duckdb 0.7.2-dev2820.0 → 0.7.2-dev2995.0
This diff represents the content of publicly available package versions that have been released to one of the supported registries. The information contained in this diff is provided for informational purposes only and reflects changes between package versions as they appear in their respective public registries.
- package/binding.gyp +1 -0
- package/package.json +1 -1
- package/src/duckdb/extension/icu/icu-datepart.cpp +55 -1
- package/src/duckdb/extension/parquet/parquet-extension.cpp +5 -4
- package/src/duckdb/src/catalog/catalog_entry/table_catalog_entry.cpp +18 -7
- package/src/duckdb/src/catalog/default/default_functions.cpp +2 -0
- package/src/duckdb/src/common/arrow/arrow_appender.cpp +3 -3
- package/src/duckdb/src/common/arrow/arrow_converter.cpp +2 -2
- package/src/duckdb/src/common/local_file_system.cpp +1 -3
- package/src/duckdb/src/common/multi_file_reader.cpp +11 -8
- package/src/duckdb/src/common/sort/partition_state.cpp +1 -1
- package/src/duckdb/src/common/string_util.cpp +6 -1
- package/src/duckdb/src/core_functions/function_list.cpp +2 -0
- package/src/duckdb/src/core_functions/scalar/string/format_bytes.cpp +29 -0
- package/src/duckdb/src/execution/index/art/art.cpp +5 -1
- package/src/duckdb/src/execution/operator/aggregate/physical_hash_aggregate.cpp +62 -43
- package/src/duckdb/src/execution/operator/aggregate/physical_perfecthash_aggregate.cpp +17 -11
- package/src/duckdb/src/execution/operator/aggregate/physical_ungrouped_aggregate.cpp +32 -39
- package/src/duckdb/src/execution/operator/aggregate/physical_window.cpp +10 -9
- package/src/duckdb/src/execution/operator/helper/physical_batch_collector.cpp +4 -4
- package/src/duckdb/src/execution/operator/helper/physical_explain_analyze.cpp +6 -21
- package/src/duckdb/src/execution/operator/helper/physical_limit.cpp +13 -13
- package/src/duckdb/src/execution/operator/helper/physical_limit_percent.cpp +15 -14
- package/src/duckdb/src/execution/operator/helper/physical_load.cpp +3 -2
- package/src/duckdb/src/execution/operator/helper/physical_materialized_collector.cpp +4 -4
- package/src/duckdb/src/execution/operator/helper/physical_pragma.cpp +4 -2
- package/src/duckdb/src/execution/operator/helper/physical_prepare.cpp +4 -2
- package/src/duckdb/src/execution/operator/helper/physical_reservoir_sample.cpp +10 -8
- package/src/duckdb/src/execution/operator/helper/physical_reset.cpp +4 -3
- package/src/duckdb/src/execution/operator/helper/physical_set.cpp +7 -6
- package/src/duckdb/src/execution/operator/helper/physical_transaction.cpp +4 -2
- package/src/duckdb/src/execution/operator/helper/physical_vacuum.cpp +8 -8
- package/src/duckdb/src/execution/operator/join/physical_asof_join.cpp +17 -16
- package/src/duckdb/src/execution/operator/join/physical_blockwise_nl_join.cpp +10 -8
- package/src/duckdb/src/execution/operator/join/physical_cross_product.cpp +3 -4
- package/src/duckdb/src/execution/operator/join/physical_delim_join.cpp +5 -5
- package/src/duckdb/src/execution/operator/join/physical_hash_join.cpp +16 -15
- package/src/duckdb/src/execution/operator/join/physical_iejoin.cpp +13 -12
- package/src/duckdb/src/execution/operator/join/physical_nested_loop_join.cpp +12 -10
- package/src/duckdb/src/execution/operator/join/physical_piecewise_merge_join.cpp +13 -11
- package/src/duckdb/src/execution/operator/join/physical_positional_join.cpp +8 -6
- package/src/duckdb/src/execution/operator/join/physical_range_join.cpp +1 -1
- package/src/duckdb/src/execution/operator/order/physical_order.cpp +13 -13
- package/src/duckdb/src/execution/operator/order/physical_top_n.cpp +8 -8
- package/src/duckdb/src/execution/operator/persistent/physical_batch_insert.cpp +160 -145
- package/src/duckdb/src/execution/operator/persistent/physical_copy_to_file.cpp +10 -25
- package/src/duckdb/src/execution/operator/persistent/physical_delete.cpp +14 -19
- package/src/duckdb/src/execution/operator/persistent/physical_export.cpp +7 -6
- package/src/duckdb/src/execution/operator/persistent/physical_insert.cpp +18 -30
- package/src/duckdb/src/execution/operator/persistent/physical_update.cpp +14 -18
- package/src/duckdb/src/execution/operator/scan/physical_column_data_scan.cpp +6 -4
- package/src/duckdb/src/execution/operator/scan/physical_dummy_scan.cpp +4 -19
- package/src/duckdb/src/execution/operator/scan/physical_empty_result.cpp +3 -2
- package/src/duckdb/src/execution/operator/scan/physical_positional_scan.cpp +14 -5
- package/src/duckdb/src/execution/operator/scan/physical_table_scan.cpp +6 -4
- package/src/duckdb/src/execution/operator/schema/physical_alter.cpp +3 -19
- package/src/duckdb/src/execution/operator/schema/physical_attach.cpp +4 -18
- package/src/duckdb/src/execution/operator/schema/physical_create_function.cpp +4 -19
- package/src/duckdb/src/execution/operator/schema/physical_create_index.cpp +8 -9
- package/src/duckdb/src/execution/operator/schema/physical_create_schema.cpp +4 -19
- package/src/duckdb/src/execution/operator/schema/physical_create_sequence.cpp +4 -19
- package/src/duckdb/src/execution/operator/schema/physical_create_table.cpp +4 -19
- package/src/duckdb/src/execution/operator/schema/physical_create_type.cpp +20 -28
- package/src/duckdb/src/execution/operator/schema/physical_create_view.cpp +4 -19
- package/src/duckdb/src/execution/operator/schema/physical_detach.cpp +4 -19
- package/src/duckdb/src/execution/operator/schema/physical_drop.cpp +3 -19
- package/src/duckdb/src/execution/operator/set/physical_recursive_cte.cpp +9 -8
- package/src/duckdb/src/execution/operator/set/physical_union.cpp +1 -1
- package/src/duckdb/src/execution/physical_operator.cpp +11 -5
- package/src/duckdb/src/execution/radix_partitioned_hashtable.cpp +16 -16
- package/src/duckdb/src/function/table/arrow_conversion.cpp +3 -3
- package/src/duckdb/src/function/table/read_csv.cpp +7 -4
- package/src/duckdb/src/function/table/version/pragma_version.cpp +2 -2
- package/src/duckdb/src/include/duckdb/catalog/catalog_entry/table_catalog_entry.hpp +7 -1
- package/src/duckdb/src/include/duckdb/common/enums/operator_result_type.hpp +16 -4
- package/src/duckdb/src/include/duckdb/common/multi_file_reader.hpp +5 -4
- package/src/duckdb/src/include/duckdb/common/optional_idx.hpp +45 -0
- package/src/duckdb/src/include/duckdb/common/set.hpp +2 -1
- package/src/duckdb/src/include/duckdb/core_functions/scalar/string_functions.hpp +15 -0
- package/src/duckdb/src/include/duckdb/execution/executor.hpp +10 -1
- package/src/duckdb/src/include/duckdb/execution/operator/aggregate/physical_hash_aggregate.hpp +5 -8
- package/src/duckdb/src/include/duckdb/execution/operator/aggregate/physical_perfecthash_aggregate.hpp +2 -4
- package/src/duckdb/src/include/duckdb/execution/operator/aggregate/physical_ungrouped_aggregate.hpp +3 -7
- package/src/duckdb/src/include/duckdb/execution/operator/aggregate/physical_window.hpp +2 -4
- package/src/duckdb/src/include/duckdb/execution/operator/helper/physical_batch_collector.hpp +1 -2
- package/src/duckdb/src/include/duckdb/execution/operator/helper/physical_explain_analyze.hpp +2 -5
- package/src/duckdb/src/include/duckdb/execution/operator/helper/physical_limit.hpp +2 -4
- package/src/duckdb/src/include/duckdb/execution/operator/helper/physical_limit_percent.hpp +2 -4
- package/src/duckdb/src/include/duckdb/execution/operator/helper/physical_load.hpp +1 -2
- package/src/duckdb/src/include/duckdb/execution/operator/helper/physical_materialized_collector.hpp +1 -2
- package/src/duckdb/src/include/duckdb/execution/operator/helper/physical_pragma.hpp +1 -2
- package/src/duckdb/src/include/duckdb/execution/operator/helper/physical_prepare.hpp +1 -2
- package/src/duckdb/src/include/duckdb/execution/operator/helper/physical_reservoir_sample.hpp +2 -4
- package/src/duckdb/src/include/duckdb/execution/operator/helper/physical_reset.hpp +1 -2
- package/src/duckdb/src/include/duckdb/execution/operator/helper/physical_set.hpp +1 -2
- package/src/duckdb/src/include/duckdb/execution/operator/helper/physical_transaction.hpp +1 -2
- package/src/duckdb/src/include/duckdb/execution/operator/helper/physical_vacuum.hpp +2 -4
- package/src/duckdb/src/include/duckdb/execution/operator/join/physical_asof_join.hpp +2 -4
- package/src/duckdb/src/include/duckdb/execution/operator/join/physical_blockwise_nl_join.hpp +2 -4
- package/src/duckdb/src/include/duckdb/execution/operator/join/physical_cross_product.hpp +1 -2
- package/src/duckdb/src/include/duckdb/execution/operator/join/physical_delim_join.hpp +1 -2
- package/src/duckdb/src/include/duckdb/execution/operator/join/physical_hash_join.hpp +2 -4
- package/src/duckdb/src/include/duckdb/execution/operator/join/physical_iejoin.hpp +2 -4
- package/src/duckdb/src/include/duckdb/execution/operator/join/physical_nested_loop_join.hpp +2 -4
- package/src/duckdb/src/include/duckdb/execution/operator/join/physical_piecewise_merge_join.hpp +2 -4
- package/src/duckdb/src/include/duckdb/execution/operator/join/physical_positional_join.hpp +2 -4
- package/src/duckdb/src/include/duckdb/execution/operator/order/physical_order.hpp +2 -4
- package/src/duckdb/src/include/duckdb/execution/operator/order/physical_top_n.hpp +2 -4
- package/src/duckdb/src/include/duckdb/execution/operator/persistent/physical_batch_insert.hpp +3 -5
- package/src/duckdb/src/include/duckdb/execution/operator/persistent/physical_copy_to_file.hpp +2 -5
- package/src/duckdb/src/include/duckdb/execution/operator/persistent/physical_delete.hpp +2 -4
- package/src/duckdb/src/include/duckdb/execution/operator/persistent/physical_export.hpp +2 -4
- package/src/duckdb/src/include/duckdb/execution/operator/persistent/physical_insert.hpp +2 -4
- package/src/duckdb/src/include/duckdb/execution/operator/persistent/physical_update.hpp +2 -4
- package/src/duckdb/src/include/duckdb/execution/operator/scan/physical_column_data_scan.hpp +1 -2
- package/src/duckdb/src/include/duckdb/execution/operator/scan/physical_dummy_scan.hpp +1 -3
- package/src/duckdb/src/include/duckdb/execution/operator/scan/physical_empty_result.hpp +1 -2
- package/src/duckdb/src/include/duckdb/execution/operator/scan/physical_positional_scan.hpp +1 -2
- package/src/duckdb/src/include/duckdb/execution/operator/scan/physical_table_scan.hpp +1 -2
- package/src/duckdb/src/include/duckdb/execution/operator/schema/physical_alter.hpp +1 -3
- package/src/duckdb/src/include/duckdb/execution/operator/schema/physical_attach.hpp +1 -3
- package/src/duckdb/src/include/duckdb/execution/operator/schema/physical_create_function.hpp +1 -3
- package/src/duckdb/src/include/duckdb/execution/operator/schema/physical_create_index.hpp +2 -4
- package/src/duckdb/src/include/duckdb/execution/operator/schema/physical_create_schema.hpp +1 -3
- package/src/duckdb/src/include/duckdb/execution/operator/schema/physical_create_sequence.hpp +1 -3
- package/src/duckdb/src/include/duckdb/execution/operator/schema/physical_create_table.hpp +1 -3
- package/src/duckdb/src/include/duckdb/execution/operator/schema/physical_create_type.hpp +6 -5
- package/src/duckdb/src/include/duckdb/execution/operator/schema/physical_create_view.hpp +1 -3
- package/src/duckdb/src/include/duckdb/execution/operator/schema/physical_detach.hpp +1 -3
- package/src/duckdb/src/include/duckdb/execution/operator/schema/physical_drop.hpp +1 -3
- package/src/duckdb/src/include/duckdb/execution/operator/set/physical_recursive_cte.hpp +2 -4
- package/src/duckdb/src/include/duckdb/execution/physical_operator.hpp +7 -4
- package/src/duckdb/src/include/duckdb/execution/physical_operator_states.hpp +26 -6
- package/src/duckdb/src/include/duckdb/execution/radix_partitioned_hashtable.hpp +5 -5
- package/src/duckdb/src/include/duckdb/function/aggregate_function.hpp +2 -1
- package/src/duckdb/src/include/duckdb/function/table_function.hpp +0 -1
- package/src/duckdb/src/include/duckdb/main/client_config.hpp +2 -0
- package/src/duckdb/src/include/duckdb/main/config.hpp +2 -0
- package/src/duckdb/src/include/duckdb/main/database.hpp +1 -0
- package/src/duckdb/src/include/duckdb/main/database_manager.hpp +3 -0
- package/src/duckdb/src/include/duckdb/main/extension_helper.hpp +0 -2
- package/src/duckdb/src/include/duckdb/parallel/event.hpp +1 -1
- package/src/duckdb/src/include/duckdb/parallel/interrupt.hpp +63 -0
- package/src/duckdb/src/include/duckdb/parallel/pipeline.hpp +16 -3
- package/src/duckdb/src/include/duckdb/parallel/pipeline_executor.hpp +51 -7
- package/src/duckdb/src/include/duckdb/parallel/task.hpp +21 -2
- package/src/duckdb/src/include/duckdb/parallel/task_counter.hpp +2 -2
- package/src/duckdb/src/include/duckdb/parallel/task_scheduler.hpp +2 -2
- package/src/duckdb/src/include/duckdb/parser/tableref/pivotref.hpp +3 -0
- package/src/duckdb/src/include/duckdb/parser/transformer.hpp +5 -1
- package/src/duckdb/src/include/duckdb/planner/column_binding.hpp +6 -0
- package/src/duckdb/src/include/duckdb/planner/expression/bound_columnref_expression.hpp +1 -0
- package/src/duckdb/src/include/duckdb/planner/operator/logical_aggregate.hpp +1 -0
- package/src/duckdb/src/include/duckdb/planner/operator/logical_column_data_get.hpp +1 -0
- package/src/duckdb/src/include/duckdb/planner/operator/logical_cteref.hpp +1 -0
- package/src/duckdb/src/include/duckdb/planner/operator/logical_delete.hpp +1 -0
- package/src/duckdb/src/include/duckdb/planner/operator/logical_delim_get.hpp +1 -0
- package/src/duckdb/src/include/duckdb/planner/operator/logical_dummy_scan.hpp +1 -0
- package/src/duckdb/src/include/duckdb/planner/operator/logical_expression_get.hpp +1 -0
- package/src/duckdb/src/include/duckdb/planner/operator/logical_insert.hpp +1 -0
- package/src/duckdb/src/include/duckdb/planner/operator/logical_pivot.hpp +1 -0
- package/src/duckdb/src/include/duckdb/planner/operator/logical_projection.hpp +1 -0
- package/src/duckdb/src/include/duckdb/planner/operator/logical_recursive_cte.hpp +1 -0
- package/src/duckdb/src/include/duckdb/planner/operator/logical_set_operation.hpp +1 -0
- package/src/duckdb/src/include/duckdb/planner/operator/logical_unnest.hpp +1 -0
- package/src/duckdb/src/include/duckdb/planner/operator/logical_update.hpp +1 -0
- package/src/duckdb/src/include/duckdb/planner/operator/logical_window.hpp +1 -0
- package/src/duckdb/src/include/duckdb/storage/data_table.hpp +1 -0
- package/src/duckdb/src/include/duckdb/storage/optimistic_data_writer.hpp +46 -0
- package/src/duckdb/src/include/duckdb/storage/partial_block_manager.hpp +24 -3
- package/src/duckdb/src/include/duckdb/storage/table/column_checkpoint_state.hpp +46 -1
- package/src/duckdb/src/include/duckdb/storage/table/column_data.hpp +9 -10
- package/src/duckdb/src/include/duckdb/storage/table/column_segment.hpp +1 -1
- package/src/duckdb/src/include/duckdb/storage/table/list_column_data.hpp +2 -2
- package/src/duckdb/src/include/duckdb/storage/table/row_group.hpp +3 -3
- package/src/duckdb/src/include/duckdb/storage/table/row_group_collection.hpp +1 -0
- package/src/duckdb/src/include/duckdb/storage/table/segment_base.hpp +1 -1
- package/src/duckdb/src/include/duckdb/storage/table/segment_tree.hpp +22 -0
- package/src/duckdb/src/include/duckdb/storage/table/standard_column_data.hpp +3 -3
- package/src/duckdb/src/include/duckdb/storage/table/struct_column_data.hpp +2 -2
- package/src/duckdb/src/include/duckdb/storage/table/update_segment.hpp +0 -2
- package/src/duckdb/src/include/duckdb/storage/table/validity_column_data.hpp +1 -2
- package/src/duckdb/src/include/duckdb/transaction/local_storage.hpp +9 -34
- package/src/duckdb/src/include/duckdb/verification/no_operator_caching_verifier.hpp +25 -0
- package/src/duckdb/src/include/duckdb/verification/statement_verifier.hpp +5 -0
- package/src/duckdb/src/main/attached_database.cpp +5 -3
- package/src/duckdb/src/main/client_verify.cpp +4 -0
- package/src/duckdb/src/main/config.cpp +4 -0
- package/src/duckdb/src/main/database.cpp +45 -48
- package/src/duckdb/src/main/extension/extension_load.cpp +32 -49
- package/src/duckdb/src/parallel/event.cpp +1 -1
- package/src/duckdb/src/parallel/executor.cpp +39 -3
- package/src/duckdb/src/parallel/executor_task.cpp +11 -0
- package/src/duckdb/src/parallel/interrupt.cpp +57 -0
- package/src/duckdb/src/parallel/pipeline.cpp +49 -6
- package/src/duckdb/src/parallel/pipeline_executor.cpp +248 -69
- package/src/duckdb/src/parallel/pipeline_initialize_event.cpp +1 -1
- package/src/duckdb/src/parallel/task_scheduler.cpp +57 -22
- package/src/duckdb/src/parser/base_expression.cpp +6 -0
- package/src/duckdb/src/parser/transform/statement/transform_create_function.cpp +1 -4
- package/src/duckdb/src/parser/transform/statement/transform_create_view.cpp +2 -4
- package/src/duckdb/src/parser/transform/statement/transform_pivot_stmt.cpp +43 -24
- package/src/duckdb/src/parser/transform/tableref/transform_pivot.cpp +3 -0
- package/src/duckdb/src/planner/binder/statement/bind_create.cpp +17 -28
- package/src/duckdb/src/planner/expression/bound_columnref_expression.cpp +17 -3
- package/src/duckdb/src/planner/expression/bound_reference_expression.cpp +8 -2
- package/src/duckdb/src/planner/operator/logical_aggregate.cpp +13 -1
- package/src/duckdb/src/planner/operator/logical_column_data_get.cpp +11 -0
- package/src/duckdb/src/planner/operator/logical_cteref.cpp +11 -0
- package/src/duckdb/src/planner/operator/logical_delete.cpp +10 -0
- package/src/duckdb/src/planner/operator/logical_delim_get.cpp +12 -1
- package/src/duckdb/src/planner/operator/logical_dummy_scan.cpp +12 -1
- package/src/duckdb/src/planner/operator/logical_expression_get.cpp +12 -1
- package/src/duckdb/src/planner/operator/logical_get.cpp +10 -4
- package/src/duckdb/src/planner/operator/logical_insert.cpp +12 -1
- package/src/duckdb/src/planner/operator/logical_pivot.cpp +11 -0
- package/src/duckdb/src/planner/operator/logical_projection.cpp +11 -0
- package/src/duckdb/src/planner/operator/logical_recursive_cte.cpp +11 -0
- package/src/duckdb/src/planner/operator/logical_set_operation.cpp +11 -0
- package/src/duckdb/src/planner/operator/logical_unnest.cpp +12 -1
- package/src/duckdb/src/planner/operator/logical_update.cpp +10 -0
- package/src/duckdb/src/planner/operator/logical_window.cpp +11 -0
- package/src/duckdb/src/storage/checkpoint_manager.cpp +1 -1
- package/src/duckdb/src/storage/data_table.cpp +5 -0
- package/src/duckdb/src/storage/local_storage.cpp +40 -110
- package/src/duckdb/src/storage/optimistic_data_writer.cpp +96 -0
- package/src/duckdb/src/storage/partial_block_manager.cpp +73 -9
- package/src/duckdb/src/storage/single_file_block_manager.cpp +3 -1
- package/src/duckdb/src/storage/standard_buffer_manager.cpp +17 -12
- package/src/duckdb/src/storage/statistics/base_statistics.cpp +3 -0
- package/src/duckdb/src/storage/table/column_checkpoint_state.cpp +90 -82
- package/src/duckdb/src/storage/table/column_data.cpp +19 -45
- package/src/duckdb/src/storage/table/column_data_checkpointer.cpp +7 -7
- package/src/duckdb/src/storage/table/column_segment.cpp +1 -1
- package/src/duckdb/src/storage/table/list_column_data.cpp +6 -11
- package/src/duckdb/src/storage/table/row_group.cpp +13 -14
- package/src/duckdb/src/storage/table/row_group_collection.cpp +10 -4
- package/src/duckdb/src/storage/table/standard_column_data.cpp +6 -10
- package/src/duckdb/src/storage/table/struct_column_data.cpp +7 -13
- package/src/duckdb/src/storage/table/update_segment.cpp +0 -25
- package/src/duckdb/src/storage/table/validity_column_data.cpp +2 -6
- package/src/duckdb/src/transaction/commit_state.cpp +4 -4
- package/src/duckdb/src/verification/no_operator_caching_verifier.cpp +13 -0
- package/src/duckdb/src/verification/statement_verifier.cpp +4 -0
- package/src/duckdb/third_party/fmt/format.cc +0 -5
- package/src/duckdb/third_party/fmt/include/fmt/core.h +10 -12
- package/src/duckdb/third_party/fmt/include/fmt/format-inl.h +2 -33
- package/src/duckdb/third_party/fmt/include/fmt/format.h +61 -24
- package/src/duckdb/third_party/fmt/include/fmt/printf.h +15 -1
- package/src/duckdb/third_party/libpg_query/include/nodes/parsenodes.hpp +1 -0
- package/src/duckdb/third_party/libpg_query/src_backend_parser_gram.cpp +10735 -10674
- package/src/duckdb/ub_src_core_functions_scalar_string.cpp +2 -0
- package/src/duckdb/ub_src_parallel.cpp +2 -0
- package/src/duckdb/ub_src_storage.cpp +2 -0
@@ -359,6 +359,7 @@ void Executor::CancelTasks() {
|
|
359
359
|
}
|
360
360
|
pipelines.clear();
|
361
361
|
root_pipelines.clear();
|
362
|
+
to_be_rescheduled_tasks.clear();
|
362
363
|
events.clear();
|
363
364
|
}
|
364
365
|
WorkOnTasks();
|
@@ -375,13 +376,44 @@ void Executor::CancelTasks() {
|
|
375
376
|
void Executor::WorkOnTasks() {
|
376
377
|
auto &scheduler = TaskScheduler::GetScheduler(context);
|
377
378
|
|
378
|
-
|
379
|
+
shared_ptr<Task> task;
|
379
380
|
while (scheduler.GetTaskFromProducer(*producer, task)) {
|
380
|
-
task->Execute(TaskExecutionMode::PROCESS_ALL);
|
381
|
+
auto res = task->Execute(TaskExecutionMode::PROCESS_ALL);
|
382
|
+
if (res == TaskExecutionResult::TASK_BLOCKED) {
|
383
|
+
task->Deschedule();
|
384
|
+
}
|
381
385
|
task.reset();
|
382
386
|
}
|
383
387
|
}
|
384
388
|
|
389
|
+
void Executor::RescheduleTask(shared_ptr<Task> &task) {
|
390
|
+
// This function will spin lock until the task provided is added to the to_be_rescheduled_tasks
|
391
|
+
while (true) {
|
392
|
+
lock_guard<mutex> l(executor_lock);
|
393
|
+
if (cancelled) {
|
394
|
+
return;
|
395
|
+
}
|
396
|
+
auto entry = to_be_rescheduled_tasks.find(task.get());
|
397
|
+
if (entry != to_be_rescheduled_tasks.end()) {
|
398
|
+
auto &scheduler = TaskScheduler::GetScheduler(context);
|
399
|
+
to_be_rescheduled_tasks.erase(task.get());
|
400
|
+
scheduler.ScheduleTask(GetToken(), task);
|
401
|
+
break;
|
402
|
+
}
|
403
|
+
}
|
404
|
+
}
|
405
|
+
|
406
|
+
void Executor::AddToBeRescheduled(shared_ptr<Task> &task) {
|
407
|
+
lock_guard<mutex> l(executor_lock);
|
408
|
+
if (cancelled) {
|
409
|
+
return;
|
410
|
+
}
|
411
|
+
if (to_be_rescheduled_tasks.find(task.get()) != to_be_rescheduled_tasks.end()) {
|
412
|
+
return;
|
413
|
+
}
|
414
|
+
to_be_rescheduled_tasks[task.get()] = std::move(task);
|
415
|
+
}
|
416
|
+
|
385
417
|
bool Executor::ExecutionIsFinished() {
|
386
418
|
return completed_pipelines >= total_pipelines || HasError();
|
387
419
|
}
|
@@ -400,7 +432,10 @@ PendingExecutionResult Executor::ExecuteTask() {
|
|
400
432
|
if (task) {
|
401
433
|
// if we have a task, partially process it
|
402
434
|
auto result = task->Execute(TaskExecutionMode::PROCESS_PARTIAL);
|
403
|
-
if (result
|
435
|
+
if (result == TaskExecutionResult::TASK_BLOCKED) {
|
436
|
+
task->Deschedule();
|
437
|
+
task.reset();
|
438
|
+
} else if (result == TaskExecutionResult::TASK_FINISHED) {
|
404
439
|
// if the task is finished, clean it up
|
405
440
|
task.reset();
|
406
441
|
}
|
@@ -444,6 +479,7 @@ void Executor::Reset() {
|
|
444
479
|
exceptions.clear();
|
445
480
|
pipelines.clear();
|
446
481
|
events.clear();
|
482
|
+
to_be_rescheduled_tasks.clear();
|
447
483
|
execution_result = PendingExecutionResult::RESULT_NOT_READY;
|
448
484
|
}
|
449
485
|
|
@@ -1,5 +1,6 @@
|
|
1
1
|
#include "duckdb/parallel/task.hpp"
|
2
2
|
#include "duckdb/execution/executor.hpp"
|
3
|
+
#include "duckdb/main/client_context.hpp"
|
3
4
|
|
4
5
|
namespace duckdb {
|
5
6
|
|
@@ -12,6 +13,16 @@ ExecutorTask::ExecutorTask(ClientContext &context) : ExecutorTask(Executor::Get(
|
|
12
13
|
ExecutorTask::~ExecutorTask() {
|
13
14
|
}
|
14
15
|
|
16
|
+
void ExecutorTask::Deschedule() {
|
17
|
+
auto this_ptr = shared_from_this();
|
18
|
+
executor.AddToBeRescheduled(this_ptr);
|
19
|
+
}
|
20
|
+
|
21
|
+
void ExecutorTask::Reschedule() {
|
22
|
+
auto this_ptr = shared_from_this();
|
23
|
+
executor.RescheduleTask(this_ptr);
|
24
|
+
}
|
25
|
+
|
15
26
|
TaskExecutionResult ExecutorTask::Execute(TaskExecutionMode mode) {
|
16
27
|
try {
|
17
28
|
return ExecuteTask(mode);
|
@@ -0,0 +1,57 @@
|
|
1
|
+
#include "duckdb/parallel/interrupt.hpp"
|
2
|
+
#include "duckdb/execution/executor.hpp"
|
3
|
+
#include "duckdb/main/client_context.hpp"
|
4
|
+
#include "duckdb/common/atomic.hpp"
|
5
|
+
#include "duckdb/common/mutex.hpp"
|
6
|
+
#include <condition_variable>
|
7
|
+
|
8
|
+
namespace duckdb {
|
9
|
+
|
10
|
+
InterruptState::InterruptState() : mode(InterruptMode::NO_INTERRUPTS) {
|
11
|
+
}
|
12
|
+
InterruptState::InterruptState(weak_ptr<Task> task) : mode(InterruptMode::TASK), current_task(std::move(task)) {
|
13
|
+
}
|
14
|
+
InterruptState::InterruptState(weak_ptr<InterruptDoneSignalState> signal_state_p)
|
15
|
+
: mode(InterruptMode::BLOCKING), signal_state(std::move(signal_state_p)) {
|
16
|
+
}
|
17
|
+
|
18
|
+
void InterruptState::Callback() const {
|
19
|
+
if (mode == InterruptMode::TASK) {
|
20
|
+
auto task = current_task.lock();
|
21
|
+
|
22
|
+
if (!task) {
|
23
|
+
return;
|
24
|
+
}
|
25
|
+
|
26
|
+
task->Reschedule();
|
27
|
+
} else if (mode == InterruptMode::BLOCKING) {
|
28
|
+
auto signal_state_l = signal_state.lock();
|
29
|
+
|
30
|
+
if (!signal_state_l) {
|
31
|
+
return;
|
32
|
+
}
|
33
|
+
|
34
|
+
// Signal the caller, who is currently blocked
|
35
|
+
signal_state_l->Signal();
|
36
|
+
} else {
|
37
|
+
throw InternalException("Callback made on InterruptState without valid interrupt mode specified");
|
38
|
+
}
|
39
|
+
}
|
40
|
+
|
41
|
+
void InterruptDoneSignalState::Signal() {
|
42
|
+
{
|
43
|
+
unique_lock<mutex> lck {lock};
|
44
|
+
done = true;
|
45
|
+
}
|
46
|
+
cv.notify_all();
|
47
|
+
}
|
48
|
+
|
49
|
+
void InterruptDoneSignalState::Await() {
|
50
|
+
std::unique_lock<std::mutex> lck(lock);
|
51
|
+
cv.wait(lck, [&]() { return done; });
|
52
|
+
|
53
|
+
// Reset after signal received
|
54
|
+
done = false;
|
55
|
+
}
|
56
|
+
|
57
|
+
} // namespace duckdb
|
@@ -12,7 +12,6 @@
|
|
12
12
|
#include "duckdb/parallel/pipeline_event.hpp"
|
13
13
|
#include "duckdb/parallel/pipeline_executor.hpp"
|
14
14
|
#include "duckdb/parallel/task_scheduler.hpp"
|
15
|
-
#include "duckdb/parallel/thread_context.hpp"
|
16
15
|
|
17
16
|
namespace duckdb {
|
18
17
|
|
@@ -33,14 +32,32 @@ public:
|
|
33
32
|
if (!pipeline_executor) {
|
34
33
|
pipeline_executor = make_uniq<PipelineExecutor>(pipeline.GetClientContext(), pipeline);
|
35
34
|
}
|
35
|
+
|
36
|
+
pipeline_executor->SetTaskForInterrupts(shared_from_this());
|
37
|
+
|
36
38
|
if (mode == TaskExecutionMode::PROCESS_PARTIAL) {
|
37
|
-
|
38
|
-
|
39
|
+
auto res = pipeline_executor->Execute(PARTIAL_CHUNK_COUNT);
|
40
|
+
|
41
|
+
switch (res) {
|
42
|
+
case PipelineExecuteResult::NOT_FINISHED:
|
39
43
|
return TaskExecutionResult::TASK_NOT_FINISHED;
|
44
|
+
case PipelineExecuteResult::INTERRUPTED:
|
45
|
+
return TaskExecutionResult::TASK_BLOCKED;
|
46
|
+
case PipelineExecuteResult::FINISHED:
|
47
|
+
break;
|
40
48
|
}
|
41
49
|
} else {
|
42
|
-
pipeline_executor->Execute();
|
50
|
+
auto res = pipeline_executor->Execute();
|
51
|
+
switch (res) {
|
52
|
+
case PipelineExecuteResult::NOT_FINISHED:
|
53
|
+
throw InternalException("Execute without limit should not return NOT_FINISHED");
|
54
|
+
case PipelineExecuteResult::INTERRUPTED:
|
55
|
+
return TaskExecutionResult::TASK_BLOCKED;
|
56
|
+
case PipelineExecuteResult::FINISHED:
|
57
|
+
break;
|
58
|
+
}
|
43
59
|
}
|
60
|
+
|
44
61
|
event->FinishTask();
|
45
62
|
pipeline_executor.reset();
|
46
63
|
return TaskExecutionResult::TASK_FINISHED;
|
@@ -68,7 +85,7 @@ bool Pipeline::GetProgress(double ¤t_percentage, idx_t &source_cardinality
|
|
68
85
|
}
|
69
86
|
|
70
87
|
void Pipeline::ScheduleSequentialTask(shared_ptr<Event> &event) {
|
71
|
-
vector<
|
88
|
+
vector<shared_ptr<Task>> tasks;
|
72
89
|
tasks.push_back(make_uniq<PipelineTask>(*this, event));
|
73
90
|
event->SetTasks(std::move(tasks));
|
74
91
|
}
|
@@ -149,7 +166,7 @@ bool Pipeline::LaunchScanTasks(shared_ptr<Event> &event, idx_t max_threads) {
|
|
149
166
|
}
|
150
167
|
|
151
168
|
// launch a task for every thread
|
152
|
-
vector<
|
169
|
+
vector<shared_ptr<Task>> tasks;
|
153
170
|
for (idx_t i = 0; i < max_threads; i++) {
|
154
171
|
tasks.push_back(make_uniq<PipelineTask>(*this, event));
|
155
172
|
}
|
@@ -265,6 +282,32 @@ vector<const_reference<PhysicalOperator>> Pipeline::GetOperators() const {
|
|
265
282
|
return result;
|
266
283
|
}
|
267
284
|
|
285
|
+
void Pipeline::ClearSource() {
|
286
|
+
source_state.reset();
|
287
|
+
batch_indexes.clear();
|
288
|
+
}
|
289
|
+
|
290
|
+
idx_t Pipeline::RegisterNewBatchIndex() {
|
291
|
+
lock_guard<mutex> l(batch_lock);
|
292
|
+
idx_t minimum = batch_indexes.empty() ? base_batch_index : *batch_indexes.begin();
|
293
|
+
batch_indexes.insert(minimum);
|
294
|
+
return minimum;
|
295
|
+
}
|
296
|
+
|
297
|
+
idx_t Pipeline::UpdateBatchIndex(idx_t old_index, idx_t new_index) {
|
298
|
+
lock_guard<mutex> l(batch_lock);
|
299
|
+
if (new_index < *batch_indexes.begin()) {
|
300
|
+
throw InternalException("Processing batch index %llu, but previous min batch index was %llu", new_index,
|
301
|
+
*batch_indexes.begin());
|
302
|
+
}
|
303
|
+
auto entry = batch_indexes.find(old_index);
|
304
|
+
if (entry == batch_indexes.end()) {
|
305
|
+
throw InternalException("Batch index %llu was not found in set of active batch indexes", old_index);
|
306
|
+
}
|
307
|
+
batch_indexes.erase(entry);
|
308
|
+
batch_indexes.insert(new_index);
|
309
|
+
return *batch_indexes.begin();
|
310
|
+
}
|
268
311
|
//===--------------------------------------------------------------------===//
|
269
312
|
// Pipeline Build State
|
270
313
|
//===--------------------------------------------------------------------===//
|
@@ -2,16 +2,29 @@
|
|
2
2
|
#include "duckdb/main/client_context.hpp"
|
3
3
|
#include "duckdb/common/limits.hpp"
|
4
4
|
|
5
|
+
#ifdef DUCKDB_DEBUG_ASYNC_SINK_SOURCE
|
6
|
+
#include <thread>
|
7
|
+
#include <chrono>
|
8
|
+
#endif
|
9
|
+
|
5
10
|
namespace duckdb {
|
6
11
|
|
7
12
|
PipelineExecutor::PipelineExecutor(ClientContext &context_p, Pipeline &pipeline_p)
|
8
13
|
: pipeline(pipeline_p), thread(context_p), context(context_p, thread, &pipeline_p) {
|
9
14
|
D_ASSERT(pipeline.source_state);
|
10
|
-
local_source_state = pipeline.source->GetLocalSourceState(context, *pipeline.source_state);
|
11
15
|
if (pipeline.sink) {
|
12
16
|
local_sink_state = pipeline.sink->GetLocalSinkState(context);
|
13
17
|
requires_batch_index = pipeline.sink->RequiresBatchIndex() && pipeline.source->SupportsBatchIndex();
|
18
|
+
if (requires_batch_index) {
|
19
|
+
auto &partition_info = local_sink_state->partition_info;
|
20
|
+
if (!partition_info.batch_index.IsValid()) {
|
21
|
+
// batch index is not set yet - initialize before fetching anything
|
22
|
+
partition_info.batch_index = pipeline.RegisterNewBatchIndex();
|
23
|
+
partition_info.min_batch_index = partition_info.batch_index;
|
24
|
+
}
|
25
|
+
}
|
14
26
|
}
|
27
|
+
local_source_state = pipeline.source->GetLocalSourceState(context, *pipeline.source_state);
|
15
28
|
|
16
29
|
intermediate_chunks.reserve(pipeline.operators.size());
|
17
30
|
intermediate_states.reserve(pipeline.operators.size());
|
@@ -35,35 +48,136 @@ PipelineExecutor::PipelineExecutor(ClientContext &context_p, Pipeline &pipeline_
|
|
35
48
|
InitializeChunk(final_chunk);
|
36
49
|
}
|
37
50
|
|
38
|
-
bool PipelineExecutor::
|
51
|
+
bool PipelineExecutor::TryFlushCachingOperators() {
|
52
|
+
if (!started_flushing) {
|
53
|
+
// Remainder of this method assumes any in process operators are from flushing
|
54
|
+
D_ASSERT(in_process_operators.empty());
|
55
|
+
started_flushing = true;
|
56
|
+
flushing_idx = IsFinished() ? idx_t(finished_processing_idx) : 0;
|
57
|
+
}
|
58
|
+
|
59
|
+
// Go over each operator and keep flushing them using `FinalExecute` until empty
|
60
|
+
while (flushing_idx < pipeline.operators.size()) {
|
61
|
+
if (!pipeline.operators[flushing_idx].get().RequiresFinalExecute()) {
|
62
|
+
flushing_idx++;
|
63
|
+
continue;
|
64
|
+
}
|
65
|
+
|
66
|
+
// This slightly awkward way of increasing the flushing idx is to make the code re-entrant: We need to call this
|
67
|
+
// method again in the case of a Sink returning BLOCKED.
|
68
|
+
if (!should_flush_current_idx && in_process_operators.empty()) {
|
69
|
+
should_flush_current_idx = true;
|
70
|
+
flushing_idx++;
|
71
|
+
continue;
|
72
|
+
}
|
73
|
+
|
74
|
+
auto &curr_chunk =
|
75
|
+
flushing_idx + 1 >= intermediate_chunks.size() ? final_chunk : *intermediate_chunks[flushing_idx + 1];
|
76
|
+
auto ¤t_operator = pipeline.operators[flushing_idx].get();
|
77
|
+
|
78
|
+
OperatorFinalizeResultType finalize_result;
|
79
|
+
OperatorResultType push_result;
|
80
|
+
|
81
|
+
if (in_process_operators.empty()) {
|
82
|
+
StartOperator(current_operator);
|
83
|
+
finalize_result = current_operator.FinalExecute(context, curr_chunk, *current_operator.op_state,
|
84
|
+
*intermediate_states[flushing_idx]);
|
85
|
+
EndOperator(current_operator, &curr_chunk);
|
86
|
+
} else {
|
87
|
+
// Reset flag and reflush the last chunk we were flushing.
|
88
|
+
finalize_result = OperatorFinalizeResultType::HAVE_MORE_OUTPUT;
|
89
|
+
}
|
90
|
+
|
91
|
+
push_result = ExecutePushInternal(curr_chunk, flushing_idx + 1);
|
92
|
+
|
93
|
+
if (finalize_result == OperatorFinalizeResultType::HAVE_MORE_OUTPUT) {
|
94
|
+
should_flush_current_idx = true;
|
95
|
+
} else {
|
96
|
+
should_flush_current_idx = false;
|
97
|
+
}
|
98
|
+
|
99
|
+
if (push_result == OperatorResultType::BLOCKED) {
|
100
|
+
remaining_sink_chunk = true;
|
101
|
+
return false;
|
102
|
+
} else if (push_result == OperatorResultType::FINISHED) {
|
103
|
+
break;
|
104
|
+
}
|
105
|
+
}
|
106
|
+
return true;
|
107
|
+
}
|
108
|
+
|
109
|
+
PipelineExecuteResult PipelineExecutor::Execute(idx_t max_chunks) {
|
39
110
|
D_ASSERT(pipeline.sink);
|
40
|
-
bool exhausted_source = false;
|
41
111
|
auto &source_chunk = pipeline.operators.empty() ? final_chunk : *intermediate_chunks[0];
|
42
112
|
for (idx_t i = 0; i < max_chunks; i++) {
|
43
|
-
if (
|
44
|
-
|
113
|
+
if (context.client.interrupted) {
|
114
|
+
throw InterruptException();
|
45
115
|
}
|
46
|
-
|
47
|
-
|
48
|
-
if (
|
49
|
-
exhausted_source = true;
|
116
|
+
|
117
|
+
OperatorResultType result;
|
118
|
+
if (exhausted_source && done_flushing && !remaining_sink_chunk && in_process_operators.empty()) {
|
50
119
|
break;
|
120
|
+
} else if (remaining_sink_chunk) {
|
121
|
+
// The pipeline was interrupted by the Sink. We should retry sinking the final chunk.
|
122
|
+
result = ExecutePushInternal(final_chunk);
|
123
|
+
remaining_sink_chunk = false;
|
124
|
+
} else if (!in_process_operators.empty() && !started_flushing) {
|
125
|
+
// The pipeline was interrupted by the Sink when pushing a source chunk through the pipeline. We need to
|
126
|
+
// re-push the same source chunk through the pipeline because there are in_process operators, meaning that
|
127
|
+
// the result for the pipeline
|
128
|
+
D_ASSERT(source_chunk.size() > 0);
|
129
|
+
result = ExecutePushInternal(source_chunk);
|
130
|
+
} else if (exhausted_source && !done_flushing) {
|
131
|
+
// The source was exhausted, try flushing all operators
|
132
|
+
auto flush_completed = TryFlushCachingOperators();
|
133
|
+
if (flush_completed) {
|
134
|
+
done_flushing = true;
|
135
|
+
break;
|
136
|
+
} else {
|
137
|
+
return PipelineExecuteResult::INTERRUPTED;
|
138
|
+
}
|
139
|
+
} else if (!exhausted_source) {
|
140
|
+
// "Regular" path: fetch a chunk from the source and push it through the pipeline
|
141
|
+
source_chunk.Reset();
|
142
|
+
SourceResultType source_result = FetchFromSource(source_chunk);
|
143
|
+
|
144
|
+
if (source_result == SourceResultType::BLOCKED) {
|
145
|
+
return PipelineExecuteResult::INTERRUPTED;
|
146
|
+
}
|
147
|
+
|
148
|
+
if (source_result == SourceResultType::FINISHED) {
|
149
|
+
exhausted_source = true;
|
150
|
+
if (source_chunk.size() == 0) {
|
151
|
+
continue;
|
152
|
+
}
|
153
|
+
}
|
154
|
+
result = ExecutePushInternal(source_chunk);
|
155
|
+
} else {
|
156
|
+
throw InternalException("Unexpected state reached in pipeline executor");
|
51
157
|
}
|
52
|
-
|
158
|
+
|
159
|
+
// SINK INTERRUPT
|
160
|
+
if (result == OperatorResultType::BLOCKED) {
|
161
|
+
remaining_sink_chunk = true;
|
162
|
+
return PipelineExecuteResult::INTERRUPTED;
|
163
|
+
}
|
164
|
+
|
53
165
|
if (result == OperatorResultType::FINISHED) {
|
54
|
-
D_ASSERT(IsFinished());
|
55
166
|
break;
|
56
167
|
}
|
57
168
|
}
|
58
|
-
|
59
|
-
|
169
|
+
|
170
|
+
if ((!exhausted_source || !done_flushing) && !IsFinished()) {
|
171
|
+
return PipelineExecuteResult::NOT_FINISHED;
|
60
172
|
}
|
173
|
+
|
61
174
|
PushFinalize();
|
62
|
-
|
175
|
+
|
176
|
+
return PipelineExecuteResult::FINISHED;
|
63
177
|
}
|
64
178
|
|
65
|
-
|
66
|
-
Execute(NumericLimits<idx_t>::Maximum());
|
179
|
+
PipelineExecuteResult PipelineExecutor::Execute() {
|
180
|
+
return Execute(NumericLimits<idx_t>::Maximum());
|
67
181
|
}
|
68
182
|
|
69
183
|
OperatorResultType PipelineExecutor::ExecutePush(DataChunk &input) { // LCOV_EXCL_START
|
@@ -84,6 +198,10 @@ OperatorResultType PipelineExecutor::ExecutePushInternal(DataChunk &input, idx_t
|
|
84
198
|
if (input.size() == 0) { // LCOV_EXCL_START
|
85
199
|
return OperatorResultType::NEED_MORE_INPUT;
|
86
200
|
} // LCOV_EXCL_STOP
|
201
|
+
|
202
|
+
// this loop will continuously push the input chunk through the pipeline as long as:
|
203
|
+
// - the OperatorResultType for the Execute is HAVE_MORE_OUTPUT
|
204
|
+
// - the Sink doesn't block
|
87
205
|
while (true) {
|
88
206
|
OperatorResultType result;
|
89
207
|
// Note: if input is the final_chunk, we don't do any executing, the chunk just needs to be sinked
|
@@ -101,9 +219,15 @@ OperatorResultType PipelineExecutor::ExecutePushInternal(DataChunk &input, idx_t
|
|
101
219
|
StartOperator(*pipeline.sink);
|
102
220
|
D_ASSERT(pipeline.sink);
|
103
221
|
D_ASSERT(pipeline.sink->sink_state);
|
104
|
-
|
222
|
+
OperatorSinkInput sink_input {*pipeline.sink->sink_state, *local_sink_state, interrupt_state};
|
223
|
+
|
224
|
+
auto sink_result = Sink(sink_chunk, sink_input);
|
225
|
+
|
105
226
|
EndOperator(*pipeline.sink, nullptr);
|
106
|
-
|
227
|
+
|
228
|
+
if (sink_result == SinkResultType::BLOCKED) {
|
229
|
+
return OperatorResultType::BLOCKED;
|
230
|
+
} else if (sink_result == SinkResultType::FINISHED) {
|
107
231
|
FinishProcessing();
|
108
232
|
return OperatorResultType::FINISHED;
|
109
233
|
}
|
@@ -114,49 +238,15 @@ OperatorResultType PipelineExecutor::ExecutePushInternal(DataChunk &input, idx_t
|
|
114
238
|
}
|
115
239
|
}
|
116
240
|
|
117
|
-
// Push all remaining cached operator output through the pipeline
|
118
|
-
void PipelineExecutor::FlushCachingOperatorsPush() {
|
119
|
-
idx_t start_idx = IsFinished() ? idx_t(finished_processing_idx) : 0;
|
120
|
-
for (idx_t op_idx = start_idx; op_idx < pipeline.operators.size(); op_idx++) {
|
121
|
-
if (!pipeline.operators[op_idx].get().RequiresFinalExecute()) {
|
122
|
-
continue;
|
123
|
-
}
|
124
|
-
|
125
|
-
OperatorFinalizeResultType finalize_result;
|
126
|
-
OperatorResultType push_result;
|
127
|
-
|
128
|
-
do {
|
129
|
-
auto &curr_chunk =
|
130
|
-
op_idx + 1 >= intermediate_chunks.size() ? final_chunk : *intermediate_chunks[op_idx + 1];
|
131
|
-
auto ¤t_operator = pipeline.operators[op_idx].get();
|
132
|
-
StartOperator(current_operator);
|
133
|
-
finalize_result = current_operator.FinalExecute(context, curr_chunk, *current_operator.op_state,
|
134
|
-
*intermediate_states[op_idx]);
|
135
|
-
EndOperator(current_operator, &curr_chunk);
|
136
|
-
push_result = ExecutePushInternal(curr_chunk, op_idx + 1);
|
137
|
-
} while (finalize_result != OperatorFinalizeResultType::FINISHED &&
|
138
|
-
push_result != OperatorResultType::FINISHED);
|
139
|
-
|
140
|
-
if (push_result == OperatorResultType::FINISHED) {
|
141
|
-
break;
|
142
|
-
}
|
143
|
-
}
|
144
|
-
}
|
145
|
-
|
146
241
|
void PipelineExecutor::PushFinalize() {
|
147
242
|
if (finalized) {
|
148
243
|
throw InternalException("Calling PushFinalize on a pipeline that has been finalized already");
|
149
244
|
}
|
150
|
-
finalized = true;
|
151
|
-
// flush all caching operators
|
152
|
-
// note that even if an operator has finished, we might still need to flush caches AFTER
|
153
|
-
// that operator e.g. if we have SOURCE -> LIMIT -> CROSS_PRODUCT -> SINK, if the
|
154
|
-
// LIMIT reports no more rows will be passed on we still need to flush caches from the CROSS_PRODUCT
|
155
|
-
D_ASSERT(in_process_operators.empty());
|
156
|
-
|
157
|
-
FlushCachingOperatorsPush();
|
158
245
|
|
159
246
|
D_ASSERT(local_sink_state);
|
247
|
+
|
248
|
+
finalized = true;
|
249
|
+
|
160
250
|
// run the combine for the sink
|
161
251
|
pipeline.sink->Combine(context, *pipeline.sink->sink_state, *local_sink_state);
|
162
252
|
|
@@ -168,6 +258,7 @@ void PipelineExecutor::PushFinalize() {
|
|
168
258
|
local_sink_state.reset();
|
169
259
|
}
|
170
260
|
|
261
|
+
// TODO: Refactoring the StreamingQueryResult to use Push-based execution should eliminate the need for this code
|
171
262
|
void PipelineExecutor::ExecutePull(DataChunk &result) {
|
172
263
|
if (IsFinished()) {
|
173
264
|
return;
|
@@ -176,12 +267,32 @@ void PipelineExecutor::ExecutePull(DataChunk &result) {
|
|
176
267
|
try {
|
177
268
|
D_ASSERT(!pipeline.sink);
|
178
269
|
auto &source_chunk = pipeline.operators.empty() ? result : *intermediate_chunks[0];
|
179
|
-
while (result.size() == 0) {
|
270
|
+
while (result.size() == 0 && !exhausted_source) {
|
180
271
|
if (in_process_operators.empty()) {
|
181
272
|
source_chunk.Reset();
|
182
|
-
|
183
|
-
|
184
|
-
|
273
|
+
|
274
|
+
auto done_signal = make_shared<InterruptDoneSignalState>();
|
275
|
+
interrupt_state = InterruptState(done_signal);
|
276
|
+
SourceResultType source_result;
|
277
|
+
|
278
|
+
// Repeatedly try to fetch from the source until it doesn't block. Note that it may block multiple times
|
279
|
+
while (true) {
|
280
|
+
source_result = FetchFromSource(source_chunk);
|
281
|
+
|
282
|
+
// No interrupt happened, all good.
|
283
|
+
if (source_result != SourceResultType::BLOCKED) {
|
284
|
+
break;
|
285
|
+
}
|
286
|
+
|
287
|
+
// Busy wait for async callback from source operator
|
288
|
+
done_signal->Await();
|
289
|
+
}
|
290
|
+
|
291
|
+
if (source_result == SourceResultType::FINISHED) {
|
292
|
+
exhausted_source = true;
|
293
|
+
if (source_chunk.size() == 0) {
|
294
|
+
break;
|
295
|
+
}
|
185
296
|
}
|
186
297
|
}
|
187
298
|
if (!pipeline.operators.empty()) {
|
@@ -265,7 +376,7 @@ OperatorResultType PipelineExecutor::Execute(DataChunk &input, DataChunk &result
|
|
265
376
|
auto operator_idx = current_idx - 1;
|
266
377
|
auto ¤t_operator = pipeline.operators[operator_idx].get();
|
267
378
|
|
268
|
-
// if current_idx > source_idx, we pass the previous
|
379
|
+
// if current_idx > source_idx, we pass the previous operators' output through the Execute of the current
|
269
380
|
// operator
|
270
381
|
StartOperator(current_operator);
|
271
382
|
auto result = current_operator.Execute(context, prev_chunk, current_chunk, *current_operator.op_state,
|
@@ -307,18 +418,86 @@ OperatorResultType PipelineExecutor::Execute(DataChunk &input, DataChunk &result
|
|
307
418
|
return in_process_operators.empty() ? OperatorResultType::NEED_MORE_INPUT : OperatorResultType::HAVE_MORE_OUTPUT;
|
308
419
|
}
|
309
420
|
|
310
|
-
void PipelineExecutor::
|
421
|
+
void PipelineExecutor::SetTaskForInterrupts(weak_ptr<Task> current_task) {
|
422
|
+
interrupt_state = InterruptState(std::move(current_task));
|
423
|
+
}
|
424
|
+
|
425
|
+
SourceResultType PipelineExecutor::GetData(DataChunk &chunk, OperatorSourceInput &input) {
|
426
|
+
//! Testing feature to enable async source on every operator
|
427
|
+
#ifdef DUCKDB_DEBUG_ASYNC_SINK_SOURCE
|
428
|
+
if (debug_blocked_source_count < debug_blocked_target_count) {
|
429
|
+
debug_blocked_source_count++;
|
430
|
+
|
431
|
+
auto &callback_state = input.interrupt_state;
|
432
|
+
std::thread rewake_thread([callback_state] {
|
433
|
+
std::this_thread::sleep_for(std::chrono::milliseconds(1));
|
434
|
+
callback_state.Callback();
|
435
|
+
});
|
436
|
+
rewake_thread.detach();
|
437
|
+
|
438
|
+
return SourceResultType::BLOCKED;
|
439
|
+
}
|
440
|
+
#endif
|
441
|
+
|
442
|
+
return pipeline.source->GetData(context, chunk, input);
|
443
|
+
}
|
444
|
+
|
445
|
+
SinkResultType PipelineExecutor::Sink(DataChunk &chunk, OperatorSinkInput &input) {
|
446
|
+
//! Testing feature to enable async sink on every operator
|
447
|
+
#ifdef DUCKDB_DEBUG_ASYNC_SINK_SOURCE
|
448
|
+
if (debug_blocked_sink_count < debug_blocked_target_count) {
|
449
|
+
debug_blocked_sink_count++;
|
450
|
+
|
451
|
+
auto &callback_state = input.interrupt_state;
|
452
|
+
std::thread rewake_thread([callback_state] {
|
453
|
+
std::this_thread::sleep_for(std::chrono::milliseconds(1));
|
454
|
+
callback_state.Callback();
|
455
|
+
});
|
456
|
+
rewake_thread.detach();
|
457
|
+
|
458
|
+
return SinkResultType::BLOCKED;
|
459
|
+
}
|
460
|
+
#endif
|
461
|
+
return pipeline.sink->Sink(context, chunk, input);
|
462
|
+
}
|
463
|
+
|
464
|
+
SourceResultType PipelineExecutor::FetchFromSource(DataChunk &result) {
|
311
465
|
StartOperator(*pipeline.source);
|
312
|
-
|
313
|
-
|
314
|
-
|
315
|
-
|
316
|
-
|
317
|
-
|
318
|
-
|
319
|
-
|
466
|
+
|
467
|
+
OperatorSourceInput source_input = {*pipeline.source_state, *local_source_state, interrupt_state};
|
468
|
+
auto res = GetData(result, source_input);
|
469
|
+
|
470
|
+
// Ensures Sinks only return empty results when Blocking or Finished
|
471
|
+
D_ASSERT(res != SourceResultType::BLOCKED || result.size() == 0);
|
472
|
+
|
473
|
+
if (requires_batch_index && res != SourceResultType::BLOCKED) {
|
474
|
+
idx_t next_batch_index;
|
475
|
+
if (result.size() == 0) {
|
476
|
+
next_batch_index = NumericLimits<int64_t>::Maximum();
|
477
|
+
} else {
|
478
|
+
next_batch_index =
|
479
|
+
pipeline.source->GetBatchIndex(context, result, *pipeline.source_state, *local_source_state);
|
480
|
+
next_batch_index += pipeline.base_batch_index;
|
481
|
+
}
|
482
|
+
auto &partition_info = local_sink_state->partition_info;
|
483
|
+
if (next_batch_index != partition_info.batch_index.GetIndex()) {
|
484
|
+
// batch index has changed - update it
|
485
|
+
if (partition_info.batch_index.GetIndex() > next_batch_index) {
|
486
|
+
throw InternalException(
|
487
|
+
"Pipeline batch index - gotten lower batch index %llu (down from previous batch index of %llu)",
|
488
|
+
next_batch_index, partition_info.batch_index.GetIndex());
|
489
|
+
}
|
490
|
+
auto current_batch = partition_info.batch_index.GetIndex();
|
491
|
+
partition_info.batch_index = next_batch_index;
|
492
|
+
// call NextBatch before updating min_batch_index to provide the opportunity to flush the previous batch
|
493
|
+
pipeline.sink->NextBatch(context, *pipeline.sink->sink_state, *local_sink_state);
|
494
|
+
partition_info.min_batch_index = pipeline.UpdateBatchIndex(current_batch, next_batch_index);
|
495
|
+
}
|
320
496
|
}
|
497
|
+
|
321
498
|
EndOperator(*pipeline.source, &result);
|
499
|
+
|
500
|
+
return res;
|
322
501
|
}
|
323
502
|
|
324
503
|
void PipelineExecutor::InitializeChunk(DataChunk &chunk) {
|
@@ -27,7 +27,7 @@ public:
|
|
27
27
|
|
28
28
|
void PipelineInitializeEvent::Schedule() {
|
29
29
|
// needs to spawn a task to get the chain of tasks for the query plan going
|
30
|
-
vector<
|
30
|
+
vector<shared_ptr<Task>> tasks;
|
31
31
|
tasks.push_back(make_uniq<PipelineInitializeTask>(*pipeline, shared_from_this()));
|
32
32
|
SetTasks(std::move(tasks));
|
33
33
|
}
|