diff --git a/source/darwin/dispatch_queue.h b/source/darwin/dispatch_queue.h index a5d1bea8d..320f066ea 100644 --- a/source/darwin/dispatch_queue.h +++ b/source/darwin/dispatch_queue.h @@ -1,5 +1,5 @@ -#ifndef AWS_IO_PRIVATE_DISPATCH_QUEUE_H -#define AWS_IO_PRIVATE_DISPATCH_QUEUE_H +#ifndef AWS_IO_DARWIN_DISPATCH_QUEUE_H +#define AWS_IO_DARWIN_DISPATCH_QUEUE_H /** * Copyright Amazon.com, Inc. or its affiliates. All Rights Reserved. * SPDX-License-Identifier: Apache-2.0. @@ -37,12 +37,16 @@ struct dispatch_scheduling_state { struct aws_linked_list scheduled_services; }; +struct dispatch_loop; +struct dispatch_loop_context; + struct dispatch_loop { struct aws_allocator *allocator; struct aws_ref_count ref_count; dispatch_queue_t dispatch_queue; struct aws_task_scheduler scheduler; struct aws_linked_list local_cross_thread_tasks; + struct aws_event_loop *base_loop; /* Apple dispatch queue uses the id string to identify the dispatch queue */ struct aws_string *dispatch_queue_id; @@ -50,7 +54,7 @@ struct dispatch_loop { struct { struct dispatch_scheduling_state scheduling_state; struct aws_linked_list cross_thread_tasks; - struct aws_mutex lock; + struct dispatch_loop_context *context; bool suspended; /* `is_executing` flag and `current_thread_id` together are used to identify the excuting * thread id for dispatch queue. See `static bool s_is_on_callers_thread(struct aws_event_loop *event_loop)` @@ -63,4 +67,4 @@ struct dispatch_loop { bool is_destroying; }; -#endif /* #ifndef AWS_IO_PRIVATE_DISPATCH_QUEUE_H */ +#endif /* #ifndef AWS_IO_DARWIN_DISPATCH_QUEUE_H */ diff --git a/source/darwin/dispatch_queue_event_loop.c b/source/darwin/dispatch_queue_event_loop.c index 6d72c3da6..1148ba212 100644 --- a/source/darwin/dispatch_queue_event_loop.c +++ b/source/darwin/dispatch_queue_event_loop.c @@ -48,41 +48,51 @@ static struct aws_event_loop_vtable s_vtable = { .is_on_callers_thread = s_is_on_callers_thread, }; +/* Internal ref-counted dispatch loop context to processing Apple Dispatch Queue Resources */ +struct dispatch_loop_context { + struct aws_mutex lock; + struct dispatch_loop *io_dispatch_loop; + struct aws_allocator *allocator; + struct aws_ref_count ref_count; +}; + struct scheduled_service_entry { struct aws_allocator *allocator; uint64_t timestamp; struct aws_linked_list_node node; - struct aws_event_loop *loop; - bool cancel; // The entry will be canceled if the event loop is destroyed. + struct dispatch_loop_context *dispatch_queue_context; }; -static struct scheduled_service_entry *scheduled_service_entry_new(struct aws_event_loop *loop, uint64_t timestamp) { - struct scheduled_service_entry *entry = aws_mem_calloc(loop->alloc, 1, sizeof(struct scheduled_service_entry)); +static struct scheduled_service_entry *s_scheduled_service_entry_new( + struct dispatch_loop_context *context, + uint64_t timestamp) { + struct scheduled_service_entry *entry = + aws_mem_calloc(context->allocator, 1, sizeof(struct scheduled_service_entry)); - entry->allocator = loop->alloc; + entry->allocator = context->allocator; entry->timestamp = timestamp; - entry->loop = loop; - struct dispatch_loop *dispatch_loop = loop->impl_data; - aws_ref_count_acquire(&dispatch_loop->ref_count); + entry->dispatch_queue_context = context; + aws_ref_count_acquire(&context->ref_count); return entry; } // may only be called when the dispatch event loop synced data lock is held -static void scheduled_service_entry_destroy(struct scheduled_service_entry *entry) { +static void s_scheduled_service_entry_destroy(struct scheduled_service_entry *entry) { if (aws_linked_list_node_is_in_list(&entry->node)) { aws_linked_list_remove(&entry->node); } - struct dispatch_loop *dispatch_loop = entry->loop->impl_data; - aws_ref_count_release(&dispatch_loop->ref_count); + struct dispatch_loop_context *dispatch_queue_context = entry->dispatch_queue_context; + aws_ref_count_release(&dispatch_queue_context->ref_count); aws_mem_release(entry->allocator, entry); - entry = NULL; } // checks to see if another scheduled iteration already exists that will either // handle our needs or reschedule at the end to do so -static bool should_schedule_iteration(struct aws_linked_list *scheduled_iterations, uint64_t proposed_iteration_time) { +static bool s_should_schedule_iteration( + struct aws_linked_list *scheduled_iterations, + uint64_t proposed_iteration_time) { if (aws_linked_list_empty(scheduled_iterations)) { return true; } @@ -94,13 +104,25 @@ static bool should_schedule_iteration(struct aws_linked_list *scheduled_iteratio return entry->timestamp > proposed_iteration_time; } +/* On dispatch event loop context ref-count reaches 0 */ +static void s_dispatch_loop_context_destroy(void *context) { + struct dispatch_loop_context *dispatch_loop_context = context; + aws_mutex_clean_up(&dispatch_loop_context->lock); + aws_mem_release(dispatch_loop_context->allocator, dispatch_loop_context); +} + +/* On dispatch event loop ref-count reaches 0 */ static void s_dispatch_event_loop_destroy(void *context) { // release dispatch loop - struct aws_event_loop *event_loop = context; struct dispatch_loop *dispatch_loop = event_loop->impl_data; - aws_mutex_clean_up(&dispatch_loop->synced_data.lock); + // Null out the dispatch queue loop context + aws_mutex_lock(&dispatch_loop->synced_data.context->lock); + dispatch_loop->synced_data.context->io_dispatch_loop = NULL; + aws_mutex_unlock(&dispatch_loop->synced_data.context->lock); + aws_ref_count_release(&dispatch_loop->synced_data.context->ref_count); + aws_string_destroy(dispatch_loop->dispatch_queue_id); aws_mem_release(dispatch_loop->allocator, dispatch_loop); aws_event_loop_clean_up_base(event_loop); @@ -148,7 +170,6 @@ struct aws_event_loop *aws_event_loop_new_with_dispatch_queue( } dispatch_loop = aws_mem_calloc(alloc, 1, sizeof(struct dispatch_loop)); - aws_ref_count_init(&dispatch_loop->ref_count, loop, s_dispatch_event_loop_destroy); dispatch_loop->dispatch_queue_id = s_get_unique_dispatch_queue_id(alloc); @@ -160,25 +181,31 @@ struct aws_event_loop *aws_event_loop_new_with_dispatch_queue( goto clean_up; } - dispatch_loop->synced_data.scheduling_state.is_executing_iteration = false; - dispatch_loop->allocator = alloc; - int err = aws_task_scheduler_init(&dispatch_loop->scheduler, alloc); if (err) { AWS_LOGF_ERROR(AWS_LS_IO_EVENT_LOOP, "id=%p: Initializing task scheduler failed", (void *)loop); goto clean_up; } + dispatch_loop->synced_data.scheduling_state.is_executing_iteration = false; + dispatch_loop->allocator = alloc; + dispatch_loop->base_loop = loop; + aws_linked_list_init(&dispatch_loop->local_cross_thread_tasks); aws_linked_list_init(&dispatch_loop->synced_data.scheduling_state.scheduled_services); aws_linked_list_init(&dispatch_loop->synced_data.cross_thread_tasks); - aws_mutex_init(&dispatch_loop->synced_data.lock); + struct dispatch_loop_context *context = aws_mem_calloc(alloc, 1, sizeof(struct dispatch_loop_context)); + aws_ref_count_init(&context->ref_count, context, s_dispatch_loop_context_destroy); + aws_mutex_init(&context->lock); + context->io_dispatch_loop = dispatch_loop; + context->allocator = alloc; + dispatch_loop->synced_data.context = context; loop->impl_data = dispatch_loop; loop->vtable = &s_vtable; - /** manually increament the thread count, so the library will wait for dispatch queue releasing */ + /** manually increment the thread count, so the library will wait for dispatch queue releasing */ aws_thread_increment_unjoined_count(); return loop; @@ -188,7 +215,7 @@ struct aws_event_loop *aws_event_loop_new_with_dispatch_queue( if (dispatch_loop->dispatch_queue) { dispatch_release(dispatch_loop->dispatch_queue); } - aws_ref_count_release(&dispatch_loop->ref_count); + s_dispatch_event_loop_destroy(loop); aws_event_loop_clean_up_base(loop); } @@ -197,58 +224,52 @@ struct aws_event_loop *aws_event_loop_new_with_dispatch_queue( return NULL; } +static void s_dispatch_queue_destroy_task(void *context) { + struct dispatch_loop *dispatch_loop = context; + + aws_mutex_lock(&dispatch_loop->synced_data.context->lock); + dispatch_loop->synced_data.current_thread_id = aws_thread_current_thread_id(); + dispatch_loop->synced_data.is_executing = true; + aws_mutex_unlock(&dispatch_loop->synced_data.context->lock); + + aws_task_scheduler_clean_up(&dispatch_loop->scheduler); + aws_mutex_lock(&dispatch_loop->synced_data.context->lock); + + while (!aws_linked_list_empty(&dispatch_loop->synced_data.cross_thread_tasks)) { + struct aws_linked_list_node *node = aws_linked_list_pop_front(&dispatch_loop->synced_data.cross_thread_tasks); + struct aws_task *task = AWS_CONTAINER_OF(node, struct aws_task, node); + task->fn(task, task->arg, AWS_TASK_STATUS_CANCELED); + } + + while (!aws_linked_list_empty(&dispatch_loop->local_cross_thread_tasks)) { + struct aws_linked_list_node *node = aws_linked_list_pop_front(&dispatch_loop->local_cross_thread_tasks); + struct aws_task *task = AWS_CONTAINER_OF(node, struct aws_task, node); + task->fn(task, task->arg, AWS_TASK_STATUS_CANCELED); + } + + dispatch_loop->synced_data.suspended = true; + dispatch_loop->synced_data.is_executing = false; + aws_mutex_unlock(&dispatch_loop->synced_data.context->lock); + + s_dispatch_event_loop_destroy(dispatch_loop->base_loop); +} + static void s_destroy(struct aws_event_loop *event_loop) { AWS_LOGF_TRACE(AWS_LS_IO_EVENT_LOOP, "id=%p: Destroying Dispatch Queue Event Loop", (void *)event_loop); struct dispatch_loop *dispatch_loop = event_loop->impl_data; - - /* To avoid double destroy */ - if (dispatch_loop->is_destroying) { + /* Avoid double release on dispatch_loop */ + if (!dispatch_loop) { return; } - dispatch_loop->is_destroying = true; /* make sure the loop is running so we can schedule a last task. */ s_run(event_loop); /* cancel outstanding tasks */ - dispatch_async_and_wait(dispatch_loop->dispatch_queue, ^{ - aws_mutex_lock(&dispatch_loop->synced_data.lock); - dispatch_loop->synced_data.current_thread_id = aws_thread_current_thread_id(); - dispatch_loop->synced_data.is_executing = true; - aws_mutex_unlock(&dispatch_loop->synced_data.lock); - - aws_task_scheduler_clean_up(&dispatch_loop->scheduler); - - while (!aws_linked_list_empty(&dispatch_loop->synced_data.cross_thread_tasks)) { - struct aws_linked_list_node *node = aws_linked_list_pop_front(&dispatch_loop->synced_data.cross_thread_tasks); - struct aws_task *task = AWS_CONTAINER_OF(node, struct aws_task, node); - task->fn(task, task->arg, AWS_TASK_STATUS_CANCELED); - } - - while (!aws_linked_list_empty(&dispatch_loop->local_cross_thread_tasks)) { - struct aws_linked_list_node *node = aws_linked_list_pop_front(&dispatch_loop->local_cross_thread_tasks); - struct aws_task *task = AWS_CONTAINER_OF(node, struct aws_task, node); - task->fn(task, task->arg, AWS_TASK_STATUS_CANCELED); - } - - aws_mutex_lock(&dispatch_loop->synced_data.lock); - /* The entries in the scheduled_services are already put on the apple dispatch queue. It would be a bad memory - * access if we destroy the entries here. We instead setting a cancel flag to cancel the task when the - * dispatch_queue execute the entry. */ - struct aws_linked_list_node *iter = NULL; - for (iter = aws_linked_list_begin(&dispatch_loop->synced_data.scheduling_state.scheduled_services); - iter != aws_linked_list_end(&dispatch_loop->synced_data.scheduling_state.scheduled_services); - iter = aws_linked_list_next(iter)) { - struct scheduled_service_entry *entry = AWS_CONTAINER_OF(iter, struct scheduled_service_entry, node); - entry->cancel = true; - } - dispatch_loop->synced_data.suspended = true; - dispatch_loop->synced_data.is_executing = false; - aws_mutex_unlock(&dispatch_loop->synced_data.lock); - }); + dispatch_async_and_wait_f(dispatch_loop->dispatch_queue, dispatch_loop, s_dispatch_queue_destroy_task); AWS_LOGF_TRACE(AWS_LS_IO_EVENT_LOOP, "id=%p: Releasing Dispatch Queue.", (void *)event_loop); - aws_ref_count_release(&dispatch_loop->ref_count); + event_loop->impl_data = NULL; } static int s_wait_for_stop_completion(struct aws_event_loop *event_loop) { @@ -260,13 +281,13 @@ static int s_wait_for_stop_completion(struct aws_event_loop *event_loop) { static int s_run(struct aws_event_loop *event_loop) { struct dispatch_loop *dispatch_loop = event_loop->impl_data; - aws_mutex_lock(&dispatch_loop->synced_data.lock); + aws_mutex_lock(&dispatch_loop->synced_data.context->lock); if (dispatch_loop->synced_data.suspended) { AWS_LOGF_INFO(AWS_LS_IO_EVENT_LOOP, "id=%p: Starting event-loop thread.", (void *)event_loop); dispatch_resume(dispatch_loop->dispatch_queue); dispatch_loop->synced_data.suspended = false; } - aws_mutex_unlock(&dispatch_loop->synced_data.lock); + aws_mutex_unlock(&dispatch_loop->synced_data.context->lock); return AWS_OP_SUCCESS; } @@ -274,7 +295,7 @@ static int s_run(struct aws_event_loop *event_loop) { static int s_stop(struct aws_event_loop *event_loop) { struct dispatch_loop *dispatch_loop = event_loop->impl_data; - aws_mutex_lock(&dispatch_loop->synced_data.lock); + aws_mutex_lock(&dispatch_loop->synced_data.context->lock); if (!dispatch_loop->synced_data.suspended) { dispatch_loop->synced_data.suspended = true; AWS_LOGF_INFO(AWS_LS_IO_EVENT_LOOP, "id=%p: Stopping event-loop thread.", (void *)event_loop); @@ -282,19 +303,24 @@ static int s_stop(struct aws_event_loop *event_loop) { * releasing the dispatch queue. */ dispatch_suspend(dispatch_loop->dispatch_queue); } - aws_mutex_unlock(&dispatch_loop->synced_data.lock); + aws_mutex_unlock(&dispatch_loop->synced_data.context->lock); return AWS_OP_SUCCESS; } -static void try_schedule_new_iteration(struct aws_event_loop *loop, uint64_t timestamp); +static void s_try_schedule_new_iteration(struct dispatch_loop_context *loop, uint64_t timestamp); // returns true if we should execute an iteration, false otherwise static bool begin_iteration(struct scheduled_service_entry *entry) { bool should_execute_iteration = false; - struct dispatch_loop *dispatch_loop = entry->loop->impl_data; + struct dispatch_loop_context *contxt = entry->dispatch_queue_context; + aws_mutex_lock(&contxt->lock); - aws_mutex_lock(&dispatch_loop->synced_data.lock); + struct dispatch_loop *dispatch_loop = entry->dispatch_queue_context->io_dispatch_loop; + if (!dispatch_loop) { + aws_mutex_unlock(&contxt->lock); + return should_execute_iteration; + } // swap the cross-thread tasks into task-local data AWS_FATAL_ASSERT(aws_linked_list_empty(&dispatch_loop->local_cross_thread_tasks)); @@ -306,59 +332,67 @@ static bool begin_iteration(struct scheduled_service_entry *entry) { aws_linked_list_remove(&entry->node); should_execute_iteration = true; - aws_mutex_unlock(&dispatch_loop->synced_data.lock); + aws_mutex_unlock(&contxt->lock); return should_execute_iteration; } // conditionally schedule another iteration as needed static void end_iteration(struct scheduled_service_entry *entry) { - struct dispatch_loop *loop = entry->loop->impl_data; - aws_mutex_lock(&loop->synced_data.lock); + struct dispatch_loop_context *contxt = entry->dispatch_queue_context; + aws_mutex_lock(&contxt->lock); + struct dispatch_loop *dispatch_loop = entry->dispatch_queue_context->io_dispatch_loop; + if (!dispatch_loop) { + aws_mutex_unlock(&contxt->lock); + return; + } - loop->synced_data.scheduling_state.is_executing_iteration = false; + dispatch_loop->synced_data.scheduling_state.is_executing_iteration = false; // if there are any cross-thread tasks, reschedule an iteration for now - if (!aws_linked_list_empty(&loop->synced_data.cross_thread_tasks)) { + if (!aws_linked_list_empty(&dispatch_loop->synced_data.cross_thread_tasks)) { // added during service which means nothing was scheduled because is_executing_iteration was true - try_schedule_new_iteration(entry->loop, 0); + s_try_schedule_new_iteration(contxt, 0); } else { // no cross thread tasks, so check internal time-based scheduler uint64_t next_task_time = 0; /* we already know it has tasks, we just scheduled one. We just want the next run time. */ - bool has_task = aws_task_scheduler_has_tasks(&loop->scheduler, &next_task_time); + bool has_task = aws_task_scheduler_has_tasks(&dispatch_loop->scheduler, &next_task_time); if (has_task) { // only schedule an iteration if there isn't an existing dispatched iteration for the next task time or // earlier - if (should_schedule_iteration(&loop->synced_data.scheduling_state.scheduled_services, next_task_time)) { - try_schedule_new_iteration(entry->loop, next_task_time); + if (s_should_schedule_iteration( + &dispatch_loop->synced_data.scheduling_state.scheduled_services, next_task_time)) { + s_try_schedule_new_iteration(contxt, next_task_time); } } } - scheduled_service_entry_destroy(entry); - aws_mutex_unlock(&loop->synced_data.lock); + s_scheduled_service_entry_destroy(entry); + aws_mutex_unlock(&contxt->lock); } -// this function is what gets scheduled and executed by the Dispatch Queue API -static void run_iteration(void *context) { +// Iteration function that scheduled and executed by the Dispatch Queue API +static void s_run_iteration(void *context) { struct scheduled_service_entry *entry = context; - struct aws_event_loop *event_loop = entry->loop; - struct dispatch_loop *dispatch_loop = event_loop->impl_data; - AWS_ASSERT(event_loop && dispatch_loop); - if (entry->cancel) { - scheduled_service_entry_destroy(entry); + + struct dispatch_loop_context *dispatch_queue_context = entry->dispatch_queue_context; + aws_mutex_lock(&dispatch_queue_context->lock); + struct dispatch_loop *dispatch_loop = entry->dispatch_queue_context->io_dispatch_loop; + aws_mutex_unlock(&dispatch_queue_context->lock); + if (!dispatch_loop) { + s_scheduled_service_entry_destroy(entry); return; } if (!begin_iteration(entry)) { - scheduled_service_entry_destroy(entry); + s_scheduled_service_entry_destroy(entry); return; } - aws_event_loop_register_tick_start(event_loop); + aws_event_loop_register_tick_start(dispatch_loop->base_loop); // run the full iteration here: local cross-thread tasks while (!aws_linked_list_empty(&dispatch_loop->local_cross_thread_tasks)) { @@ -373,20 +407,20 @@ static void run_iteration(void *context) { } } - aws_mutex_lock(&dispatch_loop->synced_data.lock); + aws_mutex_lock(&dispatch_queue_context->lock); dispatch_loop->synced_data.current_thread_id = aws_thread_current_thread_id(); dispatch_loop->synced_data.is_executing = true; - aws_mutex_unlock(&dispatch_loop->synced_data.lock); + aws_mutex_unlock(&dispatch_queue_context->lock); // run all scheduled tasks uint64_t now_ns = 0; - aws_event_loop_current_clock_time(event_loop, &now_ns); + aws_event_loop_current_clock_time(dispatch_loop->base_loop, &now_ns); aws_task_scheduler_run_all(&dispatch_loop->scheduler, now_ns); - aws_event_loop_register_tick_end(event_loop); + aws_event_loop_register_tick_end(dispatch_loop->base_loop); - aws_mutex_lock(&dispatch_loop->synced_data.lock); + aws_mutex_lock(&dispatch_queue_context->lock); dispatch_loop->synced_data.is_executing = false; - aws_mutex_unlock(&dispatch_loop->synced_data.lock); + aws_mutex_unlock(&dispatch_queue_context->lock); end_iteration(entry); } @@ -399,22 +433,22 @@ static void run_iteration(void *context) { * * The function should be wrapped with dispatch_loop->synced_data->lock */ -static void try_schedule_new_iteration(struct aws_event_loop *loop, uint64_t timestamp) { - struct dispatch_loop *dispatch_loop = loop->impl_data; +static void s_try_schedule_new_iteration(struct dispatch_loop_context *dispatch_loop_context, uint64_t timestamp) { + struct dispatch_loop *dispatch_loop = dispatch_loop_context->io_dispatch_loop; if (dispatch_loop->synced_data.suspended) return; - if (!should_schedule_iteration(&dispatch_loop->synced_data.scheduling_state.scheduled_services, timestamp)) { + if (!s_should_schedule_iteration(&dispatch_loop->synced_data.scheduling_state.scheduled_services, timestamp)) { return; } - struct scheduled_service_entry *entry = scheduled_service_entry_new(loop, timestamp); + struct scheduled_service_entry *entry = s_scheduled_service_entry_new(dispatch_loop_context, timestamp); aws_linked_list_push_front(&dispatch_loop->synced_data.scheduling_state.scheduled_services, &entry->node); - dispatch_async_f(dispatch_loop->dispatch_queue, entry, run_iteration); + dispatch_async_f(dispatch_loop->dispatch_queue, entry, s_run_iteration); } static void s_schedule_task_common(struct aws_event_loop *event_loop, struct aws_task *task, uint64_t run_at_nanos) { struct dispatch_loop *dispatch_loop = event_loop->impl_data; - aws_mutex_lock(&dispatch_loop->synced_data.lock); + aws_mutex_lock(&dispatch_loop->synced_data.context->lock); bool should_schedule = false; bool is_empty = aws_linked_list_empty(&dispatch_loop->synced_data.cross_thread_tasks); @@ -437,15 +471,15 @@ static void s_schedule_task_common(struct aws_event_loop *event_loop, struct aws /** If there is no currently running iteration, then we check if we have already scheduled an iteration * scheduled before this task's run time. */ should_schedule = - should_schedule_iteration(&dispatch_loop->synced_data.scheduling_state.scheduled_services, run_at_nanos); + s_should_schedule_iteration(&dispatch_loop->synced_data.scheduling_state.scheduled_services, run_at_nanos); } // If there is no scheduled iteration, start one right now to process the `cross_thread_task`. if (should_schedule) { - try_schedule_new_iteration(event_loop, 0); + s_try_schedule_new_iteration(dispatch_loop->synced_data.context, 0); } - aws_mutex_unlock(&dispatch_loop->synced_data.lock); + aws_mutex_unlock(&dispatch_loop->synced_data.context->lock); } static void s_schedule_task_now(struct aws_event_loop *event_loop, struct aws_task *task) { @@ -491,10 +525,10 @@ static int s_unsubscribe_from_io_events(struct aws_event_loop *event_loop, struc // dispatch queue. static bool s_is_on_callers_thread(struct aws_event_loop *event_loop) { struct dispatch_loop *dispatch_queue = event_loop->impl_data; - aws_mutex_lock(&dispatch_queue->synced_data.lock); + aws_mutex_lock(&dispatch_queue->synced_data.context->lock); bool result = dispatch_queue->synced_data.is_executing && aws_thread_thread_id_equal(dispatch_queue->synced_data.current_thread_id, aws_thread_current_thread_id()); - aws_mutex_unlock(&dispatch_queue->synced_data.lock); + aws_mutex_unlock(&dispatch_queue->synced_data.context->lock); return result; } diff --git a/tests/event_loop_test.c b/tests/event_loop_test.c index 791f3d8c1..477547cad 100644 --- a/tests/event_loop_test.c +++ b/tests/event_loop_test.c @@ -1026,8 +1026,7 @@ static int s_test_event_loop_all_types_creation(struct aws_allocator *allocator, enable_iocp = true; #endif #ifdef AWS_ENABLE_DISPATCH_QUEUE -// TODO: Dispatch queue support is not yet implemented. Uncomment the following line once the dispatch queue is ready. -// enable_dispatch_queue = true; + enable_dispatch_queue = true; #endif return s_test_event_loop_creation(allocator, AWS_EVENT_LOOP_EPOLL, enable_epoll) ||