From 0f47295c3a643d9d9475bf22c960606eb790c389 Mon Sep 17 00:00:00 2001 From: Laurens Kuiper Date: Thu, 29 Sep 2022 11:23:56 +0200 Subject: [PATCH 01/49] init streaming partitioner --- src/common/types/CMakeLists.txt | 1 + 1 file changed, 1 insertion(+) diff --git a/src/common/types/CMakeLists.txt b/src/common/types/CMakeLists.txt index 34b7acce12fd..f5dc95175834 100644 --- a/src/common/types/CMakeLists.txt +++ b/src/common/types/CMakeLists.txt @@ -13,6 +13,7 @@ add_library_unity( column_data_allocator.cpp column_data_collection.cpp column_data_collection_segment.cpp + streaming_partitioner.cpp data_chunk.cpp date.cpp decimal.cpp From 94dcde8be63227b467b5655d15d5aef525c043f8 Mon Sep 17 00:00:00 2001 From: Laurens Kuiper Date: Thu, 29 Sep 2022 18:00:24 +0200 Subject: [PATCH 02/49] initial implementation of PartitionedColumnData --- src/common/radix_partitioning.cpp | 86 +++++++++++++---- src/common/types/CMakeLists.txt | 2 +- src/common/types/partitioned_column_data.cpp | 82 ++++++++++++++++ src/execution/join_hashtable.cpp | 6 +- .../duckdb/common/radix_partitioning.hpp | 26 ++++- .../common/types/partitioned_column_data.hpp | 94 +++++++++++++++++++ 6 files changed, 268 insertions(+), 28 deletions(-) create mode 100644 src/common/types/partitioned_column_data.cpp create mode 100644 src/include/duckdb/common/types/partitioned_column_data.hpp diff --git a/src/common/radix_partitioning.cpp b/src/common/radix_partitioning.cpp index 87c40107d2f3..32d6b2bb8740 100644 --- a/src/common/radix_partitioning.cpp +++ b/src/common/radix_partitioning.cpp @@ -1,6 +1,7 @@ #include "duckdb/common/radix_partitioning.hpp" #include "duckdb/common/row_operations/row_operations.hpp" +#include "duckdb/common/types/partitioned_column_data.hpp" #include "duckdb/common/types/row_data_collection.hpp" #include "duckdb/common/types/row_layout.hpp" #include "duckdb/common/types/vector.hpp" @@ -95,6 +96,32 @@ RETURN_TYPE DoubleRadixBitsSwitch1(idx_t radix_bits_1, idx_t radix_bits_2, ARGS } } +template +struct RadixLessThan { + static inline bool Operation(hash_t hash, hash_t cutoff) { + using CONSTANTS = RadixPartitioningConstants; + return CONSTANTS::ApplyMask(hash) < cutoff; + } +}; + +struct SelectFunctor { + template + static idx_t Operation(Vector &hashes, const SelectionVector *sel, idx_t count, idx_t cutoff, + SelectionVector *true_sel, SelectionVector *false_sel) { + Vector cutoff_vector(Value::HASH(cutoff)); + return BinaryExecutor::Select>(hashes, cutoff_vector, sel, count, + true_sel, false_sel); + } +}; + +idx_t RadixPartitioning::Select(Vector &hashes, const SelectionVector *sel, idx_t count, idx_t radix_bits, idx_t cutoff, + SelectionVector *true_sel, SelectionVector *false_sel) { + return RadixBitsSwitch(radix_bits, hashes, sel, count, cutoff, true_sel, false_sel); +} + +//===--------------------------------------------------------------------===// +// Row Data Partitioning +//===--------------------------------------------------------------------===// template static void InitPartitions(BufferManager &buffer_manager, vector> &partition_collections, RowDataBlock *partition_blocks[], vector &partition_handles, @@ -369,35 +396,56 @@ struct PartitionFunctor { } }; -void RadixPartitioning::Partition(BufferManager &buffer_manager, const RowLayout &layout, const idx_t hash_offset, - RowDataCollection &block_collection, RowDataCollection &string_heap, - vector> &partition_block_collections, - vector> &partition_string_heaps, idx_t radix_bits) { +void RadixPartitioning::PartitionRowData(BufferManager &buffer_manager, const RowLayout &layout, + const idx_t hash_offset, RowDataCollection &block_collection, + RowDataCollection &string_heap, + vector> &partition_block_collections, + vector> &partition_string_heaps, + idx_t radix_bits) { return RadixBitsSwitch(radix_bits, buffer_manager, layout, hash_offset, block_collection, string_heap, partition_block_collections, partition_string_heaps); } -template -struct RadixLessThan { - static inline bool Operation(hash_t hash, hash_t cutoff) { - using CONSTANTS = RadixPartitioningConstants; - return CONSTANTS::ApplyMask(hash) < cutoff; +//===--------------------------------------------------------------------===// +// Column Data Partitioning +//===--------------------------------------------------------------------===// +RadixPartitionedColumnData::RadixPartitionedColumnData(ClientContext &context_p, vector types_p, + idx_t radix_bits_p, idx_t hash_col_idx_p) + : PartitionedColumnData(context_p, move(types_p)), radix_bits(radix_bits_p), hash_col_idx(hash_col_idx_p) { + D_ASSERT(hash_col_idx < types.size()); + // We know the number of partitions beforehand, so we can just create them + const auto num_partitions = RadixPartitioning::NumberOfPartitions(radix_bits); + partition_allocators.reserve(num_partitions); + for (idx_t i = 0; i < num_partitions; i++) { + partition_allocators.emplace_back(make_shared(BufferManager::GetBufferManager(context))); } -}; +} -struct SelectFunctor { +void RadixPartitionedColumnData::InitializeAppendStateInternal(PartitionedColumnDataAppendState &state) { + // We know the number of partitions beforehand, so we can just initialize them + const auto num_partitions = RadixPartitioning::NumberOfPartitions(radix_bits); + state.partition_buffers.reserve(num_partitions); + state.partitions.reserve(num_partitions); + for (idx_t i = 0; i < num_partitions; i++) { + state.partition_buffers.emplace_back(CreateAppendPartitionBuffer()); + state.partitions.emplace_back(CreateAppendPartition(i)); + } +} + +struct ComputePartitionIndicesFunctor { template - static idx_t Operation(Vector &hashes, const SelectionVector *sel, idx_t count, idx_t cutoff, - SelectionVector *true_sel, SelectionVector *false_sel) { - Vector cutoff_vector(Value::HASH(cutoff)); - return BinaryExecutor::Select>(hashes, cutoff_vector, sel, count, - true_sel, false_sel); + static void Operation(Vector &hashes, Vector &partition_indices, idx_t count) { + UnaryExecutor::Execute(hashes, partition_indices, count, [&](hash_t hash) { + using CONSTANTS = RadixPartitioningConstants; + return CONSTANTS::ApplyMask(hash); + }); } }; -idx_t RadixPartitioning::Select(Vector &hashes, const SelectionVector *sel, idx_t count, idx_t radix_bits, idx_t cutoff, - SelectionVector *true_sel, SelectionVector *false_sel) { - return RadixBitsSwitch(radix_bits, hashes, sel, count, cutoff, true_sel, false_sel); +void RadixPartitionedColumnData::ComputePartitionIndices(PartitionedColumnDataAppendState &state, DataChunk &input) { + + RadixBitsSwitch(radix_bits, input.data[hash_col_idx], state.partition_indices, + input.size()); } } // namespace duckdb diff --git a/src/common/types/CMakeLists.txt b/src/common/types/CMakeLists.txt index f5dc95175834..883424a37edd 100644 --- a/src/common/types/CMakeLists.txt +++ b/src/common/types/CMakeLists.txt @@ -13,7 +13,6 @@ add_library_unity( column_data_allocator.cpp column_data_collection.cpp column_data_collection_segment.cpp - streaming_partitioner.cpp data_chunk.cpp date.cpp decimal.cpp @@ -22,6 +21,7 @@ add_library_unity( uuid.cpp hyperloglog.cpp interval.cpp + partitioned_column_data.cpp row_data_collection.cpp row_data_collection_scanner.cpp row_layout.cpp diff --git a/src/common/types/partitioned_column_data.cpp b/src/common/types/partitioned_column_data.cpp new file mode 100644 index 000000000000..abcf6328d8e3 --- /dev/null +++ b/src/common/types/partitioned_column_data.cpp @@ -0,0 +1,82 @@ +#include "duckdb/common/types/partitioned_column_data.hpp" + +namespace duckdb { + +PartitionedColumnData::PartitionedColumnData(ClientContext &context_p, vector types_p) + : context(context_p), types(move(types_p)) { +} + +PartitionedColumnData::~PartitionedColumnData() { +} + +void PartitionedColumnData::InitializeAppendState(PartitionedColumnDataAppendState &state) { + state.partition_sel.Initialize(); + InitializeAppendStateInternal(state); +} + +void PartitionedColumnData::Append(PartitionedColumnDataAppendState &state, DataChunk &input) { + // Compute partition indices and store them in state.partition_indices + ComputePartitionIndices(state, input); + + // Figure out how many of each partition there are in the input chunk + const auto count = input.size(); + unordered_map partition_counts; + const auto partition_indices = FlatVector::GetData(state.partition_indices); + for (idx_t i = 0; i < count; i++) { + partition_counts[partition_indices[i]]++; + } + + // Now, for each partition, we append to the buffers, and flush the buffers if necessary + for (auto &pc : partition_counts) { + const auto &partition_index = pc.first; + const auto &partition_count = pc.second; + + auto &partition_buffer = *state.partition_buffers[partition_index]; + if (partition_buffer.size() + partition_count > STANDARD_VECTOR_SIZE) { + // Next batch won't fit in the buffer, flush it to the partition + state.partitions[partition_index]->Append(partition_buffer); + partition_buffer.Reset(); + } + + // Create a selection vector for this partition + idx_t sel_idx = 0; + for (idx_t i = 0; i < count; i++) { + if (partition_indices[i] != partition_index) { + continue; + } + state.partition_sel[sel_idx] = i; + sel_idx++; + } + D_ASSERT(sel_idx == partition_count); + + // Append the input chunk to the partition buffer using the selection vector + partition_buffer.Append(input, false, &state.partition_sel, partition_count); + } +} + +void PartitionedColumnData::AppendLocalState(PartitionedColumnDataAppendState &state) { + // Flush any remaining data in the buffers + D_ASSERT(state.partition_buffers.size() == state.partitions.size()); + for (idx_t i = 0; i < state.partitions.size(); i++) { + auto &partition_buffer = *state.partition_buffers[i]; + if (partition_buffer.size() > 0) { + state.partitions[i]->Append(partition_buffer); + } + } + + // Now combine the state's partitions into this + lock_guard guard(lock); + D_ASSERT(state.partitions.size() == NumberOfPartitions()); + if (partitions.empty()) { + // This is the first merge, we just copy them over + partitions = move(state.partitions); + } else { + // Combine the append state's partitions into this PartitionedColumnData + for (idx_t i = 0; i < NumberOfPartitions(); i++) { + partitions[i]->Combine(*state.partitions[i]); + } + // TODO: sort CDC segments or chunks on their block ID, so we don't read, e.g., 1 chunk per buffer block + } +} + +} // namespace duckdb diff --git a/src/execution/join_hashtable.cpp b/src/execution/join_hashtable.cpp index 835701b32742..f729bba2def8 100644 --- a/src/execution/join_hashtable.cpp +++ b/src/execution/join_hashtable.cpp @@ -1064,9 +1064,9 @@ void JoinHashTable::Partition(JoinHashTable &global_ht) { // Swizzle and Partition SwizzleBlocks(); - RadixPartitioning::Partition(global_ht.buffer_manager, global_ht.layout, global_ht.pointer_offset, - *swizzled_block_collection, *swizzled_string_heap, partition_block_collections, - partition_string_heaps, global_ht.radix_bits); + RadixPartitioning::PartitionRowData(global_ht.buffer_manager, global_ht.layout, global_ht.pointer_offset, + *swizzled_block_collection, *swizzled_string_heap, partition_block_collections, + partition_string_heaps, global_ht.radix_bits); // Add to global HT global_ht.Merge(*this); diff --git a/src/include/duckdb/common/radix_partitioning.hpp b/src/include/duckdb/common/radix_partitioning.hpp index 84ad8f85956c..608e6ac2c5a9 100644 --- a/src/include/duckdb/common/radix_partitioning.hpp +++ b/src/include/duckdb/common/radix_partitioning.hpp @@ -9,6 +9,7 @@ #pragma once #include "duckdb/common/fast_mem.hpp" +#include "duckdb/common/types/partitioned_column_data.hpp" namespace duckdb { @@ -45,14 +46,29 @@ struct RadixPartitioning { return (idx_t)1 << radix_bits; } - //! Partition the data in block_collection/string_heap to multiple partitions - static void Partition(BufferManager &buffer_manager, const RowLayout &layout, const idx_t hash_offset, - RowDataCollection &block_collection, RowDataCollection &string_heap, - vector> &partition_block_collections, - vector> &partition_string_heaps, idx_t radix_bits); //! Select using a cutoff on the radix bits of the hash static idx_t Select(Vector &hashes, const SelectionVector *sel, idx_t count, idx_t radix_bits, idx_t cutoff, SelectionVector *true_sel, SelectionVector *false_sel); + + //! Partition the data in block_collection/string_heap to multiple partitions + static void PartitionRowData(BufferManager &buffer_manager, const RowLayout &layout, const idx_t hash_offset, + RowDataCollection &block_collection, RowDataCollection &string_heap, + vector> &partition_block_collections, + vector> &partition_string_heaps, idx_t radix_bits); +}; + +class RadixPartitionedColumnData : public PartitionedColumnData { +public: + RadixPartitionedColumnData(ClientContext &context, vector types, idx_t radix_bits, idx_t hash_col_idx); + + void InitializeAppendStateInternal(PartitionedColumnDataAppendState &state) override; + void ComputePartitionIndices(PartitionedColumnDataAppendState &state, DataChunk &input) override; + +private: + //! The number of radix bits + const idx_t radix_bits; + //! The index of the column holding the hashes + const idx_t hash_col_idx; }; } // namespace duckdb diff --git a/src/include/duckdb/common/types/partitioned_column_data.hpp b/src/include/duckdb/common/types/partitioned_column_data.hpp new file mode 100644 index 000000000000..31b69c88c43a --- /dev/null +++ b/src/include/duckdb/common/types/partitioned_column_data.hpp @@ -0,0 +1,94 @@ +//===----------------------------------------------------------------------===// +// DuckDB +// +// duckdb/common/types/partitioned_column_data.hpp +// +// +//===----------------------------------------------------------------------===// + +#pragma once + +#include "duckdb/common/types/column_data_allocator.hpp" +#include "duckdb/common/types/column_data_collection.hpp" + +namespace duckdb { + +struct PartitionedColumnDataAppendState; +class PartitionedColumnData; + +//! Local state for partioning in parallel +struct PartitionedColumnDataAppendState { + Vector partition_indices = Vector(LogicalType::UBIGINT); + SelectionVector partition_sel; + + vector> partition_buffers; + vector> partitions; +}; + +//! PartitionedColumnData represents partitioned columnar data, which serves as an interface for different flavors of +//! partitioning, e.g., radix, hive +class PartitionedColumnData { +public: + PartitionedColumnData(ClientContext &context, vector types); + ~PartitionedColumnData(); + +public: + //! The types of columns in the PartitionedColumnData + const vector &Types() const { + return types; + } + //! The number of columns in the PartitionedColumnData + idx_t ColumnCount() const { + return types.size(); + } + //! The number of partitions in the PartitionedColumnData + idx_t NumberOfPartitions() const { + return partition_allocators.size(); + } + +public: + //! Initializes a local state for parallel partitioning that can be merged into this PartitionedColumnData + void InitializeAppendState(PartitionedColumnDataAppendState &state); + + //! Appends a DataChunk to the PartitionedColumnDataAppendState + virtual void Append(PartitionedColumnDataAppendState &state, DataChunk &input); + + //! Appends a local state into this PartitionedColumnData + void AppendLocalState(PartitionedColumnDataAppendState &state); + +private: + //===--------------------------------------------------------------------===// + // Partitioning flavor implementation interface + //===--------------------------------------------------------------------===// + //! Initialize a PartitionedColumnDataAppendState for this flavor of partitioning (optional) + virtual void InitializeAppendStateInternal(PartitionedColumnDataAppendState &state) { + } + + //! Compute the partition indices for this flavor of partioning for the input DataChunk and store them in the + //! `partition_data` of the local state. If this flavor creates partitions on the fly (for, e.g., hive), this + //! function is also in charge of creating new partitions and mapping the input data to a partition index + virtual void ComputePartitionIndices(PartitionedColumnDataAppendState &state, DataChunk &input) { + throw NotImplementedException("ComputePartitionIndices for this flavor of PartitionedColumnData"); + } + +protected: + unique_ptr CreateAppendPartition(idx_t partition_index) { + return make_unique(partition_allocators[partition_index], types); + } + + unique_ptr CreateAppendPartitionBuffer() { + auto result = make_unique(); + result->Initialize(Allocator::Get(context), types); + return result; + } + +protected: + ClientContext &context; + vector types; + + vector> partition_allocators; + vector> partitions; + mutex lock; +}; + +} // namespace duckdb From fff7c52ad5017a744a4a6bf41082a851b3473ee0 Mon Sep 17 00:00:00 2001 From: Laurens Kuiper Date: Fri, 30 Sep 2022 13:27:07 +0200 Subject: [PATCH 03/49] lock for shared column data allocator --- src/common/types/column_data_allocator.cpp | 13 +++++++++++-- src/common/types/partitioned_column_data.cpp | 2 +- .../duckdb/common/types/column_data_allocator.hpp | 5 +++++ 3 files changed, 17 insertions(+), 3 deletions(-) diff --git a/src/common/types/column_data_allocator.cpp b/src/common/types/column_data_allocator.cpp index 7cb767ce9951..700cc96e7343 100644 --- a/src/common/types/column_data_allocator.cpp +++ b/src/common/types/column_data_allocator.cpp @@ -41,8 +41,7 @@ void ColumnDataAllocator::AllocateBlock() { blocks.push_back(move(data)); } -void ColumnDataAllocator::AllocateData(idx_t size, uint32_t &block_id, uint32_t &offset, - ChunkManagementState *chunk_state) { +void ColumnDataAllocator::AllocateDataInternal(idx_t size, uint32_t &block_id, uint32_t &offset, ChunkManagementState *chunk_state) { if (type == ColumnDataAllocatorType::IN_MEMORY_ALLOCATOR) { // in-memory allocator auto allocated = alloc.allocator->Allocate(size); @@ -74,6 +73,16 @@ void ColumnDataAllocator::AllocateData(idx_t size, uint32_t &block_id, uint32_t block.size += size; } +void ColumnDataAllocator::AllocateData(idx_t size, uint32_t &block_id, uint32_t &offset, + ChunkManagementState *chunk_state) { + if (shared) { + lock_guard guard(lock); + AllocateDataInternal(size, block_id, offset, chunk_state); + } else { + AllocateDataInternal(size, block_id, offset, chunk_state); + } +} + void ColumnDataAllocator::Initialize(ColumnDataAllocator &other) { D_ASSERT(other.HasBlocks()); blocks.push_back(other.blocks.back()); diff --git a/src/common/types/partitioned_column_data.cpp b/src/common/types/partitioned_column_data.cpp index abcf6328d8e3..debe23d0b7f5 100644 --- a/src/common/types/partitioned_column_data.cpp +++ b/src/common/types/partitioned_column_data.cpp @@ -30,8 +30,8 @@ void PartitionedColumnData::Append(PartitionedColumnDataAppendState &state, Data for (auto &pc : partition_counts) { const auto &partition_index = pc.first; const auto &partition_count = pc.second; - auto &partition_buffer = *state.partition_buffers[partition_index]; + if (partition_buffer.size() + partition_count > STANDARD_VECTOR_SIZE) { // Next batch won't fit in the buffer, flush it to the partition state.partitions[partition_index]->Append(partition_buffer); diff --git a/src/include/duckdb/common/types/column_data_allocator.hpp b/src/include/duckdb/common/types/column_data_allocator.hpp index 7ec708c8f6f9..28fe7e7e4df8 100644 --- a/src/include/duckdb/common/types/column_data_allocator.hpp +++ b/src/include/duckdb/common/types/column_data_allocator.hpp @@ -46,6 +46,7 @@ class ColumnDataAllocator { data_ptr_t GetDataPointer(ChunkManagementState &state, uint32_t block_id, uint32_t offset); private: + void AllocateDataInternal(idx_t size, uint32_t &block_id, uint32_t &offset, ChunkManagementState *chunk_state); void AllocateBlock(); BufferHandle Pin(uint32_t block_id); @@ -65,6 +66,10 @@ class ColumnDataAllocator { vector blocks; //! The set of allocated data vector allocated_data; + //! Whether this ColumnDataAllocator is shared across ColumnDataCollections that allocate in parallel + bool shared; + //! Lock used in case this ColumnDataAllocator is shared across threads + mutex lock; }; } // namespace duckdb From 0e68b4f28a2d7141d3c33126daca076c23103e0c Mon Sep 17 00:00:00 2001 From: Laurens Kuiper Date: Fri, 30 Sep 2022 15:28:03 +0200 Subject: [PATCH 04/49] probe-side partitioning seems to (kind of) work! --- src/common/radix_partitioning.cpp | 4 + src/common/types/column_data_allocator.cpp | 6 +- src/execution/join_hashtable.cpp | 77 ++++++++++++++++-- .../operator/join/physical_hash_join.cpp | 79 +++++++++---------- .../duckdb/common/radix_partitioning.hpp | 1 + .../common/types/column_data_allocator.hpp | 5 +- .../common/types/partitioned_column_data.hpp | 8 +- .../duckdb/execution/join_hashtable.hpp | 36 ++++++++- 8 files changed, 162 insertions(+), 54 deletions(-) diff --git a/src/common/radix_partitioning.cpp b/src/common/radix_partitioning.cpp index 32d6b2bb8740..2be009daa16b 100644 --- a/src/common/radix_partitioning.cpp +++ b/src/common/radix_partitioning.cpp @@ -418,9 +418,13 @@ RadixPartitionedColumnData::RadixPartitionedColumnData(ClientContext &context_p, partition_allocators.reserve(num_partitions); for (idx_t i = 0; i < num_partitions; i++) { partition_allocators.emplace_back(make_shared(BufferManager::GetBufferManager(context))); + partition_allocators.back()->MakeShared(); } } +RadixPartitionedColumnData::~RadixPartitionedColumnData() { +} + void RadixPartitionedColumnData::InitializeAppendStateInternal(PartitionedColumnDataAppendState &state) { // We know the number of partitions beforehand, so we can just initialize them const auto num_partitions = RadixPartitioning::NumberOfPartitions(radix_bits); diff --git a/src/common/types/column_data_allocator.cpp b/src/common/types/column_data_allocator.cpp index 700cc96e7343..57e25bc2d17e 100644 --- a/src/common/types/column_data_allocator.cpp +++ b/src/common/types/column_data_allocator.cpp @@ -1,6 +1,7 @@ #include "duckdb/common/types/column_data_allocator.hpp" -#include "duckdb/storage/buffer_manager.hpp" + #include "duckdb/common/types/column_data_collection_segment.hpp" +#include "duckdb/storage/buffer_manager.hpp" namespace duckdb { @@ -41,7 +42,8 @@ void ColumnDataAllocator::AllocateBlock() { blocks.push_back(move(data)); } -void ColumnDataAllocator::AllocateDataInternal(idx_t size, uint32_t &block_id, uint32_t &offset, ChunkManagementState *chunk_state) { +void ColumnDataAllocator::AllocateDataInternal(idx_t size, uint32_t &block_id, uint32_t &offset, + ChunkManagementState *chunk_state) { if (type == ColumnDataAllocatorType::IN_MEMORY_ALLOCATOR) { // in-memory allocator auto allocated = alloc.allocator->Allocate(size); diff --git a/src/execution/join_hashtable.cpp b/src/execution/join_hashtable.cpp index f729bba2def8..dacaea486dea 100644 --- a/src/execution/join_hashtable.cpp +++ b/src/execution/join_hashtable.cpp @@ -12,6 +12,7 @@ namespace duckdb { using ValidityBytes = JoinHashTable::ValidityBytes; using ScanStructure = JoinHashTable::ScanStructure; +using ProbeSpill = JoinHashTable::ProbeSpill; JoinHashTable::JoinHashTable(BufferManager &buffer_manager, const vector &conditions, vector btypes, JoinType type) @@ -1144,10 +1145,8 @@ static void CreateSpillChunk(DataChunk &spill_chunk, DataChunk &keys, DataChunk spill_chunk.data[spill_col_idx].Reference(hashes); } -unique_ptr JoinHashTable::ProbeAndSpill(DataChunk &keys, DataChunk &payload, - ColumnDataCollection &spill_collection, - ColumnDataAppendState &spill_append_state, - DataChunk &spill_chunk) { +unique_ptr JoinHashTable::ProbeAndSpill(DataChunk &keys, DataChunk &payload, ProbeSpill &probe_spill, + idx_t thread_idx, DataChunk &spill_chunk) { // hash all the keys Vector hashes(LogicalType::HASH); Hash(keys, *FlatVector::IncrementalSelectionVector(), keys.size(), hashes); @@ -1165,7 +1164,7 @@ unique_ptr JoinHashTable::ProbeAndSpill(DataChunk &keys, DataChun CreateSpillChunk(spill_chunk, keys, payload, hashes); spill_chunk.Slice(false_sel, false_count); spill_chunk.Verify(); - spill_collection.Append(spill_append_state, spill_chunk); + probe_spill.Append(spill_chunk, thread_idx); // slice the stuff we CAN probe right now hashes.Slice(true_sel, true_count); @@ -1187,4 +1186,72 @@ unique_ptr JoinHashTable::ProbeAndSpill(DataChunk &keys, DataChun return ss; } +ProbeSpill::ProbeSpill(JoinHashTable &ht, ClientContext &context, const vector &probe_types) + : context(context), probe_types(probe_types) { + if (ht.total_count - ht.Count() <= ht.tuples_per_round) { + // No need to partition as we will only have one more probe round + partitioned = false; + } else { + // More than one probe round to go, so we need to partition + partitioned = true; + partitioned_data = + make_unique(context, probe_types, ht.radix_bits, probe_types.size() - 1); + } +} + +idx_t ProbeSpill::RegisterThread() { + lock_guard guard(lock); + if (partitioned) { + idx_t thread_idx = partition_append_states.size(); + partition_append_states.emplace_back(); + partitioned_data->InitializeAppendState(partition_append_states.back()); + return thread_idx; + } else { + idx_t thread_idx = local_spill_collections.size(); + local_spill_collections.emplace_back(make_unique(context, probe_types)); + spill_append_states.emplace_back(); + local_spill_collections.back()->InitializeAppend(spill_append_states.back()); + return thread_idx; + } +} + +void ProbeSpill::Append(DataChunk &chunk, idx_t thread_idx) { + if (partitioned) { + partitioned_data->Append(partition_append_states[thread_idx], chunk); + } else { + local_spill_collections[thread_idx]->Append(spill_append_states[thread_idx], chunk); + } +} + +void ProbeSpill::Finalize() { + if (partitioned) { + for (auto &append_state : partition_append_states) { + partitioned_data->AppendLocalState(append_state); + } + partition_append_states.clear(); + } else { + global_spill_collection = move(local_spill_collections[0]); + for (idx_t i = 1; i < local_spill_collections.size(); i++) { + global_spill_collection->Combine(*local_spill_collections[i]); + } + local_spill_collections.clear(); + spill_append_states.clear(); + } +} + +unique_ptr ProbeSpill::GetNextProbeCollection(JoinHashTable &ht) { + if (partitioned) { + auto &partitions = partitioned_data->GetPartitions(); + auto merged_partitions = move(partitions[ht.partition_start]); + for (idx_t p_idx = ht.partition_start + 1; p_idx < ht.partition_end; p_idx++) { + auto &partition = partitions[p_idx]; + merged_partitions->Combine(*partition); + partition.reset(); + } + return merged_partitions; + } else { + return move(global_spill_collection); + } +} + } // namespace duckdb diff --git a/src/execution/operator/join/physical_hash_join.cpp b/src/execution/operator/join/physical_hash_join.cpp index d4998ba95edd..8fe27adb39eb 100644 --- a/src/execution/operator/join/physical_hash_join.cpp +++ b/src/execution/operator/join/physical_hash_join.cpp @@ -4,6 +4,7 @@ #include "duckdb/common/vector_operations/vector_operations.hpp" #include "duckdb/execution/expression_executor.hpp" #include "duckdb/function/aggregate/distributive_functions.hpp" +#include "duckdb/function/function_binder.hpp" #include "duckdb/main/client_context.hpp" #include "duckdb/main/query_profiler.hpp" #include "duckdb/parallel/base_pipeline_event.hpp" @@ -11,7 +12,6 @@ #include "duckdb/parallel/thread_context.hpp" #include "duckdb/storage/buffer_manager.hpp" #include "duckdb/storage/storage_manager.hpp" -#include "duckdb/function/function_binder.hpp" namespace duckdb { @@ -72,6 +72,7 @@ class HashJoinGlobalSinkState : public GlobalSinkState { } void ScheduleFinalize(Pipeline &pipeline, Event &event); + void InitializeProbeSpill(ClientContext &context); public: //! Global HT used by the join @@ -93,7 +94,7 @@ class HashJoinGlobalSinkState : public GlobalSinkState { //! Excess probe data gathered during Sink vector probe_types; - vector> spill_collections; + unique_ptr probe_spill; //! Whether or not we have started scanning data using GetData atomic scanned_data; @@ -312,6 +313,13 @@ void HashJoinGlobalSinkState::ScheduleFinalize(Pipeline &pipeline, Event &event) event.InsertEvent(move(new_event)); } +void HashJoinGlobalSinkState::InitializeProbeSpill(ClientContext &context) { + lock_guard guard(lock); + if (!probe_spill) { + probe_spill = make_unique(*hash_table, context, probe_types); + } +} + class HashJoinPartitionTask : public ExecutorTask { public: HashJoinPartitionTask(shared_ptr event_p, ClientContext &context, JoinHashTable &global_ht, @@ -405,7 +413,8 @@ SinkFinalizeType PhysicalHashJoin::Finalize(Pipeline &pipeline, Event &event, Cl //===--------------------------------------------------------------------===// class HashJoinOperatorState : public OperatorState { public: - explicit HashJoinOperatorState(Allocator &allocator) : probe_executor(allocator), spill_collection(nullptr) { + explicit HashJoinOperatorState(Allocator &allocator) + : probe_executor(allocator), thread_idx(DConstants::INVALID_INDEX) { } DataChunk join_keys; @@ -413,9 +422,8 @@ class HashJoinOperatorState : public OperatorState { unique_ptr scan_structure; unique_ptr perfect_hash_join_state; - //! Collection and chunk to sink data into for external join - ColumnDataCollection *spill_collection; - ColumnDataAppendState spill_append_state; + //! Chunk to sink data into for external join + idx_t thread_idx; DataChunk spill_chunk; public: @@ -438,11 +446,6 @@ unique_ptr PhysicalHashJoin::GetOperatorState(ExecutionContext &c } if (sink.external) { state->spill_chunk.Initialize(allocator, sink.probe_types); - lock_guard local_ht_lock(sink.lock); - sink.spill_collections.push_back( - make_unique(BufferManager::GetBufferManager(context.client), sink.probe_types)); - state->spill_collection = sink.spill_collections.back().get(); - state->spill_collection->InitializeAppend(state->spill_append_state); } return move(state); @@ -480,14 +483,26 @@ OperatorResultType PhysicalHashJoin::Execute(ExecutionContext &context, DataChun return OperatorResultType::NEED_MORE_INPUT; } + // some initialization for external hash join + if (sink.external) { + if (!sink.probe_spill) { + // initialize probe spill if not yet done + sink.InitializeProbeSpill(context.client); + } + if (state.thread_idx == DConstants::INVALID_INDEX) { + // assign thread index + state.thread_idx = sink.probe_spill->RegisterThread(); + } + } + // resolve the join keys for the left chunk state.join_keys.Reset(); state.probe_executor.Execute(input, state.join_keys); // perform the actual probe if (sink.external) { - state.scan_structure = sink.hash_table->ProbeAndSpill(state.join_keys, input, *state.spill_collection, - state.spill_append_state, state.spill_chunk); + state.scan_structure = sink.hash_table->ProbeAndSpill(state.join_keys, input, *sink.probe_spill, + state.thread_idx, state.spill_chunk); } else { state.scan_structure = sink.hash_table->Probe(state.join_keys); } @@ -508,8 +523,6 @@ class HashJoinGlobalSourceState : public GlobalSourceState { //! Initialize this source state using the info in the sink void Initialize(HashJoinGlobalSinkState &sink); - //! Partition the probe-side data - void PartitionProbeSide(HashJoinGlobalSinkState &sink); //! Prepare the next build/probe stage for external hash join (must hold lock) void PrepareBuild(HashJoinGlobalSinkState &sink); void PrepareProbe(HashJoinGlobalSinkState &sink); @@ -542,7 +555,6 @@ class HashJoinGlobalSourceState : public GlobalSourceState { ColumnDataParallelScanState probe_global_scan; idx_t probe_chunk_count; idx_t probe_chunk_done; - atomic probe_side_partitioned; //! For full/outer synchronization JoinHTScanState full_outer_scan; @@ -610,7 +622,7 @@ unique_ptr PhysicalHashJoin::GetLocalSourceState(ExecutionCont HashJoinGlobalSourceState::HashJoinGlobalSourceState(const PhysicalHashJoin &op, ClientContext &context) : join_type(op.join_type), initialized(false), global_stage(HashJoinSourceStage::INIT), - lock(probe_global_scan.lock), probe_side_partitioned(false), probe_count(op.children[0]->estimated_cardinality), + lock(probe_global_scan.lock), probe_count(op.children[0]->estimated_cardinality), parallel_scan_chunk_count(context.config.verify_parallelism ? 1 : 120) { } @@ -629,31 +641,10 @@ void HashJoinGlobalSourceState::Initialize(HashJoinGlobalSinkState &sink) { build_blocks_per_thread = MaxValue(idx_t(parallel_scan_chunk_count * STANDARD_VECTOR_SIZE) / block_capacity, 1); - initialized = true; -} + // Finalize the probe spill too + sink.probe_spill->Finalize(); -void HashJoinGlobalSourceState::PartitionProbeSide(HashJoinGlobalSinkState &sink) { - if (probe_side_partitioned) { - return; - } - lock_guard guard(lock); - if (probe_side_partitioned) { - return; - } - - // For now we actually don't partition the probe side TODO - for (auto &spill_collection : sink.spill_collections) { - if (!probe_collection) { - probe_collection = move(spill_collection); - } else { - probe_collection->Combine(*spill_collection); - } - } - sink.spill_collections.clear(); - - probe_chunk_count = probe_collection->ChunkCount(); - - probe_side_partitioned = true; + initialized = true; } void HashJoinGlobalSourceState::PrepareBuild(HashJoinGlobalSinkState &sink) { @@ -676,7 +667,9 @@ void HashJoinGlobalSourceState::PrepareBuild(HashJoinGlobalSinkState &sink) { } void HashJoinGlobalSourceState::PrepareProbe(HashJoinGlobalSinkState &sink) { + probe_collection = sink.probe_spill->GetNextProbeCollection(*sink.hash_table); probe_collection->InitializeScan(probe_global_scan); + probe_chunk_count = probe_collection->ChunkCount(); probe_chunk_done = 0; if (IsRightOuterJoin(join_type)) { @@ -867,10 +860,12 @@ void PhysicalHashJoin::GetData(ExecutionContext &context, DataChunk &chunk, Glob return; } D_ASSERT(can_go_external); + // TODO: scan and consume CDC's + // String handling in CDC's + // sorting CDC blocks by block ID if (gstate.global_stage == HashJoinSourceStage::INIT) { gstate.Initialize(sink); - gstate.PartitionProbeSide(sink); lock_guard lock(gstate.lock); if (gstate.global_stage == HashJoinSourceStage::INIT) { diff --git a/src/include/duckdb/common/radix_partitioning.hpp b/src/include/duckdb/common/radix_partitioning.hpp index 608e6ac2c5a9..4adaa02cc7c8 100644 --- a/src/include/duckdb/common/radix_partitioning.hpp +++ b/src/include/duckdb/common/radix_partitioning.hpp @@ -60,6 +60,7 @@ struct RadixPartitioning { class RadixPartitionedColumnData : public PartitionedColumnData { public: RadixPartitionedColumnData(ClientContext &context, vector types, idx_t radix_bits, idx_t hash_col_idx); + ~RadixPartitionedColumnData() override; void InitializeAppendStateInternal(PartitionedColumnDataAppendState &state) override; void ComputePartitionIndices(PartitionedColumnDataAppendState &state, DataChunk &input) override; diff --git a/src/include/duckdb/common/types/column_data_allocator.hpp b/src/include/duckdb/common/types/column_data_allocator.hpp index 28fe7e7e4df8..6a67e846350e 100644 --- a/src/include/duckdb/common/types/column_data_allocator.hpp +++ b/src/include/duckdb/common/types/column_data_allocator.hpp @@ -37,6 +37,9 @@ class ColumnDataAllocator { ColumnDataAllocatorType GetType() { return type; } + void MakeShared() { + shared = true; + } public: void AllocateData(idx_t size, uint32_t &block_id, uint32_t &offset, ChunkManagementState *chunk_state); @@ -67,7 +70,7 @@ class ColumnDataAllocator { //! The set of allocated data vector allocated_data; //! Whether this ColumnDataAllocator is shared across ColumnDataCollections that allocate in parallel - bool shared; + bool shared = false; //! Lock used in case this ColumnDataAllocator is shared across threads mutex lock; }; diff --git a/src/include/duckdb/common/types/partitioned_column_data.hpp b/src/include/duckdb/common/types/partitioned_column_data.hpp index 31b69c88c43a..0e1234dd56b4 100644 --- a/src/include/duckdb/common/types/partitioned_column_data.hpp +++ b/src/include/duckdb/common/types/partitioned_column_data.hpp @@ -30,7 +30,7 @@ struct PartitionedColumnDataAppendState { class PartitionedColumnData { public: PartitionedColumnData(ClientContext &context, vector types); - ~PartitionedColumnData(); + virtual ~PartitionedColumnData(); public: //! The types of columns in the PartitionedColumnData @@ -45,6 +45,10 @@ class PartitionedColumnData { idx_t NumberOfPartitions() const { return partition_allocators.size(); } + //! The partitions in this PartitionedColumnData + vector> &GetPartitions() { + return partitions; + } public: //! Initializes a local state for parallel partitioning that can be merged into this PartitionedColumnData @@ -53,7 +57,7 @@ class PartitionedColumnData { //! Appends a DataChunk to the PartitionedColumnDataAppendState virtual void Append(PartitionedColumnDataAppendState &state, DataChunk &input); - //! Appends a local state into this PartitionedColumnData + //! Appends a local state into this PartitionedColumnData TODO: rename this void AppendLocalState(PartitionedColumnDataAppendState &state); private: diff --git a/src/include/duckdb/execution/join_hashtable.hpp b/src/include/duckdb/execution/join_hashtable.hpp index 221c32c85bc7..70072cd44130 100644 --- a/src/include/duckdb/execution/join_hashtable.hpp +++ b/src/include/duckdb/execution/join_hashtable.hpp @@ -241,6 +241,38 @@ class JoinHashTable { //===--------------------------------------------------------------------===// // External Join //===--------------------------------------------------------------------===// + struct ProbeSpill { + public: + ProbeSpill(JoinHashTable &ht, ClientContext &context, const vector &probe_types); + + //! Create an append state for a new thread and assign an index + idx_t RegisterThread(); + //! Append a chunk to this ProbeSpill + void Append(DataChunk &chunk, idx_t thread_idx); + //! Finalize by merging the thread-local accumulated data + void Finalize(); + + //! Get the probe collection for the next probe round + unique_ptr GetNextProbeCollection(JoinHashTable &ht); + + private: + mutex lock; + ClientContext &context; + + //! The types of the probe DataChunks + const vector &probe_types; + //! Whether the probe data is partitioned + bool partitioned; + + //! The partitioned probe data (if partitioned) and append states + unique_ptr partitioned_data; + vector partition_append_states; + //! The probe data (if not partitioned) and append states + unique_ptr global_spill_collection; + vector> local_spill_collections; + vector spill_append_states; + }; + //! Whether we are doing an external hash join bool external; //! The current number of radix bits used to partition @@ -281,8 +313,8 @@ class JoinHashTable { //! Build HT for the next partitioned probe round bool PrepareExternalFinalize(); //! Probe whatever we can, sink the rest into a thread-local HT - unique_ptr ProbeAndSpill(DataChunk &keys, DataChunk &payload, ColumnDataCollection &spill_collection, - ColumnDataAppendState &spill_append_state, DataChunk &spill_chunk); + unique_ptr ProbeAndSpill(DataChunk &keys, DataChunk &payload, ProbeSpill &probe_spill, + idx_t thread_idx, DataChunk &spill_chunk); private: //! First and last partition of the current partitioned round From b041d835284a31fdd0d50418ae5785804ad12cb1 Mon Sep 17 00:00:00 2001 From: Laurens Kuiper Date: Mon, 3 Oct 2022 10:09:17 +0200 Subject: [PATCH 05/49] probe-side partitioning unit tests passing! --- src/common/radix_partitioning.cpp | 3 ++- src/common/types/partitioned_column_data.cpp | 6 ++++++ src/execution/join_hashtable.cpp | 18 ++++++++++++------ .../operator/join/physical_hash_join.cpp | 12 ++++++++++++ .../common/types/partitioned_column_data.hpp | 8 +++++++- .../duckdb/execution/join_hashtable.hpp | 2 +- 6 files changed, 40 insertions(+), 9 deletions(-) diff --git a/src/common/radix_partitioning.cpp b/src/common/radix_partitioning.cpp index 2be009daa16b..0ad38180ffb6 100644 --- a/src/common/radix_partitioning.cpp +++ b/src/common/radix_partitioning.cpp @@ -447,7 +447,8 @@ struct ComputePartitionIndicesFunctor { }; void RadixPartitionedColumnData::ComputePartitionIndices(PartitionedColumnDataAppendState &state, DataChunk &input) { - + D_ASSERT(state.partitions.size() == RadixPartitioning::NumberOfPartitions(radix_bits)); + D_ASSERT(state.partition_buffers.size() == RadixPartitioning::NumberOfPartitions(radix_bits)); RadixBitsSwitch(radix_bits, input.data[hash_col_idx], state.partition_indices, input.size()); } diff --git a/src/common/types/partitioned_column_data.cpp b/src/common/types/partitioned_column_data.cpp index debe23d0b7f5..5a91a4ac8f18 100644 --- a/src/common/types/partitioned_column_data.cpp +++ b/src/common/types/partitioned_column_data.cpp @@ -38,6 +38,12 @@ void PartitionedColumnData::Append(PartitionedColumnDataAppendState &state, Data partition_buffer.Reset(); } + if (partition_count == input.size()) { + // Whole chunk is a single partition + partition_buffer.Append(input); + return; + } + // Create a selection vector for this partition idx_t sel_idx = 0; for (idx_t i = 0; i < count; i++) { diff --git a/src/execution/join_hashtable.cpp b/src/execution/join_hashtable.cpp index dacaea486dea..71fbc577ff23 100644 --- a/src/execution/join_hashtable.cpp +++ b/src/execution/join_hashtable.cpp @@ -1160,8 +1160,9 @@ unique_ptr JoinHashTable::ProbeAndSpill(DataChunk &keys, DataChun radix_bits, partition_end, &true_sel, &false_sel); auto false_count = keys.size() - true_count; - // slice the stuff we CAN'T probe right now and append to spill collection CreateSpillChunk(spill_chunk, keys, payload, hashes); + + // can't probe these values right now, append to spill spill_chunk.Slice(false_sel, false_count); spill_chunk.Verify(); probe_spill.Append(spill_chunk, thread_idx); @@ -1203,8 +1204,8 @@ idx_t ProbeSpill::RegisterThread() { lock_guard guard(lock); if (partitioned) { idx_t thread_idx = partition_append_states.size(); - partition_append_states.emplace_back(); - partitioned_data->InitializeAppendState(partition_append_states.back()); + partition_append_states.emplace_back(make_unique()); + partitioned_data->InitializeAppendState(*partition_append_states.back()); return thread_idx; } else { idx_t thread_idx = local_spill_collections.size(); @@ -1217,7 +1218,7 @@ idx_t ProbeSpill::RegisterThread() { void ProbeSpill::Append(DataChunk &chunk, idx_t thread_idx) { if (partitioned) { - partitioned_data->Append(partition_append_states[thread_idx], chunk); + partitioned_data->Append(*partition_append_states[thread_idx], chunk); } else { local_spill_collections[thread_idx]->Append(spill_append_states[thread_idx], chunk); } @@ -1226,7 +1227,7 @@ void ProbeSpill::Append(DataChunk &chunk, idx_t thread_idx) { void ProbeSpill::Finalize() { if (partitioned) { for (auto &append_state : partition_append_states) { - partitioned_data->AppendLocalState(append_state); + partitioned_data->AppendLocalState(*append_state); } partition_append_states.clear(); } else { @@ -1242,6 +1243,9 @@ void ProbeSpill::Finalize() { unique_ptr ProbeSpill::GetNextProbeCollection(JoinHashTable &ht) { if (partitioned) { auto &partitions = partitioned_data->GetPartitions(); + if (ht.partition_start == partitions.size()) { + return nullptr; + } auto merged_partitions = move(partitions[ht.partition_start]); for (idx_t p_idx = ht.partition_start + 1; p_idx < ht.partition_end; p_idx++) { auto &partition = partitions[p_idx]; @@ -1250,7 +1254,9 @@ unique_ptr ProbeSpill::GetNextProbeCollection(JoinHashTabl } return merged_partitions; } else { - return move(global_spill_collection); + auto result = move(global_spill_collection); + global_spill_collection = nullptr; + return result; } } diff --git a/src/execution/operator/join/physical_hash_join.cpp b/src/execution/operator/join/physical_hash_join.cpp index 8fe27adb39eb..c063f7402f32 100644 --- a/src/execution/operator/join/physical_hash_join.cpp +++ b/src/execution/operator/join/physical_hash_join.cpp @@ -668,6 +668,8 @@ void HashJoinGlobalSourceState::PrepareBuild(HashJoinGlobalSinkState &sink) { void HashJoinGlobalSourceState::PrepareProbe(HashJoinGlobalSinkState &sink) { probe_collection = sink.probe_spill->GetNextProbeCollection(*sink.hash_table); + D_ASSERT(probe_collection); + probe_collection->InitializeScan(probe_global_scan); probe_chunk_count = probe_collection->ChunkCount(); probe_chunk_done = 0; @@ -678,6 +680,16 @@ void HashJoinGlobalSourceState::PrepareProbe(HashJoinGlobalSinkState &sink) { } global_stage = HashJoinSourceStage::PROBE; + if (probe_chunk_count > 0) { + return; + } + + // Empty probe collection, go straight into the next stage + if (IsRightOuterJoin(join_type)) { + global_stage = HashJoinSourceStage::SCAN_HT; + } else { + PrepareBuild(sink); + } } bool HashJoinGlobalSourceState::AssignTask(HashJoinGlobalSinkState &sink, HashJoinLocalSourceState &lstate) { diff --git a/src/include/duckdb/common/types/partitioned_column_data.hpp b/src/include/duckdb/common/types/partitioned_column_data.hpp index 0e1234dd56b4..58c069101b16 100644 --- a/src/include/duckdb/common/types/partitioned_column_data.hpp +++ b/src/include/duckdb/common/types/partitioned_column_data.hpp @@ -18,7 +18,13 @@ class PartitionedColumnData; //! Local state for partioning in parallel struct PartitionedColumnDataAppendState { - Vector partition_indices = Vector(LogicalType::UBIGINT); + explicit PartitionedColumnDataAppendState() : partition_indices(LogicalType::UBIGINT) { + } + + // Implicit copying is not allowed + PartitionedColumnDataAppendState(const PartitionedColumnDataAppendState &) = delete; + + Vector partition_indices; SelectionVector partition_sel; vector> partition_buffers; diff --git a/src/include/duckdb/execution/join_hashtable.hpp b/src/include/duckdb/execution/join_hashtable.hpp index 70072cd44130..04a9184ec47e 100644 --- a/src/include/duckdb/execution/join_hashtable.hpp +++ b/src/include/duckdb/execution/join_hashtable.hpp @@ -266,7 +266,7 @@ class JoinHashTable { //! The partitioned probe data (if partitioned) and append states unique_ptr partitioned_data; - vector partition_append_states; + vector> partition_append_states; //! The probe data (if not partitioned) and append states unique_ptr global_spill_collection; vector> local_spill_collections; From 7354665c47988704d5652f04374eaac041359a30 Mon Sep 17 00:00:00 2001 From: Laurens Kuiper Date: Mon, 3 Oct 2022 12:51:00 +0200 Subject: [PATCH 06/49] chunk partitioning performance improvements --- src/common/types/column_data_allocator.cpp | 11 ++- src/common/types/partitioned_column_data.cpp | 77 ++++++++++++------- src/execution/join_hashtable.cpp | 10 +-- .../common/types/column_data_allocator.hpp | 1 + .../common/types/partitioned_column_data.hpp | 15 ++-- .../duckdb/common/types/selection_vector.hpp | 4 +- .../duckdb/execution/join_hashtable.hpp | 2 +- 7 files changed, 77 insertions(+), 43 deletions(-) diff --git a/src/common/types/column_data_allocator.cpp b/src/common/types/column_data_allocator.cpp index 57e25bc2d17e..01e687365ef8 100644 --- a/src/common/types/column_data_allocator.cpp +++ b/src/common/types/column_data_allocator.cpp @@ -112,7 +112,7 @@ Allocator &ColumnDataAllocator::GetAllocator() { : alloc.buffer_manager->GetBufferAllocator(); } -void ColumnDataAllocator::InitializeChunkState(ChunkManagementState &state, ChunkMetaData &chunk) { +void ColumnDataAllocator::InitializeChunkStateInternal(ChunkManagementState &state, ChunkMetaData &chunk) { if (type != ColumnDataAllocatorType::BUFFER_MANAGER_ALLOCATOR) { // nothing to pin return; @@ -142,6 +142,15 @@ void ColumnDataAllocator::InitializeChunkState(ChunkManagementState &state, Chun } } +void ColumnDataAllocator::InitializeChunkState(ChunkManagementState &state, ChunkMetaData &chunk) { + if (shared) { + lock_guard guard(lock); + InitializeChunkStateInternal(state, chunk); + } else { + InitializeChunkStateInternal(state, chunk); + } +} + uint32_t BlockMetaData::Capacity() { D_ASSERT(size <= capacity); return capacity - size; diff --git a/src/common/types/partitioned_column_data.cpp b/src/common/types/partitioned_column_data.cpp index 5a91a4ac8f18..6cc2af3aef7f 100644 --- a/src/common/types/partitioned_column_data.cpp +++ b/src/common/types/partitioned_column_data.cpp @@ -14,53 +14,74 @@ void PartitionedColumnData::InitializeAppendState(PartitionedColumnDataAppendSta InitializeAppendStateInternal(state); } -void PartitionedColumnData::Append(PartitionedColumnDataAppendState &state, DataChunk &input) { +void PartitionedColumnData::AppendChunk(PartitionedColumnDataAppendState &state, DataChunk &input) { // Compute partition indices and store them in state.partition_indices ComputePartitionIndices(state, input); - // Figure out how many of each partition there are in the input chunk + // Compute the counts per partition const auto count = input.size(); - unordered_map partition_counts; + map partition_entries; const auto partition_indices = FlatVector::GetData(state.partition_indices); for (idx_t i = 0; i < count; i++) { - partition_counts[partition_indices[i]]++; + const auto &partition_index = partition_indices[i]; + auto partition_entry = partition_entries.find(partition_index); + if (partition_entry == partition_entries.end()) { + partition_entries[partition_index] = list_entry_t(0, 1); + } else { + partition_entry->second.length++; + } + } + + // Early out: check if everything belongs to a single partition + if (partition_entries.size() == 1) { + auto partition_entry = *partition_entries.begin(); + state.partition_buffers[partition_entry.first]->Append(input); + return; + } + + // Compute offsets from the counts + idx_t offset = 0; + for (auto &pc : partition_entries) { + auto &partition_entry = pc.second; + partition_entry.offset = offset; + offset += partition_entry.length; + } + + // Now initialize a single selection vector that acts as a selection vector for every partition + auto &all_partitions_sel = state.partition_sel; + for (idx_t i = 0; i < count; i++) { + const auto &partition_index = partition_indices[i]; + auto &partition_offset = partition_entries[partition_index].offset; + all_partitions_sel[partition_offset++] = i; } - // Now, for each partition, we append to the buffers, and flush the buffers if necessary - for (auto &pc : partition_counts) { + // Loop through the partitions to append the new data to the partition buffers, and flush the buffers if necessary + SelectionVector partition_sel; + for (auto &pc : partition_entries) { const auto &partition_index = pc.first; - const auto &partition_count = pc.second; - auto &partition_buffer = *state.partition_buffers[partition_index]; + const auto &partition_entry = pc.second; - if (partition_buffer.size() + partition_count > STANDARD_VECTOR_SIZE) { + // Length and offset into the selection vector for this chunk, for this partition + const auto &partition_length = partition_entry.length; + const auto partition_offset = partition_entry.offset - partition_length; + + auto &partition_buffer = *state.partition_buffers[partition_index]; + if (partition_buffer.size() + partition_length > STANDARD_VECTOR_SIZE) { // Next batch won't fit in the buffer, flush it to the partition - state.partitions[partition_index]->Append(partition_buffer); + auto &partition = *state.partitions[partition_index]; + partition.Append(partition_buffer); partition_buffer.Reset(); } - if (partition_count == input.size()) { - // Whole chunk is a single partition - partition_buffer.Append(input); - return; - } - - // Create a selection vector for this partition - idx_t sel_idx = 0; - for (idx_t i = 0; i < count; i++) { - if (partition_indices[i] != partition_index) { - continue; - } - state.partition_sel[sel_idx] = i; - sel_idx++; - } - D_ASSERT(sel_idx == partition_count); + // Create a selection vector for this partition using the offset into the single selection vector + partition_sel.Initialize(all_partitions_sel, partition_offset); // Append the input chunk to the partition buffer using the selection vector - partition_buffer.Append(input, false, &state.partition_sel, partition_count); + partition_buffer.Append(input, false, &partition_sel, partition_length); } } -void PartitionedColumnData::AppendLocalState(PartitionedColumnDataAppendState &state) { +void PartitionedColumnData::CombineLocalState(PartitionedColumnDataAppendState &state) { // Flush any remaining data in the buffers D_ASSERT(state.partition_buffers.size() == state.partitions.size()); for (idx_t i = 0; i < state.partitions.size(); i++) { diff --git a/src/execution/join_hashtable.cpp b/src/execution/join_hashtable.cpp index 71fbc577ff23..ea97ce323796 100644 --- a/src/execution/join_hashtable.cpp +++ b/src/execution/join_hashtable.cpp @@ -1210,24 +1210,24 @@ idx_t ProbeSpill::RegisterThread() { } else { idx_t thread_idx = local_spill_collections.size(); local_spill_collections.emplace_back(make_unique(context, probe_types)); - spill_append_states.emplace_back(); - local_spill_collections.back()->InitializeAppend(spill_append_states.back()); + spill_append_states.emplace_back(make_unique()); + local_spill_collections.back()->InitializeAppend(*spill_append_states.back()); return thread_idx; } } void ProbeSpill::Append(DataChunk &chunk, idx_t thread_idx) { if (partitioned) { - partitioned_data->Append(*partition_append_states[thread_idx], chunk); + partitioned_data->AppendChunk(*partition_append_states[thread_idx], chunk); } else { - local_spill_collections[thread_idx]->Append(spill_append_states[thread_idx], chunk); + local_spill_collections[thread_idx]->Append(*spill_append_states[thread_idx], chunk); } } void ProbeSpill::Finalize() { if (partitioned) { for (auto &append_state : partition_append_states) { - partitioned_data->AppendLocalState(*append_state); + partitioned_data->CombineLocalState(*append_state); } partition_append_states.clear(); } else { diff --git a/src/include/duckdb/common/types/column_data_allocator.hpp b/src/include/duckdb/common/types/column_data_allocator.hpp index 6a67e846350e..4b7d361bf7c6 100644 --- a/src/include/duckdb/common/types/column_data_allocator.hpp +++ b/src/include/duckdb/common/types/column_data_allocator.hpp @@ -50,6 +50,7 @@ class ColumnDataAllocator { private: void AllocateDataInternal(idx_t size, uint32_t &block_id, uint32_t &offset, ChunkManagementState *chunk_state); + void InitializeChunkStateInternal(ChunkManagementState &state, ChunkMetaData &meta_data); void AllocateBlock(); BufferHandle Pin(uint32_t block_id); diff --git a/src/include/duckdb/common/types/partitioned_column_data.hpp b/src/include/duckdb/common/types/partitioned_column_data.hpp index 58c069101b16..cc54ff3cb663 100644 --- a/src/include/duckdb/common/types/partitioned_column_data.hpp +++ b/src/include/duckdb/common/types/partitioned_column_data.hpp @@ -18,17 +18,20 @@ class PartitionedColumnData; //! Local state for partioning in parallel struct PartitionedColumnDataAppendState { +public: explicit PartitionedColumnDataAppendState() : partition_indices(LogicalType::UBIGINT) { } - // Implicit copying is not allowed - PartitionedColumnDataAppendState(const PartitionedColumnDataAppendState &) = delete; - +public: Vector partition_indices; SelectionVector partition_sel; vector> partition_buffers; vector> partitions; + +private: + //! Implicit copying is not allowed + PartitionedColumnDataAppendState(const PartitionedColumnDataAppendState &) = delete; }; //! PartitionedColumnData represents partitioned columnar data, which serves as an interface for different flavors of @@ -61,10 +64,10 @@ class PartitionedColumnData { void InitializeAppendState(PartitionedColumnDataAppendState &state); //! Appends a DataChunk to the PartitionedColumnDataAppendState - virtual void Append(PartitionedColumnDataAppendState &state, DataChunk &input); + virtual void AppendChunk(PartitionedColumnDataAppendState &state, DataChunk &input); - //! Appends a local state into this PartitionedColumnData TODO: rename this - void AppendLocalState(PartitionedColumnDataAppendState &state); + //! Combine a local state into this PartitionedColumnData + void CombineLocalState(PartitionedColumnDataAppendState &state); private: //===--------------------------------------------------------------------===// diff --git a/src/include/duckdb/common/types/selection_vector.hpp b/src/include/duckdb/common/types/selection_vector.hpp index a1e41d47ebc3..c33fa4124712 100644 --- a/src/include/duckdb/common/types/selection_vector.hpp +++ b/src/include/duckdb/common/types/selection_vector.hpp @@ -56,9 +56,9 @@ struct SelectionVector { selection_data = move(data); sel_vector = selection_data->owned_data.get(); } - void Initialize(const SelectionVector &other) { + void Initialize(const SelectionVector &other, idx_t offset = 0) { selection_data = other.selection_data; - sel_vector = other.sel_vector; + sel_vector = other.sel_vector + offset; } inline void set_index(idx_t idx, idx_t loc) { diff --git a/src/include/duckdb/execution/join_hashtable.hpp b/src/include/duckdb/execution/join_hashtable.hpp index 04a9184ec47e..ac61cd0e0567 100644 --- a/src/include/duckdb/execution/join_hashtable.hpp +++ b/src/include/duckdb/execution/join_hashtable.hpp @@ -270,7 +270,7 @@ class JoinHashTable { //! The probe data (if not partitioned) and append states unique_ptr global_spill_collection; vector> local_spill_collections; - vector spill_append_states; + vector> spill_append_states; }; //! Whether we are doing an external hash join From 014c1211f8a04e80c67fa348e667936e73669c73 Mon Sep 17 00:00:00 2001 From: Laurens Kuiper Date: Mon, 3 Oct 2022 15:36:40 +0200 Subject: [PATCH 07/49] offset pointer to sel vector instead of passing offset --- src/common/radix_partitioning.cpp | 2 +- src/common/types/column_data_collection_segment.cpp | 8 +++++++- src/common/types/partitioned_column_data.cpp | 2 +- .../common/types/column_data_collection_segment.hpp | 4 +++- src/include/duckdb/common/types/selection_vector.hpp | 4 ++-- 5 files changed, 14 insertions(+), 6 deletions(-) diff --git a/src/common/radix_partitioning.cpp b/src/common/radix_partitioning.cpp index 0ad38180ffb6..5913c0ca7941 100644 --- a/src/common/radix_partitioning.cpp +++ b/src/common/radix_partitioning.cpp @@ -302,7 +302,7 @@ struct PartitionFunctor { static inline void FlushTempBuf(data_ptr_t &data_ptr, const idx_t &row_width, uint32_t &block_count, const data_ptr_t &tmp_buf, uint32_t &pos, const idx_t count) { pos -= count; - memcpy(data_ptr, tmp_buf + pos * row_width, count * row_width); + FastMemcpy(data_ptr, tmp_buf + pos * row_width, count * row_width); data_ptr += count * row_width; block_count += count; } diff --git a/src/common/types/column_data_collection_segment.cpp b/src/common/types/column_data_collection_segment.cpp index 710daf3fd52e..8058d55987dc 100644 --- a/src/common/types/column_data_collection_segment.cpp +++ b/src/common/types/column_data_collection_segment.cpp @@ -214,8 +214,14 @@ void ColumnDataCollectionSegment::FetchChunk(idx_t chunk_idx, DataChunk &result) void ColumnDataCollectionSegment::FetchChunk(idx_t chunk_idx, DataChunk &result, const vector &column_ids) { D_ASSERT(chunk_idx < chunk_data.size()); ChunkManagementState state; - InitializeChunkState(chunk_idx, state); state.properties = ColumnDataScanProperties::DISALLOW_ZERO_COPY; + FetchChunk(state, chunk_idx, result, column_ids); +} + +void ColumnDataCollectionSegment::FetchChunk(ChunkManagementState &state, idx_t chunk_idx, DataChunk &result, + const vector &column_ids) { + D_ASSERT(chunk_idx < chunk_data.size()); + InitializeChunkState(chunk_idx, state); ReadChunk(chunk_idx, state, result, column_ids); } diff --git a/src/common/types/partitioned_column_data.cpp b/src/common/types/partitioned_column_data.cpp index 6cc2af3aef7f..d2f143063b2f 100644 --- a/src/common/types/partitioned_column_data.cpp +++ b/src/common/types/partitioned_column_data.cpp @@ -74,7 +74,7 @@ void PartitionedColumnData::AppendChunk(PartitionedColumnDataAppendState &state, } // Create a selection vector for this partition using the offset into the single selection vector - partition_sel.Initialize(all_partitions_sel, partition_offset); + partition_sel.Initialize(all_partitions_sel.data() + partition_offset); // Append the input chunk to the partition buffer using the selection vector partition_buffer.Append(input, false, &partition_sel, partition_length); diff --git a/src/include/duckdb/common/types/column_data_collection_segment.hpp b/src/include/duckdb/common/types/column_data_collection_segment.hpp index bdbda3a52d40..b290523a48c8 100644 --- a/src/include/duckdb/common/types/column_data_collection_segment.hpp +++ b/src/include/duckdb/common/types/column_data_collection_segment.hpp @@ -8,8 +8,8 @@ #pragma once -#include "duckdb/common/types/column_data_collection.hpp" #include "duckdb/common/types/column_data_allocator.hpp" +#include "duckdb/common/types/column_data_collection.hpp" namespace duckdb { @@ -112,6 +112,8 @@ class ColumnDataCollectionSegment { idx_t ChunkCount() const; void FetchChunk(idx_t chunk_idx, DataChunk &result); void FetchChunk(idx_t chunk_idx, DataChunk &result, const vector &column_ids); + void FetchChunk(ChunkManagementState &state, idx_t chunk_idx, DataChunk &result, + const vector &column_ids); void Verify(); diff --git a/src/include/duckdb/common/types/selection_vector.hpp b/src/include/duckdb/common/types/selection_vector.hpp index c33fa4124712..a1e41d47ebc3 100644 --- a/src/include/duckdb/common/types/selection_vector.hpp +++ b/src/include/duckdb/common/types/selection_vector.hpp @@ -56,9 +56,9 @@ struct SelectionVector { selection_data = move(data); sel_vector = selection_data->owned_data.get(); } - void Initialize(const SelectionVector &other, idx_t offset = 0) { + void Initialize(const SelectionVector &other) { selection_data = other.selection_data; - sel_vector = other.sel_vector + offset; + sel_vector = other.sel_vector; } inline void set_index(idx_t idx, idx_t loc) { From 8f8dbbebea7ad221ebe83de085d96859c46ce0c5 Mon Sep 17 00:00:00 2001 From: Laurens Kuiper Date: Tue, 4 Oct 2022 10:47:08 +0200 Subject: [PATCH 08/49] sort segments, ensure block size is BLOCK_SIZE --- src/common/symbols.cpp | 8 +++ src/common/types/column_data_collection.cpp | 26 ++++++++++ .../types/column_data_collection_segment.cpp | 1 - src/common/types/row_data_collection.cpp | 2 +- src/execution/join_hashtable.cpp | 51 ++++++++++++++----- .../operator/join/physical_hash_join.cpp | 29 +++++------ .../common/types/column_data_collection.hpp | 4 ++ .../common/types/partitioned_column_data.hpp | 1 + .../common/types/row_data_collection.hpp | 5 +- .../duckdb/execution/join_hashtable.hpp | 16 +++++- 10 files changed, 110 insertions(+), 33 deletions(-) diff --git a/src/common/symbols.cpp b/src/common/symbols.cpp index 20020ad10f25..c963880a9020 100644 --- a/src/common/symbols.cpp +++ b/src/common/symbols.cpp @@ -35,6 +35,8 @@ #include "duckdb/storage/data_table.hpp" #include "duckdb/storage/write_ahead_log.hpp" #include "duckdb/transaction/transaction.hpp" +#include "duckdb/common/types/column_data_collection.hpp" +#include "duckdb/common/types/column_data_allocator.hpp" using namespace duckdb; @@ -84,6 +86,9 @@ template class std::shared_ptr; template class std::weak_ptr; template class std::unique_ptr; template class std::shared_ptr; +template class std::unique_ptr; +template class std::shared_ptr; +template class std::unique_ptr; template class std::shared_ptr; template class std::unique_ptr; @@ -127,6 +132,7 @@ template class std::unique_ptr; template class std::unique_ptr; template class std::unique_ptr; template class std::unique_ptr; +template class std::unique_ptr; template class std::unique_ptr; template class std::unique_ptr; template class std::unique_ptr; @@ -172,6 +178,8 @@ INSTANTIATE_VECTOR(std::vector>) template class std::vector>; template class std::vector; INSTANTIATE_VECTOR(std::vector>) +INSTANTIATE_VECTOR(std::vector>) +INSTANTIATE_VECTOR(std::vector>) #if !defined(__clang__) template struct std::atomic; diff --git a/src/common/types/column_data_collection.cpp b/src/common/types/column_data_collection.cpp index 5bc5b116ae93..ad72e62889a5 100644 --- a/src/common/types/column_data_collection.cpp +++ b/src/common/types/column_data_collection.cpp @@ -812,4 +812,30 @@ bool ColumnDataCollection::ResultEquals(const ColumnDataCollection &left, const return true; } +vector> ColumnDataCollection::GetChunkReferences(bool sort) { + // Create a vector containing a reference to every chunk + vector> result; + result.reserve(ChunkCount()); + for (auto &segment : segments) { + for (idx_t chunk_index = 0; chunk_index < segment->chunk_data.size(); chunk_index++) { + result.emplace_back(segment.get(), chunk_index); + } + } + + if (sort) { + // Sort them by lowest block id + std::sort(result.begin(), result.end(), + [](const pair &lhs, + const pair &rhs) -> bool { + const auto &lhs_block_ids = lhs.first->chunk_data[lhs.second].block_ids; + const auto &rhs_block_ids = rhs.first->chunk_data[rhs.second].block_ids; + const uint32_t lhs_min_block_id = *std::min_element(lhs_block_ids.begin(), lhs_block_ids.end()); + const uint32_t rhs_min_block_id = *std::min_element(rhs_block_ids.begin(), rhs_block_ids.end()); + return lhs_min_block_id < rhs_min_block_id; + }); + } + + return result; +} + } // namespace duckdb diff --git a/src/common/types/column_data_collection_segment.cpp b/src/common/types/column_data_collection_segment.cpp index 8058d55987dc..a6f19968b498 100644 --- a/src/common/types/column_data_collection_segment.cpp +++ b/src/common/types/column_data_collection_segment.cpp @@ -221,7 +221,6 @@ void ColumnDataCollectionSegment::FetchChunk(idx_t chunk_idx, DataChunk &result, void ColumnDataCollectionSegment::FetchChunk(ChunkManagementState &state, idx_t chunk_idx, DataChunk &result, const vector &column_ids) { D_ASSERT(chunk_idx < chunk_data.size()); - InitializeChunkState(chunk_idx, state); ReadChunk(chunk_idx, state, result, column_ids); } diff --git a/src/common/types/row_data_collection.cpp b/src/common/types/row_data_collection.cpp index bda851c23f2d..9cdd0d79de63 100644 --- a/src/common/types/row_data_collection.cpp +++ b/src/common/types/row_data_collection.cpp @@ -6,7 +6,7 @@ RowDataCollection::RowDataCollection(BufferManager &buffer_manager, idx_t block_ bool keep_pinned) : buffer_manager(buffer_manager), count(0), block_capacity(block_capacity), entry_size(entry_size), keep_pinned(keep_pinned) { - D_ASSERT(block_capacity * entry_size >= Storage::BLOCK_SIZE); + D_ASSERT(block_capacity * entry_size + entry_size > Storage::BLOCK_SIZE); } idx_t RowDataCollection::AppendToBlock(RowDataBlock &block, BufferHandle &handle, diff --git a/src/execution/join_hashtable.cpp b/src/execution/join_hashtable.cpp index ea97ce323796..e1861d00af79 100644 --- a/src/execution/join_hashtable.cpp +++ b/src/execution/join_hashtable.cpp @@ -2,6 +2,7 @@ #include "duckdb/common/exception.hpp" #include "duckdb/common/row_operations/row_operations.hpp" +#include "duckdb/common/types/column_data_collection_segment.hpp" #include "duckdb/common/types/row_data_collection.hpp" #include "duckdb/common/types/row_data_collection_scanner.hpp" #include "duckdb/common/vector_operations/vector_operations.hpp" @@ -58,7 +59,7 @@ JoinHashTable::JoinHashTable(BufferManager &buffer_manager, const vector(STANDARD_VECTOR_SIZE, (Storage::BLOCK_SIZE / entry_size) + 1); + idx_t block_capacity = Storage::BLOCK_SIZE / entry_size; block_collection = make_unique(buffer_manager, block_capacity, entry_size); string_heap = make_unique(buffer_manager, (idx_t)Storage::BLOCK_SIZE, 1, true); swizzled_block_collection = block_collection->CloneEmpty(); @@ -1198,6 +1199,10 @@ ProbeSpill::ProbeSpill(JoinHashTable &ht, ClientContext &context, const vector(context, probe_types, ht.radix_bits, probe_types.size() - 1); } + column_ids.reserve(probe_types.size()); + for (column_t column_id = 0; column_id < probe_types.size(); column_id++) { + column_ids.emplace_back(column_id); + } } idx_t ProbeSpill::RegisterThread() { @@ -1240,24 +1245,46 @@ void ProbeSpill::Finalize() { } } -unique_ptr ProbeSpill::GetNextProbeCollection(JoinHashTable &ht) { +void ProbeSpill::PrepareNextProbeCollection(JoinHashTable &ht) { + // Reset previous probe collection + current_probe_collection = nullptr; + chunk_references.clear(); if (partitioned) { auto &partitions = partitioned_data->GetPartitions(); if (ht.partition_start == partitions.size()) { - return nullptr; + return; + } + + // Pre-allocate + idx_t chunks = 0; + for (idx_t p_idx = ht.partition_start; p_idx < ht.partition_end; p_idx++) { + chunks += partitions[p_idx]->ChunkCount(); } - auto merged_partitions = move(partitions[ht.partition_start]); - for (idx_t p_idx = ht.partition_start + 1; p_idx < ht.partition_end; p_idx++) { - auto &partition = partitions[p_idx]; - merged_partitions->Combine(*partition); - partition.reset(); + chunk_references.reserve(chunks); + + // Get sorted chunk references per partition + for (idx_t p_idx = ht.partition_start; p_idx < ht.partition_end; p_idx++) { + auto partition_chunk_references = partitions[p_idx]->GetChunkReferences(true); + chunk_references.insert(chunk_references.end(), partition_chunk_references.begin(), + partition_chunk_references.end()); } - return merged_partitions; + + // TODO: maybe deal with partition collections? } else { - auto result = move(global_spill_collection); - global_spill_collection = nullptr; - return result; + current_probe_collection = move(global_spill_collection); + chunk_references = global_spill_collection->GetChunkReferences(false); } } +void ProbeSpill::ScanChunk(ChunkManagementState &state, idx_t chunk_idx, DataChunk &chunk) { + auto &chunk_ref = chunk_references[chunk_idx]; + auto &segment = *chunk_ref.first; + auto &idx_within_segment = chunk_ref.second; + + state.handles.clear(); + chunk.Reset(); + segment.FetchChunk(state, idx_within_segment, chunk, column_ids); + chunk.Verify(); +} + } // namespace duckdb diff --git a/src/execution/operator/join/physical_hash_join.cpp b/src/execution/operator/join/physical_hash_join.cpp index c063f7402f32..9687dce2e68a 100644 --- a/src/execution/operator/join/physical_hash_join.cpp +++ b/src/execution/operator/join/physical_hash_join.cpp @@ -537,9 +537,6 @@ class HashJoinGlobalSourceState : public GlobalSourceState { //! The JoinType of the PhysicalHashJoin JoinType join_type; - //! Probe-side data that was spilled during Execute - unique_ptr probe_collection = nullptr; - //! For synchronizing the external hash join atomic initialized; atomic global_stage; @@ -553,6 +550,7 @@ class HashJoinGlobalSourceState : public GlobalSourceState { //! For probe synchronization ColumnDataParallelScanState probe_global_scan; + idx_t probe_chunk_idx; idx_t probe_chunk_count; idx_t probe_chunk_done; @@ -591,11 +589,9 @@ class HashJoinLocalSourceState : public LocalSourceState { idx_t build_block_idx_end; //! Local scan state for probe collection - ColumnDataLocalScanState probe_local_scan; - //! Indices for ColumnDataCollection::NextScanIndex - idx_t chunk_index; - idx_t segment_index; - idx_t row_index; + ChunkManagementState probe_chunk_management_state; + //! Index of probe chunk to scan next + idx_t probe_chunk_index; //! Chunks for holding the scanned probe collection DataChunk probe_chunk; DataChunk join_keys; @@ -667,11 +663,10 @@ void HashJoinGlobalSourceState::PrepareBuild(HashJoinGlobalSinkState &sink) { } void HashJoinGlobalSourceState::PrepareProbe(HashJoinGlobalSinkState &sink) { - probe_collection = sink.probe_spill->GetNextProbeCollection(*sink.hash_table); - D_ASSERT(probe_collection); + sink.probe_spill->PrepareNextProbeCollection(*sink.hash_table); - probe_collection->InitializeScan(probe_global_scan); - probe_chunk_count = probe_collection->ChunkCount(); + probe_chunk_idx = 0; + probe_chunk_count = sink.probe_spill->ChunkCount(); probe_chunk_done = 0; if (IsRightOuterJoin(join_type)) { @@ -707,9 +702,9 @@ bool HashJoinGlobalSourceState::AssignTask(HashJoinGlobalSinkState &sink, HashJo } break; case HashJoinSourceStage::PROBE: - if (probe_collection->NextScanIndex(probe_global_scan.scan_state, lstate.chunk_index, lstate.segment_index, - lstate.row_index)) { + if (probe_chunk_idx != probe_chunk_count) { lstate.local_stage = global_stage; + lstate.probe_chunk_index = probe_chunk_idx++; return true; } break; @@ -730,6 +725,8 @@ bool HashJoinGlobalSourceState::AssignTask(HashJoinGlobalSinkState &sink, HashJo HashJoinLocalSourceState::HashJoinLocalSourceState(const PhysicalHashJoin &op, Allocator &allocator) : local_stage(HashJoinSourceStage::INIT), addresses(LogicalType::POINTER) { + probe_chunk_management_state.properties = ColumnDataScanProperties::ALLOW_ZERO_COPY; + auto &sink = (HashJoinGlobalSinkState &)*op.sink_state; probe_chunk.Initialize(allocator, sink.probe_types); join_keys.Initialize(allocator, op.condition_types); @@ -807,13 +804,13 @@ void HashJoinLocalSourceState::ExternalProbe(HashJoinGlobalSinkState &sink, Hash gstate.PrepareBuild(sink); } } + // TODO: delete blocks that we passed (blocks could have different allocators ...) } return; } // Scan input chunk for next probe - gstate.probe_collection->ScanAtIndex(gstate.probe_global_scan, probe_local_scan, probe_chunk, chunk_index, - segment_index, row_index); + sink.probe_spill->ScanChunk(probe_chunk_management_state, probe_chunk_index, probe_chunk); // Get the probe chunk columns/hashes join_keys.ReferenceColumns(probe_chunk, join_key_indices); diff --git a/src/include/duckdb/common/types/column_data_collection.hpp b/src/include/duckdb/common/types/column_data_collection.hpp index e9d6fd96fd52..62622240699e 100644 --- a/src/include/duckdb/common/types/column_data_collection.hpp +++ b/src/include/duckdb/common/types/column_data_collection.hpp @@ -8,6 +8,7 @@ #pragma once +#include "duckdb/common/pair.hpp" #include "duckdb/common/types/column_data_collection_iterators.hpp" namespace duckdb { @@ -136,6 +137,9 @@ class ColumnDataCollection { void ScanAtIndex(ColumnDataParallelScanState &state, ColumnDataLocalScanState &lstate, DataChunk &result, idx_t chunk_index, idx_t segment_index, idx_t row_index) const; + //! Get a vector of references to every chunk (segment, index in segment), and optionally sort by block id + vector> GetChunkReferences(bool sort); + private: //! Initialize the column data collection void Initialize(vector types); diff --git a/src/include/duckdb/common/types/partitioned_column_data.hpp b/src/include/duckdb/common/types/partitioned_column_data.hpp index cc54ff3cb663..2db3eb04afe7 100644 --- a/src/include/duckdb/common/types/partitioned_column_data.hpp +++ b/src/include/duckdb/common/types/partitioned_column_data.hpp @@ -8,6 +8,7 @@ #pragma once +#include "duckdb/common/pair.hpp" #include "duckdb/common/types/column_data_allocator.hpp" #include "duckdb/common/types/column_data_collection.hpp" diff --git a/src/include/duckdb/common/types/row_data_collection.hpp b/src/include/duckdb/common/types/row_data_collection.hpp index 5f4070b40f16..4f28d2dc118c 100644 --- a/src/include/duckdb/common/types/row_data_collection.hpp +++ b/src/include/duckdb/common/types/row_data_collection.hpp @@ -18,7 +18,8 @@ struct RowDataBlock { public: RowDataBlock(BufferManager &buffer_manager, idx_t capacity, idx_t entry_size) : capacity(capacity), entry_size(entry_size), count(0), byte_offset(0) { - block = buffer_manager.RegisterMemory(capacity * entry_size, false); + idx_t size = MaxValue(Storage::BLOCK_SIZE, capacity * entry_size); + block = buffer_manager.RegisterMemory(size, false); } explicit RowDataBlock(idx_t entry_size) : entry_size(entry_size) { } @@ -106,7 +107,7 @@ class RowDataCollection { } static inline idx_t EntriesPerBlock(idx_t width) { - return (Storage::BLOCK_SIZE + width * STANDARD_VECTOR_SIZE - 1) / width; + return Storage::BLOCK_SIZE / width; } private: diff --git a/src/include/duckdb/execution/join_hashtable.hpp b/src/include/duckdb/execution/join_hashtable.hpp index ac61cd0e0567..1417d1637e3b 100644 --- a/src/include/duckdb/execution/join_hashtable.hpp +++ b/src/include/duckdb/execution/join_hashtable.hpp @@ -253,7 +253,13 @@ class JoinHashTable { void Finalize(); //! Get the probe collection for the next probe round - unique_ptr GetNextProbeCollection(JoinHashTable &ht); + void PrepareNextProbeCollection(JoinHashTable &ht); + //! Number of chunks in the current probe collection + idx_t ChunkCount() const { + return chunk_references.size(); + } + //! Scans the chunk with the given index + void ScanChunk(ChunkManagementState &state, idx_t chunk_idx, DataChunk &chunk); private: mutex lock; @@ -261,16 +267,24 @@ class JoinHashTable { //! The types of the probe DataChunks const vector &probe_types; + //! The column ids + vector column_ids; //! Whether the probe data is partitioned bool partitioned; //! The partitioned probe data (if partitioned) and append states unique_ptr partitioned_data; vector> partition_append_states; + //! The probe data (if not partitioned) and append states unique_ptr global_spill_collection; vector> local_spill_collections; vector> spill_append_states; + + //! The probe collection currently being read + unique_ptr current_probe_collection; + //! The references (in order) to the chunks of the current probe collection + vector> chunk_references; }; //! Whether we are doing an external hash join From fbe4685e90473cc06b69be098db72be60513e1fa Mon Sep 17 00:00:00 2001 From: Laurens Kuiper Date: Tue, 4 Oct 2022 13:32:37 +0200 Subject: [PATCH 09/49] less locking and add ColumnDataAppendState to PartitionedColumnDataAppendState --- src/common/radix_partitioning.cpp | 3 +++ src/common/types/column_data_allocator.cpp | 18 +++++++----------- src/common/types/partitioned_column_data.cpp | 5 +++-- .../common/types/column_data_allocator.hpp | 1 - .../common/types/partitioned_column_data.hpp | 1 + 5 files changed, 14 insertions(+), 14 deletions(-) diff --git a/src/common/radix_partitioning.cpp b/src/common/radix_partitioning.cpp index cc98bccf63af..360caf83582c 100644 --- a/src/common/radix_partitioning.cpp +++ b/src/common/radix_partitioning.cpp @@ -430,9 +430,12 @@ void RadixPartitionedColumnData::InitializeAppendStateInternal(PartitionedColumn const auto num_partitions = RadixPartitioning::NumberOfPartitions(radix_bits); state.partition_buffers.reserve(num_partitions); state.partitions.reserve(num_partitions); + state.partition_append_states.reserve(num_partitions); for (idx_t i = 0; i < num_partitions; i++) { state.partition_buffers.emplace_back(CreateAppendPartitionBuffer()); state.partitions.emplace_back(CreateAppendPartition(i)); + state.partition_append_states.emplace_back(); + state.partitions.back()->InitializeAppend(state.partition_append_states.back()); } } diff --git a/src/common/types/column_data_allocator.cpp b/src/common/types/column_data_allocator.cpp index 01e687365ef8..adf903dc57c3 100644 --- a/src/common/types/column_data_allocator.cpp +++ b/src/common/types/column_data_allocator.cpp @@ -30,7 +30,12 @@ ColumnDataAllocator::ColumnDataAllocator(ClientContext &context, ColumnDataAlloc BufferHandle ColumnDataAllocator::Pin(uint32_t block_id) { D_ASSERT(type == ColumnDataAllocatorType::BUFFER_MANAGER_ALLOCATOR); - return alloc.buffer_manager->Pin(blocks[block_id].handle); + if (shared) { + lock_guard guard(lock); + return alloc.buffer_manager->Pin(blocks[block_id].handle); + } else { + return alloc.buffer_manager->Pin(blocks[block_id].handle); + } } void ColumnDataAllocator::AllocateBlock() { @@ -112,7 +117,7 @@ Allocator &ColumnDataAllocator::GetAllocator() { : alloc.buffer_manager->GetBufferAllocator(); } -void ColumnDataAllocator::InitializeChunkStateInternal(ChunkManagementState &state, ChunkMetaData &chunk) { +void ColumnDataAllocator::InitializeChunkState(ChunkManagementState &state, ChunkMetaData &chunk) { if (type != ColumnDataAllocatorType::BUFFER_MANAGER_ALLOCATOR) { // nothing to pin return; @@ -142,15 +147,6 @@ void ColumnDataAllocator::InitializeChunkStateInternal(ChunkManagementState &sta } } -void ColumnDataAllocator::InitializeChunkState(ChunkManagementState &state, ChunkMetaData &chunk) { - if (shared) { - lock_guard guard(lock); - InitializeChunkStateInternal(state, chunk); - } else { - InitializeChunkStateInternal(state, chunk); - } -} - uint32_t BlockMetaData::Capacity() { D_ASSERT(size <= capacity); return capacity - size; diff --git a/src/common/types/partitioned_column_data.cpp b/src/common/types/partitioned_column_data.cpp index d2f143063b2f..5b14f90d7777 100644 --- a/src/common/types/partitioned_column_data.cpp +++ b/src/common/types/partitioned_column_data.cpp @@ -20,7 +20,7 @@ void PartitionedColumnData::AppendChunk(PartitionedColumnDataAppendState &state, // Compute the counts per partition const auto count = input.size(); - map partition_entries; + unordered_map partition_entries; const auto partition_indices = FlatVector::GetData(state.partition_indices); for (idx_t i = 0; i < count; i++) { const auto &partition_index = partition_indices[i]; @@ -69,7 +69,8 @@ void PartitionedColumnData::AppendChunk(PartitionedColumnDataAppendState &state, if (partition_buffer.size() + partition_length > STANDARD_VECTOR_SIZE) { // Next batch won't fit in the buffer, flush it to the partition auto &partition = *state.partitions[partition_index]; - partition.Append(partition_buffer); + auto &partition_append_state = state.partition_append_states[partition_index]; + partition.Append(partition_append_state, partition_buffer); partition_buffer.Reset(); } diff --git a/src/include/duckdb/common/types/column_data_allocator.hpp b/src/include/duckdb/common/types/column_data_allocator.hpp index 4b7d361bf7c6..6a67e846350e 100644 --- a/src/include/duckdb/common/types/column_data_allocator.hpp +++ b/src/include/duckdb/common/types/column_data_allocator.hpp @@ -50,7 +50,6 @@ class ColumnDataAllocator { private: void AllocateDataInternal(idx_t size, uint32_t &block_id, uint32_t &offset, ChunkManagementState *chunk_state); - void InitializeChunkStateInternal(ChunkManagementState &state, ChunkMetaData &meta_data); void AllocateBlock(); BufferHandle Pin(uint32_t block_id); diff --git a/src/include/duckdb/common/types/partitioned_column_data.hpp b/src/include/duckdb/common/types/partitioned_column_data.hpp index 2db3eb04afe7..bab71ea39299 100644 --- a/src/include/duckdb/common/types/partitioned_column_data.hpp +++ b/src/include/duckdb/common/types/partitioned_column_data.hpp @@ -29,6 +29,7 @@ struct PartitionedColumnDataAppendState { vector> partition_buffers; vector> partitions; + vector partition_append_states; private: //! Implicit copying is not allowed From cdfd9fbbca33769bda81b195df4c3a4f5f586f87 Mon Sep 17 00:00:00 2001 From: Laurens Kuiper Date: Tue, 4 Oct 2022 14:59:13 +0200 Subject: [PATCH 10/49] BLOCK_SIZE instead of BLOCK_ALLOC_SIZE, less tasks for ExternalBuild in external hash join --- src/common/types/column_data_allocator.cpp | 4 +- src/common/types/partitioned_column_data.cpp | 43 ++++++++++++------- src/execution/join_hashtable.cpp | 4 +- .../operator/join/physical_hash_join.cpp | 26 ++++++----- .../common/types/partitioned_column_data.hpp | 1 + 5 files changed, 45 insertions(+), 33 deletions(-) diff --git a/src/common/types/column_data_allocator.cpp b/src/common/types/column_data_allocator.cpp index adf903dc57c3..5539581ccb6b 100644 --- a/src/common/types/column_data_allocator.cpp +++ b/src/common/types/column_data_allocator.cpp @@ -42,8 +42,8 @@ void ColumnDataAllocator::AllocateBlock() { D_ASSERT(type == ColumnDataAllocatorType::BUFFER_MANAGER_ALLOCATOR); BlockMetaData data; data.size = 0; - data.capacity = Storage::BLOCK_ALLOC_SIZE; - data.handle = alloc.buffer_manager->RegisterMemory(Storage::BLOCK_ALLOC_SIZE, false); + data.capacity = Storage::BLOCK_SIZE; + data.handle = alloc.buffer_manager->RegisterMemory(Storage::BLOCK_SIZE, false); blocks.push_back(move(data)); } diff --git a/src/common/types/partitioned_column_data.cpp b/src/common/types/partitioned_column_data.cpp index 5b14f90d7777..edd37b703dc1 100644 --- a/src/common/types/partitioned_column_data.cpp +++ b/src/common/types/partitioned_column_data.cpp @@ -11,6 +11,7 @@ PartitionedColumnData::~PartitionedColumnData() { void PartitionedColumnData::InitializeAppendState(PartitionedColumnDataAppendState &state) { state.partition_sel.Initialize(); + state.slice_chunk.Initialize(context, types); InitializeAppendStateInternal(state); } @@ -34,8 +35,10 @@ void PartitionedColumnData::AppendChunk(PartitionedColumnDataAppendState &state, // Early out: check if everything belongs to a single partition if (partition_entries.size() == 1) { - auto partition_entry = *partition_entries.begin(); - state.partition_buffers[partition_entry.first]->Append(input); + const auto &partition_index = partition_entries.begin()->first; + auto &partition = *state.partitions[partition_index]; + auto &partition_append_state = state.partition_append_states[partition_index]; + partition.Append(partition_append_state, input); return; } @@ -59,26 +62,37 @@ void PartitionedColumnData::AppendChunk(PartitionedColumnDataAppendState &state, SelectionVector partition_sel; for (auto &pc : partition_entries) { const auto &partition_index = pc.first; - const auto &partition_entry = pc.second; + + // Partition, buffer, and append state for this partition index + auto &partition = *state.partitions[partition_index]; + auto &partition_buffer = *state.partition_buffers[partition_index]; + auto &partition_append_state = state.partition_append_states[partition_index]; // Length and offset into the selection vector for this chunk, for this partition + const auto &partition_entry = pc.second; const auto &partition_length = partition_entry.length; const auto partition_offset = partition_entry.offset - partition_length; - auto &partition_buffer = *state.partition_buffers[partition_index]; - if (partition_buffer.size() + partition_length > STANDARD_VECTOR_SIZE) { - // Next batch won't fit in the buffer, flush it to the partition - auto &partition = *state.partitions[partition_index]; - auto &partition_append_state = state.partition_append_states[partition_index]; - partition.Append(partition_append_state, partition_buffer); - partition_buffer.Reset(); - } - // Create a selection vector for this partition using the offset into the single selection vector partition_sel.Initialize(all_partitions_sel.data() + partition_offset); - // Append the input chunk to the partition buffer using the selection vector - partition_buffer.Append(input, false, &partition_sel, partition_length); + if (partition_length >= 64) { + // Slice the input chunk using the selection vector + state.slice_chunk.Reset(); + state.slice_chunk.Slice(input, partition_sel, partition_length); + + // Append it to the partition + partition.Append(partition_append_state, state.slice_chunk); + } else { + // Append the input chunk to the partition buffer using the selection vector + partition_buffer.Append(input, false, &partition_sel, partition_length); + + if (partition_buffer.size() >= 64) { + // Next batch won't fit in the buffer, flush it to the partition + partition.Append(partition_append_state, partition_buffer); + partition_buffer.Reset(); // TODO: Reset sets the capacity back to STANDARD_VECTOR_SIZE + } + } } } @@ -103,7 +117,6 @@ void PartitionedColumnData::CombineLocalState(PartitionedColumnDataAppendState & for (idx_t i = 0; i < NumberOfPartitions(); i++) { partitions[i]->Combine(*state.partitions[i]); } - // TODO: sort CDC segments or chunks on their block ID, so we don't read, e.g., 1 chunk per buffer block } } diff --git a/src/execution/join_hashtable.cpp b/src/execution/join_hashtable.cpp index e1861d00af79..6defb4e1974f 100644 --- a/src/execution/join_hashtable.cpp +++ b/src/execution/join_hashtable.cpp @@ -1269,10 +1269,10 @@ void ProbeSpill::PrepareNextProbeCollection(JoinHashTable &ht) { partition_chunk_references.end()); } - // TODO: maybe deal with partition collections? + // TODO: what to do with partition collections? } else { current_probe_collection = move(global_spill_collection); - chunk_references = global_spill_collection->GetChunkReferences(false); + chunk_references = current_probe_collection->GetChunkReferences(false); } } diff --git a/src/execution/operator/join/physical_hash_join.cpp b/src/execution/operator/join/physical_hash_join.cpp index 9687dce2e68a..1292dcaf653b 100644 --- a/src/execution/operator/join/physical_hash_join.cpp +++ b/src/execution/operator/join/physical_hash_join.cpp @@ -522,7 +522,7 @@ class HashJoinGlobalSourceState : public GlobalSourceState { HashJoinGlobalSourceState(const PhysicalHashJoin &op, ClientContext &context); //! Initialize this source state using the info in the sink - void Initialize(HashJoinGlobalSinkState &sink); + void Initialize(ClientContext &context, HashJoinGlobalSinkState &sink); //! Prepare the next build/probe stage for external hash join (must hold lock) void PrepareBuild(HashJoinGlobalSinkState &sink); void PrepareProbe(HashJoinGlobalSinkState &sink); @@ -534,8 +534,7 @@ class HashJoinGlobalSourceState : public GlobalSourceState { } public: - //! The JoinType of the PhysicalHashJoin - JoinType join_type; + const PhysicalHashJoin &op; //! For synchronizing the external hash join atomic initialized; @@ -617,12 +616,12 @@ unique_ptr PhysicalHashJoin::GetLocalSourceState(ExecutionCont } HashJoinGlobalSourceState::HashJoinGlobalSourceState(const PhysicalHashJoin &op, ClientContext &context) - : join_type(op.join_type), initialized(false), global_stage(HashJoinSourceStage::INIT), - lock(probe_global_scan.lock), probe_count(op.children[0]->estimated_cardinality), + : op(op), initialized(false), global_stage(HashJoinSourceStage::INIT), lock(probe_global_scan.lock), + probe_count(op.children[0]->estimated_cardinality), parallel_scan_chunk_count(context.config.verify_parallelism ? 1 : 120) { } -void HashJoinGlobalSourceState::Initialize(HashJoinGlobalSinkState &sink) { +void HashJoinGlobalSourceState::Initialize(ClientContext &context, HashJoinGlobalSinkState &sink) { if (initialized) { return; } @@ -633,9 +632,9 @@ void HashJoinGlobalSourceState::Initialize(HashJoinGlobalSinkState &sink) { } full_outer_scan.total = sink.hash_table->Count(); - auto block_capacity = sink.hash_table->GetBlockCollection().block_capacity; - build_blocks_per_thread = - MaxValue(idx_t(parallel_scan_chunk_count * STANDARD_VECTOR_SIZE) / block_capacity, 1); + idx_t num_blocks = sink.hash_table->GetBlockCollection().blocks.size(); + idx_t num_threads = TaskScheduler::GetScheduler(context).NumberOfThreads(); + build_blocks_per_thread = MaxValue((num_blocks + num_threads - 1) / num_threads, 1); // Finalize the probe spill too sink.probe_spill->Finalize(); @@ -669,7 +668,7 @@ void HashJoinGlobalSourceState::PrepareProbe(HashJoinGlobalSinkState &sink) { probe_chunk_count = sink.probe_spill->ChunkCount(); probe_chunk_done = 0; - if (IsRightOuterJoin(join_type)) { + if (IsRightOuterJoin(op.join_type)) { full_outer_scan.Reset(); full_outer_scan.total = sink.hash_table->Count(); } @@ -680,7 +679,7 @@ void HashJoinGlobalSourceState::PrepareProbe(HashJoinGlobalSinkState &sink) { } // Empty probe collection, go straight into the next stage - if (IsRightOuterJoin(join_type)) { + if (IsRightOuterJoin(op.join_type)) { global_stage = HashJoinSourceStage::SCAN_HT; } else { PrepareBuild(sink); @@ -798,7 +797,7 @@ void HashJoinLocalSourceState::ExternalProbe(HashJoinGlobalSinkState &sink, Hash scan_structure = nullptr; lock_guard lock(gstate.lock); if (++gstate.probe_chunk_done == gstate.probe_chunk_count) { - if (IsRightOuterJoin(gstate.join_type)) { + if (IsRightOuterJoin(gstate.op.join_type)) { gstate.global_stage = HashJoinSourceStage::SCAN_HT; } else { gstate.PrepareBuild(sink); @@ -871,10 +870,9 @@ void PhysicalHashJoin::GetData(ExecutionContext &context, DataChunk &chunk, Glob D_ASSERT(can_go_external); // TODO: scan and consume CDC's // String handling in CDC's - // sorting CDC blocks by block ID if (gstate.global_stage == HashJoinSourceStage::INIT) { - gstate.Initialize(sink); + gstate.Initialize(context.client, sink); lock_guard lock(gstate.lock); if (gstate.global_stage == HashJoinSourceStage::INIT) { diff --git a/src/include/duckdb/common/types/partitioned_column_data.hpp b/src/include/duckdb/common/types/partitioned_column_data.hpp index bab71ea39299..14364c5f4d6a 100644 --- a/src/include/duckdb/common/types/partitioned_column_data.hpp +++ b/src/include/duckdb/common/types/partitioned_column_data.hpp @@ -26,6 +26,7 @@ struct PartitionedColumnDataAppendState { public: Vector partition_indices; SelectionVector partition_sel; + DataChunk slice_chunk; vector> partition_buffers; vector> partitions; From de6b7d53f9649b87fd2f819c504293728139c4e8 Mon Sep 17 00:00:00 2001 From: Laurens Kuiper Date: Wed, 5 Oct 2022 09:23:38 +0200 Subject: [PATCH 11/49] add a bunch of TODO's so I don't forget --- src/execution/join_hashtable.cpp | 16 +++++++++++++++- .../operator/join/physical_hash_join.cpp | 1 + 2 files changed, 16 insertions(+), 1 deletion(-) diff --git a/src/execution/join_hashtable.cpp b/src/execution/join_hashtable.cpp index 6defb4e1974f..5a0397c9470a 100644 --- a/src/execution/join_hashtable.cpp +++ b/src/execution/join_hashtable.cpp @@ -1044,6 +1044,8 @@ void JoinHashTable::ComputePartitionSizes(ClientConfig &config, vector Date: Wed, 5 Oct 2022 13:02:36 +0200 Subject: [PATCH 12/49] rework PartitionedColumnData API a bit --- src/common/radix_partitioning.cpp | 28 ++++--- src/common/types/data_chunk.cpp | 6 +- src/common/types/partitioned_column_data.cpp | 53 ++++++++---- src/common/types/vector_cache.cpp | 19 +++-- src/execution/join_hashtable.cpp | 40 +++++---- .../duckdb/common/radix_partitioning.hpp | 7 +- .../duckdb/common/types/data_chunk.hpp | 14 ++-- .../common/types/partitioned_column_data.hpp | 81 +++++++++++-------- .../duckdb/common/types/vector_cache.hpp | 7 +- .../duckdb/execution/join_hashtable.hpp | 7 +- 10 files changed, 163 insertions(+), 99 deletions(-) diff --git a/src/common/radix_partitioning.cpp b/src/common/radix_partitioning.cpp index 360caf83582c..7cb6b081873b 100644 --- a/src/common/radix_partitioning.cpp +++ b/src/common/radix_partitioning.cpp @@ -411,31 +411,39 @@ void RadixPartitioning::PartitionRowData(BufferManager &buffer_manager, const Ro //===--------------------------------------------------------------------===// RadixPartitionedColumnData::RadixPartitionedColumnData(ClientContext &context_p, vector types_p, idx_t radix_bits_p, idx_t hash_col_idx_p) - : PartitionedColumnData(context_p, move(types_p)), radix_bits(radix_bits_p), hash_col_idx(hash_col_idx_p) { + : PartitionedColumnData(PartitionedColumnDataType::RADIX, context_p, move(types_p)), radix_bits(radix_bits_p), + hash_col_idx(hash_col_idx_p) { D_ASSERT(hash_col_idx < types.size()); + // We know the number of partitions beforehand, so we can just create them const auto num_partitions = RadixPartitioning::NumberOfPartitions(radix_bits); - partition_allocators.reserve(num_partitions); + + allocators->allocators.reserve(num_partitions); for (idx_t i = 0; i < num_partitions; i++) { - partition_allocators.emplace_back(make_shared(BufferManager::GetBufferManager(context))); - partition_allocators.back()->MakeShared(); + CreateAllocator(); + } + D_ASSERT(allocators->allocators.size() == num_partitions); +} + +RadixPartitionedColumnData::RadixPartitionedColumnData(const RadixPartitionedColumnData &other) + : PartitionedColumnData(other), radix_bits(other.radix_bits), hash_col_idx(other.hash_col_idx) { + for (idx_t i = 0; i < RadixPartitioning::NumberOfPartitions(radix_bits); i++) { + partitions.emplace_back(CreateCollectionForPartition(i)); } } RadixPartitionedColumnData::~RadixPartitionedColumnData() { } -void RadixPartitionedColumnData::InitializeAppendStateInternal(PartitionedColumnDataAppendState &state) { +void RadixPartitionedColumnData::InitializeAppendStateInternal(PartitionedColumnDataAppendState &state) const { // We know the number of partitions beforehand, so we can just initialize them const auto num_partitions = RadixPartitioning::NumberOfPartitions(radix_bits); state.partition_buffers.reserve(num_partitions); - state.partitions.reserve(num_partitions); state.partition_append_states.reserve(num_partitions); for (idx_t i = 0; i < num_partitions; i++) { - state.partition_buffers.emplace_back(CreateAppendPartitionBuffer()); - state.partitions.emplace_back(CreateAppendPartition(i)); state.partition_append_states.emplace_back(); - state.partitions.back()->InitializeAppend(state.partition_append_states.back()); + partitions[i]->InitializeAppend(state.partition_append_states[i]); + state.partition_buffers.emplace_back(CreateAppendPartitionBuffer()); } } @@ -450,7 +458,7 @@ struct ComputePartitionIndicesFunctor { }; void RadixPartitionedColumnData::ComputePartitionIndices(PartitionedColumnDataAppendState &state, DataChunk &input) { - D_ASSERT(state.partitions.size() == RadixPartitioning::NumberOfPartitions(radix_bits)); + D_ASSERT(partitions.size() == RadixPartitioning::NumberOfPartitions(radix_bits)); D_ASSERT(state.partition_buffers.size() == RadixPartitioning::NumberOfPartitions(radix_bits)); RadixBitsSwitch(radix_bits, input.data[hash_col_idx], state.partition_indices, input.size()); diff --git a/src/common/types/data_chunk.cpp b/src/common/types/data_chunk.cpp index 18ea638001a7..ad992f269cc9 100644 --- a/src/common/types/data_chunk.cpp +++ b/src/common/types/data_chunk.cpp @@ -37,12 +37,12 @@ void DataChunk::InitializeEmpty(const vector &types) { } } -void DataChunk::Initialize(Allocator &allocator, const vector &types) { +void DataChunk::Initialize(Allocator &allocator, const vector &types, idx_t capacity_p) { D_ASSERT(data.empty()); // can only be initialized once D_ASSERT(!types.empty()); // empty chunk not allowed - capacity = STANDARD_VECTOR_SIZE; + capacity = capacity_p; for (idx_t i = 0; i < types.size(); i++) { - VectorCache cache(allocator, types[i]); + VectorCache cache(allocator, types[i], capacity); data.emplace_back(cache); vector_caches.push_back(move(cache)); } diff --git a/src/common/types/partitioned_column_data.cpp b/src/common/types/partitioned_column_data.cpp index edd37b703dc1..2fadc53072e8 100644 --- a/src/common/types/partitioned_column_data.cpp +++ b/src/common/types/partitioned_column_data.cpp @@ -1,9 +1,25 @@ #include "duckdb/common/types/partitioned_column_data.hpp" +#include "duckdb/common/radix_partitioning.hpp" + namespace duckdb { -PartitionedColumnData::PartitionedColumnData(ClientContext &context_p, vector types_p) - : context(context_p), types(move(types_p)) { +PartitionedColumnData::PartitionedColumnData(PartitionedColumnDataType type_p, ClientContext &context_p, + vector types_p) + : type(type_p), context(context_p), types(move(types_p)), allocators(make_shared()) { +} + +PartitionedColumnData::PartitionedColumnData(const PartitionedColumnData &other) + : type(other.type), context(other.context), types(other.types), allocators(other.allocators) { +} + +unique_ptr PartitionedColumnData::CreateShared() { + switch (type) { + case PartitionedColumnDataType::RADIX: + return make_unique((RadixPartitionedColumnData &)*this); + default: + throw NotImplementedException("CreateShared for this type of PartitionedColumnData"); + } } PartitionedColumnData::~PartitionedColumnData() { @@ -15,7 +31,7 @@ void PartitionedColumnData::InitializeAppendState(PartitionedColumnDataAppendSta InitializeAppendStateInternal(state); } -void PartitionedColumnData::AppendChunk(PartitionedColumnDataAppendState &state, DataChunk &input) { +void PartitionedColumnData::Append(PartitionedColumnDataAppendState &state, DataChunk &input) { // Compute partition indices and store them in state.partition_indices ComputePartitionIndices(state, input); @@ -36,7 +52,7 @@ void PartitionedColumnData::AppendChunk(PartitionedColumnDataAppendState &state, // Early out: check if everything belongs to a single partition if (partition_entries.size() == 1) { const auto &partition_index = partition_entries.begin()->first; - auto &partition = *state.partitions[partition_index]; + auto &partition = *partitions[partition_index]; auto &partition_append_state = state.partition_append_states[partition_index]; partition.Append(partition_append_state, input); return; @@ -64,7 +80,7 @@ void PartitionedColumnData::AppendChunk(PartitionedColumnDataAppendState &state, const auto &partition_index = pc.first; // Partition, buffer, and append state for this partition index - auto &partition = *state.partitions[partition_index]; + auto &partition = *partitions[partition_index]; auto &partition_buffer = *state.partition_buffers[partition_index]; auto &partition_append_state = state.partition_append_states[partition_index]; @@ -76,7 +92,7 @@ void PartitionedColumnData::AppendChunk(PartitionedColumnDataAppendState &state, // Create a selection vector for this partition using the offset into the single selection vector partition_sel.Initialize(all_partitions_sel.data() + partition_offset); - if (partition_length >= 64) { + if (partition_length >= HalfBufferSize()) { // Slice the input chunk using the selection vector state.slice_chunk.Reset(); state.slice_chunk.Slice(input, partition_sel, partition_length); @@ -87,37 +103,42 @@ void PartitionedColumnData::AppendChunk(PartitionedColumnDataAppendState &state, // Append the input chunk to the partition buffer using the selection vector partition_buffer.Append(input, false, &partition_sel, partition_length); - if (partition_buffer.size() >= 64) { + if (partition_buffer.size() >= HalfBufferSize()) { // Next batch won't fit in the buffer, flush it to the partition partition.Append(partition_append_state, partition_buffer); - partition_buffer.Reset(); // TODO: Reset sets the capacity back to STANDARD_VECTOR_SIZE + partition_buffer.Reset(); + partition_buffer.SetCapacity(BufferSize()); } } } } -void PartitionedColumnData::CombineLocalState(PartitionedColumnDataAppendState &state) { - // Flush any remaining data in the buffers - D_ASSERT(state.partition_buffers.size() == state.partitions.size()); - for (idx_t i = 0; i < state.partitions.size(); i++) { +void PartitionedColumnData::FlushAppendState(PartitionedColumnDataAppendState &state) { + for (idx_t i = 0; i < partitions.size(); i++) { auto &partition_buffer = *state.partition_buffers[i]; if (partition_buffer.size() > 0) { - state.partitions[i]->Append(partition_buffer); + partitions[i]->Append(partition_buffer); } } +} +void PartitionedColumnData::Combine(PartitionedColumnData &other) { // Now combine the state's partitions into this lock_guard guard(lock); - D_ASSERT(state.partitions.size() == NumberOfPartitions()); if (partitions.empty()) { // This is the first merge, we just copy them over - partitions = move(state.partitions); + partitions = move(other.partitions); } else { // Combine the append state's partitions into this PartitionedColumnData for (idx_t i = 0; i < NumberOfPartitions(); i++) { - partitions[i]->Combine(*state.partitions[i]); + partitions[i]->Combine(*other.partitions[i]); } } } +void PartitionedColumnData::CreateAllocator() { + allocators->allocators.emplace_back(make_shared(BufferManager::GetBufferManager(context))); + allocators->allocators.back()->MakeShared(); +} + } // namespace duckdb diff --git a/src/common/types/vector_cache.cpp b/src/common/types/vector_cache.cpp index 7537cf225781..d32fb626d789 100644 --- a/src/common/types/vector_cache.cpp +++ b/src/common/types/vector_cache.cpp @@ -1,18 +1,19 @@ #include "duckdb/common/types/vector_cache.hpp" -#include "duckdb/common/types/vector.hpp" + #include "duckdb/common/allocator.hpp" +#include "duckdb/common/types/vector.hpp" namespace duckdb { class VectorCacheBuffer : public VectorBuffer { public: - explicit VectorCacheBuffer(Allocator &allocator, const LogicalType &type_p) - : VectorBuffer(VectorBufferType::OPAQUE_BUFFER), type(type_p) { + explicit VectorCacheBuffer(Allocator &allocator, const LogicalType &type_p, idx_t capacity_p = STANDARD_VECTOR_SIZE) + : VectorBuffer(VectorBufferType::OPAQUE_BUFFER), type(type_p), capacity(capacity_p) { auto internal_type = type.InternalType(); switch (internal_type) { case PhysicalType::LIST: { // memory for the list offsets - owned_data = allocator.Allocate(STANDARD_VECTOR_SIZE * GetTypeIdSize(internal_type)); + owned_data = allocator.Allocate(capacity * GetTypeIdSize(internal_type)); // child data of the list auto &child_type = ListType::GetChildType(type); child_caches.push_back(make_buffer(allocator, child_type)); @@ -30,7 +31,7 @@ class VectorCacheBuffer : public VectorBuffer { break; } default: - owned_data = allocator.Allocate(STANDARD_VECTOR_SIZE * GetTypeIdSize(internal_type)); + owned_data = allocator.Allocate(capacity * GetTypeIdSize(internal_type)); break; } } @@ -48,7 +49,7 @@ class VectorCacheBuffer : public VectorBuffer { AssignSharedPointer(result.auxiliary, auxiliary); // propagate through child auto &list_buffer = (VectorListBuffer &)*result.auxiliary; - list_buffer.capacity = STANDARD_VECTOR_SIZE; + list_buffer.capacity = capacity; list_buffer.size = 0; list_buffer.SetAuxiliaryData(nullptr); @@ -92,10 +93,12 @@ class VectorCacheBuffer : public VectorBuffer { vector> child_caches; //! Aux data for the vector (if any) buffer_ptr auxiliary; + //! Capacity of the vector + idx_t capacity; }; -VectorCache::VectorCache(Allocator &allocator, const LogicalType &type_p) { - buffer = make_unique(allocator, type_p); +VectorCache::VectorCache(Allocator &allocator, const LogicalType &type_p, idx_t capacity_p) { + buffer = make_unique(allocator, type_p, capacity_p); } void VectorCache::ResetFromCache(Vector &result) const { diff --git a/src/execution/join_hashtable.cpp b/src/execution/join_hashtable.cpp index 5a0397c9470a..b7d4c69005af 100644 --- a/src/execution/join_hashtable.cpp +++ b/src/execution/join_hashtable.cpp @@ -1045,7 +1045,7 @@ void JoinHashTable::ComputePartitionSizes(ClientConfig &config, vector(context, probe_types, ht.radix_bits, probe_types.size() - 1); } column_ids.reserve(probe_types.size()); @@ -1208,42 +1208,48 @@ ProbeSpill::ProbeSpill(JoinHashTable &ht, ClientContext &context, const vector guard(lock); if (partitioned) { - idx_t thread_idx = partition_append_states.size(); - partition_append_states.emplace_back(make_unique()); - partitioned_data->InitializeAppendState(*partition_append_states.back()); - return thread_idx; + thread_idx = local_partition_append_states.size(); + local_partitions.emplace_back(global_partitions->CreateShared()); + local_partition_append_states.emplace_back(make_unique()); + local_partitions.back()->InitializeAppendState(*local_partition_append_states.back()); } else { - idx_t thread_idx = local_spill_collections.size(); + thread_idx = local_spill_collections.size(); local_spill_collections.emplace_back(make_unique(context, probe_types)); - spill_append_states.emplace_back(make_unique()); - local_spill_collections.back()->InitializeAppend(*spill_append_states.back()); - return thread_idx; + local_spill_append_states.emplace_back(make_unique()); + local_spill_collections.back()->InitializeAppend(*local_spill_append_states.back()); } + return thread_idx; } void ProbeSpill::Append(DataChunk &chunk, idx_t thread_idx) { if (partitioned) { - partitioned_data->AppendChunk(*partition_append_states[thread_idx], chunk); + local_partitions[thread_idx]->Append(*local_partition_append_states[thread_idx], chunk); } else { - local_spill_collections[thread_idx]->Append(*spill_append_states[thread_idx], chunk); + local_spill_collections[thread_idx]->Append(*local_spill_append_states[thread_idx], chunk); } } void ProbeSpill::Finalize() { if (partitioned) { - for (auto &append_state : partition_append_states) { - partitioned_data->CombineLocalState(*append_state); + D_ASSERT(local_partitions.size() == local_partition_append_states.size()); + for (idx_t i = 0; i < local_partition_append_states.size(); i++) { + local_partitions[i]->FlushAppendState(*local_partition_append_states[i]); } - partition_append_states.clear(); + for (auto &local_partition : local_partitions) { + global_partitions->Combine(*local_partition); + } + local_partitions.clear(); + local_spill_append_states.clear(); } else { global_spill_collection = move(local_spill_collections[0]); for (idx_t i = 1; i < local_spill_collections.size(); i++) { global_spill_collection->Combine(*local_spill_collections[i]); } local_spill_collections.clear(); - spill_append_states.clear(); + local_spill_append_states.clear(); } } @@ -1252,7 +1258,7 @@ void ProbeSpill::PrepareNextProbeCollection(JoinHashTable &ht) { current_probe_collection = nullptr; chunk_references.clear(); if (partitioned) { - auto &partitions = partitioned_data->GetPartitions(); + auto &partitions = global_partitions->GetPartitions(); if (ht.partition_start == partitions.size()) { return; } diff --git a/src/include/duckdb/common/radix_partitioning.hpp b/src/include/duckdb/common/radix_partitioning.hpp index 4adaa02cc7c8..5cedb087bfe8 100644 --- a/src/include/duckdb/common/radix_partitioning.hpp +++ b/src/include/duckdb/common/radix_partitioning.hpp @@ -60,9 +60,14 @@ struct RadixPartitioning { class RadixPartitionedColumnData : public PartitionedColumnData { public: RadixPartitionedColumnData(ClientContext &context, vector types, idx_t radix_bits, idx_t hash_col_idx); + RadixPartitionedColumnData(const RadixPartitionedColumnData &other); ~RadixPartitionedColumnData() override; - void InitializeAppendStateInternal(PartitionedColumnDataAppendState &state) override; +protected: + virtual idx_t BufferSize() const override { + return STANDARD_VECTOR_SIZE; + } + void InitializeAppendStateInternal(PartitionedColumnDataAppendState &state) const override; void ComputePartitionIndices(PartitionedColumnDataAppendState &state, DataChunk &input) override; private: diff --git a/src/include/duckdb/common/types/data_chunk.hpp b/src/include/duckdb/common/types/data_chunk.hpp index f5680e8cab7c..9a1fec51fe63 100644 --- a/src/include/duckdb/common/types/data_chunk.hpp +++ b/src/include/duckdb/common/types/data_chunk.hpp @@ -8,11 +8,11 @@ #pragma once +#include "duckdb/common/allocator.hpp" +#include "duckdb/common/arrow/arrow_wrapper.hpp" #include "duckdb/common/common.hpp" #include "duckdb/common/types/vector.hpp" #include "duckdb/common/winapi.hpp" -#include "duckdb/common/allocator.hpp" -#include "duckdb/common/arrow/arrow_wrapper.hpp" struct ArrowArray; @@ -61,10 +61,13 @@ class DataChunk { this->count = count_p; } inline void SetCardinality(const DataChunk &other) { - this->count = other.size(); + SetCardinality(other.size()); + } + inline void SetCapacity(idx_t capacity_p) { + this->capacity = capacity_p; } inline void SetCapacity(const DataChunk &other) { - this->capacity = other.capacity; + SetCapacity(other.capacity); } DUCKDB_API Value GetValue(idx_t col_idx, idx_t index) const; @@ -82,7 +85,8 @@ class DataChunk { //! This will create one vector of the specified type for each LogicalType in the //! types list. The vector will be referencing vector to the data owned by //! the DataChunk. - DUCKDB_API void Initialize(Allocator &allocator, const vector &types); + DUCKDB_API void Initialize(Allocator &allocator, const vector &types, + idx_t capacity = STANDARD_VECTOR_SIZE); DUCKDB_API void Initialize(ClientContext &context, const vector &types); //! Initializes an empty DataChunk with the given types. The vectors will *not* have any data allocated for them. DUCKDB_API void InitializeEmpty(const vector &types); diff --git a/src/include/duckdb/common/types/partitioned_column_data.hpp b/src/include/duckdb/common/types/partitioned_column_data.hpp index 14364c5f4d6a..8dcbe5a8d789 100644 --- a/src/include/duckdb/common/types/partitioned_column_data.hpp +++ b/src/include/duckdb/common/types/partitioned_column_data.hpp @@ -14,10 +14,12 @@ namespace duckdb { +enum class PartitionedColumnDataType : uint8_t { RADIX, INVALID }; + struct PartitionedColumnDataAppendState; class PartitionedColumnData; -//! Local state for partioning in parallel +//! Local state for parallel partitioning struct PartitionedColumnDataAppendState { public: explicit PartitionedColumnDataAppendState() : partition_indices(LogicalType::UBIGINT) { @@ -29,7 +31,6 @@ struct PartitionedColumnDataAppendState { DataChunk slice_chunk; vector> partition_buffers; - vector> partitions; vector partition_append_states; private: @@ -37,25 +38,23 @@ struct PartitionedColumnDataAppendState { PartitionedColumnDataAppendState(const PartitionedColumnDataAppendState &) = delete; }; -//! PartitionedColumnData represents partitioned columnar data, which serves as an interface for different flavors of +//! Shared allocators for parallel partitioning +struct PartitionAllocators { + mutex lock; + vector> allocators; +}; + +//! PartitionedColumnData represents partitioned columnar data, which serves as an interface for different types of //! partitioning, e.g., radix, hive class PartitionedColumnData { public: - PartitionedColumnData(ClientContext &context, vector types); + unique_ptr CreateShared(); virtual ~PartitionedColumnData(); public: - //! The types of columns in the PartitionedColumnData - const vector &Types() const { - return types; - } - //! The number of columns in the PartitionedColumnData - idx_t ColumnCount() const { - return types.size(); - } //! The number of partitions in the PartitionedColumnData idx_t NumberOfPartitions() const { - return partition_allocators.size(); + return allocators->allocators.size(); } //! The partitions in this PartitionedColumnData vector> &GetPartitions() { @@ -65,46 +64,62 @@ class PartitionedColumnData { public: //! Initializes a local state for parallel partitioning that can be merged into this PartitionedColumnData void InitializeAppendState(PartitionedColumnDataAppendState &state); - - //! Appends a DataChunk to the PartitionedColumnDataAppendState - virtual void AppendChunk(PartitionedColumnDataAppendState &state, DataChunk &input); - - //! Combine a local state into this PartitionedColumnData - void CombineLocalState(PartitionedColumnDataAppendState &state); + //! Appends a DataChunk to this PartitionedColumnData + void Append(PartitionedColumnDataAppendState &state, DataChunk &input); + //! Flushes any remaining data in the append state into this PartitionedColumnData + void FlushAppendState(PartitionedColumnDataAppendState &state); + //! Combine another PartitionedColumnData into this PartitionedColumnData + void Combine(PartitionedColumnData &other); private: //===--------------------------------------------------------------------===// - // Partitioning flavor implementation interface + // Partitioning type implementation interface //===--------------------------------------------------------------------===// - //! Initialize a PartitionedColumnDataAppendState for this flavor of partitioning (optional) - virtual void InitializeAppendStateInternal(PartitionedColumnDataAppendState &state) { + //! Size of the buffers in the append states for this type of partitioning + virtual idx_t BufferSize() const { + return 128; } - - //! Compute the partition indices for this flavor of partioning for the input DataChunk and store them in the - //! `partition_data` of the local state. If this flavor creates partitions on the fly (for, e.g., hive), this + //! Initialize a PartitionedColumnDataAppendState for this type of partitioning (optional) + virtual void InitializeAppendStateInternal(PartitionedColumnDataAppendState &state) const { + } + //! Compute the partition indices for this type of partioning for the input DataChunk and store them in the + //! `partition_data` of the local state. If this type creates partitions on the fly (for, e.g., hive), this //! function is also in charge of creating new partitions and mapping the input data to a partition index virtual void ComputePartitionIndices(PartitionedColumnDataAppendState &state, DataChunk &input) { - throw NotImplementedException("ComputePartitionIndices for this flavor of PartitionedColumnData"); + throw NotImplementedException("ComputePartitionIndices for this type of PartitionedColumnData"); } protected: - unique_ptr CreateAppendPartition(idx_t partition_index) { - return make_unique(partition_allocators[partition_index], types); + //! PartitionedColumnData can only be instantiated by derived classes + PartitionedColumnData(PartitionedColumnDataType type, ClientContext &context, vector types); + PartitionedColumnData(const PartitionedColumnData &other); + + inline idx_t HalfBufferSize() const { + // Buffersize should be a power of two + D_ASSERT((BufferSize() & (BufferSize() - 1)) == 0); + return BufferSize() / 2; } - - unique_ptr CreateAppendPartitionBuffer() { + //! Create a new shared allocator + void CreateAllocator(); + //! Create a collection for a specific a partition + unique_ptr CreateCollectionForPartition(idx_t partition_index) const { + return make_unique(allocators->allocators[partition_index], types); + } + //! Create a DataChunk used for buffering appends to the partition + unique_ptr CreateAppendPartitionBuffer() const { auto result = make_unique(); - result->Initialize(Allocator::Get(context), types); + result->Initialize(Allocator::Get(context), types, BufferSize()); return result; } protected: + PartitionedColumnDataType type; ClientContext &context; vector types; - vector> partition_allocators; - vector> partitions; mutex lock; + shared_ptr allocators; + vector> partitions; }; } // namespace duckdb diff --git a/src/include/duckdb/common/types/vector_cache.hpp b/src/include/duckdb/common/types/vector_cache.hpp index 799a38138373..333950ce7e56 100644 --- a/src/include/duckdb/common/types/vector_cache.hpp +++ b/src/include/duckdb/common/types/vector_cache.hpp @@ -9,8 +9,8 @@ #pragma once #include "duckdb/common/types.hpp" -#include "duckdb/common/vector.hpp" #include "duckdb/common/types/vector_buffer.hpp" +#include "duckdb/common/vector.hpp" namespace duckdb { class Allocator; @@ -19,8 +19,9 @@ class Vector; //! The VectorCache holds cached data that allows for re-use of the same memory by vectors class VectorCache { public: - //! Instantiate a vector cache with the given type - DUCKDB_API explicit VectorCache(Allocator &allocator, const LogicalType &type); + //! Instantiate a vector cache with the given type and capacity + DUCKDB_API explicit VectorCache(Allocator &allocator, const LogicalType &type, + idx_t capacity = STANDARD_VECTOR_SIZE); buffer_ptr buffer; diff --git a/src/include/duckdb/execution/join_hashtable.hpp b/src/include/duckdb/execution/join_hashtable.hpp index 1417d1637e3b..d71d96a7d8ee 100644 --- a/src/include/duckdb/execution/join_hashtable.hpp +++ b/src/include/duckdb/execution/join_hashtable.hpp @@ -273,13 +273,14 @@ class JoinHashTable { bool partitioned; //! The partitioned probe data (if partitioned) and append states - unique_ptr partitioned_data; - vector> partition_append_states; + unique_ptr global_partitions; + vector> local_partitions; + vector> local_partition_append_states; //! The probe data (if not partitioned) and append states unique_ptr global_spill_collection; vector> local_spill_collections; - vector> spill_append_states; + vector> local_spill_append_states; //! The probe collection currently being read unique_ptr current_probe_collection; From e2cfe1c9dfe6eaad730ca6d8f15c861dbfcbab02 Mon Sep 17 00:00:00 2001 From: Laurens Kuiper Date: Wed, 5 Oct 2022 16:30:12 +0200 Subject: [PATCH 13/49] some code cleanup --- src/common/radix_partitioning.cpp | 8 ++--- src/common/types/partitioned_column_data.cpp | 12 ++++--- .../duckdb/common/radix_partitioning.hpp | 6 +++- .../common/types/partitioned_column_data.hpp | 34 ++++++------------- 4 files changed, 26 insertions(+), 34 deletions(-) diff --git a/src/common/radix_partitioning.cpp b/src/common/radix_partitioning.cpp index 7cb6b081873b..c9bd0c58e304 100644 --- a/src/common/radix_partitioning.cpp +++ b/src/common/radix_partitioning.cpp @@ -414,10 +414,7 @@ RadixPartitionedColumnData::RadixPartitionedColumnData(ClientContext &context_p, : PartitionedColumnData(PartitionedColumnDataType::RADIX, context_p, move(types_p)), radix_bits(radix_bits_p), hash_col_idx(hash_col_idx_p) { D_ASSERT(hash_col_idx < types.size()); - - // We know the number of partitions beforehand, so we can just create them const auto num_partitions = RadixPartitioning::NumberOfPartitions(radix_bits); - allocators->allocators.reserve(num_partitions); for (idx_t i = 0; i < num_partitions; i++) { CreateAllocator(); @@ -428,7 +425,7 @@ RadixPartitionedColumnData::RadixPartitionedColumnData(ClientContext &context_p, RadixPartitionedColumnData::RadixPartitionedColumnData(const RadixPartitionedColumnData &other) : PartitionedColumnData(other), radix_bits(other.radix_bits), hash_col_idx(other.hash_col_idx) { for (idx_t i = 0; i < RadixPartitioning::NumberOfPartitions(radix_bits); i++) { - partitions.emplace_back(CreateCollectionForPartition(i)); + partitions.emplace_back(CreatePartitionCollection(i)); } } @@ -436,14 +433,13 @@ RadixPartitionedColumnData::~RadixPartitionedColumnData() { } void RadixPartitionedColumnData::InitializeAppendStateInternal(PartitionedColumnDataAppendState &state) const { - // We know the number of partitions beforehand, so we can just initialize them const auto num_partitions = RadixPartitioning::NumberOfPartitions(radix_bits); state.partition_buffers.reserve(num_partitions); state.partition_append_states.reserve(num_partitions); for (idx_t i = 0; i < num_partitions; i++) { state.partition_append_states.emplace_back(); partitions[i]->InitializeAppend(state.partition_append_states[i]); - state.partition_buffers.emplace_back(CreateAppendPartitionBuffer()); + state.partition_buffers.emplace_back(CreatePartitionBuffer()); } } diff --git a/src/common/types/partitioned_column_data.cpp b/src/common/types/partitioned_column_data.cpp index 2fadc53072e8..e3aa91e9cb65 100644 --- a/src/common/types/partitioned_column_data.cpp +++ b/src/common/types/partitioned_column_data.cpp @@ -25,7 +25,7 @@ unique_ptr PartitionedColumnData::CreateShared() { PartitionedColumnData::~PartitionedColumnData() { } -void PartitionedColumnData::InitializeAppendState(PartitionedColumnDataAppendState &state) { +void PartitionedColumnData::InitializeAppendState(PartitionedColumnDataAppendState &state) const { state.partition_sel.Initialize(); state.slice_chunk.Initialize(context, types); InitializeAppendStateInternal(state); @@ -97,7 +97,7 @@ void PartitionedColumnData::Append(PartitionedColumnDataAppendState &state, Data state.slice_chunk.Reset(); state.slice_chunk.Slice(input, partition_sel, partition_length); - // Append it to the partition + // Append it to the partition directly partition.Append(partition_append_state, state.slice_chunk); } else { // Append the input chunk to the partition buffer using the selection vector @@ -114,7 +114,7 @@ void PartitionedColumnData::Append(PartitionedColumnDataAppendState &state, Data } void PartitionedColumnData::FlushAppendState(PartitionedColumnDataAppendState &state) { - for (idx_t i = 0; i < partitions.size(); i++) { + for (idx_t i = 0; i < state.partition_buffers.size(); i++) { auto &partition_buffer = *state.partition_buffers[i]; if (partition_buffer.size() > 0) { partitions[i]->Append(partition_buffer); @@ -130,12 +130,16 @@ void PartitionedColumnData::Combine(PartitionedColumnData &other) { partitions = move(other.partitions); } else { // Combine the append state's partitions into this PartitionedColumnData - for (idx_t i = 0; i < NumberOfPartitions(); i++) { + for (idx_t i = 0; i < other.partitions.size(); i++) { partitions[i]->Combine(*other.partitions[i]); } } } +vector> &PartitionedColumnData::GetPartitions() { + return partitions; +} + void PartitionedColumnData::CreateAllocator() { allocators->allocators.emplace_back(make_shared(BufferManager::GetBufferManager(context))); allocators->allocators.back()->MakeShared(); diff --git a/src/include/duckdb/common/radix_partitioning.hpp b/src/include/duckdb/common/radix_partitioning.hpp index 5cedb087bfe8..ffb8f5973e7b 100644 --- a/src/include/duckdb/common/radix_partitioning.hpp +++ b/src/include/duckdb/common/radix_partitioning.hpp @@ -57,6 +57,7 @@ struct RadixPartitioning { vector> &partition_string_heaps, idx_t radix_bits); }; +//! RadixPartitionedColumnData is a PartitionedColumnData that partitions input based on the radix of a hash class RadixPartitionedColumnData : public PartitionedColumnData { public: RadixPartitionedColumnData(ClientContext &context, vector types, idx_t radix_bits, idx_t hash_col_idx); @@ -64,7 +65,10 @@ class RadixPartitionedColumnData : public PartitionedColumnData { ~RadixPartitionedColumnData() override; protected: - virtual idx_t BufferSize() const override { + //===--------------------------------------------------------------------===// + // Radix Partitioning interface implementation + //===--------------------------------------------------------------------===// + idx_t BufferSize() const override { return STANDARD_VECTOR_SIZE; } void InitializeAppendStateInternal(PartitionedColumnDataAppendState &state) const override; diff --git a/src/include/duckdb/common/types/partitioned_column_data.hpp b/src/include/duckdb/common/types/partitioned_column_data.hpp index 8dcbe5a8d789..88f1a3cb73ff 100644 --- a/src/include/duckdb/common/types/partitioned_column_data.hpp +++ b/src/include/duckdb/common/types/partitioned_column_data.hpp @@ -8,17 +8,11 @@ #pragma once -#include "duckdb/common/pair.hpp" #include "duckdb/common/types/column_data_allocator.hpp" #include "duckdb/common/types/column_data_collection.hpp" namespace duckdb { -enum class PartitionedColumnDataType : uint8_t { RADIX, INVALID }; - -struct PartitionedColumnDataAppendState; -class PartitionedColumnData; - //! Local state for parallel partitioning struct PartitionedColumnDataAppendState { public: @@ -38,6 +32,8 @@ struct PartitionedColumnDataAppendState { PartitionedColumnDataAppendState(const PartitionedColumnDataAppendState &) = delete; }; +enum class PartitionedColumnDataType : uint8_t { RADIX, INVALID }; + //! Shared allocators for parallel partitioning struct PartitionAllocators { mutex lock; @@ -51,38 +47,30 @@ class PartitionedColumnData { unique_ptr CreateShared(); virtual ~PartitionedColumnData(); -public: - //! The number of partitions in the PartitionedColumnData - idx_t NumberOfPartitions() const { - return allocators->allocators.size(); - } - //! The partitions in this PartitionedColumnData - vector> &GetPartitions() { - return partitions; - } - public: //! Initializes a local state for parallel partitioning that can be merged into this PartitionedColumnData - void InitializeAppendState(PartitionedColumnDataAppendState &state); + void InitializeAppendState(PartitionedColumnDataAppendState &state) const; //! Appends a DataChunk to this PartitionedColumnData void Append(PartitionedColumnDataAppendState &state, DataChunk &input); //! Flushes any remaining data in the append state into this PartitionedColumnData void FlushAppendState(PartitionedColumnDataAppendState &state); //! Combine another PartitionedColumnData into this PartitionedColumnData void Combine(PartitionedColumnData &other); + //! Get the partitions in this PartitionedColumnData + vector> &GetPartitions(); private: //===--------------------------------------------------------------------===// // Partitioning type implementation interface //===--------------------------------------------------------------------===// - //! Size of the buffers in the append states for this type of partitioning + //! Size of the buffers in the append states for this type of partitioning (default 128) virtual idx_t BufferSize() const { return 128; } //! Initialize a PartitionedColumnDataAppendState for this type of partitioning (optional) virtual void InitializeAppendStateInternal(PartitionedColumnDataAppendState &state) const { } - //! Compute the partition indices for this type of partioning for the input DataChunk and store them in the + //! Compute the partition indices for this type of partitioning for the input DataChunk and store them in the //! `partition_data` of the local state. If this type creates partitions on the fly (for, e.g., hive), this //! function is also in charge of creating new partitions and mapping the input data to a partition index virtual void ComputePartitionIndices(PartitionedColumnDataAppendState &state, DataChunk &input) { @@ -94,19 +82,19 @@ class PartitionedColumnData { PartitionedColumnData(PartitionedColumnDataType type, ClientContext &context, vector types); PartitionedColumnData(const PartitionedColumnData &other); + //! If the buffer is half full, we append to the partition inline idx_t HalfBufferSize() const { - // Buffersize should be a power of two - D_ASSERT((BufferSize() & (BufferSize() - 1)) == 0); + D_ASSERT((BufferSize() & (BufferSize() - 1)) == 0); // BufferSize should be a power of two return BufferSize() / 2; } //! Create a new shared allocator void CreateAllocator(); //! Create a collection for a specific a partition - unique_ptr CreateCollectionForPartition(idx_t partition_index) const { + unique_ptr CreatePartitionCollection(idx_t partition_index) const { return make_unique(allocators->allocators[partition_index], types); } //! Create a DataChunk used for buffering appends to the partition - unique_ptr CreateAppendPartitionBuffer() const { + unique_ptr CreatePartitionBuffer() const { auto result = make_unique(); result->Initialize(Allocator::Get(context), types, BufferSize()); return result; From 0d4dbbb5d65f0a86988ddf91c8d48008e94723d7 Mon Sep 17 00:00:00 2001 From: Laurens Kuiper Date: Thu, 6 Oct 2022 15:58:05 +0200 Subject: [PATCH 14/49] progress on scanning/consuming a ColumnDataCollection --- src/common/types/column_data_allocator.cpp | 14 ++- src/common/types/column_data_collection.cpp | 24 ++-- .../types/column_data_collection_segment.cpp | 6 - src/execution/join_hashtable.cpp | 105 ++++++++++++------ .../operator/join/physical_hash_join.cpp | 30 +++-- .../common/types/column_data_allocator.hpp | 2 +- .../common/types/column_data_collection.hpp | 24 ++-- .../types/column_data_collection_segment.hpp | 2 - .../common/types/column_data_scan_states.hpp | 2 + .../duckdb/execution/join_hashtable.hpp | 43 +++++-- .../duckdb/storage/buffer/block_handle.hpp | 6 +- 11 files changed, 159 insertions(+), 99 deletions(-) diff --git a/src/common/types/column_data_allocator.cpp b/src/common/types/column_data_allocator.cpp index 5539581ccb6b..0cb2bedefbf7 100644 --- a/src/common/types/column_data_allocator.cpp +++ b/src/common/types/column_data_allocator.cpp @@ -28,13 +28,19 @@ ColumnDataAllocator::ColumnDataAllocator(ClientContext &context, ColumnDataAlloc } } -BufferHandle ColumnDataAllocator::Pin(uint32_t block_id) { +BufferHandle ColumnDataAllocator::Pin(uint32_t block_id, bool consume) { D_ASSERT(type == ColumnDataAllocatorType::BUFFER_MANAGER_ALLOCATOR); if (shared) { lock_guard guard(lock); - return alloc.buffer_manager->Pin(blocks[block_id].handle); + auto &block_handle = blocks[block_id].handle; + auto result = alloc.buffer_manager->Pin(block_handle); + block_handle->SetCanDestroy(consume); + return result; } else { - return alloc.buffer_manager->Pin(blocks[block_id].handle); + auto &block_handle = blocks[block_id].handle; + auto result = alloc.buffer_manager->Pin(block_handle); + block_handle->SetCanDestroy(consume); + return result; } } @@ -143,7 +149,7 @@ void ColumnDataAllocator::InitializeChunkState(ChunkManagementState &state, Chun // already pinned: don't need to do anything continue; } - state.handles[block_id] = Pin(block_id); + state.handles[block_id] = Pin(block_id, state.consume); } } diff --git a/src/common/types/column_data_collection.cpp b/src/common/types/column_data_collection.cpp index ad72e62889a5..c79ae5907e28 100644 --- a/src/common/types/column_data_collection.cpp +++ b/src/common/types/column_data_collection.cpp @@ -593,34 +593,36 @@ void ColumnDataCollection::Append(DataChunk &input) { //===--------------------------------------------------------------------===// // Scan //===--------------------------------------------------------------------===// -void ColumnDataCollection::InitializeScan(ColumnDataScanState &state, ColumnDataScanProperties properties) const { +void ColumnDataCollection::InitializeScan(ColumnDataScanState &state, ColumnDataScanProperties properties, + bool consume) const { vector column_ids; column_ids.reserve(types.size()); for (idx_t i = 0; i < types.size(); i++) { column_ids.push_back(i); } - InitializeScan(state, move(column_ids), properties); + InitializeScan(state, move(column_ids), properties, consume); } void ColumnDataCollection::InitializeScan(ColumnDataScanState &state, vector column_ids, - ColumnDataScanProperties properties) const { + ColumnDataScanProperties properties, bool consume) const { state.chunk_index = 0; state.segment_index = 0; state.current_row_index = 0; state.next_row_index = 0; state.current_chunk_state.handles.clear(); state.properties = properties; + state.consume = consume; state.column_ids = move(column_ids); } -void ColumnDataCollection::InitializeScan(ColumnDataParallelScanState &state, - ColumnDataScanProperties properties) const { - InitializeScan(state.scan_state, properties); +void ColumnDataCollection::InitializeScan(ColumnDataParallelScanState &state, ColumnDataScanProperties properties, + bool consume) const { + InitializeScan(state.scan_state, properties, consume); } void ColumnDataCollection::InitializeScan(ColumnDataParallelScanState &state, vector column_ids, - ColumnDataScanProperties properties) const { - InitializeScan(state.scan_state, move(column_ids), properties); + ColumnDataScanProperties properties, bool consume) const { + InitializeScan(state.scan_state, move(column_ids), properties, consume); } bool ColumnDataCollection::Scan(ColumnDataParallelScanState &state, ColumnDataLocalScanState &lstate, @@ -689,6 +691,7 @@ void ColumnDataCollection::ScanAtIndex(ColumnDataParallelScanState &state, Colum } auto &segment = *segments[segment_index]; lstate.current_chunk_state.properties = state.scan_state.properties; + lstate.current_chunk_state.consume = state.scan_state.consume; segment.ReadChunk(chunk_index, lstate.current_chunk_state, result, state.scan_state.column_ids); lstate.current_row_index = row_index; result.Verify(); @@ -815,6 +818,10 @@ bool ColumnDataCollection::ResultEquals(const ColumnDataCollection &left, const vector> ColumnDataCollection::GetChunkReferences(bool sort) { // Create a vector containing a reference to every chunk vector> result; + if (Count() == 0) { + return result; + } + result.reserve(ChunkCount()); for (auto &segment : segments) { for (idx_t chunk_index = 0; chunk_index < segment->chunk_data.size(); chunk_index++) { @@ -831,6 +838,7 @@ vector> ColumnDataCollection::GetChun const auto &rhs_block_ids = rhs.first->chunk_data[rhs.second].block_ids; const uint32_t lhs_min_block_id = *std::min_element(lhs_block_ids.begin(), lhs_block_ids.end()); const uint32_t rhs_min_block_id = *std::min_element(rhs_block_ids.begin(), rhs_block_ids.end()); + return lhs_min_block_id < rhs_min_block_id; }); } diff --git a/src/common/types/column_data_collection_segment.cpp b/src/common/types/column_data_collection_segment.cpp index a6f19968b498..2d6d771cd596 100644 --- a/src/common/types/column_data_collection_segment.cpp +++ b/src/common/types/column_data_collection_segment.cpp @@ -215,12 +215,6 @@ void ColumnDataCollectionSegment::FetchChunk(idx_t chunk_idx, DataChunk &result, D_ASSERT(chunk_idx < chunk_data.size()); ChunkManagementState state; state.properties = ColumnDataScanProperties::DISALLOW_ZERO_COPY; - FetchChunk(state, chunk_idx, result, column_ids); -} - -void ColumnDataCollectionSegment::FetchChunk(ChunkManagementState &state, idx_t chunk_idx, DataChunk &result, - const vector &column_ids) { - D_ASSERT(chunk_idx < chunk_data.size()); ReadChunk(chunk_idx, state, result, column_ids); } diff --git a/src/execution/join_hashtable.cpp b/src/execution/join_hashtable.cpp index b7d4c69005af..209755175cc8 100644 --- a/src/execution/join_hashtable.cpp +++ b/src/execution/join_hashtable.cpp @@ -14,6 +14,8 @@ namespace duckdb { using ValidityBytes = JoinHashTable::ValidityBytes; using ScanStructure = JoinHashTable::ScanStructure; using ProbeSpill = JoinHashTable::ProbeSpill; +using ProbeSpillLocalScanState = JoinHashTable::ProbeSpillLocalScanState; +using ProbeSpillGlobalScanState = JoinHashTable::ProbeSpillGlobalScanState; JoinHashTable::JoinHashTable(BufferManager &buffer_manager, const vector &conditions, vector btypes, JoinType type) @@ -1191,7 +1193,7 @@ unique_ptr JoinHashTable::ProbeAndSpill(DataChunk &keys, DataChun } ProbeSpill::ProbeSpill(JoinHashTable &ht, ClientContext &context, const vector &probe_types) - : context(context), probe_types(probe_types) { + : ht(ht), context(context), probe_types(probe_types) { if (ht.total_count - ht.Count() <= ht.tuples_per_round) { // No need to partition as we will only have one more probe round partitioned = false; @@ -1253,57 +1255,86 @@ void ProbeSpill::Finalize() { } } -void ProbeSpill::PrepareNextProbeCollection(JoinHashTable &ht) { - // Reset previous probe collection - current_probe_collection = nullptr; - chunk_references.clear(); +idx_t ProbeSpill::ChunkCount(ProbeSpillGlobalScanState &gstate) const { + if (partitioned) { + idx_t count = 0; + for (idx_t i = ht.partition_start; i < ht.partition_end; i++) { + count += gstate.partition_chunk_references[i].size(); + } + return count; + } else { + return global_spill_collection->ChunkCount(); + } +} + +void ProbeSpill::PrepareNextProbe(ProbeSpillGlobalScanState &gstate) { if (partitioned) { auto &partitions = global_partitions->GetPartitions(); + if (gstate.partition_chunk_references.empty()) { + // First call to prepare - initialize chunk references for all partitions + for (idx_t i = 0; i < partitions.size(); i++) { + gstate.partition_chunk_references.push_back(partitions[i]->GetChunkReferences(true)); + } + } + if (ht.partition_start == partitions.size()) { return; } - // Pre-allocate - idx_t chunks = 0; - for (idx_t p_idx = ht.partition_start; p_idx < ht.partition_end; p_idx++) { - chunks += partitions[p_idx]->ChunkCount(); + // Reset indices + gstate.probe_partition_idx = ht.partition_start; + gstate.probe_chunk_idx = 0; + } else { + global_spill_collection->InitializeScan(gstate.scan_state, ColumnDataScanProperties::ALLOW_ZERO_COPY, true); + } +} + +bool ProbeSpill::GetScanIndex(ProbeSpillGlobalScanState &gstate, ProbeSpillLocalScanState &lstate) { + if (partitioned) { + // Check if done + if (gstate.probe_partition_idx == ht.partition_end) { + return false; } - chunk_references.reserve(chunks); - // Get sorted chunk references per partition - for (idx_t p_idx = ht.partition_start; p_idx < ht.partition_end; p_idx++) { - auto partition_chunk_references = partitions[p_idx]->GetChunkReferences(true); - chunk_references.insert(chunk_references.end(), partition_chunk_references.begin(), - partition_chunk_references.end()); + // Get indices + auto next_partition_index = gstate.probe_partition_idx; + auto next_chunk_index = gstate.probe_chunk_idx++; + + if (lstate.probe_partition_index != next_partition_index) { + // Previously scanned a different partition, need to reset local state pins + lstate.scan_state.current_chunk_state.handles.clear(); } - // TODO: what to do with partition collections? + // Assign them to local state + lstate.probe_partition_index = next_partition_index; + lstate.probe_chunk_index = next_chunk_index; + + // Increment to next partition, if needed + if (gstate.probe_chunk_idx == gstate.partition_chunk_references[gstate.probe_partition_idx].size()) { + gstate.probe_partition_idx++; + gstate.probe_chunk_idx = 0; + } + return true; } else { - current_probe_collection = move(global_spill_collection); - chunk_references = current_probe_collection->GetChunkReferences(false); + return global_spill_collection->NextScanIndex(gstate.scan_state.scan_state, lstate.probe_chunk_index, + lstate.probe_segment_index, lstate.probe_row_index); } } -void ProbeSpill::ScanChunk(ChunkManagementState &state, idx_t chunk_idx, DataChunk &chunk) { - auto &chunk_ref = chunk_references[chunk_idx]; - auto &segment = *chunk_ref.first; - auto &idx_within_segment = chunk_ref.second; - - // TODO: delete handles of passed blocks - // Difficulties: - // - segment contains a ColumnDataAllocator and a vector of ChunkMetaData, which is indexed by idx_within_segment - // - The ChunkMetaData contains a vector of block IDs that must be kept alive - // - The block IDs index the vector of BlockMetaData in the ColumnDataAllocator, which is a private field ... - // - We've sorted these chunk references by their minimum block ID - // Solution? - // - When the minimum block ID of a ChunkMetaData is N, we can delete all BlockMetaData up to index N - // - When the allocator changes, we can delete it! - Also need to reset handles in local scan state - // Plan of attack: Add InitializeScanAndConsume to ColumnDataCollection? Or a bool "consume" to InitializeScan - // Need a few fields in the scan states to accomplish this - - state.handles.clear(); // TODO: don't clear if same allocator as before +void ProbeSpill::ScanChunk(ProbeSpillGlobalScanState &gstate, ProbeSpillLocalScanState &lstate, DataChunk &chunk) { chunk.Reset(); - segment.FetchChunk(state, idx_within_segment, chunk, column_ids); + if (partitioned) { + auto &chunk_ref = gstate.partition_chunk_references[lstate.probe_partition_index][lstate.probe_chunk_index]; + auto &segment = *chunk_ref.first; + auto &idx_within_segment = chunk_ref.second; + + segment.ReadChunk(idx_within_segment, lstate.scan_state.current_chunk_state, chunk, column_ids); + } else { + D_ASSERT(gstate.scan_state.scan_state.consume); + D_ASSERT(gstate.scan_state.scan_state.properties == ColumnDataScanProperties::ALLOW_ZERO_COPY); + global_spill_collection->ScanAtIndex(gstate.scan_state, lstate.scan_state, chunk, lstate.probe_chunk_index, + lstate.probe_segment_index, lstate.probe_row_index); + } chunk.Verify(); } diff --git a/src/execution/operator/join/physical_hash_join.cpp b/src/execution/operator/join/physical_hash_join.cpp index d3a6bfbf5d66..c656adecaf97 100644 --- a/src/execution/operator/join/physical_hash_join.cpp +++ b/src/execution/operator/join/physical_hash_join.cpp @@ -548,8 +548,7 @@ class HashJoinGlobalSourceState : public GlobalSourceState { idx_t build_blocks_per_thread; //! For probe synchronization - ColumnDataParallelScanState probe_global_scan; - idx_t probe_chunk_idx; + JoinHashTable::ProbeSpillGlobalScanState probe_global_scan; idx_t probe_chunk_count; idx_t probe_chunk_done; @@ -587,10 +586,8 @@ class HashJoinLocalSourceState : public LocalSourceState { idx_t build_block_idx_start; idx_t build_block_idx_end; - //! Local scan state for probe collection - ChunkManagementState probe_chunk_management_state; - //! Index of probe chunk to scan next - idx_t probe_chunk_index; + //! Local scan state for probe spill + JoinHashTable::ProbeSpillLocalScanState probe_local_scan; //! Chunks for holding the scanned probe collection DataChunk probe_chunk; DataChunk join_keys; @@ -616,7 +613,7 @@ unique_ptr PhysicalHashJoin::GetLocalSourceState(ExecutionCont } HashJoinGlobalSourceState::HashJoinGlobalSourceState(const PhysicalHashJoin &op, ClientContext &context) - : op(op), initialized(false), global_stage(HashJoinSourceStage::INIT), lock(probe_global_scan.lock), + : op(op), initialized(false), global_stage(HashJoinSourceStage::INIT), lock(probe_global_scan.scan_state.lock), probe_count(op.children[0]->estimated_cardinality), parallel_scan_chunk_count(context.config.verify_parallelism ? 1 : 120) { } @@ -646,7 +643,7 @@ void HashJoinGlobalSourceState::PrepareBuild(HashJoinGlobalSinkState &sink) { D_ASSERT(global_stage != HashJoinSourceStage::BUILD); auto &ht = *sink.hash_table; - // Put the next partitions in the block collection + // Try to put the next partitions in the block collection of the HT if (!ht.PrepareExternalFinalize()) { global_stage = HashJoinSourceStage::DONE; return; @@ -662,10 +659,9 @@ void HashJoinGlobalSourceState::PrepareBuild(HashJoinGlobalSinkState &sink) { } void HashJoinGlobalSourceState::PrepareProbe(HashJoinGlobalSinkState &sink) { - sink.probe_spill->PrepareNextProbeCollection(*sink.hash_table); + sink.probe_spill->PrepareNextProbe(probe_global_scan); - probe_chunk_idx = 0; - probe_chunk_count = sink.probe_spill->ChunkCount(); + probe_chunk_count = sink.probe_spill->ChunkCount(probe_global_scan); probe_chunk_done = 0; if (IsRightOuterJoin(op.join_type)) { @@ -701,9 +697,8 @@ bool HashJoinGlobalSourceState::AssignTask(HashJoinGlobalSinkState &sink, HashJo } break; case HashJoinSourceStage::PROBE: - if (probe_chunk_idx != probe_chunk_count) { + if (sink.probe_spill->GetScanIndex(probe_global_scan, lstate.probe_local_scan)) { lstate.local_stage = global_stage; - lstate.probe_chunk_index = probe_chunk_idx++; return true; } break; @@ -724,9 +719,12 @@ bool HashJoinGlobalSourceState::AssignTask(HashJoinGlobalSinkState &sink, HashJo HashJoinLocalSourceState::HashJoinLocalSourceState(const PhysicalHashJoin &op, Allocator &allocator) : local_stage(HashJoinSourceStage::INIT), addresses(LogicalType::POINTER) { - probe_chunk_management_state.properties = ColumnDataScanProperties::ALLOW_ZERO_COPY; - auto &sink = (HashJoinGlobalSinkState &)*op.sink_state; + + auto &chunk_state = probe_local_scan.scan_state.current_chunk_state; + chunk_state.properties = ColumnDataScanProperties::ALLOW_ZERO_COPY; + chunk_state.consume = true; + probe_chunk.Initialize(allocator, sink.probe_types); join_keys.Initialize(allocator, op.condition_types); payload.Initialize(allocator, op.children[0]->types); @@ -809,7 +807,7 @@ void HashJoinLocalSourceState::ExternalProbe(HashJoinGlobalSinkState &sink, Hash } // Scan input chunk for next probe - sink.probe_spill->ScanChunk(probe_chunk_management_state, probe_chunk_index, probe_chunk); + sink.probe_spill->ScanChunk(gstate.probe_global_scan, probe_local_scan, probe_chunk); // Get the probe chunk columns/hashes join_keys.ReferenceColumns(probe_chunk, join_key_indices); diff --git a/src/include/duckdb/common/types/column_data_allocator.hpp b/src/include/duckdb/common/types/column_data_allocator.hpp index 6a67e846350e..532c161c4b5c 100644 --- a/src/include/duckdb/common/types/column_data_allocator.hpp +++ b/src/include/duckdb/common/types/column_data_allocator.hpp @@ -51,7 +51,7 @@ class ColumnDataAllocator { private: void AllocateDataInternal(idx_t size, uint32_t &block_id, uint32_t &offset, ChunkManagementState *chunk_state); void AllocateBlock(); - BufferHandle Pin(uint32_t block_id); + BufferHandle Pin(uint32_t block_id, bool consume); bool HasBlocks() { return !blocks.empty(); diff --git a/src/include/duckdb/common/types/column_data_collection.hpp b/src/include/duckdb/common/types/column_data_collection.hpp index 62622240699e..eca9a17ab4fd 100644 --- a/src/include/duckdb/common/types/column_data_collection.hpp +++ b/src/include/duckdb/common/types/column_data_collection.hpp @@ -71,21 +71,21 @@ class ColumnDataCollection { //! Initializes a chunk with the correct types for a given scan state DUCKDB_API void InitializeScanChunk(ColumnDataScanState &state, DataChunk &chunk) const; //! Initializes a Scan state for scanning all columns - DUCKDB_API void - InitializeScan(ColumnDataScanState &state, - ColumnDataScanProperties properties = ColumnDataScanProperties::ALLOW_ZERO_COPY) const; + DUCKDB_API void InitializeScan(ColumnDataScanState &state, + ColumnDataScanProperties properties = ColumnDataScanProperties::ALLOW_ZERO_COPY, + bool consume = false) const; //! Initializes a Scan state for scanning a subset of the columns - DUCKDB_API void - InitializeScan(ColumnDataScanState &state, vector column_ids, - ColumnDataScanProperties properties = ColumnDataScanProperties::ALLOW_ZERO_COPY) const; + DUCKDB_API void InitializeScan(ColumnDataScanState &state, vector column_ids, + ColumnDataScanProperties properties = ColumnDataScanProperties::ALLOW_ZERO_COPY, + bool consume = false) const; //! Initialize a parallel scan over the column data collection over all columns - DUCKDB_API void - InitializeScan(ColumnDataParallelScanState &state, - ColumnDataScanProperties properties = ColumnDataScanProperties::ALLOW_ZERO_COPY) const; + DUCKDB_API void InitializeScan(ColumnDataParallelScanState &state, + ColumnDataScanProperties properties = ColumnDataScanProperties::ALLOW_ZERO_COPY, + bool consume = false) const; //! Initialize a parallel scan over the column data collection over a subset of the columns - DUCKDB_API void - InitializeScan(ColumnDataParallelScanState &state, vector column_ids, - ColumnDataScanProperties properties = ColumnDataScanProperties::ALLOW_ZERO_COPY) const; + DUCKDB_API void InitializeScan(ColumnDataParallelScanState &state, vector column_ids, + ColumnDataScanProperties properties = ColumnDataScanProperties::ALLOW_ZERO_COPY, + bool consume = false) const; //! Scans a DataChunk from the ColumnDataCollection DUCKDB_API bool Scan(ColumnDataScanState &state, DataChunk &result) const; //! Scans a DataChunk from the ColumnDataCollection diff --git a/src/include/duckdb/common/types/column_data_collection_segment.hpp b/src/include/duckdb/common/types/column_data_collection_segment.hpp index b290523a48c8..d926cf0dc9fd 100644 --- a/src/include/duckdb/common/types/column_data_collection_segment.hpp +++ b/src/include/duckdb/common/types/column_data_collection_segment.hpp @@ -112,8 +112,6 @@ class ColumnDataCollectionSegment { idx_t ChunkCount() const; void FetchChunk(idx_t chunk_idx, DataChunk &result); void FetchChunk(idx_t chunk_idx, DataChunk &result, const vector &column_ids); - void FetchChunk(ChunkManagementState &state, idx_t chunk_idx, DataChunk &result, - const vector &column_ids); void Verify(); diff --git a/src/include/duckdb/common/types/column_data_scan_states.hpp b/src/include/duckdb/common/types/column_data_scan_states.hpp index 59a76c536196..431d221b2f10 100644 --- a/src/include/duckdb/common/types/column_data_scan_states.hpp +++ b/src/include/duckdb/common/types/column_data_scan_states.hpp @@ -36,6 +36,7 @@ enum class ColumnDataScanProperties : uint8_t { struct ChunkManagementState { unordered_map handles; ColumnDataScanProperties properties = ColumnDataScanProperties::INVALID; + bool consume = false; }; struct ColumnDataAppendState { @@ -50,6 +51,7 @@ struct ColumnDataScanState { idx_t current_row_index; idx_t next_row_index; ColumnDataScanProperties properties; + bool consume; vector column_ids; }; diff --git a/src/include/duckdb/execution/join_hashtable.hpp b/src/include/duckdb/execution/join_hashtable.hpp index d71d96a7d8ee..0a421f515d17 100644 --- a/src/include/duckdb/execution/join_hashtable.hpp +++ b/src/include/duckdb/execution/join_hashtable.hpp @@ -241,6 +241,28 @@ class JoinHashTable { //===--------------------------------------------------------------------===// // External Join //===--------------------------------------------------------------------===// + //! Thread-global scan state for the probe spill + struct ProbeSpillGlobalScanState { + ColumnDataParallelScanState scan_state; + vector>> partition_chunk_references; + + idx_t probe_partition_idx; + idx_t probe_chunk_idx; + }; + + //! Thread-local scan state for the probe spill + struct ProbeSpillLocalScanState { + ColumnDataLocalScanState scan_state; + + idx_t probe_partition_index = DConstants::INVALID_INDEX; + idx_t probe_chunk_index; + idx_t probe_segment_index; + idx_t probe_row_index; + }; + + //! ProbeSpill represents materialized probe-side data that could not be probed during PhysicalHashJoin::Execute + //! because the HashTable did not fit in memory. The ProbeSpill is not partitioned if the remaining data can be + //! dealt with in just 1 more round of probing, otherwise it is radix partitioned in the same way as the HashTable struct ProbeSpill { public: ProbeSpill(JoinHashTable &ht, ClientContext &context, const vector &probe_types); @@ -252,16 +274,18 @@ class JoinHashTable { //! Finalize by merging the thread-local accumulated data void Finalize(); - //! Get the probe collection for the next probe round - void PrepareNextProbeCollection(JoinHashTable &ht); - //! Number of chunks in the current probe collection - idx_t ChunkCount() const { - return chunk_references.size(); - } + //! The number of chunks in the current probe round + idx_t ChunkCount(ProbeSpillGlobalScanState &gstate) const; + + //! Prepare the next probe round + void PrepareNextProbe(ProbeSpillGlobalScanState &gstate); + //! Get the indices to scan next + bool GetScanIndex(ProbeSpillGlobalScanState &gstate, ProbeSpillLocalScanState &lstate); //! Scans the chunk with the given index - void ScanChunk(ChunkManagementState &state, idx_t chunk_idx, DataChunk &chunk); + void ScanChunk(ProbeSpillGlobalScanState &gstate, ProbeSpillLocalScanState &lstate, DataChunk &chunk); private: + JoinHashTable &ht; mutex lock; ClientContext &context; @@ -281,11 +305,6 @@ class JoinHashTable { unique_ptr global_spill_collection; vector> local_spill_collections; vector> local_spill_append_states; - - //! The probe collection currently being read - unique_ptr current_probe_collection; - //! The references (in order) to the chunks of the current probe collection - vector> chunk_references; }; //! Whether we are doing an external hash join diff --git a/src/include/duckdb/storage/buffer/block_handle.hpp b/src/include/duckdb/storage/buffer/block_handle.hpp index de88f05e4d17..936e2e78cb6e 100644 --- a/src/include/duckdb/storage/buffer/block_handle.hpp +++ b/src/include/duckdb/storage/buffer/block_handle.hpp @@ -53,6 +53,10 @@ class BlockHandle { unswizzled = unswizzler; } + inline void SetCanDestroy(bool can_destroy_p) { + can_destroy = can_destroy_p; + } + private: static BufferHandle Load(shared_ptr &handle, unique_ptr buffer = nullptr); unique_ptr UnloadAndTakeBlock(); @@ -72,7 +76,7 @@ class BlockHandle { //! Internal eviction timestamp atomic eviction_timestamp; //! Whether or not the buffer can be destroyed (only used for temporary buffers) - const bool can_destroy; + bool can_destroy; //! The memory usage of the block idx_t memory_usage; //! Does the block contain any memory pointers? From 6709a48b9473ed2588cbac37e1a7e42d7a48bfdf Mon Sep 17 00:00:00 2001 From: Laurens Kuiper Date: Fri, 7 Oct 2022 12:30:04 +0200 Subject: [PATCH 15/49] start work on ColumnDataConsumer --- benchmark/include/benchmark.hpp | 2 +- src/common/types/CMakeLists.txt | 1 + src/common/types/column_data_allocator.cpp | 14 ++- src/common/types/column_data_collection.cpp | 50 +++-------- src/common/types/column_data_consumer.cpp | 40 +++++++++ src/execution/join_hashtable.cpp | 6 +- .../operator/join/physical_hash_join.cpp | 1 - .../common/types/column_data_allocator.hpp | 2 +- .../common/types/column_data_collection.hpp | 26 +++--- .../common/types/column_data_consumer.hpp | 85 +++++++++++++++++++ .../common/types/column_data_scan_states.hpp | 2 - .../join/external/simple_external_join.test | 3 + 12 files changed, 168 insertions(+), 64 deletions(-) create mode 100644 src/common/types/column_data_consumer.cpp create mode 100644 src/include/duckdb/common/types/column_data_consumer.hpp diff --git a/benchmark/include/benchmark.hpp b/benchmark/include/benchmark.hpp index f25a6b8deb73..ad79e4571f58 100644 --- a/benchmark/include/benchmark.hpp +++ b/benchmark/include/benchmark.hpp @@ -31,7 +31,7 @@ struct BenchmarkState { //! new benchmarks class Benchmark { constexpr static size_t DEFAULT_NRUNS = 5; - constexpr static size_t DEFAULT_TIMEOUT = 30; + constexpr static size_t DEFAULT_TIMEOUT = 300; Benchmark(Benchmark &) = delete; diff --git a/src/common/types/CMakeLists.txt b/src/common/types/CMakeLists.txt index 883424a37edd..c13db1129220 100644 --- a/src/common/types/CMakeLists.txt +++ b/src/common/types/CMakeLists.txt @@ -13,6 +13,7 @@ add_library_unity( column_data_allocator.cpp column_data_collection.cpp column_data_collection_segment.cpp + column_data_consumer.cpp data_chunk.cpp date.cpp decimal.cpp diff --git a/src/common/types/column_data_allocator.cpp b/src/common/types/column_data_allocator.cpp index 0cb2bedefbf7..b215c07c9cf7 100644 --- a/src/common/types/column_data_allocator.cpp +++ b/src/common/types/column_data_allocator.cpp @@ -28,18 +28,16 @@ ColumnDataAllocator::ColumnDataAllocator(ClientContext &context, ColumnDataAlloc } } -BufferHandle ColumnDataAllocator::Pin(uint32_t block_id, bool consume) { +BufferHandle ColumnDataAllocator::Pin(uint32_t block_id) { D_ASSERT(type == ColumnDataAllocatorType::BUFFER_MANAGER_ALLOCATOR); if (shared) { lock_guard guard(lock); auto &block_handle = blocks[block_id].handle; auto result = alloc.buffer_manager->Pin(block_handle); - block_handle->SetCanDestroy(consume); return result; } else { auto &block_handle = blocks[block_id].handle; auto result = alloc.buffer_manager->Pin(block_handle); - block_handle->SetCanDestroy(consume); return result; } } @@ -135,8 +133,12 @@ void ColumnDataAllocator::InitializeChunkState(ChunkManagementState &state, Chun for (auto it = state.handles.begin(); it != state.handles.end(); it++) { if (chunk.block_ids.find(it->first) != chunk.block_ids.end()) { // still required: do not release + Printer::Print(StringUtil::Format(to_string((uint64_t)&state) + ": still required " + + to_string(it->first) + " " + to_string(it->second.IsValid()))); continue; } + Printer::Print(StringUtil::Format(to_string((uint64_t)&state) + ": not required " + to_string(it->first) + + " " + to_string(it->second.IsValid()))); state.handles.erase(it); found_handle = true; break; @@ -147,9 +149,13 @@ void ColumnDataAllocator::InitializeChunkState(ChunkManagementState &state, Chun for (auto &block_id : chunk.block_ids) { if (state.handles.find(block_id) != state.handles.end()) { // already pinned: don't need to do anything + Printer::Print(StringUtil::Format(to_string((uint64_t)&state) + ": already pinned " + to_string(block_id) + + " " + to_string(state.handles[block_id].IsValid()))); continue; } - state.handles[block_id] = Pin(block_id, state.consume); + state.handles[block_id] = Pin(block_id); + Printer::Print(StringUtil::Format(to_string((uint64_t)&state) + ": newly pinned " + to_string(block_id) + " " + + to_string(state.handles[block_id].IsValid()))); } } diff --git a/src/common/types/column_data_collection.cpp b/src/common/types/column_data_collection.cpp index c79ae5907e28..fd3afdbdd45e 100644 --- a/src/common/types/column_data_collection.cpp +++ b/src/common/types/column_data_collection.cpp @@ -593,36 +593,34 @@ void ColumnDataCollection::Append(DataChunk &input) { //===--------------------------------------------------------------------===// // Scan //===--------------------------------------------------------------------===// -void ColumnDataCollection::InitializeScan(ColumnDataScanState &state, ColumnDataScanProperties properties, - bool consume) const { +void ColumnDataCollection::InitializeScan(ColumnDataScanState &state, ColumnDataScanProperties properties) const { vector column_ids; column_ids.reserve(types.size()); for (idx_t i = 0; i < types.size(); i++) { column_ids.push_back(i); } - InitializeScan(state, move(column_ids), properties, consume); + InitializeScan(state, move(column_ids), properties); } void ColumnDataCollection::InitializeScan(ColumnDataScanState &state, vector column_ids, - ColumnDataScanProperties properties, bool consume) const { + ColumnDataScanProperties properties) const { state.chunk_index = 0; state.segment_index = 0; state.current_row_index = 0; state.next_row_index = 0; state.current_chunk_state.handles.clear(); state.properties = properties; - state.consume = consume; state.column_ids = move(column_ids); } -void ColumnDataCollection::InitializeScan(ColumnDataParallelScanState &state, ColumnDataScanProperties properties, - bool consume) const { - InitializeScan(state.scan_state, properties, consume); +void ColumnDataCollection::InitializeScan(ColumnDataParallelScanState &state, + ColumnDataScanProperties properties) const { + InitializeScan(state.scan_state, properties); } void ColumnDataCollection::InitializeScan(ColumnDataParallelScanState &state, vector column_ids, - ColumnDataScanProperties properties, bool consume) const { - InitializeScan(state.scan_state, move(column_ids), properties, consume); + ColumnDataScanProperties properties) const { + InitializeScan(state.scan_state, move(column_ids), properties); } bool ColumnDataCollection::Scan(ColumnDataParallelScanState &state, ColumnDataLocalScanState &lstate, @@ -691,7 +689,6 @@ void ColumnDataCollection::ScanAtIndex(ColumnDataParallelScanState &state, Colum } auto &segment = *segments[segment_index]; lstate.current_chunk_state.properties = state.scan_state.properties; - lstate.current_chunk_state.consume = state.scan_state.consume; segment.ReadChunk(chunk_index, lstate.current_chunk_state, result, state.scan_state.column_ids); lstate.current_row_index = row_index; result.Verify(); @@ -815,35 +812,8 @@ bool ColumnDataCollection::ResultEquals(const ColumnDataCollection &left, const return true; } -vector> ColumnDataCollection::GetChunkReferences(bool sort) { - // Create a vector containing a reference to every chunk - vector> result; - if (Count() == 0) { - return result; - } - - result.reserve(ChunkCount()); - for (auto &segment : segments) { - for (idx_t chunk_index = 0; chunk_index < segment->chunk_data.size(); chunk_index++) { - result.emplace_back(segment.get(), chunk_index); - } - } - - if (sort) { - // Sort them by lowest block id - std::sort(result.begin(), result.end(), - [](const pair &lhs, - const pair &rhs) -> bool { - const auto &lhs_block_ids = lhs.first->chunk_data[lhs.second].block_ids; - const auto &rhs_block_ids = rhs.first->chunk_data[rhs.second].block_ids; - const uint32_t lhs_min_block_id = *std::min_element(lhs_block_ids.begin(), lhs_block_ids.end()); - const uint32_t rhs_min_block_id = *std::min_element(rhs_block_ids.begin(), rhs_block_ids.end()); - - return lhs_min_block_id < rhs_min_block_id; - }); - } - - return result; +const vector> &ColumnDataCollection::GetSegments() const { + return segments; } } // namespace duckdb diff --git a/src/common/types/column_data_consumer.cpp b/src/common/types/column_data_consumer.cpp new file mode 100644 index 000000000000..0e06f6144466 --- /dev/null +++ b/src/common/types/column_data_consumer.cpp @@ -0,0 +1,40 @@ +#include "duckdb/common/types/column_data_consumer.hpp" + +namespace duckdb { + +using ChunkReferenceState = ColumnDataConsumer::ChunkReferenceState; +using ChunkReference = ColumnDataConsumer::ChunkReference; + +ColumnDataConsumer::ColumnDataConsumer(ColumnDataCollection &collection_p) : collection(collection_p) { +} + +void ColumnDataConsumer::InitializeScan() { + chunk_count = collection.ChunkCount(); + // Initialize chunk references and sort them so we can scan them in a sane order + chunk_references.reserve(chunk_count); + for (auto &segment : collection.GetSegments()) { + for (idx_t chunk_index = 0; chunk_index < segment->chunk_data.size(); chunk_index++) { + chunk_references.emplace_back(segment.get(), chunk_index); + } + } + std::sort(chunk_references.begin(), chunk_references.end()); + current_chunk_index = 0; + minimum_chunk_index_in_progress = DConstants::INVALID_INDEX; +} + +bool ColumnDataConsumer::AssignChunk(idx_t &assigned_chunk_index) { + lock_guard guard(lock); + if (current_chunk_index == chunk_count) { + return false; + } + assigned_chunk_index = current_chunk_index++; + auto &chunk_ref = chunk_references[assigned_chunk_index]; + chunk_ref.state = ChunkReferenceState::IN_PROGRESS; + return true; +} + +void ColumnDataConsumer::Scan(ColumnDataConsumerLocalState &state, idx_t chunk_index, DataChunk &chunk) { + auto &chunk_ref = chunk_references[chunk_index]; +} + +} // namespace duckdb \ No newline at end of file diff --git a/src/execution/join_hashtable.cpp b/src/execution/join_hashtable.cpp index 209755175cc8..c8db0ce6e3c2 100644 --- a/src/execution/join_hashtable.cpp +++ b/src/execution/join_hashtable.cpp @@ -9,6 +9,8 @@ #include "duckdb/main/client_context.hpp" #include "duckdb/storage/buffer_manager.hpp" +#include + namespace duckdb { using ValidityBytes = JoinHashTable::ValidityBytes; @@ -1199,6 +1201,7 @@ ProbeSpill::ProbeSpill(JoinHashTable &ht, ClientContext &context, const vector(context, probe_types, ht.radix_bits, probe_types.size() - 1); @@ -1285,7 +1288,7 @@ void ProbeSpill::PrepareNextProbe(ProbeSpillGlobalScanState &gstate) { gstate.probe_partition_idx = ht.partition_start; gstate.probe_chunk_idx = 0; } else { - global_spill_collection->InitializeScan(gstate.scan_state, ColumnDataScanProperties::ALLOW_ZERO_COPY, true); + global_spill_collection->InitializeScan(gstate.scan_state); } } @@ -1330,7 +1333,6 @@ void ProbeSpill::ScanChunk(ProbeSpillGlobalScanState &gstate, ProbeSpillLocalSca segment.ReadChunk(idx_within_segment, lstate.scan_state.current_chunk_state, chunk, column_ids); } else { - D_ASSERT(gstate.scan_state.scan_state.consume); D_ASSERT(gstate.scan_state.scan_state.properties == ColumnDataScanProperties::ALLOW_ZERO_COPY); global_spill_collection->ScanAtIndex(gstate.scan_state, lstate.scan_state, chunk, lstate.probe_chunk_index, lstate.probe_segment_index, lstate.probe_row_index); diff --git a/src/execution/operator/join/physical_hash_join.cpp b/src/execution/operator/join/physical_hash_join.cpp index c656adecaf97..ccdc99fc8902 100644 --- a/src/execution/operator/join/physical_hash_join.cpp +++ b/src/execution/operator/join/physical_hash_join.cpp @@ -723,7 +723,6 @@ HashJoinLocalSourceState::HashJoinLocalSourceState(const PhysicalHashJoin &op, A auto &chunk_state = probe_local_scan.scan_state.current_chunk_state; chunk_state.properties = ColumnDataScanProperties::ALLOW_ZERO_COPY; - chunk_state.consume = true; probe_chunk.Initialize(allocator, sink.probe_types); join_keys.Initialize(allocator, op.condition_types); diff --git a/src/include/duckdb/common/types/column_data_allocator.hpp b/src/include/duckdb/common/types/column_data_allocator.hpp index 532c161c4b5c..6a67e846350e 100644 --- a/src/include/duckdb/common/types/column_data_allocator.hpp +++ b/src/include/duckdb/common/types/column_data_allocator.hpp @@ -51,7 +51,7 @@ class ColumnDataAllocator { private: void AllocateDataInternal(idx_t size, uint32_t &block_id, uint32_t &offset, ChunkManagementState *chunk_state); void AllocateBlock(); - BufferHandle Pin(uint32_t block_id, bool consume); + BufferHandle Pin(uint32_t block_id); bool HasBlocks() { return !blocks.empty(); diff --git a/src/include/duckdb/common/types/column_data_collection.hpp b/src/include/duckdb/common/types/column_data_collection.hpp index eca9a17ab4fd..957207d73772 100644 --- a/src/include/duckdb/common/types/column_data_collection.hpp +++ b/src/include/duckdb/common/types/column_data_collection.hpp @@ -71,21 +71,21 @@ class ColumnDataCollection { //! Initializes a chunk with the correct types for a given scan state DUCKDB_API void InitializeScanChunk(ColumnDataScanState &state, DataChunk &chunk) const; //! Initializes a Scan state for scanning all columns - DUCKDB_API void InitializeScan(ColumnDataScanState &state, - ColumnDataScanProperties properties = ColumnDataScanProperties::ALLOW_ZERO_COPY, - bool consume = false) const; + DUCKDB_API void + InitializeScan(ColumnDataScanState &state, + ColumnDataScanProperties properties = ColumnDataScanProperties::ALLOW_ZERO_COPY) const; //! Initializes a Scan state for scanning a subset of the columns - DUCKDB_API void InitializeScan(ColumnDataScanState &state, vector column_ids, - ColumnDataScanProperties properties = ColumnDataScanProperties::ALLOW_ZERO_COPY, - bool consume = false) const; + DUCKDB_API void + InitializeScan(ColumnDataScanState &state, vector column_ids, + ColumnDataScanProperties properties = ColumnDataScanProperties::ALLOW_ZERO_COPY) const; //! Initialize a parallel scan over the column data collection over all columns - DUCKDB_API void InitializeScan(ColumnDataParallelScanState &state, - ColumnDataScanProperties properties = ColumnDataScanProperties::ALLOW_ZERO_COPY, - bool consume = false) const; + DUCKDB_API void + InitializeScan(ColumnDataParallelScanState &state, + ColumnDataScanProperties properties = ColumnDataScanProperties::ALLOW_ZERO_COPY) const; //! Initialize a parallel scan over the column data collection over a subset of the columns - DUCKDB_API void InitializeScan(ColumnDataParallelScanState &state, vector column_ids, - ColumnDataScanProperties properties = ColumnDataScanProperties::ALLOW_ZERO_COPY, - bool consume = false) const; + DUCKDB_API void + InitializeScan(ColumnDataParallelScanState &state, vector column_ids, + ColumnDataScanProperties properties = ColumnDataScanProperties::ALLOW_ZERO_COPY) const; //! Scans a DataChunk from the ColumnDataCollection DUCKDB_API bool Scan(ColumnDataScanState &state, DataChunk &result) const; //! Scans a DataChunk from the ColumnDataCollection @@ -138,7 +138,7 @@ class ColumnDataCollection { idx_t chunk_index, idx_t segment_index, idx_t row_index) const; //! Get a vector of references to every chunk (segment, index in segment), and optionally sort by block id - vector> GetChunkReferences(bool sort); + const vector> &GetSegments() const; private: //! Initialize the column data collection diff --git a/src/include/duckdb/common/types/column_data_consumer.hpp b/src/include/duckdb/common/types/column_data_consumer.hpp new file mode 100644 index 000000000000..9f8d1942f64e --- /dev/null +++ b/src/include/duckdb/common/types/column_data_consumer.hpp @@ -0,0 +1,85 @@ +//===----------------------------------------------------------------------===// +// DuckDB +// +// duckdb/common/types/column_data_consumer.hpp +// +// +//===----------------------------------------------------------------------===// + +#pragma once + +#include "duckdb/common/types/column_data_collection.hpp" + +namespace duckdb { + +struct ColumnDataConsumerLocalState { + ColumnDataAllocator *allocator; + ChunkManagementState current_chunk_state; +}; + +//! ColumnDataConsumer can scan a ColumnDataCollection, and consume it in the process, i.e., read blocks are deleted +class ColumnDataConsumer { +public: + enum class ChunkReferenceState : uint8_t { + //! Ready to scan + READY, + //! Scan in progress + IN_PROGRESS, + //! Scanning done + DONE + }; + + struct ChunkReference { + public: + ChunkReference(ColumnDataCollectionSegment *segment_p, uint32_t chunk_index_p) + : segment(segment_p), chunk_index_in_segment(chunk_index_p), state(ChunkReferenceState::READY) { + } + + inline uint32_t GetMinimumBlockID() const { + const auto &block_ids = segment->chunk_data[chunk_index_in_segment].block_ids; + return *std::min_element(block_ids.begin(), block_ids.end()); + } + + friend bool operator<(const ChunkReference &lhs, const ChunkReference &rhs) { + // Sort by allocator first + if (lhs.segment->allocator.get() != rhs.segment->allocator.get()) { + return lhs.segment->allocator.get() < rhs.segment->allocator.get(); + } + // Then by minimum block id + return lhs.GetMinimumBlockID() < rhs.GetMinimumBlockID(); + } + + public: + ColumnDataCollectionSegment *segment; + uint32_t chunk_index_in_segment; + ChunkReferenceState state; + }; + +public: + explicit ColumnDataConsumer(ColumnDataCollection &collection); + +public: + //! Initialize the scan of the ColumnDataCollection + void InitializeScan(); + //! Assign a chunk index to scan + bool Assign(idx_t &assigned_chunk_index); + //! Scan the chunk given the index + void Scan(ColumnDataConsumerLocalState &state, idx_t chunk_index, DataChunk &chunk); + //! Indicate that scanning the chunk is done + void MarkAsDone(idx_t chunk_index); + +private: + mutex lock; + //! The collection being scanned + ColumnDataCollection &collection; + //! The number of chunk references + idx_t chunk_count; + //! The chunks (in order) to be scanned + vector chunk_references; + //! Current index into "chunks" + idx_t current_chunk_index; + //! Of all the chunks that are marked IN_PROGRESS, this is the lowest index + idx_t minimum_chunk_index_in_progress; +}; + +} // namespace duckdb diff --git a/src/include/duckdb/common/types/column_data_scan_states.hpp b/src/include/duckdb/common/types/column_data_scan_states.hpp index 431d221b2f10..59a76c536196 100644 --- a/src/include/duckdb/common/types/column_data_scan_states.hpp +++ b/src/include/duckdb/common/types/column_data_scan_states.hpp @@ -36,7 +36,6 @@ enum class ColumnDataScanProperties : uint8_t { struct ChunkManagementState { unordered_map handles; ColumnDataScanProperties properties = ColumnDataScanProperties::INVALID; - bool consume = false; }; struct ColumnDataAppendState { @@ -51,7 +50,6 @@ struct ColumnDataScanState { idx_t current_row_index; idx_t next_row_index; ColumnDataScanProperties properties; - bool consume; vector column_ids; }; diff --git a/test/sql/join/external/simple_external_join.test b/test/sql/join/external/simple_external_join.test index 15c620ad709d..806069448a9a 100644 --- a/test/sql/join/external/simple_external_join.test +++ b/test/sql/join/external/simple_external_join.test @@ -8,6 +8,9 @@ pragma verify_external statement ok pragma verify_parallelism +#statement ok +#pragma threads=1 + # Create tables with large string values so that we have to swizzle strings # The tables have some overlapping values to keep the join result small statement ok From 146282f81e8c3d279dbdba6781676c678c8ae2e9 Mon Sep 17 00:00:00 2001 From: Laurens Kuiper Date: Fri, 7 Oct 2022 15:15:58 +0200 Subject: [PATCH 16/49] properly implement ColumnDataConsumer for external hash join --- src/common/types/column_data_allocator.cpp | 17 ++-- src/common/types/column_data_consumer.cpp | 72 +++++++++++++--- src/execution/join_hashtable.cpp | 86 ++----------------- .../operator/join/physical_hash_join.cpp | 22 +++-- .../common/types/column_data_allocator.hpp | 5 ++ .../common/types/column_data_consumer.hpp | 47 +++++----- .../duckdb/execution/join_hashtable.hpp | 33 ++----- 7 files changed, 121 insertions(+), 161 deletions(-) diff --git a/src/common/types/column_data_allocator.cpp b/src/common/types/column_data_allocator.cpp index b215c07c9cf7..f6f6ae0e7b0c 100644 --- a/src/common/types/column_data_allocator.cpp +++ b/src/common/types/column_data_allocator.cpp @@ -116,6 +116,15 @@ data_ptr_t ColumnDataAllocator::GetDataPointer(ChunkManagementState &state, uint return state.handles[block_id].Ptr() + offset; } +void ColumnDataAllocator::Reset() { + blocks.clear(); + allocated_data.clear(); +} + +void ColumnDataAllocator::DeleteBlock(uint32_t block_id) { + blocks[block_id].handle->SetCanDestroy(true); +} + Allocator &ColumnDataAllocator::GetAllocator() { return type == ColumnDataAllocatorType::IN_MEMORY_ALLOCATOR ? *alloc.allocator : alloc.buffer_manager->GetBufferAllocator(); @@ -133,12 +142,8 @@ void ColumnDataAllocator::InitializeChunkState(ChunkManagementState &state, Chun for (auto it = state.handles.begin(); it != state.handles.end(); it++) { if (chunk.block_ids.find(it->first) != chunk.block_ids.end()) { // still required: do not release - Printer::Print(StringUtil::Format(to_string((uint64_t)&state) + ": still required " + - to_string(it->first) + " " + to_string(it->second.IsValid()))); continue; } - Printer::Print(StringUtil::Format(to_string((uint64_t)&state) + ": not required " + to_string(it->first) + - " " + to_string(it->second.IsValid()))); state.handles.erase(it); found_handle = true; break; @@ -149,13 +154,9 @@ void ColumnDataAllocator::InitializeChunkState(ChunkManagementState &state, Chun for (auto &block_id : chunk.block_ids) { if (state.handles.find(block_id) != state.handles.end()) { // already pinned: don't need to do anything - Printer::Print(StringUtil::Format(to_string((uint64_t)&state) + ": already pinned " + to_string(block_id) + - " " + to_string(state.handles[block_id].IsValid()))); continue; } state.handles[block_id] = Pin(block_id); - Printer::Print(StringUtil::Format(to_string((uint64_t)&state) + ": newly pinned " + to_string(block_id) + " " + - to_string(state.handles[block_id].IsValid()))); } } diff --git a/src/common/types/column_data_consumer.cpp b/src/common/types/column_data_consumer.cpp index 0e06f6144466..120bfc378972 100644 --- a/src/common/types/column_data_consumer.cpp +++ b/src/common/types/column_data_consumer.cpp @@ -2,15 +2,16 @@ namespace duckdb { -using ChunkReferenceState = ColumnDataConsumer::ChunkReferenceState; -using ChunkReference = ColumnDataConsumer::ChunkReference; - -ColumnDataConsumer::ColumnDataConsumer(ColumnDataCollection &collection_p) : collection(collection_p) { +ColumnDataConsumer::ColumnDataConsumer(ColumnDataCollection &collection_p, vector column_ids) + : collection(collection_p), column_ids(move(column_ids)) { } void ColumnDataConsumer::InitializeScan() { chunk_count = collection.ChunkCount(); - // Initialize chunk references and sort them so we can scan them in a sane order + current_chunk_index = 0; + current_chunk_index = 0; + + // Initialize chunk references and sort them, so we can scan them in a sane order, regardless of how it was created chunk_references.reserve(chunk_count); for (auto &segment : collection.GetSegments()) { for (idx_t chunk_index = 0; chunk_index < segment->chunk_data.size(); chunk_index++) { @@ -18,23 +19,66 @@ void ColumnDataConsumer::InitializeScan() { } } std::sort(chunk_references.begin(), chunk_references.end()); - current_chunk_index = 0; - minimum_chunk_index_in_progress = DConstants::INVALID_INDEX; } -bool ColumnDataConsumer::AssignChunk(idx_t &assigned_chunk_index) { +bool ColumnDataConsumer::AssignChunk(ColumnDataConsumerScanState &state) { lock_guard guard(lock); if (current_chunk_index == chunk_count) { + // All chunks have been assigned + state.chunk_index = DConstants::INVALID_INDEX; return false; } - assigned_chunk_index = current_chunk_index++; - auto &chunk_ref = chunk_references[assigned_chunk_index]; - chunk_ref.state = ChunkReferenceState::IN_PROGRESS; + // Assign chunk index + state.chunk_index = current_chunk_index++; + D_ASSERT(chunks_in_progress.find(state.chunk_index) == chunks_in_progress.end()); + chunks_in_progress.insert(state.chunk_index); return true; } -void ColumnDataConsumer::Scan(ColumnDataConsumerLocalState &state, idx_t chunk_index, DataChunk &chunk) { - auto &chunk_ref = chunk_references[chunk_index]; +void ColumnDataConsumer::ScanChunk(ColumnDataConsumerScanState &state, DataChunk &chunk) const { + D_ASSERT(state.chunk_index < chunk_count); + auto &chunk_ref = chunk_references[state.chunk_index]; + if (state.allocator != chunk_ref.segment->allocator.get()) { + // Previously scanned a chunk from a different allocator, reset the handles + state.allocator = chunk_ref.segment->allocator.get(); + state.current_chunk_state.handles.clear(); + } + chunk_ref.segment->ReadChunk(chunk_ref.chunk_index_in_segment, state.current_chunk_state, chunk, column_ids); +} + +void ColumnDataConsumer::FinishChunk(ColumnDataConsumerScanState &state) { + D_ASSERT(state.chunk_index < chunk_count); + idx_t delete_index_start; + idx_t delete_index_end; + { + lock_guard guard(lock); + D_ASSERT(chunks_in_progress.find(state.chunk_index) != chunks_in_progress.end()); + delete_index_start = chunk_delete_index; + delete_index_end = *std::min_element(chunks_in_progress.begin(), chunks_in_progress.end()); + chunks_in_progress.erase(state.chunk_index); + chunk_delete_index = delete_index_end; + } + ConsumeChunks(delete_index_start, delete_index_end); +} +void ColumnDataConsumer::ConsumeChunks(idx_t delete_index_start, idx_t delete_index_end) { + for (idx_t chunk_index = delete_index_start; chunk_index < delete_index_end; chunk_index++) { + if (chunk_index == 0) { + continue; + } + auto &prev_chunk_ref = chunk_references[chunk_index - 1]; + auto &curr_chunk_ref = chunk_references[chunk_index]; + if (prev_chunk_ref.segment->allocator.get() != curr_chunk_ref.segment->allocator.get()) { + // Moved to the next allocator, reset the previous one + prev_chunk_ref.segment->allocator->Reset(); + continue; + } + // Same allocator, see if we can delete blocks + auto prev_min_block_id = prev_chunk_ref.GetMinimumBlockID(); + auto curr_min_block_id = curr_chunk_ref.GetMinimumBlockID(); + for (idx_t block_id = prev_min_block_id; block_id < curr_min_block_id; block_id++) { + prev_chunk_ref.segment->allocator->DeleteBlock(block_id); + } + } } -} // namespace duckdb \ No newline at end of file +} // namespace duckdb diff --git a/src/execution/join_hashtable.cpp b/src/execution/join_hashtable.cpp index c8db0ce6e3c2..b0cdcc9c7b28 100644 --- a/src/execution/join_hashtable.cpp +++ b/src/execution/join_hashtable.cpp @@ -16,8 +16,6 @@ namespace duckdb { using ValidityBytes = JoinHashTable::ValidityBytes; using ScanStructure = JoinHashTable::ScanStructure; using ProbeSpill = JoinHashTable::ProbeSpill; -using ProbeSpillLocalScanState = JoinHashTable::ProbeSpillLocalScanState; -using ProbeSpillGlobalScanState = JoinHashTable::ProbeSpillGlobalScanState; JoinHashTable::JoinHashTable(BufferManager &buffer_manager, const vector &conditions, vector btypes, JoinType type) @@ -1048,8 +1046,6 @@ void JoinHashTable::ComputePartitionSizes(ClientConfig &config, vector(context, probe_types, ht.radix_bits, probe_types.size() - 1); @@ -1247,7 +1242,7 @@ void ProbeSpill::Finalize() { global_partitions->Combine(*local_partition); } local_partitions.clear(); - local_spill_append_states.clear(); + local_partition_append_states.clear(); } else { global_spill_collection = move(local_spill_collections[0]); for (idx_t i = 1; i < local_spill_collections.size(); i++) { @@ -1258,86 +1253,21 @@ void ProbeSpill::Finalize() { } } -idx_t ProbeSpill::ChunkCount(ProbeSpillGlobalScanState &gstate) const { - if (partitioned) { - idx_t count = 0; - for (idx_t i = ht.partition_start; i < ht.partition_end; i++) { - count += gstate.partition_chunk_references[i].size(); - } - return count; - } else { - return global_spill_collection->ChunkCount(); - } -} - -void ProbeSpill::PrepareNextProbe(ProbeSpillGlobalScanState &gstate) { +void ProbeSpill::PrepareNextProbe() { if (partitioned) { auto &partitions = global_partitions->GetPartitions(); - if (gstate.partition_chunk_references.empty()) { - // First call to prepare - initialize chunk references for all partitions - for (idx_t i = 0; i < partitions.size(); i++) { - gstate.partition_chunk_references.push_back(partitions[i]->GetChunkReferences(true)); - } - } - if (ht.partition_start == partitions.size()) { return; } - // Reset indices - gstate.probe_partition_idx = ht.partition_start; - gstate.probe_chunk_idx = 0; - } else { - global_spill_collection->InitializeScan(gstate.scan_state); - } -} - -bool ProbeSpill::GetScanIndex(ProbeSpillGlobalScanState &gstate, ProbeSpillLocalScanState &lstate) { - if (partitioned) { - // Check if done - if (gstate.probe_partition_idx == ht.partition_end) { - return false; - } - - // Get indices - auto next_partition_index = gstate.probe_partition_idx; - auto next_chunk_index = gstate.probe_chunk_idx++; - - if (lstate.probe_partition_index != next_partition_index) { - // Previously scanned a different partition, need to reset local state pins - lstate.scan_state.current_chunk_state.handles.clear(); - } - - // Assign them to local state - lstate.probe_partition_index = next_partition_index; - lstate.probe_chunk_index = next_chunk_index; - - // Increment to next partition, if needed - if (gstate.probe_chunk_idx == gstate.partition_chunk_references[gstate.probe_partition_idx].size()) { - gstate.probe_partition_idx++; - gstate.probe_chunk_idx = 0; + // Move specific partitions to the global spill collection + global_spill_collection = move(partitions[ht.partition_start]); + for (idx_t i = ht.partition_start + 1; i < ht.partition_end; i++) { + global_spill_collection->Combine(*partitions[i]); } - return true; - } else { - return global_spill_collection->NextScanIndex(gstate.scan_state.scan_state, lstate.probe_chunk_index, - lstate.probe_segment_index, lstate.probe_row_index); - } -} - -void ProbeSpill::ScanChunk(ProbeSpillGlobalScanState &gstate, ProbeSpillLocalScanState &lstate, DataChunk &chunk) { - chunk.Reset(); - if (partitioned) { - auto &chunk_ref = gstate.partition_chunk_references[lstate.probe_partition_index][lstate.probe_chunk_index]; - auto &segment = *chunk_ref.first; - auto &idx_within_segment = chunk_ref.second; - - segment.ReadChunk(idx_within_segment, lstate.scan_state.current_chunk_state, chunk, column_ids); - } else { - D_ASSERT(gstate.scan_state.scan_state.properties == ColumnDataScanProperties::ALLOW_ZERO_COPY); - global_spill_collection->ScanAtIndex(gstate.scan_state, lstate.scan_state, chunk, lstate.probe_chunk_index, - lstate.probe_segment_index, lstate.probe_row_index); } - chunk.Verify(); + consumer = make_unique(*global_spill_collection, column_ids); + consumer->InitializeScan(); } } // namespace duckdb diff --git a/src/execution/operator/join/physical_hash_join.cpp b/src/execution/operator/join/physical_hash_join.cpp index ccdc99fc8902..6ed67469e8b0 100644 --- a/src/execution/operator/join/physical_hash_join.cpp +++ b/src/execution/operator/join/physical_hash_join.cpp @@ -539,7 +539,7 @@ class HashJoinGlobalSourceState : public GlobalSourceState { //! For synchronizing the external hash join atomic initialized; atomic global_stage; - mutex &lock; + mutex lock; //! For HT build synchronization idx_t build_block_idx; @@ -548,7 +548,6 @@ class HashJoinGlobalSourceState : public GlobalSourceState { idx_t build_blocks_per_thread; //! For probe synchronization - JoinHashTable::ProbeSpillGlobalScanState probe_global_scan; idx_t probe_chunk_count; idx_t probe_chunk_done; @@ -587,7 +586,7 @@ class HashJoinLocalSourceState : public LocalSourceState { idx_t build_block_idx_end; //! Local scan state for probe spill - JoinHashTable::ProbeSpillLocalScanState probe_local_scan; + ColumnDataConsumerScanState probe_local_scan; //! Chunks for holding the scanned probe collection DataChunk probe_chunk; DataChunk join_keys; @@ -613,7 +612,7 @@ unique_ptr PhysicalHashJoin::GetLocalSourceState(ExecutionCont } HashJoinGlobalSourceState::HashJoinGlobalSourceState(const PhysicalHashJoin &op, ClientContext &context) - : op(op), initialized(false), global_stage(HashJoinSourceStage::INIT), lock(probe_global_scan.scan_state.lock), + : op(op), initialized(false), global_stage(HashJoinSourceStage::INIT), probe_count(op.children[0]->estimated_cardinality), parallel_scan_chunk_count(context.config.verify_parallelism ? 1 : 120) { } @@ -659,9 +658,9 @@ void HashJoinGlobalSourceState::PrepareBuild(HashJoinGlobalSinkState &sink) { } void HashJoinGlobalSourceState::PrepareProbe(HashJoinGlobalSinkState &sink) { - sink.probe_spill->PrepareNextProbe(probe_global_scan); + sink.probe_spill->PrepareNextProbe(); - probe_chunk_count = sink.probe_spill->ChunkCount(probe_global_scan); + probe_chunk_count = sink.probe_spill->consumer->ChunkCount(); probe_chunk_done = 0; if (IsRightOuterJoin(op.join_type)) { @@ -697,7 +696,7 @@ bool HashJoinGlobalSourceState::AssignTask(HashJoinGlobalSinkState &sink, HashJo } break; case HashJoinSourceStage::PROBE: - if (sink.probe_spill->GetScanIndex(probe_global_scan, lstate.probe_local_scan)) { + if (sink.probe_spill->consumer->AssignChunk(lstate.probe_local_scan)) { lstate.local_stage = global_stage; return true; } @@ -719,11 +718,10 @@ bool HashJoinGlobalSourceState::AssignTask(HashJoinGlobalSinkState &sink, HashJo HashJoinLocalSourceState::HashJoinLocalSourceState(const PhysicalHashJoin &op, Allocator &allocator) : local_stage(HashJoinSourceStage::INIT), addresses(LogicalType::POINTER) { - auto &sink = (HashJoinGlobalSinkState &)*op.sink_state; - - auto &chunk_state = probe_local_scan.scan_state.current_chunk_state; + auto &chunk_state = probe_local_scan.current_chunk_state; chunk_state.properties = ColumnDataScanProperties::ALLOW_ZERO_COPY; + auto &sink = (HashJoinGlobalSinkState &)*op.sink_state; probe_chunk.Initialize(allocator, sink.probe_types); join_keys.Initialize(allocator, op.condition_types); payload.Initialize(allocator, op.children[0]->types); @@ -792,6 +790,7 @@ void HashJoinLocalSourceState::ExternalProbe(HashJoinGlobalSinkState &sink, Hash scan_structure->Next(join_keys, payload, chunk); if (chunk.size() == 0) { scan_structure = nullptr; + sink.probe_spill->consumer->FinishChunk(probe_local_scan); lock_guard lock(gstate.lock); if (++gstate.probe_chunk_done == gstate.probe_chunk_count) { if (IsRightOuterJoin(gstate.op.join_type)) { @@ -800,13 +799,12 @@ void HashJoinLocalSourceState::ExternalProbe(HashJoinGlobalSinkState &sink, Hash gstate.PrepareBuild(sink); } } - // TODO: delete blocks that we passed (blocks could have different allocators ...) } return; } // Scan input chunk for next probe - sink.probe_spill->ScanChunk(gstate.probe_global_scan, probe_local_scan, probe_chunk); + sink.probe_spill->consumer->ScanChunk(probe_local_scan, probe_chunk); // Get the probe chunk columns/hashes join_keys.ReferenceColumns(probe_chunk, join_key_indices); diff --git a/src/include/duckdb/common/types/column_data_allocator.hpp b/src/include/duckdb/common/types/column_data_allocator.hpp index 6a67e846350e..bf44da4328dd 100644 --- a/src/include/duckdb/common/types/column_data_allocator.hpp +++ b/src/include/duckdb/common/types/column_data_allocator.hpp @@ -48,6 +48,11 @@ class ColumnDataAllocator { void InitializeChunkState(ChunkManagementState &state, ChunkMetaData &meta_data); data_ptr_t GetDataPointer(ChunkManagementState &state, uint32_t block_id, uint32_t offset); + //! Resets all owned data + void Reset(); + //! Deletes the block with the given id + void DeleteBlock(uint32_t block_id); + private: void AllocateDataInternal(idx_t size, uint32_t &block_id, uint32_t &offset, ChunkManagementState *chunk_state); void AllocateBlock(); diff --git a/src/include/duckdb/common/types/column_data_consumer.hpp b/src/include/duckdb/common/types/column_data_consumer.hpp index 9f8d1942f64e..3e54ac2953cf 100644 --- a/src/include/duckdb/common/types/column_data_consumer.hpp +++ b/src/include/duckdb/common/types/column_data_consumer.hpp @@ -9,30 +9,23 @@ #pragma once #include "duckdb/common/types/column_data_collection.hpp" +#include "duckdb/common/types/column_data_collection_segment.hpp" namespace duckdb { -struct ColumnDataConsumerLocalState { - ColumnDataAllocator *allocator; +struct ColumnDataConsumerScanState { + ColumnDataAllocator *allocator = nullptr; ChunkManagementState current_chunk_state; + idx_t chunk_index; }; //! ColumnDataConsumer can scan a ColumnDataCollection, and consume it in the process, i.e., read blocks are deleted class ColumnDataConsumer { -public: - enum class ChunkReferenceState : uint8_t { - //! Ready to scan - READY, - //! Scan in progress - IN_PROGRESS, - //! Scanning done - DONE - }; - +private: struct ChunkReference { public: ChunkReference(ColumnDataCollectionSegment *segment_p, uint32_t chunk_index_p) - : segment(segment_p), chunk_index_in_segment(chunk_index_p), state(ChunkReferenceState::READY) { + : segment(segment_p), chunk_index_in_segment(chunk_index_p) { } inline uint32_t GetMinimumBlockID() const { @@ -52,34 +45,44 @@ class ColumnDataConsumer { public: ColumnDataCollectionSegment *segment; uint32_t chunk_index_in_segment; - ChunkReferenceState state; }; public: - explicit ColumnDataConsumer(ColumnDataCollection &collection); + ColumnDataConsumer(ColumnDataCollection &collection, vector column_ids); + + idx_t ChunkCount() const { + return chunk_count; + } public: //! Initialize the scan of the ColumnDataCollection void InitializeScan(); - //! Assign a chunk index to scan - bool Assign(idx_t &assigned_chunk_index); - //! Scan the chunk given the index - void Scan(ColumnDataConsumerLocalState &state, idx_t chunk_index, DataChunk &chunk); + //! Assign a chunk to the scan state + bool AssignChunk(ColumnDataConsumerScanState &state); + //! Scan the assigned chunk + void ScanChunk(ColumnDataConsumerScanState &state, DataChunk &chunk) const; //! Indicate that scanning the chunk is done - void MarkAsDone(idx_t chunk_index); + void FinishChunk(ColumnDataConsumerScanState &state); + +private: + void ConsumeChunks(idx_t delete_index_start, idx_t delete_index_end); private: mutex lock; //! The collection being scanned ColumnDataCollection &collection; + //! The column ids to scan + vector column_ids; //! The number of chunk references idx_t chunk_count; //! The chunks (in order) to be scanned vector chunk_references; //! Current index into "chunks" idx_t current_chunk_index; - //! Of all the chunks that are marked IN_PROGRESS, this is the lowest index - idx_t minimum_chunk_index_in_progress; + //! Chunks currently in progress + unordered_set chunks_in_progress; + //! The data has been consumed up to this chunk index + idx_t chunk_delete_index; }; } // namespace duckdb diff --git a/src/include/duckdb/execution/join_hashtable.hpp b/src/include/duckdb/execution/join_hashtable.hpp index 0a421f515d17..bdd1f0c09e1f 100644 --- a/src/include/duckdb/execution/join_hashtable.hpp +++ b/src/include/duckdb/execution/join_hashtable.hpp @@ -10,6 +10,7 @@ #include "duckdb/common/common.hpp" #include "duckdb/common/radix_partitioning.hpp" +#include "duckdb/common/types/column_data_consumer.hpp" #include "duckdb/common/types/data_chunk.hpp" #include "duckdb/common/types/null_value.hpp" #include "duckdb/common/types/row_data_collection.hpp" @@ -241,25 +242,6 @@ class JoinHashTable { //===--------------------------------------------------------------------===// // External Join //===--------------------------------------------------------------------===// - //! Thread-global scan state for the probe spill - struct ProbeSpillGlobalScanState { - ColumnDataParallelScanState scan_state; - vector>> partition_chunk_references; - - idx_t probe_partition_idx; - idx_t probe_chunk_idx; - }; - - //! Thread-local scan state for the probe spill - struct ProbeSpillLocalScanState { - ColumnDataLocalScanState scan_state; - - idx_t probe_partition_index = DConstants::INVALID_INDEX; - idx_t probe_chunk_index; - idx_t probe_segment_index; - idx_t probe_row_index; - }; - //! ProbeSpill represents materialized probe-side data that could not be probed during PhysicalHashJoin::Execute //! because the HashTable did not fit in memory. The ProbeSpill is not partitioned if the remaining data can be //! dealt with in just 1 more round of probing, otherwise it is radix partitioned in the same way as the HashTable @@ -267,6 +249,7 @@ class JoinHashTable { public: ProbeSpill(JoinHashTable &ht, ClientContext &context, const vector &probe_types); + public: //! Create an append state for a new thread and assign an index idx_t RegisterThread(); //! Append a chunk to this ProbeSpill @@ -274,15 +257,11 @@ class JoinHashTable { //! Finalize by merging the thread-local accumulated data void Finalize(); - //! The number of chunks in the current probe round - idx_t ChunkCount(ProbeSpillGlobalScanState &gstate) const; - + public: //! Prepare the next probe round - void PrepareNextProbe(ProbeSpillGlobalScanState &gstate); - //! Get the indices to scan next - bool GetScanIndex(ProbeSpillGlobalScanState &gstate, ProbeSpillLocalScanState &lstate); - //! Scans the chunk with the given index - void ScanChunk(ProbeSpillGlobalScanState &gstate, ProbeSpillLocalScanState &lstate, DataChunk &chunk); + void PrepareNextProbe(); + //! Scans and consumes the ColumnDataCollection + unique_ptr consumer; private: JoinHashTable &ht; From bd167f618aa317836cd8310a8b3f48ecc0da7279 Mon Sep 17 00:00:00 2001 From: Laurens Kuiper Date: Fri, 7 Oct 2022 15:46:57 +0200 Subject: [PATCH 17/49] always init probespill and re-enable anti/outer external joins --- .../operator/join/physical_hash_join.cpp | 24 +++++++++---------- src/execution/operator/join/physical_join.cpp | 3 +-- 2 files changed, 13 insertions(+), 14 deletions(-) diff --git a/src/execution/operator/join/physical_hash_join.cpp b/src/execution/operator/join/physical_hash_join.cpp index 6ed67469e8b0..6eee70b853b8 100644 --- a/src/execution/operator/join/physical_hash_join.cpp +++ b/src/execution/operator/join/physical_hash_join.cpp @@ -458,6 +458,18 @@ OperatorResultType PhysicalHashJoin::Execute(ExecutionContext &context, DataChun D_ASSERT(sink.finalized); D_ASSERT(!sink.scanned_data); + // some initialization for external hash join + if (sink.external) { + if (!sink.probe_spill) { + // initialize probe spill if not yet done + sink.InitializeProbeSpill(context.client); + } + if (state.thread_idx == DConstants::INVALID_INDEX) { + // assign thread index + state.thread_idx = sink.probe_spill->RegisterThread(); + } + } + if (sink.hash_table->Count() == 0 && EmptyResultIfRHSIsEmpty()) { return OperatorResultType::FINISHED; } @@ -483,18 +495,6 @@ OperatorResultType PhysicalHashJoin::Execute(ExecutionContext &context, DataChun return OperatorResultType::NEED_MORE_INPUT; } - // some initialization for external hash join - if (sink.external) { - if (!sink.probe_spill) { - // initialize probe spill if not yet done - sink.InitializeProbeSpill(context.client); - } - if (state.thread_idx == DConstants::INVALID_INDEX) { - // assign thread index - state.thread_idx = sink.probe_spill->RegisterThread(); - } - } - // resolve the join keys for the left chunk state.join_keys.Reset(); state.probe_executor.Execute(input, state.join_keys); diff --git a/src/execution/operator/join/physical_join.cpp b/src/execution/operator/join/physical_join.cpp index b181d32b9848..2d9788a8d46d 100644 --- a/src/execution/operator/join/physical_join.cpp +++ b/src/execution/operator/join/physical_join.cpp @@ -47,8 +47,7 @@ void PhysicalJoin::BuildJoinPipelines(Executor &executor, Pipeline ¤t, Pip if (join_op.type == PhysicalOperatorType::HASH_JOIN) { auto &hash_join_op = (PhysicalHashJoin &)join_op; - hash_join_op.can_go_external = !state.recursive_cte && !IsRightOuterJoin(join_op.join_type) && - join_op.join_type != JoinType::ANTI && join_op.join_type != JoinType::MARK; + hash_join_op.can_go_external = !state.recursive_cte && join_op.join_type != JoinType::MARK; if (hash_join_op.can_go_external) { add_child_pipeline = true; } From 3bcde351b38ff0cdec716bac23c05a00e0ff470a Mon Sep 17 00:00:00 2001 From: Laurens Kuiper Date: Mon, 10 Oct 2022 13:20:16 +0200 Subject: [PATCH 18/49] delete block handles/pins more cleanly --- src/common/types/column_data_allocator.cpp | 5 -- src/common/types/column_data_consumer.cpp | 16 ++-- .../operator/join/physical_hash_join.cpp | 74 ++++++++++--------- .../common/types/column_data_allocator.hpp | 7 +- 4 files changed, 55 insertions(+), 47 deletions(-) diff --git a/src/common/types/column_data_allocator.cpp b/src/common/types/column_data_allocator.cpp index f6f6ae0e7b0c..e932ab8beda0 100644 --- a/src/common/types/column_data_allocator.cpp +++ b/src/common/types/column_data_allocator.cpp @@ -116,11 +116,6 @@ data_ptr_t ColumnDataAllocator::GetDataPointer(ChunkManagementState &state, uint return state.handles[block_id].Ptr() + offset; } -void ColumnDataAllocator::Reset() { - blocks.clear(); - allocated_data.clear(); -} - void ColumnDataAllocator::DeleteBlock(uint32_t block_id) { blocks[block_id].handle->SetCanDestroy(true); } diff --git a/src/common/types/column_data_consumer.cpp b/src/common/types/column_data_consumer.cpp index 120bfc378972..a91ee5786ecd 100644 --- a/src/common/types/column_data_consumer.cpp +++ b/src/common/types/column_data_consumer.cpp @@ -9,7 +9,7 @@ ColumnDataConsumer::ColumnDataConsumer(ColumnDataCollection &collection_p, vecto void ColumnDataConsumer::InitializeScan() { chunk_count = collection.ChunkCount(); current_chunk_index = 0; - current_chunk_index = 0; + chunk_delete_index = DConstants::INVALID_INDEX; // Initialize chunk references and sort them, so we can scan them in a sane order, regardless of how it was created chunk_references.reserve(chunk_count); @@ -25,6 +25,7 @@ bool ColumnDataConsumer::AssignChunk(ColumnDataConsumerScanState &state) { lock_guard guard(lock); if (current_chunk_index == chunk_count) { // All chunks have been assigned + state.current_chunk_state.handles.clear(); state.chunk_index = DConstants::INVALID_INDEX; return false; } @@ -67,15 +68,18 @@ void ColumnDataConsumer::ConsumeChunks(idx_t delete_index_start, idx_t delete_in } auto &prev_chunk_ref = chunk_references[chunk_index - 1]; auto &curr_chunk_ref = chunk_references[chunk_index]; + auto prev_min_block_id = prev_chunk_ref.GetMinimumBlockID(); + auto curr_min_block_id = curr_chunk_ref.GetMinimumBlockID(); if (prev_chunk_ref.segment->allocator.get() != curr_chunk_ref.segment->allocator.get()) { - // Moved to the next allocator, reset the previous one - prev_chunk_ref.segment->allocator->Reset(); + // Moved to the next allocator, delete all remaining blocks in the previous one + auto &prev_allocator = *prev_chunk_ref.segment->allocator; + for (uint32_t block_id = prev_min_block_id; block_id < prev_allocator.BlockCount(); block_id++) { + prev_allocator.DeleteBlock(block_id); + } continue; } // Same allocator, see if we can delete blocks - auto prev_min_block_id = prev_chunk_ref.GetMinimumBlockID(); - auto curr_min_block_id = curr_chunk_ref.GetMinimumBlockID(); - for (idx_t block_id = prev_min_block_id; block_id < curr_min_block_id; block_id++) { + for (uint32_t block_id = prev_min_block_id; block_id < curr_min_block_id; block_id++) { prev_chunk_ref.segment->allocator->DeleteBlock(block_id); } } diff --git a/src/execution/operator/join/physical_hash_join.cpp b/src/execution/operator/join/physical_hash_join.cpp index 6eee70b853b8..ca3b6ca8ee24 100644 --- a/src/execution/operator/join/physical_hash_join.cpp +++ b/src/execution/operator/join/physical_hash_join.cpp @@ -523,6 +523,8 @@ class HashJoinGlobalSourceState : public GlobalSourceState { //! Initialize this source state using the info in the sink void Initialize(ClientContext &context, HashJoinGlobalSinkState &sink); + //! Try to prepare the next stage + void TryPrepareNextStage(HashJoinGlobalSinkState &sink); //! Prepare the next build/probe stage for external hash join (must hold lock) void PrepareBuild(HashJoinGlobalSinkState &sink); void PrepareProbe(HashJoinGlobalSinkState &sink); @@ -638,6 +640,35 @@ void HashJoinGlobalSourceState::Initialize(ClientContext &context, HashJoinGloba initialized = true; } +void HashJoinGlobalSourceState::TryPrepareNextStage(HashJoinGlobalSinkState &sink) { + lock_guard guard(lock); + switch (global_stage.load()) { + case HashJoinSourceStage::BUILD: + if (build_block_done == build_block_count) { + sink.hash_table->finalized = true; + PrepareProbe(sink); + } + break; + case HashJoinSourceStage::PROBE: + if (probe_chunk_done == probe_chunk_count) { + if (IsRightOuterJoin(op.join_type)) { + global_stage = HashJoinSourceStage::SCAN_HT; + } else { + PrepareBuild(sink); + } + } + break; + case HashJoinSourceStage::SCAN_HT: + if (full_outer_scan.scanned == full_outer_scan.total) { + PrepareBuild(sink); + return; + } + break; + default: + break; + } +} + void HashJoinGlobalSourceState::PrepareBuild(HashJoinGlobalSinkState &sink) { D_ASSERT(global_stage != HashJoinSourceStage::BUILD); auto &ht = *sink.hash_table; @@ -669,16 +700,6 @@ void HashJoinGlobalSourceState::PrepareProbe(HashJoinGlobalSinkState &sink) { } global_stage = HashJoinSourceStage::PROBE; - if (probe_chunk_count > 0) { - return; - } - - // Empty probe collection, go straight into the next stage - if (IsRightOuterJoin(op.join_type)) { - global_stage = HashJoinSourceStage::SCAN_HT; - } else { - PrepareBuild(sink); - } } bool HashJoinGlobalSourceState::AssignTask(HashJoinGlobalSinkState &sink, HashJoinLocalSourceState &lstate) { @@ -775,10 +796,6 @@ void HashJoinLocalSourceState::ExternalBuild(HashJoinGlobalSinkState &sink, Hash lock_guard guard(gstate.lock); gstate.build_block_done += build_block_idx_end - build_block_idx_start; - if (gstate.build_block_done == gstate.build_block_count) { - ht.finalized = true; - gstate.PrepareProbe(sink); - } } void HashJoinLocalSourceState::ExternalProbe(HashJoinGlobalSinkState &sink, HashJoinGlobalSourceState &gstate, @@ -792,13 +809,7 @@ void HashJoinLocalSourceState::ExternalProbe(HashJoinGlobalSinkState &sink, Hash scan_structure = nullptr; sink.probe_spill->consumer->FinishChunk(probe_local_scan); lock_guard lock(gstate.lock); - if (++gstate.probe_chunk_done == gstate.probe_chunk_count) { - if (IsRightOuterJoin(gstate.op.join_type)) { - gstate.global_stage = HashJoinSourceStage::SCAN_HT; - } else { - gstate.PrepareBuild(sink); - } - } + gstate.probe_chunk_done++; } return; } @@ -831,10 +842,6 @@ void HashJoinLocalSourceState::ExternalScan(HashJoinGlobalSinkState &sink, HashJ auto &fo_ss = gstate.full_outer_scan; fo_ss.scanned += full_outer_in_progress; full_outer_in_progress = 0; - if (fo_ss.scanned == fo_ss.total) { - gstate.PrepareBuild(sink); - return; - } } void HashJoinLocalSourceState::ScanFullOuter(HashJoinGlobalSinkState &sink, HashJoinGlobalSourceState &gstate) { @@ -863,9 +870,7 @@ void PhysicalHashJoin::GetData(ExecutionContext &context, DataChunk &chunk, Glob return; } D_ASSERT(can_go_external); - // TODO: scan and consume CDC's - // Allocate buffer chunks with capacity of 128 - // String handling in CDC's + // TODO: String handling in CDC's if (gstate.global_stage == HashJoinSourceStage::INIT) { gstate.Initialize(context.client, sink); @@ -883,12 +888,15 @@ void PhysicalHashJoin::GetData(ExecutionContext &context, DataChunk &chunk, Glob // Any call to GetData must produce tuples, otherwise the pipeline executor thinks that we're done // Therefore, we loop until we've produced tuples, or until the operator is actually done while (gstate.global_stage != HashJoinSourceStage::DONE && chunk.size() == 0) { - if (lstate.TaskFinished()) { - if (!gstate.AssignTask(sink, lstate)) { - continue; // Cannot assign work, spinlock - } + if (!lstate.TaskFinished()) { + lstate.ExecuteTask(sink, gstate, chunk); + } else if (!gstate.AssignTask(sink, lstate)) { + gstate.TryPrepareNextStage(sink); } - lstate.ExecuteTask(sink, gstate, chunk); + } + + if (gstate.global_stage == HashJoinSourceStage::DONE) { + lstate.probe_local_scan.current_chunk_state.handles.clear(); } } diff --git a/src/include/duckdb/common/types/column_data_allocator.hpp b/src/include/duckdb/common/types/column_data_allocator.hpp index bf44da4328dd..d0082b7f343b 100644 --- a/src/include/duckdb/common/types/column_data_allocator.hpp +++ b/src/include/duckdb/common/types/column_data_allocator.hpp @@ -40,6 +40,9 @@ class ColumnDataAllocator { void MakeShared() { shared = true; } + idx_t BlockCount() const { + return blocks.size(); + } public: void AllocateData(idx_t size, uint32_t &block_id, uint32_t &offset, ChunkManagementState *chunk_state); @@ -48,8 +51,6 @@ class ColumnDataAllocator { void InitializeChunkState(ChunkManagementState &state, ChunkMetaData &meta_data); data_ptr_t GetDataPointer(ChunkManagementState &state, uint32_t block_id, uint32_t offset); - //! Resets all owned data - void Reset(); //! Deletes the block with the given id void DeleteBlock(uint32_t block_id); @@ -58,7 +59,7 @@ class ColumnDataAllocator { void AllocateBlock(); BufferHandle Pin(uint32_t block_id); - bool HasBlocks() { + bool HasBlocks() const { return !blocks.empty(); } From 39ef799f3ab43a057a0193079080fe9380db9f83 Mon Sep 17 00:00:00 2001 From: Laurens Kuiper Date: Mon, 10 Oct 2022 14:33:58 +0200 Subject: [PATCH 19/49] enable out-of-core for anti/semi/mark joins --- src/execution/operator/join/physical_hash_join.cpp | 14 ++++++-------- src/execution/operator/join/physical_join.cpp | 2 +- 2 files changed, 7 insertions(+), 9 deletions(-) diff --git a/src/execution/operator/join/physical_hash_join.cpp b/src/execution/operator/join/physical_hash_join.cpp index ca3b6ca8ee24..5c7a63bca198 100644 --- a/src/execution/operator/join/physical_hash_join.cpp +++ b/src/execution/operator/join/physical_hash_join.cpp @@ -572,7 +572,7 @@ class HashJoinLocalSourceState : public LocalSourceState { //! Build, probe and scan for external hash join void ExternalBuild(HashJoinGlobalSinkState &sink, HashJoinGlobalSourceState &gstate); void ExternalProbe(HashJoinGlobalSinkState &sink, HashJoinGlobalSourceState &gstate, DataChunk &chunk); - void ExternalScan(HashJoinGlobalSinkState &sink, HashJoinGlobalSourceState &gstate, DataChunk &chunk); + void ExternalScanHT(HashJoinGlobalSinkState &sink, HashJoinGlobalSourceState &gstate, DataChunk &chunk); //! Scans the HT for full/outer join void ScanFullOuter(HashJoinGlobalSinkState &sink, HashJoinGlobalSourceState &gstate); @@ -661,7 +661,6 @@ void HashJoinGlobalSourceState::TryPrepareNextStage(HashJoinGlobalSinkState &sin case HashJoinSourceStage::SCAN_HT: if (full_outer_scan.scanned == full_outer_scan.total) { PrepareBuild(sink); - return; } break; default: @@ -767,7 +766,7 @@ void HashJoinLocalSourceState::ExecuteTask(HashJoinGlobalSinkState &sink, HashJo ExternalProbe(sink, gstate, chunk); break; case HashJoinSourceStage::SCAN_HT: - ExternalScan(sink, gstate, chunk); + ExternalScanHT(sink, gstate, chunk); break; default: throw InternalException("Unexpected HashJoinSourceStage in ExecuteTask!"); @@ -827,8 +826,8 @@ void HashJoinLocalSourceState::ExternalProbe(HashJoinGlobalSinkState &sink, Hash scan_structure->Next(join_keys, payload, chunk); } -void HashJoinLocalSourceState::ExternalScan(HashJoinGlobalSinkState &sink, HashJoinGlobalSourceState &gstate, - DataChunk &chunk) { +void HashJoinLocalSourceState::ExternalScanHT(HashJoinGlobalSinkState &sink, HashJoinGlobalSourceState &gstate, + DataChunk &chunk) { D_ASSERT(local_stage == HashJoinSourceStage::SCAN_HT && full_outer_in_progress != 0); if (full_outer_found_entries != 0) { @@ -870,7 +869,6 @@ void PhysicalHashJoin::GetData(ExecutionContext &context, DataChunk &chunk, Glob return; } D_ASSERT(can_go_external); - // TODO: String handling in CDC's if (gstate.global_stage == HashJoinSourceStage::INIT) { gstate.Initialize(context.client, sink); @@ -888,9 +886,9 @@ void PhysicalHashJoin::GetData(ExecutionContext &context, DataChunk &chunk, Glob // Any call to GetData must produce tuples, otherwise the pipeline executor thinks that we're done // Therefore, we loop until we've produced tuples, or until the operator is actually done while (gstate.global_stage != HashJoinSourceStage::DONE && chunk.size() == 0) { - if (!lstate.TaskFinished()) { + if (!lstate.TaskFinished() || gstate.AssignTask(sink, lstate)) { lstate.ExecuteTask(sink, gstate, chunk); - } else if (!gstate.AssignTask(sink, lstate)) { + } else { gstate.TryPrepareNextStage(sink); } } diff --git a/src/execution/operator/join/physical_join.cpp b/src/execution/operator/join/physical_join.cpp index 2d9788a8d46d..b3c118a7dde9 100644 --- a/src/execution/operator/join/physical_join.cpp +++ b/src/execution/operator/join/physical_join.cpp @@ -47,7 +47,7 @@ void PhysicalJoin::BuildJoinPipelines(Executor &executor, Pipeline ¤t, Pip if (join_op.type == PhysicalOperatorType::HASH_JOIN) { auto &hash_join_op = (PhysicalHashJoin &)join_op; - hash_join_op.can_go_external = !state.recursive_cte && join_op.join_type != JoinType::MARK; + hash_join_op.can_go_external = !state.recursive_cte; if (hash_join_op.can_go_external) { add_child_pipeline = true; } From 7eba41d32a24072e86548ff858aa063889567a88 Mon Sep 17 00:00:00 2001 From: Laurens Kuiper Date: Mon, 10 Oct 2022 16:00:55 +0200 Subject: [PATCH 20/49] add ProbeSpillLocalState to prevent some segfaultage --- src/execution/join_hashtable.cpp | 25 +++++++++++-------- .../operator/join/physical_hash_join.cpp | 15 +++++------ .../duckdb/execution/join_hashtable.hpp | 15 ++++++++--- 3 files changed, 34 insertions(+), 21 deletions(-) diff --git a/src/execution/join_hashtable.cpp b/src/execution/join_hashtable.cpp index b0cdcc9c7b28..2452d453fed1 100644 --- a/src/execution/join_hashtable.cpp +++ b/src/execution/join_hashtable.cpp @@ -16,6 +16,7 @@ namespace duckdb { using ValidityBytes = JoinHashTable::ValidityBytes; using ScanStructure = JoinHashTable::ScanStructure; using ProbeSpill = JoinHashTable::ProbeSpill; +using ProbeSpillLocalState = JoinHashTable::ProbeSpillLocalState; JoinHashTable::JoinHashTable(BufferManager &buffer_manager, const vector &conditions, vector btypes, JoinType type) @@ -1149,7 +1150,7 @@ static void CreateSpillChunk(DataChunk &spill_chunk, DataChunk &keys, DataChunk } unique_ptr JoinHashTable::ProbeAndSpill(DataChunk &keys, DataChunk &payload, ProbeSpill &probe_spill, - idx_t thread_idx, DataChunk &spill_chunk) { + ProbeSpillLocalState &spill_state, DataChunk &spill_chunk) { // hash all the keys Vector hashes(LogicalType::HASH); Hash(keys, *FlatVector::IncrementalSelectionVector(), keys.size(), hashes); @@ -1168,7 +1169,7 @@ unique_ptr JoinHashTable::ProbeAndSpill(DataChunk &keys, DataChun // can't probe these values right now, append to spill spill_chunk.Slice(false_sel, false_count); spill_chunk.Verify(); - probe_spill.Append(spill_chunk, thread_idx); + probe_spill.Append(spill_chunk, spill_state); // slice the stuff we CAN probe right now hashes.Slice(true_sel, true_count); @@ -1207,28 +1208,32 @@ ProbeSpill::ProbeSpill(JoinHashTable &ht, ClientContext &context, const vector guard(lock); if (partitioned) { - thread_idx = local_partition_append_states.size(); local_partitions.emplace_back(global_partitions->CreateShared()); local_partition_append_states.emplace_back(make_unique()); local_partitions.back()->InitializeAppendState(*local_partition_append_states.back()); + + result.local_partition = local_partitions.back().get(); + result.local_partition_append_state = local_partition_append_states.back().get(); } else { - thread_idx = local_spill_collections.size(); local_spill_collections.emplace_back(make_unique(context, probe_types)); local_spill_append_states.emplace_back(make_unique()); local_spill_collections.back()->InitializeAppend(*local_spill_append_states.back()); + + result.local_spill_collection = local_spill_collections.back().get(); + result.local_spill_append_state = local_spill_append_states.back().get(); } - return thread_idx; + return result; } -void ProbeSpill::Append(DataChunk &chunk, idx_t thread_idx) { +void ProbeSpill::Append(DataChunk &chunk, ProbeSpillLocalState &local_state) { if (partitioned) { - local_partitions[thread_idx]->Append(*local_partition_append_states[thread_idx], chunk); + local_state.local_partition->Append(*local_state.local_partition_append_state, chunk); } else { - local_spill_collections[thread_idx]->Append(*local_spill_append_states[thread_idx], chunk); + local_state.local_spill_collection->Append(*local_state.local_spill_append_state, chunk); } } diff --git a/src/execution/operator/join/physical_hash_join.cpp b/src/execution/operator/join/physical_hash_join.cpp index 5c7a63bca198..1af16b3a5b63 100644 --- a/src/execution/operator/join/physical_hash_join.cpp +++ b/src/execution/operator/join/physical_hash_join.cpp @@ -413,8 +413,7 @@ SinkFinalizeType PhysicalHashJoin::Finalize(Pipeline &pipeline, Event &event, Cl //===--------------------------------------------------------------------===// class HashJoinOperatorState : public OperatorState { public: - explicit HashJoinOperatorState(Allocator &allocator) - : probe_executor(allocator), thread_idx(DConstants::INVALID_INDEX) { + explicit HashJoinOperatorState(Allocator &allocator) : probe_executor(allocator), initialized(false) { } DataChunk join_keys; @@ -422,8 +421,9 @@ class HashJoinOperatorState : public OperatorState { unique_ptr scan_structure; unique_ptr perfect_hash_join_state; + bool initialized; + JoinHashTable::ProbeSpillLocalState spill_state; //! Chunk to sink data into for external join - idx_t thread_idx; DataChunk spill_chunk; public: @@ -464,9 +464,10 @@ OperatorResultType PhysicalHashJoin::Execute(ExecutionContext &context, DataChun // initialize probe spill if not yet done sink.InitializeProbeSpill(context.client); } - if (state.thread_idx == DConstants::INVALID_INDEX) { - // assign thread index - state.thread_idx = sink.probe_spill->RegisterThread(); + if (!state.initialized) { + // initialize local state if not yet done + state.spill_state = sink.probe_spill->RegisterThread(); + state.initialized = true; } } @@ -502,7 +503,7 @@ OperatorResultType PhysicalHashJoin::Execute(ExecutionContext &context, DataChun // perform the actual probe if (sink.external) { state.scan_structure = sink.hash_table->ProbeAndSpill(state.join_keys, input, *sink.probe_spill, - state.thread_idx, state.spill_chunk); + state.spill_state, state.spill_chunk); } else { state.scan_structure = sink.hash_table->Probe(state.join_keys); } diff --git a/src/include/duckdb/execution/join_hashtable.hpp b/src/include/duckdb/execution/join_hashtable.hpp index bdd1f0c09e1f..72cf24598d24 100644 --- a/src/include/duckdb/execution/join_hashtable.hpp +++ b/src/include/duckdb/execution/join_hashtable.hpp @@ -242,6 +242,13 @@ class JoinHashTable { //===--------------------------------------------------------------------===// // External Join //===--------------------------------------------------------------------===// + struct ProbeSpillLocalState { + PartitionedColumnData *local_partition; + PartitionedColumnDataAppendState *local_partition_append_state; + + ColumnDataCollection *local_spill_collection; + ColumnDataAppendState *local_spill_append_state; + }; //! ProbeSpill represents materialized probe-side data that could not be probed during PhysicalHashJoin::Execute //! because the HashTable did not fit in memory. The ProbeSpill is not partitioned if the remaining data can be //! dealt with in just 1 more round of probing, otherwise it is radix partitioned in the same way as the HashTable @@ -250,10 +257,10 @@ class JoinHashTable { ProbeSpill(JoinHashTable &ht, ClientContext &context, const vector &probe_types); public: - //! Create an append state for a new thread and assign an index - idx_t RegisterThread(); + //! Create a state for a new thread + ProbeSpillLocalState RegisterThread(); //! Append a chunk to this ProbeSpill - void Append(DataChunk &chunk, idx_t thread_idx); + void Append(DataChunk &chunk, ProbeSpillLocalState &local_state); //! Finalize by merging the thread-local accumulated data void Finalize(); @@ -327,7 +334,7 @@ class JoinHashTable { bool PrepareExternalFinalize(); //! Probe whatever we can, sink the rest into a thread-local HT unique_ptr ProbeAndSpill(DataChunk &keys, DataChunk &payload, ProbeSpill &probe_spill, - idx_t thread_idx, DataChunk &spill_chunk); + ProbeSpillLocalState &spill_state, DataChunk &spill_chunk); private: //! First and last partition of the current partitioned round From 8cfb1891dc483898f102e49dc3c94b678a01c62e Mon Sep 17 00:00:00 2001 From: Laurens Kuiper Date: Mon, 10 Oct 2022 16:15:02 +0200 Subject: [PATCH 21/49] simplify external GetData init --- .../operator/join/physical_hash_join.cpp | 29 ++++--------------- 1 file changed, 6 insertions(+), 23 deletions(-) diff --git a/src/execution/operator/join/physical_hash_join.cpp b/src/execution/operator/join/physical_hash_join.cpp index 1af16b3a5b63..6bbf8c53bbbf 100644 --- a/src/execution/operator/join/physical_hash_join.cpp +++ b/src/execution/operator/join/physical_hash_join.cpp @@ -540,7 +540,6 @@ class HashJoinGlobalSourceState : public GlobalSourceState { const PhysicalHashJoin &op; //! For synchronizing the external hash join - atomic initialized; atomic global_stage; mutex lock; @@ -615,18 +614,15 @@ unique_ptr PhysicalHashJoin::GetLocalSourceState(ExecutionCont } HashJoinGlobalSourceState::HashJoinGlobalSourceState(const PhysicalHashJoin &op, ClientContext &context) - : op(op), initialized(false), global_stage(HashJoinSourceStage::INIT), + : op(op), global_stage(HashJoinSourceStage::INIT), probe_chunk_count(0), probe_chunk_done(0), probe_count(op.children[0]->estimated_cardinality), parallel_scan_chunk_count(context.config.verify_parallelism ? 1 : 120) { } void HashJoinGlobalSourceState::Initialize(ClientContext &context, HashJoinGlobalSinkState &sink) { - if (initialized) { - return; - } lock_guard init_lock(lock); - if (initialized) { - // Have to check if anything changed since we got the lock + if (global_stage != HashJoinSourceStage::INIT) { + // Another thread initialized return; } full_outer_scan.total = sink.hash_table->Count(); @@ -638,7 +634,7 @@ void HashJoinGlobalSourceState::Initialize(ClientContext &context, HashJoinGloba // Finalize the probe spill too sink.probe_spill->Finalize(); - initialized = true; + global_stage = HashJoinSourceStage::PROBE; } void HashJoinGlobalSourceState::TryPrepareNextStage(HashJoinGlobalSinkState &sink) { @@ -717,7 +713,7 @@ bool HashJoinGlobalSourceState::AssignTask(HashJoinGlobalSinkState &sink, HashJo } break; case HashJoinSourceStage::PROBE: - if (sink.probe_spill->consumer->AssignChunk(lstate.probe_local_scan)) { + if (sink.probe_spill->consumer && sink.probe_spill->consumer->AssignChunk(lstate.probe_local_scan)) { lstate.local_stage = global_stage; return true; } @@ -869,19 +865,10 @@ void PhysicalHashJoin::GetData(ExecutionContext &context, DataChunk &chunk, Glob } return; } - D_ASSERT(can_go_external); + D_ASSERT(can_go_external); if (gstate.global_stage == HashJoinSourceStage::INIT) { gstate.Initialize(context.client, sink); - - lock_guard lock(gstate.lock); - if (gstate.global_stage == HashJoinSourceStage::INIT) { - if (IsRightOuterJoin(join_type)) { - gstate.global_stage = HashJoinSourceStage::SCAN_HT; - } else { - gstate.PrepareBuild(sink); - } - } } // Any call to GetData must produce tuples, otherwise the pipeline executor thinks that we're done @@ -893,10 +880,6 @@ void PhysicalHashJoin::GetData(ExecutionContext &context, DataChunk &chunk, Glob gstate.TryPrepareNextStage(sink); } } - - if (gstate.global_stage == HashJoinSourceStage::DONE) { - lstate.probe_local_scan.current_chunk_state.handles.clear(); - } } } // namespace duckdb From 8d015076cdf134fc89ab476a042576f56d71f6cd Mon Sep 17 00:00:00 2001 From: Laurens Kuiper Date: Tue, 11 Oct 2022 09:44:50 +0200 Subject: [PATCH 22/49] some code cleanup and trying to make CI happy --- src/common/radix_partitioning.cpp | 6 +++- src/common/types/column_data_consumer.cpp | 24 +++++++++++--- src/common/types/data_chunk.cpp | 2 +- src/common/types/partitioned_column_data.cpp | 1 + src/execution/join_hashtable.cpp | 11 ++++--- .../operator/join/physical_hash_join.cpp | 2 +- .../common/types/column_data_consumer.hpp | 13 ++------ .../duckdb/execution/join_hashtable.hpp | 32 +++++++++---------- 8 files changed, 52 insertions(+), 39 deletions(-) diff --git a/src/common/radix_partitioning.cpp b/src/common/radix_partitioning.cpp index c9bd0c58e304..e598cfad0f69 100644 --- a/src/common/radix_partitioning.cpp +++ b/src/common/radix_partitioning.cpp @@ -6,6 +6,7 @@ #include "duckdb/common/types/row_layout.hpp" #include "duckdb/common/types/vector.hpp" #include "duckdb/common/vector_operations/binary_executor.hpp" +#include "duckdb/common/vector_operations/unary_executor.hpp" namespace duckdb { @@ -184,7 +185,10 @@ struct PartitionFunctor { auto &data_blocks = block_collection.blocks; auto &heap_blocks = string_heap.blocks; - for (idx_t block_idx = 0; block_idx < data_blocks.size(); block_idx++) { + for (idx_t block_idx_plus_one = data_blocks.size(); block_idx_plus_one > 0; block_idx_plus_one--) { + // We loop through blocks in reverse to save some of that PRECIOUS I/O + idx_t block_idx = block_idx_plus_one - 1; + RowDataBlock *data_block; BufferHandle data_handle; data_ptr_t data_ptr; diff --git a/src/common/types/column_data_consumer.cpp b/src/common/types/column_data_consumer.cpp index a91ee5786ecd..d71f35d64b79 100644 --- a/src/common/types/column_data_consumer.cpp +++ b/src/common/types/column_data_consumer.cpp @@ -1,7 +1,20 @@ #include "duckdb/common/types/column_data_consumer.hpp" +#include + namespace duckdb { +using ChunkReference = ColumnDataConsumer::ChunkReference; + +ChunkReference::ChunkReference(ColumnDataCollectionSegment *segment_p, uint32_t chunk_index_p) + : segment(segment_p), chunk_index_in_segment(chunk_index_p) { +} + +uint32_t ChunkReference::GetMinimumBlockID() const { + const auto &block_ids = segment->chunk_data[chunk_index_in_segment].block_ids; + return *std::min_element(block_ids.begin(), block_ids.end()); +} + ColumnDataConsumer::ColumnDataConsumer(ColumnDataCollection &collection_p, vector column_ids) : collection(collection_p), column_ids(move(column_ids)) { } @@ -68,19 +81,20 @@ void ColumnDataConsumer::ConsumeChunks(idx_t delete_index_start, idx_t delete_in } auto &prev_chunk_ref = chunk_references[chunk_index - 1]; auto &curr_chunk_ref = chunk_references[chunk_index]; + auto prev_allocator = prev_chunk_ref.segment->allocator.get(); + auto curr_allocator = curr_chunk_ref.segment->allocator.get(); auto prev_min_block_id = prev_chunk_ref.GetMinimumBlockID(); auto curr_min_block_id = curr_chunk_ref.GetMinimumBlockID(); - if (prev_chunk_ref.segment->allocator.get() != curr_chunk_ref.segment->allocator.get()) { + if (prev_allocator != curr_allocator) { // Moved to the next allocator, delete all remaining blocks in the previous one - auto &prev_allocator = *prev_chunk_ref.segment->allocator; - for (uint32_t block_id = prev_min_block_id; block_id < prev_allocator.BlockCount(); block_id++) { - prev_allocator.DeleteBlock(block_id); + for (uint32_t block_id = prev_min_block_id; block_id < prev_allocator->BlockCount(); block_id++) { + prev_allocator->DeleteBlock(block_id); } continue; } // Same allocator, see if we can delete blocks for (uint32_t block_id = prev_min_block_id; block_id < curr_min_block_id; block_id++) { - prev_chunk_ref.segment->allocator->DeleteBlock(block_id); + prev_allocator->DeleteBlock(block_id); } } } diff --git a/src/common/types/data_chunk.cpp b/src/common/types/data_chunk.cpp index b144f4c90fde..f3e0ba0c1e2b 100644 --- a/src/common/types/data_chunk.cpp +++ b/src/common/types/data_chunk.cpp @@ -145,8 +145,8 @@ void DataChunk::Split(DataChunk &other, idx_t split_idx) { data.pop_back(); vector_caches.pop_back(); } - other.SetCardinality(*this); other.SetCapacity(*this); + other.SetCardinality(*this); } void DataChunk::Fuse(DataChunk &other) { diff --git a/src/common/types/partitioned_column_data.cpp b/src/common/types/partitioned_column_data.cpp index e3aa91e9cb65..dc2eb92a3810 100644 --- a/src/common/types/partitioned_column_data.cpp +++ b/src/common/types/partitioned_column_data.cpp @@ -1,6 +1,7 @@ #include "duckdb/common/types/partitioned_column_data.hpp" #include "duckdb/common/radix_partitioning.hpp" +#include "duckdb/storage/buffer_manager.hpp" namespace duckdb { diff --git a/src/execution/join_hashtable.cpp b/src/execution/join_hashtable.cpp index 2452d453fed1..77e43e245024 100644 --- a/src/execution/join_hashtable.cpp +++ b/src/execution/join_hashtable.cpp @@ -16,7 +16,7 @@ namespace duckdb { using ValidityBytes = JoinHashTable::ValidityBytes; using ScanStructure = JoinHashTable::ScanStructure; using ProbeSpill = JoinHashTable::ProbeSpill; -using ProbeSpillLocalState = JoinHashTable::ProbeSpillLocalState; +using ProbeSpillLocalState = JoinHashTable::ProbeSpillLocalAppendState; JoinHashTable::JoinHashTable(BufferManager &buffer_manager, const vector &conditions, vector btypes, JoinType type) @@ -90,7 +90,7 @@ void JoinHashTable::Merge(JoinHashTable &other) { } } - lock_guard lock(partition_lock); + lock_guard lock(partitioned_data_lock); if (partition_block_collections.empty()) { D_ASSERT(partition_string_heaps.empty()); // Move partitions to this HT @@ -1150,7 +1150,8 @@ static void CreateSpillChunk(DataChunk &spill_chunk, DataChunk &keys, DataChunk } unique_ptr JoinHashTable::ProbeAndSpill(DataChunk &keys, DataChunk &payload, ProbeSpill &probe_spill, - ProbeSpillLocalState &spill_state, DataChunk &spill_chunk) { + ProbeSpillLocalAppendState &spill_state, + DataChunk &spill_chunk) { // hash all the keys Vector hashes(LogicalType::HASH); Hash(keys, *FlatVector::IncrementalSelectionVector(), keys.size(), hashes); @@ -1209,7 +1210,7 @@ ProbeSpill::ProbeSpill(JoinHashTable &ht, ClientContext &context, const vector guard(lock); if (partitioned) { local_partitions.emplace_back(global_partitions->CreateShared()); @@ -1229,7 +1230,7 @@ ProbeSpillLocalState ProbeSpill::RegisterThread() { return result; } -void ProbeSpill::Append(DataChunk &chunk, ProbeSpillLocalState &local_state) { +void ProbeSpill::Append(DataChunk &chunk, ProbeSpillLocalAppendState &local_state) { if (partitioned) { local_state.local_partition->Append(*local_state.local_partition_append_state, chunk); } else { diff --git a/src/execution/operator/join/physical_hash_join.cpp b/src/execution/operator/join/physical_hash_join.cpp index 6bbf8c53bbbf..667176bd5948 100644 --- a/src/execution/operator/join/physical_hash_join.cpp +++ b/src/execution/operator/join/physical_hash_join.cpp @@ -422,7 +422,7 @@ class HashJoinOperatorState : public OperatorState { unique_ptr perfect_hash_join_state; bool initialized; - JoinHashTable::ProbeSpillLocalState spill_state; + JoinHashTable::ProbeSpillLocalAppendState spill_state; //! Chunk to sink data into for external join DataChunk spill_chunk; diff --git a/src/include/duckdb/common/types/column_data_consumer.hpp b/src/include/duckdb/common/types/column_data_consumer.hpp index 3e54ac2953cf..cea95f08e337 100644 --- a/src/include/duckdb/common/types/column_data_consumer.hpp +++ b/src/include/duckdb/common/types/column_data_consumer.hpp @@ -21,18 +21,11 @@ struct ColumnDataConsumerScanState { //! ColumnDataConsumer can scan a ColumnDataCollection, and consume it in the process, i.e., read blocks are deleted class ColumnDataConsumer { -private: +public: struct ChunkReference { public: - ChunkReference(ColumnDataCollectionSegment *segment_p, uint32_t chunk_index_p) - : segment(segment_p), chunk_index_in_segment(chunk_index_p) { - } - - inline uint32_t GetMinimumBlockID() const { - const auto &block_ids = segment->chunk_data[chunk_index_in_segment].block_ids; - return *std::min_element(block_ids.begin(), block_ids.end()); - } - + ChunkReference(ColumnDataCollectionSegment *segment_p, uint32_t chunk_index_p); + uint32_t GetMinimumBlockID() const; friend bool operator<(const ChunkReference &lhs, const ChunkReference &rhs) { // Sort by allocator first if (lhs.segment->allocator.get() != rhs.segment->allocator.get()) { diff --git a/src/include/duckdb/execution/join_hashtable.hpp b/src/include/duckdb/execution/join_hashtable.hpp index 72cf24598d24..773e49414007 100644 --- a/src/include/duckdb/execution/join_hashtable.hpp +++ b/src/include/duckdb/execution/join_hashtable.hpp @@ -242,10 +242,11 @@ class JoinHashTable { //===--------------------------------------------------------------------===// // External Join //===--------------------------------------------------------------------===// - struct ProbeSpillLocalState { + struct ProbeSpillLocalAppendState { + //! Local partition and append state (if partitioned) PartitionedColumnData *local_partition; PartitionedColumnDataAppendState *local_partition_append_state; - + //! Local spill and append state (if not partitioned) ColumnDataCollection *local_spill_collection; ColumnDataAppendState *local_spill_append_state; }; @@ -258,9 +259,9 @@ class JoinHashTable { public: //! Create a state for a new thread - ProbeSpillLocalState RegisterThread(); + ProbeSpillLocalAppendState RegisterThread(); //! Append a chunk to this ProbeSpill - void Append(DataChunk &chunk, ProbeSpillLocalState &local_state); + void Append(DataChunk &chunk, ProbeSpillLocalAppendState &local_state); //! Finalize by merging the thread-local accumulated data void Finalize(); @@ -275,12 +276,12 @@ class JoinHashTable { mutex lock; ClientContext &context; + //! Whether the probe data is partitioned + bool partitioned; //! The types of the probe DataChunks const vector &probe_types; //! The column ids vector column_ids; - //! Whether the probe data is partitioned - bool partitioned; //! The partitioned probe data (if partitioned) and append states unique_ptr global_partitions; @@ -303,18 +304,19 @@ class JoinHashTable { idx_t tuples_per_round; //! The number of tuples that are swizzled - idx_t SwizzledCount() { + idx_t SwizzledCount() const { return swizzled_block_collection->count; } //! Size of the in-memory data - idx_t SizeInBytes() { + idx_t SizeInBytes() const { return block_collection->SizeInBytes() + string_heap->SizeInBytes(); } //! Size of the swizzled data - idx_t SwizzledSize() { + idx_t SwizzledSize() const { return swizzled_block_collection->SizeInBytes() + swizzled_string_heap->SizeInBytes(); } - idx_t PointerTableCapacity(idx_t count) { + //! Capacity of the pointer table given the + static idx_t PointerTableCapacity(idx_t count) { return NextPowerOfTwo(MaxValue(count * 2, (Storage::BLOCK_SIZE / sizeof(data_ptr_t)) + 1)); } @@ -334,21 +336,19 @@ class JoinHashTable { bool PrepareExternalFinalize(); //! Probe whatever we can, sink the rest into a thread-local HT unique_ptr ProbeAndSpill(DataChunk &keys, DataChunk &payload, ProbeSpill &probe_spill, - ProbeSpillLocalState &spill_state, DataChunk &spill_chunk); + ProbeSpillLocalAppendState &spill_state, DataChunk &spill_chunk); private: - //! First and last partition of the current partitioned round + //! First and last partition of the current probe round idx_t partition_start; idx_t partition_end; - //! The RowDataCollection holding the swizzled main data of the hash table + //! Swizzled row data unique_ptr swizzled_block_collection; - //! The stringheap accompanying the swizzled main data unique_ptr swizzled_string_heap; - //! Partitioned data lock - mutex partition_lock; //! Partitioned data + mutex partitioned_data_lock; vector> partition_block_collections; vector> partition_string_heaps; }; From 96ef927290d3c43282a328077e17c483a56f867a Mon Sep 17 00:00:00 2001 From: Laurens Kuiper Date: Tue, 11 Oct 2022 15:29:41 +0200 Subject: [PATCH 23/49] fix bug with CountValid (took me FOREVER to find out this was the problem) --- src/include/duckdb/common/types/validity_mask.hpp | 6 +++--- 1 file changed, 3 insertions(+), 3 deletions(-) diff --git a/src/include/duckdb/common/types/validity_mask.hpp b/src/include/duckdb/common/types/validity_mask.hpp index e5a86f59587a..8dd21c45ec90 100644 --- a/src/include/duckdb/common/types/validity_mask.hpp +++ b/src/include/duckdb/common/types/validity_mask.hpp @@ -9,9 +9,9 @@ #pragma once #include "duckdb/common/common.hpp" +#include "duckdb/common/to_string.hpp" #include "duckdb/common/types.hpp" #include "duckdb/common/vector_size.hpp" -#include "duckdb/common/to_string.hpp" namespace duckdb { struct ValidityMask; @@ -115,8 +115,8 @@ struct TemplatedValidityMask { const auto entry_count = EntryCount(count); for (idx_t entry_idx = 0; entry_idx < entry_count;) { auto entry = GetValidityEntry(entry_idx++); - // Handle ragged end - if (entry_idx == entry_count) { + // Handle ragged end (if not exactly multiple of BITS_PER_VALUE) + if (entry_idx == entry_count && count % BITS_PER_VALUE != 0) { idx_t idx_in_entry; GetEntryIndex(count, entry_idx, idx_in_entry); for (idx_t i = 0; i < idx_in_entry; ++i) { From d22121b700b113e273f5cfb251637f75eab24ea3 Mon Sep 17 00:00:00 2001 From: Laurens Kuiper Date: Tue, 11 Oct 2022 15:30:05 +0200 Subject: [PATCH 24/49] always init probespill --- src/execution/join_hashtable.cpp | 17 +++++++++-------- .../operator/join/physical_hash_join.cpp | 19 ++++++++----------- 2 files changed, 17 insertions(+), 19 deletions(-) diff --git a/src/execution/join_hashtable.cpp b/src/execution/join_hashtable.cpp index 77e43e245024..25f2ed270b79 100644 --- a/src/execution/join_hashtable.cpp +++ b/src/execution/join_hashtable.cpp @@ -1262,14 +1262,15 @@ void ProbeSpill::Finalize() { void ProbeSpill::PrepareNextProbe() { if (partitioned) { auto &partitions = global_partitions->GetPartitions(); - if (ht.partition_start == partitions.size()) { - return; - } - - // Move specific partitions to the global spill collection - global_spill_collection = move(partitions[ht.partition_start]); - for (idx_t i = ht.partition_start + 1; i < ht.partition_end; i++) { - global_spill_collection->Combine(*partitions[i]); + if (partitions.empty() || ht.partition_start == partitions.size()) { + // Can't probe, just put an empty one + global_spill_collection = make_unique(context, probe_types); + } else { + // Move specific partitions to the global spill collection + global_spill_collection = move(partitions[ht.partition_start]); + for (idx_t i = ht.partition_start + 1; i < ht.partition_end; i++) { + global_spill_collection->Combine(*partitions[i]); + } } } consumer = make_unique(*global_spill_collection, column_ids); diff --git a/src/execution/operator/join/physical_hash_join.cpp b/src/execution/operator/join/physical_hash_join.cpp index 667176bd5948..64cbec0b115e 100644 --- a/src/execution/operator/join/physical_hash_join.cpp +++ b/src/execution/operator/join/physical_hash_join.cpp @@ -446,6 +446,7 @@ unique_ptr PhysicalHashJoin::GetOperatorState(ExecutionContext &c } if (sink.external) { state->spill_chunk.Initialize(allocator, sink.probe_types); + sink.InitializeProbeSpill(context.client); } return move(state); @@ -459,16 +460,10 @@ OperatorResultType PhysicalHashJoin::Execute(ExecutionContext &context, DataChun D_ASSERT(!sink.scanned_data); // some initialization for external hash join - if (sink.external) { - if (!sink.probe_spill) { - // initialize probe spill if not yet done - sink.InitializeProbeSpill(context.client); - } - if (!state.initialized) { - // initialize local state if not yet done - state.spill_state = sink.probe_spill->RegisterThread(); - state.initialized = true; - } + if (sink.external && !state.initialized) { + // initialize local state if not yet done + state.spill_state = sink.probe_spill->RegisterThread(); + state.initialized = true; } if (sink.hash_table->Count() == 0 && EmptyResultIfRHSIsEmpty()) { @@ -632,7 +627,9 @@ void HashJoinGlobalSourceState::Initialize(ClientContext &context, HashJoinGloba build_blocks_per_thread = MaxValue((num_blocks + num_threads - 1) / num_threads, 1); // Finalize the probe spill too - sink.probe_spill->Finalize(); + if (sink.probe_spill) { + sink.probe_spill->Finalize(); + } global_stage = HashJoinSourceStage::PROBE; } From 652ae1a171e486e72e5012623d8b24be1560fc33 Mon Sep 17 00:00:00 2001 From: Laurens Kuiper Date: Tue, 11 Oct 2022 15:54:22 +0200 Subject: [PATCH 25/49] trying to please windows CI --- src/common/radix_partitioning.cpp | 4 ++-- src/common/types/partitioned_column_data.cpp | 6 +++--- src/include/duckdb/common/types/column_data_consumer.hpp | 1 + .../duckdb/common/types/partitioned_column_data.hpp | 8 ++------ 4 files changed, 8 insertions(+), 11 deletions(-) diff --git a/src/common/radix_partitioning.cpp b/src/common/radix_partitioning.cpp index e598cfad0f69..b05b2995cd35 100644 --- a/src/common/radix_partitioning.cpp +++ b/src/common/radix_partitioning.cpp @@ -441,8 +441,8 @@ void RadixPartitionedColumnData::InitializeAppendStateInternal(PartitionedColumn state.partition_buffers.reserve(num_partitions); state.partition_append_states.reserve(num_partitions); for (idx_t i = 0; i < num_partitions; i++) { - state.partition_append_states.emplace_back(); - partitions[i]->InitializeAppend(state.partition_append_states[i]); + state.partition_append_states.emplace_back(make_unique()); + partitions[i]->InitializeAppend(*state.partition_append_states[i]); state.partition_buffers.emplace_back(CreatePartitionBuffer()); } } diff --git a/src/common/types/partitioned_column_data.cpp b/src/common/types/partitioned_column_data.cpp index dc2eb92a3810..8f731da363d2 100644 --- a/src/common/types/partitioned_column_data.cpp +++ b/src/common/types/partitioned_column_data.cpp @@ -55,7 +55,7 @@ void PartitionedColumnData::Append(PartitionedColumnDataAppendState &state, Data const auto &partition_index = partition_entries.begin()->first; auto &partition = *partitions[partition_index]; auto &partition_append_state = state.partition_append_states[partition_index]; - partition.Append(partition_append_state, input); + partition.Append(*partition_append_state, input); return; } @@ -99,14 +99,14 @@ void PartitionedColumnData::Append(PartitionedColumnDataAppendState &state, Data state.slice_chunk.Slice(input, partition_sel, partition_length); // Append it to the partition directly - partition.Append(partition_append_state, state.slice_chunk); + partition.Append(*partition_append_state, state.slice_chunk); } else { // Append the input chunk to the partition buffer using the selection vector partition_buffer.Append(input, false, &partition_sel, partition_length); if (partition_buffer.size() >= HalfBufferSize()) { // Next batch won't fit in the buffer, flush it to the partition - partition.Append(partition_append_state, partition_buffer); + partition.Append(*partition_append_state, partition_buffer); partition_buffer.Reset(); partition_buffer.SetCapacity(BufferSize()); } diff --git a/src/include/duckdb/common/types/column_data_consumer.hpp b/src/include/duckdb/common/types/column_data_consumer.hpp index cea95f08e337..2b6117f69b4d 100644 --- a/src/include/duckdb/common/types/column_data_consumer.hpp +++ b/src/include/duckdb/common/types/column_data_consumer.hpp @@ -10,6 +10,7 @@ #include "duckdb/common/types/column_data_collection.hpp" #include "duckdb/common/types/column_data_collection_segment.hpp" +#include "duckdb/common/types/column_data_scan_states.hpp" namespace duckdb { diff --git a/src/include/duckdb/common/types/partitioned_column_data.hpp b/src/include/duckdb/common/types/partitioned_column_data.hpp index 88f1a3cb73ff..3023b2585a2c 100644 --- a/src/include/duckdb/common/types/partitioned_column_data.hpp +++ b/src/include/duckdb/common/types/partitioned_column_data.hpp @@ -16,7 +16,7 @@ namespace duckdb { //! Local state for parallel partitioning struct PartitionedColumnDataAppendState { public: - explicit PartitionedColumnDataAppendState() : partition_indices(LogicalType::UBIGINT) { + PartitionedColumnDataAppendState() : partition_indices(LogicalType::UBIGINT) { } public: @@ -25,11 +25,7 @@ struct PartitionedColumnDataAppendState { DataChunk slice_chunk; vector> partition_buffers; - vector partition_append_states; - -private: - //! Implicit copying is not allowed - PartitionedColumnDataAppendState(const PartitionedColumnDataAppendState &) = delete; + vector> partition_append_states; }; enum class PartitionedColumnDataType : uint8_t { RADIX, INVALID }; From fb6bfc01a0e7f9cfcb056205aa4cffad3e03954c Mon Sep 17 00:00:00 2001 From: Laurens Kuiper Date: Tue, 11 Oct 2022 16:30:16 +0200 Subject: [PATCH 26/49] fix race condition and nullptr --- src/execution/join_hashtable.cpp | 12 ++++++++---- src/execution/operator/join/physical_hash_join.cpp | 4 +++- 2 files changed, 11 insertions(+), 5 deletions(-) diff --git a/src/execution/join_hashtable.cpp b/src/execution/join_hashtable.cpp index 25f2ed270b79..2575488839ca 100644 --- a/src/execution/join_hashtable.cpp +++ b/src/execution/join_hashtable.cpp @@ -1250,9 +1250,13 @@ void ProbeSpill::Finalize() { local_partitions.clear(); local_partition_append_states.clear(); } else { - global_spill_collection = move(local_spill_collections[0]); - for (idx_t i = 1; i < local_spill_collections.size(); i++) { - global_spill_collection->Combine(*local_spill_collections[i]); + if (local_spill_collections.empty()) { + global_spill_collection = make_unique(context, probe_types); + } else { + global_spill_collection = move(local_spill_collections[0]); + for (idx_t i = 1; i < local_spill_collections.size(); i++) { + global_spill_collection->Combine(*local_spill_collections[i]); + } } local_spill_collections.clear(); local_spill_append_states.clear(); @@ -1263,7 +1267,7 @@ void ProbeSpill::PrepareNextProbe() { if (partitioned) { auto &partitions = global_partitions->GetPartitions(); if (partitions.empty() || ht.partition_start == partitions.size()) { - // Can't probe, just put an empty one + // Can't probe, just make an empty one global_spill_collection = make_unique(context, probe_types); } else { // Move specific partitions to the global spill collection diff --git a/src/execution/operator/join/physical_hash_join.cpp b/src/execution/operator/join/physical_hash_join.cpp index 64cbec0b115e..e43003d2ce20 100644 --- a/src/execution/operator/join/physical_hash_join.cpp +++ b/src/execution/operator/join/physical_hash_join.cpp @@ -461,7 +461,9 @@ OperatorResultType PhysicalHashJoin::Execute(ExecutionContext &context, DataChun // some initialization for external hash join if (sink.external && !state.initialized) { - // initialize local state if not yet done + if (!sink.probe_spill) { + sink.InitializeProbeSpill(context.client); + } state.spill_state = sink.probe_spill->RegisterThread(); state.initialized = true; } From 917706f0037057324d3ab5d9ed11da90bbc138d2 Mon Sep 17 00:00:00 2001 From: Laurens Kuiper Date: Tue, 11 Oct 2022 16:40:24 +0200 Subject: [PATCH 27/49] revert some change to prep for PR --- benchmark/include/benchmark.hpp | 2 +- src/include/duckdb/common/radix_partitioning.hpp | 14 +++++++++++++- .../common/types/partitioned_column_data.hpp | 2 +- test/sql/join/external/simple_external_join.test | 3 --- 4 files changed, 15 insertions(+), 6 deletions(-) diff --git a/benchmark/include/benchmark.hpp b/benchmark/include/benchmark.hpp index ad79e4571f58..f25a6b8deb73 100644 --- a/benchmark/include/benchmark.hpp +++ b/benchmark/include/benchmark.hpp @@ -31,7 +31,7 @@ struct BenchmarkState { //! new benchmarks class Benchmark { constexpr static size_t DEFAULT_NRUNS = 5; - constexpr static size_t DEFAULT_TIMEOUT = 300; + constexpr static size_t DEFAULT_TIMEOUT = 30; Benchmark(Benchmark &) = delete; diff --git a/src/include/duckdb/common/radix_partitioning.hpp b/src/include/duckdb/common/radix_partitioning.hpp index ffb8f5973e7b..10b161002a07 100644 --- a/src/include/duckdb/common/radix_partitioning.hpp +++ b/src/include/duckdb/common/radix_partitioning.hpp @@ -69,7 +69,19 @@ class RadixPartitionedColumnData : public PartitionedColumnData { // Radix Partitioning interface implementation //===--------------------------------------------------------------------===// idx_t BufferSize() const override { - return STANDARD_VECTOR_SIZE; + switch (radix_bits) { + case 1: + case 2: + case 3: + case 4: + return 1024; + case 5: + return 512; + case 6: + return 256; + default: + return 128; + } } void InitializeAppendStateInternal(PartitionedColumnDataAppendState &state) const override; void ComputePartitionIndices(PartitionedColumnDataAppendState &state, DataChunk &input) override; diff --git a/src/include/duckdb/common/types/partitioned_column_data.hpp b/src/include/duckdb/common/types/partitioned_column_data.hpp index 3023b2585a2c..ea94dcb72c81 100644 --- a/src/include/duckdb/common/types/partitioned_column_data.hpp +++ b/src/include/duckdb/common/types/partitioned_column_data.hpp @@ -55,7 +55,7 @@ class PartitionedColumnData { //! Get the partitions in this PartitionedColumnData vector> &GetPartitions(); -private: +protected: //===--------------------------------------------------------------------===// // Partitioning type implementation interface //===--------------------------------------------------------------------===// diff --git a/test/sql/join/external/simple_external_join.test b/test/sql/join/external/simple_external_join.test index 806069448a9a..15c620ad709d 100644 --- a/test/sql/join/external/simple_external_join.test +++ b/test/sql/join/external/simple_external_join.test @@ -8,9 +8,6 @@ pragma verify_external statement ok pragma verify_parallelism -#statement ok -#pragma threads=1 - # Create tables with large string values so that we have to swizzle strings # The tables have some overlapping values to keep the join result small statement ok From b24f48d0b7abf4425ccc14151f3b2fb73f526534 Mon Sep 17 00:00:00 2001 From: Laurens Kuiper Date: Wed, 12 Oct 2022 10:20:36 +0200 Subject: [PATCH 28/49] make BufferSize never exceed STANDARD_VECTOR_SIZE --- src/common/radix_partitioning.cpp | 1 + src/common/types/partitioned_column_data.cpp | 23 +++++++++++++------ .../duckdb/common/radix_partitioning.hpp | 12 ++++++---- .../common/types/partitioned_column_data.hpp | 2 +- 4 files changed, 26 insertions(+), 12 deletions(-) diff --git a/src/common/radix_partitioning.cpp b/src/common/radix_partitioning.cpp index b05b2995cd35..14770f90f5d8 100644 --- a/src/common/radix_partitioning.cpp +++ b/src/common/radix_partitioning.cpp @@ -441,6 +441,7 @@ void RadixPartitionedColumnData::InitializeAppendStateInternal(PartitionedColumn state.partition_buffers.reserve(num_partitions); state.partition_append_states.reserve(num_partitions); for (idx_t i = 0; i < num_partitions; i++) { + // TODO only initialize the append if partition idx > ... state.partition_append_states.emplace_back(make_unique()); partitions[i]->InitializeAppend(*state.partition_append_states[i]); state.partition_buffers.emplace_back(CreatePartitionBuffer()); diff --git a/src/common/types/partitioned_column_data.cpp b/src/common/types/partitioned_column_data.cpp index 8f731da363d2..2dc7a41d57d7 100644 --- a/src/common/types/partitioned_column_data.cpp +++ b/src/common/types/partitioned_column_data.cpp @@ -40,14 +40,23 @@ void PartitionedColumnData::Append(PartitionedColumnDataAppendState &state, Data const auto count = input.size(); unordered_map partition_entries; const auto partition_indices = FlatVector::GetData(state.partition_indices); - for (idx_t i = 0; i < count; i++) { - const auto &partition_index = partition_indices[i]; - auto partition_entry = partition_entries.find(partition_index); - if (partition_entry == partition_entries.end()) { - partition_entries[partition_index] = list_entry_t(0, 1); - } else { - partition_entry->second.length++; + switch (state.partition_indices.GetVectorType()) { + case VectorType::FLAT_VECTOR: + for (idx_t i = 0; i < count; i++) { + const auto &partition_index = partition_indices[i]; + auto partition_entry = partition_entries.find(partition_index); + if (partition_entry == partition_entries.end()) { + partition_entries[partition_index] = list_entry_t(0, 1); + } else { + partition_entry->second.length++; + } } + break; + case VectorType::CONSTANT_VECTOR: + partition_entries[partition_indices[0]] = list_entry_t(0, count); + break; + default: + throw InternalException("Unexpected VectorType in PartitionedColumnData::Append"); } // Early out: check if everything belongs to a single partition diff --git a/src/include/duckdb/common/radix_partitioning.hpp b/src/include/duckdb/common/radix_partitioning.hpp index 10b161002a07..fd98ffb5c87e 100644 --- a/src/include/duckdb/common/radix_partitioning.hpp +++ b/src/include/duckdb/common/radix_partitioning.hpp @@ -74,18 +74,22 @@ class RadixPartitionedColumnData : public PartitionedColumnData { case 2: case 3: case 4: - return 1024; + return GetBufferSize(1); case 5: - return 512; + return GetBufferSize(2); case 6: - return 256; + return GetBufferSize(3); default: - return 128; + return GetBufferSize(4); } } void InitializeAppendStateInternal(PartitionedColumnDataAppendState &state) const override; void ComputePartitionIndices(PartitionedColumnDataAppendState &state, DataChunk &input) override; + static constexpr idx_t GetBufferSize(idx_t div) { + return STANDARD_VECTOR_SIZE / div == 0 ? 1 : STANDARD_VECTOR_SIZE / div; + } + private: //! The number of radix bits const idx_t radix_bits; diff --git a/src/include/duckdb/common/types/partitioned_column_data.hpp b/src/include/duckdb/common/types/partitioned_column_data.hpp index ea94dcb72c81..ce382d99ff6c 100644 --- a/src/include/duckdb/common/types/partitioned_column_data.hpp +++ b/src/include/duckdb/common/types/partitioned_column_data.hpp @@ -61,7 +61,7 @@ class PartitionedColumnData { //===--------------------------------------------------------------------===// //! Size of the buffers in the append states for this type of partitioning (default 128) virtual idx_t BufferSize() const { - return 128; + return MinValue(128, STANDARD_VECTOR_SIZE); } //! Initialize a PartitionedColumnDataAppendState for this type of partitioning (optional) virtual void InitializeAppendStateInternal(PartitionedColumnDataAppendState &state) const { From 9f8aba0cb99b7ebcbe0576accd26de5fa46caeec Mon Sep 17 00:00:00 2001 From: Laurens Kuiper Date: Tue, 18 Oct 2022 16:18:14 +0200 Subject: [PATCH 29/49] trying to debug union pipeline scheduling --- src/common/symbols.cpp | 1 + 1 file changed, 1 insertion(+) diff --git a/src/common/symbols.cpp b/src/common/symbols.cpp index c963880a9020..8021f771b615 100644 --- a/src/common/symbols.cpp +++ b/src/common/symbols.cpp @@ -175,6 +175,7 @@ template class std::vector; INSTANTIATE_VECTOR(std::vector>) INSTANTIATE_VECTOR(std::vector>) INSTANTIATE_VECTOR(std::vector>) +INSTANTIATE_VECTOR(std::vector>) template class std::vector>; template class std::vector; INSTANTIATE_VECTOR(std::vector>) From 009c61f9453d7a931dbced9c94e878d905209b2e Mon Sep 17 00:00:00 2001 From: Laurens Kuiper Date: Thu, 20 Oct 2022 09:40:32 +0200 Subject: [PATCH 30/49] add verification code for executor --- src/include/duckdb/execution/executor.hpp | 10 +++-- src/include/duckdb/parallel/event.hpp | 8 +++- src/parallel/event.cpp | 10 +++++ src/parallel/executor.cpp | 52 +++++++++++++++++++++++ 4 files changed, 75 insertions(+), 5 deletions(-) diff --git a/src/include/duckdb/execution/executor.hpp b/src/include/duckdb/execution/executor.hpp index 6dfbe9bafa7a..1a21fed75c80 100644 --- a/src/include/duckdb/execution/executor.hpp +++ b/src/include/duckdb/execution/executor.hpp @@ -9,11 +9,11 @@ #pragma once #include "duckdb/common/common.hpp" +#include "duckdb/common/enums/pending_execution_result.hpp" #include "duckdb/common/mutex.hpp" -#include "duckdb/parallel/pipeline.hpp" -#include "duckdb/common/unordered_map.hpp" #include "duckdb/common/pair.hpp" -#include "duckdb/common/enums/pending_execution_result.hpp" +#include "duckdb/common/unordered_map.hpp" +#include "duckdb/parallel/pipeline.hpp" namespace duckdb { class ClientContext; @@ -94,6 +94,10 @@ class Executor { void ScheduleEvents(); static void ScheduleEventsInternal(ScheduleEventData &event_data); + static void VerifyScheduledEvents(ScheduleEventData &event_data); + static void VerifyScheduledEventsInternal(const idx_t i, const vector> &vertices, + vector &visited, vector &recursion_stack); + static void SchedulePipeline(const shared_ptr &pipeline, ScheduleEventData &event_data, vector &scheduled_pipelines); static void ScheduleChildPipeline(Pipeline *parent, const shared_ptr &pipeline, diff --git a/src/include/duckdb/parallel/event.hpp b/src/include/duckdb/parallel/event.hpp index 088a8310985e..e7a08601afb4 100644 --- a/src/include/duckdb/parallel/event.hpp +++ b/src/include/duckdb/parallel/event.hpp @@ -34,16 +34,18 @@ class Event : public std::enable_shared_from_this { void Finish(); void AddDependency(Event &event); - bool HasDependencies() { + bool HasDependencies() const { return total_dependencies != 0; } + vector GetDependenciesVerification() const; + void CompleteDependency(); void SetTasks(vector> tasks); void InsertEvent(shared_ptr replacement_event); - bool IsFinished() { + bool IsFinished() const { return finished; } @@ -62,6 +64,8 @@ class Event : public std::enable_shared_from_this { //! The events that depend on this event to run vector> parents; + //! Raw pointers to the parents (used for verification only) + vector parents_raw; //! Whether or not the event is finished executing atomic finished; diff --git a/src/parallel/event.cpp b/src/parallel/event.cpp index bf46534654b0..4c80a52c0b4e 100644 --- a/src/parallel/event.cpp +++ b/src/parallel/event.cpp @@ -43,6 +43,13 @@ void Event::Finish() { void Event::AddDependency(Event &event) { total_dependencies++; event.parents.push_back(weak_ptr(shared_from_this())); +#ifdef DEBUG + event.parents_raw.push_back(this); +#endif +} + +vector Event::GetDependenciesVerification() const { + return parents_raw; } void Event::FinishTask() { @@ -57,6 +64,9 @@ void Event::FinishTask() { void Event::InsertEvent(shared_ptr replacement_event) { replacement_event->parents = move(parents); +#ifdef DEBUG + replacement_event->parents_raw = move(parents_raw); +#endif replacement_event->AddDependency(*this); executor.AddEvent(move(replacement_event)); } diff --git a/src/parallel/executor.cpp b/src/parallel/executor.cpp index 657837a65457..4dd1cd6903a4 100644 --- a/src/parallel/executor.cpp +++ b/src/parallel/executor.cpp @@ -211,6 +211,7 @@ void Executor::ScheduleEventsInternal(ScheduleEventData &event_data) { event->Schedule(); } } + VerifyScheduledEvents(event_data); } void Executor::ScheduleEvents() { @@ -218,6 +219,57 @@ void Executor::ScheduleEvents() { ScheduleEventsInternal(event_data); } +void Executor::VerifyScheduledEvents(ScheduleEventData &event_data) { +#ifdef DEBUG + const auto &vertices = event_data.events; + const idx_t count = vertices.size(); + vector visited(count, false); + vector recursion_stack(count, false); + for (idx_t i = 0; i < count; i++) { + VerifyScheduledEventsInternal(i, vertices, visited, recursion_stack); + } +#endif +} + +void Executor::VerifyScheduledEventsInternal(const idx_t vertex, const vector> &vertices, + vector &visited, vector &recursion_stack) { + if (visited[vertex]) { + return; // early out: we already visited this node + } + + auto dependencies = vertices[vertex]->GetDependenciesVerification(); + if (dependencies.empty()) { + return; // early out: no dependencies + } + + // create a vector the indices of the adjacent events + vector adjacent; + const idx_t count = vertices.size(); + for (auto &dep : dependencies) { + idx_t i; + for (i = 0; i < count; i++) { + if (vertices[i].get() == dep) { + adjacent.push_back(i); + break; + } + } + D_ASSERT(i != count); // dependency must be in there somewhere + } + + // mark node as visited and add to recursion stack + visited[vertex] = true; + recursion_stack[vertex] = true; + + // recurse into adjacent vertices + for (const auto &i : adjacent) { + D_ASSERT(!recursion_stack[i]); // adjacent node is in the recursion stack: circular dependency! + VerifyScheduledEventsInternal(i, vertices, visited, recursion_stack); + } + + // remove vertex from recursion stack + recursion_stack[vertex] = false; +} + void Executor::ReschedulePipelines(const vector> &pipelines, vector> &events) { unordered_map>> child_pipelines; unordered_map>> union_pipelines; From d078a2ac9a1483bc4af3ea23c4c16ca39995a9b4 Mon Sep 17 00:00:00 2001 From: Laurens Kuiper Date: Thu, 20 Oct 2022 14:01:55 +0200 Subject: [PATCH 31/49] clean up verification code a bit (still struggling to find pipeline scheduling problem) --- src/execution/operator/set/physical_union.cpp | 30 ++++++++++++++++++- src/include/duckdb/execution/executor.hpp | 6 ++-- src/parallel/event.cpp | 1 + src/parallel/executor.cpp | 22 ++++++++------ .../full_outer/full_outer_join_union.test | 14 +++++---- 5 files changed, 55 insertions(+), 18 deletions(-) diff --git a/src/execution/operator/set/physical_union.cpp b/src/execution/operator/set/physical_union.cpp index 451c9b8abcb8..c3b60d7e494d 100644 --- a/src/execution/operator/set/physical_union.cpp +++ b/src/execution/operator/set/physical_union.cpp @@ -1,6 +1,7 @@ #include "duckdb/execution/operator/set/physical_union.hpp" -#include "duckdb/parallel/thread_context.hpp" + #include "duckdb/parallel/pipeline.hpp" +#include "duckdb/parallel/thread_context.hpp" namespace duckdb { @@ -23,6 +24,18 @@ void PhysicalUnion::BuildPipelines(Executor &executor, Pipeline ¤t, Pipeli auto union_pipeline = make_shared(executor); auto pipeline_ptr = union_pipeline.get(); + + // child pipeline dependencies are in reverse order of scheduling: for 'current', dependencies are set up correctly + // however, the union pipeline may also have child pipelines + // all child pipelines of 'current' that are scheduled before going further down the pipeline + // depend on the child pipelines of the union pipeline being finished too + auto &child_pipelines = state.GetChildPipelines(executor); + vector> child_pipelines_before; + auto it = child_pipelines.find(¤t); + if (it != child_pipelines.end()) { + child_pipelines_before = it->second; + } + auto &union_pipelines = state.GetUnionPipelines(executor); // for the current pipeline, continue building on the LHS state.SetPipelineOperators(*union_pipeline, state.GetPipelineOperators(current)); @@ -33,6 +46,21 @@ void PhysicalUnion::BuildPipelines(Executor &executor, Pipeline ¤t, Pipeli // for the union pipeline, build on the RHS state.SetPipelineSink(*pipeline_ptr, state.GetPipelineSink(current)); children[1]->BuildPipelines(executor, *pipeline_ptr, state); + + if (child_pipelines_before.empty()) { + // no child pipelines, no need to set up dependencies + return; + } + + // as stated above, child_pipelines_before must depend on the child pipelines of the union pipeline + // also as stated above, child pipelines are dependent of each other in reverse order + // therefore it suffices to make child_pipelines_before[0] dependent on the 1st child pipeline of the union pipeline + it = child_pipelines.find(pipeline_ptr); + if (it != child_pipelines.end()) { + auto &union_child_pipelines = it->second; + // union_child_pipelines[0]->AddDependency(child_pipelines_before[0]); +// child_pipelines_before[0]->AddDependency(union_child_pipelines[0]); + } } vector PhysicalUnion::GetSources() const { diff --git a/src/include/duckdb/execution/executor.hpp b/src/include/duckdb/execution/executor.hpp index 1a21fed75c80..1c3e62b43569 100644 --- a/src/include/duckdb/execution/executor.hpp +++ b/src/include/duckdb/execution/executor.hpp @@ -94,9 +94,9 @@ class Executor { void ScheduleEvents(); static void ScheduleEventsInternal(ScheduleEventData &event_data); - static void VerifyScheduledEvents(ScheduleEventData &event_data); - static void VerifyScheduledEventsInternal(const idx_t i, const vector> &vertices, - vector &visited, vector &recursion_stack); + static void VerifyScheduledEvents(const ScheduleEventData &event_data); + static void VerifyScheduledEventsInternal(const idx_t i, const vector &vertices, vector &visited, + vector &recursion_stack); static void SchedulePipeline(const shared_ptr &pipeline, ScheduleEventData &event_data, vector &scheduled_pipelines); diff --git a/src/parallel/event.cpp b/src/parallel/event.cpp index 4c80a52c0b4e..254cee553170 100644 --- a/src/parallel/event.cpp +++ b/src/parallel/event.cpp @@ -49,6 +49,7 @@ void Event::AddDependency(Event &event) { } vector Event::GetDependenciesVerification() const { + D_ASSERT(parents.size() == parents_raw.size()); return parents_raw; } diff --git a/src/parallel/executor.cpp b/src/parallel/executor.cpp index 4dd1cd6903a4..581d06cd2804 100644 --- a/src/parallel/executor.cpp +++ b/src/parallel/executor.cpp @@ -219,10 +219,14 @@ void Executor::ScheduleEvents() { ScheduleEventsInternal(event_data); } -void Executor::VerifyScheduledEvents(ScheduleEventData &event_data) { +void Executor::VerifyScheduledEvents(const ScheduleEventData &event_data) { #ifdef DEBUG - const auto &vertices = event_data.events; - const idx_t count = vertices.size(); + const idx_t count = event_data.events.size(); + vector vertices; + vertices.reserve(count); + for (const auto &event : event_data.events) { + vertices.push_back(event.get()); + } vector visited(count, false); vector recursion_stack(count, false); for (idx_t i = 0; i < count; i++) { @@ -231,11 +235,12 @@ void Executor::VerifyScheduledEvents(ScheduleEventData &event_data) { #endif } -void Executor::VerifyScheduledEventsInternal(const idx_t vertex, const vector> &vertices, - vector &visited, vector &recursion_stack) { +void Executor::VerifyScheduledEventsInternal(const idx_t vertex, const vector &vertices, vector &visited, + vector &recursion_stack) { if (visited[vertex]) { - return; // early out: we already visited this node + return; // early out: we already visited this vertex } + D_ASSERT(!recursion_stack[vertex]); // this vertex is in the recursion stack: circular dependency! auto dependencies = vertices[vertex]->GetDependenciesVerification(); if (dependencies.empty()) { @@ -248,7 +253,7 @@ void Executor::VerifyScheduledEventsInternal(const idx_t vertex, const vector Date: Tue, 25 Oct 2022 16:59:39 +0200 Subject: [PATCH 32/49] initial implementation of MetaPipeline - not working yet --- src/common/symbols.cpp | 5 + .../operator/helper/physical_execute.cpp | 5 +- .../helper/physical_result_collector.cpp | 17 +- .../operator/join/physical_cross_product.cpp | 6 +- .../operator/join/physical_delim_join.cpp | 24 +- .../operator/join/physical_hash_join.cpp | 7 +- .../operator/join/physical_iejoin.cpp | 32 +- .../operator/join/physical_index_join.cpp | 12 +- src/execution/operator/join/physical_join.cpp | 37 ++- .../operator/persistent/physical_export.cpp | 6 +- .../scan/physical_column_data_scan.cpp | 9 +- .../operator/set/physical_recursive_cte.cpp | 65 ++-- src/execution/operator/set/physical_union.cpp | 51 +--- src/execution/physical_operator.cpp | 46 +-- src/include/duckdb/execution/executor.hpp | 26 +- .../operator/helper/physical_execute.hpp | 2 +- .../helper/physical_result_collector.hpp | 2 +- .../operator/join/physical_cross_product.hpp | 2 +- .../operator/join/physical_delim_join.hpp | 2 +- .../operator/join/physical_hash_join.hpp | 7 +- .../operator/join/physical_iejoin.hpp | 2 +- .../operator/join/physical_index_join.hpp | 2 +- .../execution/operator/join/physical_join.hpp | 6 +- .../operator/persistent/physical_export.hpp | 2 +- .../scan/physical_column_data_scan.hpp | 2 +- .../operator/set/physical_recursive_cte.hpp | 6 +- .../execution/operator/set/physical_union.hpp | 2 +- .../duckdb/execution/physical_operator.hpp | 10 +- .../duckdb/parallel/base_pipeline_event.hpp | 4 + src/include/duckdb/parallel/event.hpp | 5 +- src/include/duckdb/parallel/meta_pipeline.hpp | 79 +++++ src/include/duckdb/parallel/pipeline.hpp | 21 +- src/parallel/CMakeLists.txt | 1 + src/parallel/executor.cpp | 286 ++++++------------ src/parallel/meta_pipeline.cpp | 144 +++++++++ src/parallel/pipeline.cpp | 43 ++- 36 files changed, 505 insertions(+), 473 deletions(-) create mode 100644 src/include/duckdb/parallel/meta_pipeline.hpp create mode 100644 src/parallel/meta_pipeline.cpp diff --git a/src/common/symbols.cpp b/src/common/symbols.cpp index 8021f771b615..24996ba65cca 100644 --- a/src/common/symbols.cpp +++ b/src/common/symbols.cpp @@ -18,6 +18,7 @@ #include "duckdb/optimizer/join_order_optimizer.hpp" #include "duckdb/optimizer/rule.hpp" #include "duckdb/parallel/pipeline.hpp" +#include "duckdb/parallel/meta_pipeline.hpp" #include "duckdb/parser/constraint.hpp" #include "duckdb/parser/constraints/list.hpp" #include "duckdb/parser/expression/list.hpp" @@ -81,9 +82,11 @@ template class std::unique_ptr; template class std::unique_ptr; template class std::unique_ptr; template class std::unique_ptr; +template class std::shared_ptr; template class std::unique_ptr; template class std::shared_ptr; template class std::weak_ptr; +template class std::shared_ptr; template class std::unique_ptr; template class std::shared_ptr; template class std::unique_ptr; @@ -173,9 +176,11 @@ template class std::vector; template class std::vector; template class std::vector; INSTANTIATE_VECTOR(std::vector>) +INSTANTIATE_VECTOR(std::vector>) INSTANTIATE_VECTOR(std::vector>) INSTANTIATE_VECTOR(std::vector>) INSTANTIATE_VECTOR(std::vector>) +INSTANTIATE_VECTOR(std::vector>) template class std::vector>; template class std::vector; INSTANTIATE_VECTOR(std::vector>) diff --git a/src/execution/operator/helper/physical_execute.cpp b/src/execution/operator/helper/physical_execute.cpp index 68658f97b958..594fe86e8a92 100644 --- a/src/execution/operator/helper/physical_execute.cpp +++ b/src/execution/operator/helper/physical_execute.cpp @@ -10,9 +10,10 @@ vector PhysicalExecute::GetChildren() const { return {plan}; } -void PhysicalExecute::BuildPipelines(Executor &executor, Pipeline ¤t, PipelineBuildState &state) { +void PhysicalExecute::BuildPipelines(Pipeline ¤t, MetaPipeline &meta_pipeline, + vector &final_pipelines) { // EXECUTE statement: build pipeline on child - plan->BuildPipelines(executor, current, state); + meta_pipeline.Build(plan); } } // namespace duckdb diff --git a/src/execution/operator/helper/physical_result_collector.cpp b/src/execution/operator/helper/physical_result_collector.cpp index 77c7de4ba903..ad0d10653abc 100644 --- a/src/execution/operator/helper/physical_result_collector.cpp +++ b/src/execution/operator/helper/physical_result_collector.cpp @@ -37,18 +37,21 @@ vector PhysicalResultCollector::GetChildren() const { return {plan}; } -void PhysicalResultCollector::BuildPipelines(Executor &executor, Pipeline ¤t, PipelineBuildState &state) { +void PhysicalResultCollector::BuildPipelines(Pipeline ¤t, MetaPipeline &meta_pipeline, + vector &final_pipelines) { // operator is a sink, build a pipeline sink_state.reset(); - // single operator: - // the operator becomes the data source of the current pipeline - state.SetPipelineSource(current, this); - // we create a new pipeline starting from the child - D_ASSERT(children.size() == 0); + D_ASSERT(children.empty()); D_ASSERT(plan); - BuildChildPipeline(executor, current, state, plan); + // single operator: the operator becomes the data source of the current pipeline + auto &state = meta_pipeline.GetState(); + state.SetPipelineSource(current, this); + + // we create a new pipeline starting from the child + auto child_meta_pipeline = meta_pipeline.CreateChildMetaPipeline(current, this); + child_meta_pipeline->Build(plan); } } // namespace duckdb diff --git a/src/execution/operator/join/physical_cross_product.cpp b/src/execution/operator/join/physical_cross_product.cpp index dd65393eea24..4235228e841c 100644 --- a/src/execution/operator/join/physical_cross_product.cpp +++ b/src/execution/operator/join/physical_cross_product.cpp @@ -1,4 +1,5 @@ #include "duckdb/execution/operator/join/physical_cross_product.hpp" + #include "duckdb/common/types/column_data_collection.hpp" #include "duckdb/common/vector_operations/vector_operations.hpp" #include "duckdb/execution/operator/join/physical_join.hpp" @@ -135,8 +136,9 @@ OperatorResultType PhysicalCrossProduct::Execute(ExecutionContext &context, Data //===--------------------------------------------------------------------===// // Pipeline Construction //===--------------------------------------------------------------------===// -void PhysicalCrossProduct::BuildPipelines(Executor &executor, Pipeline ¤t, PipelineBuildState &state) { - PhysicalJoin::BuildJoinPipelines(executor, current, state, *this); +void PhysicalCrossProduct::BuildPipelines(Pipeline ¤t, MetaPipeline &meta_pipeline, + vector &final_pipelines) { + PhysicalJoin::BuildJoinPipelines(current, meta_pipeline, final_pipelines, *this); } vector PhysicalCrossProduct::GetSources() const { diff --git a/src/execution/operator/join/physical_delim_join.cpp b/src/execution/operator/join/physical_delim_join.cpp index 345d9c1253bb..bb52bd558b6a 100644 --- a/src/execution/operator/join/physical_delim_join.cpp +++ b/src/execution/operator/join/physical_delim_join.cpp @@ -119,34 +119,24 @@ string PhysicalDelimJoin::ParamsToString() const { //===--------------------------------------------------------------------===// // Pipeline Construction //===--------------------------------------------------------------------===// -void PhysicalDelimJoin::BuildPipelines(Executor &executor, Pipeline ¤t, PipelineBuildState &state) { +void PhysicalDelimJoin::BuildPipelines(Pipeline ¤t, MetaPipeline &meta_pipeline, + vector &final_pipelines) { op_state.reset(); sink_state.reset(); - // duplicate eliminated join - auto pipeline = make_shared(executor); - state.SetPipelineSink(*pipeline, this); - current.AddDependency(pipeline); + auto child_meta_pipeline = meta_pipeline.CreateChildMetaPipeline(current, this); + child_meta_pipeline->Build(children[0].get()); - // recurse into the pipeline child - children[0]->BuildPipelines(executor, *pipeline, state); if (type == PhysicalOperatorType::DELIM_JOIN) { // recurse into the actual join // any pipelines in there depend on the main pipeline // any scan of the duplicate eliminated data on the RHS depends on this pipeline // we add an entry to the mapping of (PhysicalOperator*) -> (Pipeline*) + auto &state = meta_pipeline.GetState(); for (auto &delim_scan : delim_scans) { - state.delim_join_dependencies[delim_scan] = pipeline.get(); + state.delim_join_dependencies[delim_scan] = child_meta_pipeline->GetRootPipeline().get(); } - join->BuildPipelines(executor, current, state); - } - if (!state.recursive_cte) { - // regular pipeline: schedule it - state.AddPipeline(executor, move(pipeline)); - } else { - // CTE pipeline! add it to the CTE pipelines - auto &cte = (PhysicalRecursiveCTE &)*state.recursive_cte; - cte.pipelines.push_back(move(pipeline)); + join->BuildPipelines(current, meta_pipeline, final_pipelines); } } diff --git a/src/execution/operator/join/physical_hash_join.cpp b/src/execution/operator/join/physical_hash_join.cpp index e43003d2ce20..0d1e7e960357 100644 --- a/src/execution/operator/join/physical_hash_join.cpp +++ b/src/execution/operator/join/physical_hash_join.cpp @@ -267,15 +267,13 @@ class HashJoinFinalizeEvent : public BasePipelineEvent { public: void Schedule() override { auto &context = pipeline->GetClientContext(); - auto parallel_construct_count = - context.config.verify_parallelism ? STANDARD_VECTOR_SIZE : PARALLEL_CONSTRUCT_COUNT; vector> finalize_tasks; auto &ht = *sink.hash_table; const auto &block_collection = ht.GetBlockCollection(); const auto &blocks = block_collection.blocks; const auto num_blocks = blocks.size(); - if (block_collection.count < parallel_construct_count) { + if (block_collection.count < PARALLEL_CONSTRUCT_THRESHOLD && !context.config.verify_parallelism) { // Single-threaded finalize finalize_tasks.push_back( make_unique(shared_from_this(), context, sink, 0, num_blocks, false)); @@ -303,8 +301,7 @@ class HashJoinFinalizeEvent : public BasePipelineEvent { sink.hash_table->finalized = true; } - // 1 << 18 TODO: tweak experimentally - static constexpr const idx_t PARALLEL_CONSTRUCT_COUNT = 262144; + static constexpr const idx_t PARALLEL_CONSTRUCT_THRESHOLD = 1048576; }; void HashJoinGlobalSinkState::ScheduleFinalize(Pipeline &pipeline, Event &event) { diff --git a/src/execution/operator/join/physical_iejoin.cpp b/src/execution/operator/join/physical_iejoin.cpp index 95de98ddf075..4156fa60b434 100644 --- a/src/execution/operator/join/physical_iejoin.cpp +++ b/src/execution/operator/join/physical_iejoin.cpp @@ -1009,32 +1009,24 @@ void PhysicalIEJoin::GetData(ExecutionContext &context, DataChunk &result, Globa //===--------------------------------------------------------------------===// // Pipeline Construction //===--------------------------------------------------------------------===// -void PhysicalIEJoin::BuildPipelines(Executor &executor, Pipeline ¤t, PipelineBuildState &state) { +void PhysicalIEJoin::BuildPipelines(Pipeline ¤t, MetaPipeline &meta_pipeline, + vector &final_pipelines) { D_ASSERT(children.size() == 2); - if (state.recursive_cte) { + auto &state = meta_pipeline.GetState(); + if (meta_pipeline.HasRecursiveCTE()) { throw NotImplementedException("IEJoins are not supported in recursive CTEs yet"); } - // Build the LHS - auto lhs_pipeline = make_shared(executor); - state.SetPipelineSink(*lhs_pipeline, this); - D_ASSERT(children[0].get()); - children[0]->BuildPipelines(executor, *lhs_pipeline, state); + // current depends on lhs + auto lhs_meta_pipeline = meta_pipeline.CreateChildMetaPipeline(current, this); + lhs_meta_pipeline->Build(children[0].get()); + auto &lhs_root_pipeline = *lhs_meta_pipeline->GetRootPipeline(); - // Build the RHS - auto rhs_pipeline = make_shared(executor); - state.SetPipelineSink(*rhs_pipeline, this); - D_ASSERT(children[1].get()); - children[1]->BuildPipelines(executor, *rhs_pipeline, state); + // lhs depends on rhs + auto rhs_pipeline = lhs_meta_pipeline->CreateChildMetaPipeline(lhs_root_pipeline, this); + rhs_pipeline->Build(children[1].get()); - // RHS => LHS => current - current.AddDependency(rhs_pipeline); - rhs_pipeline->AddDependency(lhs_pipeline); - - state.AddPipeline(executor, move(lhs_pipeline)); - state.AddPipeline(executor, move(rhs_pipeline)); - - // Now build both and scan + // this operator becomes a source for the current pipeline after RHS => LHS have been built state.SetPipelineSource(current, this); } diff --git a/src/execution/operator/join/physical_index_join.cpp b/src/execution/operator/join/physical_index_join.cpp index a9ea18f00737..8c35fdce90a6 100644 --- a/src/execution/operator/join/physical_index_join.cpp +++ b/src/execution/operator/join/physical_index_join.cpp @@ -1,14 +1,15 @@ #include "duckdb/execution/operator/join/physical_index_join.hpp" -#include "duckdb/parallel/thread_context.hpp" + #include "duckdb/common/vector_operations/vector_operations.hpp" #include "duckdb/execution/expression_executor.hpp" #include "duckdb/execution/index/art/art.hpp" #include "duckdb/execution/operator/scan/physical_table_scan.hpp" #include "duckdb/function/table/table_scan.hpp" +#include "duckdb/parallel/thread_context.hpp" #include "duckdb/storage/buffer_manager.hpp" #include "duckdb/storage/storage_manager.hpp" -#include "duckdb/transaction/transaction.hpp" #include "duckdb/storage/table/append_state.hpp" +#include "duckdb/transaction/transaction.hpp" namespace duckdb { @@ -210,12 +211,13 @@ OperatorResultType PhysicalIndexJoin::Execute(ExecutionContext &context, DataChu //===--------------------------------------------------------------------===// // Pipeline Construction //===--------------------------------------------------------------------===// -void PhysicalIndexJoin::BuildPipelines(Executor &executor, Pipeline ¤t, PipelineBuildState &state) { +void PhysicalIndexJoin::BuildPipelines(Pipeline ¤t, MetaPipeline &meta_pipeline, + vector &final_pipelines) { // index join: we only continue into the LHS // the right side is probed by the index join // so we don't need to do anything in the pipeline with this child - state.AddPipelineOperator(current, this); - children[0]->BuildPipelines(executor, current, state); + meta_pipeline.GetState().AddPipelineOperator(current, this); + children[0]->BuildPipelines(current, meta_pipeline, final_pipelines); } vector PhysicalIndexJoin::GetSources() const { diff --git a/src/execution/operator/join/physical_join.cpp b/src/execution/operator/join/physical_join.cpp index b3c118a7dde9..d18517360a68 100644 --- a/src/execution/operator/join/physical_join.cpp +++ b/src/execution/operator/join/physical_join.cpp @@ -25,12 +25,13 @@ bool PhysicalJoin::EmptyResultIfRHSIsEmpty() const { //===--------------------------------------------------------------------===// // Pipeline Construction //===--------------------------------------------------------------------===// -void PhysicalJoin::BuildJoinPipelines(Executor &executor, Pipeline ¤t, PipelineBuildState &state, - PhysicalOperator &op) { +void PhysicalJoin::BuildJoinPipelines(Pipeline ¤t, MetaPipeline &meta_pipeline, + vector &final_pipelines, PhysicalOperator &op) { op.op_state.reset(); op.sink_state.reset(); // 'current' is the probe pipeline: add this operator + auto &state = meta_pipeline.GetState(); state.AddPipelineOperator(current, &op); // Join can become a source operator if it's RIGHT/OUTER, or if the hash join goes out-of-core @@ -39,7 +40,7 @@ void PhysicalJoin::BuildJoinPipelines(Executor &executor, Pipeline ¤t, Pip if (op.type != PhysicalOperatorType::CROSS_PRODUCT) { auto &join_op = (PhysicalJoin &)op; if (IsRightOuterJoin(join_op.join_type)) { - if (state.recursive_cte) { + if (meta_pipeline.HasRecursiveCTE()) { throw NotImplementedException("FULL and RIGHT outer joins are not supported in recursive CTEs yet"); } add_child_pipeline = true; @@ -47,26 +48,38 @@ void PhysicalJoin::BuildJoinPipelines(Executor &executor, Pipeline ¤t, Pip if (join_op.type == PhysicalOperatorType::HASH_JOIN) { auto &hash_join_op = (PhysicalHashJoin &)join_op; - hash_join_op.can_go_external = !state.recursive_cte; + hash_join_op.can_go_external = !meta_pipeline.HasRecursiveCTE(); if (hash_join_op.can_go_external) { add_child_pipeline = true; } } + } - if (add_child_pipeline) { - state.AddChildPipeline(executor, current); + if (add_child_pipeline) { + // create child pipeline + auto child_pipeline = meta_pipeline.CreateChildPipeline(current); + // create a new vector to set up dependencies + vector child_pipeline_dependencies; + // continue building the LHS pipeline (probe child) + op.children[0]->BuildPipelines(current, meta_pipeline, child_pipeline_dependencies); + // child pipeline depends on the downstream child pipelines to have finished (if any) + for (auto dependee : child_pipeline_dependencies) { + meta_pipeline.AddInterPipelineDependency(child_pipeline, dependee); } + // the child pipeline needs to finish before the MetaPipeline is finished + final_pipelines.push_back(child_pipeline); + } else { + // continue building the LHS pipeline (probe child) + op.children[0]->BuildPipelines(current, meta_pipeline, final_pipelines); } - // continue building the LHS pipeline (probe child) - op.children[0]->BuildPipelines(executor, current, state); - // on the RHS (build side), we construct a new child pipeline with this pipeline as its source - op.BuildChildPipeline(executor, current, state, op.children[1].get()); + auto child_meta_pipeline = meta_pipeline.CreateChildMetaPipeline(current, &op); + child_meta_pipeline->Build(op.children[1].get()); } -void PhysicalJoin::BuildPipelines(Executor &executor, Pipeline ¤t, PipelineBuildState &state) { - PhysicalJoin::BuildJoinPipelines(executor, current, state, *this); +void PhysicalJoin::BuildPipelines(Pipeline ¤t, MetaPipeline &meta_pipeline, vector &final_pipelines) { + PhysicalJoin::BuildJoinPipelines(current, meta_pipeline, final_pipelines, *this); } vector PhysicalJoin::GetSources() const { diff --git a/src/execution/operator/persistent/physical_export.cpp b/src/execution/operator/persistent/physical_export.cpp index 77e886f3781b..44ca823653ef 100644 --- a/src/execution/operator/persistent/physical_export.cpp +++ b/src/execution/operator/persistent/physical_export.cpp @@ -191,14 +191,16 @@ SinkResultType PhysicalExport::Sink(ExecutionContext &context, GlobalSinkState & //===--------------------------------------------------------------------===// // Pipeline Construction //===--------------------------------------------------------------------===// -void PhysicalExport::BuildPipelines(Executor &executor, Pipeline ¤t, PipelineBuildState &state) { +void PhysicalExport::BuildPipelines(Pipeline ¤t, MetaPipeline &meta_pipeline, + vector &final_pipelines) { // EXPORT has an optional child // we only need to schedule child pipelines if there is a child + auto &state = meta_pipeline.GetState(); state.SetPipelineSource(current, this); if (children.empty()) { return; } - PhysicalOperator::BuildPipelines(executor, current, state); + PhysicalOperator::BuildPipelines(current, meta_pipeline, final_pipelines); } vector PhysicalExport::GetSources() const { diff --git a/src/execution/operator/scan/physical_column_data_scan.cpp b/src/execution/operator/scan/physical_column_data_scan.cpp index 46f591fb6d8d..1de87d9cedd6 100644 --- a/src/execution/operator/scan/physical_column_data_scan.cpp +++ b/src/execution/operator/scan/physical_column_data_scan.cpp @@ -1,6 +1,7 @@ #include "duckdb/execution/operator/scan/physical_column_data_scan.hpp" -#include "duckdb/parallel/pipeline.hpp" + #include "duckdb/execution/operator/join/physical_delim_join.hpp" +#include "duckdb/parallel/pipeline.hpp" namespace duckdb { @@ -35,8 +36,10 @@ void PhysicalColumnDataScan::GetData(ExecutionContext &context, DataChunk &chunk //===--------------------------------------------------------------------===// // Pipeline Construction //===--------------------------------------------------------------------===// -void PhysicalColumnDataScan::BuildPipelines(Executor &executor, Pipeline ¤t, PipelineBuildState &state) { +void PhysicalColumnDataScan::BuildPipelines(Pipeline ¤t, MetaPipeline &meta_pipeline, + vector &final_pipelines) { // check if there is any additional action we need to do depending on the type + auto &state = meta_pipeline.GetState(); switch (type) { case PhysicalOperatorType::DELIM_SCAN: { auto entry = state.delim_join_dependencies.find(this); @@ -53,7 +56,7 @@ void PhysicalColumnDataScan::BuildPipelines(Executor &executor, Pipeline ¤ return; } case PhysicalOperatorType::RECURSIVE_CTE_SCAN: - if (!state.recursive_cte) { + if (!meta_pipeline.HasRecursiveCTE()) { throw InternalException("Recursive CTE scan found without recursive CTE node"); } break; diff --git a/src/execution/operator/set/physical_recursive_cte.cpp b/src/execution/operator/set/physical_recursive_cte.cpp index a42fecb8f75c..1ee30386b358 100644 --- a/src/execution/operator/set/physical_recursive_cte.cpp +++ b/src/execution/operator/set/physical_recursive_cte.cpp @@ -1,14 +1,13 @@ #include "duckdb/execution/operator/set/physical_recursive_cte.hpp" -#include "duckdb/common/vector_operations/vector_operations.hpp" - #include "duckdb/common/types/column_data_collection.hpp" +#include "duckdb/common/vector_operations/vector_operations.hpp" #include "duckdb/execution/aggregate_hashtable.hpp" -#include "duckdb/parallel/pipeline.hpp" -#include "duckdb/storage/buffer_manager.hpp" -#include "duckdb/parallel/task_scheduler.hpp" #include "duckdb/execution/executor.hpp" #include "duckdb/parallel/event.hpp" +#include "duckdb/parallel/pipeline.hpp" +#include "duckdb/parallel/task_scheduler.hpp" +#include "duckdb/storage/buffer_manager.hpp" namespace duckdb { @@ -119,27 +118,16 @@ void PhysicalRecursiveCTE::GetData(ExecutionContext &context, DataChunk &chunk, } void PhysicalRecursiveCTE::ExecuteRecursivePipelines(ExecutionContext &context) const { - if (pipelines.empty()) { + if (!recursive_meta_pipeline) { throw InternalException("Missing pipelines for recursive CTE"); } + D_ASSERT(recursive_meta_pipeline->HasRecursiveCTE()); - for (auto &pipeline : pipelines) { - auto sink = pipeline->GetSink(); - if (sink != this) { - // reset the sink state for any intermediate sinks - sink->sink_state = sink->GetGlobalSinkState(context.client); - } - for (auto &op : pipeline->GetOperators()) { - if (op) { - op->op_state = op->GetGlobalOperatorState(context.client); - } - } - pipeline->Reset(); - } - auto &executor = pipelines[0]->executor; - + auto &executor = recursive_meta_pipeline->GetExecutor(); + vector> meta_pipelines; + recursive_meta_pipeline->GetMetaPipelines(meta_pipelines, true); vector> events; - executor.ReschedulePipelines(pipelines, events); + executor.ReschedulePipelines(meta_pipelines, events); while (true) { executor.WorkOnTasks(); @@ -163,32 +151,31 @@ void PhysicalRecursiveCTE::ExecuteRecursivePipelines(ExecutionContext &context) //===--------------------------------------------------------------------===// // Pipeline Construction //===--------------------------------------------------------------------===// -void PhysicalRecursiveCTE::BuildPipelines(Executor &executor, Pipeline ¤t, PipelineBuildState &state) { +void PhysicalRecursiveCTE::BuildPipelines(Pipeline ¤t, MetaPipeline &meta_pipeline, + vector &final_pipelines) { op_state.reset(); sink_state.reset(); - pipelines.clear(); + recursive_meta_pipeline.reset(); - // recursive CTE + auto &state = meta_pipeline.GetState(); state.SetPipelineSource(current, this); - // the LHS of the recursive CTE is our initial state - // we build this pipeline as normal - auto pipeline_child = children[0].get(); - // for the RHS, we gather all pipelines that depend on the recursive cte - // these pipelines need to be rerun - if (state.recursive_cte) { + + if (meta_pipeline.HasRecursiveCTE()) { throw InternalException("Recursive CTE detected WITHIN a recursive CTE node"); } - state.recursive_cte = this; - - auto recursive_pipeline = make_shared(executor); - state.SetPipelineSink(*recursive_pipeline, this); - children[1]->BuildPipelines(executor, *recursive_pipeline, state); - pipelines.push_back(move(recursive_pipeline)); + // the LHS of the recursive CTE is our initial state, we build this pipeline as normal + auto initial_state_pipeline = meta_pipeline.CreateChildMetaPipeline(current, this); + initial_state_pipeline->Build(children[0].get()); + auto &initial_state_root_pipeline = *initial_state_pipeline->GetRootPipeline(); - state.recursive_cte = nullptr; + // the RHS is the recursive pipeline, which depends on the initial pipeline + auto recursive_pipeline = initial_state_pipeline->CreateChildMetaPipeline(initial_state_root_pipeline, this); + recursive_pipeline->SetRecursiveCTE(this); + recursive_pipeline->Build(children[1].get()); - BuildChildPipeline(executor, current, state, pipeline_child); + // set the recursive meta pipeline in this operator + recursive_meta_pipeline = recursive_pipeline->shared_from_this(); } vector PhysicalRecursiveCTE::GetSources() const { diff --git a/src/execution/operator/set/physical_union.cpp b/src/execution/operator/set/physical_union.cpp index c3b60d7e494d..c1ba08386133 100644 --- a/src/execution/operator/set/physical_union.cpp +++ b/src/execution/operator/set/physical_union.cpp @@ -15,52 +15,17 @@ PhysicalUnion::PhysicalUnion(vector types, unique_ptr &final_pipelines) { op_state.reset(); sink_state.reset(); - auto union_pipeline = make_shared(executor); - auto pipeline_ptr = union_pipeline.get(); - - // child pipeline dependencies are in reverse order of scheduling: for 'current', dependencies are set up correctly - // however, the union pipeline may also have child pipelines - // all child pipelines of 'current' that are scheduled before going further down the pipeline - // depend on the child pipelines of the union pipeline being finished too - auto &child_pipelines = state.GetChildPipelines(executor); - vector> child_pipelines_before; - auto it = child_pipelines.find(¤t); - if (it != child_pipelines.end()) { - child_pipelines_before = it->second; - } - - auto &union_pipelines = state.GetUnionPipelines(executor); - // for the current pipeline, continue building on the LHS - state.SetPipelineOperators(*union_pipeline, state.GetPipelineOperators(current)); - children[0]->BuildPipelines(executor, current, state); - // insert the union pipeline as a union pipeline of the current node - union_pipelines[¤t].push_back(move(union_pipeline)); - - // for the union pipeline, build on the RHS - state.SetPipelineSink(*pipeline_ptr, state.GetPipelineSink(current)); - children[1]->BuildPipelines(executor, *pipeline_ptr, state); - - if (child_pipelines_before.empty()) { - // no child pipelines, no need to set up dependencies - return; - } - - // as stated above, child_pipelines_before must depend on the child pipelines of the union pipeline - // also as stated above, child pipelines are dependent of each other in reverse order - // therefore it suffices to make child_pipelines_before[0] dependent on the 1st child pipeline of the union pipeline - it = child_pipelines.find(pipeline_ptr); - if (it != child_pipelines.end()) { - auto &union_child_pipelines = it->second; - // union_child_pipelines[0]->AddDependency(child_pipelines_before[0]); -// child_pipelines_before[0]->AddDependency(union_child_pipelines[0]); - } + // create a union pipeline that is identical + auto union_pipeline = meta_pipeline.CreateUnionPipeline(current); + // continue building the current pipeline + children[0]->BuildPipelines(current, meta_pipeline, final_pipelines); + // continue building the union pipeline + children[1]->BuildPipelines(*union_pipeline, meta_pipeline, final_pipelines); } vector PhysicalUnion::GetSources() const { diff --git a/src/execution/physical_operator.cpp b/src/execution/physical_operator.cpp index e7c493ebe4f8..d183dc7bbc5c 100644 --- a/src/execution/physical_operator.cpp +++ b/src/execution/physical_operator.cpp @@ -2,12 +2,13 @@ #include "duckdb/common/printer.hpp" #include "duckdb/common/string_util.hpp" +#include "duckdb/common/tree_renderer.hpp" #include "duckdb/execution/execution_context.hpp" +#include "duckdb/execution/operator/set/physical_recursive_cte.hpp" #include "duckdb/main/client_context.hpp" -#include "duckdb/parallel/thread_context.hpp" -#include "duckdb/common/tree_renderer.hpp" +#include "duckdb/parallel/meta_pipeline.hpp" #include "duckdb/parallel/pipeline.hpp" -#include "duckdb/execution/operator/set/physical_recursive_cte.hpp" +#include "duckdb/parallel/thread_context.hpp" #include "duckdb/storage/buffer_manager.hpp" namespace duckdb { @@ -118,41 +119,22 @@ idx_t PhysicalOperator::GetMaxThreadMemory(ClientContext &context) { //===--------------------------------------------------------------------===// // Pipeline Construction //===--------------------------------------------------------------------===// -void PhysicalOperator::AddPipeline(Executor &executor, shared_ptr pipeline, PipelineBuildState &state) { - if (!state.recursive_cte) { - // regular pipeline: schedule it - state.AddPipeline(executor, move(pipeline)); - } else { - // CTE pipeline! add it to the CTE pipelines - auto &cte = (PhysicalRecursiveCTE &)*state.recursive_cte; - cte.pipelines.push_back(move(pipeline)); - } -} - -void PhysicalOperator::BuildChildPipeline(Executor &executor, Pipeline ¤t, PipelineBuildState &state, - PhysicalOperator *pipeline_child) { - auto pipeline = make_shared(executor); - state.SetPipelineSink(*pipeline, this); - // the current is dependent on this pipeline to complete - current.AddDependency(pipeline); - // recurse into the pipeline child - pipeline_child->BuildPipelines(executor, *pipeline, state); - AddPipeline(executor, move(pipeline), state); -} - -void PhysicalOperator::BuildPipelines(Executor &executor, Pipeline ¤t, PipelineBuildState &state) { +void PhysicalOperator::BuildPipelines(Pipeline ¤t, MetaPipeline &meta_pipeline, + vector &final_pipelines) { op_state.reset(); + + auto &state = meta_pipeline.GetState(); if (IsSink()) { // operator is a sink, build a pipeline sink_state.reset(); + D_ASSERT(children.size() == 1); - // single operator: - // the operator becomes the data source of the current pipeline + // single operator: the operator becomes the data source of the current pipeline state.SetPipelineSource(current, this); - // we create a new pipeline starting from the child - D_ASSERT(children.size() == 1); - BuildChildPipeline(executor, current, state, children[0].get()); + // we create a new pipeline starting from the child + auto child_meta_pipeline = meta_pipeline.CreateChildMetaPipeline(current, this); + child_meta_pipeline->Build(children[0].get()); } else { // operator is not a sink! recurse in children if (children.empty()) { @@ -163,7 +145,7 @@ void PhysicalOperator::BuildPipelines(Executor &executor, Pipeline ¤t, Pip throw InternalException("Operator not supported in BuildPipelines"); } state.AddPipelineOperator(current, this); - children[0]->BuildPipelines(executor, current, state); + children[0]->BuildPipelines(current, meta_pipeline, final_pipelines); } } } diff --git a/src/include/duckdb/execution/executor.hpp b/src/include/duckdb/execution/executor.hpp index 1c3e62b43569..ab5f18cf2baf 100644 --- a/src/include/duckdb/execution/executor.hpp +++ b/src/include/duckdb/execution/executor.hpp @@ -81,7 +81,7 @@ class Executor { } void AddEvent(shared_ptr event); - void ReschedulePipelines(const vector> &pipelines, vector> &events); + void ReschedulePipelines(const vector> &pipelines, vector> &events); //! Whether or not the root of the pipeline is a result collector object bool HasResultCollector(); @@ -98,14 +98,11 @@ class Executor { static void VerifyScheduledEventsInternal(const idx_t i, const vector &vertices, vector &visited, vector &recursion_stack); - static void SchedulePipeline(const shared_ptr &pipeline, ScheduleEventData &event_data, - vector &scheduled_pipelines); - static void ScheduleChildPipeline(Pipeline *parent, const shared_ptr &pipeline, - ScheduleEventData &event_data); - void ExtractPipelines(shared_ptr &pipeline, vector> &result); + static void SchedulePipeline(const shared_ptr &pipeline, ScheduleEventData &event_data); + bool NextExecutor(); - void AddChildPipeline(Pipeline *current); + shared_ptr CreateChildPipeline(Pipeline *current); void VerifyPipeline(Pipeline &pipeline); void VerifyPipelines(); @@ -116,10 +113,10 @@ class Executor { mutex executor_lock; mutex error_lock; - //! The pipelines of the current query + //! All pipelines of the query plan vector> pipelines; - //! The root pipeline of the query - vector> root_pipelines; + //! The meta pipelines of the query + vector> root_pipelines; //! The pipeline executor for the root pipeline unique_ptr root_executor; //! The current root pipeline index @@ -140,15 +137,6 @@ class Executor { //! Whether or not execution is cancelled bool cancelled; - //! The adjacent union pipelines of each pipeline - //! Union pipelines have the same sink, but can be run concurrently along with this pipeline - unordered_map>> union_pipelines; - //! Child pipelines of this pipeline - //! Like union pipelines, child pipelines share the same sink - //! Unlike union pipelines, child pipelines should be run AFTER their dependencies are completed - //! i.e. they should be run after the dependencies are completed, but before finalize is called on the sink - unordered_map>> child_pipelines; - //! The last pending execution result (if any) PendingExecutionResult execution_result; //! The current task in process (if any) diff --git a/src/include/duckdb/execution/operator/helper/physical_execute.hpp b/src/include/duckdb/execution/operator/helper/physical_execute.hpp index c673e7d2735a..4820e61b8db2 100644 --- a/src/include/duckdb/execution/operator/helper/physical_execute.hpp +++ b/src/include/duckdb/execution/operator/helper/physical_execute.hpp @@ -25,7 +25,7 @@ class PhysicalExecute : public PhysicalOperator { vector GetChildren() const override; public: - void BuildPipelines(Executor &executor, Pipeline ¤t, PipelineBuildState &state) override; + void BuildPipelines(Pipeline ¤t, MetaPipeline &meta_pipeline, vector &final_pipelines) override; }; } // namespace duckdb diff --git a/src/include/duckdb/execution/operator/helper/physical_result_collector.hpp b/src/include/duckdb/execution/operator/helper/physical_result_collector.hpp index fe1c630bb052..cbd9b45b610c 100644 --- a/src/include/duckdb/execution/operator/helper/physical_result_collector.hpp +++ b/src/include/duckdb/execution/operator/helper/physical_result_collector.hpp @@ -38,7 +38,7 @@ class PhysicalResultCollector : public PhysicalOperator { public: vector GetChildren() const override; - void BuildPipelines(Executor &executor, Pipeline ¤t, PipelineBuildState &state) override; + void BuildPipelines(Pipeline ¤t, MetaPipeline &meta_pipeline, vector &final_pipelines) override; }; } // namespace duckdb diff --git a/src/include/duckdb/execution/operator/join/physical_cross_product.hpp b/src/include/duckdb/execution/operator/join/physical_cross_product.hpp index 850cd7ff7854..850b2572382c 100644 --- a/src/include/duckdb/execution/operator/join/physical_cross_product.hpp +++ b/src/include/duckdb/execution/operator/join/physical_cross_product.hpp @@ -47,7 +47,7 @@ class PhysicalCrossProduct : public PhysicalOperator { } public: - void BuildPipelines(Executor &executor, Pipeline ¤t, PipelineBuildState &state) override; + void BuildPipelines(Pipeline ¤t, MetaPipeline &meta_pipeline, vector &final_pipelines) override; vector GetSources() const override; }; diff --git a/src/include/duckdb/execution/operator/join/physical_delim_join.hpp b/src/include/duckdb/execution/operator/join/physical_delim_join.hpp index c921fa3685f4..e18bb194d4f5 100644 --- a/src/include/duckdb/execution/operator/join/physical_delim_join.hpp +++ b/src/include/duckdb/execution/operator/join/physical_delim_join.hpp @@ -47,7 +47,7 @@ class PhysicalDelimJoin : public PhysicalOperator { string ParamsToString() const override; public: - void BuildPipelines(Executor &executor, Pipeline ¤t, PipelineBuildState &state) override; + void BuildPipelines(Pipeline ¤t, MetaPipeline &meta_pipeline, vector &final_pipelines) override; }; } // namespace duckdb diff --git a/src/include/duckdb/execution/operator/join/physical_hash_join.hpp b/src/include/duckdb/execution/operator/join/physical_hash_join.hpp index cb1b87581db3..f3fe75e33da7 100644 --- a/src/include/duckdb/execution/operator/join/physical_hash_join.hpp +++ b/src/include/duckdb/execution/operator/join/physical_hash_join.hpp @@ -18,11 +18,6 @@ namespace duckdb { -class HashJoinGlobalSinkState; - -class HashJoinLocalSourceState; -class HashJoinGlobalSourceState; - //! PhysicalHashJoin represents a hash loop join between two tables class PhysicalHashJoin : public PhysicalComparisonJoin { public: @@ -46,7 +41,7 @@ class PhysicalHashJoin : public PhysicalComparisonJoin { vector delim_types; //! Used in perfect hash join PerfectHashJoinStats perfect_join_statistics; - //! Whether we can go external (can't yet if recursive CTE or full outer TODO) + //! Whether we can go external (can't yet if recursive CTE) bool can_go_external; public: diff --git a/src/include/duckdb/execution/operator/join/physical_iejoin.hpp b/src/include/duckdb/execution/operator/join/physical_iejoin.hpp index a811c8850dd9..fce03d4a7884 100644 --- a/src/include/duckdb/execution/operator/join/physical_iejoin.hpp +++ b/src/include/duckdb/execution/operator/join/physical_iejoin.hpp @@ -62,7 +62,7 @@ class PhysicalIEJoin : public PhysicalRangeJoin { } public: - void BuildPipelines(Executor &executor, Pipeline ¤t, PipelineBuildState &state) override; + void BuildPipelines(Pipeline ¤t, MetaPipeline &meta_pipeline, vector &final_pipelines) override; private: // resolve joins that can potentially output N*M elements (INNER, LEFT, FULL) diff --git a/src/include/duckdb/execution/operator/join/physical_index_join.hpp b/src/include/duckdb/execution/operator/join/physical_index_join.hpp index 8a2fd926d0df..1ef0d9da96be 100644 --- a/src/include/duckdb/execution/operator/join/physical_index_join.hpp +++ b/src/include/duckdb/execution/operator/join/physical_index_join.hpp @@ -63,7 +63,7 @@ class PhysicalIndexJoin : public PhysicalOperator { } public: - void BuildPipelines(Executor &executor, Pipeline ¤t, PipelineBuildState &state) override; + void BuildPipelines(Pipeline ¤t, MetaPipeline &meta_pipeline, vector &final_pipelines) override; vector GetSources() const override; private: diff --git a/src/include/duckdb/execution/operator/join/physical_join.hpp b/src/include/duckdb/execution/operator/join/physical_join.hpp index 97cfaad9f15b..56907e405f42 100644 --- a/src/include/duckdb/execution/operator/join/physical_join.hpp +++ b/src/include/duckdb/execution/operator/join/physical_join.hpp @@ -29,9 +29,9 @@ class PhysicalJoin : public PhysicalOperator { bool has_null); public: - static void BuildJoinPipelines(Executor &executor, Pipeline ¤t, PipelineBuildState &state, - PhysicalOperator &op); - void BuildPipelines(Executor &executor, Pipeline ¤t, PipelineBuildState &state) override; + static void BuildJoinPipelines(Pipeline ¤t, MetaPipeline &confluent_pipelines, + vector &final_pipelines, PhysicalOperator &op); + void BuildPipelines(Pipeline ¤t, MetaPipeline &meta_pipeline, vector &final_pipelines) override; vector GetSources() const override; }; diff --git a/src/include/duckdb/execution/operator/persistent/physical_export.hpp b/src/include/duckdb/execution/operator/persistent/physical_export.hpp index 14c156a030e7..6c0fe285457f 100644 --- a/src/include/duckdb/execution/operator/persistent/physical_export.hpp +++ b/src/include/duckdb/execution/operator/persistent/physical_export.hpp @@ -51,7 +51,7 @@ class PhysicalExport : public PhysicalOperator { } public: - void BuildPipelines(Executor &executor, Pipeline ¤t, PipelineBuildState &state) override; + void BuildPipelines(Pipeline ¤t, MetaPipeline &meta_pipeline, vector &final_pipelines) override; vector GetSources() const override; }; diff --git a/src/include/duckdb/execution/operator/scan/physical_column_data_scan.hpp b/src/include/duckdb/execution/operator/scan/physical_column_data_scan.hpp index 4a0f852184c9..3dc2b1a23f4a 100644 --- a/src/include/duckdb/execution/operator/scan/physical_column_data_scan.hpp +++ b/src/include/duckdb/execution/operator/scan/physical_column_data_scan.hpp @@ -31,7 +31,7 @@ class PhysicalColumnDataScan : public PhysicalOperator { LocalSourceState &lstate) const override; public: - void BuildPipelines(Executor &executor, Pipeline ¤t, PipelineBuildState &state) override; + void BuildPipelines(Pipeline ¤t, MetaPipeline &meta_pipeline, vector &final_pipelines) override; }; } // namespace duckdb diff --git a/src/include/duckdb/execution/operator/set/physical_recursive_cte.hpp b/src/include/duckdb/execution/operator/set/physical_recursive_cte.hpp index 7d5f966c9c4f..67b1f658a094 100644 --- a/src/include/duckdb/execution/operator/set/physical_recursive_cte.hpp +++ b/src/include/duckdb/execution/operator/set/physical_recursive_cte.hpp @@ -8,8 +8,8 @@ #pragma once -#include "duckdb/execution/physical_operator.hpp" #include "duckdb/common/types/column_data_collection.hpp" +#include "duckdb/execution/physical_operator.hpp" namespace duckdb { class Pipeline; @@ -23,7 +23,7 @@ class PhysicalRecursiveCTE : public PhysicalOperator { bool union_all; std::shared_ptr working_table; - vector> pipelines; + shared_ptr recursive_meta_pipeline; public: // Source interface @@ -42,7 +42,7 @@ class PhysicalRecursiveCTE : public PhysicalOperator { } public: - void BuildPipelines(Executor &executor, Pipeline ¤t, PipelineBuildState &state) override; + void BuildPipelines(Pipeline ¤t, MetaPipeline &meta_pipeline, vector &final_pipelines) override; vector GetSources() const override; diff --git a/src/include/duckdb/execution/operator/set/physical_union.hpp b/src/include/duckdb/execution/operator/set/physical_union.hpp index 3c0fd72615bb..7bf451f9ed96 100644 --- a/src/include/duckdb/execution/operator/set/physical_union.hpp +++ b/src/include/duckdb/execution/operator/set/physical_union.hpp @@ -17,7 +17,7 @@ class PhysicalUnion : public PhysicalOperator { idx_t estimated_cardinality); public: - void BuildPipelines(Executor &executor, Pipeline ¤t, PipelineBuildState &state) override; + void BuildPipelines(Pipeline ¤t, MetaPipeline &meta_pipeline, vector &final_pipelines) override; vector GetSources() const override; }; diff --git a/src/include/duckdb/execution/physical_operator.hpp b/src/include/duckdb/execution/physical_operator.hpp index 68de90fb85dc..d707ad79aeb2 100644 --- a/src/include/duckdb/execution/physical_operator.hpp +++ b/src/include/duckdb/execution/physical_operator.hpp @@ -9,12 +9,12 @@ #pragma once #include "duckdb/catalog/catalog.hpp" -#include "duckdb/optimizer/join_node.hpp" #include "duckdb/common/common.hpp" +#include "duckdb/common/enums/operator_result_type.hpp" #include "duckdb/common/enums/physical_operator_type.hpp" #include "duckdb/common/types/data_chunk.hpp" #include "duckdb/execution/execution_context.hpp" -#include "duckdb/common/enums/operator_result_type.hpp" +#include "duckdb/optimizer/join_node.hpp" namespace duckdb { class Event; @@ -22,6 +22,7 @@ class Executor; class PhysicalOperator; class Pipeline; class PipelineBuildState; +class MetaPipeline; // LCOV_EXCL_START class OperatorState { @@ -218,10 +219,7 @@ class PhysicalOperator { bool AllSourcesSupportBatchIndex() const; bool AllOperatorsPreserveOrder() const; - void AddPipeline(Executor &executor, shared_ptr current, PipelineBuildState &state); - virtual void BuildPipelines(Executor &executor, Pipeline ¤t, PipelineBuildState &state); - void BuildChildPipeline(Executor &executor, Pipeline ¤t, PipelineBuildState &state, - PhysicalOperator *pipeline_child); + virtual void BuildPipelines(Pipeline ¤t, MetaPipeline &meta_pipeline, vector &final_pipelines); }; } // namespace duckdb diff --git a/src/include/duckdb/parallel/base_pipeline_event.hpp b/src/include/duckdb/parallel/base_pipeline_event.hpp index 1878aaae0a1b..97f9d6e33125 100644 --- a/src/include/duckdb/parallel/base_pipeline_event.hpp +++ b/src/include/duckdb/parallel/base_pipeline_event.hpp @@ -19,6 +19,10 @@ class BasePipelineEvent : public Event { BasePipelineEvent(shared_ptr pipeline); BasePipelineEvent(Pipeline &pipeline); + void PrintPipeline() override { + pipeline->Print(); + } + //! The pipeline that this event belongs to shared_ptr pipeline; }; diff --git a/src/include/duckdb/parallel/event.hpp b/src/include/duckdb/parallel/event.hpp index e7a08601afb4..e4cf6ef5c2b7 100644 --- a/src/include/duckdb/parallel/event.hpp +++ b/src/include/duckdb/parallel/event.hpp @@ -8,8 +8,8 @@ #pragma once -#include "duckdb/common/common.hpp" #include "duckdb/common/atomic.hpp" +#include "duckdb/common/common.hpp" #include "duckdb/common/vector.hpp" namespace duckdb { @@ -49,6 +49,9 @@ class Event : public std::enable_shared_from_this { return finished; } + virtual void PrintPipeline() { + } + protected: Executor &executor; //! The current threads working on the event diff --git a/src/include/duckdb/parallel/meta_pipeline.hpp b/src/include/duckdb/parallel/meta_pipeline.hpp new file mode 100644 index 000000000000..d4824906ecdd --- /dev/null +++ b/src/include/duckdb/parallel/meta_pipeline.hpp @@ -0,0 +1,79 @@ +//===----------------------------------------------------------------------===// +// DuckDB +// +// duckdb/parallel/meta_pipeline.hpp +// +// +//===----------------------------------------------------------------------===// + +#pragma once + +#include "duckdb/execution/physical_operator.hpp" + +namespace duckdb { + +class PhysicalRecursiveCTE; + +//! MetaPipeline represents a set of pipelines that have the same sink +class MetaPipeline : public std::enable_shared_from_this { +public: + explicit MetaPipeline(Executor &executor, PipelineBuildState &state, PhysicalOperator *sink); + + Executor &GetExecutor() const; + PipelineBuildState &GetState() const; + PhysicalOperator *GetSink() const; + + bool HasRecursiveCTE() const; + PhysicalRecursiveCTE *GetRecursiveCTE() const; + void SetRecursiveCTE(PhysicalOperator *recursive_cte); + + shared_ptr &GetRootPipeline(); + void GetPipelines(vector> &result, bool recursive); + void GetMetaPipelines(vector> &result, bool recursive); + vector> &GetChildren(); + const vector *GetDependencies(Pipeline *pipeline) const; + vector &GetFinalPipelines(); + +public: + void Build(PhysicalOperator *op); + void Ready(); + void Reset(ClientContext &context, bool reset_sink); + + Pipeline *CreateUnionPipeline(Pipeline ¤t); + Pipeline *CreateChildPipeline(Pipeline ¤t); + MetaPipeline *CreateChildMetaPipeline(Pipeline ¤t, PhysicalOperator *op); + + void AddInterPipelineDependency(Pipeline *dependant, Pipeline *dependee); + +private: + Pipeline *CreatePipeline(); + //! The executor for all MetaPipelines in the query plan + Executor &executor; + //! The PipelineBuildState for all MetaPipelines in the query plan + PipelineBuildState &state; + //! The sink of all pipelines within this MetaPipeline + PhysicalOperator *sink; + //! The recursive CTE of this MetaPipeline (if any) + PhysicalRecursiveCTE *recursive_cte = nullptr; + //! All pipelines with a different source, but the same sink + vector> pipelines; + //! The pipelines that must finish before the MetaPipeline is finished + vector final_pipelines; + //! Dependencies between the confluent pipelines + unordered_map> inter_pipeline_dependencies; + //! Other MetaPipelines that this MetaPipeline depends on + vector> children; +}; + +// class ConfluentPipelineBuildState { +// public: +// explicit ConfluentPipelineBuildState(MetaPipeline &pipeline_sink); +// +// public: +// //! The confluent pipelines that the current pipeline belongs to +// MetaPipeline &pipeline_sink; +// //! Pipelines with the same sink, that the current pipeline being built may depend on +// vector> sibling_pipelines; +// }; + +} // namespace duckdb diff --git a/src/include/duckdb/parallel/pipeline.hpp b/src/include/duckdb/parallel/pipeline.hpp index e06057b13de4..195edfa06ecb 100644 --- a/src/include/duckdb/parallel/pipeline.hpp +++ b/src/include/duckdb/parallel/pipeline.hpp @@ -8,11 +8,12 @@ #pragma once +#include "duckdb/common/atomic.hpp" #include "duckdb/common/unordered_set.hpp" #include "duckdb/execution/physical_operator.hpp" #include "duckdb/function/table_function.hpp" +#include "duckdb/parallel/meta_pipeline.hpp" #include "duckdb/parallel/task_scheduler.hpp" -#include "duckdb/common/atomic.hpp" namespace duckdb { class Executor; @@ -24,38 +25,29 @@ class PipelineBuildState { constexpr static idx_t BATCH_INCREMENT = 10000000000000; public: - //! The current recursive CTE node (if any) - PhysicalOperator *recursive_cte = nullptr; - //! Duplicate eliminated join scan dependencies unordered_map delim_join_dependencies; - //! The number of pipelines that have each specific sink as their sink - unordered_map sink_pipeline_count; - public: void SetPipelineSource(Pipeline &pipeline, PhysicalOperator *op); - void SetPipelineSink(Pipeline &pipeline, PhysicalOperator *op); + void SetPipelineSink(Pipeline &pipeline, PhysicalOperator *op, idx_t sink_pipeline_count); void SetPipelineOperators(Pipeline &pipeline, vector operators); void AddPipelineOperator(Pipeline &pipeline, PhysicalOperator *op); - void AddPipeline(Executor &executor, shared_ptr pipeline); - void AddChildPipeline(Executor &executor, Pipeline &pipeline); - - unordered_map>> &GetUnionPipelines(Executor &executor); - unordered_map>> &GetChildPipelines(Executor &executor); + shared_ptr CreateChildPipeline(Executor &executor, Pipeline &pipeline); PhysicalOperator *GetPipelineSource(Pipeline &pipeline); PhysicalOperator *GetPipelineSink(Pipeline &pipeline); vector GetPipelineOperators(Pipeline &pipeline); }; -//! The Pipeline class represents an execution pipeline +//! The Pipeline class represents an execution pipeline starting at a class Pipeline : public std::enable_shared_from_this { friend class Executor; friend class PipelineExecutor; friend class PipelineEvent; friend class PipelineFinishEvent; friend class PipelineBuildState; + friend class MetaPipeline; public: explicit Pipeline(Executor &execution_context); @@ -77,6 +69,7 @@ class Pipeline : public std::enable_shared_from_this { string ToString() const; void Print() const; + void PrintDependencies() const; //! Returns query progress bool GetProgress(double ¤t_percentage, idx_t &estimated_cardinality); diff --git a/src/parallel/CMakeLists.txt b/src/parallel/CMakeLists.txt index 440e654c3e59..e18789ef5742 100644 --- a/src/parallel/CMakeLists.txt +++ b/src/parallel/CMakeLists.txt @@ -2,6 +2,7 @@ add_library_unity( duckdb_parallel OBJECT base_pipeline_event.cpp + meta_pipeline.cpp executor_task.cpp executor.cpp event.cpp diff --git a/src/parallel/executor.cpp b/src/parallel/executor.cpp index 581d06cd2804..0822acaac8e2 100644 --- a/src/parallel/executor.cpp +++ b/src/parallel/executor.cpp @@ -43,179 +43,110 @@ struct PipelineEventStack { using event_map_t = unordered_map; struct ScheduleEventData { - ScheduleEventData(const vector> &pipelines, - unordered_map>> &child_pipelines, - unordered_map>> &union_pipelines, - vector> &events, bool initial_schedule) - : pipelines(pipelines), child_pipelines(child_pipelines), union_pipelines(union_pipelines), events(events), - initial_schedule(initial_schedule) { + ScheduleEventData(const vector> &root_pipelines, vector> &events, + bool initial_schedule) + : root_pipelines(root_pipelines), events(events), initial_schedule(initial_schedule) { } - const vector> &pipelines; - unordered_map>> &child_pipelines; - unordered_map>> &union_pipelines; - unordered_map> scheduled_pipelines; + const vector> &root_pipelines; vector> &events; bool initial_schedule; event_map_t event_map; }; -void Executor::SchedulePipeline(const shared_ptr &pipeline, ScheduleEventData &event_data, - vector &scheduled_pipelines) { - D_ASSERT(pipeline); - - auto &event_map = event_data.event_map; +void Executor::SchedulePipeline(const shared_ptr &meta_pipeline, ScheduleEventData &event_data) { + D_ASSERT(meta_pipeline); auto &events = event_data.events; - auto &union_pipelines = event_data.union_pipelines; - pipeline->Ready(); - - auto pipeline_event = make_shared(pipeline); - - PipelineEventStack stack; - stack.pipeline_event = pipeline_event.get(); - if (!scheduled_pipelines.empty()) { - // this pipeline has a parent pipeline - i.e. it is scheduled as part of a `UNION` - // set up the events - auto parent = scheduled_pipelines.back(); - auto parent_stack_entry = event_map.find(parent); - D_ASSERT(parent_stack_entry != event_map.end()); - - auto &parent_stack = parent_stack_entry->second; - stack.pipeline_finish_event = parent_stack.pipeline_finish_event; - stack.pipeline_complete_event = parent_stack.pipeline_complete_event; - - stack.pipeline_event->AddDependency(*parent_stack.pipeline_event); - parent_stack.pipeline_finish_event->AddDependency(*pipeline_event); - } else { - // stand-alone pipeline - auto pipeline_finish_event = make_shared(pipeline); - auto pipeline_complete_event = - make_shared(pipeline->executor, event_data.initial_schedule); - - pipeline_finish_event->AddDependency(*pipeline_event); - pipeline_complete_event->AddDependency(*pipeline_finish_event); - - stack.pipeline_finish_event = pipeline_finish_event.get(); - stack.pipeline_complete_event = pipeline_complete_event.get(); - - events.push_back(move(pipeline_finish_event)); - events.push_back(move(pipeline_complete_event)); - } - - events.push_back(move(pipeline_event)); - event_map.insert(make_pair(pipeline.get(), stack)); + auto &event_map = event_data.event_map; - scheduled_pipelines.push_back(pipeline.get()); - auto union_entry = union_pipelines.find(pipeline.get()); - if (union_entry != union_pipelines.end()) { - for (auto &entry : union_entry->second) { - SchedulePipeline(entry, event_data, scheduled_pipelines); - } + // create events/stack for the root pipeline + auto root_pipeline = meta_pipeline->GetRootPipeline(); + auto root_event = make_shared(root_pipeline); + auto root_finish_event = make_shared(root_pipeline); + auto root_complete_event = make_shared(root_pipeline->executor, event_data.initial_schedule); + root_finish_event->AddDependency(*root_event); + root_complete_event->AddDependency(*root_finish_event); + PipelineEventStack root_stack {root_event.get(), root_finish_event.get(), root_complete_event.get()}; + + // create an event and stack for all pipelines in the MetaPipeline + vector> pipelines; + meta_pipeline->GetPipelines(pipelines, false); + for (idx_t i = 1; i < pipelines.size(); i++) { + auto &pipeline = pipelines[i]; + D_ASSERT(pipeline); + + // create events/stack for this pipeline + auto pipeline_event = make_shared(pipeline); + PipelineEventStack pipeline_stack {pipeline_event.get(), root_stack.pipeline_finish_event, + root_stack.pipeline_complete_event}; + + // add pipeline info the the event data + events.push_back(move(pipeline_event)); + event_map.insert(make_pair(pipeline.get(), move(pipeline_stack))); } -} - -void Executor::ScheduleChildPipeline(Pipeline *parent, const shared_ptr &pipeline, - ScheduleEventData &event_data) { - auto &events = event_data.events; - pipeline->Ready(); - - auto child_ptr = pipeline.get(); - D_ASSERT(event_data.union_pipelines.find(child_ptr) == event_data.union_pipelines.end()); - // create the pipeline event and the event stack - auto pipeline_event = make_shared(pipeline); - - auto &event_map = event_data.event_map; - auto parent_entry = event_map.find(parent); - D_ASSERT(parent_entry != event_map.end()); - - PipelineEventStack stack; - stack.pipeline_event = pipeline_event.get(); - stack.pipeline_finish_event = parent_entry->second.pipeline_finish_event; - stack.pipeline_complete_event = parent_entry->second.pipeline_complete_event; - // set up the dependencies for this child pipeline - unordered_set finish_events; - - vector remaining_pipelines; - unordered_set already_scheduled; - remaining_pipelines.push_back(parent); - for (idx_t i = 0; i < remaining_pipelines.size(); i++) { - auto dep = remaining_pipelines[i]; - if (already_scheduled.find(dep) != already_scheduled.end()) { - continue; - } - already_scheduled.insert(dep); - auto dep_scheduled = event_data.scheduled_pipelines.find(dep); - if (dep_scheduled != event_data.scheduled_pipelines.end()) { - for (auto &next_dep : dep_scheduled->second) { - remaining_pipelines.push_back(next_dep); + // set up dependencies within the MetaPipeline + for (idx_t i = 1; i < pipelines.size(); i++) { + auto pipeline = pipelines[i].get(); + auto &pipeline_stack = event_map[pipeline]; + auto dependencies = meta_pipeline->GetDependencies(pipeline); + if (dependencies) { + // explicit dependencies - set them up + for (auto dep : *dependencies) { + auto &dep_stack = event_map[dep]; + pipeline_stack.pipeline_event->AddDependency(*dep_stack.pipeline_event); + dep_stack.pipeline_finish_event->AddDependency(*pipeline_stack.pipeline_event); } + } else { + // no explicit dependencies - set up defaults + pipeline_stack.pipeline_event->AddDependency(*root_stack.pipeline_event); + root_stack.pipeline_finish_event->AddDependency(*pipeline_stack.pipeline_event); } - - auto dep_entry = event_map.find(dep); - D_ASSERT(dep_entry != event_map.end()); - D_ASSERT(dep_entry->second.pipeline_event); - D_ASSERT(dep_entry->second.pipeline_finish_event); - - auto finish_event = dep_entry->second.pipeline_finish_event; - stack.pipeline_event->AddDependency(*dep_entry->second.pipeline_event); - if (finish_events.find(finish_event) == finish_events.end()) { - finish_event->AddDependency(*stack.pipeline_event); - finish_events.insert(finish_event); - } - - event_data.scheduled_pipelines[dep].push_back(child_ptr); } - events.push_back(move(pipeline_event)); - event_map.insert(make_pair(child_ptr, stack)); + // add root pipeline info to the event data + events.push_back(move(root_event)); + events.push_back(move(root_finish_event)); + events.push_back(move(root_complete_event)); + event_map.insert(make_pair(root_pipeline.get(), move(root_stack))); } void Executor::ScheduleEventsInternal(ScheduleEventData &event_data) { auto &events = event_data.events; D_ASSERT(events.empty()); // create all the required pipeline events - auto &event_map = event_data.event_map; - for (auto &pipeline : event_data.pipelines) { - vector scheduled_pipelines; - SchedulePipeline(pipeline, event_data, scheduled_pipelines); - - event_data.scheduled_pipelines[pipeline.get()] = move(scheduled_pipelines); - } - // schedule child pipelines - for (auto &entry : event_data.child_pipelines) { - // iterate in reverse order - // since child entries are added from top to bottom - // dependencies are in reverse order (bottom to top) - for (idx_t i = entry.second.size(); i > 0; i--) { - auto &child_entry = entry.second[i - 1]; - ScheduleChildPipeline(entry.first, child_entry, event_data); + for (auto &pipeline : event_data.root_pipelines) { + if (!pipeline->GetSink()) { + continue; } + SchedulePipeline(pipeline, event_data); } // set up the dependencies between pipeline events - for (auto &entry : event_map) { - auto pipeline = entry.first; - for (auto &dependency : pipeline->dependencies) { - auto dep = dependency.lock(); - D_ASSERT(dep); - auto event_map_entry = event_map.find(dep.get()); - D_ASSERT(event_map_entry != event_map.end()); - auto &dep_entry = event_map_entry->second; - D_ASSERT(dep_entry.pipeline_complete_event); - entry.second.pipeline_event->AddDependency(*dep_entry.pipeline_complete_event); + auto &event_map = event_data.event_map; + for (auto &pipeline : event_data.root_pipelines) { + if (!pipeline->GetSink()) { + continue; + } + auto pipeline_root = pipeline->GetRootPipeline().get(); + auto &root_entry = event_map[pipeline_root]; + for (auto &child_pipeline : pipeline->GetChildren()) { + auto child_root = child_pipeline->GetRootPipeline().get(); + auto &child_entry = event_map[child_root]; + root_entry.pipeline_event->AddDependency(*child_entry.pipeline_complete_event); } } + // verify that we have no cyclic dependencies + VerifyScheduledEvents(event_data); // schedule the pipelines that do not have dependencies for (auto &event : events) { if (!event->HasDependencies()) { event->Schedule(); } } - VerifyScheduledEvents(event_data); } void Executor::ScheduleEvents() { - ScheduleEventData event_data(pipelines, child_pipelines, union_pipelines, events, true); + ScheduleEventData event_data(root_pipelines, events, true); ScheduleEventsInternal(event_data); } @@ -237,20 +168,19 @@ void Executor::VerifyScheduledEvents(const ScheduleEventData &event_data) { void Executor::VerifyScheduledEventsInternal(const idx_t vertex, const vector &vertices, vector &visited, vector &recursion_stack) { + D_ASSERT(!recursion_stack[vertex]); // this vertex is in the recursion stack: circular dependency! if (visited[vertex]) { return; // early out: we already visited this vertex } - D_ASSERT(!recursion_stack[vertex]); // this vertex is in the recursion stack: circular dependency! - auto dependencies = vertices[vertex]->GetDependenciesVerification(); - if (dependencies.empty()) { + if (!vertices[vertex]->HasDependencies()) { return; // early out: no dependencies } // create a vector the indices of the adjacent events vector adjacent; const idx_t count = vertices.size(); - for (auto &dep : dependencies) { + for (auto &dep : vertices[vertex]->GetDependenciesVerification()) { idx_t i; for (i = 0; i < count; i++) { if (vertices[i] == dep) { @@ -274,41 +204,18 @@ void Executor::VerifyScheduledEventsInternal(const idx_t vertex, const vector> &pipelines, vector> &events) { - unordered_map>> child_pipelines; - unordered_map>> union_pipelines; - ScheduleEventData event_data(pipelines, child_pipelines, union_pipelines, events, false); +void Executor::ReschedulePipelines(const vector> &pipelines_p, + vector> &events_p) { + ScheduleEventData event_data(pipelines_p, events_p, false); ScheduleEventsInternal(event_data); } -void Executor::ExtractPipelines(shared_ptr &pipeline, vector> &result) { - pipeline->Ready(); - - auto pipeline_ptr = pipeline.get(); - result.push_back(move(pipeline)); - auto union_entry = union_pipelines.find(pipeline_ptr); - if (union_entry != union_pipelines.end()) { - auto &union_pipeline_list = union_entry->second; - for (auto &pipeline : union_pipeline_list) { - ExtractPipelines(pipeline, result); - } - union_pipelines.erase(pipeline_ptr); - } - auto child_entry = child_pipelines.find(pipeline_ptr); - if (child_entry != child_pipelines.end()) { - for (auto entry = child_entry->second.rbegin(); entry != child_entry->second.rend(); ++entry) { - ExtractPipelines(*entry, result); - } - child_pipelines.erase(pipeline_ptr); - } -} - bool Executor::NextExecutor() { if (root_pipeline_idx >= root_pipelines.size()) { return false; } - root_pipelines[root_pipeline_idx]->Reset(); - root_executor = make_unique(context, *root_pipelines[root_pipeline_idx]); + root_pipelines[root_pipeline_idx]->Reset(context, true); + root_executor = make_unique(context, *root_pipelines[root_pipeline_idx]->GetRootPipeline()); root_pipeline_idx++; return true; } @@ -337,9 +244,6 @@ void Executor::VerifyPipelines() { for (auto &pipeline : pipelines) { VerifyPipeline(*pipeline); } - for (auto &pipeline : root_pipelines) { - VerifyPipeline(*pipeline); - } #endif } @@ -365,16 +269,17 @@ void Executor::InitializeInternal(PhysicalOperator *plan) { profiler->Initialize(physical_plan); this->producer = scheduler.CreateProducer(); - auto root_pipeline = make_shared(*this); - root_pipeline->sink = nullptr; - PipelineBuildState state; - physical_plan->BuildPipelines(*this, *root_pipeline, state); + auto root_pipeline = make_shared(*this, state, nullptr); - this->total_pipelines = pipelines.size(); + root_pipeline->Build(physical_plan); + root_pipeline->Ready(); + root_pipeline->GetMetaPipelines(root_pipelines, true); root_pipeline_idx = 0; - ExtractPipelines(root_pipeline, root_pipelines); + + root_pipeline->GetPipelines(pipelines, true); + total_pipelines = pipelines.size(); VerifyPipelines(); @@ -390,24 +295,12 @@ void Executor::CancelTasks() { vector> weak_references; { lock_guard elock(executor_lock); - cancelled = true; weak_references.reserve(pipelines.size()); + cancelled = true; for (auto &pipeline : pipelines) { weak_references.push_back(weak_ptr(pipeline)); } - for (auto &kv : union_pipelines) { - for (auto &pipeline : kv.second) { - weak_references.push_back(weak_ptr(pipeline)); - } - } - for (auto &kv : child_pipelines) { - for (auto &pipeline : kv.second) { - weak_references.push_back(weak_ptr(pipeline)); - } - } pipelines.clear(); - union_pipelines.clear(); - child_pipelines.clear(); events.clear(); } WorkOnTasks(); @@ -489,12 +382,10 @@ void Executor::Reset() { exceptions.clear(); pipelines.clear(); events.clear(); - union_pipelines.clear(); - child_pipelines.clear(); execution_result = PendingExecutionResult::RESULT_NOT_READY; } -void Executor::AddChildPipeline(Pipeline *current) { +shared_ptr Executor::CreateChildPipeline(Pipeline *current) { D_ASSERT(!current->operators.empty()); // found another operator that is a source // schedule a child pipeline @@ -505,9 +396,7 @@ void Executor::AddChildPipeline(Pipeline *current) { D_ASSERT(child_pipeline->source->IsSource()); child_pipeline->operators.pop_back(); - vector dependencies; - dependencies.push_back(current); - child_pipelines[current].push_back(move(child_pipeline)); + return child_pipeline; } vector Executor::GetTypes() { @@ -548,6 +437,7 @@ bool Executor::GetPipelinesProgress(double ¤t_progress) { // LCOV_EXCL_STA for (auto &pipeline : pipelines) { double child_percentage; idx_t child_cardinality; + if (!pipeline->GetProgress(child_percentage, child_cardinality)) { return false; } diff --git a/src/parallel/meta_pipeline.cpp b/src/parallel/meta_pipeline.cpp new file mode 100644 index 000000000000..503eb500319c --- /dev/null +++ b/src/parallel/meta_pipeline.cpp @@ -0,0 +1,144 @@ +#include "duckdb/parallel/meta_pipeline.hpp" + +#include "duckdb/execution/operator/set/physical_recursive_cte.hpp" + +namespace duckdb { + +MetaPipeline::MetaPipeline(Executor &executor_p, PipelineBuildState &state_p, PhysicalOperator *sink_p) + : executor(executor_p), state(state_p), sink(sink_p) { + auto root_pipeline = CreatePipeline(); + state.SetPipelineSink(*root_pipeline, sink, 0); +} + +Executor &MetaPipeline::GetExecutor() const { + return executor; +} + +PipelineBuildState &MetaPipeline::GetState() const { + return state; +} + +PhysicalOperator *MetaPipeline::GetSink() const { + return sink; +} + +bool MetaPipeline::HasRecursiveCTE() const { + return recursive_cte != nullptr; +} + +PhysicalRecursiveCTE *MetaPipeline::GetRecursiveCTE() const { + return (PhysicalRecursiveCTE *)recursive_cte; +} + +void MetaPipeline::SetRecursiveCTE(PhysicalOperator *recursive_cte_p) { + D_ASSERT(recursive_cte_p->type == PhysicalOperatorType::RECURSIVE_CTE); + recursive_cte = (PhysicalRecursiveCTE *)recursive_cte_p; +} + +shared_ptr &MetaPipeline::GetRootPipeline() { + return pipelines[0]; +} + +void MetaPipeline::GetPipelines(vector> &result, bool recursive) { + if (!sink) { + return; + } + result.insert(result.end(), pipelines.begin(), pipelines.end()); + if (recursive) { + for (auto &child : children) { + child->GetPipelines(result, true); + } + } +} + +void MetaPipeline::GetMetaPipelines(vector> &result, bool recursive) { + result.push_back(shared_from_this()); + if (recursive) { + for (auto &child : children) { + child->GetMetaPipelines(result, true); + } + } +} + +vector> &MetaPipeline::GetChildren() { + return children; +} + +const vector *MetaPipeline::GetDependencies(Pipeline *pipeline) const { + auto it = inter_pipeline_dependencies.find(pipeline); + if (it == inter_pipeline_dependencies.end()) { + return nullptr; + } else { + return &it->second; + } +} + +vector &MetaPipeline::GetFinalPipelines() { + return final_pipelines; +} + +void MetaPipeline::Build(PhysicalOperator *op) { + D_ASSERT(pipelines.size() == 1); + D_ASSERT(children.empty()); + D_ASSERT(final_pipelines.empty()); + op->BuildPipelines(*pipelines.back(), *this, final_pipelines); +} + +void MetaPipeline::Ready() { + for (auto &pipeline : pipelines) { + pipeline->Ready(); + } + for (auto &child : children) { + child->Ready(); + } +} + +void MetaPipeline::Reset(ClientContext &context, bool reset_sink) { + if (sink && reset_sink) { + D_ASSERT(!HasRecursiveCTE()); + sink->sink_state = sink->GetGlobalSinkState(context); + } + for (auto &pipeline : pipelines) { + for (auto &op : pipeline->GetOperators()) { + op->op_state = op->GetGlobalOperatorState(context); + } + pipeline->Reset(); + } + for (auto &child : children) { + child->Reset(context, true); + } +} + +MetaPipeline *MetaPipeline::CreateChildMetaPipeline(Pipeline ¤t, PhysicalOperator *op) { + children.push_back(make_unique(executor, state, op)); + auto child_meta_pipeline = children.back().get(); + // child MetaPipeline must finish completely before this MetaPipeline can start + current.AddDependency(child_meta_pipeline->GetRootPipeline()); + return child_meta_pipeline; +} + +Pipeline *MetaPipeline::CreatePipeline() { + pipelines.emplace_back(make_unique(executor)); + return pipelines.back().get(); +} + +Pipeline *MetaPipeline::CreateUnionPipeline(Pipeline ¤t) { + if (HasRecursiveCTE()) { + throw NotImplementedException("UNIONS are not supported in recursive CTEs yet"); + } + auto union_pipeline = CreatePipeline(); + state.SetPipelineOperators(*union_pipeline, state.GetPipelineOperators(current)); + state.SetPipelineSink(*union_pipeline, sink, pipelines.size() - 1); + return union_pipeline; +} + +Pipeline *MetaPipeline::CreateChildPipeline(Pipeline ¤t) { + pipelines.emplace_back(state.CreateChildPipeline(executor, current)); + return pipelines.back().get(); +} + +void MetaPipeline::AddInterPipelineDependency(Pipeline *dependant, Pipeline *dependee) { + inter_pipeline_dependencies[dependant].push_back(dependee); +} + +} // namespace duckdb \ No newline at end of file diff --git a/src/parallel/pipeline.cpp b/src/parallel/pipeline.cpp index 7e7321b277dc..f3ce86479164 100644 --- a/src/parallel/pipeline.cpp +++ b/src/parallel/pipeline.cpp @@ -1,19 +1,18 @@ #include "duckdb/parallel/pipeline.hpp" +#include "duckdb/common/algorithm.hpp" #include "duckdb/common/printer.hpp" +#include "duckdb/common/tree_renderer.hpp" #include "duckdb/execution/executor.hpp" -#include "duckdb/main/client_context.hpp" -#include "duckdb/parallel/thread_context.hpp" -#include "duckdb/parallel/task_scheduler.hpp" -#include "duckdb/main/database.hpp" - #include "duckdb/execution/operator/aggregate/physical_ungrouped_aggregate.hpp" #include "duckdb/execution/operator/scan/physical_table_scan.hpp" -#include "duckdb/parallel/pipeline_executor.hpp" +#include "duckdb/execution/operator/set/physical_recursive_cte.hpp" +#include "duckdb/main/client_context.hpp" +#include "duckdb/main/database.hpp" #include "duckdb/parallel/pipeline_event.hpp" - -#include "duckdb/common/algorithm.hpp" -#include "duckdb/common/tree_renderer.hpp" +#include "duckdb/parallel/pipeline_executor.hpp" +#include "duckdb/parallel/task_scheduler.hpp" +#include "duckdb/parallel/thread_context.hpp" namespace duckdb { @@ -205,6 +204,12 @@ void Pipeline::Print() const { Printer::Print(ToString()); } +void Pipeline::PrintDependencies() const { + for (auto &dep : dependencies) { + shared_ptr(dep)->Print(); + } +} + vector Pipeline::GetOperators() const { vector result; D_ASSERT(source); @@ -223,22 +228,16 @@ void PipelineBuildState::SetPipelineSource(Pipeline &pipeline, PhysicalOperator pipeline.source = op; } -void PipelineBuildState::SetPipelineSink(Pipeline &pipeline, PhysicalOperator *op) { +void PipelineBuildState::SetPipelineSink(Pipeline &pipeline, PhysicalOperator *op, idx_t sink_pipeline_count) { pipeline.sink = op; // set the base batch index of this pipeline based on how many other pipelines have this node as their sink - pipeline.base_batch_index = BATCH_INCREMENT * sink_pipeline_count[op]; - // increment the number of nodes that have this pipeline as their sink - sink_pipeline_count[op]++; + pipeline.base_batch_index = BATCH_INCREMENT * sink_pipeline_count; } void PipelineBuildState::AddPipelineOperator(Pipeline &pipeline, PhysicalOperator *op) { pipeline.operators.push_back(op); } -void PipelineBuildState::AddPipeline(Executor &executor, shared_ptr pipeline) { - executor.pipelines.push_back(move(pipeline)); -} - PhysicalOperator *PipelineBuildState::GetPipelineSource(Pipeline &pipeline) { return pipeline.source; } @@ -251,16 +250,10 @@ void PipelineBuildState::SetPipelineOperators(Pipeline &pipeline, vector PipelineBuildState::CreateChildPipeline(Executor &executor, Pipeline &pipeline) { + return executor.CreateChildPipeline(&pipeline); } -unordered_map>> &PipelineBuildState::GetUnionPipelines(Executor &executor) { - return executor.union_pipelines; -} -unordered_map>> &PipelineBuildState::GetChildPipelines(Executor &executor) { - return executor.child_pipelines; -} vector PipelineBuildState::GetPipelineOperators(Pipeline &pipeline) { return pipeline.operators; } From 99dfdd80e0afb1fb2405eda919a9bf5f3cb2b98e Mon Sep 17 00:00:00 2001 From: Laurens Kuiper Date: Wed, 26 Oct 2022 16:38:29 +0200 Subject: [PATCH 33/49] add PipelineInitializeEvent and refactor so union pipelines can run in parallel --- .../operator/helper/physical_execute.cpp | 3 +- .../helper/physical_result_collector.cpp | 3 +- .../operator/join/physical_cross_product.cpp | 5 +- .../operator/join/physical_delim_join.cpp | 7 +- .../operator/join/physical_iejoin.cpp | 11 +- .../operator/join/physical_index_join.cpp | 5 +- src/execution/operator/join/physical_join.cpp | 35 ++--- .../operator/persistent/physical_export.cpp | 5 +- .../scan/physical_column_data_scan.cpp | 3 +- .../operator/set/physical_recursive_cte.cpp | 9 +- src/execution/operator/set/physical_union.cpp | 12 +- src/execution/physical_operator.cpp | 5 +- src/include/duckdb/execution/executor.hpp | 8 +- .../operator/helper/physical_execute.hpp | 2 +- .../helper/physical_result_collector.hpp | 2 +- .../operator/join/physical_cross_product.hpp | 2 +- .../operator/join/physical_delim_join.hpp | 2 +- .../operator/join/physical_iejoin.hpp | 2 +- .../operator/join/physical_index_join.hpp | 2 +- .../execution/operator/join/physical_join.hpp | 5 +- .../operator/persistent/physical_export.hpp | 2 +- .../scan/physical_column_data_scan.hpp | 2 +- .../operator/set/physical_recursive_cte.hpp | 2 +- .../execution/operator/set/physical_union.hpp | 2 +- .../duckdb/execution/physical_operator.hpp | 2 +- src/include/duckdb/parallel/meta_pipeline.hpp | 61 +++++--- src/include/duckdb/parallel/pipeline.hpp | 2 +- .../parallel/pipeline_initialize_event.hpp | 26 ++++ src/parallel/CMakeLists.txt | 1 + src/parallel/executor.cpp | 140 ++++++++++-------- src/parallel/meta_pipeline.cpp | 98 +++++++----- src/parallel/pipeline.cpp | 8 +- src/parallel/pipeline_initialize_event.cpp | 16 ++ 33 files changed, 278 insertions(+), 212 deletions(-) create mode 100644 src/include/duckdb/parallel/pipeline_initialize_event.hpp create mode 100644 src/parallel/pipeline_initialize_event.cpp diff --git a/src/execution/operator/helper/physical_execute.cpp b/src/execution/operator/helper/physical_execute.cpp index 594fe86e8a92..64790b16aa18 100644 --- a/src/execution/operator/helper/physical_execute.cpp +++ b/src/execution/operator/helper/physical_execute.cpp @@ -10,8 +10,7 @@ vector PhysicalExecute::GetChildren() const { return {plan}; } -void PhysicalExecute::BuildPipelines(Pipeline ¤t, MetaPipeline &meta_pipeline, - vector &final_pipelines) { +void PhysicalExecute::BuildPipelines(Pipeline ¤t, MetaPipeline &meta_pipeline) { // EXECUTE statement: build pipeline on child meta_pipeline.Build(plan); } diff --git a/src/execution/operator/helper/physical_result_collector.cpp b/src/execution/operator/helper/physical_result_collector.cpp index ad0d10653abc..86eba57caaec 100644 --- a/src/execution/operator/helper/physical_result_collector.cpp +++ b/src/execution/operator/helper/physical_result_collector.cpp @@ -37,8 +37,7 @@ vector PhysicalResultCollector::GetChildren() const { return {plan}; } -void PhysicalResultCollector::BuildPipelines(Pipeline ¤t, MetaPipeline &meta_pipeline, - vector &final_pipelines) { +void PhysicalResultCollector::BuildPipelines(Pipeline ¤t, MetaPipeline &meta_pipeline) { // operator is a sink, build a pipeline sink_state.reset(); diff --git a/src/execution/operator/join/physical_cross_product.cpp b/src/execution/operator/join/physical_cross_product.cpp index 4235228e841c..49c4eee773da 100644 --- a/src/execution/operator/join/physical_cross_product.cpp +++ b/src/execution/operator/join/physical_cross_product.cpp @@ -136,9 +136,8 @@ OperatorResultType PhysicalCrossProduct::Execute(ExecutionContext &context, Data //===--------------------------------------------------------------------===// // Pipeline Construction //===--------------------------------------------------------------------===// -void PhysicalCrossProduct::BuildPipelines(Pipeline ¤t, MetaPipeline &meta_pipeline, - vector &final_pipelines) { - PhysicalJoin::BuildJoinPipelines(current, meta_pipeline, final_pipelines, *this); +void PhysicalCrossProduct::BuildPipelines(Pipeline ¤t, MetaPipeline &meta_pipeline) { + PhysicalJoin::BuildJoinPipelines(current, meta_pipeline, *this); } vector PhysicalCrossProduct::GetSources() const { diff --git a/src/execution/operator/join/physical_delim_join.cpp b/src/execution/operator/join/physical_delim_join.cpp index bb52bd558b6a..5aff5f87e699 100644 --- a/src/execution/operator/join/physical_delim_join.cpp +++ b/src/execution/operator/join/physical_delim_join.cpp @@ -119,8 +119,7 @@ string PhysicalDelimJoin::ParamsToString() const { //===--------------------------------------------------------------------===// // Pipeline Construction //===--------------------------------------------------------------------===// -void PhysicalDelimJoin::BuildPipelines(Pipeline ¤t, MetaPipeline &meta_pipeline, - vector &final_pipelines) { +void PhysicalDelimJoin::BuildPipelines(Pipeline ¤t, MetaPipeline &meta_pipeline) { op_state.reset(); sink_state.reset(); @@ -134,9 +133,9 @@ void PhysicalDelimJoin::BuildPipelines(Pipeline ¤t, MetaPipeline &meta_pip // we add an entry to the mapping of (PhysicalOperator*) -> (Pipeline*) auto &state = meta_pipeline.GetState(); for (auto &delim_scan : delim_scans) { - state.delim_join_dependencies[delim_scan] = child_meta_pipeline->GetRootPipeline().get(); + state.delim_join_dependencies[delim_scan] = child_meta_pipeline->GetBasePipeline().get(); } - join->BuildPipelines(current, meta_pipeline, final_pipelines); + join->BuildPipelines(current, meta_pipeline); } } diff --git a/src/execution/operator/join/physical_iejoin.cpp b/src/execution/operator/join/physical_iejoin.cpp index 4156fa60b434..1a1cd112f327 100644 --- a/src/execution/operator/join/physical_iejoin.cpp +++ b/src/execution/operator/join/physical_iejoin.cpp @@ -1009,25 +1009,24 @@ void PhysicalIEJoin::GetData(ExecutionContext &context, DataChunk &result, Globa //===--------------------------------------------------------------------===// // Pipeline Construction //===--------------------------------------------------------------------===// -void PhysicalIEJoin::BuildPipelines(Pipeline ¤t, MetaPipeline &meta_pipeline, - vector &final_pipelines) { +void PhysicalIEJoin::BuildPipelines(Pipeline ¤t, MetaPipeline &meta_pipeline) { D_ASSERT(children.size() == 2); auto &state = meta_pipeline.GetState(); if (meta_pipeline.HasRecursiveCTE()) { throw NotImplementedException("IEJoins are not supported in recursive CTEs yet"); } + // this operator becomes a source for the current pipeline after RHS => LHS have been built + state.SetPipelineSource(current, this); + // current depends on lhs auto lhs_meta_pipeline = meta_pipeline.CreateChildMetaPipeline(current, this); lhs_meta_pipeline->Build(children[0].get()); - auto &lhs_root_pipeline = *lhs_meta_pipeline->GetRootPipeline(); + auto &lhs_root_pipeline = *lhs_meta_pipeline->GetBasePipeline(); // lhs depends on rhs auto rhs_pipeline = lhs_meta_pipeline->CreateChildMetaPipeline(lhs_root_pipeline, this); rhs_pipeline->Build(children[1].get()); - - // this operator becomes a source for the current pipeline after RHS => LHS have been built - state.SetPipelineSource(current, this); } } // namespace duckdb diff --git a/src/execution/operator/join/physical_index_join.cpp b/src/execution/operator/join/physical_index_join.cpp index 8c35fdce90a6..546f2ac83635 100644 --- a/src/execution/operator/join/physical_index_join.cpp +++ b/src/execution/operator/join/physical_index_join.cpp @@ -211,13 +211,12 @@ OperatorResultType PhysicalIndexJoin::Execute(ExecutionContext &context, DataChu //===--------------------------------------------------------------------===// // Pipeline Construction //===--------------------------------------------------------------------===// -void PhysicalIndexJoin::BuildPipelines(Pipeline ¤t, MetaPipeline &meta_pipeline, - vector &final_pipelines) { +void PhysicalIndexJoin::BuildPipelines(Pipeline ¤t, MetaPipeline &meta_pipeline) { // index join: we only continue into the LHS // the right side is probed by the index join // so we don't need to do anything in the pipeline with this child meta_pipeline.GetState().AddPipelineOperator(current, this); - children[0]->BuildPipelines(current, meta_pipeline, final_pipelines); + children[0]->BuildPipelines(current, meta_pipeline); } vector PhysicalIndexJoin::GetSources() const { diff --git a/src/execution/operator/join/physical_join.cpp b/src/execution/operator/join/physical_join.cpp index d18517360a68..d651318c98ee 100644 --- a/src/execution/operator/join/physical_join.cpp +++ b/src/execution/operator/join/physical_join.cpp @@ -25,8 +25,7 @@ bool PhysicalJoin::EmptyResultIfRHSIsEmpty() const { //===--------------------------------------------------------------------===// // Pipeline Construction //===--------------------------------------------------------------------===// -void PhysicalJoin::BuildJoinPipelines(Pipeline ¤t, MetaPipeline &meta_pipeline, - vector &final_pipelines, PhysicalOperator &op) { +void PhysicalJoin::BuildJoinPipelines(Pipeline ¤t, MetaPipeline &meta_pipeline, PhysicalOperator &op) { op.op_state.reset(); op.sink_state.reset(); @@ -34,8 +33,14 @@ void PhysicalJoin::BuildJoinPipelines(Pipeline ¤t, MetaPipeline &meta_pipe auto &state = meta_pipeline.GetState(); state.AddPipelineOperator(current, &op); + // on the RHS (build side), we construct a child MetaPipeline with this operator as its sink + auto child_meta_pipeline = meta_pipeline.CreateChildMetaPipeline(current, &op); + child_meta_pipeline->Build(op.children[1].get()); + + // continue building the current pipeline on the LHS (probe side) + op.children[0]->BuildPipelines(current, meta_pipeline); + // Join can become a source operator if it's RIGHT/OUTER, or if the hash join goes out-of-core - // this pipeline has to happen AFTER all the probing has happened bool add_child_pipeline = false; if (op.type != PhysicalOperatorType::CROSS_PRODUCT) { auto &join_op = (PhysicalJoin &)op; @@ -56,30 +61,12 @@ void PhysicalJoin::BuildJoinPipelines(Pipeline ¤t, MetaPipeline &meta_pipe } if (add_child_pipeline) { - // create child pipeline - auto child_pipeline = meta_pipeline.CreateChildPipeline(current); - // create a new vector to set up dependencies - vector child_pipeline_dependencies; - // continue building the LHS pipeline (probe child) - op.children[0]->BuildPipelines(current, meta_pipeline, child_pipeline_dependencies); - // child pipeline depends on the downstream child pipelines to have finished (if any) - for (auto dependee : child_pipeline_dependencies) { - meta_pipeline.AddInterPipelineDependency(child_pipeline, dependee); - } - // the child pipeline needs to finish before the MetaPipeline is finished - final_pipelines.push_back(child_pipeline); - } else { - // continue building the LHS pipeline (probe child) - op.children[0]->BuildPipelines(current, meta_pipeline, final_pipelines); + meta_pipeline.CreateChildPipeline(current, &op); } - - // on the RHS (build side), we construct a new child pipeline with this pipeline as its source - auto child_meta_pipeline = meta_pipeline.CreateChildMetaPipeline(current, &op); - child_meta_pipeline->Build(op.children[1].get()); } -void PhysicalJoin::BuildPipelines(Pipeline ¤t, MetaPipeline &meta_pipeline, vector &final_pipelines) { - PhysicalJoin::BuildJoinPipelines(current, meta_pipeline, final_pipelines, *this); +void PhysicalJoin::BuildPipelines(Pipeline ¤t, MetaPipeline &meta_pipeline) { + PhysicalJoin::BuildJoinPipelines(current, meta_pipeline, *this); } vector PhysicalJoin::GetSources() const { diff --git a/src/execution/operator/persistent/physical_export.cpp b/src/execution/operator/persistent/physical_export.cpp index 44ca823653ef..3195bbe567b3 100644 --- a/src/execution/operator/persistent/physical_export.cpp +++ b/src/execution/operator/persistent/physical_export.cpp @@ -191,8 +191,7 @@ SinkResultType PhysicalExport::Sink(ExecutionContext &context, GlobalSinkState & //===--------------------------------------------------------------------===// // Pipeline Construction //===--------------------------------------------------------------------===// -void PhysicalExport::BuildPipelines(Pipeline ¤t, MetaPipeline &meta_pipeline, - vector &final_pipelines) { +void PhysicalExport::BuildPipelines(Pipeline ¤t, MetaPipeline &meta_pipeline) { // EXPORT has an optional child // we only need to schedule child pipelines if there is a child auto &state = meta_pipeline.GetState(); @@ -200,7 +199,7 @@ void PhysicalExport::BuildPipelines(Pipeline ¤t, MetaPipeline &meta_pipeli if (children.empty()) { return; } - PhysicalOperator::BuildPipelines(current, meta_pipeline, final_pipelines); + PhysicalOperator::BuildPipelines(current, meta_pipeline); } vector PhysicalExport::GetSources() const { diff --git a/src/execution/operator/scan/physical_column_data_scan.cpp b/src/execution/operator/scan/physical_column_data_scan.cpp index 1de87d9cedd6..f8ed0eabda3c 100644 --- a/src/execution/operator/scan/physical_column_data_scan.cpp +++ b/src/execution/operator/scan/physical_column_data_scan.cpp @@ -36,8 +36,7 @@ void PhysicalColumnDataScan::GetData(ExecutionContext &context, DataChunk &chunk //===--------------------------------------------------------------------===// // Pipeline Construction //===--------------------------------------------------------------------===// -void PhysicalColumnDataScan::BuildPipelines(Pipeline ¤t, MetaPipeline &meta_pipeline, - vector &final_pipelines) { +void PhysicalColumnDataScan::BuildPipelines(Pipeline ¤t, MetaPipeline &meta_pipeline) { // check if there is any additional action we need to do depending on the type auto &state = meta_pipeline.GetState(); switch (type) { diff --git a/src/execution/operator/set/physical_recursive_cte.cpp b/src/execution/operator/set/physical_recursive_cte.cpp index 1ee30386b358..e780f84c23c1 100644 --- a/src/execution/operator/set/physical_recursive_cte.cpp +++ b/src/execution/operator/set/physical_recursive_cte.cpp @@ -125,7 +125,7 @@ void PhysicalRecursiveCTE::ExecuteRecursivePipelines(ExecutionContext &context) auto &executor = recursive_meta_pipeline->GetExecutor(); vector> meta_pipelines; - recursive_meta_pipeline->GetMetaPipelines(meta_pipelines, true); + recursive_meta_pipeline->GetMetaPipelines(meta_pipelines, true, false); vector> events; executor.ReschedulePipelines(meta_pipelines, events); @@ -151,8 +151,7 @@ void PhysicalRecursiveCTE::ExecuteRecursivePipelines(ExecutionContext &context) //===--------------------------------------------------------------------===// // Pipeline Construction //===--------------------------------------------------------------------===// -void PhysicalRecursiveCTE::BuildPipelines(Pipeline ¤t, MetaPipeline &meta_pipeline, - vector &final_pipelines) { +void PhysicalRecursiveCTE::BuildPipelines(Pipeline ¤t, MetaPipeline &meta_pipeline) { op_state.reset(); sink_state.reset(); recursive_meta_pipeline.reset(); @@ -167,9 +166,9 @@ void PhysicalRecursiveCTE::BuildPipelines(Pipeline ¤t, MetaPipeline &meta_ // the LHS of the recursive CTE is our initial state, we build this pipeline as normal auto initial_state_pipeline = meta_pipeline.CreateChildMetaPipeline(current, this); initial_state_pipeline->Build(children[0].get()); - auto &initial_state_root_pipeline = *initial_state_pipeline->GetRootPipeline(); + auto &initial_state_root_pipeline = *initial_state_pipeline->GetBasePipeline(); - // the RHS is the recursive pipeline, which depends on the initial pipeline + // the RHS is the recursive pipeline, which depends on the initial state auto recursive_pipeline = initial_state_pipeline->CreateChildMetaPipeline(initial_state_root_pipeline, this); recursive_pipeline->SetRecursiveCTE(this); recursive_pipeline->Build(children[1].get()); diff --git a/src/execution/operator/set/physical_union.cpp b/src/execution/operator/set/physical_union.cpp index c1ba08386133..70e307b1820a 100644 --- a/src/execution/operator/set/physical_union.cpp +++ b/src/execution/operator/set/physical_union.cpp @@ -15,17 +15,15 @@ PhysicalUnion::PhysicalUnion(vector types, unique_ptr &final_pipelines) { +void PhysicalUnion::BuildPipelines(Pipeline ¤t, MetaPipeline &meta_pipeline) { op_state.reset(); sink_state.reset(); - // create a union pipeline that is identical + // create a union pipeline that is identical, inheriting any dependencies encountered so far auto union_pipeline = meta_pipeline.CreateUnionPipeline(current); - // continue building the current pipeline - children[0]->BuildPipelines(current, meta_pipeline, final_pipelines); - // continue building the union pipeline - children[1]->BuildPipelines(*union_pipeline, meta_pipeline, final_pipelines); + // the remainder of the top and bottom pipelines of the union are independent of each other + children[0]->BuildPipelines(current, meta_pipeline); + children[1]->BuildPipelines(*union_pipeline, meta_pipeline); } vector PhysicalUnion::GetSources() const { diff --git a/src/execution/physical_operator.cpp b/src/execution/physical_operator.cpp index d183dc7bbc5c..40589380c82e 100644 --- a/src/execution/physical_operator.cpp +++ b/src/execution/physical_operator.cpp @@ -119,8 +119,7 @@ idx_t PhysicalOperator::GetMaxThreadMemory(ClientContext &context) { //===--------------------------------------------------------------------===// // Pipeline Construction //===--------------------------------------------------------------------===// -void PhysicalOperator::BuildPipelines(Pipeline ¤t, MetaPipeline &meta_pipeline, - vector &final_pipelines) { +void PhysicalOperator::BuildPipelines(Pipeline ¤t, MetaPipeline &meta_pipeline) { op_state.reset(); auto &state = meta_pipeline.GetState(); @@ -145,7 +144,7 @@ void PhysicalOperator::BuildPipelines(Pipeline ¤t, MetaPipeline &meta_pipe throw InternalException("Operator not supported in BuildPipelines"); } state.AddPipelineOperator(current, this); - children[0]->BuildPipelines(current, meta_pipeline, final_pipelines); + children[0]->BuildPipelines(current, meta_pipeline); } } } diff --git a/src/include/duckdb/execution/executor.hpp b/src/include/duckdb/execution/executor.hpp index ab5f18cf2baf..c573bb637a99 100644 --- a/src/include/duckdb/execution/executor.hpp +++ b/src/include/duckdb/execution/executor.hpp @@ -91,7 +91,7 @@ class Executor { private: void InitializeInternal(PhysicalOperator *physical_plan); - void ScheduleEvents(); + void ScheduleEvents(const vector> &meta_pipelines); static void ScheduleEventsInternal(ScheduleEventData &event_data); static void VerifyScheduledEvents(const ScheduleEventData &event_data); @@ -102,7 +102,7 @@ class Executor { bool NextExecutor(); - shared_ptr CreateChildPipeline(Pipeline *current); + shared_ptr CreateChildPipeline(Pipeline *current, PhysicalOperator *op); void VerifyPipeline(Pipeline &pipeline); void VerifyPipelines(); @@ -115,8 +115,8 @@ class Executor { mutex error_lock; //! All pipelines of the query plan vector> pipelines; - //! The meta pipelines of the query - vector> root_pipelines; + //! The root pipelines of the query + vector> root_pipelines; //! The pipeline executor for the root pipeline unique_ptr root_executor; //! The current root pipeline index diff --git a/src/include/duckdb/execution/operator/helper/physical_execute.hpp b/src/include/duckdb/execution/operator/helper/physical_execute.hpp index 4820e61b8db2..e74d7ab43d2a 100644 --- a/src/include/duckdb/execution/operator/helper/physical_execute.hpp +++ b/src/include/duckdb/execution/operator/helper/physical_execute.hpp @@ -25,7 +25,7 @@ class PhysicalExecute : public PhysicalOperator { vector GetChildren() const override; public: - void BuildPipelines(Pipeline ¤t, MetaPipeline &meta_pipeline, vector &final_pipelines) override; + void BuildPipelines(Pipeline ¤t, MetaPipeline &meta_pipeline) override; }; } // namespace duckdb diff --git a/src/include/duckdb/execution/operator/helper/physical_result_collector.hpp b/src/include/duckdb/execution/operator/helper/physical_result_collector.hpp index cbd9b45b610c..2a2f4bd98e88 100644 --- a/src/include/duckdb/execution/operator/helper/physical_result_collector.hpp +++ b/src/include/duckdb/execution/operator/helper/physical_result_collector.hpp @@ -38,7 +38,7 @@ class PhysicalResultCollector : public PhysicalOperator { public: vector GetChildren() const override; - void BuildPipelines(Pipeline ¤t, MetaPipeline &meta_pipeline, vector &final_pipelines) override; + void BuildPipelines(Pipeline ¤t, MetaPipeline &meta_pipeline) override; }; } // namespace duckdb diff --git a/src/include/duckdb/execution/operator/join/physical_cross_product.hpp b/src/include/duckdb/execution/operator/join/physical_cross_product.hpp index 850b2572382c..36afc42ecd45 100644 --- a/src/include/duckdb/execution/operator/join/physical_cross_product.hpp +++ b/src/include/duckdb/execution/operator/join/physical_cross_product.hpp @@ -47,7 +47,7 @@ class PhysicalCrossProduct : public PhysicalOperator { } public: - void BuildPipelines(Pipeline ¤t, MetaPipeline &meta_pipeline, vector &final_pipelines) override; + void BuildPipelines(Pipeline ¤t, MetaPipeline &meta_pipeline) override; vector GetSources() const override; }; diff --git a/src/include/duckdb/execution/operator/join/physical_delim_join.hpp b/src/include/duckdb/execution/operator/join/physical_delim_join.hpp index e18bb194d4f5..0007eb907d34 100644 --- a/src/include/duckdb/execution/operator/join/physical_delim_join.hpp +++ b/src/include/duckdb/execution/operator/join/physical_delim_join.hpp @@ -47,7 +47,7 @@ class PhysicalDelimJoin : public PhysicalOperator { string ParamsToString() const override; public: - void BuildPipelines(Pipeline ¤t, MetaPipeline &meta_pipeline, vector &final_pipelines) override; + void BuildPipelines(Pipeline ¤t, MetaPipeline &meta_pipeline) override; }; } // namespace duckdb diff --git a/src/include/duckdb/execution/operator/join/physical_iejoin.hpp b/src/include/duckdb/execution/operator/join/physical_iejoin.hpp index fce03d4a7884..34fa8941558c 100644 --- a/src/include/duckdb/execution/operator/join/physical_iejoin.hpp +++ b/src/include/duckdb/execution/operator/join/physical_iejoin.hpp @@ -62,7 +62,7 @@ class PhysicalIEJoin : public PhysicalRangeJoin { } public: - void BuildPipelines(Pipeline ¤t, MetaPipeline &meta_pipeline, vector &final_pipelines) override; + void BuildPipelines(Pipeline ¤t, MetaPipeline &meta_pipeline) override; private: // resolve joins that can potentially output N*M elements (INNER, LEFT, FULL) diff --git a/src/include/duckdb/execution/operator/join/physical_index_join.hpp b/src/include/duckdb/execution/operator/join/physical_index_join.hpp index 1ef0d9da96be..1d5e89bfb1d0 100644 --- a/src/include/duckdb/execution/operator/join/physical_index_join.hpp +++ b/src/include/duckdb/execution/operator/join/physical_index_join.hpp @@ -63,7 +63,7 @@ class PhysicalIndexJoin : public PhysicalOperator { } public: - void BuildPipelines(Pipeline ¤t, MetaPipeline &meta_pipeline, vector &final_pipelines) override; + void BuildPipelines(Pipeline ¤t, MetaPipeline &meta_pipeline) override; vector GetSources() const override; private: diff --git a/src/include/duckdb/execution/operator/join/physical_join.hpp b/src/include/duckdb/execution/operator/join/physical_join.hpp index 56907e405f42..d1d276e6efc1 100644 --- a/src/include/duckdb/execution/operator/join/physical_join.hpp +++ b/src/include/duckdb/execution/operator/join/physical_join.hpp @@ -29,9 +29,8 @@ class PhysicalJoin : public PhysicalOperator { bool has_null); public: - static void BuildJoinPipelines(Pipeline ¤t, MetaPipeline &confluent_pipelines, - vector &final_pipelines, PhysicalOperator &op); - void BuildPipelines(Pipeline ¤t, MetaPipeline &meta_pipeline, vector &final_pipelines) override; + static void BuildJoinPipelines(Pipeline ¤t, MetaPipeline &confluent_pipelines, PhysicalOperator &op); + void BuildPipelines(Pipeline ¤t, MetaPipeline &meta_pipeline) override; vector GetSources() const override; }; diff --git a/src/include/duckdb/execution/operator/persistent/physical_export.hpp b/src/include/duckdb/execution/operator/persistent/physical_export.hpp index 6c0fe285457f..399679234c50 100644 --- a/src/include/duckdb/execution/operator/persistent/physical_export.hpp +++ b/src/include/duckdb/execution/operator/persistent/physical_export.hpp @@ -51,7 +51,7 @@ class PhysicalExport : public PhysicalOperator { } public: - void BuildPipelines(Pipeline ¤t, MetaPipeline &meta_pipeline, vector &final_pipelines) override; + void BuildPipelines(Pipeline ¤t, MetaPipeline &meta_pipeline) override; vector GetSources() const override; }; diff --git a/src/include/duckdb/execution/operator/scan/physical_column_data_scan.hpp b/src/include/duckdb/execution/operator/scan/physical_column_data_scan.hpp index 3dc2b1a23f4a..3d0c5c21112b 100644 --- a/src/include/duckdb/execution/operator/scan/physical_column_data_scan.hpp +++ b/src/include/duckdb/execution/operator/scan/physical_column_data_scan.hpp @@ -31,7 +31,7 @@ class PhysicalColumnDataScan : public PhysicalOperator { LocalSourceState &lstate) const override; public: - void BuildPipelines(Pipeline ¤t, MetaPipeline &meta_pipeline, vector &final_pipelines) override; + void BuildPipelines(Pipeline ¤t, MetaPipeline &meta_pipeline) override; }; } // namespace duckdb diff --git a/src/include/duckdb/execution/operator/set/physical_recursive_cte.hpp b/src/include/duckdb/execution/operator/set/physical_recursive_cte.hpp index 67b1f658a094..7fefb321c687 100644 --- a/src/include/duckdb/execution/operator/set/physical_recursive_cte.hpp +++ b/src/include/duckdb/execution/operator/set/physical_recursive_cte.hpp @@ -42,7 +42,7 @@ class PhysicalRecursiveCTE : public PhysicalOperator { } public: - void BuildPipelines(Pipeline ¤t, MetaPipeline &meta_pipeline, vector &final_pipelines) override; + void BuildPipelines(Pipeline ¤t, MetaPipeline &meta_pipeline) override; vector GetSources() const override; diff --git a/src/include/duckdb/execution/operator/set/physical_union.hpp b/src/include/duckdb/execution/operator/set/physical_union.hpp index 7bf451f9ed96..f5738dce460d 100644 --- a/src/include/duckdb/execution/operator/set/physical_union.hpp +++ b/src/include/duckdb/execution/operator/set/physical_union.hpp @@ -17,7 +17,7 @@ class PhysicalUnion : public PhysicalOperator { idx_t estimated_cardinality); public: - void BuildPipelines(Pipeline ¤t, MetaPipeline &meta_pipeline, vector &final_pipelines) override; + void BuildPipelines(Pipeline ¤t, MetaPipeline &meta_pipeline) override; vector GetSources() const override; }; diff --git a/src/include/duckdb/execution/physical_operator.hpp b/src/include/duckdb/execution/physical_operator.hpp index d707ad79aeb2..2286214951ef 100644 --- a/src/include/duckdb/execution/physical_operator.hpp +++ b/src/include/duckdb/execution/physical_operator.hpp @@ -219,7 +219,7 @@ class PhysicalOperator { bool AllSourcesSupportBatchIndex() const; bool AllOperatorsPreserveOrder() const; - virtual void BuildPipelines(Pipeline ¤t, MetaPipeline &meta_pipeline, vector &final_pipelines); + virtual void BuildPipelines(Pipeline ¤t, MetaPipeline &meta_pipeline); }; } // namespace duckdb diff --git a/src/include/duckdb/parallel/meta_pipeline.hpp b/src/include/duckdb/parallel/meta_pipeline.hpp index d4824906ecdd..e4fe9c617cd0 100644 --- a/src/include/duckdb/parallel/meta_pipeline.hpp +++ b/src/include/duckdb/parallel/meta_pipeline.hpp @@ -14,39 +14,63 @@ namespace duckdb { class PhysicalRecursiveCTE; -//! MetaPipeline represents a set of pipelines that have the same sink +//! MetaPipeline represents a set of pipelines that all have the same sink class MetaPipeline : public std::enable_shared_from_this { + //! We follow these rules when building: + //! 1. When you encounter a join, build out the blocking side first + //! - The current streaming pipeline will have an intra-MetaPipeline dependency on it + //! - Unions of this streaming pipeline will automatically inherit this dependency + //! 2. Build child pipelines last (e.g., Hash Join becomes source after probe is done: scan HT for FULL OUTER JOIN) + //! - 'last' means after building out all other pipelines associated with this operator + //! - The child pipeline automatically has inter-MetaPipeline dependencies on: + //! * The 'current' streaming pipeline + //! * And all pipelines that were added to the MetaPipeline after 'current' public: + //! Create a MetaPipeline with the given sink explicit MetaPipeline(Executor &executor, PipelineBuildState &state, PhysicalOperator *sink); +public: + //! Get the Executor for this MetaPipeline Executor &GetExecutor() const; + //! Get the PipelineBuildState for this MetaPipeline PipelineBuildState &GetState() const; + //! Get the sink operator for this MetaPipeline PhysicalOperator *GetSink() const; + //! Get the initial pipeline of this MetaPipeline + shared_ptr &GetBasePipeline(); + //! Get the pipelines of this MetaPipeline + void GetPipelines(vector> &result, bool recursive, bool skip); + //! Get the MetaPipeline children of this MetaPipeline + void GetMetaPipelines(vector> &result, bool recursive, bool skip); + //! Get the inter-MetaPipeline dependencies of the given Pipeline + const vector *GetDependencies(Pipeline *dependant) const; + + //! Recursive CTE stuff bool HasRecursiveCTE() const; PhysicalRecursiveCTE *GetRecursiveCTE() const; void SetRecursiveCTE(PhysicalOperator *recursive_cte); - shared_ptr &GetRootPipeline(); - void GetPipelines(vector> &result, bool recursive); - void GetMetaPipelines(vector> &result, bool recursive); - vector> &GetChildren(); - const vector *GetDependencies(Pipeline *pipeline) const; - vector &GetFinalPipelines(); - public: + //! Build the MetaPipeline with 'op' as the first operator (excl. the shared sink) void Build(PhysicalOperator *op); + //! Ready all the pipelines (recursively) void Ready(); + //! All pipelines (recursively) void Reset(ClientContext &context, bool reset_sink); + //! Create a union pipeline (clone of 'current') Pipeline *CreateUnionPipeline(Pipeline ¤t); - Pipeline *CreateChildPipeline(Pipeline ¤t); + //! Create a child pipeline starting at 'op' + void CreateChildPipeline(Pipeline ¤t, PhysicalOperator *op); + //! Create a MetaPipeline child that 'current' depends on MetaPipeline *CreateChildMetaPipeline(Pipeline ¤t, PhysicalOperator *op); - void AddInterPipelineDependency(Pipeline *dependant, Pipeline *dependee); - private: + //! Create an empty pipeline within this MetaPipeline Pipeline *CreatePipeline(); + +private: //! The executor for all MetaPipelines in the query plan Executor &executor; //! The PipelineBuildState for all MetaPipelines in the query plan @@ -59,21 +83,10 @@ class MetaPipeline : public std::enable_shared_from_this { vector> pipelines; //! The pipelines that must finish before the MetaPipeline is finished vector final_pipelines; - //! Dependencies between the confluent pipelines - unordered_map> inter_pipeline_dependencies; + //! Dependencies within this MetaPipeline + unordered_map> dependencies; //! Other MetaPipelines that this MetaPipeline depends on vector> children; }; -// class ConfluentPipelineBuildState { -// public: -// explicit ConfluentPipelineBuildState(MetaPipeline &pipeline_sink); -// -// public: -// //! The confluent pipelines that the current pipeline belongs to -// MetaPipeline &pipeline_sink; -// //! Pipelines with the same sink, that the current pipeline being built may depend on -// vector> sibling_pipelines; -// }; - } // namespace duckdb diff --git a/src/include/duckdb/parallel/pipeline.hpp b/src/include/duckdb/parallel/pipeline.hpp index 195edfa06ecb..068e26b153c3 100644 --- a/src/include/duckdb/parallel/pipeline.hpp +++ b/src/include/duckdb/parallel/pipeline.hpp @@ -33,7 +33,7 @@ class PipelineBuildState { void SetPipelineSink(Pipeline &pipeline, PhysicalOperator *op, idx_t sink_pipeline_count); void SetPipelineOperators(Pipeline &pipeline, vector operators); void AddPipelineOperator(Pipeline &pipeline, PhysicalOperator *op); - shared_ptr CreateChildPipeline(Executor &executor, Pipeline &pipeline); + shared_ptr CreateChildPipeline(Executor &executor, Pipeline &pipeline, PhysicalOperator *op); PhysicalOperator *GetPipelineSource(Pipeline &pipeline); PhysicalOperator *GetPipelineSink(Pipeline &pipeline); diff --git a/src/include/duckdb/parallel/pipeline_initialize_event.hpp b/src/include/duckdb/parallel/pipeline_initialize_event.hpp new file mode 100644 index 000000000000..d3c79119ae96 --- /dev/null +++ b/src/include/duckdb/parallel/pipeline_initialize_event.hpp @@ -0,0 +1,26 @@ +//===----------------------------------------------------------------------===// +// DuckDB +// +// duckdb/parallel/pipeline_finish_event.hpp +// +// +//===----------------------------------------------------------------------===// + +#pragma once + +#include "duckdb/parallel/base_pipeline_event.hpp" + +namespace duckdb { + +class Executor; + +class PipelineInitializeEvent : public BasePipelineEvent { +public: + PipelineInitializeEvent(shared_ptr pipeline); + +public: + void Schedule() override; + void FinishEvent() override; +}; + +} // namespace duckdb diff --git a/src/parallel/CMakeLists.txt b/src/parallel/CMakeLists.txt index e18789ef5742..85b538550e3a 100644 --- a/src/parallel/CMakeLists.txt +++ b/src/parallel/CMakeLists.txt @@ -11,6 +11,7 @@ add_library_unity( pipeline_event.cpp pipeline_executor.cpp pipeline_finish_event.cpp + pipeline_initialize_event.cpp task_scheduler.cpp thread_context.cpp) set(ALL_OBJECT_FILES diff --git a/src/parallel/executor.cpp b/src/parallel/executor.cpp index 0822acaac8e2..5c44fa9e8ca3 100644 --- a/src/parallel/executor.cpp +++ b/src/parallel/executor.cpp @@ -9,6 +9,7 @@ #include "duckdb/parallel/pipeline_event.hpp" #include "duckdb/parallel/pipeline_executor.hpp" #include "duckdb/parallel/pipeline_finish_event.hpp" +#include "duckdb/parallel/pipeline_initialize_event.hpp" #include "duckdb/parallel/task_scheduler.hpp" #include "duckdb/parallel/thread_context.hpp" @@ -35,6 +36,7 @@ void Executor::AddEvent(shared_ptr event) { } struct PipelineEventStack { + Event *pipeline_initialize_event; Event *pipeline_event; Event *pipeline_finish_event; Event *pipeline_complete_event; @@ -43,12 +45,12 @@ struct PipelineEventStack { using event_map_t = unordered_map; struct ScheduleEventData { - ScheduleEventData(const vector> &root_pipelines, vector> &events, + ScheduleEventData(const vector> &meta_pipelines, vector> &events, bool initial_schedule) - : root_pipelines(root_pipelines), events(events), initial_schedule(initial_schedule) { + : meta_pipelines(meta_pipelines), events(events), initial_schedule(initial_schedule) { } - const vector> &root_pipelines; + const vector> &meta_pipelines; vector> &events; bool initial_schedule; event_map_t event_map; @@ -59,84 +61,92 @@ void Executor::SchedulePipeline(const shared_ptr &meta_pipeline, S auto &events = event_data.events; auto &event_map = event_data.event_map; - // create events/stack for the root pipeline - auto root_pipeline = meta_pipeline->GetRootPipeline(); - auto root_event = make_shared(root_pipeline); - auto root_finish_event = make_shared(root_pipeline); - auto root_complete_event = make_shared(root_pipeline->executor, event_data.initial_schedule); - root_finish_event->AddDependency(*root_event); - root_complete_event->AddDependency(*root_finish_event); - PipelineEventStack root_stack {root_event.get(), root_finish_event.get(), root_complete_event.get()}; + // create events/stack for the base pipeline + auto base_pipeline = meta_pipeline->GetBasePipeline(); + auto base_initialize_event = make_shared(base_pipeline); + auto base_event = make_shared(base_pipeline); + auto base_finish_event = make_shared(base_pipeline); + auto base_complete_event = make_shared(base_pipeline->executor, event_data.initial_schedule); + PipelineEventStack base_stack {base_initialize_event.get(), base_event.get(), base_finish_event.get(), + base_complete_event.get()}; + + // initialize -> event -> finish -> complete + base_event->AddDependency(*base_initialize_event); + base_finish_event->AddDependency(*base_event); + base_complete_event->AddDependency(*base_finish_event); // create an event and stack for all pipelines in the MetaPipeline vector> pipelines; - meta_pipeline->GetPipelines(pipelines, false); + meta_pipeline->GetPipelines(pipelines, false, false); for (idx_t i = 1; i < pipelines.size(); i++) { auto &pipeline = pipelines[i]; D_ASSERT(pipeline); // create events/stack for this pipeline auto pipeline_event = make_shared(pipeline); - PipelineEventStack pipeline_stack {pipeline_event.get(), root_stack.pipeline_finish_event, - root_stack.pipeline_complete_event}; + PipelineEventStack pipeline_stack {base_stack.pipeline_initialize_event, pipeline_event.get(), + base_stack.pipeline_finish_event, base_stack.pipeline_complete_event}; + + // base_initialize -> pipeline_event -> base_finish + pipeline_stack.pipeline_event->AddDependency(*base_stack.pipeline_initialize_event); + base_stack.pipeline_finish_event->AddDependency(*pipeline_stack.pipeline_event); // add pipeline info the the event data events.push_back(move(pipeline_event)); event_map.insert(make_pair(pipeline.get(), move(pipeline_stack))); } - // set up dependencies within the MetaPipeline - for (idx_t i = 1; i < pipelines.size(); i++) { - auto pipeline = pipelines[i].get(); - auto &pipeline_stack = event_map[pipeline]; - auto dependencies = meta_pipeline->GetDependencies(pipeline); - if (dependencies) { - // explicit dependencies - set them up - for (auto dep : *dependencies) { - auto &dep_stack = event_map[dep]; - pipeline_stack.pipeline_event->AddDependency(*dep_stack.pipeline_event); - dep_stack.pipeline_finish_event->AddDependency(*pipeline_stack.pipeline_event); - } - } else { - // no explicit dependencies - set up defaults - pipeline_stack.pipeline_event->AddDependency(*root_stack.pipeline_event); - root_stack.pipeline_finish_event->AddDependency(*pipeline_stack.pipeline_event); + // set up the inter-MetaPipeline dependencies + for (auto &pipeline : pipelines) { + auto dependencies = meta_pipeline->GetDependencies(pipeline.get()); + if (!dependencies) { + continue; + } + auto &pipeline_stack = event_map[pipeline.get()]; + for (auto &dependency : *dependencies) { + auto &dependency_stack = event_map[dependency]; + pipeline_stack.pipeline_event->AddDependency(*dependency_stack.pipeline_event); } } - // add root pipeline info to the event data - events.push_back(move(root_event)); - events.push_back(move(root_finish_event)); - events.push_back(move(root_complete_event)); - event_map.insert(make_pair(root_pipeline.get(), move(root_stack))); + // add base pipeline info to the event data too + events.push_back(move(base_initialize_event)); + events.push_back(move(base_event)); + events.push_back(move(base_finish_event)); + events.push_back(move(base_complete_event)); + event_map.insert(make_pair(base_pipeline.get(), move(base_stack))); } void Executor::ScheduleEventsInternal(ScheduleEventData &event_data) { auto &events = event_data.events; D_ASSERT(events.empty()); + // create all the required pipeline events - for (auto &pipeline : event_data.root_pipelines) { + for (auto &pipeline : event_data.meta_pipelines) { if (!pipeline->GetSink()) { continue; } SchedulePipeline(pipeline, event_data); } - // set up the dependencies between pipeline events + + // set up the intra-MetaPipeline dependencies auto &event_map = event_data.event_map; - for (auto &pipeline : event_data.root_pipelines) { - if (!pipeline->GetSink()) { - continue; - } - auto pipeline_root = pipeline->GetRootPipeline().get(); - auto &root_entry = event_map[pipeline_root]; - for (auto &child_pipeline : pipeline->GetChildren()) { - auto child_root = child_pipeline->GetRootPipeline().get(); - auto &child_entry = event_map[child_root]; - root_entry.pipeline_event->AddDependency(*child_entry.pipeline_complete_event); + for (auto &entry : event_map) { + auto pipeline = entry.first; + for (auto &dependency : pipeline->dependencies) { + auto dep = dependency.lock(); + D_ASSERT(dep); + auto event_map_entry = event_map.find(dep.get()); + D_ASSERT(event_map_entry != event_map.end()); + auto &dep_entry = event_map_entry->second; + D_ASSERT(dep_entry.pipeline_complete_event); + entry.second.pipeline_event->AddDependency(*dep_entry.pipeline_complete_event); } } + // verify that we have no cyclic dependencies VerifyScheduledEvents(event_data); + // schedule the pipelines that do not have dependencies for (auto &event : events) { if (!event->HasDependencies()) { @@ -145,8 +155,8 @@ void Executor::ScheduleEventsInternal(ScheduleEventData &event_data) { } } -void Executor::ScheduleEvents() { - ScheduleEventData event_data(root_pipelines, events, true); +void Executor::ScheduleEvents(const vector> &meta_pipelines) { + ScheduleEventData event_data(meta_pipelines, events, true); ScheduleEventsInternal(event_data); } @@ -214,8 +224,8 @@ bool Executor::NextExecutor() { if (root_pipeline_idx >= root_pipelines.size()) { return false; } - root_pipelines[root_pipeline_idx]->Reset(context, true); - root_executor = make_unique(context, *root_pipelines[root_pipeline_idx]->GetRootPipeline()); + root_pipelines[root_pipeline_idx]->Reset(); + root_executor = make_unique(context, *root_pipelines[root_pipeline_idx]); root_pipeline_idx++; return true; } @@ -271,19 +281,20 @@ void Executor::InitializeInternal(PhysicalOperator *plan) { PipelineBuildState state; auto root_pipeline = make_shared(*this, state, nullptr); - root_pipeline->Build(physical_plan); root_pipeline->Ready(); - root_pipeline->GetMetaPipelines(root_pipelines, true); + root_pipeline->GetPipelines(root_pipelines, false, false); root_pipeline_idx = 0; - root_pipeline->GetPipelines(pipelines, true); + root_pipeline->GetPipelines(pipelines, true, true); total_pipelines = pipelines.size(); VerifyPipelines(); - ScheduleEvents(); + vector> to_schedule; + root_pipeline->GetMetaPipelines(to_schedule, true, true); + ScheduleEvents(to_schedule); } } @@ -385,16 +396,23 @@ void Executor::Reset() { execution_result = PendingExecutionResult::RESULT_NOT_READY; } -shared_ptr Executor::CreateChildPipeline(Pipeline *current) { +shared_ptr Executor::CreateChildPipeline(Pipeline *current, PhysicalOperator *op) { D_ASSERT(!current->operators.empty()); - // found another operator that is a source - // schedule a child pipeline + D_ASSERT(op->IsSource()); + // found another operator that is a source, schedule a child pipeline + // 'op' is the source, and the sink is the same auto child_pipeline = make_shared(*this); child_pipeline->sink = current->sink; + child_pipeline->source = op; + + // the child pipeline has the same operators up until 'op' + for (auto ¤t_op : current->operators) { + if (current_op == op) { + break; + } + child_pipeline->operators.push_back(current_op); + } child_pipeline->operators = current->operators; - child_pipeline->source = current->operators.back(); - D_ASSERT(child_pipeline->source->IsSource()); - child_pipeline->operators.pop_back(); return child_pipeline; } diff --git a/src/parallel/meta_pipeline.cpp b/src/parallel/meta_pipeline.cpp index 503eb500319c..cefa2d85de4a 100644 --- a/src/parallel/meta_pipeline.cpp +++ b/src/parallel/meta_pipeline.cpp @@ -22,66 +22,59 @@ PhysicalOperator *MetaPipeline::GetSink() const { return sink; } -bool MetaPipeline::HasRecursiveCTE() const { - return recursive_cte != nullptr; -} - -PhysicalRecursiveCTE *MetaPipeline::GetRecursiveCTE() const { - return (PhysicalRecursiveCTE *)recursive_cte; -} - -void MetaPipeline::SetRecursiveCTE(PhysicalOperator *recursive_cte_p) { - D_ASSERT(recursive_cte_p->type == PhysicalOperatorType::RECURSIVE_CTE); - recursive_cte = (PhysicalRecursiveCTE *)recursive_cte_p; -} - -shared_ptr &MetaPipeline::GetRootPipeline() { +shared_ptr &MetaPipeline::GetBasePipeline() { return pipelines[0]; } -void MetaPipeline::GetPipelines(vector> &result, bool recursive) { - if (!sink) { - return; +void MetaPipeline::GetPipelines(vector> &result, bool recursive, bool skip) { + if (!skip) { + result.insert(result.end(), pipelines.begin(), pipelines.end()); } - result.insert(result.end(), pipelines.begin(), pipelines.end()); if (recursive) { for (auto &child : children) { - child->GetPipelines(result, true); + child->GetPipelines(result, true, false); } } } -void MetaPipeline::GetMetaPipelines(vector> &result, bool recursive) { - result.push_back(shared_from_this()); +void MetaPipeline::GetMetaPipelines(vector> &result, bool recursive, bool skip) { + if (!skip) { + result.push_back(shared_from_this()); + } if (recursive) { for (auto &child : children) { - child->GetMetaPipelines(result, true); + child->GetMetaPipelines(result, true, false); } } } -vector> &MetaPipeline::GetChildren() { - return children; -} - -const vector *MetaPipeline::GetDependencies(Pipeline *pipeline) const { - auto it = inter_pipeline_dependencies.find(pipeline); - if (it == inter_pipeline_dependencies.end()) { +const vector *MetaPipeline::GetDependencies(Pipeline *dependant) const { + auto it = dependencies.find(dependant); + if (it == dependencies.end()) { return nullptr; } else { return &it->second; } } -vector &MetaPipeline::GetFinalPipelines() { - return final_pipelines; +bool MetaPipeline::HasRecursiveCTE() const { + return recursive_cte != nullptr; +} + +PhysicalRecursiveCTE *MetaPipeline::GetRecursiveCTE() const { + return (PhysicalRecursiveCTE *)recursive_cte; +} + +void MetaPipeline::SetRecursiveCTE(PhysicalOperator *recursive_cte_p) { + D_ASSERT(recursive_cte_p->type == PhysicalOperatorType::RECURSIVE_CTE); + recursive_cte = (PhysicalRecursiveCTE *)recursive_cte_p; } void MetaPipeline::Build(PhysicalOperator *op) { D_ASSERT(pipelines.size() == 1); D_ASSERT(children.empty()); D_ASSERT(final_pipelines.empty()); - op->BuildPipelines(*pipelines.back(), *this, final_pipelines); + op->BuildPipelines(*pipelines.back(), *this); } void MetaPipeline::Ready() { @@ -110,10 +103,11 @@ void MetaPipeline::Reset(ClientContext &context, bool reset_sink) { } MetaPipeline *MetaPipeline::CreateChildMetaPipeline(Pipeline ¤t, PhysicalOperator *op) { + D_ASSERT(pipelines.back().get() == ¤t); // rule 1 children.push_back(make_unique(executor, state, op)); auto child_meta_pipeline = children.back().get(); // child MetaPipeline must finish completely before this MetaPipeline can start - current.AddDependency(child_meta_pipeline->GetRootPipeline()); + current.AddDependency(child_meta_pipeline->GetBasePipeline()); return child_meta_pipeline; } @@ -126,19 +120,43 @@ Pipeline *MetaPipeline::CreateUnionPipeline(Pipeline ¤t) { if (HasRecursiveCTE()) { throw NotImplementedException("UNIONS are not supported in recursive CTEs yet"); } + + // create the union pipeline auto union_pipeline = CreatePipeline(); state.SetPipelineOperators(*union_pipeline, state.GetPipelineOperators(current)); state.SetPipelineSink(*union_pipeline, sink, pipelines.size() - 1); + + // 'union_pipeline' inherits ALL dependencies of 'current' (intra- and inter-MetaPipeline) + union_pipeline->dependencies = current.dependencies; + auto current_inter_deps = GetDependencies(¤t); + if (current_inter_deps) { + dependencies[union_pipeline] = *current_inter_deps; + } + return union_pipeline; } -Pipeline *MetaPipeline::CreateChildPipeline(Pipeline ¤t) { - pipelines.emplace_back(state.CreateChildPipeline(executor, current)); - return pipelines.back().get(); -} +void MetaPipeline::CreateChildPipeline(Pipeline ¤t, PhysicalOperator *op) { + D_ASSERT(pipelines.back().get() != ¤t); // rule 2 + if (HasRecursiveCTE()) { + throw NotImplementedException("Child pipelines are not supported in recursive CTEs yet"); + } + + // child pipeline has an inter-MetaPipeline depency on all pipelines that were scheduled between 'current' and now + // (including 'current') - gather them + auto it = pipelines.begin(); + while (it->get() != ¤t) { + it++; + } + vector scheduled_between; + while (it != pipelines.end()) { + scheduled_between.push_back(it->get()); + } + D_ASSERT(!scheduled_between.empty()); -void MetaPipeline::AddInterPipelineDependency(Pipeline *dependant, Pipeline *dependee) { - inter_pipeline_dependencies[dependant].push_back(dependee); + // finally, create the child pipeline and set the dependencies + pipelines.emplace_back(state.CreateChildPipeline(executor, current, op)); + dependencies[pipelines.back().get()] = move(scheduled_between); } -} // namespace duckdb \ No newline at end of file +} // namespace duckdb diff --git a/src/parallel/pipeline.cpp b/src/parallel/pipeline.cpp index f3ce86479164..3bdf8da63de9 100644 --- a/src/parallel/pipeline.cpp +++ b/src/parallel/pipeline.cpp @@ -118,7 +118,6 @@ bool Pipeline::IsOrderDependent() const { void Pipeline::Schedule(shared_ptr &event) { D_ASSERT(ready); D_ASSERT(sink); - Reset(); if (!ScheduleParallel(event)) { // could not parallelize this pipeline: push a sequential task instead ScheduleSequentialTask(event); @@ -147,7 +146,7 @@ bool Pipeline::LaunchScanTasks(shared_ptr &event, idx_t max_threads) { } void Pipeline::Reset() { - if (sink && !sink->sink_state) { + if (!sink->sink_state) { sink->sink_state = sink->GetGlobalSinkState(GetClientContext()); } @@ -250,8 +249,9 @@ void PipelineBuildState::SetPipelineOperators(Pipeline &pipeline, vector PipelineBuildState::CreateChildPipeline(Executor &executor, Pipeline &pipeline) { - return executor.CreateChildPipeline(&pipeline); +shared_ptr PipelineBuildState::CreateChildPipeline(Executor &executor, Pipeline &pipeline, + PhysicalOperator *op) { + return executor.CreateChildPipeline(&pipeline, op); } vector PipelineBuildState::GetPipelineOperators(Pipeline &pipeline) { diff --git a/src/parallel/pipeline_initialize_event.cpp b/src/parallel/pipeline_initialize_event.cpp new file mode 100644 index 000000000000..cfcd08c488cf --- /dev/null +++ b/src/parallel/pipeline_initialize_event.cpp @@ -0,0 +1,16 @@ +#include "duckdb/parallel/pipeline_initialize_event.hpp" +#include "duckdb/execution/executor.hpp" + +namespace duckdb { + +PipelineInitializeEvent::PipelineInitializeEvent(shared_ptr pipeline_p) : BasePipelineEvent(move(pipeline_p)) { +} + +void PipelineInitializeEvent::Schedule() { + pipeline->Reset(); +} + +void PipelineInitializeEvent::FinishEvent() { +} + +} // namespace duckdb From 2676db14cfbe5307f85cfa0ccc0c82e6caca2978 Mon Sep 17 00:00:00 2001 From: Laurens Kuiper Date: Thu, 27 Oct 2022 12:35:29 +0200 Subject: [PATCH 34/49] refactor pipeline construction: got queries working again (somewhat) --- .../operator/join/physical_iejoin.cpp | 4 +-- .../operator/set/physical_recursive_cte.cpp | 5 ++-- .../duckdb/parallel/base_pipeline_event.hpp | 4 +-- src/include/duckdb/parallel/meta_pipeline.hpp | 7 ++--- src/include/duckdb/parallel/pipeline.hpp | 5 ++-- .../duckdb/parallel/pipeline_finish_event.hpp | 2 +- .../parallel/pipeline_initialize_event.hpp | 2 +- src/parallel/executor.cpp | 17 +++++------- src/parallel/meta_pipeline.cpp | 27 ++++++++++--------- src/parallel/pipeline.cpp | 8 ++++-- src/parallel/pipeline_initialize_event.cpp | 26 ++++++++++++++++-- .../full_outer/full_outer_join_union.test | 6 +++++ 12 files changed, 70 insertions(+), 43 deletions(-) diff --git a/src/execution/operator/join/physical_iejoin.cpp b/src/execution/operator/join/physical_iejoin.cpp index 1a1cd112f327..f3e2d3f3edfa 100644 --- a/src/execution/operator/join/physical_iejoin.cpp +++ b/src/execution/operator/join/physical_iejoin.cpp @@ -1022,10 +1022,10 @@ void PhysicalIEJoin::BuildPipelines(Pipeline ¤t, MetaPipeline &meta_pipeli // current depends on lhs auto lhs_meta_pipeline = meta_pipeline.CreateChildMetaPipeline(current, this); lhs_meta_pipeline->Build(children[0].get()); - auto &lhs_root_pipeline = *lhs_meta_pipeline->GetBasePipeline(); + auto &lhs_base_pipeline = *lhs_meta_pipeline->GetBasePipeline(); // lhs depends on rhs - auto rhs_pipeline = lhs_meta_pipeline->CreateChildMetaPipeline(lhs_root_pipeline, this); + auto rhs_pipeline = lhs_meta_pipeline->CreateChildMetaPipeline(lhs_base_pipeline, this); rhs_pipeline->Build(children[1].get()); } diff --git a/src/execution/operator/set/physical_recursive_cte.cpp b/src/execution/operator/set/physical_recursive_cte.cpp index e780f84c23c1..a422f5ab6a9c 100644 --- a/src/execution/operator/set/physical_recursive_cte.cpp +++ b/src/execution/operator/set/physical_recursive_cte.cpp @@ -166,11 +166,10 @@ void PhysicalRecursiveCTE::BuildPipelines(Pipeline ¤t, MetaPipeline &meta_ // the LHS of the recursive CTE is our initial state, we build this pipeline as normal auto initial_state_pipeline = meta_pipeline.CreateChildMetaPipeline(current, this); initial_state_pipeline->Build(children[0].get()); - auto &initial_state_root_pipeline = *initial_state_pipeline->GetBasePipeline(); + auto &initial_state_base_pipeline = *initial_state_pipeline->GetBasePipeline(); // the RHS is the recursive pipeline, which depends on the initial state - auto recursive_pipeline = initial_state_pipeline->CreateChildMetaPipeline(initial_state_root_pipeline, this); - recursive_pipeline->SetRecursiveCTE(this); + auto recursive_pipeline = initial_state_pipeline->CreateChildMetaPipeline(initial_state_base_pipeline, this); recursive_pipeline->Build(children[1].get()); // set the recursive meta pipeline in this operator diff --git a/src/include/duckdb/parallel/base_pipeline_event.hpp b/src/include/duckdb/parallel/base_pipeline_event.hpp index 97f9d6e33125..a4d909eb9256 100644 --- a/src/include/duckdb/parallel/base_pipeline_event.hpp +++ b/src/include/duckdb/parallel/base_pipeline_event.hpp @@ -16,8 +16,8 @@ namespace duckdb { //! A BasePipelineEvent is used as the basis of any event that belongs to a specific pipeline class BasePipelineEvent : public Event { public: - BasePipelineEvent(shared_ptr pipeline); - BasePipelineEvent(Pipeline &pipeline); + explicit BasePipelineEvent(shared_ptr pipeline); + explicit BasePipelineEvent(Pipeline &pipeline); void PrintPipeline() override { pipeline->Print(); diff --git a/src/include/duckdb/parallel/meta_pipeline.hpp b/src/include/duckdb/parallel/meta_pipeline.hpp index e4fe9c617cd0..4774021c3947 100644 --- a/src/include/duckdb/parallel/meta_pipeline.hpp +++ b/src/include/duckdb/parallel/meta_pipeline.hpp @@ -17,7 +17,7 @@ class PhysicalRecursiveCTE; //! MetaPipeline represents a set of pipelines that all have the same sink class MetaPipeline : public std::enable_shared_from_this { //! We follow these rules when building: - //! 1. When you encounter a join, build out the blocking side first + //! 1. Add join operators to the current pipeline, and build out the blocking side before going down the probe side //! - The current streaming pipeline will have an intra-MetaPipeline dependency on it //! - Unions of this streaming pipeline will automatically inherit this dependency //! 2. Build child pipelines last (e.g., Hash Join becomes source after probe is done: scan HT for FULL OUTER JOIN) @@ -45,11 +45,8 @@ class MetaPipeline : public std::enable_shared_from_this { void GetMetaPipelines(vector> &result, bool recursive, bool skip); //! Get the inter-MetaPipeline dependencies of the given Pipeline const vector *GetDependencies(Pipeline *dependant) const; - - //! Recursive CTE stuff + //! Whether this MetaPipeline has a recursive CTE bool HasRecursiveCTE() const; - PhysicalRecursiveCTE *GetRecursiveCTE() const; - void SetRecursiveCTE(PhysicalOperator *recursive_cte); public: //! Build the MetaPipeline with 'op' as the first operator (excl. the shared sink) diff --git a/src/include/duckdb/parallel/pipeline.hpp b/src/include/duckdb/parallel/pipeline.hpp index 068e26b153c3..a0ff10dbf517 100644 --- a/src/include/duckdb/parallel/pipeline.hpp +++ b/src/include/duckdb/parallel/pipeline.hpp @@ -61,6 +61,7 @@ class Pipeline : public std::enable_shared_from_this { void Ready(); void Reset(); + void ResetSink(); void ResetSource(); void Schedule(shared_ptr &event); @@ -90,11 +91,11 @@ class Pipeline : public std::enable_shared_from_this { //! Whether or not the pipeline has been initialized atomic initialized; //! The source of this pipeline - PhysicalOperator *source; + PhysicalOperator *source = nullptr; //! The chain of intermediate operators vector operators; //! The sink (i.e. destination) for data; this is e.g. a hash table to-be-built - PhysicalOperator *sink; + PhysicalOperator *sink = nullptr; //! The global source state unique_ptr source_state; diff --git a/src/include/duckdb/parallel/pipeline_finish_event.hpp b/src/include/duckdb/parallel/pipeline_finish_event.hpp index b5bdc750a605..1a486a61ff9a 100644 --- a/src/include/duckdb/parallel/pipeline_finish_event.hpp +++ b/src/include/duckdb/parallel/pipeline_finish_event.hpp @@ -15,7 +15,7 @@ class Executor; class PipelineFinishEvent : public BasePipelineEvent { public: - PipelineFinishEvent(shared_ptr pipeline); + explicit PipelineFinishEvent(shared_ptr pipeline); public: void Schedule() override; diff --git a/src/include/duckdb/parallel/pipeline_initialize_event.hpp b/src/include/duckdb/parallel/pipeline_initialize_event.hpp index d3c79119ae96..664717dc4bdf 100644 --- a/src/include/duckdb/parallel/pipeline_initialize_event.hpp +++ b/src/include/duckdb/parallel/pipeline_initialize_event.hpp @@ -16,7 +16,7 @@ class Executor; class PipelineInitializeEvent : public BasePipelineEvent { public: - PipelineInitializeEvent(shared_ptr pipeline); + explicit PipelineInitializeEvent(shared_ptr pipeline); public: void Schedule() override; diff --git a/src/parallel/executor.cpp b/src/parallel/executor.cpp index 5c44fa9e8ca3..50261ae40e05 100644 --- a/src/parallel/executor.cpp +++ b/src/parallel/executor.cpp @@ -96,6 +96,13 @@ void Executor::SchedulePipeline(const shared_ptr &meta_pipeline, S event_map.insert(make_pair(pipeline.get(), move(pipeline_stack))); } + // add base pipeline info to the event data too + events.push_back(move(base_initialize_event)); + events.push_back(move(base_event)); + events.push_back(move(base_finish_event)); + events.push_back(move(base_complete_event)); + event_map.insert(make_pair(base_pipeline.get(), move(base_stack))); + // set up the inter-MetaPipeline dependencies for (auto &pipeline : pipelines) { auto dependencies = meta_pipeline->GetDependencies(pipeline.get()); @@ -108,13 +115,6 @@ void Executor::SchedulePipeline(const shared_ptr &meta_pipeline, S pipeline_stack.pipeline_event->AddDependency(*dependency_stack.pipeline_event); } } - - // add base pipeline info to the event data too - events.push_back(move(base_initialize_event)); - events.push_back(move(base_event)); - events.push_back(move(base_finish_event)); - events.push_back(move(base_complete_event)); - event_map.insert(make_pair(base_pipeline.get(), move(base_stack))); } void Executor::ScheduleEventsInternal(ScheduleEventData &event_data) { @@ -123,9 +123,6 @@ void Executor::ScheduleEventsInternal(ScheduleEventData &event_data) { // create all the required pipeline events for (auto &pipeline : event_data.meta_pipelines) { - if (!pipeline->GetSink()) { - continue; - } SchedulePipeline(pipeline, event_data); } diff --git a/src/parallel/meta_pipeline.cpp b/src/parallel/meta_pipeline.cpp index cefa2d85de4a..75cb61463f11 100644 --- a/src/parallel/meta_pipeline.cpp +++ b/src/parallel/meta_pipeline.cpp @@ -6,8 +6,11 @@ namespace duckdb { MetaPipeline::MetaPipeline(Executor &executor_p, PipelineBuildState &state_p, PhysicalOperator *sink_p) : executor(executor_p), state(state_p), sink(sink_p) { - auto root_pipeline = CreatePipeline(); - state.SetPipelineSink(*root_pipeline, sink, 0); + auto base_pipeline = CreatePipeline(); + state.SetPipelineSink(*base_pipeline, sink, 0); + if (sink_p && sink_p->type == PhysicalOperatorType::RECURSIVE_CTE) { + recursive_cte = (PhysicalRecursiveCTE *)sink; + } } Executor &MetaPipeline::GetExecutor() const { @@ -61,15 +64,6 @@ bool MetaPipeline::HasRecursiveCTE() const { return recursive_cte != nullptr; } -PhysicalRecursiveCTE *MetaPipeline::GetRecursiveCTE() const { - return (PhysicalRecursiveCTE *)recursive_cte; -} - -void MetaPipeline::SetRecursiveCTE(PhysicalOperator *recursive_cte_p) { - D_ASSERT(recursive_cte_p->type == PhysicalOperatorType::RECURSIVE_CTE); - recursive_cte = (PhysicalRecursiveCTE *)recursive_cte_p; -} - void MetaPipeline::Build(PhysicalOperator *op) { D_ASSERT(pipelines.size() == 1); D_ASSERT(children.empty()); @@ -103,11 +97,16 @@ void MetaPipeline::Reset(ClientContext &context, bool reset_sink) { } MetaPipeline *MetaPipeline::CreateChildMetaPipeline(Pipeline ¤t, PhysicalOperator *op) { - D_ASSERT(pipelines.back().get() == ¤t); // rule 1 + // rule 1: make sure that child MetaPipeline is created immediately after 'op' is added to 'current' + D_ASSERT(op == current.source || current.operators.back() == op); children.push_back(make_unique(executor, state, op)); auto child_meta_pipeline = children.back().get(); // child MetaPipeline must finish completely before this MetaPipeline can start current.AddDependency(child_meta_pipeline->GetBasePipeline()); + // child meta pipeline is part of the recursive CTE too + if (HasRecursiveCTE()) { + child_meta_pipeline->recursive_cte = recursive_cte; + } return child_meta_pipeline; } @@ -137,7 +136,8 @@ Pipeline *MetaPipeline::CreateUnionPipeline(Pipeline ¤t) { } void MetaPipeline::CreateChildPipeline(Pipeline ¤t, PhysicalOperator *op) { - D_ASSERT(pipelines.back().get() != ¤t); // rule 2 + // rule 2: 'current' must be fully built (down to the source) before creating the child pipeline + D_ASSERT(current.source); if (HasRecursiveCTE()) { throw NotImplementedException("Child pipelines are not supported in recursive CTEs yet"); } @@ -151,6 +151,7 @@ void MetaPipeline::CreateChildPipeline(Pipeline ¤t, PhysicalOperator *op) vector scheduled_between; while (it != pipelines.end()) { scheduled_between.push_back(it->get()); + it++; } D_ASSERT(!scheduled_between.empty()); diff --git a/src/parallel/pipeline.cpp b/src/parallel/pipeline.cpp index 3bdf8da63de9..e63a0449d3c9 100644 --- a/src/parallel/pipeline.cpp +++ b/src/parallel/pipeline.cpp @@ -118,6 +118,7 @@ bool Pipeline::IsOrderDependent() const { void Pipeline::Schedule(shared_ptr &event) { D_ASSERT(ready); D_ASSERT(sink); + Reset(); if (!ScheduleParallel(event)) { // could not parallelize this pipeline: push a sequential task instead ScheduleSequentialTask(event); @@ -145,11 +146,14 @@ bool Pipeline::LaunchScanTasks(shared_ptr &event, idx_t max_threads) { return true; } -void Pipeline::Reset() { - if (!sink->sink_state) { +void Pipeline::ResetSink() { + if (sink && !sink->sink_state) { sink->sink_state = sink->GetGlobalSinkState(GetClientContext()); } +} +void Pipeline::Reset() { + ResetSink(); for (auto &op : operators) { if (op && !op->op_state) { op->op_state = op->GetGlobalOperatorState(GetClientContext()); diff --git a/src/parallel/pipeline_initialize_event.cpp b/src/parallel/pipeline_initialize_event.cpp index cfcd08c488cf..cc8d5db44021 100644 --- a/src/parallel/pipeline_initialize_event.cpp +++ b/src/parallel/pipeline_initialize_event.cpp @@ -1,13 +1,35 @@ #include "duckdb/parallel/pipeline_initialize_event.hpp" + #include "duckdb/execution/executor.hpp" namespace duckdb { -PipelineInitializeEvent::PipelineInitializeEvent(shared_ptr pipeline_p) : BasePipelineEvent(move(pipeline_p)) { +PipelineInitializeEvent::PipelineInitializeEvent(shared_ptr pipeline_p) + : BasePipelineEvent(move(pipeline_p)) { } +class PipelineInitializeTask : public ExecutorTask { +public: + explicit PipelineInitializeTask(Pipeline &pipeline_p, shared_ptr event_p) + : ExecutorTask(pipeline_p.executor), pipeline(pipeline_p), event(move(event_p)) { + } + + Pipeline &pipeline; + shared_ptr event; + +public: + TaskExecutionResult ExecuteTask(TaskExecutionMode mode) override { + pipeline.ResetSink(); + event->FinishTask(); + return TaskExecutionResult::TASK_FINISHED; + } +}; + void PipelineInitializeEvent::Schedule() { - pipeline->Reset(); + // needs to spawn a task to get the chain of tasks for the query plan going + vector> tasks; + tasks.push_back(make_unique(*pipeline, shared_from_this())); + SetTasks(move(tasks)); } void PipelineInitializeEvent::FinishEvent() { diff --git a/test/sql/join/full_outer/full_outer_join_union.test b/test/sql/join/full_outer/full_outer_join_union.test index 70d8918b962b..f8c97221ed1b 100644 --- a/test/sql/join/full_outer/full_outer_join_union.test +++ b/test/sql/join/full_outer/full_outer_join_union.test @@ -83,6 +83,12 @@ NULL NULL 2 20 mode unskip +#query T +#select 1 union all select 42 +#---- +#1 +#42 + # many unions/full outer joins with views statement ok CREATE VIEW v1 AS From 9bd9e6e04e3d9bfce936c08b3c441d26182459f9 Mon Sep 17 00:00:00 2001 From: Laurens Kuiper Date: Thu, 27 Oct 2022 14:35:19 +0200 Subject: [PATCH 35/49] combinations of union/child pipelines seem to work - still some bugs to fix though --- src/execution/operator/join/physical_join.cpp | 28 +++++----- src/include/duckdb/parallel/event.hpp | 2 +- src/parallel/event.cpp | 2 +- src/parallel/executor.cpp | 53 +++++++++++-------- .../full_outer/full_outer_join_union.test | 20 ++----- 5 files changed, 52 insertions(+), 53 deletions(-) diff --git a/src/execution/operator/join/physical_join.cpp b/src/execution/operator/join/physical_join.cpp index d651318c98ee..a807cbabfbd6 100644 --- a/src/execution/operator/join/physical_join.cpp +++ b/src/execution/operator/join/physical_join.cpp @@ -40,23 +40,25 @@ void PhysicalJoin::BuildJoinPipelines(Pipeline ¤t, MetaPipeline &meta_pipe // continue building the current pipeline on the LHS (probe side) op.children[0]->BuildPipelines(current, meta_pipeline); + if (op.type == PhysicalOperatorType::CROSS_PRODUCT) { + return; + } + // Join can become a source operator if it's RIGHT/OUTER, or if the hash join goes out-of-core bool add_child_pipeline = false; - if (op.type != PhysicalOperatorType::CROSS_PRODUCT) { - auto &join_op = (PhysicalJoin &)op; - if (IsRightOuterJoin(join_op.join_type)) { - if (meta_pipeline.HasRecursiveCTE()) { - throw NotImplementedException("FULL and RIGHT outer joins are not supported in recursive CTEs yet"); - } - add_child_pipeline = true; + auto &join_op = (PhysicalJoin &)op; + if (IsRightOuterJoin(join_op.join_type)) { + if (meta_pipeline.HasRecursiveCTE()) { + throw NotImplementedException("FULL and RIGHT outer joins are not supported in recursive CTEs yet"); } + add_child_pipeline = true; + } - if (join_op.type == PhysicalOperatorType::HASH_JOIN) { - auto &hash_join_op = (PhysicalHashJoin &)join_op; - hash_join_op.can_go_external = !meta_pipeline.HasRecursiveCTE(); - if (hash_join_op.can_go_external) { - add_child_pipeline = true; - } + if (join_op.type == PhysicalOperatorType::HASH_JOIN) { + auto &hash_join_op = (PhysicalHashJoin &)join_op; + hash_join_op.can_go_external = !meta_pipeline.HasRecursiveCTE(); + if (hash_join_op.can_go_external) { + add_child_pipeline = true; } } diff --git a/src/include/duckdb/parallel/event.hpp b/src/include/duckdb/parallel/event.hpp index e4cf6ef5c2b7..36b25f3896e8 100644 --- a/src/include/duckdb/parallel/event.hpp +++ b/src/include/duckdb/parallel/event.hpp @@ -37,7 +37,7 @@ class Event : public std::enable_shared_from_this { bool HasDependencies() const { return total_dependencies != 0; } - vector GetDependenciesVerification() const; + const vector &GetParentsVerification() const; void CompleteDependency(); diff --git a/src/parallel/event.cpp b/src/parallel/event.cpp index 254cee553170..52073d8618b1 100644 --- a/src/parallel/event.cpp +++ b/src/parallel/event.cpp @@ -48,7 +48,7 @@ void Event::AddDependency(Event &event) { #endif } -vector Event::GetDependenciesVerification() const { +const vector &Event::GetParentsVerification() const { D_ASSERT(parents.size() == parents_raw.size()); return parents_raw; } diff --git a/src/parallel/executor.cpp b/src/parallel/executor.cpp index 50261ae40e05..c2bf35fbd754 100644 --- a/src/parallel/executor.cpp +++ b/src/parallel/executor.cpp @@ -69,11 +69,15 @@ void Executor::SchedulePipeline(const shared_ptr &meta_pipeline, S auto base_complete_event = make_shared(base_pipeline->executor, event_data.initial_schedule); PipelineEventStack base_stack {base_initialize_event.get(), base_event.get(), base_finish_event.get(), base_complete_event.get()}; + events.push_back(move(base_initialize_event)); + events.push_back(move(base_event)); + events.push_back(move(base_finish_event)); + events.push_back(move(base_complete_event)); - // initialize -> event -> finish -> complete - base_event->AddDependency(*base_initialize_event); - base_finish_event->AddDependency(*base_event); - base_complete_event->AddDependency(*base_finish_event); + // dependencies: initialize -> event -> finish -> complete + base_stack.pipeline_event->AddDependency(*base_stack.pipeline_initialize_event); + base_stack.pipeline_finish_event->AddDependency(*base_stack.pipeline_event); + base_stack.pipeline_complete_event->AddDependency(*base_stack.pipeline_finish_event); // create an event and stack for all pipelines in the MetaPipeline vector> pipelines; @@ -86,21 +90,17 @@ void Executor::SchedulePipeline(const shared_ptr &meta_pipeline, S auto pipeline_event = make_shared(pipeline); PipelineEventStack pipeline_stack {base_stack.pipeline_initialize_event, pipeline_event.get(), base_stack.pipeline_finish_event, base_stack.pipeline_complete_event}; + events.push_back(move(pipeline_event)); - // base_initialize -> pipeline_event -> base_finish + // dependencies: base_initialize -> pipeline_event -> base_finish pipeline_stack.pipeline_event->AddDependency(*base_stack.pipeline_initialize_event); base_stack.pipeline_finish_event->AddDependency(*pipeline_stack.pipeline_event); - // add pipeline info the the event data - events.push_back(move(pipeline_event)); + // add pipeline stack to event map event_map.insert(make_pair(pipeline.get(), move(pipeline_stack))); } - // add base pipeline info to the event data too - events.push_back(move(base_initialize_event)); - events.push_back(move(base_event)); - events.push_back(move(base_finish_event)); - events.push_back(move(base_complete_event)); + // add base stack to the event data too event_map.insert(make_pair(base_pipeline.get(), move(base_stack))); // set up the inter-MetaPipeline dependencies @@ -180,17 +180,18 @@ void Executor::VerifyScheduledEventsInternal(const idx_t vertex, const vectorHasDependencies()) { - return; // early out: no dependencies + auto &parents = vertices[vertex]->GetParentsVerification(); + if (parents.empty()) { + return; // early out: outgoing edges } // create a vector the indices of the adjacent events vector adjacent; const idx_t count = vertices.size(); - for (auto &dep : vertices[vertex]->GetDependenciesVerification()) { + for (auto parent : parents) { idx_t i; for (i = 0; i < count; i++) { - if (vertices[i] == dep) { + if (vertices[i] == parent) { adjacent.push_back(i); break; } @@ -276,21 +277,28 @@ void Executor::InitializeInternal(PhysicalOperator *plan) { profiler->Initialize(physical_plan); this->producer = scheduler.CreateProducer(); + // build and ready the pipelines PipelineBuildState state; auto root_pipeline = make_shared(*this, state, nullptr); root_pipeline->Build(physical_plan); root_pipeline->Ready(); + // set root pipelines, i.e., all pipelines that end in the final sink root_pipeline->GetPipelines(root_pipelines, false, false); root_pipeline_idx = 0; - root_pipeline->GetPipelines(pipelines, true, true); - total_pipelines = pipelines.size(); - - VerifyPipelines(); - + // collect all meta-pipelines from the root pipeline vector> to_schedule; root_pipeline->GetMetaPipelines(to_schedule, true, true); + + // number of 'complete events' is equal to the number of meta pipelines, so we have to set it here + total_pipelines = to_schedule.size(); + + // collect all pipelines from the root pipelines (recursively) for the progress bar and verify them + root_pipeline->GetPipelines(pipelines, true, false); + + // finally, verify and schedule + VerifyPipelines(); ScheduleEvents(to_schedule); } } @@ -403,13 +411,12 @@ shared_ptr Executor::CreateChildPipeline(Pipeline *current, PhysicalOp child_pipeline->source = op; // the child pipeline has the same operators up until 'op' - for (auto ¤t_op : current->operators) { + for (auto current_op : current->operators) { if (current_op == op) { break; } child_pipeline->operators.push_back(current_op); } - child_pipeline->operators = current->operators; return child_pipeline; } diff --git a/test/sql/join/full_outer/full_outer_join_union.test b/test/sql/join/full_outer/full_outer_join_union.test index f8c97221ed1b..fa7b1c999bbe 100644 --- a/test/sql/join/full_outer/full_outer_join_union.test +++ b/test/sql/join/full_outer/full_outer_join_union.test @@ -2,11 +2,11 @@ # description: Test FULL OUTER JOIN with unions # group: [full_outer] -#statement ok -#PRAGMA enable_verification +statement ok +PRAGMA enable_verification statement ok -pragma debug_force_external=true +pragma verify_external statement ok CREATE TABLE integers(i INTEGER, j INTEGER); @@ -20,8 +20,6 @@ CREATE TABLE integers2(k INTEGER, l INTEGER); statement ok INSERT INTO integers2 VALUES (1, 10), (2, 20); -mode skip - # equality join query IIII SELECT i, j, k, l FROM integers FULL OUTER JOIN integers2 ON integers.i=integers2.k @@ -81,14 +79,6 @@ NULL NULL 2 20 1 1 1 10 3 3 NULL NULL -mode unskip - -#query T -#select 1 union all select 42 -#---- -#1 -#42 - # many unions/full outer joins with views statement ok CREATE VIEW v1 AS @@ -96,7 +86,7 @@ SELECT i, j, k, l FROM integers FULL OUTER JOIN integers2 ON integers.i=integers UNION ALL SELECT i, j, k, l FROM integers FULL OUTER JOIN integers2 ON integers.i=integers2.k -#loop i 0 10 +loop i 0 10 query IIIIII SELECT * FROM v1 FULL OUTER JOIN v1 v2 USING (i, j) ORDER BY 1, 2, 3, 4, 5, 6 @@ -114,4 +104,4 @@ NULL NULL 2 20 NULL NULL 3 3 NULL NULL NULL NULL 3 3 NULL NULL NULL NULL -#endloop +endloop From faf8f82f576393af120187c2938d920bd0790e25 Mon Sep 17 00:00:00 2001 From: Laurens Kuiper Date: Thu, 27 Oct 2022 19:47:04 +0200 Subject: [PATCH 36/49] fix recursive CTE after pipeline build refactor --- .../operator/set/physical_recursive_cte.cpp | 10 +++++-- src/include/duckdb/parallel/meta_pipeline.hpp | 6 ++--- src/parallel/executor.cpp | 11 ++++---- src/parallel/meta_pipeline.cpp | 26 ++++++++++--------- 4 files changed, 31 insertions(+), 22 deletions(-) diff --git a/src/execution/operator/set/physical_recursive_cte.cpp b/src/execution/operator/set/physical_recursive_cte.cpp index a422f5ab6a9c..ba1b32dcad43 100644 --- a/src/execution/operator/set/physical_recursive_cte.cpp +++ b/src/execution/operator/set/physical_recursive_cte.cpp @@ -119,13 +119,19 @@ void PhysicalRecursiveCTE::GetData(ExecutionContext &context, DataChunk &chunk, void PhysicalRecursiveCTE::ExecuteRecursivePipelines(ExecutionContext &context) const { if (!recursive_meta_pipeline) { - throw InternalException("Missing pipelines for recursive CTE"); + throw InternalException("Missing meta pipeline for recursive CTE"); } D_ASSERT(recursive_meta_pipeline->HasRecursiveCTE()); - auto &executor = recursive_meta_pipeline->GetExecutor(); + // reset pipelines (excluding sink state of the top-level meta-pipeline, which is this recursive CTE) + recursive_meta_pipeline->Reset(context.client, false); + + // get the MetaPipelines in the recursive_meta_pipeline vector> meta_pipelines; recursive_meta_pipeline->GetMetaPipelines(meta_pipelines, true, false); + + // reschedule them + auto &executor = recursive_meta_pipeline->GetExecutor(); vector> events; executor.ReschedulePipelines(meta_pipelines, events); diff --git a/src/include/duckdb/parallel/meta_pipeline.hpp b/src/include/duckdb/parallel/meta_pipeline.hpp index 4774021c3947..2a2e144e4c86 100644 --- a/src/include/duckdb/parallel/meta_pipeline.hpp +++ b/src/include/duckdb/parallel/meta_pipeline.hpp @@ -17,7 +17,7 @@ class PhysicalRecursiveCTE; //! MetaPipeline represents a set of pipelines that all have the same sink class MetaPipeline : public std::enable_shared_from_this { //! We follow these rules when building: - //! 1. Add join operators to the current pipeline, and build out the blocking side before going down the probe side + //! 1. For joins, build out the blocking side before going down the probe side //! - The current streaming pipeline will have an intra-MetaPipeline dependency on it //! - Unions of this streaming pipeline will automatically inherit this dependency //! 2. Build child pipelines last (e.g., Hash Join becomes source after probe is done: scan HT for FULL OUTER JOIN) @@ -40,7 +40,7 @@ class MetaPipeline : public std::enable_shared_from_this { //! Get the initial pipeline of this MetaPipeline shared_ptr &GetBasePipeline(); //! Get the pipelines of this MetaPipeline - void GetPipelines(vector> &result, bool recursive, bool skip); + void GetPipelines(vector> &result, bool recursive); //! Get the MetaPipeline children of this MetaPipeline void GetMetaPipelines(vector> &result, bool recursive, bool skip); //! Get the inter-MetaPipeline dependencies of the given Pipeline @@ -54,7 +54,7 @@ class MetaPipeline : public std::enable_shared_from_this { //! Ready all the pipelines (recursively) void Ready(); //! All pipelines (recursively) - void Reset(ClientContext &context, bool reset_sink); + void Reset(bool reset_sink); //! Create a union pipeline (clone of 'current') Pipeline *CreateUnionPipeline(Pipeline ¤t); diff --git a/src/parallel/executor.cpp b/src/parallel/executor.cpp index c2bf35fbd754..b8652fc9f447 100644 --- a/src/parallel/executor.cpp +++ b/src/parallel/executor.cpp @@ -81,8 +81,8 @@ void Executor::SchedulePipeline(const shared_ptr &meta_pipeline, S // create an event and stack for all pipelines in the MetaPipeline vector> pipelines; - meta_pipeline->GetPipelines(pipelines, false, false); - for (idx_t i = 1; i < pipelines.size(); i++) { + meta_pipeline->GetPipelines(pipelines, false); + for (idx_t i = 1; i < pipelines.size(); i++) { // loop starts at 1 because 0 is the base pipeline auto &pipeline = pipelines[i]; D_ASSERT(pipeline); @@ -284,18 +284,18 @@ void Executor::InitializeInternal(PhysicalOperator *plan) { root_pipeline->Ready(); // set root pipelines, i.e., all pipelines that end in the final sink - root_pipeline->GetPipelines(root_pipelines, false, false); + root_pipeline->GetPipelines(root_pipelines, false); root_pipeline_idx = 0; // collect all meta-pipelines from the root pipeline vector> to_schedule; root_pipeline->GetMetaPipelines(to_schedule, true, true); - // number of 'complete events' is equal to the number of meta pipelines, so we have to set it here + // number of 'PipelineCompleteEvent's is equal to the number of meta pipelines, so we have to set it here total_pipelines = to_schedule.size(); // collect all pipelines from the root pipelines (recursively) for the progress bar and verify them - root_pipeline->GetPipelines(pipelines, true, false); + root_pipeline->GetPipelines(pipelines, true); // finally, verify and schedule VerifyPipelines(); @@ -317,6 +317,7 @@ void Executor::CancelTasks() { weak_references.push_back(weak_ptr(pipeline)); } pipelines.clear(); + root_pipelines.clear(); events.clear(); } WorkOnTasks(); diff --git a/src/parallel/meta_pipeline.cpp b/src/parallel/meta_pipeline.cpp index 75cb61463f11..bfc41da00977 100644 --- a/src/parallel/meta_pipeline.cpp +++ b/src/parallel/meta_pipeline.cpp @@ -1,6 +1,8 @@ #include "duckdb/parallel/meta_pipeline.hpp" +#include "duckdb/execution/executor.hpp" #include "duckdb/execution/operator/set/physical_recursive_cte.hpp" +#include "duckdb/execution/physical_plan_generator.hpp" namespace duckdb { @@ -29,13 +31,11 @@ shared_ptr &MetaPipeline::GetBasePipeline() { return pipelines[0]; } -void MetaPipeline::GetPipelines(vector> &result, bool recursive, bool skip) { - if (!skip) { - result.insert(result.end(), pipelines.begin(), pipelines.end()); - } +void MetaPipeline::GetPipelines(vector> &result, bool recursive) { + result.insert(result.end(), pipelines.begin(), pipelines.end()); if (recursive) { for (auto &child : children) { - child->GetPipelines(result, true, false); + child->GetPipelines(result, true); } } } @@ -80,25 +80,22 @@ void MetaPipeline::Ready() { } } -void MetaPipeline::Reset(ClientContext &context, bool reset_sink) { +void MetaPipeline::Reset(bool reset_sink) { if (sink && reset_sink) { - D_ASSERT(!HasRecursiveCTE()); - sink->sink_state = sink->GetGlobalSinkState(context); + sink->sink_state = sink->GetGlobalSinkState(executor.context); } for (auto &pipeline : pipelines) { for (auto &op : pipeline->GetOperators()) { - op->op_state = op->GetGlobalOperatorState(context); + op->op_state = op->GetGlobalOperatorState(executor.context); } pipeline->Reset(); } for (auto &child : children) { - child->Reset(context, true); + child->Reset(true); } } MetaPipeline *MetaPipeline::CreateChildMetaPipeline(Pipeline ¤t, PhysicalOperator *op) { - // rule 1: make sure that child MetaPipeline is created immediately after 'op' is added to 'current' - D_ASSERT(op == current.source || current.operators.back() == op); children.push_back(make_unique(executor, state, op)); auto child_meta_pipeline = children.back().get(); // child MetaPipeline must finish completely before this MetaPipeline can start @@ -132,6 +129,11 @@ Pipeline *MetaPipeline::CreateUnionPipeline(Pipeline ¤t) { dependencies[union_pipeline] = *current_inter_deps; } + if (sink && (!sink->ParallelSink() || PhysicalPlanGenerator::)) { + // if the sink it not parallel, the union pipeline has to come after the current pipeline + dependencies[union_pipeline].push_back(¤t); + } + return union_pipeline; } From d35afb052ee3fd43ed7736e5752472349261ab6c Mon Sep 17 00:00:00 2001 From: Laurens Kuiper Date: Fri, 28 Oct 2022 18:23:36 +0200 Subject: [PATCH 37/49] pipeline build rework: just one test failing (recursive cte's grr) --- .../operator/helper/physical_execute.cpp | 5 +++ .../helper/physical_result_collector.cpp | 5 +++ .../operator/join/physical_iejoin.cpp | 19 ++++---- src/execution/operator/join/physical_join.cpp | 7 ++- src/execution/operator/set/physical_union.cpp | 15 ++++--- src/execution/physical_operator.cpp | 2 +- src/include/duckdb/execution/executor.hpp | 2 + .../operator/helper/physical_execute.hpp | 1 + .../helper/physical_result_collector.hpp | 4 +- .../operator/order/physical_order.hpp | 4 ++ .../duckdb/execution/physical_operator.hpp | 2 +- src/include/duckdb/parallel/meta_pipeline.hpp | 22 +++++++--- src/include/duckdb/parallel/pipeline.hpp | 2 + src/parallel/executor.cpp | 9 ++++ src/parallel/meta_pipeline.cpp | 43 +++++++++++++------ src/parallel/pipeline.cpp | 19 ++++++++ .../statistics/statistics_setop.test | 6 +++ .../binder/test_null_type_propagation.test | 8 ++-- .../cte/recursive_cte_complex_pipelines.test | 8 ++++ test/sql/order/test_limit_percent.test | 13 ++++++ .../intraquery/test_simple_parallelism.test | 6 +-- test/sql/types/union/union_cast.test | 6 +-- 22 files changed, 161 insertions(+), 47 deletions(-) diff --git a/src/execution/operator/helper/physical_execute.cpp b/src/execution/operator/helper/physical_execute.cpp index 07c8f2c3240a..7d0ed9f20c8d 100644 --- a/src/execution/operator/helper/physical_execute.cpp +++ b/src/execution/operator/helper/physical_execute.cpp @@ -12,6 +12,11 @@ vector PhysicalExecute::GetChildren() const { return {plan}; } +bool PhysicalExecute::AllOperatorsPreserveOrder() const { + D_ASSERT(plan); + return plan->AllOperatorsPreserveOrder(); +} + void PhysicalExecute::BuildPipelines(Pipeline ¤t, MetaPipeline &meta_pipeline) { // EXECUTE statement: build pipeline on child meta_pipeline.Build(plan); diff --git a/src/execution/operator/helper/physical_result_collector.cpp b/src/execution/operator/helper/physical_result_collector.cpp index 90e3888ff7ff..cf2dc5f59000 100644 --- a/src/execution/operator/helper/physical_result_collector.cpp +++ b/src/execution/operator/helper/physical_result_collector.cpp @@ -35,6 +35,11 @@ vector PhysicalResultCollector::GetChildren() const { return {plan}; } +bool PhysicalResultCollector::AllOperatorsPreserveOrder() const { + D_ASSERT(plan); + return plan->AllOperatorsPreserveOrder(); +} + void PhysicalResultCollector::BuildPipelines(Pipeline ¤t, MetaPipeline &meta_pipeline) { // operator is a sink, build a pipeline sink_state.reset(); diff --git a/src/execution/operator/join/physical_iejoin.cpp b/src/execution/operator/join/physical_iejoin.cpp index 4e96f845e30f..33588c6328b9 100644 --- a/src/execution/operator/join/physical_iejoin.cpp +++ b/src/execution/operator/join/physical_iejoin.cpp @@ -1012,22 +1012,23 @@ void PhysicalIEJoin::GetData(ExecutionContext &context, DataChunk &result, Globa //===--------------------------------------------------------------------===// void PhysicalIEJoin::BuildPipelines(Pipeline ¤t, MetaPipeline &meta_pipeline) { D_ASSERT(children.size() == 2); - auto &state = meta_pipeline.GetState(); if (meta_pipeline.HasRecursiveCTE()) { throw NotImplementedException("IEJoins are not supported in recursive CTEs yet"); } - // this operator becomes a source for the current pipeline after RHS => LHS have been built - state.SetPipelineSource(current, this); - - // current depends on lhs + // create LHS as a child MetaPipeline auto lhs_meta_pipeline = meta_pipeline.CreateChildMetaPipeline(current, this); lhs_meta_pipeline->Build(children[0].get()); - auto &lhs_base_pipeline = *lhs_meta_pipeline->GetBasePipeline(); - // lhs depends on rhs - auto rhs_pipeline = lhs_meta_pipeline->CreateChildMetaPipeline(lhs_base_pipeline, this); - rhs_pipeline->Build(children[1].get()); + // create RHS as a child MetaPipeline + auto rhs_meta_pipeline = meta_pipeline.CreateChildMetaPipeline(current, this); + rhs_meta_pipeline->Build(children[1].get()); + + // RHS depends on LHS + rhs_meta_pipeline->GetBasePipeline()->AddDependency(lhs_meta_pipeline->GetBasePipeline()); + + // this operator becomes a source for the current pipeline after RHS and LHS have been built + meta_pipeline.GetState().SetPipelineSource(current, this); } } // namespace duckdb diff --git a/src/execution/operator/join/physical_join.cpp b/src/execution/operator/join/physical_join.cpp index 769f9e8969fe..3433ecd53bfe 100644 --- a/src/execution/operator/join/physical_join.cpp +++ b/src/execution/operator/join/physical_join.cpp @@ -34,6 +34,11 @@ void PhysicalJoin::BuildJoinPipelines(Pipeline ¤t, MetaPipeline &meta_pipe auto &state = meta_pipeline.GetState(); state.AddPipelineOperator(current, &op); + // save the last added pipeline (in case we need to add a child pipeline) + vector> pipelines_so_far; + meta_pipeline.GetPipelines(pipelines_so_far, false); + auto last_pipeline = pipelines_so_far.back().get(); + // on the RHS (build side), we construct a child MetaPipeline with this operator as its sink auto child_meta_pipeline = meta_pipeline.CreateChildMetaPipeline(current, &op); child_meta_pipeline->Build(op.children[1].get()); @@ -64,7 +69,7 @@ void PhysicalJoin::BuildJoinPipelines(Pipeline ¤t, MetaPipeline &meta_pipe } if (add_child_pipeline) { - meta_pipeline.CreateChildPipeline(current, &op); + meta_pipeline.CreateChildPipeline(current, &op, last_pipeline); } } diff --git a/src/execution/operator/set/physical_union.cpp b/src/execution/operator/set/physical_union.cpp index f72365359e05..01cdd9237e6b 100644 --- a/src/execution/operator/set/physical_union.cpp +++ b/src/execution/operator/set/physical_union.cpp @@ -20,20 +20,25 @@ void PhysicalUnion::BuildPipelines(Pipeline ¤t, MetaPipeline &meta_pipelin op_state.reset(); sink_state.reset(); - // create a union pipeline that is identical, inheriting any dependencies encountered so far + // create a union pipeline that is identical to 'current' auto union_pipeline = meta_pipeline.CreateUnionPipeline(current); // continue with the current pipeline children[0]->BuildPipelines(current, meta_pipeline); - if (meta_pipeline.PreservesOrder()) { - // 'union_pipeline' must come after all pipelines created by building out 'current' if we want to preserve order - meta_pipeline.AddDependenciesFrom(union_pipeline, union_pipeline); - // FIXME: use batch index to parallelize/preserve order + auto sink = meta_pipeline.GetSink(); + if (sink && sink->IsOrderPreserving() && !sink->RequiresBatchIndex()) { + // we want to preserve order but we can't use batch indices, therefore + // 'union_pipeline' must come after all pipelines created by building out 'current' + meta_pipeline.AddDependenciesFrom(union_pipeline, union_pipeline, false); } // build the union pipeline children[1]->BuildPipelines(*union_pipeline, meta_pipeline); + + // Assign proper batch index to the union pipeline + // This needs to happen after the pipelines have been built because unions can be nested + meta_pipeline.AssignNextBatchIndex(union_pipeline); } vector PhysicalUnion::GetSources() const { diff --git a/src/execution/physical_operator.cpp b/src/execution/physical_operator.cpp index ff121bf440e4..c093e373c2e4 100644 --- a/src/execution/physical_operator.cpp +++ b/src/execution/physical_operator.cpp @@ -189,7 +189,7 @@ bool PhysicalOperator::AllOperatorsPreserveOrder() const { return false; } for (auto &child : children) { - if (!child->IsOrderPreserving()) { + if (!child->AllOperatorsPreserveOrder()) { return false; } } diff --git a/src/include/duckdb/execution/executor.hpp b/src/include/duckdb/execution/executor.hpp index c573bb637a99..577268be06a8 100644 --- a/src/include/duckdb/execution/executor.hpp +++ b/src/include/duckdb/execution/executor.hpp @@ -117,6 +117,8 @@ class Executor { vector> pipelines; //! The root pipelines of the query vector> root_pipelines; + //! The recursive CTE's in this query plan + vector recursive_ctes; //! The pipeline executor for the root pipeline unique_ptr root_executor; //! The current root pipeline index diff --git a/src/include/duckdb/execution/operator/helper/physical_execute.hpp b/src/include/duckdb/execution/operator/helper/physical_execute.hpp index e74d7ab43d2a..7a15067aa526 100644 --- a/src/include/duckdb/execution/operator/helper/physical_execute.hpp +++ b/src/include/duckdb/execution/operator/helper/physical_execute.hpp @@ -25,6 +25,7 @@ class PhysicalExecute : public PhysicalOperator { vector GetChildren() const override; public: + bool AllOperatorsPreserveOrder() const override; void BuildPipelines(Pipeline ¤t, MetaPipeline &meta_pipeline) override; }; diff --git a/src/include/duckdb/execution/operator/helper/physical_result_collector.hpp b/src/include/duckdb/execution/operator/helper/physical_result_collector.hpp index 2a2f4bd98e88..ce69f5c9e5ea 100644 --- a/src/include/duckdb/execution/operator/helper/physical_result_collector.hpp +++ b/src/include/duckdb/execution/operator/helper/physical_result_collector.hpp @@ -17,7 +17,7 @@ class PreparedStatementData; //! PhysicalResultCollector is an abstract class that is used to generate the final result of a query class PhysicalResultCollector : public PhysicalOperator { public: - PhysicalResultCollector(PreparedStatementData &data); + explicit PhysicalResultCollector(PreparedStatementData &data); StatementType statement_type; StatementProperties properties; @@ -37,7 +37,7 @@ class PhysicalResultCollector : public PhysicalOperator { public: vector GetChildren() const override; - + bool AllOperatorsPreserveOrder() const override; void BuildPipelines(Pipeline ¤t, MetaPipeline &meta_pipeline) override; }; diff --git a/src/include/duckdb/execution/operator/order/physical_order.hpp b/src/include/duckdb/execution/operator/order/physical_order.hpp index ea5125ae90dc..98718269a1d8 100644 --- a/src/include/duckdb/execution/operator/order/physical_order.hpp +++ b/src/include/duckdb/execution/operator/order/physical_order.hpp @@ -33,6 +33,10 @@ class PhysicalOrder : public PhysicalOperator { void GetData(ExecutionContext &context, DataChunk &chunk, GlobalSourceState &gstate, LocalSourceState &lstate) const override; + bool IsOrderPreserving() const override { + return false; + } + public: unique_ptr GetLocalSinkState(ExecutionContext &context) const override; unique_ptr GetGlobalSinkState(ClientContext &context) const override; diff --git a/src/include/duckdb/execution/physical_operator.hpp b/src/include/duckdb/execution/physical_operator.hpp index 9e3e380a2063..7640ee2ef1c7 100644 --- a/src/include/duckdb/execution/physical_operator.hpp +++ b/src/include/duckdb/execution/physical_operator.hpp @@ -219,7 +219,7 @@ class PhysicalOperator { // Pipeline construction virtual vector GetSources() const; bool AllSourcesSupportBatchIndex() const; - bool AllOperatorsPreserveOrder() const; + virtual bool AllOperatorsPreserveOrder() const; virtual void BuildPipelines(Pipeline ¤t, MetaPipeline &meta_pipeline); }; diff --git a/src/include/duckdb/parallel/meta_pipeline.hpp b/src/include/duckdb/parallel/meta_pipeline.hpp index a621feb7c611..59e8e9628e48 100644 --- a/src/include/duckdb/parallel/meta_pipeline.hpp +++ b/src/include/duckdb/parallel/meta_pipeline.hpp @@ -45,12 +45,17 @@ class MetaPipeline : public std::enable_shared_from_this { void GetMetaPipelines(vector> &result, bool recursive, bool skip); //! Get the inter-MetaPipeline dependencies of the given Pipeline const vector *GetDependencies(Pipeline *dependant) const; - //! Whether this MetaPipeline has a recursive CTE - bool HasRecursiveCTE() const; //! Whether the query plan preserves order bool PreservesOrder() const; - //! Let 'dependant' depend on all pipeline that were created since 'start' (including) - void AddDependenciesFrom(Pipeline *dependant, Pipeline *start); + //! Whether this MetaPipeline has a recursive CTE + bool HasRecursiveCTE() const; + //! Gets all recursive CTE's in the MetaPipeline + void GetRecursiveCTEs(vector &result) const; + //! Assign a batch index to the given pipeline + void AssignNextBatchIndex(Pipeline *pipeline); + //! Let 'dependant' depend on all pipeline that were created since 'start', + //! where 'including' determines whether 'start' is added to the dependencies + void AddDependenciesFrom(Pipeline *dependant, Pipeline *start, bool including); public: //! Build the MetaPipeline with 'op' as the first operator (excl. the shared sink) @@ -62,8 +67,9 @@ class MetaPipeline : public std::enable_shared_from_this { //! Create a union pipeline (clone of 'current') Pipeline *CreateUnionPipeline(Pipeline ¤t); - //! Create a child pipeline starting at 'op' - void CreateChildPipeline(Pipeline ¤t, PhysicalOperator *op); + //! Create a child pipeline op 'current' starting at 'op', + //! where 'last_pipeline' is the last pipeline added before building out 'current' + void CreateChildPipeline(Pipeline ¤t, PhysicalOperator *op, Pipeline *last_pipeline); //! Create a MetaPipeline child that 'current' depends on MetaPipeline *CreateChildMetaPipeline(Pipeline ¤t, PhysicalOperator *op); @@ -88,7 +94,9 @@ class MetaPipeline : public std::enable_shared_from_this { unordered_map> dependencies; //! Other MetaPipelines that this MetaPipeline depends on vector> children; - //! Whether the entire plan preserves order + //! Next batch index + idx_t next_batch_index; + //! Whether the query plan preserves order bool preserves_order; }; diff --git a/src/include/duckdb/parallel/pipeline.hpp b/src/include/duckdb/parallel/pipeline.hpp index a4f6d60ade24..3dd3d7fcaa76 100644 --- a/src/include/duckdb/parallel/pipeline.hpp +++ b/src/include/duckdb/parallel/pipeline.hpp @@ -89,6 +89,8 @@ class Pipeline : public std::enable_shared_from_this { //! Returns whether any of the operators in the pipeline care about preserving insertion order bool IsOrderDependent() const; + //! Returns whether all operators in this pipeline preserve order + bool PreservesOrder() const; private: //! Whether or not the pipeline has been readied diff --git a/src/parallel/executor.cpp b/src/parallel/executor.cpp index e4f77a205453..3b25dff54e8a 100644 --- a/src/parallel/executor.cpp +++ b/src/parallel/executor.cpp @@ -3,6 +3,7 @@ #include "duckdb/execution/execution_context.hpp" #include "duckdb/execution/operator/helper/physical_result_collector.hpp" #include "duckdb/execution/physical_operator.hpp" +#include "duckdb/execution/physical_plan_generator.hpp" #include "duckdb/main/client_context.hpp" #include "duckdb/main/client_data.hpp" #include "duckdb/parallel/pipeline_complete_event.hpp" @@ -288,6 +289,9 @@ void Executor::InitializeInternal(PhysicalOperator *plan) { root_pipeline->GetPipelines(root_pipelines, false); root_pipeline_idx = 0; + // collect the recursive CTE's (needed to be able to cancel query execution + root_pipeline->GetRecursiveCTEs(recursive_ctes); + // collect all meta-pipelines from the root pipeline vector> to_schedule; root_pipeline->GetMetaPipelines(to_schedule, true, true); @@ -317,6 +321,11 @@ void Executor::CancelTasks() { for (auto &pipeline : pipelines) { weak_references.push_back(weak_ptr(pipeline)); } + for (auto op : recursive_ctes) { + D_ASSERT(op->type == PhysicalOperatorType::RECURSIVE_CTE); + auto &rec_cte = (PhysicalRecursiveCTE &)*op; + rec_cte.recursive_meta_pipeline.reset(); + } pipelines.clear(); root_pipelines.clear(); events.clear(); diff --git a/src/parallel/meta_pipeline.cpp b/src/parallel/meta_pipeline.cpp index 2384b1515499..90673516c535 100644 --- a/src/parallel/meta_pipeline.cpp +++ b/src/parallel/meta_pipeline.cpp @@ -2,15 +2,14 @@ #include "duckdb/execution/executor.hpp" #include "duckdb/execution/operator/set/physical_recursive_cte.hpp" -#include "duckdb/execution/physical_plan_generator.hpp" namespace duckdb { MetaPipeline::MetaPipeline(Executor &executor_p, PipelineBuildState &state_p, PhysicalOperator *sink_p, bool preserves_order) - : executor(executor_p), state(state_p), sink(sink_p), preserves_order(preserves_order) { + : executor(executor_p), state(state_p), sink(sink_p), next_batch_index(0), preserves_order(preserves_order) { auto base_pipeline = CreatePipeline(); - state.SetPipelineSink(*base_pipeline, sink, 0); + state.SetPipelineSink(*base_pipeline, sink, next_batch_index++); if (sink_p && sink_p->type == PhysicalOperatorType::RECURSIVE_CTE) { recursive_cte = (PhysicalRecursiveCTE *)sink; } @@ -61,12 +60,25 @@ const vector *MetaPipeline::GetDependencies(Pipeline *dependant) con } } +bool MetaPipeline::PreservesOrder() const { + return preserves_order; +} + bool MetaPipeline::HasRecursiveCTE() const { return recursive_cte != nullptr; } -bool MetaPipeline::PreservesOrder() const { - return preserves_order; +void MetaPipeline::GetRecursiveCTEs(vector &result) const { + if (sink && sink->type == PhysicalOperatorType::RECURSIVE_CTE) { + result.push_back(sink); + } + for (auto &child : children) { + child->GetRecursiveCTEs(result); + } +} + +void MetaPipeline::AssignNextBatchIndex(Pipeline *pipeline) { + pipeline->base_batch_index = next_batch_index++ * PipelineBuildState::BATCH_INCREMENT; } void MetaPipeline::Build(PhysicalOperator *op) { @@ -91,7 +103,9 @@ void MetaPipeline::Reset(bool reset_sink) { } for (auto &pipeline : pipelines) { for (auto &op : pipeline->GetOperators()) { - op->op_state = op->GetGlobalOperatorState(executor.context); + if (op) { + op->op_state = op->GetGlobalOperatorState(executor.context); + } } pipeline->Reset(); } @@ -117,12 +131,16 @@ Pipeline *MetaPipeline::CreatePipeline() { return pipelines.back().get(); } -void MetaPipeline::AddDependenciesFrom(Pipeline *dependant, Pipeline *start) { +void MetaPipeline::AddDependenciesFrom(Pipeline *dependant, Pipeline *start, bool including) { // find 'start' auto it = pipelines.begin(); for (; it->get() != start; it++) { } + if (!including) { + it++; + } + // collect pipelines that were created from then vector created_pipelines; for (; it != pipelines.end(); it++) { @@ -146,7 +164,7 @@ Pipeline *MetaPipeline::CreateUnionPipeline(Pipeline ¤t) { // create the union pipeline auto union_pipeline = CreatePipeline(); state.SetPipelineOperators(*union_pipeline, state.GetPipelineOperators(current)); - state.SetPipelineSink(*union_pipeline, sink, pipelines.size() - 1); + state.SetPipelineSink(*union_pipeline, sink, 0); // 'union_pipeline' inherits ALL dependencies of 'current' (intra- and inter-MetaPipeline) union_pipeline->dependencies = current.dependencies; @@ -155,15 +173,15 @@ Pipeline *MetaPipeline::CreateUnionPipeline(Pipeline ¤t) { dependencies[union_pipeline] = *current_inter_deps; } - if (sink && !sink->ParallelSink()) { - // if the sink is not parallel, we set a dependency + if (sink && sink->IsOrderPreserving() && !sink->RequiresBatchIndex()) { + // if we need to preserve order, or if the sink is not parallel, we set a dependency dependencies[union_pipeline].push_back(¤t); } return union_pipeline; } -void MetaPipeline::CreateChildPipeline(Pipeline ¤t, PhysicalOperator *op) { +void MetaPipeline::CreateChildPipeline(Pipeline ¤t, PhysicalOperator *op, Pipeline *last_pipeline) { // rule 2: 'current' must be fully built (down to the source) before creating the child pipeline D_ASSERT(current.source); if (HasRecursiveCTE()) { @@ -176,7 +194,8 @@ void MetaPipeline::CreateChildPipeline(Pipeline ¤t, PhysicalOperator *op) // child pipeline has an inter-MetaPipeline depency on all pipelines that were scheduled between 'current' and now // (including 'current') - set them up - AddDependenciesFrom(child_pipeline, ¤t); + dependencies[child_pipeline].push_back(¤t); + AddDependenciesFrom(child_pipeline, last_pipeline, false); D_ASSERT(!GetDependencies(child_pipeline)->empty()); } diff --git a/src/parallel/pipeline.cpp b/src/parallel/pipeline.cpp index e63a0449d3c9..b768f78555f1 100644 --- a/src/parallel/pipeline.cpp +++ b/src/parallel/pipeline.cpp @@ -115,6 +115,25 @@ bool Pipeline::IsOrderDependent() const { return false; } +bool Pipeline::PreservesOrder() const { + auto &config = DBConfig::GetConfig(executor.context); + if (!config.options.preserve_insertion_order) { + return false; + } + if (sink && sink->IsOrderPreserving()) { + return true; + } + if (source && source->IsOrderPreserving()) { + return true; + } + for (auto &op : operators) { + if (!op->IsOrderPreserving()) { + return false; + } + } + return true; +} + void Pipeline::Schedule(shared_ptr &event) { D_ASSERT(ready); D_ASSERT(sink); diff --git a/test/optimizer/statistics/statistics_setop.test b/test/optimizer/statistics/statistics_setop.test index 4420ef6d9149..c8fe6483d47c 100644 --- a/test/optimizer/statistics/statistics_setop.test +++ b/test/optimizer/statistics/statistics_setop.test @@ -11,6 +11,8 @@ CREATE TABLE integers2 AS SELECT * FROM (VALUES (4), (5), (6)) tbl(i); statement ok CREATE TABLE integers3 AS SELECT * FROM (VALUES (4), (5), (NULL)) tbl(i); +mode skip + statement ok PRAGMA enable_verification; @@ -105,6 +107,8 @@ SELECT i IS NULL FROM (SELECT * FROM integers UNION ALL SELECT * FROM integers2) 0 0 +mode unskip + query I SELECT i IS NULL FROM (SELECT * FROM integers UNION ALL SELECT * FROM integers2 UNION ALL SELECT * FROM integers3) tbl(i); ---- @@ -118,6 +122,8 @@ SELECT i IS NULL FROM (SELECT * FROM integers UNION ALL SELECT * FROM integers2 0 1 +mode skip + query I SELECT i=7 FROM (SELECT * FROM integers2 EXCEPT SELECT * FROM integers3) tbl(i); ---- diff --git a/test/sql/binder/test_null_type_propagation.test b/test/sql/binder/test_null_type_propagation.test index c49c2f4d31d6..b839e1031318 100644 --- a/test/sql/binder/test_null_type_propagation.test +++ b/test/sql/binder/test_null_type_propagation.test @@ -81,14 +81,16 @@ SELECT bool_and(i), bool_and(j) FROM (SELECT NULL) tbl(i), (SELECT NULL) tbl2(j) ---- NULL NULL +# queries with UNION (not UNION ALL) need the 'sort' because UNION is an aggregate - no order-preserving guarantees + # Issue #1049: Should a UNION with untyped NULL preserve the type? -query III +query III sort SELECT NULL as a, NULL as b, 1 as id UNION SELECT CAST(1 AS BOOLEAN) as a, CAST(0 AS BOOLEAN) as b, 2 as id ---- -NULL NULL 1 True False 2 +NULL NULL 1 -query III +query III sort SELECT CAST(1 AS BOOLEAN) as a, CAST(0 AS BOOLEAN) as b, 1 as id UNION SELECT NULL as a, NULL as b, 2 as id ---- True False 1 diff --git a/test/sql/cte/recursive_cte_complex_pipelines.test b/test/sql/cte/recursive_cte_complex_pipelines.test index 9fbe836958a7..260fc45704ac 100644 --- a/test/sql/cte/recursive_cte_complex_pipelines.test +++ b/test/sql/cte/recursive_cte_complex_pipelines.test @@ -2,6 +2,8 @@ # description: Recursive CTEs with complex pipelines # group: [cte] +mode skip + statement ok PRAGMA enable_verification @@ -43,9 +45,13 @@ SELECT * FROM t ORDER BY 1; 81 243 +mode unskip + statement ok CREATE TABLE a AS SELECT * FROM range(100) t1(i) +mode skip + # recursive CTE with aggregates query I WITH RECURSIVE t AS @@ -64,6 +70,8 @@ SELECT * FROM t ORDER BY 1 NULLS LAST; 1000000 NULL +mode unskip + # the same but with a hash join query I WITH RECURSIVE t AS diff --git a/test/sql/order/test_limit_percent.test b/test/sql/order/test_limit_percent.test index 4ea6215ef1a3..cea7be4207ad 100644 --- a/test/sql/order/test_limit_percent.test +++ b/test/sql/order/test_limit_percent.test @@ -228,3 +228,16 @@ SELECT * FROM integers as int LIMIT (SELECT max(integers.i) FROM integers where 3 4 5 + +# unions are usually executed in parallel, and kept in order using batch indices +# limit percent is not a parallel sink, and does not support batch indices +# this tests that the order is still preserved +loop i 0 10 + +query T +select 1 union all select 2 union all select 3 union all select 4 limit 50% +---- +1 +2 + +endloop diff --git a/test/sql/parallelism/intraquery/test_simple_parallelism.test b/test/sql/parallelism/intraquery/test_simple_parallelism.test index f842cb328ce2..1bf2862df086 100644 --- a/test/sql/parallelism/intraquery/test_simple_parallelism.test +++ b/test/sql/parallelism/intraquery/test_simple_parallelism.test @@ -44,11 +44,11 @@ statement error SELECT SUM(i) FROM integers UNION ALL SELECT AVG(i) FROM integers UNION ALL SELECT MIN(i::DATE) FROM integers UNION ALL SELECT MAX(i::DATE) FROM integers; # errors are properly cleared -query R +query R sort SELECT * FROM v1 t1 JOIN v1 t2 USING (i); ---- -6.000000 -2.000000 1.000000 +2.000000 3.000000 +6.000000 diff --git a/test/sql/types/union/union_cast.test b/test/sql/types/union/union_cast.test index e8a26e2402c7..43605c4bad6f 100644 --- a/test/sql/types/union/union_cast.test +++ b/test/sql/types/union/union_cast.test @@ -54,7 +54,7 @@ NULL five NULL NULL NULL 6.0 query I -SELECT * FROM tbl2 UNION SELECT * FROM tbl1; +SELECT * FROM tbl2 UNION ALL SELECT * FROM tbl1; ---- five 4 @@ -64,7 +64,7 @@ two three query I -SELECT * FROM tbl1 UNION SELECT * FROM tbl2; +SELECT * FROM tbl1 UNION ALL SELECT * FROM tbl2; ---- 1 two @@ -74,7 +74,7 @@ five 6.0 query III -SELECT u.i32, u.str, u.f32 FROM tbl2 UNION SELECT u.i32, u.str, NULL FROM tbl1 ORDER BY ALL; +SELECT u.i32, u.str, u.f32 FROM tbl2 UNION ALL SELECT u.i32, u.str, NULL FROM tbl1 ORDER BY ALL; ---- NULL NULL 6.0 NULL five NULL From 8f789a95f33580d727941005b0c953d1b3308ce1 Mon Sep 17 00:00:00 2001 From: Laurens Kuiper Date: Mon, 31 Oct 2022 15:31:03 +0100 Subject: [PATCH 38/49] fix last issues with iejoin --- .../operator/join/physical_iejoin.cpp | 22 +++++++++++----- src/execution/operator/join/physical_join.cpp | 2 +- src/include/duckdb/parallel/meta_pipeline.hpp | 12 ++++----- src/parallel/executor.cpp | 26 ++++++++++++++----- src/parallel/meta_pipeline.cpp | 21 ++++++++------- .../binder/test_null_type_propagation.test | 6 ++--- 6 files changed, 55 insertions(+), 34 deletions(-) diff --git a/src/execution/operator/join/physical_iejoin.cpp b/src/execution/operator/join/physical_iejoin.cpp index 33588c6328b9..0e163b6f283b 100644 --- a/src/execution/operator/join/physical_iejoin.cpp +++ b/src/execution/operator/join/physical_iejoin.cpp @@ -1016,19 +1016,27 @@ void PhysicalIEJoin::BuildPipelines(Pipeline ¤t, MetaPipeline &meta_pipeli throw NotImplementedException("IEJoins are not supported in recursive CTEs yet"); } - // create LHS as a child MetaPipeline + // becomes a source after both children fully sink their data + meta_pipeline.GetState().SetPipelineSource(current, this); + + // create LHS as a child MetaPipeline of 'current', then build auto lhs_meta_pipeline = meta_pipeline.CreateChildMetaPipeline(current, this); lhs_meta_pipeline->Build(children[0].get()); - // create RHS as a child MetaPipeline + // create RHS as a child MetaPipeline of 'current', then build auto rhs_meta_pipeline = meta_pipeline.CreateChildMetaPipeline(current, this); rhs_meta_pipeline->Build(children[1].get()); - // RHS depends on LHS - rhs_meta_pipeline->GetBasePipeline()->AddDependency(lhs_meta_pipeline->GetBasePipeline()); - - // this operator becomes a source for the current pipeline after RHS and LHS have been built - meta_pipeline.GetState().SetPipelineSource(current, this); + // RHS pipelines depend on everything in LHS, set up the dependencies correctly + vector> lhs_pipelines; + lhs_meta_pipeline->GetPipelines(lhs_pipelines, false); + vector> rhs_pipelines; + rhs_meta_pipeline->GetPipelines(rhs_pipelines, false); + for (auto &rhs_pipeline : rhs_pipelines) { + for (auto &lhs_pipeline : lhs_pipelines) { + rhs_pipeline->AddDependency(lhs_pipeline); + } + } } } // namespace duckdb diff --git a/src/execution/operator/join/physical_join.cpp b/src/execution/operator/join/physical_join.cpp index 3433ecd53bfe..40cc3d984c29 100644 --- a/src/execution/operator/join/physical_join.cpp +++ b/src/execution/operator/join/physical_join.cpp @@ -34,7 +34,7 @@ void PhysicalJoin::BuildJoinPipelines(Pipeline ¤t, MetaPipeline &meta_pipe auto &state = meta_pipeline.GetState(); state.AddPipelineOperator(current, &op); - // save the last added pipeline (in case we need to add a child pipeline) + // save the last added pipeline to set up dependencies later (in case we need to add a child pipeline) vector> pipelines_so_far; meta_pipeline.GetPipelines(pipelines_so_far, false); auto last_pipeline = pipelines_so_far.back().get(); diff --git a/src/include/duckdb/parallel/meta_pipeline.hpp b/src/include/duckdb/parallel/meta_pipeline.hpp index b55c80cbc28f..cf455f1e2403 100644 --- a/src/include/duckdb/parallel/meta_pipeline.hpp +++ b/src/include/duckdb/parallel/meta_pipeline.hpp @@ -18,11 +18,11 @@ class PhysicalRecursiveCTE; class MetaPipeline : public std::enable_shared_from_this { //! We follow these rules when building: //! 1. For joins, build out the blocking side before going down the probe side - //! - The current streaming pipeline will have an intra-MetaPipeline dependency on it + //! - The current streaming pipeline will have a dependency on it (dependency across MetaPipelines) //! - Unions of this streaming pipeline will automatically inherit this dependency //! 2. Build child pipelines last (e.g., Hash Join becomes source after probe is done: scan HT for FULL OUTER JOIN) //! - 'last' means after building out all other pipelines associated with this operator - //! - The child pipeline automatically has inter-MetaPipeline dependencies on: + //! - The child pipeline automatically has dependencies (within this MetaPipeline) on: //! * The 'current' streaming pipeline //! * And all pipelines that were added to the MetaPipeline after 'current' public: @@ -43,7 +43,7 @@ class MetaPipeline : public std::enable_shared_from_this { void GetPipelines(vector> &result, bool recursive); //! Get the MetaPipeline children of this MetaPipeline void GetMetaPipelines(vector> &result, bool recursive, bool skip); - //! Get the inter-MetaPipeline dependencies of the given Pipeline + //! Get the dependencies (within this MetaPipeline) of the given Pipeline const vector *GetDependencies(Pipeline *dependant) const; //! Whether this MetaPipeline has a recursive CTE bool HasRecursiveCTE() const; @@ -59,6 +59,8 @@ class MetaPipeline : public std::enable_shared_from_this { //! Ready all the pipelines (recursively) void Ready(); + //! Create an empty pipeline within this MetaPipeline + Pipeline *CreatePipeline(); //! Create a union pipeline (clone of 'current') Pipeline *CreateUnionPipeline(Pipeline ¤t); //! Create a child pipeline op 'current' starting at 'op', @@ -67,10 +69,6 @@ class MetaPipeline : public std::enable_shared_from_this { //! Create a MetaPipeline child that 'current' depends on MetaPipeline *CreateChildMetaPipeline(Pipeline ¤t, PhysicalOperator *op); -private: - //! Create an empty pipeline within this MetaPipeline - Pipeline *CreatePipeline(); - private: //! The executor for all MetaPipelines in the query plan Executor &executor; diff --git a/src/parallel/executor.cpp b/src/parallel/executor.cpp index eecb70938470..e19fbd6d93ad 100644 --- a/src/parallel/executor.cpp +++ b/src/parallel/executor.cpp @@ -54,6 +54,7 @@ struct ScheduleEventData { vector> &events; bool initial_schedule; event_map_t event_map; + unordered_map sink_init_map; }; void Executor::SchedulePipeline(const shared_ptr &meta_pipeline, ScheduleEventData &event_data) { @@ -63,13 +64,26 @@ void Executor::SchedulePipeline(const shared_ptr &meta_pipeline, S // create events/stack for the base pipeline auto base_pipeline = meta_pipeline->GetBasePipeline(); - auto base_initialize_event = make_shared(base_pipeline); + auto shared_sink = meta_pipeline->GetSink(); + + Event *init_event; + auto &sink_init_map = event_data.sink_init_map; + auto it = sink_init_map.find(shared_sink); + if (it == sink_init_map.end()) { + auto base_initialize_event = make_shared(base_pipeline); + init_event = base_initialize_event.get(); + sink_init_map[shared_sink] = init_event; + events.push_back(move(base_initialize_event)); + } else { + // sink shared across multiple MetaPipelines (double pipeline breakers like IEJoin) + init_event = it->second; + } + auto base_event = make_shared(base_pipeline); auto base_finish_event = make_shared(base_pipeline); auto base_complete_event = make_shared(base_pipeline->executor, event_data.initial_schedule); - PipelineEventStack base_stack {base_initialize_event.get(), base_event.get(), base_finish_event.get(), - base_complete_event.get()}; - events.push_back(move(base_initialize_event)); + PipelineEventStack base_stack {init_event, base_event.get(), base_finish_event.get(), base_complete_event.get()}; + events.push_back(move(base_event)); events.push_back(move(base_finish_event)); events.push_back(move(base_complete_event)); @@ -103,7 +117,7 @@ void Executor::SchedulePipeline(const shared_ptr &meta_pipeline, S // add base stack to the event data too event_map.insert(make_pair(base_pipeline.get(), move(base_stack))); - // set up the inter-MetaPipeline dependencies + // set up the dependencies within this MetaPipeline for (auto &pipeline : pipelines) { auto dependencies = meta_pipeline->GetDependencies(pipeline.get()); if (!dependencies) { @@ -126,7 +140,7 @@ void Executor::ScheduleEventsInternal(ScheduleEventData &event_data) { SchedulePipeline(pipeline, event_data); } - // set up the intra-MetaPipeline dependencies + // set up the dependencies across MetaPipelines auto &event_map = event_data.event_map; for (auto &entry : event_map) { auto pipeline = entry.first; diff --git a/src/parallel/meta_pipeline.cpp b/src/parallel/meta_pipeline.cpp index 609a847e400e..23651377ce81 100644 --- a/src/parallel/meta_pipeline.cpp +++ b/src/parallel/meta_pipeline.cpp @@ -7,8 +7,7 @@ namespace duckdb { MetaPipeline::MetaPipeline(Executor &executor_p, PipelineBuildState &state_p, PhysicalOperator *sink_p) : executor(executor_p), state(state_p), sink(sink_p), next_batch_index(0) { - auto base_pipeline = CreatePipeline(); - state.SetPipelineSink(*base_pipeline, sink, next_batch_index++); + CreatePipeline(); if (sink_p && sink_p->type == PhysicalOperatorType::RECURSIVE_CTE) { recursive_cte = (PhysicalRecursiveCTE *)sink; } @@ -97,6 +96,7 @@ MetaPipeline *MetaPipeline::CreateChildMetaPipeline(Pipeline ¤t, PhysicalO Pipeline *MetaPipeline::CreatePipeline() { pipelines.emplace_back(make_unique(executor)); + state.SetPipelineSink(*pipelines.back(), sink, next_batch_index++); return pipelines.back().get(); } @@ -130,16 +130,16 @@ Pipeline *MetaPipeline::CreateUnionPipeline(Pipeline ¤t) { throw NotImplementedException("UNIONS are not supported in recursive CTEs yet"); } - // create the union pipeline + // create the union pipeline (batch index 0, should be set correctly afterwards) auto union_pipeline = CreatePipeline(); state.SetPipelineOperators(*union_pipeline, state.GetPipelineOperators(current)); state.SetPipelineSink(*union_pipeline, sink, 0); - // 'union_pipeline' inherits ALL dependencies of 'current' (intra- and inter-MetaPipeline) + // 'union_pipeline' inherits ALL dependencies of 'current' (within this MetaPipeline, and across MetaPipelines) union_pipeline->dependencies = current.dependencies; - auto current_inter_deps = GetDependencies(¤t); - if (current_inter_deps) { - dependencies[union_pipeline] = *current_inter_deps; + auto current_deps = GetDependencies(¤t); + if (current_deps) { + dependencies[union_pipeline] = *current_deps; } if (sink && sink->IsOrderPreserving() && !sink->RequiresBatchIndex()) { @@ -157,12 +157,13 @@ void MetaPipeline::CreateChildPipeline(Pipeline ¤t, PhysicalOperator *op, throw NotImplementedException("Child pipelines are not supported in recursive CTEs yet"); } - // create the child pipeline + // create the child pipeline (same batch index) pipelines.emplace_back(state.CreateChildPipeline(executor, current, op)); auto child_pipeline = pipelines.back().get(); + child_pipeline->base_batch_index = current.base_batch_index; - // child pipeline has an inter-MetaPipeline depency on all pipelines that were scheduled between 'current' and now - // (including 'current') - set them up + // child pipeline has a depency (within this MetaPipeline on all pipelines that were scheduled + // between 'current' and now (including 'current') - set them up dependencies[child_pipeline].push_back(¤t); AddDependenciesFrom(child_pipeline, last_pipeline, false); D_ASSERT(!GetDependencies(child_pipeline)->empty()); diff --git a/test/sql/binder/test_null_type_propagation.test b/test/sql/binder/test_null_type_propagation.test index b839e1031318..35fa1f74f129 100644 --- a/test/sql/binder/test_null_type_propagation.test +++ b/test/sql/binder/test_null_type_propagation.test @@ -97,13 +97,13 @@ True False 1 NULL NULL 2 # old behavior: cast NULL to integer explicitly -query III +query III sort SELECT NULL::INTEGER as a, NULL::INTEGER as b, 1 as id UNION SELECT CAST(1 AS BOOLEAN) as a, CAST(0 AS BOOLEAN) as b, 2 as id ---- -NULL NULL 1 1 0 2 +NULL NULL 1 -query III +query III sort SELECT CAST(1 AS BOOLEAN) as a, CAST(0 AS BOOLEAN) as b, 1 as id UNION SELECT NULL::INTEGER as a, NULL::INTEGER as b, 2 as id ---- 1 0 1 From fd943f9c2ff85cb25d5eac8f4aacfc3b27be9bbe Mon Sep 17 00:00:00 2001 From: Laurens Kuiper Date: Tue, 1 Nov 2022 09:50:32 +0100 Subject: [PATCH 39/49] fix bug with union all order preservation --- src/execution/operator/set/physical_union.cpp | 13 ++++++++----- src/include/duckdb/parallel/meta_pipeline.hpp | 2 +- src/parallel/meta_pipeline.cpp | 4 ++-- src/parallel/pipeline.cpp | 2 +- 4 files changed, 12 insertions(+), 9 deletions(-) diff --git a/src/execution/operator/set/physical_union.cpp b/src/execution/operator/set/physical_union.cpp index 01cdd9237e6b..2bab7a593db6 100644 --- a/src/execution/operator/set/physical_union.cpp +++ b/src/execution/operator/set/physical_union.cpp @@ -20,16 +20,19 @@ void PhysicalUnion::BuildPipelines(Pipeline ¤t, MetaPipeline &meta_pipelin op_state.reset(); sink_state.reset(); + // order matters if any of the downstream operators are order dependent, + // or if the sink preserves order, but does not support batch indices to do so + auto snk = meta_pipeline.GetSink(); + bool order_matters = current.IsOrderDependent() || (snk && snk->IsOrderPreserving() && !snk->RequiresBatchIndex()); + // create a union pipeline that is identical to 'current' - auto union_pipeline = meta_pipeline.CreateUnionPipeline(current); + auto union_pipeline = meta_pipeline.CreateUnionPipeline(current, order_matters); // continue with the current pipeline children[0]->BuildPipelines(current, meta_pipeline); - auto sink = meta_pipeline.GetSink(); - if (sink && sink->IsOrderPreserving() && !sink->RequiresBatchIndex()) { - // we want to preserve order but we can't use batch indices, therefore - // 'union_pipeline' must come after all pipelines created by building out 'current' + if (order_matters) { + // order matters, so 'union_pipeline' must come after all pipelines created by building out 'current' meta_pipeline.AddDependenciesFrom(union_pipeline, union_pipeline, false); } diff --git a/src/include/duckdb/parallel/meta_pipeline.hpp b/src/include/duckdb/parallel/meta_pipeline.hpp index cf455f1e2403..6041da91b2f1 100644 --- a/src/include/duckdb/parallel/meta_pipeline.hpp +++ b/src/include/duckdb/parallel/meta_pipeline.hpp @@ -62,7 +62,7 @@ class MetaPipeline : public std::enable_shared_from_this { //! Create an empty pipeline within this MetaPipeline Pipeline *CreatePipeline(); //! Create a union pipeline (clone of 'current') - Pipeline *CreateUnionPipeline(Pipeline ¤t); + Pipeline *CreateUnionPipeline(Pipeline ¤t, bool order_matters); //! Create a child pipeline op 'current' starting at 'op', //! where 'last_pipeline' is the last pipeline added before building out 'current' void CreateChildPipeline(Pipeline ¤t, PhysicalOperator *op, Pipeline *last_pipeline); diff --git a/src/parallel/meta_pipeline.cpp b/src/parallel/meta_pipeline.cpp index 23651377ce81..75f4d0ac0208 100644 --- a/src/parallel/meta_pipeline.cpp +++ b/src/parallel/meta_pipeline.cpp @@ -125,7 +125,7 @@ void MetaPipeline::AddDependenciesFrom(Pipeline *dependant, Pipeline *start, boo deps.insert(deps.begin(), created_pipelines.begin(), created_pipelines.end()); } -Pipeline *MetaPipeline::CreateUnionPipeline(Pipeline ¤t) { +Pipeline *MetaPipeline::CreateUnionPipeline(Pipeline ¤t, bool order_matters) { if (HasRecursiveCTE()) { throw NotImplementedException("UNIONS are not supported in recursive CTEs yet"); } @@ -142,7 +142,7 @@ Pipeline *MetaPipeline::CreateUnionPipeline(Pipeline ¤t) { dependencies[union_pipeline] = *current_deps; } - if (sink && sink->IsOrderPreserving() && !sink->RequiresBatchIndex()) { + if (order_matters) { // if we need to preserve order, or if the sink is not parallel, we set a dependency dependencies[union_pipeline].push_back(¤t); } diff --git a/src/parallel/pipeline.cpp b/src/parallel/pipeline.cpp index e63a0449d3c9..d160a18b4b87 100644 --- a/src/parallel/pipeline.cpp +++ b/src/parallel/pipeline.cpp @@ -104,7 +104,7 @@ bool Pipeline::IsOrderDependent() const { if (sink && sink->IsOrderDependent()) { return true; } - if (source->IsOrderDependent()) { + if (source && source->IsOrderDependent()) { return true; } for (auto &op : operators) { From e366a0004ba2a3eacc4336cbcd55e719fbf3283b Mon Sep 17 00:00:00 2001 From: Laurens Kuiper Date: Tue, 1 Nov 2022 10:15:09 +0100 Subject: [PATCH 40/49] refactor IEJoin pipelines to have a single MetaPipeline --- .../operator/join/physical_iejoin.cpp | 34 +++++++++--------- src/include/duckdb/parallel/meta_pipeline.hpp | 6 ++++ src/parallel/executor.cpp | 36 +++++++++---------- src/parallel/meta_pipeline.cpp | 8 +++++ 4 files changed, 46 insertions(+), 38 deletions(-) diff --git a/src/execution/operator/join/physical_iejoin.cpp b/src/execution/operator/join/physical_iejoin.cpp index 0e163b6f283b..c31cb7c2c0c1 100644 --- a/src/execution/operator/join/physical_iejoin.cpp +++ b/src/execution/operator/join/physical_iejoin.cpp @@ -1019,24 +1019,22 @@ void PhysicalIEJoin::BuildPipelines(Pipeline ¤t, MetaPipeline &meta_pipeli // becomes a source after both children fully sink their data meta_pipeline.GetState().SetPipelineSource(current, this); - // create LHS as a child MetaPipeline of 'current', then build - auto lhs_meta_pipeline = meta_pipeline.CreateChildMetaPipeline(current, this); - lhs_meta_pipeline->Build(children[0].get()); - - // create RHS as a child MetaPipeline of 'current', then build - auto rhs_meta_pipeline = meta_pipeline.CreateChildMetaPipeline(current, this); - rhs_meta_pipeline->Build(children[1].get()); - - // RHS pipelines depend on everything in LHS, set up the dependencies correctly - vector> lhs_pipelines; - lhs_meta_pipeline->GetPipelines(lhs_pipelines, false); - vector> rhs_pipelines; - rhs_meta_pipeline->GetPipelines(rhs_pipelines, false); - for (auto &rhs_pipeline : rhs_pipelines) { - for (auto &lhs_pipeline : lhs_pipelines) { - rhs_pipeline->AddDependency(lhs_pipeline); - } - } + // Create one child meta pipeline that will hold the LHS and RHS pipelines + auto child_meta_pipeline = meta_pipeline.CreateChildMetaPipeline(current, this); + auto lhs_pipeline = child_meta_pipeline->GetBasePipeline(); + auto rhs_pipeline = child_meta_pipeline->CreatePipeline(); + + // Build out LHS + children[0]->BuildPipelines(*lhs_pipeline, *child_meta_pipeline); + + // RHS depends on everything in LHS + child_meta_pipeline->AddDependenciesFrom(rhs_pipeline, lhs_pipeline.get(), true); + + // Build out RHS + children[1]->BuildPipelines(*rhs_pipeline, *child_meta_pipeline); + + // Despite having the same sink, RHS needs its own PipelineFinishEvent + child_meta_pipeline->AddFinishEvent(rhs_pipeline); } } // namespace duckdb diff --git a/src/include/duckdb/parallel/meta_pipeline.hpp b/src/include/duckdb/parallel/meta_pipeline.hpp index 6041da91b2f1..642a912bd539 100644 --- a/src/include/duckdb/parallel/meta_pipeline.hpp +++ b/src/include/duckdb/parallel/meta_pipeline.hpp @@ -52,6 +52,10 @@ class MetaPipeline : public std::enable_shared_from_this { //! Let 'dependant' depend on all pipeline that were created since 'start', //! where 'including' determines whether 'start' is added to the dependencies void AddDependenciesFrom(Pipeline *dependant, Pipeline *start, bool including); + //! Make sure that the given pipeline has its own PipelineFinishEvent (e.g., for IEJoin - double Finalize) + void AddFinishEvent(Pipeline *pipeline); + //! Whether the pipeline needs its own PipelineFinishEvent + bool HasFinishEvent(Pipeline *pipeline); public: //! Build the MetaPipeline with 'op' as the first operator (excl. the shared sink) @@ -88,6 +92,8 @@ class MetaPipeline : public std::enable_shared_from_this { vector> children; //! Next batch index idx_t next_batch_index; + //! Pipelines (other than the base pipeline) that need their own PipelineFinishEvent (e.g., for IEJoin) + unordered_set finish_pipelines; }; } // namespace duckdb diff --git a/src/parallel/executor.cpp b/src/parallel/executor.cpp index e19fbd6d93ad..822a6ea5e2c2 100644 --- a/src/parallel/executor.cpp +++ b/src/parallel/executor.cpp @@ -54,7 +54,6 @@ struct ScheduleEventData { vector> &events; bool initial_schedule; event_map_t event_map; - unordered_map sink_init_map; }; void Executor::SchedulePipeline(const shared_ptr &meta_pipeline, ScheduleEventData &event_data) { @@ -64,26 +63,13 @@ void Executor::SchedulePipeline(const shared_ptr &meta_pipeline, S // create events/stack for the base pipeline auto base_pipeline = meta_pipeline->GetBasePipeline(); - auto shared_sink = meta_pipeline->GetSink(); - - Event *init_event; - auto &sink_init_map = event_data.sink_init_map; - auto it = sink_init_map.find(shared_sink); - if (it == sink_init_map.end()) { - auto base_initialize_event = make_shared(base_pipeline); - init_event = base_initialize_event.get(); - sink_init_map[shared_sink] = init_event; - events.push_back(move(base_initialize_event)); - } else { - // sink shared across multiple MetaPipelines (double pipeline breakers like IEJoin) - init_event = it->second; - } - + auto base_initialize_event = make_shared(base_pipeline); auto base_event = make_shared(base_pipeline); auto base_finish_event = make_shared(base_pipeline); auto base_complete_event = make_shared(base_pipeline->executor, event_data.initial_schedule); - PipelineEventStack base_stack {init_event, base_event.get(), base_finish_event.get(), base_complete_event.get()}; - + PipelineEventStack base_stack {base_initialize_event.get(), base_event.get(), base_finish_event.get(), + base_complete_event.get()}; + events.push_back(move(base_initialize_event)); events.push_back(move(base_event)); events.push_back(move(base_finish_event)); events.push_back(move(base_complete_event)); @@ -102,13 +88,23 @@ void Executor::SchedulePipeline(const shared_ptr &meta_pipeline, S // create events/stack for this pipeline auto pipeline_event = make_shared(pipeline); + Event *pipeline_finish_event_ptr; + if (meta_pipeline->HasFinishEvent(pipeline.get())) { + // this pipeline has its own finish event (despite going into the same sink - Finalize twice!) + auto pipeline_finish_event = make_unique(pipeline); + pipeline_finish_event_ptr = pipeline_finish_event.get(); + events.push_back(move(pipeline_finish_event)); + base_stack.pipeline_complete_event->AddDependency(*pipeline_finish_event_ptr); + } else { + pipeline_finish_event_ptr = base_stack.pipeline_finish_event; + } PipelineEventStack pipeline_stack {base_stack.pipeline_initialize_event, pipeline_event.get(), - base_stack.pipeline_finish_event, base_stack.pipeline_complete_event}; + pipeline_finish_event_ptr, base_stack.pipeline_complete_event}; events.push_back(move(pipeline_event)); // dependencies: base_initialize -> pipeline_event -> base_finish pipeline_stack.pipeline_event->AddDependency(*base_stack.pipeline_initialize_event); - base_stack.pipeline_finish_event->AddDependency(*pipeline_stack.pipeline_event); + pipeline_stack.pipeline_finish_event->AddDependency(*pipeline_stack.pipeline_event); // add pipeline stack to event map event_map.insert(make_pair(pipeline.get(), move(pipeline_stack))); diff --git a/src/parallel/meta_pipeline.cpp b/src/parallel/meta_pipeline.cpp index 75f4d0ac0208..779bf368f3ea 100644 --- a/src/parallel/meta_pipeline.cpp +++ b/src/parallel/meta_pipeline.cpp @@ -125,6 +125,14 @@ void MetaPipeline::AddDependenciesFrom(Pipeline *dependant, Pipeline *start, boo deps.insert(deps.begin(), created_pipelines.begin(), created_pipelines.end()); } +void MetaPipeline::AddFinishEvent(Pipeline *pipeline) { + finish_pipelines.insert(pipeline); +} + +bool MetaPipeline::HasFinishEvent(Pipeline *pipeline) { + return finish_pipelines.find(pipeline) != finish_pipelines.end(); +} + Pipeline *MetaPipeline::CreateUnionPipeline(Pipeline ¤t, bool order_matters) { if (HasRecursiveCTE()) { throw NotImplementedException("UNIONS are not supported in recursive CTEs yet"); From 2433adebf3ca863343012a8d8fd3345b02111d71 Mon Sep 17 00:00:00 2001 From: Laurens Kuiper Date: Tue, 1 Nov 2022 10:30:27 +0100 Subject: [PATCH 41/49] properly set recursive CTE in MetaPipeline and add some missing includes --- .../operator/join/physical_index_join.cpp | 1 + .../operator/set/physical_recursive_cte.cpp | 1 + src/include/duckdb/parallel/meta_pipeline.hpp | 6 ++++-- src/parallel/executor.cpp | 2 ++ src/parallel/meta_pipeline.cpp | 15 +++++++-------- 5 files changed, 15 insertions(+), 10 deletions(-) diff --git a/src/execution/operator/join/physical_index_join.cpp b/src/execution/operator/join/physical_index_join.cpp index e7a9adff8f3b..5ce78b172b86 100644 --- a/src/execution/operator/join/physical_index_join.cpp +++ b/src/execution/operator/join/physical_index_join.cpp @@ -5,6 +5,7 @@ #include "duckdb/execution/index/art/art.hpp" #include "duckdb/execution/operator/scan/physical_table_scan.hpp" #include "duckdb/function/table/table_scan.hpp" +#include "duckdb/parallel/meta_pipeline.hpp" #include "duckdb/parallel/thread_context.hpp" #include "duckdb/storage/buffer_manager.hpp" #include "duckdb/storage/storage_manager.hpp" diff --git a/src/execution/operator/set/physical_recursive_cte.cpp b/src/execution/operator/set/physical_recursive_cte.cpp index b3a7c863e7db..1146200c8402 100644 --- a/src/execution/operator/set/physical_recursive_cte.cpp +++ b/src/execution/operator/set/physical_recursive_cte.cpp @@ -191,6 +191,7 @@ void PhysicalRecursiveCTE::BuildPipelines(Pipeline ¤t, MetaPipeline &meta_ // the RHS is the recursive pipeline recursive_meta_pipeline = make_shared(executor, state, this); + recursive_meta_pipeline->SetRecursiveCTE(); recursive_meta_pipeline->Build(children[1].get()); } diff --git a/src/include/duckdb/parallel/meta_pipeline.hpp b/src/include/duckdb/parallel/meta_pipeline.hpp index 642a912bd539..e1b754dad653 100644 --- a/src/include/duckdb/parallel/meta_pipeline.hpp +++ b/src/include/duckdb/parallel/meta_pipeline.hpp @@ -47,6 +47,8 @@ class MetaPipeline : public std::enable_shared_from_this { const vector *GetDependencies(Pipeline *dependant) const; //! Whether this MetaPipeline has a recursive CTE bool HasRecursiveCTE() const; + //! Set the flag that this MetaPipeline is a recursive CTE pipeline + void SetRecursiveCTE(); //! Assign a batch index to the given pipeline void AssignNextBatchIndex(Pipeline *pipeline); //! Let 'dependant' depend on all pipeline that were created since 'start', @@ -80,8 +82,8 @@ class MetaPipeline : public std::enable_shared_from_this { PipelineBuildState &state; //! The sink of all pipelines within this MetaPipeline PhysicalOperator *sink; - //! The recursive CTE of this MetaPipeline (if any) - PhysicalRecursiveCTE *recursive_cte = nullptr; + //! Whether this MetaPipeline is a the recursive pipeline of a recursive CTE + bool recursive_cte; //! All pipelines with a different source, but the same sink vector> pipelines; //! The pipelines that must finish before the MetaPipeline is finished diff --git a/src/parallel/executor.cpp b/src/parallel/executor.cpp index 822a6ea5e2c2..c1b01277ba21 100644 --- a/src/parallel/executor.cpp +++ b/src/parallel/executor.cpp @@ -2,9 +2,11 @@ #include "duckdb/execution/execution_context.hpp" #include "duckdb/execution/operator/helper/physical_result_collector.hpp" +#include "duckdb/execution/operator/set/physical_recursive_cte.hpp" #include "duckdb/execution/physical_operator.hpp" #include "duckdb/main/client_context.hpp" #include "duckdb/main/client_data.hpp" +#include "duckdb/parallel/meta_pipeline.hpp" #include "duckdb/parallel/pipeline_complete_event.hpp" #include "duckdb/parallel/pipeline_event.hpp" #include "duckdb/parallel/pipeline_executor.hpp" diff --git a/src/parallel/meta_pipeline.cpp b/src/parallel/meta_pipeline.cpp index 779bf368f3ea..2ed724be70a5 100644 --- a/src/parallel/meta_pipeline.cpp +++ b/src/parallel/meta_pipeline.cpp @@ -6,11 +6,8 @@ namespace duckdb { MetaPipeline::MetaPipeline(Executor &executor_p, PipelineBuildState &state_p, PhysicalOperator *sink_p) - : executor(executor_p), state(state_p), sink(sink_p), next_batch_index(0) { + : executor(executor_p), state(state_p), sink(sink_p), recursive_cte(false), next_batch_index(0) { CreatePipeline(); - if (sink_p && sink_p->type == PhysicalOperatorType::RECURSIVE_CTE) { - recursive_cte = (PhysicalRecursiveCTE *)sink; - } } Executor &MetaPipeline::GetExecutor() const { @@ -59,7 +56,11 @@ const vector *MetaPipeline::GetDependencies(Pipeline *dependant) con } bool MetaPipeline::HasRecursiveCTE() const { - return recursive_cte != nullptr; + return recursive_cte; +} + +void MetaPipeline::SetRecursiveCTE() { + recursive_cte = true; } void MetaPipeline::AssignNextBatchIndex(Pipeline *pipeline) { @@ -88,9 +89,7 @@ MetaPipeline *MetaPipeline::CreateChildMetaPipeline(Pipeline ¤t, PhysicalO // child MetaPipeline must finish completely before this MetaPipeline can start current.AddDependency(child_meta_pipeline->GetBasePipeline()); // child meta pipeline is part of the recursive CTE too - if (HasRecursiveCTE()) { - child_meta_pipeline->recursive_cte = recursive_cte; - } + child_meta_pipeline->recursive_cte = recursive_cte; return child_meta_pipeline; } From 34a0ef5683bafd66805c39ea842720af5de609d1 Mon Sep 17 00:00:00 2001 From: Laurens Kuiper Date: Tue, 1 Nov 2022 15:23:13 +0100 Subject: [PATCH 42/49] trying to please the CI --- src/parallel/executor.cpp | 4 ++-- src/parallel/pipeline.cpp | 1 - src/parallel/pipeline_initialize_event.cpp | 2 +- .../intraquery/parallel_materialization.test_slow | 6 +++--- 4 files changed, 6 insertions(+), 7 deletions(-) diff --git a/src/parallel/executor.cpp b/src/parallel/executor.cpp index c1b01277ba21..28033ee2fedf 100644 --- a/src/parallel/executor.cpp +++ b/src/parallel/executor.cpp @@ -109,11 +109,11 @@ void Executor::SchedulePipeline(const shared_ptr &meta_pipeline, S pipeline_stack.pipeline_finish_event->AddDependency(*pipeline_stack.pipeline_event); // add pipeline stack to event map - event_map.insert(make_pair(pipeline.get(), move(pipeline_stack))); + event_map.insert(make_pair(pipeline.get(), pipeline_stack)); } // add base stack to the event data too - event_map.insert(make_pair(base_pipeline.get(), move(base_stack))); + event_map.insert(make_pair(base_pipeline.get(), base_stack)); // set up the dependencies within this MetaPipeline for (auto &pipeline : pipelines) { diff --git a/src/parallel/pipeline.cpp b/src/parallel/pipeline.cpp index d160a18b4b87..b9ab74d9f1bc 100644 --- a/src/parallel/pipeline.cpp +++ b/src/parallel/pipeline.cpp @@ -118,7 +118,6 @@ bool Pipeline::IsOrderDependent() const { void Pipeline::Schedule(shared_ptr &event) { D_ASSERT(ready); D_ASSERT(sink); - Reset(); if (!ScheduleParallel(event)) { // could not parallelize this pipeline: push a sequential task instead ScheduleSequentialTask(event); diff --git a/src/parallel/pipeline_initialize_event.cpp b/src/parallel/pipeline_initialize_event.cpp index cc8d5db44021..2f305b38de68 100644 --- a/src/parallel/pipeline_initialize_event.cpp +++ b/src/parallel/pipeline_initialize_event.cpp @@ -19,7 +19,7 @@ class PipelineInitializeTask : public ExecutorTask { public: TaskExecutionResult ExecuteTask(TaskExecutionMode mode) override { - pipeline.ResetSink(); + pipeline.Reset(); event->FinishTask(); return TaskExecutionResult::TASK_FINISHED; } diff --git a/test/sql/parallelism/intraquery/parallel_materialization.test_slow b/test/sql/parallelism/intraquery/parallel_materialization.test_slow index 34704ebfe5ba..b609d1effa68 100644 --- a/test/sql/parallelism/intraquery/parallel_materialization.test_slow +++ b/test/sql/parallelism/intraquery/parallel_materialization.test_slow @@ -55,13 +55,13 @@ SELECT * FROM integers WHERE i > 337 AND i < 340 339 # IN-clause -query I +query I sort SELECT * FROM integers WHERE i IN (337, 948247, 1779793, 4779793, 99999999999999) ---- -337 -948247 1779793 +337 4779793 +948247 # more complex where clause query I From 148f699c2aa04d6ccadbbc94a3165746d7908c69 Mon Sep 17 00:00:00 2001 From: Laurens Kuiper Date: Wed, 2 Nov 2022 09:01:16 +0100 Subject: [PATCH 43/49] still trying to dodge threadsan data races --- src/common/types/data_chunk.cpp | 2 +- src/parallel/pipeline.cpp | 4 ++-- 2 files changed, 3 insertions(+), 3 deletions(-) diff --git a/src/common/types/data_chunk.cpp b/src/common/types/data_chunk.cpp index 508d775e213e..e5e2d6eecd47 100644 --- a/src/common/types/data_chunk.cpp +++ b/src/common/types/data_chunk.cpp @@ -107,8 +107,8 @@ bool DataChunk::AllConstant() const { void DataChunk::Reference(DataChunk &chunk) { D_ASSERT(chunk.ColumnCount() <= ColumnCount()); - SetCardinality(chunk); SetCapacity(chunk); + SetCardinality(chunk); for (idx_t i = 0; i < chunk.ColumnCount(); i++) { data[i].Reference(chunk.data[i]); } diff --git a/src/parallel/pipeline.cpp b/src/parallel/pipeline.cpp index 48747d12f8f1..65e97c9da51f 100644 --- a/src/parallel/pipeline.cpp +++ b/src/parallel/pipeline.cpp @@ -147,7 +147,7 @@ bool Pipeline::LaunchScanTasks(shared_ptr &event, idx_t max_threads) { } void Pipeline::ResetSink() { - if (sink && !sink->sink_state) { + if (sink) { lock_guard guard(sink->lock); if (!sink->sink_state) { sink->sink_state = sink->GetGlobalSinkState(GetClientContext()); @@ -158,7 +158,7 @@ void Pipeline::ResetSink() { void Pipeline::Reset() { ResetSink(); for (auto &op : operators) { - if (op && !op->op_state) { + if (op) { lock_guard guard(op->lock); if (!op->op_state) { op->op_state = op->GetGlobalOperatorState(GetClientContext()); From 691c0cf8365de612afa55d559ae58bf9607c93cd Mon Sep 17 00:00:00 2001 From: Laurens Kuiper Date: Wed, 2 Nov 2022 13:17:36 +0100 Subject: [PATCH 44/49] Add lock to RandomEngine to make RandomInitLocalState thread-safe --- src/function/scalar/math/random.cpp | 1 + src/include/duckdb/common/random_engine.hpp | 2 ++ 2 files changed, 3 insertions(+) diff --git a/src/function/scalar/math/random.cpp b/src/function/scalar/math/random.cpp index 2f3a2a3ed458..53f14bc55abf 100644 --- a/src/function/scalar/math/random.cpp +++ b/src/function/scalar/math/random.cpp @@ -51,6 +51,7 @@ static unique_ptr RandomInitLocalState(const BoundFunctionEx FunctionData *bind_data) { auto &info = (RandomBindData &)*bind_data; auto &random_engine = RandomEngine::Get(info.context); + lock_guard guard(random_engine.lock); return make_unique(random_engine.NextRandomInteger()); } diff --git a/src/include/duckdb/common/random_engine.hpp b/src/include/duckdb/common/random_engine.hpp index b522b0efe724..93e39d201d51 100644 --- a/src/include/duckdb/common/random_engine.hpp +++ b/src/include/duckdb/common/random_engine.hpp @@ -32,6 +32,8 @@ struct RandomEngine { static RandomEngine &Get(ClientContext &context); + mutex lock; + private: unique_ptr random_state; }; From e87e469898a9327f24de1a465ec0b49fa6a95c7c Mon Sep 17 00:00:00 2001 From: Laurens Kuiper Date: Wed, 2 Nov 2022 15:50:17 +0100 Subject: [PATCH 45/49] add missing include --- src/include/duckdb/common/random_engine.hpp | 2 ++ 1 file changed, 2 insertions(+) diff --git a/src/include/duckdb/common/random_engine.hpp b/src/include/duckdb/common/random_engine.hpp index 93e39d201d51..1185dbcb3b3f 100644 --- a/src/include/duckdb/common/random_engine.hpp +++ b/src/include/duckdb/common/random_engine.hpp @@ -10,6 +10,8 @@ #include "duckdb/common/common.hpp" #include "duckdb/common/limits.hpp" +#include "duckdb/common/mutex.hpp" + #include namespace duckdb { From 0829e3530d290f505f12ff4b7523518696b20349 Mon Sep 17 00:00:00 2001 From: Laurens Kuiper Date: Fri, 4 Nov 2022 14:19:06 +0100 Subject: [PATCH 46/49] reset source in main thread for R --- src/execution/operator/set/physical_recursive_cte.cpp | 2 +- src/include/duckdb/parallel/pipeline.hpp | 2 +- src/parallel/executor.cpp | 7 +++++++ src/parallel/pipeline.cpp | 10 +++++++--- 4 files changed, 16 insertions(+), 5 deletions(-) diff --git a/src/execution/operator/set/physical_recursive_cte.cpp b/src/execution/operator/set/physical_recursive_cte.cpp index 1146200c8402..8f7cdc2e3309 100644 --- a/src/execution/operator/set/physical_recursive_cte.cpp +++ b/src/execution/operator/set/physical_recursive_cte.cpp @@ -138,7 +138,7 @@ void PhysicalRecursiveCTE::ExecuteRecursivePipelines(ExecutionContext &context) op->op_state = op->GetGlobalOperatorState(context.client); } } - pipeline->Reset(); + pipeline->ResetSource(true); } // get the MetaPipelines in the recursive_meta_pipeline and reschedule them diff --git a/src/include/duckdb/parallel/pipeline.hpp b/src/include/duckdb/parallel/pipeline.hpp index a4f6d60ade24..6642b63cc738 100644 --- a/src/include/duckdb/parallel/pipeline.hpp +++ b/src/include/duckdb/parallel/pipeline.hpp @@ -63,7 +63,7 @@ class Pipeline : public std::enable_shared_from_this { void Ready(); void Reset(); void ResetSink(); - void ResetSource(); + void ResetSource(bool force); void Schedule(shared_ptr &event); //! Finalize this pipeline diff --git a/src/parallel/executor.cpp b/src/parallel/executor.cpp index 28033ee2fedf..b3860c251bc5 100644 --- a/src/parallel/executor.cpp +++ b/src/parallel/executor.cpp @@ -126,6 +126,13 @@ void Executor::SchedulePipeline(const shared_ptr &meta_pipeline, S auto &dependency_stack = event_map[dependency]; pipeline_stack.pipeline_event->AddDependency(*dependency_stack.pipeline_event); } + + auto source = pipeline->GetSource(); + if (source->type == PhysicalOperatorType::TABLE_SCAN) { + // we have to reset the source here (in the main thread), because some of our clients (looking at you, R) + // do not like it when threads other than the main thread call into R, for e.g., arrow scans + pipeline->ResetSource(true); + } } } diff --git a/src/parallel/pipeline.cpp b/src/parallel/pipeline.cpp index 65e97c9da51f..45f12a0a302f 100644 --- a/src/parallel/pipeline.cpp +++ b/src/parallel/pipeline.cpp @@ -165,12 +165,16 @@ void Pipeline::Reset() { } } } - ResetSource(); + ResetSource(false); + // we no longer reset source here because this function is no longer guaranteed to be called by the main thread + // source reset needs to be called by the main thread because resetting a source may call into clients like R initialized = true; } -void Pipeline::ResetSource() { - source_state = source->GetGlobalSourceState(GetClientContext()); +void Pipeline::ResetSource(bool force) { + if (force || !source_state) { + source_state = source->GetGlobalSourceState(GetClientContext()); + } } void Pipeline::Ready() { From 1ddc86d2c0e1117fa464d8a3ac5a7a74cc7600f5 Mon Sep 17 00:00:00 2001 From: Laurens Kuiper Date: Fri, 4 Nov 2022 14:42:15 +0100 Subject: [PATCH 47/49] implement PR feedback and fix skipped tests --- src/common/types/column_data_allocator.cpp | 14 +++++++------ src/execution/join_hashtable.cpp | 2 -- .../common/types/column_data_allocator.hpp | 1 + .../statistics/statistics_setop.test | 6 ------ .../cte/recursive_cte_complex_pipelines.test | 8 -------- test/sql/setops/test_union_all_by_name.test | 20 ++++++++----------- 6 files changed, 17 insertions(+), 34 deletions(-) diff --git a/src/common/types/column_data_allocator.cpp b/src/common/types/column_data_allocator.cpp index 7fab5d0cb553..973bebda0066 100644 --- a/src/common/types/column_data_allocator.cpp +++ b/src/common/types/column_data_allocator.cpp @@ -28,17 +28,19 @@ ColumnDataAllocator::ColumnDataAllocator(ClientContext &context, ColumnDataAlloc } } +BufferHandle ColumnDataAllocator::PinInternal(uint32_t block_id) { + auto &block_handle = blocks[block_id].handle; + auto result = alloc.buffer_manager->Pin(block_handle); + return result; +} + BufferHandle ColumnDataAllocator::Pin(uint32_t block_id) { D_ASSERT(type == ColumnDataAllocatorType::BUFFER_MANAGER_ALLOCATOR); if (shared) { lock_guard guard(lock); - auto &block_handle = blocks[block_id].handle; - auto result = alloc.buffer_manager->Pin(block_handle); - return result; + return PinInternal(block_id); } else { - auto &block_handle = blocks[block_id].handle; - auto result = alloc.buffer_manager->Pin(block_handle); - return result; + return PinInternal(block_id); } } diff --git a/src/execution/join_hashtable.cpp b/src/execution/join_hashtable.cpp index a1159feffe33..d9951a3c9851 100644 --- a/src/execution/join_hashtable.cpp +++ b/src/execution/join_hashtable.cpp @@ -9,8 +9,6 @@ #include "duckdb/main/client_context.hpp" #include "duckdb/storage/buffer_manager.hpp" -#include - namespace duckdb { using ValidityBytes = JoinHashTable::ValidityBytes; diff --git a/src/include/duckdb/common/types/column_data_allocator.hpp b/src/include/duckdb/common/types/column_data_allocator.hpp index 6d55bad6d695..408d398f1b93 100644 --- a/src/include/duckdb/common/types/column_data_allocator.hpp +++ b/src/include/duckdb/common/types/column_data_allocator.hpp @@ -57,6 +57,7 @@ class ColumnDataAllocator { private: void AllocateEmptyBlock(idx_t size); void AllocateBlock(); + BufferHandle PinInternal(uint32_t block_id); BufferHandle Pin(uint32_t block_id); bool HasBlocks() const { diff --git a/test/optimizer/statistics/statistics_setop.test b/test/optimizer/statistics/statistics_setop.test index c8fe6483d47c..4420ef6d9149 100644 --- a/test/optimizer/statistics/statistics_setop.test +++ b/test/optimizer/statistics/statistics_setop.test @@ -11,8 +11,6 @@ CREATE TABLE integers2 AS SELECT * FROM (VALUES (4), (5), (6)) tbl(i); statement ok CREATE TABLE integers3 AS SELECT * FROM (VALUES (4), (5), (NULL)) tbl(i); -mode skip - statement ok PRAGMA enable_verification; @@ -107,8 +105,6 @@ SELECT i IS NULL FROM (SELECT * FROM integers UNION ALL SELECT * FROM integers2) 0 0 -mode unskip - query I SELECT i IS NULL FROM (SELECT * FROM integers UNION ALL SELECT * FROM integers2 UNION ALL SELECT * FROM integers3) tbl(i); ---- @@ -122,8 +118,6 @@ SELECT i IS NULL FROM (SELECT * FROM integers UNION ALL SELECT * FROM integers2 0 1 -mode skip - query I SELECT i=7 FROM (SELECT * FROM integers2 EXCEPT SELECT * FROM integers3) tbl(i); ---- diff --git a/test/sql/cte/recursive_cte_complex_pipelines.test b/test/sql/cte/recursive_cte_complex_pipelines.test index 260fc45704ac..9fbe836958a7 100644 --- a/test/sql/cte/recursive_cte_complex_pipelines.test +++ b/test/sql/cte/recursive_cte_complex_pipelines.test @@ -2,8 +2,6 @@ # description: Recursive CTEs with complex pipelines # group: [cte] -mode skip - statement ok PRAGMA enable_verification @@ -45,13 +43,9 @@ SELECT * FROM t ORDER BY 1; 81 243 -mode unskip - statement ok CREATE TABLE a AS SELECT * FROM range(100) t1(i) -mode skip - # recursive CTE with aggregates query I WITH RECURSIVE t AS @@ -70,8 +64,6 @@ SELECT * FROM t ORDER BY 1 NULLS LAST; 1000000 NULL -mode unskip - # the same but with a hash join query I WITH RECURSIVE t AS diff --git a/test/sql/setops/test_union_all_by_name.test b/test/sql/setops/test_union_all_by_name.test index 95b1700c1ad3..b96a361f672f 100644 --- a/test/sql/setops/test_union_all_by_name.test +++ b/test/sql/setops/test_union_all_by_name.test @@ -17,8 +17,6 @@ CREATE TABLE t2 (y INT, z INT); statement ok INSERT INTO t2 VALUES (2, 2), (4, 4); -mode skip - query III SELECT x, y FROM t1 UNION ALL BY NAME SELECT y, z FROM t2; ---- @@ -83,8 +81,6 @@ NULL 4 3 NULL 3 NULL -mode unskip - ########## test limit query III @@ -96,8 +92,6 @@ NULL 3 NULL NULL 4 NULL NULL 5 NULL -mode skip - ########## test order by query III @@ -147,13 +141,13 @@ NULL 1 1 NULL NULL NULL 2 2 NULL NULL 4 4 - -query IIII +# need to sort because our hash-based union does not preserve order +query IIII sort SELECT 1, 2 FROM t1 UNION SELECT x, y FROM t1 UNION ALL BY NAME SELECT y, z FROM t2; ---- +1 1 NULL NULL 1 2 NULL NULL 3 3 NULL NULL -1 1 NULL NULL NULL NULL 2 2 NULL NULL 4 4 @@ -173,11 +167,12 @@ NULL 2 2 statement error SELECT x, y FROM t1 UNION ALL BY NAME SELECT y, z FROM t2 EXCEPT SELECT 2, 2 FROM t1; -query III +# have to sort here too because EXCEPT does not preserve order +query III sort (SELECT x, y FROM t1 UNION ALL BY NAME SELECT y, z FROM t2) EXCEPT SELECT NULL, 2, 2 FROM t1; ---- -3 3 NULL 1 1 NULL +3 3 NULL NULL 4 4 @@ -213,7 +208,8 @@ SELECT DISTINCT ON(x) x FROM (SELECT 1 as x UNION ALL BY NAME SELECT '1' as x); ---- 1 -query I +# have to sort here too because distinct on is hash-based, does not preserve order +query I sort SELECT DISTINCT ON(x) x FROM (SELECT 1 as x UNION ALL BY NAME SELECT 1.1 as x); ---- 1 From cf5744d600df6f8d751544625a44182ad0eb49e7 Mon Sep 17 00:00:00 2001 From: Laurens Kuiper Date: Mon, 7 Nov 2022 09:35:07 +0100 Subject: [PATCH 48/49] init global source state before calling continue in loop ... oops --- src/parallel/executor.cpp | 14 +++++++------- 1 file changed, 7 insertions(+), 7 deletions(-) diff --git a/src/parallel/executor.cpp b/src/parallel/executor.cpp index b3860c251bc5..92af3478fd3a 100644 --- a/src/parallel/executor.cpp +++ b/src/parallel/executor.cpp @@ -117,6 +117,13 @@ void Executor::SchedulePipeline(const shared_ptr &meta_pipeline, S // set up the dependencies within this MetaPipeline for (auto &pipeline : pipelines) { + auto source = pipeline->GetSource(); + if (source->type == PhysicalOperatorType::TABLE_SCAN) { + // we have to reset the source here (in the main thread), because some of our clients (looking at you, R) + // do not like it when threads other than the main thread call into R, for e.g., arrow scans + pipeline->ResetSource(true); + } + auto dependencies = meta_pipeline->GetDependencies(pipeline.get()); if (!dependencies) { continue; @@ -126,13 +133,6 @@ void Executor::SchedulePipeline(const shared_ptr &meta_pipeline, S auto &dependency_stack = event_map[dependency]; pipeline_stack.pipeline_event->AddDependency(*dependency_stack.pipeline_event); } - - auto source = pipeline->GetSource(); - if (source->type == PhysicalOperatorType::TABLE_SCAN) { - // we have to reset the source here (in the main thread), because some of our clients (looking at you, R) - // do not like it when threads other than the main thread call into R, for e.g., arrow scans - pipeline->ResetSource(true); - } } } From 9090797259592d1c5c5b6b6d20f0faf3dc7d39a8 Mon Sep 17 00:00:00 2001 From: Laurens Kuiper Date: Mon, 7 Nov 2022 13:19:20 +0100 Subject: [PATCH 49/49] add missing return statement --- src/common/types/column_data_allocator.cpp | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/src/common/types/column_data_allocator.cpp b/src/common/types/column_data_allocator.cpp index 4f57236f7a88..185d2284cfe4 100644 --- a/src/common/types/column_data_allocator.cpp +++ b/src/common/types/column_data_allocator.cpp @@ -32,13 +32,13 @@ BufferHandle ColumnDataAllocator::Pin(uint32_t block_id) { D_ASSERT(type == ColumnDataAllocatorType::BUFFER_MANAGER_ALLOCATOR); shared_ptr *block_handle; if (shared) { - // need to grab handle from vector within a lock else threadsan will complain + // need to grab handle from the vector within a lock else threadsan will complain lock_guard guard(lock); block_handle = &blocks[block_id].handle; } else { block_handle = &blocks[block_id].handle; } - alloc.buffer_manager->Pin(*block_handle); + return alloc.buffer_manager->Pin(*block_handle); } void ColumnDataAllocator::AllocateBlock() {