From f6e2bc8cfdf40f90526fb948131594efff9b3534 Mon Sep 17 00:00:00 2001 From: JinZhou5042 Date: Thu, 1 May 2025 18:20:38 -0400 Subject: [PATCH 001/113] vine: separate pending and ready tasks --- taskvine/src/manager/vine_manager.c | 282 +++++++++++---------------- taskvine/src/manager/vine_manager.h | 1 + taskvine/src/manager/vine_schedule.c | 35 +++- taskvine/src/manager/vine_schedule.h | 1 + 4 files changed, 150 insertions(+), 169 deletions(-) diff --git a/taskvine/src/manager/vine_manager.c b/taskvine/src/manager/vine_manager.c index 1ceeabda5f..a7a797917c 100644 --- a/taskvine/src/manager/vine_manager.c +++ b/taskvine/src/manager/vine_manager.c @@ -168,6 +168,7 @@ static void delete_uncacheable_files(struct vine_manager *q, struct vine_worker_ static int delete_worker_file(struct vine_manager *q, struct vine_worker_info *w, const char *filename, vine_cache_level_t cache_level, vine_cache_level_t delete_upto_level); struct vine_task *send_library_to_worker(struct vine_manager *q, struct vine_worker_info *w, const char *name); +static void push_task_to_ready_tasks(struct vine_manager *q, struct vine_task *t); /* Return the number of workers matching a given type: WORKER, STATUS, etc */ @@ -1422,82 +1423,6 @@ static int fetch_outputs_from_worker(struct vine_manager *q, struct vine_worker_ return 1; } -/* -Consider the set of tasks that are waiting but not running. -Cancel those that cannot run for unfixable policy reasons, -such as exceeded the absolute end time, no library task available, etc. -This is done in a separate iteration outside of scheduling -to avoid the cost of these checks in the critical path. -*/ - -static int expire_waiting_tasks(struct vine_manager *q) -{ - struct vine_task *t; - int t_idx; - int expired = 0; - - /* Measure the current time once for the whole iteration. */ - double current_time = timestamp_get() / ONE_SECOND; - - /* Only work through the queue up to iter_depth. */ - int iter_count = 0; - int iter_depth = MIN(priority_queue_size(q->ready_tasks), q->attempt_schedule_depth); - - PRIORITY_QUEUE_STATIC_ITERATE(q->ready_tasks, t_idx, t, iter_count, iter_depth) - { - /* In this loop, use VINE_RESULT_SUCCESS as an indication of "still ok to run". */ - vine_result_t result = VINE_RESULT_SUCCESS; - - /* Consider each of the possible task expiration reasons. */ - - if (t->resources_requested->end > 0 && t->resources_requested->end <= current_time) { - debug(D_VINE, "task %d has exceeded its end time", t->task_id); - result = VINE_RESULT_MAX_END_TIME; - } else if (t->needs_library && !hash_table_lookup(q->library_templates, t->needs_library)) { - debug(D_VINE, "task %d does not match any submitted library named \"%s\"", t->task_id, t->needs_library); - result = VINE_RESULT_MISSING_LIBRARY; - } - - /* If any of the reasons fired, then expire the task and put in the retrieved queue. */ - if (result != VINE_RESULT_SUCCESS) { - vine_task_set_result(t, result); - priority_queue_remove(q->ready_tasks, t_idx); - change_task_state(q, t, VINE_TASK_RETRIEVED); - expired++; - } - } - - /* Return the number of tasks expired. */ - return expired; -} - -/* -Consider the set of tasks that are waiting with strict inputs -Terminate those to which no such worker exists. -*/ - -static int enforce_waiting_fixed_locations(struct vine_manager *q) -{ - int t_idx; - struct vine_task *t; - int terminated = 0; - - int iter_count = 0; - int iter_depth = priority_queue_size(q->ready_tasks); - - PRIORITY_QUEUE_BASE_ITERATE(q->ready_tasks, t_idx, t, iter_count, iter_depth) - { - if (t->has_fixed_locations && !vine_schedule_check_fixed_location(q, t)) { - vine_task_set_result(t, VINE_RESULT_FIXED_LOCATION_MISSING); - change_task_state(q, t, VINE_TASK_RETRIEVED); - priority_queue_remove(q->ready_tasks, t_idx); - terminated++; - } - } - - return terminated; -} - /* This function handles app-level failures. It remove the task from WQ and marks the task as complete so it is returned to the application. @@ -3506,83 +3431,133 @@ Advance the state of the system by selecting one task available to run, finding the best worker for that task, and then committing the task to the worker. */ - static int send_one_task(struct vine_manager *q) { - int t_idx; - struct vine_task *t; + int committable_cores = vine_schedule_find_commitable_cores(q); + if (committable_cores == 0) { + return 0; + } - int iter_count = 0; - int iter_depth = MIN(priority_queue_size(q->ready_tasks), q->attempt_schedule_depth); - - // Iterate over the ready tasks by priority. - // The first time we arrive here, the task with the highest priority is considered. However, there may be various reasons - // that this particular task is not eligible to run, such as: 1) the task requires more resources than the workers have; - // 2) the task requires input files that are not available; 3) the task failed recently; etc. (check consider_task function) - // Therefore, we may permit occasional skips of the highest priority task, and consider the next one in the queue. Similarly, - // other tasks may be skipped, too, until we find a task that is able to run. - // For a priority queue, iterating over tasks by priority is expensive, as it requires a full sort of the queue. Therefore, - // we simply iterate by numerical index if the task at the top is unable to run, and reset the cursor to the top if events - // that may enable tasks prior to the current cursor to run occur. Specifically, the following events should trigger a reset: - // 1. Task retrieval from worker (resources released or inputs available) - // 2. New worker connection (more resources available) - // 3. Delete/Insert an element prior/equal to the rotate cursor (tasks prior to the current cursor changed) - // 1 and 2 are explicitly handled by the manager where calls priority_queue_rotate_reset, while 3 is implicitly handled by - // the priority queue data structure where also invokes priority_queue_rotate_reset. - PRIORITY_QUEUE_ROTATE_ITERATE(q->ready_tasks, t_idx, t, iter_count, iter_depth) - { + int committed_tasks = 0; + int tasks_considered = 0; + int tasks_to_consider = MIN(priority_queue_size(q->ready_tasks), q->attempt_schedule_depth); + + /* temporarily skipped tasks that are runnable but cannot fit on any current worker */ + struct list *skipped_tasks = list_create(); + + while ((tasks_considered++ < tasks_to_consider) && (committed_tasks < committable_cores)) { + + struct vine_task *t = priority_queue_pop(q->ready_tasks); + if (!t) { + break; + } + + /* this task is not runnable at all, put it back in the pending queue */ if (!consider_task(q, t)) { + list_push_tail(q->pending_tasks, t); continue; } - // Find the best worker for the task - q->stats_measure->time_scheduling = timestamp_get(); + /* select a worker for the task */ struct vine_worker_info *w = vine_schedule_task_to_worker(q, t); - q->stats->time_scheduling += timestamp_get() - q->stats_measure->time_scheduling; - if (w) { - priority_queue_remove(q->ready_tasks, t_idx); + /* task is runnable but no worker is fit, silently skip it */ + if (!w) { + list_push_tail(skipped_tasks, t); + continue; + } - // do not continue if this worker is running a group task - if (q->task_groups_enabled) { - struct vine_task *it; - uint64_t taskid; - ITABLE_ITERATE(w->current_tasks, taskid, it) - { - if (it->group_id) { - return 0; - } - } - } + /* commit the task to the worker */ + vine_result_code_t result; + if (q->task_groups_enabled) { + result = commit_task_group_to_worker(q, w, t); + } else { + result = commit_task_to_worker(q, w, t); + } - vine_result_code_t result; - if (q->task_groups_enabled) { - result = commit_task_group_to_worker(q, w, t); - } else { - result = commit_task_to_worker(q, w, t); - } + switch (result) { + case VINE_SUCCESS: + committed_tasks++; + break; + case VINE_APP_FAILURE: + case VINE_WORKER_FAILURE: + /* failed to dispatch, commit put the task back in the right place. */ + break; + case VINE_MGR_FAILURE: + /* special case, commit had a chained failure. */ + list_push_tail(skipped_tasks, t); + break; + case VINE_END_OF_LIST: + /* shouldn't happen, keep going */ + break; + } + } - switch (result) { - case VINE_SUCCESS: - /* return on successful commit. */ - return 1; - break; - case VINE_APP_FAILURE: - case VINE_WORKER_FAILURE: - /* failed to dispatch, commit put the task back in the right place. */ - break; - case VINE_MGR_FAILURE: - /* special case, commit had a chained failure. */ - priority_queue_push(q->ready_tasks, t, t->priority); - break; - case VINE_END_OF_LIST: - /* shouldn't happen, keep going */ - break; - } + /* put back all tasks that were skipped */ + struct vine_task *t; + while ((t = list_pop_head(skipped_tasks))) { + push_task_to_ready_tasks(q, t); + } + list_delete(skipped_tasks); + + return committed_tasks; +} + +/* +Rotate pending tasks to the ready queue if they are runnable. +*/ +int rotate_pending_tasks(struct vine_manager *q) +{ + int runnable_tasks = 0; + + int tasks_considered = 0; + int tasks_to_consider = MIN(list_size(q->pending_tasks), q->attempt_schedule_depth); + struct vine_task *t = NULL; + + double current_time = timestamp_get() / ONE_SECOND; + + while (tasks_considered++ < tasks_to_consider) { + t = list_pop_head(q->pending_tasks); + if (!t) { + break; } + + /* if the task is runnable, push it to the ready queue */ + if (consider_task(q, t)) { + push_task_to_ready_tasks(q, t); + runnable_tasks++; + continue; + } + + /* otherwise, check if the task has exceeded its end time or does not match any submitted library */ + /* In this loop, use VINE_RESULT_SUCCESS as an indication of "still ok to run". */ + vine_result_t result = VINE_RESULT_SUCCESS; + if (t->resources_requested->end > 0 && t->resources_requested->end <= current_time) { + debug(D_VINE, "task %d has exceeded its end time", t->task_id); + result = VINE_RESULT_MAX_END_TIME; + } + if (t->needs_library && !hash_table_lookup(q->library_templates, t->needs_library)) { + debug(D_VINE, "task %d does not match any submitted library named \"%s\"", t->task_id, t->needs_library); + result = VINE_RESULT_MISSING_LIBRARY; + } + if (result != VINE_RESULT_SUCCESS) { + vine_task_set_result(t, result); + change_task_state(q, t, VINE_TASK_RETRIEVED); + continue; + } + + /* enforce fixed locations */ + if (q->fixed_location_in_queue && t->has_fixed_locations && !vine_schedule_check_fixed_location(q, t)) { + vine_task_set_result(t, VINE_RESULT_FIXED_LOCATION_MISSING); + change_task_state(q, t, VINE_TASK_RETRIEVED); + continue; + } + + /* if the task is not runnable, put it back in the pending queue */ + list_push_tail(q->pending_tasks, t); } - return 0; + return runnable_tasks; } /* @@ -4036,6 +4011,7 @@ struct vine_manager *vine_ssl_create(int port, const char *key, const char *cert q->next_task_id = 1; q->fixed_location_in_queue = 0; + q->pending_tasks = list_create(); q->ready_tasks = priority_queue_create(0); q->running_table = itable_create(0); q->waiting_retrieval_list = list_create(); @@ -4416,6 +4392,7 @@ void vine_delete(struct vine_manager *q) hash_table_clear(q->categories, (void *)category_free); hash_table_delete(q->categories); + list_delete(q->pending_tasks); priority_queue_delete(q->ready_tasks); itable_delete(q->running_table); list_delete(q->waiting_retrieval_list); @@ -4566,19 +4543,17 @@ char *vine_monitor_wrap(struct vine_manager *q, struct vine_worker_info *w, stru } /* Put a given task on the ready list, taking into account the task priority and the manager schedule. */ - static void push_task_to_ready_tasks(struct vine_manager *q, struct vine_task *t) { if (t->result == VINE_RESULT_RESOURCE_EXHAUSTION) { /* when a task is resubmitted given resource exhaustion, we - * increment its priority by 1, so it gets to run as soon + * increment its priority a bit, so it gets to run as soon * as possible among those with the same priority. This avoids * the issue in which all 'big' tasks fail because the first * allocation is too small. */ - priority_queue_push(q->ready_tasks, t, t->priority + 1); - } else { - priority_queue_push(q->ready_tasks, t, t->priority); + t->priority *= 1.05; } + priority_queue_push(q->ready_tasks, t, t->priority); /* If the task has been used before, clear out accumulated state. */ vine_task_clean(t); @@ -5294,27 +5269,6 @@ static struct vine_task *vine_wait_internal(struct vine_manager *q, int timeout, } while (q->max_retrievals < 0 || retrieved_this_cycle < q->max_retrievals || !priority_queue_size(q->ready_tasks)); END_ACCUM_TIME(q, time_receive); - // check for tasks that cannot run at all - BEGIN_ACCUM_TIME(q, time_internal); - result = expire_waiting_tasks(q); - END_ACCUM_TIME(q, time_internal); - if (result > 0) { - retrieved_this_cycle += result; - events++; - } - - // only check for fixed location if any are present (high overhead) - if (q->fixed_location_in_queue) { - - BEGIN_ACCUM_TIME(q, time_internal); - result = enforce_waiting_fixed_locations(q); - END_ACCUM_TIME(q, time_internal); - if (result > 0) { - retrieved_this_cycle += result; - events++; - } - } - if (retrieved_this_cycle) { // reset the rotate cursor on task retrieval priority_queue_rotate_reset(q->ready_tasks); diff --git a/taskvine/src/manager/vine_manager.h b/taskvine/src/manager/vine_manager.h index 32e187a506..672eb2f632 100644 --- a/taskvine/src/manager/vine_manager.h +++ b/taskvine/src/manager/vine_manager.h @@ -102,6 +102,7 @@ struct vine_manager { /* Primary data structures for tracking task state. */ struct itable *tasks; /* Maps task_id -> vine_task of all tasks in any state. */ + struct list *pending_tasks; /* List of vine_task that are waiting to be dispatched. */ struct priority_queue *ready_tasks; /* Priority queue of vine_task that are waiting to execute. */ struct itable *running_table; /* Table of vine_task that are running at workers. */ struct list *waiting_retrieval_list; /* List of vine_task that are waiting to be retrieved. */ diff --git a/taskvine/src/manager/vine_schedule.c b/taskvine/src/manager/vine_schedule.c index 80417922f5..c2eded45ef 100644 --- a/taskvine/src/manager/vine_schedule.c +++ b/taskvine/src/manager/vine_schedule.c @@ -149,6 +149,35 @@ int check_worker_have_enough_disk_with_inputs(struct vine_manager *q, struct vin return ok; } +/* Find the number of committable cores across all connected workers. */ +int vine_schedule_find_commitable_cores(struct vine_manager *q) +{ + int committable_cores = 0; + uint64_t library_task_id = 0; + struct vine_task *library_task = NULL; + char *key; + struct vine_worker_info *w; + + HASH_TABLE_ITERATE(q->worker_table, key, w) + { + if (!w->resources || w->resources->cores.total <= 0) { + continue; + } + if (w->current_libraries && itable_size(w->current_libraries) > 0) { + ITABLE_ITERATE(w->current_libraries, library_task_id, library_task) + { + if (!library_task || !library_task->provides_library) { + continue; + } + committable_cores += (library_task->function_slots_total - library_task->function_slots_inuse); + } + } + committable_cores += (overcommitted_resource_total(q, w->resources->cores.total) - w->resources->cores.inuse); + } + + return committable_cores; +} + /* Check if this worker has committable resources for any type of task. * If it returns false, neither a function task, library task nor a regular task can run on this worker. * If it returns true, the worker has either free slots for function calls or sufficient resources for regular tasks. @@ -644,7 +673,6 @@ This is quite an expensive function and so is invoked only periodically. void vine_schedule_check_for_large_tasks(struct vine_manager *q) { - int t_idx; struct vine_task *t; int unfit_core = 0; int unfit_mem = 0; @@ -653,10 +681,7 @@ void vine_schedule_check_for_large_tasks(struct vine_manager *q) struct rmsummary *largest_unfit_task = rmsummary_create(-1); - int iter_count = 0; - int iter_depth = priority_queue_size(q->ready_tasks); - - PRIORITY_QUEUE_BASE_ITERATE(q->ready_tasks, t_idx, t, iter_count, iter_depth) + LIST_ITERATE(q->pending_tasks, t) { // check each task against the queue of connected workers vine_resource_bitmask_t bit_set = is_task_larger_than_any_worker(q, t); diff --git a/taskvine/src/manager/vine_schedule.h b/taskvine/src/manager/vine_schedule.h index 4ef0c613a9..cb4e2a37e5 100644 --- a/taskvine/src/manager/vine_schedule.h +++ b/taskvine/src/manager/vine_schedule.h @@ -24,4 +24,5 @@ int vine_schedule_check_fixed_location(struct vine_manager *q, struct vine_task int vine_schedule_in_ramp_down(struct vine_manager *q); struct vine_task *vine_schedule_find_library(struct vine_manager *q, struct vine_worker_info *w, const char *library_name); int check_worker_against_task(struct vine_manager *q, struct vine_worker_info *w, struct vine_task *t); +int vine_schedule_find_commitable_cores(struct vine_manager *q); #endif From 9abf3e5fbaa39afd66430251d536b8b132029227 Mon Sep 17 00:00:00 2001 From: JinZhou5042 Date: Fri, 2 May 2025 00:07:42 -0400 Subject: [PATCH 002/113] rename func name --- taskvine/src/manager/vine_manager.c | 20 ++++++++++++-------- taskvine/src/manager/vine_schedule.c | 2 +- taskvine/src/manager/vine_schedule.h | 2 +- 3 files changed, 14 insertions(+), 10 deletions(-) diff --git a/taskvine/src/manager/vine_manager.c b/taskvine/src/manager/vine_manager.c index a7a797917c..1f907c887a 100644 --- a/taskvine/src/manager/vine_manager.c +++ b/taskvine/src/manager/vine_manager.c @@ -3433,8 +3433,7 @@ the task to the worker. */ static int send_one_task(struct vine_manager *q) { - int committable_cores = vine_schedule_find_commitable_cores(q); - if (committable_cores == 0) { + if (vine_schedule_count_commitable_cores(q) == 0) { return 0; } @@ -3445,9 +3444,9 @@ static int send_one_task(struct vine_manager *q) /* temporarily skipped tasks that are runnable but cannot fit on any current worker */ struct list *skipped_tasks = list_create(); - while ((tasks_considered++ < tasks_to_consider) && (committed_tasks < committable_cores)) { + struct vine_task *t; - struct vine_task *t = priority_queue_pop(q->ready_tasks); + while ((tasks_considered++ < tasks_to_consider) && (t = priority_queue_pop(q->ready_tasks))) { if (!t) { break; } @@ -3491,10 +3490,14 @@ static int send_one_task(struct vine_manager *q) /* shouldn't happen, keep going */ break; } + + /* if we have committed a task, we are done */ + if (committed_tasks > 0) { + break; + } } /* put back all tasks that were skipped */ - struct vine_task *t; while ((t = list_pop_head(skipped_tasks))) { push_task_to_ready_tasks(q, t); } @@ -3503,9 +3506,7 @@ static int send_one_task(struct vine_manager *q) return committed_tasks; } -/* -Rotate pending tasks to the ready queue if they are runnable. -*/ +/* Rotate pending tasks to the ready queue if they are runnable. */ int rotate_pending_tasks(struct vine_manager *q) { int runnable_tasks = 0; @@ -5285,6 +5286,9 @@ static struct vine_task *vine_wait_internal(struct vine_manager *q, int timeout, } // tasks waiting to be dispatched? BEGIN_ACCUM_TIME(q, time_send); + // rotate pending tasks before dispatching + rotate_pending_tasks(q); + // find a task to dispatch result = send_one_task(q); END_ACCUM_TIME(q, time_send); if (result) { diff --git a/taskvine/src/manager/vine_schedule.c b/taskvine/src/manager/vine_schedule.c index c2eded45ef..bc19793d94 100644 --- a/taskvine/src/manager/vine_schedule.c +++ b/taskvine/src/manager/vine_schedule.c @@ -150,7 +150,7 @@ int check_worker_have_enough_disk_with_inputs(struct vine_manager *q, struct vin } /* Find the number of committable cores across all connected workers. */ -int vine_schedule_find_commitable_cores(struct vine_manager *q) +int vine_schedule_count_commitable_cores(struct vine_manager *q) { int committable_cores = 0; uint64_t library_task_id = 0; diff --git a/taskvine/src/manager/vine_schedule.h b/taskvine/src/manager/vine_schedule.h index cb4e2a37e5..ad237a910d 100644 --- a/taskvine/src/manager/vine_schedule.h +++ b/taskvine/src/manager/vine_schedule.h @@ -24,5 +24,5 @@ int vine_schedule_check_fixed_location(struct vine_manager *q, struct vine_task int vine_schedule_in_ramp_down(struct vine_manager *q); struct vine_task *vine_schedule_find_library(struct vine_manager *q, struct vine_worker_info *w, const char *library_name); int check_worker_against_task(struct vine_manager *q, struct vine_worker_info *w, struct vine_task *t); -int vine_schedule_find_commitable_cores(struct vine_manager *q); +int vine_schedule_count_commitable_cores(struct vine_manager *q); #endif From 89fd46790ee1a0fd780c8ce423696648c981df50 Mon Sep 17 00:00:00 2001 From: JinZhou5042 Date: Fri, 2 May 2025 00:36:34 -0400 Subject: [PATCH 003/113] aggressively send tasks --- taskvine/src/manager/vine_manager.c | 11 ++++------- 1 file changed, 4 insertions(+), 7 deletions(-) diff --git a/taskvine/src/manager/vine_manager.c b/taskvine/src/manager/vine_manager.c index 1f907c887a..5a7a20bc59 100644 --- a/taskvine/src/manager/vine_manager.c +++ b/taskvine/src/manager/vine_manager.c @@ -3433,7 +3433,8 @@ the task to the worker. */ static int send_one_task(struct vine_manager *q) { - if (vine_schedule_count_commitable_cores(q) == 0) { + int committable_cores = vine_schedule_count_commitable_cores(q); + if (committable_cores == 0) { return 0; } @@ -3446,7 +3447,7 @@ static int send_one_task(struct vine_manager *q) struct vine_task *t; - while ((tasks_considered++ < tasks_to_consider) && (t = priority_queue_pop(q->ready_tasks))) { + while ((committable_cores > 0) && (tasks_considered++ < tasks_to_consider) && (t = priority_queue_pop(q->ready_tasks))) { if (!t) { break; } @@ -3477,6 +3478,7 @@ static int send_one_task(struct vine_manager *q) switch (result) { case VINE_SUCCESS: committed_tasks++; + committable_cores--; break; case VINE_APP_FAILURE: case VINE_WORKER_FAILURE: @@ -3490,11 +3492,6 @@ static int send_one_task(struct vine_manager *q) /* shouldn't happen, keep going */ break; } - - /* if we have committed a task, we are done */ - if (committed_tasks > 0) { - break; - } } /* put back all tasks that were skipped */ From fbcb4f4a355e67631d1bc0b81682c06d272d6dd2 Mon Sep 17 00:00:00 2001 From: JinZhou5042 Date: Fri, 2 May 2025 10:02:49 -0400 Subject: [PATCH 004/113] just send one task --- taskvine/src/manager/vine_manager.c | 7 +++---- 1 file changed, 3 insertions(+), 4 deletions(-) diff --git a/taskvine/src/manager/vine_manager.c b/taskvine/src/manager/vine_manager.c index 5a7a20bc59..8e661c4020 100644 --- a/taskvine/src/manager/vine_manager.c +++ b/taskvine/src/manager/vine_manager.c @@ -3433,8 +3433,8 @@ the task to the worker. */ static int send_one_task(struct vine_manager *q) { - int committable_cores = vine_schedule_count_commitable_cores(q); - if (committable_cores == 0) { + /* return if no committable cores */ + if (vine_schedule_count_commitable_cores(q) < 1) { return 0; } @@ -3447,7 +3447,7 @@ static int send_one_task(struct vine_manager *q) struct vine_task *t; - while ((committable_cores > 0) && (tasks_considered++ < tasks_to_consider) && (t = priority_queue_pop(q->ready_tasks))) { + while ((committed_tasks == 0) && (tasks_considered++ < tasks_to_consider) && (t = priority_queue_pop(q->ready_tasks))) { if (!t) { break; } @@ -3478,7 +3478,6 @@ static int send_one_task(struct vine_manager *q) switch (result) { case VINE_SUCCESS: committed_tasks++; - committable_cores--; break; case VINE_APP_FAILURE: case VINE_WORKER_FAILURE: From 8f234f300b2146e2f79958e74d80f57b29512349 Mon Sep 17 00:00:00 2001 From: Jin Zhou Date: Sat, 24 May 2025 22:55:24 -0400 Subject: [PATCH 005/113] use vine_schedule_have_committable_resources --- taskvine/src/manager/vine_manager.c | 14 +++++++++----- taskvine/src/manager/vine_schedule.c | 19 +++++++++++-------- taskvine/src/manager/vine_schedule.h | 3 ++- 3 files changed, 22 insertions(+), 14 deletions(-) diff --git a/taskvine/src/manager/vine_manager.c b/taskvine/src/manager/vine_manager.c index da969850a5..f8ab7bd1a4 100644 --- a/taskvine/src/manager/vine_manager.c +++ b/taskvine/src/manager/vine_manager.c @@ -3418,10 +3418,13 @@ the task to the worker. static int send_one_task(struct vine_manager *q) { /* return if no committable cores */ - if (vine_schedule_count_commitable_cores(q) < 1) { + if (!vine_schedule_have_committable_resources(q)) { return 0; } + /* rotate pending tasks before dispatching any tasks */ + rotate_pending_tasks(q); + int committed_tasks = 0; int tasks_considered = 0; int tasks_to_consider = MIN(priority_queue_size(q->ready_tasks), q->attempt_schedule_depth); @@ -3431,10 +3434,12 @@ static int send_one_task(struct vine_manager *q) struct vine_task *t; - while ((committed_tasks == 0) && (tasks_considered++ < tasks_to_consider) && (t = priority_queue_pop(q->ready_tasks))) { + while ((committed_tasks == 0) && (tasks_considered < tasks_to_consider)) { + t = priority_queue_pop(q->ready_tasks); if (!t) { break; } + tasks_considered++; /* this task is not runnable at all, put it back in the pending queue */ if (!consider_task(q, t)) { @@ -3469,7 +3474,8 @@ static int send_one_task(struct vine_manager *q) break; case VINE_MGR_FAILURE: /* special case, commit had a chained failure. */ - list_push_tail(skipped_tasks, t); + debug(D_VINE, "Special case, failed to commit task %d to worker %s", t->task_id, w->hostname); + list_push_tail(q->pending_tasks, t); break; case VINE_END_OF_LIST: /* shouldn't happen, keep going */ @@ -5210,8 +5216,6 @@ static struct vine_task *vine_wait_internal(struct vine_manager *q, int timeout, } // tasks waiting to be dispatched? BEGIN_ACCUM_TIME(q, time_send); - // rotate pending tasks before dispatching - rotate_pending_tasks(q); // find a task to dispatch result = send_one_task(q); END_ACCUM_TIME(q, time_send); diff --git a/taskvine/src/manager/vine_schedule.c b/taskvine/src/manager/vine_schedule.c index bc19793d94..ddfb88f548 100644 --- a/taskvine/src/manager/vine_schedule.c +++ b/taskvine/src/manager/vine_schedule.c @@ -149,12 +149,9 @@ int check_worker_have_enough_disk_with_inputs(struct vine_manager *q, struct vin return ok; } -/* Find the number of committable cores across all connected workers. */ -int vine_schedule_count_commitable_cores(struct vine_manager *q) +/* Check if any worker has committable resources for any type of task, for now only check for available cores or slots. */ +int vine_schedule_have_committable_resources(struct vine_manager *q) { - int committable_cores = 0; - uint64_t library_task_id = 0; - struct vine_task *library_task = NULL; char *key; struct vine_worker_info *w; @@ -164,18 +161,24 @@ int vine_schedule_count_commitable_cores(struct vine_manager *q) continue; } if (w->current_libraries && itable_size(w->current_libraries) > 0) { + uint64_t library_task_id = 0; + struct vine_task *library_task = NULL; ITABLE_ITERATE(w->current_libraries, library_task_id, library_task) { if (!library_task || !library_task->provides_library) { continue; } - committable_cores += (library_task->function_slots_total - library_task->function_slots_inuse); + if (library_task->function_slots_total > library_task->function_slots_inuse) { + return 1; + } } } - committable_cores += (overcommitted_resource_total(q, w->resources->cores.total) - w->resources->cores.inuse); + if (overcommitted_resource_total(q, w->resources->cores.total) > w->resources->cores.inuse) { + return 1; + } } - return committable_cores; + return 0; } /* Check if this worker has committable resources for any type of task. diff --git a/taskvine/src/manager/vine_schedule.h b/taskvine/src/manager/vine_schedule.h index ad237a910d..8f9a4aca86 100644 --- a/taskvine/src/manager/vine_schedule.h +++ b/taskvine/src/manager/vine_schedule.h @@ -24,5 +24,6 @@ int vine_schedule_check_fixed_location(struct vine_manager *q, struct vine_task int vine_schedule_in_ramp_down(struct vine_manager *q); struct vine_task *vine_schedule_find_library(struct vine_manager *q, struct vine_worker_info *w, const char *library_name); int check_worker_against_task(struct vine_manager *q, struct vine_worker_info *w, struct vine_task *t); -int vine_schedule_count_commitable_cores(struct vine_manager *q); +int vine_schedule_have_committable_resources(struct vine_manager *q); + #endif From 2333d651f0aa05b6687ffed97ae53e3e7be06084 Mon Sep 17 00:00:00 2001 From: Jin Zhou Date: Sat, 24 May 2025 23:00:09 -0400 Subject: [PATCH 006/113] static --- taskvine/src/manager/vine_manager.c | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/taskvine/src/manager/vine_manager.c b/taskvine/src/manager/vine_manager.c index f8ab7bd1a4..973eddc6b7 100644 --- a/taskvine/src/manager/vine_manager.c +++ b/taskvine/src/manager/vine_manager.c @@ -3493,7 +3493,7 @@ static int send_one_task(struct vine_manager *q) } /* Rotate pending tasks to the ready queue if they are runnable. */ -int rotate_pending_tasks(struct vine_manager *q) +static int rotate_pending_tasks(struct vine_manager *q) { int runnable_tasks = 0; From 904b08be1bc1c144c2b43cb6769a142e1119eeee Mon Sep 17 00:00:00 2001 From: Jin Zhou Date: Sat, 24 May 2025 23:05:17 -0400 Subject: [PATCH 007/113] vine_schedule_rotate_pending --- taskvine/src/manager/vine_manager.c | 57 +--------------------------- taskvine/src/manager/vine_schedule.c | 55 +++++++++++++++++++++++++++ taskvine/src/manager/vine_schedule.h | 1 + 3 files changed, 57 insertions(+), 56 deletions(-) diff --git a/taskvine/src/manager/vine_manager.c b/taskvine/src/manager/vine_manager.c index 973eddc6b7..346295efdb 100644 --- a/taskvine/src/manager/vine_manager.c +++ b/taskvine/src/manager/vine_manager.c @@ -3423,7 +3423,7 @@ static int send_one_task(struct vine_manager *q) } /* rotate pending tasks before dispatching any tasks */ - rotate_pending_tasks(q); + vine_schedule_rotate_pending_tasks(q); int committed_tasks = 0; int tasks_considered = 0; @@ -3492,61 +3492,6 @@ static int send_one_task(struct vine_manager *q) return committed_tasks; } -/* Rotate pending tasks to the ready queue if they are runnable. */ -static int rotate_pending_tasks(struct vine_manager *q) -{ - int runnable_tasks = 0; - - int tasks_considered = 0; - int tasks_to_consider = MIN(list_size(q->pending_tasks), q->attempt_schedule_depth); - struct vine_task *t = NULL; - - double current_time = timestamp_get() / ONE_SECOND; - - while (tasks_considered++ < tasks_to_consider) { - t = list_pop_head(q->pending_tasks); - if (!t) { - break; - } - - /* if the task is runnable, push it to the ready queue */ - if (consider_task(q, t)) { - push_task_to_ready_tasks(q, t); - runnable_tasks++; - continue; - } - - /* otherwise, check if the task has exceeded its end time or does not match any submitted library */ - /* In this loop, use VINE_RESULT_SUCCESS as an indication of "still ok to run". */ - vine_result_t result = VINE_RESULT_SUCCESS; - if (t->resources_requested->end > 0 && t->resources_requested->end <= current_time) { - debug(D_VINE, "task %d has exceeded its end time", t->task_id); - result = VINE_RESULT_MAX_END_TIME; - } - if (t->needs_library && !hash_table_lookup(q->library_templates, t->needs_library)) { - debug(D_VINE, "task %d does not match any submitted library named \"%s\"", t->task_id, t->needs_library); - result = VINE_RESULT_MISSING_LIBRARY; - } - if (result != VINE_RESULT_SUCCESS) { - vine_task_set_result(t, result); - change_task_state(q, t, VINE_TASK_RETRIEVED); - continue; - } - - /* enforce fixed locations */ - if (q->fixed_location_in_queue && t->has_fixed_locations && !vine_schedule_check_fixed_location(q, t)) { - vine_task_set_result(t, VINE_RESULT_FIXED_LOCATION_MISSING); - change_task_state(q, t, VINE_TASK_RETRIEVED); - continue; - } - - /* if the task is not runnable, put it back in the pending queue */ - list_push_tail(q->pending_tasks, t); - } - - return runnable_tasks; -} - /* Finding a worker that has tasks waiting to be retrieved, then fetch the outputs of those tasks. Returns the number of tasks received. diff --git a/taskvine/src/manager/vine_schedule.c b/taskvine/src/manager/vine_schedule.c index ddfb88f548..9a13a642a1 100644 --- a/taskvine/src/manager/vine_schedule.c +++ b/taskvine/src/manager/vine_schedule.c @@ -746,3 +746,58 @@ int vine_schedule_check_fixed_location(struct vine_manager *q, struct vine_task debug(D_VINE, "Missing fixed_location dependencies for task: %d", t->task_id); return 0; } + +/* Rotate pending tasks to the ready queue if they are runnable. */ +int vine_schedule_rotate_pending_tasks(struct vine_manager *q) +{ + int runnable_tasks = 0; + + int tasks_considered = 0; + int tasks_to_consider = MIN(list_size(q->pending_tasks), q->attempt_schedule_depth); + struct vine_task *t = NULL; + + double current_time = timestamp_get() / ONE_SECOND; + + while (tasks_considered++ < tasks_to_consider) { + t = list_pop_head(q->pending_tasks); + if (!t) { + break; + } + + /* if the task is runnable, push it to the ready queue */ + if (consider_task(q, t)) { + push_task_to_ready_tasks(q, t); + runnable_tasks++; + continue; + } + + /* otherwise, check if the task has exceeded its end time or does not match any submitted library */ + /* In this loop, use VINE_RESULT_SUCCESS as an indication of "still ok to run". */ + vine_result_t result = VINE_RESULT_SUCCESS; + if (t->resources_requested->end > 0 && t->resources_requested->end <= current_time) { + debug(D_VINE, "task %d has exceeded its end time", t->task_id); + result = VINE_RESULT_MAX_END_TIME; + } + if (t->needs_library && !hash_table_lookup(q->library_templates, t->needs_library)) { + debug(D_VINE, "task %d does not match any submitted library named \"%s\"", t->task_id, t->needs_library); + result = VINE_RESULT_MISSING_LIBRARY; + } + if (result != VINE_RESULT_SUCCESS) { + vine_task_set_result(t, result); + change_task_state(q, t, VINE_TASK_RETRIEVED); + continue; + } + + /* enforce fixed locations */ + if (q->fixed_location_in_queue && t->has_fixed_locations && !vine_schedule_check_fixed_location(q, t)) { + vine_task_set_result(t, VINE_RESULT_FIXED_LOCATION_MISSING); + change_task_state(q, t, VINE_TASK_RETRIEVED); + continue; + } + + /* if the task is not runnable, put it back in the pending queue */ + list_push_tail(q->pending_tasks, t); + } + + return runnable_tasks; +} \ No newline at end of file diff --git a/taskvine/src/manager/vine_schedule.h b/taskvine/src/manager/vine_schedule.h index 8f9a4aca86..bce0b6e2f4 100644 --- a/taskvine/src/manager/vine_schedule.h +++ b/taskvine/src/manager/vine_schedule.h @@ -25,5 +25,6 @@ int vine_schedule_in_ramp_down(struct vine_manager *q); struct vine_task *vine_schedule_find_library(struct vine_manager *q, struct vine_worker_info *w, const char *library_name); int check_worker_against_task(struct vine_manager *q, struct vine_worker_info *w, struct vine_task *t); int vine_schedule_have_committable_resources(struct vine_manager *q); +int vine_schedule_rotate_pending_tasks(struct vine_manager *q); #endif From 96162e9330990a7882d6bf0ebed06cb4ca437c7e Mon Sep 17 00:00:00 2001 From: Jin Zhou Date: Sat, 24 May 2025 23:07:04 -0400 Subject: [PATCH 008/113] revert --- taskvine/src/manager/vine_manager.c | 57 +++++++++++++++++++++++++++- taskvine/src/manager/vine_schedule.c | 55 --------------------------- taskvine/src/manager/vine_schedule.h | 1 - 3 files changed, 56 insertions(+), 57 deletions(-) diff --git a/taskvine/src/manager/vine_manager.c b/taskvine/src/manager/vine_manager.c index 346295efdb..dcedee6e9a 100644 --- a/taskvine/src/manager/vine_manager.c +++ b/taskvine/src/manager/vine_manager.c @@ -3410,6 +3410,61 @@ int consider_task(struct vine_manager *q, struct vine_task *t) return 1; } +/* Rotate pending tasks to the ready queue if they are runnable. */ +static int rotate_pending_tasks(struct vine_manager *q) +{ + int runnable_tasks = 0; + + int tasks_considered = 0; + int tasks_to_consider = MIN(list_size(q->pending_tasks), q->attempt_schedule_depth); + struct vine_task *t = NULL; + + double current_time = timestamp_get() / ONE_SECOND; + + while (tasks_considered++ < tasks_to_consider) { + t = list_pop_head(q->pending_tasks); + if (!t) { + break; + } + + /* if the task is runnable, push it to the ready queue */ + if (consider_task(q, t)) { + push_task_to_ready_tasks(q, t); + runnable_tasks++; + continue; + } + + /* otherwise, check if the task has exceeded its end time or does not match any submitted library */ + /* In this loop, use VINE_RESULT_SUCCESS as an indication of "still ok to run". */ + vine_result_t result = VINE_RESULT_SUCCESS; + if (t->resources_requested->end > 0 && t->resources_requested->end <= current_time) { + debug(D_VINE, "task %d has exceeded its end time", t->task_id); + result = VINE_RESULT_MAX_END_TIME; + } + if (t->needs_library && !hash_table_lookup(q->library_templates, t->needs_library)) { + debug(D_VINE, "task %d does not match any submitted library named \"%s\"", t->task_id, t->needs_library); + result = VINE_RESULT_MISSING_LIBRARY; + } + if (result != VINE_RESULT_SUCCESS) { + vine_task_set_result(t, result); + change_task_state(q, t, VINE_TASK_RETRIEVED); + continue; + } + + /* enforce fixed locations */ + if (q->fixed_location_in_queue && t->has_fixed_locations && !vine_schedule_check_fixed_location(q, t)) { + vine_task_set_result(t, VINE_RESULT_FIXED_LOCATION_MISSING); + change_task_state(q, t, VINE_TASK_RETRIEVED); + continue; + } + + /* if the task is not runnable, put it back in the pending queue */ + list_push_tail(q->pending_tasks, t); + } + + return runnable_tasks; +} + /* Advance the state of the system by selecting one task available to run, finding the best worker for that task, and then committing @@ -3423,7 +3478,7 @@ static int send_one_task(struct vine_manager *q) } /* rotate pending tasks before dispatching any tasks */ - vine_schedule_rotate_pending_tasks(q); + rotate_pending_tasks(q); int committed_tasks = 0; int tasks_considered = 0; diff --git a/taskvine/src/manager/vine_schedule.c b/taskvine/src/manager/vine_schedule.c index 9a13a642a1..c75fdc9f6a 100644 --- a/taskvine/src/manager/vine_schedule.c +++ b/taskvine/src/manager/vine_schedule.c @@ -745,59 +745,4 @@ int vine_schedule_check_fixed_location(struct vine_manager *q, struct vine_task } debug(D_VINE, "Missing fixed_location dependencies for task: %d", t->task_id); return 0; -} - -/* Rotate pending tasks to the ready queue if they are runnable. */ -int vine_schedule_rotate_pending_tasks(struct vine_manager *q) -{ - int runnable_tasks = 0; - - int tasks_considered = 0; - int tasks_to_consider = MIN(list_size(q->pending_tasks), q->attempt_schedule_depth); - struct vine_task *t = NULL; - - double current_time = timestamp_get() / ONE_SECOND; - - while (tasks_considered++ < tasks_to_consider) { - t = list_pop_head(q->pending_tasks); - if (!t) { - break; - } - - /* if the task is runnable, push it to the ready queue */ - if (consider_task(q, t)) { - push_task_to_ready_tasks(q, t); - runnable_tasks++; - continue; - } - - /* otherwise, check if the task has exceeded its end time or does not match any submitted library */ - /* In this loop, use VINE_RESULT_SUCCESS as an indication of "still ok to run". */ - vine_result_t result = VINE_RESULT_SUCCESS; - if (t->resources_requested->end > 0 && t->resources_requested->end <= current_time) { - debug(D_VINE, "task %d has exceeded its end time", t->task_id); - result = VINE_RESULT_MAX_END_TIME; - } - if (t->needs_library && !hash_table_lookup(q->library_templates, t->needs_library)) { - debug(D_VINE, "task %d does not match any submitted library named \"%s\"", t->task_id, t->needs_library); - result = VINE_RESULT_MISSING_LIBRARY; - } - if (result != VINE_RESULT_SUCCESS) { - vine_task_set_result(t, result); - change_task_state(q, t, VINE_TASK_RETRIEVED); - continue; - } - - /* enforce fixed locations */ - if (q->fixed_location_in_queue && t->has_fixed_locations && !vine_schedule_check_fixed_location(q, t)) { - vine_task_set_result(t, VINE_RESULT_FIXED_LOCATION_MISSING); - change_task_state(q, t, VINE_TASK_RETRIEVED); - continue; - } - - /* if the task is not runnable, put it back in the pending queue */ - list_push_tail(q->pending_tasks, t); - } - - return runnable_tasks; } \ No newline at end of file diff --git a/taskvine/src/manager/vine_schedule.h b/taskvine/src/manager/vine_schedule.h index bce0b6e2f4..8f9a4aca86 100644 --- a/taskvine/src/manager/vine_schedule.h +++ b/taskvine/src/manager/vine_schedule.h @@ -25,6 +25,5 @@ int vine_schedule_in_ramp_down(struct vine_manager *q); struct vine_task *vine_schedule_find_library(struct vine_manager *q, struct vine_worker_info *w, const char *library_name); int check_worker_against_task(struct vine_manager *q, struct vine_worker_info *w, struct vine_task *t); int vine_schedule_have_committable_resources(struct vine_manager *q); -int vine_schedule_rotate_pending_tasks(struct vine_manager *q); #endif From de0d70d27511ff8423efaad95f2201e1a0956f3b Mon Sep 17 00:00:00 2001 From: Jin Zhou Date: Sun, 25 May 2025 03:52:28 -0400 Subject: [PATCH 009/113] adjust location --- taskvine/src/manager/vine_manager.c | 41 +++++++++++++--------------- taskvine/src/manager/vine_schedule.c | 1 - 2 files changed, 19 insertions(+), 23 deletions(-) diff --git a/taskvine/src/manager/vine_manager.c b/taskvine/src/manager/vine_manager.c index dcedee6e9a..30b28f1c30 100644 --- a/taskvine/src/manager/vine_manager.c +++ b/taskvine/src/manager/vine_manager.c @@ -168,7 +168,7 @@ static void delete_uncacheable_files(struct vine_manager *q, struct vine_worker_ static int delete_worker_file(struct vine_manager *q, struct vine_worker_info *w, const char *filename, vine_cache_level_t cache_level, vine_cache_level_t delete_upto_level); struct vine_task *send_library_to_worker(struct vine_manager *q, struct vine_worker_info *w, const char *name); -static void push_task_to_ready_tasks(struct vine_manager *q, struct vine_task *t); +static void enqueue_ready_task(struct vine_manager *q, struct vine_task *t); /* Return the number of workers matching a given type: WORKER, STATUS, etc */ @@ -3427,37 +3427,34 @@ static int rotate_pending_tasks(struct vine_manager *q) break; } - /* if the task is runnable, push it to the ready queue */ - if (consider_task(q, t)) { - push_task_to_ready_tasks(q, t); - runnable_tasks++; - continue; - } - - /* otherwise, check if the task has exceeded its end time or does not match any submitted library */ - /* In this loop, use VINE_RESULT_SUCCESS as an indication of "still ok to run". */ - vine_result_t result = VINE_RESULT_SUCCESS; + /* first check if the task has exceeded its end time or does not match any submitted library */ + /* If any of the reasons fired, then expire the task and put in the retrieved queue. */ if (t->resources_requested->end > 0 && t->resources_requested->end <= current_time) { debug(D_VINE, "task %d has exceeded its end time", t->task_id); - result = VINE_RESULT_MAX_END_TIME; + vine_task_set_result(t, VINE_RESULT_MAX_END_TIME); + change_task_state(q, t, VINE_TASK_RETRIEVED); + continue; } if (t->needs_library && !hash_table_lookup(q->library_templates, t->needs_library)) { debug(D_VINE, "task %d does not match any submitted library named \"%s\"", t->task_id, t->needs_library); - result = VINE_RESULT_MISSING_LIBRARY; - } - if (result != VINE_RESULT_SUCCESS) { - vine_task_set_result(t, result); + vine_task_set_result(t, VINE_RESULT_MISSING_LIBRARY); change_task_state(q, t, VINE_TASK_RETRIEVED); continue; } - - /* enforce fixed locations */ if (q->fixed_location_in_queue && t->has_fixed_locations && !vine_schedule_check_fixed_location(q, t)) { + debug(D_VINE, "Missing fixed_location dependencies for task: %d", t->task_id); vine_task_set_result(t, VINE_RESULT_FIXED_LOCATION_MISSING); change_task_state(q, t, VINE_TASK_RETRIEVED); continue; } + /* eligible to run, push it to the ready queue */ + if (consider_task(q, t)) { + enqueue_ready_task(q, t); + runnable_tasks++; + continue; + } + /* if the task is not runnable, put it back in the pending queue */ list_push_tail(q->pending_tasks, t); } @@ -3540,7 +3537,7 @@ static int send_one_task(struct vine_manager *q) /* put back all tasks that were skipped */ while ((t = list_pop_head(skipped_tasks))) { - push_task_to_ready_tasks(q, t); + enqueue_ready_task(q, t); } list_delete(skipped_tasks); @@ -4494,8 +4491,8 @@ char *vine_monitor_wrap(struct vine_manager *q, struct vine_worker_info *w, stru return wrap_cmd; } -/* Put a given task on the ready list, taking into account the task priority and the manager schedule. */ -static void push_task_to_ready_tasks(struct vine_manager *q, struct vine_task *t) +/* Put a given task on the ready queue, taking into account the task priority and the manager schedule. */ +static void enqueue_ready_task(struct vine_manager *q, struct vine_task *t) { if (t->result == VINE_RESULT_RESOURCE_EXHAUSTION) { /* when a task is resubmitted given resource exhaustion, we @@ -4559,7 +4556,7 @@ static vine_task_state_t change_task_state(struct vine_manager *q, struct vine_t break; case VINE_TASK_READY: vine_task_set_result(t, VINE_RESULT_UNKNOWN); - push_task_to_ready_tasks(q, t); + enqueue_ready_task(q, t); c->vine_stats->tasks_waiting++; break; case VINE_TASK_RUNNING: diff --git a/taskvine/src/manager/vine_schedule.c b/taskvine/src/manager/vine_schedule.c index c75fdc9f6a..9cdc8c0f5a 100644 --- a/taskvine/src/manager/vine_schedule.c +++ b/taskvine/src/manager/vine_schedule.c @@ -743,6 +743,5 @@ int vine_schedule_check_fixed_location(struct vine_manager *q, struct vine_task return 1; } } - debug(D_VINE, "Missing fixed_location dependencies for task: %d", t->task_id); return 0; } \ No newline at end of file From aff40f7bcdf321778fe25f82e82131ee056db4f6 Mon Sep 17 00:00:00 2001 From: Jin Zhou Date: Sun, 25 May 2025 04:51:06 -0400 Subject: [PATCH 010/113] fix int done --- taskvine/src/manager/vine_manager.c | 16 ++++++++-------- 1 file changed, 8 insertions(+), 8 deletions(-) diff --git a/taskvine/src/manager/vine_manager.c b/taskvine/src/manager/vine_manager.c index 30b28f1c30..ea98ad43be 100644 --- a/taskvine/src/manager/vine_manager.c +++ b/taskvine/src/manager/vine_manager.c @@ -3413,8 +3413,11 @@ int consider_task(struct vine_manager *q, struct vine_task *t) /* Rotate pending tasks to the ready queue if they are runnable. */ static int rotate_pending_tasks(struct vine_manager *q) { - int runnable_tasks = 0; + if (list_size(q->pending_tasks) == 0) { + return 0; + } + int runnable_tasks = 0; int tasks_considered = 0; int tasks_to_consider = MIN(list_size(q->pending_tasks), q->attempt_schedule_depth); struct vine_task *t = NULL; @@ -3448,15 +3451,12 @@ static int rotate_pending_tasks(struct vine_manager *q) continue; } - /* eligible to run, push it to the ready queue */ if (consider_task(q, t)) { enqueue_ready_task(q, t); runnable_tasks++; - continue; + } else { + list_push_tail(q->pending_tasks, t); } - - /* if the task is not runnable, put it back in the pending queue */ - list_push_tail(q->pending_tasks, t); } return runnable_tasks; @@ -3469,7 +3469,7 @@ the task to the worker. */ static int send_one_task(struct vine_manager *q) { - /* return if no committable cores */ + /* return early if no committable cores */ if (!vine_schedule_have_committable_resources(q)) { return 0; } @@ -5273,7 +5273,7 @@ static struct vine_task *vine_wait_internal(struct vine_manager *q, int timeout, // in this wait. if (events > 0) { BEGIN_ACCUM_TIME(q, time_internal); - int done = !priority_queue_size(q->ready_tasks) && !list_size(q->waiting_retrieval_list) && !itable_size(q->running_table); + int done = !priority_queue_size(q->ready_tasks) && !list_size(q->pending_tasks) && !list_size(q->waiting_retrieval_list) && !itable_size(q->running_table); END_ACCUM_TIME(q, time_internal); if (done) { From 7f589f22cd30241acc7b25de94dea3d4e6a59b23 Mon Sep 17 00:00:00 2001 From: Jin Zhou Date: Fri, 20 Jun 2025 09:25:42 -0400 Subject: [PATCH 011/113] vine_schedule_count_committable_cores --- taskvine/src/manager/vine_manager.c | 16 ++++++++++++++-- taskvine/src/manager/vine_schedule.c | 28 ++++++++++++++++++++-------- taskvine/src/manager/vine_schedule.h | 2 +- 3 files changed, 35 insertions(+), 11 deletions(-) diff --git a/taskvine/src/manager/vine_manager.c b/taskvine/src/manager/vine_manager.c index 227603d72c..f1b55de3c1 100644 --- a/taskvine/src/manager/vine_manager.c +++ b/taskvine/src/manager/vine_manager.c @@ -3502,7 +3502,8 @@ the task to the worker. static int send_one_task(struct vine_manager *q) { /* return early if no committable cores */ - if (!vine_schedule_have_committable_resources(q)) { + int committable_cores = vine_schedule_count_committable_cores(q); + if (committable_cores == 0) { return 0; } @@ -3518,7 +3519,7 @@ static int send_one_task(struct vine_manager *q) struct vine_task *t; - while ((committed_tasks == 0) && (tasks_considered < tasks_to_consider)) { + while (tasks_considered < tasks_to_consider) { t = priority_queue_pop(q->ready_tasks); if (!t) { break; @@ -3565,6 +3566,17 @@ static int send_one_task(struct vine_manager *q) /* shouldn't happen, keep going */ break; } + + /* continue dispatching tasks if q->prefer_dispatch is set */ + if (q->prefer_dispatch && committed_tasks < committable_cores) { + continue; + } + + /* stop when q->prefer_dispatch is not set and at least one task has been committed, + * or when it is set and all committable cores have been used */ + if (committed_tasks > 0) { + break; + } } /* put back all tasks that were skipped */ diff --git a/taskvine/src/manager/vine_schedule.c b/taskvine/src/manager/vine_schedule.c index 9cdc8c0f5a..1fd3e72a1f 100644 --- a/taskvine/src/manager/vine_schedule.c +++ b/taskvine/src/manager/vine_schedule.c @@ -149,17 +149,24 @@ int check_worker_have_enough_disk_with_inputs(struct vine_manager *q, struct vin return ok; } -/* Check if any worker has committable resources for any type of task, for now only check for available cores or slots. */ -int vine_schedule_have_committable_resources(struct vine_manager *q) +/* Count the number of committable cores for all workers. */ +int vine_schedule_count_committable_cores(struct vine_manager *q) { + int count = 0; + char *key; struct vine_worker_info *w; - HASH_TABLE_ITERATE(q->worker_table, key, w) { - if (!w->resources || w->resources->cores.total <= 0) { + /* skip if the worker hasn't reported any resources yet */ + if (!w->resources) { + continue; + } + /* skip if the worker has no cores or gpus */ + if (w->resources->cores.total <= 0 && w->resources->gpus.total <= 0) { continue; } + /* count the number of free slots on running libraries */ if (w->current_libraries && itable_size(w->current_libraries) > 0) { uint64_t library_task_id = 0; struct vine_task *library_task = NULL; @@ -169,16 +176,21 @@ int vine_schedule_have_committable_resources(struct vine_manager *q) continue; } if (library_task->function_slots_total > library_task->function_slots_inuse) { - return 1; + count += library_task->function_slots_total - library_task->function_slots_inuse; } } } - if (overcommitted_resource_total(q, w->resources->cores.total) > w->resources->cores.inuse) { - return 1; + /* count the number of free cores */ + if (w->resources->cores.total > 0 && overcommitted_resource_total(q, w->resources->cores.total) > w->resources->cores.inuse) { + count += overcommitted_resource_total(q, w->resources->cores.total) - w->resources->cores.inuse; + } + /* count the number of free gpus */ + if (w->resources->gpus.total > 0 && overcommitted_resource_total(q, w->resources->gpus.total) > w->resources->gpus.inuse) { + count += overcommitted_resource_total(q, w->resources->gpus.total) - w->resources->gpus.inuse; } } - return 0; + return count; } /* Check if this worker has committable resources for any type of task. diff --git a/taskvine/src/manager/vine_schedule.h b/taskvine/src/manager/vine_schedule.h index 8f9a4aca86..6455c18798 100644 --- a/taskvine/src/manager/vine_schedule.h +++ b/taskvine/src/manager/vine_schedule.h @@ -24,6 +24,6 @@ int vine_schedule_check_fixed_location(struct vine_manager *q, struct vine_task int vine_schedule_in_ramp_down(struct vine_manager *q); struct vine_task *vine_schedule_find_library(struct vine_manager *q, struct vine_worker_info *w, const char *library_name); int check_worker_against_task(struct vine_manager *q, struct vine_worker_info *w, struct vine_task *t); -int vine_schedule_have_committable_resources(struct vine_manager *q); +int vine_schedule_count_committable_cores(struct vine_manager *q); #endif From 6714bca758e8f15641e1f9eca1a63411385bfdeb Mon Sep 17 00:00:00 2001 From: Jin Zhou Date: Mon, 13 Oct 2025 16:08:32 -0400 Subject: [PATCH 012/113] vine: function infile load mode --- poncho/src/poncho/library_network_code.py | 30 +++++++++++++++---- poncho/src/poncho/package_serverize.py | 11 +++++-- .../python3/ndcctools/taskvine/manager.py | 9 ++++-- 3 files changed, 39 insertions(+), 11 deletions(-) diff --git a/poncho/src/poncho/library_network_code.py b/poncho/src/poncho/library_network_code.py index db6127cc71..97d490ea01 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. @@ -84,6 +87,18 @@ def sigchld_handler(signum, frame): os.writev(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") 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 @@ -130,8 +145,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) @@ -160,11 +174,10 @@ def start_function(in_pipe_fd, thread_limit=1): return -1, function_id else: 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 p = os.fork() if p == 0: @@ -382,11 +395,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/src/bindings/python3/ndcctools/taskvine/manager.py b/taskvine/src/bindings/python3/ndcctools/taskvine/manager.py index b6767a6b56..55be0e835d 100644 --- a/taskvine/src/bindings/python3/ndcctools/taskvine/manager.py +++ b/taskvine/src/bindings/python3/ndcctools/taskvine/manager.py @@ -936,8 +936,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 @@ -959,7 +960,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" @@ -1007,7 +1009,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) From a78dc95adf8d3d26fa28feb4d0be7513e2066aba Mon Sep 17 00:00:00 2001 From: Jin Zhou Date: Mon, 13 Oct 2025 16:18:57 -0400 Subject: [PATCH 013/113] specify utf-8 --- poncho/src/poncho/library_network_code.py | 19 ++++++------------- 1 file changed, 6 insertions(+), 13 deletions(-) diff --git a/poncho/src/poncho/library_network_code.py b/poncho/src/poncho/library_network_code.py index db6127cc71..9e533ee53f 100755 --- a/poncho/src/poncho/library_network_code.py +++ b/poncho/src/poncho/library_network_code.py @@ -174,12 +174,7 @@ def start_function(in_pipe_fd, thread_limit=1): stdout_timed_message(f"TASK {function_id} {function_name} arrives, starting to run in process {os.getpid()}") - try: - exit_status = 1 - except Exception: - stdout_timed_message(f"TASK {function_id} error: can't load the arguments from infile") - exit_status = 2 - raise + exit_status = 1 try: # setup stdout/err for a function call so we can capture them. @@ -199,7 +194,7 @@ def start_function(in_pipe_fd, thread_limit=1): os.dup2(library_fd, sys.stdout.fileno()) except Exception: stdout_timed_message(f"TASK {function_id} error: can't execute this function") - exit_status = 3 + exit_status = 2 raise finally: if function_stdout_fd in locals(): @@ -210,14 +205,14 @@ def start_function(in_pipe_fd, thread_limit=1): cloudpickle.dump(result, f) except Exception: stdout_timed_message(f"TASK {function_id} error: can't load the result from outfile") - exit_status = 4 - if os.path.exits("outfile"): + exit_status = 3 + if os.path.exists("outfile"): os.remove("outfile") raise try: if not result["Success"]: - exit_status = 5 + exit_status = 4 except Exception: stdout_timed_message(f"TASK {function_id} error: the result is invalid") exit_status = 5 @@ -232,14 +227,12 @@ def start_function(in_pipe_fd, thread_limit=1): os._exit(exit_status) elif p < 0: stdout_timed_message(f"TASK {function_id} error: unable to fork to execute {function_name}") - return -1 + return -1, function_id # return pid and function id of child process to parent. else: return p, function_id - return -1 - # Send result of a function execution to worker. Wake worker up to do work with SIGCHLD. def send_result(out_pipe_fd, worker_pid, task_id, exit_code): From 83cd050f07b7ccbecc9f2e89389702c98672e2da Mon Sep 17 00:00:00 2001 From: Jin Zhou Date: Mon, 13 Oct 2025 16:19:50 -0400 Subject: [PATCH 014/113] utf-8 --- poncho/src/poncho/library_network_code.py | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/poncho/src/poncho/library_network_code.py b/poncho/src/poncho/library_network_code.py index 97d490ea01..31dc5b2927 100755 --- a/poncho/src/poncho/library_network_code.py +++ b/poncho/src/poncho/library_network_code.py @@ -93,7 +93,7 @@ def load_function_infile(in_file_path): with open(in_file_path, "rb") as f: return cloudpickle.load(f) elif function_infile_load_mode == "json": - with open(in_file_path, "r") as f: + 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}") From d6f2fea0cb83b27693b36bd4795224679b93d0e5 Mon Sep 17 00:00:00 2001 From: Jin Zhou Date: Mon, 13 Oct 2025 16:56:48 -0400 Subject: [PATCH 015/113] vine: some bug fixes for the library code --- poncho/src/poncho/library_network_code.py | 51 +++++++++++------------ 1 file changed, 25 insertions(+), 26 deletions(-) diff --git a/poncho/src/poncho/library_network_code.py b/poncho/src/poncho/library_network_code.py index 9e533ee53f..b696ee5e57 100755 --- a/poncho/src/poncho/library_network_code.py +++ b/poncho/src/poncho/library_network_code.py @@ -81,7 +81,10 @@ def remote_wrapper(event): # Handler to sigchld when child exits. def sigchld_handler(signum, frame): # write any byte to signal that there's at least 1 child - os.writev(w, [b"a"]) + try: + os.write(w, b"a") + except OSError: + pass # Read data from worker, start function, and dump result to `outfile`. @@ -150,10 +153,7 @@ def start_function(in_pipe_fd, thread_limit=1): raise except Exception as e: - stdout_timed_message( - f"Library code: Function call failed due to {e}", - file=sys.stderr, - ) + stdout_timed_message(f"Library code: Function call failed due to {e}") sys.exit(1) finally: os.chdir(library_sandbox) @@ -165,40 +165,31 @@ def start_function(in_pipe_fd, thread_limit=1): event = cloudpickle.load(f) except Exception: stdout_timed_message(f"TASK {function_id} error: can't load the arguments from {arg_infile}") - return + return -1, function_id p = os.fork() if p == 0: + exit_status = 1 + try: # change the working directory to the function's sandbox os.chdir(function_sandbox) stdout_timed_message(f"TASK {function_id} {function_name} arrives, starting to run in process {os.getpid()}") - exit_status = 1 - try: - # setup stdout/err for a function call so we can capture them. - function_stdout_fd = os.open( - function_stdout_filename, os.O_WRONLY | os.O_CREAT | os.O_TRUNC - ) - # store the library's stdout fd - library_fd = os.dup(sys.stdout.fileno()) + # each child process independently redirects its own stdout/stderr. + with open(function_stdout_filename, "wb", buffering=0) as f: + os.dup2(f.fileno(), 1) # redirect stdout + os.dup2(f.fileno(), 2) # redirect stderr - # only redirect the stdout of a specific FunctionCall task into its own stdout fd, - # otherwise use the library's stdout - os.dup2(function_stdout_fd, sys.stdout.fileno()) - os.dup2(function_stdout_fd, sys.stderr.fileno()) - result = globals()[function_name](event) + stdout_timed_message(f"TASK {function_id} {function_name} starts in PID {os.getpid()}") + result = globals()[function_name](event) + stdout_timed_message(f"TASK {function_id} {function_name} finished") - # restore to the library's stdout fd on completion - os.dup2(library_fd, sys.stdout.fileno()) except Exception: - stdout_timed_message(f"TASK {function_id} error: can't execute this function") + stdout_timed_message(f"TASK {function_id} error: can't execute {function_name} due to {traceback.format_exc()}") exit_status = 2 raise - finally: - if function_stdout_fd in locals(): - os.close(function_stdout_fd) try: with open("outfile", "wb") as f: @@ -424,7 +415,15 @@ def main(): ) else: pid, func_id = start_function(in_pipe_fd, thread_limit) - pid_to_func_id[pid] = func_id + if pid == -1: + send_result( + out_pipe_fd, + args.worker_pid, + func_id, + 1, + ) + else: + pid_to_func_id[pid] = func_id else: # at least 1 child exits, reap all. # read only once as os.read is blocking if there's nothing to read. From d79b1edcc019cab0167d7e17e3de46f01cf5ed9a Mon Sep 17 00:00:00 2001 From: Jin Zhou Date: Tue, 14 Oct 2025 10:30:06 -0400 Subject: [PATCH 016/113] dttools: progress_bar src and test --- dttools/src/.gitignore | 3 +- dttools/src/Makefile | 5 +- dttools/src/progress_bar.c | 271 ++++++++++++++++++++++++++++++++ dttools/src/progress_bar.h | 86 ++++++++++ dttools/src/progress_bar_test.c | 33 ++++ 5 files changed, 396 insertions(+), 2 deletions(-) create mode 100644 dttools/src/progress_bar.c create mode 100644 dttools/src/progress_bar.h create mode 100644 dttools/src/progress_bar_test.c diff --git a/dttools/src/.gitignore b/dttools/src/.gitignore index 019e34fc65..1af0628763 100644 --- a/dttools/src/.gitignore +++ b/dttools/src/.gitignore @@ -41,4 +41,5 @@ bucketing_manager_test hash_table_fromkey_test hash_table_offset_test hash_table_benchmark -priority_queue_test \ No newline at end of file +priority_queue_test +progress_bar_test \ No newline at end of file diff --git a/dttools/src/Makefile b/dttools/src/Makefile index 829e42ef5c..c5103861b0 100644 --- a/dttools/src/Makefile +++ b/dttools/src/Makefile @@ -94,6 +94,8 @@ SOURCES = \ priority_queue.c \ priority_queue_test.c \ process.c \ + progress_bar.c \ + progress_bar_test.c \ random.c \ rmonitor.c \ rmonitor_poll.c \ @@ -164,6 +166,7 @@ HEADERS_PUBLIC = \ macros.h \ path.h \ priority_queue.h \ + progress_bar.h \ rmonitor_poll.h \ rmsummary.h \ stringtools.h \ @@ -193,7 +196,7 @@ PROGRAMS = $(MOST_PROGRAMS) catalog_query SCRIPTS = cctools_gpu_autodetect TARGETS = $(LIBRARIES) $(PRELOAD_LIBRARIES) $(PROGRAMS) $(TEST_PROGRAMS) -TEST_PROGRAMS = auth_test disk_alloc_test jx_test microbench multirun jx_count_obj_test jx_canonicalize_test jx_merge_test hash_table_offset_test hash_table_fromkey_test hash_table_benchmark histogram_test category_test jx_binary_test bucketing_base_test bucketing_manager_test priority_queue_test +TEST_PROGRAMS = auth_test disk_alloc_test jx_test microbench multirun jx_count_obj_test jx_canonicalize_test jx_merge_test hash_table_offset_test hash_table_fromkey_test hash_table_benchmark histogram_test category_test jx_binary_test bucketing_base_test bucketing_manager_test priority_queue_test progress_bar_test all: $(TARGETS) catalog_query diff --git a/dttools/src/progress_bar.c b/dttools/src/progress_bar.c new file mode 100644 index 0000000000..14e1995b42 --- /dev/null +++ b/dttools/src/progress_bar.c @@ -0,0 +1,271 @@ +/* +Copyright (C) 2025 The University of Notre Dame +This software is distributed under the GNU General Public License. +See the file COPYING for details. +*/ + +/** @file progress_bar.c +Implementation of a terminal progress bar with multiple parts. +*/ + +#include "progress_bar.h" +#include "xxmalloc.h" +#include "macros.h" +#include +#include +#include +#include +#include +#include +#include +#include +#include + +/* Max bar width (in block characters) for single-line rendering. */ +#define MAX_BAR_WIDTH 30 +/* Minimum redraw interval to avoid flicker. */ +#define PROGRESS_BAR_UPDATE_INTERVAL ((USECOND) * 0.1) + +#define COLOR_RESET "\033[0m" +#define COLOR_GREEN "\033[32m" +#define COLOR_CYAN "\033[38;2;0;255;255m" +#define COLOR_ORANGE "\033[38;2;255;165;0m" +#define COLOR_PURPLE "\033[38;2;128;0;128m" +#define COLOR_PINK "\033[38;2;255;192;203m" +#define COLOR_YELLOW "\033[38;2;255;255;0m" + +/** Get terminal width in columns; return 80 on failure. */ +static int get_terminal_width() +{ + struct winsize w; + + if (ioctl(STDOUT_FILENO, TIOCGWINSZ, &w) == -1) { + return 80; + } + + return w.ws_col; +} + +/** Compute bar width based on terminal and labels; clamp to bounds. */ +static int compute_bar_width(const char *label, int part_text_len) +{ + if (!label) { + return 0; + } + + int term_width = get_terminal_width(); + int label_len = strlen(label); + int bar_width = term_width - label_len - part_text_len - 28; + + if (bar_width > MAX_BAR_WIDTH) { + bar_width = MAX_BAR_WIDTH; + } + + if (bar_width < 10) { + bar_width = 10; + } + + return (int)(bar_width * 0.8); +} + +/** Render one-line progress bar with aggregated totals, progress, and elapsed time. */ +static void print_progress_bar(struct ProgressBar *bar) +{ + if (!bar) { + return; + } + + bar->last_draw_time = timestamp_get(); + + char part_text[256]; + char *ptr = part_text; + int remain = sizeof(part_text); + int written = snprintf(ptr, remain, "["); + ptr += written; + remain -= written; + + uint64_t total_sum = 0; + uint64_t current_sum = 0; + + bool first = true; + struct ProgressBarPart *p; + LIST_ITERATE(bar->parts, p) + { + total_sum += p->total; + current_sum += p->current; + + if (!first) { + written = snprintf(ptr, remain, ", "); + ptr += written; + remain -= written; + } + + written = snprintf(ptr, remain, "%s: %" PRIu64 "/%" PRIu64, p->label, p->current, p->total); + ptr += written; + remain -= written; + + first = false; + } + snprintf(ptr, remain, "]"); + part_text[sizeof(part_text) - 1] = '\0'; + + float progress = (total_sum > 0) ? ((float)current_sum / total_sum) : 0.0f; + if (progress > 1.0f) { + progress = 1.0f; + } + + timestamp_t elapsed = timestamp_get() - bar->start_time; + int h = elapsed / (3600LL * USECOND); + int m = (elapsed % (3600LL * USECOND)) / (60LL * USECOND); + int s = (elapsed % (60LL * USECOND)) / USECOND; + + if (bar->has_drawn_once) { + printf("\r\033[2K"); + } else { + bar->has_drawn_once = 1; + } + + int part_text_len = (int)(ptr - part_text) + 1; + int bar_width = compute_bar_width(bar->label, part_text_len); + int filled = (int)(progress * bar_width); + + char bar_line[MAX_BAR_WIDTH * 3 + 1]; + int offset = 0; + const char *block = "━"; + + for (int i = 0; i < filled; ++i) { + memcpy(bar_line + offset, block, 3); + offset += 3; + } + + memset(bar_line + offset, ' ', (bar_width - filled)); + offset += (bar_width - filled); + bar_line[offset] = '\0'; + + printf("%s " COLOR_GREEN "%s %" PRIu64 "/%" PRIu64 COLOR_YELLOW " %s" COLOR_CYAN " %.1f%%" COLOR_ORANGE " %02d:%02d:%02d" COLOR_RESET, + bar->label ? bar->label : "", + bar_line, + current_sum, + total_sum, + part_text, + progress * 100, + h, + m, + s); + + fflush(stdout); +} + +/** Create and initialize a progress bar. */ +struct ProgressBar *progress_bar_init(const char *label) +{ + if (!label) { + return NULL; + } + + struct ProgressBar *bar = xxmalloc(sizeof(struct ProgressBar)); + + bar->label = xxstrdup(label); + bar->parts = list_create(); + bar->start_time = timestamp_get(); + bar->last_draw_time = timestamp_get(); + bar->has_drawn_once = 0; + + return bar; +} + +/** Create a new part. */ +struct ProgressBarPart *progress_bar_create_part(const char *label, uint64_t total) +{ + if (!label) { + return NULL; + } + + struct ProgressBarPart *part = xxmalloc(sizeof(struct ProgressBarPart)); + + part->label = xxstrdup(label); + part->total = total; + part->current = 0; + + return part; +} + +/** Bind a part to the progress bar. */ +void progress_bar_bind_part(struct ProgressBar *bar, struct ProgressBarPart *part) +{ + if (!bar || !part) { + return; + } + + list_push_tail(bar->parts, part); + print_progress_bar(bar); +} + +/** Set the total for a part. */ +void progress_bar_set_part_total(struct ProgressBar *bar, struct ProgressBarPart *part, uint64_t new_total) +{ + if (!bar || !part) { + return; + } + part->total = new_total; + + print_progress_bar(bar); +} + +/** Advance a part's current value, redraw if needed. */ +void progress_bar_update_part(struct ProgressBar *bar, struct ProgressBarPart *part, uint64_t increment) +{ + if (!bar || !part) { + return; + } + + part->current += increment; + if (part->current > part->total) { + part->current = part->total; + } + + if (timestamp_get() - bar->last_draw_time < PROGRESS_BAR_UPDATE_INTERVAL) { + return; + } + + 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) +{ + if (!bar) { + return; + } + + bar->start_time = start_time; +} + +/** Final render and newline. */ +void progress_bar_finish(struct ProgressBar *bar) +{ + if (!bar) { + return; + } + + print_progress_bar(bar); + printf("\n"); +} + +/** Free the progress bar, its parts, and internal resources. */ +void progress_bar_delete(struct ProgressBar *bar) +{ + if (!bar) { + return; + } + + free(bar->label); + struct ProgressBarPart *p; + LIST_ITERATE(bar->parts, p) + { + free(p->label); + free(p); + } + list_delete(bar->parts); + free(bar); +} diff --git a/dttools/src/progress_bar.h b/dttools/src/progress_bar.h new file mode 100644 index 0000000000..6a38a95424 --- /dev/null +++ b/dttools/src/progress_bar.h @@ -0,0 +1,86 @@ +/* +Copyright (C) 2025 The University of Notre Dame +This software is distributed under the GNU General Public License. +See the file COPYING for details. +*/ + +/** @file progress_bar.h +Terminal progress bar API with multiple parts. +*/ + +#ifndef PROGRESS_BAR_H +#define PROGRESS_BAR_H + +#include "list.h" +#include "timestamp.h" +#include +#include + +/** A part of a progress bar. */ +struct ProgressBarPart { + char *label; + uint64_t total; + uint64_t current; +}; + +/** Progress bar object. */ +struct ProgressBar { + char *label; + struct list *parts; + timestamp_t start_time; + timestamp_t last_draw_time; + int has_drawn_once; +}; + +/* Progress Bar Part API */ + +/** Create a progress bar. +@param label Progress bar label (internally duplicated). +@return New progress bar. +*/ +struct ProgressBar *progress_bar_init(const char *label); + +/** Create a new part. +@param label Part label (internally duplicated). +@param total Total units for the part. +@return New part. +*/ +struct ProgressBarPart *progress_bar_create_part(const char *label, uint64_t total); + +/** Bind a part to the progress bar. +@param bar Progress bar. +@param part Part to bind. +*/ +void progress_bar_bind_part(struct ProgressBar *bar, struct ProgressBarPart *part); + +/** Set the total for a part. +@param bar Progress bar. +@param part Part to update. +@param new_total New total units. +*/ +void progress_bar_set_part_total(struct ProgressBar *bar, struct ProgressBarPart *part, uint64_t new_total); + +/** Update the current value for a part, redraw if needed. +@param bar Progress bar. +@param part Part to advance. +@param increment Amount to add. +*/ +void progress_bar_update_part(struct ProgressBar *bar, struct ProgressBarPart *part, uint64_t increment); + +/** Set the start time for the progress bar. +@param bar Progress bar. +@param start_time Start timestamp. +*/ +void progress_bar_set_start_time(struct ProgressBar *bar, timestamp_t start_time); + +/** Finish the progress bar: draw once and print a newline. +@param bar Progress bar. +*/ +void progress_bar_finish(struct ProgressBar *bar); + +/** Delete the progress bar and free all parts. +@param bar Progress bar. +*/ +void progress_bar_delete(struct ProgressBar *bar); + +#endif diff --git a/dttools/src/progress_bar_test.c b/dttools/src/progress_bar_test.c new file mode 100644 index 0000000000..6eaab8510d --- /dev/null +++ b/dttools/src/progress_bar_test.c @@ -0,0 +1,33 @@ +#include "progress_bar.h" +#include "list.h" +#include "timestamp.h" +#include +#include + +int main() +{ + uint64_t total = 100000; + struct ProgressBarPart *part1 = progress_bar_create_part("step", total); + struct ProgressBarPart *part2 = progress_bar_create_part("fetch", total); + struct ProgressBarPart *part3 = progress_bar_create_part("commit", total); + + struct ProgressBar *bar = progress_bar_init("Compute"); + progress_bar_bind_part(bar, part1); + progress_bar_bind_part(bar, part2); + progress_bar_bind_part(bar, part3); + + timestamp_t start_time = timestamp_get(); + for (uint64_t i = 0; i < total; i++) { + progress_bar_update_part(bar, part1, 1); + progress_bar_update_part(bar, part2, 1); + progress_bar_update_part(bar, part3, 1); + } + + progress_bar_finish(bar); + progress_bar_delete(bar); + + timestamp_t end_time = timestamp_get(); + printf("time taken: %ld\n", end_time - start_time); + + return 0; +} From 354a6d4fe7f699894720c3ef480343598dcff571 Mon Sep 17 00:00:00 2001 From: Jin Zhou Date: Tue, 14 Oct 2025 10:32:13 -0400 Subject: [PATCH 017/113] add unit --- dttools/src/progress_bar_test.c | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/dttools/src/progress_bar_test.c b/dttools/src/progress_bar_test.c index 6eaab8510d..29177fafb1 100644 --- a/dttools/src/progress_bar_test.c +++ b/dttools/src/progress_bar_test.c @@ -27,7 +27,7 @@ int main() progress_bar_delete(bar); timestamp_t end_time = timestamp_get(); - printf("time taken: %ld\n", end_time - start_time); + printf("time taken: %ld us\n", end_time - start_time); return 0; } From 81b228f04c1fb83a892f80c478a3e24548448003 Mon Sep 17 00:00:00 2001 From: Jin Zhou Date: Tue, 14 Oct 2025 10:38:21 -0400 Subject: [PATCH 018/113] format issue --- dttools/src/progress_bar_test.c | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/dttools/src/progress_bar_test.c b/dttools/src/progress_bar_test.c index 29177fafb1..4c72d09425 100644 --- a/dttools/src/progress_bar_test.c +++ b/dttools/src/progress_bar_test.c @@ -27,7 +27,7 @@ int main() progress_bar_delete(bar); timestamp_t end_time = timestamp_get(); - printf("time taken: %ld us\n", end_time - start_time); + printf("time taken: %" PRIu64 "\n", end_time - start_time); return 0; } From d5dad06e85a91f4c73ae9568ce27bd10e2c76428 Mon Sep 17 00:00:00 2001 From: Jin Zhou Date: Tue, 14 Oct 2025 10:51:03 -0400 Subject: [PATCH 019/113] vine: valid link when sending message on worker --- taskvine/src/worker/vine_worker.c | 12 ++++++++++++ 1 file changed, 12 insertions(+) diff --git a/taskvine/src/worker/vine_worker.c b/taskvine/src/worker/vine_worker.c index d285d5bb4c..d1ab432cef 100644 --- a/taskvine/src/worker/vine_worker.c +++ b/taskvine/src/worker/vine_worker.c @@ -270,6 +270,10 @@ void deliver_async_messages(struct link *l) void send_async_message(struct link *l, const char *fmt, ...) { + if (!l) { + return; + } + va_list va; char *message = malloc(VINE_LINE_MAX); va_start(va, fmt); @@ -498,6 +502,10 @@ its size in bytes and transfer time in usec. void vine_worker_send_cache_update(struct link *manager, const char *cachename, struct vine_cache_file *f) { + if (!manager) { + return; + } + char *transfer_id = hash_table_remove(current_transfers, cachename); if (!transfer_id) { transfer_id = xxstrdup("X"); @@ -524,6 +532,10 @@ could not be loaded. Accompanied by a corresponding error message. void vine_worker_send_cache_invalid(struct link *manager, const char *cachename, const char *message) { + if (!manager) { + return; + } + int length = strlen(message); char *transfer_id = hash_table_remove(current_transfers, cachename); if (transfer_id) { From 39b1d66baf42cbaac0bc24dd805677dbad8254ab Mon Sep 17 00:00:00 2001 From: Jin Zhou Date: Tue, 14 Oct 2025 15:47:17 -0400 Subject: [PATCH 020/113] vine: LIST_ITERATE_REVERSE --- dttools/src/list.c | 13 +++++++++++++ dttools/src/list.h | 29 +++++++++++++++++++++++++++++ 2 files changed, 42 insertions(+) diff --git a/dttools/src/list.c b/dttools/src/list.c index 7cba58fb7c..251fce77fa 100644 --- a/dttools/src/list.c +++ b/dttools/src/list.c @@ -629,6 +629,11 @@ void list_first_item(struct list *list) list_seek(list->iter, 0); } +void list_last_item(struct list *list) +{ + list_seek(list->iter, -1); +} + void *list_next_item(struct list *list) { void *item = NULL; @@ -637,6 +642,14 @@ void *list_next_item(struct list *list) return item; } +void *list_prev_item(struct list *list) +{ + void *item = NULL; + list_get(list->iter, &item); + list_prev(list->iter); + return item; +} + struct list *list_duplicate(struct list *src) { void *item; diff --git a/dttools/src/list.h b/dttools/src/list.h index 4a75ad2379..22ec74c1c2 100644 --- a/dttools/src/list.h +++ b/dttools/src/list.h @@ -374,6 +374,14 @@ Call @ref list_next_item to begin returning the items. void list_first_item(struct list *list); +/** Begin traversing a list in reverse. +This function sets the internal list iterator to the last item. +Call @ref list_prev_item to begin returning the items in reverse order. +@param list The list to traverse. +*/ + +void list_last_item(struct list *list); + /** Continue traversing a list. This function returns the current list item, and advances the internal iterator to the next item. @@ -383,6 +391,15 @@ and advances the internal iterator to the next item. void *list_next_item(struct list *list); +/** Continue traversing a list in reverse. +This function returns the current list item, +and advances the internal iterator to the previous item. +@param list The list to traverse. +@return The current item in the list, NULL if end of list. +*/ + +void *list_prev_item(struct list *list); + /** Apply a function to a list. Invokes op on every member of the list. @param list The list to operate on. @@ -421,4 +438,16 @@ LIST_ITERATE( list, s ) { #define LIST_ITERATE( list, item ) list_first_item(list); while((item=list_next_item(list))) +/** Macro to iterate over a list in reverse order. +Note that a statement or code block must follow the macro, like this: +
+char *s;
+LIST_ITERATE_REVERSE( list, s ) {
+	printf("%s\n",s);
+}
+
+*/ + +#define LIST_ITERATE_REVERSE( list, item ) list_last_item(list); while((item=list_prev_item(list))) + #endif From e94875e447d6470eaed87a8168993441d8d5356c Mon Sep 17 00:00:00 2001 From: Jin Zhou Date: Tue, 14 Oct 2025 15:47:49 -0400 Subject: [PATCH 021/113] temp --- .../ndcctools/taskvine/graph_definition.py | 331 ++++++++ .../ndcctools/taskvine/graph_executor.py | 251 ++++++ taskvine/src/manager/Makefile | 5 +- taskvine/src/manager/taskvine.h | 121 +++ .../src/manager/vine_file_replica_table.c | 3 +- taskvine/src/manager/vine_manager.c | 392 +++++++--- taskvine/src/manager/vine_manager.h | 19 +- taskvine/src/manager/vine_task_graph.c | 739 ++++++++++++++++++ taskvine/src/manager/vine_task_graph.h | 24 + taskvine/src/manager/vine_task_node.c | 611 +++++++++++++++ taskvine/src/manager/vine_task_node.h | 76 ++ taskvine/src/manager/vine_temp.c | 288 +++++++ taskvine/src/manager/vine_temp.h | 16 + 13 files changed, 2757 insertions(+), 119 deletions(-) create mode 100644 taskvine/src/bindings/python3/ndcctools/taskvine/graph_definition.py create mode 100644 taskvine/src/bindings/python3/ndcctools/taskvine/graph_executor.py create mode 100644 taskvine/src/manager/vine_task_graph.c create mode 100644 taskvine/src/manager/vine_task_graph.h create mode 100644 taskvine/src/manager/vine_task_node.c create mode 100644 taskvine/src/manager/vine_task_node.h create mode 100644 taskvine/src/manager/vine_temp.c create mode 100644 taskvine/src/manager/vine_temp.h diff --git a/taskvine/src/bindings/python3/ndcctools/taskvine/graph_definition.py b/taskvine/src/bindings/python3/ndcctools/taskvine/graph_definition.py new file mode 100644 index 0000000000..b9db690ad2 --- /dev/null +++ b/taskvine/src/bindings/python3/ndcctools/taskvine/graph_definition.py @@ -0,0 +1,331 @@ +from ndcctools.taskvine.utils import load_variable_from_library + +import os +import hashlib +import time +import cloudpickle +import collections +import uuid +import random +from collections import deque + +try: + import dask +except ImportError: + dask = None + +try: + import dask._task_spec as dts +except ImportError: + dts = None + + +def hash_name(*args): + out_str = "" + for arg in args: + out_str += str(arg) + return hashlib.sha256(out_str.encode('utf-8')).hexdigest()[:32] + + +def hashable(s): + try: + hash(s) + return True + except TypeError: + return False + + +def dist_func(mode, low, high): + if not mode: + return 0 + + assert mode in ["uniform", "normal", "lognormal", "pareto", "mix"] + # uniform distribution, flat spread + def uniform_dist(): + return random.uniform(low, high) + + # normal distribution, centered in the middle + def normal_dist(): + mu, sigma = (low + high) / 2, (high - low) / 6 + return min(max(random.gauss(mu, sigma), low), high) + + # lognormal distribution, long tail + def lognormal_dist(): + val = random.lognormvariate(0, 1) + val = val / (1 + val) + return low + (high - low) * val + + # pareto distribution, very heavy tail + def pareto_dist(alpha=2.0): + val = random.paretovariate(alpha) + val = val / (1 + val) + return low + (high - low) * val + + # mixture: most small values, few large ones + def mix_dist(): + if random.random() < 0.9: + return random.uniform(low, (low + high) / 2) + else: + return random.uniform((low + high) / 2, high) + + return { + "uniform": uniform_dist, + "normal": normal_dist, + "lognormal": lognormal_dist, + "pareto": pareto_dist, + "mix": mix_dist, + }[mode]() + + +class GraphKeyResult: + # extra_size_mb is used to allocate more space for this object in testing mode to evaluate storage consumption + # and peer transfer performance across all workers. + 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 + + +class TaskGraph: + def __init__(self, task_dict, + shared_file_system_dir=None, + staging_dir=None, + extra_task_output_size_mb=["uniform", 0, 0], + extra_task_sleep_time=["uniform", 0, 0]): + self.task_dict = task_dict + self.shared_file_system_dir = shared_file_system_dir + self.staging_dir = staging_dir + + if self.shared_file_system_dir: + os.makedirs(self.shared_file_system_dir, exist_ok=True) + + if dts: + for k, v in self.task_dict.items(): + if isinstance(v, dts.GraphNode): + assert isinstance(v, (dts.Alias, dts.Task, dts.DataNode)), f"Unsupported task type for key {k}: {v.__class__}" + + self.parents_of, self.children_of = self._build_dependencies(self.task_dict) + self.depth_of = self._calculate_depths() + + self.vine_key_of = {k: hash_name(k) for k in task_dict.keys()} + self.key_of_vine_key = {hash_name(k): k for k in task_dict.keys()} + + self.outfile_remote_name = {key: f"{uuid.uuid4()}.pkl" for key in self.task_dict.keys()} + self.outfile_type = {key: None for key in self.task_dict.keys()} + + # testing params + self.extra_task_output_size_mb = self._calculate_extra_size_mb_of(extra_task_output_size_mb) + self.extra_sleep_time_of = self._calculate_extra_sleep_time_of(extra_task_sleep_time) + + def set_outfile_type_of(self, k, outfile_type_str): + assert outfile_type_str in ["local", "shared-file-system", "temp"] + self.outfile_type[k] = outfile_type_str + if outfile_type_str == "shared-file-system": + self.outfile_remote_name[k] = os.path.join(self.shared_file_system_dir, self.outfile_remote_name[k]) + + def _calculate_extra_size_mb_of(self, extra_task_output_size_mb): + assert isinstance(extra_task_output_size_mb, list) and len(extra_task_output_size_mb) == 3 + mode, low, high = extra_task_output_size_mb + low, high = int(low), int(high) + assert low <= high + + max_depth = max(depth for depth in self.depth_of.values()) + extra_size_mb_of = {} + for k in self.task_dict.keys(): + if self.depth_of[k] == max_depth or self.depth_of[k] == max_depth - 1: + extra_size_mb_of[k] = 0 + continue + extra_size_mb_of[k] = dist_func(mode, low, high) + + return extra_size_mb_of + + def _calculate_extra_sleep_time_of(self, extra_task_sleep_time): + assert isinstance(extra_task_sleep_time, list) and len(extra_task_sleep_time) == 3 + mode, low, high = extra_task_sleep_time + low, high = int(low), int(high) + assert low <= high + + extra_sleep_time_of = {} + for k in self.task_dict.keys(): + extra_sleep_time_of[k] = dist_func(mode, low, high) + + return extra_sleep_time_of + + def _calculate_depths(self): + depth_of = {key: 0 for key in self.task_dict.keys()} + + topo_order = self.get_topological_order() + for key in topo_order: + if self.parents_of[key]: + depth_of[key] = max(depth_of[parent] for parent in self.parents_of[key]) + 1 + else: + depth_of[key] = 0 + + return depth_of + + def set_outfile_remote_name_of(self, key, outfile_remote_name): + self.outfile_remote_name[key] = outfile_remote_name + + def is_dts_key(self, k): + if not hasattr(dask, "_task_spec"): + return False + import dask._task_spec as dts + return isinstance(self.task_dict[k], (dts.Task, dts.TaskRef, dts.Alias, dts.DataNode, dts.NestedContainer)) + + def _build_dependencies(self, task_dict): + def _find_sexpr_parents(sexpr): + if hashable(sexpr) and sexpr in task_dict.keys(): + return {sexpr} + elif isinstance(sexpr, (list, tuple)): + deps = set() + for x in sexpr: + deps |= _find_sexpr_parents(x) + return deps + elif isinstance(sexpr, dict): + deps = set() + for k, v in sexpr.items(): + deps |= _find_sexpr_parents(k) + deps |= _find_sexpr_parents(v) + return deps + else: + return set() + + parents_of = collections.defaultdict(set) + children_of = collections.defaultdict(set) + + for k, value in task_dict.items(): + if self.is_dts_key(k): + # in the new Dask expression, each value is an object from dask._task_spec, could be + # a Task, Alias, TaskRef, etc., but they all share the same base class the dependencies + # field is of type frozenset(), without recursive ancestor dependencies involved + parents_of[k] = value.dependencies + else: + # the value could be a sexpr, e.g., the old Dask representation + parents_of[k] = _find_sexpr_parents(value) + + for k, deps in parents_of.items(): + for dep in deps: + children_of[dep].add(k) + + return parents_of, children_of + + def save_result_of_key(self, key, result): + with open(self.outfile_remote_name[key], "wb") as f: + result_obj = GraphKeyResult(result, extra_size_mb=self.extra_task_output_size_mb[key]) + cloudpickle.dump(result_obj, f) + + def load_result_of_key(self, key): + try: + with open(self.outfile_remote_name[key], "rb") as f: + result_obj = cloudpickle.load(f) + assert isinstance(result_obj, GraphKeyResult), "Loaded object is not of type GraphKeyResult" + return result_obj.result + except FileNotFoundError: + raise FileNotFoundError(f"Output file for key {key} not found at {self.outfile_remote_name[key]}") + + def get_topological_order(self): + in_degree = {key: len(self.parents_of[key]) for key in self.task_dict.keys()} + queue = deque([key for key, degree in in_degree.items() if degree == 0]) + topo_order = [] + + while queue: + current = queue.popleft() + topo_order.append(current) + + for child in self.children_of[current]: + in_degree[child] -= 1 + if in_degree[child] == 0: + queue.append(child) + + if len(topo_order) != len(self.task_dict): + print(f"len(topo_order): {len(topo_order)}") + print(f"len(self.task_dict): {len(self.task_dict)}") + raise ValueError("Failed to create topo order, the dependencies may be cyclic or problematic") + + return topo_order + + def __del__(self): + if hasattr(self, 'outfile_remote_name') and self.outfile_remote_name: + for k in self.outfile_remote_name.keys(): + if self.outfile_type.get(k) == "shared-file-system" and os.path.exists(self.outfile_remote_name[k]): + os.remove(self.outfile_remote_name[k]) + + +def init_task_graph_context(task_graph_path=None): + if task_graph_path is None: + + raise ValueError("task_graph_path must be provided to initialize the task graph context") + if not os.path.exists(task_graph_path): + raise FileNotFoundError(f"Task graph file not found at {task_graph_path}") + + with open(task_graph_path, 'rb') as f: + task_graph = cloudpickle.load(f) + + return { + 'task_graph': task_graph, + } + + +def compute_dts_key(task_graph, k, v): + try: + import dask._task_spec as dts + except ImportError: + raise ImportError("Dask is not installed") + + input_dict = {dep: task_graph.load_result_of_key(dep) for dep in v.dependencies} + + try: + if isinstance(v, dts.Alias): + assert len(v.dependencies) == 1, "Expected exactly one dependency" + return task_graph.load_result_of_key(next(iter(v.dependencies))) + elif isinstance(v, dts.Task): + return v(input_dict) + elif isinstance(v, dts.DataNode): + return v.value + else: + raise TypeError(f"unexpected node type: {type(v)} for key {k}") + except Exception as e: + raise Exception(f"Error while executing task {k}: {e}") + + +def compute_sexpr_key(task_graph, k, v): + input_dict = {parent: task_graph.load_result_of_key(parent) for parent in task_graph.parents_of[k]} + + def _rec_call(expr): + try: + if expr in input_dict.keys(): + return input_dict[expr] + except TypeError: + pass + if isinstance(expr, list): + return [_rec_call(e) for e in expr] + if isinstance(expr, tuple) and len(expr) > 0 and callable(expr[0]): + res = expr[0](*[_rec_call(a) for a in expr[1:]]) + return res + return expr + + try: + return _rec_call(v) + except Exception as e: + raise Exception(f"Failed to invoke _rec_call(): {e}") + + +def compute_single_key(vine_key): + task_graph = load_variable_from_library('task_graph') + + k = task_graph.key_of_vine_key[vine_key] + v = task_graph.task_dict[k] + + if task_graph.is_dts_key(k): + result = compute_dts_key(task_graph, k, v) + else: + result = compute_sexpr_key(task_graph, k, v) + + task_graph.save_result_of_key(k, result) + if not os.path.exists(task_graph.outfile_remote_name[k]): + raise Exception(f"Output file {task_graph.outfile_remote_name[k]} does not exist after writing") + if os.stat(task_graph.outfile_remote_name[k]).st_size == 0: + raise Exception(f"Output file {task_graph.outfile_remote_name[k]} is empty after writing") + + time.sleep(task_graph.extra_sleep_time_of[k]) + + return True diff --git a/taskvine/src/bindings/python3/ndcctools/taskvine/graph_executor.py b/taskvine/src/bindings/python3/ndcctools/taskvine/graph_executor.py new file mode 100644 index 0000000000..7efca4a554 --- /dev/null +++ b/taskvine/src/bindings/python3/ndcctools/taskvine/graph_executor.py @@ -0,0 +1,251 @@ +from ndcctools.taskvine import cvine +from ndcctools.taskvine.manager import Manager +from ndcctools.taskvine.utils import load_variable_from_library, delete_all_files, get_c_constant +from ndcctools.taskvine.graph_definition import GraphKeyResult, TaskGraph, init_task_graph_context, compute_dts_key, compute_sexpr_key, compute_single_key, hash_name, hashable + +import cloudpickle +import os +import collections +import inspect +import types +import signal +import hashlib +import time +import random +import uuid + +try: + import dask +except ImportError: + dask = None + +try: + from dask.base import is_dask_collection +except ImportError: + is_dask_collection = None + +try: + import dask._task_spec as dts +except ImportError: + dts = None + + +def dask_collections_to_task_dict(collection_dict): + assert is_dask_collection is not None + from dask.highlevelgraph import HighLevelGraph, ensure_dict + + if not isinstance(collection_dict, dict): + raise TypeError("Input must be a dict or a HighLevelGraph") + + for k, v in collection_dict.items(): + if not is_dask_collection(v): + raise TypeError(f"Input must be a dict of DaskCollection, but found {k} with type {type(v)}") + + if dts: + sub_hlgs = [v.dask for v in collection_dict.values()] + hlg = HighLevelGraph.merge(*sub_hlgs).to_dict() + else: + hlg = dask.base.collections_to_dsk(collection_dict.values()) + + return ensure_dict(hlg) + + +def ensure_task_dict(collection_dict): + if is_dask_collection and any(is_dask_collection(v) for v in collection_dict.values()): + task_dict = dask_collections_to_task_dict(collection_dict) + else: + task_dict = collection_dict + + if dts: + return dts.convert_legacy_graph(task_dict) + else: + return task_dict + + +class GraphExecutor(Manager): + def __init__(self, + *args, + libcores=1, + hoisting_modules=[], + libtask_env_files={}, + **kwargs): + + signal.signal(signal.SIGINT, self._on_sigint) + + # delete all files in the run info template directory, do this before super().__init__() + self.run_info_path = kwargs.get('run_info_path') + self.run_info_template = kwargs.get('run_info_template') + self.run_info_path_absolute = os.path.join(self.run_info_path, self.run_info_template) + if self.run_info_path and self.run_info_template: + delete_all_files(self.run_info_path_absolute) + + # initialize the manager + super_params = set(inspect.signature(Manager.__init__).parameters) + super_kwargs = {k: v for k, v in kwargs.items() if k in super_params} + + super().__init__(*args, **super_kwargs) + print(f"TaskVine manager \033[92m{self.name}\033[0m listening on port \033[92m{self.port}\033[0m") + + # tune the manager + leftover_kwargs = {k: v for k, v in kwargs.items() if k not in super_params} + for key, value in leftover_kwargs.items(): + try: + vine_param = key.replace("_", "-") + self.tune(vine_param, value) + print(f"Tuned {vine_param} to {value}") + except Exception as e: + print(f"Failed to tune {key} with value {value}: {e}") + exit(1) + self.tune("worker-source-max-transfers", 100) + self.tune("max-retrievals", -1) + self.tune("prefer-dispatch", 1) + self.tune("transient-error-interval", 1) + self.tune("attempt-schedule-depth", 1000) + + # initialize the task graph + self._vine_task_graph = cvine.vine_task_graph_create(self._taskvine) + + # create library task with specified resources + self._create_library_task(libcores, hoisting_modules, libtask_env_files) + + def _create_library_task(self, libcores=1, hoisting_modules=[], libtask_env_files={}): + assert cvine.vine_task_graph_get_function_name(self._vine_task_graph) == compute_single_key.__name__ + + self.task_graph_pkl_file_name = f"library-task-graph-{uuid.uuid4()}.pkl" + self.task_graph_pkl_file_local_path = self.task_graph_pkl_file_name + self.task_graph_pkl_file_remote_path = self.task_graph_pkl_file_name + + hoisting_modules += [os, cloudpickle, GraphKeyResult, TaskGraph, uuid, hashlib, random, types, collections, time, + load_variable_from_library, compute_dts_key, compute_sexpr_key, compute_single_key, hash_name, hashable] + if dask: + hoisting_modules += [dask] + self.libtask = self.create_library_from_functions( + cvine.vine_task_graph_get_library_name(self._vine_task_graph), + compute_single_key, + library_context_info=[init_task_graph_context, [], {"task_graph_path": self.task_graph_pkl_file_remote_path}], + add_env=False, + infile_load_mode="text", + hoisting_modules=hoisting_modules + ) + self.libtask.add_input(self.declare_file(self.task_graph_pkl_file_local_path), self.task_graph_pkl_file_remote_path) + for local_file_path, remote_file_path in libtask_env_files.items(): + self.libtask.add_input(self.declare_file(local_file_path, cache=True, peer_transfer=True), remote_file_path) + self.libtask.set_cores(libcores) + self.libtask.set_function_slots(libcores) + self.install_library(self.libtask) + + def run(self, + collection_dict, + target_keys=[], + replica_placement_policy="random", + priority_mode="largest-input-first", + scheduling_mode="files", + extra_task_output_size_mb=["uniform", 0, 0], + extra_task_sleep_time=["uniform", 0, 0], + prune_depth=1, + shared_file_system_dir="/project01/ndcms/jzhou24/shared_file_system", + staging_dir="/project01/ndcms/jzhou24/staging", + failure_injection_step_percent=-1, + balance_worker_disk_load=0, + outfile_type={ + "temp": 1.0, + "shared-file-system": 0.0, + }): + self.target_keys = target_keys + self.task_dict = ensure_task_dict(collection_dict) + + self.tune("balance-worker-disk-load", balance_worker_disk_load) + + cvine.vine_task_graph_set_failure_injection_step_percent(self._vine_task_graph, failure_injection_step_percent) + + if balance_worker_disk_load: + replica_placement_policy = "disk-load" + scheduling_mode = "worst" + + self.set_scheduler(scheduling_mode) + + # create task graph in the python side + print("Initializing TaskGraph object") + self.shared_file_system_dir = shared_file_system_dir + self.staging_dir = staging_dir + self.task_graph = TaskGraph(self.task_dict, + staging_dir=self.staging_dir, + shared_file_system_dir=self.shared_file_system_dir, + extra_task_output_size_mb=extra_task_output_size_mb, + extra_task_sleep_time=extra_task_sleep_time) + topo_order = self.task_graph.get_topological_order() + + # the sum of the values in outfile_type must be 1.0 + assert sum(list(outfile_type.values())) == 1.0 + + # set replica placement policy + cvine.vine_set_replica_placement_policy(self._taskvine, get_c_constant(f"replica_placement_policy_{replica_placement_policy.replace('-', '_')}")) + + # create task graph in the python side + print("Initializing task graph in TaskVine") + for k in topo_order: + cvine.vine_task_graph_create_node(self._vine_task_graph, + self.task_graph.vine_key_of[k], + self.staging_dir, + prune_depth, + get_c_constant(f"task_priority_mode_{priority_mode.replace('-', '_')}")) + for pk in self.task_graph.parents_of.get(k, []): + cvine.vine_task_graph_add_dependency(self._vine_task_graph, self.task_graph.vine_key_of[pk], self.task_graph.vine_key_of[k]) + + # we must finalize the graph in c side after all nodes and dependencies are added + # this includes computing various metrics for each node, such as depth, height, heavy score, etc. + cvine.vine_task_graph_finalize_metrics(self._vine_task_graph) + + # then we can use the heavy score to sort the nodes and specify their outfile remote names + heavy_scores = {} + for k in self.task_graph.task_dict.keys(): + heavy_scores[k] = cvine.vine_task_graph_get_node_heavy_score(self._vine_task_graph, self.task_graph.vine_key_of[k]) + + # keys with larger heavy score should be stored into the shared file system + sorted_keys = sorted(heavy_scores, key=lambda x: heavy_scores[x], reverse=True) + shared_file_system_size = round(len(sorted_keys) * outfile_type["shared-file-system"]) + for i, k in enumerate(sorted_keys): + if k in self.target_keys: + choice = "local" + else: + if i < shared_file_system_size: + choice = "shared-file-system" + else: + choice = "temp" + self.task_graph.set_outfile_type_of(k, choice) + outfile_type_str = f"NODE_OUTFILE_TYPE_{choice.upper().replace('-', '_')}" + cvine.vine_task_graph_set_node_outfile(self._vine_task_graph, + self.task_graph.vine_key_of[k], + get_c_constant(outfile_type_str), + self.task_graph.outfile_remote_name[k]) + + # save the task graph to a pickle file, will be sent to the remote workers + with open(self.task_graph_pkl_file_local_path, 'wb') as f: + cloudpickle.dump(self.task_graph, f) + + # now execute the vine graph + print(f"Executing task graph, logs will be written into {self.run_info_path_absolute}") + cvine.vine_task_graph_execute(self._vine_task_graph) + + # after execution, we need to load results of target keys + results = {} + for k in self.target_keys: + local_outfile_path = cvine.vine_task_graph_get_node_local_outfile_source(self._vine_task_graph, self.task_graph.vine_key_of[k]) + if not os.path.exists(local_outfile_path): + results[k] = "NOT_FOUND" + continue + with open(local_outfile_path, 'rb') as f: + result_obj = cloudpickle.load(f) + assert isinstance(result_obj, GraphKeyResult), "Loaded object is not of type GraphKeyResult" + results[k] = result_obj.result + return results + + def _on_sigint(self, signum, frame): + self.__del__() + + def __del__(self): + if hasattr(self, '_vine_task_graph') and self._vine_task_graph: + cvine.vine_task_graph_delete(self._vine_task_graph) + + if hasattr(self, 'task_graph_pkl_file_local_path') and os.path.exists(self.task_graph_pkl_file_local_path): + os.remove(self.task_graph_pkl_file_local_path) diff --git a/taskvine/src/manager/Makefile b/taskvine/src/manager/Makefile index a036e9bd64..1246f83737 100644 --- a/taskvine/src/manager/Makefile +++ b/taskvine/src/manager/Makefile @@ -28,7 +28,10 @@ SOURCES = \ vine_file_replica_table.c \ vine_fair.c \ vine_runtime_dir.c \ - vine_task_groups.c + vine_task_groups.c \ + vine_temp.c \ + vine_task_node.c \ + vine_task_graph.c PUBLIC_HEADERS = taskvine.h diff --git a/taskvine/src/manager/taskvine.h b/taskvine/src/manager/taskvine.h index 16152b7102..9e1d9b7da3 100644 --- a/taskvine/src/manager/taskvine.h +++ b/taskvine/src/manager/taskvine.h @@ -15,6 +15,7 @@ See the file COPYING for details. struct vine_manager; struct vine_task; struct vine_file; +struct vine_task_graph; /** @file taskvine.h The public API for the taskvine distributed application framework. A taskvine application consists of a manager process and a larger number of worker @@ -149,6 +150,30 @@ typedef enum { VINE_MINI_TASK, /**< A file obtained by executing a Unix command line. */ } vine_file_type_t; +/* Select while type of workers to place the redundant file replicas */ +typedef enum { + VINE_REPLICA_PLACEMENT_POLICY_RANDOM = 0, /* select a random worker */ + VINE_REPLICA_PLACEMENT_POLICY_DISK_LOAD, /* select a worker with the most free disk space */ + VINE_REPLICA_PLACEMENT_POLICY_TRANSFER_LOAD /* select a worker with the least incoming transfer load */ +} vine_replica_placement_policy_t; + +/** Select the type of the node-output file. */ +typedef enum { + VINE_NODE_OUTFILE_TYPE_LOCAL = 0, /* Node-output file will be stored locally on the manager's staging directory */ + VINE_NODE_OUTFILE_TYPE_TEMP, /* Node-output file will be stored in the temporary node-local storage */ + VINE_NODE_OUTFILE_TYPE_SHARED_FILE_SYSTEM, /* Node-output file will be stored in the persistent shared file system */ +} vine_task_node_outfile_type_t; + +/** Select priority algorithm for task graph task scheduling. */ +typedef enum { + VINE_TASK_PRIORITY_MODE_RANDOM = 0, /**< Assign random priority to tasks */ + VINE_TASK_PRIORITY_MODE_DEPTH_FIRST, /**< Prioritize deeper tasks first */ + VINE_TASK_PRIORITY_MODE_BREADTH_FIRST, /**< Prioritize shallower tasks first */ + VINE_TASK_PRIORITY_MODE_FIFO, /**< First in, first out priority */ + VINE_TASK_PRIORITY_MODE_LIFO, /**< Last in, first out priority */ + VINE_TASK_PRIORITY_MODE_LARGEST_INPUT_FIRST, /**< Prioritize tasks with larger inputs first */ + VINE_TASK_PRIORITY_MODE_LARGEST_STORAGE_FOOTPRINT_FIRST /**< Prioritize tasks with larger storage footprint first */ +} vine_task_node_priority_mode_t; /** Statistics describing a manager. */ struct vine_stats { @@ -1117,6 +1142,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 @@ -1556,6 +1589,94 @@ char *vine_get_path_library_log(struct vine_manager *m, const char *path); */ char *vine_get_path_cache(struct vine_manager *m, const char *path); +/** Create a task graph object and return it. +@param q Reference to the current manager object. +@return A new task graph object. +*/ +struct vine_task_graph *vine_task_graph_create(struct vine_manager *q); + +/** Delete a task graph object. +@param tg Reference to the task graph object. +*/ +void vine_task_graph_delete(struct vine_task_graph *tg); + +/** Add a dependency between two nodes in the task graph. +@param tg Reference to the task graph object. +@param parent_key Reference to the parent node key. +@param child_key Reference to the child node key. +*/ +void vine_task_graph_add_dependency(struct vine_task_graph *tg, const char *parent_key, const char *child_key); + +/** Create a new node in the task graph. +@param tg Reference to the task graph object. +@param node_key Reference to the node key. +@param staging_dir Reference to the staging directory. +@param prune_depth Reference to the prune depth. +@param priority_mode Reference to the priority mode. +@return A new node object. +*/ +struct vine_task_node *vine_task_graph_create_node(struct vine_task_graph *tg, + const char *node_key, + const char *staging_dir, + int prune_depth, + vine_task_node_priority_mode_t priority_mode); + +/** Set the replica placement policy (which worker do we prefer to place the redundant file replicas). +@param q Reference to the current manager object. +@param policy Reference to the replica placement policy. +*/ +void vine_set_replica_placement_policy(struct vine_manager *q, vine_replica_placement_policy_t policy); + +/** Set the type of the node-output file. +@param tg Reference to the task graph object. +@param node_key Reference to the node key. +@param outfile_type Reference to the output file type. +@param outfile_remote_name Reference to the output file remote name. +*/ +void vine_task_graph_set_node_outfile(struct vine_task_graph *tg, const char *node_key, vine_task_node_outfile_type_t outfile_type, const char *outfile_remote_name); + +/** Finalize the metrics of the task graph. +@param tg Reference to the task graph object. +*/ +void vine_task_graph_finalize_metrics(struct vine_task_graph *tg); + +/** Get the library name of the task graph. +@param tg Reference to the task graph object. +@return The library name. +*/ +const char *vine_task_graph_get_library_name(const struct vine_task_graph *tg); + +/** Get the function name of the task graph. +@param tg Reference to the task graph object. +@return The function name. +*/ +const char *vine_task_graph_get_function_name(const struct vine_task_graph *tg); + +/** Get the heavy score of a node in the task graph. +@param tg Reference to the task graph object. +@param node_key Reference to the node key. +@return The heavy score. +*/ +double vine_task_graph_get_node_heavy_score(const struct vine_task_graph *tg, const char *node_key); + +/** Execute the task graph. +@param tg Reference to the task graph object. +*/ +void vine_task_graph_execute(struct vine_task_graph *tg); + +/** Get the local outfile source of a node in the task graph. +@param tg Reference to the task graph object. +@param node_key Reference to the node key. +@return The local outfile source. +*/ +const char *vine_task_graph_get_node_local_outfile_source(const struct vine_task_graph *tg, const char *node_key); + +/** Set the failure injection step percent. +@param tg Reference to the task graph object. +@param percent Reference to the failure injection step percent. +*/ +void vine_task_graph_set_failure_injection_step_percent(struct vine_task_graph *tg, double percent); + //@} #endif diff --git a/taskvine/src/manager/vine_file_replica_table.c b/taskvine/src/manager/vine_file_replica_table.c index d6ba403f5b..7619a01833 100644 --- a/taskvine/src/manager/vine_file_replica_table.c +++ b/taskvine/src/manager/vine_file_replica_table.c @@ -25,7 +25,8 @@ See the file COPYING for details. int vine_file_replica_table_insert(struct vine_manager *m, struct vine_worker_info *w, const char *cachename, struct vine_file_replica *replica) { if (hash_table_lookup(w->current_files, cachename)) { - return 0; + // delete the previous replcia because the replica's size might have changed + vine_file_replica_table_remove(m, w, cachename); } double prev_available = w->resources->disk.total - BYTES_TO_MEGABYTES(w->inuse_cache); diff --git a/taskvine/src/manager/vine_manager.c b/taskvine/src/manager/vine_manager.c index ca72d257d9..07bdb9a5a1 100644 --- a/taskvine/src/manager/vine_manager.c +++ b/taskvine/src/manager/vine_manager.c @@ -29,6 +29,7 @@ See the file COPYING for details. #include "vine_taskgraph_log.h" #include "vine_txn_log.h" #include "vine_worker_info.h" +#include "vine_temp.h" #include "address.h" #include "buffer.h" @@ -165,11 +166,12 @@ static int vine_manager_check_inputs_available(struct vine_manager *q, struct vi static void vine_manager_consider_recovery_task(struct vine_manager *q, struct vine_file *lost_file, struct vine_task *rt); static void delete_uncacheable_files(struct vine_manager *q, struct vine_worker_info *w, struct vine_task *t); -static int delete_worker_file(struct vine_manager *q, struct vine_worker_info *w, const char *filename, vine_cache_level_t cache_level, vine_cache_level_t delete_upto_level); static int release_worker(struct vine_manager *q, struct vine_worker_info *w); struct vine_task *send_library_to_worker(struct vine_manager *q, struct vine_worker_info *w, const char *name); +static void clean_redundant_replicas(struct vine_manager *q, struct vine_file *f); + /* Return the number of workers matching a given type: WORKER, STATUS, etc */ static int count_workers(struct vine_manager *q, vine_worker_type_t type) @@ -418,9 +420,19 @@ static vine_msg_code_t handle_cache_update(struct vine_manager *q, struct vine_w f->state = VINE_FILE_STATE_CREATED; f->size = size; - /* And if the file is a newly created temporary, replicate as needed. */ - if (f->type == VINE_TEMP && *id == 'X' && q->temp_replica_count > 1) { - hash_table_insert(q->temp_files_to_replicate, f->cached_name, NULL); + /* If the replica's type was a URL, it means the manager expected the destination worker to download it + * from elsewhere. Now that it's physically present, we can resolve its type back to the original */ + if (replica->type == VINE_URL) { + replica->type = f->type; + } + + /* If a TEMP file, replicate as needed. */ + if (f->type == VINE_TEMP) { + vine_temp_replicate_file_later(q, f); + + if (q->balance_worker_disk_load) { + clean_redundant_replicas(q, f); + } } } } @@ -476,6 +488,9 @@ static vine_msg_code_t handle_cache_invalid(struct vine_manager *q, struct vine_ w->last_failure_time = timestamp_get(); } + /* If the creation failed, we may want to backup the file somewhere else. */ + vine_temp_handle_file_lost(q, cachename); + /* Successfully processed this message. */ return VINE_MSG_PROCESSED; } else { @@ -650,6 +665,15 @@ static vine_result_code_t get_completion_result(struct vine_manager *q, struct v itable_remove(q->running_table, t->task_id); vine_task_set_result(t, task_status); + /* Clean redundant replicas for the inputs */ + struct vine_mount *input_mount; + LIST_ITERATE(t->input_mounts, input_mount) + { + if (input_mount->file && input_mount->file->type == VINE_TEMP) { + clean_redundant_replicas(q, input_mount->file); + } + } + return VINE_SUCCESS; } @@ -935,8 +959,74 @@ static void cleanup_worker_files(struct vine_manager *q, struct vine_worker_info hash_table_free_keys_array(cachenames); } -/* -This function enforces a target worker eviction rate (1 every X seconds). +/** Check if a file is busy by checking if it is an input file of any task. */ +static int is_file_busy(struct vine_manager *q, struct vine_worker_info *w, struct vine_file *f) +{ + if (!q || !w || !f) { + return 0; + } + + uint64_t task_id; + struct vine_task *task; + ITABLE_ITERATE(w->current_tasks, task_id, task) + { + struct vine_mount *input_mount; + LIST_ITERATE(task->input_mounts, input_mount) + { + if (f == input_mount->file) { + return 1; + } + } + } + + return 0; +} + +/** Evict a random worker to simulate a worker failure. */ +int evict_random_worker(struct vine_manager *q) +{ + if (!q) { + return 0; + } + + if (hash_table_size(q->worker_table) == 0) { + return 0; + } + + int removed = 0; + + /* collect removable workers */ + struct list *candidates_list = list_create(); + char *key; + struct vine_worker_info *w; + HASH_TABLE_ITERATE(q->worker_table, key, w) + { + list_push_tail(candidates_list, w); + } + + /* release a random worker if any */ + int random_number = random_int64(); + if (random_number < 0) { + random_number = -random_number; + } + int index = (int)(random_number % list_size(candidates_list)); + int i = 0; + while ((w = list_pop_head(candidates_list))) { + if (i++ == index) { + /* evict this worker */ + debug(D_VINE | D_NOTICE, "Intentionally evicting worker %s", w->hostname); + release_worker(q, w); + removed = 1; + break; + } + } + + list_delete(candidates_list); + return removed; +} + +/** +Enforces a target worker eviction rate (1 every X seconds). If the observed eviction interval is shorter than the desired one, we randomly evict one worker to keep the eviction pace aligned with the target. This includes all types of removals, whether graceful or due to failures. @@ -973,32 +1063,150 @@ static int enforce_worker_eviction_interval(struct vine_manager *q) return 0; } - /* collect removable workers */ - struct list *candidates_list = list_create(); + /* evict a random worker if any */ + return evict_random_worker(q); +} + +/** Get the available disk space in bytes for a worker. */ +int64_t get_worker_available_disk_bytes(struct vine_worker_info *w) +{ + if (!w || !w->resources) { + return 0; + } + + return (int64_t)MEGABYTES_TO_BYTES(w->resources->disk.total) - w->inuse_cache; +} + +/** Clean redundant replicas of a temporary file. */ +static void clean_redundant_replicas(struct vine_manager *q, struct vine_file *f) +{ + if (!f || f->type != VINE_TEMP) { + return; + } + + // remove excess replicas of temporary files + struct set *source_workers = hash_table_lookup(q->file_worker_table, f->cached_name); + if (!source_workers) { + // no surprise - a cache-update may trigger a file deletion! + return; + } + int replicas_to_remove = set_size(source_workers) - q->temp_replica_count; + if (replicas_to_remove <= 0) { + return; + } + // note that this replica can be a source to a peer transfer, if this is unlinked, + // a corresponding transfer may fail and result in a forsaken task + // therefore, we need to wait until all replicas are ready + if (vine_file_replica_table_count_replicas(q, f->cached_name, VINE_FILE_REPLICA_STATE_READY) != set_size(source_workers)) { + return; + } + + struct priority_queue *offload_from_workers = priority_queue_create(0); + + struct vine_worker_info *source_worker = NULL; + SET_ITERATE(source_workers, source_worker) + { + // workers with more used disk are prioritized for removing + if (is_file_busy(q, source_worker, f)) { + continue; + } + + priority_queue_push(offload_from_workers, source_worker, source_worker->inuse_cache); + } + + struct vine_worker_info *offload_from_worker = NULL; + while (replicas_to_remove-- > 0 && (offload_from_worker = priority_queue_pop(offload_from_workers))) { + delete_worker_file(q, offload_from_worker, f->cached_name, 0, 0); + } + priority_queue_delete(offload_from_workers); + + return; +} + +/* Shift disk load between workers to balance the disk usage. */ +static void rebalance_worker_disk_usage(struct vine_manager *q) +{ + if (!q) { + return; + } + + struct vine_worker_info *worker_with_min_disk_usage = NULL; + struct vine_worker_info *worker_with_max_disk_usage = NULL; + char *key; struct vine_worker_info *w; HASH_TABLE_ITERATE(q->worker_table, key, w) { - if (w->type != VINE_WORKER_TYPE_WORKER) { + if (!w->transfer_port_active) { continue; } - list_push_tail(candidates_list, w); + if (w->draining) { + continue; + } + if (!w->resources) { + continue; + } + if (w->resources->tag < 0 || w->resources->disk.total < 1) { + continue; + } + if (!worker_with_min_disk_usage || w->inuse_cache < worker_with_min_disk_usage->inuse_cache) { + if (w->incoming_xfer_counter < q->worker_source_max_transfers) { + worker_with_min_disk_usage = w; + } + } + if (!worker_with_max_disk_usage || w->inuse_cache > worker_with_max_disk_usage->inuse_cache) { + if (w->outgoing_xfer_counter < q->worker_source_max_transfers) { + worker_with_max_disk_usage = w; + } + } } - /* release a random worker if any */ - int index = (int)(random_int64() % list_size(candidates_list)); - int i = 0; - while ((w = list_pop_head(candidates_list))) { - if (i++ == index) { - /* evict this worker */ - debug(D_VINE | D_NOTICE, "Intentionally evicting worker %s", w->hostname); - release_worker(q, w); + if (!worker_with_min_disk_usage || !worker_with_max_disk_usage || worker_with_min_disk_usage == worker_with_max_disk_usage) { + return; + } + + int64_t min_inuse_cache = worker_with_min_disk_usage->inuse_cache; + int64_t max_inuse_cache = worker_with_max_disk_usage->inuse_cache; + + if (min_inuse_cache * 1.2 >= max_inuse_cache) { + return; + } + + if (max_inuse_cache <= q->peak_used_cache) { + return; + } + q->peak_used_cache = max_inuse_cache; + + int64_t bytes_to_offload = (int64_t)((max_inuse_cache - min_inuse_cache) / 2); + + char *cachename; + struct vine_file_replica *replica; + HASH_TABLE_ITERATE(worker_with_max_disk_usage->current_files, cachename, replica) + { + if (replica->type != VINE_TEMP) { + continue; + } + struct vine_file *f = hash_table_lookup(q->file_table, cachename); + if (!f) { + continue; + } + if (vine_file_replica_table_lookup(worker_with_min_disk_usage, cachename)) { + continue; + } + + vine_temp_start_peer_transfer(q, f, worker_with_max_disk_usage, worker_with_min_disk_usage); + bytes_to_offload -= replica->size; + if (bytes_to_offload <= 0) { break; } - } - list_delete(candidates_list); - return 1; + if (worker_with_min_disk_usage->incoming_xfer_counter >= q->worker_source_max_transfers) { + break; + } + if (worker_with_max_disk_usage->outgoing_xfer_counter >= q->worker_source_max_transfers) { + break; + } + } } /* Remove all tasks and other associated state from a given worker. */ @@ -1036,85 +1244,6 @@ static void cleanup_worker(struct vine_manager *q, struct vine_worker_info *w) cleanup_worker_files(q, w); } -/* Start replicating files that may need replication */ -static int consider_tempfile_replications(struct vine_manager *q) -{ - if (hash_table_size(q->temp_files_to_replicate) <= 0) { - return 0; - } - - char *cached_name = NULL; - void *empty_val = NULL; - int total_replication_request_sent = 0; - - static char key_start[PATH_MAX] = "random init"; - int iter_control; - int iter_count_var; - - struct list *to_remove = list_create(); - - HASH_TABLE_ITERATE_FROM_KEY(q->temp_files_to_replicate, iter_control, iter_count_var, key_start, cached_name, empty_val) - { - struct vine_file *f = hash_table_lookup(q->file_table, cached_name); - - if (!f) { - continue; - } - - /* are there any available source workers? */ - struct set *source_workers = hash_table_lookup(q->file_worker_table, f->cached_name); - if (!source_workers) { - /* If no source workers found, it indicates that the file doesn't exist, either pruned or lost. - Because a pruned file is removed from the recovery queue, so it definitely indicates that the file is lost. */ - if (q->transfer_temps_recovery && file_needs_recovery(q, f)) { - vine_manager_consider_recovery_task(q, f, f->recovery_task); - } - list_push_tail(to_remove, xxstrdup(f->cached_name)); - continue; - } - - /* at least one source is able to transfer? */ - int has_valid_source = 0; - struct vine_worker_info *s; - SET_ITERATE(source_workers, s) - { - if (s->transfer_port_active && s->outgoing_xfer_counter < q->worker_source_max_transfers && !s->draining) { - has_valid_source = 1; - break; - } - } - if (!has_valid_source) { - continue; - } - - /* has this file been fully replicated? */ - int nsource_workers = set_size(source_workers); - int to_find = MIN(q->temp_replica_count - nsource_workers, q->transfer_replica_per_cycle); - if (to_find <= 0) { - list_push_tail(to_remove, xxstrdup(f->cached_name)); - continue; - } - - // debug(D_VINE, "Found %d workers holding %s, %d replicas needed", nsource_workers, f->cached_name, to_find); - - int round_replication_request_sent = vine_file_replica_table_replicate(q, f, source_workers, to_find); - total_replication_request_sent += round_replication_request_sent; - - if (total_replication_request_sent >= q->attempt_schedule_depth) { - break; - } - } - - while ((cached_name = list_pop_head(to_remove))) { - hash_table_remove(q->temp_files_to_replicate, cached_name); - free(cached_name); - } - - list_delete(to_remove); - - return total_replication_request_sent; -} - /* Insert into hashtable temp files that may need replication. */ static void recall_worker_lost_temp_files(struct vine_manager *q, struct vine_worker_info *w) @@ -1127,11 +1256,7 @@ static void recall_worker_lost_temp_files(struct vine_manager *q, struct vine_wo // Iterate over files we want might want to recover HASH_TABLE_ITERATE(w->current_files, cached_name, info) { - struct vine_file *f = hash_table_lookup(q->file_table, cached_name); - - if (f && f->type == VINE_TEMP) { - hash_table_insert(q->temp_files_to_replicate, cached_name, NULL); - } + vine_temp_handle_file_lost(q, cached_name); } } @@ -1243,7 +1368,7 @@ static void add_worker(struct vine_manager *q) /* Delete a single file on a remote worker except those with greater delete_upto_level cache level */ -static int delete_worker_file(struct vine_manager *q, struct vine_worker_info *w, const char *filename, vine_cache_level_t cache_level, vine_cache_level_t delete_upto_level) +int delete_worker_file(struct vine_manager *q, struct vine_worker_info *w, const char *filename, vine_cache_level_t cache_level, vine_cache_level_t delete_upto_level) { if (cache_level <= delete_upto_level) { process_replica_on_event(q, w, filename, VINE_FILE_REPLICA_STATE_TRANSITION_EVENT_UNLINK); @@ -3519,7 +3644,7 @@ static void vine_manager_consider_recovery_task(struct vine_manager *q, struct v case VINE_TASK_INITIAL: /* The recovery task has never been run, so submit it now. */ vine_submit(q, rt); - notice(D_VINE, "Submitted recovery task %d (%s) to re-create lost temporary file %s.", rt->task_id, rt->command_line, lost_file->cached_name); + debug(D_VINE, "Submitted recovery task %d (%s) to re-create lost temporary file %s.", rt->task_id, rt->command_line, lost_file->cached_name); break; case VINE_TASK_READY: case VINE_TASK_RUNNING: @@ -3533,7 +3658,7 @@ static void vine_manager_consider_recovery_task(struct vine_manager *q, struct v * here. */ vine_task_reset(rt); vine_submit(q, rt); - notice(D_VINE, "Submitted recovery task %d (%s) to re-create lost temporary file %s.", rt->task_id, rt->command_line, lost_file->cached_name); + debug(D_VINE, "Submitted recovery task %d (%s) to re-create lost temporary file %s.", rt->task_id, rt->command_line, lost_file->cached_name); break; } } @@ -4135,7 +4260,7 @@ struct vine_manager *vine_ssl_create(int port, const char *key, const char *cert q->worker_table = hash_table_create(0, 0); q->file_worker_table = hash_table_create(0, 0); - q->temp_files_to_replicate = hash_table_create(0, 0); + q->temp_files_to_replicate = priority_queue_create(0); q->worker_blocklist = hash_table_create(0, 0); q->file_table = hash_table_create(0, 0); @@ -4244,6 +4369,15 @@ struct vine_manager *vine_ssl_create(int port, const char *key, const char *cert q->enforce_worker_eviction_interval = 0; q->time_start_worker_eviction = 0; + q->return_recovery_tasks = 0; + q->num_submitted_recovery_tasks = 0; + q->balance_worker_disk_load = 0; + q->when_last_offloaded = 0; + q->peak_used_cache = 0; + q->shutting_down = 0; + vine_set_replica_placement_policy(q, VINE_REPLICA_PLACEMENT_POLICY_RANDOM); + + if ((envstring = getenv("VINE_BANDWIDTH"))) { q->bandwidth_limit = string_metric_parse(envstring); if (q->bandwidth_limit < 0) { @@ -4320,6 +4454,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"); @@ -4464,6 +4612,8 @@ void vine_delete(struct vine_manager *q) * disable the immediate recovery to avoid submitting recovery tasks for lost files */ q->immediate_recovery = 0; + q->shutting_down = 1; + vine_fair_write_workflow_info(q); release_all_workers(q); @@ -4487,8 +4637,7 @@ void vine_delete(struct vine_manager *q) hash_table_clear(q->file_worker_table, (void *)set_delete); hash_table_delete(q->file_worker_table); - hash_table_clear(q->temp_files_to_replicate, 0); - hash_table_delete(q->temp_files_to_replicate); + priority_queue_delete(q->temp_files_to_replicate); hash_table_clear(q->factory_table, (void *)vine_factory_info_delete); hash_table_delete(q->factory_table); @@ -4869,6 +5018,7 @@ int vine_submit(struct vine_manager *q, struct vine_task *t) * this distinction is important when many files are lost and the workflow is effectively rerun from scratch. */ if (t->type == VINE_TASK_TYPE_RECOVERY) { vine_task_set_priority(t, t->priority + priority_queue_get_top_priority(q->ready_tasks) + 1); + q->num_submitted_recovery_tasks++; } if (t->has_fixed_locations) { @@ -5232,7 +5382,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. @@ -5446,9 +5600,15 @@ static struct vine_task *vine_wait_internal(struct vine_manager *q, int timeout, } } + // Check if any worker is overloaded and rebalance the disk usage + if (q->balance_worker_disk_load && (timestamp_get() - q->when_last_offloaded > 5 * 1e6)) { + rebalance_worker_disk_usage(q); + q->when_last_offloaded = timestamp_get(); + } + // Check if any temp files need replication and start replicating BEGIN_ACCUM_TIME(q, time_internal); - result = consider_tempfile_replications(q); + result = vine_temp_start_replication(q); END_ACCUM_TIME(q, time_internal); if (result) { // recovered at least one temp file @@ -5980,6 +6140,9 @@ int vine_tune(struct vine_manager *q, const char *name, double value) } else if (!strcmp(name, "enforce-worker-eviction-interval")) { q->enforce_worker_eviction_interval = (timestamp_t)(MAX(0, (int)value) * ONE_SECOND); + } else if (!strcmp(name, "balance-worker-disk-load")) { + q->balance_worker_disk_load = !!((int)value); + } else { debug(D_NOTICE | D_VINE, "Warning: tuning parameter \"%s\" not recognized\n", name); return -1; @@ -6460,9 +6623,6 @@ void vine_prune_file(struct vine_manager *m, struct vine_file *f) set_free_values_array(workers_array); } } - - /* also remove from the replication table. */ - hash_table_remove(m->temp_files_to_replicate, f->cached_name); } /* diff --git a/taskvine/src/manager/vine_manager.h b/taskvine/src/manager/vine_manager.h index bcf2405616..8b4fb9826a 100644 --- a/taskvine/src/manager/vine_manager.h +++ b/taskvine/src/manager/vine_manager.h @@ -123,7 +123,7 @@ struct vine_manager { struct hash_table *file_table; /* Maps fileid -> struct vine_file.* */ struct hash_table *file_worker_table; /* Maps cachename -> struct set of workers with a replica of the file.* */ - struct hash_table *temp_files_to_replicate; /* Maps cachename -> NULL. Used as a set of temp files to be replicated */ + struct priority_queue *temp_files_to_replicate; /* Maps cachename -> NULL. Used as a set of temp files to be replicated */ /* Primary scheduling controls. */ @@ -232,6 +232,14 @@ struct vine_manager { double sandbox_grow_factor; /* When task disk sandboxes are exhausted, increase the allocation using their measured valued times this factor */ double disk_proportion_available_to_task; /* intentionally reduces disk allocation for tasks to reserve some space for cache growth. */ + int return_recovery_tasks; /* If true, recovery tasks are returned by vine_wait to the user. By default they are handled internally. */ + int num_submitted_recovery_tasks; + vine_replica_placement_policy_t replica_placement_policy; /* Mode for selecting best worker for placing a new replica of a temp file */ + int balance_worker_disk_load; /* If true, offload replicas from workers that are overloaded with temp files. */ + timestamp_t when_last_offloaded; + int64_t peak_used_cache; + int shutting_down; + /* todo: confirm datatype. int or int64 */ int max_task_stdout_storage; /* Maximum size of standard output from task. (If larger, send to a separate file.) */ int max_new_workers; /* Maximum number of workers to add in a single cycle before dealing with other matters. */ @@ -291,6 +299,15 @@ void vine_manager_remove_worker(struct vine_manager *q, struct vine_worker_info /* Check if the worker is able to transfer the necessary files for this task. */ int vine_manager_transfer_capacity_available(struct vine_manager *q, struct vine_worker_info *w, struct vine_task *t); +/* Delete a file from a worker. */ +int delete_worker_file(struct vine_manager *q, struct vine_worker_info *w, const char *filename, vine_cache_level_t cache_level, vine_cache_level_t delete_upto_level); + +/* Evict a random worker to simulate a worker failure. */ +int evict_random_worker(struct vine_manager *q); + +/* Get the available disk space in bytes for a worker. */ +int64_t get_worker_available_disk_bytes(struct vine_worker_info *w); + /* The expected format of files created by the resource monitor.*/ #define RESOURCE_MONITOR_TASK_LOCAL_NAME "vine-task-%d" #define RESOURCE_MONITOR_REMOTE_NAME "cctools-monitor" diff --git a/taskvine/src/manager/vine_task_graph.c b/taskvine/src/manager/vine_task_graph.c new file mode 100644 index 0000000000..0ae4e0f052 --- /dev/null +++ b/taskvine/src/manager/vine_task_graph.c @@ -0,0 +1,739 @@ +#include "vine_task_graph.h" +#include "taskvine.h" +#include "vine_manager.h" +#include "vine_worker_info.h" +#include "priority_queue.h" +#include +#include +#include +#include "debug.h" +#include "stringtools.h" +#include "xxmalloc.h" +#include "priority_queue.h" +#include +#include "hash_table.h" +#include +#include "itable.h" +#include "list.h" +#include "vine_task.h" +#include "timestamp.h" +#include "vine_file.h" +#include "set.h" +#include "vine_mount.h" +#include "progress_bar.h" +#include "assert.h" +#include "macros.h" +#include +#include + +static volatile sig_atomic_t interrupted = 0; + +/*************************************************************/ +/* Private Functions */ +/*************************************************************/ + +static void handle_sigint(int signal) +{ + interrupted = 1; +} + +static void submit_node_task(struct vine_task_graph *tg, struct vine_task_node *node) +{ + if (!tg || !node) { + return; + } + + int task_id = vine_task_node_submit(node); + itable_insert(tg->task_id_to_node, task_id, node); + + node->retry_attempts_left--; + if (node->retry_attempts_left < 0) { + debug(D_ERROR, "Aborting, node %s has exhausted all retry attempts", node->node_key); + vine_task_graph_delete(tg); + exit(1); + } + + return; +} + +static void submit_node_ready_children(struct vine_task_graph *tg, struct vine_task_node *node) +{ + if (!tg || !node) { + return; + } + + struct vine_task_node *child_node; + LIST_ITERATE(node->children, child_node) + { + /* Remove this parent from the child's pending set if it exists */ + if (child_node->pending_parents) { + /* Assert that this parent is indeed pending for the child */ + assert(set_lookup(child_node->pending_parents, node)); + set_remove(child_node->pending_parents, node); + } + + /* If no more parents are pending, submit the child */ + if (!child_node->pending_parents || set_size(child_node->pending_parents) == 0) { + submit_node_task(tg, child_node); + } + } + + return; +} + +static struct list *get_topological_order(struct vine_task_graph *tg) +{ + if (!tg) { + return NULL; + } + + int total_nodes = hash_table_size(tg->nodes); + struct list *topo_order = list_create(); + struct hash_table *in_degree_map = hash_table_create(0, 0); + struct priority_queue *pq = priority_queue_create(total_nodes); + + char *key; + struct vine_task_node *node; + HASH_TABLE_ITERATE(tg->nodes, key, node) + { + int deg = list_size(node->parents); + hash_table_insert(in_degree_map, key, (void *)(intptr_t)deg); + if (deg == 0) { + priority_queue_push(pq, node, compute_lex_priority(node->node_key)); + } + } + + while (priority_queue_size(pq) > 0) { + struct vine_task_node *current = priority_queue_pop(pq); + list_push_tail(topo_order, current); + + struct vine_task_node *child; + LIST_ITERATE(current->children, child) + { + intptr_t raw_deg = (intptr_t)hash_table_lookup(in_degree_map, child->node_key); + int deg = (int)raw_deg - 1; + + hash_table_remove(in_degree_map, child->node_key); + hash_table_insert(in_degree_map, child->node_key, (void *)(intptr_t)deg); + + if (deg == 0) { + priority_queue_push(pq, child, compute_lex_priority(child->node_key)); + } + } + } + + if (list_size(topo_order) != total_nodes) { + debug(D_ERROR, "Error: task graph contains cycles or is malformed.\n"); + debug(D_ERROR, "Expected %d nodes, but only sorted %d.\n", total_nodes, list_size(topo_order)); + + HASH_TABLE_ITERATE(tg->nodes, key, node) + { + intptr_t raw_deg = (intptr_t)hash_table_lookup(in_degree_map, key); + int deg = (int)raw_deg; + if (deg > 0) { + debug(D_ERROR, " Node %s has in-degree %d. Parents:\n", key, deg); + struct vine_task_node *p; + LIST_ITERATE(node->parents, p) + { + debug(D_ERROR, " -> %s\n", p->node_key); + } + } + } + + list_delete(topo_order); + exit(1); + } + + hash_table_delete(in_degree_map); + priority_queue_delete(pq); + return topo_order; +} + +static struct list *extract_weakly_connected_components(struct vine_task_graph *tg) +{ + if (!tg) { + return NULL; + } + + struct set *visited = set_create(0); + struct list *components = list_create(); + + char *node_key; + struct vine_task_node *node; + HASH_TABLE_ITERATE(tg->nodes, node_key, 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_task_node *curr = list_pop_head(queue); + + struct vine_task_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_task_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; +} + +static double compute_node_heavy_score(struct vine_task_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 struct vine_task_node *get_node_by_task(struct vine_task_graph *tg, struct vine_task *task) +{ + if (!tg || !task) { + return NULL; + } + + if (task->type == VINE_TASK_TYPE_STANDARD) { + return itable_lookup(tg->task_id_to_node, task->task_id); + } else if (task->type == VINE_TASK_TYPE_RECOVERY) { + /* note that recovery tasks are not mapped to any node but we still need the original node for pruning, + * so we look up the outfile of the task, then map it back to get the original node */ + struct vine_mount *mount; + LIST_ITERATE(task->output_mounts, mount) + { + if (mount->file->original_producer_task_id > 0) { + return itable_lookup(tg->task_id_to_node, mount->file->original_producer_task_id); + } + } + } + + debug(D_ERROR, "task %d has no original producer task id", task->task_id); + + return NULL; +} + +static void handle_checkpoint_worker_stagein(struct vine_task_graph *tg, struct vine_file *f) +{ + if (!tg || !f) { + return; + } + + struct vine_task_node *this_node = hash_table_lookup(tg->outfile_cachename_to_node, f->cached_name); + if (!this_node) { + return; + } + + vine_task_node_prune_ancestors(this_node); +} + +/*************************************************************/ +/* Public APIs */ +/*************************************************************/ + +void vine_task_graph_execute(struct vine_task_graph *tg) +{ + if (!tg) { + return; + } + + char *node_key; + struct vine_task_node *node; + + /* create mapping from task_id and outfile cached_name to node */ + HASH_TABLE_ITERATE(tg->nodes, node_key, node) + { + if (node->outfile) { + hash_table_insert(tg->outfile_cachename_to_node, node->outfile->cached_name, node); + } + } + + /* add the parents' outfiles as inputs to the task */ + struct list *topo_order = get_topological_order(tg); + LIST_ITERATE(topo_order, node) + { + struct vine_task_node *parent_node; + LIST_ITERATE(node->parents, parent_node) + { + if (parent_node->outfile) { + vine_task_add_input(node->task, parent_node->outfile, parent_node->outfile_remote_name, VINE_TRANSFER_ALWAYS); + } + } + } + + /* initialize pending_parents for all nodes */ + HASH_TABLE_ITERATE(tg->nodes, node_key, node) + { + struct vine_task_node *parent_node; + LIST_ITERATE(node->parents, parent_node) + { + if (node->pending_parents) { + /* Use parent_node->node_key to ensure pointer consistency */ + set_insert(node->pending_parents, parent_node); + } + } + } + + /* enable return recovery tasks */ + vine_enable_return_recovery_tasks(tg->manager); + + /* enqueue those without dependencies */ + HASH_TABLE_ITERATE(tg->nodes, node_key, node) + { + if (!node->pending_parents || set_size(node->pending_parents) == 0) { + submit_node_task(tg, node); + } + } + + /* calculate steps to inject failure */ + double next_failure_threshold = -1.0; + if (tg->failure_injection_step_percent > 0) { + next_failure_threshold = tg->failure_injection_step_percent / 100.0; + } + + struct ProgressBar *pbar = progress_bar_init("Executing Tasks"); + struct ProgressBarPart *regular_tasks_part = progress_bar_create_part("Regular", hash_table_size(tg->nodes)); + struct ProgressBarPart *recovery_tasks_part = progress_bar_create_part("Recovery", 0); + progress_bar_bind_part(pbar, regular_tasks_part); + progress_bar_bind_part(pbar, recovery_tasks_part); + + int wait_timeout = 2; + + while (regular_tasks_part->current < regular_tasks_part->total) { + if (interrupted) { + break; + } + + struct vine_task *task = vine_wait(tg->manager, wait_timeout); + progress_bar_set_part_total(pbar, recovery_tasks_part, tg->manager->num_submitted_recovery_tasks); + if (task) { + /* retrieve all possible tasks */ + wait_timeout = 0; + + /* get the original node by task id */ + struct vine_task_node *node = get_node_by_task(tg, 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); + } + + /* in case of failure, resubmit this task */ + if (node->task->result != VINE_RESULT_SUCCESS || node->task->exit_code != 0) { + debug(D_VINE | D_NOTICE, "Task %d failed with result %d and exit code %d, resubmitting...", task->task_id, node->task->result, node->task->exit_code); + vine_task_reset(node->task); + submit_node_task(tg, node); + continue; + } + + /* if the outfile is set to save on the sharedfs, stat to get the size of the file */ + switch (node->outfile_type) { + case VINE_NODE_OUTFILE_TYPE_SHARED_FILE_SYSTEM: + struct stat info; + int result = stat(node->outfile_remote_name, &info); + if (result < 0) { + debug(D_VINE | D_NOTICE, "Task %d succeeded but output file %s does not exist on the shared file system", task->task_id, node->outfile_remote_name); + vine_task_reset(node->task); + submit_node_task(tg, node); + continue; + } + node->outfile_size_bytes = info.st_size; + break; + case VINE_NODE_OUTFILE_TYPE_LOCAL: + case VINE_NODE_OUTFILE_TYPE_TEMP: + node->outfile_size_bytes = node->outfile->size; + break; + } + debug(D_VINE, "Node %s completed with outfile %s size: %ld bytes", node->node_key, node->outfile_remote_name, node->outfile_size_bytes); + + /* mark the node as completed */ + node->completed = 1; + + /* prune nodes on task completion */ + vine_task_node_prune_ancestors(node); + + /* skip recovery tasks */ + if (task->type == VINE_TASK_TYPE_RECOVERY) { + progress_bar_update_part(pbar, recovery_tasks_part, 1); + continue; + } + + /* set the start time to the submit time of the first regular task */ + if (regular_tasks_part->current == 0) { + progress_bar_set_start_time(pbar, task->time_when_commit_start); + } + + /* update critical time */ + vine_task_node_update_critical_time(node, task->time_workers_execute_last); + + /* mark this regular task as completed */ + progress_bar_update_part(pbar, regular_tasks_part, 1); + + /* inject failure */ + if (tg->failure_injection_step_percent > 0) { + double progress = (double)regular_tasks_part->current / (double)regular_tasks_part->total; + if (progress >= next_failure_threshold && evict_random_worker(tg->manager)) { + debug(D_VINE, "evicted a worker at %.2f%% (threshold %.2f%%)", progress * 100, next_failure_threshold * 100); + next_failure_threshold += tg->failure_injection_step_percent / 100.0; + } + } + + /* enqueue the output file for replication or checkpointing */ + switch (node->outfile_type) { + case VINE_NODE_OUTFILE_TYPE_TEMP: + vine_task_node_replicate_outfile(node); + break; + case VINE_NODE_OUTFILE_TYPE_LOCAL: + case VINE_NODE_OUTFILE_TYPE_SHARED_FILE_SYSTEM: + break; + } + + /* submit children nodes with dependencies all resolved */ + submit_node_ready_children(tg, node); + } else { + wait_timeout = 2; + progress_bar_update_part(pbar, recovery_tasks_part, 0); + } + } + + progress_bar_finish(pbar); + progress_bar_delete(pbar); + + double total_time_spent_on_unlink_local_files = 0; + double total_time_spent_on_prune_ancestors_of_temp_node = 0; + double total_time_spent_on_prune_ancestors_of_persisted_node = 0; + HASH_TABLE_ITERATE(tg->nodes, node_key, node) + { + total_time_spent_on_unlink_local_files += node->time_spent_on_unlink_local_files; + total_time_spent_on_prune_ancestors_of_temp_node += node->time_spent_on_prune_ancestors_of_temp_node; + total_time_spent_on_prune_ancestors_of_persisted_node += node->time_spent_on_prune_ancestors_of_persisted_node; + } + total_time_spent_on_unlink_local_files /= 1e6; + total_time_spent_on_prune_ancestors_of_temp_node /= 1e6; + total_time_spent_on_prune_ancestors_of_persisted_node /= 1e6; + + debug(D_VINE, "total time spent on prune ancestors of temp node: %.6f seconds\n", total_time_spent_on_prune_ancestors_of_temp_node); + debug(D_VINE, "total time spent on prune ancestors of persisted node: %.6f seconds\n", total_time_spent_on_prune_ancestors_of_persisted_node); + debug(D_VINE, "total time spent on unlink local files: %.6f seconds\n", total_time_spent_on_unlink_local_files); + + return; +} + +void vine_task_graph_finalize_metrics(struct vine_task_graph *tg) +{ + if (!tg) { + return; + } + + /* get nodes in topological order */ + struct list *topo_order = get_topological_order(tg); + if (!topo_order) { + return; + } + + char *node_key; + struct vine_task_node *node; + struct vine_task_node *parent_node; + struct vine_task_node *child_node; + + /* compute the depth of the node */ + 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; + } + } + } + + /* compute the height of the node */ + 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; + } + } + } + + /* compute the upstream and downstream counts for each node */ + struct hash_table *upstream_map = hash_table_create(0, 0); + struct hash_table *downstream_map = hash_table_create(0, 0); + HASH_TABLE_ITERATE(tg->nodes, node_key, node) + { + struct set *upstream = set_create(0); + struct set *downstream = set_create(0); + hash_table_insert(upstream_map, node_key, upstream); + hash_table_insert(downstream_map, node_key, downstream); + } + LIST_ITERATE(topo_order, node) + { + struct set *upstream = hash_table_lookup(upstream_map, node->node_key); + LIST_ITERATE(node->parents, parent_node) + { + struct set *parent_upstream = hash_table_lookup(upstream_map, parent_node->node_key); + set_union(upstream, parent_upstream); + set_insert(upstream, parent_node); + } + } + LIST_ITERATE_REVERSE(topo_order, node) + { + struct set *downstream = hash_table_lookup(downstream_map, node->node_key); + LIST_ITERATE(node->children, child_node) + { + struct set *child_downstream = hash_table_lookup(downstream_map, child_node->node_key); + set_union(downstream, child_downstream); + set_insert(downstream, child_node); + } + } + LIST_ITERATE(topo_order, node) + { + node->upstream_subgraph_size = set_size(hash_table_lookup(upstream_map, node->node_key)); + node->downstream_subgraph_size = set_size(hash_table_lookup(downstream_map, node->node_key)); + node->fan_in = list_size(node->parents); + node->fan_out = list_size(node->children); + set_delete(hash_table_lookup(upstream_map, node->node_key)); + set_delete(hash_table_lookup(downstream_map, node->node_key)); + } + hash_table_delete(upstream_map); + hash_table_delete(downstream_map); + + /* compute the heavy score */ + LIST_ITERATE(topo_order, node) + { + node->heavy_score = compute_node_heavy_score(node); + } + + /* extract weakly connected components */ + struct list *weakly_connected_components = extract_weakly_connected_components(tg); + 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)); + component_index++; + } + list_delete(weakly_connected_components); + + list_delete(topo_order); + + return; +} + +struct vine_task_graph *vine_task_graph_create(struct vine_manager *q) +{ + struct vine_task_graph *tg = xxmalloc(sizeof(struct vine_task_graph)); + tg->nodes = hash_table_create(0, 0); + tg->task_id_to_node = itable_create(0); + tg->outfile_cachename_to_node = hash_table_create(0, 0); + + tg->library_name = xxstrdup("vine_task_graph_library"); + tg->function_name = xxstrdup("compute_single_key"); + tg->manager = q; + + tg->failure_injection_step_percent = -1.0; + + /* enable debug system for C code since it uses a separate debug system instance + * from the Python bindings. Use the same function that the manager uses. */ + char *debug_tmp = string_format("%s/vine-logs/debug", tg->manager->runtime_directory); + vine_enable_debug_log(debug_tmp); + free(debug_tmp); + + signal(SIGINT, handle_sigint); + + return tg; +} + +void vine_task_graph_set_failure_injection_step_percent(struct vine_task_graph *tg, double percent) +{ + if (!tg) { + return; + } + + if (percent <= 0 || percent > 100) { + return; + } + + debug(D_VINE, "setting failure injection step percent to %lf", percent); + tg->failure_injection_step_percent = percent; +} + +struct vine_task_node *vine_task_graph_create_node( + struct vine_task_graph *tg, + const char *node_key, + const char *staging_dir, + int prune_depth, + vine_task_node_priority_mode_t priority_mode) +{ + if (!tg || !node_key) { + return NULL; + } + + struct vine_task_node *node = hash_table_lookup(tg->nodes, node_key); + if (!node) { + node = vine_task_node_create(tg->manager, + node_key, + tg->library_name, + tg->function_name, + staging_dir, + prune_depth, + priority_mode); + hash_table_insert(tg->nodes, node_key, node); + } + + return node; +} + +void vine_task_graph_add_dependency(struct vine_task_graph *tg, const char *parent_key, const char *child_key) +{ + if (!tg || !parent_key || !child_key) { + return; + } + + struct vine_task_node *parent_node = hash_table_lookup(tg->nodes, parent_key); + struct vine_task_node *child_node = hash_table_lookup(tg->nodes, child_key); + if (!parent_node) { + debug(D_ERROR, "parent node %s not found", parent_key); + exit(1); + } + if (!child_node) { + debug(D_ERROR, "child node %s not found", child_key); + exit(1); + } + + list_push_tail(child_node->parents, parent_node); + list_push_tail(parent_node->children, child_node); + debug(D_VINE, "added dependency: %s -> %s", parent_key, child_key); +} + +const char *vine_task_graph_get_library_name(const struct vine_task_graph *tg) +{ + if (!tg) { + return NULL; + } + return tg->library_name; +} + +const char *vine_task_graph_get_function_name(const struct vine_task_graph *tg) +{ + if (!tg) { + return NULL; + } + return tg->function_name; +} + +double vine_task_graph_get_node_heavy_score(const struct vine_task_graph *tg, const char *node_key) +{ + if (!tg) { + return -1; + } + + struct vine_task_node *node = hash_table_lookup(tg->nodes, node_key); + if (!node) { + return -1; + } + + return node->heavy_score; +} + +const char *vine_task_graph_get_node_local_outfile_source(const struct vine_task_graph *tg, const char *node_key) +{ + if (!tg || !node_key) { + return NULL; + } + + struct vine_task_node *node = hash_table_lookup(tg->nodes, node_key); + if (!node) { + debug(D_ERROR, "node %s not found", node_key); + exit(1); + } + + if (node->outfile_type != VINE_NODE_OUTFILE_TYPE_LOCAL) { + debug(D_ERROR, "node %s is not a local output file", node_key); + exit(1); + } + + return node->outfile->source; +} + +void vine_task_graph_set_node_outfile(struct vine_task_graph *tg, const char *node_key, vine_task_node_outfile_type_t outfile_type, const char *outfile_remote_name) +{ + if (!tg || !node_key || !outfile_remote_name) { + return; + } + + struct vine_task_node *node = hash_table_lookup(tg->nodes, node_key); + if (!node) { + return; + } + + vine_task_node_set_outfile(node, outfile_type, outfile_remote_name); + + return; +} + +void vine_task_graph_delete(struct vine_task_graph *tg) +{ + if (!tg) { + return; + } + + char *node_key; + struct vine_task_node *node; + HASH_TABLE_ITERATE(tg->nodes, node_key, node) + { + if (node->infile) { + vine_prune_file(tg->manager, node->infile); + hash_table_remove(tg->manager->file_table, node->infile->cached_name); + } + if (node->outfile) { + vine_prune_file(tg->manager, node->outfile); + hash_table_remove(tg->outfile_cachename_to_node, node->outfile->cached_name); + hash_table_remove(tg->manager->file_table, node->outfile->cached_name); + } + vine_task_node_delete(node); + } + + vine_delete(tg->manager); + + free(tg->library_name); + free(tg->function_name); + + hash_table_delete(tg->nodes); + itable_delete(tg->task_id_to_node); + hash_table_delete(tg->outfile_cachename_to_node); + free(tg); +} diff --git a/taskvine/src/manager/vine_task_graph.h b/taskvine/src/manager/vine_task_graph.h new file mode 100644 index 0000000000..1692d1fc9e --- /dev/null +++ b/taskvine/src/manager/vine_task_graph.h @@ -0,0 +1,24 @@ +#ifndef VINE_TASK_GRAPH_H +#define VINE_TASK_GRAPH_H + +#include "vine_task.h" +#include "hash_table.h" +#include "list.h" +#include "vine_manager.h" +#include "set.h" +#include "vine_task_node.h" + +struct vine_task_graph { + struct vine_manager *manager; + struct hash_table *nodes; + struct itable *task_id_to_node; + struct hash_table *outfile_cachename_to_node; + + char *library_name; + char *function_name; + + double failure_injection_step_percent; // 0 - 100, the percentage of steps to inject failure +}; + + +#endif // VINE_TASK_GRAPH_H diff --git a/taskvine/src/manager/vine_task_node.c b/taskvine/src/manager/vine_task_node.c new file mode 100644 index 0000000000..ae99004ba8 --- /dev/null +++ b/taskvine/src/manager/vine_task_node.c @@ -0,0 +1,611 @@ +#include "vine_task_node.h" +#include "vine_manager.h" +#include "vine_task.h" +#include "vine_file.h" +#include "vine_task_graph.h" +#include "xxmalloc.h" +#include "stringtools.h" +#include "taskvine.h" +#include "timestamp.h" +#include "set.h" +#include "hash_table.h" +#include "unistd.h" +#include "debug.h" +#include "assert.h" +#include "vine_worker_info.h" +#include "vine_temp.h" +#include "random.h" + +double compute_lex_priority(const char *key) +{ + double score = 0.0; + double factor = 1.0; + for (int i = 0; i < 8 && key[i] != '\0'; i++) { + score += key[i] * factor; + factor *= 0.01; + } + return -score; +} + +/* + * Create a new node in the graph. + * The caller must construct the graph nodes in a topological order. + */ + +struct vine_task_node *vine_task_node_create( + struct vine_manager *manager, + const char *node_key, + const char *library_name, + const char *function_name, + const char *staging_dir, + int prune_depth, + vine_task_node_priority_mode_t priority_mode) +{ + if (!manager || !node_key || !library_name || !function_name || !staging_dir) { + return NULL; + } + + struct vine_task_node *node = xxmalloc(sizeof(struct vine_task_node)); + + node->manager = manager; + node->node_key = xxstrdup(node_key); + node->staging_dir = xxstrdup(staging_dir); + node->priority_mode = priority_mode; + node->prune_status = PRUNE_STATUS_NOT_PRUNED; + node->parents = list_create(); + node->children = list_create(); + node->pending_parents = set_create(0); + node->completed = 0; + node->prune_depth = prune_depth; + node->retry_attempts_left = 1; + + node->outfile_size_bytes = 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->time_spent_on_unlink_local_files = 0; + node->time_spent_on_prune_ancestors_of_temp_node = 0; + node->time_spent_on_prune_ancestors_of_persisted_node = 0; + + node->critical_time = -1; + + debug(D_VINE, "node info: key=%s, staging_dir=%s, priority_mode=%d, prune_depth=%d", node->node_key, node->staging_dir, node->priority_mode, node->prune_depth); + + /* create the task */ + node->task = vine_task_create(function_name); + vine_task_set_library_required(node->task, library_name); + vine_task_addref(node->task); + + /* add the infile as the key of the graph to compute */ + char *infile_content = string_format("%s", node->node_key); + node->infile = vine_declare_buffer(node->manager, infile_content, strlen(infile_content), VINE_CACHE_LEVEL_TASK, VINE_UNLINK_WHEN_DONE); + free(infile_content); + vine_task_add_input(node->task, node->infile, "infile", VINE_TRANSFER_ALWAYS); + + return node; +} + +void vine_task_node_set_outfile(struct vine_task_node *node, vine_task_node_outfile_type_t outfile_type, const char *outfile_remote_name) +{ + if (!node) { + return; + } + + assert(outfile_remote_name != NULL); + + node->outfile_type = outfile_type; + node->outfile_remote_name = xxstrdup(outfile_remote_name); + + /* create the output file */ + switch (node->outfile_type) { + case VINE_NODE_OUTFILE_TYPE_LOCAL: + char *persistent_path = string_format("%s/outputs/%s", node->staging_dir, node->outfile_remote_name); + node->outfile = vine_declare_file(node->manager, persistent_path, VINE_CACHE_LEVEL_WORKFLOW, 0); + free(persistent_path); + debug(D_VINE, "node %s: outfile type = VINE_NODE_OUTFILE_TYPE_LOCAL, outfile = %s", node->node_key, node->outfile->cached_name); + break; + case VINE_NODE_OUTFILE_TYPE_TEMP: + node->outfile = vine_declare_temp(node->manager); + debug(D_VINE, "node %s: outfile type = VINE_NODE_OUTFILE_TYPE_TEMP, outfile = %s", node->node_key, node->outfile->cached_name); + break; + case VINE_NODE_OUTFILE_TYPE_SHARED_FILE_SYSTEM: + /* no explicit output file declaration needed */ + node->outfile = NULL; + debug(D_VINE, "node %s: outfile type = VINE_NODE_OUTFILE_TYPE_SHARED_FILE_SYSTEM", node->node_key); + break; + } + if (node->outfile) { + vine_task_add_output(node->task, node->outfile, node->outfile_remote_name, VINE_TRANSFER_ALWAYS); + } + + return; +} + +static int _node_outfile_is_persisted(struct vine_task_node *node) +{ + if (!node) { + return 0; + } + + /* if the node is not completed then the outfile is definitely not persisted */ + if (!node->completed) { + return 0; + } + + /* if the outfile pointer is NULL, it means the node is producing a shared file system file */ + if (!node->outfile) { + return 1; + } + + switch (node->outfile->type) { + case VINE_FILE: + return 1; + case VINE_TEMP: + /* check if this file is in the any of the checkpoint workers */ + return file_has_been_checkpointed(node->manager, node->outfile); + default: + /* not supported at the moment */ + debug(D_ERROR, "unsupported outfile type: %d", node->outfile->type); + return 0; + } +} + +double vine_task_node_calculate_priority(struct vine_task_node *node) +{ + if (!node) { + return 0; + } + + double priority = 0; + timestamp_t current_time = timestamp_get(); + + struct vine_task_node *parent_node; + + switch (node->priority_mode) { + case VINE_TASK_PRIORITY_MODE_RANDOM: + priority = random_double(); + break; + case VINE_TASK_PRIORITY_MODE_DEPTH_FIRST: + priority = (double)node->depth; + break; + case VINE_TASK_PRIORITY_MODE_BREADTH_FIRST: + priority = -(double)node->depth; + break; + case VINE_TASK_PRIORITY_MODE_FIFO: + priority = -(double)current_time; + break; + case VINE_TASK_PRIORITY_MODE_LIFO: + priority = (double)current_time; + break; + case VINE_TASK_PRIORITY_MODE_LARGEST_INPUT_FIRST: + LIST_ITERATE(node->parents, parent_node) + { + if (!parent_node->outfile) { + continue; + } + priority += (double)vine_file_size(parent_node->outfile); + } + break; + case VINE_TASK_PRIORITY_MODE_LARGEST_STORAGE_FOOTPRINT_FIRST: + LIST_ITERATE(node->parents, parent_node) + { + if (!parent_node->outfile) { + continue; + } + timestamp_t parent_task_completion_time = parent_node->task->time_workers_execute_last; + priority += (double)vine_file_size(parent_node->outfile) * (double)parent_task_completion_time; + } + break; + } + + return priority; +} + +void vine_task_node_update_critical_time(struct vine_task_node *node, timestamp_t execution_time) +{ + timestamp_t max_parent_critical_time = 0; + struct vine_task_node *parent_node; + LIST_ITERATE(node->parents, parent_node) + { + if (parent_node->critical_time > max_parent_critical_time) { + max_parent_critical_time = parent_node->critical_time; + } + } + node->critical_time = max_parent_critical_time + execution_time; +} + +/* delete all of the replicas present at remote workers. */ +int _prune_outfile_from_regular_workers(struct vine_task_node *node) +{ + if (!node || !node->outfile) { + return 0; + } + + struct set *source_workers = hash_table_lookup(node->manager->file_worker_table, node->outfile->cached_name); + if (!source_workers) { + return 0; + } + + int pruned_replica_count = 0; + + struct vine_worker_info *source_worker; + SET_ITERATE(source_workers, source_worker) + { + if (is_checkpoint_worker(node->manager, source_worker)) { + continue; + } + delete_worker_file(node->manager, source_worker, node->outfile->cached_name, 0, 0); + pruned_replica_count++; + } + + return pruned_replica_count; +} + +/* the dfs helper function for finding parents in a specific depth */ +static void _find_parents_dfs(struct vine_task_node *node, int remaining_depth, struct list *result, struct set *visited) +{ + if (!node || set_lookup(visited, node)) { + return; + } + set_insert(visited, node); + if (remaining_depth == 0) { + list_push_tail(result, node); + return; + } + struct vine_task_node *parent_node; + LIST_ITERATE(node->parents, parent_node) + { + _find_parents_dfs(parent_node, remaining_depth - 1, result, visited); + } +} + +/* find all parents in a specific depth of the node */ +static struct list *_find_parents_in_depth(struct vine_task_node *node, int depth) +{ + if (!node || depth < 0) { + return NULL; + } + struct list *result = list_create(); + struct set *visited = set_create(0); + _find_parents_dfs(node, depth, result, visited); + set_delete(visited); + + return result; +} + +int _prune_ancestors_of_temp_node(struct vine_task_node *node) +{ + if (!node || !node->outfile || node->prune_depth <= 0) { + return 0; + } + + timestamp_t start_time = timestamp_get(); + + int pruned_replica_count = 0; + + struct list *parents = _find_parents_in_depth(node, node->prune_depth); + + struct vine_task_node *parent_node; + LIST_ITERATE(parents, parent_node) + { + /* skip if the parent produces a shared file system file */ + if (!parent_node->outfile) { + continue; + } + /* skip if the parent produces a non-temp file */ + if (parent_node->outfile->type != VINE_TEMP) { + continue; + } + + /* a file is prunable if its outfile is no longer needed by any child node: + * 1. it has no pending dependents + * 2. all completed dependents have also completed their corresponding recovery tasks, if any */ + int all_children_completed = 1; + struct vine_task_node *child_node; + LIST_ITERATE(parent_node->children, child_node) + { + /* break early if the child node is not completed */ + if (!child_node->completed) { + all_children_completed = 0; + break; + } + /* if the task produces a temp file and the recovery task is running, the parent is not prunable */ + if (child_node->outfile && child_node->outfile->type == VINE_TEMP) { + struct vine_task *child_node_recovery_task = child_node->outfile->recovery_task; + if (child_node_recovery_task && (child_node_recovery_task->state != VINE_TASK_INITIAL && child_node_recovery_task->state != VINE_TASK_DONE)) { + all_children_completed = 0; + break; + } + } + } + if (!all_children_completed) { + continue; + } + + pruned_replica_count += _prune_outfile_from_regular_workers(parent_node); + /* this parent is pruned because a successor that produces a temp file is completed, it is unsafe because the + * manager may submit a recovery task to bring it back in case of worker failures. */ + parent_node->prune_status = PRUNE_STATUS_UNSAFE; + } + + list_delete(parents); + + node->time_spent_on_prune_ancestors_of_temp_node += timestamp_get() - start_time; + + return pruned_replica_count; +} + +static struct set *_find_safe_ancestors(struct vine_task_node *start_node) +{ + if (!start_node) { + return NULL; + } + + struct set *visited_nodes = set_create(0); + struct list *bfs_nodes = list_create(); + + list_push_tail(bfs_nodes, start_node); + set_insert(visited_nodes, start_node); + + while (list_size(bfs_nodes) > 0) { + struct vine_task_node *current = list_pop_head(bfs_nodes); + + struct vine_task_node *parent_node; + LIST_ITERATE(current->parents, parent_node) + { + if (set_lookup(visited_nodes, parent_node)) { + continue; + } + + /* shortcut if this parent was already safely pruned */ + if (parent_node->prune_status == PRUNE_STATUS_SAFE) { + continue; + } + + /* check if all children are safe */ + int all_children_safe = 1; + struct vine_task_node *child_node; + LIST_ITERATE(parent_node->children, child_node) + { + /* shortcut if this child is part of the recovery subgraph */ + if (set_lookup(visited_nodes, child_node)) { + continue; + } + /* shortcut if this outside child is not persisted */ + if (!_node_outfile_is_persisted(child_node)) { + all_children_safe = 0; + break; + } + /* shortcut if this outside child is unsafely pruned */ + if (child_node->prune_status == PRUNE_STATUS_UNSAFE) { + all_children_safe = 0; + break; + } + } + + if (!all_children_safe) { + continue; + } + + set_insert(visited_nodes, parent_node); + list_push_tail(bfs_nodes, parent_node); + } + } + + list_delete(bfs_nodes); + set_remove(visited_nodes, start_node); + + return visited_nodes; +} + +int _prune_ancestors_of_persisted_node(struct vine_task_node *node) +{ + if (!node) { + return 0; + } + + timestamp_t start_time = timestamp_get(); + + int pruned_replica_count = 0; + + /* find all safe ancestors */ + struct set *safe_ancestors = _find_safe_ancestors(node); + if (!safe_ancestors) { + return 0; + } + + /* prune all safe ancestors */ + struct vine_task_node *ancestor_node; + SET_ITERATE(safe_ancestors, ancestor_node) + { + /* unlink the shared file system file */ + if (!ancestor_node->outfile) { + timestamp_t start_time = timestamp_get(); + unlink(ancestor_node->outfile_remote_name); + node->time_spent_on_unlink_local_files += timestamp_get() - start_time; + debug(D_VINE, "unlinked %s size: %ld bytes, time: %ld", ancestor_node->outfile_remote_name, ancestor_node->outfile_size_bytes, node->time_spent_on_unlink_local_files); + } else { + switch (ancestor_node->outfile->type) { + case VINE_FILE: + /* do not prune the staging dir file */ + break; + case VINE_TEMP: + /* prune the temp file */ + vine_prune_file(node->manager, ancestor_node->outfile); + break; + default: + debug(D_ERROR, "unsupported outfile type: %d", ancestor_node->outfile->type); + break; + } + } + ancestor_node->prune_status = PRUNE_STATUS_SAFE; + pruned_replica_count++; + } + + set_delete(safe_ancestors); + + node->time_spent_on_prune_ancestors_of_persisted_node += timestamp_get() - start_time; + + return pruned_replica_count; +} + +int vine_task_node_submit(struct vine_task_node *node) +{ + if (!node) { + return -1; + } + + double priority = vine_task_node_calculate_priority(node); + vine_task_set_priority(node->task, priority); + + debug(D_VINE, "node %s: priority_mode=%d, depth=%d, calculated_priority=%.6f", node->node_key, node->priority_mode, node->depth, priority); + + return vine_submit(node->manager, node->task); +} + +void vine_task_node_print_info(struct vine_task_node *node) +{ + if (!node) { + return; + } + + debug(D_VINE, "node info %s task_id: %d", node->node_key, node->task->task_id); + debug(D_VINE, "node info %s depth: %d", node->node_key, node->depth); + debug(D_VINE, "node info %s outfile remote name: %s", node->node_key, node->outfile_remote_name); + + if (node->outfile) { + switch (node->outfile->type) { + case VINE_FILE: + debug(D_VINE, "node info %s outfile type: VINE_FILE, cached name: %s", node->node_key, node->outfile->cached_name); + break; + case VINE_TEMP: + debug(D_VINE, "node info %s outfile type: VINE_TEMP, cached name: %s", node->node_key, node->outfile->cached_name); + break; + default: + debug(D_ERROR, "unsupported outfile type: %d", node->outfile->type); + break; + } + } + + char *parent_keys = NULL; + struct vine_task_node *parent_node; + LIST_ITERATE(node->parents, parent_node) + { + if (!parent_keys) { + parent_keys = string_format("%s", parent_node->node_key); + } else { + char *tmp = string_format("%s, %s", parent_keys, parent_node->node_key); + free(parent_keys); + parent_keys = tmp; + } + } + debug(D_VINE, "node info %s parents: %s", node->node_key, parent_keys); + free(parent_keys); + + char *child_keys = NULL; + struct vine_task_node *child_node; + LIST_ITERATE(node->children, child_node) + { + if (!child_keys) { + child_keys = string_format("%s", child_node->node_key); + } else { + char *tmp = string_format("%s, %s", child_keys, child_node->node_key); + free(child_keys); + child_keys = tmp; + } + } + debug(D_VINE, "node info %s children: %s", node->node_key, child_keys); + free(child_keys); + + return; +} + +void vine_task_node_prune_ancestors(struct vine_task_node *node) +{ + if (!node) { + return; + } + + timestamp_t start_time = timestamp_get(); + + int pruned_replica_count = 0; + + if (_node_outfile_is_persisted(node)) { + pruned_replica_count = _prune_ancestors_of_persisted_node(node); + } else { + pruned_replica_count = _prune_ancestors_of_temp_node(node); + } + + timestamp_t elapsed_time = timestamp_get() - start_time; + + debug(D_VINE, "pruned %d ancestors of node %s in %.6f seconds", pruned_replica_count, node->node_key, elapsed_time / 1000000.0); +} + +void vine_task_node_replicate_outfile(struct vine_task_node *node) +{ + if (!node || !node->outfile) { + return; + } + + vine_temp_replicate_file_later(node->manager, node->outfile); +} + +void vine_task_node_checkpoint_outfile(struct vine_task_node *node) +{ + if (!node || !node->outfile) { + return; + } + + vine_temp_checkpoint_file_later(node->manager, node->outfile); +} + +int vine_task_node_set_outfile_size_bytes(struct vine_task_node *node, size_t outfile_size_bytes) +{ + if (!node || !node->outfile) { + return -1; + } + + node->outfile_size_bytes = outfile_size_bytes; + + return 0; +} + +/* delete the node and all of its associated resources. */ +void vine_task_node_delete(struct vine_task_node *node) +{ + if (!node) { + return; + } + + if (node->node_key) { + free(node->node_key); + } + if (node->outfile_remote_name) { + free(node->outfile_remote_name); + } + + vine_task_delete(node->task); + node->task = NULL; + + if (node->infile) { + vine_file_delete(node->infile); + node->infile = NULL; + } + if (node->outfile) { + vine_file_delete(node->outfile); + node->outfile = NULL; + } + + list_delete(node->parents); + list_delete(node->children); + + if (node->pending_parents) { + set_delete(node->pending_parents); + } + free(node); +} \ No newline at end of file diff --git a/taskvine/src/manager/vine_task_node.h b/taskvine/src/manager/vine_task_node.h new file mode 100644 index 0000000000..13486e12c0 --- /dev/null +++ b/taskvine/src/manager/vine_task_node.h @@ -0,0 +1,76 @@ +#ifndef VINE_TASK_NODE_H +#define VINE_TASK_NODE_H + +#include "vine_task.h" +#include "hash_table.h" +#include "list.h" +#include "vine_manager.h" +#include "set.h" +#include "taskvine.h" + +typedef enum { + PRUNE_STATUS_NOT_PRUNED = 0, + PRUNE_STATUS_SAFE, + PRUNE_STATUS_UNSAFE +} prune_status_t; + +struct vine_task_node { + char *node_key; + + struct vine_manager *manager; + struct vine_task *task; + struct vine_file *infile; + struct vine_file *outfile; + char *outfile_remote_name; + char *staging_dir; + size_t outfile_size_bytes; + + struct list *parents; + struct list *children; + struct set *pending_parents; + + int retry_attempts_left; + int completed; + int prune_depth; + + int depth; + int height; + int upstream_subgraph_size; + int downstream_subgraph_size; + int fan_in; + int fan_out; + double heavy_score; + + timestamp_t critical_time; + timestamp_t time_spent_on_unlink_local_files; + timestamp_t time_spent_on_prune_ancestors_of_temp_node; + timestamp_t time_spent_on_prune_ancestors_of_persisted_node; + + vine_task_node_priority_mode_t priority_mode; + vine_task_node_outfile_type_t outfile_type; + prune_status_t prune_status; +}; + +struct vine_task_node *vine_task_node_create( + struct vine_manager *manager, + const char *node_key, + const char *library_name, + const char *function_name, + const char *staging_dir, + int prune_depth, + vine_task_node_priority_mode_t priority_mode +); + +void vine_task_node_delete(struct vine_task_node *node); +double compute_lex_priority(const char *key); +void vine_task_node_prune_ancestors(struct vine_task_node *node); +double vine_task_node_calculate_priority(struct vine_task_node *node); +void vine_task_node_print_info(struct vine_task_node *node); +void vine_task_node_update_critical_time(struct vine_task_node *node, timestamp_t execution_time); +void vine_task_node_replicate_outfile(struct vine_task_node *node); +int vine_task_node_submit(struct vine_task_node *node); +void vine_task_node_set_outfile(struct vine_task_node *node, vine_task_node_outfile_type_t outfile_type, const char *outfile_remote_name); +int vine_task_node_set_outfile_size_bytes(struct vine_task_node *node, size_t outfile_size_bytes); +void vine_task_node_checkpoint_outfile(struct vine_task_node *node); + +#endif \ No newline at end of file diff --git a/taskvine/src/manager/vine_temp.c b/taskvine/src/manager/vine_temp.c new file mode 100644 index 0000000000..914de9a893 --- /dev/null +++ b/taskvine/src/manager/vine_temp.c @@ -0,0 +1,288 @@ +#include "vine_temp.h" +#include "priority_queue.h" +#include "vine_file.h" +#include "vine_worker_info.h" +#include "vine_file_replica_table.h" +#include "macros.h" +#include "stringtools.h" +#include "vine_manager.h" +#include "debug.h" +#include "random.h" +#include "vine_manager_put.h" +#include "xxmalloc.h" + +/*************************************************************/ +/* Private Functions */ +/*************************************************************/ + +int is_checkpoint_worker(struct vine_manager *q, struct vine_worker_info *w) +{ + if (!q || !w || !w->features) { + return 0; + } + + return hash_table_lookup(w->features, "checkpoint-worker") != NULL; +} + +static struct vine_worker_info *get_best_source_worker(struct vine_manager *q, struct vine_file *f) +{ + if (!q || !f || f->type != VINE_TEMP) { + return NULL; + } + + struct set *sources = hash_table_lookup(q->file_worker_table, f->cached_name); + if (!sources) { + return NULL; + } + + struct priority_queue *valid_sources_queue = priority_queue_create(0); + struct vine_worker_info *w = NULL; + SET_ITERATE(sources, w) + { + /* skip if transfer port is not active or in draining mode */ + if (!w->transfer_port_active || w->draining) { + continue; + } + /* skip if incoming transfer counter is too high */ + if (w->outgoing_xfer_counter >= q->worker_source_max_transfers) { + continue; + } + /* skip if the worker does not have this file */ + struct vine_file_replica *replica = vine_file_replica_table_lookup(w, f->cached_name); + if (!replica) { + continue; + } + /* skip if the file is not ready */ + if (replica->state != VINE_FILE_REPLICA_STATE_READY) { + continue; + } + /* those with less outgoing_xfer_counter are preferred */ + priority_queue_push(valid_sources_queue, w, -w->outgoing_xfer_counter); + } + + struct vine_worker_info *best_source = priority_queue_pop(valid_sources_queue); + priority_queue_delete(valid_sources_queue); + + return best_source; +} + +static struct vine_worker_info *get_best_dest_worker(struct vine_manager *q, struct vine_file *f) +{ + if (!q || !f || f->type != VINE_TEMP) { + return NULL; + } + + struct priority_queue *valid_destinations = priority_queue_create(0); + + char *key; + struct vine_worker_info *w; + HASH_TABLE_ITERATE(q->worker_table, key, w) + { + /* skip if transfer port is not active or in draining mode */ + if (!w->transfer_port_active || w->draining) { + continue; + } + /* skip if the incoming transfer counter is too high */ + if (w->incoming_xfer_counter >= q->worker_source_max_transfers) { + continue; + } + /* skip if the worker is a checkpoint worker */ + if (is_checkpoint_worker(q, w)) { + continue; + } + /* skip if the worker already has this file */ + struct vine_file_replica *replica = vine_file_replica_table_lookup(w, f->cached_name); + if (replica) { + continue; + } + /* skip if the worker does not have enough disk space */ + int64_t available_disk_space = get_worker_available_disk_bytes(w); + if ((int64_t)f->size > available_disk_space) { + continue; + } + /* workers with more available disk space are preferred */ + priority_queue_push(valid_destinations, w, available_disk_space); + switch (q->replica_placement_policy) { + case VINE_REPLICA_PLACEMENT_POLICY_RANDOM: + priority_queue_push(valid_destinations, w, random_double()); + break; + case VINE_REPLICA_PLACEMENT_POLICY_DISK_LOAD: + priority_queue_push(valid_destinations, w, available_disk_space); + break; + case VINE_REPLICA_PLACEMENT_POLICY_TRANSFER_LOAD: + priority_queue_push(valid_destinations, w, -w->incoming_xfer_counter); + break; + } + } + + struct vine_worker_info *best_destination = priority_queue_pop(valid_destinations); + priority_queue_delete(valid_destinations); + + return best_destination; +} + +int file_has_been_checkpointed(struct vine_manager *q, struct vine_file *f) +{ + if (!q || !f || f->type != VINE_TEMP) { + return 0; + } + + /* check if the file has been checkpointed by any checkpoint worker */ + struct set *source_workers = hash_table_lookup(q->file_worker_table, f->cached_name); + if (!source_workers) { + return 0; + } + + struct vine_worker_info *source_worker; + SET_ITERATE(source_workers, source_worker) + { + /* skip if not a checkpoint worker */ + if (!is_checkpoint_worker(q, source_worker)) { + continue; + } + /* it is already checkpointed */ + return 1; + } + + return 0; +} + +void vine_temp_start_peer_transfer(struct vine_manager *q, struct vine_file *f, struct vine_worker_info *source_worker, struct vine_worker_info *dest_worker) +{ + if (!q || !f || f->type != VINE_TEMP || !source_worker || !dest_worker) { + return; + } + + char *source_addr = string_format("%s/%s", source_worker->transfer_url, f->cached_name); + vine_manager_put_url_now(q, dest_worker, source_worker, source_addr, f); + free(source_addr); +} + +int vine_temp_replicate_file_now(struct vine_manager *q, struct vine_file *f) +{ + if (!q || !f || f->type != VINE_TEMP) { + return 0; + } + + struct vine_worker_info *source_worker = get_best_source_worker(q, f); + if (!source_worker) { + return 0; + } + + struct vine_worker_info *dest_worker = get_best_dest_worker(q, f); + if (!dest_worker) { + return 0; + } + + vine_temp_start_peer_transfer(q, f, source_worker, dest_worker); + + return 1; +} + +int vine_temp_start_replication(struct vine_manager *q) +{ + if (!q) { + return 0; + } + + int processed = 0; + int iter_count = 0; + int iter_depth = MIN(q->attempt_schedule_depth, priority_queue_size(q->temp_files_to_replicate)); + struct list *skipped = list_create(); + + struct vine_file *f; + while ((f = priority_queue_pop(q->temp_files_to_replicate)) && (iter_count++ < iter_depth)) { + if (!f || f->type != VINE_TEMP || f->state != VINE_FILE_STATE_CREATED) { + continue; + } + + /* skip if the file has enough replicas or no replicas */ + int current_replica_count = vine_file_replica_count(q, f); + if (current_replica_count >= q->temp_replica_count || current_replica_count == 0) { + continue; + } + /* skip if the file has no ready replicas */ + int current_ready_replica_count = vine_file_replica_table_count_replicas(q, f->cached_name, VINE_FILE_REPLICA_STATE_READY); + if (current_ready_replica_count == 0) { + continue; + } + + /* if reach here, it means the file needs to be replicated and there is at least one ready replica. */ + if (!vine_temp_replicate_file_now(q, f)) { + list_push_tail(skipped, f); + continue; + } + + processed++; + + /* push back and keep evaluating the same file with a lower priority, until no more source + * or destination workers are available, or the file has enough replicas. */ + vine_temp_replicate_file_later(q, f); + } + + while ((f = list_pop_head(skipped))) { + vine_temp_replicate_file_later(q, f); + } + list_delete(skipped); + + return processed; +} + +/*************************************************************/ +/* Public Functions */ +/*************************************************************/ + +int vine_temp_replicate_file_later(struct vine_manager *q, struct vine_file *f) +{ + if (!q || !f || f->type != VINE_TEMP || f->state != VINE_FILE_STATE_CREATED) { + return 0; + } + + int current_replica_count = vine_file_replica_count(q, f); + if (current_replica_count == 0 || current_replica_count >= q->temp_replica_count) { + return 0; + } + + priority_queue_push(q->temp_files_to_replicate, f, -current_replica_count); + + return 1; +} + +int vine_temp_handle_file_lost(struct vine_manager *q, char *cachename) +{ + if (!q || !cachename) { + return 0; + } + + struct vine_file *f = hash_table_lookup(q->file_table, cachename); + if (!f || f->type != VINE_TEMP || f->state != VINE_FILE_STATE_CREATED) { + return 0; + } + + vine_temp_replicate_file_later(q, f); + + return 1; +} + +void vine_set_replica_placement_policy(struct vine_manager *q, vine_replica_placement_policy_t policy) +{ + if (!q) { + return; + } + + switch (policy) { + case VINE_REPLICA_PLACEMENT_POLICY_RANDOM: + debug(D_VINE, "Setting replica placement policy to RANDOM"); + q->replica_placement_policy = policy; + break; + case VINE_REPLICA_PLACEMENT_POLICY_DISK_LOAD: + debug(D_VINE, "Setting replica placement policy to DISK_LOAD"); + q->replica_placement_policy = policy; + break; + case VINE_REPLICA_PLACEMENT_POLICY_TRANSFER_LOAD: + q->replica_placement_policy = policy; + break; + default: + debug(D_ERROR, "Invalid replica placement policy: %d", policy); + } +} diff --git a/taskvine/src/manager/vine_temp.h b/taskvine/src/manager/vine_temp.h new file mode 100644 index 0000000000..8398fd26dd --- /dev/null +++ b/taskvine/src/manager/vine_temp.h @@ -0,0 +1,16 @@ +#ifndef vine_temp_H +#define vine_temp_H + +#include "vine_manager.h" + +int is_checkpoint_worker(struct vine_manager *q, struct vine_worker_info *w); +int vine_temp_replicate_file_now(struct vine_manager *q, struct vine_file *f); +int vine_temp_replicate_file_later(struct vine_manager *q, struct vine_file *f); +int vine_temp_checkpoint_file_later(struct vine_manager *q, struct vine_file *f); +int vine_temp_handle_file_lost(struct vine_manager *q, char *cachename); +int vine_temp_start_replication(struct vine_manager *q); +int vine_temp_start_checkpointing(struct vine_manager *q); +void vine_temp_start_peer_transfer(struct vine_manager *q, struct vine_file *f, struct vine_worker_info *source_worker, struct vine_worker_info *dest_worker); +int file_has_been_checkpointed(struct vine_manager *q, struct vine_file *f); + +#endif \ No newline at end of file From 40e9679b3686cbdbc0fd3321b10fb3fd3de10e0f Mon Sep 17 00:00:00 2001 From: Jin Zhou Date: Tue, 14 Oct 2025 16:09:33 -0400 Subject: [PATCH 022/113] vine: task graph executor in C --- poncho/src/poncho/library_network_code.py | 2 +- .../ndcctools/taskvine/graph_definition.py | 1 + .../ndcctools/taskvine/graph_executor.py | 2 +- .../python3/ndcctools/taskvine/utils.py | 12 ++++++ taskvine/src/manager/vine_manager.c | 1 - taskvine/src/manager/vine_task_graph.c | 20 ++------- taskvine/src/manager/vine_task_node.c | 41 ++++++++----------- taskvine/src/manager/vine_temp.c | 26 ------------ taskvine/src/manager/vine_temp.h | 3 -- 9 files changed, 36 insertions(+), 72 deletions(-) diff --git a/poncho/src/poncho/library_network_code.py b/poncho/src/poncho/library_network_code.py index 6f3e1db092..cbc88209ef 100755 --- a/poncho/src/poncho/library_network_code.py +++ b/poncho/src/poncho/library_network_code.py @@ -177,7 +177,7 @@ def start_function(in_pipe_fd, thread_limit=1): 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: diff --git a/taskvine/src/bindings/python3/ndcctools/taskvine/graph_definition.py b/taskvine/src/bindings/python3/ndcctools/taskvine/graph_definition.py index b9db690ad2..1602621bbb 100644 --- a/taskvine/src/bindings/python3/ndcctools/taskvine/graph_definition.py +++ b/taskvine/src/bindings/python3/ndcctools/taskvine/graph_definition.py @@ -40,6 +40,7 @@ def dist_func(mode, low, high): return 0 assert mode in ["uniform", "normal", "lognormal", "pareto", "mix"] + # uniform distribution, flat spread def uniform_dist(): return random.uniform(low, high) diff --git a/taskvine/src/bindings/python3/ndcctools/taskvine/graph_executor.py b/taskvine/src/bindings/python3/ndcctools/taskvine/graph_executor.py index 7efca4a554..b32e10e328 100644 --- a/taskvine/src/bindings/python3/ndcctools/taskvine/graph_executor.py +++ b/taskvine/src/bindings/python3/ndcctools/taskvine/graph_executor.py @@ -124,7 +124,7 @@ def _create_library_task(self, libcores=1, hoisting_modules=[], libtask_env_file compute_single_key, library_context_info=[init_task_graph_context, [], {"task_graph_path": self.task_graph_pkl_file_remote_path}], add_env=False, - infile_load_mode="text", + function_infile_load_mode="json", hoisting_modules=hoisting_modules ) self.libtask.add_input(self.declare_file(self.task_graph_pkl_file_local_path), self.task_graph_pkl_file_remote_path) diff --git a/taskvine/src/bindings/python3/ndcctools/taskvine/utils.py b/taskvine/src/bindings/python3/ndcctools/taskvine/utils.py index b54d9c7bf4..4ced92406a 100644 --- a/taskvine/src/bindings/python3/ndcctools/taskvine/utils.py +++ b/taskvine/src/bindings/python3/ndcctools/taskvine/utils.py @@ -37,4 +37,16 @@ def load_variable_from_library(var_name): return globals()[var_name] +# helper function that deletes all files in a directory +def delete_all_files(root_dir): + 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}") + # vim: set sts=4 sw=4 ts=4 expandtab ft=python: diff --git a/taskvine/src/manager/vine_manager.c b/taskvine/src/manager/vine_manager.c index 07bdb9a5a1..4e46bcf232 100644 --- a/taskvine/src/manager/vine_manager.c +++ b/taskvine/src/manager/vine_manager.c @@ -4377,7 +4377,6 @@ struct vine_manager *vine_ssl_create(int port, const char *key, const char *cert q->shutting_down = 0; vine_set_replica_placement_policy(q, VINE_REPLICA_PLACEMENT_POLICY_RANDOM); - if ((envstring = getenv("VINE_BANDWIDTH"))) { q->bandwidth_limit = string_metric_parse(envstring); if (q->bandwidth_limit < 0) { diff --git a/taskvine/src/manager/vine_task_graph.c b/taskvine/src/manager/vine_task_graph.c index 0ae4e0f052..592bd069b2 100644 --- a/taskvine/src/manager/vine_task_graph.c +++ b/taskvine/src/manager/vine_task_graph.c @@ -242,20 +242,6 @@ static struct vine_task_node *get_node_by_task(struct vine_task_graph *tg, struc return NULL; } -static void handle_checkpoint_worker_stagein(struct vine_task_graph *tg, struct vine_file *f) -{ - if (!tg || !f) { - return; - } - - struct vine_task_node *this_node = hash_table_lookup(tg->outfile_cachename_to_node, f->cached_name); - if (!this_node) { - return; - } - - vine_task_node_prune_ancestors(this_node); -} - /*************************************************************/ /* Public APIs */ /*************************************************************/ @@ -390,10 +376,10 @@ void vine_task_graph_execute(struct vine_task_graph *tg) if (regular_tasks_part->current == 0) { progress_bar_set_start_time(pbar, task->time_when_commit_start); } - + /* update critical time */ vine_task_node_update_critical_time(node, task->time_workers_execute_last); - + /* mark this regular task as completed */ progress_bar_update_part(pbar, regular_tasks_part, 1); @@ -406,7 +392,7 @@ void vine_task_graph_execute(struct vine_task_graph *tg) } } - /* enqueue the output file for replication or checkpointing */ + /* enqueue the output file for replication */ switch (node->outfile_type) { case VINE_NODE_OUTFILE_TYPE_TEMP: vine_task_node_replicate_outfile(node); diff --git a/taskvine/src/manager/vine_task_node.c b/taskvine/src/manager/vine_task_node.c index ae99004ba8..5a3145548d 100644 --- a/taskvine/src/manager/vine_task_node.c +++ b/taskvine/src/manager/vine_task_node.c @@ -3,6 +3,8 @@ #include "vine_task.h" #include "vine_file.h" #include "vine_task_graph.h" +#include "jx.h" +#include "jx_print.h" #include "xxmalloc.h" #include "stringtools.h" #include "taskvine.h" @@ -81,8 +83,16 @@ struct vine_task_node *vine_task_node_create( vine_task_set_library_required(node->task, library_name); vine_task_addref(node->task); - /* add the infile as the key of the graph to compute */ - char *infile_content = string_format("%s", node->node_key); + /* build JSON infile expected by library: {"fn_args": [key], "fn_kwargs": {}} */ + struct jx *event = jx_object(NULL); + struct jx *args = jx_array(NULL); + jx_array_append(args, jx_string(node->node_key)); + jx_insert(event, jx_string("fn_args"), args); + jx_insert(event, jx_string("fn_kwargs"), jx_object(NULL)); + + char *infile_content = jx_print_string(event); + jx_delete(event); + node->infile = vine_declare_buffer(node->manager, infile_content, strlen(infile_content), VINE_CACHE_LEVEL_TASK, VINE_UNLINK_WHEN_DONE); free(infile_content); vine_task_add_input(node->task, node->infile, "infile", VINE_TRANSFER_ALWAYS); @@ -137,22 +147,16 @@ static int _node_outfile_is_persisted(struct vine_task_node *node) return 0; } - /* if the outfile pointer is NULL, it means the node is producing a shared file system file */ - if (!node->outfile) { + switch (node->outfile_type) { + case VINE_NODE_OUTFILE_TYPE_LOCAL: return 1; - } - - switch (node->outfile->type) { - case VINE_FILE: + case VINE_NODE_OUTFILE_TYPE_SHARED_FILE_SYSTEM: return 1; - case VINE_TEMP: - /* check if this file is in the any of the checkpoint workers */ - return file_has_been_checkpointed(node->manager, node->outfile); - default: - /* not supported at the moment */ - debug(D_ERROR, "unsupported outfile type: %d", node->outfile->type); + case VINE_NODE_OUTFILE_TYPE_TEMP: return 0; } + + return 0; } double vine_task_node_calculate_priority(struct vine_task_node *node) @@ -555,15 +559,6 @@ void vine_task_node_replicate_outfile(struct vine_task_node *node) vine_temp_replicate_file_later(node->manager, node->outfile); } -void vine_task_node_checkpoint_outfile(struct vine_task_node *node) -{ - if (!node || !node->outfile) { - return; - } - - vine_temp_checkpoint_file_later(node->manager, node->outfile); -} - int vine_task_node_set_outfile_size_bytes(struct vine_task_node *node, size_t outfile_size_bytes) { if (!node || !node->outfile) { diff --git a/taskvine/src/manager/vine_temp.c b/taskvine/src/manager/vine_temp.c index 914de9a893..4d94326d38 100644 --- a/taskvine/src/manager/vine_temp.c +++ b/taskvine/src/manager/vine_temp.c @@ -121,32 +121,6 @@ static struct vine_worker_info *get_best_dest_worker(struct vine_manager *q, str return best_destination; } -int file_has_been_checkpointed(struct vine_manager *q, struct vine_file *f) -{ - if (!q || !f || f->type != VINE_TEMP) { - return 0; - } - - /* check if the file has been checkpointed by any checkpoint worker */ - struct set *source_workers = hash_table_lookup(q->file_worker_table, f->cached_name); - if (!source_workers) { - return 0; - } - - struct vine_worker_info *source_worker; - SET_ITERATE(source_workers, source_worker) - { - /* skip if not a checkpoint worker */ - if (!is_checkpoint_worker(q, source_worker)) { - continue; - } - /* it is already checkpointed */ - return 1; - } - - return 0; -} - void vine_temp_start_peer_transfer(struct vine_manager *q, struct vine_file *f, struct vine_worker_info *source_worker, struct vine_worker_info *dest_worker) { if (!q || !f || f->type != VINE_TEMP || !source_worker || !dest_worker) { diff --git a/taskvine/src/manager/vine_temp.h b/taskvine/src/manager/vine_temp.h index 8398fd26dd..d9e4d6b428 100644 --- a/taskvine/src/manager/vine_temp.h +++ b/taskvine/src/manager/vine_temp.h @@ -6,11 +6,8 @@ int is_checkpoint_worker(struct vine_manager *q, struct vine_worker_info *w); int vine_temp_replicate_file_now(struct vine_manager *q, struct vine_file *f); int vine_temp_replicate_file_later(struct vine_manager *q, struct vine_file *f); -int vine_temp_checkpoint_file_later(struct vine_manager *q, struct vine_file *f); int vine_temp_handle_file_lost(struct vine_manager *q, char *cachename); int vine_temp_start_replication(struct vine_manager *q); -int vine_temp_start_checkpointing(struct vine_manager *q); void vine_temp_start_peer_transfer(struct vine_manager *q, struct vine_file *f, struct vine_worker_info *source_worker, struct vine_worker_info *dest_worker); -int file_has_been_checkpointed(struct vine_manager *q, struct vine_file *f); #endif \ No newline at end of file From 21b25a4c04a6e9aee223b5081d3037d2254447e9 Mon Sep 17 00:00:00 2001 From: Jin Zhou Date: Tue, 14 Oct 2025 16:14:11 -0400 Subject: [PATCH 023/113] simple fix --- taskvine/src/manager/vine_task_node.c | 5 +++-- 1 file changed, 3 insertions(+), 2 deletions(-) diff --git a/taskvine/src/manager/vine_task_node.c b/taskvine/src/manager/vine_task_node.c index 5a3145548d..569e1db095 100644 --- a/taskvine/src/manager/vine_task_node.c +++ b/taskvine/src/manager/vine_task_node.c @@ -113,12 +113,13 @@ void vine_task_node_set_outfile(struct vine_task_node *node, vine_task_node_outf /* create the output file */ switch (node->outfile_type) { - case VINE_NODE_OUTFILE_TYPE_LOCAL: + case VINE_NODE_OUTFILE_TYPE_LOCAL: { char *persistent_path = string_format("%s/outputs/%s", node->staging_dir, node->outfile_remote_name); node->outfile = vine_declare_file(node->manager, persistent_path, VINE_CACHE_LEVEL_WORKFLOW, 0); free(persistent_path); debug(D_VINE, "node %s: outfile type = VINE_NODE_OUTFILE_TYPE_LOCAL, outfile = %s", node->node_key, node->outfile->cached_name); break; + } case VINE_NODE_OUTFILE_TYPE_TEMP: node->outfile = vine_declare_temp(node->manager); debug(D_VINE, "node %s: outfile type = VINE_NODE_OUTFILE_TYPE_TEMP, outfile = %s", node->node_key, node->outfile->cached_name); @@ -432,7 +433,7 @@ int _prune_ancestors_of_persisted_node(struct vine_task_node *node) timestamp_t start_time = timestamp_get(); unlink(ancestor_node->outfile_remote_name); node->time_spent_on_unlink_local_files += timestamp_get() - start_time; - debug(D_VINE, "unlinked %s size: %ld bytes, time: %ld", ancestor_node->outfile_remote_name, ancestor_node->outfile_size_bytes, node->time_spent_on_unlink_local_files); + debug(D_VINE, "unlinked %s size: %ld bytes, time: %llu", ancestor_node->outfile_remote_name, ancestor_node->outfile_size_bytes, node->time_spent_on_unlink_local_files); } else { switch (ancestor_node->outfile->type) { case VINE_FILE: From 648071fe91921af53c356c5702f3efbcdb5c4f9b Mon Sep 17 00:00:00 2001 From: Jin Zhou Date: Tue, 14 Oct 2025 16:17:52 -0400 Subject: [PATCH 024/113] braces in block --- taskvine/src/manager/vine_task_graph.c | 3 ++- 1 file changed, 2 insertions(+), 1 deletion(-) diff --git a/taskvine/src/manager/vine_task_graph.c b/taskvine/src/manager/vine_task_graph.c index 592bd069b2..0e37061ebe 100644 --- a/taskvine/src/manager/vine_task_graph.c +++ b/taskvine/src/manager/vine_task_graph.c @@ -342,7 +342,7 @@ void vine_task_graph_execute(struct vine_task_graph *tg) /* if the outfile is set to save on the sharedfs, stat to get the size of the file */ switch (node->outfile_type) { - case VINE_NODE_OUTFILE_TYPE_SHARED_FILE_SYSTEM: + case VINE_NODE_OUTFILE_TYPE_SHARED_FILE_SYSTEM: { struct stat info; int result = stat(node->outfile_remote_name, &info); if (result < 0) { @@ -353,6 +353,7 @@ void vine_task_graph_execute(struct vine_task_graph *tg) } node->outfile_size_bytes = info.st_size; break; + } case VINE_NODE_OUTFILE_TYPE_LOCAL: case VINE_NODE_OUTFILE_TYPE_TEMP: node->outfile_size_bytes = node->outfile->size; From 824213656d039eee7ce862a243b776ff3b6fdc7a Mon Sep 17 00:00:00 2001 From: Jin Zhou Date: Tue, 14 Oct 2025 16:19:26 -0400 Subject: [PATCH 025/113] type fix --- taskvine/src/manager/vine_task_node.c | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/taskvine/src/manager/vine_task_node.c b/taskvine/src/manager/vine_task_node.c index 569e1db095..cb5342dc67 100644 --- a/taskvine/src/manager/vine_task_node.c +++ b/taskvine/src/manager/vine_task_node.c @@ -433,7 +433,7 @@ int _prune_ancestors_of_persisted_node(struct vine_task_node *node) timestamp_t start_time = timestamp_get(); unlink(ancestor_node->outfile_remote_name); node->time_spent_on_unlink_local_files += timestamp_get() - start_time; - debug(D_VINE, "unlinked %s size: %ld bytes, time: %llu", ancestor_node->outfile_remote_name, ancestor_node->outfile_size_bytes, node->time_spent_on_unlink_local_files); + debug(D_VINE, "unlinked %s size: %ld bytes, time: %lu", ancestor_node->outfile_remote_name, ancestor_node->outfile_size_bytes, node->time_spent_on_unlink_local_files); } else { switch (ancestor_node->outfile->type) { case VINE_FILE: From c841f6cb013e507906d312d188ad0564dd3804a4 Mon Sep 17 00:00:00 2001 From: Jin Zhou Date: Tue, 14 Oct 2025 16:25:56 -0400 Subject: [PATCH 026/113] type fix --- taskvine/src/manager/vine_task_node.c | 4 +++- 1 file changed, 3 insertions(+), 1 deletion(-) diff --git a/taskvine/src/manager/vine_task_node.c b/taskvine/src/manager/vine_task_node.c index cb5342dc67..fce542f585 100644 --- a/taskvine/src/manager/vine_task_node.c +++ b/taskvine/src/manager/vine_task_node.c @@ -18,6 +18,8 @@ #include "vine_temp.h" #include "random.h" +#include + double compute_lex_priority(const char *key) { double score = 0.0; @@ -433,7 +435,7 @@ int _prune_ancestors_of_persisted_node(struct vine_task_node *node) timestamp_t start_time = timestamp_get(); unlink(ancestor_node->outfile_remote_name); node->time_spent_on_unlink_local_files += timestamp_get() - start_time; - debug(D_VINE, "unlinked %s size: %ld bytes, time: %lu", ancestor_node->outfile_remote_name, ancestor_node->outfile_size_bytes, node->time_spent_on_unlink_local_files); + debug(D_VINE, "unlinked %s size: %ld bytes, time: %" PRIu64, ancestor_node->outfile_remote_name, ancestor_node->outfile_size_bytes, (uint64_t) node->time_spent_on_unlink_local_files); } else { switch (ancestor_node->outfile->type) { case VINE_FILE: From 39168ef708115a1b9e0e21512cf364065ab9dec6 Mon Sep 17 00:00:00 2001 From: Jin Zhou Date: Tue, 14 Oct 2025 16:30:54 -0400 Subject: [PATCH 027/113] lint --- taskvine/src/manager/vine_task_graph.c | 2 +- taskvine/src/manager/vine_task_node.c | 4 ++-- 2 files changed, 3 insertions(+), 3 deletions(-) diff --git a/taskvine/src/manager/vine_task_graph.c b/taskvine/src/manager/vine_task_graph.c index 0e37061ebe..1c0ec8bfbe 100644 --- a/taskvine/src/manager/vine_task_graph.c +++ b/taskvine/src/manager/vine_task_graph.c @@ -353,7 +353,7 @@ void vine_task_graph_execute(struct vine_task_graph *tg) } node->outfile_size_bytes = info.st_size; break; - } + } case VINE_NODE_OUTFILE_TYPE_LOCAL: case VINE_NODE_OUTFILE_TYPE_TEMP: node->outfile_size_bytes = node->outfile->size; diff --git a/taskvine/src/manager/vine_task_node.c b/taskvine/src/manager/vine_task_node.c index fce542f585..8bb01eacb9 100644 --- a/taskvine/src/manager/vine_task_node.c +++ b/taskvine/src/manager/vine_task_node.c @@ -121,7 +121,7 @@ void vine_task_node_set_outfile(struct vine_task_node *node, vine_task_node_outf free(persistent_path); debug(D_VINE, "node %s: outfile type = VINE_NODE_OUTFILE_TYPE_LOCAL, outfile = %s", node->node_key, node->outfile->cached_name); break; - } + } case VINE_NODE_OUTFILE_TYPE_TEMP: node->outfile = vine_declare_temp(node->manager); debug(D_VINE, "node %s: outfile type = VINE_NODE_OUTFILE_TYPE_TEMP, outfile = %s", node->node_key, node->outfile->cached_name); @@ -435,7 +435,7 @@ int _prune_ancestors_of_persisted_node(struct vine_task_node *node) timestamp_t start_time = timestamp_get(); unlink(ancestor_node->outfile_remote_name); node->time_spent_on_unlink_local_files += timestamp_get() - start_time; - debug(D_VINE, "unlinked %s size: %ld bytes, time: %" PRIu64, ancestor_node->outfile_remote_name, ancestor_node->outfile_size_bytes, (uint64_t) node->time_spent_on_unlink_local_files); + debug(D_VINE, "unlinked %s size: %ld bytes, time: %" PRIu64, ancestor_node->outfile_remote_name, ancestor_node->outfile_size_bytes, (uint64_t)node->time_spent_on_unlink_local_files); } else { switch (ancestor_node->outfile->type) { case VINE_FILE: From e8c23e6c9155a8bf2fba12ead9445220fcc8c330 Mon Sep 17 00:00:00 2001 From: Jin Zhou Date: Tue, 14 Oct 2025 22:50:54 -0400 Subject: [PATCH 028/113] some fixes --- taskvine/src/manager/vine_task_graph.c | 2 +- taskvine/src/manager/vine_task_node.c | 47 +++++++++++++++++++------- 2 files changed, 36 insertions(+), 13 deletions(-) diff --git a/taskvine/src/manager/vine_task_graph.c b/taskvine/src/manager/vine_task_graph.c index 1c0ec8bfbe..c6fdcb24b6 100644 --- a/taskvine/src/manager/vine_task_graph.c +++ b/taskvine/src/manager/vine_task_graph.c @@ -359,7 +359,7 @@ void vine_task_graph_execute(struct vine_task_graph *tg) node->outfile_size_bytes = node->outfile->size; break; } - debug(D_VINE, "Node %s completed with outfile %s size: %ld bytes", node->node_key, node->outfile_remote_name, node->outfile_size_bytes); + debug(D_VINE, "Node %s completed with outfile %s size: %zu bytes", node->node_key, node->outfile_remote_name, node->outfile_size_bytes); /* mark the node as completed */ node->completed = 1; diff --git a/taskvine/src/manager/vine_task_node.c b/taskvine/src/manager/vine_task_node.c index 8bb01eacb9..983a3cc09b 100644 --- a/taskvine/src/manager/vine_task_node.c +++ b/taskvine/src/manager/vine_task_node.c @@ -11,7 +11,6 @@ #include "timestamp.h" #include "set.h" #include "hash_table.h" -#include "unistd.h" #include "debug.h" #include "assert.h" #include "vine_worker_info.h" @@ -19,6 +18,11 @@ #include "random.h" #include +#include +#include +#include +#include +#include double compute_lex_priority(const char *key) { @@ -53,7 +57,7 @@ struct vine_task_node *vine_task_node_create( node->manager = manager; node->node_key = xxstrdup(node_key); - node->staging_dir = xxstrdup(staging_dir); + node->priority_mode = priority_mode; node->prune_status = PRUNE_STATUS_NOT_PRUNED; node->parents = list_create(); @@ -116,9 +120,12 @@ void vine_task_node_set_outfile(struct vine_task_node *node, vine_task_node_outf /* create the output file */ switch (node->outfile_type) { case VINE_NODE_OUTFILE_TYPE_LOCAL: { - char *persistent_path = string_format("%s/outputs/%s", node->staging_dir, node->outfile_remote_name); - node->outfile = vine_declare_file(node->manager, persistent_path, VINE_CACHE_LEVEL_WORKFLOW, 0); - free(persistent_path); + char *local_output_dir = string_format("%s/outputs", node->staging_dir); + mkdir(local_output_dir, 0777); // ignore errors if exists + char *local_output_path = string_format("%s/%s", local_output_dir, node->outfile_remote_name); + node->outfile = vine_declare_file(node->manager, local_output_path, VINE_CACHE_LEVEL_WORKFLOW, 0); + free(local_output_dir); + free(local_output_path); debug(D_VINE, "node %s: outfile type = VINE_NODE_OUTFILE_TYPE_LOCAL, outfile = %s", node->node_key, node->outfile->cached_name); break; } @@ -259,6 +266,7 @@ static void _find_parents_dfs(struct vine_task_node *node, int remaining_depth, if (!node || set_lookup(visited, node)) { return; } + set_insert(visited, node); if (remaining_depth == 0) { list_push_tail(result, node); @@ -277,7 +285,9 @@ static struct list *_find_parents_in_depth(struct vine_task_node *node, int dept if (!node || depth < 0) { return NULL; } + struct list *result = list_create(); + struct set *visited = set_create(0); _find_parents_dfs(node, depth, result, visited); set_delete(visited); @@ -432,10 +442,12 @@ int _prune_ancestors_of_persisted_node(struct vine_task_node *node) { /* unlink the shared file system file */ if (!ancestor_node->outfile) { - timestamp_t start_time = timestamp_get(); - unlink(ancestor_node->outfile_remote_name); - node->time_spent_on_unlink_local_files += timestamp_get() - start_time; - debug(D_VINE, "unlinked %s size: %ld bytes, time: %" PRIu64, ancestor_node->outfile_remote_name, ancestor_node->outfile_size_bytes, (uint64_t)node->time_spent_on_unlink_local_files); + timestamp_t unlink_start = timestamp_get(); + if (ancestor_node->outfile_remote_name) { + unlink(ancestor_node->outfile_remote_name); // system call + } + node->time_spent_on_unlink_local_files += timestamp_get() - unlink_start; + debug(D_VINE, "unlinked %s size: %zu bytes, time: %" PRIu64, ancestor_node->outfile_remote_name, ancestor_node->outfile_size_bytes, node->time_spent_on_unlink_local_files); } else { switch (ancestor_node->outfile->type) { case VINE_FILE: @@ -483,7 +495,9 @@ void vine_task_node_print_info(struct vine_task_node *node) debug(D_VINE, "node info %s task_id: %d", node->node_key, node->task->task_id); debug(D_VINE, "node info %s depth: %d", node->node_key, node->depth); - debug(D_VINE, "node info %s outfile remote name: %s", node->node_key, node->outfile_remote_name); + if (node->outfile_remote_name) { + debug(D_VINE, "node info %s outfile remote name: %s", node->node_key, node->outfile_remote_name); + } if (node->outfile) { switch (node->outfile->type) { @@ -511,8 +525,9 @@ void vine_task_node_print_info(struct vine_task_node *node) parent_keys = tmp; } } - debug(D_VINE, "node info %s parents: %s", node->node_key, parent_keys); + debug(D_VINE, "node info %s parents: %s", node->node_key, parent_keys ? parent_keys : "(none)"); free(parent_keys); + parent_keys = NULL; char *child_keys = NULL; struct vine_task_node *child_node; @@ -526,8 +541,9 @@ void vine_task_node_print_info(struct vine_task_node *node) child_keys = tmp; } } - debug(D_VINE, "node info %s children: %s", node->node_key, child_keys); + debug(D_VINE, "node info %s children: %s", node->node_key, child_keys ? child_keys : "(none)"); free(child_keys); + child_keys = NULL; return; } @@ -559,6 +575,10 @@ void vine_task_node_replicate_outfile(struct vine_task_node *node) return; } + if (node->outfile->type != VINE_TEMP) { + return; + } + vine_temp_replicate_file_later(node->manager, node->outfile); } @@ -586,6 +606,9 @@ void vine_task_node_delete(struct vine_task_node *node) if (node->outfile_remote_name) { free(node->outfile_remote_name); } + if (node->staging_dir) { + free(node->staging_dir); + } vine_task_delete(node->task); node->task = NULL; From ef0e8a2af0442abfbde681ca2e5822d04771c872 Mon Sep 17 00:00:00 2001 From: Jin Zhou Date: Tue, 14 Oct 2025 23:06:16 -0400 Subject: [PATCH 029/113] bug --- taskvine/src/manager/taskvine.h | 3 +- taskvine/src/manager/vine_manager.c | 11 ++++-- taskvine/src/manager/vine_task_node.c | 55 ++++++--------------------- taskvine/src/manager/vine_task_node.h | 1 - taskvine/src/manager/vine_temp.c | 13 ------- taskvine/src/manager/vine_temp.h | 1 - 6 files changed, 22 insertions(+), 62 deletions(-) diff --git a/taskvine/src/manager/taskvine.h b/taskvine/src/manager/taskvine.h index 9e1d9b7da3..4ea0258407 100644 --- a/taskvine/src/manager/taskvine.h +++ b/taskvine/src/manager/taskvine.h @@ -954,8 +954,9 @@ The given file or directory object is deleted from all worker's caches, but is still available on the manager's site, and can be recovered by submitting a recovery task. @param m A manager object @param f Any file object. +@return The number of replicas pruned. */ -void vine_prune_file(struct vine_manager *m, struct vine_file *f); +int vine_prune_file(struct vine_manager *m, struct vine_file *f); //@} diff --git a/taskvine/src/manager/vine_manager.c b/taskvine/src/manager/vine_manager.c index 4e46bcf232..3ff5965403 100644 --- a/taskvine/src/manager/vine_manager.c +++ b/taskvine/src/manager/vine_manager.c @@ -6600,16 +6600,18 @@ Should be invoked by the application when a file will never be needed again, to free up available space. */ -void vine_prune_file(struct vine_manager *m, struct vine_file *f) +int vine_prune_file(struct vine_manager *m, struct vine_file *f) { if (!f) { - return; + return 0; } if (!m) { - return; + return 0; } + int pruned_replica_count = 0; + /* delete all of the replicas present at remote workers. */ struct set *source_workers = hash_table_lookup(m->file_worker_table, f->cached_name); if (source_workers && set_size(source_workers) > 0) { @@ -6618,10 +6620,13 @@ void vine_prune_file(struct vine_manager *m, struct vine_file *f) for (int i = 0; workers_array[i] != NULL; i++) { struct vine_worker_info *w = (struct vine_worker_info *)workers_array[i]; delete_worker_file(m, w, f->cached_name, 0, 0); + pruned_replica_count++; } set_free_values_array(workers_array); } } + + return pruned_replica_count; } /* diff --git a/taskvine/src/manager/vine_task_node.c b/taskvine/src/manager/vine_task_node.c index 983a3cc09b..4970b70410 100644 --- a/taskvine/src/manager/vine_task_node.c +++ b/taskvine/src/manager/vine_task_node.c @@ -21,6 +21,7 @@ #include #include #include +#include #include #include @@ -29,7 +30,7 @@ double compute_lex_priority(const char *key) double score = 0.0; double factor = 1.0; for (int i = 0; i < 8 && key[i] != '\0'; i++) { - score += key[i] * factor; + score += (unsigned char)key[i] * factor; factor *= 0.01; } return -score; @@ -57,6 +58,7 @@ struct vine_task_node *vine_task_node_create( node->manager = manager; node->node_key = xxstrdup(node_key); + node->staging_dir = xxstrdup(staging_dir); node->priority_mode = priority_mode; node->prune_status = PRUNE_STATUS_NOT_PRUNED; @@ -66,8 +68,8 @@ struct vine_task_node *vine_task_node_create( node->completed = 0; node->prune_depth = prune_depth; node->retry_attempts_left = 1; + node->outfile_size_bytes = 0; - node->outfile_size_bytes = 0; node->depth = -1; node->height = -1; node->upstream_subgraph_size = -1; @@ -121,7 +123,9 @@ void vine_task_node_set_outfile(struct vine_task_node *node, vine_task_node_outf switch (node->outfile_type) { case VINE_NODE_OUTFILE_TYPE_LOCAL: { char *local_output_dir = string_format("%s/outputs", node->staging_dir); - mkdir(local_output_dir, 0777); // ignore errors if exists + if (mkdir(local_output_dir, 0777) != 0 && errno != EEXIST) { + debug(D_ERROR, "failed to mkdir %s (errno=%d)", local_output_dir, errno); + } char *local_output_path = string_format("%s/%s", local_output_dir, node->outfile_remote_name); node->outfile = vine_declare_file(node->manager, local_output_path, VINE_CACHE_LEVEL_WORKFLOW, 0); free(local_output_dir); @@ -233,33 +237,6 @@ void vine_task_node_update_critical_time(struct vine_task_node *node, timestamp_ node->critical_time = max_parent_critical_time + execution_time; } -/* delete all of the replicas present at remote workers. */ -int _prune_outfile_from_regular_workers(struct vine_task_node *node) -{ - if (!node || !node->outfile) { - return 0; - } - - struct set *source_workers = hash_table_lookup(node->manager->file_worker_table, node->outfile->cached_name); - if (!source_workers) { - return 0; - } - - int pruned_replica_count = 0; - - struct vine_worker_info *source_worker; - SET_ITERATE(source_workers, source_worker) - { - if (is_checkpoint_worker(node->manager, source_worker)) { - continue; - } - delete_worker_file(node->manager, source_worker, node->outfile->cached_name, 0, 0); - pruned_replica_count++; - } - - return pruned_replica_count; -} - /* the dfs helper function for finding parents in a specific depth */ static void _find_parents_dfs(struct vine_task_node *node, int remaining_depth, struct list *result, struct set *visited) { @@ -344,7 +321,7 @@ int _prune_ancestors_of_temp_node(struct vine_task_node *node) continue; } - pruned_replica_count += _prune_outfile_from_regular_workers(parent_node); + pruned_replica_count += vine_prune_file(node->manager, parent_node->outfile); /* this parent is pruned because a successor that produces a temp file is completed, it is unsafe because the * manager may submit a recovery task to bring it back in case of worker failures. */ parent_node->prune_status = PRUNE_STATUS_UNSAFE; @@ -447,7 +424,10 @@ int _prune_ancestors_of_persisted_node(struct vine_task_node *node) unlink(ancestor_node->outfile_remote_name); // system call } node->time_spent_on_unlink_local_files += timestamp_get() - unlink_start; - debug(D_VINE, "unlinked %s size: %zu bytes, time: %" PRIu64, ancestor_node->outfile_remote_name, ancestor_node->outfile_size_bytes, node->time_spent_on_unlink_local_files); + debug(D_VINE, "unlinked %s size: %zu bytes, time: %" PRIu64, + ancestor_node->outfile_remote_name ? ancestor_node->outfile_remote_name : "(null)", + ancestor_node->outfile_size_bytes, + (uint64_t)node->time_spent_on_unlink_local_files); } else { switch (ancestor_node->outfile->type) { case VINE_FILE: @@ -582,17 +562,6 @@ void vine_task_node_replicate_outfile(struct vine_task_node *node) vine_temp_replicate_file_later(node->manager, node->outfile); } -int vine_task_node_set_outfile_size_bytes(struct vine_task_node *node, size_t outfile_size_bytes) -{ - if (!node || !node->outfile) { - return -1; - } - - node->outfile_size_bytes = outfile_size_bytes; - - return 0; -} - /* delete the node and all of its associated resources. */ void vine_task_node_delete(struct vine_task_node *node) { diff --git a/taskvine/src/manager/vine_task_node.h b/taskvine/src/manager/vine_task_node.h index 13486e12c0..b480acfc7a 100644 --- a/taskvine/src/manager/vine_task_node.h +++ b/taskvine/src/manager/vine_task_node.h @@ -70,7 +70,6 @@ void vine_task_node_update_critical_time(struct vine_task_node *node, timestamp_ void vine_task_node_replicate_outfile(struct vine_task_node *node); int vine_task_node_submit(struct vine_task_node *node); void vine_task_node_set_outfile(struct vine_task_node *node, vine_task_node_outfile_type_t outfile_type, const char *outfile_remote_name); -int vine_task_node_set_outfile_size_bytes(struct vine_task_node *node, size_t outfile_size_bytes); void vine_task_node_checkpoint_outfile(struct vine_task_node *node); #endif \ No newline at end of file diff --git a/taskvine/src/manager/vine_temp.c b/taskvine/src/manager/vine_temp.c index 4d94326d38..12be13a26c 100644 --- a/taskvine/src/manager/vine_temp.c +++ b/taskvine/src/manager/vine_temp.c @@ -15,15 +15,6 @@ /* Private Functions */ /*************************************************************/ -int is_checkpoint_worker(struct vine_manager *q, struct vine_worker_info *w) -{ - if (!q || !w || !w->features) { - return 0; - } - - return hash_table_lookup(w->features, "checkpoint-worker") != NULL; -} - static struct vine_worker_info *get_best_source_worker(struct vine_manager *q, struct vine_file *f) { if (!q || !f || f->type != VINE_TEMP) { @@ -86,10 +77,6 @@ static struct vine_worker_info *get_best_dest_worker(struct vine_manager *q, str if (w->incoming_xfer_counter >= q->worker_source_max_transfers) { continue; } - /* skip if the worker is a checkpoint worker */ - if (is_checkpoint_worker(q, w)) { - continue; - } /* skip if the worker already has this file */ struct vine_file_replica *replica = vine_file_replica_table_lookup(w, f->cached_name); if (replica) { diff --git a/taskvine/src/manager/vine_temp.h b/taskvine/src/manager/vine_temp.h index d9e4d6b428..ff2efb5011 100644 --- a/taskvine/src/manager/vine_temp.h +++ b/taskvine/src/manager/vine_temp.h @@ -3,7 +3,6 @@ #include "vine_manager.h" -int is_checkpoint_worker(struct vine_manager *q, struct vine_worker_info *w); int vine_temp_replicate_file_now(struct vine_manager *q, struct vine_file *f); int vine_temp_replicate_file_later(struct vine_manager *q, struct vine_file *f); int vine_temp_handle_file_lost(struct vine_manager *q, char *cachename); From 6b527b192f145019fc2d050b5f11dadb4a504a6d Mon Sep 17 00:00:00 2001 From: Jin Zhou Date: Wed, 15 Oct 2025 10:59:25 -0400 Subject: [PATCH 030/113] use proxy function and library --- .../ndcctools/taskvine/graph_executor.py | 8 +- taskvine/src/manager/taskvine.h | 6 +- taskvine/src/manager/vine_task_graph.c | 124 ++++++++++++---- taskvine/src/manager/vine_task_graph.h | 4 +- taskvine/src/manager/vine_task_node.c | 138 +++++++++--------- taskvine/src/manager/vine_task_node.h | 4 +- 6 files changed, 179 insertions(+), 105 deletions(-) diff --git a/taskvine/src/bindings/python3/ndcctools/taskvine/graph_executor.py b/taskvine/src/bindings/python3/ndcctools/taskvine/graph_executor.py index b32e10e328..5d7137f25b 100644 --- a/taskvine/src/bindings/python3/ndcctools/taskvine/graph_executor.py +++ b/taskvine/src/bindings/python3/ndcctools/taskvine/graph_executor.py @@ -109,7 +109,7 @@ def __init__(self, self._create_library_task(libcores, hoisting_modules, libtask_env_files) def _create_library_task(self, libcores=1, hoisting_modules=[], libtask_env_files={}): - assert cvine.vine_task_graph_get_function_name(self._vine_task_graph) == compute_single_key.__name__ + assert cvine.vine_task_graph_get_proxy_function_name(self._vine_task_graph) == compute_single_key.__name__ self.task_graph_pkl_file_name = f"library-task-graph-{uuid.uuid4()}.pkl" self.task_graph_pkl_file_local_path = self.task_graph_pkl_file_name @@ -120,7 +120,7 @@ def _create_library_task(self, libcores=1, hoisting_modules=[], libtask_env_file if dask: hoisting_modules += [dask] self.libtask = self.create_library_from_functions( - cvine.vine_task_graph_get_library_name(self._vine_task_graph), + cvine.vine_task_graph_get_proxy_library_name(self._vine_task_graph), compute_single_key, library_context_info=[init_task_graph_context, [], {"task_graph_path": self.task_graph_pkl_file_remote_path}], add_env=False, @@ -194,7 +194,7 @@ def run(self, # we must finalize the graph in c side after all nodes and dependencies are added # this includes computing various metrics for each node, such as depth, height, heavy score, etc. - cvine.vine_task_graph_finalize_metrics(self._vine_task_graph) + cvine.vine_task_graph_compute_topology_metrics(self._vine_task_graph) # then we can use the heavy score to sort the nodes and specify their outfile remote names heavy_scores = {} @@ -212,7 +212,9 @@ def run(self, choice = "shared-file-system" else: choice = "temp" + # set on the Python side, will be installed on the remote workers self.task_graph.set_outfile_type_of(k, choice) + # set on the C side, so the manager knows where the data is stored outfile_type_str = f"NODE_OUTFILE_TYPE_{choice.upper().replace('-', '_')}" cvine.vine_task_graph_set_node_outfile(self._vine_task_graph, self.task_graph.vine_key_of[k], diff --git a/taskvine/src/manager/taskvine.h b/taskvine/src/manager/taskvine.h index 4ea0258407..ce4311d73d 100644 --- a/taskvine/src/manager/taskvine.h +++ b/taskvine/src/manager/taskvine.h @@ -1639,19 +1639,19 @@ void vine_task_graph_set_node_outfile(struct vine_task_graph *tg, const char *no /** Finalize the metrics of the task graph. @param tg Reference to the task graph object. */ -void vine_task_graph_finalize_metrics(struct vine_task_graph *tg); +void vine_task_graph_compute_topology_metrics(struct vine_task_graph *tg); /** Get the library name of the task graph. @param tg Reference to the task graph object. @return The library name. */ -const char *vine_task_graph_get_library_name(const struct vine_task_graph *tg); +const char *vine_task_graph_get_proxy_library_name(const struct vine_task_graph *tg); /** Get the function name of the task graph. @param tg Reference to the task graph object. @return The function name. */ -const char *vine_task_graph_get_function_name(const struct vine_task_graph *tg); +const char *vine_task_graph_get_proxy_function_name(const struct vine_task_graph *tg); /** Get the heavy score of a node in the task graph. @param tg Reference to the task graph object. diff --git a/taskvine/src/manager/vine_task_graph.c b/taskvine/src/manager/vine_task_graph.c index c6fdcb24b6..5d5c75b910 100644 --- a/taskvine/src/manager/vine_task_graph.c +++ b/taskvine/src/manager/vine_task_graph.c @@ -1,6 +1,7 @@ #include "vine_task_graph.h" #include "taskvine.h" #include "vine_manager.h" +#include "vine_task_node.h" #include "vine_worker_info.h" #include "priority_queue.h" #include @@ -9,7 +10,6 @@ #include "debug.h" #include "stringtools.h" #include "xxmalloc.h" -#include "priority_queue.h" #include #include "hash_table.h" #include @@ -46,13 +46,6 @@ static void submit_node_task(struct vine_task_graph *tg, struct vine_task_node * int task_id = vine_task_node_submit(node); itable_insert(tg->task_id_to_node, task_id, node); - node->retry_attempts_left--; - if (node->retry_attempts_left < 0) { - debug(D_ERROR, "Aborting, node %s has exhausted all retry attempts", node->node_key); - vine_task_graph_delete(tg); - exit(1); - } - return; } @@ -68,8 +61,11 @@ static void submit_node_ready_children(struct vine_task_graph *tg, struct vine_t /* Remove this parent from the child's pending set if it exists */ if (child_node->pending_parents) { /* Assert that this parent is indeed pending for the child */ - assert(set_lookup(child_node->pending_parents, node)); - set_remove(child_node->pending_parents, node); + if (child_node->pending_parents && set_lookup(child_node->pending_parents, node)) { + set_remove(child_node->pending_parents, node); + } else { + debug(D_ERROR, "inconsistent pending set: child=%s missing parent=%s", child_node->node_key, node->node_key); + } } /* If no more parents are pending, submit the child */ @@ -141,6 +137,8 @@ static struct list *get_topological_order(struct vine_task_graph *tg) } list_delete(topo_order); + hash_table_delete(in_degree_map); + priority_queue_delete(pq); exit(1); } @@ -251,9 +249,19 @@ void vine_task_graph_execute(struct vine_task_graph *tg) if (!tg) { return; } - + + debug(D_VINE, "start executing task graph"); + + /* print the info of all nodes */ char *node_key; struct vine_task_node *node; + HASH_TABLE_ITERATE(tg->nodes, node_key, node) + { + vine_task_node_print_info(node); + } + + /* enable return recovery tasks */ + vine_enable_return_recovery_tasks(tg->manager); /* create mapping from task_id and outfile cached_name to node */ HASH_TABLE_ITERATE(tg->nodes, node_key, node) @@ -289,9 +297,6 @@ void vine_task_graph_execute(struct vine_task_graph *tg) } } - /* enable return recovery tasks */ - vine_enable_return_recovery_tasks(tg->manager); - /* enqueue those without dependencies */ HASH_TABLE_ITERATE(tg->nodes, node_key, node) { @@ -334,7 +339,15 @@ void vine_task_graph_execute(struct vine_task_graph *tg) /* in case of failure, resubmit this task */ if (node->task->result != VINE_RESULT_SUCCESS || node->task->exit_code != 0) { - debug(D_VINE | D_NOTICE, "Task %d failed with result %d and exit code %d, resubmitting...", task->task_id, node->task->result, node->task->exit_code); + if (node->retry_attempts_left <= 0) { + debug(D_ERROR, "Task %d failed (result=%d, exit=%d). Node %s has no retries left. Aborting.", + task->task_id, node->task->result, node->task->exit_code, node->node_key); + vine_task_graph_delete(tg); + exit(1); + } + node->retry_attempts_left--; + debug(D_VINE | D_NOTICE, "Task %d failed (result=%d, exit=%d). Retrying node %s (remaining=%d)...", + task->task_id, node->task->result, node->task->exit_code, node->node_key, node->retry_attempts_left); vine_task_reset(node->task); submit_node_task(tg, node); continue; @@ -346,7 +359,15 @@ void vine_task_graph_execute(struct vine_task_graph *tg) struct stat info; int result = stat(node->outfile_remote_name, &info); if (result < 0) { - debug(D_VINE | D_NOTICE, "Task %d succeeded but output file %s does not exist on the shared file system", task->task_id, node->outfile_remote_name); + if (node->retry_attempts_left <= 0) { + debug(D_ERROR, "Task %d succeeded but missing sharedfs output %s; no retries left for node %s. Aborting.", + task->task_id, node->outfile_remote_name, node->node_key); + vine_task_graph_delete(tg); + exit(1); + } + node->retry_attempts_left--; + debug(D_VINE | D_NOTICE, "Task %d succeeded but missing sharedfs output %s; retrying node %s (remaining=%d)...", + task->task_id, node->outfile_remote_name, node->node_key, node->retry_attempts_left); vine_task_reset(node->task); submit_node_task(tg, node); continue; @@ -407,7 +428,7 @@ void vine_task_graph_execute(struct vine_task_graph *tg) submit_node_ready_children(tg, node); } else { wait_timeout = 2; - progress_bar_update_part(pbar, recovery_tasks_part, 0); + progress_bar_update_part(pbar, recovery_tasks_part, 0); // refresh the time and total for recovery tasks } } @@ -434,7 +455,7 @@ void vine_task_graph_execute(struct vine_task_graph *tg) return; } -void vine_task_graph_finalize_metrics(struct vine_task_graph *tg) +void vine_task_graph_compute_topology_metrics(struct vine_task_graph *tg) { if (!tg) { return; @@ -531,6 +552,7 @@ void vine_task_graph_finalize_metrics(struct vine_task_graph *tg) 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); @@ -547,8 +569,8 @@ struct vine_task_graph *vine_task_graph_create(struct vine_manager *q) tg->task_id_to_node = itable_create(0); tg->outfile_cachename_to_node = hash_table_create(0, 0); - tg->library_name = xxstrdup("vine_task_graph_library"); - tg->function_name = xxstrdup("compute_single_key"); + tg->proxy_library_name = xxstrdup("vine_task_graph_library"); + tg->proxy_function_name = xxstrdup("compute_single_key"); tg->manager = q; tg->failure_injection_step_percent = -1.0; @@ -593,8 +615,8 @@ struct vine_task_node *vine_task_graph_create_node( if (!node) { node = vine_task_node_create(tg->manager, node_key, - tg->library_name, - tg->function_name, + tg->proxy_library_name, + tg->proxy_function_name, staging_dir, prune_depth, priority_mode); @@ -604,6 +626,13 @@ struct vine_task_node *vine_task_graph_create_node( return node; } +/** + * Add a dependency between two nodes in the task graph. Note that the input-output file relationship + * is not handled here, because their file names may have not been determined yet. + * @param tg Reference to the task graph object. + * @param parent_key Reference to the parent node key. + * @param child_key Reference to the child node key. + */ void vine_task_graph_add_dependency(struct vine_task_graph *tg, const char *parent_key, const char *child_key) { if (!tg || !parent_key || !child_key) { @@ -623,25 +652,44 @@ void vine_task_graph_add_dependency(struct vine_task_graph *tg, const char *pare list_push_tail(child_node->parents, parent_node); list_push_tail(parent_node->children, child_node); - debug(D_VINE, "added dependency: %s -> %s", parent_key, child_key); + + return; } -const char *vine_task_graph_get_library_name(const struct vine_task_graph *tg) +/** + * Get the library name of the task graph. + * @param tg Reference to the task graph object. + * @return The library name. + */ +const char *vine_task_graph_get_proxy_library_name(const struct vine_task_graph *tg) { if (!tg) { return NULL; } - return tg->library_name; + + return tg->proxy_library_name; } -const char *vine_task_graph_get_function_name(const struct vine_task_graph *tg) +/** + * Get the function name of the task graph. + * @param tg Reference to the task graph object. + * @return The function name. + */ +const char *vine_task_graph_get_proxy_function_name(const struct vine_task_graph *tg) { if (!tg) { return NULL; } - return tg->function_name; + + return tg->proxy_function_name; } +/** + * Get the heavy score of a node in the task graph. + * @param tg Reference to the task graph object. + * @param node_key Reference to the node key. + * @return The heavy score. + */ double vine_task_graph_get_node_heavy_score(const struct vine_task_graph *tg, const char *node_key) { if (!tg) { @@ -656,6 +704,13 @@ double vine_task_graph_get_node_heavy_score(const struct vine_task_graph *tg, co return node->heavy_score; } +/** + * Get the local outfile source of a node in the task graph, only valid for local output files. + * The source of a local output file is the path on the local filesystem. + * @param tg Reference to the task graph object. + * @param node_key Reference to the node key. + * @return The local outfile source. + */ const char *vine_task_graph_get_node_local_outfile_source(const struct vine_task_graph *tg, const char *node_key) { if (!tg || !node_key) { @@ -676,6 +731,13 @@ const char *vine_task_graph_get_node_local_outfile_source(const struct vine_task return node->outfile->source; } +/** + * Set the outfile of a node in the task graph. This involves declaring the output file and adding it to the task. + * @param tg Reference to the task graph object. + * @param node_key Reference to the node key. + * @param outfile_type Reference to the outfile type. + * @param outfile_remote_name Reference to the outfile remote name that the task will generate remotely. + */ void vine_task_graph_set_node_outfile(struct vine_task_graph *tg, const char *node_key, vine_task_node_outfile_type_t outfile_type, const char *outfile_remote_name) { if (!tg || !node_key || !outfile_remote_name) { @@ -692,6 +754,10 @@ void vine_task_graph_set_node_outfile(struct vine_task_graph *tg, const char *no return; } +/** + * Delete a task graph object. + * @param tg Reference to the task graph object. + */ void vine_task_graph_delete(struct vine_task_graph *tg) { if (!tg) { @@ -716,8 +782,8 @@ void vine_task_graph_delete(struct vine_task_graph *tg) vine_delete(tg->manager); - free(tg->library_name); - free(tg->function_name); + free(tg->proxy_library_name); + free(tg->proxy_function_name); hash_table_delete(tg->nodes); itable_delete(tg->task_id_to_node); diff --git a/taskvine/src/manager/vine_task_graph.h b/taskvine/src/manager/vine_task_graph.h index 1692d1fc9e..a3ed8d3c6b 100644 --- a/taskvine/src/manager/vine_task_graph.h +++ b/taskvine/src/manager/vine_task_graph.h @@ -14,8 +14,8 @@ struct vine_task_graph { struct itable *task_id_to_node; struct hash_table *outfile_cachename_to_node; - char *library_name; - char *function_name; + char *proxy_library_name; + char *proxy_function_name; double failure_injection_step_percent; // 0 - 100, the percentage of steps to inject failure }; diff --git a/taskvine/src/manager/vine_task_node.c b/taskvine/src/manager/vine_task_node.c index 4970b70410..aa6d38bc9b 100644 --- a/taskvine/src/manager/vine_task_node.c +++ b/taskvine/src/manager/vine_task_node.c @@ -44,13 +44,13 @@ double compute_lex_priority(const char *key) struct vine_task_node *vine_task_node_create( struct vine_manager *manager, const char *node_key, - const char *library_name, - const char *function_name, + const char *proxy_library_name, + const char *proxy_function_name, const char *staging_dir, int prune_depth, vine_task_node_priority_mode_t priority_mode) { - if (!manager || !node_key || !library_name || !function_name || !staging_dir) { + if (!manager || !node_key || !proxy_library_name || !proxy_function_name || !staging_dir) { return NULL; } @@ -84,27 +84,25 @@ struct vine_task_node *vine_task_node_create( node->critical_time = -1; - debug(D_VINE, "node info: key=%s, staging_dir=%s, priority_mode=%d, prune_depth=%d", node->node_key, node->staging_dir, node->priority_mode, node->prune_depth); - /* create the task */ - node->task = vine_task_create(function_name); - vine_task_set_library_required(node->task, library_name); + node->task = vine_task_create(proxy_function_name); + vine_task_set_library_required(node->task, proxy_library_name); vine_task_addref(node->task); - + /* build JSON infile expected by library: {"fn_args": [key], "fn_kwargs": {}} */ struct jx *event = jx_object(NULL); struct jx *args = jx_array(NULL); jx_array_append(args, jx_string(node->node_key)); jx_insert(event, jx_string("fn_args"), args); jx_insert(event, jx_string("fn_kwargs"), jx_object(NULL)); - + char *infile_content = jx_print_string(event); jx_delete(event); - + node->infile = vine_declare_buffer(node->manager, infile_content, strlen(infile_content), VINE_CACHE_LEVEL_TASK, VINE_UNLINK_WHEN_DONE); free(infile_content); vine_task_add_input(node->task, node->infile, "infile", VINE_TRANSFER_ALWAYS); - + return node; } @@ -130,17 +128,14 @@ void vine_task_node_set_outfile(struct vine_task_node *node, vine_task_node_outf node->outfile = vine_declare_file(node->manager, local_output_path, VINE_CACHE_LEVEL_WORKFLOW, 0); free(local_output_dir); free(local_output_path); - debug(D_VINE, "node %s: outfile type = VINE_NODE_OUTFILE_TYPE_LOCAL, outfile = %s", node->node_key, node->outfile->cached_name); break; } case VINE_NODE_OUTFILE_TYPE_TEMP: node->outfile = vine_declare_temp(node->manager); - debug(D_VINE, "node %s: outfile type = VINE_NODE_OUTFILE_TYPE_TEMP, outfile = %s", node->node_key, node->outfile->cached_name); break; case VINE_NODE_OUTFILE_TYPE_SHARED_FILE_SYSTEM: /* no explicit output file declaration needed */ node->outfile = NULL; - debug(D_VINE, "node %s: outfile type = VINE_NODE_OUTFILE_TYPE_SHARED_FILE_SYSTEM", node->node_key); break; } if (node->outfile) { @@ -462,70 +457,81 @@ int vine_task_node_submit(struct vine_task_node *node) double priority = vine_task_node_calculate_priority(node); vine_task_set_priority(node->task, priority); - debug(D_VINE, "node %s: priority_mode=%d, depth=%d, calculated_priority=%.6f", node->node_key, node->priority_mode, node->depth, priority); - return vine_submit(node->manager, node->task); } +/* print the info of the node */ void vine_task_node_print_info(struct vine_task_node *node) { - if (!node) { - return; - } + if (!node) { + return; + } + + if (!node->task) { + debug(D_ERROR, "node %s has no task", node->node_key); + return; + } + + debug(D_VINE, "---------------- Node Info ----------------"); + debug(D_VINE, "key: %s", node->node_key); + debug(D_VINE, "task_id: %d", node->task->task_id); + debug(D_VINE, "staging_dir: %s", node->staging_dir ? node->staging_dir : "(null)"); + debug(D_VINE, "depth: %d", node->depth); + debug(D_VINE, "height: %d", node->height); + debug(D_VINE, "prune_depth: %d", node->prune_depth); + debug(D_VINE, "priority_mode: %d", node->priority_mode); - debug(D_VINE, "node info %s task_id: %d", node->node_key, node->task->task_id); - debug(D_VINE, "node info %s depth: %d", node->node_key, node->depth); if (node->outfile_remote_name) { - debug(D_VINE, "node info %s outfile remote name: %s", node->node_key, node->outfile_remote_name); + debug(D_VINE, "outfile_remote_name: %s", node->outfile_remote_name); } - if (node->outfile) { - switch (node->outfile->type) { - case VINE_FILE: - debug(D_VINE, "node info %s outfile type: VINE_FILE, cached name: %s", node->node_key, node->outfile->cached_name); - break; - case VINE_TEMP: - debug(D_VINE, "node info %s outfile type: VINE_TEMP, cached name: %s", node->node_key, node->outfile->cached_name); - break; - default: - debug(D_ERROR, "unsupported outfile type: %d", node->outfile->type); - break; - } - } + 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_keys = NULL; - struct vine_task_node *parent_node; - LIST_ITERATE(node->parents, parent_node) - { - if (!parent_keys) { - parent_keys = string_format("%s", parent_node->node_key); - } else { - char *tmp = string_format("%s, %s", parent_keys, parent_node->node_key); - free(parent_keys); - parent_keys = tmp; - } - } - debug(D_VINE, "node info %s parents: %s", node->node_key, parent_keys ? parent_keys : "(none)"); - free(parent_keys); - parent_keys = NULL; + /* print parent and child node keys */ + char *parent_keys = NULL; + struct vine_task_node *p; + LIST_ITERATE(node->parents, p) { + if (!parent_keys) { + parent_keys = string_format("%s", p->node_key); + } else { + char *tmp = string_format("%s, %s", parent_keys, p->node_key); + free(parent_keys); + parent_keys = tmp; + } + } - char *child_keys = NULL; - struct vine_task_node *child_node; - LIST_ITERATE(node->children, child_node) - { - if (!child_keys) { - child_keys = string_format("%s", child_node->node_key); - } else { - char *tmp = string_format("%s, %s", child_keys, child_node->node_key); - free(child_keys); - child_keys = tmp; - } - } - debug(D_VINE, "node info %s children: %s", node->node_key, child_keys ? child_keys : "(none)"); - free(child_keys); - child_keys = NULL; + char *child_keys = NULL; + struct vine_task_node *c; + LIST_ITERATE(node->children, c) { + if (!child_keys) { + child_keys = string_format("%s", c->node_key); + } else { + char *tmp = string_format("%s, %s", child_keys, c->node_key); + free(child_keys); + child_keys = tmp; + } + } - return; + debug(D_VINE, "parents: %s", parent_keys ? parent_keys : "(none)"); + debug(D_VINE, "children: %s", child_keys ? child_keys : "(none)"); + + free(parent_keys); + free(child_keys); + + debug(D_VINE, "-------------------------------------------"); } void vine_task_node_prune_ancestors(struct vine_task_node *node) diff --git a/taskvine/src/manager/vine_task_node.h b/taskvine/src/manager/vine_task_node.h index b480acfc7a..cd8e98afe3 100644 --- a/taskvine/src/manager/vine_task_node.h +++ b/taskvine/src/manager/vine_task_node.h @@ -54,8 +54,8 @@ struct vine_task_node { struct vine_task_node *vine_task_node_create( struct vine_manager *manager, const char *node_key, - const char *library_name, - const char *function_name, + const char *proxy_library_name, + const char *proxy_function_name, const char *staging_dir, int prune_depth, vine_task_node_priority_mode_t priority_mode From b4e5a7b0ee1e1b3ea9a33c4e19637a25607f3f4d Mon Sep 17 00:00:00 2001 From: Jin Zhou Date: Wed, 15 Oct 2025 11:48:22 -0400 Subject: [PATCH 031/113] comment functions --- .../ndcctools/taskvine/graph_executor.py | 10 +- taskvine/src/manager/taskvine.h | 2 +- taskvine/src/manager/vine_task_graph.c | 626 ++++++++++-------- taskvine/src/manager/vine_task_node.c | 306 ++++++--- 4 files changed, 551 insertions(+), 393 deletions(-) diff --git a/taskvine/src/bindings/python3/ndcctools/taskvine/graph_executor.py b/taskvine/src/bindings/python3/ndcctools/taskvine/graph_executor.py index 5d7137f25b..01e7cb8811 100644 --- a/taskvine/src/bindings/python3/ndcctools/taskvine/graph_executor.py +++ b/taskvine/src/bindings/python3/ndcctools/taskvine/graph_executor.py @@ -184,11 +184,11 @@ def run(self, # create task graph in the python side print("Initializing task graph in TaskVine") for k in topo_order: - cvine.vine_task_graph_create_node(self._vine_task_graph, - self.task_graph.vine_key_of[k], - self.staging_dir, - prune_depth, - get_c_constant(f"task_priority_mode_{priority_mode.replace('-', '_')}")) + cvine.vine_task_graph_add_node(self._vine_task_graph, + self.task_graph.vine_key_of[k], + self.staging_dir, + prune_depth, + get_c_constant(f"task_priority_mode_{priority_mode.replace('-', '_')}")) for pk in self.task_graph.parents_of.get(k, []): cvine.vine_task_graph_add_dependency(self._vine_task_graph, self.task_graph.vine_key_of[pk], self.task_graph.vine_key_of[k]) diff --git a/taskvine/src/manager/taskvine.h b/taskvine/src/manager/taskvine.h index ce4311d73d..ff75a8f6d2 100644 --- a/taskvine/src/manager/taskvine.h +++ b/taskvine/src/manager/taskvine.h @@ -1616,7 +1616,7 @@ void vine_task_graph_add_dependency(struct vine_task_graph *tg, const char *pare @param priority_mode Reference to the priority mode. @return A new node object. */ -struct vine_task_node *vine_task_graph_create_node(struct vine_task_graph *tg, +struct vine_task_node *vine_task_graph_add_node(struct vine_task_graph *tg, const char *node_key, const char *staging_dir, int prune_depth, diff --git a/taskvine/src/manager/vine_task_graph.c b/taskvine/src/manager/vine_task_graph.c index 5d5c75b910..5f64287d1c 100644 --- a/taskvine/src/manager/vine_task_graph.c +++ b/taskvine/src/manager/vine_task_graph.c @@ -32,11 +32,20 @@ static volatile sig_atomic_t interrupted = 0; /* Private Functions */ /*************************************************************/ +/** + * Handle the SIGINT signal. + * @param signal Reference to the signal. + */ static void handle_sigint(int signal) { interrupted = 1; } +/** + * Submit a node to the taskvine manager. + * @param tg Reference to the task graph object. + * @param node Reference to the node object. + */ static void submit_node_task(struct vine_task_graph *tg, struct vine_task_node *node) { if (!tg || !node) { @@ -49,7 +58,12 @@ static void submit_node_task(struct vine_task_graph *tg, struct vine_task_node * return; } -static void submit_node_ready_children(struct vine_task_graph *tg, struct vine_task_node *node) +/** + * Submit the children of a node if all its dependencies are resolved. + * @param tg Reference to the task graph object. + * @param node Reference to the node object. + */ +static void submit_unblocked_children(struct vine_task_graph *tg, struct vine_task_node *node) { if (!tg || !node) { return; @@ -77,6 +91,12 @@ static void submit_node_ready_children(struct vine_task_graph *tg, struct vine_t return; } +/** + * Get the topological order of the task graph. + * Must be called after all nodes and dependencies are added and the topology metrics are computed. + * @param tg Reference to the task graph object. + * @return The list of nodes in topological order. + */ static struct list *get_topological_order(struct vine_task_graph *tg) { if (!tg) { @@ -147,6 +167,12 @@ static struct list *get_topological_order(struct vine_task_graph *tg) return topo_order; } +/** + * Extract the weakly connected components of the task graph. + * This function is used only for debugging purposes at the moment. + * @param tg Reference to the task graph object. + * @return The list of weakly connected components. + */ static struct list *extract_weakly_connected_components(struct vine_task_graph *tg) { if (!tg) { @@ -203,6 +229,11 @@ static struct list *extract_weakly_connected_components(struct vine_task_graph * return components; } +/** + * Compute the heavy score of a node in the task graph. + * @param node Reference to the node object. + * @return The heavy score. + */ static double compute_node_heavy_score(struct vine_task_node *node) { if (!node) { @@ -215,6 +246,12 @@ static double compute_node_heavy_score(struct vine_task_node *node) return up_score / (down_score + 1); } +/** + * Map a task to a node in the task graph. + * @param tg Reference to the task graph object. + * @param task Reference to the task object. + * @return The node object. + */ static struct vine_task_node *get_node_by_task(struct vine_task_graph *tg, struct vine_task *task) { if (!tg || !task) { @@ -222,6 +259,7 @@ static struct vine_task_node *get_node_by_task(struct vine_task_graph *tg, struc } if (task->type == VINE_TASK_TYPE_STANDARD) { + /* standard tasks are mapped directly to a node */ return itable_lookup(tg->task_id_to_node, task->task_id); } else if (task->type == VINE_TASK_TYPE_RECOVERY) { /* note that recovery tasks are not mapped to any node but we still need the original node for pruning, @@ -244,217 +282,86 @@ static struct vine_task_node *get_node_by_task(struct vine_task_graph *tg, struc /* Public APIs */ /*************************************************************/ -void vine_task_graph_execute(struct vine_task_graph *tg) +/** + * Get the library name of the task graph. + * @param tg Reference to the task graph object. + * @return The library name. + */ +const char *vine_task_graph_get_proxy_library_name(const struct vine_task_graph *tg) { if (!tg) { - return; - } - - debug(D_VINE, "start executing task graph"); - - /* print the info of all nodes */ - char *node_key; - struct vine_task_node *node; - HASH_TABLE_ITERATE(tg->nodes, node_key, node) - { - vine_task_node_print_info(node); + return NULL; } - - /* enable return recovery tasks */ - vine_enable_return_recovery_tasks(tg->manager); - /* create mapping from task_id and outfile cached_name to node */ - HASH_TABLE_ITERATE(tg->nodes, node_key, node) - { - if (node->outfile) { - hash_table_insert(tg->outfile_cachename_to_node, node->outfile->cached_name, node); - } - } + return tg->proxy_library_name; +} - /* add the parents' outfiles as inputs to the task */ - struct list *topo_order = get_topological_order(tg); - LIST_ITERATE(topo_order, node) - { - struct vine_task_node *parent_node; - LIST_ITERATE(node->parents, parent_node) - { - if (parent_node->outfile) { - vine_task_add_input(node->task, parent_node->outfile, parent_node->outfile_remote_name, VINE_TRANSFER_ALWAYS); - } - } +/** + * Get the function name of the task graph. + * @param tg Reference to the task graph object. + * @return The function name. + */ +const char *vine_task_graph_get_proxy_function_name(const struct vine_task_graph *tg) +{ + if (!tg) { + return NULL; } - /* initialize pending_parents for all nodes */ - HASH_TABLE_ITERATE(tg->nodes, node_key, node) - { - struct vine_task_node *parent_node; - LIST_ITERATE(node->parents, parent_node) - { - if (node->pending_parents) { - /* Use parent_node->node_key to ensure pointer consistency */ - set_insert(node->pending_parents, parent_node); - } - } - } + return tg->proxy_function_name; +} - /* enqueue those without dependencies */ - HASH_TABLE_ITERATE(tg->nodes, node_key, node) - { - if (!node->pending_parents || set_size(node->pending_parents) == 0) { - submit_node_task(tg, node); - } +/** + * Get the heavy score of a node in the task graph. + * @param tg Reference to the task graph object. + * @param node_key Reference to the node key. + * @return The heavy score. + */ +double vine_task_graph_get_node_heavy_score(const struct vine_task_graph *tg, const char *node_key) +{ + if (!tg) { + return -1; } - /* calculate steps to inject failure */ - double next_failure_threshold = -1.0; - if (tg->failure_injection_step_percent > 0) { - next_failure_threshold = tg->failure_injection_step_percent / 100.0; + struct vine_task_node *node = hash_table_lookup(tg->nodes, node_key); + if (!node) { + return -1; } - struct ProgressBar *pbar = progress_bar_init("Executing Tasks"); - struct ProgressBarPart *regular_tasks_part = progress_bar_create_part("Regular", hash_table_size(tg->nodes)); - struct ProgressBarPart *recovery_tasks_part = progress_bar_create_part("Recovery", 0); - progress_bar_bind_part(pbar, regular_tasks_part); - progress_bar_bind_part(pbar, recovery_tasks_part); - - int wait_timeout = 2; - - while (regular_tasks_part->current < regular_tasks_part->total) { - if (interrupted) { - break; - } - - struct vine_task *task = vine_wait(tg->manager, wait_timeout); - progress_bar_set_part_total(pbar, recovery_tasks_part, tg->manager->num_submitted_recovery_tasks); - if (task) { - /* retrieve all possible tasks */ - wait_timeout = 0; - - /* get the original node by task id */ - struct vine_task_node *node = get_node_by_task(tg, 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); - } - - /* in case of failure, resubmit this task */ - if (node->task->result != VINE_RESULT_SUCCESS || node->task->exit_code != 0) { - if (node->retry_attempts_left <= 0) { - debug(D_ERROR, "Task %d failed (result=%d, exit=%d). Node %s has no retries left. Aborting.", - task->task_id, node->task->result, node->task->exit_code, node->node_key); - vine_task_graph_delete(tg); - exit(1); - } - node->retry_attempts_left--; - debug(D_VINE | D_NOTICE, "Task %d failed (result=%d, exit=%d). Retrying node %s (remaining=%d)...", - task->task_id, node->task->result, node->task->exit_code, node->node_key, node->retry_attempts_left); - vine_task_reset(node->task); - submit_node_task(tg, node); - continue; - } - - /* if the outfile is set to save on the sharedfs, stat to get the size of the file */ - switch (node->outfile_type) { - case VINE_NODE_OUTFILE_TYPE_SHARED_FILE_SYSTEM: { - struct stat info; - int result = stat(node->outfile_remote_name, &info); - if (result < 0) { - if (node->retry_attempts_left <= 0) { - debug(D_ERROR, "Task %d succeeded but missing sharedfs output %s; no retries left for node %s. Aborting.", - task->task_id, node->outfile_remote_name, node->node_key); - vine_task_graph_delete(tg); - exit(1); - } - node->retry_attempts_left--; - debug(D_VINE | D_NOTICE, "Task %d succeeded but missing sharedfs output %s; retrying node %s (remaining=%d)...", - task->task_id, node->outfile_remote_name, node->node_key, node->retry_attempts_left); - vine_task_reset(node->task); - submit_node_task(tg, node); - continue; - } - node->outfile_size_bytes = info.st_size; - break; - } - case VINE_NODE_OUTFILE_TYPE_LOCAL: - case VINE_NODE_OUTFILE_TYPE_TEMP: - node->outfile_size_bytes = node->outfile->size; - break; - } - debug(D_VINE, "Node %s completed with outfile %s size: %zu bytes", node->node_key, node->outfile_remote_name, node->outfile_size_bytes); - - /* mark the node as completed */ - node->completed = 1; - - /* prune nodes on task completion */ - vine_task_node_prune_ancestors(node); - - /* skip recovery tasks */ - if (task->type == VINE_TASK_TYPE_RECOVERY) { - progress_bar_update_part(pbar, recovery_tasks_part, 1); - continue; - } - - /* set the start time to the submit time of the first regular task */ - if (regular_tasks_part->current == 0) { - progress_bar_set_start_time(pbar, task->time_when_commit_start); - } - - /* update critical time */ - vine_task_node_update_critical_time(node, task->time_workers_execute_last); - - /* mark this regular task as completed */ - progress_bar_update_part(pbar, regular_tasks_part, 1); - - /* inject failure */ - if (tg->failure_injection_step_percent > 0) { - double progress = (double)regular_tasks_part->current / (double)regular_tasks_part->total; - if (progress >= next_failure_threshold && evict_random_worker(tg->manager)) { - debug(D_VINE, "evicted a worker at %.2f%% (threshold %.2f%%)", progress * 100, next_failure_threshold * 100); - next_failure_threshold += tg->failure_injection_step_percent / 100.0; - } - } - - /* enqueue the output file for replication */ - switch (node->outfile_type) { - case VINE_NODE_OUTFILE_TYPE_TEMP: - vine_task_node_replicate_outfile(node); - break; - case VINE_NODE_OUTFILE_TYPE_LOCAL: - case VINE_NODE_OUTFILE_TYPE_SHARED_FILE_SYSTEM: - break; - } + return node->heavy_score; +} - /* submit children nodes with dependencies all resolved */ - submit_node_ready_children(tg, node); - } else { - wait_timeout = 2; - progress_bar_update_part(pbar, recovery_tasks_part, 0); // refresh the time and total for recovery tasks - } +/** + * Get the local outfile source of a node in the task graph, only valid for local output files. + * The source of a local output file is the path on the local filesystem. + * @param tg Reference to the task graph object. + * @param node_key Reference to the node key. + * @return The local outfile source. + */ +const char *vine_task_graph_get_node_local_outfile_source(const struct vine_task_graph *tg, const char *node_key) +{ + if (!tg || !node_key) { + return NULL; } - progress_bar_finish(pbar); - progress_bar_delete(pbar); - - double total_time_spent_on_unlink_local_files = 0; - double total_time_spent_on_prune_ancestors_of_temp_node = 0; - double total_time_spent_on_prune_ancestors_of_persisted_node = 0; - HASH_TABLE_ITERATE(tg->nodes, node_key, node) - { - total_time_spent_on_unlink_local_files += node->time_spent_on_unlink_local_files; - total_time_spent_on_prune_ancestors_of_temp_node += node->time_spent_on_prune_ancestors_of_temp_node; - total_time_spent_on_prune_ancestors_of_persisted_node += node->time_spent_on_prune_ancestors_of_persisted_node; + struct vine_task_node *node = hash_table_lookup(tg->nodes, node_key); + if (!node) { + debug(D_ERROR, "node %s not found", node_key); + exit(1); } - total_time_spent_on_unlink_local_files /= 1e6; - total_time_spent_on_prune_ancestors_of_temp_node /= 1e6; - total_time_spent_on_prune_ancestors_of_persisted_node /= 1e6; - debug(D_VINE, "total time spent on prune ancestors of temp node: %.6f seconds\n", total_time_spent_on_prune_ancestors_of_temp_node); - debug(D_VINE, "total time spent on prune ancestors of persisted node: %.6f seconds\n", total_time_spent_on_prune_ancestors_of_persisted_node); - debug(D_VINE, "total time spent on unlink local files: %.6f seconds\n", total_time_spent_on_unlink_local_files); + if (node->outfile_type != VINE_NODE_OUTFILE_TYPE_LOCAL) { + debug(D_ERROR, "node %s is not a local output file", node_key); + exit(1); + } - return; + return node->outfile->source; } +/** + * Compute the topology metrics of the task graph, including depth, height, upstream and downstream counts, + * heavy scores, and weakly connected components. + * @param tg Reference to the task graph object. + */ void vine_task_graph_compute_topology_metrics(struct vine_task_graph *tg) { if (!tg) { @@ -538,7 +445,7 @@ void vine_task_graph_compute_topology_metrics(struct vine_task_graph *tg) hash_table_delete(upstream_map); hash_table_delete(downstream_map); - /* compute the heavy score */ + /* compute the heavy score for each node */ LIST_ITERATE(topo_order, node) { node->heavy_score = compute_node_heavy_score(node); @@ -562,6 +469,47 @@ void vine_task_graph_compute_topology_metrics(struct vine_task_graph *tg) return; } +/** + * Create a new node and track it in the task graph. + * @param tg Reference to the task graph object. + * @param node_key Reference to the node key. + * @param staging_dir Reference to the staging directory. + * @param prune_depth Reference to the prune depth. + * @param priority_mode Reference to the priority mode. + * @return A new node object. + */ +struct vine_task_node *vine_task_graph_add_node( + struct vine_task_graph *tg, + const char *node_key, + const char *staging_dir, + int prune_depth, + vine_task_node_priority_mode_t priority_mode) +{ + if (!tg || !node_key) { + return NULL; + } + + /* if the node already exists, skip creating a new one */ + struct vine_task_node *node = hash_table_lookup(tg->nodes, node_key); + if (!node) { + node = vine_task_node_create(tg->manager, + node_key, + tg->proxy_library_name, + tg->proxy_function_name, + staging_dir, + prune_depth, + priority_mode); + hash_table_insert(tg->nodes, node_key, node); + } + + return node; +} + +/** + * Create a new task graph object and bind a manager to it. + * @param q Reference to the manager object. + * @return A new task graph object. + */ struct vine_task_graph *vine_task_graph_create(struct vine_manager *q) { struct vine_task_graph *tg = xxmalloc(sizeof(struct vine_task_graph)); @@ -586,6 +534,11 @@ struct vine_task_graph *vine_task_graph_create(struct vine_manager *q) return tg; } +/** + * Set the failure injection step percent, meaning we will evict a randome worker at every X% of the DAG completion. + * @param tg Reference to the task graph object. + * @param percent Reference to the failure injection step percent. + */ void vine_task_graph_set_failure_injection_step_percent(struct vine_task_graph *tg, double percent) { if (!tg) { @@ -600,32 +553,6 @@ void vine_task_graph_set_failure_injection_step_percent(struct vine_task_graph * tg->failure_injection_step_percent = percent; } -struct vine_task_node *vine_task_graph_create_node( - struct vine_task_graph *tg, - const char *node_key, - const char *staging_dir, - int prune_depth, - vine_task_node_priority_mode_t priority_mode) -{ - if (!tg || !node_key) { - return NULL; - } - - struct vine_task_node *node = hash_table_lookup(tg->nodes, node_key); - if (!node) { - node = vine_task_node_create(tg->manager, - node_key, - tg->proxy_library_name, - tg->proxy_function_name, - staging_dir, - prune_depth, - priority_mode); - hash_table_insert(tg->nodes, node_key, node); - } - - return node; -} - /** * Add a dependency between two nodes in the task graph. Note that the input-output file relationship * is not handled here, because their file names may have not been determined yet. @@ -657,99 +584,236 @@ void vine_task_graph_add_dependency(struct vine_task_graph *tg, const char *pare } /** - * Get the library name of the task graph. + * Set the outfile of a node in the task graph. + * This involves declaring the output file and adding it to the task. * @param tg Reference to the task graph object. - * @return The library name. + * @param node_key Reference to the node key. + * @param outfile_type Reference to the outfile type. + * @param outfile_remote_name Reference to the outfile remote name that the task will generate remotely. */ -const char *vine_task_graph_get_proxy_library_name(const struct vine_task_graph *tg) +void vine_task_graph_set_node_outfile(struct vine_task_graph *tg, const char *node_key, vine_task_node_outfile_type_t outfile_type, const char *outfile_remote_name) { - if (!tg) { - return NULL; + if (!tg || !node_key || !outfile_remote_name) { + return; } - return tg->proxy_library_name; + struct vine_task_node *node = hash_table_lookup(tg->nodes, node_key); + if (!node) { + return; + } + + vine_task_node_set_outfile(node, outfile_type, outfile_remote_name); + + return; } /** - * Get the function name of the task graph. + * Execute the task graph. This must be called after all nodes and dependencies are added and the topology metrics are computed. * @param tg Reference to the task graph object. - * @return The function name. */ -const char *vine_task_graph_get_proxy_function_name(const struct vine_task_graph *tg) +void vine_task_graph_execute(struct vine_task_graph *tg) { if (!tg) { - return NULL; + return; } - return tg->proxy_function_name; -} + debug(D_VINE, "start executing task graph"); -/** - * Get the heavy score of a node in the task graph. - * @param tg Reference to the task graph object. - * @param node_key Reference to the node key. - * @return The heavy score. - */ -double vine_task_graph_get_node_heavy_score(const struct vine_task_graph *tg, const char *node_key) -{ - if (!tg) { - return -1; + /* print the info of all nodes */ + char *node_key; + struct vine_task_node *node; + HASH_TABLE_ITERATE(tg->nodes, node_key, node) + { + vine_task_node_print_info(node); } - struct vine_task_node *node = hash_table_lookup(tg->nodes, node_key); - if (!node) { - return -1; + /* enable return recovery tasks */ + vine_enable_return_recovery_tasks(tg->manager); + + /* create mapping from task_id and outfile cached_name to node */ + HASH_TABLE_ITERATE(tg->nodes, node_key, node) + { + if (node->outfile) { + hash_table_insert(tg->outfile_cachename_to_node, node->outfile->cached_name, node); + } } - return node->heavy_score; -} + /* add the parents' outfiles as inputs to the task */ + struct list *topo_order = get_topological_order(tg); + LIST_ITERATE(topo_order, node) + { + struct vine_task_node *parent_node; + LIST_ITERATE(node->parents, parent_node) + { + if (parent_node->outfile) { + vine_task_add_input(node->task, parent_node->outfile, parent_node->outfile_remote_name, VINE_TRANSFER_ALWAYS); + } + } + } -/** - * Get the local outfile source of a node in the task graph, only valid for local output files. - * The source of a local output file is the path on the local filesystem. - * @param tg Reference to the task graph object. - * @param node_key Reference to the node key. - * @return The local outfile source. - */ -const char *vine_task_graph_get_node_local_outfile_source(const struct vine_task_graph *tg, const char *node_key) -{ - if (!tg || !node_key) { - return NULL; + /* initialize pending_parents for all nodes */ + HASH_TABLE_ITERATE(tg->nodes, node_key, node) + { + struct vine_task_node *parent_node; + LIST_ITERATE(node->parents, parent_node) + { + if (node->pending_parents) { + /* Use parent_node->node_key to ensure pointer consistency */ + set_insert(node->pending_parents, parent_node); + } + } } - struct vine_task_node *node = hash_table_lookup(tg->nodes, node_key); - if (!node) { - debug(D_ERROR, "node %s not found", node_key); - exit(1); + /* enqueue those without dependencies */ + HASH_TABLE_ITERATE(tg->nodes, node_key, node) + { + if (!node->pending_parents || set_size(node->pending_parents) == 0) { + submit_node_task(tg, node); + } } - if (node->outfile_type != VINE_NODE_OUTFILE_TYPE_LOCAL) { - debug(D_ERROR, "node %s is not a local output file", node_key); - exit(1); + /* calculate steps to inject failure */ + double next_failure_threshold = -1.0; + if (tg->failure_injection_step_percent > 0) { + next_failure_threshold = tg->failure_injection_step_percent / 100.0; } - return node->outfile->source; -} + struct ProgressBar *pbar = progress_bar_init("Executing Tasks"); + struct ProgressBarPart *regular_tasks_part = progress_bar_create_part("Regular", hash_table_size(tg->nodes)); + struct ProgressBarPart *recovery_tasks_part = progress_bar_create_part("Recovery", 0); + progress_bar_bind_part(pbar, regular_tasks_part); + progress_bar_bind_part(pbar, recovery_tasks_part); -/** - * Set the outfile of a node in the task graph. This involves declaring the output file and adding it to the task. - * @param tg Reference to the task graph object. - * @param node_key Reference to the node key. - * @param outfile_type Reference to the outfile type. - * @param outfile_remote_name Reference to the outfile remote name that the task will generate remotely. - */ -void vine_task_graph_set_node_outfile(struct vine_task_graph *tg, const char *node_key, vine_task_node_outfile_type_t outfile_type, const char *outfile_remote_name) -{ - if (!tg || !node_key || !outfile_remote_name) { - return; + int wait_timeout = 2; + + while (regular_tasks_part->current < regular_tasks_part->total) { + if (interrupted) { + break; + } + + struct vine_task *task = vine_wait(tg->manager, wait_timeout); + progress_bar_set_part_total(pbar, recovery_tasks_part, tg->manager->num_submitted_recovery_tasks); + if (task) { + /* retrieve all possible tasks */ + wait_timeout = 0; + + /* get the original node by task id */ + struct vine_task_node *node = get_node_by_task(tg, 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); + } + + /* in case of failure, resubmit this task */ + if (node->task->result != VINE_RESULT_SUCCESS || node->task->exit_code != 0) { + if (node->retry_attempts_left <= 0) { + debug(D_ERROR, "Task %d failed (result=%d, exit=%d). Node %s has no retries left. Aborting.", task->task_id, node->task->result, node->task->exit_code, node->node_key); + vine_task_graph_delete(tg); + exit(1); + } + node->retry_attempts_left--; + debug(D_VINE | D_NOTICE, "Task %d failed (result=%d, exit=%d). Retrying node %s (remaining=%d)...", task->task_id, node->task->result, node->task->exit_code, node->node_key, node->retry_attempts_left); + vine_task_reset(node->task); + submit_node_task(tg, node); + continue; + } + + /* if the outfile is set to save on the sharedfs, stat to get the size of the file */ + switch (node->outfile_type) { + case VINE_NODE_OUTFILE_TYPE_SHARED_FILE_SYSTEM: { + struct stat info; + int result = stat(node->outfile_remote_name, &info); + if (result < 0) { + if (node->retry_attempts_left <= 0) { + debug(D_ERROR, "Task %d succeeded but missing sharedfs output %s; no retries left for node %s. Aborting.", task->task_id, node->outfile_remote_name, node->node_key); + vine_task_graph_delete(tg); + exit(1); + } + node->retry_attempts_left--; + debug(D_VINE | D_NOTICE, "Task %d succeeded but missing sharedfs output %s; retrying node %s (remaining=%d)...", task->task_id, node->outfile_remote_name, node->node_key, node->retry_attempts_left); + vine_task_reset(node->task); + submit_node_task(tg, node); + continue; + } + node->outfile_size_bytes = info.st_size; + break; + } + case VINE_NODE_OUTFILE_TYPE_LOCAL: + case VINE_NODE_OUTFILE_TYPE_TEMP: + node->outfile_size_bytes = node->outfile->size; + break; + } + debug(D_VINE, "Node %s completed with outfile %s size: %zu bytes", node->node_key, node->outfile_remote_name, node->outfile_size_bytes); + + /* mark the node as completed */ + node->completed = 1; + + /* prune nodes on task completion */ + vine_task_node_prune_ancestors(node); + + /* skip recovery tasks */ + if (task->type == VINE_TASK_TYPE_RECOVERY) { + progress_bar_update_part(pbar, recovery_tasks_part, 1); + continue; + } + + /* set the start time to the submit time of the first regular task */ + if (regular_tasks_part->current == 0) { + progress_bar_set_start_time(pbar, task->time_when_commit_start); + } + + /* update critical time */ + vine_task_node_update_critical_time(node, task->time_workers_execute_last); + + /* mark this regular task as completed */ + progress_bar_update_part(pbar, regular_tasks_part, 1); + + /* inject failure */ + if (tg->failure_injection_step_percent > 0) { + double progress = (double)regular_tasks_part->current / (double)regular_tasks_part->total; + if (progress >= next_failure_threshold && evict_random_worker(tg->manager)) { + debug(D_VINE, "evicted a worker at %.2f%% (threshold %.2f%%)", progress * 100, next_failure_threshold * 100); + next_failure_threshold += tg->failure_injection_step_percent / 100.0; + } + } + + /* enqueue the output file for replication */ + switch (node->outfile_type) { + case VINE_NODE_OUTFILE_TYPE_TEMP: + vine_task_node_replicate_outfile(node); + break; + case VINE_NODE_OUTFILE_TYPE_LOCAL: + case VINE_NODE_OUTFILE_TYPE_SHARED_FILE_SYSTEM: + break; + } + + /* submit children nodes with dependencies all resolved */ + submit_unblocked_children(tg, node); + } else { + wait_timeout = 2; + progress_bar_update_part(pbar, recovery_tasks_part, 0); // refresh the time and total for recovery tasks + } } - struct vine_task_node *node = hash_table_lookup(tg->nodes, node_key); - if (!node) { - return; + progress_bar_finish(pbar); + progress_bar_delete(pbar); + + double total_time_spent_on_unlink_local_files = 0; + double total_time_spent_on_prune_ancestors_of_temp_node = 0; + double total_time_spent_on_prune_ancestors_of_persisted_node = 0; + HASH_TABLE_ITERATE(tg->nodes, node_key, node) + { + total_time_spent_on_unlink_local_files += node->time_spent_on_unlink_local_files; + total_time_spent_on_prune_ancestors_of_temp_node += node->time_spent_on_prune_ancestors_of_temp_node; + total_time_spent_on_prune_ancestors_of_persisted_node += node->time_spent_on_prune_ancestors_of_persisted_node; } + total_time_spent_on_unlink_local_files /= 1e6; + total_time_spent_on_prune_ancestors_of_temp_node /= 1e6; + total_time_spent_on_prune_ancestors_of_persisted_node /= 1e6; - vine_task_node_set_outfile(node, outfile_type, outfile_remote_name); + debug(D_VINE, "total time spent on prune ancestors of temp node: %.6f seconds\n", total_time_spent_on_prune_ancestors_of_temp_node); + debug(D_VINE, "total time spent on prune ancestors of persisted node: %.6f seconds\n", total_time_spent_on_prune_ancestors_of_persisted_node); + debug(D_VINE, "total time spent on unlink local files: %.6f seconds\n", total_time_spent_on_unlink_local_files); return; } diff --git a/taskvine/src/manager/vine_task_node.c b/taskvine/src/manager/vine_task_node.c index aa6d38bc9b..eca0c551c4 100644 --- a/taskvine/src/manager/vine_task_node.c +++ b/taskvine/src/manager/vine_task_node.c @@ -25,6 +25,12 @@ #include #include +/** + * Compute a lexicographic priority score from the node key. + * Used during topological sorting to break ties deterministically. + * @param key Reference to the node key. + * @return The lexical priority. + */ double compute_lex_priority(const char *key) { double score = 0.0; @@ -36,11 +42,17 @@ double compute_lex_priority(const char *key) return -score; } -/* - * Create a new node in the graph. - * The caller must construct the graph nodes in a topological order. +/** + * Create a new node object. + * @param manager Reference to the manager object. + * @param node_key Reference to the node key. + * @param proxy_library_name Reference to the proxy library name. + * @param proxy_function_name Reference to the proxy function name. + * @param staging_dir Reference to the staging directory. + * @param prune_depth Reference to the prune depth. + * @param priority_mode Reference to the priority mode. + * @return A new node object. */ - struct vine_task_node *vine_task_node_create( struct vine_manager *manager, const char *node_key, @@ -58,8 +70,8 @@ struct vine_task_node *vine_task_node_create( node->manager = manager; node->node_key = xxstrdup(node_key); - node->staging_dir = xxstrdup(staging_dir); - + node->staging_dir = xxstrdup(staging_dir); + node->priority_mode = priority_mode; node->prune_status = PRUNE_STATUS_NOT_PRUNED; node->parents = list_create(); @@ -68,7 +80,7 @@ struct vine_task_node *vine_task_node_create( node->completed = 0; node->prune_depth = prune_depth; node->retry_attempts_left = 1; - node->outfile_size_bytes = 0; + node->outfile_size_bytes = 0; node->depth = -1; node->height = -1; @@ -88,24 +100,31 @@ struct vine_task_node *vine_task_node_create( node->task = vine_task_create(proxy_function_name); vine_task_set_library_required(node->task, proxy_library_name); vine_task_addref(node->task); - + /* build JSON infile expected by library: {"fn_args": [key], "fn_kwargs": {}} */ struct jx *event = jx_object(NULL); struct jx *args = jx_array(NULL); jx_array_append(args, jx_string(node->node_key)); jx_insert(event, jx_string("fn_args"), args); jx_insert(event, jx_string("fn_kwargs"), jx_object(NULL)); - + char *infile_content = jx_print_string(event); jx_delete(event); - + node->infile = vine_declare_buffer(node->manager, infile_content, strlen(infile_content), VINE_CACHE_LEVEL_TASK, VINE_UNLINK_WHEN_DONE); free(infile_content); vine_task_add_input(node->task, node->infile, "infile", VINE_TRANSFER_ALWAYS); - + return node; } +/** + * Set the outfile of a node in the task graph. + * This involves declaring the output file and adding it to the task. + * @param node Reference to the node object. + * @param outfile_type Reference to the outfile type. + * @param outfile_remote_name Reference to the outfile remote name. + */ void vine_task_node_set_outfile(struct vine_task_node *node, vine_task_node_outfile_type_t outfile_type, const char *outfile_remote_name) { if (!node) { @@ -121,10 +140,10 @@ void vine_task_node_set_outfile(struct vine_task_node *node, vine_task_node_outf switch (node->outfile_type) { case VINE_NODE_OUTFILE_TYPE_LOCAL: { char *local_output_dir = string_format("%s/outputs", node->staging_dir); - if (mkdir(local_output_dir, 0777) != 0 && errno != EEXIST) { - debug(D_ERROR, "failed to mkdir %s (errno=%d)", local_output_dir, errno); - } - char *local_output_path = string_format("%s/%s", local_output_dir, node->outfile_remote_name); + if (mkdir(local_output_dir, 0777) != 0 && errno != EEXIST) { + debug(D_ERROR, "failed to mkdir %s (errno=%d)", local_output_dir, errno); + } + char *local_output_path = string_format("%s/%s", local_output_dir, node->outfile_remote_name); node->outfile = vine_declare_file(node->manager, local_output_path, VINE_CACHE_LEVEL_WORKFLOW, 0); free(local_output_dir); free(local_output_path); @@ -145,6 +164,13 @@ void vine_task_node_set_outfile(struct vine_task_node *node, vine_task_node_outf return; } +/** + * Check if the outfile of a node is persisted. + * A node is considered persisted if it has completed and 1) the outfile is written to the shared file system, + * 2) the outfile is written to the local staging directory. + * @param node Reference to the node object. + * @return 1 if the outfile is persisted, 0 otherwise. + */ static int _node_outfile_is_persisted(struct vine_task_node *node) { if (!node) { @@ -168,6 +194,11 @@ static int _node_outfile_is_persisted(struct vine_task_node *node) return 0; } +/** + * Calculate the priority of a node given the priority mode. + * @param node Reference to the node object. + * @return The priority. + */ double vine_task_node_calculate_priority(struct vine_task_node *node) { if (!node) { @@ -219,6 +250,11 @@ double vine_task_node_calculate_priority(struct vine_task_node *node) return priority; } +/** + * Update the critical time of a node. + * @param node Reference to the node object. + * @param execution_time Reference to the execution time of the node. + */ void vine_task_node_update_critical_time(struct vine_task_node *node, timestamp_t execution_time) { timestamp_t max_parent_critical_time = 0; @@ -232,7 +268,13 @@ void vine_task_node_update_critical_time(struct vine_task_node *node, timestamp_ node->critical_time = max_parent_critical_time + execution_time; } -/* the dfs helper function for finding parents in a specific depth */ +/** + * The dfs helper function for finding parents in a specific depth. + * @param node Reference to the node object. + * @param remaining_depth Reference to the remaining depth. + * @param result Reference to the result list. + * @param visited Reference to the visited set. + */ static void _find_parents_dfs(struct vine_task_node *node, int remaining_depth, struct list *result, struct set *visited) { if (!node || set_lookup(visited, node)) { @@ -251,7 +293,12 @@ static void _find_parents_dfs(struct vine_task_node *node, int remaining_depth, } } -/* find all parents in a specific depth of the node */ +/** + * Find all parents in a specific depth of the node. + * @param node Reference to the node object. + * @param depth Reference to the depth. + * @return The list of parents. + */ static struct list *_find_parents_in_depth(struct vine_task_node *node, int depth) { if (!node || depth < 0) { @@ -267,7 +314,14 @@ static struct list *_find_parents_in_depth(struct vine_task_node *node, int dept return result; } -int _prune_ancestors_of_temp_node(struct vine_task_node *node) +/** + * Prune the ancestors of a temp node. This is only used for temp nodes that produce temp files. + * All ancestors of this node we consider here are temp nodes, we can not safely prune those stored in the shared file system + * because temp nodes are not considered safe enough to trigger the deletion of upstream persisted files. + * @param node Reference to the node object. + * @return The number of pruned replicas. + */ +static int prune_ancestors_of_temp_node(struct vine_task_node *node) { if (!node || !node->outfile || node->prune_depth <= 0) { return 0; @@ -329,6 +383,11 @@ int _prune_ancestors_of_temp_node(struct vine_task_node *node) return pruned_replica_count; } +/** + * Find all safe ancestors of a node. + * @param start_node Reference to the start node. + * @return The set of safe ancestors. + */ static struct set *_find_safe_ancestors(struct vine_task_node *start_node) { if (!start_node) { @@ -392,7 +451,14 @@ static struct set *_find_safe_ancestors(struct vine_task_node *start_node) return visited_nodes; } -int _prune_ancestors_of_persisted_node(struct vine_task_node *node) +/** + * Prune the ancestors of a persisted node. This is only used for persisted nodes that produce persisted files. + * All ancestors we consider here include both temp nodes and persisted nodes, becasue data written to shared file system + * is safe and can definitely trigger upstream data redundancy to be released. + * @param node Reference to the node object. + * @return The number of pruned replicas. + */ +static int prune_ancestors_of_persisted_node(struct vine_task_node *node) { if (!node) { return 0; @@ -414,15 +480,12 @@ int _prune_ancestors_of_persisted_node(struct vine_task_node *node) { /* unlink the shared file system file */ if (!ancestor_node->outfile) { - timestamp_t unlink_start = timestamp_get(); - if (ancestor_node->outfile_remote_name) { - unlink(ancestor_node->outfile_remote_name); // system call - } + timestamp_t unlink_start = timestamp_get(); + if (ancestor_node->outfile_remote_name) { + unlink(ancestor_node->outfile_remote_name); // system call + } node->time_spent_on_unlink_local_files += timestamp_get() - unlink_start; - debug(D_VINE, "unlinked %s size: %zu bytes, time: %" PRIu64, - ancestor_node->outfile_remote_name ? ancestor_node->outfile_remote_name : "(null)", - ancestor_node->outfile_size_bytes, - (uint64_t)node->time_spent_on_unlink_local_files); + debug(D_VINE, "unlinked %s size: %zu bytes, time: %" PRIu64, ancestor_node->outfile_remote_name ? ancestor_node->outfile_remote_name : "(null)", ancestor_node->outfile_size_bytes, (uint64_t)node->time_spent_on_unlink_local_files); } else { switch (ancestor_node->outfile->type) { case VINE_FILE: @@ -448,6 +511,11 @@ int _prune_ancestors_of_persisted_node(struct vine_task_node *node) return pruned_replica_count; } +/** + * Submit a node to the taskvine manager. + * @param node Reference to the node object. + * @return The task id. + */ int vine_task_node_submit(struct vine_task_node *node) { if (!node) { @@ -460,80 +528,99 @@ int vine_task_node_submit(struct vine_task_node *node) return vine_submit(node->manager, node->task); } -/* print the info of the node */ +/** + * Print the info of the node. + * @param node Reference to the node object. + */ void vine_task_node_print_info(struct vine_task_node *node) { - if (!node) { - return; - } - - if (!node->task) { - debug(D_ERROR, "node %s has no task", node->node_key); - return; - } - - debug(D_VINE, "---------------- Node Info ----------------"); - debug(D_VINE, "key: %s", node->node_key); - debug(D_VINE, "task_id: %d", node->task->task_id); - debug(D_VINE, "staging_dir: %s", node->staging_dir ? node->staging_dir : "(null)"); - debug(D_VINE, "depth: %d", node->depth); - debug(D_VINE, "height: %d", node->height); - debug(D_VINE, "prune_depth: %d", node->prune_depth); - debug(D_VINE, "priority_mode: %d", node->priority_mode); - - 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"); - } - - /* print parent and child node keys */ - char *parent_keys = NULL; - struct vine_task_node *p; - LIST_ITERATE(node->parents, p) { - if (!parent_keys) { - parent_keys = string_format("%s", p->node_key); - } else { - char *tmp = string_format("%s, %s", parent_keys, p->node_key); - free(parent_keys); - parent_keys = tmp; - } - } - - char *child_keys = NULL; - struct vine_task_node *c; - LIST_ITERATE(node->children, c) { - if (!child_keys) { - child_keys = string_format("%s", c->node_key); - } else { - char *tmp = string_format("%s, %s", child_keys, c->node_key); - free(child_keys); - child_keys = tmp; - } - } - - debug(D_VINE, "parents: %s", parent_keys ? parent_keys : "(none)"); - debug(D_VINE, "children: %s", child_keys ? child_keys : "(none)"); - - free(parent_keys); - free(child_keys); - - debug(D_VINE, "-------------------------------------------"); + if (!node) { + return; + } + + if (!node->task) { + debug(D_ERROR, "node %s has no task", node->node_key); + return; + } + + debug(D_VINE, "---------------- Node Info ----------------"); + debug(D_VINE, "key: %s", node->node_key); + debug(D_VINE, "task_id: %d", node->task->task_id); + debug(D_VINE, "staging_dir: %s", node->staging_dir ? node->staging_dir : "(null)"); + debug(D_VINE, "depth: %d", node->depth); + debug(D_VINE, "height: %d", node->height); + debug(D_VINE, "prune_depth: %d", node->prune_depth); + debug(D_VINE, "priority_mode: %d", node->priority_mode); + + 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"); + } + + /* print parent and child node keys */ + char *parent_keys = NULL; + struct vine_task_node *p; + LIST_ITERATE(node->parents, p) + { + if (!parent_keys) { + parent_keys = string_format("%s", p->node_key); + } else { + char *tmp = string_format("%s, %s", parent_keys, p->node_key); + free(parent_keys); + parent_keys = tmp; + } + } + + char *child_keys = NULL; + struct vine_task_node *c; + LIST_ITERATE(node->children, c) + { + if (!child_keys) { + child_keys = string_format("%s", c->node_key); + } else { + char *tmp = string_format("%s, %s", child_keys, c->node_key); + free(child_keys); + child_keys = tmp; + } + } + + debug(D_VINE, "parents: %s", parent_keys ? parent_keys : "(none)"); + debug(D_VINE, "children: %s", child_keys ? child_keys : "(none)"); + + free(parent_keys); + free(child_keys); + + debug(D_VINE, "-------------------------------------------"); } +/** + * Prune the ancestors of a node when it is completed. + * @param node Reference to the node object. + */ void vine_task_node_prune_ancestors(struct vine_task_node *node) { if (!node) { @@ -545,9 +632,9 @@ void vine_task_node_prune_ancestors(struct vine_task_node *node) int pruned_replica_count = 0; if (_node_outfile_is_persisted(node)) { - pruned_replica_count = _prune_ancestors_of_persisted_node(node); + pruned_replica_count = prune_ancestors_of_persisted_node(node); } else { - pruned_replica_count = _prune_ancestors_of_temp_node(node); + pruned_replica_count = prune_ancestors_of_temp_node(node); } timestamp_t elapsed_time = timestamp_get() - start_time; @@ -555,20 +642,27 @@ void vine_task_node_prune_ancestors(struct vine_task_node *node) debug(D_VINE, "pruned %d ancestors of node %s in %.6f seconds", pruned_replica_count, node->node_key, elapsed_time / 1000000.0); } +/** + * Replicate the outfile of a node if it is a temp file. + * @param node Reference to the node object. + */ void vine_task_node_replicate_outfile(struct vine_task_node *node) { if (!node || !node->outfile) { return; } - if (node->outfile->type != VINE_TEMP) { - return; - } + if (node->outfile->type != VINE_TEMP) { + return; + } vine_temp_replicate_file_later(node->manager, node->outfile); } -/* delete the node and all of its associated resources. */ +/** + * Delete the node and all of its associated resources. + * @param node Reference to the node object. + */ void vine_task_node_delete(struct vine_task_node *node) { if (!node) { @@ -581,9 +675,9 @@ void vine_task_node_delete(struct vine_task_node *node) if (node->outfile_remote_name) { free(node->outfile_remote_name); } - if (node->staging_dir) { - free(node->staging_dir); - } + if (node->staging_dir) { + free(node->staging_dir); + } vine_task_delete(node->task); node->task = NULL; From 1955fd147491c19501c6cd1112f1750190b699fc Mon Sep 17 00:00:00 2001 From: Jin Zhou Date: Wed, 15 Oct 2025 16:40:52 -0400 Subject: [PATCH 032/113] use a separate directory --- taskvine/src/Makefile | 4 +- taskvine/src/bindings/python3/Makefile | 9 ++-- taskvine/src/graph/GraphInvoker/.gitignore | 0 taskvine/src/graph/GraphInvoker/.gitkeep | 0 taskvine/src/graph/GraphInvoker/Makefile | 38 ++++++++++++++++ .../GraphInvoker}/graph_definition.py | 0 .../GraphInvoker}/graph_executor.py | 0 .../GraphInvoker}/vine_task_graph.c | 0 .../src/graph/GraphInvoker/vine_task_graph.h | 43 +++++++++++++++++++ .../GraphInvoker}/vine_task_node.c | 0 .../GraphInvoker}/vine_task_node.h | 0 taskvine/src/graph/Makefile | 13 ++++++ taskvine/src/graph/README.md | 2 + taskvine/src/manager/Makefile | 5 +-- taskvine/src/manager/vine_task_graph.h | 24 ----------- 15 files changed, 106 insertions(+), 32 deletions(-) create mode 100644 taskvine/src/graph/GraphInvoker/.gitignore create mode 100644 taskvine/src/graph/GraphInvoker/.gitkeep create mode 100644 taskvine/src/graph/GraphInvoker/Makefile rename taskvine/src/{bindings/python3/ndcctools/taskvine => graph/GraphInvoker}/graph_definition.py (100%) rename taskvine/src/{bindings/python3/ndcctools/taskvine => graph/GraphInvoker}/graph_executor.py (100%) rename taskvine/src/{manager => graph/GraphInvoker}/vine_task_graph.c (100%) create mode 100644 taskvine/src/graph/GraphInvoker/vine_task_graph.h rename taskvine/src/{manager => graph/GraphInvoker}/vine_task_node.c (100%) rename taskvine/src/{manager => graph/GraphInvoker}/vine_task_node.h (100%) create mode 100644 taskvine/src/graph/Makefile create mode 100644 taskvine/src/graph/README.md delete mode 100644 taskvine/src/manager/vine_task_graph.h diff --git a/taskvine/src/Makefile b/taskvine/src/Makefile index 8f828fd7bf..5c8858a2e4 100644 --- a/taskvine/src/Makefile +++ b/taskvine/src/Makefile @@ -1,12 +1,12 @@ 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 +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..580e25b515 100644 --- a/taskvine/src/bindings/python3/Makefile +++ b/taskvine/src/bindings/python3/Makefile @@ -4,8 +4,8 @@ include ../../../../rules.mk # Python always uses 'so' for its modules (even on Darwin) CCTOOLS_DYNAMIC_SUFFIX = so # SWIG produces code that causes a lot of warnings, so use -w to turn those off. -LOCAL_CCFLAGS = -w -fPIC -DNDEBUG $(CCTOOLS_PYTHON3_CCFLAGS) -I ../../manager -LOCAL_LINKAGE = $(CCTOOLS_PYTHON3_LDFLAGS) -lz $(CCTOOLS_OPENSSL_LDFLAGS) $(CCTOOLS_HOME)/taskvine/src/manager/libtaskvine.a $(CCTOOLS_HOME)/dttools/src/libdttools.a +LOCAL_CCFLAGS = -w -fPIC -DNDEBUG $(CCTOOLS_PYTHON3_CCFLAGS) -I ../../manager -I ../../graph/GraphInvoker +LOCAL_LINKAGE = $(CCTOOLS_PYTHON3_LDFLAGS) -lz $(CCTOOLS_OPENSSL_LDFLAGS) $(CCTOOLS_HOME)/taskvine/src/graph/GraphInvoker/libgraphinvoker.a $(CCTOOLS_HOME)/taskvine/src/manager/libtaskvine.a $(CCTOOLS_HOME)/dttools/src/libdttools.a CCTOOLS_FLAKE8_IGNORE_FILES = "cvine.py" @@ -18,7 +18,7 @@ all: $(TARGETS) vine_wrap.c: taskvine.i @echo "SWIG taskvine.i (python)" - @$(CCTOOLS_SWIG) -o vine_wrap.c -outdir ndcctools/taskvine -python -I$(CCTOOLS_HOME)/dttools/src -I$(CCTOOLS_HOME)/taskvine/src/manager taskvine.i + @$(CCTOOLS_SWIG) -o vine_wrap.c -outdir ndcctools/taskvine -python -I$(CCTOOLS_HOME)/dttools/src -I$(CCTOOLS_HOME)/taskvine/src/manager -I$(CCTOOLS_HOME)/taskvine/src/graph/GraphInvoker taskvine.i $(DSPYTHONSO): vine_wrap.o $(EXTERNAL_DEPENDENCIES) @@ -35,3 +35,6 @@ install: all cp ndcctools/taskvine/*.py $(DSPYTHONSO) $(CCTOOLS_PYTHON3_PATH)/ndcctools/taskvine cp ndcctools/taskvine/compat/*.py $(CCTOOLS_PYTHON3_PATH)/ndcctools/taskvine/compat cp taskvine.py $(CCTOOLS_PYTHON3_PATH)/ + # install GraphInvoker python modules into package path to preserve API + cp $(CCTOOLS_HOME)/taskvine/src/graph/GraphInvoker/graph_definition.py $(CCTOOLS_PYTHON3_PATH)/ndcctools/taskvine/graph_definition.py + cp $(CCTOOLS_HOME)/taskvine/src/graph/GraphInvoker/graph_executor.py $(CCTOOLS_PYTHON3_PATH)/ndcctools/taskvine/graph_executor.py diff --git a/taskvine/src/graph/GraphInvoker/.gitignore b/taskvine/src/graph/GraphInvoker/.gitignore new file mode 100644 index 0000000000..e69de29bb2 diff --git a/taskvine/src/graph/GraphInvoker/.gitkeep b/taskvine/src/graph/GraphInvoker/.gitkeep new file mode 100644 index 0000000000..e69de29bb2 diff --git a/taskvine/src/graph/GraphInvoker/Makefile b/taskvine/src/graph/GraphInvoker/Makefile new file mode 100644 index 0000000000..cce5063e1a --- /dev/null +++ b/taskvine/src/graph/GraphInvoker/Makefile @@ -0,0 +1,38 @@ +include ../../../../config.mk +include ../../../../rules.mk + +LOCAL_LINKAGE+=${CCTOOLS_HOME}/taskvine/src/manager/libtaskvine.a ${CCTOOLS_HOME}/dttools/src/libdttools.a +LOCAL_CCFLAGS+=-I ${CCTOOLS_HOME}/taskvine/src/manager + +SOURCES = vine_task_node.c vine_task_graph.c +OBJECTS = $(SOURCES:%.c=%.o) + +LIBRARIES = libgraphinvoker.a +PROGRAMS = +SCRIPTS = +TARGETS = $(LIBRARIES) $(PROGRAMS) + +all: $(TARGETS) + +libgraphinvoker.a: $(OBJECTS) + +$(PROGRAMS): $(EXTERNALS) + +install: all + mkdir -p $(CCTOOLS_INSTALL_DIR)/lib + cp $(LIBRARIES) $(CCTOOLS_INSTALL_DIR)/lib/ + mkdir -p $(CCTOOLS_INSTALL_DIR)/graph/GraphInvoker/include + cp ${CCTOOLS_HOME}/taskvine/src/manager/taskvine.h $(CCTOOLS_INSTALL_DIR)/graph/GraphInvoker/include/ + +clean: + rm -rf $(PROGRAMS) $(OBJECTS) *.o + +test: all + +lint: + +format: + +.PHONY: all clean install test lint format + + diff --git a/taskvine/src/bindings/python3/ndcctools/taskvine/graph_definition.py b/taskvine/src/graph/GraphInvoker/graph_definition.py similarity index 100% rename from taskvine/src/bindings/python3/ndcctools/taskvine/graph_definition.py rename to taskvine/src/graph/GraphInvoker/graph_definition.py diff --git a/taskvine/src/bindings/python3/ndcctools/taskvine/graph_executor.py b/taskvine/src/graph/GraphInvoker/graph_executor.py similarity index 100% rename from taskvine/src/bindings/python3/ndcctools/taskvine/graph_executor.py rename to taskvine/src/graph/GraphInvoker/graph_executor.py diff --git a/taskvine/src/manager/vine_task_graph.c b/taskvine/src/graph/GraphInvoker/vine_task_graph.c similarity index 100% rename from taskvine/src/manager/vine_task_graph.c rename to taskvine/src/graph/GraphInvoker/vine_task_graph.c diff --git a/taskvine/src/graph/GraphInvoker/vine_task_graph.h b/taskvine/src/graph/GraphInvoker/vine_task_graph.h new file mode 100644 index 0000000000..658686b514 --- /dev/null +++ b/taskvine/src/graph/GraphInvoker/vine_task_graph.h @@ -0,0 +1,43 @@ +#ifndef VINE_TASK_GRAPH_H +#define VINE_TASK_GRAPH_H + +#include "vine_task.h" +#include "hash_table.h" +#include "list.h" +#include "vine_manager.h" +#include "set.h" +#include "vine_task_node.h" + +struct vine_task_graph { + struct vine_manager *manager; + struct hash_table *nodes; + struct itable *task_id_to_node; + struct hash_table *outfile_cachename_to_node; + + char *proxy_library_name; + char *proxy_function_name; + + double failure_injection_step_percent; // 0 - 100, the percentage of steps to inject failure +}; + + +/* Public APIs for operating the task graph */ +const char *vine_task_graph_get_proxy_library_name(const struct vine_task_graph *tg); +const char *vine_task_graph_get_proxy_function_name(const struct vine_task_graph *tg); +double vine_task_graph_get_node_heavy_score(const struct vine_task_graph *tg, const char *node_key); +const char *vine_task_graph_get_node_local_outfile_source(const struct vine_task_graph *tg, const char *node_key); +void vine_task_graph_compute_topology_metrics(struct vine_task_graph *tg); +struct vine_task_node *vine_task_graph_add_node(struct vine_task_graph *tg, + const char *node_key, + const char *staging_dir, + int prune_depth, + vine_task_node_priority_mode_t priority_mode); +struct vine_task_graph *vine_task_graph_create(struct vine_manager *q); +void vine_task_graph_set_failure_injection_step_percent(struct vine_task_graph *tg, double percent); +void vine_task_graph_add_dependency(struct vine_task_graph *tg, const char *parent_key, const char *child_key); +void vine_task_graph_set_node_outfile(struct vine_task_graph *tg, const char *node_key, vine_task_node_outfile_type_t outfile_type, const char *outfile_remote_name); +void vine_task_graph_execute(struct vine_task_graph *tg); +void vine_task_graph_delete(struct vine_task_graph *tg); + + +#endif // VINE_TASK_GRAPH_H diff --git a/taskvine/src/manager/vine_task_node.c b/taskvine/src/graph/GraphInvoker/vine_task_node.c similarity index 100% rename from taskvine/src/manager/vine_task_node.c rename to taskvine/src/graph/GraphInvoker/vine_task_node.c diff --git a/taskvine/src/manager/vine_task_node.h b/taskvine/src/graph/GraphInvoker/vine_task_node.h similarity index 100% rename from taskvine/src/manager/vine_task_node.h rename to taskvine/src/graph/GraphInvoker/vine_task_node.h diff --git a/taskvine/src/graph/Makefile b/taskvine/src/graph/Makefile new file mode 100644 index 0000000000..257fe8ce69 --- /dev/null +++ b/taskvine/src/graph/Makefile @@ -0,0 +1,13 @@ +include ../../../config.mk +include ../../../rules.mk + +SUBDIRS = GraphInvoker + +all clean install test lint format: $(SUBDIRS) + +$(SUBDIRS): %: + $(MAKE) -C $@ $(MAKECMDGOALS) + +.PHONY: all clean install test lint format $(SUBDIRS) + + diff --git a/taskvine/src/graph/README.md b/taskvine/src/graph/README.md new file mode 100644 index 0000000000..8d7a3f59fc --- /dev/null +++ b/taskvine/src/graph/README.md @@ -0,0 +1,2 @@ +# Graph Executors + diff --git a/taskvine/src/manager/Makefile b/taskvine/src/manager/Makefile index 1246f83737..5c56c9f09b 100644 --- a/taskvine/src/manager/Makefile +++ b/taskvine/src/manager/Makefile @@ -29,9 +29,7 @@ SOURCES = \ vine_fair.c \ vine_runtime_dir.c \ vine_task_groups.c \ - vine_temp.c \ - vine_task_node.c \ - vine_task_graph.c + vine_temp.c PUBLIC_HEADERS = taskvine.h @@ -42,6 +40,7 @@ TARGETS = $(LIBRARIES) all: $(TARGETS) + libtaskvine.a: $(OBJECTS) install: all diff --git a/taskvine/src/manager/vine_task_graph.h b/taskvine/src/manager/vine_task_graph.h deleted file mode 100644 index a3ed8d3c6b..0000000000 --- a/taskvine/src/manager/vine_task_graph.h +++ /dev/null @@ -1,24 +0,0 @@ -#ifndef VINE_TASK_GRAPH_H -#define VINE_TASK_GRAPH_H - -#include "vine_task.h" -#include "hash_table.h" -#include "list.h" -#include "vine_manager.h" -#include "set.h" -#include "vine_task_node.h" - -struct vine_task_graph { - struct vine_manager *manager; - struct hash_table *nodes; - struct itable *task_id_to_node; - struct hash_table *outfile_cachename_to_node; - - char *proxy_library_name; - char *proxy_function_name; - - double failure_injection_step_percent; // 0 - 100, the percentage of steps to inject failure -}; - - -#endif // VINE_TASK_GRAPH_H From 6b1a5db7ea21f0d9baeda382d600230a70b57fc5 Mon Sep 17 00:00:00 2001 From: Jin Zhou Date: Wed, 15 Oct 2025 21:45:24 -0400 Subject: [PATCH 033/113] backup --- .../graph/GraphInvoker/graph_definition.py | 4 + .../src/graph/GraphInvoker/vine_task_graph.c | 73 +++++++++++++------ .../src/graph/GraphInvoker/vine_task_graph.h | 4 +- .../src/graph/GraphInvoker/vine_task_node.c | 31 ++++---- 4 files changed, 74 insertions(+), 38 deletions(-) diff --git a/taskvine/src/graph/GraphInvoker/graph_definition.py b/taskvine/src/graph/GraphInvoker/graph_definition.py index 1602621bbb..1bf509f72d 100644 --- a/taskvine/src/graph/GraphInvoker/graph_definition.py +++ b/taskvine/src/graph/GraphInvoker/graph_definition.py @@ -1,3 +1,7 @@ +# 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.utils import load_variable_from_library import os diff --git a/taskvine/src/graph/GraphInvoker/vine_task_graph.c b/taskvine/src/graph/GraphInvoker/vine_task_graph.c index 5f64287d1c..394f98c2f8 100644 --- a/taskvine/src/graph/GraphInvoker/vine_task_graph.c +++ b/taskvine/src/graph/GraphInvoker/vine_task_graph.c @@ -1,30 +1,32 @@ -#include "vine_task_graph.h" -#include "taskvine.h" -#include "vine_manager.h" -#include "vine_task_node.h" -#include "vine_worker_info.h" -#include "priority_queue.h" #include #include #include -#include "debug.h" -#include "stringtools.h" -#include "xxmalloc.h" +#include #include -#include "hash_table.h" +#include +#include #include -#include "itable.h" + +#include "priority_queue.h" #include "list.h" -#include "vine_task.h" -#include "timestamp.h" -#include "vine_file.h" +#include "debug.h" +#include "itable.h" +#include "xxmalloc.h" +#include "stringtools.h" +#include "hash_table.h" #include "set.h" -#include "vine_mount.h" +#include "timestamp.h" #include "progress_bar.h" -#include "assert.h" #include "macros.h" -#include -#include + +#include "vine_task_graph.h" +#include "vine_manager.h" +#include "vine_task_node.h" +#include "vine_worker_info.h" +#include "vine_task.h" +#include "vine_file.h" +#include "vine_mount.h" + static volatile sig_atomic_t interrupted = 0; @@ -283,7 +285,34 @@ static struct vine_task_node *get_node_by_task(struct vine_task_graph *tg, struc /*************************************************************/ /** - * Get the library name of the task graph. + * Set the proxy library and function names (Python-side), shared by all tasks. + * @param tg Reference to the task graph object. + * @param proxy_library_name Reference to the proxy library name. + * @param proxy_function_name Reference to the proxy function name. + */ +void vine_task_graph_set_proxy_library_and_function_names(struct vine_task_graph *tg, + const char *proxy_library_name, + const char *proxy_function_name) +{ + if (!tg || !proxy_library_name || !proxy_function_name) { + return; + } + + /* free the existing proxy library and function names if they exist */ + if (tg->proxy_library_name) { + free(tg->proxy_library_name); + } + if (tg->proxy_function_name) { + free(tg->proxy_function_name); + } + + /* set the new proxy library and function names */ + tg->proxy_library_name = xxstrdup(proxy_library_name); + tg->proxy_function_name = xxstrdup(proxy_function_name); +} + +/** + * Get the proxy library name (Python-side), shared by all tasks. * @param tg Reference to the task graph object. * @return The library name. */ @@ -297,7 +326,7 @@ const char *vine_task_graph_get_proxy_library_name(const struct vine_task_graph } /** - * Get the function name of the task graph. + * Get the proxy function name (Python-side), shared by all tasks. * @param tg Reference to the task graph object. * @return The function name. */ @@ -517,8 +546,8 @@ struct vine_task_graph *vine_task_graph_create(struct vine_manager *q) tg->task_id_to_node = itable_create(0); tg->outfile_cachename_to_node = hash_table_create(0, 0); - tg->proxy_library_name = xxstrdup("vine_task_graph_library"); - tg->proxy_function_name = xxstrdup("compute_single_key"); + tg->proxy_library_name = xxstrdup("vine_task_graph_library"); // Python-side proxy library name (shared by all tasks) + tg->proxy_function_name = xxstrdup("compute_single_key"); // Python-side proxy function name (shared by all tasks) tg->manager = q; tg->failure_injection_step_percent = -1.0; diff --git a/taskvine/src/graph/GraphInvoker/vine_task_graph.h b/taskvine/src/graph/GraphInvoker/vine_task_graph.h index 658686b514..956b9d8200 100644 --- a/taskvine/src/graph/GraphInvoker/vine_task_graph.h +++ b/taskvine/src/graph/GraphInvoker/vine_task_graph.h @@ -38,6 +38,8 @@ void vine_task_graph_add_dependency(struct vine_task_graph *tg, const char *pare void vine_task_graph_set_node_outfile(struct vine_task_graph *tg, const char *node_key, vine_task_node_outfile_type_t outfile_type, const char *outfile_remote_name); void vine_task_graph_execute(struct vine_task_graph *tg); void vine_task_graph_delete(struct vine_task_graph *tg); - +void vine_task_graph_set_proxy_library_and_function_names(struct vine_task_graph *tg, + const char *proxy_library_name, + const char *proxy_function_name); #endif // VINE_TASK_GRAPH_H diff --git a/taskvine/src/graph/GraphInvoker/vine_task_node.c b/taskvine/src/graph/GraphInvoker/vine_task_node.c index eca0c551c4..f3e3e4103d 100644 --- a/taskvine/src/graph/GraphInvoker/vine_task_node.c +++ b/taskvine/src/graph/GraphInvoker/vine_task_node.c @@ -1,29 +1,30 @@ -#include "vine_task_node.h" -#include "vine_manager.h" -#include "vine_task.h" -#include "vine_file.h" -#include "vine_task_graph.h" +#include +#include +#include +#include +#include +#include +#include +#include + #include "jx.h" #include "jx_print.h" #include "xxmalloc.h" #include "stringtools.h" -#include "taskvine.h" #include "timestamp.h" #include "set.h" #include "hash_table.h" #include "debug.h" -#include "assert.h" +#include "random.h" + +#include "vine_manager.h" +#include "vine_file.h" +#include "vine_task.h" +#include "vine_task_graph.h" #include "vine_worker_info.h" #include "vine_temp.h" -#include "random.h" +#include "vine_task_node.h" -#include -#include -#include -#include -#include -#include -#include /** * Compute a lexicographic priority score from the node key. From 8cd855ff6a5f7d4d151a51989b2bf1277c5faf57 Mon Sep 17 00:00:00 2001 From: Jin Zhou Date: Wed, 15 Oct 2025 22:18:30 -0400 Subject: [PATCH 034/113] libtask --- taskvine/src/bindings/python3/Makefile | 2 + taskvine/src/graph/GraphInvoker/config.py | 37 +++++++++++++ .../src/graph/GraphInvoker/graph_executor.py | 38 +++---------- .../src/graph/GraphInvoker/library_task.py | 53 +++++++++++++++++++ taskvine/src/graph/GraphInvoker/utils.py | 17 ++++++ 5 files changed, 116 insertions(+), 31 deletions(-) create mode 100644 taskvine/src/graph/GraphInvoker/config.py create mode 100644 taskvine/src/graph/GraphInvoker/library_task.py create mode 100644 taskvine/src/graph/GraphInvoker/utils.py diff --git a/taskvine/src/bindings/python3/Makefile b/taskvine/src/bindings/python3/Makefile index 580e25b515..07e837f6e6 100644 --- a/taskvine/src/bindings/python3/Makefile +++ b/taskvine/src/bindings/python3/Makefile @@ -35,6 +35,8 @@ install: all cp ndcctools/taskvine/*.py $(DSPYTHONSO) $(CCTOOLS_PYTHON3_PATH)/ndcctools/taskvine cp ndcctools/taskvine/compat/*.py $(CCTOOLS_PYTHON3_PATH)/ndcctools/taskvine/compat cp taskvine.py $(CCTOOLS_PYTHON3_PATH)/ + # install GraphInvoker python modules into package path to preserve API cp $(CCTOOLS_HOME)/taskvine/src/graph/GraphInvoker/graph_definition.py $(CCTOOLS_PYTHON3_PATH)/ndcctools/taskvine/graph_definition.py cp $(CCTOOLS_HOME)/taskvine/src/graph/GraphInvoker/graph_executor.py $(CCTOOLS_PYTHON3_PATH)/ndcctools/taskvine/graph_executor.py + cp $(CCTOOLS_HOME)/taskvine/src/graph/GraphInvoker/library_task.py $(CCTOOLS_PYTHON3_PATH)/ndcctools/taskvine/library_task.py diff --git a/taskvine/src/graph/GraphInvoker/config.py b/taskvine/src/graph/GraphInvoker/config.py new file mode 100644 index 0000000000..e3b8a41fec --- /dev/null +++ b/taskvine/src/graph/GraphInvoker/config.py @@ -0,0 +1,37 @@ +# config.py +# 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 dataclasses import dataclass, field + + +@dataclass(frozen=True) +class DefaultTuneParams: + worker_source_max_transfers: int = 100 + max_retrievals: int = -1 + prefer_dispatch: int = 1 + transient_error_interval: int = 1 + attempt_schedule_depth: int = 1000 + + +@dataclass(frozen=True) +class DefaultPaths: + shared_file_system_dir: str = "/project01/ndcms/jzhou24/shared_file_system" + staging_dir: str = "/project01/ndcms/jzhou24/staging" + + +@dataclass(frozen=True) +class DefaultPolicies: + replica_placement_policy: str = "random" + priority_mode: str = "largest-input-first" + scheduling_mode: str = "files" + prune_depth: int = 1 + + +@dataclass(frozen=True) +class DefaultOutfileType: + outfile_type: dict = field(default_factory=lambda: { + "temp": 1.0, + "shared-file-system": 0.0, + }) diff --git a/taskvine/src/graph/GraphInvoker/graph_executor.py b/taskvine/src/graph/GraphInvoker/graph_executor.py index 01e7cb8811..b36ba3f129 100644 --- a/taskvine/src/graph/GraphInvoker/graph_executor.py +++ b/taskvine/src/graph/GraphInvoker/graph_executor.py @@ -2,6 +2,7 @@ from ndcctools.taskvine.manager import Manager from ndcctools.taskvine.utils import load_variable_from_library, delete_all_files, get_c_constant from ndcctools.taskvine.graph_definition import GraphKeyResult, TaskGraph, init_task_graph_context, compute_dts_key, compute_sexpr_key, compute_single_key, hash_name, hashable +from ndcctools.taskvine.library_task import LibraryTask import cloudpickle import os @@ -105,34 +106,9 @@ def __init__(self, # initialize the task graph self._vine_task_graph = cvine.vine_task_graph_create(self._taskvine) - # create library task with specified resources - self._create_library_task(libcores, hoisting_modules, libtask_env_files) - - def _create_library_task(self, libcores=1, hoisting_modules=[], libtask_env_files={}): - assert cvine.vine_task_graph_get_proxy_function_name(self._vine_task_graph) == compute_single_key.__name__ - - self.task_graph_pkl_file_name = f"library-task-graph-{uuid.uuid4()}.pkl" - self.task_graph_pkl_file_local_path = self.task_graph_pkl_file_name - self.task_graph_pkl_file_remote_path = self.task_graph_pkl_file_name - - hoisting_modules += [os, cloudpickle, GraphKeyResult, TaskGraph, uuid, hashlib, random, types, collections, time, - load_variable_from_library, compute_dts_key, compute_sexpr_key, compute_single_key, hash_name, hashable] - if dask: - hoisting_modules += [dask] - self.libtask = self.create_library_from_functions( - cvine.vine_task_graph_get_proxy_library_name(self._vine_task_graph), - compute_single_key, - library_context_info=[init_task_graph_context, [], {"task_graph_path": self.task_graph_pkl_file_remote_path}], - add_env=False, - function_infile_load_mode="json", - hoisting_modules=hoisting_modules - ) - self.libtask.add_input(self.declare_file(self.task_graph_pkl_file_local_path), self.task_graph_pkl_file_remote_path) - for local_file_path, remote_file_path in libtask_env_files.items(): - self.libtask.add_input(self.declare_file(local_file_path, cache=True, peer_transfer=True), remote_file_path) - self.libtask.set_cores(libcores) - self.libtask.set_function_slots(libcores) - self.install_library(self.libtask) + # create library task + self.libtask = LibraryTask(libcores=libcores, hoisting_modules=hoisting_modules, env_files=libtask_env_files) + self.libtask.install(self, self._vine_task_graph) def run(self, collection_dict, @@ -222,7 +198,7 @@ def run(self, self.task_graph.outfile_remote_name[k]) # save the task graph to a pickle file, will be sent to the remote workers - with open(self.task_graph_pkl_file_local_path, 'wb') as f: + with open(self.libtask.local_path, 'wb') as f: cloudpickle.dump(self.task_graph, f) # now execute the vine graph @@ -249,5 +225,5 @@ def __del__(self): if hasattr(self, '_vine_task_graph') and self._vine_task_graph: cvine.vine_task_graph_delete(self._vine_task_graph) - if hasattr(self, 'task_graph_pkl_file_local_path') and os.path.exists(self.task_graph_pkl_file_local_path): - os.remove(self.task_graph_pkl_file_local_path) + if hasattr(self, 'libtask') and self.libtask.local_path and os.path.exists(self.libtask.local_path): + os.remove(self.libtask.local_path) diff --git a/taskvine/src/graph/GraphInvoker/library_task.py b/taskvine/src/graph/GraphInvoker/library_task.py new file mode 100644 index 0000000000..4b3064907c --- /dev/null +++ b/taskvine/src/graph/GraphInvoker/library_task.py @@ -0,0 +1,53 @@ +import os, uuid, cloudpickle, types, time, random, hashlib, collections +from ndcctools.taskvine import cvine +from ndcctools.taskvine.graph_definition import ( + GraphKeyResult, TaskGraph, compute_dts_key, compute_sexpr_key, + compute_single_key, hash_name, hashable, init_task_graph_context +) +from ndcctools.taskvine.utils import load_variable_from_library + + +class LibraryTask: + def __init__(self, libcores=16, hoisting_modules=[], env_files={}): + self._libtask = None + + self.libcores = libcores + self.hoisting_modules = hoisting_modules + self.env_files = env_files + + self.local_path = None + self.remote_path = None + + 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 install(self, manager, vine_graph): + assert cvine.vine_task_graph_get_proxy_function_name(vine_graph) == compute_single_key.__name__ + + self.local_path = f"library-task-graph-{uuid.uuid4()}.pkl" + self.remote_path = self.local_path + + self.hoisting_modules += [ + os, cloudpickle, GraphKeyResult, TaskGraph, uuid, hashlib, random, types, collections, time, + load_variable_from_library, compute_dts_key, compute_sexpr_key, compute_single_key, hash_name, hashable + ] + lib_name = cvine.vine_task_graph_get_proxy_library_name(vine_graph) + self._libtask = manager.create_library_from_functions( + lib_name, + compute_single_key, + library_context_info=[init_task_graph_context, [], {"task_graph_path": self.remote_path}], + add_env=False, + function_infile_load_mode="json", + hoisting_modules=self.hoisting_modules, + ) + self._libtask.add_input(manager.declare_file(self.local_path), self.remote_path) + for local, remote in self.env_files.items(): + self._libtask.add_input(manager.declare_file(local, cache=True, peer_transfer=True), remote) + self._libtask.set_cores(self.libcores) + self._libtask.set_function_slots(self.libcores) + manager.install_library(self._libtask) \ No newline at end of file diff --git a/taskvine/src/graph/GraphInvoker/utils.py b/taskvine/src/graph/GraphInvoker/utils.py new file mode 100644 index 0000000000..d180f368da --- /dev/null +++ b/taskvine/src/graph/GraphInvoker/utils.py @@ -0,0 +1,17 @@ +import inspect + +def extract_manager_kwargs(kwargs, base_class): + params = set(inspect.signature(base_class.__init__).parameters) + super_kwargs = {k: v for k, v in kwargs.items() if k in params} + leftover_kwargs = {k: v for k, v in kwargs.items() if k not in params} + return super_kwargs, leftover_kwargs + +def apply_tuning(manager, tune_dict): + for k, v in tune_dict.items(): + try: + vine_param = k.replace("_", "-") + manager.tune(vine_param, v) + print(f"Tuned {vine_param} to {v}") + except Exception as e: + print(f"Failed to tune {k}={v}: {e}") + raise From 5cb9b499836d02109c4bf8dbf086d519165cfcdc Mon Sep 17 00:00:00 2001 From: Jin Zhou Date: Thu, 16 Oct 2025 01:52:29 -0400 Subject: [PATCH 035/113] dagvine --- taskvine/src/bindings/python3/Makefile | 19 ++-- taskvine/src/graph/Makefile | 2 +- .../{GraphInvoker => dagvine}/.gitignore | 0 .../graph/{GraphInvoker => dagvine}/.gitkeep | 0 .../graph/{GraphInvoker => dagvine}/Makefile | 8 +- .../graph/{GraphInvoker => dagvine}/config.py | 0 .../graph_definition.py | 0 .../graph_executor.py | 71 +++++++------- .../library_task.py => dagvine/library.py} | 24 ++--- taskvine/src/graph/dagvine/params.py | 78 +++++++++++++++ .../graph/{GraphInvoker => dagvine}/utils.py | 0 .../vine_task_graph.c | 90 ++++++++++++++++-- .../vine_task_graph.h | 5 +- .../vine_task_node.c | 81 +--------------- .../vine_task_node.h | 6 +- taskvine/src/manager/stnPTyT6 | Bin 0 -> 1422314 bytes taskvine/src/manager/taskvine.h | 12 ++- taskvine/src/manager/vine_manager.c | 2 + taskvine/src/manager/vine_temp.c | 5 +- 19 files changed, 243 insertions(+), 160 deletions(-) rename taskvine/src/graph/{GraphInvoker => dagvine}/.gitignore (100%) rename taskvine/src/graph/{GraphInvoker => dagvine}/.gitkeep (100%) rename taskvine/src/graph/{GraphInvoker => dagvine}/Makefile (80%) rename taskvine/src/graph/{GraphInvoker => dagvine}/config.py (100%) rename taskvine/src/graph/{GraphInvoker => dagvine}/graph_definition.py (100%) rename taskvine/src/graph/{GraphInvoker => dagvine}/graph_executor.py (77%) rename taskvine/src/graph/{GraphInvoker/library_task.py => dagvine/library.py} (75%) create mode 100644 taskvine/src/graph/dagvine/params.py rename taskvine/src/graph/{GraphInvoker => dagvine}/utils.py (100%) rename taskvine/src/graph/{GraphInvoker => dagvine}/vine_task_graph.c (92%) rename taskvine/src/graph/{GraphInvoker => dagvine}/vine_task_graph.h (93%) rename taskvine/src/graph/{GraphInvoker => dagvine}/vine_task_node.c (89%) rename taskvine/src/graph/{GraphInvoker => dagvine}/vine_task_node.h (88%) create mode 100644 taskvine/src/manager/stnPTyT6 diff --git a/taskvine/src/bindings/python3/Makefile b/taskvine/src/bindings/python3/Makefile index 07e837f6e6..8b9f1bf14e 100644 --- a/taskvine/src/bindings/python3/Makefile +++ b/taskvine/src/bindings/python3/Makefile @@ -1,11 +1,13 @@ include ../../../../config.mk include ../../../../rules.mk +GRAPH_EXECUTOR_MODULE_NAME = dagvine + # Python always uses 'so' for its modules (even on Darwin) CCTOOLS_DYNAMIC_SUFFIX = so # SWIG produces code that causes a lot of warnings, so use -w to turn those off. -LOCAL_CCFLAGS = -w -fPIC -DNDEBUG $(CCTOOLS_PYTHON3_CCFLAGS) -I ../../manager -I ../../graph/GraphInvoker -LOCAL_LINKAGE = $(CCTOOLS_PYTHON3_LDFLAGS) -lz $(CCTOOLS_OPENSSL_LDFLAGS) $(CCTOOLS_HOME)/taskvine/src/graph/GraphInvoker/libgraphinvoker.a $(CCTOOLS_HOME)/taskvine/src/manager/libtaskvine.a $(CCTOOLS_HOME)/dttools/src/libdttools.a +LOCAL_CCFLAGS = -w -fPIC -DNDEBUG $(CCTOOLS_PYTHON3_CCFLAGS) -I ../../manager -I ../../graph/${GRAPH_EXECUTOR_MODULE_NAME} +LOCAL_LINKAGE = $(CCTOOLS_PYTHON3_LDFLAGS) -lz $(CCTOOLS_OPENSSL_LDFLAGS) $(CCTOOLS_HOME)/taskvine/src/graph/${GRAPH_EXECUTOR_MODULE_NAME}/lib${GRAPH_EXECUTOR_MODULE_NAME}.a $(CCTOOLS_HOME)/taskvine/src/manager/libtaskvine.a $(CCTOOLS_HOME)/dttools/src/libdttools.a CCTOOLS_FLAKE8_IGNORE_FILES = "cvine.py" @@ -18,7 +20,7 @@ all: $(TARGETS) vine_wrap.c: taskvine.i @echo "SWIG taskvine.i (python)" - @$(CCTOOLS_SWIG) -o vine_wrap.c -outdir ndcctools/taskvine -python -I$(CCTOOLS_HOME)/dttools/src -I$(CCTOOLS_HOME)/taskvine/src/manager -I$(CCTOOLS_HOME)/taskvine/src/graph/GraphInvoker taskvine.i + @$(CCTOOLS_SWIG) -o vine_wrap.c -outdir ndcctools/taskvine -python -I$(CCTOOLS_HOME)/dttools/src -I$(CCTOOLS_HOME)/taskvine/src/manager -I$(CCTOOLS_HOME)/taskvine/src/graph/${GRAPH_EXECUTOR_MODULE_NAME} taskvine.i $(DSPYTHONSO): vine_wrap.o $(EXTERNAL_DEPENDENCIES) @@ -36,7 +38,10 @@ install: all cp ndcctools/taskvine/compat/*.py $(CCTOOLS_PYTHON3_PATH)/ndcctools/taskvine/compat cp taskvine.py $(CCTOOLS_PYTHON3_PATH)/ - # install GraphInvoker python modules into package path to preserve API - cp $(CCTOOLS_HOME)/taskvine/src/graph/GraphInvoker/graph_definition.py $(CCTOOLS_PYTHON3_PATH)/ndcctools/taskvine/graph_definition.py - cp $(CCTOOLS_HOME)/taskvine/src/graph/GraphInvoker/graph_executor.py $(CCTOOLS_PYTHON3_PATH)/ndcctools/taskvine/graph_executor.py - cp $(CCTOOLS_HOME)/taskvine/src/graph/GraphInvoker/library_task.py $(CCTOOLS_PYTHON3_PATH)/ndcctools/taskvine/library_task.py + mkdir -p $(CCTOOLS_PYTHON3_PATH)/ndcctools/taskvine/${GRAPH_EXECUTOR_MODULE_NAME} + cp $(CCTOOLS_HOME)/taskvine/src/graph/${GRAPH_EXECUTOR_MODULE_NAME}/graph_definition.py $(CCTOOLS_PYTHON3_PATH)/ndcctools/taskvine/${GRAPH_EXECUTOR_MODULE_NAME}/graph_definition.py + cp $(CCTOOLS_HOME)/taskvine/src/graph/${GRAPH_EXECUTOR_MODULE_NAME}/graph_executor.py $(CCTOOLS_PYTHON3_PATH)/ndcctools/taskvine/${GRAPH_EXECUTOR_MODULE_NAME}/graph_executor.py + cp $(CCTOOLS_HOME)/taskvine/src/graph/${GRAPH_EXECUTOR_MODULE_NAME}/library.py $(CCTOOLS_PYTHON3_PATH)/ndcctools/taskvine/${GRAPH_EXECUTOR_MODULE_NAME}/library.py + cp $(CCTOOLS_HOME)/taskvine/src/graph/${GRAPH_EXECUTOR_MODULE_NAME}/params.py $(CCTOOLS_PYTHON3_PATH)/ndcctools/taskvine/${GRAPH_EXECUTOR_MODULE_NAME}/params.py + cp $(CCTOOLS_HOME)/taskvine/src/graph/${GRAPH_EXECUTOR_MODULE_NAME}/utils.py $(CCTOOLS_PYTHON3_PATH)/ndcctools/taskvine/${GRAPH_EXECUTOR_MODULE_NAME}/utils.py + @echo "# ${GRAPH_EXECUTOR_MODULE_NAME} package" > $(CCTOOLS_PYTHON3_PATH)/ndcctools/taskvine/${GRAPH_EXECUTOR_MODULE_NAME}/__init__.py diff --git a/taskvine/src/graph/Makefile b/taskvine/src/graph/Makefile index 257fe8ce69..04598b4683 100644 --- a/taskvine/src/graph/Makefile +++ b/taskvine/src/graph/Makefile @@ -1,7 +1,7 @@ include ../../../config.mk include ../../../rules.mk -SUBDIRS = GraphInvoker +SUBDIRS = dagvine all clean install test lint format: $(SUBDIRS) diff --git a/taskvine/src/graph/GraphInvoker/.gitignore b/taskvine/src/graph/dagvine/.gitignore similarity index 100% rename from taskvine/src/graph/GraphInvoker/.gitignore rename to taskvine/src/graph/dagvine/.gitignore diff --git a/taskvine/src/graph/GraphInvoker/.gitkeep b/taskvine/src/graph/dagvine/.gitkeep similarity index 100% rename from taskvine/src/graph/GraphInvoker/.gitkeep rename to taskvine/src/graph/dagvine/.gitkeep diff --git a/taskvine/src/graph/GraphInvoker/Makefile b/taskvine/src/graph/dagvine/Makefile similarity index 80% rename from taskvine/src/graph/GraphInvoker/Makefile rename to taskvine/src/graph/dagvine/Makefile index cce5063e1a..497c798a4f 100644 --- a/taskvine/src/graph/GraphInvoker/Makefile +++ b/taskvine/src/graph/dagvine/Makefile @@ -7,22 +7,22 @@ LOCAL_CCFLAGS+=-I ${CCTOOLS_HOME}/taskvine/src/manager SOURCES = vine_task_node.c vine_task_graph.c OBJECTS = $(SOURCES:%.c=%.o) -LIBRARIES = libgraphinvoker.a +LIBRARIES = libdagvine.a PROGRAMS = SCRIPTS = TARGETS = $(LIBRARIES) $(PROGRAMS) all: $(TARGETS) -libgraphinvoker.a: $(OBJECTS) +libdagvine.a: $(OBJECTS) $(PROGRAMS): $(EXTERNALS) install: all mkdir -p $(CCTOOLS_INSTALL_DIR)/lib cp $(LIBRARIES) $(CCTOOLS_INSTALL_DIR)/lib/ - mkdir -p $(CCTOOLS_INSTALL_DIR)/graph/GraphInvoker/include - cp ${CCTOOLS_HOME}/taskvine/src/manager/taskvine.h $(CCTOOLS_INSTALL_DIR)/graph/GraphInvoker/include/ + mkdir -p $(CCTOOLS_INSTALL_DIR)/graph/dagvine/include + cp ${CCTOOLS_HOME}/taskvine/src/manager/taskvine.h $(CCTOOLS_INSTALL_DIR)/graph/dagvine/include/ clean: rm -rf $(PROGRAMS) $(OBJECTS) *.o diff --git a/taskvine/src/graph/GraphInvoker/config.py b/taskvine/src/graph/dagvine/config.py similarity index 100% rename from taskvine/src/graph/GraphInvoker/config.py rename to taskvine/src/graph/dagvine/config.py diff --git a/taskvine/src/graph/GraphInvoker/graph_definition.py b/taskvine/src/graph/dagvine/graph_definition.py similarity index 100% rename from taskvine/src/graph/GraphInvoker/graph_definition.py rename to taskvine/src/graph/dagvine/graph_definition.py diff --git a/taskvine/src/graph/GraphInvoker/graph_executor.py b/taskvine/src/graph/dagvine/graph_executor.py similarity index 77% rename from taskvine/src/graph/GraphInvoker/graph_executor.py rename to taskvine/src/graph/dagvine/graph_executor.py index b36ba3f129..7b2f4c7323 100644 --- a/taskvine/src/graph/GraphInvoker/graph_executor.py +++ b/taskvine/src/graph/dagvine/graph_executor.py @@ -1,8 +1,11 @@ from ndcctools.taskvine import cvine from ndcctools.taskvine.manager import Manager -from ndcctools.taskvine.utils import load_variable_from_library, delete_all_files, get_c_constant -from ndcctools.taskvine.graph_definition import GraphKeyResult, TaskGraph, init_task_graph_context, compute_dts_key, compute_sexpr_key, compute_single_key, hash_name, hashable -from ndcctools.taskvine.library_task import LibraryTask +from ndcctools.taskvine.utils import delete_all_files, get_c_constant + +from ndcctools.taskvine.dagvine.graph_definition import GraphKeyResult, TaskGraph +from ndcctools.taskvine.dagvine.params import ManagerTuningParams +from ndcctools.taskvine.dagvine.params import VineConstantParams + import cloudpickle import os @@ -66,9 +69,9 @@ def ensure_task_dict(collection_dict): class GraphExecutor(Manager): def __init__(self, *args, - libcores=1, - hoisting_modules=[], - libtask_env_files={}, + manager_tuning_params=None, + library=None, + worker_cores=16, **kwargs): signal.signal(signal.SIGINT, self._on_sigint) @@ -85,44 +88,35 @@ def __init__(self, super_kwargs = {k: v for k, v in kwargs.items() if k in super_params} super().__init__(*args, **super_kwargs) - print(f"TaskVine manager \033[92m{self.name}\033[0m listening on port \033[92m{self.port}\033[0m") + print(f"TaskVine Manager \033[92m{self.name}\033[0m listening on port \033[92m{self.port}\033[0m") # tune the manager - leftover_kwargs = {k: v for k, v in kwargs.items() if k not in super_params} - for key, value in leftover_kwargs.items(): - try: - vine_param = key.replace("_", "-") - self.tune(vine_param, value) - print(f"Tuned {vine_param} to {value}") - except Exception as e: - print(f"Failed to tune {key} with value {value}: {e}") - exit(1) - self.tune("worker-source-max-transfers", 100) - self.tune("max-retrievals", -1) - self.tune("prefer-dispatch", 1) - self.tune("transient-error-interval", 1) - self.tune("attempt-schedule-depth", 1000) + if manager_tuning_params: + assert isinstance(manager_tuning_params, ManagerTuningParams), "manager_tuning_params must be an instance of ManagerTuningParams" + for k, v in manager_tuning_params.to_dict().items(): + print(f"Tuning {k} to {v}") + self.tune(k, v) # initialize the task graph self._vine_task_graph = cvine.vine_task_graph_create(self._taskvine) # create library task - self.libtask = LibraryTask(libcores=libcores, hoisting_modules=hoisting_modules, env_files=libtask_env_files) - self.libtask.install(self, self._vine_task_graph) + self.library = library + self.library.install(self, worker_cores, self._vine_task_graph) def run(self, collection_dict, target_keys=[], replica_placement_policy="random", priority_mode="largest-input-first", - scheduling_mode="files", + # scheduling_mode="files", extra_task_output_size_mb=["uniform", 0, 0], extra_task_sleep_time=["uniform", 0, 0], prune_depth=1, shared_file_system_dir="/project01/ndcms/jzhou24/shared_file_system", staging_dir="/project01/ndcms/jzhou24/staging", failure_injection_step_percent=-1, - balance_worker_disk_load=0, + vine_constant_params=None, outfile_type={ "temp": 1.0, "shared-file-system": 0.0, @@ -130,15 +124,9 @@ def run(self, self.target_keys = target_keys self.task_dict = ensure_task_dict(collection_dict) - self.tune("balance-worker-disk-load", balance_worker_disk_load) - cvine.vine_task_graph_set_failure_injection_step_percent(self._vine_task_graph, failure_injection_step_percent) - if balance_worker_disk_load: - replica_placement_policy = "disk-load" - scheduling_mode = "worst" - - self.set_scheduler(scheduling_mode) + # self.set_scheduler(scheduling_mode) # create task graph in the python side print("Initializing TaskGraph object") @@ -154,8 +142,14 @@ def run(self, # the sum of the values in outfile_type must be 1.0 assert sum(list(outfile_type.values())) == 1.0 - # set replica placement policy - cvine.vine_set_replica_placement_policy(self._taskvine, get_c_constant(f"replica_placement_policy_{replica_placement_policy.replace('-', '_')}")) + if vine_constant_params: + assert isinstance(vine_constant_params, VineConstantParams), "vine_constant_params must be an instance of VineConstantParams" + # set replica placement policy + cvine.vine_set_replica_placement_policy(self._taskvine, vine_constant_params.get_c_constant_of("replica_placement_policy")) + # set worker scheduling algorithm + cvine.vine_set_scheduler(self._taskvine, vine_constant_params.get_c_constant_of("schedule")) + # set task priority mode + cvine.vine_task_graph_set_task_priority_mode(self._vine_task_graph, vine_constant_params.get_c_constant_of("task_priority_mode")) # create task graph in the python side print("Initializing task graph in TaskVine") @@ -163,8 +157,7 @@ def run(self, cvine.vine_task_graph_add_node(self._vine_task_graph, self.task_graph.vine_key_of[k], self.staging_dir, - prune_depth, - get_c_constant(f"task_priority_mode_{priority_mode.replace('-', '_')}")) + prune_depth) for pk in self.task_graph.parents_of.get(k, []): cvine.vine_task_graph_add_dependency(self._vine_task_graph, self.task_graph.vine_key_of[pk], self.task_graph.vine_key_of[k]) @@ -198,7 +191,7 @@ def run(self, self.task_graph.outfile_remote_name[k]) # save the task graph to a pickle file, will be sent to the remote workers - with open(self.libtask.local_path, 'wb') as f: + with open(self.library.local_path, 'wb') as f: cloudpickle.dump(self.task_graph, f) # now execute the vine graph @@ -225,5 +218,5 @@ def __del__(self): if hasattr(self, '_vine_task_graph') and self._vine_task_graph: cvine.vine_task_graph_delete(self._vine_task_graph) - if hasattr(self, 'libtask') and self.libtask.local_path and os.path.exists(self.libtask.local_path): - os.remove(self.libtask.local_path) + if hasattr(self, 'library') and self.library.local_path and os.path.exists(self.library.local_path): + os.remove(self.library.local_path) diff --git a/taskvine/src/graph/GraphInvoker/library_task.py b/taskvine/src/graph/dagvine/library.py similarity index 75% rename from taskvine/src/graph/GraphInvoker/library_task.py rename to taskvine/src/graph/dagvine/library.py index 4b3064907c..f66badf033 100644 --- a/taskvine/src/graph/GraphInvoker/library_task.py +++ b/taskvine/src/graph/dagvine/library.py @@ -7,14 +7,14 @@ from ndcctools.taskvine.utils import load_variable_from_library -class LibraryTask: - def __init__(self, libcores=16, hoisting_modules=[], env_files={}): - self._libtask = None +class Library: + def __init__(self, hoisting_modules=[], env_files={}): + self.libtask = None - self.libcores = libcores self.hoisting_modules = hoisting_modules self.env_files = env_files + self.libcores = -1 self.local_path = None self.remote_path = None @@ -26,7 +26,9 @@ 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 install(self, manager, vine_graph): + def install(self, manager, libcores, vine_graph): + self.libcores = libcores + assert cvine.vine_task_graph_get_proxy_function_name(vine_graph) == compute_single_key.__name__ self.local_path = f"library-task-graph-{uuid.uuid4()}.pkl" @@ -37,7 +39,7 @@ def install(self, manager, vine_graph): load_variable_from_library, compute_dts_key, compute_sexpr_key, compute_single_key, hash_name, hashable ] lib_name = cvine.vine_task_graph_get_proxy_library_name(vine_graph) - self._libtask = manager.create_library_from_functions( + self.libtask = manager.create_library_from_functions( lib_name, compute_single_key, library_context_info=[init_task_graph_context, [], {"task_graph_path": self.remote_path}], @@ -45,9 +47,9 @@ def install(self, manager, vine_graph): function_infile_load_mode="json", hoisting_modules=self.hoisting_modules, ) - self._libtask.add_input(manager.declare_file(self.local_path), self.remote_path) + self.libtask.add_input(manager.declare_file(self.local_path), self.remote_path) for local, remote in self.env_files.items(): - self._libtask.add_input(manager.declare_file(local, cache=True, peer_transfer=True), remote) - self._libtask.set_cores(self.libcores) - self._libtask.set_function_slots(self.libcores) - manager.install_library(self._libtask) \ No newline at end of file + self.libtask.add_input(manager.declare_file(local, cache=True, peer_transfer=True), remote) + self.libtask.set_cores(self.libcores) + self.libtask.set_function_slots(self.libcores) + manager.install_library(self.libtask) \ No newline at end of file diff --git a/taskvine/src/graph/dagvine/params.py b/taskvine/src/graph/dagvine/params.py new file mode 100644 index 0000000000..4e21e10916 --- /dev/null +++ b/taskvine/src/graph/dagvine/params.py @@ -0,0 +1,78 @@ +from dataclasses import dataclass +from ndcctools.taskvine.utils import get_c_constant + + +@dataclass +class ManagerTuningParams: + worker_source_max_transfers: int = 100 + max_retrievals: int = -1 + prefer_dispatch: int = 1 + transient_error_interval: int = 1 + attempt_schedule_depth: int = 10000 + temp_replica_count: int = 1 + enforce_worker_eviction_interval: int = -1 + balance_worker_disk_load: int = 0 + + def update(self, params: dict): + """Update configuration from a dict. + - Converts '-' to '_' in all keys. + - Creates new attributes if they don't exist. + """ + if not isinstance(params, dict): + raise TypeError(f"update() expects a dict, got {type(params).__name__}") + + for k, v in params.items(): + normalized_key = k.replace("-", "_") + setattr(self, normalized_key, v) + return self + + def to_dict(self): + """Convert all current attributes (including dynamically added ones) + to a dict, replacing '_' with '-'. + """ + return {k.replace("_", "-"): v for k, v in self.__dict__.items()} + + +@dataclass +class VineConstantParams: + """ + All attributes are accessed in lower case for the convenience of the users. + If there is a need to use these values in the C code, convert them to uppercase and call the get_c_constant_of method to get the C constant. + """ + + replica_placement_policy: str = "random" + schedule: str = "worst" + task_priority_mode: str = "largest_input_first" + + valid_normalized_values = { + "replica_placement_policy": {"random", "disk_load", "transfer_load"}, + "schedule": {"files", "time", "rand", "worst", "disk"}, + "task_priority_mode": {"random", "depth_first", "breadth_first", "fifo", "lifo", "largest_input_first", "largest_storage_footprint_first"}, + } + + def normalize(self, obj): + """Normalize a string by converting '-' to '_' and uppercase the string.""" + return obj.replace("-", "_").lower() + + def update(self, params: dict): + """Update configuration from a dict. + - Converts '-' to '_' in all keys and values, and uppercase the values. + - Creates new attributes if they don't exist. + """ + if not isinstance(params, dict): + raise TypeError(f"update() expects a dict, got {type(params).__name__}") + + for k, v in params.items(): + normalized_key = self.normalize(k) + normalized_value = self.normalize(v) + assert normalized_key in self.valid_normalized_values, f"Invalid key: {normalized_key}" + assert normalized_value in self.valid_normalized_values[normalized_key], f"Invalid value: {normalized_value} for key: {normalized_key}" + setattr(self, normalized_key, normalized_value) + return self + + def get_c_constant_of(self, key): + """Get the C constant of a key.""" + normalized_key = self.normalize(key) + if normalized_key not in self.valid_normalized_values: + raise ValueError(f"Invalid key: {normalized_key}") + return get_c_constant(f"{normalized_key.upper()}_{getattr(self, normalized_key).upper()}") \ No newline at end of file diff --git a/taskvine/src/graph/GraphInvoker/utils.py b/taskvine/src/graph/dagvine/utils.py similarity index 100% rename from taskvine/src/graph/GraphInvoker/utils.py rename to taskvine/src/graph/dagvine/utils.py diff --git a/taskvine/src/graph/GraphInvoker/vine_task_graph.c b/taskvine/src/graph/dagvine/vine_task_graph.c similarity index 92% rename from taskvine/src/graph/GraphInvoker/vine_task_graph.c rename to taskvine/src/graph/dagvine/vine_task_graph.c index 394f98c2f8..cdc37dddc4 100644 --- a/taskvine/src/graph/GraphInvoker/vine_task_graph.c +++ b/taskvine/src/graph/dagvine/vine_task_graph.c @@ -13,6 +13,7 @@ #include "itable.h" #include "xxmalloc.h" #include "stringtools.h" +#include "random.h" #include "hash_table.h" #include "set.h" #include "timestamp.h" @@ -43,6 +44,63 @@ static void handle_sigint(int signal) interrupted = 1; } +/** + * Calculate the priority of a node given the priority mode. + * @param node Reference to the node object. + * @param priority_mode Reference to the priority mode. + * @return The priority. + */ +static double vine_task_node_calculate_priority(struct vine_task_node *node, vine_task_priority_mode_t priority_mode) +{ + if (!node) { + return 0; + } + + double priority = 0; + timestamp_t current_time = timestamp_get(); + + struct vine_task_node *parent_node; + + switch (priority_mode) { + case VINE_TASK_PRIORITY_MODE_RANDOM: + priority = random_double(); + break; + case VINE_TASK_PRIORITY_MODE_DEPTH_FIRST: + priority = (double)node->depth; + break; + case VINE_TASK_PRIORITY_MODE_BREADTH_FIRST: + priority = -(double)node->depth; + break; + case VINE_TASK_PRIORITY_MODE_FIFO: + priority = -(double)current_time; + break; + case VINE_TASK_PRIORITY_MODE_LIFO: + priority = (double)current_time; + break; + case VINE_TASK_PRIORITY_MODE_LARGEST_INPUT_FIRST: + LIST_ITERATE(node->parents, parent_node) + { + if (!parent_node->outfile) { + continue; + } + priority += (double)vine_file_size(parent_node->outfile); + } + break; + case VINE_TASK_PRIORITY_MODE_LARGEST_STORAGE_FOOTPRINT_FIRST: + LIST_ITERATE(node->parents, parent_node) + { + if (!parent_node->outfile) { + continue; + } + timestamp_t parent_task_completion_time = parent_node->task->time_workers_execute_last; + priority += (double)vine_file_size(parent_node->outfile) * (double)parent_task_completion_time; + } + break; + } + + return priority; +} + /** * Submit a node to the taskvine manager. * @param tg Reference to the task graph object. @@ -54,7 +112,12 @@ static void submit_node_task(struct vine_task_graph *tg, struct vine_task_node * return; } - int task_id = vine_task_node_submit(node); + /* calculate the priority of the node */ + double priority = vine_task_node_calculate_priority(node, tg->task_priority_mode); + vine_task_set_priority(node->task, priority); + /* submit the task to the manager */ + int task_id = vine_submit(node->manager, node->task); + /* insert the task id to the task id to node map */ itable_insert(tg->task_id_to_node, task_id, node); return; @@ -504,15 +567,13 @@ void vine_task_graph_compute_topology_metrics(struct vine_task_graph *tg) * @param node_key Reference to the node key. * @param staging_dir Reference to the staging directory. * @param prune_depth Reference to the prune depth. - * @param priority_mode Reference to the priority mode. * @return A new node object. */ struct vine_task_node *vine_task_graph_add_node( struct vine_task_graph *tg, const char *node_key, const char *staging_dir, - int prune_depth, - vine_task_node_priority_mode_t priority_mode) + int prune_depth) { if (!tg || !node_key) { return NULL; @@ -526,8 +587,7 @@ struct vine_task_node *vine_task_graph_add_node( tg->proxy_library_name, tg->proxy_function_name, staging_dir, - prune_depth, - priority_mode); + prune_depth); hash_table_insert(tg->nodes, node_key, node); } @@ -550,6 +610,7 @@ struct vine_task_graph *vine_task_graph_create(struct vine_manager *q) tg->proxy_function_name = xxstrdup("compute_single_key"); // Python-side proxy function name (shared by all tasks) tg->manager = q; + tg->task_priority_mode = VINE_TASK_PRIORITY_MODE_LARGEST_INPUT_FIRST; tg->failure_injection_step_percent = -1.0; /* enable debug system for C code since it uses a separate debug system instance @@ -580,6 +641,23 @@ void vine_task_graph_set_failure_injection_step_percent(struct vine_task_graph * debug(D_VINE, "setting failure injection step percent to %lf", percent); tg->failure_injection_step_percent = percent; + + return; +} + +/** + * Set the task priority mode for the task graph. + * @param tg Reference to the task graph object. + * @param priority_mode Reference to the priority mode. + */ +void vine_task_graph_set_task_priority_mode(struct vine_task_graph *tg, vine_task_priority_mode_t priority_mode) +{ + if (!tg) { + return; + } + + tg->task_priority_mode = priority_mode; + return; } /** diff --git a/taskvine/src/graph/GraphInvoker/vine_task_graph.h b/taskvine/src/graph/dagvine/vine_task_graph.h similarity index 93% rename from taskvine/src/graph/GraphInvoker/vine_task_graph.h rename to taskvine/src/graph/dagvine/vine_task_graph.h index 956b9d8200..488d520bca 100644 --- a/taskvine/src/graph/GraphInvoker/vine_task_graph.h +++ b/taskvine/src/graph/dagvine/vine_task_graph.h @@ -7,6 +7,7 @@ #include "vine_manager.h" #include "set.h" #include "vine_task_node.h" +#include "taskvine.h" struct vine_task_graph { struct vine_manager *manager; @@ -17,6 +18,7 @@ struct vine_task_graph { char *proxy_library_name; char *proxy_function_name; + vine_task_priority_mode_t task_priority_mode; // priority mode for task graph task scheduling double failure_injection_step_percent; // 0 - 100, the percentage of steps to inject failure }; @@ -30,8 +32,7 @@ void vine_task_graph_compute_topology_metrics(struct vine_task_graph *tg); struct vine_task_node *vine_task_graph_add_node(struct vine_task_graph *tg, const char *node_key, const char *staging_dir, - int prune_depth, - vine_task_node_priority_mode_t priority_mode); + int prune_depth); struct vine_task_graph *vine_task_graph_create(struct vine_manager *q); void vine_task_graph_set_failure_injection_step_percent(struct vine_task_graph *tg, double percent); void vine_task_graph_add_dependency(struct vine_task_graph *tg, const char *parent_key, const char *child_key); diff --git a/taskvine/src/graph/GraphInvoker/vine_task_node.c b/taskvine/src/graph/dagvine/vine_task_node.c similarity index 89% rename from taskvine/src/graph/GraphInvoker/vine_task_node.c rename to taskvine/src/graph/dagvine/vine_task_node.c index f3e3e4103d..c563ff459f 100644 --- a/taskvine/src/graph/GraphInvoker/vine_task_node.c +++ b/taskvine/src/graph/dagvine/vine_task_node.c @@ -20,10 +20,10 @@ #include "vine_manager.h" #include "vine_file.h" #include "vine_task.h" -#include "vine_task_graph.h" #include "vine_worker_info.h" #include "vine_temp.h" #include "vine_task_node.h" +#include "taskvine.h" /** @@ -51,7 +51,6 @@ double compute_lex_priority(const char *key) * @param proxy_function_name Reference to the proxy function name. * @param staging_dir Reference to the staging directory. * @param prune_depth Reference to the prune depth. - * @param priority_mode Reference to the priority mode. * @return A new node object. */ struct vine_task_node *vine_task_node_create( @@ -60,8 +59,7 @@ struct vine_task_node *vine_task_node_create( const char *proxy_library_name, const char *proxy_function_name, const char *staging_dir, - int prune_depth, - vine_task_node_priority_mode_t priority_mode) + int prune_depth) { if (!manager || !node_key || !proxy_library_name || !proxy_function_name || !staging_dir) { return NULL; @@ -73,7 +71,6 @@ struct vine_task_node *vine_task_node_create( node->node_key = xxstrdup(node_key); node->staging_dir = xxstrdup(staging_dir); - node->priority_mode = priority_mode; node->prune_status = PRUNE_STATUS_NOT_PRUNED; node->parents = list_create(); node->children = list_create(); @@ -195,62 +192,6 @@ static int _node_outfile_is_persisted(struct vine_task_node *node) return 0; } -/** - * Calculate the priority of a node given the priority mode. - * @param node Reference to the node object. - * @return The priority. - */ -double vine_task_node_calculate_priority(struct vine_task_node *node) -{ - if (!node) { - return 0; - } - - double priority = 0; - timestamp_t current_time = timestamp_get(); - - struct vine_task_node *parent_node; - - switch (node->priority_mode) { - case VINE_TASK_PRIORITY_MODE_RANDOM: - priority = random_double(); - break; - case VINE_TASK_PRIORITY_MODE_DEPTH_FIRST: - priority = (double)node->depth; - break; - case VINE_TASK_PRIORITY_MODE_BREADTH_FIRST: - priority = -(double)node->depth; - break; - case VINE_TASK_PRIORITY_MODE_FIFO: - priority = -(double)current_time; - break; - case VINE_TASK_PRIORITY_MODE_LIFO: - priority = (double)current_time; - break; - case VINE_TASK_PRIORITY_MODE_LARGEST_INPUT_FIRST: - LIST_ITERATE(node->parents, parent_node) - { - if (!parent_node->outfile) { - continue; - } - priority += (double)vine_file_size(parent_node->outfile); - } - break; - case VINE_TASK_PRIORITY_MODE_LARGEST_STORAGE_FOOTPRINT_FIRST: - LIST_ITERATE(node->parents, parent_node) - { - if (!parent_node->outfile) { - continue; - } - timestamp_t parent_task_completion_time = parent_node->task->time_workers_execute_last; - priority += (double)vine_file_size(parent_node->outfile) * (double)parent_task_completion_time; - } - break; - } - - return priority; -} - /** * Update the critical time of a node. * @param node Reference to the node object. @@ -512,23 +453,6 @@ static int prune_ancestors_of_persisted_node(struct vine_task_node *node) return pruned_replica_count; } -/** - * Submit a node to the taskvine manager. - * @param node Reference to the node object. - * @return The task id. - */ -int vine_task_node_submit(struct vine_task_node *node) -{ - if (!node) { - return -1; - } - - double priority = vine_task_node_calculate_priority(node); - vine_task_set_priority(node->task, priority); - - return vine_submit(node->manager, node->task); -} - /** * Print the info of the node. * @param node Reference to the node object. @@ -551,7 +475,6 @@ void vine_task_node_print_info(struct vine_task_node *node) debug(D_VINE, "depth: %d", node->depth); debug(D_VINE, "height: %d", node->height); debug(D_VINE, "prune_depth: %d", node->prune_depth); - debug(D_VINE, "priority_mode: %d", node->priority_mode); if (node->outfile_remote_name) { debug(D_VINE, "outfile_remote_name: %s", node->outfile_remote_name); diff --git a/taskvine/src/graph/GraphInvoker/vine_task_node.h b/taskvine/src/graph/dagvine/vine_task_node.h similarity index 88% rename from taskvine/src/graph/GraphInvoker/vine_task_node.h rename to taskvine/src/graph/dagvine/vine_task_node.h index cd8e98afe3..12fea69230 100644 --- a/taskvine/src/graph/GraphInvoker/vine_task_node.h +++ b/taskvine/src/graph/dagvine/vine_task_node.h @@ -46,7 +46,6 @@ struct vine_task_node { timestamp_t time_spent_on_prune_ancestors_of_temp_node; timestamp_t time_spent_on_prune_ancestors_of_persisted_node; - vine_task_node_priority_mode_t priority_mode; vine_task_node_outfile_type_t outfile_type; prune_status_t prune_status; }; @@ -57,18 +56,15 @@ struct vine_task_node *vine_task_node_create( const char *proxy_library_name, const char *proxy_function_name, const char *staging_dir, - int prune_depth, - vine_task_node_priority_mode_t priority_mode + int prune_depth ); void vine_task_node_delete(struct vine_task_node *node); double compute_lex_priority(const char *key); void vine_task_node_prune_ancestors(struct vine_task_node *node); -double vine_task_node_calculate_priority(struct vine_task_node *node); void vine_task_node_print_info(struct vine_task_node *node); void vine_task_node_update_critical_time(struct vine_task_node *node, timestamp_t execution_time); void vine_task_node_replicate_outfile(struct vine_task_node *node); -int vine_task_node_submit(struct vine_task_node *node); void vine_task_node_set_outfile(struct vine_task_node *node, vine_task_node_outfile_type_t outfile_type, const char *outfile_remote_name); void vine_task_node_checkpoint_outfile(struct vine_task_node *node); diff --git a/taskvine/src/manager/stnPTyT6 b/taskvine/src/manager/stnPTyT6 new file mode 100644 index 0000000000000000000000000000000000000000..8e72ad4194d84b33a7e9579a081911edd52e2da0 GIT binary patch literal 1422314 zcmeFa3wT`Bbv8W0PH>FLk-;HuLYhf3E;3lQ6fUue0gWxenKHIjTLuv>vMh}(1zA!x zGqxOV#-p))auAZsRkw{HbBr@-B|Igq54{306%l1h4Z^p+y5Gnn8&)MIP z6#f0{w#du{+u!-`x2HvB>c3MyeMLn6{{HO9Ddz8)OCzV8;NPs}r$zpE{X6yLO_Bdq zfA78Pg^2xo_^%`HJ;6U`{K81-U+uXIBSn7?eL7P6t4HK-_PPrr_V4zmBeTPQ-}`A~ zcIfZ4#Kn=f{rlG^Bd49f-|6nnk$3$2!LK88nvcEn-@ZeUIrw`Y_`bs5;HK1Y*WkvD zeVI(EyJI+&9?TARr8+W$na=*mroMqx$HvZq&JC&Ij&y3EyEr8?+&Pf$K~|=3V=A&C zmFd_#ID9>xJ2!Rq^>?oCPjz(nrLXT;za^7ON4it}sZ7cg>FMiFMN%VunU3z%`s{{| zjj41R%~WZit8=Kct1q)9fUF9p2Bejt6p_%_q*WGLu9~N-4W}~M;Q@5Ii&WUsk?Bld zUsC&RikP;C2ZsiSGkt>to&A=6B~XUmrVJ}9jME@sx~n(Uo$XHz2T22+Ag+uADI7ep z_|S0Q;4tVCl$Jhp^rw0<9bJRjfe`4fP7LYb28`n{daE&1Kn90W!(kvpo#`}&QV+M3 z(J37=Zi0;=@W#P`z6`J!*r2e`&mEhGJBL&XxGLL^zGXs%G*k5rq=8j`M}Ob?;m%>* z55sXieFNPlrz4ZvIMm;n(WF@4KiGABNAF-7JlB`$6%HKC>fQ#Ks{QOhwG=cRkgj{U zvu^<10KW?4fYfL@OrGmgsiDsPzD=o)zJUy=X=Vxu736VJSd^G$x-~SM!lW6-bn6-% z7)W)IcQolVRRY%P!nEi}_ku^dOz+a!jT<|M`#xi)QfMUCcMf!K?&}6U`ui}W6;N_b zs;7tT0ApJu$ixU_G?3oxzy?eN2!x?5QY~UCT~*P@sc8#_DoO<-^>mab%FOJPex}Pa zG3hpdRe-aBg;B`#fxze@m^QTm%mw)&OsAi>pzz@J*`bcX;f}81lw}KpTVGm}wj0UlIJGArIq~$qs17?dJmE=m*`ZPO{=|a`Q$+lWg-!OWCc`%41c5OozEEhCu3vL|j)?Faj zAh~mdY}}OvLmO$M32Vz3928VD4JfI?=$Xz|%Qxto{I1@?!E`DFp$E)BpsJa!&VjB} zKj*SE)K{XrhEp3M^o?|>SS8b`d0T^`un>_X05gMVn);)-R|A6`n>)i(PiB{;mnxhc z>Lvq#K$OVE0HBRFsT59J*3O28L-(a)qoKyoFjy-jm9qogsjmLc;c%Lvn4U9%2sat! zRmxNJpjMOio!#Ang~2tuo&pG*k?!lJa<=1Q+SCJ}EQVqdq9?U@-#>gQlnm$EZUwjHZ7uYN0%Es9_vKk?gQm876^> zNnvPG)@LDZwFWn77_hz&6-+WTbZ1G)A>A4?7VCP*2|o%m^#H!;{HalIR=e$i^X=P!Uy4_of$^cBBRd zvm1H?Fq38q^ykj@P-te|BA+FRsAo|Vi^o2mN-PE>36iJ`LKKxj4);_gph>Mf44DsV>MdV^Vk{SwB zSXGr&DX|)CF1al$<~sVENsWXDgbgZtp#x5n>#bA8R#pZ^l}zw!-|*nTMr+JVH4bum zpmPW-(XfUO@<`agZDG@k{S!91Sl@?;5{Nik7Tf~oL8wevI7%tbVY;x&NjWeA z){|lWV6rjtEW9*KOtav&iQ=y6DxTx4U4|kPF$}|jZ^y!*%OY?r*yN2A2wkCGYPjN! z3$rR01a!|8*=QDw3SmB{jkiH+QGj(=;nmSI3|l&E{z)MYf4QV9FHFM_dX9cR zg@h!8C4VnC5|#)IITswMaiB{v}}}#G+W2!9-HvIRgfaWR|2fTn8Zp zdOM+Ldpid<7|}03>*DYB5d)K`k5_5;jSxpZU_K}GEh?+X}%O0Au1(2hPsS3KDc7w z%3(7ck46TElRA1+omldzHj4^}48Y>_5;sRGk#nHcsJ6^@Z;`l*wHw zUmM?*vW)($UKwjDXwXbp(pWE-B3&_^!d@_) *yuaH~9Pb^&802!n@fF3L#F-$V8 zEU@y)2+7nL@;d9~b}(7XX0PaCELb`^2egbL3S=uFD3M`~A%F5D7DDE_F_Q={PC?dZ~6Z_K2 zPimxYP@INgNv3un2D>;^pbBDnmL{~Ap1s3^nT+A{@}i=;up4FV#mYs}*)F=;t2Id9 z1J=>mwC;I1kj;HV#vxw~h%6By1Fj}3@dMfCV|xMDA8|blBmou7Z%M3m5r4+ zHhUBT6|#$%n+nAM5A7yZs3wd5u&Az3Mnf`6qe2O3!?YOIrUnVs1=wNf3><-j8&jKm zQ)VGh0Ujw-Odph^LkL$E`Gewvg zR~v?8;ozd;G?=MnX&CzxWN6e;Kqs68aDR7$_(ylfw%;-U4NJH5~t5|G` zmBBWpOwQPxa63xy*et_^;C`DH;xVQbQ4k5yaAelU&uhHk)DaU^&$sgX$U!FSt-Ci&GAxu|BTDjK~_B_Tb2WlQ5! z{ECEkEWS40wswu{{l?Axtj_fx?X8VOx|9Bbd37k3^!j6SU{**<9xU| z5^?>9SkN77i_LJo|4>=^@yuDS-yECKyrj1``@I$UW0~2mKQuS#y=e+$_s6FuCim=O zvANyuIn$+?>}hW9vASmO_|@0OuZdq9zb@XfPt~9YXkI#ke;4ELX8a9wViN`25}nuI z9Z_B&`AOf8{GfM7eke9mzahgUtw(uEq5yQM{@DYQzxzU@%v69i(1$w%tQn6l}g~Y?uI4d0;TrZ&v-4RA_z&ME- zgPndLZp(ASJKCM<>7)Z8JJ8=Z0Ft5K^U{leeJg}ow9N#KMAiOQcWE-ZK!x2MSuLA= zQ4=f@mG0VFS0q-pbgYc8Xo)P3x5bmIF6(ett!azYUlBb^O+u2q`>aTmDQ3AtBAylL zg8CTjPuFXwh}fy=g|bYK*4G=aOT9`*4Zx_^Zr}RB0r-6JY~m0|A3V}Y5BCM>e|D_32)GIHsv)obFHx2W9J zEp4k4Eo-FYmGNX!6>W-Gxt5>Zy&9|+NuUvC!i%GpS(caSBkz56OqbZBc z^7zW-Ey-j{b0nEqwmQE0D&Ui7i>!&SY+kks*o2T%z3b5Bgny|C6uX1a>R)8>!iI$n z(dL#bT9T_;SG25bi-doNb;rW4NX6I=>_PVpMZ!@f_>Bc3c4?u&kd+;{9=fx5Cpp4R zpv{=0+5YZWoM*9!155cA4JbDJCC@@;S!MY>XvaFmA-QeP3qhG4arIZzMts@bW~co7Y>`n2rA*CXB}(}#6*YBugrcs z8XyAU7^2#AQydIY-?ar+q+$nbgVz&W8t^pHrAi+dzaBAAgTn!txa|&MHLM30)CIA% zQZs5IM0?JIdPTK*mNclOP1%Zh{ezv|^`c|ay_kOZ4KV2~V8V^U%;fw4C8c+QXOg^L zS;_&C)dZm}8Wa?SSXlXy@ijP1M?BL9Qigxi5!u(2k*pWDNdQ4ME9#8`)ble;OQ@8SEPDk09150!9LtAnDUk z+7X%aqL#H3z&2-v*tLOFhU0{F6ij@~$rOwQF$OrKGzQcYqsqiJtRCT$F{7&z7_)3GtV0nM_UPzyn$>Iusj*e4(rv!EO3^^*8I zRaxz?uz+b*EP1p!FgO&+)L*QUqZeHi-5j~T55~~E?wL%-5N1-*i=!g;+v003?^xXu zZ@$XPUrhodoGg{P2NAKRx{f6?4IuTPVxjnK5D|YyX&4zfJrq?YkLeV)HA@SEdRR%9 z6xWGt0bS-&94x6LU5SP$-E7vHk7*DS(MAR%+Mf1deKx(tK74u*+ATm`xk|#%Z~~-5 z1qeyD0WAcoANwF{Q$-hvivUYO)Ec%@(l=nd)JSh<7LumBL+eD%2=HMrx-r>$t{@!5bNIWiLk_Mpr z(uPSGeQ5*$`3X7t(MmyhV^jvI5&mt&u}Rx3LD{zTWr=DW#!YFq<7xiW~_RpLdS=!rpvEhj*!s2B0%v%-;gbT zSxXy1xovH&i{_o(y%2w}Eww;y!=TaZ)mnrF^`w^=I;F%!qKM9hUXa~Uoz9h}tTv^v zrj3-=>o)bJiply^N?(0#q^JJkkb^^bzaA?$^4mh?N7)iFCMn0beqQ%HaAom=$(CHG z4AzAjLTNFTTbim1D|)fx9DQ&iqeHtR4I}fw8raZ;RG6twFhC%oSOm)&GYPN=gsfd? zm~f${j#0@P@rBUg=!}9-p%fh)P>jCeGkrtR`X$l&KV|#uAY;4CA{hx7?`WW8EUHW< zzhQ2RvVd|>yb*70O(xX3h~V&vUm3q5zBb|g&h?(bddd5x>wRaf>piyyUXZ!k(UEX& z-oGOMLud3XxK9%Pjj;yn=}5qR!KWUSnQ;Bx#{tsKPdTHb_@Sydjw9b4yNInFb!NjW zqSB8#weMmYst$bL_4f5Riw-+GUR9;0$CBPpY`N*N{CIXFD)h%@pxCRMIvbC<-s`M6 zan!j+z^KBy9oO4m*iG2G(7I~c0<(?&ST*|6=FLE>v_ZoDF4{Wi?D$Ina>vBy<9gpu zdavuIa*xe*y(59AW@;V^Mw}0C+RKLO!I{sr>3URKOuxuXR=ji(f`SZ*bWBcm@|4WfB1hfgyC~@pQ(*FxBrEt?ASS* zeyrEWkGjs;%Nl@O)7(w(0yUcE+SDed)~VDwNp)(47Ob;oQ2PC{sgK#!J>a0#YiE2k=r1^Tr{T}ZeD=flOvl63LaTE+)Me!IkXcLscS zQL@86=ffD_BV^^sYh`iO$Lhz8CB7x%%7Xw_udWp;ylBOUMlVBRe^zG3!hH znhF2wG3L46Pu;PM!4!_oo7rnnXxA?0<_~3_OZcNPlFN)|QyoTS*<3w>YK)r}Gp)|` z+`8iQxlD%yaTgb-yG)NV?I1EityZ_tOO7Yg7!FK0XOG4zv0#8wz>1sB6=>CX>dA^c zCEN#&bBX5x0(cgv%q8UsKEe@!Tvl>lJ5^zl zi({2WV^mbC_){ZrGIHHT6$67@i+p1iDf`4qe*w5K>UO51*?~S7PEqX{VtaAyX5ysE ziNj$fe3I%I7Ht8owDq~XXkQvCXm~g~lv#*!ra#%VQioAQ5_5qYHD*baZky;}PjueG zkN5XPS1elub10~{nJW|oN1zpPDtAZI@DB`hr}<=IWO~u60U+ID+x%?q9Hy7~LRgNp zMA#73Eb-PMji~r>Y5KBd%+(9Ff{<7}y=r>IYZgn9LAWC;1Pc6Wh zQ1C2chl&~U(-}@57zk!z6k(ye5ov}uYlz)|l>#I-ssnbRN*p+3l~DrJgZsjy_ooIn zV5R^`IoF0}xf7u&OQOxZSUyqZnvS-uUb_S&vrSQBksl~~!f(3FI0 zIZ7pk}SQ#&OTbdULek#dAQe9W(yRm1%W89HIag0Uje`0sF8K&$U zkdg0N9IzuIsk*^o=p(}m0aJ5=aBDV=Ap{@7=|6x8(%YF4nvvnL z3!)<*NgM>#0Up3HIF#$G1%{u%bhe(7AgGz{!X)kL#lV9-xP?s^FiJZCsGdNg@K7TL z4tukPF@Fzy#BnVsVbaK^(}Ji*0}G=}6pPO^no@2!yYi4OnA5{i-BX2esCuZr_GgI% z03&UH)!h|hW%PPLs0DXMF*xbc@6n&$PFULzQyb{%6W0u;G*cR>? z6-D})5iL{2i;wYt^<#zY2TD}D^Z1t+vpR&rs5E+6S zCn%>LbVPtJri4;*;Sv!{dIboBJe&#^XQZ*BEn^`9eW!Otqsy1Kaj%b3fCsIxI^%?+ zHsuj)a22(Jc%$blalm6=5Dt*yXa?ng4o-}8Z_$Li(&)h<#PVSoh(XmIG%alGOK;>X zztCi`F)ncd$&{IwULbPFj8RdRv}BG(mjVF=>3ue|sGHY4UpRj~Tzy28Zc7d_)}_E6 z3)ZXWrLVm(8c$={O}QcqV67TCA&r-*lAD=Q6MJEpWWw!(Jr_c=V)twTzf4%QJcPH_ z6rr&8Hq0VDMGx1fNNfhNf^woyV4;ZV0i6ldpleRB5}GL9r7MM^4KuZ6CkV;b*zeIm zF??IQ3~&qYLtwu`<|93pmRo6l(7Og)&RuwJe~(rW%`%@)q@vOUYsK!<`;OHclb94y zd~6d{Te#(Op|OP^8noqv`Ih1V7b&ZmfGsVT5vNj6(N$3$cw%)HT}Akml{2gi7E>;< zK9Mym)B$@kKTmI`Xhd=bYPXq{M{l=9S0CPflhKSs)Uoh-8(mW$+NDg}jWtA?a6=xF zgzbUH>a9z$J~JrWKpI{Okrmt|R1*t(|2XN&@@Ai37mL{Wj|1lH3KTYOi@+{S^rQ3A z9~JFOnYv*BCL?@Q5#w+*yRT7(f=AO>!u0j@si`WUjom`WEX13{&Xav1Ixxd&e5s$B zeIlm3b#aG;v=(qWXhhXGvGSPAuS^~eM0_;h*o}1 z!Bg6k$RW`)Z`WY+U>4@3+8id|kx#%AdU>Szg z?GarM1|4rhse?k49;oG0MBL3V>jP)7e3th2T^wE7 z8STZk=tUo0q?^)v9Ur~;PqHbj%h$;6#?nQd*o@Ghl6o1K^0tzUMM<5K@`_Tn#+7pG z&Iz<*rjgmL`RK)~tu0m5BK-&HbUnJA86NDf2YX?=dDUyze zmlkj*M3K5fMor>Gw0lrkSa=WCs2cb+aGi_XQX9@u?wk;W>Yx}ZLNT*b!MVVHvfe8D zhL1}Jtr@R{Z0TrV+VG(mvvQDx=L(AZ(hv=mTr3v?ZS7p3oh-q#kQNrnh0(-@0lEH$ zasvwj_RRWG%ti`DLopuGuLY2;@b!y@Y$hnp4z1WNzbw$!3nU+GIlJ4BqRQ%~T9^F| zUBmX38-I`tDwGfS!TQGN0Qu0yj!;A~=4#lM91qnK&WiA+82m1Z(=>Pjf$v4m{fmsr z4g~a(l`moz<@1j9^w~_)aaAjVz$h*=gsJ4p4Ap;-KV@b~+!Jtx*4R58Y zE{tv&%to;dPw}df1a4ZU4yi6WTBc@Zb6;1A9>_IT%M~rLIG0qxK@54l?i+WtMRx@2 zI?8r-Xf+nN5~mO_GaL!jGiv!KoyLEFtA?EZ2^}3#x;A22`l-}p;;N?7^ivQMa>Nfa z6s>QGDsu>IYS~Ja3iE-SPE)SLkUdXUCm)!(27cf|U9VL1LRI+$sj|7~!q9`s=XEbI ztNO6w)YC2056&wsEFR7!pYE`N+~J|-KbimZG!UEQYS)nPLiR&gX8RUYVM>4Pq2l41Jh7n3eZVg}_ z#{xbbwa7Q)zr>^C!uG6~qMp~6@%UPlxHLJD20Oo6yyvKp&Fa8zi9i=4WU z9bho5C?4W(uL@HhIhw`@?7~Eb+YjIsw6L6!`Dp_`jrz3~M+xBs-*+rcx%l?@VpO4d zarFdMSt&fSZVA?Hn&D(}F$N0t2H`J&Hi+7`LcC=L+U6CrDI#WYc$#iM1%z1W&>@j| z`V5GSMid%Myo6EkkZ}tNU#J>1lf-kq9uW+fpTl;nfvl;iv|q?xP$3K|+yujt?l*b? z;4tDAU6xaGx4ckkInEXE6Nw5ju^B@iCDv#pWU*!IL7_#AB;6AFFs6pEK{HY2u@2A*HE=t5M>|C?M>H2c;q_tu4F4;F%6%+{6GtFh*Us|E5 z(24C-#z%06SDFl$zpS);w9v@%yQdLt|4!*9&aKRmopmSw7L#v%jMOj@5wee64SpFu zt%Ya*^KI}sqZFUo!vBTxRlsK#Y`rm7o64Syuh|2cb9Dz%+LBpOrf>~_lz^@+T!uu{ zJgqysA77_OvX3UcALHwObPioS&nf7kUgvAK`5 zqYU=$WBOE4cWm(r?=e+h^6VHOzC{vLUc!HvF$jr?=`psV62V2I7h&UFXT~4FZ=L0i zHOH{q&YgPv!u~qKZiGK{S+Wf_@k1N?&giqK&5iB4q`#Z{`H9>mk&$N@oPdK%|EZdk zeAnN~P4`62^jHgyHr@D(xHEXNGkY4pFROKCFRztfQT&R^uLk^TkY7#s)s*lrYjwTl ztxQ3^mWX7bVvDWVVk@@TiY>ZgOKVHxvBJ3^JK`*WnI)P0*3mhWFRC~ai5Ebe!1S02 zPf;D=R{nUxn;v7276%8g9Bk#ghJ)?tE7DwjIPrh$dT(lu5)n`v#hAJ}9>euUV>6II z&U{6+l{;RO-JHmMJvPDt%G~IVeJMt9pk=~OY*0xs70Wr50U50SAYs@VD2KVo_Uv5D zUVwQ>lq3{I=Ii7OAO<6{fYz}`r{hZ@mH@7VS@<-Fsgl64ziXUg_)HL{)iT!#h<&a% z5fE@KSgcqyk*sUMZWU6arKzCw4A(n+^|kv1Gxc8_(P%A=uLVc(qB>9qljb}=BIfoukUEwsao+zoZp zGN)OvcKnmxD_|ab5Ya{17p_K4ecmt}Y9t7#_(09|-<9T-vY_s(ialGJy8+vx> zOfUZi=HcWE>Je?3kL$gG5Wb8sBPS&$*xCCb%62rIX2{BUB;t&5XW_X-7;(gQKBue| z%xn(YT$0zjEruvCNYByOR{6dywiDl@lL&$2aC*xY_f(a;k`;}%svOw_`Q(EB)MY3%JbXxzN-Y@FATC( z=T>s;3&L#@u$S~+hRRUb=8m6~`FOR-Faq`k9(FQZ2Ubz~z%*#&3FN_<57K$jc+4`Z zm%AIx**HG=_dUDH%R8fYa~%AKwxW>h{YarRIY;SV%DWl4#nYV82ScSeyjF)Hl=690 zXSAWJi-bRS{mU^=*OZT*Dw6QG&j3*&#~2?wCWAQD6t&|zUN{E3jC#SCwpw1Ie>b}R z4T}{R+r`Ew8V^aN?Q%#KeMlQ~^j`qP^y9pks9u{5Y3Ah5!6c>*u7WIrG-IkI0P6;@ z3iQh?Z9FCvx5z*!?*_3VBekU?;d;4V8H&RDrF%`4`gexNcmyg3{V4nnnx~8s&S)K= zz)aCrB@j@;dxkEnQ1|D=$Xp<<0b!?-?b}W?fEkBDEo%2qk*vzFKl~U`^qf9v54C*z z0W!|5Z#6A*BF=W|>&8P=rY4rVZ~MI{z+CsupHu!J02J}s3NB(nr^gcB9v#-niii$A z`;KpeJB~Wny}T`5Iy+xh1tj|U0C3q8 z1i_bLU$_W#68l`N2$ z#5`ItxL^Ut3^@MZfEj`DM-Vhn*s&Oj3>MQK=d(1=;)1D@@5b*$*I(U<%;b_EIJf;0 zXeYfNxMMfe=t1)GRKle6=uwKd{EN4IU_ZVQFx@yFQu%8BuJBjZuJA8ya=qsp$NjZ- z-dMsYs8%6(FC;xkK!o2ze(4tdud>C2e>2$p1-I*fI}P+R*r1rZWS=uS4Tz%=+A82L z3*Js1152>qKYgwsETj$NywJ#J@&hTDe-TK7v7b7M#mr;mrZFD`zQjp9<@yqKug7c8 zTYu>Kiv?iZJD^&7geJc=84C}H>`?NbunF=T00}0&N_apqjmDI$ydCw4iJpD!=q!d# z%EAiXV7TPLyPd$)W_HI-a)eE1UA55~2D z;%OQhD|}2)Ze}kuzl{g{wio<1UPyr)J-eE{srh<^GI>AnNOa|ehrCz(73G5tQ^1|S z*J=$dtCHt*)A~@uzYo$>p~r#8;@|B$=+3FpD7OA0UlmP-j4clHbH zsHMBVxU4tcE4D9HJ5tOc=nUDuFoec|X85DdFZerTDFtQve%Eh`x_e%l$7vG`{3O`V>4EeqPh~gz|^f(jS6}v^2(EL%{OMUm9cE&l`=dj5CO1H9-K$Tb zAr@nKNGr7HkVl>C{lfJ(X}geXoTUlxF*_jfbMkc<5Oi@?e*ET-aF+Y&xuV?_fuIPD zNw?(j%mUglb=F1Tk5F$Dtk|c8lvj99Ta3 z!G+4@sSDi!6g!tVL|uQxRT$d!TuC}XKIndOZuTdWcX7hXT5pQeHct|&%7x9MJ+_kW zet&G%)Hi#qys;}DLi#{U<2c64UxV`(`|ZJ}+-o)I!^JeozMu3!-GC%&C8-)@L;3^J z|09l}JC+}Rokq#n-AAUTs2cVsyk8~9#_r)Goh-?*{O9poGd5Q)ZnS81ank|V&8+~m zTZ-K^jxRpmr@!yzx0wyVFPb|(gZJ{Z90-l&4>^`&sl!*B`*yAct5VKjt>p`DScY)W zzDjV-zQ#!!OZtMX>f_e{2&2ogJ8R@rX(R;jlA3^QE0U-B-AJ@ z!}BgSd=SW1JQr5^3{T0HUZ&FXxZJe!ryie z9uoexaeOr%O1y~$I8@;2crqk`qEy68__wmF(49{v&bhS~Kb@UVD%vSsnehJ@tI*Zu zI&il*6kw~M?p||p5Pauv`!?$5j^E;p?!k|XXbwB0e+0<6<2O6`>(CPsoc?KFyBS%H z;~}drur^e0_0)kxwPr|BMv!b~3GKQhcN7Z00LjC?PY_A*uH&}xnI94`&X{t+9aD6ufI9~QUTOwAq{H-BN-A8P=?^0 z(u?HyC-a}j`op=6>^6Bw_&}~NGfqcu4_bBePh_x8|Ce?!C)$A3?Rs2m6xl)kCuN5U z;@$d8WYNA{_pI<6VKnMft>m>QQaiG^Oax$U-IVaLSOpWgyMZfkh+K_2YfMq8R#N`K zbwV#Oo3txRY&O@Qp=cTch>`?&BJB@Bas}?;M_b1`zb0(ZElz|%+tseV4JH$ z^3k1`#%7o!q;UO{gT~nY=2=D0jpH}~?Y`kXK-VO4*xu!sh;W6%Mud`g{adgfFqL_? zM#}7&LOm5@)52cQxcXXyj9NbxHHP+Sy3k9n;QNo(ff^bz466578`LUkn0zdQ=HQ0k z3+7kw*-3tceug}7|F|d{<^lSL*^99Um_sL)Uj0yM{mnR0cK%J<;)Lh#=m#Ny%FGG* z#k6oA)YkTIm8gQr1)$h@S)g}bz7$)GgoO7^7t+c5f$A7n)5>!sOssqBuZ6HWlsV0? zHcRNN+)Z^6Y+5LdPkN!R%Rh!2dw>5F02*O!J3iT1~^wq zD%b4f*!501eg*mr6<+9FZfa=r2Q?|YDYt9-gKo{ish6xi!_vte6M^;rnQyYp|M~oH zLIH^{HPP$lWnzr6tywnB9osiWTiq+VTtLkXeW2;=jHrX1HX_sC3r!EQikvE=2b#kqnF2}Gzx z7Pms#{?ItM6MKn(O%C=;ai@aFLn;F-RJnj%+$0Ow>3Vx?G**WMnDE@TN+KvA8pRS4 z+lf&kh&_^76zwa#f;;y4SY1N|`;~7by|>mVe6Q=s9eFCK&6~JHn@^!Yhv5 z1Xt@xQ{uUQ$68h*U6S5W8+79BgB9|f zW-G#BkCYg8q?0(Z>2buKUwkTqK9(9)D?q2YE6`w_!Xg2mQ(cj{`l+j+HS=T|KmSY7 zlea9S^>e3;lr?~y{A^GiGM@t)xIB~{E$HiR&W3#->R;}~O5H=}o`++B?ZuZIzUjXO z&ik?+7z$%U6bRs7M0WfNp8^LeKQ{?rc0QA1;-QruYfz8QKph3e^w7Ee1E^hCj{mR@ zPy}p>?`Boo{&YOBy?}18%k*MEVZha%%9Wz_KRYpWiaW6%1;g{Dahz+UAmC$h{!kF| zVGNJ{SRLF$&Rtju;38(k|G3xyR?8p!h!QyXY?i`cG8*z{k=x-ZNx;lgxv+hfVY~5O zJ;$O}3}76dEpcnjhyAp;*?XgCjJL8i+P(3q{c&#xM`H4yP2aU!GFs%1G?&g`;V0Kh zYcw5Mcfrx>H%$jtOMz9d#tM#6dzWexbJ7ks=2 zUIJeQ0?m8iheglUR8^+1nbG+dGd*VXjTHqHF870H~Agc7dI3fcVgveo^6fq?tMRHqpNt>#X7RcUQ{GRvif1jADP(kJ@k(=L-o0o z|BCEs;~*QyhDVP3P*?A(yx$jp=l#Bs^F!XD>+DV-dcBpSAx-IhJ>!BCc~Q-OGWj)D zgTMR1t>}$hQXb3_80JU}g+$#STZ{y3vN1kjE9$xa1A2qpP)aynJxqSc+LyFtSw0ZK zM1R-Iw zw;x0&7*m9$*=<5YN-RoWbqmukoWW$wrMQ`apV!3N@qcq{agQ_jGJdtidK2FE5e#XM zv+)N#PCqvJ9=e<8IkW%aIAEaMOGw^}>{pO>`&<--V;PY;$i$D^Vsu6lk2-uiXWSNx zsuH8Idr-nTP6D8bT^$qK!uAZL+t#%QcTW&B0E`o5i<#)4!)%>sy^Gw~$o z5w1rXidw1?Z5lHWf?A}ZFS{^FMczvIrucU;6p)WtSpjUx$_gn(5}5w^@V1GV4o9X6 zFTwq$+C`t#$Jk>MsCk^AlR7*p>3u!+DiRV{c46$+sfth+>o`}FTy-LslRD|&7ULvL z0=u2~=6>muN&oY)c09OUcgA+1twh(3pP^H6|AANW<$U@0k~f_dZ$63=5o~mw=ieW@ z8$aQUSg8`Rmy0+q+$rT;BU)PaU*X*Txrh<8fG~#QT(~Y=)OKOP?jknMmk)DC&{hDb z;tEMLELUhfAu!>~f2OnuJ$AM9sbg0T#!=FHJ~{tc_I(|@h9now987xOO3r_ZWFT|! z8nsp=A8?&7tDC=8%-`eAKb}PK#oRb-!ouIVeJ^d%t{ulg4SHYOU9SyF+<%ml(p_@U zbymp^Im(+UnSYTw4#px%JGcMyG*)bjwKu-l_*zTj4+*3Wh9`r6Uj`|ay`uq|cq~J_ zr)4wZ5`yrza?FtN;?zF)?RmA`qJ5iBL4vsGVGV*Pkk_>?gX~^I?l(!+H>3c@;c+P| zQ5;E-QEiF#2;dwc{^JBljP7J@jqUW_X_?b4(9QI z*V?gXe$6~k5qlTchyupbnaIlg)&M3bK9E&(Ga!N3O@p%!r+cW>b({80QAM*e961h8^|2N6ts^Fmt|gqO15 zxAL;UcrP$MC@X{Cx39J)R&-@lk`7m+lB%gXjLDbW%T*2KUtm&RgB1-$WJuQTLZ4uc zgXhR^g*zc&EnRFn_lp?mg*hZF#5!t=t`6K@cc5@@AeS*t&gc%17Xlf6Z&;m1QQWbc zlA>m>`^h`*#jou7{ul3Oic;gMl#I~)$Nk)W$eI5thDzI(A^js-AD&VI(XgS$bA+WH zJ*ho;kG7ll0h!<8*nGb%eC6}-P}Wb zh-8+BtmGr%!yDdDY4}#$61ZAgrjJ66gKE`ZtXsEhWLLucX$iyCX{_z58^?>i8_-uL zihp|;e?AEQ)e#N%P4`9N81#U4zz)WS1p+7I0-K z>F)z#*xlH@RldEwnfHRjvDiL%wO&6_Up_)>L`(}k?3kc4{IC3yn$_FJs&F`kvIqjZ z3kSjKoOayzfWj%diWg9!p=A2P<((d(EJIik1M@y}Znm*1Kcd{Yuq5Eg4#`}c=MX2I2L(~JRAwFoia4G zwR=#^hQGzDqp%yR2#DZZR72F_q!tVLl;2qgE>%fM*RcusylGO)kcj%R+@tCEX)x}zyko5nKxf8i2AjPFCijU)`4m_O5 zy@3Z9swhFH3qL4vz}BZzxZRFaolU1~VPjat3XaZfe=ahKDgXjix~xp5JFy&zlKcje zCvw~xhhc|$pqs+w52H{dQp@rZDylHV#6lqlIlvLW z{t&+~?3{6}cpQKEhj^TwyO)qAa`)gXa@T`1@pCu8HC>+6{Z-_K07DekT;vY1AU^oX zm@EBrH28U&{)ut94?i{4ii{#mbnM#et!6fWze8fAgYjS@@7;@-@U{4%{Axy;2ewqP z8?iKB1!tJj3IU&@tHdS(y~sI67Sg2dwY~!<|1I`1_t=P>GBg2xT91N;df0gou-%@P z0LXS^xTi?Ildt~`oZ}zIf6Cq}67nLJfAQATls>0nDdIg|_#Tq!-wmW=;b*_b`b~8~ ztTO~xzNq?EScavY*rV2f3mwzM(Ak22fPzJA{2-603%wd&G%Cl0vlJ%bm6rjw|1}u`LUs^W>w7aaX3W4KnF?ucd73kN zAMhwo&$gAf`YKwT+)2Ap<#~C|oI+Ztx0Rhldd9{exn0LJegDMJx4fN9vOFz|$hVsyatu7cD@-YRh2(6zjBkauV2ZJ=#zSb$1!*b$3in{- zPM|EE`Grj7vL94%!?=8d#arIKIQobfF~Q`wEPh7i;>repd^$;fcg4b)q4LHu*OaF@qmNhQEobzfDyC;IEE$Wr#9{f{842{cD%n$jA|+us=#tF!aM}|Sx%p9*^pDY7XH3+ z;fsvL1L^~BzO_T)uiiR=C-I{qcTf5vsGiSI*{nW0%1IB$Pk)J+}$laRk?Vn0EiIjzo}f-fTk4z4x=b$ zsAgg4gp1Q=Da0!mFB^~{e+9t@G8u&T)cFI?q=EF9k1BW<%Uo34PP$!@vxKoGYl7b5~qUlG}I6f!7mkI zm>+kYW;nwosqhb#!OjD?ZjnH+wHAE?X4YeyrYmD>cAh3+VPoZrm^a0$>>3tsLSaBe zye_z1UTm?)payOyZkH6U1pihT$Pc5QOf`J>ivam1gvZ!Af-SsRu?jBVsD)Yv;$pzb z3V-u-AhzGj)5b0QOXYg!qMnvcwQzBu{$6K$9%U(r@pRDHz8Ozw{;pnv`E+G4A1RvY zedBVL^ltQ*=568p%9YCI0*&Ru3>3ybwG1KP*{oahpjdDlQ%^v`9C zlHAIoOjp>|Q%%X-TeX>{+*`eyXXoDfgtOz%Q7>1Rk$d^g%=REI!u6iby`s;j7Totd z%T9A=?TNxXa5v5`)a+)s&(yLubc4U#nul7^=2Y(McqlAEA!WCDuOo(DjaV4*yTo)4 z;!WfjkOb4{^#XVu;P96#EOwp0JK*Fx0JHECtSTfkzV}-Ss)_)f2pi|fcfF%}3aX6% z^763uaY|tGWekkI(8Ia2;Jrltd3b_%$(%lRnShbdE_>yt!wP`@KJRl}d{0hRbrd3l zMc24p`s^fj`&8^4c3cpP*S=1|n%CM3lg!3I?y(tI6!9%s6DIhYgwr2!XWFBF{we;N zUho@t{8+w(n5ZNEZWH_sud6vEh@OOz%j1%0^kEN7_l+vgAj2 z304?(etgSANC+HrdR_$)TsT|>-J&o7_>|gM)3^QP7Dq;~Pg;8|$75V|pSx4FDZZeJ zhuiRf)8Zay4o$wZTU{2B98{Qt_lmV+qow^Da8=hivscWW++pKX5a(cB4Ox2Waq3jx z;}g2&L&xmg`m=lCC<8fFK%47*DRws!5E^O@jGE&={{bJVoWHPZOL}f9#Xc3KS@W~F zn(~))9W>*1R|1?)Mwpiv1N4a(!7QM7Ccyp)}4jFc&JW&PbR7z(!PNnpFs-)9os#N~I zO6kD{jGXc=W-(s#MsKXy7(!SPg+U-qW@96s^1}Kp1A(H>6Jj?DC!}FgK1VT)K$9Y^ zlDnw^Rrh5+Mv-S;a7ZO#;l+6Y&{iN$eWBdoxrcYE=h;hR&d#rLW2Z>#@L)4e@u>+w zzzi>uL*)hptjOsx#GYv>upf()@8ItMjuj(7W#L=S4#w%D^gK3X&mBYDIuI6E08rmWFc6|d zTAb^i<$rrv_(@eG|Aa2xU`rol=_e|d_M2Kk8(kx6YdpsqM5zou#zK~FpE+45kvS7C zpr_;~DE1BFw#V7t29nXAiKJJY9TzAX(9(I<*-i-_4Bv|!wRRdA-*R$?F=+TzaJJup zLROcyO4_ztkf!;LSx9`w$(gkVo(>GZS2zm*xpjbni91(~#<;Xo(U_8Gr2SWwm1opb zW&YOfphy3z5E68sM`Qb(ZGR(4i=K40oe!%_aNO}Ceg&OVrT`dZ_T|RZ`=+z)64*;j zirQXmyAVmG)tXqQuJB=G+5K&D^f+e4VO9m-$y_k+HAU=B(R)qnaACj|dkfFP*=FJ^ zQRPFNJ^=Al;T||9EjCxf|F)f2kEeDqlAc4f6ge&>)JpmtB?bWt;H{F56)$|!WX3y_FF`7faD03yU9hc zpkfz8ztDMW*otSmhx+bCwO^dMOwR`xReV(~`ibDw}0tHp! z8TiQbeEEs9{UCbYIF7!*AJN`kQ8LzkKDQElLcxnMQNlF)y z#u!~h?p+5qt$Ugiav%O{+mE@u@c5=P$wK=$&(Xlzr#Q{WDTBaKS3J+ck0}YHCfJkC zjwj6IMc&i43C`MQIKLpD>(Ef!_fdD@Z#Fd&Joruo-df@R!-V=R)ZF{*yNSrRQSF)+ z&?edg8=?~!)H|dV2#e2+k3y1WKx4Q9`xZ*F=IIVg%s1sP9+P(4SG2t8ef z(B%IZLg4QU)#Uz(vxC1N=9}_Mrx4w-`B)|toaW5gAQ}_NsQ0QBL;>tt?^LdAB9=AB9*0{baEGv?F&fV zMKU?tzK@^a>?lhVj^M${59EXMF962`Ge`+3C1Z1w$SRy6{3Z98KwkzX78E5c_B1$q zd4qVE(O%tQ(1ruKuW$`!O(0KRcDC=AxmNev!XEq(KM2}?8NCwkiRO3UyHHSE^o+sT z*|rQRg-QGu*cr<)@UK6S@b4N@BLh0+(Lb^TBF2|BA$Qz8XW3#ICQ$+S58OEielLj8 zH{hD>A$`#QdeBSi0qtiC_l4vI`todHha_wz^$fnflGMa!Vkt7i+4gzBEVKYiwBLZX zX)~ZjwUvDkEv%XAY+FZJVg_Rk9thWbc)NqL#`G#;?Hal6S+7SEein9Au)O!K?R{h} zrbak;%iCucBnZh=4guVv$M_ghiVXP2bK#n6kY8A3x=r#+o>xcljK9`a*W3h&6(9_C zqa`0A4E^8c-VP0P07 zjKJ1f;;v6dCH(DMkvY}06j&I=AfBbyO~`BDUwqmdS~}_PntB4SrY>y7sB6E2cjS>M zfxHj@MRE@v#Q(Ivmk*&edPJPje?gLM(=Iw~;|+zIWtu7d=62y~r-w%bO>-lfL344T z&9<>S|Hd#o~dcF^g4S$Eas+g-2#>IY%@dWOYH=O(jrUT8H zbPx*dG%kLw35dhPL#xuQ$&Ux@Dx?u3Zz9KBesrze4T~f27Dx$Z+2n>uhA(Wt2Ss_N zb-usj8~7>KEl^y!cj|yADY^o}W{}N~pWB815w5}R@Pj)q)iyD1EXua>+;9;y`OByC z2GH$X7Uo{%>DA4(52A-0dQ9F|34$I^wTe>wRbZmxBbNVq)w6G(g@H~&iJg98XWz7( z1xbi18A^3g(2TXW#ket~WdhfXe@1(W*YqU) zmZl);Pz*^}Sg?Vwg7M#RkTMSUX5Gr-MnD2C`GI%sNcn^xkD_c~fo^)gXb&{wu^z^G<#p2zx>Ah_{gkJ=I%!|=QcUc>a){j70=?FvG=6`qpbjs6_~oM z!u59_M=c3+|7ZLNhjn9it`LVhe72hZgkYLDq#JzgI1gS#7*leoUH4iKe9-_1qJA3g zZ(q?!A!Z!Q{^?-o>~h@}n9GW1)j)x!w;I3-Wr&h0H~Cq?QI{!&f`JJU%)7i-=0V{X zut!AY`KJfK_xwa3y(@fLnwE8q-YwI|KyeBbQ^3pvrB_yK`GKq5?D0?gL*WRzn-BDv zjc}(V^0#xE^AfVr8jPE>gX)hHdGsVaa?a=f4k=0hA1DV_1f5jsCsPfz}LsnUD7xA3JEF6Yans=!vbNb_whm_RM8T^agh{+$>j9JdYFk zs!!nLZYy1kXwcZnX{(8 zSyUbZuf8b3ccPt*nf?2=qE>LSgNs#!0emu$IQ|HxA>Xhg3>5cpV5gX^+sW2i zpwJ(Hp=6{zWrMbH)VyIe)U9wX93rm#T==A?P&M+%Rqw9g8Ygse2WP60z<0q=GIe74 zK{xRjUc9AhC;fF%aU9!w3wf(Iv`4~w+(c9ZuV!2^-Gsk(u*vOj0HWEG#U(UG1>ky9 zV@dsfTY?pK|Jq^*3R0TIGElY6!?K`Ow!W@X^=N4#-VVVlV(sp+4B`J|V&fqJsnxGH z0Y`%d8*C1hY^NS22F834V%tgFcmWdLbs8JVWt4_?u3f5Q{%7Gl8LER?jE>zuzU;XP+*t zht-Ss!`;*%6x39+o|}?==)&ok;(AU1HT%)=1(jkoaKeciT9bj(U8bgbo`geI*+*hz zsQFF1l9zj^87%w!fttqsqeTksW1RU(tI^E*Gjvh((~1nGFU1ZZ0n^;f zJ(zr=2|G!L0STyS0|RG-4zw9ucWB=&H)`}oM~{<1_Nl_jUb?- z#1uXBwjmv6F%DLxL0!DJUBb4!9YaR0Ox{&bZ{|o>DDPHDPgY59t&-kWCB3~$`ZZP3 z*HuaHu9DtgDc$hKi4t?D5)cqMQzd<*O8Siz)2qg%Y^_-Q&e&*`bb25vjp1!o(&@vi zRQ}E?=@sF_drm1{1_$Z)R7tI{pCk>R7FIIZQ9W$X}jtF>B+*ZFcDUU@8NGrz`u=WeKtIHNPj8o5tKG974B z_a4E&cA;D0U0iU2fe1z);C}Mj2hlQ(aD^ST!)4Y%46>+Hqv8EKc+43a{P_t~GO5`0 zS_}(GIjZo+K5JJfobV!rzz}DHz#1cgfKt8awSG2x7{0ItmXldO*=D-#;OP_h9BB>r z5#2bZ=#W2-bz+eTs~@QApeJSM%_x|lU2l19VVvvEUGCV>yR3#SoMqQ#z*0WklsH9} ziQ0vP#Za0v`UT)$DgCx8>GU^LD$l6< z^7M=e>XE8mX?N<%ny54igp>%U-t@q!ph7p*>bpA9Nosjgu%o|zwcCD zS}FM9+T7M?$x^{Z0^Y1VIjd*a*00B!K;K9PZ@sz!9nmk^=7PZv14iyJkxRa5^9Q}m#OE@n4(aPaDMtmicgK%yqRSMd&W+)s}#mmyu43h@{G z`0{=qyd}L6uk{f=Bbz~(wL_RK)xzwi0}1oCy%k;m%wx$MKKKq3H4tfqr1xvpeY^>g z8s+BIn((;^hc8=P;k}f|{p1aKK_qVJz*MiwWXbuztsqb8P~KmA0K+v$Fmmz?B`A9l zP!p1bbwhF|yR<@D1<{*XARWPlkbEVg%rC?f$oj}8=r=KOpQ5}SaBNxIU#gIHcX?Xo zBiI^r6aFLPls%pfEWkS`PlhDWj8z6ab2^>t0k(w~D4@!V!Og+lE$LC=-zt~EYnIkD z>AE0w$ssuz(}Kvqa_~*EQCM8xZWSi{_wuzP*v70djnF@C&Q?JA%o4Ikq?qikPu^dl z{QHW_1Mt(_qAW)iBnuC4;U6m$^IC4B|M^xh7`fDMrDhsi(+Zx_5Y>pt=Htu2Y$C@c zD*mC^4Xt{({D<0+5;uW8zJO`r{v@}VH6Vi3CQ2|(bF7FioKgWdB9_yL-JuXG=DDc@ z*=yagwZJ>w$|)M)R)m*&@HL1u_hyR~halm%+*ZhAD_CP&SOI37Xt|9ueDVjVV)dDw zZc_)G(fg5KPI|6DV6`$5PIhTWrBn+h!-_jm9b{;^4~XyL58SZ6;$HsPog8bq2S22b zmO|Jj${l~VbNe3xMAF}g{r8)8LyzwS*<3$+x9Xc(pgW^~gLZQ_?TR@04fv(t3WzHxBqh7havwrR(B-=U!XU8rK6lBkohW?M$*w0q^kU=3M!Y7v&Wl0j1+g#S z0}xM-ACHih{Ui}a8CmFJ;}u&UKg?Y`EqlZOoO}-W(Yh#H4Nh%*qIj~$Cq`q3kqI0l zg*Pct%w5i9@231)$*p!0^*YIYI`=ZqsiN+GxbJzIRN(!vfAhqitf9*X==G*107*gOtI4P4g z`rVfhAK$=Qnl5ude4|>C6bl2k}*~KZVA`E*mEcL-Z!G zE}5bC%vk@q(4kbk_!~X-XKDo(+Jc8%L5{)VYY{wP;9!GGEP?78R7j>?$n2s+EFZFP zL|EV@;7GL_cBaGha@ci8_F*#&;Z(e*8pFdS3R7LYdR&c_X}RX+%Xtt-*$_ZWdRyQ& zvf3ZHyopRhrwd*^W*R725fqxFIPD?*O6_+j&ns~i0%km}{2G(wQfDy{R6qkGO!x+l z_NXi+)SB1DM557dXc~TZLKOsEppPYGGpW$bYxK|?Sr4l50C}2ZtTj>?51fzJJPoon zDO@RZ{=0alJ}=7@hI}RQ%30YX1q{Ua;uWYUUP9vOnfCD89k6Rzm68<}B%9gdt&foe zW{r|9a{!i$g^@CO9rr^gl}XeR*~Koj2(+)$f@H&;?>jDa6Sw0#$G zVGCN&%I*Y_s!Y>Hz}X7sOJ(1SulPXrWPB~_&eR9Eb5>U8y}~UD!g3f08T_2ot~nBg zF?b>w;KkFiesqcbVd&ZcaeBC3aO=SI7zF0reXNf6mmP=vTJpwCGrR}*%X>qmOD6s! zu*<2*FJX0A>k*b6#>}bKpNL=f1V((;>k8}z%bt<%rc)e8Al-~anm9wwR-w28bym^{{ za`GR|Z{V7p6!U&ZY_3tDl@Xjd+fu_C>fEjsZKQGV4wHua;9s*i+jZ#Va{;{?U!43_ z)ri8_2Pm9P_~!||0z^Vzyf?k?;Ls>e#v;y%K4$o#JqTFO9Noeo&5*P06y!z_l<^(n zYjTf4(;cxdAZ#*(SLtxam0mF+fxnl5C+Pc#gZ+3!S6y_Cz^g??0{s!nDVj3|1L}n@ zVjyfpPH@|$kq659aYi3QvBoEKU;|u{S~j63kBz|O*5iQJT+q3KS@CX_qW*6@X4=F7 z3)EmCu^V!SaTM)`*}0zFvq?iml9j@M|0;K2y2_gTfyiA2&d3`of35ATPz&8ZEtAVw zK8i0nycOj=rUYI*is3Jw-|;KH2KJ9=^GLkqGQr8M53t{s6fk3>?DtIyMX)Y|k7Wr2 z?ZtLOu6*#Z^>;NsVWM1AO+XB{8QN*k?$2Wqk#bV@97QGi`D}bRU1_=)Fwk2OV&4dM z&ghHi&F_n$?wX1?4^>9LbPp?a&u);=am!A=Q}>AaiImg*yBb1GJ9C40O*6V-BAfvO z&Fsfi%0CzRpo)A%9*GdFZC0_{I4SAmvzR$b6=3woT5G(|Q5oPpqMl}RoR>jN_rI~& zSMI_8yselkHGgazOv07yA;9<7OvnC{+(#eqjhRd8bV;wkHMdz%nwhTX$ww1hG@2o& zyhhG7Z#-)X;+5`2i>|)k#ebM_K&lE}r3-mt)Yyj8r;e9S7uv~Dxxa;~S`KRLqTj7I2 zJ2>8P`4Cha1JU7!u!sJ!9qP1C$LSAsTuF6wDxZOrmwgV`n-jjWWKL`X??wjR)Lj0| z!Z2E3H|&SL5{YT1v7pZ^b!}?JxuyKETRs)UhpXd3P$cigTlccpgt&#LeH0p4v*08g zx{r0Y-PEyi5L3fU3z`3+xM{9u?^L0@@5=bX2xfj1OFyU;K~sCwYfZa%i0O`ru``Cw z8b=TuI<;L&yP|xjJy%@WuHzRiU2AAF^t8i}6UKVz$)%9oiH8@_8*-UrOE4eK=YyCaGR|bRs zr$GbYWgG?M5(v8yuetu8xUToKG7|w__Y#Hf<89>h8p5ogzs4Em#kpL&&_7@{ywGx! zbJydAKSG}WC%7#hV|>^6M;-Xq9g(D@L5`zxEkOE~%FUr`$v_gd1a&g5>2oRmE!;g81X zI-6>mE5d}n0^YM>hBXTp{?}voqHDz|^y;e=z8p773kSRCU&EwE7)U1K9ipq3xQBsGKf3@uGl5KPGHT(|-2(!+S zG`D73J4@7TbMZAfYT_gphjz*SiHp+QW3Ex1vLXeZ(3=Va1NPxvfME}9;$$ZtC-U?$ z@}o*&0^H7{*Qm;DU6rd+u!J-EkYvFHF)~A@{XKk^O;_(bBy`BIrhzDR761;;@x;Xn zQgo9=zd?)TLSx2tJQksFPw{O?fo&=4d%8e*k1(?5#v#?Sr^PuToKb2syIy{OA~zxU z<>lZpmc7QV54B@uJO*6Gv6xY~{2oQ`ND3?|&CA{J{z+4go*aar*n!35`Zmjk$3|Xq ziLqe>tM-@&>>Lc3k!o9|y)cp_I^T9IJu(Jo!oL#hAn@z0m}P}CWlA>T;3e=>2(PUT znnP-Xq*^g*2p<|@#Ijm@i%WY%rP?VZ#n{;fc2;vbI{RI@RYL|TQ7nm^{0IsXFRBEq`;g~_8vvHpkM=MZb3!()rc5g)52A+#C<4#XLUAR!2?(^8}2a%yHX3NT|?`!NnAK33K)T;p{!&sqo&%aVw#+Q=+mni^z&>T{B9Oh>}(Iu58!p zBFZXzM4D16iZZgwib^P3Zj@Qk;QyRE(EI)Qe81n{|Gr+_d-ikAdCs%WbIu{hJ^3pz zwkn03`|+x{C#wVG48^N6tO~3E`lMkc%c|J6l~|G$WVL%543fi2hUMNNQFj^m)d4a% ztL2y{s{`Z=sjF-FE8w;|#cy>Wcy%CZb>R7`0J(+uRdG*N0>~OdLJaitZ_8H*qB&&U zb@@a$=oe)92ls?$Rkh@%gjdBq`70p5I>l^tz-ViTrAen$z zD{w*J^Te`b`ygp(R-{NUWRQD79u%hVPe?FCZn(t4Xtb9{3h!yKo=OI;CU!k8Vv3sr

9oMZ@p@%NL;C=_%FpG-|Cz zD$#RBBHmKeP|ViaQb#q_9m-p{ajQ{s$vdg@s67}pq^E^a?(wk=`wsPalxjA`b_zs@ zQ*XBN)Z0o`>McO>-A8K2dbIWSEX|J@HthMs0c{Cytwj>rzMyqa8vW%9dYNwBCCzy<$2SPbO!>y^NRL0$b;q_E}232*+*%`i6 z4Oz??V5AaMWcWVijEtV}g>s@Vf_8OOoY9vkO{fI&cRU5H?|KT(ZH-W(%0bQ{--7b- z6trnX1yc@pqm_#iqU^V(-c!()5|*blnNvki(U761Z;%oCEGXG6#510C%S#gV`ILzeaL(Pkg~F3of22u;K-O=RkY zZAa5(_41!^op^LKVHNv0eV(o`S9@)jRR0srwYpZ`zIpHH_Nh`$xer}^?YxgAEWfZQ zc!H?CkX_MW<;QPVBFAS$#AHQR9Jw;u*+0@5_`E)SS^b1Hm)+hyzioExjOp;E9W!}1 zvWZ5-_e-fbiz;k%ZhC0TYZJX+cEuvas^8Q5wxAW2Kdg0Z^yXXYSA=c57BunA zvkPLEvQHPK4_&b^wpOxn-L|mTW15Ypn9Ae&2t^mMl}+`(PM`R2HTOZ;;i>Lo{f~N& zE^XP;5!1RX{bEwvU~yVOZu#j?&7wZ+j}2Fp7|w_(OV6%}7_wabtYKDMrNa}80ej)A zCpUj-*>TT4X1clFz&MG7t0(hivs^Z@)txztD?}oo%rxson4BxbYROdOW8#9al>fna+M9`Iw&U zJidIXxi@)yl7$13ctmp9=@a+s>CBrT=QVv`%Sqm|FW%@)yt17wJ0)0;cNJ%e@Wg}& zSE)vQGj^>MpQCB{Y~?A13CcCM6D3^LiqcD`_Q;-Ni_&Eo&s}Ik~o}y7LHn~GcxA2=;i)~r-8@5T^EvmW3qjh%l^QSj` zikWY%wc`9Qp_Q5^)71n|3Z6XDI$Pt8Sba|S?8>cCK36){U+brHm+96$m@V-wFX2cF zmuhMI5suUr0o8}?GxWAcNjSHZX(uKcsZ@!FInlOGyiOC})j!u+joms*{HK&E2Apb z2ZvIhrUxZ|7jiyR@itLcTBSUL|Etx4@a>;h$>ndDCblF=&^OWl$s6r{+Kvq&iSjSm zc?{STr1-2wgpZ~sv(GmYwlJN<;hj4!U#?3}desCCVGlj^-88#oHob%XoB<7k`jNH1 z4Wqqr@*MJCS|@g_y*5`aG&P#P+Vt?I@!Imoq8)#QaNxu%#d zQ2+kKDGuM{zNYA`f@Z#i)63(uJRNzg_7|v(#2{oqA+W9<9NmrnDn7+UY7xRm26 ztHw+CUFlJ3+)FJks(hJG%S`Z<-4Vxe-P^7@N=WqWT&Fd@xMHi^&&~;^ksp)=p0^0? zm6+;zg2z*{HF$Yxb=?lD8y7>%eWO;cjO2e1K9aTGQMmEWwP`22Z}Eof7EJ0`7gc!8 zXMaz|nv1sSJ)zTmlDl8c@fJ2b<5(l^J4xAS+CiiH#-V1{Co3ynw|6>b92T3>`eb(b z@jGGPBAQ(D4NJ!LPEx+56PD7yx!kz^sB*4tzDddSXS40tzNXq5R2cgQKPq0*G}W(7 zrCdyFPPvMAf4)9AD7%t<~IE*I9S6&i#1BpTg%FWqYahUgPgMDm}aFax!^u zf&{gAu)VSFD^;tSV;j%+tnKy^9zR-PJpK1|#jb(t<~5&29h=h3nx$5b7f4w&bJ3ib zlvkEb_0tceI=(Emt~q^^NBFIPoN>T)4Ex@i_ah>{uBc9Uv!rEbQwzYIjS{8j!t|7+AWYGuV%KIOa zd*mCC%IU+l)R2Z=Th)@1#T6CO5vOUo97{ z!bG0QYuzdpJ=8r+58L`>W7FD(T}E%kD{k3FYjnjWci*~tsZGt-bkNTEi0o*lsd1O~ zRr3`m_9fABx4Y4#s%O!58|j*@(bqj?uw{{d;IU=X#t)WFFmvIIh&edn$b}V!=28qa+c>+A*eISWT(RUpuYj#!z4d|In3IQ`4xPT?opfLJ)D-nso!+aaeYo8cRc*X} zM%~48<@x2$YC8pEAGf5e4m z23Kba*U0_mT)jZv%#DFBdyUVx35RY-S>jQxWjdko-lXu1;f7yjB@53#nI;vvHr#6U zfwfsq?pZbq#nSgWr6`<|Z3`V*Rx-EvVcBHqpd0U!W!FEF?*3M(uxKF0b8~y>{qJ6n zb&@?ku69ZNHmJkr=iqs%wENOq|NNP+{X0)O#ojr5ws3Gp?Ec5gekaryRZGk)p7uG% zx?$<>GfUlGZTMEVsx>yWa`KnEN;)MnA(u1r>-65-FmmFu8hmp$VTvo~lbY8ltD~kJ zIdHPFHOEf(Q|g7;*=DG>$K)#U^ib`qbt;!(GH{aJix0qiu2Q3y*%-9PsA7oV4os)`@yb zhRQ#d(>lBMURyEhedx)(^25taPI69B-M7{KB<TMxL30JEGaH1h7qPxFb7*{Z&EX5AVGt`#v4zG|P&(aQ2>G zg6@g;OC#URt>~-lxFo-1*u3z4!>doCp|vwR3k-6Y_t)6I^We8M9FNKg-*tT@yS`9u zWW@Tw$ccMRy^1d6F7PsNbv=G4L+qouv4^Rz>6s`RzwM;e7B0)A_gBlQidIRBD;h@8 zhM#b2E)GmPbg1Lw!RHOrO=p!9<(6*xkZ7D_88BzE$5m^kTZYR+9Z#npe}Dd=#SPC_ zlFL4iDtf1`F`sm6@hYn=#euyYtL6-?t_~V^vvvOEMK?1#9`awmW1cdQe?;TPwh@B_ z+l>3;i^BR}dpA5dEc(g%tH!116SsFf+mu{TJtKNg?3c2I-%1`VjS>D{%H4hKL0Q(? zA3=5M6_-b1{C_pPjawLTXO+jo##yVZmzFec^wkruO>lg-{*1?}i6>?owb~A9w``Iz zS}(R*`$p$M&YMpX@*f8m%(S&ISiIHYPWiXV6F8EW4W!F(zMD2hfd2O1c64S&|M>C0 z)|j=R)|i!P{Qj%)`{-KnB6ivq4fgAVycH3T%K5V1+1c6Sg|jp5kww*ws)o<|KWV7f zFUxp4XU>c>^=qcIxP*KS${3e(X{T_*@Pj()F(ZTVd4I~sbA0}3SAVMq<`eZNb!N2& zPPa&XZ(;ea|1$T69F|3PuV?4M~#Jj$P5ur;M+%Zu+kd^L8^ zvN?2=-K}eto4Ml#b-lzYiHK`~od>_geY_uFzDQvDo5hjSx6K<;S>P_}nb|?F*6e;c z=hn8bFLgXQx@CVhe>fi3E3vw0qu=Oej&8nzc-4lg`I*!87qMO5UR3hy+Ky$zpr+m`(&BkOhu6g()uqi#GBYjd!D@V>dCHM{y8PSDF(-; zZ~eu0Eb2sjW5&;uCcF2{*kpfvXdgYG#r0e>Jwr9%MfvVo%X!xG1&TNoC-HqZo8HO( zY5degWuYm%g}=*=LpK`QRe+?w#({stsB^N&pxVF+r#PVk)zsSB{Ba>v3}vM z849kGW5=flT11}yb+wW!D``|~o6)$9Q&N5nESY7ocObUbwvvVXiKhNHsRN2;D3Ps_~n+hZ6R%oMBe0mAHVq0-b1Yg z(RY%hngo=J_dS$7AUDa(Kx(y2wX9x!=lSQ4(s>ouW}VHRYAnUW-jVk}vtvn5=bgPi z#^<&@nQfcBSJ;+U`mD&CMu}ewM_=jNUQ!gX`n{Ux;8knoB|-YuAN11<9^WmfS<~9~ zTD7F?Y?zYS*|XLBNvoF4)y`Tyv|@Ti_4enfcXLGS=P%C*-)LDW+-viO!_m>yZHv7{ zrR}hJn%sHOE6?s~y^A@d1KlkaE*17iQYaYeyP941cwl%(@ z{@Tf|NAV-F9Ug_ZjgsFUEAZdac(?pXS?q=InZ* zZFo|};Jdz%%cgPPgI#CN>ik%CUg-Gxlsgi;#9Rg&-!9kqY}4X+xm$0c<(=m9PafE| zb@r;#3SNQ3`}FCPpBx|ulld;j&qFn3qNO{5*Xq2__k&C#e%+yU#~iT|Ge2z zdEwUW=?~gx`kin7op$$ZKijh5gt)hPv3k{?ca~mysw|;h^7C4u@`!LzXyBxcQ-_D< zAFSZ^fA}H3EbZm8jqVjjY;BWlBgKz7DsHRfo}C(beCi6DkiO>LOZ$>uYH(~T_W6;r z?%Q3n&!3B&?ti)w*8gJP*=zSRTG`Dh%bwkKYQ>Ekqfg|oO7TRUXs@$#)@-=$6sA3z zGk3@|$NRFKMetSYEo!;v4#%b(dT5uwC%1Rvqq`0Jo~7DKeJSo=uBqU^e6CR1<89AY z4*iPD5Uvp}?wDlsVWBzVH9y zS-Ej|IQ)9(D$j9GignBL($9a(ewf~THTFu|+XXJg15HnpK5(Voc~cO>v*p8)R?nSE z(|av^y-feMA6xpOG|!kk;__lS+n4S;`HxpJ}&&S`|Xp$ z*UmyKo-lkyI;KQ_|i$FpPal5?w8?!BlJ@G$IC=#71+wAdd-=I@ly zuwFYM@VuXsLJe>J{Bimt%MA0HTaYTBrKZ|3Ek)NOuQvk&zO&eQMTH!?$Pef#-q zXP!J?Eq2{QNpj_4k2d8G#XdYLCEJB%HZ99bnKy+?tgqZG>cjMB@7~G(JhO5829uIG zJCheX-*~!KIJPw`xACp*KA)c8TE!#Qyr0AkO*HPXDV9_}3Y(mueD360QO><93maDH zFI7_sE9`t+AL(8{xi8AijDADz-86L#{%WO|C6K=a;Mm&a6E1cRq60&VAzWO9rb2AgBqa zqT;WtF<0CW<~Ken&8X*QwR~VpeoKmasb|uR9E*hGn+;~joUq*9n){G@l7@uVp6;9R zWrO#+*yeXUHyo07NM1E*c!|lqh1n0wZwW_9T%WSJ{?mE#a@?FvN&ce47b_iH- zby(kT+}?KbIQy;R-8RYs?OnRVeoKYA4wv6}tH0+_dHa``i-+Qttxj3g{Ml zt(yDz4jSg}klrH}wfV)HwjE0^%g^)jdpzRsQE!nL#|7U{vx2qb4o=zfqAc&HGV!^`D6KxTLy21yR`WqUAlAK%J6~8C7+#=*D`tD z+SQt!J|#WwwH{4qZkU77)vi5H6*NK)tCZ4rh0i-?8)+?Y^oqW8oRi3GnY6|BoDwsB zI$p|{#xIuL#?$HdWZI-ju_23&POK18iA}bd*L=pldB&$|@k>or(QVuBEH}Na@1VDF z>$P;bYpd?dzd7x=cZZ07Wcm7mJ7Vt>Oee28mbdrq!oqt!wCd-Z`3H4YY8ISYWo&#x zeXC}KW4mH+?d}X+j?tMPHDfZhW*QZ|5f*ix;J9SGbBAC1(!+d{H5K=68#pnCy|2-J zk7*#S>T2|?LH;Ek2cvrqDXJwk@f_KfR9NkjpzeM>q)txOK&yXygoE&G>*q(42m$W^|6^B`vexZC=xchh6X8cCB2lxp;fJ(807*nO}Z?Te&Iw z(bq?Jf|h(Z+;dL)&20TuKlWZIPRtBF?faa!R?4lmw!e&b;6T@RcI#!1no&|C>1H+l zJ3MC$r3Ae;HmDyo*S!&3&CXG^H(;~sr4C)4-N!z6=zEyHwDVTCb2PEKx8g=d>Gr+r zo?X=B_s(cuvt_Tojh~E)_(+w^fI-8zx~b(`pUxM1-M>^-y2-J*#K)<w44Lek6HQ#%nSUe7<|V5hrrZLa+_zqNo-xWmeXc>TDotVXY%4T zO*7ugW^>ynh*$TNuj|=8e$ai#gt96F$x2gEeo5ceJB|GL9w}C zhI6oUQ_t7YUA0miqd|QyJ8s`uKE+Grpt9Sw;rz4wbyMt1B|~bix$iJ6{8pdRF4cA- zbm0be!C6re&uD%9$}csXb}ef8@aX65FwYxRI*NVMMtP^sp6fPu$Dzhck5g{QtO*Se zo0a)Y-7rCl`Wk#@(G{tymhJ|wQ|8@cJEXzID_WNjm~*A+rK(WHH7UpFgToE9v`O){ zk1v$;X6wj%ue$9#<6Q0O!;*oSOFngF7ryZOu|#6+=IX_J;si#X<#@=s){CW2p3l4P z!K$ORAAVK*<~8xP%inSDTXXRAg3j-!tIygUb#c(M+n^bd`MvgQ?n{rLkcrZzDWBL< z)I~Sj32SVcwVj*artYC|#*2NOhb28<`(6IDliMYG=YIE9gM8(sGEx@m%3S*Ee+gHp zDQ4uKO^W`qNLt&^X4=fui5`bdrr$Vnz;$_R_U9cxcGx=o(CSFpW5gfjQM28)@bP7t zaR-X~tWDjv9eoxr6ym~DyRzH%vV>ah>Gf+LRjI$&eKOO;tFd0|<*cf7F6oteIn^Ow zLuZ|`c6|BzdVqyn+O$4JK6BRt`Ztf))!5I}3Ew{D`Yl_p>gzJcHs{UF&ikryEd9-c zHA*Li40x^qP>+tpQ`yp0|L{Q}DaqS|*`G}jz^`r^Fh!{@RB(w;w7F6D@PyjXpf z>WUE|u{>S2hyCZxb*|W+d22Lrl*7Js*8y}Zj_sWb{bg`=VT6!a z^6JwMWqkRm#jaOg`aF1I?ZR0?mb*)T>i!sLE0&XKOYQn5`c7o3gTCvuDVVD%2=OzDO#V}OhA&|6R%H==0X`!FBBN_q8MvZuVSo5jb_O>qZ3rP zhTa%?=rh;2F5+N6L#=X6A!=!BeO~lz4xZmXiYj3sAheyGDsL-?zJO%XI3oc)j;CxB z1SF{?vZ*u%0^BQLmLAIZzX8?v?`vLzQxVBH7_-3$NJTmRGsBrF!|`WE*FR~{Tvj%g zSz)LuIB+oLwcr@@$eilM|7T7!K0A}~f&b65keJK-g)0EZa`?!6#)UC~0HZ(0|1Ye; z&3uLvCPecyX zWl=gb7kn(9@E4Azz_706jpr1e{EIw#CUdO(D;W83Q}4{c=x5Z&2I@7IJeo5-79acz zpFnk_vEu9lCpRx&Cr@f`TT(mf>6oDKFb^j;mmn86Z|WJC zfAoYWivFd5r*{aYhIg2U|G&!$@Q4}<2YGn-QVXwB#fEwR^FR$uUH;D+;r^5lp@7CY z^wS((I~+~)LjiF#M;CK^T|*p=rI^p7KBIv6Eb4{17J^hj98KxLd^v(tKwOG?VZItc zDxfO~dSSjEK`M|OjSncGe59!tmfwUR6%a>r!!SqV0}647@N-)RsZxNSc;mmRJzvZ&l7mda^A0O&73P>MxLB>1;K`J1QMkmaZ5TpX) zJk$&GGz6)DIBHYOPasGI#8I1K{(5J%5y;DD#{V@^%Er;AZH1#ft-#C zRgCw4?CA^rJfa^6297l}c zX>f}tL+}!a&m;Igu)i+B@me(KeuK4KA@D1_MiSQNE7%i1yySy+!}*+t z{)p!Iqk#Da@c-Ea7lv^|o!}ubPg#%Pdm#T61ebz(*%7=GDL@6h)+E-)3*^HHc{T7) ze0{|7FTwtJ?I+AnGP9>0B=Q-A{P7yiSbinQ-yr18;QI1_;5=aWc7p4G|Gy*nDCjvv z@O#jYIN-X0^(ljkrx3xlpNtB z^(Exn!Ox=zE(QLK_qM_L@Iw3UA>Sug1>@xlq0wT*iD(>^P%6V6WkEmU6bHWU^iWYyTUlO zl;BywEeZY@DBj^aI?!A>aq1 z1h)qLr3t3@EveHY9x3oTxUB7o&~&@;CKzmZv_7e`iz6|0NZ~7 zaAAT=0GA+mFtno*!F7QzBDgi!*_hy3z%2 z0Y6FbIOwN%&l_CsE$9yy2>B^6?%_2tvHVWpm4tjF*ykp}4PYFuBlv2teO7Ri|xM+>>o|Y*TMB}Bf+4x`zPj>J!1QDhc^O=r4~6?g8;Og6D%BUJ^VB^nXk6SHM~K9AM}J_Td7(aJ{NQ ze?EdAgz;H~;CK%&*1ZT&{z{O)1R?(t{7{DAN-!QO5qvZFxgo)0!4H=aoFDwqoZuBu zt}Vgwo-(ckj|4vqB=|+>UmFOn2iN@+f^UcG)Nz92@8d-TmxKH(3BC;OlkXDT8RGc< z9Yv8TUx9sI6Y|aAA9$R@@=b8RG(yPZHUD{GoWSyGkdHXQxxt@h3BDBUsYdYEU>{wA z*ML0ho(8C1%5Ys+OUMsFyL%Gc2HHK8;BP=af#8a8y+|i`2lS(Cf|o;o$s_oA7?<$( z5^OiTCrmXV{{rmaKya$cjL=5#HW&wb2%Z7gwXXzs0{>y%>j2pkf8P;=aSrQw0m_|D zaBJWS1ebySzJTC(&p<xuv_z$piB*7iPuQn4r7Op#c2#)t) zIZE(M7?;iwJOKJBUIQK5Z6@e>mylP0@u7v_Tfm;(1h0npAi-V1pV`4qxZHB^XXZL8 z|NQQUzjsIw^7Eme&LKGd-l0Ws{5^$E@JCRtEx|3o4(uBAX(_?++V%K%ajcIU%)EEXF}cz#thS!gD}C*gMUa6oDS`xMDS$biwKU#RbztVHNh+iz7y=@MDRhl z?)VXW2Gk4h`GM;d4()<}=f*r5+8vK)m}|rMe1gay?}<}Pa6Y(RR1y3&jE@ZjSA}}D z6Pyj~)=TiW;J4oi{st*b1ukeetp7Y1w3*)~#!HeK}%DVRgQqdp$A%T#`dr#rtU2wgszz#Wtd=}KJfZ#8|4@(Jt z6YdXd366iKZYH=p^v5oOPl9p>2#$Z>93i+Nv*O(?Gv<1m}Ww_aXRK=#NnZ-wN_u2|fkJfqeuw0Xyds z+#lMdh~W32UtA-28N}}q`~djRGlEY8d-f3gJdCSD1kVS*8VBPOZm+N4S0V&I0bG*c z1#sP&OYlU9FDAGv*mF6-Tj2V>hTsNZH&23320sZScpubv6TwR%|6K&fdnz3!_-PnV z@c+TEebS-6<%IkMu+JTW|AzK`LU6o?#cP5u2K_%1JP+=}@x*}f}E{5{H@;CS!r0D^CUeiujZIJgd`5d0kNVi zgPo5P{3`fc5yA0Zc~u0T4)(c4@XMg*3xbORA0l`m_$@!!6}Joiy<3LhGeFOU1jqk} zSV8a+=PXar{5PS;wa|Xfvz@IM?d>YiNmf)9AMX1n1a81bPJHhezJ_-6Owu3wP z^AzCdvI{QD$bS)wJgO>~m$sPTaG9blBRKPU0BZg@!L=B<(`c3~91V>(puO-sJESKn z*Ae8MS>zG71@2Gqoxp<$o(MdGg`;x$!OznO-ir#M!hRNx&^-pF(Xt4B1NdPUj`GL< z*UM+&C?B+xE(&J|KE%Yd3oIPvV*&lHf`y}eRzp7536A$BtzqHF#JIhl0mtnw0e1LE za0SqZ5Bv+uF943`Gh%K6T#JxLi^ZZ~LU3-lj$0F4AGiy_t$+s*96#@{6*%_uN8r?H zEb=hKF!aIKPwa1a-sU-$e2|?zK>rGY?*e|E;Hkh{34RfH2f^`t#4Z+&+8z7hN8l(q zn1G@GcNUK7HJ?GLbPMZfCmG|?Xn1X z1dBYlAmdC1i#)Rb1F%~*!EXaULgWL(E$s}!AxgW!k`J=;MHt}jv&h4xm!Z#l7I~!4 z5R^MW@B!ez2`&ze#s=3J>~A8#@qIB$j>-iwnlyaCJeZ*kjkb^_AEZAX=d}rL2m0s{ z+yM+^^cTL2;P`rke@Dda$Oqp?;IhMbew6h(q6_*WHCVY5!Bru?i{KN$KKBWZ#~B`| zAI=~1H3Ua{CZOQQ!db_G-GAYD9yB`5+K(DprkM)G)fA(}loV8pX z5FpE?vvAgOrGO*PmDGM`h_>a`GE%z9Q$7= z!DT=`ir^~1@pS;}qYZo$A-^2>HiFv&-%jxLz;_cI&;QRLcqGUl1&++Z+AnbbMx50@ z(Ow}apaW}v&tT!K{waxssep=O%^&+S;;i}Oc0rsqe>{#M&YJ)K&E8`FJd^mBjm7>E z_{*+h{|KO`8p(ah{yF=fClHB!!~)%2!d&K3n~Ke&p5hEaJD2_ceOEE`bq%=`9?z+d zjGzZc@&NAl2hXBDA;#)+=%gNcqXMhXjTM)pAo@y2rZ+;uV+6ANd6W!dr2cqZ+eqXg zM8xM%aipUj!Pme5t_oa|IwCpv`uXR1aZ8GgRemGIQGnYiW`uet%SYFdvC2;#qkL3H zvV1fyj#d5u#ZkcbvCPO`8Xd*N)DAxmXF54vC1z(kP7(w4Kw!N^3nJ@ zR{8l#%oN`tA4w{T#&WX!D-=T_=w6aF6oEs?vqI`KPLGc@6dNo33rH_Xq{r=tkLbG2 zP!VT{XJh_!t5T?41b z<)b}7#wuSAzSr$W7h;xx(_^Iam-4Hi{G*Uwl1hNSa^v*4d_5>%6`z2NTj2K;JpT*u zpY#l!DQNna^v+Vu6!`w;Z|TwVl_)P&EC5G8Y32ucg5z@W(H+tgVzhmbp5!~Q^|NDE2H9}uNI_NPJ(WAb{DloSkx{d})<`ziIcZLAt4Q(n$ zPrF`3Nx-%@TdKlDsLf@l&6BwCu1Q9Wy_C3bQYrLpjOTsOw`S~_B~R^{HC+}Nk)E;+ z?V0uKUpZ~5iZ`A&N>BHb;fAc!Do{DgQf@Khef*xg(7sJVX*6mhB^_#eFLauoZY?83 zPw8jMP$$~|y|+`iGNqcl8QP*uf1SR=P3GQB^t27Q7_*d*e>N5CD_~|rGxYOt3GohC zFXyQseLT?Pj#kKl^Hh^P@ern1RMU&K#qL z<8E`H&O@gPUuu+QIF6q3&Mc*iQkbbZsrNkC$%xL@!`Kjwa@l{JVlzXZ|0y~Bb`{#+ z=)XxRf1g*yG{&;DO5~JgDc#ftaMZR-O!`irO2^25j0)bM^rf1CCrxBt)s}~X`>DO~ zY8ZV7CHT8Mdm1~7ARa?rz#xd@Q8_1D;({i~-?oL8o*>@IA-{Qo zc-pN+^8A5&c7$&$)qh~|eI7iP%>bi7az zP0dVFzx(<_QQbH3jQh9QcW4gsX@-i=*AKw)f-Y@U78#Il=G5u5Kt*?dW+Uz5nF;!CBs)x8+icSvDTZ zJ#V_(Zab~Ct6z0|m*Os|5l;7HU%lw$q)W#p+&!gunr36ndD`dhF`s-@8&l(at-F`f z+QM!>Qt%MFcFM+F)^Gdu#JhPJ7Mu}X3UPiXmcJjAiu21=i|7;^{1jZ!yrciI{}TQ1 zi(dqM=UY{NxVqwJivfRe$A_P#hYL>Fovv>7^x?j=Wvb6@+rC!|dKc`tm_9YYWM|(< zg#E%K-IZMm9=X%nhVure_a`_{3`kgf@@aO-VEHsK?ip-yW~M@_f-D!Q9!F)4XZkLf+h`lQgz#h`i=laJoJrbQ|Xi@9x6S zd3GWjCij1^Pg0v0d7{@h`ca69@x|)5J67gQh^c%lanp+M(wr|P?L~_Klko#KB_yVPRH9y z)=l&6rjZS`=LhY7h}hTNJNP^K&6881c3Ha`vK|;|D16Dn?426Qg1!L*_Or6q)gB5Am@)pR^IbmEqJ%34)?~(BpZfjv znbhrV;w`6rW$wK*5ju4zUV6!V`PRVNVfK1A?W^oc#jPg1_kJ%Oy0B@#9s8WXRuDc7hv>rgvViAX2huX&rOTtj!BHgAHL^iq!4WFKeU zV7Z`VagH3xe)eZ67lM;`jaD3sUVOm+g+x{H>ELyHjJ}n3D`ksxtS&J;xQAw;vgKmmk)M;R@@?B_Tetq4FC^#4CNAeQ-hzI0 zf~r^Z%A-n6T&kOz0}~xa6{5AH4t*c6vp4CR$QjtKm?OIC9cgQB_%^O?$PflGkjufZX$k+qAm4t6o@tWaC>I&gUFdFp-vIb!eP;POk<<*VDN2 ze8y9exP8QY!_mi87n`%~{UQ+#<5}OdTuXKW{<>UD;&VN54 zU%&O4!a*%l&eOcsElY17a5$dq|2Zbo{h_>&>^ezfc}c^X#1JVqYoTRF_t4pFjFwoe zIFUFtmzFP^nxK^{p1E=#?W+-&mt;9D{pWA38M`knOE0tDYOUC?y1QDT+b?kTPDO(E;F zW$f4v3F+ARcs9?Ck@D~QvdPOPCFN3Du4ns$`rJUn)ke=>s_imA+pQ((qo7uAY+WL{ zJS9G1r^V7i!~AuFldNai3EHH_?`uCPXp_rtaj9p@dnq5I2TSwb{dk;ecTB5ksaJWA z=Js;O!AF)~xbBJRIDMc0rQP_TgH`dWLG?MMD{_xe+B)ee=ZZDnjA;_>@ouusw;7bu zZIhZ?Y;UyA!)a3Gy$7YqroIQgv`g%3ZiVkmF6=ufHTV24<8@BYCx7TNajJPUl%L%9 zfv3rLxIfMDZKTs6*Y#ggbMK28tvjfaTDr(dG&ua4)U(@*Os&?PF_b#@WR>vhp3N z3|V_4v18%6JH?;onznCDo*NoD=jUc^p{B{T0d|)&J=t8&)jn9!togR+P4H5?c)OD# zDeEi0&fXn%@nECT;e#EUXIq~8q1N;KT=(9DjmtFsqr01x%0p^oGA`Cv?AuT^&t~UA z)9#5=rkt5vHg(|qgYB6cmu+Z&S^oIGO!V`5l}5RH{5MF1$%EzNr@%*-V^;*suG0W$!?Y%Vn)`H_LKA}&BpY1*Tap{9;+8HYv zZt}%jHM($#8~2_#6xEjVJRthy7ZW-DhS>F=rfF;%O&v7Z&#_>_X5Pw24rawdIld?Qi?1ck?9+?rK6$cD zb4K}xSCPLKMrgl%va+LL%A?b5(W8}ZQlYgUdpWKj37Yrw^x5~#<{xI1{T{IXaLznZ zXWkXx{cT^LrZvVnM>j-I+&nEPetEvuv+D=0HEgucyk4ZEFDB47k)uAOVAyDN>s`eN zkJ}kLe!tB)v*Ce~n&uUc_0OgqP`KQ(CszGN#mJ70fjb5|w~KeY>)f7SS1EjJcG--I z3ooAe?@W}gI5=(M8v74|H?9hg4BU_X=00zHF(1cT`;OocmC4VYLt0(6P3QQ|Ov{S? zuJhr^Bag7J+mGCRa?x>Uy{Aua*4OD7+85bEl`G7puCRMPJpZcZXJmFuZ+lAC(+x{` zZkI@{H#_8c{nTr#i{H|AJW$p$bL$I#c60it7sKPVtByF@a6M9*_V8Wx{*KmnZ{9_8 za4$G=^jg+?UX8{Ii#L($-XEIqYsNu6L)}})L+pR&9BADpn%5(mx8{jt^RkE@-8Tz8 zX0J){n-JL^_)4;^sPJo}pU$+625%0jJ-N>IN;Sb2->3D%mt`EM_Cm63v%H}?soi42 z7Wy$GZ)xtOWzigm-X%@o*kL}%G0d-9H6^_E$meg1m)X(dOc$(waJXaQlTBhvzD(ZO zl@&FZxSZ!j!a_M`y=^`tU1BQn)7Eij(R`*T)~#l5j_ZkS49%wT-?Yvy$hJMD$@O+x z{}$Tw3+6-0;*%wmTWm$|W!^C|WCk)G=$pD7j+aM${%(z8v9PegjYpW4^@aQX1B zsT@rLc5FSitxF%56?op*W4J=JhG!&n$D=su`R&K$ek6#VzwcgC{`J7?o7-DWO`lCN zxwpo1yR5SE%M*&}oX;-=^LOZ+SkO<%xkmRECl_g+aS zuBSdr{+^4}#y=X#zkDJmXNHY&ML zykt>hgIJMBliRdgy%Mh+Z%mo^snI>nLtXus?8I(43q|X-U!&A^3R-H)>?>^Is{7_! zz`eqF*}E&&33E9)YgV3p`E+Xhhnqe-)NR+EZAhGT|3Z~R9_J2`MP2S3a>Kz@CPr6d zH8{95)HjQENhT)EDv{-impbJ&VbI$rc};TVxS_4d%1x{9ZhKw!c$iyf>c+(9zJAra z&#LNpTCP1=I&k-E7#ncpO-92hySCnAh2;$qaSB5sO1|fd zVxNur7%uT>aV=H9(9*}J@mswytU)iO?z3F^|3lt;$K~|KkN+(dO{p}DN*YL;iuT?^ zNlSb0B_X9qQ&EaGqFvgG6b%xUlt{yf7DY*t@jchwb-3RbeSe?t@B91jcX_zaInVRD zUa#wVUF+O;=bYD!#a3pgxI^S0Z|zI?B?U{k{|;xQH#apkwUCf)puMjUM#^&i z(x<@3Utc`_n)>)_+2gNiF2}Y~z9%8sJ40SboX-XvVCj#ZSli{8NF=0(C6Xdhx6IU3t`~ zYxtk7)+~-!yt6I*DlbnIPu_ts1{eFStY0)ILc5i&SA1e{dCS0f=Xt|1Hb2w%?{8gP z7#@6IwtvloLe8tqYk!Qcx8zC<&LKPh$+d92)rDh=wa;nxygOfaX-3dCa9uGwVXT)R zqmz|$g<6`=D)G*>*sZ3cowNt_G+S1Uny;1!BIRR~V=Shxq~maT*j?E2dB@N$GV!fp zJ*0sY&r}V^zp=E6w{tu`Cw#b-^&PJtr@6=@bw7?8-R|4Ej31MxM&-tp=i)~Ar2F){ z>vropF1_|t;+VWxv~c$FT$+9ZZFX)Az3j=cMb}LqXi`F)N7sHUCS{q_*= zlz`o%(u70CNr(AmPF6(JM7mF19_4#WxkheN!qbAHv%)1(T?b_}=&Je;#@iQ+yf`s_ z^s^oI+I|Y|B+HxBn${CG3wXgu&S zSKggOyyHLZvH|?iilsm8a_Rsf7Emi87EpVqN8jF2#c~(*b!M}Z=QOhn7*}gXX@+Ts zrBqut=5dk;y6>$vDbTpQI!8E9KU>Y<=kt!2xu%L8_TCIbW)#P7m|lB&{^PqwbIMl{ zli>m;MOy1RXS=^-WvvqCEIBsf`f1LjuymiZYjJ<-%>Fj{Sozx>Z<8{FQ>|K;cb@Lxi8c1KbGg?BN^V^j3ibtNr zUUfHrWj?eznmns@HI?RoJmp%$?!yU6eT4>_1TU0`Wm1prSKqUDZ!*=`woi6DIUKF#+0ulK_Ex6i?GI85w+_F%#b;SE zbDR0ehg*q7?RF2ePYkVD9U+?Rv}&Qit=R3y<5jLK72zBgcGhN(ol?+A5X-kJ896=i zX7uLZF*BiB2J5Vv^l{S?#tY_wREHl(#ho|WcXBPwmj{zi!XCXeplk2C#2S2c{D^zO z)61*MC1OKkmGcI+e3xZB!!FoF{)OpH!HntXi>N1&Q;%Y{P0eJiqwvaB&f0N7=(g^= z$^tf}jaJhyVs1qH4&|tE4oM{@_my#{ITRXSxLUC_F=q5rUD0b-xtrNGVUVAa#Xma$j@%G?c0EwEH3SRkEPrN&id?lexr?YmDDGas6i6l zjoY46#5m?DJ4@K{tiHqZgZF!Pmy5t9Y%Of<{xS3;tvN)Vd{FG5Fr_miV9#rpSc_sf(~xt}Cm zK{sS_rY1J9oP9DltaNmPL)JYzFIE?)ob&$C@>VT;^Q|A6X$GUudKj)aRcB@(8R#l&~ zF9%L2DvTt@Uf90QG;eZAlXf7-{Sy1p%TLW`npL+9U6G`o_;R#NdHRRy&x*PxZxx-Z z3x#D78^~7|MQX7PacF(cy1TYEz4P`}Z<|z_9EoXFXD5x5>)ZVlSE;wN>Yl} zIt-qu8|H8%mH)Wocz0+QnbpQt(-3CgRR0GX8{;V^$z|OYEPChJhLT^=c9bX0dg<&H zREy;)6lP^D+23`pQ`02N*z}NW@W-9e^2ITB-`*&xnD|70*!+5(MHid=P*rj&Q!r_G zZcz~}}UMTEREG+1|Y-FX)! zCCbLKAwlI=R?;$JYHITu>8ss}Pi|p&JDIyr<)F{iXD`2W^I zkTi>jFTU;J3BSVPaOqOOWGVSNx>MN}TAG{=!wKJczoZ$QTeHkygd0(>qdFpUElFW^UQ{-a+P+ zMe$wk-mKu0vAx+%jQiZ9pI%shvHD4OgpgCL?k9s{$|3QSzf=RAjEh3I7i-W8Z&s(& zFpJW-J$~|RhHC%U+ltzIV$07Z-hA@7PxB3Z)Vp!+n&QNsY!~{`TPIXC+#H62bWIj2 zXS)4F&zvO{sDBvpIq@X@xUa_2k)^LvW@vULxM79Bai0_)q!lUdPk;tKCfNL>n6_V#t*d_JN3fzgyCt>2A_+x*um ze|ze!t=f@>6!AxHKI!Jn`b;LgxmTw|YskQaEbSTv)xnK*7iC%3D{GxgdmdP|pN2nB za_=h*vAT#cwrs=Gaf_uJ-NzTY)}PIz^>xr^Jf-jIKqaptn)KrCv*T=y0g_FYYg~-R zG>;e+hD~J5iL}KX2)N)$8EJRKc2kG1wwU%Vl1SHr1@(8zEO)i|@$s+q)}&cKV0qpDVGQdtY2N ze|S~>P`{2|osggPwM$PE?6SJ|NE8icuj@~`PBU=uP-Mfz-df5+YBh#^Ie`H}&5NZv zc~>r!@7$m{veD#)n|7t`&N7LEw_P*dZ0ytP3s`?nwE0m?o^MtM&-F6%=n~rI$-6xR z(jph`&rxz_^jsv(rtCFr?ioGiYjx_C&_O;C!LdnAH%HO8pSJVI)ii2sZ}_~J@Y<{S zgEyCrQ=J3TS@sVGJ(jvZZKngc!meD7z4;?*Riudi;9iDLUp3h~q+{bV;L*RnZ(TKf?$#p0*uk*%$%jp!FMT!*FXOA6+H$y!d!oPp%QfbDW}|O* z?RGoZ>3GG?zWgQ{`S}>xRVfPX5Q%*xtO4$)1+JC%us4v#tjYLjCsHn`Bt97`b`Mu)jTLs%}zP=aka};HZtc3>hbfs1Mht+1#Ar_ zclS5f?FrDwqSO46O}t-KEA+@j*im-ZPcBCH&x7%&T;{S?mnB^aAAd;Ma#weOotE97 zg{4ee?(&t%QfQ55xW8^D3WJgU_!PsvRFm z)2Cs2>-|5EP6PgQ&4v93_a_p^roj#g36D*KUuWab`k=9Cu)k3eVgP_lU>_u~6(Q{X zxpxdaf{Fcki;xJn->A?tnysOy)}~P*#HW$M&N#sH;Jzaeb0(flm7XmQ#GHwwP^1@# zSfxU5Kz4Kuy?_cm7Z$k&lfXjHsM8Ss?wkYjK7o19kSWo##gi-1bH#zYk*gHxMItEG zF$L9Y=tVGsX~iU#btux1=wSIqp^ho21KP)|MPe~;2=+zDJTd!N-rx4MXxtHkm1z0d zC~*G?NP^%4tFHrAQ{YD($ax^QB6t8C{GEX4%CU1mEB>ASPh#YN-2aM&v&?D#R!)$o zTY-b8ApTC|ggY7j#o>)W$SHCX{K!MI6L49E{CCg%Zys{O-5&pC z;E0@r>u*Tl+<)=qTe$xA&+^w(+TZWs$1{Td51zELA#DGA0L{d)ZSXO)7vuQ;A8G6v zxfQq>LC>E%HSxWG736~n^7tJqe_{e8t{{JzAP??TSWbj7itU2!zsh-U1>U^^2V?JF zdKOmTq}agtm;BllI1k2|NdEL&!aRZLR$W1UA3^?4e*D5Mbwj6cp!>tCyqoZ@tU;{)&9_ z=X{_SVV}gsmJs2_&co5#&(YKSkRw`v0B>uz!^C2QFgI?8t!@4MY=enU5MtH1JN}1( zI5hxa^8Nwf{u!`G$cdHE-_b8%g@8GK71SCF{bOwxY)y!3gNMi=+?b9Q_^*0-cze4C z`Vf~CnnqpbzwAFp7!&AE83;G({5}a<@%OWqpuNFdfyWdf11GTrIJlP#=Xj3=9NbTi z^Uc^kNPvS|HE<5@`34E_jo5#jgY^|8z`?z4IM)CWOMnBNIQIY$OMr7>|8bs;?Slk3 zH})UrMF3(6aL{LQUI`$U0N;ZB$2q+9mcWVa8UkS~p>{);G2wnZF!$nm&O#34w-RyU za)QUNarrJHdE)Xc&(F(tfe&$cFt36H+6RnEoVx;uCDg9(GA3|vhZ-(_0dk-RTm#^I z`B&$^^*n^~KtEUwarxIo9Gt^BaX%q0|4JO6`N51%yo%xSfP?w<7d9V(1aDa;+$WCR zJ4FK4Cy?N+tiSm(B_V;2yc0OMw;H#>2L*usKeOwf^#`{b05eDv$%FD7hYOo3!gs+3 zZFCU60Zyd{5bgyF?u2kWc7Z3tzd`+AJp~CbC;n~@u%3d1b75%r^fLa-al1T0-&-u&kG;G)=!a8}rkc>l-sm%xU7fyiG6jfSPM<;`#X zG0+cl2tSB-9B2pEPfxWhNQZC*c!>w@@dgQ($KOrMk8pW-j9*jZ^6Ic1@q6iU9sw@G zv1EYg!QXudV%>rCTORts1L62HcwikTmN$j|k3x9UvKR?|Z!WIC6#4+1LJl2Z{!tr-8$|3v%9Pc^^$KTOtj_@TY?}BjrorQ-GegXRJ6v8v%d=igvPWaX8 zBEmbMfAE-OxPS0>Mph#7p|IU&5l#*J=OV(d!ST*OzT9rO{%cSV3&Qbt3~?fy2995D zgnPnceuSIA`U)W&k1<0W!v;(-d*FDNMsc*Nh?j+dVm)>jpgcZ2PsgYXvUmjejD z3){;9;rKgQd=P#H>JLFUKlD!w!h7MkNJTgvVBIdB|yAo6&5rV(xj z>-7!cxI7hXA6)-oD9?;=@cjfNZiM6IltB0eD6fEU+^#voal6h4$L$6n9JfmxlLqu3 z3Mh90k)ML?m4$FeIM3uF+zawTgx`hh6#ni$+;5*CuR-MR!~T2=;rKhZ?jjt2m+F0l zA!V_S9oe=H@$BQq* z7vOk5hVYHBz3_MA+yu+4)3QpcZK?&A^Jn$ zeDxOLym0=RMR)^TM}8qZ5q$Z;5-s!xZr2g&XGgdeP7V8K0Kx^K{27FoK!2tq{4n%?KEk`<^>a1CEnvUG*B{((Y|x)w zh`bl{{~*H6VLzEbIV|TFgg3zYlEd{Mx4RW`JO&5O^P&GYBJw6sFa8d8Tz)ldm)(dw z{w_upgujD&3=!S}^&CVvIqZM#2tN+zlfwvCf%Q6t@NckxCL!D!&OetBP6Pd1hHznc zd>`R4&~6XHgP{IFgl~iWAAhGG?$5Q*KXZt@CG1!DLxQ-xE$k8-j=xhs5aBtny}}Wm2>p!3?}XR?^$5r3!+Qufhw~>Mg9GlGLPF+O42ey|v z!hPX5bw;=-r@4#u$z6apAID^O=z;dP|Tnh4hgwMlv#QQI8Lsc_y1LpUB&^bEp-pg%JZ&IjvN zh;UNqw;Kq*3+vT}a6A@iFT$%~|9_3}2sqw9A{>u7{1f3yu>a7)_Q%`N3~~;HyFf00 z@LE{@T?prd^B#U(jO)kWO>Tt9cSHZ!Ap8>KUI_PsJ9RWkWx+ zB0L;kpYS3af5)L1!gb&{RY152Y+pTuUxVY$65*eqyc@#t?-hp;o(%mEiST$>uVjQL z!G3rd;r`HX_}_nU|KRULzKzId!twhM;X=?Z{w{Z1&n9Sh9FfQU{srM2&_5KgU*LN1 z>jpf=5YCsN92X+r0_UZj2-kz{i^n#__24nVv=Di1I9?7QoY0xE4=058!}&OO#(*pr4Z9_OMEDwI&gzFK8<5~@ocZKb4jPN+vXCiGVY!uw(SHX-~bY~N0V|Agai5a9{1 z-SO)zyqx&&gI^JO{O?SZu)MfDA6yrh5l#d1^C0{R{JtiF@Eq9g@(9Pj=W8My|GvEs z;dl%IJA{wHapZw;YiO4^1`Kfh1;|e!^0~0x&mkPY)$S_7U7_73gxkV->?OjPp!^cT zYoR>;_afZ?dT?CqhULY%HdY6a^buYU`@bW?U0}aDgz!?hzQiG1751OY2zQ75umRy8 zq5f`!FT#F~zq1c7e+%qCRM6iz7lwXW3prR`!8tsM5Xpln8>%OfKsa;*i8K)hvL9gv zWg-srY=;Ka5zY^}4iN_`R>5{LA>u%f8k9FfxDw=~68!!kz`;-#fIJ=169{<)!iypABH}=g4df$49F(UU^7jaT2RRj-7x8(Z z3KR@Ww2*^C{L&Ec?L_jxKMhb`7~$2BOAvA3XDv9X$Psa%X9~(IA^Z*G`2AvdIq~^q zKao6fJ32rPvV-!AVgEtGW69v$WEqnrAoAxS$78wxdEjR})gba$;E7IzKY@G@a=iR9 zu;0ETl7~wIA;&6klQEWnJn#?x&iB;_Uklg$^$1smoD<>lkaH1n;5Xc#!U)I5g)HR2 z51^d*ysd?B{QgoqgzI2=K=MYo&oU;#V}arNaet;E@?}uI0^zlgH$jg3nHSDqtwi$h z(u$B{lt>=<*$(<=9N`Bc|3Jin-}2$Qxj@8$o)9R%gz#X<@kd|qa^iko%|cc z<>?Tf3^^0zz;B>l_}|aIjxB}sPA>C2ag$p z_uFE~T@d~FSUOIm2b9MVD8o`TkvvpE;5mpqULNAuFhCyY0;v^|?}7T;SKyrp$FHw? z5RU)-wIAUW@H*`U!dHO?!V=y-;1Ku-Pdnfa%GD7LyEBP35huPrIkE!B=QE%m$m4pd ziR8h0fv2x<2gnoiJpf<{&W{RI0tNOaGH2NR71EL)NfD3iO2Qv z6*zuhD9}$lUdoB&iTgj^PeEJJK$~EL_b;FaW+j~EB783UhggpY5hpIE0ud*!uK^J! z_L~h6CvF!XB2L^N!iYF=e}lt-(2m6Ahtm~-6PKTxNDp!O@pS-guU$m)#N}5c;>6|0 z*9V}7xcvKxcZ<827sl*j9Z z%Qr%f_fwqzEA|Y~^RL)5fWL-%@aq&@4}M=b{yhWd_m_+xU!P}KDf9n_y6x`KH%9)&^7UO9NQ%%1-L~wup9rz+}Ivq#A6AZQ~)3UNN9e3 zOa_dT|F`)?vAjU%O6A9|Log9SLXW|9B^d7tgtLD>4s2cChnJ}&pqxMsAJaHD#n`{> zxlsE?P_F{EB~Cv7un%;h_QAUGFZ(u7nGoFK8`M5nkN#EvC}>{mK{5E{cIdDIK@`Gzi+&NRPehwUvf-pb6LF>OB%lt3<*MUJS;q?TE;JOC2 zuegH!ZfKteC?QIy{k<#LzyA;W;7t_;lpmBC&ySZsa|Qb@u%T3N!*Gk|$1!+L>tFT_z{HBBXUiMH{L6vuY@EK0VMr8jC3r{_eK^KNppM%8wUoXodXwpt4w^#|4)Uf7c(!AMpI}Jie*fxV$HT z1GvTW;oC1TKO#ml4D+KL&xdpHJTDb?fNlPhUc&q$|Dk{Ck7Mw?4CqAjGXiI0$p$fq zwjcQ2`Ct8a9Ns|PM9@M)=)Yh|!}H_)=U=u#7N5<_8M%ORECE0MYyJS!6P&7m*zc6M zWqG$5DL-;Zn3MmNKi_gWzX0MGG{3NzgcKK1`hSYWBrYYkOM)Q(Kaa)qHR^Y@NE8UW z3fOutiiNfL6|iTYMl2N;0Vja+R}>>wR*hJ75UU3qf=9XuF_c0Xm!r&J5qT_@LohAJ zIBNc*mawO2>ZHNo)doHuDDHP+@VKM+SM(i?|HNpL(unTRh(E^5$RJ3;3ypxo<=sZm z$ePh@%STQO-W0r=(f1hEVg-K@K>IsB5EckWGuoO__ta--8>GY%YZ12irnW{!iSe}Jo(ZGfw{7m2@jpr5@X2^P9ZhHH6W28-h)gT-qK^z#6_ z|2+lXniJso`y9}_&oKbg=FAn~XY1wf zeYm{6Fr8dM-hS?me$Wz7f+ZtIEa;R!(6oFO$k{pC+XnhOa(O#(`8oPH+6I77QvQFL z^2am*Lj*lo>Hi4$VS%Xp9Jrjkv0}M;5W-jmxdym!xf1laVnMCE1HBw}aM=Y0a1pr$ zF9Igu73ksdf2ViZt$(RB@v_C-%o(8m%XDua2xa`A$HgA7775C36;V=gwSMcq^P$~P1Q)q zSW!tw6+H`%30eO-p{l3y7yB>3O%k>mr@kvl8iX>7p4N%( zht-rSWC_5XUoCG8nMb^G3O+gGc0SI zYGv50^4lSzO}~Yq93juLC|0Dv)TwfUJc#Y38LOqB8>_1z+N2w+%BUG@#ApID{b{)6 zoS?L9qCGm%V>;1K6@P_;PJ(Sf>n{t`V*r~*v5Jl<{yGIZ54HhqvwZphVfr(lV8cU^ zk?aGjN5p%^znti(fjJSxBoo$%R>7PX>!*Ot<{Ht=_!5fI~E5 zvAMq~V5>$n1t#DFaxELzh{e{AFwkr6g@KY0gdO$;?nq|F-3OUiO&z<4ZH-EtwYm~p*G zY(NIA#kr!N;XUua>k2+Bu|D^wF|gi;McX4Z|8gloRKw+k3OI`m4+MwBq+>W)gw=D| zwr1=tUQn0c!3?qfg)_9A(ao3-#RviVG^7OrC^e*Ar~`Xeidb0pyZ>T=43~{fZCLJK zgud_#R!Rdu`&&#Sx)p1!z-NB|)}}e9t8IZN>#^7oW!UiT-re z?KrTH;;I`<;RmiYMkUPmLVJ07)@m&*{P_9BOPI9!5pC^K@Y11$ABzpOGvyTnBHtrj z**mS{zn_eI@?F($Zq!CM(O;ZvjQpNc=N9SX(zK7}g%9?4P`urnckS>A%W|KscT-jT zKC(IGYnQ4<>YSTt=&7C#p0;wN=^2yPa_Zcj+_5Oq!7;_Rn8z&pE{n=eRcJc+pyzYN zF@qgrM~knx`)8zHX3ZYEYC!h#@JDuOM*c8?Pn)?EGbjoTtU5$4aXiR%E7(w&HET$h zob8rNW*qb2N?Lx7SH}k z5?jj-R&-O-Q@#82z^sYKj^cW9{wnh*!zKuyks=(w#pUG3&(z$02% z&%4RTk58xh+|Jt38Yv|n`4Cq9?D?QhMml$$rS`=X%CH|(3QqLpq2!}XLnO-0KZO{1 zj#G)~y>|{{`*bu+(O#a$m{FEw%l)6^B2kCwZKR&RI3XULNn+pebBhoCC!QvSN^7#V zb!8-RGD{{lJmoYt)B`t96tuXLym@?l;|0a{@wZ-YNQ>G{N}1QdVB<5egzbcoU^_D; zw0cXjVRu|NWAW?0aAP}-3?fn=7_jp{);7JYk|)J*qX)YYCMJ%Z4V_-Cw?t!n2FpM- z%%evVXS~GwF-XU^##ZUHK+?CwS;Nx>VbSY1SE>eI%i!d*A!S`laalz!kxNN|Wi6#! z6t#i2%84VF8C|$y!qhIaX{`0li54JzxAtli4-e^%tq)GHIv=A8q}0D7Qt+lONJ^Q? zUno@f`@5I@_hzbxUfyi;o6`?o)Si8_dwt{a@06ccKM4uS%l4^c`F=#d`)KI<7fH8k zf82W6f6M)fFl)d}z4K$ik$mZyIji>>8S!WCH}+L+qSLCoy4~VYIDdF#(fK&KSaa1= z0lUtne=LZLyt@=3^Xnu})Miyu8&QjW`7Rkx_}kwt_H$ga^R78C@kB8wZA~j_r~DXV0t^@g1Z9U(DVy%|@jP0hZZ z;q$iiI$2cm@b&h+Brkc{Hc?+c{7R7|ul}RW_BnRCev*vtyUnDXwy!GZazzA>(cIl~ zJw@YF2V?0er+l^7rF?zTH+JbuQkv%%#`^3q7)xp3ICn=OUU>ULhf7YUFiVM{WHP;9 zm0qCY*?e6|Q6BGSxoNbX`z2q8Y%Gk;(d%oy|NLrNxcg9{w#dSK{?1c}3U9CaV01M{ zwy-0IywJ6qyDwn;x*O}=`3r9J0hW@cjTd`s=dWmns5hqdp8xhd_uW)bQvHJ;K6a1! zgyQFiG@H+nirV?`GJkK<)Y<8M&`ITP@MfER5x1;w1XGEsZCN`q#l7*?s!Ok4m8IpZ zSrZa$yD_wSmE*!`$rzdk_F~oo=X`YnRa9vs&gRE?iVx0&DmGq_;3U75Bi}?KoL`uD zx%ax|czh&#p6$8uqfn~A@@QRER8`>TqQ z*;mX0S(1%}US2Q}Qeht}U}bsm)jZcc<5o{_vr%7A>|68L$@;tbaYlXf-7`nYXg3}yp}?9URG8pfV$y2W$<2ePe@01!sJl z%f5xDZ|n1I^EzuQNi|e5_qJ22@(ah#^)!xq=YAb|cvpR}EY2)B*XI;wgM#+h#=zNiCpY0NkztTT-cY6E!9D1srvX($&UYTCn3=R49 zyF1pLn2D2Tk4lviWZHPcS)^!PK6T@{&bg$f8wZ0wzC0K7@MAAiEz5zpDT6q7?TVNa z+E>YEbGMkL+$eDAIXC2_GX0FJ-Cew^j!fvQ@l*fr{C}J?08OOa_nsX}h%;9(61QtepS6_L2Kds}}xSDmYw{_bK z|3^n=KKvM57j2L&y=A<^a!Ob}yIv;kcu&Y`=c1sbPZPnzROfdbblITzb*yT4iUeo2 ziB?U<7vq3azRBcjzh(!M-16R?)_z+p?$22)PUYAr_hV#f`|GTu*Q7Q040Y0{r~ZzohXy<*X05C~d_N$hN7r%kW@WvwZGFkE>J23|N5gCTW=+{{ zjveNt8v2o{kWKx}BJ5I0XwNh6OXVRSGK(^w4xfJdr8=lYZ92k|D!Wi=uH$xvl9*%0 z?fIEja;N*TneS3foy0m0XgQ~>E;#MofyK{Dt#j`Z?D_So<+Y)%cOQMHrPb>yzuXMd zh6yo6dcm0`iHllY9l6d6tTJvE`_)VOuhcCH>~VW>*jxSl>obB|WP>#LJ^J;=Q}mu) z;vQkNve}#6cPh{Ok@DMtw4tK459u1TVnm-!@C{|{99^f$Tp{jG`)<$H$&B|OzKt8b zu}yyPL-}yP)}@n{3k9dg>BlB&`OR8{-o-s!wYcH?-u8D^(>siBO{BH$_j=^^Hug#L zr8*`Sm(1>)WF<+rSQKv7cFxw^+?Qt(G8J%iMCOOc_K6LPW>dE$*UjFqF28(lUwCm^ zS>a2S%E6?cGzC&U&ff+&$s3xDBcHk7@;LK=!@8$|>FE;@Z%Id4`IS#>4E8vhvxdV!DV~isP5e3xhXxzGxApyn+^gq_V&s=HF)vZduMeg5k{lu(2KO|!%Q!Bj%9gUo@+G7<-*dcOLnWo-mub7XyL_vz z+#>x5kv_d4-KO^;%6U)@$yUitab|eJ|;xcJkPR!RQ^CrYSz0g#jOH zOFAtRTSu3z*7f+Y1e|qXdsa~Ik#e0vs43_hkxz34kvJX7i zU^saw-L%*x_rxi&)fUXLp)EDEF9PrX9M2F~swGR4u(;{kAk(ILPhfA`)yT{l)$;E} z#$RcQlRM0O`yxMS4jg&*CHj$y>C!ECGq~8fu z4a37lK|;rJt$Ft7FwniJNv~oNWUopcVDL?HVyjNdV<)?LZ1=74;jb=XKj>>WO08Qo zKhk(dy*;fxStU2RT7=FGX-%a<;(wS>8By=jkK-@(&ag6@R0t_kUGbwDZXDLpJ_5&&+6NZu<{6EMy*?Su(le zdR!=HC`2-drl5~!i+lD1Wd~aQ{8yX&?!=$u^bX?=jd6u%a~ zRk!hBqx~wj)a@eAUDIVt7dKUqPjZ}K_CEVS^u(ZpsuxwknO9R!drkVU(|5hzCF0gx z&(bNRuxDd!2^xl0pzw>)asr&lSML_XLXy4Cw%VoO#>>af^`A{W^K<8mgt+ufo5Yq=X^}Ls znKjR|o+R3sFgZ_>?yY>Lnf~GN*y_TJ3(lJ#rm~2?U93oFcu6}YCEn5Zoq?nD* z;TF}pWA@u;f`luJ#40C~=gzy>57rE89k|iP)2n%UA)7qjsKF&J&)&*1Fwo*#;DX^) z=-@!lB2oYECX5E+@v!GlY}BT+q%4*yZ`?0^VqrI2Jm~U$zr=i|%dyoG^CR7(s@-32 z(LN(3G4G(b{U`sqzw}MnFULr=A$elZbhqij-B)c*^l$duHCxaAN=Hb_oqJtI@R~aw zssbNWM4Y*pk10pRG@0F^B^lD8#BN-|$7vM!ivpJLaOLn1{OLUPl|4b%&QQS)8__omwMfRbJl9AzOYPB0_Z4c8&9MaU&yJXFi zx%IQPse}T5&;z?om1NF)lX48KIma0zGETBiTAps@I9RrHrL*x-OHJx`%}eXQo?jhY z5y{DRe8D*ACB>YQaX?1XI&;^xB}OqRbR0E5K2ClR`z}Upq^PC!EUMd0fBN}p)t9Bo z*V|*<)7~!G)VOoV87g)R%AU|V@m_as*VuO-)B2-YOiRXB&mHbAY2DI&MPz$#_0tk+ z&FK@~n@uh6uGU^r??;|+FF9-79#oLaI$3-C!K1qxq*pfgo!0C( zdPg6rY7}tSY0U*IU%{4w>zW<6*G{|-C5yjfxoXI_B=O#m>35+V?H#PP6yo1ihcDm0 z-#a&WunVt>4Y_0r3sw6wM!~6annomLOZO!Yn z7PQ&AEewJzqD#Ff=S4lvAAjz<)05}cu@b%UUiL8{#-#C%c0aYrlTs2qhoWrXDoLg>Dq9y zCbHXq{kgIecBQ{adZdEN|J&b(;k_V#`hSQSQEb|z%;$x{7pPRPtut|GzuLhYtfa`x zYuoyQEXQ%CziWb0!Kg;Uq~Lz8M$Y4qZ=WajXDwY5%zQ7i@BX0M+v<-GGBZBM2i!P! zuT<_??P32iodTLy4}VpEz8mb=aNnfJ@6E{Qr4NHXCTu}W`y0mx_FS@#Pdz;NCi>~F zxF%UQhc-4c`qiBLBy4U+<9mHu|MiZd3hDdyZ)_OK#4dKFK52KVvAm!6R++DI zvo-&Vd8aR?eDdCTBNdN4KB;Qd%}iQqy-MqRXMU)LxvgDkkGI=f-F;MBLcSV!TBPkA zc21S7-n;utZi#(m=hLQwJ#VX?HK&U0v?ws0C@CDu+Qh5x%H2VEM!KjXxoX$Q)&hY0>f50oucnVLPD~FkzV6yEGdR=G_wA=DxgzV~COWnY z+0$#SyLNxd_jJ2eQ{(LEi z((CHxiT*2-8*Xl7GrIO}_iWoF+xvUZ_a+s+50*>UV-vcVQCguHOjcMNc`?0D=mhW2Ljrp@yDl%%k zA$xtEb)N2A*a;Czku$?*vsWpFwA2M%y`9CSpS0O;F0gv=_~N%)LnjrI*BniLuqoi< z0iCgPWC`QZVd}%XRqT&#yIZvj4<FAoV{?j?Jn4J+nxFURlFPbDgxhJW9&d;oBV0!6m>B^l~5pUv);!OI?oLXll4Ny@9@vWW!InUMZ963YX66y_21H zJ(5OpH;8mLcNu1B7Ut#@e%DA(j*l_3E->Fn!WaAO_Gh_UQ4RKyMe}vS!okIFPYnlB z-8rpMJ4v~J!Y1@UWNbp8Wr-I{-@W!DTllk6iyl22t#H+|+CEr5_g+cnTa$&SeS@*h zOlo4Far2Q8(j{-uVc{$Z(S*WAyZmGsQM&s#`+gZtXu zN8%6Py)bE`lA73kh@&h<=4k-076*+fP2P#YEn2Aw0i;D z`FVlVwVnU*BZIvH;@;A?IqIJD>M5%(_8+k+s_|9UR3ADm#9w||YAQ-Gc6TpblKSm< zj#-qk{s8X6bRR*U>RbepQ`i_sMM zMZ+FV=4YGA`HDA3=x)-Nw0YSei+$VP&cy6}$tH40aJA#=Nd-Ha=kFJyUY_2za3=hD z9v0wSM9-Cd`?n}=B|X+7W%YhinN_^)OcS(TQ<}5Bd$JR3^s;Ug>KEZZ* z^h0s@W*$0~HBCxs_XA%3P?;R1j9#N!=c;vf_AJ{@rhWA+`*#hL&ofRW%JDztDX+id zsj9x%=_s@zs=R+nig~j0>5pdnZjH?-AtsXVu`M}Q2qSwi5Iu^cReX%=n%Hcn|$rD zCX|`{!C;9WqnoP7`*pIaDS~t)3fuPd*K4TC`gN(sHnsm!EWf^Ki_J$f!JnK zUeRr#HmhsaEz+b`Ilt*uJWs{}!9fiM3$gE)r*4!x(d<~FmX}kT4E?I7Tl~l*r}t=A zmgA1C)gg~5#N~dPb?#fJJI3)yWbh?hU7qvmw_7+ED(4)2Tvt59tDx_r;eN7Bwjoe$A>Pka;Ykz>)U zp>$qg|FcW=;aH?{9WgtdSGHv0-IH5Q!oErrTDM#~JYKS%hmZNA5PqqcN_@FDuB7>1sdN z(aU4(nGw}-cC?qPhkiz!|9XwaFpaE9!bG^ruBIerwek89{gRjW=G9uH$@$e3hC`1> zyx8fQQ9$My%YG{1kf4U{wdwK8H*dA?H?SMI`_h(#TXM~@)wkOU=m+lk9?X0#QWaWY zFqOR5{_XlU&%}oX!5bEMPg<(oy`Hq*mbc&DGj5Wa@v_Fd_qR>I<-{1BPqutVmgJjK zdE!N%Wxx%)MtY+t!!Rwy_p^L1DV#O~@=dnWWoMb~_qoTE*^^Mj z$f3(Po_UcmFoZ*D%j2qHhjvY0^Mo-&ULj?Hty?x0I`9-ry)=E1dT(*|s4(?`uFtY& z4BvIXeBPTp+1b!4qgMZ7%Dn1<^sx}bpUMwpzfQ-h7Gdgq-g)%J)sn|0j-hP~wdpEb18 z#Hw#HhL?Rhee&zR(<*{9cHiZ_H*dDxe>vi7U$^RG8#jqiZ+c&)#Y4Nb6SJf92=R$@tdru{Yn+V_NUM zJ2$kbjV}Z45jAd|Bm9Z>e>-K$d=jb#@IkJn7wmniVI%2=6Zd!l-_VLBoS95O)SWjjTrJTR) z-7)Yeq4bMLlO5^RO}p%@QlpEL-q3O1*nWLn#0tCb@?nvn#)*B@X-|b_j#M$Vi7HQM zp0VfIw;}c;i)%nH$)R**EgSE%^xsqDgGIhwvntEZj(B%z?a()EsuT;YsVnR=r;cWi zteG0FE2VeV%V+iBl zfAAQ>_@g2Cy8_S{!r=B__|=I-5!)wGqi2jD^J=E2#_o;54y6FY(qr&}MeI!=)1hZO zL#{^86;Gi`FA#^t1r~{V{4Y_EUtoXv4n6F>fg%YrV#Sf08R9v=UKxRgNK$YIcg8qOay@3H{ z4m*pOQ>7t0Krf})jGbu13i1#u$s;V@a3>bDxC^rh>ct2hr3JQFz2bq*IPyR7hSlj6 z$o3)8g;i+C9OyOnVS$Vf&}*pCD_~iH?<}FNQ`kOf;!k>dH?c}9`KPi@R1t54cRc}7v;bFqDVuC z4Gr!m1nCR-z{(Rxra{jZ31U4*P#9v>?pi~Lv5eIOi}8%*!ESjYQGj{yegwA0kio}* zfVLGADE9wJEM@Q<4H@C7g=LkBM4JDj@7y^C_z{{ncla_nW`y}#4SF`57UAPYGmD(S!UaQzK2 zLQcZF0*4R!A!o^dzd7r_C&)-DUSIejn_}}fQ3FQeJq7OYN55^R6+DAB` zAxR{L*!}^?#}RG?xfC3i_-Q-H@pqQu+!=Cw9OHcXuKYEG_@+1mTVKM#8WQ~77Wla! z!VV3|SHx}z#=$s6QxXRL#V=r-iRg)B!u*c0$`$1At-!li;QcG`mn-nu6*xIoPp02b zB*C=r^N@&L|BJLY0m$L|0{)+hO424NqD7_BzR+T6RZ&ExNG0t^ zX`u*_QnF=$>Ybn*t+^~?_BZW1V#kF8@~TC%g5K>pX28j6&5xx(kD7P#GgJ$!-u@O+n;%Z zb|~`>?l3=2Objfu-W~pbeVO;)Rl%#qSqgxV-#SMHM}$R1F}LGTV|cIm=03}U+%e@7 z6cE+DZhlx)2y?d<_ovgLzVm&90wUl==03a|eR}AGKYmrsvIAZg&eccmk7AT@?>zG1 z-XY|}Uj28oi1m@te$0(Vk-TdEdiY0A)-bo0F)u{-Veaf=zo-2F_D1y{ek{HyBs_q& zoceEGNMsc4%s*IoH23B2A@jQ3$ra^R>k|?c5c($%4-5AR4)FaiO1RPxxINet&b;)U z=~C|PRXvFKF?YE|u$#>TJxDX14}HOX5k0HxxXUDWX^}6ckUc*+TA=UN9!VfmFt_Ak zE>~yfsk|eLlWi!U!Q$lngYrlgAHm}JEKY2r@;6yro5g8fA$97qI6b#w(iw9+`=1_a zKVfn#9>dI&LO97~=}A_&R2u6k-oK3Z?Q}31kY< z!*1_xF6GIy^^~$WxhR}Ik|%YMLgd+grpL#G zk7Yho|8!=a6vEl|L~-fg_OSbrJ-+f-^5oc_>Zkok^jola6*Er?;pCW@%GVOe6vEkg z;~|&wI2mhH{xyqRvA8s=f7yQP4e}&Dm00p*jF3X=W&6#7K&BAR_7B@{?EXsTO7r4b znVF|{lH+qyu>Jxb=8l(P%siDp2^`B;vgFCx8I@=Ijcq?U4kv}^Vf#UZOL=w@Ur;^t z7>dZ-vUnemC)|$3g@F@&{b8O=`oTQm1DVe+=nr~iOSmk^kO@5_rd$rz(IY*=2QVKR z_w)>uaJntogtUG-1c4hKE4NInMYU5$7yoFcD9oa81_XuMJV`OrCgc&oeZR{-Z7 zLA(`C9ca8!JvPvf)kvN`8@UnjDbV3B5kCrk?nHb&4M3=e)++-6*bnhC2$VsHtBLXy z4@aE7E7%C}L*Rca#Dn1y5xKdM6l#wg#FaPV^xawW5Pt{DixJN!LCq9;eSzw)=V4Ad zlBe%j+=F;68N^IEhPVa{AbOpG>X!ol+(PoRVcPKYhj})uRoAm|nl2ycF6+ zo|QrhxZ-F}^{t5UD#D4-0M*JDX;X=fNz;B6& zdqE;xhxkj#1N0fU)DLnn?h256FQ~T|@vRX59}pi5@%a_;2pI2z&~LO}`pztQ#06lS zY9Jl~<6cL7jgQI>G_DCg8kGI#ET&QlMyclKWs&OGmN`k zi1&y62z^H>_1jW7&^v?VW1wBv5HEuGr|-O^dZMA9o+J5Ku(K0!`tD%*E=;PY3F?&r zyC|pcf>T6%84+NLCgQ{3_+b>{&Jb5Ni1&dwnS%Hu$j{!0--h^^i}+;7KQV|af`1Yb zpA7xJ332+)>s-XGAs!AOz8B*56yj>oUzZVI1%9YS{0hWLGvegBFe$GQr|)L{hWIV0 zR|xdceiVWJl1AJY{HcWaXc*Tzh?C=MQp^#j@4mA~TpIMychXaPa(QYvvyuD`=*KX` zvthp@O-1{{``L z8SzT!_nV0GLBBjhJRI^v8{$ULU+)mN0)Ku&To%T~55z@b{PKgJXk2}U@z@9PWpLc0 zhq!uYjEfh;N4Da}&e^!O!CmmxcY~G{lcX{Le(3zO!jI;xECUA&6H& zyP^@#1wSMpJ{x!%;s$V`Cl_%a@be+WsUG_5T-q;3!On|F{tb+~+lVK_{;LV`gCPF} zaeAED8^#l@m!79cA>IytRzRGd?`R=T&yQ>oKL`Cd5%C&`hbf3(f_U&iJObj}AMvf= zhXsgVfc+^wJ|(Nk5T*W4Meb zpJ9l5!G3%x;=dqHQW2-`1kXUc1oZDkd?PHEBEApevl4L)i2vJ&OM>5;5U2ac*NDr2 zf4(Dr3F4W)Tb=gTOla>w$TyVB16M_S4A^go_&YdHu|%AnFHJ_g0>+US;$;w5a}lqG z_=`o{3G&!##9P5{TM?)4FyD>%b%=+fi0i{~!Vkpzfq(i#+)=+hgX1eX#N8oJDj-h% zrHZ&3*rSDb0r<@X@h;GBgE-AgQxK=`y7xxh6ZTVc5nl&+BL;Cxh}%TOX}@nmoL;xx zkGMRH_oIlPh5T>=apvcm+;SH2A>f~jh;M}BqDI84VEjHoyaW3CCE`bU)pA}V-Us5O z6LET+{T1;l$dkVj_knf^g1t1ZvLK$N5H|ul6%dz%JfMZRA#hW~Hv%7zcq_Em1@UYc zr?U|E1Uo|!9}n@o1abPV<`l$5A>Otj{u1IOAMrlmw_?OQK+k!^kAi$P;!~krj}U(b z`K=xCX%L@Z5nl`WN(kCb`|$+i;Q@$;K)Z$^9uHg}ar*A=afq9O{Z5E40Y7*mt_J-c zf_M?c)gr{hfhQt98}7O_JOc6z&C}GM^I`nZcOg@*1bOTw z($fj`enh+q_9Ft2Kd7E1kmn^4Uj_CkARYsGKpS!LZUa)x5&sVDbwGSA#G5>q1@{jQDkkAA0;nx4u7%rTph<|`QkcxN;H>xXE74%mV z;@^P3MqCc~cf=#X{ytDY_5WvxpMi+ifIn3c*MRY2i1=)X4@<;DNkvSVjJPStdm&EW z**X_-8Sqam;&Kots}ZMvhu(^~C+OLYcsk^%qljm~csz%AKUltrxHTLfHX`l@{%l2j z0*s5#h|}vJyvjxLga0`-kSoW7%WBI5L&ywee%3w{ei zybbJ$Mm!1Hn}E0-T#u#K^Qir1kcV@TyaxD_<_{`w0_{DGR4 z5x)rgJ9|id&7jaF<+s6>6eg59((-=to(^ zD_37L_X0mJLtF}$w<5j>;-M6A1MvS1#Cg5W z;j|!r1nPYYoP-nXLb#iVL%b6|kPxBk`Xdg*mm>?D>H%{&^f;aR(-suzu;d}-aqFFg z^pD_D=DjV54+rj!^guXsf>|7fGzhrR|YV`MB?^&GKX#?`V5a-?h#{9k##tqe<1@iPA^@Ni{ zG#Ex4Yb1XL_C1S}dcT7FN5tO)|Hk4(&nw6ey&%ug z@imNe1XF|&R{*Zg;zZ9z*k2g1II)M^M@Py?#7%&k11F2bpY(T79xP7!i~fFbHj5K^ zN9IFH7~(YE;t-GFiF2|L&j4P8cs}s+h@S%f0P$PEKO;`#P#p3cZErKk4@aE#*Eq!K zeRfX3X?zZY{fawFp5{VQ#X^=m>6bw8|6;`H?@gDpIO*R?*ng$4IMK5S^sGaC4e%1g zY2G-A_(_mI3!E$x`*oQQDOHHi;!@@vM2OS#xX#Qu8n#grc`c{r8g^1(0;X#771 zdlV3F0CmygBI6V966d=mXbbad!WjusD$q0{PL1`vJFPaiWLb2R#uu>329ZVdj`; zIUspAIBxT1aZ+zQ=%0;v9Pl8Pe$wxX%!icah}-cnCkOEi;3rv}>@PL~zsur8|3%<$ zSe(ee0xmLyH$&P>d!r-6PVJM#Yw#xATN%18gOYAC-%1xB}^I0 z;zZ9CkXJ_hEO33qe*rf}{0ne1;Kc2u-&ZmpQruab#2YPfEKcN;VMPk!zkwI9IMHJd z{1S^3J&MrpuYgm3jwCZo`M{EgO9osnslc6Z{ZIU(3M&U7POmczL)-%7ha)~3xDn!> zz)cW$2R;TkX%0a&XH7#~ky%HIKT8kk_mOZNU_MJ8x{52mjwMgpy9nw{Lp%(4I*Swi zbpN}P#fhGLkl%xNHt-@ACwl1qqL{^r9{RiF6Nq02J!e>)=%LqbF0(k%^Bd%=5dR9i zn#GA8`gh4D#OZlKD~l8TDsVpa8gcsf>u-qD^RHisTY(-SMMf!8NWYW*@*oRLkx=AL zxbaDjS?DpbB8wAyW`Z6S#Ag84Mtm-CeZ=YCAI%V_zk42wcns*VLwpuB@c&6ye*V?6CzLiUkdH& zh4_BpeSnkp5=JdeB z5U1DWA`#yX`8Eb|di^a9@trUpX`bU_+A0A)AQLT^C!BpmIbk!VHlod=4z{ zWpQXKm(=#azac&bmQ{#irjQBI<4zWsVu|=^9_9qGI32rW6@3RiSk-P@zdBNiB z^S(Y1lw>uVt0GR%cj*0qR33JV9Csw|2KC0VI2@XA*Ie#_zw3d^!tolZm*`RCYGLl> zN1V1d7;zd0D-gE>J$Wonjt6L=_nnb?*<1&9ikQzv+z=GSAx`6OM-Tie;?#G)5uXP7 zC1D&8dmv?S*Ql{L%yPM94}2!#BsY;l?@J>3!2~WZV9B%j-5xkK4(m|`qbaBRFFVB3 zxwUdQUWilqc|Gu@J@D;_2ZH|NEDj;bT?3fAKZQdjE~hz%*w2<%xg-qDvoyHg8 z?D2jGloHO)hx1tbarStB!s6`lZVE;aJ?!!RgvHt8-4%=@^6c?W&vyxDk9S38nG|9! z+YfX+62|re%^QTX{jif+CWUaeALw&<2xt3Y7BPe=gtPq+&*E%9Tx4;!A0&tZrjQBS z4>l~$_JaqDv;Dx)iHr?p9+AdOqat|IU^*8S5EI4G@%N4L<><_29xJC45irkJhfp2v z1JeHa5J4>T_mk?#v;O|K6g-qqCnAg(K&;c((Py?}Eiz9Y{Li94p5DjeJv_>dh|?FF z|38za57+zO(yRyS{cmCRL-YQ(H2aZu|67{(FuVUPhn{N(P6qFCpMc0bATThDiG_gK z(Lp}Gvu8&HEc|0N^K3@)Kt38HAQ#~~KPcc&7BN5QZ^3yX^8=`IrlaXB!Q}am938)~ z&`{>Z^c)@Lk&6L3%yYuEBcpu%=E8E&{AiuSo8N59?4kLNx|LgYp%(g^lPxf0f>)7Rw z+`z2RVm>4fS}-4W35GwY=>gPbrbLeBKyu7N3ONt@SN$8gYvFn9Fi$49{wWOmSN+?e zeyWGmPuhsJKN7DVn@V_cyMeT&Mb1{pxT*M-R3n^^>}B{W}>(s-nk#%q+L; zfchs;08_x+Z3sHdeEzjPJ?_(jybhap*H86S!uGiT6-lF*LjRt|Tl(Ao<;;qIwci8U zUjZAE_LKbL>d_Tz?-%|G(rN&Cq)3EY12I}GOkU;W<hdGk^ zh;pWo`em38uD_zE`r)9AUPmE1NWt}!=Ue})|Aoi#Zf^@w0{uu!S$2pjgk6Mn&2ps6Bwop<4wB16}#tnGfJ?1w_;dt z58GF>4_29Vw0^2ap4t9i{&NQV)nObH`^mKkx;<@|b`SN_-%(bP#<2?Bo+67Lwtoc` z^g}yVyuEo3_Qx#d)euY?#}s0}BJ-i!Q~TX|sQ(K&uwcq0-lY4RZcl-85A`?01r9}M zKdE1t`OxiY{UJTnZ!gcQf$X2zg>Fxoe-HI@;&~G6`c)W2x2N?l>!JRgQ2#e(J-g8D zDU0o)e#I3$iFeR{r2T5lhi*^nUjy~)(G{SW{+*NF2M)M5`=n{G#^!R*$&%d zyQuwCxE|`q+h2w4>GK1rJl_5`v*EwSzXa@f+4)n3&Skq&1{1teC| z4pruZwH4RMBWIPZ+P6HXLbiTx+Zh#?W~hO$Y#?VFKtcza!gQAUcaqCaY+p0TN+ zp@F`lqJfeANP|(lZ+$(zkw%Q9q3I|xVahC#<+hIF`T73S$j9vT{#)YJPrl{H9*xy$vx2apZy6D!15{UkG~ zr+F*h@K!jaT(nP3ILXL5B|eK1br@lzB5a?+1%cvkD?W%zaJOfsBzbc_*o$j2a60$k zE>K+kYM{7lRiL=egGF;0HRPC^+3FgRvR8CSO-yFI`$2prBc*Gvx3KpIaY=hUht#vp z%udigrQ3qbjCE0R_9<8FQy%lIa!mQmn@Ox#Hp9~~rP<4JrlpsqcV(csf(_RStqP|0 z``pUJB}iN_6}#1tim!eUzwkhivFqUnagXba)cHX1X%||(e*gY$tN+Z?i_r$kHeyq{ zt8Gsu8abA34kxG_F;VZ3Xf&>{N|-$sesr?fHFJEpwk*$uwq#vn0@>(EZ| zW#G(i|243D_59C z<&u5M1s-mVV#K+1kuUqyb7XM*=Ef6QPUsM}PhH6L_&L%Du$D1jh6HQEMPZ&~x4rrm z1Xglf=7xlahs+OBI~mc; z2<8rIZm4l9_IFG*QdMURa!iR*5q3!V)NP6QsxJ&CgE-|14>ALN{WD^_-+f#C-ypay zN~ISQoAw#@pTDQn@u0W@CzXdmD5aA0jAv?sEAhciX1)p5j7X?qkb`-mGjj$K;R_Mw z$g(4&e1lY1%hm@wo*Rk7af7quo*~Q;}L@#b*qHRgXJEnZ$ zb-Sgvr4Nj6`*kkP_UpKVhjHmSrZr6A1OK;S(mIvqD>f>mpP1~%wA3-hUxjJkJQWGDXwoiYpTJNi zq8wFZ?M<0MC%%e2HJR%bhtx$(m@)sk=^2zWU2ucz*UPC~hnjM|yY!?(syicD>_IkO z#~q~l6|hO_*+P@W0ndrOI#tIOze45O!wQ^uX9vkeC?`Y&yytkiA!9O zD2AuTsYp;Bea(^CfMH7{98wo{R&f`;-QIZe1ymAyN~`ZQ+dHw0<)VWhBz+{)}PEK};p{-VpxMZb}kJy7OwuqW9E#;QGS9~pSM z_NgaH_Hj&k0p4KXBOB9Tbw*5lONIFH3ckwjHPN?8s!6#IDU(}0huKSgbopns%$myX zYMGs?ZFj516)c!&r6ZlGQj_dMx-DeD6B2@pnC4s*hAdr4{l_5v3Qw;}UiBn@rBqu^ zvrV}SS_5BP5an*4>MR^+DL(%S`MPDF!tAPNpLaBU8pVV+cjsiU zQRB$h4ErU=6klfW*r+%Mir4emr!a3YXYz-A%2igk@bUr~Va|>zPi;~@*`&}M2s`Mv zOfB5e`CrHhw;zWFa3@E`@|N_hdDC}rCd(v6URh41ncm_S!(qHBU5?3Gx4mzabe-Il zpkojGM;5>B2wc>u#;Hgv=kjq#yd z=WIw?BU@TpRJvwDXq>#btkJ5d8qn;P*6Ljrad5&7HG=||^*OdpLA59J%`=*>hu4l+ zJydSMDW}3-4rAxv+m@DXwB7m05W#CI_io=`*lr%_`78F*=bI^N-@m7?eD*o?YuA{I zoKt&O`y7urru-!Hz}t?hRRwcrtST3m^?7r{In+#SiDP4NZ0HyqZF^cQ?ST- zVzhJ1(!AE4B5K)1n=}%N)?B;fdh=P}KHsX@`vR&0_XT};6)WUc+0U_HXsqXyVVfu0 z{dU(+++J_+ysi9f+oK!OwUwg^Kg$%H9O_Z{*fT?_?0wF@&cy}FW5TVb^sSvW=H1%G zCk)oz9V1kCnyH{DAw;@Xr{K0@wxeHHR(9K?OGfLzKe%aRIqFwY!;)!(Mb2E`Fn!J7 z&+;Wft==&f&0|e00$cg+O5a`k-B3?%pQ6Sl!Htv6rluG@u4FiILx>#H7>^KHdSRpgF?xmn7FMY&nC$=B3sbs;x~R~R6>Qlzb~j?XEZ>vAH&%~?<4$0w!5 zyNPa%4@|lpuPy4w(H3pt929NgdoAh~FP4-N?J!nM2#hPOK^vEQt%d(9e~zV)HgjkEENNjjW~m2v#d0Wne)X8Oy8lJ!RS zn{|8pIU#9-C;<=47YZB!xnIJwOf#zalk__TjvT)+kmD!k-RnrrjcuIGvfhG6+HRcV z6c}`IJXXFK5#LwNTj0phoA>ylMTLHzZF8%Brxt%Hq`dd61v^&rX-(=BCQAa$k{wxE z%1N#5=zYZGrZ`!WW!5bdW!B~Ldt|*B7ZL1m)a0fvhgmaXwaLxH`X(}aV@1M~pEPl% z?Ta1sZshINgyRGr!v#r@%MDq4qo_iazfB&*^CCoE?W#>(MD?WTn zj!3=tr=>S@+l^8$9)IbP@Xn6!Vc7Da@uO^)i&mV@9w%1l`%>^i{~OC5?zlF;-?LMN zz8&A*ueJCwKi5e9>V~PNE0ot}Es<0=mJpDhSvtP2sNN0IigzxPA*xog7dgQ zAYrY)utXu}U7)wkmU)$2Gtc>OjeO05yx5KH4tc4T zdsoR+#a4HDnZC==+1fw1bxFiCLoc6guz1Orx=kUC(v>HWm;LuqRx z6FTLU#cz!ofASTF|8DWq1jm7P4n_m4`W^|?Oq3a4DDGy}ad%GMb&pS0sTQSHsnZ`? zO{}>hn0HyC&(SHtA(p%6_-?H^HnVE3OqqdE`ZKK(`Mb&`{C71=r2K*hmWkX|DiJK1 z7_43B5_|yB2k;o_#%XYImJ=*_) zR+plVi07)I{H4M_2i?`|5}pw}u&vi?iH3e{5)GA43M72Q%o4vWJYx51f!MgjFB59C zyQE)DS1On8=>OF7{4%9-nJ(e?XPcw*f=oOHJe_+%NvV9eyyUn3SN2;~FF2#wHJX`; z%Qw-vxM)p(tGFT)jf*GNxL7R@%udYsvMG&O617ealkv`Tk8PBGnmXPlIc(5^yis3{ zt8}@S>+~7&?E9vg%s5}E#i_c@R;w-Dtin1csdSAh@7L#SWHS@TT0Rz3DzBO+X|`UX$o7??$8nKt&Lg$1 z-t9A9jVK@0C;OS_*_wNCky2)t8v4x@Oxz(~r#)@N9g{qV;Cr%~`H8Dtl#D7Q6O*cx z4>wE+E*yPq^qg${ycNllzpm}mFCt4n@BDg0wJ4c4@8s=6&wa~(8s;bY*cYpt7>mX#jzx3+s`E4;Nn#~2Z0?&Tw%T3u4 zC|{zr`?hdzdBdwx;t{>ay>Q9Mcx74M)b?m6DhwJ%`6x5e6<-|c?0-A!VM zY`?dT_0~?u9lT<-N-y;e_M4r1*E4NKkh{C@hT##aWif6y2DQaCwmuo*JjZif{)Lvf zEq5e~FJFva(DFzl+H}Gtg}RW9Va~q_{fjQWIv-RtMmB!6_mROO&zd%Ws9z*qRCqW% zu_|}eFx?EFliv9U9=EAWw#+z`c{o4+r+liZap>F1@iQ7GUY8HJvhyq7H@-VD<1?p? zAMLCg^~vD+H(9rRA-Wgcw(PII>9IQ}TFzH9@%;JmgW3z*-mfgEwzUJkK>X`>W>@`_Bpv9m#JLh~bf7$YL zen!vROOw5uyFw(#+ze16~r#r&dZNosq$mWg%6u6z3V!v zPSn1x`aJ%%k+x%>+`Vgm%vkcQvhUsSL-k{GJLF>|ROe*3yx0?MC%;nN`*60sT=e@n zuT!^#B)#7@{Z@Kfl)%nW3jgZgZaDu2kAF zr`o*MStFL{({A*B9 zZAPcgvG&4Nf5Ggfhm@2ax8305Wow@QbU1oH)G%wQ!{XJmS8d79 z_1txBdRmt9j*rWNvU-&i*tNS{I^#b;qxjUZRY5@syO*ADsm|TE@J-1&#l|sW)2DWH zW-e>15cP`~JoWgVK1>kTgybFHlXQD2^VJ#j>Eq8`TZ){fFKWq7cp|9l&_^L|VZtbv zg!a;jFXp{Iea>ssF17usg}H)px!%LwHcT#Byh7X6Y=51{x)BwYz+xrcW$K9 zsm-+;wuhXo*rVbV^I}-wjrgw4tryB$f9~1*^XalX8Zl=kS*>@Hir6#ZxtRH=(yfoK zHOz}n@;k1*L?><6O7VHsw0vbE0HOuLUJZ*IhdFCA5{>0$mmg|^{tLm0|Hue z6thdVno5TW`8W(-vqItM^{H}u-3-k{P|POBK|zTa)S%@|DA7zPW7C6!9t=#Rm*1D1;4E-E1zoW`s|b%kZ5&X zH-dBM3EzRK`_F8-7pbPCqOMRh;pQ#J&rv7TDm6yuJ)@FCkJi1w$W>a!N^6gd;RdYIivUT*O@YlqBK4}SEKnckGISo zblUoT$M7*r$K5r|T~Ree-Pk={z$8|Glx@I=y!B;kEBg)TbNyFe_pP2LcV~aISlWBQ zIm2mAcT;sAFIiPMFuCzl$r1Vd*;<*opNtP?f3+B9T`&D@*^N3U=K#UF{>p1rI(J7K z6u+t1cwm^+;ps018_1Mw+I=ijY^z7v=ta`=Oi!KFFn_SfQNeNX{vXTR9gZZv4>{8N z%s;oXby2f2e*zP@L3?{?LkQH>K%sO>B9JrZ$8@YdofIkz+n zqdIJN^lLt5SL>dpC%xuL1OL+t_lHGP1PUFxCuS3LN${aw+D%cu6YaYkQ~ffo>81`i zaUEpW-xgIh8Fn=3g4&Ox z`s;7?&yUGbF)OIiSl%UJr#&cO;_TJZ3-2nJ-?%+}(39yYL#ExAOsgC7YSN(QTTAXc zKJD{3M_V#yVpmdE$NJ}&r&*t9TkCqm=e}98+bA)<%fsg#RJ$X5M@}y0f#Au8jT5|0 z6ZL~UMpW*azV5zt%i2is_5OzSVL6v??+rS%ElNpr{;i5{H`Yd7&AzfG?!?}hfo{SX zSNpWrm${$%epFN7bKX!BtwqnjANg5soTNGT%z5FFF&kbFdf2j5zr1>NNBYU+T?(r^ zglww{^!iE{?6M!Aeq*q@s6$$b##;WwJoQbzMf~cPFLrdiSA1AIQ!(jE-r7esi?=T* zIyZZZLTt+7Rcpt*+>x_1ym}Kp1n)=Ej~Q)VVdS@qvvPK zAMqc0_gd|+vC@%5&7}`CE2d3}ZLtlm>^ScCerVyVk~_AN zmo=ZrZ)wSL^3_}uv*LoQ^!usOk}(5JS5A!ccadGAJY}bzwC#ml&auc~uPxWiLQ}-b zZI|10Z5SJTEclT8DQmySt;a%aoj2CLoe<};W2Z~4#>p?P<}SywLQK8f_Vj%u;xJ?K zD*oXk8(f2%4hITNYDycp`etLWb>2kz;{vY>S}Z)GX0`b7pWE#>%dWaLqGHT&dq4S^ zw@W&HP91u6#AA{5M>WnkX|LJgu_r+3g+aij+=So5y~JOz#?=ur$e8lqxbm#5sHnKe z;g=V^s<=Zk@%lUYb=9d=0eW^Vp1X@S1X?|J&nr4?)ADw9g&615u3m|MZr_8xs*3X! zoPxagM5P`184HYu9TV0~eZ?2(&!;bwU8m0fRO^n2XoudfP&JcIQ^TxleSZJDE<#^T$nk`~>HcSTsH@XjXx&6c{6Fz)cY#jX9rG`&>YrJjn zx51`sHVMBgIUE@K#C)w}n36ePF8{2dNoDG%OMPz7Ha|4HO1OWoduiKTI^{F?O=bu4 z^*hli-L_|1%!em>HA^cjL{x;nIV%{q=!xDo$W>F{Zu8dN`9*26(#_{r4tI^b)v>tY zv_XUa>A7+JZd#nXcslN5*i=uBR9_Hv&uD@Vr?2{>=1Yl6`(>1m zZ&rR~p*&nLI61{#)n2(_esO`JxZF5dql`uVoA?y&3C&fnvOO{7%(Z2i-v(tCj{7YC zd32HgLYH@m+xpCPymTYyMN;_QXT4kpe;ww$JjrwBj}3cc>>hnsP{h}`<$YVfb9qlY zHg$eZKC-`K{Cy`k*QV9w+jVwSFA4EAlMn4}sTzK*#nNuH?2HAmYV%j86_3ttF7kh? z{>jArWw`p2R^xjbZ;lO+-!#~4X6-49HRG-urSnhn;Cw%CaJSG)EWdf%x8$!I6*hbA z@q8%ua*y0)^Bp5Kepw!U8uVLt&Cj*Ac9N5_?3RxE{#tX=o?RoZYPhaS*|cJ4!je?q z8|Gsg&u3m(cjH=}w%)?a##LWJwunvC-Y>hTEytwP+(kP{CwauWMFWge8sAPa+_p{C zI_p$PdxXDEgnFXzuW-{6f%=LA-)3}kGUTEbg+Hr%2Sc5&sP0`6bPkn`7)w7TK< z7bZI$e9*B#Y5;LcJ@SA#>>s-)9d!#nECO9)YwSx8My|I8W+Jt#>^?rnABOP87ID8LwADPh{G4;9vqKO z+~79;iiNE9q|4>%?e}+WJ}9y8V3el$v>AQ&&vaiBm$&EM`=ar4PPtDj$sU+}FmdGG zd8>m(g3LT?KDs|Scz3>?ZoWs6<6Xf#(`#ii{zAF=9LT)B)6d&`%beBr#hyL`Gc#M?o@jtg6vasRd@zN{f7G~?HH=mgF{&3}~y$_6@wJ#fKcc*@H zv`)^4OGBQ!hQxTOG_5k5_TzrMsz~_mLDsX%WqN&FyfM2)zH-Md%U){|4j7H|klhp2 z_4?(Hkg=JQcJ-@$87H+ub>V<`jWxHPW|Ym15-iS&<69+XI{jP6@_x=Mriwh+77$WA z?R>b#>HC23`7z4?-FA2c{0+6^`0}!1dOZlL3dj zb~xMjQn0+IdDB=*v+rk)gwV;e2kSmQjm!F>S}$IG{pP*Zdv6Y!H)Z>7{=Uw8tTxR_ z(XGGp#!cAAKXpgN${U}yIzD(}klH%@RjTBu#QPiOJ+5k3Yn(J!GjCS($rBp}4Se#r z>Qnrlj-thdiR&ty!){vcFRFirMax+C_>v zFE)q#+OAvwG$f_}xQl~0UXKLqJD)$(X*8Aa$*}k#?DIBo|22iqA3Af_91R~yjoh@_>&D;Q;kcKOcq;wOWwI%zPaha2&)diu-Bc}*E`=owKYP5 z|G9fvR7s3n+g5NufY}C|`4j)5ReE-?w(0TVeU><@h@uS zyuw+$q{tD?e|@(<@rKviF@U+Jyto9_VnmZQIY{iCxCf5$)!j z;n96AR6X8t$oBRegMDjPNvMj>mY$f{v2041`Z2YVA70v@EeGw?-?r=HC;qvamo$BH z?Os-Hv>IdCwDa!Z-rD|kvzEV|F>Al{>Mj#Stu-fhO>3!`)}V293^G9uYCNArQrbnvBR>N_oG*bVuqRQu?8 zr{agAa}iBeb)yu^L}O#G+1TB6S@z<3vagW%)L#PL%A7L|wWmC)KP2avZR#Z*cqQ!F ze5r&H$1j>ZoF4Pkm+!IWSLIikY3CliYwcyD_~L8s*U=&ejrj7zFBi^anwi`AG9vogn71ryKr8}w1Ie(Y_>6>ma6lH53>Q@Q#8_ts(j8{#7{K?s_9S(XTV(A5~d^bE|s`Bz>uGdvQGo5_ow3(Hd zYoXtXBWj;Z;igi)Zoe%{rB|W7yX-uNUS?EuRrrW9B84eO39zTI;D}hwZs~ zcg^}V>x;{cih5^Et?BF0)*-#ZpgyFR<+G0yhW(PB8Tj(zC99HSKctt9U7fppb*@ik zby!At_;BICU%x5}nufQxRzCVL_qgqk$!@LU58}flh9(vs+88O-nK(5-N-(YRmYQV2 z>dy`@_NJcv-3raWg4)w0D2?@Qjz_y2jxT%`Z5vdiYg1 zvUeok_c)z(&Z3e|H*RI`nr2?Tp!tki>9UdDk%yLz6{|T}7AG$}B2eG;mVN%k0^=Pf zN<$^3w=8Zc6I}ds-AIQ$UpPL!UPb9e%S#oj46jXaylyYm{>>z*c52FI@v$mnvIH|G z9y*)*)Anh$wY+Rrbik?UN3e_%h?rHA@U*PV#AZc<%lqvR##dG(TJs;GPG)U&(vB!@l`GG9HV zPBYl?W&f1V{3p_FlXX;F-QqtB%_;lxc=4z08y1Qz=o&P0!(f5(WMyTa4MF{l=1grr zvcnUElJYz$SinX*K%6>M&SF7!8Dv$ zxx04m;@h}C-NrjuL4r>&%RISRL|IU*+A#WvnrT>qg74<~c@7J@@|`z56pGojxKHKp z=a)s5ybc=b+~S=2{bP-N_I2GCGQ!b|7ytOS$+i7=-pSJs+C1+s>}3_ZSnBiU2XURF zFDtCmiW7dERJ&1eL3N=^L6VYTsrS}3!3Ty`ri(uvy{>SWlQ~!5ZN`lu>`=!T~la^c$zCNVpaZ|;~w$nEq+`cEcZ<}c&*M7S2<2LhWS8pw_T)S<; zWc%KBsea+AA#=(P8x88Onbo+%eUs+2>}8#8i(^`JfDjNeuCU)p+awy(+1sM#9^?>bWGVrUfQL{e(=BD`yS+jTd_`+mY$?*jQxNXd6_#!Z7(ZA#OEo$=kabwi$sob>AKBPTIT-qTK&G3jVsEZU1)ltRlV|nO!8PA z<6ouc%zp^%Y~+jYzOFF&uj{P`r@o$4sVCaew>0pG(+`U?6Sk^(&rRP@HL0qqc5`5M zFRQIdB|eKz-QSk_JZ1Fp;m7V9tWPhQv%5`eWZ6oUtNaQ+*7ns)YRr;m1!(k@*qYUz zQ)eP=7SLa!zIluLm=D9242ivSe(*>BSHt-Xhctg!Ja5DIva6-4xjNBr`(M?Px4I!H z`mEGR@a=-)yOxJPrnkST>DaONx9npDakDiMZ%Z7tz8M@4+Iw||aj4FPPwQ-~$BwMD z63sD4=}bPRXgbq!S42x`lGWwWZ&sd|x^PpL%V^z(GKq{|v85&7I$y7uVl~@KYt1{k zFKZue8kd+^a_H{l6RN_8(jqrG3aVCq+uaoUdi$Y7$8z1>)x#6K+6>JG2pybv`*E4g z$ogkZ`@=enjfQgUtgX!y)&0We`}=AKhD8JgCR=>knl!E*pb-=ZoA-kl8vpz zB-Zm=iOH@Luo6>T+sjf+J*l^)m|lV)f0Kxqx{a73vyPFm5)-!T8^;h!F-2QStom|H z7!~uF4=FD}-#UIqVyytPZBj2AG4+Jrc48I+ZhXwfBbZKGQUFP6mqxx?;a>( z+ZkY^$kawm!b?O+Zn&F7qIR21JmjxK^x)+C&mwxdh!F=yr)8{=*4;e-w&W! zf&$E%e-(kh?{wpe33T6?@#j_o^hUTpF>}v_C?dQ?fxqsUqg=cPc@k-V7Xc2rTk`KB zz`cv{UmX5u!CJ+;g-eb_c(aP!$={q{0q$LTT!BBn1iDA&ADEGci!6U)hASb$TNL=~ zjy}p&dyrTAi}NI9|LQT`z0&l~oc}h{<<83fb?+h>6MsGhjSD!uTetu2FWwI;#s7*R z@ib4?L>}zPAT{qz8VQ;ZiK?xjB(x4L!(r^uRy&!1vILUMW(tn=e zQmi%He9YKM?tT52{F@&5j~=)rli&ZPN3RD??)m(eyj2gJ*!wT}SqztA{?P)rjB?1k zbN(ft(F5Pd)kF8{91eFs!6>ce^8fM@cVEEpuRZADaG!+|7&0%w2kwOMVV?LhFF?nS zqvL38M7|8j7xS1EM{7OuWk|k^$=68oHL52KjK0vYg#jENJIBdZmX1D?$B%chb@g$z zv~sld;RO3e2J>q2nHLr|H#(e6U`z5m7S?QJ$l?GFVg3R00-^$XtT$wSB=gh{wmu+a zTh^>!M1b%AMGctsw}1S?qM65z{LQ(3^9c-@@9)FAiHEuGWxjt{DA#7*tPf*3_n{@k zl=;jO&3de_e_zClr z6Cc0ex!h;f^k@gYuZ8Ebg}(EmnWwHW4^Cn3bMavgFGGF*@DR_uxdE|kw-Eyw^oMQ4 zaPSPH%r7*YXF#a$eBYpe2%qriC?E2uoB3gjIDruXJ$DkoxFChtL-x+JUPs{A zem53ZX33LzQm{QEd6>(|F*?<61)S8&Zm$cAlV@^Jc~2H+`(Y)Ev+chHoY+I|PojGG zSnXxk%k~4iUKy4=yWYtxKd}4VkHyKc5UrQ_Gc~u6_Ok7~!jdPkMCBi|INSa{JT-s$ znYo9FTd@D(_7SeU5+gzN!!784>9OFc`AgoF#mTW8)f2+vL?`8RKSS)%V)4x^d2$OX zl`m#-wm-#Lej`W4RGu8SkV5q9FdxdvGgC+*oE%$HPUDSmwjU-j%cKyl&wQx-egc_7 zINP3jF6E7vLCidrXOAxf7XJwHM2{hhlUu4uA)FlRQax<@$z6(+Ph!cFF-3VYa1vMK zoPzQ!mOR@(r7TX4^{D)37B^w>K}0cAh@EUspAAen+t0)XQV3`Jjl=*cgqtxRS}$oM zDTJFdAIc{X$P}!{pNIdFr$Sf!*j+5ccdTOUSOAniGXYsKt z`NJ$uY@>R}H40M5kvRMKjvj#%PLAT}vomRX2_L|GXdyX`6v|1Al0v71lWm;|WD4DX z5#FCmcwb~}l0rDCcN26hiB(c4|1HK_rsIopCujuq8|C)UP#@lA-Fo&3@&MgGQu!p3 zESWNwH|dtQAUhML&^$xscf+NHc<$Q&>|7wk1FI2FB^Sn+vX(dL)}I0O?m)bkC=cW! zt_;gGZ%}*8`|wtrK=QG~sZ6QlO}gvtfDWuiyb?P35#rxrxfSuBWM{>ckGx5@{sssf zx*w!={sRB^gFHaFIqX;lAwB`xH5~Co2sHYGEUG6J{AP*dbzzz2H!9x+@ir65D~t0Q zGzW1Rh&Os2jp|7QKO`Y}A;@dwJd_kFPv7}N&O_Neo443~y^YGdkQt_wBl$i8JbnZ5 zE8w37#20~o$TOcvq52yj0enK-8ua&qc2Ifx4r(dH10b#x5nls(br6^A%d>+%OPA`u z2!5V`@4ljT#z8)mf&QWVCfKin_;APr7KrDhVm%TzXIv8hxV>VyqavtluX3egM1<4wh*_ah<|{7xq!F}Pd%p^@z

tY!{|Rvm$PWUrAE50ef5RX}0`VJ=SBD_}73|bRd^-4Z6yn9O^Rz*nz5{j&;#CkQ z-iTj-IGKyM2=ExhSAc&K5pRI~-6q6`((!^geK+d?#N{E*Pa!@O`u8&8^qn)ch~I*7 z*NnI-wCgqE2H?-{h~I?%?gRR0|2~EO8i@EVuwM`H0;tyv@d;3$4dPcJ{wE=x41T8X zaH96`o^8b81S9zZXje4iM2cv3#HHytMSLpsw=ncGZPz2Pe*of7VgIU(IL(K; zh|~Nz8gZH@9T2|`<9Zt6#gNDR5ch|84o94xA1*~)7y369aeK(O8HnG3^NYQR)BIbC zcpmiE1;oSX`2ym)z#k&62jjOD@s$v_pAe_-q8ETXLHltR*e`*&1mvF~h~I<$)kK`W zn{E{1^5B0P#3Nu_OhH^2;?EoLTxic+#Fqn)L3}gtbi}hEA7&x$0{LVY;yw^pg@}j1 z@!Mgr5dRACV1Rf8l{@#B)IZRm2B@J+}})1LNo};sxNh2E>O!etUwr3q3wWJOtwME8^E+{~`$E zfVOuz)Y~6%322uh;`CkQ+KAt!$Bl@efjG27d==Q^hPWo(?;w63>|B6&DO~Sbj`&1~ z&-I8?J@nmsWHo8(H?VULlJ^6D9!1<3+ItT1uh75u5vRxb&k?8Ri60QB=b^t5r{|lZ za6Cd*6Z`4;sVd^haG*8J^oWgybSWEA>zv5H%r9%z(11_p8$!-J1O2gxi-7*s zh<}9kZ$&&3`h7RzS)l(Y;`Dd1=MbmICpQrn1OGH4-WSGuE8=CafBB5~YT&&gKhggB z33;+V;*ViJqJ%ijhq{P|!Z;d(_>>{c;AYB1#O1)B|A)Lc53A{WAHPon8IlG=2x%as zfzp69Xp~AL4Jyq^rFlYxq!5{fN@&m^Ng|3Q^B5(H43W9WPhJMe(^*3qWPEy_UwoF zH?YHC#Lb}pISTRJP;L|u?+3}gZNp1E`M?!w}gK|OhOCIuN z7~+Z0F3BOT2=#d?;;Y~~tBd#(XqPMyr|&_X5zmKm=8t$2=#N62*J^M$>kxMUJ8wfg z96V$pPVfJV5zmDBcouOvs4v$LKM3XRA>zu=4!lC#6?hlot08{{A%AFo(eVf3h&x01 zlt%m#*i#AdF6f`nL_8AO2Q$P=K>q^7hd{i&5l@4Br}s%T{%&BO)kyvZ$fqKn0r#Q% z5q}1Hjv}50<-P*(iO@f+Lp&Fb?<1ZA<>xu#gTbC1h@S_$(ZMJ*y{DjFi$HmyJO=D9 zg}65GNr=yb>y9SkTj1CT@jS4nE#h9l7bETqb_+)QEW~F8;=drhDTo__{ym8A0{iD6 zUJC6=8R9CC-WtRmp+4S0+zRwRLHq^e(|g1%A^tsxAEeI>p}nE`)d+Hui1!2e@rY|e zyRC+Jf3T-M;&hE%`aYb7`xWfqiR52EdP5N(1lO;Xh|~N16vXF1`P_{-eg1P8ar!*! zIO6oVXBFaKAiu67PM=>rM4UdqYC~Ka+L_OYpMiFO55l2#>xBM*2;$Lj-5-T`3g}lr zdeoUWPdjreS6XO<#94D?4Mo(Sb|BjT+P|C5Mcg6qoz#C5=q zuMl4i<*FO;b6`(-D8JOs^PzpVMcfCPH!IU9J{p zEHUDmz%^N%=(dOW8?ZPDHvr^~5%&ac&f-K5ZTB2loaosF@=l1S0(WC^qKCG7OIV!f zsRj8U#Lof`V{xK~wtFjCoapHQ`2@tMGVtYyPX@jLaYNvn5H|px#^NNN{Ll~I$KoWO_8^~$xHa&@EKc;$@=V9^kvt)K zf|pFLU5KS@J}WE*xjFIMK5bA;?chTo1Szi<9`!_%B49erM$ioYZSlA7?{5yd24El87-b z5%K$QU$6z~u?9T{5HI4X;T%Q06*zrRCWj^R+V*C)7i8#H!pF(^Lv{$zgr}GIs2Tse$ zM(7Z{XUW4YC^y_eP;qE^_JndGhPX5EVJuF{5ABahvp6ZwbUe2l;u}EEL>4D{==EzF zixWL(L0$uKI{slMixWMxoEWh<(bEii%n+yVTP<0f=m{Wyj9bX!M9(+SK0 z{x&GoFIb%DUkQ3%BfcE?2c&-|QO>v?#82`tX9#qVXnf?MUQc0h+BOg`vss)(kzQXM z5T673gAiW;JPz@t!1p4)9Qbj>Y5P_UoVE{v@I2%yOCC(a)zr?CC-&S5C*LE!33xZ+ z1;BqJPRIQVO7kL2<3syhA}mhAtph#ch}Qrg4xD5MLGXBnBZv3|<{WWTrMa6w->-dw z=WH{96FpF+xq?o>X*>A@!gWRb5pYiyC*_m2lYuNw($x*}A&ArO5F&xo_|Vr58(H#1 zKOIN4lO+$A3g#?_QwE&IpMIZEfw(vn-b;wP1HX#69q?N$&Q9+`7ANV=2Ki>hcL9IK z;zSQ^uimjZ(enc2KOx=(yo<$&o*~q51>D*oUOo z5FXy#X34{)h^y!$OP<)l2-5o*aUI}%GF)$e>>mca58?s92e3Ga53NT-Se%5L1@e-J z?*TrV#fctTt|qWJ(Q^mnCnH`5T!qDn9(m|jX|p)d(-%6YI*9WFH(+t1hnA~(EKc+& zfc$*KrGVQ5=VN3^zKmyn#04U*&BL6nh~EKTjJOR^$hbPheSp72{3`IDh_?cll;!E8 z<%%B)krd)3&~T4KoW7@!N4%9NWZWdg=Rc;XJehxtILhhsA|u4<^CCK52bJgfX8MUp9=a2pR2FAHFCz1X5l8f}IsM)l zbM|^#Z25Qru9`o3%z1~L{k?Gd-6z#=2lDTbd=~KEh*SAd(D|TxsJuGjMWDwBaazv2 zSR9%#u8*`{_&LOtL4OP4bAW$G+y}T6^lM1`Nw}2j^}-!{;mZ)G{quOl&qKIrEDpQe zGljkI%ZSr<;VFwl61jSMdf{WC-$Bv~MUN}LxEFq?7yc4)Bd!)^E)>W-sy`RFJmPi0 zjaeLu823zgFZ^IH{BAE?2(CXQK1BaD2zMgl_C$bjkSuQ7Fqnt9|M2*mTff+LlLi^8 zpL(Isp$TW_i!MtZvWQEpSR9hVC2lOvuJ6Gt&ThxzS)APtr?NQvIzqdfB%UPPwID~^ zEy78-w7#>`MYuo6>%o@DlW=Kyn9t&DyE(HsJKud+oNc#dEY7xDEQ_=4wvNTwcH6?@ zY`g7cakkwKu{hgqGQf!rQWx#vcs$};;dm0_PvBSyaaCeK#!cmIxaE-Or|*fTBYqO( zbrGlUi3|~!g-kF*d=_v^#Od|V7I90EUx+xpE-pfR5y*QW?ho7-@ets&KSS*t13U!D z)A>yz5#I#zafp-oJ&9Yz+x)R}0dQJ=Xt<@olac&I;Iw{G`3B(Ik$fBQJ&1P#KY%!y zKa{v^-sVp{X}>fV@xf4;3K5q9ejITX;N^&G0)uPk7NU)T&@K| zgtO~4eSb(eyIyZ)$+PSA8c<5)+4cH4C?%Za0d+2*lyG*vZeVeCy_N=#BzShZXnRW- zJ6%Uv^6Yd~u{b+jPgtCtE=3YS#u5GObj7haJ6)wL&Q2Gd?}+GOr;DQ$8oiX6DV5oV zh48k%bT5pVZh^6JPf1u>bPS%&+_3q&7If&x1@_`(Fs78CL%bAOA6v>VM(q-;=7+ zBzc5}5p(dY!yKqGQw4LA$DPEMnS|Fnh{96e5&t?J5aba+=5+lRbZ4fh z^hPpLxO^tDTJdTFfU(M?-2K;UZh_9e|4i61*FRKMT&IMF+GIpMGv`iToT5v`{bl9 z`TL8*{wFXpq|5og>tB`e`#;6s2?Z#M;-ttNllDm$abLL6?Fas;9CVMlbE^_BPR#Fb z;b(w9I=+Rf$KjLy(ci)kh6g^rtnf)5;qce?5`HQel=fqL3qPJY@wf0BX7GH#urco3 zsw8Y2KDmGWYx*I44W61m|2X}5%$dK1{}IAh0iVSFq@Bm%UuPHzpY{uxU9M|@@Jab# zJL(_I%6ZD1{9F9#{9W|D3)O++|A;y9xA;rX{UXH$3=6YiOaQw-9i+_v1=q#QOcKIWIas0{nzrTfl3oha@ zG+@}`@Qr&3KT3z^AqxBv`-w9@96sq+{VjYe5R3-@bc4f>?j`(>e+i#F=f&Zd^b-Cp z5Uij9!xo2M&`bEfvw0p&z#mCJc@~JnZ-(%RZP+dqI!m;i(R0*4-M;C?e=+#S@ukOf z`wRU4i5LV_1^)d}14P56@%i3M`imj`b-FxBlKzp*kNT(S8o+eu7#07VAp49{8P410 zaZmF0$v^c^$>?7EKY;+~b44nT{Yx_b|5ksZ;(0!3JtXlbuNtU-8vm)igs&;Z^KZu6 z{Q0N;DVWkr_^-%?fpH&s8*ckeo?TG?G< z(m#&*QU5f2(mwqy{ErZR7PR9uJnEkwll!E<*Yib1kCK*niNLEhn?@F^zu=Uo4Ao^Bea@b(lafGzb$x5?li ziE;So0~Bu7U>@1r5wft zOw7t5a-oc1`AeMvL7S%K0^g981+ zf&{BB)vA_eF$cfR14=-t3aH}nXIh}8$ng5b2TM>FiAQOC|SZ} z?VT`Dla2Hgk0&PtFhy}~)!?<>Oo0I1yi=-o{zRqrPiVZ@nkLo$7c!6l&|;c&38F%E z)W9*5^m$3&O?XqB`E$%Sa_b(GRMjdydzx^br}-eIXE)zG=_&Y6de1~P=@Mh7gc_B9 zg<UiN=Z=>LR0>26TZj^58Y;jB(AA+L#DZ-a>aiz)nIu-6IA%n-plMPRXiF5Mlc*gitjd}7xa%h?mC#T5?TYQ#OP(}%8L`E}D zYBeF*3sHEj|Lgbf!DH7D^&FF=c1R7Ae@&UiZ)ENBY*nqVzRoP=GwWkBaQy)J z#J-t^tC>l|GhF=!#%yrB@l_z|;*Ozd zRjEP>JE}Scyzn@vB7X0D)KsGpNv96VSs2~yY)>c`%H60IueM%ojhcYp_&Z0XXBT8E z8C>5dDmCO@acF+W%LOANUcAqKthc;xWXTAx?}MLPEVFiRv$(D^{PTqyKUJz2p}9&? z3MUE|Jqp-fF*!Z-a`PUUso@vR!bWc>Dt@FsuD`^k5Kp(ZRT($3?^S-PO>OBp{CW4I zluu#f)g>)`LgtzWnl7+V*cP|H>0w%B-0I4j4-13Oy|`a{CuaAUxv~34ys|#v<8A(4 z`9%8WwLO->v8J`ZzxGdEw`F9uSazdHXzELujWX8_M&8|gLbJSjW%-)Za~6%BP@y*I zuC?N`H}dc2t#lpK6LP#}>)Z0jfmE3<8UFKX&_`>0BdwiT025nQgWDxNd$$|}dj zL$q^}l^0BJ8L+wU%aWG@y0exIx}|H&xn8p%W4ol#QTbYdr8R;X`*QfFi%YC3s7Z_$ zSv<8kZ5Mx-xXr4^bJUA>e(P6!pnQy``6JF}Q4K@EBkT9{IGnk8cZtAB&W?HWo_S|i z)NlVXE$>HEwO*0e`7-g2%cJE&f^7ZwX>2_uarx0g<-}Cq6W1-rkKD-LdH34AE}saF z@Y3TFZgIN~2#B?gh_7C}qW^P=bNrH8FCSF7Wl6Y6?rz~+Sl_~*GV%C8NrSh2CEZ^3 zdE&^-EOYRBkJv8rFh1c~ojy^D6R8!_ZDXz>XY<7I`E<>rh=&Psii52muSlu0@ zqJLd4Ud&%aAjK+F&~440gB;m=QUa2hhlQ8}4L+kBEfGefKO^#skJ&kQp(FNUKXo4t zb7JqX%$K8QO3B{)%vU7V#{-vJK1V7?en7l-mgugzL-@YT;rEs9%Xhs@ zW0|pr!nA_hJ<6koXbhV+O|$;ntfQ}@HqUxhxmnOD?W@j;CnaC~uTEHf?xT~Dj^rj` zO|$jyWS_0Rl%h0X-?6%-k+or?*Ix^^?w;d$IL%Q==6L!&ZPDHDvZrsXduP!eFZDh9 z?v%ZK@8_9}_BT6tZ*-sE9rjrv^+!TJbQu*`UBBG&B-bo__{!~e2lSNH^P;T^+K*f9 z^VfN)>0o!@LEA=$B1hYji=#62PG~BHU(bu}h?#jJZ-?#@ySbUu>!;2%INiR;_n^O4 zsa<`)@Em*H8Furmj`+)$I@LQ%PixBe*L%F8e$uOA|L|io=WlL*x~rybVtre(+o5n^6u*`&r=Q=Rd?`g&2rs(d+DaN2Y*Z&Vz+#BvX%Z9N!z?1eQgW<6DQfWT#LSZ z+BSDgVxFDl6NUK)f0T8WevLbkw~>jegqp*N5yuN1mKQBpA+RH-?D>b(4}}lg&8~Xa z4=#RFmbZWP{k;9_9@@^X?e{KE#JAFaRO$Q^`>vL4*c*_z;q=>x`kUn``Q;}f^7gNd z$Spth-hNr+rUq5+Ed-(oIw#HFMttwX)EzH;6P@!F~>bof~vCXA^@63;R z&H7jS)-TJMAD(6274KbddhbL|l$@{KzVm@+?7vu+2H5OdH^y$CyL0}Hj-Jf+7h(&) zh*(PJ>C^_AZ{Ah1DrK9qTDM-)mP1l)tKSYAm4C*6=uP3C?isP_Lyx6@SN)w+p*6nE z?rPig&3(3pTE72Ovin=UPwJ58Qpb)DHr>*5aMjX~bDd>F>e76!%q|>I5HRtEwpg{w zvZ4j|g{NQ%&Vms4Uhj!Uc6CRmXS_q~&w>4~#{9M}3d}k){`Z8S*D{koII9%6ElX2B<65nsUvj{~(c|u{ zIa`;UKALP+`>gx5<&Fbe5`)iJjxO;zU47MIvs8IZj>FH_$@vCdpTn;8Q;YW7=i)x} z``V5j-QybFi@uJZUTpAa`kRo0W(V7v3$|SA`7-(D%!42A?2f(r_QA#C)^^{RQs@1~$FCl4$NGk%g;YVp2nk+=PpZ8y(fJ>&U9c0-}& zs?$n^V?K6H2ukzv7!)*6aqO$^%N55Dy>ihhIck4%$-!YW&v%qME^MnvdzO9W#qba2 zB}HSTGhe^hlv?cSP|e2-ICJpOYpmIC*H|Ma`;tedRQ zs|w%OjtZ9ZzNR1diImtN zc6oc$FiRPS&B8gOIsEHi4wHOnFw1yK_YIDHX>rM+>eR^@rw%FqnD4pyjDUN&fYZ+b zV{dgGuN|~EqCufnbg`^Vf2p16}IX}H3TPywU?^Sb8kO2 zAScY%VPVCIzFXyG&Q5(kX<4EF>lUX0pVx;J* zV(sxko;G6>rgwz~EwqT7b}e_XY=S{?shV@u1*3P%o&>z^{8;hb;%l_NMD855ktxl0 z*HjD}IqO^faKmjn(>xrALP=HF`feHOQ6q4%#y=&ceJia!2_rhV**4^;=emO3ok3w+5p)EsYZq-%n z+O8_OH^RP_Gwl8;^AvZ*$-8GPh`(pF)tD%X5>Q>awO=%b;&NoTywor_((la9?=~&rkszw<>oG+52d&mO~ zUb&!;BzPVCgXn<)kMag{_kqV3e_oIuOJoUw7rWEU*c&t)!|LE+``}8 znyrJMSvQN;SUwZKtg*NpC30@a)OG-H^cKs#?HFr0XyTUJ)_dsE@%pA(iFdV z$+_51DtYS#4sXZ_e+`DSi9>$E?R3lg7RG>`y6fk%>Q> zuCsj4{I!>)52ooEPO(gw^<%i$_m_IR?KNa}ojZ{tGv~#@9J?Za3D@Vx#_~T-){Agl zTm9zs$=Wr>Prsy|JMHj~Lu^ai`gehR1P*Wy$pg-+n1R{&l+WnT__i!-MO;7F6%AZIl||^P=&) z@jHRBab10@ALeIVRJ?QNH5}TV-W4tb`8gta;!N3Pgl;ei4E3^q7xa`-M;{1To zakKKvwudbqASaV`Z0q?4`3nxzomn(Oxpi()nipU5m4l)4Hkp)|w)?mYE?&QKgzSZ` z$Au9N8s~nm`11awYxB>J&G}1q%|Gy~xG=*0=_C>R-^;9@9&PGs{}B~lxngDb=*6AJ z8`U2qzB+Msg?@ax(12ipb@w+vFmZXlH|pMuWG|sNh27Izueukj8@|`}{Uux;d9GaS z;TWkt6B`x3q&+=wMCyW5NU7Y@f}kJYv)5H=@o8^6w)(uwjG`@RF(1D@6xlb_>v5!y zV(`}LMyndT$)D8PCLMhK^rek@%aqBdb{D)nGsGl7G(^9z^^qsC?k#yb>CW5i$1T+T z99P+0e6%JraP;DoZ~HuLyX}8V#zr153hDMc-u!q)Ma$B(*n$CKzk9?)rqzh|IBgiO zfBwe8NYqRAKgpH5Ph?UupaCZb-p(q9|qnjiU9er=J^ z{^@ZIfjQ1AUZzWLTPOZKpnCBauQKOr7bQPE@3^HgM=j;YjRmR3ZB?4~voCpW^$WRP zf91Zed`MkeU|#o&R`2m2b{^l`Ug)^=#m|u%XD?3)`TFtL!*~zx&}IGZR2+W(RmA?r z@WIpMmMCfa40<84$!pZuVm-rC)2I6uJE<7Ey&UkdL%~8~V%{R-Ahj%6CyA`Ooq`G# z=Hmq(zV1-Z(P@s0ds4O1nWGqg*WKJWMD@V97~?s?Th2DMi?1p=V6p6)`M$9&8>$zL zwA|vgZiT*BU5!$$h@Na-o!0OD3mn5M10FSANgFfMSw%g=;ehYLH?|k5lPV{mr zwo0=dGW`q1AI!4@ zYx}htlpl~&*jCszJKR7w)$Q&UsjBlszHA$sX5V)HhlNyQ=JAZ^zA`uM$7ZF5%-%ID zcdL-fnbh~e{j_&1a@nrjG|M(mA|YsIeq4NX~*5V0-ZaRBA$j*HjlHotlY3IHgB}|(1#mM$6xZblOK1nBI7~( z{5x8wN;hkFJ-K!CpqXHa>}=1EP4n!opL40pQq|6VY<~N$i{--HEkkwZR+LSw%RI34 zd-k%B(T1xhP2~%G6uWiysjVSH-DNlCx$PEkyck_ML5VZ9v|TW!B5}2+-FE%0kBxhB zCoDhRUA)M9b;7gjx@%?6XziEjbeg*V_S6Ww=8T8e${GZWMVdFg_Z%Q=EtB_pUt5X2 zrB!vx%5R37*Z-V*FH^FmpNgngeb~N0;q9-69GE{*+@(0vL`2Tzd6_xC>h_h>)z5^* zJ{Z5TIx_g1YQp3bN|QQ-cUjA3hPSQWHTKG_q~!y?P1taK{TksZiuUb$Eld}->W}vx zAK7(boZbq_c-8kRMUy|RDV`^^MQmM2PNk5`#&j!f^LeATO;7FrG^4-jDbElWPo=c= zE}crt&sdKAZ9ewg;ms{;gjMEea71NZ&eMsO$vo%XwAZrpOF_frru2CdmW$61pS!cs zds_Fptp@%k0`-m>bs>jyzf2F+bI*TlEjL;>Q>H|2xKh?m@6nBGmt7tZE7vbGdB~N0 z*V~f5hmN;dq&RV*I^-N5QfOOu zG->^VS1;FYi*0!P`1-*~Az4fXT1vQZg*>$?24@X62mYq&>D8!4=l;?Y0f zTXA%0k=o{*VP{g_RXE6PH_A^kUzE9jz=+_O{9mHtb+bDb)tVkRk7-!aZ=LSD154tD zj4j9uH?|X)xqD>IXYXh4S_>;AWlF>qMY8%kmalfvo1QUZ+QvCj^4^}oA{VCQZkV`F z(nmM5k4tpKq$^AANn2Ql)Z7#vmgFS->7g#iP_!dz<7!Ls!>Qf(oFq#N+oVzoHosn) zZep~^-P)~d;_YjWW)ZsauX7_OoIPJUz-Cv8T$;pf1IL?ZE-cU2Jpa#%)kq=Ki{@kQpsVbkWen@1Ndy7NXfc=&3GJ{s8$HFMId z-l|P%IJq<|+u^dgwEqs~VM{>azbtR$}ycFnNw>&(F`?KPYNXUKkk@gVP-Mx%b#(1$sPBPAcr6}j>) zV8Q&pe(}-|R6Dh{DXok27m%2K$84m2^phvIH&0!Vx?_=L%>DdF9ttME`?w1jFz>`@ zKaY-)rpHm~1zR2mUXT`E+pVAU;EVC;U0svJ`1Vxvk#LugkNu&gHoS88J=4=qr4(-@ zdRdyBJuP{y;IW?jff0c%4{Vz^yA@;)DZjeHTzcJy>#5hH3L@@E2A7Er+FjlD`P6xI8 z;*YDG^=kgNdOxk)FX0PLzxJDdbN|J0rm-#?JBO>6r0jRN_)5cFby?h*>g+s$oS3~| z>bA^H8L#~2+0xnmZzeQt@L%(O@alE-nk_psWhzu-w-~;-T{qKMLce6_zJBsA0?o(0 zsuZ21d)>!X#U$bLiEh3RGuk{k#s&u3(hAHwGEes@%=<4NFKHhS4{23(B~>MrDN`rH z9=E&9eSa&>?2?-iX80m=Tf4~d@z9>#HZCnIL2#qRZbcVqz|0Wg)V_QpC+v z#8*!w*ny!*14K0Eh^Xj^C@}sQ|BN1ek$&^oOb57u2qqEtZ{S^+IlQ+`;=>+IRx9F6>pP2#w zIl&vU!_Gn8Io=o}!T<2V8)_rSk^V;ocaN8sf`7hgrKiVncLe`@LrwX3;HnhBb~1O% zz198K8Qv>?K@R<8V9t(1IBklz}3-KkuixFQ2d=BCf+~Yof*5Ib6<3OJK(1b!} zyNdaVv&t=f6mfvzqAWSOd!iS9t`~l#7k;-F{=66dz86j!^uNWkZ!erYC;FSbTrYfL zFML`rT)P*(xECJQ3nyb7{}#{WUO4BUK_wm>C36E6^2ZG5H#e9{{-%*XHS(uU{+NOB z<_22iPn9RgJ+I1@SLJ$88@h7*qsVA@W{3sZ3tH^&)5x*TcNRCytD3 zByJ+}lVyICk0X$AM4q(Cl#_8##9?_29_Gr6F#A+qkHyJVgK|^gM2{ScTeG-4i@Wr~ zJ%N*Wl5$G*^HR-=Cu!R$U&)duS1Zcb11Ire%kN{!lWP!_Kgg12+wCYzo}J#4EO~Z% z&$8s%`Y*BM+4^s>=$&+h5P9==gWyf*TkLuaN zlAp%n^aTnTme-g0QOE1Y^1d4(=bnJQNF5=Luz}n?-XA>v|HDZeO&o2Xi9895a$bLx zr>8#=1V{3MI3iDedf-8aK*mu%6C!KI!+-c7=<3c#d>KTXw!>7?#To5s_PG!l%<{&4!O))U0*!TxU%9}emIhPW|=D+uL<#>XAT>%%fXJv5qA{gov22femi~>@wH%|$B4^=eaJT}#L@K5AUljB-;xkV zIsFD*2-*$GhmnggDqvFjT0JAW2mPWko--E&n?6!L%!2*@2Q?AWQTEd zE+5LLK>IKr+AYehL60KhC!n5AMLZwuK)=7--8}!#HT>Mcp$z3>==Z2JmeRh8=A)R3D}3O0Y*7pOL`m9a|`01iTEI> z7lnw6LAp*OUJkq-@ocCst%%e3^Z|R*cuGUM#1N!W^WW;H>xrpBeyA&Z_2<71f;)9@_IgR)*;MIty!gcBr;^%;0M?4wRP1w+_1X#X zTf`&d=$iCY&u3_#>D;)KD?oi&kMyWOeW!D7QTdAyE?xVU@?_{&(79hJw}9()DbjBU zcBAEv%G3TzBa+tx{T~n?0_{AVyNl|fYeMyh^ie(st|P+`FM;&RA}#{@>6-LZ&rG`HYSA)C*;wPbAdm^3$$IB28$tgDtO85&s5uo`85b zaJq&r)lb(x)kE^vpk1;+{3fK=32{32tuNw#4WmGa9_AL*Q@ zl)r*>(K+WR7YF-KM|#{LT?UAMg>=zx{;8hZ(2sIK^4s9pA8~Kc6N&gTh|e0t>7315 z5Z8co?MJ*X++XA)9tn0UM_d~ECl?XVhT}VkS3$Tfh(|#Fy+vFE?D-vWd8pTYpZ@+uHdW5gqfN52Nw?M=lp`6De zE(Ptzdc=)EemmlWA>1s)lfeE(hz|n$oI$)C^jt=K8WFKSh()&Lp%n`e;(q8Af6`?-w3=0@zaoxw-J|viqwoaUGw=3;&D(< zzadW7VDAIzAg9SM4$7N2;$6_rj76NT@uG-$JII?MPWu5?h|~V7J>uq2ADt0D1bi{# zbZ$Ri#HT?!9Ef-=*gqWcDzH}y;(<_4w<4|xcB5-`Q+s}Z@~|JtTSEEFM*Kb0w>-q} zGI>MXF~mbaz7Fw5Xn$@Xo&3!Wd#IwOZzYuqa`YQ z`8g2rOz=M(ak_@B9OAN&kE)2P1J^-39r#?tFY?UHS%A1KJmB|2Tp02(1aS>Gjz`=Z z?7s=|Q&3LQ5nl`SC=cIVbQEp6j7}t#Sa3DT!5vTR-2jYoP-}^y+QvKQx zpJ9lngMT^1bHL715vT7>brGM%i$2E!@ncXgoDp9L`Rk8(D3q%x#D_vXU5EHyNY^&R z-N6o7i0^=O6(jBsA$}R^MGfL(pq|`8{5O=JCy1MXJ>MfPOANxe9>fK~ZUeyH)Na2)P7?8R zz{ex*0M{Kg#P34<^%2hkds-px0(=qT&R{n(lMit;o@3!S260zN?|Q^%f&Lwci-O&< z5&r>tN)XqE^wQ^lG(I!HJ~xp36Ug63h&zIP+7YjYa{dkRdC(5@h45*(rND~vm&me9H?bvn1ze9dKLEIele?@#2*pt2|r1?Vcw`HN+{o#@^@+3LXmEmfNW$7VJ7#T}P+)BhjJtqZm2c9@56>&@8`w*w&eliiK*SiA5 zcYvN_h;Id6iFgU{YQ*z^*RnXV(IvR9-DGiMhX#pJH($uL}GEixc@Hz#CW`niB4rRu(7n zr$PQJi?i*}7uq>$=c^z;ipAObWpN_^0pu64IFVlj+?U0P{BPjP zS)9oG08eCbB2S+ur?WUa{)btd$Xk+xGww8tv*TZf_;Q{&rwQ>W;BQ!*==T8q{m5Wn z#!)+m0w2xdB%TUfDGp}>ixWLtL4GESlX%kaT}%op6wuij>QS0&jGFgCv7eH z(SF4Z#Jiv!CS$pYqxRt)aX1TLhjJPp`n>?NZT^H?1iT0F z)4-v-$xRd7if|>N8}UaEc;|kjLH21pDVKpAVjI%+F@a;+u2a4S_$1g;POoe9dO|r} z3w}-vrzdaaxaD5vOYudLrHh zdi)S4bC44k$lGx9i`ZEJ%55m(gMmjOJ`y->FKN0a0$+{f>A3ks#Oazo^g2oPkh!vn z+rr!Yi6^b6I}oR9L+wR85cFgqPREJn0Vif5^_Mzn*dgB8{C{l?9M%#uWRV!w;a^sq zVJ0^I>o;KRRa5@2U44aoi$d~~fthY)fEEH#B^@oxz;=oBwa%A%S{?E!>WA{SZX%V~Brjzjz3M ze!og&iKFc{?}!=K%j}SPpi2R4i)DE`3!u>_-+z!WX!vkh<9;v94T_!s{|ZpKBH_Lg z@E`y5n6yiO3%}n)o<#9v-j;+e zs32@<_!MttcK#Osa$BB64hWL?llGYUr}00^Fcd|uk4gw%g?seRAC^7Bocde%Zg2sj z{z>@6BslykhGCry;=F_KX*>xd4$G4Jl)r_a3gJ`#Bz&S0hhI;1(i6-vas8m6>YxCT z80Quv`?v6OApG^Dp=TTkpFE18{%O3PGK>U)*Df%D=QdrvlwTUHx4rn+B@G+nMo~fD z;h*p+CjBmM_;8+XoXAIaj8ouk{`^yablZp7q5FS?n4JXh&t_yF%MN7#>5_t({)p4H zFKGFpC#e1C@e~Lj`!9#^{GFD7e$hXvU1rc{3;K6orgwaX+4W`Sqs z;nxE(o`6XbnKwTC(Z3;25l_O&ONe>_th*?67B$*aoo_O$1HA#w-qYDdwKoU17>d98F-<#9+rWS^-)^--Qrt=J4 zO)ZQpUFVoubL2yn$&3I~m~Z1l$pN#x9P>@R57|{x;+^tv_we;1Ai!^N2=iS#Pj+cg zXxNkxFArvY2J*ouTaSFGJX;q%WP7LVUtw_J97B6Ub4#mvh8DKG&-X*ZnH3tJ|r$cOx#Ft4S-0c2@G`Gr&DmrjxQw3RoJpC@m_^)Q9;FooN1q`Awi!LaJBf;2OU zj7joLc?AZWB%P6yWPs{#FwsWFf+o!D_+ZAfq(5N7Kv9$Ap+t>IQuRNQqP*RBO=Tw6 zncSf^|7WQH)|v!wOp=;m68;KO$a#LxzGL?PlE00aX68lmXHDlpA;I?N3bpM?d7!{!dVU!u)SMhlQ~`(q#V+ zfrZWaM;_KlFXBU{DBoeXaUls1&)d(Yw{G+IR3AM zm;3)ic-$#hLYY}eXj4X7C)YoX7_XW8kMrD?j+*FSThfdWmjZnWdK{I$y>Vc$ZT$#l z&5n}Ij~<`i>p6DZUO_p2v-ac{;{=VJ(jCo&{B7nZ*)(OHSU*^0nxOFL{M<72Z@QxS z6J-Wz+m~M3wrRn^(Mud}w;p#Zur@s8IC@BK>Edd!xi`Q5UK#trWouZl53^K9TYC4t z(Dy68rjL3%=DnusrSeTbQ??gvn^rSnqU+EP_vY;|3OtmfHS)BldtzGai`)T24rk5W zr8&?;vs`ty(eridj`wu#ZCCEP85AgdBWOu#r*irGO1-1kD)rmMe&kNyXF*4% z+6rx-_IhlC#{@f{l4EV_tK4QRA)$^JUGy1m*{}FR=M(6YH$Eaaz)e zt_DW+BLA5u_V3YMnmvBc^JQHP#`~@<>$0&)IsKsPWTTeInd1XyTf0A={@b`>l!orI z!7qzu-&x+z_F4IoL*^@<9?ecpnXNo=$-oyUjWQ)}pHB~+Y2CUiCTUy9L5tTFs%4k^ z-+6AOez$ht9fg_4JXYNm?@sD=o?|R_T4$Tau|R1vMFGvT4TE$yly6Cp59IHtxz4#e zxlBsZ;t{`^*vZ3k1$&e1>X&y}g-yM<{$-TRf$ja(u0*GqPM;}umv4MxY~?TCG+B;_ zO7xH_Rr$T*xh8&+mk;uFrAzfGZhP*VVIq~lnNoACIloI~qvXJA(tB3fw`vOd&UoED zP&s7lC4ta|p)x85B!x;P3^USpal*tTRt2_ty7f^V@oJo7{G!E^iuY^wX)sA2@LB8# z-_M%itESYbxlLZCUA*rbv%H0-Km&7S@82apt)rt;N~P)n=aSX5Q^L94^3PA@%wJv^ zk-T)F%BJp$jW4cv)tlUF`+3~q?Lv$D5AVG)ceYpf{-o*2-hmNc;-_BtK+ErM_tndZt$xyZn>%gG%F8W^Z$Cg7eog$wH^z|I``tyy9!r%)0QR*teubCdswxH9LO^@p2PBWmp9C%#!K;4QY2Z`Zlz zxmArd`F2fmyOSPp#vlKxUFF%;%-MCV*&_b(+sZyFyDK9JR$_1jpSkykSqxszVel+Y zT4T4U+tk>x@-M_ZnT>$CchE|O-IbQATH}vLiZ z%gcjDD>(DLbClLN+O;G+v>ooA5^C2u?ZVs58m{4{&#spxM${kD%6((&vLpKTrlWJi zweA%-atf@b)JVnHI_OsB8QzMq6Ji#TkZiY^vg(JOgT9mPz7tbR3i9%;PP%+C@IBJr z9<%$lQhoTs%Qq|Y#y&7Dd}kB>Y{eL>DG4HZflsEVW(Gdj$#9%MxoK^_WB>Y`73&Oh zGAc&RZ0sXT~HK4b40GX-QdL zv!2V|c9&Oi^-5<)<~Kh(eav6Ou`FLFUfVX-=<=9bxj!bf7g~SXo-y~t@YBmqw!8YC z(mye{V%`_kPRA1mK8(#1h}&zw{ECKXz23_}dskI2fv1i2zZRaH75Y_EzS(`@WSy;-IZ+B*|7}I`0X``0LLpPlV zbNBVwW*&QC>>d#5b@jdH^cxEUx3`=18~Rrmb1sQa=^%9bEYA<% zwO&hEp(Lp7M|9EnHOVjDduhB%{rEw~SZl-x?ZYPmpOiK$&Rg<0N#O98r?>95UmL7i zaQ*vDje=O6mQo$XdHL0s>ux2QhFb>tKF|M|`>r_OT=UOP(7#1V;CabZ*(RqrT*G zb+EF+&Hd3{v!raaN910b6;bG`F+nV_?a1n*olS2(O4%uvbi3s`TN~f;p0D}pa?F{- z8G5JJiD>5rH(VHDd$m?0Uqxk#*QGP1R_zW(rJku`Q~D?RpNMRWShRS9-m=@nf;?5< zds&4RY%D3T_4rURQ`5Bj+0I#`P8$qbJ+g7`k~f2|3O1j-uzyD5_6hZg`e&uAEXFl^3lxDn|6VVOw#1zrlz7QucN)9saFl*h}BS zfi{7D!h5ECNt9`7JGMmMZpzaLqYEd-G+#Mz?o7w?#6;_c@kO&Ae!XC@QagJ|B4*oLjqi1`(c-zXaF*UnL|C9DR6W~D)J&sCWYTabfq&p;C;!;EKNP`mj z8^e4TNUzy3U9F%gBq^h{|2cd8D$9?2Ch_-nuCVTop5?ReYDMei`)}Xw);ex5WlrP7 z1E2cue5u)Yq(Q@Y&?%y62I8S2p@QdafHk@4Kl(wUXVBk~U5G zQ`^lJ9~x!CiR&}E-@8AR3BRVDSgzgXF*{*}?M zwy8o7yPG2ub3PZ&OS|E6zHs5l*)3ih&PJTq7Ju``aOpb5=6!iF*W~k>4qJ2_{Q9Im zRBYbw;qQhj$CqsoiJEqP?Y@R?^CJ;ArQ!z}%&%ScLZSZDUcJ77e!lrGC)>qrpET_; zc)IS_kMPMQx5|EeGqf+zEV_C`Y4^Yn{ra39Y2Y^?e|Xo1;US|{a#NZclvVauUzk2< z>V^3$53byI(r;cy!s`o%t7c4Gx29*se7+~e{eEPe;*9NCV&b5)q0;v4cbn}aPlWBw zIAFZzg-mM8bD=oLOIAA1zCS%dG% z`4Iw=xA(9IPo*xCue-NBrA;#P>Bj8sLlrJuNt!4! z-I{OAY+sAPUSjtW)~ww>)I&PiPV#Pza9^V}v3GY1i%#rgsH}X;dC|BG*X=2*6g?(L zUsV=)wU9IGo$2>Kb}A;7fN}rW$>~B>RaFg#UrM+xV=Ld#HJP8Zc3FKNFP#0!C-YNh zzmM)5j{io1ES|SN{^HPaRC>M}&aBK`K3Rvq<=DXFiwT#K#rNrOuNBrwyUD3*d~klCU@Y(Ug#DeN z5)lc4nv3;}g}$X4CCEzqzuqq1AYsc)Hc4)|tW?qvCct z+Kyc?{B-M{*X%~3ZHokpX9e=-8;zS15$G6vuD;sr%f$yZba@^%I13<*CNr(yPHzq?$dy`W;-hHVYy z3dT>arB@YYp2#~rZOP-GAGxw;%?+Q!bu@GCp)*~xEbZ1mF`l_CXZ)2qgL>|afb_Dn zGTrTBZ#P+%9vfYyyForfzB%(r>CYP(?j7|xPF4m-SJ(cue_4FBJMl-S;KOdYLH6nQ z-9pcf{d{giNbuDuLnhL__a*W}#FkX*J|FmCFw-zz_2$pd@?UPJH`>R@{0#Dtb-S*R z6jykSVbyyxsIqUfnSIL$1l)@c7;7TQq%^$=yuz7p>{)>C~tSiYxv! zS37ia?{13sxhs7t`?%uLrg zjX8^=?~1Bd}a(Vk~^?*(+ z6`AJi-hu<=i}(BIR`A&6SF8)2^JQ;#ptas)C;o8dsqOwjo39oh&k?ES3N9^9ss8r# z!R3y#kJ}zx`!W3Oef&4iPku^%UFo@b@0T^)hqY-bqb0L8<|LC_gYg% zU!J7n{`SSU7`wNvkIWp_|p4y$xg%Oum7cKiJ^;!S<`{disle?6ua5alYEn*9u zG&g*-&G)=Q!Dnp|t9WeRickn*9xbs=)P@kAWLf6gw zdj6IlQJ3>B`}oGMjW3FrW**RaUjE##4{@9e7UbM&4K%&BYG&*5V-NB}uMaIO4ZCro zy0WQUZP6YzBg30JKJ1NssOZHbK6%pP>yLdePx^82+=KY)j^+pUDQn&xq+->C{}7RJ zHIUtVB~mdp_~dH!*o%D|1dDhAqZ6ZpC%od?@b0<7@tif=cBXsBG_-NbjV@2CFcsga z<0-T3Kp{sIW0LCafD8Rs^ZI(WkF9fxoI54Bb3@*o<*RlVzpZh9T*aHR`1I3=viDDq zva1Fj_CDjqc}HqvaqsIN)h%tFvZc@OUugf@xBJkKyoz1E{<|WzT52TTw)h9loTsyH z&5_eDq^r7aKkELvptVtBLz%lKZ+F00W19%)sfy|ee*I!nkLQ zVw>8YwnK_%lZIvgqjOU%W^5IeUGdBDgXWj0y}|e04w|m~>08Hfc-i}LPGr#)&={?mGW~gSVZMbN5na$;;?q;1!4|0c$C;eQ0SZ`srv2&Q|x*6`R-jfv; zaNo)=tao`ZSy)(i>TMmvt4*!nB6;gm^vXgHHtmbAkj%O#aOFkjslEQ}CkqshA5Wdg z8P*rNd2>Z^ZAoX_v?i&<$NL1QJs5xL(1LCJnF$GlqF+V_iluI!Ma8FCzjaK@_@zA? zX7DG~Zi-!59tYOXnIdHrtd>iUF6Pno{s%_oX&EuB!VM)P9#w`vs(kS7 zwdgbsDdFy|#cy3Ehu>el=H$gASDkXaO4QZXrN_9Po+T3*a?-f#xZC6>mmRiGxtW+D z-^BUj@D{<$llRs-ZLMv(H*jNF+7%boZu6WMdM}RMR5GnHs;-InQ8+g3LSXQscXOl< zHohrR3CL9BI{qa@W>nVbJ>TzKxi)>uw+iJQul=PH!_x%f1tQb36y#?%E%=~VejtD8 zi)CWx9y_xCeE02V*q6baeC^SKRJ0`9s)bg3Jk^b@Qtam|>m1f?Y)H#q`Xv3Sp>vd{ z<6)}{&UHuICvGZR5IUc)T1)lFTr<-!KV9pG(jFUMw6h8K2S@IVdn9Yo+@)r`CHl6^ zoq|)V4)sKTkq^q)aWiRV@w(>^;@x(AW=~Drs@rqgI(wr{XDa(BKOr%*G1~I3B8Htj zs&@`=6S;HC#^g%4@#Ze}qSozSj8r>};|~QY8m?FR^ukduPtfA52%pGI*Qs|;wKup4 zOPqYq8GCae>1wWj?uot5l~!Bg<4^81npd_*p|j(HVUWXtcUi)6J9QlwZ|OEa&=Ngv zn%0BsZ9|s4N3J&{Y*f3VYq&}E+gzcf6>;G+B~Gs1@x4)RpMCh9S1UHmR9dklcxmgS z&uRnXe=Ti3>hRO>(S1Du&*z{0yvAO)SYwm)?YmRrcbA^DR#D8*~v9ROTQZgYi_kqNLDC~E7MjKG-_S(dAeHm znJcfpd%AC&nzZeMUDk!3W%d)kWFM(a?%!(LZ&jEz%VquEFN>zkn|09n*!2_ZzV@em z809!lC-`*L{h#+3jvnVXhV§@%FpKEvfPY zFSosuFx7i~d1=9%CyOuSC27pfyl`d>zq7h&&D0G~_<66Xn8i;}lhyWL&{ff;biFX7 z^Y-Qf+4HgE)hzeE-|D?yQ0aP=^b4b$q!%JNtCso7j#*#m@+q_C{=5xF2L(D)SG^iE z3VyXaW5D8`v)d_s*+ISa!%LqXdFAINv;MGFNJ(V9{QYnDp4GlQc1D!X>yEyCw$xPz zAr0eUo`lpQgL#%-S$b3V>WYg8Pts(6ULLga=8G4t8l?eJ5t5-?Up<6Nb~c-wZ(F)$ zfBUhh4jp6PR~{ip-d|I1AG1_&Skxi^JZqK)zK>)7&+~K6fo7j_74E1$U7iK`kCih@ zjNd%7yY+s_sQJcU`^#6$nOexLT6Zx*qHlYdS))*SWk*u=Bi{3ek~reu92w^{*s#Lo z@U?47vV0~YX-00J8&)}V#-5@1%q5YYrHEW=gGOJ+&p{eX3kdbEs^)uHQMpM zc6@p@KtknE&dHVAD~CVMU&xT1yr=If^}@oZV+!u(jBSrvqi=0C$PTh^%lW?fVgL1? z8`cy{<<9STx%*mTU)#5{kJUZr#;?t>%WM*pyY^w zbw~B?x=S*aX5|7IS>to+om?&izPXY&zERAxcOKi*8_go7+)eZLD1SJ;?9#4@PoI4r zuu?IMf0y%}{p^tF|03)Rdi;^3KIo5x|2HvbkPi408zY$7W|;Ew?__5;9>dFRKGuwv z8$C0lnp{kMP+E=t)8mzNqy&~xliJW6NE^hc#EqV*b$I!s*mZa(?xez%l`ZDzV;jRe zQID5jhnJhVydCwI)sgpS=Ax6Rgjg!tpc0+sv(N}bvl{LKJC45Td0moZ-$G?YAu22V zv8YV_Aj_N=P1&%TvRP*=W!oxVNpxHo8t31R!>RxILZ|#YMh-M`;;g^L&Jm_$*l4ql z@Z2PH>i<4=4w}jIXF_vZ*_f-wvpCkT3mos`ZzT2aSUb!)0xU(e|2go24K#)}IhZq} z2*~6@|AkDByzysROYO2zEl(0KOsQk;aWLS89XN)*T&NF{$4LTjP~RLhPQw(RMBg3R z=3vek8`&M%;9yJ{*`{xBFjt4C6V^5dL+ZbfA{@+U6aPKLG4iI{pT{^Ds5bvhzRX>E z?Xivr04F`f?Az1~IO#vtFj((Aax>S-rytr0dKS?kwH*z71;O_MwUlvJSBPPR&K6cF{OTrY{uu8|S44n!j;0 zXY+4-#y>ckL;E-RmH*)A-8Xs+B<6Te*|_B&^4tHxkN$(7{|CQJaY4rKag2E_q4ItC z5Bd6kaP+Rt-^%;uA6(-f{Nq3P*MD#nbK-CL^8bTR{Rf}^4?goBT$|#8v?va9p@Eq( z`R{QXa}L@=@tM-WyuMQ3Xf3|Kl?$zSe@gv1MTt3s>5th<%-Koy_HKbb&i3}o zs*I6MW|*h{_o!0LFj36-P|V0s^!QS&$Wp9mQmpt=tcX*rC{yzo&R(vLK7oFJ8vgg7 zQ8?C;*ZNHX_HI-G1MI1IQ{D_mXB4$*Boq_#u#f8&IQr*VC=k>i1VTau`gpDP*bLhS5!zwXU6~rHI>ZOCy2okMs>@aHEZwaw8_^i zkjjGTTF?+&w$$$;Nt4<&v0V~>T+Yaxp`6kV>mkb`h%DLbK1zn zI()}TbqnTPcpNK+55@rn>U$RT!5rTgA+AE>iqtkrq#h)L5|XF>3NxYy-xrYb`bd~c zNPaf;!TFjXNF`E!B?~j<=TO^NejRWuKgbFd_RE_lkH#)6&qc_82YGxSgzT9|lgD>* zSU=W-{ea}vY4T_rCB)}bA1tqpAeGR0GpP^e*bk)sWk{GxNRI~f!SYiPq!Qu_s1N4K z2vQ00h13Ugy8ne~+>qKv3CU|wA1rTyAe9iO`@xAxS@nYSz@ODdV(B1xR7R|4Gsq*3 zY`{DcI4UoCw!%CfIFesXu1n(;G)~X=K8@4M)c_pXgX$XF(?*kDLgRfj zPPb8{3I|h!V;d*^2pG1gV6s z3#QZub2PT2gyhjN%yki@65{A-6OExL8L_^Y9JM7ZZwVabxtzwWfs^^V(BxOpIuPu$xKnY$$I&!9Ag)+#q;8RBRQ&7omnhrDoRGh=07HV&4%w z1nlo2_yJxP7$W#>@F#jV7$xk_bg)N|;P&7*34$v?J)+;0=z5x=J)qxB=-dPL4G69X z<+3C=UK7-b;04f5{0MFa?IDcdcny*`g5$MCj}yF-Wi|ufhhYEU7*hp=yb!dv3WA%1 ze{hUItmhHj^EA4dcK3-ju7%C(CAF{Io{BxC%$FVQa?@%b=d~ZPeXM=ji{4=ySdh9-=zY_X8jy;3r$3cDJ zH6k&`F-mYfV{Qfct|9Wp&)Y5pe-8b9Bf+v;$5)`5`6*DY^?JK+98aP(UPN;t+0 z)-M3#ul6ogPCE`Hc(O;ax)h3AE?O1Xl$Aw-I~^w6|V@2>ufK9bT&%+kYSI;e`6Z{4D4hOK@S>7bG~2ohVN5wXm;1a4l%(I94UjcM7!I zC4_uDTrbQBJ`d`}mf(A!T%H8SF;8&}8=S8$w8Kb3{x0k%5WENMNg?=b=!}aHd%-_j365h9L=*f9^t(d@KMeJl zM(|f~pLUVp_oNb3*^d^~eTX zkl+r`FC++#?*nEK+z-YD4T9rw)_~wRHYko&gYEHx_UB2+=fODONAP1X&Ib`3$F>M1 zIKFS*LGVjZkMx*$=zeKA_&=VI$FaaN2%Zc6JWucp=ofheKMU=lklwfjphylc9gnWAUN-dIx&63Hd#6AGnO*r-54$Tn6ghmEcX_XMciE5M><*CwKzX zV;sTD!G0Xu1|3HBzlL_0Nyu-6c7kKHV|g4qm>!D{>G=Wu{~@7=1IpD(a2%5j$8N{^ zZ^AhFgOmrqje&6j%OAn@OK?06$`V`+`o&y=S>=A&>uFxQ^gG zP+kv$;}~c`1fLJ}8%gjc=ywMQ&ISE6mEaDL?*)S6nDWI0cY<LX9U4Hp!~*033Bg)TCe-U|c2S+rZ9p zLLT*9lxhfhb-2DX(&WJfO!-%Y{5u%W-xK^g*oj{;!S#z{;ZA|}i}_imc52QZ!PQ}$ zu_U+x$ZsI{E0Eto@X62)lL>@FlYj-}8<@H)`bO>jr>GZ((UrtCxaQHP)(sSg2#eCqX~{B8VTMC`&|V82>pv4^1=1^6Z}7s z;DKQOT!QPuc(|0{N5MZPz>!*TIkV=S3HfPoeZl2M_MlrvP|w&%lSdqeRYnkvqxwbo;L)?{iTP- zkv$$D|DE7Iz^8&kaDAD>cqk1V*@?@9PSGUf8$gd9jU#_925v}j{C6=^8mIet4RDk% zGFcMzIM6t<2ip`(aO~%OG*0*P2^y#C$FYP_zH~nq(c}@we!flP$QJD9hXlud9srL0 ztOVo0PntY5SxOr<_Z7w!+-|*?lv)p+;4Z*rX&m{p0k{f{qk6&fcjprP5Xi42cn$E? z1iu5^8aVPBvR?*$m*Fb-iqcYz*Wg1-eWMB{WnPoZ(-XFlW*DorCe7jRkNXcyTt9SKuOoyL*h!kCo7 z(52}?`P#tvyNo6eMP;7hNs~wR%!7P=2(AKr6OGf$g=4{C|EvS~?KF9$AHA1`QWS8s zi|oPWIzr>9T)xx?CHx$Q>#G+EagHXBj^Tvwcd$L%A>UfU9y}lV5slN!)k5RQejKX} zKd)f@$)LZTum_jxD~+Ra;daZ8lu`+oD+;cswTJG5728b_OW-V$D4 z3EO!G>{O@8qw;=$@!W;r?_sd

edy1~+&91pfv4H`6%MABGH|(ta98o9I0`lnxS{ z7x*#Y*bnwBaYi8_zZT?+3BC&WO+pWDS4{-R&*MD=$MrsbBFg~m&yAp8fZ*Q1MSx@b zuYrG*XdJbN6;LiU8b|eh8{|z1j-R795d0a)M-W^J^dBMkY~Z;BHv#^T;A?<)6Fd<( z8%#n)W}tH6Btqk;T&LiG8o^6}TM_&&a6f|A1CJ(nJMd&0NA^HBWaQ8|vga$v*ASc= z{P~99V!*%AIMOc-dU#NyrxFrF`jtRlp5P0CFDAG?a4Ul2zvFw;ILcR<`k=Ip#!}(*uvzOl|jQ@~FJIP_Fv~zX<#Zp$Bg7882z_s4L@pon9J8`tO0B zK7yA5A0+hS_CEn$a6t8r?4L`0P?93J5eqYB5qvf9g#_OS+??RJT=u}RpYiVo&ICUV z1@<5~{$0Y0;8rkjY$P~-t`8)5IXs7k5PT8bw}kzJAa@Hjw}MFr>b{X&f%?Op@{sexKm@y3kGIxa}ask20hZ`l56Edko_A`%i0{Je}{L zapVh}GXB9I(Ks4sY~cj--abm`Fyi=nCqeKYBt)fk1P^3khCjiJA-{bzP9LxGXdG1) z_C*!Jas8n&79~_x%+NmE4l?bBDo2*!P(_+{_X4Cy?M&oqUol;wLgWa8_EEWu-8AD0)^6_S4r9AB3( zpM^4_5^mR+@F;?(0LS%- z^F0e3_Y2Ig06#?NDF%L=;P|?HlHdum#A#l`e=>2gz zv?KII?~nNX9mMJVF%|S7j>?SF3mT{QM{&pu$#4`p$23~)r}Dl?Zn z!O+zy(A}OH;!oLyx$WrWcCoqfH%QRqWuAKw61WonhdnapKVVBg&*P}$Gl*N=*O77PZsdf2=9J5o!w z{NE>ZEkaEg-}hjRi}42W4b6RoPNTLEqxW5Oa54JtFNr)~j*>yQ=>Iw&Zr?{Vk^8FD z5v229_6Lug$=cv2_GeK0XlPnYaBjH3lRu5aa7^|R{mDVAK&jF9Vn6g2mO8IZ{?4G{L7eUvGPaPM>2m|$e(P#5pdB*;{!e6 z^O4OcX~Owr|Gxz1`>@1D{wM8M`-lCD)mW!ULPp4c!qf-nkK3=qKk}CZgZ@wXqv!6w z`Og6!80h(4F<9X=oD zkJdx{TmBw!Gl$o?!TI6y@jm)H&fm_T4-e9E%!63^cm9~%q>lfs{^dPb5_jQzWdBs^ zgY(DsH&e%%BKQC(#(#g;fqnc%h%wN5J%7vp<9gN^AK^rlza;g+=i~g*`&!h$-v`h( zmGIx^Oj<5TRmA4^!%kLgwMzMqjiP;mj58?tW-klQPLAWA2alKvcHwT zZUak#e*Kc65I!I0kFJv_3o^ESE9$INDn}PRTEgdJvnK!J{7gyKNq9aBRzaSxgwMwU z@cm>OYeSykIA8pY)+@l*zh9tG}4T`fo7Du?&quM$Zoq4^xkE@Ghfa9hOk^#Pw?*6`kev;E&7)zKd zQnL8w^#LBgkNj56&{?HpX%zEgm0`?)1v9R4Y?vV{&Tjlt&AcNpgg=b3LGbmz=|LG>)`?A$L5pPU zUBaeAMbt){EYo6=kwPj~xlzn*7BPwGHH?9P??$o3GIjjaapyjhnD?yXIwjNzlnE?T z{zrx|%fmYFUHi{pzgXpS?!)R?iYX&l))>W9U{g9qOnJvft?fvw9?CzZOw(ANU{yRi zj;uo|8eL$ig!ic!(v4dKgdb1g`gqJF?^sJsZHre>pG;vS<5h`_te(K$w28r!mhG3c zT5{?vTSAoJ^nz&f`C1AMf+3k^8)oX}Bo&-}XmKS!e}eUSha9I3TEY@%3j98D&pcz* z$+oN`thBT#Ioj{puycD@PtEh-;r16VTht~*CAc3gcr~p2Xl!NS-HVdy_7`=90-MIY z^Ut1Ut$g?1qjNgff@2Q{O_J|u5T03Q?UflJr*Kzi&U44m+hX1ObMknPoc~mPZA0}^ zmnwT@>-`FgEE8t>&UpG(aX~}tnzyEVdp}p7(kpJB)p53ZdB`~_TWyD5=Vn!^nAjGq z=(yUcwo$ow#l>j8?Gryqq#m5lciNw=JknD>(bz)6Qrgwr)N<-$lU35F55-k~UQ#AEHb;rCpYn><8DtCh?!ClFgS{FB|tG&p7`L*_H z<-~De<+rUBuSkSz^c^l}_DTiR)CXm-4BO^N{t75OEqM=5g*v z|HEAdE;4nsGt;;8I?bpvI=$T|O{8<$`v|+Vsh!ed5%=|^FYPL1Z(MG}(=B;{JxEY! zdrQURGP^XM#uYYW#U*9gc1L$@Pp@!U`g&A-Qr*Q46J9?dhocRa-zu`!zY5&4q@n*! z*}~$q{b6sXZJQWUQP*m+K7Y-aj;@rf)WUKlYwp~#hZVEAmW{2pF<7f}Mos&X$&od> z@&iX}<5R9o4@q*@ZK}N+EOK{x5PO_bhg`&xAHoj%H5PFRR?K1;BvBEkBO1r6{7CB9 z?%B46;auOu5hwM9eVmEAgrCMI!7};4Fb*62Vs;gmACz<>0blH^ZQyPr2-F5HClZR|hw>v5%eF(27X-AkIMv zZlpGTOrZpS#dY>3Bzy>EsN^L-+I}~vwym8ytkF8NZApYRco^D@?zbFHX3p zJ;spTTTiua`9a1MJ%u`>gB&kTjLWGC-B~NYZL8?FA^+U`+&IT$xAJ0Mra$udkT++% z>BOlw2Hj_H&0Tx#RSjFBTjcj%#RY2C15L>pK81Scw#U|#&z3Q*$@i(Ar+Rk&=kVu_ z>sz%qTrzQPy1SqGcHQ^om=|JOP9+x z2V^9Fa7)h#DmbyXHt(`?aR2HBC&nANH?InrQOPgpU4Gay?s($)fm%E1x_r?y7eO4#&`{tEA5-Y>W#e? zNM7gDlv%ebqqj5DBhV(?PEK!walrl*!GX5m6oG9#nsV!^%?tLgt8`r5aHJtUcjhz3 zZ9V%X684HA6OCi$>8PdMH(S>q_;rHsWBHI#o`PCY<1Vi4pIP*{s8pn0B-6BieuMI} zJ?DD(x1BW!XIrMXLRPHp#a2;ozAmjI{g?W)8>EM(HYg2gH^_yIac`XQj7>{%Xj;C0 z|J4RaZ;7&HAoyVs9rpe84$7x~axm zDr3%Wy;*)%6NZnMXLsuJo|AIZmeVuYwmjzO_ap(;ZyIt#GhfeGt}0qJL3L7|zDb05 zeecown4^5t^-Q9T^?g;cgoYE=CU;6c(yTvG)1~LT>AGd%7_EfP2PV(-dS`DC z8s_x9tRFD3`I+9*)Zn!Aq^Rf@uNUg|Yk%n$p0e}0Ugf!d$-0?ClM8Z$RbRgsNM4-x zNNI@cOSe$+j(K~u9c*Hk&nXc~epRC%aF{2hQ`}iyc~bGz#a5nG2Q#*usS{Z8)2-9b zVrW5yUm!{Ukr{NR0vSXUK|MO|vr&Uf7o9O@C@o0fc_~s3_ zt}k({>fz<+-zk=$R zoTzNR=hr>Csm~k6GMnuw`>tM)br|bwYdfJZP_4dp`I3gmkF8^e*GGG<49R@G$?Pe6 zDDQjChzB~R?1SPB6Gp)Cu6u8{fC+OY1B}Sur zb4+vN2f>qZ?@SI_hz4>kuuaoai8t@txG3qmyqj{;DJ5CzzjM17?wy{h+B1)xI=5gN z^Kc|$Uxqv8T{N&h7dvn!xb#-|qK*lD108qw4623KP0-v~ z^K`m&K%F)3g7ZoPzv2xheL23f@6DmXqW6oU`Vxm`ybby8Ak=Iz{oy>moSTz6u3aBp zq~>>Z^7r1TIhhyJmX+B0KIGKs4efr|(py~oF>RnLFlXQK9~)md&0kT!p)FeOhs8u` zKLMqhx>}RRh6~fK4R+c763m+?aiusTu{!SBmDO1Z-^TZk9je!_d2;Dci-_jCZ4K#u zej6Gt&Zx;(x4n9`!eh$rzOeifiPVy{Pv&hp>=<`PVEsbLUsvn#|FI+;(i0rly(6utSTh!CN&=*%syWyj7KHTaTDuv)XMPTCe+hcBJK$7lmslJhIm* z-mu*~*`?u0`U}@hb`}?29_&(o>vnw4sjNZ4bhB^wWrV!iOLY!K!%x;D_piM0VD|fd z)_bMxid&2Mlg?Nwj=deCn-$>~#t7T)VJgfcmeP5S$NblxnB#+I`#XX+&#V(X8IpXg zBsp12vE0A%N7+~Lm<6N#CzXvpe}nz_o_lg}B?q5fIVjwi9dRao1?S|U*GJjcuAGoJ z(Y_%0tit7*rMDW4jvTx&aoU<+=Q%>hwJTUFn+awI2{PXDODFV&yJ@#RyOflB+O={| zNqbOxzmC1`%Z~e};>&!FJ|3%cxLW*bP=3Gz$)GV=ZXq&9p6bpq66ueTdUR1cwNlbZ z*JogdQEI*9F8^84dgFTKF7*fO-%u>@o$L6@UFw#f3aXwNd8%=(`)s;>^%X76jhxmW z47R@7nxFn7;JR$!vB*xwZcpj#9*+{a*21x^75!%4JCDCTyxs0l$epF(5vI*$S?hRf zS945yo%^A$`{2|MA}Q~CqpnY^`&IGQZT5Qqy0nBX880Wz`mrYLwdj-1{S&fX%N`r2 z{(5>KA#7Y(;PnkMno&iL?6*|dxr3Q-(OM7rMv?%$%~oP<1Ew_MZ_J;irn~`i7!S z)3kM8XnjrDTKKhivw3#*d&LD;X4F;bq)u5;DjCfeJ6>dRkG*+O2Y17(d%0S9C90!C z#G4L#@TO@9S|vG1n|R4gIAQ4OF1X;>s?S2=t9)-gy)U4vHT%8x<0F#m)&;)Iytpgo z%Brt>htBW#n4h*YC8)OU=~Him4_giRR{SVw<0yXd#-eVP`eVbA?$pYUr~TIWIJG1@ zWXcJ=s~UhYGtE>n!wK|)E~!3whnTPY_fi&!}W;&=7jgRBn6t67<1_GryMJp zq`XCTzt%J(N8XArmjp%$-JF*ruUK&W@*=b5F0YD*ii7M!j3IXIh~z=RAg9;e6Z!b= ziCpg6*0*iW_e6fil5v|E5z+P^-d=KCY%q>fWX7#&fe&SOujan_SS5STSes4tX|;6+ z_2TN%>|d0y&klCe3G%OykBYfv0Eynq_CbIJRFQtVluS{&Z>e|y~T@Sy#X58X>FQ)mCxx(fS*=y>P?p+Dow81lB za5zY?-R5B5f}~8}r=c-zkBwvby4AczpU-OaRme?zW@NahW6O_6UKZ1GE_y7rch%V3 z5v-ELbS{#t8x-`SIHcU)9@$Y5fV(nhiV+^w5M`SQ+N3LN`fYTTdj z&h_ci-8voHnmin{Gt|=>cAA?jTOT_zTr|I6=cHeX#(jbVt~nLg-q~5YM}}B@i*Qw3 zAL3}A`NH_8d%WDD;$q*Bwff#q-&9SkSF}AiB>y?F%3i9-{5w}r{kfhAU!UDxK3}|4 z{Di1cRAPzZ_OrQjiZ5yGv2YF%nv@mW(QevceEd{-$)W>uMPdgut(Aj}4)6KyH6eDe z(%6GQ;TY5V#=oP0-PYYcwE?kc79YW-s%6iN_&y1?p-6Pagy`Rf4xxr zo~$EtJ9qt_>a4fEx(&gv)!+TB{(Q0Y#*d#JzNNpu#LrrDrTg#LD+O|w6F<(i9%y(f4VS8#SO2*1WYBjiT3b&;`##XqC#_pbPYwq)HNcKg~(<1)zLHE4a zgW7Y2w{CShWw&hXlL&2N(xQG&CFWMqTD>ty6awS z?>4$$t#00OWvS%6*Z}>3FY83AqO(JLWNLMLbzeb*Yd zO+VYeIF^>T1@~ny7RZ-A!8bZ!dQo~y#zxDP0wMb=?y(i7hOSyM&sqJ(-lNBC+Ql<3 zr6}5K#TL33+%w%{G1}mbYr0R}@uM|U+Z*qFiL4OMx^d0lOv2D^ZcqR0`}gYys#6Qc z?^v8`y~{Zy%3pNwQ5GFoE4H+S+6xe1)Z>o@Neyc%9R_H{^b$=e2g zd%c|Zywe=N9Q4`SH0kR*+v8dpmp2%V@k-lKVExQBi$|khOxn;jyf0P_zh-{P@Bj1<%@@(VV7R9EI95db| zpSL!j>ppY2XD5I9?2|Ke4_2FvDiX2ORI^+9a`oOBM?3O0xm+jfXL!s#=&|Ka*HA>8 z(QPfM>Vlt}_HVP8WS-gaYJTt%!wnT31s;5c)35Emlq{PW#n{1lPrUG9&Oz-qUb~Q! z7ZQa{0w;5Xtrc6^KDS@oQ&ZD9!o@Oj_N^iA^V3egX{;-}p6XS1;rR`o)jP(H+G8gj zV7rJBYo{W6Y~^u&v-yWN{^SaD(r#Q7ZDCTU(_yn@SpkmJpJp&TEsXMt6*$oJl+Af8Ra?bJ1c9oNJYekf-GSuZlmS=H3 z5lgbnJ=r#Yn%%yD@kKK}9#u483pg`4$ByU0{0rmQo?q`?-}_^>^|cqSVNNR)*Rk;l zRF$vTFv}FlFBpB9A5hH>&joYTlE4ZN2j&7ZE)E< z+{52KyLYwU1mC6|wOJa@izey^BN_WQ#TP9G*q+D>45GFWwQy4dPU zO|1zcWz%F2`lgrV2Hx;mUe3Sx^MUD&!JV2%rF=5Qn{8`;x+Qwc`pd<#Z%Y(1A9Zn} zqDaZgqSq>?cU}&tl{j@+Yf75BX#b?vQ$J&Na+punoarVTWar_qBkRzhYb9fR{E?#0 zZzXHo=4R>7;ot8!`rIm4+jD8B2P(^U_as_(WE7Zm)K>ly{XEdQ)p4H3=|tXrQy)ow zyWG;UWE}VDrY}LcF@VYN**RQZ{N<;1SF2Dr-tqejs)t$*!!n(?apgsi)6sU#D`PxDY0% zEMqdot3`9X-z0 zk`BMDcroup=i2veHX?r7(u4O8HkDlUt4(OLKD3xkU={DK)1USXOcG~1vN9vNK4opI zN>oI=b5_n7g{UOYmMde@-+ozn)n57a%!OC)D^BJ}5r}!_p!F+BUv0D6BBqd9LOTdt74aX61%Z_pj13 zHTC8P{g}97!NM-jISp#T!jbu$q2~P2z26!vFI0{XygMesdlT2|8)Gb{epLF_Z-2kc zs=-!dMpJ_4<*+*aqq1hAWee9QbFXh69@f&B_cZU|uu)?|S>$lxjxy<1r}~p0)e5vX zYb_JA%Xyl3YII~KUu0CztxY^}g=&ZMcBnXAJMP@88DchdSI#@h8w(%B-HnrcwXpc8 z=dNVOz2mfP*o6kxb<phvBvp8zv8*aT&J{8L$92z)%(<&j+ng1%eI)Wjoh-vxtY4HO`h?ksGEbgwWXja`nzfDo$%r9oE@w88FAQ3mjqq+ z(Q*6s>(XOIwoe=5-`YGf+|5&{zV^JF{OLQD_cgCPc;EOW#G7~T%Z8oLG^DSyy%b(C zTDbY_R?jCb_i~&X%{8a^iT~KHB;dYZbDC?$?9}JIO%LAjduH%eWX|9`_27Hw-5pV5 zR^O@Ssd`E<7Mje<*+?kx5$I&imVn}0If z#K<@2t6nR}uU^dUsv@rx_h5H!f%Av4wX2HPS+|;MymarF-y=Ff_-oTwp7t5m0-DcT zzh-heX>%8=`n+uUAR;~-cxv6zLu*^6R@D}6uR3!p_0w4{8!PbzaW>Z4%PluXA85CH zew@eWzRluWp**Knj~d@=!6CCPW_GJ`yra-(+h*JD^tWN&H_IE0RBCTo`*rTQcg4-h z#;f|m(vF-%j?~|&c!?udO4Oi*qjdXdx^-Q|Ryb1QAue2PK9?aRl^H;K;$wJ$w zriCxZr|rqJsBPJCliTd|gqD6e_Cce9oQHE`KZUlA)}Qlo_D|)F3df%Em_KvBY%AO_ z{@IOpYgf%Z>88-#73O-6;vd{UlcLyv?uquZbxFq zZ0I>3(eF1(?^G{oS-x~-qIPA>=f++2+`gaZ7;Fl7)NJ1DUb@caJ?F;uqPvgohB_^9 zl!zMY2>PbCx=Uv7QNG7<$?|Xd_x+F=5O_S^Zt0|i87XStGgHelHQ$xrT*&42V7=eF zANyXJTk0tY?Y_X)mm)Ll6FBkQ%%rEwHMXSaA3gcw%j69I=QVMqp@W|bN`r%**W^8& zcH_H*)(jgosgX+f`J);BhK={PB;~BMreczh>)Iul)5TF_+S-_%em}nKz)rWbCqG;n zZW`^fxYY2G@O18V@~x747L7jPlydFqg1d1suZB16`8L44Zdu&E-I1Fj-*}%*; znY}%6^u--{M^n~zDeT{OGn2!?YGRH5j;`k5`4UC4oA>G72%6{8ddMPfCA*+mo$}=8 z2iiSGog1*v`fgF$YyVTIaNEL(Mq9cH;^vkpcipoP_P+X~IKo~=WQopDmf+4Kx-Q8H zN(U+@Uy@%GzExgi##o;%S7jeGZ9Vkl))m40-1_fo!p}5c^0xgf-oG_;Rad#lrW1Ds z)<~K=&k+;9Jadp!>KBg^W9gD5nv(KVR7DrZnN+kycUMVwXJ<)OHDy(0m6@{@V2kND zG@n6|`k<%j|C^ZO)Sr<@;v2zqCT0wA6^f(8j47te%fFXhmv>?u2Ni8RYLp(Y{7z0i zUKMtSF}(5=p`v(G61u$HOUF`SzX`$~Odx}?D9`;&?Z4MjNB}yj-a_B{oqnA~0aj%hz{9px<9k2^fj?N-qd{ zyTu=$%i5u;aHORvP}`_ab44`w^rP;-b_Cb9@0r9L+bwT$pNCC?SsK#A2=h z9NB~BYGaOLKp}a$J>E1u=>2&tA5PGaL_vnC}G|W<(fqBr!fBvy;DG! zw}M7Dhv3<;znI_`pk9^}Tpl`v4Z#P1dlEbW{IG@KE~t}HDT?5_;C~$J6_>Y%CC;4d zjy5qDflhFV(6bBvZh3>?n&K=)H3Xjw{=sY3VEq^2)~1t?zYY4o5L^chLR3O?vQff% zRzkT13H}Yvmn1k|b6c6<<Ujh#x^yEN0+(Gd3 z;I{;VYeD-wM(`9EAI=cG8~lUUD!}%%gFSeS56nZ+29u67b!1`Tdh!tWwtd$8VQLLRTlhGPI@ z`R6P(jH`q^j^R>4@b_@vQcv&&aJ_s%a1pppbrJkB^v6MhcR)RI!hI{Y^9}e-kl;a3 zFA@aT0RPV*csaBud|!<90MuOwJi%wdeiFfdf*-R8E(rT~3GNN;q?X{bp}#Z`9LIjb&+FLFcumAsLLRS;*-mi0 zR(v4~}h# z^9_J@h_64GPlEGT5PC*|9(RHZpcAR&O>kML-%SKJgK;vL;CSu5aDtDA`<7h}w!@elN55o_6vj~pYv{xfIj-|VR;P%kJbO>G!{$EP)rO@w8 z369r}wIp~s)QcCv??C&f$Dl*5ya?@f3nBjo?8Iv>qr*r(6zT=9ZGibgus@d2(+~Nk z6FdaQc^u;t>yd|gxk$*D!9G2f9kM3_^w$#dX>eU=Ah;v6lV=21hj!IU@UL+F?I5@r zjAMNS4+a0@1-sB;q*mp?(cW9qFguE)$%W{IV zL%D1SJ{IisB)B-V2OLWt+n)sELlhzZ3-%8ZoCmHKX#_t5{=Y=_%`Uc>W3?coy&-1kZ=|yr1B!!7nKU{|4jsd4dwCKeJ1#H7*FsYX>fgcLwn-||6$Gp&r{+A{{-d5F@3T8dT4L+3Hf3e4-E)D z0R7IA;8oBboCxj+`T7xj8nnYOf?Gqm;s~w_?eI9k{lIUT1m6$szkuM=!M_y*uYz{_ zkl^`XXDh)Spk8qM#s2Jv>+%mmo)_+Oa125$9|Pn3L}*`_F93UF34RTpTjvrS$41a4 z_-oK_MsP5PVMp*%@Us`e@35LDBZT0G;eG`_cVhc-42#2r{3IBkPZQh^+D{(AZ$N#O z5xfU@Ey4MKKPUJlupj>pf$cPfabu8>$1%c2<9egcN2-6oeX=mYRpERog5#K~Dg+mT zaYmcqS@7J7@2|0)qTru(guDW@6C8^j%j1~mL4^Ei&=W~;U#OP@1m6e!F_qv?p#5JU z_2BJLH4?U<%hAeuBrt^%TeG!t$Kp z&l!aLR;U+Eg3E#53<-V?>{(6lN6=1O2yPAf{RwUk<8TDQ&p`S25nKhzn?!JLXy@4k zZ-M$PB={Kc&mDsEfc*^wp9SMU8^J$-o{t0%fOhhe;8`$kVC6p2=C6OufbN0P!=X};D z&+~nr-=Dwzl6%hkti9IS^EvmPa~5rvD)4_4)HmfbVZN0?TnOq@74cA*m$eXo349Ua zyP*D8AzldM$PRH8Sbsecr`L@Eh|_noMQ(e~N``yz;Uk!FABR&xNjV|H>^!qx* zQ(#>)L;N1JuO;Fk&~IH3zXAKch|_jw#}Xv##dm0zL?lmtpUpyi9*oCJhz|x{intV< zUsWS64)y;6@j%iLOnQg70MzqO#C2dj9RTw^ZO3cS4@V%b0sW1}jHmLZP@jrO-X7-3 zxro#A>P3jt>&F#{AA)($7IAH8Uk}7-Oq^|qheJP!K%9=#Lx?Yd{(lm2V;EoO5U1aV z79w5;{rL{!yw>7yo*+IO&P$sS_kwldBjPg9fB2vsXuEX({)PAvFgzOZxxi%*cY^+= zjQCRMS2QLiwM%~oHA3=&(681YUJvVqBjQm|&l?dhgmD*ycn$co2k~Ige;n~#sK--? z+rl`Zu?=atZi1dtBu`^%JVM+R>az*)-7x=uKwJ>~;e&BT?XHFXFM>Gz9f}F-c85vRXfEkt}H*fm4^3)F)(;(oABxFb$um~KP-9jxP#i1&xr*TZxJ5=``;0#=K})JPiZ^GK)p#I{uRcB4C4J^ z9*{?T8NE(K{4wxZh_8h4Yl8T4nBP_+eh%8v0`W-DV~e;u@b!pS!g}qFxGwbbO^AIe}>AXVQYYOxe8rzWayU<>JVSS@~BJ|G* zhzr2J9O7T#d|MasSx_!>#P>rzY)1S7l#3l>kjw*a;O9vsUkKwaAMs{b9}9t#?gU+s zJCoJ3Q}7vRItxq4(p`@26?GXpFuKEP0~m4fONt zEKc;;fIl}8Hw9kC;zZATsMIPJCwj;`s!4i`cmnW7;AEGm2w?t5YGcXMK1BF;7AN{k zK@a^sjh43rxB#qsls^ML5OMM@XOe~?J^<=@B;tL6OCT-fv*Bi?Ye{AO^62pk48KScsk;DNTo69++X-Z;I#g&!Jit$ zX^f-ih|}jFnh2o9Pv;@N6V_9h*13-fvDYhu7H$rDK0@?E zRxTIk{?BV)b~`GtIMqpz5Q`J}0N97uJaPLETmNJv54)ThEY8+10i5_xY#G7+M8v~k zUl#GluuuCf)gwy^&Lkz?2lu`aqMx=S9p6-*KF6Yo&VNA&a*6OZtoV&W0rQ%pR< z>3X!1*(QnlGY=}&oQJ!)9&l^KX}@$pd@aatK-?C%2jV@O$86cdl|V9*nY^z;-H zkI45F6OZt8&=ZaHF1z8e>9xDa!vl8XNn0SoVXxII0<2j`yf6 zPl6Yu?El|@WlS5AA*dNTuV*BL#=V z3N$|zfKO*Qp_M_L+NGZ}AwLr1#L^RDbb#igSdRf@UV&<&m~#;F)A3IaQ2*(^CD_OL zZ$bWeDhQuAKY4zS1KT;lSn`ChiC;xLe_uwFDj-L8InbYvv zMe?ahlK(jz17d)c&@Oj<>M6f~1uxTjlmYF+yHad7xXHUtBcX`?rt6rWg84H*yZiMdA{P>V_{9(+%~K63`1jyMig4LFhm!8b0TXLAbV*fB}<% z2}=|{R7yAW8t)JjC`eQ{w0^agxt0YxW)jO6vZ=LPYZ-}yWJWBh+31G%mX=|#Zm71D z3=@QiiMS-hghwI{_hB6NA}rigO2$Ta%22MLNQgGm`iwC(UT7m?u8128X(QwFcP4~S z1JfmVF6o8d)C(X^BGC}Z01j&7)_lz?uyH)HP+l%2_#7qm%+kN7I!oe|EJv=iwl)D5jR2yOmS z(NR|;lz8|7ZjtVL-4EZ%EmEQPEhy4X@=j{M0#X`Y zMiN0PgPRNXXX%J#PF)e%Tb&|<45S3?L}Z}#!-F<4D?v~fpORGLxG^iz@G?>~S4W+FxDhQ4Kb z7w_N&rdc?zUor$!HyIHy#PqlU-QBIfVj;xJ z3i21=^L}rl8~TAeB0{%P&zPnbBK0FMbEgbb@B7XD2cqbQC{#4+*BwzeR${HxA*)yO>U=8~T&!(?l81-xN(cBg_jO=H;U^^M{{@*6$lZv-rKA1^F^1t6^MYqGN&15KuK6{xM7;GI}{kNZEBbjnAm!GIl9=3d)T@;J6c&KR({ku5Ad?sOoGmgbLRozIBZxU}9wA6Ri?A%>Xsvq9p zpI)4!X_RiYC%rf+$Mi&^)$1Qu>g&r51x$@Tl)Tycwl@DuMY-$SpY5-T^oy9NQe|$x z`j$?#IofnLFZ=zTxJl!yXFOK*wB7bDx%^7pKFRU(vpx5(skiBEtC5lJHo2l?&=J4K zD;;(WyLsSQ;DAGkg-_FF)&6|*bkTw>-|MHDx3<@>JpB0TdF{_TS5-@$m$BdSqmPe< z>L82ex6NiFwl!{*vb|dN?aQ~R_M8ieQ6FAe%_uBgm$Fs#IRDZ{4b8)9eM}qg%M}Hu z%fzIg%uXF8tGPJABXyPRf%K%#)Ge|O!`nU8UE-amS2m^YkyTLq)nppslsMqd_L8`} zcCxlNkHp;_F1ulYyw}{Ti-sMk)=IgvI@M#k$HS#llnW!?c+Gv9I>^o~>qCBE`biZN z$+Y%|GEV)JpOjxM`C6+J*`zir_5I0Dfe-3kti`&D6pk4`d6IqCSAVs8zo8;8`nH&C z87ikE)HY#qjQyIa*ThFGk-zqG=E!#mH7KbYQ- z*>hii`qgBGg$tuC?0mLp#aOu5Z=ZPedPVQW{ntYl%^%p?DO5Iow^&5q)+wh8{P*n9=aC~L{?Opa4(X`cy$`}fXEZ%~rs?XMd~TXIZ zZ;m(FofP$*6utOOFR@Xdr)%!oG<@CBnyVZpavERX$e**9Eb!t)h<{(MsnPg|Q&jp) zXrT0GPK4a|FNI=_Ist)pS9=>seeM;Z{(acGQI-7_dbexz8ZqQM=VsF7@dp=Q8s-1s zYw9F(ezBvUwF0g=Ugc9zYoE^vdN6k&vvHoUZ^}>M!iJnE&OpQOYsqmD1}8Dce@$Yr zM)`r|Vb>hT_hWLGTw(T$iWSw`V+z}DG;<`Y=JvT%+*GNMAQ`K7*{Dydwdv{5-9|6N zI}IMI8Wi@LDj}fVf3)t93GsVnIO)3-mMqo|93|1$U_zs&+){yugY@E$$@ChvYv?`Q zKuKD3ITYN)=UBKVY(V&khH|Jb9=sQpaAqlfCbf$ipT zuUB3>cRqB3O5V(xS5AWGdQJ0{%RKx;di6cejsd|zZ}Zz1Uz@hM`dgvSvy~z)>ef-K zJWb+@PmR>?_Tq*hob99m`B9<#ynTmC=6Pn8Zy8Zgep^Z}A%EFQ&0d32OZMEB@!sr_I%J?(y6XaYpDm}B-C30B zW14jMZklc`=hVaW;q$!HrzC8tl888~RV6mM`QxM1RmKIyuPyVP923T@U$%Bg+ly2k zZ@Gu*1;S1yYopfJ#INeRVcDtRyo|%~W7Ey<$yYh0&iCfeSy**6c8_;@b>oTytqWSB zTTfc$CrYNTZtRpFV^Vi`?}cBwRbk&Z&-*l|CAoE@ytQJh@M-JxyyU4q`_spUrN1qb zI=rk-#4NpfzFE4^lHAt7cYMuB3qFa@(8=w;a>=RI3G0t|`gy0{S#sG)@L~Luu%)W$ zdBe&^yPayS)Lvh{`E7c;R@&ptEo1u64{zqvOzQT2UEOsm8Al`{{U{VqJ4D|>Q=T)OqGiL1e+nd%!; zvkLaijMAyo4jI@MmU_g=)$4O+(hh%X*MZA#UkneCuAMa@<5Z$iuTs8B)%UvPYfoPP zxp%Lf+o1tpzQ~z8=euGnJ6qJE-$B!z8QYcmD@F;64Cu@6(mH0Gv+b4rAv>iu-mL9C zbN0}6hGWLPS`<`Nw8*5t>Ib2y!a;Udo;$7EV>xF~Rk8XS+4*0lRj-LJ8WuRq`t9P) zN+%b2t54axKd*9CveefP!M^h)nl9YE)NhZz`NrbxeG8J$cce@DT^KjSkI&pYd3<5U z?vG#AdM6KPFk868ZSuq&&Fy-+`voEjM|7(DUkx*zf6h!YUpM1s!-^oMsX4mJbIT+4 z9$r$rFFJPb;T2QEEzY%GicQr~mX;2;%V_;^A$75`T!iRyKcOe7={nnvn2nhC)NDk; zE8!}yF;7nv^nPBSP%!jK(l_;oY3V@)rqko+rysgEsA_716X|6$_ocsjt!m_S z#s5?C>3bc=9-jYLHYr#0s-Ll7UDokxPes4yiIv;aT(;^I9n>B=sL(3+o5R)^lT(R~ z*Q_L0wPfwr-6vhE5HXQ&hk1^`x*Q!74*%4{eVx9}({3F6(7tT7qz>O}v8z?O4h_rW zOm;{I-F$sE`p}QWX(P|BXjnII+6DDxmt@m|qV{#!9b7T_)u0N6v)R2*ACoZ}qfo0S zy=;%>aP5!1y0#R{mf6Ud^{v;K99CcVHuj~i$xLPAJ;fo%o%dOlDn%L(9jNr-rD~47 z;goUquWk*Q)iAtSS2{^kCG?k?*BW6<$@b=3oPkpgX9?9zI=3Vv_0}ora~YMRBku|> zigB2lFIMH0eAwpTRJnqYCq5a=W|{An{>CsR%jgHRFQL4zuWomdWP_>gO*$cdd`3WAp6Eac#;bUeY_>N5=Y&z87)h*wU>v zQ!mI@wdco54e1=~;&=PQf?fW7;5P^*X!-GOeyGT|EXN4vxl=0(R(EbWdPS}y^U=lAy&oOPeR6q5M{4=H(S8f< z4@yq&*EH(*oVK*i@&ZGR*18gVF}szjTYRQBj%zh}Ie*W1rSD$rx0$9r^*_1hWJ|by zUG?-PrKD-)u|93PqdX#i_kF)`(dO8!ygcM>mF9Sh`m)zS5d){Nv}qrcN89 zefe>Aeeb^&x{>eGlv023Y2OW}qMhrPj~n;m>dcs}Ef#I!_pbP@{k^j0@mYEGyFQzK zSQYE0lsY6`-M4SB>A2(WJeSqhz0*%O7mrwi7jdU4%yY1}q zDOoq3Muhfq{cLb^w`y3?zWW#794hlzy!lw#u$xK-)oB@#wg;2@-Fj}jsOi-6w{x;b zAE-0?QR*4*JW=0KaP>j8wW|--h|9&lUH&d+b=LaR-|`fNQa%-2>vBqa^l&)tkXD$k|N^2uNcq@vn6}vgp<9zMa zBcC4B%6yUjak^tk7r(~b50Z~YL^j_XWAN_0W6PbN+g63&teCM#cR|jH!FRtHtlX8n z^t)`G>Z)|=W--qDKoURooLetKsoqQ5`Q1e!+;#wWS zy;`xY?ow{?3itQDvmMnda-{Yshlx|p>{FAwo#W}2)Vr7Z>QObO?|yu#J3eDGpJ!gg zwDb)dnwh9W`@aPmS}r=LvMXC!GV$Te&c}*ry&hD1XbwA`8&|aZvCQNghosNvJv(mC zx_MY5$EBlM=c-$(i?MmCy=&hgFE0BO9{U!h_&vk@T)y`ASu^hcZt=eT;oZ%9au(;D z-5Z?qdhJYiZWce;^mf6=Ejw%)cJ=SNpBSWh?onN9MbU6&x!;fGA2{KdJ>bJG-J^gmkLm@C&g|Ba$v^CTgtwDR6O4 z-RQvS7B9Q6Kk2*E{oBko#ZE zlk~2IY)=Z-oagjJZHD%W%$QLApi6-#Vlsk)g2KNEMkp1iolz9Kd-1?r^Y7;KcRu@G z*Ri$U=k3h%Kb=PHa9lHei{Ie($QctGZihP7cx+kVAK)uxUp+E&x!01%)@PPK()uYD zvHQ7J$v8vKmvlZoHKo^IHrDtCUYn=WZ{J!bBxmNWRcntw{BkYHZ%$#pSi_C;Vb=^OjTz8pDfUPi5`T;N_UsRH%g;pKbA{N~Fn zH91-oRpBmk#%|Z9J6_kO-u6(b|1#J{e_>f%v90nX(~pBTP2AQvZG=v~q~p7}JNjkb z?&vC(_IC+OeK>L0lT43Cg{$4Z2D>g%?{oL#>A`aavMjnpCDrQOa@F7V4T|KCxg%1p zdQ$z3SyPLw`pf+zV^^9vEq^cPwb@-_Q^x+TqL8$Js-i2eE~-Zg3eQbySD9DEf9ZKZ z^XeA~Rr}>*RS&7woA|XmsXh+wC~2R({p9|yHxI1;Wx8@!aJAI^$qC6KYIPC=;|za( z*mm()m)tFt!?LeU^^`kw(?6&D8of#G%C2!E^kkMwJnQp3D0*MQ6TK~}`HLg-GM?ON z%_!4|6?Hl0a%Z{DhtOVX?x(U%C$E2=ma}V$Os)Qyig`;{&AN1Fw?=mA!r@65{Egcz zByy#{cPZx0LGoU^E24K&^d@ZBv)y*efTcGSl*bxYW<8X={eJx2 zu=M@BuSL51WdPdWbTs2n?819Y?81o(37vwKFF%Pb{d(4+bIM32ieaZ5zb5U+S^vtv z`c7eC;qAg}g`ATK0U?2*A#qVUH9^HC$EBtCR&I|A`Po}CyHI1n{u2fwhoVDP`5C&~ZxOgrh5Et7 zJ~WwQFm$Qtv(2to-8;HWmQT}^7$X%n^sQq;==FpX<_4(|B~g)wAFN9%wS2fESZ3%% zi5oSg(mC_2>}L<2VsJ)PH#PF?^gV{7?VpYF$-Xt*)$WR_>nq>-OB=Q~UlY7!bgX$$ z-d07=^1%yE#c3R-w@QHViYmK?ShOg<| zpz<`^`4VMsCtjUn^(5NlY2<=7-Kc;!*96)>e~q1~)#vefjZ0b|YeFVZ@RkbN*zUNg zDo$zk?xRmqZmo@$8Sivxul1DhCfTV`{nyT0c1@@Mff-Xnrw7fwCf9F{{5AbkHCH^9 zH}s!b8nwO1xJ%|+?w#4+`t{AdyLMt;TcSyM<37=?9RfE;xHk-z$eq6H?V=e&C+7Fj zZyag7_eODVjkBwjx9n5DXl}pU>Am)u$X5qb?p{k>zq&E&hq3Y^liIE0Dg$mv?sjb! zt~=H7Bv+>SO|Dh+_~Ma)KP5bDyB@3=q+(;V_xYUptCL=qx$&V~ zdX#x#ihS-u6G7dq{D+ARB5unQp|%JZ*|4(Ue~TB z_fAUP^!ii&mJ2PxH)sr%Svt$zPv_ZwmE7Io zKc+|^CwzZq!9B;j2W%paYPANn9lXC`Y+=pz?|r;pR`2fJXP^3zq059m*M3|y-{kAn zff~;?H=f;e+NRP`=Enn7>-+~BJ!XC$SpTGUVUg=^(_=9=ENd3aFaK)Ter4Emha;1x zoc7z>lx?1OHfHG$CD#DMqK#+$ObzY4>;&JI{1QAg#?7DhQ)TiaC95BAI_k&7ntZA@ zxHzS&-|XDd3g@cXFSYi(UNLN?$4X;)Bbi;n+Nbpb76t6TY@v}dnxj{0n9f{s2;MvJ zAjN3DdH$qHm-iWNmMeQU=Hl|_UmLZ|a^-_Fer25)QISy*e!IPmHXYHdvI)h+GXr&sM;{>yCKQ~%D$W1?~X)i-A)SjbL0n_V(wgS}__QRm#r zvYxS<(>_cZG<5)Tv(}lP=h8#6M#&kSP}zHD`#$FtF1{~9)?_}vb8i0%f$v+)o+eE_ zRpehkJj0-3f7z$4d22J5cl!Iql)fpcb3XrA{)%nq2bW*UPvw_cT#q?YxAoWt70X%U z=bApP9=7aMW~h7i#GBXNSEmT)-MZ|tYWDI4Wri28d_Hq3Ub)LSDMJ1Fv%D|a9ZrsK z^u~xfzjm%~^7^gw&2Xeoyw2pgt8P7!7&FQ2(cU)=8AE6I#YdQ)Hk&)mJvXUltn(uo zu^&%b*X7Qi`m*wg#18qIDWAeCd{>nD?6`2Y-D26rjIZ_EEd09KqC~d+XlpH4y4KuZ zN#*>4=SMmsf0@eq-1qkL_PbPIzU@-M#w%U1FXGyQI&!Y~SLdG}+YltEpt`MK!R|R0 zHOq|GM<#rJwhrR@# zFK~?r7;89(FCwDfp3h&ibWShTn2?e#qh9II^v!TYtby5(8C`a@uG6oUxopg7cJ?kR z6F;C+6>{-Qc-1@?`y*M`M^+Ww-{LqlAUa^4?f@eRO%J~*2g2{Dj4~Ov=Cp77*6S-S z3^MP!;{W6DoZIbpcZ$BOTQKya!`4?J%f1W`xuLnoD@$ojuY{NH2F5SBzRj)hyUs?9 z?-S1ZM77l&xVW=!c<<<^_L0|5Mu&WSIAB(F`m9#b5?zk+wcay_={lNex5fRi$nx!~ zSP&BHZmXjp`fjN+|Lfoz20v;tzSOmBu}%%xKI!@6359EXN(^(QJAIP3HyzfpT@t(C zLA`y3=f}mpCW@Ff|A_JbY5P)ME^4yD0?(JbPVt>uXVaLJm$4*3fU|Q^UfwYNSMv(L z=ZQBqpC7#Tyw+pFxz z>VWn6$Le+k@jn{YzvGMBd%LrT{LJlC6VIOPrMP74_6Y{N;_UJw1Fd#GP!_fJ)(%{o zuyK~Cb<O+H0%#eUR`!5tfv7t)J+^t}W)?=RZ6ZniSh=o>MSupri2Ik~zWi z(nr5`^3@-<_x6Qc?Yq{)3>?A_@%tZ>8|EQ;(Z%(oaGwiC3ax6AwFWciCh(b_^bv^Y z^|H&~AuHX{*3xLf{(_DM$9=!|R<5hQW29hVem^nStazsIm~)AhDG8-b;%=*F)#P0i92tq+AiQd$X*pxHaR4{{cQuo7u!Qk9~Ld8#za5)%sb}hB^G4X*w!b zGuB)7gyEwGlf)X`UGlP=Fv-@p`K?<;q>?9UmTGLwb6py;BSO|HWZ`Q*3(3!=F0#6l zkKD@>$yML+`I+O;b!`^~-@RFO#Cb|ZzPG~qGiqn_J*EX-lz!cL_`#_W(~TFcudBIK ztTkVH(A{BK@jqpwq(=4^?0hbr$ApQMoWFaf<%kCTms2Jk%W>gTn(<>#+NQ;iI*#0q zNpBp%e=H_?OVOy_wu8s;&rN?fFG9U|${>X;b}9MWU$64ec=;w?cf_R~$8Ij0o@yJ} zdaBgfS52+Genyg@{L6r)BaN3_s9&rvvuonWi|>+GNr$facqvFCO=aS);N=qA74`Wy z*S+0e)_>IKoSYj`*5(FH`pFZu11>&Z((Apqhjm3EbZ;Ri|7*0_Jg`vQC@PxO>T{|EPG#8eoJw;hmlUjoAM&QMK#MZCh3(eOtc=& zoJky8V^^PTJ5<4Oh3=kkzD?&d{5O|pA2~kcnufM=YN%psf#c+Stt`$C?>+Hq*EbKa z*)lgmTvgiPxO2S=eeBQdnPM_*cb(L{u`w&H>b0e8jC2(| z`+bt~*ib)y-t(V%)$zl=9Sff2{loLNOxiTz9npJ+Ohp};Z7UWJoMM|1Ci47+byt^9SJ(8dKctg`!nO8iFP60SKfa;a zN40I5l}PQ;^y>$#f|b;wdhO@L3G^zR^!VudwaL53>a`num>zoRj*8J;rO!eJaoW-y z4i0MDv=6M4Tk>({F#m@a%XMN*cf1(zSnbXD`1W0WhMbQR5_Hx(tS2VDLo9BV_9>BL zam~Faef3qj>~Y^eE6dSzO?}jxVGl-T_}6H`ntPB^tSt(XB;efEkCco$jX*Xx|Uo*J9y|G`K`R+CA#GS&+Tf7F{dZV!AK`x)v zk-DEh-@Lcqdwbus7Zdai)b2(YD3q(-KX5_lLwMS=_mRqrwOXRJf?=;->-Zlwxa0lZQZlYI+=cAKX%L=wKeIj;*7{M zt4|3eXa~t0E=%Y$`P##IE2I5$wjS+Nj||%=^(rwl>hQU~$`A6S_NR~Wnwwo&Jp5^~ zpUTL}7dc9OC54_o*lkug`QvQ=!04U3W*?6{DKpzzbo1JQ%hf7trL{g?@(6eW%4cx`bT{o8I~k|P57PE=E1M*RF92Xd^f1#d0>M}-u5=RhIbmFX zrM^1HUKMFNiimV>%vLiz|IezdA*# ztxhl3Lrke()dG`&=dRz=TllE)fqT)?K|9CKuu7UedPn8&2bOc9W>y|}9@{_RVeAX3 zq0<69H^-hA@00V`uJrsS=cwk_n+{q=%SIby9{M@T@ZQDgciTOF`b7&*T`KSCVfRe! zi%?6w-4_w-;~jMw0S`6^hR@5A8u>PN&?yh$s9`Ze`cfwh<-!iGvb;xWn@YVtR~>h{dl>#$u`5u~ z_=Dgi)rYMIn|7_3^zLrpcF9)ZX-WF2qvHlHI+pOL`1rRat7Y@e7WZ>@N-575tqagd z6zN)*>~$j1LSHzhsNi*eL(3Yq`3*&5wtI_h%>8}h;YeBaADgP*uPe0*u6gZYxT$0L z_HCR4Ki~Z9zCJbm*Y|j*mbM!*F&t|Jb1ly-{-JqEKX#(a`u$N>lWdz_>&;VLERZ~* z{lK&XABtz*ZP8lbW|E;Qp(q-D{;pzjN!&g49qU#OHqg>8F0m6=jvQ8zzgWTXdF*oj zdvkopZh3fZ^e3spyHm?AjYw$@n*HXmzUpv4b>$$V?ITW#TB(Q!nm@H^7_J%~CzkW8 z%=@D2`Szl116{`(d5L|k&N0Z`+P|&B#KvX1#i?JTJZH36&l9BL2kOq8DDiQ z2A@kcdo|F@F?Ii#Z85h}3%(Ulh(21v3Oo9039AM5(-wByIzU*B z0b0T`i~u9Z3l*u&+_WyvWIxLMk>pPPvk(>y>a|c<+-iWZs1^w)$+K5TjxkA|6!4Ed zogVFJ_Ikw_WL*EZL2_>_z3+G}QD`VEzFt_sNLZX#7cxXYlhVk4`otIS0m1@Wf;P@d@005$^b5A0m%t3}&@sARN?YKAGF={*_k!g**I(6Ma4D z*}-sjEKcsYWLVN)96|>fBP3M#0-F*^i22*pYa#2{}*ol7fzn_@5%0- zzwqOK;p7=pcC1hC+{Bdi?qB4ax$?|p1&KRPG28si{MeH}BmTmt{DqTu8}y{d;V*pa zUpRSYd2siW!W<5F4rKg1&E5q_wrxco&Q!Z35Lo+w%lCZFoH0yEH}Acthe3FcAfY^ zkUis*kE_RerW!0aGEq~x<+5>PV#``>_OkWlFqLZKy1~N6)tf0W$AVP2^?D0yhxHbA ze`2gM&r(>RnsT=EVw&RL1+;ecc7f{mBH4HiNHaUSy5Qn?I&QK3XM|UFEO|L*nmkqfl0-P!rt(?@ zGKp}alX7_ElFKDnJb>9I3EMr!!(1-UY*Rg_S)4p`M)@@sC%Pyn&jOG{7@I%mQXZSe zY*YCT7FS^LL43ReLmKOkd@<{QYvyeaU z1R0}c|ONuoRj zPQ^93^be;ouM7~s)RzZL5HEvwv%#Z<-TDXdPT)BkkbDAM8h9a2-?d8Hh5BhgIy;jB zxc~q0^BLIPjkqf`LJZ<$?2$wsEg^~OAJ?C^pTYa+<~K>_V$yluM>jVllMs`Ncpu%I zzT2@HaXL;K5myKQ-y$vt9poG0qU3>CCiR7JNBut!ooEQ+a!@XD#Ob@NWp`=Gzkb%)BkL4PCH2_#Ye9_n)&;`Ciak%*rmgOW){5P!hKoHWE~ z%%$^)_vPnhC`O#V`--l2)XzrfSI?2W0vVJ{>Ofo#^#4RWf}FfCiI$J*FNAW@d4h6U zo=Gr&baS%7BxS^vpg+tHrAAoV8 zi+DQdF++Sbd=+Db_#@Eoig+T7qpgUqg8mSScniqWcY)G+I0EgGgygGWL1S*a;wCEZ z&)el(Mtn8Y|82y{I!w|_#LvO}@E-9j7$?6F9|Ge-5Y`RqPc8TI_>joem3iAwocP`am1O5LnlK&0$ zkb*dkiIju53#{YU5U+-DSAlo}lp5zm44`+|5e_}K^S&~mMTaXJoh38-fo z#C1Wh65=LAA(Lh!o(t=y9^&ppdGhR7%cQBEL%FPx{4eN-Zix4R{^N(Z9h55!aTBPY z1BlOqc27n;2kPxC;vUc+t|0yp`fVBFW1(C$W*@Ep6HpILNS?-+>_nWtd+|5o#^C=z z=wDR-QLsA_abak$iHQ5ad^jEPBQUNt5r<~sEJj=j_E#b933}`i&*OCmju+y`p&bJe z4}^7pH{y*jj~zjL3C#bgh|57g&qdq{=8fx!--CL)i}+NqTZ_0o9jA!fK>K}0{5|O5 zhjByO@fEDE^gNVuIv?yZ7(61e*|G(q5LVduNdNopq`}>-v(R>@lViR8iYIr(TL~5`OkjDN5J`89OCOX+?Y(%;)bA-wEyh8SxoVZ$A*94E>59{G#P;g7zJS zxB<*F;)qAWJTnpT70@nI5jUahJK`^(z0?qY%WD>n2I3YVzYK9fs2_UXLCaMI#0BvaP|x0o(^&Emh$q50V#ib_^FTPX%RwYR5BdXLm#F`3&>z?_m5ClP zn9tcUl?gu&fICVV__c1&f$={uX*F_j6w3+1gsey#<(>{!S|-Us}r*X7jzaOgkm zSja@4#(-tVLMGe}&P)3c1x%uPF2i`Fv9l?sF;2%J`2|qkDTs#vS3~?Wa2>=OxLz=^ z01*EO--B5oei-VF9mANED~|lfq%BC^9{R&B#Ob@?>HI_v6BULqPtsV=l>dPI*+`EJ zoW~X-9ti6rJN7WqzZ1$;i{ytu{j?&kMB5i}OW5xVcE~Q#PoK9Liuf()KN5)716M%& zE%3RByMX@&i0^@UX%*tDpIv<*C0+~+`d9wAMAcaTpH@L7nG0sL+1-o#NEK3afsi9{x%hH8q-uA z@wd=V==Vv~E{%1!49SR_i#6zLIej&aK+GQa03tFEpFfWZlJPY<^5U+rBM+I>$ z@JAc*P*@jef24K=p}(y|@-()X3*yWa!A&$4Io0D1^&E!e<3au);%#u=a}x2daGsEZ zcnSq(;G*-zD#IHg>iA6jb^dutQ4&|lq$ffnu z4CC<%lK%nyzZ~%@=s!;or?Dbh5U20-{fxLLv`cUBi~2)jhz&-3AI$&ah^N51Bae7K ztf#XP7ld&~=WlA4#@JbbccX<#r^0652f)aSND-nOAvm6V)RN z^_7Wu7?igF@oX56w-EmY?Mq{OQT;S_=}RPk0{VF;;yYj-pg+n|JwsuB6Ndgyxg^Ms zL0li^ZCS*ZK>Mm8UI5qK^xei({|cy2`uhgu^TB@$q=&9s8xW`RzKC1FI0{94Bd?h_ zafoL^J7ywY0D3MUUI*j181YFkF76|q1oHKWx59k*25}M4^A+*gP|tl~eA9ZPu|tO- zo&@7X9B~>;nSMV<^*BO#)segc^xK7q`@s2zDdGaKZ;7}el*l@Mx4f^y@dD`sP|IDX}eb=PW#~t#HT}h(fOGAF9hZ45Az)5 zx*$Is@lYbbBuT^@;Cw~_aT@b=4&uY1{`C>x4E4Vp@l4<}1}OD27Ul;}B!3_;I!1KKMN@!!zjG7zV+Lh1TI{Yiszl_2>TXvZqVM?$&SF@wpJNzZ%UBl&F5{~Pgj zFn)z#9;N=2!F)a%@xjpUlM&a3dQ(N5ejliV_#&tu6U4{D{A`JMAoPD%#I=C?A#M)- zha+AG{hJ*dnAF=1u$zM9Uqk)V*s`>K29cZlm{f%1&k_M9RU&>8^w%RU1@lik;%|U| zN1U!_{h@r+A0Jp}$0AOD@0pDF0_guLh@XY>Y9l@r+cN2>3XV#I9*Q-5nlr1Vm0D)TsR<3&j)-F zpAP+GFXH-8FG+|;Lcc9QoL&c3Bfbx=m)S9TNfkebd1(Zk7t?y)2>B%uUk!F=B3=RW zqdDUAy2}Z1bzZmP1R}l}#`|8xTfiT7EL-CLGmy_m@=DOJ9wROQ{pTyxB@B4{e zQU&5%%ES<5=^-4dhePi-q<#iNxoFHIEz7kIFvu|SgNjSaV@ipR;LC;&{ zza&x4q<(OHK*UJ7mJ!ILvA}6P(DMKqla5VrzOW31A8 zTLs*LB~R*YHe9DWvN*9zzkl0+co69E08Vy^|BB2XN%VVh%AI(abCRWpYZvMTSe)qZ2mOH^!j%pXa!S)BMo`_BRvC;Cr;9(}|!fE%$m(X)X#%%qhpPV`&? z`L&4C??bGBlV&CTCz$yoDFE?A9_GX$o(=pW;>ExpBTnamkBHNGj}O)*>W>cS5k`D1 z@X?6Z0-ug}6Yxcde*kWcIEf8Kk~?rxy`-KUU|re5l7~}IBF;I)k|*^~V?D+pUJC7! zj(C4KUpGv6= zx=20H=M!GBD4Dtzx z)AN=Kh=+sxJ;V;4EmmZWWn8iuCHi7&u#5V%p zgZNJ1`w*wUw;l#g>p33!Lki;L`EQbP5Pt-`8u72dUm)HO=B4k5TLB*n51x_i#Pty1 zN{G(_{d$Nm0=^7!d*B-pr_XilMw~v^k%Tyj5=PQp#OeD5>VT8}M&`+Ta9;8QaXJt8 z8&9LIuEbup-5okAo!2gL%80%ZMief5_r6M7c-k^JPR2 zo2!#f!z9YZ$p(|?b0L(|b-@P7Tkym=n-KQ^eh~2x;OGCs8xbD`{&XQu&+Ga_VW^+< ze2M3KpNi-XJD5*Po4xBP{RLMJBX*m~3ur}O8E zzi`jL@R+~wT*PVrq+@~18?cme%O%SFpVxobeu}d=6p>51uV0xxFvaDX-2eadbJ*>E zzZj_p_B^nTCC{E`U^?UWXX4KUkekSoC;qH}eO1J1|ItU>7216*i<3Ip0Q;^i&aQ_o zEY7Zn5Ef_GLlleCIw$Bbi?iz?nZ?=lAOW1T6=@?~*r)YCc`$HU_@w+k?8_rA3H6|a z_(tILx`gUU1FnhWzXR7noSrA^BR&MiqY>g`fSV#N4SXfya=^_Ip8?z&aZTV3h|}2Z zwEby$mw-H-|0t*5kNF@yHXy$haaZ7hi2DQIfjC`1==CVIy9eZ>k$f`neTe4*r~QrU zq2I$CNAi^*Px~R2uK}Kht{|RUN z6VL3EL^#`@8(hlcY=1<~#sc|~T|*4a{?2+4C_JH>jv)IZ-3)}tVO0~PB!nivD3;p4iuK>VZi;7t`?4tuM;AL+rn+pcyzMP4t*kt3H-au6a(1Sr-p#|+&DO(fGZ?dVu(0#6++a(3%l~^Km&15$5UrZzzaA2F)9xb^ zm@UHCV}cyiB>(l0aQ;b*i~#fZf6YIY$xC$7b&=WRCQbNck1>L%efmk{$TpT+!Z4Bo zc^}>XlQo^4W-&Wt+-tHS*$0Q#0heMv$vi;HN)r4$#=UOr$$kS@3f}Ju+vEe=w_#XM z_T$06D495zMC_A!1>4`lu%7I@L4&w6$Jq(nk6;e=Wd98OpsfIziG4DEVfz;u)|35r zV4q%hf|`GoKaV-kll=y;FHV#*iP$H%pke!s4C~3hI855~yps{*CS3kHhW2Eieo@RW zKRHLn_W2p(#6Ir70brl5vs~k>{7@{;2*{87uO#W)&&vHuS22dMLM68nRhKWu-=U+kyCeJJ$%;=kD^-%~LE z{wM~)fML81dL9S3+dkbV??LUU{@h^T)IdKb{u7nh|4o0fF9s8YCN%_~G(W|jf3dIY z&67w4L1Lf$G)VK)`6uEp_UV1+UXY)D(EN0N=U?o<1N#EZagvCA^3^oWPwglC#r{1I zxB&AP(LoZ;&)Z|Zhy9EFu8lkicKwsJj^?NK$$baJMFn~Q6z?L!Oll^R1}jl9l82-V zf64Dg7B(i)@5gw1f9%uzw`hJU0H1j>ybbv1K1%b^&q~POEdU$Oa7N%XAC-U30J1Ki zYUVURemZ|qc>%fspX7HJq7&yAoy-$hfehmOADQDljh}mP0Ur%^NdJ*w{%C&Mem&VH zSsY}^{~S&iQO+c3=5ILjhx3y)37>l~h_h}Q`JcI84iyX)hx=~Q9`pP1^3(TU^7de^ Jp*zgye*s{hI)?xN literal 0 HcmV?d00001 diff --git a/taskvine/src/manager/taskvine.h b/taskvine/src/manager/taskvine.h index ff75a8f6d2..af5f9faec5 100644 --- a/taskvine/src/manager/taskvine.h +++ b/taskvine/src/manager/taskvine.h @@ -173,7 +173,7 @@ typedef enum { VINE_TASK_PRIORITY_MODE_LIFO, /**< Last in, first out priority */ VINE_TASK_PRIORITY_MODE_LARGEST_INPUT_FIRST, /**< Prioritize tasks with larger inputs first */ VINE_TASK_PRIORITY_MODE_LARGEST_STORAGE_FOOTPRINT_FIRST /**< Prioritize tasks with larger storage footprint first */ -} vine_task_node_priority_mode_t; +} vine_task_priority_mode_t; /** Statistics describing a manager. */ struct vine_stats { @@ -1613,14 +1613,12 @@ void vine_task_graph_add_dependency(struct vine_task_graph *tg, const char *pare @param node_key Reference to the node key. @param staging_dir Reference to the staging directory. @param prune_depth Reference to the prune depth. -@param priority_mode Reference to the priority mode. @return A new node object. */ struct vine_task_node *vine_task_graph_add_node(struct vine_task_graph *tg, const char *node_key, const char *staging_dir, - int prune_depth, - vine_task_node_priority_mode_t priority_mode); + int prune_depth); /** Set the replica placement policy (which worker do we prefer to place the redundant file replicas). @param q Reference to the current manager object. @@ -1628,6 +1626,12 @@ struct vine_task_node *vine_task_graph_add_node(struct vine_task_graph *tg, */ void vine_set_replica_placement_policy(struct vine_manager *q, vine_replica_placement_policy_t policy); +/** Set the task priority mode for the task graph. +@param tg Reference to the task graph object. +@param priority_mode Reference to the priority mode. +*/ +void vine_task_graph_set_task_priority_mode(struct vine_task_graph *tg, vine_task_priority_mode_t priority_mode); + /** Set the type of the node-output file. @param tg Reference to the task graph object. @param node_key Reference to the node key. diff --git a/taskvine/src/manager/vine_manager.c b/taskvine/src/manager/vine_manager.c index 3ff5965403..9ecde5a14a 100644 --- a/taskvine/src/manager/vine_manager.c +++ b/taskvine/src/manager/vine_manager.c @@ -6132,10 +6132,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); diff --git a/taskvine/src/manager/vine_temp.c b/taskvine/src/manager/vine_temp.c index 12be13a26c..f6663d0989 100644 --- a/taskvine/src/manager/vine_temp.c +++ b/taskvine/src/manager/vine_temp.c @@ -233,14 +233,15 @@ void vine_set_replica_placement_policy(struct vine_manager *q, vine_replica_plac switch (policy) { case VINE_REPLICA_PLACEMENT_POLICY_RANDOM: - debug(D_VINE, "Setting replica placement policy to RANDOM"); + debug(D_VINE | D_NOTICE, "Setting replica placement policy to RANDOM"); q->replica_placement_policy = policy; break; case VINE_REPLICA_PLACEMENT_POLICY_DISK_LOAD: - debug(D_VINE, "Setting replica placement policy to DISK_LOAD"); + debug(D_VINE | D_NOTICE, "Setting replica placement policy to DISK_LOAD"); q->replica_placement_policy = policy; break; case VINE_REPLICA_PLACEMENT_POLICY_TRANSFER_LOAD: + debug(D_VINE | D_NOTICE, "Setting replica placement policy to TRANSFER_LOAD"); q->replica_placement_policy = policy; break; default: From 38c7cff0f2c7956f291e7da97c94311e1de7aa9a Mon Sep 17 00:00:00 2001 From: Jin Zhou Date: Thu, 16 Oct 2025 02:35:30 -0400 Subject: [PATCH 036/113] new interface --- taskvine/src/graph/dagvine/graph_executor.py | 110 ++++++++----------- taskvine/src/graph/dagvine/params.py | 39 ++++++- 2 files changed, 85 insertions(+), 64 deletions(-) diff --git a/taskvine/src/graph/dagvine/graph_executor.py b/taskvine/src/graph/dagvine/graph_executor.py index 7b2f4c7323..ac910b8d35 100644 --- a/taskvine/src/graph/dagvine/graph_executor.py +++ b/taskvine/src/graph/dagvine/graph_executor.py @@ -5,7 +5,7 @@ from ndcctools.taskvine.dagvine.graph_definition import GraphKeyResult, TaskGraph from ndcctools.taskvine.dagvine.params import ManagerTuningParams from ndcctools.taskvine.dagvine.params import VineConstantParams - +from ndcctools.taskvine.dagvine.params import RegularParams import cloudpickle import os @@ -69,95 +69,81 @@ def ensure_task_dict(collection_dict): class GraphExecutor(Manager): def __init__(self, *args, - manager_tuning_params=None, - library=None, - worker_cores=16, + manager_tuning_params=ManagerTuningParams(), + regular_params=RegularParams(), + vine_constant_params=VineConstantParams(), **kwargs): signal.signal(signal.SIGINT, self._on_sigint) - # delete all files in the run info template directory, do this before super().__init__() - self.run_info_path = kwargs.get('run_info_path') - self.run_info_template = kwargs.get('run_info_template') - self.run_info_path_absolute = os.path.join(self.run_info_path, self.run_info_template) - if self.run_info_path and self.run_info_template: - delete_all_files(self.run_info_path_absolute) + self.manager_tuning_params = manager_tuning_params + self.vine_constant_params = vine_constant_params + self.regular_params = regular_params - # initialize the manager - super_params = set(inspect.signature(Manager.__init__).parameters) - super_kwargs = {k: v for k, v in kwargs.items() if k in super_params} + # delete all files in the run info directory, do this before super().__init__() + run_info_path = self.regular_params.run_info_path + run_info_template = self.regular_params.run_info_template + self.run_info_path_absolute = os.path.join(run_info_path, run_info_template) + if run_info_path and run_info_template: + delete_all_files(self.run_info_path_absolute) - super().__init__(*args, **super_kwargs) + kwargs["run_info_path"] = run_info_path + kwargs["run_info_template"] = run_info_template + super().__init__(*args, **kwargs) print(f"TaskVine Manager \033[92m{self.name}\033[0m listening on port \033[92m{self.port}\033[0m") - # tune the manager - if manager_tuning_params: - assert isinstance(manager_tuning_params, ManagerTuningParams), "manager_tuning_params must be an instance of ManagerTuningParams" - for k, v in manager_tuning_params.to_dict().items(): - print(f"Tuning {k} to {v}") - self.tune(k, v) - # initialize the task graph self._vine_task_graph = cvine.vine_task_graph_create(self._taskvine) - # create library task - self.library = library - self.library.install(self, worker_cores, self._vine_task_graph) + def tune_manager(self): + for k, v in self.manager_tuning_params.to_dict().items(): + print(f"Tuning {k} to {v}") + self.tune(k, v) + + def set_policy(self): + # set replica placement policy + cvine.vine_set_replica_placement_policy(self._taskvine, self.vine_constant_params.get_c_constant_of("replica_placement_policy")) + # set worker scheduling algorithm + cvine.vine_set_scheduler(self._taskvine, self.vine_constant_params.get_c_constant_of("schedule")) + # set task priority mode + cvine.vine_task_graph_set_task_priority_mode(self._vine_task_graph, self.vine_constant_params.get_c_constant_of("task_priority_mode")) def run(self, collection_dict, target_keys=[], - replica_placement_policy="random", - priority_mode="largest-input-first", - # scheduling_mode="files", - extra_task_output_size_mb=["uniform", 0, 0], - extra_task_sleep_time=["uniform", 0, 0], - prune_depth=1, - shared_file_system_dir="/project01/ndcms/jzhou24/shared_file_system", - staging_dir="/project01/ndcms/jzhou24/staging", - failure_injection_step_percent=-1, - vine_constant_params=None, - outfile_type={ - "temp": 1.0, - "shared-file-system": 0.0, - }): + library=None, + ): + + # tune the manager every time we start a new run as the parameters may have changed + self.tune_manager() + # set the policy every time we start a new run as the parameters may have changed + self.set_policy() + self.target_keys = target_keys self.task_dict = ensure_task_dict(collection_dict) - cvine.vine_task_graph_set_failure_injection_step_percent(self._vine_task_graph, failure_injection_step_percent) + # create library task + self.library = library + self.library.install(self, self.regular_params.libcores, self._vine_task_graph) - # self.set_scheduler(scheduling_mode) + cvine.vine_task_graph_set_failure_injection_step_percent(self._vine_task_graph, self.regular_params.failure_injection_step_percent) # create task graph in the python side print("Initializing TaskGraph object") - self.shared_file_system_dir = shared_file_system_dir - self.staging_dir = staging_dir self.task_graph = TaskGraph(self.task_dict, - staging_dir=self.staging_dir, - shared_file_system_dir=self.shared_file_system_dir, - extra_task_output_size_mb=extra_task_output_size_mb, - extra_task_sleep_time=extra_task_sleep_time) + staging_dir=self.regular_params.staging_dir, + shared_file_system_dir=self.regular_params.shared_file_system_dir, + extra_task_output_size_mb=self.regular_params.extra_task_output_size_mb, + extra_task_sleep_time=self.regular_params.extra_task_sleep_time) topo_order = self.task_graph.get_topological_order() - # the sum of the values in outfile_type must be 1.0 - assert sum(list(outfile_type.values())) == 1.0 - - if vine_constant_params: - assert isinstance(vine_constant_params, VineConstantParams), "vine_constant_params must be an instance of VineConstantParams" - # set replica placement policy - cvine.vine_set_replica_placement_policy(self._taskvine, vine_constant_params.get_c_constant_of("replica_placement_policy")) - # set worker scheduling algorithm - cvine.vine_set_scheduler(self._taskvine, vine_constant_params.get_c_constant_of("schedule")) - # set task priority mode - cvine.vine_task_graph_set_task_priority_mode(self._vine_task_graph, vine_constant_params.get_c_constant_of("task_priority_mode")) - # create task graph in the python side print("Initializing task graph in TaskVine") for k in topo_order: cvine.vine_task_graph_add_node(self._vine_task_graph, self.task_graph.vine_key_of[k], - self.staging_dir, - prune_depth) + self.regular_params.staging_dir, + self.regular_params.prune_depth) for pk in self.task_graph.parents_of.get(k, []): cvine.vine_task_graph_add_dependency(self._vine_task_graph, self.task_graph.vine_key_of[pk], self.task_graph.vine_key_of[k]) @@ -172,7 +158,7 @@ def run(self, # keys with larger heavy score should be stored into the shared file system sorted_keys = sorted(heavy_scores, key=lambda x: heavy_scores[x], reverse=True) - shared_file_system_size = round(len(sorted_keys) * outfile_type["shared-file-system"]) + shared_file_system_size = round(len(sorted_keys) * self.regular_params.outfile_type["shared-file-system"]) for i, k in enumerate(sorted_keys): if k in self.target_keys: choice = "local" @@ -195,7 +181,7 @@ def run(self, cloudpickle.dump(self.task_graph, f) # now execute the vine graph - print(f"Executing task graph, logs will be written into {self.run_info_path_absolute}") + print(f"\033[92mExecuting task graph, logs will be written into {self.run_info_path_absolute}\033[0m") cvine.vine_task_graph_execute(self._vine_task_graph) # after execution, we need to load results of target keys diff --git a/taskvine/src/graph/dagvine/params.py b/taskvine/src/graph/dagvine/params.py index 4e21e10916..1f4a363760 100644 --- a/taskvine/src/graph/dagvine/params.py +++ b/taskvine/src/graph/dagvine/params.py @@ -1,9 +1,11 @@ -from dataclasses import dataclass +from dataclasses import dataclass, field from ndcctools.taskvine.utils import get_c_constant @dataclass class ManagerTuningParams: + """These parameters are used to tune the manager at the C level + and should comply with the TaskVine manager API.""" worker_source_max_transfers: int = 100 max_retrievals: int = -1 prefer_dispatch: int = 1 @@ -75,4 +77,37 @@ def get_c_constant_of(self, key): normalized_key = self.normalize(key) if normalized_key not in self.valid_normalized_values: raise ValueError(f"Invalid key: {normalized_key}") - return get_c_constant(f"{normalized_key.upper()}_{getattr(self, normalized_key).upper()}") \ No newline at end of file + return get_c_constant(f"{normalized_key.upper()}_{getattr(self, normalized_key).upper()}") + + +@dataclass +class RegularParams: + """Regular parameters that will be used directly by the graph executor.""" + run_info_path: str = None + run_info_template: str = None + libcores: int = 16 + failure_injection_step_percent: int = -1 + prune_depth: int = 1 + staging_dir: str = "./staging" + shared_file_system_dir: str = "./shared_file_system" + extra_task_output_size_mb: list[str, float, float] = field(default_factory=lambda: ["uniform", 0, 0]) + extra_task_sleep_time: list[str, float, float] = field(default_factory=lambda: ["uniform", 0, 0]) + outfile_type: dict[str, float] = field(default_factory=lambda: { + "temp": 1.0, + "shared-file-system": 0.0, + }) + + def update(self, params: dict): + """Update configuration from a dict. + - Convert '-' in keys to '_', values are as is. + """ + if not isinstance(params, dict): + raise TypeError(f"update() expects a dict, got {type(params).__name__}") + + for k, v in params.items(): + normalized_key = k.replace("-", "_") + if normalized_key not in self.__dict__.keys(): + raise ValueError(f"Invalid param key: {normalized_key}") + + setattr(self, normalized_key, v) + return self From dbe57a8f96026761146ff3c42270a46d0dddfb04 Mon Sep 17 00:00:00 2001 From: Jin Zhou Date: Thu, 16 Oct 2025 14:12:26 -0400 Subject: [PATCH 037/113] api change --- taskvine/src/Makefile | 1 + taskvine/src/bindings/python3/taskvine.i | 2 +- taskvine/src/graph/dagvine/.gitignore | 5 + taskvine/src/graph/dagvine/Makefile | 27 ++++- taskvine/src/graph/dagvine/cdagvine.py | 107 ++++++++++++++++++ taskvine/src/graph/dagvine/cvine_dagvine.py | 107 ++++++++++++++++++ taskvine/src/graph/dagvine/graph_executor.py | 27 ++--- taskvine/src/graph/dagvine/vine_task_graph.c | 40 +++++-- taskvine/src/graph/dagvine/vine_task_graph.h | 113 +++++++++++++++++-- taskvine/src/graph/dagvine/vine_task_graph.i | 18 +++ taskvine/src/manager/taskvine.h | 96 +--------------- taskvine/src/manager/vine_manager.c | 2 +- taskvine/src/manager/vine_temp.c | 2 +- 13 files changed, 410 insertions(+), 137 deletions(-) create mode 100644 taskvine/src/graph/dagvine/cdagvine.py create mode 100644 taskvine/src/graph/dagvine/cvine_dagvine.py create mode 100644 taskvine/src/graph/dagvine/vine_task_graph.i diff --git a/taskvine/src/Makefile b/taskvine/src/Makefile index 5c8858a2e4..1bd3ef4f43 100644 --- a/taskvine/src/Makefile +++ b/taskvine/src/Makefile @@ -6,6 +6,7 @@ TARGETS=manager worker tools bindings examples graph all: $(TARGETS) worker: manager +graph: manager bindings: manager graph tools: manager examples: manager worker tools bindings diff --git a/taskvine/src/bindings/python3/taskvine.i b/taskvine/src/bindings/python3/taskvine.i index ba4e66cb74..1d875f97be 100644 --- a/taskvine/src/bindings/python3/taskvine.i +++ b/taskvine/src/bindings/python3/taskvine.i @@ -1,5 +1,5 @@ /* taskvine.i */ -%module cvine +%module(package="ndcctools.taskvine") cvine %include carrays.i %array_functions(struct rmsummary *, rmsummayArray); diff --git a/taskvine/src/graph/dagvine/.gitignore b/taskvine/src/graph/dagvine/.gitignore index e69de29bb2..3e98d74473 100644 --- a/taskvine/src/graph/dagvine/.gitignore +++ b/taskvine/src/graph/dagvine/.gitignore @@ -0,0 +1,5 @@ +*.a +*.so +*.o +*_wrap.c +*_wrap.0 \ No newline at end of file diff --git a/taskvine/src/graph/dagvine/Makefile b/taskvine/src/graph/dagvine/Makefile index 497c798a4f..ba1b84c43b 100644 --- a/taskvine/src/graph/dagvine/Makefile +++ b/taskvine/src/graph/dagvine/Makefile @@ -7,10 +7,14 @@ LOCAL_CCFLAGS+=-I ${CCTOOLS_HOME}/taskvine/src/manager SOURCES = vine_task_node.c vine_task_graph.c OBJECTS = $(SOURCES:%.c=%.o) +SWIG_I = vine_task_graph.i +SWIG_WRAP = vine_task_graph_wrap.c +PYMODULE = _cdagvine.$(CCTOOLS_DYNAMIC_SUFFIX) + LIBRARIES = libdagvine.a PROGRAMS = SCRIPTS = -TARGETS = $(LIBRARIES) $(PROGRAMS) +TARGETS = $(LIBRARIES) $(PYMODULE) $(PROGRAMS) all: $(TARGETS) @@ -18,14 +22,33 @@ libdagvine.a: $(OBJECTS) $(PROGRAMS): $(EXTERNALS) +$(SWIG_WRAP): $(SWIG_I) vine_task_graph.h + $(CCTOOLS_SWIG) -python -threads -relativeimport -I$(CCTOOLS_HOME)/taskvine/src/manager -I$(CCTOOLS_HOME)/dttools/src -o $@ $< + +# Build Python module +vine_task_graph_wrap.o: $(SWIG_WRAP) + $(CCTOOLS_CC) -o $@ -c $(CCTOOLS_INTERNAL_CCFLAGS) $(CCTOOLS_PYTHON3_CCFLAGS) $(LOCAL_CCFLAGS) $< + +$(PYMODULE): vine_task_graph_wrap.o libdagvine.a +ifeq ($(CCTOOLS_STATIC),1) + $(CCTOOLS_LD) -o $@ $(CCTOOLS_DYNAMIC_FLAG) $(CCTOOLS_INTERNAL_LDFLAGS) $(LOCAL_LDFLAGS) $^ $(LOCAL_LINKAGE) $(CCTOOLS_PYTHON3_LDFLAGS) $(CCTOOLS_EXTERNAL_LINKAGE) +else + $(CCTOOLS_LD) -o $@ $(CCTOOLS_DYNAMIC_FLAG) $(CCTOOLS_INTERNAL_LDFLAGS) $(LOCAL_LDFLAGS) $^ $(LOCAL_LINKAGE) $(CCTOOLS_PYTHON3_LDFLAGS) $(CCTOOLS_EXTERNAL_LINKAGE) +endif + + + install: all mkdir -p $(CCTOOLS_INSTALL_DIR)/lib cp $(LIBRARIES) $(CCTOOLS_INSTALL_DIR)/lib/ mkdir -p $(CCTOOLS_INSTALL_DIR)/graph/dagvine/include cp ${CCTOOLS_HOME}/taskvine/src/manager/taskvine.h $(CCTOOLS_INSTALL_DIR)/graph/dagvine/include/ + mkdir -p $(CCTOOLS_PYTHON3_PATH)/ndcctools/taskvine/dagvine + cp $(PYMODULE) $(CCTOOLS_PYTHON3_PATH)/ndcctools/taskvine/dagvine/ + cp cdagvine.py $(CCTOOLS_PYTHON3_PATH)/ndcctools/taskvine/dagvine/ clean: - rm -rf $(PROGRAMS) $(OBJECTS) *.o + rm -rf $(PROGRAMS) $(OBJECTS) *.o $(SWIG_WRAP) vine_task_graph_wrap.o $(PYMODULE) cdagvine.py test: all diff --git a/taskvine/src/graph/dagvine/cdagvine.py b/taskvine/src/graph/dagvine/cdagvine.py new file mode 100644 index 0000000000..920389fda9 --- /dev/null +++ b/taskvine/src/graph/dagvine/cdagvine.py @@ -0,0 +1,107 @@ +# This file was automatically generated by SWIG (https://www.swig.org). +# Version 4.3.1 +# +# Do not make changes to this file unless you know what you are doing - modify +# the SWIG interface file instead. + +from sys import version_info as _swig_python_version_info +# Import the low-level C/C++ module +if __package__ or "." in __name__: + from . import _cdagvine +else: + import _cdagvine + +try: + import builtins as __builtin__ +except ImportError: + import __builtin__ + +def _swig_repr(self): + try: + strthis = "proxy of " + self.this.__repr__() + except __builtin__.Exception: + strthis = "" + return "<%s.%s; %s >" % (self.__class__.__module__, self.__class__.__name__, strthis,) + + +def _swig_setattr_nondynamic_instance_variable(set): + def set_instance_attr(self, name, value): + if name == "this": + set(self, name, value) + elif name == "thisown": + self.this.own(value) + elif hasattr(self, name) and isinstance(getattr(type(self), name), property): + set(self, name, value) + else: + raise AttributeError("You cannot add instance attributes to %s" % self) + return set_instance_attr + + +def _swig_setattr_nondynamic_class_variable(set): + def set_class_attr(cls, name, value): + if hasattr(cls, name) and not isinstance(getattr(cls, name), property): + set(cls, name, value) + else: + raise AttributeError("You cannot add class attributes to %s" % cls) + return set_class_attr + + +def _swig_add_metaclass(metaclass): + """Class decorator for adding a metaclass to a SWIG wrapped class - a slimmed down version of six.add_metaclass""" + def wrapper(cls): + return metaclass(cls.__name__, cls.__bases__, cls.__dict__.copy()) + return wrapper + + +class _SwigNonDynamicMeta(type): + """Meta class to enforce nondynamic attributes (no new attributes) for a class""" + __setattr__ = _swig_setattr_nondynamic_class_variable(type.__setattr__) + + +import ndcctools.taskvine.cvine +class vine_task_graph(object): + thisown = property(lambda x: x.this.own(), lambda x, v: x.this.own(v), doc="The membership flag") + __repr__ = _swig_repr + manager = property(_cdagvine.vine_task_graph_manager_get, _cdagvine.vine_task_graph_manager_set) + nodes = property(_cdagvine.vine_task_graph_nodes_get, _cdagvine.vine_task_graph_nodes_set) + task_id_to_node = property(_cdagvine.vine_task_graph_task_id_to_node_get, _cdagvine.vine_task_graph_task_id_to_node_set) + outfile_cachename_to_node = property(_cdagvine.vine_task_graph_outfile_cachename_to_node_get, _cdagvine.vine_task_graph_outfile_cachename_to_node_set) + proxy_library_name = property(_cdagvine.vine_task_graph_proxy_library_name_get, _cdagvine.vine_task_graph_proxy_library_name_set) + proxy_function_name = property(_cdagvine.vine_task_graph_proxy_function_name_get, _cdagvine.vine_task_graph_proxy_function_name_set) + task_priority_mode = property(_cdagvine.vine_task_graph_task_priority_mode_get, _cdagvine.vine_task_graph_task_priority_mode_set) + failure_injection_step_percent = property(_cdagvine.vine_task_graph_failure_injection_step_percent_get, _cdagvine.vine_task_graph_failure_injection_step_percent_set) + + def __init__(self): + _cdagvine.vine_task_graph_swiginit(self, _cdagvine.new_vine_task_graph()) + __swig_destroy__ = _cdagvine.delete_vine_task_graph + +# Register vine_task_graph in _cdagvine: +_cdagvine.vine_task_graph_swigregister(vine_task_graph) + +def vine_task_graph_create(q): + return _cdagvine.vine_task_graph_create(q) + +def vine_task_graph_set_task_priority_mode(tg, priority_mode): + return _cdagvine.vine_task_graph_set_task_priority_mode(tg, priority_mode) + +def vine_task_graph_set_failure_injection_step_percent(tg, percent): + return _cdagvine.vine_task_graph_set_failure_injection_step_percent(tg, percent) + +def vine_task_graph_add_node(tg, node_key, staging_dir, prune_depth): + return _cdagvine.vine_task_graph_add_node(tg, node_key, staging_dir, prune_depth) + +def vine_task_graph_add_dependency(tg, parent_key, child_key): + return _cdagvine.vine_task_graph_add_dependency(tg, parent_key, child_key) + +def vine_task_graph_compute_topology_metrics(tg): + return _cdagvine.vine_task_graph_compute_topology_metrics(tg) + +def vine_task_graph_get_node_heavy_score(tg, node_key): + return _cdagvine.vine_task_graph_get_node_heavy_score(tg, node_key) + +def vine_task_graph_set_node_outfile(tg, node_key, outfile_type, outfile_remote_name): + return _cdagvine.vine_task_graph_set_node_outfile(tg, node_key, outfile_type, outfile_remote_name) + +def vine_task_graph_set_proxy_library_and_function_names(tg, proxy_library_name, proxy_function_name): + return _cdagvine.vine_task_graph_set_proxy_library_and_function_names(tg, proxy_library_name, proxy_function_name) + diff --git a/taskvine/src/graph/dagvine/cvine_dagvine.py b/taskvine/src/graph/dagvine/cvine_dagvine.py new file mode 100644 index 0000000000..c705763a26 --- /dev/null +++ b/taskvine/src/graph/dagvine/cvine_dagvine.py @@ -0,0 +1,107 @@ +# This file was automatically generated by SWIG (https://www.swig.org). +# Version 4.3.1 +# +# Do not make changes to this file unless you know what you are doing - modify +# the SWIG interface file instead. + +from sys import version_info as _swig_python_version_info +# Import the low-level C/C++ module +if __package__ or "." in __name__: + from . import _cvine_dagvine +else: + import _cvine_dagvine + +try: + import builtins as __builtin__ +except ImportError: + import __builtin__ + +def _swig_repr(self): + try: + strthis = "proxy of " + self.this.__repr__() + except __builtin__.Exception: + strthis = "" + return "<%s.%s; %s >" % (self.__class__.__module__, self.__class__.__name__, strthis,) + + +def _swig_setattr_nondynamic_instance_variable(set): + def set_instance_attr(self, name, value): + if name == "this": + set(self, name, value) + elif name == "thisown": + self.this.own(value) + elif hasattr(self, name) and isinstance(getattr(type(self), name), property): + set(self, name, value) + else: + raise AttributeError("You cannot add instance attributes to %s" % self) + return set_instance_attr + + +def _swig_setattr_nondynamic_class_variable(set): + def set_class_attr(cls, name, value): + if hasattr(cls, name) and not isinstance(getattr(cls, name), property): + set(cls, name, value) + else: + raise AttributeError("You cannot add class attributes to %s" % cls) + return set_class_attr + + +def _swig_add_metaclass(metaclass): + """Class decorator for adding a metaclass to a SWIG wrapped class - a slimmed down version of six.add_metaclass""" + def wrapper(cls): + return metaclass(cls.__name__, cls.__bases__, cls.__dict__.copy()) + return wrapper + + +class _SwigNonDynamicMeta(type): + """Meta class to enforce nondynamic attributes (no new attributes) for a class""" + __setattr__ = _swig_setattr_nondynamic_class_variable(type.__setattr__) + + +import cvine +class vine_task_graph(object): + thisown = property(lambda x: x.this.own(), lambda x, v: x.this.own(v), doc="The membership flag") + __repr__ = _swig_repr + manager = property(_cvine_dagvine.vine_task_graph_manager_get, _cvine_dagvine.vine_task_graph_manager_set) + nodes = property(_cvine_dagvine.vine_task_graph_nodes_get, _cvine_dagvine.vine_task_graph_nodes_set) + task_id_to_node = property(_cvine_dagvine.vine_task_graph_task_id_to_node_get, _cvine_dagvine.vine_task_graph_task_id_to_node_set) + outfile_cachename_to_node = property(_cvine_dagvine.vine_task_graph_outfile_cachename_to_node_get, _cvine_dagvine.vine_task_graph_outfile_cachename_to_node_set) + proxy_library_name = property(_cvine_dagvine.vine_task_graph_proxy_library_name_get, _cvine_dagvine.vine_task_graph_proxy_library_name_set) + proxy_function_name = property(_cvine_dagvine.vine_task_graph_proxy_function_name_get, _cvine_dagvine.vine_task_graph_proxy_function_name_set) + task_priority_mode = property(_cvine_dagvine.vine_task_graph_task_priority_mode_get, _cvine_dagvine.vine_task_graph_task_priority_mode_set) + failure_injection_step_percent = property(_cvine_dagvine.vine_task_graph_failure_injection_step_percent_get, _cvine_dagvine.vine_task_graph_failure_injection_step_percent_set) + + def __init__(self): + _cvine_dagvine.vine_task_graph_swiginit(self, _cvine_dagvine.new_vine_task_graph()) + __swig_destroy__ = _cvine_dagvine.delete_vine_task_graph + +# Register vine_task_graph in _cvine_dagvine: +_cvine_dagvine.vine_task_graph_swigregister(vine_task_graph) + +def vine_task_graph_create(q): + return _cvine_dagvine.vine_task_graph_create(q) + +def vine_task_graph_set_task_priority_mode(tg, priority_mode): + return _cvine_dagvine.vine_task_graph_set_task_priority_mode(tg, priority_mode) + +def vine_task_graph_set_failure_injection_step_percent(tg, percent): + return _cvine_dagvine.vine_task_graph_set_failure_injection_step_percent(tg, percent) + +def vine_task_graph_add_node(tg, node_key, staging_dir, prune_depth): + return _cvine_dagvine.vine_task_graph_add_node(tg, node_key, staging_dir, prune_depth) + +def vine_task_graph_add_dependency(tg, parent_key, child_key): + return _cvine_dagvine.vine_task_graph_add_dependency(tg, parent_key, child_key) + +def vine_task_graph_compute_topology_metrics(tg): + return _cvine_dagvine.vine_task_graph_compute_topology_metrics(tg) + +def vine_task_graph_get_node_heavy_score(tg, node_key): + return _cvine_dagvine.vine_task_graph_get_node_heavy_score(tg, node_key) + +def vine_task_graph_set_node_outfile(tg, node_key, outfile_type, outfile_remote_name): + return _cvine_dagvine.vine_task_graph_set_node_outfile(tg, node_key, outfile_type, outfile_remote_name) + +def vine_task_graph_set_proxy_library_and_function_names(tg, proxy_library_name, proxy_function_name): + return _cvine_dagvine.vine_task_graph_set_proxy_library_and_function_names(tg, proxy_library_name, proxy_function_name) + diff --git a/taskvine/src/graph/dagvine/graph_executor.py b/taskvine/src/graph/dagvine/graph_executor.py index ac910b8d35..c0ae5757a6 100644 --- a/taskvine/src/graph/dagvine/graph_executor.py +++ b/taskvine/src/graph/dagvine/graph_executor.py @@ -1,4 +1,5 @@ from ndcctools.taskvine import cvine +from ndcctools.taskvine.dagvine import cdagvine from ndcctools.taskvine.manager import Manager from ndcctools.taskvine.utils import delete_all_files, get_c_constant @@ -74,7 +75,7 @@ def __init__(self, vine_constant_params=VineConstantParams(), **kwargs): - signal.signal(signal.SIGINT, self._on_sigint) + # Use C-side SIGINT handler in cdagvine to handle Ctrl+C gracefully self.manager_tuning_params = manager_tuning_params self.vine_constant_params = vine_constant_params @@ -93,7 +94,7 @@ def __init__(self, print(f"TaskVine Manager \033[92m{self.name}\033[0m listening on port \033[92m{self.port}\033[0m") # initialize the task graph - self._vine_task_graph = cvine.vine_task_graph_create(self._taskvine) + self._vine_task_graph = cdagvine.vine_task_graph_create(self._taskvine) def tune_manager(self): for k, v in self.manager_tuning_params.to_dict().items(): @@ -102,11 +103,11 @@ def tune_manager(self): def set_policy(self): # set replica placement policy - cvine.vine_set_replica_placement_policy(self._taskvine, self.vine_constant_params.get_c_constant_of("replica_placement_policy")) + cvine.vine_temp_set_replica_placement_policy(self._taskvine, self.vine_constant_params.get_c_constant_of("replica_placement_policy")) # set worker scheduling algorithm cvine.vine_set_scheduler(self._taskvine, self.vine_constant_params.get_c_constant_of("schedule")) # set task priority mode - cvine.vine_task_graph_set_task_priority_mode(self._vine_task_graph, self.vine_constant_params.get_c_constant_of("task_priority_mode")) + cdagvine.vine_task_graph_set_task_priority_mode(self._vine_task_graph, self.vine_constant_params.get_c_constant_of("task_priority_mode")) def run(self, collection_dict, @@ -126,7 +127,7 @@ def run(self, self.library = library self.library.install(self, self.regular_params.libcores, self._vine_task_graph) - cvine.vine_task_graph_set_failure_injection_step_percent(self._vine_task_graph, self.regular_params.failure_injection_step_percent) + cdagvine.vine_task_graph_set_failure_injection_step_percent(self._vine_task_graph, self.regular_params.failure_injection_step_percent) # create task graph in the python side print("Initializing TaskGraph object") @@ -140,21 +141,21 @@ def run(self, # create task graph in the python side print("Initializing task graph in TaskVine") for k in topo_order: - cvine.vine_task_graph_add_node(self._vine_task_graph, + cdagvine.vine_task_graph_add_node(self._vine_task_graph, self.task_graph.vine_key_of[k], self.regular_params.staging_dir, self.regular_params.prune_depth) for pk in self.task_graph.parents_of.get(k, []): - cvine.vine_task_graph_add_dependency(self._vine_task_graph, self.task_graph.vine_key_of[pk], self.task_graph.vine_key_of[k]) + cdagvine.vine_task_graph_add_dependency(self._vine_task_graph, self.task_graph.vine_key_of[pk], self.task_graph.vine_key_of[k]) # we must finalize the graph in c side after all nodes and dependencies are added # this includes computing various metrics for each node, such as depth, height, heavy score, etc. - cvine.vine_task_graph_compute_topology_metrics(self._vine_task_graph) + cdagvine.vine_task_graph_compute_topology_metrics(self._vine_task_graph) # then we can use the heavy score to sort the nodes and specify their outfile remote names heavy_scores = {} for k in self.task_graph.task_dict.keys(): - heavy_scores[k] = cvine.vine_task_graph_get_node_heavy_score(self._vine_task_graph, self.task_graph.vine_key_of[k]) + heavy_scores[k] = cdagvine.vine_task_graph_get_node_heavy_score(self._vine_task_graph, self.task_graph.vine_key_of[k]) # keys with larger heavy score should be stored into the shared file system sorted_keys = sorted(heavy_scores, key=lambda x: heavy_scores[x], reverse=True) @@ -171,7 +172,7 @@ def run(self, self.task_graph.set_outfile_type_of(k, choice) # set on the C side, so the manager knows where the data is stored outfile_type_str = f"NODE_OUTFILE_TYPE_{choice.upper().replace('-', '_')}" - cvine.vine_task_graph_set_node_outfile(self._vine_task_graph, + cdagvine.vine_task_graph_set_node_outfile(self._vine_task_graph, self.task_graph.vine_key_of[k], get_c_constant(outfile_type_str), self.task_graph.outfile_remote_name[k]) @@ -182,12 +183,12 @@ def run(self, # now execute the vine graph print(f"\033[92mExecuting task graph, logs will be written into {self.run_info_path_absolute}\033[0m") - cvine.vine_task_graph_execute(self._vine_task_graph) + cdagvine.vine_task_graph_execute(self._vine_task_graph) # after execution, we need to load results of target keys results = {} for k in self.target_keys: - local_outfile_path = cvine.vine_task_graph_get_node_local_outfile_source(self._vine_task_graph, self.task_graph.vine_key_of[k]) + local_outfile_path = cdagvine.vine_task_graph_get_node_local_outfile_source(self._vine_task_graph, self.task_graph.vine_key_of[k]) if not os.path.exists(local_outfile_path): results[k] = "NOT_FOUND" continue @@ -202,7 +203,7 @@ def _on_sigint(self, signum, frame): def __del__(self): if hasattr(self, '_vine_task_graph') and self._vine_task_graph: - cvine.vine_task_graph_delete(self._vine_task_graph) + cdagvine.vine_task_graph_delete(self._vine_task_graph) if hasattr(self, 'library') and self.library.local_path and os.path.exists(self.library.local_path): os.remove(self.library.local_path) diff --git a/taskvine/src/graph/dagvine/vine_task_graph.c b/taskvine/src/graph/dagvine/vine_task_graph.c index cdc37dddc4..829a3fb4e9 100644 --- a/taskvine/src/graph/dagvine/vine_task_graph.c +++ b/taskvine/src/graph/dagvine/vine_task_graph.c @@ -27,6 +27,7 @@ #include "vine_task.h" #include "vine_file.h" #include "vine_mount.h" +#include "taskvine.h" static volatile sig_atomic_t interrupted = 0; @@ -134,7 +135,7 @@ static void submit_unblocked_children(struct vine_task_graph *tg, struct vine_ta return; } - struct vine_task_node *child_node; + struct vine_task_node *child_node; LIST_ITERATE(node->children, child_node) { /* Remove this parent from the child's pending set if it exists */ @@ -348,30 +349,45 @@ static struct vine_task_node *get_node_by_task(struct vine_task_graph *tg, struc /*************************************************************/ /** - * Set the proxy library and function names (Python-side), shared by all tasks. + * Set the proxy library name (Python-side), shared by all tasks. * @param tg Reference to the task graph object. * @param proxy_library_name Reference to the proxy library name. - * @param proxy_function_name Reference to the proxy function name. */ -void vine_task_graph_set_proxy_library_and_function_names(struct vine_task_graph *tg, - const char *proxy_library_name, - const char *proxy_function_name) +void vine_task_graph_set_proxy_library_name(struct vine_task_graph *tg, const char *proxy_library_name) { - if (!tg || !proxy_library_name || !proxy_function_name) { + if (!tg || !proxy_library_name) { return; } - /* free the existing proxy library and function names if they exist */ + /* free the existing proxy library name if it exists */ if (tg->proxy_library_name) { free(tg->proxy_library_name); } + + tg->proxy_library_name = xxstrdup(proxy_library_name); + + return; +} + +/** + * Set the proxy function name (Python-side), shared by all tasks. + * @param tg Reference to the task graph object. + * @param proxy_function_name Reference to the proxy function name. + */ +void vine_task_graph_set_proxy_function_name(struct vine_task_graph *tg, const char *proxy_function_name) +{ + if (!tg || !proxy_function_name) { + return; + } + + /* free the existing proxy function name if it exists */ if (tg->proxy_function_name) { free(tg->proxy_function_name); } - /* set the new proxy library and function names */ - tg->proxy_library_name = xxstrdup(proxy_library_name); tg->proxy_function_name = xxstrdup(proxy_function_name); + + return; } /** @@ -619,8 +635,6 @@ struct vine_task_graph *vine_task_graph_create(struct vine_manager *q) vine_enable_debug_log(debug_tmp); free(debug_tmp); - signal(SIGINT, handle_sigint); - return tg; } @@ -724,6 +738,8 @@ void vine_task_graph_execute(struct vine_task_graph *tg) return; } + signal(SIGINT, handle_sigint); + debug(D_VINE, "start executing task graph"); /* print the info of all nodes */ diff --git a/taskvine/src/graph/dagvine/vine_task_graph.h b/taskvine/src/graph/dagvine/vine_task_graph.h index 488d520bca..5fd81fdd6c 100644 --- a/taskvine/src/graph/dagvine/vine_task_graph.h +++ b/taskvine/src/graph/dagvine/vine_task_graph.h @@ -9,6 +9,14 @@ #include "vine_task_node.h" #include "taskvine.h" +/** Select the type of the node-output file. */ +typedef enum { + VINE_NODE_OUTFILE_TYPE_LOCAL = 0, /* Node-output file will be stored locally on the manager's staging directory */ + VINE_NODE_OUTFILE_TYPE_TEMP, /* Node-output file will be stored in the temporary node-local storage */ + VINE_NODE_OUTFILE_TYPE_SHARED_FILE_SYSTEM, /* Node-output file will be stored in the persistent shared file system */ +} vine_task_node_outfile_type_t; + +/** The task graph object. */ struct vine_task_graph { struct vine_manager *manager; struct hash_table *nodes; @@ -24,23 +32,104 @@ struct vine_task_graph { /* Public APIs for operating the task graph */ -const char *vine_task_graph_get_proxy_library_name(const struct vine_task_graph *tg); -const char *vine_task_graph_get_proxy_function_name(const struct vine_task_graph *tg); -double vine_task_graph_get_node_heavy_score(const struct vine_task_graph *tg, const char *node_key); -const char *vine_task_graph_get_node_local_outfile_source(const struct vine_task_graph *tg, const char *node_key); -void vine_task_graph_compute_topology_metrics(struct vine_task_graph *tg); -struct vine_task_node *vine_task_graph_add_node(struct vine_task_graph *tg, - const char *node_key, - const char *staging_dir, - int prune_depth); + +/** Create a task graph object and return it. +@param q Reference to the current manager object. +@return A new task graph object. +*/ struct vine_task_graph *vine_task_graph_create(struct vine_manager *q); + +/** Set the task priority mode for the task graph. +@param tg Reference to the task graph object. +@param priority_mode Reference to the priority mode. +*/ +void vine_task_graph_set_task_priority_mode(struct vine_task_graph *tg, vine_task_priority_mode_t priority_mode); + +/** Set the failure injection step percent. +@param tg Reference to the task graph object. +@param percent Reference to the failure injection step percent. +*/ void vine_task_graph_set_failure_injection_step_percent(struct vine_task_graph *tg, double percent); + +/** Create a new node in the task graph. +@param tg Reference to the task graph object. +@param node_key Reference to the node key. +@param staging_dir Reference to the staging directory. +@param prune_depth Reference to the prune depth. +@return A new node object. +*/ +struct vine_task_node *vine_task_graph_add_node(struct vine_task_graph *tg, + const char *node_key, + const char *staging_dir, + int prune_depth); + +/** Add a dependency between two nodes in the task graph. +@param tg Reference to the task graph object. +@param parent_key Reference to the parent node key. +@param child_key Reference to the child node key. +*/ void vine_task_graph_add_dependency(struct vine_task_graph *tg, const char *parent_key, const char *child_key); + +/** Finalize the metrics of the task graph. +@param tg Reference to the task graph object. +*/ +void vine_task_graph_compute_topology_metrics(struct vine_task_graph *tg); + +/** Get the heavy score of a node in the task graph. +@param tg Reference to the task graph object. +@param node_key Reference to the node key. +@return The heavy score. +*/ +double vine_task_graph_get_node_heavy_score(const struct vine_task_graph *tg, const char *node_key); + +/** Set the type of the node-output file. +@param tg Reference to the task graph object. +@param node_key Reference to the node key. +@param outfile_type Reference to the output file type. +@param outfile_remote_name Reference to the output file remote name. +*/ void vine_task_graph_set_node_outfile(struct vine_task_graph *tg, const char *node_key, vine_task_node_outfile_type_t outfile_type, const char *outfile_remote_name); + +/** Execute the task graph. +@param tg Reference to the task graph object. +*/ void vine_task_graph_execute(struct vine_task_graph *tg); + +/** Get the local outfile source of a node in the task graph. +@param tg Reference to the task graph object. +@param node_key Reference to the node key. +@return The local outfile source. +*/ +const char *vine_task_graph_get_node_local_outfile_source(const struct vine_task_graph *tg, const char *node_key); + +/** Delete a task graph object. +@param tg Reference to the task graph object. +*/ void vine_task_graph_delete(struct vine_task_graph *tg); -void vine_task_graph_set_proxy_library_and_function_names(struct vine_task_graph *tg, - const char *proxy_library_name, - const char *proxy_function_name); + +/** Get the library name of the task graph. +@param tg Reference to the task graph object. +@return The library name. +*/ +const char *vine_task_graph_get_proxy_library_name(const struct vine_task_graph *tg); + +/** Get the function name of the task graph. +@param tg Reference to the task graph object. +@return The function name. +*/ +const char *vine_task_graph_get_proxy_function_name(const struct vine_task_graph *tg); + +/** Set the proxy library name (Python-side), shared by all tasks. +@param tg Reference to the task graph object. +@param proxy_library_name Reference to the proxy library name. +*/ +void vine_task_graph_set_proxy_library_name(struct vine_task_graph *tg, const char *proxy_library_name); + +/** Set the proxy function name (Python-side), shared by all tasks. +@param tg Reference to the task graph object. +@param proxy_function_name Reference to the proxy function name. +*/ +void vine_task_graph_set_proxy_function_name(struct vine_task_graph *tg, const char *proxy_function_name); + #endif // VINE_TASK_GRAPH_H diff --git a/taskvine/src/graph/dagvine/vine_task_graph.i b/taskvine/src/graph/dagvine/vine_task_graph.i new file mode 100644 index 0000000000..b2a86b0c69 --- /dev/null +++ b/taskvine/src/graph/dagvine/vine_task_graph.i @@ -0,0 +1,18 @@ +/* SWIG interface for local dagvine graph bindings */ +%module cdagvine + +%{ +#include "int_sizes.h" +#include "vine_task_graph.h" +%} + +%include "stdint.i" +%include "int_sizes.h" + +/* Import existing SWIG interface for type information (do not wrap again) */ +%import "../../bindings/python3/taskvine.i" + +/* Expose only the dagvine task graph APIs */ +%include "vine_task_graph.h" + + diff --git a/taskvine/src/manager/taskvine.h b/taskvine/src/manager/taskvine.h index af5f9faec5..380d584d60 100644 --- a/taskvine/src/manager/taskvine.h +++ b/taskvine/src/manager/taskvine.h @@ -15,7 +15,6 @@ See the file COPYING for details. struct vine_manager; struct vine_task; struct vine_file; -struct vine_task_graph; /** @file taskvine.h The public API for the taskvine distributed application framework. A taskvine application consists of a manager process and a larger number of worker @@ -157,13 +156,6 @@ typedef enum { VINE_REPLICA_PLACEMENT_POLICY_TRANSFER_LOAD /* select a worker with the least incoming transfer load */ } vine_replica_placement_policy_t; -/** Select the type of the node-output file. */ -typedef enum { - VINE_NODE_OUTFILE_TYPE_LOCAL = 0, /* Node-output file will be stored locally on the manager's staging directory */ - VINE_NODE_OUTFILE_TYPE_TEMP, /* Node-output file will be stored in the temporary node-local storage */ - VINE_NODE_OUTFILE_TYPE_SHARED_FILE_SYSTEM, /* Node-output file will be stored in the persistent shared file system */ -} vine_task_node_outfile_type_t; - /** Select priority algorithm for task graph task scheduling. */ typedef enum { VINE_TASK_PRIORITY_MODE_RANDOM = 0, /**< Assign random priority to tasks */ @@ -1590,97 +1582,11 @@ char *vine_get_path_library_log(struct vine_manager *m, const char *path); */ char *vine_get_path_cache(struct vine_manager *m, const char *path); -/** Create a task graph object and return it. -@param q Reference to the current manager object. -@return A new task graph object. -*/ -struct vine_task_graph *vine_task_graph_create(struct vine_manager *q); - -/** Delete a task graph object. -@param tg Reference to the task graph object. -*/ -void vine_task_graph_delete(struct vine_task_graph *tg); - -/** Add a dependency between two nodes in the task graph. -@param tg Reference to the task graph object. -@param parent_key Reference to the parent node key. -@param child_key Reference to the child node key. -*/ -void vine_task_graph_add_dependency(struct vine_task_graph *tg, const char *parent_key, const char *child_key); - -/** Create a new node in the task graph. -@param tg Reference to the task graph object. -@param node_key Reference to the node key. -@param staging_dir Reference to the staging directory. -@param prune_depth Reference to the prune depth. -@return A new node object. -*/ -struct vine_task_node *vine_task_graph_add_node(struct vine_task_graph *tg, - const char *node_key, - const char *staging_dir, - int prune_depth); - /** Set the replica placement policy (which worker do we prefer to place the redundant file replicas). @param q Reference to the current manager object. @param policy Reference to the replica placement policy. */ -void vine_set_replica_placement_policy(struct vine_manager *q, vine_replica_placement_policy_t policy); - -/** Set the task priority mode for the task graph. -@param tg Reference to the task graph object. -@param priority_mode Reference to the priority mode. -*/ -void vine_task_graph_set_task_priority_mode(struct vine_task_graph *tg, vine_task_priority_mode_t priority_mode); - -/** Set the type of the node-output file. -@param tg Reference to the task graph object. -@param node_key Reference to the node key. -@param outfile_type Reference to the output file type. -@param outfile_remote_name Reference to the output file remote name. -*/ -void vine_task_graph_set_node_outfile(struct vine_task_graph *tg, const char *node_key, vine_task_node_outfile_type_t outfile_type, const char *outfile_remote_name); - -/** Finalize the metrics of the task graph. -@param tg Reference to the task graph object. -*/ -void vine_task_graph_compute_topology_metrics(struct vine_task_graph *tg); - -/** Get the library name of the task graph. -@param tg Reference to the task graph object. -@return The library name. -*/ -const char *vine_task_graph_get_proxy_library_name(const struct vine_task_graph *tg); - -/** Get the function name of the task graph. -@param tg Reference to the task graph object. -@return The function name. -*/ -const char *vine_task_graph_get_proxy_function_name(const struct vine_task_graph *tg); - -/** Get the heavy score of a node in the task graph. -@param tg Reference to the task graph object. -@param node_key Reference to the node key. -@return The heavy score. -*/ -double vine_task_graph_get_node_heavy_score(const struct vine_task_graph *tg, const char *node_key); - -/** Execute the task graph. -@param tg Reference to the task graph object. -*/ -void vine_task_graph_execute(struct vine_task_graph *tg); - -/** Get the local outfile source of a node in the task graph. -@param tg Reference to the task graph object. -@param node_key Reference to the node key. -@return The local outfile source. -*/ -const char *vine_task_graph_get_node_local_outfile_source(const struct vine_task_graph *tg, const char *node_key); - -/** Set the failure injection step percent. -@param tg Reference to the task graph object. -@param percent Reference to the failure injection step percent. -*/ -void vine_task_graph_set_failure_injection_step_percent(struct vine_task_graph *tg, double percent); +void vine_temp_set_replica_placement_policy(struct vine_manager *q, vine_replica_placement_policy_t policy); //@} diff --git a/taskvine/src/manager/vine_manager.c b/taskvine/src/manager/vine_manager.c index 9ecde5a14a..44841b4534 100644 --- a/taskvine/src/manager/vine_manager.c +++ b/taskvine/src/manager/vine_manager.c @@ -4375,7 +4375,7 @@ struct vine_manager *vine_ssl_create(int port, const char *key, const char *cert q->when_last_offloaded = 0; q->peak_used_cache = 0; q->shutting_down = 0; - vine_set_replica_placement_policy(q, VINE_REPLICA_PLACEMENT_POLICY_RANDOM); + vine_temp_set_replica_placement_policy(q, VINE_REPLICA_PLACEMENT_POLICY_RANDOM); if ((envstring = getenv("VINE_BANDWIDTH"))) { q->bandwidth_limit = string_metric_parse(envstring); diff --git a/taskvine/src/manager/vine_temp.c b/taskvine/src/manager/vine_temp.c index f6663d0989..3d14dd0e91 100644 --- a/taskvine/src/manager/vine_temp.c +++ b/taskvine/src/manager/vine_temp.c @@ -225,7 +225,7 @@ int vine_temp_handle_file_lost(struct vine_manager *q, char *cachename) return 1; } -void vine_set_replica_placement_policy(struct vine_manager *q, vine_replica_placement_policy_t policy) +void vine_temp_set_replica_placement_policy(struct vine_manager *q, vine_replica_placement_policy_t policy) { if (!q) { return; From 6bf5de352f5dcdbb0434f0433dc002615e05b0aa Mon Sep 17 00:00:00 2001 From: Jin Zhou Date: Thu, 16 Oct 2025 15:17:49 -0400 Subject: [PATCH 038/113] new --- .../python3/ndcctools/taskvine/utils.py | 17 +++- taskvine/src/graph/dagvine/cdagvine.py | 96 ++++++++++++++++++- taskvine/src/graph/dagvine/library.py | 5 +- taskvine/src/graph/dagvine/vine_task_graph.h | 15 ++- taskvine/src/graph/dagvine/vine_task_graph.i | 2 + taskvine/src/graph/dagvine/vine_task_node.h | 10 +- taskvine/src/manager/taskvine.h | 23 ++--- 7 files changed, 139 insertions(+), 29 deletions(-) diff --git a/taskvine/src/bindings/python3/ndcctools/taskvine/utils.py b/taskvine/src/bindings/python3/ndcctools/taskvine/utils.py index 4ced92406a..aa2c588357 100644 --- a/taskvine/src/bindings/python3/ndcctools/taskvine/utils.py +++ b/taskvine/src/bindings/python3/ndcctools/taskvine/utils.py @@ -3,6 +3,11 @@ # See the file COPYING for details. from . import cvine +try: + # cdagvine contains DAG-specific enums (e.g., VINE_TASK_PRIORITY_MODE_*) + from .dagvine import cdagvine # type: ignore +except Exception: + cdagvine = None import os @@ -12,7 +17,17 @@ def get_c_constant(constant): "result_success" -> VINE_RESULT_SUCCESS """ constant = f"VINE_{constant.upper()}" - return getattr(cvine, constant) + # First try the base cvine module + value = getattr(cvine, constant, None) + if value is not None: + return value + # Fallback to cdagvine if available (for DAG-specific constants) + if cdagvine is not None: + value = getattr(cdagvine, constant, None) + if value is not None: + return value + # If still missing, raise a clear error + raise AttributeError(f"C constant {constant} not found in cvine or cdagvine") def set_port_range(port): diff --git a/taskvine/src/graph/dagvine/cdagvine.py b/taskvine/src/graph/dagvine/cdagvine.py index 920389fda9..5222c14fa6 100644 --- a/taskvine/src/graph/dagvine/cdagvine.py +++ b/taskvine/src/graph/dagvine/cdagvine.py @@ -59,6 +59,80 @@ class _SwigNonDynamicMeta(type): import ndcctools.taskvine.cvine +VINE_NODE_OUTFILE_TYPE_LOCAL = _cdagvine.VINE_NODE_OUTFILE_TYPE_LOCAL +VINE_NODE_OUTFILE_TYPE_TEMP = _cdagvine.VINE_NODE_OUTFILE_TYPE_TEMP +VINE_NODE_OUTFILE_TYPE_SHARED_FILE_SYSTEM = _cdagvine.VINE_NODE_OUTFILE_TYPE_SHARED_FILE_SYSTEM +PRUNE_STATUS_NOT_PRUNED = _cdagvine.PRUNE_STATUS_NOT_PRUNED +PRUNE_STATUS_SAFE = _cdagvine.PRUNE_STATUS_SAFE +PRUNE_STATUS_UNSAFE = _cdagvine.PRUNE_STATUS_UNSAFE +class vine_task_node(object): + thisown = property(lambda x: x.this.own(), lambda x, v: x.this.own(v), doc="The membership flag") + __repr__ = _swig_repr + node_key = property(_cdagvine.vine_task_node_node_key_get, _cdagvine.vine_task_node_node_key_set) + manager = property(_cdagvine.vine_task_node_manager_get, _cdagvine.vine_task_node_manager_set) + task = property(_cdagvine.vine_task_node_task_get, _cdagvine.vine_task_node_task_set) + infile = property(_cdagvine.vine_task_node_infile_get, _cdagvine.vine_task_node_infile_set) + outfile = property(_cdagvine.vine_task_node_outfile_get, _cdagvine.vine_task_node_outfile_set) + outfile_remote_name = property(_cdagvine.vine_task_node_outfile_remote_name_get, _cdagvine.vine_task_node_outfile_remote_name_set) + staging_dir = property(_cdagvine.vine_task_node_staging_dir_get, _cdagvine.vine_task_node_staging_dir_set) + outfile_size_bytes = property(_cdagvine.vine_task_node_outfile_size_bytes_get, _cdagvine.vine_task_node_outfile_size_bytes_set) + parents = property(_cdagvine.vine_task_node_parents_get, _cdagvine.vine_task_node_parents_set) + children = property(_cdagvine.vine_task_node_children_get, _cdagvine.vine_task_node_children_set) + pending_parents = property(_cdagvine.vine_task_node_pending_parents_get, _cdagvine.vine_task_node_pending_parents_set) + retry_attempts_left = property(_cdagvine.vine_task_node_retry_attempts_left_get, _cdagvine.vine_task_node_retry_attempts_left_set) + completed = property(_cdagvine.vine_task_node_completed_get, _cdagvine.vine_task_node_completed_set) + prune_depth = property(_cdagvine.vine_task_node_prune_depth_get, _cdagvine.vine_task_node_prune_depth_set) + depth = property(_cdagvine.vine_task_node_depth_get, _cdagvine.vine_task_node_depth_set) + height = property(_cdagvine.vine_task_node_height_get, _cdagvine.vine_task_node_height_set) + upstream_subgraph_size = property(_cdagvine.vine_task_node_upstream_subgraph_size_get, _cdagvine.vine_task_node_upstream_subgraph_size_set) + downstream_subgraph_size = property(_cdagvine.vine_task_node_downstream_subgraph_size_get, _cdagvine.vine_task_node_downstream_subgraph_size_set) + fan_in = property(_cdagvine.vine_task_node_fan_in_get, _cdagvine.vine_task_node_fan_in_set) + fan_out = property(_cdagvine.vine_task_node_fan_out_get, _cdagvine.vine_task_node_fan_out_set) + heavy_score = property(_cdagvine.vine_task_node_heavy_score_get, _cdagvine.vine_task_node_heavy_score_set) + critical_time = property(_cdagvine.vine_task_node_critical_time_get, _cdagvine.vine_task_node_critical_time_set) + time_spent_on_unlink_local_files = property(_cdagvine.vine_task_node_time_spent_on_unlink_local_files_get, _cdagvine.vine_task_node_time_spent_on_unlink_local_files_set) + time_spent_on_prune_ancestors_of_temp_node = property(_cdagvine.vine_task_node_time_spent_on_prune_ancestors_of_temp_node_get, _cdagvine.vine_task_node_time_spent_on_prune_ancestors_of_temp_node_set) + time_spent_on_prune_ancestors_of_persisted_node = property(_cdagvine.vine_task_node_time_spent_on_prune_ancestors_of_persisted_node_get, _cdagvine.vine_task_node_time_spent_on_prune_ancestors_of_persisted_node_set) + outfile_type = property(_cdagvine.vine_task_node_outfile_type_get, _cdagvine.vine_task_node_outfile_type_set) + prune_status = property(_cdagvine.vine_task_node_prune_status_get, _cdagvine.vine_task_node_prune_status_set) + + def __init__(self): + _cdagvine.vine_task_node_swiginit(self, _cdagvine.new_vine_task_node()) + __swig_destroy__ = _cdagvine.delete_vine_task_node + +# Register vine_task_node in _cdagvine: +_cdagvine.vine_task_node_swigregister(vine_task_node) + +def vine_task_node_create(manager, node_key, proxy_library_name, proxy_function_name, staging_dir, prune_depth): + return _cdagvine.vine_task_node_create(manager, node_key, proxy_library_name, proxy_function_name, staging_dir, prune_depth) + +def vine_task_node_delete(node): + return _cdagvine.vine_task_node_delete(node) + +def compute_lex_priority(key): + return _cdagvine.compute_lex_priority(key) + +def vine_task_node_prune_ancestors(node): + return _cdagvine.vine_task_node_prune_ancestors(node) + +def vine_task_node_print_info(node): + return _cdagvine.vine_task_node_print_info(node) + +def vine_task_node_update_critical_time(node, execution_time): + return _cdagvine.vine_task_node_update_critical_time(node, execution_time) + +def vine_task_node_replicate_outfile(node): + return _cdagvine.vine_task_node_replicate_outfile(node) + +def vine_task_node_set_outfile(node, outfile_type, outfile_remote_name): + return _cdagvine.vine_task_node_set_outfile(node, outfile_type, outfile_remote_name) +VINE_TASK_PRIORITY_MODE_RANDOM = _cdagvine.VINE_TASK_PRIORITY_MODE_RANDOM +VINE_TASK_PRIORITY_MODE_DEPTH_FIRST = _cdagvine.VINE_TASK_PRIORITY_MODE_DEPTH_FIRST +VINE_TASK_PRIORITY_MODE_BREADTH_FIRST = _cdagvine.VINE_TASK_PRIORITY_MODE_BREADTH_FIRST +VINE_TASK_PRIORITY_MODE_FIFO = _cdagvine.VINE_TASK_PRIORITY_MODE_FIFO +VINE_TASK_PRIORITY_MODE_LIFO = _cdagvine.VINE_TASK_PRIORITY_MODE_LIFO +VINE_TASK_PRIORITY_MODE_LARGEST_INPUT_FIRST = _cdagvine.VINE_TASK_PRIORITY_MODE_LARGEST_INPUT_FIRST +VINE_TASK_PRIORITY_MODE_LARGEST_STORAGE_FOOTPRINT_FIRST = _cdagvine.VINE_TASK_PRIORITY_MODE_LARGEST_STORAGE_FOOTPRINT_FIRST class vine_task_graph(object): thisown = property(lambda x: x.this.own(), lambda x, v: x.this.own(v), doc="The membership flag") __repr__ = _swig_repr @@ -102,6 +176,24 @@ def vine_task_graph_get_node_heavy_score(tg, node_key): def vine_task_graph_set_node_outfile(tg, node_key, outfile_type, outfile_remote_name): return _cdagvine.vine_task_graph_set_node_outfile(tg, node_key, outfile_type, outfile_remote_name) -def vine_task_graph_set_proxy_library_and_function_names(tg, proxy_library_name, proxy_function_name): - return _cdagvine.vine_task_graph_set_proxy_library_and_function_names(tg, proxy_library_name, proxy_function_name) +def vine_task_graph_execute(tg): + return _cdagvine.vine_task_graph_execute(tg) + +def vine_task_graph_get_node_local_outfile_source(tg, node_key): + return _cdagvine.vine_task_graph_get_node_local_outfile_source(tg, node_key) + +def vine_task_graph_delete(tg): + return _cdagvine.vine_task_graph_delete(tg) + +def vine_task_graph_get_proxy_library_name(tg): + return _cdagvine.vine_task_graph_get_proxy_library_name(tg) + +def vine_task_graph_get_proxy_function_name(tg): + return _cdagvine.vine_task_graph_get_proxy_function_name(tg) + +def vine_task_graph_set_proxy_library_name(tg, proxy_library_name): + return _cdagvine.vine_task_graph_set_proxy_library_name(tg, proxy_library_name) + +def vine_task_graph_set_proxy_function_name(tg, proxy_function_name): + return _cdagvine.vine_task_graph_set_proxy_function_name(tg, proxy_function_name) diff --git a/taskvine/src/graph/dagvine/library.py b/taskvine/src/graph/dagvine/library.py index f66badf033..7a441e5aba 100644 --- a/taskvine/src/graph/dagvine/library.py +++ b/taskvine/src/graph/dagvine/library.py @@ -1,5 +1,6 @@ import os, uuid, cloudpickle, types, time, random, hashlib, collections from ndcctools.taskvine import cvine +from ndcctools.taskvine.dagvine import cdagvine from ndcctools.taskvine.graph_definition import ( GraphKeyResult, TaskGraph, compute_dts_key, compute_sexpr_key, compute_single_key, hash_name, hashable, init_task_graph_context @@ -29,7 +30,7 @@ def add_env_files(self, new_env_files): def install(self, manager, libcores, vine_graph): self.libcores = libcores - assert cvine.vine_task_graph_get_proxy_function_name(vine_graph) == compute_single_key.__name__ + assert cdagvine.vine_task_graph_get_proxy_function_name(vine_graph) == compute_single_key.__name__ self.local_path = f"library-task-graph-{uuid.uuid4()}.pkl" self.remote_path = self.local_path @@ -38,7 +39,7 @@ def install(self, manager, libcores, vine_graph): os, cloudpickle, GraphKeyResult, TaskGraph, uuid, hashlib, random, types, collections, time, load_variable_from_library, compute_dts_key, compute_sexpr_key, compute_single_key, hash_name, hashable ] - lib_name = cvine.vine_task_graph_get_proxy_library_name(vine_graph) + lib_name = cdagvine.vine_task_graph_get_proxy_library_name(vine_graph) self.libtask = manager.create_library_from_functions( lib_name, compute_single_key, diff --git a/taskvine/src/graph/dagvine/vine_task_graph.h b/taskvine/src/graph/dagvine/vine_task_graph.h index 5fd81fdd6c..d0160d9051 100644 --- a/taskvine/src/graph/dagvine/vine_task_graph.h +++ b/taskvine/src/graph/dagvine/vine_task_graph.h @@ -9,12 +9,17 @@ #include "vine_task_node.h" #include "taskvine.h" -/** Select the type of the node-output file. */ + +/** Select priority algorithm for task graph task scheduling. */ typedef enum { - VINE_NODE_OUTFILE_TYPE_LOCAL = 0, /* Node-output file will be stored locally on the manager's staging directory */ - VINE_NODE_OUTFILE_TYPE_TEMP, /* Node-output file will be stored in the temporary node-local storage */ - VINE_NODE_OUTFILE_TYPE_SHARED_FILE_SYSTEM, /* Node-output file will be stored in the persistent shared file system */ -} vine_task_node_outfile_type_t; + VINE_TASK_PRIORITY_MODE_RANDOM = 0, /**< Assign random priority to tasks */ + VINE_TASK_PRIORITY_MODE_DEPTH_FIRST, /**< Prioritize deeper tasks first */ + VINE_TASK_PRIORITY_MODE_BREADTH_FIRST, /**< Prioritize shallower tasks first */ + VINE_TASK_PRIORITY_MODE_FIFO, /**< First in, first out priority */ + VINE_TASK_PRIORITY_MODE_LIFO, /**< Last in, first out priority */ + VINE_TASK_PRIORITY_MODE_LARGEST_INPUT_FIRST, /**< Prioritize tasks with larger inputs first */ + VINE_TASK_PRIORITY_MODE_LARGEST_STORAGE_FOOTPRINT_FIRST /**< Prioritize tasks with larger storage footprint first */ +} vine_task_priority_mode_t; /** The task graph object. */ struct vine_task_graph { diff --git a/taskvine/src/graph/dagvine/vine_task_graph.i b/taskvine/src/graph/dagvine/vine_task_graph.i index b2a86b0c69..adc36a9b93 100644 --- a/taskvine/src/graph/dagvine/vine_task_graph.i +++ b/taskvine/src/graph/dagvine/vine_task_graph.i @@ -4,6 +4,7 @@ %{ #include "int_sizes.h" #include "vine_task_graph.h" +#include "vine_task_node.h" /* expose outfile type enum to SWIG */ %} %include "stdint.i" @@ -13,6 +14,7 @@ %import "../../bindings/python3/taskvine.i" /* Expose only the dagvine task graph APIs */ +%include "vine_task_node.h" /* export vine_task_node_outfile_type_t values */ %include "vine_task_graph.h" diff --git a/taskvine/src/graph/dagvine/vine_task_node.h b/taskvine/src/graph/dagvine/vine_task_node.h index 12fea69230..5b6b00272a 100644 --- a/taskvine/src/graph/dagvine/vine_task_node.h +++ b/taskvine/src/graph/dagvine/vine_task_node.h @@ -8,9 +8,16 @@ #include "set.h" #include "taskvine.h" +/** Select the type of the node-output file. */ +typedef enum { + VINE_NODE_OUTFILE_TYPE_LOCAL = 0, /* Node-output file will be stored locally on the manager's staging directory */ + VINE_NODE_OUTFILE_TYPE_TEMP, /* Node-output file will be stored in the temporary node-local storage */ + VINE_NODE_OUTFILE_TYPE_SHARED_FILE_SYSTEM, /* Node-output file will be stored in the persistent shared file system */ +} vine_task_node_outfile_type_t; + typedef enum { PRUNE_STATUS_NOT_PRUNED = 0, - PRUNE_STATUS_SAFE, + PRUNE_STATUS_SAFE, PRUNE_STATUS_UNSAFE } prune_status_t; @@ -66,6 +73,5 @@ void vine_task_node_print_info(struct vine_task_node *node); void vine_task_node_update_critical_time(struct vine_task_node *node, timestamp_t execution_time); void vine_task_node_replicate_outfile(struct vine_task_node *node); void vine_task_node_set_outfile(struct vine_task_node *node, vine_task_node_outfile_type_t outfile_type, const char *outfile_remote_name); -void vine_task_node_checkpoint_outfile(struct vine_task_node *node); #endif \ No newline at end of file diff --git a/taskvine/src/manager/taskvine.h b/taskvine/src/manager/taskvine.h index 380d584d60..eb2ed880e0 100644 --- a/taskvine/src/manager/taskvine.h +++ b/taskvine/src/manager/taskvine.h @@ -156,17 +156,6 @@ typedef enum { VINE_REPLICA_PLACEMENT_POLICY_TRANSFER_LOAD /* select a worker with the least incoming transfer load */ } vine_replica_placement_policy_t; -/** Select priority algorithm for task graph task scheduling. */ -typedef enum { - VINE_TASK_PRIORITY_MODE_RANDOM = 0, /**< Assign random priority to tasks */ - VINE_TASK_PRIORITY_MODE_DEPTH_FIRST, /**< Prioritize deeper tasks first */ - VINE_TASK_PRIORITY_MODE_BREADTH_FIRST, /**< Prioritize shallower tasks first */ - VINE_TASK_PRIORITY_MODE_FIFO, /**< First in, first out priority */ - VINE_TASK_PRIORITY_MODE_LIFO, /**< Last in, first out priority */ - VINE_TASK_PRIORITY_MODE_LARGEST_INPUT_FIRST, /**< Prioritize tasks with larger inputs first */ - VINE_TASK_PRIORITY_MODE_LARGEST_STORAGE_FOOTPRINT_FIRST /**< Prioritize tasks with larger storage footprint first */ -} vine_task_priority_mode_t; - /** Statistics describing a manager. */ struct vine_stats { /* Stats for the current state of workers: */ @@ -1143,6 +1132,12 @@ int vine_enable_return_recovery_tasks(struct vine_manager *m); Recovery tasks will be handled internally by the manager. **/ int vine_disable_return_recovery_tasks(struct vine_manager *m); +/** Set the replica placement policy (which worker do we prefer to place the redundant file replicas). +@param q Reference to the current manager object. +@param policy Reference to the replica placement policy. +*/ +void vine_temp_set_replica_placement_policy(struct vine_manager *q, vine_replica_placement_policy_t policy); + /** 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 @@ -1582,12 +1577,6 @@ char *vine_get_path_library_log(struct vine_manager *m, const char *path); */ char *vine_get_path_cache(struct vine_manager *m, const char *path); -/** Set the replica placement policy (which worker do we prefer to place the redundant file replicas). -@param q Reference to the current manager object. -@param policy Reference to the replica placement policy. -*/ -void vine_temp_set_replica_placement_policy(struct vine_manager *q, vine_replica_placement_policy_t policy); - //@} #endif From 20328154c137f6b4ddc6c5bdc7840bcd5a5f4a27 Mon Sep 17 00:00:00 2001 From: Jin Zhou Date: Thu, 16 Oct 2025 16:12:17 -0400 Subject: [PATCH 039/113] vine task graph tuning --- taskvine/src/bindings/python3/Makefile | 2 +- .../python3/ndcctools/taskvine/utils.py | 2 +- taskvine/src/graph/dagvine/.gitignore | 3 +- taskvine/src/graph/dagvine/Makefile | 41 ++-- taskvine/src/graph/dagvine/cdagvine.py | 199 ------------------ taskvine/src/graph/dagvine/config.py | 37 ---- taskvine/src/graph/dagvine/cvine_dagvine.py | 4 +- taskvine/src/graph/dagvine/graph_executor.py | 38 ++-- taskvine/src/graph/dagvine/params.py | 8 +- taskvine/src/graph/dagvine/vine_task_graph.c | 94 +++++---- taskvine/src/graph/dagvine/vine_task_graph.h | 37 ++-- taskvine/src/graph/dagvine/vine_task_graph.i | 1 + taskvine/src/manager/taskvine.h | 13 -- taskvine/src/manager/vine_manager.c | 1 - taskvine/src/manager/vine_manager.h | 1 - taskvine/src/manager/vine_temp.c | 37 +--- 16 files changed, 116 insertions(+), 402 deletions(-) delete mode 100644 taskvine/src/graph/dagvine/cdagvine.py diff --git a/taskvine/src/bindings/python3/Makefile b/taskvine/src/bindings/python3/Makefile index 8b9f1bf14e..24557af1b0 100644 --- a/taskvine/src/bindings/python3/Makefile +++ b/taskvine/src/bindings/python3/Makefile @@ -7,7 +7,7 @@ GRAPH_EXECUTOR_MODULE_NAME = dagvine CCTOOLS_DYNAMIC_SUFFIX = so # SWIG produces code that causes a lot of warnings, so use -w to turn those off. LOCAL_CCFLAGS = -w -fPIC -DNDEBUG $(CCTOOLS_PYTHON3_CCFLAGS) -I ../../manager -I ../../graph/${GRAPH_EXECUTOR_MODULE_NAME} -LOCAL_LINKAGE = $(CCTOOLS_PYTHON3_LDFLAGS) -lz $(CCTOOLS_OPENSSL_LDFLAGS) $(CCTOOLS_HOME)/taskvine/src/graph/${GRAPH_EXECUTOR_MODULE_NAME}/lib${GRAPH_EXECUTOR_MODULE_NAME}.a $(CCTOOLS_HOME)/taskvine/src/manager/libtaskvine.a $(CCTOOLS_HOME)/dttools/src/libdttools.a +LOCAL_LINKAGE = $(CCTOOLS_PYTHON3_LDFLAGS) -lz $(CCTOOLS_OPENSSL_LDFLAGS) $(CCTOOLS_HOME)/taskvine/src/manager/libtaskvine.a $(CCTOOLS_HOME)/dttools/src/libdttools.a CCTOOLS_FLAKE8_IGNORE_FILES = "cvine.py" diff --git a/taskvine/src/bindings/python3/ndcctools/taskvine/utils.py b/taskvine/src/bindings/python3/ndcctools/taskvine/utils.py index aa2c588357..bd909b664b 100644 --- a/taskvine/src/bindings/python3/ndcctools/taskvine/utils.py +++ b/taskvine/src/bindings/python3/ndcctools/taskvine/utils.py @@ -4,7 +4,7 @@ from . import cvine try: - # cdagvine contains DAG-specific enums (e.g., VINE_TASK_PRIORITY_MODE_*) + # cdagvine contains DAG-specific enums (e.g., VINE_TASK_GRAPH_PRIORITY_MODE_*) from .dagvine import cdagvine # type: ignore except Exception: cdagvine = None diff --git a/taskvine/src/graph/dagvine/.gitignore b/taskvine/src/graph/dagvine/.gitignore index 3e98d74473..38280b8491 100644 --- a/taskvine/src/graph/dagvine/.gitignore +++ b/taskvine/src/graph/dagvine/.gitignore @@ -2,4 +2,5 @@ *.so *.o *_wrap.c -*_wrap.0 \ No newline at end of file +*_wrap.0 +build/ \ No newline at end of file diff --git a/taskvine/src/graph/dagvine/Makefile b/taskvine/src/graph/dagvine/Makefile index ba1b84c43b..018a48a227 100644 --- a/taskvine/src/graph/dagvine/Makefile +++ b/taskvine/src/graph/dagvine/Makefile @@ -4,51 +4,62 @@ include ../../../../rules.mk LOCAL_LINKAGE+=${CCTOOLS_HOME}/taskvine/src/manager/libtaskvine.a ${CCTOOLS_HOME}/dttools/src/libdttools.a LOCAL_CCFLAGS+=-I ${CCTOOLS_HOME}/taskvine/src/manager + + SOURCES = vine_task_node.c vine_task_graph.c OBJECTS = $(SOURCES:%.c=%.o) SWIG_I = vine_task_graph.i -SWIG_WRAP = vine_task_graph_wrap.c -PYMODULE = _cdagvine.$(CCTOOLS_DYNAMIC_SUFFIX) -LIBRARIES = libdagvine.a +# put SWIG generated sources and Python extension artifacts into build/ directory +BUILD_DIR := build +SWIG_WRAP = $(BUILD_DIR)/vine_task_graph_wrap.c +WRAP_OBJ = $(BUILD_DIR)/vine_task_graph_wrap.o +PYMODULE = $(BUILD_DIR)/_cdagvine.$(CCTOOLS_DYNAMIC_SUFFIX) + +LIBRARIES = +PYDEPS = $(WRAP_OBJ) $(OBJECTS) +PYLINK_INPUT = $(WRAP_OBJ) $(OBJECTS) PROGRAMS = SCRIPTS = TARGETS = $(LIBRARIES) $(PYMODULE) $(PROGRAMS) all: $(TARGETS) -libdagvine.a: $(OBJECTS) + $(PROGRAMS): $(EXTERNALS) -$(SWIG_WRAP): $(SWIG_I) vine_task_graph.h - $(CCTOOLS_SWIG) -python -threads -relativeimport -I$(CCTOOLS_HOME)/taskvine/src/manager -I$(CCTOOLS_HOME)/dttools/src -o $@ $< +$(BUILD_DIR): + mkdir -p $(BUILD_DIR) + +$(SWIG_WRAP): $(SWIG_I) vine_task_graph.h | $(BUILD_DIR) + $(CCTOOLS_SWIG) -python -threads -relativeimport -I$(CCTOOLS_HOME)/taskvine/src/manager -I$(CCTOOLS_HOME)/dttools/src -outdir $(BUILD_DIR) -o $@ $< # Build Python module -vine_task_graph_wrap.o: $(SWIG_WRAP) - $(CCTOOLS_CC) -o $@ -c $(CCTOOLS_INTERNAL_CCFLAGS) $(CCTOOLS_PYTHON3_CCFLAGS) $(LOCAL_CCFLAGS) $< +$(WRAP_OBJ): $(SWIG_WRAP) + $(CCTOOLS_CC) -o $@ -c $(CCTOOLS_INTERNAL_CCFLAGS) $(CCTOOLS_PYTHON3_CCFLAGS) $(LOCAL_CCFLAGS) -Wno-error=cast-function-type $< -$(PYMODULE): vine_task_graph_wrap.o libdagvine.a +$(PYMODULE): $(PYDEPS) ifeq ($(CCTOOLS_STATIC),1) - $(CCTOOLS_LD) -o $@ $(CCTOOLS_DYNAMIC_FLAG) $(CCTOOLS_INTERNAL_LDFLAGS) $(LOCAL_LDFLAGS) $^ $(LOCAL_LINKAGE) $(CCTOOLS_PYTHON3_LDFLAGS) $(CCTOOLS_EXTERNAL_LINKAGE) + $(CCTOOLS_LD) -o $@ $(CCTOOLS_DYNAMIC_FLAG) $(CCTOOLS_INTERNAL_LDFLAGS) $(LOCAL_LDFLAGS) $(PYLINK_INPUT) $(LOCAL_LINKAGE) $(CCTOOLS_PYTHON3_LDFLAGS) $(CCTOOLS_EXTERNAL_LINKAGE) else - $(CCTOOLS_LD) -o $@ $(CCTOOLS_DYNAMIC_FLAG) $(CCTOOLS_INTERNAL_LDFLAGS) $(LOCAL_LDFLAGS) $^ $(LOCAL_LINKAGE) $(CCTOOLS_PYTHON3_LDFLAGS) $(CCTOOLS_EXTERNAL_LINKAGE) + $(CCTOOLS_LD) -o $@ $(CCTOOLS_DYNAMIC_FLAG) $(CCTOOLS_INTERNAL_LDFLAGS) $(LOCAL_LDFLAGS) $(PYLINK_INPUT) $(LOCAL_LINKAGE) $(CCTOOLS_PYTHON3_LDFLAGS) $(CCTOOLS_EXTERNAL_LINKAGE) endif install: all - mkdir -p $(CCTOOLS_INSTALL_DIR)/lib - cp $(LIBRARIES) $(CCTOOLS_INSTALL_DIR)/lib/ mkdir -p $(CCTOOLS_INSTALL_DIR)/graph/dagvine/include cp ${CCTOOLS_HOME}/taskvine/src/manager/taskvine.h $(CCTOOLS_INSTALL_DIR)/graph/dagvine/include/ mkdir -p $(CCTOOLS_PYTHON3_PATH)/ndcctools/taskvine/dagvine cp $(PYMODULE) $(CCTOOLS_PYTHON3_PATH)/ndcctools/taskvine/dagvine/ - cp cdagvine.py $(CCTOOLS_PYTHON3_PATH)/ndcctools/taskvine/dagvine/ + cp $(BUILD_DIR)/cdagvine.py $(CCTOOLS_PYTHON3_PATH)/ndcctools/taskvine/dagvine/ + # remove build artifacts after installation + rm -rf $(BUILD_DIR) clean: - rm -rf $(PROGRAMS) $(OBJECTS) *.o $(SWIG_WRAP) vine_task_graph_wrap.o $(PYMODULE) cdagvine.py + rm -rf $(PROGRAMS) $(OBJECTS) *.o $(BUILD_DIR) $(PYMODULE) cdagvine.py test: all diff --git a/taskvine/src/graph/dagvine/cdagvine.py b/taskvine/src/graph/dagvine/cdagvine.py deleted file mode 100644 index 5222c14fa6..0000000000 --- a/taskvine/src/graph/dagvine/cdagvine.py +++ /dev/null @@ -1,199 +0,0 @@ -# This file was automatically generated by SWIG (https://www.swig.org). -# Version 4.3.1 -# -# Do not make changes to this file unless you know what you are doing - modify -# the SWIG interface file instead. - -from sys import version_info as _swig_python_version_info -# Import the low-level C/C++ module -if __package__ or "." in __name__: - from . import _cdagvine -else: - import _cdagvine - -try: - import builtins as __builtin__ -except ImportError: - import __builtin__ - -def _swig_repr(self): - try: - strthis = "proxy of " + self.this.__repr__() - except __builtin__.Exception: - strthis = "" - return "<%s.%s; %s >" % (self.__class__.__module__, self.__class__.__name__, strthis,) - - -def _swig_setattr_nondynamic_instance_variable(set): - def set_instance_attr(self, name, value): - if name == "this": - set(self, name, value) - elif name == "thisown": - self.this.own(value) - elif hasattr(self, name) and isinstance(getattr(type(self), name), property): - set(self, name, value) - else: - raise AttributeError("You cannot add instance attributes to %s" % self) - return set_instance_attr - - -def _swig_setattr_nondynamic_class_variable(set): - def set_class_attr(cls, name, value): - if hasattr(cls, name) and not isinstance(getattr(cls, name), property): - set(cls, name, value) - else: - raise AttributeError("You cannot add class attributes to %s" % cls) - return set_class_attr - - -def _swig_add_metaclass(metaclass): - """Class decorator for adding a metaclass to a SWIG wrapped class - a slimmed down version of six.add_metaclass""" - def wrapper(cls): - return metaclass(cls.__name__, cls.__bases__, cls.__dict__.copy()) - return wrapper - - -class _SwigNonDynamicMeta(type): - """Meta class to enforce nondynamic attributes (no new attributes) for a class""" - __setattr__ = _swig_setattr_nondynamic_class_variable(type.__setattr__) - - -import ndcctools.taskvine.cvine -VINE_NODE_OUTFILE_TYPE_LOCAL = _cdagvine.VINE_NODE_OUTFILE_TYPE_LOCAL -VINE_NODE_OUTFILE_TYPE_TEMP = _cdagvine.VINE_NODE_OUTFILE_TYPE_TEMP -VINE_NODE_OUTFILE_TYPE_SHARED_FILE_SYSTEM = _cdagvine.VINE_NODE_OUTFILE_TYPE_SHARED_FILE_SYSTEM -PRUNE_STATUS_NOT_PRUNED = _cdagvine.PRUNE_STATUS_NOT_PRUNED -PRUNE_STATUS_SAFE = _cdagvine.PRUNE_STATUS_SAFE -PRUNE_STATUS_UNSAFE = _cdagvine.PRUNE_STATUS_UNSAFE -class vine_task_node(object): - thisown = property(lambda x: x.this.own(), lambda x, v: x.this.own(v), doc="The membership flag") - __repr__ = _swig_repr - node_key = property(_cdagvine.vine_task_node_node_key_get, _cdagvine.vine_task_node_node_key_set) - manager = property(_cdagvine.vine_task_node_manager_get, _cdagvine.vine_task_node_manager_set) - task = property(_cdagvine.vine_task_node_task_get, _cdagvine.vine_task_node_task_set) - infile = property(_cdagvine.vine_task_node_infile_get, _cdagvine.vine_task_node_infile_set) - outfile = property(_cdagvine.vine_task_node_outfile_get, _cdagvine.vine_task_node_outfile_set) - outfile_remote_name = property(_cdagvine.vine_task_node_outfile_remote_name_get, _cdagvine.vine_task_node_outfile_remote_name_set) - staging_dir = property(_cdagvine.vine_task_node_staging_dir_get, _cdagvine.vine_task_node_staging_dir_set) - outfile_size_bytes = property(_cdagvine.vine_task_node_outfile_size_bytes_get, _cdagvine.vine_task_node_outfile_size_bytes_set) - parents = property(_cdagvine.vine_task_node_parents_get, _cdagvine.vine_task_node_parents_set) - children = property(_cdagvine.vine_task_node_children_get, _cdagvine.vine_task_node_children_set) - pending_parents = property(_cdagvine.vine_task_node_pending_parents_get, _cdagvine.vine_task_node_pending_parents_set) - retry_attempts_left = property(_cdagvine.vine_task_node_retry_attempts_left_get, _cdagvine.vine_task_node_retry_attempts_left_set) - completed = property(_cdagvine.vine_task_node_completed_get, _cdagvine.vine_task_node_completed_set) - prune_depth = property(_cdagvine.vine_task_node_prune_depth_get, _cdagvine.vine_task_node_prune_depth_set) - depth = property(_cdagvine.vine_task_node_depth_get, _cdagvine.vine_task_node_depth_set) - height = property(_cdagvine.vine_task_node_height_get, _cdagvine.vine_task_node_height_set) - upstream_subgraph_size = property(_cdagvine.vine_task_node_upstream_subgraph_size_get, _cdagvine.vine_task_node_upstream_subgraph_size_set) - downstream_subgraph_size = property(_cdagvine.vine_task_node_downstream_subgraph_size_get, _cdagvine.vine_task_node_downstream_subgraph_size_set) - fan_in = property(_cdagvine.vine_task_node_fan_in_get, _cdagvine.vine_task_node_fan_in_set) - fan_out = property(_cdagvine.vine_task_node_fan_out_get, _cdagvine.vine_task_node_fan_out_set) - heavy_score = property(_cdagvine.vine_task_node_heavy_score_get, _cdagvine.vine_task_node_heavy_score_set) - critical_time = property(_cdagvine.vine_task_node_critical_time_get, _cdagvine.vine_task_node_critical_time_set) - time_spent_on_unlink_local_files = property(_cdagvine.vine_task_node_time_spent_on_unlink_local_files_get, _cdagvine.vine_task_node_time_spent_on_unlink_local_files_set) - time_spent_on_prune_ancestors_of_temp_node = property(_cdagvine.vine_task_node_time_spent_on_prune_ancestors_of_temp_node_get, _cdagvine.vine_task_node_time_spent_on_prune_ancestors_of_temp_node_set) - time_spent_on_prune_ancestors_of_persisted_node = property(_cdagvine.vine_task_node_time_spent_on_prune_ancestors_of_persisted_node_get, _cdagvine.vine_task_node_time_spent_on_prune_ancestors_of_persisted_node_set) - outfile_type = property(_cdagvine.vine_task_node_outfile_type_get, _cdagvine.vine_task_node_outfile_type_set) - prune_status = property(_cdagvine.vine_task_node_prune_status_get, _cdagvine.vine_task_node_prune_status_set) - - def __init__(self): - _cdagvine.vine_task_node_swiginit(self, _cdagvine.new_vine_task_node()) - __swig_destroy__ = _cdagvine.delete_vine_task_node - -# Register vine_task_node in _cdagvine: -_cdagvine.vine_task_node_swigregister(vine_task_node) - -def vine_task_node_create(manager, node_key, proxy_library_name, proxy_function_name, staging_dir, prune_depth): - return _cdagvine.vine_task_node_create(manager, node_key, proxy_library_name, proxy_function_name, staging_dir, prune_depth) - -def vine_task_node_delete(node): - return _cdagvine.vine_task_node_delete(node) - -def compute_lex_priority(key): - return _cdagvine.compute_lex_priority(key) - -def vine_task_node_prune_ancestors(node): - return _cdagvine.vine_task_node_prune_ancestors(node) - -def vine_task_node_print_info(node): - return _cdagvine.vine_task_node_print_info(node) - -def vine_task_node_update_critical_time(node, execution_time): - return _cdagvine.vine_task_node_update_critical_time(node, execution_time) - -def vine_task_node_replicate_outfile(node): - return _cdagvine.vine_task_node_replicate_outfile(node) - -def vine_task_node_set_outfile(node, outfile_type, outfile_remote_name): - return _cdagvine.vine_task_node_set_outfile(node, outfile_type, outfile_remote_name) -VINE_TASK_PRIORITY_MODE_RANDOM = _cdagvine.VINE_TASK_PRIORITY_MODE_RANDOM -VINE_TASK_PRIORITY_MODE_DEPTH_FIRST = _cdagvine.VINE_TASK_PRIORITY_MODE_DEPTH_FIRST -VINE_TASK_PRIORITY_MODE_BREADTH_FIRST = _cdagvine.VINE_TASK_PRIORITY_MODE_BREADTH_FIRST -VINE_TASK_PRIORITY_MODE_FIFO = _cdagvine.VINE_TASK_PRIORITY_MODE_FIFO -VINE_TASK_PRIORITY_MODE_LIFO = _cdagvine.VINE_TASK_PRIORITY_MODE_LIFO -VINE_TASK_PRIORITY_MODE_LARGEST_INPUT_FIRST = _cdagvine.VINE_TASK_PRIORITY_MODE_LARGEST_INPUT_FIRST -VINE_TASK_PRIORITY_MODE_LARGEST_STORAGE_FOOTPRINT_FIRST = _cdagvine.VINE_TASK_PRIORITY_MODE_LARGEST_STORAGE_FOOTPRINT_FIRST -class vine_task_graph(object): - thisown = property(lambda x: x.this.own(), lambda x, v: x.this.own(v), doc="The membership flag") - __repr__ = _swig_repr - manager = property(_cdagvine.vine_task_graph_manager_get, _cdagvine.vine_task_graph_manager_set) - nodes = property(_cdagvine.vine_task_graph_nodes_get, _cdagvine.vine_task_graph_nodes_set) - task_id_to_node = property(_cdagvine.vine_task_graph_task_id_to_node_get, _cdagvine.vine_task_graph_task_id_to_node_set) - outfile_cachename_to_node = property(_cdagvine.vine_task_graph_outfile_cachename_to_node_get, _cdagvine.vine_task_graph_outfile_cachename_to_node_set) - proxy_library_name = property(_cdagvine.vine_task_graph_proxy_library_name_get, _cdagvine.vine_task_graph_proxy_library_name_set) - proxy_function_name = property(_cdagvine.vine_task_graph_proxy_function_name_get, _cdagvine.vine_task_graph_proxy_function_name_set) - task_priority_mode = property(_cdagvine.vine_task_graph_task_priority_mode_get, _cdagvine.vine_task_graph_task_priority_mode_set) - failure_injection_step_percent = property(_cdagvine.vine_task_graph_failure_injection_step_percent_get, _cdagvine.vine_task_graph_failure_injection_step_percent_set) - - def __init__(self): - _cdagvine.vine_task_graph_swiginit(self, _cdagvine.new_vine_task_graph()) - __swig_destroy__ = _cdagvine.delete_vine_task_graph - -# Register vine_task_graph in _cdagvine: -_cdagvine.vine_task_graph_swigregister(vine_task_graph) - -def vine_task_graph_create(q): - return _cdagvine.vine_task_graph_create(q) - -def vine_task_graph_set_task_priority_mode(tg, priority_mode): - return _cdagvine.vine_task_graph_set_task_priority_mode(tg, priority_mode) - -def vine_task_graph_set_failure_injection_step_percent(tg, percent): - return _cdagvine.vine_task_graph_set_failure_injection_step_percent(tg, percent) - -def vine_task_graph_add_node(tg, node_key, staging_dir, prune_depth): - return _cdagvine.vine_task_graph_add_node(tg, node_key, staging_dir, prune_depth) - -def vine_task_graph_add_dependency(tg, parent_key, child_key): - return _cdagvine.vine_task_graph_add_dependency(tg, parent_key, child_key) - -def vine_task_graph_compute_topology_metrics(tg): - return _cdagvine.vine_task_graph_compute_topology_metrics(tg) - -def vine_task_graph_get_node_heavy_score(tg, node_key): - return _cdagvine.vine_task_graph_get_node_heavy_score(tg, node_key) - -def vine_task_graph_set_node_outfile(tg, node_key, outfile_type, outfile_remote_name): - return _cdagvine.vine_task_graph_set_node_outfile(tg, node_key, outfile_type, outfile_remote_name) - -def vine_task_graph_execute(tg): - return _cdagvine.vine_task_graph_execute(tg) - -def vine_task_graph_get_node_local_outfile_source(tg, node_key): - return _cdagvine.vine_task_graph_get_node_local_outfile_source(tg, node_key) - -def vine_task_graph_delete(tg): - return _cdagvine.vine_task_graph_delete(tg) - -def vine_task_graph_get_proxy_library_name(tg): - return _cdagvine.vine_task_graph_get_proxy_library_name(tg) - -def vine_task_graph_get_proxy_function_name(tg): - return _cdagvine.vine_task_graph_get_proxy_function_name(tg) - -def vine_task_graph_set_proxy_library_name(tg, proxy_library_name): - return _cdagvine.vine_task_graph_set_proxy_library_name(tg, proxy_library_name) - -def vine_task_graph_set_proxy_function_name(tg, proxy_function_name): - return _cdagvine.vine_task_graph_set_proxy_function_name(tg, proxy_function_name) - diff --git a/taskvine/src/graph/dagvine/config.py b/taskvine/src/graph/dagvine/config.py index e3b8a41fec..e69de29bb2 100644 --- a/taskvine/src/graph/dagvine/config.py +++ b/taskvine/src/graph/dagvine/config.py @@ -1,37 +0,0 @@ -# config.py -# 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 dataclasses import dataclass, field - - -@dataclass(frozen=True) -class DefaultTuneParams: - worker_source_max_transfers: int = 100 - max_retrievals: int = -1 - prefer_dispatch: int = 1 - transient_error_interval: int = 1 - attempt_schedule_depth: int = 1000 - - -@dataclass(frozen=True) -class DefaultPaths: - shared_file_system_dir: str = "/project01/ndcms/jzhou24/shared_file_system" - staging_dir: str = "/project01/ndcms/jzhou24/staging" - - -@dataclass(frozen=True) -class DefaultPolicies: - replica_placement_policy: str = "random" - priority_mode: str = "largest-input-first" - scheduling_mode: str = "files" - prune_depth: int = 1 - - -@dataclass(frozen=True) -class DefaultOutfileType: - outfile_type: dict = field(default_factory=lambda: { - "temp": 1.0, - "shared-file-system": 0.0, - }) diff --git a/taskvine/src/graph/dagvine/cvine_dagvine.py b/taskvine/src/graph/dagvine/cvine_dagvine.py index c705763a26..df154836f1 100644 --- a/taskvine/src/graph/dagvine/cvine_dagvine.py +++ b/taskvine/src/graph/dagvine/cvine_dagvine.py @@ -81,8 +81,8 @@ def __init__(self): def vine_task_graph_create(q): return _cvine_dagvine.vine_task_graph_create(q) -def vine_task_graph_set_task_priority_mode(tg, priority_mode): - return _cvine_dagvine.vine_task_graph_set_task_priority_mode(tg, priority_mode) +def vine_task_graph_set_priority_mode(tg, priority_mode): + return _cvine_dagvine.vine_task_graph_set_priority_mode(tg, priority_mode) def vine_task_graph_set_failure_injection_step_percent(tg, percent): return _cvine_dagvine.vine_task_graph_set_failure_injection_step_percent(tg, percent) diff --git a/taskvine/src/graph/dagvine/graph_executor.py b/taskvine/src/graph/dagvine/graph_executor.py index c0ae5757a6..34e1c0a64a 100644 --- a/taskvine/src/graph/dagvine/graph_executor.py +++ b/taskvine/src/graph/dagvine/graph_executor.py @@ -75,39 +75,30 @@ def __init__(self, vine_constant_params=VineConstantParams(), **kwargs): - # Use C-side SIGINT handler in cdagvine to handle Ctrl+C gracefully + # handle SIGINT correctly + signal.signal(signal.SIGINT, self._on_sigint) self.manager_tuning_params = manager_tuning_params self.vine_constant_params = vine_constant_params self.regular_params = regular_params # delete all files in the run info directory, do this before super().__init__() - run_info_path = self.regular_params.run_info_path - run_info_template = self.regular_params.run_info_template - self.run_info_path_absolute = os.path.join(run_info_path, run_info_template) - if run_info_path and run_info_template: - delete_all_files(self.run_info_path_absolute) - - kwargs["run_info_path"] = run_info_path - kwargs["run_info_template"] = run_info_template - super().__init__(*args, **kwargs) - print(f"TaskVine Manager \033[92m{self.name}\033[0m listening on port \033[92m{self.port}\033[0m") + 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: + delete_all_files(self.run_info_template_path) - # initialize the task graph - self._vine_task_graph = cdagvine.vine_task_graph_create(self._taskvine) + # initialize the manager + super().__init__(*args, **kwargs) def tune_manager(self): for k, v in self.manager_tuning_params.to_dict().items(): print(f"Tuning {k} to {v}") self.tune(k, v) - def set_policy(self): - # set replica placement policy - cvine.vine_temp_set_replica_placement_policy(self._taskvine, self.vine_constant_params.get_c_constant_of("replica_placement_policy")) # set worker scheduling algorithm cvine.vine_set_scheduler(self._taskvine, self.vine_constant_params.get_c_constant_of("schedule")) - # set task priority mode - cdagvine.vine_task_graph_set_task_priority_mode(self._vine_task_graph, self.vine_constant_params.get_c_constant_of("task_priority_mode")) def run(self, collection_dict, @@ -115,10 +106,13 @@ def run(self, library=None, ): + # initialize the task graph (note: the lifetime of this object is limited to this function) + self._vine_task_graph = cdagvine.vine_task_graph_create(self._taskvine) + cdagvine.vine_task_graph_tune(self._vine_task_graph, "failure-injection-step-percent", str(self.regular_params.failure_injection_step_percent)) + cdagvine.vine_task_graph_tune(self._vine_task_graph, "task-priority-mode", self.vine_constant_params.task_priority_mode) + # tune the manager every time we start a new run as the parameters may have changed self.tune_manager() - # set the policy every time we start a new run as the parameters may have changed - self.set_policy() self.target_keys = target_keys self.task_dict = ensure_task_dict(collection_dict) @@ -127,8 +121,6 @@ def run(self, self.library = library self.library.install(self, self.regular_params.libcores, self._vine_task_graph) - cdagvine.vine_task_graph_set_failure_injection_step_percent(self._vine_task_graph, self.regular_params.failure_injection_step_percent) - # create task graph in the python side print("Initializing TaskGraph object") self.task_graph = TaskGraph(self.task_dict, @@ -182,7 +174,7 @@ def run(self, cloudpickle.dump(self.task_graph, f) # now execute the vine graph - print(f"\033[92mExecuting task graph, logs will be written into {self.run_info_path_absolute}\033[0m") + print(f"\033[92mExecuting task graph, logs will be written into {self.run_info_template_path}\033[0m") cdagvine.vine_task_graph_execute(self._vine_task_graph) # after execution, we need to load results of target keys diff --git a/taskvine/src/graph/dagvine/params.py b/taskvine/src/graph/dagvine/params.py index 1f4a363760..41a81e23fc 100644 --- a/taskvine/src/graph/dagvine/params.py +++ b/taskvine/src/graph/dagvine/params.py @@ -42,14 +42,12 @@ class VineConstantParams: If there is a need to use these values in the C code, convert them to uppercase and call the get_c_constant_of method to get the C constant. """ - replica_placement_policy: str = "random" schedule: str = "worst" - task_priority_mode: str = "largest_input_first" + task_priority_mode: str = "largest-input-first" valid_normalized_values = { - "replica_placement_policy": {"random", "disk_load", "transfer_load"}, "schedule": {"files", "time", "rand", "worst", "disk"}, - "task_priority_mode": {"random", "depth_first", "breadth_first", "fifo", "lifo", "largest_input_first", "largest_storage_footprint_first"}, + "task_priority_mode": {"random", "depth-first", "breadth-first", "fifo", "lifo", "largest-input-first", "largest-storage-footprint-first"}, } def normalize(self, obj): @@ -83,8 +81,6 @@ def get_c_constant_of(self, key): @dataclass class RegularParams: """Regular parameters that will be used directly by the graph executor.""" - run_info_path: str = None - run_info_template: str = None libcores: int = 16 failure_injection_step_percent: int = -1 prune_depth: int = 1 diff --git a/taskvine/src/graph/dagvine/vine_task_graph.c b/taskvine/src/graph/dagvine/vine_task_graph.c index 829a3fb4e9..2cb1574a73 100644 --- a/taskvine/src/graph/dagvine/vine_task_graph.c +++ b/taskvine/src/graph/dagvine/vine_task_graph.c @@ -51,7 +51,7 @@ static void handle_sigint(int signal) * @param priority_mode Reference to the priority mode. * @return The priority. */ -static double vine_task_node_calculate_priority(struct vine_task_node *node, vine_task_priority_mode_t priority_mode) +static double vine_task_node_calculate_priority(struct vine_task_node *node, vine_task_graph_priority_mode_t priority_mode) { if (!node) { return 0; @@ -63,22 +63,22 @@ static double vine_task_node_calculate_priority(struct vine_task_node *node, vin struct vine_task_node *parent_node; switch (priority_mode) { - case VINE_TASK_PRIORITY_MODE_RANDOM: + case VINE_TASK_GRAPH_PRIORITY_MODE_RANDOM: priority = random_double(); break; - case VINE_TASK_PRIORITY_MODE_DEPTH_FIRST: + case VINE_TASK_GRAPH_PRIORITY_MODE_DEPTH_FIRST: priority = (double)node->depth; break; - case VINE_TASK_PRIORITY_MODE_BREADTH_FIRST: + case VINE_TASK_GRAPH_PRIORITY_MODE_BREADTH_FIRST: priority = -(double)node->depth; break; - case VINE_TASK_PRIORITY_MODE_FIFO: + case VINE_TASK_GRAPH_PRIORITY_MODE_FIFO: priority = -(double)current_time; break; - case VINE_TASK_PRIORITY_MODE_LIFO: + case VINE_TASK_GRAPH_PRIORITY_MODE_LIFO: priority = (double)current_time; break; - case VINE_TASK_PRIORITY_MODE_LARGEST_INPUT_FIRST: + case VINE_TASK_GRAPH_PRIORITY_MODE_LARGEST_INPUT_FIRST: LIST_ITERATE(node->parents, parent_node) { if (!parent_node->outfile) { @@ -87,7 +87,7 @@ static double vine_task_node_calculate_priority(struct vine_task_node *node, vin priority += (double)vine_file_size(parent_node->outfile); } break; - case VINE_TASK_PRIORITY_MODE_LARGEST_STORAGE_FOOTPRINT_FIRST: + case VINE_TASK_GRAPH_PRIORITY_MODE_LARGEST_STORAGE_FOOTPRINT_FIRST: LIST_ITERATE(node->parents, parent_node) { if (!parent_node->outfile) { @@ -348,6 +348,46 @@ static struct vine_task_node *get_node_by_task(struct vine_task_graph *tg, struc /* Public APIs */ /*************************************************************/ +/** Tune the task graph. + *@param tg Reference to the task graph object. + *@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_task_graph_tune(struct vine_task_graph *tg, const char *name, const char *value) +{ + if (!tg || !name || !value) { + return -1; + } + + if (strcmp(name, "failure-injection-step-percent") == 0) { + tg->failure_injection_step_percent = atof(value); + + } else if (strcmp(name, "task-priority-mode") == 0) { + if (strcmp(value, "random") == 0) { + tg->task_priority_mode = VINE_TASK_GRAPH_PRIORITY_MODE_RANDOM; + } else if (strcmp(value, "depth-first") == 0) { + tg->task_priority_mode = VINE_TASK_GRAPH_PRIORITY_MODE_DEPTH_FIRST; + } else if (strcmp(value, "breadth-first") == 0) { + tg->task_priority_mode = VINE_TASK_GRAPH_PRIORITY_MODE_BREADTH_FIRST; + } else if (strcmp(value, "fifo") == 0) { + tg->task_priority_mode = VINE_TASK_GRAPH_PRIORITY_MODE_FIFO; + } else if (strcmp(value, "lifo") == 0) { + tg->task_priority_mode = VINE_TASK_GRAPH_PRIORITY_MODE_LIFO; + } else if (strcmp(value, "largest-input-first") == 0) { + tg->task_priority_mode = VINE_TASK_GRAPH_PRIORITY_MODE_LARGEST_INPUT_FIRST; + } else if (strcmp(value, "largest-storage-footprint-first") == 0) { + tg->task_priority_mode = VINE_TASK_GRAPH_PRIORITY_MODE_LARGEST_STORAGE_FOOTPRINT_FIRST; + } else { + debug(D_ERROR, "invalid priority mode: %s", value); + return -1; + } + + } + + return 0; +} + /** * Set the proxy library name (Python-side), shared by all tasks. * @param tg Reference to the task graph object. @@ -626,7 +666,7 @@ struct vine_task_graph *vine_task_graph_create(struct vine_manager *q) tg->proxy_function_name = xxstrdup("compute_single_key"); // Python-side proxy function name (shared by all tasks) tg->manager = q; - tg->task_priority_mode = VINE_TASK_PRIORITY_MODE_LARGEST_INPUT_FIRST; + tg->task_priority_mode = VINE_TASK_GRAPH_PRIORITY_MODE_LARGEST_INPUT_FIRST; tg->failure_injection_step_percent = -1.0; /* enable debug system for C code since it uses a separate debug system instance @@ -638,42 +678,6 @@ struct vine_task_graph *vine_task_graph_create(struct vine_manager *q) return tg; } -/** - * Set the failure injection step percent, meaning we will evict a randome worker at every X% of the DAG completion. - * @param tg Reference to the task graph object. - * @param percent Reference to the failure injection step percent. - */ -void vine_task_graph_set_failure_injection_step_percent(struct vine_task_graph *tg, double percent) -{ - if (!tg) { - return; - } - - if (percent <= 0 || percent > 100) { - return; - } - - debug(D_VINE, "setting failure injection step percent to %lf", percent); - tg->failure_injection_step_percent = percent; - - return; -} - -/** - * Set the task priority mode for the task graph. - * @param tg Reference to the task graph object. - * @param priority_mode Reference to the priority mode. - */ -void vine_task_graph_set_task_priority_mode(struct vine_task_graph *tg, vine_task_priority_mode_t priority_mode) -{ - if (!tg) { - return; - } - - tg->task_priority_mode = priority_mode; - return; -} - /** * Add a dependency between two nodes in the task graph. Note that the input-output file relationship * is not handled here, because their file names may have not been determined yet. diff --git a/taskvine/src/graph/dagvine/vine_task_graph.h b/taskvine/src/graph/dagvine/vine_task_graph.h index d0160d9051..6f4b39b398 100644 --- a/taskvine/src/graph/dagvine/vine_task_graph.h +++ b/taskvine/src/graph/dagvine/vine_task_graph.h @@ -12,14 +12,14 @@ /** Select priority algorithm for task graph task scheduling. */ typedef enum { - VINE_TASK_PRIORITY_MODE_RANDOM = 0, /**< Assign random priority to tasks */ - VINE_TASK_PRIORITY_MODE_DEPTH_FIRST, /**< Prioritize deeper tasks first */ - VINE_TASK_PRIORITY_MODE_BREADTH_FIRST, /**< Prioritize shallower tasks first */ - VINE_TASK_PRIORITY_MODE_FIFO, /**< First in, first out priority */ - VINE_TASK_PRIORITY_MODE_LIFO, /**< Last in, first out priority */ - VINE_TASK_PRIORITY_MODE_LARGEST_INPUT_FIRST, /**< Prioritize tasks with larger inputs first */ - VINE_TASK_PRIORITY_MODE_LARGEST_STORAGE_FOOTPRINT_FIRST /**< Prioritize tasks with larger storage footprint first */ -} vine_task_priority_mode_t; + VINE_TASK_GRAPH_PRIORITY_MODE_RANDOM = 0, /**< Assign random priority to tasks */ + VINE_TASK_GRAPH_PRIORITY_MODE_DEPTH_FIRST, /**< Prioritize deeper tasks first */ + VINE_TASK_GRAPH_PRIORITY_MODE_BREADTH_FIRST, /**< Prioritize shallower tasks first */ + VINE_TASK_GRAPH_PRIORITY_MODE_FIFO, /**< First in, first out priority */ + VINE_TASK_GRAPH_PRIORITY_MODE_LIFO, /**< Last in, first out priority */ + VINE_TASK_GRAPH_PRIORITY_MODE_LARGEST_INPUT_FIRST, /**< Prioritize tasks with larger inputs first */ + VINE_TASK_GRAPH_PRIORITY_MODE_LARGEST_STORAGE_FOOTPRINT_FIRST /**< Prioritize tasks with larger storage footprint first */ +} vine_task_graph_priority_mode_t; /** The task graph object. */ struct vine_task_graph { @@ -31,7 +31,7 @@ struct vine_task_graph { char *proxy_library_name; char *proxy_function_name; - vine_task_priority_mode_t task_priority_mode; // priority mode for task graph task scheduling + vine_task_graph_priority_mode_t task_priority_mode; // priority mode for task graph task scheduling double failure_injection_step_percent; // 0 - 100, the percentage of steps to inject failure }; @@ -44,18 +44,6 @@ struct vine_task_graph { */ struct vine_task_graph *vine_task_graph_create(struct vine_manager *q); -/** Set the task priority mode for the task graph. -@param tg Reference to the task graph object. -@param priority_mode Reference to the priority mode. -*/ -void vine_task_graph_set_task_priority_mode(struct vine_task_graph *tg, vine_task_priority_mode_t priority_mode); - -/** Set the failure injection step percent. -@param tg Reference to the task graph object. -@param percent Reference to the failure injection step percent. -*/ -void vine_task_graph_set_failure_injection_step_percent(struct vine_task_graph *tg, double percent); - /** Create a new node in the task graph. @param tg Reference to the task graph object. @param node_key Reference to the node key. @@ -136,5 +124,12 @@ void vine_task_graph_set_proxy_library_name(struct vine_task_graph *tg, const ch */ void vine_task_graph_set_proxy_function_name(struct vine_task_graph *tg, const char *proxy_function_name); +/** Tune the task graph. +@param tg Reference to the task graph object. +@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_task_graph_tune(struct vine_task_graph *tg, const char *name, const char *value); #endif // VINE_TASK_GRAPH_H diff --git a/taskvine/src/graph/dagvine/vine_task_graph.i b/taskvine/src/graph/dagvine/vine_task_graph.i index adc36a9b93..4f1f2d95fd 100644 --- a/taskvine/src/graph/dagvine/vine_task_graph.i +++ b/taskvine/src/graph/dagvine/vine_task_graph.i @@ -14,6 +14,7 @@ %import "../../bindings/python3/taskvine.i" /* Expose only the dagvine task graph APIs */ +%ignore vine_task_node_checkpoint_outfile; /* avoid exporting unimplemented/optional symbol */ %include "vine_task_node.h" /* export vine_task_node_outfile_type_t values */ %include "vine_task_graph.h" diff --git a/taskvine/src/manager/taskvine.h b/taskvine/src/manager/taskvine.h index eb2ed880e0..eda05ebc4d 100644 --- a/taskvine/src/manager/taskvine.h +++ b/taskvine/src/manager/taskvine.h @@ -149,13 +149,6 @@ typedef enum { VINE_MINI_TASK, /**< A file obtained by executing a Unix command line. */ } vine_file_type_t; -/* Select while type of workers to place the redundant file replicas */ -typedef enum { - VINE_REPLICA_PLACEMENT_POLICY_RANDOM = 0, /* select a random worker */ - VINE_REPLICA_PLACEMENT_POLICY_DISK_LOAD, /* select a worker with the most free disk space */ - VINE_REPLICA_PLACEMENT_POLICY_TRANSFER_LOAD /* select a worker with the least incoming transfer load */ -} vine_replica_placement_policy_t; - /** Statistics describing a manager. */ struct vine_stats { /* Stats for the current state of workers: */ @@ -1132,12 +1125,6 @@ int vine_enable_return_recovery_tasks(struct vine_manager *m); Recovery tasks will be handled internally by the manager. **/ int vine_disable_return_recovery_tasks(struct vine_manager *m); -/** Set the replica placement policy (which worker do we prefer to place the redundant file replicas). -@param q Reference to the current manager object. -@param policy Reference to the replica placement policy. -*/ -void vine_temp_set_replica_placement_policy(struct vine_manager *q, vine_replica_placement_policy_t policy); - /** 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 diff --git a/taskvine/src/manager/vine_manager.c b/taskvine/src/manager/vine_manager.c index 44841b4534..9953df9450 100644 --- a/taskvine/src/manager/vine_manager.c +++ b/taskvine/src/manager/vine_manager.c @@ -4375,7 +4375,6 @@ struct vine_manager *vine_ssl_create(int port, const char *key, const char *cert q->when_last_offloaded = 0; q->peak_used_cache = 0; q->shutting_down = 0; - vine_temp_set_replica_placement_policy(q, VINE_REPLICA_PLACEMENT_POLICY_RANDOM); if ((envstring = getenv("VINE_BANDWIDTH"))) { q->bandwidth_limit = string_metric_parse(envstring); diff --git a/taskvine/src/manager/vine_manager.h b/taskvine/src/manager/vine_manager.h index 8b4fb9826a..92482dc08c 100644 --- a/taskvine/src/manager/vine_manager.h +++ b/taskvine/src/manager/vine_manager.h @@ -234,7 +234,6 @@ struct vine_manager { int return_recovery_tasks; /* If true, recovery tasks are returned by vine_wait to the user. By default they are handled internally. */ int num_submitted_recovery_tasks; - vine_replica_placement_policy_t replica_placement_policy; /* Mode for selecting best worker for placing a new replica of a temp file */ int balance_worker_disk_load; /* If true, offload replicas from workers that are overloaded with temp files. */ timestamp_t when_last_offloaded; int64_t peak_used_cache; diff --git a/taskvine/src/manager/vine_temp.c b/taskvine/src/manager/vine_temp.c index 3d14dd0e91..2f171f5efe 100644 --- a/taskvine/src/manager/vine_temp.c +++ b/taskvine/src/manager/vine_temp.c @@ -87,19 +87,8 @@ static struct vine_worker_info *get_best_dest_worker(struct vine_manager *q, str if ((int64_t)f->size > available_disk_space) { continue; } - /* workers with more available disk space are preferred */ + /* workers with more available disk space are preferred to hold the file */ priority_queue_push(valid_destinations, w, available_disk_space); - switch (q->replica_placement_policy) { - case VINE_REPLICA_PLACEMENT_POLICY_RANDOM: - priority_queue_push(valid_destinations, w, random_double()); - break; - case VINE_REPLICA_PLACEMENT_POLICY_DISK_LOAD: - priority_queue_push(valid_destinations, w, available_disk_space); - break; - case VINE_REPLICA_PLACEMENT_POLICY_TRANSFER_LOAD: - priority_queue_push(valid_destinations, w, -w->incoming_xfer_counter); - break; - } } struct vine_worker_info *best_destination = priority_queue_pop(valid_destinations); @@ -224,27 +213,3 @@ int vine_temp_handle_file_lost(struct vine_manager *q, char *cachename) return 1; } - -void vine_temp_set_replica_placement_policy(struct vine_manager *q, vine_replica_placement_policy_t policy) -{ - if (!q) { - return; - } - - switch (policy) { - case VINE_REPLICA_PLACEMENT_POLICY_RANDOM: - debug(D_VINE | D_NOTICE, "Setting replica placement policy to RANDOM"); - q->replica_placement_policy = policy; - break; - case VINE_REPLICA_PLACEMENT_POLICY_DISK_LOAD: - debug(D_VINE | D_NOTICE, "Setting replica placement policy to DISK_LOAD"); - q->replica_placement_policy = policy; - break; - case VINE_REPLICA_PLACEMENT_POLICY_TRANSFER_LOAD: - debug(D_VINE | D_NOTICE, "Setting replica placement policy to TRANSFER_LOAD"); - q->replica_placement_policy = policy; - break; - default: - debug(D_ERROR, "Invalid replica placement policy: %d", policy); - } -} From 642d144e1e0653dc186dd55fcc69897cccdffb6b Mon Sep 17 00:00:00 2001 From: Jin Zhou Date: Fri, 17 Oct 2025 16:47:34 -0400 Subject: [PATCH 040/113] new version --- .../src/graph/dagvine/graph_definition.py | 28 +- taskvine/src/graph/dagvine/graph_executor.py | 276 ++++++++++++------ taskvine/src/graph/dagvine/library.py | 64 ++-- taskvine/src/graph/dagvine/vine_task_graph.c | 103 +++---- taskvine/src/graph/dagvine/vine_task_graph.h | 31 +- taskvine/src/graph/dagvine/vine_task_node.c | 39 ++- taskvine/src/graph/dagvine/vine_task_node.h | 4 +- 7 files changed, 333 insertions(+), 212 deletions(-) diff --git a/taskvine/src/graph/dagvine/graph_definition.py b/taskvine/src/graph/dagvine/graph_definition.py index 1bf509f72d..3dcead36df 100644 --- a/taskvine/src/graph/dagvine/graph_definition.py +++ b/taskvine/src/graph/dagvine/graph_definition.py @@ -93,12 +93,10 @@ def __init__(self, result, extra_size_mb=None): class TaskGraph: def __init__(self, task_dict, shared_file_system_dir=None, - staging_dir=None, extra_task_output_size_mb=["uniform", 0, 0], extra_task_sleep_time=["uniform", 0, 0]): self.task_dict = task_dict self.shared_file_system_dir = shared_file_system_dir - self.staging_dir = staging_dir if self.shared_file_system_dir: os.makedirs(self.shared_file_system_dir, exist_ok=True) @@ -248,6 +246,18 @@ def get_topological_order(self): return topo_order + def load_result_of_target_key(self, target_results_dir, key): + assert self.outfile_type[key] == "local", "Only local output files are supported for target keys" + outfile_path = os.path.join(target_results_dir, self.outfile_remote_name[key]) + if not os.path.exists(outfile_path): + result = "NOT_FOUND" + else: + with open(outfile_path, "rb") as f: + result_obj = cloudpickle.load(f) + assert isinstance(result_obj, GraphKeyResult), "Loaded object is not of type GraphKeyResult" + result = result_obj.result + return result + def __del__(self): if hasattr(self, 'outfile_remote_name') and self.outfile_remote_name: for k in self.outfile_remote_name.keys(): @@ -255,18 +265,14 @@ def __del__(self): os.remove(self.outfile_remote_name[k]) -def init_task_graph_context(task_graph_path=None): - if task_graph_path is None: - - raise ValueError("task_graph_path must be provided to initialize the task graph context") - if not os.path.exists(task_graph_path): - raise FileNotFoundError(f"Task graph file not found at {task_graph_path}") +def task_graph_loader(task_graph_pkl): + task_graph = cloudpickle.loads(task_graph_pkl) - with open(task_graph_path, 'rb') as f: - task_graph = cloudpickle.load(f) + if not isinstance(task_graph, TaskGraph): + raise TypeError("Task graph is not of type TaskGraph") return { - 'task_graph': task_graph, + "task_graph": task_graph, } diff --git a/taskvine/src/graph/dagvine/graph_executor.py b/taskvine/src/graph/dagvine/graph_executor.py index 34e1c0a64a..8393323359 100644 --- a/taskvine/src/graph/dagvine/graph_executor.py +++ b/taskvine/src/graph/dagvine/graph_executor.py @@ -1,12 +1,15 @@ + from ndcctools.taskvine import cvine from ndcctools.taskvine.dagvine import cdagvine from ndcctools.taskvine.manager import Manager from ndcctools.taskvine.utils import delete_all_files, get_c_constant from ndcctools.taskvine.dagvine.graph_definition import GraphKeyResult, TaskGraph -from ndcctools.taskvine.dagvine.params import ManagerTuningParams -from ndcctools.taskvine.dagvine.params import VineConstantParams -from ndcctools.taskvine.dagvine.params import RegularParams +from ndcctools.taskvine.dagvine.library import Library +from ndcctools.taskvine.dagvine.graph_definition import ( + GraphKeyResult, TaskGraph, compute_dts_key, compute_sexpr_key, + compute_single_key, hash_name, hashable, task_graph_loader +) import cloudpickle import os @@ -67,20 +70,69 @@ def ensure_task_dict(collection_dict): return task_dict +class GraphParams: + def __init__(self): + self.vine_manager_tuning_params = { + "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, + "balance-worker-disk-load": 0, + } + self.vine_task_graph_tuning_params = { + "failure-injection-step-percent": -1, + "task-priority-mode": "largest-input-first", + "proxy-library-name": "vine_task_graph_library", + "proxy-function-name": "compute_single_key", + "prune-depth": 1, + } + self.other_params = { + "schedule": "worst", + "libcores": 16, + "failure-injection-step-percent": -1, + "target-results-dir": "./target_results", + "shared-file-system-dir": "./shared_file_system", + "extra-task-output-size-mb": ["uniform", 0, 0], + "extra-task-sleep-time": ["uniform", 0, 0], + "outfile-type": { + "temp": 1.0, + "shared-file-system": 0.0, + }, + } + + def update_param(self, param_name, new_value): + if param_name in self.vine_manager_tuning_params: + self.vine_manager_tuning_params[param_name] = new_value + elif param_name in self.vine_task_graph_tuning_params: + self.vine_task_graph_tuning_params[param_name] = new_value + elif param_name in self.other_params: + self.other_params[param_name] = new_value + else: + self.vine_manager_tuning_params[param_name] = new_value + + def get_value_of(self, param_name): + if param_name in self.vine_manager_tuning_params: + return self.vine_manager_tuning_params[param_name] + elif param_name in self.vine_task_graph_tuning_params: + return self.vine_task_graph_tuning_params[param_name] + elif param_name in self.other_params: + return self.other_params[param_name] + else: + raise ValueError(f"Invalid param name: {param_name}") + + class GraphExecutor(Manager): def __init__(self, *args, - manager_tuning_params=ManagerTuningParams(), - regular_params=RegularParams(), - vine_constant_params=VineConstantParams(), **kwargs): # handle SIGINT correctly signal.signal(signal.SIGINT, self._on_sigint) - self.manager_tuning_params = manager_tuning_params - self.vine_constant_params = vine_constant_params - self.regular_params = regular_params + self.params = GraphParams() # delete all files in the run info directory, do this before super().__init__() run_info_path = kwargs.get("run_info_path", None) @@ -92,6 +144,8 @@ def __init__(self, # initialize the manager super().__init__(*args, **kwargs) + self._vine_task_graph = None + def tune_manager(self): for k, v in self.manager_tuning_params.to_dict().items(): print(f"Tuning {k} to {v}") @@ -100,102 +154,138 @@ def tune_manager(self): # set worker scheduling algorithm cvine.vine_set_scheduler(self._taskvine, self.vine_constant_params.get_c_constant_of("schedule")) - def run(self, - collection_dict, - target_keys=[], - library=None, - ): + if self._vine_task_graph: + cdagvine.vine_task_graph_tune(self._vine_task_graph, "failure-injection-step-percent", str(self.regular_params.failure_injection_step_percent)) + cdagvine.vine_task_graph_tune(self._vine_task_graph, "task-priority-mode", self.vine_constant_params.task_priority_mode) - # initialize the task graph (note: the lifetime of this object is limited to this function) - self._vine_task_graph = cdagvine.vine_task_graph_create(self._taskvine) - cdagvine.vine_task_graph_tune(self._vine_task_graph, "failure-injection-step-percent", str(self.regular_params.failure_injection_step_percent)) - cdagvine.vine_task_graph_tune(self._vine_task_graph, "task-priority-mode", self.vine_constant_params.task_priority_mode) + def param(self, param_name): + return self.params.get_value_of(param_name) - # tune the manager every time we start a new run as the parameters may have changed - self.tune_manager() + def update_params(self, new_params): + 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) - self.target_keys = target_keys - self.task_dict = ensure_task_dict(collection_dict) + def get_run_info_path(self): + return os.path.join(self.param("run-info-path"), self.param("run-info-template")) - # create library task - self.library = library - self.library.install(self, self.regular_params.libcores, self._vine_task_graph) - - # create task graph in the python side - print("Initializing TaskGraph object") - self.task_graph = TaskGraph(self.task_dict, - staging_dir=self.regular_params.staging_dir, - shared_file_system_dir=self.regular_params.shared_file_system_dir, - extra_task_output_size_mb=self.regular_params.extra_task_output_size_mb, - extra_task_sleep_time=self.regular_params.extra_task_sleep_time) - topo_order = self.task_graph.get_topological_order() - - # create task graph in the python side - print("Initializing task graph in TaskVine") - for k in topo_order: - cdagvine.vine_task_graph_add_node(self._vine_task_graph, - self.task_graph.vine_key_of[k], - self.regular_params.staging_dir, - self.regular_params.prune_depth) - for pk in self.task_graph.parents_of.get(k, []): - cdagvine.vine_task_graph_add_dependency(self._vine_task_graph, self.task_graph.vine_key_of[pk], self.task_graph.vine_key_of[k]) - - # we must finalize the graph in c side after all nodes and dependencies are added - # this includes computing various metrics for each node, such as depth, height, heavy score, etc. - cdagvine.vine_task_graph_compute_topology_metrics(self._vine_task_graph) - - # then we can use the heavy score to sort the nodes and specify their outfile remote names + def tune_vine_manager(self): + for k, v in self.params.vine_manager_tuning_params.items(): + print(f"Tuning {k} to {v}") + self.tune(k, v) + + def tune_vine_task_graph(self, c_graph): + for k, v in self.params.vine_task_graph_tuning_params.items(): + print(f"Tuning {k} to {v}") + cdagvine.vine_task_graph_tune(c_graph, k, str(v)) + + def assign_outfile_types(self, target_keys, py_graph, c_graph): + assert py_graph is not None, "Python graph must be built first" + assert c_graph is not None, "C graph must be built first" + + # get heavy score from C side heavy_scores = {} - for k in self.task_graph.task_dict.keys(): - heavy_scores[k] = cdagvine.vine_task_graph_get_node_heavy_score(self._vine_task_graph, self.task_graph.vine_key_of[k]) + for k in py_graph.task_dict.keys(): + heavy_scores[k] = cdagvine.vine_task_graph_get_node_heavy_score(c_graph, py_graph.vine_key_of[k]) + + # sort keys by heavy score descending + sorted_keys = sorted(heavy_scores, key=lambda k: heavy_scores[k], reverse=True) - # keys with larger heavy score should be stored into the shared file system - sorted_keys = sorted(heavy_scores, key=lambda x: heavy_scores[x], reverse=True) - shared_file_system_size = round(len(sorted_keys) * self.regular_params.outfile_type["shared-file-system"]) + # determine how many go to shared FS + sharedfs_count = round(self.param("outfile-type")["shared-file-system"] * len(sorted_keys)) + + # assign outfile types for i, k in enumerate(sorted_keys): - if k in self.target_keys: + if k in target_keys: choice = "local" + elif i < sharedfs_count: + choice = "shared-file-system" else: - if i < shared_file_system_size: - choice = "shared-file-system" - else: - choice = "temp" - # set on the Python side, will be installed on the remote workers - self.task_graph.set_outfile_type_of(k, choice) - # set on the C side, so the manager knows where the data is stored - outfile_type_str = f"NODE_OUTFILE_TYPE_{choice.upper().replace('-', '_')}" - cdagvine.vine_task_graph_set_node_outfile(self._vine_task_graph, - self.task_graph.vine_key_of[k], - get_c_constant(outfile_type_str), - self.task_graph.outfile_remote_name[k]) - - # save the task graph to a pickle file, will be sent to the remote workers - with open(self.library.local_path, 'wb') as f: - cloudpickle.dump(self.task_graph, f) - - # now execute the vine graph - print(f"\033[92mExecuting task graph, logs will be written into {self.run_info_template_path}\033[0m") - cdagvine.vine_task_graph_execute(self._vine_task_graph) - - # after execution, we need to load results of target keys + choice = "temp" + + py_graph.set_outfile_type_of(k, choice) + outfile_type_enum = get_c_constant(f"NODE_OUTFILE_TYPE_{choice.upper().replace('-', '_')}") + cdagvine.vine_task_graph_set_node_outfile( + c_graph, + py_graph.vine_key_of[k], + outfile_type_enum, + py_graph.outfile_remote_name[k] + ) + + def build_python_graph(self): + py_graph = TaskGraph( + self.task_dict, + shared_file_system_dir=self.param("shared-file-system-dir"), + extra_task_output_size_mb=self.param("extra-task-output-size-mb"), + extra_task_sleep_time=self.param("extra-task-sleep-time") + ) + + return py_graph + + def build_c_graph(self, py_graph): + assert py_graph is not None, "Python graph must be built before building the C graph" + + c_graph = cdagvine.vine_task_graph_create(self._taskvine) + + # C side vine task graph must be tuned before adding nodes and dependencies + self.tune_vine_manager() + self.tune_vine_task_graph(c_graph) + + topo_order = py_graph.get_topological_order() + for k in topo_order: + cdagvine.vine_task_graph_add_node( + c_graph, + py_graph.vine_key_of[k], + ) + for pk in py_graph.parents_of[k]: + cdagvine.vine_task_graph_add_dependency(c_graph, py_graph.vine_key_of[pk], py_graph.vine_key_of[k]) + + cdagvine.vine_task_graph_compute_topology_metrics(c_graph) + + return c_graph + + def build_graphs(self, target_keys): + # build Python DAG (logical topology) + py_graph = self.build_python_graph() + + # build C DAG (physical topology) + c_graph = self.build_c_graph(py_graph) + + # assign outfile types + self.assign_outfile_types(target_keys, py_graph, c_graph) + + return py_graph, c_graph + + def new_run(self, collection_dict, target_keys=None, params={}, hoisting_modules=[], env_files={}): + # first update the params so that they can be used for the following construction + self.update_params(params) + + self.task_dict = ensure_task_dict(collection_dict) + + # build graphs in both Python and C sides + py_graph, c_graph = self.build_graphs(target_keys) + + library = Library(self, self.param("proxy-library-name"), self.param("libcores")) + + library.add_hoisting_modules(hoisting_modules) + library.add_env_files(env_files) + library.set_context_loader(task_graph_loader, context_loader_args=[cloudpickle.dumps(py_graph)]) + library.install() + + cdagvine.vine_task_graph_execute(c_graph) + + # delete the C graph immediately after execution, so that the lifetime of this object is limited to the execution + cdagvine.vine_task_graph_delete(c_graph) + + # clean up the library instances and template on the manager + library.uninstall() + + # load results of target keys results = {} - for k in self.target_keys: - local_outfile_path = cdagvine.vine_task_graph_get_node_local_outfile_source(self._vine_task_graph, self.task_graph.vine_key_of[k]) - if not os.path.exists(local_outfile_path): - results[k] = "NOT_FOUND" - continue - with open(local_outfile_path, 'rb') as f: - result_obj = cloudpickle.load(f) - assert isinstance(result_obj, GraphKeyResult), "Loaded object is not of type GraphKeyResult" - results[k] = result_obj.result + for k in target_keys: + results[k] = py_graph.load_result_of_target_key(self.param("target-results-dir"), k) return results def _on_sigint(self, signum, frame): self.__del__() - def __del__(self): - if hasattr(self, '_vine_task_graph') and self._vine_task_graph: - cdagvine.vine_task_graph_delete(self._vine_task_graph) - - if hasattr(self, 'library') and self.library.local_path and os.path.exists(self.library.local_path): - os.remove(self.library.local_path) diff --git a/taskvine/src/graph/dagvine/library.py b/taskvine/src/graph/dagvine/library.py index 7a441e5aba..3e4a431483 100644 --- a/taskvine/src/graph/dagvine/library.py +++ b/taskvine/src/graph/dagvine/library.py @@ -3,19 +3,38 @@ from ndcctools.taskvine.dagvine import cdagvine from ndcctools.taskvine.graph_definition import ( GraphKeyResult, TaskGraph, compute_dts_key, compute_sexpr_key, - compute_single_key, hash_name, hashable, init_task_graph_context + compute_single_key, hash_name, hashable ) from ndcctools.taskvine.utils import load_variable_from_library class Library: - def __init__(self, hoisting_modules=[], env_files={}): + def __init__(self, py_manager, libname, libcores): + self.py_manager = py_manager + self.libname = libname + assert self.libname is not None, "libname must be provided" + + self.libcores = libcores + assert self.libcores is not None, "libcores must be provided" + self.libtask = None - self.hoisting_modules = hoisting_modules - self.env_files = env_files + # 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, GraphKeyResult, TaskGraph, uuid, hashlib, random, types, collections, time, + load_variable_from_library, compute_dts_key, compute_sexpr_key, compute_single_key, hash_name, hashable + ] + + # 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.libcores = -1 self.local_path = None self.remote_path = None @@ -27,30 +46,29 @@ 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 install(self, manager, libcores, vine_graph): - self.libcores = libcores - - assert cdagvine.vine_task_graph_get_proxy_function_name(vine_graph) == compute_single_key.__name__ + 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 - self.local_path = f"library-task-graph-{uuid.uuid4()}.pkl" - self.remote_path = self.local_path - - self.hoisting_modules += [ - os, cloudpickle, GraphKeyResult, TaskGraph, uuid, hashlib, random, types, collections, time, - load_variable_from_library, compute_dts_key, compute_sexpr_key, compute_single_key, hash_name, hashable - ] - lib_name = cdagvine.vine_task_graph_get_proxy_library_name(vine_graph) - self.libtask = manager.create_library_from_functions( - lib_name, + def install(self): + self.libtask = self.py_manager.create_library_from_functions( + self.libname, compute_single_key, - library_context_info=[init_task_graph_context, [], {"task_graph_path": self.remote_path}], + 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, ) - self.libtask.add_input(manager.declare_file(self.local_path), self.remote_path) for local, remote in self.env_files.items(): - self.libtask.add_input(manager.declare_file(local, cache=True, peer_transfer=True), remote) + # 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) - manager.install_library(self.libtask) \ No newline at end of file + self.py_manager.install_library(self.libtask) + + def uninstall(self): + self.py_manager.remove_library(self.libname) diff --git a/taskvine/src/graph/dagvine/vine_task_graph.c b/taskvine/src/graph/dagvine/vine_task_graph.c index 2cb1574a73..8bbd01a0f2 100644 --- a/taskvine/src/graph/dagvine/vine_task_graph.c +++ b/taskvine/src/graph/dagvine/vine_task_graph.c @@ -4,6 +4,7 @@ #include #include #include +#include #include #include @@ -383,51 +384,38 @@ int vine_task_graph_tune(struct vine_task_graph *tg, const char *name, const cha return -1; } - } - - return 0; -} - -/** - * Set the proxy library name (Python-side), shared by all tasks. - * @param tg Reference to the task graph object. - * @param proxy_library_name Reference to the proxy library name. - */ -void vine_task_graph_set_proxy_library_name(struct vine_task_graph *tg, const char *proxy_library_name) -{ - if (!tg || !proxy_library_name) { - return; - } - - /* free the existing proxy library name if it exists */ - if (tg->proxy_library_name) { - free(tg->proxy_library_name); - } + } else if (strcmp(name, "target-results-dir") == 0) { + if (tg->target_results_dir) { + free(tg->target_results_dir); + } + if (mkdir(value, 0777) != 0 && errno != EEXIST) { + debug(D_ERROR, "failed to mkdir %s (errno=%d)", value, errno); + return -1; + } + tg->target_results_dir = xxstrdup(value); - tg->proxy_library_name = xxstrdup(proxy_library_name); + } else if (strcmp(name, "proxy-library-name") == 0) { + if (tg->proxy_library_name) { + free(tg->proxy_library_name); + } + tg->proxy_library_name = xxstrdup(value); - return; -} + } else if (strcmp(name, "proxy-function-name") == 0) { + if (tg->proxy_function_name) { + free(tg->proxy_function_name); + } + tg->proxy_function_name = xxstrdup(value); -/** - * Set the proxy function name (Python-side), shared by all tasks. - * @param tg Reference to the task graph object. - * @param proxy_function_name Reference to the proxy function name. - */ -void vine_task_graph_set_proxy_function_name(struct vine_task_graph *tg, const char *proxy_function_name) -{ - if (!tg || !proxy_function_name) { - return; - } + } else if (strcmp(name, "prune-depth") == 0) { + tg->prune_depth = atoi(value); + + } else { + debug(D_ERROR, "invalid parameter name: %s", name); + return -1; - /* free the existing proxy function name if it exists */ - if (tg->proxy_function_name) { - free(tg->proxy_function_name); } - tg->proxy_function_name = xxstrdup(proxy_function_name); - - return; + return 0; } /** @@ -621,15 +609,9 @@ void vine_task_graph_compute_topology_metrics(struct vine_task_graph *tg) * Create a new node and track it in the task graph. * @param tg Reference to the task graph object. * @param node_key Reference to the node key. - * @param staging_dir Reference to the staging directory. - * @param prune_depth Reference to the prune depth. * @return A new node object. */ -struct vine_task_node *vine_task_graph_add_node( - struct vine_task_graph *tg, - const char *node_key, - const char *staging_dir, - int prune_depth) +struct vine_task_node *vine_task_graph_add_node(struct vine_task_graph *tg, const char *node_key) { if (!tg || !node_key) { return NULL; @@ -642,8 +624,15 @@ struct vine_task_node *vine_task_graph_add_node( node_key, tg->proxy_library_name, tg->proxy_function_name, - staging_dir, - prune_depth); + tg->target_results_dir, + tg->prune_depth); + + if (!node) { + debug(D_ERROR, "failed to create node %s", node_key); + vine_task_graph_delete(tg); + exit(1); + } + hash_table_insert(tg->nodes, node_key, node); } @@ -658,13 +647,18 @@ struct vine_task_node *vine_task_graph_add_node( struct vine_task_graph *vine_task_graph_create(struct vine_manager *q) { struct vine_task_graph *tg = xxmalloc(sizeof(struct vine_task_graph)); + + tg->manager = q; + tg->nodes = hash_table_create(0, 0); tg->task_id_to_node = itable_create(0); tg->outfile_cachename_to_node = hash_table_create(0, 0); - tg->proxy_library_name = xxstrdup("vine_task_graph_library"); // Python-side proxy library name (shared by all tasks) - tg->proxy_function_name = xxstrdup("compute_single_key"); // Python-side proxy function name (shared by all tasks) - tg->manager = q; + tg->target_results_dir = xxstrdup(tg->manager->runtime_directory); // default to current working directory + tg->proxy_library_name = NULL; + tg->proxy_function_name = NULL; + + tg->prune_depth = 1; tg->task_priority_mode = VINE_TASK_GRAPH_PRIORITY_MODE_LARGEST_INPUT_FIRST; tg->failure_injection_step_percent = -1.0; @@ -695,6 +689,13 @@ void vine_task_graph_add_dependency(struct vine_task_graph *tg, const char *pare struct vine_task_node *child_node = hash_table_lookup(tg->nodes, child_key); if (!parent_node) { debug(D_ERROR, "parent node %s not found", parent_key); + char *node_key = NULL; + struct vine_task_node *node; + printf("parent_keys:\n"); + HASH_TABLE_ITERATE(tg->nodes, node_key, node) + { + printf(" %s\n", node->node_key); + } exit(1); } if (!child_node) { diff --git a/taskvine/src/graph/dagvine/vine_task_graph.h b/taskvine/src/graph/dagvine/vine_task_graph.h index 6f4b39b398..47990e7211 100644 --- a/taskvine/src/graph/dagvine/vine_task_graph.h +++ b/taskvine/src/graph/dagvine/vine_task_graph.h @@ -28,8 +28,16 @@ struct vine_task_graph { struct itable *task_id_to_node; struct hash_table *outfile_cachename_to_node; - char *proxy_library_name; - char *proxy_function_name; + /* Results of target keys will be stored in this directory. + * This dir path can not necessarily be a shared file system directory, + * output files will be retrieved through the network instead, + * as long as the manager can access it. */ + char *target_results_dir; + + char *proxy_library_name; // Python-side proxy library name (shared by all tasks) + char *proxy_function_name; // Python-side proxy function name (shared by all tasks) + + int prune_depth; vine_task_graph_priority_mode_t task_priority_mode; // priority mode for task graph task scheduling double failure_injection_step_percent; // 0 - 100, the percentage of steps to inject failure @@ -47,14 +55,9 @@ struct vine_task_graph *vine_task_graph_create(struct vine_manager *q); /** Create a new node in the task graph. @param tg Reference to the task graph object. @param node_key Reference to the node key. -@param staging_dir Reference to the staging directory. -@param prune_depth Reference to the prune depth. @return A new node object. */ -struct vine_task_node *vine_task_graph_add_node(struct vine_task_graph *tg, - const char *node_key, - const char *staging_dir, - int prune_depth); +struct vine_task_node *vine_task_graph_add_node(struct vine_task_graph *tg, const char *node_key); /** Add a dependency between two nodes in the task graph. @param tg Reference to the task graph object. @@ -112,18 +115,6 @@ const char *vine_task_graph_get_proxy_library_name(const struct vine_task_graph */ const char *vine_task_graph_get_proxy_function_name(const struct vine_task_graph *tg); -/** Set the proxy library name (Python-side), shared by all tasks. -@param tg Reference to the task graph object. -@param proxy_library_name Reference to the proxy library name. -*/ -void vine_task_graph_set_proxy_library_name(struct vine_task_graph *tg, const char *proxy_library_name); - -/** Set the proxy function name (Python-side), shared by all tasks. -@param tg Reference to the task graph object. -@param proxy_function_name Reference to the proxy function name. -*/ -void vine_task_graph_set_proxy_function_name(struct vine_task_graph *tg, const char *proxy_function_name); - /** Tune the task graph. @param tg Reference to the task graph object. @param name Reference to the name of the parameter to tune. diff --git a/taskvine/src/graph/dagvine/vine_task_node.c b/taskvine/src/graph/dagvine/vine_task_node.c index c563ff459f..0539c59f94 100644 --- a/taskvine/src/graph/dagvine/vine_task_node.c +++ b/taskvine/src/graph/dagvine/vine_task_node.c @@ -3,7 +3,6 @@ #include #include #include -#include #include #include @@ -58,10 +57,27 @@ struct vine_task_node *vine_task_node_create( const char *node_key, const char *proxy_library_name, const char *proxy_function_name, - const char *staging_dir, + const char *target_results_dir, int prune_depth) { - if (!manager || !node_key || !proxy_library_name || !proxy_function_name || !staging_dir) { + if (!manager) { + debug(D_ERROR, "Cannot create node because manager is NULL"); + return NULL; + } + if (!node_key) { + debug(D_ERROR, "Cannot create node because node_key is NULL"); + return NULL; + } + if (!proxy_library_name) { + debug(D_ERROR, "Cannot create node because proxy_library_name is NULL"); + return NULL; + } + if (!proxy_function_name) { + debug(D_ERROR, "Cannot create node because proxy_function_name is NULL"); + return NULL; + } + if (!target_results_dir) { + debug(D_ERROR, "Cannot create node because target_results_dir is NULL"); return NULL; } @@ -69,7 +85,7 @@ struct vine_task_node *vine_task_node_create( node->manager = manager; node->node_key = xxstrdup(node_key); - node->staging_dir = xxstrdup(staging_dir); + node->target_results_dir = xxstrdup(target_results_dir); node->prune_status = PRUNE_STATUS_NOT_PRUNED; node->parents = list_create(); @@ -137,13 +153,12 @@ void vine_task_node_set_outfile(struct vine_task_node *node, vine_task_node_outf /* create the output file */ switch (node->outfile_type) { case VINE_NODE_OUTFILE_TYPE_LOCAL: { - char *local_output_dir = string_format("%s/outputs", node->staging_dir); - if (mkdir(local_output_dir, 0777) != 0 && errno != EEXIST) { - debug(D_ERROR, "failed to mkdir %s (errno=%d)", local_output_dir, errno); + if (!node->target_results_dir) { + debug(D_ERROR, "Cannot create local output file for node %s because target_results_dir is NULL", node->node_key); + exit(1); } - char *local_output_path = string_format("%s/%s", local_output_dir, node->outfile_remote_name); + char *local_output_path = string_format("%s/%s", node->target_results_dir, node->outfile_remote_name); node->outfile = vine_declare_file(node->manager, local_output_path, VINE_CACHE_LEVEL_WORKFLOW, 0); - free(local_output_dir); free(local_output_path); break; } @@ -471,7 +486,7 @@ void vine_task_node_print_info(struct vine_task_node *node) debug(D_VINE, "---------------- Node Info ----------------"); debug(D_VINE, "key: %s", node->node_key); debug(D_VINE, "task_id: %d", node->task->task_id); - debug(D_VINE, "staging_dir: %s", node->staging_dir ? node->staging_dir : "(null)"); + debug(D_VINE, "target_results_dir: %s", node->target_results_dir ? node->target_results_dir : "(null)"); debug(D_VINE, "depth: %d", node->depth); debug(D_VINE, "height: %d", node->height); debug(D_VINE, "prune_depth: %d", node->prune_depth); @@ -599,8 +614,8 @@ void vine_task_node_delete(struct vine_task_node *node) if (node->outfile_remote_name) { free(node->outfile_remote_name); } - if (node->staging_dir) { - free(node->staging_dir); + if (node->target_results_dir) { + free(node->target_results_dir); } vine_task_delete(node->task); diff --git a/taskvine/src/graph/dagvine/vine_task_node.h b/taskvine/src/graph/dagvine/vine_task_node.h index 5b6b00272a..c9d58bf520 100644 --- a/taskvine/src/graph/dagvine/vine_task_node.h +++ b/taskvine/src/graph/dagvine/vine_task_node.h @@ -29,7 +29,7 @@ struct vine_task_node { struct vine_file *infile; struct vine_file *outfile; char *outfile_remote_name; - char *staging_dir; + char *target_results_dir; size_t outfile_size_bytes; struct list *parents; @@ -62,7 +62,7 @@ struct vine_task_node *vine_task_node_create( const char *node_key, const char *proxy_library_name, const char *proxy_function_name, - const char *staging_dir, + const char *target_results_dir, int prune_depth ); From 3fab11052e6a54747cba77ebc8c478b546e3a47b Mon Sep 17 00:00:00 2001 From: Jin Zhou Date: Sat, 18 Oct 2025 00:20:25 -0400 Subject: [PATCH 041/113] remove _task_graph --- taskvine/src/graph/dagvine/graph_executor.py | 30 ++++++-------------- taskvine/src/graph/dagvine/vine_task_graph.c | 2 -- taskvine/src/manager/taskvine.h | 6 ++++ taskvine/src/manager/vine_manager.c | 1 + taskvine/src/manager/vine_runtime_dir.c | 5 ++++ 5 files changed, 21 insertions(+), 23 deletions(-) diff --git a/taskvine/src/graph/dagvine/graph_executor.py b/taskvine/src/graph/dagvine/graph_executor.py index 8393323359..a06c904270 100644 --- a/taskvine/src/graph/dagvine/graph_executor.py +++ b/taskvine/src/graph/dagvine/graph_executor.py @@ -4,23 +4,12 @@ from ndcctools.taskvine.manager import Manager from ndcctools.taskvine.utils import delete_all_files, get_c_constant -from ndcctools.taskvine.dagvine.graph_definition import GraphKeyResult, TaskGraph from ndcctools.taskvine.dagvine.library import Library -from ndcctools.taskvine.dagvine.graph_definition import ( - GraphKeyResult, TaskGraph, compute_dts_key, compute_sexpr_key, - compute_single_key, hash_name, hashable, task_graph_loader -) +from ndcctools.taskvine.dagvine.graph_definition import TaskGraph, task_graph_loader import cloudpickle import os -import collections -import inspect -import types import signal -import hashlib -import time -import random -import uuid try: import dask @@ -38,6 +27,7 @@ dts = None +# convert Dask collection to task dictionary def dask_collections_to_task_dict(collection_dict): assert is_dask_collection is not None from dask.highlevelgraph import HighLevelGraph, ensure_dict @@ -58,6 +48,7 @@ def dask_collections_to_task_dict(collection_dict): return ensure_dict(hlg) +# compatibility for Dask-created collections def ensure_task_dict(collection_dict): if is_dask_collection and any(is_dask_collection(v) for v in collection_dict.values()): task_dict = dask_collections_to_task_dict(collection_dict) @@ -144,7 +135,7 @@ def __init__(self, # initialize the manager super().__init__(*args, **kwargs) - self._vine_task_graph = None + self.runtime_directory = cvine.vine_get_runtime_directory(self._taskvine) def tune_manager(self): for k, v in self.manager_tuning_params.to_dict().items(): @@ -154,10 +145,6 @@ def tune_manager(self): # set worker scheduling algorithm cvine.vine_set_scheduler(self._taskvine, self.vine_constant_params.get_c_constant_of("schedule")) - if self._vine_task_graph: - cdagvine.vine_task_graph_tune(self._vine_task_graph, "failure-injection-step-percent", str(self.regular_params.failure_injection_step_percent)) - cdagvine.vine_task_graph_tune(self._vine_task_graph, "task-priority-mode", self.vine_constant_params.task_priority_mode) - def param(self, param_name): return self.params.get_value_of(param_name) @@ -256,7 +243,7 @@ def build_graphs(self, target_keys): return py_graph, c_graph - def new_run(self, collection_dict, target_keys=None, params={}, hoisting_modules=[], env_files={}): + def run(self, collection_dict, target_keys=None, params={}, hoisting_modules=[], env_files={}): # first update the params so that they can be used for the following construction self.update_params(params) @@ -272,13 +259,14 @@ def new_run(self, collection_dict, target_keys=None, params={}, hoisting_modules library.set_context_loader(task_graph_loader, context_loader_args=[cloudpickle.dumps(py_graph)]) library.install() + # execute the graph on the C side + print(f"Executing task graph, logs will be written to {self.runtime_directory}") cdagvine.vine_task_graph_execute(c_graph) - # delete the C graph immediately after execution, so that the lifetime of this object is limited to the execution - cdagvine.vine_task_graph_delete(c_graph) - # clean up the library instances and template on the manager library.uninstall() + # delete the C graph immediately after execution, so that the lifetime of this object is limited to the execution + cdagvine.vine_task_graph_delete(c_graph) # load results of target keys results = {} diff --git a/taskvine/src/graph/dagvine/vine_task_graph.c b/taskvine/src/graph/dagvine/vine_task_graph.c index 8bbd01a0f2..5e376812b7 100644 --- a/taskvine/src/graph/dagvine/vine_task_graph.c +++ b/taskvine/src/graph/dagvine/vine_task_graph.c @@ -972,8 +972,6 @@ void vine_task_graph_delete(struct vine_task_graph *tg) vine_task_node_delete(node); } - vine_delete(tg->manager); - free(tg->proxy_library_name); free(tg->proxy_function_name); diff --git a/taskvine/src/manager/taskvine.h b/taskvine/src/manager/taskvine.h index eda05ebc4d..ebce1899a2 100644 --- a/taskvine/src/manager/taskvine.h +++ b/taskvine/src/manager/taskvine.h @@ -1536,6 +1536,12 @@ void vine_counters_print(); */ char *vine_version_string(); +/** Returns the runtime directory +@param m Reference to the current manager object. +@return A string. +*/ +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 9953df9450..27832876f9 100644 --- a/taskvine/src/manager/vine_manager.c +++ b/taskvine/src/manager/vine_manager.c @@ -5157,6 +5157,7 @@ void vine_manager_remove_library(struct vine_manager *q, const char *name) struct vine_task *library = vine_schedule_find_library(q, w, name); while (library) { vine_cancel_by_task_id(q, library->task_id); + itable_remove(w->current_libraries, library->task_id); library = vine_schedule_find_library(q, w, name); } hash_table_remove(q->library_templates, name); diff --git a/taskvine/src/manager/vine_runtime_dir.c b/taskvine/src/manager/vine_runtime_dir.c index a2bc4194a0..5f5f586b32 100644 --- a/taskvine/src/manager/vine_runtime_dir.c +++ b/taskvine/src/manager/vine_runtime_dir.c @@ -134,6 +134,11 @@ char *vine_runtime_directory_create() return runtime_dir; } +char *vine_get_runtime_directory(struct vine_manager *m) +{ + 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 : ""); From 5b85f46ad5e860188ff046f2230757209c6c6648 Mon Sep 17 00:00:00 2001 From: Jin Zhou Date: Sat, 18 Oct 2025 02:32:35 -0400 Subject: [PATCH 042/113] new --- .../src/graph/dagvine/graph_definition.py | 28 +++++++++---------- taskvine/src/graph/dagvine/graph_executor.py | 6 ++-- 2 files changed, 16 insertions(+), 18 deletions(-) diff --git a/taskvine/src/graph/dagvine/graph_definition.py b/taskvine/src/graph/dagvine/graph_definition.py index 3dcead36df..0b22fa4e7e 100644 --- a/taskvine/src/graph/dagvine/graph_definition.py +++ b/taskvine/src/graph/dagvine/graph_definition.py @@ -216,14 +216,24 @@ def save_result_of_key(self, key, result): result_obj = GraphKeyResult(result, extra_size_mb=self.extra_task_output_size_mb[key]) cloudpickle.dump(result_obj, f) - def load_result_of_key(self, key): + def load_result_of_key(self, key, target_results_dir=None): + if self.outfile_type[key] == "local": + # nodes with local outfile type will have their results stored in the target results directory + # when the execution is done, users can call this function to load the results from the target results directory + # note that data is transferred through the link in sequence, not in parallel through the shared file system + outfile_path = os.path.join(target_results_dir, self.outfile_remote_name[key]) + else: + # workers user this branch to load results from either local or shared file system + # if a node-local output, then data is stored in the task sandbox and the remote name is just the filename + # if a shared file system output, then remote name is the full path to the file + outfile_path = self.outfile_remote_name[key] try: - with open(self.outfile_remote_name[key], "rb") as f: + with open(outfile_path, "rb") as f: result_obj = cloudpickle.load(f) assert isinstance(result_obj, GraphKeyResult), "Loaded object is not of type GraphKeyResult" return result_obj.result except FileNotFoundError: - raise FileNotFoundError(f"Output file for key {key} not found at {self.outfile_remote_name[key]}") + raise FileNotFoundError(f"Output file for key {key} not found at {outfile_path}") def get_topological_order(self): in_degree = {key: len(self.parents_of[key]) for key in self.task_dict.keys()} @@ -246,18 +256,6 @@ def get_topological_order(self): return topo_order - def load_result_of_target_key(self, target_results_dir, key): - assert self.outfile_type[key] == "local", "Only local output files are supported for target keys" - outfile_path = os.path.join(target_results_dir, self.outfile_remote_name[key]) - if not os.path.exists(outfile_path): - result = "NOT_FOUND" - else: - with open(outfile_path, "rb") as f: - result_obj = cloudpickle.load(f) - assert isinstance(result_obj, GraphKeyResult), "Loaded object is not of type GraphKeyResult" - result = result_obj.result - return result - def __del__(self): if hasattr(self, 'outfile_remote_name') and self.outfile_remote_name: for k in self.outfile_remote_name.keys(): diff --git a/taskvine/src/graph/dagvine/graph_executor.py b/taskvine/src/graph/dagvine/graph_executor.py index a06c904270..733fe845f8 100644 --- a/taskvine/src/graph/dagvine/graph_executor.py +++ b/taskvine/src/graph/dagvine/graph_executor.py @@ -79,12 +79,12 @@ def __init__(self): "proxy-library-name": "vine_task_graph_library", "proxy-function-name": "compute_single_key", "prune-depth": 1, + "target-results-dir": "./target_results", } self.other_params = { "schedule": "worst", "libcores": 16, "failure-injection-step-percent": -1, - "target-results-dir": "./target_results", "shared-file-system-dir": "./shared_file_system", "extra-task-output-size-mb": ["uniform", 0, 0], "extra-task-sleep-time": ["uniform", 0, 0], @@ -252,8 +252,8 @@ def run(self, collection_dict, target_keys=None, params={}, hoisting_modules=[], # build graphs in both Python and C sides py_graph, c_graph = self.build_graphs(target_keys) + # create and install the library template on the manager library = Library(self, self.param("proxy-library-name"), self.param("libcores")) - library.add_hoisting_modules(hoisting_modules) library.add_env_files(env_files) library.set_context_loader(task_graph_loader, context_loader_args=[cloudpickle.dumps(py_graph)]) @@ -271,7 +271,7 @@ def run(self, collection_dict, target_keys=None, params={}, hoisting_modules=[], # load results of target keys results = {} for k in target_keys: - results[k] = py_graph.load_result_of_target_key(self.param("target-results-dir"), k) + results[k] = py_graph.load_result_of_key(k, target_results_dir=self.param("target-results-dir")) return results def _on_sigint(self, signum, frame): From ec96041caf881d3172c11577a112411f1cfc0c07 Mon Sep 17 00:00:00 2001 From: Jin Zhou Date: Sat, 18 Oct 2025 14:45:46 -0400 Subject: [PATCH 043/113] reg --- taskvine/src/bindings/python3/Makefile | 2 +- .../src/graph/dagvine/graph_definition.py | 340 ------------------ taskvine/src/graph/dagvine/graph_executor.py | 46 +-- taskvine/src/graph/dagvine/library.py | 6 +- 4 files changed, 27 insertions(+), 367 deletions(-) delete mode 100644 taskvine/src/graph/dagvine/graph_definition.py diff --git a/taskvine/src/bindings/python3/Makefile b/taskvine/src/bindings/python3/Makefile index 24557af1b0..c81a88efb1 100644 --- a/taskvine/src/bindings/python3/Makefile +++ b/taskvine/src/bindings/python3/Makefile @@ -39,7 +39,7 @@ install: all cp taskvine.py $(CCTOOLS_PYTHON3_PATH)/ mkdir -p $(CCTOOLS_PYTHON3_PATH)/ndcctools/taskvine/${GRAPH_EXECUTOR_MODULE_NAME} - cp $(CCTOOLS_HOME)/taskvine/src/graph/${GRAPH_EXECUTOR_MODULE_NAME}/graph_definition.py $(CCTOOLS_PYTHON3_PATH)/ndcctools/taskvine/${GRAPH_EXECUTOR_MODULE_NAME}/graph_definition.py + cp $(CCTOOLS_HOME)/taskvine/src/graph/${GRAPH_EXECUTOR_MODULE_NAME}/runtime_execution_graph.py $(CCTOOLS_PYTHON3_PATH)/ndcctools/taskvine/${GRAPH_EXECUTOR_MODULE_NAME}/runtime_execution_graph.py cp $(CCTOOLS_HOME)/taskvine/src/graph/${GRAPH_EXECUTOR_MODULE_NAME}/graph_executor.py $(CCTOOLS_PYTHON3_PATH)/ndcctools/taskvine/${GRAPH_EXECUTOR_MODULE_NAME}/graph_executor.py cp $(CCTOOLS_HOME)/taskvine/src/graph/${GRAPH_EXECUTOR_MODULE_NAME}/library.py $(CCTOOLS_PYTHON3_PATH)/ndcctools/taskvine/${GRAPH_EXECUTOR_MODULE_NAME}/library.py cp $(CCTOOLS_HOME)/taskvine/src/graph/${GRAPH_EXECUTOR_MODULE_NAME}/params.py $(CCTOOLS_PYTHON3_PATH)/ndcctools/taskvine/${GRAPH_EXECUTOR_MODULE_NAME}/params.py diff --git a/taskvine/src/graph/dagvine/graph_definition.py b/taskvine/src/graph/dagvine/graph_definition.py deleted file mode 100644 index 0b22fa4e7e..0000000000 --- a/taskvine/src/graph/dagvine/graph_definition.py +++ /dev/null @@ -1,340 +0,0 @@ -# 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.utils import load_variable_from_library - -import os -import hashlib -import time -import cloudpickle -import collections -import uuid -import random -from collections import deque - -try: - import dask -except ImportError: - dask = None - -try: - import dask._task_spec as dts -except ImportError: - dts = None - - -def hash_name(*args): - out_str = "" - for arg in args: - out_str += str(arg) - return hashlib.sha256(out_str.encode('utf-8')).hexdigest()[:32] - - -def hashable(s): - try: - hash(s) - return True - except TypeError: - return False - - -def dist_func(mode, low, high): - if not mode: - return 0 - - assert mode in ["uniform", "normal", "lognormal", "pareto", "mix"] - - # uniform distribution, flat spread - def uniform_dist(): - return random.uniform(low, high) - - # normal distribution, centered in the middle - def normal_dist(): - mu, sigma = (low + high) / 2, (high - low) / 6 - return min(max(random.gauss(mu, sigma), low), high) - - # lognormal distribution, long tail - def lognormal_dist(): - val = random.lognormvariate(0, 1) - val = val / (1 + val) - return low + (high - low) * val - - # pareto distribution, very heavy tail - def pareto_dist(alpha=2.0): - val = random.paretovariate(alpha) - val = val / (1 + val) - return low + (high - low) * val - - # mixture: most small values, few large ones - def mix_dist(): - if random.random() < 0.9: - return random.uniform(low, (low + high) / 2) - else: - return random.uniform((low + high) / 2, high) - - return { - "uniform": uniform_dist, - "normal": normal_dist, - "lognormal": lognormal_dist, - "pareto": pareto_dist, - "mix": mix_dist, - }[mode]() - - -class GraphKeyResult: - # extra_size_mb is used to allocate more space for this object in testing mode to evaluate storage consumption - # and peer transfer performance across all workers. - 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 - - -class TaskGraph: - def __init__(self, task_dict, - shared_file_system_dir=None, - extra_task_output_size_mb=["uniform", 0, 0], - extra_task_sleep_time=["uniform", 0, 0]): - self.task_dict = task_dict - self.shared_file_system_dir = shared_file_system_dir - - if self.shared_file_system_dir: - os.makedirs(self.shared_file_system_dir, exist_ok=True) - - if dts: - for k, v in self.task_dict.items(): - if isinstance(v, dts.GraphNode): - assert isinstance(v, (dts.Alias, dts.Task, dts.DataNode)), f"Unsupported task type for key {k}: {v.__class__}" - - self.parents_of, self.children_of = self._build_dependencies(self.task_dict) - self.depth_of = self._calculate_depths() - - self.vine_key_of = {k: hash_name(k) for k in task_dict.keys()} - self.key_of_vine_key = {hash_name(k): k for k in task_dict.keys()} - - self.outfile_remote_name = {key: f"{uuid.uuid4()}.pkl" for key in self.task_dict.keys()} - self.outfile_type = {key: None for key in self.task_dict.keys()} - - # testing params - self.extra_task_output_size_mb = self._calculate_extra_size_mb_of(extra_task_output_size_mb) - self.extra_sleep_time_of = self._calculate_extra_sleep_time_of(extra_task_sleep_time) - - def set_outfile_type_of(self, k, outfile_type_str): - assert outfile_type_str in ["local", "shared-file-system", "temp"] - self.outfile_type[k] = outfile_type_str - if outfile_type_str == "shared-file-system": - self.outfile_remote_name[k] = os.path.join(self.shared_file_system_dir, self.outfile_remote_name[k]) - - def _calculate_extra_size_mb_of(self, extra_task_output_size_mb): - assert isinstance(extra_task_output_size_mb, list) and len(extra_task_output_size_mb) == 3 - mode, low, high = extra_task_output_size_mb - low, high = int(low), int(high) - assert low <= high - - max_depth = max(depth for depth in self.depth_of.values()) - extra_size_mb_of = {} - for k in self.task_dict.keys(): - if self.depth_of[k] == max_depth or self.depth_of[k] == max_depth - 1: - extra_size_mb_of[k] = 0 - continue - extra_size_mb_of[k] = dist_func(mode, low, high) - - return extra_size_mb_of - - def _calculate_extra_sleep_time_of(self, extra_task_sleep_time): - assert isinstance(extra_task_sleep_time, list) and len(extra_task_sleep_time) == 3 - mode, low, high = extra_task_sleep_time - low, high = int(low), int(high) - assert low <= high - - extra_sleep_time_of = {} - for k in self.task_dict.keys(): - extra_sleep_time_of[k] = dist_func(mode, low, high) - - return extra_sleep_time_of - - def _calculate_depths(self): - depth_of = {key: 0 for key in self.task_dict.keys()} - - topo_order = self.get_topological_order() - for key in topo_order: - if self.parents_of[key]: - depth_of[key] = max(depth_of[parent] for parent in self.parents_of[key]) + 1 - else: - depth_of[key] = 0 - - return depth_of - - def set_outfile_remote_name_of(self, key, outfile_remote_name): - self.outfile_remote_name[key] = outfile_remote_name - - def is_dts_key(self, k): - if not hasattr(dask, "_task_spec"): - return False - import dask._task_spec as dts - return isinstance(self.task_dict[k], (dts.Task, dts.TaskRef, dts.Alias, dts.DataNode, dts.NestedContainer)) - - def _build_dependencies(self, task_dict): - def _find_sexpr_parents(sexpr): - if hashable(sexpr) and sexpr in task_dict.keys(): - return {sexpr} - elif isinstance(sexpr, (list, tuple)): - deps = set() - for x in sexpr: - deps |= _find_sexpr_parents(x) - return deps - elif isinstance(sexpr, dict): - deps = set() - for k, v in sexpr.items(): - deps |= _find_sexpr_parents(k) - deps |= _find_sexpr_parents(v) - return deps - else: - return set() - - parents_of = collections.defaultdict(set) - children_of = collections.defaultdict(set) - - for k, value in task_dict.items(): - if self.is_dts_key(k): - # in the new Dask expression, each value is an object from dask._task_spec, could be - # a Task, Alias, TaskRef, etc., but they all share the same base class the dependencies - # field is of type frozenset(), without recursive ancestor dependencies involved - parents_of[k] = value.dependencies - else: - # the value could be a sexpr, e.g., the old Dask representation - parents_of[k] = _find_sexpr_parents(value) - - for k, deps in parents_of.items(): - for dep in deps: - children_of[dep].add(k) - - return parents_of, children_of - - def save_result_of_key(self, key, result): - with open(self.outfile_remote_name[key], "wb") as f: - result_obj = GraphKeyResult(result, extra_size_mb=self.extra_task_output_size_mb[key]) - cloudpickle.dump(result_obj, f) - - def load_result_of_key(self, key, target_results_dir=None): - if self.outfile_type[key] == "local": - # nodes with local outfile type will have their results stored in the target results directory - # when the execution is done, users can call this function to load the results from the target results directory - # note that data is transferred through the link in sequence, not in parallel through the shared file system - outfile_path = os.path.join(target_results_dir, self.outfile_remote_name[key]) - else: - # workers user this branch to load results from either local or shared file system - # if a node-local output, then data is stored in the task sandbox and the remote name is just the filename - # if a shared file system output, then remote name is the full path to the file - outfile_path = self.outfile_remote_name[key] - try: - with open(outfile_path, "rb") as f: - result_obj = cloudpickle.load(f) - assert isinstance(result_obj, GraphKeyResult), "Loaded object is not of type GraphKeyResult" - return result_obj.result - except FileNotFoundError: - raise FileNotFoundError(f"Output file for key {key} not found at {outfile_path}") - - def get_topological_order(self): - in_degree = {key: len(self.parents_of[key]) for key in self.task_dict.keys()} - queue = deque([key for key, degree in in_degree.items() if degree == 0]) - topo_order = [] - - while queue: - current = queue.popleft() - topo_order.append(current) - - for child in self.children_of[current]: - in_degree[child] -= 1 - if in_degree[child] == 0: - queue.append(child) - - if len(topo_order) != len(self.task_dict): - print(f"len(topo_order): {len(topo_order)}") - print(f"len(self.task_dict): {len(self.task_dict)}") - raise ValueError("Failed to create topo order, the dependencies may be cyclic or problematic") - - return topo_order - - def __del__(self): - if hasattr(self, 'outfile_remote_name') and self.outfile_remote_name: - for k in self.outfile_remote_name.keys(): - if self.outfile_type.get(k) == "shared-file-system" and os.path.exists(self.outfile_remote_name[k]): - os.remove(self.outfile_remote_name[k]) - - -def task_graph_loader(task_graph_pkl): - task_graph = cloudpickle.loads(task_graph_pkl) - - if not isinstance(task_graph, TaskGraph): - raise TypeError("Task graph is not of type TaskGraph") - - return { - "task_graph": task_graph, - } - - -def compute_dts_key(task_graph, k, v): - try: - import dask._task_spec as dts - except ImportError: - raise ImportError("Dask is not installed") - - input_dict = {dep: task_graph.load_result_of_key(dep) for dep in v.dependencies} - - try: - if isinstance(v, dts.Alias): - assert len(v.dependencies) == 1, "Expected exactly one dependency" - return task_graph.load_result_of_key(next(iter(v.dependencies))) - elif isinstance(v, dts.Task): - return v(input_dict) - elif isinstance(v, dts.DataNode): - return v.value - else: - raise TypeError(f"unexpected node type: {type(v)} for key {k}") - except Exception as e: - raise Exception(f"Error while executing task {k}: {e}") - - -def compute_sexpr_key(task_graph, k, v): - input_dict = {parent: task_graph.load_result_of_key(parent) for parent in task_graph.parents_of[k]} - - def _rec_call(expr): - try: - if expr in input_dict.keys(): - return input_dict[expr] - except TypeError: - pass - if isinstance(expr, list): - return [_rec_call(e) for e in expr] - if isinstance(expr, tuple) and len(expr) > 0 and callable(expr[0]): - res = expr[0](*[_rec_call(a) for a in expr[1:]]) - return res - return expr - - try: - return _rec_call(v) - except Exception as e: - raise Exception(f"Failed to invoke _rec_call(): {e}") - - -def compute_single_key(vine_key): - task_graph = load_variable_from_library('task_graph') - - k = task_graph.key_of_vine_key[vine_key] - v = task_graph.task_dict[k] - - if task_graph.is_dts_key(k): - result = compute_dts_key(task_graph, k, v) - else: - result = compute_sexpr_key(task_graph, k, v) - - task_graph.save_result_of_key(k, result) - if not os.path.exists(task_graph.outfile_remote_name[k]): - raise Exception(f"Output file {task_graph.outfile_remote_name[k]} does not exist after writing") - if os.stat(task_graph.outfile_remote_name[k]).st_size == 0: - raise Exception(f"Output file {task_graph.outfile_remote_name[k]} is empty after writing") - - time.sleep(task_graph.extra_sleep_time_of[k]) - - return True diff --git a/taskvine/src/graph/dagvine/graph_executor.py b/taskvine/src/graph/dagvine/graph_executor.py index 733fe845f8..d9846463fb 100644 --- a/taskvine/src/graph/dagvine/graph_executor.py +++ b/taskvine/src/graph/dagvine/graph_executor.py @@ -5,7 +5,7 @@ from ndcctools.taskvine.utils import delete_all_files, get_c_constant from ndcctools.taskvine.dagvine.library import Library -from ndcctools.taskvine.dagvine.graph_definition import TaskGraph, task_graph_loader +from ndcctools.taskvine.dagvine.runtime_execution_graph import RuntimeExecutionGraph import cloudpickle import os @@ -166,14 +166,14 @@ def tune_vine_task_graph(self, c_graph): print(f"Tuning {k} to {v}") cdagvine.vine_task_graph_tune(c_graph, k, str(v)) - def assign_outfile_types(self, target_keys, py_graph, c_graph): - assert py_graph is not None, "Python graph must be built first" + def assign_outfile_types(self, target_keys, reg, c_graph): + assert reg is not None, "Python graph must be built first" assert c_graph is not None, "C graph must be built first" # get heavy score from C side heavy_scores = {} - for k in py_graph.task_dict.keys(): - heavy_scores[k] = cdagvine.vine_task_graph_get_node_heavy_score(c_graph, py_graph.vine_key_of[k]) + for k in reg.task_dict.keys(): + heavy_scores[k] = cdagvine.vine_task_graph_get_node_heavy_score(c_graph, reg.vine_key_of[k]) # sort keys by heavy score descending sorted_keys = sorted(heavy_scores, key=lambda k: heavy_scores[k], reverse=True) @@ -190,27 +190,27 @@ def assign_outfile_types(self, target_keys, py_graph, c_graph): else: choice = "temp" - py_graph.set_outfile_type_of(k, choice) + reg.set_outfile_type_of(k, choice) outfile_type_enum = get_c_constant(f"NODE_OUTFILE_TYPE_{choice.upper().replace('-', '_')}") cdagvine.vine_task_graph_set_node_outfile( c_graph, - py_graph.vine_key_of[k], + reg.vine_key_of[k], outfile_type_enum, - py_graph.outfile_remote_name[k] + reg.outfile_remote_name[k] ) def build_python_graph(self): - py_graph = TaskGraph( + reg = RuntimeExecutionGraph( self.task_dict, shared_file_system_dir=self.param("shared-file-system-dir"), extra_task_output_size_mb=self.param("extra-task-output-size-mb"), extra_task_sleep_time=self.param("extra-task-sleep-time") ) - return py_graph + return reg - def build_c_graph(self, py_graph): - assert py_graph is not None, "Python graph must be built before building the C graph" + def build_c_graph(self, reg): + assert reg is not None, "Python graph must be built before building the C graph" c_graph = cdagvine.vine_task_graph_create(self._taskvine) @@ -218,14 +218,14 @@ def build_c_graph(self, py_graph): self.tune_vine_manager() self.tune_vine_task_graph(c_graph) - topo_order = py_graph.get_topological_order() + topo_order = reg.get_topological_order() for k in topo_order: cdagvine.vine_task_graph_add_node( c_graph, - py_graph.vine_key_of[k], + reg.vine_key_of[k], ) - for pk in py_graph.parents_of[k]: - cdagvine.vine_task_graph_add_dependency(c_graph, py_graph.vine_key_of[pk], py_graph.vine_key_of[k]) + for pk in reg.parents_of[k]: + cdagvine.vine_task_graph_add_dependency(c_graph, reg.vine_key_of[pk], reg.vine_key_of[k]) cdagvine.vine_task_graph_compute_topology_metrics(c_graph) @@ -233,15 +233,15 @@ def build_c_graph(self, py_graph): def build_graphs(self, target_keys): # build Python DAG (logical topology) - py_graph = self.build_python_graph() + reg = self.build_python_graph() # build C DAG (physical topology) - c_graph = self.build_c_graph(py_graph) + c_graph = self.build_c_graph(reg) # assign outfile types - self.assign_outfile_types(target_keys, py_graph, c_graph) + self.assign_outfile_types(target_keys, reg, c_graph) - return py_graph, c_graph + return reg, c_graph def run(self, collection_dict, target_keys=None, params={}, hoisting_modules=[], env_files={}): # first update the params so that they can be used for the following construction @@ -250,13 +250,13 @@ def run(self, collection_dict, target_keys=None, params={}, hoisting_modules=[], self.task_dict = ensure_task_dict(collection_dict) # build graphs in both Python and C sides - py_graph, c_graph = self.build_graphs(target_keys) + reg, c_graph = self.build_graphs(target_keys) # create and install the library template on the manager library = Library(self, self.param("proxy-library-name"), self.param("libcores")) library.add_hoisting_modules(hoisting_modules) library.add_env_files(env_files) - library.set_context_loader(task_graph_loader, context_loader_args=[cloudpickle.dumps(py_graph)]) + library.set_context_loader(RuntimeExecutionGraph.context_loader_func, context_loader_args=[cloudpickle.dumps(reg)]) library.install() # execute the graph on the C side @@ -271,7 +271,7 @@ def run(self, collection_dict, target_keys=None, params={}, hoisting_modules=[], # load results of target keys results = {} for k in target_keys: - results[k] = py_graph.load_result_of_key(k, target_results_dir=self.param("target-results-dir")) + results[k] = reg.load_result_of_key(k, target_results_dir=self.param("target-results-dir")) return results def _on_sigint(self, signum, frame): diff --git a/taskvine/src/graph/dagvine/library.py b/taskvine/src/graph/dagvine/library.py index 3e4a431483..cd8f7d498e 100644 --- a/taskvine/src/graph/dagvine/library.py +++ b/taskvine/src/graph/dagvine/library.py @@ -1,8 +1,8 @@ import os, uuid, cloudpickle, types, time, random, hashlib, collections from ndcctools.taskvine import cvine from ndcctools.taskvine.dagvine import cdagvine -from ndcctools.taskvine.graph_definition import ( - GraphKeyResult, TaskGraph, compute_dts_key, compute_sexpr_key, +from ndcctools.taskvine.dagvine.runtime_execution_graph import ( + GraphKeyResult, RuntimeExecutionGraph, compute_dts_key, compute_sexpr_key, compute_single_key, hash_name, hashable ) from ndcctools.taskvine.utils import load_variable_from_library @@ -22,7 +22,7 @@ def __init__(self, py_manager, libname, libcores): # 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, GraphKeyResult, TaskGraph, uuid, hashlib, random, types, collections, time, + os, cloudpickle, GraphKeyResult, RuntimeExecutionGraph, uuid, hashlib, random, types, collections, time, load_variable_from_library, compute_dts_key, compute_sexpr_key, compute_single_key, hash_name, hashable ] From 767988fdb9e9ab3d19d157b2187d60146dcc7d46 Mon Sep 17 00:00:00 2001 From: Jin Zhou Date: Sat, 18 Oct 2025 15:43:30 -0400 Subject: [PATCH 044/113] sog && reg --- .../python3/ndcctools/taskvine/utils.py | 3 +- taskvine/src/graph/dagvine/Makefile | 10 +- taskvine/src/graph/dagvine/graph_executor.py | 62 +- taskvine/src/graph/dagvine/library.py | 10 +- taskvine/src/graph/dagvine/vine_task_graph.c | 982 ------------------ taskvine/src/graph/dagvine/vine_task_graph.h | 126 --- taskvine/src/graph/dagvine/vine_task_graph.i | 21 - 7 files changed, 42 insertions(+), 1172 deletions(-) delete mode 100644 taskvine/src/graph/dagvine/vine_task_graph.c delete mode 100644 taskvine/src/graph/dagvine/vine_task_graph.h delete mode 100644 taskvine/src/graph/dagvine/vine_task_graph.i diff --git a/taskvine/src/bindings/python3/ndcctools/taskvine/utils.py b/taskvine/src/bindings/python3/ndcctools/taskvine/utils.py index bd909b664b..e493e7c04a 100644 --- a/taskvine/src/bindings/python3/ndcctools/taskvine/utils.py +++ b/taskvine/src/bindings/python3/ndcctools/taskvine/utils.py @@ -4,8 +4,7 @@ from . import cvine try: - # cdagvine contains DAG-specific enums (e.g., VINE_TASK_GRAPH_PRIORITY_MODE_*) - from .dagvine import cdagvine # type: ignore + from .dagvine import cdagvine except Exception: cdagvine = None diff --git a/taskvine/src/graph/dagvine/Makefile b/taskvine/src/graph/dagvine/Makefile index 018a48a227..4286e36cad 100644 --- a/taskvine/src/graph/dagvine/Makefile +++ b/taskvine/src/graph/dagvine/Makefile @@ -6,15 +6,15 @@ LOCAL_CCFLAGS+=-I ${CCTOOLS_HOME}/taskvine/src/manager -SOURCES = vine_task_node.c vine_task_graph.c +SOURCES = vine_task_node.c strategic_orchestration_graph.c OBJECTS = $(SOURCES:%.c=%.o) -SWIG_I = vine_task_graph.i +SWIG_I = strategic_orchestration_graph.i # put SWIG generated sources and Python extension artifacts into build/ directory BUILD_DIR := build -SWIG_WRAP = $(BUILD_DIR)/vine_task_graph_wrap.c -WRAP_OBJ = $(BUILD_DIR)/vine_task_graph_wrap.o +SWIG_WRAP = $(BUILD_DIR)/strategic_orchestration_graph_wrap.c +WRAP_OBJ = $(BUILD_DIR)/strategic_orchestration_graph_wrap.o PYMODULE = $(BUILD_DIR)/_cdagvine.$(CCTOOLS_DYNAMIC_SUFFIX) LIBRARIES = @@ -33,7 +33,7 @@ $(PROGRAMS): $(EXTERNALS) $(BUILD_DIR): mkdir -p $(BUILD_DIR) -$(SWIG_WRAP): $(SWIG_I) vine_task_graph.h | $(BUILD_DIR) +$(SWIG_WRAP): $(SWIG_I) strategic_orchestration_graph.h | $(BUILD_DIR) $(CCTOOLS_SWIG) -python -threads -relativeimport -I$(CCTOOLS_HOME)/taskvine/src/manager -I$(CCTOOLS_HOME)/dttools/src -outdir $(BUILD_DIR) -o $@ $< # Build Python module diff --git a/taskvine/src/graph/dagvine/graph_executor.py b/taskvine/src/graph/dagvine/graph_executor.py index d9846463fb..0554074c4f 100644 --- a/taskvine/src/graph/dagvine/graph_executor.py +++ b/taskvine/src/graph/dagvine/graph_executor.py @@ -73,10 +73,9 @@ def __init__(self): "enforce-worker-eviction-interval": -1, "balance-worker-disk-load": 0, } - self.vine_task_graph_tuning_params = { + self.sog_tuning_params = { "failure-injection-step-percent": -1, - "task-priority-mode": "largest-input-first", - "proxy-library-name": "vine_task_graph_library", + "priority-mode": "largest-input-first", "proxy-function-name": "compute_single_key", "prune-depth": 1, "target-results-dir": "./target_results", @@ -97,8 +96,8 @@ def __init__(self): def update_param(self, param_name, new_value): if param_name in self.vine_manager_tuning_params: self.vine_manager_tuning_params[param_name] = new_value - elif param_name in self.vine_task_graph_tuning_params: - self.vine_task_graph_tuning_params[param_name] = new_value + elif param_name in self.sog_tuning_params: + self.sog_tuning_params[param_name] = new_value elif param_name in self.other_params: self.other_params[param_name] = new_value else: @@ -107,8 +106,8 @@ def update_param(self, param_name, new_value): def get_value_of(self, param_name): if param_name in self.vine_manager_tuning_params: return self.vine_manager_tuning_params[param_name] - elif param_name in self.vine_task_graph_tuning_params: - return self.vine_task_graph_tuning_params[param_name] + elif param_name in self.sog_tuning_params: + return self.sog_tuning_params[param_name] elif param_name in self.other_params: return self.other_params[param_name] else: @@ -161,19 +160,19 @@ def tune_vine_manager(self): print(f"Tuning {k} to {v}") self.tune(k, v) - def tune_vine_task_graph(self, c_graph): - for k, v in self.params.vine_task_graph_tuning_params.items(): + def tune_sog(self, sog): + for k, v in self.params.sog_tuning_params.items(): print(f"Tuning {k} to {v}") - cdagvine.vine_task_graph_tune(c_graph, k, str(v)) + cdagvine.sog_tune(sog, k, str(v)) - def assign_outfile_types(self, target_keys, reg, c_graph): - assert reg is not None, "Python graph must be built first" - assert c_graph is not None, "C graph must be built first" + def assign_outfile_types(self, target_keys, reg, sog): + assert reg is not None, "Runtime execution graph must be built first" + assert sog is not None, "Strategic orchestration graph must be built first" # get heavy score from C side heavy_scores = {} for k in reg.task_dict.keys(): - heavy_scores[k] = cdagvine.vine_task_graph_get_node_heavy_score(c_graph, reg.vine_key_of[k]) + heavy_scores[k] = cdagvine.sog_get_node_heavy_score(sog, reg.vine_key_of[k]) # sort keys by heavy score descending sorted_keys = sorted(heavy_scores, key=lambda k: heavy_scores[k], reverse=True) @@ -192,8 +191,8 @@ def assign_outfile_types(self, target_keys, reg, c_graph): reg.set_outfile_type_of(k, choice) outfile_type_enum = get_c_constant(f"NODE_OUTFILE_TYPE_{choice.upper().replace('-', '_')}") - cdagvine.vine_task_graph_set_node_outfile( - c_graph, + cdagvine.sog_set_node_outfile( + sog, reg.vine_key_of[k], outfile_type_enum, reg.outfile_remote_name[k] @@ -209,39 +208,39 @@ def build_python_graph(self): return reg - def build_c_graph(self, reg): + def build_sog(self, reg): assert reg is not None, "Python graph must be built before building the C graph" - c_graph = cdagvine.vine_task_graph_create(self._taskvine) + sog = cdagvine.sog_create(self._taskvine) # C side vine task graph must be tuned before adding nodes and dependencies self.tune_vine_manager() - self.tune_vine_task_graph(c_graph) + self.tune_sog(sog) topo_order = reg.get_topological_order() for k in topo_order: - cdagvine.vine_task_graph_add_node( - c_graph, + cdagvine.sog_add_node( + sog, reg.vine_key_of[k], ) for pk in reg.parents_of[k]: - cdagvine.vine_task_graph_add_dependency(c_graph, reg.vine_key_of[pk], reg.vine_key_of[k]) + cdagvine.sog_add_dependency(sog, reg.vine_key_of[pk], reg.vine_key_of[k]) - cdagvine.vine_task_graph_compute_topology_metrics(c_graph) + cdagvine.sog_compute_topology_metrics(sog) - return c_graph + return sog def build_graphs(self, target_keys): # build Python DAG (logical topology) reg = self.build_python_graph() # build C DAG (physical topology) - c_graph = self.build_c_graph(reg) + sog = self.build_sog(reg) # assign outfile types - self.assign_outfile_types(target_keys, reg, c_graph) + self.assign_outfile_types(target_keys, reg, sog) - return reg, c_graph + return reg, sog def run(self, collection_dict, target_keys=None, params={}, hoisting_modules=[], env_files={}): # first update the params so that they can be used for the following construction @@ -250,23 +249,24 @@ def run(self, collection_dict, target_keys=None, params={}, hoisting_modules=[], self.task_dict = ensure_task_dict(collection_dict) # build graphs in both Python and C sides - reg, c_graph = self.build_graphs(target_keys) + reg, sog = self.build_graphs(target_keys) # create and install the library template on the manager - library = Library(self, self.param("proxy-library-name"), self.param("libcores")) + library = Library(self, self.param("libcores")) library.add_hoisting_modules(hoisting_modules) library.add_env_files(env_files) library.set_context_loader(RuntimeExecutionGraph.context_loader_func, context_loader_args=[cloudpickle.dumps(reg)]) library.install() + cdagvine.sog_set_proxy_library_name(sog, library.get_name()) # execute the graph on the C side print(f"Executing task graph, logs will be written to {self.runtime_directory}") - cdagvine.vine_task_graph_execute(c_graph) + cdagvine.sog_execute(sog) # clean up the library instances and template on the manager library.uninstall() # delete the C graph immediately after execution, so that the lifetime of this object is limited to the execution - cdagvine.vine_task_graph_delete(c_graph) + cdagvine.sog_delete(sog) # load results of target keys results = {} diff --git a/taskvine/src/graph/dagvine/library.py b/taskvine/src/graph/dagvine/library.py index cd8f7d498e..f5158d6656 100644 --- a/taskvine/src/graph/dagvine/library.py +++ b/taskvine/src/graph/dagvine/library.py @@ -1,6 +1,4 @@ import os, uuid, cloudpickle, types, time, random, hashlib, collections -from ndcctools.taskvine import cvine -from ndcctools.taskvine.dagvine import cdagvine from ndcctools.taskvine.dagvine.runtime_execution_graph import ( GraphKeyResult, RuntimeExecutionGraph, compute_dts_key, compute_sexpr_key, compute_single_key, hash_name, hashable @@ -9,10 +7,9 @@ class Library: - def __init__(self, py_manager, libname, libcores): + def __init__(self, py_manager, libcores): self.py_manager = py_manager - self.libname = libname - assert self.libname is not None, "libname must be provided" + self.libname = f"proxy-library-{str(uuid.uuid4())}" self.libcores = libcores assert self.libcores is not None, "libcores must be provided" @@ -38,6 +35,9 @@ def __init__(self, py_manager, libname, libcores): self.local_path = None self.remote_path = None + def get_name(self): + return self.libname + 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) diff --git a/taskvine/src/graph/dagvine/vine_task_graph.c b/taskvine/src/graph/dagvine/vine_task_graph.c deleted file mode 100644 index 5e376812b7..0000000000 --- a/taskvine/src/graph/dagvine/vine_task_graph.c +++ /dev/null @@ -1,982 +0,0 @@ -#include -#include -#include -#include -#include -#include -#include -#include -#include - -#include "priority_queue.h" -#include "list.h" -#include "debug.h" -#include "itable.h" -#include "xxmalloc.h" -#include "stringtools.h" -#include "random.h" -#include "hash_table.h" -#include "set.h" -#include "timestamp.h" -#include "progress_bar.h" -#include "macros.h" - -#include "vine_task_graph.h" -#include "vine_manager.h" -#include "vine_task_node.h" -#include "vine_worker_info.h" -#include "vine_task.h" -#include "vine_file.h" -#include "vine_mount.h" -#include "taskvine.h" - - -static volatile sig_atomic_t interrupted = 0; - -/*************************************************************/ -/* Private Functions */ -/*************************************************************/ - -/** - * Handle the SIGINT signal. - * @param signal Reference to the signal. - */ -static void handle_sigint(int signal) -{ - interrupted = 1; -} - -/** - * Calculate the priority of a node given the priority mode. - * @param node Reference to the node object. - * @param priority_mode Reference to the priority mode. - * @return The priority. - */ -static double vine_task_node_calculate_priority(struct vine_task_node *node, vine_task_graph_priority_mode_t priority_mode) -{ - if (!node) { - return 0; - } - - double priority = 0; - timestamp_t current_time = timestamp_get(); - - struct vine_task_node *parent_node; - - switch (priority_mode) { - case VINE_TASK_GRAPH_PRIORITY_MODE_RANDOM: - priority = random_double(); - break; - case VINE_TASK_GRAPH_PRIORITY_MODE_DEPTH_FIRST: - priority = (double)node->depth; - break; - case VINE_TASK_GRAPH_PRIORITY_MODE_BREADTH_FIRST: - priority = -(double)node->depth; - break; - case VINE_TASK_GRAPH_PRIORITY_MODE_FIFO: - priority = -(double)current_time; - break; - case VINE_TASK_GRAPH_PRIORITY_MODE_LIFO: - priority = (double)current_time; - break; - case VINE_TASK_GRAPH_PRIORITY_MODE_LARGEST_INPUT_FIRST: - LIST_ITERATE(node->parents, parent_node) - { - if (!parent_node->outfile) { - continue; - } - priority += (double)vine_file_size(parent_node->outfile); - } - break; - case VINE_TASK_GRAPH_PRIORITY_MODE_LARGEST_STORAGE_FOOTPRINT_FIRST: - LIST_ITERATE(node->parents, parent_node) - { - if (!parent_node->outfile) { - continue; - } - timestamp_t parent_task_completion_time = parent_node->task->time_workers_execute_last; - priority += (double)vine_file_size(parent_node->outfile) * (double)parent_task_completion_time; - } - break; - } - - return priority; -} - -/** - * Submit a node to the taskvine manager. - * @param tg Reference to the task graph object. - * @param node Reference to the node object. - */ -static void submit_node_task(struct vine_task_graph *tg, struct vine_task_node *node) -{ - if (!tg || !node) { - return; - } - - /* calculate the priority of the node */ - double priority = vine_task_node_calculate_priority(node, tg->task_priority_mode); - vine_task_set_priority(node->task, priority); - /* submit the task to the manager */ - int task_id = vine_submit(node->manager, node->task); - /* insert the task id to the task id to node map */ - itable_insert(tg->task_id_to_node, task_id, node); - - return; -} - -/** - * Submit the children of a node if all its dependencies are resolved. - * @param tg Reference to the task graph object. - * @param node Reference to the node object. - */ -static void submit_unblocked_children(struct vine_task_graph *tg, struct vine_task_node *node) -{ - if (!tg || !node) { - return; - } - - struct vine_task_node *child_node; - LIST_ITERATE(node->children, child_node) - { - /* Remove this parent from the child's pending set if it exists */ - if (child_node->pending_parents) { - /* Assert that this parent is indeed pending for the child */ - if (child_node->pending_parents && set_lookup(child_node->pending_parents, node)) { - set_remove(child_node->pending_parents, node); - } else { - debug(D_ERROR, "inconsistent pending set: child=%s missing parent=%s", child_node->node_key, node->node_key); - } - } - - /* If no more parents are pending, submit the child */ - if (!child_node->pending_parents || set_size(child_node->pending_parents) == 0) { - submit_node_task(tg, child_node); - } - } - - return; -} - -/** - * Get the topological order of the task graph. - * Must be called after all nodes and dependencies are added and the topology metrics are computed. - * @param tg Reference to the task graph object. - * @return The list of nodes in topological order. - */ -static struct list *get_topological_order(struct vine_task_graph *tg) -{ - if (!tg) { - return NULL; - } - - int total_nodes = hash_table_size(tg->nodes); - struct list *topo_order = list_create(); - struct hash_table *in_degree_map = hash_table_create(0, 0); - struct priority_queue *pq = priority_queue_create(total_nodes); - - char *key; - struct vine_task_node *node; - HASH_TABLE_ITERATE(tg->nodes, key, node) - { - int deg = list_size(node->parents); - hash_table_insert(in_degree_map, key, (void *)(intptr_t)deg); - if (deg == 0) { - priority_queue_push(pq, node, compute_lex_priority(node->node_key)); - } - } - - while (priority_queue_size(pq) > 0) { - struct vine_task_node *current = priority_queue_pop(pq); - list_push_tail(topo_order, current); - - struct vine_task_node *child; - LIST_ITERATE(current->children, child) - { - intptr_t raw_deg = (intptr_t)hash_table_lookup(in_degree_map, child->node_key); - int deg = (int)raw_deg - 1; - - hash_table_remove(in_degree_map, child->node_key); - hash_table_insert(in_degree_map, child->node_key, (void *)(intptr_t)deg); - - if (deg == 0) { - priority_queue_push(pq, child, compute_lex_priority(child->node_key)); - } - } - } - - if (list_size(topo_order) != total_nodes) { - debug(D_ERROR, "Error: task graph contains cycles or is malformed.\n"); - debug(D_ERROR, "Expected %d nodes, but only sorted %d.\n", total_nodes, list_size(topo_order)); - - HASH_TABLE_ITERATE(tg->nodes, key, node) - { - intptr_t raw_deg = (intptr_t)hash_table_lookup(in_degree_map, key); - int deg = (int)raw_deg; - if (deg > 0) { - debug(D_ERROR, " Node %s has in-degree %d. Parents:\n", key, deg); - struct vine_task_node *p; - LIST_ITERATE(node->parents, p) - { - debug(D_ERROR, " -> %s\n", p->node_key); - } - } - } - - list_delete(topo_order); - hash_table_delete(in_degree_map); - priority_queue_delete(pq); - exit(1); - } - - hash_table_delete(in_degree_map); - priority_queue_delete(pq); - return topo_order; -} - -/** - * Extract the weakly connected components of the task graph. - * This function is used only for debugging purposes at the moment. - * @param tg Reference to the task graph object. - * @return The list of weakly connected components. - */ -static struct list *extract_weakly_connected_components(struct vine_task_graph *tg) -{ - if (!tg) { - return NULL; - } - - struct set *visited = set_create(0); - struct list *components = list_create(); - - char *node_key; - struct vine_task_node *node; - HASH_TABLE_ITERATE(tg->nodes, node_key, 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_task_node *curr = list_pop_head(queue); - - struct vine_task_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_task_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 task graph. - * @param node Reference to the node object. - * @return The heavy score. - */ -static double compute_node_heavy_score(struct vine_task_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); -} - -/** - * Map a task to a node in the task graph. - * @param tg Reference to the task graph object. - * @param task Reference to the task object. - * @return The node object. - */ -static struct vine_task_node *get_node_by_task(struct vine_task_graph *tg, struct vine_task *task) -{ - if (!tg || !task) { - return NULL; - } - - if (task->type == VINE_TASK_TYPE_STANDARD) { - /* standard tasks are mapped directly to a node */ - return itable_lookup(tg->task_id_to_node, task->task_id); - } else if (task->type == VINE_TASK_TYPE_RECOVERY) { - /* note that recovery tasks are not mapped to any node but we still need the original node for pruning, - * so we look up the outfile of the task, then map it back to get the original node */ - struct vine_mount *mount; - LIST_ITERATE(task->output_mounts, mount) - { - if (mount->file->original_producer_task_id > 0) { - return itable_lookup(tg->task_id_to_node, mount->file->original_producer_task_id); - } - } - } - - debug(D_ERROR, "task %d has no original producer task id", task->task_id); - - return NULL; -} - -/*************************************************************/ -/* Public APIs */ -/*************************************************************/ - -/** Tune the task graph. - *@param tg Reference to the task graph object. - *@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_task_graph_tune(struct vine_task_graph *tg, const char *name, const char *value) -{ - if (!tg || !name || !value) { - return -1; - } - - if (strcmp(name, "failure-injection-step-percent") == 0) { - tg->failure_injection_step_percent = atof(value); - - } else if (strcmp(name, "task-priority-mode") == 0) { - if (strcmp(value, "random") == 0) { - tg->task_priority_mode = VINE_TASK_GRAPH_PRIORITY_MODE_RANDOM; - } else if (strcmp(value, "depth-first") == 0) { - tg->task_priority_mode = VINE_TASK_GRAPH_PRIORITY_MODE_DEPTH_FIRST; - } else if (strcmp(value, "breadth-first") == 0) { - tg->task_priority_mode = VINE_TASK_GRAPH_PRIORITY_MODE_BREADTH_FIRST; - } else if (strcmp(value, "fifo") == 0) { - tg->task_priority_mode = VINE_TASK_GRAPH_PRIORITY_MODE_FIFO; - } else if (strcmp(value, "lifo") == 0) { - tg->task_priority_mode = VINE_TASK_GRAPH_PRIORITY_MODE_LIFO; - } else if (strcmp(value, "largest-input-first") == 0) { - tg->task_priority_mode = VINE_TASK_GRAPH_PRIORITY_MODE_LARGEST_INPUT_FIRST; - } else if (strcmp(value, "largest-storage-footprint-first") == 0) { - tg->task_priority_mode = VINE_TASK_GRAPH_PRIORITY_MODE_LARGEST_STORAGE_FOOTPRINT_FIRST; - } else { - debug(D_ERROR, "invalid priority mode: %s", value); - return -1; - } - - } else if (strcmp(name, "target-results-dir") == 0) { - if (tg->target_results_dir) { - free(tg->target_results_dir); - } - if (mkdir(value, 0777) != 0 && errno != EEXIST) { - debug(D_ERROR, "failed to mkdir %s (errno=%d)", value, errno); - return -1; - } - tg->target_results_dir = xxstrdup(value); - - } else if (strcmp(name, "proxy-library-name") == 0) { - if (tg->proxy_library_name) { - free(tg->proxy_library_name); - } - tg->proxy_library_name = xxstrdup(value); - - } else if (strcmp(name, "proxy-function-name") == 0) { - if (tg->proxy_function_name) { - free(tg->proxy_function_name); - } - tg->proxy_function_name = xxstrdup(value); - - } else if (strcmp(name, "prune-depth") == 0) { - tg->prune_depth = atoi(value); - - } else { - debug(D_ERROR, "invalid parameter name: %s", name); - return -1; - - } - - return 0; -} - -/** - * Get the proxy library name (Python-side), shared by all tasks. - * @param tg Reference to the task graph object. - * @return The library name. - */ -const char *vine_task_graph_get_proxy_library_name(const struct vine_task_graph *tg) -{ - if (!tg) { - return NULL; - } - - return tg->proxy_library_name; -} - -/** - * Get the proxy function name (Python-side), shared by all tasks. - * @param tg Reference to the task graph object. - * @return The function name. - */ -const char *vine_task_graph_get_proxy_function_name(const struct vine_task_graph *tg) -{ - if (!tg) { - return NULL; - } - - return tg->proxy_function_name; -} - -/** - * Get the heavy score of a node in the task graph. - * @param tg Reference to the task graph object. - * @param node_key Reference to the node key. - * @return The heavy score. - */ -double vine_task_graph_get_node_heavy_score(const struct vine_task_graph *tg, const char *node_key) -{ - if (!tg) { - return -1; - } - - struct vine_task_node *node = hash_table_lookup(tg->nodes, node_key); - if (!node) { - return -1; - } - - return node->heavy_score; -} - -/** - * Get the local outfile source of a node in the task graph, only valid for local output files. - * The source of a local output file is the path on the local filesystem. - * @param tg Reference to the task graph object. - * @param node_key Reference to the node key. - * @return The local outfile source. - */ -const char *vine_task_graph_get_node_local_outfile_source(const struct vine_task_graph *tg, const char *node_key) -{ - if (!tg || !node_key) { - return NULL; - } - - struct vine_task_node *node = hash_table_lookup(tg->nodes, node_key); - if (!node) { - debug(D_ERROR, "node %s not found", node_key); - exit(1); - } - - if (node->outfile_type != VINE_NODE_OUTFILE_TYPE_LOCAL) { - debug(D_ERROR, "node %s is not a local output file", node_key); - exit(1); - } - - return node->outfile->source; -} - -/** - * Compute the topology metrics of the task graph, including depth, height, upstream and downstream counts, - * heavy scores, and weakly connected components. - * @param tg Reference to the task graph object. - */ -void vine_task_graph_compute_topology_metrics(struct vine_task_graph *tg) -{ - if (!tg) { - return; - } - - /* get nodes in topological order */ - struct list *topo_order = get_topological_order(tg); - if (!topo_order) { - return; - } - - char *node_key; - struct vine_task_node *node; - struct vine_task_node *parent_node; - struct vine_task_node *child_node; - - /* compute the depth of the node */ - 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; - } - } - } - - /* compute the height of the node */ - 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; - } - } - } - - /* compute the upstream and downstream counts for each node */ - struct hash_table *upstream_map = hash_table_create(0, 0); - struct hash_table *downstream_map = hash_table_create(0, 0); - HASH_TABLE_ITERATE(tg->nodes, node_key, node) - { - struct set *upstream = set_create(0); - struct set *downstream = set_create(0); - hash_table_insert(upstream_map, node_key, upstream); - hash_table_insert(downstream_map, node_key, downstream); - } - LIST_ITERATE(topo_order, node) - { - struct set *upstream = hash_table_lookup(upstream_map, node->node_key); - LIST_ITERATE(node->parents, parent_node) - { - struct set *parent_upstream = hash_table_lookup(upstream_map, parent_node->node_key); - set_union(upstream, parent_upstream); - set_insert(upstream, parent_node); - } - } - LIST_ITERATE_REVERSE(topo_order, node) - { - struct set *downstream = hash_table_lookup(downstream_map, node->node_key); - LIST_ITERATE(node->children, child_node) - { - struct set *child_downstream = hash_table_lookup(downstream_map, child_node->node_key); - set_union(downstream, child_downstream); - set_insert(downstream, child_node); - } - } - LIST_ITERATE(topo_order, node) - { - node->upstream_subgraph_size = set_size(hash_table_lookup(upstream_map, node->node_key)); - node->downstream_subgraph_size = set_size(hash_table_lookup(downstream_map, node->node_key)); - node->fan_in = list_size(node->parents); - node->fan_out = list_size(node->children); - set_delete(hash_table_lookup(upstream_map, node->node_key)); - set_delete(hash_table_lookup(downstream_map, node->node_key)); - } - hash_table_delete(upstream_map); - hash_table_delete(downstream_map); - - /* compute the heavy score for each node */ - LIST_ITERATE(topo_order, node) - { - node->heavy_score = compute_node_heavy_score(node); - } - - /* extract weakly connected components */ - struct list *weakly_connected_components = extract_weakly_connected_components(tg); - 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_delete(topo_order); - - return; -} - -/** - * Create a new node and track it in the task graph. - * @param tg Reference to the task graph object. - * @param node_key Reference to the node key. - * @return A new node object. - */ -struct vine_task_node *vine_task_graph_add_node(struct vine_task_graph *tg, const char *node_key) -{ - if (!tg || !node_key) { - return NULL; - } - - /* if the node already exists, skip creating a new one */ - struct vine_task_node *node = hash_table_lookup(tg->nodes, node_key); - if (!node) { - node = vine_task_node_create(tg->manager, - node_key, - tg->proxy_library_name, - tg->proxy_function_name, - tg->target_results_dir, - tg->prune_depth); - - if (!node) { - debug(D_ERROR, "failed to create node %s", node_key); - vine_task_graph_delete(tg); - exit(1); - } - - hash_table_insert(tg->nodes, node_key, node); - } - - return node; -} - -/** - * Create a new task graph object and bind a manager to it. - * @param q Reference to the manager object. - * @return A new task graph object. - */ -struct vine_task_graph *vine_task_graph_create(struct vine_manager *q) -{ - struct vine_task_graph *tg = xxmalloc(sizeof(struct vine_task_graph)); - - tg->manager = q; - - tg->nodes = hash_table_create(0, 0); - tg->task_id_to_node = itable_create(0); - tg->outfile_cachename_to_node = hash_table_create(0, 0); - - tg->target_results_dir = xxstrdup(tg->manager->runtime_directory); // default to current working directory - tg->proxy_library_name = NULL; - tg->proxy_function_name = NULL; - - tg->prune_depth = 1; - - tg->task_priority_mode = VINE_TASK_GRAPH_PRIORITY_MODE_LARGEST_INPUT_FIRST; - tg->failure_injection_step_percent = -1.0; - - /* enable debug system for C code since it uses a separate debug system instance - * from the Python bindings. Use the same function that the manager uses. */ - char *debug_tmp = string_format("%s/vine-logs/debug", tg->manager->runtime_directory); - vine_enable_debug_log(debug_tmp); - free(debug_tmp); - - return tg; -} - -/** - * Add a dependency between two nodes in the task graph. Note that the input-output file relationship - * is not handled here, because their file names may have not been determined yet. - * @param tg Reference to the task graph object. - * @param parent_key Reference to the parent node key. - * @param child_key Reference to the child node key. - */ -void vine_task_graph_add_dependency(struct vine_task_graph *tg, const char *parent_key, const char *child_key) -{ - if (!tg || !parent_key || !child_key) { - return; - } - - struct vine_task_node *parent_node = hash_table_lookup(tg->nodes, parent_key); - struct vine_task_node *child_node = hash_table_lookup(tg->nodes, child_key); - if (!parent_node) { - debug(D_ERROR, "parent node %s not found", parent_key); - char *node_key = NULL; - struct vine_task_node *node; - printf("parent_keys:\n"); - HASH_TABLE_ITERATE(tg->nodes, node_key, node) - { - printf(" %s\n", node->node_key); - } - exit(1); - } - if (!child_node) { - debug(D_ERROR, "child node %s not found", child_key); - exit(1); - } - - list_push_tail(child_node->parents, parent_node); - list_push_tail(parent_node->children, child_node); - - return; -} - -/** - * Set the outfile of a node in the task graph. - * This involves declaring the output file and adding it to the task. - * @param tg Reference to the task graph object. - * @param node_key Reference to the node key. - * @param outfile_type Reference to the outfile type. - * @param outfile_remote_name Reference to the outfile remote name that the task will generate remotely. - */ -void vine_task_graph_set_node_outfile(struct vine_task_graph *tg, const char *node_key, vine_task_node_outfile_type_t outfile_type, const char *outfile_remote_name) -{ - if (!tg || !node_key || !outfile_remote_name) { - return; - } - - struct vine_task_node *node = hash_table_lookup(tg->nodes, node_key); - if (!node) { - return; - } - - vine_task_node_set_outfile(node, outfile_type, outfile_remote_name); - - return; -} - -/** - * Execute the task graph. This must be called after all nodes and dependencies are added and the topology metrics are computed. - * @param tg Reference to the task graph object. - */ -void vine_task_graph_execute(struct vine_task_graph *tg) -{ - if (!tg) { - return; - } - - signal(SIGINT, handle_sigint); - - debug(D_VINE, "start executing task graph"); - - /* print the info of all nodes */ - char *node_key; - struct vine_task_node *node; - HASH_TABLE_ITERATE(tg->nodes, node_key, node) - { - vine_task_node_print_info(node); - } - - /* enable return recovery tasks */ - vine_enable_return_recovery_tasks(tg->manager); - - /* create mapping from task_id and outfile cached_name to node */ - HASH_TABLE_ITERATE(tg->nodes, node_key, node) - { - if (node->outfile) { - hash_table_insert(tg->outfile_cachename_to_node, node->outfile->cached_name, node); - } - } - - /* add the parents' outfiles as inputs to the task */ - struct list *topo_order = get_topological_order(tg); - LIST_ITERATE(topo_order, node) - { - struct vine_task_node *parent_node; - LIST_ITERATE(node->parents, parent_node) - { - if (parent_node->outfile) { - vine_task_add_input(node->task, parent_node->outfile, parent_node->outfile_remote_name, VINE_TRANSFER_ALWAYS); - } - } - } - - /* initialize pending_parents for all nodes */ - HASH_TABLE_ITERATE(tg->nodes, node_key, node) - { - struct vine_task_node *parent_node; - LIST_ITERATE(node->parents, parent_node) - { - if (node->pending_parents) { - /* Use parent_node->node_key to ensure pointer consistency */ - set_insert(node->pending_parents, parent_node); - } - } - } - - /* enqueue those without dependencies */ - HASH_TABLE_ITERATE(tg->nodes, node_key, node) - { - if (!node->pending_parents || set_size(node->pending_parents) == 0) { - submit_node_task(tg, node); - } - } - - /* calculate steps to inject failure */ - double next_failure_threshold = -1.0; - if (tg->failure_injection_step_percent > 0) { - next_failure_threshold = tg->failure_injection_step_percent / 100.0; - } - - struct ProgressBar *pbar = progress_bar_init("Executing Tasks"); - struct ProgressBarPart *regular_tasks_part = progress_bar_create_part("Regular", hash_table_size(tg->nodes)); - struct ProgressBarPart *recovery_tasks_part = progress_bar_create_part("Recovery", 0); - progress_bar_bind_part(pbar, regular_tasks_part); - progress_bar_bind_part(pbar, recovery_tasks_part); - - int wait_timeout = 2; - - while (regular_tasks_part->current < regular_tasks_part->total) { - if (interrupted) { - break; - } - - struct vine_task *task = vine_wait(tg->manager, wait_timeout); - progress_bar_set_part_total(pbar, recovery_tasks_part, tg->manager->num_submitted_recovery_tasks); - if (task) { - /* retrieve all possible tasks */ - wait_timeout = 0; - - /* get the original node by task id */ - struct vine_task_node *node = get_node_by_task(tg, 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); - } - - /* in case of failure, resubmit this task */ - if (node->task->result != VINE_RESULT_SUCCESS || node->task->exit_code != 0) { - if (node->retry_attempts_left <= 0) { - debug(D_ERROR, "Task %d failed (result=%d, exit=%d). Node %s has no retries left. Aborting.", task->task_id, node->task->result, node->task->exit_code, node->node_key); - vine_task_graph_delete(tg); - exit(1); - } - node->retry_attempts_left--; - debug(D_VINE | D_NOTICE, "Task %d failed (result=%d, exit=%d). Retrying node %s (remaining=%d)...", task->task_id, node->task->result, node->task->exit_code, node->node_key, node->retry_attempts_left); - vine_task_reset(node->task); - submit_node_task(tg, node); - continue; - } - - /* if the outfile is set to save on the sharedfs, stat to get the size of the file */ - switch (node->outfile_type) { - case VINE_NODE_OUTFILE_TYPE_SHARED_FILE_SYSTEM: { - struct stat info; - int result = stat(node->outfile_remote_name, &info); - if (result < 0) { - if (node->retry_attempts_left <= 0) { - debug(D_ERROR, "Task %d succeeded but missing sharedfs output %s; no retries left for node %s. Aborting.", task->task_id, node->outfile_remote_name, node->node_key); - vine_task_graph_delete(tg); - exit(1); - } - node->retry_attempts_left--; - debug(D_VINE | D_NOTICE, "Task %d succeeded but missing sharedfs output %s; retrying node %s (remaining=%d)...", task->task_id, node->outfile_remote_name, node->node_key, node->retry_attempts_left); - vine_task_reset(node->task); - submit_node_task(tg, node); - continue; - } - node->outfile_size_bytes = info.st_size; - break; - } - case VINE_NODE_OUTFILE_TYPE_LOCAL: - case VINE_NODE_OUTFILE_TYPE_TEMP: - node->outfile_size_bytes = node->outfile->size; - break; - } - debug(D_VINE, "Node %s completed with outfile %s size: %zu bytes", node->node_key, node->outfile_remote_name, node->outfile_size_bytes); - - /* mark the node as completed */ - node->completed = 1; - - /* prune nodes on task completion */ - vine_task_node_prune_ancestors(node); - - /* skip recovery tasks */ - if (task->type == VINE_TASK_TYPE_RECOVERY) { - progress_bar_update_part(pbar, recovery_tasks_part, 1); - continue; - } - - /* set the start time to the submit time of the first regular task */ - if (regular_tasks_part->current == 0) { - progress_bar_set_start_time(pbar, task->time_when_commit_start); - } - - /* update critical time */ - vine_task_node_update_critical_time(node, task->time_workers_execute_last); - - /* mark this regular task as completed */ - progress_bar_update_part(pbar, regular_tasks_part, 1); - - /* inject failure */ - if (tg->failure_injection_step_percent > 0) { - double progress = (double)regular_tasks_part->current / (double)regular_tasks_part->total; - if (progress >= next_failure_threshold && evict_random_worker(tg->manager)) { - debug(D_VINE, "evicted a worker at %.2f%% (threshold %.2f%%)", progress * 100, next_failure_threshold * 100); - next_failure_threshold += tg->failure_injection_step_percent / 100.0; - } - } - - /* enqueue the output file for replication */ - switch (node->outfile_type) { - case VINE_NODE_OUTFILE_TYPE_TEMP: - vine_task_node_replicate_outfile(node); - break; - case VINE_NODE_OUTFILE_TYPE_LOCAL: - case VINE_NODE_OUTFILE_TYPE_SHARED_FILE_SYSTEM: - break; - } - - /* submit children nodes with dependencies all resolved */ - submit_unblocked_children(tg, node); - } else { - wait_timeout = 2; - progress_bar_update_part(pbar, recovery_tasks_part, 0); // refresh the time and total for recovery tasks - } - } - - progress_bar_finish(pbar); - progress_bar_delete(pbar); - - double total_time_spent_on_unlink_local_files = 0; - double total_time_spent_on_prune_ancestors_of_temp_node = 0; - double total_time_spent_on_prune_ancestors_of_persisted_node = 0; - HASH_TABLE_ITERATE(tg->nodes, node_key, node) - { - total_time_spent_on_unlink_local_files += node->time_spent_on_unlink_local_files; - total_time_spent_on_prune_ancestors_of_temp_node += node->time_spent_on_prune_ancestors_of_temp_node; - total_time_spent_on_prune_ancestors_of_persisted_node += node->time_spent_on_prune_ancestors_of_persisted_node; - } - total_time_spent_on_unlink_local_files /= 1e6; - total_time_spent_on_prune_ancestors_of_temp_node /= 1e6; - total_time_spent_on_prune_ancestors_of_persisted_node /= 1e6; - - debug(D_VINE, "total time spent on prune ancestors of temp node: %.6f seconds\n", total_time_spent_on_prune_ancestors_of_temp_node); - debug(D_VINE, "total time spent on prune ancestors of persisted node: %.6f seconds\n", total_time_spent_on_prune_ancestors_of_persisted_node); - debug(D_VINE, "total time spent on unlink local files: %.6f seconds\n", total_time_spent_on_unlink_local_files); - - return; -} - -/** - * Delete a task graph object. - * @param tg Reference to the task graph object. - */ -void vine_task_graph_delete(struct vine_task_graph *tg) -{ - if (!tg) { - return; - } - - char *node_key; - struct vine_task_node *node; - HASH_TABLE_ITERATE(tg->nodes, node_key, node) - { - if (node->infile) { - vine_prune_file(tg->manager, node->infile); - hash_table_remove(tg->manager->file_table, node->infile->cached_name); - } - if (node->outfile) { - vine_prune_file(tg->manager, node->outfile); - hash_table_remove(tg->outfile_cachename_to_node, node->outfile->cached_name); - hash_table_remove(tg->manager->file_table, node->outfile->cached_name); - } - vine_task_node_delete(node); - } - - free(tg->proxy_library_name); - free(tg->proxy_function_name); - - hash_table_delete(tg->nodes); - itable_delete(tg->task_id_to_node); - hash_table_delete(tg->outfile_cachename_to_node); - free(tg); -} diff --git a/taskvine/src/graph/dagvine/vine_task_graph.h b/taskvine/src/graph/dagvine/vine_task_graph.h deleted file mode 100644 index 47990e7211..0000000000 --- a/taskvine/src/graph/dagvine/vine_task_graph.h +++ /dev/null @@ -1,126 +0,0 @@ -#ifndef VINE_TASK_GRAPH_H -#define VINE_TASK_GRAPH_H - -#include "vine_task.h" -#include "hash_table.h" -#include "list.h" -#include "vine_manager.h" -#include "set.h" -#include "vine_task_node.h" -#include "taskvine.h" - - -/** Select priority algorithm for task graph task scheduling. */ -typedef enum { - VINE_TASK_GRAPH_PRIORITY_MODE_RANDOM = 0, /**< Assign random priority to tasks */ - VINE_TASK_GRAPH_PRIORITY_MODE_DEPTH_FIRST, /**< Prioritize deeper tasks first */ - VINE_TASK_GRAPH_PRIORITY_MODE_BREADTH_FIRST, /**< Prioritize shallower tasks first */ - VINE_TASK_GRAPH_PRIORITY_MODE_FIFO, /**< First in, first out priority */ - VINE_TASK_GRAPH_PRIORITY_MODE_LIFO, /**< Last in, first out priority */ - VINE_TASK_GRAPH_PRIORITY_MODE_LARGEST_INPUT_FIRST, /**< Prioritize tasks with larger inputs first */ - VINE_TASK_GRAPH_PRIORITY_MODE_LARGEST_STORAGE_FOOTPRINT_FIRST /**< Prioritize tasks with larger storage footprint first */ -} vine_task_graph_priority_mode_t; - -/** The task graph object. */ -struct vine_task_graph { - struct vine_manager *manager; - struct hash_table *nodes; - struct itable *task_id_to_node; - struct hash_table *outfile_cachename_to_node; - - /* Results of target keys will be stored in this directory. - * This dir path can not necessarily be a shared file system directory, - * output files will be retrieved through the network instead, - * as long as the manager can access it. */ - char *target_results_dir; - - char *proxy_library_name; // Python-side proxy library name (shared by all tasks) - char *proxy_function_name; // Python-side proxy function name (shared by all tasks) - - int prune_depth; - - vine_task_graph_priority_mode_t task_priority_mode; // priority mode for task graph task scheduling - double failure_injection_step_percent; // 0 - 100, the percentage of steps to inject failure -}; - - -/* Public APIs for operating the task graph */ - -/** Create a task graph object and return it. -@param q Reference to the current manager object. -@return A new task graph object. -*/ -struct vine_task_graph *vine_task_graph_create(struct vine_manager *q); - -/** Create a new node in the task graph. -@param tg Reference to the task graph object. -@param node_key Reference to the node key. -@return A new node object. -*/ -struct vine_task_node *vine_task_graph_add_node(struct vine_task_graph *tg, const char *node_key); - -/** Add a dependency between two nodes in the task graph. -@param tg Reference to the task graph object. -@param parent_key Reference to the parent node key. -@param child_key Reference to the child node key. -*/ -void vine_task_graph_add_dependency(struct vine_task_graph *tg, const char *parent_key, const char *child_key); - -/** Finalize the metrics of the task graph. -@param tg Reference to the task graph object. -*/ -void vine_task_graph_compute_topology_metrics(struct vine_task_graph *tg); - -/** Get the heavy score of a node in the task graph. -@param tg Reference to the task graph object. -@param node_key Reference to the node key. -@return The heavy score. -*/ -double vine_task_graph_get_node_heavy_score(const struct vine_task_graph *tg, const char *node_key); - -/** Set the type of the node-output file. -@param tg Reference to the task graph object. -@param node_key Reference to the node key. -@param outfile_type Reference to the output file type. -@param outfile_remote_name Reference to the output file remote name. -*/ -void vine_task_graph_set_node_outfile(struct vine_task_graph *tg, const char *node_key, vine_task_node_outfile_type_t outfile_type, const char *outfile_remote_name); - -/** Execute the task graph. -@param tg Reference to the task graph object. -*/ -void vine_task_graph_execute(struct vine_task_graph *tg); - -/** Get the local outfile source of a node in the task graph. -@param tg Reference to the task graph object. -@param node_key Reference to the node key. -@return The local outfile source. -*/ -const char *vine_task_graph_get_node_local_outfile_source(const struct vine_task_graph *tg, const char *node_key); - -/** Delete a task graph object. -@param tg Reference to the task graph object. -*/ -void vine_task_graph_delete(struct vine_task_graph *tg); - -/** Get the library name of the task graph. -@param tg Reference to the task graph object. -@return The library name. -*/ -const char *vine_task_graph_get_proxy_library_name(const struct vine_task_graph *tg); - -/** Get the function name of the task graph. -@param tg Reference to the task graph object. -@return The function name. -*/ -const char *vine_task_graph_get_proxy_function_name(const struct vine_task_graph *tg); - -/** Tune the task graph. -@param tg Reference to the task graph object. -@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_task_graph_tune(struct vine_task_graph *tg, const char *name, const char *value); - -#endif // VINE_TASK_GRAPH_H diff --git a/taskvine/src/graph/dagvine/vine_task_graph.i b/taskvine/src/graph/dagvine/vine_task_graph.i deleted file mode 100644 index 4f1f2d95fd..0000000000 --- a/taskvine/src/graph/dagvine/vine_task_graph.i +++ /dev/null @@ -1,21 +0,0 @@ -/* SWIG interface for local dagvine graph bindings */ -%module cdagvine - -%{ -#include "int_sizes.h" -#include "vine_task_graph.h" -#include "vine_task_node.h" /* expose outfile type enum to SWIG */ -%} - -%include "stdint.i" -%include "int_sizes.h" - -/* Import existing SWIG interface for type information (do not wrap again) */ -%import "../../bindings/python3/taskvine.i" - -/* Expose only the dagvine task graph APIs */ -%ignore vine_task_node_checkpoint_outfile; /* avoid exporting unimplemented/optional symbol */ -%include "vine_task_node.h" /* export vine_task_node_outfile_type_t values */ -%include "vine_task_graph.h" - - From fa5095877d44ce31eac0f426ed6ca17452ac9404 Mon Sep 17 00:00:00 2001 From: Jin Zhou Date: Sat, 18 Oct 2025 15:43:54 -0400 Subject: [PATCH 045/113] sog --- .../graph/dagvine/runtime_execution_graph.py | 386 +++++++ .../dagvine/strategic_orchestration_graph.c | 987 ++++++++++++++++++ .../dagvine/strategic_orchestration_graph.h | 132 +++ .../dagvine/strategic_orchestration_graph.i | 21 + 4 files changed, 1526 insertions(+) create mode 100644 taskvine/src/graph/dagvine/runtime_execution_graph.py create mode 100644 taskvine/src/graph/dagvine/strategic_orchestration_graph.c create mode 100644 taskvine/src/graph/dagvine/strategic_orchestration_graph.h create mode 100644 taskvine/src/graph/dagvine/strategic_orchestration_graph.i diff --git a/taskvine/src/graph/dagvine/runtime_execution_graph.py b/taskvine/src/graph/dagvine/runtime_execution_graph.py new file mode 100644 index 0000000000..24e246f152 --- /dev/null +++ b/taskvine/src/graph/dagvine/runtime_execution_graph.py @@ -0,0 +1,386 @@ +# 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.utils import load_variable_from_library + +import os +import hashlib +import time +import cloudpickle +import collections +import uuid +import random +from collections import deque + +try: + import dask +except ImportError: + dask = None + +try: + import dask._task_spec as dts +except ImportError: + dts = None + + +def hash_name(*args): + out_str = "" + for arg in args: + out_str += str(arg) + return hashlib.sha256(out_str.encode('utf-8')).hexdigest()[:32] + + +def hashable(s): + try: + hash(s) + return True + except TypeError: + return False + + +def dist_func(mode, low, high): + """ Generate a random value from a distribution. """ + if not mode: + return 0 + + assert mode in ["uniform", "normal", "lognormal", "pareto", "mix"] + + # uniform distribution, flat spread + def uniform_dist(): + return random.uniform(low, high) + + # normal distribution, centered in the middle + def normal_dist(): + mu, sigma = (low + high) / 2, (high - low) / 6 + return min(max(random.gauss(mu, sigma), low), high) + + # lognormal distribution, long tail + def lognormal_dist(): + val = random.lognormvariate(0, 1) + val = val / (1 + val) + return low + (high - low) * val + + # pareto distribution, very heavy tail + def pareto_dist(alpha=2.0): + val = random.paretovariate(alpha) + val = val / (1 + val) + return low + (high - low) * val + + # mixture: most small values, few large ones + def mix_dist(): + if random.random() < 0.9: + return random.uniform(low, (low + high) / 2) + else: + return random.uniform((low + high) / 2, high) + + return { + "uniform": uniform_dist, + "normal": normal_dist, + "lognormal": lognormal_dist, + "pareto": pareto_dist, + "mix": mix_dist, + }[mode]() + + +class GraphKeyResult: + """ A wrapper class to store the result of a task and the extra size in MB to allocate for this object in testing mode to evaluate storage consumption + and peer transfer performance across all workers. """ + 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 + + +class RuntimeExecutionGraph: + """ + The RuntimeExecutionGraph class constructs the task graph and manages task dependencies. + This class is cloudpickled and sent to workers, where it is hoisted by the library instance. + The global RuntimeExecutionGraph object then serves as the execution context: whenever a task key is invoked, + the system resolves the corresponding graph key from the Vine key and executes the mapped function + to produce the result. + """ + def __init__(self, task_dict, + shared_file_system_dir=None, + extra_task_output_size_mb=["uniform", 0, 0], + extra_task_sleep_time=["uniform", 0, 0]): + self.task_dict = task_dict + self.shared_file_system_dir = shared_file_system_dir + + if self.shared_file_system_dir: + os.makedirs(self.shared_file_system_dir, exist_ok=True) + + if dts: + for k, v in self.task_dict.items(): + if isinstance(v, dts.GraphNode): + assert isinstance(v, (dts.Alias, dts.Task, dts.DataNode)), f"Unsupported task type for key {k}: {v.__class__}" + + self.parents_of, self.children_of = self._build_dependencies(self.task_dict) + self.depth_of = self._calculate_depths() + + self.vine_key_of = {k: hash_name(k) for k in task_dict.keys()} + self.key_of_vine_key = {hash_name(k): k for k in task_dict.keys()} + + self.outfile_remote_name = {key: f"{uuid.uuid4()}.pkl" for key in self.task_dict.keys()} + self.outfile_type = {key: None for key in self.task_dict.keys()} + + # testing params + self.extra_task_output_size_mb = self._calculate_extra_size_mb_of(extra_task_output_size_mb) + self.extra_sleep_time_of = self._calculate_extra_sleep_time_of(extra_task_sleep_time) + + def set_outfile_type_of(self, k, outfile_type_str): + assert outfile_type_str in ["local", "shared-file-system", "temp"] + self.outfile_type[k] = outfile_type_str + if outfile_type_str == "shared-file-system": + self.outfile_remote_name[k] = os.path.join(self.shared_file_system_dir, self.outfile_remote_name[k]) + + def _calculate_extra_size_mb_of(self, extra_task_output_size_mb): + assert isinstance(extra_task_output_size_mb, list) and len(extra_task_output_size_mb) == 3 + mode, low, high = extra_task_output_size_mb + low, high = int(low), int(high) + assert low <= high + + max_depth = max(depth for depth in self.depth_of.values()) + extra_size_mb_of = {} + for k in self.task_dict.keys(): + if self.depth_of[k] == max_depth or self.depth_of[k] == max_depth - 1: + extra_size_mb_of[k] = 0 + continue + extra_size_mb_of[k] = dist_func(mode, low, high) + + return extra_size_mb_of + + def _calculate_extra_sleep_time_of(self, extra_task_sleep_time): + assert isinstance(extra_task_sleep_time, list) and len(extra_task_sleep_time) == 3 + mode, low, high = extra_task_sleep_time + low, high = int(low), int(high) + assert low <= high + + extra_sleep_time_of = {} + for k in self.task_dict.keys(): + extra_sleep_time_of[k] = dist_func(mode, low, high) + + return extra_sleep_time_of + + def _calculate_depths(self): + depth_of = {key: 0 for key in self.task_dict.keys()} + + topo_order = self.get_topological_order() + for key in topo_order: + if self.parents_of[key]: + depth_of[key] = max(depth_of[parent] for parent in self.parents_of[key]) + 1 + else: + depth_of[key] = 0 + + return depth_of + + def set_outfile_remote_name_of(self, key, outfile_remote_name): + self.outfile_remote_name[key] = outfile_remote_name + + def is_dts_key(self, k): + if not hasattr(dask, "_task_spec"): + return False + import dask._task_spec as dts + return isinstance(self.task_dict[k], (dts.Task, dts.TaskRef, dts.Alias, dts.DataNode, dts.NestedContainer)) + + def _build_dependencies(self, task_dict): + def _find_sexpr_parents(sexpr): + if hashable(sexpr) and sexpr in task_dict.keys(): + return {sexpr} + elif isinstance(sexpr, (list, tuple)): + deps = set() + for x in sexpr: + deps |= _find_sexpr_parents(x) + return deps + elif isinstance(sexpr, dict): + deps = set() + for k, v in sexpr.items(): + deps |= _find_sexpr_parents(k) + deps |= _find_sexpr_parents(v) + return deps + else: + return set() + + parents_of = collections.defaultdict(set) + children_of = collections.defaultdict(set) + + for k, value in task_dict.items(): + if self.is_dts_key(k): + # in the new Dask expression, each value is an object from dask._task_spec, could be + # a Task, Alias, TaskRef, etc., but they all share the same base class the dependencies + # field is of type frozenset(), without recursive ancestor dependencies involved + parents_of[k] = value.dependencies + else: + # the value could be a sexpr, e.g., the old Dask representation + parents_of[k] = _find_sexpr_parents(value) + + for k, deps in parents_of.items(): + for dep in deps: + children_of[dep].add(k) + + return parents_of, children_of + + def save_result_of_key(self, key, result): + with open(self.outfile_remote_name[key], "wb") as f: + result_obj = GraphKeyResult(result, extra_size_mb=self.extra_task_output_size_mb[key]) + cloudpickle.dump(result_obj, f) + + def load_result_of_key(self, key, target_results_dir=None): + if self.outfile_type[key] == "local": + # nodes with local outfile type will have their results stored in the target results directory + # when the execution is done, users can call this function to load the results from the target results directory + # note that data is transferred through the link in sequence, not in parallel through the shared file system + outfile_path = os.path.join(target_results_dir, self.outfile_remote_name[key]) + else: + # workers user this branch to load results from either local or shared file system + # if a node-local output, then data is stored in the task sandbox and the remote name is just the filename + # if a shared file system output, then remote name is the full path to the file + outfile_path = self.outfile_remote_name[key] + try: + with open(outfile_path, "rb") as f: + result_obj = cloudpickle.load(f) + assert isinstance(result_obj, GraphKeyResult), "Loaded object is not of type GraphKeyResult" + return result_obj.result + except FileNotFoundError: + raise FileNotFoundError(f"Output file for key {key} not found at {outfile_path}") + + def get_topological_order(self): + in_degree = {key: len(self.parents_of[key]) for key in self.task_dict.keys()} + queue = deque([key for key, degree in in_degree.items() if degree == 0]) + topo_order = [] + + while queue: + current = queue.popleft() + topo_order.append(current) + + for child in self.children_of[current]: + in_degree[child] -= 1 + if in_degree[child] == 0: + queue.append(child) + + if len(topo_order) != len(self.task_dict): + print(f"len(topo_order): {len(topo_order)}") + print(f"len(self.task_dict): {len(self.task_dict)}") + raise ValueError("Failed to create topo order, the dependencies may be cyclic or problematic") + + return topo_order + + def __del__(self): + if hasattr(self, 'outfile_remote_name') and self.outfile_remote_name: + for k in self.outfile_remote_name.keys(): + if self.outfile_type.get(k) == "shared-file-system" and os.path.exists(self.outfile_remote_name[k]): + os.remove(self.outfile_remote_name[k]) + + @staticmethod + def context_loader_func(reg_pkl): + reg = cloudpickle.loads(reg_pkl) + + if not isinstance(reg, RuntimeExecutionGraph): + raise TypeError("reg_pkl is not of type RuntimeExecutionGraph") + + return { + "reg": reg, + } + + +def compute_dts_key(reg, k, v): + """ + Compute the result of a Dask task node from dask._task_spec. + + Each value `v` may be an instance of Task, Alias, or DataNode, all of which + inherit from the same base class. The `dependencies` field is a frozenset + containing direct dependencies only (no recursive ancestry). + + The function resolves each dependency from the reg, constructs an + input dictionary, and then executes the node according to its type. + """ + try: + import dask._task_spec as dts + except ImportError: + raise ImportError("Dask is not installed") + + input_dict = {dep: reg.load_result_of_key(dep) for dep in v.dependencies} + + try: + if isinstance(v, dts.Alias): + assert len(v.dependencies) == 1, "Expected exactly one dependency" + return reg.load_result_of_key(next(iter(v.dependencies))) + elif isinstance(v, dts.Task): + return v(input_dict) + elif isinstance(v, dts.DataNode): + return v.value + else: + raise TypeError(f"unexpected node type: {type(v)} for key {k}") + except Exception as e: + raise Exception(f"Error while executing task {k}: {e}") + + +def compute_sexpr_key(reg, k, v): + """ + Evaluate a symbolic expression (S-expression) task within the task graph. + + Both DAGVine and legacy Dask represent computations as symbolic + expression trees (S-expressions). Each task value `v` encodes a nested + structure where: + - Leaf nodes are constants or task keys referencing parent results. + - Lists are recursively evaluated. + - Tuples of the form (func, arg1, arg2, ...) represent function calls. + + This function builds an input dictionary from all parent keys, then + recursively resolves and executes the expression until a final value + is produced. + """ + input_dict = {parent: reg.load_result_of_key(parent) for parent in reg.parents_of[k]} + + def _rec_call(expr): + try: + if expr in input_dict.keys(): + return input_dict[expr] + except TypeError: + pass + if isinstance(expr, list): + return [_rec_call(e) for e in expr] + if isinstance(expr, tuple) and len(expr) > 0 and callable(expr[0]): + res = expr[0](*[_rec_call(a) for a in expr[1:]]) + return res + return expr + + try: + return _rec_call(v) + except Exception as e: + raise Exception(f"Failed to invoke _rec_call(): {e}") + + +def compute_single_key(vine_key): + """ + Compute a single task identified by a Vine key within the current RuntimeExecutionGraph. + + The function retrieves the corresponding graph key and task object from the + global reg, determines the task type, and dispatches to the appropriate + execution interface — e.g., `compute_dts_key` for Dask-style task specs or + `compute_sexpr_key` for S-expression graphs. + + This design allows extensibility: for new graph representations, additional + compute interfaces can be introduced and registered here to handle new key types. + + After computation, the result is saved, the output file is validated, and + an optional delay (`extra_sleep_time_of`) is applied before returning. + """ + reg = load_variable_from_library('reg') + + k = reg.key_of_vine_key[vine_key] + v = reg.task_dict[k] + + if reg.is_dts_key(k): + result = compute_dts_key(reg, k, v) + else: + result = compute_sexpr_key(reg, k, v) + + reg.save_result_of_key(k, result) + if not os.path.exists(reg.outfile_remote_name[k]): + raise Exception(f"Output file {reg.outfile_remote_name[k]} does not exist after writing") + if os.stat(reg.outfile_remote_name[k]).st_size == 0: + raise Exception(f"Output file {reg.outfile_remote_name[k]} is empty after writing") + + time.sleep(reg.extra_sleep_time_of[k]) + + return True diff --git a/taskvine/src/graph/dagvine/strategic_orchestration_graph.c b/taskvine/src/graph/dagvine/strategic_orchestration_graph.c new file mode 100644 index 0000000000..c9c84eb9d2 --- /dev/null +++ b/taskvine/src/graph/dagvine/strategic_orchestration_graph.c @@ -0,0 +1,987 @@ +#include +#include +#include +#include +#include +#include +#include +#include +#include + +#include "priority_queue.h" +#include "list.h" +#include "debug.h" +#include "itable.h" +#include "xxmalloc.h" +#include "stringtools.h" +#include "random.h" +#include "hash_table.h" +#include "set.h" +#include "timestamp.h" +#include "progress_bar.h" +#include "macros.h" + +#include "strategic_orchestration_graph.h" +#include "vine_manager.h" +#include "vine_task_node.h" +#include "vine_worker_info.h" +#include "vine_task.h" +#include "vine_file.h" +#include "vine_mount.h" +#include "taskvine.h" + + +static volatile sig_atomic_t interrupted = 0; + +/*************************************************************/ +/* Private Functions */ +/*************************************************************/ + +/** + * Handle the SIGINT signal. + * @param signal Reference to the signal. + */ +static void handle_sigint(int signal) +{ + interrupted = 1; +} + +/** + * Calculate the priority of a node given the priority mode. + * @param node Reference to the node object. + * @param priority_mode Reference to the priority mode. + * @return The priority. + */ +static double vine_task_node_calculate_priority(struct vine_task_node *node, task_priority_mode_t priority_mode) +{ + if (!node) { + return 0; + } + + double priority = 0; + timestamp_t current_time = timestamp_get(); + + struct vine_task_node *parent_node; + + switch (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; + break; + case TASK_PRIORITY_MODE_LIFO: + priority = (double)current_time; + break; + case TASK_PRIORITY_MODE_LARGEST_INPUT_FIRST: + LIST_ITERATE(node->parents, parent_node) + { + if (!parent_node->outfile) { + continue; + } + priority += (double)vine_file_size(parent_node->outfile); + } + break; + case TASK_PRIORITY_MODE_LARGEST_STORAGE_FOOTPRINT_FIRST: + LIST_ITERATE(node->parents, parent_node) + { + if (!parent_node->outfile) { + continue; + } + timestamp_t parent_task_completion_time = parent_node->task->time_workers_execute_last; + priority += (double)vine_file_size(parent_node->outfile) * (double)parent_task_completion_time; + } + break; + } + + return priority; +} + +/** + * Submit a node to the taskvine manager. + * @param sog Reference to the strategic orchestration graph object. + * @param node Reference to the node object. + */ +static void submit_node_task(struct strategic_orchestration_graph *sog, struct vine_task_node *node) +{ + if (!sog || !node) { + return; + } + + /* calculate the priority of the node */ + double priority = vine_task_node_calculate_priority(node, sog->task_priority_mode); + vine_task_set_priority(node->task, priority); + /* submit the task to the manager */ + int task_id = vine_submit(sog->manager, node->task); + /* insert the task id to the task id to node map */ + itable_insert(sog->task_id_to_node, task_id, node); + + return; +} + +/** + * Submit the children of a node if all its dependencies are resolved. + * @param sog Reference to the strategic orchestration graph object. + * @param node Reference to the node object. + */ +static void submit_unblocked_children(struct strategic_orchestration_graph *sog, struct vine_task_node *node) +{ + if (!sog || !node) { + return; + } + + struct vine_task_node *child_node; + LIST_ITERATE(node->children, child_node) + { + /* Remove this parent from the child's pending set if it exists */ + if (child_node->pending_parents) { + /* Assert that this parent is indeed pending for the child */ + if (child_node->pending_parents && set_lookup(child_node->pending_parents, node)) { + set_remove(child_node->pending_parents, node); + } else { + debug(D_ERROR, "inconsistent pending set: child=%s missing parent=%s", child_node->node_key, node->node_key); + } + } + + /* If no more parents are pending, submit the child */ + if (!child_node->pending_parents || set_size(child_node->pending_parents) == 0) { + submit_node_task(sog, child_node); + } + } + + return; +} + +/** + * Get the topological order of the strategic orchestration graph. + * Must be called after all nodes and dependencies are added and the topology metrics are computed. + * @param sog Reference to the strategic orchestration graph object. + * @return The list of nodes in topological order. + */ +static struct list *get_topological_order(struct strategic_orchestration_graph *sog) +{ + if (!sog) { + return NULL; + } + + int total_nodes = hash_table_size(sog->nodes); + struct list *topo_order = list_create(); + struct hash_table *in_degree_map = hash_table_create(0, 0); + struct priority_queue *pq = priority_queue_create(total_nodes); + + char *key; + struct vine_task_node *node; + HASH_TABLE_ITERATE(sog->nodes, key, node) + { + int deg = list_size(node->parents); + hash_table_insert(in_degree_map, key, (void *)(intptr_t)deg); + if (deg == 0) { + priority_queue_push(pq, node, compute_lex_priority(node->node_key)); + } + } + + while (priority_queue_size(pq) > 0) { + struct vine_task_node *current = priority_queue_pop(pq); + list_push_tail(topo_order, current); + + struct vine_task_node *child; + LIST_ITERATE(current->children, child) + { + intptr_t raw_deg = (intptr_t)hash_table_lookup(in_degree_map, child->node_key); + int deg = (int)raw_deg - 1; + + hash_table_remove(in_degree_map, child->node_key); + hash_table_insert(in_degree_map, child->node_key, (void *)(intptr_t)deg); + + if (deg == 0) { + priority_queue_push(pq, child, compute_lex_priority(child->node_key)); + } + } + } + + if (list_size(topo_order) != total_nodes) { + debug(D_ERROR, "Error: strategic orchestration graph contains cycles or is malformed.\n"); + debug(D_ERROR, "Expected %d nodes, but only sorted %d.\n", total_nodes, list_size(topo_order)); + + HASH_TABLE_ITERATE(sog->nodes, key, node) + { + intptr_t raw_deg = (intptr_t)hash_table_lookup(in_degree_map, key); + int deg = (int)raw_deg; + if (deg > 0) { + debug(D_ERROR, " Node %s has in-degree %d. Parents:\n", key, deg); + struct vine_task_node *p; + LIST_ITERATE(node->parents, p) + { + debug(D_ERROR, " -> %s\n", p->node_key); + } + } + } + + list_delete(topo_order); + hash_table_delete(in_degree_map); + priority_queue_delete(pq); + exit(1); + } + + hash_table_delete(in_degree_map); + priority_queue_delete(pq); + return topo_order; +} + +/** + * Extract the weakly connected components of the strategic orchestration graph. + * This function is used only for debugging purposes at the moment. + * @param sog Reference to the strategic orchestration graph object. + * @return The list of weakly connected components. + */ +static struct list *extract_weakly_connected_components(struct strategic_orchestration_graph *sog) +{ + if (!sog) { + return NULL; + } + + struct set *visited = set_create(0); + struct list *components = list_create(); + + char *node_key; + struct vine_task_node *node; + HASH_TABLE_ITERATE(sog->nodes, node_key, 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_task_node *curr = list_pop_head(queue); + + struct vine_task_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_task_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 strategic orchestration graph. + * @param node Reference to the node object. + * @return The heavy score. + */ +static double compute_node_heavy_score(struct vine_task_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); +} + +/** + * Map a task to a node in the strategic orchestration graph. + * @param sog Reference to the strategic orchestration graph object. + * @param task Reference to the task object. + * @return The node object. + */ +static struct vine_task_node *get_node_by_task(struct strategic_orchestration_graph *sog, struct vine_task *task) +{ + if (!sog || !task) { + return NULL; + } + + if (task->type == VINE_TASK_TYPE_STANDARD) { + /* standard tasks are mapped directly to a node */ + return itable_lookup(sog->task_id_to_node, task->task_id); + } else if (task->type == VINE_TASK_TYPE_RECOVERY) { + /* note that recovery tasks are not mapped to any node but we still need the original node for pruning, + * so we look up the outfile of the task, then map it back to get the original node */ + struct vine_mount *mount; + LIST_ITERATE(task->output_mounts, mount) + { + if (mount->file->original_producer_task_id > 0) { + return itable_lookup(sog->task_id_to_node, mount->file->original_producer_task_id); + } + } + } + + debug(D_ERROR, "task %d has no original producer task id", task->task_id); + + return NULL; +} + +/*************************************************************/ +/* Public APIs */ +/*************************************************************/ + +/** Tune the strategic orchestration graph. +*@param sog Reference to the strategic orchestration graph object. + *@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 sog_tune(struct strategic_orchestration_graph *sog, const char *name, const char *value) +{ + if (!sog || !name || !value) { + return -1; + } + + if (strcmp(name, "failure-injection-step-percent") == 0) { + sog->failure_injection_step_percent = atof(value); + + } else if (strcmp(name, "task-priority-mode") == 0) { + if (strcmp(value, "random") == 0) { + sog->task_priority_mode = TASK_PRIORITY_MODE_RANDOM; + } else if (strcmp(value, "depth-first") == 0) { + sog->task_priority_mode = TASK_PRIORITY_MODE_DEPTH_FIRST; + } else if (strcmp(value, "breadth-first") == 0) { + sog->task_priority_mode = TASK_PRIORITY_MODE_BREADTH_FIRST; + } else if (strcmp(value, "fifo") == 0) { + sog->task_priority_mode = TASK_PRIORITY_MODE_FIFO; + } else if (strcmp(value, "lifo") == 0) { + sog->task_priority_mode = TASK_PRIORITY_MODE_LIFO; + } else if (strcmp(value, "largest-input-first") == 0) { + sog->task_priority_mode = TASK_PRIORITY_MODE_LARGEST_INPUT_FIRST; + } else if (strcmp(value, "largest-storage-footprint-first") == 0) { + sog->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, "target-results-dir") == 0) { + if (sog->target_results_dir) { + free(sog->target_results_dir); + } + if (mkdir(value, 0777) != 0 && errno != EEXIST) { + debug(D_ERROR, "failed to mkdir %s (errno=%d)", value, errno); + return -1; + } + sog->target_results_dir = xxstrdup(value); + + } else if (strcmp(name, "proxy-function-name") == 0) { + if (sog->proxy_function_name) { + free(sog->proxy_function_name); + } + sog->proxy_function_name = xxstrdup(value); + + } else if (strcmp(name, "prune-depth") == 0) { + sog->prune_depth = atoi(value); + + } else { + debug(D_ERROR, "invalid parameter name: %s", name); + return -1; + + } + + return 0; +} + +void sog_set_proxy_library_name(struct strategic_orchestration_graph *sog, const char *proxy_library_name) +{ + if (!sog || !proxy_library_name) { + return; + } + + if (sog->proxy_library_name) { + free(sog->proxy_library_name); + } + + sog->proxy_library_name = xxstrdup(proxy_library_name); +} + +/** + * Get the proxy library name (Python-side), shared by all tasks. + * @param sog Reference to the strategic orchestration graph object. + * @return The library name. + */ +const char *sog_get_proxy_library_name(const struct strategic_orchestration_graph *sog) +{ + if (!sog) { + return NULL; + } + + return sog->proxy_library_name; +} + +/** + * Get the proxy function name (Python-side), shared by all tasks. + * @param sog Reference to the strategic orchestration graph object. + * @return The function name. + */ +const char *sog_get_proxy_function_name(const struct strategic_orchestration_graph *sog) +{ + if (!sog) { + return NULL; + } + + return sog->proxy_function_name; +} + +/** + * Get the heavy score of a node in the strategic orchestration graph. + * @param sog Reference to the strategic orchestration graph object. + * @param node_key Reference to the node key. + * @return The heavy score. + */ +double sog_get_node_heavy_score(const struct strategic_orchestration_graph *sog, const char *node_key) +{ + if (!sog) { + return -1; + } + + struct vine_task_node *node = hash_table_lookup(sog->nodes, node_key); + if (!node) { + return -1; + } + + return node->heavy_score; +} + +/** + * Get the local outfile source of a node in the strategic orchestration graph, only valid for local output files. + * The source of a local output file is the path on the local filesystem. + * @param sog Reference to the strategic orchestration graph object. + * @param node_key Reference to the node key. + * @return The local outfile source. + */ +const char *sog_get_node_local_outfile_source(const struct strategic_orchestration_graph *sog, const char *node_key) +{ + if (!sog || !node_key) { + return NULL; + } + + struct vine_task_node *node = hash_table_lookup(sog->nodes, node_key); + if (!node) { + debug(D_ERROR, "node %s not found", node_key); + exit(1); + } + + if (node->outfile_type != VINE_NODE_OUTFILE_TYPE_LOCAL) { + debug(D_ERROR, "node %s is not a local output file", node_key); + exit(1); + } + + return node->outfile->source; +} + +/** + * Compute the topology metrics of the strategic orchestration graph, including depth, height, upstream and downstream counts, + * heavy scores, and weakly connected components. + * @param sog Reference to the strategic orchestration graph object. + */ +void sog_compute_topology_metrics(struct strategic_orchestration_graph *sog) +{ + if (!sog) { + return; + } + + /* get nodes in topological order */ + struct list *topo_order = get_topological_order(sog); + if (!topo_order) { + return; + } + + char *node_key; + struct vine_task_node *node; + struct vine_task_node *parent_node; + struct vine_task_node *child_node; + + /* compute the depth of the node */ + 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; + } + } + } + + /* compute the height of the node */ + 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; + } + } + } + + /* compute the upstream and downstream counts for each node */ + struct hash_table *upstream_map = hash_table_create(0, 0); + struct hash_table *downstream_map = hash_table_create(0, 0); + HASH_TABLE_ITERATE(sog->nodes, node_key, node) + { + struct set *upstream = set_create(0); + struct set *downstream = set_create(0); + hash_table_insert(upstream_map, node_key, upstream); + hash_table_insert(downstream_map, node_key, downstream); + } + LIST_ITERATE(topo_order, node) + { + struct set *upstream = hash_table_lookup(upstream_map, node->node_key); + LIST_ITERATE(node->parents, parent_node) + { + struct set *parent_upstream = hash_table_lookup(upstream_map, parent_node->node_key); + set_union(upstream, parent_upstream); + set_insert(upstream, parent_node); + } + } + LIST_ITERATE_REVERSE(topo_order, node) + { + struct set *downstream = hash_table_lookup(downstream_map, node->node_key); + LIST_ITERATE(node->children, child_node) + { + struct set *child_downstream = hash_table_lookup(downstream_map, child_node->node_key); + set_union(downstream, child_downstream); + set_insert(downstream, child_node); + } + } + LIST_ITERATE(topo_order, node) + { + node->upstream_subgraph_size = set_size(hash_table_lookup(upstream_map, node->node_key)); + node->downstream_subgraph_size = set_size(hash_table_lookup(downstream_map, node->node_key)); + node->fan_in = list_size(node->parents); + node->fan_out = list_size(node->children); + set_delete(hash_table_lookup(upstream_map, node->node_key)); + set_delete(hash_table_lookup(downstream_map, node->node_key)); + } + hash_table_delete(upstream_map); + hash_table_delete(downstream_map); + + /* compute the heavy score for each node */ + LIST_ITERATE(topo_order, node) + { + node->heavy_score = compute_node_heavy_score(node); + } + + /* extract weakly connected components */ + struct list *weakly_connected_components = extract_weakly_connected_components(sog); + 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_delete(topo_order); + + return; +} + +/** + * Create a new node and track it in the strategic orchestration graph. + * @param sog Reference to the strategic orchestration graph object. + * @param node_key Reference to the node key. + * @return A new node object. + */ +void sog_add_node(struct strategic_orchestration_graph *sog, const char *node_key) +{ + if (!sog || !node_key) { + return; + } + + /* if the node already exists, skip creating a new one */ + struct vine_task_node *node = hash_table_lookup(sog->nodes, node_key); + if (!node) { + node = vine_task_node_create(sog->manager, + node_key, + sog->proxy_library_name, + sog->proxy_function_name, + sog->target_results_dir, + sog->prune_depth); + + if (!node) { + debug(D_ERROR, "failed to create node %s", node_key); + sog_delete(sog); + exit(1); + } + + hash_table_insert(sog->nodes, node_key, node); + } +} + +/** + * Create a new strategic orchestration graph object and bind a manager to it. + * @param q Reference to the manager object. + * @return A new strategic orchestration graph object. + */ +struct strategic_orchestration_graph *sog_create(struct vine_manager *q) +{ + struct strategic_orchestration_graph *sog = xxmalloc(sizeof(struct strategic_orchestration_graph)); + + sog->manager = q; + + sog->nodes = hash_table_create(0, 0); + sog->task_id_to_node = itable_create(0); + sog->outfile_cachename_to_node = hash_table_create(0, 0); + + sog->target_results_dir = xxstrdup(sog->manager->runtime_directory); // default to current working directory + sog->proxy_library_name = NULL; + sog->proxy_function_name = NULL; + + sog->prune_depth = 1; + + sog->task_priority_mode = TASK_PRIORITY_MODE_LARGEST_INPUT_FIRST; + sog->failure_injection_step_percent = -1.0; + + /* enable debug system for C code since it uses a separate debug system instance + * from the Python bindings. Use the same function that the manager uses. */ + char *debug_tmp = string_format("%s/vine-logs/debug", sog->manager->runtime_directory); + vine_enable_debug_log(debug_tmp); + free(debug_tmp); + + return sog; +} + +/** + * Add a dependency between two nodes in the strategic orchestration graph. Note that the input-output file relationship + * is not handled here, because their file names may have not been determined yet. + * @param sog Reference to the strategic orchestration graph object. + * @param parent_key Reference to the parent node key. + * @param child_key Reference to the child node key. + */ +void sog_add_dependency(struct strategic_orchestration_graph *sog, const char *parent_key, const char *child_key) +{ + if (!sog || !parent_key || !child_key) { + return; + } + + struct vine_task_node *parent_node = hash_table_lookup(sog->nodes, parent_key); + struct vine_task_node *child_node = hash_table_lookup(sog->nodes, child_key); + if (!parent_node) { + debug(D_ERROR, "parent node %s not found", parent_key); + char *node_key = NULL; + struct vine_task_node *node; + printf("parent_keys:\n"); + HASH_TABLE_ITERATE(sog->nodes, node_key, node) + { + printf(" %s\n", node->node_key); + } + exit(1); + } + if (!child_node) { + debug(D_ERROR, "child node %s not found", child_key); + exit(1); + } + + list_push_tail(child_node->parents, parent_node); + list_push_tail(parent_node->children, child_node); + + return; +} + +/** + * Set the outfile of a node in the strategic orchestration graph. + * This involves declaring the output file and adding it to the task. + * @param sog Reference to the strategic orchestration graph object. + * @param node_key Reference to the node key. + * @param outfile_type Reference to the outfile type. + * @param outfile_remote_name Reference to the outfile remote name that the task will generate remotely. + */ +void sog_set_node_outfile(struct strategic_orchestration_graph *sog, const char *node_key, vine_task_node_outfile_type_t outfile_type, const char *outfile_remote_name) +{ + if (!sog || !node_key || !outfile_remote_name) { + return; + } + + struct vine_task_node *node = hash_table_lookup(sog->nodes, node_key); + if (!node) { + return; + } + + vine_task_node_set_outfile(node, outfile_type, outfile_remote_name); + + return; +} + +/** + * Execute the strategic orchestration graph. This must be called after all nodes and dependencies are added and the topology metrics are computed. + * @param sog Reference to the strategic orchestration graph object. + */ +void sog_execute(struct strategic_orchestration_graph *sog) +{ + if (!sog) { + return; + } + + signal(SIGINT, handle_sigint); + + debug(D_VINE, "start executing strategic orchestration graph"); + + /* print the info of all nodes */ + char *node_key; + struct vine_task_node *node; + HASH_TABLE_ITERATE(sog->nodes, node_key, node) + { + vine_task_node_print_info(node); + } + + /* enable return recovery tasks */ + vine_enable_return_recovery_tasks(sog->manager); + + /* create mapping from task_id and outfile cached_name to node */ + HASH_TABLE_ITERATE(sog->nodes, node_key, node) + { + if (node->outfile) { + hash_table_insert(sog->outfile_cachename_to_node, node->outfile->cached_name, node); + } + } + + /* add the parents' outfiles as inputs to the task */ + struct list *topo_order = get_topological_order(sog); + LIST_ITERATE(topo_order, node) + { + struct vine_task_node *parent_node; + LIST_ITERATE(node->parents, parent_node) + { + if (parent_node->outfile) { + vine_task_add_input(node->task, parent_node->outfile, parent_node->outfile_remote_name, VINE_TRANSFER_ALWAYS); + } + } + } + + /* initialize pending_parents for all nodes */ + HASH_TABLE_ITERATE(sog->nodes, node_key, node) + { + struct vine_task_node *parent_node; + LIST_ITERATE(node->parents, parent_node) + { + if (node->pending_parents) { + /* Use parent_node->node_key to ensure pointer consistency */ + set_insert(node->pending_parents, parent_node); + } + } + } + + /* enqueue those without dependencies */ + HASH_TABLE_ITERATE(sog->nodes, node_key, node) + { + if (!node->pending_parents || set_size(node->pending_parents) == 0) { + submit_node_task(sog, node); + } + } + + /* calculate steps to inject failure */ + double next_failure_threshold = -1.0; + if (sog->failure_injection_step_percent > 0) { + next_failure_threshold = sog->failure_injection_step_percent / 100.0; + } + + struct ProgressBar *pbar = progress_bar_init("Executing Tasks"); + struct ProgressBarPart *regular_tasks_part = progress_bar_create_part("Regular", hash_table_size(sog->nodes)); + struct ProgressBarPart *recovery_tasks_part = progress_bar_create_part("Recovery", 0); + progress_bar_bind_part(pbar, regular_tasks_part); + progress_bar_bind_part(pbar, recovery_tasks_part); + + int wait_timeout = 2; + + while (regular_tasks_part->current < regular_tasks_part->total) { + if (interrupted) { + break; + } + + struct vine_task *task = vine_wait(sog->manager, wait_timeout); + progress_bar_set_part_total(pbar, recovery_tasks_part, sog->manager->num_submitted_recovery_tasks); + if (task) { + /* retrieve all possible tasks */ + wait_timeout = 0; + + /* get the original node by task id */ + struct vine_task_node *node = get_node_by_task(sog, 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); + } + + /* in case of failure, resubmit this task */ + if (node->task->result != VINE_RESULT_SUCCESS || node->task->exit_code != 0) { + if (node->retry_attempts_left <= 0) { + debug(D_ERROR, "Task %d failed (result=%d, exit=%d). Node %s has no retries left. Aborting.", task->task_id, node->task->result, node->task->exit_code, node->node_key); + sog_delete(sog); + exit(1); + } + node->retry_attempts_left--; + debug(D_VINE | D_NOTICE, "Task %d failed (result=%d, exit=%d). Retrying node %s (remaining=%d)...", task->task_id, node->task->result, node->task->exit_code, node->node_key, node->retry_attempts_left); + vine_task_reset(node->task); + submit_node_task(sog, node); + continue; + } + + /* if the outfile is set to save on the sharedfs, stat to get the size of the file */ + switch (node->outfile_type) { + case VINE_NODE_OUTFILE_TYPE_SHARED_FILE_SYSTEM: { + struct stat info; + int result = stat(node->outfile_remote_name, &info); + if (result < 0) { + if (node->retry_attempts_left <= 0) { + debug(D_ERROR, "Task %d succeeded but missing sharedfs output %s; no retries left for node %s. Aborting.", task->task_id, node->outfile_remote_name, node->node_key); + sog_delete(sog); + exit(1); + } + node->retry_attempts_left--; + debug(D_VINE | D_NOTICE, "Task %d succeeded but missing sharedfs output %s; retrying node %s (remaining=%d)...", task->task_id, node->outfile_remote_name, node->node_key, node->retry_attempts_left); + vine_task_reset(node->task); + submit_node_task(sog, node); + continue; + } + node->outfile_size_bytes = info.st_size; + break; + } + case VINE_NODE_OUTFILE_TYPE_LOCAL: + case VINE_NODE_OUTFILE_TYPE_TEMP: + node->outfile_size_bytes = node->outfile->size; + break; + } + debug(D_VINE, "Node %s completed with outfile %s size: %zu bytes", node->node_key, node->outfile_remote_name, node->outfile_size_bytes); + + /* mark the node as completed */ + node->completed = 1; + + /* prune nodes on task completion */ + vine_task_node_prune_ancestors(node); + + /* skip recovery tasks */ + if (task->type == VINE_TASK_TYPE_RECOVERY) { + progress_bar_update_part(pbar, recovery_tasks_part, 1); + continue; + } + + /* set the start time to the submit time of the first regular task */ + if (regular_tasks_part->current == 0) { + progress_bar_set_start_time(pbar, task->time_when_commit_start); + } + + /* update critical time */ + vine_task_node_update_critical_time(node, task->time_workers_execute_last); + + /* mark this regular task as completed */ + progress_bar_update_part(pbar, regular_tasks_part, 1); + + /* inject failure */ + if (sog->failure_injection_step_percent > 0) { + double progress = (double)regular_tasks_part->current / (double)regular_tasks_part->total; + if (progress >= next_failure_threshold && evict_random_worker(sog->manager)) { + debug(D_VINE, "evicted a worker at %.2f%% (threshold %.2f%%)", progress * 100, next_failure_threshold * 100); + next_failure_threshold += sog->failure_injection_step_percent / 100.0; + } + } + + /* enqueue the output file for replication */ + switch (node->outfile_type) { + case VINE_NODE_OUTFILE_TYPE_TEMP: + vine_task_node_replicate_outfile(node); + break; + case VINE_NODE_OUTFILE_TYPE_LOCAL: + case VINE_NODE_OUTFILE_TYPE_SHARED_FILE_SYSTEM: + break; + } + + /* submit children nodes with dependencies all resolved */ + submit_unblocked_children(sog, node); + } else { + wait_timeout = 2; + progress_bar_update_part(pbar, recovery_tasks_part, 0); // refresh the time and total for recovery tasks + } + } + + progress_bar_finish(pbar); + progress_bar_delete(pbar); + + double total_time_spent_on_unlink_local_files = 0; + double total_time_spent_on_prune_ancestors_of_temp_node = 0; + double total_time_spent_on_prune_ancestors_of_persisted_node = 0; + HASH_TABLE_ITERATE(sog->nodes, node_key, node) + { + total_time_spent_on_unlink_local_files += node->time_spent_on_unlink_local_files; + total_time_spent_on_prune_ancestors_of_temp_node += node->time_spent_on_prune_ancestors_of_temp_node; + total_time_spent_on_prune_ancestors_of_persisted_node += node->time_spent_on_prune_ancestors_of_persisted_node; + } + total_time_spent_on_unlink_local_files /= 1e6; + total_time_spent_on_prune_ancestors_of_temp_node /= 1e6; + total_time_spent_on_prune_ancestors_of_persisted_node /= 1e6; + + debug(D_VINE, "total time spent on prune ancestors of temp node: %.6f seconds\n", total_time_spent_on_prune_ancestors_of_temp_node); + debug(D_VINE, "total time spent on prune ancestors of persisted node: %.6f seconds\n", total_time_spent_on_prune_ancestors_of_persisted_node); + debug(D_VINE, "total time spent on unlink local files: %.6f seconds\n", total_time_spent_on_unlink_local_files); + + return; +} + +/** + * Delete a strategic orchestration graph object. + * @param sog Reference to the strategic orchestration graph object. + */ +void sog_delete(struct strategic_orchestration_graph *sog) +{ + if (!sog) { + return; + } + + char *node_key; + struct vine_task_node *node; + HASH_TABLE_ITERATE(sog->nodes, node_key, node) + { + if (node->infile) { + vine_prune_file(sog->manager, node->infile); + hash_table_remove(sog->manager->file_table, node->infile->cached_name); + } + if (node->outfile) { + vine_prune_file(sog->manager, node->outfile); + hash_table_remove(sog->outfile_cachename_to_node, node->outfile->cached_name); + hash_table_remove(sog->manager->file_table, node->outfile->cached_name); + } + vine_task_node_delete(node); + } + + free(sog->proxy_library_name); + free(sog->proxy_function_name); + + hash_table_delete(sog->nodes); + itable_delete(sog->task_id_to_node); + hash_table_delete(sog->outfile_cachename_to_node); + free(sog); +} diff --git a/taskvine/src/graph/dagvine/strategic_orchestration_graph.h b/taskvine/src/graph/dagvine/strategic_orchestration_graph.h new file mode 100644 index 0000000000..2cbbcbcce1 --- /dev/null +++ b/taskvine/src/graph/dagvine/strategic_orchestration_graph.h @@ -0,0 +1,132 @@ +#ifndef STRATEGIC_ORCHESTRATION_GRAPH_H +#define STRATEGIC_ORCHESTRATION_GRAPH_H + +#include "vine_task.h" +#include "hash_table.h" +#include "list.h" +#include "vine_manager.h" +#include "set.h" +#include "vine_task_node.h" +#include "taskvine.h" + + +/** Select priority algorithm for strategic orchestration graph task scheduling. */ +typedef enum { + TASK_PRIORITY_MODE_RANDOM = 0, /**< Assign random priority to tasks */ + TASK_PRIORITY_MODE_DEPTH_FIRST, /**< Prioritize deeper tasks first */ + TASK_PRIORITY_MODE_BREADTH_FIRST, /**< Prioritize shallower tasks first */ + TASK_PRIORITY_MODE_FIFO, /**< First in, first out priority */ + TASK_PRIORITY_MODE_LIFO, /**< Last in, first out priority */ + TASK_PRIORITY_MODE_LARGEST_INPUT_FIRST, /**< Prioritize tasks with larger inputs first */ + TASK_PRIORITY_MODE_LARGEST_STORAGE_FOOTPRINT_FIRST /**< Prioritize tasks with larger storage footprint first */ +} task_priority_mode_t; + + +/** The strategic orchestration graph object (logical scheduling layer). */ +struct strategic_orchestration_graph { + struct vine_manager *manager; + struct hash_table *nodes; + struct itable *task_id_to_node; + struct hash_table *outfile_cachename_to_node; + + /* Results of target keys will be stored in this directory. + * This dir path can not necessarily be a shared file system directory, + * output files will be retrieved through the network instead, + * as long as the manager can access it. */ + char *target_results_dir; + + char *proxy_library_name; // Python-side proxy library name (shared by all tasks) + char *proxy_function_name; // Python-side proxy function name (shared by all tasks) + + int prune_depth; + + task_priority_mode_t task_priority_mode; // priority mode for task graph task scheduling + double failure_injection_step_percent; // 0 - 100, the percentage of steps to inject failure +}; + + +/* Public APIs for operating the strategic orchestration graph */ + +/** Create a strategic orchestration graph object and return it. +@param q Reference to the current manager object. +@return A new strategic orchestration graph object. +*/ +struct strategic_orchestration_graph *sog_create(struct vine_manager *q); + +/** Create a new node in the strategic orchestration graph. +@param sog Reference to the strategic orchestration graph object. +@param node_key Reference to the node key. +*/ +void sog_add_node(struct strategic_orchestration_graph *sog, const char *node_key); + +/** Add a dependency between two nodes in the strategic orchestration graph. +@param sog Reference to the strategic orchestration graph object. +@param parent_key Reference to the parent node key. +@param child_key Reference to the child node key. +*/ +void sog_add_dependency(struct strategic_orchestration_graph *sog, const char *parent_key, const char *child_key); + +/** Finalize the metrics of the strategic orchestration graph. +@param sog Reference to the strategic orchestration graph object. +*/ +void sog_compute_topology_metrics(struct strategic_orchestration_graph *sog); + +/** Get the heavy score of a node in the strategic orchestration graph. +@param sog Reference to the strategic orchestration graph object. +@param node_key Reference to the node key. +@return The heavy score. +*/ +double sog_get_node_heavy_score(const struct strategic_orchestration_graph *sog, const char *node_key); + +/** Set the type of the node-output file. +@param sog Reference to the strategic orchestration graph object. +@param node_key Reference to the node key. +@param outfile_type Reference to the output file type. +@param outfile_remote_name Reference to the output file remote name. +*/ +void sog_set_node_outfile(struct strategic_orchestration_graph *sog, const char *node_key, vine_task_node_outfile_type_t outfile_type, const char *outfile_remote_name); + +/** Execute the task graph. +@param sog Reference to the strategic orchestration graph object. +*/ +void sog_execute(struct strategic_orchestration_graph *sog); + +/** Get the local outfile source of a node in the strategic orchestration graph. +@param sog Reference to the strategic orchestration graph object. +@param node_key Reference to the node key. +@return The local outfile source. +*/ +const char *sog_get_node_local_outfile_source(const struct strategic_orchestration_graph *sog, const char *node_key); + +/** Delete a strategic orchestration graph object. +@param sog Reference to the strategic orchestration graph object. +*/ +void sog_delete(struct strategic_orchestration_graph *sog); + +/** Set the Python-side proxy library name of the strategic orchestration graph. +@param sog Reference to the strategic orchestration graph object. +@param proxy_library_name Reference to the Python-side proxy library name. +*/ +void sog_set_proxy_library_name(struct strategic_orchestration_graph *sog, const char *proxy_library_name); + +/** Get the library name of the strategic orchestration graph. +@param sog Reference to the strategic orchestration graph object. +@return The library name. +*/ +const char *sog_get_proxy_library_name(const struct strategic_orchestration_graph *sog); + +/** Get the function name of the strategic orchestration graph. +@param sog Reference to the strategic orchestration graph object. +@return The function name. +*/ +const char *sog_get_proxy_function_name(const struct strategic_orchestration_graph *sog); + +/** Tune the strategic orchestration graph. +@param sog Reference to the strategic orchestration graph object. +@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 sog_tune(struct strategic_orchestration_graph *sog, const char *name, const char *value); + +#endif // STRATEGIC_ORCHESTRATION_GRAPH_H diff --git a/taskvine/src/graph/dagvine/strategic_orchestration_graph.i b/taskvine/src/graph/dagvine/strategic_orchestration_graph.i new file mode 100644 index 0000000000..2ac5e9a3f4 --- /dev/null +++ b/taskvine/src/graph/dagvine/strategic_orchestration_graph.i @@ -0,0 +1,21 @@ +/* SWIG interface for local dagvine graph bindings */ +%module cdagvine + +%{ +#include "int_sizes.h" +#include "strategic_orchestration_graph.h" +#include "vine_task_node.h" /* expose outfile type enum to SWIG */ +%} + +%include "stdint.i" +%include "int_sizes.h" + +/* Import existing SWIG interface for type information (do not wrap again) */ +%import "../../bindings/python3/taskvine.i" + +/* Expose only the dagvine task graph APIs */ +%ignore vine_task_node_checkpoint_outfile; /* avoid exporting unimplemented/optional symbol */ +%include "vine_task_node.h" /* export vine_task_node_outfile_type_t values */ +%include "strategic_orchestration_graph.h" + + From 0a0dd4094aa823523cb9295903d275b7dbdb4949 Mon Sep 17 00:00:00 2001 From: Jin Zhou Date: Sun, 19 Oct 2025 03:55:44 -0400 Subject: [PATCH 046/113] milestone --- taskvine/src/bindings/python3/Makefile | 4 +- taskvine/src/graph/dagvine/Makefile | 2 +- taskvine/src/graph/dagvine/graph_executor.py | 114 ++-- taskvine/src/graph/dagvine/library.py | 74 -- .../graph/dagvine/runtime_execution_graph.py | 159 +---- .../dagvine/strategic_orchestration_graph.c | 226 ++++--- .../dagvine/strategic_orchestration_graph.h | 35 +- .../dagvine/strategic_orchestration_graph.i | 6 +- taskvine/src/graph/dagvine/vine_task_node.c | 640 ------------------ taskvine/src/graph/dagvine/vine_task_node.h | 77 --- taskvine/src/manager/vine_manager.c | 1 + 11 files changed, 229 insertions(+), 1109 deletions(-) delete mode 100644 taskvine/src/graph/dagvine/library.py delete mode 100644 taskvine/src/graph/dagvine/vine_task_node.c delete mode 100644 taskvine/src/graph/dagvine/vine_task_node.h diff --git a/taskvine/src/bindings/python3/Makefile b/taskvine/src/bindings/python3/Makefile index c81a88efb1..3289a4da46 100644 --- a/taskvine/src/bindings/python3/Makefile +++ b/taskvine/src/bindings/python3/Makefile @@ -41,7 +41,9 @@ install: all mkdir -p $(CCTOOLS_PYTHON3_PATH)/ndcctools/taskvine/${GRAPH_EXECUTOR_MODULE_NAME} cp $(CCTOOLS_HOME)/taskvine/src/graph/${GRAPH_EXECUTOR_MODULE_NAME}/runtime_execution_graph.py $(CCTOOLS_PYTHON3_PATH)/ndcctools/taskvine/${GRAPH_EXECUTOR_MODULE_NAME}/runtime_execution_graph.py cp $(CCTOOLS_HOME)/taskvine/src/graph/${GRAPH_EXECUTOR_MODULE_NAME}/graph_executor.py $(CCTOOLS_PYTHON3_PATH)/ndcctools/taskvine/${GRAPH_EXECUTOR_MODULE_NAME}/graph_executor.py - cp $(CCTOOLS_HOME)/taskvine/src/graph/${GRAPH_EXECUTOR_MODULE_NAME}/library.py $(CCTOOLS_PYTHON3_PATH)/ndcctools/taskvine/${GRAPH_EXECUTOR_MODULE_NAME}/library.py + cp $(CCTOOLS_HOME)/taskvine/src/graph/${GRAPH_EXECUTOR_MODULE_NAME}/proxy_library.py $(CCTOOLS_PYTHON3_PATH)/ndcctools/taskvine/${GRAPH_EXECUTOR_MODULE_NAME}/proxy_library.py + cp $(CCTOOLS_HOME)/taskvine/src/graph/${GRAPH_EXECUTOR_MODULE_NAME}/proxy_functions.py $(CCTOOLS_PYTHON3_PATH)/ndcctools/taskvine/${GRAPH_EXECUTOR_MODULE_NAME}/proxy_functions.py + cp $(CCTOOLS_HOME)/taskvine/src/graph/${GRAPH_EXECUTOR_MODULE_NAME}/strategic_orchestration_graph.py $(CCTOOLS_PYTHON3_PATH)/ndcctools/taskvine/${GRAPH_EXECUTOR_MODULE_NAME}/strategic_orchestration_graph.py cp $(CCTOOLS_HOME)/taskvine/src/graph/${GRAPH_EXECUTOR_MODULE_NAME}/params.py $(CCTOOLS_PYTHON3_PATH)/ndcctools/taskvine/${GRAPH_EXECUTOR_MODULE_NAME}/params.py cp $(CCTOOLS_HOME)/taskvine/src/graph/${GRAPH_EXECUTOR_MODULE_NAME}/utils.py $(CCTOOLS_PYTHON3_PATH)/ndcctools/taskvine/${GRAPH_EXECUTOR_MODULE_NAME}/utils.py @echo "# ${GRAPH_EXECUTOR_MODULE_NAME} package" > $(CCTOOLS_PYTHON3_PATH)/ndcctools/taskvine/${GRAPH_EXECUTOR_MODULE_NAME}/__init__.py diff --git a/taskvine/src/graph/dagvine/Makefile b/taskvine/src/graph/dagvine/Makefile index 4286e36cad..c2e3a38236 100644 --- a/taskvine/src/graph/dagvine/Makefile +++ b/taskvine/src/graph/dagvine/Makefile @@ -6,7 +6,7 @@ LOCAL_CCFLAGS+=-I ${CCTOOLS_HOME}/taskvine/src/manager -SOURCES = vine_task_node.c strategic_orchestration_graph.c +SOURCES = strategic_orchestration_node.c strategic_orchestration_graph.c OBJECTS = $(SOURCES:%.c=%.o) SWIG_I = strategic_orchestration_graph.i diff --git a/taskvine/src/graph/dagvine/graph_executor.py b/taskvine/src/graph/dagvine/graph_executor.py index 0554074c4f..163992ba68 100644 --- a/taskvine/src/graph/dagvine/graph_executor.py +++ b/taskvine/src/graph/dagvine/graph_executor.py @@ -1,11 +1,11 @@ from ndcctools.taskvine import cvine -from ndcctools.taskvine.dagvine import cdagvine from ndcctools.taskvine.manager import Manager -from ndcctools.taskvine.utils import delete_all_files, get_c_constant +from ndcctools.taskvine.utils import delete_all_files -from ndcctools.taskvine.dagvine.library import Library -from ndcctools.taskvine.dagvine.runtime_execution_graph import RuntimeExecutionGraph +from ndcctools.taskvine.dagvine.proxy_library import ProxyLibrary +from ndcctools.taskvine.dagvine.runtime_execution_graph import RuntimeExecutionGraph, GraphKeyResult +from ndcctools.taskvine.dagvine.strategic_orchestration_graph import StrategicOrchestrationGraph import cloudpickle import os @@ -76,9 +76,9 @@ def __init__(self): self.sog_tuning_params = { "failure-injection-step-percent": -1, "priority-mode": "largest-input-first", - "proxy-function-name": "compute_single_key", "prune-depth": 1, "target-results-dir": "./target_results", + "checkpoint-fraction": 0, } self.other_params = { "schedule": "worst", @@ -163,42 +163,9 @@ def tune_vine_manager(self): def tune_sog(self, sog): for k, v in self.params.sog_tuning_params.items(): print(f"Tuning {k} to {v}") - cdagvine.sog_tune(sog, k, str(v)) - - def assign_outfile_types(self, target_keys, reg, sog): - assert reg is not None, "Runtime execution graph must be built first" - assert sog is not None, "Strategic orchestration graph must be built first" - - # get heavy score from C side - heavy_scores = {} - for k in reg.task_dict.keys(): - heavy_scores[k] = cdagvine.sog_get_node_heavy_score(sog, reg.vine_key_of[k]) - - # sort keys by heavy score descending - sorted_keys = sorted(heavy_scores, key=lambda k: heavy_scores[k], reverse=True) - - # determine how many go to shared FS - sharedfs_count = round(self.param("outfile-type")["shared-file-system"] * len(sorted_keys)) - - # assign outfile types - for i, k in enumerate(sorted_keys): - if k in target_keys: - choice = "local" - elif i < sharedfs_count: - choice = "shared-file-system" - else: - choice = "temp" - - reg.set_outfile_type_of(k, choice) - outfile_type_enum = get_c_constant(f"NODE_OUTFILE_TYPE_{choice.upper().replace('-', '_')}") - cdagvine.sog_set_node_outfile( - sog, - reg.vine_key_of[k], - outfile_type_enum, - reg.outfile_remote_name[k] - ) - - def build_python_graph(self): + sog.tune(k, str(v)) + + def build_reg(self): reg = RuntimeExecutionGraph( self.task_dict, shared_file_system_dir=self.param("shared-file-system-dir"), @@ -208,10 +175,12 @@ def build_python_graph(self): return reg - def build_sog(self, reg): + def build_sog(self, reg, target_keys): assert reg is not None, "Python graph must be built before building the C graph" - sog = cdagvine.sog_create(self._taskvine) + sog = StrategicOrchestrationGraph(self._taskvine) + + sog.set_proxy_function_name("compute_single_key") # C side vine task graph must be tuned before adding nodes and dependencies self.tune_vine_manager() @@ -219,59 +188,66 @@ def build_sog(self, reg): topo_order = reg.get_topological_order() for k in topo_order: - cdagvine.sog_add_node( - sog, - reg.vine_key_of[k], - ) - for pk in reg.parents_of[k]: - cdagvine.sog_add_dependency(sog, reg.vine_key_of[pk], reg.vine_key_of[k]) + sog.add_node(reg.sog_node_key_of[k], int(k in target_keys)) + for pk in reg.parents_of[k]: #!!!!!!!!!!!!!!!!!!!!!!! TODO: fix sog-reg key mapping + sog.add_dependency(reg.sog_node_key_of[pk], reg.sog_node_key_of[k]) - cdagvine.sog_compute_topology_metrics(sog) + sog.compute_topology_metrics() return sog def build_graphs(self, target_keys): # build Python DAG (logical topology) - reg = self.build_python_graph() - + reg = self.build_reg() # build C DAG (physical topology) - sog = self.build_sog(reg) + sog = self.build_sog(reg, target_keys) - # assign outfile types - self.assign_outfile_types(target_keys, reg, sog) + # set outfile remote names in reg from sog, note that these names are automatically generated + # with regard to the checkpointing strategy and the shared file system directory + for sog_node_key in reg.sog_node_key_of.values(): + outfile_remote_name = sog.get_node_outfile_remote_name(sog_node_key) + reg.set_outfile_remote_name_of(reg.reg_node_key_of[sog_node_key], outfile_remote_name) return reg, sog - def run(self, collection_dict, target_keys=None, params={}, hoisting_modules=[], env_files={}): + def create_proxy_library(self, reg, sog, hoisting_modules, env_files): + proxy_library = ProxyLibrary(self) + proxy_library.add_hoisting_modules(hoisting_modules) + proxy_library.add_env_files(env_files) + proxy_library.set_context_loader(RuntimeExecutionGraph.context_loader_func, context_loader_args=[cloudpickle.dumps(reg)]) + proxy_library.set_libcores(self.param("libcores")) + proxy_library.set_name(sog.get_proxy_library_name()) + proxy_library.install() + + return proxy_library + + def run(self, collection_dict, target_keys=[], params={}, hoisting_modules=[], env_files={}): # first update the params so that they can be used for the following construction self.update_params(params) self.task_dict = ensure_task_dict(collection_dict) - - # build graphs in both Python and C sides + + # build graphs from both sides reg, sog = self.build_graphs(target_keys) - # create and install the library template on the manager - library = Library(self, self.param("libcores")) - library.add_hoisting_modules(hoisting_modules) - library.add_env_files(env_files) - library.set_context_loader(RuntimeExecutionGraph.context_loader_func, context_loader_args=[cloudpickle.dumps(reg)]) - library.install() - cdagvine.sog_set_proxy_library_name(sog, library.get_name()) + # create and install the proxy library on the manager + proxy_library = self.create_proxy_library(reg, sog, hoisting_modules, env_files) # execute the graph on the C side print(f"Executing task graph, logs will be written to {self.runtime_directory}") - cdagvine.sog_execute(sog) + sog.execute() # clean up the library instances and template on the manager - library.uninstall() + proxy_library.uninstall() + # delete the C graph immediately after execution, so that the lifetime of this object is limited to the execution - cdagvine.sog_delete(sog) + sog.delete() # load results of target keys results = {} for k in target_keys: - results[k] = reg.load_result_of_key(k, target_results_dir=self.param("target-results-dir")) + outfile_path = os.path.join(self.param("target-results-dir"), reg.outfile_remote_name[k]) + results[k] = GraphKeyResult.load_from_path(outfile_path) return results def _on_sigint(self, signum, frame): diff --git a/taskvine/src/graph/dagvine/library.py b/taskvine/src/graph/dagvine/library.py deleted file mode 100644 index f5158d6656..0000000000 --- a/taskvine/src/graph/dagvine/library.py +++ /dev/null @@ -1,74 +0,0 @@ -import os, uuid, cloudpickle, types, time, random, hashlib, collections -from ndcctools.taskvine.dagvine.runtime_execution_graph import ( - GraphKeyResult, RuntimeExecutionGraph, compute_dts_key, compute_sexpr_key, - compute_single_key, hash_name, hashable -) -from ndcctools.taskvine.utils import load_variable_from_library - - -class Library: - def __init__(self, py_manager, libcores): - self.py_manager = py_manager - self.libname = f"proxy-library-{str(uuid.uuid4())}" - - self.libcores = libcores - assert self.libcores is not None, "libcores must be provided" - - 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, GraphKeyResult, RuntimeExecutionGraph, uuid, hashlib, random, types, collections, time, - load_variable_from_library, compute_dts_key, compute_sexpr_key, compute_single_key, hash_name, hashable - ] - - # 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 get_name(self): - return self.libname - - 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): - self.libtask = self.py_manager.create_library_from_functions( - self.libname, - compute_single_key, - 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.libname) diff --git a/taskvine/src/graph/dagvine/runtime_execution_graph.py b/taskvine/src/graph/dagvine/runtime_execution_graph.py index 24e246f152..4e4f7c81b5 100644 --- a/taskvine/src/graph/dagvine/runtime_execution_graph.py +++ b/taskvine/src/graph/dagvine/runtime_execution_graph.py @@ -2,8 +2,6 @@ # This software is distributed under the GNU General Public License. # See the file COPYING for details. -from ndcctools.taskvine.utils import load_variable_from_library - import os import hashlib import time @@ -90,6 +88,16 @@ 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, GraphKeyResult), "Loaded object is not of type GraphKeyResult" + return result_obj.result + except FileNotFoundError: + raise FileNotFoundError(f"Output file not found at {path}") + class RuntimeExecutionGraph: """ @@ -117,22 +125,16 @@ def __init__(self, task_dict, self.parents_of, self.children_of = self._build_dependencies(self.task_dict) self.depth_of = self._calculate_depths() - self.vine_key_of = {k: hash_name(k) for k in task_dict.keys()} - self.key_of_vine_key = {hash_name(k): k for k in task_dict.keys()} + self.sog_node_key_of = {k: hash_name(k) for k in task_dict.keys()} + self.reg_node_key_of = {hash_name(k): k for k in task_dict.keys()} - self.outfile_remote_name = {key: f"{uuid.uuid4()}.pkl" for key in self.task_dict.keys()} - self.outfile_type = {key: None for key in self.task_dict.keys()} + # will be set from sog + self.outfile_remote_name = {key: None for key in self.task_dict.keys()} # testing params self.extra_task_output_size_mb = self._calculate_extra_size_mb_of(extra_task_output_size_mb) self.extra_sleep_time_of = self._calculate_extra_sleep_time_of(extra_task_sleep_time) - def set_outfile_type_of(self, k, outfile_type_str): - assert outfile_type_str in ["local", "shared-file-system", "temp"] - self.outfile_type[k] = outfile_type_str - if outfile_type_str == "shared-file-system": - self.outfile_remote_name[k] = os.path.join(self.shared_file_system_dir, self.outfile_remote_name[k]) - def _calculate_extra_size_mb_of(self, extra_task_output_size_mb): assert isinstance(extra_task_output_size_mb, list) and len(extra_task_output_size_mb) == 3 mode, low, high = extra_task_output_size_mb @@ -224,24 +226,12 @@ def save_result_of_key(self, key, result): result_obj = GraphKeyResult(result, extra_size_mb=self.extra_task_output_size_mb[key]) cloudpickle.dump(result_obj, f) - def load_result_of_key(self, key, target_results_dir=None): - if self.outfile_type[key] == "local": - # nodes with local outfile type will have their results stored in the target results directory - # when the execution is done, users can call this function to load the results from the target results directory - # note that data is transferred through the link in sequence, not in parallel through the shared file system - outfile_path = os.path.join(target_results_dir, self.outfile_remote_name[key]) - else: - # workers user this branch to load results from either local or shared file system - # if a node-local output, then data is stored in the task sandbox and the remote name is just the filename - # if a shared file system output, then remote name is the full path to the file - outfile_path = self.outfile_remote_name[key] - try: - with open(outfile_path, "rb") as f: - result_obj = cloudpickle.load(f) - assert isinstance(result_obj, GraphKeyResult), "Loaded object is not of type GraphKeyResult" - return result_obj.result - except FileNotFoundError: - raise FileNotFoundError(f"Output file for key {key} not found at {outfile_path}") + def load_result_of_key(self, key): + # workers user this function to load results from either local or shared file system + # if a node-local output, then data is stored in the task sandbox and the remote name is just the filename + # if a shared file system output, then remote name is the full path to the file + outfile_path = self.outfile_remote_name[key] + return GraphKeyResult.load_from_path(outfile_path) def get_topological_order(self): in_degree = {key: len(self.parents_of[key]) for key in self.task_dict.keys()} @@ -264,12 +254,6 @@ def get_topological_order(self): return topo_order - def __del__(self): - if hasattr(self, 'outfile_remote_name') and self.outfile_remote_name: - for k in self.outfile_remote_name.keys(): - if self.outfile_type.get(k) == "shared-file-system" and os.path.exists(self.outfile_remote_name[k]): - os.remove(self.outfile_remote_name[k]) - @staticmethod def context_loader_func(reg_pkl): reg = cloudpickle.loads(reg_pkl) @@ -281,106 +265,3 @@ def context_loader_func(reg_pkl): "reg": reg, } - -def compute_dts_key(reg, k, v): - """ - Compute the result of a Dask task node from dask._task_spec. - - Each value `v` may be an instance of Task, Alias, or DataNode, all of which - inherit from the same base class. The `dependencies` field is a frozenset - containing direct dependencies only (no recursive ancestry). - - The function resolves each dependency from the reg, constructs an - input dictionary, and then executes the node according to its type. - """ - try: - import dask._task_spec as dts - except ImportError: - raise ImportError("Dask is not installed") - - input_dict = {dep: reg.load_result_of_key(dep) for dep in v.dependencies} - - try: - if isinstance(v, dts.Alias): - assert len(v.dependencies) == 1, "Expected exactly one dependency" - return reg.load_result_of_key(next(iter(v.dependencies))) - elif isinstance(v, dts.Task): - return v(input_dict) - elif isinstance(v, dts.DataNode): - return v.value - else: - raise TypeError(f"unexpected node type: {type(v)} for key {k}") - except Exception as e: - raise Exception(f"Error while executing task {k}: {e}") - - -def compute_sexpr_key(reg, k, v): - """ - Evaluate a symbolic expression (S-expression) task within the task graph. - - Both DAGVine and legacy Dask represent computations as symbolic - expression trees (S-expressions). Each task value `v` encodes a nested - structure where: - - Leaf nodes are constants or task keys referencing parent results. - - Lists are recursively evaluated. - - Tuples of the form (func, arg1, arg2, ...) represent function calls. - - This function builds an input dictionary from all parent keys, then - recursively resolves and executes the expression until a final value - is produced. - """ - input_dict = {parent: reg.load_result_of_key(parent) for parent in reg.parents_of[k]} - - def _rec_call(expr): - try: - if expr in input_dict.keys(): - return input_dict[expr] - except TypeError: - pass - if isinstance(expr, list): - return [_rec_call(e) for e in expr] - if isinstance(expr, tuple) and len(expr) > 0 and callable(expr[0]): - res = expr[0](*[_rec_call(a) for a in expr[1:]]) - return res - return expr - - try: - return _rec_call(v) - except Exception as e: - raise Exception(f"Failed to invoke _rec_call(): {e}") - - -def compute_single_key(vine_key): - """ - Compute a single task identified by a Vine key within the current RuntimeExecutionGraph. - - The function retrieves the corresponding graph key and task object from the - global reg, determines the task type, and dispatches to the appropriate - execution interface — e.g., `compute_dts_key` for Dask-style task specs or - `compute_sexpr_key` for S-expression graphs. - - This design allows extensibility: for new graph representations, additional - compute interfaces can be introduced and registered here to handle new key types. - - After computation, the result is saved, the output file is validated, and - an optional delay (`extra_sleep_time_of`) is applied before returning. - """ - reg = load_variable_from_library('reg') - - k = reg.key_of_vine_key[vine_key] - v = reg.task_dict[k] - - if reg.is_dts_key(k): - result = compute_dts_key(reg, k, v) - else: - result = compute_sexpr_key(reg, k, v) - - reg.save_result_of_key(k, result) - if not os.path.exists(reg.outfile_remote_name[k]): - raise Exception(f"Output file {reg.outfile_remote_name[k]} does not exist after writing") - if os.stat(reg.outfile_remote_name[k]).st_size == 0: - raise Exception(f"Output file {reg.outfile_remote_name[k]} is empty after writing") - - time.sleep(reg.extra_sleep_time_of[k]) - - return True diff --git a/taskvine/src/graph/dagvine/strategic_orchestration_graph.c b/taskvine/src/graph/dagvine/strategic_orchestration_graph.c index c9c84eb9d2..45a6e2246b 100644 --- a/taskvine/src/graph/dagvine/strategic_orchestration_graph.c +++ b/taskvine/src/graph/dagvine/strategic_orchestration_graph.c @@ -20,10 +20,11 @@ #include "timestamp.h" #include "progress_bar.h" #include "macros.h" +#include "uuid.h" +#include "strategic_orchestration_node.h" #include "strategic_orchestration_graph.h" #include "vine_manager.h" -#include "vine_task_node.h" #include "vine_worker_info.h" #include "vine_task.h" #include "vine_file.h" @@ -52,7 +53,7 @@ static void handle_sigint(int signal) * @param priority_mode Reference to the priority mode. * @return The priority. */ -static double vine_task_node_calculate_priority(struct vine_task_node *node, task_priority_mode_t priority_mode) +static double calculate_task_priority(struct strategic_orchestration_node *node, task_priority_mode_t priority_mode) { if (!node) { return 0; @@ -61,7 +62,7 @@ static double vine_task_node_calculate_priority(struct vine_task_node *node, tas double priority = 0; timestamp_t current_time = timestamp_get(); - struct vine_task_node *parent_node; + struct strategic_orchestration_node *parent_node; switch (priority_mode) { case TASK_PRIORITY_MODE_RANDOM: @@ -108,20 +109,24 @@ static double vine_task_node_calculate_priority(struct vine_task_node *node, tas * @param sog Reference to the strategic orchestration graph object. * @param node Reference to the node object. */ -static void submit_node_task(struct strategic_orchestration_graph *sog, struct vine_task_node *node) +static void submit_node_task(struct strategic_orchestration_graph *sog, struct strategic_orchestration_node *node) { if (!sog || !node) { return; } /* calculate the priority of the node */ - double priority = vine_task_node_calculate_priority(node, sog->task_priority_mode); + double priority = calculate_task_priority(node, sog->task_priority_mode); vine_task_set_priority(node->task, priority); + /* submit the task to the manager */ int task_id = vine_submit(sog->manager, node->task); + /* insert the task id to the task id to node map */ itable_insert(sog->task_id_to_node, task_id, node); + debug(D_VINE, "submitted node %s with task id %d", node->node_key, task_id); + return; } @@ -130,13 +135,13 @@ static void submit_node_task(struct strategic_orchestration_graph *sog, struct v * @param sog Reference to the strategic orchestration graph object. * @param node Reference to the node object. */ -static void submit_unblocked_children(struct strategic_orchestration_graph *sog, struct vine_task_node *node) +static void submit_unblocked_children(struct strategic_orchestration_graph *sog, struct strategic_orchestration_node *node) { if (!sog || !node) { return; } - struct vine_task_node *child_node; + struct strategic_orchestration_node *child_node; LIST_ITERATE(node->children, child_node) { /* Remove this parent from the child's pending set if it exists */ @@ -176,7 +181,7 @@ static struct list *get_topological_order(struct strategic_orchestration_graph * struct priority_queue *pq = priority_queue_create(total_nodes); char *key; - struct vine_task_node *node; + struct strategic_orchestration_node *node; HASH_TABLE_ITERATE(sog->nodes, key, node) { int deg = list_size(node->parents); @@ -187,10 +192,10 @@ static struct list *get_topological_order(struct strategic_orchestration_graph * } while (priority_queue_size(pq) > 0) { - struct vine_task_node *current = priority_queue_pop(pq); + struct strategic_orchestration_node *current = priority_queue_pop(pq); list_push_tail(topo_order, current); - struct vine_task_node *child; + struct strategic_orchestration_node *child; LIST_ITERATE(current->children, child) { intptr_t raw_deg = (intptr_t)hash_table_lookup(in_degree_map, child->node_key); @@ -215,7 +220,7 @@ static struct list *get_topological_order(struct strategic_orchestration_graph * int deg = (int)raw_deg; if (deg > 0) { debug(D_ERROR, " Node %s has in-degree %d. Parents:\n", key, deg); - struct vine_task_node *p; + struct strategic_orchestration_node *p; LIST_ITERATE(node->parents, p) { debug(D_ERROR, " -> %s\n", p->node_key); @@ -250,7 +255,7 @@ static struct list *extract_weakly_connected_components(struct strategic_orchest struct list *components = list_create(); char *node_key; - struct vine_task_node *node; + struct strategic_orchestration_node *node; HASH_TABLE_ITERATE(sog->nodes, node_key, node) { if (set_lookup(visited, node)) { @@ -265,9 +270,9 @@ static struct list *extract_weakly_connected_components(struct strategic_orchest list_push_tail(component, node); while (list_size(queue) > 0) { - struct vine_task_node *curr = list_pop_head(queue); + struct strategic_orchestration_node *curr = list_pop_head(queue); - struct vine_task_node *p; + struct strategic_orchestration_node *p; LIST_ITERATE(curr->parents, p) { if (!set_lookup(visited, p)) { @@ -277,7 +282,7 @@ static struct list *extract_weakly_connected_components(struct strategic_orchest } } - struct vine_task_node *c; + struct strategic_orchestration_node *c; LIST_ITERATE(curr->children, c) { if (!set_lookup(visited, c)) { @@ -301,7 +306,7 @@ static struct list *extract_weakly_connected_components(struct strategic_orchest * @param node Reference to the node object. * @return The heavy score. */ -static double compute_node_heavy_score(struct vine_task_node *node) +static double compute_node_heavy_score(struct strategic_orchestration_node *node) { if (!node) { return 0; @@ -319,7 +324,7 @@ static double compute_node_heavy_score(struct vine_task_node *node) * @param task Reference to the task object. * @return The node object. */ -static struct vine_task_node *get_node_by_task(struct strategic_orchestration_graph *sog, struct vine_task *task) +static struct strategic_orchestration_node *get_node_by_task(struct strategic_orchestration_graph *sog, struct vine_task *task) { if (!sog || !task) { return NULL; @@ -394,15 +399,17 @@ int sog_tune(struct strategic_orchestration_graph *sog, const char *name, const } sog->target_results_dir = xxstrdup(value); - } else if (strcmp(name, "proxy-function-name") == 0) { - if (sog->proxy_function_name) { - free(sog->proxy_function_name); - } - sog->proxy_function_name = xxstrdup(value); - } else if (strcmp(name, "prune-depth") == 0) { sog->prune_depth = atoi(value); + } 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; + } + sog->checkpoint_fraction = fraction; + } else { debug(D_ERROR, "invalid parameter name: %s", name); return -1; @@ -412,23 +419,30 @@ int sog_tune(struct strategic_orchestration_graph *sog, const char *name, const return 0; } -void sog_set_proxy_library_name(struct strategic_orchestration_graph *sog, const char *proxy_library_name) +/** + * Get the outfile remote name of a node in the strategic orchestration graph. + * @param sog Reference to the strategic orchestration graph object. + * @param node_key Reference to the node key. + * @return The outfile remote name. + */ +const char *sog_get_node_outfile_remote_name(const struct strategic_orchestration_graph *sog, const char *node_key) { - if (!sog || !proxy_library_name) { - return; + if (!sog || !node_key) { + return NULL; } - if (sog->proxy_library_name) { - free(sog->proxy_library_name); + struct strategic_orchestration_node *node = hash_table_lookup(sog->nodes, node_key); + if (!node) { + return NULL; } - sog->proxy_library_name = xxstrdup(proxy_library_name); + return node->outfile_remote_name; } /** - * Get the proxy library name (Python-side), shared by all tasks. + * Get the proxy library name of the strategic orchestration graph. * @param sog Reference to the strategic orchestration graph object. - * @return The library name. + * @return The proxy library name. */ const char *sog_get_proxy_library_name(const struct strategic_orchestration_graph *sog) { @@ -440,17 +454,21 @@ const char *sog_get_proxy_library_name(const struct strategic_orchestration_grap } /** - * Get the proxy function name (Python-side), shared by all tasks. + * Set the proxy function name of the strategic orchestration graph. * @param sog Reference to the strategic orchestration graph object. - * @return The function name. + * @param proxy_function_name Reference to the proxy function name. */ -const char *sog_get_proxy_function_name(const struct strategic_orchestration_graph *sog) +void sog_set_proxy_function_name(struct strategic_orchestration_graph *sog, const char *proxy_function_name) { - if (!sog) { - return NULL; + if (!sog || !proxy_function_name) { + return; } - return sog->proxy_function_name; + if (sog->proxy_function_name) { + free(sog->proxy_function_name); + } + + sog->proxy_function_name = xxstrdup(proxy_function_name); } /** @@ -465,7 +483,7 @@ double sog_get_node_heavy_score(const struct strategic_orchestration_graph *sog, return -1; } - struct vine_task_node *node = hash_table_lookup(sog->nodes, node_key); + struct strategic_orchestration_node *node = hash_table_lookup(sog->nodes, node_key); if (!node) { return -1; } @@ -486,7 +504,7 @@ const char *sog_get_node_local_outfile_source(const struct strategic_orchestrati return NULL; } - struct vine_task_node *node = hash_table_lookup(sog->nodes, node_key); + struct strategic_orchestration_node *node = hash_table_lookup(sog->nodes, node_key); if (!node) { debug(D_ERROR, "node %s not found", node_key); exit(1); @@ -502,7 +520,7 @@ const char *sog_get_node_local_outfile_source(const struct strategic_orchestrati /** * Compute the topology metrics of the strategic orchestration graph, including depth, height, upstream and downstream counts, - * heavy scores, and weakly connected components. + * heavy scores, and weakly connected components. Must be called after all nodes and dependencies are added. * @param sog Reference to the strategic orchestration graph object. */ void sog_compute_topology_metrics(struct strategic_orchestration_graph *sog) @@ -518,9 +536,9 @@ void sog_compute_topology_metrics(struct strategic_orchestration_graph *sog) } char *node_key; - struct vine_task_node *node; - struct vine_task_node *parent_node; - struct vine_task_node *child_node; + struct strategic_orchestration_node *node; + struct strategic_orchestration_node *parent_node; + struct strategic_orchestration_node *child_node; /* compute the depth of the node */ LIST_ITERATE(topo_order, node) @@ -594,6 +612,57 @@ void sog_compute_topology_metrics(struct strategic_orchestration_graph *sog) node->heavy_score = compute_node_heavy_score(node); } + /* sort nodes using priority queue */ + int total_nodes = list_size(topo_order); + int total_target_nodes = 0; + struct priority_queue *sorted_nodes = priority_queue_create(total_nodes); + LIST_ITERATE(topo_order, node) + { + if (node->is_target_key) { + total_target_nodes++; + } + priority_queue_push(sorted_nodes, node, node->heavy_score); + } + /* calculate the number of nodes to be checkpointed */ + int checkpoint_count = (int)((total_nodes - total_target_nodes) * sog->checkpoint_fraction); + if (checkpoint_count < 0) { + checkpoint_count = 0; + } + + /* assign outfile types to each node */ + int assigned_checkpoint_count = 0; + while ((node = priority_queue_pop(sorted_nodes))) { + if (node->is_target_key) { + /* declare the output file as a vine_file so that it can be retrieved by the manager as usual */ + node->outfile_type = VINE_NODE_OUTFILE_TYPE_LOCAL; + char *local_outfile_path = string_format("%s/%s", sog->target_results_dir, node->outfile_remote_name); + node->outfile = vine_declare_file(node->manager, local_outfile_path, VINE_CACHE_LEVEL_WORKFLOW, 0); + free(local_outfile_path); + continue; + } + if (assigned_checkpoint_count < checkpoint_count) { + /* checkpointed files will be written directly to the shared file system, no need to manage them in the manager */ + node->outfile_type = VINE_NODE_OUTFILE_TYPE_SHARED_FILE_SYSTEM; + char *shared_file_system_outfile_path = string_format("%s/%s", sog->target_results_dir, node->outfile_remote_name); + free(node->outfile_remote_name); + node->outfile_remote_name = shared_file_system_outfile_path; + node->outfile = NULL; + assigned_checkpoint_count++; + } else { + /* other nodes will be declared as temp files to leverage node-local storage */ + node->outfile_type = VINE_NODE_OUTFILE_TYPE_TEMP; + node->outfile = vine_declare_temp(sog->manager); + } + } + /* track the output dependencies of regular and vine_temp nodes */ + LIST_ITERATE(topo_order, node) + { + if (node->outfile) { + vine_task_add_output(node->task, node->outfile, node->outfile_remote_name, VINE_TRANSFER_ALWAYS); + } + } + priority_queue_delete(sorted_nodes); + /* extract weakly connected components */ struct list *weakly_connected_components = extract_weakly_connected_components(sog); struct list *component; @@ -616,19 +685,21 @@ void sog_compute_topology_metrics(struct strategic_orchestration_graph *sog) * Create a new node and track it in the strategic orchestration graph. * @param sog Reference to the strategic orchestration graph object. * @param node_key Reference to the node key. + * @param is_target_key Reference to whether the node is a target key. * @return A new node object. */ -void sog_add_node(struct strategic_orchestration_graph *sog, const char *node_key) +void sog_add_node(struct strategic_orchestration_graph *sog, const char *node_key, int is_target_key) { if (!sog || !node_key) { return; } /* if the node already exists, skip creating a new one */ - struct vine_task_node *node = hash_table_lookup(sog->nodes, node_key); + struct strategic_orchestration_node *node = hash_table_lookup(sog->nodes, node_key); if (!node) { - node = vine_task_node_create(sog->manager, + node = son_create(sog->manager, node_key, + is_target_key, sog->proxy_library_name, sog->proxy_function_name, sog->target_results_dir, @@ -651,6 +722,10 @@ void sog_add_node(struct strategic_orchestration_graph *sog, const char *node_ke */ struct strategic_orchestration_graph *sog_create(struct vine_manager *q) { + if (!q) { + return NULL; + } + struct strategic_orchestration_graph *sog = xxmalloc(sizeof(struct strategic_orchestration_graph)); sog->manager = q; @@ -660,7 +735,11 @@ struct strategic_orchestration_graph *sog_create(struct vine_manager *q) sog->outfile_cachename_to_node = hash_table_create(0, 0); sog->target_results_dir = xxstrdup(sog->manager->runtime_directory); // default to current working directory - sog->proxy_library_name = NULL; + + cctools_uuid_t proxy_library_name_id; + cctools_uuid_create(&proxy_library_name_id); + sog->proxy_library_name = xxstrdup(proxy_library_name_id.str); + sog->proxy_function_name = NULL; sog->prune_depth = 1; @@ -690,12 +769,12 @@ void sog_add_dependency(struct strategic_orchestration_graph *sog, const char *p return; } - struct vine_task_node *parent_node = hash_table_lookup(sog->nodes, parent_key); - struct vine_task_node *child_node = hash_table_lookup(sog->nodes, child_key); + struct strategic_orchestration_node *parent_node = hash_table_lookup(sog->nodes, parent_key); + struct strategic_orchestration_node *child_node = hash_table_lookup(sog->nodes, child_key); if (!parent_node) { debug(D_ERROR, "parent node %s not found", parent_key); char *node_key = NULL; - struct vine_task_node *node; + struct strategic_orchestration_node *node; printf("parent_keys:\n"); HASH_TABLE_ITERATE(sog->nodes, node_key, node) { @@ -714,30 +793,6 @@ void sog_add_dependency(struct strategic_orchestration_graph *sog, const char *p return; } -/** - * Set the outfile of a node in the strategic orchestration graph. - * This involves declaring the output file and adding it to the task. - * @param sog Reference to the strategic orchestration graph object. - * @param node_key Reference to the node key. - * @param outfile_type Reference to the outfile type. - * @param outfile_remote_name Reference to the outfile remote name that the task will generate remotely. - */ -void sog_set_node_outfile(struct strategic_orchestration_graph *sog, const char *node_key, vine_task_node_outfile_type_t outfile_type, const char *outfile_remote_name) -{ - if (!sog || !node_key || !outfile_remote_name) { - return; - } - - struct vine_task_node *node = hash_table_lookup(sog->nodes, node_key); - if (!node) { - return; - } - - vine_task_node_set_outfile(node, outfile_type, outfile_remote_name); - - return; -} - /** * Execute the strategic orchestration graph. This must be called after all nodes and dependencies are added and the topology metrics are computed. * @param sog Reference to the strategic orchestration graph object. @@ -754,10 +809,10 @@ void sog_execute(struct strategic_orchestration_graph *sog) /* print the info of all nodes */ char *node_key; - struct vine_task_node *node; + struct strategic_orchestration_node *node; HASH_TABLE_ITERATE(sog->nodes, node_key, node) { - vine_task_node_print_info(node); + son_print_info(node); } /* enable return recovery tasks */ @@ -775,7 +830,7 @@ void sog_execute(struct strategic_orchestration_graph *sog) struct list *topo_order = get_topological_order(sog); LIST_ITERATE(topo_order, node) { - struct vine_task_node *parent_node; + struct strategic_orchestration_node *parent_node; LIST_ITERATE(node->parents, parent_node) { if (parent_node->outfile) { @@ -787,7 +842,7 @@ void sog_execute(struct strategic_orchestration_graph *sog) /* initialize pending_parents for all nodes */ HASH_TABLE_ITERATE(sog->nodes, node_key, node) { - struct vine_task_node *parent_node; + struct strategic_orchestration_node *parent_node; LIST_ITERATE(node->parents, parent_node) { if (node->pending_parents) { @@ -831,7 +886,7 @@ void sog_execute(struct strategic_orchestration_graph *sog) wait_timeout = 0; /* get the original node by task id */ - struct vine_task_node *node = get_node_by_task(sog, task); + struct strategic_orchestration_node *node = get_node_by_task(sog, 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); @@ -882,7 +937,7 @@ void sog_execute(struct strategic_orchestration_graph *sog) node->completed = 1; /* prune nodes on task completion */ - vine_task_node_prune_ancestors(node); + son_prune_ancestors(node); /* skip recovery tasks */ if (task->type == VINE_TASK_TYPE_RECOVERY) { @@ -896,7 +951,7 @@ void sog_execute(struct strategic_orchestration_graph *sog) } /* update critical time */ - vine_task_node_update_critical_time(node, task->time_workers_execute_last); + son_update_critical_time(node, task->time_workers_execute_last); /* mark this regular task as completed */ progress_bar_update_part(pbar, regular_tasks_part, 1); @@ -913,7 +968,7 @@ void sog_execute(struct strategic_orchestration_graph *sog) /* enqueue the output file for replication */ switch (node->outfile_type) { case VINE_NODE_OUTFILE_TYPE_TEMP: - vine_task_node_replicate_outfile(node); + son_replicate_outfile(node); break; case VINE_NODE_OUTFILE_TYPE_LOCAL: case VINE_NODE_OUTFILE_TYPE_SHARED_FILE_SYSTEM: @@ -962,7 +1017,7 @@ void sog_delete(struct strategic_orchestration_graph *sog) } char *node_key; - struct vine_task_node *node; + struct strategic_orchestration_node *node; HASH_TABLE_ITERATE(sog->nodes, node_key, node) { if (node->infile) { @@ -974,7 +1029,10 @@ void sog_delete(struct strategic_orchestration_graph *sog) hash_table_remove(sog->outfile_cachename_to_node, node->outfile->cached_name); hash_table_remove(sog->manager->file_table, node->outfile->cached_name); } - vine_task_node_delete(node); + if (node->outfile_type == VINE_NODE_OUTFILE_TYPE_SHARED_FILE_SYSTEM) { + unlink(node->outfile_remote_name); + } + son_delete(node); } free(sog->proxy_library_name); diff --git a/taskvine/src/graph/dagvine/strategic_orchestration_graph.h b/taskvine/src/graph/dagvine/strategic_orchestration_graph.h index 2cbbcbcce1..db8009c9fa 100644 --- a/taskvine/src/graph/dagvine/strategic_orchestration_graph.h +++ b/taskvine/src/graph/dagvine/strategic_orchestration_graph.h @@ -6,7 +6,7 @@ #include "list.h" #include "vine_manager.h" #include "set.h" -#include "vine_task_node.h" +#include "strategic_orchestration_node.h" #include "taskvine.h" @@ -39,6 +39,7 @@ struct strategic_orchestration_graph { char *proxy_function_name; // Python-side proxy function name (shared by all tasks) int prune_depth; + double checkpoint_fraction; // 0 - 1, the fraction of intermediate results to checkpoint task_priority_mode_t task_priority_mode; // priority mode for task graph task scheduling double failure_injection_step_percent; // 0 - 100, the percentage of steps to inject failure @@ -56,8 +57,9 @@ struct strategic_orchestration_graph *sog_create(struct vine_manager *q); /** Create a new node in the strategic orchestration graph. @param sog Reference to the strategic orchestration graph object. @param node_key Reference to the node key. +@param is_target_key Reference to whether the node is a target key. */ -void sog_add_node(struct strategic_orchestration_graph *sog, const char *node_key); +void sog_add_node(struct strategic_orchestration_graph *sog, const char *node_key, int is_target_key); /** Add a dependency between two nodes in the strategic orchestration graph. @param sog Reference to the strategic orchestration graph object. @@ -78,18 +80,17 @@ void sog_compute_topology_metrics(struct strategic_orchestration_graph *sog); */ double sog_get_node_heavy_score(const struct strategic_orchestration_graph *sog, const char *node_key); -/** Set the type of the node-output file. +/** Execute the task graph. @param sog Reference to the strategic orchestration graph object. -@param node_key Reference to the node key. -@param outfile_type Reference to the output file type. -@param outfile_remote_name Reference to the output file remote name. */ -void sog_set_node_outfile(struct strategic_orchestration_graph *sog, const char *node_key, vine_task_node_outfile_type_t outfile_type, const char *outfile_remote_name); +void sog_execute(struct strategic_orchestration_graph *sog); -/** Execute the task graph. +/** Get the outfile remote name of a node in the strategic orchestration graph. @param sog Reference to the strategic orchestration graph object. +@param node_key Reference to the node key. +@return The outfile remote name. */ -void sog_execute(struct strategic_orchestration_graph *sog); +const char *sog_get_node_outfile_remote_name(const struct strategic_orchestration_graph *sog, const char *node_key); /** Get the local outfile source of a node in the strategic orchestration graph. @param sog Reference to the strategic orchestration graph object. @@ -103,23 +104,17 @@ const char *sog_get_node_local_outfile_source(const struct strategic_orchestrati */ void sog_delete(struct strategic_orchestration_graph *sog); -/** Set the Python-side proxy library name of the strategic orchestration graph. -@param sog Reference to the strategic orchestration graph object. -@param proxy_library_name Reference to the Python-side proxy library name. -*/ -void sog_set_proxy_library_name(struct strategic_orchestration_graph *sog, const char *proxy_library_name); - -/** Get the library name of the strategic orchestration graph. +/** Get the proxy library name of the strategic orchestration graph. @param sog Reference to the strategic orchestration graph object. -@return The library name. +@return The proxy library name. */ const char *sog_get_proxy_library_name(const struct strategic_orchestration_graph *sog); -/** Get the function name of the strategic orchestration graph. +/** Set the proxy function name of the strategic orchestration graph. @param sog Reference to the strategic orchestration graph object. -@return The function name. +@param proxy_function_name Reference to the proxy function name. */ -const char *sog_get_proxy_function_name(const struct strategic_orchestration_graph *sog); +void sog_set_proxy_function_name(struct strategic_orchestration_graph *sog, const char *proxy_function_name); /** Tune the strategic orchestration graph. @param sog Reference to the strategic orchestration graph object. diff --git a/taskvine/src/graph/dagvine/strategic_orchestration_graph.i b/taskvine/src/graph/dagvine/strategic_orchestration_graph.i index 2ac5e9a3f4..86295b3949 100644 --- a/taskvine/src/graph/dagvine/strategic_orchestration_graph.i +++ b/taskvine/src/graph/dagvine/strategic_orchestration_graph.i @@ -3,8 +3,8 @@ %{ #include "int_sizes.h" +#include "strategic_orchestration_node.h" #include "strategic_orchestration_graph.h" -#include "vine_task_node.h" /* expose outfile type enum to SWIG */ %} %include "stdint.i" @@ -13,9 +13,7 @@ /* Import existing SWIG interface for type information (do not wrap again) */ %import "../../bindings/python3/taskvine.i" -/* Expose only the dagvine task graph APIs */ -%ignore vine_task_node_checkpoint_outfile; /* avoid exporting unimplemented/optional symbol */ -%include "vine_task_node.h" /* export vine_task_node_outfile_type_t values */ +%include "strategic_orchestration_node.h" %include "strategic_orchestration_graph.h" diff --git a/taskvine/src/graph/dagvine/vine_task_node.c b/taskvine/src/graph/dagvine/vine_task_node.c deleted file mode 100644 index 0539c59f94..0000000000 --- a/taskvine/src/graph/dagvine/vine_task_node.c +++ /dev/null @@ -1,640 +0,0 @@ -#include -#include -#include -#include -#include -#include -#include - -#include "jx.h" -#include "jx_print.h" -#include "xxmalloc.h" -#include "stringtools.h" -#include "timestamp.h" -#include "set.h" -#include "hash_table.h" -#include "debug.h" -#include "random.h" - -#include "vine_manager.h" -#include "vine_file.h" -#include "vine_task.h" -#include "vine_worker_info.h" -#include "vine_temp.h" -#include "vine_task_node.h" -#include "taskvine.h" - - -/** - * Compute a lexicographic priority score from the node key. - * Used during topological sorting to break ties deterministically. - * @param key Reference to the node key. - * @return The lexical priority. - */ -double compute_lex_priority(const char *key) -{ - double score = 0.0; - double factor = 1.0; - for (int i = 0; i < 8 && key[i] != '\0'; i++) { - score += (unsigned char)key[i] * factor; - factor *= 0.01; - } - return -score; -} - -/** - * Create a new node object. - * @param manager Reference to the manager object. - * @param node_key Reference to the node key. - * @param proxy_library_name Reference to the proxy library name. - * @param proxy_function_name Reference to the proxy function name. - * @param staging_dir Reference to the staging directory. - * @param prune_depth Reference to the prune depth. - * @return A new node object. - */ -struct vine_task_node *vine_task_node_create( - struct vine_manager *manager, - const char *node_key, - const char *proxy_library_name, - const char *proxy_function_name, - const char *target_results_dir, - int prune_depth) -{ - if (!manager) { - debug(D_ERROR, "Cannot create node because manager is NULL"); - return NULL; - } - if (!node_key) { - debug(D_ERROR, "Cannot create node because node_key is NULL"); - return NULL; - } - if (!proxy_library_name) { - debug(D_ERROR, "Cannot create node because proxy_library_name is NULL"); - return NULL; - } - if (!proxy_function_name) { - debug(D_ERROR, "Cannot create node because proxy_function_name is NULL"); - return NULL; - } - if (!target_results_dir) { - debug(D_ERROR, "Cannot create node because target_results_dir is NULL"); - return NULL; - } - - struct vine_task_node *node = xxmalloc(sizeof(struct vine_task_node)); - - node->manager = manager; - node->node_key = xxstrdup(node_key); - node->target_results_dir = xxstrdup(target_results_dir); - - node->prune_status = PRUNE_STATUS_NOT_PRUNED; - node->parents = list_create(); - node->children = list_create(); - node->pending_parents = set_create(0); - node->completed = 0; - node->prune_depth = prune_depth; - node->retry_attempts_left = 1; - node->outfile_size_bytes = 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->time_spent_on_unlink_local_files = 0; - node->time_spent_on_prune_ancestors_of_temp_node = 0; - node->time_spent_on_prune_ancestors_of_persisted_node = 0; - - node->critical_time = -1; - - /* create the task */ - node->task = vine_task_create(proxy_function_name); - vine_task_set_library_required(node->task, proxy_library_name); - vine_task_addref(node->task); - - /* build JSON infile expected by library: {"fn_args": [key], "fn_kwargs": {}} */ - struct jx *event = jx_object(NULL); - struct jx *args = jx_array(NULL); - jx_array_append(args, jx_string(node->node_key)); - jx_insert(event, jx_string("fn_args"), args); - jx_insert(event, jx_string("fn_kwargs"), jx_object(NULL)); - - char *infile_content = jx_print_string(event); - jx_delete(event); - - node->infile = vine_declare_buffer(node->manager, infile_content, strlen(infile_content), VINE_CACHE_LEVEL_TASK, VINE_UNLINK_WHEN_DONE); - free(infile_content); - vine_task_add_input(node->task, node->infile, "infile", VINE_TRANSFER_ALWAYS); - - return node; -} - -/** - * Set the outfile of a node in the task graph. - * This involves declaring the output file and adding it to the task. - * @param node Reference to the node object. - * @param outfile_type Reference to the outfile type. - * @param outfile_remote_name Reference to the outfile remote name. - */ -void vine_task_node_set_outfile(struct vine_task_node *node, vine_task_node_outfile_type_t outfile_type, const char *outfile_remote_name) -{ - if (!node) { - return; - } - - assert(outfile_remote_name != NULL); - - node->outfile_type = outfile_type; - node->outfile_remote_name = xxstrdup(outfile_remote_name); - - /* create the output file */ - switch (node->outfile_type) { - case VINE_NODE_OUTFILE_TYPE_LOCAL: { - if (!node->target_results_dir) { - debug(D_ERROR, "Cannot create local output file for node %s because target_results_dir is NULL", node->node_key); - exit(1); - } - char *local_output_path = string_format("%s/%s", node->target_results_dir, node->outfile_remote_name); - node->outfile = vine_declare_file(node->manager, local_output_path, VINE_CACHE_LEVEL_WORKFLOW, 0); - free(local_output_path); - break; - } - case VINE_NODE_OUTFILE_TYPE_TEMP: - node->outfile = vine_declare_temp(node->manager); - break; - case VINE_NODE_OUTFILE_TYPE_SHARED_FILE_SYSTEM: - /* no explicit output file declaration needed */ - node->outfile = NULL; - break; - } - if (node->outfile) { - vine_task_add_output(node->task, node->outfile, node->outfile_remote_name, VINE_TRANSFER_ALWAYS); - } - - return; -} - -/** - * Check if the outfile of a node is persisted. - * A node is considered persisted if it has completed and 1) the outfile is written to the shared file system, - * 2) the outfile is written to the local staging directory. - * @param node Reference to the node object. - * @return 1 if the outfile is persisted, 0 otherwise. - */ -static int _node_outfile_is_persisted(struct vine_task_node *node) -{ - if (!node) { - return 0; - } - - /* if the node is not completed then the outfile is definitely not persisted */ - if (!node->completed) { - return 0; - } - - switch (node->outfile_type) { - case VINE_NODE_OUTFILE_TYPE_LOCAL: - return 1; - case VINE_NODE_OUTFILE_TYPE_SHARED_FILE_SYSTEM: - return 1; - case VINE_NODE_OUTFILE_TYPE_TEMP: - return 0; - } - - return 0; -} - -/** - * Update the critical time of a node. - * @param node Reference to the node object. - * @param execution_time Reference to the execution time of the node. - */ -void vine_task_node_update_critical_time(struct vine_task_node *node, timestamp_t execution_time) -{ - timestamp_t max_parent_critical_time = 0; - struct vine_task_node *parent_node; - LIST_ITERATE(node->parents, parent_node) - { - if (parent_node->critical_time > max_parent_critical_time) { - max_parent_critical_time = parent_node->critical_time; - } - } - node->critical_time = max_parent_critical_time + execution_time; -} - -/** - * The dfs helper function for finding parents in a specific depth. - * @param node Reference to the node object. - * @param remaining_depth Reference to the remaining depth. - * @param result Reference to the result list. - * @param visited Reference to the visited set. - */ -static void _find_parents_dfs(struct vine_task_node *node, int remaining_depth, struct list *result, struct set *visited) -{ - if (!node || set_lookup(visited, node)) { - return; - } - - set_insert(visited, node); - if (remaining_depth == 0) { - list_push_tail(result, node); - return; - } - struct vine_task_node *parent_node; - LIST_ITERATE(node->parents, parent_node) - { - _find_parents_dfs(parent_node, remaining_depth - 1, result, visited); - } -} - -/** - * Find all parents in a specific depth of the node. - * @param node Reference to the node object. - * @param depth Reference to the depth. - * @return The list of parents. - */ -static struct list *_find_parents_in_depth(struct vine_task_node *node, int depth) -{ - if (!node || depth < 0) { - return NULL; - } - - struct list *result = list_create(); - - struct set *visited = set_create(0); - _find_parents_dfs(node, depth, result, visited); - set_delete(visited); - - return result; -} - -/** - * Prune the ancestors of a temp node. This is only used for temp nodes that produce temp files. - * All ancestors of this node we consider here are temp nodes, we can not safely prune those stored in the shared file system - * because temp nodes are not considered safe enough to trigger the deletion of upstream persisted files. - * @param node Reference to the node object. - * @return The number of pruned replicas. - */ -static int prune_ancestors_of_temp_node(struct vine_task_node *node) -{ - if (!node || !node->outfile || node->prune_depth <= 0) { - return 0; - } - - timestamp_t start_time = timestamp_get(); - - int pruned_replica_count = 0; - - struct list *parents = _find_parents_in_depth(node, node->prune_depth); - - struct vine_task_node *parent_node; - LIST_ITERATE(parents, parent_node) - { - /* skip if the parent produces a shared file system file */ - if (!parent_node->outfile) { - continue; - } - /* skip if the parent produces a non-temp file */ - if (parent_node->outfile->type != VINE_TEMP) { - continue; - } - - /* a file is prunable if its outfile is no longer needed by any child node: - * 1. it has no pending dependents - * 2. all completed dependents have also completed their corresponding recovery tasks, if any */ - int all_children_completed = 1; - struct vine_task_node *child_node; - LIST_ITERATE(parent_node->children, child_node) - { - /* break early if the child node is not completed */ - if (!child_node->completed) { - all_children_completed = 0; - break; - } - /* if the task produces a temp file and the recovery task is running, the parent is not prunable */ - if (child_node->outfile && child_node->outfile->type == VINE_TEMP) { - struct vine_task *child_node_recovery_task = child_node->outfile->recovery_task; - if (child_node_recovery_task && (child_node_recovery_task->state != VINE_TASK_INITIAL && child_node_recovery_task->state != VINE_TASK_DONE)) { - all_children_completed = 0; - break; - } - } - } - if (!all_children_completed) { - continue; - } - - pruned_replica_count += vine_prune_file(node->manager, parent_node->outfile); - /* this parent is pruned because a successor that produces a temp file is completed, it is unsafe because the - * manager may submit a recovery task to bring it back in case of worker failures. */ - parent_node->prune_status = PRUNE_STATUS_UNSAFE; - } - - list_delete(parents); - - node->time_spent_on_prune_ancestors_of_temp_node += timestamp_get() - start_time; - - return pruned_replica_count; -} - -/** - * Find all safe ancestors of a node. - * @param start_node Reference to the start node. - * @return The set of safe ancestors. - */ -static struct set *_find_safe_ancestors(struct vine_task_node *start_node) -{ - if (!start_node) { - return NULL; - } - - struct set *visited_nodes = set_create(0); - struct list *bfs_nodes = list_create(); - - list_push_tail(bfs_nodes, start_node); - set_insert(visited_nodes, start_node); - - while (list_size(bfs_nodes) > 0) { - struct vine_task_node *current = list_pop_head(bfs_nodes); - - struct vine_task_node *parent_node; - LIST_ITERATE(current->parents, parent_node) - { - if (set_lookup(visited_nodes, parent_node)) { - continue; - } - - /* shortcut if this parent was already safely pruned */ - if (parent_node->prune_status == PRUNE_STATUS_SAFE) { - continue; - } - - /* check if all children are safe */ - int all_children_safe = 1; - struct vine_task_node *child_node; - LIST_ITERATE(parent_node->children, child_node) - { - /* shortcut if this child is part of the recovery subgraph */ - if (set_lookup(visited_nodes, child_node)) { - continue; - } - /* shortcut if this outside child is not persisted */ - if (!_node_outfile_is_persisted(child_node)) { - all_children_safe = 0; - break; - } - /* shortcut if this outside child is unsafely pruned */ - if (child_node->prune_status == PRUNE_STATUS_UNSAFE) { - all_children_safe = 0; - break; - } - } - - if (!all_children_safe) { - continue; - } - - set_insert(visited_nodes, parent_node); - list_push_tail(bfs_nodes, parent_node); - } - } - - list_delete(bfs_nodes); - set_remove(visited_nodes, start_node); - - return visited_nodes; -} - -/** - * Prune the ancestors of a persisted node. This is only used for persisted nodes that produce persisted files. - * All ancestors we consider here include both temp nodes and persisted nodes, becasue data written to shared file system - * is safe and can definitely trigger upstream data redundancy to be released. - * @param node Reference to the node object. - * @return The number of pruned replicas. - */ -static int prune_ancestors_of_persisted_node(struct vine_task_node *node) -{ - if (!node) { - return 0; - } - - timestamp_t start_time = timestamp_get(); - - int pruned_replica_count = 0; - - /* find all safe ancestors */ - struct set *safe_ancestors = _find_safe_ancestors(node); - if (!safe_ancestors) { - return 0; - } - - /* prune all safe ancestors */ - struct vine_task_node *ancestor_node; - SET_ITERATE(safe_ancestors, ancestor_node) - { - /* unlink the shared file system file */ - if (!ancestor_node->outfile) { - timestamp_t unlink_start = timestamp_get(); - if (ancestor_node->outfile_remote_name) { - unlink(ancestor_node->outfile_remote_name); // system call - } - node->time_spent_on_unlink_local_files += timestamp_get() - unlink_start; - debug(D_VINE, "unlinked %s size: %zu bytes, time: %" PRIu64, ancestor_node->outfile_remote_name ? ancestor_node->outfile_remote_name : "(null)", ancestor_node->outfile_size_bytes, (uint64_t)node->time_spent_on_unlink_local_files); - } else { - switch (ancestor_node->outfile->type) { - case VINE_FILE: - /* do not prune the staging dir file */ - break; - case VINE_TEMP: - /* prune the temp file */ - vine_prune_file(node->manager, ancestor_node->outfile); - break; - default: - debug(D_ERROR, "unsupported outfile type: %d", ancestor_node->outfile->type); - break; - } - } - ancestor_node->prune_status = PRUNE_STATUS_SAFE; - pruned_replica_count++; - } - - set_delete(safe_ancestors); - - node->time_spent_on_prune_ancestors_of_persisted_node += timestamp_get() - start_time; - - return pruned_replica_count; -} - -/** - * Print the info of the node. - * @param node Reference to the node object. - */ -void vine_task_node_print_info(struct vine_task_node *node) -{ - if (!node) { - return; - } - - if (!node->task) { - debug(D_ERROR, "node %s has no task", node->node_key); - return; - } - - debug(D_VINE, "---------------- Node Info ----------------"); - debug(D_VINE, "key: %s", node->node_key); - debug(D_VINE, "task_id: %d", node->task->task_id); - debug(D_VINE, "target_results_dir: %s", node->target_results_dir ? node->target_results_dir : "(null)"); - debug(D_VINE, "depth: %d", node->depth); - debug(D_VINE, "height: %d", node->height); - debug(D_VINE, "prune_depth: %d", node->prune_depth); - - 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"); - } - - /* print parent and child node keys */ - char *parent_keys = NULL; - struct vine_task_node *p; - LIST_ITERATE(node->parents, p) - { - if (!parent_keys) { - parent_keys = string_format("%s", p->node_key); - } else { - char *tmp = string_format("%s, %s", parent_keys, p->node_key); - free(parent_keys); - parent_keys = tmp; - } - } - - char *child_keys = NULL; - struct vine_task_node *c; - LIST_ITERATE(node->children, c) - { - if (!child_keys) { - child_keys = string_format("%s", c->node_key); - } else { - char *tmp = string_format("%s, %s", child_keys, c->node_key); - free(child_keys); - child_keys = tmp; - } - } - - debug(D_VINE, "parents: %s", parent_keys ? parent_keys : "(none)"); - debug(D_VINE, "children: %s", child_keys ? child_keys : "(none)"); - - free(parent_keys); - free(child_keys); - - debug(D_VINE, "-------------------------------------------"); -} - -/** - * Prune the ancestors of a node when it is completed. - * @param node Reference to the node object. - */ -void vine_task_node_prune_ancestors(struct vine_task_node *node) -{ - if (!node) { - return; - } - - timestamp_t start_time = timestamp_get(); - - int pruned_replica_count = 0; - - if (_node_outfile_is_persisted(node)) { - pruned_replica_count = prune_ancestors_of_persisted_node(node); - } else { - pruned_replica_count = prune_ancestors_of_temp_node(node); - } - - timestamp_t elapsed_time = timestamp_get() - start_time; - - debug(D_VINE, "pruned %d ancestors of node %s in %.6f seconds", pruned_replica_count, node->node_key, elapsed_time / 1000000.0); -} - -/** - * Replicate the outfile of a node if it is a temp file. - * @param node Reference to the node object. - */ -void vine_task_node_replicate_outfile(struct vine_task_node *node) -{ - if (!node || !node->outfile) { - return; - } - - if (node->outfile->type != VINE_TEMP) { - return; - } - - vine_temp_replicate_file_later(node->manager, node->outfile); -} - -/** - * Delete the node and all of its associated resources. - * @param node Reference to the node object. - */ -void vine_task_node_delete(struct vine_task_node *node) -{ - if (!node) { - return; - } - - if (node->node_key) { - free(node->node_key); - } - if (node->outfile_remote_name) { - free(node->outfile_remote_name); - } - if (node->target_results_dir) { - free(node->target_results_dir); - } - - vine_task_delete(node->task); - node->task = NULL; - - if (node->infile) { - vine_file_delete(node->infile); - node->infile = NULL; - } - if (node->outfile) { - vine_file_delete(node->outfile); - node->outfile = NULL; - } - - list_delete(node->parents); - list_delete(node->children); - - if (node->pending_parents) { - set_delete(node->pending_parents); - } - free(node); -} \ No newline at end of file diff --git a/taskvine/src/graph/dagvine/vine_task_node.h b/taskvine/src/graph/dagvine/vine_task_node.h deleted file mode 100644 index c9d58bf520..0000000000 --- a/taskvine/src/graph/dagvine/vine_task_node.h +++ /dev/null @@ -1,77 +0,0 @@ -#ifndef VINE_TASK_NODE_H -#define VINE_TASK_NODE_H - -#include "vine_task.h" -#include "hash_table.h" -#include "list.h" -#include "vine_manager.h" -#include "set.h" -#include "taskvine.h" - -/** Select the type of the node-output file. */ -typedef enum { - VINE_NODE_OUTFILE_TYPE_LOCAL = 0, /* Node-output file will be stored locally on the manager's staging directory */ - VINE_NODE_OUTFILE_TYPE_TEMP, /* Node-output file will be stored in the temporary node-local storage */ - VINE_NODE_OUTFILE_TYPE_SHARED_FILE_SYSTEM, /* Node-output file will be stored in the persistent shared file system */ -} vine_task_node_outfile_type_t; - -typedef enum { - PRUNE_STATUS_NOT_PRUNED = 0, - PRUNE_STATUS_SAFE, - PRUNE_STATUS_UNSAFE -} prune_status_t; - -struct vine_task_node { - char *node_key; - - struct vine_manager *manager; - struct vine_task *task; - struct vine_file *infile; - struct vine_file *outfile; - char *outfile_remote_name; - char *target_results_dir; - size_t outfile_size_bytes; - - struct list *parents; - struct list *children; - struct set *pending_parents; - - int retry_attempts_left; - int completed; - int prune_depth; - - int depth; - int height; - int upstream_subgraph_size; - int downstream_subgraph_size; - int fan_in; - int fan_out; - double heavy_score; - - timestamp_t critical_time; - timestamp_t time_spent_on_unlink_local_files; - timestamp_t time_spent_on_prune_ancestors_of_temp_node; - timestamp_t time_spent_on_prune_ancestors_of_persisted_node; - - vine_task_node_outfile_type_t outfile_type; - prune_status_t prune_status; -}; - -struct vine_task_node *vine_task_node_create( - struct vine_manager *manager, - const char *node_key, - const char *proxy_library_name, - const char *proxy_function_name, - const char *target_results_dir, - int prune_depth -); - -void vine_task_node_delete(struct vine_task_node *node); -double compute_lex_priority(const char *key); -void vine_task_node_prune_ancestors(struct vine_task_node *node); -void vine_task_node_print_info(struct vine_task_node *node); -void vine_task_node_update_critical_time(struct vine_task_node *node, timestamp_t execution_time); -void vine_task_node_replicate_outfile(struct vine_task_node *node); -void vine_task_node_set_outfile(struct vine_task_node *node, vine_task_node_outfile_type_t outfile_type, const char *outfile_remote_name); - -#endif \ No newline at end of file diff --git a/taskvine/src/manager/vine_manager.c b/taskvine/src/manager/vine_manager.c index 27832876f9..20fda511f8 100644 --- a/taskvine/src/manager/vine_manager.c +++ b/taskvine/src/manager/vine_manager.c @@ -3686,6 +3686,7 @@ static int vine_manager_check_inputs_available(struct vine_manager *q, struct vi all_available = 0; } } + return all_available; } From 73308fb528f51b4f4947a1b895579d51ee532b89 Mon Sep 17 00:00:00 2001 From: Jin Zhou Date: Sun, 19 Oct 2025 03:56:06 -0400 Subject: [PATCH 047/113] new files --- taskvine/src/graph/dagvine/proxy_functions.py | 106 +++ taskvine/src/graph/dagvine/proxy_library.py | 79 +++ .../dagvine/strategic_orchestration_graph.py | 33 + .../dagvine/strategic_orchestration_node.c | 608 ++++++++++++++++++ .../dagvine/strategic_orchestration_node.h | 92 +++ 5 files changed, 918 insertions(+) create mode 100644 taskvine/src/graph/dagvine/proxy_functions.py create mode 100644 taskvine/src/graph/dagvine/proxy_library.py create mode 100644 taskvine/src/graph/dagvine/strategic_orchestration_graph.py create mode 100644 taskvine/src/graph/dagvine/strategic_orchestration_node.c create mode 100644 taskvine/src/graph/dagvine/strategic_orchestration_node.h diff --git a/taskvine/src/graph/dagvine/proxy_functions.py b/taskvine/src/graph/dagvine/proxy_functions.py new file mode 100644 index 0000000000..c415cf9d40 --- /dev/null +++ b/taskvine/src/graph/dagvine/proxy_functions.py @@ -0,0 +1,106 @@ +import os, time +from ndcctools.taskvine.utils import load_variable_from_library + + +def compute_dts_key(reg, k, v): + """ + Compute the result of a Dask task node from dask._task_spec. + + Each value `v` may be an instance of Task, Alias, or DataNode, all of which + inherit from the same base class. The `dependencies` field is a frozenset + containing direct dependencies only (no recursive ancestry). + + The function resolves each dependency from the reg, constructs an + input dictionary, and then executes the node according to its type. + """ + try: + import dask._task_spec as dts + except ImportError: + raise ImportError("Dask is not installed") + + input_dict = {dep: reg.load_result_of_key(dep) for dep in v.dependencies} + + try: + if isinstance(v, dts.Alias): + assert len(v.dependencies) == 1, "Expected exactly one dependency" + return reg.load_result_of_key(next(iter(v.dependencies))) + elif isinstance(v, dts.Task): + return v(input_dict) + elif isinstance(v, dts.DataNode): + return v.value + else: + raise TypeError(f"unexpected node type: {type(v)} for key {k}") + except Exception as e: + raise Exception(f"Error while executing task {k}: {e}") + + +def compute_sexpr_key(reg, k, v): + """ + Evaluate a symbolic expression (S-expression) task within the task graph. + + Both DAGVine and legacy Dask represent computations as symbolic + expression trees (S-expressions). Each task value `v` encodes a nested + structure where: + - Leaf nodes are constants or task keys referencing parent results. + - Lists are recursively evaluated. + - Tuples of the form (func, arg1, arg2, ...) represent function calls. + + This function builds an input dictionary from all parent keys, then + recursively resolves and executes the expression until a final value + is produced. + """ + input_dict = {parent: reg.load_result_of_key(parent) for parent in reg.parents_of[k]} + + def _rec_call(expr): + try: + if expr in input_dict.keys(): + return input_dict[expr] + except TypeError: + pass + if isinstance(expr, list): + return [_rec_call(e) for e in expr] + if isinstance(expr, tuple) and len(expr) > 0 and callable(expr[0]): + res = expr[0](*[_rec_call(a) for a in expr[1:]]) + return res + return expr + + try: + return _rec_call(v) + except Exception as e: + raise Exception(f"Failed to invoke _rec_call(): {e}") + + +def compute_single_key(vine_key): + """ + Compute a single task identified by a Vine key within the current RuntimeExecutionGraph. + + The function retrieves the corresponding graph key and task object from the + global reg, determines the task type, and dispatches to the appropriate + execution interface — e.g., `compute_dts_key` for Dask-style task specs or + `compute_sexpr_key` for S-expression graphs. + + This design allows extensibility: for new graph representations, additional + compute interfaces can be introduced and registered here to handle new key types. + + After computation, the result is saved, the output file is validated, and + an optional delay (`extra_sleep_time_of`) is applied before returning. + """ + reg = load_variable_from_library('reg') + + k = reg.reg_node_key_of[vine_key] + v = reg.task_dict[k] + + if reg.is_dts_key(k): + result = compute_dts_key(reg, k, v) + else: + result = compute_sexpr_key(reg, k, v) + + reg.save_result_of_key(k, result) + if not os.path.exists(reg.outfile_remote_name[k]): + raise Exception(f"Output file {reg.outfile_remote_name[k]} does not exist after writing") + if os.stat(reg.outfile_remote_name[k]).st_size == 0: + raise Exception(f"Output file {reg.outfile_remote_name[k]} is empty after writing") + + time.sleep(reg.extra_sleep_time_of[k]) + + return True diff --git a/taskvine/src/graph/dagvine/proxy_library.py b/taskvine/src/graph/dagvine/proxy_library.py new file mode 100644 index 0000000000..aa14b4a607 --- /dev/null +++ b/taskvine/src/graph/dagvine/proxy_library.py @@ -0,0 +1,79 @@ +import os, uuid, cloudpickle, types, time, random, hashlib, collections +from ndcctools.taskvine.dagvine.runtime_execution_graph import ( + GraphKeyResult, RuntimeExecutionGraph, hash_name, hashable +) +from ndcctools.taskvine.dagvine.proxy_functions import compute_dts_key, compute_sexpr_key, compute_single_key +from ndcctools.taskvine.utils import load_variable_from_library + + +class ProxyLibrary: + 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, GraphKeyResult, RuntimeExecutionGraph, uuid, hashlib, random, types, collections, time, + load_variable_from_library, compute_dts_key, compute_sexpr_key, compute_single_key, hash_name, hashable + ] + + # 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, + compute_single_key, + 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/graph/dagvine/strategic_orchestration_graph.py b/taskvine/src/graph/dagvine/strategic_orchestration_graph.py new file mode 100644 index 0000000000..c441997a14 --- /dev/null +++ b/taskvine/src/graph/dagvine/strategic_orchestration_graph.py @@ -0,0 +1,33 @@ +from ndcctools.taskvine.dagvine import cdagvine + + +class StrategicOrchestrationGraph: + def __init__(self, c_taskvine): + self._c_sog = cdagvine.sog_create(c_taskvine) + + def tune(self, name, value): + cdagvine.sog_tune(self._c_sog, name, value) + + def add_node(self, key, is_target_key): + cdagvine.sog_add_node(self._c_sog, key, is_target_key) + + def add_dependency(self, parent_key, child_key): + cdagvine.sog_add_dependency(self._c_sog, parent_key, child_key) + + def compute_topology_metrics(self): + cdagvine.sog_compute_topology_metrics(self._c_sog) + + def get_node_outfile_remote_name(self, key): + return cdagvine.sog_get_node_outfile_remote_name(self._c_sog, key) + + def get_proxy_library_name(self): + return cdagvine.sog_get_proxy_library_name(self._c_sog) + + def set_proxy_function_name(self, proxy_function_name): + cdagvine.sog_set_proxy_function_name(self._c_sog, proxy_function_name) + + def execute(self): + cdagvine.sog_execute(self._c_sog) + + def delete(self): + cdagvine.sog_delete(self._c_sog) \ No newline at end of file diff --git a/taskvine/src/graph/dagvine/strategic_orchestration_node.c b/taskvine/src/graph/dagvine/strategic_orchestration_node.c new file mode 100644 index 0000000000..a9ec6209e6 --- /dev/null +++ b/taskvine/src/graph/dagvine/strategic_orchestration_node.c @@ -0,0 +1,608 @@ +#include +#include +#include +#include +#include +#include +#include + +#include "jx.h" +#include "jx_print.h" +#include "xxmalloc.h" +#include "stringtools.h" +#include "timestamp.h" +#include "set.h" +#include "hash_table.h" +#include "debug.h" +#include "random.h" +#include "uuid.h" + +#include "vine_manager.h" +#include "vine_file.h" +#include "vine_task.h" +#include "vine_worker_info.h" +#include "vine_temp.h" +#include "strategic_orchestration_node.h" +#include "taskvine.h" + + +/** + * Compute a lexicographic priority score from the node key. + * Used during topological sorting to break ties deterministically. + * @param key Reference to the node key. + * @return The lexical priority. + */ +double compute_lex_priority(const char *key) +{ + double score = 0.0; + double factor = 1.0; + for (int i = 0; i < 8 && key[i] != '\0'; i++) { + score += (unsigned char)key[i] * factor; + factor *= 0.01; + } + return -score; +} + +/** + * Create a new node object. + * @param manager Reference to the manager object. + * @param node_key Reference to the node key. + * @param is_target_key Reference to whether the node is a target key. + * @param proxy_library_name Reference to the proxy library name. + * @param proxy_function_name Reference to the proxy function name. + * @param staging_dir Reference to the staging directory. + * @param prune_depth Reference to the prune depth. + * @return A new node object. + */ +struct strategic_orchestration_node *son_create( + struct vine_manager *manager, + const char *node_key, + int is_target_key, + const char *proxy_library_name, + const char *proxy_function_name, + const char *target_results_dir, + int prune_depth) +{ + if (!manager) { + debug(D_ERROR, "Cannot create node because manager is NULL"); + return NULL; + } + if (!node_key) { + debug(D_ERROR, "Cannot create node because node_key is NULL"); + return NULL; + } + if (!proxy_library_name) { + debug(D_ERROR, "Cannot create node because proxy_library_name is NULL"); + return NULL; + } + if (!proxy_function_name) { + debug(D_ERROR, "Cannot create node because proxy_function_name is NULL"); + return NULL; + } + if (!target_results_dir) { + debug(D_ERROR, "Cannot create node because target_results_dir is NULL"); + return NULL; + } + if (is_target_key != 0 && is_target_key != 1) { + debug(D_ERROR, "Cannot create node because is_target_key is not 0 or 1"); + return NULL; + } + + struct strategic_orchestration_node *node = xxmalloc(sizeof(struct strategic_orchestration_node)); + + node->manager = manager; + node->is_target_key = is_target_key; + node->node_key = xxstrdup(node_key); + node->target_results_dir = xxstrdup(target_results_dir); + + /* create a unique outfile remote name for the node */ + cctools_uuid_t id; + cctools_uuid_create(&id); + node->outfile_remote_name = xxstrdup(id.str); + + node->prune_status = PRUNE_STATUS_NOT_PRUNED; + node->parents = list_create(); + node->children = list_create(); + node->pending_parents = set_create(0); + node->completed = 0; + node->prune_depth = prune_depth; + node->retry_attempts_left = 1; + node->outfile_size_bytes = 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->time_spent_on_unlink_local_files = 0; + node->time_spent_on_prune_ancestors_of_temp_node = 0; + node->time_spent_on_prune_ancestors_of_persisted_node = 0; + + node->critical_time = -1; + + /* create the task */ + node->task = vine_task_create(proxy_function_name); + vine_task_set_library_required(node->task, proxy_library_name); + vine_task_addref(node->task); + + /* build JSON infile expected by library: {"fn_args": [key], "fn_kwargs": {}} */ + struct jx *event = jx_object(NULL); + struct jx *args = jx_array(NULL); + jx_array_append(args, jx_string(node->node_key)); + jx_insert(event, jx_string("fn_args"), args); + jx_insert(event, jx_string("fn_kwargs"), jx_object(NULL)); + + char *infile_content = jx_print_string(event); + jx_delete(event); + + node->infile = vine_declare_buffer(node->manager, infile_content, strlen(infile_content), VINE_CACHE_LEVEL_TASK, VINE_UNLINK_WHEN_DONE); + free(infile_content); + vine_task_add_input(node->task, node->infile, "infile", VINE_TRANSFER_ALWAYS); + + return node; +} + +/** + * Check if the outfile of a node is persisted. + * A node is considered persisted if it has completed and 1) the outfile is written to the shared file system, + * 2) the outfile is written to the local staging directory. + * @param node Reference to the node object. + * @return 1 if the outfile is persisted, 0 otherwise. + */ +static int _node_outfile_is_persisted(struct strategic_orchestration_node *node) +{ + if (!node) { + return 0; + } + + /* if the node is not completed then the outfile is definitely not persisted */ + if (!node->completed) { + return 0; + } + + switch (node->outfile_type) { + case VINE_NODE_OUTFILE_TYPE_LOCAL: + return 1; + case VINE_NODE_OUTFILE_TYPE_SHARED_FILE_SYSTEM: + return 1; + case VINE_NODE_OUTFILE_TYPE_TEMP: + return 0; + } + + return 0; +} + +/** + * Update the critical time of a node. + * @param node Reference to the node object. + * @param execution_time Reference to the execution time of the node. + */ +void son_update_critical_time(struct strategic_orchestration_node *node, timestamp_t execution_time) +{ + timestamp_t max_parent_critical_time = 0; + struct strategic_orchestration_node *parent_node; + LIST_ITERATE(node->parents, parent_node) + { + if (parent_node->critical_time > max_parent_critical_time) { + max_parent_critical_time = parent_node->critical_time; + } + } + node->critical_time = max_parent_critical_time + execution_time; +} + +/** + * The dfs helper function for finding parents in a specific depth. + * @param node Reference to the node object. + * @param remaining_depth Reference to the remaining depth. + * @param result Reference to the result list. + * @param visited Reference to the visited set. + */ +static void _find_parents_dfs(struct strategic_orchestration_node *node, int remaining_depth, struct list *result, struct set *visited) +{ + if (!node || set_lookup(visited, node)) { + return; + } + + set_insert(visited, node); + if (remaining_depth == 0) { + list_push_tail(result, node); + return; + } + struct strategic_orchestration_node *parent_node; + LIST_ITERATE(node->parents, parent_node) + { + _find_parents_dfs(parent_node, remaining_depth - 1, result, visited); + } +} + +/** + * Find all parents in a specific depth of the node. + * @param node Reference to the node object. + * @param depth Reference to the depth. + * @return The list of parents. + */ +static struct list *_find_parents_in_depth(struct strategic_orchestration_node *node, int depth) +{ + if (!node || depth < 0) { + return NULL; + } + + struct list *result = list_create(); + + struct set *visited = set_create(0); + _find_parents_dfs(node, depth, result, visited); + set_delete(visited); + + return result; +} + +/** + * Prune the ancestors of a temp node. This is only used for temp nodes that produce temp files. + * All ancestors of this node we consider here are temp nodes, we can not safely prune those stored in the shared file system + * because temp nodes are not considered safe enough to trigger the deletion of upstream persisted files. + * @param node Reference to the node object. + * @return The number of pruned replicas. + */ +static int prune_ancestors_of_temp_node(struct strategic_orchestration_node *node) +{ + if (!node || !node->outfile || node->prune_depth <= 0) { + return 0; + } + + timestamp_t start_time = timestamp_get(); + + int pruned_replica_count = 0; + + struct list *parents = _find_parents_in_depth(node, node->prune_depth); + + struct strategic_orchestration_node *parent_node; + LIST_ITERATE(parents, parent_node) + { + /* skip if the parent produces a shared file system file */ + if (!parent_node->outfile) { + continue; + } + /* skip if the parent produces a non-temp file */ + if (parent_node->outfile->type != VINE_TEMP) { + continue; + } + + /* a file is prunable if its outfile is no longer needed by any child node: + * 1. it has no pending dependents + * 2. all completed dependents have also completed their corresponding recovery tasks, if any */ + int all_children_completed = 1; + struct strategic_orchestration_node *child_node; + LIST_ITERATE(parent_node->children, child_node) + { + /* break early if the child node is not completed */ + if (!child_node->completed) { + all_children_completed = 0; + break; + } + /* if the task produces a temp file and the recovery task is running, the parent is not prunable */ + if (child_node->outfile && child_node->outfile->type == VINE_TEMP) { + struct vine_task *child_node_recovery_task = child_node->outfile->recovery_task; + if (child_node_recovery_task && (child_node_recovery_task->state != VINE_TASK_INITIAL && child_node_recovery_task->state != VINE_TASK_DONE)) { + all_children_completed = 0; + break; + } + } + } + if (!all_children_completed) { + continue; + } + + pruned_replica_count += vine_prune_file(node->manager, parent_node->outfile); + /* this parent is pruned because a successor that produces a temp file is completed, it is unsafe because the + * manager may submit a recovery task to bring it back in case of worker failures. */ + parent_node->prune_status = PRUNE_STATUS_UNSAFE; + } + + list_delete(parents); + + node->time_spent_on_prune_ancestors_of_temp_node += timestamp_get() - start_time; + + return pruned_replica_count; +} + +/** + * Find all safe ancestors of a node. + * @param start_node Reference to the start node. + * @return The set of safe ancestors. + */ +static struct set *_find_safe_ancestors(struct strategic_orchestration_node *start_node) +{ + if (!start_node) { + return NULL; + } + + struct set *visited_nodes = set_create(0); + struct list *bfs_nodes = list_create(); + + list_push_tail(bfs_nodes, start_node); + set_insert(visited_nodes, start_node); + + while (list_size(bfs_nodes) > 0) { + struct strategic_orchestration_node *current = list_pop_head(bfs_nodes); + + struct strategic_orchestration_node *parent_node; + LIST_ITERATE(current->parents, parent_node) + { + if (set_lookup(visited_nodes, parent_node)) { + continue; + } + + /* shortcut if this parent was already safely pruned */ + if (parent_node->prune_status == PRUNE_STATUS_SAFE) { + continue; + } + + /* check if all children are safe */ + int all_children_safe = 1; + struct strategic_orchestration_node *child_node; + LIST_ITERATE(parent_node->children, child_node) + { + /* shortcut if this child is part of the recovery subgraph */ + if (set_lookup(visited_nodes, child_node)) { + continue; + } + /* shortcut if this outside child is not persisted */ + if (!_node_outfile_is_persisted(child_node)) { + all_children_safe = 0; + break; + } + /* shortcut if this outside child is unsafely pruned */ + if (child_node->prune_status == PRUNE_STATUS_UNSAFE) { + all_children_safe = 0; + break; + } + } + + if (!all_children_safe) { + continue; + } + + set_insert(visited_nodes, parent_node); + list_push_tail(bfs_nodes, parent_node); + } + } + + list_delete(bfs_nodes); + set_remove(visited_nodes, start_node); + + return visited_nodes; +} + +/** + * Prune the ancestors of a persisted node. This is only used for persisted nodes that produce persisted files. + * All ancestors we consider here include both temp nodes and persisted nodes, becasue data written to shared file system + * is safe and can definitely trigger upstream data redundancy to be released. + * @param node Reference to the node object. + * @return The number of pruned replicas. + */ +static int prune_ancestors_of_persisted_node(struct strategic_orchestration_node *node) +{ + if (!node) { + return 0; + } + + timestamp_t start_time = timestamp_get(); + + int pruned_replica_count = 0; + + /* find all safe ancestors */ + struct set *safe_ancestors = _find_safe_ancestors(node); + if (!safe_ancestors) { + return 0; + } + + /* prune all safe ancestors */ + struct strategic_orchestration_node *ancestor_node; + SET_ITERATE(safe_ancestors, ancestor_node) + { + /* unlink the shared file system file */ + if (!ancestor_node->outfile) { + timestamp_t unlink_start = timestamp_get(); + if (ancestor_node->outfile_remote_name) { + unlink(ancestor_node->outfile_remote_name); // system call + } + node->time_spent_on_unlink_local_files += timestamp_get() - unlink_start; + debug(D_VINE, "unlinked %s size: %zu bytes, time: %" PRIu64, ancestor_node->outfile_remote_name ? ancestor_node->outfile_remote_name : "(null)", ancestor_node->outfile_size_bytes, (uint64_t)node->time_spent_on_unlink_local_files); + } else { + switch (ancestor_node->outfile->type) { + case VINE_FILE: + /* do not prune the staging dir file */ + break; + case VINE_TEMP: + /* prune the temp file */ + vine_prune_file(node->manager, ancestor_node->outfile); + break; + default: + debug(D_ERROR, "unsupported outfile type: %d", ancestor_node->outfile->type); + break; + } + } + ancestor_node->prune_status = PRUNE_STATUS_SAFE; + pruned_replica_count++; + } + + set_delete(safe_ancestors); + + node->time_spent_on_prune_ancestors_of_persisted_node += timestamp_get() - start_time; + + return pruned_replica_count; +} + +/** + * Print the info of the node. + * @param node Reference to the node object. + */ +void son_print_info(struct strategic_orchestration_node *node) +{ + if (!node) { + return; + } + + if (!node->task) { + debug(D_ERROR, "node %s has no task", node->node_key); + return; + } + + debug(D_VINE, "---------------- Node Info ----------------"); + debug(D_VINE, "key: %s", node->node_key); + debug(D_VINE, "task_id: %d", node->task->task_id); + debug(D_VINE, "target_results_dir: %s", node->target_results_dir ? node->target_results_dir : "(null)"); + debug(D_VINE, "depth: %d", node->depth); + debug(D_VINE, "height: %d", node->height); + debug(D_VINE, "prune_depth: %d", node->prune_depth); + + 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"); + } + + /* print parent and child node keys */ + char *parent_keys = NULL; + struct strategic_orchestration_node *p; + LIST_ITERATE(node->parents, p) + { + if (!parent_keys) { + parent_keys = string_format("%s", p->node_key); + } else { + char *tmp = string_format("%s, %s", parent_keys, p->node_key); + free(parent_keys); + parent_keys = tmp; + } + } + + char *child_keys = NULL; + struct strategic_orchestration_node *c; + LIST_ITERATE(node->children, c) + { + if (!child_keys) { + child_keys = string_format("%s", c->node_key); + } else { + char *tmp = string_format("%s, %s", child_keys, c->node_key); + free(child_keys); + child_keys = tmp; + } + } + + debug(D_VINE, "parents: %s", parent_keys ? parent_keys : "(none)"); + debug(D_VINE, "children: %s", child_keys ? child_keys : "(none)"); + + free(parent_keys); + free(child_keys); + + debug(D_VINE, "-------------------------------------------"); +} + +/** + * Prune the ancestors of a node when it is completed. + * @param node Reference to the node object. + */ +void son_prune_ancestors(struct strategic_orchestration_node *node) +{ + if (!node) { + return; + } + + timestamp_t start_time = timestamp_get(); + + int pruned_replica_count = 0; + + if (_node_outfile_is_persisted(node)) { + pruned_replica_count = prune_ancestors_of_persisted_node(node); + } else { + pruned_replica_count = prune_ancestors_of_temp_node(node); + } + + timestamp_t elapsed_time = timestamp_get() - start_time; + + debug(D_VINE, "pruned %d ancestors of node %s in %.6f seconds", pruned_replica_count, node->node_key, elapsed_time / 1000000.0); +} + +/** + * Replicate the outfile of a node if it is a temp file. + * @param node Reference to the node object. + */ +void son_replicate_outfile(struct strategic_orchestration_node *node) +{ + if (!node || !node->outfile) { + return; + } + + if (node->outfile->type != VINE_TEMP) { + return; + } + + vine_temp_replicate_file_later(node->manager, node->outfile); +} + +/** + * Delete the node and all of its associated resources. + * @param node Reference to the node object. + */ +void son_delete(struct strategic_orchestration_node *node) +{ + if (!node) { + return; + } + + if (node->node_key) { + free(node->node_key); + } + if (node->outfile_remote_name) { + free(node->outfile_remote_name); + } + if (node->target_results_dir) { + free(node->target_results_dir); + } + + vine_task_delete(node->task); + node->task = NULL; + + if (node->infile) { + vine_file_delete(node->infile); + node->infile = NULL; + } + if (node->outfile) { + vine_file_delete(node->outfile); + node->outfile = NULL; + } + + list_delete(node->parents); + list_delete(node->children); + + if (node->pending_parents) { + set_delete(node->pending_parents); + } + free(node); +} \ No newline at end of file diff --git a/taskvine/src/graph/dagvine/strategic_orchestration_node.h b/taskvine/src/graph/dagvine/strategic_orchestration_node.h new file mode 100644 index 0000000000..ac5bc70ec0 --- /dev/null +++ b/taskvine/src/graph/dagvine/strategic_orchestration_node.h @@ -0,0 +1,92 @@ +#ifndef STRATEGIC_ORCHESTRATION_NODE_H +#define STRATEGIC_ORCHESTRATION_NODE_H + +#include "vine_task.h" +#include "hash_table.h" +#include "list.h" +#include "vine_manager.h" +#include "set.h" +#include "taskvine.h" + +/** Select the type of the node-output file. */ +typedef enum { + VINE_NODE_OUTFILE_TYPE_LOCAL = 0, /* Node-output file will be stored locally on the manager's staging directory */ + VINE_NODE_OUTFILE_TYPE_TEMP, /* Node-output file will be stored in the temporary node-local storage */ + VINE_NODE_OUTFILE_TYPE_SHARED_FILE_SYSTEM, /* Node-output file will be stored in the persistent shared file system */ +} node_outfile_type_t; + +typedef enum { + PRUNE_STATUS_NOT_PRUNED = 0, + PRUNE_STATUS_SAFE, + PRUNE_STATUS_UNSAFE +} prune_status_t; + +struct strategic_orchestration_node { + char *node_key; + + /* Whether the node is a target key. + * If true, the output of the node will be declared as a vine_file and retrieved through the network. */ + int is_target_key; + + struct vine_manager *manager; + struct vine_task *task; + struct vine_file *infile; + struct vine_file *outfile; + char *outfile_remote_name; + char *target_results_dir; + size_t outfile_size_bytes; + + struct list *parents; + struct list *children; + struct set *pending_parents; + + int retry_attempts_left; + int completed; + int prune_depth; + + int depth; + int height; + int upstream_subgraph_size; + int downstream_subgraph_size; + int fan_in; + int fan_out; + double heavy_score; + + timestamp_t critical_time; + timestamp_t time_spent_on_unlink_local_files; + timestamp_t time_spent_on_prune_ancestors_of_temp_node; + timestamp_t time_spent_on_prune_ancestors_of_persisted_node; + + node_outfile_type_t outfile_type; + prune_status_t prune_status; +}; + +/** Create a new strategic orchestration node object. +@param manager Reference to the manager object. +@param node_key Reference to the node key. +@param is_target_key Reference to whether the node is a target key. +@param proxy_library_name Reference to the proxy library name. +@param proxy_function_name Reference to the proxy function name. +@param target_results_dir Reference to the target results directory. +@param prune_depth Reference to the prune depth. +@return A new strategic orchestration node object. +*/ +struct strategic_orchestration_node *son_create( + struct vine_manager *manager, + const char *node_key, + int is_target_key, + const char *proxy_library_name, + const char *proxy_function_name, + const char *target_results_dir, + int prune_depth +); + +double compute_lex_priority(const char *key); + +void son_delete(struct strategic_orchestration_node *node); +void son_prune_ancestors(struct strategic_orchestration_node *node); +void son_print_info(struct strategic_orchestration_node *node); +void son_update_critical_time(struct strategic_orchestration_node *node, timestamp_t execution_time); +void son_replicate_outfile(struct strategic_orchestration_node *node); + +#endif \ No newline at end of file From 90476b8e2ebacc3d47366a9218251cf24e703fdd Mon Sep 17 00:00:00 2001 From: Jin Zhou Date: Sun, 19 Oct 2025 14:45:08 -0400 Subject: [PATCH 048/113] lint and format --- taskvine/src/bindings/python3/Makefile | 3 +- .../python3/ndcctools/taskvine/utils.py | 9 - taskvine/src/graph/dagvine/Makefile | 23 +- taskvine/src/graph/dagvine/cvine_dagvine.py | 107 -------- taskvine/src/graph/dagvine/graph_executor.py | 255 ------------------ taskvine/src/graph/dagvine/params.py | 2 +- taskvine/src/graph/dagvine/proxy_functions.py | 9 +- taskvine/src/graph/dagvine/proxy_library.py | 13 +- .../graph/dagvine/runtime_execution_graph.py | 7 +- .../dagvine/strategic_orchestration_graph.c | 31 ++- .../dagvine/strategic_orchestration_graph.h | 41 ++- .../dagvine/strategic_orchestration_graph.i | 8 +- .../dagvine/strategic_orchestration_graph.py | 24 +- .../dagvine/strategic_orchestration_node.c | 18 -- .../dagvine/strategic_orchestration_node.h | 123 +++++---- taskvine/src/graph/dagvine/utils.py | 17 -- 16 files changed, 166 insertions(+), 524 deletions(-) delete mode 100644 taskvine/src/graph/dagvine/cvine_dagvine.py delete mode 100644 taskvine/src/graph/dagvine/graph_executor.py delete mode 100644 taskvine/src/graph/dagvine/utils.py diff --git a/taskvine/src/bindings/python3/Makefile b/taskvine/src/bindings/python3/Makefile index 3289a4da46..16d9cd5ceb 100644 --- a/taskvine/src/bindings/python3/Makefile +++ b/taskvine/src/bindings/python3/Makefile @@ -40,10 +40,9 @@ install: all mkdir -p $(CCTOOLS_PYTHON3_PATH)/ndcctools/taskvine/${GRAPH_EXECUTOR_MODULE_NAME} cp $(CCTOOLS_HOME)/taskvine/src/graph/${GRAPH_EXECUTOR_MODULE_NAME}/runtime_execution_graph.py $(CCTOOLS_PYTHON3_PATH)/ndcctools/taskvine/${GRAPH_EXECUTOR_MODULE_NAME}/runtime_execution_graph.py - cp $(CCTOOLS_HOME)/taskvine/src/graph/${GRAPH_EXECUTOR_MODULE_NAME}/graph_executor.py $(CCTOOLS_PYTHON3_PATH)/ndcctools/taskvine/${GRAPH_EXECUTOR_MODULE_NAME}/graph_executor.py + cp $(CCTOOLS_HOME)/taskvine/src/graph/${GRAPH_EXECUTOR_MODULE_NAME}/executor.py $(CCTOOLS_PYTHON3_PATH)/ndcctools/taskvine/${GRAPH_EXECUTOR_MODULE_NAME}/executor.py cp $(CCTOOLS_HOME)/taskvine/src/graph/${GRAPH_EXECUTOR_MODULE_NAME}/proxy_library.py $(CCTOOLS_PYTHON3_PATH)/ndcctools/taskvine/${GRAPH_EXECUTOR_MODULE_NAME}/proxy_library.py cp $(CCTOOLS_HOME)/taskvine/src/graph/${GRAPH_EXECUTOR_MODULE_NAME}/proxy_functions.py $(CCTOOLS_PYTHON3_PATH)/ndcctools/taskvine/${GRAPH_EXECUTOR_MODULE_NAME}/proxy_functions.py cp $(CCTOOLS_HOME)/taskvine/src/graph/${GRAPH_EXECUTOR_MODULE_NAME}/strategic_orchestration_graph.py $(CCTOOLS_PYTHON3_PATH)/ndcctools/taskvine/${GRAPH_EXECUTOR_MODULE_NAME}/strategic_orchestration_graph.py cp $(CCTOOLS_HOME)/taskvine/src/graph/${GRAPH_EXECUTOR_MODULE_NAME}/params.py $(CCTOOLS_PYTHON3_PATH)/ndcctools/taskvine/${GRAPH_EXECUTOR_MODULE_NAME}/params.py - cp $(CCTOOLS_HOME)/taskvine/src/graph/${GRAPH_EXECUTOR_MODULE_NAME}/utils.py $(CCTOOLS_PYTHON3_PATH)/ndcctools/taskvine/${GRAPH_EXECUTOR_MODULE_NAME}/utils.py @echo "# ${GRAPH_EXECUTOR_MODULE_NAME} package" > $(CCTOOLS_PYTHON3_PATH)/ndcctools/taskvine/${GRAPH_EXECUTOR_MODULE_NAME}/__init__.py diff --git a/taskvine/src/bindings/python3/ndcctools/taskvine/utils.py b/taskvine/src/bindings/python3/ndcctools/taskvine/utils.py index e493e7c04a..166b2800f2 100644 --- a/taskvine/src/bindings/python3/ndcctools/taskvine/utils.py +++ b/taskvine/src/bindings/python3/ndcctools/taskvine/utils.py @@ -3,10 +3,6 @@ # See the file COPYING for details. from . import cvine -try: - from .dagvine import cdagvine -except Exception: - cdagvine = None import os @@ -20,11 +16,6 @@ def get_c_constant(constant): value = getattr(cvine, constant, None) if value is not None: return value - # Fallback to cdagvine if available (for DAG-specific constants) - if cdagvine is not None: - value = getattr(cdagvine, constant, None) - if value is not None: - return value # If still missing, raise a clear error raise AttributeError(f"C constant {constant} not found in cvine or cdagvine") diff --git a/taskvine/src/graph/dagvine/Makefile b/taskvine/src/graph/dagvine/Makefile index c2e3a38236..78eb2b4d7e 100644 --- a/taskvine/src/graph/dagvine/Makefile +++ b/taskvine/src/graph/dagvine/Makefile @@ -4,7 +4,9 @@ include ../../../../rules.mk LOCAL_LINKAGE+=${CCTOOLS_HOME}/taskvine/src/manager/libtaskvine.a ${CCTOOLS_HOME}/dttools/src/libdttools.a LOCAL_CCFLAGS+=-I ${CCTOOLS_HOME}/taskvine/src/manager - + +DAGVINE_SOURCE_DIR = $(CCTOOLS_HOME)/taskvine/src/graph/dagvine + SOURCES = strategic_orchestration_node.c strategic_orchestration_graph.c OBJECTS = $(SOURCES:%.c=%.o) @@ -15,7 +17,7 @@ SWIG_I = strategic_orchestration_graph.i BUILD_DIR := build SWIG_WRAP = $(BUILD_DIR)/strategic_orchestration_graph_wrap.c WRAP_OBJ = $(BUILD_DIR)/strategic_orchestration_graph_wrap.o -PYMODULE = $(BUILD_DIR)/_cdagvine.$(CCTOOLS_DYNAMIC_SUFFIX) +PYMODULE = $(BUILD_DIR)/_capi.$(CCTOOLS_DYNAMIC_SUFFIX) LIBRARIES = PYDEPS = $(WRAP_OBJ) $(OBJECTS) @@ -54,18 +56,27 @@ install: all cp ${CCTOOLS_HOME}/taskvine/src/manager/taskvine.h $(CCTOOLS_INSTALL_DIR)/graph/dagvine/include/ mkdir -p $(CCTOOLS_PYTHON3_PATH)/ndcctools/taskvine/dagvine cp $(PYMODULE) $(CCTOOLS_PYTHON3_PATH)/ndcctools/taskvine/dagvine/ - cp $(BUILD_DIR)/cdagvine.py $(CCTOOLS_PYTHON3_PATH)/ndcctools/taskvine/dagvine/ - # remove build artifacts after installation - rm -rf $(BUILD_DIR) + cp $(BUILD_DIR)/capi.py $(CCTOOLS_PYTHON3_PATH)/ndcctools/taskvine/dagvine/ clean: - rm -rf $(PROGRAMS) $(OBJECTS) *.o $(BUILD_DIR) $(PYMODULE) cdagvine.py + rm -rf $(PROGRAMS) $(OBJECTS) *.o $(BUILD_DIR) $(PYMODULE) capi.py test: all lint: + if ( ! clang-format -Werror --dry-run --style='file:${CCTOOLS_HOME}/.clang-format' $(DAGVINE_SOURCE_DIR)/*.c $(DAGVINE_SOURCE_DIR)/*.h);\ + then\ + echo "========================================================";\ + echo "NOTICE: Run \`make format\` to format your latest changes.";\ + echo "========================================================";\ + exit 1;\ + fi + flake8 --ignore=$(CCTOOLS_FLAKE8_IGNORE_ERRORS) \ + --exclude=$(CCTOOLS_FLAKE8_IGNORE_FILES),$(BUILD_DIR) \ + $(DAGVINE_SOURCE_DIR)/ format: + clang-format -i $(DAGVINE_SOURCE_DIR)/*.c $(DAGVINE_SOURCE_DIR)/*.h .PHONY: all clean install test lint format diff --git a/taskvine/src/graph/dagvine/cvine_dagvine.py b/taskvine/src/graph/dagvine/cvine_dagvine.py deleted file mode 100644 index df154836f1..0000000000 --- a/taskvine/src/graph/dagvine/cvine_dagvine.py +++ /dev/null @@ -1,107 +0,0 @@ -# This file was automatically generated by SWIG (https://www.swig.org). -# Version 4.3.1 -# -# Do not make changes to this file unless you know what you are doing - modify -# the SWIG interface file instead. - -from sys import version_info as _swig_python_version_info -# Import the low-level C/C++ module -if __package__ or "." in __name__: - from . import _cvine_dagvine -else: - import _cvine_dagvine - -try: - import builtins as __builtin__ -except ImportError: - import __builtin__ - -def _swig_repr(self): - try: - strthis = "proxy of " + self.this.__repr__() - except __builtin__.Exception: - strthis = "" - return "<%s.%s; %s >" % (self.__class__.__module__, self.__class__.__name__, strthis,) - - -def _swig_setattr_nondynamic_instance_variable(set): - def set_instance_attr(self, name, value): - if name == "this": - set(self, name, value) - elif name == "thisown": - self.this.own(value) - elif hasattr(self, name) and isinstance(getattr(type(self), name), property): - set(self, name, value) - else: - raise AttributeError("You cannot add instance attributes to %s" % self) - return set_instance_attr - - -def _swig_setattr_nondynamic_class_variable(set): - def set_class_attr(cls, name, value): - if hasattr(cls, name) and not isinstance(getattr(cls, name), property): - set(cls, name, value) - else: - raise AttributeError("You cannot add class attributes to %s" % cls) - return set_class_attr - - -def _swig_add_metaclass(metaclass): - """Class decorator for adding a metaclass to a SWIG wrapped class - a slimmed down version of six.add_metaclass""" - def wrapper(cls): - return metaclass(cls.__name__, cls.__bases__, cls.__dict__.copy()) - return wrapper - - -class _SwigNonDynamicMeta(type): - """Meta class to enforce nondynamic attributes (no new attributes) for a class""" - __setattr__ = _swig_setattr_nondynamic_class_variable(type.__setattr__) - - -import cvine -class vine_task_graph(object): - thisown = property(lambda x: x.this.own(), lambda x, v: x.this.own(v), doc="The membership flag") - __repr__ = _swig_repr - manager = property(_cvine_dagvine.vine_task_graph_manager_get, _cvine_dagvine.vine_task_graph_manager_set) - nodes = property(_cvine_dagvine.vine_task_graph_nodes_get, _cvine_dagvine.vine_task_graph_nodes_set) - task_id_to_node = property(_cvine_dagvine.vine_task_graph_task_id_to_node_get, _cvine_dagvine.vine_task_graph_task_id_to_node_set) - outfile_cachename_to_node = property(_cvine_dagvine.vine_task_graph_outfile_cachename_to_node_get, _cvine_dagvine.vine_task_graph_outfile_cachename_to_node_set) - proxy_library_name = property(_cvine_dagvine.vine_task_graph_proxy_library_name_get, _cvine_dagvine.vine_task_graph_proxy_library_name_set) - proxy_function_name = property(_cvine_dagvine.vine_task_graph_proxy_function_name_get, _cvine_dagvine.vine_task_graph_proxy_function_name_set) - task_priority_mode = property(_cvine_dagvine.vine_task_graph_task_priority_mode_get, _cvine_dagvine.vine_task_graph_task_priority_mode_set) - failure_injection_step_percent = property(_cvine_dagvine.vine_task_graph_failure_injection_step_percent_get, _cvine_dagvine.vine_task_graph_failure_injection_step_percent_set) - - def __init__(self): - _cvine_dagvine.vine_task_graph_swiginit(self, _cvine_dagvine.new_vine_task_graph()) - __swig_destroy__ = _cvine_dagvine.delete_vine_task_graph - -# Register vine_task_graph in _cvine_dagvine: -_cvine_dagvine.vine_task_graph_swigregister(vine_task_graph) - -def vine_task_graph_create(q): - return _cvine_dagvine.vine_task_graph_create(q) - -def vine_task_graph_set_priority_mode(tg, priority_mode): - return _cvine_dagvine.vine_task_graph_set_priority_mode(tg, priority_mode) - -def vine_task_graph_set_failure_injection_step_percent(tg, percent): - return _cvine_dagvine.vine_task_graph_set_failure_injection_step_percent(tg, percent) - -def vine_task_graph_add_node(tg, node_key, staging_dir, prune_depth): - return _cvine_dagvine.vine_task_graph_add_node(tg, node_key, staging_dir, prune_depth) - -def vine_task_graph_add_dependency(tg, parent_key, child_key): - return _cvine_dagvine.vine_task_graph_add_dependency(tg, parent_key, child_key) - -def vine_task_graph_compute_topology_metrics(tg): - return _cvine_dagvine.vine_task_graph_compute_topology_metrics(tg) - -def vine_task_graph_get_node_heavy_score(tg, node_key): - return _cvine_dagvine.vine_task_graph_get_node_heavy_score(tg, node_key) - -def vine_task_graph_set_node_outfile(tg, node_key, outfile_type, outfile_remote_name): - return _cvine_dagvine.vine_task_graph_set_node_outfile(tg, node_key, outfile_type, outfile_remote_name) - -def vine_task_graph_set_proxy_library_and_function_names(tg, proxy_library_name, proxy_function_name): - return _cvine_dagvine.vine_task_graph_set_proxy_library_and_function_names(tg, proxy_library_name, proxy_function_name) - diff --git a/taskvine/src/graph/dagvine/graph_executor.py b/taskvine/src/graph/dagvine/graph_executor.py deleted file mode 100644 index 163992ba68..0000000000 --- a/taskvine/src/graph/dagvine/graph_executor.py +++ /dev/null @@ -1,255 +0,0 @@ - -from ndcctools.taskvine import cvine -from ndcctools.taskvine.manager import Manager -from ndcctools.taskvine.utils import delete_all_files - -from ndcctools.taskvine.dagvine.proxy_library import ProxyLibrary -from ndcctools.taskvine.dagvine.runtime_execution_graph import RuntimeExecutionGraph, GraphKeyResult -from ndcctools.taskvine.dagvine.strategic_orchestration_graph import StrategicOrchestrationGraph - -import cloudpickle -import os -import signal - -try: - import dask -except ImportError: - dask = None - -try: - from dask.base import is_dask_collection -except ImportError: - is_dask_collection = None - -try: - import dask._task_spec as dts -except ImportError: - dts = None - - -# convert Dask collection to task dictionary -def dask_collections_to_task_dict(collection_dict): - assert is_dask_collection is not None - from dask.highlevelgraph import HighLevelGraph, ensure_dict - - if not isinstance(collection_dict, dict): - raise TypeError("Input must be a dict or a HighLevelGraph") - - for k, v in collection_dict.items(): - if not is_dask_collection(v): - raise TypeError(f"Input must be a dict of DaskCollection, but found {k} with type {type(v)}") - - if dts: - sub_hlgs = [v.dask for v in collection_dict.values()] - hlg = HighLevelGraph.merge(*sub_hlgs).to_dict() - else: - hlg = dask.base.collections_to_dsk(collection_dict.values()) - - return ensure_dict(hlg) - - -# compatibility for Dask-created collections -def ensure_task_dict(collection_dict): - if is_dask_collection and any(is_dask_collection(v) for v in collection_dict.values()): - task_dict = dask_collections_to_task_dict(collection_dict) - else: - task_dict = collection_dict - - if dts: - return dts.convert_legacy_graph(task_dict) - else: - return task_dict - - -class GraphParams: - def __init__(self): - self.vine_manager_tuning_params = { - "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, - "balance-worker-disk-load": 0, - } - self.sog_tuning_params = { - "failure-injection-step-percent": -1, - "priority-mode": "largest-input-first", - "prune-depth": 1, - "target-results-dir": "./target_results", - "checkpoint-fraction": 0, - } - self.other_params = { - "schedule": "worst", - "libcores": 16, - "failure-injection-step-percent": -1, - "shared-file-system-dir": "./shared_file_system", - "extra-task-output-size-mb": ["uniform", 0, 0], - "extra-task-sleep-time": ["uniform", 0, 0], - "outfile-type": { - "temp": 1.0, - "shared-file-system": 0.0, - }, - } - - def update_param(self, param_name, new_value): - if param_name in self.vine_manager_tuning_params: - self.vine_manager_tuning_params[param_name] = new_value - elif param_name in self.sog_tuning_params: - self.sog_tuning_params[param_name] = new_value - elif param_name in self.other_params: - self.other_params[param_name] = new_value - else: - self.vine_manager_tuning_params[param_name] = new_value - - def get_value_of(self, param_name): - if param_name in self.vine_manager_tuning_params: - return self.vine_manager_tuning_params[param_name] - elif param_name in self.sog_tuning_params: - return self.sog_tuning_params[param_name] - elif param_name in self.other_params: - return self.other_params[param_name] - else: - raise ValueError(f"Invalid param name: {param_name}") - - -class GraphExecutor(Manager): - def __init__(self, - *args, - **kwargs): - - # handle SIGINT correctly - signal.signal(signal.SIGINT, self._on_sigint) - - self.params = GraphParams() - - # delete all files in the run info directory, do this before super().__init__() - 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: - delete_all_files(self.run_info_template_path) - - # initialize the manager - super().__init__(*args, **kwargs) - - self.runtime_directory = cvine.vine_get_runtime_directory(self._taskvine) - - def tune_manager(self): - for k, v in self.manager_tuning_params.to_dict().items(): - print(f"Tuning {k} to {v}") - self.tune(k, v) - - # set worker scheduling algorithm - cvine.vine_set_scheduler(self._taskvine, self.vine_constant_params.get_c_constant_of("schedule")) - - def param(self, param_name): - return self.params.get_value_of(param_name) - - def update_params(self, new_params): - 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 get_run_info_path(self): - return os.path.join(self.param("run-info-path"), self.param("run-info-template")) - - def tune_vine_manager(self): - for k, v in self.params.vine_manager_tuning_params.items(): - print(f"Tuning {k} to {v}") - self.tune(k, v) - - def tune_sog(self, sog): - for k, v in self.params.sog_tuning_params.items(): - print(f"Tuning {k} to {v}") - sog.tune(k, str(v)) - - def build_reg(self): - reg = RuntimeExecutionGraph( - self.task_dict, - shared_file_system_dir=self.param("shared-file-system-dir"), - extra_task_output_size_mb=self.param("extra-task-output-size-mb"), - extra_task_sleep_time=self.param("extra-task-sleep-time") - ) - - return reg - - def build_sog(self, reg, target_keys): - assert reg is not None, "Python graph must be built before building the C graph" - - sog = StrategicOrchestrationGraph(self._taskvine) - - sog.set_proxy_function_name("compute_single_key") - - # C side vine task graph must be tuned before adding nodes and dependencies - self.tune_vine_manager() - self.tune_sog(sog) - - topo_order = reg.get_topological_order() - for k in topo_order: - sog.add_node(reg.sog_node_key_of[k], int(k in target_keys)) - for pk in reg.parents_of[k]: #!!!!!!!!!!!!!!!!!!!!!!! TODO: fix sog-reg key mapping - sog.add_dependency(reg.sog_node_key_of[pk], reg.sog_node_key_of[k]) - - sog.compute_topology_metrics() - - return sog - - def build_graphs(self, target_keys): - # build Python DAG (logical topology) - reg = self.build_reg() - # build C DAG (physical topology) - sog = self.build_sog(reg, target_keys) - - # set outfile remote names in reg from sog, note that these names are automatically generated - # with regard to the checkpointing strategy and the shared file system directory - for sog_node_key in reg.sog_node_key_of.values(): - outfile_remote_name = sog.get_node_outfile_remote_name(sog_node_key) - reg.set_outfile_remote_name_of(reg.reg_node_key_of[sog_node_key], outfile_remote_name) - - return reg, sog - - def create_proxy_library(self, reg, sog, hoisting_modules, env_files): - proxy_library = ProxyLibrary(self) - proxy_library.add_hoisting_modules(hoisting_modules) - proxy_library.add_env_files(env_files) - proxy_library.set_context_loader(RuntimeExecutionGraph.context_loader_func, context_loader_args=[cloudpickle.dumps(reg)]) - proxy_library.set_libcores(self.param("libcores")) - proxy_library.set_name(sog.get_proxy_library_name()) - proxy_library.install() - - return proxy_library - - def run(self, collection_dict, target_keys=[], params={}, hoisting_modules=[], env_files={}): - # first update the params so that they can be used for the following construction - self.update_params(params) - - self.task_dict = ensure_task_dict(collection_dict) - - # build graphs from both sides - reg, sog = self.build_graphs(target_keys) - - # create and install the proxy library on the manager - proxy_library = self.create_proxy_library(reg, sog, hoisting_modules, env_files) - - # execute the graph on the C side - print(f"Executing task graph, logs will be written to {self.runtime_directory}") - sog.execute() - - # clean up the library instances and template on the manager - proxy_library.uninstall() - - # delete the C graph immediately after execution, so that the lifetime of this object is limited to the execution - sog.delete() - - # load results of target keys - results = {} - for k in target_keys: - outfile_path = os.path.join(self.param("target-results-dir"), reg.outfile_remote_name[k]) - results[k] = GraphKeyResult.load_from_path(outfile_path) - return results - - def _on_sigint(self, signum, frame): - self.__del__() - diff --git a/taskvine/src/graph/dagvine/params.py b/taskvine/src/graph/dagvine/params.py index 41a81e23fc..d513f8b2ac 100644 --- a/taskvine/src/graph/dagvine/params.py +++ b/taskvine/src/graph/dagvine/params.py @@ -4,7 +4,7 @@ @dataclass class ManagerTuningParams: - """These parameters are used to tune the manager at the C level + """These parameters are used to tune the manager at the C level and should comply with the TaskVine manager API.""" worker_source_max_transfers: int = 100 max_retrievals: int = -1 diff --git a/taskvine/src/graph/dagvine/proxy_functions.py b/taskvine/src/graph/dagvine/proxy_functions.py index c415cf9d40..e79243f987 100644 --- a/taskvine/src/graph/dagvine/proxy_functions.py +++ b/taskvine/src/graph/dagvine/proxy_functions.py @@ -1,4 +1,9 @@ -import os, time +# 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 time from ndcctools.taskvine.utils import load_variable_from_library @@ -87,7 +92,7 @@ def compute_single_key(vine_key): """ reg = load_variable_from_library('reg') - k = reg.reg_node_key_of[vine_key] + k = reg.sog_key_to_reg_key[vine_key] v = reg.task_dict[k] if reg.is_dts_key(k): diff --git a/taskvine/src/graph/dagvine/proxy_library.py b/taskvine/src/graph/dagvine/proxy_library.py index aa14b4a607..01e9bda3be 100644 --- a/taskvine/src/graph/dagvine/proxy_library.py +++ b/taskvine/src/graph/dagvine/proxy_library.py @@ -1,4 +1,15 @@ -import os, uuid, cloudpickle, types, time, random, hashlib, collections +# 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 ndcctools.taskvine.dagvine.runtime_execution_graph import ( GraphKeyResult, RuntimeExecutionGraph, hash_name, hashable ) diff --git a/taskvine/src/graph/dagvine/runtime_execution_graph.py b/taskvine/src/graph/dagvine/runtime_execution_graph.py index 4e4f7c81b5..bc5b8049f4 100644 --- a/taskvine/src/graph/dagvine/runtime_execution_graph.py +++ b/taskvine/src/graph/dagvine/runtime_execution_graph.py @@ -4,10 +4,8 @@ import os import hashlib -import time import cloudpickle import collections -import uuid import random from collections import deque @@ -125,8 +123,8 @@ def __init__(self, task_dict, self.parents_of, self.children_of = self._build_dependencies(self.task_dict) self.depth_of = self._calculate_depths() - self.sog_node_key_of = {k: hash_name(k) for k in task_dict.keys()} - self.reg_node_key_of = {hash_name(k): k for k in task_dict.keys()} + self.reg_key_to_sog_key = {k: hash_name(k) for k in task_dict.keys()} + self.sog_key_to_reg_key = {hash_name(k): k for k in task_dict.keys()} # will be set from sog self.outfile_remote_name = {key: None for key in self.task_dict.keys()} @@ -264,4 +262,3 @@ def context_loader_func(reg_pkl): return { "reg": reg, } - diff --git a/taskvine/src/graph/dagvine/strategic_orchestration_graph.c b/taskvine/src/graph/dagvine/strategic_orchestration_graph.c index 45a6e2246b..464cd2a969 100644 --- a/taskvine/src/graph/dagvine/strategic_orchestration_graph.c +++ b/taskvine/src/graph/dagvine/strategic_orchestration_graph.c @@ -31,7 +31,6 @@ #include "vine_mount.h" #include "taskvine.h" - static volatile sig_atomic_t interrupted = 0; /*************************************************************/ @@ -47,6 +46,23 @@ static void handle_sigint(int signal) interrupted = 1; } +/** + * Compute a lexicographic priority score from the node key. + * Used during topological sorting to break ties deterministically. + * @param key Reference to the node key. + * @return The lexical priority. + */ +static double compute_lex_priority(const char *key) +{ + double score = 0.0; + double factor = 1.0; + for (int i = 0; i < 8 && key[i] != '\0'; i++) { + score += (unsigned char)key[i] * factor; + factor *= 0.01; + } + return -score; +} + /** * Calculate the priority of a node given the priority mode. * @param node Reference to the node object. @@ -141,7 +157,7 @@ static void submit_unblocked_children(struct strategic_orchestration_graph *sog, return; } - struct strategic_orchestration_node *child_node; + struct strategic_orchestration_node *child_node; LIST_ITERATE(node->children, child_node) { /* Remove this parent from the child's pending set if it exists */ @@ -355,7 +371,7 @@ static struct strategic_orchestration_node *get_node_by_task(struct strategic_or /*************************************************************/ /** Tune the strategic orchestration graph. -*@param sog Reference to the strategic orchestration graph object. + *@param sog Reference to the strategic orchestration graph object. *@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. @@ -387,7 +403,7 @@ int sog_tune(struct strategic_orchestration_graph *sog, const char *name, const } else { debug(D_ERROR, "invalid priority mode: %s", value); return -1; - } + } } else if (strcmp(name, "target-results-dir") == 0) { if (sog->target_results_dir) { @@ -401,7 +417,7 @@ int sog_tune(struct strategic_orchestration_graph *sog, const char *name, const } else if (strcmp(name, "prune-depth") == 0) { sog->prune_depth = atoi(value); - + } else if (strcmp(name, "checkpoint-fraction") == 0) { double fraction = atof(value); if (fraction < 0.0 || fraction > 1.0) { @@ -413,7 +429,6 @@ int sog_tune(struct strategic_orchestration_graph *sog, const char *name, const } else { debug(D_ERROR, "invalid parameter name: %s", name); return -1; - } return 0; @@ -729,12 +744,12 @@ struct strategic_orchestration_graph *sog_create(struct vine_manager *q) struct strategic_orchestration_graph *sog = xxmalloc(sizeof(struct strategic_orchestration_graph)); sog->manager = q; - + sog->nodes = hash_table_create(0, 0); sog->task_id_to_node = itable_create(0); sog->outfile_cachename_to_node = hash_table_create(0, 0); - sog->target_results_dir = xxstrdup(sog->manager->runtime_directory); // default to current working directory + sog->target_results_dir = xxstrdup(sog->manager->runtime_directory); // default to current working directory cctools_uuid_t proxy_library_name_id; cctools_uuid_create(&proxy_library_name_id); diff --git a/taskvine/src/graph/dagvine/strategic_orchestration_graph.h b/taskvine/src/graph/dagvine/strategic_orchestration_graph.h index db8009c9fa..affadd39b1 100644 --- a/taskvine/src/graph/dagvine/strategic_orchestration_graph.h +++ b/taskvine/src/graph/dagvine/strategic_orchestration_graph.h @@ -9,43 +9,40 @@ #include "strategic_orchestration_node.h" #include "taskvine.h" - /** Select priority algorithm for strategic orchestration graph task scheduling. */ typedef enum { - TASK_PRIORITY_MODE_RANDOM = 0, /**< Assign random priority to tasks */ - TASK_PRIORITY_MODE_DEPTH_FIRST, /**< Prioritize deeper tasks first */ - TASK_PRIORITY_MODE_BREADTH_FIRST, /**< Prioritize shallower tasks first */ - TASK_PRIORITY_MODE_FIFO, /**< First in, first out priority */ - TASK_PRIORITY_MODE_LIFO, /**< Last in, first out priority */ - TASK_PRIORITY_MODE_LARGEST_INPUT_FIRST, /**< Prioritize tasks with larger inputs first */ - TASK_PRIORITY_MODE_LARGEST_STORAGE_FOOTPRINT_FIRST /**< Prioritize tasks with larger storage footprint first */ + TASK_PRIORITY_MODE_RANDOM = 0, /**< Assign random priority to tasks */ + TASK_PRIORITY_MODE_DEPTH_FIRST, /**< Prioritize deeper tasks first */ + TASK_PRIORITY_MODE_BREADTH_FIRST, /**< Prioritize shallower tasks first */ + TASK_PRIORITY_MODE_FIFO, /**< First in, first out priority */ + TASK_PRIORITY_MODE_LIFO, /**< Last in, first out priority */ + TASK_PRIORITY_MODE_LARGEST_INPUT_FIRST, /**< Prioritize tasks with larger inputs first */ + TASK_PRIORITY_MODE_LARGEST_STORAGE_FOOTPRINT_FIRST /**< Prioritize tasks with larger storage footprint first */ } task_priority_mode_t; - /** The strategic orchestration graph object (logical scheduling layer). */ struct strategic_orchestration_graph { - struct vine_manager *manager; + struct vine_manager *manager; struct hash_table *nodes; struct itable *task_id_to_node; struct hash_table *outfile_cachename_to_node; - /* Results of target keys will be stored in this directory. - * This dir path can not necessarily be a shared file system directory, - * output files will be retrieved through the network instead, - * as long as the manager can access it. */ - char *target_results_dir; + /* Results of target keys will be stored in this directory. + * This dir path can not necessarily be a shared file system directory, + * output files will be retrieved through the network instead, + * as long as the manager can access it. */ + char *target_results_dir; - char *proxy_library_name; // Python-side proxy library name (shared by all tasks) - char *proxy_function_name; // Python-side proxy function name (shared by all tasks) + char *proxy_library_name; // Python-side proxy library name (shared by all tasks) + char *proxy_function_name; // Python-side proxy function name (shared by all tasks) - int prune_depth; - double checkpoint_fraction; // 0 - 1, the fraction of intermediate results to checkpoint + int prune_depth; + double checkpoint_fraction; // 0 - 1, the fraction of intermediate results to checkpoint - task_priority_mode_t task_priority_mode; // priority mode for task graph task scheduling - double failure_injection_step_percent; // 0 - 100, the percentage of steps to inject failure + task_priority_mode_t task_priority_mode; // priority mode for task graph task scheduling + double failure_injection_step_percent; // 0 - 100, the percentage of steps to inject failure }; - /* Public APIs for operating the strategic orchestration graph */ /** Create a strategic orchestration graph object and return it. diff --git a/taskvine/src/graph/dagvine/strategic_orchestration_graph.i b/taskvine/src/graph/dagvine/strategic_orchestration_graph.i index 86295b3949..c2bc74659e 100644 --- a/taskvine/src/graph/dagvine/strategic_orchestration_graph.i +++ b/taskvine/src/graph/dagvine/strategic_orchestration_graph.i @@ -1,9 +1,8 @@ -/* SWIG interface for local dagvine graph bindings */ -%module cdagvine +/* SWIG interface for local dagvine graph API bindings */ +%module capi %{ #include "int_sizes.h" -#include "strategic_orchestration_node.h" #include "strategic_orchestration_graph.h" %} @@ -13,7 +12,4 @@ /* Import existing SWIG interface for type information (do not wrap again) */ %import "../../bindings/python3/taskvine.i" -%include "strategic_orchestration_node.h" %include "strategic_orchestration_graph.h" - - diff --git a/taskvine/src/graph/dagvine/strategic_orchestration_graph.py b/taskvine/src/graph/dagvine/strategic_orchestration_graph.py index c441997a14..c7f9fb04a6 100644 --- a/taskvine/src/graph/dagvine/strategic_orchestration_graph.py +++ b/taskvine/src/graph/dagvine/strategic_orchestration_graph.py @@ -1,33 +1,33 @@ -from ndcctools.taskvine.dagvine import cdagvine +from ndcctools.taskvine.dagvine import capi class StrategicOrchestrationGraph: def __init__(self, c_taskvine): - self._c_sog = cdagvine.sog_create(c_taskvine) + self._c_sog = capi.sog_create(c_taskvine) def tune(self, name, value): - cdagvine.sog_tune(self._c_sog, name, value) + capi.sog_tune(self._c_sog, name, value) def add_node(self, key, is_target_key): - cdagvine.sog_add_node(self._c_sog, key, is_target_key) + capi.sog_add_node(self._c_sog, key, is_target_key) def add_dependency(self, parent_key, child_key): - cdagvine.sog_add_dependency(self._c_sog, parent_key, child_key) + capi.sog_add_dependency(self._c_sog, parent_key, child_key) def compute_topology_metrics(self): - cdagvine.sog_compute_topology_metrics(self._c_sog) + capi.sog_compute_topology_metrics(self._c_sog) def get_node_outfile_remote_name(self, key): - return cdagvine.sog_get_node_outfile_remote_name(self._c_sog, key) + return capi.sog_get_node_outfile_remote_name(self._c_sog, key) def get_proxy_library_name(self): - return cdagvine.sog_get_proxy_library_name(self._c_sog) + return capi.sog_get_proxy_library_name(self._c_sog) - def set_proxy_function_name(self, proxy_function_name): - cdagvine.sog_set_proxy_function_name(self._c_sog, proxy_function_name) + def set_proxy_function(self, proxy_function): + capi.sog_set_proxy_function_name(self._c_sog, proxy_function.__name__) def execute(self): - cdagvine.sog_execute(self._c_sog) + capi.sog_execute(self._c_sog) def delete(self): - cdagvine.sog_delete(self._c_sog) \ No newline at end of file + capi.sog_delete(self._c_sog) diff --git a/taskvine/src/graph/dagvine/strategic_orchestration_node.c b/taskvine/src/graph/dagvine/strategic_orchestration_node.c index a9ec6209e6..ee757a4e8c 100644 --- a/taskvine/src/graph/dagvine/strategic_orchestration_node.c +++ b/taskvine/src/graph/dagvine/strategic_orchestration_node.c @@ -25,24 +25,6 @@ #include "strategic_orchestration_node.h" #include "taskvine.h" - -/** - * Compute a lexicographic priority score from the node key. - * Used during topological sorting to break ties deterministically. - * @param key Reference to the node key. - * @return The lexical priority. - */ -double compute_lex_priority(const char *key) -{ - double score = 0.0; - double factor = 1.0; - for (int i = 0; i < 8 && key[i] != '\0'; i++) { - score += (unsigned char)key[i] * factor; - factor *= 0.01; - } - return -score; -} - /** * Create a new node object. * @param manager Reference to the manager object. diff --git a/taskvine/src/graph/dagvine/strategic_orchestration_node.h b/taskvine/src/graph/dagvine/strategic_orchestration_node.h index ac5bc70ec0..71390e02ca 100644 --- a/taskvine/src/graph/dagvine/strategic_orchestration_node.h +++ b/taskvine/src/graph/dagvine/strategic_orchestration_node.h @@ -10,55 +10,55 @@ /** Select the type of the node-output file. */ typedef enum { - VINE_NODE_OUTFILE_TYPE_LOCAL = 0, /* Node-output file will be stored locally on the manager's staging directory */ - VINE_NODE_OUTFILE_TYPE_TEMP, /* Node-output file will be stored in the temporary node-local storage */ - VINE_NODE_OUTFILE_TYPE_SHARED_FILE_SYSTEM, /* Node-output file will be stored in the persistent shared file system */ + VINE_NODE_OUTFILE_TYPE_LOCAL = 0, /* Node-output file will be stored locally on the manager's staging directory */ + VINE_NODE_OUTFILE_TYPE_TEMP, /* Node-output file will be stored in the temporary node-local storage */ + VINE_NODE_OUTFILE_TYPE_SHARED_FILE_SYSTEM, /* Node-output file will be stored in the persistent shared file system */ } node_outfile_type_t; typedef enum { - PRUNE_STATUS_NOT_PRUNED = 0, - PRUNE_STATUS_SAFE, - PRUNE_STATUS_UNSAFE + PRUNE_STATUS_NOT_PRUNED = 0, + PRUNE_STATUS_SAFE, + PRUNE_STATUS_UNSAFE } prune_status_t; struct strategic_orchestration_node { - char *node_key; - - /* Whether the node is a target key. - * If true, the output of the node will be declared as a vine_file and retrieved through the network. */ - int is_target_key; - - struct vine_manager *manager; - struct vine_task *task; - struct vine_file *infile; - struct vine_file *outfile; - char *outfile_remote_name; - char *target_results_dir; - size_t outfile_size_bytes; - - struct list *parents; - struct list *children; - struct set *pending_parents; - - int retry_attempts_left; - int completed; - int prune_depth; - - int depth; - int height; - int upstream_subgraph_size; - int downstream_subgraph_size; - int fan_in; - int fan_out; - double heavy_score; - - timestamp_t critical_time; - timestamp_t time_spent_on_unlink_local_files; - timestamp_t time_spent_on_prune_ancestors_of_temp_node; - timestamp_t time_spent_on_prune_ancestors_of_persisted_node; - - node_outfile_type_t outfile_type; - prune_status_t prune_status; + char *node_key; + + /* Whether the node is a target key. + * If true, the output of the node will be declared as a vine_file and retrieved through the network. */ + int is_target_key; + + struct vine_manager *manager; + struct vine_task *task; + struct vine_file *infile; + struct vine_file *outfile; + char *outfile_remote_name; + char *target_results_dir; + size_t outfile_size_bytes; + + struct list *parents; + struct list *children; + struct set *pending_parents; + + int retry_attempts_left; + int completed; + int prune_depth; + + int depth; + int height; + int upstream_subgraph_size; + int downstream_subgraph_size; + int fan_in; + int fan_out; + double heavy_score; + + timestamp_t critical_time; + timestamp_t time_spent_on_unlink_local_files; + timestamp_t time_spent_on_prune_ancestors_of_temp_node; + timestamp_t time_spent_on_prune_ancestors_of_persisted_node; + + node_outfile_type_t outfile_type; + prune_status_t prune_status; }; /** Create a new strategic orchestration node object. @@ -72,21 +72,38 @@ struct strategic_orchestration_node { @return A new strategic orchestration node object. */ struct strategic_orchestration_node *son_create( - struct vine_manager *manager, - const char *node_key, - int is_target_key, - const char *proxy_library_name, - const char *proxy_function_name, - const char *target_results_dir, - int prune_depth -); - -double compute_lex_priority(const char *key); + struct vine_manager *manager, + const char *node_key, + int is_target_key, + const char *proxy_library_name, + const char *proxy_function_name, + const char *target_results_dir, + int prune_depth); +/** Delete a strategic orchestration node object. +@param node Reference to the strategic orchestration node object. +*/ void son_delete(struct strategic_orchestration_node *node); + +/** Prune the ancestors of a strategic orchestration node object. +@param node Reference to the strategic orchestration node object. +*/ void son_prune_ancestors(struct strategic_orchestration_node *node); + +/** Print information about a strategic orchestration node object. +@param node Reference to the strategic orchestration node object. +*/ void son_print_info(struct strategic_orchestration_node *node); + +/** Update the critical time of a strategic orchestration node object. +@param node Reference to the strategic orchestration node object. +@param execution_time Reference to the execution time of the node. +*/ void son_update_critical_time(struct strategic_orchestration_node *node, timestamp_t execution_time); + +/** Replicate the outfile of a strategic orchestration node object. +@param node Reference to the strategic orchestration node object. +*/ void son_replicate_outfile(struct strategic_orchestration_node *node); #endif \ No newline at end of file diff --git a/taskvine/src/graph/dagvine/utils.py b/taskvine/src/graph/dagvine/utils.py deleted file mode 100644 index d180f368da..0000000000 --- a/taskvine/src/graph/dagvine/utils.py +++ /dev/null @@ -1,17 +0,0 @@ -import inspect - -def extract_manager_kwargs(kwargs, base_class): - params = set(inspect.signature(base_class.__init__).parameters) - super_kwargs = {k: v for k, v in kwargs.items() if k in params} - leftover_kwargs = {k: v for k, v in kwargs.items() if k not in params} - return super_kwargs, leftover_kwargs - -def apply_tuning(manager, tune_dict): - for k, v in tune_dict.items(): - try: - vine_param = k.replace("_", "-") - manager.tune(vine_param, v) - print(f"Tuned {vine_param} to {v}") - except Exception as e: - print(f"Failed to tune {k}={v}: {e}") - raise From 256c02f875baea1d888f4dec3a3de0c33d20aba5 Mon Sep 17 00:00:00 2001 From: Jin Zhou Date: Sun, 19 Oct 2025 14:45:28 -0400 Subject: [PATCH 049/113] rename executor --- taskvine/src/graph/dagvine/executor.py | 255 +++++++++++++++++++++++++ 1 file changed, 255 insertions(+) create mode 100644 taskvine/src/graph/dagvine/executor.py diff --git a/taskvine/src/graph/dagvine/executor.py b/taskvine/src/graph/dagvine/executor.py new file mode 100644 index 0000000000..9879116663 --- /dev/null +++ b/taskvine/src/graph/dagvine/executor.py @@ -0,0 +1,255 @@ + +from ndcctools.taskvine import cvine +from ndcctools.taskvine.manager import Manager +from ndcctools.taskvine.utils import delete_all_files + +from ndcctools.taskvine.dagvine.proxy_library import ProxyLibrary +from ndcctools.taskvine.dagvine.proxy_functions import compute_single_key +from ndcctools.taskvine.dagvine.runtime_execution_graph import RuntimeExecutionGraph, GraphKeyResult +from ndcctools.taskvine.dagvine.strategic_orchestration_graph import StrategicOrchestrationGraph + +import cloudpickle +import os +import signal +import hashlib + +try: + import dask +except ImportError: + dask = None + +try: + from dask.base import is_dask_collection +except ImportError: + is_dask_collection = None + +try: + import dask._task_spec as dts +except ImportError: + dts = None + + +def hash_name(*args): + out_str = "" + for arg in args: + out_str += str(arg) + return hashlib.sha256(out_str.encode('utf-8')).hexdigest()[:32] + + +# convert Dask collection to task dictionary +def dask_collections_to_task_dict(collection_dict): + assert is_dask_collection is not None + from dask.highlevelgraph import HighLevelGraph, ensure_dict + + if not isinstance(collection_dict, dict): + raise TypeError("Input must be a dict or a HighLevelGraph") + + for k, v in collection_dict.items(): + if not is_dask_collection(v): + raise TypeError(f"Input must be a dict of DaskCollection, but found {k} with type {type(v)}") + + if dts: + sub_hlgs = [v.dask for v in collection_dict.values()] + hlg = HighLevelGraph.merge(*sub_hlgs).to_dict() + else: + hlg = dask.base.collections_to_dsk(collection_dict.values()) + + return ensure_dict(hlg) + + +# compatibility for Dask-created collections +def ensure_task_dict(collection_dict): + if is_dask_collection and any(is_dask_collection(v) for v in collection_dict.values()): + task_dict = dask_collections_to_task_dict(collection_dict) + else: + task_dict = collection_dict + + if dts: + return dts.convert_legacy_graph(task_dict) + else: + return task_dict + + +class GraphParams: + def __init__(self): + self.vine_manager_tuning_params = { + "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, + "balance-worker-disk-load": 0, + } + self.sog_tuning_params = { + "failure-injection-step-percent": -1, + "priority-mode": "largest-input-first", + "prune-depth": 1, + "target-results-dir": "./target_results", + "checkpoint-fraction": 0, + } + self.other_params = { + "schedule": "worst", + "libcores": 16, + "failure-injection-step-percent": -1, + "shared-file-system-dir": "./shared_file_system", + "extra-task-output-size-mb": ["uniform", 0, 0], + "extra-task-sleep-time": ["uniform", 0, 0], + "outfile-type": { + "temp": 1.0, + "shared-file-system": 0.0, + }, + } + + def update_param(self, param_name, new_value): + if param_name in self.vine_manager_tuning_params: + self.vine_manager_tuning_params[param_name] = new_value + elif param_name in self.sog_tuning_params: + self.sog_tuning_params[param_name] = new_value + elif param_name in self.other_params: + self.other_params[param_name] = new_value + else: + self.vine_manager_tuning_params[param_name] = new_value + + def get_value_of(self, param_name): + if param_name in self.vine_manager_tuning_params: + return self.vine_manager_tuning_params[param_name] + elif param_name in self.sog_tuning_params: + return self.sog_tuning_params[param_name] + elif param_name in self.other_params: + return self.other_params[param_name] + else: + raise ValueError(f"Invalid param name: {param_name}") + + +class Executor(Manager): + def __init__(self, + *args, + **kwargs): + + # handle SIGINT correctly + signal.signal(signal.SIGINT, self._on_sigint) + + self.params = GraphParams() + + # delete all files in the run info directory, do this before super().__init__() + 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: + delete_all_files(self.run_info_template_path) + + # initialize the manager + super().__init__(*args, **kwargs) + + self.runtime_directory = cvine.vine_get_runtime_directory(self._taskvine) + + def param(self, param_name): + return self.params.get_value_of(param_name) + + def update_params(self, new_params): + 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 get_run_info_path(self): + return os.path.join(self.param("run-info-path"), self.param("run-info-template")) + + def tune_manager(self): + for k, v in self.params.vine_manager_tuning_params.items(): + print(f"Tuning {k} to {v}") + self.tune(k, v) + + def tune_sog(self, sog): + for k, v in self.params.sog_tuning_params.items(): + print(f"Tuning {k} to {v}") + sog.tune(k, str(v)) + + def build_reg(self): + reg = RuntimeExecutionGraph( + self.task_dict, + shared_file_system_dir=self.param("shared-file-system-dir"), + extra_task_output_size_mb=self.param("extra-task-output-size-mb"), + extra_task_sleep_time=self.param("extra-task-sleep-time") + ) + + return reg + + def build_sog(self, reg, target_keys): + assert reg is not None, "Python graph must be built before building the C graph" + + sog = StrategicOrchestrationGraph(self._taskvine) + + sog.set_proxy_function(compute_single_key) + + # C side vine task graph must be tuned before adding nodes and dependencies + self.tune_manager() + self.tune_sog(sog) + + topo_order = reg.get_topological_order() + for k in topo_order: + sog.add_node(reg.reg_key_to_sog_key[k], int(k in target_keys)) + for pk in reg.parents_of[k]: + sog.add_dependency(reg.reg_key_to_sog_key[pk], reg.reg_key_to_sog_key[k]) + + sog.compute_topology_metrics() + + return sog + + def build_graphs(self, target_keys): + # build Python DAG (logical topology) + reg = self.build_reg() + # build C DAG (physical topology) + sog = self.build_sog(reg, target_keys) + + # set outfile remote names in reg from sog, note that these names are automatically generated + # with regard to the checkpointing strategy and the shared file system directory + for sog_key in reg.reg_key_to_sog_key.values(): + outfile_remote_name = sog.get_node_outfile_remote_name(sog_key) + reg.set_outfile_remote_name_of(reg.sog_key_to_reg_key[sog_key], outfile_remote_name) + + return reg, sog + + def create_proxy_library(self, reg, sog, hoisting_modules, env_files): + proxy_library = ProxyLibrary(self) + proxy_library.add_hoisting_modules(hoisting_modules) + proxy_library.add_env_files(env_files) + proxy_library.set_context_loader(RuntimeExecutionGraph.context_loader_func, context_loader_args=[cloudpickle.dumps(reg)]) + proxy_library.set_libcores(self.param("libcores")) + proxy_library.set_name(sog.get_proxy_library_name()) + + return proxy_library + + def run(self, collection_dict, target_keys=[], params={}, hoisting_modules=[], env_files={}): + # first update the params so that they can be used for the following construction + self.update_params(params) + + self.task_dict = ensure_task_dict(collection_dict) + + # build graphs from both sides + reg, sog = self.build_graphs(target_keys) + + # create and install the proxy library on the manager + proxy_library = self.create_proxy_library(reg, sog, hoisting_modules, env_files) + proxy_library.install() + + # execute the graph on the C side + print(f"Executing task graph, logs will be written to {self.runtime_directory}") + sog.execute() + + # clean up the library instances and template on the manager + proxy_library.uninstall() + + # delete the C graph immediately after execution, so that the lifetime of this object is limited to the execution + sog.delete() + + # load results of target keys + results = {} + for k in target_keys: + outfile_path = os.path.join(self.param("target-results-dir"), reg.outfile_remote_name[k]) + results[k] = GraphKeyResult.load_from_path(outfile_path) + return results + + def _on_sigint(self, signum, frame): + self.__del__() From ea9285f14ca3759d2ae89d83530f162b5047a8e0 Mon Sep 17 00:00:00 2001 From: Jin Zhou Date: Sun, 19 Oct 2025 16:52:07 -0400 Subject: [PATCH 050/113] checkpoint works --- taskvine/src/graph/dagvine/executor.py | 18 +- .../graph/dagvine/runtime_execution_graph.py | 5 - .../dagvine/strategic_orchestration_graph.c | 258 +++++++++-- .../dagvine/strategic_orchestration_graph.h | 24 +- .../dagvine/strategic_orchestration_node.c | 437 ++++++------------ .../dagvine/strategic_orchestration_node.h | 58 ++- 6 files changed, 410 insertions(+), 390 deletions(-) diff --git a/taskvine/src/graph/dagvine/executor.py b/taskvine/src/graph/dagvine/executor.py index 9879116663..ba80833339 100644 --- a/taskvine/src/graph/dagvine/executor.py +++ b/taskvine/src/graph/dagvine/executor.py @@ -84,22 +84,18 @@ def __init__(self): } self.sog_tuning_params = { "failure-injection-step-percent": -1, - "priority-mode": "largest-input-first", + "task-priority-mode": "largest-input-first", "prune-depth": 1, - "target-results-dir": "./target_results", + "output-dir": "./outputs", + "checkpoint-dir": "./checkpoints", "checkpoint-fraction": 0, } self.other_params = { "schedule": "worst", "libcores": 16, "failure-injection-step-percent": -1, - "shared-file-system-dir": "./shared_file_system", "extra-task-output-size-mb": ["uniform", 0, 0], "extra-task-sleep-time": ["uniform", 0, 0], - "outfile-type": { - "temp": 1.0, - "shared-file-system": 0.0, - }, } def update_param(self, param_name, new_value): @@ -159,7 +155,10 @@ def get_run_info_path(self): def tune_manager(self): for k, v in self.params.vine_manager_tuning_params.items(): print(f"Tuning {k} to {v}") - self.tune(k, v) + try: + self.tune(k, v) + except: + raise ValueError(f"Unrecognized parameter: {k}") def tune_sog(self, sog): for k, v in self.params.sog_tuning_params.items(): @@ -169,7 +168,6 @@ def tune_sog(self, sog): def build_reg(self): reg = RuntimeExecutionGraph( self.task_dict, - shared_file_system_dir=self.param("shared-file-system-dir"), extra_task_output_size_mb=self.param("extra-task-output-size-mb"), extra_task_sleep_time=self.param("extra-task-sleep-time") ) @@ -247,7 +245,7 @@ def run(self, collection_dict, target_keys=[], params={}, hoisting_modules=[], e # load results of target keys results = {} for k in target_keys: - outfile_path = os.path.join(self.param("target-results-dir"), reg.outfile_remote_name[k]) + outfile_path = os.path.join(self.param("output-dir"), reg.outfile_remote_name[k]) results[k] = GraphKeyResult.load_from_path(outfile_path) return results diff --git a/taskvine/src/graph/dagvine/runtime_execution_graph.py b/taskvine/src/graph/dagvine/runtime_execution_graph.py index bc5b8049f4..50565ae9f0 100644 --- a/taskvine/src/graph/dagvine/runtime_execution_graph.py +++ b/taskvine/src/graph/dagvine/runtime_execution_graph.py @@ -106,14 +106,9 @@ class RuntimeExecutionGraph: to produce the result. """ def __init__(self, task_dict, - shared_file_system_dir=None, extra_task_output_size_mb=["uniform", 0, 0], extra_task_sleep_time=["uniform", 0, 0]): self.task_dict = task_dict - self.shared_file_system_dir = shared_file_system_dir - - if self.shared_file_system_dir: - os.makedirs(self.shared_file_system_dir, exist_ok=True) if dts: for k, v in self.task_dict.items(): diff --git a/taskvine/src/graph/dagvine/strategic_orchestration_graph.c b/taskvine/src/graph/dagvine/strategic_orchestration_graph.c index 464cd2a969..c8db15ebde 100644 --- a/taskvine/src/graph/dagvine/strategic_orchestration_graph.c +++ b/taskvine/src/graph/dagvine/strategic_orchestration_graph.c @@ -30,6 +30,7 @@ #include "vine_file.h" #include "vine_mount.h" #include "taskvine.h" +#include "vine_temp.h" static volatile sig_atomic_t interrupted = 0; @@ -366,6 +367,169 @@ static struct strategic_orchestration_node *get_node_by_task(struct strategic_or return NULL; } +/** + * Prune the ancestors of a persisted node. This is only used for persisted nodes that produce persisted files. + * All ancestors we consider here include both temp nodes and persisted nodes, becasue data written to shared file system + * is safe and can definitely trigger upstream data redundancy to be released. + * @param sog Reference to the strategic orchestration graph object. + * @param node Reference to the node object. + * @return The number of pruned replicas. + */ +static int prune_ancestors_of_persisted_node(struct strategic_orchestration_graph *sog, struct strategic_orchestration_node *node) +{ + if (!sog || !node) { + return -1; + } + + /* find all safe ancestors */ + struct set *safe_ancestors = son_find_safe_ancestors(node); + if (!safe_ancestors) { + return 0; + } + + int pruned_replica_count = 0; + + timestamp_t start_time = timestamp_get(); + + /* prune all safe ancestors */ + struct strategic_orchestration_node *ancestor_node; + SET_ITERATE(safe_ancestors, ancestor_node) + { + switch (ancestor_node->outfile_type) { + case NODE_OUTFILE_TYPE_LOCAL: + /* do not prune the local file */ + break; + case NODE_OUTFILE_TYPE_TEMP: + /* prune the temp file */ + vine_prune_file(sog->manager, ancestor_node->outfile); + break; + case NODE_OUTFILE_TYPE_SHARED_FILE_SYSTEM: + /* unlink directly from the shared file system */ + unlink(ancestor_node->outfile_remote_name); + break; + } + ancestor_node->prune_status = PRUNE_STATUS_SAFE; + pruned_replica_count++; + } + + set_delete(safe_ancestors); + + node->time_spent_on_prune_ancestors_of_persisted_node += timestamp_get() - start_time; + + return pruned_replica_count; +} + +/** + * Prune the ancestors of a temp node. + * This function opportunistically releases upstream temporary files + * that are no longer needed once this temp-producing node has completed. + * + * Only ancestors producing temporary outputs are considered here. + * Files stored in the shared filesystem are never pruned by this function, + * because temp outputs are not considered sufficiently safe to trigger + * deletion of persisted data upstream. + * @param sog Reference to the strategic orchestration graph object. + * @param node Reference to the node object. + * @return The number of pruned replicas. + */ +static int prune_ancestors_of_temp_node(struct strategic_orchestration_graph *sog, struct strategic_orchestration_node *node) +{ + if (!sog || !node || !node->outfile || node->prune_depth <= 0) { + return 0; + } + + timestamp_t start_time = timestamp_get(); + + int pruned_replica_count = 0; + + struct list *parents = son_find_parents_by_depth(node, node->prune_depth); + + struct strategic_orchestration_node *parent_node; + LIST_ITERATE(parents, parent_node) + { + /* skip if the parent does not produce a temp file */ + if (parent_node->outfile_type != NODE_OUTFILE_TYPE_TEMP) { + continue; + } + + /* a file is prunable if its outfile is no longer needed by any child node: + * 1. it has no pending dependents + * 2. all completed dependents have also completed their corresponding recovery tasks, if any */ + int all_children_completed = 1; + struct strategic_orchestration_node *child_node; + LIST_ITERATE(parent_node->children, child_node) + { + /* break early if the child node is not completed */ + if (!child_node->completed) { + all_children_completed = 0; + break; + } + /* if the task produces a temp file and the recovery task is running, the parent is not prunable */ + if (child_node->outfile && child_node->outfile->type == VINE_TEMP) { + struct vine_task *child_node_recovery_task = child_node->outfile->recovery_task; + if (child_node_recovery_task && (child_node_recovery_task->state != VINE_TASK_INITIAL && child_node_recovery_task->state != VINE_TASK_DONE)) { + all_children_completed = 0; + break; + } + } + } + if (!all_children_completed) { + continue; + } + + pruned_replica_count += vine_prune_file(sog->manager, parent_node->outfile); + /* this parent is pruned because a successor that produces a temp file is completed, it is unsafe because the + * manager may submit a recovery task to bring it back in case of worker failures. */ + parent_node->prune_status = PRUNE_STATUS_UNSAFE; + } + + list_delete(parents); + + node->time_spent_on_prune_ancestors_of_temp_node += timestamp_get() - start_time; + + return pruned_replica_count; +} + +/** + * Prune the ancestors of a node when it is completed. + * @param node Reference to the node object. + */ +static void prune_ancestors_of_node(struct strategic_orchestration_graph *sog, struct strategic_orchestration_node *node) +{ + if (!sog || !node) { + return; + } + + /* do not prune if the node is not completed */ + if (!node->completed) { + return; + } + + timestamp_t start_time = timestamp_get(); + + int pruned_replica_count = 0; + + switch (node->outfile_type) { + case NODE_OUTFILE_TYPE_LOCAL: + case NODE_OUTFILE_TYPE_SHARED_FILE_SYSTEM: + /* If the outfile was declared as a VINE_FILE or was written to the shared fs, then it is guaranteed to be persisted + * and there is no chance that it will be lost unexpectedly. So we can safely prune all ancestors of this node. */ + pruned_replica_count = prune_ancestors_of_persisted_node(sog, node); + break; + case NODE_OUTFILE_TYPE_TEMP: + /* Otherwise, if the node outfile is a temp file, we need to be careful about pruning, because temp files are prone + * to failures, while means they can be lost due to node evictions or failures. */ + pruned_replica_count = prune_ancestors_of_temp_node(sog, node); + break; + } + + timestamp_t elapsed_time = timestamp_get() - start_time; + + debug(D_VINE, "pruned %d ancestors of node %s in %.6f seconds", pruned_replica_count, node->node_key, elapsed_time / 1000000.0); + + return; +} + /*************************************************************/ /* Public APIs */ /*************************************************************/ @@ -405,15 +569,15 @@ int sog_tune(struct strategic_orchestration_graph *sog, const char *name, const return -1; } - } else if (strcmp(name, "target-results-dir") == 0) { - if (sog->target_results_dir) { - free(sog->target_results_dir); + } else if (strcmp(name, "output-dir") == 0) { + if (sog->output_dir) { + free(sog->output_dir); } if (mkdir(value, 0777) != 0 && errno != EEXIST) { debug(D_ERROR, "failed to mkdir %s (errno=%d)", value, errno); return -1; } - sog->target_results_dir = xxstrdup(value); + sog->output_dir = xxstrdup(value); } else if (strcmp(name, "prune-depth") == 0) { sog->prune_depth = atoi(value); @@ -426,6 +590,16 @@ int sog_tune(struct strategic_orchestration_graph *sog, const char *name, const } sog->checkpoint_fraction = fraction; + } else if (strcmp(name, "checkpoint-dir") == 0) { + if (sog->checkpoint_dir) { + free(sog->checkpoint_dir); + } + if (mkdir(value, 0777) != 0 && errno != EEXIST) { + debug(D_ERROR, "failed to mkdir %s (errno=%d)", value, errno); + return -1; + } + sog->checkpoint_dir = xxstrdup(value); + } else { debug(D_ERROR, "invalid parameter name: %s", name); return -1; @@ -525,7 +699,7 @@ const char *sog_get_node_local_outfile_source(const struct strategic_orchestrati exit(1); } - if (node->outfile_type != VINE_NODE_OUTFILE_TYPE_LOCAL) { + if (node->outfile_type != NODE_OUTFILE_TYPE_LOCAL) { debug(D_ERROR, "node %s is not a local output file", node_key); exit(1); } @@ -649,23 +823,23 @@ void sog_compute_topology_metrics(struct strategic_orchestration_graph *sog) while ((node = priority_queue_pop(sorted_nodes))) { if (node->is_target_key) { /* declare the output file as a vine_file so that it can be retrieved by the manager as usual */ - node->outfile_type = VINE_NODE_OUTFILE_TYPE_LOCAL; - char *local_outfile_path = string_format("%s/%s", sog->target_results_dir, node->outfile_remote_name); - node->outfile = vine_declare_file(node->manager, local_outfile_path, VINE_CACHE_LEVEL_WORKFLOW, 0); + node->outfile_type = NODE_OUTFILE_TYPE_LOCAL; + char *local_outfile_path = string_format("%s/%s", sog->output_dir, node->outfile_remote_name); + node->outfile = vine_declare_file(sog->manager, local_outfile_path, VINE_CACHE_LEVEL_WORKFLOW, 0); free(local_outfile_path); continue; } if (assigned_checkpoint_count < checkpoint_count) { /* checkpointed files will be written directly to the shared file system, no need to manage them in the manager */ - node->outfile_type = VINE_NODE_OUTFILE_TYPE_SHARED_FILE_SYSTEM; - char *shared_file_system_outfile_path = string_format("%s/%s", sog->target_results_dir, node->outfile_remote_name); + node->outfile_type = NODE_OUTFILE_TYPE_SHARED_FILE_SYSTEM; + char *shared_file_system_outfile_path = string_format("%s/%s", sog->checkpoint_dir, node->outfile_remote_name); free(node->outfile_remote_name); node->outfile_remote_name = shared_file_system_outfile_path; node->outfile = NULL; assigned_checkpoint_count++; } else { /* other nodes will be declared as temp files to leverage node-local storage */ - node->outfile_type = VINE_NODE_OUTFILE_TYPE_TEMP; + node->outfile_type = NODE_OUTFILE_TYPE_TEMP; node->outfile = vine_declare_temp(sog->manager); } } @@ -712,13 +886,7 @@ void sog_add_node(struct strategic_orchestration_graph *sog, const char *node_ke /* if the node already exists, skip creating a new one */ struct strategic_orchestration_node *node = hash_table_lookup(sog->nodes, node_key); if (!node) { - node = son_create(sog->manager, - node_key, - is_target_key, - sog->proxy_library_name, - sog->proxy_function_name, - sog->target_results_dir, - sog->prune_depth); + node = son_create(node_key, is_target_key); if (!node) { debug(D_ERROR, "failed to create node %s", node_key); @@ -726,6 +894,32 @@ void sog_add_node(struct strategic_orchestration_graph *sog, const char *node_ke exit(1); } + if (!sog->proxy_function_name) { + debug(D_ERROR, "proxy function name is not set"); + sog_delete(sog); + exit(1); + } + + if (!sog->proxy_library_name) { + debug(D_ERROR, "proxy library name is not set"); + sog_delete(sog); + exit(1); + } + + /* create node task */ + node->task = vine_task_create(sog->proxy_function_name); + vine_task_set_library_required(node->task, sog->proxy_library_name); + vine_task_addref(node->task); + + /* construct the task arguments and declare the infile */ + char *task_arguments = son_construct_task_arguments(node); + node->infile = vine_declare_buffer(sog->manager, task_arguments, strlen(task_arguments), VINE_CACHE_LEVEL_TASK, VINE_UNLINK_WHEN_DONE); + free(task_arguments); + vine_task_add_input(node->task, node->infile, "infile", VINE_TRANSFER_ALWAYS); + + /* initialize the pruning depth of each node, currently statically set to the global prune depth */ + node->prune_depth = sog->prune_depth; + hash_table_insert(sog->nodes, node_key, node); } } @@ -745,12 +939,13 @@ struct strategic_orchestration_graph *sog_create(struct vine_manager *q) sog->manager = q; + sog->checkpoint_dir = xxstrdup(sog->manager->runtime_directory); // default to current working directory + sog->output_dir = xxstrdup(sog->manager->runtime_directory); // default to current working directory + sog->nodes = hash_table_create(0, 0); sog->task_id_to_node = itable_create(0); sog->outfile_cachename_to_node = hash_table_create(0, 0); - sog->target_results_dir = xxstrdup(sog->manager->runtime_directory); // default to current working directory - cctools_uuid_t proxy_library_name_id; cctools_uuid_create(&proxy_library_name_id); sog->proxy_library_name = xxstrdup(proxy_library_name_id.str); @@ -827,7 +1022,7 @@ void sog_execute(struct strategic_orchestration_graph *sog) struct strategic_orchestration_node *node; HASH_TABLE_ITERATE(sog->nodes, node_key, node) { - son_print_info(node); + son_debug_print(node); } /* enable return recovery tasks */ @@ -923,7 +1118,7 @@ void sog_execute(struct strategic_orchestration_graph *sog) /* if the outfile is set to save on the sharedfs, stat to get the size of the file */ switch (node->outfile_type) { - case VINE_NODE_OUTFILE_TYPE_SHARED_FILE_SYSTEM: { + case NODE_OUTFILE_TYPE_SHARED_FILE_SYSTEM: { struct stat info; int result = stat(node->outfile_remote_name, &info); if (result < 0) { @@ -941,8 +1136,8 @@ void sog_execute(struct strategic_orchestration_graph *sog) node->outfile_size_bytes = info.st_size; break; } - case VINE_NODE_OUTFILE_TYPE_LOCAL: - case VINE_NODE_OUTFILE_TYPE_TEMP: + case NODE_OUTFILE_TYPE_LOCAL: + case NODE_OUTFILE_TYPE_TEMP: node->outfile_size_bytes = node->outfile->size; break; } @@ -952,7 +1147,7 @@ void sog_execute(struct strategic_orchestration_graph *sog) node->completed = 1; /* prune nodes on task completion */ - son_prune_ancestors(node); + prune_ancestors_of_node(sog, node); /* skip recovery tasks */ if (task->type == VINE_TASK_TYPE_RECOVERY) { @@ -966,7 +1161,7 @@ void sog_execute(struct strategic_orchestration_graph *sog) } /* update critical time */ - son_update_critical_time(node, task->time_workers_execute_last); + son_update_critical_path_time(node, task->time_workers_execute_last); /* mark this regular task as completed */ progress_bar_update_part(pbar, regular_tasks_part, 1); @@ -982,11 +1177,12 @@ void sog_execute(struct strategic_orchestration_graph *sog) /* enqueue the output file for replication */ switch (node->outfile_type) { - case VINE_NODE_OUTFILE_TYPE_TEMP: - son_replicate_outfile(node); + case NODE_OUTFILE_TYPE_TEMP: + /* replicate the outfile of the temp node */ + vine_temp_replicate_file_later(sog->manager, node->outfile); break; - case VINE_NODE_OUTFILE_TYPE_LOCAL: - case VINE_NODE_OUTFILE_TYPE_SHARED_FILE_SYSTEM: + case NODE_OUTFILE_TYPE_LOCAL: + case NODE_OUTFILE_TYPE_SHARED_FILE_SYSTEM: break; } @@ -1044,7 +1240,7 @@ void sog_delete(struct strategic_orchestration_graph *sog) hash_table_remove(sog->outfile_cachename_to_node, node->outfile->cached_name); hash_table_remove(sog->manager->file_table, node->outfile->cached_name); } - if (node->outfile_type == VINE_NODE_OUTFILE_TYPE_SHARED_FILE_SYSTEM) { + if (node->outfile_type == NODE_OUTFILE_TYPE_SHARED_FILE_SYSTEM) { unlink(node->outfile_remote_name); } son_delete(node); diff --git a/taskvine/src/graph/dagvine/strategic_orchestration_graph.h b/taskvine/src/graph/dagvine/strategic_orchestration_graph.h index affadd39b1..f440ba0872 100644 --- a/taskvine/src/graph/dagvine/strategic_orchestration_graph.h +++ b/taskvine/src/graph/dagvine/strategic_orchestration_graph.h @@ -27,20 +27,32 @@ struct strategic_orchestration_graph { struct itable *task_id_to_node; struct hash_table *outfile_cachename_to_node; + /* The directory to store the checkpointed results. + * Only intermediate results can be checkpointed, the fraction of intermediate results to checkpoint is controlled by the checkpoint-fraction parameter. */ + char *checkpoint_dir; + /* Results of target keys will be stored in this directory. * This dir path can not necessarily be a shared file system directory, * output files will be retrieved through the network instead, * as long as the manager can access it. */ - char *target_results_dir; + char *output_dir; + + /* Python-side proxy library name, while is responsible for managing the runtime execution graph (reg), + * receiving function calls emitted by the strategic orchestration graph (sog), and executing them via + * the proxy function. The proxy library name is generated by the sog, and then synchronized to reg. */ + char *proxy_library_name; - char *proxy_library_name; // Python-side proxy library name (shared by all tasks) - char *proxy_function_name; // Python-side proxy function name (shared by all tasks) + /* The proxy function is hoisted in the proxy library, it receives a key (in string format) in the rog, + * converts it to the reg-based key, looks up the function definition and arguments from reg, + * and then executes the function. This name is generated by reg and synchronized to sog. */ + char *proxy_function_name; + /* The depth of the pruning strategy. 0 means no pruning, 1 means the most aggressive pruning. */ int prune_depth; - double checkpoint_fraction; // 0 - 1, the fraction of intermediate results to checkpoint + double checkpoint_fraction; /* 0 - 1, the fraction of intermediate results to checkpoint */ - task_priority_mode_t task_priority_mode; // priority mode for task graph task scheduling - double failure_injection_step_percent; // 0 - 100, the percentage of steps to inject failure + task_priority_mode_t task_priority_mode; /* priority mode for task graph task scheduling */ + double failure_injection_step_percent; /* 0 - 100, the percentage of steps to inject failure */ }; /* Public APIs for operating the strategic orchestration graph */ diff --git a/taskvine/src/graph/dagvine/strategic_orchestration_node.c b/taskvine/src/graph/dagvine/strategic_orchestration_node.c index ee757a4e8c..9f2442213d 100644 --- a/taskvine/src/graph/dagvine/strategic_orchestration_node.c +++ b/taskvine/src/graph/dagvine/strategic_orchestration_node.c @@ -17,7 +17,6 @@ #include "random.h" #include "uuid.h" -#include "vine_manager.h" #include "vine_file.h" #include "vine_task.h" #include "vine_worker_info.h" @@ -25,107 +24,9 @@ #include "strategic_orchestration_node.h" #include "taskvine.h" -/** - * Create a new node object. - * @param manager Reference to the manager object. - * @param node_key Reference to the node key. - * @param is_target_key Reference to whether the node is a target key. - * @param proxy_library_name Reference to the proxy library name. - * @param proxy_function_name Reference to the proxy function name. - * @param staging_dir Reference to the staging directory. - * @param prune_depth Reference to the prune depth. - * @return A new node object. - */ -struct strategic_orchestration_node *son_create( - struct vine_manager *manager, - const char *node_key, - int is_target_key, - const char *proxy_library_name, - const char *proxy_function_name, - const char *target_results_dir, - int prune_depth) -{ - if (!manager) { - debug(D_ERROR, "Cannot create node because manager is NULL"); - return NULL; - } - if (!node_key) { - debug(D_ERROR, "Cannot create node because node_key is NULL"); - return NULL; - } - if (!proxy_library_name) { - debug(D_ERROR, "Cannot create node because proxy_library_name is NULL"); - return NULL; - } - if (!proxy_function_name) { - debug(D_ERROR, "Cannot create node because proxy_function_name is NULL"); - return NULL; - } - if (!target_results_dir) { - debug(D_ERROR, "Cannot create node because target_results_dir is NULL"); - return NULL; - } - if (is_target_key != 0 && is_target_key != 1) { - debug(D_ERROR, "Cannot create node because is_target_key is not 0 or 1"); - return NULL; - } - - struct strategic_orchestration_node *node = xxmalloc(sizeof(struct strategic_orchestration_node)); - - node->manager = manager; - node->is_target_key = is_target_key; - node->node_key = xxstrdup(node_key); - node->target_results_dir = xxstrdup(target_results_dir); - - /* create a unique outfile remote name for the node */ - cctools_uuid_t id; - cctools_uuid_create(&id); - node->outfile_remote_name = xxstrdup(id.str); - - node->prune_status = PRUNE_STATUS_NOT_PRUNED; - node->parents = list_create(); - node->children = list_create(); - node->pending_parents = set_create(0); - node->completed = 0; - node->prune_depth = prune_depth; - node->retry_attempts_left = 1; - node->outfile_size_bytes = 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->time_spent_on_unlink_local_files = 0; - node->time_spent_on_prune_ancestors_of_temp_node = 0; - node->time_spent_on_prune_ancestors_of_persisted_node = 0; - - node->critical_time = -1; - - /* create the task */ - node->task = vine_task_create(proxy_function_name); - vine_task_set_library_required(node->task, proxy_library_name); - vine_task_addref(node->task); - - /* build JSON infile expected by library: {"fn_args": [key], "fn_kwargs": {}} */ - struct jx *event = jx_object(NULL); - struct jx *args = jx_array(NULL); - jx_array_append(args, jx_string(node->node_key)); - jx_insert(event, jx_string("fn_args"), args); - jx_insert(event, jx_string("fn_kwargs"), jx_object(NULL)); - - char *infile_content = jx_print_string(event); - jx_delete(event); - - node->infile = vine_declare_buffer(node->manager, infile_content, strlen(infile_content), VINE_CACHE_LEVEL_TASK, VINE_UNLINK_WHEN_DONE); - free(infile_content); - vine_task_add_input(node->task, node->infile, "infile", VINE_TRANSFER_ALWAYS); - - return node; -} +/*************************************************************/ +/* Private Functions */ +/*************************************************************/ /** * Check if the outfile of a node is persisted. @@ -134,7 +35,7 @@ struct strategic_orchestration_node *son_create( * @param node Reference to the node object. * @return 1 if the outfile is persisted, 0 otherwise. */ -static int _node_outfile_is_persisted(struct strategic_orchestration_node *node) +static int node_outfile_has_been_persisted(struct strategic_orchestration_node *node) { if (!node) { return 0; @@ -146,11 +47,11 @@ static int _node_outfile_is_persisted(struct strategic_orchestration_node *node) } switch (node->outfile_type) { - case VINE_NODE_OUTFILE_TYPE_LOCAL: + case NODE_OUTFILE_TYPE_LOCAL: return 1; - case VINE_NODE_OUTFILE_TYPE_SHARED_FILE_SYSTEM: + case NODE_OUTFILE_TYPE_SHARED_FILE_SYSTEM: return 1; - case VINE_NODE_OUTFILE_TYPE_TEMP: + case NODE_OUTFILE_TYPE_TEMP: return 0; } @@ -158,21 +59,21 @@ static int _node_outfile_is_persisted(struct strategic_orchestration_node *node) } /** - * Update the critical time of a node. + * 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 son_update_critical_time(struct strategic_orchestration_node *node, timestamp_t execution_time) +void son_update_critical_path_time(struct strategic_orchestration_node *node, timestamp_t execution_time) { - timestamp_t max_parent_critical_time = 0; + timestamp_t max_parent_critical_path_time = 0; struct strategic_orchestration_node *parent_node; LIST_ITERATE(node->parents, parent_node) { - if (parent_node->critical_time > max_parent_critical_time) { - max_parent_critical_time = parent_node->critical_time; + if (parent_node->critical_path_time > max_parent_critical_path_time) { + max_parent_critical_path_time = parent_node->critical_path_time; } } - node->critical_time = max_parent_critical_time + execution_time; + node->critical_path_time = max_parent_critical_path_time + execution_time; } /** @@ -182,7 +83,7 @@ void son_update_critical_time(struct strategic_orchestration_node *node, timesta * @param result Reference to the result list. * @param visited Reference to the visited set. */ -static void _find_parents_dfs(struct strategic_orchestration_node *node, int remaining_depth, struct list *result, struct set *visited) +static void find_parents_dfs(struct strategic_orchestration_node *node, int remaining_depth, struct list *result, struct set *visited) { if (!node || set_lookup(visited, node)) { return; @@ -196,133 +97,163 @@ static void _find_parents_dfs(struct strategic_orchestration_node *node, int rem struct strategic_orchestration_node *parent_node; LIST_ITERATE(node->parents, parent_node) { - _find_parents_dfs(parent_node, remaining_depth - 1, result, visited); + find_parents_dfs(parent_node, remaining_depth - 1, result, visited); } } +/*************************************************************/ +/* Public APIs */ +/*************************************************************/ + /** - * Find all parents in a specific depth of the node. - * @param node Reference to the node object. - * @param depth Reference to the depth. - * @return The list of parents. + * Create a new node object. + * @param node_key Reference to the node key. + * @param is_target_key Reference to whether the node is a target key. + * @return A new node object. */ -static struct list *_find_parents_in_depth(struct strategic_orchestration_node *node, int depth) +struct strategic_orchestration_node *son_create(const char *node_key, int is_target_key) { - if (!node || depth < 0) { + if (!node_key) { + debug(D_ERROR, "Cannot create node because node_key is NULL"); + return NULL; + } + if (is_target_key != 0 && is_target_key != 1) { + debug(D_ERROR, "Cannot create node because is_target_key is not 0 or 1"); return NULL; } - struct list *result = list_create(); + struct strategic_orchestration_node *node = xxmalloc(sizeof(struct strategic_orchestration_node)); - struct set *visited = set_create(0); - _find_parents_dfs(node, depth, result, visited); - set_delete(visited); + node->is_target_key = is_target_key; + node->node_key = xxstrdup(node_key); - return result; + /* create a unique outfile remote name for the node */ + cctools_uuid_t id; + cctools_uuid_create(&id); + node->outfile_remote_name = xxstrdup(id.str); + + node->prune_status = PRUNE_STATUS_NOT_PRUNED; + node->parents = list_create(); + node->children = list_create(); + node->pending_parents = set_create(0); + node->completed = 0; + node->prune_depth = 0; + node->retry_attempts_left = 1; + node->outfile_size_bytes = 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->time_spent_on_unlink_local_files = 0; + node->time_spent_on_prune_ancestors_of_temp_node = 0; + node->time_spent_on_prune_ancestors_of_persisted_node = 0; + + node->critical_path_time = -1; + + return node; } /** - * Prune the ancestors of a temp node. This is only used for temp nodes that produce temp files. - * All ancestors of this node we consider here are temp nodes, we can not safely prune those stored in the shared file system - * because temp nodes are not considered safe enough to trigger the deletion of upstream persisted files. + * Construct the task arguments for the node. * @param node Reference to the node object. - * @return The number of pruned replicas. + * @return The task arguments in JSON format: {"fn_args": [key], "fn_kwargs": {}}. */ -static int prune_ancestors_of_temp_node(struct strategic_orchestration_node *node) +char *son_construct_task_arguments(struct strategic_orchestration_node *node) { - if (!node || !node->outfile || node->prune_depth <= 0) { - return 0; + if (!node) { + return NULL; } - timestamp_t start_time = timestamp_get(); - - int pruned_replica_count = 0; - - struct list *parents = _find_parents_in_depth(node, node->prune_depth); + struct jx *event = jx_object(NULL); + struct jx *args = jx_array(NULL); + jx_array_append(args, jx_string(node->node_key)); + jx_insert(event, jx_string("fn_args"), args); + jx_insert(event, jx_string("fn_kwargs"), jx_object(NULL)); - struct strategic_orchestration_node *parent_node; - LIST_ITERATE(parents, parent_node) - { - /* skip if the parent produces a shared file system file */ - if (!parent_node->outfile) { - continue; - } - /* skip if the parent produces a non-temp file */ - if (parent_node->outfile->type != VINE_TEMP) { - continue; - } + char *infile_content = jx_print_string(event); + jx_delete(event); - /* a file is prunable if its outfile is no longer needed by any child node: - * 1. it has no pending dependents - * 2. all completed dependents have also completed their corresponding recovery tasks, if any */ - int all_children_completed = 1; - struct strategic_orchestration_node *child_node; - LIST_ITERATE(parent_node->children, child_node) - { - /* break early if the child node is not completed */ - if (!child_node->completed) { - all_children_completed = 0; - break; - } - /* if the task produces a temp file and the recovery task is running, the parent is not prunable */ - if (child_node->outfile && child_node->outfile->type == VINE_TEMP) { - struct vine_task *child_node_recovery_task = child_node->outfile->recovery_task; - if (child_node_recovery_task && (child_node_recovery_task->state != VINE_TASK_INITIAL && child_node_recovery_task->state != VINE_TASK_DONE)) { - all_children_completed = 0; - break; - } - } - } - if (!all_children_completed) { - continue; - } + return infile_content; +} - pruned_replica_count += vine_prune_file(node->manager, parent_node->outfile); - /* this parent is pruned because a successor that produces a temp file is completed, it is unsafe because the - * manager may submit a recovery task to bring it back in case of worker failures. */ - parent_node->prune_status = PRUNE_STATUS_UNSAFE; +/** + * Find all parents in a specific depth of the node. + * @param node Reference to the node object. + * @param depth Reference to the depth. + * @return The list of parents. + */ +struct list *son_find_parents_by_depth(struct strategic_orchestration_node *node, int depth) +{ + if (!node || depth < 0) { + return NULL; } - list_delete(parents); + struct list *result = list_create(); - node->time_spent_on_prune_ancestors_of_temp_node += timestamp_get() - start_time; + struct set *visited = set_create(0); + find_parents_dfs(node, depth, result, visited); + set_delete(visited); - return pruned_replica_count; + return result; } /** - * Find all safe ancestors of a node. - * @param start_node Reference to the start node. - * @return The set of safe ancestors. + * Perform a reverse BFS traversal to identify all ancestors of a given node + * whose outputs can be safely pruned. + * + * A parent node is considered "safe" if: + * 1. All of its child nodes are either: + * - already persisted (their outputs are stored in a reliable location), or + * - already marked as safely pruned. + * 2. None of its child nodes remain in an unsafe or incomplete state. + * + * This function starts from the given node and iteratively walks up the DAG, + * collecting all such "safe" ancestors into a set. Nodes that have already + * been marked as PRUNE_STATUS_SAFE are skipped early. + * + * The returned set contains all ancestors that can be safely pruned once the + * current node’s output has been persisted. + * + * @param start_node The node from which to begin the reverse search. + * @return A set of ancestor nodes that are safe to prune (excluding start_node). */ -static struct set *_find_safe_ancestors(struct strategic_orchestration_node *start_node) +struct set *son_find_safe_ancestors(struct strategic_orchestration_node *start_node) { if (!start_node) { return NULL; } struct set *visited_nodes = set_create(0); - struct list *bfs_nodes = list_create(); + struct set *safe_ancestors = set_create(0); - list_push_tail(bfs_nodes, start_node); - set_insert(visited_nodes, start_node); + struct list *queue = list_create(); - while (list_size(bfs_nodes) > 0) { - struct strategic_orchestration_node *current = list_pop_head(bfs_nodes); + list_push_tail(queue, start_node); + set_insert(visited_nodes, start_node); + while (list_size(queue) > 0) { + struct strategic_orchestration_node *current_node = list_pop_head(queue); struct strategic_orchestration_node *parent_node; - LIST_ITERATE(current->parents, parent_node) + + LIST_ITERATE(current_node->parents, parent_node) { if (set_lookup(visited_nodes, parent_node)) { continue; } - /* shortcut if this parent was already safely pruned */ + set_insert(visited_nodes, parent_node); + + /* shortcut if this parent has already been marked as safely pruned */ if (parent_node->prune_status == PRUNE_STATUS_SAFE) { continue; } - /* check if all children are safe */ + /* check if all children of this parent are safe */ int all_children_safe = 1; struct strategic_orchestration_node *child_node; LIST_ITERATE(parent_node->children, child_node) @@ -332,7 +263,7 @@ static struct set *_find_safe_ancestors(struct strategic_orchestration_node *sta continue; } /* shortcut if this outside child is not persisted */ - if (!_node_outfile_is_persisted(child_node)) { + if (!node_outfile_has_been_persisted(child_node)) { all_children_safe = 0; break; } @@ -343,86 +274,24 @@ static struct set *_find_safe_ancestors(struct strategic_orchestration_node *sta } } - if (!all_children_safe) { - continue; - } - - set_insert(visited_nodes, parent_node); - list_push_tail(bfs_nodes, parent_node); - } - } - - list_delete(bfs_nodes); - set_remove(visited_nodes, start_node); - - return visited_nodes; -} - -/** - * Prune the ancestors of a persisted node. This is only used for persisted nodes that produce persisted files. - * All ancestors we consider here include both temp nodes and persisted nodes, becasue data written to shared file system - * is safe and can definitely trigger upstream data redundancy to be released. - * @param node Reference to the node object. - * @return The number of pruned replicas. - */ -static int prune_ancestors_of_persisted_node(struct strategic_orchestration_node *node) -{ - if (!node) { - return 0; - } - - timestamp_t start_time = timestamp_get(); - - int pruned_replica_count = 0; - - /* find all safe ancestors */ - struct set *safe_ancestors = _find_safe_ancestors(node); - if (!safe_ancestors) { - return 0; - } - - /* prune all safe ancestors */ - struct strategic_orchestration_node *ancestor_node; - SET_ITERATE(safe_ancestors, ancestor_node) - { - /* unlink the shared file system file */ - if (!ancestor_node->outfile) { - timestamp_t unlink_start = timestamp_get(); - if (ancestor_node->outfile_remote_name) { - unlink(ancestor_node->outfile_remote_name); // system call - } - node->time_spent_on_unlink_local_files += timestamp_get() - unlink_start; - debug(D_VINE, "unlinked %s size: %zu bytes, time: %" PRIu64, ancestor_node->outfile_remote_name ? ancestor_node->outfile_remote_name : "(null)", ancestor_node->outfile_size_bytes, (uint64_t)node->time_spent_on_unlink_local_files); - } else { - switch (ancestor_node->outfile->type) { - case VINE_FILE: - /* do not prune the staging dir file */ - break; - case VINE_TEMP: - /* prune the temp file */ - vine_prune_file(node->manager, ancestor_node->outfile); - break; - default: - debug(D_ERROR, "unsupported outfile type: %d", ancestor_node->outfile->type); - break; + if (all_children_safe) { + set_insert(safe_ancestors, parent_node); + list_push_tail(queue, parent_node); } } - ancestor_node->prune_status = PRUNE_STATUS_SAFE; - pruned_replica_count++; } - set_delete(safe_ancestors); + list_delete(queue); + set_delete(visited_nodes); - node->time_spent_on_prune_ancestors_of_persisted_node += timestamp_get() - start_time; - - return pruned_replica_count; + return safe_ancestors; } /** * Print the info of the node. * @param node Reference to the node object. */ -void son_print_info(struct strategic_orchestration_node *node) +void son_debug_print(struct strategic_orchestration_node *node) { if (!node) { return; @@ -436,7 +305,6 @@ void son_print_info(struct strategic_orchestration_node *node) debug(D_VINE, "---------------- Node Info ----------------"); debug(D_VINE, "key: %s", node->node_key); debug(D_VINE, "task_id: %d", node->task->task_id); - debug(D_VINE, "target_results_dir: %s", node->target_results_dir ? node->target_results_dir : "(null)"); debug(D_VINE, "depth: %d", node->depth); debug(D_VINE, "height: %d", node->height); debug(D_VINE, "prune_depth: %d", node->prune_depth); @@ -506,48 +374,6 @@ void son_print_info(struct strategic_orchestration_node *node) debug(D_VINE, "-------------------------------------------"); } -/** - * Prune the ancestors of a node when it is completed. - * @param node Reference to the node object. - */ -void son_prune_ancestors(struct strategic_orchestration_node *node) -{ - if (!node) { - return; - } - - timestamp_t start_time = timestamp_get(); - - int pruned_replica_count = 0; - - if (_node_outfile_is_persisted(node)) { - pruned_replica_count = prune_ancestors_of_persisted_node(node); - } else { - pruned_replica_count = prune_ancestors_of_temp_node(node); - } - - timestamp_t elapsed_time = timestamp_get() - start_time; - - debug(D_VINE, "pruned %d ancestors of node %s in %.6f seconds", pruned_replica_count, node->node_key, elapsed_time / 1000000.0); -} - -/** - * Replicate the outfile of a node if it is a temp file. - * @param node Reference to the node object. - */ -void son_replicate_outfile(struct strategic_orchestration_node *node) -{ - if (!node || !node->outfile) { - return; - } - - if (node->outfile->type != VINE_TEMP) { - return; - } - - vine_temp_replicate_file_later(node->manager, node->outfile); -} - /** * Delete the node and all of its associated resources. * @param node Reference to the node object. @@ -564,9 +390,6 @@ void son_delete(struct strategic_orchestration_node *node) if (node->outfile_remote_name) { free(node->outfile_remote_name); } - if (node->target_results_dir) { - free(node->target_results_dir); - } vine_task_delete(node->task); node->task = NULL; diff --git a/taskvine/src/graph/dagvine/strategic_orchestration_node.h b/taskvine/src/graph/dagvine/strategic_orchestration_node.h index 71390e02ca..a2f421dd06 100644 --- a/taskvine/src/graph/dagvine/strategic_orchestration_node.h +++ b/taskvine/src/graph/dagvine/strategic_orchestration_node.h @@ -4,23 +4,24 @@ #include "vine_task.h" #include "hash_table.h" #include "list.h" -#include "vine_manager.h" #include "set.h" #include "taskvine.h" /** Select the type of the node-output file. */ typedef enum { - VINE_NODE_OUTFILE_TYPE_LOCAL = 0, /* Node-output file will be stored locally on the manager's staging directory */ - VINE_NODE_OUTFILE_TYPE_TEMP, /* Node-output file will be stored in the temporary node-local storage */ - VINE_NODE_OUTFILE_TYPE_SHARED_FILE_SYSTEM, /* Node-output file will be stored in the persistent shared file system */ + NODE_OUTFILE_TYPE_LOCAL = 0, /* Node-output file will be stored locally on the local staging directory */ + NODE_OUTFILE_TYPE_TEMP, /* Node-output file will be stored in the temporary node-local storage */ + NODE_OUTFILE_TYPE_SHARED_FILE_SYSTEM, /* Node-output file will be stored in the persistent shared file system */ } node_outfile_type_t; +/** The status of an output file of a node. */ typedef enum { PRUNE_STATUS_NOT_PRUNED = 0, PRUNE_STATUS_SAFE, PRUNE_STATUS_UNSAFE } prune_status_t; +/** The strategic orchestration node object. */ struct strategic_orchestration_node { char *node_key; @@ -28,12 +29,10 @@ struct strategic_orchestration_node { * If true, the output of the node will be declared as a vine_file and retrieved through the network. */ int is_target_key; - struct vine_manager *manager; struct vine_task *task; struct vine_file *infile; struct vine_file *outfile; char *outfile_remote_name; - char *target_results_dir; size_t outfile_size_bytes; struct list *parents; @@ -52,7 +51,7 @@ struct strategic_orchestration_node { int fan_out; double heavy_score; - timestamp_t critical_time; + timestamp_t critical_path_time; timestamp_t time_spent_on_unlink_local_files; timestamp_t time_spent_on_prune_ancestors_of_temp_node; timestamp_t time_spent_on_prune_ancestors_of_persisted_node; @@ -62,48 +61,45 @@ struct strategic_orchestration_node { }; /** Create a new strategic orchestration node object. -@param manager Reference to the manager object. @param node_key Reference to the node key. @param is_target_key Reference to whether the node is a target key. -@param proxy_library_name Reference to the proxy library name. -@param proxy_function_name Reference to the proxy function name. -@param target_results_dir Reference to the target results directory. -@param prune_depth Reference to the prune depth. @return A new strategic orchestration node object. */ -struct strategic_orchestration_node *son_create( - struct vine_manager *manager, - const char *node_key, - int is_target_key, - const char *proxy_library_name, - const char *proxy_function_name, - const char *target_results_dir, - int prune_depth); +struct strategic_orchestration_node *son_create(const char *node_key, int is_target_key); -/** Delete a strategic orchestration node object. +/** Create the task arguments for a strategic orchestration node object. @param node Reference to the strategic orchestration node object. +@return The task arguments in JSON format: {"fn_args": [key], "fn_kwargs": {}}. */ -void son_delete(struct strategic_orchestration_node *node); +char *son_construct_task_arguments(struct strategic_orchestration_node *node); -/** Prune the ancestors of a strategic orchestration node object. +/** Delete a strategic orchestration node object. @param node Reference to the strategic orchestration node object. */ -void son_prune_ancestors(struct strategic_orchestration_node *node); +void son_delete(struct strategic_orchestration_node *node); /** Print information about a strategic orchestration node object. @param node Reference to the strategic orchestration node object. */ -void son_print_info(struct strategic_orchestration_node *node); +void son_debug_print(struct strategic_orchestration_node *node); -/** Update the critical time of a strategic orchestration node object. -@param node Reference to the strategic orchestration node object. -@param execution_time Reference to the execution time of the node. +/** Find all safe ancestors of a strategic orchestration node object. +@param start_node Reference to the start node. +@return The set of safe ancestors. +*/ +struct set *son_find_safe_ancestors(struct strategic_orchestration_node *start_node); + +/** Find all parents in a specific depth of a strategic orchestration node object. +@param node Reference to the node object. +@param depth Reference to the depth. +@return The list of parents. */ -void son_update_critical_time(struct strategic_orchestration_node *node, timestamp_t execution_time); +struct list *son_find_parents_by_depth(struct strategic_orchestration_node *node, int depth); -/** Replicate the outfile of a strategic orchestration node object. +/** Update the critical time of a strategic orchestration node object. @param node Reference to the strategic orchestration node object. +@param execution_time Reference to the execution time of the node. */ -void son_replicate_outfile(struct strategic_orchestration_node *node); +void son_update_critical_path_time(struct strategic_orchestration_node *node, timestamp_t execution_time); #endif \ No newline at end of file From 35243c507c2a148890c007dcfbe0f4cf0eafac73 Mon Sep 17 00:00:00 2001 From: Jin Zhou Date: Sun, 19 Oct 2025 16:57:17 -0400 Subject: [PATCH 051/113] update apis --- taskvine/src/graph/dagvine/executor.py | 19 ++++++------------- 1 file changed, 6 insertions(+), 13 deletions(-) diff --git a/taskvine/src/graph/dagvine/executor.py b/taskvine/src/graph/dagvine/executor.py index ba80833339..a02a16cb43 100644 --- a/taskvine/src/graph/dagvine/executor.py +++ b/taskvine/src/graph/dagvine/executor.py @@ -29,13 +29,6 @@ dts = None -def hash_name(*args): - out_str = "" - for arg in args: - out_str += str(arg) - return hashlib.sha256(out_str.encode('utf-8')).hexdigest()[:32] - - # convert Dask collection to task dictionary def dask_collections_to_task_dict(collection_dict): assert is_dask_collection is not None @@ -165,9 +158,9 @@ def tune_sog(self, sog): print(f"Tuning {k} to {v}") sog.tune(k, str(v)) - def build_reg(self): + def build_reg(self, task_dict): reg = RuntimeExecutionGraph( - self.task_dict, + task_dict, extra_task_output_size_mb=self.param("extra-task-output-size-mb"), extra_task_sleep_time=self.param("extra-task-sleep-time") ) @@ -195,9 +188,9 @@ def build_sog(self, reg, target_keys): return sog - def build_graphs(self, target_keys): + def build_graphs(self, task_dict, target_keys): # build Python DAG (logical topology) - reg = self.build_reg() + reg = self.build_reg(task_dict) # build C DAG (physical topology) sog = self.build_sog(reg, target_keys) @@ -223,10 +216,10 @@ def run(self, collection_dict, target_keys=[], params={}, hoisting_modules=[], e # first update the params so that they can be used for the following construction self.update_params(params) - self.task_dict = ensure_task_dict(collection_dict) + task_dict = ensure_task_dict(collection_dict) # build graphs from both sides - reg, sog = self.build_graphs(target_keys) + reg, sog = self.build_graphs(task_dict, target_keys) # create and install the proxy library on the manager proxy_library = self.create_proxy_library(reg, sog, hoisting_modules, env_files) From 7ba4fac3cc7e51018514314528991115094fd2cc Mon Sep 17 00:00:00 2001 From: Jin Zhou Date: Sun, 19 Oct 2025 17:47:40 -0400 Subject: [PATCH 052/113] make --- taskvine/src/graph/dagvine/Makefile | 6 +++--- taskvine/src/graph/dagvine/executor.py | 2 -- 2 files changed, 3 insertions(+), 5 deletions(-) diff --git a/taskvine/src/graph/dagvine/Makefile b/taskvine/src/graph/dagvine/Makefile index 78eb2b4d7e..210d2a1033 100644 --- a/taskvine/src/graph/dagvine/Makefile +++ b/taskvine/src/graph/dagvine/Makefile @@ -36,11 +36,11 @@ $(BUILD_DIR): mkdir -p $(BUILD_DIR) $(SWIG_WRAP): $(SWIG_I) strategic_orchestration_graph.h | $(BUILD_DIR) - $(CCTOOLS_SWIG) -python -threads -relativeimport -I$(CCTOOLS_HOME)/taskvine/src/manager -I$(CCTOOLS_HOME)/dttools/src -outdir $(BUILD_DIR) -o $@ $< + $(CCTOOLS_SWIG) -python -threads -relativeimport -I$(CCTOOLS_HOME)/taskvine/src/manager -I$(CCTOOLS_HOME)/dttools/src -I$(CCTOOLS_HOME)/taskvine/src/graph/dagvine -outdir $(BUILD_DIR) -o $@ $< -# Build Python module +# Build Python module (mimic bindings: silence SWIG warnings and build PIC) $(WRAP_OBJ): $(SWIG_WRAP) - $(CCTOOLS_CC) -o $@ -c $(CCTOOLS_INTERNAL_CCFLAGS) $(CCTOOLS_PYTHON3_CCFLAGS) $(LOCAL_CCFLAGS) -Wno-error=cast-function-type $< + $(CCTOOLS_CC) -o $@ -c $(CCTOOLS_INTERNAL_CCFLAGS) $(LOCAL_CCFLAGS) $(CCTOOLS_PYTHON3_CCFLAGS) -w -fPIC -DNDEBUG $< $(PYMODULE): $(PYDEPS) ifeq ($(CCTOOLS_STATIC),1) diff --git a/taskvine/src/graph/dagvine/executor.py b/taskvine/src/graph/dagvine/executor.py index a02a16cb43..7141dcde42 100644 --- a/taskvine/src/graph/dagvine/executor.py +++ b/taskvine/src/graph/dagvine/executor.py @@ -147,7 +147,6 @@ def get_run_info_path(self): def tune_manager(self): for k, v in self.params.vine_manager_tuning_params.items(): - print(f"Tuning {k} to {v}") try: self.tune(k, v) except: @@ -155,7 +154,6 @@ def tune_manager(self): def tune_sog(self, sog): for k, v in self.params.sog_tuning_params.items(): - print(f"Tuning {k} to {v}") sog.tune(k, str(v)) def build_reg(self, task_dict): From 728531b5aa0e88b9740b75f787358127c09db54b Mon Sep 17 00:00:00 2001 From: Jin Zhou Date: Sun, 19 Oct 2025 17:50:40 -0400 Subject: [PATCH 053/113] lint --- taskvine/src/graph/dagvine/executor.py | 3 +-- taskvine/src/graph/dagvine/runtime_execution_graph.py | 1 - 2 files changed, 1 insertion(+), 3 deletions(-) diff --git a/taskvine/src/graph/dagvine/executor.py b/taskvine/src/graph/dagvine/executor.py index 7141dcde42..e89c15de56 100644 --- a/taskvine/src/graph/dagvine/executor.py +++ b/taskvine/src/graph/dagvine/executor.py @@ -11,7 +11,6 @@ import cloudpickle import os import signal -import hashlib try: import dask @@ -149,7 +148,7 @@ def tune_manager(self): for k, v in self.params.vine_manager_tuning_params.items(): try: self.tune(k, v) - except: + except Exception: raise ValueError(f"Unrecognized parameter: {k}") def tune_sog(self, sog): diff --git a/taskvine/src/graph/dagvine/runtime_execution_graph.py b/taskvine/src/graph/dagvine/runtime_execution_graph.py index 50565ae9f0..37161b1bbc 100644 --- a/taskvine/src/graph/dagvine/runtime_execution_graph.py +++ b/taskvine/src/graph/dagvine/runtime_execution_graph.py @@ -2,7 +2,6 @@ # This software is distributed under the GNU General Public License. # See the file COPYING for details. -import os import hashlib import cloudpickle import collections From ac40b747a9b8ee700b71cba8356ab0582095c01e Mon Sep 17 00:00:00 2001 From: Jin Zhou Date: Sun, 19 Oct 2025 17:53:33 -0400 Subject: [PATCH 054/113] make format --- taskvine/src/bindings/python3/Makefile | 7 +- taskvine/src/graph/dagvine/params.py | 109 ------------------------- 2 files changed, 1 insertion(+), 115 deletions(-) delete mode 100644 taskvine/src/graph/dagvine/params.py diff --git a/taskvine/src/bindings/python3/Makefile b/taskvine/src/bindings/python3/Makefile index 16d9cd5ceb..e769fa0f8c 100644 --- a/taskvine/src/bindings/python3/Makefile +++ b/taskvine/src/bindings/python3/Makefile @@ -39,10 +39,5 @@ install: all cp taskvine.py $(CCTOOLS_PYTHON3_PATH)/ mkdir -p $(CCTOOLS_PYTHON3_PATH)/ndcctools/taskvine/${GRAPH_EXECUTOR_MODULE_NAME} - cp $(CCTOOLS_HOME)/taskvine/src/graph/${GRAPH_EXECUTOR_MODULE_NAME}/runtime_execution_graph.py $(CCTOOLS_PYTHON3_PATH)/ndcctools/taskvine/${GRAPH_EXECUTOR_MODULE_NAME}/runtime_execution_graph.py - cp $(CCTOOLS_HOME)/taskvine/src/graph/${GRAPH_EXECUTOR_MODULE_NAME}/executor.py $(CCTOOLS_PYTHON3_PATH)/ndcctools/taskvine/${GRAPH_EXECUTOR_MODULE_NAME}/executor.py - cp $(CCTOOLS_HOME)/taskvine/src/graph/${GRAPH_EXECUTOR_MODULE_NAME}/proxy_library.py $(CCTOOLS_PYTHON3_PATH)/ndcctools/taskvine/${GRAPH_EXECUTOR_MODULE_NAME}/proxy_library.py - cp $(CCTOOLS_HOME)/taskvine/src/graph/${GRAPH_EXECUTOR_MODULE_NAME}/proxy_functions.py $(CCTOOLS_PYTHON3_PATH)/ndcctools/taskvine/${GRAPH_EXECUTOR_MODULE_NAME}/proxy_functions.py - cp $(CCTOOLS_HOME)/taskvine/src/graph/${GRAPH_EXECUTOR_MODULE_NAME}/strategic_orchestration_graph.py $(CCTOOLS_PYTHON3_PATH)/ndcctools/taskvine/${GRAPH_EXECUTOR_MODULE_NAME}/strategic_orchestration_graph.py - cp $(CCTOOLS_HOME)/taskvine/src/graph/${GRAPH_EXECUTOR_MODULE_NAME}/params.py $(CCTOOLS_PYTHON3_PATH)/ndcctools/taskvine/${GRAPH_EXECUTOR_MODULE_NAME}/params.py + cp $(CCTOOLS_HOME)/taskvine/src/graph/${GRAPH_EXECUTOR_MODULE_NAME}/*.py $(CCTOOLS_PYTHON3_PATH)/ndcctools/taskvine/${GRAPH_EXECUTOR_MODULE_NAME}/ @echo "# ${GRAPH_EXECUTOR_MODULE_NAME} package" > $(CCTOOLS_PYTHON3_PATH)/ndcctools/taskvine/${GRAPH_EXECUTOR_MODULE_NAME}/__init__.py diff --git a/taskvine/src/graph/dagvine/params.py b/taskvine/src/graph/dagvine/params.py deleted file mode 100644 index d513f8b2ac..0000000000 --- a/taskvine/src/graph/dagvine/params.py +++ /dev/null @@ -1,109 +0,0 @@ -from dataclasses import dataclass, field -from ndcctools.taskvine.utils import get_c_constant - - -@dataclass -class ManagerTuningParams: - """These parameters are used to tune the manager at the C level - and should comply with the TaskVine manager API.""" - worker_source_max_transfers: int = 100 - max_retrievals: int = -1 - prefer_dispatch: int = 1 - transient_error_interval: int = 1 - attempt_schedule_depth: int = 10000 - temp_replica_count: int = 1 - enforce_worker_eviction_interval: int = -1 - balance_worker_disk_load: int = 0 - - def update(self, params: dict): - """Update configuration from a dict. - - Converts '-' to '_' in all keys. - - Creates new attributes if they don't exist. - """ - if not isinstance(params, dict): - raise TypeError(f"update() expects a dict, got {type(params).__name__}") - - for k, v in params.items(): - normalized_key = k.replace("-", "_") - setattr(self, normalized_key, v) - return self - - def to_dict(self): - """Convert all current attributes (including dynamically added ones) - to a dict, replacing '_' with '-'. - """ - return {k.replace("_", "-"): v for k, v in self.__dict__.items()} - - -@dataclass -class VineConstantParams: - """ - All attributes are accessed in lower case for the convenience of the users. - If there is a need to use these values in the C code, convert them to uppercase and call the get_c_constant_of method to get the C constant. - """ - - schedule: str = "worst" - task_priority_mode: str = "largest-input-first" - - valid_normalized_values = { - "schedule": {"files", "time", "rand", "worst", "disk"}, - "task_priority_mode": {"random", "depth-first", "breadth-first", "fifo", "lifo", "largest-input-first", "largest-storage-footprint-first"}, - } - - def normalize(self, obj): - """Normalize a string by converting '-' to '_' and uppercase the string.""" - return obj.replace("-", "_").lower() - - def update(self, params: dict): - """Update configuration from a dict. - - Converts '-' to '_' in all keys and values, and uppercase the values. - - Creates new attributes if they don't exist. - """ - if not isinstance(params, dict): - raise TypeError(f"update() expects a dict, got {type(params).__name__}") - - for k, v in params.items(): - normalized_key = self.normalize(k) - normalized_value = self.normalize(v) - assert normalized_key in self.valid_normalized_values, f"Invalid key: {normalized_key}" - assert normalized_value in self.valid_normalized_values[normalized_key], f"Invalid value: {normalized_value} for key: {normalized_key}" - setattr(self, normalized_key, normalized_value) - return self - - def get_c_constant_of(self, key): - """Get the C constant of a key.""" - normalized_key = self.normalize(key) - if normalized_key not in self.valid_normalized_values: - raise ValueError(f"Invalid key: {normalized_key}") - return get_c_constant(f"{normalized_key.upper()}_{getattr(self, normalized_key).upper()}") - - -@dataclass -class RegularParams: - """Regular parameters that will be used directly by the graph executor.""" - libcores: int = 16 - failure_injection_step_percent: int = -1 - prune_depth: int = 1 - staging_dir: str = "./staging" - shared_file_system_dir: str = "./shared_file_system" - extra_task_output_size_mb: list[str, float, float] = field(default_factory=lambda: ["uniform", 0, 0]) - extra_task_sleep_time: list[str, float, float] = field(default_factory=lambda: ["uniform", 0, 0]) - outfile_type: dict[str, float] = field(default_factory=lambda: { - "temp": 1.0, - "shared-file-system": 0.0, - }) - - def update(self, params: dict): - """Update configuration from a dict. - - Convert '-' in keys to '_', values are as is. - """ - if not isinstance(params, dict): - raise TypeError(f"update() expects a dict, got {type(params).__name__}") - - for k, v in params.items(): - normalized_key = k.replace("-", "_") - if normalized_key not in self.__dict__.keys(): - raise ValueError(f"Invalid param key: {normalized_key}") - - setattr(self, normalized_key, v) - return self From 76d55c81b56b246894072d1b19fbf351c8ea8afc Mon Sep 17 00:00:00 2001 From: Jin Zhou Date: Sun, 19 Oct 2025 20:43:19 -0400 Subject: [PATCH 055/113] comment --- taskvine/src/graph/dagvine/executor.py | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/taskvine/src/graph/dagvine/executor.py b/taskvine/src/graph/dagvine/executor.py index e89c15de56..0ae3d9fbec 100644 --- a/taskvine/src/graph/dagvine/executor.py +++ b/taskvine/src/graph/dagvine/executor.py @@ -116,7 +116,7 @@ def __init__(self, *args, **kwargs): - # handle SIGINT correctly + # handle SIGINT signal.signal(signal.SIGINT, self._on_sigint) self.params = GraphParams() From 945e26f73b1b957447b1fedf1f4e434553913080 Mon Sep 17 00:00:00 2001 From: Jin Zhou Date: Sun, 19 Oct 2025 22:12:48 -0400 Subject: [PATCH 056/113] file name fix --- taskvine/src/bindings/python3/Makefile | 12 +++--------- taskvine/src/graph/dagvine/Makefile | 13 +++++++------ taskvine/src/graph/dagvine/__init__.py | 7 +++++++ .../src/graph/dagvine/{executor.py => dagvine.py} | 2 +- 4 files changed, 18 insertions(+), 16 deletions(-) create mode 100644 taskvine/src/graph/dagvine/__init__.py rename taskvine/src/graph/dagvine/{executor.py => dagvine.py} (99%) diff --git a/taskvine/src/bindings/python3/Makefile b/taskvine/src/bindings/python3/Makefile index e769fa0f8c..905b2703ca 100644 --- a/taskvine/src/bindings/python3/Makefile +++ b/taskvine/src/bindings/python3/Makefile @@ -1,12 +1,10 @@ include ../../../../config.mk include ../../../../rules.mk -GRAPH_EXECUTOR_MODULE_NAME = dagvine - # Python always uses 'so' for its modules (even on Darwin) CCTOOLS_DYNAMIC_SUFFIX = so # SWIG produces code that causes a lot of warnings, so use -w to turn those off. -LOCAL_CCFLAGS = -w -fPIC -DNDEBUG $(CCTOOLS_PYTHON3_CCFLAGS) -I ../../manager -I ../../graph/${GRAPH_EXECUTOR_MODULE_NAME} +LOCAL_CCFLAGS = -w -fPIC -DNDEBUG $(CCTOOLS_PYTHON3_CCFLAGS) -I ../../manager LOCAL_LINKAGE = $(CCTOOLS_PYTHON3_LDFLAGS) -lz $(CCTOOLS_OPENSSL_LDFLAGS) $(CCTOOLS_HOME)/taskvine/src/manager/libtaskvine.a $(CCTOOLS_HOME)/dttools/src/libdttools.a CCTOOLS_FLAKE8_IGNORE_FILES = "cvine.py" @@ -20,7 +18,7 @@ all: $(TARGETS) vine_wrap.c: taskvine.i @echo "SWIG taskvine.i (python)" - @$(CCTOOLS_SWIG) -o vine_wrap.c -outdir ndcctools/taskvine -python -I$(CCTOOLS_HOME)/dttools/src -I$(CCTOOLS_HOME)/taskvine/src/manager -I$(CCTOOLS_HOME)/taskvine/src/graph/${GRAPH_EXECUTOR_MODULE_NAME} taskvine.i + @$(CCTOOLS_SWIG) -o vine_wrap.c -outdir ndcctools/taskvine -python -I$(CCTOOLS_HOME)/dttools/src -I$(CCTOOLS_HOME)/taskvine/src/manager taskvine.i $(DSPYTHONSO): vine_wrap.o $(EXTERNAL_DEPENDENCIES) @@ -36,8 +34,4 @@ 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 - cp taskvine.py $(CCTOOLS_PYTHON3_PATH)/ - - mkdir -p $(CCTOOLS_PYTHON3_PATH)/ndcctools/taskvine/${GRAPH_EXECUTOR_MODULE_NAME} - cp $(CCTOOLS_HOME)/taskvine/src/graph/${GRAPH_EXECUTOR_MODULE_NAME}/*.py $(CCTOOLS_PYTHON3_PATH)/ndcctools/taskvine/${GRAPH_EXECUTOR_MODULE_NAME}/ - @echo "# ${GRAPH_EXECUTOR_MODULE_NAME} package" > $(CCTOOLS_PYTHON3_PATH)/ndcctools/taskvine/${GRAPH_EXECUTOR_MODULE_NAME}/__init__.py + cp taskvine.py $(CCTOOLS_PYTHON3_PATH)/ \ No newline at end of file diff --git a/taskvine/src/graph/dagvine/Makefile b/taskvine/src/graph/dagvine/Makefile index 210d2a1033..95c300402c 100644 --- a/taskvine/src/graph/dagvine/Makefile +++ b/taskvine/src/graph/dagvine/Makefile @@ -6,6 +6,7 @@ LOCAL_CCFLAGS+=-I ${CCTOOLS_HOME}/taskvine/src/manager DAGVINE_SOURCE_DIR = $(CCTOOLS_HOME)/taskvine/src/graph/dagvine +DAGVINE_MODULE_DIR = $(CCTOOLS_PYTHON3_PATH)/ndcctools/taskvine/dagvine SOURCES = strategic_orchestration_node.c strategic_orchestration_graph.c @@ -28,7 +29,7 @@ TARGETS = $(LIBRARIES) $(PYMODULE) $(PROGRAMS) all: $(TARGETS) - + $(PROGRAMS): $(EXTERNALS) @@ -54,15 +55,15 @@ endif install: all mkdir -p $(CCTOOLS_INSTALL_DIR)/graph/dagvine/include cp ${CCTOOLS_HOME}/taskvine/src/manager/taskvine.h $(CCTOOLS_INSTALL_DIR)/graph/dagvine/include/ - mkdir -p $(CCTOOLS_PYTHON3_PATH)/ndcctools/taskvine/dagvine - cp $(PYMODULE) $(CCTOOLS_PYTHON3_PATH)/ndcctools/taskvine/dagvine/ - cp $(BUILD_DIR)/capi.py $(CCTOOLS_PYTHON3_PATH)/ndcctools/taskvine/dagvine/ + mkdir -p $(DAGVINE_MODULE_DIR) + cp $(PYMODULE) $(DAGVINE_MODULE_DIR) + cp $(BUILD_DIR)/capi.py $(DAGVINE_MODULE_DIR) + # also install pure Python sources for the dagvine package + cp $(DAGVINE_SOURCE_DIR)/*.py $(DAGVINE_MODULE_DIR) clean: rm -rf $(PROGRAMS) $(OBJECTS) *.o $(BUILD_DIR) $(PYMODULE) capi.py -test: all - lint: if ( ! clang-format -Werror --dry-run --style='file:${CCTOOLS_HOME}/.clang-format' $(DAGVINE_SOURCE_DIR)/*.c $(DAGVINE_SOURCE_DIR)/*.h);\ then\ diff --git a/taskvine/src/graph/dagvine/__init__.py b/taskvine/src/graph/dagvine/__init__.py new file mode 100644 index 0000000000..42b763acb3 --- /dev/null +++ b/taskvine/src/graph/dagvine/__init__.py @@ -0,0 +1,7 @@ +# 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 .executor import DAGVine + +__all__ = ["DAGVine"] diff --git a/taskvine/src/graph/dagvine/executor.py b/taskvine/src/graph/dagvine/dagvine.py similarity index 99% rename from taskvine/src/graph/dagvine/executor.py rename to taskvine/src/graph/dagvine/dagvine.py index 0ae3d9fbec..6e6a825121 100644 --- a/taskvine/src/graph/dagvine/executor.py +++ b/taskvine/src/graph/dagvine/dagvine.py @@ -111,7 +111,7 @@ def get_value_of(self, param_name): raise ValueError(f"Invalid param name: {param_name}") -class Executor(Manager): +class DAGVine(Manager): def __init__(self, *args, **kwargs): From c66818400531073c731335223e1a7d93d92cb93e Mon Sep 17 00:00:00 2001 From: Jin Zhou Date: Mon, 20 Oct 2025 21:40:47 -0400 Subject: [PATCH 057/113] rename to vinedag --- dttools/src/progress_bar.c | 56 +++++++++++++------ dttools/src/progress_bar.h | 20 +++++-- dttools/src/progress_bar_test.c | 4 +- .../python3/ndcctools/taskvine/utils.py | 21 +------ taskvine/src/graph/Makefile | 2 +- .../src/graph/{dagvine => vinedag}/.gitignore | 0 .../src/graph/{dagvine => vinedag}/.gitkeep | 0 .../src/graph/{dagvine => vinedag}/Makefile | 26 ++++----- .../graph/{dagvine => vinedag}/__init__.py | 4 +- .../src/graph/{dagvine => vinedag}/config.py | 0 .../{dagvine => vinedag}/proxy_functions.py | 2 +- .../{dagvine => vinedag}/proxy_library.py | 9 ++- .../runtime_execution_graph.py | 0 .../strategic_orchestration_graph.c | 13 +++++ .../strategic_orchestration_graph.h | 2 + .../strategic_orchestration_graph.i | 2 +- .../strategic_orchestration_graph.py | 2 +- .../strategic_orchestration_node.c | 0 .../strategic_orchestration_node.h | 0 .../dagvine.py => vinedag/vinedag.py} | 45 ++++++++++++--- 20 files changed, 138 insertions(+), 70 deletions(-) rename taskvine/src/graph/{dagvine => vinedag}/.gitignore (100%) rename taskvine/src/graph/{dagvine => vinedag}/.gitkeep (100%) rename taskvine/src/graph/{dagvine => vinedag}/Makefile (77%) rename taskvine/src/graph/{dagvine => vinedag}/__init__.py (75%) rename taskvine/src/graph/{dagvine => vinedag}/config.py (100%) rename taskvine/src/graph/{dagvine => vinedag}/proxy_functions.py (98%) rename taskvine/src/graph/{dagvine => vinedag}/proxy_library.py (92%) rename taskvine/src/graph/{dagvine => vinedag}/runtime_execution_graph.py (100%) rename taskvine/src/graph/{dagvine => vinedag}/strategic_orchestration_graph.c (98%) rename taskvine/src/graph/{dagvine => vinedag}/strategic_orchestration_graph.h (98%) rename taskvine/src/graph/{dagvine => vinedag}/strategic_orchestration_graph.i (83%) rename taskvine/src/graph/{dagvine => vinedag}/strategic_orchestration_graph.py (95%) rename taskvine/src/graph/{dagvine => vinedag}/strategic_orchestration_node.c (100%) rename taskvine/src/graph/{dagvine => vinedag}/strategic_orchestration_node.h (100%) rename taskvine/src/graph/{dagvine/dagvine.py => vinedag/vinedag.py} (85%) diff --git a/dttools/src/progress_bar.c b/dttools/src/progress_bar.c index 14e1995b42..012d80e003 100644 --- a/dttools/src/progress_bar.c +++ b/dttools/src/progress_bar.c @@ -23,8 +23,13 @@ Implementation of a terminal progress bar with multiple parts. /* Max bar width (in block characters) for single-line rendering. */ #define MAX_BAR_WIDTH 30 -/* Minimum redraw interval to avoid flicker. */ -#define PROGRESS_BAR_UPDATE_INTERVAL ((USECOND) * 0.1) +/* Typed time constants (microseconds). */ +static const timestamp_t SECOND_US = 1000000ULL; +static const timestamp_t MILLISECOND_US = 1000ULL; +static const timestamp_t MICROSECOND_US = 1ULL; + +/* Minimum redraw interval to avoid flicker (200ms). */ +#define PROGRESS_BAR_UPDATE_INTERVAL_US (SECOND_US / 5) #define COLOR_RESET "\033[0m" #define COLOR_GREEN "\033[32m" @@ -75,7 +80,7 @@ static void print_progress_bar(struct ProgressBar *bar) return; } - bar->last_draw_time = timestamp_get(); + bar->last_draw_time_us = timestamp_get(); char part_text[256]; char *ptr = part_text; @@ -114,10 +119,10 @@ static void print_progress_bar(struct ProgressBar *bar) progress = 1.0f; } - timestamp_t elapsed = timestamp_get() - bar->start_time; - int h = elapsed / (3600LL * USECOND); - int m = (elapsed % (3600LL * USECOND)) / (60LL * USECOND); - int s = (elapsed % (60LL * USECOND)) / USECOND; + timestamp_t elapsed = timestamp_get() - bar->start_time_us; + int h = elapsed / (3600LL * SECOND_US); + int m = (elapsed % (3600LL * SECOND_US)) / (60LL * SECOND_US); + int s = (elapsed % (60LL * SECOND_US)) / SECOND_US; if (bar->has_drawn_once) { printf("\r\033[2K"); @@ -167,13 +172,34 @@ struct ProgressBar *progress_bar_init(const char *label) bar->label = xxstrdup(label); bar->parts = list_create(); - bar->start_time = timestamp_get(); - bar->last_draw_time = timestamp_get(); + bar->start_time_us = timestamp_get(); + bar->last_draw_time_us = 0; + bar->update_interval_us = PROGRESS_BAR_UPDATE_INTERVAL_US; + bar->update_interval_sec = (double)bar->update_interval_us / SECOND_US; bar->has_drawn_once = 0; return bar; } +/** Set the update interval for the progress bar. */ +void progress_bar_set_update_interval(struct ProgressBar *bar, double update_interval_sec) +{ + if (!bar) { + return; + } + + if (update_interval_sec < 0) { + update_interval_sec = 0; + } + bar->update_interval_sec = update_interval_sec; + /* Convert seconds to microseconds with saturation to avoid overflow. */ + if (update_interval_sec >= (double)UINT64_MAX / (double)SECOND_US) { + bar->update_interval_us = (timestamp_t)UINT64_MAX; + } else { + bar->update_interval_us = (timestamp_t)(update_interval_sec * (double)SECOND_US); + } +} + /** Create a new part. */ struct ProgressBarPart *progress_bar_create_part(const char *label, uint64_t total) { @@ -207,9 +233,8 @@ void progress_bar_set_part_total(struct ProgressBar *bar, struct ProgressBarPart if (!bar || !part) { return; } - part->total = new_total; - print_progress_bar(bar); + part->total = new_total; } /** Advance a part's current value, redraw if needed. */ @@ -224,11 +249,10 @@ void progress_bar_update_part(struct ProgressBar *bar, struct ProgressBarPart *p part->current = part->total; } - if (timestamp_get() - bar->last_draw_time < PROGRESS_BAR_UPDATE_INTERVAL) { - return; + 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); } - - print_progress_bar(bar); } /** Set the start time for the progress bar. */ @@ -238,7 +262,7 @@ void progress_bar_set_start_time(struct ProgressBar *bar, timestamp_t start_time return; } - bar->start_time = start_time; + bar->start_time_us = start_time; } /** Final render and newline. */ diff --git a/dttools/src/progress_bar.h b/dttools/src/progress_bar.h index 6a38a95424..7fe3171a3f 100644 --- a/dttools/src/progress_bar.h +++ b/dttools/src/progress_bar.h @@ -25,11 +25,15 @@ struct ProgressBarPart { /** Progress bar object. */ struct ProgressBar { - char *label; - struct list *parts; - timestamp_t start_time; - timestamp_t last_draw_time; - int has_drawn_once; + /* User-facing interval in seconds; internal comparisons use *_us. */ + double update_interval_sec; + char *label; + struct list *parts; + /* Timestamps in microseconds. */ + timestamp_t start_time_us; + timestamp_t last_draw_time_us; + timestamp_t update_interval_us; + int has_drawn_once; }; /* Progress Bar Part API */ @@ -40,6 +44,12 @@ struct ProgressBar { */ struct ProgressBar *progress_bar_init(const char *label); +/** Set the update interval for the progress bar. +@param bar Progress bar. +@param update_interval_sec Update interval in seconds. +*/ +void progress_bar_set_update_interval(struct ProgressBar *bar, double update_interval_sec); + /** Create a new part. @param label Part label (internally duplicated). @param total Total units for the part. diff --git a/dttools/src/progress_bar_test.c b/dttools/src/progress_bar_test.c index 4c72d09425..4a7b70ed1c 100644 --- a/dttools/src/progress_bar_test.c +++ b/dttools/src/progress_bar_test.c @@ -6,12 +6,14 @@ int main() { - uint64_t total = 100000; + uint64_t total = 1000000; struct ProgressBarPart *part1 = progress_bar_create_part("step", total); struct ProgressBarPart *part2 = progress_bar_create_part("fetch", total); struct ProgressBarPart *part3 = progress_bar_create_part("commit", total); struct ProgressBar *bar = progress_bar_init("Compute"); + progress_bar_set_update_interval(bar, 0.5); + progress_bar_bind_part(bar, part1); progress_bar_bind_part(bar, part2); progress_bar_bind_part(bar, part3); diff --git a/taskvine/src/bindings/python3/ndcctools/taskvine/utils.py b/taskvine/src/bindings/python3/ndcctools/taskvine/utils.py index 166b2800f2..c11c5d5b0d 100644 --- a/taskvine/src/bindings/python3/ndcctools/taskvine/utils.py +++ b/taskvine/src/bindings/python3/ndcctools/taskvine/utils.py @@ -12,12 +12,7 @@ def get_c_constant(constant): "result_success" -> VINE_RESULT_SUCCESS """ constant = f"VINE_{constant.upper()}" - # First try the base cvine module - value = getattr(cvine, constant, None) - if value is not None: - return value - # If still missing, raise a clear error - raise AttributeError(f"C constant {constant} not found in cvine or cdagvine") + return getattr(cvine, constant) def set_port_range(port): @@ -42,16 +37,4 @@ def load_variable_from_library(var_name): return globals()[var_name] -# helper function that deletes all files in a directory -def delete_all_files(root_dir): - 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}") - -# vim: set sts=4 sw=4 ts=4 expandtab ft=python: +# vim: set sts=4 sw=4 ts=4 expandtab ft=python: \ No newline at end of file diff --git a/taskvine/src/graph/Makefile b/taskvine/src/graph/Makefile index 04598b4683..fa4c7dc2c5 100644 --- a/taskvine/src/graph/Makefile +++ b/taskvine/src/graph/Makefile @@ -1,7 +1,7 @@ include ../../../config.mk include ../../../rules.mk -SUBDIRS = dagvine +SUBDIRS = vinedag all clean install test lint format: $(SUBDIRS) diff --git a/taskvine/src/graph/dagvine/.gitignore b/taskvine/src/graph/vinedag/.gitignore similarity index 100% rename from taskvine/src/graph/dagvine/.gitignore rename to taskvine/src/graph/vinedag/.gitignore diff --git a/taskvine/src/graph/dagvine/.gitkeep b/taskvine/src/graph/vinedag/.gitkeep similarity index 100% rename from taskvine/src/graph/dagvine/.gitkeep rename to taskvine/src/graph/vinedag/.gitkeep diff --git a/taskvine/src/graph/dagvine/Makefile b/taskvine/src/graph/vinedag/Makefile similarity index 77% rename from taskvine/src/graph/dagvine/Makefile rename to taskvine/src/graph/vinedag/Makefile index 95c300402c..f3882a646c 100644 --- a/taskvine/src/graph/dagvine/Makefile +++ b/taskvine/src/graph/vinedag/Makefile @@ -4,9 +4,10 @@ include ../../../../rules.mk LOCAL_LINKAGE+=${CCTOOLS_HOME}/taskvine/src/manager/libtaskvine.a ${CCTOOLS_HOME}/dttools/src/libdttools.a LOCAL_CCFLAGS+=-I ${CCTOOLS_HOME}/taskvine/src/manager +PROJECT_NAME = vinedag -DAGVINE_SOURCE_DIR = $(CCTOOLS_HOME)/taskvine/src/graph/dagvine -DAGVINE_MODULE_DIR = $(CCTOOLS_PYTHON3_PATH)/ndcctools/taskvine/dagvine +SOURCE_DIR = $(CCTOOLS_HOME)/taskvine/src/graph/$(PROJECT_NAME) +MODULE_DIR = $(CCTOOLS_PYTHON3_PATH)/ndcctools/taskvine/$(PROJECT_NAME) SOURCES = strategic_orchestration_node.c strategic_orchestration_graph.c @@ -37,7 +38,7 @@ $(BUILD_DIR): mkdir -p $(BUILD_DIR) $(SWIG_WRAP): $(SWIG_I) strategic_orchestration_graph.h | $(BUILD_DIR) - $(CCTOOLS_SWIG) -python -threads -relativeimport -I$(CCTOOLS_HOME)/taskvine/src/manager -I$(CCTOOLS_HOME)/dttools/src -I$(CCTOOLS_HOME)/taskvine/src/graph/dagvine -outdir $(BUILD_DIR) -o $@ $< + $(CCTOOLS_SWIG) -python -threads -relativeimport -I$(CCTOOLS_HOME)/taskvine/src/manager -I$(CCTOOLS_HOME)/dttools/src -I$(CCTOOLS_HOME)/taskvine/src/graph/$(PROJECT_NAME) -outdir $(BUILD_DIR) -o $@ $< # Build Python module (mimic bindings: silence SWIG warnings and build PIC) $(WRAP_OBJ): $(SWIG_WRAP) @@ -53,19 +54,18 @@ endif install: all - mkdir -p $(CCTOOLS_INSTALL_DIR)/graph/dagvine/include - cp ${CCTOOLS_HOME}/taskvine/src/manager/taskvine.h $(CCTOOLS_INSTALL_DIR)/graph/dagvine/include/ - mkdir -p $(DAGVINE_MODULE_DIR) - cp $(PYMODULE) $(DAGVINE_MODULE_DIR) - cp $(BUILD_DIR)/capi.py $(DAGVINE_MODULE_DIR) - # also install pure Python sources for the dagvine package - cp $(DAGVINE_SOURCE_DIR)/*.py $(DAGVINE_MODULE_DIR) + mkdir -p $(CCTOOLS_INSTALL_DIR)/graph/$(PROJECT_NAME)/include + cp ${CCTOOLS_HOME}/taskvine/src/manager/taskvine.h $(CCTOOLS_INSTALL_DIR)/graph/$(PROJECT_NAME)/include/ + mkdir -p $(MODULE_DIR) + cp $(PYMODULE) $(MODULE_DIR) + cp $(BUILD_DIR)/capi.py $(MODULE_DIR) + cp $(SOURCE_DIR)/*.py $(MODULE_DIR) clean: rm -rf $(PROGRAMS) $(OBJECTS) *.o $(BUILD_DIR) $(PYMODULE) capi.py lint: - if ( ! clang-format -Werror --dry-run --style='file:${CCTOOLS_HOME}/.clang-format' $(DAGVINE_SOURCE_DIR)/*.c $(DAGVINE_SOURCE_DIR)/*.h);\ + if ( ! clang-format -Werror --dry-run --style='file:${CCTOOLS_HOME}/.clang-format' $(SOURCE_DIR)/*.c $(SOURCE_DIR)/*.h);\ then\ echo "========================================================";\ echo "NOTICE: Run \`make format\` to format your latest changes.";\ @@ -74,10 +74,10 @@ lint: fi flake8 --ignore=$(CCTOOLS_FLAKE8_IGNORE_ERRORS) \ --exclude=$(CCTOOLS_FLAKE8_IGNORE_FILES),$(BUILD_DIR) \ - $(DAGVINE_SOURCE_DIR)/ + $(SOURCE_DIR)/ format: - clang-format -i $(DAGVINE_SOURCE_DIR)/*.c $(DAGVINE_SOURCE_DIR)/*.h + clang-format -i $(SOURCE_DIR)/*.c $(SOURCE_DIR)/*.h .PHONY: all clean install test lint format diff --git a/taskvine/src/graph/dagvine/__init__.py b/taskvine/src/graph/vinedag/__init__.py similarity index 75% rename from taskvine/src/graph/dagvine/__init__.py rename to taskvine/src/graph/vinedag/__init__.py index 42b763acb3..0a3da3f715 100644 --- a/taskvine/src/graph/dagvine/__init__.py +++ b/taskvine/src/graph/vinedag/__init__.py @@ -2,6 +2,6 @@ # This software is distributed under the GNU General Public License. # See the file COPYING for details. -from .executor import DAGVine +from .vinedag import VineDAG -__all__ = ["DAGVine"] +__all__ = ["VineDAG"] diff --git a/taskvine/src/graph/dagvine/config.py b/taskvine/src/graph/vinedag/config.py similarity index 100% rename from taskvine/src/graph/dagvine/config.py rename to taskvine/src/graph/vinedag/config.py diff --git a/taskvine/src/graph/dagvine/proxy_functions.py b/taskvine/src/graph/vinedag/proxy_functions.py similarity index 98% rename from taskvine/src/graph/dagvine/proxy_functions.py rename to taskvine/src/graph/vinedag/proxy_functions.py index e79243f987..2cbbc8d1e1 100644 --- a/taskvine/src/graph/dagvine/proxy_functions.py +++ b/taskvine/src/graph/vinedag/proxy_functions.py @@ -43,7 +43,7 @@ def compute_sexpr_key(reg, k, v): """ Evaluate a symbolic expression (S-expression) task within the task graph. - Both DAGVine and legacy Dask represent computations as symbolic + Both VineDAG and legacy Dask represent computations as symbolic expression trees (S-expressions). Each task value `v` encodes a nested structure where: - Leaf nodes are constants or task keys referencing parent results. diff --git a/taskvine/src/graph/dagvine/proxy_library.py b/taskvine/src/graph/vinedag/proxy_library.py similarity index 92% rename from taskvine/src/graph/dagvine/proxy_library.py rename to taskvine/src/graph/vinedag/proxy_library.py index 01e9bda3be..0f1d1ea1b4 100644 --- a/taskvine/src/graph/dagvine/proxy_library.py +++ b/taskvine/src/graph/vinedag/proxy_library.py @@ -10,10 +10,10 @@ import random import hashlib import collections -from ndcctools.taskvine.dagvine.runtime_execution_graph import ( +from ndcctools.taskvine.vinedag.runtime_execution_graph import ( GraphKeyResult, RuntimeExecutionGraph, hash_name, hashable ) -from ndcctools.taskvine.dagvine.proxy_functions import compute_dts_key, compute_sexpr_key, compute_single_key +from ndcctools.taskvine.vinedag.proxy_functions import compute_dts_key, compute_sexpr_key, compute_single_key from ndcctools.taskvine.utils import load_variable_from_library @@ -64,6 +64,11 @@ def set_context_loader(self, context_loader_func, context_loader_args=[], contex self.context_loader_args = context_loader_args self.context_loader_kwargs = context_loader_kwargs + def get_context_size(self): + dumped_data = self.context_loader_args[0] + serialized = round(len(dumped_data) / 1024 / 1024, 2) + return serialized + 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)" diff --git a/taskvine/src/graph/dagvine/runtime_execution_graph.py b/taskvine/src/graph/vinedag/runtime_execution_graph.py similarity index 100% rename from taskvine/src/graph/dagvine/runtime_execution_graph.py rename to taskvine/src/graph/vinedag/runtime_execution_graph.py diff --git a/taskvine/src/graph/dagvine/strategic_orchestration_graph.c b/taskvine/src/graph/vinedag/strategic_orchestration_graph.c similarity index 98% rename from taskvine/src/graph/dagvine/strategic_orchestration_graph.c rename to taskvine/src/graph/vinedag/strategic_orchestration_graph.c index c8db15ebde..2f3ba66d00 100644 --- a/taskvine/src/graph/dagvine/strategic_orchestration_graph.c +++ b/taskvine/src/graph/vinedag/strategic_orchestration_graph.c @@ -137,7 +137,12 @@ static void submit_node_task(struct strategic_orchestration_graph *sog, struct s vine_task_set_priority(node->task, priority); /* submit the task to the manager */ + timestamp_t time_start = timestamp_get(); int task_id = vine_submit(sog->manager, node->task); + double time_taken = (double)(timestamp_get() - time_start) / 1e6; + FILE *fp = fopen("vinedag_submission_time.txt", "a"); + fprintf(fp, "%.6f\n", time_taken); + fclose(fp); /* insert the task id to the task id to node map */ itable_insert(sog->task_id_to_node, task_id, node); @@ -600,6 +605,10 @@ int sog_tune(struct strategic_orchestration_graph *sog, const char *name, const } sog->checkpoint_dir = xxstrdup(value); + } else if (strcmp(name, "progress-bar-update-interval-sec") == 0) { + double val = atof(value); + sog->progress_bar_update_interval_sec = (val > 0.0) ? val : 0.1; + } else { debug(D_ERROR, "invalid parameter name: %s", name); return -1; @@ -957,6 +966,8 @@ struct strategic_orchestration_graph *sog_create(struct vine_manager *q) sog->task_priority_mode = TASK_PRIORITY_MODE_LARGEST_INPUT_FIRST; sog->failure_injection_step_percent = -1.0; + sog->progress_bar_update_interval_sec = 0.1; + /* enable debug system for C code since it uses a separate debug system instance * from the Python bindings. Use the same function that the manager uses. */ char *debug_tmp = string_format("%s/vine-logs/debug", sog->manager->runtime_directory); @@ -1077,6 +1088,8 @@ void sog_execute(struct strategic_orchestration_graph *sog) } struct ProgressBar *pbar = progress_bar_init("Executing Tasks"); + progress_bar_set_update_interval(pbar, sog->progress_bar_update_interval_sec); + struct ProgressBarPart *regular_tasks_part = progress_bar_create_part("Regular", hash_table_size(sog->nodes)); struct ProgressBarPart *recovery_tasks_part = progress_bar_create_part("Recovery", 0); progress_bar_bind_part(pbar, regular_tasks_part); diff --git a/taskvine/src/graph/dagvine/strategic_orchestration_graph.h b/taskvine/src/graph/vinedag/strategic_orchestration_graph.h similarity index 98% rename from taskvine/src/graph/dagvine/strategic_orchestration_graph.h rename to taskvine/src/graph/vinedag/strategic_orchestration_graph.h index f440ba0872..840f64cacd 100644 --- a/taskvine/src/graph/dagvine/strategic_orchestration_graph.h +++ b/taskvine/src/graph/vinedag/strategic_orchestration_graph.h @@ -53,6 +53,8 @@ struct strategic_orchestration_graph { task_priority_mode_t task_priority_mode; /* priority mode for task graph task scheduling */ double failure_injection_step_percent; /* 0 - 100, the percentage of steps to inject failure */ + + double progress_bar_update_interval_sec; /* update interval for the progress bar in seconds */ }; /* Public APIs for operating the strategic orchestration graph */ diff --git a/taskvine/src/graph/dagvine/strategic_orchestration_graph.i b/taskvine/src/graph/vinedag/strategic_orchestration_graph.i similarity index 83% rename from taskvine/src/graph/dagvine/strategic_orchestration_graph.i rename to taskvine/src/graph/vinedag/strategic_orchestration_graph.i index c2bc74659e..1f7f7be8eb 100644 --- a/taskvine/src/graph/dagvine/strategic_orchestration_graph.i +++ b/taskvine/src/graph/vinedag/strategic_orchestration_graph.i @@ -1,4 +1,4 @@ -/* SWIG interface for local dagvine graph API bindings */ +/* SWIG interface for local vinedag graph API bindings */ %module capi %{ diff --git a/taskvine/src/graph/dagvine/strategic_orchestration_graph.py b/taskvine/src/graph/vinedag/strategic_orchestration_graph.py similarity index 95% rename from taskvine/src/graph/dagvine/strategic_orchestration_graph.py rename to taskvine/src/graph/vinedag/strategic_orchestration_graph.py index c7f9fb04a6..2c8f30d5f5 100644 --- a/taskvine/src/graph/dagvine/strategic_orchestration_graph.py +++ b/taskvine/src/graph/vinedag/strategic_orchestration_graph.py @@ -1,4 +1,4 @@ -from ndcctools.taskvine.dagvine import capi +from ndcctools.taskvine.vinedag import capi class StrategicOrchestrationGraph: diff --git a/taskvine/src/graph/dagvine/strategic_orchestration_node.c b/taskvine/src/graph/vinedag/strategic_orchestration_node.c similarity index 100% rename from taskvine/src/graph/dagvine/strategic_orchestration_node.c rename to taskvine/src/graph/vinedag/strategic_orchestration_node.c diff --git a/taskvine/src/graph/dagvine/strategic_orchestration_node.h b/taskvine/src/graph/vinedag/strategic_orchestration_node.h similarity index 100% rename from taskvine/src/graph/dagvine/strategic_orchestration_node.h rename to taskvine/src/graph/vinedag/strategic_orchestration_node.h diff --git a/taskvine/src/graph/dagvine/dagvine.py b/taskvine/src/graph/vinedag/vinedag.py similarity index 85% rename from taskvine/src/graph/dagvine/dagvine.py rename to taskvine/src/graph/vinedag/vinedag.py index 6e6a825121..e1bf7838bd 100644 --- a/taskvine/src/graph/dagvine/dagvine.py +++ b/taskvine/src/graph/vinedag/vinedag.py @@ -1,16 +1,16 @@ from ndcctools.taskvine import cvine from ndcctools.taskvine.manager import Manager -from ndcctools.taskvine.utils import delete_all_files -from ndcctools.taskvine.dagvine.proxy_library import ProxyLibrary -from ndcctools.taskvine.dagvine.proxy_functions import compute_single_key -from ndcctools.taskvine.dagvine.runtime_execution_graph import RuntimeExecutionGraph, GraphKeyResult -from ndcctools.taskvine.dagvine.strategic_orchestration_graph import StrategicOrchestrationGraph +from ndcctools.taskvine.vinedag.proxy_library import ProxyLibrary +from ndcctools.taskvine.vinedag.proxy_functions import compute_single_key +from ndcctools.taskvine.vinedag.runtime_execution_graph import RuntimeExecutionGraph, GraphKeyResult +from ndcctools.taskvine.vinedag.strategic_orchestration_graph import StrategicOrchestrationGraph import cloudpickle import os import signal +import json try: import dask @@ -28,6 +28,23 @@ dts = None +def delete_all_files(root_dir): + 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}") + + +# color the text with the given color code +def color_text(text, color_code): + return f"\033[{color_code}m{text}\033[0m" + + # convert Dask collection to task dictionary def dask_collections_to_task_dict(collection_dict): assert is_dask_collection is not None @@ -81,6 +98,7 @@ def __init__(self): "output-dir": "./outputs", "checkpoint-dir": "./checkpoints", "checkpoint-fraction": 0, + "progress-bar-update-interval-sec": 0.1, } self.other_params = { "schedule": "worst", @@ -90,6 +108,10 @@ def __init__(self): "extra-task-sleep-time": ["uniform", 0, 0], } + def print_params(self): + all_params = {**self.vine_manager_tuning_params, **self.sog_tuning_params, **self.other_params} + print(json.dumps(all_params, indent=4)) + def update_param(self, param_name, new_value): if param_name in self.vine_manager_tuning_params: self.vine_manager_tuning_params[param_name] = new_value @@ -111,7 +133,7 @@ def get_value_of(self, param_name): raise ValueError(f"Invalid param name: {param_name}") -class DAGVine(Manager): +class VineDAG(Manager): def __init__(self, *args, **kwargs): @@ -130,9 +152,12 @@ def __init__(self, # initialize the manager super().__init__(*args, **kwargs) - self.runtime_directory = cvine.vine_get_runtime_directory(self._taskvine) + 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)}") + def param(self, param_name): return self.params.get_value_of(param_name) @@ -215,6 +240,9 @@ def run(self, collection_dict, target_keys=[], params={}, hoisting_modules=[], e task_dict = ensure_task_dict(collection_dict) + with open("test_dv5.pkl", "wb") as f: + cloudpickle.dump(task_dict, f) + # build graphs from both sides reg, sog = self.build_graphs(task_dict, target_keys) @@ -222,8 +250,9 @@ def run(self, collection_dict, target_keys=[], params={}, hoisting_modules=[], e proxy_library = self.create_proxy_library(reg, sog, hoisting_modules, env_files) proxy_library.install() + print(f"=== Library serialized size: {color_text(proxy_library.get_context_size(), 92)} MB") + # execute the graph on the C side - print(f"Executing task graph, logs will be written to {self.runtime_directory}") sog.execute() # clean up the library instances and template on the manager From b8bca0d78a67d64858d45e00deb0d26925461cb6 Mon Sep 17 00:00:00 2001 From: Jin Zhou Date: Mon, 20 Oct 2025 21:51:51 -0400 Subject: [PATCH 058/113] add three fields --- taskvine/src/graph/vinedag/strategic_orchestration_graph.c | 6 ++---- taskvine/src/graph/vinedag/strategic_orchestration_node.c | 5 +++++ taskvine/src/graph/vinedag/strategic_orchestration_node.h | 5 +++++ 3 files changed, 12 insertions(+), 4 deletions(-) diff --git a/taskvine/src/graph/vinedag/strategic_orchestration_graph.c b/taskvine/src/graph/vinedag/strategic_orchestration_graph.c index 2f3ba66d00..216cfe23b5 100644 --- a/taskvine/src/graph/vinedag/strategic_orchestration_graph.c +++ b/taskvine/src/graph/vinedag/strategic_orchestration_graph.c @@ -139,10 +139,7 @@ static void submit_node_task(struct strategic_orchestration_graph *sog, struct s /* submit the task to the manager */ timestamp_t time_start = timestamp_get(); int task_id = vine_submit(sog->manager, node->task); - double time_taken = (double)(timestamp_get() - time_start) / 1e6; - FILE *fp = fopen("vinedag_submission_time.txt", "a"); - fprintf(fp, "%.6f\n", time_taken); - fclose(fp); + node->submission_time = timestamp_get() - time_start; /* insert the task id to the task id to node map */ itable_insert(sog->task_id_to_node, task_id, node); @@ -1158,6 +1155,7 @@ void sog_execute(struct strategic_orchestration_graph *sog) /* mark the node as completed */ node->completed = 1; + /* prune nodes on task completion */ prune_ancestors_of_node(sog, node); diff --git a/taskvine/src/graph/vinedag/strategic_orchestration_node.c b/taskvine/src/graph/vinedag/strategic_orchestration_node.c index 9f2442213d..8fb47a8a92 100644 --- a/taskvine/src/graph/vinedag/strategic_orchestration_node.c +++ b/taskvine/src/graph/vinedag/strategic_orchestration_node.c @@ -153,6 +153,11 @@ struct strategic_orchestration_node *son_create(const char *node_key, int is_tar node->time_spent_on_prune_ancestors_of_temp_node = 0; node->time_spent_on_prune_ancestors_of_persisted_node = 0; + node->submission_time = 0; + node->scheduling_time = 0; + node->execution_time = 0; + node->retrieval_time = 0; + node->critical_path_time = -1; return node; diff --git a/taskvine/src/graph/vinedag/strategic_orchestration_node.h b/taskvine/src/graph/vinedag/strategic_orchestration_node.h index a2f421dd06..32c347caf4 100644 --- a/taskvine/src/graph/vinedag/strategic_orchestration_node.h +++ b/taskvine/src/graph/vinedag/strategic_orchestration_node.h @@ -56,6 +56,11 @@ struct strategic_orchestration_node { timestamp_t time_spent_on_prune_ancestors_of_temp_node; timestamp_t time_spent_on_prune_ancestors_of_persisted_node; + timestamp_t submission_time; + timestamp_t scheduling_time; + timestamp_t execution_time; + timestamp_t retrieval_time; + node_outfile_type_t outfile_type; prune_status_t prune_status; }; From 1575045275e17ce1bff965a940d32863ac7aa94c Mon Sep 17 00:00:00 2001 From: Jin Zhou Date: Mon, 20 Oct 2025 21:56:41 -0400 Subject: [PATCH 059/113] calculate time spent on scheduling --- taskvine/src/manager/vine_manager.c | 3 +++ taskvine/src/manager/vine_task.c | 2 ++ taskvine/src/manager/vine_task.h | 2 ++ 3 files changed, 7 insertions(+) diff --git a/taskvine/src/manager/vine_manager.c b/taskvine/src/manager/vine_manager.c index 9b3cae6a46..4bca134957 100644 --- a/taskvine/src/manager/vine_manager.c +++ b/taskvine/src/manager/vine_manager.c @@ -3617,6 +3617,8 @@ static int send_one_task(struct vine_manager *q, int *tasks_ready_left_to_consid struct vine_task *t; while (tasks_considered < tasks_to_consider) { + timestamp_t time_when_scheduling_start = timestamp_get(); + t = priority_queue_pop(q->ready_tasks); if (!t) { break; @@ -3649,6 +3651,7 @@ static int send_one_task(struct vine_manager *q, int *tasks_ready_left_to_consid switch (result) { case VINE_SUCCESS: committed_tasks++; + t->time_spent_on_scheduling = timestamp_get() - time_when_scheduling_start; break; case VINE_APP_FAILURE: case VINE_WORKER_FAILURE: diff --git a/taskvine/src/manager/vine_task.c b/taskvine/src/manager/vine_task.c index c0db71c030..cde256c47a 100644 --- a/taskvine/src/manager/vine_task.c +++ b/taskvine/src/manager/vine_task.c @@ -153,6 +153,8 @@ void vine_task_reset(struct vine_task *t) t->time_workers_execute_exhaustion = 0; t->time_workers_execute_failure = 0; + t->time_spent_on_scheduling = 0; + rmsummary_delete(t->resources_measured); rmsummary_delete(t->resources_allocated); t->resources_measured = rmsummary_create(-1); diff --git a/taskvine/src/manager/vine_task.h b/taskvine/src/manager/vine_task.h index d97e0b9530..d76d38e72b 100644 --- a/taskvine/src/manager/vine_task.h +++ b/taskvine/src/manager/vine_task.h @@ -122,6 +122,8 @@ struct vine_task { timestamp_t time_workers_execute_exhaustion; /**< Accumulated time spent in attempts that exhausted resources. */ timestamp_t time_workers_execute_failure; /**< Accumulated time for runs that terminated in worker failure/disconnection. */ + timestamp_t time_spent_on_scheduling; /**< Time spent on scheduling the task, only consider the successful scheduling. */ + int64_t bytes_received; /**< Number of bytes received since task has last started receiving input data. */ int64_t bytes_sent; /**< Number of bytes sent since task has last started sending input data. */ int64_t bytes_transferred; /**< Number of bytes transferred since task has last started transferring input data. */ From c107a9cdc516d891748c129b6375edefacab04ed Mon Sep 17 00:00:00 2001 From: Jin Zhou Date: Wed, 22 Oct 2025 12:05:39 -0400 Subject: [PATCH 060/113] gpus --- taskvine/src/manager/vine_schedule.c | 3 ++- 1 file changed, 2 insertions(+), 1 deletion(-) diff --git a/taskvine/src/manager/vine_schedule.c b/taskvine/src/manager/vine_schedule.c index b60c86e4d5..f4ce49f2c5 100644 --- a/taskvine/src/manager/vine_schedule.c +++ b/taskvine/src/manager/vine_schedule.c @@ -189,7 +189,8 @@ int vine_schedule_count_committable_cores(struct vine_manager *q) } /* count the number of free gpus */ if (w->resources->gpus.total > 0 && overcommitted_resource_total(q, w->resources->gpus.total) > w->resources->gpus.inuse) { - count += overcommitted_resource_total(q, w->resources->gpus.total) - w->resources->gpus.inuse; + // Don't count gpus for now, because the manager has not yet fully supported scheduling tasks to GPUs. + // count += overcommitted_resource_total(q, w->resources->gpus.total) - w->resources->gpus.inuse; } } From cc2710977fd4b7f68a71185cb4a0325faa4bccc1 Mon Sep 17 00:00:00 2001 From: Jin Zhou Date: Wed, 29 Oct 2025 10:14:43 -0400 Subject: [PATCH 061/113] time metrics --- .../vinedag/strategic_orchestration_graph.c | 106 ++++- .../vinedag/strategic_orchestration_graph.h | 5 + .../vinedag/strategic_orchestration_node.c | 3 + .../vinedag/strategic_orchestration_node.h | 3 + taskvine/src/graph/vinedag/vinedag.py | 3 + taskvine/src/manager/vine_manager.c | 6 +- taskvine/src/manager/vine_manager_put.c | 371 +++++++++++++++++- 7 files changed, 490 insertions(+), 7 deletions(-) diff --git a/taskvine/src/graph/vinedag/strategic_orchestration_graph.c b/taskvine/src/graph/vinedag/strategic_orchestration_graph.c index 216cfe23b5..895502ef23 100644 --- a/taskvine/src/graph/vinedag/strategic_orchestration_graph.c +++ b/taskvine/src/graph/vinedag/strategic_orchestration_graph.c @@ -532,6 +532,41 @@ static void prune_ancestors_of_node(struct strategic_orchestration_graph *sog, s return; } +/** + * Print the time metrics of the strategic orchestration graph to a csv file. + * @param sog Reference to the strategic orchestration graph object. + * @param filename Reference to the filename of the csv file. + */ +static void print_time_metrics(struct strategic_orchestration_graph *sog, const char *filename) +{ + if (!sog) { + return; + } + + /* first delete the file if it exists */ + if (access(filename, F_OK) != -1) { + unlink(filename); + } + + /* print the header as a csv file */ + FILE *fp = fopen(filename, "w"); + if (!fp) { + debug(D_ERROR, "failed to open file %s", filename); + return; + } + fprintf(fp, "node_key,submission_time_us,scheduling_time_us,commit_time_us,execution_time_us,retrieval_time_us,pruning_time_us,postprocessing_time_us\n"); + + char *node_key; + struct strategic_orchestration_node *node; + HASH_TABLE_ITERATE(sog->nodes, node_key, node) + { + fprintf(fp, "%s,%lu,%lu,%lu,%lu,%lu,%lu,%lu\n", node_key, node->submission_time, node->scheduling_time, node->commit_time, node->execution_time, node->retrieval_time, node->pruning_time, node->postprocessing_time); + } + fclose(fp); + + return; +} + /*************************************************************/ /* Public APIs */ /*************************************************************/ @@ -606,6 +641,52 @@ int sog_tune(struct strategic_orchestration_graph *sog, const char *name, const double val = atof(value); sog->progress_bar_update_interval_sec = (val > 0.0) ? val : 0.1; + } else if (strcmp(name, "time-metrics-filename") == 0) { + if (strcmp(value, "0") == 0) { + return 0; + } + + if (sog->time_metrics_filename) { + free(sog->time_metrics_filename); + } + + sog->time_metrics_filename = xxstrdup(value); + + /** Extract parent directory inline **/ + const char *slash = strrchr(sog->time_metrics_filename, '/'); + if (slash) { + size_t len = slash - sog->time_metrics_filename; + char *parent = malloc(len + 1); + memcpy(parent, sog->time_metrics_filename, len); + parent[len] = '\0'; + + /** Ensure the parent directory exists **/ + if (mkdir(parent, 0777) != 0 && errno != EEXIST) { + debug(D_ERROR, "failed to mkdir %s (errno=%d)", parent, errno); + free(parent); + return -1; + } + free(parent); + } + + /** Truncate or create the file **/ + FILE *fp = fopen(sog->time_metrics_filename, "w"); + if (!fp) { + debug(D_ERROR, "failed to create file %s (errno=%d)", sog->time_metrics_filename, errno); + return -1; + } + fclose(fp); + + } else if (strcmp(name, "enable-debug-log") == 0) { + if (sog->enable_debug_log == 0) { + return -1; + } + sog->enable_debug_log = (atoi(value) == 1) ? 1 : 0; + if (sog->enable_debug_log == 0) { + debug_flags_clear(); + debug_close(); + } + } else { debug(D_ERROR, "invalid parameter name: %s", name); return -1; @@ -971,6 +1052,10 @@ struct strategic_orchestration_graph *sog_create(struct vine_manager *q) vine_enable_debug_log(debug_tmp); free(debug_tmp); + sog->time_metrics_filename = NULL; + + sog->enable_debug_log = 1; + return sog; } @@ -1092,7 +1177,7 @@ void sog_execute(struct strategic_orchestration_graph *sog) progress_bar_bind_part(pbar, regular_tasks_part); progress_bar_bind_part(pbar, recovery_tasks_part); - int wait_timeout = 2; + int wait_timeout = 1; while (regular_tasks_part->current < regular_tasks_part->total) { if (interrupted) { @@ -1105,6 +1190,8 @@ void sog_execute(struct strategic_orchestration_graph *sog) /* retrieve all possible tasks */ wait_timeout = 0; + timestamp_t time_when_postprocessing_start = timestamp_get(); + /* get the original node by task id */ struct strategic_orchestration_node *node = get_node_by_task(sog, task); if (!node) { @@ -1155,7 +1242,10 @@ void sog_execute(struct strategic_orchestration_graph *sog) /* mark the node as completed */ node->completed = 1; - + node->scheduling_time = task->time_spent_on_scheduling; + node->commit_time = task->time_when_commit_end - task->time_when_commit_start; + node->execution_time = task->time_workers_execute_last; + node->retrieval_time = task->time_when_done - task->time_when_retrieval; /* prune nodes on task completion */ prune_ancestors_of_node(sog, node); @@ -1172,7 +1262,7 @@ void sog_execute(struct strategic_orchestration_graph *sog) } /* update critical time */ - son_update_critical_path_time(node, task->time_workers_execute_last); + son_update_critical_path_time(node, node->execution_time); /* mark this regular task as completed */ progress_bar_update_part(pbar, regular_tasks_part, 1); @@ -1199,8 +1289,11 @@ void sog_execute(struct strategic_orchestration_graph *sog) /* submit children nodes with dependencies all resolved */ submit_unblocked_children(sog, node); + + timestamp_t time_when_postprocessing_end = timestamp_get(); + node->postprocessing_time = time_when_postprocessing_end - time_when_postprocessing_start; } else { - wait_timeout = 2; + wait_timeout = 1; progress_bar_update_part(pbar, recovery_tasks_part, 0); // refresh the time and total for recovery tasks } } @@ -1216,6 +1309,7 @@ void sog_execute(struct strategic_orchestration_graph *sog) total_time_spent_on_unlink_local_files += node->time_spent_on_unlink_local_files; total_time_spent_on_prune_ancestors_of_temp_node += node->time_spent_on_prune_ancestors_of_temp_node; total_time_spent_on_prune_ancestors_of_persisted_node += node->time_spent_on_prune_ancestors_of_persisted_node; + node->pruning_time = node->time_spent_on_unlink_local_files + node->time_spent_on_prune_ancestors_of_temp_node + node->time_spent_on_prune_ancestors_of_persisted_node; } total_time_spent_on_unlink_local_files /= 1e6; total_time_spent_on_prune_ancestors_of_temp_node /= 1e6; @@ -1225,6 +1319,10 @@ void sog_execute(struct strategic_orchestration_graph *sog) debug(D_VINE, "total time spent on prune ancestors of persisted node: %.6f seconds\n", total_time_spent_on_prune_ancestors_of_persisted_node); debug(D_VINE, "total time spent on unlink local files: %.6f seconds\n", total_time_spent_on_unlink_local_files); + if (sog->time_metrics_filename) { + print_time_metrics(sog, sog->time_metrics_filename); + } + return; } diff --git a/taskvine/src/graph/vinedag/strategic_orchestration_graph.h b/taskvine/src/graph/vinedag/strategic_orchestration_graph.h index 840f64cacd..0cbf29d0ef 100644 --- a/taskvine/src/graph/vinedag/strategic_orchestration_graph.h +++ b/taskvine/src/graph/vinedag/strategic_orchestration_graph.h @@ -55,6 +55,11 @@ struct strategic_orchestration_graph { double failure_injection_step_percent; /* 0 - 100, the percentage of steps to inject failure */ double progress_bar_update_interval_sec; /* update interval for the progress bar in seconds */ + + /* The filename of the csv file to store the time metrics of the strategic orchestration graph. */ + char *time_metrics_filename; + + int enable_debug_log; /* whether to enable debug log */ }; /* Public APIs for operating the strategic orchestration graph */ diff --git a/taskvine/src/graph/vinedag/strategic_orchestration_node.c b/taskvine/src/graph/vinedag/strategic_orchestration_node.c index 8fb47a8a92..3735a92fd8 100644 --- a/taskvine/src/graph/vinedag/strategic_orchestration_node.c +++ b/taskvine/src/graph/vinedag/strategic_orchestration_node.c @@ -155,8 +155,11 @@ struct strategic_orchestration_node *son_create(const char *node_key, int is_tar node->submission_time = 0; node->scheduling_time = 0; + node->commit_time = 0; node->execution_time = 0; node->retrieval_time = 0; + node->pruning_time = 0; + node->postprocessing_time = 0; node->critical_path_time = -1; diff --git a/taskvine/src/graph/vinedag/strategic_orchestration_node.h b/taskvine/src/graph/vinedag/strategic_orchestration_node.h index 32c347caf4..6ca4ca1c99 100644 --- a/taskvine/src/graph/vinedag/strategic_orchestration_node.h +++ b/taskvine/src/graph/vinedag/strategic_orchestration_node.h @@ -58,8 +58,11 @@ struct strategic_orchestration_node { timestamp_t submission_time; timestamp_t scheduling_time; + timestamp_t commit_time; timestamp_t execution_time; timestamp_t retrieval_time; + timestamp_t pruning_time; + timestamp_t postprocessing_time; node_outfile_type_t outfile_type; prune_status_t prune_status; diff --git a/taskvine/src/graph/vinedag/vinedag.py b/taskvine/src/graph/vinedag/vinedag.py index e1bf7838bd..ce4a46511a 100644 --- a/taskvine/src/graph/vinedag/vinedag.py +++ b/taskvine/src/graph/vinedag/vinedag.py @@ -11,6 +11,7 @@ import os import signal import json +import tqdm try: import dask @@ -99,6 +100,8 @@ def __init__(self): "checkpoint-dir": "./checkpoints", "checkpoint-fraction": 0, "progress-bar-update-interval-sec": 0.1, + "time-metrics-filename": "time_metrics.csv", + "enable-debug-log": 1, } self.other_params = { "schedule": "worst", diff --git a/taskvine/src/manager/vine_manager.c b/taskvine/src/manager/vine_manager.c index bdc938a088..91495b1308 100644 --- a/taskvine/src/manager/vine_manager.c +++ b/taskvine/src/manager/vine_manager.c @@ -3143,7 +3143,7 @@ static vine_result_code_t commit_task_to_worker(struct vine_manager *q, struct v /* Kill unused libraries on this worker to reclaim resources. */ /* Matches assumption in vine_schedule.c:check_worker_have_enough_resources() */ - kill_empty_libraries_on_worker(q, w, t); + // kill_empty_libraries_on_worker(q, w, t); /* If this is a function needing a library, dispatch the library. */ if (t->needs_library) { @@ -3760,6 +3760,8 @@ static int send_one_task(struct vine_manager *q, int *tasks_ready_left_to_consid /* select a worker for the task */ struct vine_worker_info *w = vine_schedule_task_to_worker(q, t); + timestamp_t time_when_scheduling_end = timestamp_get(); + /* task is runnable but no worker is fit, silently skip it */ if (!w) { list_push_tail(skipped_tasks, t); @@ -3777,7 +3779,7 @@ static int send_one_task(struct vine_manager *q, int *tasks_ready_left_to_consid switch (result) { case VINE_SUCCESS: committed_tasks++; - t->time_spent_on_scheduling = timestamp_get() - time_when_scheduling_start; + t->time_spent_on_scheduling = time_when_scheduling_end - time_when_scheduling_start; break; case VINE_APP_FAILURE: case VINE_WORKER_FAILURE: diff --git a/taskvine/src/manager/vine_manager_put.c b/taskvine/src/manager/vine_manager_put.c index 9076124698..d25223b74d 100644 --- a/taskvine/src/manager/vine_manager_put.c +++ b/taskvine/src/manager/vine_manager_put.c @@ -476,7 +476,7 @@ It does not perform any resource management. This allows it to be used for both regular tasks and mini tasks. */ -vine_result_code_t vine_manager_put_task( +vine_result_code_t vine_manager_put_task_old( struct vine_manager *q, struct vine_worker_info *w, struct vine_task *t, const char *command_line, struct rmsummary *limits, struct vine_file *target) { if (target) { @@ -590,3 +590,372 @@ vine_result_code_t vine_manager_put_task( return VINE_WORKER_FAILURE; } } + +/* + * Extreme-optimized implementation of vine_manager_put_task() + * + * Key ideas: + * - Build the entire task definition (header + command payload + tail) + * in one contiguous growable buffer. + * - Send the full buffer in one (or very few) link_putlstring() calls. + * - No nested functions, no temp stack buffers, no redundant syscalls. + * - Protocol and API remain fully backward compatible. + * - Achieves order-of-magnitude throughput improvement for short tasks. + */ + + #include + #include + #include + #include + #include + #include + + /* ---------------------------------------------------------------------- */ + /* Small dynamic buffer utility (local to this file) */ + /* ---------------------------------------------------------------------- */ + + struct sbuf { + char *buf; + size_t len; + size_t cap; + }; + + #define SB_INIT_CAP 4096 + #define SB_MAX_CHUNK (1024 * 1024) /* 1 MB max per write */ + + /* Reserve space for additional bytes. Grows buffer exponentially. */ + #define SB_RESERVE(B, NEED) do { \ + size_t need = (B).len + (size_t)(NEED) + 1; \ + if (need > (B).cap) { \ + size_t cap = (B).cap ? (B).cap : SB_INIT_CAP; \ + while (cap < need) cap <<= 1; \ + char *nb = (char*)realloc((B).buf, cap); \ + if (!nb) { free((B).buf); return VINE_WORKER_FAILURE; } \ + (B).buf = nb; (B).cap = cap; \ + } \ + } while (0) + + /* Append formatted text directly into buffer. */ + static inline int sb_vprintf(struct sbuf *B, const char *fmt, va_list ap0) + { + va_list ap; + va_copy(ap, ap0); + + size_t avail = (B->cap > B->len) ? (B->cap - B->len) : 0; + if (avail < 64) { SB_RESERVE((*B), 64); avail = B->cap - B->len; } + + int n = vsnprintf(B->buf + B->len, avail, fmt, ap); + va_end(ap); + if (n < 0) return -1; + + /* If output was truncated, expand and retry */ + if ((size_t)n >= avail) { + SB_RESERVE((*B), (size_t)n); + va_copy(ap, ap0); + n = vsnprintf(B->buf + B->len, B->cap - B->len, fmt, ap); + va_end(ap); + if (n < 0) return -1; + } + + B->len += (size_t)n; + return n; + } + + /* Append formatted string. */ + static inline int sb_printf(struct sbuf *B, const char *fmt, ...) + { + va_list ap; va_start(ap, fmt); + int n = sb_vprintf(B, fmt, ap); + va_end(ap); + return n; + } + + /* Append raw bytes. */ + static inline int sb_append(struct sbuf *B, const void *p, size_t n) + { + SB_RESERVE((*B), n); + memcpy(B->buf + B->len, p, n); + B->len += n; + B->buf[B->len] = '\0'; + return 0; + } + + /* ---------------------------------------------------------------------- */ + /* Main function: vine_manager_put_task() */ + /* ---------------------------------------------------------------------- */ + + vine_result_code_t vine_manager_put_task_old2( + struct vine_manager *q, struct vine_worker_info *w, + struct vine_task *t, const char *command_line, + struct rmsummary *limits, struct vine_file *target) + { + /* Check if target already exists on worker */ + if (target && vine_file_replica_table_lookup(w, target->cached_name)) { + debug(D_NOTICE, "mini_task %s already at %s", target->cached_name, w->addrport); + return VINE_SUCCESS; + } + + /* Send input files first (unchanged path) */ + vine_result_code_t result = vine_manager_put_input_files(q, w, t); + if (result != VINE_SUCCESS) + return result; + + if (!command_line) + command_line = t->command_line; + const long long cmd_len = (long long)strlen(command_line); + + struct sbuf sb = {0}; + + /* -------- Header: task/mini_task + cmd length -------- */ + if (target) { + int mode = target->mode ? target->mode : 0755; + sb_printf(&sb, "mini_task %s %s %d %lld 0%o\n", + target->source, target->cached_name, + target->cache_level, (long long)target->size, mode); + } else { + sb_printf(&sb, "task %lld\n", (long long)t->task_id); + } + sb_printf(&sb, "cmd %lld\n", cmd_len); + + /* -------- Append command payload directly -------- */ + sb_append(&sb, command_line, (size_t)cmd_len); + sb_printf(&sb, "\n"); + + /* -------- Tail: metadata, env, mounts, etc. -------- */ + if (t->needs_library) + sb_printf(&sb, "needs_library %s\n", t->needs_library); + + if (t->provides_library) { + sb_printf(&sb, "provides_library %s\n", t->provides_library); + sb_printf(&sb, "function_slots %d\n", t->function_slots_total); + sb_printf(&sb, "func_exec_mode %d\n", t->func_exec_mode); + } + + sb_printf(&sb, "category %s\n", t->category); + + if (limits) { + sb_printf(&sb, "cores %s\n", rmsummary_resource_to_str("cores", limits->cores, 0)); + sb_printf(&sb, "gpus %s\n", rmsummary_resource_to_str("gpus", limits->gpus, 0)); + sb_printf(&sb, "memory %s\n", rmsummary_resource_to_str("memory", limits->memory, 0)); + sb_printf(&sb, "disk %s\n", rmsummary_resource_to_str("disk", limits->disk, 0)); + + if (q->monitor_mode == VINE_MON_DISABLED) { + if (limits->end > 0) + sb_printf(&sb, "end_time %s\n", rmsummary_resource_to_str("end", limits->end, 0)); + if (limits->wall_time > 0) + sb_printf(&sb, "wall_time %s\n", rmsummary_resource_to_str("wall_time", limits->wall_time, 0)); + } + } + + char *var; + LIST_ITERATE(t->env_list, var) { + size_t L = strlen(var); + sb_printf(&sb, "env %zu\n", L); + sb_append(&sb, var, L); + sb_printf(&sb, "\n"); + } + + if (t->input_mounts) { + struct vine_mount *m; + LIST_ITERATE(t->input_mounts, m) { + char enc[PATH_MAX]; + url_encode(m->remote_name, enc, PATH_MAX); + sb_printf(&sb, "infile %s %s %d\n", m->file->cached_name, enc, m->flags); + } + } + + if (t->output_mounts) { + struct vine_mount *m; + LIST_ITERATE(t->output_mounts, m) { + char enc[PATH_MAX]; + url_encode(m->remote_name, enc, PATH_MAX); + sb_printf(&sb, "outfile %s %s %d\n", m->file->cached_name, enc, m->flags); + } + } + + if (t->group_id) + sb_printf(&sb, "groupid %d\n", t->group_id); + + sb_printf(&sb, "end\n"); + + // debug(D_VINE, "sending task %lld (%zu bytes)\n", (long long)t->task_id, sb.len); + + /* -------- Send in one or very few chunks -------- */ + const char *p = sb.buf; + size_t left = sb.len; + time_t deadline = time(0) + q->short_timeout; + int r = 0; + + while (left > 0) { + size_t chunk = left > SB_MAX_CHUNK ? SB_MAX_CHUNK : left; + r = link_putlstring(w->link, p, chunk, deadline); + if (r < 0) + break; + p += chunk; + left -= chunk; + } + + /* Finalize and cleanup */ + if (r >= 0 && left == 0) { + if (target) + vine_file_replica_table_get_or_create(q, w, target->cached_name, + target->type, target->cache_level, target->size, target->mtime); + free(sb.buf); + return VINE_SUCCESS; + } else { + free(sb.buf); + return VINE_WORKER_FAILURE; + } + + #undef SB_RESERVE + } + + + vine_result_code_t vine_manager_put_task( + struct vine_manager *q, struct vine_worker_info *w, + struct vine_task *t, const char *command_line, + struct rmsummary *limits, struct vine_file *target) +{ + /* If target already on worker */ + if (target && vine_file_replica_table_lookup(w, target->cached_name)) { + debug(D_NOTICE, "mini_task %s already at %s", target->cached_name, w->addrport); + return VINE_SUCCESS; + } + + /* Send input files first */ + vine_result_code_t result = vine_manager_put_input_files(q, w, t); + if (result != VINE_SUCCESS) return result; + + if (!command_line) command_line = t->command_line; + const long long cmd_len = (long long)strlen(command_line); + + struct sbuf sb = {0}; + + /* helper: exact same debug prefix format as vine_manager_send() */ + auto void dbg_tx(struct vine_worker_info *ww, const char *s, size_t n) { + if (!s || n == 0) return; + debug(D_VINE, "tx to %s (%s): %.*s", ww->hostname, ww->addrport, (int)n, s); + } + + /* helper: format one logical line, append to sb, and debug it */ + #ifndef VINE_LINE_MAX + #define VINE_LINE_MAX 65536 + #endif + auto void emitf(const char *fmt, ...) { + char line[VINE_LINE_MAX]; + va_list ap; + va_start(ap, fmt); + int n = vsnprintf(line, sizeof(line), fmt, ap); + va_end(ap); + if (n < 0) n = 0; + size_t len = (size_t)n; + if (len >= sizeof(line)) len = sizeof(line) - 1; /* truncate for debug parity safety */ + sb_printf(&sb, "%.*s", (int)len, line); + dbg_tx(w, line, len); + } + + /* Header */ + if (target) { + int mode = target->mode ? target->mode : 0755; + emitf("mini_task %s %s %d %lld 0%o\n", + target->source, target->cached_name, + target->cache_level, (long long)target->size, mode); + } else { + emitf("task %lld\n", (long long)t->task_id); + } + + /* cmd header + payload */ + emitf("cmd %lld\n", cmd_len); + sb_append(&sb, command_line, (size_t)cmd_len); + dbg_tx(w, command_line, (size_t)cmd_len); + emitf("\n"); + + /* extras */ + if (t->needs_library) { + emitf("needs_library %s\n", t->needs_library); + } + + if (t->provides_library) { + emitf("provides_library %s\n", t->provides_library); + emitf("function_slots %d\n", t->function_slots_total); + emitf("func_exec_mode %d\n", t->func_exec_mode); + } + + emitf("category %s\n", t->category); + + if (limits) { + emitf("cores %s\n", rmsummary_resource_to_str("cores", limits->cores, 0)); + emitf("gpus %s\n", rmsummary_resource_to_str("gpus", limits->gpus, 0)); + emitf("memory %s\n", rmsummary_resource_to_str("memory", limits->memory, 0)); + emitf("disk %s\n", rmsummary_resource_to_str("disk", limits->disk, 0)); + + if (q->monitor_mode == VINE_MON_DISABLED) { + if (limits->end > 0) + emitf("end_time %s\n", rmsummary_resource_to_str("end", limits->end, 0)); + if (limits->wall_time > 0) + emitf("wall_time %s\n", rmsummary_resource_to_str("wall_time", limits->wall_time, 0)); + } + } + + /* env list */ + char *var; + LIST_ITERATE(t->env_list, var) { + size_t L = strlen(var); + emitf("env %zu\n", L); + sb_append(&sb, var, L); + dbg_tx(w, var, L); + emitf("\n"); + } + + /* infile mounts */ + if (t->input_mounts) { + struct vine_mount *m; + LIST_ITERATE(t->input_mounts, m) { + char enc[PATH_MAX]; + url_encode(m->remote_name, enc, PATH_MAX); + emitf("infile %s %s %d\n", m->file->cached_name, enc, m->flags); + } + } + + /* outfile mounts */ + if (t->output_mounts) { + struct vine_mount *m; + LIST_ITERATE(t->output_mounts, m) { + char enc[PATH_MAX]; + url_encode(m->remote_name, enc, PATH_MAX); + emitf("outfile %s %s %d\n", m->file->cached_name, enc, m->flags); + } + } + + if (t->group_id) { + emitf("groupid %d\n", t->group_id); + } + + emitf("end\n"); + + /* Send in chunks (no extra debug here to avoid duplicate logs) */ + const char *p = sb.buf; + size_t left = sb.len; + time_t deadline = time(0) + q->short_timeout; + int r = 0; + + while (left > 0) { + size_t chunk = left > SB_MAX_CHUNK ? SB_MAX_CHUNK : left; + r = link_putlstring(w->link, p, chunk, deadline); + if (r < 0) break; + p += chunk; + left -= chunk; + } + + if (r >= 0 && left == 0) { + if (target) { + vine_file_replica_table_get_or_create(q, w, target->cached_name, + target->type, target->cache_level, target->size, target->mtime); + } + free(sb.buf); + return VINE_SUCCESS; + } else { + free(sb.buf); + return VINE_WORKER_FAILURE; + } +} From a9b95b20cedbb7e7e1952aa93255855d584077eb Mon Sep 17 00:00:00 2001 From: Jin Zhou Date: Wed, 29 Oct 2025 12:17:03 -0400 Subject: [PATCH 062/113] optimized put --- taskvine/src/manager/vine_manager_put.c | 606 ++++++++++-------------- 1 file changed, 252 insertions(+), 354 deletions(-) diff --git a/taskvine/src/manager/vine_manager_put.c b/taskvine/src/manager/vine_manager_put.c index d25223b74d..0b95d62c31 100644 --- a/taskvine/src/manager/vine_manager_put.c +++ b/taskvine/src/manager/vine_manager_put.c @@ -603,359 +603,257 @@ vine_result_code_t vine_manager_put_task_old( * - Achieves order-of-magnitude throughput improvement for short tasks. */ - #include - #include - #include - #include - #include - #include - - /* ---------------------------------------------------------------------- */ - /* Small dynamic buffer utility (local to this file) */ - /* ---------------------------------------------------------------------- */ - - struct sbuf { - char *buf; - size_t len; - size_t cap; - }; - - #define SB_INIT_CAP 4096 - #define SB_MAX_CHUNK (1024 * 1024) /* 1 MB max per write */ - - /* Reserve space for additional bytes. Grows buffer exponentially. */ - #define SB_RESERVE(B, NEED) do { \ - size_t need = (B).len + (size_t)(NEED) + 1; \ - if (need > (B).cap) { \ - size_t cap = (B).cap ? (B).cap : SB_INIT_CAP; \ - while (cap < need) cap <<= 1; \ - char *nb = (char*)realloc((B).buf, cap); \ - if (!nb) { free((B).buf); return VINE_WORKER_FAILURE; } \ - (B).buf = nb; (B).cap = cap; \ - } \ - } while (0) - - /* Append formatted text directly into buffer. */ - static inline int sb_vprintf(struct sbuf *B, const char *fmt, va_list ap0) - { - va_list ap; - va_copy(ap, ap0); - - size_t avail = (B->cap > B->len) ? (B->cap - B->len) : 0; - if (avail < 64) { SB_RESERVE((*B), 64); avail = B->cap - B->len; } - - int n = vsnprintf(B->buf + B->len, avail, fmt, ap); - va_end(ap); - if (n < 0) return -1; - - /* If output was truncated, expand and retry */ - if ((size_t)n >= avail) { - SB_RESERVE((*B), (size_t)n); - va_copy(ap, ap0); - n = vsnprintf(B->buf + B->len, B->cap - B->len, fmt, ap); - va_end(ap); - if (n < 0) return -1; - } - - B->len += (size_t)n; - return n; - } - - /* Append formatted string. */ - static inline int sb_printf(struct sbuf *B, const char *fmt, ...) - { - va_list ap; va_start(ap, fmt); - int n = sb_vprintf(B, fmt, ap); - va_end(ap); - return n; - } - - /* Append raw bytes. */ - static inline int sb_append(struct sbuf *B, const void *p, size_t n) - { - SB_RESERVE((*B), n); - memcpy(B->buf + B->len, p, n); - B->len += n; - B->buf[B->len] = '\0'; - return 0; - } - - /* ---------------------------------------------------------------------- */ - /* Main function: vine_manager_put_task() */ - /* ---------------------------------------------------------------------- */ - - vine_result_code_t vine_manager_put_task_old2( - struct vine_manager *q, struct vine_worker_info *w, - struct vine_task *t, const char *command_line, - struct rmsummary *limits, struct vine_file *target) - { - /* Check if target already exists on worker */ - if (target && vine_file_replica_table_lookup(w, target->cached_name)) { - debug(D_NOTICE, "mini_task %s already at %s", target->cached_name, w->addrport); - return VINE_SUCCESS; - } - - /* Send input files first (unchanged path) */ - vine_result_code_t result = vine_manager_put_input_files(q, w, t); - if (result != VINE_SUCCESS) - return result; - - if (!command_line) - command_line = t->command_line; - const long long cmd_len = (long long)strlen(command_line); - - struct sbuf sb = {0}; - - /* -------- Header: task/mini_task + cmd length -------- */ - if (target) { - int mode = target->mode ? target->mode : 0755; - sb_printf(&sb, "mini_task %s %s %d %lld 0%o\n", - target->source, target->cached_name, - target->cache_level, (long long)target->size, mode); - } else { - sb_printf(&sb, "task %lld\n", (long long)t->task_id); - } - sb_printf(&sb, "cmd %lld\n", cmd_len); - - /* -------- Append command payload directly -------- */ - sb_append(&sb, command_line, (size_t)cmd_len); - sb_printf(&sb, "\n"); - - /* -------- Tail: metadata, env, mounts, etc. -------- */ - if (t->needs_library) - sb_printf(&sb, "needs_library %s\n", t->needs_library); - - if (t->provides_library) { - sb_printf(&sb, "provides_library %s\n", t->provides_library); - sb_printf(&sb, "function_slots %d\n", t->function_slots_total); - sb_printf(&sb, "func_exec_mode %d\n", t->func_exec_mode); - } - - sb_printf(&sb, "category %s\n", t->category); - - if (limits) { - sb_printf(&sb, "cores %s\n", rmsummary_resource_to_str("cores", limits->cores, 0)); - sb_printf(&sb, "gpus %s\n", rmsummary_resource_to_str("gpus", limits->gpus, 0)); - sb_printf(&sb, "memory %s\n", rmsummary_resource_to_str("memory", limits->memory, 0)); - sb_printf(&sb, "disk %s\n", rmsummary_resource_to_str("disk", limits->disk, 0)); - - if (q->monitor_mode == VINE_MON_DISABLED) { - if (limits->end > 0) - sb_printf(&sb, "end_time %s\n", rmsummary_resource_to_str("end", limits->end, 0)); - if (limits->wall_time > 0) - sb_printf(&sb, "wall_time %s\n", rmsummary_resource_to_str("wall_time", limits->wall_time, 0)); - } - } - - char *var; - LIST_ITERATE(t->env_list, var) { - size_t L = strlen(var); - sb_printf(&sb, "env %zu\n", L); - sb_append(&sb, var, L); - sb_printf(&sb, "\n"); - } - - if (t->input_mounts) { - struct vine_mount *m; - LIST_ITERATE(t->input_mounts, m) { - char enc[PATH_MAX]; - url_encode(m->remote_name, enc, PATH_MAX); - sb_printf(&sb, "infile %s %s %d\n", m->file->cached_name, enc, m->flags); - } - } - - if (t->output_mounts) { - struct vine_mount *m; - LIST_ITERATE(t->output_mounts, m) { - char enc[PATH_MAX]; - url_encode(m->remote_name, enc, PATH_MAX); - sb_printf(&sb, "outfile %s %s %d\n", m->file->cached_name, enc, m->flags); - } - } - - if (t->group_id) - sb_printf(&sb, "groupid %d\n", t->group_id); - - sb_printf(&sb, "end\n"); - - // debug(D_VINE, "sending task %lld (%zu bytes)\n", (long long)t->task_id, sb.len); - - /* -------- Send in one or very few chunks -------- */ - const char *p = sb.buf; - size_t left = sb.len; - time_t deadline = time(0) + q->short_timeout; - int r = 0; - - while (left > 0) { - size_t chunk = left > SB_MAX_CHUNK ? SB_MAX_CHUNK : left; - r = link_putlstring(w->link, p, chunk, deadline); - if (r < 0) - break; - p += chunk; - left -= chunk; - } - - /* Finalize and cleanup */ - if (r >= 0 && left == 0) { - if (target) - vine_file_replica_table_get_or_create(q, w, target->cached_name, - target->type, target->cache_level, target->size, target->mtime); - free(sb.buf); - return VINE_SUCCESS; - } else { - free(sb.buf); - return VINE_WORKER_FAILURE; - } - - #undef SB_RESERVE - } - - - vine_result_code_t vine_manager_put_task( - struct vine_manager *q, struct vine_worker_info *w, - struct vine_task *t, const char *command_line, - struct rmsummary *limits, struct vine_file *target) +#include +#include +#include +#include +#include +#include + +/* ---------------------------------------------------------------------- */ +/* Small dynamic buffer utility (local to this file) */ +/* ---------------------------------------------------------------------- */ + +struct sbuf { + char *buf; + size_t len; + size_t cap; +}; + +#define SB_INIT_CAP 4096 +#define SB_MAX_CHUNK (1024 * 1024) /* 1 MB max per write */ + +/* Reserve space for additional bytes. Grows buffer exponentially. */ +#define SB_RESERVE(B, NEED) \ + do { \ + size_t need = (B).len + (size_t)(NEED) + 1; \ + if (need > (B).cap) { \ + size_t cap = (B).cap ? (B).cap : SB_INIT_CAP; \ + while (cap < need) \ + cap <<= 1; \ + char *nb = (char *)realloc((B).buf, cap); \ + if (!nb) { \ + free((B).buf); \ + return VINE_WORKER_FAILURE; \ + } \ + (B).buf = nb; \ + (B).cap = cap; \ + } \ + } while (0) + +/* Append formatted text directly into buffer. */ +static inline int sb_vprintf(struct sbuf *B, const char *fmt, va_list ap0) { - /* If target already on worker */ - if (target && vine_file_replica_table_lookup(w, target->cached_name)) { - debug(D_NOTICE, "mini_task %s already at %s", target->cached_name, w->addrport); - return VINE_SUCCESS; - } - - /* Send input files first */ - vine_result_code_t result = vine_manager_put_input_files(q, w, t); - if (result != VINE_SUCCESS) return result; - - if (!command_line) command_line = t->command_line; - const long long cmd_len = (long long)strlen(command_line); - - struct sbuf sb = {0}; - - /* helper: exact same debug prefix format as vine_manager_send() */ - auto void dbg_tx(struct vine_worker_info *ww, const char *s, size_t n) { - if (!s || n == 0) return; - debug(D_VINE, "tx to %s (%s): %.*s", ww->hostname, ww->addrport, (int)n, s); - } - - /* helper: format one logical line, append to sb, and debug it */ - #ifndef VINE_LINE_MAX - #define VINE_LINE_MAX 65536 - #endif - auto void emitf(const char *fmt, ...) { - char line[VINE_LINE_MAX]; - va_list ap; - va_start(ap, fmt); - int n = vsnprintf(line, sizeof(line), fmt, ap); - va_end(ap); - if (n < 0) n = 0; - size_t len = (size_t)n; - if (len >= sizeof(line)) len = sizeof(line) - 1; /* truncate for debug parity safety */ - sb_printf(&sb, "%.*s", (int)len, line); - dbg_tx(w, line, len); - } - - /* Header */ - if (target) { - int mode = target->mode ? target->mode : 0755; - emitf("mini_task %s %s %d %lld 0%o\n", - target->source, target->cached_name, - target->cache_level, (long long)target->size, mode); - } else { - emitf("task %lld\n", (long long)t->task_id); - } - - /* cmd header + payload */ - emitf("cmd %lld\n", cmd_len); - sb_append(&sb, command_line, (size_t)cmd_len); - dbg_tx(w, command_line, (size_t)cmd_len); - emitf("\n"); - - /* extras */ - if (t->needs_library) { - emitf("needs_library %s\n", t->needs_library); - } - - if (t->provides_library) { - emitf("provides_library %s\n", t->provides_library); - emitf("function_slots %d\n", t->function_slots_total); - emitf("func_exec_mode %d\n", t->func_exec_mode); - } - - emitf("category %s\n", t->category); - - if (limits) { - emitf("cores %s\n", rmsummary_resource_to_str("cores", limits->cores, 0)); - emitf("gpus %s\n", rmsummary_resource_to_str("gpus", limits->gpus, 0)); - emitf("memory %s\n", rmsummary_resource_to_str("memory", limits->memory, 0)); - emitf("disk %s\n", rmsummary_resource_to_str("disk", limits->disk, 0)); - - if (q->monitor_mode == VINE_MON_DISABLED) { - if (limits->end > 0) - emitf("end_time %s\n", rmsummary_resource_to_str("end", limits->end, 0)); - if (limits->wall_time > 0) - emitf("wall_time %s\n", rmsummary_resource_to_str("wall_time", limits->wall_time, 0)); - } - } - - /* env list */ - char *var; - LIST_ITERATE(t->env_list, var) { - size_t L = strlen(var); - emitf("env %zu\n", L); - sb_append(&sb, var, L); - dbg_tx(w, var, L); - emitf("\n"); - } - - /* infile mounts */ - if (t->input_mounts) { - struct vine_mount *m; - LIST_ITERATE(t->input_mounts, m) { - char enc[PATH_MAX]; - url_encode(m->remote_name, enc, PATH_MAX); - emitf("infile %s %s %d\n", m->file->cached_name, enc, m->flags); - } - } - - /* outfile mounts */ - if (t->output_mounts) { - struct vine_mount *m; - LIST_ITERATE(t->output_mounts, m) { - char enc[PATH_MAX]; - url_encode(m->remote_name, enc, PATH_MAX); - emitf("outfile %s %s %d\n", m->file->cached_name, enc, m->flags); - } - } - - if (t->group_id) { - emitf("groupid %d\n", t->group_id); - } - - emitf("end\n"); - - /* Send in chunks (no extra debug here to avoid duplicate logs) */ - const char *p = sb.buf; - size_t left = sb.len; - time_t deadline = time(0) + q->short_timeout; - int r = 0; - - while (left > 0) { - size_t chunk = left > SB_MAX_CHUNK ? SB_MAX_CHUNK : left; - r = link_putlstring(w->link, p, chunk, deadline); - if (r < 0) break; - p += chunk; - left -= chunk; - } - - if (r >= 0 && left == 0) { - if (target) { - vine_file_replica_table_get_or_create(q, w, target->cached_name, - target->type, target->cache_level, target->size, target->mtime); - } - free(sb.buf); - return VINE_SUCCESS; - } else { - free(sb.buf); - return VINE_WORKER_FAILURE; - } + va_list ap; + va_copy(ap, ap0); + + size_t avail = (B->cap > B->len) ? (B->cap - B->len) : 0; + if (avail < 64) { + SB_RESERVE((*B), 64); + avail = B->cap - B->len; + } + + int n = vsnprintf(B->buf + B->len, avail, fmt, ap); + va_end(ap); + if (n < 0) + return -1; + + /* If output was truncated, expand and retry */ + if ((size_t)n >= avail) { + SB_RESERVE((*B), (size_t)n); + va_copy(ap, ap0); + n = vsnprintf(B->buf + B->len, B->cap - B->len, fmt, ap); + va_end(ap); + if (n < 0) + return -1; + } + + B->len += (size_t)n; + return n; +} + +/* Append formatted string. */ +static inline int sb_printf(struct sbuf *B, const char *fmt, ...) +{ + va_list ap; + va_start(ap, fmt); + int n = sb_vprintf(B, fmt, ap); + va_end(ap); + return n; +} + +/* Append raw bytes. */ +static inline int sb_append(struct sbuf *B, const void *p, size_t n) +{ + SB_RESERVE((*B), n); + memcpy(B->buf + B->len, p, n); + B->len += n; + B->buf[B->len] = '\0'; + return 0; +} + +/* ---------------------------------------------------------------------- */ +/* Main function: vine_manager_put_task() */ +/* ---------------------------------------------------------------------- */ + +vine_result_code_t vine_manager_put_task( + struct vine_manager *q, struct vine_worker_info *w, + struct vine_task *t, const char *command_line, + struct rmsummary *limits, struct vine_file *target) +{ + /* If target already on worker */ + if (target && vine_file_replica_table_lookup(w, target->cached_name)) { + debug(D_NOTICE, "mini_task %s already at %s", target->cached_name, w->addrport); + return VINE_SUCCESS; + } + + /* Send input files first */ + vine_result_code_t result = vine_manager_put_input_files(q, w, t); + if (result != VINE_SUCCESS) + return result; + + if (!command_line) + command_line = t->command_line; + const long long cmd_len = (long long)strlen(command_line); + + struct sbuf sb = {0}; + + /* helper: exact same debug prefix format as vine_manager_send() */ + auto void dbg_tx(struct vine_worker_info * ww, const char *s, size_t n) + { + if (!s || n == 0) + return; + debug(D_VINE, "tx to %s (%s): %.*s", ww->hostname, ww->addrport, (int)n, s); + } + +/* helper: format one logical line, append to sb, and debug it */ +#ifndef VINE_LINE_MAX +#define VINE_LINE_MAX 65536 +#endif + auto void emitf(const char *fmt, ...) + { + char line[VINE_LINE_MAX]; + va_list ap; + va_start(ap, fmt); + int n = vsnprintf(line, sizeof(line), fmt, ap); + va_end(ap); + if (n < 0) + n = 0; + size_t len = (size_t)n; + if (len >= sizeof(line)) + len = sizeof(line) - 1; /* truncate for debug parity safety */ + sb_printf(&sb, "%.*s", (int)len, line); + dbg_tx(w, line, len); + } + + /* Header */ + if (target) { + int mode = target->mode ? target->mode : 0755; + emitf("mini_task %s %s %d %lld 0%o\n", + target->source, + target->cached_name, + target->cache_level, + (long long)target->size, + mode); + } else { + emitf("task %lld\n", (long long)t->task_id); + } + + /* cmd header + payload */ + emitf("cmd %lld\n", cmd_len); + sb_append(&sb, command_line, (size_t)cmd_len); + dbg_tx(w, command_line, (size_t)cmd_len); + emitf("\n"); + + /* extras */ + if (t->needs_library) { + emitf("needs_library %s\n", t->needs_library); + } + + if (t->provides_library) { + emitf("provides_library %s\n", t->provides_library); + emitf("function_slots %d\n", t->function_slots_total); + emitf("func_exec_mode %d\n", t->func_exec_mode); + } + + emitf("category %s\n", t->category); + + if (limits) { + emitf("cores %s\n", rmsummary_resource_to_str("cores", limits->cores, 0)); + emitf("gpus %s\n", rmsummary_resource_to_str("gpus", limits->gpus, 0)); + emitf("memory %s\n", rmsummary_resource_to_str("memory", limits->memory, 0)); + emitf("disk %s\n", rmsummary_resource_to_str("disk", limits->disk, 0)); + + if (q->monitor_mode == VINE_MON_DISABLED) { + if (limits->end > 0) + emitf("end_time %s\n", rmsummary_resource_to_str("end", limits->end, 0)); + if (limits->wall_time > 0) + emitf("wall_time %s\n", rmsummary_resource_to_str("wall_time", limits->wall_time, 0)); + } + } + + /* env list */ + char *var; + LIST_ITERATE(t->env_list, var) + { + size_t L = strlen(var); + emitf("env %zu\n", L); + sb_append(&sb, var, L); + dbg_tx(w, var, L); + emitf("\n"); + } + + /* infile mounts */ + if (t->input_mounts) { + struct vine_mount *m; + LIST_ITERATE(t->input_mounts, m) + { + char enc[PATH_MAX]; + url_encode(m->remote_name, enc, PATH_MAX); + emitf("infile %s %s %d\n", m->file->cached_name, enc, m->flags); + } + } + + /* outfile mounts */ + if (t->output_mounts) { + struct vine_mount *m; + LIST_ITERATE(t->output_mounts, m) + { + char enc[PATH_MAX]; + url_encode(m->remote_name, enc, PATH_MAX); + emitf("outfile %s %s %d\n", m->file->cached_name, enc, m->flags); + } + } + + if (t->group_id) { + emitf("groupid %d\n", t->group_id); + } + + emitf("end\n"); + + /* Send in chunks (no extra debug here to avoid duplicate logs) */ + const char *p = sb.buf; + size_t left = sb.len; + time_t deadline = time(0) + q->short_timeout; + int r = 0; + + while (left > 0) { + size_t chunk = left > SB_MAX_CHUNK ? SB_MAX_CHUNK : left; + r = link_putlstring(w->link, p, chunk, deadline); + if (r < 0) + break; + p += chunk; + left -= chunk; + } + + if (r >= 0 && left == 0) { + if (target) { + vine_file_replica_table_get_or_create(q, w, target->cached_name, target->type, target->cache_level, target->size, target->mtime); + } + free(sb.buf); + return VINE_SUCCESS; + } else { + free(sb.buf); + return VINE_WORKER_FAILURE; + } } From 363b5e7e85348b26ed92c444889fa384a1d46bb3 Mon Sep 17 00:00:00 2001 From: Jin Zhou Date: Wed, 29 Oct 2025 12:17:15 -0400 Subject: [PATCH 063/113] lint --- taskvine/src/bindings/python3/ndcctools/taskvine/utils.py | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/taskvine/src/bindings/python3/ndcctools/taskvine/utils.py b/taskvine/src/bindings/python3/ndcctools/taskvine/utils.py index c11c5d5b0d..b54d9c7bf4 100644 --- a/taskvine/src/bindings/python3/ndcctools/taskvine/utils.py +++ b/taskvine/src/bindings/python3/ndcctools/taskvine/utils.py @@ -37,4 +37,4 @@ def load_variable_from_library(var_name): return globals()[var_name] -# vim: set sts=4 sw=4 ts=4 expandtab ft=python: \ No newline at end of file +# vim: set sts=4 sw=4 ts=4 expandtab ft=python: From c7e9d658fd56588e4a0e3630577c16ad264afce0 Mon Sep 17 00:00:00 2001 From: Jin Zhou Date: Wed, 29 Oct 2025 12:17:46 -0400 Subject: [PATCH 064/113] lint --- taskvine/src/graph/vinedag/vinedag.py | 1 - 1 file changed, 1 deletion(-) diff --git a/taskvine/src/graph/vinedag/vinedag.py b/taskvine/src/graph/vinedag/vinedag.py index ce4a46511a..9ec10aefc3 100644 --- a/taskvine/src/graph/vinedag/vinedag.py +++ b/taskvine/src/graph/vinedag/vinedag.py @@ -11,7 +11,6 @@ import os import signal import json -import tqdm try: import dask From 2faa0d1061e900197a0c70e0c76b514e1ce747a7 Mon Sep 17 00:00:00 2001 From: Jin Zhou Date: Wed, 29 Oct 2025 12:19:33 -0400 Subject: [PATCH 065/113] time metrics --- .../vinedag/strategic_orchestration_graph.c | 17 ++++++++--------- .../vinedag/strategic_orchestration_graph.h | 2 +- .../vinedag/strategic_orchestration_node.c | 1 - .../vinedag/strategic_orchestration_node.h | 1 - taskvine/src/manager/vine_manager.c | 15 ++++++++++----- taskvine/src/manager/vine_task.c | 16 +++++++++++++++- taskvine/src/manager/vine_task.h | 8 ++++++-- 7 files changed, 40 insertions(+), 20 deletions(-) diff --git a/taskvine/src/graph/vinedag/strategic_orchestration_graph.c b/taskvine/src/graph/vinedag/strategic_orchestration_graph.c index 895502ef23..a790c8b39b 100644 --- a/taskvine/src/graph/vinedag/strategic_orchestration_graph.c +++ b/taskvine/src/graph/vinedag/strategic_orchestration_graph.c @@ -554,13 +554,13 @@ static void print_time_metrics(struct strategic_orchestration_graph *sog, const debug(D_ERROR, "failed to open file %s", filename); return; } - fprintf(fp, "node_key,submission_time_us,scheduling_time_us,commit_time_us,execution_time_us,retrieval_time_us,pruning_time_us,postprocessing_time_us\n"); + fprintf(fp, "node_key,submission_time_us,scheduling_time_us,commit_time_us,execution_time_us,retrieval_time_us,postprocessing_time_us\n"); char *node_key; struct strategic_orchestration_node *node; HASH_TABLE_ITERATE(sog->nodes, node_key, node) { - fprintf(fp, "%s,%lu,%lu,%lu,%lu,%lu,%lu,%lu\n", node_key, node->submission_time, node->scheduling_time, node->commit_time, node->execution_time, node->retrieval_time, node->pruning_time, node->postprocessing_time); + fprintf(fp, "%s,%lu,%lu,%lu,%lu,%lu,%lu\n", node_key, node->submission_time, node->scheduling_time, node->commit_time, node->execution_time, node->retrieval_time, node->postprocessing_time); } fclose(fp); @@ -649,9 +649,9 @@ int sog_tune(struct strategic_orchestration_graph *sog, const char *name, const if (sog->time_metrics_filename) { free(sog->time_metrics_filename); } - + sog->time_metrics_filename = xxstrdup(value); - + /** Extract parent directory inline **/ const char *slash = strrchr(sog->time_metrics_filename, '/'); if (slash) { @@ -659,7 +659,7 @@ int sog_tune(struct strategic_orchestration_graph *sog, const char *name, const char *parent = malloc(len + 1); memcpy(parent, sog->time_metrics_filename, len); parent[len] = '\0'; - + /** Ensure the parent directory exists **/ if (mkdir(parent, 0777) != 0 && errno != EEXIST) { debug(D_ERROR, "failed to mkdir %s (errno=%d)", parent, errno); @@ -668,7 +668,7 @@ int sog_tune(struct strategic_orchestration_graph *sog, const char *name, const } free(parent); } - + /** Truncate or create the file **/ FILE *fp = fopen(sog->time_metrics_filename, "w"); if (!fp) { @@ -1242,10 +1242,10 @@ void sog_execute(struct strategic_orchestration_graph *sog) /* mark the node as completed */ node->completed = 1; - node->scheduling_time = task->time_spent_on_scheduling; + node->scheduling_time = task->time_when_scheduling_end - task->time_when_scheduling_start; node->commit_time = task->time_when_commit_end - task->time_when_commit_start; node->execution_time = task->time_workers_execute_last; - node->retrieval_time = task->time_when_done - task->time_when_retrieval; + node->retrieval_time = task->time_when_get_result_end - task->time_when_get_result_start; /* prune nodes on task completion */ prune_ancestors_of_node(sog, node); @@ -1309,7 +1309,6 @@ void sog_execute(struct strategic_orchestration_graph *sog) total_time_spent_on_unlink_local_files += node->time_spent_on_unlink_local_files; total_time_spent_on_prune_ancestors_of_temp_node += node->time_spent_on_prune_ancestors_of_temp_node; total_time_spent_on_prune_ancestors_of_persisted_node += node->time_spent_on_prune_ancestors_of_persisted_node; - node->pruning_time = node->time_spent_on_unlink_local_files + node->time_spent_on_prune_ancestors_of_temp_node + node->time_spent_on_prune_ancestors_of_persisted_node; } total_time_spent_on_unlink_local_files /= 1e6; total_time_spent_on_prune_ancestors_of_temp_node /= 1e6; diff --git a/taskvine/src/graph/vinedag/strategic_orchestration_graph.h b/taskvine/src/graph/vinedag/strategic_orchestration_graph.h index 0cbf29d0ef..65db266b20 100644 --- a/taskvine/src/graph/vinedag/strategic_orchestration_graph.h +++ b/taskvine/src/graph/vinedag/strategic_orchestration_graph.h @@ -54,7 +54,7 @@ struct strategic_orchestration_graph { task_priority_mode_t task_priority_mode; /* priority mode for task graph task scheduling */ double failure_injection_step_percent; /* 0 - 100, the percentage of steps to inject failure */ - double progress_bar_update_interval_sec; /* update interval for the progress bar in seconds */ + double progress_bar_update_interval_sec; /* update interval for the progress bar in seconds */ /* The filename of the csv file to store the time metrics of the strategic orchestration graph. */ char *time_metrics_filename; diff --git a/taskvine/src/graph/vinedag/strategic_orchestration_node.c b/taskvine/src/graph/vinedag/strategic_orchestration_node.c index 3735a92fd8..517e3333c4 100644 --- a/taskvine/src/graph/vinedag/strategic_orchestration_node.c +++ b/taskvine/src/graph/vinedag/strategic_orchestration_node.c @@ -158,7 +158,6 @@ struct strategic_orchestration_node *son_create(const char *node_key, int is_tar node->commit_time = 0; node->execution_time = 0; node->retrieval_time = 0; - node->pruning_time = 0; node->postprocessing_time = 0; node->critical_path_time = -1; diff --git a/taskvine/src/graph/vinedag/strategic_orchestration_node.h b/taskvine/src/graph/vinedag/strategic_orchestration_node.h index 6ca4ca1c99..eb1dcb4ad5 100644 --- a/taskvine/src/graph/vinedag/strategic_orchestration_node.h +++ b/taskvine/src/graph/vinedag/strategic_orchestration_node.h @@ -61,7 +61,6 @@ struct strategic_orchestration_node { timestamp_t commit_time; timestamp_t execution_time; timestamp_t retrieval_time; - timestamp_t pruning_time; timestamp_t postprocessing_time; node_outfile_type_t outfile_type; diff --git a/taskvine/src/manager/vine_manager.c b/taskvine/src/manager/vine_manager.c index 91495b1308..7dc74facaa 100644 --- a/taskvine/src/manager/vine_manager.c +++ b/taskvine/src/manager/vine_manager.c @@ -147,6 +147,7 @@ static vine_msg_code_t handle_manager_status(struct vine_manager *q, struct vine static vine_msg_code_t handle_resources(struct vine_manager *q, struct vine_worker_info *w, time_t stoptime); static vine_msg_code_t handle_feature(struct vine_manager *q, struct vine_worker_info *w, const char *line); static void handle_library_update(struct vine_manager *q, struct vine_worker_info *w, const char *line); +static int receive_tasks_from_worker(struct vine_manager *q, struct vine_worker_info *w, int count_received_so_far); static struct jx *manager_to_jx(struct vine_manager *q); static struct jx *manager_lean_to_jx(struct vine_manager *q); @@ -588,6 +589,8 @@ static vine_result_code_t get_completion_result(struct vine_manager *q, struct v return VINE_SUCCESS; } + t->time_when_get_result_start = timestamp_get(); + if (task_status != VINE_RESULT_SUCCESS) { w->last_failure_time = timestamp_get(); t->time_when_last_failure = w->last_failure_time; @@ -675,6 +678,8 @@ static vine_result_code_t get_completion_result(struct vine_manager *q, struct v } } + t->time_when_get_result_end = timestamp_get(); + return VINE_SUCCESS; } @@ -687,6 +692,7 @@ static vine_msg_code_t handle_complete(struct vine_manager *q, struct vine_worke { vine_result_code_t result = get_completion_result(q, w, line); if (result == VINE_SUCCESS) { + receive_tasks_from_worker(q, w, 0); return VINE_MSG_PROCESSED; } return VINE_MSG_NOT_PROCESSED; @@ -3743,14 +3749,14 @@ static int send_one_task(struct vine_manager *q, int *tasks_ready_left_to_consid struct vine_task *t; while (tasks_considered < tasks_to_consider) { - timestamp_t time_when_scheduling_start = timestamp_get(); - t = priority_queue_pop(q->ready_tasks); if (!t) { break; } tasks_considered++; + t->time_when_scheduling_start = timestamp_get(); + /* this task is not runnable at all, put it back in the pending queue */ if (!consider_task(q, t)) { list_push_tail(q->pending_tasks, t); @@ -3760,8 +3766,8 @@ static int send_one_task(struct vine_manager *q, int *tasks_ready_left_to_consid /* select a worker for the task */ struct vine_worker_info *w = vine_schedule_task_to_worker(q, t); - timestamp_t time_when_scheduling_end = timestamp_get(); - + t->time_when_scheduling_end = timestamp_get(); + /* task is runnable but no worker is fit, silently skip it */ if (!w) { list_push_tail(skipped_tasks, t); @@ -3779,7 +3785,6 @@ static int send_one_task(struct vine_manager *q, int *tasks_ready_left_to_consid switch (result) { case VINE_SUCCESS: committed_tasks++; - t->time_spent_on_scheduling = time_when_scheduling_end - time_when_scheduling_start; break; case VINE_APP_FAILURE: case VINE_WORKER_FAILURE: diff --git a/taskvine/src/manager/vine_task.c b/taskvine/src/manager/vine_task.c index cde256c47a..5c69187345 100644 --- a/taskvine/src/manager/vine_task.c +++ b/taskvine/src/manager/vine_task.c @@ -88,8 +88,15 @@ struct vine_task *vine_task_create(const char *command_line) void vine_task_clean(struct vine_task *t) { + t->time_when_scheduling_start = 0; + t->time_when_scheduling_end = 0; + t->time_when_commit_start = 0; t->time_when_commit_end = 0; + + t->time_when_get_result_start = 0; + t->time_when_get_result_end = 0; + t->time_when_retrieval = 0; t->time_when_done = 0; @@ -153,7 +160,14 @@ void vine_task_reset(struct vine_task *t) t->time_workers_execute_exhaustion = 0; t->time_workers_execute_failure = 0; - t->time_spent_on_scheduling = 0; + t->time_when_scheduling_start = 0; + t->time_when_scheduling_end = 0; + + t->time_when_commit_start = 0; + t->time_when_commit_end = 0; + + t->time_when_get_result_start = 0; + t->time_when_get_result_end = 0; rmsummary_delete(t->resources_measured); rmsummary_delete(t->resources_allocated); diff --git a/taskvine/src/manager/vine_task.h b/taskvine/src/manager/vine_task.h index d76d38e72b..4867e4f5a6 100644 --- a/taskvine/src/manager/vine_task.h +++ b/taskvine/src/manager/vine_task.h @@ -106,9 +106,15 @@ struct vine_task { timestamp_t time_when_submitted; /**< The time at which this task was added to the queue. */ timestamp_t time_when_done; /**< The time at which the task is mark as retrieved, after transfering output files and other final processing. */ + timestamp_t time_when_scheduling_start; /**< The time when the task starts to be considered for scheduling. */ + timestamp_t time_when_scheduling_end; /**< The time when the task is mapped to a worker and ready to be committed. */ + timestamp_t time_when_commit_start; /**< The time when the task starts to be transfered to a worker. */ timestamp_t time_when_commit_end; /**< The time when the task is completely transfered to a worker. */ + timestamp_t time_when_get_result_start; /**< The time when the task starts to get the result from the worker. */ + timestamp_t time_when_get_result_end; /**< The time when the task gets the result from the worker. */ + timestamp_t time_when_retrieval; /**< The time when output files start to be transfered back to the manager. time_done - time_when_retrieval is the time taken to transfer output files. */ timestamp_t time_when_last_failure; /**< If larger than 0, the time at which the last task failure was detected. */ @@ -122,8 +128,6 @@ struct vine_task { timestamp_t time_workers_execute_exhaustion; /**< Accumulated time spent in attempts that exhausted resources. */ timestamp_t time_workers_execute_failure; /**< Accumulated time for runs that terminated in worker failure/disconnection. */ - timestamp_t time_spent_on_scheduling; /**< Time spent on scheduling the task, only consider the successful scheduling. */ - int64_t bytes_received; /**< Number of bytes received since task has last started receiving input data. */ int64_t bytes_sent; /**< Number of bytes sent since task has last started sending input data. */ int64_t bytes_transferred; /**< Number of bytes transferred since task has last started transferring input data. */ From fd06a2658eefa794f9d0999d0db2ee02ac9c3913 Mon Sep 17 00:00:00 2001 From: Jin Zhou Date: Fri, 31 Oct 2025 23:59:17 -0400 Subject: [PATCH 066/113] new structure --- taskvine/src/graph/Makefile | 2 - taskvine/src/graph/vinedag/Makefile | 90 +-- taskvine/src/graph/vinedag/config.py | 0 .../src/graph/vinedag/context_graph/Makefile | 31 + .../graph/vinedag/context_graph/__init__.py | 18 + .../core.py} | 151 ++-- .../{ => context_graph}/proxy_functions.py | 40 +- .../{ => context_graph}/proxy_library.py | 11 +- .../vinedag/strategic_orchestration_graph.h | 143 ---- .../vinedag/strategic_orchestration_graph.py | 33 - .../vinedag/strategic_orchestration_node.h | 112 --- .../src/graph/vinedag/vine_graph/Makefile | 90 +++ .../src/graph/vinedag/vine_graph/__init__.py | 14 + .../vine_graph.c} | 717 +++++++++--------- .../src/graph/vinedag/vine_graph/vine_graph.h | 150 ++++ .../vine_graph.i} | 6 +- .../vinedag/vine_graph/vine_graph_client.py | 75 ++ .../vine_node.c} | 109 ++- .../src/graph/vinedag/vine_graph/vine_node.h | 115 +++ taskvine/src/graph/vinedag/vinedag.py | 173 +++-- taskvine/src/manager/vine_manager.c | 6 + taskvine/src/worker/vine_worker.c | 2 +- 22 files changed, 1108 insertions(+), 980 deletions(-) delete mode 100644 taskvine/src/graph/vinedag/config.py create mode 100644 taskvine/src/graph/vinedag/context_graph/Makefile create mode 100644 taskvine/src/graph/vinedag/context_graph/__init__.py rename taskvine/src/graph/vinedag/{runtime_execution_graph.py => context_graph/core.py} (58%) rename taskvine/src/graph/vinedag/{ => context_graph}/proxy_functions.py (70%) rename taskvine/src/graph/vinedag/{ => context_graph}/proxy_library.py (89%) delete mode 100644 taskvine/src/graph/vinedag/strategic_orchestration_graph.h delete mode 100644 taskvine/src/graph/vinedag/strategic_orchestration_graph.py delete mode 100644 taskvine/src/graph/vinedag/strategic_orchestration_node.h create mode 100644 taskvine/src/graph/vinedag/vine_graph/Makefile create mode 100644 taskvine/src/graph/vinedag/vine_graph/__init__.py rename taskvine/src/graph/vinedag/{strategic_orchestration_graph.c => vine_graph/vine_graph.c} (53%) create mode 100644 taskvine/src/graph/vinedag/vine_graph/vine_graph.h rename taskvine/src/graph/vinedag/{strategic_orchestration_graph.i => vine_graph/vine_graph.i} (72%) create mode 100644 taskvine/src/graph/vinedag/vine_graph/vine_graph_client.py rename taskvine/src/graph/vinedag/{strategic_orchestration_node.c => vine_graph/vine_node.c} (74%) create mode 100644 taskvine/src/graph/vinedag/vine_graph/vine_node.h diff --git a/taskvine/src/graph/Makefile b/taskvine/src/graph/Makefile index fa4c7dc2c5..f961ec9e86 100644 --- a/taskvine/src/graph/Makefile +++ b/taskvine/src/graph/Makefile @@ -9,5 +9,3 @@ $(SUBDIRS): %: $(MAKE) -C $@ $(MAKECMDGOALS) .PHONY: all clean install test lint format $(SUBDIRS) - - diff --git a/taskvine/src/graph/vinedag/Makefile b/taskvine/src/graph/vinedag/Makefile index f3882a646c..32dff04894 100644 --- a/taskvine/src/graph/vinedag/Makefile +++ b/taskvine/src/graph/vinedag/Makefile @@ -1,84 +1,36 @@ include ../../../../config.mk include ../../../../rules.mk -LOCAL_LINKAGE+=${CCTOOLS_HOME}/taskvine/src/manager/libtaskvine.a ${CCTOOLS_HOME}/dttools/src/libdttools.a -LOCAL_CCFLAGS+=-I ${CCTOOLS_HOME}/taskvine/src/manager +MODULE_DIR := $(CCTOOLS_PYTHON3_PATH)/ndcctools/taskvine/vinedag +SUBDIRS := context_graph vine_graph -PROJECT_NAME = vinedag - -SOURCE_DIR = $(CCTOOLS_HOME)/taskvine/src/graph/$(PROJECT_NAME) -MODULE_DIR = $(CCTOOLS_PYTHON3_PATH)/ndcctools/taskvine/$(PROJECT_NAME) - - -SOURCES = strategic_orchestration_node.c strategic_orchestration_graph.c -OBJECTS = $(SOURCES:%.c=%.o) - -SWIG_I = strategic_orchestration_graph.i - -# put SWIG generated sources and Python extension artifacts into build/ directory -BUILD_DIR := build -SWIG_WRAP = $(BUILD_DIR)/strategic_orchestration_graph_wrap.c -WRAP_OBJ = $(BUILD_DIR)/strategic_orchestration_graph_wrap.o -PYMODULE = $(BUILD_DIR)/_capi.$(CCTOOLS_DYNAMIC_SUFFIX) - -LIBRARIES = -PYDEPS = $(WRAP_OBJ) $(OBJECTS) -PYLINK_INPUT = $(WRAP_OBJ) $(OBJECTS) -PROGRAMS = -SCRIPTS = -TARGETS = $(LIBRARIES) $(PYMODULE) $(PROGRAMS) - -all: $(TARGETS) - - - -$(PROGRAMS): $(EXTERNALS) - -$(BUILD_DIR): - mkdir -p $(BUILD_DIR) - -$(SWIG_WRAP): $(SWIG_I) strategic_orchestration_graph.h | $(BUILD_DIR) - $(CCTOOLS_SWIG) -python -threads -relativeimport -I$(CCTOOLS_HOME)/taskvine/src/manager -I$(CCTOOLS_HOME)/dttools/src -I$(CCTOOLS_HOME)/taskvine/src/graph/$(PROJECT_NAME) -outdir $(BUILD_DIR) -o $@ $< - -# Build Python module (mimic bindings: silence SWIG warnings and build PIC) -$(WRAP_OBJ): $(SWIG_WRAP) - $(CCTOOLS_CC) -o $@ -c $(CCTOOLS_INTERNAL_CCFLAGS) $(LOCAL_CCFLAGS) $(CCTOOLS_PYTHON3_CCFLAGS) -w -fPIC -DNDEBUG $< - -$(PYMODULE): $(PYDEPS) -ifeq ($(CCTOOLS_STATIC),1) - $(CCTOOLS_LD) -o $@ $(CCTOOLS_DYNAMIC_FLAG) $(CCTOOLS_INTERNAL_LDFLAGS) $(LOCAL_LDFLAGS) $(PYLINK_INPUT) $(LOCAL_LINKAGE) $(CCTOOLS_PYTHON3_LDFLAGS) $(CCTOOLS_EXTERNAL_LINKAGE) -else - $(CCTOOLS_LD) -o $@ $(CCTOOLS_DYNAMIC_FLAG) $(CCTOOLS_INTERNAL_LDFLAGS) $(LOCAL_LDFLAGS) $(PYLINK_INPUT) $(LOCAL_LINKAGE) $(CCTOOLS_PYTHON3_LDFLAGS) $(CCTOOLS_EXTERNAL_LINKAGE) -endif +.PHONY: all install clean lint format $(SUBDIRS) +all: $(SUBDIRS) +$(SUBDIRS): + $(MAKE) -C $@ all install: all - mkdir -p $(CCTOOLS_INSTALL_DIR)/graph/$(PROJECT_NAME)/include - cp ${CCTOOLS_HOME}/taskvine/src/manager/taskvine.h $(CCTOOLS_INSTALL_DIR)/graph/$(PROJECT_NAME)/include/ mkdir -p $(MODULE_DIR) - cp $(PYMODULE) $(MODULE_DIR) - cp $(BUILD_DIR)/capi.py $(MODULE_DIR) - cp $(SOURCE_DIR)/*.py $(MODULE_DIR) + cp vinedag.py $(MODULE_DIR) + cp __init__.py $(MODULE_DIR) + @for dir in $(SUBDIRS); do \ + $(MAKE) -C $$dir install; \ + done clean: - rm -rf $(PROGRAMS) $(OBJECTS) *.o $(BUILD_DIR) $(PYMODULE) capi.py + @for dir in $(SUBDIRS); do \ + $(MAKE) -C $$dir clean; \ + done + rm -rf build lint: - if ( ! clang-format -Werror --dry-run --style='file:${CCTOOLS_HOME}/.clang-format' $(SOURCE_DIR)/*.c $(SOURCE_DIR)/*.h);\ - then\ - echo "========================================================";\ - echo "NOTICE: Run \`make format\` to format your latest changes.";\ - echo "========================================================";\ - exit 1;\ - fi - flake8 --ignore=$(CCTOOLS_FLAKE8_IGNORE_ERRORS) \ - --exclude=$(CCTOOLS_FLAKE8_IGNORE_FILES),$(BUILD_DIR) \ - $(SOURCE_DIR)/ + @for dir in $(SUBDIRS); do \ + $(MAKE) -C $$dir lint; \ + done format: - clang-format -i $(SOURCE_DIR)/*.c $(SOURCE_DIR)/*.h - -.PHONY: all clean install test lint format - - + @for dir in $(SUBDIRS); do \ + $(MAKE) -C $$dir format; \ + done diff --git a/taskvine/src/graph/vinedag/config.py b/taskvine/src/graph/vinedag/config.py deleted file mode 100644 index e69de29bb2..0000000000 diff --git a/taskvine/src/graph/vinedag/context_graph/Makefile b/taskvine/src/graph/vinedag/context_graph/Makefile new file mode 100644 index 0000000000..bdee79013e --- /dev/null +++ b/taskvine/src/graph/vinedag/context_graph/Makefile @@ -0,0 +1,31 @@ +include ../../../../../config.mk +include ../../../../../rules.mk + +PROJECT_NAME = vinedag + +SOURCE_DIR = $(CCTOOLS_HOME)/taskvine/src/graph/$(PROJECT_NAME)/context_graph +MODULE_ROOT = $(CCTOOLS_PYTHON3_PATH)/ndcctools/taskvine/$(PROJECT_NAME) +MODULE_DIR = $(MODULE_ROOT)/context_graph + +PY_SOURCES = $(wildcard $(SOURCE_DIR)/*.py) + +.PHONY: all install clean lint format + +all: + @true + +install: + mkdir -p $(MODULE_DIR) + cp $(PY_SOURCES) $(MODULE_DIR) + +clean: + rm -rf __pycache__ + +lint: + flake8 --ignore=$(CCTOOLS_FLAKE8_IGNORE_ERRORS) \ + --exclude=$(CCTOOLS_FLAKE8_IGNORE_FILES) \ + $(SOURCE_DIR)/ + +format: + @true + diff --git a/taskvine/src/graph/vinedag/context_graph/__init__.py b/taskvine/src/graph/vinedag/context_graph/__init__.py new file mode 100644 index 0000000000..cbb2cafd99 --- /dev/null +++ b/taskvine/src/graph/vinedag/context_graph/__init__.py @@ -0,0 +1,18 @@ +# 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 .core import ContextGraph, GraphKeyResult +from .proxy_functions import compute_single_key, compute_dts_key, compute_sexpr_key +from .proxy_library import ProxyLibrary + + +__all__ = [ + "ContextGraph", + "GraphKeyResult", + "compute_single_key", + "compute_dts_key", + "compute_sexpr_key", + "ProxyLibrary", +] diff --git a/taskvine/src/graph/vinedag/runtime_execution_graph.py b/taskvine/src/graph/vinedag/context_graph/core.py similarity index 58% rename from taskvine/src/graph/vinedag/runtime_execution_graph.py rename to taskvine/src/graph/vinedag/context_graph/core.py index 37161b1bbc..c3708c590c 100644 --- a/taskvine/src/graph/vinedag/runtime_execution_graph.py +++ b/taskvine/src/graph/vinedag/context_graph/core.py @@ -2,31 +2,27 @@ # This software is distributed under the GNU General Public License. # See the file COPYING for details. -import hashlib import cloudpickle import collections import random from collections import deque +# Attempt to import Dask helpers. When they are unavailable we fall back to +# None so environments without Dask continue to work. If Dask is present—either +# the legacy graph or the newer TaskSpec API—we normalize it into our internal +# task representation. try: import dask except ImportError: dask = None - try: import dask._task_spec as dts except ImportError: dts = None -def hash_name(*args): - out_str = "" - for arg in args: - out_str += str(arg) - return hashlib.sha256(out_str.encode('utf-8')).hexdigest()[:32] - - def hashable(s): + """Used while wiring dependencies to spot values we can treat as node keys.""" try: hash(s) return True @@ -34,59 +30,20 @@ def hashable(s): return False -def dist_func(mode, low, high): - """ Generate a random value from a distribution. """ - if not mode: - return 0 - - assert mode in ["uniform", "normal", "lognormal", "pareto", "mix"] - - # uniform distribution, flat spread - def uniform_dist(): - return random.uniform(low, high) - - # normal distribution, centered in the middle - def normal_dist(): - mu, sigma = (low + high) / 2, (high - low) / 6 - return min(max(random.gauss(mu, sigma), low), high) - - # lognormal distribution, long tail - def lognormal_dist(): - val = random.lognormvariate(0, 1) - val = val / (1 + val) - return low + (high - low) * val - - # pareto distribution, very heavy tail - def pareto_dist(alpha=2.0): - val = random.paretovariate(alpha) - val = val / (1 + val) - return low + (high - low) * val - - # mixture: most small values, few large ones - def mix_dist(): - if random.random() < 0.9: - return random.uniform(low, (low + high) / 2) - else: - return random.uniform((low + high) / 2, high) - - return { - "uniform": uniform_dist, - "normal": normal_dist, - "lognormal": lognormal_dist, - "pareto": pareto_dist, - "mix": mix_dist, - }[mode]() - - +# Lightweight wrapper around task results that optionally pads the payload. The +# padding lets tests model large outputs without altering the logical result. class GraphKeyResult: - """ A wrapper class to store the result of a task and the extra size in MB to allocate for this object in testing mode to evaluate storage consumption - and peer transfer performance across all workers. """ def __init__(self, result, extra_size_mb=None): + """Store the real user result plus optional padding used during regression tests.""" 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): + """Workers call this while recovering an output produced by save_result_of_key from disk. + If a node-local output, then data is stored in the task sandbox and the path is just the filename + If a shared file system output, then path is the full path to the file + If a target result, the path is the full path to the file in the manager's output directory""" try: with open(path, "rb") as f: result_obj = cloudpickle.load(f) @@ -96,17 +53,15 @@ def load_from_path(path): raise FileNotFoundError(f"Output file not found at {path}") -class RuntimeExecutionGraph: - """ - The RuntimeExecutionGraph class constructs the task graph and manages task dependencies. - This class is cloudpickled and sent to workers, where it is hoisted by the library instance. - The global RuntimeExecutionGraph object then serves as the execution context: whenever a task key is invoked, - the system resolves the corresponding graph key from the Vine key and executes the mapped function - to produce the result. - """ +# ContextGraph builds the logical DAG and manages dependencies. The +# object is cloudpickled, shipped with the proxy library, and hoisted on worker +# nodes. When a task key executes we map from the Vine key back to the original +# graph key, run the user function, and persist the result. +class ContextGraph: def __init__(self, task_dict, - extra_task_output_size_mb=["uniform", 0, 0], - extra_task_sleep_time=["uniform", 0, 0]): + extra_task_output_size_mb=[0, 0], + extra_task_sleep_time=[0, 0]): + """Capture the Python DAG that VineDAG hands us before we mirror it in C.""" self.task_dict = task_dict if dts: @@ -115,12 +70,12 @@ def __init__(self, task_dict, assert isinstance(v, (dts.Alias, dts.Task, dts.DataNode)), f"Unsupported task type for key {k}: {v.__class__}" self.parents_of, self.children_of = self._build_dependencies(self.task_dict) - self.depth_of = self._calculate_depths() - self.reg_key_to_sog_key = {k: hash_name(k) for k in task_dict.keys()} - self.sog_key_to_reg_key = {hash_name(k): k for k in task_dict.keys()} + # these mappings are set after node ids are assigned in the C vine graph + self.ckey2vid = {} + self.vid2ckey = {} - # will be set from sog + # will be set from vine graph self.outfile_remote_name = {key: None for key in self.task_dict.keys()} # testing params @@ -128,56 +83,34 @@ def __init__(self, task_dict, self.extra_sleep_time_of = self._calculate_extra_sleep_time_of(extra_task_sleep_time) def _calculate_extra_size_mb_of(self, extra_task_output_size_mb): - assert isinstance(extra_task_output_size_mb, list) and len(extra_task_output_size_mb) == 3 - mode, low, high = extra_task_output_size_mb + """Sample a uniform byte budget between low/high for every node.""" + assert isinstance(extra_task_output_size_mb, list) and len(extra_task_output_size_mb) == 2 + low, high = extra_task_output_size_mb low, high = int(low), int(high) assert low <= high - max_depth = max(depth for depth in self.depth_of.values()) - extra_size_mb_of = {} - for k in self.task_dict.keys(): - if self.depth_of[k] == max_depth or self.depth_of[k] == max_depth - 1: - extra_size_mb_of[k] = 0 - continue - extra_size_mb_of[k] = dist_func(mode, low, high) - - return extra_size_mb_of + return {k: random.uniform(low, high) for k in self.task_dict.keys()} def _calculate_extra_sleep_time_of(self, extra_task_sleep_time): - assert isinstance(extra_task_sleep_time, list) and len(extra_task_sleep_time) == 3 - mode, low, high = extra_task_sleep_time + """Pick a uniform delay between low/high so tests can fake runtime.""" + assert isinstance(extra_task_sleep_time, list) and len(extra_task_sleep_time) == 2 + low, high = extra_task_sleep_time low, high = int(low), int(high) assert low <= high - extra_sleep_time_of = {} - for k in self.task_dict.keys(): - extra_sleep_time_of[k] = dist_func(mode, low, high) - - return extra_sleep_time_of - - def _calculate_depths(self): - depth_of = {key: 0 for key in self.task_dict.keys()} - - topo_order = self.get_topological_order() - for key in topo_order: - if self.parents_of[key]: - depth_of[key] = max(depth_of[parent] for parent in self.parents_of[key]) + 1 - else: - depth_of[key] = 0 - - return depth_of - - def set_outfile_remote_name_of(self, key, outfile_remote_name): - self.outfile_remote_name[key] = outfile_remote_name + return {k: random.uniform(low, high) for k in self.task_dict.keys()} def is_dts_key(self, k): + """Gate the Dask-specific branch when we parse task definitions.""" if not hasattr(dask, "_task_spec"): return False import dask._task_spec as dts return isinstance(self.task_dict[k], (dts.Task, dts.TaskRef, dts.Alias, dts.DataNode, dts.NestedContainer)) def _build_dependencies(self, task_dict): + """Normalize mixed Dask/s-expression inputs into our parent/child lookup tables.""" def _find_sexpr_parents(sexpr): + """Resolve the immediate parents inside one symbolic expression node.""" if hashable(sexpr) and sexpr in task_dict.keys(): return {sexpr} elif isinstance(sexpr, (list, tuple)): @@ -214,11 +147,13 @@ def _find_sexpr_parents(sexpr): return parents_of, children_of def save_result_of_key(self, key, result): + """Called from the proxy function to persist a result into disk after the worker finishes.""" with open(self.outfile_remote_name[key], "wb") as f: result_obj = GraphKeyResult(result, extra_size_mb=self.extra_task_output_size_mb[key]) cloudpickle.dump(result_obj, f) def load_result_of_key(self, key): + """Used by downstream tasks to pull inputs from disk or the shared store.""" # workers user this function to load results from either local or shared file system # if a node-local output, then data is stored in the task sandbox and the remote name is just the filename # if a shared file system output, then remote name is the full path to the file @@ -226,6 +161,7 @@ def load_result_of_key(self, key): return GraphKeyResult.load_from_path(outfile_path) def get_topological_order(self): + """Produce the order VineDAG uses when assigning node IDs to the C graph.""" in_degree = {key: len(self.parents_of[key]) for key in self.task_dict.keys()} queue = deque([key for key, degree in in_degree.items() if degree == 0]) topo_order = [] @@ -247,12 +183,13 @@ def get_topological_order(self): return topo_order @staticmethod - def context_loader_func(reg_pkl): - reg = cloudpickle.loads(reg_pkl) + def context_loader_func(context_graph_pkl): + """Entry point the proxy library invokes to restore the serialized ContextGraph.""" + context_graph = cloudpickle.loads(context_graph_pkl) - if not isinstance(reg, RuntimeExecutionGraph): - raise TypeError("reg_pkl is not of type RuntimeExecutionGraph") + if not isinstance(context_graph, ContextGraph): + raise TypeError("context_graph_pkl is not of type ContextGraph") return { - "reg": reg, + "context_graph": context_graph, } diff --git a/taskvine/src/graph/vinedag/proxy_functions.py b/taskvine/src/graph/vinedag/context_graph/proxy_functions.py similarity index 70% rename from taskvine/src/graph/vinedag/proxy_functions.py rename to taskvine/src/graph/vinedag/context_graph/proxy_functions.py index 2cbbc8d1e1..4218756007 100644 --- a/taskvine/src/graph/vinedag/proxy_functions.py +++ b/taskvine/src/graph/vinedag/context_graph/proxy_functions.py @@ -7,7 +7,7 @@ from ndcctools.taskvine.utils import load_variable_from_library -def compute_dts_key(reg, k, v): +def compute_dts_key(context_graph, k, v): """ Compute the result of a Dask task node from dask._task_spec. @@ -15,7 +15,7 @@ def compute_dts_key(reg, k, v): inherit from the same base class. The `dependencies` field is a frozenset containing direct dependencies only (no recursive ancestry). - The function resolves each dependency from the reg, constructs an + The function resolves each dependency from the context_graph, constructs an input dictionary, and then executes the node according to its type. """ try: @@ -23,12 +23,12 @@ def compute_dts_key(reg, k, v): except ImportError: raise ImportError("Dask is not installed") - input_dict = {dep: reg.load_result_of_key(dep) for dep in v.dependencies} + input_dict = {dep: context_graph.load_result_of_key(dep) for dep in v.dependencies} try: if isinstance(v, dts.Alias): assert len(v.dependencies) == 1, "Expected exactly one dependency" - return reg.load_result_of_key(next(iter(v.dependencies))) + return context_graph.load_result_of_key(next(iter(v.dependencies))) elif isinstance(v, dts.Task): return v(input_dict) elif isinstance(v, dts.DataNode): @@ -39,7 +39,7 @@ def compute_dts_key(reg, k, v): raise Exception(f"Error while executing task {k}: {e}") -def compute_sexpr_key(reg, k, v): +def compute_sexpr_key(context_graph, k, v): """ Evaluate a symbolic expression (S-expression) task within the task graph. @@ -54,7 +54,7 @@ def compute_sexpr_key(reg, k, v): recursively resolves and executes the expression until a final value is produced. """ - input_dict = {parent: reg.load_result_of_key(parent) for parent in reg.parents_of[k]} + input_dict = {parent: context_graph.load_result_of_key(parent) for parent in context_graph.parents_of[k]} def _rec_call(expr): try: @@ -77,10 +77,10 @@ def _rec_call(expr): def compute_single_key(vine_key): """ - Compute a single task identified by a Vine key within the current RuntimeExecutionGraph. + Compute a single task identified by a Vine key within the current ContextGraph. The function retrieves the corresponding graph key and task object from the - global reg, determines the task type, and dispatches to the appropriate + global context_graph, determines the task type, and dispatches to the appropriate execution interface — e.g., `compute_dts_key` for Dask-style task specs or `compute_sexpr_key` for S-expression graphs. @@ -90,22 +90,22 @@ def compute_single_key(vine_key): After computation, the result is saved, the output file is validated, and an optional delay (`extra_sleep_time_of`) is applied before returning. """ - reg = load_variable_from_library('reg') + context_graph = load_variable_from_library('context_graph') - k = reg.sog_key_to_reg_key[vine_key] - v = reg.task_dict[k] + k = context_graph.vid2ckey[vine_key] + v = context_graph.task_dict[k] - if reg.is_dts_key(k): - result = compute_dts_key(reg, k, v) + if context_graph.is_dts_key(k): + result = compute_dts_key(context_graph, k, v) else: - result = compute_sexpr_key(reg, k, v) + result = compute_sexpr_key(context_graph, k, v) - reg.save_result_of_key(k, result) - if not os.path.exists(reg.outfile_remote_name[k]): - raise Exception(f"Output file {reg.outfile_remote_name[k]} does not exist after writing") - if os.stat(reg.outfile_remote_name[k]).st_size == 0: - raise Exception(f"Output file {reg.outfile_remote_name[k]} is empty after writing") + context_graph.save_result_of_key(k, result) + if not os.path.exists(context_graph.outfile_remote_name[k]): + raise Exception(f"Output file {context_graph.outfile_remote_name[k]} does not exist after writing") + if os.stat(context_graph.outfile_remote_name[k]).st_size == 0: + raise Exception(f"Output file {context_graph.outfile_remote_name[k]} is empty after writing") - time.sleep(reg.extra_sleep_time_of[k]) + time.sleep(context_graph.extra_sleep_time_of[k]) return True diff --git a/taskvine/src/graph/vinedag/proxy_library.py b/taskvine/src/graph/vinedag/context_graph/proxy_library.py similarity index 89% rename from taskvine/src/graph/vinedag/proxy_library.py rename to taskvine/src/graph/vinedag/context_graph/proxy_library.py index 0f1d1ea1b4..2bc4dd3c7c 100644 --- a/taskvine/src/graph/vinedag/proxy_library.py +++ b/taskvine/src/graph/vinedag/context_graph/proxy_library.py @@ -10,10 +10,9 @@ import random import hashlib import collections -from ndcctools.taskvine.vinedag.runtime_execution_graph import ( - GraphKeyResult, RuntimeExecutionGraph, hash_name, hashable -) -from ndcctools.taskvine.vinedag.proxy_functions import compute_dts_key, compute_sexpr_key, compute_single_key + +from ndcctools.taskvine.vinedag.context_graph.core import GraphKeyResult, ContextGraph +from ndcctools.taskvine.vinedag.context_graph.proxy_functions import compute_dts_key, compute_sexpr_key, compute_single_key from ndcctools.taskvine.utils import load_variable_from_library @@ -29,8 +28,8 @@ def __init__(self, py_manager): # 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, GraphKeyResult, RuntimeExecutionGraph, uuid, hashlib, random, types, collections, time, - load_variable_from_library, compute_dts_key, compute_sexpr_key, compute_single_key, hash_name, hashable + os, cloudpickle, GraphKeyResult, ContextGraph, uuid, hashlib, random, types, collections, time, + load_variable_from_library, compute_dts_key, compute_sexpr_key, compute_single_key ] # environment files serve as additional inputs to the library task, where each key is the local path and the value is the remote path diff --git a/taskvine/src/graph/vinedag/strategic_orchestration_graph.h b/taskvine/src/graph/vinedag/strategic_orchestration_graph.h deleted file mode 100644 index 65db266b20..0000000000 --- a/taskvine/src/graph/vinedag/strategic_orchestration_graph.h +++ /dev/null @@ -1,143 +0,0 @@ -#ifndef STRATEGIC_ORCHESTRATION_GRAPH_H -#define STRATEGIC_ORCHESTRATION_GRAPH_H - -#include "vine_task.h" -#include "hash_table.h" -#include "list.h" -#include "vine_manager.h" -#include "set.h" -#include "strategic_orchestration_node.h" -#include "taskvine.h" - -/** Select priority algorithm for strategic orchestration graph task scheduling. */ -typedef enum { - TASK_PRIORITY_MODE_RANDOM = 0, /**< Assign random priority to tasks */ - TASK_PRIORITY_MODE_DEPTH_FIRST, /**< Prioritize deeper tasks first */ - TASK_PRIORITY_MODE_BREADTH_FIRST, /**< Prioritize shallower tasks first */ - TASK_PRIORITY_MODE_FIFO, /**< First in, first out priority */ - TASK_PRIORITY_MODE_LIFO, /**< Last in, first out priority */ - TASK_PRIORITY_MODE_LARGEST_INPUT_FIRST, /**< Prioritize tasks with larger inputs first */ - TASK_PRIORITY_MODE_LARGEST_STORAGE_FOOTPRINT_FIRST /**< Prioritize tasks with larger storage footprint first */ -} task_priority_mode_t; - -/** The strategic orchestration graph object (logical scheduling layer). */ -struct strategic_orchestration_graph { - struct vine_manager *manager; - struct hash_table *nodes; - struct itable *task_id_to_node; - struct hash_table *outfile_cachename_to_node; - - /* The directory to store the checkpointed results. - * Only intermediate results can be checkpointed, the fraction of intermediate results to checkpoint is controlled by the checkpoint-fraction parameter. */ - char *checkpoint_dir; - - /* Results of target keys will be stored in this directory. - * This dir path can not necessarily be a shared file system directory, - * output files will be retrieved through the network instead, - * as long as the manager can access it. */ - char *output_dir; - - /* Python-side proxy library name, while is responsible for managing the runtime execution graph (reg), - * receiving function calls emitted by the strategic orchestration graph (sog), and executing them via - * the proxy function. The proxy library name is generated by the sog, and then synchronized to reg. */ - char *proxy_library_name; - - /* The proxy function is hoisted in the proxy library, it receives a key (in string format) in the rog, - * converts it to the reg-based key, looks up the function definition and arguments from reg, - * and then executes the function. This name is generated by reg and synchronized to sog. */ - char *proxy_function_name; - - /* The depth of the pruning strategy. 0 means no pruning, 1 means the most aggressive pruning. */ - int prune_depth; - double checkpoint_fraction; /* 0 - 1, the fraction of intermediate results to checkpoint */ - - task_priority_mode_t task_priority_mode; /* priority mode for task graph task scheduling */ - double failure_injection_step_percent; /* 0 - 100, the percentage of steps to inject failure */ - - double progress_bar_update_interval_sec; /* update interval for the progress bar in seconds */ - - /* The filename of the csv file to store the time metrics of the strategic orchestration graph. */ - char *time_metrics_filename; - - int enable_debug_log; /* whether to enable debug log */ -}; - -/* Public APIs for operating the strategic orchestration graph */ - -/** Create a strategic orchestration graph object and return it. -@param q Reference to the current manager object. -@return A new strategic orchestration graph object. -*/ -struct strategic_orchestration_graph *sog_create(struct vine_manager *q); - -/** Create a new node in the strategic orchestration graph. -@param sog Reference to the strategic orchestration graph object. -@param node_key Reference to the node key. -@param is_target_key Reference to whether the node is a target key. -*/ -void sog_add_node(struct strategic_orchestration_graph *sog, const char *node_key, int is_target_key); - -/** Add a dependency between two nodes in the strategic orchestration graph. -@param sog Reference to the strategic orchestration graph object. -@param parent_key Reference to the parent node key. -@param child_key Reference to the child node key. -*/ -void sog_add_dependency(struct strategic_orchestration_graph *sog, const char *parent_key, const char *child_key); - -/** Finalize the metrics of the strategic orchestration graph. -@param sog Reference to the strategic orchestration graph object. -*/ -void sog_compute_topology_metrics(struct strategic_orchestration_graph *sog); - -/** Get the heavy score of a node in the strategic orchestration graph. -@param sog Reference to the strategic orchestration graph object. -@param node_key Reference to the node key. -@return The heavy score. -*/ -double sog_get_node_heavy_score(const struct strategic_orchestration_graph *sog, const char *node_key); - -/** Execute the task graph. -@param sog Reference to the strategic orchestration graph object. -*/ -void sog_execute(struct strategic_orchestration_graph *sog); - -/** Get the outfile remote name of a node in the strategic orchestration graph. -@param sog Reference to the strategic orchestration graph object. -@param node_key Reference to the node key. -@return The outfile remote name. -*/ -const char *sog_get_node_outfile_remote_name(const struct strategic_orchestration_graph *sog, const char *node_key); - -/** Get the local outfile source of a node in the strategic orchestration graph. -@param sog Reference to the strategic orchestration graph object. -@param node_key Reference to the node key. -@return The local outfile source. -*/ -const char *sog_get_node_local_outfile_source(const struct strategic_orchestration_graph *sog, const char *node_key); - -/** Delete a strategic orchestration graph object. -@param sog Reference to the strategic orchestration graph object. -*/ -void sog_delete(struct strategic_orchestration_graph *sog); - -/** Get the proxy library name of the strategic orchestration graph. -@param sog Reference to the strategic orchestration graph object. -@return The proxy library name. -*/ -const char *sog_get_proxy_library_name(const struct strategic_orchestration_graph *sog); - -/** Set the proxy function name of the strategic orchestration graph. -@param sog Reference to the strategic orchestration graph object. -@param proxy_function_name Reference to the proxy function name. -*/ -void sog_set_proxy_function_name(struct strategic_orchestration_graph *sog, const char *proxy_function_name); - -/** Tune the strategic orchestration graph. -@param sog Reference to the strategic orchestration graph object. -@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 sog_tune(struct strategic_orchestration_graph *sog, const char *name, const char *value); - -#endif // STRATEGIC_ORCHESTRATION_GRAPH_H diff --git a/taskvine/src/graph/vinedag/strategic_orchestration_graph.py b/taskvine/src/graph/vinedag/strategic_orchestration_graph.py deleted file mode 100644 index 2c8f30d5f5..0000000000 --- a/taskvine/src/graph/vinedag/strategic_orchestration_graph.py +++ /dev/null @@ -1,33 +0,0 @@ -from ndcctools.taskvine.vinedag import capi - - -class StrategicOrchestrationGraph: - def __init__(self, c_taskvine): - self._c_sog = capi.sog_create(c_taskvine) - - def tune(self, name, value): - capi.sog_tune(self._c_sog, name, value) - - def add_node(self, key, is_target_key): - capi.sog_add_node(self._c_sog, key, is_target_key) - - def add_dependency(self, parent_key, child_key): - capi.sog_add_dependency(self._c_sog, parent_key, child_key) - - def compute_topology_metrics(self): - capi.sog_compute_topology_metrics(self._c_sog) - - def get_node_outfile_remote_name(self, key): - return capi.sog_get_node_outfile_remote_name(self._c_sog, key) - - def get_proxy_library_name(self): - return capi.sog_get_proxy_library_name(self._c_sog) - - def set_proxy_function(self, proxy_function): - capi.sog_set_proxy_function_name(self._c_sog, proxy_function.__name__) - - def execute(self): - capi.sog_execute(self._c_sog) - - def delete(self): - capi.sog_delete(self._c_sog) diff --git a/taskvine/src/graph/vinedag/strategic_orchestration_node.h b/taskvine/src/graph/vinedag/strategic_orchestration_node.h deleted file mode 100644 index eb1dcb4ad5..0000000000 --- a/taskvine/src/graph/vinedag/strategic_orchestration_node.h +++ /dev/null @@ -1,112 +0,0 @@ -#ifndef STRATEGIC_ORCHESTRATION_NODE_H -#define STRATEGIC_ORCHESTRATION_NODE_H - -#include "vine_task.h" -#include "hash_table.h" -#include "list.h" -#include "set.h" -#include "taskvine.h" - -/** Select the type of the node-output file. */ -typedef enum { - NODE_OUTFILE_TYPE_LOCAL = 0, /* Node-output file will be stored locally on the local staging directory */ - NODE_OUTFILE_TYPE_TEMP, /* Node-output file will be stored in the temporary node-local storage */ - NODE_OUTFILE_TYPE_SHARED_FILE_SYSTEM, /* Node-output file will be stored in the persistent shared file system */ -} node_outfile_type_t; - -/** The status of an output file of a node. */ -typedef enum { - PRUNE_STATUS_NOT_PRUNED = 0, - PRUNE_STATUS_SAFE, - PRUNE_STATUS_UNSAFE -} prune_status_t; - -/** The strategic orchestration node object. */ -struct strategic_orchestration_node { - char *node_key; - - /* Whether the node is a target key. - * If true, the output of the node will be declared as a vine_file and retrieved through the network. */ - int is_target_key; - - struct vine_task *task; - struct vine_file *infile; - struct vine_file *outfile; - char *outfile_remote_name; - size_t outfile_size_bytes; - - struct list *parents; - struct list *children; - struct set *pending_parents; - - int retry_attempts_left; - int completed; - int prune_depth; - - 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; - timestamp_t time_spent_on_unlink_local_files; - timestamp_t time_spent_on_prune_ancestors_of_temp_node; - timestamp_t time_spent_on_prune_ancestors_of_persisted_node; - - timestamp_t submission_time; - timestamp_t scheduling_time; - timestamp_t commit_time; - timestamp_t execution_time; - timestamp_t retrieval_time; - timestamp_t postprocessing_time; - - node_outfile_type_t outfile_type; - prune_status_t prune_status; -}; - -/** Create a new strategic orchestration node object. -@param node_key Reference to the node key. -@param is_target_key Reference to whether the node is a target key. -@return A new strategic orchestration node object. -*/ -struct strategic_orchestration_node *son_create(const char *node_key, int is_target_key); - -/** Create the task arguments for a strategic orchestration node object. -@param node Reference to the strategic orchestration node object. -@return The task arguments in JSON format: {"fn_args": [key], "fn_kwargs": {}}. -*/ -char *son_construct_task_arguments(struct strategic_orchestration_node *node); - -/** Delete a strategic orchestration node object. -@param node Reference to the strategic orchestration node object. -*/ -void son_delete(struct strategic_orchestration_node *node); - -/** Print information about a strategic orchestration node object. -@param node Reference to the strategic orchestration node object. -*/ -void son_debug_print(struct strategic_orchestration_node *node); - -/** Find all safe ancestors of a strategic orchestration node object. -@param start_node Reference to the start node. -@return The set of safe ancestors. -*/ -struct set *son_find_safe_ancestors(struct strategic_orchestration_node *start_node); - -/** Find all parents in a specific depth of a strategic orchestration node object. -@param node Reference to the node object. -@param depth Reference to the depth. -@return The list of parents. -*/ -struct list *son_find_parents_by_depth(struct strategic_orchestration_node *node, int depth); - -/** Update the critical time of a strategic orchestration node object. -@param node Reference to the strategic orchestration node object. -@param execution_time Reference to the execution time of the node. -*/ -void son_update_critical_path_time(struct strategic_orchestration_node *node, timestamp_t execution_time); - -#endif \ No newline at end of file diff --git a/taskvine/src/graph/vinedag/vine_graph/Makefile b/taskvine/src/graph/vinedag/vine_graph/Makefile new file mode 100644 index 0000000000..45fc3f7b42 --- /dev/null +++ b/taskvine/src/graph/vinedag/vine_graph/Makefile @@ -0,0 +1,90 @@ +include ../../../../../config.mk +include ../../../../../rules.mk + +PROJECT_NAME = vinedag + +LOCAL_LINKAGE+=${CCTOOLS_HOME}/taskvine/src/manager/libtaskvine.a ${CCTOOLS_HOME}/dttools/src/libdttools.a +LOCAL_CCFLAGS+=-I ${CCTOOLS_HOME}/taskvine/src/manager +LOCAL_CCFLAGS+=-I ${CCTOOLS_HOME}/taskvine/src/graph/$(PROJECT_NAME)/vine_graph + +SOURCE_DIR = $(CCTOOLS_HOME)/taskvine/src/graph/$(PROJECT_NAME)/vine_graph +MODULE_ROOT = $(CCTOOLS_PYTHON3_PATH)/ndcctools/taskvine/$(PROJECT_NAME) +MODULE_DIR = $(MODULE_ROOT)/vine_graph + +SOURCES = vine_node.c vine_graph.c +OBJECTS = $(SOURCES:%.c=%.o) + +BUILD_DIR := ../build + +# put SWIG generated sources and Python extension artifacts into ../build/ +SWIG_I = vine_graph.i + +WRAP_NAME = vine_graph_wrap +MODULE_NAME = vine_graph_capi + +SWIG_WRAP = $(BUILD_DIR)/$(WRAP_NAME).c +WRAP_OBJ = $(BUILD_DIR)/$(WRAP_NAME).o +PYMODULE = $(BUILD_DIR)/_$(MODULE_NAME).$(CCTOOLS_DYNAMIC_SUFFIX) + +LIBRARIES = +PYDEPS = $(WRAP_OBJ) $(OBJECTS) +PYLINK_INPUT = $(WRAP_OBJ) $(OBJECTS) +PROGRAMS = +SCRIPTS = +TARGETS = $(LIBRARIES) $(PYMODULE) $(PROGRAMS) + +.PHONY: all install clean lint format + +all: $(TARGETS) + +$(PROGRAMS): $(EXTERNALS) + +$(BUILD_DIR): + mkdir -p $(BUILD_DIR) + +$(SWIG_WRAP): $(SWIG_I) vine_graph.h | $(BUILD_DIR) + $(CCTOOLS_SWIG) -python -threads -relativeimport \ + -I$(CCTOOLS_HOME)/taskvine/src/manager \ + -I$(CCTOOLS_HOME)/dttools/src \ + -I$(CCTOOLS_HOME)/taskvine/src/graph/$(PROJECT_NAME) \ + -I$(CCTOOLS_HOME)/taskvine/src/graph/$(PROJECT_NAME)/vine_graph \ + -outdir $(BUILD_DIR) -o $@ $< + +# Build Python module (mimic bindings: silence SWIG warnings and build PIC) +$(WRAP_OBJ): $(SWIG_WRAP) + $(CCTOOLS_CC) -o $@ -c $(CCTOOLS_INTERNAL_CCFLAGS) $(LOCAL_CCFLAGS) $(CCTOOLS_PYTHON3_CCFLAGS) -w -fPIC -DNDEBUG $< + +$(PYMODULE): $(PYDEPS) +ifeq ($(CCTOOLS_STATIC),1) + $(CCTOOLS_LD) -o $@ $(CCTOOLS_DYNAMIC_FLAG) $(CCTOOLS_INTERNAL_LDFLAGS) $(LOCAL_LDFLAGS) $(PYLINK_INPUT) $(LOCAL_LINKAGE) $(CCTOOLS_PYTHON3_LDFLAGS) $(CCTOOLS_EXTERNAL_LINKAGE) +else + $(CCTOOLS_LD) -o $@ $(CCTOOLS_DYNAMIC_FLAG) $(CCTOOLS_INTERNAL_LDFLAGS) $(LOCAL_LDFLAGS) $(PYLINK_INPUT) $(LOCAL_LINKAGE) $(CCTOOLS_PYTHON3_LDFLAGS) $(CCTOOLS_EXTERNAL_LINKAGE) +endif + +install: all + mkdir -p $(CCTOOLS_INSTALL_DIR)/graph/$(PROJECT_NAME)/include + cp ${CCTOOLS_HOME}/taskvine/src/manager/taskvine.h $(CCTOOLS_INSTALL_DIR)/graph/$(PROJECT_NAME)/include/ + mkdir -p $(MODULE_DIR) + cp $(PYMODULE) $(MODULE_DIR) + cp $(BUILD_DIR)/$(MODULE_NAME).py $(MODULE_DIR) + cp $(SOURCE_DIR)/__init__.py $(MODULE_DIR) + cp $(SOURCE_DIR)/vine_graph_client.py $(MODULE_DIR) + +clean: + rm -f $(PROGRAMS) $(OBJECTS) $(WRAP_OBJ) + rm -f $(PYMODULE) $(BUILD_DIR)/$(MODULE_NAME).py + rm -rf $(BUILD_DIR) + +lint: + if ( ! clang-format -Werror --dry-run --style='file:${CCTOOLS_HOME}/.clang-format' $(SOURCE_DIR)/*.c $(SOURCE_DIR)/*.h); \ + then \ + echo "========================================================"; \ + echo "NOTICE: Run \`make format\` to format your latest changes."; \ + echo "========================================================"; \ + exit 1; \ + fi + +format: + clang-format -i $(SOURCE_DIR)/*.c $(SOURCE_DIR)/*.h + + diff --git a/taskvine/src/graph/vinedag/vine_graph/__init__.py b/taskvine/src/graph/vinedag/vine_graph/__init__.py new file mode 100644 index 0000000000..4dd6f218ce --- /dev/null +++ b/taskvine/src/graph/vinedag/vine_graph/__init__.py @@ -0,0 +1,14 @@ +# Copyright (C) 2025 The University of Notre Dame +# This software is distributed under the GNU General Public License. +# See the file COPYING for details. + +"""Namespace for the C vine graph bindings and Python client.""" + +try: # pragma: no cover - module only exists after building the SWIG bindings + from . import vine_graph_capi +except ImportError: + vine_graph_capi = None + +from .vine_graph_client import VineGraphClient + +__all__ = ["vine_graph_capi", "VineGraphClient"] diff --git a/taskvine/src/graph/vinedag/strategic_orchestration_graph.c b/taskvine/src/graph/vinedag/vine_graph/vine_graph.c similarity index 53% rename from taskvine/src/graph/vinedag/strategic_orchestration_graph.c rename to taskvine/src/graph/vinedag/vine_graph/vine_graph.c index a790c8b39b..3ef646c768 100644 --- a/taskvine/src/graph/vinedag/strategic_orchestration_graph.c +++ b/taskvine/src/graph/vinedag/vine_graph/vine_graph.c @@ -1,6 +1,7 @@ #include #include #include +#include #include #include #include @@ -22,8 +23,8 @@ #include "macros.h" #include "uuid.h" -#include "strategic_orchestration_node.h" -#include "strategic_orchestration_graph.h" +#include "vine_node.h" +#include "vine_graph.h" #include "vine_manager.h" #include "vine_worker_info.h" #include "vine_task.h" @@ -47,30 +48,13 @@ static void handle_sigint(int signal) interrupted = 1; } -/** - * Compute a lexicographic priority score from the node key. - * Used during topological sorting to break ties deterministically. - * @param key Reference to the node key. - * @return The lexical priority. - */ -static double compute_lex_priority(const char *key) -{ - double score = 0.0; - double factor = 1.0; - for (int i = 0; i < 8 && key[i] != '\0'; i++) { - score += (unsigned char)key[i] * factor; - factor *= 0.01; - } - return -score; -} - /** * Calculate the priority of a node given the priority mode. * @param node Reference to the node object. * @param priority_mode Reference to the priority mode. * @return The priority. */ -static double calculate_task_priority(struct strategic_orchestration_node *node, task_priority_mode_t priority_mode) +static double calculate_task_priority(struct vine_node *node, task_priority_mode_t priority_mode) { if (!node) { return 0; @@ -79,7 +63,7 @@ static double calculate_task_priority(struct strategic_orchestration_node *node, double priority = 0; timestamp_t current_time = timestamp_get(); - struct strategic_orchestration_node *parent_node; + struct vine_node *parent_node; switch (priority_mode) { case TASK_PRIORITY_MODE_RANDOM: @@ -122,45 +106,45 @@ static double calculate_task_priority(struct strategic_orchestration_node *node, } /** - * Submit a node to the taskvine manager. - * @param sog Reference to the strategic orchestration graph object. - * @param node Reference to the node object. + * Submit a node to the TaskVine manager via the vine graph. + * @param vg Reference to the vine graph. + * @param node Reference to the node. */ -static void submit_node_task(struct strategic_orchestration_graph *sog, struct strategic_orchestration_node *node) +static void submit_node_task(struct vine_graph *vg, struct vine_node *node) { - if (!sog || !node) { + if (!vg || !node) { return; } /* calculate the priority of the node */ - double priority = calculate_task_priority(node, sog->task_priority_mode); + double priority = calculate_task_priority(node, vg->task_priority_mode); vine_task_set_priority(node->task, priority); /* submit the task to the manager */ timestamp_t time_start = timestamp_get(); - int task_id = vine_submit(sog->manager, node->task); + int task_id = vine_submit(vg->manager, node->task); node->submission_time = timestamp_get() - time_start; /* insert the task id to the task id to node map */ - itable_insert(sog->task_id_to_node, task_id, node); + itable_insert(vg->task_id_to_node, (uint64_t)task_id, node); - debug(D_VINE, "submitted node %s with task id %d", node->node_key, task_id); + debug(D_VINE, "submitted node %" PRIu64 " with task id %d", node->node_id, task_id); return; } /** - * Submit the children of a node if all its dependencies are resolved. - * @param sog Reference to the strategic orchestration graph object. - * @param node Reference to the node object. + * Submit the children of a node once every dependency has completed. + * @param vg Reference to the vine graph. + * @param node Reference to the node. */ -static void submit_unblocked_children(struct strategic_orchestration_graph *sog, struct strategic_orchestration_node *node) +static void submit_unblocked_children(struct vine_graph *vg, struct vine_node *node) { - if (!sog || !node) { + if (!vg || !node) { return; } - struct strategic_orchestration_node *child_node; + struct vine_node *child_node; LIST_ITERATE(node->children, child_node) { /* Remove this parent from the child's pending set if it exists */ @@ -169,13 +153,13 @@ static void submit_unblocked_children(struct strategic_orchestration_graph *sog, if (child_node->pending_parents && set_lookup(child_node->pending_parents, node)) { set_remove(child_node->pending_parents, node); } else { - debug(D_ERROR, "inconsistent pending set: child=%s missing parent=%s", child_node->node_key, node->node_key); + debug(D_ERROR, "inconsistent pending set: child=%" PRIu64 " missing parent=%" PRIu64, child_node->node_id, node->node_id); } } /* If no more parents are pending, submit the child */ if (!child_node->pending_parents || set_size(child_node->pending_parents) == 0) { - submit_node_task(sog, child_node); + submit_node_task(vg, child_node); } } @@ -183,99 +167,98 @@ static void submit_unblocked_children(struct strategic_orchestration_graph *sog, } /** - * Get the topological order of the strategic orchestration graph. - * Must be called after all nodes and dependencies are added and the topology metrics are computed. - * @param sog Reference to the strategic orchestration graph object. - * @return The list of nodes in topological order. + * Compute a topological ordering of the vine graph. + * Call only after all nodes, edges, and metrics have been populated. + * @param vg Reference to the vine graph. + * @return Nodes in topological order. */ -static struct list *get_topological_order(struct strategic_orchestration_graph *sog) +static struct list *get_topological_order(struct vine_graph *vg) { - if (!sog) { + if (!vg) { return NULL; } - int total_nodes = hash_table_size(sog->nodes); + int total_nodes = itable_size(vg->nodes); struct list *topo_order = list_create(); - struct hash_table *in_degree_map = hash_table_create(0, 0); + struct itable *in_degree_map = itable_create(0); struct priority_queue *pq = priority_queue_create(total_nodes); - char *key; - struct strategic_orchestration_node *node; - HASH_TABLE_ITERATE(sog->nodes, key, node) + uint64_t nid; + struct vine_node *node; + ITABLE_ITERATE(vg->nodes, nid, node) { int deg = list_size(node->parents); - hash_table_insert(in_degree_map, key, (void *)(intptr_t)deg); + itable_insert(in_degree_map, nid, (void *)(intptr_t)deg); if (deg == 0) { - priority_queue_push(pq, node, compute_lex_priority(node->node_key)); + priority_queue_push(pq, node, -(double)node->node_id); } } while (priority_queue_size(pq) > 0) { - struct strategic_orchestration_node *current = priority_queue_pop(pq); + struct vine_node *current = priority_queue_pop(pq); list_push_tail(topo_order, current); - struct strategic_orchestration_node *child; + struct vine_node *child; LIST_ITERATE(current->children, child) { - intptr_t raw_deg = (intptr_t)hash_table_lookup(in_degree_map, child->node_key); + intptr_t raw_deg = (intptr_t)itable_lookup(in_degree_map, child->node_id); int deg = (int)raw_deg - 1; - - hash_table_remove(in_degree_map, child->node_key); - hash_table_insert(in_degree_map, child->node_key, (void *)(intptr_t)deg); + itable_insert(in_degree_map, child->node_id, (void *)(intptr_t)deg); if (deg == 0) { - priority_queue_push(pq, child, compute_lex_priority(child->node_key)); + priority_queue_push(pq, child, -(double)child->node_id); } } } if (list_size(topo_order) != total_nodes) { - debug(D_ERROR, "Error: strategic orchestration graph contains cycles or is malformed.\n"); - debug(D_ERROR, "Expected %d nodes, but only sorted %d.\n", total_nodes, list_size(topo_order)); + debug(D_ERROR, "Error: vine graph contains cycles or is malformed."); + debug(D_ERROR, "Expected %d nodes, but only sorted %d.", total_nodes, list_size(topo_order)); - HASH_TABLE_ITERATE(sog->nodes, key, node) + uint64_t id; + ITABLE_ITERATE(vg->nodes, id, node) { - intptr_t raw_deg = (intptr_t)hash_table_lookup(in_degree_map, key); + intptr_t raw_deg = (intptr_t)itable_lookup(in_degree_map, id); int deg = (int)raw_deg; if (deg > 0) { - debug(D_ERROR, " Node %s has in-degree %d. Parents:\n", key, deg); - struct strategic_orchestration_node *p; + debug(D_ERROR, " Node %" PRIu64 " has in-degree %d. Parents:", id, deg); + struct vine_node *p; LIST_ITERATE(node->parents, p) { - debug(D_ERROR, " -> %s\n", p->node_key); + debug(D_ERROR, " -> %" PRIu64, p->node_id); } } } list_delete(topo_order); - hash_table_delete(in_degree_map); + itable_delete(in_degree_map); priority_queue_delete(pq); exit(1); } - hash_table_delete(in_degree_map); + itable_delete(in_degree_map); priority_queue_delete(pq); return topo_order; } /** - * Extract the weakly connected components of the strategic orchestration graph. - * This function is used only for debugging purposes at the moment. - * @param sog Reference to the strategic orchestration graph object. - * @return The list of weakly connected components. + * Extract weakly connected components of the vine graph. + * Currently used for debugging and instrumentation only. + * @param vg Reference to the vine graph. + * @return List of weakly connected components. */ -static struct list *extract_weakly_connected_components(struct strategic_orchestration_graph *sog) +static struct list *extract_weakly_connected_components(struct vine_graph *vg) { - if (!sog) { + if (!vg) { return NULL; } struct set *visited = set_create(0); struct list *components = list_create(); - char *node_key; - struct strategic_orchestration_node *node; - HASH_TABLE_ITERATE(sog->nodes, node_key, node) + uint64_t nid; + struct vine_node *node; + ITABLE_ITERATE(vg->nodes, nid, node) { if (set_lookup(visited, node)) { continue; @@ -289,9 +272,9 @@ static struct list *extract_weakly_connected_components(struct strategic_orchest list_push_tail(component, node); while (list_size(queue) > 0) { - struct strategic_orchestration_node *curr = list_pop_head(queue); + struct vine_node *curr = list_pop_head(queue); - struct strategic_orchestration_node *p; + struct vine_node *p; LIST_ITERATE(curr->parents, p) { if (!set_lookup(visited, p)) { @@ -301,7 +284,7 @@ static struct list *extract_weakly_connected_components(struct strategic_orchest } } - struct strategic_orchestration_node *c; + struct vine_node *c; LIST_ITERATE(curr->children, c) { if (!set_lookup(visited, c)) { @@ -321,11 +304,11 @@ static struct list *extract_weakly_connected_components(struct strategic_orchest } /** - * Compute the heavy score of a node in the strategic orchestration graph. - * @param node Reference to the node object. - * @return The heavy score. + * Compute the heavy score of a node in the vine graph. + * @param node Reference to the node. + * @return Heavy score. */ -static double compute_node_heavy_score(struct strategic_orchestration_node *node) +static double compute_node_heavy_score(struct vine_node *node) { if (!node) { return 0; @@ -338,28 +321,29 @@ static double compute_node_heavy_score(struct strategic_orchestration_node *node } /** - * Map a task to a node in the strategic orchestration graph. - * @param sog Reference to the strategic orchestration graph object. - * @param task Reference to the task object. - * @return The node object. + * Map a TaskVine task back to its vine node. + * @param vg Reference to the vine graph. + * @param task Task reported by the manager. + * @return Matching node. */ -static struct strategic_orchestration_node *get_node_by_task(struct strategic_orchestration_graph *sog, struct vine_task *task) +static struct vine_node *get_node_by_task(struct vine_graph *vg, struct vine_task *task) { - if (!sog || !task) { + if (!vg || !task) { return NULL; } if (task->type == VINE_TASK_TYPE_STANDARD) { /* standard tasks are mapped directly to a node */ - return itable_lookup(sog->task_id_to_node, task->task_id); + return itable_lookup(vg->task_id_to_node, (uint64_t)task->task_id); } else if (task->type == VINE_TASK_TYPE_RECOVERY) { /* note that recovery tasks are not mapped to any node but we still need the original node for pruning, * so we look up the outfile of the task, then map it back to get the original node */ struct vine_mount *mount; LIST_ITERATE(task->output_mounts, mount) { - if (mount->file->original_producer_task_id > 0) { - return itable_lookup(sog->task_id_to_node, mount->file->original_producer_task_id); + uint64_t original_producer_task_id = mount->file->original_producer_task_id; + if (original_producer_task_id > 0) { + return itable_lookup(vg->task_id_to_node, original_producer_task_id); } } } @@ -371,20 +355,20 @@ static struct strategic_orchestration_node *get_node_by_task(struct strategic_or /** * Prune the ancestors of a persisted node. This is only used for persisted nodes that produce persisted files. - * All ancestors we consider here include both temp nodes and persisted nodes, becasue data written to shared file system + * All ancestors we consider here include both temp nodes and persisted nodes, because data written to the shared file system * is safe and can definitely trigger upstream data redundancy to be released. - * @param sog Reference to the strategic orchestration graph object. + * @param vg Reference to the vine graph. * @param node Reference to the node object. * @return The number of pruned replicas. */ -static int prune_ancestors_of_persisted_node(struct strategic_orchestration_graph *sog, struct strategic_orchestration_node *node) +static int prune_ancestors_of_persisted_node(struct vine_graph *vg, struct vine_node *node) { - if (!sog || !node) { + if (!vg || !node) { return -1; } /* find all safe ancestors */ - struct set *safe_ancestors = son_find_safe_ancestors(node); + struct set *safe_ancestors = vine_node_find_safe_ancestors(node); if (!safe_ancestors) { return 0; } @@ -394,7 +378,7 @@ static int prune_ancestors_of_persisted_node(struct strategic_orchestration_grap timestamp_t start_time = timestamp_get(); /* prune all safe ancestors */ - struct strategic_orchestration_node *ancestor_node; + struct vine_node *ancestor_node; SET_ITERATE(safe_ancestors, ancestor_node) { switch (ancestor_node->outfile_type) { @@ -403,7 +387,7 @@ static int prune_ancestors_of_persisted_node(struct strategic_orchestration_grap break; case NODE_OUTFILE_TYPE_TEMP: /* prune the temp file */ - vine_prune_file(sog->manager, ancestor_node->outfile); + vine_prune_file(vg->manager, ancestor_node->outfile); break; case NODE_OUTFILE_TYPE_SHARED_FILE_SYSTEM: /* unlink directly from the shared file system */ @@ -430,13 +414,13 @@ static int prune_ancestors_of_persisted_node(struct strategic_orchestration_grap * Files stored in the shared filesystem are never pruned by this function, * because temp outputs are not considered sufficiently safe to trigger * deletion of persisted data upstream. - * @param sog Reference to the strategic orchestration graph object. + * @param vg Reference to the vine graph. * @param node Reference to the node object. * @return The number of pruned replicas. */ -static int prune_ancestors_of_temp_node(struct strategic_orchestration_graph *sog, struct strategic_orchestration_node *node) +static int prune_ancestors_of_temp_node(struct vine_graph *vg, struct vine_node *node) { - if (!sog || !node || !node->outfile || node->prune_depth <= 0) { + if (!vg || !node || !node->outfile || node->prune_depth <= 0) { return 0; } @@ -444,9 +428,9 @@ static int prune_ancestors_of_temp_node(struct strategic_orchestration_graph *so int pruned_replica_count = 0; - struct list *parents = son_find_parents_by_depth(node, node->prune_depth); + struct list *parents = vine_node_find_parents_by_depth(node, node->prune_depth); - struct strategic_orchestration_node *parent_node; + struct vine_node *parent_node; LIST_ITERATE(parents, parent_node) { /* skip if the parent does not produce a temp file */ @@ -458,7 +442,7 @@ static int prune_ancestors_of_temp_node(struct strategic_orchestration_graph *so * 1. it has no pending dependents * 2. all completed dependents have also completed their corresponding recovery tasks, if any */ int all_children_completed = 1; - struct strategic_orchestration_node *child_node; + struct vine_node *child_node; LIST_ITERATE(parent_node->children, child_node) { /* break early if the child node is not completed */ @@ -479,7 +463,7 @@ static int prune_ancestors_of_temp_node(struct strategic_orchestration_graph *so continue; } - pruned_replica_count += vine_prune_file(sog->manager, parent_node->outfile); + pruned_replica_count += vine_prune_file(vg->manager, parent_node->outfile); /* this parent is pruned because a successor that produces a temp file is completed, it is unsafe because the * manager may submit a recovery task to bring it back in case of worker failures. */ parent_node->prune_status = PRUNE_STATUS_UNSAFE; @@ -496,13 +480,13 @@ static int prune_ancestors_of_temp_node(struct strategic_orchestration_graph *so * Prune the ancestors of a node when it is completed. * @param node Reference to the node object. */ -static void prune_ancestors_of_node(struct strategic_orchestration_graph *sog, struct strategic_orchestration_node *node) +static void prune_ancestors_of_node(struct vine_graph *vg, struct vine_node *node) { - if (!sog || !node) { + if (!vg || !node) { return; } - /* do not prune if the node is not completed */ + /* do not prune if the node has not completed */ if (!node->completed) { return; } @@ -516,30 +500,30 @@ static void prune_ancestors_of_node(struct strategic_orchestration_graph *sog, s case NODE_OUTFILE_TYPE_SHARED_FILE_SYSTEM: /* If the outfile was declared as a VINE_FILE or was written to the shared fs, then it is guaranteed to be persisted * and there is no chance that it will be lost unexpectedly. So we can safely prune all ancestors of this node. */ - pruned_replica_count = prune_ancestors_of_persisted_node(sog, node); + pruned_replica_count = prune_ancestors_of_persisted_node(vg, node); break; case NODE_OUTFILE_TYPE_TEMP: /* Otherwise, if the node outfile is a temp file, we need to be careful about pruning, because temp files are prone * to failures, while means they can be lost due to node evictions or failures. */ - pruned_replica_count = prune_ancestors_of_temp_node(sog, node); + pruned_replica_count = prune_ancestors_of_temp_node(vg, node); break; } timestamp_t elapsed_time = timestamp_get() - start_time; - debug(D_VINE, "pruned %d ancestors of node %s in %.6f seconds", pruned_replica_count, node->node_key, elapsed_time / 1000000.0); + debug(D_VINE, "pruned %d ancestors of node %" PRIu64 " in %.6f seconds", pruned_replica_count, node->node_id, elapsed_time / 1000000.0); return; } /** - * Print the time metrics of the strategic orchestration graph to a csv file. - * @param sog Reference to the strategic orchestration graph object. + * Print the time metrics of the vine graph to a csv file. + * @param vg Reference to the vine graph. * @param filename Reference to the filename of the csv file. */ -static void print_time_metrics(struct strategic_orchestration_graph *sog, const char *filename) +static void print_time_metrics(struct vine_graph *vg, const char *filename) { - if (!sog) { + if (!vg) { return; } @@ -554,13 +538,13 @@ static void print_time_metrics(struct strategic_orchestration_graph *sog, const debug(D_ERROR, "failed to open file %s", filename); return; } - fprintf(fp, "node_key,submission_time_us,scheduling_time_us,commit_time_us,execution_time_us,retrieval_time_us,postprocessing_time_us\n"); + fprintf(fp, "node_id,submission_time_us,scheduling_time_us,commit_time_us,execution_time_us,retrieval_time_us,postprocessing_time_us\n"); - char *node_key; - struct strategic_orchestration_node *node; - HASH_TABLE_ITERATE(sog->nodes, node_key, node) + uint64_t nid; + struct vine_node *node; + ITABLE_ITERATE(vg->nodes, nid, node) { - fprintf(fp, "%s,%lu,%lu,%lu,%lu,%lu,%lu\n", node_key, node->submission_time, node->scheduling_time, node->commit_time, node->execution_time, node->retrieval_time, node->postprocessing_time); + fprintf(fp, "%" PRIu64 ",%lu,%lu,%lu,%lu,%lu,%lu\n", node->node_id, node->submission_time, node->scheduling_time, node->commit_time, node->execution_time, node->retrieval_time, node->postprocessing_time); } fclose(fp); @@ -571,53 +555,53 @@ static void print_time_metrics(struct strategic_orchestration_graph *sog, const /* Public APIs */ /*************************************************************/ -/** Tune the strategic orchestration graph. - *@param sog Reference to the strategic orchestration graph object. +/** Tune the vine graph. + *@param vg Reference to the vine 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 sog_tune(struct strategic_orchestration_graph *sog, const char *name, const char *value) +int vine_graph_tune(struct vine_graph *vg, const char *name, const char *value) { - if (!sog || !name || !value) { + if (!vg || !name || !value) { return -1; } if (strcmp(name, "failure-injection-step-percent") == 0) { - sog->failure_injection_step_percent = atof(value); + vg->failure_injection_step_percent = atof(value); } else if (strcmp(name, "task-priority-mode") == 0) { if (strcmp(value, "random") == 0) { - sog->task_priority_mode = TASK_PRIORITY_MODE_RANDOM; + vg->task_priority_mode = TASK_PRIORITY_MODE_RANDOM; } else if (strcmp(value, "depth-first") == 0) { - sog->task_priority_mode = TASK_PRIORITY_MODE_DEPTH_FIRST; + vg->task_priority_mode = TASK_PRIORITY_MODE_DEPTH_FIRST; } else if (strcmp(value, "breadth-first") == 0) { - sog->task_priority_mode = TASK_PRIORITY_MODE_BREADTH_FIRST; + vg->task_priority_mode = TASK_PRIORITY_MODE_BREADTH_FIRST; } else if (strcmp(value, "fifo") == 0) { - sog->task_priority_mode = TASK_PRIORITY_MODE_FIFO; + vg->task_priority_mode = TASK_PRIORITY_MODE_FIFO; } else if (strcmp(value, "lifo") == 0) { - sog->task_priority_mode = TASK_PRIORITY_MODE_LIFO; + vg->task_priority_mode = TASK_PRIORITY_MODE_LIFO; } else if (strcmp(value, "largest-input-first") == 0) { - sog->task_priority_mode = TASK_PRIORITY_MODE_LARGEST_INPUT_FIRST; + vg->task_priority_mode = TASK_PRIORITY_MODE_LARGEST_INPUT_FIRST; } else if (strcmp(value, "largest-storage-footprint-first") == 0) { - sog->task_priority_mode = TASK_PRIORITY_MODE_LARGEST_STORAGE_FOOTPRINT_FIRST; + vg->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, "output-dir") == 0) { - if (sog->output_dir) { - free(sog->output_dir); + if (vg->output_dir) { + free(vg->output_dir); } if (mkdir(value, 0777) != 0 && errno != EEXIST) { debug(D_ERROR, "failed to mkdir %s (errno=%d)", value, errno); return -1; } - sog->output_dir = xxstrdup(value); + vg->output_dir = xxstrdup(value); } else if (strcmp(name, "prune-depth") == 0) { - sog->prune_depth = atoi(value); + vg->prune_depth = atoi(value); } else if (strcmp(name, "checkpoint-fraction") == 0) { double fraction = atof(value); @@ -625,39 +609,39 @@ int sog_tune(struct strategic_orchestration_graph *sog, const char *name, const debug(D_ERROR, "invalid checkpoint fraction: %s (must be between 0.0 and 1.0)", value); return -1; } - sog->checkpoint_fraction = fraction; + vg->checkpoint_fraction = fraction; } else if (strcmp(name, "checkpoint-dir") == 0) { - if (sog->checkpoint_dir) { - free(sog->checkpoint_dir); + if (vg->checkpoint_dir) { + free(vg->checkpoint_dir); } if (mkdir(value, 0777) != 0 && errno != EEXIST) { debug(D_ERROR, "failed to mkdir %s (errno=%d)", value, errno); return -1; } - sog->checkpoint_dir = xxstrdup(value); + vg->checkpoint_dir = xxstrdup(value); } else if (strcmp(name, "progress-bar-update-interval-sec") == 0) { double val = atof(value); - sog->progress_bar_update_interval_sec = (val > 0.0) ? val : 0.1; + vg->progress_bar_update_interval_sec = (val > 0.0) ? val : 0.1; } else if (strcmp(name, "time-metrics-filename") == 0) { if (strcmp(value, "0") == 0) { return 0; } - if (sog->time_metrics_filename) { - free(sog->time_metrics_filename); + if (vg->time_metrics_filename) { + free(vg->time_metrics_filename); } - sog->time_metrics_filename = xxstrdup(value); + vg->time_metrics_filename = xxstrdup(value); /** Extract parent directory inline **/ - const char *slash = strrchr(sog->time_metrics_filename, '/'); + const char *slash = strrchr(vg->time_metrics_filename, '/'); if (slash) { - size_t len = slash - sog->time_metrics_filename; + size_t len = slash - vg->time_metrics_filename; char *parent = malloc(len + 1); - memcpy(parent, sog->time_metrics_filename, len); + memcpy(parent, vg->time_metrics_filename, len); parent[len] = '\0'; /** Ensure the parent directory exists **/ @@ -670,19 +654,19 @@ int sog_tune(struct strategic_orchestration_graph *sog, const char *name, const } /** Truncate or create the file **/ - FILE *fp = fopen(sog->time_metrics_filename, "w"); + FILE *fp = fopen(vg->time_metrics_filename, "w"); if (!fp) { - debug(D_ERROR, "failed to create file %s (errno=%d)", sog->time_metrics_filename, errno); + debug(D_ERROR, "failed to create file %s (errno=%d)", vg->time_metrics_filename, errno); return -1; } fclose(fp); } else if (strcmp(name, "enable-debug-log") == 0) { - if (sog->enable_debug_log == 0) { + if (vg->enable_debug_log == 0) { return -1; } - sog->enable_debug_log = (atoi(value) == 1) ? 1 : 0; - if (sog->enable_debug_log == 0) { + vg->enable_debug_log = (atoi(value) == 1) ? 1 : 0; + if (vg->enable_debug_log == 0) { debug_flags_clear(); debug_close(); } @@ -696,18 +680,18 @@ int sog_tune(struct strategic_orchestration_graph *sog, const char *name, const } /** - * Get the outfile remote name of a node in the strategic orchestration graph. - * @param sog Reference to the strategic orchestration graph object. - * @param node_key Reference to the node key. + * Get the outfile remote name of a node in the vine graph. + * @param vg Reference to the vine graph. + * @param node_id Reference to the node id. * @return The outfile remote name. */ -const char *sog_get_node_outfile_remote_name(const struct strategic_orchestration_graph *sog, const char *node_key) +const char *vine_graph_get_node_outfile_remote_name(const struct vine_graph *vg, uint64_t node_id) { - if (!sog || !node_key) { + if (!vg) { return NULL; } - struct strategic_orchestration_node *node = hash_table_lookup(sog->nodes, node_key); + struct vine_node *node = itable_lookup(vg->nodes, node_id); if (!node) { return NULL; } @@ -716,50 +700,50 @@ const char *sog_get_node_outfile_remote_name(const struct strategic_orchestratio } /** - * Get the proxy library name of the strategic orchestration graph. - * @param sog Reference to the strategic orchestration graph object. + * Get the proxy library name of the vine graph. + * @param vg Reference to the vine graph. * @return The proxy library name. */ -const char *sog_get_proxy_library_name(const struct strategic_orchestration_graph *sog) +const char *vine_graph_get_proxy_library_name(const struct vine_graph *vg) { - if (!sog) { + if (!vg) { return NULL; } - return sog->proxy_library_name; + return vg->proxy_library_name; } /** - * Set the proxy function name of the strategic orchestration graph. - * @param sog Reference to the strategic orchestration graph object. + * Set the proxy function name of the vine graph. + * @param vg Reference to the vine graph. * @param proxy_function_name Reference to the proxy function name. */ -void sog_set_proxy_function_name(struct strategic_orchestration_graph *sog, const char *proxy_function_name) +void vine_graph_set_proxy_function_name(struct vine_graph *vg, const char *proxy_function_name) { - if (!sog || !proxy_function_name) { + if (!vg || !proxy_function_name) { return; } - if (sog->proxy_function_name) { - free(sog->proxy_function_name); + if (vg->proxy_function_name) { + free(vg->proxy_function_name); } - sog->proxy_function_name = xxstrdup(proxy_function_name); + vg->proxy_function_name = xxstrdup(proxy_function_name); } /** - * Get the heavy score of a node in the strategic orchestration graph. - * @param sog Reference to the strategic orchestration graph object. - * @param node_key Reference to the node key. + * Get the heavy score of a node in the vine graph. + * @param vg Reference to the vine graph. + * @param node_id Reference to the node id. * @return The heavy score. */ -double sog_get_node_heavy_score(const struct strategic_orchestration_graph *sog, const char *node_key) +double vine_graph_get_node_heavy_score(const struct vine_graph *vg, uint64_t node_id) { - if (!sog) { + if (!vg) { return -1; } - struct strategic_orchestration_node *node = hash_table_lookup(sog->nodes, node_key); + struct vine_node *node = itable_lookup(vg->nodes, node_id); if (!node) { return -1; } @@ -768,26 +752,26 @@ double sog_get_node_heavy_score(const struct strategic_orchestration_graph *sog, } /** - * Get the local outfile source of a node in the strategic orchestration graph, only valid for local output files. + * Get the local outfile source of a node in the vine graph, only valid for local output files. * The source of a local output file is the path on the local filesystem. - * @param sog Reference to the strategic orchestration graph object. - * @param node_key Reference to the node key. + * @param vg Reference to the vine graph. + * @param node_id Reference to the node id. * @return The local outfile source. */ -const char *sog_get_node_local_outfile_source(const struct strategic_orchestration_graph *sog, const char *node_key) +const char *vine_graph_get_node_local_outfile_source(const struct vine_graph *vg, uint64_t node_id) { - if (!sog || !node_key) { + if (!vg) { return NULL; } - struct strategic_orchestration_node *node = hash_table_lookup(sog->nodes, node_key); + struct vine_node *node = itable_lookup(vg->nodes, node_id); if (!node) { - debug(D_ERROR, "node %s not found", node_key); + debug(D_ERROR, "node %" PRIu64 " not found", node_id); exit(1); } if (node->outfile_type != NODE_OUTFILE_TYPE_LOCAL) { - debug(D_ERROR, "node %s is not a local output file", node_key); + debug(D_ERROR, "node %" PRIu64 " is not a local output file", node_id); exit(1); } @@ -795,26 +779,25 @@ const char *sog_get_node_local_outfile_source(const struct strategic_orchestrati } /** - * Compute the topology metrics of the strategic orchestration graph, including depth, height, upstream and downstream counts, + * Compute the topology metrics of the vine 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 sog Reference to the strategic orchestration graph object. + * @param vg Reference to the vine graph. */ -void sog_compute_topology_metrics(struct strategic_orchestration_graph *sog) +void vine_graph_compute_topology_metrics(struct vine_graph *vg) { - if (!sog) { + if (!vg) { return; } /* get nodes in topological order */ - struct list *topo_order = get_topological_order(sog); + struct list *topo_order = get_topological_order(vg); if (!topo_order) { return; } - char *node_key; - struct strategic_orchestration_node *node; - struct strategic_orchestration_node *parent_node; - struct strategic_orchestration_node *child_node; + struct vine_node *node; + struct vine_node *parent_node; + struct vine_node *child_node; /* compute the depth of the node */ LIST_ITERATE(topo_order, node) @@ -841,46 +824,47 @@ void sog_compute_topology_metrics(struct strategic_orchestration_graph *sog) } /* compute the upstream and downstream counts for each node */ - struct hash_table *upstream_map = hash_table_create(0, 0); - struct hash_table *downstream_map = hash_table_create(0, 0); - HASH_TABLE_ITERATE(sog->nodes, node_key, node) + struct itable *upstream_map = itable_create(0); + struct itable *downstream_map = itable_create(0); + uint64_t nid_tmp; + ITABLE_ITERATE(vg->nodes, nid_tmp, node) { struct set *upstream = set_create(0); struct set *downstream = set_create(0); - hash_table_insert(upstream_map, node_key, upstream); - hash_table_insert(downstream_map, node_key, downstream); + itable_insert(upstream_map, node->node_id, upstream); + itable_insert(downstream_map, node->node_id, downstream); } LIST_ITERATE(topo_order, node) { - struct set *upstream = hash_table_lookup(upstream_map, node->node_key); + struct set *upstream = itable_lookup(upstream_map, node->node_id); LIST_ITERATE(node->parents, parent_node) { - struct set *parent_upstream = hash_table_lookup(upstream_map, parent_node->node_key); + struct set *parent_upstream = itable_lookup(upstream_map, parent_node->node_id); set_union(upstream, parent_upstream); set_insert(upstream, parent_node); } } LIST_ITERATE_REVERSE(topo_order, node) { - struct set *downstream = hash_table_lookup(downstream_map, node->node_key); + struct set *downstream = itable_lookup(downstream_map, node->node_id); LIST_ITERATE(node->children, child_node) { - struct set *child_downstream = hash_table_lookup(downstream_map, child_node->node_key); + struct set *child_downstream = itable_lookup(downstream_map, child_node->node_id); set_union(downstream, child_downstream); set_insert(downstream, child_node); } } LIST_ITERATE(topo_order, node) { - node->upstream_subgraph_size = set_size(hash_table_lookup(upstream_map, node->node_key)); - node->downstream_subgraph_size = set_size(hash_table_lookup(downstream_map, node->node_key)); + 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(hash_table_lookup(upstream_map, node->node_key)); - set_delete(hash_table_lookup(downstream_map, node->node_key)); + set_delete(itable_lookup(upstream_map, node->node_id)); + set_delete(itable_lookup(downstream_map, node->node_id)); } - hash_table_delete(upstream_map); - hash_table_delete(downstream_map); + itable_delete(upstream_map); + itable_delete(downstream_map); /* compute the heavy score for each node */ LIST_ITERATE(topo_order, node) @@ -894,13 +878,13 @@ void sog_compute_topology_metrics(struct strategic_orchestration_graph *sog) struct priority_queue *sorted_nodes = priority_queue_create(total_nodes); LIST_ITERATE(topo_order, node) { - if (node->is_target_key) { + if (node->is_target) { total_target_nodes++; } priority_queue_push(sorted_nodes, node, node->heavy_score); } /* calculate the number of nodes to be checkpointed */ - int checkpoint_count = (int)((total_nodes - total_target_nodes) * sog->checkpoint_fraction); + int checkpoint_count = (int)((total_nodes - total_target_nodes) * vg->checkpoint_fraction); if (checkpoint_count < 0) { checkpoint_count = 0; } @@ -908,18 +892,18 @@ void sog_compute_topology_metrics(struct strategic_orchestration_graph *sog) /* assign outfile types to each node */ int assigned_checkpoint_count = 0; while ((node = priority_queue_pop(sorted_nodes))) { - if (node->is_target_key) { + if (node->is_target) { /* declare the output file as a vine_file so that it can be retrieved by the manager as usual */ node->outfile_type = NODE_OUTFILE_TYPE_LOCAL; - char *local_outfile_path = string_format("%s/%s", sog->output_dir, node->outfile_remote_name); - node->outfile = vine_declare_file(sog->manager, local_outfile_path, VINE_CACHE_LEVEL_WORKFLOW, 0); + char *local_outfile_path = string_format("%s/%s", vg->output_dir, node->outfile_remote_name); + node->outfile = vine_declare_file(vg->manager, local_outfile_path, VINE_CACHE_LEVEL_WORKFLOW, 0); free(local_outfile_path); continue; } if (assigned_checkpoint_count < checkpoint_count) { /* checkpointed files will be written directly to the shared file system, no need to manage them in the manager */ node->outfile_type = NODE_OUTFILE_TYPE_SHARED_FILE_SYSTEM; - char *shared_file_system_outfile_path = string_format("%s/%s", sog->checkpoint_dir, node->outfile_remote_name); + char *shared_file_system_outfile_path = string_format("%s/%s", vg->checkpoint_dir, node->outfile_remote_name); free(node->outfile_remote_name); node->outfile_remote_name = shared_file_system_outfile_path; node->outfile = NULL; @@ -927,7 +911,7 @@ void sog_compute_topology_metrics(struct strategic_orchestration_graph *sog) } else { /* other nodes will be declared as temp files to leverage node-local storage */ node->outfile_type = NODE_OUTFILE_TYPE_TEMP; - node->outfile = vine_declare_temp(sog->manager); + node->outfile = vine_declare_temp(vg->manager); } } /* track the output dependencies of regular and vine_temp nodes */ @@ -940,7 +924,7 @@ void sog_compute_topology_metrics(struct strategic_orchestration_graph *sog) priority_queue_delete(sorted_nodes); /* extract weakly connected components */ - struct list *weakly_connected_components = extract_weakly_connected_components(sog); + struct list *weakly_connected_components = extract_weakly_connected_components(vg); struct list *component; int component_index = 0; debug(D_VINE, "graph has %d weakly connected components\n", list_size(weakly_connected_components)); @@ -958,135 +942,156 @@ void sog_compute_topology_metrics(struct strategic_orchestration_graph *sog) } /** - * Create a new node and track it in the strategic orchestration graph. - * @param sog Reference to the strategic orchestration graph object. - * @param node_key Reference to the node key. - * @param is_target_key Reference to whether the node is a target key. - * @return A new node object. + * Create a new node and track it in the vine graph. + * @param vg Reference to the vine graph. + * @return The auto-assigned node id. */ -void sog_add_node(struct strategic_orchestration_graph *sog, const char *node_key, int is_target_key) +uint64_t vine_graph_add_node(struct vine_graph *vg) { - if (!sog || !node_key) { - return; + if (!vg) { + return 0; } - /* if the node already exists, skip creating a new one */ - struct strategic_orchestration_node *node = hash_table_lookup(sog->nodes, node_key); + /* assign a new id based on current node count, ensure uniqueness */ + uint64_t candidate_id = itable_size(vg->nodes); + candidate_id += 1; + while (itable_lookup(vg->nodes, candidate_id)) { + candidate_id++; + } + uint64_t node_id = candidate_id; + + /* create the backing node (defaults to non-target) */ + struct vine_node *node = vine_node_create(node_id); + if (!node) { - node = son_create(node_key, is_target_key); + debug(D_ERROR, "failed to create node %" PRIu64, node_id); + vine_graph_delete(vg); + exit(1); + } - if (!node) { - debug(D_ERROR, "failed to create node %s", node_key); - sog_delete(sog); - exit(1); - } + if (!vg->proxy_function_name) { + debug(D_ERROR, "proxy function name is not set"); + vine_graph_delete(vg); + exit(1); + } - if (!sog->proxy_function_name) { - debug(D_ERROR, "proxy function name is not set"); - sog_delete(sog); - exit(1); - } + if (!vg->proxy_library_name) { + debug(D_ERROR, "proxy library name is not set"); + vine_graph_delete(vg); + exit(1); + } - if (!sog->proxy_library_name) { - debug(D_ERROR, "proxy library name is not set"); - sog_delete(sog); - exit(1); - } + /* create node task */ + node->task = vine_task_create(vg->proxy_function_name); + vine_task_set_library_required(node->task, vg->proxy_library_name); + vine_task_addref(node->task); - /* create node task */ - node->task = vine_task_create(sog->proxy_function_name); - vine_task_set_library_required(node->task, sog->proxy_library_name); - vine_task_addref(node->task); + /* construct the task arguments and declare the infile */ + char *task_arguments = vine_node_construct_task_arguments(node); + node->infile = vine_declare_buffer(vg->manager, task_arguments, strlen(task_arguments), VINE_CACHE_LEVEL_TASK, VINE_UNLINK_WHEN_DONE); + free(task_arguments); + vine_task_add_input(node->task, node->infile, "infile", VINE_TRANSFER_ALWAYS); - /* construct the task arguments and declare the infile */ - char *task_arguments = son_construct_task_arguments(node); - node->infile = vine_declare_buffer(sog->manager, task_arguments, strlen(task_arguments), VINE_CACHE_LEVEL_TASK, VINE_UNLINK_WHEN_DONE); - free(task_arguments); - vine_task_add_input(node->task, node->infile, "infile", VINE_TRANSFER_ALWAYS); + /* initialize the pruning depth of each node, currently statically set to the global prune depth */ + node->prune_depth = vg->prune_depth; - /* initialize the pruning depth of each node, currently statically set to the global prune depth */ - node->prune_depth = sog->prune_depth; + itable_insert(vg->nodes, node_id, node); - hash_table_insert(sog->nodes, node_key, node); + return node_id; +} + +/** + * Mark a node as a retrieval target. + */ +void vine_graph_set_target(struct vine_graph *vg, uint64_t node_id) +{ + if (!vg) { + return; + } + struct vine_node *node = itable_lookup(vg->nodes, node_id); + if (!node) { + debug(D_ERROR, "node %" PRIu64 " not found", node_id); + exit(1); } + node->is_target = 1; } /** - * Create a new strategic orchestration graph object and bind a manager to it. + * Create a new vine graph and bind a manager to it. * @param q Reference to the manager object. - * @return A new strategic orchestration graph object. + * @return A new vine graph instance. */ -struct strategic_orchestration_graph *sog_create(struct vine_manager *q) +struct vine_graph *vine_graph_create(struct vine_manager *q) { if (!q) { return NULL; } - struct strategic_orchestration_graph *sog = xxmalloc(sizeof(struct strategic_orchestration_graph)); + struct vine_graph *vg = xxmalloc(sizeof(struct vine_graph)); - sog->manager = q; + vg->manager = q; - sog->checkpoint_dir = xxstrdup(sog->manager->runtime_directory); // default to current working directory - sog->output_dir = xxstrdup(sog->manager->runtime_directory); // default to current working directory + vg->checkpoint_dir = xxstrdup(vg->manager->runtime_directory); // default to current working directory + vg->output_dir = xxstrdup(vg->manager->runtime_directory); // default to current working directory - sog->nodes = hash_table_create(0, 0); - sog->task_id_to_node = itable_create(0); - sog->outfile_cachename_to_node = hash_table_create(0, 0); + vg->nodes = itable_create(0); + vg->task_id_to_node = itable_create(0); + vg->outfile_cachename_to_node = hash_table_create(0, 0); cctools_uuid_t proxy_library_name_id; cctools_uuid_create(&proxy_library_name_id); - sog->proxy_library_name = xxstrdup(proxy_library_name_id.str); + vg->proxy_library_name = xxstrdup(proxy_library_name_id.str); - sog->proxy_function_name = NULL; + vg->proxy_function_name = NULL; - sog->prune_depth = 1; + vg->prune_depth = 1; - sog->task_priority_mode = TASK_PRIORITY_MODE_LARGEST_INPUT_FIRST; - sog->failure_injection_step_percent = -1.0; + vg->task_priority_mode = TASK_PRIORITY_MODE_LARGEST_INPUT_FIRST; + vg->failure_injection_step_percent = -1.0; - sog->progress_bar_update_interval_sec = 0.1; + vg->progress_bar_update_interval_sec = 0.1; /* enable debug system for C code since it uses a separate debug system instance * from the Python bindings. Use the same function that the manager uses. */ - char *debug_tmp = string_format("%s/vine-logs/debug", sog->manager->runtime_directory); + char *debug_tmp = string_format("%s/vine-logs/debug", vg->manager->runtime_directory); vine_enable_debug_log(debug_tmp); free(debug_tmp); - sog->time_metrics_filename = NULL; + vg->time_metrics_filename = NULL; - sog->enable_debug_log = 1; + vg->enable_debug_log = 1; - return sog; + return vg; } /** - * Add a dependency between two nodes in the strategic orchestration graph. Note that the input-output file relationship - * is not handled here, because their file names may have not been determined yet. - * @param sog Reference to the strategic orchestration graph object. - * @param parent_key Reference to the parent node key. - * @param child_key Reference to the child node key. + * Add a dependency between two nodes in the vine graph. Note that the input-output file relationship + * is not handled here, because their file names might not have been determined yet. + * @param vg Reference to the vine graph. + * @param parent_id Reference to the parent node id. + * @param child_id Reference to the child node id. */ -void sog_add_dependency(struct strategic_orchestration_graph *sog, const char *parent_key, const char *child_key) +void vine_graph_add_dependency(struct vine_graph *vg, uint64_t parent_id, uint64_t child_id) { - if (!sog || !parent_key || !child_key) { + if (!vg) { return; } - struct strategic_orchestration_node *parent_node = hash_table_lookup(sog->nodes, parent_key); - struct strategic_orchestration_node *child_node = hash_table_lookup(sog->nodes, child_key); + struct vine_node *parent_node = itable_lookup(vg->nodes, parent_id); + struct vine_node *child_node = itable_lookup(vg->nodes, child_id); if (!parent_node) { - debug(D_ERROR, "parent node %s not found", parent_key); - char *node_key = NULL; - struct strategic_orchestration_node *node; - printf("parent_keys:\n"); - HASH_TABLE_ITERATE(sog->nodes, node_key, node) + debug(D_ERROR, "parent node %" PRIu64 " not found", parent_id); + uint64_t nid; + struct vine_node *node; + printf("parent_ids:\n"); + ITABLE_ITERATE(vg->nodes, nid, node) { - printf(" %s\n", node->node_key); + printf(" %" PRIu64 "\n", node->node_id); } exit(1); } if (!child_node) { - debug(D_ERROR, "child node %s not found", child_key); + debug(D_ERROR, "child node %" PRIu64 " not found", child_id); exit(1); } @@ -1097,43 +1102,43 @@ void sog_add_dependency(struct strategic_orchestration_graph *sog, const char *p } /** - * Execute the strategic orchestration graph. This must be called after all nodes and dependencies are added and the topology metrics are computed. - * @param sog Reference to the strategic orchestration graph object. + * Execute the vine graph. This must be called after all nodes and dependencies are added and the topology metrics are computed. + * @param vg Reference to the vine graph. */ -void sog_execute(struct strategic_orchestration_graph *sog) +void vine_graph_execute(struct vine_graph *vg) { - if (!sog) { + if (!vg) { return; } signal(SIGINT, handle_sigint); - debug(D_VINE, "start executing strategic orchestration graph"); + debug(D_VINE, "start executing vine graph"); /* print the info of all nodes */ - char *node_key; - struct strategic_orchestration_node *node; - HASH_TABLE_ITERATE(sog->nodes, node_key, node) + uint64_t nid_iter; + struct vine_node *node; + ITABLE_ITERATE(vg->nodes, nid_iter, node) { - son_debug_print(node); + vine_node_debug_print(node); } /* enable return recovery tasks */ - vine_enable_return_recovery_tasks(sog->manager); + vine_enable_return_recovery_tasks(vg->manager); - /* create mapping from task_id and outfile cached_name to node */ - HASH_TABLE_ITERATE(sog->nodes, node_key, node) + /* create mappings from task IDs and outfile cache names to nodes */ + ITABLE_ITERATE(vg->nodes, nid_iter, node) { if (node->outfile) { - hash_table_insert(sog->outfile_cachename_to_node, node->outfile->cached_name, node); + hash_table_insert(vg->outfile_cachename_to_node, node->outfile->cached_name, node); } } /* add the parents' outfiles as inputs to the task */ - struct list *topo_order = get_topological_order(sog); + struct list *topo_order = get_topological_order(vg); LIST_ITERATE(topo_order, node) { - struct strategic_orchestration_node *parent_node; + struct vine_node *parent_node; LIST_ITERATE(node->parents, parent_node) { if (parent_node->outfile) { @@ -1143,36 +1148,36 @@ void sog_execute(struct strategic_orchestration_graph *sog) } /* initialize pending_parents for all nodes */ - HASH_TABLE_ITERATE(sog->nodes, node_key, node) + ITABLE_ITERATE(vg->nodes, nid_iter, node) { - struct strategic_orchestration_node *parent_node; + struct vine_node *parent_node; LIST_ITERATE(node->parents, parent_node) { if (node->pending_parents) { - /* Use parent_node->node_key to ensure pointer consistency */ + /* Use parent pointer to ensure pointer consistency */ set_insert(node->pending_parents, parent_node); } } } /* enqueue those without dependencies */ - HASH_TABLE_ITERATE(sog->nodes, node_key, node) + ITABLE_ITERATE(vg->nodes, nid_iter, node) { if (!node->pending_parents || set_size(node->pending_parents) == 0) { - submit_node_task(sog, node); + submit_node_task(vg, node); } } /* calculate steps to inject failure */ double next_failure_threshold = -1.0; - if (sog->failure_injection_step_percent > 0) { - next_failure_threshold = sog->failure_injection_step_percent / 100.0; + if (vg->failure_injection_step_percent > 0) { + next_failure_threshold = vg->failure_injection_step_percent / 100.0; } struct ProgressBar *pbar = progress_bar_init("Executing Tasks"); - progress_bar_set_update_interval(pbar, sog->progress_bar_update_interval_sec); + progress_bar_set_update_interval(pbar, vg->progress_bar_update_interval_sec); - struct ProgressBarPart *regular_tasks_part = progress_bar_create_part("Regular", hash_table_size(sog->nodes)); + struct ProgressBarPart *regular_tasks_part = progress_bar_create_part("Regular", itable_size(vg->nodes)); struct ProgressBarPart *recovery_tasks_part = progress_bar_create_part("Recovery", 0); progress_bar_bind_part(pbar, regular_tasks_part); progress_bar_bind_part(pbar, recovery_tasks_part); @@ -1184,8 +1189,8 @@ void sog_execute(struct strategic_orchestration_graph *sog) break; } - struct vine_task *task = vine_wait(sog->manager, wait_timeout); - progress_bar_set_part_total(pbar, recovery_tasks_part, sog->manager->num_submitted_recovery_tasks); + struct vine_task *task = vine_wait(vg->manager, wait_timeout); + progress_bar_set_part_total(pbar, recovery_tasks_part, vg->manager->num_submitted_recovery_tasks); if (task) { /* retrieve all possible tasks */ wait_timeout = 0; @@ -1193,7 +1198,7 @@ void sog_execute(struct strategic_orchestration_graph *sog) timestamp_t time_when_postprocessing_start = timestamp_get(); /* get the original node by task id */ - struct strategic_orchestration_node *node = get_node_by_task(sog, task); + struct vine_node *node = get_node_by_task(vg, 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); @@ -1202,14 +1207,14 @@ void sog_execute(struct strategic_orchestration_graph *sog) /* in case of failure, resubmit this task */ if (node->task->result != VINE_RESULT_SUCCESS || node->task->exit_code != 0) { if (node->retry_attempts_left <= 0) { - debug(D_ERROR, "Task %d failed (result=%d, exit=%d). Node %s has no retries left. Aborting.", task->task_id, node->task->result, node->task->exit_code, node->node_key); - sog_delete(sog); + debug(D_ERROR, "Task %d failed (result=%d, exit=%d). Node %" PRIu64 " has no retries left. Aborting.", task->task_id, node->task->result, node->task->exit_code, node->node_id); + vine_graph_delete(vg); exit(1); } node->retry_attempts_left--; - debug(D_VINE | D_NOTICE, "Task %d failed (result=%d, exit=%d). Retrying node %s (remaining=%d)...", task->task_id, node->task->result, node->task->exit_code, node->node_key, node->retry_attempts_left); + debug(D_VINE | D_NOTICE, "Task %d failed (result=%d, exit=%d). Retrying node %" PRIu64 " (remaining=%d)...", task->task_id, node->task->result, node->task->exit_code, node->node_id, node->retry_attempts_left); vine_task_reset(node->task); - submit_node_task(sog, node); + submit_node_task(vg, node); continue; } @@ -1220,14 +1225,14 @@ void sog_execute(struct strategic_orchestration_graph *sog) int result = stat(node->outfile_remote_name, &info); if (result < 0) { if (node->retry_attempts_left <= 0) { - debug(D_ERROR, "Task %d succeeded but missing sharedfs output %s; no retries left for node %s. Aborting.", task->task_id, node->outfile_remote_name, node->node_key); - sog_delete(sog); + debug(D_ERROR, "Task %d succeeded but missing sharedfs output %s; no retries left for node %" PRIu64 ". Aborting.", task->task_id, node->outfile_remote_name, node->node_id); + vine_graph_delete(vg); exit(1); } node->retry_attempts_left--; - debug(D_VINE | D_NOTICE, "Task %d succeeded but missing sharedfs output %s; retrying node %s (remaining=%d)...", task->task_id, node->outfile_remote_name, node->node_key, node->retry_attempts_left); + debug(D_VINE | D_NOTICE, "Task %d succeeded but missing sharedfs output %s; retrying node %" PRIu64 " (remaining=%d)...", task->task_id, node->outfile_remote_name, node->node_id, node->retry_attempts_left); vine_task_reset(node->task); - submit_node_task(sog, node); + submit_node_task(vg, node); continue; } node->outfile_size_bytes = info.st_size; @@ -1238,7 +1243,7 @@ void sog_execute(struct strategic_orchestration_graph *sog) node->outfile_size_bytes = node->outfile->size; break; } - debug(D_VINE, "Node %s completed with outfile %s size: %zu bytes", node->node_key, node->outfile_remote_name, node->outfile_size_bytes); + debug(D_VINE, "Node %" PRIu64 " completed with outfile %s size: %zu bytes", node->node_id, node->outfile_remote_name, node->outfile_size_bytes); /* mark the node as completed */ node->completed = 1; @@ -1248,7 +1253,7 @@ void sog_execute(struct strategic_orchestration_graph *sog) node->retrieval_time = task->time_when_get_result_end - task->time_when_get_result_start; /* prune nodes on task completion */ - prune_ancestors_of_node(sog, node); + prune_ancestors_of_node(vg, node); /* skip recovery tasks */ if (task->type == VINE_TASK_TYPE_RECOVERY) { @@ -1262,17 +1267,17 @@ void sog_execute(struct strategic_orchestration_graph *sog) } /* update critical time */ - son_update_critical_path_time(node, node->execution_time); + vine_node_update_critical_path_time(node, node->execution_time); /* mark this regular task as completed */ progress_bar_update_part(pbar, regular_tasks_part, 1); /* inject failure */ - if (sog->failure_injection_step_percent > 0) { + if (vg->failure_injection_step_percent > 0) { double progress = (double)regular_tasks_part->current / (double)regular_tasks_part->total; - if (progress >= next_failure_threshold && evict_random_worker(sog->manager)) { + if (progress >= next_failure_threshold && evict_random_worker(vg->manager)) { debug(D_VINE, "evicted a worker at %.2f%% (threshold %.2f%%)", progress * 100, next_failure_threshold * 100); - next_failure_threshold += sog->failure_injection_step_percent / 100.0; + next_failure_threshold += vg->failure_injection_step_percent / 100.0; } } @@ -1280,7 +1285,7 @@ void sog_execute(struct strategic_orchestration_graph *sog) switch (node->outfile_type) { case NODE_OUTFILE_TYPE_TEMP: /* replicate the outfile of the temp node */ - vine_temp_replicate_file_later(sog->manager, node->outfile); + vine_temp_replicate_file_later(vg->manager, node->outfile); break; case NODE_OUTFILE_TYPE_LOCAL: case NODE_OUTFILE_TYPE_SHARED_FILE_SYSTEM: @@ -1288,7 +1293,7 @@ void sog_execute(struct strategic_orchestration_graph *sog) } /* submit children nodes with dependencies all resolved */ - submit_unblocked_children(sog, node); + submit_unblocked_children(vg, node); timestamp_t time_when_postprocessing_end = timestamp_get(); node->postprocessing_time = time_when_postprocessing_end - time_when_postprocessing_start; @@ -1304,7 +1309,7 @@ void sog_execute(struct strategic_orchestration_graph *sog) double total_time_spent_on_unlink_local_files = 0; double total_time_spent_on_prune_ancestors_of_temp_node = 0; double total_time_spent_on_prune_ancestors_of_persisted_node = 0; - HASH_TABLE_ITERATE(sog->nodes, node_key, node) + ITABLE_ITERATE(vg->nodes, nid_iter, node) { total_time_spent_on_unlink_local_files += node->time_spent_on_unlink_local_files; total_time_spent_on_prune_ancestors_of_temp_node += node->time_spent_on_prune_ancestors_of_temp_node; @@ -1318,47 +1323,47 @@ void sog_execute(struct strategic_orchestration_graph *sog) debug(D_VINE, "total time spent on prune ancestors of persisted node: %.6f seconds\n", total_time_spent_on_prune_ancestors_of_persisted_node); debug(D_VINE, "total time spent on unlink local files: %.6f seconds\n", total_time_spent_on_unlink_local_files); - if (sog->time_metrics_filename) { - print_time_metrics(sog, sog->time_metrics_filename); + if (vg->time_metrics_filename) { + print_time_metrics(vg, vg->time_metrics_filename); } return; } /** - * Delete a strategic orchestration graph object. - * @param sog Reference to the strategic orchestration graph object. + * Delete a vine graph instance. + * @param vg Reference to the vine graph. */ -void sog_delete(struct strategic_orchestration_graph *sog) +void vine_graph_delete(struct vine_graph *vg) { - if (!sog) { + if (!vg) { return; } - char *node_key; - struct strategic_orchestration_node *node; - HASH_TABLE_ITERATE(sog->nodes, node_key, node) + uint64_t nid; + struct vine_node *node; + ITABLE_ITERATE(vg->nodes, nid, node) { if (node->infile) { - vine_prune_file(sog->manager, node->infile); - hash_table_remove(sog->manager->file_table, node->infile->cached_name); + vine_prune_file(vg->manager, node->infile); + hash_table_remove(vg->manager->file_table, node->infile->cached_name); } if (node->outfile) { - vine_prune_file(sog->manager, node->outfile); - hash_table_remove(sog->outfile_cachename_to_node, node->outfile->cached_name); - hash_table_remove(sog->manager->file_table, node->outfile->cached_name); + vine_prune_file(vg->manager, node->outfile); + hash_table_remove(vg->outfile_cachename_to_node, node->outfile->cached_name); + hash_table_remove(vg->manager->file_table, node->outfile->cached_name); } if (node->outfile_type == NODE_OUTFILE_TYPE_SHARED_FILE_SYSTEM) { unlink(node->outfile_remote_name); } - son_delete(node); + vine_node_delete(node); } - free(sog->proxy_library_name); - free(sog->proxy_function_name); + free(vg->proxy_library_name); + free(vg->proxy_function_name); - hash_table_delete(sog->nodes); - itable_delete(sog->task_id_to_node); - hash_table_delete(sog->outfile_cachename_to_node); - free(sog); + itable_delete(vg->nodes); + itable_delete(vg->task_id_to_node); + hash_table_delete(vg->outfile_cachename_to_node); + free(vg); } diff --git a/taskvine/src/graph/vinedag/vine_graph/vine_graph.h b/taskvine/src/graph/vinedag/vine_graph/vine_graph.h new file mode 100644 index 0000000000..f749f442ab --- /dev/null +++ b/taskvine/src/graph/vinedag/vine_graph/vine_graph.h @@ -0,0 +1,150 @@ +#ifndef VINE_GRAPH_H +#define VINE_GRAPH_H + +#include + +#include "vine_task.h" +#include "hash_table.h" +#include "itable.h" +#include "list.h" +#include "vine_manager.h" +#include "set.h" +#include "vine_node.h" +#include "taskvine.h" + +/** The task priority algorithm used for vine graph scheduling. */ +typedef enum { + TASK_PRIORITY_MODE_RANDOM = 0, /**< Assign random priority to tasks */ + TASK_PRIORITY_MODE_DEPTH_FIRST, /**< Prioritize deeper tasks first */ + TASK_PRIORITY_MODE_BREADTH_FIRST, /**< Prioritize shallower tasks first */ + TASK_PRIORITY_MODE_FIFO, /**< First in, first out priority */ + TASK_PRIORITY_MODE_LIFO, /**< Last in, first out priority */ + TASK_PRIORITY_MODE_LARGEST_INPUT_FIRST, /**< Prioritize tasks with larger inputs first */ + TASK_PRIORITY_MODE_LARGEST_STORAGE_FOOTPRINT_FIRST /**< Prioritize tasks with larger storage footprint first */ +} task_priority_mode_t; + +/** The vine graph (logical scheduling layer). */ +struct vine_graph { + struct vine_manager *manager; + struct itable *nodes; + struct itable *task_id_to_node; + struct hash_table *outfile_cachename_to_node; + + /* The directory to store the checkpointed results. + * Only intermediate results can be checkpointed, the fraction of intermediate results to checkpoint is controlled by the checkpoint-fraction parameter. */ + char *checkpoint_dir; + + /* Results of target nodes will be stored in this directory. + * This dir path can not necessarily be a shared file system directory, + * output files will be retrieved through the network instead, + * as long as the manager can access it. */ + char *output_dir; + + /* Python-side proxy library name. ContextGraph owns this library and sends calls into the vine graph + * so the manager can execute them through the proxy function. */ + char *proxy_library_name; + + /* The proxy function lives inside that library. It receives vine node IDs, looks up the + * Python callable and arguments inside ContextGraph, and executes the work. ContextGraph generates the name + * and shares it with the vine graph. */ + char *proxy_function_name; + + /* The depth of the pruning strategy. 0 means no pruning, 1 means the most aggressive pruning. */ + int prune_depth; + double checkpoint_fraction; /* 0 - 1, the fraction of intermediate results to checkpoint */ + + task_priority_mode_t task_priority_mode; /* priority mode for task graph task scheduling */ + double failure_injection_step_percent; /* 0 - 100, the percentage of steps to inject failure */ + + double progress_bar_update_interval_sec; /* update interval for the progress bar in seconds */ + + /* The filename of the csv file to store the time metrics of the vine graph. */ + char *time_metrics_filename; + + int enable_debug_log; /* whether to enable debug log */ +}; + +/* Public APIs for operating the vine graph */ + +/** Create a vine graph and return it. +@param q Reference to the current manager object. +@return A new vine graph. +*/ +struct vine_graph *vine_graph_create(struct vine_manager *q); + +/** Create a new node in the vine graph. +@param vg Reference to the vine graph. +@return The auto-assigned node id. +*/ +uint64_t vine_graph_add_node(struct vine_graph *vg); + +/** Mark a node as a retrieval target. +@param vg Reference to the vine graph. +@param node_id Identifier of the node to mark as target. +*/ +void vine_graph_set_target(struct vine_graph *vg, uint64_t node_id); + +/** Add a dependency between two nodes in the vine graph. +@param vg Reference to the vine 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 *vg, uint64_t parent_id, uint64_t child_id); + +/** Finalize the metrics of the vine graph. +@param vg Reference to the vine graph. +*/ +void vine_graph_compute_topology_metrics(struct vine_graph *vg); + +/** Get the heavy score of a node in the vine graph. +@param vg Reference to the vine graph. +@param node_id Identifier of the node. +@return The heavy score. +*/ +double vine_graph_get_node_heavy_score(const struct vine_graph *vg, uint64_t node_id); + +/** Execute the task graph. +@param vg Reference to the vine graph. +*/ +void vine_graph_execute(struct vine_graph *vg); + +/** Get the outfile remote name of a node in the vine graph. +@param vg Reference to the vine 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 *vg, uint64_t node_id); + +/** Get the local outfile source of a node in the vine graph. +@param vg Reference to the vine graph. +@param node_id Identifier of the node. +@return The local outfile source, or NULL if the node does not produce a local file. +*/ +const char *vine_graph_get_node_local_outfile_source(const struct vine_graph *vg, uint64_t node_id); + +/** Delete a vine graph. +@param vg Reference to the vine graph. +*/ +void vine_graph_delete(struct vine_graph *vg); + +/** Get the proxy library name of the vine graph. +@param vg Reference to the vine graph. +@return The proxy library name. +*/ +const char *vine_graph_get_proxy_library_name(const struct vine_graph *vg); + +/** Set the proxy function name of the vine graph. +@param vg Reference to the vine graph. +@param proxy_function_name Reference to the proxy function name. +*/ +void vine_graph_set_proxy_function_name(struct vine_graph *vg, const char *proxy_function_name); + +/** Tune the vine graph. +@param vg Reference to the vine 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 *vg, const char *name, const char *value); + +#endif // VINE_GRAPH_H diff --git a/taskvine/src/graph/vinedag/strategic_orchestration_graph.i b/taskvine/src/graph/vinedag/vine_graph/vine_graph.i similarity index 72% rename from taskvine/src/graph/vinedag/strategic_orchestration_graph.i rename to taskvine/src/graph/vinedag/vine_graph/vine_graph.i index 1f7f7be8eb..b87d428c01 100644 --- a/taskvine/src/graph/vinedag/strategic_orchestration_graph.i +++ b/taskvine/src/graph/vinedag/vine_graph/vine_graph.i @@ -1,9 +1,9 @@ /* SWIG interface for local vinedag graph API bindings */ -%module capi +%module vine_graph_capi %{ #include "int_sizes.h" -#include "strategic_orchestration_graph.h" +#include "vine_graph.h" %} %include "stdint.i" @@ -12,4 +12,4 @@ /* Import existing SWIG interface for type information (do not wrap again) */ %import "../../bindings/python3/taskvine.i" -%include "strategic_orchestration_graph.h" +%include "vine_graph.h" diff --git a/taskvine/src/graph/vinedag/vine_graph/vine_graph_client.py b/taskvine/src/graph/vinedag/vine_graph/vine_graph_client.py new file mode 100644 index 0000000000..6019e6e74c --- /dev/null +++ b/taskvine/src/graph/vinedag/vine_graph/vine_graph_client.py @@ -0,0 +1,75 @@ +# Copyright (C) 2025 The University of Notre Dame +# This software is distributed under the GNU General Public License. +# See the file COPYING for details. + +"""High-level client that exposes the C vine graph to Python callers.""" + +from . import vine_graph_capi + + +class VineGraphClient: + """Python-friendly wrapper that hides the raw SWIG API surface.""" + + def __init__(self, c_taskvine): + """Create and own the lifecycle of the backing C vine graph instance.""" + self._c_graph = vine_graph_capi.vine_graph_create(c_taskvine) + self._key_to_id = {} + self._id_to_key = {} + + def tune(self, name, value): + """Forward tuning parameters directly to the C vine graph.""" + vine_graph_capi.vine_graph_tune(self._c_graph, name, value) + + def add_node(self, key, is_target=None): + """Create a node in the C graph and remember the key↔id mapping.""" + node_id = vine_graph_capi.vine_graph_add_node(self._c_graph) + self._key_to_id[key] = node_id + self._id_to_key[node_id] = 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, key): + """Mark an existing node as a target output.""" + node_id = self._key_to_id.get(key) + if node_id is None: + raise KeyError(f"Key not found: {key}") + vine_graph_capi.vine_graph_set_target(self._c_graph, node_id) + + def add_dependency(self, parent_key, child_key): + """Add an edge in the C graph using the remembered id mapping.""" + if parent_key not in self._key_to_id or child_key not in self._key_to_id: + raise KeyError("parent_key or child_key missing in mapping; call add_node() first") + vine_graph_capi.vine_graph_add_dependency( + self._c_graph, self._key_to_id[parent_key], self._key_to_id[child_key] + ) + + def compute_topology_metrics(self): + """Trigger the C graph to compute depth/height, heavy-score, etc.""" + vine_graph_capi.vine_graph_compute_topology_metrics(self._c_graph) + + def get_node_outfile_remote_name(self, key): + """Ask the C layer where a node's output will be stored.""" + if key not in self._key_to_id: + raise KeyError(f"Key not found: {key}") + return vine_graph_capi.vine_graph_get_node_outfile_remote_name( + self._c_graph, self._key_to_id[key] + ) + + def get_proxy_library_name(self): + """Expose the randomly generated proxy library name from the C side.""" + return vine_graph_capi.vine_graph_get_proxy_library_name(self._c_graph) + + def set_proxy_function(self, proxy_function): + """Tell the C graph which Python function should run on the workers.""" + vine_graph_capi.vine_graph_set_proxy_function_name( + self._c_graph, proxy_function.__name__ + ) + + def execute(self): + """Kick off execution; runs through SWIG down into the C orchestration loop.""" + vine_graph_capi.vine_graph_execute(self._c_graph) + + def delete(self): + """Release the C resources and clear the client.""" + vine_graph_capi.vine_graph_delete(self._c_graph) diff --git a/taskvine/src/graph/vinedag/strategic_orchestration_node.c b/taskvine/src/graph/vinedag/vine_graph/vine_node.c similarity index 74% rename from taskvine/src/graph/vinedag/strategic_orchestration_node.c rename to taskvine/src/graph/vinedag/vine_graph/vine_node.c index 517e3333c4..86e0dfffb4 100644 --- a/taskvine/src/graph/vinedag/strategic_orchestration_node.c +++ b/taskvine/src/graph/vinedag/vine_graph/vine_node.c @@ -21,7 +21,7 @@ #include "vine_task.h" #include "vine_worker_info.h" #include "vine_temp.h" -#include "strategic_orchestration_node.h" +#include "vine_node.h" #include "taskvine.h" /*************************************************************/ @@ -35,7 +35,7 @@ * @param node Reference to the node object. * @return 1 if the outfile is persisted, 0 otherwise. */ -static int node_outfile_has_been_persisted(struct strategic_orchestration_node *node) +static int node_outfile_has_been_persisted(struct vine_node *node) { if (!node) { return 0; @@ -63,10 +63,10 @@ static int node_outfile_has_been_persisted(struct strategic_orchestration_node * * @param node Reference to the node object. * @param execution_time Reference to the execution time of the node. */ -void son_update_critical_path_time(struct strategic_orchestration_node *node, timestamp_t execution_time) +void vine_node_update_critical_path_time(struct vine_node *node, timestamp_t execution_time) { timestamp_t max_parent_critical_path_time = 0; - struct strategic_orchestration_node *parent_node; + struct vine_node *parent_node; LIST_ITERATE(node->parents, parent_node) { if (parent_node->critical_path_time > max_parent_critical_path_time) { @@ -83,7 +83,7 @@ void son_update_critical_path_time(struct strategic_orchestration_node *node, ti * @param result Reference to the result list. * @param visited Reference to the visited set. */ -static void find_parents_dfs(struct strategic_orchestration_node *node, int remaining_depth, struct list *result, struct set *visited) +static void find_parents_dfs(struct vine_node *node, int remaining_depth, struct list *result, struct set *visited) { if (!node || set_lookup(visited, node)) { return; @@ -94,7 +94,7 @@ static void find_parents_dfs(struct strategic_orchestration_node *node, int rema list_push_tail(result, node); return; } - struct strategic_orchestration_node *parent_node; + struct vine_node *parent_node; LIST_ITERATE(node->parents, parent_node) { find_parents_dfs(parent_node, remaining_depth - 1, result, visited); @@ -106,31 +106,21 @@ static void find_parents_dfs(struct strategic_orchestration_node *node, int rema /*************************************************************/ /** - * Create a new node object. - * @param node_key Reference to the node key. - * @param is_target_key Reference to whether the node is a target key. - * @return A new node object. + * Create a new vine node owned by the C-side graph. + * @param node_id Graph-assigned identifier that keeps C and Python in sync. + * @return Newly allocated vine node. */ -struct strategic_orchestration_node *son_create(const char *node_key, int is_target_key) +struct vine_node *vine_node_create(uint64_t node_id) { - if (!node_key) { - debug(D_ERROR, "Cannot create node because node_key is NULL"); - return NULL; - } - if (is_target_key != 0 && is_target_key != 1) { - debug(D_ERROR, "Cannot create node because is_target_key is not 0 or 1"); - return NULL; - } - - struct strategic_orchestration_node *node = xxmalloc(sizeof(struct strategic_orchestration_node)); + struct vine_node *node = xxmalloc(sizeof(struct vine_node)); - node->is_target_key = is_target_key; - node->node_key = xxstrdup(node_key); + node->is_target = 0; + node->node_id = node_id; - /* create a unique outfile remote name for the node */ - cctools_uuid_t id; - cctools_uuid_create(&id); - node->outfile_remote_name = xxstrdup(id.str); + /* create a unique UUID-based remote outfile name for this node */ + cctools_uuid_t uuid; + cctools_uuid_create(&uuid); + node->outfile_remote_name = xxstrdup(uuid.str); node->prune_status = PRUNE_STATUS_NOT_PRUNED; node->parents = list_create(); @@ -168,9 +158,9 @@ struct strategic_orchestration_node *son_create(const char *node_key, int is_tar /** * Construct the task arguments for the node. * @param node Reference to the node object. - * @return The task arguments in JSON format: {"fn_args": [key], "fn_kwargs": {}}. + * @return The task arguments in JSON format: {"fn_args": [node_id], "fn_kwargs": {}}. */ -char *son_construct_task_arguments(struct strategic_orchestration_node *node) +char *vine_node_construct_task_arguments(struct vine_node *node) { if (!node) { return NULL; @@ -178,7 +168,7 @@ char *son_construct_task_arguments(struct strategic_orchestration_node *node) struct jx *event = jx_object(NULL); struct jx *args = jx_array(NULL); - jx_array_append(args, jx_string(node->node_key)); + jx_array_append(args, jx_integer(node->node_id)); jx_insert(event, jx_string("fn_args"), args); jx_insert(event, jx_string("fn_kwargs"), jx_object(NULL)); @@ -194,7 +184,7 @@ char *son_construct_task_arguments(struct strategic_orchestration_node *node) * @param depth Reference to the depth. * @return The list of parents. */ -struct list *son_find_parents_by_depth(struct strategic_orchestration_node *node, int depth) +struct list *vine_node_find_parents_by_depth(struct vine_node *node, int depth) { if (!node || depth < 0) { return NULL; @@ -229,7 +219,7 @@ struct list *son_find_parents_by_depth(struct strategic_orchestration_node *node * @param start_node The node from which to begin the reverse search. * @return A set of ancestor nodes that are safe to prune (excluding start_node). */ -struct set *son_find_safe_ancestors(struct strategic_orchestration_node *start_node) +struct set *vine_node_find_safe_ancestors(struct vine_node *start_node) { if (!start_node) { return NULL; @@ -244,8 +234,8 @@ struct set *son_find_safe_ancestors(struct strategic_orchestration_node *start_n set_insert(visited_nodes, start_node); while (list_size(queue) > 0) { - struct strategic_orchestration_node *current_node = list_pop_head(queue); - struct strategic_orchestration_node *parent_node; + struct vine_node *current_node = list_pop_head(queue); + struct vine_node *parent_node; LIST_ITERATE(current_node->parents, parent_node) { @@ -262,7 +252,7 @@ struct set *son_find_safe_ancestors(struct strategic_orchestration_node *start_n /* check if all children of this parent are safe */ int all_children_safe = 1; - struct strategic_orchestration_node *child_node; + struct vine_node *child_node; LIST_ITERATE(parent_node->children, child_node) { /* shortcut if this child is part of the recovery subgraph */ @@ -298,19 +288,19 @@ struct set *son_find_safe_ancestors(struct strategic_orchestration_node *start_n * Print the info of the node. * @param node Reference to the node object. */ -void son_debug_print(struct strategic_orchestration_node *node) +void vine_node_debug_print(struct vine_node *node) { if (!node) { return; } if (!node->task) { - debug(D_ERROR, "node %s has no task", node->node_key); + debug(D_ERROR, "node %" PRIu64 " has no task", node->node_id); return; } debug(D_VINE, "---------------- Node Info ----------------"); - debug(D_VINE, "key: %s", node->node_key); + debug(D_VINE, "node_id: %" PRIu64, node->node_id); debug(D_VINE, "task_id: %d", node->task->task_id); debug(D_VINE, "depth: %d", node->depth); debug(D_VINE, "height: %d", node->height); @@ -345,38 +335,38 @@ void son_debug_print(struct strategic_orchestration_node *node) debug(D_VINE, "outfile_type: SHARED_FILE_SYSTEM or none"); } - /* print parent and child node keys */ - char *parent_keys = NULL; - struct strategic_orchestration_node *p; + /* print parent and child node ids */ + char *parent_ids = NULL; + struct vine_node *p; LIST_ITERATE(node->parents, p) { - if (!parent_keys) { - parent_keys = string_format("%s", p->node_key); + if (!parent_ids) { + parent_ids = string_format("%" PRIu64, p->node_id); } else { - char *tmp = string_format("%s, %s", parent_keys, p->node_key); - free(parent_keys); - parent_keys = tmp; + char *tmp = string_format("%s, %" PRIu64, parent_ids, p->node_id); + free(parent_ids); + parent_ids = tmp; } } - char *child_keys = NULL; - struct strategic_orchestration_node *c; + char *child_ids = NULL; + struct vine_node *c; LIST_ITERATE(node->children, c) { - if (!child_keys) { - child_keys = string_format("%s", c->node_key); + if (!child_ids) { + child_ids = string_format("%" PRIu64, c->node_id); } else { - char *tmp = string_format("%s, %s", child_keys, c->node_key); - free(child_keys); - child_keys = tmp; + char *tmp = string_format("%s, %" PRIu64, child_ids, c->node_id); + free(child_ids); + child_ids = tmp; } } - debug(D_VINE, "parents: %s", parent_keys ? parent_keys : "(none)"); - debug(D_VINE, "children: %s", child_keys ? child_keys : "(none)"); + debug(D_VINE, "parents: %s", parent_ids ? parent_ids : "(none)"); + debug(D_VINE, "children: %s", child_ids ? child_ids : "(none)"); - free(parent_keys); - free(child_keys); + free(parent_ids); + free(child_ids); debug(D_VINE, "-------------------------------------------"); } @@ -385,15 +375,12 @@ void son_debug_print(struct strategic_orchestration_node *node) * Delete the node and all of its associated resources. * @param node Reference to the node object. */ -void son_delete(struct strategic_orchestration_node *node) +void vine_node_delete(struct vine_node *node) { if (!node) { return; } - if (node->node_key) { - free(node->node_key); - } if (node->outfile_remote_name) { free(node->outfile_remote_name); } diff --git a/taskvine/src/graph/vinedag/vine_graph/vine_node.h b/taskvine/src/graph/vinedag/vine_graph/vine_node.h new file mode 100644 index 0000000000..a838fb6e88 --- /dev/null +++ b/taskvine/src/graph/vinedag/vine_graph/vine_node.h @@ -0,0 +1,115 @@ +#ifndef VINE_NODE_H +#define VINE_NODE_H + +#include + +#include "vine_task.h" +#include "hash_table.h" +#include "list.h" +#include "set.h" +#include "taskvine.h" + +/** The storage type of the node's output file. */ +typedef enum { + NODE_OUTFILE_TYPE_LOCAL = 0, /* Node-output file will be stored locally on the local staging directory */ + NODE_OUTFILE_TYPE_TEMP, /* Node-output file will be stored in the temporary node-local storage */ + NODE_OUTFILE_TYPE_SHARED_FILE_SYSTEM, /* Node-output file will be stored in the persistent shared file system */ +} node_outfile_type_t; + +/** The status of an output file of a node. */ +typedef enum { + PRUNE_STATUS_NOT_PRUNED = 0, + PRUNE_STATUS_SAFE, + PRUNE_STATUS_UNSAFE +} prune_status_t; + +/** The vine node object. */ +struct vine_node { + /* Identity */ + uint64_t node_id; /* Unique identifier assigned by the graph when the node is created. */ + int is_target; /* If true, the output of the node is retrieved when the task finishes. */ + + /* Task and files */ + struct vine_task *task; + struct vine_file *infile; + struct vine_file *outfile; + char *outfile_remote_name; + size_t outfile_size_bytes; + node_outfile_type_t outfile_type; + + /* Graph relationships */ + struct list *parents; + struct list *children; + + /* Execution and scheduling state */ + struct set *pending_parents; + int retry_attempts_left; + int completed; + prune_status_t prune_status; + + /* Structural metrics */ + int prune_depth; + int depth; + int height; + int upstream_subgraph_size; + int downstream_subgraph_size; + int fan_in; + int fan_out; + double heavy_score; + + /* Time metrics */ + timestamp_t critical_path_time; + timestamp_t time_spent_on_unlink_local_files; + timestamp_t time_spent_on_prune_ancestors_of_temp_node; + timestamp_t time_spent_on_prune_ancestors_of_persisted_node; + + timestamp_t submission_time; + timestamp_t scheduling_time; + timestamp_t commit_time; + timestamp_t execution_time; + timestamp_t retrieval_time; + timestamp_t postprocessing_time; +}; + +/** Create a new vine node. +@param node_id Unique node identifier supplied by the owning graph. +@return Newly allocated vine node instance. +*/ +struct vine_node *vine_node_create(uint64_t node_id); + +/** Create the task arguments for a vine node. +@param node Reference to the vine node. +@return The task arguments in JSON format: {"fn_args": [node_id], "fn_kwargs": {}}. +*/ +char *vine_node_construct_task_arguments(struct vine_node *node); + +/** Delete a vine node and release owned resources. +@param node Reference to the vine node. +*/ +void vine_node_delete(struct vine_node *node); + +/** Print information about a vine node. +@param node Reference to the vine node. +*/ +void vine_node_debug_print(struct vine_node *node); + +/** Find all safe ancestors of a vine node. +@param start_node Reference to the start node. +@return The set of safe ancestors. +*/ +struct set *vine_node_find_safe_ancestors(struct vine_node *start_node); + +/** Find all parents of a vine node at a specific depth. +@param node Reference to the node. +@param depth Reference to the depth. +@return The list of parents. +*/ +struct list *vine_node_find_parents_by_depth(struct vine_node *node, int depth); + +/** Update the critical path time of a vine node. +@param node Reference to the vine node. +@param execution_time Reference to the execution time of the node. +*/ +void vine_node_update_critical_path_time(struct vine_node *node, timestamp_t execution_time); + +#endif // VINE_NODE_H \ No newline at end of file diff --git a/taskvine/src/graph/vinedag/vinedag.py b/taskvine/src/graph/vinedag/vinedag.py index 9ec10aefc3..165badaf42 100644 --- a/taskvine/src/graph/vinedag/vinedag.py +++ b/taskvine/src/graph/vinedag/vinedag.py @@ -1,11 +1,14 @@ +# 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 import cvine from ndcctools.taskvine.manager import Manager -from ndcctools.taskvine.vinedag.proxy_library import ProxyLibrary -from ndcctools.taskvine.vinedag.proxy_functions import compute_single_key -from ndcctools.taskvine.vinedag.runtime_execution_graph import RuntimeExecutionGraph, GraphKeyResult -from ndcctools.taskvine.vinedag.strategic_orchestration_graph import StrategicOrchestrationGraph +from ndcctools.taskvine.vinedag.context_graph.proxy_library import ProxyLibrary +from ndcctools.taskvine.vinedag.context_graph.proxy_functions import compute_single_key +from ndcctools.taskvine.vinedag.context_graph.core import ContextGraph, GraphKeyResult +from ndcctools.taskvine.vinedag.vine_graph.vine_graph_client import VineGraphClient import cloudpickle import os @@ -29,6 +32,7 @@ def delete_all_files(root_dir): + """Clean the run-info template directory between runs so stale files never leak into a new DAG.""" if not os.path.exists(root_dir): return for dirpath, dirnames, filenames in os.walk(root_dir): @@ -40,34 +44,45 @@ def delete_all_files(root_dir): print(f"Failed to delete file {file_path}") -# color the text with the given color code +# Nicely format terminal output when printing manager metadata. def color_text(text, color_code): + """Render a colored string for the friendly status banners Vineyard prints at start-up.""" return f"\033[{color_code}m{text}\033[0m" -# convert Dask collection to task dictionary +# Flatten Dask collections into the dict-of-tasks structure the rest of the +# pipeline expects. VineDAG clients often hand us a dict like +# {"result": dask.delayed(...)}; we merge the underlying HighLevelGraphs so +# `ContextGraph` sees the same dict representation C does. def dask_collections_to_task_dict(collection_dict): + """Merge user-facing Dask collections into the flattened task dict the ContextGraph expects.""" assert is_dask_collection is not None from dask.highlevelgraph import HighLevelGraph, ensure_dict if not isinstance(collection_dict, dict): - raise TypeError("Input must be a dict or a HighLevelGraph") + raise TypeError("Input must be a dict") for k, v in collection_dict.items(): if not is_dask_collection(v): raise TypeError(f"Input must be a dict of DaskCollection, but found {k} with type {type(v)}") if dts: + # the new Dask API sub_hlgs = [v.dask for v in collection_dict.values()] hlg = HighLevelGraph.merge(*sub_hlgs).to_dict() else: + # the old Dask API hlg = dask.base.collections_to_dsk(collection_dict.values()) return ensure_dict(hlg) -# compatibility for Dask-created collections +# Accept both plain dicts and Dask collections from callers. Most library users +# hand us `{key: delayed / value}` directly, while some experiments pass a +# fully-expanded legacy Dask dict. This helper normalises both cases so the rest +# of the pipeline only deals with `{task_key: task_expression}`. def ensure_task_dict(collection_dict): + """Normalize user input (raw dict or Dask collection) into a plain `{task_key: expr}` mapping.""" if is_dask_collection and any(is_dask_collection(v) for v in collection_dict.values()): task_dict = dask_collections_to_task_dict(collection_dict) else: @@ -81,6 +96,8 @@ def ensure_task_dict(collection_dict): class GraphParams: def __init__(self): + """Hold all tweakable knobs (manager-side, vine_graph-side, and misc).""" + # Manager-level knobs: fed into `Manager.tune(...)` before execution. self.vine_manager_tuning_params = { "worker-source-max-transfers": 100, "max-retrievals": -1, @@ -91,7 +108,8 @@ def __init__(self): "enforce-worker-eviction-interval": -1, "balance-worker-disk-load": 0, } - self.sog_tuning_params = { + # VineGraph-level knobs: forwarded to the underlying vine graph via VineGraphClient. + self.vine_graph_tuning_params = { "failure-injection-step-percent": -1, "task-priority-mode": "largest-input-first", "prune-depth": 1, @@ -102,33 +120,37 @@ def __init__(self): "time-metrics-filename": "time_metrics.csv", "enable-debug-log": 1, } + # Misc knobs used purely on the Python side (e.g., generate fake outputs). self.other_params = { "schedule": "worst", "libcores": 16, "failure-injection-step-percent": -1, - "extra-task-output-size-mb": ["uniform", 0, 0], - "extra-task-sleep-time": ["uniform", 0, 0], + "extra-task-output-size-mb": [0, 0], + "extra-task-sleep-time": [0, 0], } def print_params(self): - all_params = {**self.vine_manager_tuning_params, **self.sog_tuning_params, **self.other_params} + """Dump current knob values to stdout for debugging.""" + all_params = {**self.vine_manager_tuning_params, **self.vine_graph_tuning_params, **self.other_params} print(json.dumps(all_params, indent=4)) def update_param(self, param_name, new_value): + """Update a single knob, falling back to manager-level if unknown.""" if param_name in self.vine_manager_tuning_params: self.vine_manager_tuning_params[param_name] = new_value - elif param_name in self.sog_tuning_params: - self.sog_tuning_params[param_name] = new_value + elif param_name in self.vine_graph_tuning_params: + self.vine_graph_tuning_params[param_name] = new_value elif param_name in self.other_params: self.other_params[param_name] = new_value else: self.vine_manager_tuning_params[param_name] = new_value def get_value_of(self, param_name): + """Helper so VineDAG can pull a knob value without caring where it lives.""" if param_name in self.vine_manager_tuning_params: return self.vine_manager_tuning_params[param_name] - elif param_name in self.sog_tuning_params: - return self.sog_tuning_params[param_name] + elif param_name in self.vine_graph_tuning_params: + return self.vine_graph_tuning_params[param_name] elif param_name in self.other_params: return self.other_params[param_name] else: @@ -139,20 +161,21 @@ class VineDAG(Manager): def __init__(self, *args, **kwargs): + """Spin up a TaskVine manager that knows how to mirror a Python DAG into the C orchestration layer.""" - # handle SIGINT + # React to Ctrl+C so we can tear down the graphs cleanly. signal.signal(signal.SIGINT, self._on_sigint) self.params = GraphParams() - # delete all files in the run info directory, do this before super().__init__() + # Ensure run-info templates don't accumulate garbage between runs. 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: delete_all_files(self.run_info_template_path) - # initialize the manager + # Boot the underlying TaskVine manager. The TaskVine manager keeps alive until the vinedag object is destroyed super().__init__(*args, **kwargs) self.runtime_directory = cvine.vine_get_runtime_directory(self._taskvine) @@ -161,114 +184,130 @@ def __init__(self, print(f"=== Runtime directory: {color_text(self.runtime_directory, 92)}") def param(self, param_name): + """Convenience accessor so callers can read tuned parameters at runtime.""" return self.params.get_value_of(param_name) def update_params(self, new_params): + """Apply a batch of overrides before constructing graphs. + + All parameter dictionaries—whether set via `update_params()` or passed + to `run(..., params={...})`—flow through here. We funnel each key into + the appropriate bucket (manager/vine_graph/misc). Subsequent runs can override + them by calling this again. + """ 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 get_run_info_path(self): - return os.path.join(self.param("run-info-path"), self.param("run-info-template")) - def tune_manager(self): + """Push our manager-side tuning knobs into the C layer.""" for k, v in self.params.vine_manager_tuning_params.items(): try: self.tune(k, v) except Exception: raise ValueError(f"Unrecognized parameter: {k}") - def tune_sog(self, sog): - for k, v in self.params.sog_tuning_params.items(): - sog.tune(k, str(v)) + def tune_vine_graph(self, vine_graph): + """Push VineGraph-specific tuning knobs before we build the graph.""" + for k, v in self.params.vine_graph_tuning_params.items(): + vine_graph.tune(k, str(v)) - def build_reg(self, task_dict): - reg = RuntimeExecutionGraph( + def build_context_graph(self, task_dict): + """Construct the Python-side DAG wrapper (ContextGraph).""" + context_graph = ContextGraph( task_dict, extra_task_output_size_mb=self.param("extra-task-output-size-mb"), extra_task_sleep_time=self.param("extra-task-sleep-time") ) - return reg + return context_graph - def build_sog(self, reg, target_keys): - assert reg is not None, "Python graph must be built before building the C graph" + def build_vine_graph(self, context_graph, target_keys): + """Mirror the ContextGraph into VineGraph, preserving ordering and targets.""" + assert context_graph is not None, "ContextGraph must be built before building the VineGraph" - sog = StrategicOrchestrationGraph(self._taskvine) + vine_graph = VineGraphClient(self._taskvine) - sog.set_proxy_function(compute_single_key) + vine_graph.set_proxy_function(compute_single_key) - # C side vine task graph must be tuned before adding nodes and dependencies + # Tune both manager and vine_graph before we start adding nodes/edges. self.tune_manager() - self.tune_sog(sog) + self.tune_vine_graph(vine_graph) - topo_order = reg.get_topological_order() + topo_order = context_graph.get_topological_order() + # Build the cross-language mapping as we walk the topo order. for k in topo_order: - sog.add_node(reg.reg_key_to_sog_key[k], int(k in target_keys)) - for pk in reg.parents_of[k]: - sog.add_dependency(reg.reg_key_to_sog_key[pk], reg.reg_key_to_sog_key[k]) + node_id = vine_graph.add_node(k) + context_graph.ckey2vid[k] = node_id + context_graph.vid2ckey[node_id] = k + for pk in context_graph.parents_of[k]: + vine_graph.add_dependency(pk, k) - sog.compute_topology_metrics() + # Now that every node is present, mark which ones are final outputs. + for k in target_keys: + vine_graph.set_target(k) - return sog + vine_graph.compute_topology_metrics() + + return vine_graph def build_graphs(self, task_dict, target_keys): - # build Python DAG (logical topology) - reg = self.build_reg(task_dict) - # build C DAG (physical topology) - sog = self.build_sog(reg, target_keys) + """Create both the ContextGraph and its C counterpart, wiring outputs for later use.""" + # Build the logical (Python) DAG. + context_graph = self.build_context_graph(task_dict) + # Build the physical (C) DAG. + vine_graph = self.build_vine_graph(context_graph, target_keys) - # set outfile remote names in reg from sog, note that these names are automatically generated - # with regard to the checkpointing strategy and the shared file system directory - for sog_key in reg.reg_key_to_sog_key.values(): - outfile_remote_name = sog.get_node_outfile_remote_name(sog_key) - reg.set_outfile_remote_name_of(reg.sog_key_to_reg_key[sog_key], outfile_remote_name) + # Cross-fill the outfile locations so the runtime graph knows where to read/write. + for k in context_graph.ckey2vid: + outfile_remote_name = vine_graph.get_node_outfile_remote_name(k) + context_graph.outfile_remote_name[k] = outfile_remote_name - return reg, sog + return context_graph, vine_graph - def create_proxy_library(self, reg, sog, hoisting_modules, env_files): + def create_proxy_library(self, context_graph, vine_graph, hoisting_modules, env_files): + """Package up the context_graph as a TaskVine library.""" proxy_library = ProxyLibrary(self) proxy_library.add_hoisting_modules(hoisting_modules) proxy_library.add_env_files(env_files) - proxy_library.set_context_loader(RuntimeExecutionGraph.context_loader_func, context_loader_args=[cloudpickle.dumps(reg)]) + proxy_library.set_context_loader(ContextGraph.context_loader_func, context_loader_args=[cloudpickle.dumps(context_graph)]) proxy_library.set_libcores(self.param("libcores")) - proxy_library.set_name(sog.get_proxy_library_name()) + proxy_library.set_name(vine_graph.get_proxy_library_name()) return proxy_library def run(self, collection_dict, target_keys=[], params={}, hoisting_modules=[], env_files={}): + """High-level entry point: normalise input, build graphs, ship the library, execute, and return results.""" # first update the params so that they can be used for the following construction self.update_params(params) task_dict = ensure_task_dict(collection_dict) - with open("test_dv5.pkl", "wb") as f: - cloudpickle.dump(task_dict, f) - - # build graphs from both sides - reg, sog = self.build_graphs(task_dict, target_keys) + # Build both the Python DAG and its C mirror. + context_graph, vine_graph = self.build_graphs(task_dict, target_keys) - # create and install the proxy library on the manager - proxy_library = self.create_proxy_library(reg, sog, hoisting_modules, env_files) + # Ship the execution context to workers via a proxy library. + proxy_library = self.create_proxy_library(context_graph, vine_graph, hoisting_modules, env_files) proxy_library.install() print(f"=== Library serialized size: {color_text(proxy_library.get_context_size(), 92)} MB") - # execute the graph on the C side - sog.execute() + # Kick off execution on the C side. + vine_graph.execute() - # clean up the library instances and template on the manager + # Tear down once we're done so successive runs start clean. proxy_library.uninstall() - # delete the C graph immediately after execution, so that the lifetime of this object is limited to the execution - sog.delete() + # Delete the C graph immediately so its lifetime matches the run. + vine_graph.delete() - # load results of target keys + # Load any requested target outputs back into Python land. results = {} for k in target_keys: - outfile_path = os.path.join(self.param("output-dir"), reg.outfile_remote_name[k]) + outfile_path = os.path.join(self.param("output-dir"), context_graph.outfile_remote_name[k]) results[k] = GraphKeyResult.load_from_path(outfile_path) return results def _on_sigint(self, signum, frame): + """SIGINT handler that delegates to Manager cleanup so workers are released promptly.""" self.__del__() diff --git a/taskvine/src/manager/vine_manager.c b/taskvine/src/manager/vine_manager.c index 7dc74facaa..4e096260e5 100644 --- a/taskvine/src/manager/vine_manager.c +++ b/taskvine/src/manager/vine_manager.c @@ -6121,6 +6121,12 @@ int vine_tune(struct vine_manager *q, const char *name, double value) } else if (!strcmp(name, "balance-worker-disk-load")) { q->balance_worker_disk_load = !!((int)value); + } else if (!strcmp(name, "enable-debug-log")) { + if (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/worker/vine_worker.c b/taskvine/src/worker/vine_worker.c index d1ab432cef..b830fb2249 100644 --- a/taskvine/src/worker/vine_worker.c +++ b/taskvine/src/worker/vine_worker.c @@ -1706,7 +1706,7 @@ static void vine_worker_serve_manager(struct link *manager) hence a maximum wait time of five seconds is enforced. */ - int wait_msec = 5000; + int wait_msec = 0; if (sigchld_received_flag) { wait_msec = 0; From cca31701e4bd45add5f83a9309dc29753142996d Mon Sep 17 00:00:00 2001 From: Jin Zhou Date: Sat, 1 Nov 2025 00:00:45 -0400 Subject: [PATCH 067/113] git ignore --- taskvine/src/graph/vinedag/vine_graph/.gitignore | 1 + 1 file changed, 1 insertion(+) create mode 100644 taskvine/src/graph/vinedag/vine_graph/.gitignore diff --git a/taskvine/src/graph/vinedag/vine_graph/.gitignore b/taskvine/src/graph/vinedag/vine_graph/.gitignore new file mode 100644 index 0000000000..15309787ad --- /dev/null +++ b/taskvine/src/graph/vinedag/vine_graph/.gitignore @@ -0,0 +1 @@ +*.o \ No newline at end of file From c1b4e67ea7ce2d2068854d78917c51c8fc958a0b Mon Sep 17 00:00:00 2001 From: Jin Zhou Date: Mon, 3 Nov 2025 09:47:39 -0500 Subject: [PATCH 068/113] revert vine_manager_put_task --- .../graph/vinedag/context_graph/__init__.py | 4 +- .../src/graph/vinedag/context_graph/core.py | 8 +- .../vinedag/context_graph/proxy_library.py | 4 +- .../src/graph/vinedag/vine_graph/vine_graph.h | 4 +- taskvine/src/graph/vinedag/vinedag.py | 4 +- taskvine/src/manager/vine_manager_put.c | 269 +----------------- 6 files changed, 13 insertions(+), 280 deletions(-) diff --git a/taskvine/src/graph/vinedag/context_graph/__init__.py b/taskvine/src/graph/vinedag/context_graph/__init__.py index cbb2cafd99..7d8b678cc5 100644 --- a/taskvine/src/graph/vinedag/context_graph/__init__.py +++ b/taskvine/src/graph/vinedag/context_graph/__init__.py @@ -3,14 +3,14 @@ # See the file COPYING for details. -from .core import ContextGraph, GraphKeyResult +from .core import ContextGraph, ContextGraphTaskResult from .proxy_functions import compute_single_key, compute_dts_key, compute_sexpr_key from .proxy_library import ProxyLibrary __all__ = [ "ContextGraph", - "GraphKeyResult", + "ContextGraphTaskResult", "compute_single_key", "compute_dts_key", "compute_sexpr_key", diff --git a/taskvine/src/graph/vinedag/context_graph/core.py b/taskvine/src/graph/vinedag/context_graph/core.py index c3708c590c..61bbeb0d30 100644 --- a/taskvine/src/graph/vinedag/context_graph/core.py +++ b/taskvine/src/graph/vinedag/context_graph/core.py @@ -32,7 +32,7 @@ def hashable(s): # Lightweight wrapper around task results that optionally pads the payload. The # padding lets tests model large outputs without altering the logical result. -class GraphKeyResult: +class ContextGraphTaskResult: def __init__(self, result, extra_size_mb=None): """Store the real user result plus optional padding used during regression tests.""" self.result = result @@ -47,7 +47,7 @@ def load_from_path(path): try: with open(path, "rb") as f: result_obj = cloudpickle.load(f) - assert isinstance(result_obj, GraphKeyResult), "Loaded object is not of type GraphKeyResult" + assert isinstance(result_obj, ContextGraphTaskResult), "Loaded object is not of type ContextGraphTaskResult" return result_obj.result except FileNotFoundError: raise FileNotFoundError(f"Output file not found at {path}") @@ -149,7 +149,7 @@ def _find_sexpr_parents(sexpr): def save_result_of_key(self, key, result): """Called from the proxy function to persist a result into disk after the worker finishes.""" with open(self.outfile_remote_name[key], "wb") as f: - result_obj = GraphKeyResult(result, extra_size_mb=self.extra_task_output_size_mb[key]) + result_obj = ContextGraphTaskResult(result, extra_size_mb=self.extra_task_output_size_mb[key]) cloudpickle.dump(result_obj, f) def load_result_of_key(self, key): @@ -158,7 +158,7 @@ def load_result_of_key(self, key): # if a node-local output, then data is stored in the task sandbox and the remote name is just the filename # if a shared file system output, then remote name is the full path to the file outfile_path = self.outfile_remote_name[key] - return GraphKeyResult.load_from_path(outfile_path) + return ContextGraphTaskResult.load_from_path(outfile_path) def get_topological_order(self): """Produce the order VineDAG uses when assigning node IDs to the C graph.""" diff --git a/taskvine/src/graph/vinedag/context_graph/proxy_library.py b/taskvine/src/graph/vinedag/context_graph/proxy_library.py index 2bc4dd3c7c..a0c4fb4377 100644 --- a/taskvine/src/graph/vinedag/context_graph/proxy_library.py +++ b/taskvine/src/graph/vinedag/context_graph/proxy_library.py @@ -11,7 +11,7 @@ import hashlib import collections -from ndcctools.taskvine.vinedag.context_graph.core import GraphKeyResult, ContextGraph +from ndcctools.taskvine.vinedag.context_graph.core import ContextGraphTaskResult, ContextGraph from ndcctools.taskvine.vinedag.context_graph.proxy_functions import compute_dts_key, compute_sexpr_key, compute_single_key from ndcctools.taskvine.utils import load_variable_from_library @@ -28,7 +28,7 @@ def __init__(self, py_manager): # 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, GraphKeyResult, ContextGraph, uuid, hashlib, random, types, collections, time, + os, cloudpickle, ContextGraphTaskResult, ContextGraph, uuid, hashlib, random, types, collections, time, load_variable_from_library, compute_dts_key, compute_sexpr_key, compute_single_key ] diff --git a/taskvine/src/graph/vinedag/vine_graph/vine_graph.h b/taskvine/src/graph/vinedag/vine_graph/vine_graph.h index f749f442ab..0d9edb2a9b 100644 --- a/taskvine/src/graph/vinedag/vine_graph/vine_graph.h +++ b/taskvine/src/graph/vinedag/vine_graph/vine_graph.h @@ -40,12 +40,12 @@ struct vine_graph { * as long as the manager can access it. */ char *output_dir; - /* Python-side proxy library name. ContextGraph owns this library and sends calls into the vine graph + /* Python-side proxy library name. The context_graph runtime owns this library and sends calls into the vine graph * so the manager can execute them through the proxy function. */ char *proxy_library_name; /* The proxy function lives inside that library. It receives vine node IDs, looks up the - * Python callable and arguments inside ContextGraph, and executes the work. ContextGraph generates the name + * Python callable and arguments inside the context_graph runtime, and executes the work. The runtime generates the name * and shares it with the vine graph. */ char *proxy_function_name; diff --git a/taskvine/src/graph/vinedag/vinedag.py b/taskvine/src/graph/vinedag/vinedag.py index 165badaf42..f3be364466 100644 --- a/taskvine/src/graph/vinedag/vinedag.py +++ b/taskvine/src/graph/vinedag/vinedag.py @@ -7,7 +7,7 @@ from ndcctools.taskvine.vinedag.context_graph.proxy_library import ProxyLibrary from ndcctools.taskvine.vinedag.context_graph.proxy_functions import compute_single_key -from ndcctools.taskvine.vinedag.context_graph.core import ContextGraph, GraphKeyResult +from ndcctools.taskvine.vinedag.context_graph.core import ContextGraph, ContextGraphTaskResult from ndcctools.taskvine.vinedag.vine_graph.vine_graph_client import VineGraphClient import cloudpickle @@ -305,7 +305,7 @@ def run(self, collection_dict, target_keys=[], params={}, hoisting_modules=[], e results = {} for k in target_keys: outfile_path = os.path.join(self.param("output-dir"), context_graph.outfile_remote_name[k]) - results[k] = GraphKeyResult.load_from_path(outfile_path) + results[k] = ContextGraphTaskResult.load_from_path(outfile_path) return results def _on_sigint(self, signum, frame): diff --git a/taskvine/src/manager/vine_manager_put.c b/taskvine/src/manager/vine_manager_put.c index 0b95d62c31..9076124698 100644 --- a/taskvine/src/manager/vine_manager_put.c +++ b/taskvine/src/manager/vine_manager_put.c @@ -476,7 +476,7 @@ It does not perform any resource management. This allows it to be used for both regular tasks and mini tasks. */ -vine_result_code_t vine_manager_put_task_old( +vine_result_code_t vine_manager_put_task( struct vine_manager *q, struct vine_worker_info *w, struct vine_task *t, const char *command_line, struct rmsummary *limits, struct vine_file *target) { if (target) { @@ -590,270 +590,3 @@ vine_result_code_t vine_manager_put_task_old( return VINE_WORKER_FAILURE; } } - -/* - * Extreme-optimized implementation of vine_manager_put_task() - * - * Key ideas: - * - Build the entire task definition (header + command payload + tail) - * in one contiguous growable buffer. - * - Send the full buffer in one (or very few) link_putlstring() calls. - * - No nested functions, no temp stack buffers, no redundant syscalls. - * - Protocol and API remain fully backward compatible. - * - Achieves order-of-magnitude throughput improvement for short tasks. - */ - -#include -#include -#include -#include -#include -#include - -/* ---------------------------------------------------------------------- */ -/* Small dynamic buffer utility (local to this file) */ -/* ---------------------------------------------------------------------- */ - -struct sbuf { - char *buf; - size_t len; - size_t cap; -}; - -#define SB_INIT_CAP 4096 -#define SB_MAX_CHUNK (1024 * 1024) /* 1 MB max per write */ - -/* Reserve space for additional bytes. Grows buffer exponentially. */ -#define SB_RESERVE(B, NEED) \ - do { \ - size_t need = (B).len + (size_t)(NEED) + 1; \ - if (need > (B).cap) { \ - size_t cap = (B).cap ? (B).cap : SB_INIT_CAP; \ - while (cap < need) \ - cap <<= 1; \ - char *nb = (char *)realloc((B).buf, cap); \ - if (!nb) { \ - free((B).buf); \ - return VINE_WORKER_FAILURE; \ - } \ - (B).buf = nb; \ - (B).cap = cap; \ - } \ - } while (0) - -/* Append formatted text directly into buffer. */ -static inline int sb_vprintf(struct sbuf *B, const char *fmt, va_list ap0) -{ - va_list ap; - va_copy(ap, ap0); - - size_t avail = (B->cap > B->len) ? (B->cap - B->len) : 0; - if (avail < 64) { - SB_RESERVE((*B), 64); - avail = B->cap - B->len; - } - - int n = vsnprintf(B->buf + B->len, avail, fmt, ap); - va_end(ap); - if (n < 0) - return -1; - - /* If output was truncated, expand and retry */ - if ((size_t)n >= avail) { - SB_RESERVE((*B), (size_t)n); - va_copy(ap, ap0); - n = vsnprintf(B->buf + B->len, B->cap - B->len, fmt, ap); - va_end(ap); - if (n < 0) - return -1; - } - - B->len += (size_t)n; - return n; -} - -/* Append formatted string. */ -static inline int sb_printf(struct sbuf *B, const char *fmt, ...) -{ - va_list ap; - va_start(ap, fmt); - int n = sb_vprintf(B, fmt, ap); - va_end(ap); - return n; -} - -/* Append raw bytes. */ -static inline int sb_append(struct sbuf *B, const void *p, size_t n) -{ - SB_RESERVE((*B), n); - memcpy(B->buf + B->len, p, n); - B->len += n; - B->buf[B->len] = '\0'; - return 0; -} - -/* ---------------------------------------------------------------------- */ -/* Main function: vine_manager_put_task() */ -/* ---------------------------------------------------------------------- */ - -vine_result_code_t vine_manager_put_task( - struct vine_manager *q, struct vine_worker_info *w, - struct vine_task *t, const char *command_line, - struct rmsummary *limits, struct vine_file *target) -{ - /* If target already on worker */ - if (target && vine_file_replica_table_lookup(w, target->cached_name)) { - debug(D_NOTICE, "mini_task %s already at %s", target->cached_name, w->addrport); - return VINE_SUCCESS; - } - - /* Send input files first */ - vine_result_code_t result = vine_manager_put_input_files(q, w, t); - if (result != VINE_SUCCESS) - return result; - - if (!command_line) - command_line = t->command_line; - const long long cmd_len = (long long)strlen(command_line); - - struct sbuf sb = {0}; - - /* helper: exact same debug prefix format as vine_manager_send() */ - auto void dbg_tx(struct vine_worker_info * ww, const char *s, size_t n) - { - if (!s || n == 0) - return; - debug(D_VINE, "tx to %s (%s): %.*s", ww->hostname, ww->addrport, (int)n, s); - } - -/* helper: format one logical line, append to sb, and debug it */ -#ifndef VINE_LINE_MAX -#define VINE_LINE_MAX 65536 -#endif - auto void emitf(const char *fmt, ...) - { - char line[VINE_LINE_MAX]; - va_list ap; - va_start(ap, fmt); - int n = vsnprintf(line, sizeof(line), fmt, ap); - va_end(ap); - if (n < 0) - n = 0; - size_t len = (size_t)n; - if (len >= sizeof(line)) - len = sizeof(line) - 1; /* truncate for debug parity safety */ - sb_printf(&sb, "%.*s", (int)len, line); - dbg_tx(w, line, len); - } - - /* Header */ - if (target) { - int mode = target->mode ? target->mode : 0755; - emitf("mini_task %s %s %d %lld 0%o\n", - target->source, - target->cached_name, - target->cache_level, - (long long)target->size, - mode); - } else { - emitf("task %lld\n", (long long)t->task_id); - } - - /* cmd header + payload */ - emitf("cmd %lld\n", cmd_len); - sb_append(&sb, command_line, (size_t)cmd_len); - dbg_tx(w, command_line, (size_t)cmd_len); - emitf("\n"); - - /* extras */ - if (t->needs_library) { - emitf("needs_library %s\n", t->needs_library); - } - - if (t->provides_library) { - emitf("provides_library %s\n", t->provides_library); - emitf("function_slots %d\n", t->function_slots_total); - emitf("func_exec_mode %d\n", t->func_exec_mode); - } - - emitf("category %s\n", t->category); - - if (limits) { - emitf("cores %s\n", rmsummary_resource_to_str("cores", limits->cores, 0)); - emitf("gpus %s\n", rmsummary_resource_to_str("gpus", limits->gpus, 0)); - emitf("memory %s\n", rmsummary_resource_to_str("memory", limits->memory, 0)); - emitf("disk %s\n", rmsummary_resource_to_str("disk", limits->disk, 0)); - - if (q->monitor_mode == VINE_MON_DISABLED) { - if (limits->end > 0) - emitf("end_time %s\n", rmsummary_resource_to_str("end", limits->end, 0)); - if (limits->wall_time > 0) - emitf("wall_time %s\n", rmsummary_resource_to_str("wall_time", limits->wall_time, 0)); - } - } - - /* env list */ - char *var; - LIST_ITERATE(t->env_list, var) - { - size_t L = strlen(var); - emitf("env %zu\n", L); - sb_append(&sb, var, L); - dbg_tx(w, var, L); - emitf("\n"); - } - - /* infile mounts */ - if (t->input_mounts) { - struct vine_mount *m; - LIST_ITERATE(t->input_mounts, m) - { - char enc[PATH_MAX]; - url_encode(m->remote_name, enc, PATH_MAX); - emitf("infile %s %s %d\n", m->file->cached_name, enc, m->flags); - } - } - - /* outfile mounts */ - if (t->output_mounts) { - struct vine_mount *m; - LIST_ITERATE(t->output_mounts, m) - { - char enc[PATH_MAX]; - url_encode(m->remote_name, enc, PATH_MAX); - emitf("outfile %s %s %d\n", m->file->cached_name, enc, m->flags); - } - } - - if (t->group_id) { - emitf("groupid %d\n", t->group_id); - } - - emitf("end\n"); - - /* Send in chunks (no extra debug here to avoid duplicate logs) */ - const char *p = sb.buf; - size_t left = sb.len; - time_t deadline = time(0) + q->short_timeout; - int r = 0; - - while (left > 0) { - size_t chunk = left > SB_MAX_CHUNK ? SB_MAX_CHUNK : left; - r = link_putlstring(w->link, p, chunk, deadline); - if (r < 0) - break; - p += chunk; - left -= chunk; - } - - if (r >= 0 && left == 0) { - if (target) { - vine_file_replica_table_get_or_create(q, w, target->cached_name, target->type, target->cache_level, target->size, target->mtime); - } - free(sb.buf); - return VINE_SUCCESS; - } else { - free(sb.buf); - return VINE_WORKER_FAILURE; - } -} From c0cc11602ec5da35da4e65d0236a6b3fd9e6672f Mon Sep 17 00:00:00 2001 From: Jin Zhou Date: Mon, 3 Nov 2025 10:38:29 -0500 Subject: [PATCH 069/113] traverse w->current_libraries instead of w->current_tasks --- taskvine/src/manager/vine_manager.c | 10 +++++----- 1 file changed, 5 insertions(+), 5 deletions(-) diff --git a/taskvine/src/manager/vine_manager.c b/taskvine/src/manager/vine_manager.c index ca72d257d9..7ebcdb3ede 100644 --- a/taskvine/src/manager/vine_manager.c +++ b/taskvine/src/manager/vine_manager.c @@ -3067,12 +3067,12 @@ static void find_max_worker(struct vine_manager *q) * are not counted towards the resources in use and will be killed if needed. */ static void kill_empty_libraries_on_worker(struct vine_manager *q, struct vine_worker_info *w, struct vine_task *t) { - uint64_t task_id; - struct vine_task *task; - ITABLE_ITERATE(w->current_tasks, task_id, task) + uint64_t libtask_id; + struct vine_task *libtask; + ITABLE_ITERATE(w->current_libraries, libtask_id, libtask) { - if (task->provides_library && task->function_slots_inuse == 0 && (!t->needs_library || strcmp(t->needs_library, task->provides_library))) { - vine_cancel_by_task_id(q, task->task_id); + if (libtask->function_slots_inuse == 0 && (!t->needs_library || strcmp(t->needs_library, libtask->provides_library))) { + vine_cancel_by_task_id(q, libtask_id); } } } From a8dd136e11112bc91d0d2a4f6f4b03a98389eaf8 Mon Sep 17 00:00:00 2001 From: Jin Zhou Date: Mon, 3 Nov 2025 10:41:48 -0500 Subject: [PATCH 070/113] merge --- taskvine/src/graph/README.md | 2 -- taskvine/src/graph/vinedag/vine_graph/__init__.py | 8 ++------ taskvine/src/graph/vinedag/vine_graph/vine_graph.h | 14 +++++++------- taskvine/src/graph/vinedag/vine_graph/vine_node.h | 6 +++--- taskvine/src/graph/vinedag/vinedag.py | 1 + taskvine/src/manager/vine_manager.c | 2 +- 6 files changed, 14 insertions(+), 19 deletions(-) delete mode 100644 taskvine/src/graph/README.md diff --git a/taskvine/src/graph/README.md b/taskvine/src/graph/README.md deleted file mode 100644 index 8d7a3f59fc..0000000000 --- a/taskvine/src/graph/README.md +++ /dev/null @@ -1,2 +0,0 @@ -# Graph Executors - diff --git a/taskvine/src/graph/vinedag/vine_graph/__init__.py b/taskvine/src/graph/vinedag/vine_graph/__init__.py index 4dd6f218ce..536eed6b88 100644 --- a/taskvine/src/graph/vinedag/vine_graph/__init__.py +++ b/taskvine/src/graph/vinedag/vine_graph/__init__.py @@ -2,13 +2,9 @@ # This software is distributed under the GNU General Public License. # See the file COPYING for details. -"""Namespace for the C vine graph bindings and Python client.""" - -try: # pragma: no cover - module only exists after building the SWIG bindings - from . import vine_graph_capi -except ImportError: - vine_graph_capi = None +from . import vine_graph_capi from .vine_graph_client import VineGraphClient + __all__ = ["vine_graph_capi", "VineGraphClient"] diff --git a/taskvine/src/graph/vinedag/vine_graph/vine_graph.h b/taskvine/src/graph/vinedag/vine_graph/vine_graph.h index 0d9edb2a9b..e3f78f5c83 100644 --- a/taskvine/src/graph/vinedag/vine_graph/vine_graph.h +++ b/taskvine/src/graph/vinedag/vine_graph/vine_graph.h @@ -14,13 +14,13 @@ /** The task priority algorithm used for vine graph scheduling. */ typedef enum { - TASK_PRIORITY_MODE_RANDOM = 0, /**< Assign random priority to tasks */ - TASK_PRIORITY_MODE_DEPTH_FIRST, /**< Prioritize deeper tasks first */ - TASK_PRIORITY_MODE_BREADTH_FIRST, /**< Prioritize shallower tasks first */ - TASK_PRIORITY_MODE_FIFO, /**< First in, first out priority */ - TASK_PRIORITY_MODE_LIFO, /**< Last in, first out priority */ - TASK_PRIORITY_MODE_LARGEST_INPUT_FIRST, /**< Prioritize tasks with larger inputs first */ - TASK_PRIORITY_MODE_LARGEST_STORAGE_FOOTPRINT_FIRST /**< Prioritize tasks with larger storage footprint first */ + TASK_PRIORITY_MODE_RANDOM = 0, /**< Assign random priority to tasks */ + TASK_PRIORITY_MODE_DEPTH_FIRST, /**< Prioritize deeper tasks first */ + TASK_PRIORITY_MODE_BREADTH_FIRST, /**< Prioritize shallower tasks first */ + TASK_PRIORITY_MODE_FIFO, /**< First in, first out priority */ + TASK_PRIORITY_MODE_LIFO, /**< Last in, first out priority */ + TASK_PRIORITY_MODE_LARGEST_INPUT_FIRST, /**< Prioritize tasks with larger inputs first */ + TASK_PRIORITY_MODE_LARGEST_STORAGE_FOOTPRINT_FIRST /**< Prioritize tasks with larger storage footprint first */ } task_priority_mode_t; /** The vine graph (logical scheduling layer). */ diff --git a/taskvine/src/graph/vinedag/vine_graph/vine_node.h b/taskvine/src/graph/vinedag/vine_graph/vine_node.h index a838fb6e88..9f01e959c0 100644 --- a/taskvine/src/graph/vinedag/vine_graph/vine_node.h +++ b/taskvine/src/graph/vinedag/vine_graph/vine_node.h @@ -11,9 +11,9 @@ /** The storage type of the node's output file. */ typedef enum { - NODE_OUTFILE_TYPE_LOCAL = 0, /* Node-output file will be stored locally on the local staging directory */ - NODE_OUTFILE_TYPE_TEMP, /* Node-output file will be stored in the temporary node-local storage */ - NODE_OUTFILE_TYPE_SHARED_FILE_SYSTEM, /* Node-output file will be stored in the persistent shared file system */ + NODE_OUTFILE_TYPE_LOCAL = 0, /* Node-output file will be stored locally on the local staging directory */ + NODE_OUTFILE_TYPE_TEMP, /* Node-output file will be stored in the temporary node-local storage */ + NODE_OUTFILE_TYPE_SHARED_FILE_SYSTEM, /* Node-output file will be stored in the persistent shared file system */ } node_outfile_type_t; /** The status of an output file of a node. */ diff --git a/taskvine/src/graph/vinedag/vinedag.py b/taskvine/src/graph/vinedag/vinedag.py index f3be364466..77975df3f5 100644 --- a/taskvine/src/graph/vinedag/vinedag.py +++ b/taskvine/src/graph/vinedag/vinedag.py @@ -177,6 +177,7 @@ def __init__(self, # Boot the underlying TaskVine manager. The TaskVine manager keeps alive until the vinedag object is destroyed super().__init__(*args, **kwargs) + print(f"cvine = {cvine}") self.runtime_directory = cvine.vine_get_runtime_directory(self._taskvine) print(f"=== Manager name: {color_text(self.name, 92)}") diff --git a/taskvine/src/manager/vine_manager.c b/taskvine/src/manager/vine_manager.c index b1a405564e..d6aa1942e0 100644 --- a/taskvine/src/manager/vine_manager.c +++ b/taskvine/src/manager/vine_manager.c @@ -3149,7 +3149,7 @@ static vine_result_code_t commit_task_to_worker(struct vine_manager *q, struct v /* Kill unused libraries on this worker to reclaim resources. */ /* Matches assumption in vine_schedule.c:check_worker_have_enough_resources() */ - // kill_empty_libraries_on_worker(q, w, t); + kill_empty_libraries_on_worker(q, w, t); /* If this is a function needing a library, dispatch the library. */ if (t->needs_library) { From 0372e722d30eae7e1f693576a45dd2dbb8cc4502 Mon Sep 17 00:00:00 2001 From: Jin Zhou Date: Mon, 3 Nov 2025 10:44:37 -0500 Subject: [PATCH 071/113] revert push_task_to_ready_tasks --- taskvine/src/manager/vine_manager.c | 10 +++++----- 1 file changed, 5 insertions(+), 5 deletions(-) diff --git a/taskvine/src/manager/vine_manager.c b/taskvine/src/manager/vine_manager.c index d6aa1942e0..e6ff9a3660 100644 --- a/taskvine/src/manager/vine_manager.c +++ b/taskvine/src/manager/vine_manager.c @@ -170,7 +170,7 @@ static void delete_uncacheable_files(struct vine_manager *q, struct vine_worker_ static int release_worker(struct vine_manager *q, struct vine_worker_info *w); struct vine_task *send_library_to_worker(struct vine_manager *q, struct vine_worker_info *w, const char *name); -static void enqueue_ready_task(struct vine_manager *q, struct vine_task *t); +static void push_task_to_ready_tasks(struct vine_manager *q, struct vine_task *t); static void clean_redundant_replicas(struct vine_manager *q, struct vine_file *f); @@ -3712,7 +3712,7 @@ static int rotate_pending_tasks(struct vine_manager *q) } if (consider_task(q, t)) { - enqueue_ready_task(q, t); + push_task_to_ready_tasks(q, t); runnable_tasks++; } else { list_push_tail(q->pending_tasks, t); @@ -3814,7 +3814,7 @@ static int send_one_task(struct vine_manager *q, int *tasks_ready_left_to_consid /* put back all tasks that were skipped */ while ((t = list_pop_head(skipped_tasks))) { - enqueue_ready_task(q, t); + push_task_to_ready_tasks(q, t); } list_delete(skipped_tasks); @@ -4809,7 +4809,7 @@ char *vine_monitor_wrap(struct vine_manager *q, struct vine_worker_info *w, stru } /* Put a given task on the ready queue, taking into account the task priority and the manager schedule. */ -static void enqueue_ready_task(struct vine_manager *q, struct vine_task *t) +static void push_task_to_ready_tasks(struct vine_manager *q, struct vine_task *t) { if (t->result == VINE_RESULT_RESOURCE_EXHAUSTION) { /* when a task is resubmitted given resource exhaustion, we @@ -4873,7 +4873,7 @@ static vine_task_state_t change_task_state(struct vine_manager *q, struct vine_t break; case VINE_TASK_READY: vine_task_set_result(t, VINE_RESULT_UNKNOWN); - enqueue_ready_task(q, t); + push_task_to_ready_tasks(q, t); c->vine_stats->tasks_waiting++; break; case VINE_TASK_RUNNING: From 10593213a2954456f8bfefa721780c48f422de9e Mon Sep 17 00:00:00 2001 From: Jin Zhou Date: Mon, 3 Nov 2025 10:49:00 -0500 Subject: [PATCH 072/113] vine: multiply task priority on resource exhaustion --- taskvine/src/manager/vine_manager.c | 7 +++---- 1 file changed, 3 insertions(+), 4 deletions(-) diff --git a/taskvine/src/manager/vine_manager.c b/taskvine/src/manager/vine_manager.c index ca72d257d9..b819b2e16c 100644 --- a/taskvine/src/manager/vine_manager.c +++ b/taskvine/src/manager/vine_manager.c @@ -4668,14 +4668,13 @@ static void push_task_to_ready_tasks(struct vine_manager *q, struct vine_task *t { if (t->result == VINE_RESULT_RESOURCE_EXHAUSTION) { /* when a task is resubmitted given resource exhaustion, we - * increment its priority by 1, so it gets to run as soon + * increment its priority a bit, so it gets to run as soon * as possible among those with the same priority. This avoids * the issue in which all 'big' tasks fail because the first * allocation is too small. */ - priority_queue_push(q->ready_tasks, t, t->priority + 1); - } else { - priority_queue_push(q->ready_tasks, t, t->priority); + t->priority *= 1.05; } + priority_queue_push(q->ready_tasks, t, t->priority); /* If the task has been used before, clear out accumulated state. */ vine_task_clean(t); From 78501414f766481e1e4e15e6ae784e200ffcaaa2 Mon Sep 17 00:00:00 2001 From: Jin Zhou Date: Mon, 3 Nov 2025 15:47:36 -0500 Subject: [PATCH 073/113] vine: clean redundant input replicas upon task completion --- taskvine/src/manager/vine_manager.c | 80 +++++++++++++++++++++++++++++ 1 file changed, 80 insertions(+) diff --git a/taskvine/src/manager/vine_manager.c b/taskvine/src/manager/vine_manager.c index ca72d257d9..d67c822179 100644 --- a/taskvine/src/manager/vine_manager.c +++ b/taskvine/src/manager/vine_manager.c @@ -170,6 +170,8 @@ static int release_worker(struct vine_manager *q, struct vine_worker_info *w); struct vine_task *send_library_to_worker(struct vine_manager *q, struct vine_worker_info *w, const char *name); +static void clean_redundant_replicas(struct vine_manager *q, struct vine_file *f); + /* Return the number of workers matching a given type: WORKER, STATUS, etc */ static int count_workers(struct vine_manager *q, vine_worker_type_t type) @@ -650,6 +652,15 @@ static vine_result_code_t get_completion_result(struct vine_manager *q, struct v itable_remove(q->running_table, t->task_id); vine_task_set_result(t, task_status); + /* Clean redundant replicas for the inputs of the task. */ + struct vine_mount *input_mount; + LIST_ITERATE(t->input_mounts, input_mount) + { + if (input_mount->file && input_mount->file->type == VINE_TEMP) { + clean_redundant_replicas(q, input_mount->file); + } + } + return VINE_SUCCESS; } @@ -1001,6 +1012,75 @@ static int enforce_worker_eviction_interval(struct vine_manager *q) return 1; } +/** Clean redundant replicas of a temporary file. +For example, a file may be transferred to another worker because a task that declares it +as input is scheduled there, resulting in an extra replica that consumes storage space. +This function evaluates whether the file has excessive replicas and removes those on +workers that do not execute their dependent tasks. */ +static void clean_redundant_replicas(struct vine_manager *q, struct vine_file *f) +{ + if (!f || f->type != VINE_TEMP) { + return; + } + + struct set *source_workers = hash_table_lookup(q->file_worker_table, f->cached_name); + if (!source_workers) { + /* no surprise - a cache-update message may trigger a file deletion. */ + return; + } + int excess_replicas = set_size(source_workers) - q->temp_replica_count; + if (excess_replicas <= 0) { + return; + } + /* Note that this replica may serve as a source for a peer transfer. If it is unlinked prematurely, + * the corresponding transfer could fail and leave a task without its required data. + * Therefore, we must wait until all replicas are confirmed ready before proceeding. */ + if (vine_file_replica_table_count_replicas(q, f->cached_name, VINE_FILE_REPLICA_STATE_READY) != set_size(source_workers)) { + return; + } + + struct priority_queue *clean_replicas_from_workers = priority_queue_create(0); + + struct vine_worker_info *source_worker = NULL; + SET_ITERATE(source_workers, source_worker) + { + /* if the file is actively in use by a task (the input to that task), we don't remove the replica on this worker */ + int file_inuse = 0; + + uint64_t task_id; + struct vine_task *task; + ITABLE_ITERATE(source_worker->current_tasks, task_id, task) + { + struct vine_mount *input_mount; + LIST_ITERATE(task->input_mounts, input_mount) + { + if (f == input_mount->file) { + file_inuse = 1; + break; + } + } + if (file_inuse) { + break; + } + } + + if (file_inuse) { + continue; + } + + priority_queue_push(clean_replicas_from_workers, source_worker, source_worker->inuse_cache); + } + + source_worker = NULL; + while (excess_replicas > 0 && (source_worker = priority_queue_pop(clean_replicas_from_workers))) { + delete_worker_file(q, source_worker, f->cached_name, 0, 0); + excess_replicas--; + } + priority_queue_delete(clean_replicas_from_workers); + + return; +} + /* Remove all tasks and other associated state from a given worker. */ static void cleanup_worker(struct vine_manager *q, struct vine_worker_info *w) { From 3bcc9d0832749a16a62d9f76cf6a7b114553443d Mon Sep 17 00:00:00 2001 From: Jin Zhou Date: Mon, 3 Nov 2025 15:51:51 -0500 Subject: [PATCH 074/113] lint --- taskvine/src/manager/vine_manager.c | 10 +++++----- 1 file changed, 5 insertions(+), 5 deletions(-) diff --git a/taskvine/src/manager/vine_manager.c b/taskvine/src/manager/vine_manager.c index d67c822179..633b91fd8b 100644 --- a/taskvine/src/manager/vine_manager.c +++ b/taskvine/src/manager/vine_manager.c @@ -1012,10 +1012,10 @@ static int enforce_worker_eviction_interval(struct vine_manager *q) return 1; } -/** Clean redundant replicas of a temporary file. -For example, a file may be transferred to another worker because a task that declares it -as input is scheduled there, resulting in an extra replica that consumes storage space. -This function evaluates whether the file has excessive replicas and removes those on +/** Clean redundant replicas of a temporary file. +For example, a file may be transferred to another worker because a task that declares it +as input is scheduled there, resulting in an extra replica that consumes storage space. +This function evaluates whether the file has excessive replicas and removes those on workers that do not execute their dependent tasks. */ static void clean_redundant_replicas(struct vine_manager *q, struct vine_file *f) { @@ -1063,7 +1063,7 @@ static void clean_redundant_replicas(struct vine_manager *q, struct vine_file *f break; } } - + if (file_inuse) { continue; } From 203736a900e23eddb005855883fc6f972b984aef Mon Sep 17 00:00:00 2001 From: Jin Zhou Date: Tue, 4 Nov 2025 13:15:27 -0500 Subject: [PATCH 075/113] make it an argument --- taskvine/src/manager/vine_manager.c | 20 ++++++++++++++------ taskvine/src/manager/vine_manager.h | 1 + 2 files changed, 15 insertions(+), 6 deletions(-) diff --git a/taskvine/src/manager/vine_manager.c b/taskvine/src/manager/vine_manager.c index 633b91fd8b..4528039273 100644 --- a/taskvine/src/manager/vine_manager.c +++ b/taskvine/src/manager/vine_manager.c @@ -652,12 +652,16 @@ static vine_result_code_t get_completion_result(struct vine_manager *q, struct v itable_remove(q->running_table, t->task_id); vine_task_set_result(t, task_status); - /* Clean redundant replicas for the inputs of the task. */ - struct vine_mount *input_mount; - LIST_ITERATE(t->input_mounts, input_mount) - { - if (input_mount->file && input_mount->file->type == VINE_TEMP) { - clean_redundant_replicas(q, input_mount->file); + /* A task scheduling may result in a redundant replica of its input due to peer transfers, which can be safely removed when completed. + * However, the general function of taskvine is to replicate files on demand, and to only clean them up when prune is called. + * So, we only clean up redundant replicas for the task-inputs when the manager is configured to do so. */ + if (q->clean_redundant_replicas) { + struct vine_mount *input_mount; + LIST_ITERATE(t->input_mounts, input_mount) + { + if (input_mount->file && input_mount->file->type == VINE_TEMP) { + clean_redundant_replicas(q, input_mount->file); + } } } @@ -4299,6 +4303,7 @@ struct vine_manager *vine_ssl_create(int port, const char *key, const char *cert q->perf_log_interval = VINE_PERF_LOG_INTERVAL; q->temp_replica_count = 1; + q->clean_redundant_replicas = 0; q->transfer_temps_recovery = 0; q->transfer_replica_per_cycle = 10; @@ -6060,6 +6065,9 @@ int vine_tune(struct vine_manager *q, const char *name, double value) } else if (!strcmp(name, "enforce-worker-eviction-interval")) { q->enforce_worker_eviction_interval = (timestamp_t)(MAX(0, (int)value) * ONE_SECOND); + } else if (!strcmp(name, "clean-redundant-replicas")) { + q->clean_redundant_replicas = !!((int)value); + } 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 bcf2405616..f0241e3d9f 100644 --- a/taskvine/src/manager/vine_manager.h +++ b/taskvine/src/manager/vine_manager.h @@ -217,6 +217,7 @@ struct vine_manager { int transfer_temps_recovery; /* If true, attempt to recover temp files from lost worker to reach threshold required */ int transfer_replica_per_cycle; /* Maximum number of replica to request per temp file per iteration */ 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. */ double resource_submit_multiplier; /* Factor to permit overcommitment of resources at each worker. */ double bandwidth_limit; /* Artificial limit on bandwidth of manager<->worker transfers. */ From af1eb9dd2fe39bb1e618cf9d89d9e70fd5b1f1e7 Mon Sep 17 00:00:00 2001 From: Jin Zhou Date: Tue, 4 Nov 2025 13:17:41 -0500 Subject: [PATCH 076/113] shift-disk-load --- taskvine/src/manager/vine_manager.c | 80 ++++++++++++++++++++++++++--- taskvine/src/manager/vine_manager.h | 2 + 2 files changed, 76 insertions(+), 6 deletions(-) diff --git a/taskvine/src/manager/vine_manager.c b/taskvine/src/manager/vine_manager.c index 633b91fd8b..2874745781 100644 --- a/taskvine/src/manager/vine_manager.c +++ b/taskvine/src/manager/vine_manager.c @@ -170,6 +170,7 @@ static int release_worker(struct vine_manager *q, struct vine_worker_info *w); struct vine_task *send_library_to_worker(struct vine_manager *q, struct vine_worker_info *w, const char *name); +static void shift_disk_load(struct vine_manager *q, struct vine_worker_info *source_worker, struct vine_file *f); static void clean_redundant_replicas(struct vine_manager *q, struct vine_file *f); /* Return the number of workers matching a given type: WORKER, STATUS, etc */ @@ -424,6 +425,10 @@ static vine_msg_code_t handle_cache_update(struct vine_manager *q, struct vine_w if (f->type == VINE_TEMP && *id == 'X' && q->temp_replica_count > 1) { hash_table_insert(q->temp_files_to_replicate, f->cached_name, NULL); } + + if (q->shift_disk_load) { + shift_disk_load(q, w, f); + } } } @@ -652,12 +657,16 @@ static vine_result_code_t get_completion_result(struct vine_manager *q, struct v itable_remove(q->running_table, t->task_id); vine_task_set_result(t, task_status); - /* Clean redundant replicas for the inputs of the task. */ - struct vine_mount *input_mount; - LIST_ITERATE(t->input_mounts, input_mount) - { - if (input_mount->file && input_mount->file->type == VINE_TEMP) { - clean_redundant_replicas(q, input_mount->file); + /* A task scheduling may result in a redundant replica of its input due to peer transfers, which can be safely removed when completed. + * However, the general function of taskvine is to replicate files on demand, and to only clean them up when prune is called. + * So, we only clean up redundant replicas for the task-inputs when the manager is configured to clean redundant replicas. */ + if (q->clean_redundant_replicas) { + struct vine_mount *input_mount; + LIST_ITERATE(t->input_mounts, input_mount) + { + if (input_mount->file && input_mount->file->type == VINE_TEMP) { + clean_redundant_replicas(q, input_mount->file); + } } } @@ -1012,6 +1021,57 @@ static int enforce_worker_eviction_interval(struct vine_manager *q) return 1; } +static void shift_disk_load(struct vine_manager *q, struct vine_worker_info *source_worker, struct vine_file *f) +{ + if (!q || !source_worker || !f) { + return; + } + + if (f->type != VINE_TEMP) { + return; + } + + /* Determine if this replica is from the heaviest worker, and if so, trigger a replication immediately + * to shift the storage burden to the most free and eligible worker. */ + struct vine_worker_info *target_worker = NULL; + + char *key; + struct vine_worker_info *w = NULL; + HASH_TABLE_ITERATE(q->worker_table, key, w) + { + if (w->type != VINE_WORKER_TYPE_WORKER) { + continue; + } + if (!w->transfer_port_active) { + continue; + } + if (w->draining) { + continue; + } + if (w->resources->tag < 0) { + continue; + } + if (vine_file_replica_table_lookup(w, f->cached_name)) { + continue; + } + if (w->inuse_cache + f->size > (source_worker->inuse_cache - f->size) * 0.8) { + continue; + } + if (!target_worker || w->inuse_cache < target_worker->inuse_cache) { + target_worker = w; + } + } + if (target_worker) { + char *source_addr = string_format("%s/%s", source_worker->transfer_url, f->cached_name); + vine_manager_put_url_now(q, target_worker, source_worker, source_addr, f); + free(source_addr); + } + + /* Shifting storage burden from heavy to light workers requires to replicate the file first, + * so we can clean up the original one safely when the replica arrives at the destination worker. */ + clean_redundant_replicas(q, f); +} + /** Clean redundant replicas of a temporary file. For example, a file may be transferred to another worker because a task that declares it as input is scheduled there, resulting in an extra replica that consumes storage space. @@ -4299,6 +4359,8 @@ struct vine_manager *vine_ssl_create(int port, const char *key, const char *cert q->perf_log_interval = VINE_PERF_LOG_INTERVAL; q->temp_replica_count = 1; + q->clean_redundant_replicas = 0; + q->shift_disk_load = 0; q->transfer_temps_recovery = 0; q->transfer_replica_per_cycle = 10; @@ -6060,6 +6122,12 @@ int vine_tune(struct vine_manager *q, const char *name, double value) } else if (!strcmp(name, "enforce-worker-eviction-interval")) { q->enforce_worker_eviction_interval = (timestamp_t)(MAX(0, (int)value) * ONE_SECOND); + } else if (!strcmp(name, "shift-disk-load")) { + q->shift_disk_load = !!((int)value); + + } else if (!strcmp(name, "clean-redundant-replicas")) { + q->clean_redundant_replicas = !!((int)value); + } 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 bcf2405616..24989ce0e3 100644 --- a/taskvine/src/manager/vine_manager.h +++ b/taskvine/src/manager/vine_manager.h @@ -217,6 +217,8 @@ struct vine_manager { int transfer_temps_recovery; /* If true, attempt to recover temp files from lost worker to reach threshold required */ int transfer_replica_per_cycle; /* Maximum number of replica to request per temp file per iteration */ 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. */ double resource_submit_multiplier; /* Factor to permit overcommitment of resources at each worker. */ double bandwidth_limit; /* Artificial limit on bandwidth of manager<->worker transfers. */ From e3e51b53c13d9cdc8e72393bc4941f3d2b8d7f26 Mon Sep 17 00:00:00 2001 From: Jin Zhou Date: Tue, 4 Nov 2025 13:25:32 -0500 Subject: [PATCH 077/113] lint --- taskvine/src/manager/vine_manager.c | 8 ++++---- 1 file changed, 4 insertions(+), 4 deletions(-) diff --git a/taskvine/src/manager/vine_manager.c b/taskvine/src/manager/vine_manager.c index 8198229812..2a1a650184 100644 --- a/taskvine/src/manager/vine_manager.c +++ b/taskvine/src/manager/vine_manager.c @@ -1031,8 +1031,8 @@ static void shift_disk_load(struct vine_manager *q, struct vine_worker_info *sou return; } - /* Determine if this replica is from the heaviest worker, and if so, trigger a replication immediately - * to shift the storage burden to the most free and eligible worker. */ + /* Determine if this replica is from the heaviest worker, and if so, trigger a replication immediately + * to shift the storage burden to the most free and eligible worker. */ struct vine_worker_info *target_worker = NULL; char *key; @@ -1067,8 +1067,8 @@ static void shift_disk_load(struct vine_manager *q, struct vine_worker_info *sou free(source_addr); } - /* Shifting storage burden from heavy to light workers requires to replicate the file first, - * so we can clean up the original one safely when the replica arrives at the destination worker. */ + /* Shifting storage burden from heavy to light workers requires to replicate the file first, + * so we can clean up the original one safely when the replica arrives at the destination worker. */ clean_redundant_replicas(q, f); } From f6e2b0ebee657f578c06638179723c92e34059ec Mon Sep 17 00:00:00 2001 From: Jin Zhou Date: Tue, 4 Nov 2025 13:36:31 -0500 Subject: [PATCH 078/113] comment --- taskvine/src/manager/vine_manager.c | 10 ++++++---- 1 file changed, 6 insertions(+), 4 deletions(-) diff --git a/taskvine/src/manager/vine_manager.c b/taskvine/src/manager/vine_manager.c index 2a1a650184..5e79d36090 100644 --- a/taskvine/src/manager/vine_manager.c +++ b/taskvine/src/manager/vine_manager.c @@ -1021,6 +1021,11 @@ static int enforce_worker_eviction_interval(struct vine_manager *q) return 1; } +/** +Shift a temp file replica away from the worker using the most cache space. +This function looks for an alternative worker that can accept the file immediately +so that the original replica can be cleaned up later by @clean_redundant_replicas(). +*/ static void shift_disk_load(struct vine_manager *q, struct vine_worker_info *source_worker, struct vine_file *f) { if (!q || !source_worker || !f) { @@ -1031,8 +1036,6 @@ static void shift_disk_load(struct vine_manager *q, struct vine_worker_info *sou return; } - /* Determine if this replica is from the heaviest worker, and if so, trigger a replication immediately - * to shift the storage burden to the most free and eligible worker. */ struct vine_worker_info *target_worker = NULL; char *key; @@ -1067,8 +1070,7 @@ static void shift_disk_load(struct vine_manager *q, struct vine_worker_info *sou free(source_addr); } - /* Shifting storage burden from heavy to light workers requires to replicate the file first, - * so we can clean up the original one safely when the replica arrives at the destination worker. */ + /* We can clean up the original one safely when the replica arrives at the destination worker. */ clean_redundant_replicas(q, f); } From caff977042b1dc83cd89715713ad070b5717b085 Mon Sep 17 00:00:00 2001 From: Jin Zhou Date: Tue, 4 Nov 2025 13:45:17 -0500 Subject: [PATCH 079/113] vine_temp --- taskvine/src/manager/Makefile | 3 +- taskvine/src/manager/vine_manager.c | 137 +----------- taskvine/src/manager/vine_temp.c | 335 ++++++++++++++++++++++++++++ taskvine/src/manager/vine_temp.h | 13 ++ 4 files changed, 359 insertions(+), 129 deletions(-) create mode 100644 taskvine/src/manager/vine_temp.c create mode 100644 taskvine/src/manager/vine_temp.h diff --git a/taskvine/src/manager/Makefile b/taskvine/src/manager/Makefile index a036e9bd64..db09403667 100644 --- a/taskvine/src/manager/Makefile +++ b/taskvine/src/manager/Makefile @@ -28,7 +28,8 @@ SOURCES = \ vine_file_replica_table.c \ vine_fair.c \ vine_runtime_dir.c \ - vine_task_groups.c + vine_task_groups.c \ + vine_temp.c PUBLIC_HEADERS = taskvine.h diff --git a/taskvine/src/manager/vine_manager.c b/taskvine/src/manager/vine_manager.c index 5e79d36090..5593e7aeda 100644 --- a/taskvine/src/manager/vine_manager.c +++ b/taskvine/src/manager/vine_manager.c @@ -29,6 +29,7 @@ See the file COPYING for details. #include "vine_taskgraph_log.h" #include "vine_txn_log.h" #include "vine_worker_info.h" +#include "vine_temp.h" #include "address.h" #include "buffer.h" @@ -170,9 +171,6 @@ static int release_worker(struct vine_manager *q, struct vine_worker_info *w); struct vine_task *send_library_to_worker(struct vine_manager *q, struct vine_worker_info *w, const char *name); -static void shift_disk_load(struct vine_manager *q, struct vine_worker_info *source_worker, struct vine_file *f); -static void clean_redundant_replicas(struct vine_manager *q, struct vine_file *f); - /* Return the number of workers matching a given type: WORKER, STATUS, etc */ static int count_workers(struct vine_manager *q, vine_worker_type_t type) @@ -427,7 +425,7 @@ static vine_msg_code_t handle_cache_update(struct vine_manager *q, struct vine_w } if (q->shift_disk_load) { - shift_disk_load(q, w, f); + vine_temp_shift_disk_load(q, w, f); } } } @@ -665,7 +663,7 @@ static vine_result_code_t get_completion_result(struct vine_manager *q, struct v LIST_ITERATE(t->input_mounts, input_mount) { if (input_mount->file && input_mount->file->type == VINE_TEMP) { - clean_redundant_replicas(q, input_mount->file); + vine_temp_clean_redundant_replicas(q, input_mount->file); } } } @@ -1021,128 +1019,6 @@ static int enforce_worker_eviction_interval(struct vine_manager *q) return 1; } -/** -Shift a temp file replica away from the worker using the most cache space. -This function looks for an alternative worker that can accept the file immediately -so that the original replica can be cleaned up later by @clean_redundant_replicas(). -*/ -static void shift_disk_load(struct vine_manager *q, struct vine_worker_info *source_worker, struct vine_file *f) -{ - if (!q || !source_worker || !f) { - return; - } - - if (f->type != VINE_TEMP) { - return; - } - - struct vine_worker_info *target_worker = NULL; - - char *key; - struct vine_worker_info *w = NULL; - HASH_TABLE_ITERATE(q->worker_table, key, w) - { - if (w->type != VINE_WORKER_TYPE_WORKER) { - continue; - } - if (!w->transfer_port_active) { - continue; - } - if (w->draining) { - continue; - } - if (w->resources->tag < 0) { - continue; - } - if (vine_file_replica_table_lookup(w, f->cached_name)) { - continue; - } - if (w->inuse_cache + f->size > (source_worker->inuse_cache - f->size) * 0.8) { - continue; - } - if (!target_worker || w->inuse_cache < target_worker->inuse_cache) { - target_worker = w; - } - } - if (target_worker) { - char *source_addr = string_format("%s/%s", source_worker->transfer_url, f->cached_name); - vine_manager_put_url_now(q, target_worker, source_worker, source_addr, f); - free(source_addr); - } - - /* We can clean up the original one safely when the replica arrives at the destination worker. */ - clean_redundant_replicas(q, f); -} - -/** Clean redundant replicas of a temporary file. -For example, a file may be transferred to another worker because a task that declares it -as input is scheduled there, resulting in an extra replica that consumes storage space. -This function evaluates whether the file has excessive replicas and removes those on -workers that do not execute their dependent tasks. */ -static void clean_redundant_replicas(struct vine_manager *q, struct vine_file *f) -{ - if (!f || f->type != VINE_TEMP) { - return; - } - - struct set *source_workers = hash_table_lookup(q->file_worker_table, f->cached_name); - if (!source_workers) { - /* no surprise - a cache-update message may trigger a file deletion. */ - return; - } - int excess_replicas = set_size(source_workers) - q->temp_replica_count; - if (excess_replicas <= 0) { - return; - } - /* Note that this replica may serve as a source for a peer transfer. If it is unlinked prematurely, - * the corresponding transfer could fail and leave a task without its required data. - * Therefore, we must wait until all replicas are confirmed ready before proceeding. */ - if (vine_file_replica_table_count_replicas(q, f->cached_name, VINE_FILE_REPLICA_STATE_READY) != set_size(source_workers)) { - return; - } - - struct priority_queue *clean_replicas_from_workers = priority_queue_create(0); - - struct vine_worker_info *source_worker = NULL; - SET_ITERATE(source_workers, source_worker) - { - /* if the file is actively in use by a task (the input to that task), we don't remove the replica on this worker */ - int file_inuse = 0; - - uint64_t task_id; - struct vine_task *task; - ITABLE_ITERATE(source_worker->current_tasks, task_id, task) - { - struct vine_mount *input_mount; - LIST_ITERATE(task->input_mounts, input_mount) - { - if (f == input_mount->file) { - file_inuse = 1; - break; - } - } - if (file_inuse) { - break; - } - } - - if (file_inuse) { - continue; - } - - priority_queue_push(clean_replicas_from_workers, source_worker, source_worker->inuse_cache); - } - - source_worker = NULL; - while (excess_replicas > 0 && (source_worker = priority_queue_pop(clean_replicas_from_workers))) { - delete_worker_file(q, source_worker, f->cached_name, 0, 0); - excess_replicas--; - } - priority_queue_delete(clean_replicas_from_workers); - - return; -} - /* Remove all tasks and other associated state from a given worker. */ static void cleanup_worker(struct vine_manager *q, struct vine_worker_info *w) { @@ -5983,7 +5859,6 @@ void vine_set_manager_preferred_connection(struct vine_manager *q, const char *p int vine_tune(struct vine_manager *q, const char *name, double value) { - if (!strcmp(name, "attempt-schedule-depth")) { q->attempt_schedule_depth = MAX(1, (int)value); @@ -6122,6 +5997,12 @@ int vine_tune(struct vine_manager *q, const char *name, double value) } else if (!strcmp(name, "enforce-worker-eviction-interval")) { q->enforce_worker_eviction_interval = (timestamp_t)(MAX(0, (int)value) * ONE_SECOND); + } else if (!strcmp(name, "clean-redundant-replicas")) { + q->clean_redundant_replicas = !!((int)value); + + } else if (!strcmp(name, "shift-disk-load")) { + q->shift_disk_load = !!((int)value); + } else { debug(D_NOTICE | D_VINE, "Warning: tuning parameter \"%s\" not recognized\n", name); return -1; diff --git a/taskvine/src/manager/vine_temp.c b/taskvine/src/manager/vine_temp.c new file mode 100644 index 0000000000..7b574653ae --- /dev/null +++ b/taskvine/src/manager/vine_temp.c @@ -0,0 +1,335 @@ +#include "vine_temp.h" +#include "priority_queue.h" +#include "vine_file.h" +#include "vine_worker_info.h" +#include "vine_file_replica_table.h" +#include "macros.h" +#include "stringtools.h" +#include "vine_manager.h" +#include "debug.h" +#include "random.h" +#include "vine_manager_put.h" +#include "xxmalloc.h" + +/*************************************************************/ +/* Private Functions */ +/*************************************************************/ + +static struct vine_worker_info *get_best_source_worker(struct vine_manager *q, struct vine_file *f) +{ + if (!q || !f || f->type != VINE_TEMP) { + return NULL; + } + + struct set *sources = hash_table_lookup(q->file_worker_table, f->cached_name); + if (!sources) { + return NULL; + } + + struct priority_queue *valid_sources_queue = priority_queue_create(0); + struct vine_worker_info *w = NULL; + SET_ITERATE(sources, w) + { + /* skip if transfer port is not active or in draining mode */ + if (!w->transfer_port_active || w->draining) { + continue; + } + /* skip if incoming transfer counter is too high */ + if (w->outgoing_xfer_counter >= q->worker_source_max_transfers) { + continue; + } + /* skip if the worker does not have this file */ + struct vine_file_replica *replica = vine_file_replica_table_lookup(w, f->cached_name); + if (!replica) { + continue; + } + /* skip if the file is not ready */ + if (replica->state != VINE_FILE_REPLICA_STATE_READY) { + continue; + } + /* those with less outgoing_xfer_counter are preferred */ + priority_queue_push(valid_sources_queue, w, -w->outgoing_xfer_counter); + } + + struct vine_worker_info *best_source = priority_queue_pop(valid_sources_queue); + priority_queue_delete(valid_sources_queue); + + return best_source; +} + +static struct vine_worker_info *get_best_dest_worker(struct vine_manager *q, struct vine_file *f) +{ + if (!q || !f || f->type != VINE_TEMP) { + return NULL; + } + + struct priority_queue *valid_destinations = priority_queue_create(0); + + char *key; + struct vine_worker_info *w; + HASH_TABLE_ITERATE(q->worker_table, key, w) + { + /* skip if transfer port is not active or in draining mode */ + if (!w->transfer_port_active || w->draining) { + continue; + } + /* skip if the incoming transfer counter is too high */ + if (w->incoming_xfer_counter >= q->worker_source_max_transfers) { + continue; + } + /* skip if the worker already has this file */ + struct vine_file_replica *replica = vine_file_replica_table_lookup(w, f->cached_name); + if (replica) { + continue; + } + /* skip if the worker does not have enough disk space */ + int64_t available_disk_space = get_worker_available_disk_bytes(w); + if ((int64_t)f->size > available_disk_space) { + continue; + } + /* workers with more available disk space are preferred to hold the file */ + priority_queue_push(valid_destinations, w, available_disk_space); + } + + struct vine_worker_info *best_destination = priority_queue_pop(valid_destinations); + priority_queue_delete(valid_destinations); + + return best_destination; +} + +static int vine_temp_replicate_file_now(struct vine_manager *q, struct vine_file *f) +{ + if (!q || !f || f->type != VINE_TEMP) { + return 0; + } + + struct vine_worker_info *source_worker = get_best_source_worker(q, f); + if (!source_worker) { + return 0; + } + + struct vine_worker_info *dest_worker = get_best_dest_worker(q, f); + if (!dest_worker) { + return 0; + } + + vine_temp_start_peer_transfer(q, f, source_worker, dest_worker); + + return 1; +} + +/*************************************************************/ +/* Public Functions */ +/*************************************************************/ + +void vine_temp_start_peer_transfer(struct vine_manager *q, struct vine_file *f, struct vine_worker_info *source_worker, struct vine_worker_info *dest_worker) +{ + if (!q || !f || f->type != VINE_TEMP || !source_worker || !dest_worker) { + return; + } + + char *source_addr = string_format("%s/%s", source_worker->transfer_url, f->cached_name); + vine_manager_put_url_now(q, dest_worker, source_worker, source_addr, f); + free(source_addr); +} + +int vine_temp_replicate_file_later(struct vine_manager *q, struct vine_file *f) +{ + if (!q || !f || f->type != VINE_TEMP || f->state != VINE_FILE_STATE_CREATED) { + return 0; + } + + int current_replica_count = vine_file_replica_count(q, f); + if (current_replica_count == 0 || current_replica_count >= q->temp_replica_count) { + return 0; + } + + priority_queue_push(q->temp_files_to_replicate, f, -current_replica_count); + + return 1; +} + +int vine_temp_handle_file_lost(struct vine_manager *q, char *cachename) +{ + if (!q || !cachename) { + return 0; + } + + struct vine_file *f = hash_table_lookup(q->file_table, cachename); + if (!f || f->type != VINE_TEMP || f->state != VINE_FILE_STATE_CREATED) { + return 0; + } + + vine_temp_replicate_file_later(q, f); + + return 1; +} + +int vine_temp_start_replication(struct vine_manager *q) +{ + if (!q) { + return 0; + } + + int processed = 0; + int iter_count = 0; + int iter_depth = MIN(q->attempt_schedule_depth, priority_queue_size(q->temp_files_to_replicate)); + struct list *skipped = list_create(); + + struct vine_file *f; + while ((f = priority_queue_pop(q->temp_files_to_replicate)) && (iter_count++ < iter_depth)) { + if (!f || f->type != VINE_TEMP || f->state != VINE_FILE_STATE_CREATED) { + continue; + } + + /* skip if the file has enough replicas or no replicas */ + int current_replica_count = vine_file_replica_count(q, f); + if (current_replica_count >= q->temp_replica_count || current_replica_count == 0) { + continue; + } + /* skip if the file has no ready replicas */ + int current_ready_replica_count = vine_file_replica_table_count_replicas(q, f->cached_name, VINE_FILE_REPLICA_STATE_READY); + if (current_ready_replica_count == 0) { + continue; + } + + /* if reach here, it means the file needs to be replicated and there is at least one ready replica. */ + if (!vine_temp_replicate_file_now(q, f)) { + list_push_tail(skipped, f); + continue; + } + + processed++; + + /* push back and keep evaluating the same file with a lower priority, until no more source + * or destination workers are available, or the file has enough replicas. */ + vine_temp_replicate_file_later(q, f); + } + + while ((f = list_pop_head(skipped))) { + vine_temp_replicate_file_later(q, f); + } + list_delete(skipped); + + return processed; +} + +/** +Clean redundant replicas of a temporary file. +For example, a file may be transferred to another worker because a task that declares it +as input is scheduled there, resulting in an extra replica that consumes storage space. +This function evaluates whether the file has excessive replicas and removes those on +workers that do not execute their dependent tasks. +*/ +void vine_temp_clean_redundant_replicas(struct vine_manager *q, struct vine_file *f) +{ + if (!f || f->type != VINE_TEMP) { + return; + } + + struct set *source_workers = hash_table_lookup(q->file_worker_table, f->cached_name); + if (!source_workers) { + /* no surprise - a cache-update message may trigger a file deletion. */ + return; + } + int excess_replicas = set_size(source_workers) - q->temp_replica_count; + if (excess_replicas <= 0) { + return; + } + /* Note that this replica may serve as a source for a peer transfer. If it is unlinked prematurely, + * the corresponding transfer could fail and leave a task without its required data. + * Therefore, we must wait until all replicas are confirmed ready before proceeding. */ + if (vine_file_replica_table_count_replicas(q, f->cached_name, VINE_FILE_REPLICA_STATE_READY) != set_size(source_workers)) { + return; + } + + struct priority_queue *clean_replicas_from_workers = priority_queue_create(0); + + struct vine_worker_info *source_worker = NULL; + SET_ITERATE(source_workers, source_worker) + { + /* if the file is actively in use by a task (the input to that task), we don't remove the replica on this worker */ + int file_inuse = 0; + + uint64_t task_id; + struct vine_task *task; + ITABLE_ITERATE(source_worker->current_tasks, task_id, task) + { + struct vine_mount *input_mount; + LIST_ITERATE(task->input_mounts, input_mount) + { + if (f == input_mount->file) { + file_inuse = 1; + break; + } + } + if (file_inuse) { + break; + } + } + + if (file_inuse) { + continue; + } + + priority_queue_push(clean_replicas_from_workers, source_worker, source_worker->inuse_cache); + } + + source_worker = NULL; + while (excess_replicas > 0 && (source_worker = priority_queue_pop(clean_replicas_from_workers))) { + delete_worker_file(q, source_worker, f->cached_name, 0, 0); + excess_replicas--; + } + priority_queue_delete(clean_replicas_from_workers); + + return; +} + +/** +Shift a temp file replica away from the worker using the most cache space. +This function looks for an alternative worker that can accept the file immediately +so that the original replica can be cleaned up later by @vine_temp_clean_redundant_replicas(). +*/ +void vine_temp_shift_disk_load(struct vine_manager *q, struct vine_worker_info *source_worker, struct vine_file *f) +{ + if (!q || !source_worker || !f || f->type != VINE_TEMP) { + return; + } + + struct vine_worker_info *target_worker = NULL; + + char *key; + struct vine_worker_info *w = NULL; + HASH_TABLE_ITERATE(q->worker_table, key, w) + { + if (w->type != VINE_WORKER_TYPE_WORKER) { + continue; + } + if (!w->transfer_port_active) { + continue; + } + if (w->draining) { + continue; + } + if (w->resources->tag < 0) { + continue; + } + if (vine_file_replica_table_lookup(w, f->cached_name)) { + continue; + } + if (w->inuse_cache + f->size > (source_worker->inuse_cache - f->size) * 0.8) { + continue; + } + if (!target_worker || w->inuse_cache < target_worker->inuse_cache) { + target_worker = w; + } + } + if (target_worker) { + char *source_addr = string_format("%s/%s", source_worker->transfer_url, f->cached_name); + vine_manager_put_url_now(q, target_worker, source_worker, source_addr, f); + free(source_addr); + } + + /* We can clean up the original one safely when the replica arrives at the destination worker. */ + vine_temp_clean_redundant_replicas(q, f); +} \ No newline at end of file diff --git a/taskvine/src/manager/vine_temp.h b/taskvine/src/manager/vine_temp.h new file mode 100644 index 0000000000..d6f2d72523 --- /dev/null +++ b/taskvine/src/manager/vine_temp.h @@ -0,0 +1,13 @@ +#ifndef vine_temp_H +#define vine_temp_H + +#include "vine_manager.h" + +int vine_temp_replicate_file_later(struct vine_manager *q, struct vine_file *f); +int vine_temp_handle_file_lost(struct vine_manager *q, char *cachename); +int vine_temp_start_replication(struct vine_manager *q); +void vine_temp_start_peer_transfer(struct vine_manager *q, struct vine_file *f, struct vine_worker_info *source_worker, struct vine_worker_info *dest_worker); + +void vine_temp_shift_disk_load(struct vine_manager *q, struct vine_worker_info *source_worker, struct vine_file *f); + +#endif \ No newline at end of file From cdd96a8dd6f0c3786a9ca2aa5276f6de1faee38c Mon Sep 17 00:00:00 2001 From: Jin Zhou Date: Tue, 4 Nov 2025 17:31:01 -0500 Subject: [PATCH 080/113] vine: vine_temp.c --- taskvine/src/manager/vine_manager.c | 114 +++++----------------------- taskvine/src/manager/vine_manager.h | 5 +- taskvine/src/manager/vine_temp.c | 90 +++++++++++++--------- taskvine/src/manager/vine_temp.h | 4 +- 4 files changed, 79 insertions(+), 134 deletions(-) diff --git a/taskvine/src/manager/vine_manager.c b/taskvine/src/manager/vine_manager.c index 5593e7aeda..dec3d533c6 100644 --- a/taskvine/src/manager/vine_manager.c +++ b/taskvine/src/manager/vine_manager.c @@ -166,7 +166,6 @@ static int vine_manager_check_inputs_available(struct vine_manager *q, struct vi static void vine_manager_consider_recovery_task(struct vine_manager *q, struct vine_file *lost_file, struct vine_task *rt); static void delete_uncacheable_files(struct vine_manager *q, struct vine_worker_info *w, struct vine_task *t); -static int delete_worker_file(struct vine_manager *q, struct vine_worker_info *w, const char *filename, vine_cache_level_t cache_level, vine_cache_level_t delete_upto_level); static int release_worker(struct vine_manager *q, struct vine_worker_info *w); struct vine_task *send_library_to_worker(struct vine_manager *q, struct vine_worker_info *w, const char *name); @@ -419,12 +418,15 @@ static vine_msg_code_t handle_cache_update(struct vine_manager *q, struct vine_w f->state = VINE_FILE_STATE_CREATED; f->size = size; - /* And if the file is a newly created temporary, replicate as needed. */ - if (f->type == VINE_TEMP && *id == 'X' && q->temp_replica_count > 1) { - hash_table_insert(q->temp_files_to_replicate, f->cached_name, NULL); + /* If the replica's type was a URL, it means the manager expected the destination worker to download it + * from elsewhere. Now that it's physically present, we can resolve its type back to the original */ + if (replica->type == VINE_URL) { + replica->type = f->type; } - if (q->shift_disk_load) { + /* If a TEMP file, replicate or shift disk load as needed. */ + if (f->type == VINE_TEMP) { + vine_temp_replicate_file_later(q, f); vine_temp_shift_disk_load(q, w, f); } } @@ -481,6 +483,9 @@ static vine_msg_code_t handle_cache_invalid(struct vine_manager *q, struct vine_ w->last_failure_time = timestamp_get(); } + /* If the creation failed, we may want to backup the file somewhere else. */ + vine_temp_rescue_lost_replica(q, cachename); + /* Successfully processed this message. */ return VINE_MSG_PROCESSED; } else { @@ -1054,85 +1059,6 @@ static void cleanup_worker(struct vine_manager *q, struct vine_worker_info *w) cleanup_worker_files(q, w); } -/* Start replicating files that may need replication */ -static int consider_tempfile_replications(struct vine_manager *q) -{ - if (hash_table_size(q->temp_files_to_replicate) <= 0) { - return 0; - } - - char *cached_name = NULL; - void *empty_val = NULL; - int total_replication_request_sent = 0; - - static char key_start[PATH_MAX] = "random init"; - int iter_control; - int iter_count_var; - - struct list *to_remove = list_create(); - - HASH_TABLE_ITERATE_FROM_KEY(q->temp_files_to_replicate, iter_control, iter_count_var, key_start, cached_name, empty_val) - { - struct vine_file *f = hash_table_lookup(q->file_table, cached_name); - - if (!f) { - continue; - } - - /* are there any available source workers? */ - struct set *source_workers = hash_table_lookup(q->file_worker_table, f->cached_name); - if (!source_workers) { - /* If no source workers found, it indicates that the file doesn't exist, either pruned or lost. - Because a pruned file is removed from the recovery queue, so it definitely indicates that the file is lost. */ - if (q->transfer_temps_recovery && file_needs_recovery(q, f)) { - vine_manager_consider_recovery_task(q, f, f->recovery_task); - } - list_push_tail(to_remove, xxstrdup(f->cached_name)); - continue; - } - - /* at least one source is able to transfer? */ - int has_valid_source = 0; - struct vine_worker_info *s; - SET_ITERATE(source_workers, s) - { - if (s->transfer_port_active && s->outgoing_xfer_counter < q->worker_source_max_transfers && !s->draining) { - has_valid_source = 1; - break; - } - } - if (!has_valid_source) { - continue; - } - - /* has this file been fully replicated? */ - int nsource_workers = set_size(source_workers); - int to_find = MIN(q->temp_replica_count - nsource_workers, q->transfer_replica_per_cycle); - if (to_find <= 0) { - list_push_tail(to_remove, xxstrdup(f->cached_name)); - continue; - } - - // debug(D_VINE, "Found %d workers holding %s, %d replicas needed", nsource_workers, f->cached_name, to_find); - - int round_replication_request_sent = vine_file_replica_table_replicate(q, f, source_workers, to_find); - total_replication_request_sent += round_replication_request_sent; - - if (total_replication_request_sent >= q->attempt_schedule_depth) { - break; - } - } - - while ((cached_name = list_pop_head(to_remove))) { - hash_table_remove(q->temp_files_to_replicate, cached_name); - free(cached_name); - } - - list_delete(to_remove); - - return total_replication_request_sent; -} - /* Insert into hashtable temp files that may need replication. */ static void recall_worker_lost_temp_files(struct vine_manager *q, struct vine_worker_info *w) @@ -1145,11 +1071,7 @@ static void recall_worker_lost_temp_files(struct vine_manager *q, struct vine_wo // Iterate over files we want might want to recover HASH_TABLE_ITERATE(w->current_files, cached_name, info) { - struct vine_file *f = hash_table_lookup(q->file_table, cached_name); - - if (f && f->type == VINE_TEMP) { - hash_table_insert(q->temp_files_to_replicate, cached_name, NULL); - } + vine_temp_rescue_lost_replica(q, cached_name); } } @@ -1261,7 +1183,7 @@ static void add_worker(struct vine_manager *q) /* Delete a single file on a remote worker except those with greater delete_upto_level cache level */ -static int delete_worker_file(struct vine_manager *q, struct vine_worker_info *w, const char *filename, vine_cache_level_t cache_level, vine_cache_level_t delete_upto_level) +int delete_worker_file(struct vine_manager *q, struct vine_worker_info *w, const char *filename, vine_cache_level_t cache_level, vine_cache_level_t delete_upto_level) { if (cache_level <= delete_upto_level) { process_replica_on_event(q, w, filename, VINE_FILE_REPLICA_STATE_TRANSITION_EVENT_UNLINK); @@ -4153,7 +4075,7 @@ struct vine_manager *vine_ssl_create(int port, const char *key, const char *cert q->worker_table = hash_table_create(0, 0); q->file_worker_table = hash_table_create(0, 0); - q->temp_files_to_replicate = hash_table_create(0, 0); + q->temp_files_to_replicate = priority_queue_create(0); q->worker_blocklist = hash_table_create(0, 0); q->file_table = hash_table_create(0, 0); @@ -4237,6 +4159,8 @@ struct vine_manager *vine_ssl_create(int port, const char *key, const char *cert q->perf_log_interval = VINE_PERF_LOG_INTERVAL; q->temp_replica_count = 1; + q->clean_redundant_replicas = 0; + q->shift_disk_load = 0; q->transfer_temps_recovery = 0; q->transfer_replica_per_cycle = 10; @@ -4505,8 +4429,7 @@ void vine_delete(struct vine_manager *q) hash_table_clear(q->file_worker_table, (void *)set_delete); hash_table_delete(q->file_worker_table); - hash_table_clear(q->temp_files_to_replicate, 0); - hash_table_delete(q->temp_files_to_replicate); + priority_queue_delete(q->temp_files_to_replicate); hash_table_clear(q->factory_table, (void *)vine_factory_info_delete); hash_table_delete(q->factory_table); @@ -5466,7 +5389,7 @@ static struct vine_task *vine_wait_internal(struct vine_manager *q, int timeout, // Check if any temp files need replication and start replicating BEGIN_ACCUM_TIME(q, time_internal); - result = consider_tempfile_replications(q); + result = vine_temp_start_replication(q); END_ACCUM_TIME(q, time_internal); if (result) { // recovered at least one temp file @@ -6483,9 +6406,6 @@ void vine_prune_file(struct vine_manager *m, struct vine_file *f) set_free_values_array(workers_array); } } - - /* also remove from the replication table. */ - hash_table_remove(m->temp_files_to_replicate, f->cached_name); } /* diff --git a/taskvine/src/manager/vine_manager.h b/taskvine/src/manager/vine_manager.h index 24989ce0e3..5b96dacb98 100644 --- a/taskvine/src/manager/vine_manager.h +++ b/taskvine/src/manager/vine_manager.h @@ -123,7 +123,7 @@ struct vine_manager { struct hash_table *file_table; /* Maps fileid -> struct vine_file.* */ struct hash_table *file_worker_table; /* Maps cachename -> struct set of workers with a replica of the file.* */ - struct hash_table *temp_files_to_replicate; /* Maps cachename -> NULL. Used as a set of temp files to be replicated */ + struct priority_queue *temp_files_to_replicate; /* Priority queue of temp files to be replicated, those with less replicas are at the top. */ /* Primary scheduling controls. */ @@ -293,6 +293,9 @@ void vine_manager_remove_worker(struct vine_manager *q, struct vine_worker_info /* Check if the worker is able to transfer the necessary files for this task. */ int vine_manager_transfer_capacity_available(struct vine_manager *q, struct vine_worker_info *w, struct vine_task *t); +/* Delete a file from a worker. */ +int delete_worker_file(struct vine_manager *q, struct vine_worker_info *w, const char *filename, vine_cache_level_t cache_level, vine_cache_level_t delete_upto_level); + /* The expected format of files created by the resource monitor.*/ #define RESOURCE_MONITOR_TASK_LOCAL_NAME "vine-task-%d" #define RESOURCE_MONITOR_REMOTE_NAME "cctools-monitor" diff --git a/taskvine/src/manager/vine_temp.c b/taskvine/src/manager/vine_temp.c index 7b574653ae..b7f1ef1c90 100644 --- a/taskvine/src/manager/vine_temp.c +++ b/taskvine/src/manager/vine_temp.c @@ -1,20 +1,45 @@ #include "vine_temp.h" -#include "priority_queue.h" #include "vine_file.h" #include "vine_worker_info.h" #include "vine_file_replica_table.h" +#include "vine_manager.h" +#include "vine_manager_put.h" +#include "vine_file_replica.h" +#include "vine_file_replica_table.h" +#include "vine_task.h" +#include "vine_mount.h" + +#include "priority_queue.h" #include "macros.h" #include "stringtools.h" -#include "vine_manager.h" #include "debug.h" #include "random.h" -#include "vine_manager_put.h" #include "xxmalloc.h" /*************************************************************/ /* Private Functions */ /*************************************************************/ +static int is_worker_active(struct vine_worker_info *w) +{ + if (!w) { + return 0; + } + if (w->type != VINE_WORKER_TYPE_WORKER) { + return 0; + } + if (!w->transfer_port_active) { + return 0; + } + if (w->draining) { + return 0; + } + if (w->resources->tag < 0) { + return 0; + } + return 1; +} + static struct vine_worker_info *get_best_source_worker(struct vine_manager *q, struct vine_file *f) { if (!q || !f || f->type != VINE_TEMP) { @@ -31,7 +56,7 @@ static struct vine_worker_info *get_best_source_worker(struct vine_manager *q, s SET_ITERATE(sources, w) { /* skip if transfer port is not active or in draining mode */ - if (!w->transfer_port_active || w->draining) { + if (!is_worker_active(w)) { continue; } /* skip if incoming transfer counter is too high */ @@ -70,7 +95,7 @@ static struct vine_worker_info *get_best_dest_worker(struct vine_manager *q, str HASH_TABLE_ITERATE(q->worker_table, key, w) { /* skip if transfer port is not active or in draining mode */ - if (!w->transfer_port_active || w->draining) { + if (!is_worker_active(w)) { continue; } /* skip if the incoming transfer counter is too high */ @@ -83,7 +108,7 @@ static struct vine_worker_info *get_best_dest_worker(struct vine_manager *q, str continue; } /* skip if the worker does not have enough disk space */ - int64_t available_disk_space = get_worker_available_disk_bytes(w); + int64_t available_disk_space = (int64_t)MEGABYTES_TO_BYTES(w->resources->disk.total) - w->inuse_cache; if ((int64_t)f->size > available_disk_space) { continue; } @@ -97,6 +122,17 @@ static struct vine_worker_info *get_best_dest_worker(struct vine_manager *q, str return best_destination; } +static void start_peer_transfer(struct vine_manager *q, struct vine_file *f, struct vine_worker_info *dest_worker, struct vine_worker_info *source_worker) +{ + if (!q || !f || f->type != VINE_TEMP || !dest_worker || !source_worker) { + return; + } + + char *source_addr = string_format("%s/%s", source_worker->transfer_url, f->cached_name); + vine_manager_put_url_now(q, dest_worker, source_worker, source_addr, f); + free(source_addr); +} + static int vine_temp_replicate_file_now(struct vine_manager *q, struct vine_file *f) { if (!q || !f || f->type != VINE_TEMP) { @@ -113,7 +149,7 @@ static int vine_temp_replicate_file_now(struct vine_manager *q, struct vine_file return 0; } - vine_temp_start_peer_transfer(q, f, source_worker, dest_worker); + start_peer_transfer(q, f, dest_worker, source_worker); return 1; } @@ -122,23 +158,16 @@ static int vine_temp_replicate_file_now(struct vine_manager *q, struct vine_file /* Public Functions */ /*************************************************************/ -void vine_temp_start_peer_transfer(struct vine_manager *q, struct vine_file *f, struct vine_worker_info *source_worker, struct vine_worker_info *dest_worker) -{ - if (!q || !f || f->type != VINE_TEMP || !source_worker || !dest_worker) { - return; - } - - char *source_addr = string_format("%s/%s", source_worker->transfer_url, f->cached_name); - vine_manager_put_url_now(q, dest_worker, source_worker, source_addr, f); - free(source_addr); -} - int vine_temp_replicate_file_later(struct vine_manager *q, struct vine_file *f) { if (!q || !f || f->type != VINE_TEMP || f->state != VINE_FILE_STATE_CREATED) { return 0; } + if (q->temp_replica_count <= 1) { + return 0; + } + int current_replica_count = vine_file_replica_count(q, f); if (current_replica_count == 0 || current_replica_count >= q->temp_replica_count) { return 0; @@ -149,7 +178,7 @@ int vine_temp_replicate_file_later(struct vine_manager *q, struct vine_file *f) return 1; } -int vine_temp_handle_file_lost(struct vine_manager *q, char *cachename) +int vine_temp_rescue_lost_replica(struct vine_manager *q, char *cachename) { if (!q || !cachename) { return 0; @@ -219,7 +248,7 @@ Clean redundant replicas of a temporary file. For example, a file may be transferred to another worker because a task that declares it as input is scheduled there, resulting in an extra replica that consumes storage space. This function evaluates whether the file has excessive replicas and removes those on -workers that do not execute their dependent tasks. +workers that do not execute their dependent tasks. */ void vine_temp_clean_redundant_replicas(struct vine_manager *q, struct vine_file *f) { @@ -296,28 +325,23 @@ void vine_temp_shift_disk_load(struct vine_manager *q, struct vine_worker_info * return; } + if (!q->shift_disk_load) { + return; + } + struct vine_worker_info *target_worker = NULL; char *key; struct vine_worker_info *w = NULL; HASH_TABLE_ITERATE(q->worker_table, key, w) { - if (w->type != VINE_WORKER_TYPE_WORKER) { - continue; - } - if (!w->transfer_port_active) { - continue; - } - if (w->draining) { - continue; - } - if (w->resources->tag < 0) { + if (!is_worker_active(w)) { continue; } if (vine_file_replica_table_lookup(w, f->cached_name)) { continue; } - if (w->inuse_cache + f->size > (source_worker->inuse_cache - f->size) * 0.8) { + if (w->inuse_cache + f->size > (source_worker->inuse_cache - f->size)) { continue; } if (!target_worker || w->inuse_cache < target_worker->inuse_cache) { @@ -325,9 +349,7 @@ void vine_temp_shift_disk_load(struct vine_manager *q, struct vine_worker_info * } } if (target_worker) { - char *source_addr = string_format("%s/%s", source_worker->transfer_url, f->cached_name); - vine_manager_put_url_now(q, target_worker, source_worker, source_addr, f); - free(source_addr); + start_peer_transfer(q, f, target_worker, source_worker); } /* We can clean up the original one safely when the replica arrives at the destination worker. */ diff --git a/taskvine/src/manager/vine_temp.h b/taskvine/src/manager/vine_temp.h index d6f2d72523..1a1ce2a096 100644 --- a/taskvine/src/manager/vine_temp.h +++ b/taskvine/src/manager/vine_temp.h @@ -4,10 +4,10 @@ #include "vine_manager.h" int vine_temp_replicate_file_later(struct vine_manager *q, struct vine_file *f); -int vine_temp_handle_file_lost(struct vine_manager *q, char *cachename); +int vine_temp_rescue_lost_replica(struct vine_manager *q, char *cachename); int vine_temp_start_replication(struct vine_manager *q); -void vine_temp_start_peer_transfer(struct vine_manager *q, struct vine_file *f, struct vine_worker_info *source_worker, struct vine_worker_info *dest_worker); +void vine_temp_clean_redundant_replicas(struct vine_manager *q, struct vine_file *f); void vine_temp_shift_disk_load(struct vine_manager *q, struct vine_worker_info *source_worker, struct vine_file *f); #endif \ No newline at end of file From d10c7ace5907b8c51fe73266da697f302b90a320 Mon Sep 17 00:00:00 2001 From: Jin Zhou Date: Tue, 4 Nov 2025 17:42:22 -0500 Subject: [PATCH 081/113] lint --- taskvine/src/manager/vine_manager.c | 2 ++ 1 file changed, 2 insertions(+) diff --git a/taskvine/src/manager/vine_manager.c b/taskvine/src/manager/vine_manager.c index 5481681251..199705c270 100644 --- a/taskvine/src/manager/vine_manager.c +++ b/taskvine/src/manager/vine_manager.c @@ -6409,6 +6409,8 @@ int vine_prune_file(struct vine_manager *m, struct vine_file *f) set_free_values_array(workers_array); } } + + return pruned_replica_count; } /* From 57673e75fcafae8ba52f8acd826ce6d4cd252f21 Mon Sep 17 00:00:00 2001 From: Jin Zhou Date: Tue, 4 Nov 2025 18:51:54 -0500 Subject: [PATCH 082/113] trigger rebuild From 1d91659507e0d68c61ff2b8a83ada6577cb1b290 Mon Sep 17 00:00:00 2001 From: Jin Zhou Date: Wed, 5 Nov 2025 00:07:39 -0500 Subject: [PATCH 083/113] comments --- taskvine/src/manager/vine_manager.c | 12 +++-- taskvine/src/manager/vine_temp.c | 79 +++++++++++++++++++++++------ taskvine/src/manager/vine_temp.h | 6 ++- 3 files changed, 76 insertions(+), 21 deletions(-) diff --git a/taskvine/src/manager/vine_manager.c b/taskvine/src/manager/vine_manager.c index 199705c270..757eda856b 100644 --- a/taskvine/src/manager/vine_manager.c +++ b/taskvine/src/manager/vine_manager.c @@ -426,8 +426,12 @@ static vine_msg_code_t handle_cache_update(struct vine_manager *q, struct vine_w /* If a TEMP file, replicate or shift disk load as needed. */ if (f->type == VINE_TEMP) { - vine_temp_replicate_file_later(q, f); - vine_temp_shift_disk_load(q, w, f); + if (q->temp_replica_count > 1) { + vine_temp_queue_for_replication(q, f); + } + if (q->shift_disk_load) { + vine_temp_shift_disk_load(q, w, f); + } } } } @@ -484,7 +488,7 @@ static vine_msg_code_t handle_cache_invalid(struct vine_manager *q, struct vine_ } /* If the creation failed, we may want to backup the file somewhere else. */ - vine_temp_rescue_lost_replica(q, cachename); + vine_temp_handle_lost_replica(q, cachename); /* Successfully processed this message. */ return VINE_MSG_PROCESSED; @@ -1071,7 +1075,7 @@ static void recall_worker_lost_temp_files(struct vine_manager *q, struct vine_wo // Iterate over files we want might want to recover HASH_TABLE_ITERATE(w->current_files, cached_name, info) { - vine_temp_rescue_lost_replica(q, cached_name); + vine_temp_handle_lost_replica(q, cached_name); } } diff --git a/taskvine/src/manager/vine_temp.c b/taskvine/src/manager/vine_temp.c index b7f1ef1c90..f7dac15109 100644 --- a/taskvine/src/manager/vine_temp.c +++ b/taskvine/src/manager/vine_temp.c @@ -20,6 +20,9 @@ /* Private Functions */ /*************************************************************/ +/** +Check whether a worker is eligible to participate in peer transfers. +*/ static int is_worker_active(struct vine_worker_info *w) { if (!w) { @@ -40,6 +43,12 @@ static int is_worker_active(struct vine_worker_info *w) return 1; } +/** +Find the most suitable worker to serve as the source of a replica transfer. +Eligible workers already host the file, have a ready replica, and are not +overloaded with outgoing transfers. Preference is given to workers with fewer +outgoing transfers to balance load. +*/ static struct vine_worker_info *get_best_source_worker(struct vine_manager *q, struct vine_file *f) { if (!q || !f || f->type != VINE_TEMP) { @@ -82,6 +91,12 @@ static struct vine_worker_info *get_best_source_worker(struct vine_manager *q, s return best_source; } +/** +Select a destination worker that can accept a new replica. Workers must be +active, not currently hosting the file, and have sufficient free cache space. +Those with more available disk space are prioritized to reduce pressure on +heavily utilized workers. +*/ static struct vine_worker_info *get_best_dest_worker(struct vine_manager *q, struct vine_file *f) { if (!q || !f || f->type != VINE_TEMP) { @@ -122,6 +137,11 @@ static struct vine_worker_info *get_best_dest_worker(struct vine_manager *q, str return best_destination; } +/** +Initiate a peer-to-peer transfer between two workers for the specified file. +The source worker provides a direct URL so the destination worker can pull the +replica immediately via `vine_manager_put_url_now`. +*/ static void start_peer_transfer(struct vine_manager *q, struct vine_file *f, struct vine_worker_info *dest_worker, struct vine_worker_info *source_worker) { if (!q || !f || f->type != VINE_TEMP || !dest_worker || !source_worker) { @@ -133,6 +153,11 @@ static void start_peer_transfer(struct vine_manager *q, struct vine_file *f, str free(source_addr); } +/** +Attempt to replicate a temporary file immediately by selecting compatible +source and destination workers. Returns 1 when a transfer is launched, or 0 if +no suitable pair of workers is currently available. +*/ static int vine_temp_replicate_file_now(struct vine_manager *q, struct vine_file *f) { if (!q || !f || f->type != VINE_TEMP) { @@ -158,7 +183,12 @@ static int vine_temp_replicate_file_now(struct vine_manager *q, struct vine_file /* Public Functions */ /*************************************************************/ -int vine_temp_replicate_file_later(struct vine_manager *q, struct vine_file *f) +/** +Queue a temporary file for replication when it still lacks the target number of +replicas. Files without any replica and those already satisfying the quota are +ignored. A lower priority value gives preference to scarcer replicas. +*/ +int vine_temp_queue_for_replication(struct vine_manager *q, struct vine_file *f) { if (!q || !f || f->type != VINE_TEMP || f->state != VINE_FILE_STATE_CREATED) { return 0; @@ -178,7 +208,15 @@ int vine_temp_replicate_file_later(struct vine_manager *q, struct vine_file *f) return 1; } -int vine_temp_rescue_lost_replica(struct vine_manager *q, char *cachename) +/** +Respond to a missing replica notification by re-queuing the corresponding file +for replication, provided the file is still valid and managed by this +coordinator. The use cases include when a cache-invalid message is received from a worker, +or when a worker disconnects unexpectedly, and we need to rescue the lost data. +If the replica does not have any ready source, it will be silently discarded in the +replication phase, so don't worry about it. +*/ +int vine_temp_handle_lost_replica(struct vine_manager *q, char *cachename) { if (!q || !cachename) { return 0; @@ -189,11 +227,19 @@ int vine_temp_rescue_lost_replica(struct vine_manager *q, char *cachename) return 0; } - vine_temp_replicate_file_later(q, f); + vine_temp_queue_for_replication(q, f); return 1; } +/** +Iterate through temporary files that still need additional replicas and +trigger peer-to-peer transfers when both a source and destination worker +are available. The function honors the manager's scheduling depth so that we +do not spend too much time evaluating the queue in a single invocation. +Files that cannot be replicated immediately are deferred by lowering their +priority and will be reconsidered in future calls. +*/ int vine_temp_start_replication(struct vine_manager *q) { if (!q) { @@ -202,27 +248,31 @@ int vine_temp_start_replication(struct vine_manager *q) int processed = 0; int iter_count = 0; + /* Only examine up to attempt_schedule_depth files to keep the event loop responsive. */ int iter_depth = MIN(q->attempt_schedule_depth, priority_queue_size(q->temp_files_to_replicate)); + /* Files that cannot be replicated now are temporarily stored and re-queued at the end. */ struct list *skipped = list_create(); struct vine_file *f; while ((f = priority_queue_pop(q->temp_files_to_replicate)) && (iter_count++ < iter_depth)) { + /* skip and discard the replication request if the file is not valid */ if (!f || f->type != VINE_TEMP || f->state != VINE_FILE_STATE_CREATED) { continue; } - /* skip if the file has enough replicas or no replicas */ + /* skip and discard the replication request if the file has enough replicas or no replicas */ int current_replica_count = vine_file_replica_count(q, f); if (current_replica_count >= q->temp_replica_count || current_replica_count == 0) { continue; } - /* skip if the file has no ready replicas */ + /* skip and discard the replication request if the file has no ready replicas */ int current_ready_replica_count = vine_file_replica_table_count_replicas(q, f->cached_name, VINE_FILE_REPLICA_STATE_READY); if (current_ready_replica_count == 0) { continue; } - /* if reach here, it means the file needs to be replicated and there is at least one ready replica. */ + /* If reaches here, the file still lacks replicas and has at least one ready source, so we start finding a valid source and destination worker + * and trigger the replication. If fails to find a valid source or destination worker, we requeue the file and will consider later. */ if (!vine_temp_replicate_file_now(q, f)) { list_push_tail(skipped, f); continue; @@ -230,13 +280,12 @@ int vine_temp_start_replication(struct vine_manager *q) processed++; - /* push back and keep evaluating the same file with a lower priority, until no more source - * or destination workers are available, or the file has enough replicas. */ - vine_temp_replicate_file_later(q, f); + /* Requeue the file with lower priority so it can accumulate replicas gradually. */ + vine_temp_queue_for_replication(q, f); } while ((f = list_pop_head(skipped))) { - vine_temp_replicate_file_later(q, f); + vine_temp_queue_for_replication(q, f); } list_delete(skipped); @@ -325,25 +374,25 @@ void vine_temp_shift_disk_load(struct vine_manager *q, struct vine_worker_info * return; } - if (!q->shift_disk_load) { - return; - } - struct vine_worker_info *target_worker = NULL; char *key; struct vine_worker_info *w = NULL; HASH_TABLE_ITERATE(q->worker_table, key, w) { + /* skip if the worker is not active */ if (!is_worker_active(w)) { continue; } + /* skip if the worker already has this file */ if (vine_file_replica_table_lookup(w, f->cached_name)) { continue; } - if (w->inuse_cache + f->size > (source_worker->inuse_cache - f->size)) { + /* skip if the worker becomes heavier after the transfer */ + if (w->inuse_cache + f->size > source_worker->inuse_cache - f->size) { continue; } + /* workers with less inuse cache space are preferred */ if (!target_worker || w->inuse_cache < target_worker->inuse_cache) { target_worker = w; } diff --git a/taskvine/src/manager/vine_temp.h b/taskvine/src/manager/vine_temp.h index 1a1ce2a096..0943b9b9cc 100644 --- a/taskvine/src/manager/vine_temp.h +++ b/taskvine/src/manager/vine_temp.h @@ -3,10 +3,12 @@ #include "vine_manager.h" -int vine_temp_replicate_file_later(struct vine_manager *q, struct vine_file *f); -int vine_temp_rescue_lost_replica(struct vine_manager *q, char *cachename); +/** Replication related functions */ +int vine_temp_queue_for_replication(struct vine_manager *q, struct vine_file *f); int vine_temp_start_replication(struct vine_manager *q); +int vine_temp_handle_lost_replica(struct vine_manager *q, char *cachename); +/** Storage management functions */ void vine_temp_clean_redundant_replicas(struct vine_manager *q, struct vine_file *f); void vine_temp_shift_disk_load(struct vine_manager *q, struct vine_worker_info *source_worker, struct vine_file *f); From ed1fc2368cbf0c341cd90570541fbebfa11212c1 Mon Sep 17 00:00:00 2001 From: Jin Zhou Date: Wed, 5 Nov 2025 00:12:13 -0500 Subject: [PATCH 084/113] remove unrelated code --- taskvine/src/manager/vine_manager.c | 6 ------ 1 file changed, 6 deletions(-) diff --git a/taskvine/src/manager/vine_manager.c b/taskvine/src/manager/vine_manager.c index 757eda856b..d04096cc8f 100644 --- a/taskvine/src/manager/vine_manager.c +++ b/taskvine/src/manager/vine_manager.c @@ -418,12 +418,6 @@ static vine_msg_code_t handle_cache_update(struct vine_manager *q, struct vine_w f->state = VINE_FILE_STATE_CREATED; f->size = size; - /* If the replica's type was a URL, it means the manager expected the destination worker to download it - * from elsewhere. Now that it's physically present, we can resolve its type back to the original */ - if (replica->type == VINE_URL) { - replica->type = f->type; - } - /* If a TEMP file, replicate or shift disk load as needed. */ if (f->type == VINE_TEMP) { if (q->temp_replica_count > 1) { From 8280ef61982e3d4dc0d595201441deb66f91a01e Mon Sep 17 00:00:00 2001 From: Jin Zhou Date: Wed, 5 Nov 2025 11:55:32 -0500 Subject: [PATCH 085/113] fix flicker --- dttools/src/progress_bar.c | 48 ++++++++++++++++++++++++--------- dttools/src/progress_bar.h | 20 ++++++++++---- dttools/src/progress_bar_test.c | 4 ++- 3 files changed, 53 insertions(+), 19 deletions(-) diff --git a/dttools/src/progress_bar.c b/dttools/src/progress_bar.c index 14e1995b42..614d4f7439 100644 --- a/dttools/src/progress_bar.c +++ b/dttools/src/progress_bar.c @@ -11,6 +11,8 @@ Implementation of a terminal progress bar with multiple parts. #include "progress_bar.h" #include "xxmalloc.h" #include "macros.h" +#include "macros.h" + #include #include #include @@ -23,8 +25,9 @@ Implementation of a terminal progress bar with multiple parts. /* Max bar width (in block characters) for single-line rendering. */ #define MAX_BAR_WIDTH 30 -/* Minimum redraw interval to avoid flicker. */ -#define PROGRESS_BAR_UPDATE_INTERVAL ((USECOND) * 0.1) + +/* Minimum redraw interval to avoid flicker (200ms). */ +#define PROGRESS_BAR_UPDATE_INTERVAL_US (USECOND / 5) #define COLOR_RESET "\033[0m" #define COLOR_GREEN "\033[32m" @@ -75,7 +78,7 @@ static void print_progress_bar(struct ProgressBar *bar) return; } - bar->last_draw_time = timestamp_get(); + bar->last_draw_time_us = timestamp_get(); char part_text[256]; char *ptr = part_text; @@ -114,7 +117,7 @@ static void print_progress_bar(struct ProgressBar *bar) progress = 1.0f; } - timestamp_t elapsed = timestamp_get() - bar->start_time; + timestamp_t elapsed = timestamp_get() - bar->start_time_us; int h = elapsed / (3600LL * USECOND); int m = (elapsed % (3600LL * USECOND)) / (60LL * USECOND); int s = (elapsed % (60LL * USECOND)) / USECOND; @@ -167,13 +170,34 @@ struct ProgressBar *progress_bar_init(const char *label) bar->label = xxstrdup(label); bar->parts = list_create(); - bar->start_time = timestamp_get(); - bar->last_draw_time = timestamp_get(); + bar->start_time_us = timestamp_get(); + bar->last_draw_time_us = 0; + bar->update_interval_us = PROGRESS_BAR_UPDATE_INTERVAL_US; + bar->update_interval_sec = (double)bar->update_interval_us / USECOND; bar->has_drawn_once = 0; return bar; } +/** Set the update interval for the progress bar. */ +void progress_bar_set_update_interval(struct ProgressBar *bar, double update_interval_sec) +{ + if (!bar) { + return; + } + + if (update_interval_sec < 0) { + update_interval_sec = 0; + } + bar->update_interval_sec = update_interval_sec; + /* Convert seconds to microseconds with saturation to avoid overflow. */ + if (update_interval_sec >= (double)UINT64_MAX / (double)USECOND) { + bar->update_interval_us = (timestamp_t)UINT64_MAX; + } else { + bar->update_interval_us = (timestamp_t)(update_interval_sec * (double)USECOND); + } +} + /** Create a new part. */ struct ProgressBarPart *progress_bar_create_part(const char *label, uint64_t total) { @@ -207,9 +231,8 @@ void progress_bar_set_part_total(struct ProgressBar *bar, struct ProgressBarPart if (!bar || !part) { return; } - part->total = new_total; - print_progress_bar(bar); + part->total = new_total; } /** Advance a part's current value, redraw if needed. */ @@ -224,11 +247,10 @@ void progress_bar_update_part(struct ProgressBar *bar, struct ProgressBarPart *p part->current = part->total; } - if (timestamp_get() - bar->last_draw_time < PROGRESS_BAR_UPDATE_INTERVAL) { - return; + 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); } - - print_progress_bar(bar); } /** Set the start time for the progress bar. */ @@ -238,7 +260,7 @@ void progress_bar_set_start_time(struct ProgressBar *bar, timestamp_t start_time return; } - bar->start_time = start_time; + bar->start_time_us = start_time; } /** Final render and newline. */ diff --git a/dttools/src/progress_bar.h b/dttools/src/progress_bar.h index 6a38a95424..7fe3171a3f 100644 --- a/dttools/src/progress_bar.h +++ b/dttools/src/progress_bar.h @@ -25,11 +25,15 @@ struct ProgressBarPart { /** Progress bar object. */ struct ProgressBar { - char *label; - struct list *parts; - timestamp_t start_time; - timestamp_t last_draw_time; - int has_drawn_once; + /* User-facing interval in seconds; internal comparisons use *_us. */ + double update_interval_sec; + char *label; + struct list *parts; + /* Timestamps in microseconds. */ + timestamp_t start_time_us; + timestamp_t last_draw_time_us; + timestamp_t update_interval_us; + int has_drawn_once; }; /* Progress Bar Part API */ @@ -40,6 +44,12 @@ struct ProgressBar { */ struct ProgressBar *progress_bar_init(const char *label); +/** Set the update interval for the progress bar. +@param bar Progress bar. +@param update_interval_sec Update interval in seconds. +*/ +void progress_bar_set_update_interval(struct ProgressBar *bar, double update_interval_sec); + /** Create a new part. @param label Part label (internally duplicated). @param total Total units for the part. diff --git a/dttools/src/progress_bar_test.c b/dttools/src/progress_bar_test.c index 4c72d09425..0d3d3fa813 100644 --- a/dttools/src/progress_bar_test.c +++ b/dttools/src/progress_bar_test.c @@ -6,12 +6,14 @@ int main() { - uint64_t total = 100000; + uint64_t total = 100000000; struct ProgressBarPart *part1 = progress_bar_create_part("step", total); struct ProgressBarPart *part2 = progress_bar_create_part("fetch", total); struct ProgressBarPart *part3 = progress_bar_create_part("commit", total); struct ProgressBar *bar = progress_bar_init("Compute"); + progress_bar_set_update_interval(bar, 1); + progress_bar_bind_part(bar, part1); progress_bar_bind_part(bar, part2); progress_bar_bind_part(bar, part3); From 176201f57ff0dd90e02ecc8ce6b9829f0cb87fd1 Mon Sep 17 00:00:00 2001 From: Jin Zhou Date: Thu, 6 Nov 2025 15:31:58 -0500 Subject: [PATCH 086/113] fix comment --- taskvine/src/manager/vine_manager.c | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/taskvine/src/manager/vine_manager.c b/taskvine/src/manager/vine_manager.c index 1886c7f34a..1da2d9c082 100644 --- a/taskvine/src/manager/vine_manager.c +++ b/taskvine/src/manager/vine_manager.c @@ -672,7 +672,7 @@ static vine_result_code_t get_completion_result(struct vine_manager *q, struct v /* A task scheduling may result in a redundant replica of its input due to peer transfers, which can be safely removed when completed. * However, the general function of taskvine is to replicate files on demand, and to only clean them up when prune is called. - * So, we only clean up redundant replicas for the task-inputs when the manager is configured to clean redundant replicas. */ + * So, we only clean up redundant replicas for the task-inputs when the manager is configured to do so. */ if (q->clean_redundant_replicas) { struct vine_mount *input_mount; LIST_ITERATE(t->input_mounts, input_mount) From b6880b2b334cb427aefa1c527dcfd5c171440f5b Mon Sep 17 00:00:00 2001 From: Jin Zhou Date: Thu, 6 Nov 2025 16:31:25 -0500 Subject: [PATCH 087/113] trigger rebuild From ba0b9853eb039bd289ef18eb6d9e442d5da21145 Mon Sep 17 00:00:00 2001 From: Jin Zhou Date: Mon, 10 Nov 2025 11:09:26 -0500 Subject: [PATCH 088/113] review fix --- poncho/src/poncho/library_network_code.py | 6 ++++-- 1 file changed, 4 insertions(+), 2 deletions(-) diff --git a/poncho/src/poncho/library_network_code.py b/poncho/src/poncho/library_network_code.py index b696ee5e57..5fab4b0b85 100755 --- a/poncho/src/poncho/library_network_code.py +++ b/poncho/src/poncho/library_network_code.py @@ -179,9 +179,11 @@ def start_function(in_pipe_fd, thread_limit=1): try: # each child process independently redirects its own stdout/stderr. with open(function_stdout_filename, "wb", buffering=0) as f: - os.dup2(f.fileno(), 1) # redirect stdout - os.dup2(f.fileno(), 2) # redirect stderr + os.dup2(f.fileno(), sys.stdout.fileno()) # redirect stdout + os.dup2(f.fileno(), sys.stderr.fileno()) # redirect stderr + # keep the function invocation inside the context + # so the stdout/stderr file stays open during execution and closes safely afterward. stdout_timed_message(f"TASK {function_id} {function_name} starts in PID {os.getpid()}") result = globals()[function_name](event) stdout_timed_message(f"TASK {function_id} {function_name} finished") From c89bd8576a9aeff952893201cce4a3f47a0f49ff Mon Sep 17 00:00:00 2001 From: Jin Zhou Date: Wed, 3 Dec 2025 13:21:18 -0500 Subject: [PATCH 089/113] modify project name --- dttools/src/progress_bar.c | 20 ++--- dttools/src/progress_bar_test.c | 6 +- taskvine/src/graph/Makefile | 2 +- .../src/graph/{vinedag => dagvine}/.gitignore | 0 .../src/graph/{vinedag => dagvine}/.gitkeep | 0 .../src/graph/{vinedag => dagvine}/Makefile | 4 +- .../graph/{vinedag => dagvine}/__init__.py | 4 +- .../context_graph/Makefile | 2 +- .../context_graph/__init__.py | 0 .../context_graph/core.py | 4 +- .../context_graph/proxy_functions.py | 2 +- .../context_graph/proxy_library.py | 4 +- .../vinedag.py => dagvine/dagvine.py} | 24 +++-- .../vine_graph/.gitignore | 0 .../{vinedag => dagvine}/vine_graph/Makefile | 2 +- .../vine_graph/__init__.py | 0 .../vine_graph/vine_graph.c | 0 .../vine_graph/vine_graph.h | 0 .../vine_graph/vine_graph.i | 2 +- .../vine_graph/vine_graph_client.py | 0 .../vine_graph/vine_node.c | 0 .../vine_graph/vine_node.h | 0 taskvine/src/manager/vine_manager.c | 57 ++++++------ taskvine/src/manager/vine_temp.c | 88 +++++++++---------- taskvine/src/manager/vine_temp.h | 1 - 25 files changed, 111 insertions(+), 111 deletions(-) rename taskvine/src/graph/{vinedag => dagvine}/.gitignore (100%) rename taskvine/src/graph/{vinedag => dagvine}/.gitkeep (100%) rename taskvine/src/graph/{vinedag => dagvine}/Makefile (85%) rename taskvine/src/graph/{vinedag => dagvine}/__init__.py (75%) rename taskvine/src/graph/{vinedag => dagvine}/context_graph/Makefile (96%) rename taskvine/src/graph/{vinedag => dagvine}/context_graph/__init__.py (100%) rename taskvine/src/graph/{vinedag => dagvine}/context_graph/core.py (98%) rename taskvine/src/graph/{vinedag => dagvine}/context_graph/proxy_functions.py (98%) rename taskvine/src/graph/{vinedag => dagvine}/context_graph/proxy_library.py (96%) rename taskvine/src/graph/{vinedag/vinedag.py => dagvine/dagvine.py} (93%) rename taskvine/src/graph/{vinedag => dagvine}/vine_graph/.gitignore (100%) rename taskvine/src/graph/{vinedag => dagvine}/vine_graph/Makefile (99%) rename taskvine/src/graph/{vinedag => dagvine}/vine_graph/__init__.py (100%) rename taskvine/src/graph/{vinedag => dagvine}/vine_graph/vine_graph.c (100%) rename taskvine/src/graph/{vinedag => dagvine}/vine_graph/vine_graph.h (100%) rename taskvine/src/graph/{vinedag => dagvine}/vine_graph/vine_graph.i (82%) rename taskvine/src/graph/{vinedag => dagvine}/vine_graph/vine_graph_client.py (100%) rename taskvine/src/graph/{vinedag => dagvine}/vine_graph/vine_node.c (100%) rename taskvine/src/graph/{vinedag => dagvine}/vine_graph/vine_node.h (100%) diff --git a/dttools/src/progress_bar.c b/dttools/src/progress_bar.c index 012d80e003..614d4f7439 100644 --- a/dttools/src/progress_bar.c +++ b/dttools/src/progress_bar.c @@ -11,6 +11,8 @@ Implementation of a terminal progress bar with multiple parts. #include "progress_bar.h" #include "xxmalloc.h" #include "macros.h" +#include "macros.h" + #include #include #include @@ -23,13 +25,9 @@ Implementation of a terminal progress bar with multiple parts. /* Max bar width (in block characters) for single-line rendering. */ #define MAX_BAR_WIDTH 30 -/* Typed time constants (microseconds). */ -static const timestamp_t SECOND_US = 1000000ULL; -static const timestamp_t MILLISECOND_US = 1000ULL; -static const timestamp_t MICROSECOND_US = 1ULL; /* Minimum redraw interval to avoid flicker (200ms). */ -#define PROGRESS_BAR_UPDATE_INTERVAL_US (SECOND_US / 5) +#define PROGRESS_BAR_UPDATE_INTERVAL_US (USECOND / 5) #define COLOR_RESET "\033[0m" #define COLOR_GREEN "\033[32m" @@ -120,9 +118,9 @@ static void print_progress_bar(struct ProgressBar *bar) } timestamp_t elapsed = timestamp_get() - bar->start_time_us; - int h = elapsed / (3600LL * SECOND_US); - int m = (elapsed % (3600LL * SECOND_US)) / (60LL * SECOND_US); - int s = (elapsed % (60LL * SECOND_US)) / SECOND_US; + int h = elapsed / (3600LL * USECOND); + int m = (elapsed % (3600LL * USECOND)) / (60LL * USECOND); + int s = (elapsed % (60LL * USECOND)) / USECOND; if (bar->has_drawn_once) { printf("\r\033[2K"); @@ -175,7 +173,7 @@ struct ProgressBar *progress_bar_init(const char *label) bar->start_time_us = timestamp_get(); bar->last_draw_time_us = 0; bar->update_interval_us = PROGRESS_BAR_UPDATE_INTERVAL_US; - bar->update_interval_sec = (double)bar->update_interval_us / SECOND_US; + bar->update_interval_sec = (double)bar->update_interval_us / USECOND; bar->has_drawn_once = 0; return bar; @@ -193,10 +191,10 @@ void progress_bar_set_update_interval(struct ProgressBar *bar, double update_int } bar->update_interval_sec = update_interval_sec; /* Convert seconds to microseconds with saturation to avoid overflow. */ - if (update_interval_sec >= (double)UINT64_MAX / (double)SECOND_US) { + if (update_interval_sec >= (double)UINT64_MAX / (double)USECOND) { bar->update_interval_us = (timestamp_t)UINT64_MAX; } else { - bar->update_interval_us = (timestamp_t)(update_interval_sec * (double)SECOND_US); + bar->update_interval_us = (timestamp_t)(update_interval_sec * (double)USECOND); } } diff --git a/dttools/src/progress_bar_test.c b/dttools/src/progress_bar_test.c index 4a7b70ed1c..1262a6071e 100644 --- a/dttools/src/progress_bar_test.c +++ b/dttools/src/progress_bar_test.c @@ -6,13 +6,13 @@ int main() { - uint64_t total = 1000000; + uint64_t total = 100000000; struct ProgressBarPart *part1 = progress_bar_create_part("step", total); struct ProgressBarPart *part2 = progress_bar_create_part("fetch", total); struct ProgressBarPart *part3 = progress_bar_create_part("commit", total); struct ProgressBar *bar = progress_bar_init("Compute"); - progress_bar_set_update_interval(bar, 0.5); + progress_bar_set_update_interval(bar, 1); progress_bar_bind_part(bar, part1); progress_bar_bind_part(bar, part2); @@ -32,4 +32,4 @@ int main() printf("time taken: %" PRIu64 "\n", end_time - start_time); return 0; -} +} \ No newline at end of file diff --git a/taskvine/src/graph/Makefile b/taskvine/src/graph/Makefile index f961ec9e86..c56a9f8090 100644 --- a/taskvine/src/graph/Makefile +++ b/taskvine/src/graph/Makefile @@ -1,7 +1,7 @@ include ../../../config.mk include ../../../rules.mk -SUBDIRS = vinedag +SUBDIRS = dagvine all clean install test lint format: $(SUBDIRS) diff --git a/taskvine/src/graph/vinedag/.gitignore b/taskvine/src/graph/dagvine/.gitignore similarity index 100% rename from taskvine/src/graph/vinedag/.gitignore rename to taskvine/src/graph/dagvine/.gitignore diff --git a/taskvine/src/graph/vinedag/.gitkeep b/taskvine/src/graph/dagvine/.gitkeep similarity index 100% rename from taskvine/src/graph/vinedag/.gitkeep rename to taskvine/src/graph/dagvine/.gitkeep diff --git a/taskvine/src/graph/vinedag/Makefile b/taskvine/src/graph/dagvine/Makefile similarity index 85% rename from taskvine/src/graph/vinedag/Makefile rename to taskvine/src/graph/dagvine/Makefile index 32dff04894..f40356744e 100644 --- a/taskvine/src/graph/vinedag/Makefile +++ b/taskvine/src/graph/dagvine/Makefile @@ -1,7 +1,7 @@ include ../../../../config.mk include ../../../../rules.mk -MODULE_DIR := $(CCTOOLS_PYTHON3_PATH)/ndcctools/taskvine/vinedag +MODULE_DIR := $(CCTOOLS_PYTHON3_PATH)/ndcctools/taskvine/dagvine SUBDIRS := context_graph vine_graph .PHONY: all install clean lint format $(SUBDIRS) @@ -13,7 +13,7 @@ $(SUBDIRS): install: all mkdir -p $(MODULE_DIR) - cp vinedag.py $(MODULE_DIR) + cp dagvine.py $(MODULE_DIR) cp __init__.py $(MODULE_DIR) @for dir in $(SUBDIRS); do \ $(MAKE) -C $$dir install; \ diff --git a/taskvine/src/graph/vinedag/__init__.py b/taskvine/src/graph/dagvine/__init__.py similarity index 75% rename from taskvine/src/graph/vinedag/__init__.py rename to taskvine/src/graph/dagvine/__init__.py index 0a3da3f715..feee0c0384 100644 --- a/taskvine/src/graph/vinedag/__init__.py +++ b/taskvine/src/graph/dagvine/__init__.py @@ -2,6 +2,6 @@ # This software is distributed under the GNU General Public License. # See the file COPYING for details. -from .vinedag import VineDAG +from .dagvine import DAGVine -__all__ = ["VineDAG"] +__all__ = ["DAGVine"] diff --git a/taskvine/src/graph/vinedag/context_graph/Makefile b/taskvine/src/graph/dagvine/context_graph/Makefile similarity index 96% rename from taskvine/src/graph/vinedag/context_graph/Makefile rename to taskvine/src/graph/dagvine/context_graph/Makefile index bdee79013e..a356267995 100644 --- a/taskvine/src/graph/vinedag/context_graph/Makefile +++ b/taskvine/src/graph/dagvine/context_graph/Makefile @@ -1,7 +1,7 @@ include ../../../../../config.mk include ../../../../../rules.mk -PROJECT_NAME = vinedag +PROJECT_NAME = dagvine SOURCE_DIR = $(CCTOOLS_HOME)/taskvine/src/graph/$(PROJECT_NAME)/context_graph MODULE_ROOT = $(CCTOOLS_PYTHON3_PATH)/ndcctools/taskvine/$(PROJECT_NAME) diff --git a/taskvine/src/graph/vinedag/context_graph/__init__.py b/taskvine/src/graph/dagvine/context_graph/__init__.py similarity index 100% rename from taskvine/src/graph/vinedag/context_graph/__init__.py rename to taskvine/src/graph/dagvine/context_graph/__init__.py diff --git a/taskvine/src/graph/vinedag/context_graph/core.py b/taskvine/src/graph/dagvine/context_graph/core.py similarity index 98% rename from taskvine/src/graph/vinedag/context_graph/core.py rename to taskvine/src/graph/dagvine/context_graph/core.py index 61bbeb0d30..f01cd35596 100644 --- a/taskvine/src/graph/vinedag/context_graph/core.py +++ b/taskvine/src/graph/dagvine/context_graph/core.py @@ -61,7 +61,7 @@ class ContextGraph: def __init__(self, task_dict, extra_task_output_size_mb=[0, 0], extra_task_sleep_time=[0, 0]): - """Capture the Python DAG that VineDAG hands us before we mirror it in C.""" + """Capture the Python DAG that DAGVine hands us before we mirror it in C.""" self.task_dict = task_dict if dts: @@ -161,7 +161,7 @@ def load_result_of_key(self, key): return ContextGraphTaskResult.load_from_path(outfile_path) def get_topological_order(self): - """Produce the order VineDAG uses when assigning node IDs to the C graph.""" + """Produce the order DAGVine uses when assigning node IDs to the C graph.""" in_degree = {key: len(self.parents_of[key]) for key in self.task_dict.keys()} queue = deque([key for key, degree in in_degree.items() if degree == 0]) topo_order = [] diff --git a/taskvine/src/graph/vinedag/context_graph/proxy_functions.py b/taskvine/src/graph/dagvine/context_graph/proxy_functions.py similarity index 98% rename from taskvine/src/graph/vinedag/context_graph/proxy_functions.py rename to taskvine/src/graph/dagvine/context_graph/proxy_functions.py index 4218756007..c466e62576 100644 --- a/taskvine/src/graph/vinedag/context_graph/proxy_functions.py +++ b/taskvine/src/graph/dagvine/context_graph/proxy_functions.py @@ -43,7 +43,7 @@ def compute_sexpr_key(context_graph, k, v): """ Evaluate a symbolic expression (S-expression) task within the task graph. - Both VineDAG and legacy Dask represent computations as symbolic + Both DAGVine and legacy Dask represent computations as symbolic expression trees (S-expressions). Each task value `v` encodes a nested structure where: - Leaf nodes are constants or task keys referencing parent results. diff --git a/taskvine/src/graph/vinedag/context_graph/proxy_library.py b/taskvine/src/graph/dagvine/context_graph/proxy_library.py similarity index 96% rename from taskvine/src/graph/vinedag/context_graph/proxy_library.py rename to taskvine/src/graph/dagvine/context_graph/proxy_library.py index a0c4fb4377..4d0873e667 100644 --- a/taskvine/src/graph/vinedag/context_graph/proxy_library.py +++ b/taskvine/src/graph/dagvine/context_graph/proxy_library.py @@ -11,8 +11,8 @@ import hashlib import collections -from ndcctools.taskvine.vinedag.context_graph.core import ContextGraphTaskResult, ContextGraph -from ndcctools.taskvine.vinedag.context_graph.proxy_functions import compute_dts_key, compute_sexpr_key, compute_single_key +from ndcctools.taskvine.dagvine.context_graph.core import ContextGraphTaskResult, ContextGraph +from ndcctools.taskvine.dagvine.context_graph.proxy_functions import compute_dts_key, compute_sexpr_key, compute_single_key from ndcctools.taskvine.utils import load_variable_from_library diff --git a/taskvine/src/graph/vinedag/vinedag.py b/taskvine/src/graph/dagvine/dagvine.py similarity index 93% rename from taskvine/src/graph/vinedag/vinedag.py rename to taskvine/src/graph/dagvine/dagvine.py index 77975df3f5..b8a51bc82d 100644 --- a/taskvine/src/graph/vinedag/vinedag.py +++ b/taskvine/src/graph/dagvine/dagvine.py @@ -5,10 +5,10 @@ from ndcctools.taskvine import cvine from ndcctools.taskvine.manager import Manager -from ndcctools.taskvine.vinedag.context_graph.proxy_library import ProxyLibrary -from ndcctools.taskvine.vinedag.context_graph.proxy_functions import compute_single_key -from ndcctools.taskvine.vinedag.context_graph.core import ContextGraph, ContextGraphTaskResult -from ndcctools.taskvine.vinedag.vine_graph.vine_graph_client import VineGraphClient +from ndcctools.taskvine.dagvine.context_graph.proxy_library import ProxyLibrary +from ndcctools.taskvine.dagvine.context_graph.proxy_functions import compute_single_key +from ndcctools.taskvine.dagvine.context_graph.core import ContextGraph, ContextGraphTaskResult +from ndcctools.taskvine.dagvine.vine_graph.vine_graph_client import VineGraphClient import cloudpickle import os @@ -51,7 +51,7 @@ def color_text(text, color_code): # Flatten Dask collections into the dict-of-tasks structure the rest of the -# pipeline expects. VineDAG clients often hand us a dict like +# pipeline expects. DAGVine clients often hand us a dict like # {"result": dask.delayed(...)}; we merge the underlying HighLevelGraphs so # `ContextGraph` sees the same dict representation C does. def dask_collections_to_task_dict(collection_dict): @@ -146,7 +146,7 @@ def update_param(self, param_name, new_value): self.vine_manager_tuning_params[param_name] = new_value def get_value_of(self, param_name): - """Helper so VineDAG can pull a knob value without caring where it lives.""" + """Helper so DAGVine can pull a knob value without caring where it lives.""" if param_name in self.vine_manager_tuning_params: return self.vine_manager_tuning_params[param_name] elif param_name in self.vine_graph_tuning_params: @@ -157,7 +157,7 @@ def get_value_of(self, param_name): raise ValueError(f"Invalid param name: {param_name}") -class VineDAG(Manager): +class DAGVine(Manager): def __init__(self, *args, **kwargs): @@ -175,7 +175,7 @@ def __init__(self, if self.run_info_template_path: delete_all_files(self.run_info_template_path) - # Boot the underlying TaskVine manager. The TaskVine manager keeps alive until the vinedag object is destroyed + # Boot the underlying TaskVine manager. The TaskVine manager keeps alive until the dagvine object is destroyed super().__init__(*args, **kwargs) print(f"cvine = {cvine}") self.runtime_directory = cvine.vine_get_runtime_directory(self._taskvine) @@ -282,6 +282,14 @@ def run(self, collection_dict, target_keys=[], params={}, hoisting_modules=[], e # first update the params so that they can be used for the following construction self.update_params(params) + # filter out target keys that are not in the collection dict + missing_keys = [k for k in target_keys if k not in collection_dict] + if missing_keys: + print(f"=== Warning: the following target keys are not in the collection dict:") + for k in missing_keys: + print(f" {k}") + target_keys = list(set(target_keys) - set(missing_keys)) + task_dict = ensure_task_dict(collection_dict) # Build both the Python DAG and its C mirror. diff --git a/taskvine/src/graph/vinedag/vine_graph/.gitignore b/taskvine/src/graph/dagvine/vine_graph/.gitignore similarity index 100% rename from taskvine/src/graph/vinedag/vine_graph/.gitignore rename to taskvine/src/graph/dagvine/vine_graph/.gitignore diff --git a/taskvine/src/graph/vinedag/vine_graph/Makefile b/taskvine/src/graph/dagvine/vine_graph/Makefile similarity index 99% rename from taskvine/src/graph/vinedag/vine_graph/Makefile rename to taskvine/src/graph/dagvine/vine_graph/Makefile index 45fc3f7b42..090682269a 100644 --- a/taskvine/src/graph/vinedag/vine_graph/Makefile +++ b/taskvine/src/graph/dagvine/vine_graph/Makefile @@ -1,7 +1,7 @@ include ../../../../../config.mk include ../../../../../rules.mk -PROJECT_NAME = vinedag +PROJECT_NAME = dagvine LOCAL_LINKAGE+=${CCTOOLS_HOME}/taskvine/src/manager/libtaskvine.a ${CCTOOLS_HOME}/dttools/src/libdttools.a LOCAL_CCFLAGS+=-I ${CCTOOLS_HOME}/taskvine/src/manager diff --git a/taskvine/src/graph/vinedag/vine_graph/__init__.py b/taskvine/src/graph/dagvine/vine_graph/__init__.py similarity index 100% rename from taskvine/src/graph/vinedag/vine_graph/__init__.py rename to taskvine/src/graph/dagvine/vine_graph/__init__.py diff --git a/taskvine/src/graph/vinedag/vine_graph/vine_graph.c b/taskvine/src/graph/dagvine/vine_graph/vine_graph.c similarity index 100% rename from taskvine/src/graph/vinedag/vine_graph/vine_graph.c rename to taskvine/src/graph/dagvine/vine_graph/vine_graph.c diff --git a/taskvine/src/graph/vinedag/vine_graph/vine_graph.h b/taskvine/src/graph/dagvine/vine_graph/vine_graph.h similarity index 100% rename from taskvine/src/graph/vinedag/vine_graph/vine_graph.h rename to taskvine/src/graph/dagvine/vine_graph/vine_graph.h diff --git a/taskvine/src/graph/vinedag/vine_graph/vine_graph.i b/taskvine/src/graph/dagvine/vine_graph/vine_graph.i similarity index 82% rename from taskvine/src/graph/vinedag/vine_graph/vine_graph.i rename to taskvine/src/graph/dagvine/vine_graph/vine_graph.i index b87d428c01..65be1a8edb 100644 --- a/taskvine/src/graph/vinedag/vine_graph/vine_graph.i +++ b/taskvine/src/graph/dagvine/vine_graph/vine_graph.i @@ -1,4 +1,4 @@ -/* SWIG interface for local vinedag graph API bindings */ +/* SWIG interface for local dagvine graph API bindings */ %module vine_graph_capi %{ diff --git a/taskvine/src/graph/vinedag/vine_graph/vine_graph_client.py b/taskvine/src/graph/dagvine/vine_graph/vine_graph_client.py similarity index 100% rename from taskvine/src/graph/vinedag/vine_graph/vine_graph_client.py rename to taskvine/src/graph/dagvine/vine_graph/vine_graph_client.py diff --git a/taskvine/src/graph/vinedag/vine_graph/vine_node.c b/taskvine/src/graph/dagvine/vine_graph/vine_node.c similarity index 100% rename from taskvine/src/graph/vinedag/vine_graph/vine_node.c rename to taskvine/src/graph/dagvine/vine_graph/vine_node.c diff --git a/taskvine/src/graph/vinedag/vine_graph/vine_node.h b/taskvine/src/graph/dagvine/vine_graph/vine_node.h similarity index 100% rename from taskvine/src/graph/vinedag/vine_graph/vine_node.h rename to taskvine/src/graph/dagvine/vine_graph/vine_node.h diff --git a/taskvine/src/manager/vine_manager.c b/taskvine/src/manager/vine_manager.c index e6ff9a3660..92bcda6101 100644 --- a/taskvine/src/manager/vine_manager.c +++ b/taskvine/src/manager/vine_manager.c @@ -966,29 +966,6 @@ static void cleanup_worker_files(struct vine_manager *q, struct vine_worker_info hash_table_free_keys_array(cachenames); } -/** Check if a file is busy by checking if it is an input file of any task. */ -static int is_file_busy(struct vine_manager *q, struct vine_worker_info *w, struct vine_file *f) -{ - if (!q || !w || !f) { - return 0; - } - - uint64_t task_id; - struct vine_task *task; - ITABLE_ITERATE(w->current_tasks, task_id, task) - { - struct vine_mount *input_mount; - LIST_ITERATE(task->input_mounts, input_mount) - { - if (f == input_mount->file) { - return 1; - } - } - } - - return 0; -} - /** Evict a random worker to simulate a worker failure. */ int evict_random_worker(struct vine_manager *q) { @@ -1108,24 +1085,42 @@ static void clean_redundant_replicas(struct vine_manager *q, struct vine_file *f return; } - struct priority_queue *offload_from_workers = priority_queue_create(0); + struct priority_queue *clean_replicas_from_workers = priority_queue_create(0); struct vine_worker_info *source_worker = NULL; SET_ITERATE(source_workers, source_worker) { - // workers with more used disk are prioritized for removing - if (is_file_busy(q, source_worker, f)) { + // if the file is actively in use by a task (the input to that task), we don't remove the replica on this worker + int file_inuse = 0; + + uint64_t task_id; + struct vine_task *task; + ITABLE_ITERATE(source_worker->current_tasks, task_id, task) + { + struct vine_mount *input_mount; + LIST_ITERATE(task->input_mounts, input_mount) + { + if (f == input_mount->file) { + file_inuse = 1; + break; + } + } + if (file_inuse) { + break; + } + } + + if (file_inuse) { continue; } - priority_queue_push(offload_from_workers, source_worker, source_worker->inuse_cache); + priority_queue_push(clean_replicas_from_workers, source_worker, source_worker->inuse_cache); } - struct vine_worker_info *offload_from_worker = NULL; - while (replicas_to_remove-- > 0 && (offload_from_worker = priority_queue_pop(offload_from_workers))) { - delete_worker_file(q, offload_from_worker, f->cached_name, 0, 0); + while (replicas_to_remove-- > 0 && (source_worker = priority_queue_pop(clean_replicas_from_workers))) { + delete_worker_file(q, source_worker, f->cached_name, 0, 0); } - priority_queue_delete(offload_from_workers); + priority_queue_delete(clean_replicas_from_workers); return; } diff --git a/taskvine/src/manager/vine_temp.c b/taskvine/src/manager/vine_temp.c index 2f171f5efe..d8a2339208 100644 --- a/taskvine/src/manager/vine_temp.c +++ b/taskvine/src/manager/vine_temp.c @@ -97,6 +97,31 @@ static struct vine_worker_info *get_best_dest_worker(struct vine_manager *q, str return best_destination; } +static int vine_temp_replicate_file_now(struct vine_manager *q, struct vine_file *f) +{ + if (!q || !f || f->type != VINE_TEMP) { + return 0; + } + + struct vine_worker_info *source_worker = get_best_source_worker(q, f); + if (!source_worker) { + return 0; + } + + struct vine_worker_info *dest_worker = get_best_dest_worker(q, f); + if (!dest_worker) { + return 0; + } + + vine_temp_start_peer_transfer(q, f, source_worker, dest_worker); + + return 1; +} + +/*************************************************************/ +/* Public Functions */ +/*************************************************************/ + void vine_temp_start_peer_transfer(struct vine_manager *q, struct vine_file *f, struct vine_worker_info *source_worker, struct vine_worker_info *dest_worker) { if (!q || !f || f->type != VINE_TEMP || !source_worker || !dest_worker) { @@ -108,23 +133,34 @@ void vine_temp_start_peer_transfer(struct vine_manager *q, struct vine_file *f, free(source_addr); } -int vine_temp_replicate_file_now(struct vine_manager *q, struct vine_file *f) +int vine_temp_replicate_file_later(struct vine_manager *q, struct vine_file *f) { - if (!q || !f || f->type != VINE_TEMP) { + if (!q || !f || f->type != VINE_TEMP || f->state != VINE_FILE_STATE_CREATED) { return 0; } - struct vine_worker_info *source_worker = get_best_source_worker(q, f); - if (!source_worker) { + int current_replica_count = vine_file_replica_count(q, f); + if (current_replica_count == 0 || current_replica_count >= q->temp_replica_count) { return 0; } - struct vine_worker_info *dest_worker = get_best_dest_worker(q, f); - if (!dest_worker) { + priority_queue_push(q->temp_files_to_replicate, f, -current_replica_count); + + return 1; +} + +int vine_temp_handle_file_lost(struct vine_manager *q, char *cachename) +{ + if (!q || !cachename) { return 0; } - vine_temp_start_peer_transfer(q, f, source_worker, dest_worker); + struct vine_file *f = hash_table_lookup(q->file_table, cachename); + if (!f || f->type != VINE_TEMP || f->state != VINE_FILE_STATE_CREATED) { + return 0; + } + + vine_temp_replicate_file_later(q, f); return 1; } @@ -176,40 +212,4 @@ int vine_temp_start_replication(struct vine_manager *q) list_delete(skipped); return processed; -} - -/*************************************************************/ -/* Public Functions */ -/*************************************************************/ - -int vine_temp_replicate_file_later(struct vine_manager *q, struct vine_file *f) -{ - if (!q || !f || f->type != VINE_TEMP || f->state != VINE_FILE_STATE_CREATED) { - return 0; - } - - int current_replica_count = vine_file_replica_count(q, f); - if (current_replica_count == 0 || current_replica_count >= q->temp_replica_count) { - return 0; - } - - priority_queue_push(q->temp_files_to_replicate, f, -current_replica_count); - - return 1; -} - -int vine_temp_handle_file_lost(struct vine_manager *q, char *cachename) -{ - if (!q || !cachename) { - return 0; - } - - struct vine_file *f = hash_table_lookup(q->file_table, cachename); - if (!f || f->type != VINE_TEMP || f->state != VINE_FILE_STATE_CREATED) { - return 0; - } - - vine_temp_replicate_file_later(q, f); - - return 1; -} +} \ No newline at end of file diff --git a/taskvine/src/manager/vine_temp.h b/taskvine/src/manager/vine_temp.h index ff2efb5011..da710d4f1e 100644 --- a/taskvine/src/manager/vine_temp.h +++ b/taskvine/src/manager/vine_temp.h @@ -3,7 +3,6 @@ #include "vine_manager.h" -int vine_temp_replicate_file_now(struct vine_manager *q, struct vine_file *f); int vine_temp_replicate_file_later(struct vine_manager *q, struct vine_file *f); int vine_temp_handle_file_lost(struct vine_manager *q, char *cachename); int vine_temp_start_replication(struct vine_manager *q); From 86cbc133cd7d190a71eabb7166734a1e52d53dcd Mon Sep 17 00:00:00 2001 From: Jin Zhou Date: Thu, 4 Dec 2025 11:12:29 -0500 Subject: [PATCH 090/113] fix --- poncho/src/poncho/library_network_code.py | 7 +++---- 1 file changed, 3 insertions(+), 4 deletions(-) diff --git a/poncho/src/poncho/library_network_code.py b/poncho/src/poncho/library_network_code.py index 5fab4b0b85..539c7464a5 100755 --- a/poncho/src/poncho/library_network_code.py +++ b/poncho/src/poncho/library_network_code.py @@ -81,10 +81,8 @@ def remote_wrapper(event): # Handler to sigchld when child exits. def sigchld_handler(signum, frame): # write any byte to signal that there's at least 1 child - try: - os.write(w, b"a") - except OSError: - pass + # if this fails, the notification mechanism has failed and the error will be raised normally + os.write(w, b"a") # Read data from worker, start function, and dump result to `outfile`. @@ -417,6 +415,7 @@ def main(): ) else: pid, func_id = start_function(in_pipe_fd, thread_limit) + # pid == -1 indicates a failure during fork/setup of the function execution if pid == -1: send_result( out_pipe_fd, From 67b29a529371981affe0e6e83c38471b7fc46b02 Mon Sep 17 00:00:00 2001 From: Jin Zhou Date: Mon, 5 Jan 2026 13:06:40 -0500 Subject: [PATCH 091/113] fix comments --- poncho/src/poncho/library_network_code.py | 14 ++++++++------ 1 file changed, 8 insertions(+), 6 deletions(-) diff --git a/poncho/src/poncho/library_network_code.py b/poncho/src/poncho/library_network_code.py index 539c7464a5..44dc47c566 100755 --- a/poncho/src/poncho/library_network_code.py +++ b/poncho/src/poncho/library_network_code.py @@ -156,7 +156,7 @@ def start_function(in_pipe_fd, thread_limit=1): finally: os.chdir(library_sandbox) return -1, function_id - else: + elif exec_method == "fork": try: arg_infile = os.path.join(function_sandbox, "infile") with open(arg_infile, "rb") as f: @@ -180,11 +180,10 @@ def start_function(in_pipe_fd, thread_limit=1): os.dup2(f.fileno(), sys.stdout.fileno()) # redirect stdout os.dup2(f.fileno(), sys.stderr.fileno()) # redirect stderr - # keep the function invocation inside the context - # so the stdout/stderr file stays open during execution and closes safely afterward. - stdout_timed_message(f"TASK {function_id} {function_name} starts in PID {os.getpid()}") - result = globals()[function_name](event) - stdout_timed_message(f"TASK {function_id} {function_name} finished") + # once the file descriptors are redirected, we can start the function execution + stdout_timed_message(f"TASK {function_id} {function_name} starts in PID {os.getpid()}") + result = globals()[function_name](event) + stdout_timed_message(f"TASK {function_id} {function_name} finished") except Exception: stdout_timed_message(f"TASK {function_id} error: can't execute {function_name} due to {traceback.format_exc()}") @@ -223,6 +222,9 @@ def start_function(in_pipe_fd, thread_limit=1): # return pid and function id of child process to parent. else: return p, function_id + else: + stdout_timed_message(f"error: invalid execution method {exec_method}") + return -1, function_id # Send result of a function execution to worker. Wake worker up to do work with SIGCHLD. From 40a09295740abb85272b7fce96099dd4c6fc0147 Mon Sep 17 00:00:00 2001 From: Jin Zhou Date: Mon, 5 Jan 2026 14:30:30 -0500 Subject: [PATCH 092/113] merge --- .../src/manager/vine_file_replica_table.c | 114 ------------------ .../src/manager/vine_file_replica_table.h | 4 - taskvine/src/manager/vine_manager.c | 14 +-- taskvine/src/manager/vine_temp.c | 37 ++++++ taskvine/src/manager/vine_temp.h | 11 +- 5 files changed, 49 insertions(+), 131 deletions(-) diff --git a/taskvine/src/manager/vine_file_replica_table.c b/taskvine/src/manager/vine_file_replica_table.c index d6ba403f5b..1dad20dd62 100644 --- a/taskvine/src/manager/vine_file_replica_table.c +++ b/taskvine/src/manager/vine_file_replica_table.c @@ -147,94 +147,6 @@ struct vine_worker_info *vine_file_replica_table_find_worker(struct vine_manager return peer_selected; } -// trigger replications of file to satisfy temp_replica_count -int vine_file_replica_table_replicate(struct vine_manager *m, struct vine_file *f, struct set *source_workers, int to_find) -{ - if (!f || !source_workers) { - return 0; - } - - int nsource_workers = set_size(source_workers); - int round_replication_request_sent = 0; - - /* get the elements of set so we can insert new replicas to sources */ - struct vine_worker_info **source_workers_frozen = (struct vine_worker_info **)set_values(source_workers); - struct vine_worker_info *source_worker; - - for (int i = 0; i < nsource_workers; i++) { - - source_worker = source_workers_frozen[i]; - int dest_workers_found = 0; - - // skip if the file on the source is not ready to transfer - struct vine_file_replica *replica = hash_table_lookup(source_worker->current_files, f->cached_name); - if (!replica || replica->state != VINE_FILE_REPLICA_STATE_READY) { - continue; - } - - // skip if the source is busy with other transfers - if (source_worker->outgoing_xfer_counter >= m->worker_source_max_transfers) { - continue; - } - - char *source_addr = string_format("%s/%s", source_worker->transfer_url, f->cached_name); - - char *id; - struct vine_worker_info *dest_worker; - int offset_bookkeep; - - HASH_TABLE_ITERATE_RANDOM_START(m->worker_table, offset_bookkeep, id, dest_worker) - { - // skip if the source and destination are on the same host - if (set_lookup(source_workers, dest_worker) || strcmp(source_worker->hostname, dest_worker->hostname) == 0) { - continue; - } - - // skip if the destination is not ready to transfer - if (!dest_worker->transfer_port_active) { - continue; - } - - // skip if the destination is draining - if (dest_worker->draining) { - continue; - } - - // skip if the destination is busy with other transfers - if (dest_worker->incoming_xfer_counter >= m->worker_source_max_transfers) { - continue; - } - - debug(D_VINE, "replicating %s from %s to %s", f->cached_name, source_worker->addrport, dest_worker->addrport); - - vine_manager_put_url_now(m, dest_worker, source_worker, source_addr, f); - - round_replication_request_sent++; - - // break if we have found enough destinations for this source - if (++dest_workers_found >= MIN(m->file_source_max_transfers, to_find)) { - break; - } - - // break if the source becomes busy with transfers - if (source_worker->outgoing_xfer_counter >= m->worker_source_max_transfers) { - break; - } - } - - free(source_addr); - - // break if we have sent enough replication requests for this file - if (round_replication_request_sent >= to_find) { - break; - } - } - - free(source_workers_frozen); - - return round_replication_request_sent; -} - /* Count number of replicas of a file in the system. */ @@ -258,32 +170,6 @@ int vine_file_replica_table_count_replicas(struct vine_manager *q, const char *c return count; } -/* -Check if a file replica exists on a worker. We accept both CREATING and READY replicas, -since a CREATING replica may already exist physically but hasn't yet received the cache-update -message from the manager. However, we do not accept DELETING replicas, as they indicate -the source worker has already been sent an unlink request—any subsequent cache-update or -cache-invalid events will lead to deletion. -*/ -int vine_file_replica_table_exists_somewhere(struct vine_manager *q, const char *cachename) -{ - struct set *workers = hash_table_lookup(q->file_worker_table, cachename); - if (!workers || set_size(workers) < 1) { - return 0; - } - - struct vine_worker_info *w; - SET_ITERATE(workers, w) - { - struct vine_file_replica *r = vine_file_replica_table_lookup(w, cachename); - if (r && (r->state == VINE_FILE_REPLICA_STATE_CREATING || r->state == VINE_FILE_REPLICA_STATE_READY)) { - return 1; - } - } - - return 0; -} - // get or create a replica for a worker and cachename struct vine_file_replica *vine_file_replica_table_get_or_create(struct vine_manager *m, struct vine_worker_info *w, const char *cachename, vine_file_type_t type, vine_cache_level_t cache_level, int64_t size, time_t mtime) { diff --git a/taskvine/src/manager/vine_file_replica_table.h b/taskvine/src/manager/vine_file_replica_table.h index c4d18b4099..cba24c0133 100644 --- a/taskvine/src/manager/vine_file_replica_table.h +++ b/taskvine/src/manager/vine_file_replica_table.h @@ -30,10 +30,6 @@ struct vine_file_replica *vine_file_replica_table_get_or_create(struct vine_mana struct vine_worker_info *vine_file_replica_table_find_worker(struct vine_manager *q, const char *cachename); -int vine_file_replica_table_replicate(struct vine_manager *q, struct vine_file *f, struct set *source_workers, int to_find); - -int vine_file_replica_table_exists_somewhere( struct vine_manager *q, const char *cachename ); - int vine_file_replica_table_count_replicas( struct vine_manager *q, const char *cachename, vine_file_replica_state_t state ); #endif \ No newline at end of file diff --git a/taskvine/src/manager/vine_manager.c b/taskvine/src/manager/vine_manager.c index 040b797e8e..42192f5cb1 100644 --- a/taskvine/src/manager/vine_manager.c +++ b/taskvine/src/manager/vine_manager.c @@ -43,6 +43,7 @@ See the file COPYING for details. #include "envtools.h" #include "hash_table.h" #include "skip_list.h" +#include "priority_queue.h" #include "int_sizes.h" #include "interfaces_address.h" #include "itable.h" @@ -923,15 +924,6 @@ void vine_update_catalog(struct vine_manager *m) } } -static int file_needs_recovery(struct vine_manager *q, struct vine_file *f) -{ - if (!f || f->type != VINE_TEMP || f->state != VINE_FILE_STATE_CREATED) { - return 0; - } - - return !vine_file_replica_table_exists_somewhere(q, f->cached_name); -} - static void cleanup_worker_files(struct vine_manager *q, struct vine_worker_info *w) { if (!q || !w || hash_table_size(w->current_files) < 1) { @@ -960,7 +952,7 @@ static void cleanup_worker_files(struct vine_manager *q, struct vine_worker_info vine_file_replica_delete(removed_replica); } /* consider if this replica needs recovery because of worker removal */ - if (q->immediate_recovery && file_needs_recovery(q, f)) { + if (q->immediate_recovery && f && f->type == VINE_TEMP && !vine_temp_exists_somewhere(q, f)) { vine_manager_consider_recovery_task(q, f, f->recovery_task); } } @@ -3475,7 +3467,7 @@ static int vine_manager_check_inputs_available(struct vine_manager *q, struct vi struct vine_file *f = m->file; if (f->type == VINE_FILE && f->state == VINE_FILE_STATE_PENDING) { all_available = 0; - } else if (file_needs_recovery(q, f)) { + } else if (f->type == VINE_TEMP && !vine_temp_exists_somewhere(q, f)) { vine_manager_consider_recovery_task(q, f, f->recovery_task); all_available = 0; } diff --git a/taskvine/src/manager/vine_temp.c b/taskvine/src/manager/vine_temp.c index cae4b183ee..269d47c8e0 100644 --- a/taskvine/src/manager/vine_temp.c +++ b/taskvine/src/manager/vine_temp.c @@ -42,6 +42,7 @@ static int worker_can_peer_transfer(struct vine_worker_info *w) if (w->resources->tag < 0) { return 0; } + return 1; } @@ -203,6 +204,42 @@ static int attempt_replication(struct vine_manager *q, struct vine_file *f) /* Public Functions */ /*************************************************************/ +/** +Check if a temporary file exists somewhere in all workers. +Returns 1 if at least one CREATING or READY replica exists, 0 otherwise. + +We accept both CREATING and READY replicas as available sources, since a CREATING +replica may already exist physically but hasn't yet received the cache-update +message from the manager. However, we do not accept DELETING replicas, as they +indicate the source worker has already been sent an unlink request—any subsequent +cache-update or cache-invalid events will lead to deletion. + +If the file's state is not CREATED, we need to wait for the producer task to +complete before checking for replicas. +*/ +int vine_temp_exists_somewhere(struct vine_manager *q, struct vine_file *f) +{ + if (!q || !f || f->type != VINE_TEMP || f->state != VINE_FILE_STATE_CREATED) { + return 0; + } + + struct set *workers = hash_table_lookup(q->file_worker_table, f->cached_name); + if (!workers || set_size(workers) < 1) { + return 0; + } + + struct vine_worker_info *w; + SET_ITERATE(workers, w) + { + struct vine_file_replica *replica = vine_file_replica_table_lookup(w, f->cached_name); + if (replica && (replica->state == VINE_FILE_REPLICA_STATE_CREATING || replica->state == VINE_FILE_REPLICA_STATE_READY)) { + return 1; + } + } + + return 0; +} + /** Queue a temporary file for replication when it still lacks the target number of replicas. Files without any replica and those already satisfying the quota are diff --git a/taskvine/src/manager/vine_temp.h b/taskvine/src/manager/vine_temp.h index de25f26a0f..d94c05d372 100644 --- a/taskvine/src/manager/vine_temp.h +++ b/taskvine/src/manager/vine_temp.h @@ -3,12 +3,19 @@ #include "vine_manager.h" -/** Replication related functions */ +/** Check if a temporary file exists somewhere in all workers. */ +int vine_temp_exists_somewhere(struct vine_manager *q, struct vine_file *f); + +/** Queue a temporary file for replication. */ int vine_temp_queue_for_replication(struct vine_manager *q, struct vine_file *f); + +/** Start replication for temporary files. */ int vine_temp_start_replication(struct vine_manager *q); -/** Storage management functions */ +/** Clean redundant replicas of a temporary file. */ void vine_temp_clean_redundant_replicas(struct vine_manager *q, struct vine_file *f); + +/** Shift a temp file replica away from the worker using the most cache space. */ void vine_temp_shift_disk_load(struct vine_manager *q, struct vine_worker_info *source_worker, struct vine_file *f); #endif \ No newline at end of file From b1e70df4ce1f5d9b1e797bd0a698e0e3324305b1 Mon Sep 17 00:00:00 2001 From: Jin Zhou Date: Mon, 5 Jan 2026 14:36:39 -0500 Subject: [PATCH 093/113] clean --- taskvine/src/manager/vine_manager.c | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/taskvine/src/manager/vine_manager.c b/taskvine/src/manager/vine_manager.c index 42192f5cb1..26ece01d74 100644 --- a/taskvine/src/manager/vine_manager.c +++ b/taskvine/src/manager/vine_manager.c @@ -952,7 +952,7 @@ static void cleanup_worker_files(struct vine_manager *q, struct vine_worker_info vine_file_replica_delete(removed_replica); } /* consider if this replica needs recovery because of worker removal */ - if (q->immediate_recovery && f && f->type == VINE_TEMP && !vine_temp_exists_somewhere(q, f)) { + if (q->immediate_recovery && f->type == VINE_TEMP && !vine_temp_exists_somewhere(q, f)) { vine_manager_consider_recovery_task(q, f, f->recovery_task); } } From 058466ea102032ca55d96ee39a9ef8ba535a1efa Mon Sep 17 00:00:00 2001 From: Jin Zhou Date: Mon, 5 Jan 2026 15:32:46 -0500 Subject: [PATCH 094/113] update --- .../src/graph/dagvine/vine_graph/vine_graph.c | 6 +- taskvine/src/manager/vine_manager.c | 80 +++---------------- taskvine/src/manager/vine_schedule.c | 5 -- 3 files changed, 12 insertions(+), 79 deletions(-) diff --git a/taskvine/src/graph/dagvine/vine_graph/vine_graph.c b/taskvine/src/graph/dagvine/vine_graph/vine_graph.c index 3ef646c768..73d1298fee 100644 --- a/taskvine/src/graph/dagvine/vine_graph/vine_graph.c +++ b/taskvine/src/graph/dagvine/vine_graph/vine_graph.c @@ -1275,8 +1275,8 @@ void vine_graph_execute(struct vine_graph *vg) /* inject failure */ if (vg->failure_injection_step_percent > 0) { double progress = (double)regular_tasks_part->current / (double)regular_tasks_part->total; - if (progress >= next_failure_threshold && evict_random_worker(vg->manager)) { - debug(D_VINE, "evicted a worker at %.2f%% (threshold %.2f%%)", progress * 100, next_failure_threshold * 100); + if (progress >= next_failure_threshold && release_random_worker(vg->manager)) { + debug(D_VINE, "released a random worker at %.2f%% (threshold %.2f%%)", progress * 100, next_failure_threshold * 100); next_failure_threshold += vg->failure_injection_step_percent / 100.0; } } @@ -1285,7 +1285,7 @@ void vine_graph_execute(struct vine_graph *vg) switch (node->outfile_type) { case NODE_OUTFILE_TYPE_TEMP: /* replicate the outfile of the temp node */ - vine_temp_replicate_file_later(vg->manager, node->outfile); + vine_temp_queue_for_replication(vg->manager, node->outfile); break; case NODE_OUTFILE_TYPE_LOCAL: case NODE_OUTFILE_TYPE_SHARED_FILE_SYSTEM: diff --git a/taskvine/src/manager/vine_manager.c b/taskvine/src/manager/vine_manager.c index afbc6d05d5..18e05326a5 100644 --- a/taskvine/src/manager/vine_manager.c +++ b/taskvine/src/manager/vine_manager.c @@ -173,8 +173,6 @@ static int release_worker(struct vine_manager *q, struct vine_worker_info *w); struct vine_task *send_library_to_worker(struct vine_manager *q, struct vine_worker_info *w, const char *name); static void push_task_to_ready_tasks(struct vine_manager *q, struct vine_task *t); -static void clean_redundant_replicas(struct vine_manager *q, struct vine_file *f); - /* Return the number of workers matching a given type: WORKER, STATUS, etc */ static int count_workers(struct vine_manager *q, vine_worker_type_t type) @@ -3533,58 +3531,6 @@ int consider_task(struct vine_manager *q, struct vine_task *t) return 1; } -/* Rotate pending tasks to the ready queue if they are runnable. */ -static int rotate_pending_tasks(struct vine_manager *q) -{ - if (list_size(q->pending_tasks) == 0) { - return 0; - } - - int runnable_tasks = 0; - int tasks_considered = 0; - int tasks_to_consider = MIN(list_size(q->pending_tasks), q->attempt_schedule_depth); - struct vine_task *t = NULL; - - double current_time = timestamp_get() / ONE_SECOND; - - while (tasks_considered++ < tasks_to_consider) { - t = list_pop_head(q->pending_tasks); - if (!t) { - break; - } - - /* first check if the task has exceeded its end time or does not match any submitted library */ - /* If any of the reasons fired, then expire the task and put in the retrieved queue. */ - if (t->resources_requested->end > 0 && t->resources_requested->end <= current_time) { - debug(D_VINE, "task %d has exceeded its end time", t->task_id); - vine_task_set_result(t, VINE_RESULT_MAX_END_TIME); - change_task_state(q, t, VINE_TASK_RETRIEVED); - continue; - } - if (t->needs_library && !hash_table_lookup(q->library_templates, t->needs_library)) { - debug(D_VINE, "task %d does not match any submitted library named \"%s\"", t->task_id, t->needs_library); - vine_task_set_result(t, VINE_RESULT_MISSING_LIBRARY); - change_task_state(q, t, VINE_TASK_RETRIEVED); - continue; - } - if (q->fixed_location_in_queue && t->has_fixed_locations && !vine_schedule_check_fixed_location(q, t)) { - debug(D_VINE, "Missing fixed_location dependencies for task: %d", t->task_id); - vine_task_set_result(t, VINE_RESULT_FIXED_LOCATION_MISSING); - change_task_state(q, t, VINE_TASK_RETRIEVED); - continue; - } - - if (consider_task(q, t)) { - push_task_to_ready_tasks(q, t); - runnable_tasks++; - } else { - list_push_tail(q->pending_tasks, t); - } - } - - return runnable_tasks; -} - /* Advance the state of the system by selecting one task available to run, finding the best worker for that task, and then committing @@ -3616,25 +3562,24 @@ static int send_one_task_with_cr(struct vine_manager *q, struct skip_list_cursor continue; } - /* this task is not runnable at all, put it back in the pending queue */ if (!consider_task(q, t)) { - list_push_tail(q->pending_tasks, t); continue; } - /* select a worker for the task */ + // Find the best worker for the task + q->stats_measure->time_scheduling = timestamp_get(); struct vine_worker_info *w = vine_schedule_task_to_worker(q, t); + q->stats->time_scheduling += timestamp_get() - q->stats_measure->time_scheduling; if (w) { task_unready = 1; - /* commit the task to the worker */ - vine_result_code_t result; - if (q->task_groups_enabled) { - result = commit_task_group_to_worker(q, w, t); - } else { - result = commit_task_to_worker(q, w, t); - } + vine_result_code_t result; + if (q->task_groups_enabled) { + result = commit_task_group_to_worker(q, w, t); + } else { + result = commit_task_to_worker(q, w, t); + } switch (result) { case VINE_SUCCESS: /* return on successful commit. */ @@ -4676,7 +4621,6 @@ static void push_task_to_ready_tasks(struct vine_manager *q, struct vine_task *t } else if (t->result == VINE_RESULT_RESOURCE_EXHAUSTION) { manager_priority = VINE_PRIORITY_EXHAUSTION; } - priority_queue_push(q->ready_tasks, t, t->priority); if (manager_priority > t->manager_priority) { t->manager_priority = manager_priority; @@ -5425,12 +5369,6 @@ static struct vine_task *vine_wait_internal(struct vine_manager *q, int timeout, } } - // Check if any worker is overloaded and rebalance the disk usage - if (q->balance_worker_disk_load && (timestamp_get() - q->when_last_offloaded > 5 * 1e6)) { - rebalance_worker_disk_usage(q); - q->when_last_offloaded = timestamp_get(); - } - // Check if any temp files need replication and start replicating BEGIN_ACCUM_TIME(q, time_internal); result = vine_temp_start_replication(q); diff --git a/taskvine/src/manager/vine_schedule.c b/taskvine/src/manager/vine_schedule.c index e383033302..284be290c1 100644 --- a/taskvine/src/manager/vine_schedule.c +++ b/taskvine/src/manager/vine_schedule.c @@ -607,10 +607,6 @@ void vine_schedule_check_for_large_tasks(struct vine_manager *q) struct rmsummary *largest_unfit_task = rmsummary_create(-1); -<<<<<<< HEAD - LIST_ITERATE(q->pending_tasks, t) - { -======= struct skip_list_cursor *cur = skip_list_cursor_create(q->ready_tasks); skip_list_seek(cur, 0); while (skip_list_get(cur, (void **)&t)) { @@ -618,7 +614,6 @@ void vine_schedule_check_for_large_tasks(struct vine_manager *q) // in case there are continues, etc. below. skip_list_next(cur); ->>>>>>> origin/master // check each task against the queue of connected workers vine_resource_bitmask_t bit_set = is_task_larger_than_any_worker(q, t); if (bit_set) { From da8aa185328df419340375cea2c6cbf74b4ee139 Mon Sep 17 00:00:00 2001 From: Jin Zhou Date: Mon, 5 Jan 2026 15:36:15 -0500 Subject: [PATCH 095/113] rename recovery_tasks_submitted --- taskvine/src/graph/dagvine/vine_graph/vine_graph.c | 2 +- taskvine/src/manager/vine_manager.c | 1 - taskvine/src/manager/vine_manager.h | 1 - 3 files changed, 1 insertion(+), 3 deletions(-) diff --git a/taskvine/src/graph/dagvine/vine_graph/vine_graph.c b/taskvine/src/graph/dagvine/vine_graph/vine_graph.c index 73d1298fee..66b0655a92 100644 --- a/taskvine/src/graph/dagvine/vine_graph/vine_graph.c +++ b/taskvine/src/graph/dagvine/vine_graph/vine_graph.c @@ -1190,7 +1190,7 @@ void vine_graph_execute(struct vine_graph *vg) } struct vine_task *task = vine_wait(vg->manager, wait_timeout); - progress_bar_set_part_total(pbar, recovery_tasks_part, vg->manager->num_submitted_recovery_tasks); + progress_bar_set_part_total(pbar, recovery_tasks_part, vg->manager->stats->recovery_tasks_submitted); if (task) { /* retrieve all possible tasks */ wait_timeout = 0; diff --git a/taskvine/src/manager/vine_manager.c b/taskvine/src/manager/vine_manager.c index 18e05326a5..e8a933b5e6 100644 --- a/taskvine/src/manager/vine_manager.c +++ b/taskvine/src/manager/vine_manager.c @@ -4166,7 +4166,6 @@ struct vine_manager *vine_ssl_create(int port, const char *key, const char *cert q->time_start_worker_eviction = 0; q->return_recovery_tasks = 0; - q->num_submitted_recovery_tasks = 0; q->balance_worker_disk_load = 0; q->when_last_offloaded = 0; q->peak_used_cache = 0; diff --git a/taskvine/src/manager/vine_manager.h b/taskvine/src/manager/vine_manager.h index ec8bdaad4c..5019c9a99b 100644 --- a/taskvine/src/manager/vine_manager.h +++ b/taskvine/src/manager/vine_manager.h @@ -237,7 +237,6 @@ struct vine_manager { double disk_proportion_available_to_task; /* intentionally reduces disk allocation for tasks to reserve some space for cache growth. */ int return_recovery_tasks; /* If true, recovery tasks are returned by vine_wait to the user. By default they are handled internally. */ - int num_submitted_recovery_tasks; int balance_worker_disk_load; /* If true, offload replicas from workers that are overloaded with temp files. */ timestamp_t when_last_offloaded; int64_t peak_used_cache; From 39ae787067b6e736f908d92d2ef770300ade52b0 Mon Sep 17 00:00:00 2001 From: Jin Zhou Date: Mon, 5 Jan 2026 15:41:24 -0500 Subject: [PATCH 096/113] timestamp format --- taskvine/src/graph/dagvine/vine_graph/vine_graph.c | 3 ++- taskvine/src/graph/dagvine/vine_graph/vine_graph.h | 14 +++++++------- taskvine/src/graph/dagvine/vine_graph/vine_node.h | 6 +++--- 3 files changed, 12 insertions(+), 11 deletions(-) diff --git a/taskvine/src/graph/dagvine/vine_graph/vine_graph.c b/taskvine/src/graph/dagvine/vine_graph/vine_graph.c index 66b0655a92..4f7ada468b 100644 --- a/taskvine/src/graph/dagvine/vine_graph/vine_graph.c +++ b/taskvine/src/graph/dagvine/vine_graph/vine_graph.c @@ -544,7 +544,7 @@ static void print_time_metrics(struct vine_graph *vg, const char *filename) struct vine_node *node; ITABLE_ITERATE(vg->nodes, nid, node) { - fprintf(fp, "%" PRIu64 ",%lu,%lu,%lu,%lu,%lu,%lu\n", node->node_id, node->submission_time, node->scheduling_time, node->commit_time, node->execution_time, node->retrieval_time, node->postprocessing_time); + fprintf(fp, "%" PRIu64 "," TIMESTAMP_FORMAT "," TIMESTAMP_FORMAT "," TIMESTAMP_FORMAT "," TIMESTAMP_FORMAT "," TIMESTAMP_FORMAT "," TIMESTAMP_FORMAT "\n", node->node_id, node->submission_time, node->scheduling_time, node->commit_time, node->execution_time, node->retrieval_time, node->postprocessing_time); } fclose(fp); @@ -1013,6 +1013,7 @@ void vine_graph_set_target(struct vine_graph *vg, uint64_t node_id) debug(D_ERROR, "node %" PRIu64 " not found", node_id); exit(1); } + node->is_target = 1; } diff --git a/taskvine/src/graph/dagvine/vine_graph/vine_graph.h b/taskvine/src/graph/dagvine/vine_graph/vine_graph.h index e3f78f5c83..0d9edb2a9b 100644 --- a/taskvine/src/graph/dagvine/vine_graph/vine_graph.h +++ b/taskvine/src/graph/dagvine/vine_graph/vine_graph.h @@ -14,13 +14,13 @@ /** The task priority algorithm used for vine graph scheduling. */ typedef enum { - TASK_PRIORITY_MODE_RANDOM = 0, /**< Assign random priority to tasks */ - TASK_PRIORITY_MODE_DEPTH_FIRST, /**< Prioritize deeper tasks first */ - TASK_PRIORITY_MODE_BREADTH_FIRST, /**< Prioritize shallower tasks first */ - TASK_PRIORITY_MODE_FIFO, /**< First in, first out priority */ - TASK_PRIORITY_MODE_LIFO, /**< Last in, first out priority */ - TASK_PRIORITY_MODE_LARGEST_INPUT_FIRST, /**< Prioritize tasks with larger inputs first */ - TASK_PRIORITY_MODE_LARGEST_STORAGE_FOOTPRINT_FIRST /**< Prioritize tasks with larger storage footprint first */ + TASK_PRIORITY_MODE_RANDOM = 0, /**< Assign random priority to tasks */ + TASK_PRIORITY_MODE_DEPTH_FIRST, /**< Prioritize deeper tasks first */ + TASK_PRIORITY_MODE_BREADTH_FIRST, /**< Prioritize shallower tasks first */ + TASK_PRIORITY_MODE_FIFO, /**< First in, first out priority */ + TASK_PRIORITY_MODE_LIFO, /**< Last in, first out priority */ + TASK_PRIORITY_MODE_LARGEST_INPUT_FIRST, /**< Prioritize tasks with larger inputs first */ + TASK_PRIORITY_MODE_LARGEST_STORAGE_FOOTPRINT_FIRST /**< Prioritize tasks with larger storage footprint first */ } task_priority_mode_t; /** The vine graph (logical scheduling layer). */ diff --git a/taskvine/src/graph/dagvine/vine_graph/vine_node.h b/taskvine/src/graph/dagvine/vine_graph/vine_node.h index 9f01e959c0..a838fb6e88 100644 --- a/taskvine/src/graph/dagvine/vine_graph/vine_node.h +++ b/taskvine/src/graph/dagvine/vine_graph/vine_node.h @@ -11,9 +11,9 @@ /** The storage type of the node's output file. */ typedef enum { - NODE_OUTFILE_TYPE_LOCAL = 0, /* Node-output file will be stored locally on the local staging directory */ - NODE_OUTFILE_TYPE_TEMP, /* Node-output file will be stored in the temporary node-local storage */ - NODE_OUTFILE_TYPE_SHARED_FILE_SYSTEM, /* Node-output file will be stored in the persistent shared file system */ + NODE_OUTFILE_TYPE_LOCAL = 0, /* Node-output file will be stored locally on the local staging directory */ + NODE_OUTFILE_TYPE_TEMP, /* Node-output file will be stored in the temporary node-local storage */ + NODE_OUTFILE_TYPE_SHARED_FILE_SYSTEM, /* Node-output file will be stored in the persistent shared file system */ } node_outfile_type_t; /** The status of an output file of a node. */ From 1b0cdf1aad728dba178e05633ca56a88b03da714 Mon Sep 17 00:00:00 2001 From: Jin Zhou Date: Mon, 5 Jan 2026 23:10:16 -0500 Subject: [PATCH 097/113] vine: check whether hash table key is present before inserting --- dttools/src/hash_table.c | 2 +- taskvine/src/manager/vine_manager.c | 4 +++- 2 files changed, 4 insertions(+), 2 deletions(-) diff --git a/dttools/src/hash_table.c b/dttools/src/hash_table.c index 4c298ca3b5..5f69f7d87d 100644 --- a/dttools/src/hash_table.c +++ b/dttools/src/hash_table.c @@ -275,7 +275,7 @@ int hash_table_insert(struct hash_table *h, const char *key, const void *value) h->cant_iterate_yet = 1; } else { /* Key already exists, free the unused entry */ - notice(D_DEBUG, "key % already exists in hash table, ignoring new value.", key); + notice(D_DEBUG, "key %s already exists in hash table, ignoring new value.", key); free(e->key); free(e); } diff --git a/taskvine/src/manager/vine_manager.c b/taskvine/src/manager/vine_manager.c index f385fbab06..09694974ad 100644 --- a/taskvine/src/manager/vine_manager.c +++ b/taskvine/src/manager/vine_manager.c @@ -704,7 +704,9 @@ static vine_msg_code_t vine_manager_recv_no_retry(struct vine_manager *q, struct string_prefix_is(line, "wable_status") || string_prefix_is(line, "resources_status")) { result = handle_manager_status(q, w, line, stoptime); } else if (string_prefix_is(line, "available_results")) { - hash_table_insert(q->workers_with_watched_file_updates, w->hashkey, w); + if (!hash_table_lookup(q->workers_with_watched_file_updates, w->hashkey)) { + hash_table_insert(q->workers_with_watched_file_updates, w->hashkey, w); + } result = VINE_MSG_PROCESSED; } else if (string_prefix_is(line, "resources")) { result = handle_resources(q, w, stoptime); From ff7eeb229fcbf4c1662d736bd841263647d552f2 Mon Sep 17 00:00:00 2001 From: Jin Zhou Date: Mon, 5 Jan 2026 23:12:27 -0500 Subject: [PATCH 098/113] fix hash table insertion --- dttools/src/hash_table.c | 2 +- taskvine/src/manager/vine_manager.c | 6 +++--- 2 files changed, 4 insertions(+), 4 deletions(-) diff --git a/dttools/src/hash_table.c b/dttools/src/hash_table.c index 4c298ca3b5..5f69f7d87d 100644 --- a/dttools/src/hash_table.c +++ b/dttools/src/hash_table.c @@ -275,7 +275,7 @@ int hash_table_insert(struct hash_table *h, const char *key, const void *value) h->cant_iterate_yet = 1; } else { /* Key already exists, free the unused entry */ - notice(D_DEBUG, "key % already exists in hash table, ignoring new value.", key); + notice(D_DEBUG, "key %s already exists in hash table, ignoring new value.", key); free(e->key); free(e); } diff --git a/taskvine/src/manager/vine_manager.c b/taskvine/src/manager/vine_manager.c index e8a933b5e6..768d356daa 100644 --- a/taskvine/src/manager/vine_manager.c +++ b/taskvine/src/manager/vine_manager.c @@ -148,7 +148,6 @@ static vine_msg_code_t handle_manager_status(struct vine_manager *q, struct vine static vine_msg_code_t handle_resources(struct vine_manager *q, struct vine_worker_info *w, time_t stoptime); static vine_msg_code_t handle_feature(struct vine_manager *q, struct vine_worker_info *w, const char *line); static void handle_library_update(struct vine_manager *q, struct vine_worker_info *w, const char *line); -static int receive_tasks_from_worker(struct vine_manager *q, struct vine_worker_info *w, int count_received_so_far); static struct jx *manager_to_jx(struct vine_manager *q); static struct jx *manager_lean_to_jx(struct vine_manager *q); @@ -700,7 +699,6 @@ static vine_msg_code_t handle_complete(struct vine_manager *q, struct vine_worke { vine_result_code_t result = get_completion_result(q, w, line); if (result == VINE_SUCCESS) { - receive_tasks_from_worker(q, w, 0); return VINE_MSG_PROCESSED; } return VINE_MSG_NOT_PROCESSED; @@ -737,7 +735,9 @@ static vine_msg_code_t vine_manager_recv_no_retry(struct vine_manager *q, struct string_prefix_is(line, "wable_status") || string_prefix_is(line, "resources_status")) { result = handle_manager_status(q, w, line, stoptime); } else if (string_prefix_is(line, "available_results")) { - hash_table_insert(q->workers_with_watched_file_updates, w->hashkey, w); + if (!hash_table_lookup(q->workers_with_watched_file_updates, w->hashkey)) { + hash_table_insert(q->workers_with_watched_file_updates, w->hashkey, w); + } result = VINE_MSG_PROCESSED; } else if (string_prefix_is(line, "resources")) { result = handle_resources(q, w, stoptime); From fe470361630b6492d67381262c5fe7f555fdf833 Mon Sep 17 00:00:00 2001 From: Jin Zhou Date: Tue, 6 Jan 2026 14:57:33 -0500 Subject: [PATCH 099/113] Add example target to Makefile for running example_blueprint --- taskvine/src/graph/Makefile | 5 +- taskvine/src/graph/__init__.py | 10 + taskvine/src/graph/high_level.py | 332 +++++++++++++++++++++++++++++++ 3 files changed, 346 insertions(+), 1 deletion(-) create mode 100644 taskvine/src/graph/__init__.py create mode 100644 taskvine/src/graph/high_level.py diff --git a/taskvine/src/graph/Makefile b/taskvine/src/graph/Makefile index c56a9f8090..50314300bc 100644 --- a/taskvine/src/graph/Makefile +++ b/taskvine/src/graph/Makefile @@ -8,4 +8,7 @@ all clean install test lint format: $(SUBDIRS) $(SUBDIRS): %: $(MAKE) -C $@ $(MAKECMDGOALS) -.PHONY: all clean install test lint format $(SUBDIRS) +.PHONY: all clean install test lint format $(SUBDIRS) example + +example: + PYTHONPATH=../../../ python -m taskvine.src.graph.example_blueprint diff --git a/taskvine/src/graph/__init__.py b/taskvine/src/graph/__init__.py new file mode 100644 index 0000000000..0231025894 --- /dev/null +++ b/taskvine/src/graph/__init__.py @@ -0,0 +1,10 @@ +"""High-level blueprint graph interfaces.""" + +from .high_level import BlueprintFile, BlueprintGraph, ReturnRef + +__all__ = [ + "BlueprintFile", + "BlueprintGraph", + "ReturnRef", +] + diff --git a/taskvine/src/graph/high_level.py b/taskvine/src/graph/high_level.py new file mode 100644 index 0000000000..1a36dc8bb2 --- /dev/null +++ b/taskvine/src/graph/high_level.py @@ -0,0 +1,332 @@ +"""High-level blueprint graph for TaskVine DAG modelling.""" + +from collections import defaultdict +import uuid + + +class ReturnRef: + """Reference to the return value (or sub-path) of another task.""" + + def __init__(self, node, path=()): + if node is None: + raise ValueError("node must not be None") + self.node = node + self.path = tuple(path) + + def derive(self, *path): + return ReturnRef(self.node, self.path + tuple(path)) + + def __getitem__(self, item): + if isinstance(item, tuple): + return self.derive(*item) + return self.derive(item) + +class BlueprintFile: + """Describes a file that tasks may produce or consume.""" + + def __init__(self, name, **metadata): + if not name or not isinstance(name, str): + raise ValueError("File name must be a non-empty string") + self.name = name + self.metadata = dict(metadata) + +class BlueprintGraphTask: + """Represents a task node within the blueprint graph.""" + + def __init__(self, graph, func, args=None, kwargs=None, name=None): + self.graph = graph + self.func = func + self.name = name or graph._generate_task_name() + self.args = tuple(args) if args is not None else tuple() + self.kwargs = dict(kwargs) if kwargs is not None else {} + self.produced_files = [] + self.consumed_files = [] + self._return_ref = ReturnRef(self) + + def produces(self, *files): + for file_obj in files: + file_obj = self.graph._ensure_file(file_obj) + self.graph._register_file_producer(self, file_obj) + if file_obj not in self.produced_files: + self.produced_files.append(file_obj) + return self + + def consumes(self, *files): + for file_obj in files: + file_obj = self.graph._ensure_file(file_obj) + self.graph._register_file_consumer(self, file_obj) + if file_obj not in self.consumed_files: + self.consumed_files.append(file_obj) + return self + + def ret(self): + return self._return_ref + + def update_kwargs(self, **new_kwargs): + if not new_kwargs: + return self + merged = dict(self.kwargs) + merged.update(new_kwargs) + self.kwargs = merged + self.graph.invalidate() + return self + +class BlueprintGraph: + """Blueprint graph supporting ReturnRef and file dependencies.""" + + _SUPPORTED_SCALARS = (str, int, float, bool, type(None), bytes) + def __init__(self): + self._tasks = [] + self._tasks_by_name = {} + self._return_edges = defaultdict(set) + self._files = {} + self._file_producer = {} + self._file_consumers = defaultdict(set) + self._dirty = True + + def define_file(self, name, **metadata): + existing = self._files.get(name) + if existing: + existing.metadata.update(metadata) + return existing + file_obj = BlueprintFile(name, **metadata) + self._files[name] = file_obj + return file_obj + + def task(self, func, *args, name=None, **kwargs): + task = BlueprintGraphTask( + self, + func, + args=args, + kwargs=kwargs, + name=name, + ) + return self._register_task(task) + + def nodes(self): + return list(self._tasks) + + def file_links(self): + links = [] + for file_name, consumers in self._file_consumers.items(): + file_obj = self._files.get(file_name) + if file_obj is None: + continue + producer = self._file_producer.get(file_name) + for consumer in consumers: + links.append((producer, consumer, file_obj)) + return links + + def return_children(self, task): + task = self._ensure_task(task) + return set(self._return_edges.get(task, set())) + + def invalidate(self): + self._dirty = True + + def validate(self): + if self._dirty: + for task in self._tasks: + self._validate_supported_value(task.args, f"{task.name}.args") + self._validate_supported_value(task.kwargs, f"{task.name}.kwargs") + self._assert_no_returnref_in_keys(task.args, f"{task.name}.args") + self._assert_no_returnref_in_keys(task.kwargs, f"{task.name}.kwargs") + self._rebuild_return_edges() + self._dirty = False + + missing_producers = [ + file_name + for file_name, consumers in self._file_consumers.items() + if consumers and file_name not in self._file_producer + ] + if missing_producers: + raise ValueError( + f"Files consumed without producers: {', '.join(sorted(missing_producers))}" + ) + + def describe(self): + self.validate() + data = { + "tasks": {}, + "return_links": [], + "file_links": [], + "files": { + name: dict(file_obj.metadata) for name, file_obj in self._files.items() + }, + } + for task in self._tasks: + data["tasks"][task.name] = { + "func": getattr(task.func, "__name__", repr(task.func)), + "args": self._serialise(task.args), + "kwargs": self._serialise(task.kwargs), + "produces": [self._serialise_file(f) for f in task.produced_files], + "consumes": [self._serialise_file(f) for f in task.consumed_files], + } + for parent, children in self._return_edges.items(): + for child in children: + data["return_links"].append( + { + "parent": parent.name, + "child": child.name, + } + ) + for file_name, consumers in self._file_consumers.items(): + file_obj = self._files.get(file_name) + if file_obj is None: + continue + producer = self._file_producer.get(file_name) + for consumer in consumers: + data["file_links"].append( + { + "parent": producer.name if producer else None, + "child": consumer.name if consumer else None, + "filename": file_obj.name, + "metadata": dict(file_obj.metadata), + } + ) + return data + + def _register_task(self, task): + canonical = self._tasks_by_name.get(task.name) + if canonical is not None: + raise ValueError(f"Task name '{task.name}' already registered") + self._tasks.append(task) + self._tasks_by_name[task.name] = task + self.invalidate() + return task + + def _register_file_producer(self, task, file_obj): + task = self._ensure_task(task) + existing = self._file_producer.get(file_obj.name) + if existing and existing is not task: + raise ValueError( + f"File '{file_obj.name}' already produced by task '{existing.name}'" + ) + self._file_producer[file_obj.name] = task + self.invalidate() + + def _register_file_consumer(self, task, file_obj): + task = self._ensure_task(task) + self._file_consumers[file_obj.name].add(task) + self.invalidate() + + def _ensure_file(self, file_obj): + if isinstance(file_obj, BlueprintFile): + existing = self._files.get(file_obj.name) + if existing: + existing.metadata.update(file_obj.metadata) + return existing + self._files[file_obj.name] = file_obj + return file_obj + if isinstance(file_obj, str): + existing = self._files.get(file_obj) + if not existing: + existing = BlueprintFile(file_obj) + self._files[file_obj] = existing + return existing + raise TypeError("File reference must be BlueprintFile or string name") + + def _rebuild_return_edges(self): + self._return_edges.clear() + for task in self._tasks: + canonical_task = self._ensure_task(task) + parents = self._collect_return_refs(task.args) | self._collect_return_refs(task.kwargs) + for parent in parents: + canonical_parent = self._ensure_task(parent) + self._return_edges[canonical_parent].add(canonical_task) + + def _collect_return_refs(self, value): + refs = set() + if isinstance(value, ReturnRef): + refs.add(value.node) + elif isinstance(value, (list, tuple, set, frozenset)): + for item in value: + refs |= self._collect_return_refs(item) + elif isinstance(value, dict): + for item in value.values(): + refs |= self._collect_return_refs(item) + return refs + + def _serialise(self, value): + if isinstance(value, ReturnRef): + return { + "type": "return_ref", + "node": value.node.name, + "path": list(value.path), + } + if isinstance(value, (list, tuple)): + return [self._serialise(item) for item in value] + if isinstance(value, (set, frozenset)): + items = [self._serialise(item) for item in value] + try: + return sorted(items, key=lambda x: str(x)) + except TypeError: + return items + if isinstance(value, dict): + return {k: self._serialise(v) for k, v in value.items()} + return value + + def _serialise_file(self, file_obj): + return { + "name": file_obj.name, + "metadata": dict(file_obj.metadata), + } + + def _ensure_task(self, task): + name = task.name if isinstance(task, BlueprintGraphTask) else str(task) + canonical = self._tasks_by_name.get(name) + if canonical is None: + raise ValueError(f"Task '{name}' does not belong to this graph") + return canonical + + def _generate_task_name(self): + return uuid.uuid4().hex + + def _validate_supported_value(self, value, context, _seen=None): + if _seen is None: + _seen = set() + value_id = id(value) + if value_id in _seen: + return + _seen.add(value_id) + + if isinstance(value, ReturnRef): + return + if isinstance(value, self._SUPPORTED_SCALARS): + return + if isinstance(value, (list, tuple, set, frozenset)): + for item in value: + self._validate_supported_value(item, context, _seen) + return + if isinstance(value, dict): + for item in value.values(): + self._validate_supported_value(item, context, _seen) + return + + raise TypeError( + f"Unsupported argument type '{type(value).__name__}' encountered in {context}. " + "Wrap custom objects in basic containers (dict/list/tuple/set) or convert them " + "to ReturnRef before building the graph." + ) + + def _assert_no_returnref_in_keys(self, value, context, _seen=None): + if _seen is None: + _seen = set() + value_id = id(value) + if value_id in _seen: + return + _seen.add(value_id) + + if isinstance(value, dict): + for key, val in value.items(): + if isinstance(key, ReturnRef): + raise TypeError( + f"ReturnRef cannot be used as a dict key in {context}" + ) + self._assert_no_returnref_in_keys(val, context, _seen) + elif isinstance(value, (list, tuple, set, frozenset)): + for item in value: + self._assert_no_returnref_in_keys(item, context, _seen) + + + From bc442cdb11ec53445ea32ee1711f4975d8ffaea1 Mon Sep 17 00:00:00 2001 From: Jin Zhou Date: Tue, 6 Jan 2026 14:57:41 -0500 Subject: [PATCH 100/113] Add example target to Makefile for running example_blueprint --- taskvine/src/graph/example_blueprint.py | 80 +++++++++++++++++++++++++ 1 file changed, 80 insertions(+) create mode 100644 taskvine/src/graph/example_blueprint.py diff --git a/taskvine/src/graph/example_blueprint.py b/taskvine/src/graph/example_blueprint.py new file mode 100644 index 0000000000..f5bd62322a --- /dev/null +++ b/taskvine/src/graph/example_blueprint.py @@ -0,0 +1,80 @@ +"""Example blueprint graph using ReturnRef and file edges.""" + +try: + from .high_level import BlueprintGraph +except ImportError: # Allow running as a standalone script + import os + import sys + + CURRENT_DIR = os.path.dirname(os.path.abspath(__file__)) + if CURRENT_DIR not in sys.path: + sys.path.insert(0, CURRENT_DIR) + from high_level import BlueprintGraph # type: ignore + + +def func_1(a, b, c): + with open("file1.txt", "w") as f: + f.write("Hello, world!") + + with open("file2.txt", "w") as f: + f.write("Goodbye, world!") + + return a + b + c + + +def func_2(number): + with open("file1.txt", "r") as f: + contents1 = f.read() + + with open("file2.txt", "r") as f: + contents2 = f.read() + + return f"{contents1} {contents2} {number}" + + +def build_blueprint_graph(): + g = BlueprintGraph() + + f1 = g.define_file("file1.txt", kind="intermediate") + f2 = g.define_file("file2.txt", kind="intermediate") + + t1 = g.task(func_1, 1, 2, 3, name="task_a").produces(f1, f2) + t2 = g.task(func_2, t1.ret(), name="task_b").consumes(f1, f2) + + return g + +def describe_graph(graph): + description = graph.describe() + print("Files:", {name: meta for name, meta in description["files"].items()}) + for name, info in description["tasks"].items(): + print(f"Node {name}:") + args = ", ".join(_format_arg(a) for a in info.get("args", [])) + kwargs = ", ".join(f"{k}={_format_arg(v)}" for k, v in info.get("kwargs", {}).items()) + produces = ", ".join(_format_arg(f) for f in info.get("produces", [])) + consumes = ", ".join(_format_arg(f) for f in info.get("consumes", [])) + print(f" args : {args or '-'}") + print(f" kwargs : {kwargs or '-'}") + print(f" produces : {produces or '-'}") + print(f" consumes : {consumes or '-'}") + print("Return links:", description["return_links"]) + print("File links :", description["file_links"]) + + +def _format_arg(value): + if isinstance(value, dict) and value.get("type") == "return_ref": + return f"ReturnRef({value['node']})" + if isinstance(value, list): + return "[" + ", ".join(_format_arg(v) for v in value) + "]" + if isinstance(value, set): + return "{" + ", ".join(_format_arg(v) for v in sorted(value, key=str)) + "}" + if isinstance(value, dict): + inner = ", ".join(f"{k}: {_format_arg(v)}" for k, v in value.items()) + return "{" + inner + "}" + return str(value) + + +if __name__ == "__main__": + graph = build_blueprint_graph() + describe_graph(graph) + + From 747a64799584420a6fa6a0ed8e679b9b261eea7d Mon Sep 17 00:00:00 2001 From: Jin Zhou Date: Wed, 7 Jan 2026 15:20:09 -0500 Subject: [PATCH 101/113] temp --- taskvine/src/graph/__init__.py | 10 - taskvine/src/graph/dagvine/Makefile | 2 +- .../src/graph/dagvine/context_graph/core.py | 16 +- .../dagvine/context_graph/proxy_functions.py | 4 +- taskvine/src/graph/dagvine/dagvine.py | 78 ++-- taskvine/src/graph/example_blueprint.py | 80 ----- taskvine/src/graph/high_level.py | 332 ------------------ 7 files changed, 59 insertions(+), 463 deletions(-) delete mode 100644 taskvine/src/graph/example_blueprint.py delete mode 100644 taskvine/src/graph/high_level.py diff --git a/taskvine/src/graph/__init__.py b/taskvine/src/graph/__init__.py index 0231025894..e69de29bb2 100644 --- a/taskvine/src/graph/__init__.py +++ b/taskvine/src/graph/__init__.py @@ -1,10 +0,0 @@ -"""High-level blueprint graph interfaces.""" - -from .high_level import BlueprintFile, BlueprintGraph, ReturnRef - -__all__ = [ - "BlueprintFile", - "BlueprintGraph", - "ReturnRef", -] - diff --git a/taskvine/src/graph/dagvine/Makefile b/taskvine/src/graph/dagvine/Makefile index f40356744e..1bfc711036 100644 --- a/taskvine/src/graph/dagvine/Makefile +++ b/taskvine/src/graph/dagvine/Makefile @@ -2,7 +2,7 @@ include ../../../../config.mk include ../../../../rules.mk MODULE_DIR := $(CCTOOLS_PYTHON3_PATH)/ndcctools/taskvine/dagvine -SUBDIRS := context_graph vine_graph +SUBDIRS := context_graph vine_graph blueprint_graph .PHONY: all install clean lint format $(SUBDIRS) diff --git a/taskvine/src/graph/dagvine/context_graph/core.py b/taskvine/src/graph/dagvine/context_graph/core.py index f01cd35596..9d0f3ac1c4 100644 --- a/taskvine/src/graph/dagvine/context_graph/core.py +++ b/taskvine/src/graph/dagvine/context_graph/core.py @@ -72,8 +72,8 @@ def __init__(self, task_dict, self.parents_of, self.children_of = self._build_dependencies(self.task_dict) # these mappings are set after node ids are assigned in the C vine graph - self.ckey2vid = {} - self.vid2ckey = {} + self.pykey2cid = {} + self.cid2pykey = {} # will be set from vine graph self.outfile_remote_name = {key: None for key in self.task_dict.keys()} @@ -181,15 +181,3 @@ def get_topological_order(self): raise ValueError("Failed to create topo order, the dependencies may be cyclic or problematic") return topo_order - - @staticmethod - def context_loader_func(context_graph_pkl): - """Entry point the proxy library invokes to restore the serialized ContextGraph.""" - context_graph = cloudpickle.loads(context_graph_pkl) - - if not isinstance(context_graph, ContextGraph): - raise TypeError("context_graph_pkl is not of type ContextGraph") - - return { - "context_graph": context_graph, - } diff --git a/taskvine/src/graph/dagvine/context_graph/proxy_functions.py b/taskvine/src/graph/dagvine/context_graph/proxy_functions.py index c466e62576..db14b1c356 100644 --- a/taskvine/src/graph/dagvine/context_graph/proxy_functions.py +++ b/taskvine/src/graph/dagvine/context_graph/proxy_functions.py @@ -90,9 +90,9 @@ def compute_single_key(vine_key): After computation, the result is saved, the output file is validated, and an optional delay (`extra_sleep_time_of`) is applied before returning. """ - context_graph = load_variable_from_library('context_graph') + context_graph = load_variable_from_library('graph') - k = context_graph.vid2ckey[vine_key] + k = context_graph.cid2pykey[vine_key] v = context_graph.task_dict[k] if context_graph.is_dts_key(k): diff --git a/taskvine/src/graph/dagvine/dagvine.py b/taskvine/src/graph/dagvine/dagvine.py index b8a51bc82d..e060dcbf3d 100644 --- a/taskvine/src/graph/dagvine/dagvine.py +++ b/taskvine/src/graph/dagvine/dagvine.py @@ -5,9 +5,9 @@ from ndcctools.taskvine import cvine from ndcctools.taskvine.manager import Manager -from ndcctools.taskvine.dagvine.context_graph.proxy_library import ProxyLibrary -from ndcctools.taskvine.dagvine.context_graph.proxy_functions import compute_single_key -from ndcctools.taskvine.dagvine.context_graph.core import ContextGraph, ContextGraphTaskResult +from ndcctools.taskvine.dagvine.blueprint_graph.proxy_library import ProxyLibrary +from ndcctools.taskvine.dagvine.blueprint_graph.proxy_functions import compute_single_key +from ndcctools.taskvine.dagvine.blueprint_graph.blueprint_graph import BlueprintGraph, TaskOutputWrapper from ndcctools.taskvine.dagvine.vine_graph.vine_graph_client import VineGraphClient import cloudpickle @@ -31,6 +31,14 @@ dts = None +def context_loader_func(graph_pkl): + graph = cloudpickle.loads(graph_pkl) + + return { + "graph": graph, + } + + def delete_all_files(root_dir): """Clean the run-info template directory between runs so stale files never leak into a new DAG.""" if not os.path.exists(root_dir): @@ -81,7 +89,7 @@ def dask_collections_to_task_dict(collection_dict): # hand us `{key: delayed / value}` directly, while some experiments pass a # fully-expanded legacy Dask dict. This helper normalises both cases so the rest # of the pipeline only deals with `{task_key: task_expression}`. -def ensure_task_dict(collection_dict): +def normalize_task_dict(collection_dict): """Normalize user input (raw dict or Dask collection) into a plain `{task_key: expr}` mapping.""" if is_dask_collection and any(is_dask_collection(v) for v in collection_dict.values()): task_dict = dask_collections_to_task_dict(collection_dict) @@ -223,9 +231,27 @@ def build_context_graph(self, task_dict): return context_graph - def build_vine_graph(self, context_graph, target_keys): - """Mirror the ContextGraph into VineGraph, preserving ordering and targets.""" - assert context_graph is not None, "ContextGraph must be built before building the VineGraph" + def build_blueprint_graph(self, task_dict): + def _unpack_sexpr(sexpr): + func = sexpr[0] + tail = sexpr[1:] + if tail and isinstance(tail[-1], dict): + return func, tail[:-1], tail[-1] + else: + return func, tail, {} + + bg = BlueprintGraph() + + for k, v in task_dict.items(): + func, args, kwargs = _unpack_sexpr(v) + assert callable(func), f"Task {k} does not have a callable" + bg.add_task(k, func, *args, **kwargs) + + return bg + + def build_vine_graph(self, py_graph, target_keys): + """Mirror the Python graph into VineGraph, preserving ordering and targets.""" + assert py_graph is not None, "Python graph must be built before building the VineGraph" vine_graph = VineGraphClient(self._taskvine) @@ -235,13 +261,14 @@ def build_vine_graph(self, context_graph, target_keys): self.tune_manager() self.tune_vine_graph(vine_graph) - topo_order = context_graph.get_topological_order() + topo_order = py_graph.get_topological_order() + # Build the cross-language mapping as we walk the topo order. for k in topo_order: node_id = vine_graph.add_node(k) - context_graph.ckey2vid[k] = node_id - context_graph.vid2ckey[node_id] = k - for pk in context_graph.parents_of[k]: + py_graph.pykey2cid[k] = node_id + py_graph.cid2pykey[node_id] = k + for pk in py_graph.parents_of[k]: vine_graph.add_dependency(pk, k) # Now that every node is present, mark which ones are final outputs. @@ -252,26 +279,29 @@ def build_vine_graph(self, context_graph, target_keys): return vine_graph - def build_graphs(self, task_dict, target_keys): + def build_graphs(self, task_dict, target_keys, blueprint_graph=False): """Create both the ContextGraph and its C counterpart, wiring outputs for later use.""" # Build the logical (Python) DAG. - context_graph = self.build_context_graph(task_dict) + if blueprint_graph == False: + py_graph = self.build_context_graph(task_dict) + else: + py_graph = self.build_blueprint_graph(task_dict) # Build the physical (C) DAG. - vine_graph = self.build_vine_graph(context_graph, target_keys) + vine_graph = self.build_vine_graph(py_graph, target_keys) # Cross-fill the outfile locations so the runtime graph knows where to read/write. - for k in context_graph.ckey2vid: + for k in py_graph.pykey2cid: outfile_remote_name = vine_graph.get_node_outfile_remote_name(k) - context_graph.outfile_remote_name[k] = outfile_remote_name + py_graph.outfile_remote_name[k] = outfile_remote_name - return context_graph, vine_graph + return py_graph, vine_graph - def create_proxy_library(self, context_graph, vine_graph, hoisting_modules, env_files): + def create_proxy_library(self, py_graph, vine_graph, hoisting_modules, env_files): """Package up the context_graph as a TaskVine library.""" proxy_library = ProxyLibrary(self) proxy_library.add_hoisting_modules(hoisting_modules) proxy_library.add_env_files(env_files) - proxy_library.set_context_loader(ContextGraph.context_loader_func, context_loader_args=[cloudpickle.dumps(context_graph)]) + proxy_library.set_context_loader(context_loader_func, context_loader_args=[cloudpickle.dumps(py_graph)]) proxy_library.set_libcores(self.param("libcores")) proxy_library.set_name(vine_graph.get_proxy_library_name()) @@ -290,13 +320,13 @@ def run(self, collection_dict, target_keys=[], params={}, hoisting_modules=[], e print(f" {k}") target_keys = list(set(target_keys) - set(missing_keys)) - task_dict = ensure_task_dict(collection_dict) + task_dict = normalize_task_dict(collection_dict) # Build both the Python DAG and its C mirror. - context_graph, vine_graph = self.build_graphs(task_dict, target_keys) + py_graph, vine_graph = self.build_graphs(task_dict, target_keys, blueprint_graph=True) # Ship the execution context to workers via a proxy library. - proxy_library = self.create_proxy_library(context_graph, vine_graph, hoisting_modules, env_files) + proxy_library = self.create_proxy_library(py_graph, vine_graph, hoisting_modules, env_files) proxy_library.install() print(f"=== Library serialized size: {color_text(proxy_library.get_context_size(), 92)} MB") @@ -313,8 +343,8 @@ def run(self, collection_dict, target_keys=[], params={}, hoisting_modules=[], e # Load any requested target outputs back into Python land. results = {} for k in target_keys: - outfile_path = os.path.join(self.param("output-dir"), context_graph.outfile_remote_name[k]) - results[k] = ContextGraphTaskResult.load_from_path(outfile_path) + outfile_path = os.path.join(self.param("output-dir"), py_graph.outfile_remote_name[k]) + results[k] = TaskOutputWrapper.load_from_path(outfile_path) return results def _on_sigint(self, signum, frame): diff --git a/taskvine/src/graph/example_blueprint.py b/taskvine/src/graph/example_blueprint.py deleted file mode 100644 index f5bd62322a..0000000000 --- a/taskvine/src/graph/example_blueprint.py +++ /dev/null @@ -1,80 +0,0 @@ -"""Example blueprint graph using ReturnRef and file edges.""" - -try: - from .high_level import BlueprintGraph -except ImportError: # Allow running as a standalone script - import os - import sys - - CURRENT_DIR = os.path.dirname(os.path.abspath(__file__)) - if CURRENT_DIR not in sys.path: - sys.path.insert(0, CURRENT_DIR) - from high_level import BlueprintGraph # type: ignore - - -def func_1(a, b, c): - with open("file1.txt", "w") as f: - f.write("Hello, world!") - - with open("file2.txt", "w") as f: - f.write("Goodbye, world!") - - return a + b + c - - -def func_2(number): - with open("file1.txt", "r") as f: - contents1 = f.read() - - with open("file2.txt", "r") as f: - contents2 = f.read() - - return f"{contents1} {contents2} {number}" - - -def build_blueprint_graph(): - g = BlueprintGraph() - - f1 = g.define_file("file1.txt", kind="intermediate") - f2 = g.define_file("file2.txt", kind="intermediate") - - t1 = g.task(func_1, 1, 2, 3, name="task_a").produces(f1, f2) - t2 = g.task(func_2, t1.ret(), name="task_b").consumes(f1, f2) - - return g - -def describe_graph(graph): - description = graph.describe() - print("Files:", {name: meta for name, meta in description["files"].items()}) - for name, info in description["tasks"].items(): - print(f"Node {name}:") - args = ", ".join(_format_arg(a) for a in info.get("args", [])) - kwargs = ", ".join(f"{k}={_format_arg(v)}" for k, v in info.get("kwargs", {}).items()) - produces = ", ".join(_format_arg(f) for f in info.get("produces", [])) - consumes = ", ".join(_format_arg(f) for f in info.get("consumes", [])) - print(f" args : {args or '-'}") - print(f" kwargs : {kwargs or '-'}") - print(f" produces : {produces or '-'}") - print(f" consumes : {consumes or '-'}") - print("Return links:", description["return_links"]) - print("File links :", description["file_links"]) - - -def _format_arg(value): - if isinstance(value, dict) and value.get("type") == "return_ref": - return f"ReturnRef({value['node']})" - if isinstance(value, list): - return "[" + ", ".join(_format_arg(v) for v in value) + "]" - if isinstance(value, set): - return "{" + ", ".join(_format_arg(v) for v in sorted(value, key=str)) + "}" - if isinstance(value, dict): - inner = ", ".join(f"{k}: {_format_arg(v)}" for k, v in value.items()) - return "{" + inner + "}" - return str(value) - - -if __name__ == "__main__": - graph = build_blueprint_graph() - describe_graph(graph) - - diff --git a/taskvine/src/graph/high_level.py b/taskvine/src/graph/high_level.py deleted file mode 100644 index 1a36dc8bb2..0000000000 --- a/taskvine/src/graph/high_level.py +++ /dev/null @@ -1,332 +0,0 @@ -"""High-level blueprint graph for TaskVine DAG modelling.""" - -from collections import defaultdict -import uuid - - -class ReturnRef: - """Reference to the return value (or sub-path) of another task.""" - - def __init__(self, node, path=()): - if node is None: - raise ValueError("node must not be None") - self.node = node - self.path = tuple(path) - - def derive(self, *path): - return ReturnRef(self.node, self.path + tuple(path)) - - def __getitem__(self, item): - if isinstance(item, tuple): - return self.derive(*item) - return self.derive(item) - -class BlueprintFile: - """Describes a file that tasks may produce or consume.""" - - def __init__(self, name, **metadata): - if not name or not isinstance(name, str): - raise ValueError("File name must be a non-empty string") - self.name = name - self.metadata = dict(metadata) - -class BlueprintGraphTask: - """Represents a task node within the blueprint graph.""" - - def __init__(self, graph, func, args=None, kwargs=None, name=None): - self.graph = graph - self.func = func - self.name = name or graph._generate_task_name() - self.args = tuple(args) if args is not None else tuple() - self.kwargs = dict(kwargs) if kwargs is not None else {} - self.produced_files = [] - self.consumed_files = [] - self._return_ref = ReturnRef(self) - - def produces(self, *files): - for file_obj in files: - file_obj = self.graph._ensure_file(file_obj) - self.graph._register_file_producer(self, file_obj) - if file_obj not in self.produced_files: - self.produced_files.append(file_obj) - return self - - def consumes(self, *files): - for file_obj in files: - file_obj = self.graph._ensure_file(file_obj) - self.graph._register_file_consumer(self, file_obj) - if file_obj not in self.consumed_files: - self.consumed_files.append(file_obj) - return self - - def ret(self): - return self._return_ref - - def update_kwargs(self, **new_kwargs): - if not new_kwargs: - return self - merged = dict(self.kwargs) - merged.update(new_kwargs) - self.kwargs = merged - self.graph.invalidate() - return self - -class BlueprintGraph: - """Blueprint graph supporting ReturnRef and file dependencies.""" - - _SUPPORTED_SCALARS = (str, int, float, bool, type(None), bytes) - def __init__(self): - self._tasks = [] - self._tasks_by_name = {} - self._return_edges = defaultdict(set) - self._files = {} - self._file_producer = {} - self._file_consumers = defaultdict(set) - self._dirty = True - - def define_file(self, name, **metadata): - existing = self._files.get(name) - if existing: - existing.metadata.update(metadata) - return existing - file_obj = BlueprintFile(name, **metadata) - self._files[name] = file_obj - return file_obj - - def task(self, func, *args, name=None, **kwargs): - task = BlueprintGraphTask( - self, - func, - args=args, - kwargs=kwargs, - name=name, - ) - return self._register_task(task) - - def nodes(self): - return list(self._tasks) - - def file_links(self): - links = [] - for file_name, consumers in self._file_consumers.items(): - file_obj = self._files.get(file_name) - if file_obj is None: - continue - producer = self._file_producer.get(file_name) - for consumer in consumers: - links.append((producer, consumer, file_obj)) - return links - - def return_children(self, task): - task = self._ensure_task(task) - return set(self._return_edges.get(task, set())) - - def invalidate(self): - self._dirty = True - - def validate(self): - if self._dirty: - for task in self._tasks: - self._validate_supported_value(task.args, f"{task.name}.args") - self._validate_supported_value(task.kwargs, f"{task.name}.kwargs") - self._assert_no_returnref_in_keys(task.args, f"{task.name}.args") - self._assert_no_returnref_in_keys(task.kwargs, f"{task.name}.kwargs") - self._rebuild_return_edges() - self._dirty = False - - missing_producers = [ - file_name - for file_name, consumers in self._file_consumers.items() - if consumers and file_name not in self._file_producer - ] - if missing_producers: - raise ValueError( - f"Files consumed without producers: {', '.join(sorted(missing_producers))}" - ) - - def describe(self): - self.validate() - data = { - "tasks": {}, - "return_links": [], - "file_links": [], - "files": { - name: dict(file_obj.metadata) for name, file_obj in self._files.items() - }, - } - for task in self._tasks: - data["tasks"][task.name] = { - "func": getattr(task.func, "__name__", repr(task.func)), - "args": self._serialise(task.args), - "kwargs": self._serialise(task.kwargs), - "produces": [self._serialise_file(f) for f in task.produced_files], - "consumes": [self._serialise_file(f) for f in task.consumed_files], - } - for parent, children in self._return_edges.items(): - for child in children: - data["return_links"].append( - { - "parent": parent.name, - "child": child.name, - } - ) - for file_name, consumers in self._file_consumers.items(): - file_obj = self._files.get(file_name) - if file_obj is None: - continue - producer = self._file_producer.get(file_name) - for consumer in consumers: - data["file_links"].append( - { - "parent": producer.name if producer else None, - "child": consumer.name if consumer else None, - "filename": file_obj.name, - "metadata": dict(file_obj.metadata), - } - ) - return data - - def _register_task(self, task): - canonical = self._tasks_by_name.get(task.name) - if canonical is not None: - raise ValueError(f"Task name '{task.name}' already registered") - self._tasks.append(task) - self._tasks_by_name[task.name] = task - self.invalidate() - return task - - def _register_file_producer(self, task, file_obj): - task = self._ensure_task(task) - existing = self._file_producer.get(file_obj.name) - if existing and existing is not task: - raise ValueError( - f"File '{file_obj.name}' already produced by task '{existing.name}'" - ) - self._file_producer[file_obj.name] = task - self.invalidate() - - def _register_file_consumer(self, task, file_obj): - task = self._ensure_task(task) - self._file_consumers[file_obj.name].add(task) - self.invalidate() - - def _ensure_file(self, file_obj): - if isinstance(file_obj, BlueprintFile): - existing = self._files.get(file_obj.name) - if existing: - existing.metadata.update(file_obj.metadata) - return existing - self._files[file_obj.name] = file_obj - return file_obj - if isinstance(file_obj, str): - existing = self._files.get(file_obj) - if not existing: - existing = BlueprintFile(file_obj) - self._files[file_obj] = existing - return existing - raise TypeError("File reference must be BlueprintFile or string name") - - def _rebuild_return_edges(self): - self._return_edges.clear() - for task in self._tasks: - canonical_task = self._ensure_task(task) - parents = self._collect_return_refs(task.args) | self._collect_return_refs(task.kwargs) - for parent in parents: - canonical_parent = self._ensure_task(parent) - self._return_edges[canonical_parent].add(canonical_task) - - def _collect_return_refs(self, value): - refs = set() - if isinstance(value, ReturnRef): - refs.add(value.node) - elif isinstance(value, (list, tuple, set, frozenset)): - for item in value: - refs |= self._collect_return_refs(item) - elif isinstance(value, dict): - for item in value.values(): - refs |= self._collect_return_refs(item) - return refs - - def _serialise(self, value): - if isinstance(value, ReturnRef): - return { - "type": "return_ref", - "node": value.node.name, - "path": list(value.path), - } - if isinstance(value, (list, tuple)): - return [self._serialise(item) for item in value] - if isinstance(value, (set, frozenset)): - items = [self._serialise(item) for item in value] - try: - return sorted(items, key=lambda x: str(x)) - except TypeError: - return items - if isinstance(value, dict): - return {k: self._serialise(v) for k, v in value.items()} - return value - - def _serialise_file(self, file_obj): - return { - "name": file_obj.name, - "metadata": dict(file_obj.metadata), - } - - def _ensure_task(self, task): - name = task.name if isinstance(task, BlueprintGraphTask) else str(task) - canonical = self._tasks_by_name.get(name) - if canonical is None: - raise ValueError(f"Task '{name}' does not belong to this graph") - return canonical - - def _generate_task_name(self): - return uuid.uuid4().hex - - def _validate_supported_value(self, value, context, _seen=None): - if _seen is None: - _seen = set() - value_id = id(value) - if value_id in _seen: - return - _seen.add(value_id) - - if isinstance(value, ReturnRef): - return - if isinstance(value, self._SUPPORTED_SCALARS): - return - if isinstance(value, (list, tuple, set, frozenset)): - for item in value: - self._validate_supported_value(item, context, _seen) - return - if isinstance(value, dict): - for item in value.values(): - self._validate_supported_value(item, context, _seen) - return - - raise TypeError( - f"Unsupported argument type '{type(value).__name__}' encountered in {context}. " - "Wrap custom objects in basic containers (dict/list/tuple/set) or convert them " - "to ReturnRef before building the graph." - ) - - def _assert_no_returnref_in_keys(self, value, context, _seen=None): - if _seen is None: - _seen = set() - value_id = id(value) - if value_id in _seen: - return - _seen.add(value_id) - - if isinstance(value, dict): - for key, val in value.items(): - if isinstance(key, ReturnRef): - raise TypeError( - f"ReturnRef cannot be used as a dict key in {context}" - ) - self._assert_no_returnref_in_keys(val, context, _seen) - elif isinstance(value, (list, tuple, set, frozenset)): - for item in value: - self._assert_no_returnref_in_keys(item, context, _seen) - - - From 97993b4c14d86c28618f78cbaa941854fd0d12b6 Mon Sep 17 00:00:00 2001 From: Jin Zhou Date: Thu, 8 Jan 2026 02:40:36 -0500 Subject: [PATCH 102/113] support new dask expr --- .../graph/dagvine/blueprint_graph/Makefile | 31 + .../graph/dagvine/blueprint_graph/__init__.py | 16 + .../graph/dagvine/blueprint_graph/adaptor.py | 568 ++++++++++++++++++ .../dagvine/blueprint_graph/adaptor_test.py | 528 ++++++++++++++++ .../blueprint_graph/blueprint_graph.py | 197 ++++++ .../blueprint_graph/proxy_functions.py | 51 ++ .../dagvine/blueprint_graph/proxy_library.py | 94 +++ taskvine/src/graph/dagvine/dagvine.py | 74 +-- 8 files changed, 1489 insertions(+), 70 deletions(-) create mode 100644 taskvine/src/graph/dagvine/blueprint_graph/Makefile create mode 100644 taskvine/src/graph/dagvine/blueprint_graph/__init__.py create mode 100644 taskvine/src/graph/dagvine/blueprint_graph/adaptor.py create mode 100644 taskvine/src/graph/dagvine/blueprint_graph/adaptor_test.py create mode 100644 taskvine/src/graph/dagvine/blueprint_graph/blueprint_graph.py create mode 100644 taskvine/src/graph/dagvine/blueprint_graph/proxy_functions.py create mode 100644 taskvine/src/graph/dagvine/blueprint_graph/proxy_library.py diff --git a/taskvine/src/graph/dagvine/blueprint_graph/Makefile b/taskvine/src/graph/dagvine/blueprint_graph/Makefile new file mode 100644 index 0000000000..55bbb1c5e4 --- /dev/null +++ b/taskvine/src/graph/dagvine/blueprint_graph/Makefile @@ -0,0 +1,31 @@ +include ../../../../../config.mk +include ../../../../../rules.mk + +PROJECT_NAME = dagvine + +SOURCE_DIR = $(CCTOOLS_HOME)/taskvine/src/graph/$(PROJECT_NAME)/blueprint_graph +MODULE_ROOT = $(CCTOOLS_PYTHON3_PATH)/ndcctools/taskvine/$(PROJECT_NAME) +MODULE_DIR = $(MODULE_ROOT)/blueprint_graph + +PY_SOURCES = $(wildcard $(SOURCE_DIR)/*.py) + +.PHONY: all install clean lint format + +all: + @true + +install: + mkdir -p $(MODULE_DIR) + cp $(PY_SOURCES) $(MODULE_DIR) + +clean: + rm -rf __pycache__ + +lint: + flake8 --ignore=$(CCTOOLS_FLAKE8_IGNORE_ERRORS) \ + --exclude=$(CCTOOLS_FLAKE8_IGNORE_FILES) \ + $(SOURCE_DIR)/ + +format: + @true + diff --git a/taskvine/src/graph/dagvine/blueprint_graph/__init__.py b/taskvine/src/graph/dagvine/blueprint_graph/__init__.py new file mode 100644 index 0000000000..f8282164a7 --- /dev/null +++ b/taskvine/src/graph/dagvine/blueprint_graph/__init__.py @@ -0,0 +1,16 @@ +# 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 .blueprint_graph import BlueprintGraph, TaskOutputRef, TaskOutputWrapper +from .proxy_functions import compute_single_key +from .adaptor import Adaptor + +__all__ = [ + "BlueprintGraph", + "TaskOutputRef", + "TaskOutputWrapper", + "compute_single_key", + "Adaptor", +] diff --git a/taskvine/src/graph/dagvine/blueprint_graph/adaptor.py b/taskvine/src/graph/dagvine/blueprint_graph/adaptor.py new file mode 100644 index 0000000000..3f5db28243 --- /dev/null +++ b/taskvine/src/graph/dagvine/blueprint_graph/adaptor.py @@ -0,0 +1,568 @@ +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: + # Treat any import failure as "no TaskSpec support" (including environments + # where the private module is unavailable or type-checkers can't resolve it). + dts = None + +from ndcctools.taskvine.dagvine.blueprint_graph.blueprint_graph import TaskOutputRef + + +def _identity(value): + """Tiny helper that just hands back whatever you pass in (e.g. `_identity(3)` -> 3).""" + return value + + +class Adaptor: + """Normalize user task inputs so `BlueprintGraph` can consume them without extra massaging.""" + + _LEAF_TYPES = (str, bytes, bytearray, memoryview, int, float, bool, type(None)) + + def __init__(self, collection_dict): + self.original_collection_dict = collection_dict + + # TaskSpec-only state used to "lift" inline Tasks that cannot be reduced to + # a pure Python value (or would be unsafe/expensive to inline). + self._lifted_nodes = {} + self._lift_cache = {} + self._lift_counter = 0 + # Global shared key-set for the whole adaptation run (original keys + lifted keys). + # IMPORTANT: TaskSpec conversion must always consult the same shared set so that + # lifted keys remain visible across subsequent conversions/dedup/reference checks. + self._task_keys = set() + + normalized = self._normalize_task_dict(collection_dict) + self.task_dict = self._convert_to_blueprint_tasks(normalized) + + def _normalize_task_dict(self, collection_dict): + """Collapse every supported input style into a classic `{key: sexpr or TaskSpec}` mapping.""" + from_dask_collection = bool( + is_dask_collection and any(is_dask_collection(v) for v in collection_dict.values()) + ) + + if from_dask_collection: + task_dict = self._dask_collections_to_task_dict(collection_dict) + else: + # IMPORTANT: treat plain user dicts as DAGVine sexprs by default. + # If we unconditionally run `dask._task_spec.convert_legacy_graph(...)` when + # dts is available, Dask will interpret our "final Mapping is kwargs" + # convention as a positional dict argument, breaking sexpr semantics. + task_dict = dict(collection_dict) + + # Only ask Dask to rewrite legacy graphs when we *know* the input came + # from a Dask collection/HLG. This keeps classic DAGVine sexprs stable + # even in environments where dask._task_spec is installed. + if from_dask_collection and dts and hasattr(dts, "convert_legacy_graph"): + task_dict = dts.convert_legacy_graph(task_dict) + + return task_dict + + def _convert_to_blueprint_tasks(self, task_dict): + """Turn each normalized entry into the `(func, args, kwargs)` triple that BlueprintGraph expects.""" + if not task_dict: + return {} + + converted = {} + # Shared task key universe for TaskSpec lifting/dedup/reference decisions. + # Keep this as a single shared set for the whole conversion. + self._task_keys = set(task_dict.keys()) + task_keys = self._task_keys + + for key, value in task_dict.items(): + if self._is_dts_node(value): + converted[key] = self._convert_dts_graph_node(key, value, task_keys) + else: + converted[key] = self._convert_sexpr_task(value, task_keys) + + # If any inline TaskSpec Tasks were lifted during conversion, convert them too. + # We do this iteratively because lifting a node can expose further inline Tasks. + while True: + pending = [(k, v) for k, v in self._lifted_nodes.items() if k not in converted] + if not pending: + break + for k, node in pending: + converted[k] = self._convert_dts_graph_node(k, node, task_keys) + + return converted + + def _convert_dts_graph_node(self, key, node, task_keys): + """Translate modern Dask TaskSpec graph nodes into blueprint expressions.""" + if not dts: + raise RuntimeError("Dask TaskSpec support unavailable: dask._task_spec is not installed") + + task_cls = getattr(dts, "Task", None) + alias_cls = getattr(dts, "Alias", None) + literal_cls = getattr(dts, "Literal", None) + datanode_cls = getattr(dts, "DataNode", None) + nested_cls = getattr(dts, "NestedContainer", None) + taskref_cls = getattr(dts, "TaskRef", None) + + if task_cls and isinstance(node, task_cls): + func = self._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 i, arg in enumerate(raw_args): + args.append(self._unwrap_dts_operand(arg, task_keys, parent_key=key)) + except Exception as e: + raise TypeError( + "Failed to adapt TaskSpec node argument while converting to BlueprintGraph.\n" + f"- parent_task_key: {key!r}\n" + f"- func: {self._safe_repr(func)}\n" + f"- arg_index: {i}\n" + f"- arg_value: {self._safe_repr(arg)}\n" + f"- raw_args: {self._safe_repr(raw_args)}\n" + f"- raw_kwargs: {self._safe_repr(raw_kwargs)}" + ) from e + + kwargs = {} + try: + for k, v in raw_kwargs.items(): + kwargs[k] = self._unwrap_dts_operand(v, task_keys, parent_key=key) + except Exception as e: + raise TypeError( + "Failed to adapt TaskSpec node kwarg while converting to BlueprintGraph.\n" + f"- parent_task_key: {key!r}\n" + f"- func: {self._safe_repr(func)}\n" + f"- kwarg_key: {k!r}\n" + f"- kwarg_value: {self._safe_repr(v)}\n" + f"- raw_args: {self._safe_repr(raw_args)}\n" + f"- raw_kwargs: {self._safe_repr(raw_kwargs)}" + ) from e + + return self._build_expr(func, args, kwargs) + + if alias_cls and isinstance(node, alias_cls): + alias_ref = self._extract_alias_target(node, task_keys) + if alias_ref is None: + raise ValueError(f"Alias {key} is missing a resolvable upstream task") + return self._build_expr(_identity, [alias_ref], {}) + + if datanode_cls and isinstance(node, datanode_cls): + return self._build_expr(_identity, [node.value], {}) + + if literal_cls and isinstance(node, literal_cls): + return self._build_expr(_identity, [node.value], {}) + + if taskref_cls and isinstance(node, taskref_cls): + ref = TaskOutputRef(node.key, getattr(node, "path", ()) or ()) + return self._build_expr(_identity, [ref], {}) + + if nested_cls and isinstance(node, nested_cls): + payload = getattr(node, "value", None) + if payload is None: + payload = getattr(node, "data", None) + value = self._unwrap_dts_operand(payload, task_keys, parent_key=key) + return self._build_expr(_identity, [value], {}) + + return self._build_expr(_identity, [node], {}) + + def _convert_sexpr_task(self, sexpr, task_keys): + """Handle legacy sexpr-style nodes by replacing embedded task keys with `TaskOutputRef`.""" + 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, task_keys) for arg in raw_args) + kwargs = {k: self._wrap_dependency(v, task_keys) for k, v in raw_kwargs.items()} + + return func, args, kwargs + + def _wrap_dependency(self, obj, task_keys): + """Wrap nested objects inside a sexpr when they point at other tasks.""" + if isinstance(obj, TaskOutputRef): + return obj + + if self._should_wrap(obj, task_keys): + return TaskOutputRef(obj) + + if isinstance(obj, list): + return [self._wrap_dependency(v, task_keys) for v in obj] + + if isinstance(obj, tuple): + if obj and callable(obj[0]): + head = obj[0] + tail = tuple(self._wrap_dependency(v, task_keys) for v in obj[1:]) + return (head, *tail) + return tuple(self._wrap_dependency(v, task_keys) for v in obj) + + if isinstance(obj, Mapping): + return {k: self._wrap_dependency(v, task_keys) for k, v in obj.items()} + + if isinstance(obj, set): + return {self._wrap_dependency(v, task_keys) for v in obj} + + if isinstance(obj, frozenset): + return frozenset(self._wrap_dependency(v, task_keys) for v in obj) + + return obj + + def _should_wrap(self, obj, task_keys): + """Decide whether a value should become a `TaskOutputRef`.""" + if isinstance(obj, self._LEAF_TYPES): + if isinstance(obj, str): + return obj in task_keys + return False + try: + return obj in task_keys + except TypeError: + return False + + # Flatten Dask collections into the dict-of-tasks structure the rest of the + # pipeline expects. DAGVine clients often hand us a dict like + # {"result": dask.delayed(...)}; we merge the underlying HighLevelGraphs so + # `ContextGraph` sees the same dict representation C does. + def _dask_collections_to_task_dict(self, collection_dict): + """Flatten Dask collections into the classic dict-of-task layout.""" + assert is_dask_collection is not None + from dask.highlevelgraph import HighLevelGraph, ensure_dict + + if not isinstance(collection_dict, dict): + raise TypeError("Input must be a dict") + + for k, v in collection_dict.items(): + if not is_dask_collection(v): + raise TypeError( + f"Input must be a dict of DaskCollection, but found {k} with type {type(v)}" + ) + + if dts: + sub_hlgs = [v.dask for v in collection_dict.values()] + hlg = HighLevelGraph.merge(*sub_hlgs).to_dict() + else: + hlg = dask.base.collections_to_dsk(collection_dict.values()) + + return ensure_dict(hlg) + + def _is_dts_node(self, value): + """Return True when the value is part of the TaskSpec family.""" + if not dts: + return False + try: + return isinstance(value, dts.GraphNode) + except AttributeError: + return False + + def _unwrap_dts_operand(self, operand, task_keys, *, parent_key=None): + """Recursively unwrap TaskSpec operands into pure Python values/containers and `TaskOutputRef`. + + Contract (TaskSpec path only): + - TaskRef/Alias become `TaskOutputRef` (references, never lifted). + - Literals/DataNode become plain Python values. + - NestedContainer unwraps recursively. + - Task inside args/kwargs is either: + - treated as a reference when it has a top-level key, or + - reduced to a pure value only for a small "pure constructor/identity" whitelist, or + - lifted into a new top-level node and replaced with `TaskOutputRef(new_key)`. + """ + if not dts: + return operand + + taskref_cls = getattr(dts, "TaskRef", None) + if taskref_cls and isinstance(operand, taskref_cls): + key = getattr(operand, "key", None) + path = getattr(operand, "path", ()) + return TaskOutputRef(key, path or ()) + + alias_cls = getattr(dts, "Alias", None) + if alias_cls and isinstance(operand, alias_cls): + alias_ref = self._extract_alias_target(operand, task_keys) + if alias_ref is None: + raise ValueError("Alias node is missing a valid upstream source") + return alias_ref + + literal_cls = getattr(dts, "Literal", None) + if literal_cls and isinstance(operand, literal_cls): + return getattr(operand, "value", None) + + datanode_cls = getattr(dts, "DataNode", None) + if datanode_cls and isinstance(operand, datanode_cls): + return operand.value + + nested_cls = getattr(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_dts_operand(payload, task_keys, parent_key=parent_key) + + task_cls = getattr(dts, "Task", None) + if task_cls and isinstance(operand, task_cls): + inline_key = getattr(operand, "key", None) + # Rule 3: if it is a real graph node (key is present and in task_keys), + # treat it as a dependency reference. + if inline_key is not None and inline_key in task_keys: + return TaskOutputRef(inline_key, ()) + + # Otherwise it is an inline expression. Reduce if safe, else lift. + func = self._extract_callable_from_task(operand) + if func is None: + return self._lift_inline_task(operand, task_keys, parent_key=parent_key) + + # Special-case: Dask internal identity-cast wrappers should not be called + # during adaptation. Reduce structurally by returning the first argument. + if self._is_identity_cast_op(func): + raw_args = getattr(operand, "args", ()) or () + if not raw_args: + return None + return self._unwrap_dts_operand(raw_args[0], task_keys, parent_key=parent_key) + + if self._is_pure_value_op(func): + reduced, used_lift = self._reduce_inline_task(operand, task_keys, parent_key=parent_key) + if used_lift: + # Rule 2: if any child required lifting/unknown handling, lift the whole expression. + return self._lift_inline_task(operand, task_keys, parent_key=parent_key) + if self._is_too_large_inline_value(reduced): + return self._lift_inline_task(operand, task_keys, parent_key=parent_key) + return reduced + + # Rule 1: unknown/unsafe op -> must lift. + return self._lift_inline_task(operand, task_keys, parent_key=parent_key) + + if isinstance(operand, list): + return [self._unwrap_dts_operand(v, task_keys, parent_key=parent_key) for v in operand] + + if isinstance(operand, tuple): + return tuple(self._unwrap_dts_operand(v, task_keys, parent_key=parent_key) for v in operand) + + if isinstance(operand, Mapping): + return {k: self._unwrap_dts_operand(v, task_keys, parent_key=parent_key) for k, v in operand.items()} + + if isinstance(operand, set): + return {self._unwrap_dts_operand(v, task_keys, parent_key=parent_key) for v in operand} + + if isinstance(operand, frozenset): + return frozenset(self._unwrap_dts_operand(v, task_keys, parent_key=parent_key) for v in operand) + + return operand + + def _extract_alias_target(self, alias_node, task_keys): + """Discover which upstream key an alias points at and return it as a `TaskOutputRef`.""" + fields = getattr(alias_node.__class__, "__dataclass_fields__", {}) if dts else {} + + path = getattr(alias_node, "path", ()) + path = tuple(path) if path else () + + for candidate in ("alias_of", "target", "source", "ref"): + if candidate in fields: + raw_value = getattr(alias_node, candidate, None) + if self._should_wrap(raw_value, task_keys): + return TaskOutputRef(raw_value, path) + + deps = getattr(alias_node, "dependencies", None) + if deps: + deps = list(deps) + if len(deps) == 1: + return TaskOutputRef(deps[0], path) + + return None + + @staticmethod + def _build_expr(func, args, kwargs): + return func, tuple(args), dict(kwargs) + + @staticmethod + def _safe_repr(value, limit=800): + """Best-effort repr that won't explode logs on huge graphs.""" + try: + text = repr(value) + except Exception as e: + text = f"" + if limit and len(text) > limit: + return text[:limit] + "..." + return text + + @staticmethod + def _is_pure_value_op(func): + """Return True if `func` is safe to execute during adaptation to build a pure value. + + This is intentionally conservative: only pure constructors/identity-like ops. + """ + if func in (dict, list, tuple, set, frozenset): + return True + return False + + @staticmethod + def _is_identity_cast_op(func): + """Detect Dask's private identity-cast op without executing it.""" + name = getattr(func, "__name__", None) + module = getattr(func, "__module__", None) + return bool(name == "_identity_cast" and module and module.startswith("dask")) + + def _reduce_inline_task(self, task_node, task_keys, *, parent_key=None): + """Best-effort reduction of an inline TaskSpec Task into a pure value. + + Returns (value, used_lift) where used_lift indicates a nested operand triggered lifting. + """ + func = self._extract_callable_from_task(task_node) + raw_args = getattr(task_node, "args", ()) or () + raw_kwargs = getattr(task_node, "kwargs", {}) or {} + + used_lift = False + + # unwrap args/kwargs; if we see a lifted ref, mark used_lift (Rule 2). + args = [] + for arg in raw_args: + before = len(self._lifted_nodes) + args.append(self._unwrap_dts_operand(arg, task_keys, parent_key=parent_key)) + used_lift = used_lift or (len(self._lifted_nodes) != before) + + kwargs = {} + for k, v in raw_kwargs.items(): + before = len(self._lifted_nodes) + kwargs[k] = self._unwrap_dts_operand(v, task_keys, parent_key=parent_key) + used_lift = used_lift or (len(self._lifted_nodes) != before) + + # Pure constructors are safe to execute even if they contain TaskOutputRefs + # (they just build containers of refs). Anything else is lifted. + try: + value = func(*args, **kwargs) + except Exception: + # If evaluation fails, prefer lifting over guessing semantics. + return self._lift_inline_task(task_node, task_keys, parent_key=parent_key), True + + return value, used_lift + + @staticmethod + def _is_too_large_inline_value(value, *, max_container_len=2000): + """Heuristic to avoid inlining huge container constructions that would bloat memory.""" + try: + if isinstance(value, (list, tuple, set, frozenset, dict)): + return len(value) > max_container_len + except Exception: + return False + return False + + def _lift_inline_task(self, task_node, task_keys, *, parent_key=None): + """Lift an inline TaskSpec Task into its own node and return a `TaskOutputRef` to it.""" + 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}") + + sig = self._dts_structural_signature(task_node, task_keys) + cached = self._lift_cache.get(sig) + if cached is not None: + return TaskOutputRef(cached, ()) + + import hashlib + + digest = hashlib.sha1(sig.encode("utf-8")).hexdigest()[:16] + base = f"__lift__{digest}" + new_key = base + # Collision handling + avoid clobbering existing user keys. + while new_key in task_keys or new_key in self._lifted_nodes: + self._lift_counter += 1 + new_key = f"{base}_{self._lift_counter}" + + self._lift_cache[sig] = new_key + self._lifted_nodes[new_key] = task_node + task_keys.add(new_key) + return TaskOutputRef(new_key, ()) + + def _dts_structural_signature(self, obj, task_keys): + """Best-effort stable signature for deduping lifted inline expressions.""" + # Keep it deterministic and conservative. If we can't make it stable, fall back to repr. + try: + taskref_cls = getattr(dts, "TaskRef", None) + alias_cls = getattr(dts, "Alias", None) + literal_cls = getattr(dts, "Literal", None) + datanode_cls = getattr(dts, "DataNode", None) + nested_cls = getattr(dts, "NestedContainer", None) + task_cls = getattr(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, task_keys) + return f"Alias({getattr(ref, 'task_key', None)!r},{getattr(ref, 'path', ())!r})" + if literal_cls and isinstance(obj, literal_cls): + return f"Literal({self._safe_repr(getattr(obj, 'value', None))})" + if datanode_cls and isinstance(obj, datanode_cls): + return f"DataNode({self._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._dts_structural_signature(payload, task_keys)})" + if task_cls and isinstance(obj, task_cls): + key = getattr(obj, "key", None) + if key is not None and key in task_keys: + return f"TaskKey({key!r})" + func = self._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._dts_structural_signature(a, task_keys) for a in args) + kw_sigs = ",".join(f"{k}={self._dts_structural_signature(v, task_keys)}" for k, v 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._dts_structural_signature(v, task_keys) for v in obj) + ")" + if isinstance(obj, tuple): + return "tuple(" + ",".join(self._dts_structural_signature(v, task_keys) for v in obj) + ")" + if isinstance(obj, dict): + items = ",".join( + f"{self._safe_repr(k)}:{self._dts_structural_signature(v, task_keys)}" + for k, v in sorted(obj.items(), key=lambda kv: repr(kv[0])) + ) + return "dict(" + items + ")" + if isinstance(obj, (set, frozenset)): + items = ",".join(sorted(self._dts_structural_signature(v, task_keys) for v in obj)) + return f"{type(obj).__name__}(" + items + ")" + + return f"py({self._safe_repr(obj)})" + except Exception: + return f"fallback({self._safe_repr(obj)})" + + @staticmethod + def _extract_callable_from_task(node): + candidates = ( + "function", + "op", + "callable", + "func", + "operation", + "callable_obj", + ) + + for attr in candidates: + if not hasattr(node, attr): + continue + value = getattr(node, attr) + if value is None: + continue + if callable(value): + return value + if hasattr(value, "__call__"): + return value + + if hasattr(node, "__call__") and callable(node): + return node + + return None diff --git a/taskvine/src/graph/dagvine/blueprint_graph/adaptor_test.py b/taskvine/src/graph/dagvine/blueprint_graph/adaptor_test.py new file mode 100644 index 0000000000..ce1a23fe68 --- /dev/null +++ b/taskvine/src/graph/dagvine/blueprint_graph/adaptor_test.py @@ -0,0 +1,528 @@ +import importlib +import importlib.util +import pathlib +import sys +import unittest +from dataclasses import dataclass, field + +from ndcctools.taskvine.dagvine.blueprint_graph.adaptor import Adaptor as _Adaptor, TaskOutputRef as _TaskOutputRef + +_MODULE_NAME = "ndcctools.taskvine.dagvine.blueprint_graph.adaptor" +_LOCAL_ADAPTOR = pathlib.Path(__file__).resolve().parent / "adaptor.py" + +if _LOCAL_ADAPTOR.exists(): + spec = importlib.util.spec_from_file_location(_MODULE_NAME, _LOCAL_ADAPTOR) + module = importlib.util.module_from_spec(spec) + assert spec.loader is not None + sys.modules[_MODULE_NAME] = module + spec.loader.exec_module(module) + Adaptor = module.Adaptor + TaskOutputRef = module.TaskOutputRef + adaptor_impl = module +else: + Adaptor = _Adaptor + TaskOutputRef = _TaskOutputRef + adaptor_impl = importlib.import_module(Adaptor.__module__) + + +class AdaptorSexprTests(unittest.TestCase): + def test_empty_graph_returns_empty_dict(self): + self.assertEqual(Adaptor({}).task_dict, {}) + + def test_wraps_references_in_args_and_kwargs(self): + def seed(): + return 1 + + def consume(value, bonus=0): + return value + bonus + + graph = { + "seed": (seed,), + "consumer": (consume, "seed", {"bonus": "seed"}), + } + + adaptor = Adaptor(graph) + adapted = adaptor.task_dict + + seed_func, seed_args, seed_kwargs = adapted["seed"] + self.assertIs(seed_func, seed) + self.assertEqual(seed_args, ()) + self.assertEqual(seed_kwargs, {}) + + func, args, kwargs = adapted["consumer"] + self.assertIs(func, consume) + self.assertEqual(len(args), 1) + self.assertIsInstance(args[0], TaskOutputRef) + self.assertEqual(args[0].task_key, "seed") + self.assertEqual(args[0].path, ()) + + self.assertIsInstance(kwargs["bonus"], TaskOutputRef) + self.assertEqual(kwargs["bonus"].task_key, "seed") + self.assertEqual(kwargs["bonus"].path, ()) + + def test_handles_nested_collections(self): + def aggregate(structure, *, options=None): + return structure, options + + graph = { + "alpha": (lambda: {"value": 1},), + "beta": (lambda: 2,), + "collector": ( + aggregate, + ["alpha", ("beta", "alpha")], + { + "mapping": { + "left": "alpha", + "right": ["beta", {"deep": "alpha"}], + }, + "flags": {"alpha", "unchanged"}, + }, + ), + } + + adaptor = Adaptor(graph) + adapted = adaptor.task_dict + + func, args, kwargs = adapted["collector"] + self.assertIs(func, aggregate) + + self.assertEqual(len(args), 1) + structure = args[0] + self.assertIsInstance(structure, list) + self.assertIsInstance(structure[0], TaskOutputRef) + self.assertEqual(structure[0].task_key, "alpha") + + tuple_fragment = structure[1] + self.assertIsInstance(tuple_fragment, tuple) + self.assertIsInstance(tuple_fragment[0], TaskOutputRef) + self.assertEqual(tuple_fragment[0].task_key, "beta") + self.assertIsInstance(tuple_fragment[1], TaskOutputRef) + self.assertEqual(tuple_fragment[1].task_key, "alpha") + + mapping = kwargs["mapping"] + self.assertIsInstance(mapping["left"], TaskOutputRef) + self.assertEqual(mapping["left"].task_key, "alpha") + + right_list = mapping["right"] + self.assertIsInstance(right_list[0], TaskOutputRef) + self.assertEqual(right_list[0].task_key, "beta") + self.assertIsInstance(right_list[1]["deep"], TaskOutputRef) + self.assertEqual(right_list[1]["deep"].task_key, "alpha") + + flags = kwargs["flags"] + self.assertIsInstance(flags, set) + ref_keys = {item.task_key for item in flags if isinstance(item, TaskOutputRef)} + self.assertEqual(ref_keys, {"alpha"}) + self.assertIn("unchanged", flags) + + def test_literal_strings_remain_literals(self): + def attach_unit(value, *, unit): + return value, unit + + graph = { + "value": (lambda: 42,), + "with_unit": ( + attach_unit, + "value", + {"unit": "kg"}, + ), + } + + adaptor = Adaptor(graph) + adapted = adaptor.task_dict + func, args, kwargs = adapted["with_unit"] + self.assertIs(func, attach_unit) + self.assertEqual(len(args), 1) + self.assertIsInstance(args[0], TaskOutputRef) + self.assertEqual(kwargs["unit"], "kg") + + def test_existing_task_output_ref_is_preserved(self): + original_ref = TaskOutputRef("seed") + + graph = { + "seed": (lambda: 5,), + "forward": (lambda x: x, original_ref), + } + + adapted = Adaptor(graph).task_dict + func, args, kwargs = adapted["forward"] + self.assertIs(func, graph["forward"][0]) + self.assertEqual(kwargs, {}) + self.assertIs(args[0], original_ref) + + def test_sets_and_frozensets_are_rewritten(self): + graph = { + "seed": (lambda: 1,), + "consumer": ( + lambda payload, *, meta=None: (payload, meta), + ( + { + "set_refs": {"seed", "literal"}, + "froze_refs": frozenset({"seed"}), + }, + ), + { + "meta": { + "labels": {"seed", "plain"}, + "deep": frozenset({"seed"}), + } + }, + ), + } + + adapted = Adaptor(graph).task_dict + func, args, kwargs = adapted["consumer"] + self.assertEqual(len(args), 1) + + self.assertIsInstance(args[0], tuple) + payload = args[0][0] + set_refs = payload["set_refs"] + self.assertIsInstance(set_refs, set) + self.assertIn("literal", set_refs) + refs = [item for item in set_refs if isinstance(item, TaskOutputRef)] + self.assertEqual(len(refs), 1) + self.assertEqual(refs[0].task_key, "seed") + + froze_refs = payload["froze_refs"] + self.assertIsInstance(froze_refs, frozenset) + sole_ref = next(iter(froze_refs)) + self.assertIsInstance(sole_ref, TaskOutputRef) + self.assertEqual(sole_ref.task_key, "seed") + + labels = kwargs["meta"]["labels"] + self.assertIsInstance(labels, set) + label_refs = [item for item in labels if isinstance(item, TaskOutputRef)] + self.assertEqual(len(label_refs), 1) + self.assertEqual(label_refs[0].task_key, "seed") + self.assertIn("plain", labels) + + deep_froze = kwargs["meta"]["deep"] + self.assertIsInstance(deep_froze, frozenset) + deep_ref = next(iter(deep_froze)) + self.assertIsInstance(deep_ref, TaskOutputRef) + self.assertEqual(deep_ref.task_key, "seed") + + def test_callable_tuple_preserves_callable(self): + def source(): + return 2 + + def apply(func_tuple): + fn, value = func_tuple + return fn(value) + + increment = lambda x: x + 1 # noqa: E731 + + graph = { + "value": (source,), + "result": ( + apply, + (increment, "value"), + ), + } + + adapted = Adaptor(graph).task_dict + func, args, kwargs = adapted["result"] + self.assertIs(func, apply) + nested = args[0] + self.assertIsInstance(nested, tuple) + self.assertIs(nested[0], increment) + self.assertIsInstance(nested[1], TaskOutputRef) + self.assertEqual(nested[1].task_key, "value") + + def test_invalid_task_definition_raises(self): + graph = {"broken": []} + with self.assertRaises(TypeError): + Adaptor(graph) + + def test_large_graph_scaling(self): + size = 500 + + graph = {"root": (lambda: 1,)} + for i in range(1, size + 1): + key = f"node_{i}" + prev_key = "root" if i == 1 else f"node_{i - 1}" + graph[key] = ( + lambda x, inc=1: x + inc, + prev_key, + {"inc": i}, + ) + + graph["fanout"] = ( + lambda *vals: sum(vals), + tuple(graph.keys()), + ) + + adapted = Adaptor(graph).task_dict + + self.assertEqual(len(adapted), len(graph)) + + fanout_func, fanout_args, fanout_kwargs = adapted["fanout"] + self.assertEqual(fanout_kwargs, {}) + self.assertEqual(len(fanout_args), 1) + arg_tuple = fanout_args[0] + self.assertEqual(len(arg_tuple), len(graph) - 1) + refs = [item for item in arg_tuple if isinstance(item, TaskOutputRef)] + self.assertEqual(len(refs), len(graph) - 1) + ref_keys = {ref.task_key for ref in refs} + expected_keys = set(graph.keys()) - {"fanout"} + self.assertEqual(ref_keys, expected_keys) + + +class _FakeGraphNode: + __slots__ = () + + +@dataclass +class _FakeTaskRef(_FakeGraphNode): + key: str + path: tuple = field(default_factory=tuple) + + +@dataclass +class _FakeAlias(_FakeGraphNode): + target: str + path: tuple = field(default_factory=tuple) + dependencies: frozenset = field(default_factory=frozenset) + + def __post_init__(self): + if not self.dependencies: + self.dependencies = frozenset({self.target}) + + +@dataclass +class _FakeLiteral(_FakeGraphNode): + value: object + + +@dataclass +class _FakeDataNode(_FakeGraphNode): + value: object + + +@dataclass +class _FakeNestedContainer(_FakeGraphNode): + value: object + + +@dataclass +class _FakeTask(_FakeGraphNode): + key: str + function: object + args: tuple = field(default_factory=tuple) + kwargs: dict = field(default_factory=dict) + dependencies: frozenset = field(default_factory=frozenset) + + def __post_init__(self): + if not self.dependencies: + deps = set() + for arg in self.args: + if isinstance(arg, _FakeTaskRef): + deps.add(arg.key) + for value in self.kwargs.values(): + if isinstance(value, _FakeTaskRef): + deps.add(value.key) + self.dependencies = frozenset(deps) + + +class _FakeDtsModule: + GraphNode = _FakeGraphNode + Task = _FakeTask + TaskRef = _FakeTaskRef + Alias = _FakeAlias + Literal = _FakeLiteral + DataNode = _FakeDataNode + NestedContainer = _FakeNestedContainer + + @staticmethod + def convert_legacy_graph(task_dict): + return task_dict + + +class AdaptorTaskSpecTests(unittest.TestCase): + @classmethod + def setUpClass(cls): + cls._original_dts = adaptor_impl.dts + adaptor_impl.dts = _FakeDtsModule() + + @classmethod + def tearDownClass(cls): + adaptor_impl.dts = cls._original_dts + + def test_adapts_taskspec_graph(self): + def add_bonus(value, bonus): + return value + bonus + + def combine(upstream, payload=None): + return upstream, payload + + graph = { + "raw": _FakeDataNode(value=7), + "literal_wrapper": _FakeTask( + key="literal_wrapper", + function=add_bonus, + args=(_FakeTaskRef("raw"),), + kwargs={"bonus": _FakeLiteral(3)}, + ), + "alias": _FakeAlias(target="literal_wrapper", path=("result",)), + "aggregate": _FakeTask( + key="aggregate", + function=combine, + args=(_FakeTaskRef("alias", path=("payload",)),), + kwargs={ + "payload": _FakeNestedContainer( + value=[ + _FakeTaskRef("raw"), + {"inner": _FakeTaskRef("alias", path=("extra",))}, + ] + ) + }, + ), + } + + adaptor = Adaptor(graph) + adapted = adaptor.task_dict + + raw_func, raw_args, raw_kwargs = adapted["raw"] + self.assertEqual(raw_args, (7,)) + self.assertEqual(raw_kwargs, {}) + self.assertEqual(raw_func(raw_args[0]), raw_args[0]) + + wrapper_func, wrapper_args, wrapper_kwargs = adapted["literal_wrapper"] + self.assertIs(wrapper_func, add_bonus) + self.assertEqual(len(wrapper_args), 1) + self.assertIsInstance(wrapper_args[0], TaskOutputRef) + self.assertEqual(wrapper_args[0].task_key, "raw") + self.assertEqual(wrapper_kwargs["bonus"], 3) + + self.assertIn("alias", adapted) + alias_func, alias_args, alias_kwargs = adapted["alias"] + self.assertIs(alias_func, adaptor_impl._identity) + self.assertEqual(alias_kwargs, {}) + self.assertEqual(len(alias_args), 1) + alias_input = alias_args[0] + self.assertIsInstance(alias_input, TaskOutputRef) + self.assertEqual(alias_input.task_key, "literal_wrapper") + self.assertEqual(alias_input.path, ("result",)) + + agg_func, agg_args, agg_kwargs = adapted["aggregate"] + self.assertIs(agg_func, combine) + self.assertEqual(len(agg_args), 1) + primary_input = agg_args[0] + self.assertIsInstance(primary_input, TaskOutputRef) + self.assertEqual(primary_input.task_key, "alias") + self.assertEqual(primary_input.path, ("payload",)) + + payload = agg_kwargs["payload"] + self.assertIsInstance(payload, list) + self.assertIsInstance(payload[0], TaskOutputRef) + self.assertEqual(payload[0].task_key, "raw") + nested_inner = payload[1]["inner"] + self.assertIsInstance(nested_inner, TaskOutputRef) + self.assertEqual(nested_inner.task_key, "alias") + self.assertEqual(nested_inner.path, ("extra",)) + + def test_taskspec_data_node_literal_passthrough(self): + graph = {"literal": _FakeDataNode(value=11)} + adapted = Adaptor(graph).task_dict + func, args, kwargs = adapted["literal"] + self.assertEqual(args, (11,)) + self.assertEqual(kwargs, {}) + self.assertEqual(func(args[0]), 11) + + def test_taskspec_alias_with_missing_target_raises(self): + alias = _FakeAlias(target="ghost", dependencies=frozenset()) + alias.dependencies = frozenset() + graph = {"alias": alias} + with self.assertRaises(ValueError): + Adaptor(graph) + + def test_taskspec_nested_container_fallback_to_data(self): + container = _FakeNestedContainer(value=None) + container.data = [_FakeTaskRef("raw")] + graph = { + "raw": _FakeDataNode(value=5), + "use_container": _FakeTask( + key="use_container", + function=lambda payload: payload, + args=(container,), + ), + } + + adapted = Adaptor(graph).task_dict + func, args, kwargs = adapted["use_container"] + self.assertEqual(kwargs, {}) + (payload,) = args + self.assertIsInstance(payload, list) + self.assertIsInstance(payload[0], TaskOutputRef) + self.assertEqual(payload[0].task_key, "raw") + + def test_taskspec_task_missing_function_raises(self): + graph = { + "broken": _FakeTask( + key="broken", + function=None, + args=(), + ) + } + with self.assertRaises(TypeError): + Adaptor(graph) + + def test_taskspec_task_nested_inside_args_is_lifted(self): + # Inline Tasks inside args/kwargs should be lifted unless they are a top-level key + # reference or a pure value constructor. Here `inner` is an inline task with + # an unknown (lambda) op, so it must be lifted to a new node. + inner = _FakeTask( + key=None, + function=lambda: 1, + args=(), + ) + graph = { + "outer": _FakeTask( + key="outer", + function=lambda x: x, + args=(inner,), + ) + } + adapted = Adaptor(graph).task_dict + outer_func, outer_args, outer_kwargs = adapted["outer"] + self.assertEqual(outer_kwargs, {}) + self.assertEqual(len(outer_args), 1) + self.assertIsInstance(outer_args[0], TaskOutputRef) + lifted_key = outer_args[0].task_key + self.assertIn(lifted_key, adapted) + self.assertNotEqual(lifted_key, "outer") + + def test_taskspec_identity_cast_is_structurally_reduced(self): + # Ensure we never execute dask private identity-cast during adaptation. + def fake_identity_cast(x, *_, **__): + raise RuntimeError("must not be executed") + + fake_identity_cast.__name__ = "_identity_cast" + fake_identity_cast.__module__ = "dask._fake" + + graph = { + "raw": _FakeDataNode(value=5), + "outer": _FakeTask( + key="outer", + function=lambda x: x, + args=( + _FakeTask( + key=None, + function=fake_identity_cast, + args=(_FakeTaskRef("raw"),), + ), + ), + ), + } + + adapted = Adaptor(graph).task_dict + _, outer_args, outer_kwargs = adapted["outer"] + self.assertEqual(outer_kwargs, {}) + self.assertEqual(len(outer_args), 1) + self.assertIsInstance(outer_args[0], TaskOutputRef) + self.assertEqual(outer_args[0].task_key, "raw") + self.assertFalse(any(str(k).startswith("__lift__") for k in adapted.keys())) + + +if __name__ == "__main__": + unittest.main() diff --git a/taskvine/src/graph/dagvine/blueprint_graph/blueprint_graph.py b/taskvine/src/graph/dagvine/blueprint_graph/blueprint_graph.py new file mode 100644 index 0000000000..4439a76d42 --- /dev/null +++ b/taskvine/src/graph/dagvine/blueprint_graph/blueprint_graph.py @@ -0,0 +1,197 @@ +# 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 + + +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}") + + +class TaskOutputRef: + __slots__ = ("task_key", "path") + + def __init__(self, task_key, path=()): + self.task_key = task_key + self.path = tuple(path) + + def __getitem__(self, key): + if isinstance(key, tuple): + return TaskOutputRef(self.task_key, self.path + key) + return TaskOutputRef(self.task_key, self.path + (key,)) + + +class BlueprintGraph: + + _LEAF_TYPES = (str, bytes, bytearray, memoryview, int, float, bool, type(None)) + + def __init__(self): + self.task_dict = {} # task_key -> (func, frozen_args, frozen_kwargs) + + self.parents_of = defaultdict(set) # task_key -> set of task_keys + self.children_of = defaultdict(set) # task_key -> set of task_keys + + self.producer_of = {} # filename -> task_key + self.consumers_of = defaultdict(set) # filename -> set of task_keys + + self.outfile_remote_name = defaultdict(lambda: None) # task_key -> remote outfile name, will be set by vine graph + + self.pykey2cid = {} # py_key -> c_id + self.cid2pykey = {} # c_id -> py_key + + 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.task_key) + return None + + self._visit_task_output_refs(obj, on_ref, rewrite=False) + return parents + + def add_task(self, task_key, func, *args, **kwargs): + if task_key in self.task_dict: + raise ValueError(f"Task {task_key} already exists") + + self.task_dict[task_key] = (func, args, kwargs) + + parents = self._find_parents(args) | self._find_parents(kwargs) + + for parent in parents: + self.parents_of[task_key].add(parent) + self.children_of[parent].add(task_key) + + def task_produces(self, task_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] = task_key + + def task_consumes(self, task_key, *filenames): + for filename in filenames: + # a file can be consumed by multiple tasks + self.consumers_of[filename].add(task_key) + + def save_task_output(self, task_key, output): + with open(self.outfile_remote_name[task_key], "wb") as f: + wrapped_output = TaskOutputWrapper(output, extra_size_mb=0) + cloudpickle.dump(wrapped_output, f) + + def load_task_output(self, task_key): + return TaskOutputWrapper.load_from_path(self.outfile_remote_name[task_key]) + + def get_topological_order(self): + indegree = {} + for task_key in self.task_dict: + indegree[task_key] = len(self.parents_of.get(task_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 verify_topo(g, topo): + pos = {k: i for i, k in enumerate(topo)} + for child, parents in g.parents_of.items(): + for p in parents: + if pos[p] > pos[child]: + raise AssertionError(f"bad topo: parent {p} after child {child}") + print("topo verified: ok") + + def finalize(self): + 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/graph/dagvine/blueprint_graph/proxy_functions.py b/taskvine/src/graph/dagvine/blueprint_graph/proxy_functions.py new file mode 100644 index 0000000000..7c3c5ecf4f --- /dev/null +++ b/taskvine/src/graph/dagvine/blueprint_graph/proxy_functions.py @@ -0,0 +1,51 @@ +# 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.utils import load_variable_from_library + + +def compute_task(bg, task_expr): + func, args, kwargs = task_expr + 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.task_key) + if x is None: + x = bg.load_task_output(r.task_key) + cache[r.task_key] = x + if r.path: + return _follow_path(x, r.path) + return x + + r_args = bg._visit_task_output_refs(args, on_ref, rewrite=True) + r_kwargs = bg._visit_task_output_refs(kwargs, on_ref, rewrite=True) + + print(f"func: {func}") + print(f"r_args: {r_args}") + print(f"r_kwargs: {r_kwargs}") + + return func(*r_args, **r_kwargs) + + +def compute_single_key(vine_key): + bg = load_variable_from_library('graph') + + task_key = bg.cid2pykey[vine_key] + task_expr = bg.task_dict[task_key] + + output = compute_task(bg, task_expr) + + bg.save_task_output(task_key, output) diff --git a/taskvine/src/graph/dagvine/blueprint_graph/proxy_library.py b/taskvine/src/graph/dagvine/blueprint_graph/proxy_library.py new file mode 100644 index 0000000000..78a7b71ca8 --- /dev/null +++ b/taskvine/src/graph/dagvine/blueprint_graph/proxy_library.py @@ -0,0 +1,94 @@ +# 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 ndcctools.taskvine.dagvine.blueprint_graph.blueprint_graph import BlueprintGraph, TaskOutputRef, TaskOutputWrapper +from ndcctools.taskvine.dagvine.blueprint_graph.proxy_functions import compute_single_key +from ndcctools.taskvine.utils import load_variable_from_library + + +class ProxyLibrary: + 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, BlueprintGraph, TaskOutputRef, TaskOutputWrapper, uuid, hashlib, random, types, collections, time, + load_variable_from_library, compute_single_key + ] + + # 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 get_context_size(self): + dumped_data = self.context_loader_args[0] + serialized = round(len(dumped_data) / 1024 / 1024, 2) + return serialized + + 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, + compute_single_key, + 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/graph/dagvine/dagvine.py b/taskvine/src/graph/dagvine/dagvine.py index e060dcbf3d..663102daec 100644 --- a/taskvine/src/graph/dagvine/dagvine.py +++ b/taskvine/src/graph/dagvine/dagvine.py @@ -5,6 +5,7 @@ from ndcctools.taskvine import cvine from ndcctools.taskvine.manager import Manager +from ndcctools.taskvine.dagvine.blueprint_graph.adaptor import Adaptor from ndcctools.taskvine.dagvine.blueprint_graph.proxy_library import ProxyLibrary from ndcctools.taskvine.dagvine.blueprint_graph.proxy_functions import compute_single_key from ndcctools.taskvine.dagvine.blueprint_graph.blueprint_graph import BlueprintGraph, TaskOutputWrapper @@ -15,21 +16,6 @@ import signal import json -try: - import dask -except ImportError: - dask = None - -try: - from dask.base import is_dask_collection -except ImportError: - is_dask_collection = None - -try: - import dask._task_spec as dts -except ImportError: - dts = None - def context_loader_func(graph_pkl): graph = cloudpickle.loads(graph_pkl) @@ -58,50 +44,6 @@ def color_text(text, color_code): return f"\033[{color_code}m{text}\033[0m" -# Flatten Dask collections into the dict-of-tasks structure the rest of the -# pipeline expects. DAGVine clients often hand us a dict like -# {"result": dask.delayed(...)}; we merge the underlying HighLevelGraphs so -# `ContextGraph` sees the same dict representation C does. -def dask_collections_to_task_dict(collection_dict): - """Merge user-facing Dask collections into the flattened task dict the ContextGraph expects.""" - assert is_dask_collection is not None - from dask.highlevelgraph import HighLevelGraph, ensure_dict - - if not isinstance(collection_dict, dict): - raise TypeError("Input must be a dict") - - for k, v in collection_dict.items(): - if not is_dask_collection(v): - raise TypeError(f"Input must be a dict of DaskCollection, but found {k} with type {type(v)}") - - if dts: - # the new Dask API - sub_hlgs = [v.dask for v in collection_dict.values()] - hlg = HighLevelGraph.merge(*sub_hlgs).to_dict() - else: - # the old Dask API - hlg = dask.base.collections_to_dsk(collection_dict.values()) - - return ensure_dict(hlg) - - -# Accept both plain dicts and Dask collections from callers. Most library users -# hand us `{key: delayed / value}` directly, while some experiments pass a -# fully-expanded legacy Dask dict. This helper normalises both cases so the rest -# of the pipeline only deals with `{task_key: task_expression}`. -def normalize_task_dict(collection_dict): - """Normalize user input (raw dict or Dask collection) into a plain `{task_key: expr}` mapping.""" - if is_dask_collection and any(is_dask_collection(v) for v in collection_dict.values()): - task_dict = dask_collections_to_task_dict(collection_dict) - else: - task_dict = collection_dict - - if dts: - return dts.convert_legacy_graph(task_dict) - else: - return task_dict - - class GraphParams: def __init__(self): """Hold all tweakable knobs (manager-side, vine_graph-side, and misc).""" @@ -179,13 +121,13 @@ def __init__(self, # Ensure run-info templates don't accumulate garbage between runs. 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: delete_all_files(self.run_info_template_path) # Boot the underlying TaskVine manager. The TaskVine manager keeps alive until the dagvine object is destroyed super().__init__(*args, **kwargs) - print(f"cvine = {cvine}") self.runtime_directory = cvine.vine_get_runtime_directory(self._taskvine) print(f"=== Manager name: {color_text(self.name, 92)}") @@ -232,18 +174,10 @@ def build_context_graph(self, task_dict): return context_graph def build_blueprint_graph(self, task_dict): - def _unpack_sexpr(sexpr): - func = sexpr[0] - tail = sexpr[1:] - if tail and isinstance(tail[-1], dict): - return func, tail[:-1], tail[-1] - else: - return func, tail, {} - bg = BlueprintGraph() for k, v in task_dict.items(): - func, args, kwargs = _unpack_sexpr(v) + func, args, kwargs = v assert callable(func), f"Task {k} does not have a callable" bg.add_task(k, func, *args, **kwargs) @@ -320,7 +254,7 @@ def run(self, collection_dict, target_keys=[], params={}, hoisting_modules=[], e print(f" {k}") target_keys = list(set(target_keys) - set(missing_keys)) - task_dict = normalize_task_dict(collection_dict) + task_dict = Adaptor(collection_dict).task_dict # Build both the Python DAG and its C mirror. py_graph, vine_graph = self.build_graphs(task_dict, target_keys, blueprint_graph=True) From 2a6d2caf7a37e3b3de573a2a1b2147c53d2d404a Mon Sep 17 00:00:00 2001 From: Jin Zhou Date: Thu, 8 Jan 2026 10:24:05 -0500 Subject: [PATCH 103/113] revert vine_worker --- taskvine/src/worker/vine_worker.c | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/taskvine/src/worker/vine_worker.c b/taskvine/src/worker/vine_worker.c index 97a62695d3..ee5f404a8e 100644 --- a/taskvine/src/worker/vine_worker.c +++ b/taskvine/src/worker/vine_worker.c @@ -1704,7 +1704,7 @@ static void vine_worker_serve_manager(struct link *manager) hence a maximum wait time of five seconds is enforced. */ - int wait_msec = 0; + int wait_msec = 5000; if (sigchld_received_flag) { wait_msec = 0; From 7d729aba4f4ba660bfbbe1ba6be5f40a8d5d5f36 Mon Sep 17 00:00:00 2001 From: Jin Zhou Date: Thu, 8 Jan 2026 15:54:21 -0500 Subject: [PATCH 104/113] support dask expressions --- .../graph/dagvine/blueprint_graph/.gitignore | 2 + .../graph/dagvine/blueprint_graph/adaptor.py | 72 ++++++++---- .../dagvine/blueprint_graph/adaptor_test.py | 17 ++- .../blueprint_graph/blueprint_graph.py | 11 +- .../blueprint_graph/proxy_functions.py | 7 +- taskvine/src/graph/dagvine/dagvine.py | 107 +++++++++--------- .../src/graph/dagvine/vine_graph/vine_graph.c | 7 +- 7 files changed, 134 insertions(+), 89 deletions(-) create mode 100644 taskvine/src/graph/dagvine/blueprint_graph/.gitignore diff --git a/taskvine/src/graph/dagvine/blueprint_graph/.gitignore b/taskvine/src/graph/dagvine/blueprint_graph/.gitignore new file mode 100644 index 0000000000..60123e6dfc --- /dev/null +++ b/taskvine/src/graph/dagvine/blueprint_graph/.gitignore @@ -0,0 +1,2 @@ +adaptor_test.py +__pycache__/ \ No newline at end of file diff --git a/taskvine/src/graph/dagvine/blueprint_graph/adaptor.py b/taskvine/src/graph/dagvine/blueprint_graph/adaptor.py index 3f5db28243..32d6045b8e 100644 --- a/taskvine/src/graph/dagvine/blueprint_graph/adaptor.py +++ b/taskvine/src/graph/dagvine/blueprint_graph/adaptor.py @@ -19,7 +19,7 @@ # where the private module is unavailable or type-checkers can't resolve it). dts = None -from ndcctools.taskvine.dagvine.blueprint_graph.blueprint_graph import TaskOutputRef +from ndcctools.taskvine.dagvine.blueprint_graph.blueprint_graph import TaskOutputRef, BlueprintGraph def _identity(value): @@ -32,8 +32,11 @@ class Adaptor: _LEAF_TYPES = (str, bytes, bytearray, memoryview, int, float, bool, type(None)) - def __init__(self, collection_dict): - self.original_collection_dict = collection_dict + def __init__(self, task_dict): + + if isinstance(task_dict, BlueprintGraph): + self.converted = task_dict + return # TaskSpec-only state used to "lift" inline Tasks that cannot be reduced to # a pure Python value (or would be unsafe/expensive to inline). @@ -45,23 +48,23 @@ def __init__(self, collection_dict): # lifted keys remain visible across subsequent conversions/dedup/reference checks. self._task_keys = set() - normalized = self._normalize_task_dict(collection_dict) - self.task_dict = self._convert_to_blueprint_tasks(normalized) + normalized = self._normalize_task_dict(task_dict) + self.converted = self._convert_to_blueprint_tasks(normalized) - def _normalize_task_dict(self, collection_dict): + def _normalize_task_dict(self, task_dict): """Collapse every supported input style into a classic `{key: sexpr or TaskSpec}` mapping.""" from_dask_collection = bool( - is_dask_collection and any(is_dask_collection(v) for v in collection_dict.values()) + is_dask_collection and any(is_dask_collection(v) for v in task_dict.values()) ) if from_dask_collection: - task_dict = self._dask_collections_to_task_dict(collection_dict) + task_dict = self._dask_collections_to_task_dict(task_dict) else: # IMPORTANT: treat plain user dicts as DAGVine sexprs by default. # If we unconditionally run `dask._task_spec.convert_legacy_graph(...)` when # dts is available, Dask will interpret our "final Mapping is kwargs" # convention as a positional dict argument, breaking sexpr semantics. - task_dict = dict(collection_dict) + task_dict = dict(task_dict) # Only ask Dask to rewrite legacy graphs when we *know* the input came # from a Dask collection/HLG. This keeps classic DAGVine sexprs stable @@ -227,10 +230,12 @@ def _should_wrap(self, obj, task_keys): """Decide whether a value should become a `TaskOutputRef`.""" if isinstance(obj, self._LEAF_TYPES): if isinstance(obj, str): - return obj in task_keys + hit = obj in task_keys + return hit return False try: - return obj in task_keys + hit = obj in task_keys + return hit except TypeError: return False @@ -238,25 +243,25 @@ def _should_wrap(self, obj, task_keys): # pipeline expects. DAGVine clients often hand us a dict like # {"result": dask.delayed(...)}; we merge the underlying HighLevelGraphs so # `ContextGraph` sees the same dict representation C does. - def _dask_collections_to_task_dict(self, collection_dict): + def _dask_collections_to_task_dict(self, task_dict): """Flatten Dask collections into the classic dict-of-task layout.""" assert is_dask_collection is not None from dask.highlevelgraph import HighLevelGraph, ensure_dict - if not isinstance(collection_dict, dict): + if not isinstance(task_dict, dict): raise TypeError("Input must be a dict") - for k, v in collection_dict.items(): + for k, v in task_dict.items(): if not is_dask_collection(v): raise TypeError( f"Input must be a dict of DaskCollection, but found {k} with type {type(v)}" ) if dts: - sub_hlgs = [v.dask for v in collection_dict.values()] + sub_hlgs = [v.dask for v in task_dict.values()] hlg = HighLevelGraph.merge(*sub_hlgs).to_dict() else: - hlg = dask.base.collections_to_dsk(collection_dict.values()) + hlg = dask.base.collections_to_dsk(task_dict.values()) return ensure_dict(hlg) @@ -299,18 +304,21 @@ def _unwrap_dts_operand(self, operand, task_keys, *, parent_key=None): literal_cls = getattr(dts, "Literal", None) if literal_cls and isinstance(operand, literal_cls): - return getattr(operand, "value", None) + value = getattr(operand, "value", None) + return value datanode_cls = getattr(dts, "DataNode", None) if datanode_cls and isinstance(operand, datanode_cls): - return operand.value + value = operand.value + return value nested_cls = getattr(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_dts_operand(payload, task_keys, parent_key=parent_key) + value = self._unwrap_dts_operand(payload, task_keys, parent_key=parent_key) + return value task_cls = getattr(dts, "Task", None) if task_cls and isinstance(operand, task_cls): @@ -323,15 +331,31 @@ def _unwrap_dts_operand(self, operand, task_keys, *, parent_key=None): # Otherwise it is an inline expression. Reduce if safe, else lift. func = self._extract_callable_from_task(operand) if func is None: - return self._lift_inline_task(operand, task_keys, parent_key=parent_key) + out = self._lift_inline_task(operand, task_keys, parent_key=parent_key) + return out # Special-case: Dask internal identity-cast wrappers should not be called - # during adaptation. Reduce structurally by returning the first argument. + # during adaptation. Reduce structurally by unwrapping all args and + # rebuilding the requested container type. This preserves dependency + # edges (critical for WCC) without executing arbitrary code. if self._is_identity_cast_op(func): raw_args = getattr(operand, "args", ()) or () - if not raw_args: - return None - return self._unwrap_dts_operand(raw_args[0], task_keys, parent_key=parent_key) + raw_kwargs = getattr(operand, "kwargs", {}) or {} + typ = raw_kwargs.get("typ", None) + + values = [self._unwrap_dts_operand(a, task_keys, parent_key=parent_key) for a in raw_args] + + # Only allow safe container constructors here; otherwise lift. + safe_types = (list, tuple, set, frozenset, dict) + if typ in safe_types: + try: + casted = typ(values) + except Exception: + return self._lift_inline_task(operand, task_keys, parent_key=parent_key) + return casted + + # Unknown/unsafe typ: lift so the worker executes the real op. + return self._lift_inline_task(operand, task_keys, parent_key=parent_key) if self._is_pure_value_op(func): reduced, used_lift = self._reduce_inline_task(operand, task_keys, parent_key=parent_key) diff --git a/taskvine/src/graph/dagvine/blueprint_graph/adaptor_test.py b/taskvine/src/graph/dagvine/blueprint_graph/adaptor_test.py index ce1a23fe68..d641ecea64 100644 --- a/taskvine/src/graph/dagvine/blueprint_graph/adaptor_test.py +++ b/taskvine/src/graph/dagvine/blueprint_graph/adaptor_test.py @@ -501,7 +501,8 @@ def fake_identity_cast(x, *_, **__): fake_identity_cast.__module__ = "dask._fake" graph = { - "raw": _FakeDataNode(value=5), + "raw0": _FakeDataNode(value=5), + "raw1": _FakeDataNode(value=6), "outer": _FakeTask( key="outer", function=lambda x: x, @@ -509,7 +510,11 @@ def fake_identity_cast(x, *_, **__): _FakeTask( key=None, function=fake_identity_cast, - args=(_FakeTaskRef("raw"),), + args=( + _FakeTaskRef("raw0"), + _FakeTaskRef("raw1"), + ), + kwargs={"typ": list}, ), ), ), @@ -519,8 +524,12 @@ def fake_identity_cast(x, *_, **__): _, outer_args, outer_kwargs = adapted["outer"] self.assertEqual(outer_kwargs, {}) self.assertEqual(len(outer_args), 1) - self.assertIsInstance(outer_args[0], TaskOutputRef) - self.assertEqual(outer_args[0].task_key, "raw") + self.assertIsInstance(outer_args[0], list) + self.assertEqual(len(outer_args[0]), 2) + self.assertIsInstance(outer_args[0][0], TaskOutputRef) + self.assertIsInstance(outer_args[0][1], TaskOutputRef) + self.assertEqual(outer_args[0][0].task_key, "raw0") + self.assertEqual(outer_args[0][1].task_key, "raw1") self.assertFalse(any(str(k).startswith("__lift__") for k in adapted.keys())) diff --git a/taskvine/src/graph/dagvine/blueprint_graph/blueprint_graph.py b/taskvine/src/graph/dagvine/blueprint_graph/blueprint_graph.py index 4439a76d42..0ff1c6caa1 100644 --- a/taskvine/src/graph/dagvine/blueprint_graph/blueprint_graph.py +++ b/taskvine/src/graph/dagvine/blueprint_graph/blueprint_graph.py @@ -8,6 +8,8 @@ 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 @@ -24,6 +26,7 @@ def load_from_path(path): 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__ = ("task_key", "path") @@ -37,6 +40,8 @@ def __getitem__(self, key): return TaskOutputRef(self.task_key, self.path + (key,)) +# The BlueprintGraph is a directed acyclic graph (DAG) that represents the logical dependencies between tasks. +# It is used to build the C vine graph. class BlueprintGraph: _LEAF_TYPES = (str, bytes, bytearray, memoryview, int, float, bool, type(None)) @@ -55,6 +60,9 @@ def __init__(self): self.pykey2cid = {} # py_key -> c_id self.cid2pykey = {} # c_id -> py_key + self.extra_task_output_size_mb = {} # task_key -> extra size in MB + self.extra_task_sleep_time = {} # task_key -> extra sleep time in seconds + def _visit_task_output_refs(self, obj, on_ref, *, rewrite: bool): seen = set() @@ -154,7 +162,7 @@ def task_consumes(self, task_key, *filenames): def save_task_output(self, task_key, output): with open(self.outfile_remote_name[task_key], "wb") as f: - wrapped_output = TaskOutputWrapper(output, extra_size_mb=0) + wrapped_output = TaskOutputWrapper(output, extra_size_mb=self.extra_task_output_size_mb[task_key]) cloudpickle.dump(wrapped_output, f) def load_task_output(self, task_key): @@ -191,6 +199,7 @@ def verify_topo(g, topo): print("topo verified: ok") 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) diff --git a/taskvine/src/graph/dagvine/blueprint_graph/proxy_functions.py b/taskvine/src/graph/dagvine/blueprint_graph/proxy_functions.py index 7c3c5ecf4f..0c50e6201c 100644 --- a/taskvine/src/graph/dagvine/blueprint_graph/proxy_functions.py +++ b/taskvine/src/graph/dagvine/blueprint_graph/proxy_functions.py @@ -4,6 +4,7 @@ from ndcctools.taskvine.utils import load_variable_from_library +import time def compute_task(bg, task_expr): @@ -33,10 +34,6 @@ def on_ref(r): r_args = bg._visit_task_output_refs(args, on_ref, rewrite=True) r_kwargs = bg._visit_task_output_refs(kwargs, on_ref, rewrite=True) - print(f"func: {func}") - print(f"r_args: {r_args}") - print(f"r_kwargs: {r_kwargs}") - return func(*r_args, **r_kwargs) @@ -48,4 +45,6 @@ def compute_single_key(vine_key): output = compute_task(bg, task_expr) + time.sleep(bg.extra_task_sleep_time[task_key]) + bg.save_task_output(task_key, output) diff --git a/taskvine/src/graph/dagvine/dagvine.py b/taskvine/src/graph/dagvine/dagvine.py index 663102daec..1869e25540 100644 --- a/taskvine/src/graph/dagvine/dagvine.py +++ b/taskvine/src/graph/dagvine/dagvine.py @@ -15,6 +15,7 @@ import os import signal import json +import random def context_loader_func(graph_pkl): @@ -133,6 +134,7 @@ def __init__(self, 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 param(self, param_name): """Convenience accessor so callers can read tuned parameters at runtime.""" @@ -163,23 +165,18 @@ def tune_vine_graph(self, vine_graph): for k, v in self.params.vine_graph_tuning_params.items(): vine_graph.tune(k, str(v)) - def build_context_graph(self, task_dict): - """Construct the Python-side DAG wrapper (ContextGraph).""" - context_graph = ContextGraph( - task_dict, - extra_task_output_size_mb=self.param("extra-task-output-size-mb"), - extra_task_sleep_time=self.param("extra-task-sleep-time") - ) - - return context_graph - def build_blueprint_graph(self, task_dict): - bg = BlueprintGraph() + if isinstance(task_dict, BlueprintGraph): + bg = task_dict + else: + bg = BlueprintGraph() - for k, v in task_dict.items(): - func, args, kwargs = v - assert callable(func), f"Task {k} does not have a callable" - bg.add_task(k, func, *args, **kwargs) + for k, v in task_dict.items(): + func, args, kwargs = v + assert callable(func), f"Task {k} does not have a callable" + bg.add_task(k, func, *args, **kwargs) + + bg.finalize() return bg @@ -213,14 +210,18 @@ def build_vine_graph(self, py_graph, target_keys): return vine_graph - def build_graphs(self, task_dict, target_keys, blueprint_graph=False): - """Create both the ContextGraph and its C counterpart, wiring outputs for later use.""" - # Build the logical (Python) DAG. - if blueprint_graph == False: - py_graph = self.build_context_graph(task_dict) - else: - py_graph = self.build_blueprint_graph(task_dict) - # Build the physical (C) DAG. + def build_graphs(self, task_dict, target_keys): + """Create both the python side graph and its C counterpart, wiring outputs for later use.""" + # Build the python side graph. + py_graph = self.build_blueprint_graph(task_dict) + + # filter out target keys that are not in the collection dict + 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)) + + # Build the c side graph. vine_graph = self.build_vine_graph(py_graph, target_keys) # Cross-fill the outfile locations so the runtime graph knows where to read/write. @@ -231,7 +232,7 @@ def build_graphs(self, task_dict, target_keys, blueprint_graph=False): return py_graph, vine_graph def create_proxy_library(self, py_graph, vine_graph, hoisting_modules, env_files): - """Package up the context_graph as a TaskVine library.""" + """Package up the python side graph as a TaskVine library.""" proxy_library = ProxyLibrary(self) proxy_library.add_hoisting_modules(hoisting_modules) proxy_library.add_env_files(env_files) @@ -241,46 +242,42 @@ def create_proxy_library(self, py_graph, vine_graph, hoisting_modules, env_files return proxy_library - def run(self, collection_dict, target_keys=[], params={}, hoisting_modules=[], env_files={}): + def run(self, task_dict, target_keys=[], params={}, hoisting_modules=[], env_files={}, adapt_dask=False): """High-level entry point: normalise input, build graphs, ship the library, execute, and return results.""" # first update the params so that they can be used for the following construction self.update_params(params) - # filter out target keys that are not in the collection dict - missing_keys = [k for k in target_keys if k not in collection_dict] - if missing_keys: - print(f"=== Warning: the following target keys are not in the collection dict:") - for k in missing_keys: - print(f" {k}") - target_keys = list(set(target_keys) - set(missing_keys)) - - task_dict = Adaptor(collection_dict).task_dict + if adapt_dask: + task_dict = Adaptor(task_dict).converted # Build both the Python DAG and its C mirror. - py_graph, vine_graph = self.build_graphs(task_dict, target_keys, blueprint_graph=True) + py_graph, vine_graph = self.build_graphs(task_dict, target_keys) - # Ship the execution context to workers via a proxy library. + # set extra task output size and sleep time for each task + for k in py_graph.task_dict: + py_graph.extra_task_output_size_mb[k] = random.uniform(*self.param("extra-task-output-size-mb")) + py_graph.extra_task_sleep_time[k] = random.uniform(*self.param("extra-task-sleep-time")) + + # Ship the execution context to workers via a proxy library proxy_library = self.create_proxy_library(py_graph, vine_graph, hoisting_modules, env_files) proxy_library.install() - print(f"=== Library serialized size: {color_text(proxy_library.get_context_size(), 92)} MB") - - # Kick off execution on the C side. - vine_graph.execute() - - # Tear down once we're done so successive runs start clean. - proxy_library.uninstall() - - # Delete the C graph immediately so its lifetime matches the run. - vine_graph.delete() - - # Load any requested target outputs back into Python land. - results = {} - for k in target_keys: - outfile_path = os.path.join(self.param("output-dir"), py_graph.outfile_remote_name[k]) - results[k] = TaskOutputWrapper.load_from_path(outfile_path) - return results + try: + print(f"=== Library serialized size: {color_text(proxy_library.get_context_size(), 92)} MB") + vine_graph.execute() + results = {} + for k in target_keys: + if k not in py_graph.task_dict: + continue + outfile_path = os.path.join(self.param("output-dir"), py_graph.outfile_remote_name[k]) + results[k] = TaskOutputWrapper.load_from_path(outfile_path) + return results + finally: + try: + proxy_library.uninstall() + finally: + vine_graph.delete() def _on_sigint(self, signum, frame): - """SIGINT handler that delegates to Manager cleanup so workers are released promptly.""" - self.__del__() + self._sigint_received = True + raise KeyboardInterrupt diff --git a/taskvine/src/graph/dagvine/vine_graph/vine_graph.c b/taskvine/src/graph/dagvine/vine_graph/vine_graph.c index 4f7ada468b..3a582aa0ed 100644 --- a/taskvine/src/graph/dagvine/vine_graph/vine_graph.c +++ b/taskvine/src/graph/dagvine/vine_graph/vine_graph.c @@ -1112,7 +1112,7 @@ void vine_graph_execute(struct vine_graph *vg) return; } - signal(SIGINT, handle_sigint); + void (*previous_sigint_handler)(int) = signal(SIGINT, handle_sigint); debug(D_VINE, "start executing vine graph"); @@ -1328,6 +1328,11 @@ void vine_graph_execute(struct vine_graph *vg) print_time_metrics(vg, vg->time_metrics_filename); } + signal(SIGINT, previous_sigint_handler); + if (interrupted) { + raise(SIGINT); + } + return; } From a7d05b19891b85f387692e5cd4864f2af5d10f93 Mon Sep 17 00:00:00 2001 From: Jin Zhou Date: Thu, 8 Jan 2026 16:10:56 -0500 Subject: [PATCH 105/113] gitignore --- taskvine/src/graph/dagvine/.gitignore | 3 +- .../src/graph/dagvine/context_graph/Makefile | 31 --- .../graph/dagvine/context_graph/__init__.py | 18 -- .../src/graph/dagvine/context_graph/core.py | 183 ------------------ .../dagvine/context_graph/proxy_functions.py | 111 ----------- .../dagvine/context_graph/proxy_library.py | 94 --------- 6 files changed, 2 insertions(+), 438 deletions(-) delete mode 100644 taskvine/src/graph/dagvine/context_graph/Makefile delete mode 100644 taskvine/src/graph/dagvine/context_graph/__init__.py delete mode 100644 taskvine/src/graph/dagvine/context_graph/core.py delete mode 100644 taskvine/src/graph/dagvine/context_graph/proxy_functions.py delete mode 100644 taskvine/src/graph/dagvine/context_graph/proxy_library.py diff --git a/taskvine/src/graph/dagvine/.gitignore b/taskvine/src/graph/dagvine/.gitignore index 38280b8491..25000a1241 100644 --- a/taskvine/src/graph/dagvine/.gitignore +++ b/taskvine/src/graph/dagvine/.gitignore @@ -3,4 +3,5 @@ *.o *_wrap.c *_wrap.0 -build/ \ No newline at end of file +build/ +context_graph/ \ No newline at end of file diff --git a/taskvine/src/graph/dagvine/context_graph/Makefile b/taskvine/src/graph/dagvine/context_graph/Makefile deleted file mode 100644 index a356267995..0000000000 --- a/taskvine/src/graph/dagvine/context_graph/Makefile +++ /dev/null @@ -1,31 +0,0 @@ -include ../../../../../config.mk -include ../../../../../rules.mk - -PROJECT_NAME = dagvine - -SOURCE_DIR = $(CCTOOLS_HOME)/taskvine/src/graph/$(PROJECT_NAME)/context_graph -MODULE_ROOT = $(CCTOOLS_PYTHON3_PATH)/ndcctools/taskvine/$(PROJECT_NAME) -MODULE_DIR = $(MODULE_ROOT)/context_graph - -PY_SOURCES = $(wildcard $(SOURCE_DIR)/*.py) - -.PHONY: all install clean lint format - -all: - @true - -install: - mkdir -p $(MODULE_DIR) - cp $(PY_SOURCES) $(MODULE_DIR) - -clean: - rm -rf __pycache__ - -lint: - flake8 --ignore=$(CCTOOLS_FLAKE8_IGNORE_ERRORS) \ - --exclude=$(CCTOOLS_FLAKE8_IGNORE_FILES) \ - $(SOURCE_DIR)/ - -format: - @true - diff --git a/taskvine/src/graph/dagvine/context_graph/__init__.py b/taskvine/src/graph/dagvine/context_graph/__init__.py deleted file mode 100644 index 7d8b678cc5..0000000000 --- a/taskvine/src/graph/dagvine/context_graph/__init__.py +++ /dev/null @@ -1,18 +0,0 @@ -# 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 .core import ContextGraph, ContextGraphTaskResult -from .proxy_functions import compute_single_key, compute_dts_key, compute_sexpr_key -from .proxy_library import ProxyLibrary - - -__all__ = [ - "ContextGraph", - "ContextGraphTaskResult", - "compute_single_key", - "compute_dts_key", - "compute_sexpr_key", - "ProxyLibrary", -] diff --git a/taskvine/src/graph/dagvine/context_graph/core.py b/taskvine/src/graph/dagvine/context_graph/core.py deleted file mode 100644 index 9d0f3ac1c4..0000000000 --- a/taskvine/src/graph/dagvine/context_graph/core.py +++ /dev/null @@ -1,183 +0,0 @@ -# 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 cloudpickle -import collections -import random -from collections import deque - -# Attempt to import Dask helpers. When they are unavailable we fall back to -# None so environments without Dask continue to work. If Dask is present—either -# the legacy graph or the newer TaskSpec API—we normalize it into our internal -# task representation. -try: - import dask -except ImportError: - dask = None -try: - import dask._task_spec as dts -except ImportError: - dts = None - - -def hashable(s): - """Used while wiring dependencies to spot values we can treat as node keys.""" - try: - hash(s) - return True - except TypeError: - return False - - -# Lightweight wrapper around task results that optionally pads the payload. The -# padding lets tests model large outputs without altering the logical result. -class ContextGraphTaskResult: - def __init__(self, result, extra_size_mb=None): - """Store the real user result plus optional padding used during regression tests.""" - 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): - """Workers call this while recovering an output produced by save_result_of_key from disk. - If a node-local output, then data is stored in the task sandbox and the path is just the filename - If a shared file system output, then path is the full path to the file - If a target result, the path is the full path to the file in the manager's output directory""" - try: - with open(path, "rb") as f: - result_obj = cloudpickle.load(f) - assert isinstance(result_obj, ContextGraphTaskResult), "Loaded object is not of type ContextGraphTaskResult" - return result_obj.result - except FileNotFoundError: - raise FileNotFoundError(f"Output file not found at {path}") - - -# ContextGraph builds the logical DAG and manages dependencies. The -# object is cloudpickled, shipped with the proxy library, and hoisted on worker -# nodes. When a task key executes we map from the Vine key back to the original -# graph key, run the user function, and persist the result. -class ContextGraph: - def __init__(self, task_dict, - extra_task_output_size_mb=[0, 0], - extra_task_sleep_time=[0, 0]): - """Capture the Python DAG that DAGVine hands us before we mirror it in C.""" - self.task_dict = task_dict - - if dts: - for k, v in self.task_dict.items(): - if isinstance(v, dts.GraphNode): - assert isinstance(v, (dts.Alias, dts.Task, dts.DataNode)), f"Unsupported task type for key {k}: {v.__class__}" - - self.parents_of, self.children_of = self._build_dependencies(self.task_dict) - - # these mappings are set after node ids are assigned in the C vine graph - self.pykey2cid = {} - self.cid2pykey = {} - - # will be set from vine graph - self.outfile_remote_name = {key: None for key in self.task_dict.keys()} - - # testing params - self.extra_task_output_size_mb = self._calculate_extra_size_mb_of(extra_task_output_size_mb) - self.extra_sleep_time_of = self._calculate_extra_sleep_time_of(extra_task_sleep_time) - - def _calculate_extra_size_mb_of(self, extra_task_output_size_mb): - """Sample a uniform byte budget between low/high for every node.""" - assert isinstance(extra_task_output_size_mb, list) and len(extra_task_output_size_mb) == 2 - low, high = extra_task_output_size_mb - low, high = int(low), int(high) - assert low <= high - - return {k: random.uniform(low, high) for k in self.task_dict.keys()} - - def _calculate_extra_sleep_time_of(self, extra_task_sleep_time): - """Pick a uniform delay between low/high so tests can fake runtime.""" - assert isinstance(extra_task_sleep_time, list) and len(extra_task_sleep_time) == 2 - low, high = extra_task_sleep_time - low, high = int(low), int(high) - assert low <= high - - return {k: random.uniform(low, high) for k in self.task_dict.keys()} - - def is_dts_key(self, k): - """Gate the Dask-specific branch when we parse task definitions.""" - if not hasattr(dask, "_task_spec"): - return False - import dask._task_spec as dts - return isinstance(self.task_dict[k], (dts.Task, dts.TaskRef, dts.Alias, dts.DataNode, dts.NestedContainer)) - - def _build_dependencies(self, task_dict): - """Normalize mixed Dask/s-expression inputs into our parent/child lookup tables.""" - def _find_sexpr_parents(sexpr): - """Resolve the immediate parents inside one symbolic expression node.""" - if hashable(sexpr) and sexpr in task_dict.keys(): - return {sexpr} - elif isinstance(sexpr, (list, tuple)): - deps = set() - for x in sexpr: - deps |= _find_sexpr_parents(x) - return deps - elif isinstance(sexpr, dict): - deps = set() - for k, v in sexpr.items(): - deps |= _find_sexpr_parents(k) - deps |= _find_sexpr_parents(v) - return deps - else: - return set() - - parents_of = collections.defaultdict(set) - children_of = collections.defaultdict(set) - - for k, value in task_dict.items(): - if self.is_dts_key(k): - # in the new Dask expression, each value is an object from dask._task_spec, could be - # a Task, Alias, TaskRef, etc., but they all share the same base class the dependencies - # field is of type frozenset(), without recursive ancestor dependencies involved - parents_of[k] = value.dependencies - else: - # the value could be a sexpr, e.g., the old Dask representation - parents_of[k] = _find_sexpr_parents(value) - - for k, deps in parents_of.items(): - for dep in deps: - children_of[dep].add(k) - - return parents_of, children_of - - def save_result_of_key(self, key, result): - """Called from the proxy function to persist a result into disk after the worker finishes.""" - with open(self.outfile_remote_name[key], "wb") as f: - result_obj = ContextGraphTaskResult(result, extra_size_mb=self.extra_task_output_size_mb[key]) - cloudpickle.dump(result_obj, f) - - def load_result_of_key(self, key): - """Used by downstream tasks to pull inputs from disk or the shared store.""" - # workers user this function to load results from either local or shared file system - # if a node-local output, then data is stored in the task sandbox and the remote name is just the filename - # if a shared file system output, then remote name is the full path to the file - outfile_path = self.outfile_remote_name[key] - return ContextGraphTaskResult.load_from_path(outfile_path) - - def get_topological_order(self): - """Produce the order DAGVine uses when assigning node IDs to the C graph.""" - in_degree = {key: len(self.parents_of[key]) for key in self.task_dict.keys()} - queue = deque([key for key, degree in in_degree.items() if degree == 0]) - topo_order = [] - - while queue: - current = queue.popleft() - topo_order.append(current) - - for child in self.children_of[current]: - in_degree[child] -= 1 - if in_degree[child] == 0: - queue.append(child) - - if len(topo_order) != len(self.task_dict): - print(f"len(topo_order): {len(topo_order)}") - print(f"len(self.task_dict): {len(self.task_dict)}") - raise ValueError("Failed to create topo order, the dependencies may be cyclic or problematic") - - return topo_order diff --git a/taskvine/src/graph/dagvine/context_graph/proxy_functions.py b/taskvine/src/graph/dagvine/context_graph/proxy_functions.py deleted file mode 100644 index db14b1c356..0000000000 --- a/taskvine/src/graph/dagvine/context_graph/proxy_functions.py +++ /dev/null @@ -1,111 +0,0 @@ -# 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 time -from ndcctools.taskvine.utils import load_variable_from_library - - -def compute_dts_key(context_graph, k, v): - """ - Compute the result of a Dask task node from dask._task_spec. - - Each value `v` may be an instance of Task, Alias, or DataNode, all of which - inherit from the same base class. The `dependencies` field is a frozenset - containing direct dependencies only (no recursive ancestry). - - The function resolves each dependency from the context_graph, constructs an - input dictionary, and then executes the node according to its type. - """ - try: - import dask._task_spec as dts - except ImportError: - raise ImportError("Dask is not installed") - - input_dict = {dep: context_graph.load_result_of_key(dep) for dep in v.dependencies} - - try: - if isinstance(v, dts.Alias): - assert len(v.dependencies) == 1, "Expected exactly one dependency" - return context_graph.load_result_of_key(next(iter(v.dependencies))) - elif isinstance(v, dts.Task): - return v(input_dict) - elif isinstance(v, dts.DataNode): - return v.value - else: - raise TypeError(f"unexpected node type: {type(v)} for key {k}") - except Exception as e: - raise Exception(f"Error while executing task {k}: {e}") - - -def compute_sexpr_key(context_graph, k, v): - """ - Evaluate a symbolic expression (S-expression) task within the task graph. - - Both DAGVine and legacy Dask represent computations as symbolic - expression trees (S-expressions). Each task value `v` encodes a nested - structure where: - - Leaf nodes are constants or task keys referencing parent results. - - Lists are recursively evaluated. - - Tuples of the form (func, arg1, arg2, ...) represent function calls. - - This function builds an input dictionary from all parent keys, then - recursively resolves and executes the expression until a final value - is produced. - """ - input_dict = {parent: context_graph.load_result_of_key(parent) for parent in context_graph.parents_of[k]} - - def _rec_call(expr): - try: - if expr in input_dict.keys(): - return input_dict[expr] - except TypeError: - pass - if isinstance(expr, list): - return [_rec_call(e) for e in expr] - if isinstance(expr, tuple) and len(expr) > 0 and callable(expr[0]): - res = expr[0](*[_rec_call(a) for a in expr[1:]]) - return res - return expr - - try: - return _rec_call(v) - except Exception as e: - raise Exception(f"Failed to invoke _rec_call(): {e}") - - -def compute_single_key(vine_key): - """ - Compute a single task identified by a Vine key within the current ContextGraph. - - The function retrieves the corresponding graph key and task object from the - global context_graph, determines the task type, and dispatches to the appropriate - execution interface — e.g., `compute_dts_key` for Dask-style task specs or - `compute_sexpr_key` for S-expression graphs. - - This design allows extensibility: for new graph representations, additional - compute interfaces can be introduced and registered here to handle new key types. - - After computation, the result is saved, the output file is validated, and - an optional delay (`extra_sleep_time_of`) is applied before returning. - """ - context_graph = load_variable_from_library('graph') - - k = context_graph.cid2pykey[vine_key] - v = context_graph.task_dict[k] - - if context_graph.is_dts_key(k): - result = compute_dts_key(context_graph, k, v) - else: - result = compute_sexpr_key(context_graph, k, v) - - context_graph.save_result_of_key(k, result) - if not os.path.exists(context_graph.outfile_remote_name[k]): - raise Exception(f"Output file {context_graph.outfile_remote_name[k]} does not exist after writing") - if os.stat(context_graph.outfile_remote_name[k]).st_size == 0: - raise Exception(f"Output file {context_graph.outfile_remote_name[k]} is empty after writing") - - time.sleep(context_graph.extra_sleep_time_of[k]) - - return True diff --git a/taskvine/src/graph/dagvine/context_graph/proxy_library.py b/taskvine/src/graph/dagvine/context_graph/proxy_library.py deleted file mode 100644 index 4d0873e667..0000000000 --- a/taskvine/src/graph/dagvine/context_graph/proxy_library.py +++ /dev/null @@ -1,94 +0,0 @@ -# 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 ndcctools.taskvine.dagvine.context_graph.core import ContextGraphTaskResult, ContextGraph -from ndcctools.taskvine.dagvine.context_graph.proxy_functions import compute_dts_key, compute_sexpr_key, compute_single_key -from ndcctools.taskvine.utils import load_variable_from_library - - -class ProxyLibrary: - 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, ContextGraphTaskResult, ContextGraph, uuid, hashlib, random, types, collections, time, - load_variable_from_library, compute_dts_key, compute_sexpr_key, compute_single_key - ] - - # 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 get_context_size(self): - dumped_data = self.context_loader_args[0] - serialized = round(len(dumped_data) / 1024 / 1024, 2) - return serialized - - 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, - compute_single_key, - 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) From 23331bf27d982d4b577fdb1ca6507ed367accbc7 Mon Sep 17 00:00:00 2001 From: Jin Zhou Date: Thu, 8 Jan 2026 16:13:36 -0500 Subject: [PATCH 106/113] remove context graph from makefile --- taskvine/src/graph/dagvine/Makefile | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/taskvine/src/graph/dagvine/Makefile b/taskvine/src/graph/dagvine/Makefile index 1bfc711036..6e08b844bd 100644 --- a/taskvine/src/graph/dagvine/Makefile +++ b/taskvine/src/graph/dagvine/Makefile @@ -2,7 +2,7 @@ include ../../../../config.mk include ../../../../rules.mk MODULE_DIR := $(CCTOOLS_PYTHON3_PATH)/ndcctools/taskvine/dagvine -SUBDIRS := context_graph vine_graph blueprint_graph +SUBDIRS := vine_graph blueprint_graph .PHONY: all install clean lint format $(SUBDIRS) From 6f6aca5ba3d247c5b6a4245f986ab3acb0561b4f Mon Sep 17 00:00:00 2001 From: Jin Zhou Date: Thu, 8 Jan 2026 21:54:17 -0500 Subject: [PATCH 107/113] convert bg to dask expr --- .../graph/dagvine/blueprint_graph/adaptor.py | 41 +++++++++++++++++++ 1 file changed, 41 insertions(+) diff --git a/taskvine/src/graph/dagvine/blueprint_graph/adaptor.py b/taskvine/src/graph/dagvine/blueprint_graph/adaptor.py index 32d6045b8e..20cd935012 100644 --- a/taskvine/src/graph/dagvine/blueprint_graph/adaptor.py +++ b/taskvine/src/graph/dagvine/blueprint_graph/adaptor.py @@ -27,6 +27,47 @@ def _identity(value): return value +def _apply_with_kwargs_kvlist(func, args_list, kwargs_kvlist): + """Execute `func(*args_list, **kwargs)` where kwargs is encoded as a list of [k, v] pairs. + + This encoding is intentional: `dask_executor.execute_graph_vertex.rec_call` only recurses + into lists and task tuples, not dicts/tuples. By representing kwargs as lists, upstream + task-key references can be resolved before we rebuild the dict here. + """ + kwargs = {k: v for (k, v) in kwargs_kvlist} + return func(*args_list, **kwargs) + + +def collections_from_blueprint_graph(bg): + assert isinstance(bg, BlueprintGraph), "bg must be a BlueprintGraph" + + def _ref_to_key(ref: TaskOutputRef): + # Replace TaskOutputRef occurrences with the referenced task_key only. + # NOTE: This intentionally drops any `path` component on the ref, per request. + return ref.task_key + + out = {} + for task_key, (func, args, kwargs) in bg.task_dict.items(): + # Only rewrite references inside args/kwargs; keep everything else unchanged. + new_args = bg._visit_task_output_refs(args, _ref_to_key, rewrite=True) + new_kwargs = bg._visit_task_output_refs(kwargs, _ref_to_key, rewrite=True) + + # IMPORTANT: `dask_executor.execute_graph_vertex` expects classic Dask sexprs: + # (func, arg1, arg2, ...) + # It does not understand the BlueprintGraph triple (func, args_tuple, kwargs_dict), + # and it also does not recurse into dicts. So: + # - No-kwargs tasks become (func, *args) + # - Kwargs tasks become (_apply_with_kwargs_kvlist, func, [*args], [[k, v], ...]) + if new_kwargs: + args_list = list(new_args) + kwargs_kvlist = [[k, v] for k, v in new_kwargs.items()] + out[task_key] = (_apply_with_kwargs_kvlist, func, args_list, kwargs_kvlist) + else: + out[task_key] = (func, *new_args) + + return out + + class Adaptor: """Normalize user task inputs so `BlueprintGraph` can consume them without extra massaging.""" From a4d078df2452aceecf65f7b4c8267085d13ad698 Mon Sep 17 00:00:00 2001 From: Jin Zhou Date: Thu, 8 Jan 2026 23:19:47 -0500 Subject: [PATCH 108/113] support internal in/out --- taskvine/src/graph/dagvine/dagvine.py | 13 +++- .../src/graph/dagvine/vine_graph/vine_graph.c | 65 +++++++++++++++++++ .../src/graph/dagvine/vine_graph/vine_graph.h | 16 +++++ .../dagvine/vine_graph/vine_graph_client.py | 14 ++++ taskvine/src/manager/vine_manager.c | 1 + 5 files changed, 108 insertions(+), 1 deletion(-) diff --git a/taskvine/src/graph/dagvine/dagvine.py b/taskvine/src/graph/dagvine/dagvine.py index 1869e25540..ddba9f673b 100644 --- a/taskvine/src/graph/dagvine/dagvine.py +++ b/taskvine/src/graph/dagvine/dagvine.py @@ -57,7 +57,8 @@ def __init__(self): "attempt-schedule-depth": 10000, "temp-replica-count": 1, "enforce-worker-eviction-interval": -1, - "balance-worker-disk-load": 0, + "shift-disk-load": 0, + "clean-redundant-replicas": 0, } # VineGraph-level knobs: forwarded to the underlying vine graph via VineGraphClient. self.vine_graph_tuning_params = { @@ -229,6 +230,16 @@ def build_graphs(self, task_dict, target_keys): outfile_remote_name = vine_graph.get_node_outfile_remote_name(k) py_graph.outfile_remote_name[k] = outfile_remote_name + # For each task, declare the input and output files in the vine graph + for filename in py_graph.producer_of: + task_key = py_graph.producer_of[filename] + print(f"adding output file {filename} to task {task_key}") + vine_graph.add_task_output(task_key, filename) + for filename in py_graph.consumers_of: + for task_key in py_graph.consumers_of[filename]: + print(f"adding input file {filename} to task {task_key}") + vine_graph.add_task_input(task_key, filename) + return py_graph, vine_graph def create_proxy_library(self, py_graph, vine_graph, hoisting_modules, env_files): diff --git a/taskvine/src/graph/dagvine/vine_graph/vine_graph.c b/taskvine/src/graph/dagvine/vine_graph/vine_graph.c index 3a582aa0ed..fb349dcbae 100644 --- a/taskvine/src/graph/dagvine/vine_graph/vine_graph.c +++ b/taskvine/src/graph/dagvine/vine_graph/vine_graph.c @@ -713,6 +713,66 @@ const char *vine_graph_get_proxy_library_name(const struct vine_graph *vg) return vg->proxy_library_name; } +/** + * Add an input file to a task. The input file will be declared as a temp file. + * @param vg Reference to the vine graph. + * @param task_id Reference to the task id. + * @param filename Reference to the filename. + */ +void vine_graph_add_task_input(struct vine_graph *vg, uint64_t task_id, const char *filename) +{ + if (!vg || !task_id || !filename) { + return; + } + + struct vine_node *node = itable_lookup(vg->nodes, task_id); + if (!node) { + return; + } + + struct vine_file *f = NULL; + const char *cached_name = hash_table_lookup(vg->inout_filename_to_cached_name, filename); + + if (cached_name) { + f = vine_manager_lookup_file(vg->manager, cached_name); + } else { + f = vine_declare_temp(vg->manager); + hash_table_insert(vg->inout_filename_to_cached_name, filename, xxstrdup(f->cached_name)); + } + + vine_task_add_input(node->task, f, filename, VINE_TRANSFER_ALWAYS); +} + +/** + * Add an output file to a task. The output file will be declared as a temp file. + * @param vg Reference to the vine graph. + * @param task_id Reference to the task id. + * @param filename Reference to the filename. + */ +void vine_graph_add_task_output(struct vine_graph *vg, uint64_t task_id, const char *filename) +{ + if (!vg || !task_id || !filename) { + return; + } + + struct vine_node *node = itable_lookup(vg->nodes, task_id); + if (!node) { + return; + } + + struct vine_file *f = NULL; + const char *cached_name = hash_table_lookup(vg->inout_filename_to_cached_name, filename); + + if (cached_name) { + f = vine_manager_lookup_file(vg->manager, cached_name); + } else { + f = vine_declare_temp(vg->manager); + hash_table_insert(vg->inout_filename_to_cached_name, filename, xxstrdup(f->cached_name)); + } + + vine_task_add_output(node->task, f, filename, VINE_TRANSFER_ALWAYS); +} + /** * Set the proxy function name of the vine graph. * @param vg Reference to the vine graph. @@ -1038,6 +1098,7 @@ struct vine_graph *vine_graph_create(struct vine_manager *q) vg->nodes = itable_create(0); vg->task_id_to_node = itable_create(0); vg->outfile_cachename_to_node = hash_table_create(0, 0); + vg->inout_filename_to_cached_name = hash_table_create(0, 0); cctools_uuid_t proxy_library_name_id; cctools_uuid_create(&proxy_library_name_id); @@ -1371,5 +1432,9 @@ void vine_graph_delete(struct vine_graph *vg) itable_delete(vg->nodes); itable_delete(vg->task_id_to_node); hash_table_delete(vg->outfile_cachename_to_node); + + hash_table_clear(vg->inout_filename_to_cached_name, (void *)free); + hash_table_delete(vg->inout_filename_to_cached_name); + free(vg); } diff --git a/taskvine/src/graph/dagvine/vine_graph/vine_graph.h b/taskvine/src/graph/dagvine/vine_graph/vine_graph.h index 0d9edb2a9b..b20132cc0e 100644 --- a/taskvine/src/graph/dagvine/vine_graph/vine_graph.h +++ b/taskvine/src/graph/dagvine/vine_graph/vine_graph.h @@ -29,6 +29,8 @@ struct vine_graph { struct itable *nodes; struct itable *task_id_to_node; struct hash_table *outfile_cachename_to_node; + /* Maps a logical in/out filename (remote_name) to a stable cached_name. */ + struct hash_table *inout_filename_to_cached_name; /* The directory to store the checkpointed results. * Only intermediate results can be checkpointed, the fraction of intermediate results to checkpoint is controlled by the checkpoint-fraction parameter. */ @@ -133,6 +135,20 @@ void vine_graph_delete(struct vine_graph *vg); */ const char *vine_graph_get_proxy_library_name(const struct vine_graph *vg); +/** Add an input file to a task. The input file will be declared as a temp file. +@param vg Reference to the vine graph. +@param task_id Identifier of the task. +@param filename Reference to the filename. +*/ +void vine_graph_add_task_input(struct vine_graph *vg, uint64_t task_id, const char *filename); + +/** Add an output file to a task. The output file will be declared as a temp file. +@param vg Reference to the vine graph. +@param task_id Identifier of the task. +@param filename Reference to the filename. +*/ +void vine_graph_add_task_output(struct vine_graph *vg, uint64_t task_id, const char *filename); + /** Set the proxy function name of the vine graph. @param vg Reference to the vine graph. @param proxy_function_name Reference to the proxy function name. diff --git a/taskvine/src/graph/dagvine/vine_graph/vine_graph_client.py b/taskvine/src/graph/dagvine/vine_graph/vine_graph_client.py index 6019e6e74c..841fb55287 100644 --- a/taskvine/src/graph/dagvine/vine_graph/vine_graph_client.py +++ b/taskvine/src/graph/dagvine/vine_graph/vine_graph_client.py @@ -66,6 +66,20 @@ def set_proxy_function(self, proxy_function): self._c_graph, proxy_function.__name__ ) + def add_task_input(self, task_key, filename): + """Add an input file to a task.""" + task_id = self._key_to_id.get(task_key) + if task_id is None: + raise KeyError(f"Task key not found: {task_key}") + vine_graph_capi.vine_graph_add_task_input(self._c_graph, task_id, filename) + + def add_task_output(self, task_key, filename): + """Add an output file to a task.""" + task_id = self._key_to_id.get(task_key) + if task_id is None: + raise KeyError(f"Task key not found: {task_key}") + vine_graph_capi.vine_graph_add_task_output(self._c_graph, task_id, filename) + def execute(self): """Kick off execution; runs through SWIG down into the C orchestration loop.""" vine_graph_capi.vine_graph_execute(self._c_graph) diff --git a/taskvine/src/manager/vine_manager.c b/taskvine/src/manager/vine_manager.c index 306589c042..7473e7dde2 100644 --- a/taskvine/src/manager/vine_manager.c +++ b/taskvine/src/manager/vine_manager.c @@ -3480,6 +3480,7 @@ static int vine_manager_check_inputs_available(struct vine_manager *q, struct vi if (f->type == VINE_FILE && f->state == VINE_FILE_STATE_PENDING) { all_available = 0; } else if (f->type == VINE_TEMP && !vine_temp_exists_somewhere(q, f)) { + printf("File %s is not available!\n", f->cached_name); vine_manager_consider_recovery_task(q, f, f->recovery_task); all_available = 0; } From 160c2b9427c463381f8bfa5d19f24ab76502a85e Mon Sep 17 00:00:00 2001 From: Jin Zhou Date: Fri, 9 Jan 2026 03:25:41 -0500 Subject: [PATCH 109/113] auto recovery --- taskvine/src/graph/dagvine/dagvine.py | 3 + .../src/graph/dagvine/vine_graph/vine_graph.c | 241 ++++++++++++++---- .../src/graph/dagvine/vine_graph/vine_graph.h | 8 + .../src/graph/dagvine/vine_graph/vine_node.c | 11 +- .../src/graph/dagvine/vine_graph/vine_node.h | 12 +- taskvine/src/manager/vine_manager.c | 10 +- taskvine/src/manager/vine_manager.h | 1 + 7 files changed, 227 insertions(+), 59 deletions(-) diff --git a/taskvine/src/graph/dagvine/dagvine.py b/taskvine/src/graph/dagvine/dagvine.py index ddba9f673b..051da7c047 100644 --- a/taskvine/src/graph/dagvine/dagvine.py +++ b/taskvine/src/graph/dagvine/dagvine.py @@ -71,6 +71,9 @@ def __init__(self): "progress-bar-update-interval-sec": 0.1, "time-metrics-filename": "time_metrics.csv", "enable-debug-log": 1, + "auto-recovery": 1, + "max-retry-attempts": 15, + "retry-interval-sec": 1, } # Misc knobs used purely on the Python side (e.g., generate fake outputs). self.other_params = { diff --git a/taskvine/src/graph/dagvine/vine_graph/vine_graph.c b/taskvine/src/graph/dagvine/vine_graph/vine_graph.c index fb349dcbae..48df5d9430 100644 --- a/taskvine/src/graph/dagvine/vine_graph/vine_graph.c +++ b/taskvine/src/graph/dagvine/vine_graph/vine_graph.c @@ -116,6 +116,18 @@ static void submit_node_task(struct vine_graph *vg, struct vine_node *node) return; } + if (!node->task) { + debug(D_ERROR, "submit_node_task: node %" PRIu64 " has no task", node->node_id); + return; + } + + /* Avoid double-submitting the same task object. This should never be needed + * for correctness and leads to task_id mapping corruption if it happens. */ + if (node->task->state != VINE_TASK_INITIAL) { + debug(D_VINE, "submit_node_task: skipping node %" PRIu64 " (task already submitted, state=%d, task_id=%d)", node->node_id, node->task->state, node->task->task_id); + return; + } + /* calculate the priority of the node */ double priority = calculate_task_priority(node, vg->task_priority_mode); vine_task_set_priority(node->task, priority); @@ -125,6 +137,11 @@ static void submit_node_task(struct vine_graph *vg, struct vine_node *node) int task_id = vine_submit(vg->manager, node->task); node->submission_time = timestamp_get() - time_start; + if (task_id <= 0) { + debug(D_ERROR, "submit_node_task: failed to submit node %" PRIu64 " (returned task_id=%d)", node->node_id, task_id); + return; + } + /* insert the task id to the task id to node map */ itable_insert(vg->task_id_to_node, (uint64_t)task_id, node); @@ -147,18 +164,26 @@ static void submit_unblocked_children(struct vine_graph *vg, struct vine_node *n struct vine_node *child_node; LIST_ITERATE(node->children, child_node) { - /* Remove this parent from the child's pending set if it exists */ - if (child_node->pending_parents) { - /* Assert that this parent is indeed pending for the child */ - if (child_node->pending_parents && set_lookup(child_node->pending_parents, node)) { - set_remove(child_node->pending_parents, node); - } else { - debug(D_ERROR, "inconsistent pending set: child=%" PRIu64 " missing parent=%" PRIu64, child_node->node_id, node->node_id); - } + if (!child_node) { + continue; } - /* If no more parents are pending, submit the child */ - if (!child_node->pending_parents || set_size(child_node->pending_parents) == 0) { + /* Edge-fired dependency resolution: each parent->child edge is consumed at most once. + * This is critical for recomputation/resubmission, where a parent may "complete" multiple times. */ + if (child_node->fired_parents && set_lookup(child_node->fired_parents, node)) { + continue; + } + if (child_node->fired_parents) { + set_insert(child_node->fired_parents, node); + } + + if (child_node->remaining_parents_count > 0) { + child_node->remaining_parents_count--; + } + + /* If no more parents are remaining, submit the child (if it is not already done / in-flight). */ + if (child_node->remaining_parents_count == 0 && !child_node->completed && child_node->task && + child_node->task->state == VINE_TASK_INITIAL) { submit_node_task(vg, child_node); } } @@ -551,6 +576,85 @@ static void print_time_metrics(struct vine_graph *vg, const char *filename) return; } +/** + * Enqueue a node to be resubmitted later. + * @param vg Reference to the vine graph. + * @param node Reference to the node. + */ +static void enqueue_resubmit_node(struct vine_graph *vg, struct vine_node *node) +{ + if (!vg || !node) { + return; + } + + if (node->in_resubmit_queue) { + return; + } + + /* if the task failed due to inputs missing, we must submit the producer tasks for the lost data */ + if (node->task->result == VINE_RESULT_INPUT_MISSING) { + struct vine_mount *m; + LIST_ITERATE(node->task->input_mounts, m) + { + struct vine_file *f = m->file; + /* this is a temp file, it has a producer task id and the task pointer is null */ + if (f->type != VINE_TEMP) { + continue; + } + if (vine_temp_exists_somewhere(vg->manager, f)) { + continue; + } + if (itable_lookup(vg->manager->tasks, f->original_producer_task_id)) { + continue; + } + /* get the original producer node by task id */ + struct vine_node *original_producer_node = itable_lookup(vg->task_id_to_node, f->original_producer_task_id); + if (!original_producer_node) { + continue; + } + enqueue_resubmit_node(vg, original_producer_node); + } + } + + node->last_failure_time = timestamp_get(); + list_push_tail(vg->resubmit_queue, node); + node->in_resubmit_queue = 1; +} + +/* Try to resubmit a previously failed node. + * @return 1 if a node was actually resubmitted (reset + submit invoked), 0 otherwise. */ +static int try_resubmitting_node(struct vine_graph *vg) +{ + if (!vg) { + return 0; + } + + struct vine_node *node = list_pop_head(vg->resubmit_queue); + if (!node) { + return 0; + } + node->in_resubmit_queue = 0; + + timestamp_t interval = timestamp_get() - node->last_failure_time; + + if (interval <= vg->retry_interval_sec * 1e6) { + enqueue_resubmit_node(vg, node); + return 0; + } + + if (node->retry_attempts_left-- <= 0) { + debug(D_ERROR, "node %" PRIu64 " has no retries left. Aborting.", node->node_id); + vine_graph_delete(vg); + exit(1); + } + + debug(D_VINE, "Resubmitting node %" PRIu64 " (remaining=%d)", node->node_id, node->retry_attempts_left); + vine_task_reset(node->task); + submit_node_task(vg, node); + + return 1; +} + /*************************************************************/ /* Public APIs */ /*************************************************************/ @@ -671,6 +775,15 @@ int vine_graph_tune(struct vine_graph *vg, const char *name, const char *value) debug_close(); } + } else if (strcmp(name, "auto-recovery") == 0) { + vg->auto_recovery = (atoi(value) == 1) ? 1 : 0; + + } else if (strcmp(name, "max-retry-attempts") == 0) { + vg->max_retry_attempts = MAX(0, atoi(value)); + + } else if (strcmp(name, "retry-interval-sec") == 0) { + vg->retry_interval_sec = MAX(0.0, atof(value)); + } else { debug(D_ERROR, "invalid parameter name: %s", name); return -1; @@ -1055,6 +1168,8 @@ uint64_t vine_graph_add_node(struct vine_graph *vg) /* initialize the pruning depth of each node, currently statically set to the global prune depth */ node->prune_depth = vg->prune_depth; + node->retry_attempts_left = vg->max_retry_attempts; + itable_insert(vg->nodes, node_id, node); return node_id; @@ -1099,6 +1214,7 @@ struct vine_graph *vine_graph_create(struct vine_manager *q) vg->task_id_to_node = itable_create(0); vg->outfile_cachename_to_node = hash_table_create(0, 0); vg->inout_filename_to_cached_name = hash_table_create(0, 0); + vg->resubmit_queue = list_create(); cctools_uuid_t proxy_library_name_id; cctools_uuid_create(&proxy_library_name_id); @@ -1123,6 +1239,14 @@ struct vine_graph *vine_graph_create(struct vine_manager *q) vg->enable_debug_log = 1; + vg->max_retry_attempts = 15; + vg->retry_interval_sec = 1.0; + + /* disable auto recovery so that the graph executor can handle all tasks with missing inputs + * this ensures that no recovery tasks are created automatically by the taskvine manager and that + * we can be in control of when to recreate what lost data. */ + vg->auto_recovery = 0; + return vg; } @@ -1188,6 +1312,8 @@ void vine_graph_execute(struct vine_graph *vg) /* enable return recovery tasks */ vine_enable_return_recovery_tasks(vg->manager); + vg->manager->auto_recovery = vg->auto_recovery; + /* create mappings from task IDs and outfile cache names to nodes */ ITABLE_ITERATE(vg->nodes, nid_iter, node) { @@ -1209,23 +1335,16 @@ void vine_graph_execute(struct vine_graph *vg) } } - /* initialize pending_parents for all nodes */ + /* initialize remaining_parents_count for all nodes */ ITABLE_ITERATE(vg->nodes, nid_iter, node) { - struct vine_node *parent_node; - LIST_ITERATE(node->parents, parent_node) - { - if (node->pending_parents) { - /* Use parent pointer to ensure pointer consistency */ - set_insert(node->pending_parents, parent_node); - } - } + node->remaining_parents_count = list_size(node->parents); } /* enqueue those without dependencies */ ITABLE_ITERATE(vg->nodes, nid_iter, node) { - if (!node->pending_parents || set_size(node->pending_parents) == 0) { + if (node->remaining_parents_count == 0) { submit_node_task(vg, node); } } @@ -1251,14 +1370,39 @@ void vine_graph_execute(struct vine_graph *vg) break; } + /* Always process graph-level resubmissions (this affects correctness), + * but the Recovery progress bar source depends on auto_recovery: + * - auto_recovery==1: Recovery tracks manager-created recovery tasks only + * - auto_recovery==0: Recovery tracks graph resubmit queue only */ + int did_resubmit = try_resubmitting_node(vg); + + if (vg->manager->auto_recovery) { + /* Recovery progress reflects manager recovery tasks. */ + (void)did_resubmit; + progress_bar_set_part_total(pbar, recovery_tasks_part, (uint64_t)vg->manager->stats->recovery_tasks_submitted); + progress_bar_update_part(pbar, recovery_tasks_part, 0); + } else { + /* Recovery progress reflects graph-level resubmissions: + * total = (already attempted resubmits) + (currently queued resubmits). */ + if (did_resubmit) { + progress_bar_update_part(pbar, recovery_tasks_part, 1); + } + uint64_t queued_resubmits = (uint64_t)list_size(vg->resubmit_queue); + progress_bar_set_part_total(pbar, recovery_tasks_part, recovery_tasks_part->current + queued_resubmits); + progress_bar_update_part(pbar, recovery_tasks_part, 0); + } + struct vine_task *task = vine_wait(vg->manager, wait_timeout); - progress_bar_set_part_total(pbar, recovery_tasks_part, vg->manager->stats->recovery_tasks_submitted); if (task) { /* retrieve all possible tasks */ wait_timeout = 0; - timestamp_t time_when_postprocessing_start = timestamp_get(); + /* If auto_recovery is enabled, recovery tasks should be reflected in the Recovery progress bar. */ + if (vg->manager->auto_recovery && task->type == VINE_TASK_TYPE_RECOVERY) { + progress_bar_update_part(pbar, recovery_tasks_part, 1); + } + /* get the original node by task id */ struct vine_node *node = get_node_by_task(vg, task); if (!node) { @@ -1268,15 +1412,8 @@ void vine_graph_execute(struct vine_graph *vg) /* in case of failure, resubmit this task */ if (node->task->result != VINE_RESULT_SUCCESS || node->task->exit_code != 0) { - if (node->retry_attempts_left <= 0) { - debug(D_ERROR, "Task %d failed (result=%d, exit=%d). Node %" PRIu64 " has no retries left. Aborting.", task->task_id, node->task->result, node->task->exit_code, node->node_id); - vine_graph_delete(vg); - exit(1); - } - node->retry_attempts_left--; - debug(D_VINE | D_NOTICE, "Task %d failed (result=%d, exit=%d). Retrying node %" PRIu64 " (remaining=%d)...", task->task_id, node->task->result, node->task->exit_code, node->node_id, node->retry_attempts_left); - vine_task_reset(node->task); - submit_node_task(vg, node); + enqueue_resubmit_node(vg, node); + debug(D_VINE, "Task %d failed (result=%d, exit=%d)", task->task_id, node->task->result, node->task->exit_code); continue; } @@ -1286,15 +1423,8 @@ void vine_graph_execute(struct vine_graph *vg) struct stat info; int result = stat(node->outfile_remote_name, &info); if (result < 0) { - if (node->retry_attempts_left <= 0) { - debug(D_ERROR, "Task %d succeeded but missing sharedfs output %s; no retries left for node %" PRIu64 ". Aborting.", task->task_id, node->outfile_remote_name, node->node_id); - vine_graph_delete(vg); - exit(1); - } - node->retry_attempts_left--; - debug(D_VINE | D_NOTICE, "Task %d succeeded but missing sharedfs output %s; retrying node %" PRIu64 " (remaining=%d)...", task->task_id, node->outfile_remote_name, node->node_id, node->retry_attempts_left); - vine_task_reset(node->task); - submit_node_task(vg, node); + debug(D_VINE, "Task %d succeeded but missing sharedfs output %s", task->task_id, node->outfile_remote_name); + enqueue_resubmit_node(vg, node); continue; } node->outfile_size_bytes = info.st_size; @@ -1307,7 +1437,10 @@ void vine_graph_execute(struct vine_graph *vg) } debug(D_VINE, "Node %" PRIu64 " completed with outfile %s size: %zu bytes", node->node_id, node->outfile_remote_name, node->outfile_size_bytes); - /* mark the node as completed */ + /* mark the node as completed + * Note: a node may complete multiple times due to resubmission/recomputation. + * Only the first completion should advance the "Regular" progress. */ + int first_completion = !node->completed; node->completed = 1; node->scheduling_time = task->time_when_scheduling_end - task->time_when_scheduling_start; node->commit_time = task->time_when_commit_end - task->time_when_commit_start; @@ -1317,22 +1450,25 @@ void vine_graph_execute(struct vine_graph *vg) /* prune nodes on task completion */ prune_ancestors_of_node(vg, node); - /* skip recovery tasks */ + /* skip manager-created recovery tasks. + * - If auto_recovery==1, we already accounted for them in the Recovery bar above. + * - If auto_recovery==0, Recovery bar tracks graph resubmits only. */ if (task->type == VINE_TASK_TYPE_RECOVERY) { - progress_bar_update_part(pbar, recovery_tasks_part, 1); continue; } - /* set the start time to the submit time of the first regular task */ - if (regular_tasks_part->current == 0) { - progress_bar_set_start_time(pbar, task->time_when_commit_start); - } + if (first_completion) { + /* set the start time to the submit time of the first regular task */ + if (regular_tasks_part->current == 0) { + progress_bar_set_start_time(pbar, task->time_when_commit_start); + } - /* update critical time */ - vine_node_update_critical_path_time(node, node->execution_time); + /* update critical time */ + vine_node_update_critical_path_time(node, node->execution_time); - /* mark this regular task as completed */ - progress_bar_update_part(pbar, regular_tasks_part, 1); + /* mark this regular task as completed */ + progress_bar_update_part(pbar, regular_tasks_part, 1); + } /* inject failure */ if (vg->failure_injection_step_percent > 0) { @@ -1361,7 +1497,6 @@ void vine_graph_execute(struct vine_graph *vg) node->postprocessing_time = time_when_postprocessing_end - time_when_postprocessing_start; } else { wait_timeout = 1; - progress_bar_update_part(pbar, recovery_tasks_part, 0); // refresh the time and total for recovery tasks } } @@ -1426,6 +1561,8 @@ void vine_graph_delete(struct vine_graph *vg) vine_node_delete(node); } + list_delete(vg->resubmit_queue); + free(vg->proxy_library_name); free(vg->proxy_function_name); diff --git a/taskvine/src/graph/dagvine/vine_graph/vine_graph.h b/taskvine/src/graph/dagvine/vine_graph/vine_graph.h index b20132cc0e..6cebe4dd70 100644 --- a/taskvine/src/graph/dagvine/vine_graph/vine_graph.h +++ b/taskvine/src/graph/dagvine/vine_graph/vine_graph.h @@ -32,6 +32,9 @@ struct vine_graph { /* Maps a logical in/out filename (remote_name) to a stable cached_name. */ struct hash_table *inout_filename_to_cached_name; + /* Unsuccessful tasks are appended to this list to be resubmitted later. */ + struct list *resubmit_queue; + /* The directory to store the checkpointed results. * Only intermediate results can be checkpointed, the fraction of intermediate results to checkpoint is controlled by the checkpoint-fraction parameter. */ char *checkpoint_dir; @@ -64,6 +67,11 @@ struct vine_graph { char *time_metrics_filename; int enable_debug_log; /* whether to enable debug log */ + + int auto_recovery; /* whether to enable auto recovery */ + + int max_retry_attempts; /* the maximum number of times to retry a task */ + double retry_interval_sec; /* the interval between retries in seconds, 0 means no retry interval */ }; /* Public APIs for operating the vine graph */ diff --git a/taskvine/src/graph/dagvine/vine_graph/vine_node.c b/taskvine/src/graph/dagvine/vine_graph/vine_node.c index 86e0dfffb4..48d43ffccb 100644 --- a/taskvine/src/graph/dagvine/vine_graph/vine_node.c +++ b/taskvine/src/graph/dagvine/vine_graph/vine_node.c @@ -125,11 +125,13 @@ struct vine_node *vine_node_create(uint64_t node_id) node->prune_status = PRUNE_STATUS_NOT_PRUNED; node->parents = list_create(); node->children = list_create(); - node->pending_parents = set_create(0); + node->remaining_parents_count = 0; + node->fired_parents = set_create(0); node->completed = 0; node->prune_depth = 0; - node->retry_attempts_left = 1; node->outfile_size_bytes = 0; + node->retry_attempts_left = 0; + node->in_resubmit_queue = 0; node->depth = -1; node->height = -1; @@ -151,6 +153,7 @@ struct vine_node *vine_node_create(uint64_t node_id) node->postprocessing_time = 0; node->critical_path_time = -1; + node->last_failure_time = 0; return node; } @@ -400,8 +403,8 @@ void vine_node_delete(struct vine_node *node) list_delete(node->parents); list_delete(node->children); - if (node->pending_parents) { - set_delete(node->pending_parents); + if (node->fired_parents) { + set_delete(node->fired_parents); } free(node); } \ No newline at end of file diff --git a/taskvine/src/graph/dagvine/vine_graph/vine_node.h b/taskvine/src/graph/dagvine/vine_graph/vine_node.h index a838fb6e88..1910a11afa 100644 --- a/taskvine/src/graph/dagvine/vine_graph/vine_node.h +++ b/taskvine/src/graph/dagvine/vine_graph/vine_node.h @@ -42,10 +42,16 @@ struct vine_node { struct list *children; /* Execution and scheduling state */ - struct set *pending_parents; - int retry_attempts_left; + /* Number of unresolved parent dependencies. This is initialized to the in-degree + * (list_size(parents)) before execution starts, and decremented exactly once per + * parent->child edge when the parent first completes. */ + int remaining_parents_count; + /* Edge-fired guard: tracks which parent edges have already been consumed for this child. */ + struct set *fired_parents; int completed; prune_status_t prune_status; + int retry_attempts_left; + int in_resubmit_queue; /* Structural metrics */ int prune_depth; @@ -69,6 +75,8 @@ struct vine_node { timestamp_t execution_time; timestamp_t retrieval_time; timestamp_t postprocessing_time; + + timestamp_t last_failure_time; }; /** Create a new vine node. diff --git a/taskvine/src/manager/vine_manager.c b/taskvine/src/manager/vine_manager.c index 7473e7dde2..5ba406a29a 100644 --- a/taskvine/src/manager/vine_manager.c +++ b/taskvine/src/manager/vine_manager.c @@ -1487,6 +1487,10 @@ static int retrieve_ready_task(struct vine_manager *q, struct vine_task *t, doub result = VINE_RESULT_FIXED_LOCATION_MISSING; } } + if (!q->auto_recovery && !vine_manager_check_inputs_available(q, t)) { + debug(D_VINE, "task %d has missing input files", t->task_id); + result = VINE_RESULT_INPUT_MISSING; + } /* If any of the reasons fired, then expire the task and put in the retrieved queue. */ if (result != VINE_RESULT_SUCCESS) { @@ -3423,6 +3427,10 @@ static void vine_manager_consider_recovery_task(struct vine_manager *q, struct v return; } + if (!q->auto_recovery) { + return; + } + /* Prevent race between original task and recovery task after worker crash. * Example: Task T completes on worker W, creates file F, T moves to WAITING_RETRIEVAL. * W crashes before stdout retrieval, T gets rescheduled to READY, F is lost and triggers @@ -3480,7 +3488,6 @@ static int vine_manager_check_inputs_available(struct vine_manager *q, struct vi if (f->type == VINE_FILE && f->state == VINE_FILE_STATE_PENDING) { all_available = 0; } else if (f->type == VINE_TEMP && !vine_temp_exists_somewhere(q, f)) { - printf("File %s is not available!\n", f->cached_name); vine_manager_consider_recovery_task(q, f, f->recovery_task); all_available = 0; } @@ -4174,6 +4181,7 @@ struct vine_manager *vine_ssl_create(int port, const char *key, const char *cert q->time_start_worker_eviction = 0; q->return_recovery_tasks = 0; + q->auto_recovery = 1; q->balance_worker_disk_load = 0; q->when_last_offloaded = 0; q->peak_used_cache = 0; diff --git a/taskvine/src/manager/vine_manager.h b/taskvine/src/manager/vine_manager.h index 166c9c9c36..3588cd443a 100644 --- a/taskvine/src/manager/vine_manager.h +++ b/taskvine/src/manager/vine_manager.h @@ -238,6 +238,7 @@ struct vine_manager { double disk_proportion_available_to_task; /* intentionally reduces disk allocation for tasks to reserve some space for cache growth. */ int return_recovery_tasks; /* If true, recovery tasks are returned by vine_wait to the user. By default they are handled internally. */ + int auto_recovery; /* If true, recovery tasks are created automatically internally. */ int balance_worker_disk_load; /* If true, offload replicas from workers that are overloaded with temp files. */ timestamp_t when_last_offloaded; int64_t peak_used_cache; From f43bd83f8382d917a1457bb8e3784154154cf43d Mon Sep 17 00:00:00 2001 From: Jin Zhou Date: Fri, 9 Jan 2026 03:32:08 -0500 Subject: [PATCH 110/113] rename to User --- .../src/graph/dagvine/vine_graph/vine_graph.c | 20 +++++++++---------- 1 file changed, 10 insertions(+), 10 deletions(-) diff --git a/taskvine/src/graph/dagvine/vine_graph/vine_graph.c b/taskvine/src/graph/dagvine/vine_graph/vine_graph.c index 48df5d9430..199798b83c 100644 --- a/taskvine/src/graph/dagvine/vine_graph/vine_graph.c +++ b/taskvine/src/graph/dagvine/vine_graph/vine_graph.c @@ -1087,7 +1087,7 @@ void vine_graph_compute_topology_metrics(struct vine_graph *vg) node->outfile = vine_declare_temp(vg->manager); } } - /* track the output dependencies of regular and vine_temp nodes */ + /* track the output dependencies of user and vine_temp nodes */ LIST_ITERATE(topo_order, node) { if (node->outfile) { @@ -1358,14 +1358,14 @@ void vine_graph_execute(struct vine_graph *vg) struct ProgressBar *pbar = progress_bar_init("Executing Tasks"); progress_bar_set_update_interval(pbar, vg->progress_bar_update_interval_sec); - struct ProgressBarPart *regular_tasks_part = progress_bar_create_part("Regular", itable_size(vg->nodes)); + struct ProgressBarPart *user_tasks_part = progress_bar_create_part("User", itable_size(vg->nodes)); struct ProgressBarPart *recovery_tasks_part = progress_bar_create_part("Recovery", 0); - progress_bar_bind_part(pbar, regular_tasks_part); + progress_bar_bind_part(pbar, user_tasks_part); progress_bar_bind_part(pbar, recovery_tasks_part); int wait_timeout = 1; - while (regular_tasks_part->current < regular_tasks_part->total) { + while (user_tasks_part->current < user_tasks_part->total) { if (interrupted) { break; } @@ -1439,7 +1439,7 @@ void vine_graph_execute(struct vine_graph *vg) /* mark the node as completed * Note: a node may complete multiple times due to resubmission/recomputation. - * Only the first completion should advance the "Regular" progress. */ + * Only the first completion should advance the "User" progress. */ int first_completion = !node->completed; node->completed = 1; node->scheduling_time = task->time_when_scheduling_end - task->time_when_scheduling_start; @@ -1458,21 +1458,21 @@ void vine_graph_execute(struct vine_graph *vg) } if (first_completion) { - /* set the start time to the submit time of the first regular task */ - if (regular_tasks_part->current == 0) { + /* set the start time to the submit time of the first user task */ + if (user_tasks_part->current == 0) { progress_bar_set_start_time(pbar, task->time_when_commit_start); } /* update critical time */ vine_node_update_critical_path_time(node, node->execution_time); - /* mark this regular task as completed */ - progress_bar_update_part(pbar, regular_tasks_part, 1); + /* mark this user task as completed */ + progress_bar_update_part(pbar, user_tasks_part, 1); } /* inject failure */ if (vg->failure_injection_step_percent > 0) { - double progress = (double)regular_tasks_part->current / (double)regular_tasks_part->total; + double progress = (double)user_tasks_part->current / (double)user_tasks_part->total; if (progress >= next_failure_threshold && release_random_worker(vg->manager)) { debug(D_VINE, "released a random worker at %.2f%% (threshold %.2f%%)", progress * 100, next_failure_threshold * 100); next_failure_threshold += vg->failure_injection_step_percent / 100.0; From 845e0a697175bbeb21e01d23cd25f4bb8b5c65bf Mon Sep 17 00:00:00 2001 From: Jin Zhou Date: Fri, 9 Jan 2026 03:43:44 -0500 Subject: [PATCH 111/113] block rec tasks w missing inputs --- .../src/graph/dagvine/vine_graph/vine_graph.c | 40 +++++++++++-------- 1 file changed, 24 insertions(+), 16 deletions(-) diff --git a/taskvine/src/graph/dagvine/vine_graph/vine_graph.c b/taskvine/src/graph/dagvine/vine_graph/vine_graph.c index 199798b83c..350dfe20cd 100644 --- a/taskvine/src/graph/dagvine/vine_graph/vine_graph.c +++ b/taskvine/src/graph/dagvine/vine_graph/vine_graph.c @@ -591,7 +591,27 @@ static void enqueue_resubmit_node(struct vine_graph *vg, struct vine_node *node) return; } + node->last_failure_time = timestamp_get(); + list_push_tail(vg->resubmit_queue, node); + node->in_resubmit_queue = 1; +} + +/* Try to resubmit a previously failed node. + * @return 1 if a node was actually resubmitted (reset + submit invoked), 0 otherwise. */ +static int try_resubmitting_node(struct vine_graph *vg) +{ + if (!vg) { + return 0; + } + + struct vine_node *node = list_pop_head(vg->resubmit_queue); + if (!node) { + return 0; + } + node->in_resubmit_queue = 0; + /* if the task failed due to inputs missing, we must submit the producer tasks for the lost data */ + int all_inputs_ready = 1; if (node->task->result == VINE_RESULT_INPUT_MISSING) { struct vine_mount *m; LIST_ITERATE(node->task->input_mounts, m) @@ -613,27 +633,15 @@ static void enqueue_resubmit_node(struct vine_graph *vg, struct vine_node *node) continue; } enqueue_resubmit_node(vg, original_producer_node); + all_inputs_ready = 0; } } - node->last_failure_time = timestamp_get(); - list_push_tail(vg->resubmit_queue, node); - node->in_resubmit_queue = 1; -} - -/* Try to resubmit a previously failed node. - * @return 1 if a node was actually resubmitted (reset + submit invoked), 0 otherwise. */ -static int try_resubmitting_node(struct vine_graph *vg) -{ - if (!vg) { - return 0; - } - - struct vine_node *node = list_pop_head(vg->resubmit_queue); - if (!node) { + /* if not all inputs are ready, enqueue the node and consider later */ + if (!all_inputs_ready) { + enqueue_resubmit_node(vg, node); return 0; } - node->in_resubmit_queue = 0; timestamp_t interval = timestamp_get() - node->last_failure_time; From 3abe87b77dd852c2abc333eae981fbc0aaa13a4f Mon Sep 17 00:00:00 2001 From: Jin Zhou Date: Thu, 15 Jan 2026 01:26:48 -0500 Subject: [PATCH 112/113] add compute makespan --- .../taskvine/compat/dask_executor.py | 16 +++++++-- taskvine/src/graph/dagvine/dagvine.py | 10 +++++- .../src/graph/dagvine/vine_graph/vine_graph.c | 36 ++++++++++++++++--- .../src/graph/dagvine/vine_graph/vine_graph.h | 11 ++++++ .../dagvine/vine_graph/vine_graph_client.py | 4 +++ taskvine/src/manager/vine_manager.c | 2 -- taskvine/src/manager/vine_task.c | 12 ------- taskvine/src/manager/vine_task.h | 7 ---- 8 files changed, 68 insertions(+), 30 deletions(-) diff --git a/taskvine/src/bindings/python3/ndcctools/taskvine/compat/dask_executor.py b/taskvine/src/bindings/python3/ndcctools/taskvine/compat/dask_executor.py index f6f4834069..2b48b2675a 100644 --- a/taskvine/src/bindings/python3/ndcctools/taskvine/compat/dask_executor.py +++ b/taskvine/src/bindings/python3/ndcctools/taskvine/compat/dask_executor.py @@ -138,8 +138,10 @@ def get(self, dsk, keys, *, hoisting_modules=None, # Deprecated, use lib_modules import_modules=None, # Deprecated, use lib_modules lazy_transfers=True, # Deprecated, use worker_tranfers + extra_serialize_time_sec=0, ): try: + self.extra_serialize_time_sec = extra_serialize_time_sec self.set_property("framework", "dask") if retries and retries < 1: raise ValueError("retries should be larger than 0") @@ -213,10 +215,13 @@ def __call__(self, *args, **kwargs): return self.get(*args, **kwargs) def _dask_execute(self, dsk, keys): + indices = {k: inds for (k, inds) in find_dask_keys(keys)} keys_flatten = indices.keys() + time_start = time.time() dag = DaskVineDag(dsk, low_memory_mode=self.low_memory_mode, prune_depth=self.prune_depth) + print(f"Time taken to enqueue tasks: {time.time() - time_start:.6f} seconds") tag = f"dag-{id(dag)}" # create Library if using 'function-calls' task mode. @@ -437,7 +442,8 @@ def _enqueue_dask_calls(self, dag, tag, rs, retries, enqueued_calls): extra_files=self.extra_files, retries=retries, worker_transfers=lazy, - wrapper=self.wrapper) + wrapper=self.wrapper, + extra_serialize_time_sec=self.extra_serialize_time_sec) t.set_priority(priority) t.set_tag(tag) # tag that identifies this dag @@ -557,7 +563,9 @@ def __init__(self, m, env_vars=None, retries=5, worker_transfers=False, - wrapper=None): + wrapper=None, + extra_serialize_time_sec=0): + time.sleep(extra_serialize_time_sec) self._key = key self._sexpr = sexpr @@ -658,7 +666,9 @@ def __init__(self, m, extra_files=None, retries=5, worker_transfers=False, - wrapper=None): + wrapper=None, + extra_serialize_time_sec=0): + time.sleep(extra_serialize_time_sec) self._key = key self.resources = resources diff --git a/taskvine/src/graph/dagvine/dagvine.py b/taskvine/src/graph/dagvine/dagvine.py index 051da7c047..b6b668b890 100644 --- a/taskvine/src/graph/dagvine/dagvine.py +++ b/taskvine/src/graph/dagvine/dagvine.py @@ -16,6 +16,7 @@ import signal import json import random +import time def context_loader_func(graph_pkl): @@ -69,7 +70,7 @@ def __init__(self): "checkpoint-dir": "./checkpoints", "checkpoint-fraction": 0, "progress-bar-update-interval-sec": 0.1, - "time-metrics-filename": "time_metrics.csv", + "time-metrics-filename": 0, "enable-debug-log": 1, "auto-recovery": 1, "max-retry-attempts": 15, @@ -258,6 +259,8 @@ def create_proxy_library(self, py_graph, vine_graph, hoisting_modules, env_files def run(self, task_dict, target_keys=[], params={}, hoisting_modules=[], env_files={}, adapt_dask=False): """High-level entry point: normalise input, build graphs, ship the library, execute, and return results.""" + time_start = time.time() + # first update the params so that they can be used for the following construction self.update_params(params) @@ -278,6 +281,7 @@ def run(self, task_dict, target_keys=[], params={}, hoisting_modules=[], env_fil try: print(f"=== Library serialized size: {color_text(proxy_library.get_context_size(), 92)} MB") + print(f"Time taken to initialize the graph in Python: {time.time() - time_start:.6f} seconds") vine_graph.execute() results = {} for k in target_keys: @@ -285,6 +289,10 @@ def run(self, task_dict, target_keys=[], params={}, hoisting_modules=[], env_fil continue outfile_path = os.path.join(self.param("output-dir"), py_graph.outfile_remote_name[k]) results[k] = TaskOutputWrapper.load_from_path(outfile_path) + makespan_s = round(vine_graph.get_makespan_us() / 1e6, 6) + throughput_tps = round(len(py_graph.task_dict) / makespan_s, 6) + print(f"Makespan: {color_text(makespan_s, 92)} seconds") + print(f"Throughput: {color_text(throughput_tps, 92)} tasks/second") return results finally: try: diff --git a/taskvine/src/graph/dagvine/vine_graph/vine_graph.c b/taskvine/src/graph/dagvine/vine_graph/vine_graph.c index 350dfe20cd..0935e4552d 100644 --- a/taskvine/src/graph/dagvine/vine_graph/vine_graph.c +++ b/taskvine/src/graph/dagvine/vine_graph/vine_graph.c @@ -563,13 +563,13 @@ static void print_time_metrics(struct vine_graph *vg, const char *filename) debug(D_ERROR, "failed to open file %s", filename); return; } - fprintf(fp, "node_id,submission_time_us,scheduling_time_us,commit_time_us,execution_time_us,retrieval_time_us,postprocessing_time_us\n"); + fprintf(fp, "node_id,submission_time_us,commit_time_us,execution_time_us,retrieval_time_us,postprocessing_time_us\n"); uint64_t nid; struct vine_node *node; ITABLE_ITERATE(vg->nodes, nid, node) { - fprintf(fp, "%" PRIu64 "," TIMESTAMP_FORMAT "," TIMESTAMP_FORMAT "," TIMESTAMP_FORMAT "," TIMESTAMP_FORMAT "," TIMESTAMP_FORMAT "," TIMESTAMP_FORMAT "\n", node->node_id, node->submission_time, node->scheduling_time, node->commit_time, node->execution_time, node->retrieval_time, node->postprocessing_time); + fprintf(fp, "%" PRIu64 "," TIMESTAMP_FORMAT "," TIMESTAMP_FORMAT "," TIMESTAMP_FORMAT "," TIMESTAMP_FORMAT "," TIMESTAMP_FORMAT "\n", node->node_id, node->submission_time, node->commit_time, node->execution_time, node->retrieval_time, node->postprocessing_time); } fclose(fp); @@ -738,7 +738,7 @@ int vine_graph_tune(struct vine_graph *vg, const char *name, const char *value) vg->progress_bar_update_interval_sec = (val > 0.0) ? val : 0.1; } else if (strcmp(name, "time-metrics-filename") == 0) { - if (strcmp(value, "0") == 0) { + if (value == NULL || strcmp(value, "0") == 0) { return 0; } @@ -1255,6 +1255,10 @@ struct vine_graph *vine_graph_create(struct vine_manager *q) * we can be in control of when to recreate what lost data. */ vg->auto_recovery = 0; + vg->time_first_task_dispatched = UINT64_MAX; + vg->time_last_task_retrieved = 0; + vg->makespan_us = 0; + return vg; } @@ -1295,6 +1299,15 @@ void vine_graph_add_dependency(struct vine_graph *vg, uint64_t parent_id, uint64 return; } +uint64_t vine_graph_get_makespan_us(const struct vine_graph *vg) +{ + if (!vg) { + return 0; + } + + return (uint64_t)vg->makespan_us; +} + /** * Execute the vine graph. This must be called after all nodes and dependencies are added and the topology metrics are computed. * @param vg Reference to the vine graph. @@ -1305,6 +1318,8 @@ void vine_graph_execute(struct vine_graph *vg) return; } + timestamp_t time_start = timestamp_get(); + void (*previous_sigint_handler)(int) = signal(SIGINT, handle_sigint); debug(D_VINE, "start executing vine graph"); @@ -1363,6 +1378,9 @@ void vine_graph_execute(struct vine_graph *vg) next_failure_threshold = vg->failure_injection_step_percent / 100.0; } + timestamp_t time_end = timestamp_get(); + printf("Time taken to initialize the graph in C: %.6f seconds\n", (double)(time_end - time_start) / 1e6); + struct ProgressBar *pbar = progress_bar_init("Executing Tasks"); progress_bar_set_update_interval(pbar, vg->progress_bar_update_interval_sec); @@ -1425,6 +1443,15 @@ void vine_graph_execute(struct vine_graph *vg) continue; } + /* update time metrics */ + vg->time_first_task_dispatched = MIN(vg->time_first_task_dispatched, task->time_when_commit_end); + vg->time_last_task_retrieved = MAX(vg->time_last_task_retrieved, task->time_when_retrieval); + if (vg->time_last_task_retrieved < vg->time_first_task_dispatched) { + debug(D_ERROR, "task %d time_last_task_retrieved < time_first_task_dispatched: %" PRIu64 " < %" PRIu64, task->task_id, vg->time_last_task_retrieved, vg->time_first_task_dispatched); + vg->time_last_task_retrieved = vg->time_first_task_dispatched; + } + vg->makespan_us = vg->time_last_task_retrieved - vg->time_first_task_dispatched; + /* if the outfile is set to save on the sharedfs, stat to get the size of the file */ switch (node->outfile_type) { case NODE_OUTFILE_TYPE_SHARED_FILE_SYSTEM: { @@ -1450,10 +1477,9 @@ void vine_graph_execute(struct vine_graph *vg) * Only the first completion should advance the "User" progress. */ int first_completion = !node->completed; node->completed = 1; - node->scheduling_time = task->time_when_scheduling_end - task->time_when_scheduling_start; node->commit_time = task->time_when_commit_end - task->time_when_commit_start; node->execution_time = task->time_workers_execute_last; - node->retrieval_time = task->time_when_get_result_end - task->time_when_get_result_start; + node->retrieval_time = task->time_when_done - task->time_when_retrieval; /* prune nodes on task completion */ prune_ancestors_of_node(vg, node); diff --git a/taskvine/src/graph/dagvine/vine_graph/vine_graph.h b/taskvine/src/graph/dagvine/vine_graph/vine_graph.h index 6cebe4dd70..4433563ec9 100644 --- a/taskvine/src/graph/dagvine/vine_graph/vine_graph.h +++ b/taskvine/src/graph/dagvine/vine_graph/vine_graph.h @@ -11,6 +11,7 @@ #include "set.h" #include "vine_node.h" #include "taskvine.h" +#include "timestamp.h" /** The task priority algorithm used for vine graph scheduling. */ typedef enum { @@ -72,6 +73,10 @@ struct vine_graph { int max_retry_attempts; /* the maximum number of times to retry a task */ double retry_interval_sec; /* the interval between retries in seconds, 0 means no retry interval */ + + timestamp_t time_first_task_dispatched; /* the time when the first task is dispatched */ + timestamp_t time_last_task_retrieved; /* the time when the last task is retrieved */ + timestamp_t makespan_us; /* the makespan of the vine graph in microseconds */ }; /* Public APIs for operating the vine graph */ @@ -171,4 +176,10 @@ void vine_graph_set_proxy_function_name(struct vine_graph *vg, const char *proxy */ int vine_graph_tune(struct vine_graph *vg, const char *name, const char *value); +/** Get the makespan of the vine graph in microseconds. +@param vg Reference to the vine graph. +@return The makespan in microseconds. +*/ +uint64_t vine_graph_get_makespan_us(const struct vine_graph *vg); + #endif // VINE_GRAPH_H diff --git a/taskvine/src/graph/dagvine/vine_graph/vine_graph_client.py b/taskvine/src/graph/dagvine/vine_graph/vine_graph_client.py index 841fb55287..43c7b7e210 100644 --- a/taskvine/src/graph/dagvine/vine_graph/vine_graph_client.py +++ b/taskvine/src/graph/dagvine/vine_graph/vine_graph_client.py @@ -84,6 +84,10 @@ def execute(self): """Kick off execution; runs through SWIG down into the C orchestration loop.""" vine_graph_capi.vine_graph_execute(self._c_graph) + def get_makespan_us(self): + """Get the makespan of the vine graph in microseconds.""" + return vine_graph_capi.vine_graph_get_makespan_us(self._c_graph) + def delete(self): """Release the C resources and clear the client.""" vine_graph_capi.vine_graph_delete(self._c_graph) diff --git a/taskvine/src/manager/vine_manager.c b/taskvine/src/manager/vine_manager.c index 5ba406a29a..e4df62c292 100644 --- a/taskvine/src/manager/vine_manager.c +++ b/taskvine/src/manager/vine_manager.c @@ -595,8 +595,6 @@ static vine_result_code_t get_completion_result(struct vine_manager *q, struct v return VINE_SUCCESS; } - t->time_when_get_result_start = timestamp_get(); - if (task_status != VINE_RESULT_SUCCESS) { w->last_failure_time = timestamp_get(); t->time_when_last_failure = w->last_failure_time; diff --git a/taskvine/src/manager/vine_task.c b/taskvine/src/manager/vine_task.c index 0adf5c7378..8616e4ecda 100644 --- a/taskvine/src/manager/vine_task.c +++ b/taskvine/src/manager/vine_task.c @@ -89,15 +89,9 @@ struct vine_task *vine_task_create(const char *command_line) void vine_task_clean(struct vine_task *t) { - t->time_when_scheduling_start = 0; - t->time_when_scheduling_end = 0; - t->time_when_commit_start = 0; t->time_when_commit_end = 0; - t->time_when_get_result_start = 0; - t->time_when_get_result_end = 0; - t->time_when_retrieval = 0; t->time_when_done = 0; @@ -161,15 +155,9 @@ void vine_task_reset(struct vine_task *t) t->time_workers_execute_exhaustion = 0; t->time_workers_execute_failure = 0; - t->time_when_scheduling_start = 0; - t->time_when_scheduling_end = 0; - t->time_when_commit_start = 0; t->time_when_commit_end = 0; - t->time_when_get_result_start = 0; - t->time_when_get_result_end = 0; - rmsummary_delete(t->resources_measured); rmsummary_delete(t->resources_allocated); t->resources_measured = rmsummary_create(-1); diff --git a/taskvine/src/manager/vine_task.h b/taskvine/src/manager/vine_task.h index fff5ea553f..ad03efd76e 100644 --- a/taskvine/src/manager/vine_task.h +++ b/taskvine/src/manager/vine_task.h @@ -117,20 +117,13 @@ struct vine_task { timestamp_t time_when_submitted; /**< The time at which this task was added to the queue. */ timestamp_t time_when_done; /**< The time at which the task is mark as retrieved, after transfering output files and other final processing. */ - timestamp_t time_when_scheduling_start; /**< The time when the task starts to be considered for scheduling. */ - timestamp_t time_when_scheduling_end; /**< The time when the task is mapped to a worker and ready to be committed. */ - timestamp_t time_when_commit_start; /**< The time when the task starts to be transfered to a worker. */ timestamp_t time_when_commit_end; /**< The time when the task is completely transfered to a worker. */ - timestamp_t time_when_get_result_start; /**< The time when the task starts to get the result from the worker. */ - timestamp_t time_when_get_result_end; /**< The time when the task gets the result from the worker. */ - timestamp_t time_when_retrieval; /**< The time when output files start to be transfered back to the manager. time_done - time_when_retrieval is the time taken to transfer output files. */ timestamp_t time_when_last_failure; /**< If larger than 0, the time at which the last task failure was detected. */ - timestamp_t time_workers_execute_last_start; /**< The time when the last complete execution for this task started at a worker. */ timestamp_t time_workers_execute_last_end; /**< The time when the last complete execution for this task ended at a worker. */ From 4222532c8f469e476d95277338c3e2ba20a85b1e Mon Sep 17 00:00:00 2001 From: Jin Zhou Date: Thu, 15 Jan 2026 17:22:07 -0500 Subject: [PATCH 113/113] use processing_transfers in vine_cache_check_files --- taskvine/src/worker/vine_cache.c | 21 ++++++++++++++++----- 1 file changed, 16 insertions(+), 5 deletions(-) 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; }

kOtYEE~6jOT5x>`+EYdod}(zNj13&ABrapN#!Gof)f*0*wP@A7uW3Nv1S`GbsVM}V6Vg%aW(ESyxs;3|>C z3E~Mii6L1aA6(Ks*hD-;dJ4)!6diy=&=LC%0xG33n}2X0KmlEgrD%J7F3{yizJ?W^nz3T=Ornau!RT)aLOW90tRd?J#YG z;Krf>MjXO6@xKTNF&jbjBhZ_A!cakZ3j8a;Ksa9s3i;|=N0t>y0}y#hGlX1QNz+1r zf7Si%Xr^$7B$r3>Z&V3?1KXPwh$!HZCj#IX_$`TR!|n;1FuX26K+?DLqojR#eQL`WmBC|InL>9GRH0S0J5*t5+r5r7Zhc~-_PkziTE zP!q0aB6y`i9(WxR%^%VfXCQCbuO+4!J6?Ds9VY?N=*j_a5#-&40ip)=y%IoJcT2z; zf(UQKAzV*z5kmvKtg@qO!o8syv}oNRuR-((v5%m>E1|4lf{{nrRpB=Ud~>63Ocf!-j}Lc_J003^X2O>74d3gjyP9d(fj zI}b>((usJ~2~8jYaE+1`Y&ju9y=u1!^a>=K5QvQ>BX`lV}4_RC6_4xGFRPsR%BGuAs-1&oMSrYSDyfOw}k28E1)r69}dE<$q2P zu{hq3uoSRp3;=z?X<#USbZK!8tAz}Y!7yoo(KH-AdQP6O+r$YYyf6U>8xO@WV&R0Z zXkw_I5JL44%fQ_%Bv_FWPe>#tEQ2Wtjs73%532}9wZgg}G`G?Y6ISux^A`3CW?CNN zF;zn}QeV`b!eCL1Z$SE)4p?$@6nee3k>*w#XTq~y3z@left!$0y-D75Fw3iO6p#}E z0Y+2w1^cmK76RPBABeH_gjh&Gv-2DpB=!nP8qAweMm%OO7%Z7$G$6_#dIGo%Hw@;*w=2B`=Q2c=ks}tO z)2nRW-?pP@poU;ILd1QQ0S)*SVTI}9lz7Oa$`qajTxK~zhKEf5keRA==>bRQ60QeHSBvdol0hJHshXQV zVphw;jFp5)M6}JwgGpFgB(%j)t2)2 zpd0qAVLjFPT!NoU0`PO*Ba$S6so^#OQ(G3Mh@GuOm;w@$+7d}uAU8w0fRk{4mwxmj z8G#X$;pU`KCrhzJg0?*B#C0ha_SY&UgrF@LzVBZ2nv~!@zC@aA7z;!jZ?Yi<40;7{ zZ?Bzyjly36EnE`f@kl^H)Ea9K?>m89F$Hr3`K~XItU?Xb#YY4bPC{vLDWU*rB>p`N zu&z)Z^jRj)=dA%txHQGVd&C=W&~gzyzj2pOPFQ?r!? zNVW@W5&aMlQwr|~)<%$*S8ye|4knu^oCXS#+Wd=)l0{8~2N-HKYq-tV8k`HPG4<-! z00IR*_8Z}RE{r6gIf2}uO%w79p=4)QTM$sly$t7pwSWt6gVLbyGhzFIlY_^-*G?e3 zaIu{$LTd7rmduJLG+DofQH-4;f;zK$DtaL4qS1=yb+^t&qGqw!EzHd{B~H7l8@jN2gcC!~;|$K?$+K z@Vpsd^EMSow}B%i|4?F3hET}xpHwrvBP1X``*&j{bCCV7E}}{(po8_{h#H_7z)!D| z5_N2nFu-1P#KKh(N-4B4#T;G+80dB|#7ey;Q(>}fsRMc!B;Z9ME8+AG9ESLR7^FY& z{-BP{u3LfVAWbR0n$$g(<)ATs}NN*NMj zqAX_t&nyYUrn!Qx;lVH@NN);nfF$B*Lgk1#kW}=))t`T>J$M?Zn4t8NO2cOHX&7Q8 zsV&P4e^>Z#>JRkKAE_F^K&}HBS!T;*a|7eQwWejH`~R&!gcD9Kl2lB%NA|zUh?V5` ztN9SwY2Z%A?kivuZ)=5@IL^AEsJlSOQ!%&9z1j4aFr{!0X^l`Q;o3JQv zBg_Ik$2bNEirJgg4_}_%1ot};xY)JL)8kqzR+Y?H@(jAO);L4jm+F z15fZEH4uZtsG@yVrYIAHzW6d^>TU5)#X?CJk*3{c9qqN z#Og65k_dxi76n5A3_6ejlm@o8h~w#E{u&XCTxK|5G;M_Za}Atp*f^Vg1YJ_g(M(tX3P&@ z%i3;)_h??g+Z}53wd2{CdToSC6ywQ+Gvps&Eder^zHB1qnV68;Xg>L8o zoY)1lLBbRzy_V1=;=@gdtw(7PV+d!PiO+Q~iC(IO-hf}kOTLFGbJ;$+v$i_`py`}x@Kghc=KrA6BvBBjx*h;fLVQpO zdZC6(N`-L*ReK894oE@<)ly%8aD z1DTN=L*$^6VDJKpq{U32z>$G4&3G>O_jj&IkT_smR_vv~5fS6Ks0G}SJShLex+Q{& zjIn5_>z9`;8kEBmD^(C$fD-^j_D-&t5WCESN%moAK8O+j9M%P*Z(vIF3it%hzrh!)xHGwFfZ^GT`}CiOh2eJLW4}kL-L~oWTYWgs7b!$4s>m3+&hd6 z?BA+L2yw!Q+IK_L+*G-yr; zlm6|MO#}jO2l^45pdj8T(X(=+#7AI|J&2w}SAb*%S|Qw>1l+d~&Ol)32@fE{V{Py@ z`gKSpvngVg1j7i)k^|k)2DE-&(bY1;qB##N|DZoXPaEJ+yP)!Cbr zjmAA}6CfMVTzTmEMeTkNqUs47N6>2|608sf;Ii^~LC8QF_GrRRAO6n}!WMm`E)e|i z3C1I-=Y&h4h&l1IpyPus7dA+`0%Z(Cj~)OLg(FOG9%sieO06RzEAfJqI+RMhAf*SY zCpC!s%j*G5z-B8R58;e`2#kO2#@VCJkrg`9jvTl#t!&B>-Qx(HY{k8469J^^Wsp8& z=&KP(AbtYC*2_C`uw>Af_T#uFy7}Q+QJ}V95WH)rUDjHk*hs;~NQTgC;IGAL0l*hFm~Y zbl^1`2qVM+M!szR0A2uux>*faH{dBUShoUK8@0h{rXfU);RRU%Yh*2hTU70yEH-15E!yl_8; z1`9R=z+>7wABiK9px85m`@(;Kz$FC9+`wr3!sMyF;TRm@dI_}D+{g=uaNQETRYwq< zNhTbm%oTx88&Y~!83+j#VZ_YfgaEitpbP#$12u7icHqd#z z2}Ge~Lik{!rwAALM8AcuknY4Fr=joY88`zdxDpdR9sL$wR9%iB_yZH~e;~u(2d=br zd4M0DS%(g2MK}D%cF28L=`ZBhC*;=zqE9kQ|9g5`f}H18dvA5Av8>w|<| zPl0F)-T@GlfDdr$+baj`n0Qk{42hUQL_NAXodT~yh*kxFLAuACg(Hk;sm1z0?^aAn z(8~vfWC6iS=0)1$*?wy9HvP*8@l<14VD6+f_i zR8|#C>!W`^pYdlB_BD z@2N6r%Kd$1boYj&jOYEok1_D__Zttk@je(gpREvJY~-t$fBK3wMs1+{V@!Lro!6%c z^&#bfo56nXcG0(TOs0hE*8UoKKl*E9PUhgBtHqp^*Ld?YCiuo!^jVId(cHns8u8hsfTgg#+BU2;U#mwV6@4=Z*OV>l?M6_tE9EO)*ia zQ5o{a_fSzbhZt>BV+`$PW8ZH?uVxt9M5oJhT}4e%t%hD#;Cgzt!2Y+kh6ZW6MkO^z znzPGjt|=NVUZXine?vgHBjKm(_a|4ggZYQ{eSZ>?oz3se_DlI;Oa@u%i)uvfFM-fOa9R$^OU@O{0f*UlVD zX!l{dL|5(`y01vwRr`ZSKZl9LuBE+3bzTw{E!EXV=PzV8eQ>$JO3Nwfc9$i!@n(pw4yYt{5GpSq&St(=iw>DTh%(BOv~({%}-@DEio$7>E?ys>*_b$jHT>Ek&1(_)8Iy%=w)zns^!cg;x8omR8mY>a$rs4)^Pl>+uM^8w>i6MQ+#OiC3Fs~D?DHz!+hToD}QEMU*b09VKv>P z%QQu=_wKy<&|L3K6vf7)=KNb{F6pVx`Zd^Wqm*C`@t`j)7M$=FxWV%6>f6`!Gdav< zadX9I&aob`I~R6}Jv2D|)cvjba{97+qvv|_P1U33>geK`@dnI1ysy*v+A@PjG1=H) zj>a~|mvj!%szFpU%$p5W#RQDv6m?ur5%@8HU|ff@;ESm8|2=i{|2j}@p~PoU0aT~a%58A=I!yx zTOP_yUlJ%iap=uddB#gKO;VQJ6MMz3;4PHo!9n$A`jy)C(>I2$gt25RW-8E7!cPstPb%~?@9FI2ScsQ>D)9dL(3PPN zc3WH1v8oD`_9wm+Lj2Zaa!|w3#{NNLNXQ8ADKV~>{L#mur!CfkxL zDfA9JzjK5?G@Wvxl9l&oDhs9&no-V83(TD~rfGQm^E;1f`HXU$Dg)&K9@Pu8%HhEm zg9}-BBz|V9Wm8iA66hH@&kT zY73Rg`pS1KV;ax7=!~w+f7G;=q;%z1T{C=sZAd$ff4_+5=4&Irm?=di`s(S792fSf z1?#^2!AWIeUb%}o`}IsfI7L$ORIn?IOkjjE0v3+1NS%G;pRwu=VO zc*;LlSv8u`?;SVYV`ev&c&w@-Wg{I8Jw}ZuCD?>!hHi=n(9?@Xmu`yBNL!Vb>JNQ` zrnNj}AWai}gSxdSB_+)$E3>Ax2;~VH)4dO-ex9n?xzPQ4Ao_I5prBQhQo4nIazgA( zl6=-2o%#p!Yg%&fl=>BEUIT$KWpM+o5zL>A4E=nyBX88-e6Dt9%}Zx9U>%~y7betX zC$=j2KdiT>AK}jWGp)d#H8P)mp|xz@qAA>LwD$zVy^~&EyiTv4O)}VbcqKgk`Fd{i zWy|sO(?U-cjgMt=dq>WihkRyQG%NH=eAU;skbdRXSq)c9+pn%8ziaf{T+HTZXd@Y0 zOphmI&mJyyi!xnY%xsAkivIX(oTKEKXJm@6@UPv!H7qN6CN+NZUt4^u(q#Lccd>ov zeWvM)?pGKKf^NM`51+c|8L}%47a04D+s31Ve#=E=i*1oY1$$0e^msa4ay`iW)G1Wx zklgbeN6a;q$O3+ubPZ`{(NAsqiSDA2qbIZWrD)8S*h@3&Dafd>4Vrp7g!t~g>_i>O z!S^6q1IOI`D(6LNq%*g9Xqy4|4ONFtk)kZ!%{d$Cz6`YKV!yjPY*b&X@SvONQbkox z>1N+3jdA9mF&g6<@p-NaUV*e-aaQtPY82-Elic0dpGs9ZI2Ogm9KD9{DON9$1!nH< zn2#558gJ;Xc{;4~u~uVr))G$9c*mDmpW{qd`l-!3Q)#og+t@j5ll%kf$gO`AyzSuLS4>%?8N681x?0o7siCh?~;+*_%{V>3@3jXYCDe2{KTwa zw^l(j^Su~1-@Azoj0vva86z|KBSSQXZv@ox&b-+-)J8eODL=Oh{i9k~&&~Jf>n6qo zs~>wK8}99`;+>f=^cXvQ!=95SA#YPF@64P9H{ayWa1GalCr!K_-~GAyTz56({AjQl zX?xaaJJPmx_`b)OhKISroK&p8!W_N2AotHiahAw^J1!<_SjFo#z(;<3VLm8@v z`f9oJ?lQ>ChWTm-9L_H5JhH|d13v^As~t_)8uY02G^wws2EJx8w3%Ra7}>o;gX(sG z>&O=VL{%;t2P0Mou~-eNFC~4gdm^@lP{rir?9W!Fk}jQk&C`0~oTPUitj8UV-0f>R zF1=#0l-#<;%h8i{NbTiOPwvb#4VGCdPvfH5t%f|9T8*Mf&RUIaw>EKOG@ffYjO4^r zOLAo&HcZP4-)$>n)xQ{@(-ps+@}QOQI?C5(!od_GBJ7&kD!w|^oHaG=TeIDg+2r%2 zS!33%wY(_Tf2-z%iomccqbc7d{&YRtZg-t<4j1u_j*scja?{9Y1=FmT(+IvG(`zC6 z+HD(^N&djwVJGIcoh(?+xWoG_f;|&z9GirCx8<~RiwekLHMUB*h(=yux!Rc1vTe&< z=1z&Zlhm7wnN7BYeht_Yd5)H%`*}_Y#p5+;@||MT+0ERahpMz+a534Gk8>hMW_0umlI_whp$@VF5oe*vgSMVQk<)o7rb^ zd6`Vbb&LkNEqrl29&3jbWWp!bMz-+%s>*TfJn9xTddlBj<{rbW+L`FwS8jQ?W%K=>4F_8r21-jp_Rpkn#Q?bR2s;JBLl!=1|v{1`)T4t1opvvFq6J7HW)!)I}U^z&F@u<&2E7 z>pk3Ac)3&3LpMJnp<_(|-|5W?AK>R9(LFKgUw3-S3Kj0Z|Ii@jeQJ7%jN65xq~SUB z;PqB5oOJO6FY&PyoR#U7YNK*S^CwcceSapjht}>DRwp<+BDDAJDl_xH@Kh->}Tz0Vub2rf~Dx%I`tXfyxu{- zdH=pGlq18rEUIr(gq%Hu;={C`Cwgs;?^tih_~xxIU8<Vbz zsY;h-qr7U}a^>Qe=mV0U#8sove|5rtA^Qvsy~d-T+B`hoDm(~H_h!7Uc^#{9lSOo> z?LFPMPi(V0&si(4Ber4=~QDQ?PeaiR2V;s24Y@#*sUmYj7B z%AeY_&Dr{yH~2noewoADe4D$RJI9jo_INsXs%>O_ezDX!m()QAi+5vXIqw#X?_lEJ z*U3`u-yna8%kzGF<%zL^x~FU8*6Ri8*Er3YURRKF>aRI|C79bP)1hSp*Uu(RPnRw_ z%F`_yJh_59&syZCw}mLq#&@*QS5MKDwVgYeb-U3K@6X1%BUs3Drb_%(*p%o^DvxeO zWrLWc4(-Z427Z}b3++_yJbLCe&g@e)>|005jG5a*(^c;T-Bsg}!+qXINqOTZ4NcJ~ z8*^?r<>^i(ReCnaEDA1Q(fjOmm2|pHau zF$i>OA4nK^rTd|$I_h%hyRox;n;%d0-~X2BQ+4y(J)bJBxeDUv;Q^>+v$4zy`s}d4r{fw`aY%vl|$Hg0msao?Rze{tE(wjUSoazy3js4 zK7#pD;@7F?F(xcDcVEwk$&Us^asPT`;*s`~dcDc1M;rBdBD+oc@j-Pt1tOZU|z(7pM(F^W1X#sO=eb9ZhHc940iM$f$Q$Hz+B zbZfJBC(fj9`FrQYNP8R+P>g#@=z+|Ml8U^UJiPFWh zI^}kMh&C2nV7TqG%VtcN-=xX!7kf;(a9`WCg@}mB691uTkvWlpk*VMSzU!~62U^(Y z9Ah-|W9LSGN-t<1doOV#=x4q@-Pt2rEzbGd?=|Y&=lQN)TvO%vUHzc{=Aj-<+%AT^ zN8Sajt=BR~Rf`HA>^4!W%Jf@*+=gcFlJnb}B5?xTuO1#eU1e&#H@ezrzheGb(;Obh zCp8rf14E`I;b-BosYW)9sU;(dAETmQv8 z%k9|eu`#0?&T7f~@=DvI?lFmdjT13_z!6~9Be1^Ar2j;d&w+8d2eP7^rL$D0KQTR$ z?N&IN{N&pv^VtcHPyDGFGtp|X5B67WI2adgeLs+ z0hz_G_@9AQ)EnRNvway$SrZXqZ87|tRY&CvkGFnRT=o0o_ssN+UnIHXpIs1r$9T9q zx8fj!@?B3qx4m}f`E7&z5;tB`-xNWAJ|Ru212cvd@jY&6a6Mi2!K1fvmp%&rHm$nY z-F?dP*#YYsMfwxIVmfR8sO+j$O*&OLKB-0rz0?10{6;}beS4(5U)9%?w1otdZ(-q2 z8Cj`(8isTAH72iF=2X!eEO737g*lq@{zW=x~HT$!TMdxe|Bu$ZAAw5p~njg>es@r#u;U4kG=Cho$^5Jpt7C72(4^XPNSBt^5z9)~S`lh8&FbTVoo5y;PCfAK8oeV?KX?JB zCwS9~9oO8Uc2D=ixzEjer?d^7d)oW^G$2( zp0agLTi)(swrK884h(ogC!6LytN%xBV_Hd9V#&PuenH%+?R7St2e!uU#>S?Z{2t!c zo5?WqV$*%2RKcN$!Hu77N_gdUn8)8U9CLYeGH--xYh&2N*&_Z8;!O=FUziFt3!K)w zaQ4lmy}#1A+QKXIQ|%i$C7pYe#AkXg*uF2DeWo8&Vs<$rX!l9!#<0c5U6LmYX7vRM zc^lb8bz*dngGGcb05v39NcCu4j zP*?YAsWg6#_8*s|f%|-NCe0rOTo=m>b)_DB)N~%{%ZM>mQ~o0w8K_B<5cVfxkqL{* zdz_To`R*vav-^&ShoNnbC$GP85g9#^9ir!5Et4qW^Z7F0LH%`9gEK9gWUcVP` zlk%<^{rQcVib=xabysmwh!fq}g8O#vc~X@p)(LzMN$8}XeW&CoE2In-z8lVb8v=CK z->`evpB{47)$-(?Lk3)*ZS>d`ii54sC2tc7do6RSSU)?>L|6Ko0bkcG-T)7y?j3sK zoZi*Fip_&p-lWkqy-_ipzE>E$)l*aEYxAv|TjqZ@VB~T4??xYF{}7R#dd1T3SAVWd zeH)|Yseuf}oUX_9Pu*nQOj3kRaG_mwV&!Hf)h}GKtvX(a%1&$)z%Dk=eXv|#X|R>S z-9Cru!pO)@?UF#g8)Ll0v`6AurQ$;?)H!`|-0MJiWP ztXX=gb4D$!el9pvSm8zW#PJ2E)||7q+A`fcT5$0NWw(mu9>e;~IJ!TfEunJNDOmh+;%`&i3Y-Tb5PX=PQX)jw4 z`*hNw@9Wix9`C)|MP7Yw`4Yv}=Ww>4g<8aZlTbzR6Fc|d@{HS7XYxm=?R!d`h4K)KDl_2e5*r$i*qzJeO@&zf zDC=@h!7$0icu`8lCwFLz+IA$UT;9JJ^Q6#j^0+l^hxXV83d*OvUz3!=j_d47c$HqW zr}T=?#dYd+MLKLhu2DAe(S=8H(8kl69QmB0>Nc!o)NT5}=f#!hmI-^BuK87_4 z!CnKMyBqtCELrfRI?Fj12ddMgwRoE+eBCLnc_)~;e*DC(O?6qgCB@U_QnlN(-RNb` z%)dL%z;&Wr^+Su%WsZB-WxIYaSSjZC9y>+r7#p+6Ox;_)PTti$#$8_fepY+>BBk21 zknF6^b@=UckM6grFn@K?3p#rsf4kM_>5dpF$#?Es<5+4X)@9>riaciaF^1WGo;Mmr!>ja@?-gNvQXT9nGmYmUn}(eyQcg!REO%#b1@y z9%;SmR*?SwZ3N>P#qZkR(%Bfzb%)d4^p0Lrx|L?owfTzA>sv3xj&`vs#0*(>iZ-tw z*E=Cs)$L+YI(oD7ZFqnOUk;adrUuhWPy1?26yB0(+56i{O+eyu~Ux9^~0 zK#jDxV7uH?*<68^n1igv`-@JiDsM5kC%P!gyDxk9H0})_rcj1jd~j2wy?N~}#=AJ? zXZJ-br)##T77Z`)D0nc|{h*obma z>e8rjQyG_r#CGh-+S`dw#71vQj0|i%C_f?7Q(N;+cc6I37az4q+haOw`11>;e6Xy6 zf*mJjteyBSxQbEp+h+ITT{!Fl<2dbZeAi5%aiX5QJLZ39eXr{F*X5xc#;qp~w-%|l zzDvlg891J7zJ-zLk@xEpq6fb%F0tkp(pR6=h@Y~`35}xqE!J88@Dxu~bQQ~%jSJKT zRKa_=co`GZjnhQ#-;Hpiv-_>gd&*py-S*s`PEqxOBO1!KSBhyXw!gL&-TYmxA*|#1 z?*lT=M1K2D7{$#SH+*PiY~Or!gehDw;LByB8?W+qvu17Bxo^Xs#I>7p73#cw=nltv zJ+ME#ruE}GZ^n}o=QnKJXr3STCxQ9laa`M-=fMhYnHT2Dj=U-4P0t%;(SGK~iL2l3 z6NA_5%AiO~q7|xC!R{4_efENHI{$?Jx0h!ADtgUNRAjX8oUP>Srq0p7a-m^_iX&C- z)vb$sCwH7-`XhNVCXe~#2QF?qjwPo%S01i^@l+ApBNO{9e)GYvo8wL%-@0YXz%Prf zS9yQ!({GTt5nsvhrQ68wk>#GvH9;v-gOwxQ61*yg@3cl43$`iOsC7CvK6*G9_S-5` zip84p#tXNbd%~u+$quCKJ|VP0?iS~Rdr1eYYP|Rk@NUpC1;RR4{& zbaR_^3teZwEysrAcJ2KejW2LwFO#QeHT& zZu-N^(!n2_c=$$Q?!-3E(;0PYf`*5Z9-BmYycISXd|;HLmFSqW8z7eWd^T2;@wUAEB=}$v%Plpwp~4qg2!ptv{+~~E0+rV z53K)oZ}|L4hs5;B{1Kd48$%g=F}LBJYnlGiqUV&SA2NT#>dDnT7E0Ri5?o!wWXW0od>* zsjedrr^dGY;hxj<`udE9|5tOK+S94(d5!z+Uffx^&fBVM65m5fzGbC<*O0rft8g~87J9*<7x)l#?;EfzEc-Zm4ro`We zf2(aK*TcP6D@7$XKYqQdiz3L@XSdr9*{iv}`l?!!8#v6^E9AOra>oi!8_(GcojEkO zf5^sYZgS}Mg$RC05w=_9VSkP;un8o`sh4`y8D6jOS}bG;#2j|po711nR;IX}LG;Xe zbrboRS+3~Qg>~mzCNTFNIL6oMyc(c!B%(J58sMjaAw2sl-CHiYZ(uYHY0&*53;6 zq&1?}yRvw=`vT9lchfam_r!{5t&bho`nCV+Ldu!6&=Wiz>)jvHUS88uT6Ie0P3n}X z#?i}RO+PcFG6t`|m!Us1G@R{i?BZc`G;Z3UqlICR;r=wJji!k|%ZBm`VzW-+_h>0i z@LTL_g(!0t*VDz?U9>oJdYFCVm&7jImUi78ceNUC=2WKUt(_klo+zUA;FkKCTn@VL zF9fT2EXMk8RXT2s^rZh-oLM75w;0pEQ%%Y1>#dIucKV$TEwVcIa$Q73#tWIrE5aK! zRH>dC2r0%tezd33|KUeVaV@cM)}xHU8`e;6^|_NI_xs!KOKSrrm$DjI#7i35MaO!i zlJ1QQzVKUX*gv6d&_5xh*X}&38$7_;a!ONu)(rdP;DwFVO4$2lO+1(c^G>n&bF-b8 z#x*-9-W}=@s)(igS*MS??VQ|L)P7R^{H^;EF2^IC+mBW_f56LXWAYqNcic#~dxhgP z&|?k_I6)Qc{89JonvL>mKj&rMQ=9F2p{{rSQ0>)$C4CQv#*Puw)^1iuPL5!sk&(9J zx+&`0yN)|g8x-z$oR%5q^5W!onSYW9BIa(p{#r}CwvlF_VbW!x!yb2jUo`0rGX^$(btH@Kg4xbwR@f8@eb ziPR9G<^x|JO{W-+Rg7d}Oc>=Sat(P4*9NS+mUF4hag-%DbDQ|(BHW3Ng5*(wjnS#% zn))1?`vkx3U}K56pp>$PeeDgUMpY|s*%=EC2fN1h64kXYhhLVcUa^Rq53JGu_&}68 zx;Wy!#kp37bp97q(nA%6xq`bnP31nMf0WXG+IWBc58-~6-ud%9Qy=uDP2FrJzl}Vs z>VFs^pO2ZWv$DPbZ?N7eQt7avt?*{;@Mh8POOB`M@cw467$jSGPwiwmZ>VDsy^iDu zqo9JUEpsg?shQiW=0qSy`PriW{?-8*SzUcOZApnsP%QSF{m*o@$&*FhGe2UU8yvjV z@8kZbtTgMTqlJ+B(B#M9{TDX&?!2k;Xe(`nn4rsVWx--oh`Do1+LM|)@=Nua28iF)@!dy;>8{(77J$Y8LpuY-E&(#bWCoWFgN_1RZw%nx-t zhQvX0riX%DTDj(25+6_hpbbg8W>)p3?x*K?iE7)J$QFelxkV~UdYzNq_bc@$rN=(( zJeu6YFaFY&^^*0u^M5?My>hapFeeo;CqBo=3lxNXe(9B_HfbnQ=&L>Kbq&VpN<;j_lvc2nREpz;xY*|9p^u{&M94X^o@4(4ab|H+AAvcFLKR&P4PGvq-off1mp5+ zJgXxdrx3eVuz*Qha;q>eeF4+kEc{y9iv0pNr=kX0z9c|*_J3}@y~Jv0bFnt|StPYY zzFC#?fj?!+r`vkO+eh2C$fY^iW6J?Om{LjR9WNMM4!q0i@usbyK4`1J zY<$#I2NQ}9`6+E+QRm2=SklmUnc7NXkIkA#Vc$MItnU9_Qv1cWCVGn?6x;;`Q)nyR z3BW|K`=NPT#BjVw1PV|Imp#h=cY-!+PvlAGNMe zagJdZdmLaIn-*2DZItmuLYwm8@&lhw5fI6%G#m+4J2il4(!2*^2k`-nU6Ld$VqfOLs z=z9K<#Y|55%GZEZanKtr+DA65k8Efk9Yy##@hp{g_gF?eAWU7zBh(qqKV{8k&5pk% z0ISMEjw7Ia|CCiYwsnLG$t`YM-@6O?H=(TUGFmRs8|>#(tGiRf@aE2EAKsik(rS5C z?9mmmN8wjazPb4-L9wSdMM{Fw*Z`Yz_l$Xws^pHkrP*%0xKmBH#k^tcfuWY&-}gA) z-6Fqddsf(N$$hG~6&?4fnm#&O*#Dm9KATdm^}+tmfNuKsHm#mvSgqM*@Cb0r$MIuz z2d>UB2o$Vg=ol9ih;~#9xWbXi@1UQMYG|BVpo3xU5RuxwF`O|?RI@qQ^Q}$my`9z1 zuLX#;M3+-jKK!B**?!NpCyPtI!uglcUbmh=&)Z@q;4?Y&*kXI$1F{o+gd&is%_f%bzqE}?pBR5cZNvX;$icY;IcJ& zXH>h@-yGa{E`#}JskgA#%ixm7v856!*-l^WIUNl+L-qu|(&6IYVf$U-%HeB{d*2FbJ42JNh+ZPK4G%5FGmsvdGYd4tXd9k-!fk0vzOBvaaA zG8i{dzdx>RwSDIaJe$Dty`MB*hx)#cO#NfgS9rBK>mFP7k#mEO#yMl{SWPectl!*y zm_h38YxNIi)?=>@+!kwhH*>jgXN_*(*QP;5AP^bg+LXtMA=B&!;G^M);Tr(zG7 z>m-tuwL`DsWUcQxP>yprNEn?|5M?doz)6M*M#ODa=u?*viLbOYcVMTgQa0ujbMrL+ zO#3?IDK*2+zB~Niq%3>{zT*F^ttY!AgE1Lh*JntPHySTrEeM5zij7h-G z(_@%q@4`O`;^T(D_>bD0QtbL1n7a<2@#e{mnIn!W_9rg;noizP&sa>p_P{)+WBuaZ zuNEHykY+NlG22Yc(xkOpMQdbo7u}}IQ zk+476M$*}J%a$(7k+i~!YolF(d0{2%D6zZt9$*PPk~Hjcui!{vrgulM`jnqsPr?TxV;#4wn;<93-L2cwS1kfU$A6aN14^<~4M^iL*L+hs*BhOAeP0s&^+Q zUo$&+%faTp|CIB=g3(gp4y^dknM>lIG}DSjj=nu~;=a?AT@SYT9(~#wwufy(P6-BG zC)0`rWly|xs@_>b-*bx1=bk|qgj0&nO&>m;VlP6=eD6+i`^O~N6Yt);J^m=<^u)j{ zc8gsmUskT;(60O&hpr81r+qlnEGm2Sj%y&NgKOWGbGEik3+JPO?VGyRn!gjf`J|jH zu$!Ahw(BJA`69b`$2%8qD7?^X{Ri_&kXz?(-k^be|K)7YWu)z@aM$Q z%pr-GbgKQy9R-Wh;m zx%RzzaBQ%xvaF%2vnbOowj|*}HS^=~Jr<9T{W;pCP!8tMz!B{&wdFmNm+o&=TDU@Q z^!DrbPY+MXd`oO?-mI)g>mm5O=euW9(WBW@q>9SVHmywYYcaA#6b>@`045!}0WCZr{lI)X=r6cc^Sp=WFPpQkal$bYmKlR6< zus+C}Q|zs)S1_AiTiw5)<|pJY$>X~|7_PCp zFaK$GSPmy+ec0W~SLQO$e;q%j>lV1MC&svS*ShH&v)4|Ue{ENPBJ^eCSEVaAg>gfD7|8SJ&9va#3gXjI* z(|a}6N?IC{TXlREy5SvZ_B;aWVg4>uKt1jX|?hF zksCV}6KUM+9;@r{WLY{*Zy29*Y78k*yc#uSo3|tPQqjqlo*my0(sR%E8uz{_9SR6 zhi~MY&G#RwIb)K{Qs_=YV+} z9h6mfmZT_~nmMx0cbRIR%Vr=+3bH?qRr zHh$yVuY@n*dVYA?L=l(QagV(eLzgvJ9&l6W@Xu1IYt#{0bq&n>)(ElQvA!{T{LEGn zvCaXzzLV;gHptO1aa7%F37~oB6YZsNXQoiUmv-k`oV#mM?gu%&V2_f7#e+jTMxAj3 znghG)qN~{o-E-rQBqt?YU|)EaMK5Wv0{;J#tr+vc*76uiC`guj_{)DgRP6%P! z&Af}LtHU-sgvP6B=s8DSeMVoEP$6EB#jb3?+j~pEeBA!R@fd}D(%k##qbOUlV-SI zF6?%@w%%d~^&=hC!nN{l4E*<;I0I8iIeGCZ_Qo?I%C8!`-OOkMF9pj6 z>8fs9`cSsDZt);iSaklX?Hivn#%fw6mVvlRw;xSD-p$Q1;d7CXj_q(7pw`%-C0Z0T zAC13Nckoz}V3lIZ6wRLbQ%!SQ&ZGs4-zk!^;}*1;{r>=!Kx)5}EoU>~SF8#=u83C= z%^<pGcw&T5$bmk+gsJJ3qORalKfI@m~EIZ$meH#^+J*l%hNR$brK< zph`V5ccue8L)EXipOx^cKIG8ry+XxHsfJ$hx|i)x|w?TBV8V=<>~9+5I< z9AhtJ>Iyx5o~;_dsgyiSP9-bl z%3Hvxlw$PJGB8bcQ5VZD>hR8y9r_!-_z6I5q7>tu)fq1psPZoB%sq~EQndSG3CIoD z-j`skyN^mpd*>!bzJ=XQ1Fv+o8T~7U1C_C}cDtn@w~G}ST4Udx7Jm@Fx;%NO)$qbM znpfC`vn*V{(eXArBMi3K0@9D1`gWALHr**4*z?6}PV2GhkkNQF-X72-sYy=gxn?QJ3ofA~{AxsA1fccmEbr=QZ@ zC*JSy<=ge{V{-JVPr2n!`ixtC|7T{jyRi(km4Pl2(59~hjN-p%+{&;9N)cM8aU2@M zqzsiaz&%qYxl{s#PT)$Kjkk({gQD?!Z>_VyV9U`hl@S`b?KU$@w)d4O zJwptRaBz&JI{oQSef@S;on}cf?j>^NImf6x7kx^VCsh5Z#P){{wPkSD?OcwVxEzYP zfK5W{ZnqXYIHp@}w>CidQ0Bd6xH3H>vi^Dpv(W`BEMv~6{i)yI!7_ZlJea&(2CkEV zTV&vV8Q3cU_DYF*I{h@|&dBK}5nbBNp`CN5Bw*C(B4rvnz5T8_tvzslB1X=&0}9F1yQ3D@3Kre`?mFfKfyRjFG$49u^k~@Pblu%j zjfk&)18Y`7^+)TkiLY1oV$i=O2buRo6%qWBR`*z(+!t7?Gwz8&Ig2f%|Eo;>Uja+x zD%$_3rWM?4?XgC%{GEBPy%Hj(LHj%;^3T%`^~X)QkJ;T}mKcq@Kt$P55~W1|&YLmd z9E$-bbYI;#-R*HKi;fZTu_zcS*_HRzJ%ujz<-^|FwB?Y@8G%KuAu^JTJ6e9Epp4ExllES=SH^y09uXvXgU#nB6x7mU)iJ<>9AK~dzW z3;+Gb$k9(~jzp2bl;~T#v0rwrGnTG}5HLr|XYf>~$^C%rQmy3kFG85D9U^45%*z5r ztuZ`a@ZpCmVQtW}j6ucby|6F%N+HSs1_FS)S6=kd9#JGvO$1&i0@X5sY9ioYtZktb znEHBp_s`SrmqK@QnphDxsuj_^9&Yr5SP?h+dRVsAZuIq)D+mu8DTg&OURH0ETOa@A z0B&!PF1kBc6~dU&anyW3LY(XCYH1DU`mCQXI`?#hpj*X9#wDHNQmj>_a0@BGC^IS6 z`mAT6*7~~Gas=&}9C0X5`Ne}}<-Vm4VHuYD@Uv2Q7+``AKf@xKL}WI~XMpDQ>U&9` zQNA{|_y4`nT4s(JM6{uTlzEk4_QObOsIRRhZ$o{hwzsl>(mFz$#!c`8Y(zbm=nwF< zw$LBo>uO8?n1w!=SCNZ8Hn})ST>AK|hkp9_tcQO3_{!yd+_iJ2Yv-}tI{5}Yg5}nU z7&}Y5Ry`fLJ&mF5|u8$uB+5-on8vF2&P+WDHu)f3j?_G~_>)S%@ zc!Vj119Af&wGP_L+i2R{M@c$Gyo&OlB&Zkzfq)HYtM9y+L5l}v8ASCdz?2da(?NQXp+ZB&R8)2e@i1p^>Or_r3 zE1ZW_PoRu1XR4n_T*X2r%3JZ%iOPKI1^$`VqYGbVS`SYhR>PRb@rzALm(TyB-3M4! z@ZTwqbHo#w%6H;0><^o;lbQA({e!9T`}ag0n?Uqy(W6(hLOHTWoxL_=;RAr4$E6y| z_vV=x@3m#k@^PcLrxtSKJjo<&ZNV8SyUt+B20Rg=+}Q6wFh6Xu zF_S)QvEvD}z#B91n4hPTy>-Kyho^)pGE?OKc$Yz{X4hV98Wow=3wbLtds}*lip&{7 zw;7gSsjbN5N3(a)(d?C(cr;ra3tgRQe=O9n>WlaC&R|`p9}KemZdplGW}z$_^RC}3 z*$6yYX*0s;kg0s0+88}LYJS_x^VlJ?T|jD{zKY7_T5+cGT@>;=zt0HAHDRNwCsFg? zXY}Hl|2|_ZsD%2y?=uzv5?v0H;e&Vo4DDzJU8#@V=j$00iNxy}-8hNYGlpZ~p~yQk zl!}Qv#Q+`1u>Yq3IB&!;>sA<5S724?*RJ{@6PE$tZWLOBks$1B$LJeGR;)56w?a5rPQ=4q=!!C+qX zv@uLq^YPfz_W7{s;4e3Z7}~d@lJ(k;mF)72qKFRu@{BVrU0bzU5peJ6`_J;otr^y? z>smET*#}R#*qY&OvkwN+@~-#)cl+UwK4UF2ngfrvAKtxB=-sj(er1Ht?Kb*fn~{#!_}gqx9W_H-Ov5s71jFZPPeGGK4&c|ZlcT-Cz+x?crI4e$L)`;Fx%{} zg`&m9gOw|^E9VQ_<2O>&z&5esWh|-#zl$IH%heg@?#KRebq0NX8q=RN4l`-{QI4KHJHufySA?lsb`9P_W3 z|Mm(v4>B4}Ze>{deG+PceY*DiH!~qczg5Ce(hAMPZm>I^w@k{}%`_V;?(*CnTyCU{ z4r|}kY+jyYO7K7d^H&a7`zDw|?r=boO!k9eQEzy(Na%XzniPddPmsK(asT%`Y+92i2NFFwpAN8*Y*FeSnud*7W5)?icbchBh1b%K8 zfo2YXSWJr-qIgC(vB=&ps5VJ4ViQGVlp>D~M9*!E1Xjy=nl1xy!Ztb}G63Jq%Rg9$ z#|4B%iY~DK>!7uoDBNQdzqL#lMv;#mj4~x;rfJenS{VXr9~~6aeJZh}7i*tiGCgBP zvhj=7QnY-^P9Z-%8u z@{L?!>-HBIY8cI=(U~@y9?Yd3#nj^*7Clau;4}Con@TOOpW{A#x`OBh>2<~NtRu75KPeYf6ybcO^jI}vO&7qgk zu{h{0CdVS35KHCCL#g+R)bS#9`YV`vKa~Z)t#7}#BTmHd+xnjNrrKr8QPYzWKkaSL zeHvu*pkOmuU_SK9_PQ#~ z%;XNWxlIy@;TljT`VO!1RV17sy2+dPkCP^d5Z>vhvXMecV@xW##!x0SIrsPcSU@Ps=| zVQtWRrgd%5JJ!-hH+tvDa`MS||_M0cgPd%fr`@kaR$>OVwxNj>S#jYfCu8%TGq zx3pe#r>b;^3A%@f?o_YxC26IQA9$nALVk?~VT$>Lt<@vqTvNFO-_XcM+V4a2@?Yr6MX1L%9+SZ}$nQh$ddRi(SA!`e_K zlyJN~@0>Q^a%qK2({iUxjGQilzcBxuo}N6CQPe9WEJ3bA7p`U~xvo+fz*+GH?~vhl z@5?Yt>fAyJ>wSh*^Nupy!tZ3V1OlfI8-?$EC&C9%qd$lq4c@~)kt(8M* z^l0EPH8Pqwh)lPWGYu1DS6LIN^X2gU@Vf{}IdcrH;W?Y|Ami}CyJFy?*i*-Q(JH>; zriR`VjZB1p$f`>T53k5Yv%#vkh~CYLe>6oqIz$YJpJz!IQ5;lLmDfhQh)JE9SN@*G zP~67|JuHHSPL-nwKgff4Nbr@F4hZXbp9e<>3voHM#C3@h2Ov$!^+W!AG{mlq8VfnP>X4G7 z7y)%v#W?e`-7}DEyA>sa4Med0EITaJp?1Pw0L@A*ur&vG-6F-QmKcHSk(GlnP zRGd3)aUw4mQih4^_@rNEma=Lnj=_)NHJzHK^$9vn;K9x5oM!CGwxXH0`x-;!#<^{Yc zVf}AzbvzxC(QkKr?;~r?gxeke_(=9JVF@bw?T%4YV5qW5x-;XG|D6iD{;yT=?~kqB zKd9j3$5s_|u&Q9CLJ+R!>i$V&LVnQ|?w}N?u+@6ZC*;DclxAdMxAnSDiKMl7#Bi_H$(%@=-eE%p7J$wHu z?c+3TpBVa88ns^9CiQX}ZIe1cq+U)lHmbgyh8tCh-&EoUcSA6wscE>p=%1lYPs4c` zq&q`!n?>B1U7DR{+NC*ZXqWJPUYfbtb)ocrp+xVZG~6>7fFr;duq-X>F^^DAbsAo7 zjPF-T@7vPcG63}S`2PQBA?vYZ=idVauvgH{3W_mn5 z7-o}?S!9!M*rPy3Gyjai^A=|cqUj$?Xu?(wqA3JA+9E<{95zC6WL>UHhUVY$TIF+* zfR+pTxBeOG+u})l0~D1aD=JO`1hsVKj=n?k(l^Y(8bV{OTuQYP%7xX+rJkXD@7_|+ z3@ma)nm|ZX2x)?+FGrf-nE*)sMcO2fA7Zh-O)=`{Y;Br{@97P9X~LbH7RLI8wv+!l zY6gCV6|aS-pJleJg=eC?D>!L+B@0?@d;Te#|Ne5ASI0t+vR_?j1WpS{E0?SsU0)f! zn>1v2f^q*!Doth0rk0sOv}LDKH++TKI+bc@>r|?tEr+D7IBMn8G#^Y=nr2tgG@%QI zRg~1;5t}3*Og$@NljMV`mC9Z1i?oMRk)82}O*Hu%uB69PbFS}fMq=30sSPZbyFZhv z+*yxAZv5KVTArG0I~@tr8wx&XR#8a{kVif%D}H;I5PaV*xbCD7!bySl3Iy5dIqv zSljIs*N%;-@A!^SkoDJ9&At0w)Oq*u&g!y=OM-tg$dpNK>_B&uwi&voh>w{gim?6n z)*`%u;hy=urrBJ+viSQtOBT?!#D?}{R5T$s)DvqRsWV>Lf7bE`xt>MtKeJzQxTH85 z&{CcVhtW~R51P@LDAjxG57z44(Hua9f77%_e~27Fsly}Zk<%L@JPvX1QqC28I%cZGNpQ*kSZpN>b5h0Yk@n;gZyj+?_8;9S-)G^~WWt zvrHE!Oz2=K7xG>#rlU5VZI3@ta+9QZU%MVA-f+t51^{5=H;-bDm|PwCDH z{+=?P6Z}18F(;@?1QSz(a5$xzf4=rtisx87yu#s)zf;`2i$(J9rDTJ%4+nV|`S(%= zbMo({%;n_YOIas|0EbfWu$F(J_Cbm{&5>WR zpSv=pUqnB5Wypx1z zcD#*`(sqnCZDm~k`v+4~s2z(4bv-IoAvlp!%Tb&bKWpU0@2WCB(IzYjoO$HyE?RKv zZzfABqi0i+Z~O%=)RAKJH8OC41h}t|f$L@9b_o!|J;VZE_}LJ^oRE?CJt_OJ^V_fz z`Nb{)9FnH<8{mLhE~9V!i*+3cEMJ#jtW#{aG21ZW7sH!VZTIS5>a^W(e8^rt#s7mt zrB6j<#mqIFeiNv0T6|@u^IO11X95UU~`9Lt;OMZ=zSkcbb zU+e4u;j;OCzedxLJk1AZSwD$nXS4V4>=d}tgc*;sQ+jdZadygNZamITSuTvnIVtAs z#@ZAdt&qv6N|_+$T=+Zpkf)~@{I>C5N6q1*XgQ{*C`WMcBUbZLaAqg2x_|yEZGH+r ziEMu0)pF4H>7&LJBcAh>?!p2t?;WMwuZYYy%GOuNX82o<8YrUTWh|;AZ4)bA#on=> zV0|oVt7Qr_a;B}8DP6d>TBeK<^K>009TjsO`KK>gt#wK9)A2O4+J7gbS$&8|{hi#C zllnV(B3Jz1$xDUePbK^27`8e+#hm;KawCdE#sSHh^7*$5l8ijX|A!9eCl8CT)B}x% z^6X@HtH=av)G#eKUb+O|^z1kA-|@-LKF64?O_Q+nkbH@LQyD0bfEXuj%styszm%oY zS;uNwDxEZpHPmCOqbV)w4cD}tQDec*K7PzPVet=hHQyKEkyaQ|{cNTZX7^a3IU|HM zy)YR*K)|YJVe%P#=AtlpoT!w=RJQ&_TBl^1R=}FAC!@Qwf|BaV*vS2hH5U~zPT(db z8?BGb||ILQ7WoZMSFG@zc3^mNpTaJDy53ZO(DcV)vnUtvYTU zPR;A((C$8NEW?Nn1@Z-}Ey|V)I0N&V6<6+j@bahQb#^a6DR{z4saU1d??luAXE{h1 zVQ;Z!Jm3z3ig%sBn(=_Ul-s-q+~wxH`UCDc;&P3Ls3h_IVR~m3=}|Y?3am2c5l!PS zn+xfv8UGhnne!s5%qkW#!E#B$dG49=O(AGm(*EN0=|Z=Q>eIfn5Z>rchI;R;-b1H~ zH@U6zVb}g)+|A=ogWfPjdN=-X$Ztfz%7pP}gekkXrH+ktdr$tMNi~911@hk1GWzb1 zsAZ(OF5j^t^8IPMbs+L61fcee9d&-}s9R!3-4{FR54EDkGH0ZCZ*BiFyxquF{COAF zQ%1(BjQ}g`T(!bpZwOSg<0NgZb@#SReb05DWvQ}r-4&J{hI8E;o)~!=3OqZ@69EiO^V^OYfdQi4-f= z61(#DU-w1vVu8Zw+Z=@sN=rw zG+Q0dXS)36RP8!era&TZl9rFjTJ&-~6A`mHKq`y@5rwf@LK1ozQ2)Y6a|n7JB&Hn{ zraziYJ4j4BoG#Oa{5L1g0pDL9=*FcnPzO``AhlxSV0N%Oi3YRgOC=;;jL>^x(-AVfYi&Dk`e zPECFn5kg%}gl|i0*-68Q^Qk0b7X4JxaFg??qzS;xKYhh)&Edw&cc{bOo}_fxL&Rz+ z7=G4aZ%^tbci5~!-H@bo&S>v$N%E~vMo)YV!uKTci|cl;!3K9vQX0R=ZVlFm2a}Xp z1x)iqQsU!yMq=MOy6Z>fLnlj;@Q0F~Akm8xsN&zPx-xng7;V&{uC+#VzNG>S;r*cgydhz zd9fhf5ew2Qu^{~#3sP2c3`pH#L7G|@(olx9I@wwwpz&XuTx;WhBw23!D{W^QQq(^8 zY;kfCJQUXFrX+QW=yN?beXcK}&kdUWC&KwgAG~Qb_QCl^AG~=r_L~Js*4h5fHhu13 zH7tL*wjn9_UI-}Nv>G{WNXjO7!`U2eOz>v}pPS^SdAV)G`C_#fS9p}ubuSVq^w=!2n{Ua)>HA}5Mr|WC<_SoCDx%QM) zq|56<0+}9+1?l}**g4Yb<=V6m$yC>-UmAC9HmycwZeFdrHVf?8)zbN2n^t@KPm|k! zq=o(<3OfbqR4ho%(qljx5DU`cSdgxc1!+$#NFT?76z7e_|A_@@a4bm6>Oz7J;bO0S zhp@|AYlZlix3&uLLoDK@XT*S191GHfSdgk>LAo^-q!(jB`Y9Hq%*+^&N@78p91GHU zu^`}{({mri6Jf#e}}X7mdJ+y#V0>fT*&tIrgaRqH)FP^(@G-aVcNx$gj|zbs594wZ0kJb1vmj9S(c;yVR{&^R-u8 zHdCA~byL^;<=Q7UXkVyk!N$!acg5^?HA~EPXnLU(qj!^lP=5x5Mzcq*oZ9>|zAn<) zm3t)YP!lU;y+Dbncdd6czU*6TUnH>o>5Z_MyokK*6(|_Dep!yg85c225;mSN*X(~uVp0Kd>|`!q+($qK_KTe6ZAjb1ZSU4g0GoHsL7G*6u+KY>Wij2 zwC8h-I$rt&OTkb}pNHSbE!rOmJhfB$E2q{@Db!Hzl-5VcSK81jU!f}B%!aYax3;0Z zeA^q=D&ON#<%5rhyeG-0Xb$H;8`e>afI_g5OrcPv(7jO|6hHxIfu5pf9kRH-QLS05 zX(VTHt?iQF1JtiTnZT|(7cI=qU~m1QZ{WM?T$90$`{!#9aVw8~(lxHIY~J_tUq=m8 zS=J|A<5EwV&R`+q8JH@K8-nAo}8?y{ett77*1+T5s$ znFF5z%C)*gaIMg*6wa>|3dWfV>mnb&f^FCT6NY#o#fp-@jA7zEajkVQ7|@J#5Lua+Eja{(u_4)L_+yYM6DZ|QN3t5&T=UqVA4fhKiLco$5}3A171=E3z=j&9WdRctPSA$ z!AzI@F}hhU<70F|psfDQbEWKOfqIJMBR0uu?x=MGOf$(~W` zhxKr1ujI)lZoL*ckMd-(rX9|swjXI#lkzPlSxtNt&Ohc`hi`YW@GkidZdL8Q0ws^< zCvj4Q z1(=h#Ott_xFE%fcFNg1@V=MC$`Iij$5WFBU(KyzEZF5PYav&SSmL@8PqI`?B4T(y( zA@U6OC5&q$4+$gP41v0j*^2Y6s33^tT+>sm-0p0{r(7Z4)-C|n>vdwmaS&%)dn}r_ zL;g%(W*X=gTDG;wYB_2%rft-=QVz^i?DOlww&|zxsG1CIhy7GuVt?LGt*HF*y|vVVwUNTx(z}1LR*-01HddTyUN%-rJ%N9b)+5opsHkTmem&AZd!{xl zQMx#wccM#lfc%};@x!`u=p|0{avv9B9Y3shkEr8^_3@S?FNgKHmTwn-ttSGF`0CxK z)?>goI!Y8@U2|$ZtFPbbN_`EUk`C|DTYO^N_yXoddZe4I^?(F;n@j`-H=SDVZH;fc zPOWzrO9+YwV)?dIl`%b`<;u*>KjY4&%)|y?{ND(oG>Indjq%Nh$R>eGgsusa@_PZ4 z+ty?JjF08pCrgPZmYN!KiHNzeN{zWy#5@xb^JIH#*No@i+Sk&7MYiDd!aCiLZj4t! zVbl>AP#Ie{6+pX)&>STZVvUcM;4w-N?F+i)j_xKteCz%7!YIoyMXW=@D+KVG6z52I z(%$a3_532VEpO|E+_t=}58;Q=-`1y_-++BvUyVHlCOfDPpkxPi;||J$I$d;y@w@d3 zir=jl^Z4ER2u^mlKHI!_Ws8o(u7iZWMeikU$GlcIjMFyVG)~v)=1qv(rS}?Lx@&O` z5$)UoiO$h+%Nrv!Pa-s5O1waNU#y!qx-65zm+O+zTcvyH^Sa3HSOT)EB6i0TjH@G$ zB}i9C9!oH;jy#@VN^v5A`d_qzZzLF{Hl3HNqvqUVw3TlpM1CIkjfBY0&Vv zO@ONOM~6DAD0~y4MaNoPRo8kIc}O&56>W}$rp8=TE4L~Yd2ejozK9Jit++PUm#WB} zq9LnjlcS==^4$q+H%}#eRE(AAsRZLg%ugkZ6}9D=1oIY%ebW1L();rX)NGc|&<@Zm zk-k0wJ_X81U!TyGlfFJ-m>^w4B>a_&mTEf^Xf^>W_k{@_h?q}_FH9)ri7!kTh&SQR zS`F{x&9f&LCQL^GX3kr#T_g#1HNBHG>k_UZg6l+USx2pfkzSWTLtDgLmGCrSt`eB5 z1m-Fkb8Z4$am+C1CUoMMa}!3IEpTqabh!J-0M2cdhEC4n1iHt7?%17}KzHm8CsGp= z+Hz7869$;1CMJ}d8J?K10!SIa763@1rU};&nWjVr_0T*4za53|EfP!vlb3*}cKxfg z!3o*sb-tNoTplMvnF;N=JediD5or~&{WV^9d`Bq1#v8f+HD1d7ukkCc@*0AD9^beG zkw1?wehCf>Ab`{HrjLDur6 z_;i{nKpJ!6ee(>vYF zQV^dcPoA{qzmA%w_*;-z1x&~KfFLcepGg8w36|4#4zdD=0j)Ti!uqI}DU)qY~d z%74ET`yr;e{m!$5=3aD~7vj9+lr9Z=)tN!q%b-!zFgrU_&?2zX%>&Nq}=SNp(~dA_vH-WUY@ywsSB0RcOx z0{&5I-6&!R2y@oKwiZ2I?2KCs#Ze-w^9(z6!X(o8?#3jN`MISI;uN-ud!fKpf@`e7 z`2C~qQO6?q{Ub}Xa@83%yD3h{A8Qp;sM-1FYqOoWLX39}&Y@=4%)fBHw%CbVs<44A zaq@*eBr?J|jEIbIb`oZ1l#^zW$pSi+>x_RA?V_>_+eLx1IKrtdaE`D{02DYUSthUx zoz_*(BB%9!pHdsV9!_;~pt3Bo<#lleKJ3Yg`T{9Nzd{1s+hyPm8F)wna%IxypYcuX zWM|qlj7<|@6PA+EGGK?6R$NxwjMZ@Iuy`#Oc$DMh z*)(r;`Ry)a^>@Ps?RsT%GtIOBMFjJMtS%)xdzedV$@^08v2I?Z93$7y!+X--$rfm@YWh5IWm@_OdK;znC4++T5%<%-0^=2#Zf{*F`TNrf_inK@G| zeJcZ-KKcQB_r~>)5OiNm=X@05Xu zWJ>!a!28k=2Y$)U`yRerQ04ssUp7^Ff5Ml|Ro*}GWm}b39||uwRk<^UN+cR$=&mYn zD}32qTn)HO1e`Hsh^h<$uKF-UT4P2jzZCD^|Cc)PBdAM3rI*cl&j zAIJRecv_^k=;MEUGvAn+AmyMGPCQhul1Mn@W9tQ(w!+SmK@hLb$B8U+I6KJJYo2U7dICBP4vyMN_15!L=CAB-J6UE4>o*mqTe*(J_8W<3Xl0xs!_HtOd(ylsSuT> zLbNOy(FPz|S12tQlno4p)9z%P@Tkd;3cV%J^Oo-V`#fO2TN?bxa$1RdPntk}=AJ0p_L@ z7(JYf5zl`zA{@E?E_LzidbK}4Ibf}v=S}%iy8~LAY$!4>CNH|e!<|NaTx+7IHPd+B`lAEzzUA?p)F=wA9vCv-2pL0l}EtvmYH z{<@>k>#ui|?g6@^mm8=PTLC?jxcfM^0=lDnjOpEpdw_K3S~Q*NwIL+d4LVn=cR?x6 zI($tx`6{H#O4}!{TNgC-?lenl(K9ToXiMp>bKTu*R?a)>m)c5iz1+!pNBugU^Lnil ziMN|hl$K{1=Y!1-_?%_7X8bH8n(?!&T9*Ge%u;^VYv&m`aCRS4)@zMYlX3fGzFynM z#=6#P``C2PdTk$@?=fX~k#QppzQN5s_A_CjBP@7cV#2iH@m9jlW?sU1`-<}iwEuTq?IOz3btrE+sp6ZsV@#!qS1+05jDahB02$h^AhK|yl=3?Cs z41#WSqh~Qx$U^JOfXfqf{qr;dB6}*BH8MnA<*x0!nsW{lf_(A45ZIXSR`>|-9U(C% zN}Xvd)jP9-?W;$ZG1~FA)86pSNLR;SQ1;(0+On?jtqI#^v6w|Z#WF1QtBE5c{c1wt z=sbku=qcZhWAxu!9X?j5G8XDo4qV!9b1+7P0$m9Z$=)WWfXFcJf>Jf}34t=g9GVg& z(l`Umb~?skO?*8iX-zyUGAH#hYP46D2M%mPz3G zniu#udNFghOuLE!7R0NbOkD58$L)63@QPgz7`0>el_u@1LDU+m+lkSWs{&6Bqyode zQvPZCL#7orXeUePa9p@lJ6TUh4Vb5D!nUa$9CzT#i>k8C%JezL~Ag@ z)MY7%r?=uQ7~E1&(0%O%;BfYbJRgD0Ne+tiU#c<{|mKfsrEXzEGPmo z;t+3*6Oi-`%YKf#-{vRJ5~uBrzsv}ysB7N*_9~_nmxwQGJ#RI?y~^)0{89zBcLEKv ze4)l(P=01nuWsR)*}^k}(*tB?3*Y_c8QwC!_+p-~CuP32I{A8vT+yvV4Wk4VLiypl zw$FSA-}f&H^7>@RE}_`$Blu`Xvw=D+nr|(pk^RNV-BI>l*)N{cx(N>srJ4 z>8~G;Aa*giUF_GgPvc=JMjv`v*&qUBjshTaJiM}KaHr$i8xZ}{(qB)=DT18zmV(H5 z{w>a#OpvR@_c35$mJmd;)5;6PU!p`?cuLbgzt$RH8#&Qp_sAOSH`R&YO<5AC05XOU zlrfqD6Dd$mfmsB|Yyu$jc6i0_hF38jFcYQmS-Eisd%5>}z>l9!bz{w>Htv`I{|QV2f$}cS z1r}CoFcy$u)-@YJ7b~TYhWXN!s&;V|e^^PC8+&FtnQ0L#WBTecPpiNih-5;5GVG`5ZOP7 zqMm*Hj52fKtn>Mdz$Lf($2;m>)2oO~Pv^+Z5Mlo+YhoJ4)ZewOBEoZQ?59x{5w6pX zR|v{HOaNmq1$KPTXWfElr);S5%!sEUJ-e$&{RW*Fay%8Hv}%8%wv@<^K#Sqp+?(wS zXF|5~3Cbv>Kp6$931HR}AaA(<-h||tD@uGy=K+xF_E%G}a)~Wg${~Ku2pR&f&Hl2Q zHhpUPv}**(Cs$_1(3IAGu8|qU{bD17pp5Pm$fm$>3XCN{KILsaQ^Vn4`V=Enlth1@ zDQ3MYgX?r^4S8E4waL0T?nCDJK@*CD?!<4C^8@O|gSX$!?@*u{1+pnHi~{)t$m({f z130c9if?(Fwv%d@)-wmFj66cxGarCR;ibaFq5M*7doO1up&2T^y@7){iy(scffLWX z;+j-z_*VDZoq-A3+2R`S!VI+(Ej5nSjCzwa1bWmb(6!+IT#aXlCG~dJtS%3IBTxTxlh|LCnbO zA_kYGkOVVI2c+Hv2rxs*5Z~W3N(?snYv%}|1~JZMizBq4e4-%m$2lpej@NM2 zF|TkQ^}1c|gXdzutdM0-PTNEK5Mwhz89NAIzDR&5Jpe%@GhcsBea8CQD+MyWK984 z++F8Q|1IFZxsC`gFwV-q+vWrEV4dA@&?KMVz^Rxyovc%q1X&Ii?{zljNvi`TDc$FJ zlX5@!kDf>6C~B*q*jM2iyLz6t)a?TSL}IF-LE!~<_Onw;7r8e;t^bCgj4c#+h64Ku z5cx}pACx#F=Y8U9$De&YjJ|&bMEoSYuKC)B#5_wp&9-f*w3gMmjo)OjM|}I3q2Be7 z>>h;57)XKP1c;1ngLBgAUCoLz@4oBB5@$6XR!>8@Jx4GB-5tq}r$*LceoAD+QzN@J zHL@S2M)p{8GE@)lo$psvs2}`5Wk5wFlzP+a-CAk;B6vFNnPP{CvnLuAka7w+gEH$- z1$i4O?amlYqbC~Q(;<-b!PH3)q(+KHQ~&gm>P^fqsaZE9V(OY=@x&(kX%$qgHNF^{ z{uzQYLj6c8p=fU@q#3V1_8FSsDMvv z@T?`!N_jv7T^l4clvR?_(2!tGq*DqSJKw>HCx=nGBUyTWfTnF}P!v{I@XmIg>~=i} zM6WbB6P$AH<#IrN*1%+wE^$XO9SGVkl(i!LuLS?^IsU~4 zTFvBIMfvYpK55$(q~#3}-E<)^$Q%lQc{%C0l|=$DuOhx?ydRxO zWkT@Rmj@Q9xgjFI_iUUO76o*c)U30uj-U>Hy%u}oSJfcx&#NU@xHzctTqEVu9Zr@@ zrLg>mB`|cYB`D)=0z_#btITfe7`{7kn8FX;VIJ$K*Ed3#DFkE8Ab>fS0*eTcLMKQ0 z?utgQp!{kMGyun6R>{^Jpa1{jG00k@xBGnv#J}|MN9Ip8db=gcv5I_sxX~+L;|lOx zBOOkx@#2zzpI6S&13ABfnDdsnBwvZW2=Y<7gh<)c6l>p37!KV*Wh`T3bV(9(3Q2m_ z#RBbAVowtdU>AA8C16w0~6kwQ6FPSAbDck5CZSSkoYwL6P8c=Utsw$Y!s5n^6RFve{JFxOF_l>m8< z2w)X3FdrqD$|nV2Zl!^D5+M4rCn-BZS$|vxNjXB3a-0BzykYRUNPK@DO6!T_3Io|D ziiXpb`PRK2{l%-K&;6?XcKRCOZ^nE0+!oN9eK-0*Oefpt{^I@O!%y0 zL8Mn$b8?R#9)zVYO%gm!<_6!-^5E$@is zLge4v% zMU$*$ed4!kpe?a|!l8#(KtjHx9sV2vBJJ$Xn;|-{GBeh!NIEkrd!T4yx^{(kzK}_O zq}l$DMxBCR4va{Wp~cg`R(P5+o|Ijp48n7Vx&X+9b&v#Q>>xls4P-qBE({0LDIh3tu1L>oa_v!B-eYy37c?vM4L(IVZmFgu;)w zxta$k@D>3g7z>_<8^M_LmVEOJB@NRtJwkx2dp!Ge_zV}{uS0DPuYyEur2XV~1Q_HW z-v0f)V>m51tWq;ORyi9j-j%thVh9c@j-3aiWD32*k(BEGB@amF_UWwaeafCvS;5+IAR zbDj&t_mGi(U^Pn+J~+-n{fGDsz&7W%jgT>nGWtFN@np+)<9ES zQ?68yZi+2O`pDNNxXL`5UAc}omrP|C6wM#9HimEko`Xux+y-C{JmzB~KAdou?km_KA+H>`;!8%4L}6J#t8+nF4J0E=JWR3LOw zmpmV5EQFGdL-&;3P?12q!sD3;n#uCe+!jYzFWNBv7%+mFa-htb9b^?j*D^2$xDwRL zGt4U+h2j+s%$&1(k4&u%h0LoA7oKu_d5r_dK}CXG;a}+ieeQR3k%ubUV+|X!^S^ow zxR$rXbNS{l^1Mk~oww2Ekga6T9ssVY)kIidXppOIy+hyhw_1keCY3Yn}@E?BaSn#2Pxmoz2$68&2>%L(TM{Z`=I|H{~l z;tleWZev}~>XU?HmSQRWt7;%XkZ7xQVRg;c6VV?H@kltTp)hzRv})Q8qv%=_c&5;e zNB4tW=|ji^rIs=(jFhM=yjB~E@Z?Qn#86B$N(7AP<7~KzY$XzlJ*pR^MVskZy=n59 zo+^XSC{dI1dKro{Y@LP*@Lqn6Zvn>F7B)*5NpL!-g?*0t&sop`sLumrW;|n=vR>4V zq5U3O#uU?9rD6#KTWUllRjaWe)Q%L2N*a~!>|82dtoQ2GFWiX3SD zB51eeLECuqhxC;8!+xatl0$C*W}UjONJz68gG)f&!@9sJ?XW)NNJ{itRnnn$SnehPncaW$kp5`j*T7OSd%`RCq|bvD$aw1zR(dTLzC|6)1aZx$T*jQy#_*%PT*z`EIKG6A~5W;`56y zg*c%0e48;7qH3*Wp~PD@Xm$kmrK4qDdqp43>ty?5O69S;o9Vht;PeNyZv^Hxjb`jYGCfbc|z|htQDk zb`e67Nr2Oz2|A3%tUNv%qDGLGDkv&+#nOhPUyTq+m5P~fqF3C>A=y9JQtXgTQ{Kp0 zbJlRdt5_{5cL}uhDcut$@12)OW_U&tPk#z8$tbc+1kRxa8Ya>l(!twhpze$DP*U!j zy#};;l1Z1VeK0xL_jiaGiJ;*mK^g=5+ynbM$u-xA8$h@<1Cca=$*fTz!+-OR)&n6Y zY;CW1hC`a0 zb)Itt{!8Qo6V92(Q$rtsEZPD>62C=Y4#F+*a0Mt^g<*Gc>P;Z5aOC2 z;)*+{en6qx%!o~tkg2jXU;;!6HG)9_iY3I^!ie1mwQr|wuK1DhDIr^lAX_P?t2~o$ zxhLrCfhRb$eJ!1u=Se*~5 zsVJzX!uD_NrPsgU14fm=SVATnP_+<+i4E7^{;>|Y)8~RpPg;V0)uFdbb@54e(cD5( zr37*Fh2~{K7(PE7(JX9^?SeYj-M6mxIX14_4wBg4XhpnFO|GVm2{KBo^7Qraf{45X@&R{R6J#TwlN52G zo{L+DQD~8ZeC6@g;H%8acg<=;hwX}x4RPuqp~(0_C-L`{S-T=*B7}AI2EYHdDo+9{ zOEd%C0MIHhkv}KeAaFbem-hq5-~M90Q-;Q$&Hr-KU|^l=M1GgBG3sUXJ=8L|2Qov* zjNM|&r6XQ)hyTb~J0W-MNR>+}FLe@ii43NC0%_j%613??!{w>ixFE{k&=^zB=!~9H z9!t6cjgUqMHjpxId~Aq3e($?{|E2H;XiAuSPEL(84qZ?EHwN@}2f~CX=OR)Lm?-%Y z{G4&+uv$2nFeF+HzSn5~>;b=Ihkcsnr#3(kkqgah6ExmCR8uSl8i(CUR^grfj2PT6 zHjL;j-*7yc7Q>6Lg6!2Fr|j0h&Pg+Qz@pO20ED>t0ZEe4F`RJex-6foH1Vt!r!c$QkQUmSwPCA%2nHssZHzBQ z+T$hES`YGtgQfoZd;Mad#fNB5{67OV8__k|WEqud@>}x|uKSCNv2jQO9U=cMzvygMs<(_Kb5bL7QCh_F$aw`*3s-@FkL}%p+ z9QX)l*c4>}7A|RAa_{1d#5jC2_kah|B0(|QH|mriT3ZmANgS|)?L5~ZtHH-W6D|9% zf&)=-t~o(D3Bte5prJg)II7s{*3w&f-CGnLzz?D?E|z3y3tO36R;SYniYJzK3NFYim!$yA8bC-5)ScUFc&GAp)J;fP z6!*hgy2<&4$_*IfNIcTaO5V)2>8aEreL;kcU3Dj$-TMpdq&H0=PX5gD8Xcqzm(u@U z>euTn)KmDU-ZfC91mwier@@OStEMqNiYNO)eR{f}T-u<$je_hTeZnFw{&dZgz+vqX zCNm|UqcQpLI3wa=4R9<|~w#9pf4jcno2Ao8T#4mpPc|?oY{1H;07x4LyZy6tsZfoxGCi(q75=P zRynUH*&Q|#QXnRnXtE_8XV@D8Kx0EP;sBwK1fQQBz&qv5V)IJR3^Q^&Huq{cvYh=A zYS@x#=#cV-0iIvMhAlMb4?nmI>P7-1Ofv?~%R&{oI-!WPZhaPF`*ndA+K+pwWKuRo zk(bhidYP+lfHi4cNTpJ7p1Y8AA9aK^V9l_f2$(`tPwkO@pV>3kk}4nYJ8CQG_4 za*|UILQ}V#$kngaPL9dIn=D`=ZU{K$d!R_1$mU?~by+~DylJroCbj9qWA+p8GeSx4 zU=OpkfEgIRV{|rfWm_7tG{j@VTYdtWTAHJ*6%_2zRClou@nn~d7%;S=Z{5@&&AQcS zOxXJV2aw(9Gza|!(|nfU5zLy+4TSinZ>`N@x4jwy6sV4DpSClQg$d|(lLZ3><7#B& zP>i7Nip}e|A4X3O{@^6W5@+*O0fK^(ITy1Fmv*VM2fp`C{>7MH6Ha=YmN2BtsSk~b zzXry8?@UvIXJPc`Ew@J0IuN#P3K6#Ss#f;j=F5tBIMea~i^fQy)@oA;M$Unre$pQM zRn6?S#!KF7u<6FAG(Th}1h@r@G@M)3K_RUeNKbGgt(bHfEjMX-1cmem^WO)#l*Chs zt1cy(Z?xU>Z+ahYa|eFZHX0h)dADSh>RBlY|DgEDf|JYPw#W?yw#dq)iet5&J-0k0{4l>!%T^g+ogyiY3H?WS*_XJfn z&QweYi}FUKIV$j<-u1D_BPC$`EXK*f#w98!GrjWB_dcU9|teg zWF^paYTp}rpB8$bCkkgRh=vMq7EPctO#ofMQUDL;s)k*zKO6B0=*x;-v!D%-@3H{} zY)KCpr)ovfs8Mw(XV<#R(_UU*NU zskY->Msu!bs+v4Gk@dHT;HwA^QEx_$JyI1<+Fn+FS$Y6%yX!BQ@?x8IdX3b^FNFIq z_PpJee|?$tS++x-9mod66%fzAcX^>N7Xr~!K$-bYM9hy6o3it`zE~>SXTx(&X5su| z{^{oCnz-(r$cE@8^!itmM{(mDOgTt6+y^i1Zg$1rvh9?afvlv4FX6(crqdFM^N&rc zh98%(DVc#@MTfVqkv2BIkmz^A-gVAE>uG%D%!DsZj#qq%pIvu$ZmQ}aX9YS77Esfc z8o{t-n<4X_Qyo@r6RI$l3sA9pFZ6rOt8Ne~aPtIUwQZOhZ^nAc#(rsEqw;aMFG%rY zk>9-o-Q2hv>A>gE^~;~`u5rIcC^BlqnbrKjH`1)i?<>$pLy3DxA4g{p9zmK`- z7}mN3sv0D5&atgV^XrYLEobt#JnGGc@u!&fY#|DmK+_}EoFbZc?CzWG<&x`s1x)ET zYT{k!pT6M(Rv_b13q7JIlULUsJ?fU-EG0d;6VA5eU|HG!JNzXSW zFcl6ZG`U&=u{TcLvjbsVdUaJQLU1GOVT2a}3^g8FVo?s>ZuTA>$v?%7->Iya^Pvs0 zXHC>miEPzTO$MKla9hGw>-dI3uZY=g2+7H*jfJUxzha6CeP|QW-TX!2A$=ZR7}bIVPWM@D1$dde;7 z)~r)G$)RG)`z+~J?>!$O&7o{CBPOY?y|SbIu5Zu7<|?yrXg)mT?y;WHkG!PzuRxaI9U;mx+5AYL6(MV0 zY$5!#OGzwCw*Mtuo`4(wTFCxJ2Zn3mKjDp3$HK^YPVaL;EO__GbAD_@dhOF7BDt-q zQf6qhEDsE+*)!Fw#Ymq0I#$nu*d1TZ(!ZNf?*)Usr)9nxlK9e=eJjzudz8Cb>b_G| zcQ(+IXluHAvq(`%f7NsO>e%!5ko(HmdcLy1zn-@^+MXF79fZJc$= zJ>DJQ8+Yu7CGV3NOl#fn=zSz?z^Ep#-fM*1{yU}+KYLSp-Q$GXj{RXH2%>}T&BuP) zvKqOidb6mmb|qR}yjSG~B5>ch->_g&tA}#y!?xl@u3HA`WYHArsUp%>i({Y}O+`PP zfVnpXe^Wj9u!pJv$-U(4lOzmJ1uWPN@S z?N}m&s79FanoHGLoWs73+426Y8N@&@aEc=x@d5aJyL93eltc3)e^%G(RZ}(4qTp!} z*y=bJ`}P$;qeT99V1(mFJ9!{@NlCx^2=*e@6tC}?S}5Mq@4X4zUvMiwNK6&==}iV^ zKf)mQC$DL2i2}(dm%H7wSn9Ce1{p-S6LaVu?8T_57L%85HUoAk)eSXC8+tX5rsHy{g9H^mNT8ZgzFNm zS?ZBM8lQ~l20w@fYBoF`a4oAbM)8u2G?v!U-wFEUT@PwG!z`EWIR9->7m3I!MmK_r zKe>jF2IGOIveT8k1~oUQQ&k9V&ExFG1FptfXypYvXE&A}5|-#ZpiYWQS=yB-Ov!qj z58Uwhq_}TTD1fsUhTad^2H4y>H6mq3PHX5;$!j?hRdJDy6NGcbrEMTEEYnSFB`odMnX>zpGe#-Qfc`41x1AbnX7*X?2Lz=%R&mBb-f0L!3=jb!BD5aLjY;O_wL- zc2HyvmnBTSxU7=kfia`M#XcuZwZ5t{7P#j6 zn>lB1-K{8*lgLu|wDE@&x;;MW5S!_;Ww?z-s)9klQ(;m5?k^UZfbzOJP}7YH@NBs@ zc%P->c?FPX<1#>NNXM7dyaJ&ukrif&L;(aO2mlYmVjDS~5C39Pebj7-16_dYhQzM9 zM2=fN^9`F*n=4{w0DWYr$8y@TV!z|UL(kP}-`+hCHfRC#TNG51_e+47tq3Okbv-RE zO-p+wAztA*?26xB9X-G?;HjUkgE+EuX@K`R-b1ky5eC+(QSt52f>ZLF1Um>Z^J{jp zZigcWsU9Sr1xt(Tm6R1DSH_De9eq^B4EQk$fgkA6SM;xlU@w@#q}B@%nRTD5j(<%f zKnIr%2g3mY4L%7Co-C3Y%RXZ&VsK!5zM#V;DkRTMQ)BJQv|MO!T!vPK?8f3*lSLI` z-mW;}Su6d)K~XoyhZ9>~D!)mm-A3iFL~K^VVQT{Mt9PR=Cytw89Hv#xn?nimzMlg^ z;ZxjHY4Bj;Z*|LCJ#ab8Z9}QXeeuh}R^t|c;H{OGCJvbTf}R+@F7%}ykwCrZDQ;9q zPkq(5ErTBDy21gzju1t$Z(<4iVqrItC-U=+fL636e1F<<;zpc_Yh`@P2u2Ix-$tuj z7prfVmUv^dvw~w#StmPnxWn}_5hmQ4b;fV0)k=@9JL(TmiG%5;L|X?|CHo8_$Vh!; zlRVCC2tf#(EmAj@FO7a1vyfqvJz{@^bp#q!dE?R@p`QLPpfdem`}C%;acST-90Uze zG@R)OIHe$>=XMxck?ChZ^dLecv!La>LbhNsL6Fi#qzQYxbg=L1yLk!r`p>0>YweSb zFjLcw%G=V4GbUOV^v>(hvUf5Mvm?#o*2;$j^Kf{Cpb^)5hOQR-iwKdgO`I~G=T*lD z#n|Z?7X08cNaV2M?bTt!-dx;ucs4H8@qj=v-3)fu&E*(IizA&I=%(OrtS4hMFo}Kh zy%aaoYSLz+*OukX=HHCmN%V2WuqJ09njL~E|6SG@6!S=AYLS|Aq)VdD4k=dAI(JBa6MM4cv2qw7d<@~cnG6)f#X)!1Z+`Y?6urLAeo zQx79qy7{wKCveTeTl+?Py~3I05PGdrULgQQWToW7h!ezcfH8z>i!N5PM+;SpM`XVM zZ7NDGm@rsR4S&x!+QkyD?r8ibcS%@pr7515Sy!GbnllK0f4)j+tjd3ONW1de+XxOd z+lwhTBN#h?&Y0La)rV_*%OVYnkc$IFi9Z%Qir&T8Slajr&$=DKrp^O1A7TN6hz!g4;^;Pw(+b)U6!%d@+>so!JF%r~UxktZg=mO&G zK+^)9!%A1yQG=b%m`r4PC^w%j6*8hpq`IG+91PVgBtNt(bo_|&E_W#G+BB_vEg&oH zT<<*8!2Hl9>mIYP6`*F}gyF#awX_w0;#J`dtOZ&&E2wEfq;jTVL5CvDb~GmJ*${UH zA*+pr+NfZ7x2BlS%os5<)Xth%d$K;dKa7AMPLXLoj)suv;^RX2SG$H=zy{W&JFXr( z*5x)FIEclUf5@7Cgn7m!!^etlc^3~^fYAKir$tx*Te}~l!2e#@B2q?U%Kr$15DYak zAY4C4MvFud3<=<*5}G2728=I;uJS7S$$U9yiV}jTSS2WC9wNH8px7s6UXO=!0{^TY zCbO$-27TqVW}_2j-stFZ$P^1s^1*cS1%551)s>*C;x*{6-%-$4FnTz3t=$~iI2r|A9| z)YU;)HRs8NC~h|(=bB6zk&ks`m1`bc-*xP#xBW*Se1O^xiWh*0lfu%gDF*I ziV!jA({OS;+0EB%PwZOMrVVinyY=zY;x!r>BLN5);upMpm;z@;9oZUUl!DTo_a!@P zN6zeuj&(uDvRB`=S7ZEE;~Gi!R+L8ipn(-)mM!5BI5Sl~D0kDmdo5pG{g-u-rj$i4 zLbiR=EUvU@fkyNt;K(#x6M|Fr7-Na)?4F;vzOmL$PulW|z0AW0)0BTL)77^3 z+3r5S@w)Ge;kvI#w53Rx(fX{gv6dlm2|B;MjQ6*vjJF>x3q_2nv1KJ+iY#-nywRbn z{I>V-;~8xe8v|Q$s&JML*{YH9yHWT;Jgl~uZY#hvR3f@q4r``ts!tSD*O6z&t zX-(srm&0Bfr-B|jp28hckHkyRS3em@3mClm)=kJD-~pc5i4~zB4za=XomOucW7o|r zutjTujrFSP`0qZ2!JKSILaR*f`ot>l=J8}A6$m#>uP**M$Q2OD zKpl!DBE{Eo6V{v34@g-K;vt zv+bLA+(A&v61asP))BPJEtAR)leg7{a_n#4`N;84IP~1xr6J)Y47QtC6l@r^QQ1$cSoM7 zcWXOjwv&HuIgEG@G=X63z2;VRNB94xqnhNHBN!pCNjPueWTP&uo@}pH;XjEGlrd`X z*q;i@le9a}!v^$qMG`-lD??6|-IvGUOrIOThYb|L?K`TZZpTa#?RdC&-K@g)cs;oZ zs-lV-OxY2|c09Fg9Yw2IBR={-PcPavxXo>tdkysiX%SY`?4a}Az>5<3^i9VG+zzP_ zSDv|bv!6Nuj5<9p>{>{c!T*{5@is#~4G1L+6zLT;0Bm|H;R>UZ6AFGdl{A5X}RTRJ*DoEqmAfz>E3^ zocgc$hl=!rc@f6xSw}uA*$}(C*Q=ni`SoWj;rVd_vWpizQIZy2|oP@Etlw z&;+!(Bo0gYrq(Oz^;75oo9j&F&t&>XUc8i|q#`=fiCr0&4;?Ky9!cpS3=gf(<~HXJ zUbF++_ODID>cbEl36oM?eoGb{d9(10WZ8C+9dC;gbMu`$p>12oc3_Zey3(B=SV+Fo zc2J4i(%1saSluz>Z0p5j5ESoJMuS)!{T6{W5Q(z1aLlC0_vLQl%}`Gw>KsyI9q>ba zi=~444$r`iCOEbrnntcHSdrLhm2I2L?hD>LHa1VN(R^h@m5i|l;bAnw>rk|h*^KS~ z9zC!q;v;~jn_!7EJGv6>N>?Jd!-sGtQ9oGg-yY39x=I>{?L>%W7A*@2gO1Coe*+nL zz`WG<<@KK^*}KgiKG_`%bbi(G)tEWTHF_`oq%P4vdSEMsv02&#_i39z%syZMFlkX= zz!>PN&pmxn)AYw{2B|^_t3uR`U!~P$`~O)Zyo)#`Vpq(xR~4~Ee-7?=e-mnc_OF}? zwV$g-1tZ-(CjTNcBQzX7mB3r}%BA{;jj@@Lg*4PMO|q7sG&7`-rvWrS&?0{^@YUG> z%4U90XJ|on%pW!ovZuU%rn{oc?_F42INbJbEOujW9Tu!Iyxl^@WUdxF{L?LXYzDq+M@-Atkb1a)?2ufyuUg2)k&gZUrgt_=&NAlO1W2OW(Sw;5m&OCe! zQ`X8vHBq$`osny_qE?l<(WTeS?%DS2H*&bykWrDIFbqC0M-zZ)+mTncq8o}5Nq$eI z#meR~NWa$|LZGSPn*1&`-WS1q_0h;$wh!>wL`Ai3Bc@Q&Hl^%~`F z)~58D8CNROTa>zIrAkr%j&wmUvkD#+E|XTa2V7D$Oj&t1VdbL|y)|xAP7d!yWDAc_ z$JUTyll|-y!KzI%1zB{`g~SuqbRkjkC;(dX!1?(dXl`Ud_D#7x+gXDzHnqocmkIR? zI#Q{P<&)Hu!)^RwOh580wj}IUop{2|Lty)rR{LwPMX$MI~&QH{^#cCo6WCo=tHrP%(QvSb*8LE< zLe=>azw4Wlx6p_30`nDLfPU>5F9VMo#CSOa;71RYt+=-TF?vb3;BGkSxWH@ze4+4TqnC^!=W=s?Akj?d z%+k^5)aIj(b%i@ZpDUB5broz2S{$3a=wx|0Z-s5Do{_K#vaYmMU5|;KrS(bGbvm1#= zTD(OMUx#+Av=O!Kb{vNST@+24=A7w;_o5)I-auSV7!$wJJuBdKe(3AoNXMqt!Up8F z26CGC2hMdS2q{d<8Qh|G{^{rZMhV+hRd1irFtC4Zpn9%rL*}D|2=c7Y1vG@I@|(J+ zTLG(x*p$V1`#&}ii+et1X|niD`*Dcz+rNjuFE+|yK4FCEUF*ccj(N2x=9c2FS=qTa zx1oz~S69kv6}sXbIG4rxPN?7#u7kt`yz$<191S|0?#3r0`b*v7u}++yN;WV}SJh`+ zl5!OQsL|{y;rux2lBg(A*7R)X*^7m{r{k^w$FiWaEs)X*vps;8U!%R?>^TJ&g>7F= zUC5zw+q#6sQNLW~t<*gD;zJBs(q8^4FTY+_C1g~_VnDF{;r4Yqwf4IUrQFW%h>>o5 zuig_imK*RE<3XlMICRWc)ZfNq|$5pXo!OLpY4*CorQs`z~FC#?7r(()Dr5v zOgXa4BVQ(NXe;xDpJ;Z_a{c4f3%$Fge;zu-V$J1;#O0FGK*G;B5o!-m!~`V_bvn~* zor*xGJSP+m=uT?$D?V__SOiCm6e)y_<}6mlXLdTJ&W^Z)(>TlaCX`boyy4VxI9LIV za#7g)0tNm~=!-4`Hn7{?o+Csy z_WZ|4T5}W+EAx0W(Yyttc_?u9LEZi{_>y1lsFt7mto0^6#|t{eD{BFq8==1Gs|X5BWxmqJK;2;?+sB7 zj0uc(8{IS`3u}!1DKjM{%Bqc-tgZny<@A~xDl4=qE#~bdo!L*UUWBIbJ%Wwmg+aiy zZAhu9AvFvN@E-%k0?wARB5+Ve60ltNeqjhJJ5jBtPnoScC%R@Up|u_%(3{*#0wdLD zicHI<9LWjg^o7(sn9yx-d8~yTK2P+2YM*=%Yka+5fmJRMZyDOHBFGo{F~>6HCc^uwjLN zbQc;-*etGwvd840i{?c~-NeH3Zg5iYtT89e+~C8yhI14x8SY=80@_N-t=J-CL|pAt z5uZ#CM84Zg_^0y+r?%RJFH>fL@z?FN8|)P2@+;gTb^8`VU@vWT=(kfieZqtF*tBYF zmt=*sn`hX(d2k1;Hg26|HT=r@=X-<2q7&Poa)(43J^+Xo>Vdj2JN?AgkUM#b?mSAF zMk?mY>mupTnGU#J5m$ytYlC(MHsGV4q`Q^t%+XtvDV6~}f_$JK*n1yuK&keq3sT#7 z@MZpH!!oc==2~}45k)|7>6B4-8g*&i5kZ%-MXVB^lX`uDcOcgpOO&|3t>Ps^wYOln z#R*Z3Up;CtA|CDL;Z0uQ|NHn%>99u{-$Rp4F!FbW#meoXy1nf~@q|ushI(UUYR@Tb z-oWEt8PpN`mtU2E+d`(3SZ)s@QftMG+qLE#H{w95OC>xzpW5-|#&%`Rq&j?ji|N^U zEY6YI@XFOfpN85cG{&p;LJ3O)B&oO>B=u7wg$ZgiA0h1a8}GbkY-~Fr`Zk*Jb1Lwi zEReC``Pe!nS6(7#+4&W0;;2Bm#{*GJ1q+%oi)5*M3rM@)b&E1LqE>MZS8*cdOA9BH zseDn2*Y&DIbLqJEEoY3yl=2}SEfm32ofGYL$savYeA)B#L*3d;_XFbAAcw$Z+K6h- zpjQ-jPx^Cp?a~Ef_d?(0_FiId9$S8moGDB{@6S|Z8{Whht7%6~?3uiSj%&n2aLBVNB6Nh3?VrVAlD!o$;O1Sz_Ax-P3uIY0=}zDbXWK+yW`w z$H>mu;^jx$ned{lT^t`Gu0m<>sC2_iG=yKa;LH*}3Sakps3$gyb}v1JaSzXP{a zn}&AVmNmT4DM(|YH1_T-zRGr%c)RE=L~fox&&ikS48;5W{wE}`2wBXBlq7_BM8qtw zrDg?hhzUOcL3E)vaoOPAy0l2GY4gbmI6|4P^qWue^T=1h zbSu8((HZJ7#&j`s50n@D#Ub<|da^{hev@|F>8AC$;fHvLs$BuwtE6KNvJdt?4u^EI zY869oTKNX9i=nr>pa>rY`FDt z=i@4{>>I21lBcl6U-f~~A)+I^=kPBkD*NC`%HOe6c;WNf3ch3m0BIW;k*69REX9(} zmp{NHS<%47?tJevno9x}ii%;Oh9Y+$*w0_DbEye`7M zM&ZsA*(j}T5I8+;funRvr@a}xk8+%*UsgNyUO`{C{F*gKT}y4# zevRo3aQ~4u?u}XEfSkPLnBY~Yk)Z=Au zzo&Da-_9xeW^bxq8zOxZVSckS0{#v&1MbPlTK6d-_2Z?@x0Y?=u=kH}n2j1eJ;UAy z0&S&Oql*2nk)%D0lF4T5)P<>< zTE>Kda|GX9r{3ea-s-OC2LxDLrp!)YYWk^~xqZ(S>KFc2 zPf*`tp`sa2gA^_1S<6?*?FXwe4>pJgbSkw#?I4SGN~0=1$|WSv~Oi4 z6s`QeEnt%Sn4W;18-SEM4@wh1!9@xLsBG*yK~lCoNt`x06en*I?rg0Mq%y9 z)RoF$6PCLnfH-ry5(Cp2bQf(?TGsHN^+i^J>_ThVLX$e%5{4@bb(Qj}olb$9bki}{ zf)-nwQb2eThM;v+N`=k+P`j$zS+ynl2LY<6pfDH|$pkg1UH{Q#N)vTgg|Y);+B_V8 zX0f{6XO(g#zfdzgFjfO-9KJ>c*Qs)FU{C)+e8@p_vDPVPY0$5u^NBhaz8)P%5^({W z&$f?pM)yL?^3YPOP1>h#Nf2No7lQYv?+*&355j`Jr{`6>vvBQ7SZSF7k0*u_Y23+@} zk^H22S5P=)TWn>nAS8{7@j|S)ymc`vY6}a~jXoC`tmhbdeKZ{jR22Kgk9(L8osN7h zMYukn=wDF#>M3(vu!K$I2K5sjPSV7N6R&AH&siSiDFxRzlK$1A11d07)G8?@D{{I` zti8VAN&LN)G$!fP0)Mw+dXH%?Ul@k-ewv`?V4B1hYDab-2Ar{>k%;704O|uQ!)3j1 z87{irkW0wcN)X?lCW?owRtWChT(DmmURwfcU)e+0Ij(A7r&dd5nI{!wJl7L#LE0m}k`ahy4HY=9B90A82h&eo3pLBuQD3J`$F+nj8tH$n|oO3SuL! zvG~O8T~a2%VI}JKKcuT4UIFmBy4%n~hLx&yFmaC7#i!PO$LrL&Fl3Fx#=$Dc(Z>B( z*9NRhMe3_#p@{gg3s9H@AK&ZcHpmi zbbfyh8)Zj>RmUxZ&sKuVgT9y6!iGh=i{v%#RRg(SxF%pxo*1Y)I(dB!v^X|f|nZu+n z2Ql-U+KCQLkdF5;#63wQ&gCNM#(vN7B>c`5Ve$c~499}}R5MenYGr=-Ts_}a*#_3w z*mLE*^3JYZ2wUOipDA1>%8!;FtE z=O#}vbByxh(<(+@q39UMFlkrRHl}+2&hB&Y*N?wEli-3KC?yN{<-FeUGONV@yK2gH zOx~JhHZtjhgHoO<;rz8@UP-)n$_FbKdkkgg-c{=~ju^l}KwtOR?Bm&@3@0tok*+l85I8%=>p zaE?LL)&*U4ub_T|ZM7xUZme!vGlY znpFIUG_ajt@>x+8=fOJ%@pcM{XF$`gE@%Bu}~5cyql$DNsEr+*&)}G)|{{eI$onnvRqD2SbVlZE2SDip=2Unp&*zdlK)ft3y8a%@% z;AMN4!4r;aWeWy_j*(uY%30usUtgH7zd!~ zqAJ|&0p9SY{^B9Erk z@t6YJV~!q$!dGF1*(Ebpr?agPfq7P~tmE2vMZrmwj}X^j{F7TzhHO?0EAj>LD0m+Z zx$H)*^1e27l;pWzKFF;!8?J%ywHDH>ozd0w#KXM!g>MaR#ZIeRD5|e~2C$+!?kHR( znRF#(N^W!7td9q2`4@)#$)**+aGMamvo?zo{z75tp{_Ay+rxh4*`eOJji5oqnR*hI zuv!Wmzzx^Hr3#eXfL>_qgvrW1;Ww-cPWl}T4)bXWAu-IpFEEuiP?O~lFW#8B-^ae*w45B&`KA zR9wY=0R}IY7f25hONMa=lNk@^{1vlrv7%<=g+Hny(ozNNa@0l^PL^B;!xPlr=re$> z`D;#Q>1<|Z-4jzCMYrvLQ(}g-xt;iY}qH6;2($j}`Qpty8Q4k;n$a z)oaR)^L{+9qEvdDOQ4Twma}zXibY_n=c3@aI8oT%BXH6_X5FrBj&7snd5YuuE@KmB z{`v>V*aN>A_;`xiF^30wi2WqK;N*UL3Gj<_<-Ahg30$5K-Yp!~S)Z;% zBbMvu{Xp4~^rran^?x{S1zg+$$pM02mH)6fGT4p0wZWY1IbNIRzPf*cmx>Mf1+|U{OIQn>QU@?a)sZrKl4-Ex{CV!h5b#rO^DbTenKwfr*>Qa zN=847-iC+Cn48@e1!)%lkUdxX_n}6i_xBI!x%?ep&d90YrftT*W6{+BD#Z}BfG zJ(8cl-`Pk^l+NG!3HaW?Q2Q%S^0~*V&=h z{A%cW#o2#=Iv8{$bVh@V4_h@^G2mYX2A))_M8f$nWzCV~i$=fI7pBwKY-GA~{T_0i zr?Z@o>v<*&na?OH&&EnKzrPG3#VIEa8;n3mkWhnHEgQU&4bd6q6l#{N8tf#LB6{;)A=Y6XNy->x|o)g~Y)R}zyFKNt4Y8CMvm zK}#eFHBuXP$i11HEzMq73QEW3SH3-`)tp*VC}Xp#FdR*l)DLX7sYcNaW)s7g02mCP zG$aMR@DDUR$*@T)g}5sQvX4fBgI7iKX{U$(cC<=SqZE%$EKX%4M!;NU1I)mYDiM4T zJ&K9?Pt+%;IwUz`N>p!`lWtJmillc>#b`;i^j;zdM^$M@XwEnw!An8oVo7NrNI>7H zWgLs{a|1X~S{=32Wsm)-Vgf|kSnM);>vzkJ@|Ox7{heJbyQQ+BUtQ999GC28YzLSPEv2^@`1H>jUXa1#D7FSy2}aLYOXtagajv} zpBQk1>V;wwrD7wJt|Uk$tP6);6`8;M$_;dp1)2T2;7Nrc0Z_9wme@9 z`khtQw81yR;}3^&7Z#z4H8PXsKV>%)*sclujnqQI8G?3^={5*#CAEvZz7Ukfy@-P( z#tX`)NPW7%d!$xc1jY!OC+GxOj!=Hxgr{lMNrK+;q7h9N6f5X`L6px2g5ZVhg5DOC zA}B=A1VO>HE0pvx>hA(UeuB(iv`q6er!P~)Oy;F5q6V6_c?tP~p!Hs-@~WT|d&xFY zP#uNZPG@SSAnL|x3cA?K?w+IR<#Y}xdLJdyLYeL)uYR&EpzWY*IZUSe$@Evk&16tb zdvt)--A#QRdQ%`BKsuFlC&mbxD`=b-8uLtg*~{~Io-!Zh<=$FN{{RL-n27)Z00000 z0002Xi2wiq00001000000001ZoXvfCd{ozUu-?0KuC7MAW=1PWgFxVo5#T+Hz)pgJ z5nwyT8hut(U$StVgRyJTmp+X`@!wn+q8+r+|--F!_VjFU8ND!inP z(;%|^;^fPBmiyj}m=SjR*Jl#Vd+*+N*R!8<&wJ{jq9Q0VMc;6WiX6vzt)iTM@TN!} zwCJK){&SrcxAN+l&m1oBuN3_W6e!4+@_*o$&iW^(-&qGVTS2z8P!)H!EBCCWj(_VE zs9dO(LZ$p}V78931+)Iic^90b68~K_z0lDP=w;ws2Ci8Mj@pw3SDR2M94>ML&KUl2 zQ}{<|lK`dHfuZ_I2)uR(nN_69Bpsg4&5=OIL!ULnfV!#iFi8P?_2N zuc{ny!&s#X%7?*u6O?b{zr!hfwJQ|HmK^MLh`X9|984D!zX>07!5kV!8iz5&BMNT= z^N;w>HVES#^g#H#c(;Pz57FRWhDsXTAhgqD2C@UJKl)JD7 znlsSkURmP<$OJecwP}ykyJz-x3vZVKB(2UWWo9c8;VLn*NlQ7u41Lb z0Pn~TL%-5#`Qg8W@3ml5%4DESh7(CbEnX2uf;OEF|I*)LO=w*rJ(TgDs$lx?QAA)| z$XftCh9tQ@cgqXW<~3W0k~hnE_O2~3Wsx$E}rE?`=Cs{a~#|VBP=ACUV)`s z^jn!C(&g$NzN?yXS25$RgxXhFqe9&s_wE?=GBov`ae1Z{Qr#BS6DQD6O^(OeEMqI1 zF&URvq6}AJp;-cDZ-Rxbv~AsXi2H?RQ?ob+eYMc8J7f4Cb-8;4+8Inv`f%akmIs=^ zxf1sm0_Udkd4j_An}pys%)~RacI}~1gf^L`dcty`2Z;kDAQXql5QG*J%dzCE2ckU9 zDyZ5ALAv}3RINyHjS7$JcMcKuV;4%r%&sF`;Xm-6r!7sWLghUvo+d9~j#k&jaw<{h8Y@A`y$EGvXEchv z{X*ZBzd%24V+0K%L}*yC?w&&JW=0<f`)9#`OuYkqsib1&|Ufa!3Nw@BgxN$e$mEtK1 z*&5P=N@SVH>mAqgtwq$ z0@hB#Y8iX2C>iwDLNlq8hy|??D+Z`gQUpI7m?o*p)%Bh@Up6EysH1zQ2%}b*DLsbU zopdj#C)y;mG$S)ZZpXKAR*4!I$#<~qf*Q6|` z2sm5Rttt1i8l2>uLek9lOBp~NoE3Be>iEzNi$q$-M(QVUn)Oges>7afs51|PdpC*K z?q#HL?j%W9ZJpwQs}!jNtzjKA(Gird2J{|cuGmT$LS8`K?Ox%eOpQ+Bf?+9^oAop9 z-PVu`c?0}n6Ah5)b0tbQ>l32CAcZ~!i?qA0 zZm6+hCUGnTw#7mOaT?*Y$RMOJ-ppGPMEowd@+~c8u?O3v$|%&`%G8~7RTGP{)T`AU zDekLSI>!gMp&&()(W>dVJjG?k)*5&pQIsJD(QrRRIsk^m%CelyQ{6mBl&C|Rz_Zz5 z7ScuCxGHanPPBS^h)dF8VhCVlUzkHO;3#q8Li)_V`!9hzk1Zk{VJ%^s?=jpvvf%pDzUS}bh*Y)_;q1k#HK{1lye%hM$NQ~En z=2qejt^wy_!u}61_&D1NAk;xI0Ff2J=`PU87w7v^Djon}K})ke(aH^7409;9vZ()6 zCKQfva_0(Ab8!5u;`Aj8wElHzJ!p+&PDg<{|0H!PW-FiYO;~AgZYxmX|72}k&W&tf zB7SD}SsWt^Tdhoil^L-q;qoJcXr3&@B>g2Bflx=**z)gO()?}Te(}V))ro;fwmkg=i^LRoW~2)d%VrD z3YF;y>9`&?>x7UeWQJzGooz2(KVaK?S*m1s?Oe0Kv@*A--N(iT&}6e%Cz20I!scEp z1+gvjt;{go9tgdOIC=N5XDpc%=`b}&l4GMu6@cF_Eo6GDC!xYLQ{_wJ$cBjN^=fr> z)W2^1|M3cW()y@Ym#1gZx*tAAboX*(t=yJYnuI;}si|9u@dV3J7(-;kzDD z)Q<~7+|P7}XjvHw`06V=;$Rqzo+^uT5i$14ZlNd3`3Xt(!5&=bFYm~Rc@!%X&}|H3 z4-PcabqBJ>gF9H~3+I^CA7k!Vi2`(&RrK%i6#Z*Oc2VCcbF?|+UJ}l#{U|-45K%fJ z#xPYfLXbQa!?CG{QR(;lnBYEEnJ$WaG)8!JA| zr*Miv_dLhb<~Erm83$x| zg$1VZY>dr9;%U?4B6DeR9++H1jqesIxd!DE{4Ma#(B2+3v%}`pp zPg+WARg$3f_#QTMuBaK2Ps6!)Lk;2Ur&*Y9>=rS82X{nHkce0u?2`!ZQ#Z<@k#|F^ z9cmL0OF%722hVS1gQa^w43>XGNFPE*6?}@WHnMtIt%503>H3l|?_2o0n=Jr14OD9J zWAprZc{UW z*MEShdhupiYKWF&%O#FSbON|%Z=zr1E4)abB7EZ&00cg7qR^ALjbytRgo~VXS zlbVVeUE_6tdK(f*@y9187UaLW6i=?q_^ifQz*fwLmiVE$Fh3g%!@cO|qGX z!v(JBfS^1l?{qK$E4sZ0E40%I!4G#V_gg`RD6(Nl%^4*4P`i^M-YF5zHbQ7SIX=2V zj&Epjpi7&j;9gJTiYKHVE+3dxDKkZl6e;d*YXNiqe2uUKTB7Hc{i@MuQHGzUge%kptJKYc*wY?UnD9` z)+qJXF*2_I#i9wR04X_oX^61V|MmVk2**;y+&?3CWYctUCh z#ZAoZ7Ju4uwuzEAyvc(@@}OQ+fcboBX|7hC*4;zLW>dhA_*K(T(*fabsM*29o4m^I zkkDg7mfjw_8>2(V%|Uh}2AjwVN{Y%oK(t%=_>m!#3F+~{W^k4i6wSXkv*VE8aY#C@ z&hOY99GX6(XNHlMKL`=JDqd`c`ZNa z>&}Bv*Uc^e&~Ld!FB97!@>~M60rpr)>_GJaa5k~9byfDM+J$`TVy{4@M|39p!bAML zz`Q>;-;=NMksg_$JfYDxTbXOEKVWQUPPS!gC%O%4_tK+J(Id-)aq*%evo(9#a4&EA z0xPrK@2`LPZP4%uQ4;Qx6>P98SS~AA>Nn0~I=4s0B%>r!tyEbGBBa}~ef{HDu#19G zA`rcYZJ+{|kI~qqWg1(vxAe#_@pj(KoxoMDQB4V`SKAU0eh(FH=h206(%AUV@n#?2 zR6@;8Zk>eQkOjEMYGID9z>7UL%xqL(Xq&9?6EwsgtP+6*3Z=;kbrT6Phr_&kw0iXR zTvdvKrnseW@NUjn(Rwq zY9<2b?}7%enN~Z#^O0UQuOc+V*UrZ|w%o@!CyRG5%$v24>=Pvk*)9~HNE#%(4@2`I zXgCOwgRqc%@wmw15b9xg$U$uanOZ?a6xwSAx~QPm>OqL`*Gx~JeqNB?B7`ABQr|w- zw@<|ch1H4v43BVur^qC;$lNE0#|W)%K#@S~5XU@kFUK7G{W}W;Sa3vDCSVDJ`k!Wp zX@gR?B*}=e71cde7t`IUY{YcXlTC4a(k5ni^#yh|t0($XC^iR2K`qwP1$-~hMm@SZ zl2`WY82g=yIgZ7tV(*4a)=8JuWt3^E67sM|1X9+~9o9k#)#p;6!U58B+mY6nX!UA; z0xEnBjGVKm(U<7;Z}{N@`46cx0`&-8lJR2>udlfWF4OF>!hTE5;2Z413zTe1K?8?g z?KDxdM<^t{^1H&Xut9MG%MTSUCxTOdmU1S@r)Dp=6P&et9y4MSE>yaXQMv9cz0>IH zQ&5{JTxDfZp2=l-5u02}^tJibJGpu$xB6TesrtqQFGfL=@rv9gCG6kL zd#@wEo6BFP8z!d=COgM?$y&&P_k{FOS)dPpYX!E(m^KGcXHDW8@0cBWX4{HRA@FC} zNGeA`t$CKMpejN)ego%F)k*5>MB$2d+oMmA+WuW;H6^I<7p%*&%UQ|9KESldKh0Dycti)#~*``1&Cy+Qh>@hntWaK%{ zLQjc)rf}s;@is95=Ms9&(m8>}ycDz$sXdIkASBu@iG?YEL1h*1jmz>wm*AMk&K-oiA|t|+{ccsGKv`?OM$AMj79;}T`@Lj zY;kPx!wTLi3d~4St%ZgH$6$^6`d$j*P}OFNM;O4_EyVU>>?sJ_XT4VVyA1-L2le&b z!IYb&QRtD*t}bF3dIzERh+CB()5(GQbC!W~mFK~t zQIXQL!s>m`K!{YXl{uDIJ|hNnjoa67 zk)eakl;kWJvQ~yiLMKeO>E=O7kc2kvM#AtszXlK6G#6Qa;ML(;d>Pfg2LVtE-K={; z$4%dikc4R_B<>EFHwvyO#|~UN)WhZv*~+tLgRs?POSR2Vf3q z#T`dzn5O#x)>_Z?v-O0a966(1=VdcdC_v#@tq+)UO((#QVeZ@U^((z2Y~l*3#LKS} zbsKe@GzG;Xxtr=IA2I{Qg+ywP+{y+zBD$Asg{b)x!311_w~NgLT!aAqdALaJVjxdJ z@di=9?gnxQ)NpkFGx%Ex?t0x54#^?FT>zKShPE{#pJ~_vsEQ?#1f!4|QPr6(c)5*K zuLlq)2|^mv5;QZ!Tq&U4g&}*~!K{~nP6?@r=+LdW`2mgSv&lC_&wScYxcW_0T_s1< zIphIu#~NDQ=%^($436g~6S8V`EI@sRf3IM+59bMDHwZSfM_G^>(Rho8&B!onY`pLb zzU^DA=_)iS=K05=lHh#P*3)vphIWKp@s7GW19uX9PO^`d13^FMw^GCMhik}${<3{~ zT*CVGCjXuUjZhjcQ6JoE#!>xM4eoO1{!=nub*#vN3^fx-$b zh|Ha9z+J&ziUF8$cO>h7kMjxFN{jTaUM&^$&CG}vWuSi0#ziAB?Hy#f(N+H!m*UO5 zi9(kfc@{T~VSkU29gBcxG-9JnyVxeg)XjZ!4VHY?7Mvgm<;R=Zu}6)ukGwcTNJ6Bi zk|J0)+xil78PEr2zK7EZ{g~>m!m=$ynBBM3i1>JBt0YxcCJ=*CNsjNnz_IZ5yBY9C zd|09$)LoNy&qrqNIm_21!>am3_SD577(~u7Hoa=Kh6hkQ!ZQIHBCv{A=lH#86#A-H zbB8uiK4_whfYnJ^>jYG%yrR{Q`jkxL^w;T>?IC2`C1L!hdC@>xGhai8@A z6eCo>mJQVw2Co!;Le5C$H!a;TZ@;*u)gCcFGI7Uodxk?KR^7L<5G|&-b#jz<80JX~ z+Oc5|^TmMQI3oa0j=MCHZ1o&6{yi}Wx3YNMh>WWA3KCWX8<-f&VJGdL7ZyY2Gf;Oq z({4I{S(RS75~a{SimWu7E%)x?^RaLx-Yjc}YZK7Am}&pDRx($r!d>7@^U`WSwUvW=-ScUeuN`Eq1lzmJYgpBc9VgGtl@6T0XPgdQ?&7%!*53*4!V>BoHt6REN1$n=mpxOYsmN~NAEa@MA$$~0SJwsBx>V%qk6&p9bYyH1*?-ETDtRvA@z1!tNCFl z#n;Pj5sTxGQF7zOrKnUlPLqA#bOXu8-nsK`QZHNzc~|cmV+z0r^3RB?QF|u1r7Eu_ ztbf1<<}x*k#GP7YqCzmJN(bd8F65R(+J``$z@e!%v?R3*9rdbYf+Y4j)th1i>kMhC4IY7> zBV$tf8&`|JFgb2;?mVC%6Y6pEzPb~zFL)BAw*P&A=2##v9 zU)Nhp{xJ_>e*%0wY%xpHWMck<(5E|+D3jFC(Iq)F#;)KFsQWttz%K11wVLCYA&z$t zR(F{kb9P&qDDxr1N!~CPa|?w*J;df>Ib9Zm(igx@L6e~a6sj`qARH#+xV+`+scmTw zL*$HnNCa0JXdhfizj4dg%neM>2StIvhQ1V`)5hQ3mUzQ0+?U&eqn?V;6jhRuGCHW! zan|{+MG~+(0nQz$G1Ju;lHr6N*Xp(egQL>NPEsEi`jG47u1xyuD|4S8y%vY2(QJfs za0_0J_(#VQr+N1(cjWkkR)~_;C6T+^Q*4dPx(HCqJZhn3RG-qS&!PPtVq;0Zvv%Yh zj?elYuoP=-{1b~9Nl$qR=tmBMsNJ|iOnO0xei*OpL^^bW69XkHSHU;(1MUIEs>feyYC~*ZlD8L6- z(P?88_sFC9DNqp(9|*f``8gG0d`_qfoWm?^hw*XD0T?>5;Yh|f| zbAJ9@Nkf4vq~t^qxmhX)?U9jTn~?x1JguL_S`VTKD5Ong0=kZB zF@w>B94h(R-jNW~o`60Hb7^gG(T4jH1s1KHC4!eq9H?E5&0hj@)y@fk&JZ^Y`e`Cg z%t}xSFg*9)@Gwe|JF7*pXo}M%ygi*we!$0A7K{z4JEoZS7GOZ%x*%(08RBI%<5@Jx zAVJM|v7b3=9=Ol@aDilIevP>6#ym+_K$C?47970rqCOVlbjl6q$BZrU^knPd?14Hu zU(yje4R6Eys8&C{IU(_E-YD4kp!wK%!iKDUaz@BnB1!N49bOHX9ikf~Y#(@8ph)Q_94d1(eE}6t&+G`O`fD`ok=|(CJ|Je6bS2 z`56fee^6AfeNav8eZ>&#+Sz~Wyk9ow)f|IG%egJ~`*uMM-*xjGC-jSK_b#4>HmOuB zm5R@(Bry16SZdV~XOf`4-Mce2bX<-C^5>)Mi);B*e?rz$aB!^9T7LJ;wH(pv3C9BM z5Ewen-630n)Zi@zmiRO+@rVF}HJUu18}GT#lFeiGWNT5@wlgzm`aOC)22NdpEvIIF`6?|<*9mSslasIj8O0m{*EtNXM>q>d7wZaz%|CzC zp4~G;=@$s#WyPOFx!^{}v&oWynh>F40ICIeIYN3XRG8IxyH&?*hN#>l28kg(s8tO~ z7%V_aw3=ptk7Pr2iIWY=$k6c+mT9M0E~cn!irlE`VEnB@Bvl=HXowERin*X%v~M@V zejYc?g_Ck_)*bCY@V@hd^&b@hTr$xBKEM7gQLc-hm}z@ zh55EQW-JctiYO3YB?$)nP<&F_7n1kv>U>Yo(xN<9?&2cr%2mNk{tFl8ztHZ#P+8!G zW%*Nyc~cpX++L1(D-@IN?YxzA8N8HpRH-aM3E)zR4u70p@#vT*O5jGkpvTD`90}!^ zjN@6U>j5b7M@BnPW=8O-7?SC%zqQH`F+wea&jfW?nI&Rn!~*E@a@ga|lTg{gdU-Xc zKHCJ9Ysd?ODWyj3NKXP*N4-#z%n9;2Z+yf+-SWL)oM&DL?6oV1oo39~J4n9pFa1 zTgFtkcpJED$mxfF`Be`6LT!<`Cdzg>sZ)aLmIU$?^-o2(MjT z?$ntXO1SwFdn|S4^RWWY%@N+{(KD;ptG~IIw#pplI0lCNKoWU8%|v}#;UKX4cBf$X z5-B})Z&GLwR{W7FR1;g;FqxJVF0Vfi6JsJ(rp8$K@-nO(@e|hxGkr{0$0$^AKyNp3 z1$v!Mf(OzpDFPqw?4#8ja%3i#I%j5eDO1DTkKfS%bwn>NLnLb?7CG8;EE_3G=TCGW zt)_a#5qa2Ef7efx%^k;?zMIqiaands3F`*V>G>G0{S`dHCM#z883J8>fX^h<(vZfP z2YN@|T>UJuW=Zd&?NN*lKem;A=c>99s8(AT9p}fxe5hjKTCpIlhL-)Xc`q#NAV=|L z%BO?7fU@pFT?H*qL--h!Ps75?$jT%5Mr$ONlEWc{k!nCxhD%Kg#f}inlfb$jIkg&q z1$+i!t2m!2JKMuzP35L(GXyB_M- zlQoePBk}3-etTY;dd+^$x@i;o8q-V8`z3U1jGb-7`n0u3C0Ri)S07ch>N>+A-p?W$ zRP$)mj`3;MQx>T{T;6%+DVT4bG92r4vm-GrLW9UWCx+GsxdT2nK4d4Swo{@TBWeCx zL^O$v$dzO~VIe}_&;Bc$@q`;wCyPjF4N)T>+?z(V%`5mdQ-cV*cP=9>o%SDr?Z5g* zL`T&_X4-};m`>jo(W=E-M_bkA4ZV+?T2cc=+Cv5Ow%5Q{Aj@9nT%Y;`srqFeDv)I2 zuHz7N-+(-$g_kNAoXr++C@h0gi-mhaArI5!a7YlvAQes`17U^g&YaXzCdwgySv-m| z&2a0z77qEEcBwp=$Dl-fXprU?!tr`24dR6C(TNe2h(&V`S!6Y05KGmQ195 z4cKP1iU`9Fs$NQzQXeDxlIr2SLo7|A@FR&QB{?b645TCTEHQ7B9RjkFkV%2ov{Ihh zSa?+b`8Mg%+pF~y(12nEVrE|B@i*GEc01YeU6G&(kf~BtWGtf)lEBBl46QKbdQp_^ z#!)44vJdZ!(&c!v;3=GsU*yc59boh#f7bx%%L54=caj_kmV#`E*v26%HGcCbOW+hg zG)S2>aUsX8fqq@l8J1|o99Aa^8KD1xyX7o8X_Vt(y_j|h)r*nANtO{dg`rkVIXyZ# zC&@K_P*DeI0ZM;D)S+$ktXO<*q9NqZ^vPCV0p+u6FL2U1*Z?+WISqu}H*(KL`$1$% zKd2?+tekHdwMU!1V>OQmj3PAT9_6x*6r1AUa=eQQ`w4wo=oaymJ_`Q{fp<*W`9e0U z4!(JrRo~RDr}*?&Us_pk)Lpb#*8uq|sFZ3pM8gA|7r zM?zMiPT$IOJ8L15oTidPjtD}4Cps$EY{cjx57HkSgwUMQrgzxFa&Ryj5AUJpWlD=fBpVIV|K9KMcu#mBPWQk3;C ztnhq-x!ElzG|D1+ko`~(Q5LSRpD`{&!khX87R|MSsQZGKjiT1X`9S&x6pb=+rsGo5 ze%2C=x->?jfqEuAB9@4`Tktl`tt{#m>HJ2P9Ts_=Q2(@g=O>{#O@D>mu}Q~rNaS|M zNxmDS_5L-^lcY^}7#dbXWCz(IbfiFZ?&s&sBmyW0&{%gO$HU7639{_?vU>GnEb_Um zP*e>LkzW1)sJY;n^+()pnGYq`13bV+Ly~D>3WC>RdyTO)kMFJw5v&Q{297B3X20XK99bigatrEs z30Cn);9U!@SW{D39YI2PB*M#*y_iku`wtXFT*TJCGJT5`JyFP$94p5n4riyX(f2`M zi9Vt|8B6W9Eo&7_db~>!lIZ<7oK3!?v`JQpvma*UE+%(8zO;1BK4^6}BQ3i+#v!7b z9Cu#+0iY5`tfJQEXVz0RSvlp6NQ|Vl7gx~D+SnWtYFC;iWSp_c^IHV>uk01CN91Uc zZ{=1cnbU`?#=m2N=Vmf6Be(xF!}1b=(JGJg%R#+>KWV6Kp5SNES4;yj?but1f^N7uo zY&`YnB!_}f62(o9w=E^I$n*S-z%5@QI~%dRI$o!8%ONlXj4DLV6E)#FM-NcG>CfzR zvS~3Ux$FFlJ2o3m+RwQ2FNW-!9YXk!ADT`IccB|P`m6r>Jri~sN4!&M-ct7Bohqg2 zAMmDRw`KOhCw%QnO}b=g=ll{pN)#O_MH_vOVE8v#csMEp|DlXqTH36o|Y?am~& z^&kW;(_4=TYTUyVeS%p~kf@oe&J@Z*#L3i4)JJYlkzn#l$nkSZRj8rcdBrIi4N~$~ zLsW72b|G|h$3Dg(I%0+7ZmX9E&sU|F;Yc^s8WJ(ooiyCGq&X!LQEwuGWXK4{t6pf} z;F4}nliD#JL8fPzSyjXe4Eu%RGolC{%*no&qHGY@8{1Hl?zs-7$fBr? z7%UO8L+MnKLsqFxnj?q^A44)~k-Ln7Q|in*6P^iHW$LHzagr!EG!$5dAC&t%lY(1R z9b(Ddj9f`loG7ELdPhvYaKNNTjNrAju2C*b=+AmN1|-0jswO#@;GJo#e&jtIfK#sS z^2tz67>ySbQ7{n4EzbAI&9{zL*|z#e@p2;Uov`k~ZVm=5na^VN3S5JK-SoN(g<$j9 z1(wx)Cx-$gw0gGJdEVzS?)Qdgcv`obj;HZ|Ps`5`{^P4>voiRSIyKo2-BXlvVD~)ge=c2^8BFC(g)6)($KOZ8(8G(Z__VZdCJDE3e{^v zFuEoGXd3)9a@6q@Pk?F!hoF9k;0W1cn^RCe$f}-=Q!1aFAt_&BkA!$B{K>80=(=cv z6_Tq`N8DDT9;%Nf^X;Nol+Vs0Wg=_g!jsU-dEBYV7$pfHBrn9YDpZTWcZ*oZ{I#{GG&pxJTbz(7Hw!XT6tz17R~E6m zWn+@sMt{o6WZ!Zm8$^>jDn@*gf#43T!yf~Ai+BFPMj}1NB6WG3J&O{# zxJ_`aR{Ef0IHF=dBO8e1u9;$Wmw2Mve$=K&;Ur@kx6mV2cFOO;5u{~*OII6#g4WzV z3Gwf?hUyh!b9LTpM&c~bAL~zJwIJooDbw zpxI)$$NxL+tM>_V;(B9WCmJ*|a8NA$ngp~>(LsN0rC9a2HTu3}<%i-IL=(H4@3H=TyOXt9GQ<$dw|rRT zn`|9(Pr-&F&c@`b(S8$S%K!}(NBpsVTG^Kz`ecpzNIy|zX=nZve8-R-UxxW)N{w{S%!bk@@0vsOdmrDVvkl~^BfsFX34tkY zSlbe@h9p=GV8J47%YQJl+@(Y3cHe_)+%b|~%TeF$&)AOJyp9DCzTr5z+z6jkWOCpt zgj@Oh%?<|(*j1q;EAAv*%<3O)|7UkkhysCtw;kLi(m(Ix@Y&O9@@Y8robZQ|j9u#! zjA0eDqz=hvJDCx6i0%Y-1~k`?5Q>N|M6z>56BBfdkc>Aanu!%P!(CJpqgehw-^^k2|vtway;?{d@5ql9KT74Y)n(WH5sSYJpb!TR6-$5V8TJ zB7Fo<^+cNS{B~jrPn4$_^2^lsY>aJ0)Q`Fkf*zw~`=|hcW+ly~3@zjYosqipJKp}O zaN?*4uphSRA4_|oypz7ILEO%P4M?|JEzG69&f>=*|~6h95~ zw{rlHM5rPgqU~!=dk9VjJoaXzf`l^Hfc=8T;Udv1Tc-$w%G5LW2!!~HOmd#8nzGA` z6Q6G=0$r7+OlNjN*UGX@6kIkW2Vq3*Dj=QJ6MC*@F=tU4qTPbnk~vI##6$dRc`e({ ztl5kJ(`vo}J@k@|a#EG280`YuL?|ThPvsV&V9pCrdcQQpUmwxqV(HJ&Y=LqK1=OMK zX#sWh>K#ciYPg1}+beg(CB#(^eLfo zbwZ@Eaj<>l?A&Q<`bH zLCwEQldGZ820%(uEcZ$A>XG`qYiKyD6b&JVuO*zRx(X1W@vqX`?3T9sX{)rmh2`dw z2`^w%j}iF&zR0o8E_1OTT^Sf>ximMq5*;_{+iI3%x977q0HRwrK%?4Ck9OOjjmIb| zP9l?6@9R%GIPJ5K)z*yPFWnB$bj;SM=I`yhw`p^K+&dq+eKh=u7?M_5lkb1?4bW_>1sz zX&OQa@rFsHi1%3$rJp?|&Ak@(+Hb?=a%2HEr>cDc$DUd^ZV_;RrpTZ5`C&6b=g=O< zJ9~AZpj)M2FUXIp55Q`VdZZ}B4JUo!svdz*$GaUvpDcN;=#8w;pK*N;q=4Hd^H89(ceLEa@XMXU3FfXzalZ3ueqYhC;9snNfLq1WYfBGGwkRvExp* zL$Uqd(fzpl+mOw)@Mk@NB0xz5LVkpBNV~l*4jpMI!CfjQ+Nx5gc*wCa=!gUA9JXpP z>R^*XpniKNZ+$zhF{0*Y=Q_6AMN?M9;=tLCGxuhUwnC&L_5OEFnq#B_mFf!SV_)1$ z%k=)emiArshCG)Cv!2T?cCwyFZqHT!OS(x1lu0KoH4i`yWrbN0l8=&dnnTG7+fFbom`kdj{rK! zN1PHum2P0ejQ!Yb$fnj?Zy#(DY+FEwK5=Dhl3dXSpK*I#NCxj&HhwXQno^@B%Gr!=J5Crz9o<@j?^2s;#)iO2#^9K?n8um6Y+v($S z^FP1>agH5!-%P3ff3kB{CF4h+q^h7%ddsjt%!kSZVlw(jp)3mUKdLWt^q2aFn=Qz{$v3gn^G%E+bylw$t9kth=wLY+$lQHmBO@EsI=08D*N{U|3_nr~S_W#3 zZ%xuWZc({duG(1`K-qz;4FNtNK9>k~x%$C71kPE~s+FN%n1&TH^jeR}d1Iw8ya>2^ z_3=YFhe#+%f~w05>7@M_%SR((Ja8w(FExfVuGFz#PUCZ*!^bFfIH|h)fjj*ti&#bM z<-&!1uHQKW_kA*m%3Y}qOrtNIF$Pl*YKU9}RwEWxvXFHF$-*#tMk@;h1Oj2JXGjuU z8O;%&BB4A6LYRKk?}-wqfQ_Zc+0I{{obu?{b7fvR&s;r3U%~aXx;!N*cQy@nF1syU z@?7feOgOWoL)j62_RbU;8)<|AHQF45g~aLzY>!*@G92>1f5QYLC(Vy|sOfSy4|lT2 z&*ej+5(5U1mZRgKbFu&};bE{^n~;E_aZ-+tPfuG$Xc+vZ5*x<#G#FG*=unev1VY4!=C~q z>7+?lJO%ERJfYmL0>+qzl?V+}+OugsE|1dSC32Z79eDZE=LGul?8FlOwMAg6D>x|&uU7kwI3!iFrSRT7bsBqR4?R^+( zpUoQf2a1jgst5bid=rnx`Hp9L8sEG|7T|C9;;P%lIL}PvA~9Ax+V7ZA6y{&wtZnLM z*tNw1!qog`<}`|h$tQ1`@OMP3?>Q9Wb&zPF;QxslN#VmGNyB$IW?)E_B+W6Zfjng( zi&dog>fX(%SkLq`wAf<@TY zF4Q=3N??i;1G2ljFKO-V$!wAZJ^dm_X@uoG$&gK44izEd!Fr=PN!3w^{XSBB`c&`! zyRt&;Wjy8YH_KiICgeO0Ez`-fl3h}`I31Mw!d@aQ>V={SE5!8e@#T79@b38m^6PSU>gXul1Ga@5ZheUSF;C8q8y=9=p zA1`c4`nee?J+FHO1p8^9p3d>!S*ntToNq=G39;cfot}7?HHFjaZANLWS)?^DQzdy6;pvzI0GYNm1#( zrMsk5bpPiwGkY|Ae*e$oVRt^ydCz<1GxM7H%*^h7m~plLEE)9uvgeO;b$`$8IrFR6 z?KrT*7B_McYr#hA>UXC<&3N4R@}1Zh#pxdw>AbLES9xBzQX1WC=&$tO-52F&nNZE>-{F*;K_VeE9XR%z@^pNB{ngevjwB=iqrLv+m_R7`OJS>viSbb$4%$ ze)Yv>jN9bg+MenApZPetn(1jep0i^a@pe~vjye32wKB9W;lN%k8EwI!(1#~g3zpq{ z_3FKkp-Wrnl3!KbZSLGFGc}w%|ENv$@#%pNA}2RZ-!i5$%ud*F`p?w@t(5c)z9UNg zAD{2?>euTdk5Bav$-mz1=&n<>NOUg`s*1?=~JbY(F_^PJVcs)0|iK zCmX&l9%Gi%?d#K!@4~E?Zyt89dsXvw`s4w9a{OIwqLY6Hq#WP(_K#Q4sc2t=aL<$7 zmOeLXu*<7dZr-J2DOeVLIHvVr-JHVa-kbeA_p++}4ZioyP=CC?8N?z22uSWN3BkALEm4@2Y!mi0AG+e&S94I+s?9Qj3jSUl+bqGClW#FPDGIXkp-& z;Cnmv>^oCAPa$r=QiVSYWqngGM(>#$Id|4augr&z`=8cNFC6_~QT~hB`DY8`;_7Z^ z-FyFMbKuGO*8*b}slS_)fBwAA2nyd7S){ywm1(`sOT8_00w)y0-@IJ1~6y<+f$h4jrlc`C{USJwXN^?w#w~ zv)in-FAVJmwSBsnJE3u4NSO+7!E?)r#5d<~_&A*E`r>66;#h3THJ4)0i(ce`$8(A@PUMW06M zXClztZU6Lm0NEa#LhZDV7Xu6y|>GMd$lW^`4OT#`e5+&2JYhn&tI)O zZuV-^)yHK!-}p~yPrC(hMvDC#+5cYE%h32Vxk=m9Wt4aAidUYoVcs_PrW#X~)P%r_k^O}E z=lflEQ~V*@a$&R6`eWBp3!SJ@oL8#;cjs~`C!a1V0VYB+^w8zSMBji@P^_MxPba8j!AESu9M{F1U{xH0^OYoLt-(*F{U+kWDU)I@< z)qcO{*3;If(FVg$aVxwB=QSr^=^}GL!=QI{ukZ)_RXr9L_L9G`(u5aWQAx?~6|Ns% zJuI`-Vc3P!i7YdtpAqqgsOUH0D*mlAJgWnYZQHk<@5SmBUmgE(YRmNSJ3U)0uXL5I z806LRG)N`pzT4*!=W4@i8U?e+~*dK7$tKdz5m%;W`zR7_C-bl{%W1-?y5HOFIVry z@Un!(>@j-x+q!boZc{Rog$n(Pm*)I>w;}koe&ya13AST?@UE*&P;BQk?_Xb?b9mbD zpt9QF-B~T|?%~BB*#~u7$5SR}U9DrMo2y56Tfw=!!ba11P50K6Mn9kI(<;kVzAXvt z1%A?&QATx5DU?~T$AIEHHrNe2yZKySXTx}bzctJ1&vNx0Dt(l~t$y{tS9ZDQJpO3; z>Wiv-*m`-lR+z*bKX`TC_C+znE%XDDU%F9VcVo6{UN!GF+I#Bz*3ZSy{$uy*btn0g z#BtUTPZCnr5#lI`kvdTQH-439oOEj!ZosyVbjTXnkFe6{{{@QjvE^4A`%(b{;q zHopI9wHHQ1jtq0Dj%H~unfo}mBEMhVPZ_<--DmzeQc(9X&Q`nN%)tdc0@cDFTucb& zrPzSyLTbEpr`}i0I<8nYN^_LqJWJf;kGe=wd-0!YW(}vj9 zPW{lECmhzb+d;OzZcAzY-W8WFq#sWDRQuv_MQHk(0Ym=2+v(7C#cm5H?VQx*eQQUJ z4v5uUbs%lPhz7HDedeyY?Xz*JX+>$lGBwY{oOlI&g_7hi{T!{#u6&+qH{IM>!6@p} z{o@NxL>8yqYs=l1^7?K4h_5e){FWX3bL7r(A^I9;rp4SaYT2;Jq&1YQ+;}k7C_d#- zTYBb@@_y^AQ`P=j^{=o`=yk1G?aa_&`vRJtrR{WiDRb&v(TNLccKj)-I@twgndJrF z_ExR9ej@vltg>#bLw9?Z@;5oB>f8<)%UZupd+*|VbJv3IyY|kz`tjcIm8M(sf>I~1 zaW^>RH2p!z+UX1Tr)U{1SsWJQyn0o)XTKAl?MqL8qZTlm8(w(uT)~B@4evw41Lf^r zzTdx0SO126N_`%GU&?>$wojNY>&mt=YALZbD&s!+dF#%t9}j9un(}jQmYT~x;s|7K z^OF0tddyrNf9GAT%;put9K!ziANw2LRu{Lt{mp>5pKrqcetq`5OYrlvZ#lZdQ=X{w z_}gXY=ec#Slm}!Dusj&|r{((B7iSetzI-mTcIwYhNlo06f$eO&(^+19NB>qmbKh!} z?0|-=)!!bTS5{Bp|5_TA_b0mLM0M4s`0$)Hvn?OA0Az z`n!EdP8{I#-mCoUw2$!(g)c35#VtNteR8MYbkgs&WrQ&P?{LjvKc}bq6I#93W+#E) ze;!bJq`y~T3o&X+tMxqWj9Qi~YY&HBObh!nn{@KE#7M~&XAmo!v;x9)Lj zKt!tgLH;s7)_T7kvPWio)8x6|(sr`VUzPo6h3U)Hd;1O%a_kNDlc_DNZezdxef3fA zwYb;08AW~t zwa+geDOq{yo#U;*8smd|m6~(olg7Pz{YJlkcG9^6ZyO)ECGi`6WrwyWR`wb`u)pqb z*TIAR%v*YPQDJ|cnKq|5WS}`W<%vUNW8Ig`GL6anKfJ@-9hR@lJ*Fke84vHNaeXvD zE6M*%k>h#Ka=pJ{G7VoF%T`P~IH&80`K=n)2iNRLRPpVtI^C%HyXhFMIL1->m3vv{XWcEleteYcAm$rh}v$4^)H{iaH?- zbrtXGKo9>5*)nd)^Ztwf&;PB(U(cJ9t8?U=cg+8pbNRn674ICgRn&+2>|DEki=2*G z`&EtfAzsT5X1$SrxZtGH`a=PGUp{a0*HXG15%})b=ajlGDUpYa)Vj|}Fj>0&#!%%w z`NG%RTE@Q9ZTVPV$BVNcWgXgQ(sPZ0tF~JwH`iqD)6-t@Hix&aI-QdJ{meaH+19o) z$2Dl=;G&7g$DPY+Ugf0a_}=!$qL5|R6l_-|UKVh?51#Pprr)A$JT`Zjt>w!zUEMd@ zmMFesImLv8dw67jvMX1OxZKn__DpT&?Up&-MM}}#X352a=P4u(HsxNhx%$FfDMA@yMBwzA1-^m^F;Q>dENV+oSM{X=Q?1E*OWQ?j*Rhd zQ1NzneY;PdoUEfEUv9GdHK(jeLdypqZ*!)b9v;4Bbdh;tlU3n_DQ!ouo<5!s@4YFt z>UpA7>cwvZ`OCDXgb3Tx!863~pZaw#+t;qk!@jzEuD6WxQy%Gj>YhTO$Lpij=CT#H zmi)Y;RJ67xWG{PrwDPN?eV#Zs_8+R7lYZ;DhTaRG@~Uu^A1NdKWVT12>$2fuRj+en z-u-@Q&`{*HeSdW~Bhxc~^<>xQ#vNHT_PzP`CPCPul5vyF&n_Jje??6_=juE81ohBL zm72G!Ob@k{x6eMN#L9p6wPd;L_ujqdd)&Kz?CWr~SzYDZpT*|wjaulT(4tl&KR$Di z(i``t9`>3)iuc}$bbk9<$NqXo?51=14xG`m6stUbeXHyjv~Zi(|7rYzxYD$X&+a2q#okj7%D~>Po~$+!XvDAN^-vvutVwO(7Wai?G-N)_a4@nvK4GfSpmTxRTC4qkJEc#m+L&#& zxh>}Dv%#*T*DTL?MMffm7?=>awn-K6y## z((>Xd!NI)tpOZG`I0=Fp&#+m%iBp=~XQkABOgh{3{p{VFmHTRJ+;LoR)<3$pE~C~3l{oHrx51#)cV7^OZnnlHnaDvMC<1K&o}3u ze!h7?vrE_L6yfc~KX2+Z9!ly`quL&;Iciypwr4Mwc@cAl)NiJymjw?T79pHz({%A< zMOVdBzY6u5v+2u9S$UAk=qyxPjv=0aTT{3s&`azfW7)K~*Zq4vExidjAI7q&1;(#Np zJNexvjB0VoC|~0GeeRw*?f>GoUb)_T&X9rEOount8=sz=R_1o{;Y0VU(;S~|@!U3M zMS11E^m?nHbB64l@*W9GKABxfv8xzgc6(sPm*YO8T4Y$VNBXzS|LfClqRU+OyKn6L zrX}{geXk|fX!O%Rt8)D|TANfRS=U;P`8JsDI$LS!UkHk_$bYu5F#^{9Heu0E^}Uo9O|Xn~g2^5D=uwWEKJ8u8;? zROQjq^2w*)hP)eO`@ojBX6zR2$@f>9#l`$mJ}tl5-fax)L#TCXbx7B>i}h{JUtJy; zRs47uOKp2YKaO**mEQ`5m=EWsE>>T4(eZ_>&NJD{R{498-S5{eW7)7>oD$0Kaay&* z!V-I$I)}f$rf$4!oO8^TZdMJ;T}}3Q#k9@y8??Z`=BMwGezM`}m3l>Mi*~J9+r22r z`O`t}w)P=j+g`q&tS@ZNi*U>wy?J9AcRA1H!jq*fPU|=xvEkyqjmDQncSb&J$k~1^;?~5f5f{4f{FeAXosnoblD;$A!!(#d$70 zk4=-AQkNHZ&TN@b*%kcb?Yl=-ofuuP=RkL>cXw94m0u&vzaP3g?XBa14G+Hu$R&Q$ zNm_4PI8t-(wGoB)vfN|0OgpX~_&B=9p5qUrw!41m@+sW)BI`)m?zAOmXFOXK6TNop zz*nVv_j~P}(Dq+rxciClvI2w01J?(IZQE+K)ygN+p-gjqvhVt+)r$7TqizK~7bse< znGj#d)pL6kF*!AsD~P!>OW(PxuA}wAWd*rMIl;0I<2I<=OW5IJe8s=qbHuyf`JS4Y zU+Qf)sZ4&j*xvN{0FyU2W_y24zdw4K1$DXgfThj!gR0l7zwu1E1gF1UW#RF%zomaP6fdtDSi&hDIO?OSf$H8y^Ty4KCLW{Ihb_AC^!$6PFI?!g1ybnd{!q7U+Q~S$cNO`#AkyfH58 zi*;y3>cZN3fw!WK!?}{5g-h&=Qm=bxchfl&`sTw=!`4r~SJf=jd(Qj$>G9O9<5ov# zv{yYSfAV?xx(i9SpG|w(I_TBJ0TWU`4oj+2*rFhp#JSs*=iOWtI&dtz z=D)95mfB`c@-e0S0gJpcO)u&MjX%Br`=`72d>_jtjoGct8{1DQSFyL%lbkH0oNpfv zbXR*i)VbIxxAgqMLAnh$?BZ=R=7o6;UtYc1%W1~=)?-`tGzv%S^}K#h|8$XI{0EkA z(V4705#vg2dg@jsscac&SM@e`o5qlssz21Tz) zw|s^rJv$Kpuz65(^NT|wPD=2JpZWf1udkjjHVj&lcIm%C)8F-c zmZvdt{?(QS@PA!jUmo+J&j8ar=g%PqYhw&97OqKqShQ@Cp_hmG>ZY~s)w@FZJN7QTmHU`15eZXl`x_>MI#|-x!<6_>sxfYPn>@G zeWvBBE;n}{Uoqy?uoTz28>!m!;;a+Yn&VE7EIZwstvFmM|4z)HkSO0r@_Ds=`%hOH z(Y|tW#8%drrvX|8L(fh7Fy>qQtuHRu-PG>v`ntq?%YoAFi>>O0o_t~PsH(>9;Fw#5 zX8Ggap1t|gYh6`Z{K5Mf4jR+5!$NgpCeD7*7Qp_zV94C|{4@FOt-sd$e&FAjvU7X< z&niya;7e~*!^~kZ<)}jMCf@K-+k9;5TIBrjhuDZo+U7x`X zyxw2^=IvO1B-(ae`SmF+;q5731e3kb7k_%x&$K*atI3y93$DKVxb?AY*ym;Mwwk{j zy{28B3cEe$kjGG^uqclknYMN#EG|*1kJcDHdF=G+$_$6QwcDc>Z9DKoI4kd}-OrmX94R4@50jX z30ZRPyxe{X-rVp774P^Jj)KWLmBNP;kAJx`W%am4PWEz1Ufa};P2Xy?vuB^su{$T3 zCZ1c+CFk8v`2dfdEsNQir?rkGOm|Oxvi!V$=&|_8bK_g){Q7gIIH__$@bKVAPRi9q z@x^lT<~@ym`RVw!T}^W5jNjz0mEttOY4`lK&!|zoBYLkebg#{O5-{fYwgZ*HvHdq~ zK7T>ez`4Eb;&86!k>huaEDoI=`Ag0;S8lZTuXkG)H?h&*P5Wz151r9Gjhs-Y$JI3omyRiY-t}<6$?V+N z#KNP0lK0E#zqFVYlb$(tRdYe!fw`GpqX%sgXT|LDu1 zi`w4p9<=D;&BHeMC_iG_PKC_rB(6A^Xdogy}lrL*p}<@S+}prxOH<6I2%m4 z%RM;r+v!s~yDGl;yLd&?gtgTAiX1tp1Uy z%hP?bJR_w(Sj~9nwxEf-ePb^_ox+LqpRPW6l#%AF;=QT0_ojwC`C>o5e#qlfK8GC3 zW*7KeJZ7O1!Qw65eY7~6Q|J13+?h@NT+Lr(AHH(s_JRXtuXa4~Z_4_eeo244aqoq7 z3PbH4WW>xn?pZK3eboN_h6aF-gXE z>z2)Sp9Z&o^w+kS;$I#YLwPEFkA0{U;uE0pvFOJ1U4BEnPN&3tc@aLr=VaUDG|b8qnWhrF!*0yYv?TroZO+w9MC=itA+_4?IDEUGlv_3mPcgHLxyxmN`ns}?1? zxhJ}=@_#OW&#S5K@ZqmLX4FhRI%dF?W8GFdzdl%zwtw4yH%f$7rN`3^_uac;J8_eS znfk|tByINmDx-6uArH(aXFUDJPSY*n8V ziw?H`(y}?GXt=QCSGdse_9<(v#4!sGYZ$mbjP8GD>a()gy_$2kq`G#g9$s8_%O);#O-sik`o_wTUZrj!Irv1KZClc&WcD^lan7t}@N-TIoB9<;COi|)A1C(l32NRc-) za1LDZBS)iW(!7fgg#Hi4Ouq8TBd74N+Y+5mzpvNT=+Djf=UpE>Zr#RJ7ORG&-QU%y zr)+h6YJlOX+H-?T?r?QiPf)y(liHH+H*fo;Q_JUR_rKl$^3SEAricD}QaAWSy!OD5 zPiiY`8b>K_%2LRh@Tc*?u=nYHD(b3TKQA9;IQh!cwY+NWV62RH?iDjcJJ8c z5Y|)ICgYH^Gqp$amzR$ATaqwy>v5Iz>lf1d_IVWTxP0%O_PxWB_O$9cfW-#s`9Avsk~6~aF(U6vFw>ErQ&%g(JnO$i3}H@uH% zG!>nCOBw#qj*!i0{CYt7`zObsP1fP=S{YW_Jr2F%-nx6#+aK(@)we4RmL9zdbL%+tS=L$BbTD0|q_~88q_1 zlR=+bmM%WIbaB@CztP%#3L6LS(tdCJDD-c$sjkgv&mWTx`55NR?N_f{e0$d|qx!XL z@>G4yKTr=;bNXDoYF_WSxP5zEQvcph3PWa(ocVWK^4V)&f+HSG{PZW$ruKYMxVM|< z!0Au=e;8jnCU{+u;`_3F>9vKS3SHtJUB6xS_WH@9L)R{rvgP@H4Yg@2JY^2W-}rI7 zH2bKDNtRadvm>7TDeC>+b-!o6Z?b>)Bd+$ZMsFx{Gl=>e$O-k&nIFCB^!9h<7yaZ+ zFHgHy==m$sgmt{{+@i~I{?h~JKY428(zuQ~9bsXy*|nh9!~E+p?UoJpr84tpD74KQ z+&5ZT_t4fimFq0-P11_=T-J9`<$-UPKYdF$k$9OO{xEe#Pu@p%_0mz7xi%Vu-YGe1 z{rvSU#5q28Y<=%-n-omThZVM}tbUr~ynCtkklPy$9NB-&$YAY_gg$fo3#jX@yf<1a zjNg~nq}_Hg^hwa$*Y*6_FYW{G7EL^~F7jPrw(-o{=>pCjRlg|SDQ@z;e4Xpmz?uYm z{q*N|%<9-L6w1~vvsk?-E_U8KyMlSuXJtHlj2cea+t`fM)bkAs_46Jc5Ec>WuNmm; ztI0Pp;2RhX7YxB6q&HdcrV0MKfIs^QlPMM@Q$aH}_@fUQe2)KZ6xAJJN@mGKN+w1{ zrmvEmd^QlV1K|B|+-~~anyZqC) zX3I6Q;_6v*HLbboAcZwo-bxYr3c|h~&>aN&0v4;;z%oc_317$jO z=Ax(ql9ys%_+KnhM&SGsX6xy9;6H;dYcku4tC=8c%N@&#Qs8P@an-G0nQRr|wf(|r zM$+$&T+JC=c^9sxEmt0#(mwm)&b$McIfDf}NPM!_f$28UCV3i?Su!ZTI{(cV_7->UW+y`>qlP$IHKtnP;<$waa{ETwk=na z<;PXG<*Hb5<;Q|3TWrU{WnE2|HIvp!hAb$L_?q2Vjo^wwEl6K6)<>#zUXpr)K4858 zeI##h*-UU4#450pn-e^6@;B43@<9n>w2}Vb>n53Q=LDeV3N3adXchWufo{Mxq22(}g7h&s{cF0c@OmBS z^uRWmuCs(L?%evZEDs{Nqav-#Qk=FK=bgc_rP~tZogfP?jg=3Y(nh3pYemWnqf^so1XpoiQk50xJZLLv(qz z<2YAw9Lt4{q~j|7N4kgqBd#6C3B*3b#rbu|a~DH#+w~G46{y(PH+sN)23oO+*Hut#!grXW?b4IIP}2Z z1PgZ9jY=J!xB5 zb`Kawj|*lvZZI8Z&sCWPV(9B|!@LL0!9nNahuM7bSOGNt?^w~I$-2O8*L35?4_pe~ zt+*|kGeF-5;u8-Yj|i- zX_S~oblR=KZ5ozkdne^PNtDk4l&==EYxKpyu>|M<&oMg8SHfA@oemieitygAjweVf zu`lA@&kB&fU5fON)8HzGWf%Q_P7yN#{-UtmRoXWkk3@Q9%~F8v1#Yb46sL3bT)DJ8 z-A21{8hU;}A2d6KnW)`p&DLY43BpLTbD(Bv`}a65osI(sJfE<~Kpj!ZWTr`z6HesoKt{QgNtw^J@oC+d6jIJz6NP}-Mh9G!#4(Q%54;W$dq zOX{)RO>8HwFC*4;aedh;(xnDofX?W;!$CT46(u?$MTzpJHKL6fixx4_rjmQK1Dd;+ z$gn@qZQ9w*7$m3jh{FDGbHFh?6|-ryMkRg1oogg1Dm@;Uf#_0f3+Jb>9l_iQ&wtpK zV9JH3`HqU7;;`u6-h$Ki0)9|E!dodUmlD2cx{UNe>q!%{&g?BZv3Fyz?+1K^;J9$w zIX$l!3+|n^ih*sk}vXr0CPE--JDo~P2QH3NE@D}ArBQlzyw z3#WCF1t*}LGU;`JbXu?s>cTdX?T{FlPXm7&ar#7B6Cl0xy5gU?7FZUL%L)fY_Ej)c zb;Q6G-KX2J@ADvx$_D4}PU87DnB=mG{=F*t9%F?6kMAJyyd7MV^!g5@OW*`>&Fr~j z=X1@hftL(4i3fJz8o>DCNOw^$@V6BET1VHu2Uo7!Qbo3r3_bMKUUlG(P3V#=o59XxfyE%#!GY`M2HZZyd2E)Rui`eq>y)vq zYp8_csfMkhH=V20=)AOGq9Ww`S*Hl z!H|1RJmjus!FqGXe!OWvU}-B^hk|~uZi^S9&Wd2RkLF(1=P%M_O2p~#?HR5=)?-#d z`vPqsz5Za?i^o`NMNlw$l|YXpwK&ZpIt`pZOaPM%8#GBAuLx$(_HZuX0ixTn?IgOb zKv}WoK}{vg!tsxu5S%~?6*rZQBJJ&;4J~^`FLeI_Zs>M&gNtRb-z3&6*k{D|M>t-? zF@}|b+=%aqPKvDe0DFmM3j=c_@TUzYe-ssruL-9Ix<>nL&eC}T1^R(B z({P$z^t~OlpJbfsuy<}t@88boT+iba*%@@qlnz^NE{c>Ua9x4(aR*__f!;R)Z5{&m z*Bt1%4Lu$;VVj?{&Hq;xFV@;lXYV#>U2LRII{|Amaa{m)B(uT1iLQ$hO;&+)W!Sd8y<h|U;+`VmDImy_;_&lB9L*MgN-MFauY)DU;pc>6dwj!_ zm{S$dUc#SB#j#;;k^&%j8~m}Pc(hbBo5CGPioJpM5)MBL#PMwLC&`ioU4al;Qv6^4 zu=h!fL1+lHm+)v>$oODWnxf!8?`#T=4F53tgbhEL_!nD3|Jq^*KbOO2Yzp>jahxc? zhO;iw8G0WN{pg7y{JagHv8kbAE*)TtKQWW0bUBCr3&7cA=wHI&=VsW7O&NBur|XF& z#o=dQVtalEdm6AQVFylsCx9iz;pbo2icOhxu%`i=GG*ZA415#=AI-qWFmNjdZo|NB z8TdE`ZqL9S7`P(?cVgf!4BVB0!`WYlI5su01ET>O?yPi-VpCH(LTEpYfzM#z^eEc# zU{iBCLO|G*R|kFp>P^DoXHM9PP5E@NrvV$zygNp*seq0U+6OW4xeRIp zV)J(D#WJ#lRA?G*;KjYL@LCl zPD`Chf^6!nlm(lrmO4Yh zJZLZBa32|4v8l@)?9YNJ9ZQPCePglxl@9haU{hB+@a-^P35R>d*osZnb+A7T?InDJ zlm(l*E_H^2H=(_R!~I@t#inj{uy2O;5)Suv#rC&4*wcVb-RZ!)vg!JfaJYAit=QDP z4)(pFy@bR4TCx3u4)!!)Q;!(~ z1%qIDBpmMNVJkNEqk}yS*wn8Md?EBB;cKKU*wi1XGf5COjpP)DlAs?JZY}V=CU7n~ z*p!5e82t|RCB*h3UI_h2INT2dkq2R>5~s4F1kjK4YE=@!rsO;Lr(@Za0s~iK;9Lf- z!oYb9ygLK$$-vbacpnBX`o^=Q4$$>#FxYD{a4iO|-GNuZCK=p;Ut!>=JbE4M5g*FH zhcR$N2F_>TLI!T!f!D+GbN;PJaCQrF-46EmF_$M&Ji&fM`>3MmCd@^9t|I;hb2$;f z@6DLwuRVxkTOrq#Q5O>wXvdu1=aW8uV?RBlAqcZ!hvHBJ2;LQQa_2)Cb0J~h19J<4 z_r;uCVGo2{m9VC$!Pwr7h#8LgOo9tBCs$_Xm`4%zV=#{;xE<#41b4!mT-mrouG(QI zX)3ngNcfqJ`9XsFVt$I?!I;+&JPdMGYCXZDu|0XGjm4b2L$1Jn$a+q}_T(Kb6LWGz z&%s=o=otl=lOxa;$W^I&qWn9sJvq|sgIrfeldwOG?MD##p1^z@!B0c3+To50KVt&> zy+(xnWo$nU%-5ukI?UG-{1)ahgr5hH>&oa!I|1Qm*uI_c^9plK*k2`yK46|o@K(%4 z`xT-T-y!EvGl}y2#r7?PpDtbKl&Vw_G4v^7t_jnLyaP+jy$Ie5^W6mRk9iBh2Vrgo z-zENsU>-woL(D4(Zi2ZyQT|bwyAj+9at;+h)Q>&pAq029Jb~bou>Yro|LNGikg)f} z_722A?~i#9!RJA)D`O<78W>uD?NbT+MUZo-|A>5-VSb+A37CuK+oEjJFn>na!|w-! z&xyVed>yt|CiGm0xe39GF&F&?NtEv{%%#_9bld^VLkT}eG1npZf0&05I!C(UP@>;0 ziSoUO?df%@^ihlX4uapnyp-T~F)t_hW6bLa-h}x>g1^PQiQp}me<1ib%)b--7v^$A zJ+Po&IaCjVqyEF8h7p{L?X3yk19NAB_rrWT!3SX;Pn2^A=1U2C^t%->ZWB5;h4!kH zCNc4{#C~!KKX%yuHDQl_`=Uz46MPc3-%aE@1N*;C`1i*AA;E(%e@*Z(?BA`cwAv_Y zA-0$P{E4<$hWSpyPa@_=2%e7p>k|C@yWy_mlx_+iXhaJ-k4^(5x%1V4-U2!db4+>YQ^F`r5B2FybV{s8k81aHJV zhv088FCsYl-5ZBGMDR9je}UkCFuz6cE^vJ3P%j9sggLq&O6pAwa#iXs(Jp#2no>Bz{IEmZ82X@a3{<~zq=IWFcI@Bg#C2PpA*~* z^IrrH#Jn48o}RVKaPZ-R%{1q8Rp{5ZiUV2;{TQc06BN9`uz zvoL>2`0>TOjo@=JXTf)gp9Pq!5TP5A0YS{%+C_M3iHbZuf_Z}!Ea*zis1J#|4Q&@n6rra;cLv* z2>ucCK?MJXxsc$$AqV#(qJPOkI}SCDuvf&~mEb&hu1d9ZlhiRC>W#TP!8I{YCHP>> z%?NIQ`ALGCU@rPyq9}I@$T^fLd>2uQvcdM#h>h>@n5Pol9rH^BpN6^UcY~rFJR#>$ zbK$#)Qd9uukp!QQc_P81FwY`*Ean>tz7q5O1W&{KKZ2w628XI4cs{moB=}~`zY}~1 z=JyDW%E6(y#60IHw(mpmGRy}Nyb^Oh!O{APLs=5M4%<5s{5Iy(2>uB3K!P`6zKG!O zFem5RpE2J+*rWR5P`e0@zJko5N(n9p&pA{z!BIVOs5=Djf$due-VbvY(XUbcawt`T z55@L_2#)HVLzxlW4BD$w4$2aZ)1fig&qTr={x|@vPiGSx{A>jr94dt19+<}vd?w~u z1owd)e11WcCm8b}f`?;28wfwBy*bn#f-lGRhY6m9`8k4TV1A9@IhZ#P9O;HbH4?lS z+kYhZZp=l$QKxH>J`O<+#&hC&kBd05AU}=$%p}Iu3z+vN_!Z2B1iyj#1cKj#9DE)} zN^}F6Na49}YR#ha|4c7~5YY>_=hyvBXE5*4X|EVeg1}9l_l&Ur6w&m^To74(87Y z?vMFLf}{S-p=5~rW+b*>MA$FEyf!ffbtOh7WTh}@c$6= zVuC-%d>6srVt$a|pCAY0Dv@tHwm(DI|Hbz8ggpngH;1Yt?7Lxpo8a9c2cLthiCPf7 z_r*Mw;98iAe$OpRH3V}n!rln;5`vpT&Y|8D<*~&4Bf;%3|3h$R?Ef|4e-h@J(qHfd zg`I)@C=lbeH|A;t55ioB;9;1LCOGQn9Lkp9%dovO!4om}A$U6EK!-&6*J8e#;04&v zV!}@mwpS;aQX*Aw~rLC&GR67@e%CDe`276xyp3K0vGw`zv{3_r)N>hAYd>ncUxt5sc;qM>)V(=r+&|fqd_z(te z!N6S@_%sGiuLnA-=O_lA#K1EcI6d$0%zrThFK6J54E!wv@5Z1XL%?}bzn+B8JO-^d zp26Ooflp`Pt@@o`ZzzL(3L?zGVsL=d#VN&SxA0GHqvH++PjGD zK{4Pl6LL}Of!`-!9>L%z8QP0VM^T_z;jtO=iTI47;4`JcSqX!ma}4|$1OE#Bdx-rX z#UXjHuHeC^aw0GVa~;4tt2bL{KSS&XE@Q!2Amk#|NS-;J$l#{{+Do1rAo8<^!TvO~ z_YvnS`npLSgZ)cr? z<+OqJk>Y&QamWkuXfcP+p$2DR41N+A_&NrDjDcSUoG0}>1oVA$@Ougd`=1P)2iC)# zUGFf!c~Z|+$iY`=3VE#fdPU>23xgjY2ELSm=K{`?dOkxb&S?*WeK`ZKWZ(}OcpC$k z@7H<#r~=NDdR~O6oCXZ`4h-BE`dJ~a&+Rz0jKO{rv`-M*i~9RX2K#H!K1pmZ>Nz(V z>_0H@FATg#f4Ux0#QtyMkTv9KVlFCY0OX?H1Ae=W2b6_?cc#OY3_Oj2XEX4F4E#R^ zUctZ{77X*#qqh#P%XT6_6K-xya8O20yBrbon=l?Za_M4{(uBaU=+s0?wnhh!KzhJf>;V zycnNRlnxFp1iUjpxzK)_*j^ilN*V0yq5TfAJ$zO=IBQ|BR~p!PzQY)}1p^Xx`X zV{zyd!+g(-!H*>acVpn^hjw0`8L&JL#N`=Y>Qu%E)f{TO%@^#591 zPWU`mT$5c$|Y=@;o#J%xfW+JP%C~^G3)e&qD*-8t`a^T=G1$ zahP`j8=;+z&)pfgKHxk`^87TgjDp8l277mCFL|z-$Uca{ekrt>FFdc%&Zw7lV_`r@Q{twoY|H(TL@XqR)5A7vS zS+d0nuwt;E1MMZxe*@c_@K^x3yeBIuy! zqOgd0{t;9_aHya3xu1WCKd^uO%|I5NT;Q16hCFkjE8Fi>O>lFm5N z6a3qE&qdzBQNf{suzZme@COP)XA>S0?Cb3r7V7E0&_6UvlpdGR*V{M9A5_Ca?~q_W zQSQN^0bx|2f0SorlwVjhxX2KH?@-{$FB<%dd2qSdXQV;6cSMw=(8w36M^QG?&EXjp z;T;+o;2+`X>mBax8yvOR(|e(JaEP~0h`#~2K<^Mx2yb!a(e+Fxr#w9aqCJq`F& zR9KXEh@|Q~L;d~z{QW50-kz|Wu$m;TjOq?vk}tUEK-_?lQJ^tIrHt_R4oB@85D_+C z)c)dpjVaKwpgLg71NShkq*ww;X8+_J>>r8ROk6%3O}BSMv@c!1pqV2*=Yxg4h1_+pLz=|Z5-ghmGYfi?r~!xs93ZbR3Cs8G14U@--N#ssPbonrp{U`cghBUo5k zq(E*_{t=4lT{03v=kg9A<0|)0)Gy<}fYa2qxwm;Ve)IpfTSFrso^sGnkfd z1QYX-0QpAP3Cx3!RfJ`Pb;QRBkTWbmtUxS5q%uB|86WA4kA%iYO5-D`@sZZ}NNjuo zDiV?#AL)&c1jiS`E8!!t@saZQNN9YdG(M6VA8Cz`WX4Ad;~S&m8lwwC!r~jF{EX2R zA^q`@0QpFDd?Y(Q(j6a3j*m3QM{?tvATLPze57$ck~bgq3cd-dR-{8d5+WZdk&mRv zM_S|~A@h+Y`AC#}q)I-LB_HXMkA%rb%H$(y@{u-ffw@9e0P2=N^r-{O&Ig-EnQBw8U7tq_S;h(s$ygRu~4 zSBNwyL=qMv2@8>0g-E+XBw8WTwGe4nh(s$yq7@?13Xy1qNUcI7Qz4SD5NTJ4L@Pw1 z6(Z3Jk!XcTrb47$A=0i8iB^b2D@39dBGC$wXoX0$LL^!t60H!4R*2LpMA{W1(F&1h zg-EnQBw8U-s}RXlh$JjT+7%+v3Xy1qNVGyES|JjxF_NM&lAVV|W29YUEPd3SjFA+L zkxY$|K8=ysjj^Cnj}xMs=J_ucorn6zz*DdY@tq$I)-hnA;XBXMH)x(GJ%gtr=0`@)pYIK(`d|%6 z0V{e{309c%y`>h*q;!ap^DMS(zL(FjtDZ1n?39{+c-gkO#V)mW-~SJ~Y-VEB zxU{ePhnFqt_qg(`iEOmVQ#DHpspKEUNDB5@t2IYFtgF?;Eurt)4Jx za>e8^X)io^XnILaPy5Zhmz_+WYxoZ4~ieXbm>TqGq_;fB~e-9lt zVdCToYv&GhtFLeiDb5Moz4%@*Yw%v1;U)bfh#% zAQ|!QRy40^(-Iuv)%8y;^dD*?XpX@zLr)jF_R2NgKC+Db4nlN;%ysEAmUooy~ z-0+%w_~5Si2~XvSG3m|2rcX}CV|J2e618o15^LDXr-)thU_$E3(<{^2$CRo{>qA9* z)5untj^)Oswpbye!J`)V5K%oQsftmsGK~o1YKM-37w;%=$fst?q|xaZZHc^VCe%b^ z-K1#p|IkZyjFt}P%;}PjQ51E`a2?p?Dp|79c(7)#tM^!wNMl@)Z*>}3CRXTFGLMP1 z$4VXs9k7(t_}>rYvam4B68^nYYv@ap!IVk=G=}&npAJ%UWy9>GDZ?gB9y57L@;}+g zFI0^$riGfS)Q`P6T1Zm!`KQ-L7Nh(U?@k9`Qzq)X5(ien$p0X57t(ob8au{JNas|w zLr<`xGutwaUda6;O(m(jOiBY>I^oQY7WJ*8xGB{>z!!~osQO(xV;(;>>(qG03l{B0 z5^s-nL^EVEUNzFsw`;ZOpnIt1!F0r(2GU8zL|-=sWor{^()ssv_C5~Mg~rb3>T;|- zu`4?6@Ch~3a|2BCeI7!y)GKVvN> zNQdegXtW&Bn`>ENHV;W!an$9MF9W3JndX%0>Pj2@vKz(9LIw08G1cbObkb?_zRxVh zMF5v*(o#3e7KWu79a}j%&0Cd+jG3I4G-=q>ae3iR@nSK+6n_C0@2DMvR5PiORca+x zym89-I=gsRVai@xQkY6sqN`M)houu+jpv1Nm3PF%p=pXtF-)(>`ML&{N|)L~t#E8r zRZTjPpQ`HvX=q882>$7{B}y&(9y!g2X$DC%cRE8JpB6i*n6HuCUnS{5vj7O$D|`_Z%yOUk@=Y6 z=_q=lk7DY1RcSeqyiVn>eOt~*FKTtAt1L@Q55;ul!);Mxb!*F_z3DQ9nr&K1q$X`w zL9pN+S2ZzRpwRlHsD;r}#*a#qb$-LpB7h;sVM8YsmzmOPDZG-#(6NDUdZoRU0X{k!dWl z`N>tbgdd&O{60>GmuwwVS(#c|y7r+lwg^)i1`?TIt7j`&F{7lFQfi1Aj%_?za;{}w zOTJ=FbBW@pQb(nsD3_op(-P<(3ZUibnAELO>$NM9={k{LLDD6oIzi5=_IhPr7hyG# z=95%le{b4})l>~1GcsLO6E5OVIZj)W!Y{AgF_Wx2R!*v(aESLH_r<#Z|18c1CrzsM zWlg%SHzHj(D^}iV37C4Ct~b?AV-wRg(li966=0f^i*>3khl(9tF+E8?{E!i~ZD`7r zbZN*`wvDxCOLQ+|b(>EEMTsZWsHQ9HM&cz3{I_w-cUNhIA6_gdD{CgCOQ#kQ8lF5w z`_D8F6qk|pEKPjF$7)ol-AWD}K77i!DVi+u1+!W?Ua)O;ovE~r%7!#N7gdL+Hkqa6 zMD2P+OF*qNa>H9<+w1vRT_xX~)V@1ic+~#Y%i*mjLA0Q{_xr@0-&SO)A+`4Xcs=cu zYjLgFRJgi^`HyCOgS~{=yl$_1XXU7LEx)$I)HV7N-QWEz;_9tv{DjG4hGTN9bz4zL z9mUqW*lJxg(RO-bTH&PWY|@zFn00J{U#HjUN8Pc_uyoFnZUCrAOZ;@zHudAwS1XDoLE3hv z0c*m@yrfC*u+>-Xd$a#_JuqJYUm_sGM^BiL=dgV5L~$>JN@GpLmdR;wshp^Tfe{l9 zv00)h@)G^hx2Il1*+gwbd_XDkucO@G6gKn8Nj|>PV$vH+F{5i+SgBbe8((scsPj8J zSH#W6FFOq?GS{oM^M*|l#d}R_i|{2fHda?l3g)D=rb^LHlYBaisf)f9$>Wf>#pyV~2GhPoD{F*!Hw6!Buum5wUX!fjGImiot|7C*1a%NLD{*`z%; zE17x853)s}ototQYU^H8X@ze!$4sg$Zs@5TQ?ukARF&h~7Y~B&8>y(Cl$MKjo|BeN zL&vE+BS%z>OgE~GS2J2NMpJAjtdY9#FU{7q=c6_0E%{8MIFd=8q!HT!sW<4*O#ao; zqEDooi>iiK51nXlU1E^1rBAWpT;helzoj^`C<1{)tJv4-43!udH*~BWtJI!N6o!|4 zre&7ZoQ?e0?BhC9Dhm5K5ov06tx?^r4PgT<87w-bNKv47qytG8#K1Nu7PZs@1*9hk=@63}%Pf0uc zG(|3P-j?5xN||0Q=Fh6BW76$swrsc3EjeClmYZB$4f3`9cU(Ki&$GAItA9Vdu-EF&DSWUfT-DrU z#b2VwsIYltvVpR0tyCQCBZummMs39vZMN=0pqE;i_>hdu zN(XV9|K)Q{Rk~7=?d$wL#U|Z*?5FrT9h)?|s=B(kida-a+D447y+1O)(JIhAPwmwDFm3N7=eJhl13e_#lwX46nRatSvXnrj(J&XAHtX(@D zJJ1PjHd>7J|L|0|$jS%v%-_h9>*B2hE=o(BG3Z={E-;s-Babu#6wVehJaV8O=Wy=j5OB6Q2$~ z5&A>H=Yk&wp1zKvNV0P{xPNXq@&ils2N_Q$Dqc)K?&nnKkAVFnp??YbSBK<4r$Mg07 zpDn%qC#&<_O)q{7_z>8kpZA=&DDq{0_fMS34&&yZgc4`m z{F6uGjGMPZamH;9>XGH`nsiDk?+NKm9-pd$=J>N+jWsU)iO_qwl<%AA$L$=Q{#KHH z2K3h!G~0hG{q6eOj7Po7s3iOPRs`37D40^zcR>4mC%x|aE{FsD84bO@UBUGSm*_8m z{ypetm+1Zfi_8D_p=bYMJpY3Im7!lj22xUa^*_1Y&&J@a-|Y%@)}IW0_Y%H$2_Fr9 z3hW#Q&i;4@IQygOPD%cs3Omyfx6{Cn2R|Kr7UIM9KNtElp!e}g`7%EKInE@l_#pjw zJlm(gm14WvH{*iOg#F>hW#_~6<94P%&-y(T{7l$+2K+2=wPB^`|4JDb{A}pgF)n`= zq#yUECG_V&zYX*sLC^KWxzO(hy}td!?Q=at{ZQyNce~!_1(l2XEle-YcGU}2IZIM{92-agGCv3z|8{0so}D^fw~k z<`xI#`&Ih!e78b;*e^W((ti#8UeM2h{o%0x4fNH}Gd_ocv)>&DeiQ8bdPC*98T>Bj zZvlVNctREbg+Cud&+`5N{kPEn0X_X+RfbcNKer;^HYI!)<9hfm{doNQKu`OX;J2l} zT|dsa9=?a2snFAo##kjiXbvk{J>&K|`0eR$*MAJoewV(+ zph$Z8r}X3cJ{cGOu-<2bvz;t3E;~P`AGhQELi`tS|K|VrM|oD5+eWPcJEGuE*cg-)`pyX~AD6^A zK0gKh9jS0`$LCq;IWMmS|Nls@xqgij|NR`GP14>8`{Q8ePuMvIcK!vup99FByTEUQ z9me@V*x~&9F7$sP-({0cB|WH(dAZg%9^=^>oc0d^zdQZycC-doDjt##&Nas=Mf=`Q z#h0ir=viNHf!~wKLYkO_PU=ZfOFnC9h~Q5-0!o!tuDt?Qohe3-))WS zp>_K4eD?(BdD@}IW1M@Z_bTaSj<<&vG;{48-2Mf|Wry>T%BUne`j#r!Yp z=UU?_NxvNQ8q<`bp68AF+wE}wLi;|yM1RgJv45#?=^McQ%_Vw2f0kZr75C?<68-bW zrC$;H#o&zFG7_c~=F`j*gZu6O-*CHkG9*SCSWevcBp`jnFVY@Rz_}1lJl>DWQs0;J?;XO3Ih>qclq5FTu`w z;G72=6zXhWdpGyf&#z>M{h~hftNkzZe%=-5*)3pab=cY7xbl4k{?kvb<^G@iq+Z{y z_J5;a>wlqd2LJVKY;K?VHU=+?=cXvH)<|w=5IE|e{L_bbUx*Yx9deEb)04*NV$XaUZC*AkrV zsug&TWLHV{HwN!p(9E^=bANn(k$w~K1EJS7bk~mrZwo#id{gkL;O)SV1m6t&MDWeQ z&j4=^em?jX;IqKD1iu=5EAX4awFdU`8ZP45B|Y(Thx*Wqd;e#>i|=5%hC%P+^{RpI zX3y6Q+{eeY1MgtZeY{mYD&J1Vo7toD;r86eN#{NuY0a$UeAoQ6!u@sL$9Q{tbiRi@ zcYmGtH}3vA_xZ{Fb$)>9JKLl4Q|-C?>wJW9_t*Joq-CyU^ zjJvvx<9-aHR>Ha!D-gNG-^OKCbzs^rH?*2MI)42QV{9NPiuk#CyyT8saHGZ@` zI)B@qyT8uo8h3x4Uu)d`b$+99_t*KY#@%1%cN%wpo!?{J{dN9;@!9t1-1qtJuk*)E z=l(i>+PM4c{5j+9uk#m;yT8s~G4B34f5W)@>--(#^X$>N##|-$*ZId8SGd2qhRcbFJN#++XK@eboJR?)#hyd!O^e?792vTywUP`|DhDt&;oeTyvz7`|DhDo09wM zTyv6=`|G@earf7GC*xXEDLKE=p1Z%!yBT+Xo$qSg{dL~QxclpT5998y^Zv%&U+062 zyT8s4Fy7f7ozJ)D?yvI^#@%1%qm8@2&Z~{Pzs_rnyT8sS8+U)5Pc!cRIzQa_5PNj~ zgFSbDogZ)9{dInlarf8xX~x}O=Vuysf1RIe-2HWap>g-u`K88>wnyhp^hZkWuk*R4 zbAO#*Yux>Hexq^s*ZHl+-CyT-8h3x4-(%eUb^d_y+4kt%uV1>q&L20O`|JE^w^LLETvq$G2+4D_vH5B`r|E3pxz1kU^`&r)?Xurej zI1QlZ`Kt0$lKy+xr=EP}LY>E%ZIX}fkM9>`hv(A1o)_o-%h&CZ`}#cchb-U7eZCXt zIm7$Vllwdp^?#Mw@>9uJ=>W8rK8;xddF-roCJ`6Htof(ix_b9(I8K zgA)BW;JWtb_WuIsKHYGU@6XUTD00pI{0hDYIO}~pxUMO9zB9lX&u78u&&J@qZnZl& zuUj1nPCI^IkmAYywSnnX?oNm&^QHe=Lcb&QzOR*i>V2G%{hgriXLh3g0C1kOO)@S! zJHyUl(5p^8&c_>%=fkJLPG{IT2fPdTmEc{$Zz4~B|B+r(x-&l%?Q<9C?=!CQ@;dKh z;JOCw{yzg=k*?u7SNWA>ryh8Nf@Z!9IP2@@q;WfaOfP#{!#f{c!n+w)zW2e;J;B-j zPXcEfgim>D3VJz3Drq_e65xD&g(|eVqr~g}lbKKy% zVmah{RbeHM!@Pd_SPAF;i+WwlP?DeI*C1{yA>S{-R|eNLcO{i;74X&4-%8Q``X#(& z32#@zhnMinOZY1#yg~hc_H(}yeqsrqSHgcQ;Wf+D<*PG!Zzrc3S3KDs*39xE&RjS0 z`WpLD4fN!^?$9vB#PeOw^s2A!sji)`1Usuk-w1pS@bzJ*2kbP5eog2bf;R$J-&B%6 zJz=M5L34X+485QGDsG%-yF$M<^jtrz1Mc&n^4%5rR>Kb0SI2^{3p=NPHvvD_xa{|W z{p+Bo{r|7y(;fcwhCjRykp3J9Kh>ALopAoz4R*L5+W`9iH*xC&`~QE4+wSn^e-pQ6 zh{Ht0lkLavZ&7<@`#B!<)ff3*1Usy+2H+bZ-_^idfOi4cnC$iI{X+h<0;iqk;NDNw z&N;tyGW+8F;C~MxiQMy z*7Wk9_s#AI-UfQ^Bie#{oRzQ6XuZ6{U}sb4y?;gh#${(9?7R#85$Q$Oe-5s-iu3Qm zIgk7Xz8UoErM#3>uD4*nxpDb32>$3yMoD_MbM<8<@ddCm1AKGjdoDOX*XHMGveO>= zo1mxtJB`cFebe6_hi9SZ{hx1wZvp$Cf^P}#=dALFd}(Evk~rHyf3-xfKJrg`-6Nr-IMe$OUp4ulBt6G}jdebH8tmtGovx-$k&) z`$7#DaoYs-wPJd|lI)Cy9Y04^oO$lFo5U$eUk!c#g64XmzS6k#-G2p^fj>m2lOoOa!I44@*WKRs>Y)~I#X2==Xf{? zc61HJ{W%O8mL>YW z&>sT*z!JU6qNImu(CaKzDf)8_^wXigwM6gdlY043=y@*3c_6DM86n%efGrlI)hS*sc;_!9jQ&>s!` zi6#2Sz!fX^^99&B26ozHU)F(O${z^u@f)xJ|}9eiHOY zl;~$ee=_vG-iZFZ0R1V@zfofUFX&H&emTsK^uI0U)6<~exNv=P{iS4=+`hF5t-)7+YY_fBmF)B&l3H(Ly9l?37LOa%73On@EuaC(7&amTpwJVmF z*AaI52fgf++3)%f`v1wk=5&vP`}059?+*Ri$q%KdZ=G?$dH%c|IOFN(@bdGr6m$2d z3i_VNx6Dso|6+VLgq_~7vjRBv&A|IWza}{Kn}YX+{y5_*7tbqBhMwm-yxgCCVCQJq9}GLZK05+j zdn>myuEfq%@Hb)qXxNzzJ8W0yLVqFjbD(FOslNgFGX773v%GJEvtGVIzE>b$?sNA- zdsrc9lvIytO%)68p<3Fe&Nbdgxa*9^>PmPuI&gK)-Q`p5Je9BlJ6$=y`6S z*m`++Zou;T^$3}}8Ttt&_K!5Khg+bZQKCN|`dgvbnWs{W&&|-^2L0V7`iG&{HA*k< zli;_*4$mJLhfkru1N#4#K-tqST5a*qNKO|3;Oj-^jdo;$qwg1U7Jvn zox5O1YdNKuuht$)(%%ieY$`?l0npzAy`OtT{lU=R3;m%b_D_cXKIqRb(a(YYe(0|+ z(ccTb#t^TUM@#fCLH{81Zk=P$HS)J91nYe zbN!%u0+m$nT)#CfXs&mzqx`&DdfqSU=g+Elu8WR^o!w9_|6UZ?c|HAjdFPs5{Autf zz`4I$HpN6q`Eot!^JcWOx#`7OUY$WHMcz~3AGoe1DJfsxhpB5uO3@#-ANpV4>LuoT zQ3>DLHjuLaXnK>!tq1sH;KNGz1mnt=^*96iGhk-{^sKM$wo#ECa=$JUd2gIcP~Q>z z@qN)we7_?*^U(gAq~}VC!+y}WEokQZgYN|161*3<#x#%TOz=VAmw+FS_*@G<82a15 zdEIIwaD7*p=et=6?_peVcm#161U=6gs=&FAQ(Y@5Zk5RQ>w;#^>yJxWzmR?i^liZJ zLLB;l>&)NFH2{1l?C5MjGX0-W7Zdd9u~V;#dQ&YkBUEu6ZdbZv1=%+lj8}xL(%+l%%gf zd3DC3B+hth?pBf;j05YP{iTihtB1$adp+NSjLSazoxby2Ne`S~?gVGPUzc#+-!c;A z(pjI9@*M@v_1hEhQ)O3*e)2jL$EE4ekB0pd!5N>cz{fy;hjA6}N%->w^o-l<&_4zJ z*Cl#iFUh|4dR{LpVx2?#8)80X{I>>w26p^DnHaY|rjPw)Z`gSjc6i@D`^zZk*yxX%X-8{6rD#WeN=ck{{w!$b9QSs#a!G$sYD-=&&QoK-eV&T-I~eiliTWJ@ z&id880ZNJ|>sQx~l?t97H`O00`8cMhtan|5Q$ef|W&3X= zT}slALmWC5H2bN0zddeaj7L514`=*&{>gc2Htg%33b%hN?6d!F49@eqUBG!R$9c9I z(70dU+0hmfhdZ^J?z1Jf4ahKfk#r^5y3@tHF7_^+Ir#>uTfj z^WX6E9_Sg*cfsjrD{%U`132^D3!MH>2d900E{5%m>sYoQuP>F0?e<>SZ-9Dy4V-cN z9DEY=ypMV^_{!E_R9{oTJAyOc-ND(;^?oJ!|2*nti-P8U$Iq*F1)qp~86U3C4}|{m zl!>?7vrG8xCA=9p=l2fa_rZQ|aO(FpuJW#ve%v39|Lnh4!VdS*Y`5&!yzawx)e&~+ zPp=X_82k{#c{(_+ryOTo<$3}2$bI6`h>!0RrRVwcT-fKiqK|vhf02GXZugexpD->x z&y$}o(Z2<~_E2vBqZ0jpq1TzY>whiL*H0QH`SUXLD;ZZmV0pP8`499PmFT%Xc?J6I zO7xnemE_NY^yB{fyf3b!3+D$wub4O=3QqmO;2%Tp`v{9?2fqkyeWP;_$b)tcz%d+ z>E}X!bcy~f=&ypF_c^c~-Ut0#(9bKe|1$I%6Fi=8l<41wz6121g7dn+kN=7%=TzaRAPLqEJkKM{Ja zD>N1<#qyp4z0RCHKIfF^uYmqT==Ir|f3mMNwvrweK>t{Y{$=Ptg8uCi{TI-G486V+ zNhy|h>D2a=^ztXrYc5lYdcS@x&g+<3lPk%N+P24S6WC$<)MrVQWQXmO=bT(umtCjU zH4Y`^%XPBOHkG3P+6yR2&vp3GCHk4rYc1sEy0AolHS{+?e@lt}A?O!E|5S->$4$M|mx{bwle&L#T3(DOc=fhGEp z(CgZe*Vlv+{gKe~K8G14`tzZ`7xphN(ccXH7tr5bqJI*4jp6S93nlvZpyz!Q3rqAr zL(lst_&H71@5;;6wFm7f+@HqAWBs;*Ue{t=zj=v%C+Hu7zGsPkALze=zOqC=9{R7L zKcqx|BJ|%tKeI$X3;O>;e@%)0F6h67{^1h+^U!|>{p%(APocjN`u~>b`FW6=py%g7 z*d7{blbn*q&zqrdYCN`w&7jvF*4x8&CHfxF-wJ)d68(YDYftL-N0sQOLa()%>yIkY zUk|>$DjMz zQK%QcUt0Pf;lJ-6<;)lTw?`P(9)uRe^*7LgR4DgFn=I-ZZCHw*7^0OxW zxc+JAS*|6{`QhiM(9=)u`{<{i@2R}>bCVP+CHbR0o%^%7ap^ZjzS~0o6ZBodX=m3G zJ9~ojdjrORAC7!il#?mR&r{Nm`?--|O7fHE9(t}M{m;p!>$fRr)*qT)bl%yx^uIvg zt3*EldbZDz(DR(+9O!?AeSV&T{#*n7BIx~`S>>btVd%97^m4IXQU5yhzeE3l@o3-M zmEyyFCC?LRf4O8+N%}uve^ukreiP_9ezqvlZv)PD&i2FWUxz|}B-+*a;Jp5&xk*WW zvL2rS=edFQZc5Vgd_ZkUNpaAbhS$ro0x88ftOb4){A>kIf3`HPhd*Jz8}uCa`hhcU z`+;*D-XH#-nDTJ{b!|vVZ__nm=lq-z$EB?#PD%Q|;3u#9a2()uRqfr}{(gm(95?zu z&A99{4(gjqiZkQD^Trd>o7}!%SCXB@@N&(G1G)hb}okf-N4z-*^hYL`3Tb|6BR4K4(|hDKe_<=6``MP zT;;kR{6Ek)1b?SQzYu!PQ$L(-F)n~7q7lxBA`lGy*^spNA8yc6M?V0oG z>d@~1J^l3gR4?nh=G>qCp{My3XF^YZ^j;-BY?yvr zuenu84~^0<=YD-tyfOI00x89KzGPevZPJh1c^`c3^tbcRj7K}aLBBriESu(fCB3{3 z?C`oL%ew{i>q5VsapgM$T-RKb#E(gTyPv$z_8923_g6}&q6zp3;Ol{(4*UFm$Sa^< zANm`hXSp7OUY{j&Kc9o1<@yTz*z~vS7lE@}4RKD`6!urgxgz~<4L#dWSMX-Avm5Nt zPu@?vA@swcXB@79{f(ga?}v%?`zq}4{!h90>j(^b^4upNn9>74(-wPy3HTzcKXBmgrxDzBTmx{uA17m&&7* zq!l`I@HjLPOi5hdQRbZIX_`Bn_b$}AJsgL2ew=aHX#+j)4`lzn2>Q0r&xL*JAA(-@ zR=YosL(lQ#ZRpvaeZM0=+riEXShp~4T<@{H`F){=lF0OILEzE0*MM_8e8PB=R%ngq<$V=;`mbwwO7fHA63=Ij zN54>AD@m`lf!ps6uFv2&=jZfifLB6)68Jdqb-|~AHvvD&xE{7jzdT=E>rs-QEHBrG zTSNbNiT*io*5e1@tjC|h8Bf1ILH@8@E2Oe0iL+dOe}c?ylYV)*cpn1GrSB$Ek{#w- zV_bH&g&p1}Ks&r2fc@Q;1I4(?bFfvEX-cxc9qczPX!eKa{M^6uTx(q^q|b3+?Je=g#%9r$_RJ;1qN7zBPM z^uxf<2cHB^z5A0~ujrV5d3;(!-w1YiKCC|Idah6C=K-*D5%hIPF{p&g+#| zf?o_ftglPJzlWaVo3Bq)UmV|jK2?0SM|}L8N_vgG9tXb;BK;2FT%SpMh+_>`XjC?l*XTDp)4)fg! zocZO>#&j$B-N9MYvU!E_=F|H?% zfgSFTRt0DMZfjije@;Jc|6u6Zel7)Pyj2L3_628ob+3_$`!oTw}p+gq`ET zSzix=-vRs2g46zo;B5cDfU~>}j4M97!k;$KbG@`9IQvU4aIQ~=8kc>ZyK-I1xIF=V zFZlB+?6ZCHdN=L({Zz`g1?+GfrX9Z@BesWj$d~aP2EHZgh2t~ph3lNVpuZe;SnoH0 z-wi#-ZS`euZy!TH1)TMM4|oF`_Y|Mrh!6Kuj1SL&cY~hmANGr07*|;@TodkI_8)@%-q6$jKycdk`<>+H!>}_JdfJ%=PCLA>X&&rw zA4fg+agRVh7xo_o_w~Q(SAE3m@gC^8{_*qtqF=y{pR1|7$0MF^!Vb%;&wMG#4*l0> zNt9%V=Z(KXPdm%W@sy&Sm5s~JWAM}WDbmx9uJI{FJ38xj&UVrjobmMgQe=nY;Q`Q- zj{#>KYK+VO$Kk)`N+sEU0$g*4Qq1=}aGtkZ3H~JP=nU5Fu>Wc<{wMxa34f)8zinLQ zqJ51aZlCq?6FB3ubn@Btw8Q&V$k!{;^ZLb8h{F!hKMlSsIL{~b858%1TxSmd#HSip z@%p4+-tP{FeuMP4^OM2pkIn?#{xi^P&F`FgjcLvqpZO*HjS~I=__OJ6_w#e(D%b8P z7w?DRyuTQBK(9TZ z^9qc^<6+0owPa@x=r@9%@!|cPdqS^kF-jW8G}d{(yBnANy}$>R=tqIS06&ie=eaBQ z`#cx*b2ItFcF6gZe6yrelAh-heF~a6&n38?=k;{%U#j8%9N3uvejhmNeE~S@U3*U@ z`N?zd#RdI4Z-}_DeX@QB!5_w-_K$^~7ok5Lobj9m&T*CTr$7Au2bT9Wv!nLIb6tJ* zQ%PKVb8l}u2&AO=^n;&jdrErPJN%6Ivcm+O6fE*sCM*g9{F zdi)E1`o38DQ__#?eH||Q^por42FUj+*rz{lqkienhtSiX267}NJ?xt_?$5@?^-bDPQ{eAvo={!2HN_k6n#N`+Go7`^SK@Uidu#jQ{J<)1Qr&DcmpWeV>%M z$G@L(mFr)K+dy#Ixe8o$?RK63r+%enbH1|wAJ}hgTyfBx?sm3^o_0=vp7nA%^o-lR z;FFQ>VsOr1TP#=SC+Dv&;IF{WWN`M!8Q}E)A>*;$pM;+NKL<`b?|?Jk?d^U}#hL4_ zeZgOa|Ge*XW$?MCm!GJ zXB^H2XTQ7IxXLvM<(daQ%k?qzY;Rvd&-z*fPJcSDQ0M>S$ag>R*HNxX;H;OK;QHSs zUN5u38MiyYwa0P&1K`yE0?ztcYsKup;>Pv!#>O{EoOULI^ZMh7;M6|`{wDnV5d2ZZ zZ5cc0RK7f)Tp9c^=v#u*PIqwH=?hLfr-O67cd>EBrxWtM8hl6a8)1j*|NFqXK70=R zEyU-2@VCJ~1Lt~j{gsk~DalWc|1H6vPOo`8X$Sr~_#WWA4t|z#)eHT944n1;8hA(e z|2gdQym~z?UQ?3&cc5@5%QgCT;<&#{BYFwce=nn*c3VODa4b~{!FUHAA==TO6 z0$yc2_TOpXY;PCA4(o9a^o;Wz;1f}gkAO4&_1CQPe<=KHVm$iU0i5$%ci8C+J9~n2 zUKJ+Lzj_8)?tc0K^tnXtF3pTKFq z_)xL6fAR)v<$6*4$=3vDJvJ}l+m!GcaITY&0Oz{&5^(1Gp79i#CB~Ugp=bO26P*3M zaihBOvcDf=T=w@uf1d$8?>9RWoPN#*=RWRkaPH$?0;l~$8fQP{&v3Md(~Zaeayj&e zruTSzcprQO>?{QT8Tys%gAmbv3-Iq?XB%*iC;Nf(K8nM^SueMOvtFJC=Y29Sf%CqI zkHLAp?<;W5r@w;#igNu0&id`LPF=lloa|*>abr6?1^O!3zZ80o^Yfq|3H@s&`uD(D zuHV6R7Ub=rp?%;)el7st$auh zMb~`Y&&`d?{%Ew5F3@wFIRpBa;Lo|xZw33efqw!1AnbHRzE480YbEaIt0nsPO7vfW zGY)^1@Ku}E#fSRMj4M96_rv|!6MFW?5#a1!UxU;B;u5}bv+R%ZWxw0pc$_affwSHA z24}uUmhgMQY3FHh+WEw|{5%Np`5t<!!Tz-i}0k?LT8YnW)e;FSqk1^t7`8oOXT$=X!Oc=E-nMiVxST{lR&TIvl(T z^;->oJp4QXoa?u1zz>3*yTCb4J`7I#FB(r-E^$8mHTVSh^C#>uZVg%}3c2m*-Y}1I z6XS{x^_zinJ=__b*G~t4bNw(Foc-c{<0_Yq#@(OCq5ra89e)}6gW>aD0sjGZx`H!bI}R-D z{{+1)AO4mv^}qZt^o+wI=*!|j{qO&m`ak~{dOP+g%B6e!ynZ(-i4Xl;s(zh*;}ZQ5 z;4h-RY0sjh{$3AuZY^l$yx;Gw63+Y27s5X8ccPswm&%5eFL`Hh_D64r$#lh3*thL| z;XlV8>6B!DS(GdNStN0N7Rlpmmy3(B`B~HG!RuXP_VelIKs$eNf5cx%Kc4JZsIVh$ zmmw;=zNCLH{j_b6-(lUo(EmH5;&(gy+w*<9R|Z=L{=v=}>=5{B26_hGW#0_O1U}<{ z42}wXPL~W$4g4^3u`?--fEYue=+cTw#)pzz*{!Y{HwrQS-Jf4 z5$?~{?X!M)8#tVwZ0)vD;MLaATLkW7!IptH8kp^O4*dE7nfDF6oz=UpMJl;JzgYj$ zUXT3wG-oJjFQDXlzyC&KBl#v4w{rtO+6Ig(0^i!!Lw?_#+n-_jhl2hLg&vk8%{Kw7n^P>Vk+uG*?fnQ|{rKba*-aFgT zy=O{Z-eb+a&J4+)-zPuUT9kb79{KrifiEz=VVYa1Kg8OP|34ge zz)#sS^NxXkY323*-{kf;@0a!6gZ_h#nfv#Dy1t9`$326-tIZ?+|DIg$-!FM!(6_TV z92EH4*3PE}zM;*({y7V`zk)5aX9j&+tFKD~ud(_4y1=h%mCJiq;5F86Ukv* z;OT>}=?MWx88xjzI)Dh zVBiy)Wj;Re4Q;%d9{5eR{yaW%yRUI(;5%5HFAn^=Rdc>q1%7Um%x?{RzuuYOANWFB zzx@*Uf4XOV$K~>)*VpD2haQ2?uyy|4fp@a@Rug!;PTB6^fuCdRnNtGqvwhZI5crub zGWYS@{p@1x>b9V7YH{}au3i6Yw`^yA&~IV>yb<^#=Km*wPivm-d>{B_%V)kg@a-*b z4efRJXMmM=oxmS5{Z@g0VdF;Uz!&$=`R*Qg|NS!GFYtqHeKjiZ$)=wi_&D3Q9uxSn z7Kbwe-^%vCmjvEz!|czsfuCdRmVX6)M~|%c?>F|it!@45g`mII+TlBacd!EhzyI9r zY-{_ApM!pi`CqR=&foP5ZGWj&P^)(LF_A7%FqbO`)xThH_eyt&1vf8g&} zKRq(=S!-s$j}QDf>ldd6{*3jb3j!Z!alSrqzfb7yz>l-_&*Oo2?Uns`Iq#Mu@zpMcGZ{}yiz^kpDtR8sNHu?SQ1b(uO+gk*F zU%RZ|F7Vz~FFOW){%%>nOW?ixWZo`T6wY-%s&O(2rXs>t74}a_bi# z1%8;VTfPl^q_u}X1OIUUY<~soH+otA>}`HF4*ZSvvc5&&yV|~Ti@=xKE$ep*yo>L5 z0`F+!^Ps@}ez=i=FWfBKpAvXeE9X&xd%mX!exdcRivz#h)~nY9e#$O6-#Y{U!P@5| zfnR9-^!dQsw#|0l4!nnr+n)tK(aQT%;OpD`wRFR5&+BnUv;2IOz+2h*&w7DBVeNdA zz(2D5whi3x|K26=mbMPtBk-A>vp<6aA2cZQg92}0{u~l`h0SNj2mXq!H_i%tqV@M# zfxl(z!y5vB($=N-1U}K~@rl4Uu=V`Qf&XRmyMLa=>&3^TuY&$!TlX#s{4<;Pm$h-u z?dUViN^1oEiS1jO1%B9pS?`~JaXV{u$$a~u|8URDdj{U4b>{s8_w~=vz9y|2pv1ZC&w0;BVP_>9@eIwt8IM z=6^3&jrG&D1Fy7mybS{1#^Tu`@I!6hZyR_&8xMC0-0z?075Msna(VX%{6^!W13#i) z){hT-bDMuB2i~h*)*l-9x_e}Pbl|?;Ju&bzt$m&r_^B5Ea{@oh_AfUC{(!aT+X8>u z+ST2G-_a`j`C#DPtv@~%_={%$nZTDaJ1+!&n9Ucj2Hw=#=i7k~?3w-fFz^#C--Ur+ zWaGovfj_W&w(~>a-P>jEpLg?idrjxe7YF^Qewi<012L9|O+0KB#SGDKk0>9Ye z<)7E{_#AERb9&HE+a}vTD)4$%zb6F#RKu)4E$|c8&3ta)Z`9BH`oMp-espW#$C^KP z2mY3|!xsYI&)UPQf$z{Q=lgcxuXWA*!@%#e_3FaF-?Mh|P2h`teP#P6kMkF{-fI~6 z`WFAy1HWR+?9bYP-`qFzrhyN(bBXN&Z)?wY4E$I-C)p)%zpmLU@c-DnvPaxP@7j9YF7tH*-_6#ytpoqj_IKL`ev|dwRUKse!oio2W@Ok@YerMpm-hDLi{Wj0~7X!c4 z=8^XT_s;`-75M8G|K9>X%hqqpTe-cy=2|~(6!?ByWPe%&zLM>awha6pn=d*CzKxyR z^bLHx**_rgYnRRW9u)YfwKDh59lQU(Y?=9qLBE~NYv%@jy{*Hq2z<2zvz=Q5Kgjx{ zf4Yl3(<$)97Kh$}zh>k1puneE{6_|Ux3#y)fuGwf=X-46m#Tm%of-I4i^HtI z7uvXbW8hP4-Fsi)jjbM^4!nuA!`A}eyjkH+#m9kn-YoO)1FyGL<`uT@_xjqw;;?ey zkK6P00`F(zaGSutu=?!~_)nd4zC8l(ZR^bbfls$_X;|P-TYIPpd`%mNj|}`R8wXAc z{1l7B#ew&+_1Lw6x3O~F75IJD9z0GS&+GQd<$XEmTU&qpAn^ZKd-x{sZ>%2w2z+hZ z->uL%=i~XFzf*p`cHoEZo_VXlk8Pj%)`7p*IrFZ8pKAW^5%`X_&+_}A-Jg|hy;mLd zjct55H1IbEobzb26S)88_e8`5`pLYVEV&nf8 zfp2f;SHA|{sxsSI*48^7pYJlRSU2#sZT;LL@PoI`Z`d+$*LMzl%LB5$Z{Tm(zH3#d=9C&Zr7aS3|-}iD#;EQa0z98`1x6A&_4Sax&&$k7>qm8Q%1-_MyKl1~>*5dX? z;O`C0`F;|)jgQ|4ey{D176-na?K>LUI>PJmew)YF3EaQ;xmDl`?fF)LziaX79QZqS zj@YuLQ} zK;Uz19s5k+R_zt92mbx)Ip1#s_xmFL4BYRlS;593_uuc^X&kuUpVK06zn^EPz|Y@4 z`{Un};Pw~olX>r;*JtFE_6&TpWilTe__@}9#|G~GZd%|z4*0o|`}1t?Y=2hJ?_%Tm z4S~OD`^|d-pKI&nCjuX8{r%;@*EIe6fgf-E=&Qibv3wT={*bLpmur&C;pOuCLe>g= zu(i+Tf%n-UzkiFs&$4yM&Vi4zaenu}FSqsT{(&!N=VSi;5ANqgYnO)v{giF9KPLo! zx9yA034AMSSN^>aZhv;CZ0DAsf6?02gMqiR{y0DIPi!6aX5fB(XJO#2?Ogh&z-P9} z{`mJ!xSyZe_^_JI&(5!~`feKdeP*X!;M44UV~4<7+Bn=RaQ~jrfq}1M>p=fr3-{-H zYqyhvepRdAV*)>+GM9H|;QMA=aarK~%+3vg58Wug;pf1|Y?JxY>*q)Jv$4$=D+li9 zT}=Xi-Qw9ga9_V}6Zm?z{^=L^9airH0^i2YH~f1x+|Rx2TyAvGH?Z~I)W9#bb;E$U>?*+b1{cPvUz;A7q`LBV0zJ2D) z*f`|z*~Hcxs|UWZ?F*U){*&o94SdhW*?z~shuJvaBk-L&XZ_xR`}yaPzz&FFt z+uoTU8u(i+GM^Fn#HN}1_mg-We0?}4=vT7#c5~p@4$5}!5Bx*xFHZ+vW9`|$x5V>p zWaIpTp#Rm@UH=Wd{!TgHKLVfHDf0$aKDR%kS>}xb?|VSz%>zHu@@*gZ1}(FG$H4t_ zgS`X)&h||M1Mg|^@$W@(KR>W}>foUFdHl%0eO+;C;5~Y0KQ9XWrH+|j6ZisaKmQ86 zrS+G`0$F+^bGdTO%pjm!&{X|=bG!ESF!)Y1# zF&2ld0{8oHx&-d`?eq)0yUnkI1K-catFeI(+O?uc(*u9DbLQS2yx zW#9{XWjnV8uJ8CzdbnBfPfxm3y@b7^?XX}Tht$%yD zHnBLY8~DdNW&6zn?{Dp=Rp9qq9NGo`oUI%E`*S>BKTqr)^ge&>6}azzD+BlaY<1wi zpPe4~@-~j07`U&C&I#Pd+nWOa(#GvO0`Fn*zc=uQY=8Z5;NR|*=E!Z|^m5=Y?wI|5Bk=b7Xa0WRT{EuuGVrR-`Gr3N-?(??^_u5L zuV3HiHwgSU^K+HJ2R6&^Un}tIWH6--0>9Ymy?Nm0?VI)OjW3m+Ze1@k{q7j_hgf-b z3w%S1|ER#%wsq-@!2eh)=i%SK?@-r4qvvjgAF+QaREUuFH*zaPo{yrNmQ|9#NkWb@AQ)}LJ8-1>FXz_+#W zZJWS{*f`cF@CR&usSdn@`FUF4JFJuayesgVt-jt4{3~0B{~oyCr?PshY|qPeb-nz2 zlfWPDn|bHJAGLX8VBqIk$D0)RTwAZ68+fmMvi<7=|7?fM9|`=KW|@B&_~q74ehBe`EPpTWsurKdfj@2Y(N=-qZ|5Gp0)J}HY=8g22Mx%4T;Tgz`#d@DA8cQ6dEnD* z-FsKy{{0vK4*USy?|&5dh8CB4t+QVqhxXPEn+HD3)-yW?KFi{>f8c*vJD(bOjqPtQ z3H*Vc*^dVTpKtT{>w)iM{qyUc@uLB>uYvwDp$&X&HV;kn@?E?Q~m&|t!ysxd> zD+6D~@;fZ>Io3Ys1pbi4?a{#f9OS*g?=^pZ3;Y&q=c~0{vRr4cke_cE_=z^Y?G<>n zjkhBL-`Vn=8u)z6_q4#9SUZ^&_-`GuKmQ8c|IhBzf!}NM@B4xKxc_tD+nPVC+qk9T zKc9XkTmNbk_)D8*fA$Uhnf)@K9{5>Szvl%$r&rehEAR!@ZeI-i92-YI4ZMDjZ0Fa& z*V;1kHQMDzFV`X6^7FQVe`o94T>`(x`rW>P`#I*=z<;xG=D5H=^7;*Y#ERLU*8`t# z{{I>H0E=gHs|PRFznW(|n+CpR@65Lk{5E^uC-CjX0?LQd!sy4s88MuF+`?rDLYxDBjw*K`vY;Wt>Hi7?S{j^iy&27Bx8~6`4?hOh2 z}36SwZOmC z0zRdU13%8jg`EQL*Cp$F2fnrWQ5E=Y7N27Rf70T4LEsnm&-QN#{JTDxKN0wvHt)U@ z_!IrI{@1|IT0Qf1w#bhjpB6TcZyxx4>t$V+z@OPM^I?Ivus9qW_zexR{*=Iv+cxv7 z0&i*KfZspjeonA;)CWQTg4Ng0fgf!7Hnj1>?YwE@$VP!bx@Pug`@mmXCG$RkuWS8o zRN$K}o%PcLA8GNN8Tg~tuI>%|CR?Ap5_nguuWtjNWb1)Nfp2Hy(kgaN=5e^u{BIR_ zg^dGU0{8o4_6~fXy>odF3Ve5)r%nibz&2TbLEx{-v6OBO{9|h;j|YBJo2-8^@WnR% ze--#^7Uy3BU#4ERvx=45<8zD6zqg}{A%`QO0Lw05|R*J(qXy$gN�$^<_8$y<2dm%r0&l-p)-MWt`8_gU%i58b_f(tTw+-AsZ{H{IgDSI~ zQGxd~J|pm1gR=g{z&kb0{IS5_u=UtmfuCgS(ys$Q!OlGv2kzGq*Rt`_%lq2;Ip4N{ ze`4)nx4l=UYEKFY@1*@4gOoAvVo_xnrU4*XPG-~Jr<3cF=H zt69JC^7e0<`KEy%y+-B(1OMLI?Zm)G+B|!5;5RLo?K~Lx^hTM#7r0;N{wwg4ZQfkd z>eKyilFd|X9r#Q;ALt+W=k|PZ;N$F^;k>|?wf28o;I~@8elGA2Y+hLq_V_Z_W5-$*W0$9X&v}2Hc$EW0oPx!a(=(x7vTKcWiuZg?EKr}b9CSrS^K{r@FC{^ zO@XhtcDDcTz?;~*@#DZdSUXu9_z)YH8reAQ{?D{|XN$laoBurnAGCY+=fJ?%w0WT> z@TIK2P7Hj3wa-ffuW#eb9fAMWGw1tk;7x3O`C;Ijt(o<|20n1D%vZL2JPtQmKW!EG z#{;vzL*UmhpLxH)+c(L)D)52U&QA#ZMH}BP3H-19vz?m)pJDyz*}&V_yyl;;@^T$y z`PSbt+jCyAZ+^aR;3r%AX&?9l`(%BO!0)hmZEWBdnV%;HzS;rVj(?wu`?IRm*R4U{ zZO^QKGVr;!9)2V6Ob{`g7YD_MV8ac3)cdU!tld|>mVf8UJ9Vc;Ih*;%PI~Hd}HHB8CSXFk4;Avml}8f53}}mg>mU0Pd}a>?VSH||9jXA zPX=zoe8qflwJZ1kHE{VqJpCxW)wy^m$}2uNKdpG*xcg(>wBlFrWsubq@o8q` z*yNyZs5ho`HuTS>=j+(H!CY|pf4}Lk4g7B7HyPJc`EyMAQF=1)OEa$cH1Io({}A}L zOXU}qvId~1vj3jxR}Q?D>6--J&G;t9y&iYA`C@D6A2r*0=nuW(;OFgw0)NTu4+dBK z|J+el(*HyQmz|2G^{b+yI&l9!_k+P@$3ItcIJoR=WOj}Yd_ChQfXmKNgVL{*&IFg8 zZl?F^rQRQRGCnuxM;gC2@FB);0$0APTO95ISH35i{(-=cHU0>=?D+j5Z-UFt)uw+p z@VUkp7*{ntntnD&KT1t^$&b$4=BE|g825Ivt_FjYIz#_RM#b+T(956uE#DD=-(h?V zIO}&Zxa=%2{j|W}F@6NN>}+oH(@Eg6Q@>vNmC|W}|7CVA4ZM}8&|!n9&H&?v8J6vIJe=U zqK$F)a}!(7YylozHj^M z@1cL#Z0KS2?)iz2WB$GTjRJ33KkHfq-ru-?{?+aDHQo+-#=ir&;^yCv-zo4Kv(p8f z@$Um(v>(&&5%}504-EVk<9;8#`+t-1QKnb?6*rIn8G(EJ=fb}HoM#KQTcCd^aZfKm zFaIC6eEoa1-Jb`He;D*%8vi8lPmF&7&iMZXu6$QoCVR9f@a2sE1`QxA8xgA{j&NcmAfzL92KRE00NpRVD()7;;KF|0I;H<~Dz-8xC z)4vz^0^?r=?%(78ZQy^J{wL$AN5$W^j}^^rAMAXi^qNvf<7!v(^He(z><0Y<5?UGo zz5HKu+5E!jz#AGL2QK^G-==}f&JLzOJn(Id9|JBseqZ-#;IcE=^k)V>(76Af951i0 z17`>QQKp|8xc~ov>%dvRcY@2G%gxR`fnQ?$$-wV1{%qiP8GpgJ>QVLL_4{YwUcYPX znnWpic~|RPJgwNkxa{1YQSo~x=;i-^EZ?qyKViHlxb|y{me0oa1(%)Wm&<%`;L92x zX59U;W6_EUfj7@)Do%hM<=epaiRXYT-^OPDg22}@erd4Z-t6BIxE-fg%nQ7S>0gI^ zwTH*-yk!CO_nB4?OZUo8d_3RO{8={e7RFZumwn%VH3C<8{r^3z8~E;KrzyDXc>8Yy zE<009zggfDjQi(ryj}S^ZKt3=&-DJi@?PGvjPDWjHyiIC_>IQ*HLiA|c=|kbbl^Tu zoew)I@56R3?e}@f&Ybk)=@IA^hi5F`Cj!6U__N@wmsi1MXW8X*4sQnjhuL`#ob~cK zxa{m;cD@dLbK~EG%Z}H}pWw3N|2JZ(-q|m&mxIjCDuMg|fmkE(S*BmxxW}_uHe1mS zdiisC)BHjQaQX9++36Jczm0bZ_FI{K|3508?+(Tf3wnE6aaQ05nEtlFM;d=9@I#Eh zVtj@4T;-a(b$;P}=r1>|9u`Bd@_ujSs<&Gf$&UD^##ak`(*{|$R^ZKyuL~}J-p}%i z7RD7HEz$h{{Iv`EUF|w-2k`kxKg8^J3Ve|9F2-eF<@J8IZ{T)Wwc_BwJEZq0og8@2 zj4Q4Qe1!3b1HaAqE8sdRZEb#j2+lZs1Fj|1?Y4hh+SWlTm-yAj8-lC6KBlh|_`LL* zl7DZjx087?m{L#ZXJu6U9%bCy!?EW7*uee&h}3|q-u-)B4g;T`_+@72sK75We!OwH zCx7fXq2glbW&c_0Ftee*EOAfspjW>0%>Ltn-)a2i!2dA*THxOp_v_X2Q~9oE880%u z^LECU-97)|d>7+u1a6mwD>gCi^)-0M{D!Teztn8#p+EF0Z%4f`CBHuG^|-n5kwJfu z@q+>%VtfL);(v)PMEv`x-2P>zKN5QRbC&Vrz-6a%|MV-RGr*;P-SlS%{(|ufj4Pgs zgB{;i+z7qw-??V`mD1hdvcK5u+#mQa#^(k5V+t$jd_CwlUNOJ0An^5#FAR2$Hako8 z&3`C6`Ll=VmkGSD@fD1FdCxIBZ2})^ylddM8Xp*VTjL`F?__+saryat`nk;RvpF96 zOH8YWYl40i+W_1Z_?i3VH$4J7gJI`G=oOz)7Kcv)A8h;!@aL1>_uW5%%g(u`UljP6 z#+S3}ie4`_8DBB*8;q}NT=gRVeIK+*;N8<}N<9M~Qqc5X*q8r{>^j4<|uC zKRtic{5dUf|K8?{1OL|avjX@3k8)$+{(ntw4ZKmq?9@YncQ*cL;2n%VWn6JnxsJ+a zDn1GNF{b}K@G9fq1UsjjomDJ;ZhyA%jRW_7v2);$nBM;ns@r+N_{5-J+xRJgw>N&d zaqmZecFQkZ4}G<1_3$k8s^4R*yw3-IgzY+|LV_PH#%-;Gkb_rOc-UzS!&>3NC+s>7Df_fXkouX6NL{(l7hL8jG1JzJ0{KJtHr<-1(qLyR{BXT3BA zmz|?b-z0GVzctOkWyk-2R$Fk{xzy}z9{BmjcMSYK<6Q#x&&&5Pu6QaAD!0;b=;hDC zb_K2&552Du^vW5~%g>iA-?IaM%J>D~jN299vQuwmSx@_bz<)P8H-O8I|KGj4z-4C( zvvXhI8ykPvxZw z^!{6|Jr6Oi`g=b8%(VE7gno45o=${b@tJ1+pAz_F<1@i!-^ZVe!DZ(h)6WWgrtzBs zzumZh50m#h|35ts2fhE_oyP+A|5Na^am88X_4fQ(;EmI3N`JwQ%C)?WhxKipE<1iE zA$<$u?&qiGf9t>(7;gtI`yRIr;ELN)tH^rVw+8;F+1V}dW~T2Kxc^_D!GZh#;~5%w z7qc_cxR=-Cc4FWjw@YD1<@$a=;eN#x(EFKzUU?LHmAAj;`()t$e}LwLGj6Ye%g%)V zL)Tq^NmYJ-0KYUW-Q6i5DP1ew-MMs2Hz+9xNS6YFgmi;QN(mAIN{4`yAfSMRfWZGf z^M23&clYV|BZ) zL!GD?x6GT2SLZDYxz1a0xa4G9E2Fv#w}hK~M^B;42$Prb?!$W9k?)50A-@VAKz;!} z6fXVCdgYiR%M9c4I!s~o`HiU~{dd9>(bXp3UJ772X7bX%{{NnnpV%(omJ}x6+*9bXC|tL@{$HPxg0{!waM$j z8yJ`QjQ^frd4zG9x8BI>`+jLZAHh2#FXNT|arh5DHFc!^6#O7(qRF>)uFGnZm-Ws> z{q^Mf`?R~s_5GRmksm{yL&l{aect+uaanIdJOH01j}N~|o(+D7JRSTQc?0-Ma{d1| zZ;c;w{XCM1bxX`f{hRKrzX8Vs-9Kkxo}-fM|LKWkT=oa; zCy8;{AC93;3i5sMw8o{5_MgMJ)Omw^ZgTxUI{D!`F6_2JEpptAf-c78IFkU!$-c&A zehRt|StgP<@f@r$F6%rQ%iplgxYTKb{LkdQ;rES8Kl#wlGviX{Q{QG9?Oe(49O5yti??pA0iDbs8f-g1jdD zQ{#3&nQC0>j7MJI?@IUQ(eN*kmwA&fbwBxzT=$bx#_fLct8w{K_mijO72R#J#KoOW z*NfAYASZcG9}g-SmmZg6ziMP$R;BwxJ96D0dXek?Fv+-#D-rg$1#sOT_T%SkD@`7| zWT?}f`Pn{9#=B)Zb%#VqE%3c^1zO zPitK2#KCxTz;(Qt!~HjuBG<>|s^nF0($R=~S#4jZ9l5@*buaQ5<$ZnzxxP=?cjPay z-ecs;@VxRIT;}0RUpjR8+~j4u9Krm*BG><$6*?(#Ih~)*IM0cV_3QoX7V5+&{|%m& zJaS^+R7UcDkfF`5AZ~^2hK3N$d|!alkbEdBCiR* zOr8b)io7H|RZIVc_JdeZfcymV;cz`Z{9D7{P|@V^R4;Tx2a}iW`#$DJ|K3!OCs}nq zDc?-zle`{W-xopajDb(4{0R6A<8r^Njq6Lu^*FzoT#xey$n`jX9xn4K^HAA+$a3A} zW&a;50k=FhF7q=VeLf?f3)lZkrsG`=kJ2hsE_B?MIxFGn$o2oXWg_2>e0I3@uiNqi z@`-+RL4ER>@RslJ!Eo*W4A%QG`6>8V@|*Apm(m-tEjH@qZ|4Nn^3Z`b+B4o^v*6`l#M{Xg{I9)wfA81m)G z3&E>Uoml9#!n^J|6974d;uO8*1%;RWIk7- zpIydfKI@_Xz2tS^N66d2kCR8he}n6|w4Zw>Fa7L9KhKRzKfO`s6?sp1)YdNK7Tvxh z;W5dF!_&iMbq(CdQp`_o^6%lr$WOx?lIwA)A9>?Q(p?aYB<~ELOgiEDay-9`*Mmx#e0}G-L{PqH5r0cpfA(~m*o0>Ac@<$qHfXc-U}x>DdDo-dZC2BC}#39 z|No$VDe}kgvc_eaYsbO5|=>I{iL!KnL&o_i?Kl;3~o5@T4-*CP@@j0$jYZ{lwXMaJ(I^K3Vm1XcK)07_|_Et z#3!!`Phwp9(bt*N8<+7eL_Rb59C!}nQs)+42qKsJA2>CvEN#k<=y@LC11>;ix zj{A_M4tf0a-rJJL#m`N8kaunB^F!gXI+@Q*o`dPe1`x2Kv|A^|Z$1yH_;%Q@$5zc+SPp*9yH7?unnr<)h z%kYZEWqawk>XPfYTETT(-F*8&f8)`dMV1S~iOH7-Nazk6+9Jaqp;z6;gYarGzHag8I_7Z z?elB6_BkBu{m!`b`4Q%Szj5hv9`fg@zV>;OT>E@Ru6;)CETCu6_0-*FJ~AwNE|IpJ7~D)c=dOz_|4J9DQ!4 z`gf#Bx9lO;K7S$CKCh5#pZDO}XN<_ce&o*nv$*uR5c3(=xb&F~`Al%#UU&Sr2YJY~ z&v0_>vpTu<*#xeA7RGve8kZK=q0b@4rO#@}PoetS=R9)lb1k{{xt(15JOtN1n_<0I zj7y6L(C1y_(&qr=-%x$+GpvgXxkcw$`%FWwedZw7J`2IM&#_oq_F8T8r6xb&&- zSJRE^Yo7zjwa@Y7+UG2C?Q=0)``nB^w;PuhZ=ug0j7y*2A%BtTYoE8twa-V!<#C}e zj{i@|KZ3tCZXXw7cXi+FmN54ruH#Ap*Kz%bKJyxvAw`bvF5FVwxQy!<^0lbGj;k5D zj;n+5qpptrAEhqjGcc~c#_hOEt8DPxC&rgUl^BhRYiU+)z|H{ zom|KDgK-&GSW0){mc!)H;irt-aa|?XaovaOxSFERDBb;f#ARF+F|K&VWn68L&kX17 zm6u${Rou9YD+l^7O`Zu}*|;57eR3UFYq*Z9H~JiCT*lQJ;~HgL#x)rEIaFV_*Aj9a z*E-`et{Uin6L}^0F5`AwhskwZXW=@o@#yn`aT(VDjO!ocGOlCD$Nk7%aSLy+l;k?D zEXHMAJ<)#-@^0_~#*e$aK7N%Z*Kt*Y>$onX&o;(oTw^eBhKu6>Rp*FI;$wa>v=?`q@H=TY>z)wuLI z8~I~YU;8{yu6_PNu6;fu*FK}*1sv^j8P=Q1xb%4geP%T-ecnO7I9#`*_F0i!`>bPJ z?spS#|7t)!7T(JEahKO|btTtv4TS5s{z0FUjmxIQYu6^!H!i*FG;}y_Jj$RY0G0jZ2@eknc$Kwa-4}+UFQ@?Q<%* z_W2cD`;3kI%NFC(N^A7F$GG&F1Nk#lU;F%xT>Jc+T>E@Wu6@So<5uPt?XxJ>o7uSZ zIShT~F)n@T`+${$>-NseKAlE)uk!zn@$+gb|aP9L6)_cjg^!X$Dylq_i zObmZT^|j9!{anZ`I?vi?N^l>Fo%fLHReeJU!x%N4h zT>G3(u6-_qYoE2S-mS)^&xi1R#--0$@N-mO`@BJ}eg0`&&Rf30`N?DQMeu)(+w+!~ z{oOaaMYorZD=A#Z^$q&WZ9IYdzL;@B-yRe(F5^0Ze08d?<7!N<<7#VM#&rPwcO>5f z?`7PMYZ$qXYdl=XrSFUTwQ<>AVKM!I<**FM`~y~B)4pM~J#j7y&r z;Pa`z_PK&w``kgUeI6p$K2O87&$(FdJ>#-J=fnQ|%((RV3-YlCx+`ws`Akl(eZEJo zeHJ3uKFh+j&+Axk6XVinWAxe1xbzt#mUI^cgQ&jF=NNMBb2hp5xtLu0To2bilViPy zjLUYfg8qLoE`8pXCf)M8amnlNEkqp@y7~Vf$1BN-+>({N0$#@`MScigmAp3k>}Xu( zb2#R67+kmS<07He2BS>AmZ#9=Vw0ES)s2+?#%1J};H!cmgweFu3o z_#Wd@=ccT0%Djn}fX6j% zkIyN|b-&66m;R;CT<$}bQpROJIg9OE*|_BMO4uzesJ@P?Gr5kduW{+WR;o~$U?6!l z_(N^&Y}{Pm=!x{~NCJnXQSx;f2Z9L{>hf z8S1ZSK1XVQJr{Xa8Jt`4!)3eF2#LR_Px*G(5!;dvP2=+)nL5=?ok=Dy`_)+NKhwxR zfiE#G`<4E`(oIxf@9%ra>tNki$VbBOQ~l?dpA^GwAjE_JRT-;Deh_(#U={0xI@pE-T`U_7}VPZp6kM1C(ht_COY zh0f1~g8myWn!MiU({^bykx93jdDkbVU7AXbu%7kM%GKDf-Mj4Q%@$a0PHosqvu-VuJ6>fpze!83Af)*#;S&}u^G+p^xN{%Vk! z9EYo*1i9`fEy*{d&H&@G-5X=OkAv&>{XK{OhUq3>%~R-d6I}8#pC4nqTgm&wcN&-N za-^!eaLb|Lp--XbaZ+bB@<+*+z)u>NI`@*e3%6Vv9{TkEbfq(>pfP#< zP<8kBXyg;%qse96vP^^PcnieyKHub5dJ0|cGI^PYXleb8d&ys89!`*#K>ieYF8EvW zQSc}@!H_;>y~E%!;gOt4pOxK*EJ?|Gcn&g=kAr_mz8BtvT<5I=`7Px8kiUYDfa^TZ z#|gnilW&Bqd|GYtGTzk~?|Sk@@GZur$9vhOiu=7#<5K4W@(0Mzzz-XjI^82nl^{56 zTHrB2QaqCxNhd8P~@4vHCzv>TE0EF5J@1xYRkFLDc=87x}RaArAT(mpbP%dmnCG>O{hRHJbcI2A}`T zxYU{Wf%h54rA{8ynM0lpKHs?1IfxU%<;JB>Bji_;*Mn~-?*-pY-W|T*xU9Ed26y3> z>&B(dZsc#1e+$27TbQwxYWssd?xa2@a)E=PKR=Sy#*qAH2j=#nNNMa>$-7S?>6LblW%}OA-@U#hx`isHC(1~ zr2D|>Ly+iGcf~E@^8D+ccyi4>uQe|DiXnB!Tk_`cI^=EOEy-uWyOMtk9|YI!wXvnY zVYJC_Mpiy8F?pG{%s3t{Cx4B3SYuq~Utcf%&bZV$ggQIPx4@5)r_JKN$t@?zlfus! zx7+=iaanJ5m_#-+~7Z0^D>F~|DP`nVr2tG^zH zJSse)ajB#450=Kb^b>)62J-UoJmk0G1;|grix`*nmTuwuENfiWI|SpZNL~$I&A8Ng z9o5%qXk6-a$?m-=d3AUz{l{B^`pBBx1=;K z;|j<5Ll)yQuAAsT2l+L4Ve&AX4;LqYiF_I3vfi3y+=W}J8khCfK%JW8)!>cbI-jR8 zuGS_mOPLZ!^q3$_PKcEAHb8tb-bry`+UZCcqOVc4t1)Le+sWpbvEuma?~^BjS0Yahuk#M?0}pd$kGhXP(a!+#`|#o9ui>M~U%{urwf_upeE;*_ z;k&6$T)a-Qk32T~D0v$A3G!6%^W=Hpm&tR&e}ilP@1g$}$tT4u_iD5CyU`O z+%nJjQFna_>MSH*2wy?I8@`5oJN#SnKj7QRZ^QSKN6iyzGdM&Z8Ge#HG5j=nLih!^ z&d*eA-`nK+e_mc0ms7OFc)cs$MBkUx5f6i>FfQjyyHP(Md8UN^`>T?lM7|yQ4fs&< z`|ug$GvoUDYsu%}b+kP9d50(X+^wdT!^)r$0gy$mv37(hyD7+~7Wq3*Qi|`8M_uy5?@51ZCd0frk;eDyjE7TcC z{sKONJS?xD$1&tF;giTy!Kaa@fX^j=559mrJA65u$Ghs8=xSkK>z~gCF<8nM~iu&)7*M}D%{}^7Fd?36G`8V)# zi!N2VCb(=Vy?~A9M2%j`^8nT;}I3>P#bl1)ocv5-$WV zAWsTkPF?}Nn*2lf7V=i`ZR9QB`^bC350ZC-ABF4ul*ImXo?PD_`+;$Lz7b6E`<>Jg zFCX7u4>K<38=X-<6Zv&Kz85ntb>77Dbt)UTb?U-(Kd+1J*wMIb$MN}mov!4c!uygh zfe$2K2p>Uy5}>AMk19ci?l$U%(fTKZh@Y>vkN9dEP*-@9%xkxZREy$V*{6 z-Y_onzXJ7NksrZ!OoU&o()~fVV_M@5v;%;@`B0M~J?!?>0kmvL3Z&mA@ymvQAq|69r5gYSgvxEi6K!{h}s_Og^ed1>Fy2$S(pASE4-1d3RcxYV6KOj$t{p2NiZg`^EejK`8N?@MTlk5NAEKEK? znsQ4G^5O8N&Z!|TWXRY%Mjuqg8T^1AG*MGJGO%lHZJov7X5!rJ_0_Ld<}d8`BL}{ z^0V+cF`D}Q4@~Q9~7_GBfkbOOP;x~ zA8$qSjPP2iAhj*ent59b+`F8l{5b~6L;uf9%7Z_J;@~7|wa2;e89(gZ#a`H{^)a2{nS;*rR3qnf{@;AumhwD7g zPv!F^OkR!y1u@=g#$~+CP`?&=9e5Y=>F|%p$HE7YABPViKL{TU*Z$L>|4Hxgl~m_3 z>Z~Py2;W3?@}T|!<8nWmg5&%d<8nV*n9^Oiz*8E5j!km-U8Yy>pGrdLxkEM7|V$g#3H>Ir3lOPss1U z6E1Qqb4w)mA?tksFG3!cT2k)!i^4v(~ue7b3sg zxa3D5f0kVT-sf-QQl~QNyf!X%wxdqc#qNq*bRK?%7d9?+dZJE6<5K4~^3BNmV?KwH zkAu%3|0Ah%5d>es^?29-$A`7X<@gY-cqkrhB!64X`%mOqk^h-IGyFbzA^1b`g76pQ z72&VR%fq8D@qOuWyE#0JygocJ`B->z@=xFy;JO{tC-dtoM)~!~mm*&YuVh?~&)qTq zO^nO@pFzF_`B8WW^0*~@KV8UU!F#~9|1;=+Ecw*rzRnWkvVS&C<$bMj**_N}f0R67 z3ZK7YT8Qy{X82n@MGw@mDci?Nu)1>y}+(BLdewsXPTA#lH*ZDk%`TWzk%x7E7!(;MR z@K@xc;BU!?!DBA_?{--Xk3&8mo|Jq)JSF)qcqa0@@NDEa;Cad8mh}A*zow4Pjl-ui=a+vf@6 z(x?8P(97hdqq`g3a+iE6{Ecy2KjupReKH)WKQM;Bo|Jq7JQKP8oUw>;TmM7jQa@5G zU%w`KTX-|_p72iObKw2R55T7xmwsa6`PV$-(vSX}XoGRv{|@7l&lTJE|0DT(`h1go z7W_JSA-qoV7x{R2)Kz{xvR!OH@r+A9(~-|cJ{Mk)Twl+uY~0qbYh3Ct#p^sBjoW-L z8yhqcM|b*=Ws?L7A|9@;LcL+u4a$@TpSKO>)<#^+~|Z-*}?*Y|T< zPre6_*Sp9c;`#d#^2j)zpCeBKze)ZG{)jwE2H*c1@{#b^Yy9nc9NPs?LH>OfpUX%7 zBfK(sovc3Jfcz!ABYA%x4|>7F-1TGbV|p}y!wBO?9es$$*)ilr;Nz+OZ|HN0aoJCL z;(5wiw__ntU1j z5&3g?qIJGrB=|EAgxzw&TuGwNg|KLIaDb$*uOZmCIr(R0v_d>6bI`2+aJ*yTM~`@L%XWCz3DRl7hSxJS%w}cnP?U>qFeH z8<6Y!0<|HJ$mrMGgM9e=-iMMm&+UB{xz=Apo+`G_?KUWWNaNFd+^Ax&_zR6$F-%y)GcI*< zA^(y*JNzwN=3n~3%M3vpIazUw-mkOi>jrSiR}3ZmMOl-V^;Qh`UXi>kyqa;_e?#L^ zr#13T$y>l%!L=Wq|B)sy^<&p`7j7A6@)bOVE|;6UtaoU*zi~DBK==mZw*Q^RrOs^R z_mEGA|6p9|=yC6)ajCNr`P1ZU;TPaC4>B&@j;~E#>ZeWbF5Hr4i~p?KHw^wBc@o@T z3XyLt;=jK-c^D2rjo{Kxc^?g34lsEc*H7X8#v$Z~;lqtf{}H%zPcSZZ?jS#z{3d(` zT>H`eWTnYV{Rws4gp#o5q|e&^YLFhT+jl`re?v~=vffsxpNG5|ydc$YhWgdW^}MSy z`3U4kkk5yIX*_|maq;{OtBlLIav}d6IUdG?bL4xGzXjKM$bs{&dnR8FS@{$jCjpX| z`5A=Jo|b$rJR|u`csAp9f5>lK*1HS&LgZWF#o>`$R{F=oV9=Oc?|c2ptGe%z zWwfa;{Vc)#ce2Uj)Gw4@XY$hLE%dXA`~rM0d5*H7GQkhzY2ZhU+xa^!J`|OI*HwfED4QE zox{i{B|ikuOnwEPo%}33w{cmozVB!eI)5o>*GTIINleKufPFxrKuAIy~w8{CNKL{ee6HS$*aRp8JGRDdU2`Z z{=b=VsWTG!>*RyszZ;i2o!^rx?)M>$OP%$|KOVmQ+Fa6(v*Cf~fr_+?`^hdr2d2#qq@&@p^|V;>p$*P$)@ zRov?e#-+{)SCM$AFU*qBK`Y1uSOg1k0_>n^#%p-q-I^P%%chU7e9_%$P zbqb-*QR9-|hx~2hk}r?^L*tS^gM8fG{tKPA*YHfnrOrpFlb5_DyqfWFmye5awIxpp zA3~l9KApS`e64ZmCo9(bopD)jcjOPk^|+^hCvwc>7rVSJ_f1~*w^Von_K-Xt{0(^v zc;xSWMcr@fz+)Jf{XBjwpG#z1*83&$$;fBGQyV|#@{e&G$YNaTS8C+z=QS?%x1&x$ z@^9fqjZ2+Jv3;F##-+{~kjP!()(Fh9@9z4^Kkg8lIMX3OpnEM0if}74SUd%iu-Gx5G=2 ze+v(XM{;A8`7h=^WT`=phpC{Aal7C4BM-y&{lvJ`ZyT!ael9_drxn2la^1eijLUlU zeCCpIsow|nqwM!z=(r}p(;1gKS<&bF#-+{-IQ zBh~o`b-I#wgZCvL1|LX11U?L|)nrh zFL;_4W5v#u}G8 zMUkICo)137xYW6o-LH49ajDY^`32-n;7g24oq6eeopr{g&M@RRkq?1?2bcMj`AOx= z1SiRJz<(vr4}VHt86I1ja7&o`kUpPchf4^T^_FlSy5u)``Ept{e@h|q@$lltrG5@P zzEm(SU+zM_3i&2@O}O@x!LK%GMvmP!=tHjK9c^5WWBa9`Tjs)b9tvW+e{J&mGdjs1 zp!}A`qHe!4E?@p$EyTeo@*D6A#$|rmm-qRraA{AUTgxqx<;69(XrJA25|@y?E}lQ8 zflHmDq3`q;;gm0c9jl_r7cuz`CNHhUtnP2@LjDHh>S0{QrT53d#-&aX)cKe^KYSEi z`j>uEyAN5WkmvCn%qK@X!CK?8-M3-8?}q>Xd?xmV4wyWqGIYc5l<%3_=bw=;&gnhU zPx$OU74lz(E;GVqy)qAttNRTH;gj*R>;by#U?NP_d%b_$a}z78<+lX;Qq4JxYU`1{0{Q*@I7!Dm-G|YeaLd& zxNKj2|Bd^mj;yz1SxLLcF_UlUDRh}!LT=IRvI^@>O}-4C!MJ>>zZa3qxQy!*@_EVk z!#^-CU+VmnF)npd*KikZDMy|hUWdFCya9QBcvHB{r_2LRmx4j$wPmGlnPptg$9|NF za?3ZyrOt?uy5p;H$^V1=PsSzxJ@U7WOMVLS&*5@azHEyfH}VPpjne-I{>#v1Cb;Bf z-af*3vypd#=b}1kQLDUh>1SLFe?u)(NBT)n#d}keFX$PTqo|Jd zGtap6(+T}7Gj*h&0tG^=4K|uQD0IUwCNKSL#Cp$?uZCZuI@-?@=)G#jl^BtUDwKOjMM4(PT@=xIN$tT0N zkbe!|N4^Yxf_ww~HhC5F|JJzlp8@AZVR%7X?~hyK_(DnH^6e3xLYKKIujgq+jLUW( zfbCm?yeqsK`6hTR@>THq#$|t76z(qE(#p83_c!F*l3#%LB2Qd9R3_+09s@quxUBaS zo|O-w^fR z8khWRgesVo-Um`Ci6XKSCjLUkbW4)2jhHef$UPQPr_4hzh zkxxLKV&rq+4aoK9Rv(dXKz=Owarjr{`tzwR7FARS~UJ3q`yaN2Caar$GUp|O>*?%GP zS=N2z!|^$xahd04J{F{YhZlj%ejxof#(GPTH-uLq?+mX&-X7kN{1bRn@?r2!aNSQL z(0}iD_;{)_8FeO+e-58Vz7W2Ud>(uS`M2;j+SfSMh+l|}f)gj~Z z_`VV6U6;tef!{JN$2Wa`@tFK8^pg|^OnuzUk=*xFlKhV6pfb7sU1vk`e~SBW=t+Ju znfIaOW2UywJzxWaPTuOyq0xxo>hy zo@=2`|NnF6&oKYR$rJlnP>uXt5r0Dy^3PIwZ%7`R~*H@DGf)D7kL0^5oqx-rD5) zd$29YTQ>6R8$kXV=g*_b^?c?_@~F7~enZ|bgRj4v{Qv*HnEXS`&u#M68GW5c@JR0J zGEbq)gxCEQeVl4s&tFeUUK^g7d?Y+O`Cxc%<8psVgY)nr#^t4(P4P)4;pHb>61>)dqb{ULHd_>UK9S{S-x=apWJsr;^u& z&m^x2|BAdDd=Ysk_$u;o@O9+F;oljTaSg(_elRZMT8I2$@)hu3;5u*H-1o@xg8YEz zAj%E@h33^Wz;*vThWc5^kHFufI(WG$C`0}b>upB95Z;M=CA=T`OZZsw*fJ2eOnry1 zf$O*~q5lo!7vS5;|A6l%zXRV7*ZItXetsrT;@b?akoSZCZR*SYt{v*9xakAZf9U>L z$zQKXKBIv5cI00s^xg$7?Je~bx*TuvvcE-Z;BTBn{v7>GH!l6O#(KXpE_JG)&LZ+s z@D1eS;akXu!neb<|AD^E;DpIbKM^ufZn*(e#Lm# zlb?d`CQshb7u!c37k-kw4*WEECHQsn3GmzGBjERp%W+cQ2kD7%+3tfeZ*Psuc(FiH%F0ACXT^z7L+pxYQ|wA4Fw0E_Lo8pNsrDJfCqHuRc#H zW?br@!g*AA<5E91&X+2a$AH&>%YG~S$y@g!OB>_S-1QC;a7#Dil1~huK(0RrTu5Fq zRK>jy5IIgigCEGJA%B&83;a3x8F-xE{TDi}^zfAAIpNvidcTW}*9qS@`2onvry3?N z^PC^^S%*A5ycPLGcw6$p@LuG1;r+pr4F8s z1#{pspJm-g?a1!JElZ6LaMw>_e^~bpKWgerKXb9(6Xa9j=gEJDUnV~UzeOGc&mZrS zzefHsT-uiY|3&}*n!Mcanqs_h@Ptc`8%0n*0eL}qO7gn!wB$A6*~q)YbCP$47bKqu zFG4;V9!|aiUY>j%yaxF`cpdWH@P=@m&sdnZkI40XS|%8m$H9kj-Gy6b8kg;&e@FHm zd3hY?e>N_4GQ^WA?)T1&OPwH!T)5Z8-{H~k`rCE;UdFh>$j`$QlRt+iC;uCsfjkio zd|Alj!QUs(1ldphRCO-^s zZ(RBuhdz54mp&gMKhe14iy}YExa8ww|KDU>^4}r9%edsTBL6G-Biv74lZU(SktN+d z|5@fo&Ic-n`Rh51OMU&hOd<0AJ|2`M{~TV6d^fx~`3rboa{WEM+2jSJVYh509}GWc zT;@$**S%o8taE+5yKP+NL0{K>Nv^N!M!oMsZqfZpe{VJ+`Df@S9r@SrT;zM^5>LH2_`rYG z{JyIGdKvQBnY>pgf7{&qX!4%Xy{{tQS>F2va(&(JFY@iE^Dnu+-Wc|$uc!S#tKzRG zh0EsKXZ$kbQ-h%uAyp8eWt{d6w z_TMkmWf0sa9|C_#j+aA%7=QVI)~StrcJc}EaB_VlZZ+@&r0R_(q){{e8&fH@_Wfo!mpC&z&t;I>+w7nwo9_dem&~?bFNI}8FBufmmE(+ zf)eEV^SdhKb+h^P)q=~k$hvjuVe)bun2!Cw5BUW6460Kag}x^51pl6V3j7@T9QZA` z&d;BC-hI#HV|CX?&?)6OM77R%?Ck?Z5& zX7Z1)e|`^_tG#`R(B)N=m+kTc=KluyHuxj*s4aZ4r{w=2ALWU^UFJ>dmqR`Vc@cOV zGcrGAxG-sh1Qf`4sX>U2x(^Q(lT9pFXCC%`L^ zXZGy{9mwT5l`NylKlL0eA`g>>-LjoL79PLOlc$6~Brk=7akOW?o*svL;W(Uy`~gnr zOOVgN{kK2)2WfoW5pZdHm8a0<*CsFfXNd@Z<5Kd1@J;0H;akaD!*?2&`-T4Br8CB5 zy`zx7Kt2q9mwYk&0r`CRW8<>k-3i@=TV5NN_3n>w7jB94+<(^nc2@+h!*$+rcn*@7 zyc|!;V1LVETkFl5we%TfWgP4d8mec!l$d zLFBq$eM+7@r>`@O{8)AGE69u0^1c}^ZJ@=_4W~{1sFO&oef~W8iwN(x$@3!r2YGh* zU&dv>)&GO}(zvX*1@dpm8^ELBg*%xy**`yUAF{+TZtst2$m9BWkegi3$BG!2e#T=sslGMc%`6aM8FNA7a4&FfMgQ!=t_Q zUr2q)eIFB3I?Oq)P(hl8=n@#ysTl;MK`f z!<)kOI5r+DXm9e9b*1j}7?YR%r(hd@<7eb~;ggKZeyD$s{DpC;(*XJTXio6s&uW{Mm;@|*U%(&EVfP4#bJWUDukxxW^l5wg3R~&!C z0=Vv1`g78iCjU8V$fqMFAKI@l568)~!cQ5O?bsRTrB{qgongpdC+`J+NWKUDgnTpn zg>gAS2||7OM| zUkdr2aPD&`x%N5MxXkAh%)_jLW!IU|c7S%ebZ> ze~aqt{68Ysas6vt#uc|63cc~4Wq!nC!^4c*aU~Jh1>-WV zI>@&*^`)P5=%*uj8hAIV(*$)!lXroCLB0gO*3_5r>U?e|*ZKUxxb*ox`Z-LV3x3MD zozJV}I-mF9I-i@-XOy@83vrpxeVEU9#$`Tt`dE+|&ij8}avfK3<1(&r^k161B)qb5 zJFfcVIo~?W%D9XxLcY;0-%x#hU380aseK>Et3$?RTrJT5QSv76 zlW^_xW32ZM`DXZQa{WI?VWBe_e?BAqe~ElH^3(1%SxUfpdsQOW?N!&fjH@5|Z%Ez? z9%0;WuWsbJy#~Q`9&U!Zl;e;eh+N9{I*si$*SL)LKJx3SzTUrflIyq*8JBTQL;pv~ zC&AAcx8wSaT*vh%T*rlnIro2c{yVOvcs)FUaT(Vh9}7yt_4v6O{e+XRgx9A!f1yri zlb8N=e)^H?{0uiP^Rol>N0V=ZPcm-j=Sy;(pJi~JpD^5yzB4Y{{dp93;g;t;qobYtyQBkKLobNAX$aUVT8kc$d3G-8v{4l(+aXWAA$aUU& z!L|P+=yRNLnYZh>zf3nS^OgenmBwf14Px7Kgy&opm=Tm3N_4(Qja@~%9k?VH+mt4;$V`7KbK6Se!CqI+J z_gR=+`>aj=cP^iA2A65?=_zzM#N=haiqYQR_z8I=_~+z#;8V%7!Dktl{cvtFU;i6& zegENu+MdPyGiM4&5Kgc(w_x{wl)E|gCFUb4C-xv?wzkHb> zRy4n3>*K{#ipH*7fNef`dNv5M)Ggq+2Fc;@$@k$MP3elHYaZiA7DJ1 zvoZ~v>b-JL=x8!f}__D{i)ESIA=gH+hFU#M?rG5|8e{Edq zERwKWQbhOLMJ^@Z7y0bQCBM#n$npXC5zj$&@~iM}elRX|Mxf4F<5H&#@{f#5ek}5@j7wgBFDO}z|IW{3d4nNF8Te)cOuubK_FyZ`4_CT=G4U-)LO& zVL0ynWL)y2kUwW!@)?nTY+Uj)kpI`Xez|eU&q98`amk-W{wL#-UyJ-5llc zB&xg7Et8B(oy@2+$GFtVjQqFcHQ)!}^6hdSk~^cjaLWmkALl7_`IpJd`N@Iy{>Hz_ zcf%v$jW49W7xa8JB+ayfGEIp4a6h*YnA$V@Rr7< zA3bmEY+ULTLcTkBet2)=_Iz)caj8=i`4Qw*;WLa&ohmqQTtdFAw!6_So5?H3^L~Il zuIJ!{ap|Wc)_aP)E&M!O?k}>RyoWk}kynJriz`Y0x|!tbhpq)dcJi+9vc{u1)APw% zyiJ$xa9S`Xt8n0A3%P$ammMv?Qb}4T=JKZziV9b zdS3U^xa9HckwH?NDC%)Z&(AZF>-l+Z<5K5ethYRQe7`w@R>q|tJ+JO;T$?JJ$WDHPZlJALpeDVqK z?Bt8!mB{zQ+ZdPi>Un-Qgf6H9OF{wFXX=^kAd@$pN&f$eSE)U zTnl3$I-!5PLSKO6aV#wFhv^|u+9{5s@M8<%`X37i{~Y;p#wGtH>enzX`S>{B=ww{-%aHGFT=FT9A5X6T7jl7d zsiWr`D~wAWec#|i+`SS#wC9g`MJg=e+c!L8khVH`!3M~q8e z-#_`5amkNH{x9Q_*Y{Np!y8=4tV(_|@=1(KJ{Qgp^BR}@JmiZRmwY+o>l>H+IgGcZ zamhDBevt7{KBl|TEu)M}z7O(W8JGM+)L(8~@)MBXZ(Q>FeB>wNl3$Gc9pjSM=Od4e zOMVCPaq)uyozKX4ojkd5$)7^LfN`mx5c!hEC4Ue3M#d$t&v#lIm%P3|_E6)J*XKK* z8kc-(xzR0O8yC{&JFARKz98}kjSK1Xos-5TUk&+t#)b6x&Qs%(Z;O0_r2oCYXU6=b zGA{Wc$QLp$R22C##w9-;`KHDbIIkSvUAU#4amlYj{uARuby5E_V7ujl+@T=KD_y9>7@PWIpV)Yo&;8kc-#uu~fqV<&lGoRBIvSUJJ>*9km%P57Gts!@yCT2Lxa0?7el{4F{3zs)8JE1i{&Uv2 zwdAGy8{d1dm`$hR;q>(!r^bu=#ZZy-O0{2qKhc~t!3>37DZzP=u{ z&$!f2h5WC^C9khX{cc?H`H+u{A5iLXOkag(%K`Hf4y7V=fd2f-V` z5eQWO+b7&vOtt<^T3i$>)HlB*%|2gKXp-k*`3$1YQrW z^FOVMzoDhck49ELeQfeF-bWbMNb=v|V~xvv>i=P!W?bszM5D9F)4?~AcZGjT-WtBc zxU4r_BX{AJgT`gOvyneSJ`;YPd=>mM`BM0A#$~;WYsp&O@5>mM^&Un3A^9QrTk;3+ zD5?DZAlpmoT!qIpF6*tG(_OeFv2n>~>lET3Ie7+n2Dq$6wtF@o4|1Bk?EkZ{{}(qd z^{b#xY4Q)@mB{ zZ-@U(eggi0ycFg+W@^8EwZ0x7vXQq&z8d*3cw6%6@Q;nlal4D`NN$;AT(cR`PX^llK%zyzl}>?pWnSUF8SY)Pm$(-^CNkEew4|$UY{RzGcNgTzKvk4amnlRqp8LvUjq5n#wD+>SA1(+^0kpa zXI%0hxer->H!k`1$j8JR2gy~*_jMn#q%bb|!N?agF8Lu6c1soGlAnxx7vqv2?mlE0 zY+Um5ke^S!4E_W8Cit)9`t#h!#$~O|4T-KWi`R(L+;OEFcgg-Ve^{?xAjZ6Ky$fr#28} zQ`)%X?;;;T{sP|5xYW6TI-eMqIx+D&#(eU`@Ktd6c6pr2i1Yaqs<}cLau**R}8NE;nyks zHgajz7sjPd-Y)LKE%V89!B>&jgRdj61m8tI1HPAhH2ehl1NbTO8}ReSWn9m(ac&rw zafNr4QMuy=c>(x+xa=)bVyNG3f=HSC7h%q%{>6Ik!Yy&(nxBY#9+Q{zVI0PSBF1f< zvc{#)!RV(Vd4G5{xUTnvG~<>bXVeS5*#bAJ2mGqbb1v*nze1K+QFAN*aoZimlQygSYo#2bP#yR>I@siYaMSKiKY z&|3LOcwgo2bAY4CrJr%b1ac|&aZGqX`S4Ea zxopoL+z=KUj3a;5&@YJ3R(=ltC|s6T>K|v*1Zyc?_M_?W|By@m1}S#J4&`;=hg5!4 zGk8O}JHC9c93PJi{-*r0ywkeba<8vx*Oyx_75iFmqukx!h*f@BAsasiF4}=cP_>1T-vTAPwzDIV1^RPJHoFbmpnK23d4iSkzeS$5%Fn_(z}@ziQ@|E5h~g!GEdDNLB)Q}-oNnVsDZdOpM&*AB z`Dc^M`=Rdp(jO$3?SC8MHzclLqYiU%B#a$ zD|erB?ydZ3+dM%Exs*E?^}Lf@miKwYKcrmRT88J9yZe)`E63Yy!8hbm?oyQd7rB&M z4PLR{rFM|`-{5u0CB6Z?h4MS#os{GDNia*wE6?*3l>bLH;%`=fGq{QZYq zmUjipEzrQWzqFh95_nN^sgFBeUrjFYFC+dM<%i)llz$4Zqx>UyL%7sR)|Y$VvlqD> zau$>{4`vulF7drWHTT1mKL}q*E_qfV&qi{|vmWtB$R&Oq;*XO{{7Z=chFsz|Bfe0> zOWV1`e}ecr%JDJRpfh=C6T1`_q65h#&&@V%kV!7_?t9^8l1qFy#6Jjk`^#$`%|tUi zr`&B<2bH_+@O9`{4F$@HNHD`BE<3Syl|9s^6SNZ+$ zVvR4YcZpvCFQI%Hysq+R;0={;f;WY``r~rO{Jxnj*Li>YG8m!U9lw%1J`?Wx&zDf{ zY~@Gb3zVOLFH-&v{8{B^;m;}m68-|*E!UMe{=THV8T@nQx0`h!!@uOxkKKvmZ1u?8 z?dJN`Nac@*S8)7irPfy z$fbXH3h_moTxti2k0@s2uOgTDJ&3PGF74^gb0WzlJ^|~cC%NQ5gZzWYC4M8~$B|2X z4&tYfbGft0CC?Ygvw>XltS@RF%&?tY;(tK=apeWDo%{%Q{duWIVVa&mN?yZTg7?&?!dxjWx$sXV#9Eq9o5cO1`D?(*a)Pm8p9X2E4C(DdPi zCn#R}^O6}hexve#&|hs=J`VA_ln;TwqWoLT--5e(K5DBIoF|vPN+>T@r7+z(1~2)V-eqrT;lO|e~_g7VdS}+oXcHAE_t3s z{Ilc|e;(!TBA57A5&s^!#Jleg|D0Un-S6TBH`p2Fk?T{MT;i`s|ImWQH5b zCH^MF_a~S5VpuOD$R)l%;wO+xd^F;x!(Bf!7yZU6g{K zSoz*-t$(C^Lu2cIDKCZklxtzz)7A4Av`=T{uAU>5yT6AWqr5Xd53or2zM-~0dzHJ- zZ-1=3*>yJlYvmhzS}%BG?)slXKT}uvX56=Ip?pA7o2R34m%oqlhx*$1QOY}4x1OQ= zD%|g#qPzl@YmxFpH`zQpmG8ub=!eRO;ri~J^6#*IFK?NG?yt#Y?#6!KJ2e&|M9pSsE)z*Y?UcV1MkT{6MtLKLjqd zLKBA*?o#o!Vr={pZHn-*B0ktD?IY(igYbr573F>4*TY>N?2bWOia%!ZT!!=Njud}i zSfYK1r+6v%4&)!LJQJQmF8S9qvUw(uOP&pgzeD*d_)NIeU+PoUrU~w+_@l<$@7pdX zm;CP{&*RG9gg>eL2l&&p=O)UB z!CNbz48K|Vcz8#+TVHcf&)bxbwbc&Nl&?bkEak-zzf3tUD}raq<>SoqNFF zAm{iG$R&ON;?KgRJ*6Lb59c8fxKZr<9h}EDQa)vXnaKz?KY10ol-n%Z)@QBqy6}zUlBaNg8~-12 z$&-Zm9m-?jd&nhEF&x)kCYL;m5dWI;d*N@AOPU~HPsI3;gP)a`h5rM0>vxL{4~n(38=h=evj2VpuRt#Oqmiew@*eP-%E!R#DjyAR zs(e1Yh4Q)ZcFLcIcT~O>-a~oB1iN6-%FDusz+L-9*k%dhJw8+ANkyL7%2VJAl+T1O zQl0}}19$bo>I?ouF8#?ZsOLU%*?&vI-yoNGTm}Xol1n^Jhl2~`5|5AD1%HxD{2KFL zhGJ2+AXm>hD7PlL9FKkrqfPnblIIBGdy`8%ZrcXgaM$lWSj{GSK)KsbA5*@(nT=ns zyu>xuw<)g}VST^yJ^id7g-f*;SqdM&p?KL}R$x7zSN<^kALaYu1=`yNk>!;;<$;9JPK{a+xL zauI$YXrC69X_xj4Dx`2q2zl%IoFfJ-)6zgdOMgBh-;csWk}h4aj&pQK)-Grs%y1{UEbm(6 z$x*%ney{SE;q#OqgfCY9J^T^nr{SxW*SaIDX0T3q1^8FW6X55RkAN5Mcxk(mzjgp#)Yr^u!L?)wJ6RX!eh{!qReUa9k?_Lugl zaJ9`}lU(v2L;Nk|5+8;5XmW}F4DoT~5}$ z^^x*<@N>!^h8MtzwA&8Xz)LBA8eUoXFl-NXm0yN-=s+&Zl&^=EQN9LVS@~gj zgz~-cy2{VO8!A5yZ=w7zcx&aqz}vxH``?6ixQ$%e^D^`s8RXKQgAqSRc@li7@&)h> z={Ldr)Ww={k6X5TW%l2FeUy$*U^78PnlsAW;QyvNbQF%}Jugbf@FYlUr z`$>RbseB~-YPf6v<5;eS3Gqqf67PPOFoRs;i`ms0%p;e0_xo8($R)lC;`b}Z#~_0*m7`hA9|qbPrXE6GZCLgF7eCIo|DKW{xQTaP`&}a zT=@?84syx=H1Z!Lm;CPYDyNm7LY@M*+9&cTd3GRA33AEvJ>r`xFNEVrPjbn#A9)6m zOP-pDpGYppso&(m4AaRaz8&J1kxTv~$iI$U;vYi%0dk3d9q~uWC4Md9&ntJogHRlA zD7f}{8+po+OP+U-rL|Iy_eTGAnq1;!IcZD0yF!wKcBn_K@}xqH9+3<9zr=<;Bs?J(au94~$Um?q8)Vzxf7R?tRK%#P+sX zxohXc%5$(iys5nB5S#xOTeC$l|a-K5@=MUYK zkAn|Vz79TA`3iU}xopp_{$t3c+_w;)ru;Q{7P;hcpWnD!`3RiPFH-LACqGRt<^F>F zTa}-OZzq@QohQwI8TOM)eNKdGu8GJcfB7kP!g1wS!QUa5JZ^hCMJ{=6K>QcVo50VK zOCI<4sy~rSo}q~UP5B`B-*C5`B$#<*xFXtaZ_d*!2i26%g|||^3f@QgKk!s?In>pG z&mym4tf;L+yNTv&ugnWb^#2+V`?Tv56E++7ppC0g731C+bv8mWAy4G+?kyZm=5cc0g~Px*{SHqj!uEJZcs zmeGGGUiOy>*e`Y{PlLaxd=>mvzZ{Q zKL~f*PaCw`i^^NWkE%TGckaGdUKQuVzk9qKKCtAekA`O}p9r6%d^~)y@_XTrD8C!N0`A&j zHI{cLxoi(rCBO`ClS{iDLi`1C+1`#zuo?a&m-r75e+@P~S8lg3+T0%^mvT45yOK+u z`S5shiFf_USaONqhxpm#68{SPL2`*tmx`HT6S>5n3Dw+3B$xOph(DzK9{7jKAB7jf ziHE$bCb`i**O5!PKgcvQ)FqerXvB9Sm-qt6--}%0;}M^sd@_6!pBY& zFZ=0gZ2w)AFM|(IeiS}L`5}0c@;~4y%723AkW0I{^Z)zErJhxC%!3*3S6(S6GzFeNK;gQOJgZEPI`k680 zvK>B#_-yhBW3_B?!Cd9|_;j#VIc{eK2bISl&!@`A!v9dd7+&Ev`^0UZr_0-l)>iKR zKB$@ULO0s@5gxx?x$A$jmAmgHn5Nu)pVeID?)$7BQ9evMZ!@fc%hFvJihVdp@v>d@ z&awWY@^0{>O)x^`X^K1}l}EzIDDMJKQ{E9i z8-5w;GacowQt`2fU#ol={7IE(B=T%mj?1RtE#(~$e?j?LshAlG4!2L_5w{pZuL^hT zYZ~fvz4A%$`pP%K8!KN2Z>jtZcpK#}!#gYg7v4?zukgOg8)5|vR9+LF3BL^OkQKIq z`Ta(Um;GoMd>*-M&jXNWf%1OvN0g6(FIS!bU#EOL{7L0m@NLRx!?!D+1>diHDg3bV z#qeXw*Tdgb{sjC7a%nerU-4gZX}9f&FE~P4-rP4a|HYq!7lXU?@`(hQp^ox9?d-uV z%4fsll;3011gXlOhtF5;?k6l)eiP#ND0kO8A1ZhK(jUrGkf$h4RNV5ui~3h2FNate zHHW))JBD^^sr(4Mz4AZcot6ItAEmtF3|si^%FDyk;8Ja=XLB1K%psR!R5W}Uxx^2L zA5xBw>js~aOP-;Kzd$Z|9)OpOwGHU%^BTOK@>B3Oof0fpURM^@3_HlBol`?Ke}6_UdD_80 zCYN~Ef1V+i_yLG77C{6=;XP$``?h zDt{9mqx>a!lJdg1k)5LaPsC>{Z;ALx${WF_D<2J?r92M)0J-$H?tb87j%n?ytLkN3p<>nG2E@Mhftqp${&EYQT`M> zO8I(tH{~zFdn(@#AE^8^{5IvE!{e0~xZ5_wXyyMPJ{>NtA?<+6vS2dB%X)P8XJ(Ph zdaQ!{_bIOke@J;t_!8yK;VYH*gRfEE8@@?-3j7)6N${P@?}qPDJ`H{l?%Jo0S!Nl& zQ{K^XaAo|Z?W(XbT!sYI$z2Yaf->Uo!ytncX;e(aeoMkIIO!;;2 zMCIM!$;#WoGnC&2AFq4@e5&$Q@EOV?j+&*QhKJkKIecja5){gm&A4^qAto(*^PX^r~K zSN@)@X7HTy257fWmA8cdtK4meRYuzYS8fX8J1TeAN5hq0gL+O>J`wd?NG``-_qpgj zu~R-UaUJgFlxK`YU($ui}(1Gj)<- z3c0MWBxw*cJU}kXE7N4yq`bQ2;3e|H#!45m6W%13Jg(n9ryL*O3yPv+l;x5#TcSSY z$tBN7^IwKYa(C`5ekq|2Vv3jTvn%$yC&*>{9DqCY!l-*b4`?RG(2o?51^GE`LV{yw6Ka(7>`7rAU#CrgBBf&_ADw`9w~ zoytFiFI0X3{*3a%X#Y2qH-dk!+#UCej>)~en-PDV^0(n_m45;6uKZj0NagPHA=Ahg zV3}m}7~Jhg^RRwbD!&`PLHP^tP0F{zpHu!de5dkH;Rlshzt2|oMdg*?@4;p9Wx2-M z@Zc|xUxfprTi!m%Q%-p|cva<7;MXgk2yX~?_4yg~xzXeORGw#%XOQwO@L1&s;0en2 z!N)3p6FyG)ad-~g)&C#V{}JV-vAsP{F2{vtcq4N^xvZD#5&to{#P>q{8FGnlj`+fO zqgh^t$0!`AN|8(aEr@SG9v)v1--2A?@iu+Xk6c2N5I>w;;;}milgTA?CE{n2OT4?U zzKUFyw?CHmNpgw58}&b|{2}-k$~VGGrrIa+D0#NRYbx(=rpa)ta<{*vkjo;SEER?a zyb*CPG^xwOMjw8K$yiGLpPzbJS6QTen>%PV;*B2P7P$+HLT z(2U&O+mj-C!LKmS<@k68+kao>r{F`CUo%ItnEQOnOTsgh-vS@6ybXLRxg5vc@A%9m zf7SRT#Lrhg0sb&t7GIWklld>hYKoWqs|$u4>?W7|3z28P@>%c;%0Ghtto%LrKg$1v z7r+O}T>D>u7gt_yuB~S&;XPuPg5cKd*c!yjX@U zNFJqJ_c_1_<>L_FQ28WyYvt~DsRohD^3KNcjwY9KUqYUl!=!i!?Vkam-L#$vh3lS@52Aifz~_V=Ua{~mZ}@>h+& ziv6gY@|WR#$vMwZa>?@v;$xJ*2TxLd#XK{U8B&x7@JzVdPVTqi!Aj-sc(S(mn3$-f8rYmiI4dmpMjx#T~N?dn!?iGK|B z8BV^&xEm(IUB8`$_PJB}DEQsV_rUK}{w(}KxGbHNTLT3@M)9)V8{z)LQ{=MV|3;oI z%72FMP+kWsa<}pb_z}3P&j{4#IK@kSZb5xckxP9BAkP=dZ-sxW{66>vknv<=&$_ z1HMA}9q{LsW3hrm${$7ihsvLX|Ev5Nc!}|LLF7@k&oAH=mAAq2-l!aJ9|ZlC&qREN z@=|D@smk5o#V=I;H1cdBm;F7dtgYxS^7W{@j84GacG40b1o=RDJ@^;OXTr}Y9}mBv z{5bq)#E58a}Q+Z!_UFBWjO_k4q zw@^M6-VW~S|Fo@6(2e4y-&l|O4 z3cuCnB^TxStqLHTKTHF9a6-;k#sx#am5@r{)K3cpc# z-3QG~X1GatO?W4`EQRb}Je(3WR#$l!cnjriuw4C> z_l3tPFM;i0s`5>!=Un9n;14UGgZgYy{zABR&HGx)UxI(2{1p5j-5#*JfjTSfG3s{1N3#;LDXSg6~ki0lr)LTKECD zERO7V&!PUWD|dfKc3!#rzOO$#e$5@Wq1^KBL%G)~-vh6%{B?M3<*&hShP(PN!FD^8 zytG-a^Elr~B$wq~gZK>P?t1V(b94}VpT%YvW~ zZrHlzT4B=!S1DhD_0?GUN_ZFLZu^W;UKh(XkzCrXLOEOT405S|dGrH|$Ty&lGTH)n z>-`I?uWiadgzr{f{2`lczw*NHSK!hs(co{GVec-OZ}U}yOM8$$tWJ~+A|jAj#fSp{*cNO zi(Jb*zC-1ik374T&xIdRz6t({@^$dHl)nW(q5L)Y1?4}(e^&lI{2%4lF0>0=V2W*j z*KTFu#gz|&mr~vbUI8xIWWBiekFNLl%_>h8@^nz10l!uG40vzl)8K>Qu0HNMccjNB zt2_@P&lKhJ;rA$C1)rmQ1$;i-)u)YZ_TU+E+5Q)yKYW2)+OsF(KU6--Oq1cPa@T+U zqTF3ymz-)7y85iPX@UmI-E~kaG8|VP5vuv#JIW)+*}hm|Uy;jk zwg&Q_Q(hJRqw-tezbcP{2h(zIZ=>Lal#hg833too&eO_!yqU^#7xLVw{0?}O@_F!1 z%ICm)Dt`>#NBJZ0+u*wX2_C;w<#`f$a+Gg?->ZBle4g^{@Wsksg+HSFMffJTtH1kv z)bk#HUFG={dEQd~3H(Fl=i#T6e+NIS{IVr>gZ@_e--!QHc}2wktGpb%&~&?fy5${- z{iU+^Z!5CQKx(S9_2rkw29^@uXmgE`O3T2v%XOIZp*>L z$}_LA6P78TgnB-%{Cp7`|Ag`%J6nHJdFK+=pH_ZNckA1f$KnqWw=0jSV&iu!-_gSwEt@O?m6DC?7k(`s>Pnz;)JJ%Kz$M<4-ED(b)Qj$|v`+{;Bd)O|5^ad~yTp z=akp1VEsGg&!PT5D({N*@~iURXrHTWxPN z?Mv#<@$PqEw0HBwCwqLl@`|XFyYC|T_rYZ3?xScwMR_iO07ncjdbG4W!&Xu23`nhT_>v;`~U*H}ZHF<*wXeNuB(PH+6)StbL$Fn`YL3t;X@uu<{;6IYP zdyq1z;pH|!>cbwVyqXOQW{|r+TqZ5|_FVQh|D1Mc`STpAI5lmxpKGt zoK)`4yW9;%mp=hz)WMBM=gIIs%E!Z#$cMvZw8-O|l}BKngXCjjGP>aLDmZVFdP;n4 z2{uD}<&lSO3!#KZ^1fzW0pxBUl48<5{(#4yRqpa1Bp*%nKjm@zA%5F$ zBvO18T(8LTvbQC7*RnF(V2@{dd@*@EmHVE@uf_%}^^thDJ=9j-5V-~_Zw_BVj!V99 z%4;6~SNXNbQ~hfDMC#-ImQG}d$LA<_d7dO6Y*U7huaZAP{*}iIma#!nABjgd5>!*( z0J&}^p8=Cmg2!ii{3+!w|BD`fUwIVD{?X%?m$d=1TvE?F5ns{cjmQTeRz|}-K1+Ed z=2@=1D}0m3_j>$Y<%^K#OOOBR@zOXzOFPJNy^lO~Jl@#@gC3g_zaIP zCGT&_bnW)Oa#Sby!{Zfkf-1|)c_NkLbTL?_9Nk^;rSd)~_;>Ptt})*=LQO32nZz1)Ra%aQaD*qZjSa~6t(G1zjw_|&FRQXZ( zLFMlA?Prw#h4|vQKz8-)UeK0XSNTYIFXfZrzda`#?j>QaaS3VZGrjp-E`8Rp|gvZY-?}c1N z(6Ca^jg-HQ$7d?P4)bgxA4GZHSB~8$_(A#2n5Rrt`$Xz9kn*%u?v7szmAiB9x0Jhn z;|Jw$qKw*D5LfPH^27`Y%4@)UZ9#uY6c0D{4Tz+#|JBS_f<2Le=iBmFkksv_+F2nQQidoUh(T~KABzW znGUb7d=k8y$79H++mzwsJdf}6_?ODNAy38>HiVdi*}+<55Pb znz{4rhQ}!%P}oj5t=z@mSu6iMV{ik()$@JiuUIE{`~=*OdsX@L2pjjM^4*9pUN?6h zcVDlU^3FJZjaBaAcYFNGdim#1^!RFzN8LOmMa4BRXpBUd40rpQ$7VgM)@B2 zeaa8R*D7~;_Ims%x$G)e;KcevkAF=r`@73uxRGrq*A5F&pR&rgz-uaZ-w)s1;~hQT z&*SkP&-A$aT_#t5SDyzx@vA((#p8RGH^qIBV;=v=P~~;Y+U;Yi@?`ir}S^+5X*qy&FA#3;8w^ z@AAhgZ-@GfQyveWqujN_T8}?VF7<3%%C_eLkH0}K^>q2aRz4N=`9t}m@KQ}|Ij%h~ z$N6-m@~#m!?snzJWkxeRs=a{q=ajqmeLqpY4Cgt$n%enXeVmV1{w2y?;_)NO&*HlB ztnzx&*keJwc+~uCX+`b;-@zL-rOxVljfA@QAe5S{zdVG$@m%`n8iIsxPuuJ)5__xYk z{cE(a^SL}F5#LIAU3k35rz>xR^WeoE-|6wQ9xriY?s}#pf28s!;q8>ah<>B5^6##) zc_%8r{A%lKm5+g+P<|cG*T3@kAIe)JPq~)4mn*r9eO<%jQOcW@v++|rzD;>g+z)zB z`3l_EDAg)=eO&x7l`n@sqr5)+UFF}yOWu^b{zXyGI?DT3uyLK0PlhKbzY_ON#wq`)rp+@`d2__C z@%SO-F8?`?mu!=JxvC(4Q{^pjUn9=ri#>i=`88OsPnBN}FVZ%5eR|_MwubVB@D9qy z+u4Ij+U4=7 zQ~XkoZ}s>w@=lcJtjCMBw?VR8Qz*WL$77ZE#5|M9J5rv?0K1Aevg{t{q@ddf$_qm<)guE7BEf;L6? zIK$)5c>HtaxJ(PK#>OGlXixbwJbpKMVH2N%Nv99)stB>>RJ>HnSu$k4}2WaQ&m-@SMi{qf^+N~1Wxw3Nicbd(W*TZu4 z^!O-`XOc_(-Tl((9$!E%^>^iNRF2cU;05KwQO{4vx4>l704EKy_~Og4KlV``Wv0mx zt9*dvAWivg@F^Z&K#o(eaE|9Zev%xgkYW699xr#Rt$-|-Ya5YP9*_0-6y?|2w82B< zIOPwg?D6;+a$Nd^@t5PkE%n5uZRjmMK7<^%BEtAA@|aNU!vh|FihKmczwPnLIB}AC za{SH8Yav$}Ic^1oQx=fp)>P zFjn~y%fTk)U%=l}KB|bFaE?3P#!e;o|<3(`bld!d6qw2zB#p!~jqcKO>W-vA%3 z++FugRNf2mkC1P*sl&&YJpPNvYoP;|`b(Y$$Ti60NgmJf_+!c=uzp|k_zxbxZkVl@ ztEco2o~?DF{g9&bL<&L^`!V4sGM13f<1;|DzcgYt4H{Bz;gDMz;)G*Z3>@$HrGf{##+uW1U#D}NjD4=Mi!zDc?JzWrUw z-RDb>DR-aqKc~EcEVvnpVFQ+z>7m$%#>!p4Jy7}Qm}ipm70Cafa`!#?Pbx1^$R;{W zKAY^pQQY^BGzY;&$1LgKC8SwroFAaC;SKU znUtsgX!}B{$DX3R7Uo&1ygmFW<@dplDt{d9@SSq^{iU^&?2N8H?sGETmAm?6d3>qz z2MXA{8$EuAJQH=7(OHj|OtwK%PmXV_+|{$I^3HblV5IVg;WNmKz-07{$4`4a@^(A3 z)RXh{S6;`41&ciXoyXgc$({c`!#+aqwHo zQ(-d7R*u^s!5rl{4G2~$ckT0{$3Iiv&XqmZK9PD#ecZNEg}jD~HRE)TAN2V3SV+mw zd7e=2miI^HuIT7A`^@Ee6%(FO{tf&)`C({+-7MV*{2vb1BcALU~o>`cZjPc$qBwM9P)?{oqZM z?}K+!eiR<3dcei1|V&%BK5WME`YU6X~zXy5x zD1REB;_=DwD@@$W=Kt=SrN%)p%i{}F9``#_Ta-T^W(k67xS8VGVJ|#QxqDx1GF;c^ zZWZ6sRx?;hz6iNxbim_Zc)Z9&JF}$aJheRDk^CV$Yxp?MgqZo)R@TM>U$`O`Q#`at<$_&Mc=;lF$QiaT@H{~+3}q4Gqu+bzmF;&?j3 z7a#joc096ls|wx6UnQ=Wc0YlU-9@^%5Wjof2S%w#o}(d+VXx{m#Wu<Pa{66KYk!KBgL|8-n@S4Z}^mx@A zJD)6Og!z91mbXccd0__Uqv6pxA%}jQeHuPyc>F<+KTTdD%wQkhQC=9+ej+aklhM`F z>3S) z4|{xr$9E`q*E_G1m$Qo)KK|+PYiC?)2Z^tbd76=zhskJw@(4^zR}KzldVD2$B^PYQ zZ+X1POdBNYi{o2)JV|+kojsVLJO;jyyf#cmn>_xK$Ip1Y;N3Q<)SvTLRbJYL1+6^Z z-{WIFKFi}zkk>((GCJ$=TC;4B)L-J8*_T0g<-Orkly8PV?(rkalcgXtoKXIT<=|W8 z*CD>ZJvKmI)di;BjSG?EGEcKK;u8(M;+?{U>RPH{Q$rsq7!pB`6 z|Hk7r?z1yXJvmQb<+wc+O!fFy_T#f~kmAm8P zBITP<#>dJ(fd8eumsG$E_2=5LtIss}Am#VMXDD~^>y=l+esNTJmx8vuQywoqFL!-h zd^Y(S8xlV5_4q%^Yha$bXh2yU$>V&W@)3xi>@wb(`W6v+j|ARc0aKR|`mvRrl2YWnQ z`SsXNW_tV~<+Tz2xW`{0-;BD*=ogPSTx5eJEyoY`_%!8(?d-v09)I5BZ+iS&j~82< zyPmGxn#vnsF?iic@%kT z@@Vp#$YaUfIgw0GCvQvfIpjB!&nIt3zLGqOd<%Jd@_pnT$&ZtFB0ojmnfyHYE#$6q zk@oLGUK*W_csKG0^6uo3 zd>{D$^5f(K$xo3FB0o<)n4EuqWC(fbLiUBUzdKhFi6H0e{YY|m%#rvg^5Fs)qRHL0 zg-nViclWx))5&86FyxTOIW*(> zuce)n$xGwJM*Md22=X!Hk>n}lQRHLEqsddrW69IW)5$Z)bI3Ev=aXlXuOy#9zJ+`; z`9AWy$d8jxAwNYvmHa$;4!Qe~mbCLU^3p}@3-RgX5#%$-BgtoyN0HAWk0zf@9!q{7 zc{=%A@*MJc(6^93M0xg+FCsrqj#JTa>M3%*FLa*#5y}%3wJ)Td zA0;nM{up@#`7-iI^5x`F z&mrGQKA-#r@|EPf$hVO1Cf`TChx|DCUh-4q`^e9e?&Q!(=jb*Acg`&FSDF9r+`T8oi@U!^aL0OyFJt~^A^W2>?xpgP;)y)5w5O*A7=kERVQ{>gn^DTJ& zBY6$;d^TQ}mIP*yQEl_Tw7f7wE%JKi_4-iF-=~u|G!_Bxqx>X1j=YPpui#naZdqm0 zY;rk2#BLruto(iCc~W_?0(Qb~kH1bX=X>eM^9lJ#n2dfPk2kOD;B`@yBQNEABfqSGe}-MlRJ&6D%phUn?(L33A3B`8a%#NxqH9kb@Ine4%feYO1|2->@za_ zsNC(p#j$+S4ifLYigNdTAdSh_n0V(_HluMSZDFkav9are1G zSFVfySH-*UwJwhi*2O!otK5ALaC?vUQ|`WxH&MC!9@o2+*C}Qj)cwwhtB?Es(#WX55lIlDq?X6Y?J9P00t6yJMG3cHfsQ z%hjCXZ>M;7Eg+M#$XihS6!II%XOp)ie~`Qt`D5g*$)6y+b{5$fF@)_hQs&YNAkTDY#)rt|`qvGwlTUDH#_y9)CjXNBF7or_Q^@}$pGscjO8Y{# zpB(bic^itKMc##cHhEw2`^blr&m|vCK94+u`~mX2$QO{$ zCVzwPyA!O5s>KdT*@@m1t&$SaetC9h4sj=U-Pdh)j98_2tmZzS(a{v`Qu@=fGP zAAc?+Jfbx%q@{2%2X!V6-* zbnW>W-0k#4y}tn0Uxg7zl6K*y>s#I z^F8kGT%8vzY}ZSQ%2NV9PI*oEc;!vtcPMWM&r#kLez)>yxci-3SI^<_2UL7Ie4%pp zet>(Q(dBsv@yk{G3ixW}8{zAfZ-cwog9~9;iLoPI7JG54(cm; zu5wAt_(N1eybSryqV|P&Ir6HgoOpTiq2v|G>s)SMNPH#oN6D*@Ctx8Z{yOrF@+k6-*ytsHd-8~D z?F;da|Krk{4=uDL$3_1bI677wFU^zq=-t$<@)hh-V65*iN49 z(2ReMvM(flf_)l3PC@4)KAC(eIv4S~$Rp6Xh)*G(NIsSPnp^A(iO(TVA)iJ*zKeY! z@zcrwC7(gQq3fmindBwVNlBhrD_1t>j0@Bk;U{85bmX#}4r#M)9jcu@8??d>Qg@$jgyONCGp+sC@X$K6E9o zNIsXm68TB;D&#fLSTgxK@;2lVjuRjDUm-xWfqDUMUBaIXI3 zIHd}G4f&u@?8DpSIMoU#oF~UARp@n!+xetioFat~OFq|SQA0n?%*1nMZ zi^!wN`TA}y`BIAiko*zypU592FILXJka8a*k0xJ6KAU_w`D5fO$ld*6$-j#HFN%Mh zy#F=!g~YEWUqilz{0#Y8@;W%_lsxOmhmo%*KSaKP{0#X<^5zxn3(5Z^c@p_1@~6l* zlb<5rLLP~eW6A#vx%>MD@vY=XDSjJy*K6$yiGQAa82NVcbn+eK?tZf5*-7s1CyT#8 zzK`^R9i%SxP>Wq$Z6E-Nl*G%_|#FRucUs#*CteC`!$x@GtwG1nUjHb4kY1tX^iLxA2)0R9#Qe0|6 zN@A|ii^<|fkIqOOE!DJD%)AuVC_XJUGb=qKExl1>KIuj!#${z^BsNOU%!tb!5SNmY z79W$9W?Cknsa*Zcm{DmNF%#mFvyxLshYOr*eVDS%l1Y`)Q#(% zcKghyF=dO3Nk|--JvwHrX@Aoeyde26ZhTyNTzqoYq)W?^XP(5=a1qlJrFGCuxz=}> z*|fm_5{6_MiDn~|g^!mtJSp!slq=`|7VYY3w*0h=tmL%RxRi@q%>Pl}|1DfnVgm@P zmMNQs*=Q58Q*w6@7pJDi$%YheGNH+KqfN4;XC$X(m^Hw$;?j>~#-t>Uy40s#+^{Yd zVtTWWnTXNRP|RG}~vc{Uy)D7`y3SS{Apm$EKww zXPMfkj?R-DlVT=h#HHt%Xu7}b6xpw6R@)Ee33WTQ>8s76ro^NqkIab6$h$BuF6^KY z6qA)WHa*4kl6kU@G+o`8n4~mwz?hJnl@#vv;V!P4=u#Y~!0sq(NZN`gg>9Fen*V&c zCYpuGGo!4ogp9c4ROweoP6{h%w;5G0vs`(%m@$co>81l6FFk3N*&w8gytK|F_oWEw z=JH!$iE>Suo{?yJMAJc;gKBDOV!Z5AoYBQy+VR3}KQqbfEAi4_s!C^Oj~yGAkv#e0 zK`U>BDSf0l>`h2cFe@b`+4TE)Vq`x`95qU+W0v${biQ$BlgY}vQzT`ljy8RcIj*E< zn}BesR8?(Ao}5@x+sB&y&KF?zw^1?w$MmopjSCyp?Bf1BvW1N{JCJFMe1gnol9g;$ zxM@(F)g|}BA@!4auW$zBV%Dlyx?Zj32(s4k}-SQ2+ct+-p$S+y4 z*&)=XBYO}f^F&-sHYz#8bVA{|SGXz0j5epSdAC`~dU2kyX$g7PUhW8U@@iH_e74zr z!_(p1hYqaiiwRwQ^DK*(Y1gl9+L*{HWx z@kwcEnRJF6o|Z~gd5((&Wz3Ok{)hkF{3il#$t!yGEB>u z^%FLdsd1QwCu2WMjZaLGUOkM9Ba;syhpnDxLk>{;_`Mw2vLWa@X+Wg zY0s(h)Y8=UnR(?Thi9=z zQhTA6&t$Xdn{$iocsa!|mpf*I&3itKgSac)267n)NXp7eHydntb(zmew_0=Y<6}(I zna2G;PEpKR?|3fAwy9iVm~KAzsgz8b5SJwPm?n6H|x|p%K z_s6jM=^3Vj#Yvv(&CIn)Vth(mhG(*CJsex~B~Gr!`9Bl=pA*qt{SRlGaS67a^65ac zN9K1n;TAx)nB)X=g_G-&gxAu=2x&>4kZU2a@7hZ~Z&YTIS=R|^6H@c(OTu(^{>&H& zlsL|W?J{o>iQ|*a7H_W567ub&FyAP1&1!A{aK^Amd$Gza*U_#QlD#%>iT2X`V#k*^ zz%GorQD9F=%_SKZaA~UCPspz$lY0#R*Bs`mJD-l!ZA{r2d5d$?9a~WOoI#4<&~S?M-0rLB;v!Pd7*Aq`dbcT3L_AsaX?KWAd9Omz?J8`O4&l zgr%B+xjQy?YQ2sv5E9J{u z$GZHVj7brC`(2Os=GXO?zi-U1{^_sV^>+#d zN&T-jNp$^d7?%)T{>tXI#CI{juYCDkwfxGDe#4d@f%&C;*(7xNgN#dvEK_%lEA2xBL~}we2tKTm~sW!i;tK%{}Eu-XwDcf&1P{`zmk$Ex%KhU-^+J z-~FDrlz+WRqRa2&DgSsITjCirod>=A@_hpNm7jFN&gj1HRLZYm66x}jJmv3eXG{Ig z%*TT+Ki*URsrPO9(U@P#uVoVH^6&SQzooq`b-y~X(&gXlDL?W7b5HpvQGSjjH-nVl(2RBYWla0#SN^^(;cUSayBPL`dDP1<->;D0 z@(=#h&dB{o<9y1O?^VdJ{3toGHN(jNQhsw!`TIVzGe%*4S$^q@^zsk#l)t4%SbW%S z_L+Uu%iq^ie(o%RiCImqW5H|D>n+!mj5kJ`AbngFTWfTb@_$O z#+zUJulyg&myJKa?f<2G%9lfuF29+l{NpHJpBL-)m+!^Mul(OoephP$w)vDF=PAE5 zR=n)9Jm~U=dCHGEZP!1yzZ{bE^3V5_ABFOVapC5f z%7ZR{k7xcZn149WY@X}+cX;Z*67`S8{IdUcGS7AUz3C~xbhMq(OwKjv^1t%TAMroV z|Bq+>NX%a{SLu0|zk1>P_rEC2-{-%~U)9Vn%NV8N0-Sio!lnJXndiFwx_HW8iSph4 zY_AWg-oF{#~B>pGW;m+iom8 z59n+1==y)`DgU?xn}I)LE3b9=?|I4(zP1Iwg!;?&+uzKu%a^~ek`Po=iU^05{p@&@ znh;_6;^VKL@?ZMK&bagn8!F`wG*5N;mzxu3DPNyImzH2NU}RsIN4@-o%#8W%zp=mC z^1EVADSxneuFG%YDL)+r%RMh1bomWD<*!Hiy#L&066x||Jmu%0d^0)Mpv%9_Q~thu z%9mrIF8>}+`SbtB<)7{;|71Sp%e`V<{^OqV-QSNFFz07H=;dGLDgQjm-;V89)_<%? zq|4v!DSwOHm^Oo%oNLhKZ}*hH{tsI)um6!IlP>>#Px<>${(sy4-tm;bFQ4)g@+top zPx;4D{;^z3pLhSe;3@wk%8$kT(*C3JDgQcif|TF>zfyJgpYq4$Q~tH4LiugKu_%Aee<{D5nLoevHxcD`#r(4UX693V2T%Fu zQT~nprT=W}DSscz|4kOo3{t+_i#Flm;MU79Px+CtHtqp)e!zoX|ARc`Z~xcszkL4m z|0p{Xc&NVj@sB0hhomgU*!QwW+1IfqTUip>Wh;cpPR166tWlOkN)lzsk~KRiYxX@$ zWGTw;UNh6jXPWtczW?)jb?>?Np67X==Q+<=W=!XL0*WyC{y6wG)=298gK!{^$-jkz z-;9TJoH&rA0Us(FT=r)Me$4i zkdYZoelZSyCW;R`(Ehiv@IT|=>)0ZB|HbdY!LK1e`v2hf=l)pudpP)1c1V6U^7QA3 zX+QY>8*Ke=n-Iw-K45vQ|X}>ZKJ{^h=N6CZyP%M0R9DF%@q+l;f9(}^(yWrrPqxk(OL6Cn3$OFx_ zpU{2*|6e{f|HB-R<^Ie6PjK)h4kP_9h~j~K1dzw%58>boqWJj0{Bkg1+W!d$-wwrx z1JA((^1-x>oUDL^X)osWYaDzXEc|3ZAEy6@aqtCE{Qt85GY)+N!Z-#?EhT?Pl2j2(>Uk=3=M9G8w?-3UM9UOcrKcpc0fAE8F@LjR+v#{`s zaq#6(e0nfA4kS$f7vkV&qWCo^S&ue#n2! ze{_I6w*D8(g^d499+BtiabAdhK3xStYOMa=k*O+pIRqU6yhO#TWEz6AKg3?S9)ACc?d zG9Zu12j9Dejc*Q4)<7CZp8gy$`NsfxZ2sRy@ekVn7SMvpm%+i;NkJk}(-2O+Bo2P2 z7}9=olqBf?3LuZkH^Ra1Lh-@hM;=U=_JjLXVY441j^wYRgh754kjLbE;o$3}Aq77i zgadg@z8enyDvE#5|J7LdNjUhhbR@X*KltGL%COm=g!r#@JWH^*!XhbWDTSn;KKt6)BZz%Jht|? z(?s$Qe*f44=)trf++PhFza}3Er9Qw0vzUBl9DFZrB%cl?58B@fBtN>{iU18+qU+oXRzYbgcB^e;?2frTy6UYa@ zOGB$e$Hhk+{8-e)F9!-ANSN{0iG!bczv19_q4?zk|2Kw$Yc6n3kM(a4k<{3l1HB~`A#_GVJLZ&8`0u_`OgkV{asP@ z_d-d6_5T9oG3y_WgFlAiTl^s-Gnn-c#=)P%;=eI0{3kg0I^bdhNWk>;uY}2ejDrui zLAD>dpMv&J0GcrEZ^gkkNAYX^MgM#JY{bDgNAVB-{(cGze-#HGh5~#32Y(R+Vt^)}u{;yPkgD>HVc|HQ$ka!2y>4)8az@L|9L8Ej%A49+UJ*aQ;) zf3!a%Adjv8zd`X~D0$HSZ9pHU|0QwoVV(PUh}D1aMRD-^J&^VvT)+Ip!oP%rAKQiG z$01LDj)1%WmGp4%|#m55*1L+7-{Lc}SKaGRm zm5t=np(H^*_B{XBTUDik01 z^T59nCjU4NJ|yRF`$@6zrE&0KC_dA_=zrS}?mLdnemWFi5G4=#pBxL{2nSyd#aHMe$kxgMS?dzXpr_R9N^4IQTXHBR>`ge-#V=2o`=B4*u%@ z*#5;h_^@1L{2grnqgeQTIQUek7dQV`{B_~r>tNv@!@>vmy9aLn)&B+mNB#;9egGCe z9Tq+f@BtYcUkAm<7=Hg6e<%TYZ2hkY3!ecCAAFxGHhut#&-WkkFMxwThJ_El6B~2< z)xp6pLh+Tr${k3U{ z5oF}mm?P!E`h)MHN6VwzkLD>e*?+iMtuPW z@&&N)xp45`pf(;TJA@`7&*=K2@3P|HcLgB%a>tP+LH=nhd?_4!*9xF8kaT5{^5`Qb z{|pX3RT7flPm7cU`9fIua1-O-?>6$`;8rlU+}l&XnA!10qwzLgCxPT$h-aK;1%#a zK2XpTxP>VX{=5e{((f1hb0AJd%6~Zh=hl7((|^FnC#>?#70CKO1nfJQFy&7R!v2+n z&WH<&Ns5RF1NVhRg~Wx=Adff2=Jy7U{k?sD3uT^d!(v;%^v%2On|w&z>phS1BE~#2(UXV zN(U(fs0ufS8xIWJWE0r!9d;Gigb-8)2q8-XCsK%_g)_(pH>XA8L8*OhmX_`=E{<*j zsBKYCA-7leMwzc`ZE0-}Yzzr#-rvj;)XnV%>=h2k0#@z6tAYmx?q|IJ1gy%bQ>P#h zRK3o^fERwCtOs`52DZ;Vs0S#G_VPbYLfHy<0@#}!c*6=z0IeA;`%it)E%Cq926%t_ zpK1VV0gL`L1OI-@SUbCT*xJFs_iw=5fLB>q171a&2SaWy&9ncp9OX%1Ye#3aCt<*@ zqx($@cxm6G|J2jk3I?>e6AX0NzD6FHm5a68KGF%?j~C`{XAQIVas^BU%I+5f%tey{ zKmOxBlwbY{n0*#*Kcn^<2GsuX)B%70kplw99fo>k3JlnQ8rW3S%^D0JAjsY z8U}1z3~ZwhHZ?MI!NNe2zbJ30hWI3JpoVCF3$BFa_09Csvb>r6mG@BQ5I_R>{|`#+jKNW+9TJG)bMf$SCkhjpfBDLh;sK15b zLDUen@JItHtw?1$p&Delci=dyiP%D3DGT+?vjBRa%u1k*Kn;az)FKV&l)!pMYEfxM zD$AiYAJD3WhDIs@Hbr~kA+;$1wTFk+s3DXkpdlZA@9hcgAs+xLVQ|Ebnzv(rB z7y%sU3syv*oes$Z+W{85O#lrE0zwQB1OkE%SqLfhr&`dEA_%es7+YX55cj9WK)vX) zfHv?a>VLFpvMZq*_Fyr5Xh=G+J_rCLtvxt_(*Cns1+dQka|4bTMe%@mh6CZGEVMaK{U^@gKrh2t-zPwSI-uyi z4Roai&?2Q7>BtUWlZR3P;|B0y5ol*R8|bN8Xv8r<0N$j;4ww)4G4wESiS8zvhy`RX zQA0GMDg-)65UlfXT7y)*1O~gF1xSC}=yBttm(2;-XnF6$zgNS$Xpbdyo^q3-#s0V!t{sa_k z1u%YrrdU85p@f(P8j%i;ZZY8cPvyd!0U-`8i z=x$WI&71A+?ZpCCzyS{;!w-lv&9u15_FrcQ<(x7^D69PSLULGh3WtM~dT2ewEZ&>}Vafoc=`U;cQD@;9pX5~#W(2CKm#7Cnsq;{jyf0s;)^o#?XY zFmeW*0~kWZ(}c3(9Z*pW#bC>WV+rQ?3^;i)6{GB*;R0)09Oww>3M>J}{D0~A(*^!y z=AZu2vF~3!IHDIG{1pa|R0pQN{S}!NJQBJ50$T%E)4&n#z-W(_BgokS z^of0OI3kVx4RBKv@g5w<_dgk<@yLnrFAu9Bn!sUK16WTfq9*xBg5YgDXb2rJtbl7h zH>3vnO5hq;w$;FaF^>*d2SM*<_6U1m;|FjOe1|j=(JFq^<0mtgVde96d z#Mr;qCy^RxzXBr_P(k#n5cC}|To0h0;4<$ib&FQC^Sz%a-GFoA{v#>Btc;0@TR0Fkf2zivRR0XVmz zycYbGSPjt+nCk#EHLy(GlE1j;&JNQACNhcr)fl>5Jxn=t3K|NoL6ssE$(4cXhJ*tb z!RH8}A>iMO;E`^?5UbA)s|GGmBVA~XhdU6*Kw#?8MUnx4y%P2S;3=p9V1X2FB;bKtkB|rgkeT>u@i5^#g@B?dratzRj`@_ti;k6d% zd`LMsqAPd;HLlIUNxDGygELLdd<3Zb&wQzg079k_Y$TIQ7w z*nzSg+A!@;e6?%0GOdk`!!1@>h zpD>`IT0o)2hfrn+Ag4q!0S%P`Z#4ho@8B`u{unfr9k@Sl2pmxBhW(IK0dJB5hkwcg zpE^t~19d}wFntfWejD@^=qOY9i)uiW10RnY0q6GVf#HcXc997P0@VB#=>~M8{bd63 z8Q=##U=l(*6&U{j^9O*zlvt<+7+8>>edT~P1>goHpdUsOfY0_9LJKCEh~JmNo&}WJ z+wZ1;tAP(1A*bZ`GmtSwu*ZPO3-vdEKi6r|DP+F-2DfTfTCDCRt1VkCx3^iXF)0wU zOp)LPtr4#YO2x10Oq}AzvofM0UyI;oi5Lto;N10v^&INPHyE@oFf29U8}Sf$V^{my zWFcSq=rNP8S3MGL%5VDK^;vm)|8UDnQ`MILm)UcplA~Mc7E9b(>kf5KTB?n`vaeO` z(ziTaC5;MoiNhbIWlMKqYax3<-SUO`xp9HTh4c0^SDU|C#K@=X^EcI93Q&*!aGOp; zQmw_dn5pz#71ZY3DAfAFz^MNtUr%o9(iw^QyHjW9Wj~lX`pc#)`pU*l`pOzXJXCvh zDt*?{({6irY!#?4DH4pG6YNokCp(+ndqM1bN7r~LB$vbDXCo|IWca>Ebld8MgpWmU zTk@9+I{egKH}3y-aga8BzvtwTb$P_UMfOlWjj0MTweKA&wl_gkTWogtPvhZK1~eSh zebnCJ3y4Z33MC$;Xr-r0T1tM@PH~f$;qjbxi|hhyJ%?hdLz`j78u2{eJEtlVxHXa% zrz_IAIfb9!U)ezvhq`1rx{+OY*WAzhG&oPm$~1OG`!Vl0v^CC&E1z>vOJA`h!;Y?7 z{_!|#7tiROaFs3zC{6T{cTLv*E)H2#859$|4ZEGM`wL1AbJVO#kp^6}XD&V3gKueN z4EZDhbtmE!q;l8ith;JCaf(mqe!^9Io1AA<0mr;Vx((B=GWC8x_3Uv2`vGQl9k%m|k8W0N*_7N8nzNZ=_31f4o zijtdSr$ZE_XHr_6skICFMfe$Zf}JkdHQb!P&LZeKu`2NbbqtNBoP?Sn_4v_R_7FEh zy&k+%RH7jk-luL+<+9oZ*_~awnM?1d(?!)w5$9>ulGKG?G=-OE7?A(%;jfKHyXqRD zZUWLN9CsZ)9@*eoO%6GKUYH|GC1=*BW3Zh(-rz#QVCe6)CBIKhzxs~On)d85u_TXC z;m;@h771|=@|S~g^j=XVrrD*zw@>>`q`*{bL8vSHD}y?M#1A1?o>z^~(RL*aJgO%f zp`)=Xj&J|;_q7lq!WGuo4-0z{f`~B*{P`Dt_!dO9E>!U3jVS# zHjDmBK%r^_QIkCy?HEy$>X22IT5CdGw_l`nwqA}4DRO63*~2?%yu(jXdzw&qRyXfQrwQ3eD!cM%WT$Yxs0)V(`tpn`;9`? zto4(+`c^;B3#UGg{b=J6t&(wl{T?sF%s1DS#!c#Xz(~D7^+FY5StN>Qoxj0WOlHw; zX4|pGmQ5yg^(|FK>*_#V@TH~Xaq_E|b&dCCY7>~((_DEyqqP@pUq6wq6|ESI;gH^J zQTG3u?0B4_@nO}%YMyaXP49$0qx-YF4yKRVUUfdz@qW8SY6yKV&DFc@DCPn<& z){K)4>Gn>ke(G>Ptb4duJzG21K;q7lrYGx?jd5^EhP^0 zvux%Yw>2-rS#r6$6YG;6Dny7#N9%tmIr%hYK6%W2^;q&vSf9u*I_Vao+{^DS-|&mp ztcXiVdZ=jJTyj{oD$3qP{A<9uq|MNk*7(Jal2Is!NwS&JlY;blj%>_E&%)@9THC_l-seDR8NM*m*lvTZTAK{zLra8ra(vtP`yadf# zwH1+HL5^o=ytSBxn@=OYv0Wn-mVkT0fHbdRdhJ z5uQa+7rC;9yCX4Cn6zYKGAXTUjVh}MEkkzFQys+BT!mnv?8p7A`U`gy!|M=TRD0K$ z_=1QNex!8)e;f1V9EzY(m?l2T5W*X82h&gX;TSIu;>qPPSRjsh)s{Sa9rsIj zrmwBy!k9OA?{g~I&#Ixz%Mn8T>kRsv6Iyk#ZAH143FcEPD)>}FXBRa$NhwLQ76SQ? zl?k=2DET(7YMv057a(oESXuhxaEg!ub(xSZb=d{g7)Y(1RGivypaKd80MlVvqHyf1*?^Ov&5(Nq#y3%;GB*Bx)^u@3c9mN}^ zqX>FE`$%fd_tMH1$wKMkbhT}Q{3M~)Ds0bXLaiI0@#`ERZ6&`^$jvV@6MGq!q+t^{ z#z`2_9hFM`JI&2i7U*t$ zLa$l-L`I@N)|A9iUdi?0CbS(qn%NwkCm{ETv3my6;HM}Gl(c^{%8=+0Uwl35qX|namE;NM zjGVufovumI&)oVYEdm%;7!swLMNh6%3XKFXxFyZ20KHNqE=EL*HQg#D)IGvpyhk!u z5`H|%j_ZomV+tRHz08$7YJo9EvBxGq)T8Ib-|4YA%xH)`R{m6I453_>v*;J5e`H#8 zH@_uwT}FTG#|uaC*WcP)Eh+h~*bqYP!ty%b@^!RD)EE$@uO6m9RDS&F*pEPi-H&Zt9;FCKBB*>AA!- zB)YH5Zzl&d(ieW-y7sBi!9D1AgpC^S+3uC$d=)OsUeWY9(cQo>%GkMRm5(xVkE0*4 z?m!TJvcjFf#X5EKPnF}a*xKk*ok8>^4&JMb{cPS+E$mu%hM!T!kybZ-jhiQ`_tj-TB42t0+)Jiq0=w$TeyB*JMCzdU(qseL9!v3 zU9-UQ=-V!_xZu+ohZDn=DYo)+J~tPRRw#^@>j(55S{`U(`vS}MJo?5;EOXoYA|rbe zuh&tiPs@9!W7D$_6C1`dwzgIwuhbI8##ZYBACXJBW(?oP^FO;KBV0)4o7S3yu&VnKa+Ia|}{n&cUTbRlakj z*{8qzL({KsOeH;Adb-v3-sEql(UX%9EHm5IUw-d;EJA%=wBKM^_hSc)`?^xi^plM( ziRy+90fKBP!`82i{#VAPY#n2HS|3rIvo6#e@KsTJdyLV`M5$=}NTww#@ns?@!{p0~ z7adk2nllxi44N;#AKRM3Gv}Ea^}$zE*cd7yJx|B6bL7kqX>CQBMS>14#xXla%i03v zORejbZ`8uyu8Ca?H(C?ZV0+}~Mq-^?6oS8zty79vqzNh3;@777Al{u%dgxi{CVNBQ z^O7e<+PjDMG8FIE4X`}Ucb!<>zCiu?$F;bePcJzhG_BAUk9Fk?;>Cr2QFv@RChE!F zTnslpo#A$0yk+)TwwUpitL}Ijjwu>fp=&q7)5nj*_lpW~$Y1TPe@uwbqG8lWSU-#B zxtp)he2G_YGfz)pIz?Vq-@ClD%;FP;Gvex?raVhrN@?bu5|a`8>zaHsF{R;DNqpx& zFPB~)zo7k;eTT2jexAp7qM{+LxbUozwE%;^p#Bs6Az+&rd`a65S4b>gZg$lJ9 zpLg6+-gcE3g3LY~VNW~NWmWjBXR?u-Pybe6&rsfjsasdPUsmX@czjuS9dntyBlk27 zC+x|9UU#Fhy*iNxU&K!dlZ7lQt)iOSQL{7h7f*J2^AyK?Ijgqp;Gz*X60lZbvwCBT zL0U}4!1?`opKGT2WY@{GYhIW0a3H2_BaLlidgUx!eWE>`UgYc^JJ;E>$uR2V6Meql zOvCy^n6rs3zEgVk^U&w?D_Iua40!Gm46yn!Y9`%A@#Z%FdEx$wBo={w#F}*=++yvsBkT!Q{_9TX}c2y5y4odp}82jqRO}T#>o{`Pol0bjAs2I|^0{N!4CchIr6k zB{e-$bgZL4xGa>fk2T96DLbGrYAjZZSwZ#6>-*$ihSTHI++SR~a@M*g)Gw+vpr<4B zcluXJ-D}Xi0kwC$T*r6UEfu|JPE=Aj9DYu`HSTT=KYfSB$fTct!3$;~Kv8dlB<>sZ^|ouaH|^ ztnvLiVFu5lW#VE*`w@lG)=kl$cAj6me#2jQw$AA>zA_1`S*(4OGPzk~^|gn}c{8iq zS@I^7%Ga-NtN&yHMUT0fgn%4e{MiYS$|8Q(`U#f60)e~7D%c4%X2LHo&A{-sh&D4&m6%MGq7=AVZb-sKi&{InHl;}nqJ`~AGQ`sdco(^)c? z2VVK@`a|b4{jCiy_Emq`Xu0zKr~5$8d792d3Ay|f;)&ok1CMitzohj_1T$%@Vy zFb?ziXO>spiVHJs0Vk3!!C(pcPT~2;+Pee*J^lgC2xa%ecrd}j92ClIW|o6PWtvT zQFKdC=dbMwGQW38SD*CkNj;`ac|<@QV|BP){_{jZjl z6RxnWx=H@NLcD8==&p;;(Gcxkj>;skB--^2doCnM&T9}H=<>CK|4nVIg^s3)V6Q4RG?olqhG{S zE^Lr~t=m%JJ;QK=tbB00+H9z3=a5TlmD7tmFS5%*0yeg4Zm&@BHN2yR)QsMbc$QD` zN%exPOo=Rk`ny!{@0k0^7X02ANa$;?G$*|wT^e~R>GXEF?2%Bx8XJnFDAS-ab9V#y zRs3&>w{v(8?S?R$Cm2sZ*{1pF-C^x1k~-m(|77R;yv!3G#lnqOhdwG0cE_sv)+S5Y z!tw|Sb!7VjZv$U*2$MW(pFjNTUUtT!kJ+iNr1!fJ=ne7<0nUy=2Jfd_(>dGnAF7!= zVGUl_F^66IHfls3ayx}n+2eG}l!OpZT&Vt|n0yLVM?IJwIqh3&@5m=Sfmk z%^$i_;SI%{g+=mLyK`MG?oT+X)D~m;o~obE`(-EJ^X~-4hj=z)@AL8;_sxF$=(5`3 zO@wg8Ac2OQ7a@}dg|K!&Op{XK#u3%*){^jcMFeja*KJLYjrSIQDG*v|g-mx}iIg`Y^M}>PXwH{WOgiuhel!XT7wB*TXMa%bJ@)KkrxPLE?ej#zCKKf9y@#E=ncc;~RByZHyJKi1@S#&S_ zqUt|&WR_;p?$LrRm*6VLMkT)Qe1?f^s14e*7VWWpGx6s8_=hsv`6FBV->hX$3~n!k$FifU#pcf2ui9x68+j?FyA)C zSYy0IP3&o-KI1nwBH6BELWMeN@;IBK~*(>d-TFaI+^R-KDJMF1ke(N*s zp9>L8cV~p8mStb)jWeawUYC9#F!{FN63LaCx4f1bX zI9giwz9-d8|BL3@iv*%j0k`!{Ql-al2%nTC)^2g1>&(6y(Iaa2@-vw)`<>n8%{7** zKW-9vdpF?@O~qpGrTiASWhX|-3>>dkvgk}Q0=k#E6#xX@NhbArj* z{dq38HI017>cS)Ixl z@#aKVk_m@J76~9950?^=^fLfoo{q^Ec?1pp7CR+PHb0=$$$G0)u~XsJ4lHn-ZO-W4 z!=92#c*pEhW73b-VQw<-iDjKbLLu81Z{`uY=LK02tBE^= zh&coq65C(iqP{tD(dmexQ)yDjYpNltVyAYNXJWS8cZwdR{e)O-;MGcgm+Ck*JX>3w zet7usiDe7w%lK9S@0+W%*`8_B2vO=qk=(bSob+P29U|7_RoCG~q|!ulvr$Cw?873F zhj>B{b72-*f;=HD-Xwc>y?FwrDeP&o0&Bzc26BQvDvKOxxL{xp(eHLwLdZbw`NY2= zfFV>75|Dcy@s|i-PhctTT)J&!yymj`a;yz<*&g_g8Y;%`B5yUyHZ-zxoNgIizVLGH zLYeC@q-@pJEIz~gntS}U?J6TzJzZe~*P!xx+>Z<4(|w+;O;8Ol)B8+vG!C+|Ydx<`(B=mg?y^P{#zID)&lC z_lFC7JO5Jrj$w0}_}PXZx2MSx98|Z(w4*<7YeyH|AJj5BS0LN>bThh}G~hAqwTMvd;}&SFJG5f2DuXNkVT9LTVwc@QKX|Jg_?J zWNMv_Zxo_q3lsp#|0SgmC5p>e?>FNIru7Xq z)cvFww&dZ2mxP;UwnG&|#cXXZB`RJYn6rp?+DV8c?Q>qY6($(ka0qs|!f>+qW{qcX zOjt?iJsVHX{&f1lul*I*34g0DZum8{|qO->Qdv|B-6-f z@uX~&tbzGfAy zx8QGUrq$NI_yI0T0y$-F9kQml!mIZqw0KKX)kE}IvyWWY<6%iSqRT^{+}h?^MV5Rf z&v{yB*Ab}jXvmYGPcIf$95!?OSW7;pTuB}fSiyVU`+-f_+0NeOKJCYQ8*P{H*^UX^ zG$d1IVB`rdz7?}1pe^uCGmOa9MPHkD{`@@>Sbo;kqjyOj4tT^RZu`+auZpa5h260>bJ+DdJQ+fmgg~ssJmeZANqk6i7l!va>!MVQxB~f^ z#9lw+-*68@#avFQs85QTpC9xZhNnEg`?$e%)TEL`*ji7LL(rjIrj)aG$w8OiO5I3u z#8yXSX8Gh7)l`r%IIz5vjOXepYQpux&Kf&e<7K2DmrA4BV(`7mIA zc04#g7Fd{@n_q?CQ4(3LMpK8j7Nwp{$O!vjZD*}Ay^}S+Vp~HAk%=Tk{htu@yefyv zps&5s3?g4#Q@8*LUAeom{AJ6E|J3#i^82-dyT=^~dOwu&CGvEzvZ}Pcq7bO#BhX?f zol6l6x#*}EQYIMU{dOh$?n;^)SEyhe!YyhWI-#}JajbN6K#2QxLS3=^T^8S_tf1v< zN7C(u@9}Bh9PfX7yIxeLew_Td3(pJMw<&hIjbS+h^x~zHNfp6to{{0Bc7|`Hbd9n|pYg+SmtIxLX&U&jq9U8f;dDM-yKlIDqZpmPdkBYIiI;EFTB|8JZ=*{v+hof_T z3^kVSa2T&Fh*vmitT&4DwFgM)n5ny*OrZ`XUCW&$RGmmyG$6&SNT3)Kv-a`km6;c zIM<>%rz^#6_>fz+r#r*)qu}SHvF}CsGHiH!0tgX5!-@FKLf+*M(axDO-1rtKL-tna z?b9ajrUz7Ru~PWZmp2KGIF;A@ebs{qO%+0I-f=btrfobY`sIFVd)y*u{ zF2_)FjaMCx?w_CIU5UEF=R4}d@ynH{%areqyYB2%9o;)={LvW0?Mu4$>cqasJVrQ* zJrc9|y0oe^D{l~bi%CopzZj#V;zWp)!!s1SS5Rs)%^wDv)GL@EE-!xmyZra z4AII}vz+@ypPl5&BD4CW{(*kbuk!&M#}1hT^@q-~yM&WvU8nE47n~k_{|B*e?oIMt zJ@N+kUiRw5^_PQuMXv^qiFIt0e}?aIuAk4NCE7cD#FC`l*l2(vZny28mgw8kG4iC# z5o8mv-(ok*$QvKPDf;x6-N!`OngmTh;2`xpbW)i(oc z+mW<4^Nk^w%az};lfHVXG= zj_G&*3(YeHH$pD<*GE!IwabaGtSY>S5UtLNzgxmk|K{lE4IWm}<1MrImkj)V=p z3{k{gOW}Iq$NX_tKd1WF=sf-Xi1gPDnbAKzmTY3ao7~aA$-yMp8(U`kc3iNY@wu9! z@J+n4lYWe2kE@9YUeKIL2_?U#lr{<(==J#-5VUmRmM157m?`_ZKD=3^o!X=yq$%+a z*$>}xc_+kvv*J`>y0e+yP<^`}OKKM(_vZ#H`Y?3?`ANC*sm0VEXMI`nf` zqK58A_180|hlGO)th{s`hV=rhSc2X!r^eS{Wa*tXoRLy(BeZA0w z>FY9Q`yH>-Wr!sa`g4l+kFJLVF%yOgew>k&=%eTQN%=EsnqJtII1}ETK0dP(AG)sJ z6si05-kV?ey5F60{8@B$X`h@q)zbHaS(W$dog5gkH*a;lW^+s119n#ry7G3ZA6}{U zRO_0GUqUz{zFqnKYF+ri=dd4K$cg8l@if$U=*E!Bk%lMRhvOq&wHQvbKt)Mf_%CEe z93Q%s5){CKpmmPKgWFg8o%ie{haAz)*L|dXq(*aVzV}IIQQ@G(#n4mvGi-MA&>z*e ze%ZYy`2sD!uo*33@KoH%u+hmdf%j>7ai3OKdh{jP{Z8lRKv^pGZf5=Zi!vd1ij)$-uid)c_zfWR*WKTa~M8< zy6!_tsgkhW+bpoa+xcy2Y)F{2Z);;lXgF`A)h(~IV!*kYg5m44zWgY%lU60g0g2NO z2O3{UetDXr^kgl4LZCeb_LRX%-&clcGxCLZLgjR_{SOMM*mn(`!yVOOk-vELw^|L} zP;?kQr44155I4A3Ku#_6B;Wo@g`$qw0QGpQ!HB;7gH?PgDc>gd7rWcz;>)(JeSS0_ z1R~!=-F-8bXJWnT`!!Jf$e8vldAE-D=3sp@%cup7%!|bJfVa-&f1ymyf6pq=XE2(w}>P1+XnM59!=31x*yA5J};0sij3^ zQxmy-8tO7vhVZX@`-%ph(tFE!H#Li0k%KE8)y)S%y{RqbXzM6@z43&E{Izev&GVI<4*{%do`$q{z@n;G>(sp8GYl(@#=QN zgx{r$7c4JZU(EEn68TKI?&WOu*=tjXtt0H$PjVUUjBf{KTy-(5U-^*J{JkX~&x6)& z^J)J}AAFhOsJ>i?{nHm8!l>r2>+7Vu*%TeoB0g(qaI0b@|Jq{5V}TdyB-CHB>nmqm zPAFHiKWJvy)0VksdUu)YwM_GlQ=|B}<2P1+S?TW{JU`ym8)?u@F}`7CxRG~PH#j0c zm7KlHuS?>t@uOySsX8TB|N67lgkxKh2G-7jC#;1M|M0u7J5JPT|!*m zz^t2EUy4J!wf_7)NHUSKq0J#VVO~|z*T&EL4*7I%^Ey3y{4vY*rAsG=!dqU%OZAKV zUN*fqZ}&>pn+7IEQXda^Qj~gGmHWZS?&8&7T_OSI>s@v#{j-@LR|>gw&l}0tYE9m# zqczpZTd9nAsysq?Qnc$h)L(tqGuwcg^>P~U*Ocd)@qV8XJI%)t5QKaN)#O;k^+d8X zZ~Eco6NP>Qqoa2iyOxtPc+C||JT9!^UE(i$b(+uWF7NYhyBII^&lh^-axOfZBjA^v zhrBG#nF;D*^li1`MC^AMw|U_6aNj~pzPs{4?iVF#we?7|;4@2MTXu%$??z6~ zFrTF#(J1`%sv-Pzz{CtBM(W1+aIAL?;bfh>=~)H7d&g|8yOK>8Q{FEW2lBdl3A~ZL znpNL?_E*TYl(B%)02AIsVrcj8-rY`F+1tM-A9Pml{+y>wKP~yoz_9s3RQ2HVkN)Xb z-MYPh&o6J!JO3Jba;`VmpX1HUGGy?Rr>(O_rch!+}E#du!uxF*xqj@|P*=J693 z1$2yJIX5|n4sTQ%b-rH`H`NiJS>z_2J$IKRq9XL0->ZA3bgOcM7d{XKkrbTCg+4mh z>Mp)e%Oc5gg_}C0v~pQXKx}KXG;wmxGS-vNS;_7OvvO&L@3Tk04QDb}o$ZGe+$yf% zJ$uE>8+FW~%$Kz4nbSGNhM-#zk-Z(yBU~}3jiVb|YXggJ)*RJhd99~6#7W`gt#=Di z6uEn@J@a${gK#<5c)3#=ehYbsu=1@MX@wIXS$c@*w(faf9di4AjuW4xB(&=OS#N6z zZ_RhrFB4}CxVGgE2W+&cCs*cwDQ7rYvmwN`5VkifH{ozTFqdVE3T66P zc2ZNawU4cM*ZGuODhQvw{dN3uV3)0cd)9B4!-qm3-B(SG`rkeO)V&Z&D5;WOs(6+C z7g4LVI`>9HuYv|eQpGG4b?clr)Re!>B;u;kYp(ofH1&n25FUAqagKEP)YOqFZ1}!w z_qVIe9&8mfCdSZ?+z)+t-($G8i?p31>+296A_0#_Rdq4MuXUW@!^k_S!o+Sif)DlK z->yZu-%Ii8W!8O`8!^i1=3YJQ=qa3c{bFnYHNS~;*SAX^562A{es6pitsXs_JST!U znP4v3qm}r{=aIgdWwcc!7PsTEa~ zTPLoHMt&N4Ap)1ncb^FX>-xBdYX%b{fowz z;*VuySzQDg+ucr|c1EZ?{MH!!Y?-B17FW7GB%4k7fAn`Ex^svaZd18dI3zq(C(=50 zW^lRB#!+yf#WU33QCv;bgZ=pE`Rle`jlj>Wocq6@)8%y^)Qi{Y6$wwA)i#uh7C$p; zzJ_nR!{IY9c%~HZ>YJx;Yy8O$^XA1w*M4{+);mReb3N;2!nJgLn)1&^U+|rbKV1*} z0d+{uFP};CDhNchsdiqVw|)1CWPsgH<$3HS&LNHyi_THeeQT{YVJAam6X!a$2L~lH z9u5%Q9R5BnhF5q}<+Xrl!L(J%g@L!+R#$E^y}SL>5LH`MM}x2dzM=~MQOU4 zO?aJwFHxSpkAls41LvLNO*+d=&AVDbC#QFuIZn9X89;A+EpZB>k|Ft?pzZX2 z55~7U<8DX04MvMNO6ZDyfT(}^QPlgGw${F;4zKFRv-Q9VnBmD+`m&?mLdOd|t1=lE zc@^(^khSDPg-$K)=b)Nq!RJ~P~zw>|mYp@y*) zl{(?OEw7zO7t5{(_xPTKbi6rM!a_JrP12YaPJUm}nV{tNX*w+?lsgi51gr?WC1D*1x7;e5C0%`ZUf|t}}-i zIX^B%2IeO-m?d_X4LH1|GmmXA|9!rPx#mQuy?Mshtng3EUV0Lj^FC`Iy}popkCAck zwW$Tid~$8SqWR&-IESiivuCYRl3M5N;wxUHZu}nSYaZz3_yB<)aGBw~hAb7N}>{E!wM~G-V+Y`1jN2-rAom4y&h>!QXCeGbexf^3hp2 zM>U_^XZ%$Sgr$%5g`ZH^%g?R!*Kf4wb?To};m41OkGgUT#*)0Hk~LYs&2ULADCeiv zj}|3(QRdorLT%3T_pg@6g|)J2rR7o>XlF#ke~)Y9**@m*D{;VxhbEwsqS3K7oGz|^ z3>L=UeNE$YDN%4P?}t+zcdbMplHM88Ja?aDvbKd1-!fS+vBFGUTMKraL9j2N$!f5y zb#+_pKB=qu65FTT({{?1r$*Xf{@f#p*1nP2=RIBQ%YF?AA5* zJ~0o%TvmlCyHXBK3QZHqZ)4^;KQ-MNRcm|SY-Bl8u0A)PWF$?{4C~={e)7TM`m4SV zueC0&>36VxI1cs^Jy}q08LMmw{E(W^#dZ7%_pREiv{7u2&D5{= za4vkl=|Xuw+1YfQ6TZQcTe5lQ(8tsEAyY-?Y}D=UPaG?b^=dFaZ8WG$VqV=#-ah?N z@7@XPs0F90fexnk@UX5c)6=gicQwrX9z2lUPE*-TS$rR|bjY#W)$We3&sba&g6g=a zd7^LL%{-}J<4G^2$2R9$PloD@xJupPT}@mYO>fVKy?zde=vt|hsDeym_JJDxG3Of#ug+8!Ix)s^WWV#LVC&9MB%9}A#FM2An3}V9my(e1XFvQo ztvt~H+T#DiMy0@FCY=d>=KMA8hSwQNOY^U&zl|}^1gRfMRp7bP`E2JxeX-}N=Iruv z&W_{8?aks}Pvy@_K|`WkqefMYo{`-7Sl)e=@tkaAN^<3V)nMF?@Zh5FvG(_U5$9K8 zXl!|R!o%9W`Fu_9a^(y7q_s=g)_`kBUqa)=Qf{>{8@&v-#;7Q9P0|pa`EX`(1XNP;gD^nU31V zHP5O3O|P|tKt?Ce;8Sp#~3T4ygPqCY}Wj{$33j%D%0M@tE{8#TAMuR zH6i-K8&%2hDU(J&jy`K}(<|EPTX~(4d;OQDM-8cX?9l1<;Cm0dNQ*9%hE-fT{k2z* z!Y_XBT!(4sHcp;D*KY9Ip>tb&d()=IAbIj&@6OJh)6MF5 zLKo|v4M$bgoNux6&d=+jSLhFZN&0mjzo~a+&ttZFw{?^6CqAh8px=7O*MoLk-n=+3 zx^tJFSNqFTG>s3xNZ4Nd!P3Ea=c<;s(sp*qJ9ukB6WPjflT16@>tgcMI5Bfm;qg{C z#*P@jt< zlVRE;w-0_htCGKE4=%j$?!xy+ufupLgiV!GkYmY)G$vYTNXe{cleT zH0aW&aY2@r(S!3(8*P8G;nCi+CcV?%ZT~7Uhz+%nba!{}B((^Qiwz5IOKuTEZiX^E zG*oKmWMgMz+t%I^w#4QT?&l;W+w|69-Fx;U_h(Woq+gj}?W6|#^xjJ}v0Egmjqfc{ z8*8Z6Sg)q`Y$7Zt$JqS?KhVr>`i8UA-SkanYIy2Pvo(G6EwXBO=-W=y^3d;)RQy=h0Wp4DI zvQqM*Kf0wUg8q}~8>Xpu*SA&gucL1$)7O^iN>T{EO^=b3L1Xn-nS%X7nKIsfb@ifZ ziqxpBrk|-U*Pl&Zx#}MJ%hmhpr>IXL%#W}>}3PAI_vc z%+?_F&eHVIx0ptKn5Lz^NJrnogL9gvu6mZfg-l=S$=pbE1(4TP&^L>0OM22aArdn+ zNZx5wUmDdnQQyK{Un*lcJ&&EdP%J0K)R?9@Q)3o!ltuSr`cjXR`X-(iND~$7Yg1)^ zPP8~p$?K{=9n}z`?cYZrR=XkcOnrAEfC$$b{dc|Ld5Br2fAc1sak@)o|A1 zem|@s!FQ4sk84Owe-|WsGJ(LCq`0GA!128*dkDOSqyr~QH~oMkEa@iz&3{=m2)LN< zSFRx7fxnQjCG=YoC)XjK+2LqmqlRQQ1Mr>zoZg3yPh*80Tq_AYLf80T5tJ&(Yf9Sk zO33zfVZVmt6yn*!zNSR9>({}#Aio^hvj8}~UoxMT0cYbNJC+AL1<9`$xTeH`3BvnU z;OU6(29Eu9K;Shbe)L`fTskUj*N|ug0*K~xq^a}}F zH_3lP#OUs;YPdS7|DWV*Rm181p#LNxVGL^XU&HJsk% z=}&r8)$qC1a2l6?(zB)-zP%b=QVqXS4S!k4ewSBR}gN@-DzJO*T}P1s*#@{$p4y;gtZCj6S>vM@2rN?dxQL`UJnHQzveYz z-9bAZSp)q^-lQ7dt{N^Q+?dlOtV4+Uh-%~|49c@V`C%Z!bvMiT{P80`5SMiDE)NI|0~r6fz1r z#Z6}c2>R^>L3`n#y^z&j&}T0m6(sH3O9Hzqdb!CIfxW!E`gryY>??Cqcn1Dn2Xc$Z z(UQc((BRkzNn%ohaEDI`+$S?IEIcH6xG0|#oH#NtF+3@d+&3{fI!O`}9335p{FIas z9Ge(He3%p!6E30m`Xti97aLQ*OTjSMH6=p8Hrg*#XVl5>tgdh17eS4j!I zVWn6)tx#}GU{YLQ1hFDCGA1re5*i(s7%rg}5_@9eX?shOW2v!{n2}UD$wcmMNmY{D zRuZ#`OJd^!No9kRqT-n1xZshJcB*zD>_^-?)QWAE~dW| zDlXm{k`qrF8%xR_9w^*THIR9jm^GH(;Z#&WYfd~J9~~7M9GDawLiPk*34x-|{$nwT zKPwO&m6$|IP9Xp3ojiZ%vEgF@M+PTGLMhR4aU)5q;rv9Ls;>1X`+p8|E5J{AGCNE}ZOOEsHNY@86QF)#Z;W%%9EXR5K zH*uV|^DxKxc)Q7Q+LxJ~&p6(d<8)r9iQ3tXyqLT(aGI?#$LYLI6P4%NMb2?oa*XK- z=eUgH@Q5Tqj~mD5aq{jQU&3+P7nuI79QWk-agOu;zf%o=QVstMoce*DU9)m^HOUT1 z*bj9%PS2T{d{d6|_IPldkB9yo=i`C>euoPIMcvO^M%AA64T^Ve+P*#GdT1VIn) zhxMEu-VfV3&f9r})8B&F#oG5AC(rxgF2{X1`KKJ`+wl{}=`|y!UxQxHC5ifhw^Ntn zyq%4JQ$LuK7t_;@ljrrc=QyuN#_4Iv>FLkO^YsnnIA5>GYIq{Yd3z>soVOe+s2<+`Qv@naij(X6JiO4{s+tLPXH-$I17l3P_^m z`jZ#aGnwPPIsP|r`bM8;#pLOoJZYkO`1TdJ3>EC*?T1G>3Y?ECIw#UZ?e9xo%uW`cl=tI!I@zX)^8Vz-iNqL%gM#sAm-7n&8hVh`$5>(CgzgF?;f%eg8(h13d{K zX%*t?6Zn&zw=z9R&~YCj`NI$o?79Jy@1!M`D-1|v zlkrF>S0C(U{4tc<1aU*S6l{ffEvOg0-c1wJqX+Thg?J$JUp9X+`PvXCAxM6Zs8$k> z_;$MJBZpeJ)A?Ld4wv~L08ADWBuClR+c6Zv_>%|QP{#IHbmy+Ygy>iZq> zY>0myy4WR&`PmG{bM}mS#?@gQGe`0=7&q(?H-mAeE8>x0XAi`Ez@7la*MZ&Ph%bWq zE&*`|sP7cSi=kgEK%725mZp5fr?(LK4#X9(9w{0(q(#8Y7rZG(6Y zjJGbBgZ*;EPeH%yhxi(pmqQTm3jI0`@$2A+iHJ`G`;~}WLi^4~oIb;wrd5ddfqu6Y z@w4FPgNW~gI5~s3CCFbx+y&b85#pK9Uat|~1M}h!#4Ev0cD;zjgHKDbUM8TA@$;}= zYKeFc@UuPQk}TV)0pe_&Z-O`*&n*$J-Ac668F4qT!yR#Uz1|P;gAmU{5q}HuFcR@5Q0_RyE1*A4 zM|>5uR}SLPERvOoZvpu&h+lwl-~i%1U>$WD@y9SeUqw6@#-)ddr$RfvLVP#0<9Ea- zLVa~$y~*0QCCqmX5Wf!Ni8oh5w8dRr3d13z@7la17Um_gLqBw+a$zI zU_MPl{1_XD5!Zr#Hw$qc@LLYz3RqV!Li{`ES%LTr@cUZCcS1WBBEB2?=}E+GU|u_i z_){3)E+IYw;`|2U5g`8>acl6~N5mUQ#Yq2wcwgZ4Y2c8=;%5i6OB2M$!~AQ8IQ?E8 zO%{k3fgT&g6JcC+MEpL)XD7r1!T&PEqhNm^;&ULL0}+=)`-UTa5d1b0@j1{RMNyc!TxtlinV=P{CB0PXc0@qsXoy+-^cjK3cdXP-C! zK>Ps2rw*)-m_6@d{A_@@8pOXj;+w!8JH+F_|6LL94C71>#0P_a0uZ-@b`M9KeLk9q z_*z&OO+|ba%y+X8e*pDbf;juW%Lc?7vT+IVW-xyhAzlK0xPtgth^vQ)H-Y`vh`Yl$ z!#>Ag?Zxg_RvY4%@obQ9jQBw4FP4bEhHC+U4 zcq)uy|Z$Pl0i17vgV3HIie9CqO^BhKKKRS>28rZ*u_%s;b$`Nk` zdOjk)1>&j(_?5NSJFv43;+j&iUd<8Dg?4O*xDLcg7sT6wojnl028jiRw z^!H@McfmZEhB&)!I0x}cXvbxUD`0;U;@4sP+K>2AsMl%4n}Z*&Aszzl@fh(cXqUH$ zzlL#74aN!9ULjz=0pe_YZ;H4N__Gb-&Jc$k5w8h$bVs}~%**`{p9<{~ig+co?yPS?jxQI_OtabYZvzU z!FME|1ml%1%(qP53)-E{!;Cw_c-0!oE1?~o5Z4Coj(9^D$NUkW4gES0@rDprF^IQ^ z`c6Q62H3AeTo3&AH{z>k1xZ?sxbS5z!gnD44d&k?i06Tw#fXO;vO*m1R(wp;x+!4Dei`CyowgBi7KbMg4~B8?65?B-+X{VXT({* z(1L!?{LmEE19cEjgL0*aM?hSWM-d2#>FEyhT4%&VL@|jE;%^{M0uX-$@gISBDa20_ z;^i>Tq#}L|){V0f?@bLRX({5z_A> zv)@@spq-hY+2_dmi1&o?zX{?!;5@|&@e5MH4U!IsKLP)EAt` z;_Uux?E81DUJ4iwvyl8+*k6cv8MH4u&h%V?_}`7>VHl7cM_iZnJH**_*pG-W1wVgB zyb!o1j9bheZD?OZ#M$@Hnjroa)*+UNmx;|KaYme-f4L)W0_%sNh<}BC7m9cj=yy?w zv)>EFA+8B}MkB5Z<9RCLcc355M4XMEix6k$4{H%;=Y6{nXXkxK5oh1eE=HWi=WWE< zx~>9oZ)nF4i2noaq5<*9+8x|4F+#i!_`wQs_I=Cth_mnOdLYie5A2UP``+&`#M$?R z$0FVf=84IO=fL=uj`%64uL^N?f3%H=uZR9vfOsf$$P%x~1p8@f95Aj%7fBu7b zCzuyLBR&?|Q49J#Yp)ErE>{Qf?qH`Bac4N6ZHss@%!8c}9|rSmcf=3W6dl?ZacA%w zTNf}p+3$5@k^FL4pJgH59QIcs-V4@IhYB9~uyv4F)oQ#P`9vA`s#+;%wc28*zJxpSOsgf$>TQ>cQ%D z2jp8L-UrU(d=WQ+bx0iIQ()W}2b_jI3`t~*e9kKs-h}Uu(GUT*OO_xGVJq2y_$V>E zWDDY z9XRukEpX%J;s$G%IN;3?cL4`k0cUz-An(F)-cB!$({_Pylk`En7w8E>d=~I@;LJ`L zG~`T9o*rXqGbc~my{(#fWIN*QcZ`LI4+r_#3&seVV$V~@Bk@bIH}+`wV$o`{1CU&5XFpohje zjW-WyFZMey%&(wwKR~%R5Pt{!0pjfUlTQ%Wg!q3BoRu2@_GrO+f$>z}=7?VfZio0? z;9U{_4!keNDcVh5GzD^;`bQ>Ei6k2F7~o2dQ$4eR&*eDPlLmY_;>Um=I(X^A}RL@b6e}wp0S~y8H;rxKbDJ{Wj3$N6z@0&rSiYQMc`j$|pvsms!VujM$E z_W^zb@d)5gfiu6o(-IFD!Fdeh{mn$qzVFO<2*|ZT@@#zeLA;P=B&iQ@S}!P4;0c^O zZI_kcpV5e?15ZO-RzpxDnT~i1;PVhK1D=ccQQ%8BPTPg~VGYNror|Fo8xS82d@IMP z9!*+tlJ;?&>KR*8z>-6V4+4IS<5Umx|9Os6J!%l_7ZLvqdaiPu>T!nk{1e1S!gYoZ zh_ml+Ygvf)vUp(E{putB^LrhLv+IG*h)ZD|>&tOkFHdc;Jb%Pn0PoLnTCbbn{~(T2 zJ?!^vVThN4o+ysf`P7xXXiDHX)ju8DaU#d5ej7TNk(7eC6u1&Ni+?t+&E@22xx)o5 z63GgVQ~eV`&uYXYfp13qEAZ`zKLx%Y@fhf*hY@GLUpj^Ob&x-Y_$lC55VwSr@*9XZ z1pWZ=Xy8u}9|8O&;(nm$0N?xMY9n92jn*)&erYc5qE%jxq{;qvGtE| zvoo^K>{NifsTF}7j{@$AcqZ^@#B+hqM0^GCe8hJG-+{OR%p)Znr=sjR5l=WyhY}Nz z|AF{L;JR&St$x+FAIu{b9H;tE2~rYCSB_Ktn_;}_f%tCV0f?6X4@dj~@M(yD13nk= znl(j(mLqNed=uhhfbTh{rzWle`C8VZ0E?d>h))N^6M*+X z{1mi%DB{hby(S=T4ed3X<5YE9uyX^)shz_>{utuV!Oyo4-wO451zd;dpyRwA=>N=d zetyva1E@S5mmWZdnuy;9ZU~(DjooLe5hqW}g>Eac;5gM^vo`rhk~QM&_d-sHyMTO0 z#GQb0CC$1 zaTnm-5ohbwFpg8zDPVsR$7xpy1wBf{+5OaZAbu3|oJHIo>h%!u3aHmt#IvCP8n*l8 z2dVZHIq&A4N&ySorPF?>G_`eh46~Maz$L0EQ@Iy_YnUA z`olO*^|0$ZaU7?jmY^pQarQgC@xYm#a#|=!YY;CGvE(?%sh`>BlI0wyRk{cAT5utV z>ZkI4z|A;L<#nN5x^bM!2ZDSk$EmzM$ft0e${U0HB92q}nIK=laXQqp=dzvQIF)|| z@%#?)FTmAcK}gk7`BwBFNzFJ;^)H8VZ8=Uwok89Y@%=DvhXQBuxe*+lg?L~+TB=0y zH{$MQB43F(yRXD1#3ju|`2xh*^|9lKv+Mblh%aj`>iL8?vq!^GJkILHuD9qT-j_Be zNevOd4)JV`IJ@4}32|00PsCdqiu(N#-vggJ3<6G@4~8Bg!#GZ!_OFRx&m_c?fX_fY zANWkfmjYjecnR=jh@S$!5%Dj;w<7)q_%4nc5&OHr_d^bIoYu=6{BsO(_I>749H)BN zb-C+^v+HtC5og!mJ|fQIp{5g2N)n3`cAc#O;_SMd4acedeqcX&u$7Rg9?HFdx92#u zpIv{Gah%F$fV?N-?DwcXz*(Ho=i1Q}$jMXvEN-JXPW9)5{usnp0#D#L)x+X>BFCwo z(;%OMIQ#u-25@HQ46t)C;!A+P{i5yxPlRPXeG)=AkMy*H3IQ9P;MOJ?0Z>>h+hJE`1oA7 zzNae|Df~}>ZQ3q8XX_ozzae>WvxGf2iptZrWGRvYk}zLc4Zn&w)Bm~}ZU~)+>Gu(f zlQ2D}I2nZVigDypXzF#0x=ljr$?YWS0Ccn#a4YD9SU#p2 zKBpS44%Zi%p6g)$AjG2~J{BSV7jSl8Bc_L4pJ(?kV*CW%AW6Hn_|7=Hp4qt?9#{=e zt%k3zhMz&a4D5M@IJ^F)Ll+GsF+17yH*>_<^_;HN@PKM~LN)xaYWT)#_({as^@_WQ zv+Em`h_my3HCX?$`Z8`+4WGnu=*oh>)>OkUR>Qwm!`b-I_VL2h0hPX_y zhMcD(&em^d5NGQ*Hs;Xo2O%$L0&yW8czdK!4)YuA3eRe!+m!R;p(}+XQO?g7egggX zJQ&1re*T59{TZLRR3S-L;@eNo`j-rG_Wj97#MwBu3~^hi^lrpi{GZ}Dtrwdw*!q;3 zM}686!8cTXB5)ShOuiVnE7GG5^z zZULOtgXwPvyf2dP0DK_gGT_4y_XZw{IQ#w^i#Jy85Ri{S@?pRe5RV2v2Js}|6A@1V zJ{55l@M(z82hQ4)*|{3{Y$U%Ccn;zPz!xHZ960M=O#e0Dc}V^d@O6lP0=^mXS}-rO z^GK$j{XT`Q+Zk^F@&}O~bKpl2ZwtH#arQitbBMF&kz7Q)C+N9~_+a3-5Dx)fhB*6v z?_~d0IA0Q6NM)KfXypfQA!qPc+AQdk%1%x2KHbyglr>PE(YgJ4+xBSpSzGd7gVXQ%>4| zZBk60Lr?ZY<6>gMV@aFE#wCT@bnn@(t?+ny*dHF7Y!jbA9weWTG!Co^j|_}RppT>f z|4#I4hcv|a*riHc>AycD2*;mCnv*TcpxgfBu0OYa-@y6lbqU%({;&D1NnWaxtq;hi zkX+%Njcc$g$kJ_Q2fa@4r~I}cFphRSF0sDM^3(MJ-Nyc7;~~2~$5gQOG28bc8+1Hz zWdPoD`->ZFz1ouO(lS{2CLl-W{XdppLlj`21A({%mrvIde=6TLK+F*U1=8~ACNBRk z!v0i#JS@B;L^VJE&c?<4F_<(EPE?ECVfHp##0zl$9B zQ~A$=#2f+8{FC+lE(kW0A!blv->@?As4Ok*K0EuWr};_~U| zoqsC7&{)hk3-Yr!T)q~`{HO8@{!l*6i_3S0@}W3ZjKr^wxQWYS`PsW1^8Zu}8iH7U zxDLz|vwUpdy&C&%!$gT(Sa?(aQI#w|>py*~DL)=An3`47|5^F;car4qmtr`OYbkD+ z!u2)4Kg(zP^mBke`Tx{WQ9>K?Q~T?a7q*|C2mPu10$VY^oA~zgKg-X+m}<%|IW0=8 z1wmRq^%=|0{5QLr@+Ed+#(R*Ty|MgkUsX-{W6y{Z?7BBCzY%${{H*-d)s$Zb<@?rP zC5i7WKcma5DZlKjC{acmha_5l6Y^sDS^3+bd|P&a3X^0DpC1$%i2Lk66C>G#bfB92 zpS;B52}}^=asG2GKOARoBjnL!w0$PZ!2 z#DDsaf$va0&VLH>v)^Aad2O}<@1IEiKlPu&5>a3_$kFy|L0&9BYd=~aTsF;8LMLXD z*!lIXGI-AY=~})sppb(r;>^IDv;n zC9u+XD>y7}N?Y0zTZQslCblECg~t+8ei;ZB+UiJRqGF>0lY$dP14(^ww?G$}kL zo~a{oDH~8wNupWS-&=WgA6@Cm`gtqg%ldjNzxC@QpZbr5R3cGD*~(Sp48!EACM_*U z9=UR{xe4KN&KvlZq5g*eXVW-|03=~d1-WV8> z+JO-ng=2k)a|)?c;TwYh4|Bse28M!Mgh9I_iQ(jcHK_&hR{6CbRaFrNrXCUrWOgAm zp^aT(d016dm4T@%X|tngWolHP{9=W?)={eUDA8K@JWt3+9Bg7)B<3dB+6vi-zBdL# zj)(;r?EBPBEG>$ZW?-65s{!SyIzYK95*df-+$zs@qKDAnXYDm;{hEhR*>i^P@uawc7yl+;DCoNjI+B3&dY zbn__LB%aX|HT-NrvE9TrR32rGN3`nEN>fXIR@xwgF1SgEQ-gmCaaZ0hWh7~GREqNE ze~Oal^GGm@@#U?2rcmA&`u7jgbDxotB;^T0bf(a4+OdBM^dl0oexyUwLkiN+L4rIP zBc`* z%8F{@Nw`-_`8blBA0Jt?SD)|sNwEZ2JBrKJz}UDkbm2kPku4K#q_XIQ@ZhjInluK(^DhKZ|gI%)D24x zrRz)S=%`?!qC$2oH#}Te|3xIk#Yn}oiJy8B!b8KOMiV<)+So@&NPXQ%-N?2r*`W(c z*zk30N0yesqlpWKlj1lkRGup@BA8S%OqvuYO(e@-YMBK+3keyQ6rM;?lBG}$mW_<> zba1~w$BFNkT z#im^$*Tdy1UspLdC5-rDEP;fxa0Z4ANT@tt6zO9C79lHIVh9 zu!Lt$VNUf9%`!hl*I}v&WV$C4U|)0XpKCG&Ija)mhy>JO(GzMgLy&m0m1mqzss|>C z#W3~cXa{;!{>P)L>{CP;$wA{(JiTLQZ7N?6Fjzdm6cS05ugP!P#9I1hR2ihVBi^DX zKHW|1KFG6$iTBSwVCn>6>HkL9;(sG-`(FynR9WWH#QPbilj8)Fg6LTrJqsD#`oB~m zWQ+fq?Y}j1wp=wSK=jdb3vqn|VdJZO^($n4Z-ifY=-D(G9nKIUmuFNZ4FOedzt)$e ztBLC^dL}O(_Eu?ms(K1BolVZVezm<^B_{(1G#OC|;$k7YNFjc-kO5Fg7iur<%KumW ze!uHiW#^IjA$>KYh=k?Om=wJrie?lFr=?@t`esxnwGxNPekL-NFO6c-4J|xXo_az@ z^@Sc8Fo=dMIhzp|Xzbi|CJlUYt$z ziByfJ4Q-H~MF%G3JF;3AGqUlYoY_IMk=3lQ@}{4r$W=p$hhX5N104N|7a5FAk-Y8=h?#fwL&$?L>L3;kJjlW4&qsM z_9jc~Rzn!|#<xx=4+{pTDtiu@~%*u zQAmuFsT!nxQ8P#nCWC$27fplo0J1^nidpnVt0WLC=#-^O)Kj^L#gbhz%{?I_I~m4B zRGuYVD{z%~+TD_qlSmWpBwz|rN9Hkdw#uIZ!1a;;SYL(86Xr*6)c{vtINAu zBnl4^|BzsS1(i%Zy)YvOz;4pclJX`(rBjF`@%w*hH@+UdOngs_uF}hn$bB zODO0_q)>h)iyTzX@;YK?NvFke<dSNTanS5CCietNF_$e-;0 zsZXKu)utZ&x4!?O4&rz;fK-E4($-tm%VdNaDOf=-@d=AI(rf{P%H4$G#rlx$CMSI+`K?BZGygk4CXAy_!M0l6HAK zv7*}RdgVqmEN1^ae)`vSzcb{z-|6zx#LJ>uV#j|>Dd{#cQJ{=|bS|V4SMi;fkBhSB5OEr*Ce4 zWTacjrN6>A)V03cKW~rQj5>YP`}R5EYmrssufEBVjeQ3CD^0_r9gcceHQ9M|a9WFV zuPS>N^!aOVi?cHhJA|FMS1;P>*qG@XU(K{`bHJg&n`uitCM0Z}SG0Z8tJ2;tzNa<+ zXWCSYWt$T=PTJS2-hFHB36(KR%oeA-e%{%zcx%^C^@u`i{ljrP{MYVqydD!arS<12 zBV!h~_&m6%$%#`-Cw6W4yu~NABRyWVaBFp>m-aYI?HM!rdgK}B7`HUMVt6C-sE4=5 zdymm8wajYzJNR#J6wvzS{@JHie{s>=WgKO-drC+DMi;HjhF*xF3>%g8mIcwbl{6`z*7$?`sF?w&9Q>U`! zdcQuSXMCKz(rUz0*~$@DKBugG86Ix-ptwti9t%zzY;hPKWiV56tVIV&=N_|yy@#FX z((m}*;F-Fvqn2y9%ltjf70zBi_8YC+(cAiJ8xO7a^9tMOJnWPI<*whk$93Xc9o6me zNiom1md_{4ey68i?|dcaeT4Czb2*g@V=Gp+DGIlo+vXbm*Y0!MMJp7Wk7$q6{H*P% zSwnlZru(*(MyYPsI=|MbqvyW1wS`-x)MRQ6#|A&52dW-ec?HNVR~Zhf<#dQV%e;0ChVZtaf*q&G`d8GbO%>VCafyxmb*>Y(%%n!)LHH0v1M zZJwJIzc5JWXvfrbu_?n+$9Gc{k8w_q(w^btxTL>JX;zq@U%S$C@2ZUcTCo1XG|MsW zyNC46|9HM+pwXKSg#~*x(kg>K-f!Kd-^9Oa@44?V=iS*Qq0gOE4O?zsX%*9ZSL-;R zh{}sgcMl%zJwdDaoS<&%(UhYl*bf45v?Yz}DH)ubs^&`{OYQw>qwi#ub>g8KLXdTr{t@4tF zU%l%VX*TwnMq^ipBNMdqZrt?NycAtoOt+%wmY#al?hi)f-{|^N9S6HicU`z?QWpI; zjsBZS{(Y}a4t%w$akT}JtWCu?k<1^jcOE{C!|LL2 z59gJ`%=XWyH|1%KgtdnK`)amMohxm4q(_Tn`y7eiy$U3SWH)A)-VjdlOGzK$z1%PTI2_>Z(K?ObInb5dNJJvDq#`s&LLLsgdx-}RE8SG2u; zGDEQ`>T!toF8j8%Hq~{q9=>nb^qtQ6tD{Ua=f{NXQ(HH;+$*nAadxZ0{@hr%iiknV z%g1)=M71j&k@sCOa<;zxCeKbsFVEL+>vCkLTIt>35#2kT-)X1arBhMTB7^?n?MmnC zl~it7&}mfX!=Yaq;5uXMlberQF#5sO*{73Cdp9fnF85UI zpY5}t*{gGEPiK|O@@z}ppP5{B_7NV!CqT(uayi?-mD{ZF$-@|6XbTa~?fQzA7HoSQ0Tl zV{~Z#v7L<_zE2$;+A1M;{DM=VUluBdtyGG>;W@Qdp>(T1Xip7dY?e{LY zYuE*?8C&ijm#&(lj4>c=o znD8O@N9v9nh3|~=og9{UE-Xr#*|VL(wDfgGd7pVhF1%}a$tu72ZSc#;&@bwx&bD(_ z?N+X9Fe$HRemnmJm(pu;m)t|m4@P{Mf50vO<+H+L@49@>P(1Y>GwGjC$F%Gly4}6} zetjK9$MGZAHSW9fjiU0=>5fWgNz1wrr`pPPr~? zqGrC2V(_WH8JCatt!&t|s(kWx>e5KHWzUrLU(Z?kY|ul+tiSzjFO6OD ze(r;5^V$w$DP%n_d}7UPUWkx^`MWYi?3s({|-+CvPj=XHYV7`p%53uaYi{buP59+}PD< z>c;f0wdlWw~+5dhq4wA%76Umvi&CMn)y*q9e0sIXhiPK*+&(7q*oj^ zwP-&zs>_}4w!7ApXAsZ_gfmshUw9o7b%0sFy}<&R2eDJo3`ZW^kd#r)Gsr(L zo6a1q{=g+9PW_$#?!C>M9jTmp(f7vf1+MEpYz=v5?YH`1i2k6m2bbs0xVXc0Q_a}# zQ6=iZm7`7+nAcWZp1j7Sg-Pz>=%IgURcwviK6+}8*AsLH?k^0FYF>J?Mxz7A&Nb~A zpZvuvvAg~`cj=<~1zJ@qLvRs5a&Bi&CMc;+=#A042v zb9#8t_c80&mDQdcr|p;Ys$fQ7pNT8yjP`X})xiJVtHqgay&J4Mz5K}E?v5SYb0el( zxL7ziwR}G6+XHbxzF##LHOTug^|Fb4q~E|9KDF*l zclxGw>Hdm>;7H5zIav;7PqK0@hnR`B*KH$h%e(KRpfx~X3mzj7US*LCULdwwomIpu`mxwGT6F1=Elkkj;o#aA`g zfIW`RK?cvJh8GlTYdq9arLFF`y?HH{xr=5#-`sw?Ji2c?=SjWm_xPf_y?LvVv*qd& z!fU$J>yq=O-PWV)WnTM(zOFQXId)^z7ip};`!RlNDpIy&UUPH&=c7+OuPyy5H=MV) zl-|Ikbkdhb>LXIUd~*#KDeGATx*r^&tZTbEa-nW&({WxyOV2Bh=QXbyAGp+iOW){N z?Z*4twOhG$&Z`cqk8WD`+9gLGmpn!`r=(y}#>+*Y6`OOG?(KiP@4g**x&6!{7P`-{ zOf8DqwO653j-23N+3H2J#iM?NAA4BGX=rSnli%0wTef51sAJcx2Q@!uJM_?-xC$N5 zB@10Y*7q9H^6)*=bC2I$x;Fgb)As7Nv(|TN9h^4Sx#h;X#RE_Go)j_l?Wa3A?HY|* z)qAmFhuOELUk{kP#V8~wdEUiZ@#jZAzqwy}arRX2L*}OIV^YWZuWIZ0`s5AselgP& zuS_+69I>}koB!99g>ABptO~sb3|)TlM2*|e79MIm=||O$ixIxdPrcP$yECVEXNxCK zQga6xUzOYU%E)jET9Dnl^UfE$t{+IyZCq#n`1;p3SqIs4&9fe{u3qi>7NfLhrmdCD zGs>ymKeAzdvA3Q}cYULTgtj|E7COb9a@8?uK7PWtmA#I)8*e|^|7l5qM_~OnvF6G% zlj8m=GMcg9{>Ou>MgF%}y>2kZdgg;e%EJXg_cK(BdYg&8cbZOju)Nn0p zn~)n`*3Ly+4>cHke(TipYGIiHRYrUNHkfzIIAh7U!Sf;?`AkghedSDMc;pp5=aws< z^m?$Np+~9eL8I*rj$2%uIl(ccDCR*^|2tih>b@xGer27%%M1VaH`g}(xOQLvyQ4CE zr}XpvJEqgO!#zF+o5ziBeOYhsnel5aAFNsG_ck@h%sD`J*7(=)jy`TP#~ByYKfBd! zTDGxbf~)%N{3}l{4_Ox3XKPKXaq7*A-F>G&Uf5G^IDSms*ltZX??`AnwQ*X=*_N|wtCI! zGI?~^kU?Xo9bbI6?V_>+qU&rv@Xj_nV!+Cp21?1$^G9idR=_+bgZII1 zQ8!{kph@ui37h5!QG`KWrO#6(`h{?ZPI*dS9?~V?-XK>#5cF#67-E42=Ja%gl+h6}& z`d;6_4NU^n>*OBt>A2M>$*NwAQ>yvn{Y*av6-OjbPHR}I(>aDJNm%8;` zdslzsn29y6-m+Wrys}{C^UCW%S8t74GjL#0vo@d72gViIt!*%};L7@}Lxbn-wjQ|R z_Tv;?pR9v5V@$Nhl&M?9_&1ZM-MSz%`E2;ZbNqOp8oJ+a@9m@1)_?G7Y>(8P^XuqX ze^&QCRQOjr+nu)$o;1nobU^ca&Dvf=oeaG0L^sRY*FVv=;_AW{etXXAbk(_&Up6dp zNX5+DhY7V7cG0`K>hZ}kN5_dKi-X&@-e?rsqrn~v-~Nx(n^^1Jbu~S+_5O+Lubx|N zyMLm=sAH-vWz!n8b80;7z=i0TQGeH)cscrk#fE9d8}904wOp0^YN21$>)n-W!=Bf) z^P695>?p<2=>tE_wwxGHzw5c9X^mG;Iu_h$|8~37L67pn?Kl2?zgevz`&(+ipSShB zPM3ae)6ORQUASQ>OII5-XtiGSiWi&r&Ca+vprz@>n>mMG1a_U3qr7Fi_oT;@Y)3_n zP{pyjQAt|8Hg5egaZ;+?-N6}$A4Jant4U&bfYq@^wa&aSo?Y_TdcWztVb_{jUVLAm z_ceU1_q1+jUbU{}Hf3E|)K|9|weMOLj@nc=bad0M|75LgwPnYK2~(TKH9o%fjJ~Gg zUbK5egEL#BFI%T-Buq+PrhjVph#UU-r-wFl+Pme%84q8Z{x`FVKJGOdtUGb{m-U0@ z%(QA_S!DBhpTVJVdq*g)xjb;Z-oCKNp-%r(mIJyklinKf<>rxB7yKPv2Ce(i-1lV7 zY5&A_Sk>XLHk-_+_nOw+e%`70Ihrf>`3yV0S*zoh3ZLG!yUS-d?U$H7TmLLGY=XzR z!`ogKZGJTJq|e=uZqFO$zx}aIqt}`_yE?iVxlW(#;_NlO?AC+V+h#nT<(j|NZql$f z*FWS=+cy4y$+7{f&5H)xFV4B&qv3_KWfpsupBtdxV?eFP^NwYXj!UZNy>8~*-iI2x z9H*K0dZ>g5yE>1KnAH+r{qh}*c_W|Zw=n|+E;9kVn4zP;|~`=r}u ze;@9ro>)@$b@+iZT@r5>zPM4icmJ1jO=HvxS2k*61J4WFFd>M^aKeUjX8U)sD8 zheocc`@CZQ{r20xIXsSjZQ#A_NJn)Whk3p3M7KK-;E{TvhV0sxtJ_9j-S+wDo{lFo zh7N2~F`=K;-Y+q=dVe}Nc;3>grSkYCIlbFeZ8z}0cJJ}hZ_-)o?X5C4J&3X>Zdq7w z_CI0Ag0u!4I#8AUm+QiR4ySJ0l6_F|GW4z54wrS#PxP{;xMkP2Yd(I< zwKEbo#s*E7~Z3n z$v*SEmeF=Di|1;5(=;vYXg#IIlw(_#&v|s+X3}=2g?kg9-kl@gcInGS_5SL8kJf2a zYPY%~GsVYJI<;cR#0K4R{8DbWALG(0s-KzD(}l<7n>-E=H}Z?CUy-t_V}VY<$FcS! zUA>1TMQP+cUuoz8?PRg+w`Ly^}th_8?{n9eF)9IBn86wIz*D*I0F@$&A&tHoEoFJ$bmF;jk_GA+0aO znC>zezv!)M;>+0$gN$Cke05-7mj|z8Cwo-Yt6@B5?snf6<>3d%M7DqXW9fHu_eXi^ zPaD2ke*28wq};D7uJu+W%ZCj+)a}ek=Z}>~Zx^l3c>S)Y&#iyb9(G{SY3lH$`|IIes%H+9<}A}2|0OWXA#mxG zc+V+;bFMr~-uu^($BV+}^?38?*36+ZI=`}ct2Xv!eXUQkpNHptskv@Sn)sty7B}?E zz%;S_v5T_<;-dTXtNU{A$K8`Qx!cFLc=o9<(^Xx^v-Yw`leE}U&B$xnd$w)z>ekrK z@MwtoBG*FewYp~mOg8JzEwM;1$kNQ68XRGt>8e-n=?_V5)i+%eS<5Kx@fQw`d3f-{ zmFouokF+;|tKs|O|8I*mvRC$`k}YjSqJ2e@vZsBcO$$YcN<^|&5+zB=5+PC{kwj!C zTZj;{WDUPLGw;Lsyt&`s@A3biN4J@I&g;C-Iq$Q~nYnXk78QT~vE=#o;5Tnm7oFG< zkyQCwYu1*&m!Hbl&S>x8+B971W!Z4IdmYEvc9VZ{PCF&cv*A;1&Ha*|W3z)FCmPys znl(J#ZkEG6wRxQ!x6D1#w)N_`Oo+!F{GhA;+F4WxnWq69y zfiJ`3oYE>PqnaKMezHm8)waD=r3LwM8Yvm7!vZFhl|6hQ8&xuLS%v)r<946+1vO9k zDXZhI6gRIVtWCh{l^*%48y%|$xgS_<^L4|Einb9yKIwT)eZE8Mnd9@7$EqgJdZBuD zOc`%@Ho=vmaNqRs)aJI_zl|!+ z%c{(uR4RShZK50j zbWlCne14|>`^qkFjrJt=zFGXK($wT=jz&t`TU9krQ#23wyq8*5nlxmmnapX2k=p7u zCzb~N8uqBuTa{rc#ZM=MYyU{Dwf9(kaHU+|T@8I>q|Yl|XxSn!NYX+y`1MN37sdyT zN|h8Jj@oQ2>S>)WW4p!if?-)F_c+52&Xbzd)^#r^Jn6VyH2F-oPs>x@?0sh3?m*tg z9hYowUb0W9_lz1}aCS)ZtTiG29j#7F*)MO^b>JSW8(SI`&3pH9&p7h^o~&1t(Hi}* z<#r1sGxxQ=qgNUDcu30Ow2mXT4_yA*{?6+QttNlB^@;yBvffQIKfpgv$EfmznZlh6 zoeicvQbP4o?(}&UacuuN_q|K3d%m~dv9pI`TKuK*@lW6G9{Kr9AJ5Ip+8>p>cJNc1 zHnpp?x>u_X_E%p)7mYH4=zY$qRg#C2o++Er65H5a?*ZJTBD z`e~aTS)E;t&*!w6H}Q7fq`sr&^q^Kx_oZ*Xzw!R3J~e0S%G4j2v|3hQ`(sWC!cM1Ci`WPYgfBRL(VD2X@uNStC zx~kdx$2Obl)m_Kehm05XdA#<;y2an*!k3QY>g{#?;!&!Z5z&QJD~)4^>y=kcb@AVG zZs6T{F$o=iN<8`~m2smkV@t-%manVZn`sZfWPf;}mtA~z50&b*gY4oz?Ylc=%_o_4 zO>z@FCLLC|Q4rSuTfw|m0b>q3DyL7BbLgA8J0m6XgyF3t-`8a+emb-`^ZDX-6VI(2 zdA90Wz5WT)os;HoyyMW^^FwC2)M3BIh^yz{-1fhCsOD6D;EDEIlHxO+rkCi)gtV?m zuxm7MK6bprZIk<1eGjSD79G!@)HeU^ataX z9h2YW;(1|hwR-QU5%*nd!cq#yrRcV)(Uf)eO3BzOw`=O)aMgD$ZNps+-(B3BYu|lP z#@;qFo8;Rstl4+GcHizE(I?lce{Gn%&-wiO1!Mi|o3~~@lKV8Y&)df%7MX5sUfEmm ztKPj+3FC@;#m>GuZdkMNjrVOn75DO+KQJS)q*8ZjNBIG@lJnGE-Ol^34!I*2qvq+K zwluB9T07mXaMrmr$O&~9Mw*bz^hgP$yI)){I(T>gMj?*nZH zc8fh)>pb(xhmrs(3)6XS&z7qAiY9kWO3xl=Y@NGf=*ZhwvqQcOyJ2)B=)cF+Q^F>5 zoUJ)?>eQ(+kz}uS=Oeduim6<8Hu}QSl@~mwMQ3f7m{2T}5}}gvzMFUR_aWKM9jCP1 z%RPVIvxOTyPSa z+H-pOv8Ybdqc?g;Jdy0W-eUZajmb*`m1OR_xxR3FP+A(arfzVnN`tk-)HYZsOqbK% z=I$AODI#*4hnB^FICr^9vzuPNLq^}Z6|9kWCc0+Jql;6A1U@q{9~s<9JM~VaZJ_jy zeVwajEV~k#)}^Yb^ep$}YtKGL|MkOKng4Ec^9?yi-;E9*PkjEAdVl_0i_eRDR;t>i zt}#8hRl41>`JKY5*M7~YPB822R6j^BBCn@G#Vk9+ip~>8n?*!+c8W;RpJUf()g|`b zq&=GEixo8A%}e%nUA{fozoz|^H4o1eKYzRZ*P=HCS6+lanjUcDgZ{4T4_A#0h|}!n zYxY(mZOY8h&yB@8D(z;s8@67q?NG~tr|C!bF8;jZb;DLgg|~ohNPvx1YD!@-b@Qcd3un)BP+o zmFuAOXXjp^oo!#CUP*6uNI*Oem=o=xpjQ#R&`hmHI=U+Jc*Q=b1NM!T}2i1i==E-_o_`RP0f}}?vqvYtfptj0gpW=TG+RF?9nPF=;%hn2@mS~ z8{a55Oc?oi=#}ZM#u&{y{KR*?>yE_+y)Vh7Z=Khx%j$!EqG8Lls`u=R_0jO2z0WvK zVdBs;r}gwN$$hv|zV%Yz{tXK)Ps>$5Qd+s)dbDP6(Z%PBPe@K`F5RYCKDX$`DwkHZ zimQ57pMSIY%96I4N&~g0`E9&dJ~HY4L*rXAx$YO&wrB_&cv*9ui_MwnfRiqv-ZJ0w zL}wZ{kGr|C$f5U?va{yrV=FYZbX}8jmJIX?3M{TjQQF$6z2=Ag`&V>&oH9CiLaB9@ zm0Fy^l2h+wJ}g&1q?3H|nVPZwI8%?Rxyz;2YNV~r3lB~@ma4TOQZ_DT=gQjQIYGe* zj{@=%J|EFHx^5HIqaf6ObWU}Mnw;E-XOeuiX20I$Ckf@xYedTlGT1zeVh~%JxW!-rQ2f>x}+x3%!gH<=baBzA#tr zS0WR5;8>TWwhm>B;^n0V?OUSU*e@kYdO`QJx}SR@cP%qt8PxOr9F4S{L#-Di997@Y zH)G7Q@#FIh3pUymI#el7pE;~SrS`$B_JwjgZk1SMYCihl*}3pjrx7z04Z7(%rj3~; z5mVCV%?_o!C4DdT>^|kl^wU%BwlsZfxY=&&tmitzDueQ^1D#FGp0@0$ee|1Rme$Ox z`&-Ug+Gk!_K6c*Ux=lWOH~nRo zDK{*X+T@*h(a=5R=BMi^-$%+wT$&hmdtiq>vKbXRD(gy0gEj6ZMS!`eH)a_Z@POlrx2eml)s8X%l=obBw&W>|+ zOx#+q#^Tmj)vMtLgDw?Jzh1K_bN6u7RGkC*CtBU@JE%F$B=viX7w3;$c(dchB0Xil z{_%5*hxuM>);&J)*s+@(ZWb}|R+%dtccOR8{QQz>ibFXVYn9BCB%8}MMWhl{dd+pL5Ps@^* z^nBS{qGrTcedRs76<2-SUvhrT%Py(W=VRCDc*Qk4Pqg&^ag| zN!2$`r>$7o`{3aDTYA(Dm}s@LS?2AlfmQz<_q(b6cRMUKW=2iWlM$Wkc3bt!MQsNcKUDrKBU4vEB;b$*|X4HR||9Z`uD%# zu&w$|z2&Uc0X}d{KuL8 z^G=R`a#!`>^LlUl7=!qG896dTUUXeNbH7{D;{4)-f;qoFD&*c@a@3$9L$2q+84gxC zJ5%MlR9iP|`@Nr4ueNWdpZ)uVarbI2n{6xUdi-E>OaHXGefloCzj_Rl&7F2%ed;^) zLwsoD<|MoKQPCx;PcCQQM%i&&p$fbw@=2cqL&3t z54=4NE||Q)&wA0|yq#rpJ{!)6IM%Plped`vy1h+$bs~Fj?!BA)v>dKFOC@T{zj!jK z|L6mo@(XvI@6bOt#H{;;UUuHWEk_OBYAiJ%b$6yuoW4iM$4O7VD_&L&ua?zWKkMFn z2Q#0#xkDdKRvysh#h{II<1)wiv~gW{bD*(R;Gz~&=Jj)mTH|s$+GW?vC!Ib&Dyo^$ z{ho?-msv^i@|Qb34LFh_Ke*9$fL}Ms4M%%)lGfW?@oRT)`!x&STs$!|WXjYTZ%PMC z6%HM`-PlPZWYm}McizlioHlgeUCYuXC)S+2G^LZooiXd)Iw^0}QMfuX} z6VF9!SDrB1QZz7a%(dqSC8``szofSxGTfr*mOb&uj)krBC45x6=?-W=xH|ga zWXGLxFRkZ0E=bJyIjbnpxpea4#2(F~7f+OtU!<=+%H`Lr8&R3xLOxpEK9_%XWAl%| zTes4bKWuoK*zvq><7S=A(vBAFEqfF?VH^W2kYx zLHg}v<7KMnR(oWY$;36DkjQN7p=|E|aMsxPtityLOMA4QG~NE0!)&EjJFo6MwWrR! ze%7Svkqgz5)a)9xuEciS{W#^_v2yL1HBH-$a~{Vl_3GV4eYC~C+oPQ)6*>mo6#ZQC zGxXr`SCzAE7x)f$dme26LU;5ti;Ud*pmPh3tJzkWZHtamnLWC?OB;)^3K2gJ=GyIg z81mAf?G-s+4g1L%9V2urWF-0-tQ@^@cjd6ynwqfH*GpcuEmHqtnDk*s>#Z|~EXW@= zI8!aU`^@yab51{B9=s=g%FZ*jUmcS=On;slsdL#XwDU#fuDahA&Y0@cYLi3aw^r|6 z0{0ntjJXxida0|P@uOZtmK59>n{#K*k<)$0?S1awvwZb{y-Sx2dpcG!zd?Fj`ut8` zEiA89jMAT$HZ0k;@b;79Cw5hNHO&(3U(_x+c<-l4K}O|`IodPV4C<5Tm*9M)i|*FJ zN^6xv{Bz^lgyyx^ahmGezojjAtMBUDTlGeo=^luD=GHqj>q=qmmv1^5=K3io;*zI$ zr3ISx>}}K>mgi~~STHss<=5BeyDu+oe*ZPe$>9DbtF7CA?U~~~@U+v>+>cJ@pPI>^ z-M#hN?yZ*s()`z4`e{Ay(xpzm`vPVk*IriH*s_0P%kU%3swH{ue6pIgMJefM?6cP{M;^QGzG-yVut$T&s`1(d zu}!ni8ay7ge@6O8#R;Y#`X$Gx<%_z-?KAo1a$>{no-cD9PcKdww)3d@b;(0Mg;!Q7 zKT{hhee~PyWkW(2jE-D(O>o;l!?vqiy5lvb~eKTs@r=vm!;?%DH^|RfFgyrb~_$9J*Iz z8Y<;+Pdg}TmY&D@o~>KAeI2!Q+u zb4(6uoIDu#{6|(;hfR|_o2ShFYOS5&_+@Fy{4p~mt&L`G^z9Z=pc>*(^;s(JQ-_uqdw=kHE=Q_-V(u(9o?UC-vIk8-J) zU^_(5(&5{)$P*t<-MH3j{r5L7T#n3osI5O?`9=K!k)k%6<$sLba=E5%enHaL_|a?1 z;?w<_oLirovAWj*rCwv#PXAD|O@2ea{M!~S)Rphl-8DaunkA<-xF(B z4WCz>>?eP(ChTU}?F&|scHIlU)zl?~?Pz{oS6r^p<=(>l?|MG3ilUHnS1FOi!Den?v`bQ|l_)Pnd;GaLLA$FR~wVKA! zu}wb~epvkDpuYR8?G4%L<6^(+KX{#KH)C~^f#qe=N3%q zyXRMA%at8_*!XIUNbYT-KgK{SqQ|xkEu!|n$X!4D#LTLk7w1GXho1EHx;>%U?Rdg+ zxdkhv##}S^k83|%Z`tKBs-?Y6kIt3U`DUD4SbM(xurYN;6J3tGn;kZsetT}N%%;>d znH3!~b2DbynYTApIdCfVkXL5&EcyMpL4kE!oR{hD)(P*T*dcK4Zbt)~je}xNo!Qx| z=VwmF=*^^qo&i`Sy*=C7<}It<{O2b1GXq$h5`6 zJ8vTtR<4}t*sfQ{wTE0EG+5lO?tHs!-uUy;IxFG=RQnFyFfieeynWGJrT(t36H_MP%&2(NwTPbNtb>rmSYisRmEoRkiYabo^`1bcD z>eH@t+wI)vY=?wj8ZI$*j|Z-<+hC`)b&rMJ7*~49YKVNB@Ut$)wJWl}21TYGT--t}$hO4_EvcQ0dqhpFFe|pd z^-%Y%%D71_r@x(P>5-omWB$=YAwsiH>hJpwCM?BoU`o$dY#0U2(gB}!R zD0#YXj#3+XtMW(CGmCx?;)D8xZA#iz6|_gq;7!=W6@D65Rq6~wCrogwY$~`j=eUAl z;?VM+6YZ=c)mnr->ec(^+`F*@ynK{O-&R#j&@x!$__cm^uh#FZa@U@#YCAJ@O^8=Y z)6P{FFJ8-x`C0U<@P4m1B^grEoh>9*4{q_uwtsM9tZU}un+==dXPlQUeO>vi+*U1o zZl`e#@mXb^b=o*TT2ZJ`G{Sb;xIyjuF8h#sZ{5%#I~4t1Rg}l8OJD80$t>K>wWr+* zjk=r(DZ%Nxay%2_Eqcjq(Qh)mqxCgDtM!9M`ulPy1a>kyRvFY7c z4c@Zj%IQhprZ%WwNOkVmF#3&E>MiaDqN<4VnLp}Bjee}_U6!G|@v&)1{qc|Wl}B{1 zXhf&pKc+dQt6O;Q%vaY_&s-ejkiFI~{#U<59kpDE8>`-jjFUO9vvEUE;D}|@wwBm; zoA6FS-|(`|ryUFT=$)6^65Gn{>^i^O&pgh3uHN$g>LRCgx1RlYy8fl&qS&BkDsfry zh0(`*EQ)A1;^>KA|x?B9O^oX`@z2Khl3hy3YE(K|v=~wY_ z@U}xMr+SXvF1tizU9TMdv#9jU{@piRHXj;z(ad`4A&XPr25nm2%PYIH_l2G*1Ny(O zQ_@Wwy>p~@{OQ_ci^+lZ{YI9uHOiuDK)p zgu(T6^RL=TS=q0W_Sa5I-1s<0ZI`XS+m9su{OHBet|=kMk1l+?eq-XKcYa4t+*w?d z*yhXB@UKIL{o0W9u&%DdSn~}(HMfT~Vc8^(lbMw8-{)deeJ+@W%ZD=ks z7?!wh+4t6G+PK6tH@{PGQL_7)KIy9C&yV@(R%`mWcsKY=GCvdCda1QVTD8TJn_E+3 zi|03~g(@_Zw{IJ7?wxlq*l+LXZhapeE=Wmz9ycNNhvumDy>ibQwyzp!I>1WFvAFHE zFwHxQURCvVG)$VLH0RQBidTTd(XnXbLlugdFHc+re^Z65nPc7L9iHGN)lpG5<7 z>{ltAj<}U*@>cP^>{t8uBO`q8YOgsz;P8P*+w7yy9w`jmxK?s`-kFTSgQw2d-`m(d za9hdv1^zn@jSZS=e%Alw?RKeolN!raSD(}#ylM59H=WW}RW2o@B;AeUFX!%3kQxngB3|{c~QBz~<(AmonB1rRkcP~IBWLuK5{+$-i5nuc)XG_F(`*|(Y zt7W-j#F_&#hNfOpw(o~XZ+_f+-1C$zr~{w=+JK4mV-9A={`+XyVCn& z<=g$D0{e#S4oi+UC;OR>Jkxi=K_hMJH5xIOhmTvsm!AoS8Mk zCD?HJ#+eh<)wjLG31g zuee zmai{Z>D|6{xnf4Nw57@8ilmK;M>%Op6nb9CUQ~W5+0}2#pcdsui+?l>EcV}|x&D0Y ziu;y>UIu0_y!oW$lvS34^G?qguPWI{=hpj{OqSi9-f-BV{Ig?zjeX31eP_KXV^q7@ zEL)Y)I!{t(?DVAL&jEO&%aE)*3{vc zQB1a5agwWqm$Tyl#qH+uK{2NV)B{37v+P>Ao`>A~pZ8?ewOg5ni>Y>pu+L^epDWa=F`=yjnK2P?nx< zYUri$?N#E2%+y}mSNCl#GoGoxsFPlep6ipGU*oJKXM}AWnX9{g_zIf`TBDsGj?{6_ zUm5Cm!L6^KWZ%K3F5eiIzk2pak%ftgo}!vtfWN!TQ0@^n++$q4+}sp3hpTI_hFp^a>mQ^&CRp3k-%eFR>nZ}u4NfP41 z9PX#qLRwZNEz93&aSs_G0cTp@{ccv z8L>U#AGY~&?Ej}NBN=Bdx&}0G{+)7vZ2ik86J#4DBoF@6mYEFatS=JQvXPQzvMV{a zu9LLk)SUfU+-DP{!9R4BPs5tz*X(zGdB2+yzjMCmCs`%?rJtp2q>?MQG$O!|OVs2q zN1KqZR!F^m1pB6B$=l~DgSF558tilRngI1``akOR@8pbl-#e4~vKsDfWz|e%6^&?%7$@WWU%*8*CD)D>r2?h6wTT?D1LFM8S~lez zPSSIS&Sy_L>&Ps6BxGF~d#8O7;wuG$rB-y}8vff6r&XQkc zy~oK0TFZtR$wo@p%Q|z$#RpQ=(xP|2{-u!OM~%PtNb#e{e>oX1BpJ?+4Iq&cp$B<$ zT!YvD;e7x8t*4_90B$jX*Z<+Qe|B+W7<@U%Fh5@Zhx7fCW9xa|{*MysoB$~i!~ZBa zjvhn}`=tJQBEGOg>h~M486(`P)bCLhbMkcZzk?M2)bM|C@-*+igOsTE-;g6xqP}9d zq8P3$hLa~b{~e@6YJWqHNQnmi#q)VWI@o^$d2}LaiTslqm|eyEIz8h2*-;9-^&iBq zkw?N1JJ=H!fsf;tfBsbipYaFrYd3+1AYKE!5OJYBN{cEHe-3&foj*HY17Gt8@oVpa z??n6yzun>YJKyMT;oU%gw@+!(vAK3jj-hN2-f<=^63X{yPe)!aEqeI}@oRFxJM+sw zJ9_~amVY>T8U^vcBGRh?r|*oXF@$%vN{cLzUI+9u5Z43lhqxi|g@~I2k3@Vt@HoWn zfF~e64R|u*vw^1}?hbqp;&Xr>Ks*pQ|J)+#9Bw%ucoEV^0IxuNIq++UuLNF;_*&rP z0cLCv<{HwXpNOaM+Z}$-%V?Z*u)~h~WM39=`i@+x-wSqxc1@{BbO`i;%(zVU3V!Jz za_0B{(esZM;o4g-q%WiPk^TbMcR>0pz|9c933dek{Ym#8=m#SGW6)>w%Rl;h;HUl| ze(ep%`9~ljF3s8f7w;g34-&)m#c)?KoIKL#U+IR4;Ty&9#bS7}7*6JJ|4R3e7=B6& zFBijai{Z6m_**gjix@7&&E@`;e;F~ns~Fx(3@7i(`7Ljs6u_biH3;#H=O*_CjWHEKklV> z<4s18f4bx!_wF~2@laEXU+1Nx_*Fi>CT~Dfh|EXSLD7kBsn);xK*u0Z%I4Dtx{^>K6IUZ24| zXV{5*+%KtAH|}Z6As!;`5e6P2C+;P9E^c$3+`Q*Hkw^53$U8&+Ym%lWyrhD>DTKT@ z&%?!!dktOy_s?DA<{J>~AtG>$73P(SXu(BV$pdqo05gWNovLIU`A6H*tEw;FPBg1G;jU;KW%j_3Z| z%OeHMmz%G4Z-Xe&Y3&flv z_o4bYZk+^ThxPvko(i0;U$z~(aT;ppKBH&zN!knv#J)TCq54e(asm4-Rlq!V=GLix zCvcMgehe3`iwJgDzr6%KvG2j?zcHMxFG&eu2hlroADV6_;3QoohST$RqG!`JWc1`J z9knBzrwaZjmn|vZ$mrWKd^>PrpY^9Hw@m`!tUpO#Ac1gC?nCXbAdm}$v;B7)Plf#S z=hms7&Qpn=Z4XC4Pq-JOe*&ENN15SofD^qp!-eZ5Lb`1j-jgXWJ6_S>T@d>!jGnv$ zk_4Pj1BUx>`)RqBGMvr7a9%FhVcSCnqjzTPkazEqK=MPb>QVdjIs?g%3&XE6c3As& z8Sc#JUoiFuFr0qJfbBOkcGz|@nu|*U(a+^RG@qpGBoNNF2XXCW3$GUR>~ZQ4aN-9w z?nCWYFnYHAu>FxePJLkXY`xR#UBo`gE44q8DX$B|X}=>{)(>QiC4q3(5A=E|u`k7a zsC`#%odm)MaUaS_8zzC+VbfLTsleHB(um>gagg|g1Y&vK++w;eJD>MkPA57{Q~CsJZ_!p={P{_3}tvFqi6Gf zjp3|+8X3-(i;mC4KJhJ0mwy&Ew@vcN`k#)&L_dtt50ctiH8+KgVfj`(-}h3kJpJ?b!e;rgGT4`TRnCO@Rj)A9<} z^ZvGfhtZE<^beW*v-Y1cdR<2UnbEWDp(QCO7sz5H_o4Y|2b|OwYrhMlAI0dE89i%X zo6+kr`caIYwQtDi$(Tv=Gl|i&_UAKteMTR_=vn(wjD8HG-^A!y`}-LESVo`E=vn)P zjNX9JS2B9m{!NAlGx|D)v;Cc1wk%XVZPd*kR`_-xxh> zXNWYH7#D~?+2dmnaFQPr?nC_<#^_xbzMSFg_`ja9W6J2$7(JVxd`54^=ueB$*D#zt z{_1dXNg(B7{m&l1Sig}u5(z{f%6({g7Zb<@!r9~9CY}oIlh~yCEZ{`X`sWJ69T|Nc zaN_4K3~vGB7~vKSXXmS|efIp6wNJnQBKeum*vSR^gfC!t6O*nb!(~WDxPZ&mpWy-A ze(DDU;ACONaC*K;xHZH57&~ma;us#r=!=1q#W;pvWAv<@7mR*9qo?VEh7|zz?SB8f(c5GX48(bji zvi+Ke+&_~4&fKT3gh0T|Zyo~WZ!PTp&DDUnLtGl{bVU3Vw41Jo)AJu{zb&`F8)(R) zJ**Sno%<+~O5g&?GYN$E;MatIau=aF^PybLB(buuG>oh^!$S6vn^C$A<{qXCxCN^$B~077p@A6-{q2l zd_F+j4Q};#iFh@%t51kaK|`R|g=jtXDw0P(Ba*sfP4t!2iPi8h`5R zU?(9X^n8}u$$)-11=(2(`FBOU8SMBYK3hq!PtQlF{SM&IRY+d}{d6F6sj+KbN zfczvP9;GDcGZEha;|#fvfCTCvYw#}}k15ZDc2b4(n&7v4h%SYz7g7g zSHv4y3Hj-Vcm-^$Bc22OeKg`yFwR&aei_D92gGe5KjeM{5@1F+jWq?2ku$Ewq#Ah&zDaJQ0_I zb{mZN4k0_DNW>MvpTeu#{=z<_ z8vI<2_*rNt*AZ_E`dY*#p&h(ITnXyyJK~XW9BmEzX*+R)`FS_Q9aMz&M(t630?e0& zA^mVEL9dUv0@Rlk;(g(GHwE!wFmAXZ-VEglM0_szc`@QXFfWQj{3+N^LVN*?gPDlG zBL(NeLBtziyedI_k02LSB5nu%zm53O0YZd2#9u;@zQNpF=zv#_el}_g55Bc!YRs7?)lnUIO*_4e=(Z$5t?}r1h8u z1g5S~*FM)Qo3-R`Fe91*z1M1@t;>u8u7Z5ju^T8{KUxj({O~j{w zKk0b`t=}+c=TDIS2BiBPaV_wh1Tnw`YR5s5-xRe*d;@Sr#2cVqR1uE^{|rIAyM$n0 z3vvGD7jD-`#HYeIFc$F!XrHEtcLx8DLtGEG-4NdZ?b!$MX)rzqAbtk?7K->K=#PsK z&jS0)5RU*mv54P>?RdnWgMK68AD}&KLEHn@s}{ZeMCGB`t>)&?IAz(r;B7a znIbeme{2ckJ>}{!-)oQfUKqzz5MK*?Fyfk!4^70k!1<#t;<9l39)q}Z3of%zYpidGZ0UMe(H_*Lb#4SAMr4l z=SL$x4f@dr#MR+Al8$%`_+dZdZDBrNggEu@1;jrg18jeaYtMp{2zpPdl*+E z5m$lnVJ+gdke^hKJg5vS$+iFgF~ zza6wYT92l1{Oy4_f72*eW;Mj0Lw@M>6>4V&#24)IQqPi4esLjR@nVru6T%v%hRelzH85U+;g(JaIRAU|^uj{*OO zA#MlTD-f@Oac2|a^g40|;x{4PJjA`hk0prvL%m!?d>z!|UBs_LJFG`M2J-m{@ipL2 zX_$vnKNo->IwRg5;wvKl2HJ@_;_skd^bubG+#2!QF#b$K{2+|mo`~OtaVZ4xJz#%1 z;`Dm!I>bw19N3O{YnWf{MSMCOrwS4O1^GXZ_zr02HxXY7^NOd4?}zim_lO^d_8@}e z1Ff%Gs7G1ECj;+=_)Zv41|ePr{dy$gtD#+)BYqmnYmfL>$iFM%c`%*_BJM{D!i6P> zKZW_p8pK9pv*i;-=uYI>dFs|L+ms4CC7` z#C4#Zw1fUZ>#+-5f9Q_*a_D!eh}%mD6|9Z86Z8vuJ(b#D4*r>d^dn)sa7Mf%9Cv&X zuYqDs60r3hLCubnO7t-}Y{1>#J z5X5&wKU#+PDbTM)d;)8^2ipC=zO#2<;XJO!rL;ODY zc^%@vfF~hN-ygaY@e^>qPv=3@4?f_pVx*_z%mu`)q2E14oZh$d4)HP=e|{p)-=NF+ zNCw7HnxDmxA9|l1<;$U8t04V7IG$=Dt^)nS0P%N_pYe!W!u*!buW7oY6orcPMEWjp zT__mwdZ^z>#Od{oc*N;-ycEQpU>w+u_;RSn!-(I8^SIN9YlDBP5U2B;8pI=^UDYGL z80IHUh|~Tl3I3*jvw`;90r7b-F7-luE~GmU@paIzM<8Ae+a`$9`yy-+uY}`^BjVrT zymAiW!(g0WfcQG-*U^a2hVf)Q;;W%OZ%2F@aTFJF5vRw^V~Eq^IK7TS{TvPL`5Mw^ zLOs!ZyU|c$ka$VVEx!AT9ypMj7Il zg=Qp>;zku=lC*o=_o=C&_CG~?mjL&TmSAqWC3GtuMt_C8m4dcKt z#B1TWKOFJ%pdXF+EGfYeMu@Kke_9~k9_9x&h|}L~dLkYL>G~qxAIcSkcoQ72!x29X z^WGT5&%wB~9`R(bza4QuXb-uF4}pG1zt5!g6$1LRNFM{|`BxFQ1HU~)yfyT@2E-#E zpI;GIgnDTS^AwsN@|-jhx*(nc7`=Iqe{aFU}TLgMCt%6g1ZcQY*r^qk(e3C8-ZWBF1 z9J-6>4dV7fbkTdn#{=gd>cp*+{1E#qfJ^riRw!QxybZ(2REwVfc4auRa~|{xh!+Cy z&2VDJL_*l1#&BY%1Mne;O99tpII$B2?MI*C#Lh&}8zN5AHDfrjL(g|?8BXlPf*pIr zqkvCmII%MrD#o4R#Lh0zdn3LLco^dJc}5Y4)8~n-0ZyWm_A`h3kg$W%ll;)@=X)7W z^7D~I)b9s2vR?!bv1QZD*jV>P5Vg>)?u_W@3Sr$y{ba-Y{^g$oOiUIp~gh?D2glCS~s zXyAFkX?cf93i`u{)BGPtcGiKNDn`$i_de2Rg8m2M^!KDa`U~mP^3vyJ4`Dd*a~#+i zh4?k#7KmF5W<^ejy8@3w{1EW1h+hU?#BfQDb%uJO_xq4dlFwV9uVeIte+1qF#$&3d z&p}gSII&NEKdi-YVn-d)oy>5eR|M|JaH1a%dUdOxt=2F{bnCea@S-iP5tzYzFPh7)}ia2tjbeI)Q{ z3@7?0!2K9b^l`u=7*6!xfp23t(I)`k&2XZp<4F<2iJtnQlHo+JMs{+cf#C$DfqiCx04}ji`;Y3e=PegyGNbS?_#e9%H8S)>; zaDs}!{w9VK`vsuSVK_k*pg+cNqQ4CKI}9i28t9)eoapa@z7g>kz+1z7ibN-RX({d> z7X~t%Na^p)M=_k}WkGL__)p+o3@3J~xep2R7*6bHNb^{ffH?g4>W8ynr=6OR5aC2m+p`?QNxJko<;uY6anK0P`_vgd zOfh-GHjJJe_v!E9?GU#liF3gjaoRrJ5uXqGAjIi&vgRY62l{Bl8-cGxoc>-tp5bKb zKb_B|NR-HMa(txow;c>8<)!Us7vk-p{p<%${7?KggZq$B!01W3zC7jn0mF%XRj_jl zaq^se5*{Hw4fs>UZGpc=JPPCWDu_eeOKVr zKWByLqTY<2_-7UHF$^dEc?))o5PuHb8u4DxFKiH}zk7E^d^+ge5jO(vhxj?*^AM-^ zr7vVSDOVPhE0*D;T;IUXYQ(<)--LLl7J{K<#9IO12Auj?g{*L458^RkzntM@yvhcB zHN#1J{sH=*h+ikhxgax0SkiRq^M%wHPP%9y^iwT{6Z;*2Ph>dBk2BU*60O(^8 zr{9ZZGMw1?0(SN@oY^7TB+6I6?Hd{GQ>&4t@Sbm%-cy7f5nMZw7XHGn_9k zv|Ag*#{r+qaQ-*~`443{vEu^zG{m<8FJd^!3O!#q$8ch&2=vbx&gTCe!-@U@=;eoS z8(bjSBb+{uMgw?Tj*)i09NMQ2;`I0bqmZ3cD6cuDK;(LJa zXE@(ZfFENxDc3$||92Qp^rt}ogyBSgg7z20M}l5vsIWr)W(`~car*ucRp7**q1C0`tNih}*zCNf|itCro8{1E!3gw8PEN4lNN+06qorLf|tHr{8CJBYq$B zzKGuez7X;6z?UH21U!o2q~58Y*D{=xi{8Jt0r4I!c{hlX7*6ca>p2+=Cw7cLpM|&q z@LYxyI}hNx@ll2oJGP)NM4WzqagyQ04!s_Cf#JjseXc|m;*Mbd8gN?gT3j3wULtNL zU{UK~!V1;T0X~r7q^sNnK8oR_T`dJZ88~g9UL>xpkPSSM%{S?F}1E0xoVu#lI9K>n8hapbueL2HPx?W&^ z1>*F%9;<=V_F%xpAz>%ta|JBAfOs762E?}k@2)AVQ2UpF>oL4N#~`}Mf#IZmz5#tO z;-7)9MqHmH#D!f9C-!MLz;I%pzBjBGIBm}*t%VrR5x0Z>_#W{DB|-lK@k!wS)-XU( z`*b|-f_MNNANwHw81gd^aeAEPUhTvOYJV~$;Doq8@HvPpK)Rub)BHywPV>JSahlIW z#A(0Si#W}H0pj$zR2LDa{=bL#Y@zs~=ZMqa_kKV;0{Z(;#NAW{L+!Nx>rZN59&u{F zKjPHBI^xv6G2$IN3hDVGPU~v{;9G-*zHSkMH{tmxJTZam49y zybN(#@0Ssefd29naa!+h5uXkI`Hncfe^*84zx74^rhz#1+i1kQfFH~er{51wLY(G* z7UDGj-iYT75E593cnYK&i?|`2M{Gu%-dD35acbu<;?&ML;AD7&A&gJCmeG^rMHGxz zb%-wl{s!@bz~3XD1N=MU^miz~5WfZb*24vdka3LUM;gYxc8E8Fz8m7|pzn_OAmGCh z9|wFC;%2~&5vR{fGDqA4^y7e2KTi~jBl1PuANX3tmjOS7cpmU;42Q1B?|H*;Qoq%p zZ#P0%q4~KBTp4kCop}Vq;Z%f=F$p-Wm*r4i2gDZwpN)7Pa96~$fzL(!8SntaZvqcP z+_a5Qun5GpfNw@T9rzZ+w*${W{3P%!#18;JfOvP}C@$n9-UfIP;-0`yAwC;;1>y<7 zFCo4f_)Wy~fZs)&-jCXV_yf>4B7PV6PsHhSVI*~haub;x_jwQcR=`O+B<;$X`;agg z@%;i8*&}`w_*}#<08c=iKA${~;iSJnHxX4aoaBc-Pw6A#^g3!gxPU|RPp@}%NBkTS za6uJuY3Quni#z#1?Ti3!i1=9G6A+&P+>7A^!Rn&thQrXw=OYU^t*;I; zJQnRiycO_##K!Mvk_kl`lSpfRX7fKGQ)}eMBoPy zr^ky+h}VMtA;Vz^A=+yuLZ7!_$}b05toCLMFYg?edf~@mS!Yh%W}d1o0x^%Mm{Wd^O^qfF~eMpO2G_ zxLOC|1(7Hfar(TQ{fL`^{t)5@z)v9V3;ZPFuE5VDz8&~Q#1nx(K>QN$$B3T=-pFt= zo+$C96^UAo5mrd^Anm~t_+a3qJrF&eM~!4S8Go9D$Bkz8B(c zfCq}P6NB`FLB9%d72q2Xw*j7rIDOvD4u%sySU~>s8BY8V3U&$*p9lOD!&y5O3@3Kh zgZ>iY@xX5)eh~Ow#B+hy0;l!+9`gT*;bi=&1m13}utN1+Vcal5oPO^=4RL3%6Nq>m z@Fc|d0zbxZlAl80R~Sy(=W*cmh|}NqwSf!TB-_LeJumLUaAJqv*F6|}$_p;ww;lNZfAd%|yh048_f2E_ zwghkibNYT&%3TGSi1sJS4*{q5lTv<(JCbT!=!P-WQa}a2R@c{RuJrff(K_ zhWCT-g-AA7`^Jd(=TjAlCL!($+!^tuz(W}hQ&N79Tf)Ar6NX(RjpH(0n3J_4N1g#6DCh zALAgSXF2`-DAt>ilOir)z60?Kf=qM@aa-U$$NoaR&WE zq^H*#n-Hhzc9an!5<5^uJQyp6)8Fr6J$>H@=JdG;n7=^wEg=8W{NzLM56RC6;C&em zuI2Yk5W^RV;kjb?V=-JFj<+QLY`Rkshtnm|2{HT%;?)1OVmN)SHcj^__(29bIp%{H z4n^T(OcTRbis8j#_-8SEAWRBLe%N%)5vS#)zvHKRHz-#c()$9hLY&U$>GMvhoepG$ z3$h)BCFP~Urf7&5K2Z#xkN6d^vq20$B!)LK9E!qQ>JO7Dl24XT5yPXz@JulrhJSv( z-4*gq7LXl&{9(siMbMD^u;ZZy!`bn{kl}2x*|Jw~v&It*^EOaDgn? z`s&7Tw!Rb@&eoR_Z$+r59^Cpw*j7b+HEa(;oX#iJ5g!ccX(1i~oYp&OA0*w&zzvaJ zQc?hBh<5{SjX3>1AT0;APk*;-kMtwK&UD0$fX_zU3OF4{seN1E-bgFN7Q7a$%2`Uu47@41#Cz8ds&zDCnc1WxOda{7E-+TSVP3;NB-{xRSwh*tnlL!6!u zW+Gk>`W(cYf$vAW4Rp$U#5)2%hIn`2#fa1Qo0cLz81&_cYXiTCxIXY|#I1qfM4a9) zSA)1O=pP|Yf1h55_;S#{M0_>yw}{ie;#48stoE;C#px(%C zc0AOD$b_@wp%Zf+%8rLmz$nqP!($9*$3tVt1ktnO;T*^W;p}**$ZeBA ziouq54NnEmmbaeaY!MCX>hpd4J)Hbp{9U{}f}8?FL!3f`eANS8fH2-tIYpAEEN6`P2 zgt!FH{htCKf8ugda8gI4QiO`)HiWXc1&7d*xO=#UdO7*{dj@bd%dfk*x(0d7hlm35 z4RG`I2@avwKo{iV@AW$cfd%<{{Vn8b%7Yr`nhISd*oohE3-I&v@aLMAe?W+bx`mCy z(BKdkH?EBQwwHgXdSFmMphr+hIONL1+sQM?#m|FlyZ?VfPtE}8nlN1aHfQ*I?Y}F~ zjeo;_+!|r**g$p~l7IgQIQ{@mLoS8xZz(Ysoszhas%v z@?RkX0~hFhhr-r>#}>lK*V$yqHsn5Rxa}!yodP(FTas~-l$8VpVIlJ3mLxAY{YT&s zdVdti|1-7`=E{Bkm3}fY%!NECC`q4;$2k22j{Pfr`a7vwZa*8?v6b*R?(?toEq#Po z*}_6Z(kJ6SPQQ?2|4QGGWR?r`J^*3sZ~q_RcKj>-Mo6Dt&nM}VOFlUL7aaRn`i)cw z?SPZ<0hj+NNB@<+;aova=ielKaxO^C()#bjB~H@E?YEIkIJuyV(x>rB91_TNYO;>o zk0P9S()%%ZG4~JWUy+Oaul#$%1idvRMDkDK()4Nmt;D1s2nX$OnlLPJ`Xqn67aiIQ=zZ(ogRoM5NEP05gC2 zKUPfomH|SHY^Ye0z8v?#>F*Phek92(7wGSFg{{BqpZu==U;b|l6=K|mafqZ(>Kvzk z1JXy$RCK+&5TCXaVM|Dhu9M{zG3lR&hO-XDBzHlXA$LSg&eO%u(K3%?r z_`eN9dVFHs`hMbcCeU~ky%dxGX$u9#N*D)7{(EyD8lU=wJP(Nb`?G_rb0JSdSehWm z()hw&Zf?x&|5yAkFrc`?I7;<6ej6_Szv}-{oe<*`8ML`T@~_B!XndOg{$kR1*5kxn zuoV`+|I_#s^bwPO`ddN4jz3BqqVZ|^x?-N>{p?0T!5%-AIYi^r z^v8=ypWa6&i3VmGpQdjiCjDve1qFS-3h}?{KhmED>C^WSQ@epi!rFljln6_jKHp|I zM4qEe)^U8rNrK*+3Wd$z<)=89#)tiMvC>XhQ=Kzt+<T&!uF5si84KM@LanTW7=wZoviTwoA>%n@dXku_z%L zD~x4Ke#xr$l0U$ZUo{jdb63zEELSy6wp{hyGOD3b?a9*qOx4|&~lZT zhUKbr!ge7^$uj0tNbA*>?M7L)UfjVls;u>@D>EI>cy{P(N+P`B;y`3>m*!P|C!Zp3 zaP@o*%a~6g@+3i!apt6cS+*vY&YS_IxsSh#uOc~3Rpd^lJ$&YSxGQ=F1^6j02>3s& zy$L*3&mZ`GZBgV(WQn?jB!pxw*M1_CW6_&S9-P|GNYpsiz}W4O!MNN1-R z;XjXaU?307YM3Y@Dvn(>Ovn`}#4Z^oyNU!DCX7N8P?cbQ%sltCMT|iV z;KUM~K$*lUpc4;(36T@DMOY@b{%il(DRQErh%u&yoJ?sk`b3>)zwItNg)74J-dD$T zffA=EB_{@puoxzcQw3t{Lds#N-zk)x|9whMxKEV>@gy|kO8-wKoOqCfoMM=PIIq{H)IG(s>khIv4 zU%!6QV0aN)49_?vy??f1@LFJ3%R~!~*shR)L|qoFDRlpib)xMQc1}*DWEGDX&v0T4 zrIxsoH!@PH4KWt%Z%K}70#(4qPQzBiPQzZq0rqqH(OX3b#27nFM0Q^GmyYbO-)RMP zO&{+tft?+Jow3HKE#e8zcwzd194I2JoPtGoslhR;yJ;k1bxeE1qJ}DFxg!=p#F3mB zNa;3P$q8m6GWv;IMeyh_FIIVOk`wR#r~Ur}g9VFtVl7?c7py-M@x--}ptOEB{|O1$ ze^4c ziPZy2b0Q~#89!D%DG$gElPS048kIH_MwOt(WkRN2GbSgh%GTk5RjDV`Z=eWuf9jST zunyh#cPB;-0V+FiHNw;*?hnhzyTBT3*35T*)P4Ug#j1mfRTq2X?kT+CUZWWnOS)mK zez`Dazmg_P>$+JB`x5%(PH(fKVJ_|s{gIZ10oG*|#@l>1^cxhLUtL2yZfU@(Q+%dk zccJsxpDVa0fo4(k`s6byYc9O| z`fhE}SvAG+n4gDM8b6sRNR1GCmu%L)|5EzH_}z!*#wR|%s0kKT9NF;bIayT9K+{KC zym@+VgfHVZ>oWb*3&9=Is?!50RZrR?HgCG{sw-jEEkB}2%0Q6kU}ML(%`P@m_pHue z^cFw2^NWkuujl7-;*ZrxFQ&TOi`?P0o}Bt-hkDE#he)MHH}wWrm*vJA&eof?q%@e- z4?7jP&dQq0=E<3F{&K-Ptt)Br?A^C2`B|qASetG;>sI&kouNtoIYYj<5cS&CX@NvH7Cz0f31$%&?~dO%h%6ULZ)kbZ+BDoBXWd(t>Wg-Te?n6ELh~r<*&*=Vt6p^ z>*gsBen-an@db^14WCh={Q9^9RtpQIbNkvRnJh zg4g_0{?SvLI|;8%t;vbYo$l(z@D**(7`kBVy;SOXN1OEfwOQuOO|4@L)z8N7@fZnh zIh9ZPO!_SRS;$CdT+chaPwRD5RN!OvlVL7d!gqBO6wfb-yENO{N#A%>QQ}}m)EoD% z)eP&d8s3;njyg@4Z&7!%zaAIEUwKnQl)l(dV(3hdker?4^7MJ@Vl&6+{n2Z#Uisc# z5|OAUsn6Kb5jbu?S}`0xxBk3Y_V$wDI9~IFjS<BH`*@wDP| zOJ#e*#B%ic9_Ky};yoF0{m4g7L&4#1zs?i92wogsELURXljH@x7`?c>3|H;V9*x{q z8RjT!B)Iynh^G5e$#-LJmLr#Lyt^0ZZM?PfR9DenZT&+$_t^Vx#HDXNQ?2@1{OaQe1(u%A9UhDBOTXn^vn;gb zPR`ZlPv?sqd3^LuZ(KSeAzZ)nw7=5v17aIHY*h9zed=3r$3la@=eXq#cf~?Sp-3KS z%M=4G1DW8@_MKwq!67z9nVt);D2J zg`~7WVs?Y$Y%sfMmuLU;BxBjgM=a+E*olWapVl6E2Pb*^0A z?P#G{O$f0KT%L6)dy}KG7GG9W_V%pPPK2xkCl1HWTFQ-BV#na?&54i<#)*S+-0q{EO0K=Fk6of9q?|GX5)Gh^~U*JN#L#$=c= z86Ml0UWcRVOZHo*5~epFTI#r`T$M2yomV(=S)6#*8KEVG&aDKq&8~Fbyrpy@Y|4@0aY|T; ztXGeI>G(LEQ}Nn9JFQl(?`RH z)wVv$ok^+F;J&V*FWex%ZEwM)N4v9ao?gD&E0@i8;q7Me0J%rQ)sph7ubC}mDu1_TUIy9Xo$CJ!U>if%OMrn-6zCv0r*lvQX#!0kzw)O6?zbhGf3o>!ee=AFEWc zhBJJ{Tq%cM?>bEZdxw=ni{WO+7$yfa>d9s-4|RV-t^|nnNI({$IgZV*QzxIt{VxlKM~4)^)XZH*t*bsj`+#SPSrzG z9O0?cf|}DGo3-96z1kYHbH?1_J%e#<=kym5&aVs(8)J5Utk!&g((;ja_EYiV6)Ftr zLc!-ZRcWrB_1*J+Q>U%;oUCVF=Xy~tM*bKzojz-!nw9d)%7q0Km;T^RzZk2=;JZTQ z@e!%us7)kIo-aov=a?KcVx&u^TZYf*)48&|6CT^YT1@a=C2#JHEdlMU*WT`kZY#UA zA@_S9-@IE$>5(tn_b9wPFZAm8>0{;1CrnPu=&nD-E@O2~j8nCs=jypzJcZ;q8>!76 z`f-Bo!d3)!%hNuDSDNdT2{rb*Ly;0Eil3yN>)WTD?0&G&MY}u9K;G|q$N5W_GfN9| z)aml09~~1@I%fUAqR_KsW1Q{|B_FxqB{vU0ktWpe>Q$w;-MH?#A@WA;6MF$!zORdI zY2H_D$~LK;EB>M-Y`r91viqdVl}`1Po~6s_A93>wj9gs#v~%#_t+DYYdC62~v*^xN zfgIiJc7fx*Mw*wMwJ*lD#Jow+?2WEqHP5bHmAzX1V2oeP$dT-Q;;Q1ooA1q?>Xgb_ zr}o0W%t9^`dZ{=C_)bLA_bJ00}$$YEX)6McW zHKl1I&;PMnd|hy%<*{GtxroGS)gjr5Ed#HdoqCr`pZ{>TzFGMCYFkO~cK^dO8r_9k z^HzU+5`KlmnD~9J_-$%MN<`^Mj?H1SYSMkX;fc5V+P}G-sXW@Q|D^Y9Zm&d=Nb}JX zUv6`78?6&uHX^~(b!SP_Mqk-<>7fSohjm2tlg2`Bm9Ms*v{OCh7NbAsvzlp*L(6vJ zS+4E3H8~!HCo-mp|B4Pz(aJvWo?Fp>yvBm-Qty~d{|mKrjt>%-2kSa#-&RPs$6fZ( zO%r)9y7xUtSC;k{y{31bjcl&x;#*s+s|NSC%PP!_l~%H81Rs2Ns=aaF!A@SgQq2HO zR_^kw)nBbx%1c+K-QaoaYq4puP-o_ZUHt;P_f&|*^qQo}#LDWs!*(^BvVsTqrJj2- zsPKW)HrBd#xc}O0LrKUFz30xWhPQ5wWh}GOA^JA?>#j~LOk!ugG#r-+H2dhqYH*`ttzZLYvXRq%^PFLw??uwPi zZzKZ~7_E1zTKveV^Qw2Yb48if#_*&K8yudFxn_!mv_%N? zybZr|LNlcF_JLi#MfcNQ`}$omGAi1&ulVVm{r1-O(`QUCTKwz@RM}Qlv2Oj3H&d-` zZ3V{;HFLXP51P9n_w30*(}e}H>QvWG)rW!pd3{xhenO92c*fm5j|#lY43}$UN;|7y z;FtD3E}iR>#Yx$18w@&L77Fl8=9wGJYptJOq*n+H8F+cd;8Fi96H`ah@Ic^krYW*tg36l~rM!gio#R#XJM9<9fc$!g*J#a!!~0y0Stg zW3=*IQfaQyuKRZB!Hcitf4WOQpF;V%f>X7{g33l6 z#X5pUCtL6}4P^FYq{LXnCb0SI3@(A;-9+2$K!A zIcw?;yjs~{^!-)9mLg2f5Vy}?yR?FSWr?P(S9hm33Zc=`LYh>Bj zufw-EI6Hj=ziNiEA7LMQxoUW-yg8#sHZ;HNIIBy^xMAkPQT-1b!L|32r}ieU_|d`^ zq*wcJi?`t;vD7R@(ae$J3D$t1GX;;tKG;nP?3nuJTA6rAv!K-RTOY@c1)tDR;S>o> z9az?UVk*LY;6c2D)4?xu^FC|4T0#qI-9J43lp?FsX2yM|z&q-xAd}DwyC>&faF>s< zE5_%1U3y(Ccy!zBK&H~&%7>dumZx=w>h`AE9+vFCbxSj}>}7Q5lb#C`8SKB7Z<9PR zi`~lM_i-!WGhW;umf^d5An>U=+qUDklG`|Mk9Dbhhz-s*YL^mq%<|RjihIy~s}<*>kMIq7>|NR4!kP3S(EQx1zU5!(d{b`=E_q(z(Y-F+gWElg+dWlXP48Kjau}-f_OFXu| zin_f@>FQFt52K$W{#iJBIsIm7YRI_K562kgsu`&j4_`I~);C7kyxzCrk@@jGcKv>K zpKDa3){mdhk5*Z^f2~A5-Oir=M7=w#+nzCQy<#GMVY|qIBCj^?$9IFz@Ctrj`1YV| zhm8RL)Tv0OV2d@HY+FZFT@JQPT%@-ju8(TVt$RR6yzohEXjXnu%s!9nh(o*{(>>Yp z$vZYo4x`QE4JnUpkN-0f`%_F$`O1yHO$Q{jTHj6;xrYqZNfH^4rGC=lzqo5|e|G8^ z&kys6&_jFDFF!E5)ccDrGB&r_lQ_t&n zXLp(`JDT^}I3R%IepmmM2(4Z}&3NHvom9(1$+X{HgiE~MbBP;DzT}jsKQf&2O%O{b>U?eD z^ilKNDQ*zSG$VHFx#kgb%cLZd;aH4$C8tdFp2l(H%{G@XdcVqqCZXGG5_isB6%Sd!`baZqk(6KLZJ=GDpjO)o_rstv6tCn0P z5ZngoDiQe<#2=P%u?}PJVuy@)Z3~y0tkh~YDs$F!&e2(U-$Ly2DrZ~6@gZ01hGF%gA>p&p>1G*8 z^*tKD@V@jk3Y(fv964F zapjRXdVKgsRYb}3hr|ei?LU^ar3?g8>Zgch4I=fa>bB~4nI3O4aV;X*v=n|?nwi?U z#Z^*&P~*h?sEK3YrVQJPJ^1KfO&EOPolRBWeD2}d&Vb5hyI|9HzbPRz_4vp@g|y?(_H~X~^WEvq*=M1twzbGbH|eI- z@*6FFJbG#0>mq*Gb1XG8PBA_e_vXmTLwa*>`^CN&KOb7cv*Ok2b?d_)4)#p?zC5L` zR@f#Yw_)$V`L9MH#k?PQ{p21cbGcX-@qM=_7_xlVY&U!@CD)YsK)E~9uZF-$XXB-v zLPEb9wz;M4r6ylm*XVTokw#Eid89(lRpw(bp|i>_}f`ml{$x@!0-M`hg7 zw;FX8uWyICMP)8ZnNEHYWQ#j+SmU(94liqY%^chovd`WmmVuIPHKGA5CeK4rhU zV!Jx8V)*_5lfC_g;tLgrbH6jST{V8)yQSQh@#>N3IfdEs-pdaY_kLXeKrZI^;}c;v zN9s>)ePDC;=6tT7?o*|{o9_9V1Gl~xzdC4b|A1PN@qP?{S19fe%ZRla2pmx2Y%W-2 z-df();nTBqSfhkjZ>81j_?ZVQmkDbumGx(hJX7sZTdLhRrAa9mG(u0x94>(S$~VU(QX~o-LP?CIM+YV)c%3itI4Sy z_beq$$265bIY?Cp*jsH1?71?y$a7~pT(9@u*>a2Px5Mx4;aw29sy5!D`muNaokJ;~ z-pxLkH+k>zR$vOw1cDT0eNIp?Ky|q{)qsG5IeV<=e#by&#p*#8SL}8jejwqAY+xN?RvX` zfeiW?$!hcDjbrt8$Ikc{^XA^Wrurnb|9I)tzO;WHmwFEr)SUfhZ(S$BYW@!J=LMDE*n+77TU#r*_-x65d3;9J1)? zU7q8V8QyUCL^-+mmk-?;P3=w|EWk?#LfFAhzdhv)SA+<=T@0Jjd84 z^TRe)MspUMWU7|gD=d4WbJs9+;N&Mmk=E9i7W?*_#=NVQ+T&1qZt#0plf}h0_o`Ko z>CBXt+~=xmZd<)%BYXGR2aGwPbzY(8j;QLb5qtG+R;b0UoaL}v>*8Hv7oXj zl<%b@t4(^Inyj|`0EZ#V%Xh)YpCm@{D+w!oemI^PKHxc^xcrIMP1T$F^V5}&U$5&( zsy2T?N7rXMFIyC4=OeBA?UmU19j`mZCLSGU8JZpx5r2^3ILcu}zqCH)NweH-{uGHN zM^AmIz8bvpq>Rm>%!eMPvV`?vcc+&PIoT(^Nd1u5FYnmT|4@Yg$7G`y;TSJ>&oG%R zNLSjvX7pB7!l8^^A4=nWN=~2LVY?`#NErUQ^5$lTo%uPzrCy1Q{mcRZDmJ?aHCGA; zS~%3+oQ*2Y8QXg`C0laxG_zu>IBTHXC1=+y;zo@(O658d7tfuEsacYkVE$#oE$7hP zZ)GE=4yV(7YdsOj;_AHcYmUE+?MA-Vec#yra&e<&CfD`( zl^v}Q^0bVd&6D=}Xurx>`}2|T>Y-MRw+3xQrV$0B=}2}ak=Y-{Qsyt(a<*-m%{s`s zkS1{D+}ii$Mdr5L#^P)NY-rmEbv;;v>{ktb4gS8Z7to9pQJ`swM9$?LC2zY?EdS$c3_UhvHIYy!7z z_GR(9w!r6c{B4%{`*J43yEfgExe;~u>3puZb>hkrS=ZXhd({oSp(Py(4813UJKpo| zbl;L1=E<27m3r?k+4HH=@{BK|?!#%Q37_H{E5?;$bJ4);*Ke z6rx|^9d^H4{oHxSisv1p!e@HlCumN8tq$wDYszgKq3e7vabkb@$J}JgLyUv#D{F;Q zIlpu7U|+L7XStR1QHg{|Ij3{2s@t#lJnIzTa6ZxQows8X=kc&>dzKlrR!C;f-&khG zrI#>r*)X@%W1Qd5pfXue!J|)7|IBgC)S?^TEg5_a-xc<8#wQ)&xBRg3i=yP#%)`kM zQ3H=jwzu^%!`zb`Tn6bXVkckoD+SPTYmG|iFUFY7Ha+>^n$;f3JbdTMx38u4baz~~ zF;?F5o*|~bU?Fccy&z;BE4i_J|8(1sPr3DV-KJZ?s*n9l-mv~y5OLjg$Kl>uP41lb zT@0k~UA^91^t*Qp2x%I(uefvhyL)?nkEU7lgH=|>bCQax*Di@nt-K>8@g;aJ?5N_# z@gEWo^0#-ye0hDIB%*tJ?C$0F!`SlT4Fg5w#T=V@zj1eGVbdR?PHe0CteEyX5AB}l z3e8~2sJ1j&c`>=^>&6_$>Ni%VX3@cwt{XYj_QhRc9M09@X_8WxG^h_^vAfG-xH&z$ z=I6(meaa`_(c4Rg8@M?=`uX~fHDkQY+>+%jJKT%gmK5iV%vbQ)a%CGx8V(vfH0r*5 z;^qm{r8PNvO&^-~rhYeO>6FljF@3_}m87VJ)8_B$<^IplPRep=`$~CF)yZE89xWrJ*6m)&w(gYkc^&I5%kPo>oQ@mD zh(BrLo^H7RZRX?SnU4n-7q!TG$xNoI)7v=*Z?}zn_!N9#Q)tnKo%u7&vYMx&6tL%n z+h6J`%g(S&9_u=_<(x*DzWAs$Po(~E=R2_s@!6-bGah#xV|AwjxgKQmXLWOUoSKce z#lO{OR9IYZ8Q;h4B{NoBjUpP){QenlqR(d(zf4+RuX17M<@=4-7}xCxVr*M3@%89> z1!=Cw-DNVT_=WwX8EvO;j`)H_H?UD#OK=*8Fw4rHMRz>U=fP#;1o&` za@yP5Ys|38+0aQ~B-nG~;hs#nJ#Q_q=Dd~SzbHelOUV^AJsa?-;UD)`Lf@lZ7(Sn> zy6|jFIMV0Iwx{KE9urfIjjo@g56$koli_hzr|WUcT5`Vdgzz_+cjSmVo#%Ee!^A76 zg&R6-FJ)K@6y=WDKJO(8FeztL(iyIZqc76rZp{D8lpX)2rS&||9mB!cxk^9n*UG`d z!;)iqdCK!%7Fy-XT~;QEdTo|hYK^ z0(0tRLu75876RSYF>^XcS5y_*C=`@C*yw-%v%~dth7FEz@#qMv$8&J;X<>Ot4Qs~ z19~y5L{jKdMYd=z9Nflnl(~0x{UST*Tkmwh!#s;2dA(w%vMS3OlJa6iA=C2jshxu- zjW65FZ26c$)^j<$eN~nWaqn*9mNk;*+C_7!b!Nq43QM9ar`3g01t}2|MuAQSpAx3@TC3>4OOw}L;!!)XVSPkF zMq`iW++vF?ZyuW#lP*X0Jr@3~C7tX^X)x36lq&FEqeCkyD;JK+!N9nrE&FAFI zT8m9QM4l?OHr;JsNw0B*bUgk3#4PV_(JPKevOR@uG&jy;}gn{%_n!2cM6v2H8tuqdV8vIcpkr)5;o@)6T{+pRQt@K z#;n&<7Y38GABUe6B{n4}UD~7L#Al%OY3=3(@5L}tjTYHM8BYuD<*&4OSbpXEEmHPa z(&}%r1{bo$7e0ut+O)f4P3BLlD{psZc)!@RL|@#WyI_OXS-x7Y#W@TjdmDIW>W5LYVU-nb|@mjOqu{igY3DQGzZ#Dgs9zEGn3}b90L77skqLOWDy=z8<8!J!(JCz3{bkV{;S#x!5ef_T(0EfuE1sP8vy> zZ=>^yyMB7%kypF+ho&nwH(zNynPz*}S2%fygXiq&e3QXTpVEfE>y^liC3OkcZ~PwM zddJ_TE2q#larSo0$6pMu-{e*QzAvWTlL*`ygk@Bl*zIbRVVhj*-RO8RO#S=0cB8Im zHfqN+Q%*RXCY>HawH5C-QKJv;xRBKJteFcqC zpUoB-Y;kisNmH@84_v<-e?PS>qi)y4sJJ*Q&sM3~;5*Y&ONAcy9X!Q1WbMCU?qK53 z@=+tv2eN0S-yWY?GbU*#JGbxbvzHe~PW4vo);t!+D9t`)Q{gt_<+Ywv!>&;AAe!V? zC7sATnzQO_ms{|amKp2A^!aQnwiEQUv{XpqF8+S5PEzjvfgWxo4;L4b>=tQRX&EWG z4e$)JC%By_VIT0b6l}E_fu2BD2N-_8MThv|QUU^9_-+DS7zf>IHfEMnn6L~MMgaQ& z=K;Pn*g4|pwb^;28T8mm2bQt(YO!-@u(Rl~8ng52P);dpD2#zD=7hkP5j)3jb{2{* zPvFN~0{Gg&&T)dO%YdC^#m$(rdEw zo?y^tCnYm#vWq7%X|T)0EzxCHie@%opAll%L%)n&31b@U;+pIvOco1+>D6OpI77$2 zC`4wDgg=s@X#MXOun!~Qy; z+kdYy;0rDOi?Q^8m5Kj#>nPs;wHx@MBz1Kc<>dDl_`NmY$WQR$TC0Db!{GrGp**d< z0HW@#!}@z~JzQRca>7DT!T~($QjV;YwSlykc0f+1h1fZ@J+S_MhZlaC`V(*#LKY6- z(Ux*#CE$A^;Fp~t2knB^QQ9W<%keN;h@JaV{Hz2=goB^0a?;diYIy2i-q`Q{{>NcY zC;w+%6A&u@3-9;~pTsyPO=(g32aK^}4UdziT~PW5jLTsBPw4^gSo#w;py*-4LQ);2 z&%jQ-{~{my7Y=@`_$U2f54b<^8-L+-f8m{f;e&tSAOFI^ULk)<59eRFz+X7nE96gl zwEx1vnv_4u`~QU>`wKt&7hdoee&;W|?k~LSFMRkfeEu(-nc>g*1@C75lO0z5g$w?L z|LynZ4i4@iel89UV6y;2Ey@?R3xT25X7D8kzU0Bz7VxD2z7)Y1w({LjOO_%dOSvvf z5mlyON_bhBKgI}uRqEgr7>F%tAUNn5?$XpSbl9bLl2g?{7eEj|Q;ur9}5x_DC2e1SJD*r!5?BnKy#rwx!y;6U*?+_3ef~~dp!uD;z zHa!UP3U?#8VCyLSeS#b)>+}B{HY5Ps#lg+N$;XF4-F*Pt4!|oIKgYHbaL2NQEy^Hx zIt6(;P<6X|1qKE0bvr=I{IH!L0D?+_v1qB`yu%!@y&r-otNnoRo&TXl>->2iBy}l0Y|Kf2>q#0z-8U zSmv=^5FA`Q_d0;JD}o?0-QEI%upWa0kbY#UI%g@FG0f%AqVo{yO_JSSN{deD4OJ zM}~%TV&f%nAYLBqgUgFR4&-UeMS+IP(#YG;=mB$XTn|`N4jj-UhkbA!Lc?k6555N& zNDo-HjLU=h1aLslX6%FWMj9?p!|^?bf&3O44%R3G2e<;G{|e+F+x#>f-?Iwjm$p7XrjZAI8g7R-H2P`fCuz6_jr=FbLHTOZ@UJxb1!y=O zxJSnvNGDhsgU8EG!D zz2Lx=g76d202dIh0OKt{IKBq@D#G_be-f0;QAY&omWDAIFE*vg@kZ=&|eSNJ_hac8sTrDeLf(3mW>+mcZ6#}{mWqc#`EnFczYQF&e-ze3d@oF# z^FjOIdtu@{1okJ_5IsaDs$Os(4je9D4DHi`aQvM+PZ1sj%N^gt64%29?Y|B$A1n@# zr3ui9Von0#e6Sx-NBDiHUl-vAVf!^ixChuI0dx3Xm3VrZp`0ZmuMh2DkMMe!PFI9W zK>K?mJOuW)0SL$6O%{sq30Th$BOL#KN&>>uVL6^dxD&`A=CTn^g!SP%!tuSKs}Md3 z%c~XP*|7h?-yx6N^DNBoG$PLm>&JJ55|aUYkbhj!3G2*=}1LpUC99>Vc>8xcMX{cQ;Mf#rq2y93W}H8{as4&zHdNy5Y)d3;RVpIfN*>suZeIKXy=^>$J>Ps z!twg%iSR&Jz99%7fc8m3xIN5oD#GVrdU6m>1UJE$yM}OG7;h!QW8i$F5#dMRINpJ9 zAL#Ex_&wN8KOx)@>KB0JgV&SoFg^ANzX;1~FT(Nrt}ukR!SN*(;a1Rp8R31fU6dfa z9`=Vd2$zBSA0WI4+W9%c>!JR42;U9I{aJ)tLp%IJcp8*vgXN6dEd=%_dz1EGr||3eszS~LwRe2pMvG+fpD<*8*sr0&w=y#D1=ME`g{W6GSHqE5&j0w2MQ5> z3D(bYgl~iCY(O{*ESJX!=YsWZ5aEf?K4S>)g#LMiABX)R11tx;yp&;nIS^h0{elSB zf#r+8iyqe_2kVC-B9Fi8NE_iUFkdDJw}AF^K=^GquKOaK1nqnf;p^c1Ed}8{Fy2cD z7lPvl{w{YsU#8#$bESxU2DEP-!k@tQ_z2jV7aeGcqG&-j_@9so-GJ3f%R4s;oo7s-G%UVuzlGg9DfI#7s7pDI}Js+ zDNKJf!sDPlPa*sZ^k*V`3AASs!Wm(^zKw7LYOxcV5Izsbk0%I!3g;~^5k3I*OdvcA z`WF$d0LNJ-Snu%i<$&`>PJ}bV`nDF~Q?Q;(B78mUH&z85dJ4D5!#tNkY;?+@z<8?4v3d@M{SKf;S)d96pdC2Xhi2{)@cf>J{Z9rW{~ET_D+tHmK~;fp z{M~qs2)_mU=T3y5h5QA=3n2f7a3XAAwNTJ7$C{!S=Wu;in;AiSRgRH~cvY zu0Iu)BmPb?oS%jE7en+sfc?fcgl~lHR14vzu$}57d=%D)T?oh5_n9L+5BhBo&I0>$ zXN1c@{{e(2!FCsc@KI=oScETu_D@3iK4|C52v>#jR}k(E?Q;X+La-i`Bb*-gllKsw z4%=5N!s(&^8N&17yz4E(J0YJz_)b{vKM?*E_CL#D`Q!CO1pHG4bNIXAaDEBaHxWeM z2=+5_2ych>S4VgWtS37Wz5=!*TZH59{PRY*6wFr`!ts724&hDk{yhcZhLGnVd<87u zn+Vs1{Z9?TpTU0kA;QC9{pm;eX~^FrJPY!9gr~uDGQ#?Tmx}}J_c#$=1?vL|;U=&i zNF!Vr@~sGe3)_VO!i(WN4Sxq8o}Qmj4u9T{^QGVfbNG0Ib6wbek0Sbi!t~(xXSn<% zESF1&JQ3Ezs|X*1{ncHBXT$!v72%dJJ2gjpfgpb1hY8K&v)a+6Rb?h8ZKM|G- zJ4`Rme?t8N2seiHN&?|@u$?L)oC_$#9R7|%Tz@&VgBc=Eg7G>d9DgnlfN&3Jw<8Fr zgZh&YPK5noI>PtCdRvHaBUt|{5FQB2r3vA_uwUvz_*vL~UL)KL*5?lh?}Yu*cZC1` z{}b%LaeH>c^5#W&F4QlE@GaD0A;=?K7xsIa2yce+#t1Kh{f|Au_rvz+gK$@vpKydn z!2aYI!Y{(|I*;%Nkmn+N8rC=b9g( z{ucUU5PlEVpEC$Ih4#!sct5n;b%d+Ha=C|a7ij->gs+4AIl^aQ|2Bf~3$WjwLwGv0 zCq2B5m)BOvIS?KQ?Y0Kt(J(!d2ww&J0TqM`K>cKduZDKRpC{qzVS?Nhk?(=|4MeyM zY>$x$_lD(r0^vN+pMh|EUR#9l-LRflB77Y37KDeuaRh&!gy)MN_S-{<{41FL8HD5S zF8Yb^N;n@|4*Ne`KNqZTJP60<_o4{L(Fwjz7<=LOA|BycOa2^Y$Ku)`w-w5J#%Zw|{*9^sZSUz!NN0reXrJQ3=*N4OK5KlmVA zALb_<;ev3SIEL_DupG}L9Dfdzi|}IDKbImL|Nn6v!i!$}_M>G*$4cmn!!dqc{Aqc+@=ZEJJz7v*LDZ($T<-11UWYi2UT_omg8C)4)mOd^6LIh!tp%@;aN~m4#ICjj?ZUt z{jVT@g2;(X$Lo_WY!5RPxm5rA-3XwPE^7l%BLhJ&J2g8FaJa1bxP?&2}R@&0EN;rQ?W zmx4t~n8WRWuR~ae@HviB;2seg28R1rt=OG+_&d@-^fqk+- zW-#}Vh6DS|!1mh@Ip~MM_`4rm#M~Ez9&F?^E7ZH95-ltR}Tm;VNTObGN zfuh(M)?bk*2a0|$u7Q0QfEy!G4!^l5aa9T?ENFxtbU{|rVhr*P@?{fSMFLEJ#0&*dQi$mjyB76wT zr{N&K=1^V<;rM(*2jTeo7JY>KKs`Gl2lk=IK6rieK=>wV_6ZRPw}m_%a=iYmhWald z{206+yo_-Cxj_NK^I^Yq72)`EiJJ)Lg!8a+gl~b*L+<>A!)ShwBQRTp9q8)WdH*Opw5GbD8j+MGQh2XM{pU~tqA&MXgF>E?DQ9&gmAom=F@Ok)j+fa zd>tH!mzLx6dYt3^0N8g1IKbg4#gG3k59Jn6Da!XD9B(i9ybrVs*rX}?;jsEY@scQ4 zsp+9jzYGnhwVOH(r>&o``=;mzdhoFsPA4fG=z)(9DSIFQRUltL4;h?b&WL*W%?lyl zgK+%5lD1vY+B29&o;F`cXgF=Y5@Kd>Q zet+$XaIha0a9-5IZ+Q{O{SYn*c@V;7A;<3v@pP&{ei)J0h5RVOO(2g)I6m)6LbwN% zKaKD($j>7jpLbnEI6gngLO9;PD z;PxDW@~u!FR1b(KWFxFE;6U5Hyl6OW`|6|NwC#&T(MHuxtKW=<)9Ux3;k5d*XgIBY zf^^UUUu>Tz>=+zKJ$mA2!Pq8G1Zix;C?|rnb5IaLI?&C>Ng7b;|I@})aL=s&OKIvx zSJHw0ARdqiS!rz7uK&B3s9RwDCoHz9*Z)gcnk}$kNEn2&Z5Fm4mb*WeX*cH(4+ke_ z=Rmj6|6KO*cLDoA;l_Y+flhuNZvSP0ejfiT=;MWL*#(tjS;bEQ2HRf|q+R^6w`==h zR`>G{c9X_7ER~{c3JUj>k9O#e1IcmQ;D1grcfGlum`awVcr}W>1>Bq+F>hWpVBYBg(6M}fSoEx zKe+!x)BgiwAbuDfc1CsTu+tUB0X&ir|1vBv2>)9#yzU9Z_uvyr$np50mjLc>{*?YS z6{-Y&6vDhj?1P8L?Z4_T@vFn%4dCnk@%ZrYxPQZ6!js_*9oQd~)}i6SzGQz&zf1@< z6xeSQI3euAiGA?!c>2};68|C;z@M`M9l+t?sXpvE>|f$Pwx22y1_eRkH^3J zFYznE|5v)h@c01_kNb`O5^TdDkdHtk00!l^C$a1 zhZ%Q-9W#g@#EFN;;}3%I%itHFVh;EK?ic(&JRTn|h6STIuqFq#XoFlK|h?b79!A3S;La z?1P41iZOKT#t=aq49u9r_p_n;U|)xOup@XZ^r!G?F#T;Hfiw>B5;iHwlW3i+qJqp8 z<;|PrWJn}Ad07QHMe4V#jEsUjCc0S`lb{4}To$2Ys7p`xuSPn88nrK`i4UBz?E4p} zQ%=#+X>x%dZSlR#f=lh42d1Xr)1y#xaG|A#$sUVV5pqmPb5|bo}8kHmo#3e}rkx7!A z{9K`mApD*hPpyWTe!_QhLa#YFp~p1g6FHeOrv0My?6Gx~pb7-@K}t@T#xI0yssycu z?DEMlVbta>C~-S-0%aT|CsT$a4clL$AQIx*NO1UH6_udc2Uq{$0tsBm{lf)N(SsA= zg<4RhW$UOZ8uP{WImP@0Q*!hqiJZ8GoVcBwNXg4Aks}oZ#E#L_;|enoj#=UgZ4q8- zU?@mVctxQsREo(PimmmdW{Hw@jqax~Qva_Xdj-HA z*?A2M4RZ}MbC48ISHna}QD7>=1RoJg{e&N~zo_c<6PolBs|z^@1gam}2{LlBnTQfO zIaq{Dy_(P%TStt!kM#iLL~ZO!Vj*~FNRGyC-NPP|6YXT&iPOJk$wZ06>R2AOintTS z@BhvQr7*D!P|5?l(&|n$Yrr!0niwyIX~eE<1E#|Q9s-8ppcLcp&|D%Iq7*3>bcSlT ze}mX|;sIufRxPtc`%X*>Q*Ks|MX(qui={~t^G?@cV!iI0Op??V` z`Vuo@BL=Xb7EiDU%tT07h$$fFi=Bx#V@y)ycY*||iRF+2N9=MdrkWDO3{xWgo*E|> zJta;oM_8PFc$}}XINyLcN3l4YKwvD+2h=$6jQozs78E!K8^}M!bl4FZ$`%X8g{2?F zi{+bQFf8vF6~`hr!xZCr$J0U0`x`tR@3C}@fpmPp($Ngl@en&wRAI`WkdrBK$zqnW z!t!O+=}sp4H4<0p7!zCS-n$czw73%k-oJAvKEvvzy9V(H#w?$@6Gyr*K7{cj*vXMM z*mW*wI;^FV6B@}0Ez}H>6Pn4OQh{I5U{(KjuCWR}OSFuIO@(qCbN}({mpgH-HdaZn z@}%Smv+5H(H_yq5&;$*AOF%jo$qASmEPGgsq7*G=8H@I@EKCio=U8H>F(!x6kzIa~ z>qi-=e(ZvN0<61+2|p&dsqunpul}o%9No-9PS9Bd?L!^3598m8V(UWsK^E`pfO-L2 z6*-ww`4e6mCj5F2@c%CWON*r|c_SmGVTS}^?cdWdVbR7;!&bvi!(PLo(OsIjHU>mT z_(UAx62ZO>nGONQL;t7jD5>!QBP;KL!=Jt1Nz#9AN3hyvZCVXeAKVjGj|>3)LV0xfUve_nHQVy@$*~hpZ93HqY%lEyQIt@%PknR6{BYqr4;}WD z1@11LSB1}q@@lQ#75EpY%#E3NcnKAUh4wv&Ejznmz@Du4iQBEh%}u0w{k(=lt8GE9 z%UYF*fG2gQp9@{tafnSMA%aeMNZdGn$8jqKvt4~=mzJ3K3n>v${2%(=t$U`(aIdyJ>Ydqc`)G)8m)L`n1+4e9P?Sy5zBaOV5cG zqljHLZ|kN-XHo)7eB-MNf{7m@hwKLlhYc1k=jFbeln))|^AP17ruPO7+-!|(D}Zk6pPUd+D!wNI}`gxo^W9qT}V?v)+JuL;RMr8bp~098WZ~CO5C2p<5BL; zZ2X?rs-1h5Z9Kx?*>dJ-RH*aTo9V3#f=$|4XVVyc#PSJtO?>LaCxkgp4|>&OUg~RQ z1)u9bBz#SHOkZ_O*HLjF`^|VI;&Z(WqJW5WFcX8ese8?Vtx)MK76mAN#iM@S9hatydrOOR`NOIY{lFnyueSGu^&Zz3{h z!j?I*-QC6Dl3+(*==vcgb74`i?tO3s(Qy4w78$b_9#NkXUJ%|sJItP@wUsDy>;+Hc zDrf(pf82Ww?y4l6cJ`cVec~c(bvFHBI59J%(xvwr8+|lT-%PM92D*TMp`;yyn6E3kX z)0JOh?t4F~YS_4J%gVP$X4|T@Ew{^VIJDQ0Ire91)1dM0`%z7zCK2}fw)~&(M``cq zUdznMn)6fStR?T~J<7p-ns+m5H%e<~8az?#Hww&ROL z@%ij?Vm*O{Rw2sWMb=S4KBnab6&HW@n{O;0xv<8f_`1OktN8Kp6v@LX=Dq3{&3iTN zip(qH*7SrJ-|${urd7Ub+|-UZW%h#bx%inzm6@QG@~sy|pT!3*_Y$3GD69zJC!08oZp|8RZx3E!9-^h+GsPPdSmdu2QZ%jp zxoC~}jjw(~g17b>zo6Sv^r4~avVO?^vSwR{TSYwH=DyMWW(%*c4wx^Px2_II9{>5! zZFbOH)p_?Vto@=#KyX?3f4 z{J5yJs@cIiAB?LX#>S7EUq51gLt}YSuQKPAm#>R7%l~<>tmmJP@)ymSjh_{Tue8ol zY5iO>&cV}KRGTy6-Q(nWZG7sD9CL8tOO3v8?o`IiBD#ky7tN=0+*8N5y;tb@mbAAg z-Zr9e$(>w_!%f`B##vl$8>XhoMH!};mClK&N@yuRW6GQGO`0uQZo7FV-egR-y<9-i z%5u0vZbQLy=>U`65ytD!irOp-GLwrcHcg>7vDj;LUQnjTyH~oZ%OF3xNA6wfdR9{_ zIo5y%19qlL7NheFwsvAc>MIL+%&e3?x2^Q{_Ow)KiAiV83*1vs{?^RR;)H4UYrh%E z-60uTdvDmd5@mYDTduv|%ruL;=*rUT7YlDm_!&HCNJzIAGBt}IpA0l9C@Qhd zuXt1x)UIt;^l2rFc8{TFanaHrpNpiAh8DVKyZ_^@&>mVOt$S*|cto?x>{Hf@B6rV| z-fCsLLY4*_his6~vTmi@Rep@&l6S(-nw<0FM9FcvAF(=z?IWB`9?U7OD!(_gj`i2F z*36b)ta?&<_AGj9F08J1*jvylW+zcAhs*?>k6*q z4uOjU$^Q>|XC9YR6F&UAMTIs?S}6)`w4zme?$WMk)4r!gn-&r+q%2WUk+KvD5vhcP z6iO+gNF+)NDk)L#%-u}xXP(FVdw+kw^Z9h2bG~!UHP@_X&gqg4WFD@#ow}8h(arD%x0)x5nPm4ld9KY~;nds{`#ojLkdR?S9 zRlT{Dm2lDCx$ePDp@H@*GtK2pU)?ybt0ermri#_Z(WQu`_=Q_+{sx(>TICgXear=& zGA&DMw1XrLp9{|jw!i5&s&M8j|Ltnq>=VM>Uc+{~3hsLv1U0Z1cBL3<6l|9@lgMC8 z6_eV{$A5p6Qf*X_x^v_AA&dR){i}Aw^}9cKGrqdr)8Xhd?N<{5A0FN~nCN5DnP?K^ z{l0#1a?-$Ga#GmjnW_cjh@4Par7-_5m&Cza(wrC9w52y>YWB3{_1y9GmUV1D5i+1r zzA(H>V|hf^ZsUahv0TC3zaJGeCNEg6SPr&@{FXM^$?C#a; zpO-Cn{9b+Ja{2EM1#8D03nI4-oGS}XA1YODmRR-psDjwjY;#GS+!b|3?$zPaX`3>d z6~A<82o~sz3I6_27Coq@5y5cyMTXbK+h-ca70Peiu>Gi}PYntRl-91Fad+1Y<%<`0 z2_1rBTbFljmWEAgbfe4>2gOAFTigd;4Enzj*(McuQu@nM}>iMsaI$dF9Rdal$l=_(4Z#5aY z&QhV?x4{$rG1K>iPNExCf7a1{7X21YX58v+!_Qux-4*25#Aba-)9%i`=jSw~WPdH$ zd7?*L<{gvL>H1NNpWy@TXNL1SFTd`~R=*iFV#aD7R2zJ|qbgf4$9%RdH;)&4G-phn zmNUZ!h9|R%R;5(#QnOWc;+0xf&(nF%jbYK7My{Xk2HAd&10`qMuRl(I&1K8jw%qCV zdvOMpRoVJKKdqi|SYoqh!S1aaU+U%ES9RxdTJvyAe825tYqv^=BbV%xHWwG&ZdhvT znlU`|(jry%Xri^K`l+geZ9gRDov|p+6MmiBm+_p%N-kSA)cf*Tp79O#o=JUnrw+Bh zW>DgbSiyB-dp+}SOUp!^=X>rR_V+!Wrn-nf-mm-Jpsm@;@6wZ8eJ?FV%MG~Idr}O~ zeaH{J67Olr@vb%UV!hA4D`#Y#ZS#Kq_;tu*MB(?XF880$8`XC@bKGf=qR89~$+SL@ zFn-Uy_3UQvu?yt^>=g?h)?5XyY-5U}J7hZQsF`&YCN60=xQ-saRub`PUGkc`LmD-n zRbR)Ad%F|;S>Em#yq8+lobqYX%cos!&D|3*Nkw8G$}JTp@(PE4Rn!YH@}{i;Za64{-cbn*r!;kxIP&aCJ;u{#>VcMlSpgmglb>4-=%B)TIbr-nq zf6qL7_6H>w#z#CX8TLj(PY-dA94KwpPduE#HXKL-nMK2g_ zqEEXY(fc8g`RvP1c9B&dz+O{`hG&#Yy812 z?yeql3pc-}U}>-V-fiI%x`#zvKQ@$0`JAuMk+=Kde8X~^ibRX-=6g>&b3>H%?=LUR z4^vsV{^w`qpXFB~O+T*nkrh7 zf!XxUha6sGo@>|T1pCe(o;7Pp-nS1+?-w_oX5Fc^Q)`Y|FW1+VBjl*9%89=G0DdH<)d$Ua_VvmW8TcayGnF)yV}RnK*Jgx z@qP|1iDpKxg+q~_gi;%$x!l5XR|clEtB>B;a_WVyMb?^`8eByJyi3cE&C!@+hq+~j1NtIs0mqcIcB){ z^IEBwF?YC=5-ud`S179}Wu~kyc^Y}ysjRv$cy#`!;^_T_dbQnk0cOBceb+X&=ptR6 zlJva?`tSbUH@foC?(E}1TS{7 z^f+=&CRnNQVPnMI^JNv^an>L@+x7x^I*0_~%QvgTssl-zU=31^jJt~mq zSE~Ja&zR_9^`kX@0n5iqK8ou;IJCpU^_t|nL$NAG*=rSq?jHHwlQm+t;;E2f|I%53 z`)kKftS_6I|BY@!Eu zrb*IU9ivBIqsBNc^VoeR`tv39KStdD!jH$>GzPUP&Ap#L%32Znt>5AK+7EYqA0Ln` zNxRx*JQ%l|O07y{E~R+c{P^CcK%p9%k6buhb4n{nwyEh&Y~P8u-ZH*CX?rF6b|;&# z@LYdA64uEz_TDx_zh%j>NpF6=y^q$JpEt2uw&+7%@4eBBtDTouL^rKw$$e{_YQ5;n zsi8~H7T1`xwZ7b6SW@1aIGj1GCm?uz=Z5`z^PUIYdA@P*=2stON0CcwR%~74u)wNw zbx<#(!{Dc1-peivFIups`R%7&q^FVaga9A=M%qQzN%35>7F{J@+P=xQGv#7w>#y7N=&HvNt{I@qQaM`mdMvN zD8tRq!x+2Gs|Q^rKdqYU_K(o_+7~u+Qs1lW8|<4@ab+mjSkMr)dBb^2(X9%+ z^0A-0J~s7@e0N@U^72Xlv5l4&Ry!~kloy4X+s2ot?|a<7W^6{DiRoy0_=H+i`r|K- zU$1q2AN;OY(e$W0#_3XK;i?i3e&4-i`DfUg-IHQA?O%J)zr$*!hPsTy7mXbeoAU}+ zN$URUT%+Q*YK^symG*_5oD(<4ZoIlkopbiryMXxtRI~H^xfUk2YrBH39!XQVGv}?{ z`figb1K&L%TQ_W}_*_DrO!!G<{?z%olFH)n%|~vaTg^bowJX0zg=6)aEr~~ttdnMy zJyTjex8{+r>?yvVjtNq)yrfvJoq7FZv+@1fZ3mV78YeCpzLH+EH$6IqJK%eiQ5%y- z(7V)&4QVQ_-4P>Q&e6du?x@+neX_{Ybp4q7uGw$9<`!))W2#uRpyh_k!ZV?vy9Pu@ zqE&a6O>{C`_dBtw+s1IN(srf}#~gRHbG5n`4zzTL>_7gfI%jZ9Ds;vU-UYp>6ZSaO6s_JY@3-uZj`;kmX}Md+?G+l8msB$+_dH9jE7Z`?>$lncr9UJp z)qTNolV}%5*YTKrl1_;`Pku1GexmSA`=g5k>Z`pkc>gvSsa*LyeNmH=mGCA7*&*v& z!t+lt>Xa|b5#9KE*$V$OD?t(2;T18)Dk{=>W;?Q?w~9M;D&=b_*7yH3iSrXyHkdz9 zqqgv;zU0@>Ss#544dytn-xmHRrki?KVQYNcc6}z>p1RSOr+>xApMNj-E1(Te>SpI=RyVAi>8DpTCW*UBBbTmRLm zmcrVvTpc4i&xPC{zRY=N@c!n)jy=cY%nc^Z$MfyW^?yb!(K>aiYG;|!skJd*jnihX zJ|ZYo8MX0Hq3_^jhab9(@0jMzkhokE#}cNe@8cGdADS;|9MPV$L(JQEEHJwGeHYTawUkko?MklxUStyPY`d<<7M zo*COCw#~in{Gk(OlKI(@$2TyK?sk9kQL;=VBYvAU0QnOp2#yx>+vR8K|5rUY%Lpq2O*=6MA~}JD)zz68{a=9j{OS3|sDI zddAF=ql@a^^_&vh-f(?^fac1SUhUDD+ZE=8*B`N;qb{3KtmXOimG|`{!=pwAmYz*q ze$`l}t-kRjPuuvXSA8SnajAP!#e(JX*r|Dkt;G4ABsOp0^=iCq@zwJF>oKvrAC9?u z$G`3=d;CqrczDe-dHbA)d{vj!d`-$y7da~yC@b3?OIj!xBfWsC^=8A+6~i+%{KBy( zcN>nhbAIz=KjFOp>iOpko9(I!XS_Uo!LxN^OG4zfC--HYz7}=1-e*ysHL>Z!jV00s zZ3c6Lde(PQee;EEbeLHLgQH#eCC+z5r0z~$U80cKZFu=v)cJdIx5cgns+U?B*n6oK zM4!lc^P@;v`q9JvC1)!*WX{I0KR1ZtT9jop%OFlFlCu7X?Yt#HI(zSk|K=ah=ug&T zlu$L=6o33opXKkxJM7OM+?jvSvFTxeHVgBKUKiRPN*e&-7o`tgl%^R#fJlNaMDr{oEBRzF6`Q9tTi^jj69$&}H zuKT8Qp$bu z$!T?h+f`ZD?}87DYsM&#r88KTv6`L%cfd}RIi6grvEugH#QRQp+1#3PN?qJqADKh~ z*SG5Qbeb2H+H^N0?%*y}X~>A{Elr3$XR_zGY>=(IVe`hb0RCAw0`jUfr>@9H_lfNWQuV>$! z-`;8@sS|L%_R%llMU|AF${hKk9yP~5o))l^S2N+!lcJ8+Y)|WP{k(Oix4p=~YihCG z{2mdHO5N2}8xD(JaLyC#`+h#`@#_xJ%j-Y(pM1)x5x#IG?+?#>?aFcA#!kUcq4R_E zwLWot|E1NS?8x}KDCV(R!zYKVy49bX9P;}(8D+~41l>+gloHN@c>ofd+!~rTC%kUhs?Kz zo0;n`cz3sAo)2sP=X;BT*G81>l?yz@?c2$;{(iHJJjWa!+vN@Rf;$WiCzFJoP!}XWkg*@jiiCC7i?*Nau9ADhRQrCj+YvJn3ycvt`ikWwHvMwqQ zFJEfeU|vb-8P=d=FI0cLB((SZC)I74Ki7zaWH(P=huudHSN&P1o;B|`xhKosH~YFr zk?}9h3d6Kz4nCO&CBv%A^Ox7{i#xQ8<6Ce2(b4+E#R|(34)*d54K2Es&bRQ#%uj5m z!hQTKhOB!s7#WvyILvUDTUF2yp!f7uR9g4G6aF2-d0Zm`=YO{Ty!l{*$ii52nV5G^ zoa$LiD;AA+-o6;rEB;}FY%>(AP=4JH;86>_vm z&VFv#P<>(Ic%DjJkDYOi-15yj1z-7_EIsl9ZFSf=Uo99m)W3MVQ{cGx%kmM6TO1k3 zug!1Ik|{A`!E<8_mGIUHu2%+h44(;u0X{DY14E<*g&~}qL6Du5?GO-#kIbOQ11V3SfHo(0 zGUGZ*F~XV&HvPoAO0DVxBe#i_+fz|_q&i&F`x z)8Z5dGTNMcAdNOB+j`=$MVe0h1Ix=y`9^yt2{XK9`p>7?;4^OMCKDD!Tc&@!CH*Io z>0eJYPx;nlq~o-@14-a>Ukr5lvLQ7FV9PlD#2%_Y6vsrz5u$CF{_&RFRE!O6!||_Y zryc*oA!eF>vyCP+{loMxEmKiUv?l?l#inyc^@AKU*dWx+wjFXnRc4 zo=N{ixYKt3-7Y?elh2d~AWl4zjrtt(^gGNrhX^E(9)-qusvrMHyT{TbNbTU8F*MFT zogVM(M;x~|=7$jn&!7Dx9HkvGPrqY~BTvwF*r(rV!2C4WA$7pGvcK@kzwqk6@aDhp zKEUVF^+VcN1`K}d;=k-!_!lns7q0skZtxclpV9uWeBnF4|Hb|O!h`<8cmIVa{)K1# zg%|#X-~J1y{Nt@I^Z;t4`$|^|d@$8<>cLdUDIW1(%rU^jBiKFUA5pG;fs~a-I&!kK zU+{XCk?ht3h+k}{;YyOYv9jn_ygVpGtyCpKXNodIR%R2IwMnUEh9%$efX7A43J+3V=(9^&W{fld7HqM|YeQ@osm zy&S>oa9i9RU4z`6L)__!hj&nLh_8DDJ@W@|#X+|_W6vT^_Bo^F&!5N`!GOxILl&r^yoItmpHw=2^0na%ELb_ zZv`O;kim05=9?h|0p_~|{9*13AqY?&zSDp?K6jIL#zA2apgf#QvHaBhH>IE6&Z+rq zN*`fS9NGi_u)HkdP+p9VuR)xY*Z+(BQaZkj zPR}YjzMPKZcd%i;uuZW&^z{SdFi!*fFhF|*z#r!H^#hXKmNbsOdnAOr#O zh2RhK#}I-5^MtPvVEz(95XgM-ePzh$^Bti{Pszje7}jGC<}(G(}L`4x10j7~p&JFZ{?LLfjra4m`RRj1=JbliZB)AvK{vlr^2?}tfr@^Fk` z{f7~UX3@_B$LTn{*1+Mc=YF8 zPt2LnlHGDD`j0#+gcHGI(Yul0lt&9#&mgR95a_>=0Q1G)h~7zy{bSD^bYjHwB-ZmD zwO=YB|CD74tQ=4gyzZEx5@LNa^E9w`x9`ax#gMhycin%_j2Vp`U`(%+OxDV=>@Lfw7 zupT`)C_&IC_+C^$8wig7(d|U=cvS!X1W!Z#Z9Bo;P`}zo@I$B`juE^L9pi6(V*3xF zddA-(!TkJGu_#r99^aW$5e)>#-{NW~cnYeYK7z-f`uR!lBzQ6bfep>`*q#^Ycmct` z!$}kb{LK+8pM%OJPsndZ`Dqed5het|gy5s7p6v-fi8y{Y73a$WUr+`Ce?tUw4#X1( zJ@XMyC3rdFrwD!#)!P+3HajNnJnJd4{0+ry0d3I2u%=7K0+{H-I*+fe_<-x|Xl zfAe@Lnr|>ajmEnq!Ox=l_ar#>#T`O$KQu0O5xgAfKS1y!XyVTzxF4Da&JjEewc~Yy z_oIGZMeqtFkI$#Ly!hjC*hj)NNA=c2=$X3j1iqC?@WrV9nNYi7J$T)`kl@8=evl-% zE2`%;1fPe-g)YHak)7rQmq+!9&*Ru0UsV6TguEjf@8JaBg6zb;HnE;YRNljc{943M z68ti%|3ZRG;{H!??7Op?;Mmtw6Tu}>J#-NK6&iQg=K;3=F*;xWB;=(LXGL2q?}f@W zpWxWn{Stz&LH$FP;9pUB*AhGl*>6wq>nL9jf*(ctqX^!M_%4DEOigc;Jp|`L?Y^Jj z*q6=`f@9wq83fP8>l1=wpH~-1j^u9;+zBlV?-Ses_1i{*W8X2@rzvhP3skONLOu=E z^C-c;q4{?PvJ2~Z2zNl3OYpa-9fb+L63x%j1aC&;NSWZ+*BD;c;Cxe2J!~N4?a{Fl z!9~%$;YV=nV;B2k#QH^1eeNOT52E=qmEhxOKEdsb^)MrTiI87BRc(};1V4!Cy@ue} zx5#sXA4PiJ6TAbp;}?QgqjktHf~TYUnTgsB+n<8!VFAG_(YO;O_(n93$r9WSDg;5D z;B!&Fh6Eo#=e11)uR!yW8^Mc_o~;Bwg!*kX!SVc@L~uNxA0s%Px6cwBUmsp2xEreH zI|Sd1%Jqcc22<5Tc|~wl#6J<-9L?L`34R={8<|l1<93{d>TM3epQHA|zO1nP1#~`@ zBIJD#S0eanG=B96Zio6m_DP2IPrV-uzR5?(V;?|U2>t}MV+6tD5lHT$_<2akzd^j2;NK7*B=|OTJW6l{BtJp$Dl{IM(L99h zS&ZhxSp?@t=R0nK_n>xMK=4bb|12cPbEhhLaRR4Dfo{q}F9_a_=IsvzFGKUvAi;G|`{En= zxV?tZyv>QWm@A_5wGhFPT8bpW@pXt2!FNq{D~dkB7ohsLBzQN<*M;DF(YOdC_y^SA zq6w~q`fD=5y%5hNcnWIYa|HKA_LmSGpO5booD13cjNtRpJk~*Qaa5rL1b>eD|1W}5 z(L6H?<%8R63u<>hfu?8i!B7lzGou| z&VsHJa6992?MCHKC**@sy`3TWDb%m75*+&^tR#3Sn*Zwwz5tz}bwtz$8GAgeu!LhGDeEo;@tU~Q>M98b6`m`nZ zSJXf8eKM>k7mY{oof2BWdi_e%uVM(tQZaO`vb5y4NP zdHW^7by0nOB)A*8F2m<-T&}yQ-I-81Fh7Up4Gw~LAudR8EH6RuZ>U|?5PZ{A^-|Uo zJPgU(65I@pM>m3dqw`K6!81@jCA^S23keEGzPAb38ipWg(xL+cG@G+wctf~cQx6FeT(1NPmE<*`pcNkSgC z?;3(*-`{2g_dxUcW`e7s{^>*T<}WjV`(fCA6*P{r33)Eme+me$ zjd&Ts@qHrfn-=TWL-sr;Lf9l^gM{dxqKLHaikd;+zLGr`NzaR9+zBYSob{5_g~k_e9P!(-!H)ThcJJK>@+9jf1~{HbsLr!K>0=!^6Sw! zJWcSqsQ+9e_$@TgG!uL;(*K#@ClSX!$gw@KX#FXK<^#D*35A|z;KNv`#?dVhMlm~S3 zP!C>bKc(YPj|Wr&LKDI5r!eIu9Y;fx7W1BtLp?{4{6~TxM7)=d!=l1x&R`g(<4`|7 zzfaI{sQ)$;27wW+i(%bEUJfA$taKde!Rvv!1jp-&g#^dzgynP`>TiL{L69Q&^C?WB z_c;&q#qFp<$oC<6yne^_e?)veA&;MPG9@^EF3K8l+zfcQ~@Uqn2M;P9K~Fq|Pces1X;!Le_$D+I^S z8(k+jJDPOwAP%#GZIc2s1)+`LPp2^DH^Jf8ZDC+R^9s}f>)#SW5b*!ial1#NeEI3* zVY}exjg}C6KayWY@I=IA2!0N61%jVJT#ewj5!WPm1>yz-uS49J;ExgCNO0`4$d=$A zkh~k>xc+g!iXk}u275Navyh%*Iu1|mc%HdM$6*^_A9QU5$G)3}2riG-1M|`O3ft*| zIKDo{Tms|)gE=9uJcTI^1mA>sG{NQ2`Xrs;T8N(}xG~~42_A{~JAxlXe2m~Zhzp=~ z7AzM;XgNR;qvNnXuON9nIt~$9o>NTeIF!fFV+GQ2C=cgQ7$WI7ly5@%v*gW$)Xj>Gwf9o1V9!5IrgIMlxx)z4*u zE22uOB)Au9@5cn^Kj9Tbjo`RkRK#()Rzc%HaHQj~TzFjj&~aEUFDMMcPC`E} z*FJ*da%B-5+nGmjZ0A+Pv7O;l*-~ohIJ6VbSIu-B+PMqK_YnHAox=pjcFsTzjoS;` ziG9pqj_nje^4Lziu27)k&`#JkFs!BH(9TR61^y<8e6gJl1jlv;5FFbXNpNgu0^-=t z3nK+qz14dQwPzmK>D!QUaik>GC;cP02B;+_N_Kzs`whkd~o zUH^vBaab-kR7sHpXF+@?9fx{cke)<34)yRM`TYdP=cB`PoTdjR0^v9vhX_BfoI~(s zNdH+n4)uIN=fTT#9O}_Q^2G$#K)e)j-0q4X4Ga$mZa9T0t#o=|zWDw|JHfe-J)Z~; zKA=bo0|dwSD+US92RkYVKM0QRlZ+GG8V&*&kUs?6&l%B##rzjOli+yW&Oz`oSZEN? z>52mD4Vg&eTD0G(?`1$PjgCDDz8Lk-V+3D^_#J}ddH5y4V^Mwf6Fe93`OrWRu$>nX z$G(R#$Je=@ggoZ^34V4;E#(ry@pX6OUpOzyA4bFa!SYcA$Jc!a{=&}_JOt&3uQRZI z{9a8xA&>c|zi&4p^+fq_aLwy!znzyPJwNR*8(sKt`%V*>rI44r#O9iW$8H7g#*_4 zf71iyolyb>w1x7J<9=&JaC0aCg74JkUv`Gkae6yr=s3NdOAv<)-nzy4Eh9KypGXq? zK001WaQr;8Ji%R&{b~d+L0pUAPZ8HAcsJtf3H}{%GlEYbj?0J3%ZkR2Eg{c^IDU@@ z%j5e+ZiM_2B=1ddNyPmLj^A4gCb%k+$MYr5cOBw82zd*{cN2Uw;RBwI|#_T{wuK z9he=x9wRzVuLsvNl&9D8zw<#&=O`LFd!{@5hu`S``eX^6YiRmt+9yi>;x&5uD=5>S z2Ekdn28X~5r}75}a^RaKG$sGv`;ms1ZLqJfpniZins|rUqV6?K7lS>>(CKQLYfkU| zbB9d797Mw<_utZ^<@mG`@))kepiVrWVnI!`BZUFl;{5Se74ZK`ABf~#i1avroZb|r zC+n{RrF%l8$LCRu4Z#jPMQdV!whgdgAb6pSM8H1G1Lw~N_F;hYCln#`FGA^}p%7gl z^M~{Df90=)PFzkXtq9n{yvh6%0sF7~W#PmD0=`c;b@XQ)fsfc_pX|f6agDf@Q3Zj?cWL*w0}AR?t>tXDx{efImO^FPLBz^zWA^7 z_#d)Ds2#C9nZ6rD{FnXKUMK|`*dpL?4*0|AvHgr7^Z&|U1YKvYMr!Z|r^m;?!Onl? zj~0SXNKgc9;W`ed$NB%whc^CyVIWG6H#j{$o&${kul!3<{^yXO2-xy~Kb#)tFZGx5 z*P!xy{}1)Q>@WE{ZJElr0i}oe!&^dR{%cVFqz`B80~+TwJ03%}!r_vuAtQF?j?_eoh_09XMhs+oe*|I02y zVJym@Oz(v9uO#Gg`7w?L>Hn+$$OcSFP?4Yr*uq;DI6dw^usmem08vDi!Cwlc11bk$ zG58Y!e`I>N<|EI&08M?Y2yF0m291Fqm>+$BWBR|+ha>y({RNzcOi%l^Td=E_yIbfM z_mu%M|C0~sRSL3eB4loD!CuJ#xX5G`;XplIvftuI^ z90apblOmCi-(C*n8+AmJn;~=wH7N>3P(jjOT2kO?l+8}d0r-Q3zF4VA6oaJOF+e9Z zDG>WI2C@(2Dtn**F$FkD9@(=6R3LN6tfW1BV8RA6v*~=40#kkhQ@E)~cc4FFYNDkW z+mwS*Y7*E5K4X*0;d7=_O!*$y$x2`dH&;oe9j(M#o9H|qE4fjVrW~U}AF$%o#D;QM zXliV@EG1+qbmJ;M?K+h!9)9Ku=u%2-p!1$eO&l^vtfTJVBF1NsI7v;q9SAgFgKq8x z2L{-Q?NmJgXRyHMGcAUiI4~udSOuNi!es-@Qv~F%Lu7e>HhYZ-Tk?)^e33743#sA5vkc&W82_lsY67{J)j==jW zYS%$6E^a+OgQPH7kVCCp8xWMN_u!Jb%azawtfEDWgJ?}!G|m4pEjrAf79|3r%wW`X zx-g!^`C>+iKc-5YG+(UeIPeX7h$aKHa z0DsRwKY$u&D)~TBtc_*i{zegy`J-uLL1kOlXmTBe-k$$u?LYG2I(lzrZ|o2QS4msX z!+$%&rD$0KrEM^0gT!H)WAmxreHeBfYElC1PSm6wpmw+oVT((e0ZYX10+-D9W&oeX zz*jM7e}FY%p8yW2p^t0elp2i*k3%pm>W*M+#8-wGP?MMpk}AahJrtWk3^hqp1W1J| zVZG27wi|RU4t!j5*>`|i=mQx55_T4jE80kh!{Hw4H^3WkYzG5OO>O*4ommBKs{*xE z*;_(uFEsk7iQkd;^V_gz1JC1%$n$sz97r#wOy&{>WxoXlpbuqk=tCJ53=~9pxt*bICYzr+Lh-XXcR-nOa4H1mK@ z^eGF0r+MHM9XOef?Sr$#lwWiz@QV&+iRaX$SX#TpDpA50fzirMiRPuo-lScZgmWY% zIRT5~YeS@f{c@FXUV_ued|Cmy;^EWMa2mlj02j4D!sPWID*ZxBUv;@~)_!URk?`cyEaIj9hqXfpx z^biHE<)Nm}z_B=CJG8M0;G{I#$@dvod<)1&D=ALw7&sMz*!WJEZ_*AYDriexP>67l z8HFn@8zh=4NUtzH&(h-lcut49x4|qyYE)*>H-Hy(E70!GLU`!w5SAmZ6lkUm>mF@d zGFl%(nec)61e~Cf_R_}KUO2|Ml7negGu72-XE!iR(HS1(RNr%O+Q^;(@F~E5mpy(i zC`He+|BhpUaV504yVEmE4|V2USkGV{w65$q2n@i@87e#7H<8o$coUGgCj|TeYp2II z7~HXaz%e@A`JW2LD(q?eaDXI!haEGHgSI~|hx8jH-lx^k2&n!fy)EEmAq1-oUNC^c z1{WmAj{Uk~vT#koMg@Cd%3v15^W&5lZLva4Qbc~$p^|K19jsl@ER`WIsj;PSilKzm z$0{*~*6Jn|!AV6gekAl3oYM8MMbwADc|9!Hw0=C_8vxhrP;WViG)U3|-L^919T;(Z z)THfVvQu^%Bw2wxEktO7yPzOLOK%02uqLPgFuLAONdYbK@PBJ`HA^dAHAoivf|{rZ z&g*FAfklFapia-ZT=9b-DVXrAsfl0zG3UjA{3uDWUxE3sB74d~GajNcfQi0mHE2-W zpN&9`B<-Tjw@GFqP&qf1X&3CF)VuHJQzfcEH|9EU1|$YiLb0PWxZ+L%#XCb|M;W=| z)PP*Re0rT6Bm((94url_-Zvjq^=3e#b^@iL? zkI!{2Wc>o5dM{^xk)`-wB$WSGd657Q5kGgJ zAp+De2>+XOZr;J@js}Hd>>V5o4fF60cXx9HAHjAG@ec48adQuJ_jhymclCA;76ISV zrWvjRN5SyIKZYaae4ym4;g|ztd@h13?fZL=a!d;F*Oo z6$Pg~t;uRICBe%ev=lP}OC`3Rqd+Ad$F$YJ!1SbIkdy}dBJ7r6%>fEZYP% zDv{Ktt#&QIMTATZj1|YjNn2A3EWC9Sdtz%Dair{Yq=+upx*mwR63&CnwPNd-!BJ}% zGuUIJCO$+}GV=>46YT<$HlFZ>7$|VuDUi%4X^WVqK~g+zw18C}SE3r|(YneZV)MD; zUri16sqqiy9P!zp^1z=E_~YY>9|7sfDNu~I=GqP~?e1IunFJ?kH$-UHh0sW@J)3C5 z3L6>U4_1qR@^uFJ_8vs@1GJEhwlMMl5=M!CDwYajM8NrXim{4%{X&`lhT z?O@g#dB_l|41U36DFYg05)=rWDVV{QPb>Bl8&}*6kO4RnLd95<>DC7) z1v_vNh5I`FMO5NFxIWWLnlFY70}V*D9IPF>>nA>V{X}>9&4aO_%eIkW_VObe>_v%dF6zOO1|7!%5TXt!#y3W zTx0Ay<|?DkCakv@j_TYRX53!o#4vGiF+*w9p^w*@IUZQfw#d;x#MR6^7@MiZU^ZJ^ zuzA)ahU>A~7rJ?(Zm-uYlZVo=BJ>a&uT`pjKY{;LC``t(~(h zSZp;{{P2wPDpqd#bo2@1x?Gzz+D)#k9Gl9@JZ^0cp8LFFuAD_`*2Z^-cgH*tu}^R7 zl{8jL>yV=977lKs{oGGi=SH##RY^-D( z$hmdZwN74C?XdGq!_WL1DFS>i7zOyt8C>~F7^>y_7?V9;G99Zd5^lI3$R?ZkPPRiJ zK{Te8{UN1APKee1X&__6lZM&HssaUM)80ut3EYdQ3^~%VBxh-M(Tz`s558N*qO;56 z(Rr4-S^b?08S7RW%?M8t@OrHCE2pHZkT1>uLcK{>*CD+}D{TGqUrBy8ePAlA@!O`x zqyLfd*z(dhg*P@7nzp4z4EN-mJ2Vzj;u-hElkY0!AzzODM-BO7F`oQ6-XAp-?lWjO z$Fyc>xQjChdB$*NPF9)C7*%GP|7?49E@$9whF0$h!Fx)Tv)Rq8hV&9@V~(&l*bTL? zH&Je}o4E|V<@aE4aE{T>%I9mPT>f~+`A#dtWjW1TbuIik?H>c+j}6PY8)K^AC*lCZ zXTOOQ@RRKs6PdNs1^n>x0SP;Hvw)$UaBni9$4vWH-4B>qiDY($AgeXXEb>=qZ{6+IiZYa2KaSb*BxrPE;iE#yG}~XD6^05wwi{k< z6=VAKBz>j-H<_>H)@FPYLQ+Ovy@g$#q8eF6LD3~&i_=c_|FW-o%Ja)!>WK9xjSgd% zd#~>U*-c`8Fgm4t->Oyd@yTL*sBu z{ZjAQ*Kz`{T`ah+?Rdc6^{@nDkr{8( zoLmm0hjFXr7xK2=579n%^N`02scRXJO=kN!t4Lj2Wgw*@b4^|UVehGi11=WNe#C`c zQInbZ=4pscH$(U<@$Rg^V(q9|wSuhiBl0%dZ`}n?a{rXdyVjs%bX|KqLm>Y4(MiGa z=VHUM8;TpWIEQTG7R_w;ynNZJn;Y1C+Gu^;?We^Ki-)Zbn`q`YaCJ(#j@+KTkkh=* zKlh>XlGbIz%uOqE@3^*#4vQTua_nYnV#pPiTR1mzmV-H)+><$rndMAp{^oPAU`u(t zHFu{=qyyWP$1@i*%iNi(J^V~6?rT?S!_$rp@!WOC=V&VhZ|vp`EYO~ubwhG^eW}%A z-L<^&+&?Rp$G!Y?n0xoh$J#PlPhCXrYujkECG6H_i!sVQI;U~w;zj#rFBUOm`^~yr zC3jNSa`~`il}c_4cgxIM2eJ|x0x~k=CT6EoyE&NswD0INFB^_(uiKa#Z2w~A@VY!b z?N3{I=lqs^vazcymnm>j%wCxv6QBp*&(6>YX=Dcs$tX;5A$Os)(0V zjiUA%-4j`r3siX`A0INEE49R1WuY#MsLR+)36X+A{r;FA+I)nP&H+Tn{LC+ZTkAvj)qF#qRt#(SR8f6A?cau zX+u-7!cBV@i&$1`Pbo4~w5f0q|88MEb`gl3Cdzq%e$=Dfe< zzMpZ!eZN(UxhY02N$gwgT7qT*^y0qXX5QgT>tB4gQVg~~TKJKz%&n$9vx@S{&e18R z@8Ws(L$^X*cdtBGI^U-0)uu2;4=-U#;ez%huMZVw1>UEu8Oo>%t<6kiVi~nKUc%h6 zv>#9^>L-@Y)kgk&f9(k&cU^ zy!J4^pDdMrov)suNcU!d&H4!SQWu-OMPHTgYRAMso%uqZWxGkyfYUsYlk*C#j0TnO z+SZAI-BoNnaW9rwa+lA}-QB)NVh zDV}r7lT~@T7lnH5gEQFToO%y&GhPs8>-V@C8pCB%#cO=1 zq$u$8()ivZ2}}|tSw-HBiuG1qCwljL#fKH1N^8BckJG*K%D#=>4<%KMR5v!-vIrG) zQ{!I5m@BJ~Fxy7tH6|SAG(FW=Ewnv?vR}Vph2qNmdq)C7)SBdT6UQ>Ioo-on=S0Za zg(r>+^&6;0>U{GuZEgJc-XHu*T!FvdV+*sdomM!;$#(r~_>%E%W1~Is{`C)2nCd=U zb=5AmaXm15M@mmz!_i}r@wOQzv+KZ&qIk5YqG5#*Z+zfknVIh@9TwQeHPom(^%^h&mjDr)n(R%0qUwA{mjdxCGr z1GdJ(Z;dxazN|6GFxI*qrn6$N*P3R=^*3`D=<-WvXr3AFcgt(J-;ffd@jcmaYcRF; z`Sz`9pC^MY@}BdVM(68#%#~@ZIub6yx;QZ9+tu)V)raAdCb=y3ZWU^rH)~|YIE!lz ztvr3c+2weM^RH9|6X(UMrn^q%{7AV~RC;LbWKM4t^QQHDqrMzVcGoJ~Tr>KFG8}uF-XJ`q&$01FZyyVT?K&rwi$XgEW(PbK&9sty#ULLmC1I}2~uq#u8Pz7hF3PlgXP}YkS5vz{*Dd`uqJAF~0J_^L~7N&1Pi!^_ zQU8>@^n{GD*-4qB?lsrkR^N+wv{Y3xNWy%ESYlzSDDS{R_t1S=?7@%2!%YLL0mZHt`!w!R=-Vd-qHz*&M~qw{|M@tb$gCx<#bU9tLn-)8^*cd4tM?pcxg_U&Uk z;4gLWo98Mwf2r7B8d@LnN;%l3eNeFcLZ`p$?Sm`gc7C~%otu>$Cd`_D`&ON@$;IW5 zy4M%{I`E>+^Tj~><&R^lTc0Z5uCtXmKOVl)@ASmE@POh?_Fhjt<_LJMV(q=q=M$20 z_@rvbO^%UH76-#CPyN`NbRMfJq#oVWcT>yJVWs-v+gmn^iIykNb+mVsZ;`Akd=TNz z^x-T^jg+v6O#i);b;A{p7(R2^{d*~PPF(=6#rIc8>38S=JF?A=2g1(`0NHr8DePKmIM2)V*p zpcr^#pUFh?Eb-b%3zoMFEYb>Z9WC*QYBS0cWl|J7k)*0=t9PR4cH`A+xi#4uFFeIB z58K=vUe0r%By#M7TK%vePm7(w?8`IDc3Cgwk*@oFcdv-N%tUjRybTj?OHEo z$A(R}&dgr zYwpR@_g?8-ZhzOZ)^rc6hR&Yl8fWiSwlk@7d3?X3Br5UUGc9iem32v3ecoz$x!RSg z*AG9TW=el_k5_%ZtaILj7l#k_^K35=c-#@psjw|_)(L;z{CvxTkVwN&v;11)u+NJ= zi{H9^`f<2%npuBH{#O0Fmx```FDFNfvzG>pKhJDOsZwOwuJS|tWXPNJq(6aRFKi$c9nQJ4_A<3Q&9^4qm0b-@Q8tuRwId3zy9 zqQP(*{}7MbD&O(>aoP1>oFfXD6eH}~^3R>;Z@pbxZ&T>oWqE0>-o=c8F8Jfn zDm9MI))d1N8)TM!3Ay+rC}Gxsf_&3}$aG5N0R(frDO}4?&aT- zqpIHCSSo1eSs`oU7_{ZcH|>Jk@2!u%ap1KL&HtvQ@AUrCWoo7KXh?^tFZgZ&UYd5_B@}H zLgI}Cwz-umxV9ep-fCNaG{QGyWP*BGJ&WbLN&T1p8CLFX!3T9eS3L95-jl16{LJ|< z%Nmm=r-X_xB{yZtub3FE_kBajdvKyR>3p=ugCjMMf7&b;4Gz?uECzEp`{A9>9)IyH zq^j^|^^}~kvkP>aTqMz4AebTX{q};TZMPL(t2iyZliAyJT6y1|7gS^Q$;TxXBOGdm z`pg+yjmO5U^7L9-CvO|(Z&!S^``o%$mn1R`U#WC9W_3%a`^hir&rz_}w)(o<(PW}# z{Y30hwS8>6ns)jgo^#=g{GxfG9!y2RHDtgeyM-b%TUneQURw)(4#OXo-&>xP2B+nQ z4d(dFcYn?M?4aLwpUw&GHym68`SK?Dja9`yoo)HH4o?}qZ<;VaD~b7N`NI9$#fZdt z+xY$3YG-}B%KPodW|Nh*@yr_q*1w{(v=#U3N>^R=dggwmChdWF%_;ASm9w`+fB)$w zA-2v@;?`vje@DmbXXo1aomn=V;_KrtT$Y#K8{}@oS<9AnVbeDL9(L!Tv-_?xh^7|Q z{8-i+@aaa=rzh#Dau$U=de2nYTbz?wRvqy>m!WmAY}ndO!~eyez@2IJ)_%J*6o>nR zzNUP?d+Nw4@l#r_-q&3`D%i8+;>*p~B_|bIA{=$zj-RyK-mo(I41;I5udvRK&V-4C zxWS$MT|X3J)b-!q(`d@IF{#S#on-RAb0P!YVy1=5%u`!@9Ddx-@>NiiCU@VJio}}x zhC;jCQ<$R`C*C{j#C_B-GfqCNH@q6QA;PY%QN5{6>a)mR zld)ve(!B@yy&@Q9Fwe04?YJXnhnKu+p031vxgz=elf$aZg6&V}9MH55Ef5%}X0hI5 zar;GZ3`2d_h7C5Yh7bid2ft=d)$rryUn~_YEL1$kaxcNO`Y~hcM8?**tk&UxuP{8H$-QC z%Kj3OcJ*=gIo7^SQZ_CZ-OjJ{yXY{y^W}5SL0@K}u9EL&g04dRU&lH0&nC_OCUQsr z#2)?#4jqN^^E>W~T@2cOd_2-EJDTT7G{>WdT#Jcw4^5=5uF!{ajWjYNT|yJX^57Y3|UewPM^}&DL2B+~U+N zcMqjIt)H>J$n0=-)R(?;1=asU+M9sY^nC&UHBB~W4cAW0 z6EzL1cM-p^mvcco%V$zmPPMQ8+Rxex*2*>)E)o_jnWy$J@7!IHSFdfBR$iOsI70v9 z5lyk0&Kt!B;xp8g4U`20gZHfqoz?VFpjaTBzr~Y()zrOV!`xIhDsH*4>e^Ekv98Vv zO)>t~)vBVQ2PICf-0F!)S-^XF1vHkPx zqWB?`+B^HbH@f6A@%o{MQo2w4Vup0Li4FX4=;_Rs$t~W~j@h463N1R2T{1!9hg|27 z*%zLMMp!SunBO5V@S%~)?VnR(jH@h4Pl!&Kni5)VEPu;*_KvB~3)(%7d}$eS-L&8H zbKbtj!>+}a*Tt47Z)+d-+xYOKkNzK>Hw$=qq{e-kz2N&0W7nqK*x-YM-m4xB*}b{u z!x@A1>9UgES>1+xXK@Ta25t8K`Eugoje~wF&yC31bf4dJS+(o%moCSYF9)gDez~H$ zY^!d!gavL9>jd69Zbym4$AdfAHc? z=%p8Rf=R)uQ&&}V6sguP+`4H*lf;zOTcR@8Md(hkHp+bR=lM>RxB8osZ#|N^TJY0o z@}1(X#c$5;*;MJ%z5Teym1h%{Y)#ND{;B@-rt{_SvZZOK%!zO;XkzngG<#Fa&r~TEFd=8Imk2rRqy6SP1!>N1eo2=Guken;! zF(f_wlID7&3ehK}kEVyV=MNSff1qL1uy<~bhEY47oV!(i|E93}{kRcr^ZIR!3)x+^ zTDkpDc*QW)6&JgFViU7>PYI7KOPf1fYv}ypt7Z+}u|af|OYrl`n2Gn+C!MT$dEt3Q z|McY=5mtt;WZiXA5^58}=B>Ti$HL40%sh=X%>&(}PPA{Xom}2W%1nOGR+ak}TJF=g z)ftu++}gJ2#4dFu3sb>UuXiW7s#m!w@7|{{W4x4~-SX2LcZ^aUdLdNRdEbXVdWDLU z)Sr8{D|dUjb)S!rGCIEQQS_`WhqUW8ca}d+J2`faTX^HGMY^-5hV5HE)2k(W_IztE z>*@O((|$i0>X-dzk^buJ*xVLV)9h1YC8aOkIaAuXG2!gz#aj1}?+zN@(7b){KeGKqowS!ol7XyXqUHeA3+{HZYW73MHN`KzX7$8) zknYnFZjJj+KG5~Q`g)zecbu5IQBli=*TPAHD!cz1SHAzxxDxGZ6D;i_yM5fNWhJ-2 zdus~zzdttj^J%?+z}?4PWTxiJXsdYk_gZ^yuE>t%-bL!c@o}$Hjx1C>7kjf(TiGtk zbDor!*ZCdiWOoc2eY0}Voi@=OnvZ@=DbBGzkv{QDV&tFiF@GY%T;fKYu0GWqw_T^T zVC%y9A~t>Wj6P^id8+s8X#HWG(IbC&`d_$qEpL)`vajFUKMtqgYK{B#y#7OB$KtDHGoJz1-T6)$fvIW`EaPd-#$_d(iq<{%h45Qr0H$`yQENw&eJx?RRsY zZ5KakU^%VbaYg6@r@S$v`VDGGsNpQRmlWZ@q+rbiqve{S9O=*z6>E-FX$)1#s{MR< zM85oG*J*NfYEmPslxLrNZd+a3Ic@&Lt*V=iJC6zby^za1^KI?8r|YYh&s%@!;8n@9 zN0YplZFr`9#Asu6BHz0X+4A@6wSB$_>JPCgHchIFSn*{+SMBfdMH})jS)0v~R38xE z&^d0@(%I9j9&&O^?_7K;)F2a;e&2e>hd=#?)y~|%ytCi_k>ARaHcwt6AD+NBu+M~D zAD1P~8tgcj&s*U1>x=WmO!?(5AAV+{I%WOmv9+;TJGB-ZP%N6UYli)9*{jK#QGGSq zyfQ-7J<_kwUiGTs+|}jQ>q_sG+_FE?G}?5;TZO3nX>-TiwTrm-;259Bm&2nk#vlE( z{K4!4`Xd~NDZG0r)TDjv?vMAGDciKq+r3#jJmSds+xC<0ukA<=*!4%w^w-YIi@bFO z+n@Pb$K*I(x0xy(|KY4?>#k(yJJs&9EiKfp)}$pAcRFu&a2#)XPJHi#$i~-!k0%?P zx3er&N?n++t4-_V=*wD5Y99ZVZWVV{JDs8Tv)k(Sy5&-}MWfdlKM?xj?WkV3WT5R+ zxz`^0K3n#hyS}tFkK(>@Rn8JWSK!v>>fA{d^)f! zWA%<9)m2MU9#`&~rD^-za{Vz4aoZ!I8!897bZi`C9sVS$-*=URGO7nJySq5$>!`JT z**fW@Lhx-nukjlk^bgFKzq`-Y_^{SK@yLuWqI={WK{al1mauKv|4QU$82dv@zs`Yx#M7^=SD%sN=2(U5!hvNJ_)zua3mzTvK)&x(SZ z!q;!iRxUsBc^;{b-(x1r)#-Kb_}~AvgToe zTdkkB`tzmt7EQV8QDmW1nq;g!e(7Rcg~kB+`LCMNjc1w+oSVNnEyrMYufMUR{IiS-+ce9J~*~+)#%?cOA-pDDFT>s7UTu_q&A7x`hp$}$I@>xwk6gHFqthw1sh7W= zT>DWw&D^imy0F;L$ad5Bw_gvu2v`<1^VYFrUbdRrBbK)hy|=|P?o{4$-G^JP9?ofB ztm(OV;1z4*-!JMW4mLWKppdO^uL3+`Grt zdIP>hSq6LVX&=7g*tKhibaf_Zh!sb44^w#1_i$F_b;Uc)(=+c>TKyUIX=J{P|MA((-PqDSeqOOaWl9TT-AHW{q8 z;>%6D*^(cSd*@BtS+n)YdlWLB`3(K_>U4x^=hus`5)Yo+6sWrHMv}Xgj=xaX?r9%J z=*|i8>v%BdY_z84d}+%onauO%zo%p7~@_MGp}k@Sztea(N>aml$O zj_T?Ml^afYH(>JJ1HSfZN}c6LX76vfBGaw?K-p;L!qfVn@06DAJ@jPEl;STlyXL8% zEwMSeY?*cALgP7MuG7;c=a#=|-?MRp=*EWWzxDMWx*A@|a~+mb^q?MYkh<6W4ND(Jjd78=QmT=6?8t{VXScl%pIq)S z(@)Y@)Gp}wW6g6V2|r3_lpIq%neX5^|K-irqbq&)jy)dIcgodspIc+q%jI&!4%!y@ z-Tq`{TtDr_SDkGK8gKUrn^e{kFxhQVUFqpnAI9X1hicp1YxVIHe4$%5z%9MK-+&D_ zW{jWi|2p0<(JIr<{Xwxy@pXHHGcEGgAAeq3a>RLedG2hV5@(mzwaaA#Iu)D7mtUEG zgnycX*l~Av=e8eiu>rGu6E?P;%<|v6?{Rvh%d&=vlcr|H3x514KP7oj+an42oP~>x z_BmyLeYgGWg#pt-jJ~`-rm=O-rRqWL>LE9t%wFgEwDHEx>}#pB?p_V-^wzWK=d(iE zx32r<&9yF%OAE)W^V51YzGZ}yos_3#M9qCyf31AE=B}*4Upe}dw^uAmn~?AE=KUVq zT>);^IOFm_HOpo5z{XlEPgxU@ProQ&20a$jdz+?6dzuc>^e?1i&OF5 z_(1-RgHFd%g=8Om&XtXjPVqI|I_cErZHI43%>40d#8uI>hcN?Y|5`eCxU+WC`74@( ze4bx=y`-*eOzQq8HAe?u)-9fK`SNoeS;5zei%zW>BB!MoGpuWWl%2Th25+}bmMbH& z_+QG6ci%J4zW9@ieaxCZhXh?$3P|V9h)dZS_rvSP-Now@V}{hs|7P%fk&aOKy%%8{ zYt3yZnA(>=4VK@M_3g^f{>nFKvlr}INX8E3X@& z`o-?s)VhizyJeH^wJ7Xt@^o2aX>yojI*1cd7p0z^lT*Fc;d1o3+l}|?yOy35*jDxC z!O+yq@S&1Rb++uE=Hb73=|Z6Y-RKvHi8H+d%!~cfW0)+=&x1Xv|$b zG)ZT6wY=T2`eDskTQUpv`+81R&%LowFHhs6*P8N?xq*gp7o9FWU1fPHKJ|gcBRzqN z{E?Ol8|;6FMhrC!?db2I{Z@NucGA`ALq>0d`sPmZ)t@p?Q6sop^Q$)Ja^F?WpMLqA z{GB(}Y+YHCm)?+<@e!A=6*;W1Y9Icr;?)ju$)=X2=Y5`cU8s1SESGe>ZtL)O=J!U} z360pXOCUs{|B}oDx5u|^v|Khzvi?o8GPjEN@8E?OFo_L?t1-Y zR2-&r#^%eB&Pd3Rd1m8e+sGpHZ6A~R^`g_Ie_2jz>7pS?Tc z@`IomF`L~({7gQzZL)qPd35{aV$(?vt~=^X%T2!C{mn?V|Bb2H_nZ4fWSH+!>bG;! zMw95Mgwn@`k>M88W?TQMPm?SCy)wL&GtGIu_E9Bi}O^%Ge~nnkp1+V z3aYau4o151t-rP4!@+A&fzN#H8aY24qY70*#QmFZYN?m1?&!)c-`cv{sHjr@&~@qj z+TT+O4$GaFU*;0{Qc&chL4Sk31D+_JjXE(;U2?yc)|`SvYkXE5NKE?Lv}*SDn{|FG zZi>2++JC^dcDBxvV^hXLS@!tlhswSit~k5No>~E*L<-3Q&-*FTJce`Gm?K; z#hutHF5OsOlcE1saASPo#OBqPBhDn|IZV4MJXrTxXLeDBPpbK2k+)r4OeJty78mzH z)nc@P$G$TexvDyY7BsJM3TQNw@4TbCZAyrV_U*9B=Lc!wmzR^E_F1~D7ld)AHI<|KIltG$BvpnNlE5kY{ys75MfSTfqQNhnyeh=|-07=w#QJrN-|>G!&}>FP|UwY~IeJb4_iGE_P%!o2K1%IWs+? z%l4(g>fI?n`11Lrhqs4CuaZg@vKCfLU16bqd3f=Nt#vtOi_7+HpZ8U|P*8LA{?)B+ z+ssQ=S>&kAE-s6A&Y10Xe{S$fUA++D?CIxIL_Vf;ZhxlS(J)4#Z&~ykW1+B&=O-sl z)N%hl;O+k445gu&Et-|a53KVqtKZ4H@U^7h+I=Ce>yM6TY@ht*YW1?vX$p1MY$AO4 zz6(m9EZRF^*MfVh#V77(KR9l<=bPs|<6j=v)k6n<-I4YDwj|#m$MefghezG(^Tk)Q z)-tL|>A2SIPtBGmm*gJ$qq;iYz6?-z;eyz>}wMR_xgv49Sn@f zD_-Z2c>BXj+h^NTcc%#nuNpY?efZ7e=Z2SjxAIfJ8!J$IXxvaEw` z-Wj96ar@2_t-gaFyj@+nJaE47WhurJ~2%B<>nFBLB$^nJHoZH7@3A8w*vEnhphYL``;gusn11S#s{BotoBWC&T1_ z%dN7sQS6iNdN!+eRqM~@RE>tHKi+zI_VLv(f6iZ`x+;C?-C=KnTO)NZ94`4dX=p~u z&~5uZzN>GX*|aHa)R@wWe+CzMeE=eUaju z9GN47@73}>Jy5vwjOMa$zUd>|o?lz!n(bybBB^ft zxOt~GKapCoM8N6Z?orVyXGZ-TqCEfEsQzOPyA%vrdA}oMv7tc6370I-W#_N256#@6 zI%OR`_n)_U&RsZ#&%0{0 z#iBjZDYpz%Zq2o;+Y;~6=skJMkVk-WS$xWWZ@si04nf@8~hHedxS~UI9pv#c!X#rGDP%3q_~yuasV$ zJXdaqROd>=L9e#Wv+S&VJutInFTaoFLSOmvZJ%{FZ3-HEzu)*(61(>uJns@SQ8uzQ zwOFUC&opDHF%9xfagNtc*bfVryb-^!;;zM}^dA?lZMZrhT0r!wWOS}@#*^POCkEs^ zS(PocK(^fMSbmj4n&Cy^VGi@0JQrn$RVgKzhr6?R|oewU^IeO5zE7Hc! zwm7T!t-#}fMFYonH{Y%K`a$Wp#pBc(+xPJ&4xesJPI%X@^GRVG+ZZK;!RKigj@-fo!Cm(YMoC4AYPKQ*?c z@BOx7^Wj&=lG>~io_)-oP?>c4PNV(1PdR(U7oJvqoP4Lg=&*Wev#;OuOTI0OPd#>@ ztSN6f_{@!OF-_BatPL{OrRWW9XfZCA8M^Xj`>VFNZ=R=HF6ze5_Ierj<<7z(zQ5vz z*m>-K74rFLdf9OQtFd|`OU9jS9BSX9AGy#v^~05qeK*+2ue*E5xMR$qw1SMM{fGaW z7O!z$^yu;ZUBwcnT5i|p1#LaMPs(?j3P+@4*5iX`s>Y3LPOaHCzh>m=CF4iBm5)h( z@>OI0dAae621~1-ReM`}Smd=}u|ms!L)R&hTdoIcPR!jOn%A;NXvg4{vCVd#`Dt>) zy!JUBe0_5Hp{sA-zjd|vJ^Aad)WFS=7xV{@l&*K0czwt2E+sy9kqYJAYidJ>{%%+L zz0drjlAX&BSH1R#yDfiAq`o_atp543IIGskRV_05#h4KjOl3NCPm0V988PyC%WRvL z){9}9MpIu7dbZzCZ&uS5_mxp+E_fvUcqlR^zJJxbj`l+;kt+%#k3V^qGr#_nm2U8{ zIp6n=j$Sg-G*x-fxQ3|_FE>w}ZFuXH|LQW#(;L z6|J?%s6j*c@j@rRvROJ8l{#x~Pur^BbgtWQ$CQH4duL583pP`IXRLPi!O-8f{@PEf z+9Sg3q`Soen%{CCdg;0DUco#SP6E9SkIgTianrs=TjY^gv}c9gB>82#ttAhPw`=wq zFk_Qz$A;7QCiw?4M9Gf`-GxFY6TdiHR z_o?KrrmEYwe80CQYqZ=g;y;wwRXAu{1X~ zlviOMC-m{uU>>{k_m%he_LkQfuc@P{t)Z(5dyK!y&kE$34|%N7(%ywxH-@j8Fj@|L za7@;OA%VV=hcA8xhc8x~ZK5Pf_vYK8#qHsQ6UTe%x~bgfk(9d2)<-@um2>K4zk_nfzv= z@)-hVqAF_zO+~d+`kIOwBnwRy4Ug_GYG5v^Z7Qn52rz<71STz0CN1)~%@LxA32(_i z=G59gwxY5r{8L5clLe-VYWFGaFDgHc8fZ0uW5S5F5m_d1*A8dmWRmnVNcw99m}F7} znTW}Kr;0lEiRv$^Jylf2l$EgA0B#=C!HJJZ-Tqe|7#)xYOVRK?u1qGFT%k-%9l)_= zqH`a3XSQLz>0h-+`DZ!1>huue1Ez+WWs}`2T0+JDv=G&q_ys$Kn5b zHOrop{5@|%=t0~ae(tJke=}yC0;cVd)!2A}TjsFO-?LIKJz3`2_@QPmeIfhHH`T_1(!Pxnh%i+s!=9oJkFx$RNfA=pxkOea&{?QMN z=|z5gFWkNt9?%Pq>V=bcI{nMeeZBC;Uic4&ORzL?$05cK(#&}DFa2u0@F~4;vMSKO z^epOyuk3|y?S+#y*#D*fG{YrW{VI1nWb*Z_7x_=U@Bz#?`Y-*8y>O#m_}pH2STB5C zFMNA1yprJ(tnq?7UNiZ9*o!>p@5T7MIo<)j-a)XKvWKU?XJBY#lm}VcLX#}|5bPNo z=o!h;v^8hecD6OwCI7~ee|qHK1oDqrblKK?BKfDI&Hc}nWENkxH6PDC)`c|wUP=Oz z(3W2UK@t%V22+dC>OEI865IFPZ%FEE1n9~k8u@;49ljRhRw z85zJB?G^0HEL9N}6dnF|Ha!AEJt90q!aaP#VnR7A;mE*vUru;LU|2+8RGi16Xy0gG z4=(%5D*>L1eLZ|b!=n8IJeWl;!lEO*nWZkcj_f(~@D2-);{@_-;2Ip{+jC3|C5I@A zk8iMVlrJZ2v2TQTSV%}<6thrBPXZoMVNssJ91oAkC{LzL6F>rTVWkBvbfAxUX-J zhbPnm>SeA6|Ev8Wp0TW?Lwt!rg#BHF#Otg$fuTL^nOQz3(A$@#f{Kw+VrnZYKgj!E zYt!`T;;#43bb;J38n&v;mh3ajW^uzL7FT3(dYww-$?TTO^X9c24&h`ybIN)1Q65)e zaero?1R_6{`B3>70+~R#8uOu?%>PIrT%Gw)p3KZwNFZE;`B1)x#Wh*{eJ|XV8ShCT zdf4(SSX_%aNA=Gk!b~9YY`^(&DUY-LmdxU8zmfPPkS#mE=3L6-?ELcPeuTx@<$jID z+2#9`#o6`YCyVQ_^bg@@4wyjfWY-647XPb0usA!v^qoCK&nV_Y^OeSuH)QdXEY7Y! zmsp%!qf$L2E(t_GxwA>R7lBM5d@}Q)d;@_@Abbk*q5K+wOdxzT^P#+zKqe5*_FI2e zIhru%sJu3dk7IFD7B^*avSv03M86sHp?a3E%Eg?;b6K2SF1J~nT`nJ4oL&DFSpJ;K z(r*TwM3Z6hAeKD4TZ0+~R# z74xCIkl81J@afEl@(TnqfpB)Yw{R(sv)iv1%m3_p8_(kGdh&tA+4A%qJtSQ=pUm<< z8H1>u4lHiN;&WNtmc{*9oGpJ2IN6Z9IaL2mmb@K{cd+D#us9#9eC=7hKdU|rW$}?L zdA5E7mOQ!JMeQ+X$+Pv_vgFDAX)5o@l4t8*#*!b-l4tiPZ2j~dqa@p8&33B4q!;~l zEd3){@{KHcwtjNGN&?pZjU~^vpWad;^@QDy$T11H-txY!;qg(-d20V`t~8Ic>t`Tv zVvjuYq4Ej9iGD{GFJ*DIJ^g_9XSQOnPc{Q#pKx-wbRP)Pet>W(kRcn=hmb%yy;di6 zi3GyQJ>2i)fCHcWO-W^Ef*EhqBR_#m z5}82Clmse28w{Mo!#z9_GUti-2GA3N_kqH}ln;v_L zMR=e9@gpE#hWK_;F_>_ix9QO{icC6~P{Z5w@TJ7rOt{P2^l;iwH6v~h_5MBLnqX%q z;vYyS!GylhuTuYbft}?3CJB_AjOHE7Bc2I*G!Qq0d>JAx2^G^4aq$5>{Vs@G2=cf$ z;xC|{EJA!Gq??HNDri9TT?{l|;*hUABwq=QpcHXOs8`1kcYu0Ti#X{cNVtvoQ8KAv z!gIu*gFSS-qjp*<@Z`IY{3WRW1EAcfya~vW#f?dzTovjGxj)P1bHH!p{w$ksCY?PK z%n?r%<#8v(3qalj@nYa1h?fITKztz7lQhKjdC77%BTnCKT!?rE^h@OaFA3BSP0+d0 zcg#_K2+I8;l6Ql07X<%N`880EbiAhg)+k8{_}>L_ zS0SE$FT@=Mc%0tHrgr{-_PYehr$T$A<2#jq4gF*;l0ORVYbWA#zFUE~F6gO3{3+y@ z&If3^sUZIn$v@&{i$m^HlR)JUkLDf!L3||)bmaau2~>U=^fUAw$CPgdc_k#T3genC z;%C7R^c}=hPa@PGTO|Jg`e!%9Llk-T`5`U^?IjZN5o8j|gyo1k(|!-}cYS&CTM?HB zzm*_f0`?z4d<{okgFW97msRA&7ld@Ep3Cw) zJ{oav@Q()KRb;Zs1arjmz;DwLH-L8MfVd6lcR^eV%6Bf}gQ1;zB7PDkY5|Co7Xy$G zg}4UvSG4|8Kj=X_N=NciVE<;s!=RtsjkrGa4~Gyx4*la<#Fu~{t|2Z5_BSD}4dvU4 zxE$o`3*vu3KmDqf<~IWTJP_(NP&sOEVCs{k8|!3ad zBmNWo6M;B+2Pg@#i0=eF%MgDJ{lH4Z={qCWAif>^zaH`Ha9oObE!5`&h(7}V97TK- z)RR+)7eISFhj!H+SBABO9}rHIpc zQZnN7`eY5_3UIswar%Aa9>nXR9qmWF3hX(G_!IDdJ>n%WzTZZC9*i$dh&zBEo+6$H zerrX12F(9IBW@1!+it|mq1;7a+$N_hSZ{5p&ak%&(N|0f|H4deAX#1BGy+=h4^*i(jh71a9^h(~}QE+MW8 z^{Nr^(~z$hh|hs`^a=4B&<}JYPLBsd{i5ZV1^yp~comF~s))-$yPJUcc&Psti2nh5 z==~v@?h~kQo=9FC{1b-wbkM&P@wL#t(h-+{`QH}A&w-v2#EqdH(ffkatIXqzbPA1R>r4{Z$;|<>3ER#IM3}4&sSWk9Q(I4D=sFTpj#y7V&!UTRq}; zp5F-BY*u4`-% zkAw1ZL;NSSFMq@XVcd*Hd?55o$%tPB{p%5T2LBfz?g#V0a>Pf#eC8D5Q(&CBg7{cS z_a5RZP#;@XUC|@}QA-iUjG{fiJUgZ?}baXykkCS)M)3;kFg;@vR*mLl#0_4YX8x1ro?5pRTa zZzG-}&ozhh9B~2g!w1A$pnY{A9s<04N7qj%si`1o3tyUb?_@58TnU0npr54Q zCsTh0Kz~SoUqJZ!T)sJrt-ACXd(GW&>zzICY7h(kGmrIIZ!U%h<^op!V%8^ zd+6_OsD2-iUxVeL-`j+^9L!I4AzlFX97MbV#)~tEKLPoxi043iypOme=xITmUe|m^ zoL;|oBYpzPQ3U)!{a^><&Je_%pnWMLehK=Safs9Y%>?ls(0=U@uY~qE2l4lC?2q_2 zD3>V2>F+_45YK>gGZEhe{Z$^~O;Emj5U1Zi9z|RM#+UPmJ3_iQ5YL2m^$77;UN_01 zzrUd6k^<$|f#j*({Lmjz`F3c(;)s8P{0>Jv7wX|y#QOm^K)ew8lWB<0f%-EO@dF@F z?>A68Cqw-SNAg0DPg;+teA6hNgVrMX$uQ5{ig+xP%O1o>$n*3ZL!5pOcmZ)^@WU;{ z>Gy!o5U1Z8zDGQaWP}O75vMOw?GJvU`89_ASr+l@FfUa`yc_&Ue-}XY^o99}8Iu1B z`W+FczyF($I30gO5T6Qujz^sK^Qnl_d0IB&v>uitZUFt>NyNXveC!h9df=Zsh(858 zpCeA&#e2kmLVf#%_r2d z?$1L!2l| zAYKmh;WotS_op3*mqI@*0O`~6dI8rn^gb@-x1b#5k^DF!zywXizk{Ahh!2GMj5XqF za6B7vEs*y^`~~#?QHVQ3Jzs%%IFxS|;uGO|WIN)Zp0f|}JkWCzabvija~ts#D6gl8 zU*Nfo(~dZ(;dCNy3H3@6@<+>s{@zR;aXMaVAWnY|X^1%e-J>Pq^mme;h<8By_eXp) z)bkL;3n9Nzi0^}WW)9+3(7uWgr{CA?N4!5=*PTL~-lw{Zcpc1_==}`pPZ6lMZAe}d z_y@#Yp?rnlx|8aWgYj`7;v&#rNg_^vCp`pl2WVgNh!2HvN*(c$Fi$o>oQ^vdh|}xq z8Hm&CWe>#Z^>Q%c+R$Io`+U?7v|Xnlc{-2DM!Y}tV}*#v!+g6O@k*#yClROj87?7y z7Rvn*;sMal)B6Y1e(LuQBtIC+^9;nU|#VW@jAG!6omV@ zG{1|$9%aOxK>q~94}u?OBi;i3iA1~&>f1)dW950}OP^Pu_RIoaf#k=5{nrrZgZl6Z z@n29beV~7$`n%!!MHX?|el!v91NCGo;`hN1a}hrZeprn7TbTExA)XKZ$p%iE2_1Wh zhmNr1N!JhNb1D&s?u2s@aWkTX33Z4Y@-U|zI89dr_)o-5fe(i9gv!qYu8(*kaAy`L zWzYuPoyAFh%Yg?0r{(n$`XBnc1ClPyD~Ym^B~Sdbm_R0MLOg(nIr%J3n$~oZ5EDvR zoai~hr5sKf;>UrXK)eAs{XIKTOY~d>ex4;y(wz+H*0VT~??*Dogqw)};bG2Q7AJZl zp#OZ%;zSR9?z#nWBhdeb#fhFtFt7W@;zZ9vkpGD|eSY{4ixWLNpuGsoGY3qde%KCr z2Fi1B56=cJ!QwQCTL);AVTY@;P4`&gl&lx^OJP7oBMx2f>k_tS-_?QjJ?-a089&u;j8i;2AH$;3F zaQeI^)&GU;Fu@ballA9FSjOU{ENNKH;>1q+{NXkhCtVdxPdNKnoXC%c^1Z_1M1L&v zA)$%IiM%Dqzh`lxe;^TNLKlk@dHOx|(9zt}9Fk3b!#kmi_z{>#(%(l=JzL>=X9dU;KLg?_+QHI8#)UY@S25yr zJ&_}bUk3R~#Ob;)^f@YO553=V8OaY4;N|NY;`BM72Z)=1{1e3Wfb+q108RHOa1jy>ar*rssXxS? z5FX~#BmHAR|4qb|fj>Ze8t^BGn*jd?oZ7h+?CGP-Gl24Yz!ebx3VZ_M4PgIN#NPmS zLVP*s2}E2O+ItM*lR-}=;(oyM5YGi(g}5G+%R|I1fYa}1s2@Cl)8FGzPM=pF0sSTI z2bRIS-wg2#NY5T|`g;Rk#CL&wIB=3~U&s^LWU=H){q%*C*@$}r-+}ll;Khh91zyJD zq+L8Cm6Qp`S)8PM801eOUIx6H#fcu8-v-2Kd!XNU(frzwM49jv$gVO~y_hJA6Mt?1{St_;1wI%!wV$qIqJ;P%=$8yxdPu!}2ld~A z#YwuaLH~5b>GP=eNPi{hpNn_{@BqYL0AI|~PfXEaJ|rZvII(jO_;C%36Fd3)5or!5 z6Y(yveRxJ zS%^Oc9)@@{)c=i$yMx~f5nlxS0^;=gt_|@sAm0JJKa&m8j>6%3XNWp4UzA6Kor;LN z!hLTo#D7A6Wr%n&gQ_2R|3yLoIcmR z1@S_VFGl<>@V$s%2Y!mhi9bu>27N7y6FUWjdG=pH{5R;ii?|xdKR{dw_*3B2Kl6F& zIqirq1pX88RNz7|F`(rhqQE<0jQBIS&uoP_Eyn=F_d$JI1f19jh?_tbOP=`08uFEm z_*CFqkRC&xnVe$8?SUU*=^^Fv8!Ay1OCG8wH^w8DJh6v<5Bm&pAF$^g;;FztA)W~Q z58@TT`)KkECjKYs?g1{u;>7;E0X(@uEKcO_fxI;0H-V2v{3q}+h|}MDtFt&smtH68 zvp7jtiWtlUL&PP3PXSKKo%neS5oUrd;wD_myzd0@VBkS44wu~AGbzBS-_#)8HHeP_ zz7cUx;F}Pi1-uaPO~7{{o&mfZ@vFcOB7P3|b>PH5Bww`s-s^>bL3+AC&v(SX1LrQX z$>=4Aq`XE${SnY&4wyjsNZ^;*)^~ARYod266hme;SLk=lP|*@O!;*L8$a3IihDa zF_H=Dh==en$A-nBiEz&ZvN*Vm%UAcp5B9?E_rg07R|R{dNr%k@Vn0cj&VSTd9GVUh z<9PMLH}}G?^}_!kPUj(tFy0b-NV=4p_QL6UAXq*H$ww38nQ#bk6%vsNv<(uoM7f0b zkIZ36p#I?;TQXOEj% zEIttS$%gI|`-y(q?&O($5-5L7c9=lNG0HzdC7sGU&6OwiY=>j|`z#_)cn9!M*iufv zuSsTcVt)l3m-oUiBYq!_pY_6jusFMY^@0464Z9pgSe#vsv>zg;+2yFfl4qBr28*-H z(SXI-~gecaawjHOA6#46G*)!w*YL(0TYyYn;yOzj>jT?7mi^%-y=T;(xc^0 z_0#t>=p#MlAa8F<=R5vS`qIv}nBg+{-Zp!U$;Be@~@i6Bq=4=Qg0+#AW; z0uMlZCh!o%=K+sE+y^+F$I^7efiFSwbp7ij#OeGz1@Rovla6>kaN6E!x_f}r>kP{2 z`)sx%J(obf0P*|4cOl*mycBWTnD!%1pMN=wxHL4rO2p~+l~st31Nrla+X26XIGw-J zevbOVALQwE1m)qtY5Sl&2KY13PpUV#fJ;1~zLO1zayfl(5#j9qBmsm7XZI)aAVfI3 zKZ$2?c7IX|2SlFTpV0dygtPmTLO39t-Ji%a$0U$EvGYay6~fs03MRr#Ae@~qT8@OX z^VLp-nLs!@U-UgbgtPOtgc!&K!rA${#^R6_uACeZWCGc+^TpAOj0<7jLda~RB6!;X zx)()W?xyME8Rf~*^onHO>KNf0?5Rnp=D%L3M&FLd68rltdVjrDktOxlYX#Y_D5NiS z`$y6-VG%*h%l+8W|MNmaRuX?3!hYr7|KS!`=x?~?R z@45WXfqxtH?|1Y4|B3(oIy;({cVrYvm=qQF&34=uFfzsH>lN+q;pycS;k)>+)6Cl# z$;%k2k3o)klb}EI(n_|Th*1Ck6$}mx^`*+md-ot6g1PVE(zY@}~hKMA6k~U6%AH)8YK7D_~TZm6L*#9NWiGQUZ>%xm+ z1rbU5WGuz$KWEs#(k~=RnLy_$yd&PdWx7wckC@ND(yxZ}7Z4#Pko3uTfzuaa68~5F zCNOx>bx4Q~5^(;>b;Q5Ymv`mK&n5#B6G-~x`VyzF-b?!OFzC_WLlGS$;PlCS{a@+F z&gRK4Cz)dcNuR{U={rFBbiPkyNyvi1bqK`g(f{I8JinLtr7#G??O+<6>uYjA<3~Vz z-2ST}KK=bIReWSa zzXvK7v0siApV}|Z*vF{&AH@b_U}1taZ`1Q1r%#@P_?Q12Pw=9AgqS4#5zL3ir|B#9 zl0N-?!9|EqH#9yyCfCjXO5X&&Ah^uU9IyS$Gm&U~nm)N+{#X7THFznspoU4~)AaRv zNx$D|o&5KDo^uIkJmC0|5blxSMwz9gCHqCRpvwE)AA$v!D*8ym6qf`hqIYvjtQhWjbc7H zJ{jil+>t>XN4OF6ni$6l*ngy({#ShMbG%4-$T%EdYofM1JLpW%*B(D{+&EnZ>+0#~ z>rUYP*U{G2*VAK|fsVcbH^PGci59lg`tbdyk&k&3>VHT78NJ%~zkL(#t_jl?Msd8O zBO;hf2kx87J(RS^`FI5S`~#9J4CcM(948kG7Yh&PnWi&coH+qukx`-KDk3bB;~C){ z!146)i3ksiU|vh^8xa{87RsUTr{_xXuK5`Cq2A%qk({WoD9>Qt8`3=(_2I$3%=_Kh zFd{lMG%(a3m{(ksZ=?r(<9r13I(v|%@1TzgWGrE>dm@<3klmh~k%OLKI?L46)Wvk> z^t9hDI`5pVlG;_6;nXVq%xcDKKDO!VN~E>frtMRdW%%^8Z{5-)?%M@oBbSq%v^Q26 z z82r!4^wWyOtyXDIcxF#$8W9&j<4t>NmF@_oH9b9Bv5*M&oTG70E3!P=bJohcfeK_R z?)hiLY{kpH@caMZJZ=9aZpr zi*~J+*|oP`XwIimE&*jSyCkw*txB^M+{Wf?kQy|0>zW4zgnp8J!MO}1I|CT6qdymmJ%|;2L72^(_op0ix*Xy>;7B(-vshZwdaVe%# zye53qz{|3`gx&fKNwqbZleKv5Xfw;jWr0I(l|FWTFxWO|qw$x$9$|&cT;p|4a;_Z< zyE(W!29ZcU7m;^IJ$|e@9lzwBXU=i42j9)ZQsw%)7mDq-IV6N*KRNsiEe})e0>`&#Kc=3vF|j8Z3T%{IcV>L1nY1f3one8dbZq)baW3 zH?D;rXB7R27-DCz=F(Mr!CFaa=`(J(k9GaZ>9S5a`Q>{?{U5W4BFbr1wW3n(Tf*`buBUff>35yY9#4Y{*zXsivQaWQ~={xS9Zi0PomI zcBjs4^AsseE)s6IqAeV|!o5Rx5{Y>39E}ws6DM1`&g6+JfOLiUdCz*63Bc zrMOn-iG(a4AmY$RQ}-HYp5JD{*iD}L&%{I`Ib9;}mW#=@@`a9@G_dlj_Q>!8&m{AY zO6HcCxv$jJJtGF046(J@aN5MyX5+mIGux|mzh2B%NY8zwa9DR?#UHuc$(ceWV_nwU zNp0I}dPAlld1lqmbq#jr3gH{KpHSDSmlSotFCebFoCgG-mCxjz}d=3bFGXAn;)apo^@}g<-ep7UYl`m*L zsc}W=!J?-XFYiy>GIOLH-=yXB3m@Be6r7pUQQi8kPBndR%h=6h9uHT+EXV2f<>MF5KeBlAPErkL# zdRjJ7t^vzZ;$xiyMzx)G6zV)Bf23cOQ-F@1)&LFDho{W0>K3Th8QR`h(YY!&d9wH$ zfyXj)_)e-Fk%;oxGvMp)hOGkM{^T?mHrz9NYdZGg5x$uUUngCbdC)InrGD~}1{p`q zi}xh!zZ%SQsEZ$MFx#2obzEFd!LG$eOefittznas;WoC zkNNJRW40AdjGhjo=6#EY+4i3%@_D*{?CV); zFKs@4#UN8q&Zu$1c9U?D{S=qBxvl?Nt9mJTL!xTO=QV>T*?hGY$t+Mhsdi4aPeuRmRi5LQKULX!M_-dD8$60o-tzK5_Y|vL3fw~VvyM}X^ zU03o?(`R_xy|%~l%)ct~Z@b9IzQs>O3i=jz6)G1c7y1>M7wQ+)6fQ0rT_{>K*fb)w zZKwQ+@n3CZ^BxWDY?=9Td&4Eu2#2;=A7^Y#Fgx3SLxNd>z!l>n4}RvK!xiW3*U`57 z9*K17wP_TMlTQ@;=)J;zbIZ?|gN^S$6wcVIW}LRMbJkfU`xqy=-DeYa*-wcs&(q7< zwdJw5fkEzrS%uN()^+r4Iaj}ahQ^2sA9Q~Vm%QuhT)s_P;!Eio*I^xtI(2F$=ij{H zB-s*kBt6Yrv2%^x#~XdVdUf|bn|s~SsO$9l1v0= zrGdAl9yss1IZRYZf1>5^uzm|)yx6l?B7R@(7cUQ0y9wD>FZu2{JhE@q#erMpPD;g< z*ZuJ_`9JKvd0b7=_xOF&AW8#Cq1-4EnuN>^q9`IEqSBlu4U`O_kR(&cm?86!sYnPR z^PG8(%*i~2XRmwK%CXh&^Z7pC=k|^(f_Dwq+^0tmvu|l%ztmv+Q=RcW{G zfUC>$*DZF;J>(i1+PKZcDJQo4j^7`$YtYDB=8K-y>+|B^@qt~sH_bk4-}zP+f zyf(WI2>hJ7W{y+qvS7AE-5yuDsGbbwKT1%6TIT zx~d5l&kS3+?#`WV z<9&QrZ7Esb`uDOj)9`IC22N~Ld-Xf3?F-AtHvHJr{P&*whkX`Yewy|uXY!k7i@tOY z*)pYc#>UqBTeaxcyM>{Od#Q<;U;4AYs3}agBCr-{}&O zc(6-^@%Tq|r#!!rRyv`*d&70ft2~G8t#^5%y?SEy>yGV>TL-T$Fo_XzsH} z!(ptaq|@Ee8FdvGdzDH?9hV=Lt$)@nzV^Wemr|Y%?Q*CfJ7JTY*~Ety526F!>TFUq zx|~sX*7Zff_{5OSCu&b@mMIq!Jz<@?ef^RfM>}5fGoRUVYDR8*#ce}^&xCHxJRRNf znne$V4~q_I3~OJSe9pn*(&!fMl75PIM!zP`o@#z$^mL=w-)9A!o_4B4>9pbH%K>Xt zXRWYQUvN(A)8Z_J9l^c|;}%Go93Jv(Ubv;hwJopw5|T3K6(nt0+dWtE*z-WSad~CV zchfP0`Zqeg{bRFZZ?a>5DDKMceembRvaY>64p)rqtU5uZXtZbM;4_T4 z?bT$A-~a?wt;2pPU0v^dOVoaDjPGpzq)YPrgCTad8Z*jPXBKU5JN>-e z*+rWT9rTYLv+uTfbcET>?+S~9i}uSM?R)8hN~vDaNt4az+pLdL+xmDxgS!Faetmnk zd(U|182h=8qZ9ItGv);cw~V`wbvM%+1f9h`p~1cW&P;h?|t;^ zG~Z@cS^CLkVtDE+zh$A`Z#v#I`1s(_*`((AGd7gu+Vsnt8m|9B+sQl8`gq0kp+irl zmbk57mYaR&!|lm4TQ_h%U-{;7Xv@5rhr-gEdG2^v%cRNRQ8^hN?-u-?dTg=vgZk>a z^+tF8eZuUk#emK#*R6j|GWr#7)T7(UbyH0`x*9#Y&|Ob5Ub9D`ET2+5c*~CdJa@#c zNcHjd_8u*fQ&aMbTq4(SnEBfGUTaU6+bv|n2sV_+8wd`_z0*JLqy3qC|A*e*lB=$T z#|qCJ+gx0z_T+(oQv31dM>EYd*B)f+8VC5uzm&NoJ9F`c8y&~3m^5ft|5~;h_ijB+ zQ?XSjpL5wR$6Wg*`xtL@sPpEA!&P$jxPFzmSnYD_DQVCzZHl+q-j=_nR><$&Gj`kn zh2}nHnYX@tK7a21Yfs(mtZn&2`)XbuAAY-YRB;1`O&0y!23|au9#yKo!s5dEU(z&- zg*T)b`pI?IU&%LXzjOA|!i!lQp7*Yej+RC)fUHfON-;Z?+p1J5&%9(TXq5{@NO~_KU(^}fR zba{^%{gXls%w|US`L%d&8|Nm`&MpdS5pl`hZReP)96R@8=sUSRwW6JdK0jK=uI={) z?OGow9Y20|3$+Ve^*5Ns{w+M0iHlSI^*QU3Jgw1NV+i@6#< z`!_nYt5;s=i2c`l-0(j5t@Kxi`?e>WF7|u9c&*!%r_G}5Wy9(y{VZg!*~Iy3(y&|O z);VgQ8&ILX|Ki~DQT=;flV6;(yrYu3)NDY!#(KjuZ5vz8fBNft%JNi+({;0K%YM&B zHh-z>f? z^Y_Q56EZK(-f?SBxY>lyDR~DbmzB*g|F9+S{QhiQLUfZUM-F6csT}xl<(;&b$@{y{vWx8=GE>Dd z@9X}1UshgeR1x%GUtHMLad|(6Kk8}vYDDXgJB#*}dHYy8RmL<7*l1OmCtVlWdeHib z`@TQkf3~($n@pp()901Mt4&Wnbyd~jVQSHr-O>q52fv!!uw72Uk%t2+u9Vq+e4KoA zL0D7iVAG!K<9m#(d}KCs@1{oWbO&`EZ1jED&{sXmj~I8`5-~TlPS>!*OAhw0=-4M) zZl-@hzRIKx-wQIDT>h|T=dCp%$@h;MDaEMnTbA_YrJ|Pk`{D(KeL@!|oHm(ltm`&w z%L@0mrxTt`-dpPNu5?d}uuVQE@o#u7 zs*`5n8)nHbU-Q|$jl<4&>x#6^>gO0;opZO;y`EC$_>)%gms36KjOdZ#_jLTI)RMO^ z=9ypFVX^nBOSDx$?vDu#1`hwZ^x5pyaT#qVCcajW$Oy1)x+|?TY-Y@@TAL=$l8ZD` z^{t$`ywafJ=hLo7MzqtPRBuRw0d<@HaNQ8r=FIZZOTGkV6?om<*mmcK!v?zF_I_NN zynD9(y00CUjZ|4%=5%$-;t9F7CJPoS-#*l=vUUEw6LadlvC`i@M`PpYR+CTLXheSN zakP6%Vt{#stHz-A1^vgrGBt}GkrDLFXvw+HOXV}CFMO=GfB3lVlH`uv%=(RWnmJVK z*1Mwx!)6bkSIO3!e>2HQ`JJzwc5$G^CGAAh-|<$RHu>#8xN~ds)j)tuwyu~uW8vsrx!}TOJDL-nC<@joRh4GMWE&^fQyOz0US*ll<`M&J76~Q{68gTV1<;tIc1Y z_u0EVnU?PdkI`YD9xAJJ1@cWJ3gw$i?P+D~IFbPb2b%~iVmG;0)h5fZo?+mzieMSd|8w+Ffa>K1&PPNS5I7aJ|&*IhdZwP(9fi^rA@&|A2whr3F9=Vfd66YIdc4lO0=cf7x4CbIyw8OJnv8`x+dw zxz4N9q-?n!1Lv>pR`2rWvfVk0jvZZEAn!AHU*Ndj+#`%_bc1QaR%n z9itt4t*)E%>)VDWq;{(f>eY4~s@|a9m$OFaI`4=my<}B#|ElhmMsdBb7#vrN?!M^M zvJoH4H=o?w()D&(hJM<(sHl-6dqp)mwWi!*U5BL^6K#*_&v=-$uA}OF#l&74&zD{p z6BLs@F0IF&{z(BR4KC}S+p(yLO?SJDtd=IO;+0wU$SG& z^$okzN;{ni-BhajqF4Hy_K)uBhaZjZu_|I?`GW9mgK}T3zp_PffK<`1_GP#6p&L9e zt~+qHo70TwZbJ{0Z{Pj3tw-qUfOGot8-5K>l`pWz~m*zb15i#hs6vEIVG=b!FASH=F%p$Lm=e*FWfI?RTTI!=9!YNB4)n2w0S1VSRm6 z@W+?RL!FA1DnG5BGB$QcadGR(6^~pj&dsXlC7Bppk~bziQfKgz!*RMh6bxj?p>Z}wcF6xRquNjgU?yJ&TGd)~AAMXUa|cTZWXQMbokmlaoQ*Lic|g>~MqI+~XqewK~b z`tvQj&$covxVvPF%^;rgurXFALxRcNAHC`4L9*-Sf94Zf;8~oGxsB$Y) zo871UvwMCF`ndY!j0SB2N1h)terrP8H}ftX8@f8t{Ca~|I#ZnVO5YWKH1t=SF>bBH z*#j;sb=7{}QfjXDCFx8vpN}h+FFtuhd%fe{EB99Sa@nCiAkeG#1*Hy8ycccnSu%5AbX_I`Z8W&h`q&wj_YnbGR6c4^^1&kcv>-|K(e zTs8TJ-FDp{nvG@+-F(Gr%ps$({;m9mVo)M2e?jAFKd4BP}Q%eVY(r91ovD7$NBH#N}bh*CMhnq*e-$ZQQ8n!27 z)vx+rzs6niSrBsg$PM)?L4KhNUlfcv*>_O7#mKDj3CS_5RoiWpN#la=f{%Y_a%Mo?Yl^geIvsCeA)#>LYZI<&Iy`*MZi&+u}1OzlVLR|O(BVUR>FPD8GVYHels0_MmV z^tl1K2Y+^I8tJv{)O6;`+o?&@6ztTD=P344GoPVkuVy{Hb|1Be(ptH7)vRsR%x%?- znF>>7PVCf_ZB^jQa1hvFVhtYKs%gxq)n83#x_mD+seCgzHJx5+8n$Z6>=g7(Sj2uH zFW`^qPOnv~3{G0J!yI5t^1%-CU{0;xYC1FIZPldH739~+)m77B4rIPEF6_$mYx)B- z3X+Q7)t#4@#pb_f>II76rMvR)$9*?}Gu|BAdsT&z`IeA&HvcDOmwa_c&Z!UC6Qjp-| zO4UhA*`5MTK~TjzA@5d2gdO2nLE^x1Thw>tcyHt`9Pfu5yLwigbwy5Nkm!Uv@-YN( zI~;io$GwqH;CKM?bdHB2U&rw%w-Va}=Q!n39ZzPkoqR0F?P1AoPMeNp`*t7lpGf7YO1o87mo3#VEQ zyj2ZcA{%sq!&`#G10qa)|MW!ii5|ncFhzKU#7Rt@?akqj4Onruw}ij;@Mi^o9pJAc z{B?pqHqyk|zBBw;2u@^fEM$%>WUee^&Mai^EILZO`Zy1;vvu|w(7U&r!(cD=O~l#3 zOA^dR9)a!J&U$vu~8Y59`R~>l4Zby-31@jVw~CMjCmcPLNM@5bFR2 zp+bTl;SqG>Vm^gMdIkEkK4mc6#Wy0B1u`o7FE9Q4Wqrrkco-I@(1>V%NkEjpzo7ER zQ`8cB8tU&8B@qEsk@*x5%z_o=9~m0#>*IwdFnD4$OlcueSyV=h4)OPmVg1mee8VF7 zAh7;%nEYrdI(P#1O>58cCP-`cGhn|;=(fa!?hnwcgu5g0L2Mrc^qaFE@j!q~fVW^j z;*kKEfE~!^#HRpc!rRFZu#6kBebR^e6a@5**pE28I|l*YlKqG`1jqz9+*65n1jqz9 z{D=<)$b{#@ed1r9DX9OWpC{rbY@h6(74bGAezykxO2nc6DB1a51BbC3AP|0NE08{n zkpO|Pz<6ik)Zzhsa}mexJTf~lK8f@pO+i3k%(1KVKU`>k{Ht#(v+^(RR|9vifqT`! zg?7Nd_TxqR&}O3RnkC{;7Kn@Up_PcQKz)eM4kEr8ImD0HPOgZH?Hm_z$dlynr5d<6 zpNr%FA?ow#t0J-k?S8UP=?LW!`VSL_JO%>c@63M0djMnt`t{k5ICgQC>5I!Lj0pn) z{RZqu`knxpfPJxkkcF=vS z|FlS7N2GtH2K^@@-bJMU9y!DtltH>Kda#0cfV3k{bMSzEV-csh8hCvpQT@B6a@Tj#(rccQ=~5r=X#O8yhvY}?Ss^1zdEu#;jbR<1BZQibPr_&1UQU&BZ0gJ zLLABl2&5eda4547ryy`I0|Blo+Y|oU;68BhcMlXACN9E8rVpIGgO`a1$7wDny!}_{ zZ&nZ#ss1GU3!#!=Lia)9rsyw~JK|~(c}&uTO_lvliUP>w_*{&LRUH3;5xIrq8!$Zk zIX(#!>JZ22K6jeq^YFx7j#D{*&hc_+yfFDJY^tuy3oe|=PhnHVi!t%FFkevo48t)Y zjX55L`cjTRhs0#klH=Zxv6)c$CHuJ;|CXG7PYfrOYtr||%tdob5l_VQ?Zw#%Lpyyr zejVd~Ajh-ucqGSZF85%Lx4?LyIdLgGG-v5FPX7bOPddjdF`vK~MG&&H1M}Nvp5yf% z=lCzdx#SYZDLgdy9ob)k`JCo}Ax`?Lm~O<&QD2wikI`QmLrD5NFq|~k3vtSqFs2NI zIQdI!2N5TKLpgo&7sh6Z?U27qIi896Z57A2V`aIX<8?5fY~i>8@*IxqV*KQC+!T2M z$Nexr9OHO<%n!vJS48~_93PJ1xyErn-`{Pnhe<#{m$MGw8yo2Lq zm|nD|5ZS+s*L9!M*Twqn6~`}Nz4Vpi&rx3y(~ay8!En;tlEf=9Up3?OQ}BA*a(pMo zPiKzjqJD3lW4U(YIL+1Q&GAtfKVcm2hxwD{_Mz}oJBijjCteT3vxc+t8tw1kcqZQ8 z4|3cc^Hnj&>tlRg=lDvD|Hm9ZkNV{tKL!D2Qps@-OqV*Cf9bl0U_5AZd^28e3y#mm zV{?wDW4+X!<576MeK~H7;Tg*DG)!MVjz{42jOBPG=GVy_pN7{Z40-!=KN^VfnaSx7 z!un?;$D=Wx_i>!YW?$gAI$qaJj>}^>-*Vgv%iTwg|Hl0Do#Xl#9-6d;(or6-S4l}o zf8uc%Z?vWp@%xxw4LJP<82`;Uz6r}|2acy=`gY}b28Pp?QrlUH?=~$cN z=dpcd%5ekCKUN%HiTR-m$M<2mu;KU(yx!g%uaDR3!f_kCUJs5B!1CqGak}qDa9kVX zaU#c8V0|*1mXj&r;p<`Y^=gyORYhW`PlpO5wD z8;;*Ze}8cN8J43u7_Vf%9=t?n(um_Ur@0Zw=V5xa=lBDRH(QR6!|)8?_#15Bj^wyD z#?Mra-@){vH9+7rAPV;~PX8>X7u~l>{}b|koc(8?(~&A$$mU?2Tp$t)^CG29)$W{99PEt5Xx~Xr|}%8 z`jE;!g=ZwD*FsJ|2ID`A;~g>oZ|67`Ye_!GyWn%yDUQcO0GM3kxB|w*BaSyhf8TO^ zvS3E?gX8qQPZ`sPu6GvZw}u=ahvlw0$KCKgYsT@1SiZV(yeY<;6UV8&I)vlYe)i?~ z2Mo_xju&9MPvLkiJWk{I5sa7R9RG=SHgnt#_47Dxf$@BTT=F*r*XZJlkvXun&WPm zzCSr0i1DU@{?qkN$8Z{O+z;Kig$O( z=+8!OE#jc>g4|BTL4P&!ks=QILy%K>qIf%id>W@e3i%un2m2++H;OpeAA@|Ch=YC^ z^3xorenh1r4t8Qt|E-9FohA@qCUvlWrFfelU`YcJ2mRKlFXebQ|RUJ(cV*{FY%<13Nh6>+e0 z8|^$3aj>%q^_B2>9f}*2J9_U!?@K@pt4OxT`r(N5!OdbBWqoToJ{Ng+ju#=fw2DE~j< z_;hq2OgJdazW{bnO`@(XC{Q@5-&h-tQ$J&Cj_*MG!#I8qc>r=s-wD`$2ovdJQ$Xfo zjz}Nem|#Ac$8kgC8Jry~wvx3Rhdz-YJ2~D;z>)%vTOdEpap=<$ls9zU4#WI1*g!qYs1160)K1sk5Lypfu-j(B&zK$HQjE8wF8%n?BJt_%u<0!AKVPUbT8z%6Y}#Mk3oK!!uc z)NXko;!sX!WBv6)#34Kj(9RoPQGqBzr)MT` z2=UVe4w&e2+*QC5Q{-Sn9DZvNhjTC<3#1#zbCBDLxH$a%L>%l;f4%`6KZ zkon5S+j5-RW0OR7#NnSQ;^6NUw4cgx>PI+F#KqxXCgNb{3EEl3@khwlA*b+9#PH{F zd;#)vB0J*nmxws{`yTDz<~Y44d?4cD@V^jou%n1Khc_ITL;fB)g?}Z6Q%P6YA&!R< zLylAX!Ghz|PH^G)L$n{v@p9ye9H;k?OE|8A@so`lt`*{;9B&$%IX*>Q@L&hW={a{d z$IHPWlRSQjZe7#^^g4Kzv zen2l}XXyR|(FhzL2PEUr9q}f3yhOys?XD9w@Gl&v`(_(VH`s{p@2(;)uIH&AGMolZ z@9P$F`c~*4jUym^st@;Y`cxklah&QOT9cXVQ2kTN=~H>9e#oRx-4?%g0+2R7pTPyPcJ=YQ&d4qTl7rR*34yc63Gl~I9@K=rS?prY(xc~gWb>Eh`1)Eg=vR#?j;I<66v?2d#3wp3G&f}nKjtckS zKqPd3B-EAdK-0*E0B(!n!kF9?HnK5@P`>D1i_KnHy<46CRL|0QEj07bToF*8SLc5g z-Z-iLfH9CzUU~nao~_P*S4@x>?7Ud`>xcTgI{yn>3$7kwJc9r5iiP(d>ecG}w*jL} zX#Nc0=-;_ApuVoo|14}+&>DE)zY6>D{_kf@7DPJ1j%9)wA`b#B5?*Z&J6E0m&#|Fk zVsf9E*h9 z6Z*k+s`KC4ThI{u53gFtJ*8j68vK8U*MD9XIKdYDlY1f>?0j|p=Xj$AmP5FHcqK>f z$$ta%pI<|+5CY5OI5`jwtK3tt+ScHH7Bgw!lv~cCm9NTZ?K0<- z#}=O*Q_2JfjwvS`li#Q(-C_;|;*@gYm8N>LBnDtDCtMs$w!RZCtR<|^7UTP887Q>H zR$aSL1y2c(tqYf5#b|LarkL%V!a`A{6qCUgz;dRgyI#%1HS8RTz|wMXrGJ)|b7V`) zL1Z|mJg3#=j?a@_T$LGiaWC}Q%tS06Qf^@&EDB{yw>en65yJN>TiD7jS7JNFSy;S` zExecVNmzT&G3C0OOY-L!Jz0ov&}FG7onmgBQ@#j56w>ZO%*l!&EY8*&7eqtFRM~mq z;FU>$db3_Eo{tFr%hA$alv+o?3i&__`mt5|s`RVQ^-6immh_`T!E@OXX)$Xpesh>D zBPXoKcbwumRaSXBvb>Yjn#EX^i#CFx>0i65r)#hiua3zA%HmxnO|Tr$aomdRA^n0H z4%O)qct1M&!He^UdPF6-T-$IvYe>D(6T=$26qWSRO44qA-OkNx-^s>@QgXN2OtkY< zNNRpG?YfKhNOuj-Sht*HyGGNLU-pmm)zV5h{JUE~%NyIeq(=Pu(DTT*-_dGr{VkqZ z>YG;#mJ9B9>cg5RlM_A~ZdDxh&|9z8wpLH|E8f>?)@H!7V764!0+T)--6nXL9;$W2 zFQ?Tr^^_%3EZ&D4*pznR^Ju-3b+$)#UG`|CzD7=EJ7tYdcKw4iZzrj(8FTuI&1&-; zzxci{Vs9OJ@$r8Cv33<_Y~McrH9J*HqmxxV*Di%8y;XG`?=RAAWE#Jq(?U;!__Nd6 z*zMTmFny!P`6hKw#;$dHk=OEa`l4%_Rd=mk87M7Xx614BmdS7S)b6sa)sQJ|uFwCJ zH&D;xNS^YSx*vAFP|#bQH{9g8N@mvkdux1h_XItEZPL4PyqoFiDFZa;cX;;uxckm8 zKS%#OGOED_<>ks(lXQ=@^_govV3@S+md&cS!xn$vbmmdu_jPkiZTp_-=s^45{cpA|Y(ejBz);PA-aTsTXCm%4K`W?Fk{ysV(MP->$Uzv1&~3nU38ayx*#E zM}4LB(Sp^xjRWUSzxymc^iv~JSO zZW|O$52t=Id1<&W_k8W+N7ft)Uh9?~|7`Qv^}}86wRb(|x}j6P_S2x3eW!o5>vQXk zQ-i17%aax#PCZ-{QD%Dk#;c%k%_SduTk6GVHFBDDtf2nI(Xpcf7v>r*dVjWU{eg$h zJ#8I#_sV4zt>aJqH4a8ty3KZ4*dt--#)028o1P3%?D{3@-NZUg62A>IO$u(aZⅆ zo`=7En)$eS-}#fypD*4qxck}0gXPEWY@U^;m}%d2&Fi!4-nD)D`R*{gZ&S5Dx>xSp zTB?`*sjWf4-dFv`x>`RNv$Foph%`&jmoMX6Ih%O~>!-cgm1Z|!=B|M4M*>c1-`vsi z@wA|ktLH@KDy(j+o!z`-{+%0k<8AUQ_MLv-FWMpcu!`ZF_TPJMFTVa_dGE#(&0qE$ zyfR~8o2W$}2!(ZOMQzW%-{c)$Z-hqEaqAYP6#8FVIqPAYM;_^V z^*w`ICZ?X~lyY@gy2n_f^u}X$pGz|@QFb#LJNefF(~0LppM73*sqf1fL*$y|$^Rat z^Xt^t0UtVl_;KRk`bmwGgUWu_Zg}lz0k*eUjS+~Ara$#X*dW_M$80oS8yNQLqkXKd zlfnsi>2o==rh|=hy3Mcsu3j_mjPGral)Y|gGw5mXnB2hpgoMRWRtw4%g3dkKtSP@L z-BbR2?zNpQhgGPQefN+|>Ercrol~KAt!MQT>ITIYw%psmC#;rrlbg%bpB^apHhI~* z@`t{BK~~s==Vz{X?sPjlW!JXPwcgk@c0D`AvCFbZ?>r>9zEyKuv4(_@X%!kOuHL*o#&|ksI_f_ zTAQ`AW9xce|Jks|jJ0zHrdW6F?NMHGbj14?6Lu__yH~%^rI%awttRTHeps*F;hA=} zUdr?Bp^hbihS{!zq6UuGa&YOEr-$|=c-~k)Ak%Hd?eRB;Dx6#Td6z@`mI>!lqQ11( zpVRAoseJmz{iW_x@~yh8f{K{pliuahaomd7`37n%ejUCHP^zU?U+vQR4Ldf}PtRWC zAa_c7v$0i@{NRTpPSvqiQko{M7tGdMTslsr?fPxEGk^aoEq|@px5MCmnH#_5=j_(p zJxQ_U#1r)j9CSX^8*yy?++2;E40*Q;9rOl&+&g#N!#xY-d@oqN8T#?7*RtI8!5952 zyBV!C%AW3ACa>qTL9g}IpA!zYskp1yyTJI3txwN|*8<h4KqTHVsmM?3;GlNXlu^FCmm&TyeFGk$V)HVwEfZB#v^IlF5QsS95{S^T^)ntE`vSdn|4*|kn5_od%%_aZ*5)LnyJ|| ztxU+@zH-2o=drKT%`Z-_7rwA-ea+%_*^8eR?@2hKquV{nb+JjEIK7y<6G!`u>-ac- z^v|-UUG`sZ*(pa}YrVDlyr8$vN-uwndin5t`L`Yg2PY)vPoL6j>9*}R_Ad&ob9dg9 zxx3nCZV8yMD{0%czHXt<9Xu0eZBWS6{G5F0_~I+c=TCQh(En`PUw_iGHzw4dc=sQC zC+dz=6Tx$WbNr<*lScXR05uMNLJ6}`AI{%G;s1&tK89$FDS z?$I}!dkN~-=Fe?ilDKPd-B+6$=nXrSzbZ2A?3-r8?_Z6rus(aE-}{n~I@h|i7;NA5 zSzNz*R*|#9&qmoNMFm-W%s4GKOVj1m;|IH&ADVhY=jef}!%wd5;25b<`7H0&X!}*_ zgTHt$y*gu9#l_>juHFh8wK%?Go4J{LLUNCmF1p>u#5T>lBz^2q={Bd%;fX%_s{0LB znZ3v~FBz(#mamf$SXwNX{%Z8X?Ym8^TPzy7CNb*S(rfj8c1?_LHE^L@(tsR;`g%`- zb3ISqyVp@Ydr{Zj?@ydIY|^(Vr1`#(1vy!-mhNdgC9M4CBOkZzsSP{-T-|b(YP#*M zh~4(@3kMekjtIMJxTVkWK9{>Y1t{D(Jr)-RoPtDn{L zXXw7zBD1NN=RaTH{KgjT{z1v@u1FEszK-2cGJCtoLJY%Q5yuADtqk`(2rDX)f#-_u*3FMY$e7W zr)f98gl{x*S9RC_wArVV6IN5^>iuk)3MflZ`8Ly=9WKV8X5t+1AS zJvO_&c`r3%ThZkDcJNLZzR~)zA4rQTJ5FH70qih0GhnX%=}NUO*ZteQBe<_gz*Nn1 z<5&0A1>VI#?EEX$&6y8c^8b9f5ZuaFy(#)*3*O(#F&D4}p?68}vfti_%e+ml>PHew0=+r}($92EMcg zo>c?iR|7v?1HW1We^vvBJ~-8dpWXVauP~n&-yq@IC4c!4 z8O!Es4+1-Xn4Z_0k89|4dFaHx}s zCj(>x96pN>pAV1;aQIw6Jd5pvKzQIY6!FafnSj2O{fI+700R0@>k>ZO4B3zLyNI|r-ufel@WA*1(jO(# zHxlsx5pOBtRFi`p_^GlV2)G5|4d=*y0CL`bs7Sw!NT2c*=)<=~vXds#7sngiUqHZLC<~;&79bO_(}?|u?*+&N z^u>0L%c#KNo=W;>kn`ccEz%d4yGNWp*@17mAiNz_QM!xm&^L0@Cp*;MsmpAM+XU2- z0S=$(@5-}3CQ#NufQ##Uh+7ci_-X&Ixu3x%lz9-ylOUiEeeOsozmdQD(BH18Kzs`x zJ8_)mv~=aTGh8?mPhlhT7sB`yxi7~XDheQk|f%Xpwahm%UKDU7oKZN0V zAmA!qivB+3xCzGFD~>P1;|h+S#AE8?O#T+49cW*H5EqWvF@}ryWITqoen5y{#c)F3 z0x|c;<8~aUKAxR9egf^lH-51ln!{E+S2W}|Tgq3Q{tUdXaU7@cEao_s=QSKB{p}p5 zFX-dqT_-qB-T;rOuQ0{K zD%8*8^r>7N=Q!oJiySw^aNglKwHqop{ubk*lH88>h!{e~eFK zj-SWl_8hN=;pxHg3utE;$0>h!aeOV>@#FYuyv_uUC*t+a=XfVPUd-{0Xy-7;haz(L4LMHF)iif9*ny<-MmzmD{T0Y* zZZ-0EKk_LeeXwJWJYB?Lm)fIyIDQ}X&u|<%Oo7}-PWc@AWP+55^f9Vr{1>NBEcWa41pb_+sRmA}%f$O+*~*JVO0u9EZNkAci6?E*Ijte!&jB9|W-!=>u0s z-buv4&Rwjp#B=%ruZ8*0RiqDgY|+kOj(11y!*Oaa(VTG*|6per>W7H*!Cxw;F(MB7 z)WZ4K*ZxwUpyBt*rE0h&83zy@j>B5ZAoM<*?7u@!`H1)zxCcW1QW9Fcx*xFfC!1Hu63G^>j>iD_2YCfV`LGYC{~t*cW`|E(Fl_^W)d*&Xsxt;`4;bB+=!6aJa9iGe zD|Q}k^)G4r{b12)%Pq!a8Igxan29!Q{n+|zLy+o{g~tj2;yVSQl> z{zIKb-q7{K`^xJ4-(OpBN6$^d(I5A84DYL|b5HF_McGNx{@49BcB;DYd$$wZSfhIg zKis4F@SkL?I{&3`u}rAlA{_m3Psc~uPIdk-#T&3ADuVye?&JNx%@}uyG8cnu@SoSc z&a|uZ|6w)$YgXfbIr&ckllji_5Bg8vVGvh^pN?O$o$CBIHWM_Au^hnlH()>VhVsu3 z#^9RxF|G67+~axL(3&vD1aM2{Xxj|kb83?J zkl57-xua8N3_wxgE>rRq-J6plZt407!y3(ClXp*ZBWIB!Zh806POmQgTrC6#A-l3&jwoNIvZGAfBa>{#K zIcEP?Yy-!%?qH7%gZyLnZzPdQBSyoZU8z)<*u_sOl}7moc=`DGv1^qI9kKi*l6L=D zz^nh#Ufuu39@cs?vl2JF794{qTm>HBuuFlZ_ES!EQ%G(99NeY$Yn$q3j80Rd4H0Uk zx@lBFRfN)1x29E42cdPUn^hGwAMrv64Etx{=>P}+jv{45I3-F}A*KHR2q%@XaC|VK zy=j_h;AUCH%(bzdi zdDq_dYqIkybYF+QZhJQ~dW5Sq^w@qVQMoR4BBja zwS0u~!W~8fHc8vnUZ|$Mw(!%T&?T1B+HWY$Id*7)eAefX_09F(#Vp(Bz0#+h-r@Pb z-!HX$cA+eF+q!N2wnc9{yRFT(PQ%LW=3dP4&{yzU^n0LNQLo>jZ9Z|wR#yy*TViFv`txpu3stM6*3_H>+L zh1GNGl(a^^_2k~Y@wWAtGkB8rA^*1ad5hPUH@ZA%W7z_Yb(_EM*ga-j@#C*XgVS5@ z>%Y12Ep^TOpmB#EhE0lW@p+J4*GHN8&yqS{AHUmk^~hhYQ?6&4rY`&Xpx)YMOV&PE zytb!cXYSRp`I_UpyzQc=zM_B5)s!`wq4~`Drl~)-WGwIv^GV<9aHhe>e6ON8H_~oe z{k zZJTM$O;3D!chddL>Q4cIC&##@ZmKky`aGjYa4W;a14T|PWBpJ%-E+TO*#k6vlI z*!oSx&NCiq+Y=(ohcr-He#R?u+10d$9Y5Tv++G~~y8DS=+XrR6K4TrTwP9rHTaOD( z3LIUVA9)^nB(V14$*!+VC+M^ZHF|jVUAx1T>iZ(vjF;VqWrg*h`1x_Xu(0q<;n763 zo8GG0N*zwRYIg`bX`>ycxA?@IH6J`;rcG^rRMTeuEbAQQag&@w6;+K?`+o3PYN#VW z%C?D7Z01WOqpJ4k(+sN$<*ML17AKTnEdGg~ zu=9uN9kWqY^uDt(aPH$-uA73I*InjVdClgzL8_mmW1`%q*2kJ_?KQ96#Pv#KBZV_g zHXn)Ep-#b$Be=A)kty_kH|XUw#7u3MAZJ9W`&cxtPaZqwMOO)@R)m5Q&cx08F? zdd`JwJ3{t1IOm=>VcXG+!XtBn->(awI`NIV*8L5IN*g4L&#zfI>Z7J{-iu3DwYE(v zc8nOGkn10`x|_kRvY`sk!o0tHewSV#XL(Y)L)CStVxx<$D{jRk*Ms3~3{dID=e4!0 zCduhq`Wxr(9oa=?cCS;@8lIWAN#Xg8CA+U4dwVIletJQR^d&Yyo?D8RP7Tvde69QF z=^Hn_ab5RKx*ghSeBae!<9@DM(9h54N#FhDVFyOf>AQN-#e6nlR z!@IL{u4FFyy~ND>+^!kLQvqJFIO|uc6kG1D*7=!fUNsZSyon&Ns=pV(C3Q z=`yb~frpYtmINqXPzp%9eXw#sc~gtVc3Uq#%(1*VA^B6sImaE|49lM7@2;=%Zh8Cp zd*yYTFB`n=M~9S*l@Yf26XJ??1@6+!R-bA6{HPp!UZhQZmu};uVhW0kdDENy}({WOhdaEf5?+>hjG3g|1WbJBa+?TyCu~pNU zUaM9z=!2~wJhu+E!nu`jt|o!}T=Q5aY|wL#s~fWf&zrSGZzpBfg6`!14A8Bz)JCRX zRb^CNA1KJnW)!PZt=jpYk4;jmax32XGg3-S!C$zFf2HQJNsazhW<*|sljd?~<*RxI z|2g3x+x<75`yz+uXt4~C9pDEn(TTyxjYSN0g}&wr5`0Bcbyi3p1qsyeV)2z7C{*=r zp>smMQjpjXz-^dps~~aWc$Dn8j>MJY4#T^O37a{;(iOSwJK&ha<^OV0@GX~H3#O(!VGJSZBNZcJcuMc$<2;5f1 z_N4%sfWAKaksYWnKtNw?pYkuSPbCWUp$w57dgudvsfbg_2Yu)_M*393d3^{Qhza`{ zupjBm9)#I3*ny`q;+AY51UTF?h|`!7;1K7;`>=fwUSBBZ0*A1XzNbw4U;SVahd3ww zu_7)G&twsYT8#8(inv%`9MAC1i1aD{K)i|bNi&fh_#Q&~_zF>$?qdCpsL#i9caa@& zJPY-v5D)Sq`$I+g;&`TSj1bRa{ZNslRj z`Bw$^VFT~EL4d<+M9QyFMnHgz>s3k@;&cu_0m$^hCiL3@X(?=ETzogB`w97bOioa= z66r&D)*!dxIQ5Nk;rL$Er+iNKvrumYr+)~Ky*a)W?SygM3OS`C*)K!AcupS=B}p8= zgvawZeh-gVaGd7d-puh~7@oZxhco~wl>Prn-56VBpOPKAuTbAO;GiFe`o*06{>Z6M7@USWh04imk-h=y!sfGx zL->919Q9EkI~1PAV3Y}Puww`ZOsJ0naIiz+q%k67rwb@Eapde!c^A)>19sA6+JDap z1x{^{kK^p};#A)ibNc%Pb;(VR z)0n|~B7dol1t%XlPIX@$sFayNFd#pWG!SvH1NSx%V-W{^YClMDS>Kjnh*PebICPreCil4cyI z{A0;+%0IRu`xpe7B~Ot)!~uoho8uHe!5pXhCye90F`VN#J{|c4j!#8SW0@e{Ae?QH zuRwj^5N{;gMI7`!@We?G2YqNmgWMHyXa|yfK@RZ@*Glae>dQzx105)!W8xQ()4I~c zi;-(``fribm{rn$jl41H!}Y?hA#xh?!N-FQr{4+n2a32j9z2nQztGM(1O}OSbNmx> ze~#0AEtumph8^03AmCh_-v-L)ACA#p)jr|%vqbvnm5f~EIMmS~<-$hhuUKCRIb1LN zNHlPV_g5<7VmnkWK_BV|5*yqBPGJE=?I7MhwR3@s?Hhn2Ccwq^sT=?o+iwoaOyHW} zTAbm4iMZVhe3py~&wH>-`X$K4?OxEQzRaERJn8E~0GRX;HdS_7Be&VyH?118RMaMA|;suA9OE2abauO|1Df60`Tc5n;z z3)r;exD6gty-ladf7*f`?DN;3#TdvaVN>-Fb&OcLvIDpm*oYz9e}@fMZ{$+8h4M#b zjoK^zj8*5q46<`{KV-Wy;r;h!v^xJ?;pWK1TlO!JNZA(J!My*;jDi1r{8692mF%2Y z$UV3L$z;Fk!v7EzsQoBYW`BJ67co{{_;WD)L1;(H^q?*$|0(|VGgh7dkx=0=Sto3& z{(1lRurt;9ucIkA5&N&o5bysD#$-XH6M{2|4rXZfIrlH&)Fe>PSLc6XeZko~tc0cP z5V91xr|W;hSatrt1EWl!@0?i3J+aShr#klC`Cn7t!msE*U;boaMVqi` zT!a5{dV-R-u#rgFwjulR{=<7QnN2!@`sVnMdkT+f!>LK2Z)0`We-r(8l^K^vq|87g z_9ORn{Wk18sPKJ5zoP$OUo7OFaK9Sd8)LpL5Ibi3y!#R4p3^Hu_r@G2e`)K7?m0Dy zDpVHL2)UzEFjod#DpnVSB^v17yov}r6n?@mMigxF?rqS0A}QjQcMo%%RF{6n%>)PD z=uXPE@a==#Q~FisH#jLo{~fU&GGGjfB<~)g8CBTcA5q5p7t}9e0sq7TWx6`|voQQL zwwB!R?#(PLgsz@aL8|(OT1cgqRu&yCI|=_5=H?x(tQhOu!J2KD{P*jfxCW8$bf30i z?|0flUrgmA!`L~YzhQuSd7u2PDfZ^ng|e5SvSV5doGW)c z`drIV9ztR-{c~DFhEwcqgxLt#;v>e6r|e~~n;lb`H-FX?F<~)aEUb;eR>tTh>wgMW zn&cETY-FWC=C|vs!-9D?|8PH8{w2ykIwCg8*Prz|jrNTSj*JP82!~ZPXWM1JJVPD{aX*_`k!Wr-lxolUU;5|~=u0UCgzzQkX zEDGHnQ-1#GGs+V419l`8Vo6v$22xWNnUpg9jGa^M&8u(?eNRD0FyNRbdt^r|lnotI z`&ly`oBzU&&3~|?!|X`^3p>*PV8@Z!as4msxc&z_%FK?&e_==C|6~WxrK)i8Hi8_< z(qyDXp{*xh+M%y7E0?JQSOJr1Ij8nvy^lLUSzy6B0$V5*RSEiEMc{v%U`+47ib$qc zZAKa)3LsTu2UJ*qEebz4ZWU@G$JE}+j&gliHP%~$Z8g}|nr*Gw)`o3u*j9&ab=X$Q zwoWGR(W_O3#9qO$K<;tJ7XOV^cD)}ka^rBYN< zqzI)&q=lkQkt4sB=8RdIsb=&tpCG6-v41B^ZziA`+t}TJSz`zyN$$FipLtZtmer+L%}u~ zl5yACZgY~}v?D)%wC@Za{MZ(@UN+TA)*>t0H)s0fhXp0Pc}1lrPE0|Au?x;8`kb(c zyC5e#&D5*h!seV|^t6ZBn{PIv-{tpwRMwqvv0W+e@t7PnvxSY)IV*E^X&ZowvZrdL7Q ztaH<|+Iv!W9Z&BH6qH?JJVE+&z_0Kwzs+UYvy-JCWl26eSH8DR`F&RXkG=6N9hVjs zRFrYml`h*HD}ObSaa`j{pMbS)?DW*kEp_1=Wa>U|y1ZupiMNI`RDv=X(oMR;&VmYF zr(FZ)RlmJD<#c3*fpTBRy!+mpJ)(NI*oet+S!i21O|`#N=J-Whe^Z1HuNJSpiq@`_ ztGULB8RLqit1F~6rdz)JHQjvt{xQe2?5CyFKA_fo9E|z3V}juUns`L-4u=<=n)7Yx zH_mBU9lQNw+8Ac0S$zJ>WzrRgopQRoKQv6rsk^-Ru|jr*=K0i&Fi|_%m6tL!12?%? z&42DjS3UHiRbAx!7OAXr+L}9sx#Wr`QJO`#S}b`wJG3=L=8Z{yPaW?ct~>Q9k5iT3 zxUjy2X&Z%S8x2+|CWMxbH#*+>Hosco<#JE=&6Srm^}T0239%E|S)e?}`gQre6J8n1 zR4w;@x7o4x>&(4BzGa0TGd55(FOb_O`*6NxfqAycQA7IYnuCSGHK}F&URFZO#PVbF zPb!DIo)q)@n`wW1#cJm{{D(x|r@eXQl;bJ8MEHH0+_a-Ih22vnDn+KfZm=u7QNt_o z^;D12wZ1QsdB?X)yR6gsoqYG_1@Et3P1bu?lznJ)Yp>L^lb!NxLK7`+%I27f=U$p= z8@2~Ax(=qTOt{|Bv+7O2r42oA>z~x+e1Dhhm@5+UW$C)MQ8-kZ`wPYXg6NNX@@0G5)MI_~ zFP*vO_Q)=rWS%mT^BFE%fc5$k^hxwB@8$?XA^tBxmIPx8JT`CxMD z)dxrS9;bUhdb?-f@!Q1oOWOs~BJSMHxU+n0pvJep;HcsQKYLzmxF7z`-@{6!ea4~g z^i^9_R>-Gc+C4aUr}-M?^H;DvpZr~xQM*I!;nZ6b7BnCG(|%KJx3f;Twr_up zM?}CY@#DGUjRX2>JVGrrHrlqlSbR4>Y<05PHxustcP~tOdR4Ifd5C>s%B+f%UV~4H zHT8}`+wXN0ZENa?k5=Fg*vIdiVOSs=ANq1($?NYTV>5WidT02`9+d50efmB9h_ZZK zXl8;_W|r-i-S-_{tP3+gw`uLglXtFP+kUff;eO8tGNu|a6D)Th{W2wWMb1L&zC^i9 z>+ycm>zx)Hke+zF%kJ63?6fl{f3?_T|5{V`yukFpbaTTQkwx<6IfnL@%ePIw0WIxKry`CIyN$WR} za_Y~%Hdps~^kgN|sY`NC`5U!)PA;jrmaeM$ZgHwp(v2SivX?tgnSEY7k&-{T)=$~w z%&k``vEfzc%7iO}1m&~Nuif6O**?{`LBID_gX*e7LbnWp*7$vwn!IqM&E^iCJHa3N zE8nX>2}y}Oyv)mCLH0S}%0y~jnQaF3-@lGkn}aGMX*U`QFf+q7gVZ&f$gdoZHbY zLb-z8GWuo8!i$xzThIA9O;}Xr`x4FfHd4tlQ*!Oi^QHXX2~2eDHcCx(|E*Db?~;t} z#wV`=3jaiAPd5KBi{citOR_RJt>&@Eg3srYDOVl2R%fL-*Yr7?UdicmaZ#l%8naqo zXMFU`J?a_PDe)`E>1gkMR$u3;+4r$0VfYIXC%$w({92C&zgzgVNC zWRZuan9=sMl2pEO`o%6;pO@W>wj3}uyVg0y#i4BGB9|RT&i6{?$2P^;8P9CfuimxH z-B9OK=9FimpN=S~e0#BJW0u-w-%oQsJ~f_uV3ts5qKw4*2j_d$4yc#hqWx`7+MedH zbD?9x??t&s^E0Zs6w7LrRA-r=7@zf9<<(mYolhCwTWR}FrT2R2YZSkJ-B^10Z|d!e zX;vp5pRhS=eqxiOiSL56aaI8y)^v~5$MyTv>}MIBxMV02G_iZL=@wo|?hWq0J5N)D zOK(5<%P5x(IGl8OS$6gD)bMflP9Axg+|;dm)^6O}*~&Y|@V!XJj}B5AV50(V=y)xM z?^0D$xMohNt4td+z3A8%i3j}?8v3n8O@F#Scyswde}u=M0q*`wqD5mV`(4Mo4~uV( z6b-VF{NCZf4G06?=xB(d6{|H*OrO$}yjcM7-|kM*asu0IUDm*1v7}h_)5pW<1EuRsCfzozT{B<$SJ>-^k#ir7I%?i$!b zS3U90OD=zd%BXP;>np7%Yg$*PPS)I)@%VgdgHOW`k7vaOK=axt}>=2VKUNfVkd|&yL0BK97AinUofJ5pzht7raP_4?2DR<6vi9lrId{S%b+0$7iE;QI6A zpZeot(dyf(G^Hc|&4dk`tJEC$x7Xd|X$;+(I=8BCj;xyVsr@oK*~M=5tJZ8SoBMRF zuIfXj$fE{wn(miSLKVz5%cm57bKYd;3-%6{`0uQ zS?isN>(qmIa($LJJbHMrze)V8S5$L!)WV3ELb;Is`Bp#YZp`z3vRt>{=lPh-3Hk4L zw^c^B&Ng}KuO>qK{6|{-#LTO^H?6dIdFE`D`A@Za%{_q?sX<>QuWa%((~nqLgD%ha zWaM8DX(_b+DKx&tZu9Bo*ThX`!gc0Q5|FAU8e2y z3btZ;*h`)Fh%5}*!4N2prB^JA@~K?%=s}0h-rhO8lVViD zhj=XS*Nxu06bv+8<+Vqhlf781AhB`XQ?DZdmgj}O)`d5VI_N#^JvWX2_V-y*{y$U9 zQ!=h4ed*l8e7;5*J#UiC z`cwB^b-y1gul0%%T`TUpA^+9@qwxpdIR5eLYP(i#-%_8Ze`Ac&#&t{LXLRL1whlN` zYIek3OI@a_@aEQuXP9mi-xThMeOvK2l+d>ogp_LtbBXZE=FwW*~>4#UUC{E_Jg}Ve0loLpDLk)T|EQOuNXA0 zu~z$0b6)enwSpxf?X*Gh3rFVk&Y8c`P-tEA_FKURG`8%xCAfZiZT%hz)6$&4lMF%F0WS7c5$ogJk$E_FT!IQ16~%1-1qp9uDeyH#`r;whi=#& z-f23EOT**MW~~!&7EZsr+A3n^X0Hjn-60L{H!1Bc;^rIsUZZ@nvfj&SM<|!Ya(+!| zt-rkdXp4@o`>XR8#)=d@x0n;Qi)%~9TJ6sHt(mhoUh#Xi^KtUmWA>MtAGQV#2!5?l zPq;>#tZrjm{q&aWw}97ThoiU8m>GM4Uj6CS!@-ry;+-zr`3pY3wmdeeF7K%6#1oUJ zH!u9z`^a!WE2&@XcZ&guzN!7+ylKe3FGB_81-l30yN0;D#!$vo!h1L9 zRu$*F$78mcQO1M_j1iwQfhQ3NPXjXOb}0LKP+dL&U+{7uycdn1UqVAB)^m(Bz@gJ( z{lNDOcd}k3TnhXykO5Z-f57@|d%5)ZMB~STHQ08;_1U7vt>!b~ie|5>ro|@;B((W> z^e3~Pf6an%fO`lt07Bxq^!Y@0k2U6_MsbhL8^b4R%qO4=QiAuE?m$7{?!q7T`;1qi z5zyvN$bf^F3!#JT4{MJ?F-2R*ST_;C7F`Nu$>DGGVjIzXT0Q|SR`y}&hz@3F^mi%Iq=6K3^mXCzkr*CfXf-!7|Mx~Cymb%Pf5Q&e5$yi= z>$#yDaaaM`j{YtsdT`($dHf{l@PM(LT1)2WqSCh9?L!7MvKk+!kVcmpJLgIRkLlz(o z?|l-T3d_Swktag1&whr0m-8$*4qk!e@w45Sv)VmKcNGR`tB>Ttt+!!rE#ij5D1OI+ zmtslcR!H8K#5W-BLE=t`ZzXZI94{r3#C?%`0*TWRKTP7=5HBF{U5LYHbck*@%fm~l zAVRPokN7PTKLGqfth?;&096zY(}VpK*+IP{@tc4X*A>J)Me8W?FIe(C;Fs;##}&Ne z0KCL^Wh+rQTxVsZbm23qBXRg$@{u_F9@9wNeiZHxxDa7wB-Z%>*c3gAJbdSAB>hKN zdWJ$+=MJE!n8k-7U1ei4s!PM4<$1nSI5Ah09UY*pfgYxO4;n^=wk1<)iuD; z-PPXR6)fHd@2PwEfo1&AO>Z|p$k~Woez8q!9RX?7E~{mgIpQ>V~$%q9prG_ zhxtkffrA|0KE`|_gup=#$L*N=LkJv9AO2&W10irw9-ebBKM5glSiT=V;sV@NHfG7g zu_czDi8z#pV?oTN5r_KWISX@`cj%xzoLFOypPhs9@Htt`X<#2ZC@%*7F<%EEaFC0G zf6VP61dg08u^fncg4yCMJuoj=&rZb2_9PPI;bU`HK9#_U_T&@v5X+&IAWzhT`ze?& zqCK?)J;d^PLeK-pEZClQ0w>z@iJ*sQ51#13bcuQY;IWfPozz-cczxbUPC{L^pnot-x zvK|vQ{zs1^;!vJg&pim7Xivx}dKd(GVtpXCN1~prQS_W6=ppt4#P&$kb88em#Bzpn z0(=~Ce+bJ{o51Ce9#{@@2wVwqvOMnRp}Y)19*^hA@@7a6l%GqGw?UjN??#Y^=NfGP z=27Ig5#(W=!ScICk;mhCXb(&q%O{N@pN%+_hh>H33r3N@NRXE&$X^{r{vJVIfgt~A z6nP%R;f#G~G&=OI8|1J}EkW0Vu?*xQ;2%5O?l8ymKv?&ngM1qJZ-XA-;IV@aa^kF- zJ&$B_s52g2GJst}YzQ%kB zlAn&kF}Fv@OPs{rQ8`PK_)8>D{8j|4hu(;*ljI{&y2NitKzaOKa(q7!+Y^QKIFR(< zafT0xpFwtRBXQiWl1Loa`(q@I?Wrd522`)``4`)Xo1{ZXkJK636K(kvL9QgT!&VxSzoKak}m#dAxq64~gSv^#e)#JIYTeiEAPMP7=R? zj#mtc<8_^xBwmW-@i!{4{W*xIlH@DV@yaCeqo`gTCvjOMe}cqO@l(!{I3LnO{MH4m z&veAEkmOgO^0`jp-AEpPUmfQQuSa^9Bp-?L+eG3gke(J2Z$tjKBp!$C`AXurT?~>q zu5VLN`q)ktRBkg!9G~y;^Eg<(581h#B#*z{p-}+w?H^=hq>y{Ks z7fF5xD$hO=A3(=sB5+u59TAryaeKsx z>wrS}0K|#wfI{w%_zIFea)|2@^h0}+5jQ7rs3!_>YZ8w@+=0ZC5O*Q*1Bm+~j?2dc z@nC{HvDEh4^p8p%<19ZWr9J69*3Wt3hlpg^$3YJ@~q?5Q!%v z{o*8EfVd=y=OI3qz+t;kL*p5F#Gx23vITCmN%FW|Y#2q}brg=*d4>6f`MQkki6ZES z9ACeVC-K`zK9R)X{aok{kI}y636rNCy2x7 zuso^YAG(*LaN>HjupIDsdyu66H?oHZ79?=EJo(T#6MusMbG()TUSAg5qr{F*kwkjP zd>Kif2oUSIaXCsm)^B|HQ9i%6e#PRikND{X|^1DeK ze*<$bfx~v8h2|HjB#!IfV4%Quj?7s_u7iDUUsBrb^LzatLI2hRy$(A3Fn z0T>MH55E2)LgLP-93%)F=2rySFO4|Vk9NTxC=Y@>)E|xX=#lsi#EnPMV@;5U{V>jh zD}h7(`ACl^iR0@m0VIyEgKQ=7Ye-KR;y7RUe4Rw#u&Lp4K1SlYAPl-90*AX;59>W< z;3w;ELh|^y;OhX7N&GRg^96~&NBj+mcOu?}I8N6B8VMX1Dp<&g`Qj&Wb(S=^H%H)5 zCDt=<6t0Ch&KDP|KYApNA4xPOar_NcGZGg>daM!0`3gsN`V%-YU)xDs70JgFILsH; zlROGPi#X1gJhG>l#OEV^mBcj=uOxANed;FSIA1x)&Nc!k=Bt~;S0VXd1P=3s^^8R) zZ)iVkUo(+BFM-4MWsU3K~n+-O|)n#8#f|3KpS+qs`fd@7RvN#Y9- zA0Tmg#K)t_6*<5Bqwx8N!?JLB$-c91+ zh<`^Mmp^~`tQ3jk`x&w%o{97pGzoT6(oKE@mdnELA;*C@qL|# zB>ou5KOymE#9xy5C&W8Q{3GH$h~x6d<@RF~E{JZN;r#wZdZv^3AH-*oI6t~CB2D7F zh|fbD+k>y~X%aZGJ{Xa>E~_?!@dklITaTmq5Jcd_`VdLtSkGPphkE8B{U-?=U3z4N zln^+S$K%gt0*C2h*G}M2{tB`azd8ZEP~H$lm;?uHz~S=2?{my2a42udk^=V?5r?|r zxOzU)zmml9INyxKdC~lLBZ=ejxhIJ?p>)GZ9P8Ob;v(q0wij_|9;yl~O@#z`xVr!a z7Lm9N;ui=U=2sZi!)pW%ckPgT6^WZ8UPs_i&sbCr4-toUA_Vp*l;rIIyb^I2#BqJNgv#HKAdgIE#YiB? zLzD4+)&nFSfy(U&iRU7oP2$H9&qo~Q7p9A*+ms6gd074i$j&l?JVGFpLU~G%hjv~> z`k#~ddBk6l^dL;>C&fE99-^TEOCFD#pcs)mu|nDP z7Uv0~z)^TKiOV5>@+dry#7&U@(kQ%+#J3`U%P9N4Xid?3^V&)2$832^v1K&)SZjVawEJ(%|qIMjpN=UCK#U_BN{&$LmvB!NRc zsz@F`uZ{JDBfgR#5A%z;0f9q3cwB8m;7~pX=?Non+$X`H{89M5Q8<2%4%!Lz;BnVD z)c=vW@F-l0z@h#4SSk`YQNPA0+;|kep1_Iv-3T1&j{^VD1&zXYkHYs8IPo|hA#kF8 z;#yTiUNMUNEdnR%$ItcQa;|5aPw61YL;aZdjl##E!3Wd>_2c@?N8m&~5~Fa%QTQ?f zC+gQHaHt>8i!Ddt?xXMk0*Cr>{ohI8MEwb)@T^hzDFP?zFC}oOAJ601-|rrpFJMmm zE)r4xH9-&5uM7U6`%K_O`N2^*USk5+2b``r8VBLyi}foKIND{UyI~Z*hrp2uEcrYV z$9?xD0!LMkCEq~cuxr6%7lc`T37QJCmX9?La-aPlCvwz;Rw7+C|{R^K~MD6UQxC1dhu9q9OtpK>Kimb&~CY?+ikR zc@EplS#uF`V!9eA9CK;pHz9Ev@;j5bFY@E@4%U-|_-=wcF<&Vpj?+Cs;yB%_B#zU) zPvSV;b^<5XlP&@$){}1pPOK-y@h7pK;Bh+4FR`BBehYG9J;BerK~AhEvJe0VH~5%B zKVBmZ@?pn|AWtk$8i5n*2`*P?BYe-!2>e4gm%U-x4mkt)vMk&qMH1wNjTLz}R}?-@Z(Y!?E^2*`=;VgZ2@+r=FMC$@_N$Y`jC z*e-04(U23{1r_+AgT+ED2mJmba7X}s zLS=LT)`wI^4<>(8MUlK(Az|@C@S&K1(F|pMA7%7l_7_tK>nRT9zX>?@!;vR@f6sB<*?@q8b~g>G0I1%3ta4|QsSf1>k{X8S}D zlmlBR2W^E8b2G3%Qu@1~5O8F>$mzpz!$|4x8X0ve1z@C^$)(EIa2z&pi~3wO!!;dco}ioXjT z(Ek)4ey3-o_?qbbAy_|%j+{RHKEO!jZz9W%^#Upf4wk<#_$QY?y#LD*!2wVhI4bhv z`|xZ!6d(JKjFP?*Ob|E|dG;21VV%Rt;Q9yWA0wsTi3U}1FXa$}VEiD-~j z^dI8GwbVx{zg>93GLfAUv>%@9aQfK($D^dLi7t+;#fG9Sj*oHUXz8O5lTeZUFnxFp z0msMbcaM^OB1#|j&qRmgWA=8G^s~?eOcwP+FnvkzkK^O?DIlC>A`ZZch1feQP<)KZ zV!ua8zs`{z1FtCy)0YDOI6h7vzFz{-cdmY%91#ahqv&17kHm2YKt>SgkYa6s|n)6t^#a^IR_M9X&4_!%V ze{y`-Jjr7}fGA6;@E45FNXkjsVZCoLQv6OdKz8~M@mb$vq&o(BEBY;@0{5TR@>Et* zQ&M3q=K0^R<>?VNWVI$MSDF+iC62;g%M7ew#$IkOrgFz}MpgF=H-5=zmK2Rq1D3!< zfvxmegY_J5pQeBSy`9FW#i2p3;HqMF8>>J}y?_9}0Cg(-j-@MHgObYn`X-o2QrRED zq`LV9P+57N#7akt<)Jz|`hrOaHPF*1Xfxf>*VQj5aC0aXOh2}Iy0}thWdi3@9erKE z%9vY%JOf;*j?~SbP63Vqp{yXNDA+N;7p$GB&Q>SmLiKPAq=F?cU8!^rM_;O+uPay^ z(?5t#bpwV`eZeHgg(?Hq&-^dq`32EOh{&=YeB&e_$k*AC4wvl=cBF%EqPkFt@-PXO zlL=G@25fcQ>114N7BY+L+oXpc51L)LXPr3)lU*KlPKo3Rm-7^p#A%@^ z0sPu3)z1wdBN&hF>Emi2>=)qe8W3pj8sh34M0W*S567TDxa=tSce?_W;^OM&2(*LjX~=+OP~qQDIm3)o9Ru8h;4D{>0%q=>&_as6Jvqn z7z>>|eHVh`2&Uuq08#;TcBcFJZ4O+3GSA9MprThGu!wE3hpVrBV33oKCmrOFz+6C0 zBY5Ey+mjAI@ESn>7sC&`4yg-pr3ZMrf+~U-sZcvmlt?@c*V+yN&c)CzP zaeymJVC7i~XgaR$aA8_lP@siVd|h2#0_{=jrl1aB2!P8d0k9>ozN?ECzlDp)4qM;W zISAwgd|Ve6Ek(wKBBQ9{4pe#i1$cr>8DW#0eO#y-tQs|m!s-+Pffvm1*fpHxAyzvo ztZ1NKcn0|S!kPqyT|h77Os9a(%p2BX@XSvI>}SCOk=Nj;(HyP~$?hI$j4E18C4V%$ z=uV6INayABqhMWOuoNomW(^kA5#{XBID00XJsbAkSAGH345L;@6MmK!=y9NxQh|Q9 zFRB6T*6M__e6>;V>LJ`69+$?5+5-co16c}I2;eRbq}4|48PXX7IF0>4a&1%r@PI$c zAXpL_E+_c}^!+HX44!P+Wx>uGjfo1Py9tDFoH=XtXvS}T!B}{j2OClL20qjSd~obQ ztButJ8|c8U|Ilt0i?L`O-eq3v1iPrHm{A>Xob!#_jXCL)AiWEi$LSKFe$mn8sUyi#fjsGWf@4NJ zo=PwYbUXvWvF?M%^F6t!3C9zzm#f@|E7XX)?1gQW`-ZP+3loBFs62P90A{2 zo9kcsH?TU@ul$>?gThv5z@ax_5jfy{(wMBMGzQIK*b)0b>bJrAiO1E3ki!2_zdzPL zWWPT_J3JUe_0jki$XN8sQ7htPkCjvOBp$X-^Bpz-U0 zxe%J|fWAfaUu~@Nql>eYa{4`Hn8Q#{K`2U={aO~veQ*w@JZO$;HFG=7fKko%xFynn z7X`2e5OgaVGmyuS@eWI`1>J5X>_Z5rbqCw})))Xo9n zv@7OzG}Pdi1nmWT*aH~)CK_{W)Y36@VH(o}^!fx_x|>+?{$SyFY+Xzxkft}&m^1;R zW$fZ47?{qPZxZYPYstUcNM1i4mKAGM1x^keD|KPJCfTXawlnB$pN;?*{ROlZE_%Nh zmO5b#ch*@Ah${Edm>~k}LhU|JC@;>8mUXzm`Qd}80sEY20ZwhOIT+HSb_h`DdNjr( zct+CLrM{5v!KgE2=+5C`+*ZC7^EV@nID8hIZk=e7!QV`T*I=%mL6BG?5<}U&q{@3zYDT|1KWO3vMl?Jfc^Z@ z89wPsEl1wgsh;0F_aZxA~-HX#{GBs`Oos*_h3Kp-{t!_ zMa#l~@ld%DRvVCA>Mrm!=>1bc>zS70m-t2a2Ev&){*&bWzo!H9*?X$R^d zyXF4>UZ15N<0drnj`A&(X`1s_bLGaNnTRf<>px|ghn;1uja6EfT8z8Ow}xjMRWHKq z=6|IV)*tM&EG*4oBMN36pl_U0KaGjcDwnB1md0oU#|ZR`G{y~dV3?Mm8+;4v%dn}_ zumd^N6XElY9jk)E!};IUPdnvG&asC!v&VaIkL~`G9~L{z-#i*4SXTKKYTWEB;I_h3 zHPqky=lT!YKhMxvgb}=X$OB9QgOt_8GdiojLj%;nL+_|ug2fa!wNDtHxcm!Fw9ORGI0GB%138C2@I72AavWkkvxXIMvX6SK4YbFlO zewpln-wc;f($Gn2WUS=RTtCPgRd74B(rbt9~I5A{8(+rtDG){f!o{7$ToD;es z>;h2x=ZJw*YtVBL*#ty3;m~f$q2u-6#kT?RZT=7O9YB1C|F`j(I-1PD&7i;;dYVif z4~Cux`v`&|rVbc^p^LjT=fDu6Cw$0*$7WbE>I$5aQ0TL1%yPJg8zc*JMiGn{7{-js z0l#2`GM*6w_h9X%ae5apzQO5t3H%)#1bz^%{1J?oIzh7o{SU0JYoq>72&OV)8$hB_ z{Z#tosQw9C$Agh9IKM>w9YdcQ^_PpT$hZW<8K2p_mL&}mq_C$#jZ1k&S=UIGng&2;g!w{B3IhruG?8qPiu5Y8V1KYPp?>4B*)E`l&LgaET>doMAgPMq-aodU@7P#8P+kaLg1;WsZwl84VcjKu$p!bj6Tntb&rdRUhqIe8nhAC}Tc z_Kd8j6i8Eo#AwVIe~`a!K|+7XjNyS<2F(~;421KS7z)WR3Zt~){+AgeKnkbA@QR^u z6X-uP8 zc}xHa!Ulpe&+4B5f~|>I9=nG8p#K^i7aWL21&l$OGj|znOvB)dq47O??#b!D&|LKY z?8ka=IgGX)x}OsWR<|2=$A}eWC>%c7$i8s{?$_9(dr*Y?K`@+sWu2JT@c68&@WfyW z^I9VC4|N<`)>>9@F#1P%5FEN*JZW|wn)kwSJes64%UO}xV|q@dCk`RO=%Nu^+~kDe zj?NMcGDZ#nm+EoPz!3&lwMG^Om$XIYl^tSBF{&P_af7)JL9w<>7$&;ZRzfbWA98+Y!@2 zBbsgrPC6z-E$kg30X*~p3AB!ofDMIiGhza0T+}Vjp6`J=Hx!$o9uJ>@`e#D|Z2Q5r zA+0crv3hs*a1(-?Q2P;T8&<=ZLd!T1*>u*^!~63+Hc~fekD9&#a-?u4qLq zwl$wuvA65tIhEyJS+Td+ACgg?F*)4g=H<%!Pp2NZ+cP(8Yw^=L(a}aBG2Aq5>Kp-W zZ5q#;iO~XbV%Z*{Q$y?Ra!+5-J0}v+@@$QrZGp+nM<;ttyKeS1PKbQ8$>+pFuX9hF za#wlgT8EXt3(Hs**f+-eL;3IDTekjHZ4du;_s)|0#VP`q(?)DXIOZJKRF(vx* zlzi{~Gm@XY*?E7;ym5bO?-Ui!Ib~v*6d|x<@y}25B=m1eW&GJ;Hvb~mjo4c!rQXcx zpBHKTc@}sMJb)4rs?n19m7`{EIRUGh4CV-AU@^W zVVlw!{mB*|a__u3t9AOs&1(q@DnIjXru5|M6c|`Dr=}0~!9{(oKK`12qN@vmdswoouv*cEKPVwkBtb9ia zpZiL6;?bN{>1+H`y`PJvzCH8V$NY)>5`OV=U*noHc(IJZFWI&0C8NCPm$)H7U{0 zdhG>5#a-6sIfrgGn4MCaa_6&7is8-0s#*{8dovmu^=;3mJ`X?q%J;`wTeqLNzwYQK zZ#cT_Z=j9jW8c@z-&x;k#vCo=|J-#W|L``6qx1U>Z{~{}RbRQi|9ha{g66hG-l<)i z_HjUf9y>E{D-De?d?AfI8Gr{(F$p`Vr&$B~zzqpg_ zvhZ4H?yDJbQH>|2#9p0V=AN}fm&a|n!RNQh%xU+j0a~|{4^FxK{ljz(@#*Unxq{N& z{iPqu=qf3Ud2gesv|{2S>xQM_H9Qxqmj0<*yP$i~V-aPu$MMBf4B~zzm zM=EYkPLhs}Nm3DWRMt|~UZ^$khJNHZp5;8Z1$v}K_Z1z^n(4SoYut@7k!zx>xTgvx z2wF(3Fi@KDa;eth8wQbL(W=}Rc+Ltvl)7rH#QTyT=r)BP8PhzjOuPYG_|MZ$?kxltJ>BQjBox5I0g-nXraQC-@_O2JPJa2;A z-U*BsoS9k^`C~PY&O~Es;_7WjSBvWT3q?F@`fVP)g?p6_)xP)d%wusc#&HL1_~aSw z$?Yys$1Gncy=dwmp@*u8(yxd>? zK83;`JDxY~ZIj{8CBeFDq^ZH*j|1W`()L@|$|W5-fPkd* zpOhCzdER_z+b8gM*KfhI+0q-SW(-9kPu0kp6UR$$wwP{h zb$%I7`SHFGC%$Xko<6zLRip2Vi6yp8@;S0v_R#m&PvREcGdV6%r1Eg!ma~ZKG5wEv zYYRN-w`-pY6~!%UXfg{AYzd?Xv)~{T(W|uBXVfE!$kP)bC~#^ctjT2NrMYD7nb@ z%O!vD*zRVX>-0kry<(px<}VxDeSv3D*L!n8k0Z_90`N!G@xk#>E5FZC!3*c5Df-BL z*UyOcXl!<;Zk9@81Pa})+%}a9{Mb=`MY}naYy8$M14@cjD$;%LRsPysIP-4hubCD` zszq%PpGB{4;69YlYaH#)^H!?iSz)l$igiko5p%r;%z3U+OSNLPXvG)X_Re28RUs+Q zzkEWsPVvPZv&GifO&gfY-ICqO6!2hXFT5EPbxHP@%AB_BPWi+TX7*X#+b-|*CMloR zmvgXqMmIY%|^_AKG5S!-*~FYE54CCCz#V7d=oDx!b7Qx^a=w)aMC@&+Ih} zrharNo7$zAmr1*lzH;rAHF;Nx9Ql>DkLjW|DdtqwxPAO|BK*zz50kI^SB~%N80S#q zSjS(fYc%%mk~vE99zEINI!%uE@;4^=r7e0NUX^S5PCPw|EaN zdmkRtSnzH7xJp;ghd=eD8Z^J{P1(AnLb>zpqK5FBY1(7&T1OVqw)FOgs@$UYt7cLY zxc)|T9obNv@OS2?Q&D%ke(G2p{9x%lZRZgS!JQS0w)e+UP4xcg7Bd^V#k()h6|ReB&c4H&UEq8QLlQK@LjG+A?I{5zHMFtuUb=CLu)m~*;k>nLS7${e z3G4cs9XVN2XsTtt+2Qjs`u=(15xZ?#sN4ZHB{R9z={X(JNewrxZ8!1RXv$w$C_cXM zs#%jtcYU*5WS)t`9f|8t!sgr-kZn4{|J6jjhrdL9=dwL}r!*Y6p;YUcS!l-i}#vwVq)hVEpQCPCZ8^n}L9JKuRyijH=Pnn^YtFIl0gpt8VSx%<_7 z-<{R1RR^N)H5(2Zyz`n{_N%m7H&`+?z4PnqNmHNi2(P2Z{kgMFH&tq)Azl2HOSZTG zf79AUmZ}e8XuBA`x+>{^nZ7}3JB%Ky8(upWCXrW6KUei#?D=AqOr4Vsza;MJ8eYq{ z6VF@H*|NLKs6|O7v$jMr?^2?$(#EV-THd8D^#fh^WuMm51w9uuE&u61=~>-(5%o1n z50dNo^48MR9Dd2K>2WKIFNy8)d-K@vQ_}!n)AX`RmtVVlL(*2xY480eF-w73ER~b* zXO*dx{JMPYFX8F#p?Rn2^Ip$s(SKvQ;ViGpC$8eX8D}k4%?MF_`o{K51e4Bo(uBrlMDQuDS)g_O~lnf38d4KbJX9rII2yZ?;6l zqs#GBu8%WRRRlZ^r|oAn)ODY0=MQVLkw3SpKQ@-UZ!n=C+zv4PoEcAK0l7;Q9 ztB<2v>o)9oy_P@RXZ+n$A#Ts77<@L_;P1Wf@`^BX%^p=Zvsdm)1JNg%i~0D!vECdtK(vla*L7nm~7?3LV|mz(N#*z1Ud8x*K}TmEgkq|LFsdK6!l4 zk)->I>%6Ha6s^y->3lW%bpGo6G*!iR3!@veZzbe~AO2N1xgdLWb>mXI^Xf-yXUs_X zw)@c6^zdSCmCedxeaYdin)hrICwZAjFOh8Rd>U?d+|4RK^1x$5pNHq~@3HN;T994Y z5w5yz%8D}{Q7G_)f?wap$b@VT-T(63 zC9$sDB9nB1}U!*3OD; z8`zVQptJJodzsxb!ThzqmR=OS&sV!`{`8bLRm$d$kNW55^_B7qpYL0He?8ZrQepq` z1;5qi|6cawSA3%RrL`xD{6zxZi^=HMMwr;G(Dr^U_$y$_F&9;e`TX^#WTzfm;&b%% zq1Wy4Ki3|eE7N&eD0-z#)dH2m6vxBdH#BWd)J%8y`QStBz`9^9i{F+(DW_W#M30n3 z=sjM0Axh{&%jp$G=G*ps71gPF_^FahuQUnRC(Owc}uG^Y{Og*fh#aP9N#Ql7?cHBLF`J!zlG7o3RyjC#T^3l4~-0Q*{AF~^h)p;tXr!3F4 zl4o=#+4`)}%@3(9njU19|Ijf%$F)MWWzC2EHwHc?Wx6>w<)|GJex$!-jdpf}=fJ}6 zl6e)|PBi=~(zf3pt$VC@U?JUAx!WUgk`Ir3$$BB77`5-$8kQJsZ_3rqkDdN&wgGLL zdih|?*5slD{Syr}McVZ;z(nW4$r`Tj4@*r~pIz86`#_>BU4L+LrtNKYw>2^MrpPav zGm)X~-gm8Gp5b;Qvtve%F>MP0IsD7Mv*29e@%3^g%L+|W=ggVp)3fuKy=w0TTFk}h z>17*1@)yHz%(!AHSG2gWa;@_MDN_gUKdNfRkCOXbpZqx`dcjZ8=uxt5`ik}Y7L6^E zr1HN0AeNzBWvmnB2U;ed*u%?nHqBi7Cv489%_0jQUTrrzSA8(LUwqCc4-v_{mlq6k zO8sWW@i)b}elbyS_M)xG*?aJ2@yC|MQWbI4#TU5Nukd)hQghGF2w9Qsz&)k6~A5{(ir# zR`&g=>vJ?_gm3zoC4Mz_{x|R49zC{E4KkBubNMz>miM=2M@^0S^?hQ&-_`^tgMmvD z$whwFVY2O7`xk1x@jezlK|;o?A}XQm!GZO^>k981X%_b_yB)p$_sqTiF&iUaPW)`W zG5hV$`lWpKmt5pS6D1CNpLvPGXm?(IqCNsh9s{A{CVbPOGldBouCiAs+ z%&vLC(E?juu`k_QPvdyn<}j@r#zkhagsInjEl7Q`f4+b zr{~?9`{HnJF>V`R7fm69?6GvR@ZW6n=U*YvZEp4-B4gGiN%_ zK6$0iRNVBrUP4}DU1^VOKx@^Zj+8xaKk4yq5tlsW=RQ5uA^u*$-J?Kt(F&{L4~HL~ zbzAS?mooQ(!h};r>xCOK+)lM{tq#%+5?QLAU45F@J5pTwP}Xw&sg+l596tGG?k?N= zlg{>*TPk~H`nN}FOtd}`yt;7P52KoG!3}##n)9nnr<&ICt+q}o-Rs!f6JPM-s%Y=- z$A=#qRd|*Eun@1m{$|IcE6c3PwN9CSa$Ei8v{te68eNIQ65q0`eRxg+aY)9(!TpYjR6tK#JvemBolv5#i(b@@!2=)QG~Fsbt$%ct(F*rBj( zceBZaO~12b{69{X4Q+T>;(RH%W~$r0iX*lk;R$CgcarDpTajHLfkQTC8KOGOcU%uJ-;t9=D2;tWch% z|4E@=tNPK2KWg(n__%+ttz4%tD_!jJbr=`TG^ z33}Zv-5e3L!1UGivW_6-(_`gVO1j0AYVDFdV`6<(x=ecCwdR#UVq0&@zfaqp@Z6%&b8*0`Ew&dv&dZjZ!$0xkyqLZ%+P{l) z26AjY4ljN2!*^cu?S01w<{ul_dTiiuZ^6L)=8I~1XEr(NF23cbuRHsrw@Fg`wsYed z{W)E6W;12}mPoDASQCCpuw6ptuhDE@ourVW@0;W|Wn|1-VpHpR^4;<5BU+~G8oV-A zOIrS%***6<<7?RXT(McP&0}~&UMw{{KW>`Afr3G;W7o_(C&*73;6It$82Ur&!lZy1 zN1_k?IQcv$^ep%*3{7oVq3_YQ`cUu*y#sGIzJFrY82@vt{;l9cUq4!OFe5U((lh)5 zlfoPJ=~vvn_*LyxM{7>_spD@BYyY%5bv!rxRP&@|?LzzSFZ}A08@~3T$_e+Bg64}y zw7$eh2glV7Eb7=qJDw4^&_aH8U~t^C1=2r!_I3_rn!ep*{Cnp0Ek362{vX=j1D>nz zkK_LsDU^hw5L$%nky5ratx84ND>I{zR0>%si87KhB1A%4%F4({GO|KeWt2kef6x7# z%kA|2{Qv*of1JnT`kec`uk$|Zea^l2e0=U*us&VaK1@0NwZTT#7{xXdi>e3W%C`5s z)7Dka|IV?$Gqq{UzRuLvSK<~69>u%VI49rN&$rer9qr^=8s2(q@3w<2w_cq|Jbm@x ze&O-`F`Et7nkH;GRB4p9H@B*rDg8kBkwGsp>+HR^`?i-`=vONXO%>12JeIr6JJB#x zp(K8h*3B2Xb30=f^;u~Beo&g^>%Q^T#z)Wh1#-ViK5q4>+y43J#+O}zi~K*>KCg^Z6p>`?RZe04 zw5``x`DD?Pr!CDdleew#iDpbnlHuAe7&~7!1>K$Um1E0->rq=Y>U8~@E zQOEN8_v^0clc$sVd!!rRxGm5yy?8+Rd7X;Ng23O0IO-+0vHO_P$2k=iYPQ<^C>3b_ zmZfXeR_dMgD{ozzJX>38!J&4Ai?ZvDE1BIZqnj$!=3nZ)CNh>YXVbFP%N2@iw*&_j zG8Kh$EPVSd=4QzY2a2w-mGWZ)9_;9ni407*b>JMw!nj9$dmn%53-(MoEZ6o~;S0MB zzdUc&)z&$Gw$rio^o+aK%EaAVsBHfGfYv7Gr|aYVYv^t|iU*dxh`6=3nW3ZIR*r=G*)-&t(PKj+KnDI?bIgQ?QGtd?>;jT_%Ix4ujbY4z*W zdDkKF?NP3&M2p|(6xWAyV`p^P*Yj!#K6K7;PM%-sY4=TGOu;kz7Bf$%>_|bbaI(f{ z`W@}&kGNksDpGYaahn6U80y zM5;L~I`PlejsxG$_1WcQ+T6N7f1h*^ueEFN&yA*PV@eCg#S`$qI#hEM&uqOq;PB~tZc>hzk5M{8SCxt}LH`ehz; zeIz~obvR|!uh$%nRaZ-d6N|npjc3O`GL>Z1T64dqz-YzK=L-vh10rIKz3*_p*(Uc> z{ZDXIa^l77hhDAP{Grxq`O%-=pWnEZ8)e;gR$-OwH+Fj+@Mg4sj?Z&;G5Z_hL5X6m z*K4{Bo?kb*`KOHO5NBAZZsxngnklzT>H1F64H<_e5#QGh;Qw1c`x}3+c&FXNJ{;2Q zo9NWvYDYITOE?${70h|8U%#i#zwUi_)_2hcAMZH=&qfT3YGWG8WiK6!K0EHBcuCg% zL-o}ho(DsBL@$U5jR=HfuV>GF$F16-)TL@(;&-<*a_gx=4@W~&QMZ!mllF`(bcZwr zOCQGTJxgI^5RVax+OoyJ&B(2|)0|>*s4#>s<(@)f(U4^PO zdrXd*M%c983S8RlS!N;I_UWvRLTExJUtwwUo@E7TgL3mNuSq5A7;>wIKGMFg#y0gr zef)Wi!tnZ=58srj7LK(G>tx-Q`(4mDd1&6dBUQqsZQN^2jkI5G8aF=hK_Xl%T3?6% zSb;~4;WrnNTccMk`Ymp_F+NbVj{SDLb5dJlMR6HjyO-4M)WtDhw2u$0(%ExDEx?wr(SO4smh`x97iPg|E%ktWInMKJeE?;)1sl{0@ zxP15Z&5FzKo9%s_w+1Q?>*e2Fl4w$QRd@FyuWcFhwMzn&QjQ-ofAo}Vs#V*GZ+COZ ztBDl{Sr_tT<=Su0_1o&vFPz8C;MGzW?VS=nkaFAds}H}_kMH46be(@>G#(Eq0^vG>cYQBCj7DsK1+kx_lp0t&)6<%3s;Bi#s{~g z6+LfmHHkDUSlw*Vcz8Zfd)7YA_Sp8HaiR50YHDXbsAX7i)Xb5KJRo`5Vv)I1z5SQ~i6RLR-() zvE<`xRMLazeqMWnM>5uIcdc;qV#y-a-23fzY;VSurntIO#BMb)mo7ZBP-C+2ChMPF zb#vSbwii2YTN{6wX_RZ9&cv%sn^SMXjKUPY7&)HZzSsBc_f*yTxQ0uIlLuAz_P9?g z#ee;xmb*99^6^TpW0U8ydAc{HGxZ1d4GKp4oi*fFc^=_DuYBqzSB867FQ0yBPYJhX zq7wbFdYjvR3?}lQbFu&HH1_V1XdN=Ybi6QV-TC8>nOfqWu5&)P@5-MOyIM+R+HcwC zpS8{1Bf%>@Bf@1$U7|K|glus%F$&95R38i6IS|79 z?a^BgzWA01sSAAi_hX;&bB0I<+-83FR(vw%mxGpJLyHU)Uw^t=h}qAJk$QSpT_8 zJlFHc@5qjdvF+bvU)U)QuP%^&buuN*O}RQ-I&7~|mgAXI(}CvqUTJBuUDH=|uTWck zJM4?Jk2X(uoauA_0?Wvd1-dm??yg-I^m1G%S60LEiW-ki;n(n@)!SAEHh4YH#2&3I zu*3ICaEm^+JE`kjsxrKYbn3Nr0D0r z-PNGDKT2<r5Ny%qBq-rxuJoTEF9*M)&S`skM-!2<#aC>KEcwlhgbv<*{*0}#(qpWN zp^opu<`vJ&t^IwkW(OA>+xZ}5p3%ayzv(`+4J3w$+z1v~vFrQ-i8E3Q0&@>M3@fva zz7Z_0H6gCWxj%%(f37^kD?DTs5F8IX{ zFku?5RfBu{roT8q?J3sH1Dl39ebC)jV&|GM{VW7g<94qHk>@H3mw8tEf#BX0D4 z(#-F*=W8cWQW?yqy~k=kD)X92@QkZYgce@%J4%1-`)+gpHMR9!x^3svKDD1Mcis^4 zeWhR2a+X1%CtMBNcW)X0dcZk?r?%pZ^`VQcQ#alx>s}HL@8?S26d3SEs z>>E+ey;nDvk0`i_zCChyS!!xcyuX72k4^2|w^MXlvN=r8V{@jumM-a~la{=GjwwTM zcR~}^LCwKmOdnV3zw}|$Xe@}eh={BVv?WiPBza-D@(b>G5?0Y*DqLW+Z=j}6RO3yjdOW*rzVFz7> z$^q-ER|DgBJ(8Gy<2n7(=~5rJ;DWV{7M1?hU+zw2-~H1l%{x6OG~t~BCAa&j>+wWC zp|?G2_>YAawAc82{hqP>=tZSJB|T?0c5>29pFMQ06Tc)H^7yxS8m&Q(ViJJXphi5lFRwrWJCprS9V z*Gz5K90NW5lqyi&3&g%xd`g2P3%wfG+^!bSH*@lN z7>if%>{z2~CVkFqRF{J>R52&mXK67jk8;4Voa1w(=Y2McE`7V%FT(so&G0Irl5Ru$ z$tl+^ zZrl9*plizY_PKvLf6I52>>sHptgbpy<<6XTKi2Zc#HAnn6%qH(uaxlWdo2Cy^XtM6 zWfykpQ@)6DkHcpc5&K^8;dRn%bo%T}T`lspPfNC0amMM0++(@SX~(6*)g02wJiuss zk)2y_eE_e{tq1-(lYiD7YT7kXvt!NGt7(!JC6&Tqz8b3=w}~Z$w(9guCpM_|S9u(I zb6==fGEw!oz+<1TfJ`<13f177sd~)I)U8GQTb1&A4p`a^-72kXa{t&F5LRjC$SmBh<_?>MbDPkq-wjh@J=vtExjZc7k6 zw`ePWYrn9TL0MVc5r5`Jk;BjF_6fT(&}FA6O}>3@@K!ult)-feY0*%Wyv`F&4#6Hl zkA?RaUuZqEnXYnaLt9NuUrUvom?QVi{L`MC;n(_6dk(hT;2KD>X@9Be zX~9?W>q3RA@iTi-ipaVN_s{GRQ4-fB=*BM050&9?bLA>_roXe&Ynch-ab|HDc{2$G zS>6j*g7&y9D%)*edM^D=l#cGZ@ba;RE9@>e?h^QVmw&s|^Iwa3%KR$Ms6|d~uvp^V z@?!a_A)fbxiHgdSi`HQOZlL0!67r0RA=$CDw6xsF;Qj{B4Jq4O(^X@Z7!-I9?d+KI zGf7qP{s_a^R@V{L#YI6|`R5*68ri?7b;Ypxsmnz}#!nY-VX$9k~_#_Wo_ zk481dT|1ieLpt)ySBghsf4;WwGD&^M6!GJeVYi$tSgf>bHRgZ1c)z>u&XVnqkIJo0 zwOnr!C>@^?qkK2Gg4r_7NPhX_`XRN!;} zUZ}87T-j*ZS5x_E)@wyaI_J@M6d$r36Inc3vh$gsKf~|pv@ZoUlY>I3zf-!CCPKpk z{Mzfn7liUHw`h7^xN?{`;`8d`mkva!*7u4DZnoBb(K^MxX#Ty>=(zeT9~}*+`ppiT zbSFeB{aGgUv3cpY`5HXmmoBMG&Ev9)QB|gJ78N(G7&{uLSsA1AY2^`p`R(smdk!e8 zarFGj*{=M0&BgK620wH%_xHTmw0`sSA`ge}J!ypn1tRy=Iv8%8vX$NM7}s|aYd-bCueRsN zQ1UtvJ!ijrx2jslL=?~H<*t3c?(=|$H@~c?#Z8%a3O}lXt{fb4iIY=E*<;1MZawps z^X|FQ6^%Ax%l%U7L%L%Z_cd(bO?ve@Wwfz)@2?FxA(i4`VI?aHymfl7vx;3Aaov{8 zylU?X+1)q1?j28Qj4CZIIw5)a*VvQY1#QBm1-md1953l`vr#^LjBstRJtH`Pk zs(g3yx1X9|KL2OoW$vUyiCx7_cl3Aeea`#h%w$#a;WtsPyXbsud9pLd_Ydu8xcrK4 zbl=vI`rnq}{EO>SmKgBx)~~H<`cpp1*b}TPDA(|N=@NrN^UiehxUb5c786Q_w+G*u z9?oUqiFtH=x>?4GGflj-B~*Lo;UbydAAub;qU*jI2{kXg@?x;)rP00@SyjP&WrwFm zJV`$S`;n;_|w^oNZFRmKO{HCCwsGykO_3LE*^lj1K-5IjG z$1gH3%(fGDP^!=!{_|_!Ek6JLaUrimcHLJCMjZ=A#SeHrZF}C+(qS6Z^}y}i13#~M z*G#1%%zPq(2Zhtu9x3JOa%(md2_IVgfi0uBV)-iZ`%!ZgZHimIwp$eM+~+Eg({-M| zVe?!6;rI_J+MmnUWTbfnAJ4zHbFP#@W!janD*|UF*ql2K@{_O7sHTa=1iT(bLE~b^|S7|k-(Kzq7X$D3hT%VB5d&PrQo<|L$T_=xi;@dD8>bQkzyZY~v4L3%Zl%BP%jh8j?mxy+%%~l-hj5r@# zzU+kdjSWs_%+o)QeRLG9x$$RC*9kW4T8y|((ZipzH@oi@A8??}txYGgG;MOOa_*B`NxC4DPdfy*zZ`50;&7Bw-bGIp@wmPpe zL|tB^lkLfd=*zn%|9D>ddT=lAfg=MMn&~}y{oA?)J}e9<5b7zCL(&bG?go#4px_Uhhu1%lhSEyN=6cl%=hi`mUSrt(RA--sBjU z@x15d_r48#R214=2KRmv5HQGZtNj*qu%Wy1$zIQ+^;@?VZXI~`nf=7Kvm1MNriE7? z+7!%k0UpaKV8hLQ#5FDa3Ja{<2T95UFUP+PVVFm+LN(a(r8dZIWhjB zjoqI8tM+)WJdAC5B~+nw<#)}mr6b5i1(%G0CPcP^Q_ zWYpw{{r+w6Tq;8}@!|ZAO|~DzxzFf|h2I%3E*N}qSSc|*=#AsGjm?xFxN_9Tdt}D$ox! zd^(z!Pm#X=#JkW|H1@zHZdcEn2jTZI8v$YjrKMbMx*$FPGjF z%xQ8U{`zu>%K5s_1eF;zdTT;Ica-vT%I?$b-u`D}6#vhwwocJ8zaFjRU|ap@?ds!( z=NWfgKeNPZ@6sC|0^Wagd)Xz%n0d|7`j1?&kbmmjMM1pBnSz@3=_bT7bvHIzEDI4R za!Omeb#zPG`1i*w9_r@dhWSI8dFs1PC**NfmnaJgysXKNFW6}tTF#iXZ5O>SlatzL z?Bz{#c};hk+vZ;mU(UUF$yeQxP4p!Rkvm$Cu2z3?GG^!JnBmnUQSm|3jIVXs9e1QW zo71yW+OI*&;#j2Ek-B@a{w+#cgR##7a>R$n>3O^yl)h|ry=|GM_OUA5BDL7kqdHJI z_*uz1-&?FVGn8LQAG9<3lP#crm$_VDX*5_*g`RDJ#`%h4J7gY~_3S)RpZ}!imrb}{ zOKp*af+@Z5yaRplI?{ZT*96O}x9#w_ac0!5q_Updm9t$~BKgX)^+W6vdoC?*mH5zP zHvjy&i(levnJVLQ<~FXs6L4jb$JOsuhK5@=8=TFFw_{UrIgO5vgMoS5x@nOpwsVB$viJiZOO;>BmIG zqZX<CVkbGbaaip)=DRTTxoBNjAj6#ki)_EX?#c`5Q#PCUVxDjD zG|A+{^Qt}xmXgi6!7AUx^E&%q_+&RE4)U?CWA9);RrK01^)S26x)hyPLCej0H?4G7 zULfZE?o@rq>FbP7_NwGnTdm;wtDdxL9y+IL=C_eob?TX!vNn)wWa3l-`H0I)$4yfP2k;*B<&y2aLRVBC0cypUO|VA1w@8IJLn z>MpDrS!Mr*=jcl|ABuR@3f83`INzEwzSq1^zKUJn{*7Gi=&AeK$`#|U+pe4pT`uCF z!BG+rkr%JN=KfWRCU)0+qUm9cXoK@Ew|P%e|6ID>p%hdlxX+YRBJii;$)y=A+V$0y zT$M{wlQlM^fAcOd;V(#U%`}Q&z9+%b<~MqtFMT<$Lf}^>eetVr>K%0%tJVbmc~!F6 zmP_Q}8d2ekscUCe#O=i!Xc7A(KcW1xD z=Hc&g62Bh5`~1z|wtMyxF^o;a<}nJSw?l@p!$RGfpqzl^}pLRVx8akd zav~J@Wj_Rz^zP}4PtX?QYwB!@6Bxd-JL%ofOM|bCtL~;4$yP33pA}GSzFGI!#$Xp; zQ8m%NE4TM=)pc;#8hYW@m|*sj2Tt`G@f-E>zZ`wV^deRDl!9&%C0+A~t$D28$!DpN z`X`)E0dEoT#3PoNTQrNEvw7C_;HxHhuQg zxeX->Cd%&b&ISECTF`Mqn`gU?`-Ps^Y`dhUcWMSkWo;oF*&luVF?w_G?xi~k#pCDL z?D%!$jw%JSU4BBT%20VB+@C7SSP*~O$}^I2ciD>isGp6wzIlC2olE8iz3Vn!?7g?Y z?pu`X4|XXkNJE-n{VX=T|-r-WTdC^)59C&%beOKhwAS+m0UH`Ow#H?bg*k zoK@eqXG|UYao@o9iQ@^nAGO_K@()(GZ@FRdZqSlbe~DRi__C-YmOC|7_g>UD`Ee<0 zm*_%qmyO{CeNLh3LuZ8w)%QOc5FKjUy0u(x`6{80cboY>Yo7?J)6$(TY>l~KdGG$u z^-iDLhZ=)k(V3oaj*!{5Ws^!};!tJ~+i8~PA2QcHJQ@43V9@u=ruNMtFE`&XQvKx?HAEpz;kNZpF8U{g?Rpi>wGVD9C(*4Ud z-{z_LA@hFz+;nhkwCBs=FFi+h9P~6#PFyy$_^DOn{?V~Arv2G5Kl-W{m;0WUc0V)` zQdOnJy*l8ESm?p2s!t|2T#U7U7sMXh8AEaYoWJtAP1~oB($n$DW-WQ1H(w2yKS_<7 zW)loz;<=L)vPsM&!B2jJ+YPNrgD6c7?ae(L;kV~C_KCYX*vYFuambo3P)dCL{iUUrR*mTRwM&-s)9Ni6^y;nV z4>xEsYBUa9&dqKQ{mLuU(YL_Hb_svLsbUK`IZ@VjgG$R4tJvLn%5H@|icnV6`r2HU zbM@(bsng-LXSsZ=!*^y+FTB*0yy;Ey{eI~MoXlAj{6bAJV_eRylcyNYsmC~ssy|$y zCL+7TlTNuRx|NxEBDei<*=<>q1{I4V-nUhR8cJ(quXi0vm}ZH5b@;ItixAIJ!_bA^ zTW(!jX>$6-^#^K-bn}f2?Tzl-GP37CFku(})S__N;Y(jUS(438x9q%1*|=o-O0h>$ z*~fQ+YHPTo^W5EPIjhGHEZL+b~wj;Eu0fq zFD_^=O3w99*KUkZ32ECkBpkKZV*Zhbw~yM~ z7RW2-@1MgSDKEvp*XC_;&g*ohH&-+rDwJ;ZC8pPOi4=UgSAEW;HtOL)-7fotH%aMt zk3TMI%jB54D_|VvyfHDWe=wx1y65*t^PK@o8AG2r3IkL9kDQehHXWub(nL-*s35^-!=Mh8Q0|54m|`GTcEujuuqfmwl7_i6{v zSdTX_B(iW@2`h~ZcFiux>{aNoCph)7bmZuf-JVLn6Nnv56fDhq@Q{OZnJN|N5WqTKh? ziK9Ps==Fa6+PF{uqNd1BhvEo_tPm@j$9{83;p=1XZj9P- zEa*A8&f(e9vXc*&T1>r^+&je=tIF8n^GCHKl95})Xjk0Z%n9br9&0(J-Hxo&PGTN& z=e;Ytzcp>`@&h-sWsmY`9DKTK<%KDRA2v~scd_d9AK?75kb0PwcuCy|TlwDl`gd3DmhvsTDjb#X={Kb7 zU)EvNc|R!~`1Ky=wo-QS{CS4#Ed~ok(sH}1E&cVFg`ev&K2TQ_sDuF9jZN$Q&%W-l&@%@#90T_vFs5 zU&Fz#G^cxXnlh$7yzLwpvA6mxE@!Ew&nqB#J<0Z5KeLEYkJ&n|l(@a_U6nC=hHN%T z{2VmsWRSUjvp2p(HZO8fpq}gTWnrdmneTXyzv*PSMltUBk+W5FmB7@|0>2actT|6E zIMR|YKaYzex#9o&bnW}?7ZWT z8+#zVCL7Ng22D0W`mJm{7*J!I2cEp5365i7703kF?ILsoENncQxKNE1e}|PTuB1@v z!6&9YLr-7JHjny#t1x)%(d@B&a148(2;~+$248K1m+diRax?lZY=VdJ-0(EunO`8l ze+|UzG)hGjBk(y}CoCbo4pAqNA9#)r2t-=Iz%$T50PP6^V5*R@;)L2}yCK^N7+yyc z7PfhYth=$wfHWTPIQ}GUY#%W2uq+VZtDkTd4;ftg#97XNu#*2^%?Jbi`4KXIkkQ|D zw`b$g#v2Zlg$sTMz8mPU02uKuEBKfcoDuJ*BJ1>ETJXL!5P*MJL3SDdEF0)Ypv}!N z_jrM2Tn73X_}c}j{jWX+$8+Fubzlo`Tg;3Xwaf#X|yj8Y&?Ms+H8Vn7#SAP;hw6Fi9uTt@6I}g#Yt@ieBXia6G7~dDxBvEfesu! zfOWMxs}1&d8}JE1Y_o!z5iE{ttg)k8W;NhFnCGBoJJ8@|(*|YR3d*L*Cgs8=h?Nzq zFDM_--~S)=1OI~_v^@`Way?j{aC$8^o-oi>fsEVOM9$37W|KP1CZfS6sK&;Fl^2u? zq|<}xaG$_Wl z;Xa5@Zde-lswXVow9Am+qQUh}W9Sqw+jT!Jkyj|2myI}PTWK@Ig!XQe|!X|>HooP}HY|a2} zjLo~Sjlo#F2W{D5aS(vtS3qV7eA^ZHJp&6J=J`P13y;AMydIPP!REtCk#&NQLHEJa z%0UL&2?US_@vbrW>P4KHKpmO6{zuxg2vf!EU4g6@)Ms|C(q|K7kf+1k0Yli?M}u`c zDQ--PD`UytK^uNp{BIld>N8U9zf5c=EQ31n!GfOx>9GKq6lC*pLZN_sG$Eq{jfC}6 z1U=Tb=NZ)51j87$*+c??&!1spSO$6*>u~HKwKKP1z5&MaS(rX!rmmO+oMG6))+NYr z8B})W3`(6N??L?*PzHPsbiq7Vk5wD#gc!&NjP4 ziZS3lM*=X)Fmty3`xpa-^KUStUrd7&XS=@-Gf=qy1~YcRi2XM(P?r4*F~WeqJ@el~ z3^P|~e;;F@EdLw8JqC&(4ZeZ~Uqyqjp}|FHaPS7he***V{r|<+{R=U|K#`=urDyJWcGuB0v}U1 zi|`o3KW~<%3K%FFh?3x1G&nwk{B;bJ9F$WPQbG6$J{TxlNqe(V1`2-FG7Dggfr4KF z{|DFohiPL31AL`qw7|#o(`fVuyiZNJRhoWT4*3{Kdcy1NCm;Klr|X z7*&6O1~;a`O=xiZis7GQ43tCvFaQ`RrZl)24Q@_@ThQQ^G`KYlZcBsP)8Gy?xDyTT zLW3Wn!QE-_qcpe|4em{YpP<1{(%`3QaQ}Ze)x+V5n{1wcjg_i#IeJ8WG?=GbAo|SYRh~6qxZl?p8uRtsV&AtL0gpCGE9^lvT-aP zfbTO=MiCy0@6W~GElWOh5$cKia^hGFaQUBV#q5HUJm&r!fT*?)T3WR9)y3+{%L_cdKTfaPRO$m-Us<}gbzdB zgzz!Q(f<05bIjl9|4#4=>Y0&xG2r`iDQgf;)y<{IBYYv$t0H_cJjb<^_>W(zHWIgWfbA3p?(VCL6HAOcsS&Yh+kcToEhO) zA!kQ;BIJt^o`Q2v{N9=DxwoNyHKNajTomDjkgq`a6Ub!|{sM9}gj3s{iK2(_TBzTS z@Hdd}M0gwI;N9fp&<(jg!uuijLii`hPa^ya#@We+VI1_we{o(KT!seMp}`MeoP%^W z!q=Lyyd7xh-DvRRGWXUV+I(XwK8**WJXA=p~d5zdxo>Z*b|)d%y8n);WftpE8lKhKYPEt0H*wobKV&{L=8Ex z0&tM-mx(oXVVuw2FB5zN&IM=e5PTQTSNy}xalUGX`@`5NoX_4bu^61s-Y-+H z^RV@8oX_4b6Z&$TgViExAX>Hq=j#YeISpgqa4tE+iGH7lbr1(ddWNgRm;laYXSfiI z$>V(XY6)LE$ByY^{J;9ek_JCcgGbWf$uxK-4gQn{ucg7eG0s6zoT=|I7-PgPssF29 z95gsD#yKcTGj zP93h-n9=LO*f0(KPh77xqbK}!8FowaU+pP@aSqCs8GRaz?ZWx&Is;}F4-PmdT4i>< z{~XSV*_9>v20qGxqIZY$wiV?=Gmwy;Uxjm`RlphrHN3<5>^cUvi-$3so5DTH%!)8GGiXns&bBzC2zu4h=>lwWtjGe-{?F>H!W3f1QnBl~kI2-3qGaUae0NZfIs-IJcucPKZ@YbdhkaH{E?=rak(_E zmzKr39R3$iCyghR#uG_XQ{h%*sF~mvWbm{ycv=}etqh)422U$P)#Daqa0{|{Dp}lu zES^>tPb-V3mBrJ_;%Q~^w6b_wIch50f*h`t!!5|+7UXaXa(G%fJgpp_R*sq$PbH6A zkjGQWm@DE03p@$I~j{X%+BP3V137+>QdCRsm0|fTvZ!(<AmL0n4YCBAIwY3ekcAM^2Q{SOwO0l%`aB*>TH+8gk$KLa0?_>Qp zarJb!GxadHcSNv@tEru}xfRuoaKACNvUargu%2}+Q%g5%a|~FTJ36{pQoOv}J>0B3 zUH>)&HEvGso=#5YZr(&9{QY2-z#pyM+$ruJR@QEA6q{pi_J8N#Xl?FHnN9pp5vYFr z*UQG-tZhhT1aCL9biuNAhIR2UcXu$g#i*mbg&Qc4oAnV-dpB$HVUUP8?CR!n)ZPj^ zOf;m8xrezUDF?Ex8S!k1fGse}Jiy!2+#Ov!+(~KwItX5}h7D0GauWlAwewNaqvmc{ zBe~f-+nQqkBIRW6ffxZJ+};{B?PTr+8+$iwd|}?} zG~*~ZceeI|m}J1w#l-;|C^oKGhkDqMP1`%WTVt(3(pq9ez>N(483P_}n0sLrGd*VR zft8zh8=w13L9DDTJZ&k2jMOBc?$lTF;r>b;=5WN$dNAn)e}6+fsi}>gfhY7;a)_4^dfo+SpjTQR{;X$pFzh**n`~^Bm#dxEx_K zDgLkCb_hJy($3V?+|t3^mh3g40Q4q`*u-p$3?$=Vt9_`l!4INQ!(0&oYt z>gh;S6)z{?nESY36DB%}om`ylJzU&~mom=UvUG8UTG-QIe|B=6Jv@^GHnKb&$wh^| ziMuw{GyfU|@G!xgT|De9$qfYN%Kx5?{~oKj?%%NruswMa0RTI9V6UGfc{PC6UiKiT zzvfcF|EUmwShu;ky8JZ>0c_{u?%{0igqGh4OtqF^HnwzfMY8?RnaBf-O%E5WpkV$+ zQeeLH*BKAf9rmIe_A+&`IBad{0b$uD&10uK%ro6^ zhX-}0PLHOT!~dTj$7bAFe{izK{ukHO(0>OjPb zZ0|wDT|GT)V1eOL5A3pp+9VLzxLI3cmz-E#z}#!+U}^)dOJ@BSzgV3)OPboaxi}Gx ziWzeLyF32-)cRKz^{=wv2q1?B8hLn;vmpt8@P)n?5JXK5K#R#D#rPBEKcZ&^$DOSbK;H_<#w13 z1eCi{FX$5aq=9Li>wY2}j#) z4JlvpxZoyna=#Gc5#$Seo+uBoLI~`GG80?}i-Q0-**>w(0qpF-HbT#Z#X$i2yVyqX z6#!xZaPs^>tS|#RV9lG*6Dy*qovoM#1k}zdD*k8O{eQ*(|87qsSbk8iLTn@2!Ggr| zNt{?&0{Q|HC)TR~zfa=CniFVekQZTpH5LZ}aIm^aaAIW@aPqwLzuMV>r3c~vtsPqB zA&(1ky9raX#r=fbZVDt$ZZ}O5C$}51(hJ)2F}4xyL#!wQ{)EK6us8^SKg2ddPcA2U zJklD!wDd&1KzowUcXHT12q0hbIAz1>znqW6$>VVWiNC<&MERGIcnOJ<%U??3_2qP_J@qBlvLuu>g1lwh=r6Kr8_M659w4u2nz)oLqj; z7a#ymEiIeS+{mL5CfB^CZ zW6c-rV8N04fpZfqPS_`oqvg3m(v$6^lQ{VHfv|H2a@5W?5+|1*lmP^wCzsz7r?cfJ zbVNBpy#OQf4TE|#-xL~lGD#d-hdfV`>qYiwa=nOo81%0x$<9iW{ZVaR8?Q1~}?3iO4VTYJ+fu3v!%uyf!o`G$I z-V#7801oCxf)nEuPrHeccp?@j?2!A*n#76o zAZni|7vKpb{U$690^kp@jj*o^AQk}6#5RIE0*D2`i?EI0<^zaswcAJ`}J)im@nG`KQ}lgqgia^R*m z*hciPDM>#>;;dL41faLYHbPH6?@W-mFct>^=cKTU2*5t53!!fY5DTcjpNet4D;6j8qmZNeKP3GTl71e^AISA3?&Cmt z+(>#s8hWz-xRdm>{AVSm0RheadQFm^oG+LI z2|KbR{V@_J=4Vh&;7f$wkfiq}aZ3{SA#u3!?}L0k{3JN=br9I8A2W7N!F?`-?}qzCdxCcUj%`FaiSr%c(VW0YYB5HZa`Yj~=Ig)-a z4f|FkeK1MyPD4-qoes5qY)SeM8hYY)ZlJ#QBz+PM{T)(y$mgm1H1tnN{1?eSaeWK& z4I%M*lAhdd9i)6iNqXXUcfd{B-}T_+1dNL%!bF zNU{UQ0%2c+WQUwD@p~R%2V6T5dRvkn_$0wyNcIg#ocNs&u#-gMK_ola53y(LTq1F> zhDq2@CfO(Zc?L;OZvRIlJ=xETY3LhBda|E4)6fr+^ueThjnL3fk~o-Kh12@U0~M1(IH$hTfROOGtWfEd&B6?nP`P?4Je@3uwN; zw?P1$e4iMDQ>vXvEKbDUD~wXCmo|-;o0SToNZ==R}k2aA0u|fFArN zm&kWDPO0q)e2L)1xIp!bp&oE>%}nTt>mnA6Z-*N2u>i&a*Tptg5Wxab9@ zK5-pO@Etr4fI#TG*{J(<)Q=gy1KN3qa76|x7(h7hQY!e0a7At^06q)? zk#8*+;8@^9_-|-m0O9_yoMH&SAc&ixC?dQG+Sf$*dl9OhSPvx1vl-esgy`j9y@+v5 z=y{;SdLeqd1yuWi2$x}^@+gGghyHd0;iq7}IS5}3Z)k|?E22DiVZEvm{a!fWS`oew z`sV<`Z$Ur&hVcE=;!}vEfbC82WH5nXK@j1N!c;DSa8Ednh-VWLcG6%u zwjuf!=qJSW6`@y$?Pi1M!&Xr9^+34(A}aSs_$8>nfbdwzuOU1M@=SztKtFkia0P0% zl$Qv<0V;xpdW1K?c?0~87zCoeDxB2)A%yF}e*cbe9@y`4u>A--i=kc};X6gC_O~E> z5Khc{5H11x*Ad|-U^}}ZeDzAIeJ_MR6{K=Mgdd0V$ytOCz;PFXaA{c13ka`<{Su9E z2k5up_s$>??Xw5Yhc^(e0{7Dq{u1V!kMK0;pT!7&4$nt52p@;_YDIW2JW=-}yb_-8 zz9L)@o{;Gn|E|{u=pVcY&xR*_5rh-v*^clM=!ZrKC-hbb?}6nxg78jwzVkykQO=79 zC%zX+Ksb?aA;O7#OAt=vTaR!e-%*5Lh36yidusrzgQXNL1X7s4y4a>_Dx;* z{DR-Vll8>7IEQe;-!3EEYc)0f4TRr-{__CgD`CDR2>$@D_rUMs$>rGz_gfKO3){I5 z;lXpM=|>U10M2he5YDuUs%K)NeiH4r1WF895_6XERPDp8{j;?1L3T&-S#7V4s0JAgj+-XDTMRG>#J~t8^ZHoEW$Uz^Hdtb z9bkJA&p0L8#|G+)5q&1SA9;mvQ`in|2)9~EZRY`mFNJ?Stn4fI!&U1ossX{#=Br(?EC_EWaMYi>Y$TL4;ev_B@R6R@k1$5YEg> zH57nwf7p+g5Z(rP62ecwaax4%V%R<}5k3m%=T3xgg#FluaN+@R!w64@_Q7)oKp@)L z4$hO`5pE0jetzVnf%Aq4!r#Gqf#(>I%b5V$jahHJbTdn7wI5E!?^BJKh=JO?Ro+J2g=npFq?gRTp3gOzYJvSj-2lfm8@PAS{rJz6TL-Y&a zJZyvToV8eDEO;P%F7!`-gj3ZNN(91{p*|krx8XdHj_?*ZZx!Ykl?u0O)dpkJLsI0rnxoJaUqc%HkA za7ICDyJaD~ACA*Jgg3zJ@!wCNh%jt@6N0{$XgdcC5ncetV=BTK;rTZc;r#2UewByt3h0Mb z2;T|Yp&sGx@Ou6&!mq$_-Hz}8c)ihs@Q-jj4kEl7w&y6qmEd`89-ME8@xlN(8^Tw> ze&<4X9K0V~hHydHzk&!S)-Qw+ehu1JLHI8?9<>lY2cBoQBRmC`#{l7lu>B7sT$Grf z5zY%;pjBxN=KM=knocP{r4!qtZ{O}-be-4Bb z>t+H74}kNq62ggf8*PNk!+tSD_)&UlIn5C60L$ZqaN_$gAB0!HdIlkU2OPgq2)BUc zNkVuFJfCJGJR0gBA^bNS7ZnKqDM+o?8-&k=^W=MkUxofNf^cFTg*@p05*k48nmxJZDLikr;D)PA{(@Eb7SB7{r9^Hep$)2U`CEeKbI-*bIHxHoK{ zF9_cX=V4-fh3GFYSlExpU{6a5&j114G=yD&R1p# z?}GJpM))W^@B1R0PL$ekAqZDpN99)#-URELig4oo^d7=_p#C|+iThLHI*Vve;(oRR z(Z7fN@)6+{&~K*^{u7?>=0QIq>|4U|z69Zha6GO;_}+Qcc921M2kc)}g!{wu^)7_B z!hH*b%R>KmMmRCgA4hmM?3Z&0C+3aI2-k!DK#V7%9eUyXb_day!u#3B2v>&f^Ah3A zaQD58H2{qq;XzryQJ7I+;;*vWJMPV|c@!ig7%I3k>w9|92GD@gULNQ6Ix{*Z|9LOB0qB0Lk8^8vz%`Kk=z z?84N1Un4vb=G%pEy46(uXN0>lQu$AWn^Q4`13&KWKXkxTv=8fp<_uR1~n= zQBhG*!EQl721P|hYyoMdq;13;F|e_*vAv32*oB>#m|$aLUEf}F*2Z<$W&Zc~d7t<8 zhnX|yd)8jD`RV7?;;uE;3#S%Uyo|EX!H0BVRK4 zDOiuWj32=AF~ajmsplH@>(Y#;;=EIXaV?gEHRBIazCGgyaa?j?JR942f5z2VzX6O} zHB`zinsF>1)kMa-pq@F5pT~94CdT3UypZ-V{vFrzCm5IW$`!`PV*h%`cyo;R732G` z{Bs%ai1VDhzAo$QKCY`vV*ijl2*>kkjLY?81IDF4j*QFkt_$Piv7mc0ULV`}P{!r= zj$w>ftEZH6G~@ER{Ak8gvA;}Ud=jpgW-(qK{aMDiANK2wj7Ok9dl;AJ1t%D9j`ekg zaU&ex9x&by=dV|cm%#Do8{^$@-Yks$M%M2ZY*%F%ABg)k8DE3_(T4FaSdSeTZ-VRN zu8g0?@*Kc;7>)yhjGw^qKZbELY`2pbPlEtOn#;Hk%CBU61Ge*Rj4#7+?-1iPa2;LXF;52WPOF@weC?y%;}$>#>YD-=&OKUYFU$ z^z#biMX>)qWIP?))f>k9p!^TUvv56J1m^+i=U?bgImZ35{A)437Ta4B z#&_X)kR9W9Fs|;57sviGi1CS7kHZ)@$M!jbarm7fNK+WUhwXem<5SSj)r`ktdH%)t zIxL4HjQ7QIxWM>PY$taazk>Dhg7H?!zcM}($6@4<7d?hU%_}gY!6!) zm+R33jMu~ZJ73-@vlb81kdNaNs=fUBO@5KH%j`1m2o--J?!1+s#)6$<; z1(bS}_hCui9nZV=Fg*r%{&JG>I7N->I^*&>!vn@U;=J{maVh_u@oTtlG{$xz{oIS^ zHI*6HVE?Mm_$wS&TQDxqQ)Ii7`oCdZJ(#?_t~HqP?>LSHF&=?-<#i6JAO2Sm(j+F2 z%TJY@&!s&44iF@HAEo5-zQKJ=kG${n6yx%~)@zK*`&Tm=U!>NVRefi?GWIw56Xw#N z0reDlBkX^YKT@!&IOFpFuB{p04;e*j#`ps4kG&bMjpOGq#sjgPBrv`P$L;luZ$tgp z7&pc7=ON?maGm^}aSQCfX4oEOIV@M}!cy5V?uggBx-z~C*9k)zm)}cGVZ0fx&zCVi z1^eAr#&=^q-eg?fAMl3p&Nxn%#5iPJi*ej&!T343o@ZQ(^1+N3L;W)tABpw5hjB9; z_pUL%u%1enUNZg)``r)5Ph$Tv$N53VWrzLRhH*>uzZ>IbC?Cjp7u=u7_!7lU)e^>c zBHzOJ2=w~|CZW1P2Tyi0L@v0z-@-`0$A`Ts30jE7^p8N#?} zeI>96#&2N#PG>kJVJehE?8;A1K7>`9h3%M)@9Bx%9jJHMk6NJN*DSwamBH>V; zy;1%p;UF)6r}q=#An$?l1{TT(>CYtORS1VAo%~(jI)sCsSt#F>@ioX@2?u}V?*R8F z9P}JQ`DnsHUj9C?ynj>rc^c(s5P8t^9Qg*qL67|Z*qwxfd=AQ|F)r63_Xwx{KPMdY zR0YRGDo{(=!Y;@M7Etz02nTsfl&`?JBl2d1gC6OB2f{&5ca(QwT;2y5Kse};i{q+kaao-x=j?8SjMLm2vqym~M>A=k)a> z9LAXh)Z;@q#5)>!0OPX0f{}w*C-EAH?J$9H`2A@}8yFvr{2b#6$g>$=h}@`-qEFg| z-|A4WLH7eO9HILLoQo=7;zn;~CJ zILQA*zLRlz-_$|m&k|(3QVV%A!a=?Y za#zAZUjDwlC*dGp8|7mecS5cu9Q4TZuX%)n9=xog+Qc}Xj;Kx%4tlm?`CKL(^t2K$ zNcS0swgM@aaL}_G^%TJMjx0A{owO(q!a@EZ%FF*tkn&?uz9W;D?Zky}@N*8z`x6fO z&!Ya}goFGhl%GyG$jj%NEg&4^PoVr?goC_%p6>y|K|UMhZxRmjIp|L&;UEvsz=HIH zaFG9oyih&eVHNCxyc)ThaFCbdRXxH%z8%Ut5)Sfmyy{Lk$oEJ2p@f6H9Iqk?2l*J3 zpG7#x%kgRv;UGT~d<`3UiXbeM3EUyJe?goFG75EkhP;UF*D)!&4J{1y-v$r#UL zKrP72aj6XBH_)#9e;3#VJ@Wa>El|FwU@#A=jm4Wt-5HnP=MG?eMG-~bpYd4qJA&~* zd7Y5)WD`YC5_0gi3ig}iKsdkLrQ{{-W*{BJNW+ljo6 zF6&X2=UbGA;(~IJb_+EWJ0byx@|5Ge8Q~z0r>m-3$fbVS{%shSezqkX%BP5-coWH% z@j}Qu6HeQSJL57gZ{#vA*`7xd4slJm-Yrei_#~#${Z85e{+HM*nv) zUIY0N!f9OR8JBV0MlRz58<5@+4spqO_B-Jamm3I+WZFpC^7^X8xQwed;b7Ad{jbM( z2joo&?=IwJJG5n7#?=+MjLS!HPvt{6#I*%^DB%!S5b}vkzpSrB#${Yf2#2_aqW>!x z_d~vc@a{rh#Tz35U2QBLBwp%eV^Rz%J`i#$`r0#5D!| zFU|M_-WV&B$e3o6*nXghO08$S)EOait=Ef?U>%jO!iaGOq80LtGco z|GybOjocV7I7|JqzDhGLd{nI?DHD`ej@`jLWz}35U2o zp#Kq!zePTVa9Uql#${X!kjuF4p`Tj_hx&qNwnExVIK=f-M@4xv{W7i$#${Yj35U4k zbHuV4FABy*`an31>u<(oT*VqI`D9$5lwDN~!a;RyjH?0R5Z6zX?}%Ks6B$=G#${an z2#2`pq5p#zm(MlxC!EF=!MKbo4!MlW1pQw~IGAjPd==pkS83!snSL495yoX)=LmnYwm6s_e*en}g+91v%`tCGvJmz5&QAEBq zkzc{&6H6)wHxhYyO%9G+Wb$wE`^^j@-;(J6#pHc(ylFK85jV$mcP>1o=wF*COA_ z_-^C}7*9ifhVd)NZ!mrr`D4cAb;Wm#e@6MAj2FQ6T%@@oAnVr@d0ECQBd^JLUF6n` z%jdSYV_c3)?u^Utl|l)JsX7<+k3=r}<2X~r?@2@+Kk5#oF!=y%hkF>GQd`k;g>m`+ z=?@scQdp6HiX7a-Aw+keFm@8@r@X#bmT@n&PFz(PIq1QmU3Z`Zlg}xw$aiD>R567I zFfRX(K9cbeMO-x&IoNd&FE#RIOnw%Q+bKjIFWu>KoFVcs4(!KrPG@{O@+*YHys!et znR|qT9{F6$Ovaz1o@a!E9vRm=aST!}lb82h6vRPLj$>*ZhYK@a8Mz7JV7DON z*Hn&h@V_O>S7N*=ay4@B6UwuOctL7LIE0GbR@H%Uke6e9AmiW@q_Kp99(x@X*VhRL zJ%f>NX51h73C6>a-(x%(`8UF$|H|)kjnH9UFSU`&{<0X$r!(U(YAFUn8UKRqb1vf} zOceR$$iZ$s1?k=o6M3kwo@n?`g8VMwpy!)-L3&O& z=vjdBFBwln{*G|aBd@!CCmi(bMftxO-+|l&7a6ks-$ri6_;ut}8UKL12IF!)QwO;$ zf2=N*J>k$kt;7pbAHt!&O9Lib#|M(5UK~EFZlfifc4ddOBe;}N;pI?N7o*0xj zXrtJX^*bE75#hA`m=g|q7NL9@#^)lh$@o6xbr|1;ycy%ykhfy|0`m6Aq5VKT)>Oh# zg)ol8jw*?9Iqq#C`XSz}(6~g}OXTq@YaPE$hD)RWEl3FL1Gr}djlIOute@;@1Wj@$rGprGDq{hAOC zdQ6IlH<8R3H%4BS@mk1hFkT(GHRBzSH)XsP@>a;9exdwj{SIVY*6&otZABhP%ZPr6 zHyIb9DMY@uPEvV0L*$_x2BSaejQ2%;g>YKG_Xr0)aVVe3_(2&eU%LpbPJh4P;m zUyA%2;k14Y@uG(8@5fNynDN8N%QAi!c}2!=A-81wGjc1&-yyd_4)qJ=FYDKhaaq4{ zjN6JlkmeKp5U>1y>|`Qup_5eJ(uh2iry2@Gq!WynKz@~R`F!4+j0d6oUBYQSJ|`UP zUP1YnjPFDKj&NF!-w6jj4#mKGai0$3jgc3`0S(lG9(le~966LfoX2^J7o=)L9`w&c zJvAAhgWQtnfpYE!NpFAfvk0etE+QQ4rlI_P!VLv;g#bl5 zO*qwag>caGN++#Sy(Jv32&Z~#*eV%iIWH=u>^DFT+2y$!9O;T2^3%0_G1`VBOgsT=#l3OQwRq=2`Hby z_$cIwgo7U04i_Vb@`3V<6E8@cZFQSGKQEx3?TnvCz6-f5XSsfR#CUH-t?Dh~amY>W zlnxG=|6{(j8bvZ(G8l9nPr+TEu-ogo7UWJl`&i+hV!( zV>}%BAjXFvAA(%^FUPmBj91lpBl;8L^^hkqj>C~^JL9rEPZAF0AB%b}5v~?&9P%v2 z=OQ=64ki6xhTNR-jmWDqeja&q#_uEV#&{txB$7Ab;Qw|V6~9eSxLPpz{|nO?m;Y~) zgj}|(^0=s9PUNvE=qwy2^3Wc3VLcvWd<*i^ghRWM_gP&f9Q52n`J0U2M1B`Jgb)7U z*EOn7jH|^lNCg~}t&Gd7in4Ev9Q4RG1xGB2JoxiB+O=Xl7r71L5SM%|Yg@uWkGh0- z6Umlw`FvqV(=?gLgFp7@&osu{Az#e6FY@J# zdmvwp9Q+5n(*Ny@%YJ&8=mGzO%8ECUP7ryi1Nc)S5B^U_yV;D#Bmc_y0pvdzm(Leg zIf51CdtR_BZI@@fzD`QqXTdmLCQ@k_*Py&7<9(3FGVYH&k@3;U*CChfSzS;$u$jnL zLRr{cV)9iAD+lio4&^C-=kx*N_p#ib5)Sp%$x6}ljd7{psFQMB`dPfXvTuSM+DQck z>E0VL`Lh^Td&Yy!luW%Cm-3#Be=*7m%ZZ*c3evqF zW%Ba50hbx~#`C+IL=S!irPK3`$;;;w7&|G(WO=${11*6Z;*x6+khEd)@_7Zej8AN! z=<5#!DeTNI2LHM1Bl8*u^FVYE+MjJm_zU^3NG>i2N1dpkK~wUkC?1((ZT0 zy-?3@!a+~o^Hp&GfqDl$^HIJ8<1>(#LN4PfFY-XLB)pK=H`Y-x9}y1aP#bwy#^pL{ z2;rayt5+37IOyq$dPX9b?RHxoL;8tamVd8`3NPGQ z*@9iXG@t|ZkxThHU_>MvB418Hy7wML9&EZ|T)i3BAn#8&EjM4nLC;8(4`e(Nc?fdw zAN-N!Hi~gsZi$4`a!V#0^vH7C%lH6s4AKq8{S>VF!1!?Frd^Z|($9G0^^g}8`%n&Z zu$--lJRFnBmB@pS%di~W7+;9ohjIDb`2fcEp?olM@DuFHb>K+G>!Cwai5~D@K38-$ zk#B~wut{O^@_Dd32#2`jbM1FCE`ML|B;%EE@;u9UDdbm?LtNm`R`G&#R}cTp^w^@F zZ;ZD`UH}^mlo@>Nj@*dxuE@;le=bk!}t@4{#RvOi@ZML^O4&#z8ra9 z#?^`&DqqI!kdI-!7xMXxhaum>xSYREGCmLG?=!v*`Dextp#Q~T@(>Ajp?uCFuf_OH z zD<#7l#!V~~UdUD1O1sIp9yVoM>Z!>1QJlzYFvmX9~%O_2vMJrEzHIL2}Ls+z&LyCSYy%(xHo1B{PAevk1P$X_u27xFKR zA3$!{L-`=<mO zLwN_r{spUZe>vl;clOyAok@sZW2rrVkGcJEOVE}UQAN-Nm0Y)(1 zL@}$H&A0>db%eup5}CFW4*GFAP#t2tFJ?H$_z>i`7!N}Jl<^egg>ZtCaUDWloAIm2 zofxkH8Aa+#xS@h{@8N{YYff-r3F9W1XN?}7h8)@n)NlP#N`@1R*F}B_Ik<!G(zLD__$af+Kf55J^eNYd-#q?yLp1X|SK>n2R z56H6_&q4kMx%3C8EmhIpN}&LUcAEqrL@Gx(^j$STk!lbQG0E!yc7#K}mi@Fh;Y~pS zrZD8PUCG~1iey|>TFEem@pj0^Gu{OGG~^Ht*tJ#EsOIY7TbZ6=sAmV`{>Tq7J`wp5 z#^aHnKra31hyFYw9LoO;mj7$Qn+jeX<-alB2zfDF@IZFF)T%pD8@Vj!rRYyR#^)e! z!t}_oqb-q#^#Bg1Di^|GTsn(-Tp2%!yf@P?{SP7>+Rr##9E~O%;wmd%kmeE&^06qt z8ac!Z>!0~J`LEZ*H#07uAG@7#3oD^aq+N`+!a;5y<45s=<6*|-^L~#pz70aC*e&B8H~&4VBcq4KEDx%58e5flj4f*y(NAIbHMANUD>WB z_d>bxOnxlNZy>xAeum96JzW0(F=PjM3zX}E`Xu+o_Ta;~{6DoZjK`zAJfD_&*3e*@aK%b#C`dE0@#JPyG}Ut~NL`A5dnk()xLiX{Dfs_d%d z?~F*3XLp#}bcyj1dU&iJzE}_6&$yfiuQT3G`cJqe@w1Gv zvIG88-a-%Wqlbqvj?Jj@BZG|-y}F*FALU=z^gYM_VrARN9=muZw9zMOFEBDySTdUzJ$ z&7^X%t&D{Qegc>6$Byx>n8#HQ_hNh>%EvH%7WrhxUm>5bhp%J&3(6m4Tz;Q;o^kp8 z;2k~u731=`83x$MWckSN(aiMl>WoK2Mv>YOjzhkVdlFtiN0c{ty%w?q$7ZHVV)C-B z>>wPcM4h4x!f~qC@xpil1Nwo>K50cbj4`t9wjmtOeI<7xT(%u?Smm#WPbSt*OqU6-E}__F5>D4eIfT=7o%}Kn4$Hm^>ddgw1E=d-HQ|;Zh>2bYs4Jn^ z+7kJCgwyK-^$GVR@^rikBfKGzk0ZPh;faJ>6P`?XW5Tx+-h}Wp!kZF)nQ%H^XA<6= z$mbAF&s$aGI)J=31=-99Z$eS)|N%m#nb~>WG$u{jC?_;Ex<@ya~r9qa(qD8{ucz z#1c;Doe6}~d1of!blyoKoX$I|2&ePTX2R*bvzu^PhJextr}NG^!s)zoop3Y!44Vgp z(|ISGa60dNAe_!SKMAMvjuBLpNU)*vjv3)}-l<49op)*xPUoFQgwuJa72&cifFJgR z(|M;W;dI{VOSo*Spl1l-blwRfoX$IPT?~iSpa9c&B41rXu}vVn2H^_`r}K`yz6*zI z6ZuUAZ7> za2U^Jx<)vicODQ9?&R|kL)9oPeeJ{wk6+#T((uo<^LDUwkNqe zRD?*f%}BlhxvlP~a_>C&^8>jo6Dco$*Qqm;FNJY-XIzckjq$q3`!U`a`C!IdBbW20 z^hf?4VgQquzcVQ3Qz_pY^@KBd`F&LkwaVKt7pq`F(K$88%IVL)U3BM7|`EPa>SI)6Nl2*J(cqr|YzaSm8o%VxpYofm+b_TF3=Sny{gmAh}izl3})9Cj&be(nqI}hlWb1vxbjGYHKU8kKO zoUYTd38(8c6YOlDhpyAC38(8cH^S*UEtYV)PFqSiU8kiHPS8_pZod*VmCm!m%ss$Y#Q^%j!rP;dK1ZBpkb%j(j2Y?|H_+u?gt-bfSlj|H*{Y@jsPttWupGdA}j(r{n)U zB2UMEdHoaQ>G)qBfJm^Rx9$s zUtU*)!)o!8Nv@B8SC>$19dKO%yawR`gwyeVI^lHuKS($o|6dbM$A7s#0lRekcf)lB za60}^C!CJ|CkSsq{J%{&9sgewPRD=U&1WJ%*tI5l)VQtyPRIYIgwyd~LwHl7Cx~!5 z{_iE6j{oU|)A2u>a610W^$++-$A2>%bb!$gDgFGGo<@3CO)A3)fqkz-# zUp~(pI353|61%qIm`qCvx06t8GYRiVxEcpd&~Hz;yx$49BjFk%--+<4ggX(wmvA|T z!(q8j1O0UTS2c_n5gh3?L~J9&m2H6h6zLlqscPus73rmFI3yxM)iB&Q$g3ex@hLLg z%RAB|I5a9GGQuM?#3S4{!dFO#hk`sPvuW7!pSw0xL3Chcy ze7!>QKP2ttx%5Y)myb`luV4Pzgp(fLp~1mkAwDW4lt20W#j&8kA>m%(BSbjEqXNTy zh0vd%|44!sDH#?X8Xf2ZlDd-lBSh8d6#`9uQ6b)ufual}f zqo=??);A>DBibuGUp-K7fxh|M2=MI-m{6Fny?Tu)Q*4)FCBy(=^%A}}J-H^f^{okfO= z)+a1^#CS!DEIuBgQITO$&X?X{Dgn4-n^YZ^cA`J@- z@eT<6Z#mJa1H(f@f_+2u6g@Q9h!C%^h=5QrqUcIDpK&AtsyHeLN+X}6Vc~(H;enBQ zT8ggshlT`3hK6GWhzio6j#>pS@z z49tJ5j)?RT?Ji#zd}9N_4;Xdx(I?}|=MYRqVWBX{u_Hr0#B>}OB8M7fiq@}GG+ll3u~0Qs42StOdN>pjs%JQa8O|eb>hkal z4-NjOPpovHspZXz`tm6&4ey9ZSZBZx?&CWo%3lmmexYL3qWkPMWJtJg^gj*QP(;-5EAD=a+3|KEZ^V%Z{a38;<#T{E2w}B>(?Py1?TP z768gkbsE_e4Q@4Fyzj`$q)J&;FTtApNJE zfql3x#qDPbmY@AN6r|_oLiAtu?+TWm{YLPP~{+rELGI(M>@E@+PasRvMv2R^O$@pLXhv~^L|L+BSqlw2q zS5JOdv`?>Tf*zh9?gJ5j|LT99n7=DEo=^V$B6EJ_7l!4Ri20%XpwIL2yQ|0kcC=5g zv4Se@|5rWv(=b21R+?Y_Dq?)hFaAu-Pp^sPmmjW!<`=*G;d(jFLHuwXlgHmqk9{+| zNc~^-Tk5f2egSy<&;AeBMY;U}dhFZ&kM`a5*tgEdzB(WKBlXx1`ycH`>9McL$9|1` z>?i55pN#h7MIM@X`8*4S~~Fu&Yz`-k+{S1(j@XfPw# zuPt8O{!=~n6Vbjq=9e38AMS6+ul)z)W8X3#`^I9Ol3)MJ`5)siDDvlLe|tXm>*Zq~ z?t943zAJv|Q&t9!TOL2$?~q^qFw8&xKk|3f<9{ytpMv?J{@}VjufL&s?8`5w{%ijB z)nh*y7ogU}K70?r?N8BTU;h3@N6arby!_(z*uRYSHN-xw)w%r*dh93S#8F)u#x1wM zMvwjSiVGO~E1oqpA7q>rHkA1Z{L^SrUn#*K_+F^w>{Zq2#CQ zXJ>)9efYjOzxd<+NBjPI?BCAEKD1?Sf1V!uiT|VhS$gc}V7`|_U>p>5H`?SIpg->kHf>%ZDhB_WVs{i`wmAZlEE z=Jv~q{Q0&2xMU@xr?SzF|GmU#9>2XF`*CRBQ>>3^;`ZC?v7ec&s ztH*vOE?ff?HF@vceypDSuH}?0-cYzC@%#~b{6Di=$+Q>a2mkwu7x#ab9{Xw7@kR6a zN8B)cjf6RYePyWpRF@L6>{5hCkEU*5Nxc^`EalNINy$GpujG{vy#Chev7eTr$R}d{sbcFZUfe$Xj(L9dpN971Fu&Yz z`)Bmncio`mNW+X^-(S4AefZsSu+N4$l`UNC5^0{Y$$MAleNsNW(PQ6iqjH?aA1DyF z{~PU#ZuF0oiT3d{JntBf|GOUh$!K5JGsHhkytw_U0_0czIcVR24FKGJ1(83$_O}=9 zC*uGJ_Ji}W-%5{tm0Adkv=N*miQ8|g$9~`KN+vr0gbF0YbxD>B{JwjB@o%rL$T^|B z+;ICn^w_uEspRm)j8nySxOj2m=(ykvf9zm)i%AV2%A8pT3}G=y83U*ZxX ze}4A&4p8EU^%*3vKSsP{erexQk9})r$R&OcPG(Sc^YY`as-_>kQ?6JCc<_WtBoq7(PL>dBu3ldVWSrJ%B#XJ6v(GCv-do0mSy zCrb%%EAz>1U(C* zo`00cpI`gAEg$Ga+RtR6N9LFH2k~&*V6G)hHX<40I!Y@(C&4~Ikz{^h{$KZAz;RnJ zCXz)lWh>u#`9bqwQ>0Gx_HadV8Wt?N*G-+(3RyS^H z)4XvD<=w{Gx_Q&4YIW1*HjS-8Q0-ZCoTE#p0tSCH8i+fy|LpxSYPtH)XItGzTa|;j zn>x?7hjKSrYjv2o`$yc3S}=$FJD)DUxMNBj zSBHkE#offd;o=rbU$r;f@2rjt4IL)(HneUM~It{v7C@SI$9_AAmu2$};RM)Pp z7Pls(grx1M(PZXVscx%6viqi)zpDv$7sRW)ef6%`rKE;7)2n6GfGf$|{& zK|}9=!0<58t^431?tg>gR{j(de?)XHXeRwOoj6N;v~%yJ(f-g(cx{~EctQt*NPkVjDBIsUDe=iH?^F4_Pc8A2_M!5@CQUQp zgz*5o!N*Oj*{U=N<21r{jp(`s?BYm+9#98{_=PKE^{=jJIjbJm_x zc9eIQx;e2vex~ozK4?tqrD+m+nQ3M;gZ!E!@DB6bUg#blOb6>0rAn?B9e)4*?WfbP zN$4Z92nX{rp`7NAoc7p+Oari_x$;s|Vi6|Z-7I2F z>sna&nGQMXXJ;C6MtnHuXBwK}XBu?dwC+F)Ebo2AMP`lmv_^Z}MVtFkvq>}QO{9rN z`%ZCRqdh4ySqU|vkrrke?JzS@*nPoM7p=GYw7qVeKW*;{#MFG4t*LpFu9M!FPCO&_ zoo9@;Fu*d_K6cT*)oATo3R9>#!MmGCc+tx&r;9z0tqp5GGi`{h6WsyxZ4Wa$^fI)Tx?FQKmw(}6RA1AmxmawaH zbYdS-t)$fo1uxtM(zVNf>3+3J;Xyi^8tq#W+N89oiGuvYmlvTzl_CT7wYkvDi_0G6 zT;#+Ne>j2=S+i^bHPEiI1>tAa1?%3ThxI-jJ;ufK*D?Rmp)s8D;WzVO(ew zDU`&9`*?>zZ^y149vT{{>f6k^l}GPhjy*kGdv@>JPvzFtRSY;_FjgFi3iF7n6{Qk4 zT*i(NyLlfPhWQ4gS@HiWLd`!$4`tM0LkX0x3R*fea${LaFEumT!2!`mHQG})X_7w_ ztrI_sVL_wK9xw>jy`mF};xg1EIJJabvaiZ@X+oSDF~DYhNJS%1&)M(O>K) z*m*)1@em&!vpDY7Z!m2xma4z9x6ya6vy z2dLH_l|^l*8vtKC`&;PHjj=jY@(rrW(dMioWOKHs%~f}sFX+L96XlQR9ft*1e#f;r zy+nRZLU*+&&hi@VwV^7Tt3y?Gx{IiP1`KWAzWw_RI4(Lm+AFBj2f{XAu!#)C0Y?l5 z<#m-UM0)E+Oi|rdy3s6#zFf?`%(-&*Z`EV z^LH(s8+mVb?lx)8+P|fR-gYn}*=axZ?j;NF=*89)<BQEe_rEAE-f&(;ab;}I8;P-@;fO6>kn+$Te>D?i zFeMb0Bbfrk#AZf`t?BMB<1faED%QqW;m@u~+WyZy{ zSaRRu$J@8(Lz-vYw+K3{t~SxCNRdZ1hub6>nYkp|US6dx-=j-s=QLyEzIK5(Y)bpZ zmF>U2<${Z;1*11!uG`>Z%C25-uFr~IH!gMege{NHq+U!3_J1Alu2t6IuW!%aEq*)t z_uf(CvR*BV5a_|4Hb?qj8(pzjz~d&BCMPd=a{S}ZGwUX_ zUbxJnZFPdauXbMYci+*gn{}8IH)25C<(T?G7r&l+^KGZ|sIiBOK0jJwZsL*6xho!T zGJO8%=lcdO*V<|xo*m!t=+>)7M~xGwb?ATap_{3RQ$52vw?iA1x>{^x!o$?(zfM&c zQLMh1p|M*H<5HS3ZR^-p7+=q@Zh;%d zFN<7PYf7cq7hGnzuG|)Lj|v5vn0+!&ak|_+*}S+(t#_3()|6>py>IuWeuhhmq^fUs zR?qNUYFj^kjNu)#$`+SrddAxpj`ucfZQj<|vugY`!{;?ycJZtp|Iu*2S=;W(m5w$l z;$wEOlc#z7!&2FHIsOF_%m>!KT*)xUVEkdrpi4)>eiU7(zFdB#Z=t<*CGYnQD}7b7 z>RGMzlYA;%i|;U@w85?lNd_HCH@R<`F{tzbtLL7VY#LW?*K*|W^wnDQYWK=p3)1`w6}evMb=6hfoow4VtQfzqOFM^cjgmq?n;lx2nRvtg z)2X=o_Mg@TbXl2U|7nqN%9QL2-!7YPn3BD7&PPN4skX(!1qw7=lDhwV`wWvyjUN<< zPP{ar*!N~l93GV2(4evX(}>HnY{P?&52_g2XVv2>o!t(WGH^VZ^mIFT;ET=5ok_N}Kh@k;p@zrAxQDJ4Pk$~*8)+Zs6@YwMLe{(yL^e7kf~4?$hks%^Pg~;nlEtk&t24rUe$fS76wN29x)F zz0q-J%!ih#lQ)=J41Vz|-CYx$)bZ&34c|;c`&(wWKbYOjwtG^sPlw(%nbBg@9^>eo z9T};0ruQ2VA09P5b?mwRgI5n4*RRQs)GKAa&-H6=vwF$#&$Dk{IFlVV&Z^FK&F>9< z(M?wu{!w#F%J71V*M0G`y7BN<`WL^=)}!4=HTizrsL90TC#FXk zr-WAcIGSOeO}4xFW##p=HR_gWchNer)Vg`@ z2KLUI#;Qh^O!vO~OFjI`=zvn%dG0kTIG*d+a$I4Du-IxV3Y;FBGq%gr)vo3-#^zm| zZho@b98mLXQTImMJs-Czc6{Ql@Y%O26y5x!c7+>{Zv+gVx1_$~sr@~^Z1**}<#=V< zkMTdpWFEf|AKfrq`=XpnnUXFRJw6vd-?pLl$iOLsr!AQ?`mwj?)+?KP+c#cu#`J81 zo6o&#%)GY5s`0ZHZLZXxSZ|7BOmKW#&k4CD+g$AbquDd>H0yJpGKyaCUv#@$?N0p` zyCmis*KeP-^W>ogxxFmfAG_oBen3CNCVj___19KwlDuh^L&Epb&acjP9I(Fsw0o^~ z*k69zE3oHV?eP%}@4Wa}bW7>UQ6{tcd@NPvT8Mw_q9z>{91EP?)I2b-UBQx27_Ca3#U-nj9{{x>n9S5Nqr zS?V+A+PRKtN52hha^)}gVM}-Izcc!~c3o1+QLm%#!g_laKX9e@_-3!W?JD57>&CO} zb!KN?)oL}jQuBo;moAy_8_}!w#Ncx~95)3d82Ka(_fPuT-)n2blh#lA9h>f7q)Gd_ z=`#j24|MLcy6%d{>JiNrG>PoH=*WqC-wY~ze`gihcjKb97kfP#=dtnezzPO0hisVE z@4&DP(=r<0>RD!I*SU>rw07)ta8PUCP7de(eptaga&oKtML#w$j&RAz+HlcGv;I<4 zP@xQ?KG}A=dlkRm_+xmjPIebJ4+#ufpWUuho$xl^!yd)WFY$d@iROJvjBU`c+LKG+ z)4un+)uTo5uJeu^etd9i*!sI!>y4FvWxlTbXmj7l28NASv}{{tXWJ^>rrvE`G`)9~ z;K`4Ag>5M6x1vJMxkv3HqZ?H{QPsayzi!9cB-HU*H-2Zz;!^$gXO(DMs&^0L5KFJq zYpnhbns&N%(M^sQ0xLBA>@{;tY^&<_-wIC*|6Imx`OS4(AC#H$tL*u|lUt4}b^Ol! zkohyN)jxmZnajQPgDNaq|8(f*>@RInk2v<;^}23L+Xtx|9~^v@Q`>h$!94~(2k&lf zzf?13=vec>kHRa zebs!_!2(~ihU|QqRU_iVUq8pDc3k(ZSoK){FI7hdcQ?uGGWN#H&((TosK$-)Y&5C< zq?en24qakXe~qgCyt^;_;`TJL-ZuEriAf>Og?1Et^y^3F+~KL6+B|yn>wba4ktf@i zKO5}X^K9m{cBNcLC4ReQ)#iP%N1ZQh|Kc)Xw7tWl^;Nd5*qt-Q;%DThdZW{~eVN>_ zOYV~S;Rj|l+i){%?U85Mbu_JCsQa0xH7Y&xaJaflgKLMb55Lv?-O<(03$%TE=6=$% z+CIyszIb1v*}5_%yzYA!UEzE-$$iZr?cpJRElIuDb$?b$!l&f#U%IV%^3ra~udH?D zug}g(ba<0xUTEW#T?S4Aj`m6E+u1L7ca7W)Miuw%n!0VOE$2u2=F)!^ToT;+VCJWF_gZYI{nz+C&fi1)u4Ng2e>>05 zqxFd;lds(|->}Om;&3P54buW!eK($WIJ@SxmxJmLuxxqb?y1ztN&D>=1m=#JYWTI! zS*ro%A2e=t>p&HYF5imZI9qIVN&oQoS-u6D42vl9_vwwzuXhZ|+S02Rlgb@-@)bP;;RW4oVN^H^{eoMwAB9pJRj(;T>Z&AUx-~A-0O|| zHCwYo*CMO8Ztm)?sd;jTaf7jCHub2xp-}w}XFM*w=;3j!a8Bh%l|4Fp9E%CB6S(;P z9gmEA%i2WvzqNW65E;_OcJ(Cn{><2-_D7hevT?z@N0-VFz$zM`?yWk(Z{L=jqhq1IH2p|#!LLRCfuuCvd|Tq zr1g~@(rzX;oa-BJH|JIJH$j7=##)rzduZ>om*G9r-Yj18cbzv@&gW_c)^9rSO_tYw zqkA_>JEX78blp4Z#p2peA}_?XcKO)X(s^2sLhUzpKJe++g1DJ|2CeV6(?-)dqpc|<%bq;r~6m7Mu&e^Y;?EC4emM_Ykzt*VJ znx*E&QybK{(@pzf$fe~KD~Cv{ooz~UMNZ)qa_fYpup{uPT22~0!{Lysd@93@M z{ICXoXDQDYW0ziV@2YX@QGU$BF)?oSeh-^&DbLUPq zYyGQnA@jbeDN|SZRWDg7$6CS)g zH0si~!yhlkyo=cuvu)kJ)W*SCbu&ghs<^&oR+*n=PVQX%&Bm^qYy1l1Uh}5-8>cxe zxOBO(SJG7nS!JWj_4P0z$+72{~wJ$%qW`PsY1xj05Uzu>myzLLCAwu4Y%ajc+Dy}ylsIXpDUJ+Y8>*-9edt=QN_N#=UW>s_`EcvfnSfk zWv_32X7sb)ny4P9qxN5N^Ln?_$z*@k>Q?LeWTb9AwcdO|p(;ho#GhQ*@6j!*1{G^9 zn7w-H(raBi^_yx_b#b{e%hwk&@NoT^l)23~tB>Kszb;pw*LdRErW4nusxyr`1U~J) z@uz*a(f#6wmwr_#SUV*6O1sJ19tI6a+J2#Gw&k)0JB>azYnOAtzx3K~-8vNdRrtqA zpE4iY>^fn&QQNPC=l2r*>O4JHZI=XnOMdnV!=U;>WT5ObiHEd$Zn=guLHkNlh z-z7y;8H{0iaueN_r zwK-pYTp9bbtDkf26L!X}vkSY_c-(yINXrdhcczahWL~fQ&SOo_T<<*SR;u@meuH10 zT3BgKG4*`UQ9~}jtybz@CEHxvofG!=PuVno(4h>wK~=J&3|4mQ_`2;zqa&@=sTxiC z&38rtfqSPO8+dTd{wHfw2Nw7?qhIL#aWgWT?rx*mHu2NB+Vj?LxNo<=($)J;Jr1mX z^L_rd#jW}+YPP8G*|uY+_!Lfj9p|)HT|GKuXyTSrH3ue|z1sh~MUm(g@ne!Q4|jVO z;a{!&_vX%*4lc=D^67NTEq`AwIoH06)#=K6u2s!Fkkx$ump3K_Hh0f4Ib=3tPMKC0 zw)Pnw)2@H@`ZHZGG+9w+$HD$vKOFyXd~v7dE1O%*+}-Oy&uNt=%(~cbQTV!&>E|~7 z3>*}6qEzLtKZ~?`KJ&<;_#Yc)nGDLQIiShK`^~;2ExEX_K;*pj?X8=xGA&xA+2!50 zW|fY1dU>ejfrs@Y(`~;FnC-lLPQAHh3-`>n8M^h>7t_exYh)S{keAw!w)>KbEreXYYjW}?sKGV<)NuZ zJ`L@Wcjo|Haf0+`yx8RBWl+M%Co_HOG2cwzj?WTShF1u>lH1lWH~V>g>&&cI ztKT=fl3UC0>5a>|UI9OY3VfJy<=&_=U6)swacSDAEbp`us?zHUZ6bQ*Gb8`lDF$|Y zGd)!X?naiLUY@bEw8r|W>88Gm(sD~XPHLR6yQ)KC zmD;gMMJ~Sf9pB_+rqzI!35AOlbKX;+?1J`7*B0@g?`hs^!p>uvn}e)|Oj-0Oebe-? z+8qWN555k1*yXOz;w7n9B`mq;Xh7Pw0c4!eFZC0lAl4;56D2FibL3>M9@o_MFdG5Y?Z>edO zhF_a>EU0bP;ZG;Kd3dGm+1nv&Y}SFSuZR0*kH{(TuFj>8Q?tj8?O1!xvQp_4-c9d* zWofEg*~-)SQ~T5xtfcCu#s>G1{uFI4AJMkGwFwJ{;P_=6TM_nW@A=(6tF#S%qhW{)cIoBJnSG9tY)gxzVou;pS}zFIW=4AZ4rh`E%tDoLv)I;=-YJjXh^wY27w#!(T;; z-@ZJrndOJE2II{a`?`D_SoT|t_Vb!zYtlCiJm|D__ZgQ0#Sh08DwlDz$nfn)yM#As zlaf8}rPHeZ;kyE2Y%(VNIx%U&>WuL(y*^9}THE(cbS?Ye2~S^Mb}{Jw*S7Nw!&8$C zetmzmGDe%Q=~9cr&B`5j^>`guv1Des>rdQ1l$r84_~g+B&NnJY4p~uf-S4q8axK@oa)%$8$wQb$>&tSyw3)v9p)r8YA!muz%*tIOz%@lSKLYx-8}^7wwH^P8PTrW~JF zd6DO=O-G|+P==&2d+i^2( zn*?88m=u`$rIp{HK7+2huDo5#W~=Yow`bN44qxH4*SKhjCJpyBS~Rs*sanIjreAVD zI=gUs$kcC9XnYTJ%)Pj{`l=|+>$53ATX zolkDy(BZ(stO0dDk9VtoY*X^_&TFUTgpcn0`^oIFhmH16A8p_FL(Gs{_1{*nq+Yym zP^0%{B2NTmM6USVJz{M67ZW_(e8)P>uC#qc|C*+^f&3y2i-+hs=wtK5Xtbu!qB-(!2Y*Mpy?wJ>uPCq+?b43M>3Fc77=BZXVgA|Ks4A zf%R_%H9j-Tq1=lTlOB(3vf6NLrIVY>NA|m0uIQ15#|Q7fV|P8G`?hygcbrI#a(K9R z*>2ZrOUqo?*d=<^<`$+sJGc0Bt=JRq2Pa&k&RsrI(mo=viQnhP1(TwUo>x0M&&E6V zcejFtbIW~QaH70!7|k0;N8ta^{25=y^c@tjS}Y^FpRIQ0=IaLs1~fDMn0DJ@BSYUlpYZEcvtCl*LHX&cvg8(=S3w`_f3W!R#`mnezx(` z?nh07_MEPJ|Ix742a8?$6|`~32>(VsDu0MdZ1>^w-H0=Nu6%8p*7H@)ixHEeDs-JQ z_WB^REzJYZ%#HorX}fB-`L_K*oy*n#Vb=TPeX9X`95;+|C~;=O1g|>hiVm;2!~J%_ zoh!EZEGss7Sn}GgBO+4}`((Yl` zg9r2*U13Mqur)uL#(t^X;QJG&O^N=);*PC29$Vq$*S|+p^jmV{`}BdiHT!$i8Q->w zgMYxF=B}O&ze7qjvTiXj_qyZz|A)Qz4vS(58h)3ghzgi7FCsxj1WcG&U|~^`fEihk zEFwryR163Rh+xjBnDCf$P6!6fIVX;aiaBSGw`-<)Y1_qfzW4p^^V~mfKP)rz>*}iR z>YkpS*u6GSnLA4(VzxhGeD3?2$P|7`CaepI})vBC*f#9p{%yc1=GXn6la; z=XSHx?UuK%oc6Hl!_{%8jRrWcDbQ-|-p@T~Z1v^sbGoD&uFnsQy8pSR-s>DKyE15)>~^#%aJR`)V*ppe(J1ii61SeEHy8gQ$5h-TZ5O)UvAZ{ z8<8=sbkn=Gxt~pQbMqtf-%PVKlpnq`cbx5*M`nwRjJsK_xLnlhv5|T21AE`bdUv}N z-RfCgi+6p!heXb3`eakj%t^IBUwXLGd-9Wvh!qZX*3QU#o^yXsK(!HX>x8}E-ejS| zc-NgJb+*o|``~!m_U(ITyF5JA__EKN(~Ek(IPi7ZRo~Td0nal`!)GknIcL~MqrV-l zA6aMNY}`H7Mrk%8(JHv@`wkPIcAmU*@n7c~4lVVyA6nh;_Q{o9YBf96Il9M!d*iNe z&EHYY`9sXhmRma3@)~sIVdaFw+ud{1o{eAT|L)l9@7J#d97x>#e8j+iZn*}v>HYD2 z!-wy8oyp$R^kQ(_PsvS>N%1nD*PjnJXpz;s&R9Li4xcw0#chp07CA92{n6joPRkk_ zx-0I+Ola5<2xtzJ$T<2BGFAehhWhKOpbiRFCYi06ZvlZVO z?lM_a7;#dx6ufLAGJja=XN=9;7X8q+F)8@pI(N?vo>?=`{!# zSufx2bmZ%~HFn+^)@xL-`RvgtI&XG3x%M~PdF)zAAG?d~TwPlfw|)Kn^~;raHw~5a zyfEX~=W0vF4!l=#!7$bI$CgVsZ;ae`M;bXU|NOfADP3~XKHH~U|9Giq+nsGbO=!L0 z^__iQcXq}ODK(Y$TNwOschj5ouiQw#_qvm|{fL>}v|DV~I;)6WHK5dQuH55?UU0^Z zgK3Qy&xkcVm-|nPp0iK=H88AJXWPqVvrav~-r=u#m8<63~zPbQPs{lMT1}V3U8BcR(o*H7OiLNE}kCj zzH@+Oq2;rPX>aV$PyE{boO6eFf0y~xba~eN#fj^YwvQVR?moq}rnBjXDe@EAS8fmJ z+IqbH9i?-8)ZV?_dp&4xKILjdtE(#;TQvMSaZB#U0k7&ujJtF)w^QCog<1c@jWuUk zCJef+>}BWkVcLX(BP)(MZJRgiW%r^XGTU7P4-c;m+CFIZcrIKC`5V1#YK`jPt`2LgK<&}pb8_(X7vm-oY^1x!#8Rs^J+jX#x z@n8MwcDKwibBy&CcPVS1T&=^ihxaqQN{5{pliu}hm3h+=wAUTDwl;E&Ril3EC-z9` z8}vOh@90ratqj*b-&cNpS?NLHCEcc>=Nk{dUu)Yg^R2Gq`X9XhJt|@U;+#c;I-G5) zS9|EYZeMIZJnc6t+vH={?Yc+pueoY&sd8=bC9h_KJ)VT$t?a++>&AoQ-+p`TezWuP zG0*00jOiWuyvI-Zk(b#{>v|3OVxHmmIoqsxx7m9`J{C9h3vBat+teqOJ{#XW+{8b- z#plD5{c`tg>0|!0$;w05v=pnO@1E}L_jrnR>oKpaN10CQ_Txust55x`hK^{h-}Bka zDmG($UPo@Lchq`D{R7ipAN;V^boU_BO)uq@mmS%2=V5Zd-PnUIx-@jH^7))HxMh03p6~Q~Y}k3J%2B`g)+TYX+UI&G*56ut`skHfcjdkv zH`IQmf49}?^i@UXgOf{ohHcvPyuIUyolXZwBsvV<{=QbzoVSq%qdq2$pY>`{7t7AC z_I}(IZDaV!q~Y#Xu`QpUcz<`!*)}7J2PMQFOkG&Gv8%?_!_f_*i!ImB9+kdlX^FDQ zfJcLLuSZQSFhBd_f??BkJJvh&|NCyII?DVH_YauqO@4YbD{Jfzn>qiuesZe1F4ID1 zL#rog3upEGIAKUqriaP`71S;>ONb z9hEV&!&+|b?LX_~#HC-Znk=gl{J}#byZ4PZdHK77;%@gd9o%VU*FKFy>O3wsZeAlG zE!c18=7&qxeQw%5eZ$kKdsgr5cKO!V(i(>=*Bx&2K74IY?GM@xi_kNbX!|wL%&C_m-J+p1|g(`c!Ucdb@=}_Z-{x7YDIpjD0 zR@^CU`zl%embH4WTRQpvXbtD5+7lNPw(os)_N6?Z34N~QSH3mf>VfC1YiD(Qe2@OU zZGMQYOQU&%sz&+y-n%u`s*j@SqG)SL^WhQ~N5{@m3-ZC+;K0`8^Uz@-(y*W)sZDz; z8!PM9wk>cCJPVg;rQ`>X&5|!)lJ^)i3V>1nYv4A0g#d?>5{)t55{*PdjfT}K>rN-a z?MNDYT~C}hu$R7Jj;2iCC`HRfU#dA+Pv1zUZ|I<}E3X=$uUm9hp)XC*%Fz_&81p4* zAykKF`W*BPr)j$A8|7%p^`+U`a(#=bI!^l5lPkIE+oe=?(vQ)qr=h2BN03b4!a-l^ zpl?Ky5XA~%TJ=0oL;}9LEqpI(yNJYZ$&*}6)^vh=Iq6G>>l-=g8_M)`g?D;yqa1zT z(U5GDhbVQlK7(=v#Q{OG#eDN|V?TQ96G`B&N&J^w2lThWekX?WAupSx2UCol=QZnuUz1 zZ?Vly{~y>?5Ie<~t_b`jv^n{&ZI0NSTuBCP&PCr#a|~&6y?<+RJ*x`sm(CM*1@k{m zlQhcT+OMF?N#9;`s9Ig_RfRSffc+gV1Qw4Q!lUiP2DHlzWcX?ySXLXLc-oRTzdq!K zl3K#v|9dhmIza^&r6s(j#X+*EC852;uW?$x%m!8-r=^-9zMR3D)-SWZl*ehQX1FhB zqC9xw_n;-=-y{`fwCD#lxFlMt-s0B?E!7)ZJZEQLDUZ`q^?>CJ*0g?^O{+XkOCl}b zVAizgi?HQnYVKqj4ShDuo+sv)Qi()(zLRjiMT_&R zRF#;NiwbK!g#X$SBiQkoAQ#rF{#Nt7kc;-q6Z95VwN-Q2(-`qvI8`MbDZr)pjTmj! z-19U#NLZ;V%s)*TEF+MM`pRSA{dn-huyQ1x_NIK1nuPR{_sYmgp@2D*ojuRwlU1w5|;zNrEZ?V~1he5B*j3j(L3%JO-R z$|}(Fp#rW&#xXUS<0jQuSKtP;xT+(rsldxUMB7gn!po^DnFQu)R>xt^T6NrC&|ltL zNm12sf!9_kraVdD)GHFz5e4R&t;rmJg)u2fy;9)i$6sN5OZa}m)#V4~lem#Ln= z1>QuJA9@7+AaI&H9Lo|Z9ny5v>91P>Z%KGf<~SaSmbwTWM*3n9#>!;XP2e;a)QI0bfpdO=kQcjBQDNwhFwZkT>{8Hid9l;NsB@03EAd6u6}* zMEJ9PZ-kp;C-`S02-*l~ zY{cY3DjOl0jgZ7fNNpn|vJq0*3X-;hq^%%nE9w!n+X~ujg;cg85z^WU+H5=h_S(5b z>Eh<;AahfCIy?KweU-j42RFG=6cgS%C+~NQTf)3qIAaRD&n|(Q8DmVv?O6T znU5lpOb!y1K;8%+86pV^B5$k@izBbalL;~yyEye$Vq#R_F!FMyBr%c9VG*1VBO#MK z1(8Wy1iyxp=_{1M(MmFb1x&?4-Wd;31_i=AGD>0c5f zlmR+}!XuTTff2))48e3MWachaeJZ2KWK_azHcFW1CnhjH97-58Dp*2tN?y>139F(4 zOl3?6)F-W)goKD-Iu%;EEWRjP4$3Df zDre-21Z1`^VVj*$tsKCNO5(e=<%z_3huG{BB|+f~NQ75ogBiUrDGQxj>{sRNn3NCMBrap&(mR22VBK8xY3-M<*_MAoU@9SnytH0Apk&dOg# zP7WgyV@CcN!<#Yu4RR=#Gs7!m{||ah7+xPaFHg_(Kpu`RG@lV<9RkQJ$dB?2fFuBS zB|pl~0VDxy86fcb zVY~$a^26#s$#4%6NApw4a8HJRWjJfM3OJGk(Cr|fB7wCu41LdWRJln2d91j8X4y2wFKEApdy z8Zz?142L>`0P-;EqVjD4k^uRKj+t_(I|v}(n*1oI=fNO9gyDK*9RkS1)etH#ULX10 zZ%{`NKo6Tw*3WD^hjj>`ANB>R-xVMUeEy*gL*VltCSpOJ%|Gi;Ya&DS{}+GSF!KMm z{7J7ULH=zS{j7i3c4$vzAn^W#wgdt6K;CG%AT0#opp$a;xEaClVPqWw$hRjyDjy4w z1m6ECA{O{ivQFivGMug7JmjDsuKH2=&B#Ij5Qfv^EXcFvI>E?CGV<3L?#ghut^on` z4&2YmY&-ACaPTd)+ndqfk>PNi0s`cdEmr^|57#oO zo-l@YV)$4_51UUq2Ri7FV&qpadf4)=t3b~#hO^}by%4}}oym{p6WS64;9ba%a#)7| z>DYYI_6(fW4}B~I(9@Ots2(Li5IzPzUy&X$+X9A$93Rvj5~|7b?C?O9aq{AyE0{I^uS`xKQyrsE5`&mG6%ZOXK9PVSZ+Dd?QxKe2&xd zujF_Y%;#o~S4F$~IbIX%v6$n{A;TnG78j~=Iad}z3CG`Kd7pE1$~77rxDO|9S6ehd`$L+a$~8nz z&m$;rhV?jx)AJYXm?UI#oX(jwljFB>hgrgLE3Du39M?yG?&0`z^qcSz6=I*}(;CZr zij()mdc4kYE7VWVYp9-3^ut?Dz832L$#F;Af2*NCs2)3P=k+;01m(>*e!7a_PDxvi zf5CdU=lBLuT;k608>oLE$9+&fp5x!qpUE6g#d4BIOoTwo^#=X6h~xB@1Z@{oz8U&+ z2Pgjk{RwlAL7?*OaKAps@w1rE+Z^AC?W&C9577@FIo=03%-O}-rQg^!(s2Ce9TWrj&H*Hb>etF_7A-|-VwP!$Bj^bB**um-$rv>irX+p7Mss9Y*%wQ zo`L=0MviBr-6I@dhIU~NDpr4A^v`*YTVZ>+%JKHJemTAg{cxY-X6Vl{jt@evf$3R-k2T3D`i9Pf|gpDi4( zf&LUeYRLFM9Lsy2lkbP^_A1AzfA4TS810sE+#SpNf#ch7yrGHpN&V@Gely_sP?T@P zarhQHgccmPK|LKfu7~>_%%KE<+O2~Ar|pY!Z?rp@lP|$~8Org7Sl&?_H$wSTj?cn+ znZfZmY|l$LJ_gGLa}crl{|EiQo8#}GAS9gPcoEk7b&gL%f0lBb*4JB(o1#CfVZNx} z=s3JS$A4nKZN_m2Y|m{uUIX`2dyap`_UX>?#kf6?7WAJifqOEf8pa#$$W-=XfBtpGb~ZLc56^ z-y^D(q;Y%z@|hf$<9@o7<4@85>p8BE`G>h$Akh3DMgPDYDlA`s{1V64qMu7R{uLef zg5&+L|M|x8j;Oy1?%y;&^|2k+?dWMe2Q313GJt;{0NjA z%*nS$e@1fL8{6|}j%Q#$kk0WC^v@iQ)APy|9H-}%TR0wxb`NoU3;N*<$HTC_-QxIj z^k*5zow0sDar`0rO$QuC0`=Qz^iM60=VCu&%yHWP)AKE=#~t(2gOjHhN_%m950?*^-yX*8njG(e`@0dxZE?S7!tv`^E>n)T!+xe4$EVP73&+>swkyZOQI99b;kh{o z^tuQw*9F{<266IZaGPGoqVj7|4~?gMGwPYh>FJF9%w&#_#P*!c@lrhQEaCVG^ygZR z2crI693P1M2*+EXo(mk;!g{~U@w%9wXB-d2di>0BIv1!8?q}3LwXl8G3zwkQP0%3zo6X7RV*PIB zxE45sgaaI3jsCpM@uk?GzvlQ^QH`WJ)-(0dk0Ymn<>xxo)0{|tFej{l9k9>c*d?HA~MEvlzEwnH;U9`w`qfLbve z^w8@P9T*Ps!KkM*$LV$29t;OP^gdKihJ&6w)Z@YN`N;h^eiC^A#|x2%Acsw8Cq2jy z!Wf3bp_;yTlg{x1)U%Y~pr;ooAz>HCgG4O3&Ttq479fAjaL_~Bj}Gn!G@s*8zA?v_ zA-82X=%?+cC&NMi5tLUloGtG#hJ&6nD4)u3&~qO70_1dmyoUYJN=BYOy9i14aq@IO zI>vCQ_mAl3lN_h@a*pB9er#}@d6VIwM~>sJyBu$i{3XX{BY(^BDab!D9PHBlNE7>Y zx*wfE`AQr=jGW%Lr+%aRQC&{HI*unBa9jgt{AKX#iisLfmT{%wgli728 z9LmcX4r3)x@`K>PaL7+M_HP3i4*A)PdIocx-fs&<4$T1C&u;RA5Y5Pg-9Uj#Boi49 z`VXUidfcY4|f7N z&8G+E^D4&&BY(zl*ngAIKOYzl{+xjFdN_`QO^{DP`34LJdH6O1gmxUK{euU`Z;9d( zKaSH6HIC99XUQ7`5=yOM;^m)$j@rz84L&gP0>G#8P57)1H(bS70MswIDOvzCc{}h zPZB#ZhI3AO8{3V_@dLf78m<7RJ3?mQw zdn>dX&v8@aqZ!Wj+o=o(J@k9i860;(J>pD4<>jUQ_Iyr05#@6^9)o->$Co1C$ngcp zw;`u~fHFWRW;h&L=ze;U;ovuTHWb1;hC{tO3zR%}2nTf%sQ$jPc^s$riC1x)-p5_TaeCi(BXaOR_)SWF5Q-QM^=l6TB%EP5$Xg0j zA}QrK?LU8T+#Thc;K>!Jh5DuKq&34qed8YN%QpA$l3n7rw7z1nUAZ#y z*u@K)A{cq_&vz_W6vsazk7YQN)e6sZ#xWf9G=WT!kj!!V9NJ`tv-LiM;h@JJ^~~XT zXXFbR&er=XhJ&6klwZSfdY^kEa&Q;;$C&&e++*ZH{|s;}2~Qb$bb-J%@Zt;gKfOO* zjp2}=g_s{hj??F&>o6Sb();yNhJ&8{sHZu{cOq}WaMsW57!G>wqP!i)>HYF<3}^kU zU^wW}z~hA*$G@Te-pIkv;J4=F2Vn@qLDWQ`63G-s57-qtGqRY?$m69HA;mgQej7IS z9Snzj();myINk*Fe~{s<--CWKzhUITpY*wh4;=4{{`|^ts7HETLK|O}fbxQ#Q7Es=@e#=D zaC`ys`W&By+=Aou`6x?{Z%26>|z&t^F2d5H4!Ier)UVuphrI!;^7aL`j3uP3bIxCZjg$iY8QUfh)2 zW8^{qWgH)tG4i;Jl2{VJ7zwnUQ~&5O99AvRu0F?`Bd^79)<2CI4y$sMH|BT`gj|W`~&5s{t0K~LH`EanPM4v@C6Ms7_x9H;jye{$Ri^=N}J5@>z`ge@{2uwpEy_a_YAOSl(5PJPzfB z6vdppIbMvp$jD=tD#({{^0Z#wF&zAmi{<^q@wv#qGhAqI=!i}nO^c11I9>?i@ zZ3BkGDjjdvXE^A2gL)csoIan_jN!0K$J>?+2R%*EpVl0&hrB(*VU<2d+JoVs#}(xr zIW9-8U^uMO=V`qe4tnTwNPRgTh{~)g?6{3ZH81zSpo02vhpNiat zHp8J@gYf*T z3By753+iviar*q8Im1EEOFYT8VL0figYBw4$Laerosnx0S;+r-@`K>Q@jW7z_;UOZ z@<5J%Kt6`yaGa&*WoZltyN)P7ljH8lmvX!x^7R}KM81dPLy=$Lcw-GQ;wi%+pEP_x z4%GpEwnBeaZYD}lPTyZOGw9&3*G!-G|O zj??FYI&oZz{T?<`VZXyuaKQoXL^lW=FSQCJp#r|W0{*-LZi@8-3s!$f1$-ID<*1jw z7fkgGME;49$6EwKjI*XF0MRUm?@Le1? z7qpQ3eGI3^YKSVWfa~JP6_ksWZ^3Zv$^{F~74V1(_-uw_*DUC{!|@1#lCd0)uOUAm z?}WT*1>A|@SVSR3Fvsb*Vm!xNqyEJV$6ZZ`IZ^?CTLG71AtC>)o}Lx(&fT(Pay?u^Hd(yNTj#|oGq_4!*LfE zh&{vE`to8pHVuIUFdSVjkQjzz5d|`l;cUB|&TzJ!=P{h!A2%~xAJ<_~fa_3Rc7H$5 za6?>&MG3BhJll`GW;olg3I|LQ2l8xxZU}$`Sg^+tDZ|<0uQkKj@qj(U@fM<>*Nfps z0ulcOFq|FFP+vhbw4b4v1~0=1`2?PiybUfv9@+`DXU}k`M+I(sF&xUZ8@Gp4z(+Bh z-7h9FoNb@k3}@TtJchIFa~Z?g_PLJXZ2R27aJGFGFr00lCmGJR&&v#_J_mG{;cWYS z!f>{IzGXPuKEE@ZZJ+ddA=DS!KG}B2wom$=1jw`PlRhsDoNb?N0FVF+wtaSHINLs* z7|ym&4~Dbtvzee>JcdZgI&D{G98bmVmK?u|+jN`)btol2S{_@Br#u1k(ShT|$n81) z4Y{1-m2rIBljF6Jdvd%1a$k;{ARoYSOXU6>Z-+dH@ny*8aGZXZXCcQoq5M*g z??%3ofX4}JQMxJdCbpM8Ewmk%6N^fmR@7RsPL)oVh@498O7~HV$V8_{9X7WI>&YU*`?rGoYFyfR7Mn3Q((%oisqL zfH*mT+9;R~Ky3s!Wq?{4VfKIuW6CEIP%DD-2dIrJpGrV&v^cAP+9;emKy4hEOaZ15 zP#cFc3aE_}=LO*FLzo?ahd4*TZxf>lgxbWBKW{AkQTNVqaIHZUrL?8^V^ z0MC0s7lwx&l|Fmb+OI3v-Kn54Sp&u%8z9mi{;3IkdUGNJ{eDI1*<%Z+aVs1R$;c)g z52=0H*ZL8r&VC*`gx>cPwf%Z65?DX-Q)fTJQB1+whvP19KantX_UU`!>@fs1^5u^s zOr8DLP9RP0KZ^@tP6jxR^Y*6^2KM>-JBEXYvzVSPKo$aBuE2G^{_+rX6Qs-k@%dju zqSWPIS1zUrz@##=gz*EP|HFhqUU>T@2-ql5DzKl2_Mv}*0QTYfG;e=x1@>d`q~YJn z59hV&%5UTXR?FXS0{d`JfwzCG0{eMr-wc#6fiM5z3hXP;KHL7A5Iwy8(hBUKNBeyl z+px;pFR8$O0NS69>1AXoB|qLioX8Lrztz7ro}5LJI5zP1;kuYQ|G!521(+D@HzPmZ zexnNP=b`<7D}OzbUY-5T3ehlI{^n}z+f`tno`e>GbC|%FAHHj?&c3~?mgaGA- zw#1j;w*vd;@xXHh5;K9f57+P1**A0(6VdDTVBdoLc>6;uu)i5ks`+spZ+}7s>FM`h za2TK}0iS+u1?kVDefAt4l z;rR$iuTqS{uQ7cCZU8~;Qk^GAdUf>|;4VridWcJ?KR6cA^t4~OS%G~+yczgI$egGH z{?qi7yr?KWbe1Go(}IedzomatLH_@mDkdnXBrYNUFh->IY5p}y15@Wed#RW%L|mx; z)ASVlsK9>sEK$OO#6tl4?Z}U&r}klRw|KTS^od~a4={Y}RU)p?Bl z4kQsxPwls=z`mWaXkp;L_}{Do`_}Wt^!;%Mg!0>|u@B#Khw@v~jETu4^G(H-0*=#7 z!DmFop@Q`Nt;F=*s37X)(|gnO7*7{-aNuP@0WPUJx*UY*IWfrre&EKXr}FT<`O0Ji ztBGPs0j9TB5pjj)pW+m>&!>Nl>F1MpHt^}klX!LQCu6ag%8+S4&=t`1wEch|c-xRN zqb>X=-{rvLr~_G7mmY>0Toxgj#IC!rkd$(f)bC=fcx{vu5u)bfjTe}AmJNJ*;Ywmu zG`T`$72QTEmD+UdVBNk`+qSkgQmJh_n+~=e#s4z4URk$F^HS7gd1inLqt)fAccC@E584%T(ceC(7NWk{LaNRcJ%aX;;wD>9!98Z7gZ z4VL*A304(pKjXU){ofQ>JupGmSR@^wFs(I`Wh;-JdLvY87i_y z>4I8ew>4-w5d>@s#1zy+^RO?V+ljdezTKj0$qertIPl@c;>CvFUJg6%G3i~Pu7|{2 zk*VWkB%jjELXqO&ntUWIPgh50lBSc!a(n5jWW7~VZzsoEa_Ji_3-1di#{cL9hCLej zYV*uip^pzN7~Q4q!FPjB%n$an`H>o+_k6m`?k0_5HIkNS^;pG*z4>fakTAacOZ!BKshhL}FCX4+<6hs^_TKE3T-)~BUK z?HSVh?&*HYj5{kz!y9k68kjax5~US2&-Xyi=ZJ)}t($B#nfZOA-AVULOjG>vqICFxlQGs?bBXk)xMs%Yey1 zZYKTBVr+LUpH@e6_0uk$W3vk)ruvzmcaBN<@W3OW-P?n9Ckm?fxY<$bo6XlM7C|QM zRwy(7j>_Bey=8u%1>Xa&_SG%SQRt;uOqf=Gj?~#cvg%3Q+E;Q7Dk-lo9C)M7&itH- zbsJPO&TCw$NY_kvsx*9?-tr59Px7YR`F?-gq6UdgHA+X+2s5=DQm=n*jT_(RS8n@K zqsy#EXVa&@J^o04`I9EEDjzsxUsUgS&cp?S3r1xudGqk$tK`Q0gSNf95p%}wmi@`- zf%|gG&g5qLZP&ZC@_g>eme+sQGkO*9+Hb<`qn2LZ*KQkrvvuf|mp4yI+emh8d$n>z zu+^E(O&)m^kDVWOW9_o`$BOL6dM+K||18p~dFwUKCyNX=FMRE3`{i|9H-(Eu=)2^x z>wnBN-ca<9##N8LCxm9J+DGYJVdd?y&vbW>BKvA-OW!$anqA8GSy*j@Z?W;H4Huek{5+%Kz94D;tj*J#`6Z7&w@hRC zfOFbMOg4M&*qagls@BKeLAQ2xY(BK6e61>^o>mE9qe+^p?#2P-pGO{?Mw#L?0#iUuGxvWFL%%8g!I1I z{^zHMgY~U%l#~qp6xvR@uB&9h!M1aYN|U|}eAuq;xAzB+?&sc%Ye^8OKmHLnRojWVQKy&b zr&t>udGl)Kv#o}W^n8j^wi~sx?%aNQ>gIHZ*K;if79X4W;j;33?4k^dJ*RfuGPkJT zXj^Wp+lBilW-Yk9=yHiH$TR#@o9}^{VJD_0zxx*O(_zJmo&OY9x*(T3c3bwiwo|P- ztv_VcJ1)JM;q}*=YD?PH8G1uwn4i%MCSP8^3>NUFX`DC^0 z8j>EHHO$P3unr!!;6N*cNY$E5wPvC!=WuiY^#kdrJoiOu$_y8GJl=58reu>nlEzia zbb^a_<~DSkGD726zV_a{Nh|a_=V>f74jgHE*1@*n^V4I$b-!8N$zs~pIyEGZJ-SrW zuAO=N;o05~zh2%`YS^@Jj_>A?Z6?lf(p~dxj^~)X{L~NgH)ND1-)J@H(CK{Nu|cN_ zCWlwQr{6=7p1Hy9^$mmfhs$)n&E5Lt`LomgHh&8<>g;=K?D($(YNr(*8hxur&JSaw zUh~erIC?cHrub9;RXs|rj^#DV{^yNn%R@KoJ)2x>#N{Ii^&0G|4Toh~uGcuWquYtO zA-<<#A4m-!`n=cb8MSpsdZXC$C++$UnCmyQk-7O4X|T_X0-fT~Nk8-*T#LWnuc381 z{!_c|RZmp$S+U)3Mf&=J;C>GWY%%**$EtX_rFG|hCX1hRiWq&awM+Ny#r^eKIaIn5 zJ7h+UrdHwIEly>w)N%<;>d^OkjU4wrm9AM>C~ozAr@!N}_rtZ)c^|Bw4qkYFuUW{9 z9$(yiOyviT)Ny?~?yc{)hqk9icrA+W*3HO1@My{5_*$JN*tuV9w>g=U`~Lgs zv9_b`%-*-$_`{`RWmCqO8eDk);G{-(bIpv=`jrCD+kf-7X{Xzw>yk|k>p3l4>2I>a z`ON3iM_boR`({LV+$r84>N@vT%SMVpfrcHUq*_bc4?k~_ZKt^FT+nYqEBB{n5!N^E z51CQt(~~}B^E_{T-}J(^%_j>>TX(a(pH~XEY<*>(6Wa90_4)6I9awbt?$j$U67?RR zxm-3b)HBp>eE%M66MvSRxvKn7Idl4Ahgu!ZcpeBSk%cWPSURidMAw@U`m$$V+S}bK zGD&PAk4|eoOF3yz^wq5Od%dhTSf}cE4{VgWvRkK!D>>Ed_gFcdo^^A`=CQr6u3K67 z#Cu2I>~jYqyYK5}t+c=Y&h>V^eHI4}WN-ZGH(2>&%KhY_Blpjm(!w<`#O1`RoXh*) z9-MaHvG%-o$>aJTH3Bm&wZiCvqSq#_rDmhXMOShO<{MMyqWYcWMR7j1GG=)Ms{D_ z_EzBA(bhR%jK56S+0x~w2_l7h8cH;7Fkzwe&>*LbwlfYMq}rc z6c1m1rpMu}?t52nnmPC4;A*x{=A1P@FH_8!^dia9c-M9Nz(qy}#~Ugb&EmPK1o_}xJoe`U^?u@N{a{Xkl ziRW^MoNlJD*BERwGV4rl1FieYt9O>&+0o`_jp0gzF(L zx$Q^B;Ti?FOA`YA6St|~;QBKJjWOO5jYLC@hSe(TPA8FYUWTXJ;&*u*^bK<~d+Hly zYsvMcQ?;G+Ehg*8^sQ4WX}arM$n>QS`bI>=L7$uqzSp2{eX)cWNqT$Zy>S+E?g%KsYRno~`ty!0BVlyLs~6QqRmdUYJ`U8&<96>zv0 zs7^kF@S2RDg?c7zR0Z-G74U394?Tk{|Hg=fISC75Hk9}PhtkE((?RB@^mKOik^3rr zWe#p~r6ekF7`d@7iibuF3sK@D9f^tLu6S@lj2IIHr*@wHH+1e*7+OF8Za{vNcLGQPaPTMPcuDOydAk3!B(m_V50wX75I|2Y z@}s;lKoWS)<_Gq5DsL@Fi}Eb*h@7`eTPo;>%ZF4CtvNWQ$IEJh2D*m;huv%Q2$e|f%%bh^8P6A#&I*~ zh)5VHE(G16>o4Si9EW=V5W>ZUO8ye^D2~%_a*gHqI?U%pj`zZCyrikp?|^(hCr{@R zT*vY2xV?wtBd}ZtIX)Hhf12Yl$S-l6ehc*h$MbQ!jN|khg|9dsj|Z_29DjxR{KoOw zxJ@s`sOlZ_SsBw&?vLB_8=;iP!GK(N%fpX zIr^+2<-L#(;`A)QdOWduI&i$n$)}_K#~g=yP!Qg8ysL;M8W>OY zpFz9&96yff8*)4kx9PnAst2xJL!jS|rkuXp>cHulB8p4sGxJnF4Y?mDPyHUjaXrlE zNRCgy{G@RFC~i;ZI6ON6VKK+=iCD6Z<8ZG4!VZqV#_dBKj}XNrr#L3pC_eXj9tUL85VhNqgk8;{>nqWC7r~4J$OMyUnJ8@IeiQ}2*H#x^A;C=!3Bp^^d zw7g1=SHkUK9G9bhcFtPR)eXx}zr{`U)BMmm@+db(J^7q|6Crox8-N^NkNPiiJRaqr zaGdrRKRHg@hc5a9vVp6DCUe|?e8S}$dheBfvzKx@exUadsh;|R7V>QbMh~R1#(dB@ zUqK#`kRpPS2fHxth7iSZD-la#kyAZ1pR}GSH%9rXoO~SLvktimq!SMsgn{vDqxe3Q<|7L+4>IMAV z6y@zXPV1503#IaKeg+|ilkY8J3H`<_Ef?J%GnxEAoC`Km`i?!7r~B7TPW~$T5zZ+f zKz*7vO2gTjG++)A2OzKPYbw86&|~TnKF( z^o&AI`yncS0=YdW{~fuU<5jSe=*jWA$UQmU6uB?Q>G);<$E{JGwjY|$j>v;J`5wsG z^C>8APvq?R6!5;tV>vzaUBOWtk3;z+j*my4%yBwy%t8)sL{WiMMZdy=-CyYV9XQ)g zq(UlD2dl@A;jErShO>J9&)%i^_2n7imN&h?!oNi!Mpp1@4W@)^BkcPO|NlD>zg!c6 zDznFW;}_UmRM?oT@ofhp?ctxAz^6ASGH|_wZ8MO}9{#Bbe0m!q1B~{wG>JW~L{flD z+V4_qNmig;+Eai_`V1Rwzi^Ei)addFUib0Q=DQ()2W6GYL~?e>XO$`j`(QCIp(EZm%LM>e63_Op!20T&Vu@ z>GMd8y8IX51;M{CCFCED)igcL|6an>*{9>G_L!b7XnMN6gRH2tAKp-u2tYxwZ$N%D zJ+*(9FwR9n3Obj3D`BhrA1`}?M5(i1jP@0n6zs$GVcz}&!qk=jJPwvhF+E+-^mO|! zSy5--5Dbz)kL$1rV@8^umj4xDV4r`6Z#Q-*t14)})DK@PNFPvzM3C@NT&Vt2`!qdV z&k*dZB*c}z(AY_^5ErWdG#y>m!t^QuT%mJd^PHxm+=Ku)zu<}~nTYnSsUR+C{weN) z_WAVm*|`iVSV8&@B)z)&vuh$IP+&T!KiH+HeOiA|9^N)2vBQ4D5F_Ef1DsRy>7knV zV-G(Idl9_1t9 zzdyrQ@}63!$2og){@-5W?jV5|eaS1JF$v_I(Xd!aoFqOXiUqDWgIH z;}T*+f+ZyJ*l|*@B5`;1Q1+Ag_{s&Fz6x*8J}wHcKE4tcZ@JuQfYPClV=uX{tA~q3 z-d`c>O^iAdXj8Nok`}DG6zp@VcpfOkGEVRmG}2@b9HnjsK2w^TPgSU z_Vo6VNc(v7^6>0Ol9#Ijk<# z?>=Y_c3=z!Z=;kJBR|%;S7@b)91=n z*KkdwaFCQn1r84kBS*!Uh+uNG5PMCQ30-88+`~x_C!altj#5U)MubHS$FZ3jJ=lHY z@s%iMjsW4&aq*pFqGRJRx#Yj3|K)fjwZOhSi5{1d$`4CqRp`o-NWMuotrWA8L=RC( z%n_=En1lafzx+rwR&@ZU-%DeQc6dtu1PCgzX#bJ*!YKU|cc@#$}3|bQ6q{4uN zZc#V=CqJoFWH6ilj`O$%vYJbXGPUO7A=YDh@zpEB@o_PzUf zc#sjGw0!USH`|Ppg(tp9##NK8|EO z?USV1e__n0ba(afAqB$5s@nANbt28TeACs%L*|x5U8m~Of)juqPoJzzgLlT2RLV}4i*!rO3^8eZ%oIE|`|KBNENFq5$ z3L+og`BT3^hq=QdhQtPv7ED^T1vcv>7#~CD>Fq0bqMd0Aw#wDiH#=623WTjB#@_s6?Cw9t!riWlOJq?VJLs1x6i!@(H~HZFD*3sBTqO=xWNDiU zDKb-=+Yw+>K*vW0saru)<_FizABxP!ip&RM6LHE~HJwB_W#U;+Rv&VTB@B!Nsl#-5 zBu{%7?;}h3u3?ZmgUFLL9fQ=z;+nQWsws?Hv(n~+jkJRJ7K$t{JJkS-XcCQ)RGJ2< z@c2ilLDN(s3r$t^hM^mC7N&xVEU_fPidJIL4AM4$t>hz6VDWuJI4{gf2v>-6rz*1g zkW%e&u zD+&tQ(MN?;Q`VfzVsheh(Sf3HSpygcTha3VqK9$xzv?00od^XdCm*R>h@})ELx%Aq zY56iA)2t}6b_k-GubvP=Qlz`&y-spQM{|$mEt^P2;8}9wH{nF7Tu&M)fp99VFmO80 zD4kC1KU+hp&WR|^+Wc88uh8S5YE^+~sv^Pg398-1>XGdpAuyq?FbCe4 zAumN%7pjM>idw|~phyhD)rI8e_M|cVvGv3J2&DB3S0H+mb=S=As%A-A4AeVUaeE-= z&^%u;B@|ZKHUa|Jy2-1ts;dYL{*(5Lf_||~{{z)2<8*Zl(o2aavgEq78iTYaDKyUi zEWvsb{jA#GgxvgAO=4O9C%I9T{sjMAY=i32o5C5l2_FArTRBS|ih%h8qsUg>|`j9r3c?SB}3R+pp57MhG zBuQK|-^((uiao5DL7M{CEZ^pK&|=t5L7w(8zP+lwseG97qmn^7++8TI%e7)%s`?L5 zXJ5Xn(*1*JZ@+e<*fxKw9fPzzB*RP%39oIF^ZDP}##HTf({Jr{QQ^7i(d zzA`UTdw+_z{+D<|w)&xdh{Ks~fLxaodYE9htmeYO&@FSNa4;mbBUTDr?I6Ds;_(!^ z##gT7FcU)@50^g_QuuFgO|BZrjp#KPd;3s>+J_0T5Ot6`BNqt8b)kygGT)TfGWAh~ z)q-8Hs^-bN_o$aY-X$3Fl_Uyc3x#+3KY|O+5Em#if2f)%UwZe<(<*DOS$&~Nsv4lE zkyxHccGvP2z#IqUuW6B7v&feLRFz7O^F}nrE%W~GN)=hNo5z%EB_?Ti2{DOkV;Haf zaw)4kB{wGcgO6nWz(ZHjL++VJh?72mllp_{%p`ItC_AaSA!#gKv9auL4kHWq%yYCt z`jC+f5#hW=eIQ1;WquXK-7?>*lXlB|svh50hlv9~EzJOAlD4&9B2tB{s)j}hHAz!} za?#H)$dq_dP}QB0FF!!{Bwh(8HvnuNvV%uU96YWU_7~zPF$ZY{2{Ec_6Onsni6XO% zz~8Q;F_rhpSjaphskA63KycB9s*KlJY}~$xPXBv8#mfm^g?l^W$9fFTDck6IC@!iv}G7 z*_Y8(<>Q?`YXQqZHALO$;4DE87w7|*M~@-D}=4WZJqMx8WM9?EgYNVdu4IK zxifAvyUaRqdhvoT3tXkaQf-GO!45TTjkOCMb>+rgQ&-AI*KXfTs;ijY&v5 zi&IX2x|3El`%^dh^d-|np4|TH(}(xZ7cATL{!^p&+kGaSbFW>gxpD2p9xZ;Bx(paN zEhhKgL#rb%jdS1G%WV&j)=n!KlxlQXCv|sanL<(ORI~Rai!2M*hs}nXo-`Y3a?*IH z`AO5K4yMc{>NauB*p=s(D4R29#Y z{(Q4VAD=9quDpNs#f2_4lAm9=)j523r4hY%Egy8`K*QkT<+%;VollsY-G9feL$bu^ z5wFfxx;1B6;1`X0pPIis5S-R~DqAT@oo!hOloFhJT122g9RV<_dYdj z?a>K$`gE)kJ+|2URPOYUu5nv>HaWNd{5!iQU&idRsa(}8xof+`y384Txz^zL!FKD-d{Mnzm;s%?m#}U)t<@t4#x2zn%ObcD7ujx#^t!%}Pf)#!Ppc)Ng~o z)!Rz5s(7y%VldnL`?|R!v-TV7whI|;+R1vZooqqFWBom>o7EfYaZzu7zp;(0wtDR_ zvE8DM7j?QF8Ggi6Ib;8z(1fp*9v_IlzI2n_Iln3gjYsE)eLUK&j&sK(#~Ob%JQ%2Z zxw^82!=$`wtrRt<7N^X)-qX6~vh2J@%NFNpwt8Z};d9FxbG% zxOS?KwoeU@T5j(&a*e}%UR}+9yuNB+rAgB>LN8uD5#OS|-ldQkKNbeN)l8dKI`P!x z%ClFkTDAK!41CnUmG{jx%S=CdE0kQnSbwbr%kug?`Jt!@646>f_#_$?n=@HKxAijl_Vu@J}E9`NwSPx9d&8q~&7| z?9Hy+`9OYl<+{f4hL+CH%o`e57JYzpEe-Cs-1?^AO3L^rSE9PcL_I7@x^qmc)s_wf zF4R5k+U8u9aQm|#o@^Uie99`%#e2M^W1aqrP78x7g`K>l`_O+*knYM>rK@$jPYSD{ z7;*BQZp*_}byvzuAM2)6$r-eN-rHt{F_VEq|jtFczs-%%RU>gQxH&)1zbBmBRN zN`0dp-Ryiji$f0cd&X!cKXUqFklL%tU5C6X-s4lOmzG5R?09Tgi$gAXH@xr4$-jRl z+HYHx>p<4J+rvL$%~?8aWiI?nnquF?^-H7Vt~1~2XpU}PI>E5mq{RehM~KRXsAKnM z%szC#gwV%^ca80)*}rt%Gx4?3s7p%+htBTJYmD=F*sQbTY?HvEus4#~tq$*-ZWye+ zpdcmB@S*V$-6Y)+laf+nHG4QrvX?1pYS$Q8#X&bpJ1}+RBwdHfGL6ETlPpMEs}@*Y zR>NUe^M}U|>{WWNJ^4Ln{E4W{$ri^lA8ebGG<#yWpO6;rx)?($8)sZCy={hc)be>&=bi zbEco47d~$A^2oqz&qB>Rytf5;{&bIZmi zkJ{00G#7qs-`1w&M7L z4$Tf6yWBW=$>M;UQ%9RG*eP8+qJnScCpfZL)``9)2pKMx(#$Z=bf zeDmoeZGN=%Zga`!dg1td6Gi0(|LBw%Uh)iY6}r5jOt+TYqetMF$vZ>M^ycNX?Uw%D zww8|DCY#C8ZEQAWnTO4fRxe83Hl6oB`@SVvsXcsg z;PS+0t<3axpSA6MBq67)`qpiRHFN@on@uSQHk-0H$?Z_S^~}sk)^jqKZ#KFV-C}0& zS5H0D1$XoN&n){?+s*bsFLUqw9#TR~JdeEZE}!Do)a$R}_r_;KyIgXaf9>#ln~Tlz zFU5z3e4VYo!ENcXhYj-2&A8fjoK3ZnONQsSv59V5D{XW)w=j#6u&*XPKiZB<+n!z) zxivCB+9E%=bFbrm`G>rHPsHa8>v-gGrR(+$bk1HfbFI`bXN!OMwp9Nm5mk>S+g|9~ zZ0YnK7U$13ePiK&PR~YfZpS8<_7;xlU$A>$YWmvEVQs7V7++o%r`YY=&D!bSy22t8 z!}b^a>sW1j-skTca}TuaTGQZo{7v0?V+(3s&@kzG(x>d=x!Qm0-PpfUx8K@D=}*kl zW9n!&u3LCz?mJoH`@C$&_a?I4mExW#3_YZl^bPTkS$l`5%})k3j=ZpI;mVkac~yScmJNzJ*68A{&Ucqb?~r%!9Qri%@ACMf)32u6F28zfqU>Da?R>({ zJD7ad-)KqFwQc=)Xu?jtI0y6gI~uQE>M`_bu0{VUi!b}?ZGN&c|Llr8$pPJ?^tI=i zoZsR;c;f4}lhU?(8pfaRAUp56Py6twh*{Fz-eec|81;0>&1SlD`{#^#>fOj-@k`$+ z+n&62U1gnRwe)hSZv2;(8|_^R=6OC%P$ZFchx@0@6WZ%kbBnYH2-usfSt-AIZENd0 z8MA|*Iu8xmTV+jlS!nEGw?$9Lesi=}dZ*Irnz7QB?b4GK-P7VDD|TvG7LBelkZLeI{MdHn&e|W@5eg471pf)wn zcM`qU2IhAT4q6uH|Eu?(iX*};{hsxe8v6e5!0sC6>-t*_3$CzObx~=(hi=aJ__x38 z&e`8jPE1r@qqJ?VjX~J45$lU48U*y8abVAY^hkrX<7X>sj#s*zG_lXM)C0OPMv6Wv zqEgoy?hIB7wy6ARC$UqxLpIXlNy(`nBYMZJRC?2Czb@m4gM#8k|GG7z#fxrN#1t03 zS61#n#AM0OBFQ&PV=W%_ugNdbcUZLdZ10##?gOQ5B1L_7T)KC;=i5KQ-Cw`*9O7M? zY&7%w;@pKPGlr|aHcNi;Cv$TzbIV=5s#XiH?y=nJw)U&nt4m({G(T&79-BQ<$32a(Z`~NYI4mN8~=Q8ts+!+}~bR zqv3wPth6O2uJh)#Syq2vlz*Zu{Y2Rs-N?|^@nwq3cP#vQMJiEDO6C6h>Ag$D10GMf z=v@7)P$;F%`M{n#68`opWzv5ud{yf9VqenepdS67TnpT>!oj9NN3(FER7c1-*{A)| zTyr&kOjErtI-$2n=;JptBcgZTY&N#^u6nWJv(~Hn3%)B;zvi@>N$j=k`@1mZ(xG*B zcBiZJ9^`Me+0gw}zY$*=Ch1QqACdTJ@?PaRb^ER?<|dz!j!m&DF47HU*Otw$4BjC> zaH~PxtGnM+{H8wicE7Q8wM#$?C5z_RfzK~595PHdUA zzR~r8)ZqCB!n0h5Z))}ln00r!+Et_4y!-O)yT*6+SXU9Pr|fj1(Mht#zEwhLS^Ax0 zZKQ?(%<9nax8*%pP z+DD$|YO1O4F&H`}s6_lxi}uo{Z90#3#wc9w&>mGWNj23&t<_cI!H-8)#Vevo``aMST3qRY~7aQ!JCD(3pA}Gg1S9Erk1bmwfWiB zp})MkJ-%P~BgNTk#=BLbZ_fX!TCmG=&!2ej!Z4{SpPS()b!&=71?R1F^t>_u*TKk# zh0=L7j^~Y>XUz6fb6z@Uu4CRq$B$RvC!4!3Y8`hpc4LeUFF^Td^4_)MC+^C~1zSh1hBxE>6K7tVo7hp%5i9GPGJ8Tz>RU6% zQp1e4!pBP`PC1Wi(Jfu$-@M>_!QE2t#mB-N8|Oa>5zibqf7hfVw`a93y&gPt*-^a< zdFIn<4w)Ld6dlbfc{uaTkAP8RR$T*@aos@+(d*Qabu_o6w{;b9$-BXXjQK^qP@#FKAO@&fsprmDYpAvM-Bl zjS3y_=-SgZ$M^gAgHD+T?>?$>iMo4see>&@{moCkLOb_;N-Xj&)1BVwJA29aULWj~ z>viA#Z1K8anSHWjb^1W*SAO@>dOVoBW66Zj#ZK=JYfii^Hbc5nab?Lml@WUnZE!myJ`ZoMBe{fP} zqD6f4`_8=blJ?aq91imS&{^Tv-plU$+Ii}kZ4vJ$t-k(JPdrin&I;4UfVV{p!cNVz z?0;*;@jIFFX`lKQN7b13tDnB_!NSIavsUlAT{qA3l%~XjJjsKnS2%obI&F{}US@IW zhwp?V{^38$DG!sdQFeSa;7NynY?4OgF_rMcp+Scp-n%8@rjd0v_=2ok zfQ+-a^N;a2+dKzPdy}*`Tzx}s-?Qq&7weZUJ63J~YWMk$B9r+BW;1-|Y1OW|^;FmC z&($>m4d9-8bHsPk$*+(53^Di79_opiA z{)hO2m8(KdzO}ekvw6X_k3A9&cXv9zOlna7*cW0WJcG8nH}zO>{nOpjVN>4AZk>^A zur7>iGRiWZT1J7ig`o?cu^LQd$a4*<>(Zpr)D;|#D zA(_79i^_-okrjt$FI;%JrTN6kKRx>-Y#QWmUle+|vcmDQ^F71jVWF3MB^hd(y;045 z(-Ee0YxU0Tw%n^5Z&eJ^H{27TF#cPmynfcool=;A&<-YxCLH}%yXiM+z2ha|zi{@Onhi>*S2#6_1Cney34buBB#ZI(dQ5 zj@+=+47VM{Pv?cKcL_-^c&%6T+I+O#Vc!N#pX*vhuOoe~e~Gxh<<;Qfb!Y6VmZXmL z-tqLc#%O2zleeZvWrsAMi&m>vXc&Ghzwxw(!GNVL=Ua^9WP9XlMfw)&jI(Qzmo8Hr z_gl^G;b$KWom)<}!BSN(&x~67WYbyANAn-w)qa$bxI63egPJYF4mVvtG(h=G!`!cD zi{m;)whq5(2lp}Q{mf8N0igTbvrK1HhTI$9`@_i#@t@tUR83e|FY2Qlf4Q<$>DuKH z(r>+7vO|Q-OT`_;W-h5OuLxdy+cTmje|Bup(dj|crcKj+lNEj`JlaXB<>%Up6;*}f zJohe8Q{Ah3e5YxWaG}1jL1LYZ`XI$38Qx>RCvbkw>~xq zY5utD(bok_gYU@}`?cpCQ=W2f?TXEBqqaIN)VQX)y(pab)-JDNyz>k*@7Cfi)=Rfr z5BfT0(e*v^gahQPr4yV+i_Lo7Q~T(=n@jcYEHvttz2nrYT6OIkvVApcMyTDnHM@C= z^V};Rzh4`;G*NYFQfumVMU{-fwa<$iS1q|&^gPn-xYd`{Sx+9PMYdboTi3mN^t^FX z&owKD9h+9JS0Ha0`|!EDlS*r_v1Rp9m9Y*3`oFUrC~6~i<9A;C2-639b*g{o6dqk` zs`a41;#1Rb#iuo==BI`>Tb%J0>GeQfEjd9ft^AYMqkWg>hF+*^(%ve4S}ZhwqqXSR z)R{N#AJ98mvQ0xG&#?P4b^nbMEao{@|In~k{QR^~&BDy((A3BmyOyhHz9?=iSlU*m zby%ZArDjEOPTpN<`y2m$k^x_bA2T%a5qSf+3x z-|v@2cF(6xjZIIMuDdYsZdS;v=G`*S@6`>|-kSV7-aIa2Y3n}id2hWWhU;|lmYOZ& z|Js7qiv<$6o~)cT#ct4anM~8r3Q2X3SVxVQBWz1H#l|G0n<(v1AFp2(b8x5a>ItP0 zGo~B&iRt^h?3Bmn16ngh=Umw&v$WM_?)3A)`$R3W66P+iD>jbUTC5wb^66q|)rEH# zO|^q86ZVh#UcdZ7q`5-_Pu%pXQmkjcpFK1BXIv9GxbWK46DLCz)-TCW4op%=ReoYy z(7e;=%YdoTmAlKfs@qi98b=6kzS1x0jMy0GM0<~bNe{fuldr{UY#cnjWm$QwhVpjD zz4C+dYKpIPU+=o@v#`QiZB@HFwio4V<{t7tp|#>zzbP}FVw1KW_l-}P@zU3M?~DYe z#JoAK$K3ubZ+!fI?1{Q>pC9-i?P6{my$|lc~zGPT6BQT~q8_*gO9%_Vq(;o(N|iwZ6L|W!bRTlMURY77ma*w{`e- zm2GLVlN)%4dR%>*HPECYMQ_5S+Dz59O$+j?T+32+7L7W$a?!^dx!=6FKse<*;{}~4 zopR#(Rd2bAovVMZ)X8*fS8e%xd1PJK%<)mS2RE4YSSH~$-D_ZiyjX(Mq)?F%?+o>% zA^8bn_hfInC;N%(ZPU_!r@L+Ku5UY&N6stF*I9M-!|rIQeBn2PH%==UbNKX>k38ek zgJjQDcZ~DOPS$AYqyI)Xu+%=%zNT;7+2pp2g!E{yPM4U<6EjXekN%U;GVb^G=Pt9X z!?$HFUUKj4mC$W-jKjJIyniyLS3uM1usXL%3!|<(e^T_XQ`~WK*G*YRH>qBm#iebx z3Qwq!J-_w3$61ArHLkXMYUrF6jTcQ0zTz+$IxAhVOqUwue za$23U2OksBi4l=pY_?>9&Cze?*Ou0Zy)qHC3<)tc+-GLr-CZtFKdgGM+ZNm3%O314 z?-Oy!rdzhzI>{Ve-M|y0b-Q=d?0z+;^vs-jizgMOsd;^iofy90zP+1ur1Tlnn2`LZ zcCCX=gtYP-1JatGoc=XM<=*{idVeIfA9z-T?^o;O)rhHl_}}*PLa@uD?Q2;^(dCko zl5(D~LXT@oNn){;o8xYH{j7+)(Rnq-b(NNU=g73keBZADzQ14jHXge3+Iip!5uU7s z_~rk#+wd>D%WrUYZUH!;<9 zW5v$Rv5)5&vr@yghxJQI#R?OPL+c(=~+NR`KmD-xg zIs_SvuT_uVe|nqcLD?rg&1Q|Lu{Cl`J~+AR&U4+AT01X?X$le>o=)!nCN1m8f-$Qf z`j_mSe0_M)-A?`0Z!bmaf9w3Hy>j;k#pRbGcZ>5n51R+~Gnpbc@w#cfuFj&Vn-p9` z49}HD+_$p1D`e_FQd)f4j@p*6(&Z8h?v_tn-v5rMkno{ComO^JJ`CF_Rek43oXW)f z?2+oxQ$9Y-f9Y{4J9323{g-OS)2_Kq82G_|WJ>1ihke~kHzf)e?(G=l`|N|l#_PRW zYb{pW>~0LHcX5gNZg(ia?~~@Ps*B=>$FF*p(z5FDgNw3P&Kx;?tyTF`Q$~V!hu!Hz zbM@c7{yDh${XD<4A>jdrR{L+?$v9N_c`vU#NX6Sr`j$<&NJXnpxVgiK#KR{no)m|d z6={A-H8HB*m{#g(vRtwEDHY3$PsdGKarMaN**Xr#avo&&oAV&!pqRp)wt34}PU&@s!m>EfMsYmS7?YW;I?|GfLV_g_$$CwpZ{ zTIlrTIAc%u+Ku9acOCs)8)WmRk7jbclH^3UUqVX;u4^j0aOY8}y-a27uJO56?KuO^ zrn;Q}mH%uUZ}$l4H$yj#zrQg5tfkptF|)zpmK8bYgoTR^&l=lr$Co*KMr4(L>s~se zd{sv0?Jpj!pFP|*+^o2B`BZj`hqJ*er}skFQ`FX-nGlive9N3McWX|@1RoWDY_2d* zaqQD?bMoE{*E;>_)!26*Hh9}>toWoO`LO#+XSqR$_tQr!V*q*xo(?B!1xbWEb@h9KvY*t9uPU$-#%GhI&?Xd0MRz_Ca z{Yp+Zep|n`^2fgFDfb5_9nzDQeBa+k>WfF%t`|QPtJBjg6mRt?lZzVgd+3~%_I*|# zGT$yMcdoYT;6R_XYQJ21-a9j2R_(fB;Yg7|W_Nxlw13=pciPgEo4S7;a!d0b{XdMf zy$u!K7P3OMwN#WGTk&xr|r4- zOkd<$$iE=g?AXzZ|F2Q=H~So%f%7s3q^H-mx#qf2O=TzbNrQ^_eN@ zv-dQ;75lNPX6S+Fl*my|8+Q*1x33HH40vX^>twC`*s}%a%Z4BAXgesEJSJ_bSJn)d zRO{1o%)KY&uGUKZ@x}4@o*|wWoNStIO*VWq%Rekdag(}lcEZDkcZ=@@`n`U3_QMOiM;I^`1 zV;|_P%D6pZ%c=~UZ9kQ#K5SnSJ!E%_-sHoH;=7)FYWx!up7ngKU%ckQW7loG)2e1~ zv))?1?X-Ss(cYx+S*OAe1g7jh(UIyhs^o>W^Rk9z>OEy|jkBG0B>&og!Heb?8?Drp z9x-`w(D|I9BfmT=xtVAGWA-lfn(v7@H=U2(IQ2NOB`iW`poXRDaxvL22Idk4>AAx< zmd%WOwxvlzZShirqK7SuHw+w6J*MN#+;L65<6K`<_NZDp*WPom`O0LYqb(n7a)XSw z{aR*bcd^gGmv2H&hDo1|dh&Sb+3&@9_MhfXSZwF)W}0{4!G(^2w|sk>_+H4kV!pp{ zsBQQBQ<@!~C-k2*e%yX9K>PNxXjR3Y zy0(_?lPdb9&J4E+ENVXKTkjh)ZT6+Kw=N2Y^S^KRw*K|;K*qX~o)e9}Pxt@z(@f~x z7;E2)^5x}Uc9ywF7={Y1EE_UtYG;xD;ED);^UIF|5=P3VpH>p`HyGgZscFm{FPC_~ z#=K^^>H+Ih$JKX!FdKDT{-fJng{v<{&GkNCA(!OQHmS8yL9TdJ{Dx?|9ZL*W&R(n2 zQ#9ggbbW@bX8q0$E%k4NJ%2qPdiQtPpz_*mwcg*Gi`1vI%bPU1r`}v9AO1eaOd%{f z!r!iEgs)ALyYH2I)3?8iz15gA!PZSJTTK7#jbpYNr9#W}tR$Wn)y|BmoZn~A2O-g? zDcj~HSNF*=6H-@({x|?U&(CTJ|k}Vq( z4BhVLt$TLHrPV{?rm~01$ttLSi^8NL`2gD$U;Lz(#qHKI z>EV6;(!y@@3VXki*gxp?^9{cm0@}pmcXf*$eV`%k!_(P5l6qGHYF`Oe|6c1B7;!4F z>a0*ikJDRJ9BpP#s`|WLZDOgo#35@#-Ye;uGfx-!9}XXSdC-q@BK3x1%{BQ=B4tTu zETxq$3O{;3bz{@ewrvBRs{FYA(r5qq1jn@Nm)z!Us;&3Sbt>4~o*Hn`zT-=>_N70~ z(=PpKxb3=F$0FkAg&d)CGIh0idY6VcE;sUOd*j{t#`{Z4<)*-Ib^2pN3oM-sjF0C^ zZua>iB7Uc*Qtj8|bEhq4m&WE?9G#-su`h0T{`r&Z%{5mo&zLeZVg9bDKc$jszDbTc zl4@JDEeGgm*9C5s)UzGir~GND@QwxRwnSaZP)!WIA^mgsw4#8*c~w^|WZiC?uFafq zaix2F<@`#MTEmv9l~pl?7Y2rN1ve{VmqZn4%|yj%H1rf zqcSSY$m&zunTMCg>xZ3e&eaefBzn&$?ozL|+p!gk^j&A%Y%tl?*z!SncFD8q^^9WJO&d!ltN{Gf+|p60Bh<-RGS z#b^7+-<~^iR`_zI8G{{e?o6;v89RCH?S`q32MfDQUnBE*bd$I60$1M_(^8`!hCQ!! zf8G>6@U?S~$b?UubzA56&YGZ8Kca8v$BQjG?{aq!d0X&$ceP*XoGfvNTU+O^(ET*$ zLb$OrcZYd)=v^7h-ucdJvbEDqo<`l;bZCcIwwbJNuuF|z!q@isE)OqkD|(QU`?87q zj}s$jPLe5-sgZi_KPuU%Nz7;G_zSPE?T%a-yT|^(o9&^e^dE<{iqwZp{&qZJrFfr# z8O>I8GsZqxH8e_n-~K(v&925(FFe_ z4n6eHuCZ=@FtYe+-$h&Z@5x#Dx_*xC=6;Ws#3i^YN$gMPXFB54uARE;hMW0Jo+#b$ zu;-Z$?i;7oAvYi4hAWjV<`=eO4MdG7tO!%nto!LuIM4h&Wm zJrgafth_KuOut#Rh9~L1TI@!X*Y?~A^{s_#-VJCyny_+k^W;anQK=)i zuD=|fM0YRj>lHBCciYKkgBbyJ`$}4mB=$_0^G0Ipyj36k_w3gyT3A#2;KHI;E`1L4 zIca=itU?_R~PDRuj%xDAPu4VoKQwsLuJ;>ID0oqJV!3SIS5>!Wvf$>d|t zQyp8?KATOEEL?t0bzm}3DWim*{@ACITKK1XpPPqY$4(iVC41*n z^0KO^--|E)j5JFcXP{}N)Z!%+lvUTP5xp^KMa$JuT1htBYirGW5dUmi<+M7-_t8}s{L81Gnm_UOy1Gf3LiG#lEVeD#rZQj3BwDL8gQuvD}A)Z~qO`aG7FR&DtC;;UQvA(h1W z+WE$sKkjcdS1>D4OYFYm{PS^QhvJM)zjmDSkcj%_RK4`DbhP5;tW!&$dI-O5o^|Z0 zoZG@>Px@7ex4yh5GF59;dX02Qd#d)dhH1klhQy`~U$|mqNW}VwpES%r44FLZ-Na#S zmXE%Ty%lh6mziA9==a|@$dgm;8$38R-qfGF&sVe$Kid_r1Au@4Rjsa-KKs>nYt9(i(NJ zMt|yyW#=VI>s9iX)la@t_`6E>WX(6ZhkMdpCa>Jwaqi|)@5g1kz4dBaWt+yzMcVx= zUvRwNG1aE=>LGf@-Pd0U>Hn-EUgq(o>lLoAUmgGY#duu&_RJLTn)akkrq@Q@vyT&Q zKYp#m+Ons7)tY0kJcV^X9qM&9`1y;uirrt&oH5EiZA0*M5SK zB9hnlwLhtC;&>-x*DtS4T$PT37YGt!}~DnIex?I<0Xrf3Ritn(tf9%~wbocAsnh zC)YG;gK1b;(cHY!>f;lP7lgMS>6;g{+rins^S9#2{iYiRmvye}VIb3gPS2$I#_zW+ z_m&^LrB~I9A6GMX+^x>|(Z6o~ow9j$Ei%C+hZh}ee`WGaV~FhLZBnkHp`vYay18+& zI;(fC6WVygQe*g+k*(SLq@ykkJyOoAc;?WyKKk5@ z82J*RB=rf^hU@Ytj#gXw;B3HTwQE_!ZKlkOI6bYdW0TNB#UWzXJ_oL=H2S<{YMq{i<0*7WF(CddYNYqeb#B zkCP;xzySrMbDl-t6uJA{l?&u+=uNme>aJ* zyn6n_>l?q+k6ygr>&^O#6Q?pNF5S8~JZoRixw+;yduWTdo{INUY#T12zWcSo{R!Rs zHU4^Az1OV4uy4&iAtkvW>;Chz&+QvSQ| z-^Jg(y!fzvZ=+J)=fV%)q;{D)=nv@-dUO1~!vuG?_~FkIm#K_6rI|IbLZZZ@xAdjw zKdmoZRqEZg_(!{h)cCScZ~Ko4Ev1jw6brR)`H*3E?y_M>p~IF9J8PS7R4HyBRbk#! zxn{zF`j(@`TJOppRSt?6cs4RuGJIiuT0!19iQ}G|`X{Zo`0ckKWHPfd`<`eQu2_$L7VCm3n-T8lKyY%npH2k+oaMW{k&!GiUn@X;!H|ors{e7A0=NM7-xakYFol!cv z%lD<#CQs+|l=gKGp1CB)tk1qTNGz;-j%oA#@A*NkPjc?0SWcedrIsBmU8YbFt*Y|M zQ9SwF=ztW%w@)p!gyQI5f?CwRn>ut_HxKehzvi+URuAjP`p#R_1vdGteO5BV5S2T95EE;dRRu(*RkiT{nC;iu2S@s+w z&(R}k0Wr#7D~nqYYoiFaFxGle3u)DLVm8v_Bf3kgT1hKe(3P^Rq(lrv5VMpLJex3p zZ;i)uC-3%T?Je%Gbs{#>I>O_n2UuL@mfEVKYi*JL(M?hZaomEewI$Ld#C>SR?VchZ zaH*Ahloau6lkmU%`m7}Uh4X@QmG$hVkn~R_Tj@2*f;Fmt^Q}N0oSlutpY`PY)`{9k zzfcm{Db$^BfXkm-1Z)y_8ocq`%K(CzN#1E(Hi=9iYlv}iSubWu))2Fi=A5|SU0R1j z>{?=$QpCnGu+heCGdB3^knwHs*CFHEc=T@@7JM7UU}Nw`8s$s>eU|g$qeEDi<66v2k#$JovFmm^+vMJ$8++2RqXj9(E)&WA|GqXG|37~Ik4^4>eQ}}v%lT~x6e9emX#S6! zza#QLt3Lhx{Qt6+n;LhR5U)3HJRCmYuNm&ylklKihLKGXPXAv?55mFvtIoDOoacG&cFL~OXQ!PhZ4cN2)CB>e@2+e9&7X?hbqw;@-f$5EqOAqP%d#LqNX)@n!sdI$Hhxa20UDdH>}x zz~lZR{#F8T^58x;Z3b?Ocq;G{h-Uy-fEGO{+8fb$fCST#06tE*Dg@G2=n(vc|wQ_%3GAzgzQLz z{tn_Yz@H+{AD6iP*%!q}3H0RI9&8ewcp*qdm?@ixH#fD!09u_X*>n}izrV6 z>Cb>(4{^bqPn2hZcp2#FnrYPjHQ=+5z6v;9`;hAI0}n&`THry5zW^SC_*>vUh<^aS z0`afFcOd?g{Cxk;|M}OX zT)wG%{&)V*zs}_N89tZiwgicP&B$$5cabkyW1}x~T_6~bdAxQ$|9f2!OAWFQSkm(3 z{qsOr*QI`8zOH^jt|4CJ`L;Y)SMTt}p02K1Iy^0VOa32SvZY7<=#xJNTS8xdb7+wBBy8NSb`G@K9kJIHJsLMZ6mw%`(|5!c#v3i31_{Zw;kJaNJ ztH(c9kAJKl|5!c#vHEy7x|#F#g%N3XBPSs7*_F+LSSK_3nzpJOOzbkow zC@(aOJM9pjy9a4^Z2N=R-$*w+^dG&?mcvxGxUn3h}=(_H_63^>PglUdWZ_ ze|-*5`R0Nb;u#2wz5lOu`h3`~ZMX*qbL(k@g+%_n(?ISM`EoDrVdnfa1Nu#F_~zf0 zp?n^2qW9xolrIKO^n(~q*PtW%MGU9c^@RH~Jc-*TiP%x#Uex{$0=b0q-6z0&J$bGp z)gJ>+^yD4`*s~}%8dQkmGM$h`+!En}pG5)+&V27-WN$m)(FaPa79T%~m zg6qY9>!*T!qG$bCz;M8x;8bj!}@=au``VE{}^M3&9|80 zte4$U=^IHRJcxTyPR^Yq!h^XNk49DgVEpW zLjMLh(Q7jLrY`jKJ|xj=F?va+Ut;5_44mk-8U4sE^yZ9ShtXSip?6{Qq^xOtJiE{@ zXY_iEKDrD2E=EtjBcS&8bfG`X=nWYCg)a0D7`-8*f6|5iJEJ#Z^zB{f$8s8yNIfBQ z73#k|fm|Y-eCI&)|gN`?nB zd=-$h5j7F*?j2(A91I9)@U@HGsd$mGkG?-Yi!^>7};*?L0mQ<8-J zB=;97XUlgpV`m&Ep#G;bdN!Vi86M5(={$t={c9P1im`9b@FIqfXZSUSTQK}K!zVDj zhT&2ir~Se!;3N+0IQxO&>^S?C;p-Ut!x1+4y@g zoQ*%*@3H5#g3+_*#rAt_e_jB35`P=U&&v$AW%xaYPh$8hhEHbrCx+WGoEH_LZ1?i9QZEorhDsJ#aGDC5dt!;2Hw_52xcRnVXSB^%9^bV=GCdUa|dv zJ)a8nWSiDcN8q>~x-t4GjNY#cJ*mSa5r0U#pmE$uAeRWA%Drg52MFX6&Nokh|J6>< zGn}o@ml@91=WoDCzT}#M=G(#O*>)!d@gsWDXHq@=UK{(#w)g3bUJLACy(MFx#GKln z+J)Ye(a&J?i@VUTWArl_eNq>CIvMl*U@Zkr?$2Xf6t{iOAc=o1(o#%+^Ccp~?r z`Ze4(NrbcYa4Vk*?2|Ew>eGM|JzFk^7|!-@CmGJx2ezJUhv%WOPT8ND;3&j60?A7%9O8U2|q^jEsj-|0gCgyC%YzGFCBzMmP+w)cTTf)9xQ zYK;KF@I<3`H`$AO$XNu*rZI0x|QzsrS;(IgQ)8|QU=`fvSq;6%^H zIjswQSLMRC3%1-ZGIrQFv-O{ia}A?s<1E}ooZ0br0e38oC);mqVR%>NV#(wi#OP_c zkodSVoRl3&xV(G?n9tesW&11kd}(3EIx|JQ!FSdcA0UpAgi zz=@xp+>4$U?QaPu_c|zN$7L^uv*R)w|B;-AB;r5Y-nIBt5Fc-Do7x`>9G@@SZ?pA> zjKw4oJzIZh|3El0H%*3H;tx@za9^z#j4GkO_?&@rT%7 z3U+8cr~HbNzyKZZDEEL9qV1P*SF*vSHT=JVYXxH80P>ARd_xZb*oOFi1p(NJc!IP5 z97KFG*g1hX`TY_}#ez3}oQXf<$R93U6TJPy?+y}x`-o46g8hKFAK3qi_<0cl5ET-< z)A;Cs|Gg1^p(xPN>j|n42K#DAuLTvy2=NPGht6-QojGv64oE)|Dk;5hO!a~_=DF{Z zkY1~sz|Tm;jUnF+h$ln7DTwFw64*I_c*k%7KZ$sJe*vfW@u;6?l?0sL$D{m?q=3`= z#*|wNFs}jG-vS-g55zA(oay~NYG)Ss*$>Kx@@REI2Fi##gI#UJQ=p!hA?^invqRiK zNnoGe$D{e)?k(W-_m-5Kf<7GCnE-qp;*)@HL0lQ?$sWXMzn6!2e<^`KXArM~exnTW zVCYck{W|LZIw<$2NM8be4utqo{VdRrMBE$7R|D}{6@i~)5LblqwL|g{sGcR@cHjX3>}CUJK>cV(JP!tT zaZ$l>)PL%aI^y+E4~-C~dP~G*K<|LK7{tc~aq4F<;`H|xD-fsolHXC1ME$4v9zdMt zdjfHq?`_0wU_5$&_!*c4Jwe>3pTLiL#7_Z#hxj7s=bI7V3j7=5#{}kiZHTXhexnm{ zS@2U#OmHBL&lPB=QiuzWLjAvtxFgi_JBTlbdi4zP`_O(G5pRHT>IdR0AwHrI zZyFyr@V^h@&*6L(5l@1Cc@*NkA#Ns!%fk6uBfbdqu82#+{3-zP*b#zqj6{44j8p3o z_kr`@hPXZG_adGE*D1#ke-CjukN6iQfjcDNft$Rsh`T~OCn5eB;yD{}Uzj&8MEp4T6N316 z;A;@y2kkTm@ovz6o4$*Fn1zCKZQEwEjeZonDB4hyFqV@jUQf4e=s1fn6iS z-++Bf#NWZV?0~okw09T8g`pf5A#MxTJ4+EC0{+AxE(QBr5&r@8a3A6mpdUDjIL)^Z zaVKbBR}nt}Y2cns8&S%}m5u?ONOp}qwn?hNg81>&9HXFTFMssevf5my9% z4j?WIaXx|gL@2Kdh+9K@zkzrZjK4LA?|}1qjd&51;}^u;pxt&L-djmlZ-n}A3UPlZmr}%EL4Bw~oQ@w)5D$j>(13Uw(Cy>pgz)a z{0_Q)h(odPlrV>VO~ju(&P5WWQx4?c+#M@wA*AMZ#P_G6eo(}DI2;w85-xz`TIw;3+i0^@RYK3?`TyIWB zd>_P%Yj|Luq`fca7y;u=scdl5H)>%qf_ z9|1eLh-<^RpO1Jx)X#H>f0Gc@pG$}*KtEoCxEIvtXNVg>d|n}*1H1w8eb631Asz|# zTM(ZLc77o~2l^i&s9&^v?|@z$@gYir^X-ZFc*wUO;t|k43`9H{%6ACj;V?g;_vffT zeWBitLV8;$M_t4}LwhtvTn5HpGsN}P1?Odj_-&}4^AT@>dhUU^JY2{5B3=OPdNJY~ zVZITD_%CRWQHVD{JfjhphjDEe;$NYE+k<#C)XzhRm%@FOqlou_@;!;TFI+dBL%bF2 z-$9)I|B#1>)9cKqh*yF?uMl^IdiV?RYKVgn^ar&5m_Yj#N1O-!Lr=tGAZ{{^pM^K#t#$3n;_qDh|hrj#|rV~P!A^~J{S672gHlP|2c@~KzRir{sH_9 zN1V=Imm@w1>Sr|KtqSd91_IIru74~2UB5OERc zH(nu5zhD21_$R1W9f;q6_TB^fOIjaxL%GN!9s%F?sUkiN=ATB0_k(`S3h`p7_l}5f zf;hM#?giyS?8h5o+`@sBWHrQ;9v zGZ5ICBH5dVva_lNl2 zMEokW(^|xTfj{pNUk(1WBAy2IO%&PzEw3dIpT3Cqg7&3^IQ_kX7UJ_P-%-94{82zW8sbmyA5;A>sBb1n?+NqT ziHOI-_%ai55$Fdz5bp-%7>sx!l*=l_TcF?Di1>SmPa5Ke;Aa-%mtdcM-%ii>D8!*0 z>FIcO7x4k)KrX#N+yVOGkBC2n^7@Uq8}ROM9@HNb7&iwXeg^ub5r|7dz6OYoh4@cE z{E-sheVzm2zab6_5T60_u>iz3fd5g5Z-x30hxiQWuju#u^nBaE|AR>X2I@mT;vEpj z62!fLS0O$c`mv{oyTJI^i1;4Z|B3h{INxp%C+fc$_$h<n*??}T_am`C{{UIXJGogY#E6`?)GAbl0Yb1UMF(7yH}9xE)US9yr9hW@J< z@r_WQZy+8ZFvEL{cs9(VULkG+*DqfXr}|FB8(`kp6ON;C$cDHnBmM@;-4Jmduw#KZ z{k_Lj#Gj}OPRtqcc&I17h%W-a!x6s${rOtNt)LwxBmNHJmVtN+w8w13>G!H<5RZUy z@G|1%&|lp_{5_1n&k(2opR*D15@;9n_cgR!_Cb6^;hl0Rn8yr5JPYcx8sZ9&pElwT zAP$CzKZp71Si~nl9A+YJ1pTBF;@zOU+z|f-dT+$_VV)3xcq-TlLEIMhBN0~yJ6jMx z2Knwl+#Ig6G7+cWR~WLHL+hDx#K|B-mVTcD3 zhq)AuxCYosLY!XLq$5tR-?I>Z1?6}eaZ?y~$`IcH?dvw;=b@i@ia71x8W7ip_S=g1 zTo@NbAg;8$xvih?YnWFrL;5K&KE@)xfnVx8-d4ovc)thn0;tcqh!;Y?T8Q{;INz&?KY(_05Ai7A z^@vwN`F%p%5$ew$#NUFR{w|NkM;q$TK!`WxZ(tl8iFiGXJBEnMLw|0GxQn_Vpwkf- zgL$AU;`IB|K*YC09F`;gMM+>k9&!48>kh=NVLo;U@fsn4odU!Ug8fp&4PkzA8*%#m z*)znABn0-G5ch`jYD3%+{1J!xLd%7Yzk?7T3UO9Loc8lZh|_tRCE~Om&PQAZ`aM6y z3t&DLfp`k+uS5I@__-Nz+AcB?7lQdf4&s|3-?ND8K)t$xcuyGDXg^KkIYm$`yn3Xc z3hm+(;vrD~{~*3pSYSvB>IJp$4c9XR5l@AA<4D9qq2HkOi`vnFe%KP}3!q+2M_d%z zqbuUH9R?y!pAWbkadVje#3P;x*SR|or{6mtLfi=Y{{qA-VH_z%{3w*;ZN&RQ`+bJ^ zOz6*>5O;<7SR3N2puLGhInX$h=UtQ3AMsM?4~HRs3hIF_;(K7<9PwZ30{c@C2lsi- zh~I#HKg4Ik{mdl9H$$AaBc2aD9r0Rd7e^4UhH@`JyaeVUC5S&2oEGmE;`I8f7V#1& z_xFhFLcZS-e-HMBpdX^;1SYNMN}ns>FZ$=XaI% z&@X8rt`6;D65^?ng5wt>o(c6o3Go9^9}XdY68hC5#BE`oQ;oPc)bn?UD?+~^4C5`W zKSLpI{SbE&=kXK$Jvi0>2E7&1FCQ+@J0Y$BdQZfU!THkp9ko9m?*F7Ay&>2yLOcb= zqZ-67!1*>KE)Ki{@uM)Fs=)OS^=F#8;Jn5lZVesQ9K^>#zwL>58u%ZKcq+`F>3t3% zE;}efe#WVco(!cB9$p&aa9X@Q3@1Zo8}!dP3@3KR@vZZCd59YUF9A;d83p_?;^Tq0 zFkF~pJIEg{(RnVlV<*78VGt+E=KwcF+zt3l#AAVb0H@{E0PW6~(ZeMW|A;t7PvYPU zb~YmJ20RV%RN%W2PXc~`;UxZ7AoDzilYC1+pO1Jk@UsjjcIfqN8N-R4I?!K5yaxCq zh7&tFP*GknoYb}szHc*gB?wV6FXTjewi?w*vSC> zSj4vjw_rFK>Sl2-k|r~p*!Sd9?%7I+?}Z79H{v-kE(9TN3H@*+aN-7urvcbcU^prF zX;81y8BX*Tpg)DU4e(OL-GM)2I5AD1-~El@#C`zi`-~85P(POeS3^7+_yolBfKNlb z0=NfoTHlgjUgyi`VJgf&B976M^1VzP2ns+5Pu2!-H6u$KfrKO?sH&3%40am zm)`%(M_dBR>ny{G9b@R1${0@Ukae0!x{5e`K6@p@iJe0*EAu@eAxY7t)uypG|- z4*i`{Bg2WEG|+!Ud@FGNL$^5{2^T3B3+_eIK$wS8K2Lyoh72br=(=C#3@3JiLGR3P zqNn|&Kf{TB1L!v}oai;c&m9aW`rV)}U^tQLf&MbXiT*U`>ln`JKQWx>%Rt{#Rj@(h za0mEE#Ge4Czdxn=Ze)W?Q$bJLMI`C$xipip15->s4`cMC9O?69mmyvP=er*9R^SPU ze+0f0aU~Icw!93)>GSS$5FZcvJjCgG+Qo=30(}YM-oURgoNcFf7*67l0Q&oguLb^? z;cPp7i#Tm}-w=NX?Xf5H>!kQdxflw-~#F_p-U210u*zraBo}gcX zI9<1IJ;RAViC||x!-)ynzH%8(^m{>njp0O3+ZTOK2KDni=)WU9ZC_$AkD&VdpjT!% zu}}4S3@853d7uO0O<>0t@$bN+5g$w(;L>*B-MMX2?$4D8-5uXWhwncm=aC^j)fKO*Q@yAV!pDoXs;UwR? zpm#<58t{b-Cw9uj1$GuQoY;{7`yq%6178Q6#H|PSG8dTPWgtEq_&J7?a*u~{FJm~d z6A1d}3@3WJ4(oe{6a5j;i)sosh*_eq2m8_tC;EJWjHiJ(tv^m{WH`yU z68JiXlYHrQ(;kKsJ$;UUHp7Wt5A0lFIML?r{5DbF`Vf4gT95~L>~gYj~2JX zC9+HOrN9RtW{^t`?TABp%M;MRyw1b^HaPW%i7?$2=I=Um|15cdOq67ewLw-H|h{2k)7 zALHo?940c7ZwKh5ffGjR+bX!v*^kjnaGWM?kxYIfdQuO0-T1paZUIStqWnA5LrcVM zKyQP%G4QzzC;rg;X&wwG`KEy08}Ve|{tPE}Xgv=HPEKEndyVH_B&}jNIbZtR_-)9J z7Tl-JWH|BXG~~Mv@dDt7k$rQBe<{OBJt^W|Bvmq;_+tn92E^wB7t!M%#xGlP|7V-&&xyE1%n}n%$qtuf5x*nA zJSB#cd_P0JnuycuQ&Zriy^!*PYQ?i>IH^}x!A}Ro%Ye^fII#oWHE#jKN&Tnw)|cVL z{&TPsfcR73k%)f+z7p{!;A?@CxRE#vgE*uiPOp#7Ax@ura0hYveAjmjC#J7*FOt48 zoaB3qPkB5^xIm!gNcA!dC;IWA*Ff9>xE12Qz+D+m{5b*KkKx2VZ5Qhir}zJNBmM&H zoI#xS|CbS`{rq!=v+;S)aN^HCXfGm$f{bWbl^fpf(;sf8apY56Lu2x!w{$UD@+mp3VK__>2-k@;(eh$ zL@}J4uMF6UVL0(e8T9)RR|kF`aXSC6K%CCsUL#KD|7{E>{tN?uM2rO+Bs=0yG~}y{ zcmnXzi01>Jh++#vQHK(B;2eXg+y;`DjM)`;%|J06JB-$g8CIFZg0xWS8N zIPs?h?Ce4OI`FfIe*s>OxDYYMrFz8s0B>bDG2z8mbALt)HfVg5K(C0n8gOn=bbg|G z`aEK8QTqSk?x1%=d}4PrP<5%{0zFr4V;f!>9{PkFp%h7&O8`caUtKg0R{fPM|biGC93Qy5P41E5~z zFr4UpK!1thL{IDCeTEY~U0^TCzi-@%cyIVVX$Ru;_i5>f*GLNN??s&cPVXS%lcD}( zBfc8y`7y-h6b1H4S(8L$efe90KlFY%=6;|dybq(_(FHH2_UlfHUoE8blr~y z?q`4)98!rTVt+UZN6$SVd1Hn&g;I>k%R{?O7(W2z(fezW9v2vALzuiQyh3_SxR4qe6IXx;!|m&{e6vk*sPfhES#{t;G-aUFFwLo@RO<-_Ym zg)&G#%A!Q#4mLw0#iQ3qc#L4j%MB+#mphyxY zw@W;UliMYg#L4ZFMdIXk$s=*(6M!z0IJsRaNu1m+H%Xk_E^Q=EZkHYsC%20je+_L2SC09<26wK35+*Eo`dl=$j@T@ z8RVBR{uy#~{*B822lDHfJRj`zs2?GDG#}LsOkM)YH(^{J@>a+}ctgY>r=fi`zCkg{ zOA;rKCvq?|kSC8PL8RjoP*#){z(ESglgE>NBu*YrIM|svfjoIUaU*f^c#=orXwk0-GtP99IFOtXw*8M1!@Nt`_1qHz+qiR>TLj)0T>!#MCK7~o|8p!@CtC;Mj` zC^(S-C;O*{#L51d0tAT!zR3PT*Fga%`v*M_0XW$|=)P3I$^M}zglzSrySNhH^kC+< zH(H{5gwrVs?k;o}ih^rM2t^?{0IUE>Wd&to)x+0C0SFOGApxPmZXO{{TYTu=PCfxn z!5)DD!9?!AGGRXc9!~Ude<$AnFQ+ZRK6H8h^RUA#ee9e2fO%(c>bqWHy1Z=52w&T!t=mk<1YWV zJDvPo{9U~MQ=zYqYp_f3*8feVOJJarhd-U+EoS4ph0sw;xqG;VdO7*{dj=3u!dP~3 zbq)3igT-Vb-vBpXpAb5T9#D2DN7UWRuv#lenbH& z3jP6f4+Ud$8~G5riyILGjCC*nP=&zYfIyF6I%;nZZzs=S7e5c8AN+rj<`GIj*M-B? zxJuperGGBKZaxl0i6y|uV-wh@3qB@Dcz$U@28>7l*Zhl#yg(;9E+Q5gNf&;j{ViN& z$bw~54qdp6*B{+afbQD_c|anMRbU-lMg_YVUE-gdrnxiM0^#7SO00rKH1-NYVIT&| zg4?%%gSQKmLtnUkCt|}y_QTQmqWpOI_YxZ>D*q5HKe`?c$`AHWc=?M7HlhEZA$0!-A;w6!{UU-+WPchw!1M+i zi3IF}{UC0?i(nJlpG|Ba(tp*zosgNxeqazIJEaBe@Q4KL1DE3VfpK6TiV;i9gx)7; zhXDM>>;Hq`6WQ0LGc%y$0AL^F#qERNNhYd)A-vFuuJbXq{k#8w--9NyA0Eogp-ZYi z*ze=^RVJ}-F_)SDJ2DKv@%n@7F%#Lp4o?`UpmIb4^#|wJxP6z&?86R@o*Mul<@jrn#BGPyl{YiF9i01O5A?eB=#-9j*Up@K5ORMzw6&ViT!Wz#4v`~ z4id00Lj2?Q*@^fBBKY_h2!x3g$NU=q#Ou#OPVE-S- z#KihHXdi|lL6`~UN8ig~{&B@njt^c)x&?xslu$l|zd<)=n7B?94r3MjuQ@sIMO zav4uzzgL-=UxL_9PAETO>nF)SZ3$Dt1Us<6^MmJDCn|r+D`tK^(*9e5Kqx;dKe%4N z2obaa7R*kaxv&L&KLXn)vETWQ zDUr#j95eo9Ca|A5iG3vvX8x`J;s1k^*bf|F=121jgZNoAf&Cn4UkN#enT=AooVjot z>u^X6R*OE37#!yUop^phbEd>MYy+x4%Kx6&K2iUv8DdIMVSz#Wfi8{m zqxhZ3Hpt0m2|g*5UU-oKbZ0N(AJ0FXU|5I|gt#9VUTB;P`!53$AGrT!5^SRUy`Pvl z#<2S1`4yE_G^nJctiDu9ZJDa7iZYd|vP5~QiaPUCSxIT>k|hLNrn;1w!I?AK(A0>P z<*!B-;x6*PYpBE&3ObetM3;eeDU!sUl;Tuyxd5C`zSVIILDJ^}tz6;5Ki zwVu^##x@^!su*!1Nr?D~QCFE;*jQUpJ$-yVeB71cH~dniDkvyW*VtH-M3%s9YG4~s zYqZML5M>i%UM&!#hWJEya8ei+ttAO_k|E=kD3T#m3epKh0Zmk`*oS2Q6v8Gtq`+3yCdNz|BMwt{^m+`8c( zn_k|Waz?U?Kmb86ZqO}A!(?}JSkX@BnPfKiKf+S)S zEv|;n)(F&B0^_EM&uPhOK&f&KQ^+*&EpvgE6iH()y%+K1U1$LE5Vc?k(~>L%QHH;A zP$yv`w4Ijty$tk;e-xilcS7Fcf7A=*{-HOvo{yIJyBfHB!$v)aKMa>J%U1{LK_rHM zN{9u$b*mEZ>)Dc=BnVtYBYZRsRg&R`#BN%Wsl>EZiH-mA##bab?hV4t|GOVd6UU%R zq~{v&J1w?e5Y@p^&xu(J-i5muKC+yjacGmbDEmB< z^;;9;f*x!-)@xmR`|4Ht;?vFi0`i=TWaccgY26unm3oRg|7>m=r|Rp5sH$hrdN#%w z4FB$o(kZ`B{0)BGx_Z-{8R8vE@^!yP+UGs)do_2IV^~4SGVh+&u|YM>gFJ6vz1tWN zJ@`3ZifwoBBF=>^UW>D)f024F!e7Vjc6+I){-cv>*V%oZmA)`@?EYijUqc^Tok(Bs zr_JN=u<9Ajr%jxDM|obGD{8zt^Tl=QJx5K8!`BA~T{WX_jD0N@sZ$E=E)&UbjeN0b z=z7o0ju@Gs`7)yZlvV8yR7~@Cs+=`hwA;i|=iHv#XU?6~Iv3Aram~7AvF3A~fcXMT z-SQkBTJ1}G8<6w)tkk&{x?|ASd)7tvg}0xUSV(%hUoMZ{X_EQKzT|GQwYuu(s`Iwz zP6&0qAAFqe;yGltr8@0gPO(+p*<XKAM!}(?!KuH7r>ct?IV?0ga`M4~($CbBpNBrt2X>XksTR$Cx1hP+ zTt;Z1|BI`1`)t~*${5a5>K3bp4OwJQIfb9xH}BL63o~nqTD>m6#p4gP@g7Frd=^cr z`Y{o%f-N*_)_e8Svf~7&U!8S2W~u9v!ZaGoH+o&@%SusOa^}#Ztw~I>!n}RI6@0-8Q?eL2EFV&qO}=*RQB_gZ;l^ zD@@hh|L_gXj-)iE{SvQ{+Rm+XbKnF;`9&T@y!p3~h4}!7;Pz3LY~D{?F-6`Br?Ai7 z&BC7hOEYlpC%G8QO$%A1_^Me1FOLT7h)g(5dD-!MzEW)XY-gp%O)*>RSB3UGeK;$C zWv}lK{+gS`YuJbdZerm+E3qH~764%-7M`&Pc8wawr218JDP8^$4YoUpK3yQxHOfH| ze_WL%&^5X{UxZlVtT1z66MQo|qqm^%tZ&^e1q-Rr)X$b5tdA$obE+Mhrxd!LeH&L{ z0+lb#QdmaRm%2(rck0O)6EV)5glz|-_${0-?`P#S8@1FG$(P&{qj^d&U)m(b=oEio zwY#%b*^KVkf(^%tbygJ5DYo=I*lAm&-7Wi!MQ&ruH<#MxqeTuDueqkS%{4tdcOcV7 z#pXw&mx|f52Zm?uc@HP~9p_i>SeGH|YPq#9I^SiLt+rg@#rmZUN8C0f`^_l6*7?X~ z!+yWhTJ?*b?Ypag!Lj|jK0SZ8!%q(gZ;D&yI~F&5b$)Z%>ub;G@dG2vo)yz= z^9Jsk*&n{uKe(L#K$m0X!EM@$wa>Sy<}O^-B9~h&QEs2N#dL0i|D*od9eTCI*Jh8# z;Jg!QhdDHVDaRM8hA!GEJ)__coqdW;#NsXN>kk(_5#paG{7KWuOR1)+Md-l8U(59G ztesiV6cYcOJ5noemR+Nsd&b%Lez8E4-#(Wj ze9T5Pjnz(6ZE2X+ATdNY3E9%jZk|gy;_VR;p?3CuvsgjiI-As!rH&ukpHepGNwHlx zc7(Ms?47t|gs(^RrQA7vwpVw4^L}*I(B`FNVaU3+m}~WweLp(e&i7LyBh=)IFWr3K zZ!@Dukyt6WtJWVK-o&%0>A2%P!`6ZRhg#0+$_t%K3g%FLB*amgC^lRo5_4EY_}X22 zXS_IQ`LuJHX8w<$H-eTKYS#IZ`Mky)*Z6sFuS$ua^h(g=*)*oF<#pM1fLIqKKGsja zB|%ful{{r6BRjU(T6Fykn>`ES)}HdrHyIYRmX8oQH#5rID0I`tgjkkqI`yqF-Fzk$Y}QHC5$HWROqB%*G7oOKFFn7``Ezc%%DvTk+(@H#g;^#-$HFI~e|e zCcnY=(0cxk)%Q4Co5Tyswp`?YI(K>RBFV8SZ@o@Q%A8qWUvqAz^5sp2cZT+co*Mf( zqwLzA^o;pIA6sN1@-0qmo}-;rFmG#;_tf@9>!)8!6t>uX@zj0Y$AYTUc36J!=4BpQ!Gn#8KuXX;lq@@DyPkuU(&`LKPmY~}V z^S>Sa&M$sXaZ}QJwHV26ZbL&Sk4XNga!>OC-Nxm;ik_~ZI$Ux)XTRYmM=DME$PxUeCG0=zjEht zUx5!hixaEv6ff$&S9(1`E^nil!m+R<$Jb9B>PH@IZ*?5KD|RW9C+WI-gnqqRo8`>8 zW=#ewyuL1eA^gs7=3GlJgX&lAj>E#zUry;~-A%gQvuSQnZRo0cfewM084l|Vt39&> zEF5-xH+0x(FQ!oL+xoF|AAgX*oY_G_wqj#!83#_S2$q-Cyw#A?Cx1!GFz;L5j8-2R z$+6jyVy!1it@)3=UT--(ci)0jAsXxCGrtYH~VsDMf7xz z?4*T$?JvJM^;YOchrsvlP9tCMwak6{?&~S`zUgk!QQPV+9*k0}|LUH)eaY91XPelE z7xM4t@1JQ`KreqeB&_<8-lRX(%j~Uo6wT{x!w!$!3Q;K;7!6Ml{x!6rhm-%C>fvr{`=_9?EA}m)!!bj`}M{*FvBm z$`hCH<68?hbSyZ(ch%S&?IU|jBW@pUm(F;%NPHyITryozwUO${@!~;I!cl|M%KH-2 z8ed&kxHBzl@9N}JJ;!F}t~;GHy&FE8qd);33aBk^gsobQS zPmDQSb{|N8=CD@kVbQHPO=S6 z)5E5{L&O%U~0%9!xm^^atw)!!#AH($Ev zibYGCVJ)>%=(M`cpeJoq-tPBM#h<*P%g4I}4@GT=x^ns4AK^n|R)aF-k^`P^wNAy} zFDcb5a2x(0?>2n#(sHwm;6#VnJKQgZRoTYg~@EbF>~1;gtz+w#dGrEKu4@ywUo2 zi}>7G`N4vT!n)H&mn^MD{vfTD!Ta(h}7h1C8{-*UWeZ4=19q;LyI;N>8 zuiTk8jr-#4V;wgeomd(*w>tdnqPm?nZfx|R^Zl_IJ*p6g}6^14`^KCp4~`iECH9h|fB@^;ZHS1aAz%tzHN{2U5i@PxE&<+<`(ocF%mf*wwX zBK9S(IO~MGj>!Lc<`nxW^V7cGBgThX)K+h+bx`d+Qp<8d%bU=oQY1Q6a&A-$&3ld! ze^!j-v8zhw;w&>h1hw~6?I~-LK9VeKabou9mqjj@L~WK7v_02&dcwi{`QSPJopJWY@SKm^#cJt^{1-_079_MDo^{8?+1c5Z!otJu z&UZhCQ$on{J4frQ*{$D#xjR`X8P8Z5m*0>)x+e~;hi7ql5cxfNot-Id7Bx3onr}Y1 zr?Df|L-^%dBlfV&aIGx0gVUR}Zcg)(x?vy8P2s;W^(x7J(7)O@*+SX8&Vu{evCgqH z6SkbR&I3DUp1W*SY+_L~m)$N&cd?8Jg=OQ8K?9L&>1=7|m5aFto@CGC`fW5tOgM4H zx3}NBK34v;Vs#D-pr8gG9!s!pOwv&&g1sNJROQKN$K?T5QX zX7GoWUP^e?yp*-FVJQ82u}Jb!UC}p<4!5KZuAEhpyPB7Mr|Po6m(koldoq1?e^PmP z!Y*3k=j?4S*9|2$UUy#aGgkOZJ9qXz$8ArpOP0KLIL3LW?r_BMqi?q#DSkK{y02gL z=XIVcx~*i1`uk+=-BC_b$3IEleYL)S%e_OdLZ&=3wQw*x^+fuWox=D}b3@>V6_Q`W zv!d)bhMcC&dX;x~;ZWm;CxeCqbvoV>zO&O)*;c*$WMw>W{Z7>-A~VvXs<&S`w?cQp zRtkMH@fOO{FdE{CR$s9i`UI?1^=;r&4NaS-Af*gVQ-e zkrjiH2O3?A90SsmHeO!U@XJ#m>08qW*XpN|&)h!wRaYc!Sry?n*WY)K?73Q<@(sQp z>j(Qq=e&Qlq4?(UdWEWJJH@b~O0%$y8%3wN*XjgT(jNU-Jm)C=*Q<9srG`G%Z38Y{P_JU)0~vtH|v6}x~%I7ix* zgZhrmm+yv3WKx&44)w_XIHo7nP~?|%=i%Fcm5-JU9i{*A_}<~wj%D30WKZ>-D>DuQz=exsxWer#eI{Bvdfs%q2$~ovYe~d$ zT)gg0$YRyKdEJg5+az};Wz0FgYAmYA+&27K=S#DZ&c3dD_7!&ye2c8uvrXoYUy=FD zt#@Zjxy$oWUf%BXD!KDG{F?ER%Z6pA7DQd-^f1_{{dHF4=`APilg^ctJd^gh@!rtr zPV=D;)T>i!Zlq5+$zitBqcNgKZhz9Qv0F{0cK6+$<;e=Xt?|(~-}>vtZr%QRh0^`v z)V%sd_pg;bsiA)vT>5w*z<=TB^7@pL0?vUOTVJavw`}fO=y$cT$iggtToqSRwrk+JLo+Q>T6^}zDrYN73Yndeq^vSOqGSB{p%CUzIp9R zN5m@cro7Gmu|#&ktc6>}lQNEe3>xL1YSB3*d1LKS>34y&K(QC~>^bjS3m!gp7QevR zw7Il>neb12v7w?auRDpJ>OOR(l_haaFSJ)WN ztry`VOT;%>&1dxobo0r4U-NG2SesFDRY~;muXQ7zBkr7` z&PvpLxNzl;0fnc+UXf4ow)DSI)-F%asSy+QDnBD5XGS~OA*i=gBvd|icvruWm;4pM zh3CJF_{L>b?T@J1e2!C2{d8xr4%hNUtlz##e6-pXQevXAJ*J^D!%TY5XpZ68#s#%!66o%hTZEIuoWpatSziO5Z4#~+fDr7K*5}@cWm)r`f zmUOtSc#$?_{bGzu*YDg8*S8s6>BCDuORvxI=MUs&8BL3-VTs{j_Dr+tXADpiC{k} zArUw@vb{2_{*|)^?{_WxjFlzlv_mc04ovI5pn38N=e)i>Igc{eSLA=4_9!r0MCN;!R_&dWZoKxZ$)8P^PQ95a{%Ol6wcx6@+6OER7kg#u zW^u3Psy=e&UYeoS?E6J`KV>&AsJ8lS*z*4ESfN9VcD=B9u>1n`u8mpWh3<8D9Pioh z`BK`Z*x>cNtwq}%wDfF6&7+jVtPC&iKeFi3k$oxdzvdmuR#++CE0=ddFl_#YnLoXA zW~!bnOza)jbemJF@zsP^Ywa@43cLQrkKN4;QZrmH*bguIaKn8u$DO_`oy#&DoID@c zX3mS0Z~mEAbgIT$$5^V*d61X3Z%J)r`VEYBd|~0&b{u?@oVc%fh4HrO z7sS}^e~eW*xJAA@l`r11^_}CxCFk-JzUBt+a!Hf^P+U~ef4T0D@e|_}npgIV-wV|k zao+8oHQ>L%)mi+hZ@^sp0hY+1u_()P2i0D1XDP{E+?;G*zEvzIbjQUl(Q8+xIM0#L zJCVm_$-~jp^Ci5{PitS{HQ#g1x6T_xpHTlNz?wP|B9;a{n3y2uNY`Lk*d_W z(OowF#<5*7hTE@|CLTYdU~a=QPwg6K!r{+$>++W>+wg2OY)Mz~ySTAGbBok!FP0m2 zEIj+tikIdKhPE4oF&JEhdv$vH=2#3(`?RvX7!evSGiRQv%?j^Sz z6eFz+Zw&6d>fM^7y4d=f=G`QfWrtsRF}^9o{=b=(qPr~$UkMq#s-7^g8fx7BKgZX9R7Yp%2SzH({e$@-4YTctJ* z5A+^IXfMAX^OJkWMWyiYQ!#EO3#=cd=r8UFlzqa_t8;pBv9?6UeqC`DnO+6Y7o}RA zUoIp*xi9S4zpmWuT+(oH*pWG@eh<_n?6@=3o-Qni>r~kBAtdsF*qfg13l~Sohdku_ zP{V5by?n*R-Ula|)XnU_dc_V?PmiPXZ(@7uN zv}cScu-HltoE;msqO`T??iSV`KKQbGXH2!<_Ph5_HZQ6DL3v?6CqYSQJDsj4HTC?9 z*HybkgD$+bG+n>OUqyswVnMh|kbz*S-gZsX1}VYwS{c)i z-*H%ZV=MpKi_^qn*z;?@TwF2r=%vH!=EXNO z93F_M+{*ns_LpU*w*KbZ2HWWIuXEx99)>%5=oaW};XbXKB z|Kn+wwdKg|-x91Es@b`6W%Er~!)43OeKadS#n;W3KUR9UKO=E4#tol&!=|M~S;NDF~ zGW&-7M^{f7etNvkeC@W;y1FyVc$b$aY0b|%aj)5Km*@r)@#g#Y_8$8aQnpO-jLxa~ zdN*sdZ*{M_@@M*{#HFd*V@*x7-LH?<{@|;7YdX_LS7dCCwY|yM*?y&K#+{kk2Yc%l zvGxy0_XG~Tnb-2$qU#IomeDT3{)KM~Og@p%GtfJ-|DF$CJkaW2N98vBqMz7t>Efu; zhjn7Chr-!ryXfAl+P+IN>RpcMyEbjXAKjO;Qt1n)Mjxi;7M1kSFYl<=RNA(Ezs~$M zRFC1F;Dy7lTyFVo4sLLGwn8!Ki=be$jL7t~~qPf5B=U_AHDVG=6yv_d6!91BFsTE~A zcI)lqlnr~yYU0FIzx6?djAkXL@$=kgBEyDv4xINtTN@N?zN@s5-*TQ)PpM_t|!?%}Jk4G?`&47m* zJH?nsiM5}Hx~h&xP_L1PZ{=Mg6Oi`>dWcjDK8QT~Sq*sv_p#~mP~)c<@W{kYHRjP} zP2!O;s^g&&83k816PfSxOp_tR^FbaWxquHN9=>?i)jWc+Y{op)ZG@FN9zG%iQ8dPV zsO><8NbuB^xiZNo@XHj+r#};IW@C8s-&@!i-kpf!cRDg<*v9Y7`DY%se>_$6cN^P3 zo+=`9^3y|P&i8kPw8ZwWJ0F=F*(kF~QhyiOD0Ba=ke1jOPiv9@Tx6rb8%eQEY~zvh zcLTUTml>UZ8CYU!m_nic6T@{7#s5sqpZ}*);4PTr8^Qf=AnL)rROF<{SeZ(JxBQK7 z0{2V-d0oc$-;ZUC#i^8ae-UFz2Xa@8>qG8_aT?^I7zekG%_7|q%jjDK^C$i%YDI82 z#zY)EQ#p$iS3)3=iFB6upGf}lB)oJIUONdVZrhnqf6pX*WD@>^;Il~a5A1=ofT+hT zQvACxF-42P|Hl7#-)255McnJfxWx-RfTrL^Q83k40w2VKY^M4u;6oLBECC;C;A1KH zPzN8x15c*J3$ln0WhKhLl`)@YLl3Aq8Jn(Isb}i6#>mLZ(AvpbZ>6cB6UCGH#M`8q z84s>GfoGcBLjx(qmz#Gm#WOJ2$Di&=@eKCxpt!lw0|IvX zKk+mk6MF`R(%mL6te=m+j}yb_q(uVvw@t3VGk|#DDwyGUbT1;foC@OV=O1Q5{+A^f z=!=LiC6++~^29wx<6H|sBH{8jOw5pT2O7vP`_NY8d+86=<|>`M{f10a!b{fC&C!6k@gB!3lhpaT660i%#TvQ%u0Eq-R+5amTl<5aB z)+2di$Z*WbKUN2^p{@c&nNP5VAK0wj~_I9Y8nIw6#e^9&Newa?u z1NsQkqd?+9B+j_;l32(6U{4v0_*bNgGhh_yMbUZ zAQE%``j_{@f*Ue7kISQXx0qu5F>or8kl&F0-@q9}LUDq)DLjPmV{RVTKLy&QW856- z*^2Q&Xm>ZpF9Bx|X)p6@Tz@gFFN!Nv9%U$(gULUH0ab|cgRlWAFfIgwjz~9|U*mQk zzy^Gb@nG1W-(p+@*6Rz#7s3GigK>8bCg6hlP`!4;fD**`5Udw^rx22#0Z%T#Gio3q zo(h8qy^|dAdr;04lfMAlYdyvz;6V_2Mhoe&hwT!E$yY;v?!q{Fhs6Pm`@loxY>dx@ z>)_cfa=nDPnClf7Kg-7Cw=s^Mks^L7Wh7L7Yv`Xh7=I5BjfOG)9qMO={T}Jr0y{Jx z#<#+HEx*9n@d2xJI+iufED4>U~Ae8_DvE&(~3TMg-vgFG0MSAjeR<8L9~ zhw<0YZ^tmM599L;#=GFSf!;lW$}=0rNfjo)4BEYe@feViNSv@8ke&!Q=<;HGF+4S! zkMVVI96;|3L3-A~IM>AF2cRBfj3>bUVvX@FP(OM%EYiQ4d4fjq$K>xr{gD{o4Et9C z#?xV(q+z@W%Adlx28`zdj0eE>s=)Yurkg3ZFunq|%Oi|yK=~nzXF`Ae!nhb}*D1`5 z$Ul$acsm#4$*`Of7(WC1oifHp;drZqab-A8uEsbGmeUU7?66%tFzx{T5RCC0=>KSp zN5cK%UX0&_`#bc`JLHF4*e`N1`9kQ2D;S>%+m~^13iJ=+AE0`6rho5Y@{1uybFU-$ zEwEm1G5K)l&k>BHda}X(jr2Ida?ZrKIuIbzB8)dfyGt)@Zn$C`&3WLBaR)f=1z`LH8^(p8pL;Q$0^{=o#?4{-j$vFI#w~h>5sHUW*uVH;+#${j>m`cu zqtFiu7)S5=UygBf9Dv@Li1cT}^4nnYC!t+ejGMsr3c&bP7(d%E9uLQ#M2t%?!<>TN z)riV-2+EztP{Ju!~P z`4Ehw@jM3Omtc94F=jJq(Ki*gm?R*<8)@KC)z!?A`q3 zEGK%0Cerf&w%2z|J{FD}>~I`H^7@dYcWxq{0Q)Z$lW&IQk;gbMF^=Y> zw8r>$*uE|pPlbN=$M_b=(f$^dpSa18krFU@)SuHZt`FBwVH~}iy8z=-aJ?DhxiC)J zF}@V`mu`&vfC3WA=>|-@+^dMHkkZ0sK*cE0?^MP z7`KG|WGlv_;W!zM@r7_d6^C&>xIaz8_#f!cRE#UZ{*sAtXBfBWxDVC04UW&nn0yv2 ze+9{Aw6CgLao= zd^YqmI$i^tK^IDc{mU4W{|)2K4C67d-&tW?8;&RIG5!;_OEAU-V4R0z{5AB$c8m+d z*0C@#z$bkZpZin*zVmJe*pW} zbBs5`dVRon9&GpT7)S4rWrzI_Yz9r$1LJiz#z$d$Q869|+e;qfHBgTx#;3sX#~9;F zpdYL;-V5W+1>?bR{PD-QJFIUc#%IAeNx--m?C)t9zY6uA!niE-V*$ojLc0|hcZBm6ULW9e{#Wb1+^o(E+mNYaM&*-F;0VA1>-W%-U^JPxtvWg z9tZtnk8yVBA5V<0f$Jd{Z-Vg?gK>WtC&?I(h5kH(@qN%gc^GGbdah!e9qtEfF@7J$ zPcz0{V845e@hT|ahw)Vw~vm zjHHioFF5{NU_1ft$2Vf!5ysCZjHC1JEf_xv?e4<34&>-}1LTL}aC|>80Xy>w}yW1!?+Y|-%*S&hWooIFy2vnNx=4csVf%JsTm}074aV2P_8rD}6C8(Gp+8XjPKW;F z!}tm~zAeCb4U}JuaXPG*CdQw`a-x2K>`I{a#pI8{{jWR5E8+MSjBycIuNaK0LcR~< zXs**NjQhaypTl@F^yhVqo5FZN=f|jC(y(8(WAcMA-dZy z(lI^^{kaq4;&6OU!8ke|I*#$}F#a!K+!Ok-4CBw>`Nd6)3qb!o!nhNRlUEo=bDV$1 zcoNjZxOtKA32LuEcs+p!jt7X>!~Q6O@lmLE5ynm7ahC?hy`VphFy02oRcnl!g8~rA z72}uT`C1^x%^}~8@mRRu*^BWixPAoV%b`E>Fg!#J11_y}xYbbSTsLG8K{lV1tj3%yev$*aL} zz!8%df$P2)N56AKVEh#vw-YeV2mP6j@ij1Rb1<$C{dooBH{iHYiShZcUQHN3hR)wH zj^?0!i*a$---j{246d`lc0hhZ*VA|~o(KDp490&!KPzB-3EZD9!T46VztF_^PssH! zo(%V2_83os_XXEVmnKpuqgsgQ5MxHlYUwqx7|#z{KHJ7GLzVf-8H zFQ+ix3)}ZB#+{-6uVFk4`u{G*M`64@!?-lm{~6=KtJGc_einA^q}2>JBb5Vh7$iEZ71mmMMT$$(C@UMJn+x~ zh*M6HMhoQ`{Oee_$za_x(gkwAg@;HnaL!OB7 zOOR(^ychDL7=HqJHi?6FNA)^S;-H*kP`&`;qmWl%9NjltgK_YRVUX@&Tnh45jHBxY zk03|>L~->J;~Gr$6c*V3ko-o-=a4vPD%gD~QX~%A%M;4$U_21=)fnFgxig6a{i={} zB5|NU6Y?z>uY!Cpi32_AkY|%P&~p>=ix__c`3({$m**~t13g1fz6;~CV7q*P9Q79& zH~@Vn$-^O@p=l-@FVVP!=9(42xFOUp3^`Z@dc_3+3%Fc`D?7 zBo6eW@n<`U1OK3uLE>=N%h+@oa#Sxg=W{W}(eXny#*e}J)?qvi^1B$n1$hg`>mcvK z_#os@F^=vpAA=l-fqKEiLkb%_E(Y}i`w+Ar;U#e}4hX|Wn2m9M$VD)|0&*(Gmq9Lz zac9WoF^=vZTZ(Zyl-I<#KjiutKMc7s#=+bSAfdTbQ2SOuZjH%bhujt8?T~w79L;UC z8FJ)*Ii^`kIL22)o=oB(K6N3_A#u=-){s|V+!69y7!QNIhs4R_Lm!C){V`B}1mjtd zbHYFZ^#XdFTdt=-UazBi3h8*={ zq(21mEtq^XlN6)3kWBeqPPr~>`$Wt(m_P=QuzXj!wU>xmdPhh+Y%I9GG zCFGYN2Vp>te-vMUll$W#7%$+99RJgyKETQGzn;X&@lPe>Kmwb|cG0*3IN5F>5GE4f zWV;s_l*!3F!8Lw6A1o&E5g5K+MPg} z@q&c^Ns#%j1o8~H|9_q}7aZ>4F%CjX7ko^RV7D0GB1$X)2962INte7rTE_EB6Wc)C z{$KMiCh`KEXg@$KGLkO*M&le@9lz#=%CVf-Fj0PVyoJU?lpp0o-(WugmdSk;$m&7? zbRG&ANPNt%@lO+C35*N6}Ny!=sh!p7$f2K z!S%t3>}GP9sLGl6}u55w)Z z5^N&-DKMa1m}-KSr>L>@$=S+ZGW2NFAvR8A zp8`)Dm0)IIADlP)a0ppE7gpUt$z0k$EEfO3;3b8Z#%b56X_) zSDwUvCnzkDq8No^E`v{$9}$B|^4md2S^P(S@ZN!m%Flf97%_J!%n!;Beu_Z$QU9@@ z#C|?Jp+oy=R9=)Hty@oG|C=jQA_WQp`vSy2%8%^(O=7=cBKv~wPy+TtP=9bK5#>kr_d)ykoR{eS z3_9p^q=fRLVu9!Xz%riy+d^h00VD`@;Q7y@{3IJ-afT=p$Y30)L*L-M0xgq8)kMi_ zf0uFD8w3EybF66^MUrG z68|VaYQKqWgDe&x1Ch}0dE&$}IM%@PgL7sWBE-_)BqPOKb^r!Q30q~nvudLJDX{#n zungELMNwHzi8SRU^}pu4R9&K>yp)mQf1mSGcl+N8=rR^{DbmE7c>LVekqV7iY-2sOYHx?Pk? z1upgR4+YH<5J>m&Bi;e$RqzDZG+uXc8+%|`V`cVmYcn7!>jX`cbU=K_+ z`3KIvsAsEZLrl$SMcXl;K?T!huH&Os10m48-DycvBxQ(2TH-DV8S3AOH&-P&@s)w$ z1I^)>C?Y{ie8!wMGOmU|-6JG$49@E5+&`8oDMnTv^Tr7f z-ITVY9@Gv@!l}1WkC?==j(4FhQQ0LDaH><~8bjp|PzBQ*E%|~3l{grnB^pQ&vv9&` zNx`g}pc1~sxJOHNkf53-ey#!sD8zJ?Fgw^BLDW&XW>w-7P<&z{OQuSu0B{eu8cMF) zcy6!}9GdCH{$iz{{#$vH0G&^daj}qLzJD9>n!O3dLc0v(K7tdj zCd|22NqRynX#W&rIM$K)WZ{kDBXYoM(vqzus6?g5Tk-!`;nlzX{7L!$Sm#&pPB(B2 zM@iB^Rd*m_62|GcTWE>3jMj!Ss7EoJBT0CfYah-%(M*oi;pc36(1a@}^2|>oh zz|p!;8#643Rwo*lutH4Gip-EBue0F_NF|6@qXB!F^F>GLL<{DOhOiS7&0SZIZ zggIb_@)1K$Hp52e2F8Fxs*p_;FW$}nf0r>H z{+nN&glA2{cVPvp<|1rE?}J|LLWDA{lsIGv_Hc3YrX@~m(&g~>aHsUshz%qc5xM^N z1U#waZVLBRro@0Pvnp{YNuA+8BgQx~185`sN6c{SHf0Bm<>sGWPX#oGF)fM6G2V2% zace-9*m@;XVz5u_*nlP%0|{bQ>80*QQCJ>EpNW-z$HlFYkE#Wz4~C>6PwG zil^#+z8YNtUrg#wI4f2XD+W+eS0_SaLAh{KekEx3A*ST%otg>#_UGl zOT}7~#ENMPzW+@4tj|AWm${(sbDdm=uZ0_R@2>3@%J)8eJnJ3yL-G4qWR#}jx`@wF zHpjO`H@x)!qnp<8$3=2vxAEZ~(@1gY^Oe8Lb&|fY~(vlz$%r{H}hIEN8_}t7s_0%GW#owx{@1@b{55- zaBk8RmR3&n%TzsdH^;8~P{6D3{v}gx7GiAYo9XUx+-7HLqPI?P(vt0YQ%=)f zZuV(Uy=q%8_l)O#+RKMQbioHfin1@%z{j;Wr6~vbKe5EDkJr@IuG7^OuX8qSxvM0h zv;V+O7n$s|lM7lJl|*LKxql8G5ir^|#aVX$URJuw63QdxC9GwFM<~mM8d*nG_Ogk% z3i4{)ZsFFbRhk*N$7NRU;yV7Ct?_4LlDOq#o~gJ~Is|js{FUxfex^Mat&xwnpT6@k z_eDwmAx4a3FU6rGsvGL`rSG7~4juz>u#O_;~sa!(MKbN*%-%abxR*pesx#h7- z_ZI7}`Q+WF9I&UsFjXhd`K8SId49{f*Iym+p4Ync^u269!^iQ(Y#-KMJ?7ovBVDyH z;$eu`a_Qje`QBF+#%&VYci85dRMQsQ%bQkmiI*j8S${3*)YzW&Y$o3GXM{@9Z+z!w zTkXB5s^U@hT29-YE`jakw;rYBuj{#Z@+P(FmTl##0J*ALkCap;vyS@C*dL+ZC2Sci zJNU$XvqVh5WwmoLCVbD_H`5;)Z_0^1$s)e(;6E_cdJmn(rl}c(89MbVh@~;)vg{o^4yxE z{NC@of#+g>h4+hnec!O1wMMPRk+pYozzQp)<$Wr?V|!1WXn*Fjbxg_d?bxed=1v!Q zQjf*eT$~|!^P{*4Uv8t)n)9LTB0&O=UfI(PGH9X~G&M$tl-GL1g|Mn_Gw1vf=YKoA zaL9(`eZhh57VqrbQ?jd^GJUI6+*>#K+zAMvWd`gHy%M)@PwmNAyKCak0*zaIm#Il| zbUbL$x-Xh3ASt$__0A=s5$-GI8f`o!efmHBg!9b5nch4+r)SO|OWo&t1o)$Gzc{+D z^W&|<%hTUakvJ3L`k>3Qe7SLt+_n_)oRFJ=DHevJC+C}9b7s4e6CzwV!}xnvMoRef z#ZuF?ZEb}cJLd>#)U@xCsMR~LU{^!n?X?4Ma%sm8jZNu%-JbdNOr-D)<>!JQXCE+# z=4o~+o4Kziy_RybO{RKk@vOGUhJ0$j_|5{+KgS+L1!-+`7ff4o{<+b+WzlNBDpQNv zqHWr*a+RC+zxH_S?df(hVJz>VN`FRFUH_8)x7B_ZJ0%Bx=`?K1)8`0j$X%GnwX(}} zhq~_DA9J}E*Xy2C)#W(7%QHX2-u?3E$075Ku|s|A$70t;2A*H?@C0W_*wl-UcJ>OFaE25q(JE*_ybsI{y`H*zKUN9G z1_uT0R-Bo?>(1IJ?R{Y!yxZtLgZ5>0$(zFG@V*V5r+39G%NwrQz@3+ohGqtM7OWo=k4|dIUJ$-MdClmFZjG_2i*)v7tt-EK|9EAK_JZ!}Cgu3cUMJ2;{+0-I=lfOLeL}8m z4xd@)Ro^txOLN;gPJZcdczwri^nFBwl0fOZ)ZZ1ydt=*Ubp*9m+;coL=ZB5$H6-Tdh`zm&(BHV)S_2@x__wk;A`_j8RbVmF7=+d zI?wU`D=x`x7oXhO6Bg|%&o_JLt*5BIjwx`r+5Hg0Yg{=hLl6S6|yO zzwPGj<8ga*zGYn2+Eji0;BS@E*{|c{Rh&1RQCw2%!SnSp_u&cwrJw-yU+JH!3J)x+ z`t@F?Y9wDJ&c;9DP3{H9W!H7*y(gwn7v~sBbi;izchO~MU?X9cL zi2#X3mFg9KQ%pW?TxWjJ|F-NI+iM58dr}{~VUt^PVYGs4%$!Rxe=zF#wTEBVe>?Mg z@y|Pv3fr#jj|+Gj;v)X^$Bo*ICW&{6$pY4wrbb0`cYS-Z$jwbPZCG^pK&M7*56eNz zhw7)#y;OS;w~nKX<5Rk>`aJCyGFf$h{ByIbZGw)|?gP8h z4LMdACCy*=)OQ~?6kgr5^{-dg&i>}mJoxmm_Hc*oNb@lLEBDAu$vlUl zy{n^c=;j$n|43zNJ^9m@>p^AC$6>33?1xs$-oNeHGG%zZ2xo$}^E$4Fg*xlm9(M{9 zINm5?$@(F7ZjWaFQ$tRz9*!?NPfzLF;#2EeerB|NX%Opy>Bodtn3gr>wuijzy;iNX zdq~r=Ka_V7SB;VC)$~i!8mb{72Q`++Esl{sa%xwpuyFZco~^^e#H4#7ZALzeW{PsX z=}s#;+jvBHq{sJuzfg;1KiV8_Zh$zXDETv>gD%tln zyGTjOk~S?Wl@^qx4Xu={C8R=0McPXt?dvmhU7yiuzV!U=`+nW8*Yn3SFPC%XJ)ik} zWiur)`ov9)AE7#7)U2Z*e{Fnac5m7s% zpLl=VX;-mPAa1_nw+n-o-yOEP#$bNbn(El3(2|wE{O_7XPO~}UKG`FG>z%Wk?d%t= zUtW?X@a0F>iJaRN@3rOfXGhdI6|k$f_c~_^zEg_apZZO&K(P3fx6YG=Xp7QO`k6#m`zxZ3ae!bi{m2L}ptEW{-Lec5^it!$fBKnH9 zTZImJ1WXxvc+bPeOOJBemD1GI-0M0&@6&RhKC^mi!DE%WH8ta7H$^FA{aE!n_C2d{ zy1szA;M%uCOSDhT7HV1GFlwC$zy8eoF`*qsf*+ntJA5TIbL9=&d!?gY9*mAUl3QX} zadh1li=IQb7eCDxv!0O{b=X7Tp8Xl|uTq9PjS{0aA2wZFP@U*^J3H^%gv51cgbT#K z4sf4n%%d!~VuR}IwJ%CODdb)+7`U(GzQ$wW^SARgyz3fN^rJ3qt`#~pF4D$avFe$Gj^uxb(~9fRcP2Z)(-;_cLO?-xC-te3ARaj>%0s3t40BP85o{ zEx5pw@wj`*=BnYkzOsix{H_n#6UFOQIwCK~WSwf*$nM2A%0$G!W_9u!SM=mtEUXSExQL8(V#pw}CHLj$J{B-;^P3y)f zjZuQ@)h`C9+S=jnJM@98V)v92UP<$oj}4p=wsc~Xbqni#*vhY> z66R+=F0IvS;+}QL*L(4rf;{2HUj*i;E&ZUXJ9Ow1(F)5gk|`67H@!gVYHxzgIE#76D zyKjl@qceF9?#`y0qLA?`?d2|$1os%OK*6sClJ#+S_*7iE_wcHo-xIgz4Fny4yH z*nOdE(L{@5o$2eULK?2P_#Kf~d){qx;<3WUEmtco)*k7q@N}Gg)jKq}IabO@GjVgM zHFwNn?G8ujYiR~fWf}5u9}Z0%B^(tf_FSND$%MNaGgF_eRXyPOeVaqhys^6^-lw09 zlhb_We0oEMuVAg?ws1Z((RlqATFU#k*kH+F~UfH^sK#t%A^{lxUld*8_jH36At| z`c*x?GDl^8V~WU-7RhA^Zzdf0;Je|JgZL`Pea<62lcJBjTWyq>zVPbXPCk)oW?93e z#%b2E8n|03PCOewDD`w1|HE1Of<=#0Qis==%Fo&D%6~m-o=}j!UfyY=*s6JlueIO^;Mpiw8D0QMv2p=4E2H8f$2MU``9fs>6m8oOh?wnMBS`SDD>Eu`HE+cwx9QS z>{U~iIf(o2_HP$o?jC8;wd<~)?fv&<*UbDC3P#Ai*-(6Buf`lJokeCU%M3qc`)K6d zek3_OTf9Mi@^bOS(;1b^W*rOa);HYp!^p?7#XXp-F|S^uBjTy$&`UfY4c{J2sD7`p zC*OFqsFB;f-BX=pEciKXeI^Bdh_Xj<@J!8wgtudQ!wtSLb=yf(W@L-&SX`iaZ;(r4sJ z>BU|CsrZ9;pKOv7>I09!DZ})d~i{F-Mhx?=*YHt&( zy#CZAPVb(NZo6rz$-9%Scczrw>wYvrO6TaiJ6Q{*Ga4%0PFf@^53A;FN_b#x=8*H- zdlZw!-`hRkpK`da zYn{Cbf6Gbb)=8IN&4|2k`HoT!_prRU+RYZm);t=P6$`E;&2>AgcBVxCd$zDth{2+s zN~QEx)$qi2^P~B-#m4qy3{EFT%jQ2dJo>ooiPx?TMIZJZ7rw9bWd5e=eY>`Ys0!~e z>Dtv;*j5o`%GXvNvBTDS`~aSMw|Bzlw=90YF)KWx_ELDH+V3|f_bInVe}CW7IsKfO z+xxPxjG#Q#AJ1mUU6EG(F8}7+^UyUxmwv4Z^0w=qRW{^zQ6igAVRAZ9Q99wuP%~*`%*tD?3&8Mo!*!OHpi}*oCC3sVnAm@4s?z{l2$~ zqqzpHF?{t=r24Z#lkL*!!e7VcSWRu{UjH$0)Xmd#G)D-%F;qDG#q(_9@bD{Y=}R7} zd%X`7$utqui5tAeU+wourL+91dn%Gx5o2=nmoBoInX*S{t=+XJr$bq7N8O6dtL+H!suC@5vXD zyp7GuT*8Kuv(`s{t!o%}^L?zn&RruZ(-KrDd@; z`Hf!5v%Ab6$T^-5A2$1J&-bn78y|G+ob0EoXz)GKaHRQ2&(mqj)6)c0B9>ZC`E^!C zZeC@&prJEE%hcL>Tt-Q(xMp|mVAMdbJ*Fa; zG-Vbho(?a36qz@mC&WGQ>9t1Q?};(Gg;tSw)qF>tJS^ect+uJcX>{ZT*+pW0rNRXx zbR?J63tIA&Oudo(=8WX3haT_jJvZH67nZkSmB0tB!>7{|g?!>O4`ocg{JMM68y|nI zA>;G)3+k3zNLVM1$bV{KcD7r7{`ZPVvzdlc@r&2&oi+E@xs)cm0*jj|dJ3LS*K!{V zehiuLnqM$Q_x)QpZaLSLA}UiY8+Wc68fs@dZmY&gb@>YZTWfs;xs(uq!D!;eaK<$)Uve<;~()z&n zqn~`s3ygo6vUW(%_{8IaJxO*=*WE&;D|wba8n}%_O*Uc86p}G)DyI8{evZy z9-^97Jq0o|Ds5J^mMR@s8C2htc0f$r?VZW&HnX% znk3`OyK7`8!NgxXHdNB5o=y$`Bk?R70VuVaL@5pH0r3n_4}!HakNT+&Kj%B z4We8(Lt+<7sAyhZol&GY+-`H&g~cyFTs-rlEG&Qj&9d6eyNR*uxjH4cP5dAs;J!8T z_x&~RZg@O<#B+`AVRdVV{y)>GKm zp!Cbh;eo@h?mfzNGfIs0R)55js|hbZo!G2#I@sH^=xxjc`}VtcycLW(^rXb!ChO?b+PvLFNajMG>KiHizMXBj& z%9~)xmw}qw2hZ@UUu5|9y2+A#j|VwC61!Ep_40@S?Ky$%=R23{6b=rnUa4y1bIv=S zhj;bCTUTO#aXAglB1?|3g#UA2C2#v1KeRS&_ zJ}L9~PWikmQJQ=@HN1i4!jF5ryCN=e=cRmh?ijFO%=W~p`vcX78rGHbt9S$??yNA+ zo13+BOnq>!l*Yw%YucjoLduWM8uf9|wICj~a{;Sb!oJ(tb*z%D%eo)9;e2`9Y7t$Q z(=4~xoX;*!3twHocBOvq>#JTr<1c#dGwXTJ|K;T`GsR!4eturjy;(fPwAMVswe(B= z_M@>!E88AVj=v+fRo(Am>WXZsjD|L+&;@aZ1&l;3}wIA_;D!D){^>=n0oH7--v z=j5J(<6HE#J7xF9Prc_JxzAQ`(s=j#`_74feB>@^Y4&50BY)ASPeZ5kjr{e$uwXPg zZ^Q53$e?nGJ+3QlLL7W?S%eIe{ve%f#>r$6+4LF4&%^`O52IIHj>LCvM+Yw0H zh5>W&+S1sB?;y5*Jwg7tgISZ29Dd-B9e&T3vnDncvm7KSY$V8UGI)Ri+C$QF@cR+{ zIUM!>=9<%ZZtDQX3=(!-Kw|#Cw-z-{?SCCN`|b9=IDYHRq0 z)WkIduNwEp;Jk*(!_w&`oE^@$I(S&aDXz)(^RS93&WX><3a2=Fbq6`-tUZo^346oC zn#=L_{$BvxrI&DaEP)48+!pwHigV@?@vxF9?h5j|DDDmXBE?q%r!b52%>bU>wcAN} z6nlr4-CqgA+jVRYPwzY3By1ya&fRrF$(oxyEIjTo-8Qy|hlTC~{pEH8zy23xvGxPM zL-8ZPJ1Bk}cpJq}0{=qsLg3uwf*8Ut0Vm(MAqTZ%V> zyeh?85f^5M{r&G_KOE<$FL`{R?u+C6^u=ZS;S>Ae)BEAG`r!usa9oakr3aT|U)-f1 zj^7IIOFpI_zP%s5uOFU^ICHIc_VEzaNmW1c&HeCpwjMm|jpNwINVMDCkNnqu`0swW zIC^=Sxpq7Ic#8DPBi>i}OzVf^Z_y=JtO-w+gU5@)vzL%pVXq;t!imC`<18=F z)~3QF;z|&JGLrY6M9_<=~iV&RAuW^WhX?9Ew9Fr=V;Jm^}Xmk zSu)>B_j0W$^c9sMHM%gqNwaF&@zQfMZDJ(~34>ln-mK(i72SZ?mF z!9lE`AZJHEH!SRf7}1R;KG<{SK>US*{qb7%NE0i-*$2&nVA{2+uqs3+uqI56U|M*_0l^P`>)#Roege}40r`0H8^&lO9*5OPXl3B zFXG`yZ7|Cj&6aTeD{202Bx!7IfsTGI{=S?@oFv;j`}=!){)yz~80?5s<%81xpLT@X z7IOTY4SP-$zO_l#;wI+<6A{Db0)%sp&-fhd?=|O_AwPClej54_`9KVj!(1Kx2;YJs za+u@qPY{mZNX8EH>F7uJK@5?@cJWyi;h7jBhq*ZV5q<_k%-3Bu z<#2qCL*&Wv3+usi0tuHy;{)2s=E%YPYmceH*crZf`J}V^h zcs#}q>zRpugyR`1*kQYp=tubOe&kOwa<5Va_aPOE%?jW;risaAx{B z*Jm7gCii2=GwUUc!41(qNk6&1qw}lIkT+t;Gt-Z+xriPskjM3c%Y<-y;B+}~t|K|= zH(|(=;~>^Efx*|Jee7sGn>d)wndPvvAI`ac;^<-4FCIg%!}{l;A4xwxMq`K5BY=K{ zGmr21{t@Ae(LQ!`IyZ1ITMrH+@|%Iv@&_35__Gm2KC>VBvkdu34Ef9b$k#FC+4r>2 zJPz!d`jKyC$m8joBt0(}^79$|J%gJuIP-jfS^gl-E+2fZ^#A~L$6mf3IP-iA@23MF z0$W^Pm=}ScTrm)S4>&$Y!j5p#-|#sJc9`RnTb9o6QidMpc;dy-!)*V740&ezU(Mjm_MgGv`i%6D_Ke#aX~SrjH5C6t z4s+aw?I6AghR9()4E?mR2f5*#O)m#8`&<|AV|hXJ!^Z&Z$ngtv=DjtXN9>5*8N3`n z84n2$!N!rp=WN&!egW?whx3RX;S!Kh6Atd>8*n2<&YZL9WRaJhg>9Q)60v&zmei=1vnsu;)e!uz(I;zL51W{+#f3N9K|=|N3l`;VDNvWco@{Xt%E@`6B2?WDQ6n|Ba)b#SQgC zcscY_VTv!t0?3V~couL)ij%eT$hRqp{)LdfIh1@4l>Z`%lm0`#(M3NAdU2E=jsb`fVX!>pZ3dIc}orfs?8Tdttp8~t}6c+}&O%%6+@_9hb_BheAF``H*h~k~QfGcZT$gq~!G=J<=3chy2P@d_VMe9g45RhLD>>agvUC6gL7r z&J=G1zbD1Xn#TSVUk3dvgyLlF<4B5cgyTvK#eJb3$(od;+{&R}Z>Hq0Lb+v9yb#(? zF2%c{e$P;RD3sfIiXVr5ahc)`z{@GF3HqxkJ{jceDgF@Tn<%~v+SLP!$Ag}Bil;)m zd`ahEQA@{Gt>;2Itk2D82*Q zl?ug^K>u`#?}h#~i{j@YzXlY~hjwU6@gm5t6UE7zBpwtef;c_he7$Dp!g|hKbI)J3G`f}_M+zXP9AiQhYzu%MXgD zL4E}wT_nH1p}!BOIC-uxp5nWpJ(F)^5@#fL(@&!D&{=rN-BVQ5!Y6b}LJNbz{+cYYLi z0r^OZ4~FrRoEMOMk^Z%lk~aqV!xWzl^>vcsZ6IGn@%`8ka@Q%&p)6Jt#mB?(;t9oD z!QVykXOJGUMkC2r2-qD&w#XnBO$5UuD6R-xmg1jb{8XVhS<79A;^lDsnn!V7Xje-p z-VEjDN^wibS0Kf2LHk)l@e*j~$rMij{rf2H0qyoU#bcnKo}+jP=&7Li1Q?&mbr=rD zNwSZLw!=QX7P@G)vs8gI=?-)?r0_-lL zcsG=TJ;m=qx_v1w2jv_|@$pc<2^7zTa@$Sukx;)G6wiinCZFPD{I8;T0QetLycpW& z6N-O>_WX+C_aU96{7Ajbg!=tN$v=ns5`^*~@~M!1F^a!|@|2{wB&0`%;=e$@JjH{6 zPo}so_|+&LO8PIw=YpO^6i)!Z4aJ{AdtO2Dh0uQdDNf!ujih)X_~R*l0{lBE9stL+ z!xUc)?d>GRn_#>p=YynPu7IBFl>B$7mnMqufcE)>;%lHibWwaZw4a|8Hw8Wj`Y}mQ z1Lz+?@d}War8sn7mI}qCz#h4tBl@2~ejO?Knb1DnDc%P8@};-}9G`+IZV2saEydS@ z-9r=~1NkCrCzEu#;2q>zDfu11KT+Hp`t<4 z)bA>alQoWZQhWi71IH+C4f!gg_9Pf%lc||ce)V>T7j>WJZ^1MJ8>IZYIhrExk2AtI6 zI~d<)G2~$=N3qa-0ERrSm&cGEM~dTl4cNIcIIdsP-&Zm?*29Mri(ClB$@9@w4371X z{usyLSkENTlR$CuKJI1)$9gW}f5`0zj+2e^OZvqbN}jayVg|={ZNctUirWCMWN>Vf zwDVgGj`etgd?UrlbI<$0Njgb;dq;6Oqzk*jqMR*}U&`^aWEdQq>;OKM!Erfk2X4sV zSiTLoHG^aMY~WrDj^)1qU&Y{9z7+Tt2FLPvz7lo^863-#>xBXa$MQlTU(VoIp44MA zgJXHpFP<>bf0mb)$eq#p5ddT_o5{fsVFzlQe9P7UXdfX{~4!944WBvM&=`aSzddPG1 zNQ&PCJ!=^p>q&=kK8eAx9`byB3&mf6o*fL1^^kGkAcJE)gFycgiVp;y3!K#BBoqg` zLWVpZ3h_H?*i|w()-MBkswqAO_#KL?0l!P}DZn29C+X2*TV=66Q`{K1usCOj$U6g< zV{lw#*MTcDIIco};6@Z*4}3X;V?B3(`!G1xvlaMyithoQM)4cKvw@TT^&ZZL3mEcn z$-$1%z>vq~zXZzd9>pzyKV)#+PRRLT2gN_bdEI9Q$NE=;p6?V70Zv}vBk7NTelZwc z?86-EA@A7>QG6fh5d)6v5$AV2`e8Sg;^e(S4F<>cQpT2MvF0*3)wOx82#oqw`2%MCsJ1$J* zB;dt#(r!hdex)c*p8JjkPV|Iw)Ufm@o&wyS;%UHxDQ*vZ3xnfv1Hbc)-2n#2?SY*C zLjK863-#>(KQKj^)>b{0;`k@=rkjNe0LA$sm7)!Ld9S4n*!ggJZOd zOJN zDZUuEkQ7HDDF44=?Z97A zoSYYNjs82oCqQ11;^ccdQozZ1#dNqHUP$pSevaOy6esVUx>9^5jN1VeC(m=EC{C{9 zHd36NC+wg&d0vxAaUC-LP<#sH>jK5eeeYt5i$H%Vr8v2tTuJfGklz~=KLzLKH~ZlY z6este?@@dj95)|QoIEdhO!2#Le*T=|FucLnO4t#3jt||BfgQ|AT#QJ4V$N*0x;PLy%t={dzd8Pg z9Olgaj<0>N!yK3AT7i_)P@wt-y_1# zl(Xs8PrjEy@*r}$6c#T4g<@wW;%-S5bGG3LyENABlg&g^%BxRWD?IkVrL zWN=&-#L2)8P9BJ`i4TJ_?WQw0({2NUGwreyg2J$Vrd^g&P=s%=qZ8T&2XeL^WG~n? zJeZ~Ag1(x>QgR9kVkrgsV-J>8Q9|FXVtUbsumW9u9F=e!rM^GNMTPhMVJLZH!XS+O9W8rLKR17rJ^W2Adq*dyK-bW}g3(8!e9-dTINLa5 zAcw5FyZ+4r{oMa6=;P_fUAeSTOj)28C3B9;RatO(rp>5)W!e$a9Eb<#v%`9&~18ybbI&y-l5}9 zK*6|u{_n=0jN)RQ_LM>5CEJ-uuCMs%AVAJ{KwWPiCh_s{1n)EZB$m~M z@r7JhVul?#@8I}2$0^*;bqRoNsv7KoVp-(HyQD9}?J)N9m%i+qLp(ALgSP+J_d)@E z+SlU5=z@sYKDJHU$K!ur_PfCT4Tw)RbpG*ir7!#OGdMAr_VIW{+dqj|U-nC2FdTye z8IHD}jl%k}AC4Oaa%&+z&Og4FP20!$!y-`KXpiICKyWGnu*J5Rt`d3rvj0V!BPGMI zkMF0`_U|EvMWDDjk$KK8{P<|9}`dvr z1}0*k#1};I`$~U(5JzGw#K-B!Nh0w{`W5=IFAVqZ$n_$zN8%Gd{!T+*_8s6vaVrR7 z`#4J^KC!Rek9~8n?@kQCmc%D`dO!BN!2U51#P&y`ABj)w{P?WZr>*k1iaj#V;mP!4(V zIRYIY50g|AV@K3fVQ+fxZ*hG8Eo*%3*H`>_NPj++f6Dv!$6J;Bl~tx|OjXxXQ&Uxu zkx`wdqM@|ljg`8%dxufw<&eeO{w)4 z5SztXi+A54=~=9F^r~N&agvX$gK_d|E?DJ0;k9u>t4Tt+aY7{!t%{J9^Ald^Rq{** z3&>}jFh`aj1dt(qbYG9hV{0u+#N z-#A$v?_I%7SfzxcOQqyZp^)?(oW&Qt_DW!kJQA-8Q2^GT@CZzhq!=ez$qKWJm023N zaBa}H-{vR0>rEmyC}EONMG|Kvt6-M!1=JZQTvaJCPMW5O3g{jq2T`cZ5-LqdDgRxh z^Amn^l+920Otj)m#a5t#7E?w!3wB#OLrEyk5G6(EtUlNJlHhi2H!t~as_sz%$)Y<2O_!|3leDzD!In}_{n|DGdWJpYXKhzvU+vzo4{RVtUJImJ1K8l779 z;?$*e5lj8YwcN=vH{brK;k$?4_RDQ+CHBf*n2;)7HrM1`kWoebAfz>?%vE;`J%%YEQk9;rK3|@vIh-3X?`zF@t$n$@gof#8>=VhZ@ZyfFE#tFlD@sg(%_fT zQyd1%j+iv(5^whBu-se&??p-rG%km(^x(bSdA6)D`HjZ(t2I>$`&QN220oRR3LLpi zNV7@by3}az+0vW?iwA`(#@Prj3txTlt+I09(m^g|+h(T}O_uETml)Ujas1)VNi|!0 zsv;)5C~M;4+PKv&EhXZJ=B&tK>GEM8wu}n>JwEncmRVq(OtWk3)3WZN+iF&*NSW~d z8g{37clz1fiANg8*td*R5v=Go*S~4;xlie;k!*7_aL4S9Q%#JKp0zi}!rredfWp(Ve1^Vb?R8FQ~Zg_^v7PSU1`+ zXT;FXg2xrJKIDI>91%St$70uy*23#Xxwhk_cDD?iFyyzVV!{Wt4bn4nTW(#-t@*XY z<6z#Y@$NOA?a_OKe%&>BTKVkUQ;C_QTedb%oc`|RnInbSh2C%4XQqC=rMaV^v9g%U z@xAiMPxogp_vuN=pP{Z)sBu?2KyO7~H11^VA4QQ?-}6a{{i#svCyyo^;Z!`i@YP*|S%zsz=sI z*OXKTewbc2_{vSskyg6(;>tr5#X`FEh5P~%{Wm9`tADYvP5xqC%jc<=wY0K-td`wb z8**_*?xb~L!-_^5Z3@3N@x5y5pzfWupedsJEG_+sP6BUY2Pc^PIWz7}`8H=MV5jl}xbO=!d-;}&_BnE1of>?~Rql zDG{JuBi^?8@xae}$9&nG(W*G_=GkJ`5^piID)GqGx6h5U+*K(tal*GfbJw==_K05M zI%Ub_ZGY;L!Cu*~T{3%es^&bux9sLy+ZpK{zTfy$*H$N%OcJqLHqAo$!^~%f1q~l6 zPfo5nlo2pwNlHM&w^xaYL$!}CHPQWk+%td8>VY{ABrHzkau0juA>tC8vY@JZxmwmd z4|^pM)eZ|OkJ$4XBh?2dy?(3b(E4bzAv*-yy-%W6yjmUR6Eb_4n_2@vnn^Y-=C*q;&ASX<0c@yl09N zKwnSnL38AArLnU7R$lHt;wnoV;( zogWSNT&!w1(!PAhwTFhWN}VxtQ){QTT-2L5ZsW?G!7~op3>X-*C35%Vr$%Q-U%p?L z^yKS0V~doO6;c|G8O@=k!frf6RR(MvuRU+nnrr^wr|wa8vetX9E}m)r!fDO2J7Q|( zwLydTx}^pMJmwA3HddbagLkI(S&3n%-}uE@o_#7d!BM+5{2QyPDJ$Rz&*!VZtL4(_ zcJF!lMfCgD$i3fE_qL4FQLTF8ps>T1H6Uk)Pr`)rSKhp<%gk(8+A%*M{le$K8955!dxf0q6mk`SYYV+&!+{(R z<3Fi4IF;|M*YLC)2jpDa92{78%u7n!%2g9qYK%W~jZ1!)^!)tPZE>1Q&M#0sv)tLo zQD5|#yGoG5%1?UY&%_;u-Ew!>DDc(^-MqDE7a*s~#15HDO#J9P!(!G#F-PI(7yBbO z6eqn~-C;U^h@NP20&j7+kx`IS?K=743iDi@^up!_3dB7eqjLJv!E`y^$0O?REYi(V z`N1{H-A!xzyx^p-;m_YhxQA{t&^0dIEj(b@Pm|8Q<9ei)?CLtGd^^x5?1jqvlDrCe z$#bcfQb+s{nzUA5t!~wOLzzQIw+N=<+AM5_!)ic52p{MQi1{2RRYd*P>aPCc(&Y^kNgm;b370g%47g4hN&Odd4 zynoCL`LR)Np8ebt8kqdO@YM5IskE1si3%Mz)rZeEbf2mAX7Wp&OoIiIom%xOiCfo9 zh}s&-CnCAAOnm2=yitjdro_(YNjUa#{^wm`mIbM2CPl109UU0g$h&mgp8d9_c8~Af z*Yi*nuX3dh6`tFIIlk$W#&6(Y_0@IY)-wou2D1#gxdGrRpec}#3@ zl;x!*F1KgI=ggVDMZk2k@02VJKl{r+J671QxOk_h?A7DbZ*2D0Ms2=+^w`RF$%iAk zyKcnQf3&>2X3gtwrJsDZ-q|uF@n^-=V?&+=#5C33_Saf@X^zM6vp*#)ukcp75n?_{fkMGYneyifw2-ADf5i#20E2qYbsj% z#IQtZR_@8&j|2U@AFOIGIxFhF_0my^LY9c|k(*7v%~#8}eQFZvM$dI zSr}PT{C!#8*U(>X5np~)Ep70aK7G{{t*<#^%ZDxUerTj#I7MyevDA>EmEYql2DbPQ zRe9ww@#m18?$#;sSr>2kTW<<+e)Pe&=4qi^wEtzUdhMP!|7%MbvhN?spnk=Re*Y(H z8O}%X(OnU?V;>X#tR>jjT85^Ar?>}P>~k$cW9qIqD1{BK&;Ofi0b(=%>udh?on-Fb zH-RAWu;FIE74kR3K5mxie<7QT{fFawXML@KH=NC3uw?IZ{&8*z^Yr!)5+uXk;rZ*i zJ4X1Pp9tg02szGAM;t%Ppq&{4eeuQpaQl9^M?V}Vyf6LH{m5_Xhwto%r}e`#`r-Ng z@Js#htNrl$e)!{l_~(8&Z$ITj-6VC!>u9kT*5WM4#a>qHZyEL~SbeXdWpD4pSx728 z99Dn|baiyWcP{%`REjK&#aVPK97Sd?PlYyk@vR{Lz+iiHBNQ)x#aL(S@71ax3_a?1 z!mk4mjd-aoe1r8b@Mobdq8dH!cg5>%QLAck)`3Dr$^KS9el|wFbsvl&a`+ZF?u~?R zzz{jiac?Ai4~EELj&FYvo`oTDnBzW4_;U=A!-psQOqB2*43Wd~_*oO-xL&cN^@sw; zdhmO3M1CYg9^dCDTn4xzV)$7S;c5(dNd_nHz+(OQ*(s4H@0#KC;4y{p9SlAAI6(L@ z2FLxE@Usj)n!zi8W4l5O-olV)+U;O)d=5bLe`9cF`IEWqba@WJ29cvVdB-1fY4jud z0~zu%3?9kg%<@lU@UaZ}R0e0ZpG*d4rvEI1k7MW|S7rDi1oL?|d1n%H{3G8s!EGKp z%$P?nYzsTW$(W1V26mWZTLIXEoIGd4<^r&fH&YlK$5{;Zp+#~0kN`Vf_J59kET0S9 zh~lx3{{<9J0C{VQ9|6BJ#ic+VA8)Y3Hu2Ai8~gsuVIurI@U`sFKl0?AV|={9j>wa5 z5u|c(FV_YCQHsk!dQMUN3gowl;-no^Q=EMBp^4&Mkp6axi-6sC6n6ywKnN%KBHz9j zrg#eIC*?}y8$iz#O5Oyx7RAp2H>5b3lS00ULiCgNw1Se40ly!`cY;5J;$OfnX-7nV z3FM2!BYZ8$lW)KfPS#*JN$KAR@MaEkqzZK-k_(?dK zOOQ_K!S7UHN4}{*)Zi@_xVv3mv5Gl}9mA^jS_adg;a#~^d8iJp~U z&xWA~zav48E6XU3&w;RWU~sI9oL6}!=zaqtXfvW*0^|A%@%%|kX zf;@AL0G!{^z^y4g`$3N{#dCnKqd2KwGFP6YClTZ?Q}QHVcPUPei{u-+xcqT_k^56* z4n3)tS&;r7&_g&G$9_=U3FOI19+4-4fs!uF~9AP^*e)}?F@O$?Sb#3cm(i+6b}P_6gWvwDx`;e ztDo?3AkX}kKDN6Fcn76F9{5K}KhZM?M@Ej6^Kp z@pcu1g9^4i@@;uymyBaa81lFrHbbOS6i)+wjpEtBA5)yn`Faf;w_Chx$4(84#r#%0 zF0-jnzdsrBn3FXx2E#=xNxv`Hok;Oe;Cd8a1AHOH@qIh&JQ*C4!sZbSj?0JKZ`}kO z=ZoDZLH}Nglk=_&iXR0dWbQc8a}xL^O1_qz8WyXH;+*@10X%^6esCx&d8`kCK&$09`kElQr~e?W1f zzl-9@P+z|(9t7zh1UHp%F#eI_%W#U5>ua*s8IdP?rc&}mk2b~Iz%E&vg6MICax~^0RA9 z80wA4gSU6BTf#~E!S|@KBb?MPd5xX$`FIC8W6q{mp70eEH-~(#q&Rqczwv|n9qv-| zAwV9sgvWzlk>dBkuSD@FkWP|*q9+o#1|?quT$kd%fg4g>2=Zl0acSTSDJ~D(isDMZ zZ7Hq+oRlX?|199nlsuUW??G|$Z6aTaF9tn<6er_e7{%Q|K8oVxc}5Jy$@7d26i)y> zNfak*h?4tuB)@w>elN)5>cHazaWb%j#hCp6%G_$slx7cCXYZg8U-t7(=KSRUe@-*1 z12P^#wXk%_4z-1D)l0{sJ*cz=dw=iH@h6~Qe)Rj_jXxR1#X8A&fp*zW7q<8r z8`BZ{WQ*^^^%b9-H_c{7M*BFe*pc{nyukZ(`K2R<9XYS(`2PD&10IKP``0A`WD7S? z$@4q>JPz9+_9Z~h2W{DDApyW0pg=4B>Rf-~gBZ&l?et~89t@a^awM^RY=W2}_QMhD z%f12#`jCR+sOgOlUKSot`ih?l*Ae7C2$3iDi64(seWhOpHw@&+^$JcuK2DMNB>m}# zQAtDuT{mK3yya|q|I@PjP)J|)x59yq#K-oBq91L)0I|OE?*R54Q8?3)_=KHAJAK(N z0sCZJ#`cBKkHjbWzm8a6_OF5cHY&p;K8nV6<>-H3_RW_e zenp(@Gex<`1tk;qA&c)TPUKA9r(l>_>?8QVjn`&ylbJhNHafHq}@B zW03wyR0PV)c@-+aHPFr8$KU;bFsBsFF`YV92D|_HoYJ*Ue=?xU-ql6#mW*MImkIXt zbw#f#`3A^@`3HKt1_s$X`}_HErjrul>E{{T8|>+V=9rQYr;wnCUXP;_BgD%ey*uU7 z8y4v5>mT};-^bN4hyk!#7kmGZKZcypr(C=l5;O~cK z)v|55IieT9h@i8hpR=paAHQq3hhs<(Qe^Ll)cFQr3#fGMgIxVk6ugHf;JSK-B4q^I zyZih99mXK2T7pA@?0tjWL6N6luxp^7qYo#3fWHrV4-VxUD{~A8@bPqZ4EFT*1MN5i zA%6s1!(E+2g26O+7`6z3JYY3W5y7rOIPt--3#oUJadPx?3G;Lb_KC_~@nVOJu^;qUZt{KR%IV9WbDhJz&D0~x%_3H9_Np=ghUqFr{Vr(Z~rtG%-$ zDq(!BfgNdb(05Sr1<6dABIE7nALfVVV!KS1At8S5*w2Z@V&SRQ7x0K=oOHz;jaTe@ z@+zM=(^@a;p<%^1DG3KGW8eKCQOZY(M*OnQ^Jq?jIPAD~=v{MEJdB%C5|GN4@EaOyTtx}}fr|3PS$wmL4jJ`{@eA0*L#kW~Lc zGW`cB^dCfux{nl2?myPl{)49V6NKv4N>&pB|5NhX{lr3Xbo&pQ(|?e0|3UNn4>IpR z$f}6*R zefrPPj(>)B^&g6h^vl14@p*JVdBU;z{~gT!!bIO=7>+IT@7Va$FaNNu`R`!ee+L`? zJ9x>zgB|`I?D6m5@cx5UKGB_meawPlU>u2PvGwf3C;IXT5Bqo;G6a2f1Rr7XS4YY? zhc#5wf3C~c;foK3KRhk@Ze)N}<7mFF$ozdB#gQ5=`6kyy`8yRBkFb!fElL@9gQF`i)=TbI0-ejRs!6F%~T{1(_C0m-@`JetOHPES`7L z@X3o-MYnePhrJrI$^G}eNS)8ooyG2*KcA(g@a4=>ZtdZ|89ng2P3fg;RdbxrB$(|w zYcKQms?GEJ^2+ua28X|fDKC9qk+I*#M%89BkET<&Xv)SLQ>PwUB{$%L-L@moCaJxB zIbHFETT5$hU7>W&_E2Bpl<&JQr#2s)J*80Vj*@~|q1*k_!OvPkE#=?-{Ox%&KWv*@ zz>KE#%Le>Bw0we$4$t;-BVFE=g2yV92Wd-ZABj>ld$wq()%v~q&n%OzxEkkF9CK6r z!1u1;=YF%1R|@WC$0>fe^;^zLrErsC_uzvCEsL`sESyrb1kZ;HuxuW0QUaTzVL4;hCKzInVcS<)VjH#nSe!=awDs?r;}Uzbdr( z-uJifH>~sKYm>9qE>`=#{lHajbDxsVySod9jWAtxTJVyxyuaqek1{`{71UkAsxMX6 zt#J?C<#J=2q5HaHJ{#o993z}NLPW!quG|>B@yY1$1m2zz5s&$UXLk*}c4C9elATf7 zqjCi$(yn+M7}3Pyx+y7EZF{}ER4Z?Vri6n)oWY=@t%c>+JZdyv$1hU35XgP$N3O5n zZl!~hoJMXuGXAzf@R6o1HoV^NKVB&J9&%A{i0TE?%|aUJ-`x!&j?>!>xI)M43~`v= zUKTS=MP!7B%-lW8G7=MR>5gB-H*MJN_5rowirp$DK4ZS})ax(jnmqQl?j#jKkMXZ# zLY=hh^_B|+iwxBr(LK>YRAtypd3mm73D#R6%?ry?$&p3r=Lz|m|nwxe-P1&f? z`a|}Mf4aCs^wO=P?$z8}t8iM}!8g@ccik7Q5`7ir0U71WT#J*xiklmJNVB=_#y54{ zJ%LjPYLwbk507wIxu~tOFZ4CPfhqTtK^K@8dj!ABv(`V*YN@4PYXHd?X`&+ zS?WS zM-b5oSXNnSjV1EyAb-})oCQI{XK!xw{1!7kUWjW~wie%y*w}+F3|7xjv#dJKAF^Gs zYvmKQ)FS801tAxNSzN;gnCyQNGiV%FgGik2Xcd;Rs7ZQ^!oW=7iRs%Exc0`Yrtgv9 zuaU2g8Gd-c5`8PKyiYX;%b18Uqaeo zRhuQ38@dOrdUtMI==@tj2bV@(4N||L^Ilhe&-tj&6`dlkI*X*6C0-w^PceNU)*u@q zWBfqsseD5D6g@u~;~M@zZapWrt@~6&?D+$|50>Y}%wJe2aZIFE(_-IIze$mgbZTcb zzBM#>Yj-ExHp6+l>Pidyr>$qa1W(*Pc2U&JqBd@?bx(a&bM~%w`+MIDkG(rF#=rB_ zoR1kIT20v(vz82(^UdZpzvMNx;n33ct@X#34hcE_E>7e4F1?4@&*~R9t~TG|#oLhM zH6`W1TkD5goWENO>29YqPtAxJFZCT(vo}bjSHn!o!w_y#3?r)kI zoPF_7NVc?rUG@k~zZ}gQp*fnhm(%Z;u3TPmq0p#s^!<$Xj{7<7AHK~Le{r{1alxsR z*(=(QootW#z9CWc-Pp#HT3tEOWij?2Rr#XDUp!AyTu_%~mA3!pG0j<@b3UD_vk-HC zb1d;rNP4^2e)FFR-h$%p9TwtQ9fKO}&v%&rOnxKS=)FAr*oqg`1|e(b**5Mok9{p; zG_KK6X_A%E`4i?pNBifSgost`wf5J`w_3ftBYX48!kks3uFGXVsaI{BS>__*Vf}M( z)jOM&X7!mf5?-CR`>3>ibhc%^bM|N3yV-Y^uRT^5usV0f4Y~O2gN6HU8)nXz@ky>e zd;6^TOa*^$r_ffb7R6nQHGc}W40(1V>G~R#XJ^6=C>KW zc>T9^nwL+mk>YyC%(XF(Ipuiqewl>IaK&YD^18yyZj4CpEgGAj)s!n8uu|{Q4*jr4 z%f~V-ys|9%ney7=c~aWOjPv(om|r|?Zku7wnqhvzJf%&{IREjQ+~hU6?W(yU%DF%L zk|d=9^`5TD^$XNr_bzEF)57b9MgP`3%SrW=HrKp1yS%og#`%wo^VQbmHmT+wPOivV zKajMgH)-m+h1V&I{%i}$sv5D?_m{7Jx%^tq^0CEvZN_S6=M8yl?yH0rA(aChIfSsN5*t z?DP07qx8vVtMzXDRK0m&YjtjuL$%wg1veVyMJ#`vOIWSnRhhYdLyX zK5%hD;c=t5+M<$gUUC(y>J+Z*x%G?T#ofEGQy6hdL^U^f(-wt8Gd3j{nb)XuA9^jy z!Op>P&@6%N=(+DV@63MBW$JiSg!PkvV7276dV^i-D#Y2NE86y~d--}G_Td{b2lh9? z{|p@OD)2a&CeHr#7c+}OY`4YUg-(>sp;t!J4S4R4&Kb@2Ni{R-^^)Ry7V_$f*d60N zHUk_NmM3+sa9~=zo44H~WcJA+bM=M)c#8$b&DlM>V2(5xZ|*$($#k_TECZVb76{Y-kg(Gw?-_t**RO^ z#9-^($fdGu_6b}b_PVhtwp?kSyI*(k^ItNN%pGUjajRy3)a%tsEw%fZ*>jTF_u>ymB^}f2tzCXw<;w4kUKOhQuJYp;Yl{CD(`vi+ zrcE-d+OKKO>q__)x8%f#*Kh$d^Dov2wDF>kf&FlgfnWy@4Gm3Qu z-@0(PtK8ekpzBn?P3c;h&0?AD#&kWEfzt4V?Iu;JiD0L#nMscPvaL#abyc;pI z;Msy=M#{3!3@o7gI19X44jvCsZSC*5Z=3+2^~!-WC`^oY~IYX<^3_myIGo@=L4QiJelpo9DjX=YFp@KYT{;XT1;EJP&wVHm{Z#Bjnb}w88UUOFn3LH(dAfZE$O-kT;QtV%F9Y zt)^tVy&q`pA#U3hC0ZJ7UTPl^*f-SrB*aJH<9nI=%e4FljkV^DYv_na9^G>N+{;Oq z+no}veOs8n_^&+2W;Q7yx%xtV?YS4TT~e!_E*m@)u4L9-6166B=SfZt&aU=O9ji;1 z$3<3V-PUzEH|}4++dR5OaEq_l@cxzWy^e>zx?20ZQ?#JzqwBy1j_$apZ_TBLhC8fd z9(nzcSkm5MaPxckjnJ@Xg+FcTU)vXjJNQ|}-SB*ty1q>&XvVAs9)ST(^^srpd}JC9 z`81=?f76q*8trmr&DLAS`?u;8`E2XG<6B@Kv{gEDaM2aj3zrkG&VDLkK9nw~X*6Tr zMMZ7y_@ggWtb2l5Uzt}8c-1!T_0T`{wk&LKW!cnNN6f0K@1yVYs>H}4wd;^VfxR;A@2;!u0;)#h96 zHak^<=KS*1u{Ap+9r@vLgty7X6=!_kUf^Hup8uNTb>>3dP5r^cY${n|d9Q6(&yZeU zBpN|+ld7^-DQZ68HTul@#Y56(h zLs4qstXi^rc{0}&2-ORnWzq?h=3V|(@1Kp{(F~psPFrinScV*w^Hc6R8g%d0*Pc=3 zRek%8-?B6@4Aw7+(=__-o-}?xIb`z>!PfKVn`$p4U7{?v%$x|dU;nb`^8As$@EzCo z+N}3`*DAml{&2SB&w$ds$rA(ju5CMT^|JKVUFupL7gO%}Pc(Qpukj}Os)jjpYq=fy zU{7Hc{4RS!^4)5i+!I#@4WmT%TaqZQR-^_biuAL#J~ES;asH<4uRYa7r(9_$g=(+xfAyI=d$#Z? zL=8TOWVky}9^^vTx-Eg#~R3p4jLzU0`%+@?QANI)|eKi(?B4pS!f38ML1DDP^YrlzqoxA+;@`P=6sI|7!Ow*C zx6j8_<;7*SK3K)5zH6zx2-6R@NB=DLxc}>Br`^k~MgI4!o=rLScOG3Zuaj~W|wty z&EBv71PPreZT;QGr(qW4d!KI65zy;qB=uSG(7M7cpm^XY86F3G-tifN*ry3)}n;-{uQE06R{O}Z@= zog8SpyYKrkt;gc?0yQFMiHqgD_f9YWb;NgjN0oTbx}6EPTy}IGv$(%k)VjP?HtV`i z{%4{%#T4kWsUIC# znN}`0HF>jZ^P;g6_u4C47p7>dErTsWi_JtfnO&*FAutMwb~dD@~9*~ zo71+ty6_?Ac;lpis&nzI#KD??^wyg*S=!o){xv_4_Q?oWy)zRdrq9>+R_so;;Chm<3&}~XUVTh>%Zhn{WENMV)DZ+bZ8bfK+kF~<>|gts?S>@3%t zl%#z~WFI>twDgDS`)7S6wY&^Jnc{C8e@Q)$L+2~xKF?r5XvQV820voVss=8xEjO-r zI&~H?hfBmNsxj7ariX={R>)&yl5-R{KcKhZo#C8s?7Moy0#18ovt6m~c$`{4u;5tT z@?Cuk@|ZW*?HsT%daKhT$>PMMq5VGom&HPf1ij`0rE29_os*0=CcO8pyYakyY;MD) z;F1Wbn&t1ks@0pnI)9EjB|f_;^|hTz%1~S7_j{kdPN-WXpYwcn>tjc8`ec4#&*m$+ zsTDiK<{!}?kvmhU%dj)Pux2z=Gv9f(v*y;r3^#c=%{Q3dDm+kmQU8X^o~^-GFT`B@ zdS=$-3-j+jL;V76y8q2^RiIyr5b*>3qse1cq4Hy(FjNB#>?%ebr-cYo7!CR|Skw0&Fe#)8OQeWc1`Dp8Qgj;Sb>354Olu&2|L4Z6`hC*>ueARAF{?QK6(0qde)lX#gf{)} z$E=$7ZwD~M^!rWlKa0VGX|6?{W~L3RtQAmyi}rsSoIh zUR7ZGl1-8tVEt>2X6BnxM}EL|88_R=)*moK8WBv zq+C+#AHm}2=ugw(;JS7?|K)V}eLB384)3GGKhxnq=y0sF6?JSX@~~8Bs44|8O4Vxc zr}*zE4m$%pgWMdp5ra^<5x2oM8k%g-Fmy1{(Xr6Bbg|5-7SFl`jux81BKxRz(G5Tcs4N)3{N+YhN=F! zhF5pU=xyof-RbB<=;-&;(Sz@o0K@a2K}TOmM}LEk{sA3*3mtto9sNf-`X6-k z=sUdl_TmR%n)O`@Ij%?VY~kgjPDgJ{$3Gei3it0xM~}WsiR%y2(Z@lK>$B+S(YtTB z9=)4@ukSrN{-4tE|AvnK6CM3GI(qcoVLU&4u!Ap%29D9FeGrEn*UQtAsu}u9eocS{RAC78#u`(3@`urkmKbqMn}Jzj$Vt7 z-ja^qg^nJL1%&7403Ce-9sg(O=u7D6%Sd`2qMT6uY@?&^Ch5t?yAe8ibP5W}pR7k? z_~7d+2rifi!}ZdTMq{SlGuf4zBlsrP3z9{gi-=Oq=}X$Ei$Nw zDQ0+{jP&c_eliE+?yy1$Vq6ziO!V#)@`J96lrg<8EPox0&jJw%v&484yN>aESU$BFSBK?;u4_>FD1)7hus4_< zjrj|{!UqiLWx&CKFf?`=;yiG@X2N_SZUW<@>l(yY!}^TIDnMKZ)(14^2I5ALZ@~Op zLT-w2TgdG(?hLsP#ycQKV|1W&x508lV+kN$4fpF4n4dU!K8nVOLVAB#KKYn_2Tb=m z#!tiTK8EqzaC?!*0tGI@;r`A9(?{ukhI%x16ykGX`TJq|J1{@!x&Y~4!+b?eXuU*WIdEZo z87zlI7_Wf!VL8SJVSPYjkRtzEX~|MFG5t>H-xT9(pq@OYD3C6P`QM4@H^KeuAjXs7 zb~%A@G=^g)#_eHwUc|T#EPr&pfYvJlwlnBmeZKMA-KM;F`fkZAjTI!K8f)i zu$*VWDN45oZbx2>e~0^>5XR@j{c9!0li>DM#rS5JkBu1bhV{%6;{`DPP8c79_0u2Y z!*Kr!!}tprj={JC%zqBXTVT4EG0qJ0S%z_Rep!R@bujz{<07zLy~MZ*tS9{#SB2$2 zf$>0Cf6(;^+Ky;U%sFuV5YK_-BZ%=LxZPJ^yaA@CjPW#>A8m~5gMgRAeh}vWD8|tkM=2Ob?O_haQTthfanw%U z#<(lYPXos1z;f=yxHDX@_ZT;Uh(V%!R@?+1*l!0q@2`lf*K=PrtY&YyN&H(o#AB@|;@+6O~3W_olu2&SM ze+~7?7{3kID+lAk(9ac&7sGm0jq&BsKe}%L7K8i*!g@{~Lly9ou>SO8e$Y1tzGA!r z>Sw_Eg8Y}l{LjVsM!4UJVB83<7aF??`6+<)YCWdk0l6W@72t7U3&yh`--hu+aD75B z{t32+2QfYg!wDGIg7rHa<7RL_y^L|Ry>DTh8j^i;Tj72oi19se`$}OPJ-<}N_(!PM#W*@&w8D5lte>tJFM;cYuJ2L) zKfrQ}!1O0zxy539KlGo0aWrOn0me7ObjvUv1k=5b@ue`_hVfF^E_Gu(0@j~T7{3YY z&rggKov*0I0hgn45QW=U0OQuMo=aey9qt#37>DjDS{Ubm?VdTt_tCOLal|+yEPp?Y z?}h2^#rO!!=TVHGhxI2F<6mLDJ&$oThTm0;Z-e?8jC;cTKfyS9-t!9MXvCaBjPHZ{ z%M`|+!uE;{t~c6VXJC2G$9N{(Z$&X)3Xex}7*B!aqmJ<*s5i#A8>}aG7=Hl$cwu}6 z%uguB-@%G<7~=SJ6V`nSQjINVw5s>E^z(hG5!s1-%N~OhugOhL#X+A$su_3to_ z`kjnm{5s4(>P(8Z*J{|_a>DgS{5uRU#<&a2zc|LTA(zFtJuH81jK7841motAyJ36_ z3`bymJ={;TFbg_Ag0A4TFo&k5^i4vCZXmq{GxmqUF$iIeqhBo6di zQ2&X<$@(c02YPgVJ0I?6AUatuMB+fd8~R^M;$*!Ji39yfsCOoDvfhWpfxZ~(qez^r zPatuiN8jwbOyXpH8Hoe^2k5_z#L4B5|@F zjqQrc=Qh+c!&gWEC+j&$9Qc0*^^zn`)+>-W(2qjBDT$NyTSy$}(fz9s5+~~;NF3-_ zpz9zK2YS$k0y|IQK)(^{Ye<}|ZzOS`cY*p|5-00NNF3<*K|Lqz7=+3b<%gfdfj$%J z6-b=ye?5r;zblE8^?oD{^w*&N zD2bExNhA*Rolsv);sC+Zb4odh1N}JEx05(P5L4ceIMB}pe}sJ}ak72}KW#bM-{qiQ zgv7~uDG~=#^bJT|5-01;NgU|?pnpFSC+kB=97vBreG-Y2^_e6Nq!*#SoWy}1?O(Md z4y5%^|AxfL`hF4z(mtr40bdZp>l-(T11TFk|Cb_hvVTPq2YPX+Hz#qj-j2k9UI*$! zNt~=dK;l5}2KAXFPS#%_aiBj4^|d5U);Ezj(4T|)eiA3^M@by$YoMMRc8J8wQ-H*Q z9zCyBByqBTbrJ`9)Opj6#L0Sh5(oMv@czjG5-01AkvPz!=d}eGN8d`UB5~jco*q-` zNF4Y<&uhCej@~=`1Ub4+iG}BdV2%^JxVf+V7JPhNRa67`o zFZH-Utq-(hDVlJK+C3Pg_+xwp44)uzayi55OD$(K52WyNlbSABFGTV~&JSwuQTam| z^<6?RkLOba3P3S}Yf3{HUWV}~7?#9%I}A%>TpT1sm>g|FT`%DO_jLruFG0Nq#?iNH zbuj)2>J2cCzO8PGarCXRO&ITiezss7_2YEF_!QK;Vw@4yA5V;<{+;N08kI9I)CXdE z^o^=ej7y^WiE$Ok4`5sa@?(&LY?JFxIf;|&j}YV_I$4kQSHQ{o|Lt!%DN|_QX9{u) z4U(pQya|1w;oolnO9%K9Z}K`x6R+LMO3VIFk>G2;|L@aJa5GZh@dewS+@^qK>fmo0 zgVp=@5@BKvuRCC&I{2H$@c80{2AngI>k^2j4*sSwJiZj60SvVxD2h6q*@Zbq=V!lK_9{cBS;^#jVKvZet86&E`4Qq-@y|V6rlr#;?qLJ z_a2Cu>Ed621-}ak;S`U5jo{PepUs{Y!4bv-`3J{d6d&dPF~O!wUjkl#%ES0*g5sla zJux#~`rBaox=;wx2e+b8e3brMf?-8OjbRJZ7omp!{NdVH#G>iaFNW#Mz^EX7a88P+ z|CM0Vtv|X?FahJE35t)xUx=CM(tiWf=VqiSLHgiU9g2_EAM_&y>Eqv?3We9PsDC=m z%b)nD9JuJjR~Dkh_>LmNDM}y3UxMOeIt5`Gkia;Kho)fAcr;Iz+6j|Gf97#5c#j91 z!op(uEp&wO(e^_NQ2tTal!y;};_>5Qe9*p-4UZ4TL6~m)eR8D5Lg|C;2QEQSe6;eb%xV>Rc{y1jAvpzEMdDji)PGqS8ASyJV!5ocA~7NCKmYa=KWxiE zHq@zkQKtw~geV%;8qvS1KnZJDX+)0`AF6pod=95oH*%1THiEvp2AAIh@Ym$7%jNgf zoy+--ZVa_ye4s7?9JzC~Q^2CT?p!`Kzc%o5%~5A_=Nhgiru@2*wQR&_{Pr3Scd2O- z&u$L=3R?KbyDqH{{x5YQAie7ze^*mOQ&-zzlc|=5r8Z@gv7w%^0fol&jIA^b^|T0R zU~FP-OflEi(ArGVBqo-6#v3VG+J@SHX0){^dVV2JKAx^Z0dCZi=0!D2a~^ttbA0_diCgFW12%#1<#knLoxvB65;} z>);JydE_J`*TKWY%%M82gP;X9JWf4lj{M0GwD9*lV-OddKQzvDXan$boLcyilT1P4 zhS6H0Y|-_`9wB>09}I#ShKUdCr}_u+ zcq4x@2Aw}N9xOn32U#7+jwj|o5(CN5|9|4LZv~P1f205Q^S55M`fju#QE0TBgAcB( z2CGFY5#n1)Y%uqINkklC0mvM%5O_g@!o;^D%#FNx&E|&f{P3vJKt9@6POq_1)aXo% zNUjcxn#_TPJA`!)9`+MGRaPuC)06MO{Bj9xr7@;6rw(lBBY@bD_K+2RWF)! zu-t3k5PWcfw(iKXma~ddWi4fn-QLZ%JRY9kG^56@<@~gFQ+lMV;kKe9%{wQdOv0z{ z)!Ld&*9`aUXJ?;%ST^MQ?MQQ#?Lj^Ln3bhsm%Fa5)Gb*1OE}|tjFO9k(p3MrR@hme z&bZ*+Gv8kBSUY2I|6cy=o6K6P*QU90v#tnboN?)m`l?ort~(x;86GkQ*>|&f&-Jmq zdSLxjY=!5g1KoiqhR%60hc4M9o?Ui6FMXAfk&T+-D@(0v4{lyZ)&gzEOpR0O)@|&M zu07FUdRwz_A~AMW=!F`w(N*aWO@16~;WHduIx~8T(@pD$?%M}*#4-*WJGj1G@oDCs zupg^?PjL>9yDf9}-3_jRUH)WFi_U4J^+s zYnb*dN&b}-zP4iYQ?rcw z;8N~dKPvM@_0i~5&{JQz1?8USZnjOlQo59}c;LnH@2v~OmmbOJy*oR>#Prxm&8Aki zlWkR@9=P$B~)JbsHI#JGMVfWw_xIv&R0@O1`Q4jIUHutp_EmmnbT~sZMD(eY)mz$Fdm* zSa;`{G#nPZWLSNFMbL8Dqf)zjHFGwm+ugI9bB5clFK6x00V4^Q(;4l3Kkska(i=5( zCu}6SJ|q2!p<2tt;TxYz&+vuI#lCr&5h5VZ-P{)crZU+6vK zmmV`<^I*Mu#cq+++uPT-8!ju&I=oNOu;-WTcgnUK3QkzLp|NuQlv{9utifINxetba z4Q;O(oDF1MKH*y!h0 z3G1Q`L`6lLZhl6Wbwy_7qggBX)>kb4B>!D;r_BCB@e7G8!Tl#91dsE| zoKItr_u4i~pgfo5>SzVa8V`5F@r1g}w6#YHdu$k_iwYw=n5RTsHP+_QFdk9=AkR?$_OU?bGF_UqS|QYh(M)*G`1)&-drQ9~@=!&NcFg*v~sF z7X)qO+7Xr`ad}RqInQLa*)bi3m)YfhUXkfjdbT;A_6?o*8s2`90 zY?gqgJ8?e~m7^(E&Y>A!{)|l?Ul%cRZu8eY3&8N?x(^t#{=^IWE8ay{VoY*| zD>m87*l`x6y?m|oy79)1==;Sb?vBS8BC>MRSH1CH5XEHimD}0wvb5a8gOkxmGu4vw z58p}J-n62cUrzUQqTpR&aYfgp@P}^sRg1RYdVEqi#d9!e$UW)pMy~7bLY5H$OC}dM zDd|}-&%RxD?!-sOW|0S7)-Hp(SuQM}27k)#P0M&I9H_8nw@-8W>Frw2>z6(nx!;?3 zGLHYq4FRv548IK1vAXfHcOPdkt##Gx z-IBwQyWcBI>aF31gDy)Soay>8wRnxXM)lDYWx+U)CYAN$m(8jRbQtA}Pac1h!^}Qk zzh$%E{?X%C)NacflwVU+tX`lZdfiTc<-?{;+~wBuIZoUvE$lu%I9ulVRduOhMa?Ep zz7H4Wz03Lqo^QM7$XYLVamPJ{f@R^xFEXE3_!li|U~{^bym*Dv(+*d?(t}3Z(z%b! zp7m9I(Y-33zl|>ug*=X^KfsN{x- zhA5}IDsdB(*Q)=52(Rdd?K{64-*%~q&q&fIx@SD!~-(H>E)ioW#l+~D!~ zLCLZ{iSv9;f6!+Cf{cdF z0{94z;oifeSuu6Yv)}!4@YG=yUVk!Z zzG`pque6mPTU>NH$`&}RAMr+M}6b@8POJE~c4@AKd29B`uYCBul+n;xbY z;jtfg_|NxY+;KZCQ~u#r^J80>*`u~T6s+=&jJ15VxM`qfz-QIY(WBA3onLYseQc z1H04zpRsHei1^@o7#Q_=0x@1~9HTxbe=HcU_As*+r$i*nMot;Vjhqr1)In;`)dKy z?Rlijqtxga9S)wU@{sQE3qx-Obo513{hw=j^o+5Cj-EDdtUK{xYU2CS)cXz6 zf5x?y17p|{&!r4CmB610_>-qH>ae%OV=hC@zXrP{`*Lsza05ftPCwMGuagh)m2O%N zT>OK<_o)N_&X0qun~xjegE|T=^-}5d(0~(VjdW%2cTU-V`0HZ2h_1f1xXxsDfmmjlF$GHr*)jf5BWj$1#oZ=8*y!7 z9vI-Do*<6eIKaXEImA7Pd0>EpJRlwfAYov=goq#F5yU(&pa-o0;-EeO103w5h^GKZ z81AQshN*h6&m;XXFTSmZe|9T`n*+00K3k>+- zBYw!g7m1Vo2hj1qi=-#}k05by{}%Z_N8)7v`E>kWCF#ljQA-N;Tr^M2{%$V;2S$?f^cQpK) zzlZDFk8yBs5ZEW$uDpEQE8YUxslv z$X8+VLdnAT8t4a&t%&>_hkkBidN#=KW4r5h<%c{*9B8-SK>uejJ=!0xVtftMqwRy%3)NRNmNm%645$T@LCg zL*a^v!TkwvFsl!_42c8(ry$oMagZN$-DCwh+9vK$?}~9$ueM|SHq?h>9JMn?F^>BD zqcQG4{$Wv3J)yC$L4H7eL&pX5E&}3rKt#gIp?~BDoVx>if~AY@ak{1IQ2h3QfKv&A_2CW<4* z(YvT_7=I4^dtn^4SN<3ugnD#c27=`6_!8;?CvQh|-2pgxI|@NP@Im%tL*isV+ew`4 z=fCUY?dIbo{ijp+zwblQ?sVZDzyGCyce4KK_D$*q{&&~$e|d13_HNhz>kGKeY zPnUiK6o$h1XoBLS@N6RTbm^l9UMN0DpO^Td_$YmGI_ZCfLU(HBP-=hTBO*d4{ZJ8_ z0-dLV^g)%v(+Ah_APKxrx+AFUgl(Wxe^Iu;ebMRS6TKd&_J%h3{g2W|@r_aZ-wN7H z6g=USz&MJBrq(b%CZiO?_&7)L5cePeIA6wA^LKn1B!p9xe}wnJ^zry_VEXY$NGCqH z4>?`^Q5U5}(1r29_5;tmQ2MC;fc3%C22r+yhgyX3!*&Qf&s{+L@c7``4nOu0h!O>M z7Q)`sCcpnddSnB#Y2(BEM`IDNAkimXh3I|$$NtAY=_)Gnid6Z3-X~p1`%jM5sk7>o zrNji6WCQ)jb)!3Nx89}oF(-Pl zf76Xoqs7pTer@|p3?#v|M4fV%n%18`kRBob0=j7H##EIN-chs~-9yzuT%ZHAZuAqv zH?=P_;SDa2sRUx(=vTU+C$?^M4dRwWh=^pnm56p%$BB3>>?#!GPjp51^$&3q3ibLOOSuSE~4M~pZrks@~?alnV6{?{Q$0mZgki0F5kAgRkVl<6xs?xH$)F< zunSOa1RgU{nCQFQvCn7I`vt6@zPx%KxTk5)(D3!q>nu0t*6KUWH!Y0WrqtG!eg0H- z#|49VV!^8IIzc})jn0W$Xb=@v(J={_& zvgh^igvx~dB7=}$$~`}B4ex2%^;qTcuB?ghwBSV@W|=PGvqN}y{(SJb=)%=CJ4;qx zOkHWR)l%}?oQ|RMo63|5w|va@&dK*W?so-(BsOVJbX;a{nCCrU zHMe0xDXE^T{rQMqW`#syK#IH4q4zey*OjEKueU9hsYqHmJaT(8uPom+w_#m3Rvn7r zlFkyZY)|#I^A+oJyYl&;YHTuRskgS!d30z^zWSyr z&o*tbx%Xcixm|c_c~XC^&v>QfrTX#Iuq7ug@|T?YaoY0q$g9;ow_?)In^zp4%6}$# zt$C33LTgghy+PLe`Cd`!CbKPOU6|u_rez}ShkHLqY+#bA#jZV#W7kz$?>tBdHBcFj zZtk|-&10foJcmpDRf~nPMc0lFfr|0PdA>583ol$}aFzVnp!43^$!(MD`~^GwYLAHy z@0n*Dx;HXE)u}?zKxKjLDeLgc05`UuZI0fuUf(@6mV13qJYe4Tb(fmJ>~r_C7~Vze zWJjm6U*#5m%Y3PJ@J+x%+1lmmYnqvZxC6Hz*zk7j;^n~a?%9bezeGrPD`cPA#;`k@ zMSV#LcdvL8Wk!^CwNUd+jyWC-cWb%!&*}DYlrQ1u7i(g;&g)xk+`X|{+L^sLMVX6# zsdSX>G2V|)$HxpM-fXBgYcmgTKO3`mVYFxSz16a<@_CCoZU=M-&KD9r;GBQnCf!dz z>hSv9tg8EolqXejCqHoi5HZ#8nZYkGB_?Eii$lhE)G9kYl*RE$ul4@?T8V15mLi7I zC?(3_#8)#K#jZ0QTls@?yXt4@YW3EA%uUNeD92QP@V!~R2s~z(k++^%MtO{n zp!-;enW}3)a)>D2M3lHBmb{1b>0f%_8Li_e~IQ=Vs0 z_K7*bGxv(yk?5yPYwND4r?W*Yn!`Gx#kfr*L4DSV8C#+`p6Iyl4>>!tSI8@ZQI@4w z)N8-A2x}1Un0nFVZR6ETO!mfBKFA$-loxq3+4hhx^K*&o^E%5r@2B1w%WXbsA9~m} zh3>3go4%XEQ=2>Gzuq~(JN|C+nVJp1(UjcsWA2d#Lxy#0xxcrq z$c=hqUZlXq+;FR9Ew|K!{-j7*&_AAEt9n#URI&eK^xA)+~kfVN@ z&!w9`KCy3J96TO-Uu&nE@7~y$_O>pYHB$W#zGQH}WHLHBsRskj9?8Dd>vTa+f$dt9px$l2kV|Jbo>6R`Q7F=-9k)mG!%tN$O;e3g=C-{J7i{&Y%>XSrFcJduMa8|DN%qnL0HedDk(G#bh3up>=BeA&q9XtM;Rd6hwFL zZ@nI+7ot@w_3c0}SH^v9!%OGRu6&|6CvhNTPU%3FmSNEv?Z%=dQT9DKtDMV!*wpT= zupV936V_~-F`?}r_=-<D828qg8Qph&!zI(2-tVX; zYqY*KTX8RrxXg2WNB)Yb4DYSYYR#p#L2O%APMvN35Mke(vaQ0XR!k#n+oES>4x_6^ z7jZ8>EI2ppQQ7>(|4eBr+<98kYkOH;EAiONLMd5aMki0-lF|6(H!DuZR~gkUd1kj( zR`A2A&?{1}Yc1R+mt{&fD1AQm;Fp2?rG%>3U7oF6vtlO4uc@-zW|ft+{M>oU^ZDS~ zXhqNRmTTfR0d|Qy70i7c_G|77`F=+Hxsg|v{%Gga%6p}`3s-!+Fl>BevB$Z$dn7%- z`4m6>%DeOE%M0#>5`+FFgHNXv6L)*>&iK&!T+aA&%C%D;#J(7;Hh!w_@oLg-Yl0I4 zmynCKTZ4$wDO1h?uJrQd!h)m*F#-(tcmlu-LZUn?eOYBVSv-TeN z^1h>ADt26G|S;ngtz)O=pfK$G_H9R66ZRmabZKbrH( ztNsyd180WWt%P+~hIBW#z8DFQJ@%ta@JmgN&Go^P$3KsJ_$Z}Rz;!W(rAO^0+aqud z3=AE2bKwag3WsKFI`PuDMt$zV9#bpHjWgX&UT3V4-1pN-_m-)!W7*jo%e2DSmsD%% z?Oh(WcIOS#u8S`lRpci_k|np_63-l2`d~zZawwxZqI6!Zu|(|ECoOztEGv|ftM|V& zbF0j$*E7yPQO97%5V&Z(dacI;rh3_#|LkNdZaY=VrYzZFchY8_i|6tBrM~&cJm-8o zd)Q3ME4?X-pYy#c1Wx~+Znre4NsccRvSzG>X~d)9`sv^|7DoQpW?a89eqD)Ax%HCw`KH?r*=wHox$OuZ>zNmMYO*F~?;*YY z>$a|&>eCt&kGxgpy*JgQZvFPF8@lefgtJgkl*k@qR>i@l| zD1EB^V7k_KrnQ{fs0$JW-P`?B?!j?)W*$)IKD;yFn8*q}Zara5)eE70-P#{Jujwv* z5?R!CCh&;;)g{_pjpN0QgW>Y88zT>z-qQ;5c)=p&Zuv>-J>NVN;U(la4W*NBV3ygTlnS7jf(S7-hK+ z^*sBgBf2G_dgdV!=a+%Z;YstORrhJ9>uxsJ>1Iv}8jV`IouO>DlJ2QxZ;vN$W0xM> z^>$)?TnG0V1=r;wl-ot=pVckhmgZ$_=DbuZs@F&$3q8 z%Nw*=r+0R+{#sLq)cI`;r(;tkJ15`dmN93QB!0HCweoQhwPj}grhB^cxK>L-XiopZ z+vhY_HmaQ;_H7c~kQTB3uAkqhnikFOV8H|VN|C8f+UsiBN4I9ZeQt6`?3eD3qjULn z=FJ>+dzbms*u!ejH+EISPl~`de)iw{GbiZyN7$eBKV-8t@U0?Gt;y#l+xtUS-7Hu; z80p+_UGmENEAL$=*jY}*N2M}{QY7jo28ui=@d+O*EPXFYg;V6$ty?!+d!BvdG2mGi4I%d~Y_4<*0W=D+Lhmjj87hm@b+`H?$Me%55i9o((vy+F* z4deFmr)GKI3WjYX7auEC@%DIsXVUs3(~%vYW|a9@K5?FU=Sbuw z%@Ee7*AuM73PqfZGn8&|`PH)!0@qtnF#U@vlv(>;3n9`xLNaY@mqG zA$+p0!Q@w2*%;PNgT_wGfFXdtpUy5-1Ysp)IH;>y7G*q4+ zHQQ*X?(=j&tees*`FjP=9aeYxqBmT z1CPY%p+&O2rSC>d%rqSf?=ZMYRhg+2MY0KuS(|11`9&RJ~O5 zCfMl0MXPS#%X%N~e|h@y`;?nXib?4a@9=rwFCOQ+Q?On^TK}S{h*0~(3yPP1W|&Wu z4hMuyT-|-Ma?jq)d3(3a6Z!dN&8MR`6Kr1cgi|!Mj&ugDck%v|Si{wBz9eVyPW!Rh zHLvI7mc5XWG@skrouyQ#lKo45z< z+34_i#lFDcFO3&oNhJq1dsi(kF|K>JPHXsr%S}-aC7vytIav1y2%f3!dQhGq@Ul~Q zo6y08&tX=YMdkMGLym6eSmT6d3*BEjx@Exf(Tf{XG6hD5LV|oF*7_bk{pQgId9BzZ z?f)Fp-6rI9h)vLZ#_d+cN8RtjShbb-xZo!l*fI)m#28M*k?I|l{$<9-!6HX?BUMk5O^yl z__1w%<*Y>a4PkLN-r2lD63_p0_OMfHnr`5sd263m7@p(` z)0i92U0x!+tTJ9$luh5lWaZUE_2C%`8_GW__NHDDx-zFPRbDQplbuCu_7<~&O9EXI z;+@>j9{#iZTFDtd`{V}R^+A3`29MpF^%z&Tq$Wuow7T7|w0{9le#hNAjvLIzj<0_C z#OS$2Z}y!Wj|-NT^M7quF?@PY{?Z+fdWVO;7eoFTs1*=&J7$_E^_BUO{P?Da`*-L4 zl$`0u?i0=}*0QF&U1Njm3F`uWRFbc zN4;AAsHa2C_(g5%zJv+(B`?Pg?qPYM!PVEbyHM3HcYB?K+tfy9{^G-azPC0gqz$F@ z+vrBtFt$GSUii!-hokh3P~2{z`K2v6j%Bys&OKnx`6^b}?Xusivz%^|Te7dQd$b0Z zE7ty49@#uvJQ=XH|Fm3t@4o!;3cCi;Zc($Fx`(3A1RpU!WwNfd?eZb#L{+!>pX~HM z+IJlY(bL@_9)I7g`jpctenpN{o5IIEd;K%wT6N|g6b`*M(L2h^lsJ0xEQ`%OwekmT z&dwL}qb7~dyC<_P-?M3--V%q_2WqVk9{0WF6J`>bU$|~jOC#eVm&9C>YwuCAZA@{*>X?06 zlgF#rIg`KMW*K($hxaFQ=!raze&sPYjPEcf<4+yd!-A_fTw-{4W{2;%&C)YvcTUJY z9^D($#K9q=8td-0LwvYm(*5(|*rmdW0#SKtbxd2=mWV%IH8D||x8xpYYG-^E^B0~a z)q58mb@kakN4DK1C_YJfJhfos2bcBYiG^Eq7bFifO|6~$k-zV;u0bU$&n+dThf9VI znn7o-1+&v%iU%i zxyI37-Kg&pB)NH@vn=d(mF{W3{&zM)+s!{LUD&niQTN#ciCuD@TOY^ACOmeQPk&p| za8o?(5le;-i|dgC>6srknR8`CanGC;<(i*8WH>C}v$@Vdq|Wj9p`yBl*)z2tCz^%r zNB+TIQb*I^{llK34b{%q z_&*)rpLU)rrDWpp?%t2@R8u@GPem$L99CAn5kMV<2wGZYJQzj)`%8>lR)=`0hUnd!MoO?G8+P}T~r6&ooToTYZ@ z-PRj3lX>fVFZA{XjTkU~->ag&?6!q`X3`OznrMcer}>lli&v;ho6S_-qkMH?Y>en# zi#fcJy+Zfw(;hFdjMxx8I*{by@!-*hTfbJd@x_FZ&rc5ib^VcR)?i#K!!gKp&E09| zJqF_4J9}S_Xp4J#Pe;Z?Uj|TCn$O8gJ9hu=HjXqS*U)dp!fKfs`Ew7vU7x;Oo#*?j zSTn`)`Alf@l;h^#APjR=FlnSzM5?YyA{iHFwXO`6Cj-R{z zlx_y=?(*DNU%lc1w>f)MM2=ni2i0^Y?r;c=LI9e{Jo?=PjPC-kWzC zRORhx(akSCxlMjst#fCrlXp>2PvKGMe56-vyK*f{uU@<_P|%KD2-j;%&dpGm#UW zSzes{#4_Lyq?rV?gi-s?5ImBR(6Z^wG6GKN7+9bG`|tF@2U=JjnEv$^#h-=r-`yi_ z#7|>P)TfJoFZl1@-2+c){x&A+h(!P9v`55$Kj@?FH{Dtp z)8U}c!E}0if=~DE+fF+Ay;S|5df`Zfft@6N)A`S%!*3FthqN83b&p^zboB4&aQH48 z?MaD)r=PpO^q+U@WdC0JAH6LX8cK{7=^DI!n&^MOC`U@#!N=d zgYn(4Gk7t^V_=0v&uLNlpgyCIF+KW@&~uEJfWjlJhc@}0&u7du@CD;lXk-2_^4>h2 z$|rpIKZwY#LYrjCmIy6K_MD=~R*`+lzV9N^Vo7^K+O!ZYlFE{zq@+Zt$Wo~!N-Aj+ zzkANPrqB7zN8ji9Jptn$+#j^{N0$p`F@pVh=%7=54u`OMV~;EqK8Q=q;42)+;E$W@HDVb6~R}7eHsXE&ciIYo#2^}Kc2&l^~C!oeJA9F zz^}L<9hNr+Ir3R42k>VpLf#7eL6zWmKg=ZrcL8}kS09&)_sO&)z6Y@`Cyv6&fV0lw!v9v-$egouRN^o@DL*XI8RY3pe1V`&>qVS&Jc>lm31n&er zxnNwxdOCnTClcHY>@P|1IOy+qpJkj+545`$A>Re{H6eHwaBG6|19u@feg-9g;3Lql zV+oFLtda@N27ZqB?Zx^RLi^?t@{SNcPw*#T->U>a5Bk>;{1^D=Q-b$Gzj{OP+t6NL z2)-Qj$NP$6{cE9IQAmgRcgSCo;8I{W^4TO!kXuN|OTzWqh~T>5C#wj)2*zheg5$Z> zz67_1dPNcZ1n9qo;I*Lt9)e$n_z{9%0e#L89PeLUM(|_MUbO_b20J_=_(y1`7X;^p ze)@snPeA{l1Q!E)jt6_-cI<%oB!a6$f1E||CD5#7eNqS>0(QtCcoX8g?7aIs$e@zg??H{$m9KKO9?&`+Wi*6 zw}Rh3B={xJ6YnFB%RL1AJs}?t?f8S>`2LRzt{XTXdGMc!1Q!4%@L}Mo1iuG%+ePrT(7qW2?*aQC zB6tDVr-a~*V9#=bB@p}*d|yZ>_;c{LY=S=k{qqTq_hG(F@J<*v@b?~U z&);C@hlG3rj0-Oao(A>mC-@M^4->ou>NNrM#pTLEKH>z&`$^9xxGLnYLGbn9=LQ7F zYXIQ;AaokJY(CW2k&wRv+>hX=LH`(n8$rKICb$9AD}&(G;8!OIo(#O0;CNs6DuN$^ z`?Lmvr$M}(;2&T->?Jw$i|+)thxX%w^XN3P+iK`{lL(#z>7@uh8SJS_@YA5r5`vFH z+>GFO-(dW^1=hy~?yvCg6PSm9pWxpEF#iPO(qv;<7AVhFmC|82u(NOLjf{Q_1 zi{SWo7CONXfdAn8GOW*a@E>OdN!QX>E0R$HYeKrvMIQVNS!Hb~XGYOs!_BloHK9DaVxIOsE z4T4ufK8*xl1p0Im`~mcf_XJOdcKS*1wctP8(Ehl+M!^3?3BC}x48dKXU*qu}=OYXD z*CXWLKzt>^gWu`tUjcrA;MTa`5xf<65y4HN zzh5OdzE8&A1F)V?K+h+HJRToj5j+|0d+~J}=aUNdL1+7LM{! zXXH-pS59y_;Q0F}vKx~30REnkcLDwhIMyG!FO3I2U@i&r_;)qT@$WGB|6Pbf6Eaem zLO!@14?x_SC4aazVaSKF$Rque7?j$toZ$HX!5dgOtKIPLDA*2{A)izhd6fSd;28ve z13a7H&w(ESj_Ql_p91^}!SVms9}s*M$m8oCw$Dr89N>?bZw5c)1CGjtF2qP7O~_;a zQD)(&zMLo_73L9ql!<9N1jqj;)+hKZ$Y&{VtPh@F=0I>s$UlOG!w|_hvzy2tukpE` z;CfK*VIm(?Hx%+%o}uJnmxsL5wz!MV_@>Z5Ga2t`u-oU!;Qt#Ag$HJH!$jHMS%wp9M7u=CAc!kM-hB6@Hm270N+G#2jJTX?g1RPC$>)j@ZE%bH1LDK zky!vS$OjhA>i;vre(1pJhu$ol)ej{hACxz1{&W`3nm@Kbl4Z>w&$B|DHUIyswb)pv zI>N34|5=NTRp!rHY{Z&wtTHe`@?Yz+IWxcNeSeL(o!ppZanukx^HP{6~>$uq>oAKxiLL;oKh&>TqQ zTK{TuDy~Pp#tPtue@@M!J|V{HW9XzFdK)8<>Ct$Mz5@N9rbpv8%JXmfl``U*(uUr#Yq6x_y9pO_&5 z%E$7U<2XKcP$w9o7=R$22bnk?r(_-pFI00>G_M|4k%(_{UGDdoq~ zAO9a=5v0cloF2#dsKjHHpAQ{B3IrvoBl?K}r^n@^wdBVtAOD|0k4nQDaC*!n|5CoB z9#ev~d^9Hpr^n@M{iXaD@OuURo`=iB>2ZAiU&_zaV@lxPZ6v8EntzYeD1qU z)`ok5z7$d63{Jp|{z;G0ps?yM>36{dAk@CB0cVG?3r^1}M14L66Y1p%j?>|z52Po= zXmxlZK5+u4!x+u0L0yy`12{H9dR2mB{W0DH<&)|0wp~S75RPQ}bSnK={wrz7OyCUZ zB&j1>9}cI-?T6~~pRy@dgc_R)N8#iw>N6TU$@FNLAg{d?qFKNf!Otkd5>R=p0rlyz z(kFoaIYa^?N>fprrzXh~{#R?msV!WvkeT3re{HxmZvSYk$N0!x8;-nY*<6#P&Z5QM zt|K*B7K}?v6Q>A0?IP;#^psolv?Oi@s6gW@ovoVAHb7TO(qo*xOIO;9E`M}|J9LH5 z^b}*Ec|&MZ^BaPq3<*JD<3DS{88fyur>FFwwdR=1%xx6M3&!Df<`^ndZ#Gu+#P^w0 z&hGTIakBDA#gxr5=t_gO-@e8e>chh6DPPfYbt&CAn~!)Qxr$ws2hdY$aXg8eYCY!u z>uGu#Y*m_#f@U{T7{3PgO%a?))t8Y1=W`pUW1-p%>A1)AG^Qf%bS0){;S=bIHAq*g zV$5{ZEEdOi{Gu`=1s1s0I0`mGml+?##1WsswkU zt(@p-hT>lY;~5&z)ApmvPzgVL78H_27Nd@g9|y`PQ5xDn-cwMXAxoWL3{F(Moq7#Y z=j2P#_c`hWL*VwN0V;-wr(nQ$#wo_CfN6$8*HC8k6kcj6Kf0h%Ero^XYUfhn-lJZFp9LMMb5w4Ej`b+B&s1g)Q7%tDGa>=h(eVp9^NBiB zht4!mXQ)U3mA~Unbf$qiGlb53qRvo}fFVyo*C*7Gl5(Zues=|hQ(KTFu(n8;QWL2j zK)qaVdkQ|;)Prn5?Fldbr?myQOrZt=YL!Asv^t`L{yOIRfv~SSw4Q!S4YL9J6q)