From f5a623d275d94559e58de3f42b9f2127cc73c670 Mon Sep 17 00:00:00 2001 From: dtenwolde Date: Fri, 15 Dec 2023 13:07:12 +0100 Subject: [PATCH 001/249] Created barebones path finding operator skeleton --- CMakeLists.txt | 18 ------------------ duckdb-pgq | 2 +- test/sql/path-finding/complex_matching.test | 1 + 3 files changed, 2 insertions(+), 19 deletions(-) diff --git a/CMakeLists.txt b/CMakeLists.txt index 48cc12d4..1d264bad 100644 --- a/CMakeLists.txt +++ b/CMakeLists.txt @@ -1,23 +1,6 @@ cmake_minimum_required(VERSION 2.8.12) # Set extension name here -#<<<<<<< HEAD -#set(TARGET_NAME duckpgq) -# -#set(EXTENSION_NAME ${TARGET_NAME}_extension) -#project(${TARGET_NAME}) -#set(CMAKE_CXX_STANDARD 11) -# -#include_directories(duckpgq/include) -#add_subdirectory(duckpgq/src) -# -#include_directories(../duckdb-pgq/third_party/libpg_query/include) -# -#add_library(${EXTENSION_NAME} STATIC ${EXTENSION_SOURCES}) -# -#set(PARAMETERS "-warnings") -#build_loadable_extension(${TARGET_NAME} ${PARAMETERS} ${EXTENSION_SOURCES}) -#======= set(TARGET_NAME duckpgq) set(CMAKE_CXX_STANDARD 11) # DuckDB's extension distribution supports vcpkg. As such, dependencies can be added in ./vcpkg.json and then @@ -41,7 +24,6 @@ build_loadable_extension(${TARGET_NAME} " " ${EXTENSION_SOURCES}) # Link OpenSSL in both the static library as the loadable extension target_link_libraries(${EXTENSION_NAME} OpenSSL::SSL OpenSSL::Crypto) target_link_libraries(${LOADABLE_EXTENSION_NAME} OpenSSL::SSL OpenSSL::Crypto) -#>>>>>>> template/main install( TARGETS ${EXTENSION_NAME} diff --git a/duckdb-pgq b/duckdb-pgq index f484fe08..5f8c4832 160000 --- a/duckdb-pgq +++ b/duckdb-pgq @@ -1 +1 @@ -Subproject commit f484fe0899e67fc29b6aec7ef1734925dca7ef45 +Subproject commit 5f8c48329da20df4736d613b3b24155b53e724bb diff --git a/test/sql/path-finding/complex_matching.test b/test/sql/path-finding/complex_matching.test index e2eb792d..3feb702c 100644 --- a/test/sql/path-finding/complex_matching.test +++ b/test/sql/path-finding/complex_matching.test @@ -49,6 +49,7 @@ EDGE TABLES ( LABEL replyOf ); +# https://github.com/cwida/duckpgq-extension/issues/62 query II -FROM GRAPH_TABLE (snb MATCH o = ANY SHORTEST (p:Person)-[w:knows]-> {1,3}(p2:Person)-[i:hasInterest]->(t:Tag) From c1d3af0ea1f1b0c1ac378105f9a2ad70b0522b20 Mon Sep 17 00:00:00 2001 From: dtenwolde Date: Mon, 18 Dec 2023 16:39:10 +0100 Subject: [PATCH 002/249] Creating path finding operator in extension module --- .../duckpgq/operators/path_finding_operator.hpp | 13 +++++++++++++ duckpgq/src/duckpgq/CMakeLists.txt | 1 + duckpgq/src/duckpgq/operators/CMakeLists.txt | 5 +++++ .../src/duckpgq/operators/path_finding_operator.cpp | 8 ++++++++ duckpgq/src/duckpgq_extension.cpp | 11 +++++++---- 5 files changed, 34 insertions(+), 4 deletions(-) create mode 100644 duckpgq/include/duckpgq/operators/path_finding_operator.hpp create mode 100644 duckpgq/src/duckpgq/operators/CMakeLists.txt create mode 100644 duckpgq/src/duckpgq/operators/path_finding_operator.cpp diff --git a/duckpgq/include/duckpgq/operators/path_finding_operator.hpp b/duckpgq/include/duckpgq/operators/path_finding_operator.hpp new file mode 100644 index 00000000..8b31b432 --- /dev/null +++ b/duckpgq/include/duckpgq/operators/path_finding_operator.hpp @@ -0,0 +1,13 @@ +#pragma once +#include + +namespace duckdb { + + class PathFindingOperator : public LogicalExtensionOperator { + + + + + }; + +} diff --git a/duckpgq/src/duckpgq/CMakeLists.txt b/duckpgq/src/duckpgq/CMakeLists.txt index 9584fb65..d4bc666f 100644 --- a/duckpgq/src/duckpgq/CMakeLists.txt +++ b/duckpgq/src/duckpgq/CMakeLists.txt @@ -1,4 +1,5 @@ add_subdirectory(functions) +add_subdirectory(operators) set(EXTENSION_SOURCES ${CMAKE_CURRENT_SOURCE_DIR}/common.cpp diff --git a/duckpgq/src/duckpgq/operators/CMakeLists.txt b/duckpgq/src/duckpgq/operators/CMakeLists.txt new file mode 100644 index 00000000..3bd9434e --- /dev/null +++ b/duckpgq/src/duckpgq/operators/CMakeLists.txt @@ -0,0 +1,5 @@ +set(EXTENSION_SOURCES + ${CMAKE_CURRENT_SOURCE_DIR}/path_finding_operator.cpp + ${EXTENSION_SOURCES} + PARENT_SCOPE +) \ No newline at end of file diff --git a/duckpgq/src/duckpgq/operators/path_finding_operator.cpp b/duckpgq/src/duckpgq/operators/path_finding_operator.cpp new file mode 100644 index 00000000..b336d0ca --- /dev/null +++ b/duckpgq/src/duckpgq/operators/path_finding_operator.cpp @@ -0,0 +1,8 @@ + +#include "duckpgq/operators/path_finding_operator.hpp" +#include + +namespace duckdb { + + +} \ No newline at end of file diff --git a/duckpgq/src/duckpgq_extension.cpp b/duckpgq/src/duckpgq_extension.cpp index 5cc1404d..e144d788 100644 --- a/duckpgq/src/duckpgq_extension.cpp +++ b/duckpgq/src/duckpgq_extension.cpp @@ -41,15 +41,15 @@ static void LoadInternal(DatabaseInstance &instance) { auto &catalog = Catalog::GetSystemCatalog(*con.context); - PGQMatchFunction match_pg_function; + const PGQMatchFunction match_pg_function; CreateTableFunctionInfo match_pg_info(match_pg_function); catalog.CreateTableFunction(*con.context, match_pg_info); - CreatePropertyGraphFunction create_pg_function; + const CreatePropertyGraphFunction create_pg_function; CreateTableFunctionInfo create_pg_info(create_pg_function); catalog.CreateTableFunction(*con.context, create_pg_info); - DropPropertyGraphFunction drop_pg_function; + const DropPropertyGraphFunction drop_pg_function; CreateTableFunctionInfo drop_pg_info(drop_pg_function); catalog.CreateTableFunction(*con.context, drop_pg_info); @@ -73,7 +73,7 @@ void DuckpgqExtension::Load(DuckDB &db) { LoadInternal(*db.instance); } ParserExtensionParseResult duckpgq_parse(ParserExtensionInfo *info, const std::string &query) { - auto parse_info = (DuckPGQParserExtensionInfo &)(info); + auto parse_info = reinterpret_cast(info); Parser parser; parser.ParseQuery((query[0] == '-') ? query.substr(1, query.length()) : query); @@ -127,6 +127,9 @@ duckpgq_plan(ParserExtensionInfo *, ClientContext &context, auto select_node = dynamic_cast(select_statement->node.get()); auto from_table_function = dynamic_cast(select_node->from_table.get()); + if (!from_table_function) { + throw Exception("Use duckpgq_bind instead"); + } auto function = dynamic_cast(from_table_function->function.get()); if (function->function_name == "duckpgq_match") { From 0e3eeb641eb4323da7c9770c6acb51e774cf7909 Mon Sep 17 00:00:00 2001 From: dtenwolde Date: Mon, 18 Dec 2023 16:56:46 +0100 Subject: [PATCH 003/249] First implementation of path finding operator --- .../duckpgq/operators/path_finding_operator.hpp | 16 ++++++++++++++++ .../duckpgq/operators/path_finding_operator.cpp | 4 +++- duckpgq/src/duckpgq_extension.cpp | 3 +++ 3 files changed, 22 insertions(+), 1 deletion(-) diff --git a/duckpgq/include/duckpgq/operators/path_finding_operator.hpp b/duckpgq/include/duckpgq/operators/path_finding_operator.hpp index 8b31b432..29b5b88d 100644 --- a/duckpgq/include/duckpgq/operators/path_finding_operator.hpp +++ b/duckpgq/include/duckpgq/operators/path_finding_operator.hpp @@ -4,6 +4,22 @@ namespace duckdb { class PathFindingOperator : public LogicalExtensionOperator { + public: + explicit PathFindingOperator(unique_ptr plan) { + children.emplace_back(std::move(plan)); + } + + void Serialize(Serializer &serializer) const override { + throw InternalException("Path Finding Operator should not be serialized"); + } + + unique_ptr CreatePlan(ClientContext &context, PhysicalPlanGenerator &generator) override; + + vector GetColumnBindings() override; + + std::string GetName() const override { + return "PATH_FINDING"; + } diff --git a/duckpgq/src/duckpgq/operators/path_finding_operator.cpp b/duckpgq/src/duckpgq/operators/path_finding_operator.cpp index b336d0ca..38027494 100644 --- a/duckpgq/src/duckpgq/operators/path_finding_operator.cpp +++ b/duckpgq/src/duckpgq/operators/path_finding_operator.cpp @@ -3,6 +3,8 @@ #include namespace duckdb { - + unique_ptr PathFindingOperator::CreatePlan(ClientContext &context, PhysicalPlanGenerator &generator) { + return unique_ptr(); // TODO IMPLEMENT ME + } } \ No newline at end of file diff --git a/duckpgq/src/duckpgq_extension.cpp b/duckpgq/src/duckpgq_extension.cpp index e144d788..006f2b12 100644 --- a/duckpgq/src/duckpgq_extension.cpp +++ b/duckpgq/src/duckpgq_extension.cpp @@ -1,6 +1,9 @@ #define DUCKDB_EXTENSION_MAIN #include "duckpgq_extension.hpp" + +#include + #include "duckdb/function/scalar_function.hpp" #include "duckpgq/duckpgq_functions.hpp" From c32e0b6d1cbbdb05d0d3d868a6366ec4c20fa974 Mon Sep 17 00:00:00 2001 From: dtenwolde Date: Tue, 19 Dec 2023 11:22:00 +0100 Subject: [PATCH 004/249] Moved physical operator to extension --- duckdb-pgq | 2 +- .../operators/path_finding_operator.hpp | 5 +- .../operators/physical_path_finding.hpp | 72 ++ duckpgq/src/duckpgq/operators/CMakeLists.txt | 2 + .../operators/path_finding_operator.cpp | 10 + .../operators/physical_path_finding.cpp | 1008 +++++++++++++++++ 6 files changed, 1094 insertions(+), 5 deletions(-) create mode 100644 duckpgq/include/duckpgq/operators/physical_path_finding.hpp create mode 100644 duckpgq/src/duckpgq/operators/physical_path_finding.cpp diff --git a/duckdb-pgq b/duckdb-pgq index 5f8c4832..925281d0 160000 --- a/duckdb-pgq +++ b/duckdb-pgq @@ -1 +1 @@ -Subproject commit 5f8c48329da20df4736d613b3b24155b53e724bb +Subproject commit 925281d061c48bf620de8171282571fcbad32e36 diff --git a/duckpgq/include/duckpgq/operators/path_finding_operator.hpp b/duckpgq/include/duckpgq/operators/path_finding_operator.hpp index 29b5b88d..d7152026 100644 --- a/duckpgq/include/duckpgq/operators/path_finding_operator.hpp +++ b/duckpgq/include/duckpgq/operators/path_finding_operator.hpp @@ -20,10 +20,7 @@ namespace duckdb { std::string GetName() const override { return "PATH_FINDING"; } - - - - }; + } diff --git a/duckpgq/include/duckpgq/operators/physical_path_finding.hpp b/duckpgq/include/duckpgq/operators/physical_path_finding.hpp new file mode 100644 index 00000000..22ba8590 --- /dev/null +++ b/duckpgq/include/duckpgq/operators/physical_path_finding.hpp @@ -0,0 +1,72 @@ +//===----------------------------------------------------------------------===// +// DuckDB +// +// duckdb/execution/operator/join/physical_piecewise_merge_join.hpp +// +// +//===----------------------------------------------------------------------===// + +#pragma once + +#include "duckdb/execution/operator/join/physical_range_join.hpp" +#include "duckdb/planner/bound_result_modifier.hpp" + +namespace duckdb { + +//! PhysicalIEJoin represents a two inequality range join between +//! two tables + class PhysicalPathFinding : public CachingPhysicalOperator { + public: + static constexpr const PhysicalOperatorType TYPE = PhysicalOperatorType::PATH_FINDING; + public: + PhysicalPathFinding(LogicalComparisonJoin &op, unique_ptr left, + unique_ptr right); + + // vector join_key_types; + // vector> lhs_orders; + // vector> rhs_orders; + + public: + // CachingOperator Interface + OperatorResultType ExecuteInternal(ExecutionContext &context, DataChunk &input, DataChunk &chunk, + GlobalOperatorState &gstate, OperatorState &state) const override; + + public: + // Source interface + unique_ptr GetLocalSourceState(ExecutionContext &context, + GlobalSourceState &gstate) const override; + unique_ptr GetGlobalSourceState(ClientContext &context) const override; + SourceResultType GetData(ExecutionContext &context, DataChunk &chunk, OperatorSourceInput &input) const override; + + bool IsSource() const override { + return true; + } + bool ParallelSource() const override { + return true; + } + + public: + // Sink Interface + unique_ptr GetGlobalSinkState(ClientContext &context) const override; + unique_ptr GetLocalSinkState(ExecutionContext &context) const override; + SinkResultType Sink(ExecutionContext &context, DataChunk &chunk, OperatorSinkInput &input) const override; + SinkCombineResultType Combine(ExecutionContext &context, OperatorSinkCombineInput &input) const override; + SinkFinalizeType Finalize(Pipeline &pipeline, Event &event, ClientContext &context, + OperatorSinkFinalizeInput &input) const override; + + bool IsSink() const override { + return true; + } + bool ParallelSink() const override { + return true; + } + + public: + void BuildPipelines(Pipeline ¤t, MetaPipeline &meta_pipeline) override; + + private: + // resolve joins that can potentially output N*M elements (INNER, LEFT, FULL) + // void ResolveComplexJoin(ExecutionContext &context, DataChunk &result, LocalSourceState &state) const; + }; + +} // namespace duckdb diff --git a/duckpgq/src/duckpgq/operators/CMakeLists.txt b/duckpgq/src/duckpgq/operators/CMakeLists.txt index 3bd9434e..17f3d436 100644 --- a/duckpgq/src/duckpgq/operators/CMakeLists.txt +++ b/duckpgq/src/duckpgq/operators/CMakeLists.txt @@ -1,5 +1,7 @@ set(EXTENSION_SOURCES ${CMAKE_CURRENT_SOURCE_DIR}/path_finding_operator.cpp + ${CMAKE_CURRENT_SOURCE_DIR}/physical_path_finding.cpp + ${EXTENSION_SOURCES} PARENT_SCOPE ) \ No newline at end of file diff --git a/duckpgq/src/duckpgq/operators/path_finding_operator.cpp b/duckpgq/src/duckpgq/operators/path_finding_operator.cpp index 38027494..31add29b 100644 --- a/duckpgq/src/duckpgq/operators/path_finding_operator.cpp +++ b/duckpgq/src/duckpgq/operators/path_finding_operator.cpp @@ -7,4 +7,14 @@ namespace duckdb { return unique_ptr(); // TODO IMPLEMENT ME } + duckdb::unique_ptr CreatePlan(duckdb::ClientContext &, + duckdb::PhysicalPlanGenerator &generator) override { + auto result = duckdb::make_uniq_base(bridge_id, types, + estimated_cardinality); + D_ASSERT(children.size() == 2); + auto plan = generator.CreatePlan(std::move(children[0])); + result->children.emplace_back(std::move(plan)); + return result; + } + } \ No newline at end of file diff --git a/duckpgq/src/duckpgq/operators/physical_path_finding.cpp b/duckpgq/src/duckpgq/operators/physical_path_finding.cpp new file mode 100644 index 00000000..d7762891 --- /dev/null +++ b/duckpgq/src/duckpgq/operators/physical_path_finding.cpp @@ -0,0 +1,1008 @@ +#include "duckpgq/operators/physical_path_finding.hpp" + +#include "duckdb/common/sort/sort.hpp" +#include "duckdb/main/client_context.hpp" +#include "duckdb/parallel/event.hpp" +#include "duckdb/parallel/meta_pipeline.hpp" +#include "duckdb/parallel/thread_context.hpp" + +#include +#include + + +namespace duckdb { + + PhysicalPathFinding::PhysicalPathFinding(LogicalComparisonJoin &op, unique_ptr left, + unique_ptr right) + : CachingPhysicalOperator(type, op.types, estimated_cardinality) { + } + +//===--------------------------------------------------------------------===// +// Sink +//===--------------------------------------------------------------------===// + class PathFindingLocalState : public LocalSinkState { + public: + // TODO Add something to the local sink state + // using LocalSortedTable = PhysicalRangeJoin::LocalSortedTable; + + PathFindingLocalState(ClientContext &context, const PhysicalPathFinding &op, const idx_t child) + : context(context), op(op), child(child) { + } + ClientContext &context; + const PhysicalPathFinding &op; + const idx_t child; + //! The local sort state + // LocalSortedTable table; + }; + + class PathFindingGlobalState : public GlobalSinkState { + public: + using GlobalSortedTable = PhysicalRangeJoin::GlobalSortedTable; + + public: + PathFindingGlobalState(ClientContext &context, const PhysicalPathFinding &op) { + // tables.resize(2); + // RowLayout lhs_layout; + // lhs_layout.Initialize(op.children[0]->types); + // vector lhs_order; + // lhs_order.emplace_back(op.lhs_orders[0][0].Copy()); + // tables[0] = make_uniq(context, lhs_order, lhs_layout); + // + // RowLayout rhs_layout; + // rhs_layout.Initialize(op.children[1]->types); + // vector rhs_order; + // rhs_order.emplace_back(op.rhs_orders[0][0].Copy()); + // tables[1] = make_uniq(context, rhs_order, rhs_layout); + } + + PathFindingGlobalState(PathFindingGlobalState &prev) + : GlobalSinkState(prev) {//, tables(std::move(prev.tables)), child(prev.child + 1) { + } + + void Sink(DataChunk &input, PathFindingLocalState &lstate) { + // auto &table = *tables[child]; + // auto &global_sort_state = table.global_sort_state; + // auto &local_sort_state = lstate.table.local_sort_state; + // + // // Sink the data into the local sort state + // lstate.table.Sink(input, global_sort_state); + // + // // When sorting data reaches a certain size, we sort it + // if (local_sort_state.SizeInBytes() >= table.memory_per_thread) { + // local_sort_state.Sort(global_sort_state, true); + // } + } + + vector> tables; + // size_t child; + }; + + unique_ptr PhysicalPathFinding::GetGlobalSinkState(ClientContext &context) const { + D_ASSERT(!sink_state); + return make_uniq(context, *this); + } + + unique_ptr PhysicalPathFinding::GetLocalSinkState(ExecutionContext &context) const { + idx_t sink_child = 0; + if (sink_state) { + // const auto &ie_sink = sink_state->Cast(); + // sink_child = ie_sink.child; + } + return make_uniq(context.client, *this, sink_child); + } + + SinkResultType PhysicalPathFinding::Sink(ExecutionContext &context, DataChunk &chunk, OperatorSinkInput &input) const { + auto &gstate = input.global_state.Cast(); + auto &lstate = input.local_state.Cast(); + + gstate.Sink(chunk, lstate); + + return SinkResultType::NEED_MORE_INPUT; + } + + SinkCombineResultType PhysicalPathFinding::Combine(ExecutionContext &context, OperatorSinkCombineInput &input) const { + // auto &gstate = input.global_state.Cast(); + // auto &lstate = input.local_state.Cast(); + // gstate.tables[gstate.child]->Combine(lstate.table); + auto &client_profiler = QueryProfiler::Get(context.client); + + // context.thread.profiler.Flush(*this, lstate.table.executor, gstate.child ? "rhs_executor" : "lhs_executor", 1); + client_profiler.Flush(context.thread.profiler); + + return SinkCombineResultType::FINISHED; + } + +//===--------------------------------------------------------------------===// +// Finalize +//===--------------------------------------------------------------------===// + SinkFinalizeType PhysicalPathFinding::Finalize(Pipeline &pipeline, Event &event, ClientContext &context, + OperatorSinkFinalizeInput &input) const { + auto &gstate = input.global_state.Cast(); + // auto &table = *gstate.tables[gstate.child]; + // auto &global_sort_state = table.global_sort_state; + + // if ((gstate.child == 1 && IsRightOuterJoin(join_type)) || (gstate.child == 0 && IsLeftOuterJoin(join_type))) { + // // for FULL/LEFT/RIGHT OUTER JOIN, initialize found_match to false for every tuple + // table.IntializeMatches(); + // } + // if (gstate.child == 1 && global_sort_state.sorted_blocks.empty() && EmptyResultIfRHSIsEmpty()) { + // // Empty input! + // return SinkFinalizeType::NO_OUTPUT_POSSIBLE; + // } + + // Sort the current input child + // table.Finalize(pipeline, event); + + // Move to the next input child + // ++gstate.child; + + return SinkFinalizeType::READY; + } + +//===--------------------------------------------------------------------===// +// Operator +//===--------------------------------------------------------------------===// + OperatorResultType PhysicalPathFinding::ExecuteInternal(ExecutionContext &context, DataChunk &input, DataChunk &chunk, + GlobalOperatorState &gstate, OperatorState &state) const { + return OperatorResultType::FINISHED; + } + +//===--------------------------------------------------------------------===// +// Source +//===--------------------------------------------------------------------===// + // struct PathFindingUnion { + // using SortedTable = PhysicalRangeJoin::GlobalSortedTable; + // + // static idx_t AppendKey(SortedTable &table, ExpressionExecutor &executor, SortedTable &marked, int64_t increment, + // int64_t base, const idx_t block_idx); + // + // static void Sort(SortedTable &table) { + // auto &global_sort_state = table.global_sort_state; + // global_sort_state.PrepareMergePhase(); + // while (global_sort_state.sorted_blocks.size() > 1) { + // global_sort_state.InitializeMergeRound(); + // MergeSorter merge_sorter(global_sort_state, global_sort_state.buffer_manager); + // merge_sorter.PerformInMergeRound(); + // global_sort_state.CompleteMergeRound(true); + // } + // } + // + // template + // static vector ExtractColumn(SortedTable &table, idx_t col_idx) { + // vector result; + // result.reserve(table.count); + // + // auto &gstate = table.global_sort_state; + // auto &blocks = *gstate.sorted_blocks[0]->payload_data; + // PayloadScanner scanner(blocks, gstate, false); + // + // DataChunk payload; + // payload.Initialize(Allocator::DefaultAllocator(), gstate.payload_layout.GetTypes()); + // for (;;) { + // scanner.Scan(payload); + // const auto count = payload.size(); + // if (!count) { + // break; + // } + // + // const auto data_ptr = FlatVector::GetData(payload.data[col_idx]); + // result.insert(result.end(), data_ptr, data_ptr + count); + // } + // + // return result; + // } + // + // IEJoinUnion(ClientContext &context, const PhysicalIEJoin &op, SortedTable &t1, const idx_t b1, SortedTable &t2, + // const idx_t b2); + // + // idx_t SearchL1(idx_t pos); + // bool NextRow(); + // + // //! Inverted loop + // idx_t JoinComplexBlocks(SelectionVector &lsel, SelectionVector &rsel); + // + // //! L1 + // unique_ptr l1; + // //! L2 + // unique_ptr l2; + // + // //! Li + // vector li; + // //! P + // vector p; + // + // //! B + // vector bit_array; + // ValidityMask bit_mask; + // + // //! Bloom Filter + // static constexpr idx_t BLOOM_CHUNK_BITS = 1024; + // idx_t bloom_count; + // vector bloom_array; + // ValidityMask bloom_filter; + // + // //! Iteration state + // idx_t n; + // idx_t i; + // idx_t j; + // unique_ptr op1; + // unique_ptr off1; + // unique_ptr op2; + // unique_ptr off2; + // int64_t lrid; + // }; + + // idx_t IEJoinUnion::AppendKey(SortedTable &table, ExpressionExecutor &executor, SortedTable &marked, int64_t increment, + // int64_t base, const idx_t block_idx) { + // LocalSortState local_sort_state; + // local_sort_state.Initialize(marked.global_sort_state, marked.global_sort_state.buffer_manager); + // + // // Reading + // const auto valid = table.count - table.has_null; + // auto &gstate = table.global_sort_state; + // PayloadScanner scanner(gstate, block_idx); + // auto table_idx = block_idx * gstate.block_capacity; + // + // DataChunk scanned; + // scanned.Initialize(Allocator::DefaultAllocator(), scanner.GetPayloadTypes()); + // + // // Writing + // auto types = local_sort_state.sort_layout->logical_types; + // const idx_t payload_idx = types.size(); + // + // const auto &payload_types = local_sort_state.payload_layout->GetTypes(); + // types.insert(types.end(), payload_types.begin(), payload_types.end()); + // const idx_t rid_idx = types.size() - 1; + // + // DataChunk keys; + // DataChunk payload; + // keys.Initialize(Allocator::DefaultAllocator(), types); + // + // idx_t inserted = 0; + // for (auto rid = base; table_idx < valid;) { + // scanner.Scan(scanned); + // + // // NULLs are at the end, so stop when we reach them + // auto scan_count = scanned.size(); + // if (table_idx + scan_count > valid) { + // scan_count = valid - table_idx; + // scanned.SetCardinality(scan_count); + // } + // if (scan_count == 0) { + // break; + // } + // table_idx += scan_count; + // + // // Compute the input columns from the payload + // keys.Reset(); + // keys.Split(payload, rid_idx); + // executor.Execute(scanned, keys); + // + // // Mark the rid column + // payload.data[0].Sequence(rid, increment, scan_count); + // payload.SetCardinality(scan_count); + // keys.Fuse(payload); + // rid += increment * scan_count; + // + // // Sort on the sort columns (which will no longer be needed) + // keys.Split(payload, payload_idx); + // local_sort_state.SinkChunk(keys, payload); + // inserted += scan_count; + // keys.Fuse(payload); + // + // // Flush when we have enough data + // if (local_sort_state.SizeInBytes() >= marked.memory_per_thread) { + // local_sort_state.Sort(marked.global_sort_state, true); + // } + // } + // marked.global_sort_state.AddLocalState(local_sort_state); + // marked.count += inserted; + // + // return inserted; + // } + // + // IEJoinUnion::IEJoinUnion(ClientContext &context, const PhysicalIEJoin &op, SortedTable &t1, const idx_t b1, + // SortedTable &t2, const idx_t b2) + // : n(0), i(0) { + // // input : query Q with 2 join predicates t1.X op1 t2.X' and t1.Y op2 t2.Y', tables T, T' of sizes m and n resp. + // // output: a list of tuple pairs (ti , tj) + // // Note that T/T' are already sorted on X/X' and contain the payload data + // // We only join the two block numbers and use the sizes of the blocks as the counts + // + // // 0. Filter out tables with no overlap + // if (!t1.BlockSize(b1) || !t2.BlockSize(b2)) { + // return; + // } + // + // const auto &cmp1 = op.conditions[0].comparison; + // SBIterator bounds1(t1.global_sort_state, cmp1); + // SBIterator bounds2(t2.global_sort_state, cmp1); + // + // // t1.X[0] op1 t2.X'[-1] + // bounds1.SetIndex(bounds1.block_capacity * b1); + // bounds2.SetIndex(bounds2.block_capacity * b2 + t2.BlockSize(b2) - 1); + // if (!bounds1.Compare(bounds2)) { + // return; + // } + // + // // 1. let L1 (resp. L2) be the array of column X (resp. Y ) + // const auto &order1 = op.lhs_orders[0][0]; + // const auto &order2 = op.lhs_orders[1][0]; + // + // // 2. if (op1 ∈ {>, ≥}) sort L1 in descending order + // // 3. else if (op1 ∈ {<, ≤}) sort L1 in ascending order + // + // // For the union algorithm, we make a unified table with the keys and the rids as the payload: + // // X/X', Y/Y', R/R'/Li + // // The first position is the sort key. + // vector types; + // types.emplace_back(order2.expression->return_type); + // types.emplace_back(LogicalType::BIGINT); + // RowLayout payload_layout; + // payload_layout.Initialize(types); + // + // // Sort on the first expression + // auto ref = make_uniq(order1.expression->return_type, 0); + // vector orders; + // orders.emplace_back(order1.type, order1.null_order, std::move(ref)); + // + // l1 = make_uniq(context, orders, payload_layout); + // + // // LHS has positive rids + // ExpressionExecutor l_executor(context); + // l_executor.AddExpression(*order1.expression); + // l_executor.AddExpression(*order2.expression); + // AppendKey(t1, l_executor, *l1, 1, 1, b1); + // + // // RHS has negative rids + // ExpressionExecutor r_executor(context); + // r_executor.AddExpression(*op.rhs_orders[0][0].expression); + // r_executor.AddExpression(*op.rhs_orders[1][0].expression); + // AppendKey(t2, r_executor, *l1, -1, -1, b2); + // + // if (l1->global_sort_state.sorted_blocks.empty()) { + // return; + // } + // + // Sort(*l1); + // + // op1 = make_uniq(l1->global_sort_state, cmp1); + // off1 = make_uniq(l1->global_sort_state, cmp1); + // + // // We don't actually need the L1 column, just its sort key, which is in the sort blocks + // li = ExtractColumn(*l1, types.size() - 1); + // + // // 4. if (op2 ∈ {>, ≥}) sort L2 in ascending order + // // 5. else if (op2 ∈ {<, ≤}) sort L2 in descending order + // + // // We sort on Y/Y' to obtain the sort keys and the permutation array. + // // For this we just need a two-column table of Y, P + // types.clear(); + // types.emplace_back(LogicalType::BIGINT); + // payload_layout.Initialize(types); + // + // // Sort on the first expression + // orders.clear(); + // ref = make_uniq(order2.expression->return_type, 0); + // orders.emplace_back(order2.type, order2.null_order, std::move(ref)); + // + // ExpressionExecutor executor(context); + // executor.AddExpression(*orders[0].expression); + // + // l2 = make_uniq(context, orders, payload_layout); + // for (idx_t base = 0, block_idx = 0; block_idx < l1->BlockCount(); ++block_idx) { + // base += AppendKey(*l1, executor, *l2, 1, base, block_idx); + // } + // + // Sort(*l2); + // + // // We don't actually need the L2 column, just its sort key, which is in the sort blocks + // + // // 6. compute the permutation array P of L2 w.r.t. L1 + // p = ExtractColumn(*l2, types.size() - 1); + // + // // 7. initialize bit-array B (|B| = n), and set all bits to 0 + // n = l2->count.load(); + // bit_array.resize(ValidityMask::EntryCount(n), 0); + // bit_mask.Initialize(bit_array.data()); + // + // // Bloom filter + // bloom_count = (n + (BLOOM_CHUNK_BITS - 1)) / BLOOM_CHUNK_BITS; + // bloom_array.resize(ValidityMask::EntryCount(bloom_count), 0); + // bloom_filter.Initialize(bloom_array.data()); + // + // // 11. for(i←1 to n) do + // const auto &cmp2 = op.conditions[1].comparison; + // op2 = make_uniq(l2->global_sort_state, cmp2); + // off2 = make_uniq(l2->global_sort_state, cmp2); + // i = 0; + // j = 0; + // (void)NextRow(); + // } + // + // idx_t IEJoinUnion::SearchL1(idx_t pos) { + // // Perform an exponential search in the appropriate direction + // op1->SetIndex(pos); + // + // idx_t step = 1; + // auto hi = pos; + // auto lo = pos; + // if (!op1->cmp) { + // // Scan left for loose inequality + // lo -= MinValue(step, lo); + // step *= 2; + // off1->SetIndex(lo); + // while (lo > 0 && op1->Compare(*off1)) { + // hi = lo; + // lo -= MinValue(step, lo); + // step *= 2; + // off1->SetIndex(lo); + // } + // } else { + // // Scan right for strict inequality + // hi += MinValue(step, n - hi); + // step *= 2; + // off1->SetIndex(hi); + // while (hi < n && !op1->Compare(*off1)) { + // lo = hi; + // hi += MinValue(step, n - hi); + // step *= 2; + // off1->SetIndex(hi); + // } + // } + // + // // Binary search the target area + // while (lo < hi) { + // const auto mid = lo + (hi - lo) / 2; + // off1->SetIndex(mid); + // if (op1->Compare(*off1)) { + // hi = mid; + // } else { + // lo = mid + 1; + // } + // } + // + // off1->SetIndex(lo); + // + // return lo; + // } + // + // bool IEJoinUnion::NextRow() { + // for (; i < n; ++i) { + // // 12. pos ← P[i] + // auto pos = p[i]; + // lrid = li[pos]; + // if (lrid < 0) { + // continue; + // } + // + // // 16. B[pos] ← 1 + // op2->SetIndex(i); + // for (; off2->GetIndex() < n; ++(*off2)) { + // if (!off2->Compare(*op2)) { + // break; + // } + // const auto p2 = p[off2->GetIndex()]; + // if (li[p2] < 0) { + // // Only mark rhs matches. + // bit_mask.SetValid(p2); + // bloom_filter.SetValid(p2 / BLOOM_CHUNK_BITS); + // } + // } + // + // // 9. if (op1 ∈ {≤,≥} and op2 ∈ {≤,≥}) eqOff = 0 + // // 10. else eqOff = 1 + // // No, because there could be more than one equal value. + // // Find the leftmost off1 where L1[pos] op1 L1[off1..n] + // // These are the rows that satisfy the op1 condition + // // and that is where we should start scanning B from + // j = SearchL1(pos); + // + // return true; + // } + // return false; + // } + // + // static idx_t NextValid(const ValidityMask &bits, idx_t j, const idx_t n) { + // if (j >= n) { + // return n; + // } + // + // // We can do a first approximation by checking entries one at a time + // // which gives 64:1. + // idx_t entry_idx, idx_in_entry; + // bits.GetEntryIndex(j, entry_idx, idx_in_entry); + // auto entry = bits.GetValidityEntry(entry_idx++); + // + // // Trim the bits before the start position + // entry &= (ValidityMask::ValidityBuffer::MAX_ENTRY << idx_in_entry); + // + // // Check the non-ragged entries + // for (const auto entry_count = bits.EntryCount(n); entry_idx < entry_count; ++entry_idx) { + // if (entry) { + // for (; idx_in_entry < bits.BITS_PER_VALUE; ++idx_in_entry, ++j) { + // if (bits.RowIsValid(entry, idx_in_entry)) { + // return j; + // } + // } + // } else { + // j += bits.BITS_PER_VALUE - idx_in_entry; + // } + // + // entry = bits.GetValidityEntry(entry_idx); + // idx_in_entry = 0; + // } + // + // // Check the final entry + // for (; j < n; ++idx_in_entry, ++j) { + // if (bits.RowIsValid(entry, idx_in_entry)) { + // return j; + // } + // } + // + // return j; + // } + // + // idx_t IEJoinUnion::JoinComplexBlocks(SelectionVector &lsel, SelectionVector &rsel) { + // // 8. initialize join result as an empty list for tuple pairs + // idx_t result_count = 0; + // + // // 11. for(i←1 to n) do + // while (i < n) { + // // 13. for (j ← pos+eqOff to n) do + // for (;;) { + // // 14. if B[j] = 1 then + // + // // Use the Bloom filter to find candidate blocks + // while (j < n) { + // auto bloom_begin = NextValid(bloom_filter, j / BLOOM_CHUNK_BITS, bloom_count) * BLOOM_CHUNK_BITS; + // auto bloom_end = MinValue(n, bloom_begin + BLOOM_CHUNK_BITS); + // + // j = MaxValue(j, bloom_begin); + // j = NextValid(bit_mask, j, bloom_end); + // if (j < bloom_end) { + // break; + // } + // } + // + // if (j >= n) { + // break; + // } + // + // // Filter out tuples with the same sign (they come from the same table) + // const auto rrid = li[j]; + // ++j; + // + // // 15. add tuples w.r.t. (L1[j], L1[i]) to join result + // if (lrid > 0 && rrid < 0) { + // lsel.set_index(result_count, sel_t(+lrid - 1)); + // rsel.set_index(result_count, sel_t(-rrid - 1)); + // ++result_count; + // if (result_count == STANDARD_VECTOR_SIZE) { + // // out of space! + // return result_count; + // } + // } + // } + // ++i; + // + // if (!NextRow()) { + // break; + // } + // } + // + // return result_count; + // } + // + class PathFindingLocalSourceState : public LocalSourceState { + public: + explicit PathFindingLocalSourceState(ClientContext &context, const PhysicalPathFinding &op) + : op(op), true_sel(STANDARD_VECTOR_SIZE) { +// left_executor(context), right_executor(context), left_matches(nullptr), right_matches(nullptr) + // auto &allocator = Allocator::Get(context); + // unprojected.Initialize(allocator, op.unprojected_types); + // + // if (op.conditions.size() < 3) { + // return; + // } + // + // vector left_types; + // vector right_types; + // for (idx_t i = 2; i < op.conditions.size(); ++i) { + // const auto &cond = op.conditions[i]; + // + // left_types.push_back(cond.left->return_type); + // left_executor.AddExpression(*cond.left); + // + // right_types.push_back(cond.left->return_type); + // right_executor.AddExpression(*cond.right); + // } + // + // left_keys.Initialize(allocator, left_types); + // right_keys.Initialize(allocator, right_types); + // } + + // idx_t SelectOuterRows(bool *matches) { + // idx_t count = 0; + // for (; outer_idx < outer_count; ++outer_idx) { + // if (!matches[outer_idx]) { + // true_sel.set_index(count++, outer_idx); + // if (count >= STANDARD_VECTOR_SIZE) { + // outer_idx++; + // break; + // } + // } + // } + + // return count; + } + + const PhysicalPathFinding &op; + + // Joining + // unique_ptr joiner; + // + // idx_t left_base; + // idx_t left_block_index; + // + // idx_t right_base; + // idx_t right_block_index; + + // Trailing predicates + SelectionVector true_sel; + + // ExpressionExecutor left_executor; + // DataChunk left_keys; + // + // ExpressionExecutor right_executor; + // DataChunk right_keys; + // + // DataChunk unprojected; + // + // // Outer joins + // idx_t outer_idx; + // idx_t outer_count; + // bool *left_matches; + // bool *right_matches; + }; + + // void PhysicalIEJoin::ResolveComplexJoin(ExecutionContext &context, DataChunk &result, LocalSourceState &state_p) const { + // auto &state = state_p.Cast(); + // auto &ie_sink = sink_state->Cast(); + // auto &left_table = *ie_sink.tables[0]; + // auto &right_table = *ie_sink.tables[1]; + // + // const auto left_cols = children[0]->GetTypes().size(); + // auto &chunk = state.unprojected; + // do { + // SelectionVector lsel(STANDARD_VECTOR_SIZE); + // SelectionVector rsel(STANDARD_VECTOR_SIZE); + // auto result_count = state.joiner->JoinComplexBlocks(lsel, rsel); + // if (result_count == 0) { + // // exhausted this pair + // return; + // } + // + // // found matches: extract them + // + // chunk.Reset(); + // SliceSortedPayload(chunk, left_table.global_sort_state, state.left_block_index, lsel, result_count, 0); + // SliceSortedPayload(chunk, right_table.global_sort_state, state.right_block_index, rsel, result_count, + // left_cols); + // chunk.SetCardinality(result_count); + // + // auto sel = FlatVector::IncrementalSelectionVector(); + // if (conditions.size() > 2) { + // // If there are more expressions to compute, + // // split the result chunk into the left and right halves + // // so we can compute the values for comparison. + // const auto tail_cols = conditions.size() - 2; + // + // DataChunk right_chunk; + // chunk.Split(right_chunk, left_cols); + // state.left_executor.SetChunk(chunk); + // state.right_executor.SetChunk(right_chunk); + // + // auto tail_count = result_count; + // auto true_sel = &state.true_sel; + // for (size_t cmp_idx = 0; cmp_idx < tail_cols; ++cmp_idx) { + // auto &left = state.left_keys.data[cmp_idx]; + // state.left_executor.ExecuteExpression(cmp_idx, left); + // + // auto &right = state.right_keys.data[cmp_idx]; + // state.right_executor.ExecuteExpression(cmp_idx, right); + // + // if (tail_count < result_count) { + // left.Slice(*sel, tail_count); + // right.Slice(*sel, tail_count); + // } + // tail_count = SelectJoinTail(conditions[cmp_idx + 2].comparison, left, right, sel, tail_count, true_sel); + // sel = true_sel; + // } + // chunk.Fuse(right_chunk); + // + // if (tail_count < result_count) { + // result_count = tail_count; + // chunk.Slice(*sel, result_count); + // } + // } + // + // // We need all of the data to compute other predicates, + // // but we only return what is in the projection map + // ProjectResult(chunk, result); + // + // // found matches: mark the found matches if required + // if (left_table.found_match) { + // for (idx_t i = 0; i < result_count; i++) { + // left_table.found_match[state.left_base + lsel[sel->get_index(i)]] = true; + // } + // } + // if (right_table.found_match) { + // for (idx_t i = 0; i < result_count; i++) { + // right_table.found_match[state.right_base + rsel[sel->get_index(i)]] = true; + // } + // } + // result.Verify(); + // } while (result.size() == 0); + // } + + class PathFindingGlobalSourceState : public GlobalSourceState { + public: + explicit PathFindingGlobalSourceState(const PhysicalPathFinding &op) + : op(op), initialized(false) { + } + + void Initialize(PathFindingGlobalState &sink_state) { + lock_guard initializing(lock); + if (initialized) { + return; + } + + // Compute the starting row for reach block + // (In theory these are all the same size, but you never know...) + // auto &left_table = *sink_state.tables[0]; + // const auto left_blocks = left_table.BlockCount(); + // idx_t left_base = 0; + // + // for (size_t lhs = 0; lhs < left_blocks; ++lhs) { + // left_bases.emplace_back(left_base); + // left_base += left_table.BlockSize(lhs); + // } + // + // auto &right_table = *sink_state.tables[1]; + // const auto right_blocks = right_table.BlockCount(); + // idx_t right_base = 0; + // for (size_t rhs = 0; rhs < right_blocks; ++rhs) { + // right_bases.emplace_back(right_base); + // right_base += right_table.BlockSize(rhs); + // } + // + // // Outer join block counts + // if (left_table.found_match) { + // left_outers = left_blocks; + // } + // + // if (right_table.found_match) { + // right_outers = right_blocks; + // } + + // Ready for action + initialized = true; + } + + public: + idx_t MaxThreads() override { + // We can't leverage any more threads than block pairs. + const auto &sink_state = (op.sink_state->Cast()); + return 1; + // return sink_state.tables[0]->BlockCount() * sink_state.tables[1]->BlockCount(); + } + + void GetNextPair(ClientContext &client, PathFindingGlobalState &gstate, PathFindingLocalSourceState &lstate) { + // auto &left_table = *gstate.tables[0]; + // auto &right_table = *gstate.tables[1]; + // + // const auto left_blocks = left_table.BlockCount(); + // const auto right_blocks = right_table.BlockCount(); + // const auto pair_count = left_blocks * right_blocks; + // + // // Regular block + // const auto i = next_pair++; + // if (i < pair_count) { + // const auto b1 = i / right_blocks; + // const auto b2 = i % right_blocks; + // + // lstate.left_block_index = b1; + // lstate.left_base = left_bases[b1]; + // + // lstate.right_block_index = b2; + // lstate.right_base = right_bases[b2]; + // + // lstate.joiner = make_uniq(client, op, left_table, b1, right_table, b2); + // return; + // } + + // // Outer joins + // if (!left_outers && !right_outers) { + // return; + // } + // + // // Spin wait for regular blocks to finish(!) + // while (completed < pair_count) { + // std::this_thread::yield(); + // } + // + // // Left outer blocks + // const auto l = next_left++; + // if (l < left_outers) { + // lstate.joiner = nullptr; + // lstate.left_block_index = l; + // lstate.left_base = left_bases[l]; + // + // lstate.left_matches = left_table.found_match.get() + lstate.left_base; + // lstate.outer_idx = 0; + // lstate.outer_count = left_table.BlockSize(l); + // return; + // } else { + // lstate.left_matches = nullptr; + // } + // + // // Right outer block + // const auto r = next_right++; + // if (r < right_outers) { + // lstate.joiner = nullptr; + // lstate.right_block_index = r; + // lstate.right_base = right_bases[r]; + // + // lstate.right_matches = right_table.found_match.get() + lstate.right_base; + // lstate.outer_idx = 0; + // lstate.outer_count = right_table.BlockSize(r); + // return; + // } else { + // lstate.right_matches = nullptr; + // } + } + + // void PairCompleted(ClientContext &client, PathFindingGlobalState &gstate, PahtFindingLocalSourceState &lstate) { + // lstate.joiner.reset(); + // ++completed; + // GetNextPair(client, gstate, lstate); + // } + + const PhysicalPathFinding &op; + + mutex lock; + bool initialized; + + // Join queue state + // std::atomic next_pair; + // std::atomic completed; + + // Block base row number + // vector left_bases; + // vector right_bases; + + // Outer joins + // idx_t left_outers; + // std::atomic next_left; + + // idx_t right_outers; + // std::atomic next_right; + }; + + unique_ptr PhysicalPathFinding::GetGlobalSourceState(ClientContext &context) const { + return make_uniq(*this); + } + + unique_ptr PhysicalPathFinding::GetLocalSourceState(ExecutionContext &context, + GlobalSourceState &gstate) const { + return make_uniq(context.client, *this); + } + + SourceResultType PhysicalPathFinding::GetData(ExecutionContext &context, DataChunk &result, + OperatorSourceInput &input) const { + auto &pf_sink = sink_state->Cast(); + auto &pf_gstate = input.global_state.Cast(); + auto &pf_lstate = input.local_state.Cast(); + + pf_gstate.Initialize(pf_sink); + + // if (!ie_lstate.joiner && !ie_lstate.left_matches && !ie_lstate.right_matches) { + // ie_gstate.GetNextPair(context.client, ie_sink, ie_lstate); + // } + + // Process INNER results + // while (pf_lstate.joiner) { + // ResolveComplexJoin(context, result, ie_lstate); + // + // if (result.size()) { + // return SourceResultType::HAVE_MORE_OUTPUT; + // } + // + // ie_gstate.PairCompleted(context.client, ie_sink, ie_lstate); + // } + + // Process LEFT OUTER results + // const auto left_cols = children[0]->GetTypes().size(); + // while (ie_lstate.left_matches) { + // const idx_t count = ie_lstate.SelectOuterRows(ie_lstate.left_matches); + // if (!count) { + // ie_gstate.GetNextPair(context.client, ie_sink, ie_lstate); + // continue; + // } + // auto &chunk = ie_lstate.unprojected; + // chunk.Reset(); + // SliceSortedPayload(chunk, ie_sink.tables[0]->global_sort_state, ie_lstate.left_block_index, ie_lstate.true_sel, + // count); + // + // // Fill in NULLs to the right + // for (auto col_idx = left_cols; col_idx < chunk.ColumnCount(); ++col_idx) { + // chunk.data[col_idx].SetVectorType(VectorType::CONSTANT_VECTOR); + // ConstantVector::SetNull(chunk.data[col_idx], true); + // } + // + // ProjectResult(chunk, result); + // result.SetCardinality(count); + // result.Verify(); + // + // return result.size() == 0 ? SourceResultType::FINISHED : SourceResultType::HAVE_MORE_OUTPUT; + // } + + // Process RIGHT OUTER results + // while (ie_lstate.right_matches) { + // const idx_t count = ie_lstate.SelectOuterRows(ie_lstate.right_matches); + // if (!count) { + // ie_gstate.GetNextPair(context.client, ie_sink, ie_lstate); + // continue; + // } + // + // auto &chunk = ie_lstate.unprojected; + // chunk.Reset(); + // SliceSortedPayload(chunk, ie_sink.tables[1]->global_sort_state, ie_lstate.right_block_index, ie_lstate.true_sel, + // count, left_cols); + // + // // Fill in NULLs to the left + // for (idx_t col_idx = 0; col_idx < left_cols; ++col_idx) { + // chunk.data[col_idx].SetVectorType(VectorType::CONSTANT_VECTOR); + // ConstantVector::SetNull(chunk.data[col_idx], true); + // } + // + // ProjectResult(chunk, result); + // result.SetCardinality(count); + // result.Verify(); + // + // break; + // } + + return result.size() == 0 ? SourceResultType::FINISHED : SourceResultType::HAVE_MORE_OUTPUT; + } + +//===--------------------------------------------------------------------===// +// Pipeline Construction +//===--------------------------------------------------------------------===// + void PhysicalPathFinding::BuildPipelines(Pipeline ¤t, MetaPipeline &meta_pipeline) { + D_ASSERT(children.size() == 2); + if (meta_pipeline.HasRecursiveCTE()) { + throw NotImplementedException("Path Finding is not supported in recursive CTEs yet"); + } + + // becomes a source after both children fully sink their data + meta_pipeline.GetState().SetPipelineSource(current, *this); + + // Create one child meta pipeline that will hold the LHS and RHS pipelines + auto &child_meta_pipeline = meta_pipeline.CreateChildMetaPipeline(current, *this); + + // Build out LHS + auto lhs_pipeline = child_meta_pipeline.GetBasePipeline(); + children[0]->BuildPipelines(*lhs_pipeline, child_meta_pipeline); + + // Build out RHS + auto rhs_pipeline = child_meta_pipeline.CreatePipeline(); + children[1]->BuildPipelines(*rhs_pipeline, child_meta_pipeline); + + // Despite having the same sink, RHS and everything created after it need their own (same) PipelineFinishEvent + child_meta_pipeline.AddFinishEvent(rhs_pipeline); + } + +} // namespace duckdb + From 18019af4769d2434eec548ecd911788e53fae2f1 Mon Sep 17 00:00:00 2001 From: dtenwolde Date: Tue, 19 Dec 2023 12:43:30 +0100 Subject: [PATCH 005/249] Compiles --- .../operators/physical_path_finding.hpp | 19 ++++++----------- .../operators/path_finding_operator.cpp | 21 +++++++++---------- .../operators/physical_path_finding.cpp | 8 +++++-- 3 files changed, 22 insertions(+), 26 deletions(-) diff --git a/duckpgq/include/duckpgq/operators/physical_path_finding.hpp b/duckpgq/include/duckpgq/operators/physical_path_finding.hpp index 22ba8590..b2bda204 100644 --- a/duckpgq/include/duckpgq/operators/physical_path_finding.hpp +++ b/duckpgq/include/duckpgq/operators/physical_path_finding.hpp @@ -1,25 +1,23 @@ //===----------------------------------------------------------------------===// // DuckDB // -// duckdb/execution/operator/join/physical_piecewise_merge_join.hpp +// duckdb/execution/operator/join/physical_path_finding.hpp // // //===----------------------------------------------------------------------===// #pragma once -#include "duckdb/execution/operator/join/physical_range_join.hpp" -#include "duckdb/planner/bound_result_modifier.hpp" +#include -namespace duckdb { +#include "duckdb/execution/physical_operator.hpp" -//! PhysicalIEJoin represents a two inequality range join between -//! two tables +namespace duckdb { class PhysicalPathFinding : public CachingPhysicalOperator { public: - static constexpr const PhysicalOperatorType TYPE = PhysicalOperatorType::PATH_FINDING; + static constexpr const PhysicalOperatorType TYPE = PhysicalOperatorType::EXTENSION; public: - PhysicalPathFinding(LogicalComparisonJoin &op, unique_ptr left, + PhysicalPathFinding(LogicalExtensionOperator &op, unique_ptr left, unique_ptr right); // vector join_key_types; @@ -61,12 +59,7 @@ namespace duckdb { return true; } - public: void BuildPipelines(Pipeline ¤t, MetaPipeline &meta_pipeline) override; - - private: - // resolve joins that can potentially output N*M elements (INNER, LEFT, FULL) - // void ResolveComplexJoin(ExecutionContext &context, DataChunk &result, LocalSourceState &state) const; }; } // namespace duckdb diff --git a/duckpgq/src/duckpgq/operators/path_finding_operator.cpp b/duckpgq/src/duckpgq/operators/path_finding_operator.cpp index 31add29b..0048545a 100644 --- a/duckpgq/src/duckpgq/operators/path_finding_operator.cpp +++ b/duckpgq/src/duckpgq/operators/path_finding_operator.cpp @@ -1,20 +1,19 @@ #include "duckpgq/operators/path_finding_operator.hpp" #include +#include namespace duckdb { - unique_ptr PathFindingOperator::CreatePlan(ClientContext &context, PhysicalPlanGenerator &generator) { - return unique_ptr(); // TODO IMPLEMENT ME - } - - duckdb::unique_ptr CreatePlan(duckdb::ClientContext &, - duckdb::PhysicalPlanGenerator &generator) override { - auto result = duckdb::make_uniq_base(bridge_id, types, - estimated_cardinality); + unique_ptr PathFindingOperator::CreatePlan(ClientContext &, + duckdb::PhysicalPlanGenerator &generator) { D_ASSERT(children.size() == 2); - auto plan = generator.CreatePlan(std::move(children[0])); - result->children.emplace_back(std::move(plan)); + auto left = generator.CreatePlan(std::move(children[0])); + auto right = generator.CreatePlan(std::move(children[1])); + + auto result = duckdb::make_uniq_base(*this, std::move(left), std::move(right)); + // auto plan = generator.CreatePlan(std::move(children[0])); + result->children.emplace_back(std::move(result)); return result; } -} \ No newline at end of file +} diff --git a/duckpgq/src/duckpgq/operators/physical_path_finding.cpp b/duckpgq/src/duckpgq/operators/physical_path_finding.cpp index d7762891..698ce7b6 100644 --- a/duckpgq/src/duckpgq/operators/physical_path_finding.cpp +++ b/duckpgq/src/duckpgq/operators/physical_path_finding.cpp @@ -5,16 +5,20 @@ #include "duckdb/parallel/event.hpp" #include "duckdb/parallel/meta_pipeline.hpp" #include "duckdb/parallel/thread_context.hpp" +#include "duckdb/execution/physical_operator.hpp" #include +#include #include namespace duckdb { - PhysicalPathFinding::PhysicalPathFinding(LogicalComparisonJoin &op, unique_ptr left, + PhysicalPathFinding::PhysicalPathFinding(LogicalExtensionOperator &op, unique_ptr left, unique_ptr right) - : CachingPhysicalOperator(type, op.types, estimated_cardinality) { + : CachingPhysicalOperator(TYPE, op.types, 0) { + children.push_back(std::move(left)); + children.push_back(std::move(right)); } //===--------------------------------------------------------------------===// From 407b46565fbf808912361f29a38a6815c15a46a3 Mon Sep 17 00:00:00 2001 From: dtenwolde Date: Wed, 10 Jan 2024 15:18:49 +0100 Subject: [PATCH 006/249] Remove comment --- duckdb-pgq | 2 +- duckpgq/src/duckpgq/operators/path_finding_operator.cpp | 1 - 2 files changed, 1 insertion(+), 2 deletions(-) diff --git a/duckdb-pgq b/duckdb-pgq index 925281d0..430336d1 160000 --- a/duckdb-pgq +++ b/duckdb-pgq @@ -1 +1 @@ -Subproject commit 925281d061c48bf620de8171282571fcbad32e36 +Subproject commit 430336d1c5ec74c73eb553ba985a5a31b2073dbc diff --git a/duckpgq/src/duckpgq/operators/path_finding_operator.cpp b/duckpgq/src/duckpgq/operators/path_finding_operator.cpp index 0048545a..b9659817 100644 --- a/duckpgq/src/duckpgq/operators/path_finding_operator.cpp +++ b/duckpgq/src/duckpgq/operators/path_finding_operator.cpp @@ -11,7 +11,6 @@ namespace duckdb { auto right = generator.CreatePlan(std::move(children[1])); auto result = duckdb::make_uniq_base(*this, std::move(left), std::move(right)); - // auto plan = generator.CreatePlan(std::move(children[0])); result->children.emplace_back(std::move(result)); return result; } From 57996010350a32a2855086936944d26ebf566f33 Mon Sep 17 00:00:00 2001 From: dtenwolde Date: Thu, 15 Feb 2024 12:07:48 +0100 Subject: [PATCH 007/249] Format --- .../operators/path_finding_operator.cpp | 24 ++++++++++--------- 1 file changed, 13 insertions(+), 11 deletions(-) diff --git a/duckpgq/src/duckpgq/operators/path_finding_operator.cpp b/duckpgq/src/duckpgq/operators/path_finding_operator.cpp index b9659817..2d88845a 100644 --- a/duckpgq/src/duckpgq/operators/path_finding_operator.cpp +++ b/duckpgq/src/duckpgq/operators/path_finding_operator.cpp @@ -1,18 +1,20 @@ #include "duckpgq/operators/path_finding_operator.hpp" -#include #include +#include namespace duckdb { - unique_ptr PathFindingOperator::CreatePlan(ClientContext &, - duckdb::PhysicalPlanGenerator &generator) { - D_ASSERT(children.size() == 2); - auto left = generator.CreatePlan(std::move(children[0])); - auto right = generator.CreatePlan(std::move(children[1])); - - auto result = duckdb::make_uniq_base(*this, std::move(left), std::move(right)); - result->children.emplace_back(std::move(result)); - return result; - } +unique_ptr +PathFindingOperator::CreatePlan(ClientContext &, + duckdb::PhysicalPlanGenerator &generator) { + D_ASSERT(children.size() == 2); + auto left = generator.CreatePlan(std::move(children[0])); + auto right = generator.CreatePlan(std::move(children[1])); + auto result = + duckdb::make_uniq_base( + *this, std::move(left), std::move(right)); + result->children.emplace_back(std::move(result)); + return result; } +} // namespace duckdb From 58db9d6e1ea4fd2d28d4ee548006d1103bb9eea3 Mon Sep 17 00:00:00 2001 From: dtenwolde Date: Thu, 15 Feb 2024 12:14:47 +0100 Subject: [PATCH 008/249] Builds --- .../operators/physical_path_finding.hpp | 87 +- .../operators/physical_path_finding.cpp | 1990 +++++++++-------- 2 files changed, 1060 insertions(+), 1017 deletions(-) diff --git a/duckpgq/include/duckpgq/operators/physical_path_finding.hpp b/duckpgq/include/duckpgq/operators/physical_path_finding.hpp index b2bda204..67e966d0 100644 --- a/duckpgq/include/duckpgq/operators/physical_path_finding.hpp +++ b/duckpgq/include/duckpgq/operators/physical_path_finding.hpp @@ -13,53 +13,58 @@ #include "duckdb/execution/physical_operator.hpp" namespace duckdb { - class PhysicalPathFinding : public CachingPhysicalOperator { - public: - static constexpr const PhysicalOperatorType TYPE = PhysicalOperatorType::EXTENSION; - public: - PhysicalPathFinding(LogicalExtensionOperator &op, unique_ptr left, - unique_ptr right); +class PhysicalPathFinding : public CachingPhysicalOperator { +public: + static constexpr const PhysicalOperatorType TYPE = + PhysicalOperatorType::EXTENSION; - // vector join_key_types; - // vector> lhs_orders; - // vector> rhs_orders; +public: + PhysicalPathFinding(LogicalExtensionOperator &op, + unique_ptr left, + unique_ptr right); - public: - // CachingOperator Interface - OperatorResultType ExecuteInternal(ExecutionContext &context, DataChunk &input, DataChunk &chunk, - GlobalOperatorState &gstate, OperatorState &state) const override; + // vector join_key_types; + // vector> lhs_orders; + // vector> rhs_orders; - public: - // Source interface - unique_ptr GetLocalSourceState(ExecutionContext &context, - GlobalSourceState &gstate) const override; - unique_ptr GetGlobalSourceState(ClientContext &context) const override; - SourceResultType GetData(ExecutionContext &context, DataChunk &chunk, OperatorSourceInput &input) const override; +public: + // CachingOperator Interface + OperatorResultType ExecuteInternal(ExecutionContext &context, + DataChunk &input, DataChunk &chunk, + GlobalOperatorState &gstate, + OperatorState &state) const override; - bool IsSource() const override { - return true; - } - bool ParallelSource() const override { - return true; - } +public: + // Source interface + unique_ptr + GetLocalSourceState(ExecutionContext &context, + GlobalSourceState &gstate) const override; + unique_ptr + GetGlobalSourceState(ClientContext &context) const override; + SourceResultType GetData(ExecutionContext &context, DataChunk &chunk, + OperatorSourceInput &input) const override; - public: - // Sink Interface - unique_ptr GetGlobalSinkState(ClientContext &context) const override; - unique_ptr GetLocalSinkState(ExecutionContext &context) const override; - SinkResultType Sink(ExecutionContext &context, DataChunk &chunk, OperatorSinkInput &input) const override; - SinkCombineResultType Combine(ExecutionContext &context, OperatorSinkCombineInput &input) const override; - SinkFinalizeType Finalize(Pipeline &pipeline, Event &event, ClientContext &context, - OperatorSinkFinalizeInput &input) const override; + bool IsSource() const override { return true; } + bool ParallelSource() const override { return true; } - bool IsSink() const override { - return true; - } - bool ParallelSink() const override { - return true; - } +public: + // Sink Interface + unique_ptr + GetGlobalSinkState(ClientContext &context) const override; + unique_ptr + GetLocalSinkState(ExecutionContext &context) const override; + SinkResultType Sink(ExecutionContext &context, DataChunk &chunk, + OperatorSinkInput &input) const override; + SinkCombineResultType Combine(ExecutionContext &context, + OperatorSinkCombineInput &input) const override; + SinkFinalizeType Finalize(Pipeline &pipeline, Event &event, + ClientContext &context, + OperatorSinkFinalizeInput &input) const override; - void BuildPipelines(Pipeline ¤t, MetaPipeline &meta_pipeline) override; - }; + bool IsSink() const override { return true; } + bool ParallelSink() const override { return true; } + + void BuildPipelines(Pipeline ¤t, MetaPipeline &meta_pipeline) override; +}; } // namespace duckdb diff --git a/duckpgq/src/duckpgq/operators/physical_path_finding.cpp b/duckpgq/src/duckpgq/operators/physical_path_finding.cpp index 698ce7b6..863331cc 100644 --- a/duckpgq/src/duckpgq/operators/physical_path_finding.cpp +++ b/duckpgq/src/duckpgq/operators/physical_path_finding.cpp @@ -1,1012 +1,1050 @@ #include "duckpgq/operators/physical_path_finding.hpp" #include "duckdb/common/sort/sort.hpp" +#include "duckdb/execution/physical_operator.hpp" #include "duckdb/main/client_context.hpp" #include "duckdb/parallel/event.hpp" #include "duckdb/parallel/meta_pipeline.hpp" #include "duckdb/parallel/thread_context.hpp" -#include "duckdb/execution/physical_operator.hpp" -#include #include #include - +#include namespace duckdb { - PhysicalPathFinding::PhysicalPathFinding(LogicalExtensionOperator &op, unique_ptr left, - unique_ptr right) - : CachingPhysicalOperator(TYPE, op.types, 0) { - children.push_back(std::move(left)); - children.push_back(std::move(right)); - } +PhysicalPathFinding::PhysicalPathFinding(LogicalExtensionOperator &op, + unique_ptr left, + unique_ptr right) + : CachingPhysicalOperator(TYPE, op.types, 0) { + children.push_back(std::move(left)); + children.push_back(std::move(right)); +} //===--------------------------------------------------------------------===// // Sink //===--------------------------------------------------------------------===// - class PathFindingLocalState : public LocalSinkState { - public: - // TODO Add something to the local sink state - // using LocalSortedTable = PhysicalRangeJoin::LocalSortedTable; - - PathFindingLocalState(ClientContext &context, const PhysicalPathFinding &op, const idx_t child) - : context(context), op(op), child(child) { - } - ClientContext &context; - const PhysicalPathFinding &op; - const idx_t child; - //! The local sort state - // LocalSortedTable table; - }; - - class PathFindingGlobalState : public GlobalSinkState { - public: - using GlobalSortedTable = PhysicalRangeJoin::GlobalSortedTable; - - public: - PathFindingGlobalState(ClientContext &context, const PhysicalPathFinding &op) { - // tables.resize(2); - // RowLayout lhs_layout; - // lhs_layout.Initialize(op.children[0]->types); - // vector lhs_order; - // lhs_order.emplace_back(op.lhs_orders[0][0].Copy()); - // tables[0] = make_uniq(context, lhs_order, lhs_layout); - // - // RowLayout rhs_layout; - // rhs_layout.Initialize(op.children[1]->types); - // vector rhs_order; - // rhs_order.emplace_back(op.rhs_orders[0][0].Copy()); - // tables[1] = make_uniq(context, rhs_order, rhs_layout); - } - - PathFindingGlobalState(PathFindingGlobalState &prev) - : GlobalSinkState(prev) {//, tables(std::move(prev.tables)), child(prev.child + 1) { - } - - void Sink(DataChunk &input, PathFindingLocalState &lstate) { - // auto &table = *tables[child]; - // auto &global_sort_state = table.global_sort_state; - // auto &local_sort_state = lstate.table.local_sort_state; - // - // // Sink the data into the local sort state - // lstate.table.Sink(input, global_sort_state); - // - // // When sorting data reaches a certain size, we sort it - // if (local_sort_state.SizeInBytes() >= table.memory_per_thread) { - // local_sort_state.Sort(global_sort_state, true); - // } - } - - vector> tables; - // size_t child; - }; - - unique_ptr PhysicalPathFinding::GetGlobalSinkState(ClientContext &context) const { - D_ASSERT(!sink_state); - return make_uniq(context, *this); - } - - unique_ptr PhysicalPathFinding::GetLocalSinkState(ExecutionContext &context) const { - idx_t sink_child = 0; - if (sink_state) { - // const auto &ie_sink = sink_state->Cast(); - // sink_child = ie_sink.child; - } - return make_uniq(context.client, *this, sink_child); - } - - SinkResultType PhysicalPathFinding::Sink(ExecutionContext &context, DataChunk &chunk, OperatorSinkInput &input) const { - auto &gstate = input.global_state.Cast(); - auto &lstate = input.local_state.Cast(); - - gstate.Sink(chunk, lstate); - - return SinkResultType::NEED_MORE_INPUT; - } - - SinkCombineResultType PhysicalPathFinding::Combine(ExecutionContext &context, OperatorSinkCombineInput &input) const { - // auto &gstate = input.global_state.Cast(); - // auto &lstate = input.local_state.Cast(); - // gstate.tables[gstate.child]->Combine(lstate.table); - auto &client_profiler = QueryProfiler::Get(context.client); - - // context.thread.profiler.Flush(*this, lstate.table.executor, gstate.child ? "rhs_executor" : "lhs_executor", 1); - client_profiler.Flush(context.thread.profiler); - - return SinkCombineResultType::FINISHED; - } +class PathFindingLocalState : public LocalSinkState { +public: + // TODO Add something to the local sink state + // using LocalSortedTable = PhysicalRangeJoin::LocalSortedTable; + + PathFindingLocalState(ClientContext &context, const PhysicalPathFinding &op, + const idx_t child) + : context(context), op(op), child(child) {} + ClientContext &context; + const PhysicalPathFinding &op; + const idx_t child; + //! The local sort state + // LocalSortedTable table; +}; + +class PathFindingGlobalState : public GlobalSinkState { +public: + using GlobalSortedTable = PhysicalRangeJoin::GlobalSortedTable; + +public: + PathFindingGlobalState(ClientContext &context, + const PhysicalPathFinding &op) { + // tables.resize(2); + // RowLayout lhs_layout; + // lhs_layout.Initialize(op.children[0]->types); + // vector lhs_order; + // lhs_order.emplace_back(op.lhs_orders[0][0].Copy()); + // tables[0] = make_uniq(context, lhs_order, lhs_layout); + // + // RowLayout rhs_layout; + // rhs_layout.Initialize(op.children[1]->types); + // vector rhs_order; + // rhs_order.emplace_back(op.rhs_orders[0][0].Copy()); + // tables[1] = make_uniq(context, rhs_order, rhs_layout); + } + + PathFindingGlobalState(PathFindingGlobalState &prev) + : GlobalSinkState( + prev) { //, tables(std::move(prev.tables)), child(prev.child + 1) { + } + + void Sink(DataChunk &input, PathFindingLocalState &lstate) { + // auto &table = *tables[child]; + // auto &global_sort_state = table.global_sort_state; + // auto &local_sort_state = lstate.table.local_sort_state; + // + // // Sink the data into the local sort state + // lstate.table.Sink(input, global_sort_state); + // + // // When sorting data reaches a certain size, we sort it + // if (local_sort_state.SizeInBytes() >= table.memory_per_thread) { + // local_sort_state.Sort(global_sort_state, true); + // } + } + + vector> tables; + // size_t child; +}; + +unique_ptr +PhysicalPathFinding::GetGlobalSinkState(ClientContext &context) const { + D_ASSERT(!sink_state); + return make_uniq(context, *this); +} + +unique_ptr +PhysicalPathFinding::GetLocalSinkState(ExecutionContext &context) const { + idx_t sink_child = 0; + if (sink_state) { + // const auto &ie_sink = sink_state->Cast(); + // sink_child = ie_sink.child; + } + return make_uniq(context.client, *this, sink_child); +} + +SinkResultType PhysicalPathFinding::Sink(ExecutionContext &context, + DataChunk &chunk, + OperatorSinkInput &input) const { + auto &gstate = input.global_state.Cast(); + auto &lstate = input.local_state.Cast(); + + gstate.Sink(chunk, lstate); + + return SinkResultType::NEED_MORE_INPUT; +} + +SinkCombineResultType +PhysicalPathFinding::Combine(ExecutionContext &context, + OperatorSinkCombineInput &input) const { + // auto &gstate = input.global_state.Cast(); + // auto &lstate = input.local_state.Cast(); + // gstate.tables[gstate.child]->Combine(lstate.table); + auto &client_profiler = QueryProfiler::Get(context.client); + + // context.thread.profiler.Flush(*this, lstate.table.executor, gstate.child ? + // "rhs_executor" : "lhs_executor", 1); + client_profiler.Flush(context.thread.profiler); + + return SinkCombineResultType::FINISHED; +} //===--------------------------------------------------------------------===// // Finalize //===--------------------------------------------------------------------===// - SinkFinalizeType PhysicalPathFinding::Finalize(Pipeline &pipeline, Event &event, ClientContext &context, - OperatorSinkFinalizeInput &input) const { - auto &gstate = input.global_state.Cast(); - // auto &table = *gstate.tables[gstate.child]; - // auto &global_sort_state = table.global_sort_state; - - // if ((gstate.child == 1 && IsRightOuterJoin(join_type)) || (gstate.child == 0 && IsLeftOuterJoin(join_type))) { - // // for FULL/LEFT/RIGHT OUTER JOIN, initialize found_match to false for every tuple - // table.IntializeMatches(); - // } - // if (gstate.child == 1 && global_sort_state.sorted_blocks.empty() && EmptyResultIfRHSIsEmpty()) { - // // Empty input! - // return SinkFinalizeType::NO_OUTPUT_POSSIBLE; - // } - - // Sort the current input child - // table.Finalize(pipeline, event); - - // Move to the next input child - // ++gstate.child; - - return SinkFinalizeType::READY; - } +SinkFinalizeType +PhysicalPathFinding::Finalize(Pipeline &pipeline, Event &event, + ClientContext &context, + OperatorSinkFinalizeInput &input) const { + auto &gstate = input.global_state.Cast(); + // auto &table = *gstate.tables[gstate.child]; + // auto &global_sort_state = table.global_sort_state; + + // if ((gstate.child == 1 && IsRightOuterJoin(join_type)) || (gstate.child == + // 0 && IsLeftOuterJoin(join_type))) { + // // for FULL/LEFT/RIGHT OUTER JOIN, initialize found_match to false for + // every tuple table.IntializeMatches(); + // } + // if (gstate.child == 1 && global_sort_state.sorted_blocks.empty() && + // EmptyResultIfRHSIsEmpty()) { + // // Empty input! + // return SinkFinalizeType::NO_OUTPUT_POSSIBLE; + // } + + // Sort the current input child + // table.Finalize(pipeline, event); + + // Move to the next input child + // ++gstate.child; + + return SinkFinalizeType::READY; +} //===--------------------------------------------------------------------===// // Operator //===--------------------------------------------------------------------===// - OperatorResultType PhysicalPathFinding::ExecuteInternal(ExecutionContext &context, DataChunk &input, DataChunk &chunk, - GlobalOperatorState &gstate, OperatorState &state) const { - return OperatorResultType::FINISHED; - } +OperatorResultType PhysicalPathFinding::ExecuteInternal( + ExecutionContext &context, DataChunk &input, DataChunk &chunk, + GlobalOperatorState &gstate, OperatorState &state) const { + return OperatorResultType::FINISHED; +} //===--------------------------------------------------------------------===// // Source //===--------------------------------------------------------------------===// - // struct PathFindingUnion { - // using SortedTable = PhysicalRangeJoin::GlobalSortedTable; - // - // static idx_t AppendKey(SortedTable &table, ExpressionExecutor &executor, SortedTable &marked, int64_t increment, - // int64_t base, const idx_t block_idx); - // - // static void Sort(SortedTable &table) { - // auto &global_sort_state = table.global_sort_state; - // global_sort_state.PrepareMergePhase(); - // while (global_sort_state.sorted_blocks.size() > 1) { - // global_sort_state.InitializeMergeRound(); - // MergeSorter merge_sorter(global_sort_state, global_sort_state.buffer_manager); - // merge_sorter.PerformInMergeRound(); - // global_sort_state.CompleteMergeRound(true); - // } - // } - // - // template - // static vector ExtractColumn(SortedTable &table, idx_t col_idx) { - // vector result; - // result.reserve(table.count); - // - // auto &gstate = table.global_sort_state; - // auto &blocks = *gstate.sorted_blocks[0]->payload_data; - // PayloadScanner scanner(blocks, gstate, false); - // - // DataChunk payload; - // payload.Initialize(Allocator::DefaultAllocator(), gstate.payload_layout.GetTypes()); - // for (;;) { - // scanner.Scan(payload); - // const auto count = payload.size(); - // if (!count) { - // break; - // } - // - // const auto data_ptr = FlatVector::GetData(payload.data[col_idx]); - // result.insert(result.end(), data_ptr, data_ptr + count); - // } - // - // return result; - // } - // - // IEJoinUnion(ClientContext &context, const PhysicalIEJoin &op, SortedTable &t1, const idx_t b1, SortedTable &t2, - // const idx_t b2); - // - // idx_t SearchL1(idx_t pos); - // bool NextRow(); - // - // //! Inverted loop - // idx_t JoinComplexBlocks(SelectionVector &lsel, SelectionVector &rsel); - // - // //! L1 - // unique_ptr l1; - // //! L2 - // unique_ptr l2; - // - // //! Li - // vector li; - // //! P - // vector p; - // - // //! B - // vector bit_array; - // ValidityMask bit_mask; - // - // //! Bloom Filter - // static constexpr idx_t BLOOM_CHUNK_BITS = 1024; - // idx_t bloom_count; - // vector bloom_array; - // ValidityMask bloom_filter; - // - // //! Iteration state - // idx_t n; - // idx_t i; - // idx_t j; - // unique_ptr op1; - // unique_ptr off1; - // unique_ptr op2; - // unique_ptr off2; - // int64_t lrid; - // }; - - // idx_t IEJoinUnion::AppendKey(SortedTable &table, ExpressionExecutor &executor, SortedTable &marked, int64_t increment, - // int64_t base, const idx_t block_idx) { - // LocalSortState local_sort_state; - // local_sort_state.Initialize(marked.global_sort_state, marked.global_sort_state.buffer_manager); - // - // // Reading - // const auto valid = table.count - table.has_null; - // auto &gstate = table.global_sort_state; - // PayloadScanner scanner(gstate, block_idx); - // auto table_idx = block_idx * gstate.block_capacity; - // - // DataChunk scanned; - // scanned.Initialize(Allocator::DefaultAllocator(), scanner.GetPayloadTypes()); - // - // // Writing - // auto types = local_sort_state.sort_layout->logical_types; - // const idx_t payload_idx = types.size(); - // - // const auto &payload_types = local_sort_state.payload_layout->GetTypes(); - // types.insert(types.end(), payload_types.begin(), payload_types.end()); - // const idx_t rid_idx = types.size() - 1; - // - // DataChunk keys; - // DataChunk payload; - // keys.Initialize(Allocator::DefaultAllocator(), types); - // - // idx_t inserted = 0; - // for (auto rid = base; table_idx < valid;) { - // scanner.Scan(scanned); - // - // // NULLs are at the end, so stop when we reach them - // auto scan_count = scanned.size(); - // if (table_idx + scan_count > valid) { - // scan_count = valid - table_idx; - // scanned.SetCardinality(scan_count); - // } - // if (scan_count == 0) { - // break; - // } - // table_idx += scan_count; - // - // // Compute the input columns from the payload - // keys.Reset(); - // keys.Split(payload, rid_idx); - // executor.Execute(scanned, keys); - // - // // Mark the rid column - // payload.data[0].Sequence(rid, increment, scan_count); - // payload.SetCardinality(scan_count); - // keys.Fuse(payload); - // rid += increment * scan_count; - // - // // Sort on the sort columns (which will no longer be needed) - // keys.Split(payload, payload_idx); - // local_sort_state.SinkChunk(keys, payload); - // inserted += scan_count; - // keys.Fuse(payload); - // - // // Flush when we have enough data - // if (local_sort_state.SizeInBytes() >= marked.memory_per_thread) { - // local_sort_state.Sort(marked.global_sort_state, true); - // } - // } - // marked.global_sort_state.AddLocalState(local_sort_state); - // marked.count += inserted; - // - // return inserted; - // } - // - // IEJoinUnion::IEJoinUnion(ClientContext &context, const PhysicalIEJoin &op, SortedTable &t1, const idx_t b1, - // SortedTable &t2, const idx_t b2) - // : n(0), i(0) { - // // input : query Q with 2 join predicates t1.X op1 t2.X' and t1.Y op2 t2.Y', tables T, T' of sizes m and n resp. - // // output: a list of tuple pairs (ti , tj) - // // Note that T/T' are already sorted on X/X' and contain the payload data - // // We only join the two block numbers and use the sizes of the blocks as the counts - // - // // 0. Filter out tables with no overlap - // if (!t1.BlockSize(b1) || !t2.BlockSize(b2)) { - // return; - // } - // - // const auto &cmp1 = op.conditions[0].comparison; - // SBIterator bounds1(t1.global_sort_state, cmp1); - // SBIterator bounds2(t2.global_sort_state, cmp1); - // - // // t1.X[0] op1 t2.X'[-1] - // bounds1.SetIndex(bounds1.block_capacity * b1); - // bounds2.SetIndex(bounds2.block_capacity * b2 + t2.BlockSize(b2) - 1); - // if (!bounds1.Compare(bounds2)) { - // return; - // } - // - // // 1. let L1 (resp. L2) be the array of column X (resp. Y ) - // const auto &order1 = op.lhs_orders[0][0]; - // const auto &order2 = op.lhs_orders[1][0]; - // - // // 2. if (op1 ∈ {>, ≥}) sort L1 in descending order - // // 3. else if (op1 ∈ {<, ≤}) sort L1 in ascending order - // - // // For the union algorithm, we make a unified table with the keys and the rids as the payload: - // // X/X', Y/Y', R/R'/Li - // // The first position is the sort key. - // vector types; - // types.emplace_back(order2.expression->return_type); - // types.emplace_back(LogicalType::BIGINT); - // RowLayout payload_layout; - // payload_layout.Initialize(types); - // - // // Sort on the first expression - // auto ref = make_uniq(order1.expression->return_type, 0); - // vector orders; - // orders.emplace_back(order1.type, order1.null_order, std::move(ref)); - // - // l1 = make_uniq(context, orders, payload_layout); - // - // // LHS has positive rids - // ExpressionExecutor l_executor(context); - // l_executor.AddExpression(*order1.expression); - // l_executor.AddExpression(*order2.expression); - // AppendKey(t1, l_executor, *l1, 1, 1, b1); - // - // // RHS has negative rids - // ExpressionExecutor r_executor(context); - // r_executor.AddExpression(*op.rhs_orders[0][0].expression); - // r_executor.AddExpression(*op.rhs_orders[1][0].expression); - // AppendKey(t2, r_executor, *l1, -1, -1, b2); - // - // if (l1->global_sort_state.sorted_blocks.empty()) { - // return; - // } - // - // Sort(*l1); - // - // op1 = make_uniq(l1->global_sort_state, cmp1); - // off1 = make_uniq(l1->global_sort_state, cmp1); - // - // // We don't actually need the L1 column, just its sort key, which is in the sort blocks - // li = ExtractColumn(*l1, types.size() - 1); - // - // // 4. if (op2 ∈ {>, ≥}) sort L2 in ascending order - // // 5. else if (op2 ∈ {<, ≤}) sort L2 in descending order - // - // // We sort on Y/Y' to obtain the sort keys and the permutation array. - // // For this we just need a two-column table of Y, P - // types.clear(); - // types.emplace_back(LogicalType::BIGINT); - // payload_layout.Initialize(types); - // - // // Sort on the first expression - // orders.clear(); - // ref = make_uniq(order2.expression->return_type, 0); - // orders.emplace_back(order2.type, order2.null_order, std::move(ref)); - // - // ExpressionExecutor executor(context); - // executor.AddExpression(*orders[0].expression); - // - // l2 = make_uniq(context, orders, payload_layout); - // for (idx_t base = 0, block_idx = 0; block_idx < l1->BlockCount(); ++block_idx) { - // base += AppendKey(*l1, executor, *l2, 1, base, block_idx); - // } - // - // Sort(*l2); - // - // // We don't actually need the L2 column, just its sort key, which is in the sort blocks - // - // // 6. compute the permutation array P of L2 w.r.t. L1 - // p = ExtractColumn(*l2, types.size() - 1); - // - // // 7. initialize bit-array B (|B| = n), and set all bits to 0 - // n = l2->count.load(); - // bit_array.resize(ValidityMask::EntryCount(n), 0); - // bit_mask.Initialize(bit_array.data()); - // - // // Bloom filter - // bloom_count = (n + (BLOOM_CHUNK_BITS - 1)) / BLOOM_CHUNK_BITS; - // bloom_array.resize(ValidityMask::EntryCount(bloom_count), 0); - // bloom_filter.Initialize(bloom_array.data()); - // - // // 11. for(i←1 to n) do - // const auto &cmp2 = op.conditions[1].comparison; - // op2 = make_uniq(l2->global_sort_state, cmp2); - // off2 = make_uniq(l2->global_sort_state, cmp2); - // i = 0; - // j = 0; - // (void)NextRow(); - // } - // - // idx_t IEJoinUnion::SearchL1(idx_t pos) { - // // Perform an exponential search in the appropriate direction - // op1->SetIndex(pos); - // - // idx_t step = 1; - // auto hi = pos; - // auto lo = pos; - // if (!op1->cmp) { - // // Scan left for loose inequality - // lo -= MinValue(step, lo); - // step *= 2; - // off1->SetIndex(lo); - // while (lo > 0 && op1->Compare(*off1)) { - // hi = lo; - // lo -= MinValue(step, lo); - // step *= 2; - // off1->SetIndex(lo); - // } - // } else { - // // Scan right for strict inequality - // hi += MinValue(step, n - hi); - // step *= 2; - // off1->SetIndex(hi); - // while (hi < n && !op1->Compare(*off1)) { - // lo = hi; - // hi += MinValue(step, n - hi); - // step *= 2; - // off1->SetIndex(hi); - // } - // } - // - // // Binary search the target area - // while (lo < hi) { - // const auto mid = lo + (hi - lo) / 2; - // off1->SetIndex(mid); - // if (op1->Compare(*off1)) { - // hi = mid; - // } else { - // lo = mid + 1; - // } - // } - // - // off1->SetIndex(lo); - // - // return lo; - // } - // - // bool IEJoinUnion::NextRow() { - // for (; i < n; ++i) { - // // 12. pos ← P[i] - // auto pos = p[i]; - // lrid = li[pos]; - // if (lrid < 0) { - // continue; - // } - // - // // 16. B[pos] ← 1 - // op2->SetIndex(i); - // for (; off2->GetIndex() < n; ++(*off2)) { - // if (!off2->Compare(*op2)) { - // break; - // } - // const auto p2 = p[off2->GetIndex()]; - // if (li[p2] < 0) { - // // Only mark rhs matches. - // bit_mask.SetValid(p2); - // bloom_filter.SetValid(p2 / BLOOM_CHUNK_BITS); - // } - // } - // - // // 9. if (op1 ∈ {≤,≥} and op2 ∈ {≤,≥}) eqOff = 0 - // // 10. else eqOff = 1 - // // No, because there could be more than one equal value. - // // Find the leftmost off1 where L1[pos] op1 L1[off1..n] - // // These are the rows that satisfy the op1 condition - // // and that is where we should start scanning B from - // j = SearchL1(pos); - // - // return true; - // } - // return false; - // } - // - // static idx_t NextValid(const ValidityMask &bits, idx_t j, const idx_t n) { - // if (j >= n) { - // return n; - // } - // - // // We can do a first approximation by checking entries one at a time - // // which gives 64:1. - // idx_t entry_idx, idx_in_entry; - // bits.GetEntryIndex(j, entry_idx, idx_in_entry); - // auto entry = bits.GetValidityEntry(entry_idx++); - // - // // Trim the bits before the start position - // entry &= (ValidityMask::ValidityBuffer::MAX_ENTRY << idx_in_entry); - // - // // Check the non-ragged entries - // for (const auto entry_count = bits.EntryCount(n); entry_idx < entry_count; ++entry_idx) { - // if (entry) { - // for (; idx_in_entry < bits.BITS_PER_VALUE; ++idx_in_entry, ++j) { - // if (bits.RowIsValid(entry, idx_in_entry)) { - // return j; - // } - // } - // } else { - // j += bits.BITS_PER_VALUE - idx_in_entry; - // } - // - // entry = bits.GetValidityEntry(entry_idx); - // idx_in_entry = 0; - // } - // - // // Check the final entry - // for (; j < n; ++idx_in_entry, ++j) { - // if (bits.RowIsValid(entry, idx_in_entry)) { - // return j; - // } - // } - // - // return j; - // } - // - // idx_t IEJoinUnion::JoinComplexBlocks(SelectionVector &lsel, SelectionVector &rsel) { - // // 8. initialize join result as an empty list for tuple pairs - // idx_t result_count = 0; - // - // // 11. for(i←1 to n) do - // while (i < n) { - // // 13. for (j ← pos+eqOff to n) do - // for (;;) { - // // 14. if B[j] = 1 then - // - // // Use the Bloom filter to find candidate blocks - // while (j < n) { - // auto bloom_begin = NextValid(bloom_filter, j / BLOOM_CHUNK_BITS, bloom_count) * BLOOM_CHUNK_BITS; - // auto bloom_end = MinValue(n, bloom_begin + BLOOM_CHUNK_BITS); - // - // j = MaxValue(j, bloom_begin); - // j = NextValid(bit_mask, j, bloom_end); - // if (j < bloom_end) { - // break; - // } - // } - // - // if (j >= n) { - // break; - // } - // - // // Filter out tuples with the same sign (they come from the same table) - // const auto rrid = li[j]; - // ++j; - // - // // 15. add tuples w.r.t. (L1[j], L1[i]) to join result - // if (lrid > 0 && rrid < 0) { - // lsel.set_index(result_count, sel_t(+lrid - 1)); - // rsel.set_index(result_count, sel_t(-rrid - 1)); - // ++result_count; - // if (result_count == STANDARD_VECTOR_SIZE) { - // // out of space! - // return result_count; - // } - // } - // } - // ++i; - // - // if (!NextRow()) { - // break; - // } - // } - // - // return result_count; - // } - // - class PathFindingLocalSourceState : public LocalSourceState { - public: - explicit PathFindingLocalSourceState(ClientContext &context, const PhysicalPathFinding &op) - : op(op), true_sel(STANDARD_VECTOR_SIZE) { -// left_executor(context), right_executor(context), left_matches(nullptr), right_matches(nullptr) - // auto &allocator = Allocator::Get(context); - // unprojected.Initialize(allocator, op.unprojected_types); - // - // if (op.conditions.size() < 3) { - // return; - // } - // - // vector left_types; - // vector right_types; - // for (idx_t i = 2; i < op.conditions.size(); ++i) { - // const auto &cond = op.conditions[i]; - // - // left_types.push_back(cond.left->return_type); - // left_executor.AddExpression(*cond.left); - // - // right_types.push_back(cond.left->return_type); - // right_executor.AddExpression(*cond.right); - // } - // - // left_keys.Initialize(allocator, left_types); - // right_keys.Initialize(allocator, right_types); - // } - - // idx_t SelectOuterRows(bool *matches) { - // idx_t count = 0; - // for (; outer_idx < outer_count; ++outer_idx) { - // if (!matches[outer_idx]) { - // true_sel.set_index(count++, outer_idx); - // if (count >= STANDARD_VECTOR_SIZE) { - // outer_idx++; - // break; - // } - // } - // } - - // return count; - } - - const PhysicalPathFinding &op; - - // Joining - // unique_ptr joiner; - // - // idx_t left_base; - // idx_t left_block_index; - // - // idx_t right_base; - // idx_t right_block_index; - - // Trailing predicates - SelectionVector true_sel; - - // ExpressionExecutor left_executor; - // DataChunk left_keys; - // - // ExpressionExecutor right_executor; - // DataChunk right_keys; - // - // DataChunk unprojected; - // - // // Outer joins - // idx_t outer_idx; - // idx_t outer_count; - // bool *left_matches; - // bool *right_matches; - }; - - // void PhysicalIEJoin::ResolveComplexJoin(ExecutionContext &context, DataChunk &result, LocalSourceState &state_p) const { - // auto &state = state_p.Cast(); - // auto &ie_sink = sink_state->Cast(); - // auto &left_table = *ie_sink.tables[0]; - // auto &right_table = *ie_sink.tables[1]; - // - // const auto left_cols = children[0]->GetTypes().size(); - // auto &chunk = state.unprojected; - // do { - // SelectionVector lsel(STANDARD_VECTOR_SIZE); - // SelectionVector rsel(STANDARD_VECTOR_SIZE); - // auto result_count = state.joiner->JoinComplexBlocks(lsel, rsel); - // if (result_count == 0) { - // // exhausted this pair - // return; - // } - // - // // found matches: extract them - // - // chunk.Reset(); - // SliceSortedPayload(chunk, left_table.global_sort_state, state.left_block_index, lsel, result_count, 0); - // SliceSortedPayload(chunk, right_table.global_sort_state, state.right_block_index, rsel, result_count, - // left_cols); - // chunk.SetCardinality(result_count); - // - // auto sel = FlatVector::IncrementalSelectionVector(); - // if (conditions.size() > 2) { - // // If there are more expressions to compute, - // // split the result chunk into the left and right halves - // // so we can compute the values for comparison. - // const auto tail_cols = conditions.size() - 2; - // - // DataChunk right_chunk; - // chunk.Split(right_chunk, left_cols); - // state.left_executor.SetChunk(chunk); - // state.right_executor.SetChunk(right_chunk); - // - // auto tail_count = result_count; - // auto true_sel = &state.true_sel; - // for (size_t cmp_idx = 0; cmp_idx < tail_cols; ++cmp_idx) { - // auto &left = state.left_keys.data[cmp_idx]; - // state.left_executor.ExecuteExpression(cmp_idx, left); - // - // auto &right = state.right_keys.data[cmp_idx]; - // state.right_executor.ExecuteExpression(cmp_idx, right); - // - // if (tail_count < result_count) { - // left.Slice(*sel, tail_count); - // right.Slice(*sel, tail_count); - // } - // tail_count = SelectJoinTail(conditions[cmp_idx + 2].comparison, left, right, sel, tail_count, true_sel); - // sel = true_sel; - // } - // chunk.Fuse(right_chunk); - // - // if (tail_count < result_count) { - // result_count = tail_count; - // chunk.Slice(*sel, result_count); - // } - // } - // - // // We need all of the data to compute other predicates, - // // but we only return what is in the projection map - // ProjectResult(chunk, result); - // - // // found matches: mark the found matches if required - // if (left_table.found_match) { - // for (idx_t i = 0; i < result_count; i++) { - // left_table.found_match[state.left_base + lsel[sel->get_index(i)]] = true; - // } - // } - // if (right_table.found_match) { - // for (idx_t i = 0; i < result_count; i++) { - // right_table.found_match[state.right_base + rsel[sel->get_index(i)]] = true; - // } - // } - // result.Verify(); - // } while (result.size() == 0); - // } - - class PathFindingGlobalSourceState : public GlobalSourceState { - public: - explicit PathFindingGlobalSourceState(const PhysicalPathFinding &op) - : op(op), initialized(false) { - } - - void Initialize(PathFindingGlobalState &sink_state) { - lock_guard initializing(lock); - if (initialized) { - return; - } - - // Compute the starting row for reach block - // (In theory these are all the same size, but you never know...) - // auto &left_table = *sink_state.tables[0]; - // const auto left_blocks = left_table.BlockCount(); - // idx_t left_base = 0; - // - // for (size_t lhs = 0; lhs < left_blocks; ++lhs) { - // left_bases.emplace_back(left_base); - // left_base += left_table.BlockSize(lhs); - // } - // - // auto &right_table = *sink_state.tables[1]; - // const auto right_blocks = right_table.BlockCount(); - // idx_t right_base = 0; - // for (size_t rhs = 0; rhs < right_blocks; ++rhs) { - // right_bases.emplace_back(right_base); - // right_base += right_table.BlockSize(rhs); - // } - // - // // Outer join block counts - // if (left_table.found_match) { - // left_outers = left_blocks; - // } - // - // if (right_table.found_match) { - // right_outers = right_blocks; - // } - - // Ready for action - initialized = true; - } - - public: - idx_t MaxThreads() override { - // We can't leverage any more threads than block pairs. - const auto &sink_state = (op.sink_state->Cast()); - return 1; - // return sink_state.tables[0]->BlockCount() * sink_state.tables[1]->BlockCount(); - } - - void GetNextPair(ClientContext &client, PathFindingGlobalState &gstate, PathFindingLocalSourceState &lstate) { - // auto &left_table = *gstate.tables[0]; - // auto &right_table = *gstate.tables[1]; - // - // const auto left_blocks = left_table.BlockCount(); - // const auto right_blocks = right_table.BlockCount(); - // const auto pair_count = left_blocks * right_blocks; - // - // // Regular block - // const auto i = next_pair++; - // if (i < pair_count) { - // const auto b1 = i / right_blocks; - // const auto b2 = i % right_blocks; - // - // lstate.left_block_index = b1; - // lstate.left_base = left_bases[b1]; - // - // lstate.right_block_index = b2; - // lstate.right_base = right_bases[b2]; - // - // lstate.joiner = make_uniq(client, op, left_table, b1, right_table, b2); - // return; - // } - - // // Outer joins - // if (!left_outers && !right_outers) { - // return; - // } - // - // // Spin wait for regular blocks to finish(!) - // while (completed < pair_count) { - // std::this_thread::yield(); - // } - // - // // Left outer blocks - // const auto l = next_left++; - // if (l < left_outers) { - // lstate.joiner = nullptr; - // lstate.left_block_index = l; - // lstate.left_base = left_bases[l]; - // - // lstate.left_matches = left_table.found_match.get() + lstate.left_base; - // lstate.outer_idx = 0; - // lstate.outer_count = left_table.BlockSize(l); - // return; - // } else { - // lstate.left_matches = nullptr; - // } - // - // // Right outer block - // const auto r = next_right++; - // if (r < right_outers) { - // lstate.joiner = nullptr; - // lstate.right_block_index = r; - // lstate.right_base = right_bases[r]; - // - // lstate.right_matches = right_table.found_match.get() + lstate.right_base; - // lstate.outer_idx = 0; - // lstate.outer_count = right_table.BlockSize(r); - // return; - // } else { - // lstate.right_matches = nullptr; - // } - } - - // void PairCompleted(ClientContext &client, PathFindingGlobalState &gstate, PahtFindingLocalSourceState &lstate) { - // lstate.joiner.reset(); - // ++completed; - // GetNextPair(client, gstate, lstate); - // } - - const PhysicalPathFinding &op; - - mutex lock; - bool initialized; - - // Join queue state - // std::atomic next_pair; - // std::atomic completed; - - // Block base row number - // vector left_bases; - // vector right_bases; - - // Outer joins - // idx_t left_outers; - // std::atomic next_left; - - // idx_t right_outers; - // std::atomic next_right; - }; - - unique_ptr PhysicalPathFinding::GetGlobalSourceState(ClientContext &context) const { - return make_uniq(*this); - } - - unique_ptr PhysicalPathFinding::GetLocalSourceState(ExecutionContext &context, - GlobalSourceState &gstate) const { - return make_uniq(context.client, *this); - } - - SourceResultType PhysicalPathFinding::GetData(ExecutionContext &context, DataChunk &result, - OperatorSourceInput &input) const { - auto &pf_sink = sink_state->Cast(); - auto &pf_gstate = input.global_state.Cast(); - auto &pf_lstate = input.local_state.Cast(); - - pf_gstate.Initialize(pf_sink); - - // if (!ie_lstate.joiner && !ie_lstate.left_matches && !ie_lstate.right_matches) { - // ie_gstate.GetNextPair(context.client, ie_sink, ie_lstate); - // } - - // Process INNER results - // while (pf_lstate.joiner) { - // ResolveComplexJoin(context, result, ie_lstate); - // - // if (result.size()) { - // return SourceResultType::HAVE_MORE_OUTPUT; - // } - // - // ie_gstate.PairCompleted(context.client, ie_sink, ie_lstate); - // } - - // Process LEFT OUTER results - // const auto left_cols = children[0]->GetTypes().size(); - // while (ie_lstate.left_matches) { - // const idx_t count = ie_lstate.SelectOuterRows(ie_lstate.left_matches); - // if (!count) { - // ie_gstate.GetNextPair(context.client, ie_sink, ie_lstate); - // continue; - // } - // auto &chunk = ie_lstate.unprojected; - // chunk.Reset(); - // SliceSortedPayload(chunk, ie_sink.tables[0]->global_sort_state, ie_lstate.left_block_index, ie_lstate.true_sel, - // count); - // - // // Fill in NULLs to the right - // for (auto col_idx = left_cols; col_idx < chunk.ColumnCount(); ++col_idx) { - // chunk.data[col_idx].SetVectorType(VectorType::CONSTANT_VECTOR); - // ConstantVector::SetNull(chunk.data[col_idx], true); - // } - // - // ProjectResult(chunk, result); - // result.SetCardinality(count); - // result.Verify(); - // - // return result.size() == 0 ? SourceResultType::FINISHED : SourceResultType::HAVE_MORE_OUTPUT; - // } - - // Process RIGHT OUTER results - // while (ie_lstate.right_matches) { - // const idx_t count = ie_lstate.SelectOuterRows(ie_lstate.right_matches); - // if (!count) { - // ie_gstate.GetNextPair(context.client, ie_sink, ie_lstate); - // continue; - // } - // - // auto &chunk = ie_lstate.unprojected; - // chunk.Reset(); - // SliceSortedPayload(chunk, ie_sink.tables[1]->global_sort_state, ie_lstate.right_block_index, ie_lstate.true_sel, - // count, left_cols); - // - // // Fill in NULLs to the left - // for (idx_t col_idx = 0; col_idx < left_cols; ++col_idx) { - // chunk.data[col_idx].SetVectorType(VectorType::CONSTANT_VECTOR); - // ConstantVector::SetNull(chunk.data[col_idx], true); - // } - // - // ProjectResult(chunk, result); - // result.SetCardinality(count); - // result.Verify(); - // - // break; - // } - - return result.size() == 0 ? SourceResultType::FINISHED : SourceResultType::HAVE_MORE_OUTPUT; - } +// struct PathFindingUnion { +// using SortedTable = PhysicalRangeJoin::GlobalSortedTable; +// +// static idx_t AppendKey(SortedTable &table, ExpressionExecutor +// &executor, SortedTable &marked, int64_t increment, int64_t base, const idx_t +// block_idx); +// +// static void Sort(SortedTable &table) { +// auto &global_sort_state = table.global_sort_state; +// global_sort_state.PrepareMergePhase(); +// while (global_sort_state.sorted_blocks.size() > 1) { +// global_sort_state.InitializeMergeRound(); +// MergeSorter merge_sorter(global_sort_state, +// global_sort_state.buffer_manager); merge_sorter.PerformInMergeRound(); +// global_sort_state.CompleteMergeRound(true); +// } +// } +// +// template +// static vector ExtractColumn(SortedTable &table, idx_t +// col_idx) { vector result; result.reserve(table.count); +// +// auto &gstate = table.global_sort_state; +// auto &blocks = *gstate.sorted_blocks[0]->payload_data; +// PayloadScanner scanner(blocks, gstate, false); +// +// DataChunk payload; +// payload.Initialize(Allocator::DefaultAllocator(), +// gstate.payload_layout.GetTypes()); for (;;) { scanner.Scan(payload); const +// auto count = payload.size(); if (!count) { break; +// } +// +// const auto data_ptr = +// FlatVector::GetData(payload.data[col_idx]); result.insert(result.end(), +// data_ptr, data_ptr + count); +// } +// +// return result; +// } +// +// IEJoinUnion(ClientContext &context, const PhysicalIEJoin &op, +// SortedTable &t1, const idx_t b1, SortedTable &t2, const idx_t b2); +// +// idx_t SearchL1(idx_t pos); +// bool NextRow(); +// +// //! Inverted loop +// idx_t JoinComplexBlocks(SelectionVector &lsel, SelectionVector +// &rsel); +// +// //! L1 +// unique_ptr l1; +// //! L2 +// unique_ptr l2; +// +// //! Li +// vector li; +// //! P +// vector p; +// +// //! B +// vector bit_array; +// ValidityMask bit_mask; +// +// //! Bloom Filter +// static constexpr idx_t BLOOM_CHUNK_BITS = 1024; +// idx_t bloom_count; +// vector bloom_array; +// ValidityMask bloom_filter; +// +// //! Iteration state +// idx_t n; +// idx_t i; +// idx_t j; +// unique_ptr op1; +// unique_ptr off1; +// unique_ptr op2; +// unique_ptr off2; +// int64_t lrid; +// }; + +// idx_t IEJoinUnion::AppendKey(SortedTable &table, ExpressionExecutor +// &executor, SortedTable &marked, int64_t increment, int64_t base, const idx_t +// block_idx) { LocalSortState local_sort_state; +// local_sort_state.Initialize(marked.global_sort_state, +// marked.global_sort_state.buffer_manager); +// +// // Reading +// const auto valid = table.count - table.has_null; +// auto &gstate = table.global_sort_state; +// PayloadScanner scanner(gstate, block_idx); +// auto table_idx = block_idx * gstate.block_capacity; +// +// DataChunk scanned; +// scanned.Initialize(Allocator::DefaultAllocator(), +// scanner.GetPayloadTypes()); +// +// // Writing +// auto types = local_sort_state.sort_layout->logical_types; +// const idx_t payload_idx = types.size(); +// +// const auto &payload_types = local_sort_state.payload_layout->GetTypes(); +// types.insert(types.end(), payload_types.begin(), payload_types.end()); +// const idx_t rid_idx = types.size() - 1; +// +// DataChunk keys; +// DataChunk payload; +// keys.Initialize(Allocator::DefaultAllocator(), types); +// +// idx_t inserted = 0; +// for (auto rid = base; table_idx < valid;) { +// scanner.Scan(scanned); +// +// // NULLs are at the end, so stop when we reach them +// auto scan_count = scanned.size(); +// if (table_idx + scan_count > valid) { +// scan_count = valid - table_idx; +// scanned.SetCardinality(scan_count); +// } +// if (scan_count == 0) { +// break; +// } +// table_idx += scan_count; +// +// // Compute the input columns from the payload +// keys.Reset(); +// keys.Split(payload, rid_idx); +// executor.Execute(scanned, keys); +// +// // Mark the rid column +// payload.data[0].Sequence(rid, increment, scan_count); +// payload.SetCardinality(scan_count); +// keys.Fuse(payload); +// rid += increment * scan_count; +// +// // Sort on the sort columns (which will no longer be needed) +// keys.Split(payload, payload_idx); +// local_sort_state.SinkChunk(keys, payload); +// inserted += scan_count; +// keys.Fuse(payload); +// +// // Flush when we have enough data +// if (local_sort_state.SizeInBytes() >= marked.memory_per_thread) +// { local_sort_state.Sort(marked.global_sort_state, true); +// } +// } +// marked.global_sort_state.AddLocalState(local_sort_state); +// marked.count += inserted; +// +// return inserted; +// } +// +// IEJoinUnion::IEJoinUnion(ClientContext &context, const PhysicalIEJoin &op, +// SortedTable &t1, const idx_t b1, SortedTable &t2, const idx_t b2) : n(0), +// i(0) { +// // input : query Q with 2 join predicates t1.X op1 t2.X' and t1.Y op2 +// t2.Y', tables T, T' of sizes m and n resp. +// // output: a list of tuple pairs (ti , tj) +// // Note that T/T' are already sorted on X/X' and contain the payload +// data +// // We only join the two block numbers and use the sizes of the blocks as +// the counts +// +// // 0. Filter out tables with no overlap +// if (!t1.BlockSize(b1) || !t2.BlockSize(b2)) { +// return; +// } +// +// const auto &cmp1 = op.conditions[0].comparison; +// SBIterator bounds1(t1.global_sort_state, cmp1); +// SBIterator bounds2(t2.global_sort_state, cmp1); +// +// // t1.X[0] op1 t2.X'[-1] +// bounds1.SetIndex(bounds1.block_capacity * b1); +// bounds2.SetIndex(bounds2.block_capacity * b2 + t2.BlockSize(b2) - 1); +// if (!bounds1.Compare(bounds2)) { +// return; +// } +// +// // 1. let L1 (resp. L2) be the array of column X (resp. Y ) +// const auto &order1 = op.lhs_orders[0][0]; +// const auto &order2 = op.lhs_orders[1][0]; +// +// // 2. if (op1 ∈ {>, ≥}) sort L1 in descending order +// // 3. else if (op1 ∈ {<, ≤}) sort L1 in ascending order +// +// // For the union algorithm, we make a unified table with the keys and +// the rids as the payload: +// // X/X', Y/Y', R/R'/Li +// // The first position is the sort key. +// vector types; +// types.emplace_back(order2.expression->return_type); +// types.emplace_back(LogicalType::BIGINT); +// RowLayout payload_layout; +// payload_layout.Initialize(types); +// +// // Sort on the first expression +// auto ref = +// make_uniq(order1.expression->return_type, 0); +// vector orders; +// orders.emplace_back(order1.type, order1.null_order, std::move(ref)); +// +// l1 = make_uniq(context, orders, payload_layout); +// +// // LHS has positive rids +// ExpressionExecutor l_executor(context); +// l_executor.AddExpression(*order1.expression); +// l_executor.AddExpression(*order2.expression); +// AppendKey(t1, l_executor, *l1, 1, 1, b1); +// +// // RHS has negative rids +// ExpressionExecutor r_executor(context); +// r_executor.AddExpression(*op.rhs_orders[0][0].expression); +// r_executor.AddExpression(*op.rhs_orders[1][0].expression); +// AppendKey(t2, r_executor, *l1, -1, -1, b2); +// +// if (l1->global_sort_state.sorted_blocks.empty()) { +// return; +// } +// +// Sort(*l1); +// +// op1 = make_uniq(l1->global_sort_state, cmp1); +// off1 = make_uniq(l1->global_sort_state, cmp1); +// +// // We don't actually need the L1 column, just its sort key, which is in +// the sort blocks li = ExtractColumn(*l1, types.size() - 1); +// +// // 4. if (op2 ∈ {>, ≥}) sort L2 in ascending order +// // 5. else if (op2 ∈ {<, ≤}) sort L2 in descending order +// +// // We sort on Y/Y' to obtain the sort keys and the permutation array. +// // For this we just need a two-column table of Y, P +// types.clear(); +// types.emplace_back(LogicalType::BIGINT); +// payload_layout.Initialize(types); +// +// // Sort on the first expression +// orders.clear(); +// ref = +// make_uniq(order2.expression->return_type, 0); +// orders.emplace_back(order2.type, order2.null_order, std::move(ref)); +// +// ExpressionExecutor executor(context); +// executor.AddExpression(*orders[0].expression); +// +// l2 = make_uniq(context, orders, payload_layout); +// for (idx_t base = 0, block_idx = 0; block_idx < l1->BlockCount(); +// ++block_idx) { base += AppendKey(*l1, executor, *l2, 1, base, block_idx); +// } +// +// Sort(*l2); +// +// // We don't actually need the L2 column, just its sort key, which is in +// the sort blocks +// +// // 6. compute the permutation array P of L2 w.r.t. L1 +// p = ExtractColumn(*l2, types.size() - 1); +// +// // 7. initialize bit-array B (|B| = n), and set all bits to 0 +// n = l2->count.load(); +// bit_array.resize(ValidityMask::EntryCount(n), 0); +// bit_mask.Initialize(bit_array.data()); +// +// // Bloom filter +// bloom_count = (n + (BLOOM_CHUNK_BITS - 1)) / BLOOM_CHUNK_BITS; +// bloom_array.resize(ValidityMask::EntryCount(bloom_count), 0); +// bloom_filter.Initialize(bloom_array.data()); +// +// // 11. for(i←1 to n) do +// const auto &cmp2 = op.conditions[1].comparison; +// op2 = make_uniq(l2->global_sort_state, cmp2); +// off2 = make_uniq(l2->global_sort_state, cmp2); +// i = 0; +// j = 0; +// (void)NextRow(); +// } +// +// idx_t IEJoinUnion::SearchL1(idx_t pos) { +// // Perform an exponential search in the appropriate direction +// op1->SetIndex(pos); +// +// idx_t step = 1; +// auto hi = pos; +// auto lo = pos; +// if (!op1->cmp) { +// // Scan left for loose inequality +// lo -= MinValue(step, lo); +// step *= 2; +// off1->SetIndex(lo); +// while (lo > 0 && op1->Compare(*off1)) { +// hi = lo; +// lo -= MinValue(step, lo); +// step *= 2; +// off1->SetIndex(lo); +// } +// } else { +// // Scan right for strict inequality +// hi += MinValue(step, n - hi); +// step *= 2; +// off1->SetIndex(hi); +// while (hi < n && !op1->Compare(*off1)) { +// lo = hi; +// hi += MinValue(step, n - hi); +// step *= 2; +// off1->SetIndex(hi); +// } +// } +// +// // Binary search the target area +// while (lo < hi) { +// const auto mid = lo + (hi - lo) / 2; +// off1->SetIndex(mid); +// if (op1->Compare(*off1)) { +// hi = mid; +// } else { +// lo = mid + 1; +// } +// } +// +// off1->SetIndex(lo); +// +// return lo; +// } +// +// bool IEJoinUnion::NextRow() { +// for (; i < n; ++i) { +// // 12. pos ← P[i] +// auto pos = p[i]; +// lrid = li[pos]; +// if (lrid < 0) { +// continue; +// } +// +// // 16. B[pos] ← 1 +// op2->SetIndex(i); +// for (; off2->GetIndex() < n; ++(*off2)) { +// if (!off2->Compare(*op2)) { +// break; +// } +// const auto p2 = p[off2->GetIndex()]; +// if (li[p2] < 0) { +// // Only mark rhs matches. +// bit_mask.SetValid(p2); +// bloom_filter.SetValid(p2 / BLOOM_CHUNK_BITS); +// } +// } +// +// // 9. if (op1 ∈ {≤,≥} and op2 ∈ {≤,≥}) eqOff = 0 +// // 10. else eqOff = 1 +// // No, because there could be more than one equal value. +// // Find the leftmost off1 where L1[pos] op1 L1[off1..n] +// // These are the rows that satisfy the op1 condition +// // and that is where we should start scanning B from +// j = SearchL1(pos); +// +// return true; +// } +// return false; +// } +// +// static idx_t NextValid(const ValidityMask &bits, idx_t j, const idx_t n) { +// if (j >= n) { +// return n; +// } +// +// // We can do a first approximation by checking entries one at a time +// // which gives 64:1. +// idx_t entry_idx, idx_in_entry; +// bits.GetEntryIndex(j, entry_idx, idx_in_entry); +// auto entry = bits.GetValidityEntry(entry_idx++); +// +// // Trim the bits before the start position +// entry &= (ValidityMask::ValidityBuffer::MAX_ENTRY << idx_in_entry); +// +// // Check the non-ragged entries +// for (const auto entry_count = bits.EntryCount(n); entry_idx < +// entry_count; ++entry_idx) { if (entry) { for (; idx_in_entry < +// bits.BITS_PER_VALUE; ++idx_in_entry, ++j) { if (bits.RowIsValid(entry, +// idx_in_entry)) { return j; +// } +// } +// } else { +// j += bits.BITS_PER_VALUE - idx_in_entry; +// } +// +// entry = bits.GetValidityEntry(entry_idx); +// idx_in_entry = 0; +// } +// +// // Check the final entry +// for (; j < n; ++idx_in_entry, ++j) { +// if (bits.RowIsValid(entry, idx_in_entry)) { +// return j; +// } +// } +// +// return j; +// } +// +// idx_t IEJoinUnion::JoinComplexBlocks(SelectionVector &lsel, SelectionVector +// &rsel) { +// // 8. initialize join result as an empty list for tuple pairs +// idx_t result_count = 0; +// +// // 11. for(i←1 to n) do +// while (i < n) { +// // 13. for (j ← pos+eqOff to n) do +// for (;;) { +// // 14. if B[j] = 1 then +// +// // Use the Bloom filter to find candidate blocks +// while (j < n) { +// auto bloom_begin = NextValid(bloom_filter, j / +// BLOOM_CHUNK_BITS, bloom_count) * BLOOM_CHUNK_BITS; auto bloom_end = +// MinValue(n, bloom_begin + BLOOM_CHUNK_BITS); +// +// j = MaxValue(j, bloom_begin); +// j = NextValid(bit_mask, j, bloom_end); +// if (j < bloom_end) { +// break; +// } +// } +// +// if (j >= n) { +// break; +// } +// +// // Filter out tuples with the same sign (they come from +// the same table) const auto rrid = li[j]; +// ++j; +// +// // 15. add tuples w.r.t. (L1[j], L1[i]) to join result +// if (lrid > 0 && rrid < 0) { +// lsel.set_index(result_count, sel_t(+lrid - 1)); +// rsel.set_index(result_count, sel_t(-rrid - 1)); +// ++result_count; +// if (result_count == STANDARD_VECTOR_SIZE) { +// // out of space! +// return result_count; +// } +// } +// } +// ++i; +// +// if (!NextRow()) { +// break; +// } +// } +// +// return result_count; +// } +// +class PathFindingLocalSourceState : public LocalSourceState { +public: + explicit PathFindingLocalSourceState(ClientContext &context, + const PhysicalPathFinding &op) + : op(op), true_sel(STANDARD_VECTOR_SIZE) { + // left_executor(context), right_executor(context), left_matches(nullptr), + // right_matches(nullptr) auto &allocator = Allocator::Get(context); + // unprojected.Initialize(allocator, op.unprojected_types); + // + // if (op.conditions.size() < 3) { + // return; + // } + // + // vector left_types; + // vector right_types; + // for (idx_t i = 2; i < op.conditions.size(); ++i) { + // const auto &cond = op.conditions[i]; + // + // left_types.push_back(cond.left->return_type); + // left_executor.AddExpression(*cond.left); + // + // right_types.push_back(cond.left->return_type); + // right_executor.AddExpression(*cond.right); + // } + // + // left_keys.Initialize(allocator, left_types); + // right_keys.Initialize(allocator, right_types); + // } + + // idx_t SelectOuterRows(bool *matches) { + // idx_t count = 0; + // for (; outer_idx < outer_count; ++outer_idx) { + // if (!matches[outer_idx]) { + // true_sel.set_index(count++, outer_idx); + // if (count >= STANDARD_VECTOR_SIZE) { + // outer_idx++; + // break; + // } + // } + // } + + // return count; + } + + const PhysicalPathFinding &op; + + // Joining + // unique_ptr joiner; + // + // idx_t left_base; + // idx_t left_block_index; + // + // idx_t right_base; + // idx_t right_block_index; + + // Trailing predicates + SelectionVector true_sel; + + // ExpressionExecutor left_executor; + // DataChunk left_keys; + // + // ExpressionExecutor right_executor; + // DataChunk right_keys; + // + // DataChunk unprojected; + // + // // Outer joins + // idx_t outer_idx; + // idx_t outer_count; + // bool *left_matches; + // bool *right_matches; +}; + +// void PhysicalIEJoin::ResolveComplexJoin(ExecutionContext &context, DataChunk +// &result, LocalSourceState &state_p) const { auto &state = +// state_p.Cast(); auto &ie_sink = +// sink_state->Cast(); auto &left_table = *ie_sink.tables[0]; +// auto &right_table = *ie_sink.tables[1]; +// +// const auto left_cols = children[0]->GetTypes().size(); +// auto &chunk = state.unprojected; +// do { +// SelectionVector lsel(STANDARD_VECTOR_SIZE); +// SelectionVector rsel(STANDARD_VECTOR_SIZE); +// auto result_count = state.joiner->JoinComplexBlocks(lsel, rsel); +// if (result_count == 0) { +// // exhausted this pair +// return; +// } +// +// // found matches: extract them +// +// chunk.Reset(); +// SliceSortedPayload(chunk, left_table.global_sort_state, +// state.left_block_index, lsel, result_count, 0); SliceSortedPayload(chunk, +// right_table.global_sort_state, state.right_block_index, rsel, result_count, +// left_cols); +// chunk.SetCardinality(result_count); +// +// auto sel = FlatVector::IncrementalSelectionVector(); +// if (conditions.size() > 2) { +// // If there are more expressions to compute, +// // split the result chunk into the left and right halves +// // so we can compute the values for comparison. +// const auto tail_cols = conditions.size() - 2; +// +// DataChunk right_chunk; +// chunk.Split(right_chunk, left_cols); +// state.left_executor.SetChunk(chunk); +// state.right_executor.SetChunk(right_chunk); +// +// auto tail_count = result_count; +// auto true_sel = &state.true_sel; +// for (size_t cmp_idx = 0; cmp_idx < tail_cols; ++cmp_idx) +// { auto &left = state.left_keys.data[cmp_idx]; +// state.left_executor.ExecuteExpression(cmp_idx, +// left); +// +// auto &right = state.right_keys.data[cmp_idx]; +// state.right_executor.ExecuteExpression(cmp_idx, +// right); +// +// if (tail_count < result_count) { +// left.Slice(*sel, tail_count); +// right.Slice(*sel, tail_count); +// } +// tail_count = SelectJoinTail(conditions[cmp_idx + +// 2].comparison, left, right, sel, tail_count, true_sel); sel = true_sel; +// } +// chunk.Fuse(right_chunk); +// +// if (tail_count < result_count) { +// result_count = tail_count; +// chunk.Slice(*sel, result_count); +// } +// } +// +// // We need all of the data to compute other predicates, +// // but we only return what is in the projection map +// ProjectResult(chunk, result); +// +// // found matches: mark the found matches if required +// if (left_table.found_match) { +// for (idx_t i = 0; i < result_count; i++) { +// left_table.found_match[state.left_base + +// lsel[sel->get_index(i)]] = true; +// } +// } +// if (right_table.found_match) { +// for (idx_t i = 0; i < result_count; i++) { +// right_table.found_match[state.right_base + +// rsel[sel->get_index(i)]] = true; +// } +// } +// result.Verify(); +// } while (result.size() == 0); +// } + +class PathFindingGlobalSourceState : public GlobalSourceState { +public: + explicit PathFindingGlobalSourceState(const PhysicalPathFinding &op) + : op(op), initialized(false) {} + + void Initialize(PathFindingGlobalState &sink_state) { + lock_guard initializing(lock); + if (initialized) { + return; + } + + // Compute the starting row for reach block + // (In theory these are all the same size, but you never know...) + // auto &left_table = *sink_state.tables[0]; + // const auto left_blocks = left_table.BlockCount(); + // idx_t left_base = 0; + // + // for (size_t lhs = 0; lhs < left_blocks; ++lhs) { + // left_bases.emplace_back(left_base); + // left_base += left_table.BlockSize(lhs); + // } + // + // auto &right_table = *sink_state.tables[1]; + // const auto right_blocks = right_table.BlockCount(); + // idx_t right_base = 0; + // for (size_t rhs = 0; rhs < right_blocks; ++rhs) { + // right_bases.emplace_back(right_base); + // right_base += right_table.BlockSize(rhs); + // } + // + // // Outer join block counts + // if (left_table.found_match) { + // left_outers = left_blocks; + // } + // + // if (right_table.found_match) { + // right_outers = right_blocks; + // } + + // Ready for action + initialized = true; + } + +public: + idx_t MaxThreads() override { + // We can't leverage any more threads than block pairs. + const auto &sink_state = (op.sink_state->Cast()); + return 1; + // return sink_state.tables[0]->BlockCount() * + // sink_state.tables[1]->BlockCount(); + } + + void GetNextPair(ClientContext &client, PathFindingGlobalState &gstate, + PathFindingLocalSourceState &lstate) { + // auto &left_table = *gstate.tables[0]; + // auto &right_table = *gstate.tables[1]; + // + // const auto left_blocks = left_table.BlockCount(); + // const auto right_blocks = right_table.BlockCount(); + // const auto pair_count = left_blocks * right_blocks; + // + // // Regular block + // const auto i = next_pair++; + // if (i < pair_count) { + // const auto b1 = i / right_blocks; + // const auto b2 = i % right_blocks; + // + // lstate.left_block_index = b1; + // lstate.left_base = left_bases[b1]; + // + // lstate.right_block_index = b2; + // lstate.right_base = right_bases[b2]; + // + // lstate.joiner = make_uniq(client, op, left_table, b1, + // right_table, b2); return; + // } + + // // Outer joins + // if (!left_outers && !right_outers) { + // return; + // } + // + // // Spin wait for regular blocks to finish(!) + // while (completed < pair_count) { + // std::this_thread::yield(); + // } + // + // // Left outer blocks + // const auto l = next_left++; + // if (l < left_outers) { + // lstate.joiner = nullptr; + // lstate.left_block_index = l; + // lstate.left_base = left_bases[l]; + // + // lstate.left_matches = left_table.found_match.get() + lstate.left_base; + // lstate.outer_idx = 0; + // lstate.outer_count = left_table.BlockSize(l); + // return; + // } else { + // lstate.left_matches = nullptr; + // } + // + // // Right outer block + // const auto r = next_right++; + // if (r < right_outers) { + // lstate.joiner = nullptr; + // lstate.right_block_index = r; + // lstate.right_base = right_bases[r]; + // + // lstate.right_matches = right_table.found_match.get() + + // lstate.right_base; lstate.outer_idx = 0; lstate.outer_count = + // right_table.BlockSize(r); return; } else { lstate.right_matches = + // nullptr; + // } + } + + // void PairCompleted(ClientContext &client, PathFindingGlobalState &gstate, + // PahtFindingLocalSourceState &lstate) { lstate.joiner.reset(); + // ++completed; + // GetNextPair(client, gstate, lstate); + // } + + const PhysicalPathFinding &op; + + mutex lock; + bool initialized; + + // Join queue state + // std::atomic next_pair; + // std::atomic completed; + + // Block base row number + // vector left_bases; + // vector right_bases; + + // Outer joins + // idx_t left_outers; + // std::atomic next_left; + + // idx_t right_outers; + // std::atomic next_right; +}; + +unique_ptr +PhysicalPathFinding::GetGlobalSourceState(ClientContext &context) const { + return make_uniq(*this); +} + +unique_ptr +PhysicalPathFinding::GetLocalSourceState(ExecutionContext &context, + GlobalSourceState &gstate) const { + return make_uniq(context.client, *this); +} + +SourceResultType +PhysicalPathFinding::GetData(ExecutionContext &context, DataChunk &result, + OperatorSourceInput &input) const { + auto &pf_sink = sink_state->Cast(); + auto &pf_gstate = input.global_state.Cast(); + auto &pf_lstate = input.local_state.Cast(); + + pf_gstate.Initialize(pf_sink); + + // if (!ie_lstate.joiner && !ie_lstate.left_matches && + // !ie_lstate.right_matches) { ie_gstate.GetNextPair(context.client, ie_sink, + // ie_lstate); + // } + + // Process INNER results + // while (pf_lstate.joiner) { + // ResolveComplexJoin(context, result, ie_lstate); + // + // if (result.size()) { + // return SourceResultType::HAVE_MORE_OUTPUT; + // } + // + // ie_gstate.PairCompleted(context.client, ie_sink, ie_lstate); + // } + + // Process LEFT OUTER results + // const auto left_cols = children[0]->GetTypes().size(); + // while (ie_lstate.left_matches) { + // const idx_t count = ie_lstate.SelectOuterRows(ie_lstate.left_matches); + // if (!count) { + // ie_gstate.GetNextPair(context.client, ie_sink, ie_lstate); + // continue; + // } + // auto &chunk = ie_lstate.unprojected; + // chunk.Reset(); + // SliceSortedPayload(chunk, ie_sink.tables[0]->global_sort_state, + // ie_lstate.left_block_index, ie_lstate.true_sel, count); + // + // // Fill in NULLs to the right + // for (auto col_idx = left_cols; col_idx < chunk.ColumnCount(); ++col_idx) + // { chunk.data[col_idx].SetVectorType(VectorType::CONSTANT_VECTOR); + // ConstantVector::SetNull(chunk.data[col_idx], true); + // } + // + // ProjectResult(chunk, result); + // result.SetCardinality(count); + // result.Verify(); + // + // return result.size() == 0 ? SourceResultType::FINISHED : + // SourceResultType::HAVE_MORE_OUTPUT; + // } + + // Process RIGHT OUTER results + // while (ie_lstate.right_matches) { + // const idx_t count = ie_lstate.SelectOuterRows(ie_lstate.right_matches); + // if (!count) { + // ie_gstate.GetNextPair(context.client, ie_sink, ie_lstate); + // continue; + // } + // + // auto &chunk = ie_lstate.unprojected; + // chunk.Reset(); + // SliceSortedPayload(chunk, ie_sink.tables[1]->global_sort_state, + // ie_lstate.right_block_index, ie_lstate.true_sel, count, left_cols); + // + // // Fill in NULLs to the left + // for (idx_t col_idx = 0; col_idx < left_cols; ++col_idx) { + // chunk.data[col_idx].SetVectorType(VectorType::CONSTANT_VECTOR); + // ConstantVector::SetNull(chunk.data[col_idx], true); + // } + // + // ProjectResult(chunk, result); + // result.SetCardinality(count); + // result.Verify(); + // + // break; + // } + + return result.size() == 0 ? SourceResultType::FINISHED + : SourceResultType::HAVE_MORE_OUTPUT; +} //===--------------------------------------------------------------------===// // Pipeline Construction //===--------------------------------------------------------------------===// - void PhysicalPathFinding::BuildPipelines(Pipeline ¤t, MetaPipeline &meta_pipeline) { - D_ASSERT(children.size() == 2); - if (meta_pipeline.HasRecursiveCTE()) { - throw NotImplementedException("Path Finding is not supported in recursive CTEs yet"); - } - - // becomes a source after both children fully sink their data - meta_pipeline.GetState().SetPipelineSource(current, *this); - - // Create one child meta pipeline that will hold the LHS and RHS pipelines - auto &child_meta_pipeline = meta_pipeline.CreateChildMetaPipeline(current, *this); - - // Build out LHS - auto lhs_pipeline = child_meta_pipeline.GetBasePipeline(); - children[0]->BuildPipelines(*lhs_pipeline, child_meta_pipeline); - - // Build out RHS - auto rhs_pipeline = child_meta_pipeline.CreatePipeline(); - children[1]->BuildPipelines(*rhs_pipeline, child_meta_pipeline); - - // Despite having the same sink, RHS and everything created after it need their own (same) PipelineFinishEvent - child_meta_pipeline.AddFinishEvent(rhs_pipeline); - } +void PhysicalPathFinding::BuildPipelines(Pipeline ¤t, + MetaPipeline &meta_pipeline) { + D_ASSERT(children.size() == 2); + if (meta_pipeline.HasRecursiveCTE()) { + throw NotImplementedException( + "Path Finding is not supported in recursive CTEs yet"); + } + + // becomes a source after both children fully sink their data + meta_pipeline.GetState().SetPipelineSource(current, *this); + + // Create one child meta pipeline that will hold the LHS and RHS pipelines + auto &child_meta_pipeline = + meta_pipeline.CreateChildMetaPipeline(current, *this); + + // Build out LHS + auto lhs_pipeline = child_meta_pipeline.GetBasePipeline(); + children[0]->BuildPipelines(*lhs_pipeline, child_meta_pipeline); + + // Build out RHS + auto &rhs_pipeline = child_meta_pipeline.CreatePipeline(); + children[1]->BuildPipelines(rhs_pipeline, child_meta_pipeline); + + // Despite having the same sink, RHS and everything created after it need + // their own (same) PipelineFinishEvent + child_meta_pipeline.AddFinishEvent(rhs_pipeline); +} } // namespace duckdb From 97d65450139dad27bfa31f2d440cef6ae7fa31ff Mon Sep 17 00:00:00 2001 From: dtenwolde Date: Thu, 15 Feb 2024 13:01:20 +0100 Subject: [PATCH 009/249] Change to range join type --- duckdb-pgq | 2 +- .../duckpgq/operators/physical_path_finding.hpp | 8 ++++++-- .../src/duckpgq/operators/physical_path_finding.cpp | 11 +++++++---- 3 files changed, 14 insertions(+), 7 deletions(-) diff --git a/duckdb-pgq b/duckdb-pgq index 3e6d77b9..39dc9f0f 160000 --- a/duckdb-pgq +++ b/duckdb-pgq @@ -1 +1 @@ -Subproject commit 3e6d77b92397f0b2ae70424a51c3c8c22c3c6123 +Subproject commit 39dc9f0fed3fc7d6228bce1c3c4b4181cd8d4dc9 diff --git a/duckpgq/include/duckpgq/operators/physical_path_finding.hpp b/duckpgq/include/duckpgq/operators/physical_path_finding.hpp index 67e966d0..be667136 100644 --- a/duckpgq/include/duckpgq/operators/physical_path_finding.hpp +++ b/duckpgq/include/duckpgq/operators/physical_path_finding.hpp @@ -12,8 +12,10 @@ #include "duckdb/execution/physical_operator.hpp" +#include + namespace duckdb { -class PhysicalPathFinding : public CachingPhysicalOperator { +class PhysicalPathFinding : public PhysicalComparisonJoin { public: static constexpr const PhysicalOperatorType TYPE = PhysicalOperatorType::EXTENSION; @@ -21,7 +23,9 @@ class PhysicalPathFinding : public CachingPhysicalOperator { public: PhysicalPathFinding(LogicalExtensionOperator &op, unique_ptr left, - unique_ptr right); + unique_ptr right, + vector cond, JoinType join_type, + idx_t estimated_cardinality); // vector join_key_types; // vector> lhs_orders; diff --git a/duckpgq/src/duckpgq/operators/physical_path_finding.cpp b/duckpgq/src/duckpgq/operators/physical_path_finding.cpp index 863331cc..223ea10d 100644 --- a/duckpgq/src/duckpgq/operators/physical_path_finding.cpp +++ b/duckpgq/src/duckpgq/operators/physical_path_finding.cpp @@ -15,8 +15,11 @@ namespace duckdb { PhysicalPathFinding::PhysicalPathFinding(LogicalExtensionOperator &op, unique_ptr left, - unique_ptr right) - : CachingPhysicalOperator(TYPE, op.types, 0) { + unique_ptr right, + vector cond, + JoinType join_type, + idx_t estimated_cardinality) + : PhysicalComparisonJoin(op, TYPE, std::move(cond), join_type, estimated_cardinality) { children.push_back(std::move(left)); children.push_back(std::move(right)); } @@ -1038,8 +1041,8 @@ void PhysicalPathFinding::BuildPipelines(Pipeline ¤t, children[0]->BuildPipelines(*lhs_pipeline, child_meta_pipeline); // Build out RHS - auto &rhs_pipeline = child_meta_pipeline.CreatePipeline(); - children[1]->BuildPipelines(rhs_pipeline, child_meta_pipeline); + auto rhs_pipeline = child_meta_pipeline.CreatePipeline(); + children[1]->BuildPipelines(*rhs_pipeline, child_meta_pipeline); // Despite having the same sink, RHS and everything created after it need // their own (same) PipelineFinishEvent From d016338c6440d4b95b712eb6f102c1b0ac822e66 Mon Sep 17 00:00:00 2001 From: dtenwolde Date: Thu, 15 Feb 2024 13:40:10 +0100 Subject: [PATCH 010/249] Removed returning PhysicaLPathFinding in planning comparison join --- .../duckpgq/operators/physical_path_finding.hpp | 3 --- .../duckpgq/operators/path_finding_operator.cpp | 17 ++++++++--------- .../duckpgq/operators/physical_path_finding.cpp | 2 +- 3 files changed, 9 insertions(+), 13 deletions(-) diff --git a/duckpgq/include/duckpgq/operators/physical_path_finding.hpp b/duckpgq/include/duckpgq/operators/physical_path_finding.hpp index be667136..d3760e07 100644 --- a/duckpgq/include/duckpgq/operators/physical_path_finding.hpp +++ b/duckpgq/include/duckpgq/operators/physical_path_finding.hpp @@ -8,10 +8,7 @@ #pragma once -#include - #include "duckdb/execution/physical_operator.hpp" - #include namespace duckdb { diff --git a/duckpgq/src/duckpgq/operators/path_finding_operator.cpp b/duckpgq/src/duckpgq/operators/path_finding_operator.cpp index 2d88845a..8e81217c 100644 --- a/duckpgq/src/duckpgq/operators/path_finding_operator.cpp +++ b/duckpgq/src/duckpgq/operators/path_finding_operator.cpp @@ -7,14 +7,13 @@ namespace duckdb { unique_ptr PathFindingOperator::CreatePlan(ClientContext &, duckdb::PhysicalPlanGenerator &generator) { - D_ASSERT(children.size() == 2); - auto left = generator.CreatePlan(std::move(children[0])); - auto right = generator.CreatePlan(std::move(children[1])); - - auto result = - duckdb::make_uniq_base( - *this, std::move(left), std::move(right)); - result->children.emplace_back(std::move(result)); - return result; + // D_ASSERT(children.size() == 2); + // auto left = generator.CreatePlan(std::move(children[0])); + // auto right = generator.CreatePlan(std::move(children[1])); + // + // auto result = + // duckdb::make_uniq_base(*this); + // result->children.emplace_back(std::move(result)); + // return result; } } // namespace duckdb diff --git a/duckpgq/src/duckpgq/operators/physical_path_finding.cpp b/duckpgq/src/duckpgq/operators/physical_path_finding.cpp index 223ea10d..3412db5d 100644 --- a/duckpgq/src/duckpgq/operators/physical_path_finding.cpp +++ b/duckpgq/src/duckpgq/operators/physical_path_finding.cpp @@ -7,7 +7,7 @@ #include "duckdb/parallel/meta_pipeline.hpp" #include "duckdb/parallel/thread_context.hpp" -#include +#include #include #include From a71a0d81208ce06bfe65d2796686db8675081867 Mon Sep 17 00:00:00 2001 From: dtenwolde Date: Thu, 15 Feb 2024 13:40:15 +0100 Subject: [PATCH 011/249] Removed returning PhysicaLPathFinding in planning comparison join --- duckdb-pgq | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/duckdb-pgq b/duckdb-pgq index 39dc9f0f..f85aa91d 160000 --- a/duckdb-pgq +++ b/duckdb-pgq @@ -1 +1 @@ -Subproject commit 39dc9f0fed3fc7d6228bce1c3c4b4181cd8d4dc9 +Subproject commit f85aa91d405c3d6e0eca1bb09238824af663c7af From 43e2423f02efaf3e1f1f15f75d55d2a819236254 Mon Sep 17 00:00:00 2001 From: dtenwolde Date: Thu, 15 Feb 2024 13:47:18 +0100 Subject: [PATCH 012/249] Remove some commented code --- .../operators/physical_path_finding.hpp | 8 ++-- .../operators/physical_path_finding.cpp | 48 ++----------------- 2 files changed, 8 insertions(+), 48 deletions(-) diff --git a/duckpgq/include/duckpgq/operators/physical_path_finding.hpp b/duckpgq/include/duckpgq/operators/physical_path_finding.hpp index d3760e07..afaf5f12 100644 --- a/duckpgq/include/duckpgq/operators/physical_path_finding.hpp +++ b/duckpgq/include/duckpgq/operators/physical_path_finding.hpp @@ -9,10 +9,10 @@ #pragma once #include "duckdb/execution/physical_operator.hpp" -#include +#include "duckdb/planner/operator/logical_extension_operator.hpp" namespace duckdb { -class PhysicalPathFinding : public PhysicalComparisonJoin { +class PhysicalPathFinding : public CachingPhysicalOperator { public: static constexpr const PhysicalOperatorType TYPE = PhysicalOperatorType::EXTENSION; @@ -20,9 +20,7 @@ class PhysicalPathFinding : public PhysicalComparisonJoin { public: PhysicalPathFinding(LogicalExtensionOperator &op, unique_ptr left, - unique_ptr right, - vector cond, JoinType join_type, - idx_t estimated_cardinality); + unique_ptr right); // vector join_key_types; // vector> lhs_orders; diff --git a/duckpgq/src/duckpgq/operators/physical_path_finding.cpp b/duckpgq/src/duckpgq/operators/physical_path_finding.cpp index 3412db5d..9509aa2c 100644 --- a/duckpgq/src/duckpgq/operators/physical_path_finding.cpp +++ b/duckpgq/src/duckpgq/operators/physical_path_finding.cpp @@ -15,11 +15,8 @@ namespace duckdb { PhysicalPathFinding::PhysicalPathFinding(LogicalExtensionOperator &op, unique_ptr left, - unique_ptr right, - vector cond, - JoinType join_type, - idx_t estimated_cardinality) - : PhysicalComparisonJoin(op, TYPE, std::move(cond), join_type, estimated_cardinality) { + unique_ptr right) + : CachingPhysicalOperator(TYPE, op.types, 0) { children.push_back(std::move(left)); children.push_back(std::move(right)); } @@ -29,61 +26,26 @@ PhysicalPathFinding::PhysicalPathFinding(LogicalExtensionOperator &op, //===--------------------------------------------------------------------===// class PathFindingLocalState : public LocalSinkState { public: - // TODO Add something to the local sink state - // using LocalSortedTable = PhysicalRangeJoin::LocalSortedTable; - PathFindingLocalState(ClientContext &context, const PhysicalPathFinding &op, const idx_t child) : context(context), op(op), child(child) {} ClientContext &context; const PhysicalPathFinding &op; const idx_t child; - //! The local sort state - // LocalSortedTable table; }; class PathFindingGlobalState : public GlobalSinkState { -public: - using GlobalSortedTable = PhysicalRangeJoin::GlobalSortedTable; public: PathFindingGlobalState(ClientContext &context, const PhysicalPathFinding &op) { - // tables.resize(2); - // RowLayout lhs_layout; - // lhs_layout.Initialize(op.children[0]->types); - // vector lhs_order; - // lhs_order.emplace_back(op.lhs_orders[0][0].Copy()); - // tables[0] = make_uniq(context, lhs_order, lhs_layout); - // - // RowLayout rhs_layout; - // rhs_layout.Initialize(op.children[1]->types); - // vector rhs_order; - // rhs_order.emplace_back(op.rhs_orders[0][0].Copy()); - // tables[1] = make_uniq(context, rhs_order, rhs_layout); - } + } PathFindingGlobalState(PathFindingGlobalState &prev) - : GlobalSinkState( - prev) { //, tables(std::move(prev.tables)), child(prev.child + 1) { - } + : GlobalSinkState(prev) {} - void Sink(DataChunk &input, PathFindingLocalState &lstate) { - // auto &table = *tables[child]; - // auto &global_sort_state = table.global_sort_state; - // auto &local_sort_state = lstate.table.local_sort_state; - // - // // Sink the data into the local sort state - // lstate.table.Sink(input, global_sort_state); - // - // // When sorting data reaches a certain size, we sort it - // if (local_sort_state.SizeInBytes() >= table.memory_per_thread) { - // local_sort_state.Sort(global_sort_state, true); - // } - } + void Sink(DataChunk &input, PathFindingLocalState &lstate) {} - vector> tables; - // size_t child; }; unique_ptr From 38c3475973bbf8a9ddab729c737655f4b6d881fb Mon Sep 17 00:00:00 2001 From: dtenwolde Date: Thu, 15 Feb 2024 14:17:24 +0100 Subject: [PATCH 013/249] Remove unused include --- .../operators/physical_path_finding.cpp | 28 ++----------------- 1 file changed, 2 insertions(+), 26 deletions(-) diff --git a/duckpgq/src/duckpgq/operators/physical_path_finding.cpp b/duckpgq/src/duckpgq/operators/physical_path_finding.cpp index 9509aa2c..0d119034 100644 --- a/duckpgq/src/duckpgq/operators/physical_path_finding.cpp +++ b/duckpgq/src/duckpgq/operators/physical_path_finding.cpp @@ -6,9 +6,6 @@ #include "duckdb/parallel/event.hpp" #include "duckdb/parallel/meta_pipeline.hpp" #include "duckdb/parallel/thread_context.hpp" - -#include -#include #include namespace duckdb { @@ -78,13 +75,11 @@ SinkResultType PhysicalPathFinding::Sink(ExecutionContext &context, SinkCombineResultType PhysicalPathFinding::Combine(ExecutionContext &context, OperatorSinkCombineInput &input) const { - // auto &gstate = input.global_state.Cast(); - // auto &lstate = input.local_state.Cast(); + auto &gstate = input.global_state.Cast(); + auto &lstate = input.local_state.Cast(); // gstate.tables[gstate.child]->Combine(lstate.table); auto &client_profiler = QueryProfiler::Get(context.client); - // context.thread.profiler.Flush(*this, lstate.table.executor, gstate.child ? - // "rhs_executor" : "lhs_executor", 1); client_profiler.Flush(context.thread.profiler); return SinkCombineResultType::FINISHED; @@ -98,25 +93,6 @@ PhysicalPathFinding::Finalize(Pipeline &pipeline, Event &event, ClientContext &context, OperatorSinkFinalizeInput &input) const { auto &gstate = input.global_state.Cast(); - // auto &table = *gstate.tables[gstate.child]; - // auto &global_sort_state = table.global_sort_state; - - // if ((gstate.child == 1 && IsRightOuterJoin(join_type)) || (gstate.child == - // 0 && IsLeftOuterJoin(join_type))) { - // // for FULL/LEFT/RIGHT OUTER JOIN, initialize found_match to false for - // every tuple table.IntializeMatches(); - // } - // if (gstate.child == 1 && global_sort_state.sorted_blocks.empty() && - // EmptyResultIfRHSIsEmpty()) { - // // Empty input! - // return SinkFinalizeType::NO_OUTPUT_POSSIBLE; - // } - - // Sort the current input child - // table.Finalize(pipeline, event); - - // Move to the next input child - // ++gstate.child; return SinkFinalizeType::READY; } From f98cf9b37f586346453733d66d1c410403708dff Mon Sep 17 00:00:00 2001 From: dtenwolde Date: Thu, 15 Feb 2024 14:51:00 +0100 Subject: [PATCH 014/249] Create between expression with tasks and csr --- duckdb-pgq | 2 +- .../functions/tablefunctions/match.hpp | 2 +- .../functions/tablefunctions/match.cpp | 115 ++++++++++-------- 3 files changed, 68 insertions(+), 51 deletions(-) diff --git a/duckdb-pgq b/duckdb-pgq index f85aa91d..2943c520 160000 --- a/duckdb-pgq +++ b/duckdb-pgq @@ -1 +1 @@ -Subproject commit f85aa91d405c3d6e0eca1bb09238824af663c7af +Subproject commit 2943c520aba15ae0167690bc8ddeabc7eb499cae diff --git a/duckpgq/include/duckpgq/functions/tablefunctions/match.hpp b/duckpgq/include/duckpgq/functions/tablefunctions/match.hpp index 4144d22a..aa2cd542 100644 --- a/duckpgq/include/duckpgq/functions/tablefunctions/match.hpp +++ b/duckpgq/include/duckpgq/functions/tablefunctions/match.hpp @@ -58,7 +58,7 @@ struct PGQMatchFunction : public TableFunction { static unique_ptr CreateCountCTESubquery(); - static unique_ptr + static unique_ptr CreateCSRCTE(const shared_ptr &edge_table, const string &edge_binding, const string &prev_binding, const string &next_binding); diff --git a/duckpgq/src/duckpgq/functions/tablefunctions/match.cpp b/duckpgq/src/duckpgq/functions/tablefunctions/match.cpp index 74059c53..35a48546 100644 --- a/duckpgq/src/duckpgq/functions/tablefunctions/match.cpp +++ b/duckpgq/src/duckpgq/functions/tablefunctions/match.cpp @@ -237,7 +237,7 @@ unique_ptr PGQMatchFunction::CreateCountCTESubquery() { return temp_cte_select_subquery; } -unique_ptr +unique_ptr PGQMatchFunction::CreateCSRCTE(const shared_ptr &edge_table, const string &prev_binding, const string &edge_binding, @@ -355,9 +355,9 @@ PGQMatchFunction::CreateCSRCTE(const shared_ptr &edge_table, auto outer_select_statement = make_uniq(); outer_select_statement->node = std::move(outer_select_node); - auto info = make_uniq(); - info->query = std::move(outer_select_statement); - return info; + auto csr_subquery = make_uniq(); + csr_subquery->subquery = std::move(outer_select_statement); + return csr_subquery; } void PGQMatchFunction::EdgeTypeAny( @@ -635,23 +635,40 @@ void PGQMatchFunction::AddPathFinding( const string &prev_binding, const string &edge_binding, const string &next_binding, const shared_ptr &edge_table, const SubPath *subpath) { + /* TODO Idea is to create a path-finding operator wih two sinks. This acts similar + to the IEJoin. We insert that in this function, instead of the iterativelength() UDF. + For this binding phase, we generate a logical query plan, so there cannot be a physical + path-finding operator inserted quite yet. We need to create the two sinks here. + One side being the src,dst pairs (tasks) and the other side being the CSR. Importantly + Without the CREATE_CSR_EDGE() UDF because that will be done in one of the sinks + of the new operator. + */ + + auto csr = CreateCSRCTE(edge_table, prev_binding, edge_binding, next_binding); + auto src_tasks = make_uniq("rowid", prev_binding); + auto dst_tasks = make_uniq("rowid", next_binding); + auto between_expression = make_uniq( + std::move(src_tasks), std::move(csr), std::move(dst_tasks)); + + conditions.push_back(std::move(between_expression)); + //! START //! FROM (SELECT count(cte1.temp) * 0 as temp from cte1) __x - select_node->cte_map.map["cte1"] = - CreateCSRCTE(edge_table, prev_binding, edge_binding, next_binding); - - auto temp_cte_select_subquery = CreateCountCTESubquery(); - - if (from_clause) { - // create a cross join since there is already something in the - // from clause - auto from_join = make_uniq(JoinRefType::CROSS); - from_join->left = std::move(from_clause); - from_join->right = std::move(temp_cte_select_subquery); - from_clause = std::move(from_join); - } else { - from_clause = std::move(temp_cte_select_subquery); - } + // select_node->cte_map.map["cte1"] = + // CreateCSRCTE(edge_table, prev_binding, edge_binding, next_binding); + + // auto temp_cte_select_subquery = CreateCountCTESubquery(); + + // if (from_clause) { + // // create a cross join since there is already something in the + // // from clause + // auto from_join = make_uniq(JoinRefType::CROSS); + // from_join->left = std::move(from_clause); + // from_join->right = std::move(temp_cte_select_subquery); + // from_clause = std::move(from_join); + // } else { + // from_clause = std::move(temp_cte_select_subquery); + // } //! END //! FROM (SELECT count(cte1.temp) * 0 as temp from cte1) __x @@ -659,36 +676,36 @@ void PGQMatchFunction::AddPathFinding( //! WHERE __x.temp + iterativelength(, (SELECT count(c.id) //! from dst c, a.rowid, b.rowid) between lower and upper - auto src_row_id = make_uniq("rowid", prev_binding); - auto dst_row_id = make_uniq("rowid", next_binding); - auto csr_id = make_uniq(Value::INTEGER(0)); - - vector> pathfinding_children; - pathfinding_children.push_back(std::move(csr_id)); - pathfinding_children.push_back( - std::move(GetCountTable(edge_table, prev_binding))); - pathfinding_children.push_back(std::move(src_row_id)); - pathfinding_children.push_back(std::move(dst_row_id)); - - auto reachability_function = make_uniq( - "iterativelength", std::move(pathfinding_children)); - - auto cte_col_ref = make_uniq("temp", "__x"); - - vector> addition_children; - addition_children.push_back(std::move(cte_col_ref)); - addition_children.push_back(std::move(reachability_function)); - - auto addition_function = - make_uniq("add", std::move(addition_children)); - auto lower_limit = make_uniq( - Value::INTEGER(static_cast(subpath->lower))); - auto upper_limit = make_uniq( - Value::INTEGER(static_cast(subpath->upper))); - auto between_expression = make_uniq( - std::move(addition_function), std::move(lower_limit), - std::move(upper_limit)); - conditions.push_back(std::move(between_expression)); + // auto src_row_id = make_uniq("rowid", prev_binding); + // auto dst_row_id = make_uniq("rowid", next_binding); + // auto csr_id = make_uniq(Value::INTEGER(0)); + // + // vector> pathfinding_children; + // pathfinding_children.push_back(std::move(csr_id)); + // pathfinding_children.push_back( + // std::move(GetCountTable(edge_table, prev_binding))); + // pathfinding_children.push_back(std::move(src_row_id)); + // pathfinding_children.push_back(std::move(dst_row_id)); + + // auto reachability_function = make_uniq( + // "iterativelength", std::move(pathfinding_children)); + + // auto cte_col_ref = make_uniq("temp", "__x"); + + // vector> addition_children; + // addition_children.push_back(std::move(cte_col_ref)); + // addition_children.push_back(std::move(reachability_function)); + + // auto addition_function = + // make_uniq("add", std::move(addition_children)); + // auto lower_limit = make_uniq( + // Value::INTEGER(static_cast(subpath->lower))); + // auto upper_limit = make_uniq( + // Value::INTEGER(static_cast(subpath->upper))); + // auto between_expression = make_uniq( + // std::move(addition_function), std::move(lower_limit), + // std::move(upper_limit)); + // conditions.push_back(std::move(between_expression)); //! END //! WHERE __x.temp + iterativelength(, (SELECT count(s.id) From e7060a0ecc5d2d20156bd16ea7f4ccfba9535c04 Mon Sep 17 00:00:00 2001 From: dtenwolde Date: Thu, 15 Feb 2024 15:00:55 +0100 Subject: [PATCH 015/249] Changed SubqueryRef to SubqueryExpression --- duckpgq/include/duckpgq/functions/tablefunctions/match.hpp | 2 +- duckpgq/src/duckpgq/functions/tablefunctions/match.cpp | 4 ++-- 2 files changed, 3 insertions(+), 3 deletions(-) diff --git a/duckpgq/include/duckpgq/functions/tablefunctions/match.hpp b/duckpgq/include/duckpgq/functions/tablefunctions/match.hpp index aa2cd542..fcc85920 100644 --- a/duckpgq/include/duckpgq/functions/tablefunctions/match.hpp +++ b/duckpgq/include/duckpgq/functions/tablefunctions/match.hpp @@ -58,7 +58,7 @@ struct PGQMatchFunction : public TableFunction { static unique_ptr CreateCountCTESubquery(); - static unique_ptr + static unique_ptr CreateCSRCTE(const shared_ptr &edge_table, const string &edge_binding, const string &prev_binding, const string &next_binding); diff --git a/duckpgq/src/duckpgq/functions/tablefunctions/match.cpp b/duckpgq/src/duckpgq/functions/tablefunctions/match.cpp index 35a48546..ca6ca6f7 100644 --- a/duckpgq/src/duckpgq/functions/tablefunctions/match.cpp +++ b/duckpgq/src/duckpgq/functions/tablefunctions/match.cpp @@ -237,7 +237,7 @@ unique_ptr PGQMatchFunction::CreateCountCTESubquery() { return temp_cte_select_subquery; } -unique_ptr +unique_ptr PGQMatchFunction::CreateCSRCTE(const shared_ptr &edge_table, const string &prev_binding, const string &edge_binding, @@ -355,7 +355,7 @@ PGQMatchFunction::CreateCSRCTE(const shared_ptr &edge_table, auto outer_select_statement = make_uniq(); outer_select_statement->node = std::move(outer_select_node); - auto csr_subquery = make_uniq(); + auto csr_subquery = make_uniq(); csr_subquery->subquery = std::move(outer_select_statement); return csr_subquery; } From 239cefe2c645d474516b63e9530d62ae502c5983 Mon Sep 17 00:00:00 2001 From: dtenwolde Date: Mon, 26 Feb 2024 12:14:32 +0100 Subject: [PATCH 016/249] Adding test --- .../operators/physical_path_finding.cpp | 2 +- .../path-finding/parallel_path_finding.test | 102 ++++++++++++++++++ 2 files changed, 103 insertions(+), 1 deletion(-) create mode 100644 test/sql/path-finding/parallel_path_finding.test diff --git a/duckpgq/src/duckpgq/operators/physical_path_finding.cpp b/duckpgq/src/duckpgq/operators/physical_path_finding.cpp index 0d119034..79878ac5 100644 --- a/duckpgq/src/duckpgq/operators/physical_path_finding.cpp +++ b/duckpgq/src/duckpgq/operators/physical_path_finding.cpp @@ -113,7 +113,7 @@ OperatorResultType PhysicalPathFinding::ExecuteInternal( // using SortedTable = PhysicalRangeJoin::GlobalSortedTable; // // static idx_t AppendKey(SortedTable &table, ExpressionExecutor -// &executor, SortedTable &marked, int64_t increment, int64_t base, const idx_t +// &executor, SortedTable &marked, int64_t increment, int64_t base, const idx_t // block_idx); // // static void Sort(SortedTable &table) { diff --git a/test/sql/path-finding/parallel_path_finding.test b/test/sql/path-finding/parallel_path_finding.test new file mode 100644 index 00000000..aee3c9a5 --- /dev/null +++ b/test/sql/path-finding/parallel_path_finding.test @@ -0,0 +1,102 @@ +# name: test/sql/join/pathfinding/test_path_finding.test +# description: Prototype the path finding operator +# group: [path-finding] + +statement ok +PRAGMA enable_verification + +require duckpgq + +statement ok +CREATE TABLE pairs(src INT, dst INT); + +statement ok +INSERT INTO pairs(src, dst) VALUES (1, 2), (2, 3), (3,1), (3, 2), (6, 1), (4,2); + +statement ok +create table student(id INT); + +statement ok +INSERT INTO student(id) VALUES (1), (2), (3); + +statement ok +create table knows(src INT, dst INT); + +statement ok +INSERT INTO knows(src, dst) VALUES (1,3), (1,1), (2,1), (2,3), (3,1), (NULL, NULL), (NULL, NULL) + +statement ok +create table foo(i bigint); + +#query II +#SELECT * +#FROM knows AS a, pairs AS b +#WHERE (cos(a.src) BETWEEN cos(b.dst) AND cos(b.src)); +#---- + +statement ok +-CREATE PROPERTY GRAPH pg +VERTEX TABLES ( +foo +) +EDGE TABLES ( +knows SOURCE KEY (src) REFERENCES foo (i) + DESTINATION KEY (dst) references foo (i) +) + +statement ok +-FROM GRAPH_TABLE(pg + MATCH (f:foo)-[k:knows]->{0,3}(f2:foo) + COLUMNS (*) + ); + +query II +SELECT * +FROM pairs AS p +WHERE p.src BETWEEN (select csr_id from (SELECT + 0 as csr_id, + (SELECT count(a.id) FROM Student a), + CAST ( + (SELECT sum(CREATE_CSR_VERTEX(0, + (SELECT count(a.id) FROM Student a), + sub.dense_id, + sub.cnt) + ) + FROM ( + SELECT a.rowid as dense_id, count(k.src) as cnt + FROM Student a + LEFT JOIN Knows k ON k.src = a.id + GROUP BY a.rowid) sub + ) AS BIGINT), + a.rowid, + c.rowid, + k.rowid FROM Knows k + JOIN student a on a.id = k.src + JOIN student c on c.id = k.dst)) AND p.dst; +---- + +# LHS of IEJOIN + + + + +# RHS of IEJoin +# 0, + # (SELECT count(a.id) FROM Student a), + # CAST ( + # (SELECT sum(CREATE_CSR_VERTEX( + # 0, + # (SELECT count(a.id) FROM Student a), + # sub.dense_id, + # sub.cnt) + # ) + # FROM ( + # SELECT a.rowid as dense_id, count(k.src) as cnt + # FROM Student a + # LEFT JOIN Know k ON k.src = a.id + # GROUP BY a.rowid) sub + # ) + # AS BIGINT), + # a.rowid, + # c.rowid, + # k.rowid \ No newline at end of file From 00e9873423db46c18ab5524c4100728f76529fe7 Mon Sep 17 00:00:00 2001 From: dtenwolde Date: Mon, 26 Feb 2024 12:17:17 +0100 Subject: [PATCH 017/249] Fix --- duckdb-pgq | 2 +- duckpgq/src/duckpgq/operators/physical_path_finding.cpp | 4 ++-- 2 files changed, 3 insertions(+), 3 deletions(-) diff --git a/duckdb-pgq b/duckdb-pgq index 70c6cac3..f0a71608 160000 --- a/duckdb-pgq +++ b/duckdb-pgq @@ -1 +1 @@ -Subproject commit 70c6cac303ec7f553a17e85d9ed8687f3d6c656e +Subproject commit f0a71608770498324167efa41cc738de11f8aa4e diff --git a/duckpgq/src/duckpgq/operators/physical_path_finding.cpp b/duckpgq/src/duckpgq/operators/physical_path_finding.cpp index 79878ac5..04525da0 100644 --- a/duckpgq/src/duckpgq/operators/physical_path_finding.cpp +++ b/duckpgq/src/duckpgq/operators/physical_path_finding.cpp @@ -979,8 +979,8 @@ void PhysicalPathFinding::BuildPipelines(Pipeline ¤t, children[0]->BuildPipelines(*lhs_pipeline, child_meta_pipeline); // Build out RHS - auto rhs_pipeline = child_meta_pipeline.CreatePipeline(); - children[1]->BuildPipelines(*rhs_pipeline, child_meta_pipeline); + auto &rhs_pipeline = child_meta_pipeline.CreatePipeline(); + children[1]->BuildPipelines(rhs_pipeline, child_meta_pipeline); // Despite having the same sink, RHS and everything created after it need // their own (same) PipelineFinishEvent From 9fcd2d401de906726c113f5dd865a7aeef0d86f3 Mon Sep 17 00:00:00 2001 From: dtenwolde Date: Mon, 26 Feb 2024 12:19:27 +0100 Subject: [PATCH 018/249] Make subquery type any --- duckpgq/src/duckpgq/functions/tablefunctions/match.cpp | 1 + 1 file changed, 1 insertion(+) diff --git a/duckpgq/src/duckpgq/functions/tablefunctions/match.cpp b/duckpgq/src/duckpgq/functions/tablefunctions/match.cpp index ca6ca6f7..a139d167 100644 --- a/duckpgq/src/duckpgq/functions/tablefunctions/match.cpp +++ b/duckpgq/src/duckpgq/functions/tablefunctions/match.cpp @@ -357,6 +357,7 @@ PGQMatchFunction::CreateCSRCTE(const shared_ptr &edge_table, outer_select_statement->node = std::move(outer_select_node); auto csr_subquery = make_uniq(); csr_subquery->subquery = std::move(outer_select_statement); + csr_subquery->subquery_type = SubqueryType::ANY; return csr_subquery; } From bde90976a57e08746be2804c7c1170c3c7fa8d1c Mon Sep 17 00:00:00 2001 From: dtenwolde Date: Mon, 26 Feb 2024 12:26:41 +0100 Subject: [PATCH 019/249] Add compare equal --- duckpgq/src/duckpgq/functions/tablefunctions/match.cpp | 1 + 1 file changed, 1 insertion(+) diff --git a/duckpgq/src/duckpgq/functions/tablefunctions/match.cpp b/duckpgq/src/duckpgq/functions/tablefunctions/match.cpp index a139d167..a1bf0860 100644 --- a/duckpgq/src/duckpgq/functions/tablefunctions/match.cpp +++ b/duckpgq/src/duckpgq/functions/tablefunctions/match.cpp @@ -358,6 +358,7 @@ PGQMatchFunction::CreateCSRCTE(const shared_ptr &edge_table, auto csr_subquery = make_uniq(); csr_subquery->subquery = std::move(outer_select_statement); csr_subquery->subquery_type = SubqueryType::ANY; + csr_subquery->comparison_type = ExpressionType::COMPARE_EQUAL; return csr_subquery; } From 26cadf12c8a3211f0e188af43e98cd1d61c931ad Mon Sep 17 00:00:00 2001 From: dtenwolde Date: Mon, 26 Feb 2024 12:29:37 +0100 Subject: [PATCH 020/249] Changing any type to scalar --- duckpgq/src/duckpgq/functions/tablefunctions/match.cpp | 3 +-- 1 file changed, 1 insertion(+), 2 deletions(-) diff --git a/duckpgq/src/duckpgq/functions/tablefunctions/match.cpp b/duckpgq/src/duckpgq/functions/tablefunctions/match.cpp index a1bf0860..8b868c5e 100644 --- a/duckpgq/src/duckpgq/functions/tablefunctions/match.cpp +++ b/duckpgq/src/duckpgq/functions/tablefunctions/match.cpp @@ -357,8 +357,7 @@ PGQMatchFunction::CreateCSRCTE(const shared_ptr &edge_table, outer_select_statement->node = std::move(outer_select_node); auto csr_subquery = make_uniq(); csr_subquery->subquery = std::move(outer_select_statement); - csr_subquery->subquery_type = SubqueryType::ANY; - csr_subquery->comparison_type = ExpressionType::COMPARE_EQUAL; + csr_subquery->subquery_type = SubqueryType::SCALAR; return csr_subquery; } From e4df3006d0b39dab2e8971b3b0be760eec0a6932 Mon Sep 17 00:00:00 2001 From: dtenwolde Date: Mon, 26 Feb 2024 14:21:41 +0100 Subject: [PATCH 021/249] Add optimizerextension --- duckpgq/include/duckpgq_extension.hpp | 12 ++++++++++++ duckpgq/src/duckpgq_extension.cpp | 1 + 2 files changed, 13 insertions(+) diff --git a/duckpgq/include/duckpgq_extension.hpp b/duckpgq/include/duckpgq_extension.hpp index 5bbe091d..15672918 100644 --- a/duckpgq/include/duckpgq_extension.hpp +++ b/duckpgq/include/duckpgq_extension.hpp @@ -17,6 +17,18 @@ class DuckpgqExtension : public Extension { std::string Name() override; }; +class DuckpgqOptimizerExtension : public OptimizerExtension { +public: + DuckpgqOptimizerExtension() { + optimize_function = DuckpgqOptimizeFunction; + } + + static void DuckpgqOptimizeFunction(ClientContext &context, OptimizerExtensionInfo *info, + duckdb::unique_ptr &plan) { + + } +}; + struct DuckPGQParserExtensionInfo : public ParserExtensionInfo { public: DuckPGQParserExtensionInfo() : ParserExtensionInfo(){}; diff --git a/duckpgq/src/duckpgq_extension.cpp b/duckpgq/src/duckpgq_extension.cpp index 50c1b57c..c80cb24c 100644 --- a/duckpgq/src/duckpgq_extension.cpp +++ b/duckpgq/src/duckpgq_extension.cpp @@ -9,6 +9,7 @@ #include "duckpgq/duckpgq_functions.hpp" #include "duckdb/parser/parser_extension.hpp" +#include "duckdb/optimizer/optimizer_extension.hpp" #include "duckdb/parser/parser.hpp" #include "duckdb/parser/tableref/table_function_ref.hpp" From ab12a42ad37f13786e700936388913beff53a407 Mon Sep 17 00:00:00 2001 From: dtenwolde Date: Mon, 26 Feb 2024 14:51:42 +0100 Subject: [PATCH 022/249] Add function to find between expression --- duckpgq/include/duckpgq_extension.hpp | 22 ++++++++++++++++++++++ duckpgq/src/duckpgq_extension.cpp | 1 + 2 files changed, 23 insertions(+) diff --git a/duckpgq/include/duckpgq_extension.hpp b/duckpgq/include/duckpgq_extension.hpp index 15672918..fa34015f 100644 --- a/duckpgq/include/duckpgq_extension.hpp +++ b/duckpgq/include/duckpgq_extension.hpp @@ -8,6 +8,7 @@ #include "duckdb/parser/simplified_token.hpp" #include "duckdb/common/compressed_sparse_row.hpp" #include "duckdb/parser/parsed_data/create_property_graph_info.hpp" +#include "duckdb/planner/operator/logical_comparison_join.hpp" namespace duckdb { @@ -23,8 +24,29 @@ class DuckpgqOptimizerExtension : public OptimizerExtension { optimize_function = DuckpgqOptimizeFunction; } + static bool HasBetweenExpression(LogicalOperator &op) { + if (op.type == LogicalOperatorType::LOGICAL_COMPARISON_JOIN) { + auto &get = op.Cast(); + for (auto &expression : get.expressions) { + if (expression->expression_class == ExpressionClass::BOUND_BETWEEN) { + return true; + } + } + } + for (auto &child : op.children) { + if (HasBetweenExpression(*child)) { + return true; + } + } + return false; + } + static void DuckpgqOptimizeFunction(ClientContext &context, OptimizerExtensionInfo *info, duckdb::unique_ptr &plan) { + if (!HasBetweenExpression(*plan)) { + return; + } + std::cout << "Between expression found"; } }; diff --git a/duckpgq/src/duckpgq_extension.cpp b/duckpgq/src/duckpgq_extension.cpp index c80cb24c..3b5544fd 100644 --- a/duckpgq/src/duckpgq_extension.cpp +++ b/duckpgq/src/duckpgq_extension.cpp @@ -44,6 +44,7 @@ static void LoadInternal(DatabaseInstance &instance) { DuckPGQParserExtension pgq_parser; config.parser_extensions.push_back(pgq_parser); config.operator_extensions.push_back(make_uniq()); + config.optimizer_extensions.push_back(DuckpgqOptimizerExtension()); Connection con(instance); con.BeginTransaction(); From d94ab617fe65a99e50d56d605add41e722c3096f Mon Sep 17 00:00:00 2001 From: dtenwolde Date: Mon, 26 Feb 2024 16:57:24 +0100 Subject: [PATCH 023/249] Changing test slightly --- duckpgq/include/duckpgq_extension.hpp | 13 +++++--- .../path-finding/parallel_path_finding.test | 33 +++++++------------ 2 files changed, 19 insertions(+), 27 deletions(-) diff --git a/duckpgq/include/duckpgq_extension.hpp b/duckpgq/include/duckpgq_extension.hpp index fa34015f..e3cf829a 100644 --- a/duckpgq/include/duckpgq_extension.hpp +++ b/duckpgq/include/duckpgq_extension.hpp @@ -1,14 +1,15 @@ #pragma once -#include "duckpgq/common.hpp" -#include "duckdb/parser/sql_statement.hpp" +#include "duckdb/common/compressed_sparse_row.hpp" +#include "duckdb/parser/column_list.hpp" +#include "duckdb/parser/parsed_data/create_property_graph_info.hpp" #include "duckdb/parser/parsed_expression.hpp" #include "duckdb/parser/query_node.hpp" -#include "duckdb/parser/column_list.hpp" #include "duckdb/parser/simplified_token.hpp" -#include "duckdb/common/compressed_sparse_row.hpp" -#include "duckdb/parser/parsed_data/create_property_graph_info.hpp" +#include "duckdb/parser/sql_statement.hpp" #include "duckdb/planner/operator/logical_comparison_join.hpp" +#include "duckpgq/common.hpp" +#include "duckpgq/operators/path_finding_operator.hpp" namespace duckdb { @@ -48,6 +49,8 @@ class DuckpgqOptimizerExtension : public OptimizerExtension { } std::cout << "Between expression found"; + plan = make_uniq(plan->children); + } }; diff --git a/test/sql/path-finding/parallel_path_finding.test b/test/sql/path-finding/parallel_path_finding.test index aee3c9a5..aeff644e 100644 --- a/test/sql/path-finding/parallel_path_finding.test +++ b/test/sql/path-finding/parallel_path_finding.test @@ -8,25 +8,13 @@ PRAGMA enable_verification require duckpgq statement ok -CREATE TABLE pairs(src INT, dst INT); +CREATE TABLE pairs(src INT, dst INT); INSERT INTO pairs(src, dst) VALUES (1, 2), (2, 3), (3,1), (3, 2), (6, 1), (4,2); statement ok -INSERT INTO pairs(src, dst) VALUES (1, 2), (2, 3), (3,1), (3, 2), (6, 1), (4,2); +create table student(id INT); INSERT INTO student(id) VALUES (1), (2), (3); statement ok -create table student(id INT); - -statement ok -INSERT INTO student(id) VALUES (1), (2), (3); - -statement ok -create table knows(src INT, dst INT); - -statement ok -INSERT INTO knows(src, dst) VALUES (1,3), (1,1), (2,1), (2,3), (3,1), (NULL, NULL), (NULL, NULL) - -statement ok -create table foo(i bigint); +create table knows(src INT, dst INT); INSERT INTO knows(src, dst) VALUES (1,3), (1,1), (2,1), (2,3), (3,1), (NULL, NULL), (NULL, NULL); #query II #SELECT * @@ -37,18 +25,19 @@ create table foo(i bigint); statement ok -CREATE PROPERTY GRAPH pg VERTEX TABLES ( -foo +student ) EDGE TABLES ( -knows SOURCE KEY (src) REFERENCES foo (i) - DESTINATION KEY (dst) references foo (i) -) +knows SOURCE KEY (src) REFERENCES student (id) + DESTINATION KEY (dst) references student (id) +); -statement ok --FROM GRAPH_TABLE(pg - MATCH (f:foo)-[k:knows]->{0,3}(f2:foo) +query II +-EXPLAIN FROM GRAPH_TABLE(pg + MATCH (f:student)-[k:knows]->{0,3}(f2:student) COLUMNS (*) ); +---- query II SELECT * From a46ff6e2eccc9a5e816703d12be41975951c213b Mon Sep 17 00:00:00 2001 From: dtenwolde Date: Mon, 26 Feb 2024 16:59:46 +0100 Subject: [PATCH 024/249] Create plan --- .../duckpgq/operators/path_finding_operator.cpp | 15 +++++++-------- 1 file changed, 7 insertions(+), 8 deletions(-) diff --git a/duckpgq/src/duckpgq/operators/path_finding_operator.cpp b/duckpgq/src/duckpgq/operators/path_finding_operator.cpp index 8e81217c..18874234 100644 --- a/duckpgq/src/duckpgq/operators/path_finding_operator.cpp +++ b/duckpgq/src/duckpgq/operators/path_finding_operator.cpp @@ -7,13 +7,12 @@ namespace duckdb { unique_ptr PathFindingOperator::CreatePlan(ClientContext &, duckdb::PhysicalPlanGenerator &generator) { - // D_ASSERT(children.size() == 2); - // auto left = generator.CreatePlan(std::move(children[0])); - // auto right = generator.CreatePlan(std::move(children[1])); - // - // auto result = - // duckdb::make_uniq_base(*this); - // result->children.emplace_back(std::move(result)); - // return result; + D_ASSERT(children.size() == 2); + auto left = generator.CreatePlan(std::move(children[0])); + auto right = generator.CreatePlan(std::move(children[1])); + return make_uniq(*this, std::move(left), std::move(right)); +} +vector PathFindingOperator::GetColumnBindings() { + return LogicalOperator::GetColumnBindings(); } } // namespace duckdb From 9f68f3d1961a4c31dc4da73c040e766c5130baed Mon Sep 17 00:00:00 2001 From: dtenwolde Date: Tue, 27 Feb 2024 10:25:22 +0100 Subject: [PATCH 025/249] Make naming consistent --- ...finding_operator.hpp => logical_path_finding_operator.hpp} | 4 ++-- ...al_path_finding.hpp => physical_path_finding_operator.hpp} | 0 2 files changed, 2 insertions(+), 2 deletions(-) rename duckpgq/include/duckpgq/operators/{path_finding_operator.hpp => logical_path_finding_operator.hpp} (79%) rename duckpgq/include/duckpgq/operators/{physical_path_finding.hpp => physical_path_finding_operator.hpp} (100%) diff --git a/duckpgq/include/duckpgq/operators/path_finding_operator.hpp b/duckpgq/include/duckpgq/operators/logical_path_finding_operator.hpp similarity index 79% rename from duckpgq/include/duckpgq/operators/path_finding_operator.hpp rename to duckpgq/include/duckpgq/operators/logical_path_finding_operator.hpp index d7152026..4172e4d5 100644 --- a/duckpgq/include/duckpgq/operators/path_finding_operator.hpp +++ b/duckpgq/include/duckpgq/operators/logical_path_finding_operator.hpp @@ -3,9 +3,9 @@ namespace duckdb { - class PathFindingOperator : public LogicalExtensionOperator { + class LogicalPathFindingOperator : public LogicalExtensionOperator { public: - explicit PathFindingOperator(unique_ptr plan) { + explicit LogicalPathFindingOperator(unique_ptr plan) { children.emplace_back(std::move(plan)); } diff --git a/duckpgq/include/duckpgq/operators/physical_path_finding.hpp b/duckpgq/include/duckpgq/operators/physical_path_finding_operator.hpp similarity index 100% rename from duckpgq/include/duckpgq/operators/physical_path_finding.hpp rename to duckpgq/include/duckpgq/operators/physical_path_finding_operator.hpp From e2349ee43ea5846843d64bea650987ceaf5ccf68 Mon Sep 17 00:00:00 2001 From: dtenwolde Date: Tue, 27 Feb 2024 10:25:38 +0100 Subject: [PATCH 026/249] Add import --- duckpgq/src/duckpgq_extension.cpp | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/duckpgq/src/duckpgq_extension.cpp b/duckpgq/src/duckpgq_extension.cpp index 3b5544fd..97ef5879 100644 --- a/duckpgq/src/duckpgq_extension.cpp +++ b/duckpgq/src/duckpgq_extension.cpp @@ -2,8 +2,8 @@ #include "duckpgq_extension.hpp" -#include #include +#include #include "duckdb/function/scalar_function.hpp" #include "duckpgq/duckpgq_functions.hpp" From 3856f221bb88acfdfe9fde650ad79dec5ea86917 Mon Sep 17 00:00:00 2001 From: dtenwolde Date: Tue, 27 Feb 2024 10:26:38 +0100 Subject: [PATCH 027/249] Rename import --- duckpgq/include/duckpgq_extension.hpp | 9 ++++----- duckpgq/src/duckpgq/operators/physical_path_finding.cpp | 2 +- 2 files changed, 5 insertions(+), 6 deletions(-) diff --git a/duckpgq/include/duckpgq_extension.hpp b/duckpgq/include/duckpgq_extension.hpp index e3cf829a..dd3619b8 100644 --- a/duckpgq/include/duckpgq_extension.hpp +++ b/duckpgq/include/duckpgq_extension.hpp @@ -9,7 +9,7 @@ #include "duckdb/parser/sql_statement.hpp" #include "duckdb/planner/operator/logical_comparison_join.hpp" #include "duckpgq/common.hpp" -#include "duckpgq/operators/path_finding_operator.hpp" +#include "duckpgq/operators/logical_path_finding_operator.hpp" namespace duckdb { @@ -28,8 +28,8 @@ class DuckpgqOptimizerExtension : public OptimizerExtension { static bool HasBetweenExpression(LogicalOperator &op) { if (op.type == LogicalOperatorType::LOGICAL_COMPARISON_JOIN) { auto &get = op.Cast(); - for (auto &expression : get.expressions) { - if (expression->expression_class == ExpressionClass::BOUND_BETWEEN) { + for (auto &condition : get.conditions) { + if (condition.comparison == ExpressionType::COMPARE_GREATERTHANOREQUALTO) { return true; } } @@ -48,8 +48,7 @@ class DuckpgqOptimizerExtension : public OptimizerExtension { return; } std::cout << "Between expression found"; - - plan = make_uniq(plan->children); + plan = make_uniq(plan->children); } }; diff --git a/duckpgq/src/duckpgq/operators/physical_path_finding.cpp b/duckpgq/src/duckpgq/operators/physical_path_finding.cpp index 04525da0..5bdf3b2e 100644 --- a/duckpgq/src/duckpgq/operators/physical_path_finding.cpp +++ b/duckpgq/src/duckpgq/operators/physical_path_finding.cpp @@ -1,4 +1,4 @@ -#include "duckpgq/operators/physical_path_finding.hpp" +#include "duckpgq/operators/physical_path_finding_operator.hpp" #include "duckdb/common/sort/sort.hpp" #include "duckdb/execution/physical_operator.hpp" From ff2790914a73520429cbd30ce1abf51981b08707 Mon Sep 17 00:00:00 2001 From: dtenwolde Date: Tue, 27 Feb 2024 10:26:44 +0100 Subject: [PATCH 028/249] Remove explain --- test/sql/path-finding/parallel_path_finding.test | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/test/sql/path-finding/parallel_path_finding.test b/test/sql/path-finding/parallel_path_finding.test index aeff644e..157c8ea7 100644 --- a/test/sql/path-finding/parallel_path_finding.test +++ b/test/sql/path-finding/parallel_path_finding.test @@ -33,7 +33,7 @@ knows SOURCE KEY (src) REFERENCES student (id) ); query II --EXPLAIN FROM GRAPH_TABLE(pg +-FROM GRAPH_TABLE(pg MATCH (f:student)-[k:knows]->{0,3}(f2:student) COLUMNS (*) ); From ebb651d041b2ede4d95bf867eb10ead55838c5b1 Mon Sep 17 00:00:00 2001 From: dtenwolde Date: Tue, 27 Feb 2024 10:28:09 +0100 Subject: [PATCH 029/249] Make names consistent --- duckpgq/src/duckpgq/operators/CMakeLists.txt | 4 ++-- ...finding_operator.cpp => logical_path_finding_operator.cpp} | 4 ++-- ...al_path_finding.cpp => physical_path_finding_operator.cpp} | 0 3 files changed, 4 insertions(+), 4 deletions(-) rename duckpgq/src/duckpgq/operators/{path_finding_operator.cpp => logical_path_finding_operator.cpp} (82%) rename duckpgq/src/duckpgq/operators/{physical_path_finding.cpp => physical_path_finding_operator.cpp} (100%) diff --git a/duckpgq/src/duckpgq/operators/CMakeLists.txt b/duckpgq/src/duckpgq/operators/CMakeLists.txt index 17f3d436..f9172760 100644 --- a/duckpgq/src/duckpgq/operators/CMakeLists.txt +++ b/duckpgq/src/duckpgq/operators/CMakeLists.txt @@ -1,6 +1,6 @@ set(EXTENSION_SOURCES - ${CMAKE_CURRENT_SOURCE_DIR}/path_finding_operator.cpp - ${CMAKE_CURRENT_SOURCE_DIR}/physical_path_finding.cpp + ${CMAKE_CURRENT_SOURCE_DIR}/logical_path_finding_operator.cpp + ${CMAKE_CURRENT_SOURCE_DIR}/physical_path_finding_operator.cpp ${EXTENSION_SOURCES} PARENT_SCOPE diff --git a/duckpgq/src/duckpgq/operators/path_finding_operator.cpp b/duckpgq/src/duckpgq/operators/logical_path_finding_operator.cpp similarity index 82% rename from duckpgq/src/duckpgq/operators/path_finding_operator.cpp rename to duckpgq/src/duckpgq/operators/logical_path_finding_operator.cpp index 18874234..d717269a 100644 --- a/duckpgq/src/duckpgq/operators/path_finding_operator.cpp +++ b/duckpgq/src/duckpgq/operators/logical_path_finding_operator.cpp @@ -1,6 +1,6 @@ -#include "duckpgq/operators/path_finding_operator.hpp" -#include +#include "duckpgq/operators/logical_path_finding_operator.hpp" +#include #include namespace duckdb { diff --git a/duckpgq/src/duckpgq/operators/physical_path_finding.cpp b/duckpgq/src/duckpgq/operators/physical_path_finding_operator.cpp similarity index 100% rename from duckpgq/src/duckpgq/operators/physical_path_finding.cpp rename to duckpgq/src/duckpgq/operators/physical_path_finding_operator.cpp From 6d7382c2ef6abaa19787c78ac3aacd3530cbe6f9 Mon Sep 17 00:00:00 2001 From: dtenwolde Date: Tue, 27 Feb 2024 10:28:31 +0100 Subject: [PATCH 030/249] Rename --- .../src/duckpgq/operators/logical_path_finding_operator.cpp | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/duckpgq/src/duckpgq/operators/logical_path_finding_operator.cpp b/duckpgq/src/duckpgq/operators/logical_path_finding_operator.cpp index d717269a..bda1d5cf 100644 --- a/duckpgq/src/duckpgq/operators/logical_path_finding_operator.cpp +++ b/duckpgq/src/duckpgq/operators/logical_path_finding_operator.cpp @@ -5,14 +5,14 @@ namespace duckdb { unique_ptr -PathFindingOperator::CreatePlan(ClientContext &, +LogicalPathFindingOperator::CreatePlan(ClientContext &, duckdb::PhysicalPlanGenerator &generator) { D_ASSERT(children.size() == 2); auto left = generator.CreatePlan(std::move(children[0])); auto right = generator.CreatePlan(std::move(children[1])); return make_uniq(*this, std::move(left), std::move(right)); } -vector PathFindingOperator::GetColumnBindings() { +vector LogicalPathFindingOperator::GetColumnBindings() { return LogicalOperator::GetColumnBindings(); } } // namespace duckdb From ad0b8fd68853597df23ba2a7b08b2a298fdc4644 Mon Sep 17 00:00:00 2001 From: dtenwolde Date: Tue, 27 Feb 2024 10:36:28 +0100 Subject: [PATCH 031/249] Fix test --- .../path-finding/parallel_path_finding.test | 60 +++++++++---------- 1 file changed, 28 insertions(+), 32 deletions(-) diff --git a/test/sql/path-finding/parallel_path_finding.test b/test/sql/path-finding/parallel_path_finding.test index 157c8ea7..7ce5f2c5 100644 --- a/test/sql/path-finding/parallel_path_finding.test +++ b/test/sql/path-finding/parallel_path_finding.test @@ -2,8 +2,8 @@ # description: Prototype the path finding operator # group: [path-finding] -statement ok -PRAGMA enable_verification +#statement ok +#PRAGMA enable_verification require duckpgq @@ -32,14 +32,13 @@ knows SOURCE KEY (src) REFERENCES student (id) DESTINATION KEY (dst) references student (id) ); -query II --FROM GRAPH_TABLE(pg - MATCH (f:student)-[k:knows]->{0,3}(f2:student) - COLUMNS (*) - ); ----- +#statement ok +#-FROM GRAPH_TABLE(pg +# MATCH (f:student)-[k:knows]->{0,3}(f2:student) +# COLUMNS (*) +# ); -query II +statement ok SELECT * FROM pairs AS p WHERE p.src BETWEEN (select csr_id from (SELECT @@ -62,30 +61,27 @@ WHERE p.src BETWEEN (select csr_id from (SELECT k.rowid FROM Knows k JOIN student a on a.id = k.src JOIN student c on c.id = k.dst)) AND p.dst; ----- +statement ok +select 1; # LHS of IEJOIN - - - - # RHS of IEJoin # 0, - # (SELECT count(a.id) FROM Student a), - # CAST ( - # (SELECT sum(CREATE_CSR_VERTEX( - # 0, - # (SELECT count(a.id) FROM Student a), - # sub.dense_id, - # sub.cnt) - # ) - # FROM ( - # SELECT a.rowid as dense_id, count(k.src) as cnt - # FROM Student a - # LEFT JOIN Know k ON k.src = a.id - # GROUP BY a.rowid) sub - # ) - # AS BIGINT), - # a.rowid, - # c.rowid, - # k.rowid \ No newline at end of file +# (SELECT count(a.id) FROM Student a), +# CAST ( +# (SELECT sum(CREATE_CSR_VERTEX( +# 0, +# (SELECT count(a.id) FROM Student a), +# sub.dense_id, +# sub.cnt) +# ) +# FROM ( +# SELECT a.rowid as dense_id, count(k.src) as cnt +# FROM Student a +# LEFT JOIN Know k ON k.src = a.id +# GROUP BY a.rowid) sub +# ) +# AS BIGINT), +# a.rowid, +# c.rowid, +# k.rowid \ No newline at end of file From f9e078da84dd8827532010f96bdce217f79a4397 Mon Sep 17 00:00:00 2001 From: dtenwolde Date: Tue, 27 Feb 2024 10:40:28 +0100 Subject: [PATCH 032/249] Format fix --- .../logical_path_finding_operator.hpp | 30 +++++++++---------- 1 file changed, 14 insertions(+), 16 deletions(-) diff --git a/duckpgq/include/duckpgq/operators/logical_path_finding_operator.hpp b/duckpgq/include/duckpgq/operators/logical_path_finding_operator.hpp index 4172e4d5..49a7877c 100644 --- a/duckpgq/include/duckpgq/operators/logical_path_finding_operator.hpp +++ b/duckpgq/include/duckpgq/operators/logical_path_finding_operator.hpp @@ -3,24 +3,22 @@ namespace duckdb { - class LogicalPathFindingOperator : public LogicalExtensionOperator { - public: - explicit LogicalPathFindingOperator(unique_ptr plan) { - children.emplace_back(std::move(plan)); - } +class LogicalPathFindingOperator : public LogicalExtensionOperator { +public: + explicit LogicalPathFindingOperator(unique_ptr plan) { + children.emplace_back(std::move(plan)); + } - void Serialize(Serializer &serializer) const override { - throw InternalException("Path Finding Operator should not be serialized"); - } + void Serialize(Serializer &serializer) const override { + throw InternalException("Path Finding Operator should not be serialized"); + } - unique_ptr CreatePlan(ClientContext &context, PhysicalPlanGenerator &generator) override; + unique_ptr + CreatePlan(ClientContext &context, PhysicalPlanGenerator &generator) override; - vector GetColumnBindings() override; + vector GetColumnBindings() override; - std::string GetName() const override { - return "PATH_FINDING"; - } - }; + std::string GetName() const override { return "PATH_FINDING"; } +}; - -} +} // namespace duckdb From b1630b087091883aea40fde3fccc81a169e9cb74 Mon Sep 17 00:00:00 2001 From: dtenwolde Date: Tue, 27 Feb 2024 10:40:44 +0100 Subject: [PATCH 033/249] Format fix --- .../operators/logical_path_finding_operator.cpp | 14 +++++++------- 1 file changed, 7 insertions(+), 7 deletions(-) diff --git a/duckpgq/src/duckpgq/operators/logical_path_finding_operator.cpp b/duckpgq/src/duckpgq/operators/logical_path_finding_operator.cpp index bda1d5cf..c5e575b8 100644 --- a/duckpgq/src/duckpgq/operators/logical_path_finding_operator.cpp +++ b/duckpgq/src/duckpgq/operators/logical_path_finding_operator.cpp @@ -4,13 +4,13 @@ #include namespace duckdb { -unique_ptr -LogicalPathFindingOperator::CreatePlan(ClientContext &, - duckdb::PhysicalPlanGenerator &generator) { - D_ASSERT(children.size() == 2); - auto left = generator.CreatePlan(std::move(children[0])); - auto right = generator.CreatePlan(std::move(children[1])); - return make_uniq(*this, std::move(left), std::move(right)); +unique_ptr LogicalPathFindingOperator::CreatePlan( + ClientContext &, duckdb::PhysicalPlanGenerator &generator) { + D_ASSERT(children.size() == 2); + auto left = generator.CreatePlan(std::move(children[0])); + auto right = generator.CreatePlan(std::move(children[1])); + return make_uniq(*this, std::move(left), + std::move(right)); } vector LogicalPathFindingOperator::GetColumnBindings() { return LogicalOperator::GetColumnBindings(); From 2d51385a331afe5562ccbe3dc4db9dda132048de Mon Sep 17 00:00:00 2001 From: dtenwolde Date: Tue, 27 Feb 2024 10:48:44 +0100 Subject: [PATCH 034/249] Change constructro --- .../duckpgq/operators/logical_path_finding_operator.hpp | 6 ++++-- 1 file changed, 4 insertions(+), 2 deletions(-) diff --git a/duckpgq/include/duckpgq/operators/logical_path_finding_operator.hpp b/duckpgq/include/duckpgq/operators/logical_path_finding_operator.hpp index 49a7877c..d51f214d 100644 --- a/duckpgq/include/duckpgq/operators/logical_path_finding_operator.hpp +++ b/duckpgq/include/duckpgq/operators/logical_path_finding_operator.hpp @@ -5,8 +5,8 @@ namespace duckdb { class LogicalPathFindingOperator : public LogicalExtensionOperator { public: - explicit LogicalPathFindingOperator(unique_ptr plan) { - children.emplace_back(std::move(plan)); + explicit LogicalPathFindingOperator(vector> &children_) : LogicalExtensionOperator() { + children = std::move(children_); } void Serialize(Serializer &serializer) const override { @@ -19,6 +19,8 @@ class LogicalPathFindingOperator : public LogicalExtensionOperator { vector GetColumnBindings() override; std::string GetName() const override { return "PATH_FINDING"; } + + void ResolveTypes() override {}; }; } // namespace duckdb From e0715e1f4504f1a83ec7752b9a3e5b9545f55004 Mon Sep 17 00:00:00 2001 From: dtenwolde Date: Tue, 27 Feb 2024 12:22:43 +0100 Subject: [PATCH 035/249] Now changing the logical operator to path finding operator --- duckpgq/include/duckpgq_extension.hpp | 26 +++++++++++++++++++------- 1 file changed, 19 insertions(+), 7 deletions(-) diff --git a/duckpgq/include/duckpgq_extension.hpp b/duckpgq/include/duckpgq_extension.hpp index dd3619b8..4e8dd28b 100644 --- a/duckpgq/include/duckpgq_extension.hpp +++ b/duckpgq/include/duckpgq_extension.hpp @@ -26,12 +26,24 @@ class DuckpgqOptimizerExtension : public OptimizerExtension { } static bool HasBetweenExpression(LogicalOperator &op) { - if (op.type == LogicalOperatorType::LOGICAL_COMPARISON_JOIN) { - auto &get = op.Cast(); - for (auto &condition : get.conditions) { - if (condition.comparison == ExpressionType::COMPARE_GREATERTHANOREQUALTO) { - return true; - } +// if (op.type == LogicalOperatorType::LOGICAL_COMPARISON_JOIN) { +// auto &get = op.Cast(); +// for (auto &condition : get.conditions) { +// if (condition.comparison == ExpressionType::COMPARE_GREATERTHANOREQUALTO) { +// unique_ptr path_finding_operator = make_uniq(op.children); +// op.children.push_back(path_finding_operator); +// return true; +// } +// } +// } + for (auto &child : op.children) { + if (child->type == LogicalOperatorType::LOGICAL_COMPARISON_JOIN) { + auto &get = child->Cast(); + auto path_finding_operator = + make_uniq(get.children); + op.children.clear(); + op.children.push_back(std::move(path_finding_operator)); + return true; } } for (auto &child : op.children) { @@ -48,7 +60,7 @@ class DuckpgqOptimizerExtension : public OptimizerExtension { return; } std::cout << "Between expression found"; - plan = make_uniq(plan->children); +// plan = make_uniq(plan->children); } }; From 184b91783a2862c5c632be0349d275d720040ad2 Mon Sep 17 00:00:00 2001 From: dtenwolde Date: Tue, 27 Feb 2024 12:22:54 +0100 Subject: [PATCH 036/249] Throw not implemented --- duckpgq/src/duckpgq/operators/logical_path_finding_operator.cpp | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/duckpgq/src/duckpgq/operators/logical_path_finding_operator.cpp b/duckpgq/src/duckpgq/operators/logical_path_finding_operator.cpp index c5e575b8..f1c3227d 100644 --- a/duckpgq/src/duckpgq/operators/logical_path_finding_operator.cpp +++ b/duckpgq/src/duckpgq/operators/logical_path_finding_operator.cpp @@ -13,6 +13,6 @@ unique_ptr LogicalPathFindingOperator::CreatePlan( std::move(right)); } vector LogicalPathFindingOperator::GetColumnBindings() { - return LogicalOperator::GetColumnBindings(); + throw NotImplementedException("Column binding for path finding operator is not yet implemented."); } } // namespace duckdb From 042abd989b7979ecd78fc53152b8006e102d347b Mon Sep 17 00:00:00 2001 From: dtenwolde Date: Tue, 27 Feb 2024 13:05:21 +0100 Subject: [PATCH 037/249] Rename method --- duckpgq/include/duckpgq_extension.hpp | 9 +++------ 1 file changed, 3 insertions(+), 6 deletions(-) diff --git a/duckpgq/include/duckpgq_extension.hpp b/duckpgq/include/duckpgq_extension.hpp index 4e8dd28b..df23eb26 100644 --- a/duckpgq/include/duckpgq_extension.hpp +++ b/duckpgq/include/duckpgq_extension.hpp @@ -25,7 +25,7 @@ class DuckpgqOptimizerExtension : public OptimizerExtension { optimize_function = DuckpgqOptimizeFunction; } - static bool HasBetweenExpression(LogicalOperator &op) { + static bool InsertPathFindingOperator(LogicalOperator &op) { // if (op.type == LogicalOperatorType::LOGICAL_COMPARISON_JOIN) { // auto &get = op.Cast(); // for (auto &condition : get.conditions) { @@ -47,7 +47,7 @@ class DuckpgqOptimizerExtension : public OptimizerExtension { } } for (auto &child : op.children) { - if (HasBetweenExpression(*child)) { + if (InsertPathFindingOperator(*child)) { return true; } } @@ -56,12 +56,9 @@ class DuckpgqOptimizerExtension : public OptimizerExtension { static void DuckpgqOptimizeFunction(ClientContext &context, OptimizerExtensionInfo *info, duckdb::unique_ptr &plan) { - if (!HasBetweenExpression(*plan)) { + if (!InsertPathFindingOperator(*plan)) { return; } - std::cout << "Between expression found"; -// plan = make_uniq(plan->children); - } }; From 2eaa6e86564dc1f57c9ead591cea4098eadab38e Mon Sep 17 00:00:00 2001 From: dtenwolde Date: Tue, 27 Feb 2024 13:05:36 +0100 Subject: [PATCH 038/249] Try column binding method --- .../src/duckpgq/operators/logical_path_finding_operator.cpp | 5 ++++- 1 file changed, 4 insertions(+), 1 deletion(-) diff --git a/duckpgq/src/duckpgq/operators/logical_path_finding_operator.cpp b/duckpgq/src/duckpgq/operators/logical_path_finding_operator.cpp index f1c3227d..44508669 100644 --- a/duckpgq/src/duckpgq/operators/logical_path_finding_operator.cpp +++ b/duckpgq/src/duckpgq/operators/logical_path_finding_operator.cpp @@ -13,6 +13,9 @@ unique_ptr LogicalPathFindingOperator::CreatePlan( std::move(right)); } vector LogicalPathFindingOperator::GetColumnBindings() { - throw NotImplementedException("Column binding for path finding operator is not yet implemented."); + auto left_bindings = children[0]->GetColumnBindings(); + auto right_bindings = children[1]->GetColumnBindings(); + left_bindings.insert(left_bindings.end(), right_bindings.begin(), right_bindings.end()); + return left_bindings; } } // namespace duckdb From be18b0ca91dd7c273bfeb56a5ee12e2bf5fe0d12 Mon Sep 17 00:00:00 2001 From: dtenwolde Date: Tue, 27 Feb 2024 14:33:19 +0100 Subject: [PATCH 039/249] Implement ResolveTypes --- .../duckpgq/operators/logical_path_finding_operator.hpp | 2 +- .../src/duckpgq/operators/logical_path_finding_operator.cpp | 6 ++++++ 2 files changed, 7 insertions(+), 1 deletion(-) diff --git a/duckpgq/include/duckpgq/operators/logical_path_finding_operator.hpp b/duckpgq/include/duckpgq/operators/logical_path_finding_operator.hpp index d51f214d..96384c81 100644 --- a/duckpgq/include/duckpgq/operators/logical_path_finding_operator.hpp +++ b/duckpgq/include/duckpgq/operators/logical_path_finding_operator.hpp @@ -20,7 +20,7 @@ class LogicalPathFindingOperator : public LogicalExtensionOperator { std::string GetName() const override { return "PATH_FINDING"; } - void ResolveTypes() override {}; + void ResolveTypes() override; }; } // namespace duckdb diff --git a/duckpgq/src/duckpgq/operators/logical_path_finding_operator.cpp b/duckpgq/src/duckpgq/operators/logical_path_finding_operator.cpp index 44508669..c65ae1e0 100644 --- a/duckpgq/src/duckpgq/operators/logical_path_finding_operator.cpp +++ b/duckpgq/src/duckpgq/operators/logical_path_finding_operator.cpp @@ -18,4 +18,10 @@ vector LogicalPathFindingOperator::GetColumnBindings() { left_bindings.insert(left_bindings.end(), right_bindings.begin(), right_bindings.end()); return left_bindings; } + +void LogicalPathFindingOperator::ResolveTypes() { + types = children[0]->types; + auto right_types = children[1]->types; + types.insert(types.end(), right_types.begin(), right_types.end()); +} } // namespace duckdb From 13a31aec3096f5e3371b8ebd9a2e4256aa415be3 Mon Sep 17 00:00:00 2001 From: dtenwolde Date: Tue, 27 Feb 2024 14:33:43 +0100 Subject: [PATCH 040/249] Change config option debug print --- duckdb-pgq | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/duckdb-pgq b/duckdb-pgq index f0a71608..21c3f0f3 160000 --- a/duckdb-pgq +++ b/duckdb-pgq @@ -1 +1 @@ -Subproject commit f0a71608770498324167efa41cc738de11f8aa4e +Subproject commit 21c3f0f30f8619e63378b7e8253c5b8e5fe4a65f From 99819b05df27d527663740563f123c2c6c5ca8e1 Mon Sep 17 00:00:00 2001 From: dtenwolde Date: Tue, 27 Feb 2024 14:56:36 +0100 Subject: [PATCH 041/249] Comment --- .../duckpgq/operators/physical_path_finding_operator.hpp | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/duckpgq/include/duckpgq/operators/physical_path_finding_operator.hpp b/duckpgq/include/duckpgq/operators/physical_path_finding_operator.hpp index afaf5f12..ef665ba1 100644 --- a/duckpgq/include/duckpgq/operators/physical_path_finding_operator.hpp +++ b/duckpgq/include/duckpgq/operators/physical_path_finding_operator.hpp @@ -1,7 +1,7 @@ //===----------------------------------------------------------------------===// // DuckDB // -// duckdb/execution/operator/join/physical_path_finding.hpp +// duckpgq/operators/physical_path_finding_operator.hpp // // //===----------------------------------------------------------------------===// From a4d85cb7b2bbd7d0e758bd56c1355baeeee34b79 Mon Sep 17 00:00:00 2001 From: dtenwolde Date: Tue, 27 Feb 2024 15:03:09 +0100 Subject: [PATCH 042/249] Remove commented code --- .../physical_path_finding_operator.cpp | 782 +----------------- 1 file changed, 2 insertions(+), 780 deletions(-) diff --git a/duckpgq/src/duckpgq/operators/physical_path_finding_operator.cpp b/duckpgq/src/duckpgq/operators/physical_path_finding_operator.cpp index 5bdf3b2e..76bd8557 100644 --- a/duckpgq/src/duckpgq/operators/physical_path_finding_operator.cpp +++ b/duckpgq/src/duckpgq/operators/physical_path_finding_operator.cpp @@ -109,617 +109,20 @@ OperatorResultType PhysicalPathFinding::ExecuteInternal( //===--------------------------------------------------------------------===// // Source //===--------------------------------------------------------------------===// -// struct PathFindingUnion { -// using SortedTable = PhysicalRangeJoin::GlobalSortedTable; -// -// static idx_t AppendKey(SortedTable &table, ExpressionExecutor -// &executor, SortedTable &marked, int64_t increment, int64_t base, const idx_t -// block_idx); -// -// static void Sort(SortedTable &table) { -// auto &global_sort_state = table.global_sort_state; -// global_sort_state.PrepareMergePhase(); -// while (global_sort_state.sorted_blocks.size() > 1) { -// global_sort_state.InitializeMergeRound(); -// MergeSorter merge_sorter(global_sort_state, -// global_sort_state.buffer_manager); merge_sorter.PerformInMergeRound(); -// global_sort_state.CompleteMergeRound(true); -// } -// } -// -// template -// static vector ExtractColumn(SortedTable &table, idx_t -// col_idx) { vector result; result.reserve(table.count); -// -// auto &gstate = table.global_sort_state; -// auto &blocks = *gstate.sorted_blocks[0]->payload_data; -// PayloadScanner scanner(blocks, gstate, false); -// -// DataChunk payload; -// payload.Initialize(Allocator::DefaultAllocator(), -// gstate.payload_layout.GetTypes()); for (;;) { scanner.Scan(payload); const -// auto count = payload.size(); if (!count) { break; -// } -// -// const auto data_ptr = -// FlatVector::GetData(payload.data[col_idx]); result.insert(result.end(), -// data_ptr, data_ptr + count); -// } -// -// return result; -// } -// -// IEJoinUnion(ClientContext &context, const PhysicalIEJoin &op, -// SortedTable &t1, const idx_t b1, SortedTable &t2, const idx_t b2); -// -// idx_t SearchL1(idx_t pos); -// bool NextRow(); -// -// //! Inverted loop -// idx_t JoinComplexBlocks(SelectionVector &lsel, SelectionVector -// &rsel); -// -// //! L1 -// unique_ptr l1; -// //! L2 -// unique_ptr l2; -// -// //! Li -// vector li; -// //! P -// vector p; -// -// //! B -// vector bit_array; -// ValidityMask bit_mask; -// -// //! Bloom Filter -// static constexpr idx_t BLOOM_CHUNK_BITS = 1024; -// idx_t bloom_count; -// vector bloom_array; -// ValidityMask bloom_filter; -// -// //! Iteration state -// idx_t n; -// idx_t i; -// idx_t j; -// unique_ptr op1; -// unique_ptr off1; -// unique_ptr op2; -// unique_ptr off2; -// int64_t lrid; -// }; - -// idx_t IEJoinUnion::AppendKey(SortedTable &table, ExpressionExecutor -// &executor, SortedTable &marked, int64_t increment, int64_t base, const idx_t -// block_idx) { LocalSortState local_sort_state; -// local_sort_state.Initialize(marked.global_sort_state, -// marked.global_sort_state.buffer_manager); -// -// // Reading -// const auto valid = table.count - table.has_null; -// auto &gstate = table.global_sort_state; -// PayloadScanner scanner(gstate, block_idx); -// auto table_idx = block_idx * gstate.block_capacity; -// -// DataChunk scanned; -// scanned.Initialize(Allocator::DefaultAllocator(), -// scanner.GetPayloadTypes()); -// -// // Writing -// auto types = local_sort_state.sort_layout->logical_types; -// const idx_t payload_idx = types.size(); -// -// const auto &payload_types = local_sort_state.payload_layout->GetTypes(); -// types.insert(types.end(), payload_types.begin(), payload_types.end()); -// const idx_t rid_idx = types.size() - 1; -// -// DataChunk keys; -// DataChunk payload; -// keys.Initialize(Allocator::DefaultAllocator(), types); -// -// idx_t inserted = 0; -// for (auto rid = base; table_idx < valid;) { -// scanner.Scan(scanned); -// -// // NULLs are at the end, so stop when we reach them -// auto scan_count = scanned.size(); -// if (table_idx + scan_count > valid) { -// scan_count = valid - table_idx; -// scanned.SetCardinality(scan_count); -// } -// if (scan_count == 0) { -// break; -// } -// table_idx += scan_count; -// -// // Compute the input columns from the payload -// keys.Reset(); -// keys.Split(payload, rid_idx); -// executor.Execute(scanned, keys); -// -// // Mark the rid column -// payload.data[0].Sequence(rid, increment, scan_count); -// payload.SetCardinality(scan_count); -// keys.Fuse(payload); -// rid += increment * scan_count; -// -// // Sort on the sort columns (which will no longer be needed) -// keys.Split(payload, payload_idx); -// local_sort_state.SinkChunk(keys, payload); -// inserted += scan_count; -// keys.Fuse(payload); -// -// // Flush when we have enough data -// if (local_sort_state.SizeInBytes() >= marked.memory_per_thread) -// { local_sort_state.Sort(marked.global_sort_state, true); -// } -// } -// marked.global_sort_state.AddLocalState(local_sort_state); -// marked.count += inserted; -// -// return inserted; -// } -// -// IEJoinUnion::IEJoinUnion(ClientContext &context, const PhysicalIEJoin &op, -// SortedTable &t1, const idx_t b1, SortedTable &t2, const idx_t b2) : n(0), -// i(0) { -// // input : query Q with 2 join predicates t1.X op1 t2.X' and t1.Y op2 -// t2.Y', tables T, T' of sizes m and n resp. -// // output: a list of tuple pairs (ti , tj) -// // Note that T/T' are already sorted on X/X' and contain the payload -// data -// // We only join the two block numbers and use the sizes of the blocks as -// the counts -// -// // 0. Filter out tables with no overlap -// if (!t1.BlockSize(b1) || !t2.BlockSize(b2)) { -// return; -// } -// -// const auto &cmp1 = op.conditions[0].comparison; -// SBIterator bounds1(t1.global_sort_state, cmp1); -// SBIterator bounds2(t2.global_sort_state, cmp1); -// -// // t1.X[0] op1 t2.X'[-1] -// bounds1.SetIndex(bounds1.block_capacity * b1); -// bounds2.SetIndex(bounds2.block_capacity * b2 + t2.BlockSize(b2) - 1); -// if (!bounds1.Compare(bounds2)) { -// return; -// } -// -// // 1. let L1 (resp. L2) be the array of column X (resp. Y ) -// const auto &order1 = op.lhs_orders[0][0]; -// const auto &order2 = op.lhs_orders[1][0]; -// -// // 2. if (op1 ∈ {>, ≥}) sort L1 in descending order -// // 3. else if (op1 ∈ {<, ≤}) sort L1 in ascending order -// -// // For the union algorithm, we make a unified table with the keys and -// the rids as the payload: -// // X/X', Y/Y', R/R'/Li -// // The first position is the sort key. -// vector types; -// types.emplace_back(order2.expression->return_type); -// types.emplace_back(LogicalType::BIGINT); -// RowLayout payload_layout; -// payload_layout.Initialize(types); -// -// // Sort on the first expression -// auto ref = -// make_uniq(order1.expression->return_type, 0); -// vector orders; -// orders.emplace_back(order1.type, order1.null_order, std::move(ref)); -// -// l1 = make_uniq(context, orders, payload_layout); -// -// // LHS has positive rids -// ExpressionExecutor l_executor(context); -// l_executor.AddExpression(*order1.expression); -// l_executor.AddExpression(*order2.expression); -// AppendKey(t1, l_executor, *l1, 1, 1, b1); -// -// // RHS has negative rids -// ExpressionExecutor r_executor(context); -// r_executor.AddExpression(*op.rhs_orders[0][0].expression); -// r_executor.AddExpression(*op.rhs_orders[1][0].expression); -// AppendKey(t2, r_executor, *l1, -1, -1, b2); -// -// if (l1->global_sort_state.sorted_blocks.empty()) { -// return; -// } -// -// Sort(*l1); -// -// op1 = make_uniq(l1->global_sort_state, cmp1); -// off1 = make_uniq(l1->global_sort_state, cmp1); -// -// // We don't actually need the L1 column, just its sort key, which is in -// the sort blocks li = ExtractColumn(*l1, types.size() - 1); -// -// // 4. if (op2 ∈ {>, ≥}) sort L2 in ascending order -// // 5. else if (op2 ∈ {<, ≤}) sort L2 in descending order -// -// // We sort on Y/Y' to obtain the sort keys and the permutation array. -// // For this we just need a two-column table of Y, P -// types.clear(); -// types.emplace_back(LogicalType::BIGINT); -// payload_layout.Initialize(types); -// -// // Sort on the first expression -// orders.clear(); -// ref = -// make_uniq(order2.expression->return_type, 0); -// orders.emplace_back(order2.type, order2.null_order, std::move(ref)); -// -// ExpressionExecutor executor(context); -// executor.AddExpression(*orders[0].expression); -// -// l2 = make_uniq(context, orders, payload_layout); -// for (idx_t base = 0, block_idx = 0; block_idx < l1->BlockCount(); -// ++block_idx) { base += AppendKey(*l1, executor, *l2, 1, base, block_idx); -// } -// -// Sort(*l2); -// -// // We don't actually need the L2 column, just its sort key, which is in -// the sort blocks -// -// // 6. compute the permutation array P of L2 w.r.t. L1 -// p = ExtractColumn(*l2, types.size() - 1); -// -// // 7. initialize bit-array B (|B| = n), and set all bits to 0 -// n = l2->count.load(); -// bit_array.resize(ValidityMask::EntryCount(n), 0); -// bit_mask.Initialize(bit_array.data()); -// -// // Bloom filter -// bloom_count = (n + (BLOOM_CHUNK_BITS - 1)) / BLOOM_CHUNK_BITS; -// bloom_array.resize(ValidityMask::EntryCount(bloom_count), 0); -// bloom_filter.Initialize(bloom_array.data()); -// -// // 11. for(i←1 to n) do -// const auto &cmp2 = op.conditions[1].comparison; -// op2 = make_uniq(l2->global_sort_state, cmp2); -// off2 = make_uniq(l2->global_sort_state, cmp2); -// i = 0; -// j = 0; -// (void)NextRow(); -// } -// -// idx_t IEJoinUnion::SearchL1(idx_t pos) { -// // Perform an exponential search in the appropriate direction -// op1->SetIndex(pos); -// -// idx_t step = 1; -// auto hi = pos; -// auto lo = pos; -// if (!op1->cmp) { -// // Scan left for loose inequality -// lo -= MinValue(step, lo); -// step *= 2; -// off1->SetIndex(lo); -// while (lo > 0 && op1->Compare(*off1)) { -// hi = lo; -// lo -= MinValue(step, lo); -// step *= 2; -// off1->SetIndex(lo); -// } -// } else { -// // Scan right for strict inequality -// hi += MinValue(step, n - hi); -// step *= 2; -// off1->SetIndex(hi); -// while (hi < n && !op1->Compare(*off1)) { -// lo = hi; -// hi += MinValue(step, n - hi); -// step *= 2; -// off1->SetIndex(hi); -// } -// } -// -// // Binary search the target area -// while (lo < hi) { -// const auto mid = lo + (hi - lo) / 2; -// off1->SetIndex(mid); -// if (op1->Compare(*off1)) { -// hi = mid; -// } else { -// lo = mid + 1; -// } -// } -// -// off1->SetIndex(lo); -// -// return lo; -// } -// -// bool IEJoinUnion::NextRow() { -// for (; i < n; ++i) { -// // 12. pos ← P[i] -// auto pos = p[i]; -// lrid = li[pos]; -// if (lrid < 0) { -// continue; -// } -// -// // 16. B[pos] ← 1 -// op2->SetIndex(i); -// for (; off2->GetIndex() < n; ++(*off2)) { -// if (!off2->Compare(*op2)) { -// break; -// } -// const auto p2 = p[off2->GetIndex()]; -// if (li[p2] < 0) { -// // Only mark rhs matches. -// bit_mask.SetValid(p2); -// bloom_filter.SetValid(p2 / BLOOM_CHUNK_BITS); -// } -// } -// -// // 9. if (op1 ∈ {≤,≥} and op2 ∈ {≤,≥}) eqOff = 0 -// // 10. else eqOff = 1 -// // No, because there could be more than one equal value. -// // Find the leftmost off1 where L1[pos] op1 L1[off1..n] -// // These are the rows that satisfy the op1 condition -// // and that is where we should start scanning B from -// j = SearchL1(pos); -// -// return true; -// } -// return false; -// } -// -// static idx_t NextValid(const ValidityMask &bits, idx_t j, const idx_t n) { -// if (j >= n) { -// return n; -// } -// -// // We can do a first approximation by checking entries one at a time -// // which gives 64:1. -// idx_t entry_idx, idx_in_entry; -// bits.GetEntryIndex(j, entry_idx, idx_in_entry); -// auto entry = bits.GetValidityEntry(entry_idx++); -// -// // Trim the bits before the start position -// entry &= (ValidityMask::ValidityBuffer::MAX_ENTRY << idx_in_entry); -// -// // Check the non-ragged entries -// for (const auto entry_count = bits.EntryCount(n); entry_idx < -// entry_count; ++entry_idx) { if (entry) { for (; idx_in_entry < -// bits.BITS_PER_VALUE; ++idx_in_entry, ++j) { if (bits.RowIsValid(entry, -// idx_in_entry)) { return j; -// } -// } -// } else { -// j += bits.BITS_PER_VALUE - idx_in_entry; -// } -// -// entry = bits.GetValidityEntry(entry_idx); -// idx_in_entry = 0; -// } -// -// // Check the final entry -// for (; j < n; ++idx_in_entry, ++j) { -// if (bits.RowIsValid(entry, idx_in_entry)) { -// return j; -// } -// } -// -// return j; -// } -// -// idx_t IEJoinUnion::JoinComplexBlocks(SelectionVector &lsel, SelectionVector -// &rsel) { -// // 8. initialize join result as an empty list for tuple pairs -// idx_t result_count = 0; -// -// // 11. for(i←1 to n) do -// while (i < n) { -// // 13. for (j ← pos+eqOff to n) do -// for (;;) { -// // 14. if B[j] = 1 then -// -// // Use the Bloom filter to find candidate blocks -// while (j < n) { -// auto bloom_begin = NextValid(bloom_filter, j / -// BLOOM_CHUNK_BITS, bloom_count) * BLOOM_CHUNK_BITS; auto bloom_end = -// MinValue(n, bloom_begin + BLOOM_CHUNK_BITS); -// -// j = MaxValue(j, bloom_begin); -// j = NextValid(bit_mask, j, bloom_end); -// if (j < bloom_end) { -// break; -// } -// } -// -// if (j >= n) { -// break; -// } -// -// // Filter out tuples with the same sign (they come from -// the same table) const auto rrid = li[j]; -// ++j; -// -// // 15. add tuples w.r.t. (L1[j], L1[i]) to join result -// if (lrid > 0 && rrid < 0) { -// lsel.set_index(result_count, sel_t(+lrid - 1)); -// rsel.set_index(result_count, sel_t(-rrid - 1)); -// ++result_count; -// if (result_count == STANDARD_VECTOR_SIZE) { -// // out of space! -// return result_count; -// } -// } -// } -// ++i; -// -// if (!NextRow()) { -// break; -// } -// } -// -// return result_count; -// } -// + class PathFindingLocalSourceState : public LocalSourceState { public: explicit PathFindingLocalSourceState(ClientContext &context, const PhysicalPathFinding &op) : op(op), true_sel(STANDARD_VECTOR_SIZE) { - // left_executor(context), right_executor(context), left_matches(nullptr), - // right_matches(nullptr) auto &allocator = Allocator::Get(context); - // unprojected.Initialize(allocator, op.unprojected_types); - // - // if (op.conditions.size() < 3) { - // return; - // } - // - // vector left_types; - // vector right_types; - // for (idx_t i = 2; i < op.conditions.size(); ++i) { - // const auto &cond = op.conditions[i]; - // - // left_types.push_back(cond.left->return_type); - // left_executor.AddExpression(*cond.left); - // - // right_types.push_back(cond.left->return_type); - // right_executor.AddExpression(*cond.right); - // } - // - // left_keys.Initialize(allocator, left_types); - // right_keys.Initialize(allocator, right_types); - // } - - // idx_t SelectOuterRows(bool *matches) { - // idx_t count = 0; - // for (; outer_idx < outer_count; ++outer_idx) { - // if (!matches[outer_idx]) { - // true_sel.set_index(count++, outer_idx); - // if (count >= STANDARD_VECTOR_SIZE) { - // outer_idx++; - // break; - // } - // } - // } - - // return count; } const PhysicalPathFinding &op; - // Joining - // unique_ptr joiner; - // - // idx_t left_base; - // idx_t left_block_index; - // - // idx_t right_base; - // idx_t right_block_index; - // Trailing predicates SelectionVector true_sel; - - // ExpressionExecutor left_executor; - // DataChunk left_keys; - // - // ExpressionExecutor right_executor; - // DataChunk right_keys; - // - // DataChunk unprojected; - // - // // Outer joins - // idx_t outer_idx; - // idx_t outer_count; - // bool *left_matches; - // bool *right_matches; }; -// void PhysicalIEJoin::ResolveComplexJoin(ExecutionContext &context, DataChunk -// &result, LocalSourceState &state_p) const { auto &state = -// state_p.Cast(); auto &ie_sink = -// sink_state->Cast(); auto &left_table = *ie_sink.tables[0]; -// auto &right_table = *ie_sink.tables[1]; -// -// const auto left_cols = children[0]->GetTypes().size(); -// auto &chunk = state.unprojected; -// do { -// SelectionVector lsel(STANDARD_VECTOR_SIZE); -// SelectionVector rsel(STANDARD_VECTOR_SIZE); -// auto result_count = state.joiner->JoinComplexBlocks(lsel, rsel); -// if (result_count == 0) { -// // exhausted this pair -// return; -// } -// -// // found matches: extract them -// -// chunk.Reset(); -// SliceSortedPayload(chunk, left_table.global_sort_state, -// state.left_block_index, lsel, result_count, 0); SliceSortedPayload(chunk, -// right_table.global_sort_state, state.right_block_index, rsel, result_count, -// left_cols); -// chunk.SetCardinality(result_count); -// -// auto sel = FlatVector::IncrementalSelectionVector(); -// if (conditions.size() > 2) { -// // If there are more expressions to compute, -// // split the result chunk into the left and right halves -// // so we can compute the values for comparison. -// const auto tail_cols = conditions.size() - 2; -// -// DataChunk right_chunk; -// chunk.Split(right_chunk, left_cols); -// state.left_executor.SetChunk(chunk); -// state.right_executor.SetChunk(right_chunk); -// -// auto tail_count = result_count; -// auto true_sel = &state.true_sel; -// for (size_t cmp_idx = 0; cmp_idx < tail_cols; ++cmp_idx) -// { auto &left = state.left_keys.data[cmp_idx]; -// state.left_executor.ExecuteExpression(cmp_idx, -// left); -// -// auto &right = state.right_keys.data[cmp_idx]; -// state.right_executor.ExecuteExpression(cmp_idx, -// right); -// -// if (tail_count < result_count) { -// left.Slice(*sel, tail_count); -// right.Slice(*sel, tail_count); -// } -// tail_count = SelectJoinTail(conditions[cmp_idx + -// 2].comparison, left, right, sel, tail_count, true_sel); sel = true_sel; -// } -// chunk.Fuse(right_chunk); -// -// if (tail_count < result_count) { -// result_count = tail_count; -// chunk.Slice(*sel, result_count); -// } -// } -// -// // We need all of the data to compute other predicates, -// // but we only return what is in the projection map -// ProjectResult(chunk, result); -// -// // found matches: mark the found matches if required -// if (left_table.found_match) { -// for (idx_t i = 0; i < result_count; i++) { -// left_table.found_match[state.left_base + -// lsel[sel->get_index(i)]] = true; -// } -// } -// if (right_table.found_match) { -// for (idx_t i = 0; i < result_count; i++) { -// right_table.found_match[state.right_base + -// rsel[sel->get_index(i)]] = true; -// } -// } -// result.Verify(); -// } while (result.size() == 0); -// } - class PathFindingGlobalSourceState : public GlobalSourceState { public: explicit PathFindingGlobalSourceState(const PhysicalPathFinding &op) @@ -730,36 +133,6 @@ class PathFindingGlobalSourceState : public GlobalSourceState { if (initialized) { return; } - - // Compute the starting row for reach block - // (In theory these are all the same size, but you never know...) - // auto &left_table = *sink_state.tables[0]; - // const auto left_blocks = left_table.BlockCount(); - // idx_t left_base = 0; - // - // for (size_t lhs = 0; lhs < left_blocks; ++lhs) { - // left_bases.emplace_back(left_base); - // left_base += left_table.BlockSize(lhs); - // } - // - // auto &right_table = *sink_state.tables[1]; - // const auto right_blocks = right_table.BlockCount(); - // idx_t right_base = 0; - // for (size_t rhs = 0; rhs < right_blocks; ++rhs) { - // right_bases.emplace_back(right_base); - // right_base += right_table.BlockSize(rhs); - // } - // - // // Outer join block counts - // if (left_table.found_match) { - // left_outers = left_blocks; - // } - // - // if (right_table.found_match) { - // right_outers = right_blocks; - // } - - // Ready for action initialized = true; } @@ -768,99 +141,17 @@ class PathFindingGlobalSourceState : public GlobalSourceState { // We can't leverage any more threads than block pairs. const auto &sink_state = (op.sink_state->Cast()); return 1; - // return sink_state.tables[0]->BlockCount() * - // sink_state.tables[1]->BlockCount(); + } void GetNextPair(ClientContext &client, PathFindingGlobalState &gstate, PathFindingLocalSourceState &lstate) { - // auto &left_table = *gstate.tables[0]; - // auto &right_table = *gstate.tables[1]; - // - // const auto left_blocks = left_table.BlockCount(); - // const auto right_blocks = right_table.BlockCount(); - // const auto pair_count = left_blocks * right_blocks; - // - // // Regular block - // const auto i = next_pair++; - // if (i < pair_count) { - // const auto b1 = i / right_blocks; - // const auto b2 = i % right_blocks; - // - // lstate.left_block_index = b1; - // lstate.left_base = left_bases[b1]; - // - // lstate.right_block_index = b2; - // lstate.right_base = right_bases[b2]; - // - // lstate.joiner = make_uniq(client, op, left_table, b1, - // right_table, b2); return; - // } - - // // Outer joins - // if (!left_outers && !right_outers) { - // return; - // } - // - // // Spin wait for regular blocks to finish(!) - // while (completed < pair_count) { - // std::this_thread::yield(); - // } - // - // // Left outer blocks - // const auto l = next_left++; - // if (l < left_outers) { - // lstate.joiner = nullptr; - // lstate.left_block_index = l; - // lstate.left_base = left_bases[l]; - // - // lstate.left_matches = left_table.found_match.get() + lstate.left_base; - // lstate.outer_idx = 0; - // lstate.outer_count = left_table.BlockSize(l); - // return; - // } else { - // lstate.left_matches = nullptr; - // } - // - // // Right outer block - // const auto r = next_right++; - // if (r < right_outers) { - // lstate.joiner = nullptr; - // lstate.right_block_index = r; - // lstate.right_base = right_bases[r]; - // - // lstate.right_matches = right_table.found_match.get() + - // lstate.right_base; lstate.outer_idx = 0; lstate.outer_count = - // right_table.BlockSize(r); return; } else { lstate.right_matches = - // nullptr; - // } } - // void PairCompleted(ClientContext &client, PathFindingGlobalState &gstate, - // PahtFindingLocalSourceState &lstate) { lstate.joiner.reset(); - // ++completed; - // GetNextPair(client, gstate, lstate); - // } - const PhysicalPathFinding &op; mutex lock; bool initialized; - - // Join queue state - // std::atomic next_pair; - // std::atomic completed; - - // Block base row number - // vector left_bases; - // vector right_bases; - - // Outer joins - // idx_t left_outers; - // std::atomic next_left; - - // idx_t right_outers; - // std::atomic next_right; }; unique_ptr @@ -883,75 +174,6 @@ PhysicalPathFinding::GetData(ExecutionContext &context, DataChunk &result, pf_gstate.Initialize(pf_sink); - // if (!ie_lstate.joiner && !ie_lstate.left_matches && - // !ie_lstate.right_matches) { ie_gstate.GetNextPair(context.client, ie_sink, - // ie_lstate); - // } - - // Process INNER results - // while (pf_lstate.joiner) { - // ResolveComplexJoin(context, result, ie_lstate); - // - // if (result.size()) { - // return SourceResultType::HAVE_MORE_OUTPUT; - // } - // - // ie_gstate.PairCompleted(context.client, ie_sink, ie_lstate); - // } - - // Process LEFT OUTER results - // const auto left_cols = children[0]->GetTypes().size(); - // while (ie_lstate.left_matches) { - // const idx_t count = ie_lstate.SelectOuterRows(ie_lstate.left_matches); - // if (!count) { - // ie_gstate.GetNextPair(context.client, ie_sink, ie_lstate); - // continue; - // } - // auto &chunk = ie_lstate.unprojected; - // chunk.Reset(); - // SliceSortedPayload(chunk, ie_sink.tables[0]->global_sort_state, - // ie_lstate.left_block_index, ie_lstate.true_sel, count); - // - // // Fill in NULLs to the right - // for (auto col_idx = left_cols; col_idx < chunk.ColumnCount(); ++col_idx) - // { chunk.data[col_idx].SetVectorType(VectorType::CONSTANT_VECTOR); - // ConstantVector::SetNull(chunk.data[col_idx], true); - // } - // - // ProjectResult(chunk, result); - // result.SetCardinality(count); - // result.Verify(); - // - // return result.size() == 0 ? SourceResultType::FINISHED : - // SourceResultType::HAVE_MORE_OUTPUT; - // } - - // Process RIGHT OUTER results - // while (ie_lstate.right_matches) { - // const idx_t count = ie_lstate.SelectOuterRows(ie_lstate.right_matches); - // if (!count) { - // ie_gstate.GetNextPair(context.client, ie_sink, ie_lstate); - // continue; - // } - // - // auto &chunk = ie_lstate.unprojected; - // chunk.Reset(); - // SliceSortedPayload(chunk, ie_sink.tables[1]->global_sort_state, - // ie_lstate.right_block_index, ie_lstate.true_sel, count, left_cols); - // - // // Fill in NULLs to the left - // for (idx_t col_idx = 0; col_idx < left_cols; ++col_idx) { - // chunk.data[col_idx].SetVectorType(VectorType::CONSTANT_VECTOR); - // ConstantVector::SetNull(chunk.data[col_idx], true); - // } - // - // ProjectResult(chunk, result); - // result.SetCardinality(count); - // result.Verify(); - // - // break; - // } - return result.size() == 0 ? SourceResultType::FINISHED : SourceResultType::HAVE_MORE_OUTPUT; } From 92536ff823a439a377038d2aef19279c801b46ae Mon Sep 17 00:00:00 2001 From: dtenwolde Date: Tue, 27 Feb 2024 15:13:46 +0100 Subject: [PATCH 043/249] Add expressionexecutor --- .../duckpgq/operators/physical_path_finding_operator.cpp | 6 +++++- 1 file changed, 5 insertions(+), 1 deletion(-) diff --git a/duckpgq/src/duckpgq/operators/physical_path_finding_operator.cpp b/duckpgq/src/duckpgq/operators/physical_path_finding_operator.cpp index 76bd8557..738cbd96 100644 --- a/duckpgq/src/duckpgq/operators/physical_path_finding_operator.cpp +++ b/duckpgq/src/duckpgq/operators/physical_path_finding_operator.cpp @@ -114,13 +114,17 @@ class PathFindingLocalSourceState : public LocalSourceState { public: explicit PathFindingLocalSourceState(ClientContext &context, const PhysicalPathFinding &op) - : op(op), true_sel(STANDARD_VECTOR_SIZE) { + : op(op), true_sel(STANDARD_VECTOR_SIZE), left_executor(context), right_executor(context){ } const PhysicalPathFinding &op; // Trailing predicates SelectionVector true_sel; + + ExpressionExecutor left_executor; + ExpressionExecutor right_executor; + }; class PathFindingGlobalSourceState : public GlobalSourceState { From 2dd2996ca695531979e02c009fd82567dcfadb67 Mon Sep 17 00:00:00 2001 From: dtenwolde Date: Tue, 27 Feb 2024 15:40:03 +0100 Subject: [PATCH 044/249] Changed base class to PhysicalComparisonJoin so two children are allowed --- .../duckpgq/operators/physical_path_finding_operator.hpp | 3 ++- .../src/duckpgq/operators/physical_path_finding_operator.cpp | 2 +- 2 files changed, 3 insertions(+), 2 deletions(-) diff --git a/duckpgq/include/duckpgq/operators/physical_path_finding_operator.hpp b/duckpgq/include/duckpgq/operators/physical_path_finding_operator.hpp index ef665ba1..fcfc11a0 100644 --- a/duckpgq/include/duckpgq/operators/physical_path_finding_operator.hpp +++ b/duckpgq/include/duckpgq/operators/physical_path_finding_operator.hpp @@ -8,11 +8,12 @@ #pragma once +#include "duckdb/execution/operator/join/physical_comparison_join.hpp" #include "duckdb/execution/physical_operator.hpp" #include "duckdb/planner/operator/logical_extension_operator.hpp" namespace duckdb { -class PhysicalPathFinding : public CachingPhysicalOperator { +class PhysicalPathFinding : public PhysicalComparisonJoin { public: static constexpr const PhysicalOperatorType TYPE = PhysicalOperatorType::EXTENSION; diff --git a/duckpgq/src/duckpgq/operators/physical_path_finding_operator.cpp b/duckpgq/src/duckpgq/operators/physical_path_finding_operator.cpp index 738cbd96..9258039e 100644 --- a/duckpgq/src/duckpgq/operators/physical_path_finding_operator.cpp +++ b/duckpgq/src/duckpgq/operators/physical_path_finding_operator.cpp @@ -13,7 +13,7 @@ namespace duckdb { PhysicalPathFinding::PhysicalPathFinding(LogicalExtensionOperator &op, unique_ptr left, unique_ptr right) - : CachingPhysicalOperator(TYPE, op.types, 0) { + : PhysicalComparisonJoin(op, TYPE, {}, JoinType::INNER, 0) { children.push_back(std::move(left)); children.push_back(std::move(right)); } From fcb9d576571285e20b58f9da18bcf797563c91a7 Mon Sep 17 00:00:00 2001 From: dtenwolde Date: Wed, 28 Feb 2024 10:13:10 +0100 Subject: [PATCH 045/249] Adding csr to operator --- .../physical_path_finding_operator.hpp | 17 +++++++++++++++++ 1 file changed, 17 insertions(+) diff --git a/duckpgq/include/duckpgq/operators/physical_path_finding_operator.hpp b/duckpgq/include/duckpgq/operators/physical_path_finding_operator.hpp index fcfc11a0..30403626 100644 --- a/duckpgq/include/duckpgq/operators/physical_path_finding_operator.hpp +++ b/duckpgq/include/duckpgq/operators/physical_path_finding_operator.hpp @@ -13,6 +13,23 @@ #include "duckdb/planner/operator/logical_extension_operator.hpp" namespace duckdb { + +class CompressedSparseRow { + CompressedSparseRow() {}; + ~CompressedSparseRow() { + delete[] v; + } + atomic *v; + vector e; + vector edge_ids; + vector w; + vector w_double; + bool initialized_v = false; + bool initialized_e = false; + bool initialized_w = false; + size_t vsize; +}; + class PhysicalPathFinding : public PhysicalComparisonJoin { public: static constexpr const PhysicalOperatorType TYPE = From 8fc86188bc4f75d671c2a17c1202ea2b93595bbe Mon Sep 17 00:00:00 2001 From: dtenwolde Date: Wed, 28 Feb 2024 10:14:48 +0100 Subject: [PATCH 046/249] Adding csr to operator --- .../src/duckpgq/operators/physical_path_finding_operator.cpp | 2 ++ 1 file changed, 2 insertions(+) diff --git a/duckpgq/src/duckpgq/operators/physical_path_finding_operator.cpp b/duckpgq/src/duckpgq/operators/physical_path_finding_operator.cpp index 9258039e..6565ff96 100644 --- a/duckpgq/src/duckpgq/operators/physical_path_finding_operator.cpp +++ b/duckpgq/src/duckpgq/operators/physical_path_finding_operator.cpp @@ -43,6 +43,8 @@ class PathFindingGlobalState : public GlobalSinkState { void Sink(DataChunk &input, PathFindingLocalState &lstate) {} + unique_ptr csr; + }; unique_ptr From a9520c48b747206ac20ce0984a9c1ae17002106b Mon Sep 17 00:00:00 2001 From: dtenwolde Date: Wed, 28 Feb 2024 11:09:55 +0100 Subject: [PATCH 047/249] Adding a local csr state --- .../physical_path_finding_operator.hpp | 58 +++++++++++++------ .../physical_path_finding_operator.cpp | 16 +++-- 2 files changed, 50 insertions(+), 24 deletions(-) diff --git a/duckpgq/include/duckpgq/operators/physical_path_finding_operator.hpp b/duckpgq/include/duckpgq/operators/physical_path_finding_operator.hpp index 30403626..d5106d7f 100644 --- a/duckpgq/include/duckpgq/operators/physical_path_finding_operator.hpp +++ b/duckpgq/include/duckpgq/operators/physical_path_finding_operator.hpp @@ -8,29 +8,53 @@ #pragma once +#include "duckdb/common/types/row/row_layout.hpp" #include "duckdb/execution/operator/join/physical_comparison_join.hpp" #include "duckdb/execution/physical_operator.hpp" #include "duckdb/planner/operator/logical_extension_operator.hpp" namespace duckdb { -class CompressedSparseRow { - CompressedSparseRow() {}; - ~CompressedSparseRow() { - delete[] v; - } - atomic *v; - vector e; - vector edge_ids; - vector w; - vector w_double; - bool initialized_v = false; - bool initialized_e = false; - bool initialized_w = false; - size_t vsize; -}; +struct GlobalCompressedSparseRow; class PhysicalPathFinding : public PhysicalComparisonJoin { +public: + class LocalCompressedSparseRow { + public: + LocalCompressedSparseRow(ClientContext &context, + const PhysicalPathFinding &op); + + void Sink(DataChunk &input, GlobalCompressedSparseRow &global_csr); + + + //! The hosting operator + const PhysicalPathFinding &op; + //! Holds a vector of incoming columns + DataChunk keys; + //! Local copy of the expression executor + ExpressionExecutor executor + + }; + + class GlobalCompressedSparseRow { + public: + GlobalCompressedSparseRow(ClientContext &context, + RowLayout &payload_layout){ + + }; + ~GlobalCompressedSparseRow() { delete[] v; } + + atomic *v; + vector e; + vector edge_ids; + vector w; + vector w_double; + bool initialized_v = false; + bool initialized_e = false; + bool initialized_w = false; + size_t vsize; + }; + public: static constexpr const PhysicalOperatorType TYPE = PhysicalOperatorType::EXTENSION; @@ -40,10 +64,6 @@ class PhysicalPathFinding : public PhysicalComparisonJoin { unique_ptr left, unique_ptr right); - // vector join_key_types; - // vector> lhs_orders; - // vector> rhs_orders; - public: // CachingOperator Interface OperatorResultType ExecuteInternal(ExecutionContext &context, diff --git a/duckpgq/src/duckpgq/operators/physical_path_finding_operator.cpp b/duckpgq/src/duckpgq/operators/physical_path_finding_operator.cpp index 6565ff96..35a3b155 100644 --- a/duckpgq/src/duckpgq/operators/physical_path_finding_operator.cpp +++ b/duckpgq/src/duckpgq/operators/physical_path_finding_operator.cpp @@ -18,17 +18,20 @@ PhysicalPathFinding::PhysicalPathFinding(LogicalExtensionOperator &op, children.push_back(std::move(right)); } +PhysicalPathFinding::LocalCompressedSparseRow::LocalCompressedSparseRow( + duckdb::ClientContext &context, const duckdb::PhysicalPathFinding &op) : + op(op), executor(context) {} + //===--------------------------------------------------------------------===// // Sink //===--------------------------------------------------------------------===// class PathFindingLocalState : public LocalSinkState { public: + using LocalCompressedSparseRow = PhysicalPathFinding::LocalCompressedSparseRow; PathFindingLocalState(ClientContext &context, const PhysicalPathFinding &op, const idx_t child) - : context(context), op(op), child(child) {} - ClientContext &context; - const PhysicalPathFinding &op; - const idx_t child; + : local_csr(context, op) {} + LocalCompressedSparseRow local_csr; }; class PathFindingGlobalState : public GlobalSinkState { @@ -36,6 +39,9 @@ class PathFindingGlobalState : public GlobalSinkState { public: PathFindingGlobalState(ClientContext &context, const PhysicalPathFinding &op) { + RowLayout rhs_layout; + rhs_layout.Initialize(op.children[1]->types); + csr = make_uniq(context, rhs_layout); } PathFindingGlobalState(PathFindingGlobalState &prev) @@ -43,7 +49,7 @@ class PathFindingGlobalState : public GlobalSinkState { void Sink(DataChunk &input, PathFindingLocalState &lstate) {} - unique_ptr csr; + unique_ptr csr; }; From 72b9c0837a0fdd0b588e7a6d33f7a7d19c0accb3 Mon Sep 17 00:00:00 2001 From: dtenwolde Date: Wed, 6 Mar 2024 16:51:40 +0100 Subject: [PATCH 048/249] Allow non-const first argument for create_csr_edge --- duckpgq/src/duckpgq/common.cpp | 25 ++++++++++--------- .../duckpgq/functions/scalar/csr_creation.cpp | 7 ++++++ 2 files changed, 20 insertions(+), 12 deletions(-) diff --git a/duckpgq/src/duckpgq/common.cpp b/duckpgq/src/duckpgq/common.cpp index 2e0a516f..e03e88aa 100644 --- a/duckpgq/src/duckpgq/common.cpp +++ b/duckpgq/src/duckpgq/common.cpp @@ -45,18 +45,19 @@ unique_ptr CSRFunctionData::CSREdgeBind(ClientContext &context, ScalarFunction &bound_function, vector> &arguments) { - if (!arguments[0]->IsFoldable()) { - throw InvalidInputException("Id must be constant."); - } - Value id = ExpressionExecutor::EvaluateScalar(context, *arguments[0]); - if (arguments.size() == 7) { - return make_uniq(context, id.GetValue(), - arguments[6]->return_type); - } else { - auto logical_type = LogicalType::SQLNULL; - return make_uniq(context, id.GetValue(), - logical_type); - } +// if (!arguments[0]->IsFoldable()) { +// throw InvalidInputException("Id must be constant."); +// } +// Value id = ExpressionExecutor::EvaluateScalar(context, *arguments[0]); +// if (arguments.size() == 7) { +// return make_uniq(context, id.GetValue(), +// arguments[6]->return_type); +// } else { +// auto logical_type = LogicalType::SQLNULL; +// return make_uniq(context, id.GetValue(), +// logical_type); +// } + return make_uniq(context, 0, LogicalType::SQLNULL); } unique_ptr diff --git a/duckpgq/src/duckpgq/functions/scalar/csr_creation.cpp b/duckpgq/src/duckpgq/functions/scalar/csr_creation.cpp index 14f954ef..0406fe45 100644 --- a/duckpgq/src/duckpgq/functions/scalar/csr_creation.cpp +++ b/duckpgq/src/duckpgq/functions/scalar/csr_creation.cpp @@ -208,6 +208,13 @@ CreateScalarFunctionInfo DuckPGQFunctions::GetCsrVertexFunction() { CreateScalarFunctionInfo DuckPGQFunctions::GetCsrEdgeFunction() { ScalarFunctionSet set("create_csr_edge"); + + set.AddFunction(ScalarFunction({LogicalType::BIGINT, LogicalType::BIGINT, + LogicalType::BIGINT, LogicalType::BIGINT, + LogicalType::BIGINT, LogicalType::BIGINT}, + LogicalType::INTEGER, CreateCsrEdgeFunction, + CSRFunctionData::CSREdgeBind)); + //! No edge weight set.AddFunction(ScalarFunction({LogicalType::INTEGER, LogicalType::BIGINT, LogicalType::BIGINT, LogicalType::BIGINT, From 4792314e8dedd99ba02185ec2a7836e68a7b88ff Mon Sep 17 00:00:00 2001 From: dtenwolde Date: Wed, 6 Mar 2024 16:52:01 +0100 Subject: [PATCH 049/249] Optimizer rule to find csr creation and insert pathfinding operator --- duckpgq/include/duckpgq_extension.hpp | 52 ++++++++++++++++++--------- 1 file changed, 36 insertions(+), 16 deletions(-) diff --git a/duckpgq/include/duckpgq_extension.hpp b/duckpgq/include/duckpgq_extension.hpp index df23eb26..3b533e8a 100644 --- a/duckpgq/include/duckpgq_extension.hpp +++ b/duckpgq/include/duckpgq_extension.hpp @@ -7,7 +7,13 @@ #include "duckdb/parser/query_node.hpp" #include "duckdb/parser/simplified_token.hpp" #include "duckdb/parser/sql_statement.hpp" +#include "duckdb/planner/logical_operator.hpp" #include "duckdb/planner/operator/logical_comparison_join.hpp" +#include "duckdb/planner/operator/logical_filter.hpp" +#include "duckdb/planner/operator/logical_aggregate.hpp" +#include "duckdb/planner/operator/logical_projection.hpp" +#include "duckdb/planner/operator/logical_limit.hpp" +#include "duckdb/planner/operator/logical_get.hpp" #include "duckpgq/common.hpp" #include "duckpgq/operators/logical_path_finding_operator.hpp" @@ -26,24 +32,38 @@ class DuckpgqOptimizerExtension : public OptimizerExtension { } static bool InsertPathFindingOperator(LogicalOperator &op) { -// if (op.type == LogicalOperatorType::LOGICAL_COMPARISON_JOIN) { -// auto &get = op.Cast(); -// for (auto &condition : get.conditions) { -// if (condition.comparison == ExpressionType::COMPARE_GREATERTHANOREQUALTO) { -// unique_ptr path_finding_operator = make_uniq(op.children); -// op.children.push_back(path_finding_operator); -// return true; -// } -// } -// } + vector> path_finding_children; + vector> path_finding_expressions; for (auto &child : op.children) { if (child->type == LogicalOperatorType::LOGICAL_COMPARISON_JOIN) { - auto &get = child->Cast(); - auto path_finding_operator = - make_uniq(get.children); - op.children.clear(); - op.children.push_back(std::move(path_finding_operator)); - return true; + auto &get_join = child->Cast(); + //! For now we assume this is enough to detect we have found a path-finding query + //! Should be improved in the future + if (get_join.children[0]->type == LogicalOperatorType::LOGICAL_FILTER && + get_join.children[1]->type == LogicalOperatorType::LOGICAL_AGGREGATE_AND_GROUP_BY) { + //! Structure is Aggregate --> Limit --> Projection(create_csr_edge(...)) + //! First check if we find the create_csr_edge function on the right side. + auto &get_aggregate = get_join.children[1]->Cast(); + auto &get_limit = get_aggregate.children[0]->Cast(); + auto &get_projection = get_limit.children[0]->Cast(); + auto &get_function_expression = get_projection.expressions[0]->Cast(); + if (get_function_expression.function.name != "create_csr_edge") { + continue; + } + auto &get_filter = get_join.children[0]->Cast(); + if (get_filter.children[0]->type == LogicalOperatorType::LOGICAL_GET) { + path_finding_children.push_back(std::move(get_filter.children[0])); + } + + path_finding_children.push_back(std::move(get_projection.children[0])); + path_finding_expressions = std::move(get_function_expression.children); + + auto path_finding_operator = + make_uniq(path_finding_children, path_finding_expressions); + op.children.clear(); + op.children.push_back(std::move(path_finding_operator)); + return true; + } } } for (auto &child : op.children) { From 6f737e843f45c1aaf5f4a2d6ab2f10bb45d23707 Mon Sep 17 00:00:00 2001 From: dtenwolde Date: Wed, 6 Mar 2024 16:52:21 +0100 Subject: [PATCH 050/249] Add ParamsToString method and add the expressions to logical operator --- .../operators/logical_path_finding_operator.hpp | 5 ++++- .../operators/logical_path_finding_operator.cpp | 10 ++++++++++ 2 files changed, 14 insertions(+), 1 deletion(-) diff --git a/duckpgq/include/duckpgq/operators/logical_path_finding_operator.hpp b/duckpgq/include/duckpgq/operators/logical_path_finding_operator.hpp index 96384c81..aad63865 100644 --- a/duckpgq/include/duckpgq/operators/logical_path_finding_operator.hpp +++ b/duckpgq/include/duckpgq/operators/logical_path_finding_operator.hpp @@ -5,7 +5,8 @@ namespace duckdb { class LogicalPathFindingOperator : public LogicalExtensionOperator { public: - explicit LogicalPathFindingOperator(vector> &children_) : LogicalExtensionOperator() { + explicit LogicalPathFindingOperator(vector> &children_, vector> &expressions_) + : LogicalExtensionOperator(std::move(expressions_)) { children = std::move(children_); } @@ -21,6 +22,8 @@ class LogicalPathFindingOperator : public LogicalExtensionOperator { std::string GetName() const override { return "PATH_FINDING"; } void ResolveTypes() override; + + string ParamsToString() const override; }; } // namespace duckdb diff --git a/duckpgq/src/duckpgq/operators/logical_path_finding_operator.cpp b/duckpgq/src/duckpgq/operators/logical_path_finding_operator.cpp index c65ae1e0..0ef3dabd 100644 --- a/duckpgq/src/duckpgq/operators/logical_path_finding_operator.cpp +++ b/duckpgq/src/duckpgq/operators/logical_path_finding_operator.cpp @@ -24,4 +24,14 @@ void LogicalPathFindingOperator::ResolveTypes() { auto right_types = children[1]->types; types.insert(types.end(), right_types.begin(), right_types.end()); } + +string LogicalPathFindingOperator::ParamsToString() const { + string extra_info; + for (auto &expr : expressions) { + extra_info += "\n"; + extra_info += expr->ToString(); + } + return extra_info; + +} } // namespace duckdb From ab1d6bca33e65b8acfedab3e98b5e60ce9798000 Mon Sep 17 00:00:00 2001 From: dtenwolde Date: Wed, 6 Mar 2024 16:54:01 +0100 Subject: [PATCH 051/249] Change order of execution children, add local and global csr to physical operator --- .../physical_path_finding_operator.hpp | 44 +++++++++--------- .../physical_path_finding_operator.cpp | 46 ++++++++++++++----- 2 files changed, 57 insertions(+), 33 deletions(-) diff --git a/duckpgq/include/duckpgq/operators/physical_path_finding_operator.hpp b/duckpgq/include/duckpgq/operators/physical_path_finding_operator.hpp index d5106d7f..5a330622 100644 --- a/duckpgq/include/duckpgq/operators/physical_path_finding_operator.hpp +++ b/duckpgq/include/duckpgq/operators/physical_path_finding_operator.hpp @@ -14,35 +14,15 @@ #include "duckdb/planner/operator/logical_extension_operator.hpp" namespace duckdb { - -struct GlobalCompressedSparseRow; - class PhysicalPathFinding : public PhysicalComparisonJoin { public: - class LocalCompressedSparseRow { - public: - LocalCompressedSparseRow(ClientContext &context, - const PhysicalPathFinding &op); - - void Sink(DataChunk &input, GlobalCompressedSparseRow &global_csr); - - - //! The hosting operator - const PhysicalPathFinding &op; - //! Holds a vector of incoming columns - DataChunk keys; - //! Local copy of the expression executor - ExpressionExecutor executor - - }; - class GlobalCompressedSparseRow { public: GlobalCompressedSparseRow(ClientContext &context, RowLayout &payload_layout){ }; - ~GlobalCompressedSparseRow() { delete[] v; } +// ~GlobalCompressedSparseRow() { delete[] v; } atomic *v; vector e; @@ -55,6 +35,24 @@ class PhysicalPathFinding : public PhysicalComparisonJoin { size_t vsize; }; + class LocalCompressedSparseRow { + public: + LocalCompressedSparseRow(ClientContext &context, + const PhysicalPathFinding &op); + + void Sink(DataChunk &input, GlobalCompressedSparseRow &global_csr); + + //! The hosting operator + const PhysicalPathFinding &op; + //! Holds a vector of incoming columns + DataChunk keys; + //! Local copy of the expression executor + ExpressionExecutor executor; + + }; + + + public: static constexpr const PhysicalOperatorType TYPE = PhysicalOperatorType::EXTENSION; @@ -63,6 +61,8 @@ class PhysicalPathFinding : public PhysicalComparisonJoin { PhysicalPathFinding(LogicalExtensionOperator &op, unique_ptr left, unique_ptr right); +public: + vector> expressions; public: // CachingOperator Interface @@ -102,6 +102,8 @@ class PhysicalPathFinding : public PhysicalComparisonJoin { bool ParallelSink() const override { return true; } void BuildPipelines(Pipeline ¤t, MetaPipeline &meta_pipeline) override; + + }; } // namespace duckdb diff --git a/duckpgq/src/duckpgq/operators/physical_path_finding_operator.cpp b/duckpgq/src/duckpgq/operators/physical_path_finding_operator.cpp index 35a3b155..ce866c77 100644 --- a/duckpgq/src/duckpgq/operators/physical_path_finding_operator.cpp +++ b/duckpgq/src/duckpgq/operators/physical_path_finding_operator.cpp @@ -8,6 +8,7 @@ #include "duckdb/parallel/thread_context.hpp" #include +#include namespace duckdb { PhysicalPathFinding::PhysicalPathFinding(LogicalExtensionOperator &op, @@ -16,12 +17,20 @@ PhysicalPathFinding::PhysicalPathFinding(LogicalExtensionOperator &op, : PhysicalComparisonJoin(op, TYPE, {}, JoinType::INNER, 0) { children.push_back(std::move(left)); children.push_back(std::move(right)); + expressions = std::move(op.expressions); } PhysicalPathFinding::LocalCompressedSparseRow::LocalCompressedSparseRow( duckdb::ClientContext &context, const duckdb::PhysicalPathFinding &op) : op(op), executor(context) {} +void PhysicalPathFinding::LocalCompressedSparseRow::Sink( + DataChunk &input, + PhysicalPathFinding::GlobalCompressedSparseRow &global_csr) { + input.Print(); + return; +} + //===--------------------------------------------------------------------===// // Sink //===--------------------------------------------------------------------===// @@ -30,26 +39,36 @@ class PathFindingLocalState : public LocalSinkState { using LocalCompressedSparseRow = PhysicalPathFinding::LocalCompressedSparseRow; PathFindingLocalState(ClientContext &context, const PhysicalPathFinding &op, const idx_t child) - : local_csr(context, op) {} + : local_csr(context, op) { + } + + //! local csr LocalCompressedSparseRow local_csr; }; class PathFindingGlobalState : public GlobalSinkState { - public: + using GlobalCompressedSparseRow = PhysicalPathFinding::GlobalCompressedSparseRow; PathFindingGlobalState(ClientContext &context, const PhysicalPathFinding &op) { + RowLayout lhs_layout; + lhs_layout.Initialize(op.children[0]->types); RowLayout rhs_layout; rhs_layout.Initialize(op.children[1]->types); - csr = make_uniq(context, rhs_layout); + global_csr = make_uniq(context, rhs_layout); } PathFindingGlobalState(PathFindingGlobalState &prev) - : GlobalSinkState(prev) {} + : GlobalSinkState(prev), global_csr(std::move(prev.global_csr)), child(prev.child+1) {} - void Sink(DataChunk &input, PathFindingLocalState &lstate) {} + void Sink(DataChunk &input, PathFindingLocalState &lstate) { - unique_ptr csr; + input.Print(); + lstate.local_csr.Sink(input, *global_csr); + } + + unique_ptr global_csr; + size_t child; }; @@ -63,8 +82,8 @@ unique_ptr PhysicalPathFinding::GetLocalSinkState(ExecutionContext &context) const { idx_t sink_child = 0; if (sink_state) { - // const auto &ie_sink = sink_state->Cast(); - // sink_child = ie_sink.child; + const auto &pathfinding_sink = sink_state->Cast(); + sink_child = pathfinding_sink.child; } return make_uniq(context.client, *this, sink_child); } @@ -74,7 +93,7 @@ SinkResultType PhysicalPathFinding::Sink(ExecutionContext &context, OperatorSinkInput &input) const { auto &gstate = input.global_state.Cast(); auto &lstate = input.local_state.Cast(); - + chunk.Print(); gstate.Sink(chunk, lstate); return SinkResultType::NEED_MORE_INPUT; @@ -111,6 +130,8 @@ PhysicalPathFinding::Finalize(Pipeline &pipeline, Event &event, OperatorResultType PhysicalPathFinding::ExecuteInternal( ExecutionContext &context, DataChunk &input, DataChunk &chunk, GlobalOperatorState &gstate, OperatorState &state) const { + input.Print(); + chunk.Print(); return OperatorResultType::FINISHED; } @@ -183,9 +204,10 @@ PhysicalPathFinding::GetData(ExecutionContext &context, DataChunk &result, auto &pf_sink = sink_state->Cast(); auto &pf_gstate = input.global_state.Cast(); auto &pf_lstate = input.local_state.Cast(); - + result.Print(); pf_gstate.Initialize(pf_sink); + return result.size() == 0 ? SourceResultType::FINISHED : SourceResultType::HAVE_MORE_OUTPUT; } @@ -210,11 +232,11 @@ void PhysicalPathFinding::BuildPipelines(Pipeline ¤t, // Build out LHS auto lhs_pipeline = child_meta_pipeline.GetBasePipeline(); - children[0]->BuildPipelines(*lhs_pipeline, child_meta_pipeline); + children[1]->BuildPipelines(*lhs_pipeline, child_meta_pipeline); // Build out RHS auto &rhs_pipeline = child_meta_pipeline.CreatePipeline(); - children[1]->BuildPipelines(rhs_pipeline, child_meta_pipeline); + children[0]->BuildPipelines(rhs_pipeline, child_meta_pipeline); // Despite having the same sink, RHS and everything created after it need // their own (same) PipelineFinishEvent From 24583253329bdf5f5f870e2ba13c52ebd02b5654 Mon Sep 17 00:00:00 2001 From: dtenwolde Date: Wed, 6 Mar 2024 16:54:24 +0100 Subject: [PATCH 052/249] Slightly change how csr is created, add more tuples to test --- .../path-finding/parallel_path_finding.test | 78 +++++++++++++------ 1 file changed, 56 insertions(+), 22 deletions(-) diff --git a/test/sql/path-finding/parallel_path_finding.test b/test/sql/path-finding/parallel_path_finding.test index 7ce5f2c5..cd885df0 100644 --- a/test/sql/path-finding/parallel_path_finding.test +++ b/test/sql/path-finding/parallel_path_finding.test @@ -8,13 +8,13 @@ require duckpgq statement ok -CREATE TABLE pairs(src INT, dst INT); INSERT INTO pairs(src, dst) VALUES (1, 2), (2, 3), (3,1), (3, 2), (6, 1), (4,2); +CREATE TABLE pairs(src INT, dst INT); INSERT INTO pairs(src, dst) VALUES (10, 20), (20, 30), (30,10); statement ok -create table student(id INT); INSERT INTO student(id) VALUES (1), (2), (3); +create table student(id INT); INSERT INTO student(id) VALUES (10), (20), (30), (40); statement ok -create table knows(src INT, dst INT); INSERT INTO knows(src, dst) VALUES (1,3), (1,1), (2,1), (2,3), (3,1), (NULL, NULL), (NULL, NULL); +create table knows(src INT, dst INT); INSERT INTO knows(src, dst) VALUES (40, 20), (10,30), (10,10), (20,10), (20,30), (30,10); #query II #SELECT * @@ -41,29 +41,63 @@ knows SOURCE KEY (src) REFERENCES student (id) statement ok SELECT * FROM pairs AS p -WHERE p.src BETWEEN (select csr_id from (SELECT - 0 as csr_id, - (SELECT count(a.id) FROM Student a), - CAST ( - (SELECT sum(CREATE_CSR_VERTEX(0, - (SELECT count(a.id) FROM Student a), - sub.dense_id, - sub.cnt) - ) - FROM ( - SELECT a.rowid as dense_id, count(k.src) as cnt - FROM Student a - LEFT JOIN Knows k ON k.src = a.id - GROUP BY a.rowid) sub - ) AS BIGINT), +WHERE p.src BETWEEN (SELECT CREATE_CSR_EDGE( + (SELECT count(a.id) as v_size FROM Student a), + (SELECT count(k.src) as e_size from knows k), a.rowid, c.rowid, - k.rowid FROM Knows k + k.rowid, + t.cnt) FROM Knows k JOIN student a on a.id = k.src - JOIN student c on c.id = k.dst)) AND p.dst; + JOIN student c on c.id = k.dst + JOIN (SELECT count(k.src) cnt, a.rowid as a_rowid + FROM student a + LEFT JOIN knows k ON k.src = a.id + GROUP BY a.rowid) t + ON t.a_rowid = a.rowid) AND p.dst; -statement ok -select 1; + +# CAST ( +# (SELECT sum(CREATE_CSR_VERTEX(0, +# (SELECT count(a.id) FROM Student a), +# sub.dense_id, +# sub.cnt) +# ) +# FROM ( +# SELECT a.rowid as dense_id, count(k.src) as cnt +# FROM Student a +# LEFT JOIN Knows k ON k.src = a.id +# GROUP BY a.rowid) sub +# ) AS BIGINT), + + +#statement ok +#SELECT * +#FROM pairs AS p +#WHERE (p.src, p.dst) BETWEEN ( +# SELECT (csr_id, 0) FROM (SELECT 0 as csr_id, +# (SELECT count(a.id) FROM Student a), +# 0, +# a.rowid, +# c.rowid, +# k.rowid FROM Knows k +# JOIN student a on a.id = k.src +# JOIN student c on c.id = k.dst)) AND (0, 3); + +#CAST ( +# (SELECT sum(CREATE_CSR_VERTEX(0, +# (SELECT count(a.id) FROM Student a), +# sub.dense_id, +# sub.cnt) +# ) +# FROM ( +# SELECT a.rowid as dense_id, count(k.src) as cnt +# FROM Student a +# LEFT JOIN Knows k ON k.src = a.id +# GROUP BY a.rowid) sub +# ) AS BIGINT), +#statement ok +#select 1; # LHS of IEJOIN # RHS of IEJoin # 0, From 11b0d10317e08a6741d5fe317867d6f7b515e61c Mon Sep 17 00:00:00 2001 From: dtenwolde Date: Wed, 6 Mar 2024 16:54:28 +0100 Subject: [PATCH 053/249] bump --- duckdb-pgq | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/duckdb-pgq b/duckdb-pgq index 21c3f0f3..e55aa6d8 160000 --- a/duckdb-pgq +++ b/duckdb-pgq @@ -1 +1 @@ -Subproject commit 21c3f0f30f8619e63378b7e8253c5b8e5fe4a65f +Subproject commit e55aa6d85d7f3162f17215ad77cc8e5c32f9c87e From 8326114834a642985ef9d0a4c4b4e8525c73f8ab Mon Sep 17 00:00:00 2001 From: dtenwolde Date: Thu, 7 Mar 2024 11:31:17 +0100 Subject: [PATCH 054/249] Add initializeVertex method --- .../physical_path_finding_operator.hpp | 8 +++-- .../physical_path_finding_operator.cpp | 31 ++++++++++++++++--- 2 files changed, 31 insertions(+), 8 deletions(-) diff --git a/duckpgq/include/duckpgq/operators/physical_path_finding_operator.hpp b/duckpgq/include/duckpgq/operators/physical_path_finding_operator.hpp index 5a330622..fec54015 100644 --- a/duckpgq/include/duckpgq/operators/physical_path_finding_operator.hpp +++ b/duckpgq/include/duckpgq/operators/physical_path_finding_operator.hpp @@ -32,7 +32,11 @@ class PhysicalPathFinding : public PhysicalComparisonJoin { bool initialized_v = false; bool initialized_e = false; bool initialized_w = false; - size_t vsize; + size_t v_size; + + std::mutex csr_lock; + public: + void InitializeVertex(int64_t v_size); }; class LocalCompressedSparseRow { @@ -51,8 +55,6 @@ class PhysicalPathFinding : public PhysicalComparisonJoin { }; - - public: static constexpr const PhysicalOperatorType TYPE = PhysicalOperatorType::EXTENSION; diff --git a/duckpgq/src/duckpgq/operators/physical_path_finding_operator.cpp b/duckpgq/src/duckpgq/operators/physical_path_finding_operator.cpp index ce866c77..0562d057 100644 --- a/duckpgq/src/duckpgq/operators/physical_path_finding_operator.cpp +++ b/duckpgq/src/duckpgq/operators/physical_path_finding_operator.cpp @@ -20,14 +20,40 @@ PhysicalPathFinding::PhysicalPathFinding(LogicalExtensionOperator &op, expressions = std::move(op.expressions); } +void PhysicalPathFinding::GlobalCompressedSparseRow::InitializeVertex(int64_t v_size_) { + lock_guard csr_init_lock(csr_lock); + + if (initialized_v) { + return; + } + v_size = v_size_ + 2; + try { + v = new std::atomic[v_size]; + } catch (std::bad_alloc const &) { + throw InternalException("Unable to initialize vector of size for csr vertex table " + "representation"); + } + for (idx_t i = 0; i < v_size; ++i) { + v[i].store(0); + } + initialized_v = true; +} + PhysicalPathFinding::LocalCompressedSparseRow::LocalCompressedSparseRow( duckdb::ClientContext &context, const duckdb::PhysicalPathFinding &op) : op(op), executor(context) {} + + void PhysicalPathFinding::LocalCompressedSparseRow::Sink( DataChunk &input, PhysicalPathFinding::GlobalCompressedSparseRow &global_csr) { input.Print(); + const auto e_size = ConstantVector::GetData(input.data[7]); + const auto v_size = ConstantVector::GetData(input.data[8]); + if (!global_csr.initialized_v) { + + } return; } @@ -62,8 +88,6 @@ class PathFindingGlobalState : public GlobalSinkState { : GlobalSinkState(prev), global_csr(std::move(prev.global_csr)), child(prev.child+1) {} void Sink(DataChunk &input, PathFindingLocalState &lstate) { - - input.Print(); lstate.local_csr.Sink(input, *global_csr); } @@ -93,7 +117,6 @@ SinkResultType PhysicalPathFinding::Sink(ExecutionContext &context, OperatorSinkInput &input) const { auto &gstate = input.global_state.Cast(); auto &lstate = input.local_state.Cast(); - chunk.Print(); gstate.Sink(chunk, lstate); return SinkResultType::NEED_MORE_INPUT; @@ -130,8 +153,6 @@ PhysicalPathFinding::Finalize(Pipeline &pipeline, Event &event, OperatorResultType PhysicalPathFinding::ExecuteInternal( ExecutionContext &context, DataChunk &input, DataChunk &chunk, GlobalOperatorState &gstate, OperatorState &state) const { - input.Print(); - chunk.Print(); return OperatorResultType::FINISHED; } From 149bab3672b73fa0c92e408eb0aa1bd680ca97e1 Mon Sep 17 00:00:00 2001 From: dtenwolde Date: Thu, 7 Mar 2024 11:45:22 +0100 Subject: [PATCH 055/249] Getting correct v_size now --- .../src/duckpgq/operators/physical_path_finding_operator.cpp | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/duckpgq/src/duckpgq/operators/physical_path_finding_operator.cpp b/duckpgq/src/duckpgq/operators/physical_path_finding_operator.cpp index 0562d057..d5c197b4 100644 --- a/duckpgq/src/duckpgq/operators/physical_path_finding_operator.cpp +++ b/duckpgq/src/duckpgq/operators/physical_path_finding_operator.cpp @@ -50,9 +50,9 @@ void PhysicalPathFinding::LocalCompressedSparseRow::Sink( PhysicalPathFinding::GlobalCompressedSparseRow &global_csr) { input.Print(); const auto e_size = ConstantVector::GetData(input.data[7]); - const auto v_size = ConstantVector::GetData(input.data[8]); + const auto v_size = input.data[8].GetValue(0).GetValue(); if (!global_csr.initialized_v) { - + global_csr.InitializeVertex(v_size); } return; } From e87dd81798eef6b390f5dece7d757515a006c999 Mon Sep 17 00:00:00 2001 From: dtenwolde Date: Thu, 7 Mar 2024 13:05:23 +0100 Subject: [PATCH 056/249] Adding print for debug --- .../physical_path_finding_operator.hpp | 29 +++++++++++++++++++ .../physical_path_finding_operator.cpp | 12 +++++++- 2 files changed, 40 insertions(+), 1 deletion(-) diff --git a/duckpgq/include/duckpgq/operators/physical_path_finding_operator.hpp b/duckpgq/include/duckpgq/operators/physical_path_finding_operator.hpp index fec54015..d39297d8 100644 --- a/duckpgq/include/duckpgq/operators/physical_path_finding_operator.hpp +++ b/duckpgq/include/duckpgq/operators/physical_path_finding_operator.hpp @@ -37,6 +37,35 @@ class PhysicalPathFinding : public PhysicalComparisonJoin { std::mutex csr_lock; public: void InitializeVertex(int64_t v_size); + void Print() { + string result; + result += "CSR:\nV: "; + if (initialized_v) { + for (idx_t i = 0; i < v_size; ++i) { + result += std::to_string(v[i]) + ' '; + } + } else { + result += "V not initialized"; + } + result += "\nE: "; + if (initialized_e) { + for (auto i : e) { + result += std::to_string(i) + " "; + } + } else { + result += "E not initialized"; + } + if (initialized_w) { + for (auto i : w) { + result += std::to_string(i) + " "; + } + } else { + result += "W not initialized"; + } + + Printer::Print(result); + + }; }; class LocalCompressedSparseRow { diff --git a/duckpgq/src/duckpgq/operators/physical_path_finding_operator.cpp b/duckpgq/src/duckpgq/operators/physical_path_finding_operator.cpp index d5c197b4..fe104c72 100644 --- a/duckpgq/src/duckpgq/operators/physical_path_finding_operator.cpp +++ b/duckpgq/src/duckpgq/operators/physical_path_finding_operator.cpp @@ -36,6 +36,7 @@ void PhysicalPathFinding::GlobalCompressedSparseRow::InitializeVertex(int64_t v_ for (idx_t i = 0; i < v_size; ++i) { v[i].store(0); } + Print(); initialized_v = true; } @@ -54,7 +55,16 @@ void PhysicalPathFinding::LocalCompressedSparseRow::Sink( if (!global_csr.initialized_v) { global_csr.InitializeVertex(v_size); } - return; + Vector result = Vector(LogicalTypeId::BIGINT); + BinaryExecutor::Execute( + input.data[6], input.data[5], result, input.size(), + [&](int64_t src, int64_t cnt) { + int64_t edge_count = 0; + global_csr.v[src + 2] = cnt; + edge_count = edge_count + cnt; + return edge_count; + }); + global_csr.Print(); } //===--------------------------------------------------------------------===// From 97a21c44c1f1079cbba81fac33f3683936432b3e Mon Sep 17 00:00:00 2001 From: dtenwolde Date: Thu, 7 Mar 2024 13:06:22 +0100 Subject: [PATCH 057/249] Add w --- .../include/duckpgq/operators/physical_path_finding_operator.hpp | 1 + 1 file changed, 1 insertion(+) diff --git a/duckpgq/include/duckpgq/operators/physical_path_finding_operator.hpp b/duckpgq/include/duckpgq/operators/physical_path_finding_operator.hpp index d39297d8..4c03139c 100644 --- a/duckpgq/include/duckpgq/operators/physical_path_finding_operator.hpp +++ b/duckpgq/include/duckpgq/operators/physical_path_finding_operator.hpp @@ -55,6 +55,7 @@ class PhysicalPathFinding : public PhysicalComparisonJoin { } else { result += "E not initialized"; } + result += "\nW: "; if (initialized_w) { for (auto i : w) { result += std::to_string(i) + " "; From 82fed5233f01cc1ef84f6bae996423bae01d65e9 Mon Sep 17 00:00:00 2001 From: dtenwolde Date: Thu, 7 Mar 2024 13:24:20 +0100 Subject: [PATCH 058/249] Initialze e array --- .../physical_path_finding_operator.hpp | 3 +-- .../physical_path_finding_operator.cpp | 24 ++++++++++++++++--- 2 files changed, 22 insertions(+), 5 deletions(-) diff --git a/duckpgq/include/duckpgq/operators/physical_path_finding_operator.hpp b/duckpgq/include/duckpgq/operators/physical_path_finding_operator.hpp index 4c03139c..d47205b1 100644 --- a/duckpgq/include/duckpgq/operators/physical_path_finding_operator.hpp +++ b/duckpgq/include/duckpgq/operators/physical_path_finding_operator.hpp @@ -37,6 +37,7 @@ class PhysicalPathFinding : public PhysicalComparisonJoin { std::mutex csr_lock; public: void InitializeVertex(int64_t v_size); + void InitializeEdge(int64_t e_size); void Print() { string result; result += "CSR:\nV: "; @@ -63,9 +64,7 @@ class PhysicalPathFinding : public PhysicalComparisonJoin { } else { result += "W not initialized"; } - Printer::Print(result); - }; }; diff --git a/duckpgq/src/duckpgq/operators/physical_path_finding_operator.cpp b/duckpgq/src/duckpgq/operators/physical_path_finding_operator.cpp index fe104c72..f4b75b96 100644 --- a/duckpgq/src/duckpgq/operators/physical_path_finding_operator.cpp +++ b/duckpgq/src/duckpgq/operators/physical_path_finding_operator.cpp @@ -36,9 +36,23 @@ void PhysicalPathFinding::GlobalCompressedSparseRow::InitializeVertex(int64_t v_ for (idx_t i = 0; i < v_size; ++i) { v[i].store(0); } - Print(); initialized_v = true; } +void PhysicalPathFinding::GlobalCompressedSparseRow::InitializeEdge( + int64_t e_size) { + const lock_guard csr_init_lock(csr_lock); + try { + e.resize(e_size, 0); + edge_ids.resize(e_size, 0); + } catch (std::bad_alloc const &) { + throw InternalException("Unable to initialize vector of size for csr " + "edge table representation"); + } + for (idx_t i = 1; i < v_size; i++) { + v[i] += v[i-1]; + } + initialized_e = true; +} PhysicalPathFinding::LocalCompressedSparseRow::LocalCompressedSparseRow( duckdb::ClientContext &context, const duckdb::PhysicalPathFinding &op) : @@ -50,9 +64,8 @@ void PhysicalPathFinding::LocalCompressedSparseRow::Sink( DataChunk &input, PhysicalPathFinding::GlobalCompressedSparseRow &global_csr) { input.Print(); - const auto e_size = ConstantVector::GetData(input.data[7]); - const auto v_size = input.data[8].GetValue(0).GetValue(); if (!global_csr.initialized_v) { + const auto v_size = input.data[8].GetValue(0).GetValue(); global_csr.InitializeVertex(v_size); } Vector result = Vector(LogicalTypeId::BIGINT); @@ -64,6 +77,11 @@ void PhysicalPathFinding::LocalCompressedSparseRow::Sink( edge_count = edge_count + cnt; return edge_count; }); + + if (!global_csr.initialized_e) { + const auto e_size = input.data[7].GetValue(0).GetValue(); + global_csr.InitializeEdge(e_size); + } global_csr.Print(); } From 9ea70a8d952653a7e5a75437f1d55570d9046524 Mon Sep 17 00:00:00 2001 From: dtenwolde Date: Thu, 7 Mar 2024 13:57:23 +0100 Subject: [PATCH 059/249] Create CSR --- .../physical_path_finding_operator.hpp | 2 ++ .../physical_path_finding_operator.cpp | 30 ++++++++++++++----- 2 files changed, 25 insertions(+), 7 deletions(-) diff --git a/duckpgq/include/duckpgq/operators/physical_path_finding_operator.hpp b/duckpgq/include/duckpgq/operators/physical_path_finding_operator.hpp index d47205b1..dc17561f 100644 --- a/duckpgq/include/duckpgq/operators/physical_path_finding_operator.hpp +++ b/duckpgq/include/duckpgq/operators/physical_path_finding_operator.hpp @@ -33,6 +33,7 @@ class PhysicalPathFinding : public PhysicalComparisonJoin { bool initialized_e = false; bool initialized_w = false; size_t v_size; + bool is_ready = false; std::mutex csr_lock; public: @@ -75,6 +76,7 @@ class PhysicalPathFinding : public PhysicalComparisonJoin { void Sink(DataChunk &input, GlobalCompressedSparseRow &global_csr); + static void CreateCSR(DataChunk &input, GlobalCompressedSparseRow &global_csr); //! The hosting operator const PhysicalPathFinding &op; //! Holds a vector of incoming columns diff --git a/duckpgq/src/duckpgq/operators/physical_path_finding_operator.cpp b/duckpgq/src/duckpgq/operators/physical_path_finding_operator.cpp index f4b75b96..5b339ff6 100644 --- a/duckpgq/src/duckpgq/operators/physical_path_finding_operator.cpp +++ b/duckpgq/src/duckpgq/operators/physical_path_finding_operator.cpp @@ -58,12 +58,8 @@ PhysicalPathFinding::LocalCompressedSparseRow::LocalCompressedSparseRow( duckdb::ClientContext &context, const duckdb::PhysicalPathFinding &op) : op(op), executor(context) {} - - -void PhysicalPathFinding::LocalCompressedSparseRow::Sink( - DataChunk &input, - PhysicalPathFinding::GlobalCompressedSparseRow &global_csr) { - input.Print(); +void PhysicalPathFinding::LocalCompressedSparseRow::CreateCSR(DataChunk &input, + PhysicalPathFinding::GlobalCompressedSparseRow &global_csr) { if (!global_csr.initialized_v) { const auto v_size = input.data[8].GetValue(0).GetValue(); global_csr.InitializeVertex(v_size); @@ -82,9 +78,29 @@ void PhysicalPathFinding::LocalCompressedSparseRow::Sink( const auto e_size = input.data[7].GetValue(0).GetValue(); global_csr.InitializeEdge(e_size); } + TernaryExecutor::Execute( + input.data[6], input.data[4], input.data[2], result, input.size(), + [&](int64_t src, int64_t dst, int64_t edge_id) { + auto pos = ++global_csr.v[src + 1]; + global_csr.e[(int64_t)pos - 1] = dst; + global_csr.edge_ids[(int64_t)pos - 1] = edge_id; + return 1; + }); global_csr.Print(); } +void PhysicalPathFinding::LocalCompressedSparseRow::Sink( + DataChunk &input, + PhysicalPathFinding::GlobalCompressedSparseRow &global_csr) { + if (global_csr.is_ready) { + // Go to path-finding --> CSR is ready + //! return for now + input.Print(); + return; + } + CreateCSR(input, global_csr); +} + //===--------------------------------------------------------------------===// // Sink //===--------------------------------------------------------------------===// @@ -146,7 +162,7 @@ SinkResultType PhysicalPathFinding::Sink(ExecutionContext &context, auto &gstate = input.global_state.Cast(); auto &lstate = input.local_state.Cast(); gstate.Sink(chunk, lstate); - + gstate.global_csr->is_ready = true; return SinkResultType::NEED_MORE_INPUT; } From 8f15a413eb923512f0ecd4b8b8c79f2d4f1036ab Mon Sep 17 00:00:00 2001 From: dtenwolde Date: Thu, 7 Mar 2024 13:59:37 +0100 Subject: [PATCH 060/249] Remove unused keys --- .../duckpgq/operators/physical_path_finding_operator.hpp | 2 -- 1 file changed, 2 deletions(-) diff --git a/duckpgq/include/duckpgq/operators/physical_path_finding_operator.hpp b/duckpgq/include/duckpgq/operators/physical_path_finding_operator.hpp index dc17561f..52e98fd5 100644 --- a/duckpgq/include/duckpgq/operators/physical_path_finding_operator.hpp +++ b/duckpgq/include/duckpgq/operators/physical_path_finding_operator.hpp @@ -79,8 +79,6 @@ class PhysicalPathFinding : public PhysicalComparisonJoin { static void CreateCSR(DataChunk &input, GlobalCompressedSparseRow &global_csr); //! The hosting operator const PhysicalPathFinding &op; - //! Holds a vector of incoming columns - DataChunk keys; //! Local copy of the expression executor ExpressionExecutor executor; From 08179bb914267c687e58b265b5bdd900f4eb8e35 Mon Sep 17 00:00:00 2001 From: dtenwolde Date: Thu, 7 Mar 2024 15:25:04 +0100 Subject: [PATCH 061/249] Add IterativeLength functions --- .../physical_path_finding_operator.hpp | 15 +- .../physical_path_finding_operator.cpp | 130 ++++++++++++++++-- 2 files changed, 132 insertions(+), 13 deletions(-) diff --git a/duckpgq/include/duckpgq/operators/physical_path_finding_operator.hpp b/duckpgq/include/duckpgq/operators/physical_path_finding_operator.hpp index 52e98fd5..65a9d2ab 100644 --- a/duckpgq/include/duckpgq/operators/physical_path_finding_operator.hpp +++ b/duckpgq/include/duckpgq/operators/physical_path_finding_operator.hpp @@ -14,7 +14,10 @@ #include "duckdb/planner/operator/logical_extension_operator.hpp" namespace duckdb { + class PhysicalPathFinding : public PhysicalComparisonJoin { +#define LANE_LIMIT 512 + public: class GlobalCompressedSparseRow { public: @@ -47,7 +50,7 @@ class PhysicalPathFinding : public PhysicalComparisonJoin { result += std::to_string(v[i]) + ' '; } } else { - result += "V not initialized"; + result += "not initialized"; } result += "\nE: "; if (initialized_e) { @@ -55,7 +58,7 @@ class PhysicalPathFinding : public PhysicalComparisonJoin { result += std::to_string(i) + " "; } } else { - result += "E not initialized"; + result += "not initialized"; } result += "\nW: "; if (initialized_w) { @@ -63,7 +66,7 @@ class PhysicalPathFinding : public PhysicalComparisonJoin { result += std::to_string(i) + " "; } } else { - result += "W not initialized"; + result += "not initialized"; } Printer::Print(result); }; @@ -77,6 +80,12 @@ class PhysicalPathFinding : public PhysicalComparisonJoin { void Sink(DataChunk &input, GlobalCompressedSparseRow &global_csr); static void CreateCSR(DataChunk &input, GlobalCompressedSparseRow &global_csr); + + bool IterativeLength(int64_t v_size, int64_t *v, vector &e, + vector> &seen, + vector> &visit, + vector> &next); + //! The hosting operator const PhysicalPathFinding &op; //! Local copy of the expression executor diff --git a/duckpgq/src/duckpgq/operators/physical_path_finding_operator.cpp b/duckpgq/src/duckpgq/operators/physical_path_finding_operator.cpp index 5b339ff6..0bf9612b 100644 --- a/duckpgq/src/duckpgq/operators/physical_path_finding_operator.cpp +++ b/duckpgq/src/duckpgq/operators/physical_path_finding_operator.cpp @@ -11,6 +11,7 @@ #include namespace duckdb { + PhysicalPathFinding::PhysicalPathFinding(LogicalExtensionOperator &op, unique_ptr left, unique_ptr right) @@ -89,13 +90,127 @@ void PhysicalPathFinding::LocalCompressedSparseRow::CreateCSR(DataChunk &input, global_csr.Print(); } +bool PhysicalPathFinding::LocalCompressedSparseRow::IterativeLength(int64_t v_size, int64_t *v, vector &e, + vector> &seen, + vector> &visit, + vector> &next) { + bool change = false; + for (auto i = 0; i < v_size; i++) { + next[i] = 0; + } + for (auto i = 0; i < v_size; i++) { + if (visit[i].any()) { + for (auto offset = v[i]; offset < v[i + 1]; offset++) { + auto n = e[offset]; + next[n] = next[n] | visit[i]; + } + } + } + for (auto i = 0; i < v_size; i++) { + next[i] = next[i] & ~seen[i]; + seen[i] = seen[i] | next[i]; + change |= next[i].any(); + } + return change; +} + + void PhysicalPathFinding::LocalCompressedSparseRow::Sink( DataChunk &input, PhysicalPathFinding::GlobalCompressedSparseRow &global_csr) { if (global_csr.is_ready) { // Go to path-finding --> CSR is ready - //! return for now input.Print(); + global_csr.Print(); + auto &src = input.data[0]; + auto &dst = input.data[1]; + + auto v_size = global_csr.v_size; + auto *v = (int64_t *)global_csr.v; + vector &e = global_csr.e; + + UnifiedVectorFormat vdata_src; + UnifiedVectorFormat vdata_dst; + src.ToUnifiedFormat(input.size(), vdata_src); + dst.ToUnifiedFormat(input.size(), vdata_dst); + auto src_data = (int64_t *)vdata_src.data; + auto dst_data = (int64_t *)vdata_dst.data; + Vector result = Vector(LogicalTypeId::BIGINT); + ValidityMask &result_validity = FlatVector::Validity(result); + result.SetVectorType(VectorType::FLAT_VECTOR); + auto result_data = FlatVector::GetData(result); + + vector> seen(v_size); + vector> visit1(v_size); + vector> visit2(v_size); + short lane_to_num[LANE_LIMIT]; + for (short & lane : lane_to_num) { + lane = -1; // inactive + } + idx_t started_searches = 0; + while (started_searches < input.size()) { + + // empty visit vectors + for (auto i = 0; i < v_size; i++) { + seen[i] = 0; + visit1[i] = 0; + } + + // add search jobs to free lanes + uint64_t active = 0; + for (int64_t lane = 0; lane < LANE_LIMIT; lane++) { + lane_to_num[lane] = -1; + while (started_searches < input.size()) { + int64_t search_num = started_searches++; + int64_t src_pos = vdata_src.sel->get_index(search_num); + int64_t dst_pos = vdata_dst.sel->get_index(search_num); + if (!vdata_src.validity.RowIsValid(src_pos)) { + result_validity.SetInvalid(search_num); + result_data[search_num] = (uint64_t)-1; /* no path */ + } else if (src_data[src_pos] == dst_data[dst_pos]) { + result_data[search_num] = + (uint64_t)0; // path of length 0 does not require a search + } else { + visit1[src_data[src_pos]][lane] = true; + lane_to_num[lane] = search_num; // active lane + active++; + break; + } + } + } + + // make passes while a lane is still active + for (int64_t iter = 1; active; iter++) { + if (!IterativeLength(v_size, v, e, seen, (iter & 1) ? visit1 : visit2, + (iter & 1) ? visit2 : visit1)) { + break; + } + // detect lanes that finished + for (int64_t lane = 0; lane < LANE_LIMIT; lane++) { + int64_t search_num = lane_to_num[lane]; + if (search_num >= 0) { // active lane + int64_t dst_pos = vdata_dst.sel->get_index(search_num); + if (seen[dst_data[dst_pos]][lane]) { + result_data[search_num] = + iter; /* found at iter => iter = path length */ + lane_to_num[lane] = -1; // mark inactive + active--; + } + } + } + } + + + // no changes anymore: any still active searches have no path + for (int64_t lane = 0; lane < LANE_LIMIT; lane++) { + int64_t search_num = lane_to_num[lane]; + if (search_num >= 0) { // active lane + result_validity.SetInvalid(search_num); + result_data[search_num] = (int64_t)-1; /* no path */ + lane_to_num[lane] = -1; // mark inactive + } + } + } return; } CreateCSR(input, global_csr); @@ -126,7 +241,7 @@ class PathFindingGlobalState : public GlobalSinkState { RowLayout rhs_layout; rhs_layout.Initialize(op.children[1]->types); global_csr = make_uniq(context, rhs_layout); - } + } PathFindingGlobalState(PathFindingGlobalState &prev) : GlobalSinkState(prev), global_csr(std::move(prev.global_csr)), child(prev.child+1) {} @@ -171,7 +286,6 @@ PhysicalPathFinding::Combine(ExecutionContext &context, OperatorSinkCombineInput &input) const { auto &gstate = input.global_state.Cast(); auto &lstate = input.local_state.Cast(); - // gstate.tables[gstate.child]->Combine(lstate.table); auto &client_profiler = QueryProfiler::Get(context.client); client_profiler.Flush(context.thread.profiler); @@ -208,17 +322,12 @@ class PathFindingLocalSourceState : public LocalSourceState { public: explicit PathFindingLocalSourceState(ClientContext &context, const PhysicalPathFinding &op) - : op(op), true_sel(STANDARD_VECTOR_SIZE), left_executor(context), right_executor(context){ + : op(op){ } const PhysicalPathFinding &op; - // Trailing predicates - SelectionVector true_sel; - - ExpressionExecutor left_executor; - ExpressionExecutor right_executor; - + DataChunk pf_results; }; class PathFindingGlobalSourceState : public GlobalSourceState { @@ -269,6 +378,7 @@ PhysicalPathFinding::GetData(ExecutionContext &context, DataChunk &result, auto &pf_sink = sink_state->Cast(); auto &pf_gstate = input.global_state.Cast(); auto &pf_lstate = input.local_state.Cast(); + result.Print(); pf_gstate.Initialize(pf_sink); From d41fc4d74531ffa69d10efe1a12b04d75cc93fe4 Mon Sep 17 00:00:00 2001 From: dtenwolde Date: Thu, 7 Mar 2024 15:49:48 +0100 Subject: [PATCH 062/249] Change tests to be rowid --- test/sql/path-finding/parallel_path_finding.test | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/test/sql/path-finding/parallel_path_finding.test b/test/sql/path-finding/parallel_path_finding.test index cd885df0..400e485b 100644 --- a/test/sql/path-finding/parallel_path_finding.test +++ b/test/sql/path-finding/parallel_path_finding.test @@ -8,7 +8,7 @@ require duckpgq statement ok -CREATE TABLE pairs(src INT, dst INT); INSERT INTO pairs(src, dst) VALUES (10, 20), (20, 30), (30,10); +CREATE TABLE pairs(src INT, dst INT); INSERT INTO pairs(src, dst) VALUES (0, 1), (1, 2), (2,0); statement ok create table student(id INT); INSERT INTO student(id) VALUES (10), (20), (30), (40); From fcc6a75d855979d9cb0234bd16f456d3ac7da4c0 Mon Sep 17 00:00:00 2001 From: dtenwolde Date: Thu, 7 Mar 2024 15:50:04 +0100 Subject: [PATCH 063/249] Fix incorrect index --- .../operators/physical_path_finding_operator.hpp | 2 +- .../operators/physical_path_finding_operator.cpp | 11 +++++------ 2 files changed, 6 insertions(+), 7 deletions(-) diff --git a/duckpgq/include/duckpgq/operators/physical_path_finding_operator.hpp b/duckpgq/include/duckpgq/operators/physical_path_finding_operator.hpp index 65a9d2ab..a4067c98 100644 --- a/duckpgq/include/duckpgq/operators/physical_path_finding_operator.hpp +++ b/duckpgq/include/duckpgq/operators/physical_path_finding_operator.hpp @@ -81,7 +81,7 @@ class PhysicalPathFinding : public PhysicalComparisonJoin { static void CreateCSR(DataChunk &input, GlobalCompressedSparseRow &global_csr); - bool IterativeLength(int64_t v_size, int64_t *v, vector &e, + static bool IterativeLength(int64_t v_size, int64_t *v, vector &e, vector> &seen, vector> &visit, vector> &next); diff --git a/duckpgq/src/duckpgq/operators/physical_path_finding_operator.cpp b/duckpgq/src/duckpgq/operators/physical_path_finding_operator.cpp index 0bf9612b..d72496bf 100644 --- a/duckpgq/src/duckpgq/operators/physical_path_finding_operator.cpp +++ b/duckpgq/src/duckpgq/operators/physical_path_finding_operator.cpp @@ -133,8 +133,8 @@ void PhysicalPathFinding::LocalCompressedSparseRow::Sink( UnifiedVectorFormat vdata_dst; src.ToUnifiedFormat(input.size(), vdata_src); dst.ToUnifiedFormat(input.size(), vdata_dst); - auto src_data = (int64_t *)vdata_src.data; - auto dst_data = (int64_t *)vdata_dst.data; + auto src_data = vdata_src.data; + auto dst_data = vdata_dst.data; Vector result = Vector(LogicalTypeId::BIGINT); ValidityMask &result_validity = FlatVector::Validity(result); result.SetVectorType(VectorType::FLAT_VECTOR); @@ -144,12 +144,11 @@ void PhysicalPathFinding::LocalCompressedSparseRow::Sink( vector> visit1(v_size); vector> visit2(v_size); short lane_to_num[LANE_LIMIT]; - for (short & lane : lane_to_num) { - lane = -1; // inactive + for (int64_t lane = 0; lane < LANE_LIMIT; lane++) { + lane_to_num[lane] = -1; // inactive } idx_t started_searches = 0; while (started_searches < input.size()) { - // empty visit vectors for (auto i = 0; i < v_size; i++) { seen[i] = 0; @@ -200,7 +199,6 @@ void PhysicalPathFinding::LocalCompressedSparseRow::Sink( } } - // no changes anymore: any still active searches have no path for (int64_t lane = 0; lane < LANE_LIMIT; lane++) { int64_t search_num = lane_to_num[lane]; @@ -211,6 +209,7 @@ void PhysicalPathFinding::LocalCompressedSparseRow::Sink( } } } + result.Print(); return; } CreateCSR(input, global_csr); From 6833842cf1d1a031e2f48cd61a1574211e8bb515 Mon Sep 17 00:00:00 2001 From: dtenwolde Date: Thu, 7 Mar 2024 16:09:19 +0100 Subject: [PATCH 064/249] Use correct types --- .../duckpgq/operators/physical_path_finding_operator.cpp | 7 ++++--- 1 file changed, 4 insertions(+), 3 deletions(-) diff --git a/duckpgq/src/duckpgq/operators/physical_path_finding_operator.cpp b/duckpgq/src/duckpgq/operators/physical_path_finding_operator.cpp index d72496bf..b5b3c955 100644 --- a/duckpgq/src/duckpgq/operators/physical_path_finding_operator.cpp +++ b/duckpgq/src/duckpgq/operators/physical_path_finding_operator.cpp @@ -125,6 +125,7 @@ void PhysicalPathFinding::LocalCompressedSparseRow::Sink( auto &src = input.data[0]; auto &dst = input.data[1]; + auto v_size = global_csr.v_size; auto *v = (int64_t *)global_csr.v; vector &e = global_csr.e; @@ -133,9 +134,9 @@ void PhysicalPathFinding::LocalCompressedSparseRow::Sink( UnifiedVectorFormat vdata_dst; src.ToUnifiedFormat(input.size(), vdata_src); dst.ToUnifiedFormat(input.size(), vdata_dst); - auto src_data = vdata_src.data; - auto dst_data = vdata_dst.data; - Vector result = Vector(LogicalTypeId::BIGINT); + auto src_data = (int32_t*)vdata_src.data; + auto dst_data = (int32_t*)vdata_dst.data; + Vector result = Vector(src.GetType()); ValidityMask &result_validity = FlatVector::Validity(result); result.SetVectorType(VectorType::FLAT_VECTOR); auto result_data = FlatVector::GetData(result); From 737db39feed0abfb8ddb0a4fc6c0bfe85c6caf62 Mon Sep 17 00:00:00 2001 From: dtenwolde Date: Tue, 12 Mar 2024 11:36:27 +0100 Subject: [PATCH 065/249] Change type of pf result, move lstate to gstate --- duckdb-pgq | 2 +- .../operators/physical_path_finding_operator.hpp | 3 +++ .../operators/physical_path_finding_operator.cpp | 16 ++++++++-------- 3 files changed, 12 insertions(+), 9 deletions(-) diff --git a/duckdb-pgq b/duckdb-pgq index e55aa6d8..8bd29f5a 160000 --- a/duckdb-pgq +++ b/duckdb-pgq @@ -1 +1 @@ -Subproject commit e55aa6d85d7f3162f17215ad77cc8e5c32f9c87e +Subproject commit 8bd29f5ac550cc57ded6073d6a9629133f137206 diff --git a/duckpgq/include/duckpgq/operators/physical_path_finding_operator.hpp b/duckpgq/include/duckpgq/operators/physical_path_finding_operator.hpp index a4067c98..734469c2 100644 --- a/duckpgq/include/duckpgq/operators/physical_path_finding_operator.hpp +++ b/duckpgq/include/duckpgq/operators/physical_path_finding_operator.hpp @@ -91,6 +91,9 @@ class PhysicalPathFinding : public PhysicalComparisonJoin { //! Local copy of the expression executor ExpressionExecutor executor; + //! Final result for the path-finding pairs + DataChunk local_results; + }; public: diff --git a/duckpgq/src/duckpgq/operators/physical_path_finding_operator.cpp b/duckpgq/src/duckpgq/operators/physical_path_finding_operator.cpp index b5b3c955..591407a0 100644 --- a/duckpgq/src/duckpgq/operators/physical_path_finding_operator.cpp +++ b/duckpgq/src/duckpgq/operators/physical_path_finding_operator.cpp @@ -139,7 +139,7 @@ void PhysicalPathFinding::LocalCompressedSparseRow::Sink( Vector result = Vector(src.GetType()); ValidityMask &result_validity = FlatVector::Validity(result); result.SetVectorType(VectorType::FLAT_VECTOR); - auto result_data = FlatVector::GetData(result); + auto result_data = FlatVector::GetData(result); vector> seen(v_size); vector> visit1(v_size); @@ -210,7 +210,9 @@ void PhysicalPathFinding::LocalCompressedSparseRow::Sink( } } } - result.Print(); + local_results.data.emplace_back(result); + local_results.SetCardinality(input); + local_results.Print(); return; } CreateCSR(input, global_csr); @@ -253,6 +255,8 @@ class PathFindingGlobalState : public GlobalSinkState { unique_ptr global_csr; size_t child; + DataChunk result; + }; unique_ptr @@ -288,6 +292,7 @@ PhysicalPathFinding::Combine(ExecutionContext &context, auto &lstate = input.local_state.Cast(); auto &client_profiler = QueryProfiler::Get(context.client); + gstate.result.Move(lstate.local_csr.local_results); client_profiler.Flush(context.thread.profiler); return SinkCombineResultType::FINISHED; @@ -326,8 +331,6 @@ class PathFindingLocalSourceState : public LocalSourceState { } const PhysicalPathFinding &op; - - DataChunk pf_results; }; class PathFindingGlobalSourceState : public GlobalSourceState { @@ -345,10 +348,8 @@ class PathFindingGlobalSourceState : public GlobalSourceState { public: idx_t MaxThreads() override { - // We can't leverage any more threads than block pairs. const auto &sink_state = (op.sink_state->Cast()); return 1; - } void GetNextPair(ClientContext &client, PathFindingGlobalState &gstate, @@ -378,11 +379,10 @@ PhysicalPathFinding::GetData(ExecutionContext &context, DataChunk &result, auto &pf_sink = sink_state->Cast(); auto &pf_gstate = input.global_state.Cast(); auto &pf_lstate = input.local_state.Cast(); - + result.Move(pf_sink.result); result.Print(); pf_gstate.Initialize(pf_sink); - return result.size() == 0 ? SourceResultType::FINISHED : SourceResultType::HAVE_MORE_OUTPUT; } From 54f4e42d6fe69884c0bbd39b44cc68dc4021aee5 Mon Sep 17 00:00:00 2001 From: dtenwolde Date: Wed, 20 Mar 2024 16:54:14 +0100 Subject: [PATCH 066/249] Changed to ColumnDataCollection --- .../physical_path_finding_operator.hpp | 2 - .../physical_path_finding_operator.cpp | 212 ++++-------------- 2 files changed, 48 insertions(+), 166 deletions(-) diff --git a/duckpgq/include/duckpgq/operators/physical_path_finding_operator.hpp b/duckpgq/include/duckpgq/operators/physical_path_finding_operator.hpp index 734469c2..c519096e 100644 --- a/duckpgq/include/duckpgq/operators/physical_path_finding_operator.hpp +++ b/duckpgq/include/duckpgq/operators/physical_path_finding_operator.hpp @@ -77,9 +77,7 @@ class PhysicalPathFinding : public PhysicalComparisonJoin { LocalCompressedSparseRow(ClientContext &context, const PhysicalPathFinding &op); - void Sink(DataChunk &input, GlobalCompressedSparseRow &global_csr); - static void CreateCSR(DataChunk &input, GlobalCompressedSparseRow &global_csr); static bool IterativeLength(int64_t v_size, int64_t *v, vector &e, vector> &seen, diff --git a/duckpgq/src/duckpgq/operators/physical_path_finding_operator.cpp b/duckpgq/src/duckpgq/operators/physical_path_finding_operator.cpp index 591407a0..51e4bf38 100644 --- a/duckpgq/src/duckpgq/operators/physical_path_finding_operator.cpp +++ b/duckpgq/src/duckpgq/operators/physical_path_finding_operator.cpp @@ -6,9 +6,9 @@ #include "duckdb/parallel/event.hpp" #include "duckdb/parallel/meta_pipeline.hpp" #include "duckdb/parallel/thread_context.hpp" +#include "duckdb/common/types/column/column_data_collection.hpp" #include -#include namespace duckdb { @@ -55,20 +55,45 @@ void PhysicalPathFinding::GlobalCompressedSparseRow::InitializeEdge( initialized_e = true; } -PhysicalPathFinding::LocalCompressedSparseRow::LocalCompressedSparseRow( - duckdb::ClientContext &context, const duckdb::PhysicalPathFinding &op) : - op(op), executor(context) {} +//===--------------------------------------------------------------------===// +// Sink +//===--------------------------------------------------------------------===// +class PathFindingLocalState : public LocalSinkState { +public: + using GlobalCompressedSparseRow = + PhysicalPathFinding::GlobalCompressedSparseRow; + PathFindingLocalState(ClientContext &context, const PhysicalPathFinding &op, + const idx_t child) : local_tasks(context, op.children[0]->GetTypes()) { + } + + void Sink(DataChunk &input, GlobalCompressedSparseRow &global_csr) { + if (global_csr.is_ready) { + // Add the tasks (src, dst) to sink + // Optimizations: Eliminate duplicate sources/destinations + input.Print(); + local_tasks.Append(input); + local_tasks.Print(); + return; + } + CreateCSR(input, global_csr); + } + + static void CreateCSR(DataChunk &input, + GlobalCompressedSparseRow &global_csr); -void PhysicalPathFinding::LocalCompressedSparseRow::CreateCSR(DataChunk &input, - PhysicalPathFinding::GlobalCompressedSparseRow &global_csr) { + ColumnDataCollection local_tasks; +}; + +void PathFindingLocalState::CreateCSR(DataChunk &input, + GlobalCompressedSparseRow &global_csr) { if (!global_csr.initialized_v) { const auto v_size = input.data[8].GetValue(0).GetValue(); global_csr.InitializeVertex(v_size); } - Vector result = Vector(LogicalTypeId::BIGINT); + auto result = Vector(LogicalTypeId::BIGINT); BinaryExecutor::Execute( input.data[6], input.data[5], result, input.size(), - [&](int64_t src, int64_t cnt) { + [&](const int64_t src, const int64_t cnt) { int64_t edge_count = 0; global_csr.v[src + 2] = cnt; edge_count = edge_count + cnt; @@ -82,162 +107,19 @@ void PhysicalPathFinding::LocalCompressedSparseRow::CreateCSR(DataChunk &input, TernaryExecutor::Execute( input.data[6], input.data[4], input.data[2], result, input.size(), [&](int64_t src, int64_t dst, int64_t edge_id) { - auto pos = ++global_csr.v[src + 1]; - global_csr.e[(int64_t)pos - 1] = dst; - global_csr.edge_ids[(int64_t)pos - 1] = edge_id; + const auto pos = ++global_csr.v[src + 1]; + global_csr.e[static_cast(pos) - 1] = dst; + global_csr.edge_ids[static_cast(pos) - 1] = edge_id; return 1; }); - global_csr.Print(); -} - -bool PhysicalPathFinding::LocalCompressedSparseRow::IterativeLength(int64_t v_size, int64_t *v, vector &e, - vector> &seen, - vector> &visit, - vector> &next) { - bool change = false; - for (auto i = 0; i < v_size; i++) { - next[i] = 0; - } - for (auto i = 0; i < v_size; i++) { - if (visit[i].any()) { - for (auto offset = v[i]; offset < v[i + 1]; offset++) { - auto n = e[offset]; - next[n] = next[n] | visit[i]; - } - } - } - for (auto i = 0; i < v_size; i++) { - next[i] = next[i] & ~seen[i]; - seen[i] = seen[i] | next[i]; - change |= next[i].any(); - } - return change; + global_csr.Print(); // Debug print } - -void PhysicalPathFinding::LocalCompressedSparseRow::Sink( - DataChunk &input, - PhysicalPathFinding::GlobalCompressedSparseRow &global_csr) { - if (global_csr.is_ready) { - // Go to path-finding --> CSR is ready - input.Print(); - global_csr.Print(); - auto &src = input.data[0]; - auto &dst = input.data[1]; - - - auto v_size = global_csr.v_size; - auto *v = (int64_t *)global_csr.v; - vector &e = global_csr.e; - - UnifiedVectorFormat vdata_src; - UnifiedVectorFormat vdata_dst; - src.ToUnifiedFormat(input.size(), vdata_src); - dst.ToUnifiedFormat(input.size(), vdata_dst); - auto src_data = (int32_t*)vdata_src.data; - auto dst_data = (int32_t*)vdata_dst.data; - Vector result = Vector(src.GetType()); - ValidityMask &result_validity = FlatVector::Validity(result); - result.SetVectorType(VectorType::FLAT_VECTOR); - auto result_data = FlatVector::GetData(result); - - vector> seen(v_size); - vector> visit1(v_size); - vector> visit2(v_size); - short lane_to_num[LANE_LIMIT]; - for (int64_t lane = 0; lane < LANE_LIMIT; lane++) { - lane_to_num[lane] = -1; // inactive - } - idx_t started_searches = 0; - while (started_searches < input.size()) { - // empty visit vectors - for (auto i = 0; i < v_size; i++) { - seen[i] = 0; - visit1[i] = 0; - } - - // add search jobs to free lanes - uint64_t active = 0; - for (int64_t lane = 0; lane < LANE_LIMIT; lane++) { - lane_to_num[lane] = -1; - while (started_searches < input.size()) { - int64_t search_num = started_searches++; - int64_t src_pos = vdata_src.sel->get_index(search_num); - int64_t dst_pos = vdata_dst.sel->get_index(search_num); - if (!vdata_src.validity.RowIsValid(src_pos)) { - result_validity.SetInvalid(search_num); - result_data[search_num] = (uint64_t)-1; /* no path */ - } else if (src_data[src_pos] == dst_data[dst_pos]) { - result_data[search_num] = - (uint64_t)0; // path of length 0 does not require a search - } else { - visit1[src_data[src_pos]][lane] = true; - lane_to_num[lane] = search_num; // active lane - active++; - break; - } - } - } - - // make passes while a lane is still active - for (int64_t iter = 1; active; iter++) { - if (!IterativeLength(v_size, v, e, seen, (iter & 1) ? visit1 : visit2, - (iter & 1) ? visit2 : visit1)) { - break; - } - // detect lanes that finished - for (int64_t lane = 0; lane < LANE_LIMIT; lane++) { - int64_t search_num = lane_to_num[lane]; - if (search_num >= 0) { // active lane - int64_t dst_pos = vdata_dst.sel->get_index(search_num); - if (seen[dst_data[dst_pos]][lane]) { - result_data[search_num] = - iter; /* found at iter => iter = path length */ - lane_to_num[lane] = -1; // mark inactive - active--; - } - } - } - } - - // no changes anymore: any still active searches have no path - for (int64_t lane = 0; lane < LANE_LIMIT; lane++) { - int64_t search_num = lane_to_num[lane]; - if (search_num >= 0) { // active lane - result_validity.SetInvalid(search_num); - result_data[search_num] = (int64_t)-1; /* no path */ - lane_to_num[lane] = -1; // mark inactive - } - } - } - local_results.data.emplace_back(result); - local_results.SetCardinality(input); - local_results.Print(); - return; - } - CreateCSR(input, global_csr); -} - -//===--------------------------------------------------------------------===// -// Sink -//===--------------------------------------------------------------------===// -class PathFindingLocalState : public LocalSinkState { -public: - using LocalCompressedSparseRow = PhysicalPathFinding::LocalCompressedSparseRow; - PathFindingLocalState(ClientContext &context, const PhysicalPathFinding &op, - const idx_t child) - : local_csr(context, op) { - } - - //! local csr - LocalCompressedSparseRow local_csr; -}; - class PathFindingGlobalState : public GlobalSinkState { public: using GlobalCompressedSparseRow = PhysicalPathFinding::GlobalCompressedSparseRow; PathFindingGlobalState(ClientContext &context, - const PhysicalPathFinding &op) { + const PhysicalPathFinding &op) : global_tasks(context, op.children[0]->GetTypes()) { RowLayout lhs_layout; lhs_layout.Initialize(op.children[0]->types); RowLayout rhs_layout; @@ -246,17 +128,19 @@ class PathFindingGlobalState : public GlobalSinkState { } PathFindingGlobalState(PathFindingGlobalState &prev) - : GlobalSinkState(prev), global_csr(std::move(prev.global_csr)), child(prev.child+1) {} + : GlobalSinkState(prev), global_tasks(prev.global_tasks), + global_csr(std::move(prev.global_csr)), child(prev.child + 1) {} - void Sink(DataChunk &input, PathFindingLocalState &lstate) { - lstate.local_csr.Sink(input, *global_csr); + void Sink(DataChunk &input, PathFindingLocalState &lstate) const { + lstate.Sink(input, *global_csr); } unique_ptr global_csr; size_t child; - DataChunk result; - + ColumnDataCollection global_tasks; + ColumnDataScanState scan_state; + ColumnDataAppendState append_state; }; unique_ptr @@ -292,9 +176,9 @@ PhysicalPathFinding::Combine(ExecutionContext &context, auto &lstate = input.local_state.Cast(); auto &client_profiler = QueryProfiler::Get(context.client); - gstate.result.Move(lstate.local_csr.local_results); + gstate.global_tasks.Combine(lstate.local_tasks); client_profiler.Flush(context.thread.profiler); - + gstate.global_tasks.Print(); return SinkCombineResultType::FINISHED; } @@ -379,7 +263,7 @@ PhysicalPathFinding::GetData(ExecutionContext &context, DataChunk &result, auto &pf_sink = sink_state->Cast(); auto &pf_gstate = input.global_state.Cast(); auto &pf_lstate = input.local_state.Cast(); - result.Move(pf_sink.result); + pf_sink.global_tasks.Scan(pf_sink.scan_state, result); result.Print(); pf_gstate.Initialize(pf_sink); From a50334c127342ddce4f4e16e8ec8e03b7a7168c6 Mon Sep 17 00:00:00 2001 From: dtenwolde Date: Wed, 20 Mar 2024 16:55:42 +0100 Subject: [PATCH 067/249] Changed to ColumnDataCollection --- duckdb-pgq | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/duckdb-pgq b/duckdb-pgq index 817b9c66..a69a7552 160000 --- a/duckdb-pgq +++ b/duckdb-pgq @@ -1 +1 @@ -Subproject commit 817b9c6681810827774038c25d63989c60772e6e +Subproject commit a69a7552378c5fc80083bb4a3aa9ffa0cfec06ee From df01f0e2db8d2e22aeafd90b54367096fd2bfc11 Mon Sep 17 00:00:00 2001 From: Pingan Ren Date: Tue, 2 Apr 2024 01:22:23 +0200 Subject: [PATCH 068/249] sequential algorithm runable --- .../logical_path_finding_operator.cpp | 1 + .../physical_path_finding_operator.cpp | 163 +++++++++++++++++- .../path-finding/parallel_path_finding.test | 5 +- 3 files changed, 160 insertions(+), 9 deletions(-) diff --git a/duckpgq/src/duckpgq/operators/logical_path_finding_operator.cpp b/duckpgq/src/duckpgq/operators/logical_path_finding_operator.cpp index 0ef3dabd..3202405f 100644 --- a/duckpgq/src/duckpgq/operators/logical_path_finding_operator.cpp +++ b/duckpgq/src/duckpgq/operators/logical_path_finding_operator.cpp @@ -23,6 +23,7 @@ void LogicalPathFindingOperator::ResolveTypes() { types = children[0]->types; auto right_types = children[1]->types; types.insert(types.end(), right_types.begin(), right_types.end()); + // types = {LogicalType::BIGINT, LogicalType::BIGINT}; } string LogicalPathFindingOperator::ParamsToString() const { diff --git a/duckpgq/src/duckpgq/operators/physical_path_finding_operator.cpp b/duckpgq/src/duckpgq/operators/physical_path_finding_operator.cpp index 51e4bf38..21a8200b 100644 --- a/duckpgq/src/duckpgq/operators/physical_path_finding_operator.cpp +++ b/duckpgq/src/duckpgq/operators/physical_path_finding_operator.cpp @@ -70,9 +70,9 @@ class PathFindingLocalState : public LocalSinkState { if (global_csr.is_ready) { // Add the tasks (src, dst) to sink // Optimizations: Eliminate duplicate sources/destinations - input.Print(); + // input.Print(); local_tasks.Append(input); - local_tasks.Print(); + // local_tasks.Print(); return; } CreateCSR(input, global_csr); @@ -112,7 +112,7 @@ void PathFindingLocalState::CreateCSR(DataChunk &input, global_csr.edge_ids[static_cast(pos) - 1] = edge_id; return 1; }); - global_csr.Print(); // Debug print + // global_csr.Print(); // Debug print } class PathFindingGlobalState : public GlobalSinkState { @@ -129,18 +129,22 @@ class PathFindingGlobalState : public GlobalSinkState { PathFindingGlobalState(PathFindingGlobalState &prev) : GlobalSinkState(prev), global_tasks(prev.global_tasks), + scan_state(prev.scan_state), append_state(prev.append_state), global_csr(std::move(prev.global_csr)), child(prev.child + 1) {} void Sink(DataChunk &input, PathFindingLocalState &lstate) const { lstate.Sink(input, *global_csr); } - unique_ptr global_csr; - size_t child; - + // pairs is a 2-column table with src and dst ColumnDataCollection global_tasks; + // pairs with path exists + // ColumnDataCollection global_results; ColumnDataScanState scan_state; ColumnDataAppendState append_state; + + unique_ptr global_csr; + size_t child; }; unique_ptr @@ -178,18 +182,163 @@ PhysicalPathFinding::Combine(ExecutionContext &context, gstate.global_tasks.Combine(lstate.local_tasks); client_profiler.Flush(context.thread.profiler); - gstate.global_tasks.Print(); + // gstate.global_tasks.Print(); return SinkCombineResultType::FINISHED; } //===--------------------------------------------------------------------===// // Finalize //===--------------------------------------------------------------------===// + +static bool IterativeLength(int64_t v_size, int64_t *v, vector &e, + vector> &seen, + vector> &visit, + vector> &next) { + bool change = false; + for (auto i = 0; i < v_size; i++) { + next[i] = 0; + } + for (auto i = 0; i < v_size; i++) { + if (visit[i].any()) { + for (auto offset = v[i]; offset < v[i + 1]; offset++) { + auto n = e[offset]; + next[n] = next[n] | visit[i]; + } + } + } + for (auto i = 0; i < v_size; i++) { + next[i] = next[i] & ~seen[i]; + seen[i] = seen[i] | next[i]; + change |= next[i].any(); + } + return change; +} + +static void IterativeLengthFunction(const unique_ptr &csr, + DataChunk &pairs, Vector &result) { + int64_t v_size = csr->v_size; + int64_t *v = (int64_t *)csr->v; + vector &e = csr->e; + + // get src and dst vectors for searches + auto &src = pairs.data[0]; + auto &dst = pairs.data[1]; + UnifiedVectorFormat vdata_src; + UnifiedVectorFormat vdata_dst; + src.ToUnifiedFormat(pairs.size(), vdata_src); + dst.ToUnifiedFormat(pairs.size(), vdata_dst); + + auto src_data = FlatVector::GetData(src); + auto dst_data = FlatVector::GetData(dst); + + ValidityMask &result_validity = FlatVector::Validity(result); + + // create result vector + result.SetVectorType(VectorType::FLAT_VECTOR); + auto result_data = FlatVector::GetData(result); + + // create temp SIMD arrays + vector> seen(v_size); + vector> visit1(v_size); + vector> visit2(v_size); + + // maps lane to search number + short lane_to_num[LANE_LIMIT]; + for (int64_t lane = 0; lane < LANE_LIMIT; lane++) { + lane_to_num[lane] = -1; // inactive + } + + idx_t started_searches = 0; + while (started_searches < pairs.size()) { + + // empty visit vectors + for (auto i = 0; i < v_size; i++) { + seen[i] = 0; + visit1[i] = 0; + } + + // add search jobs to free lanes + uint64_t active = 0; + for (int64_t lane = 0; lane < LANE_LIMIT; lane++) { + lane_to_num[lane] = -1; + while (started_searches < pairs.size()) { + int64_t search_num = started_searches++; + int64_t src_pos = vdata_src.sel->get_index(search_num); + int64_t dst_pos = vdata_dst.sel->get_index(search_num); + if (!vdata_src.validity.RowIsValid(src_pos)) { + result_validity.SetInvalid(search_num); + result_data[search_num] = (uint64_t)-1; /* no path */ + } else if (src_data[src_pos] == dst_data[dst_pos]) { + result_data[search_num] = + (uint64_t)0; // path of length 0 does not require a search + } else { + visit1[src_data[src_pos]][lane] = true; + lane_to_num[lane] = search_num; // active lane + active++; + break; + } + } + } + + // make passes while a lane is still active + for (int64_t iter = 1; active; iter++) { + if (!IterativeLength(v_size, v, e, seen, (iter & 1) ? visit1 : visit2, + (iter & 1) ? visit2 : visit1)) { + break; + } + // detect lanes that finished + for (int64_t lane = 0; lane < LANE_LIMIT; lane++) { + int64_t search_num = lane_to_num[lane]; + if (search_num >= 0) { // active lane + int64_t dst_pos = vdata_dst.sel->get_index(search_num); + if (seen[dst_data[dst_pos]][lane]) { + result_data[search_num] = + iter; /* found at iter => iter = path length */ + lane_to_num[lane] = -1; // mark inactive + active--; + } + } + } + } + + // no changes anymore: any still active searches have no path + for (int64_t lane = 0; lane < LANE_LIMIT; lane++) { + int64_t search_num = lane_to_num[lane]; + if (search_num >= 0) { // active lane + result_validity.SetInvalid(search_num); + result_data[search_num] = (int64_t)-1; /* no path */ + lane_to_num[lane] = -1; // mark inactive + } + } + } +} + + SinkFinalizeType PhysicalPathFinding::Finalize(Pipeline &pipeline, Event &event, ClientContext &context, OperatorSinkFinalizeInput &input) const { auto &gstate = input.global_state.Cast(); + auto &csr = gstate.global_csr; + auto &global_tasks = gstate.global_tasks; + if (global_tasks.Count() != 0) { + DataChunk pairs; + global_tasks.InitializeScanChunk(pairs); + ColumnDataScanState scan_state; + global_tasks.InitializeScan(scan_state); + while (global_tasks.Scan(scan_state, pairs)) { + Vector result(LogicalType::BIGINT, true, true); + IterativeLengthFunction(csr, pairs, result); + // store the result + // gstate.global_results.InitializeAppend(gstate.append_state); + // gstate.global_results.Append(gstate.append_state, pairs); + // // debug print + // gstate.global_results.Print(); + } + } + + // Move to the next input child + ++gstate.child; return SinkFinalizeType::READY; } diff --git a/test/sql/path-finding/parallel_path_finding.test b/test/sql/path-finding/parallel_path_finding.test index 400e485b..378d7bab 100644 --- a/test/sql/path-finding/parallel_path_finding.test +++ b/test/sql/path-finding/parallel_path_finding.test @@ -8,7 +8,7 @@ require duckpgq statement ok -CREATE TABLE pairs(src INT, dst INT); INSERT INTO pairs(src, dst) VALUES (0, 1), (1, 2), (2,0); +CREATE TABLE pairs(src BIGINT, dst BIGINT); INSERT INTO pairs(src, dst) VALUES (0, 1), (1, 2), (2,0); statement ok create table student(id INT); INSERT INTO student(id) VALUES (10), (20), (30), (40); @@ -38,7 +38,7 @@ knows SOURCE KEY (src) REFERENCES student (id) # COLUMNS (*) # ); -statement ok +query II SELECT * FROM pairs AS p WHERE p.src BETWEEN (SELECT CREATE_CSR_EDGE( @@ -55,6 +55,7 @@ WHERE p.src BETWEEN (SELECT CREATE_CSR_EDGE( LEFT JOIN knows k ON k.src = a.id GROUP BY a.rowid) t ON t.a_rowid = a.rowid) AND p.dst; +---- # CAST ( From e31b83376e7b6ceb6f9216f56462dfca7d910219 Mon Sep 17 00:00:00 2001 From: Pingan Ren Date: Tue, 2 Apr 2024 16:13:48 +0200 Subject: [PATCH 069/249] clean code --- .../operators/physical_path_finding_operator.cpp | 12 ++++-------- 1 file changed, 4 insertions(+), 8 deletions(-) diff --git a/duckpgq/src/duckpgq/operators/physical_path_finding_operator.cpp b/duckpgq/src/duckpgq/operators/physical_path_finding_operator.cpp index 21a8200b..74a79b58 100644 --- a/duckpgq/src/duckpgq/operators/physical_path_finding_operator.cpp +++ b/duckpgq/src/duckpgq/operators/physical_path_finding_operator.cpp @@ -129,7 +129,6 @@ class PathFindingGlobalState : public GlobalSinkState { PathFindingGlobalState(PathFindingGlobalState &prev) : GlobalSinkState(prev), global_tasks(prev.global_tasks), - scan_state(prev.scan_state), append_state(prev.append_state), global_csr(std::move(prev.global_csr)), child(prev.child + 1) {} void Sink(DataChunk &input, PathFindingLocalState &lstate) const { @@ -329,11 +328,8 @@ PhysicalPathFinding::Finalize(Pipeline &pipeline, Event &event, while (global_tasks.Scan(scan_state, pairs)) { Vector result(LogicalType::BIGINT, true, true); IterativeLengthFunction(csr, pairs, result); - // store the result - // gstate.global_results.InitializeAppend(gstate.append_state); - // gstate.global_results.Append(gstate.append_state, pairs); - // // debug print - // gstate.global_results.Print(); + // debug print + Printer::Print(result.ToString(pairs.size())); } } @@ -381,7 +377,7 @@ class PathFindingGlobalSourceState : public GlobalSourceState { public: idx_t MaxThreads() override { - const auto &sink_state = (op.sink_state->Cast()); + // const auto &sink_state = (op.sink_state->Cast()); return 1; } @@ -411,7 +407,7 @@ PhysicalPathFinding::GetData(ExecutionContext &context, DataChunk &result, OperatorSourceInput &input) const { auto &pf_sink = sink_state->Cast(); auto &pf_gstate = input.global_state.Cast(); - auto &pf_lstate = input.local_state.Cast(); + // auto &pf_lstate = input.local_state.Cast(); pf_sink.global_tasks.Scan(pf_sink.scan_state, result); result.Print(); pf_gstate.Initialize(pf_sink); From 8642326719ff71189473548041183ca2d53d464b Mon Sep 17 00:00:00 2001 From: Pingan Ren Date: Thu, 4 Apr 2024 23:22:00 +0200 Subject: [PATCH 070/249] make bfs support parallel --- .../physical_path_finding_operator.hpp | 3 +- .../physical_path_finding_operator.cpp | 226 +++++++++++++++++- 2 files changed, 223 insertions(+), 6 deletions(-) diff --git a/duckpgq/include/duckpgq/operators/physical_path_finding_operator.hpp b/duckpgq/include/duckpgq/operators/physical_path_finding_operator.hpp index c519096e..e5dead10 100644 --- a/duckpgq/include/duckpgq/operators/physical_path_finding_operator.hpp +++ b/duckpgq/include/duckpgq/operators/physical_path_finding_operator.hpp @@ -144,7 +144,8 @@ class PhysicalPathFinding : public PhysicalComparisonJoin { void BuildPipelines(Pipeline ¤t, MetaPipeline &meta_pipeline) override; - + // //! Schedules tasks to calculate the next iteration of the path-finding + static void ScheduleBFSTasks(Pipeline &pipeline, Event &event, GlobalSinkState &state); }; } // namespace duckdb diff --git a/duckpgq/src/duckpgq/operators/physical_path_finding_operator.cpp b/duckpgq/src/duckpgq/operators/physical_path_finding_operator.cpp index 74a79b58..49f1ae8b 100644 --- a/duckpgq/src/duckpgq/operators/physical_path_finding_operator.cpp +++ b/duckpgq/src/duckpgq/operators/physical_path_finding_operator.cpp @@ -6,6 +6,7 @@ #include "duckdb/parallel/event.hpp" #include "duckdb/parallel/meta_pipeline.hpp" #include "duckdb/parallel/thread_context.hpp" +#include "duckdb/parallel/base_pipeline_event.hpp" #include "duckdb/common/types/column/column_data_collection.hpp" #include @@ -115,6 +116,49 @@ void PathFindingLocalState::CreateCSR(DataChunk &input, // global_csr.Print(); // Debug print } +class GlobalBFSState { +public: + GlobalBFSState() = default; + + GlobalBFSState(int64_t v_size_, idx_t pairs_size_, int64_t *src_, int64_t *dst_, + UnifiedVectorFormat &vdata_src_, UnifiedVectorFormat &vdata_dst_) + : iter(1), v_size(v_size_), src(src_), dst(dst_), vdata_src(std::move(vdata_src_)), + vdata_dst(std::move(vdata_dst_)), started_searches(0), + seen(v_size_), visit1(v_size_), visit2(v_size_), + change(false), result(LogicalTypeId::BIGINT, true, true, pairs_size_) { + for (auto i = 0; i < LANE_LIMIT; i++) { + lane_to_num[i] = -1; + } + } + + void clear() { + iter = 1; + change = false; + for (auto i = 0; i < LANE_LIMIT; i++) { + lane_to_num[i] = -1; + } + // empty visit vectors + for (auto i = 0; i < v_size; i++) { + seen[i] = 0; + visit1[i] = 0; + } + } +public: + int64_t iter; + int64_t v_size; + int64_t *src; + int64_t *dst; + UnifiedVectorFormat vdata_src; + UnifiedVectorFormat vdata_dst; + idx_t started_searches; + idx_t lane_to_num[LANE_LIMIT]; + vector> seen; + vector> visit1; + vector> visit2; + bool change; + Vector result; +}; + class PathFindingGlobalState : public GlobalSinkState { public: using GlobalCompressedSparseRow = PhysicalPathFinding::GlobalCompressedSparseRow; @@ -129,7 +173,9 @@ class PathFindingGlobalState : public GlobalSinkState { PathFindingGlobalState(PathFindingGlobalState &prev) : GlobalSinkState(prev), global_tasks(prev.global_tasks), - global_csr(std::move(prev.global_csr)), child(prev.child + 1) {} + global_csr(std::move(prev.global_csr)), child(prev.child + 1) { + + } void Sink(DataChunk &input, PathFindingLocalState &lstate) const { lstate.Sink(input, *global_csr); @@ -144,6 +190,9 @@ class PathFindingGlobalState : public GlobalSinkState { unique_ptr global_csr; size_t child; + + // state for BFS + unique_ptr global_bfs_state; }; unique_ptr @@ -312,6 +361,114 @@ static void IterativeLengthFunction(const unique_ptr event_p, ClientContext &context, PathFindingGlobalState &state, idx_t start, idx_t end) + : ExecutorTask(context, std::move(event_p)), context(context), state(state), start(start), end(end) { + } + + TaskExecutionResult ExecuteTask(TaskExecutionMode mode) override { + auto& change = state.global_bfs_state->change; + auto& v_size = state.global_bfs_state->v_size; + auto& seen = state.global_bfs_state->seen; + auto& visit = state.global_bfs_state->iter & 1 ? state.global_bfs_state->visit1 : state.global_bfs_state->visit2; + auto& next = state.global_bfs_state->iter & 1 ? state.global_bfs_state->visit2 : state.global_bfs_state->visit1; + int64_t *v = (int64_t *)state.global_csr->v; + vector &e = state.global_csr->e; + + for (auto i = 0; i < v_size; i++) { + next[i] = 0; + } + for (auto i = 0; i < v_size; i++) { + if (visit[i].any()) { + for (auto offset = v[i]; offset < v[i + 1]; offset++) { + auto n = e[offset]; + next[n] = next[n] | visit[i]; + } + } + } + for (auto i = 0; i < v_size; i++) { + next[i] = next[i] & ~seen[i]; + seen[i] = seen[i] | next[i]; + change |= next[i].any(); + } + + event->FinishTask(); + return TaskExecutionResult::TASK_FINISHED; + } + +private: + ClientContext &context; + PathFindingGlobalState &state; + idx_t start; + idx_t end; +}; + +class BFSIterativeEvent : public BasePipelineEvent { +public: + BFSIterativeEvent(PathFindingGlobalState &gstate_p, Pipeline &pipeline_p) + : BasePipelineEvent(pipeline_p), gstate(gstate_p) { + } + + PathFindingGlobalState &gstate; + +public: + void Schedule() override { + auto &context = pipeline->GetClientContext(); + + // Schedule tasks equal to the number of threads, which will each merge multiple partitions + auto &ts = TaskScheduler::GetScheduler(context); + idx_t num_threads = ts.NumberOfThreads(); + + vector> bfs_tasks; + bfs_tasks.push_back(make_uniq(shared_from_this(), context, gstate, 0, 0)); + // for (idx_t tnum = 0; tnum < num_threads; tnum++) { + // bfs_tasks.push_back(make_uniq(shared_from_this(), context, gstate)); + // } + SetTasks(std::move(bfs_tasks)); + } + + void FinishEvent() override { + auto& bfs_state = gstate.global_bfs_state; + + auto result_data = FlatVector::GetData(bfs_state->result); + ValidityMask &result_validity = FlatVector::Validity(bfs_state->result); + + if (bfs_state->change) { + // detect lanes that finished + for (int64_t lane = 0; lane < LANE_LIMIT; lane++) { + int64_t search_num = bfs_state->lane_to_num[lane]; + if (search_num >= 0) { // active lane + int64_t dst_pos = bfs_state->vdata_dst.sel->get_index(search_num); + if (bfs_state->seen[bfs_state->dst[dst_pos]][lane]) { + result_data[search_num] = + bfs_state->iter; /* found at iter => iter = path length */ + bfs_state->lane_to_num[lane] = -1; // mark inactive + } + } + } + // into the next iteration + auto bfs_event = std::make_shared(gstate, *pipeline); + this->InsertEvent(std::dynamic_pointer_cast(bfs_event)); + } else { + // no changes anymore: any still active searches have no path + for (int64_t lane = 0; lane < LANE_LIMIT; lane++) { + int64_t search_num = bfs_state->lane_to_num[lane]; + if (search_num >= 0) { // active lane + result_validity.SetInvalid(search_num); + result_data[search_num] = (int64_t)-1; /* no path */ + bfs_state->lane_to_num[lane] = -1; // mark inactive + } + } + + // if remaining pairs, schedule the BFS for the next batch + if (bfs_state->started_searches < gstate.global_tasks.Count()) { + PhysicalPathFinding::ScheduleBFSTasks(*pipeline, *this, gstate); + } + } + } +}; + SinkFinalizeType PhysicalPathFinding::Finalize(Pipeline &pipeline, Event &event, @@ -321,24 +478,83 @@ PhysicalPathFinding::Finalize(Pipeline &pipeline, Event &event, auto &csr = gstate.global_csr; auto &global_tasks = gstate.global_tasks; if (global_tasks.Count() != 0) { + DataChunk all_pairs; DataChunk pairs; global_tasks.InitializeScanChunk(pairs); ColumnDataScanState scan_state; global_tasks.InitializeScan(scan_state); while (global_tasks.Scan(scan_state, pairs)) { - Vector result(LogicalType::BIGINT, true, true); - IterativeLengthFunction(csr, pairs, result); - // debug print - Printer::Print(result.ToString(pairs.size())); + all_pairs.Append(pairs, true); + } + // debug print + all_pairs.Print(); + + auto &src = all_pairs.data[0]; + auto &dst = all_pairs.data[1]; + UnifiedVectorFormat vdata_src; + UnifiedVectorFormat vdata_dst; + src.ToUnifiedFormat(all_pairs.size(), vdata_src); + dst.ToUnifiedFormat(all_pairs.size(), vdata_dst); + auto src_data = FlatVector::GetData(src); + auto dst_data = FlatVector::GetData(dst); + + gstate.global_bfs_state = make_uniq(csr->v_size, + global_tasks.Count(), src_data, dst_data, vdata_src, vdata_dst); + + // Schedule the first round of BFS tasks + if (all_pairs.size() > 0) { + ScheduleBFSTasks(pipeline, event, gstate); } } + // debug print + gstate.global_bfs_state->result.Print(global_tasks.Count()); + // Move to the next input child ++gstate.child; return SinkFinalizeType::READY; } +void ScheduleBFSTasks(Pipeline &pipeline, Event &event, GlobalSinkState &state) { + auto &gstate = state.Cast(); + auto &bfs_state = gstate.global_bfs_state; + + // for every batch of pairs, schedule a BFS task + bfs_state->clear(); + + // remaining pairs + if (bfs_state->started_searches < gstate.global_tasks.Count()) { + + auto result_data = FlatVector::GetData(bfs_state->result); + auto& result_validity = FlatVector::Validity(bfs_state->result); + + for (int64_t lane = 0; lane < LANE_LIMIT; lane++) { + bfs_state->lane_to_num[lane] = -1; + while (bfs_state->started_searches < gstate.global_tasks.Count()) { + int64_t search_num = bfs_state->started_searches++; + int64_t src_pos = bfs_state->vdata_src.sel->get_index(search_num); + int64_t dst_pos = bfs_state->vdata_src.sel->get_index(search_num); + if (!bfs_state->vdata_src.validity.RowIsValid(src_pos)) { + result_validity.SetInvalid(search_num); + result_data[search_num] = (uint64_t)-1; /* no path */ + } else if (bfs_state->src[src_pos] == bfs_state->dst[dst_pos]) { + result_data[search_num] = + (uint64_t)0; // path of length 0 does not require a search + } else { + bfs_state->visit1[bfs_state->src[src_pos]][lane] = true; + bfs_state->lane_to_num[lane] = search_num; // active lane + break; + } + } + } + + auto bfs_event = make_shared(gstate, pipeline); + event.InsertEvent(std::move(std::dynamic_pointer_cast(bfs_event))); + } +} + + //===--------------------------------------------------------------------===// // Operator //===--------------------------------------------------------------------===// From 8720721dea79c852a2da81aa2d3fe9bfac74232d Mon Sep 17 00:00:00 2001 From: Pingan Ren Date: Thu, 4 Apr 2024 23:43:19 +0200 Subject: [PATCH 071/249] update .gitignore --- .gitignore | 1 + 1 file changed, 1 insertion(+) diff --git a/.gitignore b/.gitignore index b9f264b9..0cdf9316 100644 --- a/.gitignore +++ b/.gitignore @@ -6,3 +6,4 @@ duckdb_unittest_tempdir/ testext test/python/__pycache__/ .Rhistory +.vscode From 8e34267ecd4f7f6eb5103eed550bc37545c61b4e Mon Sep 17 00:00:00 2001 From: Pingan Ren Date: Fri, 5 Apr 2024 13:21:54 +0200 Subject: [PATCH 072/249] update --- .../physical_path_finding_operator.cpp | 112 ++++++++++-------- 1 file changed, 63 insertions(+), 49 deletions(-) diff --git a/duckpgq/src/duckpgq/operators/physical_path_finding_operator.cpp b/duckpgq/src/duckpgq/operators/physical_path_finding_operator.cpp index 49f1ae8b..462912bb 100644 --- a/duckpgq/src/duckpgq/operators/physical_path_finding_operator.cpp +++ b/duckpgq/src/duckpgq/operators/physical_path_finding_operator.cpp @@ -118,17 +118,28 @@ void PathFindingLocalState::CreateCSR(DataChunk &input, class GlobalBFSState { public: - GlobalBFSState() = default; - - GlobalBFSState(int64_t v_size_, idx_t pairs_size_, int64_t *src_, int64_t *dst_, - UnifiedVectorFormat &vdata_src_, UnifiedVectorFormat &vdata_dst_) - : iter(1), v_size(v_size_), src(src_), dst(dst_), vdata_src(std::move(vdata_src_)), - vdata_dst(std::move(vdata_dst_)), started_searches(0), - seen(v_size_), visit1(v_size_), visit2(v_size_), - change(false), result(LogicalTypeId::BIGINT, true, true, pairs_size_) { + + void init(int64_t v_size_, idx_t pairs_size_, int64_t *src_, int64_t *dst_, + UnifiedVectorFormat &vdata_src_, UnifiedVectorFormat &vdata_dst_) { + iter = 1; + v_size = v_size_; + src = src_; + dst = dst_; + vdata_src = std::move(vdata_src_); + vdata_dst = std::move(vdata_dst_); + started_searches = 0; for (auto i = 0; i < LANE_LIMIT; i++) { lane_to_num[i] = -1; } + seen.resize(v_size_); + visit1.resize(v_size_); + visit2.resize(v_size_); + for (auto i = 0; i < v_size; i++) { + seen[i] = 0; + visit1[i] = 0; + } + change = false; + result = make_uniq(LogicalType::BIGINT, true, true, pairs_size_); } void clear() { @@ -156,7 +167,7 @@ class GlobalBFSState { vector> visit1; vector> visit2; bool change; - Vector result; + unique_ptr result; }; class PathFindingGlobalState : public GlobalSinkState { @@ -173,9 +184,9 @@ class PathFindingGlobalState : public GlobalSinkState { PathFindingGlobalState(PathFindingGlobalState &prev) : GlobalSinkState(prev), global_tasks(prev.global_tasks), - global_csr(std::move(prev.global_csr)), child(prev.child + 1) { - - } + global_csr(std::move(prev.global_csr)), + global_bfs_state(std::move(prev.global_bfs_state)), child(prev.child + 1) { + } void Sink(DataChunk &input, PathFindingLocalState &lstate) const { lstate.Sink(input, *global_csr); @@ -189,10 +200,10 @@ class PathFindingGlobalState : public GlobalSinkState { ColumnDataAppendState append_state; unique_ptr global_csr; - size_t child; - // state for BFS - unique_ptr global_bfs_state; + GlobalBFSState global_bfs_state; + + size_t child; }; unique_ptr @@ -368,11 +379,11 @@ class PhysicalBFSTask : public ExecutorTask { } TaskExecutionResult ExecuteTask(TaskExecutionMode mode) override { - auto& change = state.global_bfs_state->change; - auto& v_size = state.global_bfs_state->v_size; - auto& seen = state.global_bfs_state->seen; - auto& visit = state.global_bfs_state->iter & 1 ? state.global_bfs_state->visit1 : state.global_bfs_state->visit2; - auto& next = state.global_bfs_state->iter & 1 ? state.global_bfs_state->visit2 : state.global_bfs_state->visit1; + auto& change = state.global_bfs_state.change; + auto& v_size = state.global_bfs_state.v_size; + auto& seen = state.global_bfs_state.seen; + auto& visit = state.global_bfs_state.iter & 1 ? state.global_bfs_state.visit1 : state.global_bfs_state.visit2; + auto& next = state.global_bfs_state.iter & 1 ? state.global_bfs_state.visit2 : state.global_bfs_state.visit1; int64_t *v = (int64_t *)state.global_csr->v; vector &e = state.global_csr->e; @@ -431,19 +442,19 @@ class BFSIterativeEvent : public BasePipelineEvent { void FinishEvent() override { auto& bfs_state = gstate.global_bfs_state; - auto result_data = FlatVector::GetData(bfs_state->result); - ValidityMask &result_validity = FlatVector::Validity(bfs_state->result); + auto result_data = FlatVector::GetData(*bfs_state.result); + ValidityMask &result_validity = FlatVector::Validity(*bfs_state.result); - if (bfs_state->change) { + if (bfs_state.change) { // detect lanes that finished for (int64_t lane = 0; lane < LANE_LIMIT; lane++) { - int64_t search_num = bfs_state->lane_to_num[lane]; + int64_t search_num = bfs_state.lane_to_num[lane]; if (search_num >= 0) { // active lane - int64_t dst_pos = bfs_state->vdata_dst.sel->get_index(search_num); - if (bfs_state->seen[bfs_state->dst[dst_pos]][lane]) { + int64_t dst_pos = bfs_state.vdata_dst.sel->get_index(search_num); + if (bfs_state.seen[bfs_state.dst[dst_pos]][lane]) { result_data[search_num] = - bfs_state->iter; /* found at iter => iter = path length */ - bfs_state->lane_to_num[lane] = -1; // mark inactive + bfs_state.iter; /* found at iter => iter = path length */ + bfs_state.lane_to_num[lane] = -1; // mark inactive } } } @@ -453,16 +464,16 @@ class BFSIterativeEvent : public BasePipelineEvent { } else { // no changes anymore: any still active searches have no path for (int64_t lane = 0; lane < LANE_LIMIT; lane++) { - int64_t search_num = bfs_state->lane_to_num[lane]; + int64_t search_num = bfs_state.lane_to_num[lane]; if (search_num >= 0) { // active lane result_validity.SetInvalid(search_num); result_data[search_num] = (int64_t)-1; /* no path */ - bfs_state->lane_to_num[lane] = -1; // mark inactive + bfs_state.lane_to_num[lane] = -1; // mark inactive } } // if remaining pairs, schedule the BFS for the next batch - if (bfs_state->started_searches < gstate.global_tasks.Count()) { + if (bfs_state.started_searches < gstate.global_tasks.Count()) { PhysicalPathFinding::ScheduleBFSTasks(*pipeline, *this, gstate); } } @@ -480,6 +491,7 @@ PhysicalPathFinding::Finalize(Pipeline &pipeline, Event &event, if (global_tasks.Count() != 0) { DataChunk all_pairs; DataChunk pairs; + global_tasks.InitializeScanChunk(all_pairs); global_tasks.InitializeScanChunk(pairs); ColumnDataScanState scan_state; global_tasks.InitializeScan(scan_state); @@ -498,17 +510,19 @@ PhysicalPathFinding::Finalize(Pipeline &pipeline, Event &event, auto src_data = FlatVector::GetData(src); auto dst_data = FlatVector::GetData(dst); - gstate.global_bfs_state = make_uniq(csr->v_size, - global_tasks.Count(), src_data, dst_data, vdata_src, vdata_dst); + // gstate.global_bfs_state = make_uniq(csr->v_size, + // global_tasks.Count(), src_data, dst_data, vdata_src, vdata_dst); + gstate.global_bfs_state.init(csr->v_size, global_tasks.Count(), src_data, dst_data, vdata_src, vdata_dst); // Schedule the first round of BFS tasks if (all_pairs.size() > 0) { ScheduleBFSTasks(pipeline, event, gstate); } + + // debug print + gstate.global_bfs_state.result->Print(global_tasks.Count()); } - // debug print - gstate.global_bfs_state->result.Print(global_tasks.Count()); // Move to the next input child ++gstate.child; @@ -516,34 +530,34 @@ PhysicalPathFinding::Finalize(Pipeline &pipeline, Event &event, return SinkFinalizeType::READY; } -void ScheduleBFSTasks(Pipeline &pipeline, Event &event, GlobalSinkState &state) { +void PhysicalPathFinding::ScheduleBFSTasks(Pipeline &pipeline, Event &event, GlobalSinkState &state) { auto &gstate = state.Cast(); auto &bfs_state = gstate.global_bfs_state; // for every batch of pairs, schedule a BFS task - bfs_state->clear(); + bfs_state.clear(); // remaining pairs - if (bfs_state->started_searches < gstate.global_tasks.Count()) { + if (bfs_state.started_searches < gstate.global_tasks.Count()) { - auto result_data = FlatVector::GetData(bfs_state->result); - auto& result_validity = FlatVector::Validity(bfs_state->result); + auto result_data = FlatVector::GetData(*bfs_state.result); + auto& result_validity = FlatVector::Validity(*bfs_state.result); for (int64_t lane = 0; lane < LANE_LIMIT; lane++) { - bfs_state->lane_to_num[lane] = -1; - while (bfs_state->started_searches < gstate.global_tasks.Count()) { - int64_t search_num = bfs_state->started_searches++; - int64_t src_pos = bfs_state->vdata_src.sel->get_index(search_num); - int64_t dst_pos = bfs_state->vdata_src.sel->get_index(search_num); - if (!bfs_state->vdata_src.validity.RowIsValid(src_pos)) { + bfs_state.lane_to_num[lane] = -1; + while (bfs_state.started_searches < gstate.global_tasks.Count()) { + int64_t search_num = bfs_state.started_searches++; + int64_t src_pos = bfs_state.vdata_src.sel->get_index(search_num); + int64_t dst_pos = bfs_state.vdata_src.sel->get_index(search_num); + if (!bfs_state.vdata_src.validity.RowIsValid(src_pos)) { result_validity.SetInvalid(search_num); result_data[search_num] = (uint64_t)-1; /* no path */ - } else if (bfs_state->src[src_pos] == bfs_state->dst[dst_pos]) { + } else if (bfs_state.src[src_pos] == bfs_state.dst[dst_pos]) { result_data[search_num] = (uint64_t)0; // path of length 0 does not require a search } else { - bfs_state->visit1[bfs_state->src[src_pos]][lane] = true; - bfs_state->lane_to_num[lane] = search_num; // active lane + bfs_state.visit1[bfs_state.src[src_pos]][lane] = true; + bfs_state.lane_to_num[lane] = search_num; // active lane break; } } From 9c0f90d2cd167ae728aa5b1459e1c076d1875e76 Mon Sep 17 00:00:00 2001 From: Pingan Ren Date: Tue, 9 Apr 2024 15:54:21 +0200 Subject: [PATCH 073/249] stupid parallel --- src/include/duckpgq_extension.hpp | 1 - .../physical_path_finding_operator.cpp | 214 ++++-------------- 2 files changed, 46 insertions(+), 169 deletions(-) diff --git a/src/include/duckpgq_extension.hpp b/src/include/duckpgq_extension.hpp index dc2e3ee4..b283a2d7 100644 --- a/src/include/duckpgq_extension.hpp +++ b/src/include/duckpgq_extension.hpp @@ -1,6 +1,5 @@ #pragma once -#include "duckdb/common/compressed_sparse_row.hpp" #include "duckdb/parser/column_list.hpp" #include "duckdb/parser/parsed_data/create_property_graph_info.hpp" #include "duckdb/parser/parsed_expression.hpp" diff --git a/src/operators/physical_path_finding_operator.cpp b/src/operators/physical_path_finding_operator.cpp index 462912bb..58deb507 100644 --- a/src/operators/physical_path_finding_operator.cpp +++ b/src/operators/physical_path_finding_operator.cpp @@ -9,6 +9,8 @@ #include "duckdb/parallel/base_pipeline_event.hpp" #include "duckdb/common/types/column/column_data_collection.hpp" #include +#include +#include namespace duckdb { @@ -119,14 +121,16 @@ void PathFindingLocalState::CreateCSR(DataChunk &input, class GlobalBFSState { public: - void init(int64_t v_size_, idx_t pairs_size_, int64_t *src_, int64_t *dst_, - UnifiedVectorFormat &vdata_src_, UnifiedVectorFormat &vdata_dst_) { + void init(shared_ptr pairs_, int64_t v_size_) { iter = 1; v_size = v_size_; - src = src_; - dst = dst_; - vdata_src = std::move(vdata_src_); - vdata_dst = std::move(vdata_dst_); + pairs = pairs_; + auto &src_data = pairs->data[0]; + auto &dst_data = pairs->data[1]; + src_data.ToUnifiedFormat(pairs->size(), vdata_src); + dst_data.ToUnifiedFormat(pairs->size(), vdata_dst); + src = FlatVector::GetData(src_data); + dst = FlatVector::GetData(dst_data); started_searches = 0; for (auto i = 0; i < LANE_LIMIT; i++) { lane_to_num[i] = -1; @@ -139,7 +143,7 @@ class GlobalBFSState { visit1[i] = 0; } change = false; - result = make_uniq(LogicalType::BIGINT, true, true, pairs_size_); + result = make_uniq(LogicalType::BIGINT, true, true, pairs->size()); } void clear() { @@ -155,6 +159,7 @@ class GlobalBFSState { } } public: + shared_ptr pairs; int64_t iter; int64_t v_size; int64_t *src; @@ -162,7 +167,7 @@ class GlobalBFSState { UnifiedVectorFormat vdata_src; UnifiedVectorFormat vdata_dst; idx_t started_searches; - idx_t lane_to_num[LANE_LIMIT]; + int64_t lane_to_num[LANE_LIMIT]; vector> seen; vector> visit1; vector> visit2; @@ -249,129 +254,6 @@ PhysicalPathFinding::Combine(ExecutionContext &context, // Finalize //===--------------------------------------------------------------------===// -static bool IterativeLength(int64_t v_size, int64_t *v, vector &e, - vector> &seen, - vector> &visit, - vector> &next) { - bool change = false; - for (auto i = 0; i < v_size; i++) { - next[i] = 0; - } - for (auto i = 0; i < v_size; i++) { - if (visit[i].any()) { - for (auto offset = v[i]; offset < v[i + 1]; offset++) { - auto n = e[offset]; - next[n] = next[n] | visit[i]; - } - } - } - for (auto i = 0; i < v_size; i++) { - next[i] = next[i] & ~seen[i]; - seen[i] = seen[i] | next[i]; - change |= next[i].any(); - } - return change; -} - -static void IterativeLengthFunction(const unique_ptr &csr, - DataChunk &pairs, Vector &result) { - int64_t v_size = csr->v_size; - int64_t *v = (int64_t *)csr->v; - vector &e = csr->e; - - // get src and dst vectors for searches - auto &src = pairs.data[0]; - auto &dst = pairs.data[1]; - UnifiedVectorFormat vdata_src; - UnifiedVectorFormat vdata_dst; - src.ToUnifiedFormat(pairs.size(), vdata_src); - dst.ToUnifiedFormat(pairs.size(), vdata_dst); - - auto src_data = FlatVector::GetData(src); - auto dst_data = FlatVector::GetData(dst); - - ValidityMask &result_validity = FlatVector::Validity(result); - - // create result vector - result.SetVectorType(VectorType::FLAT_VECTOR); - auto result_data = FlatVector::GetData(result); - - // create temp SIMD arrays - vector> seen(v_size); - vector> visit1(v_size); - vector> visit2(v_size); - - // maps lane to search number - short lane_to_num[LANE_LIMIT]; - for (int64_t lane = 0; lane < LANE_LIMIT; lane++) { - lane_to_num[lane] = -1; // inactive - } - - idx_t started_searches = 0; - while (started_searches < pairs.size()) { - - // empty visit vectors - for (auto i = 0; i < v_size; i++) { - seen[i] = 0; - visit1[i] = 0; - } - - // add search jobs to free lanes - uint64_t active = 0; - for (int64_t lane = 0; lane < LANE_LIMIT; lane++) { - lane_to_num[lane] = -1; - while (started_searches < pairs.size()) { - int64_t search_num = started_searches++; - int64_t src_pos = vdata_src.sel->get_index(search_num); - int64_t dst_pos = vdata_dst.sel->get_index(search_num); - if (!vdata_src.validity.RowIsValid(src_pos)) { - result_validity.SetInvalid(search_num); - result_data[search_num] = (uint64_t)-1; /* no path */ - } else if (src_data[src_pos] == dst_data[dst_pos]) { - result_data[search_num] = - (uint64_t)0; // path of length 0 does not require a search - } else { - visit1[src_data[src_pos]][lane] = true; - lane_to_num[lane] = search_num; // active lane - active++; - break; - } - } - } - - // make passes while a lane is still active - for (int64_t iter = 1; active; iter++) { - if (!IterativeLength(v_size, v, e, seen, (iter & 1) ? visit1 : visit2, - (iter & 1) ? visit2 : visit1)) { - break; - } - // detect lanes that finished - for (int64_t lane = 0; lane < LANE_LIMIT; lane++) { - int64_t search_num = lane_to_num[lane]; - if (search_num >= 0) { // active lane - int64_t dst_pos = vdata_dst.sel->get_index(search_num); - if (seen[dst_data[dst_pos]][lane]) { - result_data[search_num] = - iter; /* found at iter => iter = path length */ - lane_to_num[lane] = -1; // mark inactive - active--; - } - } - } - } - - // no changes anymore: any still active searches have no path - for (int64_t lane = 0; lane < LANE_LIMIT; lane++) { - int64_t search_num = lane_to_num[lane]; - if (search_num >= 0) { // active lane - result_validity.SetInvalid(search_num); - result_data[search_num] = (int64_t)-1; /* no path */ - lane_to_num[lane] = -1; // mark inactive - } - } - } -} - class PhysicalBFSTask : public ExecutorTask { public: PhysicalBFSTask(shared_ptr event_p, ClientContext &context, PathFindingGlobalState &state, idx_t start, idx_t end) @@ -380,29 +262,32 @@ class PhysicalBFSTask : public ExecutorTask { TaskExecutionResult ExecuteTask(TaskExecutionMode mode) override { auto& change = state.global_bfs_state.change; - auto& v_size = state.global_bfs_state.v_size; auto& seen = state.global_bfs_state.seen; auto& visit = state.global_bfs_state.iter & 1 ? state.global_bfs_state.visit1 : state.global_bfs_state.visit2; auto& next = state.global_bfs_state.iter & 1 ? state.global_bfs_state.visit2 : state.global_bfs_state.visit1; int64_t *v = (int64_t *)state.global_csr->v; vector &e = state.global_csr->e; - for (auto i = 0; i < v_size; i++) { + for (auto i = start; i < end; i++) { next[i] = 0; } - for (auto i = 0; i < v_size; i++) { + for (auto i = start; i < end; i++) { if (visit[i].any()) { for (auto offset = v[i]; offset < v[i + 1]; offset++) { auto n = e[offset]; next[n] = next[n] | visit[i]; + + next[n] = next[n] & ~seen[n]; + seen[n] = seen[n] | next[n]; + change |= next[n].any(); } } } - for (auto i = 0; i < v_size; i++) { - next[i] = next[i] & ~seen[i]; - seen[i] = seen[i] | next[i]; - change |= next[i].any(); - } + // for (auto i = start; i < end; i++) { + // next[i] = next[i] & ~seen[i]; + // seen[i] = seen[i] | next[i]; + // change |= next[i].any(); + // } event->FinishTask(); return TaskExecutionResult::TASK_FINISHED; @@ -411,6 +296,7 @@ class PhysicalBFSTask : public ExecutorTask { private: ClientContext &context; PathFindingGlobalState &state; + // [start, end) idx_t start; idx_t end; }; @@ -425,17 +311,22 @@ class BFSIterativeEvent : public BasePipelineEvent { public: void Schedule() override { + auto &bfs_state = gstate.global_bfs_state; auto &context = pipeline->GetClientContext(); + bfs_state.change = false; + // Schedule tasks equal to the number of threads, which will each merge multiple partitions auto &ts = TaskScheduler::GetScheduler(context); - idx_t num_threads = ts.NumberOfThreads(); + // idx_t num_threads = ts.NumberOfThreads(); + idx_t num_threads = std::min((int64_t)ts.NumberOfThreads(), bfs_state.v_size); + idx_t blocks = floor(bfs_state.v_size / (float)num_threads); vector> bfs_tasks; - bfs_tasks.push_back(make_uniq(shared_from_this(), context, gstate, 0, 0)); - // for (idx_t tnum = 0; tnum < num_threads; tnum++) { - // bfs_tasks.push_back(make_uniq(shared_from_this(), context, gstate)); - // } + for (idx_t tnum = 0; tnum < num_threads; tnum++) { + bfs_tasks.push_back(make_uniq(shared_from_this(), context, gstate, + tnum * blocks, std::min(tnum * blocks + blocks, (idx_t)bfs_state.v_size))); + } SetTasks(std::move(bfs_tasks)); } @@ -459,6 +350,7 @@ class BFSIterativeEvent : public BasePipelineEvent { } } // into the next iteration + bfs_state.iter++; auto bfs_event = std::make_shared(gstate, *pipeline); this->InsertEvent(std::dynamic_pointer_cast(bfs_event)); } else { @@ -489,38 +381,24 @@ PhysicalPathFinding::Finalize(Pipeline &pipeline, Event &event, auto &csr = gstate.global_csr; auto &global_tasks = gstate.global_tasks; if (global_tasks.Count() != 0) { - DataChunk all_pairs; + auto all_pairs = make_shared(); DataChunk pairs; - global_tasks.InitializeScanChunk(all_pairs); + global_tasks.InitializeScanChunk(*all_pairs); global_tasks.InitializeScanChunk(pairs); ColumnDataScanState scan_state; global_tasks.InitializeScan(scan_state); while (global_tasks.Scan(scan_state, pairs)) { - all_pairs.Append(pairs, true); + all_pairs->Append(pairs, true); } // debug print - all_pairs.Print(); - - auto &src = all_pairs.data[0]; - auto &dst = all_pairs.data[1]; - UnifiedVectorFormat vdata_src; - UnifiedVectorFormat vdata_dst; - src.ToUnifiedFormat(all_pairs.size(), vdata_src); - dst.ToUnifiedFormat(all_pairs.size(), vdata_dst); - auto src_data = FlatVector::GetData(src); - auto dst_data = FlatVector::GetData(dst); - - // gstate.global_bfs_state = make_uniq(csr->v_size, - // global_tasks.Count(), src_data, dst_data, vdata_src, vdata_dst); - gstate.global_bfs_state.init(csr->v_size, global_tasks.Count(), src_data, dst_data, vdata_src, vdata_dst); + all_pairs->Print(); + gstate.global_bfs_state.init(all_pairs, csr->v_size); // Schedule the first round of BFS tasks - if (all_pairs.size() > 0) { + if (all_pairs->size() > 0) { ScheduleBFSTasks(pipeline, event, gstate); } - // debug print - gstate.global_bfs_state.result->Print(global_tasks.Count()); } @@ -548,7 +426,7 @@ void PhysicalPathFinding::ScheduleBFSTasks(Pipeline &pipeline, Event &event, Glo while (bfs_state.started_searches < gstate.global_tasks.Count()) { int64_t search_num = bfs_state.started_searches++; int64_t src_pos = bfs_state.vdata_src.sel->get_index(search_num); - int64_t dst_pos = bfs_state.vdata_src.sel->get_index(search_num); + int64_t dst_pos = bfs_state.vdata_dst.sel->get_index(search_num); if (!bfs_state.vdata_src.validity.RowIsValid(src_pos)) { result_validity.SetInvalid(search_num); result_data[search_num] = (uint64_t)-1; /* no path */ @@ -637,10 +515,10 @@ PhysicalPathFinding::GetData(ExecutionContext &context, DataChunk &result, OperatorSourceInput &input) const { auto &pf_sink = sink_state->Cast(); auto &pf_gstate = input.global_state.Cast(); + auto &pf_bfs_state = pf_sink.global_bfs_state; // auto &pf_lstate = input.local_state.Cast(); - pf_sink.global_tasks.Scan(pf_sink.scan_state, result); - result.Print(); - pf_gstate.Initialize(pf_sink); + pf_bfs_state.result->Print(pf_bfs_state.pairs->size()); + // pf_gstate.Initialize(pf_sink); return result.size() == 0 ? SourceResultType::FINISHED : SourceResultType::HAVE_MORE_OUTPUT; From e62a5cf8910953fdb7135f6c4c32bc6a27972ed4 Mon Sep 17 00:00:00 2001 From: Pingan Ren Date: Sun, 14 Apr 2024 20:38:16 +0200 Subject: [PATCH 074/249] lock and barrier sync --- .../physical_path_finding_operator.cpp | 156 +++++++++++------- 1 file changed, 95 insertions(+), 61 deletions(-) diff --git a/src/operators/physical_path_finding_operator.cpp b/src/operators/physical_path_finding_operator.cpp index 58deb507..5388eacd 100644 --- a/src/operators/physical_path_finding_operator.cpp +++ b/src/operators/physical_path_finding_operator.cpp @@ -118,35 +118,57 @@ void PathFindingLocalState::CreateCSR(DataChunk &input, // global_csr.Print(); // Debug print } +class Barrier { +public: + void Init(std::size_t iCount) { + mThreshold = iCount; + mCount = iCount; + mGeneration = 0; + } + + void Wait() { + std::unique_lock lLock{mMutex}; + auto lGen = mGeneration; + if (!--mCount) { + mGeneration++; + mCount = mThreshold; + mCond.notify_all(); + } else { + mCond.wait(lLock, [this, lGen] { return lGen != mGeneration; }); + } + } + +private: + std::mutex mMutex; + std::condition_variable mCond; + std::size_t mThreshold; + std::size_t mCount; + std::size_t mGeneration; +}; + class GlobalBFSState { public: - void init(shared_ptr pairs_, int64_t v_size_) { - iter = 1; - v_size = v_size_; - pairs = pairs_; + GlobalBFSState(shared_ptr pairs_, int64_t v_size_) + : pairs(pairs_), iter(1), v_size(v_size_), change(false), started_searches(0), + seen(v_size_), visit1(v_size_), visit2(v_size_), + result(make_uniq(LogicalType::BIGINT, true, true, pairs_->size())) { + for (auto i = 0; i < LANE_LIMIT; i++) { + lane_to_num[i] = -1; + } auto &src_data = pairs->data[0]; auto &dst_data = pairs->data[1]; src_data.ToUnifiedFormat(pairs->size(), vdata_src); dst_data.ToUnifiedFormat(pairs->size(), vdata_dst); src = FlatVector::GetData(src_data); dst = FlatVector::GetData(dst_data); - started_searches = 0; - for (auto i = 0; i < LANE_LIMIT; i++) { - lane_to_num[i] = -1; - } - seen.resize(v_size_); - visit1.resize(v_size_); - visit2.resize(v_size_); for (auto i = 0; i < v_size; i++) { seen[i] = 0; visit1[i] = 0; } - change = false; - result = make_uniq(LogicalType::BIGINT, true, true, pairs->size()); } - void clear() { + void Clear() { iter = 1; change = false; for (auto i = 0; i < LANE_LIMIT; i++) { @@ -162,17 +184,22 @@ class GlobalBFSState { shared_ptr pairs; int64_t iter; int64_t v_size; + bool change; + idx_t started_searches; int64_t *src; int64_t *dst; UnifiedVectorFormat vdata_src; UnifiedVectorFormat vdata_dst; - idx_t started_searches; int64_t lane_to_num[LANE_LIMIT]; vector> seen; vector> visit1; vector> visit2; - bool change; unique_ptr result; + + Barrier barrier; + + // lock for next + mutable mutex lock; }; class PathFindingGlobalState : public GlobalSinkState { @@ -206,7 +233,7 @@ class PathFindingGlobalState : public GlobalSinkState { unique_ptr global_csr; // state for BFS - GlobalBFSState global_bfs_state; + unique_ptr global_bfs_state; size_t child; }; @@ -261,10 +288,13 @@ class PhysicalBFSTask : public ExecutorTask { } TaskExecutionResult ExecuteTask(TaskExecutionMode mode) override { - auto& change = state.global_bfs_state.change; - auto& seen = state.global_bfs_state.seen; - auto& visit = state.global_bfs_state.iter & 1 ? state.global_bfs_state.visit1 : state.global_bfs_state.visit2; - auto& next = state.global_bfs_state.iter & 1 ? state.global_bfs_state.visit2 : state.global_bfs_state.visit1; + auto& bfs_state = state.global_bfs_state; + auto& change = bfs_state->change; + auto& seen = bfs_state->seen; + auto& visit = bfs_state->iter & 1 ? bfs_state->visit1 : bfs_state->visit2; + auto& next = bfs_state->iter & 1 ? bfs_state->visit2 : bfs_state->visit1; + auto& barrier = bfs_state->barrier; + int64_t *v = (int64_t *)state.global_csr->v; vector &e = state.global_csr->e; @@ -275,19 +305,21 @@ class PhysicalBFSTask : public ExecutorTask { if (visit[i].any()) { for (auto offset = v[i]; offset < v[i + 1]; offset++) { auto n = e[offset]; + lock_guard lock(bfs_state->lock); next[n] = next[n] | visit[i]; - - next[n] = next[n] & ~seen[n]; - seen[n] = seen[n] | next[n]; - change |= next[n].any(); } } } - // for (auto i = start; i < end; i++) { - // next[i] = next[i] & ~seen[i]; - // seen[i] = seen[i] | next[i]; - // change |= next[i].any(); - // } + + barrier.Wait(); + + for (auto i = start; i < end; i++) { + if (next[i].any()) { + next[i] = next[i] & ~seen[i]; + seen[i] = seen[i] | next[i]; + change |= next[i].any(); + } + } event->FinishTask(); return TaskExecutionResult::TASK_FINISHED; @@ -314,18 +346,20 @@ class BFSIterativeEvent : public BasePipelineEvent { auto &bfs_state = gstate.global_bfs_state; auto &context = pipeline->GetClientContext(); - bfs_state.change = false; + bfs_state->change = false; // Schedule tasks equal to the number of threads, which will each merge multiple partitions auto &ts = TaskScheduler::GetScheduler(context); // idx_t num_threads = ts.NumberOfThreads(); - idx_t num_threads = std::min((int64_t)ts.NumberOfThreads(), bfs_state.v_size); - idx_t blocks = floor(bfs_state.v_size / (float)num_threads); + idx_t num_threads = std::min((int64_t)ts.NumberOfThreads(), bfs_state->v_size); + idx_t blocks = floor(bfs_state->v_size / (float)num_threads); + + bfs_state->barrier.Init(num_threads); vector> bfs_tasks; for (idx_t tnum = 0; tnum < num_threads; tnum++) { bfs_tasks.push_back(make_uniq(shared_from_this(), context, gstate, - tnum * blocks, std::min(tnum * blocks + blocks, (idx_t)bfs_state.v_size))); + tnum * blocks, std::min(tnum * blocks + blocks, (idx_t)bfs_state->v_size))); } SetTasks(std::move(bfs_tasks)); } @@ -333,39 +367,39 @@ class BFSIterativeEvent : public BasePipelineEvent { void FinishEvent() override { auto& bfs_state = gstate.global_bfs_state; - auto result_data = FlatVector::GetData(*bfs_state.result); - ValidityMask &result_validity = FlatVector::Validity(*bfs_state.result); + auto result_data = FlatVector::GetData(*bfs_state->result); + ValidityMask &result_validity = FlatVector::Validity(*bfs_state->result); - if (bfs_state.change) { + if (bfs_state->change) { // detect lanes that finished for (int64_t lane = 0; lane < LANE_LIMIT; lane++) { - int64_t search_num = bfs_state.lane_to_num[lane]; + int64_t search_num = bfs_state->lane_to_num[lane]; if (search_num >= 0) { // active lane - int64_t dst_pos = bfs_state.vdata_dst.sel->get_index(search_num); - if (bfs_state.seen[bfs_state.dst[dst_pos]][lane]) { + int64_t dst_pos = bfs_state->vdata_dst.sel->get_index(search_num); + if (bfs_state->seen[bfs_state->dst[dst_pos]][lane]) { result_data[search_num] = - bfs_state.iter; /* found at iter => iter = path length */ - bfs_state.lane_to_num[lane] = -1; // mark inactive + bfs_state->iter; /* found at iter => iter = path length */ + bfs_state->lane_to_num[lane] = -1; // mark inactive } } } // into the next iteration - bfs_state.iter++; + bfs_state->iter++; auto bfs_event = std::make_shared(gstate, *pipeline); this->InsertEvent(std::dynamic_pointer_cast(bfs_event)); } else { // no changes anymore: any still active searches have no path for (int64_t lane = 0; lane < LANE_LIMIT; lane++) { - int64_t search_num = bfs_state.lane_to_num[lane]; + int64_t search_num = bfs_state->lane_to_num[lane]; if (search_num >= 0) { // active lane result_validity.SetInvalid(search_num); result_data[search_num] = (int64_t)-1; /* no path */ - bfs_state.lane_to_num[lane] = -1; // mark inactive + bfs_state->lane_to_num[lane] = -1; // mark inactive } } // if remaining pairs, schedule the BFS for the next batch - if (bfs_state.started_searches < gstate.global_tasks.Count()) { + if (bfs_state->started_searches < gstate.global_tasks.Count()) { PhysicalPathFinding::ScheduleBFSTasks(*pipeline, *this, gstate); } } @@ -392,7 +426,7 @@ PhysicalPathFinding::Finalize(Pipeline &pipeline, Event &event, } // debug print all_pairs->Print(); - gstate.global_bfs_state.init(all_pairs, csr->v_size); + gstate.global_bfs_state = make_uniq(all_pairs, csr->v_size); // Schedule the first round of BFS tasks if (all_pairs->size() > 0) { @@ -413,29 +447,29 @@ void PhysicalPathFinding::ScheduleBFSTasks(Pipeline &pipeline, Event &event, Glo auto &bfs_state = gstate.global_bfs_state; // for every batch of pairs, schedule a BFS task - bfs_state.clear(); + bfs_state->Clear(); // remaining pairs - if (bfs_state.started_searches < gstate.global_tasks.Count()) { + if (bfs_state->started_searches < gstate.global_tasks.Count()) { - auto result_data = FlatVector::GetData(*bfs_state.result); - auto& result_validity = FlatVector::Validity(*bfs_state.result); + auto result_data = FlatVector::GetData(*bfs_state->result); + auto& result_validity = FlatVector::Validity(*bfs_state->result); for (int64_t lane = 0; lane < LANE_LIMIT; lane++) { - bfs_state.lane_to_num[lane] = -1; - while (bfs_state.started_searches < gstate.global_tasks.Count()) { - int64_t search_num = bfs_state.started_searches++; - int64_t src_pos = bfs_state.vdata_src.sel->get_index(search_num); - int64_t dst_pos = bfs_state.vdata_dst.sel->get_index(search_num); - if (!bfs_state.vdata_src.validity.RowIsValid(src_pos)) { + bfs_state->lane_to_num[lane] = -1; + while (bfs_state->started_searches < gstate.global_tasks.Count()) { + int64_t search_num = bfs_state->started_searches++; + int64_t src_pos = bfs_state->vdata_src.sel->get_index(search_num); + int64_t dst_pos = bfs_state->vdata_dst.sel->get_index(search_num); + if (!bfs_state->vdata_src.validity.RowIsValid(src_pos)) { result_validity.SetInvalid(search_num); result_data[search_num] = (uint64_t)-1; /* no path */ - } else if (bfs_state.src[src_pos] == bfs_state.dst[dst_pos]) { + } else if (bfs_state->src[src_pos] == bfs_state->dst[dst_pos]) { result_data[search_num] = (uint64_t)0; // path of length 0 does not require a search } else { - bfs_state.visit1[bfs_state.src[src_pos]][lane] = true; - bfs_state.lane_to_num[lane] = search_num; // active lane + bfs_state->visit1[bfs_state->src[src_pos]][lane] = true; + bfs_state->lane_to_num[lane] = search_num; // active lane break; } } @@ -517,7 +551,7 @@ PhysicalPathFinding::GetData(ExecutionContext &context, DataChunk &result, auto &pf_gstate = input.global_state.Cast(); auto &pf_bfs_state = pf_sink.global_bfs_state; // auto &pf_lstate = input.local_state.Cast(); - pf_bfs_state.result->Print(pf_bfs_state.pairs->size()); + pf_bfs_state->result->Print(pf_bfs_state->pairs->size()); // pf_gstate.Initialize(pf_sink); return result.size() == 0 ? SourceResultType::FINISHED From 5be7d827ddae248bb6e79b9935d0a2e4ac49ce4b Mon Sep 17 00:00:00 2001 From: Pingan Ren Date: Tue, 16 Apr 2024 14:46:47 +0200 Subject: [PATCH 075/249] Bottom-up BFS, but not applied --- .../physical_path_finding_operator.hpp | 2 + .../physical_path_finding_operator.cpp | 115 ++++++++++++++++-- 2 files changed, 110 insertions(+), 7 deletions(-) diff --git a/src/include/duckpgq/operators/physical_path_finding_operator.hpp b/src/include/duckpgq/operators/physical_path_finding_operator.hpp index e5dead10..98e50a44 100644 --- a/src/include/duckpgq/operators/physical_path_finding_operator.hpp +++ b/src/include/duckpgq/operators/physical_path_finding_operator.hpp @@ -28,7 +28,9 @@ class PhysicalPathFinding : public PhysicalComparisonJoin { // ~GlobalCompressedSparseRow() { delete[] v; } atomic *v; + atomic *reverse_v; vector e; + vector reverse_e; vector edge_ids; vector w; vector w_double; diff --git a/src/operators/physical_path_finding_operator.cpp b/src/operators/physical_path_finding_operator.cpp index 5388eacd..03c9495d 100644 --- a/src/operators/physical_path_finding_operator.cpp +++ b/src/operators/physical_path_finding_operator.cpp @@ -33,12 +33,14 @@ void PhysicalPathFinding::GlobalCompressedSparseRow::InitializeVertex(int64_t v_ v_size = v_size_ + 2; try { v = new std::atomic[v_size]; + reverse_v = new std::atomic[v_size]; } catch (std::bad_alloc const &) { throw InternalException("Unable to initialize vector of size for csr vertex table " "representation"); } for (idx_t i = 0; i < v_size; ++i) { v[i].store(0); + reverse_v[i].store(0); } initialized_v = true; } @@ -47,6 +49,7 @@ void PhysicalPathFinding::GlobalCompressedSparseRow::InitializeEdge( const lock_guard csr_init_lock(csr_lock); try { e.resize(e_size, 0); + reverse_e.resize(e_size, 0); edge_ids.resize(e_size, 0); } catch (std::bad_alloc const &) { throw InternalException("Unable to initialize vector of size for csr " @@ -152,7 +155,8 @@ class GlobalBFSState { GlobalBFSState(shared_ptr pairs_, int64_t v_size_) : pairs(pairs_), iter(1), v_size(v_size_), change(false), started_searches(0), seen(v_size_), visit1(v_size_), visit2(v_size_), - result(make_uniq(LogicalType::BIGINT, true, true, pairs_->size())) { + result(make_uniq(LogicalType::BIGINT, true, true, pairs_->size())), + frontier_size(0), unseen_size(v_size_) { for (auto i = 0; i < LANE_LIMIT; i++) { lane_to_num[i] = -1; } @@ -171,6 +175,8 @@ class GlobalBFSState { void Clear() { iter = 1; change = false; + frontier_size = 0; + unseen_size = v_size; for (auto i = 0; i < LANE_LIMIT; i++) { lane_to_num[i] = -1; } @@ -196,6 +202,12 @@ class GlobalBFSState { vector> visit2; unique_ptr result; + atomic frontier_size; + atomic unseen_size; + constexpr static int64_t alpha = 1024; + constexpr static int64_t beta = 64; + bool is_top_down = true; + Barrier barrier; // lock for next @@ -281,9 +293,9 @@ PhysicalPathFinding::Combine(ExecutionContext &context, // Finalize //===--------------------------------------------------------------------===// -class PhysicalBFSTask : public ExecutorTask { +class PhysicalBFSTopDownTask : public ExecutorTask { public: - PhysicalBFSTask(shared_ptr event_p, ClientContext &context, PathFindingGlobalState &state, idx_t start, idx_t end) + PhysicalBFSTopDownTask(shared_ptr event_p, ClientContext &context, PathFindingGlobalState &state, idx_t start, idx_t end) : ExecutorTask(context, std::move(event_p)), context(context), state(state), start(start), end(end) { } @@ -294,6 +306,8 @@ class PhysicalBFSTask : public ExecutorTask { auto& visit = bfs_state->iter & 1 ? bfs_state->visit1 : bfs_state->visit2; auto& next = bfs_state->iter & 1 ? bfs_state->visit2 : bfs_state->visit1; auto& barrier = bfs_state->barrier; + auto& frontier_size = bfs_state->frontier_size; + auto& unseen_size = bfs_state->unseen_size; int64_t *v = (int64_t *)state.global_csr->v; vector &e = state.global_csr->e; @@ -301,6 +315,9 @@ class PhysicalBFSTask : public ExecutorTask { for (auto i = start; i < end; i++) { next[i] = 0; } + + barrier.Wait(); + for (auto i = start; i < end; i++) { if (visit[i].any()) { for (auto offset = v[i]; offset < v[i + 1]; offset++) { @@ -318,7 +335,64 @@ class PhysicalBFSTask : public ExecutorTask { next[i] = next[i] & ~seen[i]; seen[i] = seen[i] | next[i]; change |= next[i].any(); + + frontier_size = next[i].any() ? frontier_size + 1 : frontier_size.load(); + unseen_size = seen[i].all() ? unseen_size - 1 : unseen_size.load(); + } + } + + event->FinishTask(); + return TaskExecutionResult::TASK_FINISHED; + } + +private: + ClientContext &context; + PathFindingGlobalState &state; + // [start, end) + idx_t start; + idx_t end; +}; + +class PhysicalBFSBottomUpTask : public ExecutorTask { +public: + PhysicalBFSBottomUpTask(shared_ptr event_p, ClientContext &context, PathFindingGlobalState &state, idx_t start, idx_t end) + : ExecutorTask(context, std::move(event_p)), context(context), state(state), start(start), end(end) { + } + + TaskExecutionResult ExecuteTask(TaskExecutionMode mode) override { + auto& bfs_state = state.global_bfs_state; + auto& change = bfs_state->change; + auto& seen = bfs_state->seen; + auto& visit = bfs_state->iter & 1 ? bfs_state->visit1 : bfs_state->visit2; + auto& next = bfs_state->iter & 1 ? bfs_state->visit2 : bfs_state->visit1; + auto& barrier = bfs_state->barrier; + auto& frontier_size = bfs_state->frontier_size; + auto& unseen_size = bfs_state->unseen_size; + + int64_t *v = (int64_t *)state.global_csr->v; + vector &e = state.global_csr->e; + + for (auto i = start; i < end; i++) { + next[i] = 0; + } + + barrier.Wait(); + + for (auto i = start; i < end; i++) { + if (seen[i].all()) { + unseen_size -= 1; + continue; + } + for (auto offset = v[i]; offset < v[i + 1]; offset++) { + auto n = e[offset]; + next[i] = next[i] | visit[n]; } + next[i] = next[i] & ~seen[i]; + seen[i] = seen[i] | next[i]; + change |= next[i].any(); + + frontier_size = next[i].any() ? frontier_size + 1 : frontier_size.load(); + unseen_size = seen[i].all() ? unseen_size - 1 : unseen_size.load(); } event->FinishTask(); @@ -348,6 +422,22 @@ class BFSIterativeEvent : public BasePipelineEvent { bfs_state->change = false; + // Determine the switch of algorithms + if (bfs_state->is_top_down) { + auto Ctb = bfs_state->unseen_size / bfs_state->alpha; + if (bfs_state->frontier_size > Ctb) { + bfs_state->is_top_down = false; + } + } else { + auto Cbt = bfs_state->v_size / bfs_state->beta; + if (bfs_state->frontier_size < Cbt) { + bfs_state->is_top_down = true; + } + } + // clear the counters after the switch + bfs_state->frontier_size = 0; + bfs_state->unseen_size = bfs_state->v_size; + // Schedule tasks equal to the number of threads, which will each merge multiple partitions auto &ts = TaskScheduler::GetScheduler(context); // idx_t num_threads = ts.NumberOfThreads(); @@ -357,10 +447,19 @@ class BFSIterativeEvent : public BasePipelineEvent { bfs_state->barrier.Init(num_threads); vector> bfs_tasks; - for (idx_t tnum = 0; tnum < num_threads; tnum++) { - bfs_tasks.push_back(make_uniq(shared_from_this(), context, gstate, + for (idx_t tnum = 0; tnum < num_threads; tnum++) { + bfs_tasks.push_back(make_uniq(shared_from_this(), context, gstate, tnum * blocks, std::min(tnum * blocks + blocks, (idx_t)bfs_state->v_size))); - } + } + // for (idx_t tnum = 0; tnum < num_threads; tnum++) { + // if (bfs_state->is_top_down) { + // bfs_tasks.push_back(make_uniq(shared_from_this(), context, gstate, + // tnum * blocks, std::min(tnum * blocks + blocks, (idx_t)bfs_state->v_size))); + // } else { + // bfs_tasks.push_back(make_uniq(shared_from_this(), context, gstate, + // tnum * blocks, std::min(tnum * blocks + blocks, (idx_t)bfs_state->v_size))); + // } + // } SetTasks(std::move(bfs_tasks)); } @@ -385,6 +484,7 @@ class BFSIterativeEvent : public BasePipelineEvent { } // into the next iteration bfs_state->iter++; + auto bfs_event = std::make_shared(gstate, *pipeline); this->InsertEvent(std::dynamic_pointer_cast(bfs_event)); } else { @@ -426,7 +526,7 @@ PhysicalPathFinding::Finalize(Pipeline &pipeline, Event &event, } // debug print all_pairs->Print(); - gstate.global_bfs_state = make_uniq(all_pairs, csr->v_size); + gstate.global_bfs_state = make_uniq(all_pairs, csr->v_size - 2); // Schedule the first round of BFS tasks if (all_pairs->size() > 0) { @@ -468,6 +568,7 @@ void PhysicalPathFinding::ScheduleBFSTasks(Pipeline &pipeline, Event &event, Glo result_data[search_num] = (uint64_t)0; // path of length 0 does not require a search } else { + bfs_state->frontier_size++; bfs_state->visit1[bfs_state->src[src_pos]][lane] = true; bfs_state->lane_to_num[lane] = search_num; // active lane break; From 5ab5e9a3c80d8fdab38263ff02aa3e6259878032 Mon Sep 17 00:00:00 2001 From: Pingan Ren Date: Wed, 17 Apr 2024 00:42:59 +0200 Subject: [PATCH 076/249] work stealing --- .../physical_path_finding_operator.cpp | 142 +++++++++++++----- 1 file changed, 106 insertions(+), 36 deletions(-) diff --git a/src/operators/physical_path_finding_operator.cpp b/src/operators/physical_path_finding_operator.cpp index 03c9495d..fb8c2df2 100644 --- a/src/operators/physical_path_finding_operator.cpp +++ b/src/operators/physical_path_finding_operator.cpp @@ -124,6 +124,7 @@ void PathFindingLocalState::CreateCSR(DataChunk &input, class Barrier { public: void Init(std::size_t iCount) { + std::unique_lock lLock{mMutex}; mThreshold = iCount; mCount = iCount; mGeneration = 0; @@ -137,7 +138,19 @@ class Barrier { mCount = mThreshold; mCond.notify_all(); } else { - mCond.wait(lLock, [this, lGen] { return lGen != mGeneration; }); + mCond.wait(lLock, [this, lGen] { return lGen != mGeneration || mCount == mThreshold; }); + } + } + + void DecreaseCount() { + std::unique_lock lLock{mMutex}; + mCount--; + mThreshold--; + + if (mCount == 0) { + mGeneration++; + mCount = mThreshold; + mCond.notify_all(); } } @@ -152,11 +165,11 @@ class Barrier { class GlobalBFSState { public: - GlobalBFSState(shared_ptr pairs_, int64_t v_size_) + GlobalBFSState(shared_ptr pairs_, int64_t v_size_, idx_t num_threads_) : pairs(pairs_), iter(1), v_size(v_size_), change(false), started_searches(0), seen(v_size_), visit1(v_size_), visit2(v_size_), result(make_uniq(LogicalType::BIGINT, true, true, pairs_->size())), - frontier_size(0), unseen_size(v_size_) { + frontier_size(0), unseen_size(v_size_), num_threads(num_threads_), task_queues(num_threads_) { for (auto i = 0; i < LANE_LIMIT; i++) { lane_to_num[i] = -1; } @@ -208,6 +221,12 @@ class GlobalBFSState { constexpr static int64_t beta = 64; bool is_top_down = true; + idx_t num_threads; + // task_queues[workerId] = {curTaskIx, queuedTasks} + // queuedTasks[curTaskIx] = {start, end} + vector, vector>>> task_queues; + constexpr static int64_t split_size = 1; + Barrier barrier; // lock for next @@ -295,8 +314,8 @@ PhysicalPathFinding::Combine(ExecutionContext &context, class PhysicalBFSTopDownTask : public ExecutorTask { public: - PhysicalBFSTopDownTask(shared_ptr event_p, ClientContext &context, PathFindingGlobalState &state, idx_t start, idx_t end) - : ExecutorTask(context, std::move(event_p)), context(context), state(state), start(start), end(end) { + PhysicalBFSTopDownTask(shared_ptr event_p, ClientContext &context, PathFindingGlobalState &state, idx_t worker_id) + : ExecutorTask(context, std::move(event_p)), context(context), state(state), worker_id(worker_id) { } TaskExecutionResult ExecuteTask(TaskExecutionMode mode) override { @@ -312,32 +331,42 @@ class PhysicalBFSTopDownTask : public ExecutorTask { int64_t *v = (int64_t *)state.global_csr->v; vector &e = state.global_csr->e; - for (auto i = start; i < end; i++) { - next[i] = 0; - } + while (true) { + auto task = fetch_task(); + if (task.first == task.second) { + barrier.DecreaseCount(); + break; + } + auto start = task.first; + auto end = task.second; - barrier.Wait(); + for (auto i = start; i < end; i++) { + next[i] = 0; + } - for (auto i = start; i < end; i++) { - if (visit[i].any()) { - for (auto offset = v[i]; offset < v[i + 1]; offset++) { - auto n = e[offset]; - lock_guard lock(bfs_state->lock); - next[n] = next[n] | visit[i]; + barrier.Wait(); + + for (auto i = start; i < end; i++) { + if (visit[i].any()) { + for (auto offset = v[i]; offset < v[i + 1]; offset++) { + auto n = e[offset]; + lock_guard lock(bfs_state->lock); + next[n] = next[n] | visit[i]; + } } } - } - barrier.Wait(); + barrier.Wait(); - for (auto i = start; i < end; i++) { - if (next[i].any()) { - next[i] = next[i] & ~seen[i]; - seen[i] = seen[i] | next[i]; - change |= next[i].any(); + for (auto i = start; i < end; i++) { + if (next[i].any()) { + next[i] = next[i] & ~seen[i]; + seen[i] = seen[i] | next[i]; + change |= next[i].any(); - frontier_size = next[i].any() ? frontier_size + 1 : frontier_size.load(); - unseen_size = seen[i].all() ? unseen_size - 1 : unseen_size.load(); + frontier_size = next[i].any() ? frontier_size + 1 : frontier_size.load(); + unseen_size = seen[i].all() ? unseen_size - 1 : unseen_size.load(); + } } } @@ -345,12 +374,29 @@ class PhysicalBFSTopDownTask : public ExecutorTask { return TaskExecutionResult::TASK_FINISHED; } +private: + pair fetch_task() { + auto& task_queue = state.global_bfs_state->task_queues; + idx_t offset = 0; + do { + auto worker_idx = (worker_id + offset) % task_queue.size(); + auto cur_task_ix = task_queue[worker_idx].first.fetch_add(1); + if (cur_task_ix < task_queue[worker_idx].second.size()) { + return task_queue[worker_idx].second[cur_task_ix]; + } else { + offset++; + } + } while (offset < task_queue.size()); + return {0, 0}; + } + private: ClientContext &context; PathFindingGlobalState &state; - // [start, end) - idx_t start; - idx_t end; + // // [start, end) + // idx_t start; + // idx_t end; + idx_t worker_id; }; class PhysicalBFSBottomUpTask : public ExecutorTask { @@ -415,6 +461,27 @@ class BFSIterativeEvent : public BasePipelineEvent { PathFindingGlobalState &gstate; +private: + void CreateTasks() { + auto &bfs_state = gstate.global_bfs_state; + + // workerTasks[workerId] = [task1, task2, ...] + vector>> worker_tasks(bfs_state->num_threads); + auto cur_worker = 0; + + for (auto offset = 0; offset < bfs_state->v_size; offset += bfs_state->split_size) { + auto worker_id = cur_worker % bfs_state->num_threads; + pair range = {offset, std::min(offset + bfs_state->split_size, bfs_state->v_size)}; + worker_tasks[worker_id].push_back(range); + cur_worker++; + } + + for (idx_t worker_id = 0; worker_id < bfs_state->num_threads; worker_id++) { + bfs_state->task_queues[worker_id].first.store(0); + bfs_state->task_queues[worker_id].second = worker_tasks[worker_id]; + } + } + public: void Schedule() override { auto &bfs_state = gstate.global_bfs_state; @@ -438,18 +505,18 @@ class BFSIterativeEvent : public BasePipelineEvent { bfs_state->frontier_size = 0; bfs_state->unseen_size = bfs_state->v_size; - // Schedule tasks equal to the number of threads, which will each merge multiple partitions - auto &ts = TaskScheduler::GetScheduler(context); + // auto &ts = TaskScheduler::GetScheduler(context); // idx_t num_threads = ts.NumberOfThreads(); - idx_t num_threads = std::min((int64_t)ts.NumberOfThreads(), bfs_state->v_size); - idx_t blocks = floor(bfs_state->v_size / (float)num_threads); + // idx_t num_threads = std::min((int64_t)ts.NumberOfThreads(), bfs_state->v_size); + // idx_t blocks = floor(bfs_state->v_size / (float)num_threads); - bfs_state->barrier.Init(num_threads); + bfs_state->barrier.Init(bfs_state->num_threads); + + CreateTasks(); vector> bfs_tasks; - for (idx_t tnum = 0; tnum < num_threads; tnum++) { - bfs_tasks.push_back(make_uniq(shared_from_this(), context, gstate, - tnum * blocks, std::min(tnum * blocks + blocks, (idx_t)bfs_state->v_size))); + for (idx_t tnum = 0; tnum < bfs_state->num_threads; tnum++) { + bfs_tasks.push_back(make_uniq(shared_from_this(), context, gstate, tnum)); } // for (idx_t tnum = 0; tnum < num_threads; tnum++) { // if (bfs_state->is_top_down) { @@ -526,7 +593,10 @@ PhysicalPathFinding::Finalize(Pipeline &pipeline, Event &event, } // debug print all_pairs->Print(); - gstate.global_bfs_state = make_uniq(all_pairs, csr->v_size - 2); + + auto &ts = TaskScheduler::GetScheduler(context); + idx_t num_threads = ts.NumberOfThreads(); + gstate.global_bfs_state = make_uniq(all_pairs, csr->v_size - 2, num_threads); // Schedule the first round of BFS tasks if (all_pairs->size() > 0) { From c99f28941ef4ad92e94fc307539c6639a7ca5573 Mon Sep 17 00:00:00 2001 From: Pingan Ren Date: Thu, 18 Apr 2024 20:18:18 +0200 Subject: [PATCH 077/249] fix parallel bug --- .../physical_path_finding_operator.cpp | 37 ++++--- .../path_finding/parallel_path_finding.test | 99 ++++++++++++++++--- 2 files changed, 106 insertions(+), 30 deletions(-) diff --git a/src/operators/physical_path_finding_operator.cpp b/src/operators/physical_path_finding_operator.cpp index fb8c2df2..3a928539 100644 --- a/src/operators/physical_path_finding_operator.cpp +++ b/src/operators/physical_path_finding_operator.cpp @@ -327,25 +327,30 @@ class PhysicalBFSTopDownTask : public ExecutorTask { auto& barrier = bfs_state->barrier; auto& frontier_size = bfs_state->frontier_size; auto& unseen_size = bfs_state->unseen_size; + auto& v_size = bfs_state->v_size; int64_t *v = (int64_t *)state.global_csr->v; vector &e = state.global_csr->e; + // clear next before each iteration + idx_t block_size = ceil((double)v_size / bfs_state->num_threads); + block_size = block_size == 0 ? 1 : block_size; + auto left = block_size * worker_id; + auto right = std::min(block_size * (worker_id + 1), (idx_t)v_size); + for (auto i = left; i < right; i++) { + next[i] = 0; + } + + barrier.Wait(); + while (true) { auto task = fetch_task(); if (task.first == task.second) { - barrier.DecreaseCount(); break; } auto start = task.first; auto end = task.second; - for (auto i = start; i < end; i++) { - next[i] = 0; - } - - barrier.Wait(); - for (auto i = start; i < end; i++) { if (visit[i].any()) { for (auto offset = v[i]; offset < v[i + 1]; offset++) { @@ -355,18 +360,18 @@ class PhysicalBFSTopDownTask : public ExecutorTask { } } } + } - barrier.Wait(); + barrier.Wait(); - for (auto i = start; i < end; i++) { - if (next[i].any()) { - next[i] = next[i] & ~seen[i]; - seen[i] = seen[i] | next[i]; - change |= next[i].any(); + for (auto i = left; i < right; i++) { + if (next[i].any()) { + next[i] = next[i] & ~seen[i]; + seen[i] = seen[i] | next[i]; + change |= next[i].any(); - frontier_size = next[i].any() ? frontier_size + 1 : frontier_size.load(); - unseen_size = seen[i].all() ? unseen_size - 1 : unseen_size.load(); - } + frontier_size = next[i].any() ? frontier_size + 1 : frontier_size.load(); + unseen_size = seen[i].all() ? unseen_size - 1 : unseen_size.load(); } } diff --git a/test/sql/path_finding/parallel_path_finding.test b/test/sql/path_finding/parallel_path_finding.test index 378d7bab..a28b6991 100644 --- a/test/sql/path_finding/parallel_path_finding.test +++ b/test/sql/path_finding/parallel_path_finding.test @@ -8,13 +8,13 @@ require duckpgq statement ok -CREATE TABLE pairs(src BIGINT, dst BIGINT); INSERT INTO pairs(src, dst) VALUES (0, 1), (1, 2), (2,0); +CREATE TABLE pair(src BIGINT, dst BIGINT); INSERT INTO pair(src, dst) VALUES (0, 1), (1, 2), (2,0); statement ok create table student(id INT); INSERT INTO student(id) VALUES (10), (20), (30), (40); statement ok -create table knows(src INT, dst INT); INSERT INTO knows(src, dst) VALUES (40, 20), (10,30), (10,10), (20,10), (20,30), (30,10); +create table know(src INT, dst INT); INSERT INTO know(src, dst) VALUES (40, 20), (10,30), (10,10), (20,10), (30,10); #query II #SELECT * @@ -28,35 +28,106 @@ VERTEX TABLES ( student ) EDGE TABLES ( -knows SOURCE KEY (src) REFERENCES student (id) +know SOURCE KEY (src) REFERENCES student (id) DESTINATION KEY (dst) references student (id) ); -#statement ok -#-FROM GRAPH_TABLE(pg +# statement ok +# -FROM GRAPH_TABLE(pg # MATCH (f:student)-[k:knows]->{0,3}(f2:student) # COLUMNS (*) # ); +# query II +# SELECT * +# FROM pair AS p +# WHERE p.src BETWEEN (SELECT CREATE_CSR_EDGE( +# (SELECT count(a.id) as v_size FROM Student a), +# (SELECT count(k.src) as e_size from know k), +# a.rowid, +# c.rowid, +# k.rowid, +# t.cnt) FROM Know k +# JOIN student a on a.id = k.src +# JOIN student c on c.id = k.dst +# JOIN (SELECT count(k.src) cnt, a.rowid as a_rowid +# FROM student a +# LEFT JOIN know k ON k.src = a.id +# GROUP BY a.rowid) t +# ON t.a_rowid = a.rowid) AND p.dst; +# ---- + +statement ok +import database 'duckdb-pgq/data/SNB0.003'; + +statement ok +CREATE TABLE pairs2(src BIGINT, dst BIGINT); + +statement ok +INSERT INTO pairs2 +SELECT a.rowid, b.rowid +FROM Person a CROSS JOIN Person b +WHERE a.rowid = 1; + +statement ok +-CREATE PROPERTY GRAPH snb +VERTEX TABLES ( + Person PROPERTIES (id) LABEL Person + ) +EDGE TABLES ( + Person_knows_Person SOURCE KEY (Person1Id) REFERENCES Person (id) + DESTINATION KEY (Person2Id) REFERENCES Person (id) + LABEL Knows + ); + query II SELECT * -FROM pairs AS p +FROM pairs2 AS p WHERE p.src BETWEEN (SELECT CREATE_CSR_EDGE( - (SELECT count(a.id) as v_size FROM Student a), - (SELECT count(k.src) as e_size from knows k), + (SELECT count(a.id) as v_size FROM Person a), + (SELECT count(k.Person1Id) as e_size FROM Person_knows_Person k), a.rowid, c.rowid, k.rowid, - t.cnt) FROM Knows k - JOIN student a on a.id = k.src - JOIN student c on c.id = k.dst - JOIN (SELECT count(k.src) cnt, a.rowid as a_rowid - FROM student a - LEFT JOIN knows k ON k.src = a.id + t.cnt) FROM Person_knows_Person k + JOIN Person a on a.id = k.Person1Id + JOIN Person c on c.id = k.Person2Id + JOIN (SELECT count(k.Person1Id) cnt, a.rowid as a_rowid + FROM Person a + LEFT JOIN Person_knows_Person k ON k.Person1Id = a.id GROUP BY a.rowid) t ON t.a_rowid = a.rowid) AND p.dst; ---- +query III +WITH cte1 AS ( + SELECT CREATE_CSR_EDGE( + 0, + (SELECT count(a.id) FROM Person a), + CAST ( + (SELECT sum(CREATE_CSR_VERTEX( + 0, + (SELECT count(a.id) FROM Person a), + sub.dense_id, + sub.cnt) + ) + FROM ( + SELECT a.rowid as dense_id, count(k.Person1Id) as cnt + FROM Person a + LEFT JOIN Person_knows_Person k ON k.Person1Id = a.id + GROUP BY a.rowid) sub + ) + AS BIGINT), + a.rowid, + c.rowid, + k.rowid) as temp + FROM Person_knows_Person k + JOIN Person a on a.id = k.Person1Id + JOIN Person c on c.id = k.Person2Id +) SELECT a.rowid, b.rowid, iterativelength(0, (select count(*) from Person), a.rowid, b.rowid) as length + FROM Person a, Person b, (select count(cte1.temp) * 0 as temp from cte1) __x + WHERE a.rowid = 1 AND __x.temp * 0 + iterativelength(0, (select count(*) from Person), a.rowid, b.rowid) >= 0; +---- # CAST ( # (SELECT sum(CREATE_CSR_VERTEX(0, From 1cc6f791f8e29bfb66bdb6c8adde8213c6118f47 Mon Sep 17 00:00:00 2001 From: Pingan Ren Date: Fri, 19 Apr 2024 01:36:27 +0200 Subject: [PATCH 078/249] disable debug info --- src/operators/physical_path_finding_operator.cpp | 4 ++-- test/sql/path_finding/parallel_path_finding.test | 2 +- 2 files changed, 3 insertions(+), 3 deletions(-) diff --git a/src/operators/physical_path_finding_operator.cpp b/src/operators/physical_path_finding_operator.cpp index 3a928539..0d463307 100644 --- a/src/operators/physical_path_finding_operator.cpp +++ b/src/operators/physical_path_finding_operator.cpp @@ -597,7 +597,7 @@ PhysicalPathFinding::Finalize(Pipeline &pipeline, Event &event, all_pairs->Append(pairs, true); } // debug print - all_pairs->Print(); + // all_pairs->Print(); auto &ts = TaskScheduler::GetScheduler(context); idx_t num_threads = ts.NumberOfThreads(); @@ -727,7 +727,7 @@ PhysicalPathFinding::GetData(ExecutionContext &context, DataChunk &result, auto &pf_gstate = input.global_state.Cast(); auto &pf_bfs_state = pf_sink.global_bfs_state; // auto &pf_lstate = input.local_state.Cast(); - pf_bfs_state->result->Print(pf_bfs_state->pairs->size()); + // pf_bfs_state->result->Print(pf_bfs_state->pairs->size()); // pf_gstate.Initialize(pf_sink); return result.size() == 0 ? SourceResultType::FINISHED diff --git a/test/sql/path_finding/parallel_path_finding.test b/test/sql/path_finding/parallel_path_finding.test index a28b6991..0928b308 100644 --- a/test/sql/path_finding/parallel_path_finding.test +++ b/test/sql/path_finding/parallel_path_finding.test @@ -126,7 +126,7 @@ WITH cte1 AS ( JOIN Person c on c.id = k.Person2Id ) SELECT a.rowid, b.rowid, iterativelength(0, (select count(*) from Person), a.rowid, b.rowid) as length FROM Person a, Person b, (select count(cte1.temp) * 0 as temp from cte1) __x - WHERE a.rowid = 1 AND __x.temp * 0 + iterativelength(0, (select count(*) from Person), a.rowid, b.rowid) >= 0; + WHERE a.rowid = 1 AND __x.temp * 0 >= 0; ---- # CAST ( From 52afb357273a9aa1c5c7c3ccf9685f4eb92ff652 Mon Sep 17 00:00:00 2001 From: Pingan Ren Date: Mon, 22 Apr 2024 14:30:24 +0200 Subject: [PATCH 079/249] update csr creation --- .../physical_path_finding_operator.hpp | 2 + .../physical_path_finding_operator.cpp | 37 +++++++++++-------- .../path_finding/parallel_path_finding.test | 28 +++++++++----- 3 files changed, 43 insertions(+), 24 deletions(-) diff --git a/src/include/duckpgq/operators/physical_path_finding_operator.hpp b/src/include/duckpgq/operators/physical_path_finding_operator.hpp index 98e50a44..d28262e1 100644 --- a/src/include/duckpgq/operators/physical_path_finding_operator.hpp +++ b/src/include/duckpgq/operators/physical_path_finding_operator.hpp @@ -40,6 +40,8 @@ class PhysicalPathFinding : public PhysicalComparisonJoin { size_t v_size; bool is_ready = false; + Vector result = Vector(LogicalType::BIGINT); + std::mutex csr_lock; public: void InitializeVertex(int64_t v_size); diff --git a/src/operators/physical_path_finding_operator.cpp b/src/operators/physical_path_finding_operator.cpp index 0d463307..be96e761 100644 --- a/src/operators/physical_path_finding_operator.cpp +++ b/src/operators/physical_path_finding_operator.cpp @@ -46,7 +46,10 @@ void PhysicalPathFinding::GlobalCompressedSparseRow::InitializeVertex(int64_t v_ } void PhysicalPathFinding::GlobalCompressedSparseRow::InitializeEdge( int64_t e_size) { - const lock_guard csr_init_lock(csr_lock); + lock_guard csr_init_lock(csr_lock); + if (initialized_e) { + return; + } try { e.resize(e_size, 0); reverse_e.resize(e_size, 0); @@ -72,16 +75,17 @@ class PathFindingLocalState : public LocalSinkState { const idx_t child) : local_tasks(context, op.children[0]->GetTypes()) { } - void Sink(DataChunk &input, GlobalCompressedSparseRow &global_csr) { - if (global_csr.is_ready) { + void Sink(DataChunk &input, GlobalCompressedSparseRow &global_csr, idx_t child) { + if (child == 1) { // Add the tasks (src, dst) to sink // Optimizations: Eliminate duplicate sources/destinations // input.Print(); local_tasks.Append(input); // local_tasks.Print(); - return; + } else { + // Create CSR + CreateCSR(input, global_csr); } - CreateCSR(input, global_csr); } static void CreateCSR(DataChunk &input, @@ -92,13 +96,15 @@ class PathFindingLocalState : public LocalSinkState { void PathFindingLocalState::CreateCSR(DataChunk &input, GlobalCompressedSparseRow &global_csr) { + // debug print + // input.Print(); if (!global_csr.initialized_v) { const auto v_size = input.data[8].GetValue(0).GetValue(); global_csr.InitializeVertex(v_size); } - auto result = Vector(LogicalTypeId::BIGINT); + // auto result = Vector(LogicalTypeId::BIGINT); BinaryExecutor::Execute( - input.data[6], input.data[5], result, input.size(), + input.data[6], input.data[5], global_csr.result, input.size(), [&](const int64_t src, const int64_t cnt) { int64_t edge_count = 0; global_csr.v[src + 2] = cnt; @@ -110,15 +116,16 @@ void PathFindingLocalState::CreateCSR(DataChunk &input, const auto e_size = input.data[7].GetValue(0).GetValue(); global_csr.InitializeEdge(e_size); } + global_csr.Print(); // Debug print TernaryExecutor::Execute( - input.data[6], input.data[4], input.data[2], result, input.size(), + input.data[6], input.data[4], input.data[2], global_csr.result, input.size(), [&](int64_t src, int64_t dst, int64_t edge_id) { const auto pos = ++global_csr.v[src + 1]; global_csr.e[static_cast(pos) - 1] = dst; global_csr.edge_ids[static_cast(pos) - 1] = edge_id; return 1; }); - // global_csr.Print(); // Debug print + global_csr.Print(); // Debug print } class Barrier { @@ -225,7 +232,7 @@ class GlobalBFSState { // task_queues[workerId] = {curTaskIx, queuedTasks} // queuedTasks[curTaskIx] = {start, end} vector, vector>>> task_queues; - constexpr static int64_t split_size = 1; + constexpr static int64_t split_size = 256; Barrier barrier; @@ -237,7 +244,7 @@ class PathFindingGlobalState : public GlobalSinkState { public: using GlobalCompressedSparseRow = PhysicalPathFinding::GlobalCompressedSparseRow; PathFindingGlobalState(ClientContext &context, - const PhysicalPathFinding &op) : global_tasks(context, op.children[0]->GetTypes()) { + const PhysicalPathFinding &op) : global_tasks(context, op.children[0]->GetTypes()), child(0) { RowLayout lhs_layout; lhs_layout.Initialize(op.children[0]->types); RowLayout rhs_layout; @@ -252,7 +259,7 @@ class PathFindingGlobalState : public GlobalSinkState { } void Sink(DataChunk &input, PathFindingLocalState &lstate) const { - lstate.Sink(input, *global_csr); + lstate.Sink(input, *global_csr, child); } // pairs is a 2-column table with src and dst @@ -291,7 +298,7 @@ SinkResultType PhysicalPathFinding::Sink(ExecutionContext &context, auto &gstate = input.global_state.Cast(); auto &lstate = input.local_state.Cast(); gstate.Sink(chunk, lstate); - gstate.global_csr->is_ready = true; + // gstate.global_csr->is_ready = true; return SinkResultType::NEED_MORE_INPUT; } @@ -597,7 +604,7 @@ PhysicalPathFinding::Finalize(Pipeline &pipeline, Event &event, all_pairs->Append(pairs, true); } // debug print - // all_pairs->Print(); + all_pairs->Print(); auto &ts = TaskScheduler::GetScheduler(context); idx_t num_threads = ts.NumberOfThreads(); @@ -727,7 +734,7 @@ PhysicalPathFinding::GetData(ExecutionContext &context, DataChunk &result, auto &pf_gstate = input.global_state.Cast(); auto &pf_bfs_state = pf_sink.global_bfs_state; // auto &pf_lstate = input.local_state.Cast(); - // pf_bfs_state->result->Print(pf_bfs_state->pairs->size()); + pf_bfs_state->result->Print(pf_bfs_state->pairs->size()); // pf_gstate.Initialize(pf_sink); return result.size() == 0 ? SourceResultType::FINISHED diff --git a/test/sql/path_finding/parallel_path_finding.test b/test/sql/path_finding/parallel_path_finding.test index 0928b308..bc4fcbce 100644 --- a/test/sql/path_finding/parallel_path_finding.test +++ b/test/sql/path_finding/parallel_path_finding.test @@ -59,15 +59,25 @@ know SOURCE KEY (src) REFERENCES student (id) statement ok import database 'duckdb-pgq/data/SNB0.003'; +# import database 'duckdb-pgq/data/SNB1-projected|'; + + +# statement ok +# DELETE FROM Person +# WHERE rowid NOT IN ( +# SELECT rowid FROM Person +# ORDER BY rowid ASC +# LIMIT 500 +# ); -statement ok -CREATE TABLE pairs2(src BIGINT, dst BIGINT); statement ok -INSERT INTO pairs2 -SELECT a.rowid, b.rowid -FROM Person a CROSS JOIN Person b -WHERE a.rowid = 1; +CREATE OR REPLACE TABLE pairs2 AS ( + SELECT src, dst + FROM (SELECT a.rowid AS src FROM Person a), + (SELECT b.rowid AS dst FROM Person b) + USING SAMPLE reservoir(64 ROWS) REPEATABLE (300) +); statement ok -CREATE PROPERTY GRAPH snb @@ -124,9 +134,9 @@ WITH cte1 AS ( FROM Person_knows_Person k JOIN Person a on a.id = k.Person1Id JOIN Person c on c.id = k.Person2Id -) SELECT a.rowid, b.rowid, iterativelength(0, (select count(*) from Person), a.rowid, b.rowid) as length - FROM Person a, Person b, (select count(cte1.temp) * 0 as temp from cte1) __x - WHERE a.rowid = 1 AND __x.temp * 0 >= 0; +) SELECT p.src, p.dst, iterativelength(0, (select count(*) from Person), p.src, p.dst) as length + FROM pairs2 p, (select count(cte1.temp) * 0 as temp from cte1) __x + WHERE __x.temp * 0 >= 0; ---- # CAST ( From 056fc3d6257284be74cf8615a058425d603af5e7 Mon Sep 17 00:00:00 2001 From: dtenwolde Date: Tue, 23 Apr 2024 15:56:40 +0200 Subject: [PATCH 080/249] Add test with iterativelength as extra column --- .../path_finding/parallel_path_finding.test | 42 ++++++++----------- 1 file changed, 18 insertions(+), 24 deletions(-) diff --git a/test/sql/path_finding/parallel_path_finding.test b/test/sql/path_finding/parallel_path_finding.test index bc4fcbce..7aabbaab 100644 --- a/test/sql/path_finding/parallel_path_finding.test +++ b/test/sql/path_finding/parallel_path_finding.test @@ -16,12 +16,6 @@ create table student(id INT); INSERT INTO student(id) VALUES (10), (20), (30), ( statement ok create table know(src INT, dst INT); INSERT INTO know(src, dst) VALUES (40, 20), (10,30), (10,10), (20,10), (30,10); -#query II -#SELECT * -#FROM knows AS a, pairs AS b -#WHERE (cos(a.src) BETWEEN cos(b.dst) AND cos(b.src)); -#---- - statement ok -CREATE PROPERTY GRAPH pg VERTEX TABLES ( @@ -38,24 +32,24 @@ know SOURCE KEY (src) REFERENCES student (id) # COLUMNS (*) # ); -# query II -# SELECT * -# FROM pair AS p -# WHERE p.src BETWEEN (SELECT CREATE_CSR_EDGE( -# (SELECT count(a.id) as v_size FROM Student a), -# (SELECT count(k.src) as e_size from know k), -# a.rowid, -# c.rowid, -# k.rowid, -# t.cnt) FROM Know k -# JOIN student a on a.id = k.src -# JOIN student c on c.id = k.dst -# JOIN (SELECT count(k.src) cnt, a.rowid as a_rowid -# FROM student a -# LEFT JOIN know k ON k.src = a.id -# GROUP BY a.rowid) t -# ON t.a_rowid = a.rowid) AND p.dst; -# ---- + query II + SELECT *, iterativelength(src, dst) as length + FROM pair AS p + WHERE p.src BETWEEN (SELECT CREATE_CSR_EDGE( + (SELECT count(a.id) as v_size FROM Student a), + (SELECT count(k.src) as e_size from know k), + a.rowid, + c.rowid, + k.rowid, + t.cnt) FROM Know k + JOIN student a on a.id = k.src + JOIN student c on c.id = k.dst + JOIN (SELECT count(k.src) cnt, a.rowid as a_rowid + FROM student a + LEFT JOIN know k ON k.src = a.id + GROUP BY a.rowid) t + ON t.a_rowid = a.rowid) AND p.dst; +---- statement ok import database 'duckdb-pgq/data/SNB0.003'; From b6671db0c12491b80c7641c19a2d71963a2f2074 Mon Sep 17 00:00:00 2001 From: dtenwolde Date: Tue, 23 Apr 2024 15:56:57 +0200 Subject: [PATCH 081/249] Change iterativelength udf to also work with two columns --- src/common.cpp | 3 +++ src/functions/scalar/iterativelength.cpp | 13 ++++++++++--- 2 files changed, 13 insertions(+), 3 deletions(-) diff --git a/src/common.cpp b/src/common.cpp index 3c2fe74d..bc4502fc 100644 --- a/src/common.cpp +++ b/src/common.cpp @@ -83,6 +83,9 @@ bool IterativeLengthFunctionData::Equals(const FunctionData &other_p) const { unique_ptr IterativeLengthFunctionData::IterativeLengthBind( ClientContext &context, ScalarFunction &bound_function, vector> &arguments) { + if (arguments.size() == 2) { + return make_uniq(context, 0); + } if (!arguments[0]->IsFoldable()) { throw InvalidInputException("Id must be constant."); } diff --git a/src/functions/scalar/iterativelength.cpp b/src/functions/scalar/iterativelength.cpp index 61746d41..0372f9fb 100644 --- a/src/functions/scalar/iterativelength.cpp +++ b/src/functions/scalar/iterativelength.cpp @@ -158,12 +158,19 @@ static void IterativeLengthFunction(DataChunk &args, ExpressionState &state, } CreateScalarFunctionInfo DuckPGQFunctions::GetIterativeLengthFunction() { - auto fun = ScalarFunction("iterativelength", + ScalarFunctionSet set("iterativelength"); + + set.AddFunction(ScalarFunction("iterativelength", {LogicalType::INTEGER, LogicalType::BIGINT, LogicalType::BIGINT, LogicalType::BIGINT}, LogicalType::BIGINT, IterativeLengthFunction, - IterativeLengthFunctionData::IterativeLengthBind); - return CreateScalarFunctionInfo(fun); + IterativeLengthFunctionData::IterativeLengthBind)); + + set.AddFunction(ScalarFunction({LogicalType::BIGINT, LogicalType::BIGINT}, + LogicalType::BIGINT, IterativeLengthFunction, + IterativeLengthFunctionData::IterativeLengthBind)); + + return CreateScalarFunctionInfo(set); } } // namespace duckdb From ba56b4e263413599fdbaae6021577e01fd1f222c Mon Sep 17 00:00:00 2001 From: dtenwolde Date: Tue, 23 Apr 2024 15:57:48 +0200 Subject: [PATCH 082/249] Change the BoundFunctionExpression of iterativlength to a BoundColumnRef --- src/include/duckpgq_extension.hpp | 12 ++++++++++++ 1 file changed, 12 insertions(+) diff --git a/src/include/duckpgq_extension.hpp b/src/include/duckpgq_extension.hpp index b283a2d7..ad8f2fb6 100644 --- a/src/include/duckpgq_extension.hpp +++ b/src/include/duckpgq_extension.hpp @@ -63,6 +63,17 @@ class DuckpgqOptimizerExtension : public OptimizerExtension { make_uniq(path_finding_children, path_finding_expressions); op.children.clear(); op.children.push_back(std::move(path_finding_operator)); + + // Iterate in reverse to not influence the upcoming iterations when erasing an element from the list. + for (int64_t i = op.expressions.size() - 1; i >= 0; --i) { + const auto& expr = op.expressions[i]; + if (expr->expression_class == ExpressionClass::BOUND_FUNCTION && + expr->Cast().function.name == "iterativelength") { + op.expressions.emplace_back(make_uniq( + expr->alias, LogicalType::BIGINT, ColumnBinding(10, 0))); + op.expressions.erase(op.expressions.begin() + i); + } + } return true; } } @@ -77,6 +88,7 @@ class DuckpgqOptimizerExtension : public OptimizerExtension { static void DuckpgqOptimizeFunction(ClientContext &context, OptimizerExtensionInfo *info, duckdb::unique_ptr &plan) { + if (!InsertPathFindingOperator(*plan)) { return; } From 59ed8d03ca7409e7d4110ed45d9cb521c9f2fde8 Mon Sep 17 00:00:00 2001 From: dtenwolde Date: Tue, 23 Apr 2024 15:58:09 +0200 Subject: [PATCH 083/249] Add the extra ColumnBinding to the logical operator --- src/operators/logical_path_finding_operator.cpp | 8 ++++---- 1 file changed, 4 insertions(+), 4 deletions(-) diff --git a/src/operators/logical_path_finding_operator.cpp b/src/operators/logical_path_finding_operator.cpp index 3202405f..f38e7ec6 100644 --- a/src/operators/logical_path_finding_operator.cpp +++ b/src/operators/logical_path_finding_operator.cpp @@ -14,15 +14,15 @@ unique_ptr LogicalPathFindingOperator::CreatePlan( } vector LogicalPathFindingOperator::GetColumnBindings() { auto left_bindings = children[0]->GetColumnBindings(); - auto right_bindings = children[1]->GetColumnBindings(); - left_bindings.insert(left_bindings.end(), right_bindings.begin(), right_bindings.end()); + left_bindings.push_back(ColumnBinding(10, 0)); return left_bindings; } void LogicalPathFindingOperator::ResolveTypes() { types = children[0]->types; - auto right_types = children[1]->types; - types.insert(types.end(), right_types.begin(), right_types.end()); + types.push_back(LogicalType::BIGINT); + // auto right_types = children[1]->types; + // types.insert(types.end(), right_types.begin(), right_types.end()); // types = {LogicalType::BIGINT, LogicalType::BIGINT}; } From 369354336942c60d0222e3575991c2b6f0005b8e Mon Sep 17 00:00:00 2001 From: dtenwolde Date: Tue, 23 Apr 2024 15:58:58 +0200 Subject: [PATCH 084/249] Format fix Add result as a DataChunk which is fused to the final result DataChunk --- .../physical_path_finding_operator.cpp | 155 +++++++++--------- 1 file changed, 73 insertions(+), 82 deletions(-) diff --git a/src/operators/physical_path_finding_operator.cpp b/src/operators/physical_path_finding_operator.cpp index be96e761..a630cdf9 100644 --- a/src/operators/physical_path_finding_operator.cpp +++ b/src/operators/physical_path_finding_operator.cpp @@ -1,20 +1,18 @@ #include "duckpgq/operators/physical_path_finding_operator.hpp" #include "duckdb/common/sort/sort.hpp" +#include "duckdb/common/types/column/column_data_collection.hpp" #include "duckdb/execution/physical_operator.hpp" #include "duckdb/main/client_context.hpp" +#include "duckdb/parallel/base_pipeline_event.hpp" #include "duckdb/parallel/event.hpp" #include "duckdb/parallel/meta_pipeline.hpp" #include "duckdb/parallel/thread_context.hpp" -#include "duckdb/parallel/base_pipeline_event.hpp" -#include "duckdb/common/types/column/column_data_collection.hpp" -#include #include -#include +#include namespace duckdb { - PhysicalPathFinding::PhysicalPathFinding(LogicalExtensionOperator &op, unique_ptr left, unique_ptr right) @@ -24,7 +22,8 @@ PhysicalPathFinding::PhysicalPathFinding(LogicalExtensionOperator &op, expressions = std::move(op.expressions); } -void PhysicalPathFinding::GlobalCompressedSparseRow::InitializeVertex(int64_t v_size_) { +void PhysicalPathFinding::GlobalCompressedSparseRow::InitializeVertex( + int64_t v_size_) { lock_guard csr_init_lock(csr_lock); if (initialized_v) { @@ -35,8 +34,9 @@ void PhysicalPathFinding::GlobalCompressedSparseRow::InitializeVertex(int64_t v_ v = new std::atomic[v_size]; reverse_v = new std::atomic[v_size]; } catch (std::bad_alloc const &) { - throw InternalException("Unable to initialize vector of size for csr vertex table " - "representation"); + throw InternalException( + "Unable to initialize vector of size for csr vertex table " + "representation"); } for (idx_t i = 0; i < v_size; ++i) { v[i].store(0); @@ -46,10 +46,7 @@ void PhysicalPathFinding::GlobalCompressedSparseRow::InitializeVertex(int64_t v_ } void PhysicalPathFinding::GlobalCompressedSparseRow::InitializeEdge( int64_t e_size) { - lock_guard csr_init_lock(csr_lock); - if (initialized_e) { - return; - } + const lock_guard csr_init_lock(csr_lock); try { e.resize(e_size, 0); reverse_e.resize(e_size, 0); @@ -59,7 +56,7 @@ void PhysicalPathFinding::GlobalCompressedSparseRow::InitializeEdge( "edge table representation"); } for (idx_t i = 1; i < v_size; i++) { - v[i] += v[i-1]; + v[i] += v[i - 1]; } initialized_e = true; } @@ -112,6 +109,7 @@ void PathFindingLocalState::CreateCSR(DataChunk &input, return edge_count; }); + global_csr.Print(); if (!global_csr.initialized_e) { const auto e_size = input.data[7].GetValue(0).GetValue(); global_csr.InitializeEdge(e_size); @@ -172,11 +170,11 @@ class Barrier { class GlobalBFSState { public: - GlobalBFSState(shared_ptr pairs_, int64_t v_size_, idx_t num_threads_) + GlobalBFSState(shared_ptr pairs_, int64_t v_size_, idx_t num_threads_, ClientContext &context_) : pairs(pairs_), iter(1), v_size(v_size_), change(false), started_searches(0), - seen(v_size_), visit1(v_size_), visit2(v_size_), - result(make_uniq(LogicalType::BIGINT, true, true, pairs_->size())), + seen(v_size_), visit1(v_size_), visit2(v_size_), context(context_), frontier_size(0), unseen_size(v_size_), num_threads(num_threads_), task_queues(num_threads_) { + result.Initialize(context, {LogicalType::BIGINT}, STANDARD_VECTOR_SIZE); for (auto i = 0; i < LANE_LIMIT; i++) { lane_to_num[i] = -1; } @@ -206,6 +204,7 @@ class GlobalBFSState { visit1[i] = 0; } } + public: shared_ptr pairs; int64_t iter; @@ -220,7 +219,8 @@ class GlobalBFSState { vector> seen; vector> visit1; vector> visit2; - unique_ptr result; + DataChunk result; + ClientContext& context; atomic frontier_size; atomic unseen_size; @@ -244,26 +244,24 @@ class PathFindingGlobalState : public GlobalSinkState { public: using GlobalCompressedSparseRow = PhysicalPathFinding::GlobalCompressedSparseRow; PathFindingGlobalState(ClientContext &context, - const PhysicalPathFinding &op) : global_tasks(context, op.children[0]->GetTypes()), child(0) { - RowLayout lhs_layout; - lhs_layout.Initialize(op.children[0]->types); - RowLayout rhs_layout; - rhs_layout.Initialize(op.children[1]->types); - global_csr = make_uniq(context, rhs_layout); + const PhysicalPathFinding &op) { + global_tasks = make_uniq(context, op.children[0]->GetTypes()); + global_csr = make_uniq(context); + child = 0; } PathFindingGlobalState(PathFindingGlobalState &prev) - : GlobalSinkState(prev), global_tasks(prev.global_tasks), - global_csr(std::move(prev.global_csr)), - global_bfs_state(std::move(prev.global_bfs_state)), child(prev.child + 1) { - } + : GlobalSinkState(prev), global_tasks(std::move(prev.global_tasks)), + global_csr(std::move(prev.global_csr)), + global_bfs_state(std::move(prev.global_bfs_state)), + child(prev.child + 1) {} void Sink(DataChunk &input, PathFindingLocalState &lstate) const { lstate.Sink(input, *global_csr, child); } // pairs is a 2-column table with src and dst - ColumnDataCollection global_tasks; + unique_ptr global_tasks; // pairs with path exists // ColumnDataCollection global_results; ColumnDataScanState scan_state; @@ -286,8 +284,8 @@ unique_ptr PhysicalPathFinding::GetLocalSinkState(ExecutionContext &context) const { idx_t sink_child = 0; if (sink_state) { - const auto &pathfinding_sink = sink_state->Cast(); - sink_child = pathfinding_sink.child; + const auto &pathfinding_sink = sink_state->Cast(); + sink_child = pathfinding_sink.child; } return make_uniq(context.client, *this, sink_child); } @@ -298,7 +296,6 @@ SinkResultType PhysicalPathFinding::Sink(ExecutionContext &context, auto &gstate = input.global_state.Cast(); auto &lstate = input.local_state.Cast(); gstate.Sink(chunk, lstate); - // gstate.global_csr->is_ready = true; return SinkResultType::NEED_MORE_INPUT; } @@ -309,7 +306,7 @@ PhysicalPathFinding::Combine(ExecutionContext &context, auto &lstate = input.local_state.Cast(); auto &client_profiler = QueryProfiler::Get(context.client); - gstate.global_tasks.Combine(lstate.local_tasks); + gstate.global_tasks->Combine(lstate.local_tasks); client_profiler.Flush(context.thread.profiler); // gstate.global_tasks.Print(); return SinkCombineResultType::FINISHED; @@ -453,13 +450,13 @@ class PhysicalBFSBottomUpTask : public ExecutorTask { unseen_size = seen[i].all() ? unseen_size - 1 : unseen_size.load(); } - event->FinishTask(); - return TaskExecutionResult::TASK_FINISHED; - } + event->FinishTask(); + return TaskExecutionResult::TASK_FINISHED; + } private: - ClientContext &context; - PathFindingGlobalState &state; + ClientContext &context; + PathFindingGlobalState &state; // [start, end) idx_t start; idx_t end; @@ -467,11 +464,10 @@ class PhysicalBFSBottomUpTask : public ExecutorTask { class BFSIterativeEvent : public BasePipelineEvent { public: - BFSIterativeEvent(PathFindingGlobalState &gstate_p, Pipeline &pipeline_p) - : BasePipelineEvent(pipeline_p), gstate(gstate_p) { - } + BFSIterativeEvent(PathFindingGlobalState &gstate_p, Pipeline &pipeline_p) + : BasePipelineEvent(pipeline_p), gstate(gstate_p) {} - PathFindingGlobalState &gstate; + PathFindingGlobalState &gstate; private: void CreateTasks() { @@ -495,9 +491,9 @@ class BFSIterativeEvent : public BasePipelineEvent { } public: - void Schedule() override { + void Schedule() override { auto &bfs_state = gstate.global_bfs_state; - auto &context = pipeline->GetClientContext(); + auto &context = pipeline->GetClientContext(); bfs_state->change = false; @@ -532,10 +528,10 @@ class BFSIterativeEvent : public BasePipelineEvent { } // for (idx_t tnum = 0; tnum < num_threads; tnum++) { // if (bfs_state->is_top_down) { - // bfs_tasks.push_back(make_uniq(shared_from_this(), context, gstate, + // bfs_tasks.push_back(make_uniq(shared_from_this(), context, gstate, // tnum * blocks, std::min(tnum * blocks + blocks, (idx_t)bfs_state->v_size))); // } else { - // bfs_tasks.push_back(make_uniq(shared_from_this(), context, gstate, + // bfs_tasks.push_back(make_uniq(shared_from_this(), context, gstate, // tnum * blocks, std::min(tnum * blocks + blocks, (idx_t)bfs_state->v_size))); // } // } @@ -543,10 +539,10 @@ class BFSIterativeEvent : public BasePipelineEvent { } void FinishEvent() override { - auto& bfs_state = gstate.global_bfs_state; + auto &bfs_state = gstate.global_bfs_state; - auto result_data = FlatVector::GetData(*bfs_state->result); - ValidityMask &result_validity = FlatVector::Validity(*bfs_state->result); + auto result_data = FlatVector::GetData(bfs_state->result.data[0]); + ValidityMask &result_validity = FlatVector::Validity(bfs_state->result.data[0]); if (bfs_state->change) { // detect lanes that finished @@ -578,14 +574,13 @@ class BFSIterativeEvent : public BasePipelineEvent { } // if remaining pairs, schedule the BFS for the next batch - if (bfs_state->started_searches < gstate.global_tasks.Count()) { + if (bfs_state->started_searches < gstate.global_tasks->Count()) { PhysicalPathFinding::ScheduleBFSTasks(*pipeline, *this, gstate); } } - } + } }; - SinkFinalizeType PhysicalPathFinding::Finalize(Pipeline &pipeline, Event &event, ClientContext &context, @@ -593,14 +588,14 @@ PhysicalPathFinding::Finalize(Pipeline &pipeline, Event &event, auto &gstate = input.global_state.Cast(); auto &csr = gstate.global_csr; auto &global_tasks = gstate.global_tasks; - if (global_tasks.Count() != 0) { + if (global_tasks->Count() != 0) { auto all_pairs = make_shared(); DataChunk pairs; - global_tasks.InitializeScanChunk(*all_pairs); - global_tasks.InitializeScanChunk(pairs); + global_tasks->InitializeScanChunk(*all_pairs); + global_tasks->InitializeScanChunk(pairs); ColumnDataScanState scan_state; - global_tasks.InitializeScan(scan_state); - while (global_tasks.Scan(scan_state, pairs)) { + global_tasks->InitializeScan(scan_state); + while (global_tasks->Scan(scan_state, pairs)) { all_pairs->Append(pairs, true); } // debug print @@ -608,23 +603,22 @@ PhysicalPathFinding::Finalize(Pipeline &pipeline, Event &event, auto &ts = TaskScheduler::GetScheduler(context); idx_t num_threads = ts.NumberOfThreads(); - gstate.global_bfs_state = make_uniq(all_pairs, csr->v_size - 2, num_threads); + gstate.global_bfs_state = make_uniq(all_pairs, csr->v_size - 2, num_threads, context); // Schedule the first round of BFS tasks if (all_pairs->size() > 0) { ScheduleBFSTasks(pipeline, event, gstate); } - } - - // Move to the next input child - ++gstate.child; + // Move to the next input child + ++gstate.child; return SinkFinalizeType::READY; } -void PhysicalPathFinding::ScheduleBFSTasks(Pipeline &pipeline, Event &event, GlobalSinkState &state) { +void PhysicalPathFinding::ScheduleBFSTasks(Pipeline &pipeline, Event &event, + GlobalSinkState &state) { auto &gstate = state.Cast(); auto &bfs_state = gstate.global_bfs_state; @@ -632,14 +626,13 @@ void PhysicalPathFinding::ScheduleBFSTasks(Pipeline &pipeline, Event &event, Glo bfs_state->Clear(); // remaining pairs - if (bfs_state->started_searches < gstate.global_tasks.Count()) { - - auto result_data = FlatVector::GetData(*bfs_state->result); - auto& result_validity = FlatVector::Validity(*bfs_state->result); + if (bfs_state->started_searches < gstate.global_tasks->Count()) { + auto result_data = FlatVector::GetData(bfs_state->result.data[0]); + auto &result_validity = FlatVector::Validity(bfs_state->result.data[0]); for (int64_t lane = 0; lane < LANE_LIMIT; lane++) { bfs_state->lane_to_num[lane] = -1; - while (bfs_state->started_searches < gstate.global_tasks.Count()) { + while (bfs_state->started_searches < gstate.global_tasks->Count()) { int64_t search_num = bfs_state->started_searches++; int64_t src_pos = bfs_state->vdata_src.sel->get_index(search_num); int64_t dst_pos = bfs_state->vdata_dst.sel->get_index(search_num); @@ -659,11 +652,11 @@ void PhysicalPathFinding::ScheduleBFSTasks(Pipeline &pipeline, Event &event, Glo } auto bfs_event = make_shared(gstate, pipeline); - event.InsertEvent(std::move(std::dynamic_pointer_cast(bfs_event))); + event.InsertEvent( + std::move(std::dynamic_pointer_cast(bfs_event))); } } - //===--------------------------------------------------------------------===// // Operator //===--------------------------------------------------------------------===// @@ -681,8 +674,7 @@ class PathFindingLocalSourceState : public LocalSourceState { public: explicit PathFindingLocalSourceState(ClientContext &context, const PhysicalPathFinding &op) - : op(op){ - } + : op(op) {} const PhysicalPathFinding &op; }; @@ -702,13 +694,11 @@ class PathFindingGlobalSourceState : public GlobalSourceState { public: idx_t MaxThreads() override { - // const auto &sink_state = (op.sink_state->Cast()); - return 1; + return 1; // TODO } void GetNextPair(ClientContext &client, PathFindingGlobalState &gstate, - PathFindingLocalSourceState &lstate) { - } + PathFindingLocalSourceState &lstate) {} const PhysicalPathFinding &op; @@ -733,12 +723,14 @@ PhysicalPathFinding::GetData(ExecutionContext &context, DataChunk &result, auto &pf_sink = sink_state->Cast(); auto &pf_gstate = input.global_state.Cast(); auto &pf_bfs_state = pf_sink.global_bfs_state; - // auto &pf_lstate = input.local_state.Cast(); - pf_bfs_state->result->Print(pf_bfs_state->pairs->size()); - // pf_gstate.Initialize(pf_sink); - - return result.size() == 0 ? SourceResultType::FINISHED - : SourceResultType::HAVE_MORE_OUTPUT; + pf_bfs_state->result.Print(); + pf_bfs_state->result.SetCardinality(*pf_bfs_state->pairs); + result.Move(*pf_bfs_state->pairs); + result.Fuse(pf_bfs_state->result); + result.Print(); + return result.size() == 0 + ? SourceResultType::FINISHED + : SourceResultType::HAVE_MORE_OUTPUT; } //===--------------------------------------------------------------------===// @@ -773,4 +765,3 @@ void PhysicalPathFinding::BuildPipelines(Pipeline ¤t, } } // namespace duckdb - From 0048c4bbe0f6eb089676da5fe51c7711cb0e9318 Mon Sep 17 00:00:00 2001 From: dtenwolde Date: Tue, 23 Apr 2024 15:59:10 +0200 Subject: [PATCH 085/249] Remove comment Update submodule --- .../duckpgq/operators/physical_path_finding_operator.hpp | 5 +---- 1 file changed, 1 insertion(+), 4 deletions(-) diff --git a/src/include/duckpgq/operators/physical_path_finding_operator.hpp b/src/include/duckpgq/operators/physical_path_finding_operator.hpp index d28262e1..89da1178 100644 --- a/src/include/duckpgq/operators/physical_path_finding_operator.hpp +++ b/src/include/duckpgq/operators/physical_path_finding_operator.hpp @@ -21,11 +21,8 @@ class PhysicalPathFinding : public PhysicalComparisonJoin { public: class GlobalCompressedSparseRow { public: - GlobalCompressedSparseRow(ClientContext &context, - RowLayout &payload_layout){ - + GlobalCompressedSparseRow(ClientContext &context){ }; -// ~GlobalCompressedSparseRow() { delete[] v; } atomic *v; atomic *reverse_v; From fe66b50e040d9addc8d608d57522e7a2b1e39304 Mon Sep 17 00:00:00 2001 From: Pingan Ren Date: Tue, 23 Apr 2024 16:42:21 +0200 Subject: [PATCH 086/249] parallel edge creation --- .../physical_path_finding_operator.hpp | 2 - .../physical_path_finding_operator.cpp | 161 +++++++++++++++--- 2 files changed, 134 insertions(+), 29 deletions(-) diff --git a/src/include/duckpgq/operators/physical_path_finding_operator.hpp b/src/include/duckpgq/operators/physical_path_finding_operator.hpp index d28262e1..98e50a44 100644 --- a/src/include/duckpgq/operators/physical_path_finding_operator.hpp +++ b/src/include/duckpgq/operators/physical_path_finding_operator.hpp @@ -40,8 +40,6 @@ class PhysicalPathFinding : public PhysicalComparisonJoin { size_t v_size; bool is_ready = false; - Vector result = Vector(LogicalType::BIGINT); - std::mutex csr_lock; public: void InitializeVertex(int64_t v_size); diff --git a/src/operators/physical_path_finding_operator.cpp b/src/operators/physical_path_finding_operator.cpp index be96e761..a27b447d 100644 --- a/src/operators/physical_path_finding_operator.cpp +++ b/src/operators/physical_path_finding_operator.cpp @@ -72,7 +72,8 @@ class PathFindingLocalState : public LocalSinkState { using GlobalCompressedSparseRow = PhysicalPathFinding::GlobalCompressedSparseRow; PathFindingLocalState(ClientContext &context, const PhysicalPathFinding &op, - const idx_t child) : local_tasks(context, op.children[0]->GetTypes()) { + const idx_t child) : local_tasks(context, op.children[0]->GetTypes()), + local_inputs(context, op.children[1]->GetTypes()) { } void Sink(DataChunk &input, GlobalCompressedSparseRow &global_csr, idx_t child) { @@ -84,17 +85,19 @@ class PathFindingLocalState : public LocalSinkState { // local_tasks.Print(); } else { // Create CSR - CreateCSR(input, global_csr); + local_inputs.Append(input); + CreateCSRVertex(input, global_csr); } } - static void CreateCSR(DataChunk &input, + static void CreateCSRVertex(DataChunk &input, GlobalCompressedSparseRow &global_csr); ColumnDataCollection local_tasks; + ColumnDataCollection local_inputs; }; -void PathFindingLocalState::CreateCSR(DataChunk &input, +void PathFindingLocalState::CreateCSRVertex(DataChunk &input, GlobalCompressedSparseRow &global_csr) { // debug print // input.Print(); @@ -102,30 +105,28 @@ void PathFindingLocalState::CreateCSR(DataChunk &input, const auto v_size = input.data[8].GetValue(0).GetValue(); global_csr.InitializeVertex(v_size); } - // auto result = Vector(LogicalTypeId::BIGINT); + auto result = Vector(LogicalTypeId::BIGINT); BinaryExecutor::Execute( - input.data[6], input.data[5], global_csr.result, input.size(), + input.data[6], input.data[5], result, input.size(), [&](const int64_t src, const int64_t cnt) { int64_t edge_count = 0; global_csr.v[src + 2] = cnt; edge_count = edge_count + cnt; return edge_count; }); - - if (!global_csr.initialized_e) { - const auto e_size = input.data[7].GetValue(0).GetValue(); - global_csr.InitializeEdge(e_size); - } - global_csr.Print(); // Debug print - TernaryExecutor::Execute( - input.data[6], input.data[4], input.data[2], global_csr.result, input.size(), - [&](int64_t src, int64_t dst, int64_t edge_id) { - const auto pos = ++global_csr.v[src + 1]; - global_csr.e[static_cast(pos) - 1] = dst; - global_csr.edge_ids[static_cast(pos) - 1] = edge_id; - return 1; - }); - global_csr.Print(); // Debug print + // if (!global_csr.initialized_e) { + // const auto e_size = input.data[7].GetValue(0).GetValue(); + // global_csr.InitializeEdge(e_size); + // } + // TernaryExecutor::Execute( + // input.data[6], input.data[4], input.data[2], result, input.size(), + // [&](int64_t src, int64_t dst, int64_t edge_id) { + // const auto pos = ++global_csr.v[src + 1]; + // global_csr.e[static_cast(pos) - 1] = dst; + // global_csr.edge_ids[static_cast(pos) - 1] = edge_id; + // return 1; + // }); + // global_csr.Print(); // Debug print } class Barrier { @@ -244,7 +245,8 @@ class PathFindingGlobalState : public GlobalSinkState { public: using GlobalCompressedSparseRow = PhysicalPathFinding::GlobalCompressedSparseRow; PathFindingGlobalState(ClientContext &context, - const PhysicalPathFinding &op) : global_tasks(context, op.children[0]->GetTypes()), child(0) { + const PhysicalPathFinding &op) : global_tasks(context, op.children[0]->GetTypes()), + global_inputs(context, op.children[1]->GetTypes()), child(0) { RowLayout lhs_layout; lhs_layout.Initialize(op.children[0]->types); RowLayout rhs_layout; @@ -253,7 +255,7 @@ class PathFindingGlobalState : public GlobalSinkState { } PathFindingGlobalState(PathFindingGlobalState &prev) - : GlobalSinkState(prev), global_tasks(prev.global_tasks), + : GlobalSinkState(prev), global_tasks(prev.global_tasks), global_inputs(prev.global_inputs), global_csr(std::move(prev.global_csr)), global_bfs_state(std::move(prev.global_bfs_state)), child(prev.child + 1) { } @@ -262,8 +264,34 @@ class PathFindingGlobalState : public GlobalSinkState { lstate.Sink(input, *global_csr, child); } + void CSRCreateEdge() { + DataChunk input; + global_inputs.InitializeScanChunk(input); + ColumnDataScanState scan_state; + global_inputs.InitializeScan(scan_state); + auto result = Vector(LogicalTypeId::BIGINT); + while (global_inputs.Scan(scan_state, input)) { + input.Print(); + if (!global_csr->initialized_e) { + const auto e_size = input.data[7].GetValue(0).GetValue(); + global_csr->InitializeEdge(e_size); + } + TernaryExecutor::Execute( + input.data[6], input.data[4], input.data[2], result, input.size(), + [&](int64_t src, int64_t dst, int64_t edge_id) { + const auto pos = ++global_csr->v[src + 1]; + global_csr->e[static_cast(pos) - 1] = dst; + global_csr->edge_ids[static_cast(pos) - 1] = edge_id; + return 1; + }); + } + // debug print + global_csr->Print(); + } + // pairs is a 2-column table with src and dst ColumnDataCollection global_tasks; + ColumnDataCollection global_inputs; // pairs with path exists // ColumnDataCollection global_results; ColumnDataScanState scan_state; @@ -310,6 +338,7 @@ PhysicalPathFinding::Combine(ExecutionContext &context, auto &client_profiler = QueryProfiler::Get(context.client); gstate.global_tasks.Combine(lstate.local_tasks); + gstate.global_inputs.Combine(lstate.local_inputs); client_profiler.Flush(context.thread.profiler); // gstate.global_tasks.Print(); return SinkCombineResultType::FINISHED; @@ -465,6 +494,82 @@ class PhysicalBFSBottomUpTask : public ExecutorTask { idx_t end; }; +class PhysicalCSREdgeCreationTask : public ExecutorTask { +public: + PhysicalCSREdgeCreationTask(shared_ptr event_p, ClientContext &context, PathFindingGlobalState &state) + : ExecutorTask(context, std::move(event_p)), context(context), state(state) { + } + + TaskExecutionResult ExecuteTask(TaskExecutionMode mode) override { + auto& global_inputs = state.global_inputs; + auto& global_csr = state.global_csr; + auto& scan_state = state.scan_state; + + DataChunk input; + global_inputs.InitializeScanChunk(input); + auto result = Vector(LogicalTypeId::BIGINT); + while (true) { + { + lock_guard lock(global_csr->csr_lock); + if (!global_inputs.Scan(scan_state, input)) { + break; + } + } + if (!global_csr->initialized_e) { + const auto e_size = input.data[7].GetValue(0).GetValue(); + global_csr->InitializeEdge(e_size); + } + TernaryExecutor::Execute( + input.data[6], input.data[4], input.data[2], result, input.size(), + [&](int64_t src, int64_t dst, int64_t edge_id) { + const auto pos = ++global_csr->v[src + 1]; + global_csr->e[static_cast(pos) - 1] = dst; + global_csr->edge_ids[static_cast(pos) - 1] = edge_id; + return 1; + }); + } + event->FinishTask(); + return TaskExecutionResult::TASK_FINISHED; + } + +private: + ClientContext &context; + PathFindingGlobalState &state; +}; + +class CSREdgeCreationEvent : public BasePipelineEvent { +public: + CSREdgeCreationEvent(PathFindingGlobalState &gstate_p, Pipeline &pipeline_p) + : BasePipelineEvent(pipeline_p), gstate(gstate_p) { + } + + PathFindingGlobalState &gstate; + + void Schedule() override { + auto &context = pipeline->GetClientContext(); + auto &ts = TaskScheduler::GetScheduler(context); + idx_t num_threads = ts.NumberOfThreads(); + auto& scan_state = gstate.scan_state; + auto& global_inputs = gstate.global_inputs; + + global_inputs.InitializeScan(scan_state); + + vector> tasks; + for (idx_t tnum = 0; tnum < num_threads; tnum++) { + tasks.push_back(make_uniq(shared_from_this(), context, gstate)); + } + SetTasks(std::move(tasks)); + } + + void FinishEvent() override { + auto &gstate = this->gstate; + auto &global_csr = gstate.global_csr; + global_csr->is_ready = true; + // debug print + global_csr->Print(); + } +}; + class BFSIterativeEvent : public BasePipelineEvent { public: BFSIterativeEvent(PathFindingGlobalState &gstate_p, Pipeline &pipeline_p) @@ -593,7 +698,11 @@ PhysicalPathFinding::Finalize(Pipeline &pipeline, Event &event, auto &gstate = input.global_state.Cast(); auto &csr = gstate.global_csr; auto &global_tasks = gstate.global_tasks; - if (global_tasks.Count() != 0) { + if (gstate.child == 0) { + auto csr_event = make_shared(gstate, pipeline); + event.InsertEvent(std::move(std::dynamic_pointer_cast(csr_event))); + // gstate.CSRCreateEdge(); + } else if (gstate.child == 1 && global_tasks.Count() > 0) { auto all_pairs = make_shared(); DataChunk pairs; global_tasks.InitializeScanChunk(*all_pairs); @@ -614,10 +723,8 @@ PhysicalPathFinding::Finalize(Pipeline &pipeline, Event &event, if (all_pairs->size() > 0) { ScheduleBFSTasks(pipeline, event, gstate); } - } - // Move to the next input child ++gstate.child; @@ -731,7 +838,7 @@ SourceResultType PhysicalPathFinding::GetData(ExecutionContext &context, DataChunk &result, OperatorSourceInput &input) const { auto &pf_sink = sink_state->Cast(); - auto &pf_gstate = input.global_state.Cast(); + // auto &pf_gstate = input.global_state.Cast(); auto &pf_bfs_state = pf_sink.global_bfs_state; // auto &pf_lstate = input.local_state.Cast(); pf_bfs_state->result->Print(pf_bfs_state->pairs->size()); From 2f29872f426ae0def32a242d221d7ef73a176863 Mon Sep 17 00:00:00 2001 From: Pingan Ren Date: Tue, 23 Apr 2024 17:07:24 +0200 Subject: [PATCH 087/249] code clear --- src/operators/physical_path_finding_operator.cpp | 6 +++--- 1 file changed, 3 insertions(+), 3 deletions(-) diff --git a/src/operators/physical_path_finding_operator.cpp b/src/operators/physical_path_finding_operator.cpp index a27b447d..67a20971 100644 --- a/src/operators/physical_path_finding_operator.cpp +++ b/src/operators/physical_path_finding_operator.cpp @@ -566,7 +566,7 @@ class CSREdgeCreationEvent : public BasePipelineEvent { auto &global_csr = gstate.global_csr; global_csr->is_ready = true; // debug print - global_csr->Print(); + // global_csr->Print(); } }; @@ -713,7 +713,7 @@ PhysicalPathFinding::Finalize(Pipeline &pipeline, Event &event, all_pairs->Append(pairs, true); } // debug print - all_pairs->Print(); + // all_pairs->Print(); auto &ts = TaskScheduler::GetScheduler(context); idx_t num_threads = ts.NumberOfThreads(); @@ -841,7 +841,7 @@ PhysicalPathFinding::GetData(ExecutionContext &context, DataChunk &result, // auto &pf_gstate = input.global_state.Cast(); auto &pf_bfs_state = pf_sink.global_bfs_state; // auto &pf_lstate = input.local_state.Cast(); - pf_bfs_state->result->Print(pf_bfs_state->pairs->size()); + // pf_bfs_state->result->Print(pf_bfs_state->pairs->size()); // pf_gstate.Initialize(pf_sink); return result.size() == 0 ? SourceResultType::FINISHED From 141cb103254f771b1039ee3c76370933ab3196e0 Mon Sep 17 00:00:00 2001 From: Pingan Ren Date: Tue, 23 Apr 2024 19:49:35 +0200 Subject: [PATCH 088/249] add sequential algorithm back --- .../physical_path_finding_operator.cpp | 134 +++++++++++++++++- 1 file changed, 130 insertions(+), 4 deletions(-) diff --git a/src/operators/physical_path_finding_operator.cpp b/src/operators/physical_path_finding_operator.cpp index 67a20971..e611880c 100644 --- a/src/operators/physical_path_finding_operator.cpp +++ b/src/operators/physical_path_finding_operator.cpp @@ -690,6 +690,129 @@ class BFSIterativeEvent : public BasePipelineEvent { } }; +static bool IterativeLength(int64_t v_size, int64_t *v, vector &e, + vector> &seen, + vector> &visit, + vector> &next) { + bool change = false; + for (auto i = 0; i < v_size; i++) { + next[i] = 0; + } + for (auto i = 0; i < v_size; i++) { + if (visit[i].any()) { + for (auto offset = v[i]; offset < v[i + 1]; offset++) { + auto n = e[offset]; + next[n] = next[n] | visit[i]; + } + } + } + for (auto i = 0; i < v_size; i++) { + next[i] = next[i] & ~seen[i]; + seen[i] = seen[i] | next[i]; + change |= next[i].any(); + } + return change; +} + +static void IterativeLengthFunction(const unique_ptr &csr, + DataChunk &pairs, Vector &result) { + int64_t v_size = csr->v_size; + int64_t *v = (int64_t *)csr->v; + vector &e = csr->e; + + // get src and dst vectors for searches + auto &src = pairs.data[0]; + auto &dst = pairs.data[1]; + UnifiedVectorFormat vdata_src; + UnifiedVectorFormat vdata_dst; + src.ToUnifiedFormat(pairs.size(), vdata_src); + dst.ToUnifiedFormat(pairs.size(), vdata_dst); + + auto src_data = FlatVector::GetData(src); + auto dst_data = FlatVector::GetData(dst); + + ValidityMask &result_validity = FlatVector::Validity(result); + + // create result vector + result.SetVectorType(VectorType::FLAT_VECTOR); + auto result_data = FlatVector::GetData(result); + + // create temp SIMD arrays + vector> seen(v_size); + vector> visit1(v_size); + vector> visit2(v_size); + + // maps lane to search number + short lane_to_num[LANE_LIMIT]; + for (int64_t lane = 0; lane < LANE_LIMIT; lane++) { + lane_to_num[lane] = -1; // inactive + } + + idx_t started_searches = 0; + while (started_searches < pairs.size()) { + + // empty visit vectors + for (auto i = 0; i < v_size; i++) { + seen[i] = 0; + visit1[i] = 0; + } + + // add search jobs to free lanes + uint64_t active = 0; + for (int64_t lane = 0; lane < LANE_LIMIT; lane++) { + lane_to_num[lane] = -1; + while (started_searches < pairs.size()) { + int64_t search_num = started_searches++; + int64_t src_pos = vdata_src.sel->get_index(search_num); + int64_t dst_pos = vdata_dst.sel->get_index(search_num); + if (!vdata_src.validity.RowIsValid(src_pos)) { + result_validity.SetInvalid(search_num); + result_data[search_num] = (uint64_t)-1; /* no path */ + } else if (src_data[src_pos] == dst_data[dst_pos]) { + result_data[search_num] = + (uint64_t)0; // path of length 0 does not require a search + } else { + visit1[src_data[src_pos]][lane] = true; + lane_to_num[lane] = search_num; // active lane + active++; + break; + } + } + } + + // make passes while a lane is still active + for (int64_t iter = 1; active; iter++) { + if (!IterativeLength(v_size, v, e, seen, (iter & 1) ? visit1 : visit2, + (iter & 1) ? visit2 : visit1)) { + break; + } + // detect lanes that finished + for (int64_t lane = 0; lane < LANE_LIMIT; lane++) { + int64_t search_num = lane_to_num[lane]; + if (search_num >= 0) { // active lane + int64_t dst_pos = vdata_dst.sel->get_index(search_num); + if (seen[dst_data[dst_pos]][lane]) { + result_data[search_num] = + iter; /* found at iter => iter = path length */ + lane_to_num[lane] = -1; // mark inactive + active--; + } + } + } + } + + // no changes anymore: any still active searches have no path + for (int64_t lane = 0; lane < LANE_LIMIT; lane++) { + int64_t search_num = lane_to_num[lane]; + if (search_num >= 0) { // active lane + result_validity.SetInvalid(search_num); + result_data[search_num] = (int64_t)-1; /* no path */ + lane_to_num[lane] = -1; // mark inactive + } + } + } +} + SinkFinalizeType PhysicalPathFinding::Finalize(Pipeline &pipeline, Event &event, @@ -719,10 +842,13 @@ PhysicalPathFinding::Finalize(Pipeline &pipeline, Event &event, idx_t num_threads = ts.NumberOfThreads(); gstate.global_bfs_state = make_uniq(all_pairs, csr->v_size - 2, num_threads); - // Schedule the first round of BFS tasks - if (all_pairs->size() > 0) { - ScheduleBFSTasks(pipeline, event, gstate); - } + auto& result = *gstate.global_bfs_state->result; + IterativeLengthFunction(csr, *all_pairs, result); + + // // Schedule the first round of BFS tasks + // if (all_pairs->size() > 0) { + // ScheduleBFSTasks(pipeline, event, gstate); + // } } // Move to the next input child From 3c15e2d18b00af99c7fcc0696daea1f6b5bb83fd Mon Sep 17 00:00:00 2001 From: Pingan Ren Date: Wed, 24 Apr 2024 11:32:14 +0200 Subject: [PATCH 089/249] code clear --- src/operators/physical_path_finding_operator.cpp | 5 +---- 1 file changed, 1 insertion(+), 4 deletions(-) diff --git a/src/operators/physical_path_finding_operator.cpp b/src/operators/physical_path_finding_operator.cpp index e611880c..c5beed2a 100644 --- a/src/operators/physical_path_finding_operator.cpp +++ b/src/operators/physical_path_finding_operator.cpp @@ -271,7 +271,6 @@ class PathFindingGlobalState : public GlobalSinkState { global_inputs.InitializeScan(scan_state); auto result = Vector(LogicalTypeId::BIGINT); while (global_inputs.Scan(scan_state, input)) { - input.Print(); if (!global_csr->initialized_e) { const auto e_size = input.data[7].GetValue(0).GetValue(); global_csr->InitializeEdge(e_size); @@ -285,8 +284,6 @@ class PathFindingGlobalState : public GlobalSinkState { return 1; }); } - // debug print - global_csr->Print(); } // pairs is a 2-column table with src and dst @@ -967,7 +964,7 @@ PhysicalPathFinding::GetData(ExecutionContext &context, DataChunk &result, // auto &pf_gstate = input.global_state.Cast(); auto &pf_bfs_state = pf_sink.global_bfs_state; // auto &pf_lstate = input.local_state.Cast(); - // pf_bfs_state->result->Print(pf_bfs_state->pairs->size()); + pf_bfs_state->result->Print(pf_bfs_state->pairs->size()); // pf_gstate.Initialize(pf_sink); return result.size() == 0 ? SourceResultType::FINISHED From 140820a1093bd93cbb5c2096433861145b34f3ee Mon Sep 17 00:00:00 2001 From: Pingan Ren Date: Wed, 24 Apr 2024 15:29:23 +0200 Subject: [PATCH 090/249] sequential CAS --- .../physical_path_finding_operator.cpp | 210 ++++++++++-------- 1 file changed, 112 insertions(+), 98 deletions(-) diff --git a/src/operators/physical_path_finding_operator.cpp b/src/operators/physical_path_finding_operator.cpp index c5beed2a..79247568 100644 --- a/src/operators/physical_path_finding_operator.cpp +++ b/src/operators/physical_path_finding_operator.cpp @@ -99,8 +99,6 @@ class PathFindingLocalState : public LocalSinkState { void PathFindingLocalState::CreateCSRVertex(DataChunk &input, GlobalCompressedSparseRow &global_csr) { - // debug print - // input.Print(); if (!global_csr.initialized_v) { const auto v_size = input.data[8].GetValue(0).GetValue(); global_csr.InitializeVertex(v_size); @@ -114,19 +112,6 @@ void PathFindingLocalState::CreateCSRVertex(DataChunk &input, edge_count = edge_count + cnt; return edge_count; }); - // if (!global_csr.initialized_e) { - // const auto e_size = input.data[7].GetValue(0).GetValue(); - // global_csr.InitializeEdge(e_size); - // } - // TernaryExecutor::Execute( - // input.data[6], input.data[4], input.data[2], result, input.size(), - // [&](int64_t src, int64_t dst, int64_t edge_id) { - // const auto pos = ++global_csr.v[src + 1]; - // global_csr.e[static_cast(pos) - 1] = dst; - // global_csr.edge_ids[static_cast(pos) - 1] = edge_id; - // return 1; - // }); - // global_csr.Print(); // Debug print } class Barrier { @@ -188,8 +173,12 @@ class GlobalBFSState { src = FlatVector::GetData(src_data); dst = FlatVector::GetData(dst_data); for (auto i = 0; i < v_size; i++) { - seen[i] = 0; - visit1[i] = 0; + for (auto j = 0; j < 8; j++) { + seen[i][j] = 0; + visit1[i][j] = 0; + } + // seen[i] = 0; + // visit1[i] = 0; } } @@ -203,8 +192,12 @@ class GlobalBFSState { } // empty visit vectors for (auto i = 0; i < v_size; i++) { - seen[i] = 0; - visit1[i] = 0; + for (auto j = 0; j < 8; j++) { + seen[i][j] = 0; + visit1[i][j] = 0; + } + // seen[i] = 0; + // visit1[i] = 0; } } public: @@ -218,9 +211,12 @@ class GlobalBFSState { UnifiedVectorFormat vdata_src; UnifiedVectorFormat vdata_dst; int64_t lane_to_num[LANE_LIMIT]; - vector> seen; - vector> visit1; - vector> visit2; + // vector> seen; + // vector> visit1; + // vector> visit2; + vector[8]> seen; + vector[8]> visit1; + vector[8]> visit2; unique_ptr result; atomic frontier_size; @@ -358,8 +354,8 @@ class PhysicalBFSTopDownTask : public ExecutorTask { auto& visit = bfs_state->iter & 1 ? bfs_state->visit1 : bfs_state->visit2; auto& next = bfs_state->iter & 1 ? bfs_state->visit2 : bfs_state->visit1; auto& barrier = bfs_state->barrier; - auto& frontier_size = bfs_state->frontier_size; - auto& unseen_size = bfs_state->unseen_size; + // auto& frontier_size = bfs_state->frontier_size; + // auto& unseen_size = bfs_state->unseen_size; auto& v_size = bfs_state->v_size; int64_t *v = (int64_t *)state.global_csr->v; @@ -371,7 +367,10 @@ class PhysicalBFSTopDownTask : public ExecutorTask { auto left = block_size * worker_id; auto right = std::min(block_size * (worker_id + 1), (idx_t)v_size); for (auto i = left; i < right; i++) { - next[i] = 0; + for (auto j = 0; j < 8; j++) { + next[i][j] = 0; + } + // next[i] = 0; } barrier.Wait(); @@ -385,27 +384,42 @@ class PhysicalBFSTopDownTask : public ExecutorTask { auto end = task.second; for (auto i = start; i < end; i++) { - if (visit[i].any()) { - for (auto offset = v[i]; offset < v[i + 1]; offset++) { - auto n = e[offset]; - lock_guard lock(bfs_state->lock); - next[n] = next[n] | visit[i]; + for (auto j = 0; j < 8; j++) { + if (visit[i][j]) { + for (auto offset = v[i]; offset < v[i + 1]; offset++) { + auto n = e[offset]; + next[n][j] = next[n][j] | visit[i][j]; + } } } + // if (visit[i].any()) { + // for (auto offset = v[i]; offset < v[i + 1]; offset++) { + // auto n = e[offset]; + // lock_guard lock(bfs_state->lock); + // next[n] = next[n] | visit[i]; + // } + // } } } barrier.Wait(); for (auto i = left; i < right; i++) { - if (next[i].any()) { - next[i] = next[i] & ~seen[i]; - seen[i] = seen[i] | next[i]; - change |= next[i].any(); - - frontier_size = next[i].any() ? frontier_size + 1 : frontier_size.load(); - unseen_size = seen[i].all() ? unseen_size - 1 : unseen_size.load(); + for (auto j = 0; j < 8; j++) { + if (next[i][j]) { + next[i][j] = next[i][j] & ~seen[i][j]; + seen[i][j] = seen[i][j] | next[i][j]; + change |= next[i][j]; + } } + // if (next[i].any()) { + // next[i] = next[i] & ~seen[i]; + // seen[i] = seen[i] | next[i]; + // change |= next[i].any(); + + // frontier_size = next[i].any() ? frontier_size + 1 : frontier_size.load(); + // unseen_size = seen[i].all() ? unseen_size - 1 : unseen_size.load(); + // } } event->FinishTask(); @@ -437,59 +451,59 @@ class PhysicalBFSTopDownTask : public ExecutorTask { idx_t worker_id; }; -class PhysicalBFSBottomUpTask : public ExecutorTask { -public: - PhysicalBFSBottomUpTask(shared_ptr event_p, ClientContext &context, PathFindingGlobalState &state, idx_t start, idx_t end) - : ExecutorTask(context, std::move(event_p)), context(context), state(state), start(start), end(end) { - } - - TaskExecutionResult ExecuteTask(TaskExecutionMode mode) override { - auto& bfs_state = state.global_bfs_state; - auto& change = bfs_state->change; - auto& seen = bfs_state->seen; - auto& visit = bfs_state->iter & 1 ? bfs_state->visit1 : bfs_state->visit2; - auto& next = bfs_state->iter & 1 ? bfs_state->visit2 : bfs_state->visit1; - auto& barrier = bfs_state->barrier; - auto& frontier_size = bfs_state->frontier_size; - auto& unseen_size = bfs_state->unseen_size; - - int64_t *v = (int64_t *)state.global_csr->v; - vector &e = state.global_csr->e; - - for (auto i = start; i < end; i++) { - next[i] = 0; - } - - barrier.Wait(); - - for (auto i = start; i < end; i++) { - if (seen[i].all()) { - unseen_size -= 1; - continue; - } - for (auto offset = v[i]; offset < v[i + 1]; offset++) { - auto n = e[offset]; - next[i] = next[i] | visit[n]; - } - next[i] = next[i] & ~seen[i]; - seen[i] = seen[i] | next[i]; - change |= next[i].any(); - - frontier_size = next[i].any() ? frontier_size + 1 : frontier_size.load(); - unseen_size = seen[i].all() ? unseen_size - 1 : unseen_size.load(); - } - - event->FinishTask(); - return TaskExecutionResult::TASK_FINISHED; - } - -private: - ClientContext &context; - PathFindingGlobalState &state; - // [start, end) - idx_t start; - idx_t end; -}; +// class PhysicalBFSBottomUpTask : public ExecutorTask { +// public: +// PhysicalBFSBottomUpTask(shared_ptr event_p, ClientContext &context, PathFindingGlobalState &state, idx_t start, idx_t end) +// : ExecutorTask(context, std::move(event_p)), context(context), state(state), start(start), end(end) { +// } + +// TaskExecutionResult ExecuteTask(TaskExecutionMode mode) override { +// auto& bfs_state = state.global_bfs_state; +// auto& change = bfs_state->change; +// auto& seen = bfs_state->seen; +// auto& visit = bfs_state->iter & 1 ? bfs_state->visit1 : bfs_state->visit2; +// auto& next = bfs_state->iter & 1 ? bfs_state->visit2 : bfs_state->visit1; +// auto& barrier = bfs_state->barrier; +// auto& frontier_size = bfs_state->frontier_size; +// auto& unseen_size = bfs_state->unseen_size; + +// int64_t *v = (int64_t *)state.global_csr->v; +// vector &e = state.global_csr->e; + +// for (auto i = start; i < end; i++) { +// next[i] = 0; +// } + +// barrier.Wait(); + +// for (auto i = start; i < end; i++) { +// if (seen[i].all()) { +// unseen_size -= 1; +// continue; +// } +// for (auto offset = v[i]; offset < v[i + 1]; offset++) { +// auto n = e[offset]; +// next[i] = next[i] | visit[n]; +// } +// next[i] = next[i] & ~seen[i]; +// seen[i] = seen[i] | next[i]; +// change |= next[i].any(); + +// frontier_size = next[i].any() ? frontier_size + 1 : frontier_size.load(); +// unseen_size = seen[i].all() ? unseen_size - 1 : unseen_size.load(); +// } + +// event->FinishTask(); +// return TaskExecutionResult::TASK_FINISHED; +// } + +// private: +// ClientContext &context; +// PathFindingGlobalState &state; +// // [start, end) +// idx_t start; +// idx_t end; +// }; class PhysicalCSREdgeCreationTask : public ExecutorTask { public: @@ -656,7 +670,7 @@ class BFSIterativeEvent : public BasePipelineEvent { int64_t search_num = bfs_state->lane_to_num[lane]; if (search_num >= 0) { // active lane int64_t dst_pos = bfs_state->vdata_dst.sel->get_index(search_num); - if (bfs_state->seen[bfs_state->dst[dst_pos]][lane]) { + if (bfs_state->seen[bfs_state->dst[dst_pos]][lane / 64] & ((idx_t)1 << (lane % 64))) { result_data[search_num] = bfs_state->iter; /* found at iter => iter = path length */ bfs_state->lane_to_num[lane] = -1; // mark inactive @@ -839,13 +853,13 @@ PhysicalPathFinding::Finalize(Pipeline &pipeline, Event &event, idx_t num_threads = ts.NumberOfThreads(); gstate.global_bfs_state = make_uniq(all_pairs, csr->v_size - 2, num_threads); - auto& result = *gstate.global_bfs_state->result; - IterativeLengthFunction(csr, *all_pairs, result); + // auto& result = *gstate.global_bfs_state->result; + // IterativeLengthFunction(csr, *all_pairs, result); - // // Schedule the first round of BFS tasks - // if (all_pairs->size() > 0) { - // ScheduleBFSTasks(pipeline, event, gstate); - // } + // Schedule the first round of BFS tasks + if (all_pairs->size() > 0) { + ScheduleBFSTasks(pipeline, event, gstate); + } } // Move to the next input child @@ -881,7 +895,7 @@ void PhysicalPathFinding::ScheduleBFSTasks(Pipeline &pipeline, Event &event, Glo (uint64_t)0; // path of length 0 does not require a search } else { bfs_state->frontier_size++; - bfs_state->visit1[bfs_state->src[src_pos]][lane] = true; + bfs_state->visit1[bfs_state->src[src_pos]][lane / 64] |= ((idx_t)1 << (lane % 64)); bfs_state->lane_to_num[lane] = search_num; // active lane break; } From 3f756586e58ee304d1106e6a3b63f5b8fa98cb37 Mon Sep 17 00:00:00 2001 From: Pingan Ren Date: Wed, 24 Apr 2024 16:37:33 +0200 Subject: [PATCH 091/249] reduce task creation cost --- .../physical_path_finding_operator.cpp | 72 ++++++++++++------- 1 file changed, 45 insertions(+), 27 deletions(-) diff --git a/src/operators/physical_path_finding_operator.cpp b/src/operators/physical_path_finding_operator.cpp index 79247568..975052e9 100644 --- a/src/operators/physical_path_finding_operator.cpp +++ b/src/operators/physical_path_finding_operator.cpp @@ -180,6 +180,9 @@ class GlobalBFSState { // seen[i] = 0; // visit1[i] = 0; } + + barrier.Init(num_threads); + CreateTasks(); } void Clear() { @@ -200,6 +203,25 @@ class GlobalBFSState { // visit1[i] = 0; } } + + void CreateTasks() { + // workerTasks[workerId] = [task1, task2, ...] + vector>> worker_tasks(num_threads); + auto cur_worker = 0; + + for (auto offset = 0; offset < v_size; offset += split_size) { + auto worker_id = cur_worker % num_threads; + pair range = {offset, std::min(offset + split_size, v_size)}; + worker_tasks[worker_id].push_back(range); + cur_worker++; + } + + for (idx_t worker_id = 0; worker_id < num_threads; worker_id++) { + task_queues[worker_id].first.store(0); + task_queues[worker_id].second = worker_tasks[worker_id]; + } + } + public: shared_ptr pairs; int64_t iter; @@ -358,14 +380,14 @@ class PhysicalBFSTopDownTask : public ExecutorTask { // auto& unseen_size = bfs_state->unseen_size; auto& v_size = bfs_state->v_size; + InitTask(); + int64_t *v = (int64_t *)state.global_csr->v; vector &e = state.global_csr->e; + BoundaryCalculation(); + // clear next before each iteration - idx_t block_size = ceil((double)v_size / bfs_state->num_threads); - block_size = block_size == 0 ? 1 : block_size; - auto left = block_size * worker_id; - auto right = std::min(block_size * (worker_id + 1), (idx_t)v_size); for (auto i = left; i < right; i++) { for (auto j = 0; j < 8; j++) { next[i][j] = 0; @@ -376,7 +398,7 @@ class PhysicalBFSTopDownTask : public ExecutorTask { barrier.Wait(); while (true) { - auto task = fetch_task(); + auto task = FetchTask(); if (task.first == task.second) { break; } @@ -427,7 +449,12 @@ class PhysicalBFSTopDownTask : public ExecutorTask { } private: - pair fetch_task() { + void InitTask() { + auto& task_queue = state.global_bfs_state->task_queues; + task_queue[worker_id].first.store(0); + } + + pair FetchTask() { auto& task_queue = state.global_bfs_state->task_queues; idx_t offset = 0; do { @@ -442,12 +469,21 @@ class PhysicalBFSTopDownTask : public ExecutorTask { return {0, 0}; } + void BoundaryCalculation() { + auto& bfs_state = state.global_bfs_state; + auto& v_size = bfs_state->v_size; + idx_t block_size = ceil((double)v_size / bfs_state->num_threads); + block_size = block_size == 0 ? 1 : block_size; + left = block_size * worker_id; + right = std::min(block_size * (worker_id + 1), (idx_t)v_size); + } + private: ClientContext &context; PathFindingGlobalState &state; - // // [start, end) - // idx_t start; - // idx_t end; + // [left, right) + idx_t left; + idx_t right; idx_t worker_id; }; @@ -633,28 +669,10 @@ class BFSIterativeEvent : public BasePipelineEvent { bfs_state->frontier_size = 0; bfs_state->unseen_size = bfs_state->v_size; - // auto &ts = TaskScheduler::GetScheduler(context); - // idx_t num_threads = ts.NumberOfThreads(); - // idx_t num_threads = std::min((int64_t)ts.NumberOfThreads(), bfs_state->v_size); - // idx_t blocks = floor(bfs_state->v_size / (float)num_threads); - - bfs_state->barrier.Init(bfs_state->num_threads); - - CreateTasks(); - vector> bfs_tasks; for (idx_t tnum = 0; tnum < bfs_state->num_threads; tnum++) { bfs_tasks.push_back(make_uniq(shared_from_this(), context, gstate, tnum)); } - // for (idx_t tnum = 0; tnum < num_threads; tnum++) { - // if (bfs_state->is_top_down) { - // bfs_tasks.push_back(make_uniq(shared_from_this(), context, gstate, - // tnum * blocks, std::min(tnum * blocks + blocks, (idx_t)bfs_state->v_size))); - // } else { - // bfs_tasks.push_back(make_uniq(shared_from_this(), context, gstate, - // tnum * blocks, std::min(tnum * blocks + blocks, (idx_t)bfs_state->v_size))); - // } - // } SetTasks(std::move(bfs_tasks)); } From d43b082180bf622968ba22a114c6d3c2b4a02a36 Mon Sep 17 00:00:00 2001 From: Pingan Ren Date: Thu, 25 Apr 2024 22:31:06 +0200 Subject: [PATCH 092/249] atomic v -> pod v --- .../physical_path_finding_operator.hpp | 2 +- .../physical_path_finding_operator.cpp | 659 ++++++++++++++---- 2 files changed, 516 insertions(+), 145 deletions(-) diff --git a/src/include/duckpgq/operators/physical_path_finding_operator.hpp b/src/include/duckpgq/operators/physical_path_finding_operator.hpp index 98e50a44..7f3d8b2a 100644 --- a/src/include/duckpgq/operators/physical_path_finding_operator.hpp +++ b/src/include/duckpgq/operators/physical_path_finding_operator.hpp @@ -27,7 +27,7 @@ class PhysicalPathFinding : public PhysicalComparisonJoin { }; // ~GlobalCompressedSparseRow() { delete[] v; } - atomic *v; + int64_t *v; atomic *reverse_v; vector e; vector reverse_e; diff --git a/src/operators/physical_path_finding_operator.cpp b/src/operators/physical_path_finding_operator.cpp index 975052e9..8a4bd943 100644 --- a/src/operators/physical_path_finding_operator.cpp +++ b/src/operators/physical_path_finding_operator.cpp @@ -32,14 +32,14 @@ void PhysicalPathFinding::GlobalCompressedSparseRow::InitializeVertex(int64_t v_ } v_size = v_size_ + 2; try { - v = new std::atomic[v_size]; + v = new int64_t[v_size]; reverse_v = new std::atomic[v_size]; } catch (std::bad_alloc const &) { throw InternalException("Unable to initialize vector of size for csr vertex table " "representation"); } for (idx_t i = 0; i < v_size; ++i) { - v[i].store(0); + v[i] = 0; reverse_v[i].store(0); } initialized_v = true; @@ -161,7 +161,9 @@ class GlobalBFSState { GlobalBFSState(shared_ptr pairs_, int64_t v_size_, idx_t num_threads_) : pairs(pairs_), iter(1), v_size(v_size_), change(false), started_searches(0), seen(v_size_), visit1(v_size_), visit2(v_size_), - result(make_uniq(LogicalType::BIGINT, true, true, pairs_->size())), + result_length(make_uniq(LogicalType::BIGINT, true, true, pairs_->size())), + result_path(make_uniq(LogicalType::LIST(LogicalType::BIGINT), true, true, pairs_->size())), + parents_v(v_size_, std::vector(LANE_LIMIT, -1)), parents_e(v_size_, std::vector(LANE_LIMIT, -1)), frontier_size(0), unseen_size(v_size_), num_threads(num_threads_), task_queues(num_threads_) { for (auto i = 0; i < LANE_LIMIT; i++) { lane_to_num[i] = -1; @@ -199,6 +201,10 @@ class GlobalBFSState { seen[i][j] = 0; visit1[i][j] = 0; } + for (auto j = 0; j < LANE_LIMIT; j++) { + parents_v[i][j] = -1; + parents_e[i][j] = -1; + } // seen[i] = 0; // visit1[i] = 0; } @@ -220,6 +226,7 @@ class GlobalBFSState { task_queues[worker_id].first.store(0); task_queues[worker_id].second = worker_tasks[worker_id]; } + } public: @@ -239,7 +246,11 @@ class GlobalBFSState { vector[8]> seen; vector[8]> visit1; vector[8]> visit2; - unique_ptr result; + unique_ptr result_length; + unique_ptr result_path; + + vector> parents_v; + vector> parents_e; atomic frontier_size; atomic unseen_size; @@ -363,9 +374,9 @@ PhysicalPathFinding::Combine(ExecutionContext &context, // Finalize //===--------------------------------------------------------------------===// -class PhysicalBFSTopDownTask : public ExecutorTask { +class PhysicalIterativeTopDownTask : public ExecutorTask { public: - PhysicalBFSTopDownTask(shared_ptr event_p, ClientContext &context, PathFindingGlobalState &state, idx_t worker_id) + PhysicalIterativeTopDownTask(shared_ptr event_p, ClientContext &context, PathFindingGlobalState &state, idx_t worker_id) : ExecutorTask(context, std::move(event_p)), context(context), state(state), worker_id(worker_id) { } @@ -378,7 +389,6 @@ class PhysicalBFSTopDownTask : public ExecutorTask { auto& barrier = bfs_state->barrier; // auto& frontier_size = bfs_state->frontier_size; // auto& unseen_size = bfs_state->unseen_size; - auto& v_size = bfs_state->v_size; InitTask(); @@ -617,35 +627,14 @@ class CSREdgeCreationEvent : public BasePipelineEvent { } }; -class BFSIterativeEvent : public BasePipelineEvent { +class ParallelIterativeEvent : public BasePipelineEvent { public: - BFSIterativeEvent(PathFindingGlobalState &gstate_p, Pipeline &pipeline_p) + ParallelIterativeEvent(PathFindingGlobalState &gstate_p, Pipeline &pipeline_p) : BasePipelineEvent(pipeline_p), gstate(gstate_p) { } PathFindingGlobalState &gstate; -private: - void CreateTasks() { - auto &bfs_state = gstate.global_bfs_state; - - // workerTasks[workerId] = [task1, task2, ...] - vector>> worker_tasks(bfs_state->num_threads); - auto cur_worker = 0; - - for (auto offset = 0; offset < bfs_state->v_size; offset += bfs_state->split_size) { - auto worker_id = cur_worker % bfs_state->num_threads; - pair range = {offset, std::min(offset + bfs_state->split_size, bfs_state->v_size)}; - worker_tasks[worker_id].push_back(range); - cur_worker++; - } - - for (idx_t worker_id = 0; worker_id < bfs_state->num_threads; worker_id++) { - bfs_state->task_queues[worker_id].first.store(0); - bfs_state->task_queues[worker_id].second = worker_tasks[worker_id]; - } - } - public: void Schedule() override { auto &bfs_state = gstate.global_bfs_state; @@ -654,24 +643,24 @@ class BFSIterativeEvent : public BasePipelineEvent { bfs_state->change = false; // Determine the switch of algorithms - if (bfs_state->is_top_down) { - auto Ctb = bfs_state->unseen_size / bfs_state->alpha; - if (bfs_state->frontier_size > Ctb) { - bfs_state->is_top_down = false; - } - } else { - auto Cbt = bfs_state->v_size / bfs_state->beta; - if (bfs_state->frontier_size < Cbt) { - bfs_state->is_top_down = true; - } - } + // if (bfs_state->is_top_down) { + // auto Ctb = bfs_state->unseen_size / bfs_state->alpha; + // if (bfs_state->frontier_size > Ctb) { + // bfs_state->is_top_down = false; + // } + // } else { + // auto Cbt = bfs_state->v_size / bfs_state->beta; + // if (bfs_state->frontier_size < Cbt) { + // bfs_state->is_top_down = true; + // } + // } // clear the counters after the switch bfs_state->frontier_size = 0; bfs_state->unseen_size = bfs_state->v_size; vector> bfs_tasks; for (idx_t tnum = 0; tnum < bfs_state->num_threads; tnum++) { - bfs_tasks.push_back(make_uniq(shared_from_this(), context, gstate, tnum)); + bfs_tasks.push_back(make_uniq(shared_from_this(), context, gstate, tnum)); } SetTasks(std::move(bfs_tasks)); } @@ -679,8 +668,8 @@ class BFSIterativeEvent : public BasePipelineEvent { void FinishEvent() override { auto& bfs_state = gstate.global_bfs_state; - auto result_data = FlatVector::GetData(*bfs_state->result); - ValidityMask &result_validity = FlatVector::Validity(*bfs_state->result); + auto result_data = FlatVector::GetData(*bfs_state->result_length); + ValidityMask &result_validity = FlatVector::Validity(*bfs_state->result_length); if (bfs_state->change) { // detect lanes that finished @@ -698,7 +687,7 @@ class BFSIterativeEvent : public BasePipelineEvent { // into the next iteration bfs_state->iter++; - auto bfs_event = std::make_shared(gstate, *pipeline); + auto bfs_event = std::make_shared(gstate, *pipeline); this->InsertEvent(std::dynamic_pointer_cast(bfs_event)); } else { // no changes anymore: any still active searches have no path @@ -719,129 +708,511 @@ class BFSIterativeEvent : public BasePipelineEvent { } }; -static bool IterativeLength(int64_t v_size, int64_t *v, vector &e, - vector> &seen, - vector> &visit, - vector> &next) { - bool change = false; - for (auto i = 0; i < v_size; i++) { - next[i] = 0; +class SequentialIterativeEvent : public BasePipelineEvent { +public: + SequentialIterativeEvent(PathFindingGlobalState &gstate_p, Pipeline &pipeline_p) + : BasePipelineEvent(pipeline_p), gstate(gstate_p) { + } + + PathFindingGlobalState &gstate; + +public: + void Schedule() override { + auto &bfs_state = gstate.global_bfs_state; + + auto& pairs = *bfs_state->pairs; + auto& result = *bfs_state->result_length; + IterativeLengthFunction(gstate.global_csr, pairs, result); } - for (auto i = 0; i < v_size; i++) { - if (visit[i].any()) { - for (auto offset = v[i]; offset < v[i + 1]; offset++) { - auto n = e[offset]; - next[n] = next[n] | visit[i]; + +private: + static bool IterativeLength(int64_t v_size, int64_t *v, vector &e, + vector> &seen, + vector> &visit, + vector> &next) { + bool change = false; + for (auto i = 0; i < v_size; i++) { + next[i] = 0; + } + for (auto i = 0; i < v_size; i++) { + if (visit[i].any()) { + for (auto offset = v[i]; offset < v[i + 1]; offset++) { + auto n = e[offset]; + next[n] = next[n] | visit[i]; + } } } + for (auto i = 0; i < v_size; i++) { + next[i] = next[i] & ~seen[i]; + seen[i] = seen[i] | next[i]; + change |= next[i].any(); + } + return change; } - for (auto i = 0; i < v_size; i++) { - next[i] = next[i] & ~seen[i]; - seen[i] = seen[i] | next[i]; - change |= next[i].any(); - } - return change; -} -static void IterativeLengthFunction(const unique_ptr &csr, - DataChunk &pairs, Vector &result) { - int64_t v_size = csr->v_size; - int64_t *v = (int64_t *)csr->v; - vector &e = csr->e; + static void IterativeLengthFunction(const unique_ptr &csr, + DataChunk &pairs, Vector &result) { + int64_t v_size = csr->v_size; + int64_t *v = (int64_t *)csr->v; + vector &e = csr->e; - // get src and dst vectors for searches - auto &src = pairs.data[0]; - auto &dst = pairs.data[1]; - UnifiedVectorFormat vdata_src; - UnifiedVectorFormat vdata_dst; - src.ToUnifiedFormat(pairs.size(), vdata_src); - dst.ToUnifiedFormat(pairs.size(), vdata_dst); + // get src and dst vectors for searches + auto &src = pairs.data[0]; + auto &dst = pairs.data[1]; + UnifiedVectorFormat vdata_src; + UnifiedVectorFormat vdata_dst; + src.ToUnifiedFormat(pairs.size(), vdata_src); + dst.ToUnifiedFormat(pairs.size(), vdata_dst); - auto src_data = FlatVector::GetData(src); - auto dst_data = FlatVector::GetData(dst); + auto src_data = FlatVector::GetData(src); + auto dst_data = FlatVector::GetData(dst); - ValidityMask &result_validity = FlatVector::Validity(result); + ValidityMask &result_validity = FlatVector::Validity(result); - // create result vector - result.SetVectorType(VectorType::FLAT_VECTOR); - auto result_data = FlatVector::GetData(result); + // create result vector + result.SetVectorType(VectorType::FLAT_VECTOR); + auto result_data = FlatVector::GetData(result); - // create temp SIMD arrays - vector> seen(v_size); - vector> visit1(v_size); - vector> visit2(v_size); + // create temp SIMD arrays + vector> seen(v_size); + vector> visit1(v_size); + vector> visit2(v_size); - // maps lane to search number - short lane_to_num[LANE_LIMIT]; - for (int64_t lane = 0; lane < LANE_LIMIT; lane++) { - lane_to_num[lane] = -1; // inactive - } + // maps lane to search number + short lane_to_num[LANE_LIMIT]; + for (int64_t lane = 0; lane < LANE_LIMIT; lane++) { + lane_to_num[lane] = -1; // inactive + } - idx_t started_searches = 0; - while (started_searches < pairs.size()) { + idx_t started_searches = 0; + while (started_searches < pairs.size()) { - // empty visit vectors - for (auto i = 0; i < v_size; i++) { - seen[i] = 0; - visit1[i] = 0; - } + // empty visit vectors + for (auto i = 0; i < v_size; i++) { + seen[i] = 0; + visit1[i] = 0; + } - // add search jobs to free lanes - uint64_t active = 0; - for (int64_t lane = 0; lane < LANE_LIMIT; lane++) { - lane_to_num[lane] = -1; - while (started_searches < pairs.size()) { - int64_t search_num = started_searches++; - int64_t src_pos = vdata_src.sel->get_index(search_num); - int64_t dst_pos = vdata_dst.sel->get_index(search_num); - if (!vdata_src.validity.RowIsValid(src_pos)) { - result_validity.SetInvalid(search_num); - result_data[search_num] = (uint64_t)-1; /* no path */ - } else if (src_data[src_pos] == dst_data[dst_pos]) { - result_data[search_num] = - (uint64_t)0; // path of length 0 does not require a search - } else { - visit1[src_data[src_pos]][lane] = true; - lane_to_num[lane] = search_num; // active lane - active++; - break; + // add search jobs to free lanes + uint64_t active = 0; + for (int64_t lane = 0; lane < LANE_LIMIT; lane++) { + lane_to_num[lane] = -1; + while (started_searches < pairs.size()) { + int64_t search_num = started_searches++; + int64_t src_pos = vdata_src.sel->get_index(search_num); + int64_t dst_pos = vdata_dst.sel->get_index(search_num); + if (!vdata_src.validity.RowIsValid(src_pos)) { + result_validity.SetInvalid(search_num); + result_data[search_num] = (uint64_t)-1; /* no path */ + } else if (src_data[src_pos] == dst_data[dst_pos]) { + result_data[search_num] = + (uint64_t)0; // path of length 0 does not require a search + } else { + visit1[src_data[src_pos]][lane] = true; + lane_to_num[lane] = search_num; // active lane + active++; + break; + } } } - } - // make passes while a lane is still active - for (int64_t iter = 1; active; iter++) { - if (!IterativeLength(v_size, v, e, seen, (iter & 1) ? visit1 : visit2, - (iter & 1) ? visit2 : visit1)) { - break; + // make passes while a lane is still active + for (int64_t iter = 1; active; iter++) { + if (!IterativeLength(v_size, v, e, seen, (iter & 1) ? visit1 : visit2, + (iter & 1) ? visit2 : visit1)) { + break; + } + // detect lanes that finished + for (int64_t lane = 0; lane < LANE_LIMIT; lane++) { + int64_t search_num = lane_to_num[lane]; + if (search_num >= 0) { // active lane + int64_t dst_pos = vdata_dst.sel->get_index(search_num); + if (seen[dst_data[dst_pos]][lane]) { + result_data[search_num] = + iter; /* found at iter => iter = path length */ + lane_to_num[lane] = -1; // mark inactive + active--; + } + } + } } - // detect lanes that finished + + // no changes anymore: any still active searches have no path for (int64_t lane = 0; lane < LANE_LIMIT; lane++) { int64_t search_num = lane_to_num[lane]; if (search_num >= 0) { // active lane - int64_t dst_pos = vdata_dst.sel->get_index(search_num); - if (seen[dst_data[dst_pos]][lane]) { - result_data[search_num] = - iter; /* found at iter => iter = path length */ - lane_to_num[lane] = -1; // mark inactive - active--; + result_validity.SetInvalid(search_num); + result_data[search_num] = (int64_t)-1; /* no path */ + lane_to_num[lane] = -1; // mark inactive + } + } + } + } +}; + +class PhysicalShortestPathTopDownTask : public ExecutorTask { +public: + PhysicalShortestPathTopDownTask(shared_ptr event_p, ClientContext &context, PathFindingGlobalState &state, idx_t worker_id) + : ExecutorTask(context, std::move(event_p)), context(context), state(state), worker_id(worker_id) { + } + + TaskExecutionResult ExecuteTask(TaskExecutionMode mode) override { + auto& bfs_state = state.global_bfs_state; + auto& change = bfs_state->change; + auto& seen = bfs_state->seen; + auto& visit = bfs_state->iter & 1 ? bfs_state->visit1 : bfs_state->visit2; + auto& next = bfs_state->iter & 1 ? bfs_state->visit2 : bfs_state->visit1; + auto& barrier = bfs_state->barrier; + // auto& frontier_size = bfs_state->frontier_size; + // auto& unseen_size = bfs_state->unseen_size; + auto& parents_v = bfs_state->parents_v; + auto& parents_e = bfs_state->parents_e; + + InitTask(); + + int64_t *v = (int64_t *)state.global_csr->v; + vector &e = state.global_csr->e; + vector &edge_ids = state.global_csr->edge_ids; + + BoundaryCalculation(); + + // clear next before each iteration + for (auto i = left; i < right; i++) { + for (auto j = 0; j < 8; j++) { + next[i][j] = 0; + } + // next[i] = 0; + } + + barrier.Wait(); + + while (true) { + auto task = FetchTask(); + if (task.first == task.second) { + break; + } + auto start = task.first; + auto end = task.second; + + for (auto i = start; i < end; i++) { + for (auto j = 0; j < 8; j++) { + if (visit[i][j]) { + for (auto offset = v[i]; offset < v[i + 1]; offset++) { + auto n = e[offset]; + auto edge_id = edge_ids[offset]; + next[n][j] = next[n][j] | visit[i][j]; + for (auto l = 0; l < LANE_LIMIT; l++) { + parents_v[n][l] = ((parents_v[n][l] == -1) && visit[i][l / 64] & ((idx_t)1 << (l % 64))) + ? i : parents_v[n][l]; + parents_e[n][l] = ((parents_e[n][l] == -1) && visit[i][l / 64] & ((idx_t)1 << (l % 64))) + ? edge_id : parents_e[n][l]; + } + } } } } } - // no changes anymore: any still active searches have no path - for (int64_t lane = 0; lane < LANE_LIMIT; lane++) { - int64_t search_num = lane_to_num[lane]; - if (search_num >= 0) { // active lane - result_validity.SetInvalid(search_num); - result_data[search_num] = (int64_t)-1; /* no path */ - lane_to_num[lane] = -1; // mark inactive + barrier.Wait(); + + for (auto i = left; i < right; i++) { + for (auto j = 0; j < 8; j++) { + if (next[i][j]) { + next[i][j] = next[i][j] & ~seen[i][j]; + seen[i][j] = seen[i][j] | next[i][j]; + change |= next[i][j]; + } } } + + event->FinishTask(); + return TaskExecutionResult::TASK_FINISHED; } -} +private: + void InitTask() { + auto& task_queue = state.global_bfs_state->task_queues; + task_queue[worker_id].first.store(0); + } + + pair FetchTask() { + auto& task_queue = state.global_bfs_state->task_queues; + idx_t offset = 0; + do { + auto worker_idx = (worker_id + offset) % task_queue.size(); + auto cur_task_ix = task_queue[worker_idx].first.fetch_add(1); + if (cur_task_ix < task_queue[worker_idx].second.size()) { + return task_queue[worker_idx].second[cur_task_ix]; + } else { + offset++; + } + } while (offset < task_queue.size()); + return {0, 0}; + } + + void BoundaryCalculation() { + auto& bfs_state = state.global_bfs_state; + auto& v_size = bfs_state->v_size; + idx_t block_size = ceil((double)v_size / bfs_state->num_threads); + block_size = block_size == 0 ? 1 : block_size; + left = block_size * worker_id; + right = std::min(block_size * (worker_id + 1), (idx_t)v_size); + } + + ClientContext &context; + PathFindingGlobalState &state; + // [left, right) + idx_t left; + idx_t right; + idx_t worker_id; +}; + +class ParallelShortestPathEvent : public BasePipelineEvent { +public: + ParallelShortestPathEvent(PathFindingGlobalState &gstate_p, Pipeline &pipeline_p) + : BasePipelineEvent(pipeline_p), gstate(gstate_p) { + } + + PathFindingGlobalState &gstate; + +public: + void Schedule() override { + auto &bfs_state = gstate.global_bfs_state; + auto &context = pipeline->GetClientContext(); + + bfs_state->change = false; + + vector> bfs_tasks; + for (idx_t tnum = 0; tnum < bfs_state->num_threads; tnum++) { + bfs_tasks.push_back(make_uniq(shared_from_this(), context, gstate, tnum)); + } + SetTasks(std::move(bfs_tasks)); + } + + void FinishEvent() override { + + } +}; + +class SequentialShortestPathEvent : public BasePipelineEvent { +public: + SequentialShortestPathEvent(PathFindingGlobalState &gstate_p, Pipeline &pipeline_p) + : BasePipelineEvent(pipeline_p), gstate(gstate_p) { + } + + PathFindingGlobalState &gstate; + +public: + void Schedule() override { + auto &bfs_state = gstate.global_bfs_state; + auto &pairs = *bfs_state->pairs; + auto &result = *bfs_state->result_path; + ShortestPathFunction(gstate.global_csr, pairs, result); + } + +private: + static bool IterativeLength(int64_t v_size, int64_t *V, vector &E, + vector &edge_ids, + vector> &parents_v, + vector> &parents_e, + vector> &seen, + vector> &visit, + vector> &next) { + bool change = false; + for (auto v = 0; v < v_size; v++) { + next[v] = 0; + } + //! Keep track of edge id through which the node was reached + for (auto v = 0; v < v_size; v++) { + if (visit[v].any()) { + for (auto e = V[v]; e < V[v + 1]; e++) { + auto n = E[e]; + auto edge_id = edge_ids[e]; + next[n] = next[n] | visit[v]; + for (auto l = 0; l < LANE_LIMIT; l++) { + parents_v[n][l] = + ((parents_v[n][l] == -1) && visit[v][l]) ? v : parents_v[n][l]; + parents_e[n][l] = ((parents_e[n][l] == -1) && visit[v][l]) + ? edge_id + : parents_e[n][l]; + } + } + } + } + + for (auto v = 0; v < v_size; v++) { + next[v] = next[v] & ~seen[v]; + seen[v] = seen[v] | next[v]; + change |= next[v].any(); + } + return change; + } + + static void ShortestPathFunction(const unique_ptr &csr, + DataChunk &pairs, Vector &result) { + int64_t v_size = csr->v_size; + int64_t *v = (int64_t *)csr->v; + vector &e = csr->e; + vector &edge_ids = csr->edge_ids; + + auto &src = pairs.data[0]; + auto &dst = pairs.data[1]; + UnifiedVectorFormat vdata_src; + UnifiedVectorFormat vdata_dst; + src.ToUnifiedFormat(pairs.size(), vdata_src); + dst.ToUnifiedFormat(pairs.size(), vdata_dst); + + auto src_data = (int64_t *)vdata_src.data; + auto dst_data = (int64_t *)vdata_dst.data; + + result.SetVectorType(VectorType::FLAT_VECTOR); + auto result_data = FlatVector::GetData(result); + ValidityMask &result_validity = FlatVector::Validity(result); + + // create temp SIMD arrays + vector> seen(v_size); + vector> visit1(v_size); + vector> visit2(v_size); + vector> parents_v(v_size, + std::vector(LANE_LIMIT, -1)); + vector> parents_e(v_size, + std::vector(LANE_LIMIT, -1)); + + // maps lane to search number + int16_t lane_to_num[LANE_LIMIT]; + for (int64_t lane = 0; lane < LANE_LIMIT; lane++) { + lane_to_num[lane] = -1; // inactive + } + int64_t total_len = 0; + + idx_t started_searches = 0; + while (started_searches < pairs.size()) { + + // empty visit vectors + for (auto i = 0; i < v_size; i++) { + seen[i] = 0; + visit1[i] = 0; + for (auto j = 0; j < LANE_LIMIT; j++) { + parents_v[i][j] = -1; + parents_e[i][j] = -1; + } + } + + // add search jobs to free lanes + uint64_t active = 0; + for (int64_t lane = 0; lane < LANE_LIMIT; lane++) { + lane_to_num[lane] = -1; + while (started_searches < pairs.size()) { + int64_t search_num = started_searches++; + int64_t src_pos = vdata_src.sel->get_index(search_num); + if (!vdata_src.validity.RowIsValid(src_pos)) { + result_validity.SetInvalid(search_num); + } else { + visit1[src_data[src_pos]][lane] = true; + parents_v[src_data[src_pos]][lane] = + src_data[src_pos]; // Mark source with source id + parents_e[src_data[src_pos]][lane] = + -2; // Mark the source with -2, there is no incoming edge for the + // source. + lane_to_num[lane] = search_num; // active lane + active++; + break; + } + } + } + + //! make passes while a lane is still active + for (int64_t iter = 1; active; iter++) { + //! Perform one step of bfs exploration + if (!IterativeLength(v_size, v, e, edge_ids, parents_v, parents_e, seen, + (iter & 1) ? visit1 : visit2, + (iter & 1) ? visit2 : visit1)) { + break; + } + int64_t finished_searches = 0; + // detect lanes that finished + for (int64_t lane = 0; lane < LANE_LIMIT; lane++) { + int64_t search_num = lane_to_num[lane]; + if (search_num >= 0) { // active lane + //! Check if dst for a source has been seen + int64_t dst_pos = vdata_dst.sel->get_index(search_num); + if (seen[dst_data[dst_pos]][lane]) { + finished_searches++; + } + } + } + if (finished_searches == LANE_LIMIT) { + break; + } + } + //! Reconstruct the paths + for (int64_t lane = 0; lane < LANE_LIMIT; lane++) { + int64_t search_num = lane_to_num[lane]; + if (search_num == -1) { // empty lanes + continue; + } + + //! Searches that have stopped have found a path + int64_t src_pos = vdata_src.sel->get_index(search_num); + int64_t dst_pos = vdata_dst.sel->get_index(search_num); + if (src_data[src_pos] == dst_data[dst_pos]) { // Source == destination + unique_ptr output = + make_uniq(LogicalType::LIST(LogicalType::BIGINT)); + ListVector::PushBack(*output, src_data[src_pos]); + ListVector::Append(result, ListVector::GetEntry(*output), + ListVector::GetListSize(*output)); + result_data[search_num].length = ListVector::GetListSize(*output); + result_data[search_num].offset = total_len; + total_len += result_data[search_num].length; + continue; + } + std::vector output_vector; + std::vector output_edge; + auto source_v = src_data[src_pos]; // Take the source + + auto parent_vertex = + parents_v[dst_data[dst_pos]] + [lane]; // Take the parent vertex of the destination vertex + auto parent_edge = + parents_e[dst_data[dst_pos]] + [lane]; // Take the parent edge of the destination vertex + + output_vector.push_back(dst_data[dst_pos]); // Add destination vertex + output_vector.push_back(parent_edge); + while (parent_vertex != source_v) { // Continue adding vertices until we + // have reached the source vertex + //! -1 is used to signify no parent + if (parent_vertex == -1 || + parent_vertex == parents_v[parent_vertex][lane]) { + result_validity.SetInvalid(search_num); + break; + } + output_vector.push_back(parent_vertex); + parent_edge = parents_e[parent_vertex][lane]; + parent_vertex = parents_v[parent_vertex][lane]; + output_vector.push_back(parent_edge); + } + + if (!result_validity.RowIsValid(search_num)) { + continue; + } + output_vector.push_back(source_v); + std::reverse(output_vector.begin(), output_vector.end()); + auto output = make_uniq(LogicalType::LIST(LogicalType::BIGINT)); + for (auto val : output_vector) { + Value value_to_insert = val; + ListVector::PushBack(*output, value_to_insert); + } + + result_data[search_num].length = ListVector::GetListSize(*output); + result_data[search_num].offset = total_len; + ListVector::Append(result, ListVector::GetEntry(*output), + ListVector::GetListSize(*output)); + total_len += result_data[search_num].length; + } + } + } +}; SinkFinalizeType PhysicalPathFinding::Finalize(Pipeline &pipeline, Event &event, @@ -871,8 +1242,8 @@ PhysicalPathFinding::Finalize(Pipeline &pipeline, Event &event, idx_t num_threads = ts.NumberOfThreads(); gstate.global_bfs_state = make_uniq(all_pairs, csr->v_size - 2, num_threads); - // auto& result = *gstate.global_bfs_state->result; - // IterativeLengthFunction(csr, *all_pairs, result); + // auto bfs_event = make_shared(gstate, pipeline); + // event.InsertEvent(std::move(std::dynamic_pointer_cast(bfs_event))); // Schedule the first round of BFS tasks if (all_pairs->size() > 0) { @@ -896,8 +1267,8 @@ void PhysicalPathFinding::ScheduleBFSTasks(Pipeline &pipeline, Event &event, Glo // remaining pairs if (bfs_state->started_searches < gstate.global_tasks.Count()) { - auto result_data = FlatVector::GetData(*bfs_state->result); - auto& result_validity = FlatVector::Validity(*bfs_state->result); + auto result_data = FlatVector::GetData(*bfs_state->result_length); + auto& result_validity = FlatVector::Validity(*bfs_state->result_length); for (int64_t lane = 0; lane < LANE_LIMIT; lane++) { bfs_state->lane_to_num[lane] = -1; @@ -920,7 +1291,7 @@ void PhysicalPathFinding::ScheduleBFSTasks(Pipeline &pipeline, Event &event, Glo } } - auto bfs_event = make_shared(gstate, pipeline); + auto bfs_event = make_shared(gstate, pipeline); event.InsertEvent(std::move(std::dynamic_pointer_cast(bfs_event))); } } @@ -996,7 +1367,7 @@ PhysicalPathFinding::GetData(ExecutionContext &context, DataChunk &result, // auto &pf_gstate = input.global_state.Cast(); auto &pf_bfs_state = pf_sink.global_bfs_state; // auto &pf_lstate = input.local_state.Cast(); - pf_bfs_state->result->Print(pf_bfs_state->pairs->size()); + pf_bfs_state->result_length->Print(pf_bfs_state->pairs->size()); // pf_gstate.Initialize(pf_sink); return result.size() == 0 ? SourceResultType::FINISHED From 64f9687d471f636783afc473089d7ac82ff2c7d3 Mon Sep 17 00:00:00 2001 From: Pingan Ren Date: Thu, 25 Apr 2024 22:37:48 +0200 Subject: [PATCH 093/249] pod v -> atomic v --- .../duckpgq/operators/physical_path_finding_operator.hpp | 2 +- src/operators/physical_path_finding_operator.cpp | 4 ++-- 2 files changed, 3 insertions(+), 3 deletions(-) diff --git a/src/include/duckpgq/operators/physical_path_finding_operator.hpp b/src/include/duckpgq/operators/physical_path_finding_operator.hpp index 7f3d8b2a..98e50a44 100644 --- a/src/include/duckpgq/operators/physical_path_finding_operator.hpp +++ b/src/include/duckpgq/operators/physical_path_finding_operator.hpp @@ -27,7 +27,7 @@ class PhysicalPathFinding : public PhysicalComparisonJoin { }; // ~GlobalCompressedSparseRow() { delete[] v; } - int64_t *v; + atomic *v; atomic *reverse_v; vector e; vector reverse_e; diff --git a/src/operators/physical_path_finding_operator.cpp b/src/operators/physical_path_finding_operator.cpp index 8a4bd943..17166b96 100644 --- a/src/operators/physical_path_finding_operator.cpp +++ b/src/operators/physical_path_finding_operator.cpp @@ -32,14 +32,14 @@ void PhysicalPathFinding::GlobalCompressedSparseRow::InitializeVertex(int64_t v_ } v_size = v_size_ + 2; try { - v = new int64_t[v_size]; + v = new std::atomic[v_size]; reverse_v = new std::atomic[v_size]; } catch (std::bad_alloc const &) { throw InternalException("Unable to initialize vector of size for csr vertex table " "representation"); } for (idx_t i = 0; i < v_size; ++i) { - v[i] = 0; + v[i].store(0); reverse_v[i].store(0); } initialized_v = true; From bdbdd969e54290de8f6e72945c3c341ee63bd436 Mon Sep 17 00:00:00 2001 From: Pingan Ren Date: Thu, 25 Apr 2024 22:44:39 +0200 Subject: [PATCH 094/249] memory order relaxed --- .../physical_path_finding_operator.cpp | 17 ++++++++++------- 1 file changed, 10 insertions(+), 7 deletions(-) diff --git a/src/operators/physical_path_finding_operator.cpp b/src/operators/physical_path_finding_operator.cpp index 17166b96..0ad82429 100644 --- a/src/operators/physical_path_finding_operator.cpp +++ b/src/operators/physical_path_finding_operator.cpp @@ -400,7 +400,7 @@ class PhysicalIterativeTopDownTask : public ExecutorTask { // clear next before each iteration for (auto i = left; i < right; i++) { for (auto j = 0; j < 8; j++) { - next[i][j] = 0; + next[i][j].store(0, std::memory_order_relaxed); } // next[i] = 0; } @@ -417,10 +417,11 @@ class PhysicalIterativeTopDownTask : public ExecutorTask { for (auto i = start; i < end; i++) { for (auto j = 0; j < 8; j++) { - if (visit[i][j]) { + if (visit[i][j].load(std::memory_order_relaxed)) { for (auto offset = v[i]; offset < v[i + 1]; offset++) { auto n = e[offset]; - next[n][j] = next[n][j] | visit[i][j]; + next[n][j].store(next[n][j].load(std::memory_order_relaxed) | visit[i][j].load(std::memory_order_relaxed), std::memory_order_relaxed); + // next[n][j] = next[n][j] | visit[i][j]; } } } @@ -438,10 +439,12 @@ class PhysicalIterativeTopDownTask : public ExecutorTask { for (auto i = left; i < right; i++) { for (auto j = 0; j < 8; j++) { - if (next[i][j]) { - next[i][j] = next[i][j] & ~seen[i][j]; - seen[i][j] = seen[i][j] | next[i][j]; - change |= next[i][j]; + if (next[i][j].load(std::memory_order_relaxed)) { + next[i][j].store(next[i][j].load(std::memory_order_relaxed) & ~seen[i][j].load(std::memory_order_relaxed), std::memory_order_relaxed); + seen[i][j].store(seen[i][j].load(std::memory_order_relaxed) | next[i][j].load(std::memory_order_relaxed), std::memory_order_relaxed); + // next[i][j] = next[i][j] & ~seen[i][j]; + // seen[i][j] = seen[i][j] | next[i][j]; + change |= next[i][j].load(std::memory_order_relaxed); } } // if (next[i].any()) { From b5e7b04bf3ee1b517ed8bbd7fb3444394f61abf8 Mon Sep 17 00:00:00 2001 From: Pingan Ren Date: Fri, 26 Apr 2024 00:37:54 +0200 Subject: [PATCH 095/249] atomic bitset --- CMakeLists.txt | 3 + .../physical_path_finding_operator.cpp | 326 +++++++++--------- 2 files changed, 170 insertions(+), 159 deletions(-) diff --git a/CMakeLists.txt b/CMakeLists.txt index f1cdc7d2..b1bea926 100644 --- a/CMakeLists.txt +++ b/CMakeLists.txt @@ -40,6 +40,9 @@ target_link_libraries(${EXTENSION_NAME} OpenSSL::SSL OpenSSL::Crypto) target_link_libraries(${LOADABLE_EXTENSION_NAME} OpenSSL::SSL OpenSSL::Crypto) # >>>>>>> template/main +target_compile_options(${EXTENSION_NAME} PRIVATE -march=native) +target_link_options(${EXTENSION_NAME} PRIVATE -latomic) + install( TARGETS ${EXTENSION_NAME} EXPORT "${DUCKDB_EXPORT_SET}" diff --git a/src/operators/physical_path_finding_operator.cpp b/src/operators/physical_path_finding_operator.cpp index 0ad82429..8a9f6175 100644 --- a/src/operators/physical_path_finding_operator.cpp +++ b/src/operators/physical_path_finding_operator.cpp @@ -175,12 +175,12 @@ class GlobalBFSState { src = FlatVector::GetData(src_data); dst = FlatVector::GetData(dst_data); for (auto i = 0; i < v_size; i++) { - for (auto j = 0; j < 8; j++) { - seen[i][j] = 0; - visit1[i][j] = 0; - } - // seen[i] = 0; - // visit1[i] = 0; + // for (auto j = 0; j < 8; j++) { + // seen[i][j] = 0; + // visit1[i][j] = 0; + // } + seen[i] = 0; + visit1[i] = 0; } barrier.Init(num_threads); @@ -197,16 +197,16 @@ class GlobalBFSState { } // empty visit vectors for (auto i = 0; i < v_size; i++) { - for (auto j = 0; j < 8; j++) { - seen[i][j] = 0; - visit1[i][j] = 0; - } + // for (auto j = 0; j < 8; j++) { + // seen[i][j] = 0; + // visit1[i][j] = 0; + // } for (auto j = 0; j < LANE_LIMIT; j++) { parents_v[i][j] = -1; parents_e[i][j] = -1; } - // seen[i] = 0; - // visit1[i] = 0; + seen[i] = 0; + visit1[i] = 0; } } @@ -240,12 +240,12 @@ class GlobalBFSState { UnifiedVectorFormat vdata_src; UnifiedVectorFormat vdata_dst; int64_t lane_to_num[LANE_LIMIT]; - // vector> seen; - // vector> visit1; - // vector> visit2; - vector[8]> seen; - vector[8]> visit1; - vector[8]> visit2; + vector>> seen; + vector>> visit1; + vector>> visit2; + // vector[8]> seen; + // vector[8]> visit1; + // vector[8]> visit2; unique_ptr result_length; unique_ptr result_path; @@ -399,10 +399,10 @@ class PhysicalIterativeTopDownTask : public ExecutorTask { // clear next before each iteration for (auto i = left; i < right; i++) { - for (auto j = 0; j < 8; j++) { - next[i][j].store(0, std::memory_order_relaxed); - } - // next[i] = 0; + // for (auto j = 0; j < 8; j++) { + // next[i][j].store(0, std::memory_order_relaxed); + // } + next[i] = 0; } barrier.Wait(); @@ -416,45 +416,49 @@ class PhysicalIterativeTopDownTask : public ExecutorTask { auto end = task.second; for (auto i = start; i < end; i++) { - for (auto j = 0; j < 8; j++) { - if (visit[i][j].load(std::memory_order_relaxed)) { - for (auto offset = v[i]; offset < v[i + 1]; offset++) { - auto n = e[offset]; - next[n][j].store(next[n][j].load(std::memory_order_relaxed) | visit[i][j].load(std::memory_order_relaxed), std::memory_order_relaxed); - // next[n][j] = next[n][j] | visit[i][j]; - } - } - } - // if (visit[i].any()) { - // for (auto offset = v[i]; offset < v[i + 1]; offset++) { - // auto n = e[offset]; - // lock_guard lock(bfs_state->lock); - // next[n] = next[n] | visit[i]; + // for (auto j = 0; j < 8; j++) { + // if (visit[i][j].load(std::memory_order_relaxed)) { + // for (auto offset = v[i]; offset < v[i + 1]; offset++) { + // auto n = e[offset]; + // next[n][j].store(next[n][j].load(std::memory_order_relaxed) | visit[i][j].load(std::memory_order_relaxed), std::memory_order_relaxed); + // // next[n][j] = next[n][j] | visit[i][j]; + // } // } // } + if (visit[i].load(std::memory_order_relaxed).any()) { + for (auto offset = v[i]; offset < v[i + 1]; offset++) { + auto n = e[offset]; + // lock_guard lock(bfs_state->lock); + next[n].store(next[n].load(std::memory_order_relaxed) | visit[i].load(std::memory_order_relaxed), std::memory_order_relaxed); + // next[n] = next[n] | visit[i]; + } + } } } barrier.Wait(); for (auto i = left; i < right; i++) { - for (auto j = 0; j < 8; j++) { - if (next[i][j].load(std::memory_order_relaxed)) { - next[i][j].store(next[i][j].load(std::memory_order_relaxed) & ~seen[i][j].load(std::memory_order_relaxed), std::memory_order_relaxed); - seen[i][j].store(seen[i][j].load(std::memory_order_relaxed) | next[i][j].load(std::memory_order_relaxed), std::memory_order_relaxed); - // next[i][j] = next[i][j] & ~seen[i][j]; - // seen[i][j] = seen[i][j] | next[i][j]; - change |= next[i][j].load(std::memory_order_relaxed); - } - } - // if (next[i].any()) { - // next[i] = next[i] & ~seen[i]; - // seen[i] = seen[i] | next[i]; - // change |= next[i].any(); - - // frontier_size = next[i].any() ? frontier_size + 1 : frontier_size.load(); - // unseen_size = seen[i].all() ? unseen_size - 1 : unseen_size.load(); + // for (auto j = 0; j < 8; j++) { + // if (next[i][j].load(std::memory_order_relaxed)) { + // next[i][j].store(next[i][j].load(std::memory_order_relaxed) & ~seen[i][j].load(std::memory_order_relaxed), std::memory_order_relaxed); + // seen[i][j].store(seen[i][j].load(std::memory_order_relaxed) | next[i][j].load(std::memory_order_relaxed), std::memory_order_relaxed); + // // next[i][j] = next[i][j] & ~seen[i][j]; + // // seen[i][j] = seen[i][j] | next[i][j]; + // change |= next[i][j].load(std::memory_order_relaxed); + // } // } + if (next[i].load(std::memory_order_relaxed).any()) { + next[i].store(next[i].load(std::memory_order_relaxed) & ~seen[i].load(std::memory_order_relaxed), std::memory_order_relaxed); + seen[i].store(seen[i].load(std::memory_order_relaxed) | next[i].load(std::memory_order_relaxed), std::memory_order_relaxed); + change |= next[i].load(std::memory_order_relaxed).any(); + // next[i] = next[i] & ~seen[i]; + // seen[i] = seen[i] | next[i]; + // change |= next[i].any(); + + // frontier_size = next[i].any() ? frontier_size + 1 : frontier_size.load(); + // unseen_size = seen[i].all() ? unseen_size - 1 : unseen_size.load(); + } } event->FinishTask(); @@ -680,7 +684,8 @@ class ParallelIterativeEvent : public BasePipelineEvent { int64_t search_num = bfs_state->lane_to_num[lane]; if (search_num >= 0) { // active lane int64_t dst_pos = bfs_state->vdata_dst.sel->get_index(search_num); - if (bfs_state->seen[bfs_state->dst[dst_pos]][lane / 64] & ((idx_t)1 << (lane % 64))) { + // if (bfs_state->seen[bfs_state->dst[dst_pos]][lane / 64] & ((idx_t)1 << (lane % 64))) { + if (bfs_state->seen[bfs_state->dst[dst_pos]].load(std::memory_order_relaxed)[lane]) { result_data[search_num] = bfs_state->iter; /* found at iter => iter = path length */ bfs_state->lane_to_num[lane] = -1; // mark inactive @@ -853,122 +858,122 @@ class SequentialIterativeEvent : public BasePipelineEvent { } }; -class PhysicalShortestPathTopDownTask : public ExecutorTask { -public: - PhysicalShortestPathTopDownTask(shared_ptr event_p, ClientContext &context, PathFindingGlobalState &state, idx_t worker_id) - : ExecutorTask(context, std::move(event_p)), context(context), state(state), worker_id(worker_id) { - } - - TaskExecutionResult ExecuteTask(TaskExecutionMode mode) override { - auto& bfs_state = state.global_bfs_state; - auto& change = bfs_state->change; - auto& seen = bfs_state->seen; - auto& visit = bfs_state->iter & 1 ? bfs_state->visit1 : bfs_state->visit2; - auto& next = bfs_state->iter & 1 ? bfs_state->visit2 : bfs_state->visit1; - auto& barrier = bfs_state->barrier; - // auto& frontier_size = bfs_state->frontier_size; - // auto& unseen_size = bfs_state->unseen_size; - auto& parents_v = bfs_state->parents_v; - auto& parents_e = bfs_state->parents_e; - - InitTask(); - - int64_t *v = (int64_t *)state.global_csr->v; - vector &e = state.global_csr->e; - vector &edge_ids = state.global_csr->edge_ids; +// class PhysicalShortestPathTopDownTask : public ExecutorTask { +// public: +// PhysicalShortestPathTopDownTask(shared_ptr event_p, ClientContext &context, PathFindingGlobalState &state, idx_t worker_id) +// : ExecutorTask(context, std::move(event_p)), context(context), state(state), worker_id(worker_id) { +// } - BoundaryCalculation(); +// TaskExecutionResult ExecuteTask(TaskExecutionMode mode) override { +// auto& bfs_state = state.global_bfs_state; +// auto& change = bfs_state->change; +// auto& seen = bfs_state->seen; +// auto& visit = bfs_state->iter & 1 ? bfs_state->visit1 : bfs_state->visit2; +// auto& next = bfs_state->iter & 1 ? bfs_state->visit2 : bfs_state->visit1; +// auto& barrier = bfs_state->barrier; +// // auto& frontier_size = bfs_state->frontier_size; +// // auto& unseen_size = bfs_state->unseen_size; +// auto& parents_v = bfs_state->parents_v; +// auto& parents_e = bfs_state->parents_e; - // clear next before each iteration - for (auto i = left; i < right; i++) { - for (auto j = 0; j < 8; j++) { - next[i][j] = 0; - } - // next[i] = 0; - } +// InitTask(); - barrier.Wait(); +// int64_t *v = (int64_t *)state.global_csr->v; +// vector &e = state.global_csr->e; +// vector &edge_ids = state.global_csr->edge_ids; - while (true) { - auto task = FetchTask(); - if (task.first == task.second) { - break; - } - auto start = task.first; - auto end = task.second; +// BoundaryCalculation(); - for (auto i = start; i < end; i++) { - for (auto j = 0; j < 8; j++) { - if (visit[i][j]) { - for (auto offset = v[i]; offset < v[i + 1]; offset++) { - auto n = e[offset]; - auto edge_id = edge_ids[offset]; - next[n][j] = next[n][j] | visit[i][j]; - for (auto l = 0; l < LANE_LIMIT; l++) { - parents_v[n][l] = ((parents_v[n][l] == -1) && visit[i][l / 64] & ((idx_t)1 << (l % 64))) - ? i : parents_v[n][l]; - parents_e[n][l] = ((parents_e[n][l] == -1) && visit[i][l / 64] & ((idx_t)1 << (l % 64))) - ? edge_id : parents_e[n][l]; - } - } - } - } - } - } +// // clear next before each iteration +// for (auto i = left; i < right; i++) { +// for (auto j = 0; j < 8; j++) { +// next[i][j] = 0; +// } +// // next[i] = 0; +// } - barrier.Wait(); +// barrier.Wait(); - for (auto i = left; i < right; i++) { - for (auto j = 0; j < 8; j++) { - if (next[i][j]) { - next[i][j] = next[i][j] & ~seen[i][j]; - seen[i][j] = seen[i][j] | next[i][j]; - change |= next[i][j]; - } - } - } +// while (true) { +// auto task = FetchTask(); +// if (task.first == task.second) { +// break; +// } +// auto start = task.first; +// auto end = task.second; + +// for (auto i = start; i < end; i++) { +// for (auto j = 0; j < 8; j++) { +// if (visit[i][j]) { +// for (auto offset = v[i]; offset < v[i + 1]; offset++) { +// auto n = e[offset]; +// auto edge_id = edge_ids[offset]; +// next[n][j] = next[n][j] | visit[i][j]; +// for (auto l = 0; l < LANE_LIMIT; l++) { +// parents_v[n][l] = ((parents_v[n][l] == -1) && visit[i][l / 64] & ((idx_t)1 << (l % 64))) +// ? i : parents_v[n][l]; +// parents_e[n][l] = ((parents_e[n][l] == -1) && visit[i][l / 64] & ((idx_t)1 << (l % 64))) +// ? edge_id : parents_e[n][l]; +// } +// } +// } +// } +// } +// } - event->FinishTask(); - return TaskExecutionResult::TASK_FINISHED; - } +// barrier.Wait(); -private: - void InitTask() { - auto& task_queue = state.global_bfs_state->task_queues; - task_queue[worker_id].first.store(0); - } +// for (auto i = left; i < right; i++) { +// for (auto j = 0; j < 8; j++) { +// if (next[i][j]) { +// next[i][j] = next[i][j] & ~seen[i][j]; +// seen[i][j] = seen[i][j] | next[i][j]; +// change |= next[i][j]; +// } +// } +// } - pair FetchTask() { - auto& task_queue = state.global_bfs_state->task_queues; - idx_t offset = 0; - do { - auto worker_idx = (worker_id + offset) % task_queue.size(); - auto cur_task_ix = task_queue[worker_idx].first.fetch_add(1); - if (cur_task_ix < task_queue[worker_idx].second.size()) { - return task_queue[worker_idx].second[cur_task_ix]; - } else { - offset++; - } - } while (offset < task_queue.size()); - return {0, 0}; - } +// event->FinishTask(); +// return TaskExecutionResult::TASK_FINISHED; +// } - void BoundaryCalculation() { - auto& bfs_state = state.global_bfs_state; - auto& v_size = bfs_state->v_size; - idx_t block_size = ceil((double)v_size / bfs_state->num_threads); - block_size = block_size == 0 ? 1 : block_size; - left = block_size * worker_id; - right = std::min(block_size * (worker_id + 1), (idx_t)v_size); - } +// private: +// void InitTask() { +// auto& task_queue = state.global_bfs_state->task_queues; +// task_queue[worker_id].first.store(0); +// } + +// pair FetchTask() { +// auto& task_queue = state.global_bfs_state->task_queues; +// idx_t offset = 0; +// do { +// auto worker_idx = (worker_id + offset) % task_queue.size(); +// auto cur_task_ix = task_queue[worker_idx].first.fetch_add(1); +// if (cur_task_ix < task_queue[worker_idx].second.size()) { +// return task_queue[worker_idx].second[cur_task_ix]; +// } else { +// offset++; +// } +// } while (offset < task_queue.size()); +// return {0, 0}; +// } - ClientContext &context; - PathFindingGlobalState &state; - // [left, right) - idx_t left; - idx_t right; - idx_t worker_id; -}; +// void BoundaryCalculation() { +// auto& bfs_state = state.global_bfs_state; +// auto& v_size = bfs_state->v_size; +// idx_t block_size = ceil((double)v_size / bfs_state->num_threads); +// block_size = block_size == 0 ? 1 : block_size; +// left = block_size * worker_id; +// right = std::min(block_size * (worker_id + 1), (idx_t)v_size); +// } + +// ClientContext &context; +// PathFindingGlobalState &state; +// // [left, right) +// idx_t left; +// idx_t right; +// idx_t worker_id; +// }; class ParallelShortestPathEvent : public BasePipelineEvent { public: @@ -987,7 +992,7 @@ class ParallelShortestPathEvent : public BasePipelineEvent { vector> bfs_tasks; for (idx_t tnum = 0; tnum < bfs_state->num_threads; tnum++) { - bfs_tasks.push_back(make_uniq(shared_from_this(), context, gstate, tnum)); + // bfs_tasks.push_back(make_uniq(shared_from_this(), context, gstate, tnum)); } SetTasks(std::move(bfs_tasks)); } @@ -1287,7 +1292,10 @@ void PhysicalPathFinding::ScheduleBFSTasks(Pipeline &pipeline, Event &event, Glo (uint64_t)0; // path of length 0 does not require a search } else { bfs_state->frontier_size++; - bfs_state->visit1[bfs_state->src[src_pos]][lane / 64] |= ((idx_t)1 << (lane % 64)); + // bfs_state->visit1[bfs_state->src[src_pos]][lane / 64] |= ((idx_t)1 << (lane % 64)); + auto new_visit = bfs_state->visit1[bfs_state->src[src_pos]].load(); + new_visit[lane] = true; + bfs_state->visit1[bfs_state->src[src_pos]].store(new_visit); bfs_state->lane_to_num[lane] = search_num; // active lane break; } From b5fd1dc21726acc9327d7ff7e3a38a6befb25388 Mon Sep 17 00:00:00 2001 From: Pingan Ren Date: Fri, 26 Apr 2024 11:37:30 +0200 Subject: [PATCH 096/249] add timer --- src/operators/physical_path_finding_operator.cpp | 13 +++++++++++++ 1 file changed, 13 insertions(+) diff --git a/src/operators/physical_path_finding_operator.cpp b/src/operators/physical_path_finding_operator.cpp index 8a9f6175..e5c663c8 100644 --- a/src/operators/physical_path_finding_operator.cpp +++ b/src/operators/physical_path_finding_operator.cpp @@ -11,6 +11,7 @@ #include #include #include +#include namespace duckdb { @@ -266,6 +267,8 @@ class GlobalBFSState { Barrier barrier; + std::chrono::milliseconds time_elapsed = std::chrono::milliseconds(0); + // lock for next mutable mutex lock; }; @@ -642,6 +645,9 @@ class ParallelIterativeEvent : public BasePipelineEvent { PathFindingGlobalState &gstate; +private: + std::chrono::time_point start_time; + public: void Schedule() override { auto &bfs_state = gstate.global_bfs_state; @@ -670,11 +676,16 @@ class ParallelIterativeEvent : public BasePipelineEvent { bfs_tasks.push_back(make_uniq(shared_from_this(), context, gstate, tnum)); } SetTasks(std::move(bfs_tasks)); + start_time = std::chrono::high_resolution_clock::now(); } void FinishEvent() override { auto& bfs_state = gstate.global_bfs_state; + auto end_time = std::chrono::high_resolution_clock::now(); + auto duration = std::chrono::duration_cast(end_time - start_time); + bfs_state->time_elapsed += duration; + auto result_data = FlatVector::GetData(*bfs_state->result_length); ValidityMask &result_validity = FlatVector::Validity(*bfs_state->result_length); @@ -1379,6 +1390,8 @@ PhysicalPathFinding::GetData(ExecutionContext &context, DataChunk &result, auto &pf_bfs_state = pf_sink.global_bfs_state; // auto &pf_lstate = input.local_state.Cast(); pf_bfs_state->result_length->Print(pf_bfs_state->pairs->size()); + string message = "Algorithm running time: " + to_string(pf_bfs_state->time_elapsed.count()) + "ms"; + Printer::Print(message); // pf_gstate.Initialize(pf_sink); return result.size() == 0 ? SourceResultType::FINISHED From 0ef2a695fcf1c823c9e13cf435ad40eb5692ec02 Mon Sep 17 00:00:00 2001 From: Pingan Ren Date: Fri, 26 Apr 2024 11:43:02 +0200 Subject: [PATCH 097/249] add timer to serial algorithm --- src/operators/physical_path_finding_operator.cpp | 7 +++++++ 1 file changed, 7 insertions(+) diff --git a/src/operators/physical_path_finding_operator.cpp b/src/operators/physical_path_finding_operator.cpp index e5c663c8..e49ee107 100644 --- a/src/operators/physical_path_finding_operator.cpp +++ b/src/operators/physical_path_finding_operator.cpp @@ -735,13 +735,20 @@ class SequentialIterativeEvent : public BasePipelineEvent { PathFindingGlobalState &gstate; +private: + std::chrono::time_point start_time; + public: void Schedule() override { auto &bfs_state = gstate.global_bfs_state; auto& pairs = *bfs_state->pairs; auto& result = *bfs_state->result_length; + start_time = std::chrono::high_resolution_clock::now(); IterativeLengthFunction(gstate.global_csr, pairs, result); + auto end_time = std::chrono::high_resolution_clock::now(); + auto duration = std::chrono::duration_cast(end_time - start_time); + bfs_state->time_elapsed += duration; } private: From b41169b7ed1e52e27e16d2b1b9d55f2c983e631a Mon Sep 17 00:00:00 2001 From: Pingan Ren Date: Sat, 27 Apr 2024 17:38:33 +0200 Subject: [PATCH 098/249] return to serial next and visit --- .../physical_path_finding_operator.cpp | 242 +++++++++++------- 1 file changed, 147 insertions(+), 95 deletions(-) diff --git a/src/operators/physical_path_finding_operator.cpp b/src/operators/physical_path_finding_operator.cpp index e49ee107..7ddb0dac 100644 --- a/src/operators/physical_path_finding_operator.cpp +++ b/src/operators/physical_path_finding_operator.cpp @@ -12,6 +12,7 @@ #include #include #include +#include namespace duckdb { @@ -117,33 +118,35 @@ void PathFindingLocalState::CreateCSRVertex(DataChunk &input, class Barrier { public: - void Init(std::size_t iCount) { - std::unique_lock lLock{mMutex}; - mThreshold = iCount; - mCount = iCount; - mGeneration = 0; + void Init(std::size_t count) { + mThreshold.store(count, std::memory_order_relaxed); + mCount.store(count, std::memory_order_relaxed); + mGeneration.store(0, std::memory_order_relaxed); } void Wait() { - std::unique_lock lLock{mMutex}; - auto lGen = mGeneration; - if (!--mCount) { - mGeneration++; - mCount = mThreshold; + int currentGen = mGeneration.load(std::memory_order_acquire); + if (mCount.fetch_sub(1, std::memory_order_acq_rel) == 1) { + // Last thread to reach the barrier + mCount.store(mThreshold.load(std::memory_order_relaxed), std::memory_order_relaxed); + mGeneration.fetch_add(1, std::memory_order_acq_rel); mCond.notify_all(); } else { - mCond.wait(lLock, [this, lGen] { return lGen != mGeneration || mCount == mThreshold; }); + std::mutex localMutex; + std::unique_lock lock(localMutex); + mCond.wait(lock, [this, currentGen]() { + return currentGen != mGeneration.load(std::memory_order_acquire); + }); } } void DecreaseCount() { - std::unique_lock lLock{mMutex}; - mCount--; - mThreshold--; + mThreshold.fetch_sub(1, std::memory_order_acq_rel); + int expectedCount = mCount.fetch_sub(1, std::memory_order_acq_rel) - 1; - if (mCount == 0) { - mGeneration++; - mCount = mThreshold; + if (expectedCount == 0) { + mCount.store(mThreshold.load(std::memory_order_relaxed), std::memory_order_relaxed); + mGeneration.fetch_add(1, std::memory_order_acq_rel); mCond.notify_all(); } } @@ -151,11 +154,14 @@ class Barrier { private: std::mutex mMutex; std::condition_variable mCond; - std::size_t mThreshold; - std::size_t mCount; - std::size_t mGeneration; + std::atomic mThreshold; + std::atomic mCount; + std::atomic mGeneration; }; + + + class GlobalBFSState { public: @@ -176,15 +182,16 @@ class GlobalBFSState { src = FlatVector::GetData(src_data); dst = FlatVector::GetData(dst_data); for (auto i = 0; i < v_size; i++) { - // for (auto j = 0; j < 8; j++) { - // seen[i][j] = 0; - // visit1[i][j] = 0; - // } - seen[i] = 0; - visit1[i] = 0; + for (auto j = 0; j < 8; j++) { + seen[i][j] = 0; + visit1[i][j] = 0; + } + // seen[i] = 0; + // visit1[i] = 0; } barrier.Init(num_threads); + // pthread_barrier_init(&barrier, NULL, num_threads); CreateTasks(); } @@ -198,16 +205,16 @@ class GlobalBFSState { } // empty visit vectors for (auto i = 0; i < v_size; i++) { - // for (auto j = 0; j < 8; j++) { - // seen[i][j] = 0; - // visit1[i][j] = 0; - // } - for (auto j = 0; j < LANE_LIMIT; j++) { - parents_v[i][j] = -1; - parents_e[i][j] = -1; + for (auto j = 0; j < 8; j++) { + seen[i][j] = 0; + visit1[i][j] = 0; } - seen[i] = 0; - visit1[i] = 0; + // for (auto j = 0; j < LANE_LIMIT; j++) { + // parents_v[i][j] = -1; + // parents_e[i][j] = -1; + // } + // seen[i] = 0; + // visit1[i] = 0; } } @@ -241,12 +248,15 @@ class GlobalBFSState { UnifiedVectorFormat vdata_src; UnifiedVectorFormat vdata_dst; int64_t lane_to_num[LANE_LIMIT]; - vector>> seen; - vector>> visit1; - vector>> visit2; - // vector[8]> seen; - // vector[8]> visit1; - // vector[8]> visit2; + // vector> seen; + // vector> visit1; + // vector> visit2; + // vector>> seen; + // vector>> visit1; + // vector>> visit2; + vector[8]> seen; + vector[8]> visit1; + vector[8]> visit2; unique_ptr result_length; unique_ptr result_path; @@ -266,8 +276,9 @@ class GlobalBFSState { constexpr static int64_t split_size = 256; Barrier barrier; + // pthread_barrier_t barrier; - std::chrono::milliseconds time_elapsed = std::chrono::milliseconds(0); + std::chrono::microseconds time_elapsed = std::chrono::microseconds(0); // lock for next mutable mutex lock; @@ -392,7 +403,7 @@ class PhysicalIterativeTopDownTask : public ExecutorTask { auto& barrier = bfs_state->barrier; // auto& frontier_size = bfs_state->frontier_size; // auto& unseen_size = bfs_state->unseen_size; - + InitTask(); int64_t *v = (int64_t *)state.global_csr->v; @@ -402,13 +413,41 @@ class PhysicalIterativeTopDownTask : public ExecutorTask { // clear next before each iteration for (auto i = left; i < right; i++) { - // for (auto j = 0; j < 8; j++) { - // next[i][j].store(0, std::memory_order_relaxed); - // } - next[i] = 0; + for (auto j = 0; j < 8; j++) { + next[i][j].store(0, std::memory_order_relaxed); + } + // next[i] = 0; } barrier.Wait(); + // pthread_barrier_wait(&barrier); + + + // for (auto i = left; i < right; i++) { + // // if (visit[i].load(std::memory_order_relaxed).any()) { + // // for (auto offset = v[i]; offset < v[i + 1]; offset++) { + // // auto n = e[offset]; + // // // lock_guard lock(bfs_state->lock); + // // next[n].store(next[n].load(std::memory_order_relaxed) | visit[i].load(std::memory_order_relaxed), std::memory_order_relaxed); + // // // next[n] = next[n] | visit[i]; + // // } + // // } + // if (visit[i].any()) { + // for (auto offset = v[i]; offset < v[i + 1]; offset++) { + // auto n = e[offset]; + // // lock_guard lock(bfs_state->lock); + // next[n] = next[n] | visit[i]; + // // next[n].store(next[n].load(std::memory_order_relaxed) | visit[i].load(std::memory_order_relaxed), std::memory_order_relaxed); + // // next[n] = next[n] | visit[i]; + // } + // } + // } + + // auto end_time = std::chrono::high_resolution_clock::now(); + // auto duration = std::chrono::duration_cast(end_time - start_time); + // if (worker_id == 0) { + // Printer::Print("Duration_2: " + std::to_string(duration.count()) + " μs"); + // } while (true) { auto task = FetchTask(); @@ -419,49 +458,60 @@ class PhysicalIterativeTopDownTask : public ExecutorTask { auto end = task.second; for (auto i = start; i < end; i++) { - // for (auto j = 0; j < 8; j++) { - // if (visit[i][j].load(std::memory_order_relaxed)) { - // for (auto offset = v[i]; offset < v[i + 1]; offset++) { - // auto n = e[offset]; - // next[n][j].store(next[n][j].load(std::memory_order_relaxed) | visit[i][j].load(std::memory_order_relaxed), std::memory_order_relaxed); - // // next[n][j] = next[n][j] | visit[i][j]; - // } - // } - // } - if (visit[i].load(std::memory_order_relaxed).any()) { - for (auto offset = v[i]; offset < v[i + 1]; offset++) { - auto n = e[offset]; - // lock_guard lock(bfs_state->lock); - next[n].store(next[n].load(std::memory_order_relaxed) | visit[i].load(std::memory_order_relaxed), std::memory_order_relaxed); - // next[n] = next[n] | visit[i]; + for (auto j = 0; j < 8; j++) { + if (visit[i][j].load(std::memory_order_relaxed)) { + for (auto offset = v[i]; offset < v[i + 1]; offset++) { + auto n = e[offset]; + next[n][j].store(next[n][j].load(std::memory_order_relaxed) | visit[i][j].load(std::memory_order_relaxed), std::memory_order_relaxed); + // next[n][j] = next[n][j] | visit[i][j]; + } } } + // if (visit[i].load(std::memory_order_relaxed).any()) { + // for (auto offset = v[i]; offset < v[i + 1]; offset++) { + // auto n = e[offset]; + // // lock_guard lock(bfs_state->lock); + // next[n].store(next[n].load(std::memory_order_relaxed) | visit[i].load(std::memory_order_relaxed), std::memory_order_relaxed); + // // next[n] = next[n] | visit[i]; + // } + // } + // if (visit[i].any()) { + // for (auto offset = v[i]; offset < v[i + 1]; offset++) { + // auto n = e[offset]; + // // lock_guard lock(bfs_state->lock); + // next[n] = next[n] | visit[i]; + // // next[n].store(next[n].load(std::memory_order_relaxed) | visit[i].load(std::memory_order_relaxed), std::memory_order_relaxed); + // // next[n] = next[n] | visit[i]; + // } + // } } } barrier.Wait(); + // pthread_barrier_wait(&barrier); for (auto i = left; i < right; i++) { - // for (auto j = 0; j < 8; j++) { - // if (next[i][j].load(std::memory_order_relaxed)) { - // next[i][j].store(next[i][j].load(std::memory_order_relaxed) & ~seen[i][j].load(std::memory_order_relaxed), std::memory_order_relaxed); - // seen[i][j].store(seen[i][j].load(std::memory_order_relaxed) | next[i][j].load(std::memory_order_relaxed), std::memory_order_relaxed); - // // next[i][j] = next[i][j] & ~seen[i][j]; - // // seen[i][j] = seen[i][j] | next[i][j]; - // change |= next[i][j].load(std::memory_order_relaxed); - // } - // } - if (next[i].load(std::memory_order_relaxed).any()) { - next[i].store(next[i].load(std::memory_order_relaxed) & ~seen[i].load(std::memory_order_relaxed), std::memory_order_relaxed); - seen[i].store(seen[i].load(std::memory_order_relaxed) | next[i].load(std::memory_order_relaxed), std::memory_order_relaxed); - change |= next[i].load(std::memory_order_relaxed).any(); - // next[i] = next[i] & ~seen[i]; - // seen[i] = seen[i] | next[i]; - // change |= next[i].any(); - - // frontier_size = next[i].any() ? frontier_size + 1 : frontier_size.load(); - // unseen_size = seen[i].all() ? unseen_size - 1 : unseen_size.load(); + for (auto j = 0; j < 8; j++) { + if (next[i][j].load(std::memory_order_relaxed)) { + next[i][j].store(next[i][j].load(std::memory_order_relaxed) & ~seen[i][j].load(std::memory_order_relaxed), std::memory_order_relaxed); + seen[i][j].store(seen[i][j].load(std::memory_order_relaxed) | next[i][j].load(std::memory_order_relaxed), std::memory_order_relaxed); + // next[i][j] = next[i][j] & ~seen[i][j]; + // seen[i][j] = seen[i][j] | next[i][j]; + change |= next[i][j].load(std::memory_order_relaxed); + } } + // if (next[i].load(std::memory_order_relaxed).any()) { + // next[i].store(next[i].load(std::memory_order_relaxed) & ~seen[i].load(std::memory_order_relaxed), std::memory_order_relaxed); + // seen[i].store(seen[i].load(std::memory_order_relaxed) | next[i].load(std::memory_order_relaxed), std::memory_order_relaxed); + // change |= next[i].load(std::memory_order_relaxed).any(); + // } + // if (next[i].any()) { + // next[i] = next[i] & ~seen[i]; + // seen[i] = seen[i] | next[i]; + // change |= next[i].any(); + // // frontier_size = next[i].any() ? frontier_size + 1 : frontier_size.load(); + // // unseen_size = seen[i].all() ? unseen_size - 1 : unseen_size.load(); + // } } event->FinishTask(); @@ -683,7 +733,7 @@ class ParallelIterativeEvent : public BasePipelineEvent { auto& bfs_state = gstate.global_bfs_state; auto end_time = std::chrono::high_resolution_clock::now(); - auto duration = std::chrono::duration_cast(end_time - start_time); + auto duration = std::chrono::duration_cast(end_time - start_time); bfs_state->time_elapsed += duration; auto result_data = FlatVector::GetData(*bfs_state->result_length); @@ -695,8 +745,9 @@ class ParallelIterativeEvent : public BasePipelineEvent { int64_t search_num = bfs_state->lane_to_num[lane]; if (search_num >= 0) { // active lane int64_t dst_pos = bfs_state->vdata_dst.sel->get_index(search_num); - // if (bfs_state->seen[bfs_state->dst[dst_pos]][lane / 64] & ((idx_t)1 << (lane % 64))) { - if (bfs_state->seen[bfs_state->dst[dst_pos]].load(std::memory_order_relaxed)[lane]) { + if (bfs_state->seen[bfs_state->dst[dst_pos]][lane / 64] & ((idx_t)1 << (lane % 64))) { + // if (bfs_state->seen[bfs_state->dst[dst_pos]].load(std::memory_order_relaxed)[lane]) { + // if (bfs_state->seen[bfs_state->dst[dst_pos]][lane]) { result_data[search_num] = bfs_state->iter; /* found at iter => iter = path length */ bfs_state->lane_to_num[lane] = -1; // mark inactive @@ -1268,13 +1319,13 @@ PhysicalPathFinding::Finalize(Pipeline &pipeline, Event &event, idx_t num_threads = ts.NumberOfThreads(); gstate.global_bfs_state = make_uniq(all_pairs, csr->v_size - 2, num_threads); - // auto bfs_event = make_shared(gstate, pipeline); - // event.InsertEvent(std::move(std::dynamic_pointer_cast(bfs_event))); + auto bfs_event = make_shared(gstate, pipeline); + event.InsertEvent(std::move(std::dynamic_pointer_cast(bfs_event))); - // Schedule the first round of BFS tasks - if (all_pairs->size() > 0) { - ScheduleBFSTasks(pipeline, event, gstate); - } + // // Schedule the first round of BFS tasks + // if (all_pairs->size() > 0) { + // ScheduleBFSTasks(pipeline, event, gstate); + // } } // Move to the next input child @@ -1310,10 +1361,11 @@ void PhysicalPathFinding::ScheduleBFSTasks(Pipeline &pipeline, Event &event, Glo (uint64_t)0; // path of length 0 does not require a search } else { bfs_state->frontier_size++; - // bfs_state->visit1[bfs_state->src[src_pos]][lane / 64] |= ((idx_t)1 << (lane % 64)); - auto new_visit = bfs_state->visit1[bfs_state->src[src_pos]].load(); - new_visit[lane] = true; - bfs_state->visit1[bfs_state->src[src_pos]].store(new_visit); + bfs_state->visit1[bfs_state->src[src_pos]][lane / 64] |= ((idx_t)1 << (lane % 64)); + // auto new_visit = bfs_state->visit1[bfs_state->src[src_pos]].load(); + // new_visit[lane] = true; + // bfs_state->visit1[bfs_state->src[src_pos]].store(new_visit); + // bfs_state->visit1[bfs_state->src[src_pos]][lane] = true; bfs_state->lane_to_num[lane] = search_num; // active lane break; } @@ -1397,7 +1449,7 @@ PhysicalPathFinding::GetData(ExecutionContext &context, DataChunk &result, auto &pf_bfs_state = pf_sink.global_bfs_state; // auto &pf_lstate = input.local_state.Cast(); pf_bfs_state->result_length->Print(pf_bfs_state->pairs->size()); - string message = "Algorithm running time: " + to_string(pf_bfs_state->time_elapsed.count()) + "ms"; + string message = "Algorithm running time: " + to_string(pf_bfs_state->time_elapsed.count()) + " us"; Printer::Print(message); // pf_gstate.Initialize(pf_sink); From e333ba2f13d3ddd99baf8275b91031e9e3c1a3ec Mon Sep 17 00:00:00 2001 From: Pingan Ren Date: Sat, 27 Apr 2024 20:52:57 +0200 Subject: [PATCH 099/249] delete barrier --- .../physical_path_finding_operator.cpp | 150 +++++++++--------- 1 file changed, 78 insertions(+), 72 deletions(-) diff --git a/src/operators/physical_path_finding_operator.cpp b/src/operators/physical_path_finding_operator.cpp index 7ddb0dac..9598f535 100644 --- a/src/operators/physical_path_finding_operator.cpp +++ b/src/operators/physical_path_finding_operator.cpp @@ -118,45 +118,30 @@ void PathFindingLocalState::CreateCSRVertex(DataChunk &input, class Barrier { public: - void Init(std::size_t count) { - mThreshold.store(count, std::memory_order_relaxed); - mCount.store(count, std::memory_order_relaxed); - mGeneration.store(0, std::memory_order_relaxed); + explicit Barrier(std::size_t iCount) : + mThreshold(iCount), + mCount(iCount), + mGeneration(0) { } void Wait() { - int currentGen = mGeneration.load(std::memory_order_acquire); - if (mCount.fetch_sub(1, std::memory_order_acq_rel) == 1) { - // Last thread to reach the barrier - mCount.store(mThreshold.load(std::memory_order_relaxed), std::memory_order_relaxed); - mGeneration.fetch_add(1, std::memory_order_acq_rel); + std::unique_lock lLock{mMutex}; + auto lGen = mGeneration; + if (!--mCount) { + mGeneration++; + mCount = mThreshold; mCond.notify_all(); } else { - std::mutex localMutex; - std::unique_lock lock(localMutex); - mCond.wait(lock, [this, currentGen]() { - return currentGen != mGeneration.load(std::memory_order_acquire); - }); - } - } - - void DecreaseCount() { - mThreshold.fetch_sub(1, std::memory_order_acq_rel); - int expectedCount = mCount.fetch_sub(1, std::memory_order_acq_rel) - 1; - - if (expectedCount == 0) { - mCount.store(mThreshold.load(std::memory_order_relaxed), std::memory_order_relaxed); - mGeneration.fetch_add(1, std::memory_order_acq_rel); - mCond.notify_all(); + mCond.wait(lLock, [this, lGen] { return lGen != mGeneration; }); } } private: std::mutex mMutex; std::condition_variable mCond; - std::atomic mThreshold; - std::atomic mCount; - std::atomic mGeneration; + std::size_t mThreshold; + std::size_t mCount; + std::size_t mGeneration; }; @@ -171,7 +156,7 @@ class GlobalBFSState { result_length(make_uniq(LogicalType::BIGINT, true, true, pairs_->size())), result_path(make_uniq(LogicalType::LIST(LogicalType::BIGINT), true, true, pairs_->size())), parents_v(v_size_, std::vector(LANE_LIMIT, -1)), parents_e(v_size_, std::vector(LANE_LIMIT, -1)), - frontier_size(0), unseen_size(v_size_), num_threads(num_threads_), task_queues(num_threads_) { + frontier_size(0), unseen_size(v_size_), num_threads(num_threads_), task_queues(num_threads_), barrier(num_threads_) { for (auto i = 0; i < LANE_LIMIT; i++) { lane_to_num[i] = -1; } @@ -190,7 +175,6 @@ class GlobalBFSState { // visit1[i] = 0; } - barrier.Init(num_threads); // pthread_barrier_init(&barrier, NULL, num_threads); CreateTasks(); } @@ -409,17 +393,17 @@ class PhysicalIterativeTopDownTask : public ExecutorTask { int64_t *v = (int64_t *)state.global_csr->v; vector &e = state.global_csr->e; - BoundaryCalculation(); + // BoundaryCalculation(); // clear next before each iteration - for (auto i = left; i < right; i++) { - for (auto j = 0; j < 8; j++) { - next[i][j].store(0, std::memory_order_relaxed); - } - // next[i] = 0; - } + // for (auto i = left; i < right; i++) { + // for (auto j = 0; j < 8; j++) { + // next[i][j].store(0, std::memory_order_relaxed); + // } + // // next[i] = 0; + // } - barrier.Wait(); + // barrier.Wait(); // pthread_barrier_wait(&barrier); @@ -487,32 +471,32 @@ class PhysicalIterativeTopDownTask : public ExecutorTask { } } - barrier.Wait(); - // pthread_barrier_wait(&barrier); + // barrier.Wait(); + // // pthread_barrier_wait(&barrier); - for (auto i = left; i < right; i++) { - for (auto j = 0; j < 8; j++) { - if (next[i][j].load(std::memory_order_relaxed)) { - next[i][j].store(next[i][j].load(std::memory_order_relaxed) & ~seen[i][j].load(std::memory_order_relaxed), std::memory_order_relaxed); - seen[i][j].store(seen[i][j].load(std::memory_order_relaxed) | next[i][j].load(std::memory_order_relaxed), std::memory_order_relaxed); - // next[i][j] = next[i][j] & ~seen[i][j]; - // seen[i][j] = seen[i][j] | next[i][j]; - change |= next[i][j].load(std::memory_order_relaxed); - } - } - // if (next[i].load(std::memory_order_relaxed).any()) { - // next[i].store(next[i].load(std::memory_order_relaxed) & ~seen[i].load(std::memory_order_relaxed), std::memory_order_relaxed); - // seen[i].store(seen[i].load(std::memory_order_relaxed) | next[i].load(std::memory_order_relaxed), std::memory_order_relaxed); - // change |= next[i].load(std::memory_order_relaxed).any(); - // } - // if (next[i].any()) { - // next[i] = next[i] & ~seen[i]; - // seen[i] = seen[i] | next[i]; - // change |= next[i].any(); - // // frontier_size = next[i].any() ? frontier_size + 1 : frontier_size.load(); - // // unseen_size = seen[i].all() ? unseen_size - 1 : unseen_size.load(); - // } - } + // for (auto i = left; i < right; i++) { + // for (auto j = 0; j < 8; j++) { + // if (next[i][j].load(std::memory_order_relaxed)) { + // next[i][j].store(next[i][j].load(std::memory_order_relaxed) & ~seen[i][j].load(std::memory_order_relaxed), std::memory_order_relaxed); + // seen[i][j].store(seen[i][j].load(std::memory_order_relaxed) | next[i][j].load(std::memory_order_relaxed), std::memory_order_relaxed); + // // next[i][j] = next[i][j] & ~seen[i][j]; + // // seen[i][j] = seen[i][j] | next[i][j]; + // change |= next[i][j].load(std::memory_order_relaxed); + // } + // } + // // if (next[i].load(std::memory_order_relaxed).any()) { + // // next[i].store(next[i].load(std::memory_order_relaxed) & ~seen[i].load(std::memory_order_relaxed), std::memory_order_relaxed); + // // seen[i].store(seen[i].load(std::memory_order_relaxed) | next[i].load(std::memory_order_relaxed), std::memory_order_relaxed); + // // change |= next[i].load(std::memory_order_relaxed).any(); + // // } + // // if (next[i].any()) { + // // next[i] = next[i] & ~seen[i]; + // // seen[i] = seen[i] | next[i]; + // // change |= next[i].any(); + // // // frontier_size = next[i].any() ? frontier_size + 1 : frontier_size.load(); + // // // unseen_size = seen[i].all() ? unseen_size - 1 : unseen_size.load(); + // // } + // } event->FinishTask(); return TaskExecutionResult::TASK_FINISHED; @@ -718,20 +702,42 @@ class ParallelIterativeEvent : public BasePipelineEvent { // } // } // clear the counters after the switch - bfs_state->frontier_size = 0; - bfs_state->unseen_size = bfs_state->v_size; + // bfs_state->frontier_size = 0; + // bfs_state->unseen_size = bfs_state->v_size; + + start_time = std::chrono::high_resolution_clock::now(); + + // clear next before each iteration + auto& next = bfs_state->iter & 1 ? bfs_state->visit2 : bfs_state->visit1; + for (auto i = 0; i < bfs_state->v_size; i++) { + for (auto j = 0; j < 8; j++) { + next[i][j].store(0, std::memory_order_relaxed); + } + } vector> bfs_tasks; for (idx_t tnum = 0; tnum < bfs_state->num_threads; tnum++) { bfs_tasks.push_back(make_uniq(shared_from_this(), context, gstate, tnum)); } SetTasks(std::move(bfs_tasks)); - start_time = std::chrono::high_resolution_clock::now(); } void FinishEvent() override { auto& bfs_state = gstate.global_bfs_state; + auto& next = bfs_state->iter & 1 ? bfs_state->visit2 : bfs_state->visit1; + auto& seen = bfs_state->seen; + auto& change = bfs_state->change; + for (auto i = 0; i < bfs_state->v_size; i++) { + for (auto j = 0; j < 8; j++) { + if (next[i][j].load(std::memory_order_relaxed)) { + next[i][j].store(next[i][j].load(std::memory_order_relaxed) & ~seen[i][j].load(std::memory_order_relaxed), std::memory_order_relaxed); + seen[i][j].store(seen[i][j].load(std::memory_order_relaxed) | next[i][j].load(std::memory_order_relaxed), std::memory_order_relaxed); + change |= next[i][j].load(std::memory_order_relaxed); + } + } + } + auto end_time = std::chrono::high_resolution_clock::now(); auto duration = std::chrono::duration_cast(end_time - start_time); bfs_state->time_elapsed += duration; @@ -1319,13 +1325,13 @@ PhysicalPathFinding::Finalize(Pipeline &pipeline, Event &event, idx_t num_threads = ts.NumberOfThreads(); gstate.global_bfs_state = make_uniq(all_pairs, csr->v_size - 2, num_threads); - auto bfs_event = make_shared(gstate, pipeline); - event.InsertEvent(std::move(std::dynamic_pointer_cast(bfs_event))); + // auto bfs_event = make_shared(gstate, pipeline); + // event.InsertEvent(std::move(std::dynamic_pointer_cast(bfs_event))); - // // Schedule the first round of BFS tasks - // if (all_pairs->size() > 0) { - // ScheduleBFSTasks(pipeline, event, gstate); - // } + // Schedule the first round of BFS tasks + if (all_pairs->size() > 0) { + ScheduleBFSTasks(pipeline, event, gstate); + } } // Move to the next input child From 7f56ffee8fea54a3b7d736697139df5291d0abd1 Mon Sep 17 00:00:00 2001 From: Pingan Ren Date: Sat, 27 Apr 2024 22:11:33 +0200 Subject: [PATCH 100/249] don't split every iteration --- .../physical_path_finding_operator.cpp | 263 +++++++----------- 1 file changed, 100 insertions(+), 163 deletions(-) diff --git a/src/operators/physical_path_finding_operator.cpp b/src/operators/physical_path_finding_operator.cpp index 9598f535..a314b237 100644 --- a/src/operators/physical_path_finding_operator.cpp +++ b/src/operators/physical_path_finding_operator.cpp @@ -385,118 +385,33 @@ class PhysicalIterativeTopDownTask : public ExecutorTask { auto& visit = bfs_state->iter & 1 ? bfs_state->visit1 : bfs_state->visit2; auto& next = bfs_state->iter & 1 ? bfs_state->visit2 : bfs_state->visit1; auto& barrier = bfs_state->barrier; + auto result_data = FlatVector::GetData(*bfs_state->result_length); + ValidityMask &result_validity = FlatVector::Validity(*bfs_state->result_length); + auto& iter = bfs_state->iter; + auto& lane_to_num = bfs_state->lane_to_num; // auto& frontier_size = bfs_state->frontier_size; // auto& unseen_size = bfs_state->unseen_size; - - InitTask(); int64_t *v = (int64_t *)state.global_csr->v; vector &e = state.global_csr->e; - // BoundaryCalculation(); - - // clear next before each iteration - // for (auto i = left; i < right; i++) { - // for (auto j = 0; j < 8; j++) { - // next[i][j].store(0, std::memory_order_relaxed); - // } - // // next[i] = 0; - // } + BoundaryCalculation(); - // barrier.Wait(); - // pthread_barrier_wait(&barrier); - - - // for (auto i = left; i < right; i++) { - // // if (visit[i].load(std::memory_order_relaxed).any()) { - // // for (auto offset = v[i]; offset < v[i + 1]; offset++) { - // // auto n = e[offset]; - // // // lock_guard lock(bfs_state->lock); - // // next[n].store(next[n].load(std::memory_order_relaxed) | visit[i].load(std::memory_order_relaxed), std::memory_order_relaxed); - // // // next[n] = next[n] | visit[i]; - // // } - // // } - // if (visit[i].any()) { - // for (auto offset = v[i]; offset < v[i + 1]; offset++) { - // auto n = e[offset]; - // // lock_guard lock(bfs_state->lock); - // next[n] = next[n] | visit[i]; - // // next[n].store(next[n].load(std::memory_order_relaxed) | visit[i].load(std::memory_order_relaxed), std::memory_order_relaxed); - // // next[n] = next[n] | visit[i]; - // } - // } - // } - - // auto end_time = std::chrono::high_resolution_clock::now(); - // auto duration = std::chrono::duration_cast(end_time - start_time); - // if (worker_id == 0) { - // Printer::Print("Duration_2: " + std::to_string(duration.count()) + " μs"); - // } + do { + change = false; + InitTask(); + IterativeLength(v, change, barrier, e, seen, visit, next); + barrier.Wait(); - while (true) { - auto task = FetchTask(); - if (task.first == task.second) { - break; + if (worker_id == 0) { + ReachDetect(result_data); } - auto start = task.first; - auto end = task.second; - for (auto i = start; i < end; i++) { - for (auto j = 0; j < 8; j++) { - if (visit[i][j].load(std::memory_order_relaxed)) { - for (auto offset = v[i]; offset < v[i + 1]; offset++) { - auto n = e[offset]; - next[n][j].store(next[n][j].load(std::memory_order_relaxed) | visit[i][j].load(std::memory_order_relaxed), std::memory_order_relaxed); - // next[n][j] = next[n][j] | visit[i][j]; - } - } - } - // if (visit[i].load(std::memory_order_relaxed).any()) { - // for (auto offset = v[i]; offset < v[i + 1]; offset++) { - // auto n = e[offset]; - // // lock_guard lock(bfs_state->lock); - // next[n].store(next[n].load(std::memory_order_relaxed) | visit[i].load(std::memory_order_relaxed), std::memory_order_relaxed); - // // next[n] = next[n] | visit[i]; - // } - // } - // if (visit[i].any()) { - // for (auto offset = v[i]; offset < v[i + 1]; offset++) { - // auto n = e[offset]; - // // lock_guard lock(bfs_state->lock); - // next[n] = next[n] | visit[i]; - // // next[n].store(next[n].load(std::memory_order_relaxed) | visit[i].load(std::memory_order_relaxed), std::memory_order_relaxed); - // // next[n] = next[n] | visit[i]; - // } - // } - } - } + } while (change); - // barrier.Wait(); - // // pthread_barrier_wait(&barrier); - - // for (auto i = left; i < right; i++) { - // for (auto j = 0; j < 8; j++) { - // if (next[i][j].load(std::memory_order_relaxed)) { - // next[i][j].store(next[i][j].load(std::memory_order_relaxed) & ~seen[i][j].load(std::memory_order_relaxed), std::memory_order_relaxed); - // seen[i][j].store(seen[i][j].load(std::memory_order_relaxed) | next[i][j].load(std::memory_order_relaxed), std::memory_order_relaxed); - // // next[i][j] = next[i][j] & ~seen[i][j]; - // // seen[i][j] = seen[i][j] | next[i][j]; - // change |= next[i][j].load(std::memory_order_relaxed); - // } - // } - // // if (next[i].load(std::memory_order_relaxed).any()) { - // // next[i].store(next[i].load(std::memory_order_relaxed) & ~seen[i].load(std::memory_order_relaxed), std::memory_order_relaxed); - // // seen[i].store(seen[i].load(std::memory_order_relaxed) | next[i].load(std::memory_order_relaxed), std::memory_order_relaxed); - // // change |= next[i].load(std::memory_order_relaxed).any(); - // // } - // // if (next[i].any()) { - // // next[i] = next[i] & ~seen[i]; - // // seen[i] = seen[i] | next[i]; - // // change |= next[i].any(); - // // // frontier_size = next[i].any() ? frontier_size + 1 : frontier_size.load(); - // // // unseen_size = seen[i].all() ? unseen_size - 1 : unseen_size.load(); - // // } - // } + if (worker_id == 0) { + UnReachableSet(result_data, result_validity); + } event->FinishTask(); return TaskExecutionResult::TASK_FINISHED; @@ -532,6 +447,86 @@ class PhysicalIterativeTopDownTask : public ExecutorTask { right = std::min(block_size * (worker_id + 1), (idx_t)v_size); } + void IterativeLength(int64_t *v, bool& change, Barrier& barrier, + vector &e, + vector[8]> &seen, + vector[8]> &visit, + vector[8]> &next) { + // clear next before each iteration + for (auto i = left; i < right; i++) { + for (auto j = 0; j < 8; j++) { + next[i][j].store(0, std::memory_order_relaxed); + } + // next[i] = 0; + } + + barrier.Wait(); + + while (true) { + auto task = FetchTask(); + if (task.first == task.second) { + break; + } + auto start = task.first; + auto end = task.second; + + for (auto i = start; i < end; i++) { + for (auto j = 0; j < 8; j++) { + if (visit[i][j].load(std::memory_order_relaxed)) { + for (auto offset = v[i]; offset < v[i + 1]; offset++) { + auto n = e[offset]; + next[n][j].store(next[n][j].load(std::memory_order_relaxed) | visit[i][j].load(std::memory_order_relaxed), std::memory_order_relaxed); + } + } + } + } + } + + barrier.Wait(); + + for (auto i = left; i < right; i++) { + for (auto j = 0; j < 8; j++) { + if (next[i][j].load(std::memory_order_relaxed)) { + next[i][j].store(next[i][j].load(std::memory_order_relaxed) & ~seen[i][j].load(std::memory_order_relaxed), std::memory_order_relaxed); + seen[i][j].store(seen[i][j].load(std::memory_order_relaxed) | next[i][j].load(std::memory_order_relaxed), std::memory_order_relaxed); + change |= next[i][j].load(std::memory_order_relaxed); + } + } + } + } + +void ReachDetect(int64_t *result_data) { + auto &bfs_state = state.global_bfs_state; + // detect lanes that finished + for (int64_t lane = 0; lane < LANE_LIMIT; lane++) { + int64_t search_num = bfs_state->lane_to_num[lane]; + if (search_num >= 0) { // active lane + int64_t dst_pos = bfs_state->vdata_dst.sel->get_index(search_num); + if (bfs_state->seen[bfs_state->dst[dst_pos]][lane / 64] & ((idx_t)1 << (lane % 64))) { + // if (bfs_state->seen[bfs_state->dst[dst_pos]].load(std::memory_order_relaxed)[lane]) { + // if (bfs_state->seen[bfs_state->dst[dst_pos]][lane]) { + result_data[search_num] = + bfs_state->iter; /* found at iter => iter = path length */ + bfs_state->lane_to_num[lane] = -1; // mark inactive + } + } + } + // into the next iteration + bfs_state->iter++; +} + +void UnReachableSet(int64_t *result_data, ValidityMask &result_validity) { + auto &bfs_state = state.global_bfs_state; + for (int64_t lane = 0; lane < LANE_LIMIT; lane++) { + int64_t search_num = bfs_state->lane_to_num[lane]; + if (search_num >= 0) { // active lane + result_validity.SetInvalid(search_num); + result_data[search_num] = (int64_t)-1; /* no path */ + bfs_state->lane_to_num[lane] = -1; // mark inactive + } + } +} + private: ClientContext &context; PathFindingGlobalState &state; @@ -687,7 +682,6 @@ class ParallelIterativeEvent : public BasePipelineEvent { auto &bfs_state = gstate.global_bfs_state; auto &context = pipeline->GetClientContext(); - bfs_state->change = false; // Determine the switch of algorithms // if (bfs_state->is_top_down) { @@ -705,81 +699,24 @@ class ParallelIterativeEvent : public BasePipelineEvent { // bfs_state->frontier_size = 0; // bfs_state->unseen_size = bfs_state->v_size; - start_time = std::chrono::high_resolution_clock::now(); - - // clear next before each iteration - auto& next = bfs_state->iter & 1 ? bfs_state->visit2 : bfs_state->visit1; - for (auto i = 0; i < bfs_state->v_size; i++) { - for (auto j = 0; j < 8; j++) { - next[i][j].store(0, std::memory_order_relaxed); - } - } - vector> bfs_tasks; for (idx_t tnum = 0; tnum < bfs_state->num_threads; tnum++) { bfs_tasks.push_back(make_uniq(shared_from_this(), context, gstate, tnum)); } SetTasks(std::move(bfs_tasks)); + start_time = std::chrono::high_resolution_clock::now(); } void FinishEvent() override { auto& bfs_state = gstate.global_bfs_state; - auto& next = bfs_state->iter & 1 ? bfs_state->visit2 : bfs_state->visit1; - auto& seen = bfs_state->seen; - auto& change = bfs_state->change; - for (auto i = 0; i < bfs_state->v_size; i++) { - for (auto j = 0; j < 8; j++) { - if (next[i][j].load(std::memory_order_relaxed)) { - next[i][j].store(next[i][j].load(std::memory_order_relaxed) & ~seen[i][j].load(std::memory_order_relaxed), std::memory_order_relaxed); - seen[i][j].store(seen[i][j].load(std::memory_order_relaxed) | next[i][j].load(std::memory_order_relaxed), std::memory_order_relaxed); - change |= next[i][j].load(std::memory_order_relaxed); - } - } - } - auto end_time = std::chrono::high_resolution_clock::now(); auto duration = std::chrono::duration_cast(end_time - start_time); bfs_state->time_elapsed += duration; - auto result_data = FlatVector::GetData(*bfs_state->result_length); - ValidityMask &result_validity = FlatVector::Validity(*bfs_state->result_length); - - if (bfs_state->change) { - // detect lanes that finished - for (int64_t lane = 0; lane < LANE_LIMIT; lane++) { - int64_t search_num = bfs_state->lane_to_num[lane]; - if (search_num >= 0) { // active lane - int64_t dst_pos = bfs_state->vdata_dst.sel->get_index(search_num); - if (bfs_state->seen[bfs_state->dst[dst_pos]][lane / 64] & ((idx_t)1 << (lane % 64))) { - // if (bfs_state->seen[bfs_state->dst[dst_pos]].load(std::memory_order_relaxed)[lane]) { - // if (bfs_state->seen[bfs_state->dst[dst_pos]][lane]) { - result_data[search_num] = - bfs_state->iter; /* found at iter => iter = path length */ - bfs_state->lane_to_num[lane] = -1; // mark inactive - } - } - } - // into the next iteration - bfs_state->iter++; - - auto bfs_event = std::make_shared(gstate, *pipeline); - this->InsertEvent(std::dynamic_pointer_cast(bfs_event)); - } else { - // no changes anymore: any still active searches have no path - for (int64_t lane = 0; lane < LANE_LIMIT; lane++) { - int64_t search_num = bfs_state->lane_to_num[lane]; - if (search_num >= 0) { // active lane - result_validity.SetInvalid(search_num); - result_data[search_num] = (int64_t)-1; /* no path */ - bfs_state->lane_to_num[lane] = -1; // mark inactive - } - } - - // if remaining pairs, schedule the BFS for the next batch - if (bfs_state->started_searches < gstate.global_tasks.Count()) { - PhysicalPathFinding::ScheduleBFSTasks(*pipeline, *this, gstate); - } + // if remaining pairs, schedule the BFS for the next batch + if (bfs_state->started_searches < gstate.global_tasks.Count()) { + PhysicalPathFinding::ScheduleBFSTasks(*pipeline, *this, gstate); } } }; From c6dbdfbe719261356fb59e3fff356e7ef1ab4238 Mon Sep 17 00:00:00 2001 From: Pingan Ren Date: Sat, 27 Apr 2024 23:16:03 +0200 Subject: [PATCH 101/249] new spin lock --- .../physical_path_finding_operator.cpp | 32 +++++++++++-------- 1 file changed, 18 insertions(+), 14 deletions(-) diff --git a/src/operators/physical_path_finding_operator.cpp b/src/operators/physical_path_finding_operator.cpp index a314b237..d0d1028e 100644 --- a/src/operators/physical_path_finding_operator.cpp +++ b/src/operators/physical_path_finding_operator.cpp @@ -125,23 +125,24 @@ class Barrier { } void Wait() { - std::unique_lock lLock{mMutex}; - auto lGen = mGeneration; - if (!--mCount) { - mGeneration++; - mCount = mThreshold; - mCond.notify_all(); + int localGeneration = mGeneration.load(std::memory_order_acquire); + // Decrement the count atomically and check if this thread is the last to reach the barrier + if (mCount.fetch_sub(1, std::memory_order_acq_rel) == 1) { + // Last thread to reach the barrier + mCount.store(mThreshold, std::memory_order_relaxed); // Reset the count for the next use + mGeneration.fetch_add(1, std::memory_order_acq_rel); // Move to the next generation } else { - mCond.wait(lLock, [this, lGen] { return lGen != mGeneration; }); + // Not the last thread; spin-wait until the generation number changes + while (mGeneration.load(std::memory_order_acquire) == localGeneration) { + // std::this_thread::yield(); // Yield to reduce the impact on CPU usage + } } } private: - std::mutex mMutex; - std::condition_variable mCond; - std::size_t mThreshold; - std::size_t mCount; - std::size_t mGeneration; + std::atomic mCount; // Atomic counter to manage the number of waiting threads + std::atomic mGeneration; // Atomic generation count to manage barrier reuse across cycles + std::size_t mThreshold; // The number of threads that must reach the barrier to proceed }; @@ -382,8 +383,6 @@ class PhysicalIterativeTopDownTask : public ExecutorTask { auto& bfs_state = state.global_bfs_state; auto& change = bfs_state->change; auto& seen = bfs_state->seen; - auto& visit = bfs_state->iter & 1 ? bfs_state->visit1 : bfs_state->visit2; - auto& next = bfs_state->iter & 1 ? bfs_state->visit2 : bfs_state->visit1; auto& barrier = bfs_state->barrier; auto result_data = FlatVector::GetData(*bfs_state->result_length); ValidityMask &result_validity = FlatVector::Validity(*bfs_state->result_length); @@ -398,8 +397,13 @@ class PhysicalIterativeTopDownTask : public ExecutorTask { BoundaryCalculation(); do { + barrier.Wait(); change = false; InitTask(); + + auto& visit = iter & 1 ? bfs_state->visit1 : bfs_state->visit2; + auto& next = iter & 1 ? bfs_state->visit2 : bfs_state->visit1; + IterativeLength(v, change, barrier, e, seen, visit, next); barrier.Wait(); From e51d40b7b462c8c492ae0b87a12b4033b06d4ea3 Mon Sep 17 00:00:00 2001 From: Pingan Ren Date: Mon, 29 Apr 2024 12:35:34 +0200 Subject: [PATCH 102/249] parallel path finding --- .../physical_path_finding_operator.cpp | 443 +++++++++++------- 1 file changed, 271 insertions(+), 172 deletions(-) diff --git a/src/operators/physical_path_finding_operator.cpp b/src/operators/physical_path_finding_operator.cpp index d0d1028e..a2c43f51 100644 --- a/src/operators/physical_path_finding_operator.cpp +++ b/src/operators/physical_path_finding_operator.cpp @@ -125,35 +125,32 @@ class Barrier { } void Wait() { - int localGeneration = mGeneration.load(std::memory_order_acquire); - // Decrement the count atomically and check if this thread is the last to reach the barrier - if (mCount.fetch_sub(1, std::memory_order_acq_rel) == 1) { - // Last thread to reach the barrier - mCount.store(mThreshold, std::memory_order_relaxed); // Reset the count for the next use - mGeneration.fetch_add(1, std::memory_order_acq_rel); // Move to the next generation + std::unique_lock lLock{mMutex}; + auto lGen = mGeneration; + if (!--mCount) { + mGeneration++; + mCount = mThreshold; + mCond.notify_all(); } else { - // Not the last thread; spin-wait until the generation number changes - while (mGeneration.load(std::memory_order_acquire) == localGeneration) { - // std::this_thread::yield(); // Yield to reduce the impact on CPU usage - } + mCond.wait(lLock, [this, lGen] { return lGen != mGeneration; }); } } private: - std::atomic mCount; // Atomic counter to manage the number of waiting threads - std::atomic mGeneration; // Atomic generation count to manage barrier reuse across cycles - std::size_t mThreshold; // The number of threads that must reach the barrier to proceed + std::mutex mMutex; + std::condition_variable mCond; + std::size_t mThreshold; + std::size_t mCount; + std::size_t mGeneration; }; - - class GlobalBFSState { public: GlobalBFSState(shared_ptr pairs_, int64_t v_size_, idx_t num_threads_) : pairs(pairs_), iter(1), v_size(v_size_), change(false), started_searches(0), - seen(v_size_), visit1(v_size_), visit2(v_size_), + total_len(0), seen(v_size_), visit1(v_size_), visit2(v_size_), result_length(make_uniq(LogicalType::BIGINT, true, true, pairs_->size())), result_path(make_uniq(LogicalType::LIST(LogicalType::BIGINT), true, true, pairs_->size())), parents_v(v_size_, std::vector(LANE_LIMIT, -1)), parents_e(v_size_, std::vector(LANE_LIMIT, -1)), @@ -172,11 +169,12 @@ class GlobalBFSState { seen[i][j] = 0; visit1[i][j] = 0; } - // seen[i] = 0; - // visit1[i] = 0; + for (auto j = 0; j < LANE_LIMIT; j++) { + parents_v[i][j] = -1; + parents_e[i][j] = -1; + } } - // pthread_barrier_init(&barrier, NULL, num_threads); CreateTasks(); } @@ -194,12 +192,10 @@ class GlobalBFSState { seen[i][j] = 0; visit1[i][j] = 0; } - // for (auto j = 0; j < LANE_LIMIT; j++) { - // parents_v[i][j] = -1; - // parents_e[i][j] = -1; - // } - // seen[i] = 0; - // visit1[i] = 0; + for (auto j = 0; j < LANE_LIMIT; j++) { + parents_v[i][j] = -1; + parents_e[i][j] = -1; + } } } @@ -228,17 +224,12 @@ class GlobalBFSState { int64_t v_size; bool change; idx_t started_searches; + int64_t total_len; int64_t *src; int64_t *dst; UnifiedVectorFormat vdata_src; UnifiedVectorFormat vdata_dst; int64_t lane_to_num[LANE_LIMIT]; - // vector> seen; - // vector> visit1; - // vector> visit2; - // vector>> seen; - // vector>> visit1; - // vector>> visit2; vector[8]> seen; vector[8]> visit1; vector[8]> visit2; @@ -261,7 +252,6 @@ class GlobalBFSState { constexpr static int64_t split_size = 256; Barrier barrier; - // pthread_barrier_t barrier; std::chrono::microseconds time_elapsed = std::chrono::microseconds(0); @@ -351,7 +341,6 @@ SinkResultType PhysicalPathFinding::Sink(ExecutionContext &context, auto &gstate = input.global_state.Cast(); auto &lstate = input.local_state.Cast(); gstate.Sink(chunk, lstate); - // gstate.global_csr->is_ready = true; return SinkResultType::NEED_MORE_INPUT; } @@ -365,7 +354,6 @@ PhysicalPathFinding::Combine(ExecutionContext &context, gstate.global_tasks.Combine(lstate.local_tasks); gstate.global_inputs.Combine(lstate.local_inputs); client_profiler.Flush(context.thread.profiler); - // gstate.global_tasks.Print(); return SinkCombineResultType::FINISHED; } @@ -387,9 +375,6 @@ class PhysicalIterativeTopDownTask : public ExecutorTask { auto result_data = FlatVector::GetData(*bfs_state->result_length); ValidityMask &result_validity = FlatVector::Validity(*bfs_state->result_length); auto& iter = bfs_state->iter; - auto& lane_to_num = bfs_state->lane_to_num; - // auto& frontier_size = bfs_state->frontier_size; - // auto& unseen_size = bfs_state->unseen_size; int64_t *v = (int64_t *)state.global_csr->v; vector &e = state.global_csr->e; @@ -461,7 +446,6 @@ class PhysicalIterativeTopDownTask : public ExecutorTask { for (auto j = 0; j < 8; j++) { next[i][j].store(0, std::memory_order_relaxed); } - // next[i] = 0; } barrier.Wait(); @@ -507,8 +491,6 @@ void ReachDetect(int64_t *result_data) { if (search_num >= 0) { // active lane int64_t dst_pos = bfs_state->vdata_dst.sel->get_index(search_num); if (bfs_state->seen[bfs_state->dst[dst_pos]][lane / 64] & ((idx_t)1 << (lane % 64))) { - // if (bfs_state->seen[bfs_state->dst[dst_pos]].load(std::memory_order_relaxed)[lane]) { - // if (bfs_state->seen[bfs_state->dst[dst_pos]][lane]) { result_data[search_num] = bfs_state->iter; /* found at iter => iter = path length */ bfs_state->lane_to_num[lane] = -1; // mark inactive @@ -686,23 +668,6 @@ class ParallelIterativeEvent : public BasePipelineEvent { auto &bfs_state = gstate.global_bfs_state; auto &context = pipeline->GetClientContext(); - - // Determine the switch of algorithms - // if (bfs_state->is_top_down) { - // auto Ctb = bfs_state->unseen_size / bfs_state->alpha; - // if (bfs_state->frontier_size > Ctb) { - // bfs_state->is_top_down = false; - // } - // } else { - // auto Cbt = bfs_state->v_size / bfs_state->beta; - // if (bfs_state->frontier_size < Cbt) { - // bfs_state->is_top_down = true; - // } - // } - // clear the counters after the switch - // bfs_state->frontier_size = 0; - // bfs_state->unseen_size = bfs_state->v_size; - vector> bfs_tasks; for (idx_t tnum = 0; tnum < bfs_state->num_threads; tnum++) { bfs_tasks.push_back(make_uniq(shared_from_this(), context, gstate, tnum)); @@ -874,122 +839,247 @@ class SequentialIterativeEvent : public BasePipelineEvent { } }; -// class PhysicalShortestPathTopDownTask : public ExecutorTask { -// public: -// PhysicalShortestPathTopDownTask(shared_ptr event_p, ClientContext &context, PathFindingGlobalState &state, idx_t worker_id) -// : ExecutorTask(context, std::move(event_p)), context(context), state(state), worker_id(worker_id) { -// } +class PhysicalShortestPathTopDownTask : public ExecutorTask { +public: + PhysicalShortestPathTopDownTask(shared_ptr event_p, ClientContext &context, PathFindingGlobalState &state, idx_t worker_id) + : ExecutorTask(context, std::move(event_p)), context(context), state(state), worker_id(worker_id) { + } -// TaskExecutionResult ExecuteTask(TaskExecutionMode mode) override { -// auto& bfs_state = state.global_bfs_state; -// auto& change = bfs_state->change; -// auto& seen = bfs_state->seen; -// auto& visit = bfs_state->iter & 1 ? bfs_state->visit1 : bfs_state->visit2; -// auto& next = bfs_state->iter & 1 ? bfs_state->visit2 : bfs_state->visit1; -// auto& barrier = bfs_state->barrier; -// // auto& frontier_size = bfs_state->frontier_size; -// // auto& unseen_size = bfs_state->unseen_size; -// auto& parents_v = bfs_state->parents_v; -// auto& parents_e = bfs_state->parents_e; + TaskExecutionResult ExecuteTask(TaskExecutionMode mode) override { + auto& bfs_state = state.global_bfs_state; + auto& change = bfs_state->change; + auto& seen = bfs_state->seen; + auto& v_size = bfs_state->v_size; + auto& barrier = bfs_state->barrier; + auto& parents_v = bfs_state->parents_v; + auto& parents_e = bfs_state->parents_e; + auto& result_path = *bfs_state->result_path; + auto result_data = FlatVector::GetData(result_path); + ValidityMask &result_validity = FlatVector::Validity(result_path); + auto& iter = bfs_state->iter; -// InitTask(); + int64_t *v = (int64_t *)state.global_csr->v; + vector &e = state.global_csr->e; + vector &edge_ids = state.global_csr->edge_ids; -// int64_t *v = (int64_t *)state.global_csr->v; -// vector &e = state.global_csr->e; -// vector &edge_ids = state.global_csr->edge_ids; + BoundaryCalculation(); -// BoundaryCalculation(); + do { + InitTask(); -// // clear next before each iteration -// for (auto i = left; i < right; i++) { -// for (auto j = 0; j < 8; j++) { -// next[i][j] = 0; -// } -// // next[i] = 0; -// } + auto& visit = iter & 1 ? bfs_state->visit1 : bfs_state->visit2; + auto& next = iter & 1 ? bfs_state->visit2 : bfs_state->visit1; -// barrier.Wait(); + IterativeLength(v_size, change, barrier, v, e, edge_ids, parents_v, parents_e, seen, visit, next); -// while (true) { -// auto task = FetchTask(); -// if (task.first == task.second) { -// break; -// } -// auto start = task.first; -// auto end = task.second; - -// for (auto i = start; i < end; i++) { -// for (auto j = 0; j < 8; j++) { -// if (visit[i][j]) { -// for (auto offset = v[i]; offset < v[i + 1]; offset++) { -// auto n = e[offset]; -// auto edge_id = edge_ids[offset]; -// next[n][j] = next[n][j] | visit[i][j]; -// for (auto l = 0; l < LANE_LIMIT; l++) { -// parents_v[n][l] = ((parents_v[n][l] == -1) && visit[i][l / 64] & ((idx_t)1 << (l % 64))) -// ? i : parents_v[n][l]; -// parents_e[n][l] = ((parents_e[n][l] == -1) && visit[i][l / 64] & ((idx_t)1 << (l % 64))) -// ? edge_id : parents_e[n][l]; -// } -// } -// } -// } -// } -// } + barrier.Wait(); -// barrier.Wait(); + if (worker_id == 0) { + ReachDetect(result_data, change); + } -// for (auto i = left; i < right; i++) { -// for (auto j = 0; j < 8; j++) { -// if (next[i][j]) { -// next[i][j] = next[i][j] & ~seen[i][j]; -// seen[i][j] = seen[i][j] | next[i][j]; -// change |= next[i][j]; -// } -// } -// } + barrier.Wait(); + } while (change); -// event->FinishTask(); -// return TaskExecutionResult::TASK_FINISHED; -// } + if (worker_id == 0) { + PathConstruction(result_path, result_data, result_validity); + } -// private: -// void InitTask() { -// auto& task_queue = state.global_bfs_state->task_queues; -// task_queue[worker_id].first.store(0); -// } - -// pair FetchTask() { -// auto& task_queue = state.global_bfs_state->task_queues; -// idx_t offset = 0; -// do { -// auto worker_idx = (worker_id + offset) % task_queue.size(); -// auto cur_task_ix = task_queue[worker_idx].first.fetch_add(1); -// if (cur_task_ix < task_queue[worker_idx].second.size()) { -// return task_queue[worker_idx].second[cur_task_ix]; -// } else { -// offset++; -// } -// } while (offset < task_queue.size()); -// return {0, 0}; -// } + event->FinishTask(); + return TaskExecutionResult::TASK_FINISHED; + } -// void BoundaryCalculation() { -// auto& bfs_state = state.global_bfs_state; -// auto& v_size = bfs_state->v_size; -// idx_t block_size = ceil((double)v_size / bfs_state->num_threads); -// block_size = block_size == 0 ? 1 : block_size; -// left = block_size * worker_id; -// right = std::min(block_size * (worker_id + 1), (idx_t)v_size); -// } - -// ClientContext &context; -// PathFindingGlobalState &state; -// // [left, right) -// idx_t left; -// idx_t right; -// idx_t worker_id; -// }; +private: + void InitTask() { + auto& task_queue = state.global_bfs_state->task_queues; + task_queue[worker_id].first.store(0); + } + + pair FetchTask() { + auto& task_queue = state.global_bfs_state->task_queues; + idx_t offset = 0; + do { + auto worker_idx = (worker_id + offset) % task_queue.size(); + auto cur_task_ix = task_queue[worker_idx].first.fetch_add(1); + if (cur_task_ix < task_queue[worker_idx].second.size()) { + return task_queue[worker_idx].second[cur_task_ix]; + } else { + offset++; + } + } while (offset < task_queue.size()); + return {0, 0}; + } + + void BoundaryCalculation() { + auto& bfs_state = state.global_bfs_state; + auto& v_size = bfs_state->v_size; + idx_t block_size = ceil((double)v_size / bfs_state->num_threads); + block_size = block_size == 0 ? 1 : block_size; + left = block_size * worker_id; + right = std::min(block_size * (worker_id + 1), (idx_t)v_size); + } + + void IterativeLength(int64_t v_size, bool& change, Barrier& barrier, int64_t *v, + vector &e, vector &edge_ids, + vector> &parents_v, + vector> &parents_e, + vector[8]> &seen, + vector[8]> &visit, + vector[8]> &next) { + // clear next before each iteration + for (auto i = left; i < right; i++) { + for (auto j = 0; j < 8; j++) { + next[i][j] = 0; + } + } + + barrier.Wait(); + + while (true) { + auto task = FetchTask(); + if (task.first == task.second) { + break; + } + auto start = task.first; + auto end = task.second; + + for (auto i = start; i < end; i++) { + for (auto j = 0; j < 8; j++) { + if (visit[i][j].load(std::memory_order_relaxed)) { + for (auto offset = v[i]; offset < v[i + 1]; offset++) { + auto n = e[offset]; + auto edge_id = edge_ids[offset]; + // lock_guard lock(state.global_bfs_state->lock); + // next[n][j] = next[n][j] | visit[i][j]; + next[n][j].store(next[n][j].load(std::memory_order_relaxed) | visit[i][j].load(std::memory_order_relaxed), std::memory_order_relaxed); + for (auto l = 0; l < LANE_LIMIT; l++) { + // parents_v[n][l] = ((parents_v[n][l] == -1) && visit[i][l / 64] & ((idx_t)1 << (l % 64))) + // ? i : parents_v[n][l]; + // parents_e[n][l] = ((parents_e[n][l] == -1) && visit[i][l / 64] & ((idx_t)1 << (l % 64))) + // ? edge_id : parents_e[n][l]; + parents_v[n][l] = ((parents_v[n][l] == -1) && visit[i][l / 64].load(std::memory_order_relaxed) & ((idx_t)1 << (l % 64))) + ? i : parents_v[n][l]; + parents_e[n][l] = ((parents_e[n][l] == -1) && visit[i][l / 64].load(std::memory_order_relaxed) & ((idx_t)1 << (l % 64))) + ? edge_id : parents_e[n][l]; + } + } + } + } + } + } + + change = false; + barrier.Wait(); + + for (auto i = left; i < right; i++) { + for (auto j = 0; j < 8; j++) { + if (next[i][j].load(std::memory_order_relaxed)) { + next[i][j].store(next[i][j].load(std::memory_order_relaxed) & ~seen[i][j].load(std::memory_order_relaxed), std::memory_order_relaxed); + seen[i][j].store(seen[i][j].load(std::memory_order_relaxed) | next[i][j].load(std::memory_order_relaxed), std::memory_order_relaxed); + change |= next[i][j].load(std::memory_order_relaxed); + } + } + } + } + + void ReachDetect(list_entry_t *result_data, bool& change) { + auto &bfs_state = state.global_bfs_state; + int64_t finished_searches = 0; + // detect lanes that finished + for (int64_t lane = 0; lane < LANE_LIMIT; lane++) { + int64_t search_num = bfs_state->lane_to_num[lane]; + + if (search_num >= 0) { // active lane + //! Check if dst for a source has been seen + int64_t dst_pos = bfs_state->vdata_dst.sel->get_index(search_num); + if (bfs_state->seen[bfs_state->dst[dst_pos]][lane / 64] & ((idx_t)1 << (lane % 64))) { + finished_searches++; + } + } + } + if (finished_searches == LANE_LIMIT) { + change = false; + } + // into the next iteration + bfs_state->iter++; + } + + void PathConstruction(Vector &result, list_entry_t *result_data, ValidityMask &result_validity) { + auto &bfs_state = state.global_bfs_state; + //! Reconstruct the paths + for (int64_t lane = 0; lane < LANE_LIMIT; lane++) { + int64_t search_num = bfs_state->lane_to_num[lane]; + if (search_num == -1) { // empty lanes + continue; + } + + //! Searches that have stopped have found a path + int64_t src_pos = bfs_state->vdata_src.sel->get_index(search_num); + int64_t dst_pos = bfs_state->vdata_dst.sel->get_index(search_num); + if (bfs_state->src[src_pos] == bfs_state->dst[dst_pos]) { // Source == destination + unique_ptr output = + make_uniq(LogicalType::LIST(LogicalType::BIGINT)); + ListVector::PushBack(*output, bfs_state->src[src_pos]); + ListVector::Append(result, ListVector::GetEntry(*output), + ListVector::GetListSize(*output)); + result_data[search_num].length = ListVector::GetListSize(*output); + result_data[search_num].offset = bfs_state->total_len; + bfs_state->total_len += result_data[search_num].length; + continue; + } + std::vector output_vector; + std::vector output_edge; + auto source_v = bfs_state->src[src_pos]; // Take the source + + auto parent_vertex = + bfs_state->parents_v[bfs_state->dst[dst_pos]] + [lane]; // Take the parent vertex of the destination vertex + auto parent_edge = + bfs_state->parents_e[bfs_state->dst[dst_pos]] + [lane]; // Take the parent edge of the destination vertex + + output_vector.push_back(bfs_state->dst[dst_pos]); // Add destination vertex + output_vector.push_back(parent_edge); + while (parent_vertex != source_v) { // Continue adding vertices until we + // have reached the source vertex + //! -1 is used to signify no parent + if (parent_vertex == -1 || + parent_vertex == bfs_state->parents_v[parent_vertex][lane]) { + result_validity.SetInvalid(search_num); + break; + } + output_vector.push_back(parent_vertex); + parent_edge = bfs_state->parents_e[parent_vertex][lane]; + parent_vertex = bfs_state->parents_v[parent_vertex][lane]; + output_vector.push_back(parent_edge); + } + + if (!result_validity.RowIsValid(search_num)) { + continue; + } + output_vector.push_back(source_v); + std::reverse(output_vector.begin(), output_vector.end()); + auto output = make_uniq(LogicalType::LIST(LogicalType::BIGINT)); + for (auto val : output_vector) { + Value value_to_insert = val; + ListVector::PushBack(*output, value_to_insert); + } + + result_data[search_num].length = ListVector::GetListSize(*output); + result_data[search_num].offset = bfs_state->total_len; + ListVector::Append(result, ListVector::GetEntry(*output), + ListVector::GetListSize(*output)); + bfs_state->total_len += result_data[search_num].length; + } + } + + ClientContext &context; + PathFindingGlobalState &state; + // [left, right) + idx_t left; + idx_t right; + idx_t worker_id; +}; class ParallelShortestPathEvent : public BasePipelineEvent { public: @@ -999,22 +1089,32 @@ class ParallelShortestPathEvent : public BasePipelineEvent { PathFindingGlobalState &gstate; +private: + std::chrono::time_point start_time; + public: void Schedule() override { auto &bfs_state = gstate.global_bfs_state; auto &context = pipeline->GetClientContext(); - bfs_state->change = false; - vector> bfs_tasks; for (idx_t tnum = 0; tnum < bfs_state->num_threads; tnum++) { - // bfs_tasks.push_back(make_uniq(shared_from_this(), context, gstate, tnum)); + bfs_tasks.push_back(make_uniq(shared_from_this(), context, gstate, tnum)); } + start_time = std::chrono::high_resolution_clock::now(); SetTasks(std::move(bfs_tasks)); } void FinishEvent() override { + auto &bfs_state = gstate.global_bfs_state; + auto end_time = std::chrono::high_resolution_clock::now(); + auto duration = std::chrono::duration_cast(end_time - start_time); + bfs_state->time_elapsed += duration; + // if remaining pairs, schedule the BFS for the next batch + if (bfs_state->started_searches < gstate.global_tasks.Count()) { + PhysicalPathFinding::ScheduleBFSTasks(*pipeline, *this, gstate); + } } }; @@ -1031,7 +1131,11 @@ class SequentialShortestPathEvent : public BasePipelineEvent { auto &bfs_state = gstate.global_bfs_state; auto &pairs = *bfs_state->pairs; auto &result = *bfs_state->result_path; + auto start_time = std::chrono::high_resolution_clock::now(); ShortestPathFunction(gstate.global_csr, pairs, result); + auto end_time = std::chrono::high_resolution_clock::now(); + auto duration = std::chrono::duration_cast(end_time - start_time); + bfs_state->time_elapsed = duration; } private: @@ -1248,7 +1352,6 @@ PhysicalPathFinding::Finalize(Pipeline &pipeline, Event &event, if (gstate.child == 0) { auto csr_event = make_shared(gstate, pipeline); event.InsertEvent(std::move(std::dynamic_pointer_cast(csr_event))); - // gstate.CSRCreateEdge(); } else if (gstate.child == 1 && global_tasks.Count() > 0) { auto all_pairs = make_shared(); DataChunk pairs; @@ -1266,7 +1369,7 @@ PhysicalPathFinding::Finalize(Pipeline &pipeline, Event &event, idx_t num_threads = ts.NumberOfThreads(); gstate.global_bfs_state = make_uniq(all_pairs, csr->v_size - 2, num_threads); - // auto bfs_event = make_shared(gstate, pipeline); + // auto bfs_event = make_shared(gstate, pipeline); // event.InsertEvent(std::move(std::dynamic_pointer_cast(bfs_event))); // Schedule the first round of BFS tasks @@ -1309,17 +1412,13 @@ void PhysicalPathFinding::ScheduleBFSTasks(Pipeline &pipeline, Event &event, Glo } else { bfs_state->frontier_size++; bfs_state->visit1[bfs_state->src[src_pos]][lane / 64] |= ((idx_t)1 << (lane % 64)); - // auto new_visit = bfs_state->visit1[bfs_state->src[src_pos]].load(); - // new_visit[lane] = true; - // bfs_state->visit1[bfs_state->src[src_pos]].store(new_visit); - // bfs_state->visit1[bfs_state->src[src_pos]][lane] = true; bfs_state->lane_to_num[lane] = search_num; // active lane break; } } } - auto bfs_event = make_shared(gstate, pipeline); + auto bfs_event = make_shared(gstate, pipeline); event.InsertEvent(std::move(std::dynamic_pointer_cast(bfs_event))); } } @@ -1395,7 +1494,7 @@ PhysicalPathFinding::GetData(ExecutionContext &context, DataChunk &result, // auto &pf_gstate = input.global_state.Cast(); auto &pf_bfs_state = pf_sink.global_bfs_state; // auto &pf_lstate = input.local_state.Cast(); - pf_bfs_state->result_length->Print(pf_bfs_state->pairs->size()); + pf_bfs_state->result_path->Print(pf_bfs_state->pairs->size()); string message = "Algorithm running time: " + to_string(pf_bfs_state->time_elapsed.count()) + " us"; Printer::Print(message); // pf_gstate.Initialize(pf_sink); From fe6d4a062c09a951f0ca29dd6b8e271930de965a Mon Sep 17 00:00:00 2001 From: dtenwolde Date: Mon, 29 Apr 2024 14:44:41 +0200 Subject: [PATCH 103/249] Fix print --- src/operators/physical_path_finding_operator.cpp | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/src/operators/physical_path_finding_operator.cpp b/src/operators/physical_path_finding_operator.cpp index 09a3600a..67e9a75e 100644 --- a/src/operators/physical_path_finding_operator.cpp +++ b/src/operators/physical_path_finding_operator.cpp @@ -1493,7 +1493,7 @@ PhysicalPathFinding::GetData(ExecutionContext &context, DataChunk &result, return SourceResultType::FINISHED; } pf_bfs_state->result.SetCardinality(*pf_bfs_state->pairs); - pf_bfs_state->result_path->Print(pf_bfs_state->pairs->size()); + pf_bfs_state->result.Print(); string message = "Algorithm running time: " + to_string(pf_bfs_state->time_elapsed.count()) + " us"; Printer::Print(message); From 8245e9ee61f15cc520f76630ac6fa8bdc048d499 Mon Sep 17 00:00:00 2001 From: dtenwolde Date: Mon, 29 Apr 2024 17:26:49 +0200 Subject: [PATCH 104/249] Add shortestpath udf with only two parameters --- src/functions/scalar/shortest_path.cpp | 20 ++++++++++++++++---- 1 file changed, 16 insertions(+), 4 deletions(-) diff --git a/src/functions/scalar/shortest_path.cpp b/src/functions/scalar/shortest_path.cpp index 9cf0ad99..2ff7dcca 100644 --- a/src/functions/scalar/shortest_path.cpp +++ b/src/functions/scalar/shortest_path.cpp @@ -230,13 +230,25 @@ static void ShortestPathFunction(DataChunk &args, ExpressionState &state, } CreateScalarFunctionInfo DuckPGQFunctions::GetShortestPathFunction() { - auto fun = ScalarFunction("shortestpath", - {LogicalType::INTEGER, LogicalType::BIGINT, + + ScalarFunctionSet set("shortestpath"); + + set.AddFunction(ScalarFunction({LogicalType::INTEGER, LogicalType::BIGINT, LogicalType::BIGINT, LogicalType::BIGINT}, LogicalType::LIST(LogicalType::BIGINT), ShortestPathFunction, - IterativeLengthFunctionData::IterativeLengthBind); - return CreateScalarFunctionInfo(fun); + IterativeLengthFunctionData::IterativeLengthBind)); + set.AddFunction(ScalarFunction({LogicalType::INTEGER, LogicalType::BIGINT, + LogicalType::BIGINT, LogicalType::BIGINT}, + LogicalType::LIST(LogicalType::BIGINT), + ShortestPathFunction, + IterativeLengthFunctionData::IterativeLengthBind)); + + set.AddFunction(ScalarFunction({LogicalType::BIGINT, LogicalType::BIGINT}, + LogicalType::LIST(LogicalType::BIGINT), ShortestPathFunction, + IterativeLengthFunctionData::IterativeLengthBind)); + + return CreateScalarFunctionInfo(set); } }; // namespace duckdb From 7b839e84fe2f1ea232ed5ea987daf79298f32a88 Mon Sep 17 00:00:00 2001 From: dtenwolde Date: Mon, 29 Apr 2024 17:26:54 +0200 Subject: [PATCH 105/249] Update the test results --- .../path_finding/parallel_path_finding.test | 21 +++++++++++++++++++ 1 file changed, 21 insertions(+) diff --git a/test/sql/path_finding/parallel_path_finding.test b/test/sql/path_finding/parallel_path_finding.test index 09d03bf3..65ca1ad5 100644 --- a/test/sql/path_finding/parallel_path_finding.test +++ b/test/sql/path_finding/parallel_path_finding.test @@ -54,6 +54,27 @@ know SOURCE KEY (src) REFERENCES student (id) 1 2 2 2 0 1 + query III + SELECT *, shortestpath(src, dst) as path + FROM pair AS p + WHERE p.src BETWEEN (SELECT CREATE_CSR_EDGE( + (SELECT count(a.id) as v_size FROM Student a), + (SELECT count(k.src) as e_size from know k), + a.rowid, + c.rowid, + k.rowid, + t.cnt) FROM Know k + JOIN student a on a.id = k.src + JOIN student c on c.id = k.dst + JOIN (SELECT count(k.src) cnt, a.rowid as a_rowid + FROM student a + LEFT JOIN know k ON k.src = a.id + GROUP BY a.rowid) t + ON t.a_rowid = a.rowid) AND p.dst; +---- +0 1 NULL +1 2 [1, 3, 0, 1, 2] +2 0 [2, 4, 0] statement ok import database 'duckdb-pgq/data/SNB0.003'; From 7a5df925bafc4a529258a17db367fa144c5be6db Mon Sep 17 00:00:00 2001 From: dtenwolde Date: Mon, 29 Apr 2024 17:27:15 +0200 Subject: [PATCH 106/249] In optimizer check for both iterativelength and shortestpath udfs --- src/include/duckpgq_extension.hpp | 20 ++++++++++++++------ 1 file changed, 14 insertions(+), 6 deletions(-) diff --git a/src/include/duckpgq_extension.hpp b/src/include/duckpgq_extension.hpp index ad8f2fb6..5f5551ca 100644 --- a/src/include/duckpgq_extension.hpp +++ b/src/include/duckpgq_extension.hpp @@ -58,13 +58,9 @@ class DuckpgqOptimizerExtension : public OptimizerExtension { path_finding_children.push_back(std::move(get_projection.children[0])); path_finding_expressions = std::move(get_function_expression.children); - - auto path_finding_operator = - make_uniq(path_finding_children, path_finding_expressions); - op.children.clear(); - op.children.push_back(std::move(path_finding_operator)); - + string mode; // Iterate in reverse to not influence the upcoming iterations when erasing an element from the list. + // Does not work if both iterativelength and shortestpath are called in the same query for now. To be improved in the future. for (int64_t i = op.expressions.size() - 1; i >= 0; --i) { const auto& expr = op.expressions[i]; if (expr->expression_class == ExpressionClass::BOUND_FUNCTION && @@ -72,8 +68,20 @@ class DuckpgqOptimizerExtension : public OptimizerExtension { op.expressions.emplace_back(make_uniq( expr->alias, LogicalType::BIGINT, ColumnBinding(10, 0))); op.expressions.erase(op.expressions.begin() + i); + mode = "iterativelength"; + } else if (expr->expression_class == ExpressionClass::BOUND_FUNCTION && + expr->Cast().function.name == "shortestpath") { + op.expressions.emplace_back(make_uniq( + expr->alias, LogicalType::LIST(LogicalType::BIGINT), ColumnBinding(10, 0))); + op.expressions.erase(op.expressions.begin() + i); + mode = "shortestpath"; } } + + auto path_finding_operator = + make_uniq(path_finding_children, path_finding_expressions, mode); + op.children.clear(); + op.children.push_back(std::move(path_finding_operator)); return true; } } From 608c872f24914d5293ba5b412207c9010c487e8a Mon Sep 17 00:00:00 2001 From: dtenwolde Date: Mon, 29 Apr 2024 17:27:36 +0200 Subject: [PATCH 107/249] Push either bigint or list(bigint) as types --- .../operators/logical_path_finding_operator.hpp | 6 +++++- src/operators/logical_path_finding_operator.cpp | 12 ++++++++---- 2 files changed, 13 insertions(+), 5 deletions(-) diff --git a/src/include/duckpgq/operators/logical_path_finding_operator.hpp b/src/include/duckpgq/operators/logical_path_finding_operator.hpp index aad63865..a98613a4 100644 --- a/src/include/duckpgq/operators/logical_path_finding_operator.hpp +++ b/src/include/duckpgq/operators/logical_path_finding_operator.hpp @@ -5,9 +5,10 @@ namespace duckdb { class LogicalPathFindingOperator : public LogicalExtensionOperator { public: - explicit LogicalPathFindingOperator(vector> &children_, vector> &expressions_) + explicit LogicalPathFindingOperator(vector> &children_, vector> &expressions_, const string& mode_) : LogicalExtensionOperator(std::move(expressions_)) { children = std::move(children_); + mode = mode_; } void Serialize(Serializer &serializer) const override { @@ -24,6 +25,9 @@ class LogicalPathFindingOperator : public LogicalExtensionOperator { void ResolveTypes() override; string ParamsToString() const override; + +public: + string mode; }; } // namespace duckdb diff --git a/src/operators/logical_path_finding_operator.cpp b/src/operators/logical_path_finding_operator.cpp index f38e7ec6..b3bef44f 100644 --- a/src/operators/logical_path_finding_operator.cpp +++ b/src/operators/logical_path_finding_operator.cpp @@ -12,6 +12,7 @@ unique_ptr LogicalPathFindingOperator::CreatePlan( return make_uniq(*this, std::move(left), std::move(right)); } + vector LogicalPathFindingOperator::GetColumnBindings() { auto left_bindings = children[0]->GetColumnBindings(); left_bindings.push_back(ColumnBinding(10, 0)); @@ -20,10 +21,13 @@ vector LogicalPathFindingOperator::GetColumnBindings() { void LogicalPathFindingOperator::ResolveTypes() { types = children[0]->types; - types.push_back(LogicalType::BIGINT); - // auto right_types = children[1]->types; - // types.insert(types.end(), right_types.begin(), right_types.end()); - // types = {LogicalType::BIGINT, LogicalType::BIGINT}; + if (mode == "iterativelength") { + types.push_back(LogicalType::BIGINT); + } else if (mode == "shortestpath") { + types.push_back(LogicalType::LIST(LogicalType::BIGINT)); + } else { + throw NotImplementedException("Unrecognized mode in PathFindingOperator: " + mode); + } } string LogicalPathFindingOperator::ParamsToString() const { From 0ccb1d8c5259f56f4456b26b0babddecac695b15 Mon Sep 17 00:00:00 2001 From: dtenwolde Date: Mon, 29 Apr 2024 17:28:01 +0200 Subject: [PATCH 108/249] Add mode to physical path finding operator and return correct column depending on mode --- .../physical_path_finding_operator.hpp | 4 +- .../physical_path_finding_operator.cpp | 38 +++++++++++++------ 2 files changed, 30 insertions(+), 12 deletions(-) diff --git a/src/include/duckpgq/operators/physical_path_finding_operator.hpp b/src/include/duckpgq/operators/physical_path_finding_operator.hpp index 9509966b..47df55ca 100644 --- a/src/include/duckpgq/operators/physical_path_finding_operator.hpp +++ b/src/include/duckpgq/operators/physical_path_finding_operator.hpp @@ -23,7 +23,6 @@ class PhysicalPathFinding : public PhysicalComparisonJoin { public: GlobalCompressedSparseRow(ClientContext &context){ }; - atomic *v; atomic *reverse_v; vector e; @@ -97,12 +96,15 @@ class PhysicalPathFinding : public PhysicalComparisonJoin { static constexpr const PhysicalOperatorType TYPE = PhysicalOperatorType::EXTENSION; + public: PhysicalPathFinding(LogicalExtensionOperator &op, unique_ptr left, unique_ptr right); public: vector> expressions; + string mode; // "iterativelength" or "shortestpath" + public: // CachingOperator Interface diff --git a/src/operators/physical_path_finding_operator.cpp b/src/operators/physical_path_finding_operator.cpp index 67e9a75e..e62d30dd 100644 --- a/src/operators/physical_path_finding_operator.cpp +++ b/src/operators/physical_path_finding_operator.cpp @@ -8,11 +8,12 @@ #include "duckdb/parallel/event.hpp" #include "duckdb/parallel/meta_pipeline.hpp" #include "duckdb/parallel/thread_context.hpp" -#include -#include #include #include +#include +#include #include +#include namespace duckdb { @@ -23,6 +24,8 @@ PhysicalPathFinding::PhysicalPathFinding(LogicalExtensionOperator &op, children.push_back(std::move(left)); children.push_back(std::move(right)); expressions = std::move(op.expressions); + auto &path_finding_op = op.Cast(); + mode = path_finding_op.mode; } void PhysicalPathFinding::GlobalCompressedSparseRow::InitializeVertex( @@ -152,7 +155,7 @@ class GlobalBFSState { seen(v_size_), visit1(v_size_), visit2(v_size_), context(context_), total_len(0), parents_v(v_size_, std::vector(LANE_LIMIT, -1)), parents_e(v_size_, std::vector(LANE_LIMIT, -1)), frontier_size(0), unseen_size(v_size_), num_threads(num_threads_), task_queues(num_threads_), barrier(num_threads_) { - result.Initialize(context, {LogicalType::BIGINT, LogicalType(LogicalType::BIGINT)}, STANDARD_VECTOR_SIZE); + result.Initialize(context, {LogicalType::BIGINT, LogicalType::LIST(LogicalType::BIGINT)}, STANDARD_VECTOR_SIZE); for (auto i = 0; i < LANE_LIMIT; i++) { lane_to_num[i] = -1; } @@ -262,20 +265,22 @@ class GlobalBFSState { class PathFindingGlobalState : public GlobalSinkState { public: - using GlobalCompressedSparseRow = PhysicalPathFinding::GlobalCompressedSparseRow; + using GlobalCompressedSparseRow = + PhysicalPathFinding::GlobalCompressedSparseRow; PathFindingGlobalState(ClientContext &context, const PhysicalPathFinding &op) { global_tasks = make_uniq(context, op.children[0]->GetTypes()); global_inputs = make_uniq(context, op.children[1]->GetTypes()); global_csr = make_uniq(context); child = 0; + mode = op.mode; } PathFindingGlobalState(PathFindingGlobalState &prev) : GlobalSinkState(prev), global_tasks(std::move(prev.global_tasks)), global_inputs(std::move(prev.global_inputs)), global_csr(std::move(prev.global_csr)), - global_bfs_state(std::move(prev.global_bfs_state)), child(prev.child + 1) { + global_bfs_state(std::move(prev.global_bfs_state)), child(prev.child + 1), mode(prev.mode) { } void Sink(DataChunk &input, PathFindingLocalState &lstate) const { @@ -315,7 +320,7 @@ class PathFindingGlobalState : public GlobalSinkState { unique_ptr global_csr; // state for BFS unique_ptr global_bfs_state; - + string mode; size_t child; }; @@ -1417,9 +1422,14 @@ void PhysicalPathFinding::ScheduleBFSTasks(Pipeline &pipeline, Event &event, } } } + if (gstate.mode == "iterativelength") { + auto bfs_event = make_shared(gstate, pipeline); + event.InsertEvent(std::move(std::dynamic_pointer_cast(bfs_event))); + } else if (gstate.mode == "shortestpath") { + auto bfs_event = make_shared(gstate, pipeline); + event.InsertEvent(std::move(std::dynamic_pointer_cast(bfs_event))); + } - auto bfs_event = make_shared(gstate, pipeline); - event.InsertEvent(std::move(std::dynamic_pointer_cast(bfs_event))); } } @@ -1493,15 +1503,21 @@ PhysicalPathFinding::GetData(ExecutionContext &context, DataChunk &result, return SourceResultType::FINISHED; } pf_bfs_state->result.SetCardinality(*pf_bfs_state->pairs); - pf_bfs_state->result.Print(); + // pf_bfs_state->result.Print(); string message = "Algorithm running time: " + to_string(pf_bfs_state->time_elapsed.count()) + " us"; Printer::Print(message); result.Move(*pf_bfs_state->pairs); auto result_path = make_uniq(); - //! Split off the path from the path length, and then fuse the path length into the result DataChunk + //! Split off the path from the path length, and then fuse into the result pf_bfs_state->result.Split(*result_path, 1); - result.Fuse(pf_bfs_state->result); + if (pf_sink.mode == "iterativelength") { + result.Fuse(pf_bfs_state->result); + } else if (pf_sink.mode == "shortestpath") { + result.Fuse(*result_path); + } else { + throw NotImplementedException("Unrecognized mode for Path Finding"); + } // result.Print(); return result.size() == 0 ? SourceResultType::FINISHED From 9e6431985d0b5103f0af7b245f525b5ee004148a Mon Sep 17 00:00:00 2001 From: Pingan Ren Date: Tue, 30 Apr 2024 23:44:48 +0200 Subject: [PATCH 109/249] add spin barrier and cas --- .../physical_path_finding_operator.cpp | 92 ++++++++++++------- 1 file changed, 61 insertions(+), 31 deletions(-) diff --git a/src/operators/physical_path_finding_operator.cpp b/src/operators/physical_path_finding_operator.cpp index e62d30dd..f6d17aab 100644 --- a/src/operators/physical_path_finding_operator.cpp +++ b/src/operators/physical_path_finding_operator.cpp @@ -120,32 +120,59 @@ void PathFindingLocalState::CreateCSRVertex(DataChunk &input, }); } +// class Barrier { +// public: +// explicit Barrier(std::size_t iCount) : +// mThreshold(iCount), +// mCount(iCount), +// mGeneration(0) { +// } + +// void Wait() { +// std::unique_lock lLock{mMutex}; +// auto lGen = mGeneration; +// if (!--mCount) { +// mGeneration++; +// mCount = mThreshold; +// mCond.notify_all(); +// } else { +// mCond.wait(lLock, [this, lGen] { return lGen != mGeneration; }); +// } +// } + +// private: +// std::mutex mMutex; +// std::condition_variable mCond; +// std::size_t mThreshold; +// std::size_t mCount; +// std::size_t mGeneration; +// }; + class Barrier { public: - explicit Barrier(std::size_t iCount) : - mThreshold(iCount), - mCount(iCount), - mGeneration(0) { - } + explicit Barrier(std::size_t iCount) : + mThreshold(iCount), + mCount(iCount), + mGeneration(0) { + } - void Wait() { - std::unique_lock lLock{mMutex}; - auto lGen = mGeneration; - if (!--mCount) { - mGeneration++; - mCount = mThreshold; - mCond.notify_all(); - } else { - mCond.wait(lLock, [this, lGen] { return lGen != mGeneration; }); - } + void Wait() { + auto lGen = mGeneration.load(); + if (!--mCount) { + mCount = mThreshold; + ++mGeneration; + } else { + while (lGen == mGeneration.load()) { + std::this_thread::yield(); + } } + } private: - std::mutex mMutex; - std::condition_variable mCond; - std::size_t mThreshold; - std::size_t mCount; - std::size_t mGeneration; + std::mutex mMutex; + std::size_t mThreshold; + std::atomic mCount; + std::atomic mGeneration; }; class GlobalBFSState { @@ -387,8 +414,6 @@ class PhysicalIterativeTopDownTask : public ExecutorTask { BoundaryCalculation(); do { - barrier.Wait(); - change = false; InitTask(); auto& visit = iter & 1 ? bfs_state->visit1 : bfs_state->visit2; @@ -400,7 +425,7 @@ class PhysicalIterativeTopDownTask : public ExecutorTask { if (worker_id == 0) { ReachDetect(result_data); } - + barrier.Wait(); } while (change); if (worker_id == 0) { @@ -463,18 +488,24 @@ class PhysicalIterativeTopDownTask : public ExecutorTask { auto start = task.first; auto end = task.second; + idx_t old_next, new_next; + for (auto i = start; i < end; i++) { for (auto j = 0; j < 8; j++) { if (visit[i][j].load(std::memory_order_relaxed)) { for (auto offset = v[i]; offset < v[i + 1]; offset++) { auto n = e[offset]; - next[n][j].store(next[n][j].load(std::memory_order_relaxed) | visit[i][j].load(std::memory_order_relaxed), std::memory_order_relaxed); + do { + old_next = next[n][j].load(); + new_next = old_next | visit[i][j].load(); + } while (!next[n][j].compare_exchange_weak(old_next, new_next)); } } } } } + change = false; barrier.Wait(); for (auto i = left; i < right; i++) { @@ -947,20 +978,19 @@ class PhysicalShortestPathTopDownTask : public ExecutorTask { auto start = task.first; auto end = task.second; + idx_t old_next, new_next; + for (auto i = start; i < end; i++) { for (auto j = 0; j < 8; j++) { if (visit[i][j].load(std::memory_order_relaxed)) { for (auto offset = v[i]; offset < v[i + 1]; offset++) { auto n = e[offset]; auto edge_id = edge_ids[offset]; - // lock_guard lock(state.global_bfs_state->lock); - // next[n][j] = next[n][j] | visit[i][j]; - next[n][j].store(next[n][j].load(std::memory_order_relaxed) | visit[i][j].load(std::memory_order_relaxed), std::memory_order_relaxed); + do { + old_next = next[n][j].load(); + new_next = old_next | visit[i][j].load(); + } while (!next[n][j].compare_exchange_weak(old_next, new_next)); for (auto l = 0; l < LANE_LIMIT; l++) { - // parents_v[n][l] = ((parents_v[n][l] == -1) && visit[i][l / 64] & ((idx_t)1 << (l % 64))) - // ? i : parents_v[n][l]; - // parents_e[n][l] = ((parents_e[n][l] == -1) && visit[i][l / 64] & ((idx_t)1 << (l % 64))) - // ? edge_id : parents_e[n][l]; parents_v[n][l] = ((parents_v[n][l] == -1) && visit[i][l / 64].load(std::memory_order_relaxed) & ((idx_t)1 << (l % 64))) ? i : parents_v[n][l]; parents_e[n][l] = ((parents_e[n][l] == -1) && visit[i][l / 64].load(std::memory_order_relaxed) & ((idx_t)1 << (l % 64))) From 231bf6363ac508916892ae7542e6ca337cb45a76 Mon Sep 17 00:00:00 2001 From: Pingan Ren Date: Thu, 2 May 2024 12:03:34 +0200 Subject: [PATCH 110/249] bug found when pairs = 1 --- .../path_finding/parallel_path_finding.test | 88 +++++++++---------- 1 file changed, 44 insertions(+), 44 deletions(-) diff --git a/test/sql/path_finding/parallel_path_finding.test b/test/sql/path_finding/parallel_path_finding.test index 65ca1ad5..b19d8fa7 100644 --- a/test/sql/path_finding/parallel_path_finding.test +++ b/test/sql/path_finding/parallel_path_finding.test @@ -32,49 +32,49 @@ know SOURCE KEY (src) REFERENCES student (id) # COLUMNS (*) # ); - query III - SELECT *, iterativelength(src, dst) as length - FROM pair AS p - WHERE p.src BETWEEN (SELECT CREATE_CSR_EDGE( - (SELECT count(a.id) as v_size FROM Student a), - (SELECT count(k.src) as e_size from know k), - a.rowid, - c.rowid, - k.rowid, - t.cnt) FROM Know k - JOIN student a on a.id = k.src - JOIN student c on c.id = k.dst - JOIN (SELECT count(k.src) cnt, a.rowid as a_rowid - FROM student a - LEFT JOIN know k ON k.src = a.id - GROUP BY a.rowid) t - ON t.a_rowid = a.rowid) AND p.dst; ----- -0 1 NULL -1 2 2 -2 0 1 - - query III - SELECT *, shortestpath(src, dst) as path - FROM pair AS p - WHERE p.src BETWEEN (SELECT CREATE_CSR_EDGE( - (SELECT count(a.id) as v_size FROM Student a), - (SELECT count(k.src) as e_size from know k), - a.rowid, - c.rowid, - k.rowid, - t.cnt) FROM Know k - JOIN student a on a.id = k.src - JOIN student c on c.id = k.dst - JOIN (SELECT count(k.src) cnt, a.rowid as a_rowid - FROM student a - LEFT JOIN know k ON k.src = a.id - GROUP BY a.rowid) t - ON t.a_rowid = a.rowid) AND p.dst; ----- -0 1 NULL -1 2 [1, 3, 0, 1, 2] -2 0 [2, 4, 0] +# query III +# SELECT *, iterativelength(src, dst) as length +# FROM pair AS p +# WHERE p.src BETWEEN (SELECT CREATE_CSR_EDGE( +# (SELECT count(a.id) as v_size FROM Student a), +# (SELECT count(k.src) as e_size from know k), +# a.rowid, +# c.rowid, +# k.rowid, +# t.cnt) FROM Know k +# JOIN student a on a.id = k.src +# JOIN student c on c.id = k.dst +# JOIN (SELECT count(k.src) cnt, a.rowid as a_rowid +# FROM student a +# LEFT JOIN know k ON k.src = a.id +# GROUP BY a.rowid) t +# ON t.a_rowid = a.rowid) AND p.dst; +# ---- +# 0 1 NULL +# 1 2 2 +# 2 0 1 + +# query III +# SELECT *, shortestpath(src, dst) as path +# FROM pair AS p +# WHERE p.src BETWEEN (SELECT CREATE_CSR_EDGE( +# (SELECT count(a.id) as v_size FROM Student a), +# (SELECT count(k.src) as e_size from know k), +# a.rowid, +# c.rowid, +# k.rowid, +# t.cnt) FROM Know k +# JOIN student a on a.id = k.src +# JOIN student c on c.id = k.dst +# JOIN (SELECT count(k.src) cnt, a.rowid as a_rowid +# FROM student a +# LEFT JOIN know k ON k.src = a.id +# GROUP BY a.rowid) t +# ON t.a_rowid = a.rowid) AND p.dst; +# ---- +# 0 1 NULL +# 1 2 [1, 3, 0, 1, 2] +# 2 0 [2, 4, 0] statement ok import database 'duckdb-pgq/data/SNB0.003'; @@ -95,7 +95,7 @@ CREATE OR REPLACE TABLE pairs2 AS ( SELECT src, dst FROM (SELECT a.rowid AS src FROM Person a), (SELECT b.rowid AS dst FROM Person b) - USING SAMPLE reservoir(64 ROWS) REPEATABLE (300) + USING SAMPLE reservoir(1 ROWS) REPEATABLE (300) ); statement ok From 3089c5e5e74f470ea0aacd5705a8c10103304abf Mon Sep 17 00:00:00 2001 From: Pingan Ren Date: Thu, 2 May 2024 13:36:27 +0200 Subject: [PATCH 111/249] solve bug when there are too many results --- src/operators/physical_path_finding_operator.cpp | 4 +--- 1 file changed, 1 insertion(+), 3 deletions(-) diff --git a/src/operators/physical_path_finding_operator.cpp b/src/operators/physical_path_finding_operator.cpp index f6d17aab..fc0e9cdc 100644 --- a/src/operators/physical_path_finding_operator.cpp +++ b/src/operators/physical_path_finding_operator.cpp @@ -182,7 +182,7 @@ class GlobalBFSState { seen(v_size_), visit1(v_size_), visit2(v_size_), context(context_), total_len(0), parents_v(v_size_, std::vector(LANE_LIMIT, -1)), parents_e(v_size_, std::vector(LANE_LIMIT, -1)), frontier_size(0), unseen_size(v_size_), num_threads(num_threads_), task_queues(num_threads_), barrier(num_threads_) { - result.Initialize(context, {LogicalType::BIGINT, LogicalType::LIST(LogicalType::BIGINT)}, STANDARD_VECTOR_SIZE); + result.Initialize(context, {LogicalType::BIGINT, LogicalType::LIST(LogicalType::BIGINT)}, pairs_->size()); for (auto i = 0; i < LANE_LIMIT; i++) { lane_to_num[i] = -1; } @@ -264,8 +264,6 @@ class GlobalBFSState { vector, 8>> seen; vector, 8>> visit1; vector, 8>> visit2; - unique_ptr result_length; - unique_ptr result_path; vector> parents_v; vector> parents_e; From 8e5d0cc6eb3fb98ecb8755316b89b0f25105fe8d Mon Sep 17 00:00:00 2001 From: dtenwolde Date: Mon, 6 May 2024 10:15:47 +0200 Subject: [PATCH 112/249] Add rule to disable path finding operator --- src/duckpgq_extension.cpp | 4 +++- src/include/duckpgq_extension.hpp | 7 ++++++- 2 files changed, 9 insertions(+), 2 deletions(-) diff --git a/src/duckpgq_extension.cpp b/src/duckpgq_extension.cpp index 3435649c..9dd143ee 100644 --- a/src/duckpgq_extension.cpp +++ b/src/duckpgq_extension.cpp @@ -48,7 +48,9 @@ static void LoadInternal(DatabaseInstance &instance) { config.parser_extensions.push_back(pgq_parser); config.operator_extensions.push_back(make_uniq()); config.optimizer_extensions.push_back(DuckpgqOptimizerExtension()); - + config.AddExtensionOption("experimental_path_finding_operator", + "Enables the experimental path finding operator to be triggered", + LogicalType::BOOLEAN, Value(false)); Connection con(instance); con.BeginTransaction(); diff --git a/src/include/duckpgq_extension.hpp b/src/include/duckpgq_extension.hpp index 5f5551ca..7133f15c 100644 --- a/src/include/duckpgq_extension.hpp +++ b/src/include/duckpgq_extension.hpp @@ -96,7 +96,12 @@ class DuckpgqOptimizerExtension : public OptimizerExtension { static void DuckpgqOptimizeFunction(ClientContext &context, OptimizerExtensionInfo *info, duckdb::unique_ptr &plan) { - + Value path_finding_operator_enabled; + context.db->TryGetCurrentSetting("experimental_path_finding_operator", path_finding_operator_enabled); + // If the path finding operator is not enabled, we do not need to do anything + if (!path_finding_operator_enabled.GetValue()) { + return; + } if (!InsertPathFindingOperator(*plan)) { return; } From 8fafd0347c387b514ce13a7205de07950011e3d7 Mon Sep 17 00:00:00 2001 From: dtenwolde Date: Mon, 6 May 2024 10:16:05 +0200 Subject: [PATCH 113/249] Add check if csr exists to prevent seg faults --- src/functions/scalar/csr_creation.cpp | 10 +++++++++- test/sql/path_finding/parallel_path_finding.test | 3 +++ 2 files changed, 12 insertions(+), 1 deletion(-) diff --git a/src/functions/scalar/csr_creation.cpp b/src/functions/scalar/csr_creation.cpp index ce066a69..f3199bcc 100644 --- a/src/functions/scalar/csr_creation.cpp +++ b/src/functions/scalar/csr_creation.cpp @@ -48,6 +48,9 @@ static void CsrInitializeEdge(DuckPGQState &context, int32_t id, int64_t v_size, const lock_guard csr_init_lock(context.csr_lock); auto csr_entry = context.csr_list.find(id); + if (csr_entry == context.csr_list.end()) { + throw InvalidInputException("CSR has not been initialized properly"); + } if (csr_entry->second->initialized_e) { return; } @@ -69,7 +72,9 @@ static void CsrInitializeWeight(DuckPGQState &context, int32_t id, int64_t e_size, PhysicalType weight_type) { const lock_guard csr_init_lock(context.csr_lock); auto csr_entry = context.csr_list.find(id); - + if (csr_entry == context.csr_list.end()) { + throw InvalidInputException("CSR has not been initialized properly"); + } if (csr_entry->second->initialized_w) { return; } @@ -148,6 +153,9 @@ static void CreateCsrEdgeFunction(DataChunk &args, ExpressionState &state, int64_t edge_size = args.data[2].GetValue(0).GetValue(); auto csr_entry = duckpgq_state->csr_list.find(info.id); + if (csr_entry == duckpgq_state->csr_list.end()) { + throw InvalidInputException("CSR has not been initialized properly"); + } if (!csr_entry->second->initialized_e) { CsrInitializeEdge(*duckpgq_state, info.id, vertex_size, edge_size); } diff --git a/test/sql/path_finding/parallel_path_finding.test b/test/sql/path_finding/parallel_path_finding.test index b19d8fa7..9eb726ab 100644 --- a/test/sql/path_finding/parallel_path_finding.test +++ b/test/sql/path_finding/parallel_path_finding.test @@ -109,6 +109,9 @@ EDGE TABLES ( LABEL Knows ); +#statement ok +#set experimental_path_finding_operator = 'true'; + query II SELECT *, iterativelength(src, dst) as length FROM pairs2 AS p From 60f1c1e570e80948dadad8d17e9e1598ee16ed06 Mon Sep 17 00:00:00 2001 From: dtenwolde Date: Mon, 6 May 2024 12:19:24 +0200 Subject: [PATCH 114/249] Make tests pass and set option get the correct value --- src/include/duckpgq_extension.hpp | 13 +- .../path_finding/parallel_path_finding.test | 118 ++++++++---------- 2 files changed, 62 insertions(+), 69 deletions(-) diff --git a/src/include/duckpgq_extension.hpp b/src/include/duckpgq_extension.hpp index 7133f15c..21664b01 100644 --- a/src/include/duckpgq_extension.hpp +++ b/src/include/duckpgq_extension.hpp @@ -14,6 +14,7 @@ #include "duckdb/planner/operator/logical_limit.hpp" #include "duckdb/planner/operator/logical_get.hpp" #include "duckpgq/common.hpp" +#include "duckdb/main/config.hpp" #include "duckpgq/operators/logical_path_finding_operator.hpp" #include "duckpgq/compressed_sparse_row.hpp" #include "duckdb/parser/parsed_data/create_property_graph_info.hpp" @@ -96,15 +97,15 @@ class DuckpgqOptimizerExtension : public OptimizerExtension { static void DuckpgqOptimizeFunction(ClientContext &context, OptimizerExtensionInfo *info, duckdb::unique_ptr &plan) { - Value path_finding_operator_enabled; - context.db->TryGetCurrentSetting("experimental_path_finding_operator", path_finding_operator_enabled); - // If the path finding operator is not enabled, we do not need to do anything - if (!path_finding_operator_enabled.GetValue()) { - return; + auto& client_config = ClientConfig::GetConfig(context); + auto const path_finding_operator_option = client_config.set_variables.find("experimental_path_finding_operator"); + if (path_finding_operator_option == client_config.set_variables.end()) { + return; // If the path finding operator is not enabled, we do not need to do anything } - if (!InsertPathFindingOperator(*plan)) { + if (!path_finding_operator_option->second.GetValue()) { return; } + InsertPathFindingOperator(*plan); } }; diff --git a/test/sql/path_finding/parallel_path_finding.test b/test/sql/path_finding/parallel_path_finding.test index 9eb726ab..3a174a2d 100644 --- a/test/sql/path_finding/parallel_path_finding.test +++ b/test/sql/path_finding/parallel_path_finding.test @@ -7,6 +7,9 @@ require duckpgq +statement ok +set experimental_path_finding_operator=true; + statement ok CREATE TABLE pair(src BIGINT, dst BIGINT); INSERT INTO pair(src, dst) VALUES (0, 1), (1, 2), (2,0); @@ -26,76 +29,62 @@ know SOURCE KEY (src) REFERENCES student (id) DESTINATION KEY (dst) references student (id) ); -# statement ok -# -FROM GRAPH_TABLE(pg -# MATCH (f:student)-[k:knows]->{0,3}(f2:student) -# COLUMNS (*) -# ); - -# query III -# SELECT *, iterativelength(src, dst) as length -# FROM pair AS p -# WHERE p.src BETWEEN (SELECT CREATE_CSR_EDGE( -# (SELECT count(a.id) as v_size FROM Student a), -# (SELECT count(k.src) as e_size from know k), -# a.rowid, -# c.rowid, -# k.rowid, -# t.cnt) FROM Know k -# JOIN student a on a.id = k.src -# JOIN student c on c.id = k.dst -# JOIN (SELECT count(k.src) cnt, a.rowid as a_rowid -# FROM student a -# LEFT JOIN know k ON k.src = a.id -# GROUP BY a.rowid) t -# ON t.a_rowid = a.rowid) AND p.dst; -# ---- -# 0 1 NULL -# 1 2 2 -# 2 0 1 - -# query III -# SELECT *, shortestpath(src, dst) as path -# FROM pair AS p -# WHERE p.src BETWEEN (SELECT CREATE_CSR_EDGE( -# (SELECT count(a.id) as v_size FROM Student a), -# (SELECT count(k.src) as e_size from know k), -# a.rowid, -# c.rowid, -# k.rowid, -# t.cnt) FROM Know k -# JOIN student a on a.id = k.src -# JOIN student c on c.id = k.dst -# JOIN (SELECT count(k.src) cnt, a.rowid as a_rowid -# FROM student a -# LEFT JOIN know k ON k.src = a.id -# GROUP BY a.rowid) t -# ON t.a_rowid = a.rowid) AND p.dst; -# ---- -# 0 1 NULL -# 1 2 [1, 3, 0, 1, 2] -# 2 0 [2, 4, 0] +query III +SELECT *, iterativelength(src, dst) as length +FROM pair AS p +WHERE p.src BETWEEN (SELECT CREATE_CSR_EDGE( + (SELECT count(a.id) as v_size FROM Student a), + (SELECT count(k.src) as e_size from know k), + a.rowid, + c.rowid, + k.rowid, + t.cnt) FROM Know k + JOIN student a on a.id = k.src + JOIN student c on c.id = k.dst + JOIN (SELECT count(k.src) cnt, a.rowid as a_rowid + FROM student a + LEFT JOIN know k ON k.src = a.id + GROUP BY a.rowid) t + ON t.a_rowid = a.rowid) AND p.dst; +---- +0 1 NULL +1 2 2 +2 0 1 + +query III +SELECT *, shortestpath(src, dst) as path +FROM pair AS p +WHERE p.src BETWEEN (SELECT CREATE_CSR_EDGE( + (SELECT count(a.id) as v_size FROM Student a), + (SELECT count(k.src) as e_size from know k), + a.rowid, + c.rowid, + k.rowid, + t.cnt) FROM Know k + JOIN student a on a.id = k.src + JOIN student c on c.id = k.dst + JOIN (SELECT count(k.src) cnt, a.rowid as a_rowid + FROM student a + LEFT JOIN know k ON k.src = a.id + GROUP BY a.rowid) t + ON t.a_rowid = a.rowid) AND p.dst; +---- +0 1 NULL +1 2 [1, 3, 0, 1, 2] +2 0 [2, 4, 0] statement ok import database 'duckdb-pgq/data/SNB0.003'; -# import database 'duckdb-pgq/data/SNB1-projected|'; - - -# statement ok -# DELETE FROM Person -# WHERE rowid NOT IN ( -# SELECT rowid FROM Person -# ORDER BY rowid ASC -# LIMIT 500 -# ); +statement ok +select setseed(0.42) statement ok CREATE OR REPLACE TABLE pairs2 AS ( SELECT src, dst FROM (SELECT a.rowid AS src FROM Person a), (SELECT b.rowid AS dst FROM Person b) - USING SAMPLE reservoir(1 ROWS) REPEATABLE (300) + USING SAMPLE reservoir(2 ROWS) REPEATABLE (300) ); statement ok @@ -109,10 +98,8 @@ EDGE TABLES ( LABEL Knows ); -#statement ok -#set experimental_path_finding_operator = 'true'; -query II +query III SELECT *, iterativelength(src, dst) as length FROM pairs2 AS p WHERE p.src BETWEEN (SELECT CREATE_CSR_EDGE( @@ -130,6 +117,9 @@ WHERE p.src BETWEEN (SELECT CREATE_CSR_EDGE( GROUP BY a.rowid) t ON t.a_rowid = a.rowid) AND p.dst; ---- +42 22 NULL +3 33 2 + query III WITH cte1 AS ( @@ -160,6 +150,8 @@ WITH cte1 AS ( FROM pairs2 p, (select count(cte1.temp) * 0 as temp from cte1) __x WHERE __x.temp * 0 >= 0; ---- +42 22 NULL +3 33 2 # CAST ( # (SELECT sum(CREATE_CSR_VERTEX(0, From 74f54ca066a4c2830aac351a8e1b1832ce637489 Mon Sep 17 00:00:00 2001 From: dtenwolde Date: Tue, 7 May 2024 14:24:30 +0200 Subject: [PATCH 115/249] Add table to scan argument to udf and tests --- src/common.cpp | 9 +- src/functions/scalar/iterativelength.cpp | 2 +- src/functions/scalar/shortest_path.cpp | 2 +- src/include/duckpgq/common.hpp | 5 +- .../path_finding/parallel_path_finding.test | 175 +++++++++++------- 5 files changed, 114 insertions(+), 79 deletions(-) diff --git a/src/common.cpp b/src/common.cpp index bc4502fc..0ae36003 100644 --- a/src/common.cpp +++ b/src/common.cpp @@ -72,7 +72,7 @@ CSRFunctionData::CSRBind(ClientContext &context, ScalarFunction &bound_function, } unique_ptr IterativeLengthFunctionData::Copy() const { - return make_uniq(context, csr_id); + return make_uniq(context, table_to_scan, csr_id); } bool IterativeLengthFunctionData::Equals(const FunctionData &other_p) const { @@ -83,8 +83,9 @@ bool IterativeLengthFunctionData::Equals(const FunctionData &other_p) const { unique_ptr IterativeLengthFunctionData::IterativeLengthBind( ClientContext &context, ScalarFunction &bound_function, vector> &arguments) { - if (arguments.size() == 2) { - return make_uniq(context, 0); + if (arguments.size() == 3) { + string table_to_scan = ExpressionExecutor::EvaluateScalar(context, *arguments[2]).GetValue(); + return make_uniq(context, table_to_scan, 0); } if (!arguments[0]->IsFoldable()) { throw InvalidInputException("Id must be constant."); @@ -93,7 +94,7 @@ unique_ptr IterativeLengthFunctionData::IterativeLengthBind( int32_t csr_id = ExpressionExecutor::EvaluateScalar(context, *arguments[0]) .GetValue(); - return make_uniq(context, csr_id); + return make_uniq(context, "", csr_id); } unique_ptr CheapestPathLengthFunctionData::CheapestPathLengthBind( diff --git a/src/functions/scalar/iterativelength.cpp b/src/functions/scalar/iterativelength.cpp index 0372f9fb..cf66b387 100644 --- a/src/functions/scalar/iterativelength.cpp +++ b/src/functions/scalar/iterativelength.cpp @@ -166,7 +166,7 @@ CreateScalarFunctionInfo DuckPGQFunctions::GetIterativeLengthFunction() { LogicalType::BIGINT, IterativeLengthFunction, IterativeLengthFunctionData::IterativeLengthBind)); - set.AddFunction(ScalarFunction({LogicalType::BIGINT, LogicalType::BIGINT}, + set.AddFunction(ScalarFunction({LogicalType::BIGINT, LogicalType::BIGINT, LogicalType::VARCHAR}, LogicalType::BIGINT, IterativeLengthFunction, IterativeLengthFunctionData::IterativeLengthBind)); diff --git a/src/functions/scalar/shortest_path.cpp b/src/functions/scalar/shortest_path.cpp index 2ff7dcca..6abac2a3 100644 --- a/src/functions/scalar/shortest_path.cpp +++ b/src/functions/scalar/shortest_path.cpp @@ -244,7 +244,7 @@ CreateScalarFunctionInfo DuckPGQFunctions::GetShortestPathFunction() { ShortestPathFunction, IterativeLengthFunctionData::IterativeLengthBind)); - set.AddFunction(ScalarFunction({LogicalType::BIGINT, LogicalType::BIGINT}, + set.AddFunction(ScalarFunction({LogicalType::BIGINT, LogicalType::BIGINT, LogicalType::VARCHAR}, LogicalType::LIST(LogicalType::BIGINT), ShortestPathFunction, IterativeLengthFunctionData::IterativeLengthBind)); diff --git a/src/include/duckpgq/common.hpp b/src/include/duckpgq/common.hpp index 7f3fb82d..c2e0ca31 100644 --- a/src/include/duckpgq/common.hpp +++ b/src/include/duckpgq/common.hpp @@ -45,10 +45,11 @@ struct CSRFunctionData : public FunctionData { struct IterativeLengthFunctionData : public FunctionData { public: ClientContext &context; + string table_to_scan; int32_t csr_id; - IterativeLengthFunctionData(ClientContext &context, int32_t csr_id) - : context(context), csr_id(csr_id) {} + IterativeLengthFunctionData(ClientContext &context, string table_to_scan, int32_t csr_id) + : context(context), table_to_scan(table_to_scan), csr_id(csr_id) {} static unique_ptr IterativeLengthBind(ClientContext &context, ScalarFunction &bound_function, vector> &arguments); diff --git a/test/sql/path_finding/parallel_path_finding.test b/test/sql/path_finding/parallel_path_finding.test index 3a174a2d..ea3ace2b 100644 --- a/test/sql/path_finding/parallel_path_finding.test +++ b/test/sql/path_finding/parallel_path_finding.test @@ -19,40 +19,40 @@ create table student(id INT); INSERT INTO student(id) VALUES (10), (20), (30), ( statement ok create table know(src INT, dst INT); INSERT INTO know(src, dst) VALUES (40, 20), (10,30), (10,10), (20,10), (30,10); -statement ok --CREATE PROPERTY GRAPH pg -VERTEX TABLES ( -student -) -EDGE TABLES ( -know SOURCE KEY (src) REFERENCES student (id) - DESTINATION KEY (dst) references student (id) -); - -query III -SELECT *, iterativelength(src, dst) as length -FROM pair AS p -WHERE p.src BETWEEN (SELECT CREATE_CSR_EDGE( - (SELECT count(a.id) as v_size FROM Student a), - (SELECT count(k.src) as e_size from know k), - a.rowid, - c.rowid, - k.rowid, - t.cnt) FROM Know k - JOIN student a on a.id = k.src - JOIN student c on c.id = k.dst - JOIN (SELECT count(k.src) cnt, a.rowid as a_rowid - FROM student a - LEFT JOIN know k ON k.src = a.id - GROUP BY a.rowid) t - ON t.a_rowid = a.rowid) AND p.dst; ----- -0 1 NULL -1 2 2 -2 0 1 +#statement ok +#-CREATE PROPERTY GRAPH pg +#VERTEX TABLES ( +#student +#) +#EDGE TABLES ( +#know SOURCE KEY (src) REFERENCES student (id) +# DESTINATION KEY (dst) references student (id) +#); +# +#query III +#SELECT *, iterativelength(src, dst) as length +#FROM pair AS p +#WHERE p.src BETWEEN (SELECT CREATE_CSR_EDGE( +# (SELECT count(a.id) as v_size FROM Student a), +# (SELECT count(k.src) as e_size from know k), +# a.rowid, +# c.rowid, +# k.rowid, +# t.cnt) FROM Know k +# JOIN student a on a.id = k.src +# JOIN student c on c.id = k.dst +# JOIN (SELECT count(k.src) cnt, a.rowid as a_rowid +# FROM student a +# LEFT JOIN know k ON k.src = a.id +# GROUP BY a.rowid) t +# ON t.a_rowid = a.rowid) AND p.dst; +#---- +#0 1 NULL +#1 2 2 +#2 0 1 query III -SELECT *, shortestpath(src, dst) as path +SELECT *, shortestpath(src, dst, 'pair') as path FROM pair AS p WHERE p.src BETWEEN (SELECT CREATE_CSR_EDGE( (SELECT count(a.id) as v_size FROM Student a), @@ -81,26 +81,26 @@ select setseed(0.42) statement ok CREATE OR REPLACE TABLE pairs2 AS ( - SELECT src, dst + SELECT src, dst FROM (SELECT a.rowid AS src FROM Person a), (SELECT b.rowid AS dst FROM Person b) USING SAMPLE reservoir(2 ROWS) REPEATABLE (300) ); -statement ok --CREATE PROPERTY GRAPH snb -VERTEX TABLES ( - Person PROPERTIES (id) LABEL Person - ) -EDGE TABLES ( - Person_knows_Person SOURCE KEY (Person1Id) REFERENCES Person (id) - DESTINATION KEY (Person2Id) REFERENCES Person (id) - LABEL Knows - ); +#statement ok +#-CREATE PROPERTY GRAPH snb +#VERTEX TABLES ( +# Person PROPERTIES (id) LABEL Person +# ) +#EDGE TABLES ( +# Person_knows_Person SOURCE KEY (Person1Id) REFERENCES Person (id) +# DESTINATION KEY (Person2Id) REFERENCES Person (id) +# LABEL Knows +# ); query III -SELECT *, iterativelength(src, dst) as length +SELECT *, iterativelength(src, dst, 'pairs2') as length FROM pairs2 AS p WHERE p.src BETWEEN (SELECT CREATE_CSR_EDGE( (SELECT count(a.id) as v_size FROM Person a), @@ -121,37 +121,70 @@ WHERE p.src BETWEEN (SELECT CREATE_CSR_EDGE( 3 33 2 +#query III +#WITH cte1 AS ( +# SELECT CREATE_CSR_EDGE( +# 0, +# (SELECT count(a.id) FROM Person a), +# CAST ( +# (SELECT sum(CREATE_CSR_VERTEX( +# 0, +# (SELECT count(a.id) FROM Person a), +# sub.dense_id, +# sub.cnt) +# ) +# FROM ( +# SELECT a.rowid as dense_id, count(k.Person1Id) as cnt +# FROM Person a +# LEFT JOIN Person_knows_Person k ON k.Person1Id = a.id +# GROUP BY a.rowid) sub +# ) +# AS BIGINT), +# a.rowid, +# c.rowid, +# k.rowid) as temp +# FROM Person_knows_Person k +# JOIN Person a on a.id = k.Person1Id +# JOIN Person c on c.id = k.Person2Id +#) SELECT p.src, p.dst, iterativelength(0, (select count(*) from Person), p.src, p.dst) as length +# FROM pairs2 p, (select count(cte1.temp) * 0 as temp from cte1) __x +# WHERE __x.temp * 0 >= 0; +#---- +#42 22 NULL +#3 33 2 + +statement ok +CREATE OR REPLACE TABLE pairs3 AS ( + SELECT src, dst + FROM (SELECT a.rowid AS src FROM Person a), + (SELECT b.rowid AS dst FROM Person b) + USING SAMPLE reservoir(1 ROWS) REPEATABLE (300) +); + +query II +from pairs3; +---- +10 2 + query III -WITH cte1 AS ( - SELECT CREATE_CSR_EDGE( - 0, - (SELECT count(a.id) FROM Person a), - CAST ( - (SELECT sum(CREATE_CSR_VERTEX( - 0, - (SELECT count(a.id) FROM Person a), - sub.dense_id, - sub.cnt) - ) - FROM ( - SELECT a.rowid as dense_id, count(k.Person1Id) as cnt - FROM Person a - LEFT JOIN Person_knows_Person k ON k.Person1Id = a.id - GROUP BY a.rowid) sub - ) - AS BIGINT), +SELECT *, iterativelength(src, dst, 'pairs3') as length +FROM pairs3 as p +WHERE p.src BETWEEN (SELECT CREATE_CSR_EDGE( + (SELECT count(a.id) as v_size FROM Person a), + (SELECT count(k.Person1Id) as e_size FROM Person_knows_Person k), a.rowid, c.rowid, - k.rowid) as temp - FROM Person_knows_Person k - JOIN Person a on a.id = k.Person1Id - JOIN Person c on c.id = k.Person2Id -) SELECT p.src, p.dst, iterativelength(0, (select count(*) from Person), p.src, p.dst) as length - FROM pairs2 p, (select count(cte1.temp) * 0 as temp from cte1) __x - WHERE __x.temp * 0 >= 0; + k.rowid, + t.cnt) FROM Person_knows_Person k + JOIN Person a on a.id = k.Person1Id + JOIN Person c on c.id = k.Person2Id + JOIN (SELECT count(k.Person1Id) cnt, a.rowid as a_rowid + FROM Person a + LEFT JOIN Person_knows_Person k ON k.Person1Id = a.id + GROUP BY a.rowid) t + ON t.a_rowid = a.rowid) AND p.dst; ---- -42 22 NULL -3 33 2 +10 2 NULL # CAST ( # (SELECT sum(CREATE_CSR_VERTEX(0, From 8230603c7c14d67a5c547476c1ac38bcd9bed59a Mon Sep 17 00:00:00 2001 From: dtenwolde Date: Tue, 7 May 2024 14:24:42 +0200 Subject: [PATCH 116/249] Update the optimizer rule to also work with 1 pair --- src/include/duckpgq_extension.hpp | 172 ++++++++++++++++++++---------- 1 file changed, 114 insertions(+), 58 deletions(-) diff --git a/src/include/duckpgq_extension.hpp b/src/include/duckpgq_extension.hpp index 21664b01..eeea9f92 100644 --- a/src/include/duckpgq_extension.hpp +++ b/src/include/duckpgq_extension.hpp @@ -1,5 +1,6 @@ #pragma once +#include "duckdb/main/config.hpp" #include "duckdb/parser/column_list.hpp" #include "duckdb/parser/parsed_data/create_property_graph_info.hpp" #include "duckdb/parser/parsed_expression.hpp" @@ -7,17 +8,19 @@ #include "duckdb/parser/simplified_token.hpp" #include "duckdb/parser/sql_statement.hpp" #include "duckdb/planner/logical_operator.hpp" +#include "duckdb/planner/operator/logical_aggregate.hpp" #include "duckdb/planner/operator/logical_comparison_join.hpp" +#include "duckdb/planner/operator/logical_empty_result.hpp" #include "duckdb/planner/operator/logical_filter.hpp" -#include "duckdb/planner/operator/logical_aggregate.hpp" -#include "duckdb/planner/operator/logical_projection.hpp" -#include "duckdb/planner/operator/logical_limit.hpp" #include "duckdb/planner/operator/logical_get.hpp" +#include "duckdb/planner/operator/logical_limit.hpp" +#include "duckdb/planner/operator/logical_projection.hpp" #include "duckpgq/common.hpp" -#include "duckdb/main/config.hpp" -#include "duckpgq/operators/logical_path_finding_operator.hpp" #include "duckpgq/compressed_sparse_row.hpp" -#include "duckdb/parser/parsed_data/create_property_graph_info.hpp" +#include "duckpgq/operators/logical_path_finding_operator.hpp" +#include "duckdb/function/table/table_scan.hpp" +#include "duckdb/main/database_manager.hpp" +#include "duckdb/catalog/catalog_entry/duck_table_entry.hpp" namespace duckdb { @@ -33,62 +36,115 @@ class DuckpgqOptimizerExtension : public OptimizerExtension { optimize_function = DuckpgqOptimizeFunction; } - static bool InsertPathFindingOperator(LogicalOperator &op) { - vector> path_finding_children; - vector> path_finding_expressions; - for (auto &child : op.children) { - if (child->type == LogicalOperatorType::LOGICAL_COMPARISON_JOIN) { - auto &get_join = child->Cast(); - //! For now we assume this is enough to detect we have found a path-finding query - //! Should be improved in the future - if (get_join.children[0]->type == LogicalOperatorType::LOGICAL_FILTER && - get_join.children[1]->type == LogicalOperatorType::LOGICAL_AGGREGATE_AND_GROUP_BY) { - //! Structure is Aggregate --> Limit --> Projection(create_csr_edge(...)) - //! First check if we find the create_csr_edge function on the right side. - auto &get_aggregate = get_join.children[1]->Cast(); - auto &get_limit = get_aggregate.children[0]->Cast(); - auto &get_projection = get_limit.children[0]->Cast(); - auto &get_function_expression = get_projection.expressions[0]->Cast(); - if (get_function_expression.function.name != "create_csr_edge") { - continue; - } - auto &get_filter = get_join.children[0]->Cast(); - if (get_filter.children[0]->type == LogicalOperatorType::LOGICAL_GET) { - path_finding_children.push_back(std::move(get_filter.children[0])); - } - - path_finding_children.push_back(std::move(get_projection.children[0])); - path_finding_expressions = std::move(get_function_expression.children); - string mode; - // Iterate in reverse to not influence the upcoming iterations when erasing an element from the list. - // Does not work if both iterativelength and shortestpath are called in the same query for now. To be improved in the future. - for (int64_t i = op.expressions.size() - 1; i >= 0; --i) { - const auto& expr = op.expressions[i]; - if (expr->expression_class == ExpressionClass::BOUND_FUNCTION && - expr->Cast().function.name == "iterativelength") { - op.expressions.emplace_back(make_uniq( - expr->alias, LogicalType::BIGINT, ColumnBinding(10, 0))); - op.expressions.erase(op.expressions.begin() + i); - mode = "iterativelength"; - } else if (expr->expression_class == ExpressionClass::BOUND_FUNCTION && - expr->Cast().function.name == "shortestpath") { - op.expressions.emplace_back(make_uniq( - expr->alias, LogicalType::LIST(LogicalType::BIGINT), ColumnBinding(10, 0))); - op.expressions.erase(op.expressions.begin() + i); - mode = "shortestpath"; - } - } - - auto path_finding_operator = - make_uniq(path_finding_children, path_finding_expressions, mode); - op.children.clear(); - op.children.push_back(std::move(path_finding_operator)); + static bool InsertPathFindingOperator(LogicalOperator &op, ClientContext &context) { + if (op.type != LogicalOperatorType::LOGICAL_PROJECTION) { // We only care about projections + for (auto &child : op.children) { // But a child might have it + if (InsertPathFindingOperator(*child, context)) { + return true; + } + } + return false; // No child had it + } + unique_ptr function_expression; + string mode; + for (int64_t i = op.expressions.size() - 1; i >= 0; --i) { + const auto &expr = op.expressions[i]; + if (expr->expression_class != ExpressionClass::BOUND_FUNCTION) { + continue; + } + auto &bound_function_expression = expr->Cast(); + if (bound_function_expression.function.name == "iterativelength") { + op.expressions.emplace_back(make_uniq( + expr->alias, LogicalType::BIGINT, ColumnBinding(10, 0))); + op.expressions.erase(op.expressions.begin() + i); + mode = "iterativelength"; + function_expression = expr->Copy(); + } else if (bound_function_expression.function.name == "shortestpath") { + op.expressions.emplace_back(make_uniq( + expr->alias, LogicalType::LIST(LogicalType::BIGINT), ColumnBinding(10, 0))); + op.expressions.erase(op.expressions.begin() + i); + mode = "shortestpath"; + function_expression = expr->Copy(); + } + } + if (!function_expression) { // Didn't find the function expression + for (auto &child : op.children) { // But a child might have it + if (InsertPathFindingOperator(*child, context)) { return true; } } + return false; // No child had it + } + + for (const auto &child : op.children) { + vector> path_finding_children; + if (child->type != LogicalOperatorType::LOGICAL_COMPARISON_JOIN) { + continue; + } + auto &get_join = child->Cast(); + //! For now we assume this is enough to detect we have found a + //! path-finding query. Should be improved in the future + if (get_join.children.size() != 2) { + continue; + } + if (get_join.children[1]->type != + LogicalOperatorType::LOGICAL_AGGREGATE_AND_GROUP_BY) { + continue; + } + auto &get_aggregate = get_join.children[1]->Cast(); + auto &get_limit = get_aggregate.children[0]->Cast(); + auto &get_projection = get_limit.children[0]->Cast(); + auto &get_function_expression = + get_projection.expressions[0]->Cast(); + if (get_function_expression.function.name != "create_csr_edge") { + continue; + } + vector> path_finding_expressions = + std::move(get_function_expression.children); + if (get_join.children[0]->type == LogicalOperatorType::LOGICAL_FILTER) { + auto &get_filter = get_join.children[0]->Cast(); + if (get_filter.children[0]->type != LogicalOperatorType::LOGICAL_GET) { + continue; + } + path_finding_children.push_back(std::move(get_filter.children[0])); + } else if (get_join.children[0]->type == + LogicalOperatorType::LOGICAL_EMPTY_RESULT) { + auto default_database = DatabaseManager::GetDefaultDatabase(context); + auto &catalog = Catalog::GetCatalog(context, default_database); + auto &bound_function_expression = function_expression->Cast(); + auto &bind_info = bound_function_expression.bind_info->Cast(); + auto &duckdb_table = catalog.GetEntry(context, DEFAULT_SCHEMA, bind_info.table_to_scan); + auto &get_empty_result = + get_join.children[0]->Cast(); + vector returned_names = {"src", "dst"}; + unique_ptr bind_data; + auto scan_function = duckdb_table.GetScanFunction(context, bind_data); + auto logical_get = make_uniq( + get_empty_result.bindings[0].table_index, scan_function, + std::move(bind_data), get_empty_result.return_types, + returned_names); + vector column_ids_vector; + for (const auto &binding : get_empty_result.bindings) { + column_ids_vector.push_back(binding.column_index); + } + logical_get->column_ids = std::move(column_ids_vector); + path_finding_children.push_back(std::move(logical_get)); + } + path_finding_children.push_back(std::move(get_projection.children[0])); + // Iterate in reverse to not influence the upcoming iterations when + // erasing an element from the list. Does not work if both iterativelength + // and shortestpath are called in the same query for now. To be improved + // in the future. + + + auto path_finding_operator = make_uniq( + path_finding_children, path_finding_expressions, mode); + op.children.clear(); + op.children.push_back(std::move(path_finding_operator)); + return true; // We have found the path-finding operator, no need to continue } for (auto &child : op.children) { - if (InsertPathFindingOperator(*child)) { + if (InsertPathFindingOperator(*child, context)) { return true; } } @@ -105,7 +161,7 @@ class DuckpgqOptimizerExtension : public OptimizerExtension { if (!path_finding_operator_option->second.GetValue()) { return; } - InsertPathFindingOperator(*plan); + InsertPathFindingOperator(*plan, context); } }; From 52468d2a9d1ebe77b6da6f6de261abe7db5688a8 Mon Sep 17 00:00:00 2001 From: dtenwolde Date: Tue, 7 May 2024 15:57:10 +0200 Subject: [PATCH 117/249] Fix function being set to incorrect column --- src/include/duckpgq_extension.hpp | 21 ++------------------- 1 file changed, 2 insertions(+), 19 deletions(-) diff --git a/src/include/duckpgq_extension.hpp b/src/include/duckpgq_extension.hpp index eeea9f92..c583b548 100644 --- a/src/include/duckpgq_extension.hpp +++ b/src/include/duckpgq_extension.hpp @@ -37,14 +37,6 @@ class DuckpgqOptimizerExtension : public OptimizerExtension { } static bool InsertPathFindingOperator(LogicalOperator &op, ClientContext &context) { - if (op.type != LogicalOperatorType::LOGICAL_PROJECTION) { // We only care about projections - for (auto &child : op.children) { // But a child might have it - if (InsertPathFindingOperator(*child, context)) { - return true; - } - } - return false; // No child had it - } unique_ptr function_expression; string mode; for (int64_t i = op.expressions.size() - 1; i >= 0; --i) { @@ -56,26 +48,17 @@ class DuckpgqOptimizerExtension : public OptimizerExtension { if (bound_function_expression.function.name == "iterativelength") { op.expressions.emplace_back(make_uniq( expr->alias, LogicalType::BIGINT, ColumnBinding(10, 0))); + function_expression = expr->Copy(); op.expressions.erase(op.expressions.begin() + i); mode = "iterativelength"; - function_expression = expr->Copy(); } else if (bound_function_expression.function.name == "shortestpath") { op.expressions.emplace_back(make_uniq( expr->alias, LogicalType::LIST(LogicalType::BIGINT), ColumnBinding(10, 0))); + function_expression = expr->Copy(); op.expressions.erase(op.expressions.begin() + i); mode = "shortestpath"; - function_expression = expr->Copy(); } } - if (!function_expression) { // Didn't find the function expression - for (auto &child : op.children) { // But a child might have it - if (InsertPathFindingOperator(*child, context)) { - return true; - } - } - return false; // No child had it - } - for (const auto &child : op.children) { vector> path_finding_children; if (child->type != LogicalOperatorType::LOGICAL_COMPARISON_JOIN) { From 923ba693a35ae516ad5f5d73dad2f2fb823a91ee Mon Sep 17 00:00:00 2001 From: Pingan Ren Date: Fri, 17 May 2024 17:12:28 +0200 Subject: [PATCH 118/249] dynamic tasks splitting --- src/duckpgq_extension.cpp | 8 + src/functions/scalar/csr_creation.cpp | 7 + .../physical_path_finding_operator.hpp | 42 + .../physical_path_finding_operator.cpp | 925 +++++++++++++----- .../path_finding/parallel_path_finding.test | 32 +- 5 files changed, 748 insertions(+), 266 deletions(-) diff --git a/src/duckpgq_extension.cpp b/src/duckpgq_extension.cpp index 9dd143ee..d924cc9b 100644 --- a/src/duckpgq_extension.cpp +++ b/src/duckpgq_extension.cpp @@ -51,6 +51,14 @@ static void LoadInternal(DatabaseInstance &instance) { config.AddExtensionOption("experimental_path_finding_operator", "Enables the experimental path finding operator to be triggered", LogicalType::BOOLEAN, Value(false)); + config.AddExtensionOption("experimental_path_finding_operator_barrier", + "Barrier variants selection", LogicalType::INTEGER, Value(0)); + config.AddExtensionOption("experimental_path_finding_operator_task_size", + "Number of vertices processed per thread at a time", LogicalType::INTEGER, Value(256)); + config.AddExtensionOption("experimental_path_finding_operator_alpha", + "Top-up and bottom-up search toggle parameter", LogicalType::INTEGER, Value(1024)); + config.AddExtensionOption("experimental_path_finding_operator_beta", + "Top-up and bottom-up search toggle parameter", LogicalType::INTEGER, Value(64)); Connection con(instance); con.BeginTransaction(); diff --git a/src/functions/scalar/csr_creation.cpp b/src/functions/scalar/csr_creation.cpp index f3199bcc..18d78b76 100644 --- a/src/functions/scalar/csr_creation.cpp +++ b/src/functions/scalar/csr_creation.cpp @@ -223,6 +223,13 @@ CreateScalarFunctionInfo DuckPGQFunctions::GetCsrEdgeFunction() { LogicalType::INTEGER, CreateCsrEdgeFunction, CSRFunctionData::CSREdgeBind)); + set.AddFunction(ScalarFunction({LogicalType::BIGINT, LogicalType::BIGINT, + LogicalType::BIGINT, LogicalType::BIGINT, + LogicalType::BIGINT, LogicalType::BIGINT, + LogicalType::BIGINT}, + LogicalType::INTEGER, CreateCsrEdgeFunction, + CSRFunctionData::CSREdgeBind)); + //! No edge weight set.AddFunction(ScalarFunction({LogicalType::INTEGER, LogicalType::BIGINT, LogicalType::BIGINT, LogicalType::BIGINT, diff --git a/src/include/duckpgq/operators/physical_path_finding_operator.hpp b/src/include/duckpgq/operators/physical_path_finding_operator.hpp index 47df55ca..1ddc6007 100644 --- a/src/include/duckpgq/operators/physical_path_finding_operator.hpp +++ b/src/include/duckpgq/operators/physical_path_finding_operator.hpp @@ -23,11 +23,21 @@ class PhysicalPathFinding : public PhysicalComparisonJoin { public: GlobalCompressedSparseRow(ClientContext &context){ }; + ~GlobalCompressedSparseRow() { + if (v) { + delete[] v; + } + if (reverse_v) { + delete[] reverse_v; + } + } + atomic *v; atomic *reverse_v; vector e; vector reverse_e; vector edge_ids; + vector reverse_edge_ids; vector w; vector w_double; bool initialized_v = false; @@ -58,6 +68,38 @@ class PhysicalPathFinding : public PhysicalComparisonJoin { } else { result += "not initialized"; } + result += "\nEdge IDs: "; + if (initialized_e) { + for (auto i : edge_ids) { + result += std::to_string(i) + " "; + } + } else { + result += "not initialized"; + } + result += "\nReverse V: "; + if (initialized_v) { + for (idx_t i = 0; i < v_size; ++i) { + result += std::to_string(reverse_v[i]) + ' '; + } + } else { + result += "not initialized"; + } + result += "\nReverse E: "; + if (initialized_e) { + for (auto i : reverse_e) { + result += std::to_string(i) + " "; + } + } else { + result += "not initialized"; + } + result += "\nReverse Edge IDs: "; + if (initialized_e) { + for (auto i : reverse_edge_ids) { + result += std::to_string(i) + " "; + } + } else { + result += "not initialized"; + } result += "\nW: "; if (initialized_w) { for (auto i : w) { diff --git a/src/operators/physical_path_finding_operator.cpp b/src/operators/physical_path_finding_operator.cpp index fc0e9cdc..16af965c 100644 --- a/src/operators/physical_path_finding_operator.cpp +++ b/src/operators/physical_path_finding_operator.cpp @@ -13,8 +13,14 @@ #include #include #include +#include #include +#define SEGMENT_BITSET +// #define ATOMIC_BITSET +// #define FINE_GRAINED_LOCK +// #define ELEMENT_LOCK + namespace duckdb { PhysicalPathFinding::PhysicalPathFinding(LogicalExtensionOperator &op, @@ -60,12 +66,14 @@ void PhysicalPathFinding::GlobalCompressedSparseRow::InitializeEdge( e.resize(e_size, 0); reverse_e.resize(e_size, 0); edge_ids.resize(e_size, 0); + reverse_edge_ids.resize(e_size, 0); } catch (std::bad_alloc const &) { throw InternalException("Unable to initialize vector of size for csr " "edge table representation"); } for (idx_t i = 1; i < v_size; i++) { v[i] += v[i - 1]; + reverse_v[i] += reverse_v[i - 1]; } initialized_e = true; } @@ -106,82 +114,98 @@ class PathFindingLocalState : public LocalSinkState { void PathFindingLocalState::CreateCSRVertex(DataChunk &input, GlobalCompressedSparseRow &global_csr) { if (!global_csr.initialized_v) { - const auto v_size = input.data[8].GetValue(0).GetValue(); + const auto v_size = input.data[9].GetValue(0).GetValue(); global_csr.InitializeVertex(v_size); } auto result = Vector(LogicalTypeId::BIGINT); BinaryExecutor::Execute( - input.data[6], input.data[5], result, input.size(), + input.data[4], input.data[3], result, input.size(), [&](const int64_t src, const int64_t cnt) { int64_t edge_count = 0; global_csr.v[src + 2] = cnt; edge_count = edge_count + cnt; return edge_count; }); + BinaryExecutor::Execute( + input.data[7], input.data[6], result, input.size(), + [&](const int64_t dst, const int64_t cnt) { + int64_t edge_count = 0; + global_csr.reverse_v[dst + 2] = cnt; + edge_count = edge_count + cnt; + return edge_count; + }); } -// class Barrier { -// public: -// explicit Barrier(std::size_t iCount) : -// mThreshold(iCount), -// mCount(iCount), -// mGeneration(0) { -// } - -// void Wait() { -// std::unique_lock lLock{mMutex}; -// auto lGen = mGeneration; -// if (!--mCount) { -// mGeneration++; -// mCount = mThreshold; -// mCond.notify_all(); -// } else { -// mCond.wait(lLock, [this, lGen] { return lGen != mGeneration; }); -// } -// } - -// private: -// std::mutex mMutex; -// std::condition_variable mCond; -// std::size_t mThreshold; -// std::size_t mCount; -// std::size_t mGeneration; -// }; - class Barrier { public: - explicit Barrier(std::size_t iCount) : + explicit Barrier(std::size_t iType, std::size_t iCount) : + mType(iType), mThreshold(iCount), mCount(iCount), mGeneration(0) { + if (iType == 2) { + pthread_barrier_init(&mBarrier, nullptr, iCount); + } } void Wait() { - auto lGen = mGeneration.load(); - if (!--mCount) { - mCount = mThreshold; - ++mGeneration; - } else { - while (lGen == mGeneration.load()) { - std::this_thread::yield(); + switch (mType) + { + case 0: + { + std::unique_lock lLock{mMutex}; + auto lGen = mGeneration.load(); + if (!--mCount) { + mGeneration++; + mCount = mThreshold; + mCond.notify_all(); + } else { + mCond.wait(lLock, [this, lGen] { return lGen != mGeneration; }); + } } + break; + case 1: + { + auto lGen = mGeneration.load(); + if (!--mCount) { + mCount = mThreshold; + ++mGeneration; + } else { + while (lGen == mGeneration.load()) { + std::this_thread::yield(); + } + } + } + break; + case 2: + pthread_barrier_wait(&mBarrier); + break; + default: + break; } } private: + std::size_t mType; + pthread_barrier_t mBarrier; std::mutex mMutex; + std::condition_variable mCond; std::size_t mThreshold; std::atomic mCount; std::atomic mGeneration; }; class GlobalBFSState { + using GlobalCompressedSparseRow = + PhysicalPathFinding::GlobalCompressedSparseRow; public: - GlobalBFSState(shared_ptr pairs_, int64_t v_size_, idx_t num_threads_, ClientContext &context_) - : pairs(pairs_), iter(1), v_size(v_size_), change(false), started_searches(0), - seen(v_size_), visit1(v_size_), visit2(v_size_), context(context_), - total_len(0), parents_v(v_size_, std::vector(LANE_LIMIT, -1)), parents_e(v_size_, std::vector(LANE_LIMIT, -1)), - frontier_size(0), unseen_size(v_size_), num_threads(num_threads_), task_queues(num_threads_), barrier(num_threads_) { + GlobalBFSState(shared_ptr csr_, shared_ptr pairs_, int64_t v_size_, + idx_t num_threads_, idx_t barrier_type_, ClientContext &context_) + : csr(csr_), pairs(pairs_), iter(1), v_size(v_size_), change(false), started_searches(0), total_len(0), context(context_), + seen(v_size_), visit1(v_size_), visit2(v_size_), + parents_v(v_size_, std::vector(LANE_LIMIT, -1)), parents_e(v_size_, std::vector(LANE_LIMIT, -1)), + frontier_size(0), unseen_size(v_size_ * 8), is_top_down(true), num_threads(num_threads_), task_queues(num_threads_), + task_queues_reverse(num_threads_), barrier(barrier_type_, num_threads_) { result.Initialize(context, {LogicalType::BIGINT, LogicalType::LIST(LogicalType::BIGINT)}, pairs_->size()); for (auto i = 0; i < LANE_LIMIT; i++) { lane_to_num[i] = -1; @@ -193,10 +217,15 @@ class GlobalBFSState { src = FlatVector::GetData(src_data); dst = FlatVector::GetData(dst_data); for (auto i = 0; i < v_size; i++) { +#ifdef SEGMENT_BITSET for (auto j = 0; j < 8; j++) { seen[i][j] = 0; visit1[i][j] = 0; } +#else + seen[i] = 0; + visit1[i] = 0; +#endif for (auto j = 0; j < LANE_LIMIT; j++) { parents_v[i][j] = -1; parents_e[i][j] = -1; @@ -210,16 +239,21 @@ class GlobalBFSState { iter = 1; change = false; frontier_size = 0; - unseen_size = v_size; + unseen_size = v_size * 8; for (auto i = 0; i < LANE_LIMIT; i++) { lane_to_num[i] = -1; } // empty visit vectors for (auto i = 0; i < v_size; i++) { +#ifdef SEGMENT_BITSET for (auto j = 0; j < 8; j++) { seen[i][j] = 0; visit1[i][j] = 0; } +#else + seen[i] = 0; + visit1[i] = 0; +#endif for (auto j = 0; j < LANE_LIMIT; j++) { parents_v[i][j] = -1; parents_e[i][j] = -1; @@ -230,28 +264,114 @@ class GlobalBFSState { void CreateTasks() { // workerTasks[workerId] = [task1, task2, ...] - vector>> worker_tasks(num_threads); - auto cur_worker = 0; - - for (auto offset = 0; offset < v_size; offset += split_size) { - auto worker_id = cur_worker % num_threads; - pair range = {offset, std::min(offset + split_size, v_size)}; - worker_tasks[worker_id].push_back(range); - cur_worker++; + auto queues = {&task_queues, &task_queues_reverse}; + is_top_down = true; + for (auto& queue : queues) { + vector>> worker_tasks(num_threads); + auto cur_worker = 0; + int64_t *v = is_top_down ? (int64_t*)csr->v : (int64_t*)csr->reverse_v; + int64_t current_task_edges = 0; + idx_t current_task_start = 0; + for (idx_t i = 0; i < (idx_t)v_size; i++) { + auto vertex_edges = v[i + 1] - v[i]; + if (current_task_edges + vertex_edges > split_size && i != current_task_start) { + auto worker_id = cur_worker % num_threads; + pair range = {current_task_start, i}; + worker_tasks[worker_id].push_back(range); + current_task_start = i; + current_task_edges = 0; + cur_worker++; + } + current_task_edges += vertex_edges; + } + if (current_task_start < (idx_t)v_size) { + auto worker_id = cur_worker % num_threads; + pair range = {current_task_start, v_size}; + worker_tasks[worker_id].push_back(range); + } + for (idx_t worker_id = 0; worker_id < num_threads; worker_id++) { + queue->at(worker_id).first.store(0); + queue->at(worker_id).second = worker_tasks[worker_id]; + } + is_top_down = false; } + is_top_down = true; + } - for (idx_t worker_id = 0; worker_id < num_threads; worker_id++) { + void InitTask(idx_t worker_id) { + if (is_top_down) { task_queues[worker_id].first.store(0); - task_queues[worker_id].second = worker_tasks[worker_id]; + } else { + task_queues_reverse[worker_id].first.store(0); } + } + + pair FetchTask(idx_t worker_id) { + auto& task_queue = is_top_down ? task_queues : task_queues_reverse; + idx_t offset = 0; + do { + auto worker_idx = (worker_id + offset) % task_queue.size(); + auto cur_task_ix = task_queue[worker_idx].first.fetch_add(1); + if (cur_task_ix < task_queue[worker_idx].second.size()) { + return task_queue[worker_idx].second[cur_task_ix]; + } else { + offset++; + } + } while (offset < task_queue.size()); + return {0, 0}; + } + pair BoundaryCalculation(idx_t worker_id) { + idx_t block_size = ceil((double)v_size / num_threads); + block_size = block_size == 0 ? 1 : block_size; + idx_t left = block_size * worker_id; + idx_t right = std::min(block_size * (worker_id + 1), (idx_t)v_size); + return {left, right}; + } + + void DirectionSwitch() { + // Debug print + string message = "frontier size: "; + message += std::to_string(frontier_size); + message += " unseen size: "; + message += std::to_string(unseen_size); + message += " iter "; + message += std::to_string(iter); + Printer::Print(message); + // Determine the switch of algorithms + if (is_top_down) { + auto Ctb = unseen_size / alpha; + if (frontier_size > Ctb) { + is_top_down = false; + } + } else { + auto Cbt = v_size / beta; + if (frontier_size < Cbt) { + is_top_down = true; + } + } + // Debug print + message = "Switch to "; + message += is_top_down ? "top-down" : "bottom-up"; + message += " at iter "; + message += std::to_string(iter); + Printer::Print(message); + if (frontier_size > 0) { + change = true; + } else { + change = false; + } + // clear the counters after the switch + frontier_size = 0; + unseen_size = v_size * 8; } public: + shared_ptr csr; shared_ptr pairs; int64_t iter; int64_t v_size; - bool change; + atomic change; idx_t started_searches; int64_t total_len; int64_t *src; @@ -261,24 +381,35 @@ class GlobalBFSState { int64_t lane_to_num[LANE_LIMIT]; DataChunk result; // 0 for length, 1 for path ClientContext& context; +#ifdef SEGMENT_BITSET vector, 8>> seen; vector, 8>> visit1; vector, 8>> visit2; +#elif defined(ATOMIC_BITSET) + vector>> seen; + vector>> visit1; + vector>> visit2; +#else + vector> seen; + vector> visit1; + vector> visit2; +#endif vector> parents_v; vector> parents_e; atomic frontier_size; atomic unseen_size; - constexpr static int64_t alpha = 1024; - constexpr static int64_t beta = 64; - bool is_top_down = true; + int64_t alpha = 1024; + int64_t beta = 64; + atomic is_top_down; idx_t num_threads; // task_queues[workerId] = {curTaskIx, queuedTasks} // queuedTasks[curTaskIx] = {start, end} vector, vector>>> task_queues; - constexpr static int64_t split_size = 256; + vector, vector>>> task_queues_reverse; + int64_t split_size = 256; Barrier barrier; @@ -342,11 +473,11 @@ class PathFindingGlobalState : public GlobalSinkState { ColumnDataScanState scan_state; ColumnDataAppendState append_state; - unique_ptr global_csr; + shared_ptr global_csr; // state for BFS unique_ptr global_bfs_state; - string mode; size_t child; + string mode; }; unique_ptr @@ -391,43 +522,40 @@ PhysicalPathFinding::Combine(ExecutionContext &context, // Finalize //===--------------------------------------------------------------------===// -class PhysicalIterativeTopDownTask : public ExecutorTask { +class PhysicalIterativeTask : public ExecutorTask { public: - PhysicalIterativeTopDownTask(shared_ptr event_p, ClientContext &context, PathFindingGlobalState &state, idx_t worker_id) + PhysicalIterativeTask(shared_ptr event_p, ClientContext &context, PathFindingGlobalState &state, idx_t worker_id) : ExecutorTask(context, std::move(event_p)), context(context), state(state), worker_id(worker_id) { } TaskExecutionResult ExecuteTask(TaskExecutionMode mode) override { auto& bfs_state = state.global_bfs_state; auto& change = bfs_state->change; - auto& seen = bfs_state->seen; auto& barrier = bfs_state->barrier; - auto result_data = FlatVector::GetData(bfs_state->result.data[0]); - ValidityMask &result_validity = FlatVector::Validity(bfs_state->result.data[0]); - auto& iter = bfs_state->iter; - int64_t *v = (int64_t *)state.global_csr->v; - vector &e = state.global_csr->e; - - BoundaryCalculation(); + auto bound = bfs_state->BoundaryCalculation(worker_id); + left = bound.first; + right = bound.second; do { - InitTask(); - - auto& visit = iter & 1 ? bfs_state->visit1 : bfs_state->visit2; - auto& next = iter & 1 ? bfs_state->visit2 : bfs_state->visit1; + bfs_state->InitTask(worker_id); - IterativeLength(v, change, barrier, e, seen, visit, next); + if (bfs_state->is_top_down) { + IterativeLengthTopDown(); + } else { + IterativeLengthBottomUp(); + } barrier.Wait(); if (worker_id == 0) { - ReachDetect(result_data); + ReachDetect(); + bfs_state->DirectionSwitch(); } barrier.Wait(); } while (change); if (worker_id == 0) { - UnReachableSet(result_data, result_validity); + UnReachableSet(); } event->FinishTask(); @@ -435,60 +563,46 @@ class PhysicalIterativeTopDownTask : public ExecutorTask { } private: - void InitTask() { - auto& task_queue = state.global_bfs_state->task_queues; - task_queue[worker_id].first.store(0); - } - - pair FetchTask() { - auto& task_queue = state.global_bfs_state->task_queues; - idx_t offset = 0; - do { - auto worker_idx = (worker_id + offset) % task_queue.size(); - auto cur_task_ix = task_queue[worker_idx].first.fetch_add(1); - if (cur_task_ix < task_queue[worker_idx].second.size()) { - return task_queue[worker_idx].second[cur_task_ix]; - } else { - offset++; - } - } while (offset < task_queue.size()); - return {0, 0}; - } - - void BoundaryCalculation() { + void IterativeLengthTopDown() { auto& bfs_state = state.global_bfs_state; - auto& v_size = bfs_state->v_size; - idx_t block_size = ceil((double)v_size / bfs_state->num_threads); - block_size = block_size == 0 ? 1 : block_size; - left = block_size * worker_id; - right = std::min(block_size * (worker_id + 1), (idx_t)v_size); - } + auto& seen = bfs_state->seen; + auto& visit = bfs_state->iter & 1 ? bfs_state->visit1 : bfs_state->visit2; + auto& next = bfs_state->iter & 1 ? bfs_state->visit2 : bfs_state->visit1; + auto& barrier = bfs_state->barrier; + int64_t *v = (int64_t *)state.global_csr->v; + vector &e = state.global_csr->e; + auto& frontier_size = bfs_state->frontier_size; + auto& unseen_size = bfs_state->unseen_size; - void IterativeLength(int64_t *v, bool& change, Barrier& barrier, - vector &e, - vector, 8>> &seen, - vector, 8>> &visit, - vector, 8>> &next) { // clear next before each iteration for (auto i = left; i < right; i++) { +#ifdef SEGMENT_BITSET for (auto j = 0; j < 8; j++) { next[i][j].store(0, std::memory_order_relaxed); } +#else + next[i] = 0; +#endif } barrier.Wait(); while (true) { - auto task = FetchTask(); + auto task = bfs_state->FetchTask(worker_id); if (task.first == task.second) { break; } auto start = task.first; auto end = task.second; +#ifdef SEGMENT_BITSET idx_t old_next, new_next; +#else + std::bitset old_next, new_next; +#endif for (auto i = start; i < end; i++) { +#ifdef SEGMENT_BITSET for (auto j = 0; j < 8; j++) { if (visit[i][j].load(std::memory_order_relaxed)) { for (auto offset = v[i]; offset < v[i + 1]; offset++) { @@ -500,53 +614,202 @@ class PhysicalIterativeTopDownTask : public ExecutorTask { } } } +#elif defined(ATOMIC_BITSET) + if (visit[i].load(std::memory_order_relaxed).any()) { + for (auto offset = v[i]; offset < v[i + 1]; offset++) { + auto n = e[offset]; + do { + old_next = next[n].load(); + new_next = old_next | visit[i].load(); + } while (!next[n].compare_exchange_weak(old_next, new_next)); + } + } +#else + if (visit[i].any()) { + for (auto offset = v[i]; offset < v[i + 1]; offset++) { + auto n = e[offset]; +#ifdef FINE_GRAINED_LOCK + std::lock_guard lock(state.global_csr->lock); +#elif defined(ELEMENT_LOCK) + std::lock_guard lock(state.global_csr->element_lock[n]); +#endif + next[n] |= visit[i]; + } + } +#endif } } - change = false; barrier.Wait(); for (auto i = left; i < right; i++) { +#ifdef SEGMENT_BITSET for (auto j = 0; j < 8; j++) { if (next[i][j].load(std::memory_order_relaxed)) { next[i][j].store(next[i][j].load(std::memory_order_relaxed) & ~seen[i][j].load(std::memory_order_relaxed), std::memory_order_relaxed); seen[i][j].store(seen[i][j].load(std::memory_order_relaxed) | next[i][j].load(std::memory_order_relaxed), std::memory_order_relaxed); - change |= next[i][j].load(std::memory_order_relaxed); + + frontier_size = next[i][j].load() ? frontier_size + 1 : frontier_size.load(); + unseen_size = (~seen[i][j].load()) == 0 ? unseen_size - 1 : unseen_size.load(); } } +#elif defined(ATOMIC_BITSET) + if (next[i].load().any()) { + next[i].store(next[i].load() & ~seen[i].load(), std::memory_order_relaxed); + seen[i].store(seen[i].load() | next[i].load(), std::memory_order_relaxed); + + frontier_size = next[i].load().any() ? frontier_size + 1 : frontier_size.load(); + unseen_size = (~seen[i].load()).all() ? unseen_size - 1 : unseen_size.load(); + } +#else + if (next[i].any()) { + next[i] &= ~seen[i]; + seen[i] |= next[i]; + + frontier_size = next[i].any() ? frontier_size + 1 : frontier_size.load(); + unseen_size = seen[i].all() ? unseen_size - 1 : unseen_size.load(); + } +#endif } } -void ReachDetect(int64_t *result_data) { - auto &bfs_state = state.global_bfs_state; - // detect lanes that finished - for (int64_t lane = 0; lane < LANE_LIMIT; lane++) { - int64_t search_num = bfs_state->lane_to_num[lane]; - if (search_num >= 0) { // active lane - int64_t dst_pos = bfs_state->vdata_dst.sel->get_index(search_num); - if (bfs_state->seen[bfs_state->dst[dst_pos]][lane / 64] & ((idx_t)1 << (lane % 64))) { - result_data[search_num] = - bfs_state->iter; /* found at iter => iter = path length */ - bfs_state->lane_to_num[lane] = -1; // mark inactive + void IterativeLengthBottomUp() { + auto& bfs_state = state.global_bfs_state; + auto& seen = bfs_state->seen; + auto& visit = bfs_state->iter & 1 ? bfs_state->visit1 : bfs_state->visit2; + auto& next = bfs_state->iter & 1 ? bfs_state->visit2 : bfs_state->visit1; + auto& barrier = bfs_state->barrier; + int64_t *v = (int64_t *)state.global_csr->reverse_v; + vector &e = state.global_csr->reverse_e; + auto& frontier_size = bfs_state->frontier_size; + auto& unseen_size = bfs_state->unseen_size; + + // clear next before each iteration + for (auto i = left; i < right; i++) { +#ifdef SEGMENT_BITSET + for (auto j = 0; j < 8; j++) { + next[i][j].store(0, std::memory_order_relaxed); + } +#else + next[i] = 0; +#endif + } + + barrier.Wait(); + + while (true) { + auto task = bfs_state->FetchTask(worker_id); + if (task.first == task.second) { + break; + } + auto start = task.first; + auto end = task.second; + +#ifdef SEGMENT_BITSET + idx_t old_next, new_next; +#else + std::bitset old_next, new_next; +#endif + + for (auto i = start; i < end; i++) { +#ifdef SEGMENT_BITSET + for (auto j = 0; j < 8; j++) { + if (~seen[i][j] == 0) { + continue; + } + + for (auto offset = v[i]; offset < v[i + 1]; offset++) { + auto n = e[offset]; + do { + old_next = next[i][j].load(); + new_next = old_next | visit[n][j].load(); + } while (!next[i][j].compare_exchange_weak(old_next, new_next)); + } + + next[i][j].store(next[i][j].load() & ~seen[i][j].load()); + seen[i][j].store(seen[i][j].load() | next[i][j].load()); + + frontier_size = next[i][j].load() ? frontier_size + 1 : frontier_size.load(); + unseen_size = (~seen[i][j].load()) == 0 ? unseen_size - 1 : unseen_size.load(); + } +#elif defined(ATOMIC_BITSET) + if (seen[i].load().all()) { + continue; + } + + for (auto offset = v[i]; offset < v[i + 1]; offset++) { + auto n = e[offset]; + do { + old_next = next[i].load(); + new_next = old_next | visit[n].load(); + } while (!next[i].compare_exchange_weak(old_next, new_next)); + } + + next[i].store(next[i].load() & ~seen[i].load()); + seen[i].store(seen[i].load() | next[i].load()); + + frontier_size = next[i].load().any() ? frontier_size + 1 : frontier_size.load(); + unseen_size = (~seen[i].load()).all() ? unseen_size - 1 : unseen_size.load(); +#else + if (seen[i].all()) { + continue; + } + + for (auto offset = v[i]; offset < v[i + 1]; offset++) { + auto n = e[offset]; + next[i] |= visit[n]; + } + + next[i] &= ~seen[i]; + seen[i] |= next[i]; + + frontier_size = next[i].any() ? frontier_size + 1 : frontier_size.load(); + unseen_size = seen[i].all() ? unseen_size - 1 : unseen_size.load(); +#endif } } } - // into the next iteration - bfs_state->iter++; -} -void UnReachableSet(int64_t *result_data, ValidityMask &result_validity) { - auto &bfs_state = state.global_bfs_state; - for (int64_t lane = 0; lane < LANE_LIMIT; lane++) { - int64_t search_num = bfs_state->lane_to_num[lane]; - if (search_num >= 0) { // active lane - result_validity.SetInvalid(search_num); - result_data[search_num] = (int64_t)-1; /* no path */ - bfs_state->lane_to_num[lane] = -1; // mark inactive + void ReachDetect() { + auto &bfs_state = state.global_bfs_state; + auto result_data = FlatVector::GetData(bfs_state->result.data[0]); + + // detect lanes that finished + for (int64_t lane = 0; lane < LANE_LIMIT; lane++) { + int64_t search_num = bfs_state->lane_to_num[lane]; + if (search_num >= 0) { // active lane + int64_t dst_pos = bfs_state->vdata_dst.sel->get_index(search_num); +#ifdef SEGMENT_BITSET + if (bfs_state->seen[bfs_state->dst[dst_pos]][lane / 64] & ((idx_t)1 << (lane % 64))) { +#elif defined(ATOMIC_BITSET) + if (bfs_state->seen[bfs_state->dst[dst_pos]].load()[lane]) { +#else + if (bfs_state->seen[bfs_state->dst[dst_pos]][lane]) { +#endif + result_data[search_num] = + bfs_state->iter; /* found at iter => iter = path length */ + bfs_state->lane_to_num[lane] = -1; // mark inactive + } + } } + // into the next iteration + bfs_state->iter++; } -} + void UnReachableSet() { + auto &bfs_state = state.global_bfs_state; + auto result_data = FlatVector::GetData(bfs_state->result.data[0]); + auto &result_validity = FlatVector::Validity(bfs_state->result.data[0]); + + for (int64_t lane = 0; lane < LANE_LIMIT; lane++) { + int64_t search_num = bfs_state->lane_to_num[lane]; + if (search_num >= 0) { // active lane + result_validity.SetInvalid(search_num); + result_data[search_num] = (int64_t)-1; /* no path */ + bfs_state->lane_to_num[lane] = -1; // mark inactive + } + } + } private: ClientContext &context; PathFindingGlobalState &state; @@ -556,60 +819,6 @@ void UnReachableSet(int64_t *result_data, ValidityMask &result_validity) { idx_t worker_id; }; -// class PhysicalBFSBottomUpTask : public ExecutorTask { -// public: -// PhysicalBFSBottomUpTask(shared_ptr event_p, ClientContext &context, PathFindingGlobalState &state, idx_t start, idx_t end) -// : ExecutorTask(context, std::move(event_p)), context(context), state(state), start(start), end(end) { -// } - -// TaskExecutionResult ExecuteTask(TaskExecutionMode mode) override { -// auto& bfs_state = state.global_bfs_state; -// auto& change = bfs_state->change; -// auto& seen = bfs_state->seen; -// auto& visit = bfs_state->iter & 1 ? bfs_state->visit1 : bfs_state->visit2; -// auto& next = bfs_state->iter & 1 ? bfs_state->visit2 : bfs_state->visit1; -// auto& barrier = bfs_state->barrier; -// auto& frontier_size = bfs_state->frontier_size; -// auto& unseen_size = bfs_state->unseen_size; - -// int64_t *v = (int64_t *)state.global_csr->v; -// vector &e = state.global_csr->e; - -// for (auto i = start; i < end; i++) { -// next[i] = 0; -// } - -// barrier.Wait(); - -// for (auto i = start; i < end; i++) { -// if (seen[i].all()) { -// unseen_size -= 1; -// continue; -// } -// for (auto offset = v[i]; offset < v[i + 1]; offset++) { -// auto n = e[offset]; -// next[i] = next[i] | visit[n]; -// } -// next[i] = next[i] & ~seen[i]; -// seen[i] = seen[i] | next[i]; -// change |= next[i].any(); - -// frontier_size = next[i].any() ? frontier_size + 1 : frontier_size.load(); -// unseen_size = seen[i].all() ? unseen_size - 1 : unseen_size.load(); -// } - -// event->FinishTask(); -// return TaskExecutionResult::TASK_FINISHED; -// } - -// private: -// ClientContext &context; -// PathFindingGlobalState &state; -// // [start, end) -// idx_t start; -// idx_t end; -// }; - class PhysicalCSREdgeCreationTask : public ExecutorTask { public: PhysicalCSREdgeCreationTask(shared_ptr event_p, ClientContext &context, PathFindingGlobalState &state) @@ -632,17 +841,25 @@ class PhysicalCSREdgeCreationTask : public ExecutorTask { } } if (!global_csr->initialized_e) { - const auto e_size = input.data[7].GetValue(0).GetValue(); + const auto e_size = input.data[8].GetValue(0).GetValue(); global_csr->InitializeEdge(e_size); } TernaryExecutor::Execute( - input.data[6], input.data[4], input.data[2], result, input.size(), + input.data[4], input.data[7], input.data[2], result, input.size(), [&](int64_t src, int64_t dst, int64_t edge_id) { const auto pos = ++global_csr->v[src + 1]; global_csr->e[static_cast(pos) - 1] = dst; global_csr->edge_ids[static_cast(pos) - 1] = edge_id; return 1; }); + TernaryExecutor::Execute( + input.data[7], input.data[4], input.data[2], result, input.size(), + [&](int64_t dst, int64_t src, int64_t edge_id) { + const auto pos = ++global_csr->reverse_v[dst + 1]; + global_csr->reverse_e[static_cast(pos) - 1] = src; + global_csr->reverse_edge_ids[static_cast(pos) - 1] = edge_id; + return 1; + }); } event->FinishTask(); return TaskExecutionResult::TASK_FINISHED; @@ -704,7 +921,7 @@ class ParallelIterativeEvent : public BasePipelineEvent { vector> bfs_tasks; for (idx_t tnum = 0; tnum < bfs_state->num_threads; tnum++) { - bfs_tasks.push_back(make_uniq(shared_from_this(), context, gstate, tnum)); + bfs_tasks.push_back(make_uniq(shared_from_this(), context, gstate, tnum)); } SetTasks(std::move(bfs_tasks)); start_time = std::chrono::high_resolution_clock::now(); @@ -773,7 +990,7 @@ class SequentialIterativeEvent : public BasePipelineEvent { return change; } - static void IterativeLengthFunction(const unique_ptr &csr, + static void IterativeLengthFunction(const shared_ptr &csr, DataChunk &pairs, Vector &result) { int64_t v_size = csr->v_size; int64_t *v = (int64_t *)csr->v; @@ -873,50 +1090,41 @@ class SequentialIterativeEvent : public BasePipelineEvent { } }; -class PhysicalShortestPathTopDownTask : public ExecutorTask { +class PhysicalShortestPathTask : public ExecutorTask { public: - PhysicalShortestPathTopDownTask(shared_ptr event_p, ClientContext &context, PathFindingGlobalState &state, idx_t worker_id) + PhysicalShortestPathTask(shared_ptr event_p, ClientContext &context, PathFindingGlobalState &state, idx_t worker_id) : ExecutorTask(context, std::move(event_p)), context(context), state(state), worker_id(worker_id) { } TaskExecutionResult ExecuteTask(TaskExecutionMode mode) override { auto& bfs_state = state.global_bfs_state; auto& change = bfs_state->change; - auto& seen = bfs_state->seen; - auto& v_size = bfs_state->v_size; auto& barrier = bfs_state->barrier; - auto& parents_v = bfs_state->parents_v; - auto& parents_e = bfs_state->parents_e; - auto& result_path = bfs_state->result.data[1]; - auto result_data = FlatVector::GetData(result_path); - ValidityMask &result_validity = FlatVector::Validity(result_path); - auto& iter = bfs_state->iter; - - int64_t *v = (int64_t *)state.global_csr->v; - vector &e = state.global_csr->e; - vector &edge_ids = state.global_csr->edge_ids; - BoundaryCalculation(); + auto bound = bfs_state->BoundaryCalculation(worker_id); + left = bound.first; + right = bound.second; do { - InitTask(); - - auto& visit = iter & 1 ? bfs_state->visit1 : bfs_state->visit2; - auto& next = iter & 1 ? bfs_state->visit2 : bfs_state->visit1; - - IterativeLength(v_size, change, barrier, v, e, edge_ids, parents_v, parents_e, seen, visit, next); + bfs_state->InitTask(worker_id); + if (bfs_state->is_top_down) { + IterativeLengthTopDown(); + } else { + IterativeLengthBottomUp(); + } barrier.Wait(); if (worker_id == 0) { - ReachDetect(result_data, change); + ReachDetect(); + bfs_state->DirectionSwitch(); } barrier.Wait(); } while (change); if (worker_id == 0) { - PathConstruction(result_path, result_data, result_validity); + PathConstruction(); } event->FinishTask(); @@ -924,61 +1132,49 @@ class PhysicalShortestPathTopDownTask : public ExecutorTask { } private: - void InitTask() { - auto& task_queue = state.global_bfs_state->task_queues; - task_queue[worker_id].first.store(0); - } - - pair FetchTask() { - auto& task_queue = state.global_bfs_state->task_queues; - idx_t offset = 0; - do { - auto worker_idx = (worker_id + offset) % task_queue.size(); - auto cur_task_ix = task_queue[worker_idx].first.fetch_add(1); - if (cur_task_ix < task_queue[worker_idx].second.size()) { - return task_queue[worker_idx].second[cur_task_ix]; - } - offset++; - } while (offset < task_queue.size()); - return {0, 0}; - } - - void BoundaryCalculation() { + void IterativeLengthTopDown() { auto& bfs_state = state.global_bfs_state; - auto& v_size = bfs_state->v_size; - idx_t block_size = ceil((double)v_size / bfs_state->num_threads); - block_size = block_size == 0 ? 1 : block_size; - left = block_size * worker_id; - right = std::min(block_size * (worker_id + 1), (idx_t)v_size); - } + auto& seen = bfs_state->seen; + auto& visit = bfs_state->iter & 1 ? bfs_state->visit1 : bfs_state->visit2; + auto& next = bfs_state->iter & 1 ? bfs_state->visit2 : bfs_state->visit1; + auto& barrier = bfs_state->barrier; + int64_t *v = (int64_t *)state.global_csr->v; + vector &e = state.global_csr->e; + auto& edge_ids = state.global_csr->edge_ids; + auto& frontier_size = bfs_state->frontier_size; + auto& unseen_size = bfs_state->unseen_size; + auto& parents_v = bfs_state->parents_v; + auto& parents_e = bfs_state->parents_e; - void IterativeLength(int64_t v_size, bool& change, Barrier& barrier, int64_t *v, - vector &e, vector &edge_ids, - vector> &parents_v, - vector> &parents_e, - vector, 8>> &seen, - vector, 8>> &visit, - vector, 8>> &next) { // clear next before each iteration for (auto i = left; i < right; i++) { +#ifdef SEGMENT_BITSET for (auto j = 0; j < 8; j++) { - next[i][j] = 0; + next[i][j].store(0, std::memory_order_relaxed); } +#else + next[i] = 0; +#endif } barrier.Wait(); while (true) { - auto task = FetchTask(); + auto task = bfs_state->FetchTask(worker_id); if (task.first == task.second) { break; } auto start = task.first; auto end = task.second; +#ifdef SEGMENT_BITSET idx_t old_next, new_next; +#else + std::bitset old_next, new_next; +#endif for (auto i = start; i < end; i++) { +#ifdef SEGMENT_BITSET for (auto j = 0; j < 8; j++) { if (visit[i][j].load(std::memory_order_relaxed)) { for (auto offset = v[i]; offset < v[i + 1]; offset++) { @@ -997,25 +1193,201 @@ class PhysicalShortestPathTopDownTask : public ExecutorTask { } } } +#elif defined(ATOMIC_BITSET) + if (visit[i].load(std::memory_order_relaxed).any()) { + for (auto offset = v[i]; offset < v[i + 1]; offset++) { + auto n = e[offset]; + auto edge_id = edge_ids[offset]; + do { + old_next = next[n].load(); + new_next = old_next | visit[i].load(); + } while (!next[n].compare_exchange_weak(old_next, new_next)); + for (auto l = 0; l < LANE_LIMIT; l++) { + parents_v[n][l] = ((parents_v[n][l] == -1) && visit[i].load()[l]) + ? i : parents_v[n][l]; + parents_e[n][l] = ((parents_e[n][l] == -1) && visit[i].load()[l]) + ? edge_id : parents_e[n][l]; + } + } + } +#else + if (visit[i].any()) { + for (auto offset = v[i]; offset < v[i + 1]; offset++) { + auto n = e[offset]; +#ifdef FINE_GRAINED_LOCK + std::lock_guard lock(state.global_csr->lock); +#elif defined(ELEMENT_LOCK) + std::lock_guard lock(state.global_csr->element_lock[n]); +#endif + next[n] |= visit[i]; + for (auto l = 0; l < LANE_LIMIT; l++) { + parents_v[n][l] = ((parents_v[n][l] == -1) && visit[i][l]) + ? i : parents_v[n][l]; + parents_e[n][l] = ((parents_e[n][l] == -1) && visit[i][l]) + ? edge_id : parents_e[n][l]; + } + } + } +#endif } } - change = false; barrier.Wait(); for (auto i = left; i < right; i++) { +#ifdef SEGMENT_BITSET for (auto j = 0; j < 8; j++) { if (next[i][j].load(std::memory_order_relaxed)) { next[i][j].store(next[i][j].load(std::memory_order_relaxed) & ~seen[i][j].load(std::memory_order_relaxed), std::memory_order_relaxed); seen[i][j].store(seen[i][j].load(std::memory_order_relaxed) | next[i][j].load(std::memory_order_relaxed), std::memory_order_relaxed); - change |= next[i][j].load(std::memory_order_relaxed); + + frontier_size = next[i][j].load() ? frontier_size + 1 : frontier_size.load(); + unseen_size = (~seen[i][j].load()) == 0 ? unseen_size - 1 : unseen_size.load(); } } +#elif defined(ATOMIC_BITSET) + if (next[i].load().any()) { + next[i].store(next[i].load() & ~seen[i].load(), std::memory_order_relaxed); + seen[i].store(seen[i].load() | next[i].load(), std::memory_order_relaxed); + + frontier_size = next[i].load().any() ? frontier_size + 1 : frontier_size.load(); + unseen_size = (~seen[i].load()).all() ? unseen_size - 1 : unseen_size.load(); + } +#else + if (next[i].any()) { + next[i] &= ~seen[i]; + seen[i] |= next[i]; + + frontier_size = next[i].any() ? frontier_size + 1 : frontier_size.load(); + unseen_size = seen[i].all() ? unseen_size - 1 : unseen_size.load(); + } +#endif } } - void ReachDetect(list_entry_t *result_data, bool& change) { + void IterativeLengthBottomUp() { + auto& bfs_state = state.global_bfs_state; + auto& seen = bfs_state->seen; + auto& visit = bfs_state->iter & 1 ? bfs_state->visit1 : bfs_state->visit2; + auto& next = bfs_state->iter & 1 ? bfs_state->visit2 : bfs_state->visit1; + auto& barrier = bfs_state->barrier; + int64_t *v = (int64_t *)state.global_csr->reverse_v; + vector &e = state.global_csr->reverse_e; + auto& edge_ids = state.global_csr->reverse_edge_ids; + auto& frontier_size = bfs_state->frontier_size; + auto& unseen_size = bfs_state->unseen_size; + auto& parents_v = bfs_state->parents_v; + auto& parents_e = bfs_state->parents_e; + + // clear next before each iteration + for (auto i = left; i < right; i++) { +#ifdef SEGMENT_BITSET + for (auto j = 0; j < 8; j++) { + next[i][j].store(0, std::memory_order_relaxed); + } +#else + next[i] = 0; +#endif + } + + barrier.Wait(); + + while (true) { + auto task = bfs_state->FetchTask(worker_id); + if (task.first == task.second) { + break; + } + auto start = task.first; + auto end = task.second; + +#ifdef SEGMENT_BITSET + idx_t old_next, new_next; +#else + std::bitset old_next, new_next; +#endif + + for (auto i = start; i < end; i++) { +#ifdef SEGMENT_BITSET + for (auto j = 0; j < 8; j++) { + if (~seen[i][j] == 0) { + continue; + } + + for (auto offset = v[i]; offset < v[i + 1]; offset++) { + auto n = e[offset]; + auto edge_id = edge_ids[offset]; + do { + old_next = next[i][j].load(); + new_next = old_next | visit[n][j].load(); + } while (!next[i][j].compare_exchange_weak(old_next, new_next)); + for (auto l = 0; l < LANE_LIMIT; l++) { + parents_v[i][l] = ((parents_v[i][l] == -1) && visit[n][l / 64].load(std::memory_order_relaxed) & ((idx_t)1 << (l % 64))) + ? n : parents_v[i][l]; + parents_e[i][l] = ((parents_e[i][l] == -1) && visit[n][l / 64].load(std::memory_order_relaxed) & ((idx_t)1 << (l % 64))) + ? edge_id : parents_e[i][l]; + } + } + + next[i][j].store(next[i][j].load() & ~seen[i][j].load()); + seen[i][j].store(seen[i][j].load() | next[i][j].load()); + + frontier_size = next[i][j].load() ? frontier_size + 1 : frontier_size.load(); + unseen_size = (~seen[i][j].load()) == 0 ? unseen_size - 1 : unseen_size.load(); + } +#elif defined(ATOMIC_BITSET) + if (seen[i].load().all()) { + continue; + } + + for (auto offset = v[i]; offset < v[i + 1]; offset++) { + auto n = e[offset]; + do { + old_next = next[i].load(); + new_next = old_next | visit[n].load(); + } while (!next[i].compare_exchange_weak(old_next, new_next)); + for (auto l = 0; l < LANE_LIMIT; l++) { + parents_v[i][l] = ((parents_v[i][l] == -1) && visit[n].load()[l]) + ? i : parents_v[i][l]; + parents_e[i][l] = ((parents_e[i][l] == -1) && visit[n].load()[l]) + ? edge_id : parents_e[i][l]; + } + } + + next[i].store(next[i].load() & ~seen[i].load()); + seen[i].store(seen[i].load() | next[i].load()); + + frontier_size = next[i].load().any() ? frontier_size + 1 : frontier_size.load(); + unseen_size = (~seen[i].load()).all() ? unseen_size - 1 : unseen_size.load(); +#else + if (seen[i].all()) { + continue; + } + + for (auto offset = v[i]; offset < v[i + 1]; offset++) { + auto n = e[offset]; + next[i] |= visit[n]; + for (auto l = 0; l < LANE_LIMIT; l++) { + parents_v[i][l] = ((parents_v[i][l] == -1) && visit[n][l]) + ? i : parents_v[i][l]; + parents_e[i][l] = ((parents_e[i][l] == -1) && visit[n][l]) + ? edge_id : parents_e[i][l]; + } + } + + next[i] &= ~seen[i]; + seen[i] |= next[i]; + + frontier_size = next[i].any() ? frontier_size + 1 : frontier_size.load(); + unseen_size = seen[i].all() ? unseen_size - 1 : unseen_size.load(); +#endif + } + } + } + + void ReachDetect() { auto &bfs_state = state.global_bfs_state; + auto &change = bfs_state->change; + auto &frontier_size = bfs_state->frontier_size; int64_t finished_searches = 0; // detect lanes that finished for (int64_t lane = 0; lane < LANE_LIMIT; lane++) { @@ -1024,20 +1396,37 @@ class PhysicalShortestPathTopDownTask : public ExecutorTask { if (search_num >= 0) { // active lane //! Check if dst for a source has been seen int64_t dst_pos = bfs_state->vdata_dst.sel->get_index(search_num); +#ifdef SEGMENT_BITSET if (bfs_state->seen[bfs_state->dst[dst_pos]][lane / 64] & ((idx_t)1 << (lane % 64))) { +#elif defined(ATOMIC_BITSET) + if (bfs_state->seen[bfs_state->dst[dst_pos]].load()[lane]) { +#else + if (bfs_state->seen[bfs_state->dst[dst_pos]][lane]) { +#endif finished_searches++; } + } else { + finished_searches++; } } if (finished_searches == LANE_LIMIT) { change = false; + } else { + if (frontier_size > 0) { + change = true; + } else { + change = false; + } } // into the next iteration bfs_state->iter++; } - void PathConstruction(Vector &result, list_entry_t *result_data, ValidityMask &result_validity) { + void PathConstruction() { auto &bfs_state = state.global_bfs_state; + auto &result = bfs_state->result.data[1]; + auto result_data = FlatVector::GetData(result); + auto &result_validity = FlatVector::Validity(result); //! Reconstruct the paths for (int64_t lane = 0; lane < LANE_LIMIT; lane++) { int64_t search_num = bfs_state->lane_to_num[lane]; @@ -1131,7 +1520,7 @@ class ParallelShortestPathEvent : public BasePipelineEvent { vector> bfs_tasks; for (idx_t tnum = 0; tnum < bfs_state->num_threads; tnum++) { - bfs_tasks.push_back(make_uniq(shared_from_this(), context, gstate, tnum)); + bfs_tasks.push_back(make_uniq(shared_from_this(), context, gstate, tnum)); } start_time = std::chrono::high_resolution_clock::now(); SetTasks(std::move(bfs_tasks)); @@ -1208,7 +1597,7 @@ class SequentialShortestPathEvent : public BasePipelineEvent { return change; } - static void ShortestPathFunction(const unique_ptr &csr, + static void ShortestPathFunction(const shared_ptr &csr, DataChunk &pairs, Vector &result) { int64_t v_size = csr->v_size; int64_t *v = (int64_t *)csr->v; @@ -1399,7 +1788,17 @@ PhysicalPathFinding::Finalize(Pipeline &pipeline, Event &event, auto &ts = TaskScheduler::GetScheduler(context); idx_t num_threads = ts.NumberOfThreads(); - gstate.global_bfs_state = make_uniq(all_pairs, csr->v_size - 2, num_threads, context); + auto& client_config = ClientConfig::GetConfig(context); + auto const barrier_type_idx = client_config.set_variables.find("experimental_path_finding_operator_barrier"); + auto barrier_type = barrier_type_idx != client_config.set_variables.end() ? barrier_type_idx->second.GetValue() : 0; + gstate.global_bfs_state = make_uniq(csr, all_pairs, csr->v_size - 2, num_threads, barrier_type, context); + + auto const alpha = client_config.set_variables.find("experimental_path_finding_operator_alpha"); + gstate.global_bfs_state->alpha = alpha != client_config.set_variables.end() ? alpha->second.GetValue() : 1024; + auto const beta = client_config.set_variables.find("experimental_path_finding_operator_beta"); + gstate.global_bfs_state->beta = beta != client_config.set_variables.end() ? beta->second.GetValue() : 64; + auto const task_size = client_config.set_variables.find("experimental_path_finding_operator_task_size"); + gstate.global_bfs_state->split_size = task_size != client_config.set_variables.end() ? task_size->second.GetValue() : 256; // auto bfs_event = make_shared(gstate, pipeline); // event.InsertEvent(std::move(std::dynamic_pointer_cast(bfs_event))); @@ -1444,7 +1843,15 @@ void PhysicalPathFinding::ScheduleBFSTasks(Pipeline &pipeline, Event &event, (uint64_t)0; // path of length 0 does not require a search } else { bfs_state->frontier_size++; +#ifdef SEGMENT_BITSET bfs_state->visit1[bfs_state->src[src_pos]][lane / 64] |= ((idx_t)1 << (lane % 64)); +#elif defined(ATOMIC_BITSET) + auto new_visit = bfs_state->visit1[bfs_state->src[src_pos]].load(); + new_visit[lane] = true; + bfs_state->visit1[bfs_state->src[src_pos]].store(new_visit); +#else + bfs_state->visit1[bfs_state->src[src_pos]][lane] = true; +#endif bfs_state->lane_to_num[lane] = search_num; // active lane break; } diff --git a/test/sql/path_finding/parallel_path_finding.test b/test/sql/path_finding/parallel_path_finding.test index ea3ace2b..921e5599 100644 --- a/test/sql/path_finding/parallel_path_finding.test +++ b/test/sql/path_finding/parallel_path_finding.test @@ -60,21 +60,27 @@ WHERE p.src BETWEEN (SELECT CREATE_CSR_EDGE( a.rowid, c.rowid, k.rowid, - t.cnt) FROM Know k + t.cnt, + t_r.cnt_r) FROM Know k JOIN student a on a.id = k.src JOIN student c on c.id = k.dst JOIN (SELECT count(k.src) cnt, a.rowid as a_rowid FROM student a LEFT JOIN know k ON k.src = a.id GROUP BY a.rowid) t - ON t.a_rowid = a.rowid) AND p.dst; + ON t.a_rowid = a.rowid + JOIN (SELECT count(k.dst) cnt_r, c.rowid as c_rowid + FROM student c + LEFT JOIN know k ON k.dst = c.id + GROUP BY c.rowid) t_r + ON t_r.c_rowid = c.rowid) AND p.dst; ---- 0 1 NULL 1 2 [1, 3, 0, 1, 2] 2 0 [2, 4, 0] statement ok -import database 'duckdb-pgq/data/SNB0.003'; +import database 'duckdb-pgq/data/SNB1-projected|'; statement ok select setseed(0.42) @@ -108,14 +114,20 @@ WHERE p.src BETWEEN (SELECT CREATE_CSR_EDGE( a.rowid, c.rowid, k.rowid, - t.cnt) FROM Person_knows_Person k + t.cnt, + t_r.cnt_r) FROM Person_knows_Person k JOIN Person a on a.id = k.Person1Id JOIN Person c on c.id = k.Person2Id JOIN (SELECT count(k.Person1Id) cnt, a.rowid as a_rowid FROM Person a LEFT JOIN Person_knows_Person k ON k.Person1Id = a.id GROUP BY a.rowid) t - ON t.a_rowid = a.rowid) AND p.dst; + ON t.a_rowid = a.rowid + JOIN (SELECT count(k.Person2Id) cnt_r, c.rowid as c_rowid + FROM Person c + LEFT JOIN Person_knows_Person k ON k.Person2Id = c.id + GROUP BY c.rowid) t_r + ON t_r.c_rowid = c.rowid) AND p.dst; ---- 42 22 NULL 3 33 2 @@ -175,14 +187,20 @@ WHERE p.src BETWEEN (SELECT CREATE_CSR_EDGE( a.rowid, c.rowid, k.rowid, - t.cnt) FROM Person_knows_Person k + t.cnt, + t_r.cnt_r) FROM Person_knows_Person k JOIN Person a on a.id = k.Person1Id JOIN Person c on c.id = k.Person2Id JOIN (SELECT count(k.Person1Id) cnt, a.rowid as a_rowid FROM Person a LEFT JOIN Person_knows_Person k ON k.Person1Id = a.id GROUP BY a.rowid) t - ON t.a_rowid = a.rowid) AND p.dst; + ON t.a_rowid = a.rowid + JOIN (SELECT count(k.Person2Id) cnt_r, c.rowid as c_rowid + FROM Person c + LEFT JOIN Person_knows_Person k ON k.Person2Id = c.id + GROUP BY c.rowid) t_r + ON t_r.c_rowid = c.rowid) AND p.dst; ---- 10 2 NULL From 1e7ec360ae55e444ffc21fd754eedebec7ae79cf Mon Sep 17 00:00:00 2001 From: Pingan Ren Date: Tue, 28 May 2024 12:08:08 +0200 Subject: [PATCH 119/249] top-down and bottom-up cost --- .../physical_path_finding_operator.cpp | 125 +++++++++++------- 1 file changed, 74 insertions(+), 51 deletions(-) diff --git a/src/operators/physical_path_finding_operator.cpp b/src/operators/physical_path_finding_operator.cpp index 16af965c..65762739 100644 --- a/src/operators/physical_path_finding_operator.cpp +++ b/src/operators/physical_path_finding_operator.cpp @@ -201,10 +201,10 @@ class GlobalBFSState { public: GlobalBFSState(shared_ptr csr_, shared_ptr pairs_, int64_t v_size_, idx_t num_threads_, idx_t barrier_type_, ClientContext &context_) - : csr(csr_), pairs(pairs_), iter(1), v_size(v_size_), change(false), started_searches(0), total_len(0), context(context_), - seen(v_size_), visit1(v_size_), visit2(v_size_), + : csr(csr_), pairs(pairs_), deduplicated_pairs(v_size_), pairs_mask(v_size_, false), iter(1), v_size(v_size_), change(false), + started_searches(0), total_len(0), context(context_), seen(v_size_), visit1(v_size_), visit2(v_size_), parents_v(v_size_, std::vector(LANE_LIMIT, -1)), parents_e(v_size_, std::vector(LANE_LIMIT, -1)), - frontier_size(0), unseen_size(v_size_ * 8), is_top_down(true), num_threads(num_threads_), task_queues(num_threads_), + top_down_cost(0), bottom_up_cost(0), is_top_down(true), num_threads(num_threads_), task_queues(num_threads_), task_queues_reverse(num_threads_), barrier(barrier_type_, num_threads_) { result.Initialize(context, {LogicalType::BIGINT, LogicalType::LIST(LogicalType::BIGINT)}, pairs_->size()); for (auto i = 0; i < LANE_LIMIT; i++) { @@ -216,6 +216,15 @@ class GlobalBFSState { dst_data.ToUnifiedFormat(pairs->size(), vdata_dst); src = FlatVector::GetData(src_data); dst = FlatVector::GetData(dst_data); + + // deduplicate pairs + for (idx_t i = 0; i < pairs->size(); i++) { + int64_t src_pos = vdata_src.sel->get_index(i); + int64_t dst_pos = vdata_dst.sel->get_index(i); + deduplicated_pairs[src[src_pos]].push_back(dst[dst_pos]); + pairs_mask[src[src_pos]] = true; + } + for (auto i = 0; i < v_size; i++) { #ifdef SEGMENT_BITSET for (auto j = 0; j < 8; j++) { @@ -238,8 +247,6 @@ class GlobalBFSState { void Clear() { iter = 1; change = false; - frontier_size = 0; - unseen_size = v_size * 8; for (auto i = 0; i < LANE_LIMIT; i++) { lane_to_num[i] = -1; } @@ -331,24 +338,16 @@ class GlobalBFSState { void DirectionSwitch() { // Debug print - string message = "frontier size: "; - message += std::to_string(frontier_size); - message += " unseen size: "; - message += std::to_string(unseen_size); - message += " iter "; - message += std::to_string(iter); + string message = "Top-down cost: "; + message += std::to_string(top_down_cost.load()); + message += ", Bottom-up cost: "; + message += std::to_string(bottom_up_cost.load()); Printer::Print(message); // Determine the switch of algorithms - if (is_top_down) { - auto Ctb = unseen_size / alpha; - if (frontier_size > Ctb) { - is_top_down = false; - } + if (top_down_cost > bottom_up_cost) { + is_top_down = false; } else { - auto Cbt = v_size / beta; - if (frontier_size < Cbt) { - is_top_down = true; - } + is_top_down = true; } // Debug print message = "Switch to "; @@ -356,19 +355,21 @@ class GlobalBFSState { message += " at iter "; message += std::to_string(iter); Printer::Print(message); - if (frontier_size > 0) { + if (top_down_cost > 0) { change = true; } else { change = false; } // clear the counters after the switch - frontier_size = 0; - unseen_size = v_size * 8; + top_down_cost = 0; + bottom_up_cost = 0; } public: shared_ptr csr; shared_ptr pairs; + vector> deduplicated_pairs; + vector pairs_mask; int64_t iter; int64_t v_size; atomic change; @@ -398,8 +399,8 @@ class GlobalBFSState { vector> parents_v; vector> parents_e; - atomic frontier_size; - atomic unseen_size; + atomic top_down_cost; + atomic bottom_up_cost; int64_t alpha = 1024; int64_t beta = 64; atomic is_top_down; @@ -570,9 +571,10 @@ class PhysicalIterativeTask : public ExecutorTask { auto& next = bfs_state->iter & 1 ? bfs_state->visit2 : bfs_state->visit1; auto& barrier = bfs_state->barrier; int64_t *v = (int64_t *)state.global_csr->v; + int64_t *reverse_v = (int64_t *)state.global_csr->reverse_v; vector &e = state.global_csr->e; - auto& frontier_size = bfs_state->frontier_size; - auto& unseen_size = bfs_state->unseen_size; + auto& top_down_cost = bfs_state->top_down_cost; + auto& bottom_up_cost = bfs_state->bottom_up_cost; // clear next before each iteration for (auto i = left; i < right; i++) { @@ -645,12 +647,16 @@ class PhysicalIterativeTask : public ExecutorTask { for (auto i = left; i < right; i++) { #ifdef SEGMENT_BITSET for (auto j = 0; j < 8; j++) { - if (next[i][j].load(std::memory_order_relaxed)) { - next[i][j].store(next[i][j].load(std::memory_order_relaxed) & ~seen[i][j].load(std::memory_order_relaxed), std::memory_order_relaxed); - seen[i][j].store(seen[i][j].load(std::memory_order_relaxed) | next[i][j].load(std::memory_order_relaxed), std::memory_order_relaxed); + if (next[i][j].load()) { + next[i][j].store(next[i][j].load() & ~seen[i][j].load()); + seen[i][j].store(seen[i][j].load() | next[i][j].load()); - frontier_size = next[i][j].load() ? frontier_size + 1 : frontier_size.load(); - unseen_size = (~seen[i][j].load()) == 0 ? unseen_size - 1 : unseen_size.load(); + if (next[i][j].load()) { + top_down_cost += v[i + 1] - v[i]; + } + if (~seen[i][j].load()) { + bottom_up_cost += reverse_v[i + 1] - reverse_v[i]; + } } } #elif defined(ATOMIC_BITSET) @@ -680,9 +686,10 @@ class PhysicalIterativeTask : public ExecutorTask { auto& next = bfs_state->iter & 1 ? bfs_state->visit2 : bfs_state->visit1; auto& barrier = bfs_state->barrier; int64_t *v = (int64_t *)state.global_csr->reverse_v; + int64_t *normal_v = (int64_t *)state.global_csr->v; vector &e = state.global_csr->reverse_e; - auto& frontier_size = bfs_state->frontier_size; - auto& unseen_size = bfs_state->unseen_size; + auto& top_down_cost = bfs_state->top_down_cost; + auto& bottom_up_cost = bfs_state->bottom_up_cost; // clear next before each iteration for (auto i = left; i < right; i++) { @@ -720,17 +727,23 @@ class PhysicalIterativeTask : public ExecutorTask { for (auto offset = v[i]; offset < v[i + 1]; offset++) { auto n = e[offset]; - do { - old_next = next[i][j].load(); - new_next = old_next | visit[n][j].load(); - } while (!next[i][j].compare_exchange_weak(old_next, new_next)); + if (visit[n][j].load(std::memory_order_relaxed)) { + do { + old_next = next[i][j].load(); + new_next = old_next | visit[n][j].load(); + } while (!next[i][j].compare_exchange_weak(old_next, new_next)); + } } next[i][j].store(next[i][j].load() & ~seen[i][j].load()); seen[i][j].store(seen[i][j].load() | next[i][j].load()); - frontier_size = next[i][j].load() ? frontier_size + 1 : frontier_size.load(); - unseen_size = (~seen[i][j].load()) == 0 ? unseen_size - 1 : unseen_size.load(); + if (next[i][j].load()) { + top_down_cost += normal_v[i + 1] - normal_v[i]; + } + if (~seen[i][j].load()) { + bottom_up_cost += v[i + 1] - v[i]; + } } #elif defined(ATOMIC_BITSET) if (seen[i].load().all()) { @@ -1139,10 +1152,11 @@ class PhysicalShortestPathTask : public ExecutorTask { auto& next = bfs_state->iter & 1 ? bfs_state->visit2 : bfs_state->visit1; auto& barrier = bfs_state->barrier; int64_t *v = (int64_t *)state.global_csr->v; + int64_t *reverse_v = (int64_t *)state.global_csr->reverse_v; vector &e = state.global_csr->e; auto& edge_ids = state.global_csr->edge_ids; - auto& frontier_size = bfs_state->frontier_size; - auto& unseen_size = bfs_state->unseen_size; + auto& top_down_cost = bfs_state->top_down_cost; + auto& bottom_up_cost = bfs_state->bottom_up_cost; auto& parents_v = bfs_state->parents_v; auto& parents_e = bfs_state->parents_e; @@ -1241,8 +1255,12 @@ class PhysicalShortestPathTask : public ExecutorTask { next[i][j].store(next[i][j].load(std::memory_order_relaxed) & ~seen[i][j].load(std::memory_order_relaxed), std::memory_order_relaxed); seen[i][j].store(seen[i][j].load(std::memory_order_relaxed) | next[i][j].load(std::memory_order_relaxed), std::memory_order_relaxed); - frontier_size = next[i][j].load() ? frontier_size + 1 : frontier_size.load(); - unseen_size = (~seen[i][j].load()) == 0 ? unseen_size - 1 : unseen_size.load(); + if (next[i][j].load(std::memory_order_relaxed)) { + top_down_cost += v[i + 1] - v[i]; + } + if (~seen[i][j].load(std::memory_order_relaxed)) { + bottom_up_cost += reverse_v[i + 1] - reverse_v[i]; + } } } #elif defined(ATOMIC_BITSET) @@ -1272,10 +1290,11 @@ class PhysicalShortestPathTask : public ExecutorTask { auto& next = bfs_state->iter & 1 ? bfs_state->visit2 : bfs_state->visit1; auto& barrier = bfs_state->barrier; int64_t *v = (int64_t *)state.global_csr->reverse_v; + int64_t *normal_v = (int64_t *)state.global_csr->v; vector &e = state.global_csr->reverse_e; auto& edge_ids = state.global_csr->reverse_edge_ids; - auto& frontier_size = bfs_state->frontier_size; - auto& unseen_size = bfs_state->unseen_size; + auto& top_down_cost = bfs_state->top_down_cost; + auto& bottom_up_cost = bfs_state->bottom_up_cost; auto& parents_v = bfs_state->parents_v; auto& parents_e = bfs_state->parents_e; @@ -1331,8 +1350,12 @@ class PhysicalShortestPathTask : public ExecutorTask { next[i][j].store(next[i][j].load() & ~seen[i][j].load()); seen[i][j].store(seen[i][j].load() | next[i][j].load()); - frontier_size = next[i][j].load() ? frontier_size + 1 : frontier_size.load(); - unseen_size = (~seen[i][j].load()) == 0 ? unseen_size - 1 : unseen_size.load(); + if (next[i][j].load()) { + top_down_cost += normal_v[i + 1] - normal_v[i]; + } + if (~seen[i][j].load()) { + bottom_up_cost += v[i + 1] - v[i]; + } } #elif defined(ATOMIC_BITSET) if (seen[i].load().all()) { @@ -1387,7 +1410,8 @@ class PhysicalShortestPathTask : public ExecutorTask { void ReachDetect() { auto &bfs_state = state.global_bfs_state; auto &change = bfs_state->change; - auto &frontier_size = bfs_state->frontier_size; + auto &top_down_cost = bfs_state->top_down_cost; + auto &bottom_up_cost = bfs_state->bottom_up_cost; int64_t finished_searches = 0; // detect lanes that finished for (int64_t lane = 0; lane < LANE_LIMIT; lane++) { @@ -1412,7 +1436,7 @@ class PhysicalShortestPathTask : public ExecutorTask { if (finished_searches == LANE_LIMIT) { change = false; } else { - if (frontier_size > 0) { + if (top_down_cost > 0 || bottom_up_cost > 0) { change = true; } else { change = false; @@ -1842,7 +1866,6 @@ void PhysicalPathFinding::ScheduleBFSTasks(Pipeline &pipeline, Event &event, result_data[search_num] = (uint64_t)0; // path of length 0 does not require a search } else { - bfs_state->frontier_size++; #ifdef SEGMENT_BITSET bfs_state->visit1[bfs_state->src[src_pos]][lane / 64] |= ((idx_t)1 << (lane % 64)); #elif defined(ATOMIC_BITSET) From b051999fe717a9ec4871b8badf2209fd389c13d7 Mon Sep 17 00:00:00 2001 From: Pingan Ren Date: Mon, 10 Jun 2024 00:45:51 +0200 Subject: [PATCH 120/249] code clean --- CMakeLists.txt | 1 + src/duckpgq_extension.cpp | 6 +- .../physical_path_finding_operator.cpp | 207 ++++++++++-------- 3 files changed, 113 insertions(+), 101 deletions(-) diff --git a/CMakeLists.txt b/CMakeLists.txt index f1cdc7d2..7b16a9f7 100644 --- a/CMakeLists.txt +++ b/CMakeLists.txt @@ -38,6 +38,7 @@ build_loadable_extension(${TARGET_NAME} " " ${EXTENSION_SOURCES}) # Link OpenSSL in both the static library as the loadable extension target_link_libraries(${EXTENSION_NAME} OpenSSL::SSL OpenSSL::Crypto) target_link_libraries(${LOADABLE_EXTENSION_NAME} OpenSSL::SSL OpenSSL::Crypto) +target_link_libraries(${EXTENSION_NAME} atomic) # >>>>>>> template/main install( diff --git a/src/duckpgq_extension.cpp b/src/duckpgq_extension.cpp index d924cc9b..6a554f3e 100644 --- a/src/duckpgq_extension.cpp +++ b/src/duckpgq_extension.cpp @@ -55,10 +55,8 @@ static void LoadInternal(DatabaseInstance &instance) { "Barrier variants selection", LogicalType::INTEGER, Value(0)); config.AddExtensionOption("experimental_path_finding_operator_task_size", "Number of vertices processed per thread at a time", LogicalType::INTEGER, Value(256)); - config.AddExtensionOption("experimental_path_finding_operator_alpha", - "Top-up and bottom-up search toggle parameter", LogicalType::INTEGER, Value(1024)); - config.AddExtensionOption("experimental_path_finding_operator_beta", - "Top-up and bottom-up search toggle parameter", LogicalType::INTEGER, Value(64)); + config.AddExtensionOption("experimental_path_finding_operator_sequential", + "Sequential path finding operator", LogicalType::BOOLEAN, Value(true)); Connection con(instance); con.BeginTransaction(); diff --git a/src/operators/physical_path_finding_operator.cpp b/src/operators/physical_path_finding_operator.cpp index 65762739..a488bd2d 100644 --- a/src/operators/physical_path_finding_operator.cpp +++ b/src/operators/physical_path_finding_operator.cpp @@ -16,8 +16,8 @@ #include #include -#define SEGMENT_BITSET -// #define ATOMIC_BITSET +// #define SEGMENT_BITSET +#define ATOMIC_BITSET // #define FINE_GRAINED_LOCK // #define ELEMENT_LOCK @@ -201,11 +201,11 @@ class GlobalBFSState { public: GlobalBFSState(shared_ptr csr_, shared_ptr pairs_, int64_t v_size_, idx_t num_threads_, idx_t barrier_type_, ClientContext &context_) - : csr(csr_), pairs(pairs_), deduplicated_pairs(v_size_), pairs_mask(v_size_, false), iter(1), v_size(v_size_), change(false), + : csr(csr_), pairs(pairs_), iter(1), v_size(v_size_), change(false), started_searches(0), total_len(0), context(context_), seen(v_size_), visit1(v_size_), visit2(v_size_), parents_v(v_size_, std::vector(LANE_LIMIT, -1)), parents_e(v_size_, std::vector(LANE_LIMIT, -1)), top_down_cost(0), bottom_up_cost(0), is_top_down(true), num_threads(num_threads_), task_queues(num_threads_), - task_queues_reverse(num_threads_), barrier(barrier_type_, num_threads_) { + task_queues_reverse(num_threads_), barrier(barrier_type_, num_threads_), locks(v_size_) { result.Initialize(context, {LogicalType::BIGINT, LogicalType::LIST(LogicalType::BIGINT)}, pairs_->size()); for (auto i = 0; i < LANE_LIMIT; i++) { lane_to_num[i] = -1; @@ -217,22 +217,14 @@ class GlobalBFSState { src = FlatVector::GetData(src_data); dst = FlatVector::GetData(dst_data); - // deduplicate pairs - for (idx_t i = 0; i < pairs->size(); i++) { - int64_t src_pos = vdata_src.sel->get_index(i); - int64_t dst_pos = vdata_dst.sel->get_index(i); - deduplicated_pairs[src[src_pos]].push_back(dst[dst_pos]); - pairs_mask[src[src_pos]] = true; - } - for (auto i = 0; i < v_size; i++) { #ifdef SEGMENT_BITSET for (auto j = 0; j < 8; j++) { - seen[i][j] = 0; + seen[i][j] = ~0; visit1[i][j] = 0; } #else - seen[i] = 0; + seen[i] = ~0; visit1[i] = 0; #endif for (auto j = 0; j < LANE_LIMIT; j++) { @@ -254,11 +246,11 @@ class GlobalBFSState { for (auto i = 0; i < v_size; i++) { #ifdef SEGMENT_BITSET for (auto j = 0; j < 8; j++) { - seen[i][j] = 0; + seen[i][j] = ~0; visit1[i][j] = 0; } #else - seen[i] = 0; + seen[i] = ~0; visit1[i] = 0; #endif for (auto j = 0; j < LANE_LIMIT; j++) { @@ -337,24 +329,13 @@ class GlobalBFSState { } void DirectionSwitch() { - // Debug print - string message = "Top-down cost: "; - message += std::to_string(top_down_cost.load()); - message += ", Bottom-up cost: "; - message += std::to_string(bottom_up_cost.load()); - Printer::Print(message); // Determine the switch of algorithms + // debug print if (top_down_cost > bottom_up_cost) { is_top_down = false; } else { is_top_down = true; } - // Debug print - message = "Switch to "; - message += is_top_down ? "top-down" : "bottom-up"; - message += " at iter "; - message += std::to_string(iter); - Printer::Print(message); if (top_down_cost > 0) { change = true; } else { @@ -368,8 +349,6 @@ class GlobalBFSState { public: shared_ptr csr; shared_ptr pairs; - vector> deduplicated_pairs; - vector pairs_mask; int64_t iter; int64_t v_size; atomic change; @@ -418,6 +397,7 @@ class GlobalBFSState { // lock for next mutable mutex lock; + mutable vector locks; }; class PathFindingGlobalState : public GlobalSinkState { @@ -575,6 +555,7 @@ class PhysicalIterativeTask : public ExecutorTask { vector &e = state.global_csr->e; auto& top_down_cost = bfs_state->top_down_cost; auto& bottom_up_cost = bfs_state->bottom_up_cost; + auto& lane_to_num = bfs_state->lane_to_num; // clear next before each iteration for (auto i = left; i < right; i++) { @@ -631,9 +612,9 @@ class PhysicalIterativeTask : public ExecutorTask { for (auto offset = v[i]; offset < v[i + 1]; offset++) { auto n = e[offset]; #ifdef FINE_GRAINED_LOCK - std::lock_guard lock(state.global_csr->lock); + std::lock_guard lock(bfs_state->lock); #elif defined(ELEMENT_LOCK) - std::lock_guard lock(state.global_csr->element_lock[n]); + std::lock_guard lock(bfs_state->locks[n]); #endif next[n] |= visit[i]; } @@ -651,29 +632,31 @@ class PhysicalIterativeTask : public ExecutorTask { next[i][j].store(next[i][j].load() & ~seen[i][j].load()); seen[i][j].store(seen[i][j].load() | next[i][j].load()); - if (next[i][j].load()) { - top_down_cost += v[i + 1] - v[i]; - } - if (~seen[i][j].load()) { - bottom_up_cost += reverse_v[i + 1] - reverse_v[i]; - } + top_down_cost += v[i + 1] - v[i]; + } + if (~seen[i][j].load()) { + bottom_up_cost += reverse_v[i + 1] - reverse_v[i]; } } #elif defined(ATOMIC_BITSET) if (next[i].load().any()) { - next[i].store(next[i].load() & ~seen[i].load(), std::memory_order_relaxed); - seen[i].store(seen[i].load() | next[i].load(), std::memory_order_relaxed); + next[i].store(next[i].load() & ~seen[i].load()); + seen[i].store(seen[i].load() | next[i].load()); - frontier_size = next[i].load().any() ? frontier_size + 1 : frontier_size.load(); - unseen_size = (~seen[i].load()).all() ? unseen_size - 1 : unseen_size.load(); + top_down_cost += v[i + 1] - v[i]; + } + if (~seen[i].load().all()) { + bottom_up_cost += reverse_v[i + 1] - reverse_v[i]; } #else if (next[i].any()) { next[i] &= ~seen[i]; seen[i] |= next[i]; - frontier_size = next[i].any() ? frontier_size + 1 : frontier_size.load(); - unseen_size = seen[i].all() ? unseen_size - 1 : unseen_size.load(); + top_down_cost += v[i + 1] - v[i]; + } + if (~seen[i].all()) { + bottom_up_cost += reverse_v[i + 1] - reverse_v[i]; } #endif } @@ -735,10 +718,10 @@ class PhysicalIterativeTask : public ExecutorTask { } } - next[i][j].store(next[i][j].load() & ~seen[i][j].load()); - seen[i][j].store(seen[i][j].load() | next[i][j].load()); - if (next[i][j].load()) { + next[i][j].store(next[i][j].load() & ~seen[i][j].load()); + seen[i][j].store(seen[i][j].load() | next[i][j].load()); + top_down_cost += normal_v[i + 1] - normal_v[i]; } if (~seen[i][j].load()) { @@ -758,11 +741,15 @@ class PhysicalIterativeTask : public ExecutorTask { } while (!next[i].compare_exchange_weak(old_next, new_next)); } - next[i].store(next[i].load() & ~seen[i].load()); - seen[i].store(seen[i].load() | next[i].load()); + if (next[i].load().any()) { + next[i].store(next[i].load() & ~seen[i].load()); + seen[i].store(seen[i].load() | next[i].load()); - frontier_size = next[i].load().any() ? frontier_size + 1 : frontier_size.load(); - unseen_size = (~seen[i].load()).all() ? unseen_size - 1 : unseen_size.load(); + top_down_cost += normal_v[i + 1] - normal_v[i]; + } + if (~seen[i].load().all()) { + bottom_up_cost += v[i + 1] - v[i]; + } #else if (seen[i].all()) { continue; @@ -773,11 +760,16 @@ class PhysicalIterativeTask : public ExecutorTask { next[i] |= visit[n]; } - next[i] &= ~seen[i]; - seen[i] |= next[i]; + if (next[i].any()) { + next[i] &= ~seen[i]; + seen[i] |= next[i]; + + top_down_cost += normal_v[i + 1] - normal_v[i]; + } + if (~seen[i].all()) { + bottom_up_cost += v[i + 1] - v[i]; + } - frontier_size = next[i].any() ? frontier_size + 1 : frontier_size.load(); - unseen_size = seen[i].all() ? unseen_size - 1 : unseen_size.load(); #endif } } @@ -1228,10 +1220,11 @@ class PhysicalShortestPathTask : public ExecutorTask { if (visit[i].any()) { for (auto offset = v[i]; offset < v[i + 1]; offset++) { auto n = e[offset]; + auto edge_id = edge_ids[offset]; #ifdef FINE_GRAINED_LOCK - std::lock_guard lock(state.global_csr->lock); + std::lock_guard lock(bfs_state->lock); #elif defined(ELEMENT_LOCK) - std::lock_guard lock(state.global_csr->element_lock[n]); + std::lock_guard lock(bfs_state->locks[n]); #endif next[n] |= visit[i]; for (auto l = 0; l < LANE_LIMIT; l++) { @@ -1251,33 +1244,35 @@ class PhysicalShortestPathTask : public ExecutorTask { for (auto i = left; i < right; i++) { #ifdef SEGMENT_BITSET for (auto j = 0; j < 8; j++) { - if (next[i][j].load(std::memory_order_relaxed)) { - next[i][j].store(next[i][j].load(std::memory_order_relaxed) & ~seen[i][j].load(std::memory_order_relaxed), std::memory_order_relaxed); - seen[i][j].store(seen[i][j].load(std::memory_order_relaxed) | next[i][j].load(std::memory_order_relaxed), std::memory_order_relaxed); + if (next[i][j].load()) { + next[i][j].store(next[i][j].load() & ~seen[i][j].load()); + seen[i][j].store(seen[i][j].load() | next[i][j].load()); - if (next[i][j].load(std::memory_order_relaxed)) { - top_down_cost += v[i + 1] - v[i]; - } - if (~seen[i][j].load(std::memory_order_relaxed)) { - bottom_up_cost += reverse_v[i + 1] - reverse_v[i]; - } + top_down_cost += v[i + 1] - v[i]; + } + if (~seen[i][j].load()) { + bottom_up_cost += reverse_v[i + 1] - reverse_v[i]; } } #elif defined(ATOMIC_BITSET) if (next[i].load().any()) { - next[i].store(next[i].load() & ~seen[i].load(), std::memory_order_relaxed); - seen[i].store(seen[i].load() | next[i].load(), std::memory_order_relaxed); + next[i].store(next[i].load() & ~seen[i].load()); + seen[i].store(seen[i].load() | next[i].load()); - frontier_size = next[i].load().any() ? frontier_size + 1 : frontier_size.load(); - unseen_size = (~seen[i].load()).all() ? unseen_size - 1 : unseen_size.load(); + top_down_cost += v[i + 1] - v[i]; + } + if (~seen[i].load().all()) { + bottom_up_cost += reverse_v[i + 1] - reverse_v[i]; } #else if (next[i].any()) { next[i] &= ~seen[i]; seen[i] |= next[i]; - frontier_size = next[i].any() ? frontier_size + 1 : frontier_size.load(); - unseen_size = seen[i].all() ? unseen_size - 1 : unseen_size.load(); + top_down_cost += v[i + 1] - v[i]; + } + if (~seen[i].all()) { + bottom_up_cost += reverse_v[i + 1] - reverse_v[i]; } #endif } @@ -1346,11 +1341,11 @@ class PhysicalShortestPathTask : public ExecutorTask { ? edge_id : parents_e[i][l]; } } - - next[i][j].store(next[i][j].load() & ~seen[i][j].load()); - seen[i][j].store(seen[i][j].load() | next[i][j].load()); - + if (next[i][j].load()) { + next[i][j].store(next[i][j].load() & ~seen[i][j].load(), std::memory_order_relaxed); + seen[i][j].store(seen[i][j].load() | next[i][j].load(), std::memory_order_relaxed); + top_down_cost += normal_v[i + 1] - normal_v[i]; } if (~seen[i][j].load()) { @@ -1364,6 +1359,7 @@ class PhysicalShortestPathTask : public ExecutorTask { for (auto offset = v[i]; offset < v[i + 1]; offset++) { auto n = e[offset]; + auto edge_id = edge_ids[offset]; do { old_next = next[i].load(); new_next = old_next | visit[n].load(); @@ -1376,11 +1372,16 @@ class PhysicalShortestPathTask : public ExecutorTask { } } - next[i].store(next[i].load() & ~seen[i].load()); - seen[i].store(seen[i].load() | next[i].load()); + if (next[i].load().any()) { + next[i].store(next[i].load() & ~seen[i].load()); + seen[i].store(seen[i].load() | next[i].load()); + + top_down_cost += normal_v[i + 1] - normal_v[i]; + } + if (~seen[i].load().all()) { + bottom_up_cost += v[i + 1] - v[i]; + } - frontier_size = next[i].load().any() ? frontier_size + 1 : frontier_size.load(); - unseen_size = (~seen[i].load()).all() ? unseen_size - 1 : unseen_size.load(); #else if (seen[i].all()) { continue; @@ -1388,6 +1389,7 @@ class PhysicalShortestPathTask : public ExecutorTask { for (auto offset = v[i]; offset < v[i + 1]; offset++) { auto n = e[offset]; + auto edge_id = edge_ids[offset]; next[i] |= visit[n]; for (auto l = 0; l < LANE_LIMIT; l++) { parents_v[i][l] = ((parents_v[i][l] == -1) && visit[n][l]) @@ -1397,11 +1399,15 @@ class PhysicalShortestPathTask : public ExecutorTask { } } - next[i] &= ~seen[i]; - seen[i] |= next[i]; + if (next[i].any()) { + next[i] &= ~seen[i]; + seen[i] |= next[i]; - frontier_size = next[i].any() ? frontier_size + 1 : frontier_size.load(); - unseen_size = seen[i].all() ? unseen_size - 1 : unseen_size.load(); + top_down_cost += normal_v[i + 1] - normal_v[i]; + } + if (~seen[i].any()) { + bottom_up_cost += v[i + 1] - v[i]; + } #endif } } @@ -1817,19 +1823,18 @@ PhysicalPathFinding::Finalize(Pipeline &pipeline, Event &event, auto barrier_type = barrier_type_idx != client_config.set_variables.end() ? barrier_type_idx->second.GetValue() : 0; gstate.global_bfs_state = make_uniq(csr, all_pairs, csr->v_size - 2, num_threads, barrier_type, context); - auto const alpha = client_config.set_variables.find("experimental_path_finding_operator_alpha"); - gstate.global_bfs_state->alpha = alpha != client_config.set_variables.end() ? alpha->second.GetValue() : 1024; - auto const beta = client_config.set_variables.find("experimental_path_finding_operator_beta"); - gstate.global_bfs_state->beta = beta != client_config.set_variables.end() ? beta->second.GetValue() : 64; auto const task_size = client_config.set_variables.find("experimental_path_finding_operator_task_size"); gstate.global_bfs_state->split_size = task_size != client_config.set_variables.end() ? task_size->second.GetValue() : 256; - // auto bfs_event = make_shared(gstate, pipeline); - // event.InsertEvent(std::move(std::dynamic_pointer_cast(bfs_event))); - - // Schedule the first round of BFS tasks - if (all_pairs->size() > 0) { - ScheduleBFSTasks(pipeline, event, gstate); + auto const sequential = client_config.set_variables.find("experimental_path_finding_operator_sequential"); + if (sequential != client_config.set_variables.end() && sequential->second.GetValue()) { + // Schedule the first round of BFS tasks + if (all_pairs->size() > 0) { + ScheduleBFSTasks(pipeline, event, gstate); + } + } else { + auto bfs_event = make_shared(gstate, pipeline); + event.InsertEvent(std::move(std::dynamic_pointer_cast(bfs_event))); } } @@ -1868,12 +1873,23 @@ void PhysicalPathFinding::ScheduleBFSTasks(Pipeline &pipeline, Event &event, } else { #ifdef SEGMENT_BITSET bfs_state->visit1[bfs_state->src[src_pos]][lane / 64] |= ((idx_t)1 << (lane % 64)); + for (int i = 0; i < bfs_state->v_size; i++) { + bfs_state->seen[i][lane / 64] &= ~((idx_t)1 << (lane % 64)); + } #elif defined(ATOMIC_BITSET) auto new_visit = bfs_state->visit1[bfs_state->src[src_pos]].load(); new_visit[lane] = true; bfs_state->visit1[bfs_state->src[src_pos]].store(new_visit); + for (int i = 0; i < bfs_state->v_size; i++) { + auto new_seen = bfs_state->seen[i].load(); + new_seen[lane] = false; + bfs_state->seen[i].store(new_seen); + } #else bfs_state->visit1[bfs_state->src[src_pos]][lane] = true; + for (int i = 0; i < bfs_state->v_size; i++) { + bfs_state->seen[i][lane] = false; + } #endif bfs_state->lane_to_num[lane] = search_num; // active lane break; @@ -1961,9 +1977,6 @@ PhysicalPathFinding::GetData(ExecutionContext &context, DataChunk &result, return SourceResultType::FINISHED; } pf_bfs_state->result.SetCardinality(*pf_bfs_state->pairs); - // pf_bfs_state->result.Print(); - string message = "Algorithm running time: " + to_string(pf_bfs_state->time_elapsed.count()) + " us"; - Printer::Print(message); result.Move(*pf_bfs_state->pairs); auto result_path = make_uniq(); From 0fa68fcc52df573d3eef7d83c3c809fdf1f4c1cb Mon Sep 17 00:00:00 2001 From: Pingan Ren Date: Mon, 10 Jun 2024 15:57:39 +0200 Subject: [PATCH 121/249] no direction switch --- .../physical_path_finding_operator.cpp | 112 ++++++++++-------- 1 file changed, 60 insertions(+), 52 deletions(-) diff --git a/src/operators/physical_path_finding_operator.cpp b/src/operators/physical_path_finding_operator.cpp index a488bd2d..721bb765 100644 --- a/src/operators/physical_path_finding_operator.cpp +++ b/src/operators/physical_path_finding_operator.cpp @@ -17,9 +17,9 @@ #include // #define SEGMENT_BITSET -#define ATOMIC_BITSET +// #define ATOMIC_BITSET // #define FINE_GRAINED_LOCK -// #define ELEMENT_LOCK +#define ELEMENT_LOCK namespace duckdb { @@ -351,7 +351,7 @@ class GlobalBFSState { shared_ptr pairs; int64_t iter; int64_t v_size; - atomic change; + bool change; idx_t started_searches; int64_t total_len; int64_t *src; @@ -521,16 +521,18 @@ class PhysicalIterativeTask : public ExecutorTask { do { bfs_state->InitTask(worker_id); - if (bfs_state->is_top_down) { - IterativeLengthTopDown(); - } else { - IterativeLengthBottomUp(); - } + // if (bfs_state->is_top_down) { + // IterativeLengthTopDown(); + // } else { + // IterativeLengthBottomUp(); + // } + IterativeLengthTopDown(); + barrier.Wait(); if (worker_id == 0) { ReachDetect(); - bfs_state->DirectionSwitch(); + // bfs_state->DirectionSwitch(); } barrier.Wait(); } while (change); @@ -556,6 +558,7 @@ class PhysicalIterativeTask : public ExecutorTask { auto& top_down_cost = bfs_state->top_down_cost; auto& bottom_up_cost = bfs_state->bottom_up_cost; auto& lane_to_num = bfs_state->lane_to_num; + auto& change = bfs_state->change; // clear next before each iteration for (auto i = left; i < right; i++) { @@ -623,6 +626,7 @@ class PhysicalIterativeTask : public ExecutorTask { } } + change = false; barrier.Wait(); for (auto i = left; i < right; i++) { @@ -631,33 +635,33 @@ class PhysicalIterativeTask : public ExecutorTask { if (next[i][j].load()) { next[i][j].store(next[i][j].load() & ~seen[i][j].load()); seen[i][j].store(seen[i][j].load() | next[i][j].load()); - - top_down_cost += v[i + 1] - v[i]; - } - if (~seen[i][j].load()) { - bottom_up_cost += reverse_v[i + 1] - reverse_v[i]; + change |= next[i][j].load(); + // top_down_cost += v[i + 1] - v[i]; } + // if (~seen[i][j].load()) { + // bottom_up_cost += reverse_v[i + 1] - reverse_v[i]; + // } } #elif defined(ATOMIC_BITSET) if (next[i].load().any()) { next[i].store(next[i].load() & ~seen[i].load()); seen[i].store(seen[i].load() | next[i].load()); - - top_down_cost += v[i + 1] - v[i]; - } - if (~seen[i].load().all()) { - bottom_up_cost += reverse_v[i + 1] - reverse_v[i]; + change |= next[i].load().any(); + // top_down_cost += v[i + 1] - v[i]; } + // if (~seen[i].load().all()) { + // bottom_up_cost += reverse_v[i + 1] - reverse_v[i]; + // } #else if (next[i].any()) { next[i] &= ~seen[i]; seen[i] |= next[i]; - - top_down_cost += v[i + 1] - v[i]; - } - if (~seen[i].all()) { - bottom_up_cost += reverse_v[i + 1] - reverse_v[i]; + change |= next[i].any(); + // top_down_cost += v[i + 1] - v[i]; } + // if (~seen[i].all()) { + // bottom_up_cost += reverse_v[i + 1] - reverse_v[i]; + // } #endif } } @@ -1113,16 +1117,17 @@ class PhysicalShortestPathTask : public ExecutorTask { do { bfs_state->InitTask(worker_id); - if (bfs_state->is_top_down) { - IterativeLengthTopDown(); - } else { - IterativeLengthBottomUp(); - } + // if (bfs_state->is_top_down) { + // IterativeLengthTopDown(); + // } else { + // IterativeLengthBottomUp(); + // } + IterativeLengthTopDown(); barrier.Wait(); if (worker_id == 0) { ReachDetect(); - bfs_state->DirectionSwitch(); + // bfs_state->DirectionSwitch(); } barrier.Wait(); @@ -1151,6 +1156,7 @@ class PhysicalShortestPathTask : public ExecutorTask { auto& bottom_up_cost = bfs_state->bottom_up_cost; auto& parents_v = bfs_state->parents_v; auto& parents_e = bfs_state->parents_e; + auto& change = bfs_state->change; // clear next before each iteration for (auto i = left; i < right; i++) { @@ -1239,6 +1245,7 @@ class PhysicalShortestPathTask : public ExecutorTask { } } + change = false; barrier.Wait(); for (auto i = left; i < right; i++) { @@ -1247,33 +1254,33 @@ class PhysicalShortestPathTask : public ExecutorTask { if (next[i][j].load()) { next[i][j].store(next[i][j].load() & ~seen[i][j].load()); seen[i][j].store(seen[i][j].load() | next[i][j].load()); - - top_down_cost += v[i + 1] - v[i]; - } - if (~seen[i][j].load()) { - bottom_up_cost += reverse_v[i + 1] - reverse_v[i]; + change |= next[i][j].load(); + // top_down_cost += v[i + 1] - v[i]; } + // if (~seen[i][j].load()) { + // bottom_up_cost += reverse_v[i + 1] - reverse_v[i]; + // } } #elif defined(ATOMIC_BITSET) if (next[i].load().any()) { next[i].store(next[i].load() & ~seen[i].load()); seen[i].store(seen[i].load() | next[i].load()); - - top_down_cost += v[i + 1] - v[i]; - } - if (~seen[i].load().all()) { - bottom_up_cost += reverse_v[i + 1] - reverse_v[i]; + change |= next[i].load().any(); + // top_down_cost += v[i + 1] - v[i]; } + // if (~seen[i].load().all()) { + // bottom_up_cost += reverse_v[i + 1] - reverse_v[i]; + // } #else if (next[i].any()) { next[i] &= ~seen[i]; seen[i] |= next[i]; - - top_down_cost += v[i + 1] - v[i]; - } - if (~seen[i].all()) { - bottom_up_cost += reverse_v[i + 1] - reverse_v[i]; + change |= next[i].any(); + // top_down_cost += v[i + 1] - v[i]; } + // if (~seen[i].all()) { + // bottom_up_cost += reverse_v[i + 1] - reverse_v[i]; + // } #endif } } @@ -1441,13 +1448,14 @@ class PhysicalShortestPathTask : public ExecutorTask { } if (finished_searches == LANE_LIMIT) { change = false; - } else { - if (top_down_cost > 0 || bottom_up_cost > 0) { - change = true; - } else { - change = false; - } - } + } + // else { + // if (top_down_cost > 0 || bottom_up_cost > 0) { + // change = true; + // } else { + // change = false; + // } + // } // into the next iteration bfs_state->iter++; } From edc725cadd77c4e030d3c409a244466727d17352 Mon Sep 17 00:00:00 2001 From: Pingan Ren Date: Thu, 13 Jun 2024 21:17:09 +0200 Subject: [PATCH 122/249] no bottom up --- .../physical_path_finding_operator.hpp | 60 +- .../physical_path_finding_operator.cpp | 535 +++++++++--------- 2 files changed, 298 insertions(+), 297 deletions(-) diff --git a/src/include/duckpgq/operators/physical_path_finding_operator.hpp b/src/include/duckpgq/operators/physical_path_finding_operator.hpp index 1ddc6007..166741eb 100644 --- a/src/include/duckpgq/operators/physical_path_finding_operator.hpp +++ b/src/include/duckpgq/operators/physical_path_finding_operator.hpp @@ -27,17 +27,17 @@ class PhysicalPathFinding : public PhysicalComparisonJoin { if (v) { delete[] v; } - if (reverse_v) { - delete[] reverse_v; - } + // if (reverse_v) { + // delete[] reverse_v; + // } } atomic *v; - atomic *reverse_v; + // atomic *reverse_v; vector e; - vector reverse_e; + // vector reverse_e; vector edge_ids; - vector reverse_edge_ids; + // vector reverse_edge_ids; vector w; vector w_double; bool initialized_v = false; @@ -76,30 +76,30 @@ class PhysicalPathFinding : public PhysicalComparisonJoin { } else { result += "not initialized"; } - result += "\nReverse V: "; - if (initialized_v) { - for (idx_t i = 0; i < v_size; ++i) { - result += std::to_string(reverse_v[i]) + ' '; - } - } else { - result += "not initialized"; - } - result += "\nReverse E: "; - if (initialized_e) { - for (auto i : reverse_e) { - result += std::to_string(i) + " "; - } - } else { - result += "not initialized"; - } - result += "\nReverse Edge IDs: "; - if (initialized_e) { - for (auto i : reverse_edge_ids) { - result += std::to_string(i) + " "; - } - } else { - result += "not initialized"; - } + // result += "\nReverse V: "; + // if (initialized_v) { + // for (idx_t i = 0; i < v_size; ++i) { + // result += std::to_string(reverse_v[i]) + ' '; + // } + // } else { + // result += "not initialized"; + // } + // result += "\nReverse E: "; + // if (initialized_e) { + // for (auto i : reverse_e) { + // result += std::to_string(i) + " "; + // } + // } else { + // result += "not initialized"; + // } + // result += "\nReverse Edge IDs: "; + // if (initialized_e) { + // for (auto i : reverse_edge_ids) { + // result += std::to_string(i) + " "; + // } + // } else { + // result += "not initialized"; + // } result += "\nW: "; if (initialized_w) { for (auto i : w) { diff --git a/src/operators/physical_path_finding_operator.cpp b/src/operators/physical_path_finding_operator.cpp index 721bb765..81a11bfb 100644 --- a/src/operators/physical_path_finding_operator.cpp +++ b/src/operators/physical_path_finding_operator.cpp @@ -44,7 +44,7 @@ void PhysicalPathFinding::GlobalCompressedSparseRow::InitializeVertex( v_size = v_size_ + 2; try { v = new std::atomic[v_size]; - reverse_v = new std::atomic[v_size]; + // reverse_v = new std::atomic[v_size]; } catch (std::bad_alloc const &) { throw InternalException( "Unable to initialize vector of size for csr vertex table " @@ -52,7 +52,7 @@ void PhysicalPathFinding::GlobalCompressedSparseRow::InitializeVertex( } for (idx_t i = 0; i < v_size; ++i) { v[i].store(0); - reverse_v[i].store(0); + // reverse_v[i].store(0); } initialized_v = true; } @@ -64,16 +64,16 @@ void PhysicalPathFinding::GlobalCompressedSparseRow::InitializeEdge( } try { e.resize(e_size, 0); - reverse_e.resize(e_size, 0); + // reverse_e.resize(e_size, 0); edge_ids.resize(e_size, 0); - reverse_edge_ids.resize(e_size, 0); + // reverse_edge_ids.resize(e_size, 0); } catch (std::bad_alloc const &) { throw InternalException("Unable to initialize vector of size for csr " "edge table representation"); } for (idx_t i = 1; i < v_size; i++) { v[i] += v[i - 1]; - reverse_v[i] += reverse_v[i - 1]; + // reverse_v[i] += reverse_v[i - 1]; } initialized_e = true; } @@ -126,14 +126,14 @@ void PathFindingLocalState::CreateCSRVertex(DataChunk &input, edge_count = edge_count + cnt; return edge_count; }); - BinaryExecutor::Execute( - input.data[7], input.data[6], result, input.size(), - [&](const int64_t dst, const int64_t cnt) { - int64_t edge_count = 0; - global_csr.reverse_v[dst + 2] = cnt; - edge_count = edge_count + cnt; - return edge_count; - }); + // BinaryExecutor::Execute( + // input.data[7], input.data[6], result, input.size(), + // [&](const int64_t dst, const int64_t cnt) { + // int64_t edge_count = 0; + // global_csr.reverse_v[dst + 2] = cnt; + // edge_count = edge_count + cnt; + // return edge_count; + // }); } class Barrier { @@ -268,7 +268,8 @@ class GlobalBFSState { for (auto& queue : queues) { vector>> worker_tasks(num_threads); auto cur_worker = 0; - int64_t *v = is_top_down ? (int64_t*)csr->v : (int64_t*)csr->reverse_v; + // int64_t *v = is_top_down ? (int64_t*)csr->v : (int64_t*)csr->reverse_v; + int64_t *v = (int64_t*)csr->v; int64_t current_task_edges = 0; idx_t current_task_start = 0; for (idx_t i = 0; i < (idx_t)v_size; i++) { @@ -553,7 +554,7 @@ class PhysicalIterativeTask : public ExecutorTask { auto& next = bfs_state->iter & 1 ? bfs_state->visit2 : bfs_state->visit1; auto& barrier = bfs_state->barrier; int64_t *v = (int64_t *)state.global_csr->v; - int64_t *reverse_v = (int64_t *)state.global_csr->reverse_v; + // int64_t *reverse_v = (int64_t *)state.global_csr->reverse_v; vector &e = state.global_csr->e; auto& top_down_cost = bfs_state->top_down_cost; auto& bottom_up_cost = bfs_state->bottom_up_cost; @@ -666,118 +667,118 @@ class PhysicalIterativeTask : public ExecutorTask { } } - void IterativeLengthBottomUp() { - auto& bfs_state = state.global_bfs_state; - auto& seen = bfs_state->seen; - auto& visit = bfs_state->iter & 1 ? bfs_state->visit1 : bfs_state->visit2; - auto& next = bfs_state->iter & 1 ? bfs_state->visit2 : bfs_state->visit1; - auto& barrier = bfs_state->barrier; - int64_t *v = (int64_t *)state.global_csr->reverse_v; - int64_t *normal_v = (int64_t *)state.global_csr->v; - vector &e = state.global_csr->reverse_e; - auto& top_down_cost = bfs_state->top_down_cost; - auto& bottom_up_cost = bfs_state->bottom_up_cost; - - // clear next before each iteration - for (auto i = left; i < right; i++) { -#ifdef SEGMENT_BITSET - for (auto j = 0; j < 8; j++) { - next[i][j].store(0, std::memory_order_relaxed); - } -#else - next[i] = 0; -#endif - } +// void IterativeLengthBottomUp() { +// auto& bfs_state = state.global_bfs_state; +// auto& seen = bfs_state->seen; +// auto& visit = bfs_state->iter & 1 ? bfs_state->visit1 : bfs_state->visit2; +// auto& next = bfs_state->iter & 1 ? bfs_state->visit2 : bfs_state->visit1; +// auto& barrier = bfs_state->barrier; +// int64_t *v = (int64_t *)state.global_csr->reverse_v; +// int64_t *normal_v = (int64_t *)state.global_csr->v; +// vector &e = state.global_csr->reverse_e; +// auto& top_down_cost = bfs_state->top_down_cost; +// auto& bottom_up_cost = bfs_state->bottom_up_cost; + +// // clear next before each iteration +// for (auto i = left; i < right; i++) { +// #ifdef SEGMENT_BITSET +// for (auto j = 0; j < 8; j++) { +// next[i][j].store(0, std::memory_order_relaxed); +// } +// #else +// next[i] = 0; +// #endif +// } - barrier.Wait(); - - while (true) { - auto task = bfs_state->FetchTask(worker_id); - if (task.first == task.second) { - break; - } - auto start = task.first; - auto end = task.second; - -#ifdef SEGMENT_BITSET - idx_t old_next, new_next; -#else - std::bitset old_next, new_next; -#endif - - for (auto i = start; i < end; i++) { -#ifdef SEGMENT_BITSET - for (auto j = 0; j < 8; j++) { - if (~seen[i][j] == 0) { - continue; - } - - for (auto offset = v[i]; offset < v[i + 1]; offset++) { - auto n = e[offset]; - if (visit[n][j].load(std::memory_order_relaxed)) { - do { - old_next = next[i][j].load(); - new_next = old_next | visit[n][j].load(); - } while (!next[i][j].compare_exchange_weak(old_next, new_next)); - } - } - - if (next[i][j].load()) { - next[i][j].store(next[i][j].load() & ~seen[i][j].load()); - seen[i][j].store(seen[i][j].load() | next[i][j].load()); - - top_down_cost += normal_v[i + 1] - normal_v[i]; - } - if (~seen[i][j].load()) { - bottom_up_cost += v[i + 1] - v[i]; - } - } -#elif defined(ATOMIC_BITSET) - if (seen[i].load().all()) { - continue; - } - - for (auto offset = v[i]; offset < v[i + 1]; offset++) { - auto n = e[offset]; - do { - old_next = next[i].load(); - new_next = old_next | visit[n].load(); - } while (!next[i].compare_exchange_weak(old_next, new_next)); - } - - if (next[i].load().any()) { - next[i].store(next[i].load() & ~seen[i].load()); - seen[i].store(seen[i].load() | next[i].load()); - - top_down_cost += normal_v[i + 1] - normal_v[i]; - } - if (~seen[i].load().all()) { - bottom_up_cost += v[i + 1] - v[i]; - } -#else - if (seen[i].all()) { - continue; - } - - for (auto offset = v[i]; offset < v[i + 1]; offset++) { - auto n = e[offset]; - next[i] |= visit[n]; - } - - if (next[i].any()) { - next[i] &= ~seen[i]; - seen[i] |= next[i]; - - top_down_cost += normal_v[i + 1] - normal_v[i]; - } - if (~seen[i].all()) { - bottom_up_cost += v[i + 1] - v[i]; - } - -#endif - } - } - } +// barrier.Wait(); + +// while (true) { +// auto task = bfs_state->FetchTask(worker_id); +// if (task.first == task.second) { +// break; +// } +// auto start = task.first; +// auto end = task.second; + +// #ifdef SEGMENT_BITSET +// idx_t old_next, new_next; +// #else +// std::bitset old_next, new_next; +// #endif + +// for (auto i = start; i < end; i++) { +// #ifdef SEGMENT_BITSET +// for (auto j = 0; j < 8; j++) { +// if (~seen[i][j] == 0) { +// continue; +// } + +// for (auto offset = v[i]; offset < v[i + 1]; offset++) { +// auto n = e[offset]; +// if (visit[n][j].load(std::memory_order_relaxed)) { +// do { +// old_next = next[i][j].load(); +// new_next = old_next | visit[n][j].load(); +// } while (!next[i][j].compare_exchange_weak(old_next, new_next)); +// } +// } + +// if (next[i][j].load()) { +// next[i][j].store(next[i][j].load() & ~seen[i][j].load()); +// seen[i][j].store(seen[i][j].load() | next[i][j].load()); + +// top_down_cost += normal_v[i + 1] - normal_v[i]; +// } +// if (~seen[i][j].load()) { +// bottom_up_cost += v[i + 1] - v[i]; +// } +// } +// #elif defined(ATOMIC_BITSET) +// if (seen[i].load().all()) { +// continue; +// } + +// for (auto offset = v[i]; offset < v[i + 1]; offset++) { +// auto n = e[offset]; +// do { +// old_next = next[i].load(); +// new_next = old_next | visit[n].load(); +// } while (!next[i].compare_exchange_weak(old_next, new_next)); +// } + +// if (next[i].load().any()) { +// next[i].store(next[i].load() & ~seen[i].load()); +// seen[i].store(seen[i].load() | next[i].load()); + +// top_down_cost += normal_v[i + 1] - normal_v[i]; +// } +// if (~seen[i].load().all()) { +// bottom_up_cost += v[i + 1] - v[i]; +// } +// #else +// if (seen[i].all()) { +// continue; +// } + +// for (auto offset = v[i]; offset < v[i + 1]; offset++) { +// auto n = e[offset]; +// next[i] |= visit[n]; +// } + +// if (next[i].any()) { +// next[i] &= ~seen[i]; +// seen[i] |= next[i]; + +// top_down_cost += normal_v[i + 1] - normal_v[i]; +// } +// if (~seen[i].all()) { +// bottom_up_cost += v[i + 1] - v[i]; +// } + +// #endif +// } +// } +// } void ReachDetect() { auto &bfs_state = state.global_bfs_state; @@ -861,14 +862,14 @@ class PhysicalCSREdgeCreationTask : public ExecutorTask { global_csr->edge_ids[static_cast(pos) - 1] = edge_id; return 1; }); - TernaryExecutor::Execute( - input.data[7], input.data[4], input.data[2], result, input.size(), - [&](int64_t dst, int64_t src, int64_t edge_id) { - const auto pos = ++global_csr->reverse_v[dst + 1]; - global_csr->reverse_e[static_cast(pos) - 1] = src; - global_csr->reverse_edge_ids[static_cast(pos) - 1] = edge_id; - return 1; - }); + // TernaryExecutor::Execute( + // input.data[7], input.data[4], input.data[2], result, input.size(), + // [&](int64_t dst, int64_t src, int64_t edge_id) { + // const auto pos = ++global_csr->reverse_v[dst + 1]; + // global_csr->reverse_e[static_cast(pos) - 1] = src; + // global_csr->reverse_edge_ids[static_cast(pos) - 1] = edge_id; + // return 1; + // }); } event->FinishTask(); return TaskExecutionResult::TASK_FINISHED; @@ -1149,7 +1150,7 @@ class PhysicalShortestPathTask : public ExecutorTask { auto& next = bfs_state->iter & 1 ? bfs_state->visit2 : bfs_state->visit1; auto& barrier = bfs_state->barrier; int64_t *v = (int64_t *)state.global_csr->v; - int64_t *reverse_v = (int64_t *)state.global_csr->reverse_v; + // int64_t *reverse_v = (int64_t *)state.global_csr->reverse_v; vector &e = state.global_csr->e; auto& edge_ids = state.global_csr->edge_ids; auto& top_down_cost = bfs_state->top_down_cost; @@ -1285,140 +1286,140 @@ class PhysicalShortestPathTask : public ExecutorTask { } } - void IterativeLengthBottomUp() { - auto& bfs_state = state.global_bfs_state; - auto& seen = bfs_state->seen; - auto& visit = bfs_state->iter & 1 ? bfs_state->visit1 : bfs_state->visit2; - auto& next = bfs_state->iter & 1 ? bfs_state->visit2 : bfs_state->visit1; - auto& barrier = bfs_state->barrier; - int64_t *v = (int64_t *)state.global_csr->reverse_v; - int64_t *normal_v = (int64_t *)state.global_csr->v; - vector &e = state.global_csr->reverse_e; - auto& edge_ids = state.global_csr->reverse_edge_ids; - auto& top_down_cost = bfs_state->top_down_cost; - auto& bottom_up_cost = bfs_state->bottom_up_cost; - auto& parents_v = bfs_state->parents_v; - auto& parents_e = bfs_state->parents_e; - - // clear next before each iteration - for (auto i = left; i < right; i++) { -#ifdef SEGMENT_BITSET - for (auto j = 0; j < 8; j++) { - next[i][j].store(0, std::memory_order_relaxed); - } -#else - next[i] = 0; -#endif - } +// void IterativeLengthBottomUp() { +// auto& bfs_state = state.global_bfs_state; +// auto& seen = bfs_state->seen; +// auto& visit = bfs_state->iter & 1 ? bfs_state->visit1 : bfs_state->visit2; +// auto& next = bfs_state->iter & 1 ? bfs_state->visit2 : bfs_state->visit1; +// auto& barrier = bfs_state->barrier; +// int64_t *v = (int64_t *)state.global_csr->reverse_v; +// int64_t *normal_v = (int64_t *)state.global_csr->v; +// vector &e = state.global_csr->reverse_e; +// auto& edge_ids = state.global_csr->reverse_edge_ids; +// auto& top_down_cost = bfs_state->top_down_cost; +// auto& bottom_up_cost = bfs_state->bottom_up_cost; +// auto& parents_v = bfs_state->parents_v; +// auto& parents_e = bfs_state->parents_e; + +// // clear next before each iteration +// for (auto i = left; i < right; i++) { +// #ifdef SEGMENT_BITSET +// for (auto j = 0; j < 8; j++) { +// next[i][j].store(0, std::memory_order_relaxed); +// } +// #else +// next[i] = 0; +// #endif +// } - barrier.Wait(); - - while (true) { - auto task = bfs_state->FetchTask(worker_id); - if (task.first == task.second) { - break; - } - auto start = task.first; - auto end = task.second; - -#ifdef SEGMENT_BITSET - idx_t old_next, new_next; -#else - std::bitset old_next, new_next; -#endif - - for (auto i = start; i < end; i++) { -#ifdef SEGMENT_BITSET - for (auto j = 0; j < 8; j++) { - if (~seen[i][j] == 0) { - continue; - } - - for (auto offset = v[i]; offset < v[i + 1]; offset++) { - auto n = e[offset]; - auto edge_id = edge_ids[offset]; - do { - old_next = next[i][j].load(); - new_next = old_next | visit[n][j].load(); - } while (!next[i][j].compare_exchange_weak(old_next, new_next)); - for (auto l = 0; l < LANE_LIMIT; l++) { - parents_v[i][l] = ((parents_v[i][l] == -1) && visit[n][l / 64].load(std::memory_order_relaxed) & ((idx_t)1 << (l % 64))) - ? n : parents_v[i][l]; - parents_e[i][l] = ((parents_e[i][l] == -1) && visit[n][l / 64].load(std::memory_order_relaxed) & ((idx_t)1 << (l % 64))) - ? edge_id : parents_e[i][l]; - } - } +// barrier.Wait(); + +// while (true) { +// auto task = bfs_state->FetchTask(worker_id); +// if (task.first == task.second) { +// break; +// } +// auto start = task.first; +// auto end = task.second; + +// #ifdef SEGMENT_BITSET +// idx_t old_next, new_next; +// #else +// std::bitset old_next, new_next; +// #endif + +// for (auto i = start; i < end; i++) { +// #ifdef SEGMENT_BITSET +// for (auto j = 0; j < 8; j++) { +// if (~seen[i][j] == 0) { +// continue; +// } + +// for (auto offset = v[i]; offset < v[i + 1]; offset++) { +// auto n = e[offset]; +// auto edge_id = edge_ids[offset]; +// do { +// old_next = next[i][j].load(); +// new_next = old_next | visit[n][j].load(); +// } while (!next[i][j].compare_exchange_weak(old_next, new_next)); +// for (auto l = 0; l < LANE_LIMIT; l++) { +// parents_v[i][l] = ((parents_v[i][l] == -1) && visit[n][l / 64].load(std::memory_order_relaxed) & ((idx_t)1 << (l % 64))) +// ? n : parents_v[i][l]; +// parents_e[i][l] = ((parents_e[i][l] == -1) && visit[n][l / 64].load(std::memory_order_relaxed) & ((idx_t)1 << (l % 64))) +// ? edge_id : parents_e[i][l]; +// } +// } - if (next[i][j].load()) { - next[i][j].store(next[i][j].load() & ~seen[i][j].load(), std::memory_order_relaxed); - seen[i][j].store(seen[i][j].load() | next[i][j].load(), std::memory_order_relaxed); - - top_down_cost += normal_v[i + 1] - normal_v[i]; - } - if (~seen[i][j].load()) { - bottom_up_cost += v[i + 1] - v[i]; - } - } -#elif defined(ATOMIC_BITSET) - if (seen[i].load().all()) { - continue; - } - - for (auto offset = v[i]; offset < v[i + 1]; offset++) { - auto n = e[offset]; - auto edge_id = edge_ids[offset]; - do { - old_next = next[i].load(); - new_next = old_next | visit[n].load(); - } while (!next[i].compare_exchange_weak(old_next, new_next)); - for (auto l = 0; l < LANE_LIMIT; l++) { - parents_v[i][l] = ((parents_v[i][l] == -1) && visit[n].load()[l]) - ? i : parents_v[i][l]; - parents_e[i][l] = ((parents_e[i][l] == -1) && visit[n].load()[l]) - ? edge_id : parents_e[i][l]; - } - } - - if (next[i].load().any()) { - next[i].store(next[i].load() & ~seen[i].load()); - seen[i].store(seen[i].load() | next[i].load()); - - top_down_cost += normal_v[i + 1] - normal_v[i]; - } - if (~seen[i].load().all()) { - bottom_up_cost += v[i + 1] - v[i]; - } - -#else - if (seen[i].all()) { - continue; - } - - for (auto offset = v[i]; offset < v[i + 1]; offset++) { - auto n = e[offset]; - auto edge_id = edge_ids[offset]; - next[i] |= visit[n]; - for (auto l = 0; l < LANE_LIMIT; l++) { - parents_v[i][l] = ((parents_v[i][l] == -1) && visit[n][l]) - ? i : parents_v[i][l]; - parents_e[i][l] = ((parents_e[i][l] == -1) && visit[n][l]) - ? edge_id : parents_e[i][l]; - } - } - - if (next[i].any()) { - next[i] &= ~seen[i]; - seen[i] |= next[i]; - - top_down_cost += normal_v[i + 1] - normal_v[i]; - } - if (~seen[i].any()) { - bottom_up_cost += v[i + 1] - v[i]; - } -#endif - } - } - } +// if (next[i][j].load()) { +// next[i][j].store(next[i][j].load() & ~seen[i][j].load(), std::memory_order_relaxed); +// seen[i][j].store(seen[i][j].load() | next[i][j].load(), std::memory_order_relaxed); + +// top_down_cost += normal_v[i + 1] - normal_v[i]; +// } +// if (~seen[i][j].load()) { +// bottom_up_cost += v[i + 1] - v[i]; +// } +// } +// #elif defined(ATOMIC_BITSET) +// if (seen[i].load().all()) { +// continue; +// } + +// for (auto offset = v[i]; offset < v[i + 1]; offset++) { +// auto n = e[offset]; +// auto edge_id = edge_ids[offset]; +// do { +// old_next = next[i].load(); +// new_next = old_next | visit[n].load(); +// } while (!next[i].compare_exchange_weak(old_next, new_next)); +// for (auto l = 0; l < LANE_LIMIT; l++) { +// parents_v[i][l] = ((parents_v[i][l] == -1) && visit[n].load()[l]) +// ? i : parents_v[i][l]; +// parents_e[i][l] = ((parents_e[i][l] == -1) && visit[n].load()[l]) +// ? edge_id : parents_e[i][l]; +// } +// } + +// if (next[i].load().any()) { +// next[i].store(next[i].load() & ~seen[i].load()); +// seen[i].store(seen[i].load() | next[i].load()); + +// top_down_cost += normal_v[i + 1] - normal_v[i]; +// } +// if (~seen[i].load().all()) { +// bottom_up_cost += v[i + 1] - v[i]; +// } + +// #else +// if (seen[i].all()) { +// continue; +// } + +// for (auto offset = v[i]; offset < v[i + 1]; offset++) { +// auto n = e[offset]; +// auto edge_id = edge_ids[offset]; +// next[i] |= visit[n]; +// for (auto l = 0; l < LANE_LIMIT; l++) { +// parents_v[i][l] = ((parents_v[i][l] == -1) && visit[n][l]) +// ? i : parents_v[i][l]; +// parents_e[i][l] = ((parents_e[i][l] == -1) && visit[n][l]) +// ? edge_id : parents_e[i][l]; +// } +// } + +// if (next[i].any()) { +// next[i] &= ~seen[i]; +// seen[i] |= next[i]; + +// top_down_cost += normal_v[i + 1] - normal_v[i]; +// } +// if (~seen[i].any()) { +// bottom_up_cost += v[i + 1] - v[i]; +// } +// #endif +// } +// } +// } void ReachDetect() { auto &bfs_state = state.global_bfs_state; From 84d9cdb1ad7975c66bbbdeb81799cf5c49e73c7f Mon Sep 17 00:00:00 2001 From: Pingan Ren Date: Thu, 13 Jun 2024 23:08:48 +0200 Subject: [PATCH 123/249] fix option --- src/duckpgq_extension.cpp | 2 +- .../physical_path_finding_operator.cpp | 19 ++++++++++++------- 2 files changed, 13 insertions(+), 8 deletions(-) diff --git a/src/duckpgq_extension.cpp b/src/duckpgq_extension.cpp index 6a554f3e..b7f7e737 100644 --- a/src/duckpgq_extension.cpp +++ b/src/duckpgq_extension.cpp @@ -56,7 +56,7 @@ static void LoadInternal(DatabaseInstance &instance) { config.AddExtensionOption("experimental_path_finding_operator_task_size", "Number of vertices processed per thread at a time", LogicalType::INTEGER, Value(256)); config.AddExtensionOption("experimental_path_finding_operator_sequential", - "Sequential path finding operator", LogicalType::BOOLEAN, Value(true)); + "Sequential path finding operator", LogicalType::BOOLEAN, Value(false)); Connection con(instance); con.BeginTransaction(); diff --git a/src/operators/physical_path_finding_operator.cpp b/src/operators/physical_path_finding_operator.cpp index 81a11bfb..478a91b0 100644 --- a/src/operators/physical_path_finding_operator.cpp +++ b/src/operators/physical_path_finding_operator.cpp @@ -114,12 +114,12 @@ class PathFindingLocalState : public LocalSinkState { void PathFindingLocalState::CreateCSRVertex(DataChunk &input, GlobalCompressedSparseRow &global_csr) { if (!global_csr.initialized_v) { - const auto v_size = input.data[9].GetValue(0).GetValue(); + const auto v_size = input.data[8].GetValue(0).GetValue(); global_csr.InitializeVertex(v_size); } auto result = Vector(LogicalTypeId::BIGINT); BinaryExecutor::Execute( - input.data[4], input.data[3], result, input.size(), + input.data[6], input.data[5], result, input.size(), [&](const int64_t src, const int64_t cnt) { int64_t edge_count = 0; global_csr.v[src + 2] = cnt; @@ -851,11 +851,11 @@ class PhysicalCSREdgeCreationTask : public ExecutorTask { } } if (!global_csr->initialized_e) { - const auto e_size = input.data[8].GetValue(0).GetValue(); + const auto e_size = input.data[7].GetValue(0).GetValue(); global_csr->InitializeEdge(e_size); } TernaryExecutor::Execute( - input.data[4], input.data[7], input.data[2], result, input.size(), + input.data[6], input.data[4], input.data[2], result, input.size(), [&](int64_t src, int64_t dst, int64_t edge_id) { const auto pos = ++global_csr->v[src + 1]; global_csr->e[static_cast(pos) - 1] = dst; @@ -1837,13 +1837,18 @@ PhysicalPathFinding::Finalize(Pipeline &pipeline, Event &event, auto const sequential = client_config.set_variables.find("experimental_path_finding_operator_sequential"); if (sequential != client_config.set_variables.end() && sequential->second.GetValue()) { + if (gstate.mode == "iterativelength") { + auto bfs_event = make_shared(gstate, pipeline); + event.InsertEvent(std::move(std::dynamic_pointer_cast(bfs_event))); + } else if (gstate.mode == "shortestpath") { + auto bfs_event = make_shared(gstate, pipeline); + event.InsertEvent(std::move(std::dynamic_pointer_cast(bfs_event))); + } + } else { // Schedule the first round of BFS tasks if (all_pairs->size() > 0) { ScheduleBFSTasks(pipeline, event, gstate); } - } else { - auto bfs_event = make_shared(gstate, pipeline); - event.InsertEvent(std::move(std::dynamic_pointer_cast(bfs_event))); } } From 6bc4e889df660fc9e4c9e8054f0daff7ddfed692 Mon Sep 17 00:00:00 2001 From: Pingan Ren Date: Mon, 17 Jun 2024 17:50:34 +0200 Subject: [PATCH 124/249] no sync --- .../physical_path_finding_operator.cpp | 10 ++--- .../path_finding/parallel_path_finding.test | 37 +++++-------------- 2 files changed, 13 insertions(+), 34 deletions(-) diff --git a/src/operators/physical_path_finding_operator.cpp b/src/operators/physical_path_finding_operator.cpp index 478a91b0..3b46e1f7 100644 --- a/src/operators/physical_path_finding_operator.cpp +++ b/src/operators/physical_path_finding_operator.cpp @@ -18,7 +18,6 @@ // #define SEGMENT_BITSET // #define ATOMIC_BITSET -// #define FINE_GRAINED_LOCK #define ELEMENT_LOCK namespace duckdb { @@ -180,6 +179,7 @@ class Barrier { case 2: pthread_barrier_wait(&mBarrier); break; + case 3: default: break; } @@ -615,9 +615,7 @@ class PhysicalIterativeTask : public ExecutorTask { if (visit[i].any()) { for (auto offset = v[i]; offset < v[i + 1]; offset++) { auto n = e[offset]; -#ifdef FINE_GRAINED_LOCK - std::lock_guard lock(bfs_state->lock); -#elif defined(ELEMENT_LOCK) +#ifdef ELEMENT_LOCK std::lock_guard lock(bfs_state->locks[n]); #endif next[n] |= visit[i]; @@ -1228,9 +1226,7 @@ class PhysicalShortestPathTask : public ExecutorTask { for (auto offset = v[i]; offset < v[i + 1]; offset++) { auto n = e[offset]; auto edge_id = edge_ids[offset]; -#ifdef FINE_GRAINED_LOCK - std::lock_guard lock(bfs_state->lock); -#elif defined(ELEMENT_LOCK) +#ifdef ELEMENT_LOCK std::lock_guard lock(bfs_state->locks[n]); #endif next[n] |= visit[i]; diff --git a/test/sql/path_finding/parallel_path_finding.test b/test/sql/path_finding/parallel_path_finding.test index 921e5599..27a1e043 100644 --- a/test/sql/path_finding/parallel_path_finding.test +++ b/test/sql/path_finding/parallel_path_finding.test @@ -9,6 +9,7 @@ require duckpgq statement ok set experimental_path_finding_operator=true; +set experimental_path_finding_operator_sequential=true; statement ok CREATE TABLE pair(src BIGINT, dst BIGINT); INSERT INTO pair(src, dst) VALUES (0, 1), (1, 2), (2,0); @@ -55,32 +56,26 @@ query III SELECT *, shortestpath(src, dst, 'pair') as path FROM pair AS p WHERE p.src BETWEEN (SELECT CREATE_CSR_EDGE( - (SELECT count(a.id) as v_size FROM Student a), - (SELECT count(k.src) as e_size from know k), + (SELECT count(a.id) as v_size FROM Student a), + (SELECT count(k.src) as e_size from know k), a.rowid, c.rowid, k.rowid, - t.cnt, - t_r.cnt_r) FROM Know k + t.cnt) FROM Know k JOIN student a on a.id = k.src JOIN student c on c.id = k.dst JOIN (SELECT count(k.src) cnt, a.rowid as a_rowid FROM student a LEFT JOIN know k ON k.src = a.id GROUP BY a.rowid) t - ON t.a_rowid = a.rowid - JOIN (SELECT count(k.dst) cnt_r, c.rowid as c_rowid - FROM student c - LEFT JOIN know k ON k.dst = c.id - GROUP BY c.rowid) t_r - ON t_r.c_rowid = c.rowid) AND p.dst; + ON t.a_rowid = a.rowid) AND p.dst; ---- 0 1 NULL 1 2 [1, 3, 0, 1, 2] 2 0 [2, 4, 0] statement ok -import database 'duckdb-pgq/data/SNB1-projected|'; +import database 'duckdb-pgq/data/SNB0.003'; statement ok select setseed(0.42) @@ -114,20 +109,14 @@ WHERE p.src BETWEEN (SELECT CREATE_CSR_EDGE( a.rowid, c.rowid, k.rowid, - t.cnt, - t_r.cnt_r) FROM Person_knows_Person k + t.cnt) FROM Person_knows_Person k JOIN Person a on a.id = k.Person1Id JOIN Person c on c.id = k.Person2Id JOIN (SELECT count(k.Person1Id) cnt, a.rowid as a_rowid FROM Person a LEFT JOIN Person_knows_Person k ON k.Person1Id = a.id GROUP BY a.rowid) t - ON t.a_rowid = a.rowid - JOIN (SELECT count(k.Person2Id) cnt_r, c.rowid as c_rowid - FROM Person c - LEFT JOIN Person_knows_Person k ON k.Person2Id = c.id - GROUP BY c.rowid) t_r - ON t_r.c_rowid = c.rowid) AND p.dst; + ON t.a_rowid = a.rowid) AND p.dst; ---- 42 22 NULL 3 33 2 @@ -187,20 +176,14 @@ WHERE p.src BETWEEN (SELECT CREATE_CSR_EDGE( a.rowid, c.rowid, k.rowid, - t.cnt, - t_r.cnt_r) FROM Person_knows_Person k + t.cnt) FROM Person_knows_Person k JOIN Person a on a.id = k.Person1Id JOIN Person c on c.id = k.Person2Id JOIN (SELECT count(k.Person1Id) cnt, a.rowid as a_rowid FROM Person a LEFT JOIN Person_knows_Person k ON k.Person1Id = a.id GROUP BY a.rowid) t - ON t.a_rowid = a.rowid - JOIN (SELECT count(k.Person2Id) cnt_r, c.rowid as c_rowid - FROM Person c - LEFT JOIN Person_knows_Person k ON k.Person2Id = c.id - GROUP BY c.rowid) t_r - ON t_r.c_rowid = c.rowid) AND p.dst; + ON t.a_rowid = a.rowid) AND p.dst; ---- 10 2 NULL From 33cbcbd65afdd9187eda00a846b524b87c3690f8 Mon Sep 17 00:00:00 2001 From: Pingan Ren Date: Tue, 25 Jun 2024 23:44:44 +0200 Subject: [PATCH 125/249] add active count --- src/functions/scalar/shortest_path.cpp | 5 -- .../physical_path_finding_operator.hpp | 2 +- .../physical_path_finding_operator.cpp | 83 ++++++++----------- .../path_finding/parallel_path_finding.test | 3 +- 4 files changed, 38 insertions(+), 55 deletions(-) diff --git a/src/functions/scalar/shortest_path.cpp b/src/functions/scalar/shortest_path.cpp index 6abac2a3..6dbb36be 100644 --- a/src/functions/scalar/shortest_path.cpp +++ b/src/functions/scalar/shortest_path.cpp @@ -238,11 +238,6 @@ CreateScalarFunctionInfo DuckPGQFunctions::GetShortestPathFunction() { LogicalType::LIST(LogicalType::BIGINT), ShortestPathFunction, IterativeLengthFunctionData::IterativeLengthBind)); - set.AddFunction(ScalarFunction({LogicalType::INTEGER, LogicalType::BIGINT, - LogicalType::BIGINT, LogicalType::BIGINT}, - LogicalType::LIST(LogicalType::BIGINT), - ShortestPathFunction, - IterativeLengthFunctionData::IterativeLengthBind)); set.AddFunction(ScalarFunction({LogicalType::BIGINT, LogicalType::BIGINT, LogicalType::VARCHAR}, LogicalType::LIST(LogicalType::BIGINT), ShortestPathFunction, diff --git a/src/include/duckpgq/operators/physical_path_finding_operator.hpp b/src/include/duckpgq/operators/physical_path_finding_operator.hpp index 166741eb..60a7b6b8 100644 --- a/src/include/duckpgq/operators/physical_path_finding_operator.hpp +++ b/src/include/duckpgq/operators/physical_path_finding_operator.hpp @@ -146,7 +146,7 @@ class PhysicalPathFinding : public PhysicalComparisonJoin { public: vector> expressions; string mode; // "iterativelength" or "shortestpath" - + std::chrono::time_point start_time; public: // CachingOperator Interface diff --git a/src/operators/physical_path_finding_operator.cpp b/src/operators/physical_path_finding_operator.cpp index 3b46e1f7..ff1847b1 100644 --- a/src/operators/physical_path_finding_operator.cpp +++ b/src/operators/physical_path_finding_operator.cpp @@ -26,6 +26,7 @@ PhysicalPathFinding::PhysicalPathFinding(LogicalExtensionOperator &op, unique_ptr left, unique_ptr right) : PhysicalComparisonJoin(op, TYPE, {}, JoinType::INNER, 0) { + start_time = std::chrono::high_resolution_clock::now(); children.push_back(std::move(left)); children.push_back(std::move(right)); expressions = std::move(op.expressions); @@ -200,16 +201,13 @@ class GlobalBFSState { PhysicalPathFinding::GlobalCompressedSparseRow; public: GlobalBFSState(shared_ptr csr_, shared_ptr pairs_, int64_t v_size_, - idx_t num_threads_, idx_t barrier_type_, ClientContext &context_) + idx_t num_threads_, idx_t barrier_type_, idx_t mode_, ClientContext &context_) : csr(csr_), pairs(pairs_), iter(1), v_size(v_size_), change(false), started_searches(0), total_len(0), context(context_), seen(v_size_), visit1(v_size_), visit2(v_size_), parents_v(v_size_, std::vector(LANE_LIMIT, -1)), parents_e(v_size_, std::vector(LANE_LIMIT, -1)), top_down_cost(0), bottom_up_cost(0), is_top_down(true), num_threads(num_threads_), task_queues(num_threads_), - task_queues_reverse(num_threads_), barrier(barrier_type_, num_threads_), locks(v_size_) { + task_queues_reverse(num_threads_), barrier(barrier_type_, num_threads_), locks(v_size_), mode(mode_) { result.Initialize(context, {LogicalType::BIGINT, LogicalType::LIST(LogicalType::BIGINT)}, pairs_->size()); - for (auto i = 0; i < LANE_LIMIT; i++) { - lane_to_num[i] = -1; - } auto &src_data = pairs->data[0]; auto &dst_data = pairs->data[1]; src_data.ToUnifiedFormat(pairs->size(), vdata_src); @@ -217,45 +215,29 @@ class GlobalBFSState { src = FlatVector::GetData(src_data); dst = FlatVector::GetData(dst_data); - for (auto i = 0; i < v_size; i++) { -#ifdef SEGMENT_BITSET - for (auto j = 0; j < 8; j++) { - seen[i][j] = ~0; - visit1[i][j] = 0; - } -#else - seen[i] = ~0; - visit1[i] = 0; -#endif - for (auto j = 0; j < LANE_LIMIT; j++) { - parents_v[i][j] = -1; - parents_e[i][j] = -1; - } - } - CreateTasks(); } void Clear() { iter = 1; + active = 0; change = false; - for (auto i = 0; i < LANE_LIMIT; i++) { - lane_to_num[i] = -1; - } // empty visit vectors for (auto i = 0; i < v_size; i++) { #ifdef SEGMENT_BITSET for (auto j = 0; j < 8; j++) { - seen[i][j] = ~0; + seen[i][j] = 0; visit1[i][j] = 0; } #else - seen[i] = ~0; + seen[i] = 0; visit1[i] = 0; #endif - for (auto j = 0; j < LANE_LIMIT; j++) { - parents_v[i][j] = -1; - parents_e[i][j] = -1; + if (mode == 1) { + for (auto j = 0; j < LANE_LIMIT; j++) { + parents_v[i][j] = -1; + parents_e[i][j] = -1; + } } } } @@ -263,7 +245,7 @@ class GlobalBFSState { void CreateTasks() { // workerTasks[workerId] = [task1, task2, ...] - auto queues = {&task_queues, &task_queues_reverse}; + auto queues = {&task_queues}; is_top_down = true; for (auto& queue : queues) { vector>> worker_tasks(num_threads); @@ -360,6 +342,7 @@ class GlobalBFSState { UnifiedVectorFormat vdata_src; UnifiedVectorFormat vdata_dst; int64_t lane_to_num[LANE_LIMIT]; + idx_t active = 0; DataChunk result; // 0 for length, 1 for path ClientContext& context; #ifdef SEGMENT_BITSET @@ -399,6 +382,8 @@ class GlobalBFSState { // lock for next mutable mutex lock; mutable vector locks; + + idx_t mode; }; class PathFindingGlobalState : public GlobalSinkState { @@ -797,9 +782,13 @@ class PhysicalIterativeTask : public ExecutorTask { result_data[search_num] = bfs_state->iter; /* found at iter => iter = path length */ bfs_state->lane_to_num[lane] = -1; // mark inactive + bfs_state->active--; } } } + if (bfs_state->active == 0) { + bfs_state->change = false; + } // into the next iteration bfs_state->iter++; } @@ -1422,7 +1411,6 @@ class PhysicalShortestPathTask : public ExecutorTask { auto &change = bfs_state->change; auto &top_down_cost = bfs_state->top_down_cost; auto &bottom_up_cost = bfs_state->bottom_up_cost; - int64_t finished_searches = 0; // detect lanes that finished for (int64_t lane = 0; lane < LANE_LIMIT; lane++) { int64_t search_num = bfs_state->lane_to_num[lane]; @@ -1437,15 +1425,13 @@ class PhysicalShortestPathTask : public ExecutorTask { #else if (bfs_state->seen[bfs_state->dst[dst_pos]][lane]) { #endif - finished_searches++; + bfs_state->active--; } - } else { - finished_searches++; } } - if (finished_searches == LANE_LIMIT) { + if (bfs_state->active == 0) { change = false; - } + } // else { // if (top_down_cost > 0 || bottom_up_cost > 0) { // change = true; @@ -1826,7 +1812,8 @@ PhysicalPathFinding::Finalize(Pipeline &pipeline, Event &event, auto& client_config = ClientConfig::GetConfig(context); auto const barrier_type_idx = client_config.set_variables.find("experimental_path_finding_operator_barrier"); auto barrier_type = barrier_type_idx != client_config.set_variables.end() ? barrier_type_idx->second.GetValue() : 0; - gstate.global_bfs_state = make_uniq(csr, all_pairs, csr->v_size - 2, num_threads, barrier_type, context); + idx_t mode = this->mode == "iterativelength" ? 0 : 1; + gstate.global_bfs_state = make_uniq(csr, all_pairs, csr->v_size - 2, num_threads, barrier_type, mode, context); auto const task_size = client_config.set_variables.find("experimental_path_finding_operator_task_size"); gstate.global_bfs_state->split_size = task_size != client_config.set_variables.end() ? task_size->second.GetValue() : 256; @@ -1883,27 +1870,18 @@ void PhysicalPathFinding::ScheduleBFSTasks(Pipeline &pipeline, Event &event, } else { #ifdef SEGMENT_BITSET bfs_state->visit1[bfs_state->src[src_pos]][lane / 64] |= ((idx_t)1 << (lane % 64)); - for (int i = 0; i < bfs_state->v_size; i++) { - bfs_state->seen[i][lane / 64] &= ~((idx_t)1 << (lane % 64)); - } #elif defined(ATOMIC_BITSET) auto new_visit = bfs_state->visit1[bfs_state->src[src_pos]].load(); new_visit[lane] = true; bfs_state->visit1[bfs_state->src[src_pos]].store(new_visit); - for (int i = 0; i < bfs_state->v_size; i++) { - auto new_seen = bfs_state->seen[i].load(); - new_seen[lane] = false; - bfs_state->seen[i].store(new_seen); - } #else bfs_state->visit1[bfs_state->src[src_pos]][lane] = true; - for (int i = 0; i < bfs_state->v_size; i++) { - bfs_state->seen[i][lane] = false; - } #endif bfs_state->lane_to_num[lane] = search_num; // active lane + bfs_state->active++; break; } + } } if (gstate.mode == "iterativelength") { @@ -1988,6 +1966,9 @@ PhysicalPathFinding::GetData(ExecutionContext &context, DataChunk &result, } pf_bfs_state->result.SetCardinality(*pf_bfs_state->pairs); + // string message = "Algorithm time elapsed: " + to_string(pf_bfs_state->time_elapsed.count()) + " microseconds"; + // Printer::Print(message); + result.Move(*pf_bfs_state->pairs); auto result_path = make_uniq(); //! Split off the path from the path length, and then fuse into the result @@ -1999,6 +1980,12 @@ PhysicalPathFinding::GetData(ExecutionContext &context, DataChunk &result, } else { throw NotImplementedException("Unrecognized mode for Path Finding"); } + + auto end_time = std::chrono::high_resolution_clock::now(); + auto duration = std::chrono::duration_cast(end_time - start_time); + // string message2 = "Total time elapsed: " + to_string(duration.count()) + " microseconds"; + // Printer::Print(message2); + // result.Print(); return result.size() == 0 ? SourceResultType::FINISHED diff --git a/test/sql/path_finding/parallel_path_finding.test b/test/sql/path_finding/parallel_path_finding.test index 27a1e043..1237e438 100644 --- a/test/sql/path_finding/parallel_path_finding.test +++ b/test/sql/path_finding/parallel_path_finding.test @@ -9,7 +9,8 @@ require duckpgq statement ok set experimental_path_finding_operator=true; -set experimental_path_finding_operator_sequential=true; +set experimental_path_finding_operator_sequential=false; +set experimental_path_finding_operator_barrier=2; statement ok CREATE TABLE pair(src BIGINT, dst BIGINT); INSERT INTO pair(src, dst) VALUES (0, 1), (1, 2), (2,0); From 9bcbf37440a79e738fd7f4e284b7fe560f6be2c1 Mon Sep 17 00:00:00 2001 From: Pingan Ren Date: Wed, 26 Jun 2024 16:05:37 +0200 Subject: [PATCH 126/249] time record --- .../physical_path_finding_operator.hpp | 1 + .../physical_path_finding_operator.cpp | 68 ++++++++++++++++--- 2 files changed, 61 insertions(+), 8 deletions(-) diff --git a/src/include/duckpgq/operators/physical_path_finding_operator.hpp b/src/include/duckpgq/operators/physical_path_finding_operator.hpp index 60a7b6b8..0b172798 100644 --- a/src/include/duckpgq/operators/physical_path_finding_operator.hpp +++ b/src/include/duckpgq/operators/physical_path_finding_operator.hpp @@ -148,6 +148,7 @@ class PhysicalPathFinding : public PhysicalComparisonJoin { string mode; // "iterativelength" or "shortestpath" std::chrono::time_point start_time; + public: // CachingOperator Interface OperatorResultType ExecuteInternal(ExecutionContext &context, diff --git a/src/operators/physical_path_finding_operator.cpp b/src/operators/physical_path_finding_operator.cpp index ff1847b1..38959a07 100644 --- a/src/operators/physical_path_finding_operator.cpp +++ b/src/operators/physical_path_finding_operator.cpp @@ -26,7 +26,6 @@ PhysicalPathFinding::PhysicalPathFinding(LogicalExtensionOperator &op, unique_ptr left, unique_ptr right) : PhysicalComparisonJoin(op, TYPE, {}, JoinType::INNER, 0) { - start_time = std::chrono::high_resolution_clock::now(); children.push_back(std::move(left)); children.push_back(std::move(right)); expressions = std::move(op.expressions); @@ -378,6 +377,7 @@ class GlobalBFSState { Barrier barrier; std::chrono::microseconds time_elapsed = std::chrono::microseconds(0); + std::chrono::microseconds time_elapsed_task_init = std::chrono::microseconds(0); // lock for next mutable mutex lock; @@ -406,8 +406,19 @@ class PathFindingGlobalState : public GlobalSinkState { global_bfs_state(std::move(prev.global_bfs_state)), child(prev.child + 1), mode(prev.mode) { } - void Sink(DataChunk &input, PathFindingLocalState &lstate) const { + void Sink(DataChunk &input, PathFindingLocalState &lstate) { + if (operator_start_time == std::chrono::time_point()) { + operator_start_time = std::chrono::high_resolution_clock::now(); + } + auto start_time = std::chrono::high_resolution_clock::now(); lstate.Sink(input, *global_csr, child); + auto end_time = std::chrono::high_resolution_clock::now(); + auto duration = std::chrono::duration_cast(end_time - start_time); + if (child == 1) { + time_elapsed_pairs += duration; + } else { + time_elapsed_csr += duration; + } } void CSRCreateEdge() { @@ -440,6 +451,11 @@ class PathFindingGlobalState : public GlobalSinkState { ColumnDataScanState scan_state; ColumnDataAppendState append_state; + std::chrono::microseconds time_elapsed_csr = std::chrono::microseconds(0); + std::chrono::microseconds time_elapsed_pairs = std::chrono::microseconds(0); + std::chrono::time_point operator_start_time; + std::chrono::microseconds time_elapsed_operator = std::chrono::microseconds(0); + shared_ptr global_csr; // state for BFS unique_ptr global_bfs_state; @@ -875,7 +891,11 @@ class CSREdgeCreationEvent : public BasePipelineEvent { PathFindingGlobalState &gstate; +private: + std::chrono::high_resolution_clock::time_point start_time; + void Schedule() override { + start_time = std::chrono::high_resolution_clock::now(); auto &context = pipeline->GetClientContext(); auto &ts = TaskScheduler::GetScheduler(context); idx_t num_threads = ts.NumberOfThreads(); @@ -895,6 +915,10 @@ class CSREdgeCreationEvent : public BasePipelineEvent { auto &gstate = this->gstate; auto &global_csr = gstate.global_csr; global_csr->is_ready = true; + auto end_time = std::chrono::high_resolution_clock::now(); + auto duration = std::chrono::duration_cast(end_time - start_time); + string msg = "CSR edge creation time: " + std::to_string(duration.count()) + " microseconds"; + Printer::Print(msg); // debug print // global_csr->Print(); } @@ -913,6 +937,8 @@ class ParallelIterativeEvent : public BasePipelineEvent { public: void Schedule() override { + start_time = std::chrono::high_resolution_clock::now(); + auto &bfs_state = gstate.global_bfs_state; auto &context = pipeline->GetClientContext(); @@ -921,7 +947,6 @@ class ParallelIterativeEvent : public BasePipelineEvent { bfs_tasks.push_back(make_uniq(shared_from_this(), context, gstate, tnum)); } SetTasks(std::move(bfs_tasks)); - start_time = std::chrono::high_resolution_clock::now(); } void FinishEvent() override { @@ -1791,10 +1816,16 @@ PhysicalPathFinding::Finalize(Pipeline &pipeline, Event &event, auto &gstate = input.global_state.Cast(); auto &csr = gstate.global_csr; auto &global_tasks = gstate.global_tasks; + if (gstate.child == 0) { auto csr_event = make_shared(gstate, pipeline); event.InsertEvent(std::move(std::dynamic_pointer_cast(csr_event))); } else if (gstate.child == 1 && global_tasks->Count() > 0) { + string msg_csr = "CSR vertex creation time: " + std::to_string(gstate.time_elapsed_csr.count()) + " microseconds"; + Printer::Print(msg_csr); + string msg_pairs = "Pair creation time: " + std::to_string(gstate.time_elapsed_pairs.count()) + " microseconds"; + Printer::Print(msg_pairs); + auto start_time = std::chrono::high_resolution_clock::now(); auto all_pairs = make_shared(); DataChunk pairs; global_tasks->InitializeScanChunk(*all_pairs); @@ -1804,9 +1835,15 @@ PhysicalPathFinding::Finalize(Pipeline &pipeline, Event &event, while (global_tasks->Scan(scan_state, pairs)) { all_pairs->Append(pairs, true); } + auto end_time = std::chrono::high_resolution_clock::now(); + auto duration = std::chrono::duration_cast(end_time - start_time); + string msg = "Time to collect all pairs: " + std::to_string(duration.count()) + " microseconds"; + Printer::Print(msg); // debug print // all_pairs->Print(); + + start_time = std::chrono::high_resolution_clock::now(); auto &ts = TaskScheduler::GetScheduler(context); idx_t num_threads = ts.NumberOfThreads(); auto& client_config = ClientConfig::GetConfig(context); @@ -1819,6 +1856,10 @@ PhysicalPathFinding::Finalize(Pipeline &pipeline, Event &event, gstate.global_bfs_state->split_size = task_size != client_config.set_variables.end() ? task_size->second.GetValue() : 256; auto const sequential = client_config.set_variables.find("experimental_path_finding_operator_sequential"); + end_time = std::chrono::high_resolution_clock::now(); + duration = std::chrono::duration_cast(end_time - start_time); + msg = "Time to initialize BFS state: " + std::to_string(duration.count()) + " microseconds"; + Printer::Print(msg); if (sequential != client_config.set_variables.end() && sequential->second.GetValue()) { if (gstate.mode == "iterativelength") { auto bfs_event = make_shared(gstate, pipeline); @@ -1843,6 +1884,7 @@ PhysicalPathFinding::Finalize(Pipeline &pipeline, Event &event, void PhysicalPathFinding::ScheduleBFSTasks(Pipeline &pipeline, Event &event, GlobalSinkState &state) { + auto start_time = std::chrono::high_resolution_clock::now(); auto &gstate = state.Cast(); auto &bfs_state = gstate.global_bfs_state; @@ -1893,6 +1935,9 @@ void PhysicalPathFinding::ScheduleBFSTasks(Pipeline &pipeline, Event &event, } } + auto end_time = std::chrono::high_resolution_clock::now(); + auto duration = std::chrono::duration_cast(end_time - start_time); + bfs_state->time_elapsed_task_init += duration; } //===--------------------------------------------------------------------===// @@ -1959,6 +2004,7 @@ PhysicalPathFinding::GetLocalSourceState(ExecutionContext &context, SourceResultType PhysicalPathFinding::GetData(ExecutionContext &context, DataChunk &result, OperatorSourceInput &input) const { + auto results_start_time = std::chrono::high_resolution_clock::now(); auto &pf_sink = sink_state->Cast(); auto &pf_bfs_state = pf_sink.global_bfs_state; if (pf_bfs_state->pairs->size() == 0) { @@ -1966,8 +2012,11 @@ PhysicalPathFinding::GetData(ExecutionContext &context, DataChunk &result, } pf_bfs_state->result.SetCardinality(*pf_bfs_state->pairs); - // string message = "Algorithm time elapsed: " + to_string(pf_bfs_state->time_elapsed.count()) + " microseconds"; - // Printer::Print(message); + string msg = "Task init time elapsed: " + to_string(pf_bfs_state->time_elapsed_task_init.count()) + " microseconds"; + Printer::Print(msg); + + string message = "Algorithm time elapsed: " + to_string(pf_bfs_state->time_elapsed.count()) + " microseconds"; + Printer::Print(message); result.Move(*pf_bfs_state->pairs); auto result_path = make_uniq(); @@ -1982,9 +2031,12 @@ PhysicalPathFinding::GetData(ExecutionContext &context, DataChunk &result, } auto end_time = std::chrono::high_resolution_clock::now(); - auto duration = std::chrono::duration_cast(end_time - start_time); - // string message2 = "Total time elapsed: " + to_string(duration.count()) + " microseconds"; - // Printer::Print(message2); + auto results_duration = std::chrono::duration_cast(end_time - results_start_time); + string message_results = "Results time elapsed: " + to_string(results_duration.count()) + " microseconds"; + Printer::Print(message_results); + auto duration = std::chrono::duration_cast(end_time - pf_sink.operator_start_time); + string message2 = "Total time elapsed: " + to_string(duration.count()) + " microseconds\n"; + Printer::Print(message2); // result.Print(); return result.size() == 0 From 0ebb0d589c16656a7115998c77dfe2f266eebfbf Mon Sep 17 00:00:00 2001 From: Pingan Ren Date: Wed, 26 Jun 2024 18:04:53 +0200 Subject: [PATCH 127/249] fix time record bug --- .../physical_path_finding_operator.cpp | 32 +++++++++++++------ 1 file changed, 22 insertions(+), 10 deletions(-) diff --git a/src/operators/physical_path_finding_operator.cpp b/src/operators/physical_path_finding_operator.cpp index 38959a07..ffc2c2a6 100644 --- a/src/operators/physical_path_finding_operator.cpp +++ b/src/operators/physical_path_finding_operator.cpp @@ -203,9 +203,10 @@ class GlobalBFSState { idx_t num_threads_, idx_t barrier_type_, idx_t mode_, ClientContext &context_) : csr(csr_), pairs(pairs_), iter(1), v_size(v_size_), change(false), started_searches(0), total_len(0), context(context_), seen(v_size_), visit1(v_size_), visit2(v_size_), - parents_v(v_size_, std::vector(LANE_LIMIT, -1)), parents_e(v_size_, std::vector(LANE_LIMIT, -1)), + parents_v(v_size_, std::vector(LANE_LIMIT)), parents_e(v_size_, std::vector(LANE_LIMIT)), top_down_cost(0), bottom_up_cost(0), is_top_down(true), num_threads(num_threads_), task_queues(num_threads_), task_queues_reverse(num_threads_), barrier(barrier_type_, num_threads_), locks(v_size_), mode(mode_) { + auto start = std::chrono::high_resolution_clock::now(); result.Initialize(context, {LogicalType::BIGINT, LogicalType::LIST(LogicalType::BIGINT)}, pairs_->size()); auto &src_data = pairs->data[0]; auto &dst_data = pairs->data[1]; @@ -215,6 +216,11 @@ class GlobalBFSState { dst = FlatVector::GetData(dst_data); CreateTasks(); + + auto end = std::chrono::high_resolution_clock::now(); + auto duration = std::chrono::duration_cast(end - start); + string message = "BFS state initialization time: " + std::to_string(duration.count()) + " microseconds"; + Printer::Print(message); } void Clear() { @@ -410,14 +416,16 @@ class PathFindingGlobalState : public GlobalSinkState { if (operator_start_time == std::chrono::time_point()) { operator_start_time = std::chrono::high_resolution_clock::now(); } - auto start_time = std::chrono::high_resolution_clock::now(); + if (child == 1 && pairs_start_time == std::chrono::time_point()) { + pairs_start_time = std::chrono::high_resolution_clock::now(); + } else if (child != 1 && csr_start_time == std::chrono::time_point()) { + csr_start_time = std::chrono::high_resolution_clock::now(); + } lstate.Sink(input, *global_csr, child); - auto end_time = std::chrono::high_resolution_clock::now(); - auto duration = std::chrono::duration_cast(end_time - start_time); if (child == 1) { - time_elapsed_pairs += duration; + pairs_end_time = std::chrono::high_resolution_clock::now(); } else { - time_elapsed_csr += duration; + csr_end_time = std::chrono::high_resolution_clock::now(); } } @@ -451,10 +459,12 @@ class PathFindingGlobalState : public GlobalSinkState { ColumnDataScanState scan_state; ColumnDataAppendState append_state; - std::chrono::microseconds time_elapsed_csr = std::chrono::microseconds(0); - std::chrono::microseconds time_elapsed_pairs = std::chrono::microseconds(0); std::chrono::time_point operator_start_time; std::chrono::microseconds time_elapsed_operator = std::chrono::microseconds(0); + std::chrono::time_point pairs_start_time; + std::chrono::time_point pairs_end_time; + std::chrono::time_point csr_start_time; + std::chrono::time_point csr_end_time; shared_ptr global_csr; // state for BFS @@ -1821,9 +1831,11 @@ PhysicalPathFinding::Finalize(Pipeline &pipeline, Event &event, auto csr_event = make_shared(gstate, pipeline); event.InsertEvent(std::move(std::dynamic_pointer_cast(csr_event))); } else if (gstate.child == 1 && global_tasks->Count() > 0) { - string msg_csr = "CSR vertex creation time: " + std::to_string(gstate.time_elapsed_csr.count()) + " microseconds"; + auto csr_duration = std::chrono::duration_cast(gstate.csr_end_time - gstate.csr_start_time); + string msg_csr = "CSR vertex creation time: " + std::to_string(csr_duration.count()) + " microseconds"; Printer::Print(msg_csr); - string msg_pairs = "Pair creation time: " + std::to_string(gstate.time_elapsed_pairs.count()) + " microseconds"; + auto pairs_duration = std::chrono::duration_cast(gstate.pairs_end_time - gstate.pairs_start_time); + string msg_pairs = "Pair creation time: " + std::to_string(pairs_duration.count()) + " microseconds"; Printer::Print(msg_pairs); auto start_time = std::chrono::high_resolution_clock::now(); auto all_pairs = make_shared(); From 67b30c2fc7ebf7a6bcfd2b971fd393f3a0789911 Mon Sep 17 00:00:00 2001 From: Pingan Ren Date: Thu, 27 Jun 2024 12:04:39 +0200 Subject: [PATCH 128/249] reduce init time --- src/operators/physical_path_finding_operator.cpp | 12 +++++------- 1 file changed, 5 insertions(+), 7 deletions(-) diff --git a/src/operators/physical_path_finding_operator.cpp b/src/operators/physical_path_finding_operator.cpp index ffc2c2a6..fc5b1b42 100644 --- a/src/operators/physical_path_finding_operator.cpp +++ b/src/operators/physical_path_finding_operator.cpp @@ -203,10 +203,8 @@ class GlobalBFSState { idx_t num_threads_, idx_t barrier_type_, idx_t mode_, ClientContext &context_) : csr(csr_), pairs(pairs_), iter(1), v_size(v_size_), change(false), started_searches(0), total_len(0), context(context_), seen(v_size_), visit1(v_size_), visit2(v_size_), - parents_v(v_size_, std::vector(LANE_LIMIT)), parents_e(v_size_, std::vector(LANE_LIMIT)), top_down_cost(0), bottom_up_cost(0), is_top_down(true), num_threads(num_threads_), task_queues(num_threads_), task_queues_reverse(num_threads_), barrier(barrier_type_, num_threads_), locks(v_size_), mode(mode_) { - auto start = std::chrono::high_resolution_clock::now(); result.Initialize(context, {LogicalType::BIGINT, LogicalType::LIST(LogicalType::BIGINT)}, pairs_->size()); auto &src_data = pairs->data[0]; auto &dst_data = pairs->data[1]; @@ -215,12 +213,12 @@ class GlobalBFSState { src = FlatVector::GetData(src_data); dst = FlatVector::GetData(dst_data); - CreateTasks(); + if (mode == 1) { + parents_v.resize(v_size_, std::vector(LANE_LIMIT)); + parents_e.resize(v_size_, std::vector(LANE_LIMIT)); + } - auto end = std::chrono::high_resolution_clock::now(); - auto duration = std::chrono::duration_cast(end - start); - string message = "BFS state initialization time: " + std::to_string(duration.count()) + " microseconds"; - Printer::Print(message); + CreateTasks(); } void Clear() { From 48d5080337f30dfabf6342172124d58554ecee2a Mon Sep 17 00:00:00 2001 From: Pingan Ren Date: Thu, 27 Jun 2024 16:31:46 +0200 Subject: [PATCH 129/249] add bottom-up again --- src/duckpgq_extension.cpp | 4 +- .../physical_path_finding_operator.hpp | 61 +- .../physical_path_finding_operator.cpp | 830 ++++-------------- .../path_finding/parallel_path_finding.test | 35 +- 4 files changed, 236 insertions(+), 694 deletions(-) diff --git a/src/duckpgq_extension.cpp b/src/duckpgq_extension.cpp index b7f7e737..5c06d178 100644 --- a/src/duckpgq_extension.cpp +++ b/src/duckpgq_extension.cpp @@ -51,8 +51,8 @@ static void LoadInternal(DatabaseInstance &instance) { config.AddExtensionOption("experimental_path_finding_operator", "Enables the experimental path finding operator to be triggered", LogicalType::BOOLEAN, Value(false)); - config.AddExtensionOption("experimental_path_finding_operator_barrier", - "Barrier variants selection", LogicalType::INTEGER, Value(0)); + config.AddExtensionOption("experimental_path_finding_operator_alpha", + "The ratio to switch top-down and bottom up", LogicalType::DOUBLE, Value(1.0)); config.AddExtensionOption("experimental_path_finding_operator_task_size", "Number of vertices processed per thread at a time", LogicalType::INTEGER, Value(256)); config.AddExtensionOption("experimental_path_finding_operator_sequential", diff --git a/src/include/duckpgq/operators/physical_path_finding_operator.hpp b/src/include/duckpgq/operators/physical_path_finding_operator.hpp index 0b172798..1ddc6007 100644 --- a/src/include/duckpgq/operators/physical_path_finding_operator.hpp +++ b/src/include/duckpgq/operators/physical_path_finding_operator.hpp @@ -27,17 +27,17 @@ class PhysicalPathFinding : public PhysicalComparisonJoin { if (v) { delete[] v; } - // if (reverse_v) { - // delete[] reverse_v; - // } + if (reverse_v) { + delete[] reverse_v; + } } atomic *v; - // atomic *reverse_v; + atomic *reverse_v; vector e; - // vector reverse_e; + vector reverse_e; vector edge_ids; - // vector reverse_edge_ids; + vector reverse_edge_ids; vector w; vector w_double; bool initialized_v = false; @@ -76,30 +76,30 @@ class PhysicalPathFinding : public PhysicalComparisonJoin { } else { result += "not initialized"; } - // result += "\nReverse V: "; - // if (initialized_v) { - // for (idx_t i = 0; i < v_size; ++i) { - // result += std::to_string(reverse_v[i]) + ' '; - // } - // } else { - // result += "not initialized"; - // } - // result += "\nReverse E: "; - // if (initialized_e) { - // for (auto i : reverse_e) { - // result += std::to_string(i) + " "; - // } - // } else { - // result += "not initialized"; - // } - // result += "\nReverse Edge IDs: "; - // if (initialized_e) { - // for (auto i : reverse_edge_ids) { - // result += std::to_string(i) + " "; - // } - // } else { - // result += "not initialized"; - // } + result += "\nReverse V: "; + if (initialized_v) { + for (idx_t i = 0; i < v_size; ++i) { + result += std::to_string(reverse_v[i]) + ' '; + } + } else { + result += "not initialized"; + } + result += "\nReverse E: "; + if (initialized_e) { + for (auto i : reverse_e) { + result += std::to_string(i) + " "; + } + } else { + result += "not initialized"; + } + result += "\nReverse Edge IDs: "; + if (initialized_e) { + for (auto i : reverse_edge_ids) { + result += std::to_string(i) + " "; + } + } else { + result += "not initialized"; + } result += "\nW: "; if (initialized_w) { for (auto i : w) { @@ -146,7 +146,6 @@ class PhysicalPathFinding : public PhysicalComparisonJoin { public: vector> expressions; string mode; // "iterativelength" or "shortestpath" - std::chrono::time_point start_time; public: diff --git a/src/operators/physical_path_finding_operator.cpp b/src/operators/physical_path_finding_operator.cpp index fc5b1b42..7856ba7d 100644 --- a/src/operators/physical_path_finding_operator.cpp +++ b/src/operators/physical_path_finding_operator.cpp @@ -16,10 +16,6 @@ #include #include -// #define SEGMENT_BITSET -// #define ATOMIC_BITSET -#define ELEMENT_LOCK - namespace duckdb { PhysicalPathFinding::PhysicalPathFinding(LogicalExtensionOperator &op, @@ -43,7 +39,7 @@ void PhysicalPathFinding::GlobalCompressedSparseRow::InitializeVertex( v_size = v_size_ + 2; try { v = new std::atomic[v_size]; - // reverse_v = new std::atomic[v_size]; + reverse_v = new std::atomic[v_size]; } catch (std::bad_alloc const &) { throw InternalException( "Unable to initialize vector of size for csr vertex table " @@ -51,7 +47,7 @@ void PhysicalPathFinding::GlobalCompressedSparseRow::InitializeVertex( } for (idx_t i = 0; i < v_size; ++i) { v[i].store(0); - // reverse_v[i].store(0); + reverse_v[i].store(0); } initialized_v = true; } @@ -63,16 +59,16 @@ void PhysicalPathFinding::GlobalCompressedSparseRow::InitializeEdge( } try { e.resize(e_size, 0); - // reverse_e.resize(e_size, 0); + reverse_e.resize(e_size, 0); edge_ids.resize(e_size, 0); - // reverse_edge_ids.resize(e_size, 0); + reverse_edge_ids.resize(e_size, 0); } catch (std::bad_alloc const &) { throw InternalException("Unable to initialize vector of size for csr " "edge table representation"); } for (idx_t i = 1; i < v_size; i++) { v[i] += v[i - 1]; - // reverse_v[i] += reverse_v[i - 1]; + reverse_v[i] += reverse_v[i - 1]; } initialized_e = true; } @@ -113,81 +109,49 @@ class PathFindingLocalState : public LocalSinkState { void PathFindingLocalState::CreateCSRVertex(DataChunk &input, GlobalCompressedSparseRow &global_csr) { if (!global_csr.initialized_v) { - const auto v_size = input.data[8].GetValue(0).GetValue(); + const auto v_size = input.data[9].GetValue(0).GetValue(); global_csr.InitializeVertex(v_size); } auto result = Vector(LogicalTypeId::BIGINT); BinaryExecutor::Execute( - input.data[6], input.data[5], result, input.size(), + input.data[4], input.data[3], result, input.size(), [&](const int64_t src, const int64_t cnt) { int64_t edge_count = 0; global_csr.v[src + 2] = cnt; edge_count = edge_count + cnt; return edge_count; }); - // BinaryExecutor::Execute( - // input.data[7], input.data[6], result, input.size(), - // [&](const int64_t dst, const int64_t cnt) { - // int64_t edge_count = 0; - // global_csr.reverse_v[dst + 2] = cnt; - // edge_count = edge_count + cnt; - // return edge_count; - // }); + BinaryExecutor::Execute( + input.data[7], input.data[6], result, input.size(), + [&](const int64_t dst, const int64_t cnt) { + int64_t edge_count = 0; + global_csr.reverse_v[dst + 2] = cnt; + edge_count = edge_count + cnt; + return edge_count; + }); } class Barrier { public: - explicit Barrier(std::size_t iType, std::size_t iCount) : - mType(iType), + explicit Barrier(std::size_t iCount) : mThreshold(iCount), mCount(iCount), mGeneration(0) { - if (iType == 2) { - pthread_barrier_init(&mBarrier, nullptr, iCount); - } } void Wait() { - switch (mType) - { - case 0: - { - std::unique_lock lLock{mMutex}; - auto lGen = mGeneration.load(); - if (!--mCount) { - mGeneration++; - mCount = mThreshold; - mCond.notify_all(); - } else { - mCond.wait(lLock, [this, lGen] { return lGen != mGeneration; }); - } - } - break; - case 1: - { - auto lGen = mGeneration.load(); - if (!--mCount) { - mCount = mThreshold; - ++mGeneration; - } else { - while (lGen == mGeneration.load()) { - std::this_thread::yield(); - } - } - } - break; - case 2: - pthread_barrier_wait(&mBarrier); - break; - case 3: - default: - break; + std::unique_lock lLock{mMutex}; + auto lGen = mGeneration.load(); + if (!--mCount) { + mGeneration++; + mCount = mThreshold; + mCond.notify_all(); + } else { + mCond.wait(lLock, [this, lGen] { return lGen != mGeneration; }); } } private: - std::size_t mType; - pthread_barrier_t mBarrier; std::mutex mMutex; std::condition_variable mCond; std::size_t mThreshold; @@ -200,11 +164,11 @@ class GlobalBFSState { PhysicalPathFinding::GlobalCompressedSparseRow; public: GlobalBFSState(shared_ptr csr_, shared_ptr pairs_, int64_t v_size_, - idx_t num_threads_, idx_t barrier_type_, idx_t mode_, ClientContext &context_) + idx_t num_threads_, idx_t mode_, ClientContext &context_) : csr(csr_), pairs(pairs_), iter(1), v_size(v_size_), change(false), started_searches(0), total_len(0), context(context_), seen(v_size_), visit1(v_size_), visit2(v_size_), top_down_cost(0), bottom_up_cost(0), is_top_down(true), num_threads(num_threads_), task_queues(num_threads_), - task_queues_reverse(num_threads_), barrier(barrier_type_, num_threads_), locks(v_size_), mode(mode_) { + task_queues_reverse(num_threads_), barrier(num_threads_), locks(v_size_), mode(mode_) { result.Initialize(context, {LogicalType::BIGINT, LogicalType::LIST(LogicalType::BIGINT)}, pairs_->size()); auto &src_data = pairs->data[0]; auto &dst_data = pairs->data[1]; @@ -227,15 +191,7 @@ class GlobalBFSState { change = false; // empty visit vectors for (auto i = 0; i < v_size; i++) { -#ifdef SEGMENT_BITSET - for (auto j = 0; j < 8; j++) { - seen[i][j] = 0; - visit1[i][j] = 0; - } -#else - seen[i] = 0; visit1[i] = 0; -#endif if (mode == 1) { for (auto j = 0; j < LANE_LIMIT; j++) { parents_v[i][j] = -1; @@ -248,13 +204,12 @@ class GlobalBFSState { void CreateTasks() { // workerTasks[workerId] = [task1, task2, ...] - auto queues = {&task_queues}; + auto queues = {&task_queues, &task_queues_reverse}; is_top_down = true; for (auto& queue : queues) { vector>> worker_tasks(num_threads); auto cur_worker = 0; - // int64_t *v = is_top_down ? (int64_t*)csr->v : (int64_t*)csr->reverse_v; - int64_t *v = (int64_t*)csr->v; + int64_t *v = is_top_down ? (int64_t*)csr->v : (int64_t*)csr->reverse_v; int64_t current_task_edges = 0; idx_t current_task_start = 0; for (idx_t i = 0; i < (idx_t)v_size; i++) { @@ -317,16 +272,12 @@ class GlobalBFSState { void DirectionSwitch() { // Determine the switch of algorithms // debug print - if (top_down_cost > bottom_up_cost) { + if (top_down_cost * alpha > bottom_up_cost) { is_top_down = false; } else { is_top_down = true; } - if (top_down_cost > 0) { - change = true; - } else { - change = false; - } + change = top_down_cost ? true : false; // clear the counters after the switch top_down_cost = 0; bottom_up_cost = 0; @@ -348,27 +299,15 @@ class GlobalBFSState { idx_t active = 0; DataChunk result; // 0 for length, 1 for path ClientContext& context; -#ifdef SEGMENT_BITSET - vector, 8>> seen; - vector, 8>> visit1; - vector, 8>> visit2; -#elif defined(ATOMIC_BITSET) - vector>> seen; - vector>> visit1; - vector>> visit2; -#else vector> seen; vector> visit1; vector> visit2; -#endif - vector> parents_v; vector> parents_e; atomic top_down_cost; atomic bottom_up_cost; - int64_t alpha = 1024; - int64_t beta = 64; + double alpha = 1; atomic is_top_down; idx_t num_threads; @@ -380,11 +319,7 @@ class GlobalBFSState { Barrier barrier; - std::chrono::microseconds time_elapsed = std::chrono::microseconds(0); - std::chrono::microseconds time_elapsed_task_init = std::chrono::microseconds(0); - // lock for next - mutable mutex lock; mutable vector locks; idx_t mode; @@ -411,42 +346,7 @@ class PathFindingGlobalState : public GlobalSinkState { } void Sink(DataChunk &input, PathFindingLocalState &lstate) { - if (operator_start_time == std::chrono::time_point()) { - operator_start_time = std::chrono::high_resolution_clock::now(); - } - if (child == 1 && pairs_start_time == std::chrono::time_point()) { - pairs_start_time = std::chrono::high_resolution_clock::now(); - } else if (child != 1 && csr_start_time == std::chrono::time_point()) { - csr_start_time = std::chrono::high_resolution_clock::now(); - } lstate.Sink(input, *global_csr, child); - if (child == 1) { - pairs_end_time = std::chrono::high_resolution_clock::now(); - } else { - csr_end_time = std::chrono::high_resolution_clock::now(); - } - } - - void CSRCreateEdge() { - DataChunk input; - global_inputs->InitializeScanChunk(input); - ColumnDataScanState scan_state; - global_inputs->InitializeScan(scan_state); - auto result = Vector(LogicalTypeId::BIGINT); - while (global_inputs->Scan(scan_state, input)) { - if (!global_csr->initialized_e) { - const auto e_size = input.data[7].GetValue(0).GetValue(); - global_csr->InitializeEdge(e_size); - } - TernaryExecutor::Execute( - input.data[6], input.data[4], input.data[2], result, input.size(), - [&](int64_t src, int64_t dst, int64_t edge_id) { - const auto pos = ++global_csr->v[src + 1]; - global_csr->e[static_cast(pos) - 1] = dst; - global_csr->edge_ids[static_cast(pos) - 1] = edge_id; - return 1; - }); - } } // pairs is a 2-column table with src and dst @@ -457,13 +357,6 @@ class PathFindingGlobalState : public GlobalSinkState { ColumnDataScanState scan_state; ColumnDataAppendState append_state; - std::chrono::time_point operator_start_time; - std::chrono::microseconds time_elapsed_operator = std::chrono::microseconds(0); - std::chrono::time_point pairs_start_time; - std::chrono::time_point pairs_end_time; - std::chrono::time_point csr_start_time; - std::chrono::time_point csr_end_time; - shared_ptr global_csr; // state for BFS unique_ptr global_bfs_state; @@ -531,18 +424,17 @@ class PhysicalIterativeTask : public ExecutorTask { do { bfs_state->InitTask(worker_id); - // if (bfs_state->is_top_down) { - // IterativeLengthTopDown(); - // } else { - // IterativeLengthBottomUp(); - // } - IterativeLengthTopDown(); + if (bfs_state->is_top_down) { + IterativeLengthTopDown(); + } else { + IterativeLengthBottomUp(); + } barrier.Wait(); if (worker_id == 0) { ReachDetect(); - // bfs_state->DirectionSwitch(); + bfs_state->DirectionSwitch(); } barrier.Wait(); } while (change); @@ -563,22 +455,15 @@ class PhysicalIterativeTask : public ExecutorTask { auto& next = bfs_state->iter & 1 ? bfs_state->visit2 : bfs_state->visit1; auto& barrier = bfs_state->barrier; int64_t *v = (int64_t *)state.global_csr->v; - // int64_t *reverse_v = (int64_t *)state.global_csr->reverse_v; + int64_t *reverse_v = (int64_t *)state.global_csr->reverse_v; vector &e = state.global_csr->e; auto& top_down_cost = bfs_state->top_down_cost; auto& bottom_up_cost = bfs_state->bottom_up_cost; auto& lane_to_num = bfs_state->lane_to_num; - auto& change = bfs_state->change; // clear next before each iteration for (auto i = left; i < right; i++) { -#ifdef SEGMENT_BITSET - for (auto j = 0; j < 8; j++) { - next[i][j].store(0, std::memory_order_relaxed); - } -#else next[i] = 0; -#endif } barrier.Wait(); @@ -591,201 +476,79 @@ class PhysicalIterativeTask : public ExecutorTask { auto start = task.first; auto end = task.second; -#ifdef SEGMENT_BITSET - idx_t old_next, new_next; -#else - std::bitset old_next, new_next; -#endif - for (auto i = start; i < end; i++) { -#ifdef SEGMENT_BITSET - for (auto j = 0; j < 8; j++) { - if (visit[i][j].load(std::memory_order_relaxed)) { - for (auto offset = v[i]; offset < v[i + 1]; offset++) { - auto n = e[offset]; - do { - old_next = next[n][j].load(); - new_next = old_next | visit[i][j].load(); - } while (!next[n][j].compare_exchange_weak(old_next, new_next)); - } - } - } -#elif defined(ATOMIC_BITSET) - if (visit[i].load(std::memory_order_relaxed).any()) { - for (auto offset = v[i]; offset < v[i + 1]; offset++) { - auto n = e[offset]; - do { - old_next = next[n].load(); - new_next = old_next | visit[i].load(); - } while (!next[n].compare_exchange_weak(old_next, new_next)); - } - } -#else if (visit[i].any()) { for (auto offset = v[i]; offset < v[i + 1]; offset++) { auto n = e[offset]; -#ifdef ELEMENT_LOCK std::lock_guard lock(bfs_state->locks[n]); -#endif next[n] |= visit[i]; } } -#endif } } - change = false; barrier.Wait(); for (auto i = left; i < right; i++) { -#ifdef SEGMENT_BITSET - for (auto j = 0; j < 8; j++) { - if (next[i][j].load()) { - next[i][j].store(next[i][j].load() & ~seen[i][j].load()); - seen[i][j].store(seen[i][j].load() | next[i][j].load()); - change |= next[i][j].load(); - // top_down_cost += v[i + 1] - v[i]; - } - // if (~seen[i][j].load()) { - // bottom_up_cost += reverse_v[i + 1] - reverse_v[i]; - // } - } -#elif defined(ATOMIC_BITSET) - if (next[i].load().any()) { - next[i].store(next[i].load() & ~seen[i].load()); - seen[i].store(seen[i].load() | next[i].load()); - change |= next[i].load().any(); - // top_down_cost += v[i + 1] - v[i]; - } - // if (~seen[i].load().all()) { - // bottom_up_cost += reverse_v[i + 1] - reverse_v[i]; - // } -#else if (next[i].any()) { next[i] &= ~seen[i]; seen[i] |= next[i]; - change |= next[i].any(); - // top_down_cost += v[i + 1] - v[i]; + top_down_cost += v[i + 1] - v[i]; + } + if (~(seen[i].all())) { + bottom_up_cost += reverse_v[i + 1] - reverse_v[i]; } - // if (~seen[i].all()) { - // bottom_up_cost += reverse_v[i + 1] - reverse_v[i]; - // } -#endif } } -// void IterativeLengthBottomUp() { -// auto& bfs_state = state.global_bfs_state; -// auto& seen = bfs_state->seen; -// auto& visit = bfs_state->iter & 1 ? bfs_state->visit1 : bfs_state->visit2; -// auto& next = bfs_state->iter & 1 ? bfs_state->visit2 : bfs_state->visit1; -// auto& barrier = bfs_state->barrier; -// int64_t *v = (int64_t *)state.global_csr->reverse_v; -// int64_t *normal_v = (int64_t *)state.global_csr->v; -// vector &e = state.global_csr->reverse_e; -// auto& top_down_cost = bfs_state->top_down_cost; -// auto& bottom_up_cost = bfs_state->bottom_up_cost; - -// // clear next before each iteration -// for (auto i = left; i < right; i++) { -// #ifdef SEGMENT_BITSET -// for (auto j = 0; j < 8; j++) { -// next[i][j].store(0, std::memory_order_relaxed); -// } -// #else -// next[i] = 0; -// #endif -// } + void IterativeLengthBottomUp() { + auto& bfs_state = state.global_bfs_state; + auto& seen = bfs_state->seen; + auto& visit = bfs_state->iter & 1 ? bfs_state->visit1 : bfs_state->visit2; + auto& next = bfs_state->iter & 1 ? bfs_state->visit2 : bfs_state->visit1; + auto& barrier = bfs_state->barrier; + int64_t *v = (int64_t *)state.global_csr->reverse_v; + int64_t *normal_v = (int64_t *)state.global_csr->v; + vector &e = state.global_csr->reverse_e; + auto& top_down_cost = bfs_state->top_down_cost; + auto& bottom_up_cost = bfs_state->bottom_up_cost; + + // clear next before each iteration + for (auto i = left; i < right; i++) { + next[i] = 0; + } -// barrier.Wait(); - -// while (true) { -// auto task = bfs_state->FetchTask(worker_id); -// if (task.first == task.second) { -// break; -// } -// auto start = task.first; -// auto end = task.second; - -// #ifdef SEGMENT_BITSET -// idx_t old_next, new_next; -// #else -// std::bitset old_next, new_next; -// #endif - -// for (auto i = start; i < end; i++) { -// #ifdef SEGMENT_BITSET -// for (auto j = 0; j < 8; j++) { -// if (~seen[i][j] == 0) { -// continue; -// } - -// for (auto offset = v[i]; offset < v[i + 1]; offset++) { -// auto n = e[offset]; -// if (visit[n][j].load(std::memory_order_relaxed)) { -// do { -// old_next = next[i][j].load(); -// new_next = old_next | visit[n][j].load(); -// } while (!next[i][j].compare_exchange_weak(old_next, new_next)); -// } -// } - -// if (next[i][j].load()) { -// next[i][j].store(next[i][j].load() & ~seen[i][j].load()); -// seen[i][j].store(seen[i][j].load() | next[i][j].load()); - -// top_down_cost += normal_v[i + 1] - normal_v[i]; -// } -// if (~seen[i][j].load()) { -// bottom_up_cost += v[i + 1] - v[i]; -// } -// } -// #elif defined(ATOMIC_BITSET) -// if (seen[i].load().all()) { -// continue; -// } - -// for (auto offset = v[i]; offset < v[i + 1]; offset++) { -// auto n = e[offset]; -// do { -// old_next = next[i].load(); -// new_next = old_next | visit[n].load(); -// } while (!next[i].compare_exchange_weak(old_next, new_next)); -// } - -// if (next[i].load().any()) { -// next[i].store(next[i].load() & ~seen[i].load()); -// seen[i].store(seen[i].load() | next[i].load()); - -// top_down_cost += normal_v[i + 1] - normal_v[i]; -// } -// if (~seen[i].load().all()) { -// bottom_up_cost += v[i + 1] - v[i]; -// } -// #else -// if (seen[i].all()) { -// continue; -// } - -// for (auto offset = v[i]; offset < v[i + 1]; offset++) { -// auto n = e[offset]; -// next[i] |= visit[n]; -// } - -// if (next[i].any()) { -// next[i] &= ~seen[i]; -// seen[i] |= next[i]; - -// top_down_cost += normal_v[i + 1] - normal_v[i]; -// } -// if (~seen[i].all()) { -// bottom_up_cost += v[i + 1] - v[i]; -// } - -// #endif -// } -// } -// } + barrier.Wait(); + + while (true) { + auto task = bfs_state->FetchTask(worker_id); + if (task.first == task.second) { + break; + } + auto start = task.first; + auto end = task.second; + + for (auto i = start; i < end; i++) { + if (seen[i].all()) { + continue; + } + + for (auto offset = v[i]; offset < v[i + 1]; offset++) { + auto n = e[offset]; + next[i] |= visit[n]; + } + + if (next[i].any()) { + next[i] &= ~seen[i]; + seen[i] |= next[i]; + top_down_cost += normal_v[i + 1] - normal_v[i]; + } + if (~(seen[i].all())) { + bottom_up_cost += v[i + 1] - v[i]; + } + } + } + } void ReachDetect() { auto &bfs_state = state.global_bfs_state; @@ -796,13 +559,7 @@ class PhysicalIterativeTask : public ExecutorTask { int64_t search_num = bfs_state->lane_to_num[lane]; if (search_num >= 0) { // active lane int64_t dst_pos = bfs_state->vdata_dst.sel->get_index(search_num); -#ifdef SEGMENT_BITSET - if (bfs_state->seen[bfs_state->dst[dst_pos]][lane / 64] & ((idx_t)1 << (lane % 64))) { -#elif defined(ATOMIC_BITSET) - if (bfs_state->seen[bfs_state->dst[dst_pos]].load()[lane]) { -#else if (bfs_state->seen[bfs_state->dst[dst_pos]][lane]) { -#endif result_data[search_num] = bfs_state->iter; /* found at iter => iter = path length */ bfs_state->lane_to_num[lane] = -1; // mark inactive @@ -862,25 +619,25 @@ class PhysicalCSREdgeCreationTask : public ExecutorTask { } } if (!global_csr->initialized_e) { - const auto e_size = input.data[7].GetValue(0).GetValue(); + const auto e_size = input.data[8].GetValue(0).GetValue(); global_csr->InitializeEdge(e_size); } TernaryExecutor::Execute( - input.data[6], input.data[4], input.data[2], result, input.size(), + input.data[4], input.data[7], input.data[2], result, input.size(), [&](int64_t src, int64_t dst, int64_t edge_id) { const auto pos = ++global_csr->v[src + 1]; global_csr->e[static_cast(pos) - 1] = dst; global_csr->edge_ids[static_cast(pos) - 1] = edge_id; return 1; }); - // TernaryExecutor::Execute( - // input.data[7], input.data[4], input.data[2], result, input.size(), - // [&](int64_t dst, int64_t src, int64_t edge_id) { - // const auto pos = ++global_csr->reverse_v[dst + 1]; - // global_csr->reverse_e[static_cast(pos) - 1] = src; - // global_csr->reverse_edge_ids[static_cast(pos) - 1] = edge_id; - // return 1; - // }); + TernaryExecutor::Execute( + input.data[7], input.data[4], input.data[2], result, input.size(), + [&](int64_t dst, int64_t src, int64_t edge_id) { + const auto pos = ++global_csr->reverse_v[dst + 1]; + global_csr->reverse_e[static_cast(pos) - 1] = src; + global_csr->reverse_edge_ids[static_cast(pos) - 1] = edge_id; + return 1; + }); } event->FinishTask(); return TaskExecutionResult::TASK_FINISHED; @@ -899,11 +656,7 @@ class CSREdgeCreationEvent : public BasePipelineEvent { PathFindingGlobalState &gstate; -private: - std::chrono::high_resolution_clock::time_point start_time; - void Schedule() override { - start_time = std::chrono::high_resolution_clock::now(); auto &context = pipeline->GetClientContext(); auto &ts = TaskScheduler::GetScheduler(context); idx_t num_threads = ts.NumberOfThreads(); @@ -923,10 +676,6 @@ class CSREdgeCreationEvent : public BasePipelineEvent { auto &gstate = this->gstate; auto &global_csr = gstate.global_csr; global_csr->is_ready = true; - auto end_time = std::chrono::high_resolution_clock::now(); - auto duration = std::chrono::duration_cast(end_time - start_time); - string msg = "CSR edge creation time: " + std::to_string(duration.count()) + " microseconds"; - Printer::Print(msg); // debug print // global_csr->Print(); } @@ -940,13 +689,8 @@ class ParallelIterativeEvent : public BasePipelineEvent { PathFindingGlobalState &gstate; -private: - std::chrono::high_resolution_clock::time_point start_time; - public: void Schedule() override { - start_time = std::chrono::high_resolution_clock::now(); - auto &bfs_state = gstate.global_bfs_state; auto &context = pipeline->GetClientContext(); @@ -960,10 +704,6 @@ class ParallelIterativeEvent : public BasePipelineEvent { void FinishEvent() override { auto& bfs_state = gstate.global_bfs_state; - auto end_time = std::chrono::high_resolution_clock::now(); - auto duration = std::chrono::duration_cast(end_time - start_time); - bfs_state->time_elapsed += duration; - // if remaining pairs, schedule the BFS for the next batch if (bfs_state->started_searches < gstate.global_tasks->Count()) { PhysicalPathFinding::ScheduleBFSTasks(*pipeline, *this, gstate); @@ -992,7 +732,6 @@ class SequentialIterativeEvent : public BasePipelineEvent { IterativeLengthFunction(gstate.global_csr, pairs, result); auto end_time = std::chrono::high_resolution_clock::now(); auto duration = std::chrono::duration_cast(end_time - start_time); - bfs_state->time_elapsed += duration; } private: @@ -1138,17 +877,16 @@ class PhysicalShortestPathTask : public ExecutorTask { do { bfs_state->InitTask(worker_id); - // if (bfs_state->is_top_down) { - // IterativeLengthTopDown(); - // } else { - // IterativeLengthBottomUp(); - // } - IterativeLengthTopDown(); + if (bfs_state->is_top_down) { + IterativePathTopDown(); + } else { + IterativePathBottomUp(); + } barrier.Wait(); if (worker_id == 0) { ReachDetect(); - // bfs_state->DirectionSwitch(); + bfs_state->DirectionSwitch(); } barrier.Wait(); @@ -1163,31 +901,24 @@ class PhysicalShortestPathTask : public ExecutorTask { } private: - void IterativeLengthTopDown() { + void IterativePathTopDown() { auto& bfs_state = state.global_bfs_state; auto& seen = bfs_state->seen; auto& visit = bfs_state->iter & 1 ? bfs_state->visit1 : bfs_state->visit2; auto& next = bfs_state->iter & 1 ? bfs_state->visit2 : bfs_state->visit1; auto& barrier = bfs_state->barrier; int64_t *v = (int64_t *)state.global_csr->v; - // int64_t *reverse_v = (int64_t *)state.global_csr->reverse_v; + int64_t *reverse_v = (int64_t *)state.global_csr->reverse_v; vector &e = state.global_csr->e; auto& edge_ids = state.global_csr->edge_ids; auto& top_down_cost = bfs_state->top_down_cost; auto& bottom_up_cost = bfs_state->bottom_up_cost; auto& parents_v = bfs_state->parents_v; auto& parents_e = bfs_state->parents_e; - auto& change = bfs_state->change; // clear next before each iteration for (auto i = left; i < right; i++) { -#ifdef SEGMENT_BITSET - for (auto j = 0; j < 8; j++) { - next[i][j].store(0, std::memory_order_relaxed); - } -#else next[i] = 0; -#endif } barrier.Wait(); @@ -1200,57 +931,12 @@ class PhysicalShortestPathTask : public ExecutorTask { auto start = task.first; auto end = task.second; -#ifdef SEGMENT_BITSET - idx_t old_next, new_next; -#else - std::bitset old_next, new_next; -#endif - for (auto i = start; i < end; i++) { -#ifdef SEGMENT_BITSET - for (auto j = 0; j < 8; j++) { - if (visit[i][j].load(std::memory_order_relaxed)) { - for (auto offset = v[i]; offset < v[i + 1]; offset++) { - auto n = e[offset]; - auto edge_id = edge_ids[offset]; - do { - old_next = next[n][j].load(); - new_next = old_next | visit[i][j].load(); - } while (!next[n][j].compare_exchange_weak(old_next, new_next)); - for (auto l = 0; l < LANE_LIMIT; l++) { - parents_v[n][l] = ((parents_v[n][l] == -1) && visit[i][l / 64].load(std::memory_order_relaxed) & ((idx_t)1 << (l % 64))) - ? i : parents_v[n][l]; - parents_e[n][l] = ((parents_e[n][l] == -1) && visit[i][l / 64].load(std::memory_order_relaxed) & ((idx_t)1 << (l % 64))) - ? edge_id : parents_e[n][l]; - } - } - } - } -#elif defined(ATOMIC_BITSET) - if (visit[i].load(std::memory_order_relaxed).any()) { - for (auto offset = v[i]; offset < v[i + 1]; offset++) { - auto n = e[offset]; - auto edge_id = edge_ids[offset]; - do { - old_next = next[n].load(); - new_next = old_next | visit[i].load(); - } while (!next[n].compare_exchange_weak(old_next, new_next)); - for (auto l = 0; l < LANE_LIMIT; l++) { - parents_v[n][l] = ((parents_v[n][l] == -1) && visit[i].load()[l]) - ? i : parents_v[n][l]; - parents_e[n][l] = ((parents_e[n][l] == -1) && visit[i].load()[l]) - ? edge_id : parents_e[n][l]; - } - } - } -#else if (visit[i].any()) { for (auto offset = v[i]; offset < v[i + 1]; offset++) { auto n = e[offset]; auto edge_id = edge_ids[offset]; -#ifdef ELEMENT_LOCK std::lock_guard lock(bfs_state->locks[n]); -#endif next[n] |= visit[i]; for (auto l = 0; l < LANE_LIMIT; l++) { parents_v[n][l] = ((parents_v[n][l] == -1) && visit[i][l]) @@ -1260,184 +946,81 @@ class PhysicalShortestPathTask : public ExecutorTask { } } } -#endif } } - change = false; barrier.Wait(); for (auto i = left; i < right; i++) { -#ifdef SEGMENT_BITSET - for (auto j = 0; j < 8; j++) { - if (next[i][j].load()) { - next[i][j].store(next[i][j].load() & ~seen[i][j].load()); - seen[i][j].store(seen[i][j].load() | next[i][j].load()); - change |= next[i][j].load(); - // top_down_cost += v[i + 1] - v[i]; - } - // if (~seen[i][j].load()) { - // bottom_up_cost += reverse_v[i + 1] - reverse_v[i]; - // } - } -#elif defined(ATOMIC_BITSET) - if (next[i].load().any()) { - next[i].store(next[i].load() & ~seen[i].load()); - seen[i].store(seen[i].load() | next[i].load()); - change |= next[i].load().any(); - // top_down_cost += v[i + 1] - v[i]; - } - // if (~seen[i].load().all()) { - // bottom_up_cost += reverse_v[i + 1] - reverse_v[i]; - // } -#else if (next[i].any()) { next[i] &= ~seen[i]; seen[i] |= next[i]; - change |= next[i].any(); - // top_down_cost += v[i + 1] - v[i]; + top_down_cost += v[i + 1] - v[i]; + } + if (~(seen[i].all())) { + bottom_up_cost += reverse_v[i + 1] - reverse_v[i]; } - // if (~seen[i].all()) { - // bottom_up_cost += reverse_v[i + 1] - reverse_v[i]; - // } -#endif } } -// void IterativeLengthBottomUp() { -// auto& bfs_state = state.global_bfs_state; -// auto& seen = bfs_state->seen; -// auto& visit = bfs_state->iter & 1 ? bfs_state->visit1 : bfs_state->visit2; -// auto& next = bfs_state->iter & 1 ? bfs_state->visit2 : bfs_state->visit1; -// auto& barrier = bfs_state->barrier; -// int64_t *v = (int64_t *)state.global_csr->reverse_v; -// int64_t *normal_v = (int64_t *)state.global_csr->v; -// vector &e = state.global_csr->reverse_e; -// auto& edge_ids = state.global_csr->reverse_edge_ids; -// auto& top_down_cost = bfs_state->top_down_cost; -// auto& bottom_up_cost = bfs_state->bottom_up_cost; -// auto& parents_v = bfs_state->parents_v; -// auto& parents_e = bfs_state->parents_e; - -// // clear next before each iteration -// for (auto i = left; i < right; i++) { -// #ifdef SEGMENT_BITSET -// for (auto j = 0; j < 8; j++) { -// next[i][j].store(0, std::memory_order_relaxed); -// } -// #else -// next[i] = 0; -// #endif -// } + void IterativePathBottomUp() { + auto& bfs_state = state.global_bfs_state; + auto& seen = bfs_state->seen; + auto& visit = bfs_state->iter & 1 ? bfs_state->visit1 : bfs_state->visit2; + auto& next = bfs_state->iter & 1 ? bfs_state->visit2 : bfs_state->visit1; + auto& barrier = bfs_state->barrier; + int64_t *v = (int64_t *)state.global_csr->reverse_v; + int64_t *normal_v = (int64_t *)state.global_csr->v; + vector &e = state.global_csr->reverse_e; + auto& edge_ids = state.global_csr->reverse_edge_ids; + auto& top_down_cost = bfs_state->top_down_cost; + auto& bottom_up_cost = bfs_state->bottom_up_cost; + auto& parents_v = bfs_state->parents_v; + auto& parents_e = bfs_state->parents_e; + + // clear next before each iteration + for (auto i = left; i < right; i++) { + next[i] = 0; + } -// barrier.Wait(); - -// while (true) { -// auto task = bfs_state->FetchTask(worker_id); -// if (task.first == task.second) { -// break; -// } -// auto start = task.first; -// auto end = task.second; - -// #ifdef SEGMENT_BITSET -// idx_t old_next, new_next; -// #else -// std::bitset old_next, new_next; -// #endif - -// for (auto i = start; i < end; i++) { -// #ifdef SEGMENT_BITSET -// for (auto j = 0; j < 8; j++) { -// if (~seen[i][j] == 0) { -// continue; -// } - -// for (auto offset = v[i]; offset < v[i + 1]; offset++) { -// auto n = e[offset]; -// auto edge_id = edge_ids[offset]; -// do { -// old_next = next[i][j].load(); -// new_next = old_next | visit[n][j].load(); -// } while (!next[i][j].compare_exchange_weak(old_next, new_next)); -// for (auto l = 0; l < LANE_LIMIT; l++) { -// parents_v[i][l] = ((parents_v[i][l] == -1) && visit[n][l / 64].load(std::memory_order_relaxed) & ((idx_t)1 << (l % 64))) -// ? n : parents_v[i][l]; -// parents_e[i][l] = ((parents_e[i][l] == -1) && visit[n][l / 64].load(std::memory_order_relaxed) & ((idx_t)1 << (l % 64))) -// ? edge_id : parents_e[i][l]; -// } -// } - -// if (next[i][j].load()) { -// next[i][j].store(next[i][j].load() & ~seen[i][j].load(), std::memory_order_relaxed); -// seen[i][j].store(seen[i][j].load() | next[i][j].load(), std::memory_order_relaxed); - -// top_down_cost += normal_v[i + 1] - normal_v[i]; -// } -// if (~seen[i][j].load()) { -// bottom_up_cost += v[i + 1] - v[i]; -// } -// } -// #elif defined(ATOMIC_BITSET) -// if (seen[i].load().all()) { -// continue; -// } - -// for (auto offset = v[i]; offset < v[i + 1]; offset++) { -// auto n = e[offset]; -// auto edge_id = edge_ids[offset]; -// do { -// old_next = next[i].load(); -// new_next = old_next | visit[n].load(); -// } while (!next[i].compare_exchange_weak(old_next, new_next)); -// for (auto l = 0; l < LANE_LIMIT; l++) { -// parents_v[i][l] = ((parents_v[i][l] == -1) && visit[n].load()[l]) -// ? i : parents_v[i][l]; -// parents_e[i][l] = ((parents_e[i][l] == -1) && visit[n].load()[l]) -// ? edge_id : parents_e[i][l]; -// } -// } - -// if (next[i].load().any()) { -// next[i].store(next[i].load() & ~seen[i].load()); -// seen[i].store(seen[i].load() | next[i].load()); - -// top_down_cost += normal_v[i + 1] - normal_v[i]; -// } -// if (~seen[i].load().all()) { -// bottom_up_cost += v[i + 1] - v[i]; -// } - -// #else -// if (seen[i].all()) { -// continue; -// } - -// for (auto offset = v[i]; offset < v[i + 1]; offset++) { -// auto n = e[offset]; -// auto edge_id = edge_ids[offset]; -// next[i] |= visit[n]; -// for (auto l = 0; l < LANE_LIMIT; l++) { -// parents_v[i][l] = ((parents_v[i][l] == -1) && visit[n][l]) -// ? i : parents_v[i][l]; -// parents_e[i][l] = ((parents_e[i][l] == -1) && visit[n][l]) -// ? edge_id : parents_e[i][l]; -// } -// } - -// if (next[i].any()) { -// next[i] &= ~seen[i]; -// seen[i] |= next[i]; - -// top_down_cost += normal_v[i + 1] - normal_v[i]; -// } -// if (~seen[i].any()) { -// bottom_up_cost += v[i + 1] - v[i]; -// } -// #endif -// } -// } -// } + barrier.Wait(); + + while (true) { + auto task = bfs_state->FetchTask(worker_id); + if (task.first == task.second) { + break; + } + auto start = task.first; + auto end = task.second; + + for (auto i = start; i < end; i++) { + if (seen[i].all()) { + continue; + } + + for (auto offset = v[i]; offset < v[i + 1]; offset++) { + auto n = e[offset]; + auto edge_id = edge_ids[offset]; + next[i] |= visit[n]; + for (auto l = 0; l < LANE_LIMIT; l++) { + parents_v[i][l] = ((parents_v[i][l] == -1) && visit[n][l]) + ? n : parents_v[i][l]; + parents_e[i][l] = ((parents_e[i][l] == -1) && visit[n][l]) + ? edge_id : parents_e[i][l]; + } + } + + if (next[i].any()) { + next[i] &= ~seen[i]; + seen[i] |= next[i]; + top_down_cost += normal_v[i + 1] - normal_v[i]; + } + if (~(seen[i].any())) { + bottom_up_cost += v[i + 1] - v[i]; + } + } + } + } void ReachDetect() { auto &bfs_state = state.global_bfs_state; @@ -1451,13 +1034,7 @@ class PhysicalShortestPathTask : public ExecutorTask { if (search_num >= 0) { // active lane //! Check if dst for a source has been seen int64_t dst_pos = bfs_state->vdata_dst.sel->get_index(search_num); -#ifdef SEGMENT_BITSET - if (bfs_state->seen[bfs_state->dst[dst_pos]][lane / 64] & ((idx_t)1 << (lane % 64))) { -#elif defined(ATOMIC_BITSET) - if (bfs_state->seen[bfs_state->dst[dst_pos]].load()[lane]) { -#else if (bfs_state->seen[bfs_state->dst[dst_pos]][lane]) { -#endif bfs_state->active--; } } @@ -1465,13 +1042,6 @@ class PhysicalShortestPathTask : public ExecutorTask { if (bfs_state->active == 0) { change = false; } - // else { - // if (top_down_cost > 0 || bottom_up_cost > 0) { - // change = true; - // } else { - // change = false; - // } - // } // into the next iteration bfs_state->iter++; } @@ -1564,9 +1134,6 @@ class ParallelShortestPathEvent : public BasePipelineEvent { PathFindingGlobalState &gstate; -private: - std::chrono::high_resolution_clock::time_point start_time; - public: void Schedule() override { auto &bfs_state = gstate.global_bfs_state; @@ -1576,15 +1143,11 @@ class ParallelShortestPathEvent : public BasePipelineEvent { for (idx_t tnum = 0; tnum < bfs_state->num_threads; tnum++) { bfs_tasks.push_back(make_uniq(shared_from_this(), context, gstate, tnum)); } - start_time = std::chrono::high_resolution_clock::now(); SetTasks(std::move(bfs_tasks)); } void FinishEvent() override { auto &bfs_state = gstate.global_bfs_state; - auto end_time = std::chrono::high_resolution_clock::now(); - auto duration = std::chrono::duration_cast(end_time - start_time); - bfs_state->time_elapsed += duration; // if remaining pairs, schedule the BFS for the next batch if (bfs_state->started_searches < gstate.global_tasks->Count()) { @@ -1610,7 +1173,6 @@ class SequentialShortestPathEvent : public BasePipelineEvent { ShortestPathFunction(gstate.global_csr, pairs, result); auto end_time = std::chrono::high_resolution_clock::now(); auto duration = std::chrono::duration_cast(end_time - start_time); - bfs_state->time_elapsed = duration; } private: @@ -1829,13 +1391,6 @@ PhysicalPathFinding::Finalize(Pipeline &pipeline, Event &event, auto csr_event = make_shared(gstate, pipeline); event.InsertEvent(std::move(std::dynamic_pointer_cast(csr_event))); } else if (gstate.child == 1 && global_tasks->Count() > 0) { - auto csr_duration = std::chrono::duration_cast(gstate.csr_end_time - gstate.csr_start_time); - string msg_csr = "CSR vertex creation time: " + std::to_string(csr_duration.count()) + " microseconds"; - Printer::Print(msg_csr); - auto pairs_duration = std::chrono::duration_cast(gstate.pairs_end_time - gstate.pairs_start_time); - string msg_pairs = "Pair creation time: " + std::to_string(pairs_duration.count()) + " microseconds"; - Printer::Print(msg_pairs); - auto start_time = std::chrono::high_resolution_clock::now(); auto all_pairs = make_shared(); DataChunk pairs; global_tasks->InitializeScanChunk(*all_pairs); @@ -1845,31 +1400,23 @@ PhysicalPathFinding::Finalize(Pipeline &pipeline, Event &event, while (global_tasks->Scan(scan_state, pairs)) { all_pairs->Append(pairs, true); } - auto end_time = std::chrono::high_resolution_clock::now(); - auto duration = std::chrono::duration_cast(end_time - start_time); - string msg = "Time to collect all pairs: " + std::to_string(duration.count()) + " microseconds"; - Printer::Print(msg); // debug print // all_pairs->Print(); - start_time = std::chrono::high_resolution_clock::now(); auto &ts = TaskScheduler::GetScheduler(context); idx_t num_threads = ts.NumberOfThreads(); auto& client_config = ClientConfig::GetConfig(context); - auto const barrier_type_idx = client_config.set_variables.find("experimental_path_finding_operator_barrier"); - auto barrier_type = barrier_type_idx != client_config.set_variables.end() ? barrier_type_idx->second.GetValue() : 0; idx_t mode = this->mode == "iterativelength" ? 0 : 1; - gstate.global_bfs_state = make_uniq(csr, all_pairs, csr->v_size - 2, num_threads, barrier_type, mode, context); + gstate.global_bfs_state = make_uniq(csr, all_pairs, csr->v_size - 2, num_threads, mode, context); auto const task_size = client_config.set_variables.find("experimental_path_finding_operator_task_size"); gstate.global_bfs_state->split_size = task_size != client_config.set_variables.end() ? task_size->second.GetValue() : 256; + auto const alpha = client_config.set_variables.find("experimental_path_finding_operator_alpha"); + gstate.global_bfs_state->alpha = alpha != client_config.set_variables.end() ? alpha->second.GetValue() : 1; + auto const sequential = client_config.set_variables.find("experimental_path_finding_operator_sequential"); - end_time = std::chrono::high_resolution_clock::now(); - duration = std::chrono::duration_cast(end_time - start_time); - msg = "Time to initialize BFS state: " + std::to_string(duration.count()) + " microseconds"; - Printer::Print(msg); if (sequential != client_config.set_variables.end() && sequential->second.GetValue()) { if (gstate.mode == "iterativelength") { auto bfs_event = make_shared(gstate, pipeline); @@ -1894,7 +1441,6 @@ PhysicalPathFinding::Finalize(Pipeline &pipeline, Event &event, void PhysicalPathFinding::ScheduleBFSTasks(Pipeline &pipeline, Event &event, GlobalSinkState &state) { - auto start_time = std::chrono::high_resolution_clock::now(); auto &gstate = state.Cast(); auto &bfs_state = gstate.global_bfs_state; @@ -1906,6 +1452,7 @@ void PhysicalPathFinding::ScheduleBFSTasks(Pipeline &pipeline, Event &event, auto result_data = FlatVector::GetData(bfs_state->result.data[0]); auto& result_validity = FlatVector::Validity(bfs_state->result.data[0]); + std::bitset seen_mask = ~0; for (int64_t lane = 0; lane < LANE_LIMIT; lane++) { bfs_state->lane_to_num[lane] = -1; @@ -1920,22 +1467,18 @@ void PhysicalPathFinding::ScheduleBFSTasks(Pipeline &pipeline, Event &event, result_data[search_num] = (uint64_t)0; // path of length 0 does not require a search } else { -#ifdef SEGMENT_BITSET - bfs_state->visit1[bfs_state->src[src_pos]][lane / 64] |= ((idx_t)1 << (lane % 64)); -#elif defined(ATOMIC_BITSET) - auto new_visit = bfs_state->visit1[bfs_state->src[src_pos]].load(); - new_visit[lane] = true; - bfs_state->visit1[bfs_state->src[src_pos]].store(new_visit); -#else bfs_state->visit1[bfs_state->src[src_pos]][lane] = true; -#endif bfs_state->lane_to_num[lane] = search_num; // active lane bfs_state->active++; + seen_mask[lane] = false; break; } - } } + for (int64_t i = 0; i < bfs_state->v_size; i++) { + bfs_state->seen[i] = seen_mask; + } + if (gstate.mode == "iterativelength") { auto bfs_event = make_shared(gstate, pipeline); event.InsertEvent(std::move(std::dynamic_pointer_cast(bfs_event))); @@ -1945,9 +1488,6 @@ void PhysicalPathFinding::ScheduleBFSTasks(Pipeline &pipeline, Event &event, } } - auto end_time = std::chrono::high_resolution_clock::now(); - auto duration = std::chrono::duration_cast(end_time - start_time); - bfs_state->time_elapsed_task_init += duration; } //===--------------------------------------------------------------------===// @@ -2022,12 +1562,6 @@ PhysicalPathFinding::GetData(ExecutionContext &context, DataChunk &result, } pf_bfs_state->result.SetCardinality(*pf_bfs_state->pairs); - string msg = "Task init time elapsed: " + to_string(pf_bfs_state->time_elapsed_task_init.count()) + " microseconds"; - Printer::Print(msg); - - string message = "Algorithm time elapsed: " + to_string(pf_bfs_state->time_elapsed.count()) + " microseconds"; - Printer::Print(message); - result.Move(*pf_bfs_state->pairs); auto result_path = make_uniq(); //! Split off the path from the path length, and then fuse into the result @@ -2040,14 +1574,6 @@ PhysicalPathFinding::GetData(ExecutionContext &context, DataChunk &result, throw NotImplementedException("Unrecognized mode for Path Finding"); } - auto end_time = std::chrono::high_resolution_clock::now(); - auto results_duration = std::chrono::duration_cast(end_time - results_start_time); - string message_results = "Results time elapsed: " + to_string(results_duration.count()) + " microseconds"; - Printer::Print(message_results); - auto duration = std::chrono::duration_cast(end_time - pf_sink.operator_start_time); - string message2 = "Total time elapsed: " + to_string(duration.count()) + " microseconds\n"; - Printer::Print(message2); - // result.Print(); return result.size() == 0 ? SourceResultType::FINISHED diff --git a/test/sql/path_finding/parallel_path_finding.test b/test/sql/path_finding/parallel_path_finding.test index 1237e438..af895f0b 100644 --- a/test/sql/path_finding/parallel_path_finding.test +++ b/test/sql/path_finding/parallel_path_finding.test @@ -10,7 +10,6 @@ require duckpgq statement ok set experimental_path_finding_operator=true; set experimental_path_finding_operator_sequential=false; -set experimental_path_finding_operator_barrier=2; statement ok CREATE TABLE pair(src BIGINT, dst BIGINT); INSERT INTO pair(src, dst) VALUES (0, 1), (1, 2), (2,0); @@ -57,19 +56,25 @@ query III SELECT *, shortestpath(src, dst, 'pair') as path FROM pair AS p WHERE p.src BETWEEN (SELECT CREATE_CSR_EDGE( - (SELECT count(a.id) as v_size FROM Student a), - (SELECT count(k.src) as e_size from know k), + (SELECT count(a.id) as v_size FROM Student a), + (SELECT count(k.src) as e_size from know k), a.rowid, c.rowid, k.rowid, - t.cnt) FROM Know k + t.cnt, + t_r.cnt_r) FROM Know k JOIN student a on a.id = k.src JOIN student c on c.id = k.dst JOIN (SELECT count(k.src) cnt, a.rowid as a_rowid FROM student a LEFT JOIN know k ON k.src = a.id GROUP BY a.rowid) t - ON t.a_rowid = a.rowid) AND p.dst; + ON t.a_rowid = a.rowid + JOIN (SELECT count(k.dst) cnt_r, c.rowid as c_rowid + FROM student c + LEFT JOIN know k ON k.dst = c.id + GROUP BY c.rowid) t_r + ON t_r.c_rowid = c.rowid) AND p.dst; ---- 0 1 NULL 1 2 [1, 3, 0, 1, 2] @@ -110,14 +115,20 @@ WHERE p.src BETWEEN (SELECT CREATE_CSR_EDGE( a.rowid, c.rowid, k.rowid, - t.cnt) FROM Person_knows_Person k + t.cnt, + t_r.cnt_r) FROM Person_knows_Person k JOIN Person a on a.id = k.Person1Id JOIN Person c on c.id = k.Person2Id JOIN (SELECT count(k.Person1Id) cnt, a.rowid as a_rowid FROM Person a LEFT JOIN Person_knows_Person k ON k.Person1Id = a.id GROUP BY a.rowid) t - ON t.a_rowid = a.rowid) AND p.dst; + ON t.a_rowid = a.rowid + JOIN (SELECT count(k.Person2Id) cnt_r, c.rowid as c_rowid + FROM Person c + LEFT JOIN Person_knows_Person k ON k.Person2Id = c.id + GROUP BY c.rowid) t_r + ON t_r.c_rowid = c.rowid) AND p.dst; ---- 42 22 NULL 3 33 2 @@ -177,14 +188,20 @@ WHERE p.src BETWEEN (SELECT CREATE_CSR_EDGE( a.rowid, c.rowid, k.rowid, - t.cnt) FROM Person_knows_Person k + t.cnt, + t_r.cnt_r) FROM Person_knows_Person k JOIN Person a on a.id = k.Person1Id JOIN Person c on c.id = k.Person2Id JOIN (SELECT count(k.Person1Id) cnt, a.rowid as a_rowid FROM Person a LEFT JOIN Person_knows_Person k ON k.Person1Id = a.id GROUP BY a.rowid) t - ON t.a_rowid = a.rowid) AND p.dst; + ON t.a_rowid = a.rowid + JOIN (SELECT count(k.Person2Id) cnt_r, c.rowid as c_rowid + FROM Person c + LEFT JOIN Person_knows_Person k ON k.Person2Id = c.id + GROUP BY c.rowid) t_r + ON t_r.c_rowid = c.rowid) AND p.dst; ---- 10 2 NULL From 11f292a5b393b7c3a9cd454a6ab9951aeb6c8615 Mon Sep 17 00:00:00 2001 From: Pingan Ren Date: Fri, 28 Jun 2024 22:42:39 +0200 Subject: [PATCH 130/249] fix seen set bug --- .../physical_path_finding_operator.cpp | 28 +++++++++++++++---- 1 file changed, 23 insertions(+), 5 deletions(-) diff --git a/src/operators/physical_path_finding_operator.cpp b/src/operators/physical_path_finding_operator.cpp index 7856ba7d..ee504260 100644 --- a/src/operators/physical_path_finding_operator.cpp +++ b/src/operators/physical_path_finding_operator.cpp @@ -277,7 +277,11 @@ class GlobalBFSState { } else { is_top_down = true; } + change = top_down_cost ? true : false; + // // debug print + // string msg = "Iter " + std::to_string(iter) + " top_down_cost: " + std::to_string(top_down_cost) + " bottom_up_cost: " + std::to_string(bottom_up_cost); + // Printer::Print(msg); // clear the counters after the switch top_down_cost = 0; bottom_up_cost = 0; @@ -495,7 +499,7 @@ class PhysicalIterativeTask : public ExecutorTask { seen[i] |= next[i]; top_down_cost += v[i + 1] - v[i]; } - if (~(seen[i].all())) { + if (seen[i].all() == false) { bottom_up_cost += reverse_v[i + 1] - reverse_v[i]; } } @@ -543,7 +547,7 @@ class PhysicalIterativeTask : public ExecutorTask { seen[i] |= next[i]; top_down_cost += normal_v[i + 1] - normal_v[i]; } - if (~(seen[i].all())) { + if (seen[i].all() == false) { bottom_up_cost += v[i + 1] - v[i]; } } @@ -957,7 +961,7 @@ class PhysicalShortestPathTask : public ExecutorTask { seen[i] |= next[i]; top_down_cost += v[i + 1] - v[i]; } - if (~(seen[i].all())) { + if (seen[i].all() == false) { bottom_up_cost += reverse_v[i + 1] - reverse_v[i]; } } @@ -1015,7 +1019,7 @@ class PhysicalShortestPathTask : public ExecutorTask { seen[i] |= next[i]; top_down_cost += normal_v[i + 1] - normal_v[i]; } - if (~(seen[i].any())) { + if (seen[i].all() == false) { bottom_up_cost += v[i + 1] - v[i]; } } @@ -1443,6 +1447,8 @@ void PhysicalPathFinding::ScheduleBFSTasks(Pipeline &pipeline, Event &event, GlobalSinkState &state) { auto &gstate = state.Cast(); auto &bfs_state = gstate.global_bfs_state; + int64_t *v = (int64_t *)gstate.global_csr->v; + int64_t *reverse_v = (int64_t *)gstate.global_csr->reverse_v; // for every batch of pairs, schedule a BFS task bfs_state->Clear(); @@ -1452,7 +1458,8 @@ void PhysicalPathFinding::ScheduleBFSTasks(Pipeline &pipeline, Event &event, auto result_data = FlatVector::GetData(bfs_state->result.data[0]); auto& result_validity = FlatVector::Validity(bfs_state->result.data[0]); - std::bitset seen_mask = ~0; + std::bitset seen_mask; + seen_mask.set(); for (int64_t lane = 0; lane < LANE_LIMIT; lane++) { bfs_state->lane_to_num[lane] = -1; @@ -1477,8 +1484,19 @@ void PhysicalPathFinding::ScheduleBFSTasks(Pipeline &pipeline, Event &event, } for (int64_t i = 0; i < bfs_state->v_size; i++) { bfs_state->seen[i] = seen_mask; + + if (bfs_state->visit1[i].any()) { + bfs_state->top_down_cost += v[i + 1] - v[i]; + } + if (bfs_state->seen[i].all() == false) { + bfs_state->bottom_up_cost += reverse_v[i + 1] - reverse_v[i]; + } } + // // debug print + // string msg = "Iter 1, top down cost: " + to_string(bfs_state->top_down_cost) + ", bottom up cost: " + to_string(bfs_state->bottom_up_cost); + // Printer::Print(msg); + if (gstate.mode == "iterativelength") { auto bfs_event = make_shared(gstate, pipeline); event.InsertEvent(std::move(std::dynamic_pointer_cast(bfs_event))); From df4ff18a9ea3eaa23794a77336026194a1e42203 Mon Sep 17 00:00:00 2001 From: Pingan Ren Date: Sat, 29 Jun 2024 00:10:28 +0200 Subject: [PATCH 131/249] fix path record sync bug --- .../physical_path_finding_operator.cpp | 44 ++++++++++++++----- 1 file changed, 33 insertions(+), 11 deletions(-) diff --git a/src/operators/physical_path_finding_operator.cpp b/src/operators/physical_path_finding_operator.cpp index ee504260..b50b5441 100644 --- a/src/operators/physical_path_finding_operator.cpp +++ b/src/operators/physical_path_finding_operator.cpp @@ -168,7 +168,7 @@ class GlobalBFSState { : csr(csr_), pairs(pairs_), iter(1), v_size(v_size_), change(false), started_searches(0), total_len(0), context(context_), seen(v_size_), visit1(v_size_), visit2(v_size_), top_down_cost(0), bottom_up_cost(0), is_top_down(true), num_threads(num_threads_), task_queues(num_threads_), - task_queues_reverse(num_threads_), barrier(num_threads_), locks(v_size_), mode(mode_) { + task_queues_reverse(num_threads_), barrier(num_threads_), element_locks(v_size_), mode(mode_) { result.Initialize(context, {LogicalType::BIGINT, LogicalType::LIST(LogicalType::BIGINT)}, pairs_->size()); auto &src_data = pairs->data[0]; auto &dst_data = pairs->data[1]; @@ -177,10 +177,23 @@ class GlobalBFSState { src = FlatVector::GetData(src_data); dst = FlatVector::GetData(dst_data); - if (mode == 1) { - parents_v.resize(v_size_, std::vector(LANE_LIMIT)); - parents_e.resize(v_size_, std::vector(LANE_LIMIT)); - } + CreateTasks(); + } + + GlobalBFSState(shared_ptr csr_, shared_ptr pairs_, int64_t v_size_, + idx_t num_threads_, idx_t mode_, ClientContext &context_, bool is_path) + : csr(csr_), pairs(pairs_), iter(1), v_size(v_size_), change(false), + started_searches(0), total_len(0), context(context_), seen(v_size_), visit1(v_size_), visit2(v_size_), + parents_v(v_size_, std::vector(LANE_LIMIT)), parents_e(v_size_, std::vector(LANE_LIMIT)), + top_down_cost(0), bottom_up_cost(0), is_top_down(true), num_threads(num_threads_), task_queues(num_threads_), + task_queues_reverse(num_threads_), barrier(num_threads_), element_locks(v_size_), lane_locks(v_size_), mode(mode_) { + result.Initialize(context, {LogicalType::BIGINT, LogicalType::LIST(LogicalType::BIGINT)}, pairs_->size()); + auto &src_data = pairs->data[0]; + auto &dst_data = pairs->data[1]; + src_data.ToUnifiedFormat(pairs->size(), vdata_src); + dst_data.ToUnifiedFormat(pairs->size(), vdata_dst); + src = FlatVector::GetData(src_data); + dst = FlatVector::GetData(dst_data); CreateTasks(); } @@ -277,7 +290,7 @@ class GlobalBFSState { } else { is_top_down = true; } - + change = top_down_cost ? true : false; // // debug print // string msg = "Iter " + std::to_string(iter) + " top_down_cost: " + std::to_string(top_down_cost) + " bottom_up_cost: " + std::to_string(bottom_up_cost); @@ -324,7 +337,8 @@ class GlobalBFSState { Barrier barrier; // lock for next - mutable vector locks; + mutable vector element_locks; + mutable vector> lane_locks; idx_t mode; }; @@ -484,7 +498,7 @@ class PhysicalIterativeTask : public ExecutorTask { if (visit[i].any()) { for (auto offset = v[i]; offset < v[i + 1]; offset++) { auto n = e[offset]; - std::lock_guard lock(bfs_state->locks[n]); + std::lock_guard lock(bfs_state->element_locks[n]); next[n] |= visit[i]; } } @@ -940,9 +954,12 @@ class PhysicalShortestPathTask : public ExecutorTask { for (auto offset = v[i]; offset < v[i + 1]; offset++) { auto n = e[offset]; auto edge_id = edge_ids[offset]; - std::lock_guard lock(bfs_state->locks[n]); - next[n] |= visit[i]; + { + std::lock_guard lock(bfs_state->element_locks[n]); + next[n] |= visit[i]; + } for (auto l = 0; l < LANE_LIMIT; l++) { + std::lock_guard lock(bfs_state->lane_locks[n][l]); parents_v[n][l] = ((parents_v[n][l] == -1) && visit[i][l]) ? i : parents_v[n][l]; parents_e[n][l] = ((parents_e[n][l] == -1) && visit[i][l]) @@ -1007,6 +1024,7 @@ class PhysicalShortestPathTask : public ExecutorTask { auto edge_id = edge_ids[offset]; next[i] |= visit[n]; for (auto l = 0; l < LANE_LIMIT; l++) { + std::lock_guard lock(bfs_state->lane_locks[i][l]); parents_v[i][l] = ((parents_v[i][l] == -1) && visit[n][l]) ? n : parents_v[i][l]; parents_e[i][l] = ((parents_e[i][l] == -1) && visit[n][l]) @@ -1412,7 +1430,11 @@ PhysicalPathFinding::Finalize(Pipeline &pipeline, Event &event, idx_t num_threads = ts.NumberOfThreads(); auto& client_config = ClientConfig::GetConfig(context); idx_t mode = this->mode == "iterativelength" ? 0 : 1; - gstate.global_bfs_state = make_uniq(csr, all_pairs, csr->v_size - 2, num_threads, mode, context); + if (mode == 0) { + gstate.global_bfs_state = make_uniq(csr, all_pairs, csr->v_size - 2, num_threads, mode, context); + } else { + gstate.global_bfs_state = make_uniq(csr, all_pairs, csr->v_size - 2, num_threads, mode, context, true); + } auto const task_size = client_config.set_variables.find("experimental_path_finding_operator_task_size"); gstate.global_bfs_state->split_size = task_size != client_config.set_variables.end() ? task_size->second.GetValue() : 256; From a7447d201d708f2df16367ba2a8f0c899ccc40e0 Mon Sep 17 00:00:00 2001 From: Pingan Ren Date: Sat, 29 Jun 2024 00:15:38 +0200 Subject: [PATCH 132/249] solve locks low performance --- src/operators/physical_path_finding_operator.cpp | 9 +++------ 1 file changed, 3 insertions(+), 6 deletions(-) diff --git a/src/operators/physical_path_finding_operator.cpp b/src/operators/physical_path_finding_operator.cpp index b50b5441..5a1c1015 100644 --- a/src/operators/physical_path_finding_operator.cpp +++ b/src/operators/physical_path_finding_operator.cpp @@ -954,12 +954,9 @@ class PhysicalShortestPathTask : public ExecutorTask { for (auto offset = v[i]; offset < v[i + 1]; offset++) { auto n = e[offset]; auto edge_id = edge_ids[offset]; - { - std::lock_guard lock(bfs_state->element_locks[n]); - next[n] |= visit[i]; - } + std::lock_guard lock(bfs_state->element_locks[n]); + next[n] |= visit[i]; for (auto l = 0; l < LANE_LIMIT; l++) { - std::lock_guard lock(bfs_state->lane_locks[n][l]); parents_v[n][l] = ((parents_v[n][l] == -1) && visit[i][l]) ? i : parents_v[n][l]; parents_e[n][l] = ((parents_e[n][l] == -1) && visit[i][l]) @@ -1023,8 +1020,8 @@ class PhysicalShortestPathTask : public ExecutorTask { auto n = e[offset]; auto edge_id = edge_ids[offset]; next[i] |= visit[n]; + std::lock_guard lock(bfs_state->element_locks[i]); for (auto l = 0; l < LANE_LIMIT; l++) { - std::lock_guard lock(bfs_state->lane_locks[i][l]); parents_v[i][l] = ((parents_v[i][l] == -1) && visit[n][l]) ? n : parents_v[i][l]; parents_e[i][l] = ((parents_e[i][l] == -1) && visit[n][l]) From 863f28bf0a2b642411770a413a669b0ffcc7cd52 Mon Sep 17 00:00:00 2001 From: Pingan Ren Date: Sat, 29 Jun 2024 01:38:43 +0200 Subject: [PATCH 133/249] atomic path record --- .../physical_path_finding_operator.cpp | 77 ++++++++++++------- 1 file changed, 48 insertions(+), 29 deletions(-) diff --git a/src/operators/physical_path_finding_operator.cpp b/src/operators/physical_path_finding_operator.cpp index 5a1c1015..21a5ae11 100644 --- a/src/operators/physical_path_finding_operator.cpp +++ b/src/operators/physical_path_finding_operator.cpp @@ -18,6 +18,13 @@ namespace duckdb { +struct ve { + int64_t v; + int64_t e; + ve() : v(-1), e(-1) {} + ve(int64_t v_, int64_t e_) : v(v_), e(e_) {} +}; + PhysicalPathFinding::PhysicalPathFinding(LogicalExtensionOperator &op, unique_ptr left, unique_ptr right) @@ -181,12 +188,12 @@ class GlobalBFSState { } GlobalBFSState(shared_ptr csr_, shared_ptr pairs_, int64_t v_size_, - idx_t num_threads_, idx_t mode_, ClientContext &context_, bool is_path) + idx_t num_threads_, idx_t mode_, ClientContext &context_, bool is_path_) : csr(csr_), pairs(pairs_), iter(1), v_size(v_size_), change(false), started_searches(0), total_len(0), context(context_), seen(v_size_), visit1(v_size_), visit2(v_size_), - parents_v(v_size_, std::vector(LANE_LIMIT)), parents_e(v_size_, std::vector(LANE_LIMIT)), + parents_ve(v_size_), top_down_cost(0), bottom_up_cost(0), is_top_down(true), num_threads(num_threads_), task_queues(num_threads_), - task_queues_reverse(num_threads_), barrier(num_threads_), element_locks(v_size_), lane_locks(v_size_), mode(mode_) { + task_queues_reverse(num_threads_), barrier(num_threads_), element_locks(v_size_), mode(mode_) { result.Initialize(context, {LogicalType::BIGINT, LogicalType::LIST(LogicalType::BIGINT)}, pairs_->size()); auto &src_data = pairs->data[0]; auto &dst_data = pairs->data[1]; @@ -207,8 +214,7 @@ class GlobalBFSState { visit1[i] = 0; if (mode == 1) { for (auto j = 0; j < LANE_LIMIT; j++) { - parents_v[i][j] = -1; - parents_e[i][j] = -1; + parents_ve[i][j] = {-1, -1}; } } } @@ -319,8 +325,8 @@ class GlobalBFSState { vector> seen; vector> visit1; vector> visit2; - vector> parents_v; - vector> parents_e; + vector, LANE_LIMIT>> parents_ve; + // vector> parents_e; atomic top_down_cost; atomic bottom_up_cost; @@ -338,7 +344,6 @@ class GlobalBFSState { // lock for next mutable vector element_locks; - mutable vector> lane_locks; idx_t mode; }; @@ -931,8 +936,9 @@ class PhysicalShortestPathTask : public ExecutorTask { auto& edge_ids = state.global_csr->edge_ids; auto& top_down_cost = bfs_state->top_down_cost; auto& bottom_up_cost = bfs_state->bottom_up_cost; - auto& parents_v = bfs_state->parents_v; - auto& parents_e = bfs_state->parents_e; + // auto& parents_v = bfs_state->parents_v; + // auto& parents_e = bfs_state->parents_e; + auto& parents_ve = bfs_state->parents_ve; // clear next before each iteration for (auto i = left; i < right; i++) { @@ -957,10 +963,13 @@ class PhysicalShortestPathTask : public ExecutorTask { std::lock_guard lock(bfs_state->element_locks[n]); next[n] |= visit[i]; for (auto l = 0; l < LANE_LIMIT; l++) { - parents_v[n][l] = ((parents_v[n][l] == -1) && visit[i][l]) - ? i : parents_v[n][l]; - parents_e[n][l] = ((parents_e[n][l] == -1) && visit[i][l]) - ? edge_id : parents_e[n][l]; + // parents_v[n][l] = ((parents_v[n][l] == -1) && visit[i][l]) + // ? i : parents_v[n][l]; + // parents_e[n][l] = ((parents_e[n][l] == -1) && visit[i][l]) + // ? edge_id : parents_e[n][l]; + if (parents_ve[n][l].load().v == -1 && visit[i][l]) { + parents_ve[n][l] = {static_cast(i), edge_id}; + } } } } @@ -993,8 +1002,9 @@ class PhysicalShortestPathTask : public ExecutorTask { auto& edge_ids = state.global_csr->reverse_edge_ids; auto& top_down_cost = bfs_state->top_down_cost; auto& bottom_up_cost = bfs_state->bottom_up_cost; - auto& parents_v = bfs_state->parents_v; - auto& parents_e = bfs_state->parents_e; + // auto& parents_v = bfs_state->parents_v; + // auto& parents_e = bfs_state->parents_e; + auto& parents_ve = bfs_state->parents_ve; // clear next before each iteration for (auto i = left; i < right; i++) { @@ -1022,10 +1032,13 @@ class PhysicalShortestPathTask : public ExecutorTask { next[i] |= visit[n]; std::lock_guard lock(bfs_state->element_locks[i]); for (auto l = 0; l < LANE_LIMIT; l++) { - parents_v[i][l] = ((parents_v[i][l] == -1) && visit[n][l]) - ? n : parents_v[i][l]; - parents_e[i][l] = ((parents_e[i][l] == -1) && visit[n][l]) - ? edge_id : parents_e[i][l]; + // parents_v[i][l] = ((parents_v[i][l] == -1) && visit[n][l]) + // ? n : parents_v[i][l]; + // parents_e[i][l] = ((parents_e[i][l] == -1) && visit[n][l]) + // ? edge_id : parents_e[i][l]; + if (parents_ve[i][l].load().v == -1 && visit[n][l]) { + parents_ve[i][l] = {n, edge_id}; + } } } @@ -1095,12 +1108,15 @@ class PhysicalShortestPathTask : public ExecutorTask { std::vector output_edge; auto source_v = bfs_state->src[src_pos]; // Take the source - auto parent_vertex = - bfs_state->parents_v[bfs_state->dst[dst_pos]] - [lane]; // Take the parent vertex of the destination vertex - auto parent_edge = - bfs_state->parents_e[bfs_state->dst[dst_pos]] - [lane]; // Take the parent edge of the destination vertex + // auto parent_vertex = + // bfs_state->parents_v[bfs_state->dst[dst_pos]] + // [lane]; // Take the parent vertex of the destination vertex + // auto parent_edge = + // bfs_state->parents_e[bfs_state->dst[dst_pos]] + // [lane]; // Take the parent edge of the destination vertex + auto ve = bfs_state->parents_ve[bfs_state->dst[dst_pos]][lane].load(); + auto parent_vertex = ve.v; + auto parent_edge = ve.e; output_vector.push_back(bfs_state->dst[dst_pos]); // Add destination vertex output_vector.push_back(parent_edge); @@ -1108,13 +1124,16 @@ class PhysicalShortestPathTask : public ExecutorTask { // have reached the source vertex //! -1 is used to signify no parent if (parent_vertex == -1 || - parent_vertex == bfs_state->parents_v[parent_vertex][lane]) { + parent_vertex == bfs_state->parents_ve[parent_vertex][lane].load().v) { result_validity.SetInvalid(search_num); break; } output_vector.push_back(parent_vertex); - parent_edge = bfs_state->parents_e[parent_vertex][lane]; - parent_vertex = bfs_state->parents_v[parent_vertex][lane]; + // parent_edge = bfs_state->parents_e[parent_vertex][lane]; + // parent_vertex = bfs_state->parents_v[parent_vertex][lane]; + auto ve = bfs_state->parents_ve[parent_vertex][lane].load(); + parent_edge = ve.e; + parent_vertex = ve.v; output_vector.push_back(parent_edge); } From 5e9787151aaf228123ad7516c53d8ec7ad923b0e Mon Sep 17 00:00:00 2001 From: Pingan Ren Date: Sat, 29 Jun 2024 02:45:35 +0200 Subject: [PATCH 134/249] lock-free path record --- .../physical_path_finding_operator.cpp | 75 +++++++++++++++---- 1 file changed, 61 insertions(+), 14 deletions(-) diff --git a/src/operators/physical_path_finding_operator.cpp b/src/operators/physical_path_finding_operator.cpp index 21a5ae11..bba50bce 100644 --- a/src/operators/physical_path_finding_operator.cpp +++ b/src/operators/physical_path_finding_operator.cpp @@ -19,10 +19,59 @@ namespace duckdb { struct ve { - int64_t v; - int64_t e; - ve() : v(-1), e(-1) {} - ve(int64_t v_, int64_t e_) : v(v_), e(e_) {} + // higher 30 bits for v, lower 34 bits for e + const uint8_t v_bits = 30; + const uint8_t e_bits = 34; + uint64_t value; + const uint64_t v_mask = UINT64_MAX << e_bits; + const uint64_t e_mask = UINT64_MAX >> v_bits; + ve() : value(UINT64_MAX) {} + ve(uint64_t value) : value(value) {} + ve(int64_t v, int64_t e) { + uint64_t new_value = 0; + if (v < 0) { + new_value |= v_mask; + } else { + new_value |= (v << e_bits); + } + if (e < 0) { + new_value |= e_mask; + } else { + new_value |= e & e_mask; + } + value = new_value; + } + ve& operator=(std::initializer_list list) { + if (list.size() != 2) { + throw std::invalid_argument("Initializer list for ve must have exactly 2 elements."); + } + uint64_t new_value = 0; + auto it = list.begin(); + if (*it < 0) { + new_value |= v_mask; + } else { + new_value |= (*it << e_bits); + } + if (*(++it) < 0) { + new_value |= e_mask; + } else { + new_value |= (*it & e_mask); + } + value = new_value; + return *this; + } + int64_t GetV() { + if ((value & v_mask) == v_mask) { + return -1; + } + return static_cast(value >> e_bits); + } + int64_t GetE() { + if ((value & e_mask) == e_mask) { + return -1; + } + return static_cast(value & e_mask); + } }; PhysicalPathFinding::PhysicalPathFinding(LogicalExtensionOperator &op, @@ -325,7 +374,7 @@ class GlobalBFSState { vector> seen; vector> visit1; vector> visit2; - vector, LANE_LIMIT>> parents_ve; + vector> parents_ve; // vector> parents_e; atomic top_down_cost; @@ -967,7 +1016,7 @@ class PhysicalShortestPathTask : public ExecutorTask { // ? i : parents_v[n][l]; // parents_e[n][l] = ((parents_e[n][l] == -1) && visit[i][l]) // ? edge_id : parents_e[n][l]; - if (parents_ve[n][l].load().v == -1 && visit[i][l]) { + if (parents_ve[n][l].GetV() == -1 && visit[i][l]) { parents_ve[n][l] = {static_cast(i), edge_id}; } } @@ -1036,7 +1085,7 @@ class PhysicalShortestPathTask : public ExecutorTask { // ? n : parents_v[i][l]; // parents_e[i][l] = ((parents_e[i][l] == -1) && visit[n][l]) // ? edge_id : parents_e[i][l]; - if (parents_ve[i][l].load().v == -1 && visit[n][l]) { + if (parents_ve[i][l].GetV() == -1 && visit[n][l]) { parents_ve[i][l] = {n, edge_id}; } } @@ -1114,9 +1163,8 @@ class PhysicalShortestPathTask : public ExecutorTask { // auto parent_edge = // bfs_state->parents_e[bfs_state->dst[dst_pos]] // [lane]; // Take the parent edge of the destination vertex - auto ve = bfs_state->parents_ve[bfs_state->dst[dst_pos]][lane].load(); - auto parent_vertex = ve.v; - auto parent_edge = ve.e; + auto parent_vertex = bfs_state->parents_ve[bfs_state->dst[dst_pos]][lane].GetV(); + auto parent_edge = bfs_state->parents_ve[bfs_state->dst[dst_pos]][lane].GetE(); output_vector.push_back(bfs_state->dst[dst_pos]); // Add destination vertex output_vector.push_back(parent_edge); @@ -1124,16 +1172,15 @@ class PhysicalShortestPathTask : public ExecutorTask { // have reached the source vertex //! -1 is used to signify no parent if (parent_vertex == -1 || - parent_vertex == bfs_state->parents_ve[parent_vertex][lane].load().v) { + parent_vertex == bfs_state->parents_ve[parent_vertex][lane].GetV()) { result_validity.SetInvalid(search_num); break; } output_vector.push_back(parent_vertex); // parent_edge = bfs_state->parents_e[parent_vertex][lane]; // parent_vertex = bfs_state->parents_v[parent_vertex][lane]; - auto ve = bfs_state->parents_ve[parent_vertex][lane].load(); - parent_edge = ve.e; - parent_vertex = ve.v; + parent_edge = bfs_state->parents_ve[parent_vertex][lane].GetE(); + parent_vertex = bfs_state->parents_ve[parent_vertex][lane].GetV(); output_vector.push_back(parent_edge); } From 1f4149a1ea702282ed080eb8a3ba03811643e7f1 Mon Sep 17 00:00:00 2001 From: Pingan Ren Date: Sat, 29 Jun 2024 03:01:55 +0200 Subject: [PATCH 135/249] remove useless code --- .../physical_path_finding_operator.cpp | 42 ++++--------------- 1 file changed, 8 insertions(+), 34 deletions(-) diff --git a/src/operators/physical_path_finding_operator.cpp b/src/operators/physical_path_finding_operator.cpp index bba50bce..f31e4831 100644 --- a/src/operators/physical_path_finding_operator.cpp +++ b/src/operators/physical_path_finding_operator.cpp @@ -29,48 +29,23 @@ struct ve { ve(uint64_t value) : value(value) {} ve(int64_t v, int64_t e) { uint64_t new_value = 0; - if (v < 0) { - new_value |= v_mask; - } else { - new_value |= (v << e_bits); - } - if (e < 0) { - new_value |= e_mask; - } else { - new_value |= e & e_mask; - } + new_value |= v < 0 ? v_mask : (v << e_bits); + new_value |= e < 0 ? e_mask : (e & e_mask); value = new_value; } ve& operator=(std::initializer_list list) { - if (list.size() != 2) { - throw std::invalid_argument("Initializer list for ve must have exactly 2 elements."); - } uint64_t new_value = 0; auto it = list.begin(); - if (*it < 0) { - new_value |= v_mask; - } else { - new_value |= (*it << e_bits); - } - if (*(++it) < 0) { - new_value |= e_mask; - } else { - new_value |= (*it & e_mask); - } + new_value |= *it < 0 ? v_mask : (*it << e_bits); + new_value |= *(++it) < 0 ? e_mask : (*it & e_mask); value = new_value; return *this; } - int64_t GetV() { - if ((value & v_mask) == v_mask) { - return -1; - } - return static_cast(value >> e_bits); + inline int64_t GetV() { + return (value & v_mask) == v_mask ? -1 : static_cast(value >> e_bits); } - int64_t GetE() { - if ((value & e_mask) == e_mask) { - return -1; - } - return static_cast(value & e_mask); + inline int64_t GetE() { + return (value & e_mask) == e_mask ? -1 : static_cast(value & e_mask); } }; @@ -345,7 +320,6 @@ class GlobalBFSState { } else { is_top_down = true; } - change = top_down_cost ? true : false; // // debug print // string msg = "Iter " + std::to_string(iter) + " top_down_cost: " + std::to_string(top_down_cost) + " bottom_up_cost: " + std::to_string(bottom_up_cost); From c6c5241952c3fcc08b82d8482c8dc35db1b5f1fa Mon Sep 17 00:00:00 2001 From: Pingan Ren Date: Tue, 9 Jul 2024 16:48:00 +0200 Subject: [PATCH 136/249] final version --- src/duckpgq_extension.cpp | 4 - .../physical_path_finding_operator.hpp | 30 - .../physical_path_finding_operator.cpp | 700 ++---------------- .../path_finding/parallel_path_finding.test | 35 +- 4 files changed, 56 insertions(+), 713 deletions(-) diff --git a/src/duckpgq_extension.cpp b/src/duckpgq_extension.cpp index 5c06d178..bc758c6b 100644 --- a/src/duckpgq_extension.cpp +++ b/src/duckpgq_extension.cpp @@ -51,12 +51,8 @@ static void LoadInternal(DatabaseInstance &instance) { config.AddExtensionOption("experimental_path_finding_operator", "Enables the experimental path finding operator to be triggered", LogicalType::BOOLEAN, Value(false)); - config.AddExtensionOption("experimental_path_finding_operator_alpha", - "The ratio to switch top-down and bottom up", LogicalType::DOUBLE, Value(1.0)); config.AddExtensionOption("experimental_path_finding_operator_task_size", "Number of vertices processed per thread at a time", LogicalType::INTEGER, Value(256)); - config.AddExtensionOption("experimental_path_finding_operator_sequential", - "Sequential path finding operator", LogicalType::BOOLEAN, Value(false)); Connection con(instance); con.BeginTransaction(); diff --git a/src/include/duckpgq/operators/physical_path_finding_operator.hpp b/src/include/duckpgq/operators/physical_path_finding_operator.hpp index 1ddc6007..3833f108 100644 --- a/src/include/duckpgq/operators/physical_path_finding_operator.hpp +++ b/src/include/duckpgq/operators/physical_path_finding_operator.hpp @@ -27,17 +27,11 @@ class PhysicalPathFinding : public PhysicalComparisonJoin { if (v) { delete[] v; } - if (reverse_v) { - delete[] reverse_v; - } } atomic *v; - atomic *reverse_v; vector e; - vector reverse_e; vector edge_ids; - vector reverse_edge_ids; vector w; vector w_double; bool initialized_v = false; @@ -76,30 +70,6 @@ class PhysicalPathFinding : public PhysicalComparisonJoin { } else { result += "not initialized"; } - result += "\nReverse V: "; - if (initialized_v) { - for (idx_t i = 0; i < v_size; ++i) { - result += std::to_string(reverse_v[i]) + ' '; - } - } else { - result += "not initialized"; - } - result += "\nReverse E: "; - if (initialized_e) { - for (auto i : reverse_e) { - result += std::to_string(i) + " "; - } - } else { - result += "not initialized"; - } - result += "\nReverse Edge IDs: "; - if (initialized_e) { - for (auto i : reverse_edge_ids) { - result += std::to_string(i) + " "; - } - } else { - result += "not initialized"; - } result += "\nW: "; if (initialized_w) { for (auto i : w) { diff --git a/src/operators/physical_path_finding_operator.cpp b/src/operators/physical_path_finding_operator.cpp index f31e4831..33f59012 100644 --- a/src/operators/physical_path_finding_operator.cpp +++ b/src/operators/physical_path_finding_operator.cpp @@ -70,7 +70,6 @@ void PhysicalPathFinding::GlobalCompressedSparseRow::InitializeVertex( v_size = v_size_ + 2; try { v = new std::atomic[v_size]; - reverse_v = new std::atomic[v_size]; } catch (std::bad_alloc const &) { throw InternalException( "Unable to initialize vector of size for csr vertex table " @@ -78,7 +77,6 @@ void PhysicalPathFinding::GlobalCompressedSparseRow::InitializeVertex( } for (idx_t i = 0; i < v_size; ++i) { v[i].store(0); - reverse_v[i].store(0); } initialized_v = true; } @@ -90,16 +88,13 @@ void PhysicalPathFinding::GlobalCompressedSparseRow::InitializeEdge( } try { e.resize(e_size, 0); - reverse_e.resize(e_size, 0); edge_ids.resize(e_size, 0); - reverse_edge_ids.resize(e_size, 0); } catch (std::bad_alloc const &) { throw InternalException("Unable to initialize vector of size for csr " "edge table representation"); } for (idx_t i = 1; i < v_size; i++) { v[i] += v[i - 1]; - reverse_v[i] += reverse_v[i - 1]; } initialized_e = true; } @@ -140,26 +135,18 @@ class PathFindingLocalState : public LocalSinkState { void PathFindingLocalState::CreateCSRVertex(DataChunk &input, GlobalCompressedSparseRow &global_csr) { if (!global_csr.initialized_v) { - const auto v_size = input.data[9].GetValue(0).GetValue(); + const auto v_size = input.data[8].GetValue(0).GetValue(); global_csr.InitializeVertex(v_size); } auto result = Vector(LogicalTypeId::BIGINT); BinaryExecutor::Execute( - input.data[4], input.data[3], result, input.size(), + input.data[6], input.data[5], result, input.size(), [&](const int64_t src, const int64_t cnt) { int64_t edge_count = 0; global_csr.v[src + 2] = cnt; edge_count = edge_count + cnt; return edge_count; }); - BinaryExecutor::Execute( - input.data[7], input.data[6], result, input.size(), - [&](const int64_t dst, const int64_t cnt) { - int64_t edge_count = 0; - global_csr.reverse_v[dst + 2] = cnt; - edge_count = edge_count + cnt; - return edge_count; - }); } class Barrier { @@ -198,8 +185,7 @@ class GlobalBFSState { idx_t num_threads_, idx_t mode_, ClientContext &context_) : csr(csr_), pairs(pairs_), iter(1), v_size(v_size_), change(false), started_searches(0), total_len(0), context(context_), seen(v_size_), visit1(v_size_), visit2(v_size_), - top_down_cost(0), bottom_up_cost(0), is_top_down(true), num_threads(num_threads_), task_queues(num_threads_), - task_queues_reverse(num_threads_), barrier(num_threads_), element_locks(v_size_), mode(mode_) { + num_threads(num_threads_), task_queues(num_threads_), barrier(num_threads_), element_locks(v_size_), mode(mode_) { result.Initialize(context, {LogicalType::BIGINT, LogicalType::LIST(LogicalType::BIGINT)}, pairs_->size()); auto &src_data = pairs->data[0]; auto &dst_data = pairs->data[1]; @@ -215,9 +201,8 @@ class GlobalBFSState { idx_t num_threads_, idx_t mode_, ClientContext &context_, bool is_path_) : csr(csr_), pairs(pairs_), iter(1), v_size(v_size_), change(false), started_searches(0), total_len(0), context(context_), seen(v_size_), visit1(v_size_), visit2(v_size_), - parents_ve(v_size_), - top_down_cost(0), bottom_up_cost(0), is_top_down(true), num_threads(num_threads_), task_queues(num_threads_), - task_queues_reverse(num_threads_), barrier(num_threads_), element_locks(v_size_), mode(mode_) { + parents_ve(v_size_), num_threads(num_threads_), task_queues(num_threads_), barrier(num_threads_), + element_locks(v_size_), mode(mode_) { result.Initialize(context, {LogicalType::BIGINT, LogicalType::LIST(LogicalType::BIGINT)}, pairs_->size()); auto &src_data = pairs->data[0]; auto &dst_data = pairs->data[1]; @@ -247,50 +232,41 @@ class GlobalBFSState { void CreateTasks() { // workerTasks[workerId] = [task1, task2, ...] - auto queues = {&task_queues, &task_queues_reverse}; - is_top_down = true; - for (auto& queue : queues) { - vector>> worker_tasks(num_threads); - auto cur_worker = 0; - int64_t *v = is_top_down ? (int64_t*)csr->v : (int64_t*)csr->reverse_v; - int64_t current_task_edges = 0; - idx_t current_task_start = 0; - for (idx_t i = 0; i < (idx_t)v_size; i++) { - auto vertex_edges = v[i + 1] - v[i]; - if (current_task_edges + vertex_edges > split_size && i != current_task_start) { - auto worker_id = cur_worker % num_threads; - pair range = {current_task_start, i}; - worker_tasks[worker_id].push_back(range); - current_task_start = i; - current_task_edges = 0; - cur_worker++; - } - current_task_edges += vertex_edges; - } - if (current_task_start < (idx_t)v_size) { + auto queues = &task_queues; + vector>> worker_tasks(num_threads); + auto cur_worker = 0; + int64_t *v = (int64_t*)csr->v; + int64_t current_task_edges = 0; + idx_t current_task_start = 0; + for (idx_t i = 0; i < (idx_t)v_size; i++) { + auto vertex_edges = v[i + 1] - v[i]; + if (current_task_edges + vertex_edges > split_size && i != current_task_start) { auto worker_id = cur_worker % num_threads; - pair range = {current_task_start, v_size}; + pair range = {current_task_start, i}; worker_tasks[worker_id].push_back(range); + current_task_start = i; + current_task_edges = 0; + cur_worker++; } - for (idx_t worker_id = 0; worker_id < num_threads; worker_id++) { - queue->at(worker_id).first.store(0); - queue->at(worker_id).second = worker_tasks[worker_id]; - } - is_top_down = false; + current_task_edges += vertex_edges; + } + if (current_task_start < (idx_t)v_size) { + auto worker_id = cur_worker % num_threads; + pair range = {current_task_start, v_size}; + worker_tasks[worker_id].push_back(range); + } + for (idx_t worker_id = 0; worker_id < num_threads; worker_id++) { + queues->at(worker_id).first.store(0); + queues->at(worker_id).second = worker_tasks[worker_id]; } - is_top_down = true; } void InitTask(idx_t worker_id) { - if (is_top_down) { - task_queues[worker_id].first.store(0); - } else { - task_queues_reverse[worker_id].first.store(0); - } + task_queues[worker_id].first.store(0); } pair FetchTask(idx_t worker_id) { - auto& task_queue = is_top_down ? task_queues : task_queues_reverse; + auto& task_queue = task_queues; idx_t offset = 0; do { auto worker_idx = (worker_id + offset) % task_queue.size(); @@ -312,23 +288,6 @@ class GlobalBFSState { return {left, right}; } - void DirectionSwitch() { - // Determine the switch of algorithms - // debug print - if (top_down_cost * alpha > bottom_up_cost) { - is_top_down = false; - } else { - is_top_down = true; - } - change = top_down_cost ? true : false; - // // debug print - // string msg = "Iter " + std::to_string(iter) + " top_down_cost: " + std::to_string(top_down_cost) + " bottom_up_cost: " + std::to_string(bottom_up_cost); - // Printer::Print(msg); - // clear the counters after the switch - top_down_cost = 0; - bottom_up_cost = 0; - } - public: shared_ptr csr; shared_ptr pairs; @@ -349,18 +308,11 @@ class GlobalBFSState { vector> visit1; vector> visit2; vector> parents_ve; - // vector> parents_e; - - atomic top_down_cost; - atomic bottom_up_cost; - double alpha = 1; - atomic is_top_down; idx_t num_threads; // task_queues[workerId] = {curTaskIx, queuedTasks} // queuedTasks[curTaskIx] = {start, end} vector, vector>>> task_queues; - vector, vector>>> task_queues_reverse; int64_t split_size = 256; Barrier barrier; @@ -470,17 +422,12 @@ class PhysicalIterativeTask : public ExecutorTask { do { bfs_state->InitTask(worker_id); - if (bfs_state->is_top_down) { - IterativeLengthTopDown(); - } else { - IterativeLengthBottomUp(); - } + IterativeLength(); barrier.Wait(); if (worker_id == 0) { ReachDetect(); - bfs_state->DirectionSwitch(); } barrier.Wait(); } while (change); @@ -494,18 +441,16 @@ class PhysicalIterativeTask : public ExecutorTask { } private: - void IterativeLengthTopDown() { + void IterativeLength() { auto& bfs_state = state.global_bfs_state; auto& seen = bfs_state->seen; auto& visit = bfs_state->iter & 1 ? bfs_state->visit1 : bfs_state->visit2; auto& next = bfs_state->iter & 1 ? bfs_state->visit2 : bfs_state->visit1; auto& barrier = bfs_state->barrier; int64_t *v = (int64_t *)state.global_csr->v; - int64_t *reverse_v = (int64_t *)state.global_csr->reverse_v; vector &e = state.global_csr->e; - auto& top_down_cost = bfs_state->top_down_cost; - auto& bottom_up_cost = bfs_state->bottom_up_cost; auto& lane_to_num = bfs_state->lane_to_num; + auto& change = bfs_state->change; // clear next before each iteration for (auto i = left; i < right; i++) { @@ -533,65 +478,14 @@ class PhysicalIterativeTask : public ExecutorTask { } } + change = false; barrier.Wait(); for (auto i = left; i < right; i++) { if (next[i].any()) { next[i] &= ~seen[i]; seen[i] |= next[i]; - top_down_cost += v[i + 1] - v[i]; - } - if (seen[i].all() == false) { - bottom_up_cost += reverse_v[i + 1] - reverse_v[i]; - } - } - } - - void IterativeLengthBottomUp() { - auto& bfs_state = state.global_bfs_state; - auto& seen = bfs_state->seen; - auto& visit = bfs_state->iter & 1 ? bfs_state->visit1 : bfs_state->visit2; - auto& next = bfs_state->iter & 1 ? bfs_state->visit2 : bfs_state->visit1; - auto& barrier = bfs_state->barrier; - int64_t *v = (int64_t *)state.global_csr->reverse_v; - int64_t *normal_v = (int64_t *)state.global_csr->v; - vector &e = state.global_csr->reverse_e; - auto& top_down_cost = bfs_state->top_down_cost; - auto& bottom_up_cost = bfs_state->bottom_up_cost; - - // clear next before each iteration - for (auto i = left; i < right; i++) { - next[i] = 0; - } - - barrier.Wait(); - - while (true) { - auto task = bfs_state->FetchTask(worker_id); - if (task.first == task.second) { - break; - } - auto start = task.first; - auto end = task.second; - - for (auto i = start; i < end; i++) { - if (seen[i].all()) { - continue; - } - - for (auto offset = v[i]; offset < v[i + 1]; offset++) { - auto n = e[offset]; - next[i] |= visit[n]; - } - - if (next[i].any()) { - next[i] &= ~seen[i]; - seen[i] |= next[i]; - top_down_cost += normal_v[i + 1] - normal_v[i]; - } - if (seen[i].all() == false) { - bottom_up_cost += v[i + 1] - v[i]; - } + change |= next[i].any(); } } } @@ -665,25 +559,17 @@ class PhysicalCSREdgeCreationTask : public ExecutorTask { } } if (!global_csr->initialized_e) { - const auto e_size = input.data[8].GetValue(0).GetValue(); + const auto e_size = input.data[7].GetValue(0).GetValue(); global_csr->InitializeEdge(e_size); } TernaryExecutor::Execute( - input.data[4], input.data[7], input.data[2], result, input.size(), + input.data[6], input.data[4], input.data[2], result, input.size(), [&](int64_t src, int64_t dst, int64_t edge_id) { const auto pos = ++global_csr->v[src + 1]; global_csr->e[static_cast(pos) - 1] = dst; global_csr->edge_ids[static_cast(pos) - 1] = edge_id; return 1; }); - TernaryExecutor::Execute( - input.data[7], input.data[4], input.data[2], result, input.size(), - [&](int64_t dst, int64_t src, int64_t edge_id) { - const auto pos = ++global_csr->reverse_v[dst + 1]; - global_csr->reverse_e[static_cast(pos) - 1] = src; - global_csr->reverse_edge_ids[static_cast(pos) - 1] = edge_id; - return 1; - }); } event->FinishTask(); return TaskExecutionResult::TASK_FINISHED; @@ -722,8 +608,6 @@ class CSREdgeCreationEvent : public BasePipelineEvent { auto &gstate = this->gstate; auto &global_csr = gstate.global_csr; global_csr->is_ready = true; - // debug print - // global_csr->Print(); } }; @@ -757,154 +641,6 @@ class ParallelIterativeEvent : public BasePipelineEvent { } }; -class SequentialIterativeEvent : public BasePipelineEvent { -public: - SequentialIterativeEvent(PathFindingGlobalState &gstate_p, Pipeline &pipeline_p) - : BasePipelineEvent(pipeline_p), gstate(gstate_p) { - } - - PathFindingGlobalState &gstate; - -private: - std::chrono::high_resolution_clock::time_point start_time; - -public: - void Schedule() override { - auto &bfs_state = gstate.global_bfs_state; - - auto& pairs = *bfs_state->pairs; - auto& result = bfs_state->result.data[0]; - start_time = std::chrono::high_resolution_clock::now(); - IterativeLengthFunction(gstate.global_csr, pairs, result); - auto end_time = std::chrono::high_resolution_clock::now(); - auto duration = std::chrono::duration_cast(end_time - start_time); - } - -private: - static bool IterativeLength(int64_t v_size, int64_t *v, vector &e, - vector> &seen, - vector> &visit, - vector> &next) { - bool change = false; - for (auto i = 0; i < v_size; i++) { - next[i] = 0; - } - for (auto i = 0; i < v_size; i++) { - if (visit[i].any()) { - for (auto offset = v[i]; offset < v[i + 1]; offset++) { - auto n = e[offset]; - next[n] = next[n] | visit[i]; - } - } - } - for (auto i = 0; i < v_size; i++) { - next[i] = next[i] & ~seen[i]; - seen[i] = seen[i] | next[i]; - change |= next[i].any(); - } - return change; - } - - static void IterativeLengthFunction(const shared_ptr &csr, - DataChunk &pairs, Vector &result) { - int64_t v_size = csr->v_size; - int64_t *v = (int64_t *)csr->v; - vector &e = csr->e; - - // get src and dst vectors for searches - auto &src = pairs.data[0]; - auto &dst = pairs.data[1]; - UnifiedVectorFormat vdata_src; - UnifiedVectorFormat vdata_dst; - src.ToUnifiedFormat(pairs.size(), vdata_src); - dst.ToUnifiedFormat(pairs.size(), vdata_dst); - - auto src_data = FlatVector::GetData(src); - auto dst_data = FlatVector::GetData(dst); - - ValidityMask &result_validity = FlatVector::Validity(result); - - // create result vector - result.SetVectorType(VectorType::FLAT_VECTOR); - auto result_data = FlatVector::GetData(result); - - // create temp SIMD arrays - vector> seen(v_size); - vector> visit1(v_size); - vector> visit2(v_size); - - // maps lane to search number - short lane_to_num[LANE_LIMIT]; - for (int64_t lane = 0; lane < LANE_LIMIT; lane++) { - lane_to_num[lane] = -1; // inactive - } - - idx_t started_searches = 0; - while (started_searches < pairs.size()) { - - // empty visit vectors - for (auto i = 0; i < v_size; i++) { - seen[i] = 0; - visit1[i] = 0; - } - - // add search jobs to free lanes - uint64_t active = 0; - for (int64_t lane = 0; lane < LANE_LIMIT; lane++) { - lane_to_num[lane] = -1; - while (started_searches < pairs.size()) { - int64_t search_num = started_searches++; - int64_t src_pos = vdata_src.sel->get_index(search_num); - int64_t dst_pos = vdata_dst.sel->get_index(search_num); - if (!vdata_src.validity.RowIsValid(src_pos)) { - result_validity.SetInvalid(search_num); - result_data[search_num] = (uint64_t)-1; /* no path */ - } else if (src_data[src_pos] == dst_data[dst_pos]) { - result_data[search_num] = - (uint64_t)0; // path of length 0 does not require a search - } else { - visit1[src_data[src_pos]][lane] = true; - lane_to_num[lane] = search_num; // active lane - active++; - break; - } - } - } - - // make passes while a lane is still active - for (int64_t iter = 1; active; iter++) { - if (!IterativeLength(v_size, v, e, seen, (iter & 1) ? visit1 : visit2, - (iter & 1) ? visit2 : visit1)) { - break; - } - // detect lanes that finished - for (int64_t lane = 0; lane < LANE_LIMIT; lane++) { - int64_t search_num = lane_to_num[lane]; - if (search_num >= 0) { // active lane - int64_t dst_pos = vdata_dst.sel->get_index(search_num); - if (seen[dst_data[dst_pos]][lane]) { - result_data[search_num] = - iter; /* found at iter => iter = path length */ - lane_to_num[lane] = -1; // mark inactive - active--; - } - } - } - } - - // no changes anymore: any still active searches have no path - for (int64_t lane = 0; lane < LANE_LIMIT; lane++) { - int64_t search_num = lane_to_num[lane]; - if (search_num >= 0) { // active lane - result_validity.SetInvalid(search_num); - result_data[search_num] = (int64_t)-1; /* no path */ - lane_to_num[lane] = -1; // mark inactive - } - } - } - } -}; - class PhysicalShortestPathTask : public ExecutorTask { public: PhysicalShortestPathTask(shared_ptr event_p, ClientContext &context, PathFindingGlobalState &state, idx_t worker_id) @@ -923,16 +659,11 @@ class PhysicalShortestPathTask : public ExecutorTask { do { bfs_state->InitTask(worker_id); - if (bfs_state->is_top_down) { - IterativePathTopDown(); - } else { - IterativePathBottomUp(); - } + IterativePath(); barrier.Wait(); if (worker_id == 0) { ReachDetect(); - bfs_state->DirectionSwitch(); } barrier.Wait(); @@ -947,21 +678,17 @@ class PhysicalShortestPathTask : public ExecutorTask { } private: - void IterativePathTopDown() { + void IterativePath() { auto& bfs_state = state.global_bfs_state; auto& seen = bfs_state->seen; auto& visit = bfs_state->iter & 1 ? bfs_state->visit1 : bfs_state->visit2; auto& next = bfs_state->iter & 1 ? bfs_state->visit2 : bfs_state->visit1; auto& barrier = bfs_state->barrier; int64_t *v = (int64_t *)state.global_csr->v; - int64_t *reverse_v = (int64_t *)state.global_csr->reverse_v; vector &e = state.global_csr->e; auto& edge_ids = state.global_csr->edge_ids; - auto& top_down_cost = bfs_state->top_down_cost; - auto& bottom_up_cost = bfs_state->bottom_up_cost; - // auto& parents_v = bfs_state->parents_v; - // auto& parents_e = bfs_state->parents_e; auto& parents_ve = bfs_state->parents_ve; + auto& change = bfs_state->change; // clear next before each iteration for (auto i = left; i < right; i++) { @@ -983,13 +710,11 @@ class PhysicalShortestPathTask : public ExecutorTask { for (auto offset = v[i]; offset < v[i + 1]; offset++) { auto n = e[offset]; auto edge_id = edge_ids[offset]; - std::lock_guard lock(bfs_state->element_locks[n]); - next[n] |= visit[i]; + { + std::lock_guard lock(bfs_state->element_locks[n]); + next[n] |= visit[i]; + } for (auto l = 0; l < LANE_LIMIT; l++) { - // parents_v[n][l] = ((parents_v[n][l] == -1) && visit[i][l]) - // ? i : parents_v[n][l]; - // parents_e[n][l] = ((parents_e[n][l] == -1) && visit[i][l]) - // ? edge_id : parents_e[n][l]; if (parents_ve[n][l].GetV() == -1 && visit[i][l]) { parents_ve[n][l] = {static_cast(i), edge_id}; } @@ -1005,74 +730,7 @@ class PhysicalShortestPathTask : public ExecutorTask { if (next[i].any()) { next[i] &= ~seen[i]; seen[i] |= next[i]; - top_down_cost += v[i + 1] - v[i]; - } - if (seen[i].all() == false) { - bottom_up_cost += reverse_v[i + 1] - reverse_v[i]; - } - } - } - - void IterativePathBottomUp() { - auto& bfs_state = state.global_bfs_state; - auto& seen = bfs_state->seen; - auto& visit = bfs_state->iter & 1 ? bfs_state->visit1 : bfs_state->visit2; - auto& next = bfs_state->iter & 1 ? bfs_state->visit2 : bfs_state->visit1; - auto& barrier = bfs_state->barrier; - int64_t *v = (int64_t *)state.global_csr->reverse_v; - int64_t *normal_v = (int64_t *)state.global_csr->v; - vector &e = state.global_csr->reverse_e; - auto& edge_ids = state.global_csr->reverse_edge_ids; - auto& top_down_cost = bfs_state->top_down_cost; - auto& bottom_up_cost = bfs_state->bottom_up_cost; - // auto& parents_v = bfs_state->parents_v; - // auto& parents_e = bfs_state->parents_e; - auto& parents_ve = bfs_state->parents_ve; - - // clear next before each iteration - for (auto i = left; i < right; i++) { - next[i] = 0; - } - - barrier.Wait(); - - while (true) { - auto task = bfs_state->FetchTask(worker_id); - if (task.first == task.second) { - break; - } - auto start = task.first; - auto end = task.second; - - for (auto i = start; i < end; i++) { - if (seen[i].all()) { - continue; - } - - for (auto offset = v[i]; offset < v[i + 1]; offset++) { - auto n = e[offset]; - auto edge_id = edge_ids[offset]; - next[i] |= visit[n]; - std::lock_guard lock(bfs_state->element_locks[i]); - for (auto l = 0; l < LANE_LIMIT; l++) { - // parents_v[i][l] = ((parents_v[i][l] == -1) && visit[n][l]) - // ? n : parents_v[i][l]; - // parents_e[i][l] = ((parents_e[i][l] == -1) && visit[n][l]) - // ? edge_id : parents_e[i][l]; - if (parents_ve[i][l].GetV() == -1 && visit[n][l]) { - parents_ve[i][l] = {n, edge_id}; - } - } - } - - if (next[i].any()) { - next[i] &= ~seen[i]; - seen[i] |= next[i]; - top_down_cost += normal_v[i + 1] - normal_v[i]; - } - if (seen[i].all() == false) { - bottom_up_cost += v[i + 1] - v[i]; - } + change |= next[i].any(); } } } @@ -1080,8 +738,6 @@ class PhysicalShortestPathTask : public ExecutorTask { void ReachDetect() { auto &bfs_state = state.global_bfs_state; auto &change = bfs_state->change; - auto &top_down_cost = bfs_state->top_down_cost; - auto &bottom_up_cost = bfs_state->bottom_up_cost; // detect lanes that finished for (int64_t lane = 0; lane < LANE_LIMIT; lane++) { int64_t search_num = bfs_state->lane_to_num[lane]; @@ -1131,12 +787,6 @@ class PhysicalShortestPathTask : public ExecutorTask { std::vector output_edge; auto source_v = bfs_state->src[src_pos]; // Take the source - // auto parent_vertex = - // bfs_state->parents_v[bfs_state->dst[dst_pos]] - // [lane]; // Take the parent vertex of the destination vertex - // auto parent_edge = - // bfs_state->parents_e[bfs_state->dst[dst_pos]] - // [lane]; // Take the parent edge of the destination vertex auto parent_vertex = bfs_state->parents_ve[bfs_state->dst[dst_pos]][lane].GetV(); auto parent_edge = bfs_state->parents_ve[bfs_state->dst[dst_pos]][lane].GetE(); @@ -1151,8 +801,6 @@ class PhysicalShortestPathTask : public ExecutorTask { break; } output_vector.push_back(parent_vertex); - // parent_edge = bfs_state->parents_e[parent_vertex][lane]; - // parent_vertex = bfs_state->parents_v[parent_vertex][lane]; parent_edge = bfs_state->parents_ve[parent_vertex][lane].GetE(); parent_vertex = bfs_state->parents_ve[parent_vertex][lane].GetV(); output_vector.push_back(parent_edge); @@ -1215,229 +863,6 @@ class ParallelShortestPathEvent : public BasePipelineEvent { } }; -class SequentialShortestPathEvent : public BasePipelineEvent { -public: - SequentialShortestPathEvent(PathFindingGlobalState &gstate_p, Pipeline &pipeline_p) - : BasePipelineEvent(pipeline_p), gstate(gstate_p) { - } - - PathFindingGlobalState &gstate; - -public: - void Schedule() override { - auto &bfs_state = gstate.global_bfs_state; - auto &pairs = *bfs_state->pairs; - auto &result = bfs_state->result.data[1]; - auto start_time = std::chrono::high_resolution_clock::now(); - ShortestPathFunction(gstate.global_csr, pairs, result); - auto end_time = std::chrono::high_resolution_clock::now(); - auto duration = std::chrono::duration_cast(end_time - start_time); - } - -private: - static bool IterativeLength(int64_t v_size, int64_t *V, vector &E, - vector &edge_ids, - vector> &parents_v, - vector> &parents_e, - vector> &seen, - vector> &visit, - vector> &next) { - bool change = false; - for (auto v = 0; v < v_size; v++) { - next[v] = 0; - } - //! Keep track of edge id through which the node was reached - for (auto v = 0; v < v_size; v++) { - if (visit[v].any()) { - for (auto e = V[v]; e < V[v + 1]; e++) { - auto n = E[e]; - auto edge_id = edge_ids[e]; - next[n] = next[n] | visit[v]; - for (auto l = 0; l < LANE_LIMIT; l++) { - parents_v[n][l] = - ((parents_v[n][l] == -1) && visit[v][l]) ? v : parents_v[n][l]; - parents_e[n][l] = ((parents_e[n][l] == -1) && visit[v][l]) - ? edge_id - : parents_e[n][l]; - } - } - } - } - - for (auto v = 0; v < v_size; v++) { - next[v] = next[v] & ~seen[v]; - seen[v] = seen[v] | next[v]; - change |= next[v].any(); - } - return change; - } - - static void ShortestPathFunction(const shared_ptr &csr, - DataChunk &pairs, Vector &result) { - int64_t v_size = csr->v_size; - int64_t *v = (int64_t *)csr->v; - vector &e = csr->e; - vector &edge_ids = csr->edge_ids; - - auto &src = pairs.data[0]; - auto &dst = pairs.data[1]; - UnifiedVectorFormat vdata_src; - UnifiedVectorFormat vdata_dst; - src.ToUnifiedFormat(pairs.size(), vdata_src); - dst.ToUnifiedFormat(pairs.size(), vdata_dst); - - auto src_data = (int64_t *)vdata_src.data; - auto dst_data = (int64_t *)vdata_dst.data; - - result.SetVectorType(VectorType::FLAT_VECTOR); - auto result_data = FlatVector::GetData(result); - ValidityMask &result_validity = FlatVector::Validity(result); - - // create temp SIMD arrays - vector> seen(v_size); - vector> visit1(v_size); - vector> visit2(v_size); - vector> parents_v(v_size, - std::vector(LANE_LIMIT, -1)); - vector> parents_e(v_size, - std::vector(LANE_LIMIT, -1)); - - // maps lane to search number - int16_t lane_to_num[LANE_LIMIT]; - for (int64_t lane = 0; lane < LANE_LIMIT; lane++) { - lane_to_num[lane] = -1; // inactive - } - int64_t total_len = 0; - - idx_t started_searches = 0; - while (started_searches < pairs.size()) { - - // empty visit vectors - for (auto i = 0; i < v_size; i++) { - seen[i] = 0; - visit1[i] = 0; - for (auto j = 0; j < LANE_LIMIT; j++) { - parents_v[i][j] = -1; - parents_e[i][j] = -1; - } - } - - // add search jobs to free lanes - uint64_t active = 0; - for (int64_t lane = 0; lane < LANE_LIMIT; lane++) { - lane_to_num[lane] = -1; - while (started_searches < pairs.size()) { - int64_t search_num = started_searches++; - int64_t src_pos = vdata_src.sel->get_index(search_num); - if (!vdata_src.validity.RowIsValid(src_pos)) { - result_validity.SetInvalid(search_num); - } else { - visit1[src_data[src_pos]][lane] = true; - parents_v[src_data[src_pos]][lane] = - src_data[src_pos]; // Mark source with source id - parents_e[src_data[src_pos]][lane] = - -2; // Mark the source with -2, there is no incoming edge for the - // source. - lane_to_num[lane] = search_num; // active lane - active++; - break; - } - } - } - - //! make passes while a lane is still active - for (int64_t iter = 1; active; iter++) { - //! Perform one step of bfs exploration - if (!IterativeLength(v_size, v, e, edge_ids, parents_v, parents_e, seen, - (iter & 1) ? visit1 : visit2, - (iter & 1) ? visit2 : visit1)) { - break; - } - int64_t finished_searches = 0; - // detect lanes that finished - for (int64_t lane = 0; lane < LANE_LIMIT; lane++) { - int64_t search_num = lane_to_num[lane]; - if (search_num >= 0) { // active lane - //! Check if dst for a source has been seen - int64_t dst_pos = vdata_dst.sel->get_index(search_num); - if (seen[dst_data[dst_pos]][lane]) { - finished_searches++; - } - } - } - if (finished_searches == LANE_LIMIT) { - break; - } - } - //! Reconstruct the paths - for (int64_t lane = 0; lane < LANE_LIMIT; lane++) { - int64_t search_num = lane_to_num[lane]; - if (search_num == -1) { // empty lanes - continue; - } - - //! Searches that have stopped have found a path - int64_t src_pos = vdata_src.sel->get_index(search_num); - int64_t dst_pos = vdata_dst.sel->get_index(search_num); - if (src_data[src_pos] == dst_data[dst_pos]) { // Source == destination - unique_ptr output = - make_uniq(LogicalType::LIST(LogicalType::BIGINT)); - ListVector::PushBack(*output, src_data[src_pos]); - ListVector::Append(result, ListVector::GetEntry(*output), - ListVector::GetListSize(*output)); - result_data[search_num].length = ListVector::GetListSize(*output); - result_data[search_num].offset = total_len; - total_len += result_data[search_num].length; - continue; - } - std::vector output_vector; - std::vector output_edge; - auto source_v = src_data[src_pos]; // Take the source - - auto parent_vertex = - parents_v[dst_data[dst_pos]] - [lane]; // Take the parent vertex of the destination vertex - auto parent_edge = - parents_e[dst_data[dst_pos]] - [lane]; // Take the parent edge of the destination vertex - - output_vector.push_back(dst_data[dst_pos]); // Add destination vertex - output_vector.push_back(parent_edge); - while (parent_vertex != source_v) { // Continue adding vertices until we - // have reached the source vertex - //! -1 is used to signify no parent - if (parent_vertex == -1 || - parent_vertex == parents_v[parent_vertex][lane]) { - result_validity.SetInvalid(search_num); - break; - } - output_vector.push_back(parent_vertex); - parent_edge = parents_e[parent_vertex][lane]; - parent_vertex = parents_v[parent_vertex][lane]; - output_vector.push_back(parent_edge); - } - - if (!result_validity.RowIsValid(search_num)) { - continue; - } - output_vector.push_back(source_v); - std::reverse(output_vector.begin(), output_vector.end()); - auto output = make_uniq(LogicalType::LIST(LogicalType::BIGINT)); - for (auto val : output_vector) { - Value value_to_insert = val; - ListVector::PushBack(*output, value_to_insert); - } - - result_data[search_num].length = ListVector::GetListSize(*output); - result_data[search_num].offset = total_len; - ListVector::Append(result, ListVector::GetEntry(*output), - ListVector::GetListSize(*output)); - total_len += result_data[search_num].length; - } - } - } -}; - SinkFinalizeType PhysicalPathFinding::Finalize(Pipeline &pipeline, Event &event, ClientContext &context, @@ -1459,9 +884,6 @@ PhysicalPathFinding::Finalize(Pipeline &pipeline, Event &event, while (global_tasks->Scan(scan_state, pairs)) { all_pairs->Append(pairs, true); } - // debug print - // all_pairs->Print(); - auto &ts = TaskScheduler::GetScheduler(context); idx_t num_threads = ts.NumberOfThreads(); @@ -1476,23 +898,9 @@ PhysicalPathFinding::Finalize(Pipeline &pipeline, Event &event, auto const task_size = client_config.set_variables.find("experimental_path_finding_operator_task_size"); gstate.global_bfs_state->split_size = task_size != client_config.set_variables.end() ? task_size->second.GetValue() : 256; - auto const alpha = client_config.set_variables.find("experimental_path_finding_operator_alpha"); - gstate.global_bfs_state->alpha = alpha != client_config.set_variables.end() ? alpha->second.GetValue() : 1; - - auto const sequential = client_config.set_variables.find("experimental_path_finding_operator_sequential"); - if (sequential != client_config.set_variables.end() && sequential->second.GetValue()) { - if (gstate.mode == "iterativelength") { - auto bfs_event = make_shared(gstate, pipeline); - event.InsertEvent(std::move(std::dynamic_pointer_cast(bfs_event))); - } else if (gstate.mode == "shortestpath") { - auto bfs_event = make_shared(gstate, pipeline); - event.InsertEvent(std::move(std::dynamic_pointer_cast(bfs_event))); - } - } else { - // Schedule the first round of BFS tasks - if (all_pairs->size() > 0) { - ScheduleBFSTasks(pipeline, event, gstate); - } + // Schedule the first round of BFS tasks + if (all_pairs->size() > 0) { + ScheduleBFSTasks(pipeline, event, gstate); } } @@ -1507,7 +915,6 @@ void PhysicalPathFinding::ScheduleBFSTasks(Pipeline &pipeline, Event &event, auto &gstate = state.Cast(); auto &bfs_state = gstate.global_bfs_state; int64_t *v = (int64_t *)gstate.global_csr->v; - int64_t *reverse_v = (int64_t *)gstate.global_csr->reverse_v; // for every batch of pairs, schedule a BFS task bfs_state->Clear(); @@ -1543,19 +950,8 @@ void PhysicalPathFinding::ScheduleBFSTasks(Pipeline &pipeline, Event &event, } for (int64_t i = 0; i < bfs_state->v_size; i++) { bfs_state->seen[i] = seen_mask; - - if (bfs_state->visit1[i].any()) { - bfs_state->top_down_cost += v[i + 1] - v[i]; - } - if (bfs_state->seen[i].all() == false) { - bfs_state->bottom_up_cost += reverse_v[i + 1] - reverse_v[i]; - } } - // // debug print - // string msg = "Iter 1, top down cost: " + to_string(bfs_state->top_down_cost) + ", bottom up cost: " + to_string(bfs_state->bottom_up_cost); - // Printer::Print(msg); - if (gstate.mode == "iterativelength") { auto bfs_event = make_shared(gstate, pipeline); event.InsertEvent(std::move(std::dynamic_pointer_cast(bfs_event))); diff --git a/test/sql/path_finding/parallel_path_finding.test b/test/sql/path_finding/parallel_path_finding.test index af895f0b..3ff5c9ef 100644 --- a/test/sql/path_finding/parallel_path_finding.test +++ b/test/sql/path_finding/parallel_path_finding.test @@ -9,7 +9,6 @@ require duckpgq statement ok set experimental_path_finding_operator=true; -set experimental_path_finding_operator_sequential=false; statement ok CREATE TABLE pair(src BIGINT, dst BIGINT); INSERT INTO pair(src, dst) VALUES (0, 1), (1, 2), (2,0); @@ -56,25 +55,19 @@ query III SELECT *, shortestpath(src, dst, 'pair') as path FROM pair AS p WHERE p.src BETWEEN (SELECT CREATE_CSR_EDGE( - (SELECT count(a.id) as v_size FROM Student a), - (SELECT count(k.src) as e_size from know k), + (SELECT count(a.id) as v_size FROM Student a), + (SELECT count(k.src) as e_size from know k), a.rowid, c.rowid, k.rowid, - t.cnt, - t_r.cnt_r) FROM Know k + t.cnt) FROM Know k JOIN student a on a.id = k.src JOIN student c on c.id = k.dst JOIN (SELECT count(k.src) cnt, a.rowid as a_rowid FROM student a LEFT JOIN know k ON k.src = a.id GROUP BY a.rowid) t - ON t.a_rowid = a.rowid - JOIN (SELECT count(k.dst) cnt_r, c.rowid as c_rowid - FROM student c - LEFT JOIN know k ON k.dst = c.id - GROUP BY c.rowid) t_r - ON t_r.c_rowid = c.rowid) AND p.dst; + ON t.a_rowid = a.rowid) AND p.dst; ---- 0 1 NULL 1 2 [1, 3, 0, 1, 2] @@ -115,20 +108,14 @@ WHERE p.src BETWEEN (SELECT CREATE_CSR_EDGE( a.rowid, c.rowid, k.rowid, - t.cnt, - t_r.cnt_r) FROM Person_knows_Person k + t.cnt) FROM Person_knows_Person k JOIN Person a on a.id = k.Person1Id JOIN Person c on c.id = k.Person2Id JOIN (SELECT count(k.Person1Id) cnt, a.rowid as a_rowid FROM Person a LEFT JOIN Person_knows_Person k ON k.Person1Id = a.id GROUP BY a.rowid) t - ON t.a_rowid = a.rowid - JOIN (SELECT count(k.Person2Id) cnt_r, c.rowid as c_rowid - FROM Person c - LEFT JOIN Person_knows_Person k ON k.Person2Id = c.id - GROUP BY c.rowid) t_r - ON t_r.c_rowid = c.rowid) AND p.dst; + ON t.a_rowid = a.rowid) AND p.dst; ---- 42 22 NULL 3 33 2 @@ -188,20 +175,14 @@ WHERE p.src BETWEEN (SELECT CREATE_CSR_EDGE( a.rowid, c.rowid, k.rowid, - t.cnt, - t_r.cnt_r) FROM Person_knows_Person k + t.cnt) FROM Person_knows_Person k JOIN Person a on a.id = k.Person1Id JOIN Person c on c.id = k.Person2Id JOIN (SELECT count(k.Person1Id) cnt, a.rowid as a_rowid FROM Person a LEFT JOIN Person_knows_Person k ON k.Person1Id = a.id GROUP BY a.rowid) t - ON t.a_rowid = a.rowid - JOIN (SELECT count(k.Person2Id) cnt_r, c.rowid as c_rowid - FROM Person c - LEFT JOIN Person_knows_Person k ON k.Person2Id = c.id - GROUP BY c.rowid) t_r - ON t_r.c_rowid = c.rowid) AND p.dst; + ON t.a_rowid = a.rowid) AND p.dst; ---- 10 2 NULL From 07bd55aa57e40a6f393e7b9802c325f6f8c29fec Mon Sep 17 00:00:00 2001 From: Pingan Ren Date: Wed, 10 Jul 2024 14:41:17 +0200 Subject: [PATCH 137/249] fix compile bug --- src/CMakeLists.txt | 1 + src/duckpgq_extension.cpp | 2 +- .../iterative_length_function_data.cpp | 4 +- src/functions/scalar/csr_creation.cpp | 9 +- .../iterative_length_function_data.hpp | 5 +- src/include/duckpgq_extension.hpp | 28 +++- .../physical_path_finding_operator.cpp | 14 +- .../path_finding/parallel_path_finding.test | 144 +----------------- 8 files changed, 42 insertions(+), 165 deletions(-) diff --git a/src/CMakeLists.txt b/src/CMakeLists.txt index 074a62d4..d374728e 100644 --- a/src/CMakeLists.txt +++ b/src/CMakeLists.txt @@ -1,4 +1,5 @@ add_subdirectory(functions) +add_subdirectory(operators) add_subdirectory(utils) set(EXTENSION_SOURCES diff --git a/src/duckpgq_extension.cpp b/src/duckpgq_extension.cpp index 9f575fda..b61086f2 100644 --- a/src/duckpgq_extension.cpp +++ b/src/duckpgq_extension.cpp @@ -55,7 +55,7 @@ static void LoadInternal(DatabaseInstance &instance) { config.optimizer_extensions.push_back(DuckpgqOptimizerExtension()); config.AddExtensionOption("experimental_path_finding_operator", "Enables the experimental path finding operator to be triggered", - LogicalType::BOOLEAN, Value(false)); + LogicalType::BOOLEAN, Value(true)); config.AddExtensionOption("experimental_path_finding_operator_task_size", "Number of vertices processed per thread at a time", LogicalType::INTEGER, Value(256)); Connection con(instance); diff --git a/src/functions/function_data/iterative_length_function_data.cpp b/src/functions/function_data/iterative_length_function_data.cpp index e687266d..0504913a 100644 --- a/src/functions/function_data/iterative_length_function_data.cpp +++ b/src/functions/function_data/iterative_length_function_data.cpp @@ -4,7 +4,7 @@ namespace duckdb { unique_ptr IterativeLengthFunctionData::Copy() const { - return make_uniq(context, csr_id); + return make_uniq(context, table_to_scan, csr_id); } bool IterativeLengthFunctionData::Equals(const FunctionData &other_p) const { @@ -23,7 +23,7 @@ unique_ptr IterativeLengthFunctionData::IterativeLengthBind( int32_t csr_id = ExpressionExecutor::EvaluateScalar(context, *arguments[0]) .GetValue(); - return make_uniq(context, csr_id); + return make_uniq(context, "", csr_id); } diff --git a/src/functions/scalar/csr_creation.cpp b/src/functions/scalar/csr_creation.cpp index c3231ae2..22e8c5d9 100644 --- a/src/functions/scalar/csr_creation.cpp +++ b/src/functions/scalar/csr_creation.cpp @@ -207,14 +207,7 @@ CreateScalarFunctionInfo DuckPGQFunctions::GetCsrEdgeFunction() { LogicalType::BIGINT, LogicalType::BIGINT, LogicalType::BIGINT, LogicalType::BIGINT}, LogicalType::INTEGER, CreateCsrEdgeFunction, - CSRFunctionData::CSREdgeBind)); - - set.AddFunction(ScalarFunction({LogicalType::BIGINT, LogicalType::BIGINT, - LogicalType::BIGINT, LogicalType::BIGINT, - LogicalType::BIGINT, LogicalType::BIGINT, - LogicalType::BIGINT}, - LogicalType::INTEGER, CreateCsrEdgeFunction, - CSRFunctionData::CSREdgeBind)); + CSRFunctionData::CSREdgeBind)); //! No edge weight set.AddFunction(ScalarFunction({LogicalType::INTEGER, LogicalType::BIGINT, diff --git a/src/include/duckpgq/functions/function_data/iterative_length_function_data.hpp b/src/include/duckpgq/functions/function_data/iterative_length_function_data.hpp index 3321ee50..c3becf78 100644 --- a/src/include/duckpgq/functions/function_data/iterative_length_function_data.hpp +++ b/src/include/duckpgq/functions/function_data/iterative_length_function_data.hpp @@ -12,10 +12,11 @@ namespace duckdb { struct IterativeLengthFunctionData final : FunctionData { ClientContext &context; + string table_to_scan; int32_t csr_id; - IterativeLengthFunctionData(ClientContext &context, int32_t csr_id) - : context(context), csr_id(csr_id) {} + IterativeLengthFunctionData(ClientContext &context, string table_to_scan, int32_t csr_id) + : context(context), table_to_scan(table_to_scan), csr_id(csr_id) {} static unique_ptr IterativeLengthBind(ClientContext &context, ScalarFunction &bound_function, vector> &arguments); diff --git a/src/include/duckpgq_extension.hpp b/src/include/duckpgq_extension.hpp index df5f6eea..8f6ab71e 100644 --- a/src/include/duckpgq_extension.hpp +++ b/src/include/duckpgq_extension.hpp @@ -1,10 +1,31 @@ #pragma once #include "duckdb.hpp" +#include "duckdb/main/config.hpp" +#include "duckdb/parser/column_list.hpp" #include "duckdb/parser/sql_statement.hpp" #include "duckdb/parser/parsed_expression.hpp" #include "duckpgq/utils/compressed_sparse_row.hpp" #include "duckdb/parser/parsed_data/create_property_graph_info.hpp" +#include "duckdb/parser/parsed_expression.hpp" +#include "duckdb/parser/query_node.hpp" +#include "duckdb/parser/simplified_token.hpp" +#include "duckdb/parser/sql_statement.hpp" +#include "duckdb/planner/logical_operator.hpp" +#include "duckdb/planner/operator/logical_aggregate.hpp" +#include "duckdb/planner/operator/logical_comparison_join.hpp" +#include "duckdb/planner/operator/logical_empty_result.hpp" +#include "duckdb/planner/operator/logical_filter.hpp" +#include "duckdb/planner/operator/logical_get.hpp" +#include "duckdb/planner/operator/logical_limit.hpp" +#include "duckdb/planner/operator/logical_projection.hpp" +#include "duckdb/planner/expression/bound_function_expression.hpp" +#include "duckpgq/operators/logical_path_finding_operator.hpp" +#include "duckpgq/functions/function_data/iterative_length_function_data.hpp" +#include "duckdb/function/table/table_scan.hpp" +#include "duckdb/main/database_manager.hpp" +#include "duckdb/catalog/catalog_entry/duck_table_entry.hpp" +#include namespace duckdb { @@ -108,6 +129,7 @@ class DuckpgqOptimizerExtension : public OptimizerExtension { path_finding_children, path_finding_expressions, mode); op.children.clear(); op.children.push_back(std::move(path_finding_operator)); + std::cout << "Found path-finding operator" << std::endl; return true; // We have found the path-finding operator, no need to continue } for (auto &child : op.children) { @@ -118,9 +140,9 @@ class DuckpgqOptimizerExtension : public OptimizerExtension { return false; } - static void DuckpgqOptimizeFunction(ClientContext &context, OptimizerExtensionInfo *info, + static void DuckpgqOptimizeFunction(OptimizerExtensionInput &input, duckdb::unique_ptr &plan) { - auto& client_config = ClientConfig::GetConfig(context); + auto& client_config = ClientConfig::GetConfig(input.context); auto const path_finding_operator_option = client_config.set_variables.find("experimental_path_finding_operator"); if (path_finding_operator_option == client_config.set_variables.end()) { return; // If the path finding operator is not enabled, we do not need to do anything @@ -128,7 +150,7 @@ class DuckpgqOptimizerExtension : public OptimizerExtension { if (!path_finding_operator_option->second.GetValue()) { return; } - InsertPathFindingOperator(*plan, context); + InsertPathFindingOperator(*plan, input.context); } }; diff --git a/src/operators/physical_path_finding_operator.cpp b/src/operators/physical_path_finding_operator.cpp index 33f59012..c2f62628 100644 --- a/src/operators/physical_path_finding_operator.cpp +++ b/src/operators/physical_path_finding_operator.cpp @@ -872,10 +872,10 @@ PhysicalPathFinding::Finalize(Pipeline &pipeline, Event &event, auto &global_tasks = gstate.global_tasks; if (gstate.child == 0) { - auto csr_event = make_shared(gstate, pipeline); - event.InsertEvent(std::move(std::dynamic_pointer_cast(csr_event))); + auto csr_event = make_shared_ptr(gstate, pipeline); + event.InsertEvent(std::move(csr_event)); } else if (gstate.child == 1 && global_tasks->Count() > 0) { - auto all_pairs = make_shared(); + auto all_pairs = make_shared_ptr(); DataChunk pairs; global_tasks->InitializeScanChunk(*all_pairs); global_tasks->InitializeScanChunk(pairs); @@ -953,11 +953,11 @@ void PhysicalPathFinding::ScheduleBFSTasks(Pipeline &pipeline, Event &event, } if (gstate.mode == "iterativelength") { - auto bfs_event = make_shared(gstate, pipeline); - event.InsertEvent(std::move(std::dynamic_pointer_cast(bfs_event))); + auto bfs_event = make_shared_ptr(gstate, pipeline); + event.InsertEvent(std::move(bfs_event)); } else if (gstate.mode == "shortestpath") { - auto bfs_event = make_shared(gstate, pipeline); - event.InsertEvent(std::move(std::dynamic_pointer_cast(bfs_event))); + auto bfs_event = make_shared_ptr(gstate, pipeline); + event.InsertEvent(std::move(bfs_event)); } } diff --git a/test/sql/path_finding/parallel_path_finding.test b/test/sql/path_finding/parallel_path_finding.test index 3ff5c9ef..9c39115a 100644 --- a/test/sql/path_finding/parallel_path_finding.test +++ b/test/sql/path_finding/parallel_path_finding.test @@ -19,42 +19,10 @@ create table student(id INT); INSERT INTO student(id) VALUES (10), (20), (30), ( statement ok create table know(src INT, dst INT); INSERT INTO know(src, dst) VALUES (40, 20), (10,30), (10,10), (20,10), (30,10); -#statement ok -#-CREATE PROPERTY GRAPH pg -#VERTEX TABLES ( -#student -#) -#EDGE TABLES ( -#know SOURCE KEY (src) REFERENCES student (id) -# DESTINATION KEY (dst) references student (id) -#); -# -#query III -#SELECT *, iterativelength(src, dst) as length -#FROM pair AS p -#WHERE p.src BETWEEN (SELECT CREATE_CSR_EDGE( -# (SELECT count(a.id) as v_size FROM Student a), -# (SELECT count(k.src) as e_size from know k), -# a.rowid, -# c.rowid, -# k.rowid, -# t.cnt) FROM Know k -# JOIN student a on a.id = k.src -# JOIN student c on c.id = k.dst -# JOIN (SELECT count(k.src) cnt, a.rowid as a_rowid -# FROM student a -# LEFT JOIN know k ON k.src = a.id -# GROUP BY a.rowid) t -# ON t.a_rowid = a.rowid) AND p.dst; -#---- -#0 1 NULL -#1 2 2 -#2 0 1 - query III SELECT *, shortestpath(src, dst, 'pair') as path FROM pair AS p -WHERE p.src BETWEEN (SELECT CREATE_CSR_EDGE( +WHERE p.src BETWEEN (SELECT CREATE_CSR_EDGE(0, (SELECT count(a.id) as v_size FROM Student a), (SELECT count(k.src) as e_size from know k), a.rowid, @@ -87,22 +55,10 @@ CREATE OR REPLACE TABLE pairs2 AS ( USING SAMPLE reservoir(2 ROWS) REPEATABLE (300) ); -#statement ok -#-CREATE PROPERTY GRAPH snb -#VERTEX TABLES ( -# Person PROPERTIES (id) LABEL Person -# ) -#EDGE TABLES ( -# Person_knows_Person SOURCE KEY (Person1Id) REFERENCES Person (id) -# DESTINATION KEY (Person2Id) REFERENCES Person (id) -# LABEL Knows -# ); - - query III SELECT *, iterativelength(src, dst, 'pairs2') as length FROM pairs2 AS p -WHERE p.src BETWEEN (SELECT CREATE_CSR_EDGE( +WHERE p.src BETWEEN (SELECT CREATE_CSR_EDGE(0, (SELECT count(a.id) as v_size FROM Person a), (SELECT count(k.Person1Id) as e_size FROM Person_knows_Person k), a.rowid, @@ -120,39 +76,6 @@ WHERE p.src BETWEEN (SELECT CREATE_CSR_EDGE( 42 22 NULL 3 33 2 - -#query III -#WITH cte1 AS ( -# SELECT CREATE_CSR_EDGE( -# 0, -# (SELECT count(a.id) FROM Person a), -# CAST ( -# (SELECT sum(CREATE_CSR_VERTEX( -# 0, -# (SELECT count(a.id) FROM Person a), -# sub.dense_id, -# sub.cnt) -# ) -# FROM ( -# SELECT a.rowid as dense_id, count(k.Person1Id) as cnt -# FROM Person a -# LEFT JOIN Person_knows_Person k ON k.Person1Id = a.id -# GROUP BY a.rowid) sub -# ) -# AS BIGINT), -# a.rowid, -# c.rowid, -# k.rowid) as temp -# FROM Person_knows_Person k -# JOIN Person a on a.id = k.Person1Id -# JOIN Person c on c.id = k.Person2Id -#) SELECT p.src, p.dst, iterativelength(0, (select count(*) from Person), p.src, p.dst) as length -# FROM pairs2 p, (select count(cte1.temp) * 0 as temp from cte1) __x -# WHERE __x.temp * 0 >= 0; -#---- -#42 22 NULL -#3 33 2 - statement ok CREATE OR REPLACE TABLE pairs3 AS ( SELECT src, dst @@ -185,66 +108,3 @@ WHERE p.src BETWEEN (SELECT CREATE_CSR_EDGE( ON t.a_rowid = a.rowid) AND p.dst; ---- 10 2 NULL - -# CAST ( -# (SELECT sum(CREATE_CSR_VERTEX(0, -# (SELECT count(a.id) FROM Student a), -# sub.dense_id, -# sub.cnt) -# ) -# FROM ( -# SELECT a.rowid as dense_id, count(k.src) as cnt -# FROM Student a -# LEFT JOIN Knows k ON k.src = a.id -# GROUP BY a.rowid) sub -# ) AS BIGINT), - - -#statement ok -#SELECT * -#FROM pairs AS p -#WHERE (p.src, p.dst) BETWEEN ( -# SELECT (csr_id, 0) FROM (SELECT 0 as csr_id, -# (SELECT count(a.id) FROM Student a), -# 0, -# a.rowid, -# c.rowid, -# k.rowid FROM Knows k -# JOIN student a on a.id = k.src -# JOIN student c on c.id = k.dst)) AND (0, 3); - -#CAST ( -# (SELECT sum(CREATE_CSR_VERTEX(0, -# (SELECT count(a.id) FROM Student a), -# sub.dense_id, -# sub.cnt) -# ) -# FROM ( -# SELECT a.rowid as dense_id, count(k.src) as cnt -# FROM Student a -# LEFT JOIN Knows k ON k.src = a.id -# GROUP BY a.rowid) sub -# ) AS BIGINT), -#statement ok -#select 1; -# LHS of IEJOIN -# RHS of IEJoin -# 0, -# (SELECT count(a.id) FROM Student a), -# CAST ( -# (SELECT sum(CREATE_CSR_VERTEX( -# 0, -# (SELECT count(a.id) FROM Student a), -# sub.dense_id, -# sub.cnt) -# ) -# FROM ( -# SELECT a.rowid as dense_id, count(k.src) as cnt -# FROM Student a -# LEFT JOIN Know k ON k.src = a.id -# GROUP BY a.rowid) sub -# ) -# AS BIGINT), -# a.rowid, -# c.rowid, -# k.rowid \ No newline at end of file From 285a285cc5d0595c27f958cd702243ffb98fd9bd Mon Sep 17 00:00:00 2001 From: dtenwolde Date: Wed, 10 Jul 2024 15:41:21 +0200 Subject: [PATCH 138/249] Remove dependency of atomic library and point extension-ci-tools --- CMakeLists.txt | 1 - extension-ci-tools | 2 +- 2 files changed, 1 insertion(+), 2 deletions(-) diff --git a/CMakeLists.txt b/CMakeLists.txt index d117257c..d6226956 100644 --- a/CMakeLists.txt +++ b/CMakeLists.txt @@ -38,7 +38,6 @@ build_loadable_extension(${TARGET_NAME} " " ${EXTENSION_SOURCES}) # Link OpenSSL in both the static library as the loadable extension target_link_libraries(${EXTENSION_NAME} OpenSSL::SSL OpenSSL::Crypto) target_link_libraries(${LOADABLE_EXTENSION_NAME} OpenSSL::SSL OpenSSL::Crypto) -target_link_libraries(${EXTENSION_NAME} atomic) # >>>>>>> template/main install( diff --git a/extension-ci-tools b/extension-ci-tools index 4672177c..6b8924c9 160000 --- a/extension-ci-tools +++ b/extension-ci-tools @@ -1 +1 @@ -Subproject commit 4672177cc5e516be1da66aec20cb8b8a5979adb8 +Subproject commit 6b8924c9beb7793460f67de027bddce25efa75fd From b7cc712689eae0292784018042893c27c7e60566 Mon Sep 17 00:00:00 2001 From: dtenwolde Date: Fri, 12 Jul 2024 15:42:39 +0200 Subject: [PATCH 139/249] Update to latest extension-ci-tools --- extension-ci-tools | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/extension-ci-tools b/extension-ci-tools index 6b8924c9..fa1bb1d2 160000 --- a/extension-ci-tools +++ b/extension-ci-tools @@ -1 +1 @@ -Subproject commit 6b8924c9beb7793460f67de027bddce25efa75fd +Subproject commit fa1bb1d2d05bd4d8c81eaf793f770e0afd060a45 From 726c116f6393cc6bfce007f3801195b3aefcc6ea Mon Sep 17 00:00:00 2001 From: dtenwolde Date: Fri, 12 Jul 2024 16:18:47 +0200 Subject: [PATCH 140/249] disabling the requirement of the csr id as first argument. Tests now pass --- .../iterative_length_function_data.cpp | 4 +++ src/utils/compressed_sparse_row.cpp | 26 ++++++++++--------- .../path_finding/parallel_path_finding.test | 2 +- 3 files changed, 19 insertions(+), 13 deletions(-) diff --git a/src/functions/function_data/iterative_length_function_data.cpp b/src/functions/function_data/iterative_length_function_data.cpp index 0504913a..018d9b41 100644 --- a/src/functions/function_data/iterative_length_function_data.cpp +++ b/src/functions/function_data/iterative_length_function_data.cpp @@ -16,6 +16,10 @@ bool IterativeLengthFunctionData::Equals(const FunctionData &other_p) const { unique_ptr IterativeLengthFunctionData::IterativeLengthBind( ClientContext &context, ScalarFunction &bound_function, vector> &arguments) { + if (arguments.size() == 3) { + string table_to_scan = ExpressionExecutor::EvaluateScalar(context, *arguments[2]).GetValue(); + return make_uniq(context, table_to_scan, 0); + } if (!arguments[0]->IsFoldable()) { throw InvalidInputException("Id must be constant."); } diff --git a/src/utils/compressed_sparse_row.cpp b/src/utils/compressed_sparse_row.cpp index bbb5dc1f..04ee35e4 100644 --- a/src/utils/compressed_sparse_row.cpp +++ b/src/utils/compressed_sparse_row.cpp @@ -65,18 +65,20 @@ unique_ptr CSRFunctionData::CSREdgeBind(ClientContext &context, ScalarFunction &bound_function, vector> &arguments) { - if (!arguments[0]->IsFoldable()) { - throw InvalidInputException("Id must be constant."); - } - Value id = ExpressionExecutor::EvaluateScalar(context, *arguments[0]); - if (arguments.size() == 7) { - return make_uniq(context, id.GetValue(), - arguments[6]->return_type); - } else { - auto logical_type = LogicalType::SQLNULL; - return make_uniq(context, id.GetValue(), - logical_type); - } + // if (!arguments[0]->IsFoldable()) { + // throw InvalidInputException("Id must be constant."); + // } + // Value id = ExpressionExecutor::EvaluateScalar(context, *arguments[0]); + // if (arguments.size() == 7) { + // return make_uniq(context, id.GetValue(), + // arguments[6]->return_type); + // } else { + // auto logical_type = LogicalType::SQLNULL; + // return make_uniq(context, id.GetValue(), + // logical_type); + // } + + return make_uniq(context, 0, LogicalType::SQLNULL); } unique_ptr diff --git a/test/sql/path_finding/parallel_path_finding.test b/test/sql/path_finding/parallel_path_finding.test index 9c39115a..940681ed 100644 --- a/test/sql/path_finding/parallel_path_finding.test +++ b/test/sql/path_finding/parallel_path_finding.test @@ -22,7 +22,7 @@ create table know(src INT, dst INT); INSERT INTO know(src, dst) VALUES (40, 20), query III SELECT *, shortestpath(src, dst, 'pair') as path FROM pair AS p -WHERE p.src BETWEEN (SELECT CREATE_CSR_EDGE(0, +WHERE p.src BETWEEN (SELECT CREATE_CSR_EDGE( (SELECT count(a.id) as v_size FROM Student a), (SELECT count(k.src) as e_size from know k), a.rowid, From 8a8e1ace7d242eee5d663f0192e7c35a23e7e518 Mon Sep 17 00:00:00 2001 From: dtenwolde Date: Mon, 12 Aug 2024 12:56:20 +0200 Subject: [PATCH 141/249] Merge with latest main branch --- src/core/CMakeLists.txt | 2 ++ src/core/module.cpp | 1 + src/core/operator/CMakeLists.txt | 2 ++ .../operator}/logical_path_finding_operator.cpp | 13 +++++++++---- .../operator}/physical_path_finding_operator.cpp | 15 ++++++++------- .../duckpgq/core/functions/table/match.hpp | 1 + .../operator}/logical_path_finding_operator.hpp | 9 +++++++-- .../operator}/physical_path_finding_operator.hpp | 10 +++++++--- src/operators/CMakeLists.txt | 7 ------- 9 files changed, 37 insertions(+), 23 deletions(-) rename src/{operators => core/operator}/logical_path_finding_operator.cpp (84%) rename src/{operators => core/operator}/physical_path_finding_operator.cpp (99%) rename src/include/duckpgq/{operators => core/operator}/logical_path_finding_operator.hpp (89%) rename src/include/duckpgq/{operators => core/operator}/physical_path_finding_operator.hpp (97%) delete mode 100644 src/operators/CMakeLists.txt diff --git a/src/core/CMakeLists.txt b/src/core/CMakeLists.txt index 142fa3b7..baefb9a8 100644 --- a/src/core/CMakeLists.txt +++ b/src/core/CMakeLists.txt @@ -1,8 +1,10 @@ add_subdirectory(functions) add_subdirectory(operator) +add_subdirectory(option) add_subdirectory(parser) add_subdirectory(utils) + set(EXTENSION_SOURCES ${CMAKE_CURRENT_SOURCE_DIR}/module.cpp ${EXTENSION_SOURCES} diff --git a/src/core/module.cpp b/src/core/module.cpp index c4887294..16532ddb 100644 --- a/src/core/module.cpp +++ b/src/core/module.cpp @@ -15,6 +15,7 @@ void CoreModule::Register(DatabaseInstance &db) { CoreScalarFunctions::Register(db); CorePGQParser::Register(db); CorePGQOperator::Register(db); + CorePGQOptions::Register(db); } diff --git a/src/core/operator/CMakeLists.txt b/src/core/operator/CMakeLists.txt index 36f54420..02c1c60f 100644 --- a/src/core/operator/CMakeLists.txt +++ b/src/core/operator/CMakeLists.txt @@ -1,5 +1,7 @@ set(EXTENSION_SOURCES ${CMAKE_CURRENT_SOURCE_DIR}/duckpgq_bind.cpp + ${CMAKE_CURRENT_SOURCE_DIR}/logical_path_finding_operator.cpp + ${CMAKE_CURRENT_SOURCE_DIR}/physical_path_finding_operator.cpp ${EXTENSION_SOURCES} PARENT_SCOPE ) \ No newline at end of file diff --git a/src/operators/logical_path_finding_operator.cpp b/src/core/operator/logical_path_finding_operator.cpp similarity index 84% rename from src/operators/logical_path_finding_operator.cpp rename to src/core/operator/logical_path_finding_operator.cpp index b3bef44f..d8e8ee72 100644 --- a/src/operators/logical_path_finding_operator.cpp +++ b/src/core/operator/logical_path_finding_operator.cpp @@ -1,9 +1,12 @@ -#include "duckpgq/operators/logical_path_finding_operator.hpp" -#include -#include +#include "duckpgq/core/operator/logical_path_finding_operator.hpp" +#include "duckpgq/core/operator/physical_path_finding_operator.hpp" +#include "duckpgq/common.hpp" + +namespace duckpgq { + +namespace core { -namespace duckdb { unique_ptr LogicalPathFindingOperator::CreatePlan( ClientContext &, duckdb::PhysicalPlanGenerator &generator) { D_ASSERT(children.size() == 2); @@ -39,4 +42,6 @@ string LogicalPathFindingOperator::ParamsToString() const { return extra_info; } + +} // namespace core } // namespace duckdb diff --git a/src/operators/physical_path_finding_operator.cpp b/src/core/operator/physical_path_finding_operator.cpp similarity index 99% rename from src/operators/physical_path_finding_operator.cpp rename to src/core/operator/physical_path_finding_operator.cpp index c2f62628..0041d6ed 100644 --- a/src/operators/physical_path_finding_operator.cpp +++ b/src/core/operator/physical_path_finding_operator.cpp @@ -1,4 +1,6 @@ -#include "duckpgq/operators/physical_path_finding_operator.hpp" +#include "duckpgq/core/operator/physical_path_finding_operator.hpp" +#include +#include "duckpgq/common.hpp" #include "duckdb/common/sort/sort.hpp" #include "duckdb/common/types/column/column_data_collection.hpp" @@ -9,14 +11,12 @@ #include "duckdb/parallel/meta_pipeline.hpp" #include "duckdb/parallel/thread_context.hpp" #include -#include #include -#include -#include -#include #include -namespace duckdb { +namespace duckpgq { + +namespace core { struct ve { // higher 30 bits for v, lower 34 bits for e @@ -26,7 +26,7 @@ struct ve { const uint64_t v_mask = UINT64_MAX << e_bits; const uint64_t e_mask = UINT64_MAX >> v_bits; ve() : value(UINT64_MAX) {} - ve(uint64_t value) : value(value) {} + explicit ve(uint64_t value) : value(value) {} ve(int64_t v, int64_t e) { uint64_t new_value = 0; new_value |= v < 0 ? v_mask : (v << e_bits); @@ -1084,4 +1084,5 @@ void PhysicalPathFinding::BuildPipelines(Pipeline ¤t, child_meta_pipeline.AddFinishEvent(rhs_pipeline); } +} // namespace core } // namespace duckdb diff --git a/src/include/duckpgq/core/functions/table/match.hpp b/src/include/duckpgq/core/functions/table/match.hpp index 036c79aa..90391901 100644 --- a/src/include/duckpgq/core/functions/table/match.hpp +++ b/src/include/duckpgq/core/functions/table/match.hpp @@ -23,6 +23,7 @@ namespace duckpgq { namespace core { + config.optimizer_extensions.push_back(DuckpgqOptimizerExtension()); config.AddExtensionOption("experimental_path_finding_operator", "Enables the experimental path finding operator to be triggered", diff --git a/src/include/duckpgq/operators/logical_path_finding_operator.hpp b/src/include/duckpgq/core/operator/logical_path_finding_operator.hpp similarity index 89% rename from src/include/duckpgq/operators/logical_path_finding_operator.hpp rename to src/include/duckpgq/core/operator/logical_path_finding_operator.hpp index a98613a4..83598a14 100644 --- a/src/include/duckpgq/operators/logical_path_finding_operator.hpp +++ b/src/include/duckpgq/core/operator/logical_path_finding_operator.hpp @@ -1,7 +1,11 @@ #pragma once +#include "duckpgq/common.hpp" #include -namespace duckdb { +namespace duckpgq { + +namespace core { + class LogicalPathFindingOperator : public LogicalExtensionOperator { public: @@ -29,5 +33,6 @@ class LogicalPathFindingOperator : public LogicalExtensionOperator { public: string mode; }; +} // namespace core -} // namespace duckdb +} // namespace duckpgq diff --git a/src/include/duckpgq/operators/physical_path_finding_operator.hpp b/src/include/duckpgq/core/operator/physical_path_finding_operator.hpp similarity index 97% rename from src/include/duckpgq/operators/physical_path_finding_operator.hpp rename to src/include/duckpgq/core/operator/physical_path_finding_operator.hpp index 3833f108..2d6ecfd2 100644 --- a/src/include/duckpgq/operators/physical_path_finding_operator.hpp +++ b/src/include/duckpgq/core/operator/physical_path_finding_operator.hpp @@ -7,13 +7,15 @@ //===----------------------------------------------------------------------===// #pragma once - +#include "duckpgq/common.hpp" #include "duckdb/common/types/row/row_layout.hpp" #include "duckdb/execution/operator/join/physical_comparison_join.hpp" #include "duckdb/execution/physical_operator.hpp" #include "duckdb/planner/operator/logical_extension_operator.hpp" -namespace duckdb { +namespace duckpgq { + +namespace core { class PhysicalPathFinding : public PhysicalComparisonJoin { #define LANE_LIMIT 512 @@ -161,4 +163,6 @@ class PhysicalPathFinding : public PhysicalComparisonJoin { static void ScheduleBFSTasks(Pipeline &pipeline, Event &event, GlobalSinkState &state); }; -} // namespace duckdb +} // namespace core +} // namespace duckpgq + diff --git a/src/operators/CMakeLists.txt b/src/operators/CMakeLists.txt deleted file mode 100644 index f9172760..00000000 --- a/src/operators/CMakeLists.txt +++ /dev/null @@ -1,7 +0,0 @@ -set(EXTENSION_SOURCES - ${CMAKE_CURRENT_SOURCE_DIR}/logical_path_finding_operator.cpp - ${CMAKE_CURRENT_SOURCE_DIR}/physical_path_finding_operator.cpp - - ${EXTENSION_SOURCES} - PARENT_SCOPE -) \ No newline at end of file From 0914e9df894a0b4f6a45b81856c58fca7740c797 Mon Sep 17 00:00:00 2001 From: dtenwolde Date: Mon, 12 Aug 2024 12:56:29 +0200 Subject: [PATCH 142/249] Add separate file for options --- src/core/option/CMakeLists.txt | 5 +++++ src/core/option/duckpgq_option.cpp | 20 +++++++++++++++++ .../duckpgq/core/option/duckpgq_option.hpp | 22 +++++++++++++++++++ 3 files changed, 47 insertions(+) create mode 100644 src/core/option/CMakeLists.txt create mode 100644 src/core/option/duckpgq_option.cpp create mode 100644 src/include/duckpgq/core/option/duckpgq_option.hpp diff --git a/src/core/option/CMakeLists.txt b/src/core/option/CMakeLists.txt new file mode 100644 index 00000000..e7d41c0a --- /dev/null +++ b/src/core/option/CMakeLists.txt @@ -0,0 +1,5 @@ +set(EXTENSION_SOURCES + ${CMAKE_CURRENT_SOURCE_DIR}/duckpgq_option.cpp + ${EXTENSION_SOURCES} + PARENT_SCOPE +) \ No newline at end of file diff --git a/src/core/option/duckpgq_option.cpp b/src/core/option/duckpgq_option.cpp new file mode 100644 index 00000000..87d238e9 --- /dev/null +++ b/src/core/option/duckpgq_option.cpp @@ -0,0 +1,20 @@ +#include "duckpgq/core/option/duckpgq_option.hpp" +#include "duckpgq/common.hpp" + +namespace duckpgq { +namespace core { + + +//------------------------------------------------------------------------------ +// Register functions +//------------------------------------------------------------------------------ +void CorePGQOptions::Register( + DatabaseInstance &db) { + auto &config = DBConfig::GetConfig(db); + DuckPGQParserExtension pgq_parser; + config.parser_extensions.push_back(pgq_parser); +} + +} // namespace core +} // namespace duckpgq + diff --git a/src/include/duckpgq/core/option/duckpgq_option.hpp b/src/include/duckpgq/core/option/duckpgq_option.hpp new file mode 100644 index 00000000..1eadc6b5 --- /dev/null +++ b/src/include/duckpgq/core/option/duckpgq_option.hpp @@ -0,0 +1,22 @@ +#pragma once + +#include "duckpgq/common.hpp" + +namespace duckpgq { + +namespace core { + +struct CorePGQOptions { + static void Register(DatabaseInstance &db) { + RegisterExperimentalPathFindingOperator(db); + RegisterPathFindingTaskSize(db); + } + +private: + static void RegisterExperimentalPathFindingOperator(DatabaseInstance &db); + static void RegisterPathFindingTaskSize(DatabaseInstance &db); +};; + +} // namespace core + +} // namespace duckpgq From 3a862a8174bdec7cd404261a3117853542639ba1 Mon Sep 17 00:00:00 2001 From: dtenwolde Date: Mon, 12 Aug 2024 12:57:34 +0200 Subject: [PATCH 143/249] Add options --- src/core/option/duckpgq_option.cpp | 20 ++++++++++++++++--- .../duckpgq/core/functions/table/match.hpp | 6 +----- 2 files changed, 18 insertions(+), 8 deletions(-) diff --git a/src/core/option/duckpgq_option.cpp b/src/core/option/duckpgq_option.cpp index 87d238e9..41f22d25 100644 --- a/src/core/option/duckpgq_option.cpp +++ b/src/core/option/duckpgq_option.cpp @@ -8,13 +8,27 @@ namespace core { //------------------------------------------------------------------------------ // Register functions //------------------------------------------------------------------------------ -void CorePGQOptions::Register( +void CorePGQOptions::RegisterExperimentalPathFindingOperator( DatabaseInstance &db) { auto &config = DBConfig::GetConfig(db); - DuckPGQParserExtension pgq_parser; - config.parser_extensions.push_back(pgq_parser); + config.AddExtensionOption("experimental_path_finding_operator", + "Enables the experimental path finding operator to be triggered", + LogicalType::BOOLEAN, Value(true)); } +//------------------------------------------------------------------------------ +// Register functions +//------------------------------------------------------------------------------ +void CorePGQOptions::RegisterPathFindingTaskSize( + DatabaseInstance &db) { + auto &config = DBConfig::GetConfig(db); + + config.AddExtensionOption("experimental_path_finding_operator_task_size", + "Number of vertices processed per thread at a time", LogicalType::INTEGER, Value(256)); +} + + + } // namespace core } // namespace duckpgq diff --git a/src/include/duckpgq/core/functions/table/match.hpp b/src/include/duckpgq/core/functions/table/match.hpp index 90391901..be0797d0 100644 --- a/src/include/duckpgq/core/functions/table/match.hpp +++ b/src/include/duckpgq/core/functions/table/match.hpp @@ -25,11 +25,7 @@ namespace core { config.optimizer_extensions.push_back(DuckpgqOptimizerExtension()); -config.AddExtensionOption("experimental_path_finding_operator", - "Enables the experimental path finding operator to be triggered", - LogicalType::BOOLEAN, Value(true)); -config.AddExtensionOption("experimental_path_finding_operator_task_size", - "Number of vertices processed per thread at a time", LogicalType::INTEGER, Value(256)); + From 39731e6520015af65706a05320b4c92db786e03a Mon Sep 17 00:00:00 2001 From: dtenwolde Date: Mon, 12 Aug 2024 13:07:08 +0200 Subject: [PATCH 144/249] Remove unused import --- src/duckpgq_extension.cpp | 5 +---- 1 file changed, 1 insertion(+), 4 deletions(-) diff --git a/src/duckpgq_extension.cpp b/src/duckpgq_extension.cpp index 86e956e3..a5be68bc 100644 --- a/src/duckpgq_extension.cpp +++ b/src/duckpgq_extension.cpp @@ -5,9 +5,6 @@ #include "duckpgq/core/module.hpp" -// #include "duckpgq/core/functions/table/drop_property_graph.hpp" - - namespace duckdb { static void LoadInternal(DatabaseInstance &instance) { @@ -27,7 +24,7 @@ DUCKDB_EXTENSION_API void duckpgq_init(DatabaseInstance &db) { } DUCKDB_EXTENSION_API const char *duckpgq_version() { - return duckdb::DuckDB::LibraryVersion(); + return DuckDB::LibraryVersion(); } } From 67c821db949d8941cbf0837e740c02b4b09d2afd Mon Sep 17 00:00:00 2001 From: dtenwolde Date: Mon, 12 Aug 2024 13:11:20 +0200 Subject: [PATCH 145/249] Adding optimizer rule --- src/core/CMakeLists.txt | 1 + src/core/module.cpp | 5 ++++- src/core/optimizer/CMakeLists.txt | 6 ++++++ src/core/optimizer/duckpgq_optimizer.cpp | 8 ++++++++ src/core/optimizer/path_finding_optimizer_rule.cpp | 10 ++++++++++ .../duckpgq/core/optimizer/duckpgq_optimizer.hpp | 10 ++++++++++ .../core/optimizer/path_finding_optimizer_rule.hpp | 8 ++++++++ 7 files changed, 47 insertions(+), 1 deletion(-) create mode 100644 src/core/optimizer/CMakeLists.txt create mode 100644 src/core/optimizer/duckpgq_optimizer.cpp create mode 100644 src/core/optimizer/path_finding_optimizer_rule.cpp create mode 100644 src/include/duckpgq/core/optimizer/duckpgq_optimizer.hpp create mode 100644 src/include/duckpgq/core/optimizer/path_finding_optimizer_rule.hpp diff --git a/src/core/CMakeLists.txt b/src/core/CMakeLists.txt index baefb9a8..e601f407 100644 --- a/src/core/CMakeLists.txt +++ b/src/core/CMakeLists.txt @@ -1,6 +1,7 @@ add_subdirectory(functions) add_subdirectory(operator) add_subdirectory(option) +add_subdirectory(optimizer) add_subdirectory(parser) add_subdirectory(utils) diff --git a/src/core/module.cpp b/src/core/module.cpp index 16532ddb..c1e7d723 100644 --- a/src/core/module.cpp +++ b/src/core/module.cpp @@ -5,6 +5,8 @@ #include "duckpgq/core/functions/scalar.hpp" #include "duckpgq/core/parser/duckpgq_parser.hpp" #include "duckpgq/core/operator/duckpgq_operator.hpp" +#include "duckpgq/core/option/duckpgq_option.hpp" +#include "duckpgq/core/optimizer/duckpgq_optimizer.hpp" namespace duckpgq { @@ -13,9 +15,10 @@ namespace core { void CoreModule::Register(DatabaseInstance &db) { CoreTableFunctions::Register(db); CoreScalarFunctions::Register(db); - CorePGQParser::Register(db); CorePGQOperator::Register(db); CorePGQOptions::Register(db); + CorePGQOptimizer::Register(db); + CorePGQParser::Register(db); } diff --git a/src/core/optimizer/CMakeLists.txt b/src/core/optimizer/CMakeLists.txt new file mode 100644 index 00000000..780f049f --- /dev/null +++ b/src/core/optimizer/CMakeLists.txt @@ -0,0 +1,6 @@ +set(EXTENSION_SOURCES + ${CMAKE_CURRENT_SOURCE_DIR}/duckpgq_optimizer.cpp + ${CMAKE_CURRENT_SOURCE_DIR}/path_finding_optimizer_rule.cpp + ${EXTENSION_SOURCES} + PARENT_SCOPE +) \ No newline at end of file diff --git a/src/core/optimizer/duckpgq_optimizer.cpp b/src/core/optimizer/duckpgq_optimizer.cpp new file mode 100644 index 00000000..60c882ee --- /dev/null +++ b/src/core/optimizer/duckpgq_optimizer.cpp @@ -0,0 +1,8 @@ +#include "duckpgq/core/optimizer/duckpgq_optimizer.hpp" + +namespace duckpgq { +namespace core { + +} // namespace core + +} // namespace duckpgq \ No newline at end of file diff --git a/src/core/optimizer/path_finding_optimizer_rule.cpp b/src/core/optimizer/path_finding_optimizer_rule.cpp new file mode 100644 index 00000000..1dd1aa6c --- /dev/null +++ b/src/core/optimizer/path_finding_optimizer_rule.cpp @@ -0,0 +1,10 @@ +#include "duckpgq/core/optimizer/path_finding_optimizer_rule.hpp" + + +namespace duckpgq { +namespace core { + + + +} // namespace core +} // namespace duckpgq \ No newline at end of file diff --git a/src/include/duckpgq/core/optimizer/duckpgq_optimizer.hpp b/src/include/duckpgq/core/optimizer/duckpgq_optimizer.hpp new file mode 100644 index 00000000..15a5e6ed --- /dev/null +++ b/src/include/duckpgq/core/optimizer/duckpgq_optimizer.hpp @@ -0,0 +1,10 @@ +#pragma once +#include "duckpgq/common.hpp" + +namespace duckpgq { + +namespace core { + +} // namespace core + +} // namespace duckpgq \ No newline at end of file diff --git a/src/include/duckpgq/core/optimizer/path_finding_optimizer_rule.hpp b/src/include/duckpgq/core/optimizer/path_finding_optimizer_rule.hpp new file mode 100644 index 00000000..c3bfa6e1 --- /dev/null +++ b/src/include/duckpgq/core/optimizer/path_finding_optimizer_rule.hpp @@ -0,0 +1,8 @@ +// +// Created by Daniel ten Wolde on 12/08/2024. +// + +#ifndef PATH_FINDING_OPTIMIZER_RULE_HPP +#define PATH_FINDING_OPTIMIZER_RULE_HPP + +#endif //PATH_FINDING_OPTIMIZER_RULE_HPP From f75bcf5e20ed6f82a0f621a701fa7c621cfab835 Mon Sep 17 00:00:00 2001 From: dtenwolde Date: Mon, 12 Aug 2024 13:12:36 +0200 Subject: [PATCH 146/249] Adding coreoptimizer struct --- .../duckpgq/core/optimizer/duckpgq_optimizer.hpp | 11 ++++++++++- .../core/optimizer/path_finding_optimizer_rule.hpp | 13 +++++++------ 2 files changed, 17 insertions(+), 7 deletions(-) diff --git a/src/include/duckpgq/core/optimizer/duckpgq_optimizer.hpp b/src/include/duckpgq/core/optimizer/duckpgq_optimizer.hpp index 15a5e6ed..eba1df00 100644 --- a/src/include/duckpgq/core/optimizer/duckpgq_optimizer.hpp +++ b/src/include/duckpgq/core/optimizer/duckpgq_optimizer.hpp @@ -5,6 +5,15 @@ namespace duckpgq { namespace core { -} // namespace core +struct CorePGQOptimizer { + static void Register(DatabaseInstance &db) { + RegisterPathFindingOptimizerRule(db); + } + +private: + static void RegisterPathFindingOptimizerRule(DatabaseInstance &db); +}; + +} // namespace core } // namespace duckpgq \ No newline at end of file diff --git a/src/include/duckpgq/core/optimizer/path_finding_optimizer_rule.hpp b/src/include/duckpgq/core/optimizer/path_finding_optimizer_rule.hpp index c3bfa6e1..cf78561f 100644 --- a/src/include/duckpgq/core/optimizer/path_finding_optimizer_rule.hpp +++ b/src/include/duckpgq/core/optimizer/path_finding_optimizer_rule.hpp @@ -1,8 +1,9 @@ -// -// Created by Daniel ten Wolde on 12/08/2024. -// +#pragma once -#ifndef PATH_FINDING_OPTIMIZER_RULE_HPP -#define PATH_FINDING_OPTIMIZER_RULE_HPP +#include "duckpgq/common.hpp" -#endif //PATH_FINDING_OPTIMIZER_RULE_HPP +namespace duckpgq { +namespace core { + +} // namespace core +} // namespace duckpgq \ No newline at end of file From bb1f54d6aadddfd2fcf241d69d1f4066d629fc38 Mon Sep 17 00:00:00 2001 From: dtenwolde Date: Mon, 12 Aug 2024 13:43:32 +0200 Subject: [PATCH 147/249] Adding optimizer rule --- .../optimizer/path_finding_optimizer_rule.cpp | 136 ++++++++++++++++++ .../duckpgq/core/functions/table/match.hpp | 2 - .../optimizer/path_finding_optimizer_rule.hpp | 12 ++ 3 files changed, 148 insertions(+), 2 deletions(-) diff --git a/src/core/optimizer/path_finding_optimizer_rule.cpp b/src/core/optimizer/path_finding_optimizer_rule.cpp index 1dd1aa6c..c0428ca5 100644 --- a/src/core/optimizer/path_finding_optimizer_rule.cpp +++ b/src/core/optimizer/path_finding_optimizer_rule.cpp @@ -1,9 +1,145 @@ #include "duckpgq/core/optimizer/path_finding_optimizer_rule.hpp" +#include "duckpgq/core/optimizer/duckpgq_optimizer.hpp" + +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include namespace duckpgq { namespace core { +bool DuckpgqOptimizerExtension::InsertPathFindingOperator(LogicalOperator &op, ClientContext &context) { + unique_ptr function_expression; + string mode; + for (int64_t i = op.expressions.size() - 1; i >= 0; --i) { + const auto &expr = op.expressions[i]; + if (expr->expression_class != ExpressionClass::BOUND_FUNCTION) { + continue; + } + auto &bound_function_expression = expr->Cast(); + if (bound_function_expression.function.name == "iterativelength") { + op.expressions.emplace_back(make_uniq( + expr->alias, LogicalType::BIGINT, ColumnBinding(10, 0))); + function_expression = expr->Copy(); + op.expressions.erase(op.expressions.begin() + i); + mode = "iterativelength"; + } else if (bound_function_expression.function.name == "shortestpath") { + op.expressions.emplace_back(make_uniq( + expr->alias, LogicalType::LIST(LogicalType::BIGINT), ColumnBinding(10, 0))); + function_expression = expr->Copy(); + op.expressions.erase(op.expressions.begin() + i); + mode = "shortestpath"; + } + } + for (const auto &child : op.children) { + vector> path_finding_children; + if (child->type != LogicalOperatorType::LOGICAL_COMPARISON_JOIN) { + continue; + } + auto &get_join = child->Cast(); + //! For now we assume this is enough to detect we have found a + //! path-finding query. Should be improved in the future + if (get_join.children.size() != 2) { + continue; + } + if (get_join.children[1]->type != + LogicalOperatorType::LOGICAL_AGGREGATE_AND_GROUP_BY) { + continue; + } + auto &get_aggregate = get_join.children[1]->Cast(); + auto &get_limit = get_aggregate.children[0]->Cast(); + auto &get_projection = get_limit.children[0]->Cast(); + auto &get_function_expression = + get_projection.expressions[0]->Cast(); + if (get_function_expression.function.name != "create_csr_edge") { + continue; + } + vector> path_finding_expressions = + std::move(get_function_expression.children); + if (get_join.children[0]->type == LogicalOperatorType::LOGICAL_FILTER) { + auto &get_filter = get_join.children[0]->Cast(); + if (get_filter.children[0]->type != LogicalOperatorType::LOGICAL_GET) { + continue; + } + path_finding_children.push_back(std::move(get_filter.children[0])); + } else if (get_join.children[0]->type == + LogicalOperatorType::LOGICAL_EMPTY_RESULT) { + auto default_database = DatabaseManager::GetDefaultDatabase(context); + auto &catalog = Catalog::GetCatalog(context, default_database); + auto &bound_function_expression = function_expression->Cast(); + auto &bind_info = bound_function_expression.bind_info->Cast(); + auto &duckdb_table = catalog.GetEntry(context, DEFAULT_SCHEMA, bind_info.table_to_scan); + auto &get_empty_result = + get_join.children[0]->Cast(); + vector returned_names = {"src", "dst"}; + unique_ptr bind_data; + auto scan_function = duckdb_table.GetScanFunction(context, bind_data); + auto logical_get = make_uniq( + get_empty_result.bindings[0].table_index, scan_function, + std::move(bind_data), get_empty_result.return_types, + returned_names); + vector column_ids_vector; + for (const auto &binding : get_empty_result.bindings) { + column_ids_vector.push_back(binding.column_index); + } + logical_get->column_ids = std::move(column_ids_vector); + path_finding_children.push_back(std::move(logical_get)); + } + path_finding_children.push_back(std::move(get_projection.children[0])); + // Iterate in reverse to not influence the upcoming iterations when + // erasing an element from the list. Does not work if both iterativelength + // and shortestpath are called in the same query for now. To be improved + // in the future. + + + auto path_finding_operator = make_uniq( + path_finding_children, path_finding_expressions, mode); + op.children.clear(); + op.children.push_back(std::move(path_finding_operator)); + std::cout << "Found path-finding operator" << std::endl; + return true; // We have found the path-finding operator, no need to continue + } + for (auto &child : op.children) { + if (InsertPathFindingOperator(*child, context)) { + return true; + } + } + return false; + } + +void DuckpgqOptimizerExtension::DuckpgqOptimizeFunction(OptimizerExtensionInput &input, + duckdb::unique_ptr &plan) { + auto& client_config = ClientConfig::GetConfig(input.context); + auto const path_finding_operator_option = client_config.set_variables.find("experimental_path_finding_operator"); + if (path_finding_operator_option == client_config.set_variables.end()) { + return; // If the path finding operator is not enabled, we do not need to do anything + } + if (!path_finding_operator_option->second.GetValue()) { + return; + } + InsertPathFindingOperator(*plan, input.context); +} + + +//------------------------------------------------------------------------------ +// Register functions +//------------------------------------------------------------------------------ +void CorePGQOptimizer::RegisterPathFindingOptimizerRule( + DatabaseInstance &db) { + auto &config = DBConfig::GetConfig(db); + + config.optimizer_extensions.push_back(DuckpgqOptimizerExtension());} + } // namespace core diff --git a/src/include/duckpgq/core/functions/table/match.hpp b/src/include/duckpgq/core/functions/table/match.hpp index be0797d0..c64c1b33 100644 --- a/src/include/duckpgq/core/functions/table/match.hpp +++ b/src/include/duckpgq/core/functions/table/match.hpp @@ -24,8 +24,6 @@ namespace duckpgq { namespace core { -config.optimizer_extensions.push_back(DuckpgqOptimizerExtension()); - diff --git a/src/include/duckpgq/core/optimizer/path_finding_optimizer_rule.hpp b/src/include/duckpgq/core/optimizer/path_finding_optimizer_rule.hpp index cf78561f..6a3ddd14 100644 --- a/src/include/duckpgq/core/optimizer/path_finding_optimizer_rule.hpp +++ b/src/include/duckpgq/core/optimizer/path_finding_optimizer_rule.hpp @@ -5,5 +5,17 @@ namespace duckpgq { namespace core { +class DuckpgqOptimizerExtension : public OptimizerExtension { +public: + DuckpgqOptimizerExtension() { + optimize_function = DuckpgqOptimizeFunction; + } + + static bool InsertPathFindingOperator(LogicalOperator &op, ClientContext &context); + + static void DuckpgqOptimizeFunction(OptimizerExtensionInput &input, + unique_ptr &plan); +}; + } // namespace core } // namespace duckpgq \ No newline at end of file From df89b627b3b4860d7d27b97bc6aa70742666bc81 Mon Sep 17 00:00:00 2001 From: dtenwolde Date: Mon, 12 Aug 2024 13:47:31 +0200 Subject: [PATCH 148/249] Format fix --- src/core/functions/table/match.cpp | 10 ++-- .../duckpgq/core/functions/table/match.hpp | 53 ++++++++++--------- 2 files changed, 32 insertions(+), 31 deletions(-) diff --git a/src/core/functions/table/match.cpp b/src/core/functions/table/match.cpp index ada7cd8a..dcfab017 100644 --- a/src/core/functions/table/match.cpp +++ b/src/core/functions/table/match.cpp @@ -800,12 +800,12 @@ void PGQMatchFunction::AddPathFinding( //! FROM (SELECT count(cte1.temp) * 0 as temp from cte1) __x // TODO wrap this with the config option for experimental path finding // auto csr = CreateCSRCTE(edge_table, prev_binding, edge_binding, next_binding); - auto src_tasks = make_uniq("rowid", prev_binding); - auto dst_tasks = make_uniq("rowid", next_binding); - auto between_expression = make_uniq( - std::move(src_tasks), std::move(csr), std::move(dst_tasks)); + // auto src_tasks = make_uniq("rowid", prev_binding); + // auto dst_tasks = make_uniq("rowid", next_binding); + // auto between_expression = make_uniq( + // std::move(src_tasks), std::move(CSR), std::move(dst_tasks)); - conditions.push_back(std::move(between_expression)); + // conditions.push_back(std::move(between_expression)); //! START //! WHERE __x.temp + iterativelength(, (SELECT count(c.id) diff --git a/src/include/duckpgq/core/functions/table/match.hpp b/src/include/duckpgq/core/functions/table/match.hpp index c64c1b33..2c09dd47 100644 --- a/src/include/duckpgq/core/functions/table/match.hpp +++ b/src/include/duckpgq/core/functions/table/match.hpp @@ -23,10 +23,6 @@ namespace duckpgq { namespace core { - - - - struct PGQMatchFunction : public TableFunction { public: PGQMatchFunction() { @@ -115,8 +111,8 @@ struct PGQMatchFunction : public TableFunction { idx_t element_idx); static unique_ptr AddPathQuantifierCondition( - const string &prev_binding, const string &next_binding, - const shared_ptr &edge_table, const SubPath *subpath); + const string &prev_binding, const string &next_binding, + const shared_ptr &edge_table, const SubPath *subpath); static unique_ptr MatchBindReplace(ClientContext &context, TableFunctionBindInput &input); @@ -126,19 +122,25 @@ struct PGQMatchFunction : public TableFunction { vector> &column_list, unordered_set &named_subpaths); - static unique_ptr GenerateShortestPathCTE(CreatePropertyGraphInfo & pg_table, SubPath * edge_subpath, - PathElement * path_element, PathElement * next_vertex_element, vector> &path_finding_conditions); + static unique_ptr GenerateShortestPathCTE( + CreatePropertyGraphInfo &pg_table, SubPath *edge_subpath, + PathElement *path_element, PathElement *next_vertex_element, + vector> &path_finding_conditions); static unique_ptr CreatePathFindingFunction(vector> &path_list, - CreatePropertyGraphInfo &pg_table, const string &path_variable, unique_ptr &final_select_node, vector> &conditions); + CreatePropertyGraphInfo &pg_table, + const string &path_variable, + unique_ptr &final_select_node, + vector> &conditions); - static void AddPathFinding( - unique_ptr &select_node, - vector> &conditions, - const string &prev_binding, const string &edge_binding, - const string &next_binding, - const shared_ptr &edge_table, - CreatePropertyGraphInfo &pg_table, SubPath *subpath); + static void AddPathFinding(unique_ptr &select_node, + vector> &conditions, + const string &prev_binding, + const string &edge_binding, + const string &next_binding, + const shared_ptr &edge_table, + CreatePropertyGraphInfo &pg_table, + SubPath *subpath); static void AddEdgeJoins(const shared_ptr &edge_table, @@ -150,17 +152,16 @@ struct PGQMatchFunction : public TableFunction { unordered_map &alias_map, int32_t &extra_alias_counter, unique_ptr &from_clause); - static void ProcessPathList( - vector> &path_pattern, - vector> &conditions, - unique_ptr &select_node, - unordered_map &alias_map, - CreatePropertyGraphInfo &pg_table, int32_t &extra_alias_counter, - MatchExpression &original_ref); + static void ProcessPathList(vector> &path_pattern, + vector> &conditions, + unique_ptr &select_node, + unordered_map &alias_map, + CreatePropertyGraphInfo &pg_table, + int32_t &extra_alias_counter, + MatchExpression &original_ref); static void - CheckNamedSubpath(SubPath &subpath, - MatchExpression &original_ref, + CheckNamedSubpath(SubPath &subpath, MatchExpression &original_ref, CreatePropertyGraphInfo &pg_table, unique_ptr &final_select_node, vector> &conditions); @@ -168,4 +169,4 @@ struct PGQMatchFunction : public TableFunction { } // namespace core -} // namespace duckdb +} // namespace duckpgq From e3d0eefef6375c4c614380b0f366567c21f7ad13 Mon Sep 17 00:00:00 2001 From: dtenwolde Date: Mon, 12 Aug 2024 14:33:11 +0200 Subject: [PATCH 149/249] Remove commented out code --- src/core/functions/table/match.cpp | 2 -- 1 file changed, 2 deletions(-) diff --git a/src/core/functions/table/match.cpp b/src/core/functions/table/match.cpp index dcfab017..2b6117d6 100644 --- a/src/core/functions/table/match.cpp +++ b/src/core/functions/table/match.cpp @@ -1132,8 +1132,6 @@ PGQMatchFunction::MatchBindReplace(ClientContext &context, // Register functions //------------------------------------------------------------------------------ void CoreTableFunctions::RegisterMatchTableFunction(DatabaseInstance &db) { - // TableFunctionSet match_set("duckpgq_match"); - // match_set.AddFunction(PGQMatchFunction()); ExtensionUtil::RegisterFunction(db, PGQMatchFunction()); } From b9c209ec8b4c56a8a6e5ed0e4e3017d1c5599522 Mon Sep 17 00:00:00 2001 From: dtenwolde Date: Mon, 12 Aug 2024 14:33:18 +0200 Subject: [PATCH 150/249] Remove unused optimizer file --- src/core/optimizer/duckpgq_optimizer.cpp | 8 -------- 1 file changed, 8 deletions(-) delete mode 100644 src/core/optimizer/duckpgq_optimizer.cpp diff --git a/src/core/optimizer/duckpgq_optimizer.cpp b/src/core/optimizer/duckpgq_optimizer.cpp deleted file mode 100644 index 60c882ee..00000000 --- a/src/core/optimizer/duckpgq_optimizer.cpp +++ /dev/null @@ -1,8 +0,0 @@ -#include "duckpgq/core/optimizer/duckpgq_optimizer.hpp" - -namespace duckpgq { -namespace core { - -} // namespace core - -} // namespace duckpgq \ No newline at end of file From 720972f20ea8bca34ce6be2b01f57d52e5f6b63a Mon Sep 17 00:00:00 2001 From: dtenwolde Date: Mon, 12 Aug 2024 14:33:40 +0200 Subject: [PATCH 151/249] Various fixes --- src/core/optimizer/CMakeLists.txt | 1 - src/core/optimizer/path_finding_optimizer_rule.cpp | 7 +++---- src/core/option/duckpgq_option.cpp | 4 ++-- src/core/parser/duckpgq_parser.cpp | 2 +- 4 files changed, 6 insertions(+), 8 deletions(-) diff --git a/src/core/optimizer/CMakeLists.txt b/src/core/optimizer/CMakeLists.txt index 780f049f..ae7ba279 100644 --- a/src/core/optimizer/CMakeLists.txt +++ b/src/core/optimizer/CMakeLists.txt @@ -1,5 +1,4 @@ set(EXTENSION_SOURCES - ${CMAKE_CURRENT_SOURCE_DIR}/duckpgq_optimizer.cpp ${CMAKE_CURRENT_SOURCE_DIR}/path_finding_optimizer_rule.cpp ${EXTENSION_SOURCES} PARENT_SCOPE diff --git a/src/core/optimizer/path_finding_optimizer_rule.cpp b/src/core/optimizer/path_finding_optimizer_rule.cpp index c0428ca5..12b31793 100644 --- a/src/core/optimizer/path_finding_optimizer_rule.cpp +++ b/src/core/optimizer/path_finding_optimizer_rule.cpp @@ -132,15 +132,14 @@ void DuckpgqOptimizerExtension::DuckpgqOptimizeFunction(OptimizerExtensionInput //------------------------------------------------------------------------------ -// Register functions +// Register optimizer //------------------------------------------------------------------------------ void CorePGQOptimizer::RegisterPathFindingOptimizerRule( DatabaseInstance &db) { auto &config = DBConfig::GetConfig(db); - config.optimizer_extensions.push_back(DuckpgqOptimizerExtension());} - - + config.optimizer_extensions.push_back(DuckpgqOptimizerExtension()); +} } // namespace core } // namespace duckpgq \ No newline at end of file diff --git a/src/core/option/duckpgq_option.cpp b/src/core/option/duckpgq_option.cpp index 41f22d25..4806078d 100644 --- a/src/core/option/duckpgq_option.cpp +++ b/src/core/option/duckpgq_option.cpp @@ -6,7 +6,7 @@ namespace core { //------------------------------------------------------------------------------ -// Register functions +// Register option //------------------------------------------------------------------------------ void CorePGQOptions::RegisterExperimentalPathFindingOperator( DatabaseInstance &db) { @@ -17,7 +17,7 @@ void CorePGQOptions::RegisterExperimentalPathFindingOperator( } //------------------------------------------------------------------------------ -// Register functions +// Register option //------------------------------------------------------------------------------ void CorePGQOptions::RegisterPathFindingTaskSize( DatabaseInstance &db) { diff --git a/src/core/parser/duckpgq_parser.cpp b/src/core/parser/duckpgq_parser.cpp index f29ef65f..ab342c8b 100644 --- a/src/core/parser/duckpgq_parser.cpp +++ b/src/core/parser/duckpgq_parser.cpp @@ -185,7 +185,7 @@ duckpgq_plan(ParserExtensionInfo *, ClientContext &context, //------------------------------------------------------------------------------ -// Register functions +// Register parser //------------------------------------------------------------------------------ void CorePGQParser::RegisterPGQParserExtension( DatabaseInstance &db) { From aaafc370df94df4af6c38a5fbc7e8f073d561136 Mon Sep 17 00:00:00 2001 From: dtenwolde Date: Mon, 12 Aug 2024 16:01:29 +0200 Subject: [PATCH 152/249] Add function to get option value --- .../optimizer/path_finding_optimizer_rule.cpp | 16 ++++++++++------ .../optimizer/path_finding_optimizer_rule.hpp | 1 + 2 files changed, 11 insertions(+), 6 deletions(-) diff --git a/src/core/optimizer/path_finding_optimizer_rule.cpp b/src/core/optimizer/path_finding_optimizer_rule.cpp index 12b31793..5fe96e91 100644 --- a/src/core/optimizer/path_finding_optimizer_rule.cpp +++ b/src/core/optimizer/path_finding_optimizer_rule.cpp @@ -18,6 +18,15 @@ namespace duckpgq { namespace core { +bool DuckpgqOptimizerExtension::GetPathFindingOption(ClientContext &context) { + auto& client_config = ClientConfig::GetConfig(context); + auto const path_finding_operator_option = client_config.set_variables.find("experimental_path_finding_operator"); + if (path_finding_operator_option == client_config.set_variables.end()) { + return false; // If the path finding operator is not enabled, we do not need to do anything + } + return path_finding_operator_option->second.GetValue(); +} + bool DuckpgqOptimizerExtension::InsertPathFindingOperator(LogicalOperator &op, ClientContext &context) { unique_ptr function_expression; string mode; @@ -119,12 +128,7 @@ bool DuckpgqOptimizerExtension::InsertPathFindingOperator(LogicalOperator &op, C void DuckpgqOptimizerExtension::DuckpgqOptimizeFunction(OptimizerExtensionInput &input, duckdb::unique_ptr &plan) { - auto& client_config = ClientConfig::GetConfig(input.context); - auto const path_finding_operator_option = client_config.set_variables.find("experimental_path_finding_operator"); - if (path_finding_operator_option == client_config.set_variables.end()) { - return; // If the path finding operator is not enabled, we do not need to do anything - } - if (!path_finding_operator_option->second.GetValue()) { + if (!GetPathFindingOption(input.context)) { return; } InsertPathFindingOperator(*plan, input.context); diff --git a/src/include/duckpgq/core/optimizer/path_finding_optimizer_rule.hpp b/src/include/duckpgq/core/optimizer/path_finding_optimizer_rule.hpp index 6a3ddd14..4c12397e 100644 --- a/src/include/duckpgq/core/optimizer/path_finding_optimizer_rule.hpp +++ b/src/include/duckpgq/core/optimizer/path_finding_optimizer_rule.hpp @@ -11,6 +11,7 @@ class DuckpgqOptimizerExtension : public OptimizerExtension { optimize_function = DuckpgqOptimizeFunction; } + static bool GetPathFindingOption(ClientContext &context); static bool InsertPathFindingOperator(LogicalOperator &op, ClientContext &context); static void DuckpgqOptimizeFunction(OptimizerExtensionInput &input, From d0a9038e2a852855b2207d298bc9c6c816868b1b Mon Sep 17 00:00:00 2001 From: dtenwolde Date: Mon, 12 Aug 2024 16:01:43 +0200 Subject: [PATCH 153/249] Fix build errors --- src/core/functions/scalar/iterativelength.cpp | 2 +- src/core/functions/scalar/shortest_path.cpp | 10 +- src/core/functions/table/match.cpp | 125 +----------------- .../duckpgq/core/functions/table/match.hpp | 5 - 4 files changed, 5 insertions(+), 137 deletions(-) diff --git a/src/core/functions/scalar/iterativelength.cpp b/src/core/functions/scalar/iterativelength.cpp index 15de84a2..4fead0a9 100644 --- a/src/core/functions/scalar/iterativelength.cpp +++ b/src/core/functions/scalar/iterativelength.cpp @@ -171,7 +171,7 @@ void CoreScalarFunctions::RegisterIterativeLengthScalarFunction( ExtensionUtil::RegisterFunction( db, ScalarFunction("iterativelength", - ScalarFunction({LogicalType::BIGINT, LogicalType::BIGINT, LogicalType::VARCHAR}, + {LogicalType::BIGINT, LogicalType::BIGINT, LogicalType::VARCHAR}, LogicalType::BIGINT, IterativeLengthFunction, IterativeLengthFunctionData::IterativeLengthBind)); } diff --git a/src/core/functions/scalar/shortest_path.cpp b/src/core/functions/scalar/shortest_path.cpp index 8812b5f5..9b0b488e 100644 --- a/src/core/functions/scalar/shortest_path.cpp +++ b/src/core/functions/scalar/shortest_path.cpp @@ -234,17 +234,13 @@ void CoreScalarFunctions::RegisterShortestPathScalarFunction( LogicalType::BIGINT, LogicalType::BIGINT}, LogicalType::LIST(LogicalType::BIGINT), ShortestPathFunction, - IterativeLengthFunctionData::IterativeLengthBind); - ExtensionUtil::RegisterFunction( + IterativeLengthFunctionData::IterativeLengthBind)); + + ExtensionUtil::RegisterFunction( db, ScalarFunction("shortestpath", {LogicalType::BIGINT, LogicalType::BIGINT, LogicalType::VARCHAR}, LogicalType::LIST(LogicalType::BIGINT), ShortestPathFunction, IterativeLengthFunctionData::IterativeLengthBind)); - - - - - return CreateScalarFunctionInfo(fun); } } // namespace core diff --git a/src/core/functions/table/match.cpp b/src/core/functions/table/match.cpp index 2b6117d6..b8b969f2 100644 --- a/src/core/functions/table/match.cpp +++ b/src/core/functions/table/match.cpp @@ -204,130 +204,6 @@ unique_ptr PGQMatchFunction::CreateCountCTESubquery() { return temp_cte_select_subquery; } -unique_ptr -PGQMatchFunction::CreateCSRCTE(const shared_ptr &edge_table, - const string &prev_binding, - const string &edge_binding, - const string &next_binding) { - auto csr_edge_id_constant = make_uniq(Value::INTEGER(0)); - auto count_create_edge_select = GetCountTable(edge_table, prev_binding); - - auto cast_subquery_expr = make_uniq(); - auto cast_select_node = make_uniq(); - - vector> csr_vertex_children; - csr_vertex_children.push_back( - make_uniq(Value::INTEGER(0))); - - auto count_create_vertex_expr = GetCountTable(edge_table, prev_binding); - - csr_vertex_children.push_back(std::move(count_create_vertex_expr)); - - csr_vertex_children.push_back( - make_uniq("dense_id", "sub")); - csr_vertex_children.push_back(make_uniq("cnt", "sub")); - - auto create_vertex_function = make_uniq( - "create_csr_vertex", std::move(csr_vertex_children)); - vector> sum_children; - sum_children.push_back(std::move(create_vertex_function)); - auto sum_function = - make_uniq("sum", std::move(sum_children)); - - auto inner_select_statement = make_uniq(); - auto inner_select_node = make_uniq(); - - auto source_rowid_colref = - make_uniq("rowid", prev_binding); - source_rowid_colref->alias = "dense_id"; - - auto count_create_inner_expr = make_uniq(); - count_create_inner_expr->subquery_type = SubqueryType::SCALAR; - auto edge_src_colref = - make_uniq(edge_table->source_fk[0], edge_binding); - vector> inner_count_children; - inner_count_children.push_back(std::move(edge_src_colref)); - auto inner_count_function = - make_uniq("count", std::move(inner_count_children)); - inner_count_function->alias = "cnt"; - - inner_select_node->select_list.push_back(std::move(source_rowid_colref)); - inner_select_node->select_list.push_back(std::move(inner_count_function)); - auto source_rowid_colref_1 = - make_uniq("rowid", prev_binding); - expression_map_t grouping_expression_map; - inner_select_node->groups.group_expressions.push_back( - std::move(source_rowid_colref_1)); - GroupingSet grouping_set = {0}; - inner_select_node->groups.grouping_sets.push_back(grouping_set); - - auto inner_join_ref = make_uniq(JoinRefType::REGULAR); - inner_join_ref->type = JoinType::LEFT; - auto left_base_ref = make_uniq(); - left_base_ref->table_name = edge_table->source_reference; - left_base_ref->alias = prev_binding; - auto right_base_ref = make_uniq(); - right_base_ref->table_name = edge_table->table_name; - right_base_ref->alias = edge_binding; - inner_join_ref->left = std::move(left_base_ref); - inner_join_ref->right = std::move(right_base_ref); - - auto edge_join_colref = - make_uniq(edge_table->source_fk[0], edge_binding); - auto vertex_join_colref = - make_uniq(edge_table->source_pk[0], prev_binding); - - inner_join_ref->condition = make_uniq( - ExpressionType::COMPARE_EQUAL, std::move(edge_join_colref), - std::move(vertex_join_colref)); - inner_select_node->from_table = std::move(inner_join_ref); - inner_select_statement->node = std::move(inner_select_node); - - auto inner_from_subquery = - make_uniq(std::move(inner_select_statement), "sub"); - - cast_select_node->from_table = std::move(inner_from_subquery); - - cast_select_node->select_list.push_back(std::move(sum_function)); - auto cast_select_stmt = make_uniq(); - cast_select_stmt->node = std::move(cast_select_node); - cast_subquery_expr->subquery = std::move(cast_select_stmt); - cast_subquery_expr->subquery_type = SubqueryType::SCALAR; - - auto src_rowid_colref = make_uniq("rowid", prev_binding); - auto dst_rowid_colref = make_uniq("rowid", next_binding); - auto edge_rowid_colref = - make_uniq("rowid", edge_binding); - - auto cast_expression = make_uniq( - LogicalType::BIGINT, std::move(cast_subquery_expr)); - - vector> csr_edge_children; - csr_edge_children.push_back(std::move(csr_edge_id_constant)); - csr_edge_children.push_back(std::move(count_create_edge_select)); - csr_edge_children.push_back(std::move(cast_expression)); - csr_edge_children.push_back(std::move(src_rowid_colref)); - csr_edge_children.push_back(std::move(dst_rowid_colref)); - csr_edge_children.push_back(std::move(edge_rowid_colref)); - - auto outer_select_node = make_uniq(); - - auto create_csr_edge_function = make_uniq( - "create_csr_edge", std::move(csr_edge_children)); - create_csr_edge_function->alias = "temp"; - - outer_select_node->select_list.push_back(std::move(create_csr_edge_function)); - outer_select_node->from_table = - GetJoinRef(edge_table, edge_binding, prev_binding, next_binding); - auto outer_select_statement = make_uniq(); - - outer_select_statement->node = std::move(outer_select_node); - auto csr_subquery = make_uniq(); - csr_subquery->subquery = std::move(outer_select_statement); - csr_subquery->subquery_type = SubqueryType::SCALAR; - return csr_subquery; -} - void PGQMatchFunction::EdgeTypeAny( const shared_ptr &edge_table, const string &edge_binding, const string &prev_binding, @@ -799,6 +675,7 @@ void PGQMatchFunction::AddPathFinding( //! END //! FROM (SELECT count(cte1.temp) * 0 as temp from cte1) __x // TODO wrap this with the config option for experimental path finding + // auto csr = CreateCSRCTE(edge_table, prev_binding, edge_binding, next_binding); // auto src_tasks = make_uniq("rowid", prev_binding); // auto dst_tasks = make_uniq("rowid", next_binding); diff --git a/src/include/duckpgq/core/functions/table/match.hpp b/src/include/duckpgq/core/functions/table/match.hpp index 2c09dd47..12d34d3e 100644 --- a/src/include/duckpgq/core/functions/table/match.hpp +++ b/src/include/duckpgq/core/functions/table/match.hpp @@ -66,11 +66,6 @@ struct PGQMatchFunction : public TableFunction { static unique_ptr CreateCountCTESubquery(); - static unique_ptr - CreateCSRCTE(const shared_ptr &edge_table, - const string &edge_binding, const string &prev_binding, - const string &next_binding); - static unique_ptr CreateWhereClause(vector> &conditions); From 7fa2095cfbb24f31529605a9c108ba06b2ead61d Mon Sep 17 00:00:00 2001 From: dtenwolde Date: Wed, 14 Aug 2024 11:58:03 +0200 Subject: [PATCH 154/249] Set default experimental path finding option to false --- src/core/option/duckpgq_option.cpp | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/src/core/option/duckpgq_option.cpp b/src/core/option/duckpgq_option.cpp index 4806078d..4ad21f10 100644 --- a/src/core/option/duckpgq_option.cpp +++ b/src/core/option/duckpgq_option.cpp @@ -13,7 +13,7 @@ void CorePGQOptions::RegisterExperimentalPathFindingOperator( auto &config = DBConfig::GetConfig(db); config.AddExtensionOption("experimental_path_finding_operator", "Enables the experimental path finding operator to be triggered", - LogicalType::BOOLEAN, Value(true)); + LogicalType::BOOLEAN, Value(false)); } //------------------------------------------------------------------------------ From 13a0cf61b7404d8eeca4954601e2cae3118dff0b Mon Sep 17 00:00:00 2001 From: dtenwolde Date: Wed, 14 Aug 2024 12:03:08 +0200 Subject: [PATCH 155/249] Get the option from the dbconfig --- src/core/optimizer/path_finding_optimizer_rule.cpp | 6 +++--- 1 file changed, 3 insertions(+), 3 deletions(-) diff --git a/src/core/optimizer/path_finding_optimizer_rule.cpp b/src/core/optimizer/path_finding_optimizer_rule.cpp index 5fe96e91..901e757a 100644 --- a/src/core/optimizer/path_finding_optimizer_rule.cpp +++ b/src/core/optimizer/path_finding_optimizer_rule.cpp @@ -19,9 +19,9 @@ namespace duckpgq { namespace core { bool DuckpgqOptimizerExtension::GetPathFindingOption(ClientContext &context) { - auto& client_config = ClientConfig::GetConfig(context); - auto const path_finding_operator_option = client_config.set_variables.find("experimental_path_finding_operator"); - if (path_finding_operator_option == client_config.set_variables.end()) { + auto& db_config = DBConfig::GetConfig(context); + auto const path_finding_operator_option = db_config.options.set_variables.find("experimental_path_finding_operator"); + if (path_finding_operator_option == db_config.options.set_variables.end()) { return false; // If the path finding operator is not enabled, we do not need to do anything } return path_finding_operator_option->second.GetValue(); From 02f4a5a69250503e0cdc276e962b946a1fb4d54c Mon Sep 17 00:00:00 2001 From: dtenwolde Date: Wed, 14 Aug 2024 12:34:09 +0200 Subject: [PATCH 156/249] Getting the option now works correctly --- src/core/optimizer/path_finding_optimizer_rule.cpp | 9 +++------ 1 file changed, 3 insertions(+), 6 deletions(-) diff --git a/src/core/optimizer/path_finding_optimizer_rule.cpp b/src/core/optimizer/path_finding_optimizer_rule.cpp index 901e757a..d2ff5ffd 100644 --- a/src/core/optimizer/path_finding_optimizer_rule.cpp +++ b/src/core/optimizer/path_finding_optimizer_rule.cpp @@ -19,12 +19,9 @@ namespace duckpgq { namespace core { bool DuckpgqOptimizerExtension::GetPathFindingOption(ClientContext &context) { - auto& db_config = DBConfig::GetConfig(context); - auto const path_finding_operator_option = db_config.options.set_variables.find("experimental_path_finding_operator"); - if (path_finding_operator_option == db_config.options.set_variables.end()) { - return false; // If the path finding operator is not enabled, we do not need to do anything - } - return path_finding_operator_option->second.GetValue(); + Value value; + context.TryGetCurrentSetting("experimental_path_finding_operator", value); + return value.GetValue(); } bool DuckpgqOptimizerExtension::InsertPathFindingOperator(LogicalOperator &op, ClientContext &context) { From dc7d66fa31b24496c2432285663cf3e40408f909 Mon Sep 17 00:00:00 2001 From: dtenwolde Date: Wed, 14 Aug 2024 12:34:32 +0200 Subject: [PATCH 157/249] Register shortest path and iterativelength functions correctly --- src/core/functions/scalar/iterativelength.cpp | 32 +++++++++++-------- src/core/functions/scalar/shortest_path.cpp | 28 +++++++++------- 2 files changed, 34 insertions(+), 26 deletions(-) diff --git a/src/core/functions/scalar/iterativelength.cpp b/src/core/functions/scalar/iterativelength.cpp index 4fead0a9..db639cf3 100644 --- a/src/core/functions/scalar/iterativelength.cpp +++ b/src/core/functions/scalar/iterativelength.cpp @@ -155,25 +155,29 @@ static void IterativeLengthFunction(DataChunk &args, ExpressionState &state, duckpgq_state->csr_to_delete.insert(info.csr_id); } -//------------------------------------------------------------------------------ -// Register functions -//------------------------------------------------------------------------------ -void CoreScalarFunctions::RegisterIterativeLengthScalarFunction( - DatabaseInstance &db) { - ExtensionUtil::RegisterFunction( - db, - ScalarFunction("iterativelength", +ScalarFunctionSet GetIterativeLengthFunction() { + ScalarFunctionSet set("iterativelength"); + + set.AddFunction(ScalarFunction( {LogicalType::INTEGER, LogicalType::BIGINT, LogicalType::BIGINT, LogicalType::BIGINT}, LogicalType::BIGINT, IterativeLengthFunction, IterativeLengthFunctionData::IterativeLengthBind)); - ExtensionUtil::RegisterFunction( - db, - ScalarFunction("iterativelength", - {LogicalType::BIGINT, LogicalType::BIGINT, LogicalType::VARCHAR}, - LogicalType::BIGINT, IterativeLengthFunction, - IterativeLengthFunctionData::IterativeLengthBind)); + set.AddFunction( + ScalarFunction( + {LogicalType::BIGINT, LogicalType::BIGINT, LogicalType::VARCHAR}, + LogicalType::BIGINT, IterativeLengthFunction, + IterativeLengthFunctionData::IterativeLengthBind)); + return set; +} + +//------------------------------------------------------------------------------ +// Register functions +//------------------------------------------------------------------------------ +void CoreScalarFunctions::RegisterIterativeLengthScalarFunction( + DatabaseInstance &db) { + ExtensionUtil::RegisterFunction(db, GetIterativeLengthFunction()); } } // namespace core diff --git a/src/core/functions/scalar/shortest_path.cpp b/src/core/functions/scalar/shortest_path.cpp index 9b0b488e..785567d4 100644 --- a/src/core/functions/scalar/shortest_path.cpp +++ b/src/core/functions/scalar/shortest_path.cpp @@ -222,25 +222,29 @@ static void ShortestPathFunction(DataChunk &args, ExpressionState &state, duckpgq_state->csr_to_delete.insert(info.csr_id); } +ScalarFunctionSet GetShortestPathFunction() { + ScalarFunctionSet set("shortestpath"); + + set.AddFunction(ScalarFunction( + {LogicalType::INTEGER, LogicalType::BIGINT, LogicalType::BIGINT, + LogicalType::BIGINT}, + LogicalType::LIST(LogicalType::BIGINT), ShortestPathFunction, + IterativeLengthFunctionData::IterativeLengthBind)); + + set.AddFunction(ScalarFunction({LogicalType::BIGINT, LogicalType::BIGINT, LogicalType::VARCHAR}, + LogicalType::LIST(LogicalType::BIGINT), ShortestPathFunction, + IterativeLengthFunctionData::IterativeLengthBind)); + return set; +} + //------------------------------------------------------------------------------ // Register functions //------------------------------------------------------------------------------ void CoreScalarFunctions::RegisterShortestPathScalarFunction( DatabaseInstance &db) { - ExtensionUtil::RegisterFunction( - db, - ScalarFunction("shortestpath", - {LogicalType::INTEGER, LogicalType::BIGINT, - LogicalType::BIGINT, LogicalType::BIGINT}, - LogicalType::LIST(LogicalType::BIGINT), - ShortestPathFunction, - IterativeLengthFunctionData::IterativeLengthBind)); ExtensionUtil::RegisterFunction( - db, - ScalarFunction("shortestpath", {LogicalType::BIGINT, LogicalType::BIGINT, LogicalType::VARCHAR}, - LogicalType::LIST(LogicalType::BIGINT), ShortestPathFunction, - IterativeLengthFunctionData::IterativeLengthBind)); + db, GetShortestPathFunction()); } } // namespace core From 59fd63e6cdf761eefc9ef1d9c859090ac9e54638 Mon Sep 17 00:00:00 2001 From: dtenwolde Date: Wed, 14 Aug 2024 12:51:30 +0200 Subject: [PATCH 158/249] Add function to get csr id or throw error --- .../cheapest_path_length_function_data.cpp | 9 +--- .../iterative_length_function_data.cpp | 10 ++--- ...l_clustering_coefficient_function_data.cpp | 9 +--- src/core/utils/compressed_sparse_row.cpp | 41 ++++++++----------- .../core/utils/compressed_sparse_row.hpp | 1 + 5 files changed, 23 insertions(+), 47 deletions(-) diff --git a/src/core/functions/function_data/cheapest_path_length_function_data.cpp b/src/core/functions/function_data/cheapest_path_length_function_data.cpp index 305fd3aa..d59d1ea1 100644 --- a/src/core/functions/function_data/cheapest_path_length_function_data.cpp +++ b/src/core/functions/function_data/cheapest_path_length_function_data.cpp @@ -1,6 +1,5 @@ #include "duckpgq/core/functions/function_data/cheapest_path_length_function_data.hpp" #include "duckpgq/core/utils/duckpgq_utils.hpp" -#include "duckdb/execution/expression_executor.hpp" namespace duckpgq { @@ -10,14 +9,8 @@ unique_ptr CheapestPathLengthFunctionData::CheapestPathLengthBind( ClientContext &context, ScalarFunction &bound_function, vector> &arguments) { - if (!arguments[0]->IsFoldable()) { - throw InvalidInputException("Id must be constant."); - } - + auto csr_id = GetCSRId(arguments[0], context); auto duckpgq_state = GetDuckPGQState(context); - - int32_t csr_id = ExpressionExecutor::EvaluateScalar(context, *arguments[0]) - .GetValue(); CSR *csr = duckpgq_state->GetCSR(csr_id); if (!(csr->initialized_v && csr->initialized_e && csr->initialized_w)) { diff --git a/src/core/functions/function_data/iterative_length_function_data.cpp b/src/core/functions/function_data/iterative_length_function_data.cpp index f98542e2..d8ef5990 100644 --- a/src/core/functions/function_data/iterative_length_function_data.cpp +++ b/src/core/functions/function_data/iterative_length_function_data.cpp @@ -1,7 +1,8 @@ #include "duckpgq/core/functions/function_data/iterative_length_function_data.hpp" -#include "duckdb/execution/expression_executor.hpp" #include "duckpgq/common.hpp" +#include + namespace duckpgq { namespace core { @@ -23,12 +24,7 @@ unique_ptr IterativeLengthFunctionData::IterativeLengthBind( string table_to_scan = ExpressionExecutor::EvaluateScalar(context, *arguments[2]).GetValue(); return make_uniq(context, table_to_scan, 0); } - if (!arguments[0]->IsFoldable()) { - throw InvalidInputException("Id must be constant."); - } - - int32_t csr_id = ExpressionExecutor::EvaluateScalar(context, *arguments[0]) - .GetValue(); + auto csr_id = GetCSRId(arguments[0], context); return make_uniq(context, "", csr_id); } diff --git a/src/core/functions/function_data/local_clustering_coefficient_function_data.cpp b/src/core/functions/function_data/local_clustering_coefficient_function_data.cpp index cbc1672e..0c1d2147 100644 --- a/src/core/functions/function_data/local_clustering_coefficient_function_data.cpp +++ b/src/core/functions/function_data/local_clustering_coefficient_function_data.cpp @@ -1,5 +1,5 @@ #include "duckpgq/core/functions/function_data/local_clustering_coefficient_function_data.hpp" -#include "duckdb/execution/expression_executor.hpp" +#include namespace duckpgq { @@ -15,12 +15,7 @@ unique_ptr LocalClusteringCoefficientFunctionData::LocalClusteringCoefficientBind( ClientContext &context, ScalarFunction &bound_function, vector> &arguments) { - if (!arguments[0]->IsFoldable()) { - throw InvalidInputException("Id must be constant."); - } - - int32_t csr_id = ExpressionExecutor::EvaluateScalar(context, *arguments[0]) - .GetValue(); + auto csr_id = GetCSRId(arguments[0], context); return make_uniq(context, csr_id); } diff --git a/src/core/utils/compressed_sparse_row.cpp b/src/core/utils/compressed_sparse_row.cpp index b3757743..dafdae5e 100644 --- a/src/core/utils/compressed_sparse_row.cpp +++ b/src/core/utils/compressed_sparse_row.cpp @@ -52,48 +52,32 @@ unique_ptr CSRFunctionData::CSRVertexBind(ClientContext &context, ScalarFunction &bound_function, vector> &arguments) { - if (!arguments[0]->IsFoldable()) { - throw InvalidInputException("Id must be constant."); - } - - Value id = ExpressionExecutor::EvaluateScalar(context, *arguments[0]); + auto csr_id = GetCSRId(arguments[0], context); if (arguments.size() == 4) { auto logical_type = LogicalType::SQLNULL; - return make_uniq(context, id.GetValue(), - logical_type); + return make_uniq(context, csr_id, logical_type); } - return make_uniq(context, id.GetValue(), - arguments[3]->return_type); - + return make_uniq(context, csr_id, arguments[3]->return_type); } unique_ptr CSRFunctionData::CSREdgeBind(ClientContext &context, ScalarFunction &bound_function, vector> &arguments) { - if (!arguments[0]->IsFoldable()) { - throw InvalidInputException("Id must be constant."); - } - Value id = ExpressionExecutor::EvaluateScalar(context, *arguments[0]); + auto csr_id = GetCSRId(arguments[0], context); if (arguments.size() == 7) { - return make_uniq(context, id.GetValue(), + return make_uniq(context, csr_id, arguments[6]->return_type); - } else { - auto logical_type = LogicalType::SQLNULL; - return make_uniq(context, id.GetValue(), - logical_type); } + auto logical_type = LogicalType::SQLNULL; + return make_uniq(context, csr_id, logical_type); } unique_ptr CSRFunctionData::CSRBind(ClientContext &context, ScalarFunction &bound_function, vector> &arguments) { - if (!arguments[0]->IsFoldable()) { - throw InvalidInputException("Id must be constant."); - } - Value id = ExpressionExecutor::EvaluateScalar(context, *arguments[0]); - return make_uniq(context, id.GetValue(), - LogicalType::BOOLEAN); + auto csr_id = GetCSRId(arguments[0], context); + return make_uniq(context, csr_id, LogicalType::BOOLEAN); } // Helper function to create a ColumnRefExpression with alias @@ -477,6 +461,13 @@ unique_ptr CreateDirectedCSRCTE(const shared_ptrquery = std::move(outer_select_statement); return info; } + +int32_t GetCSRId(const unique_ptr &expr, ClientContext &context) { + if (!expr->IsFoldable()) { + throw InvalidInputException("CSR ID must be constant."); + } + return ExpressionExecutor::EvaluateScalar(context, *expr).GetValue(); +} } // namespace core } // namespace duckpgq \ No newline at end of file diff --git a/src/include/duckpgq/core/utils/compressed_sparse_row.hpp b/src/include/duckpgq/core/utils/compressed_sparse_row.hpp index bf80794f..e64067fc 100644 --- a/src/include/duckpgq/core/utils/compressed_sparse_row.hpp +++ b/src/include/duckpgq/core/utils/compressed_sparse_row.hpp @@ -78,6 +78,7 @@ unique_ptr GetJoinRef(const shared_ptr &edge_table, unique_ptr GetCountTable(const shared_ptr &edge_table, const string &prev_binding); unique_ptr CreateColumnRef(const std::string &column_name, const std::string &table_name, const std::string &alias); void SetupSelectNode(unique_ptr &select_node, const shared_ptr &edge_table, bool reverse = false); +int32_t GetCSRId(const unique_ptr &expr, ClientContext &context); } // namespace core From 57ceaf4f2cad50e3aba57ba21560ed439e469837 Mon Sep 17 00:00:00 2001 From: dtenwolde Date: Wed, 14 Aug 2024 14:45:58 +0200 Subject: [PATCH 159/249] Fix path in comment --- .../local_clustering_coefficient_function_data.hpp | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/src/include/duckpgq/core/functions/function_data/local_clustering_coefficient_function_data.hpp b/src/include/duckpgq/core/functions/function_data/local_clustering_coefficient_function_data.hpp index 696ea755..d3fb177a 100644 --- a/src/include/duckpgq/core/functions/function_data/local_clustering_coefficient_function_data.hpp +++ b/src/include/duckpgq/core/functions/function_data/local_clustering_coefficient_function_data.hpp @@ -1,7 +1,7 @@ //===----------------------------------------------------------------------===// // DuckPGQ // -// duckpgq/functions/function_data/local_clustering_coefficient_function_data.hpp +// duckpgq/core/functions/function_data/local_clustering_coefficient_function_data.hpp // // //===----------------------------------------------------------------------===// From d5847ce0e3f0419d199c5af2c48d26dd9946db6d Mon Sep 17 00:00:00 2001 From: dtenwolde Date: Wed, 14 Aug 2024 14:46:16 +0200 Subject: [PATCH 160/249] Add separate UDF for shortest path operator to keep logic separate --- src/core/functions/scalar/CMakeLists.txt | 2 ++ .../scalar/shortest_path_operator.cpp | 20 +++++++++++ .../shortest_path_operator_function_data.hpp | 35 +++++++++++++++++++ src/include/duckpgq/core/functions/scalar.hpp | 4 ++- 4 files changed, 60 insertions(+), 1 deletion(-) create mode 100644 src/core/functions/scalar/shortest_path_operator.cpp create mode 100644 src/include/duckpgq/core/functions/function_data/shortest_path_operator_function_data.hpp diff --git a/src/core/functions/scalar/CMakeLists.txt b/src/core/functions/scalar/CMakeLists.txt index aecc74e2..75f02a88 100644 --- a/src/core/functions/scalar/CMakeLists.txt +++ b/src/core/functions/scalar/CMakeLists.txt @@ -9,6 +9,8 @@ set(EXTENSION_SOURCES ${CMAKE_CURRENT_SOURCE_DIR}/iterativelength_bidirectional.cpp ${CMAKE_CURRENT_SOURCE_DIR}/reachability.cpp ${CMAKE_CURRENT_SOURCE_DIR}/shortest_path.cpp + ${CMAKE_CURRENT_SOURCE_DIR}/shortest_path_operator.cpp + ${CMAKE_CURRENT_SOURCE_DIR}/csr_creation.cpp ${CMAKE_CURRENT_SOURCE_DIR}/local_clustering_coefficient.cpp PARENT_SCOPE diff --git a/src/core/functions/scalar/shortest_path_operator.cpp b/src/core/functions/scalar/shortest_path_operator.cpp new file mode 100644 index 00000000..270648e7 --- /dev/null +++ b/src/core/functions/scalar/shortest_path_operator.cpp @@ -0,0 +1,20 @@ +#include "duckpgq/core/functions/function_data/shortest_path_operator_function_data.hpp" + +namespace duckpgq { + +namespace core { + +//------------------------------------------------------------------------------ +// Register functions +//------------------------------------------------------------------------------ +void CoreScalarFunctions::RegisterShortestPathScalarFunction( + DatabaseInstance &db) { + + ExtensionUtil::RegisterFunction( + db, GetShortestPathFunction()); +} + + +} // namespace core + +} // namespace duckpgq \ No newline at end of file diff --git a/src/include/duckpgq/core/functions/function_data/shortest_path_operator_function_data.hpp b/src/include/duckpgq/core/functions/function_data/shortest_path_operator_function_data.hpp new file mode 100644 index 00000000..0e5b1fa5 --- /dev/null +++ b/src/include/duckpgq/core/functions/function_data/shortest_path_operator_function_data.hpp @@ -0,0 +1,35 @@ +//===----------------------------------------------------------------------===// +// DuckPGQ +// +// duckpgq/core/functions/function_data/shortest_path_operator_function_data.hpp +// +// +//===----------------------------------------------------------------------===// + +#pragma once +#include "duckdb/main/client_context.hpp" +#include "duckpgq/common.hpp" + + +namespace duckpgq { + +namespace core { + +struct IterativeLengthFunctionData final : FunctionData { + ClientContext &context; + string table_to_scan; + int32_t csr_id; + + IterativeLengthFunctionData(ClientContext &context, string table_to_scan, int32_t csr_id) + : context(context), table_to_scan(table_to_scan), csr_id(csr_id) {} + static unique_ptr + IterativeLengthBind(ClientContext &context, ScalarFunction &bound_function, + vector> &arguments); + + unique_ptr Copy() const override; + bool Equals(const FunctionData &other_p) const override; +}; + +} // namespace core + +} // namespace duckpgq \ No newline at end of file diff --git a/src/include/duckpgq/core/functions/scalar.hpp b/src/include/duckpgq/core/functions/scalar.hpp index ed1a8541..1e502559 100644 --- a/src/include/duckpgq/core/functions/scalar.hpp +++ b/src/include/duckpgq/core/functions/scalar.hpp @@ -17,6 +17,8 @@ struct CoreScalarFunctions { RegisterLocalClusteringCoefficientScalarFunction(db); RegisterReachabilityScalarFunction(db); RegisterShortestPathScalarFunction(db); + RegisterShortestPathOperatorScalarFunction(db); + } private: @@ -30,7 +32,7 @@ struct CoreScalarFunctions { static void RegisterLocalClusteringCoefficientScalarFunction(DatabaseInstance &db); static void RegisterReachabilityScalarFunction(DatabaseInstance &db); static void RegisterShortestPathScalarFunction(DatabaseInstance &db); - + static void RegisterShortestPathOperatorScalarFunction(DatabaseInstance &db); }; From 57f48928db042b1c1f845c3c8c6b912dfa83e43e Mon Sep 17 00:00:00 2001 From: dtenwolde Date: Wed, 14 Aug 2024 14:49:38 +0200 Subject: [PATCH 161/249] Register function --- src/core/functions/scalar/CMakeLists.txt | 1 - src/core/functions/scalar/shortest_path.cpp | 4 +--- src/core/functions/scalar/shortest_path_operator.cpp | 7 ++++++- .../shortest_path_operator_function_data.hpp | 8 ++++---- 4 files changed, 11 insertions(+), 9 deletions(-) diff --git a/src/core/functions/scalar/CMakeLists.txt b/src/core/functions/scalar/CMakeLists.txt index 75f02a88..ba22a32e 100644 --- a/src/core/functions/scalar/CMakeLists.txt +++ b/src/core/functions/scalar/CMakeLists.txt @@ -10,7 +10,6 @@ set(EXTENSION_SOURCES ${CMAKE_CURRENT_SOURCE_DIR}/reachability.cpp ${CMAKE_CURRENT_SOURCE_DIR}/shortest_path.cpp ${CMAKE_CURRENT_SOURCE_DIR}/shortest_path_operator.cpp - ${CMAKE_CURRENT_SOURCE_DIR}/csr_creation.cpp ${CMAKE_CURRENT_SOURCE_DIR}/local_clustering_coefficient.cpp PARENT_SCOPE diff --git a/src/core/functions/scalar/shortest_path.cpp b/src/core/functions/scalar/shortest_path.cpp index 785567d4..211831ab 100644 --- a/src/core/functions/scalar/shortest_path.cpp +++ b/src/core/functions/scalar/shortest_path.cpp @@ -231,9 +231,7 @@ ScalarFunctionSet GetShortestPathFunction() { LogicalType::LIST(LogicalType::BIGINT), ShortestPathFunction, IterativeLengthFunctionData::IterativeLengthBind)); - set.AddFunction(ScalarFunction({LogicalType::BIGINT, LogicalType::BIGINT, LogicalType::VARCHAR}, - LogicalType::LIST(LogicalType::BIGINT), ShortestPathFunction, - IterativeLengthFunctionData::IterativeLengthBind)); + return set; } diff --git a/src/core/functions/scalar/shortest_path_operator.cpp b/src/core/functions/scalar/shortest_path_operator.cpp index 270648e7..f66e8890 100644 --- a/src/core/functions/scalar/shortest_path_operator.cpp +++ b/src/core/functions/scalar/shortest_path_operator.cpp @@ -1,5 +1,8 @@ + #include "duckpgq/core/functions/function_data/shortest_path_operator_function_data.hpp" +#include + namespace duckpgq { namespace core { @@ -11,7 +14,9 @@ void CoreScalarFunctions::RegisterShortestPathScalarFunction( DatabaseInstance &db) { ExtensionUtil::RegisterFunction( - db, GetShortestPathFunction()); + db, ScalarFunction("shortestpathoperator", {LogicalType::BIGINT, LogicalType::BIGINT, LogicalType::VARCHAR}, + LogicalType::LIST(LogicalType::BIGINT), ShortestPathOperatorFunction, + ShortestPathOperatorData::ShortestPathOperatorBind)); } diff --git a/src/include/duckpgq/core/functions/function_data/shortest_path_operator_function_data.hpp b/src/include/duckpgq/core/functions/function_data/shortest_path_operator_function_data.hpp index 0e5b1fa5..3ccb635a 100644 --- a/src/include/duckpgq/core/functions/function_data/shortest_path_operator_function_data.hpp +++ b/src/include/duckpgq/core/functions/function_data/shortest_path_operator_function_data.hpp @@ -15,15 +15,15 @@ namespace duckpgq { namespace core { -struct IterativeLengthFunctionData final : FunctionData { +struct ShortestPathOperatorData final : FunctionData { ClientContext &context; string table_to_scan; int32_t csr_id; - IterativeLengthFunctionData(ClientContext &context, string table_to_scan, int32_t csr_id) - : context(context), table_to_scan(table_to_scan), csr_id(csr_id) {} + ShortestPathOperatorData(ClientContext &context, string table_to_scan, int32_t csr_id) + : context(context), table_to_scan(std::move(table_to_scan)), csr_id(csr_id) {} static unique_ptr - IterativeLengthBind(ClientContext &context, ScalarFunction &bound_function, + ShortestPathOperatorBind(ClientContext &context, ScalarFunction &bound_function, vector> &arguments); unique_ptr Copy() const override; From 587d2a7be1fb26a35c972737e8a65cbef7bbde58 Mon Sep 17 00:00:00 2001 From: dtenwolde Date: Wed, 14 Aug 2024 14:52:17 +0200 Subject: [PATCH 162/249] Throw not implemented for the UDF itsefl --- src/core/functions/scalar/shortest_path_operator.cpp | 5 +++++ 1 file changed, 5 insertions(+) diff --git a/src/core/functions/scalar/shortest_path_operator.cpp b/src/core/functions/scalar/shortest_path_operator.cpp index f66e8890..e8abc450 100644 --- a/src/core/functions/scalar/shortest_path_operator.cpp +++ b/src/core/functions/scalar/shortest_path_operator.cpp @@ -7,6 +7,11 @@ namespace duckpgq { namespace core { +static void ShortestPathOperatorFunction(DataChunk &args, ExpressionState &state, + Vector &result) { + throw NotImplementedException("ShortestPathOperatorFunction not implemented, should have gone to the operator instead."); +} + //------------------------------------------------------------------------------ // Register functions //------------------------------------------------------------------------------ From 85f15c74f35e1f06eae541102e43873fb65a30d9 Mon Sep 17 00:00:00 2001 From: dtenwolde Date: Wed, 14 Aug 2024 14:57:28 +0200 Subject: [PATCH 163/249] Implement copy and equals and bind --- .../functions/function_data/CMakeLists.txt | 2 ++ .../shortest_path_operator_function_data.cpp | 27 +++++++++++++++++++ .../shortest_path_operator_function_data.hpp | 5 ++-- 3 files changed, 31 insertions(+), 3 deletions(-) create mode 100644 src/core/functions/function_data/shortest_path_operator_function_data.cpp diff --git a/src/core/functions/function_data/CMakeLists.txt b/src/core/functions/function_data/CMakeLists.txt index 1e5e3c75..d2d59899 100644 --- a/src/core/functions/function_data/CMakeLists.txt +++ b/src/core/functions/function_data/CMakeLists.txt @@ -3,5 +3,7 @@ set(EXTENSION_SOURCES ${CMAKE_CURRENT_SOURCE_DIR}/cheapest_path_length_function_data.cpp ${CMAKE_CURRENT_SOURCE_DIR}/iterative_length_function_data.cpp ${CMAKE_CURRENT_SOURCE_DIR}/local_clustering_coefficient_function_data.cpp + ${CMAKE_CURRENT_SOURCE_DIR}/shortest_path_operator_function_data.cpp + PARENT_SCOPE ) \ No newline at end of file diff --git a/src/core/functions/function_data/shortest_path_operator_function_data.cpp b/src/core/functions/function_data/shortest_path_operator_function_data.cpp new file mode 100644 index 00000000..239fefd6 --- /dev/null +++ b/src/core/functions/function_data/shortest_path_operator_function_data.cpp @@ -0,0 +1,27 @@ +#include "duckpgq/core/functions/function_data/shortest_path_operator_function_data.hpp" + +#include + +namespace duckpgq { + +namespace core { + +unique_ptr +ShortestPathOperatorData::ShortestPathOperatorBind( + ClientContext &context, ScalarFunction &bound_function, + vector> &arguments) { + // TODO look into the arguments and bound_function to see if we need to do anything + return make_uniq(context); +} +unique_ptr ShortestPathOperatorData::Copy() const { + return make_uniq(context, table_to_scan); +} + +bool ShortestPathOperatorData::Equals(const FunctionData &other_p) const { + auto &other = (const ShortestPathOperatorData &)other_p; + return other.table_to_scan == table_to_scan; +} + +} // namespace core + +} // namespace duckpgq \ No newline at end of file diff --git a/src/include/duckpgq/core/functions/function_data/shortest_path_operator_function_data.hpp b/src/include/duckpgq/core/functions/function_data/shortest_path_operator_function_data.hpp index 3ccb635a..f6761ed3 100644 --- a/src/include/duckpgq/core/functions/function_data/shortest_path_operator_function_data.hpp +++ b/src/include/duckpgq/core/functions/function_data/shortest_path_operator_function_data.hpp @@ -18,10 +18,9 @@ namespace core { struct ShortestPathOperatorData final : FunctionData { ClientContext &context; string table_to_scan; - int32_t csr_id; - ShortestPathOperatorData(ClientContext &context, string table_to_scan, int32_t csr_id) - : context(context), table_to_scan(std::move(table_to_scan)), csr_id(csr_id) {} + ShortestPathOperatorData(ClientContext &context, string table_to_scan) + : context(context), table_to_scan(std::move(table_to_scan)) {} static unique_ptr ShortestPathOperatorBind(ClientContext &context, ScalarFunction &bound_function, vector> &arguments); From 0bde3d9ae4f5c27a30cdbc17245b683962a85ba1 Mon Sep 17 00:00:00 2001 From: dtenwolde Date: Wed, 14 Aug 2024 14:58:34 +0200 Subject: [PATCH 164/249] Remove imports --- .../function_data/shortest_path_operator_function_data.cpp | 1 - src/core/functions/scalar/csr_creation.cpp | 2 -- 2 files changed, 3 deletions(-) diff --git a/src/core/functions/function_data/shortest_path_operator_function_data.cpp b/src/core/functions/function_data/shortest_path_operator_function_data.cpp index 239fefd6..1a43a137 100644 --- a/src/core/functions/function_data/shortest_path_operator_function_data.cpp +++ b/src/core/functions/function_data/shortest_path_operator_function_data.cpp @@ -1,5 +1,4 @@ #include "duckpgq/core/functions/function_data/shortest_path_operator_function_data.hpp" - #include namespace duckpgq { diff --git a/src/core/functions/scalar/csr_creation.cpp b/src/core/functions/scalar/csr_creation.cpp index 4aca6bcb..5a388240 100644 --- a/src/core/functions/scalar/csr_creation.cpp +++ b/src/core/functions/scalar/csr_creation.cpp @@ -6,8 +6,6 @@ #include #include #include -#include -#include namespace duckpgq { From 7d62c6a833bfb6f8505cee86fbc76d27b148cf33 Mon Sep 17 00:00:00 2001 From: dtenwolde Date: Wed, 14 Aug 2024 15:08:31 +0200 Subject: [PATCH 165/249] Move csr function data to separate file for consistency --- .../functions/function_data/CMakeLists.txt | 1 + .../function_data/csr_function_data.cpp | 56 +++++++++++++++++++ src/core/utils/compressed_sparse_row.cpp | 45 --------------- .../function_data/csr_function_data.hpp | 30 ++++++++++ .../core/utils/compressed_sparse_row.hpp | 19 ------- 5 files changed, 87 insertions(+), 64 deletions(-) create mode 100644 src/core/functions/function_data/csr_function_data.cpp create mode 100644 src/include/duckpgq/core/functions/function_data/csr_function_data.hpp diff --git a/src/core/functions/function_data/CMakeLists.txt b/src/core/functions/function_data/CMakeLists.txt index d2d59899..beff5953 100644 --- a/src/core/functions/function_data/CMakeLists.txt +++ b/src/core/functions/function_data/CMakeLists.txt @@ -1,6 +1,7 @@ set(EXTENSION_SOURCES ${EXTENSION_SOURCES} ${CMAKE_CURRENT_SOURCE_DIR}/cheapest_path_length_function_data.cpp + ${CMAKE_CURRENT_SOURCE_DIR}/csr_function_data.cpp ${CMAKE_CURRENT_SOURCE_DIR}/iterative_length_function_data.cpp ${CMAKE_CURRENT_SOURCE_DIR}/local_clustering_coefficient_function_data.cpp ${CMAKE_CURRENT_SOURCE_DIR}/shortest_path_operator_function_data.cpp diff --git a/src/core/functions/function_data/csr_function_data.cpp b/src/core/functions/function_data/csr_function_data.cpp new file mode 100644 index 00000000..437b0a12 --- /dev/null +++ b/src/core/functions/function_data/csr_function_data.cpp @@ -0,0 +1,56 @@ +#include "duckpgq/core/functions/function_data/csr_function_data.hpp" +#include + +namespace duckpgq { + +namespace core { + +CSRFunctionData::CSRFunctionData(ClientContext &context, int32_t id, + LogicalType weight_type) + : context(context), id(id), weight_type(std::move(weight_type)) {} + +unique_ptr CSRFunctionData::Copy() const { + return make_uniq(context, id, weight_type); +} + +bool CSRFunctionData::Equals(const FunctionData &other_p) const { + auto &other = (const CSRFunctionData &)other_p; + return id == other.id && weight_type == other.weight_type; +} + + +unique_ptr +CSRFunctionData::CSRVertexBind(ClientContext &context, + ScalarFunction &bound_function, + vector> &arguments) { + auto csr_id = GetCSRId(arguments[0], context); + if (arguments.size() == 4) { + auto logical_type = LogicalType::SQLNULL; + return make_uniq(context, csr_id, logical_type); + } + return make_uniq(context, csr_id, arguments[3]->return_type); +} + +unique_ptr +CSRFunctionData::CSREdgeBind(ClientContext &context, + ScalarFunction &bound_function, + vector> &arguments) { + auto csr_id = GetCSRId(arguments[0], context); + if (arguments.size() == 7) { + return make_uniq(context, csr_id, + arguments[6]->return_type); + } + auto logical_type = LogicalType::SQLNULL; + return make_uniq(context, csr_id, logical_type); +} + +unique_ptr +CSRFunctionData::CSRBind(ClientContext &context, ScalarFunction &bound_function, + vector> &arguments) { + auto csr_id = GetCSRId(arguments[0], context); + return make_uniq(context, csr_id, LogicalType::BOOLEAN); +} + +} // namespace core + +} // namespace duckpgq diff --git a/src/core/utils/compressed_sparse_row.cpp b/src/core/utils/compressed_sparse_row.cpp index dafdae5e..be1137ae 100644 --- a/src/core/utils/compressed_sparse_row.cpp +++ b/src/core/utils/compressed_sparse_row.cpp @@ -34,51 +34,6 @@ string CSR::ToString() const { return result; } -CSRFunctionData::CSRFunctionData(ClientContext &context, int32_t id, - LogicalType weight_type) - : context(context), id(id), weight_type(std::move(weight_type)) {} - -unique_ptr CSRFunctionData::Copy() const { - return make_uniq(context, id, weight_type); -} - -bool CSRFunctionData::Equals(const FunctionData &other_p) const { - auto &other = (const CSRFunctionData &)other_p; - return id == other.id && weight_type == other.weight_type; -} - - -unique_ptr -CSRFunctionData::CSRVertexBind(ClientContext &context, - ScalarFunction &bound_function, - vector> &arguments) { - auto csr_id = GetCSRId(arguments[0], context); - if (arguments.size() == 4) { - auto logical_type = LogicalType::SQLNULL; - return make_uniq(context, csr_id, logical_type); - } - return make_uniq(context, csr_id, arguments[3]->return_type); -} - -unique_ptr -CSRFunctionData::CSREdgeBind(ClientContext &context, - ScalarFunction &bound_function, - vector> &arguments) { - auto csr_id = GetCSRId(arguments[0], context); - if (arguments.size() == 7) { - return make_uniq(context, csr_id, - arguments[6]->return_type); - } - auto logical_type = LogicalType::SQLNULL; - return make_uniq(context, csr_id, logical_type); -} - -unique_ptr -CSRFunctionData::CSRBind(ClientContext &context, ScalarFunction &bound_function, - vector> &arguments) { - auto csr_id = GetCSRId(arguments[0], context); - return make_uniq(context, csr_id, LogicalType::BOOLEAN); -} // Helper function to create a ColumnRefExpression with alias unique_ptr CreateColumnRef(const std::string &column_name, const std::string &table_name, const std::string &alias) { diff --git a/src/include/duckpgq/core/functions/function_data/csr_function_data.hpp b/src/include/duckpgq/core/functions/function_data/csr_function_data.hpp new file mode 100644 index 00000000..4e819226 --- /dev/null +++ b/src/include/duckpgq/core/functions/function_data/csr_function_data.hpp @@ -0,0 +1,30 @@ +#pragma once + +#include "duckpgq/common.hpp" + +namespace duckpgq { + +namespace core { + +struct CSRFunctionData : FunctionData { + CSRFunctionData(ClientContext &context, int32_t id, LogicalType weight_type); + unique_ptr Copy() const override; + bool Equals(const FunctionData &other_p) const override; + static unique_ptr + CSRVertexBind(ClientContext &context, ScalarFunction &bound_function, + vector> &arguments); + static unique_ptr + CSREdgeBind(ClientContext &context, ScalarFunction &bound_function, + vector> &arguments); + static unique_ptr + CSRBind(ClientContext &context, ScalarFunction &bound_function, + vector> &arguments); + + ClientContext &context; + const int32_t id; + const LogicalType weight_type; +}; + +} // namespace core + +} // namespace duckpgq \ No newline at end of file diff --git a/src/include/duckpgq/core/utils/compressed_sparse_row.hpp b/src/include/duckpgq/core/utils/compressed_sparse_row.hpp index e64067fc..74b48224 100644 --- a/src/include/duckpgq/core/utils/compressed_sparse_row.hpp +++ b/src/include/duckpgq/core/utils/compressed_sparse_row.hpp @@ -46,25 +46,6 @@ class CSR { string ToString() const; }; -struct CSRFunctionData : FunctionData { - CSRFunctionData(ClientContext &context, int32_t id, LogicalType weight_type); - unique_ptr Copy() const override; - bool Equals(const FunctionData &other_p) const override; - static unique_ptr - CSRVertexBind(ClientContext &context, ScalarFunction &bound_function, - vector> &arguments); - static unique_ptr - CSREdgeBind(ClientContext &context, ScalarFunction &bound_function, - vector> &arguments); - static unique_ptr - CSRBind(ClientContext &context, ScalarFunction &bound_function, - vector> &arguments); - - ClientContext &context; - const int32_t id; - const LogicalType weight_type; -}; - // CSR BindReplace functions unique_ptr CreateUndirectedCSRCTE(const shared_ptr &edge_table); unique_ptr CreateDirectedCSRCTE(const shared_ptr &edge_table, const string &prev_binding, const string &edge_binding, const string &next_binding); From 751262e11f55b78542b2541e62d2b9bc0f797e5a Mon Sep 17 00:00:00 2001 From: dtenwolde Date: Wed, 14 Aug 2024 15:17:51 +0200 Subject: [PATCH 166/249] Create separate csr operator function data --- .../functions/function_data/CMakeLists.txt | 1 + .../csr_operator_function_data.cpp | 21 ++++++++++++++++ .../csr_operator_function_data.hpp | 24 +++++++++++++++++++ 3 files changed, 46 insertions(+) create mode 100644 src/core/functions/function_data/csr_operator_function_data.cpp create mode 100644 src/include/duckpgq/core/functions/function_data/csr_operator_function_data.hpp diff --git a/src/core/functions/function_data/CMakeLists.txt b/src/core/functions/function_data/CMakeLists.txt index beff5953..c33588e7 100644 --- a/src/core/functions/function_data/CMakeLists.txt +++ b/src/core/functions/function_data/CMakeLists.txt @@ -2,6 +2,7 @@ set(EXTENSION_SOURCES ${EXTENSION_SOURCES} ${CMAKE_CURRENT_SOURCE_DIR}/cheapest_path_length_function_data.cpp ${CMAKE_CURRENT_SOURCE_DIR}/csr_function_data.cpp + ${CMAKE_CURRENT_SOURCE_DIR}/csr_operator_function_data.cpp ${CMAKE_CURRENT_SOURCE_DIR}/iterative_length_function_data.cpp ${CMAKE_CURRENT_SOURCE_DIR}/local_clustering_coefficient_function_data.cpp ${CMAKE_CURRENT_SOURCE_DIR}/shortest_path_operator_function_data.cpp diff --git a/src/core/functions/function_data/csr_operator_function_data.cpp b/src/core/functions/function_data/csr_operator_function_data.cpp new file mode 100644 index 00000000..15dc4189 --- /dev/null +++ b/src/core/functions/function_data/csr_operator_function_data.cpp @@ -0,0 +1,21 @@ +#include "duckpgq/core/functions/function_data/csr_operator_function_data.hpp" + +namespace duckpgq { + +namespace core { + +CSROperatorFunctionData::CSROperatorFunctionData(ClientContext &context) + : context(context) {} + +unique_ptr CSROperatorFunctionData::Copy() const { + return make_uniq(context); +} + +bool CSROperatorFunctionData::Equals(const FunctionData &other_p) const { + // TODO implement me + // auto &other = (const CSROperatorFunctionData &)other_p; + return true; +} +} // namespace core + +} // namespace duckpgq \ No newline at end of file diff --git a/src/include/duckpgq/core/functions/function_data/csr_operator_function_data.hpp b/src/include/duckpgq/core/functions/function_data/csr_operator_function_data.hpp new file mode 100644 index 00000000..dfdd7dda --- /dev/null +++ b/src/include/duckpgq/core/functions/function_data/csr_operator_function_data.hpp @@ -0,0 +1,24 @@ +#pragma once +#include "duckpgq/common.hpp" + +namespace duckpgq { + +namespace core { + +struct CSROperatorFunctionData : FunctionData { + CSROperatorFunctionData(ClientContext &context); + unique_ptr Copy() const override; + bool Equals(const FunctionData &other_p) const override; + // static unique_ptr + // CSREdgeBind(ClientContext &context, ScalarFunction &bound_function, + // vector> &arguments); + // static unique_ptr + // CSRBind(ClientContext &context, ScalarFunction &bound_function, + // vector> &arguments); + + ClientContext &context; +}; + +} // namespace core + +} // namespace duckpgq \ No newline at end of file From 30b3e546777f3986d7ff3244ff82bedbc78eed48 Mon Sep 17 00:00:00 2001 From: dtenwolde Date: Wed, 14 Aug 2024 15:21:35 +0200 Subject: [PATCH 167/249] Fix import --- src/core/functions/scalar/csr_creation.cpp | 1 + 1 file changed, 1 insertion(+) diff --git a/src/core/functions/scalar/csr_creation.cpp b/src/core/functions/scalar/csr_creation.cpp index 5a388240..c76a1bd4 100644 --- a/src/core/functions/scalar/csr_creation.cpp +++ b/src/core/functions/scalar/csr_creation.cpp @@ -4,6 +4,7 @@ #include "duckpgq/common.hpp" #include "duckpgq/core/utils/compressed_sparse_row.hpp" #include +#include #include #include From 1ee8ad04b24f08fce5b55f4fa516ab119fa03d82 Mon Sep 17 00:00:00 2001 From: dtenwolde Date: Wed, 14 Aug 2024 17:03:56 +0200 Subject: [PATCH 168/249] Create separate logic for csr creation for operator --- .../csr_operator_function_data.cpp | 7 +++++ src/core/functions/scalar/CMakeLists.txt | 1 + src/core/functions/scalar/csr_deletion.cpp | 2 +- .../scalar/csr_operator_creation.cpp | 29 +++++++++++++++++++ .../csr_operator_function_data.hpp | 9 ++---- src/include/duckpgq/core/functions/scalar.hpp | 4 +-- 6 files changed, 43 insertions(+), 9 deletions(-) create mode 100644 src/core/functions/scalar/csr_operator_creation.cpp diff --git a/src/core/functions/function_data/csr_operator_function_data.cpp b/src/core/functions/function_data/csr_operator_function_data.cpp index 15dc4189..fbe4d9a4 100644 --- a/src/core/functions/function_data/csr_operator_function_data.cpp +++ b/src/core/functions/function_data/csr_operator_function_data.cpp @@ -11,6 +11,13 @@ unique_ptr CSROperatorFunctionData::Copy() const { return make_uniq(context); } +unique_ptr +CSROperatorFunctionData::CSRBind(ClientContext &context, + ScalarFunction &bound_function, + vector> &arguments) { + return make_uniq(context); +} + bool CSROperatorFunctionData::Equals(const FunctionData &other_p) const { // TODO implement me // auto &other = (const CSROperatorFunctionData &)other_p; diff --git a/src/core/functions/scalar/CMakeLists.txt b/src/core/functions/scalar/CMakeLists.txt index ba22a32e..7ada6107 100644 --- a/src/core/functions/scalar/CMakeLists.txt +++ b/src/core/functions/scalar/CMakeLists.txt @@ -4,6 +4,7 @@ set(EXTENSION_SOURCES ${CMAKE_CURRENT_SOURCE_DIR}/csr_creation.cpp ${CMAKE_CURRENT_SOURCE_DIR}/csr_deletion.cpp ${CMAKE_CURRENT_SOURCE_DIR}/csr_get_w_type.cpp + ${CMAKE_CURRENT_SOURCE_DIR}/csr_operator_creation.cpp ${CMAKE_CURRENT_SOURCE_DIR}/iterativelength.cpp ${CMAKE_CURRENT_SOURCE_DIR}/iterativelength2.cpp ${CMAKE_CURRENT_SOURCE_DIR}/iterativelength_bidirectional.cpp diff --git a/src/core/functions/scalar/csr_deletion.cpp b/src/core/functions/scalar/csr_deletion.cpp index b1d8c4ac..5bc0e841 100644 --- a/src/core/functions/scalar/csr_deletion.cpp +++ b/src/core/functions/scalar/csr_deletion.cpp @@ -1,9 +1,9 @@ #include "duckdb/main/client_data.hpp" #include "duckdb/planner/expression/bound_function_expression.hpp" #include "duckpgq/common.hpp" +#include #include #include -#include namespace duckpgq { diff --git a/src/core/functions/scalar/csr_operator_creation.cpp b/src/core/functions/scalar/csr_operator_creation.cpp new file mode 100644 index 00000000..acb8bf36 --- /dev/null +++ b/src/core/functions/scalar/csr_operator_creation.cpp @@ -0,0 +1,29 @@ +#include "duckpgq/common.hpp" +#include +#include +#include "duckdb/function/scalar_function.hpp" + +namespace duckpgq { + +namespace core { + +static void CreateCSROperatorFunction(DataChunk &args, ExpressionState &state, + Vector &result) { + throw NotImplementedException("CSR operator creation function not implemented, should have gone to operator instead."); +} + +//------------------------------------------------------------------------------ +// Register functions +//------------------------------------------------------------------------------ +void CoreScalarFunctions::RegisterCSROperatorCreationScalarFunctions(DatabaseInstance &db) { + ExtensionUtil::RegisterFunction(db, ScalarFunction("csr_operator", {LogicalType::INTEGER, LogicalType::BIGINT, + LogicalType::BIGINT, LogicalType::BIGINT, + LogicalType::BIGINT, LogicalType::BIGINT, + LogicalType::BIGINT}, + LogicalType::INTEGER, CreateCSROperatorFunction, + CSROperatorFunctionData::CSRBind)); +} + +} // namespace core + +} // namespace duckpgq \ No newline at end of file diff --git a/src/include/duckpgq/core/functions/function_data/csr_operator_function_data.hpp b/src/include/duckpgq/core/functions/function_data/csr_operator_function_data.hpp index dfdd7dda..95e48b9c 100644 --- a/src/include/duckpgq/core/functions/function_data/csr_operator_function_data.hpp +++ b/src/include/duckpgq/core/functions/function_data/csr_operator_function_data.hpp @@ -9,12 +9,9 @@ struct CSROperatorFunctionData : FunctionData { CSROperatorFunctionData(ClientContext &context); unique_ptr Copy() const override; bool Equals(const FunctionData &other_p) const override; - // static unique_ptr - // CSREdgeBind(ClientContext &context, ScalarFunction &bound_function, - // vector> &arguments); - // static unique_ptr - // CSRBind(ClientContext &context, ScalarFunction &bound_function, - // vector> &arguments); + static unique_ptr CSRBind(ClientContext &context, + ScalarFunction &bound_function, + vector> &arguments); ClientContext &context; }; diff --git a/src/include/duckpgq/core/functions/scalar.hpp b/src/include/duckpgq/core/functions/scalar.hpp index 1e502559..974240f7 100644 --- a/src/include/duckpgq/core/functions/scalar.hpp +++ b/src/include/duckpgq/core/functions/scalar.hpp @@ -18,7 +18,7 @@ struct CoreScalarFunctions { RegisterReachabilityScalarFunction(db); RegisterShortestPathScalarFunction(db); RegisterShortestPathOperatorScalarFunction(db); - + RegisterCSROperatorCreationScalarFunctions(db); } private: @@ -33,7 +33,7 @@ struct CoreScalarFunctions { static void RegisterReachabilityScalarFunction(DatabaseInstance &db); static void RegisterShortestPathScalarFunction(DatabaseInstance &db); static void RegisterShortestPathOperatorScalarFunction(DatabaseInstance &db); - + static void RegisterCSROperatorCreationScalarFunctions(DatabaseInstance &db); }; From 10e6e5185d8a766b14544a6fd3f3411aed1ddfe8 Mon Sep 17 00:00:00 2001 From: dtenwolde Date: Wed, 14 Aug 2024 17:04:17 +0200 Subject: [PATCH 169/249] Explicit --- .../core/functions/function_data/csr_operator_function_data.hpp | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/src/include/duckpgq/core/functions/function_data/csr_operator_function_data.hpp b/src/include/duckpgq/core/functions/function_data/csr_operator_function_data.hpp index 95e48b9c..d19f27b5 100644 --- a/src/include/duckpgq/core/functions/function_data/csr_operator_function_data.hpp +++ b/src/include/duckpgq/core/functions/function_data/csr_operator_function_data.hpp @@ -6,7 +6,7 @@ namespace duckpgq { namespace core { struct CSROperatorFunctionData : FunctionData { - CSROperatorFunctionData(ClientContext &context); + explicit CSROperatorFunctionData(ClientContext &context); unique_ptr Copy() const override; bool Equals(const FunctionData &other_p) const override; static unique_ptr CSRBind(ClientContext &context, From 7f7793678dcf5519236ca1080cbad8cd1deb7d0d Mon Sep 17 00:00:00 2001 From: dtenwolde Date: Wed, 14 Aug 2024 17:06:01 +0200 Subject: [PATCH 170/249] Format fix --- .../scalar/csr_operator_creation.cpp | 24 +++++++++++-------- 1 file changed, 14 insertions(+), 10 deletions(-) diff --git a/src/core/functions/scalar/csr_operator_creation.cpp b/src/core/functions/scalar/csr_operator_creation.cpp index acb8bf36..349b7fab 100644 --- a/src/core/functions/scalar/csr_operator_creation.cpp +++ b/src/core/functions/scalar/csr_operator_creation.cpp @@ -1,27 +1,31 @@ +#include "duckdb/function/scalar_function.hpp" #include "duckpgq/common.hpp" #include #include -#include "duckdb/function/scalar_function.hpp" namespace duckpgq { namespace core { static void CreateCSROperatorFunction(DataChunk &args, ExpressionState &state, - Vector &result) { - throw NotImplementedException("CSR operator creation function not implemented, should have gone to operator instead."); + Vector &result) { + throw NotImplementedException( + "CSR operator creation function not implemented, should have gone to " + "operator instead."); } //------------------------------------------------------------------------------ // Register functions //------------------------------------------------------------------------------ -void CoreScalarFunctions::RegisterCSROperatorCreationScalarFunctions(DatabaseInstance &db) { - ExtensionUtil::RegisterFunction(db, ScalarFunction("csr_operator", {LogicalType::INTEGER, LogicalType::BIGINT, - LogicalType::BIGINT, LogicalType::BIGINT, - LogicalType::BIGINT, LogicalType::BIGINT, - LogicalType::BIGINT}, - LogicalType::INTEGER, CreateCSROperatorFunction, - CSROperatorFunctionData::CSRBind)); +void CoreScalarFunctions::RegisterCSROperatorCreationScalarFunctions( + DatabaseInstance &db) { + ExtensionUtil::RegisterFunction( + db, ScalarFunction("csr_operator", + {LogicalType::BIGINT, LogicalType::BIGINT, + LogicalType::BIGINT, LogicalType::BIGINT, + LogicalType::BIGINT, LogicalType::BIGINT}, + LogicalType::INTEGER, CreateCSROperatorFunction, + CSROperatorFunctionData::CSRBind)); } } // namespace core From d2982ff050df37e0dd38bafe9e63e369bf05afe3 Mon Sep 17 00:00:00 2001 From: dtenwolde Date: Wed, 14 Aug 2024 17:06:07 +0200 Subject: [PATCH 171/249] Update optimizer rule --- src/core/optimizer/path_finding_optimizer_rule.cpp | 7 ++++--- 1 file changed, 4 insertions(+), 3 deletions(-) diff --git a/src/core/optimizer/path_finding_optimizer_rule.cpp b/src/core/optimizer/path_finding_optimizer_rule.cpp index d2ff5ffd..13231623 100644 --- a/src/core/optimizer/path_finding_optimizer_rule.cpp +++ b/src/core/optimizer/path_finding_optimizer_rule.cpp @@ -33,13 +33,14 @@ bool DuckpgqOptimizerExtension::InsertPathFindingOperator(LogicalOperator &op, C continue; } auto &bound_function_expression = expr->Cast(); - if (bound_function_expression.function.name == "iterativelength") { + // TODO create iterativelengthoperator UDF + if (bound_function_expression.function.name == "iterativelengthoperator") { op.expressions.emplace_back(make_uniq( expr->alias, LogicalType::BIGINT, ColumnBinding(10, 0))); function_expression = expr->Copy(); op.expressions.erase(op.expressions.begin() + i); mode = "iterativelength"; - } else if (bound_function_expression.function.name == "shortestpath") { + } else if (bound_function_expression.function.name == "shortestpathoperator") { op.expressions.emplace_back(make_uniq( expr->alias, LogicalType::LIST(LogicalType::BIGINT), ColumnBinding(10, 0))); function_expression = expr->Copy(); @@ -67,7 +68,7 @@ bool DuckpgqOptimizerExtension::InsertPathFindingOperator(LogicalOperator &op, C auto &get_projection = get_limit.children[0]->Cast(); auto &get_function_expression = get_projection.expressions[0]->Cast(); - if (get_function_expression.function.name != "create_csr_edge") { + if (get_function_expression.function.name != "csr_operator") { continue; } vector> path_finding_expressions = From e00af177c3ecc96f42b1676c185c1e36bdeeb5aa Mon Sep 17 00:00:00 2001 From: dtenwolde Date: Wed, 14 Aug 2024 17:08:55 +0200 Subject: [PATCH 172/249] Fix constructor and add import --- .../function_data/shortest_path_operator_function_data.cpp | 3 ++- src/core/functions/scalar/csr_get_w_type.cpp | 1 + 2 files changed, 3 insertions(+), 1 deletion(-) diff --git a/src/core/functions/function_data/shortest_path_operator_function_data.cpp b/src/core/functions/function_data/shortest_path_operator_function_data.cpp index 1a43a137..1fbcd10c 100644 --- a/src/core/functions/function_data/shortest_path_operator_function_data.cpp +++ b/src/core/functions/function_data/shortest_path_operator_function_data.cpp @@ -10,7 +10,8 @@ ShortestPathOperatorData::ShortestPathOperatorBind( ClientContext &context, ScalarFunction &bound_function, vector> &arguments) { // TODO look into the arguments and bound_function to see if we need to do anything - return make_uniq(context); + string table_to_scan = ExpressionExecutor::EvaluateScalar(context, *arguments[2]).GetValue(); + return make_uniq(context, table_to_scan); } unique_ptr ShortestPathOperatorData::Copy() const { return make_uniq(context, table_to_scan); diff --git a/src/core/functions/scalar/csr_get_w_type.cpp b/src/core/functions/scalar/csr_get_w_type.cpp index ea735b0c..85a85d36 100644 --- a/src/core/functions/scalar/csr_get_w_type.cpp +++ b/src/core/functions/scalar/csr_get_w_type.cpp @@ -2,6 +2,7 @@ #include "duckpgq/common.hpp" #include +#include #include #include From 6716a088a3691812f86514d0ded2786345eb7115 Mon Sep 17 00:00:00 2001 From: dtenwolde Date: Wed, 14 Aug 2024 17:12:53 +0200 Subject: [PATCH 173/249] Fix wrong function name --- src/core/functions/scalar/shortest_path_operator.cpp | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/src/core/functions/scalar/shortest_path_operator.cpp b/src/core/functions/scalar/shortest_path_operator.cpp index e8abc450..8b7c3a51 100644 --- a/src/core/functions/scalar/shortest_path_operator.cpp +++ b/src/core/functions/scalar/shortest_path_operator.cpp @@ -15,7 +15,7 @@ static void ShortestPathOperatorFunction(DataChunk &args, ExpressionState &state //------------------------------------------------------------------------------ // Register functions //------------------------------------------------------------------------------ -void CoreScalarFunctions::RegisterShortestPathScalarFunction( +void CoreScalarFunctions::RegisterShortestPathOperatorScalarFunction( DatabaseInstance &db) { ExtensionUtil::RegisterFunction( From 2faf8825c747fecc058d1645e71db565a2fe4a8a Mon Sep 17 00:00:00 2001 From: dtenwolde Date: Wed, 14 Aug 2024 17:15:01 +0200 Subject: [PATCH 174/249] Adjust the tests to the correct function names --- .../path_finding/parallel_path_finding.test | 38 ++++++++++++++++--- 1 file changed, 32 insertions(+), 6 deletions(-) diff --git a/test/sql/path_finding/parallel_path_finding.test b/test/sql/path_finding/parallel_path_finding.test index 940681ed..c90731e1 100644 --- a/test/sql/path_finding/parallel_path_finding.test +++ b/test/sql/path_finding/parallel_path_finding.test @@ -10,6 +10,11 @@ require duckpgq statement ok set experimental_path_finding_operator=true; +query I +SELECT current_setting('experimental_path_finding_operator'); +---- +true + statement ok CREATE TABLE pair(src BIGINT, dst BIGINT); INSERT INTO pair(src, dst) VALUES (0, 1), (1, 2), (2,0); @@ -20,9 +25,9 @@ statement ok create table know(src INT, dst INT); INSERT INTO know(src, dst) VALUES (40, 20), (10,30), (10,10), (20,10), (30,10); query III -SELECT *, shortestpath(src, dst, 'pair') as path +SELECT *, shortestpathoperator(src, dst, 'pair') as path FROM pair AS p -WHERE p.src BETWEEN (SELECT CREATE_CSR_EDGE( +WHERE p.src BETWEEN (SELECT CSR_OPERATOR( (SELECT count(a.id) as v_size FROM Student a), (SELECT count(k.src) as e_size from know k), a.rowid, @@ -56,9 +61,30 @@ CREATE OR REPLACE TABLE pairs2 AS ( ); query III -SELECT *, iterativelength(src, dst, 'pairs2') as length +SELECT *, shortestpathoperator(src, dst, 'pairs2') as length +FROM pairs2 AS p +WHERE p.src BETWEEN (SELECT CSR_OPERATOR( + (SELECT count(a.id) as v_size FROM Person a), + (SELECT count(k.Person1Id) as e_size FROM Person_knows_Person k), + a.rowid, + c.rowid, + k.rowid, + t.cnt) FROM Person_knows_Person k + JOIN Person a on a.id = k.Person1Id + JOIN Person c on c.id = k.Person2Id + JOIN (SELECT count(k.Person1Id) cnt, a.rowid as a_rowid + FROM Person a + LEFT JOIN Person_knows_Person k ON k.Person1Id = a.id + GROUP BY a.rowid) t + ON t.a_rowid = a.rowid) AND p.dst; +---- +42 22 NULL +3 33 [3, 13, 26, 64, 33] + +query III +SELECT *, iterativelengthoperator(src, dst, 'pairs2') as length FROM pairs2 AS p -WHERE p.src BETWEEN (SELECT CREATE_CSR_EDGE(0, +WHERE p.src BETWEEN (SELECT CSR_OPERATOR( (SELECT count(a.id) as v_size FROM Person a), (SELECT count(k.Person1Id) as e_size FROM Person_knows_Person k), a.rowid, @@ -90,9 +116,9 @@ from pairs3; 10 2 query III -SELECT *, iterativelength(src, dst, 'pairs3') as length +SELECT *, iterativelengthoperator(src, dst, 'pairs3') as length FROM pairs3 as p -WHERE p.src BETWEEN (SELECT CREATE_CSR_EDGE( +WHERE p.src BETWEEN (SELECT CSR_OPERATOR( (SELECT count(a.id) as v_size FROM Person a), (SELECT count(k.Person1Id) as e_size FROM Person_knows_Person k), a.rowid, From 6c5750a8c0990b6dce88978960adf09aa0e80733 Mon Sep 17 00:00:00 2001 From: dtenwolde Date: Wed, 14 Aug 2024 17:15:18 +0200 Subject: [PATCH 175/249] Remove todo --- .../function_data/shortest_path_operator_function_data.cpp | 1 - 1 file changed, 1 deletion(-) diff --git a/src/core/functions/function_data/shortest_path_operator_function_data.cpp b/src/core/functions/function_data/shortest_path_operator_function_data.cpp index 1fbcd10c..4b21c107 100644 --- a/src/core/functions/function_data/shortest_path_operator_function_data.cpp +++ b/src/core/functions/function_data/shortest_path_operator_function_data.cpp @@ -9,7 +9,6 @@ unique_ptr ShortestPathOperatorData::ShortestPathOperatorBind( ClientContext &context, ScalarFunction &bound_function, vector> &arguments) { - // TODO look into the arguments and bound_function to see if we need to do anything string table_to_scan = ExpressionExecutor::EvaluateScalar(context, *arguments[2]).GetValue(); return make_uniq(context, table_to_scan); } From 6776629d4446f7cce4289cd7bd476981ef3819f2 Mon Sep 17 00:00:00 2001 From: dtenwolde Date: Wed, 14 Aug 2024 17:27:22 +0200 Subject: [PATCH 176/249] Adding iterativelength operator as well. All tests pass --- src/core/functions/scalar/CMakeLists.txt | 1 + .../scalar/iterativelength_operator.cpp | 27 +++++++++++++++++++ .../optimizer/path_finding_optimizer_rule.cpp | 1 - src/include/duckpgq/core/functions/scalar.hpp | 3 +++ 4 files changed, 31 insertions(+), 1 deletion(-) create mode 100644 src/core/functions/scalar/iterativelength_operator.cpp diff --git a/src/core/functions/scalar/CMakeLists.txt b/src/core/functions/scalar/CMakeLists.txt index 7ada6107..86c9038a 100644 --- a/src/core/functions/scalar/CMakeLists.txt +++ b/src/core/functions/scalar/CMakeLists.txt @@ -8,6 +8,7 @@ set(EXTENSION_SOURCES ${CMAKE_CURRENT_SOURCE_DIR}/iterativelength.cpp ${CMAKE_CURRENT_SOURCE_DIR}/iterativelength2.cpp ${CMAKE_CURRENT_SOURCE_DIR}/iterativelength_bidirectional.cpp + ${CMAKE_CURRENT_SOURCE_DIR}/iterativelength_operator.cpp ${CMAKE_CURRENT_SOURCE_DIR}/reachability.cpp ${CMAKE_CURRENT_SOURCE_DIR}/shortest_path.cpp ${CMAKE_CURRENT_SOURCE_DIR}/shortest_path_operator.cpp diff --git a/src/core/functions/scalar/iterativelength_operator.cpp b/src/core/functions/scalar/iterativelength_operator.cpp new file mode 100644 index 00000000..bb66d4a4 --- /dev/null +++ b/src/core/functions/scalar/iterativelength_operator.cpp @@ -0,0 +1,27 @@ + +#include "duckpgq/core/functions/function_data/shortest_path_operator_function_data.hpp" + +#include + +namespace duckpgq { + +namespace core { + +static void IterativeLengthOperatorFunction(DataChunk &args, ExpressionState &state, + Vector &result) { + throw NotImplementedException("IterativeLengthOperatorFunction not implemented, should have gone to the operator instead."); +} + +//------------------------------------------------------------------------------ +// Register functions +//------------------------------------------------------------------------------ +void CoreScalarFunctions::RegisterIterativeLengthOperatorScalarFunction(DatabaseInstance &db) { + ExtensionUtil::RegisterFunction( + db, ScalarFunction("iterativelengthoperator", {LogicalType::BIGINT, LogicalType::BIGINT, LogicalType::VARCHAR}, + LogicalType::BIGINT, IterativeLengthOperatorFunction, + ShortestPathOperatorData::ShortestPathOperatorBind)); +} + +} // namespace core + +} // namespace duckpgq \ No newline at end of file diff --git a/src/core/optimizer/path_finding_optimizer_rule.cpp b/src/core/optimizer/path_finding_optimizer_rule.cpp index 13231623..0469d3a5 100644 --- a/src/core/optimizer/path_finding_optimizer_rule.cpp +++ b/src/core/optimizer/path_finding_optimizer_rule.cpp @@ -113,7 +113,6 @@ bool DuckpgqOptimizerExtension::InsertPathFindingOperator(LogicalOperator &op, C path_finding_children, path_finding_expressions, mode); op.children.clear(); op.children.push_back(std::move(path_finding_operator)); - std::cout << "Found path-finding operator" << std::endl; return true; // We have found the path-finding operator, no need to continue } for (auto &child : op.children) { diff --git a/src/include/duckpgq/core/functions/scalar.hpp b/src/include/duckpgq/core/functions/scalar.hpp index 974240f7..26778d7b 100644 --- a/src/include/duckpgq/core/functions/scalar.hpp +++ b/src/include/duckpgq/core/functions/scalar.hpp @@ -19,6 +19,7 @@ struct CoreScalarFunctions { RegisterShortestPathScalarFunction(db); RegisterShortestPathOperatorScalarFunction(db); RegisterCSROperatorCreationScalarFunctions(db); + RegisterIterativeLengthOperatorScalarFunction(db); } private: @@ -32,6 +33,8 @@ struct CoreScalarFunctions { static void RegisterLocalClusteringCoefficientScalarFunction(DatabaseInstance &db); static void RegisterReachabilityScalarFunction(DatabaseInstance &db); static void RegisterShortestPathScalarFunction(DatabaseInstance &db); + static void RegisterIterativeLengthOperatorScalarFunction(DatabaseInstance &db); + static void RegisterShortestPathOperatorScalarFunction(DatabaseInstance &db); static void RegisterCSROperatorCreationScalarFunctions(DatabaseInstance &db); }; From 626599913f42b591f1d0127ecf9d219b0fdf2ced Mon Sep 17 00:00:00 2001 From: dtenwolde Date: Thu, 15 Aug 2024 10:07:13 +0200 Subject: [PATCH 177/249] Make filename consistent --- .../path_finding/{path-finding-cte.test => path_finding_cte.test} | 0 1 file changed, 0 insertions(+), 0 deletions(-) rename test/sql/path_finding/{path-finding-cte.test => path_finding_cte.test} (100%) diff --git a/test/sql/path_finding/path-finding-cte.test b/test/sql/path_finding/path_finding_cte.test similarity index 100% rename from test/sql/path_finding/path-finding-cte.test rename to test/sql/path_finding/path_finding_cte.test From 329785678f736605dcbe98204dd7cd9acbd63a71 Mon Sep 17 00:00:00 2001 From: dtenwolde Date: Thu, 15 Aug 2024 10:07:40 +0200 Subject: [PATCH 178/249] Remove todo --- src/core/optimizer/path_finding_optimizer_rule.cpp | 1 - 1 file changed, 1 deletion(-) diff --git a/src/core/optimizer/path_finding_optimizer_rule.cpp b/src/core/optimizer/path_finding_optimizer_rule.cpp index 0469d3a5..f62596e1 100644 --- a/src/core/optimizer/path_finding_optimizer_rule.cpp +++ b/src/core/optimizer/path_finding_optimizer_rule.cpp @@ -33,7 +33,6 @@ bool DuckpgqOptimizerExtension::InsertPathFindingOperator(LogicalOperator &op, C continue; } auto &bound_function_expression = expr->Cast(); - // TODO create iterativelengthoperator UDF if (bound_function_expression.function.name == "iterativelengthoperator") { op.expressions.emplace_back(make_uniq( expr->alias, LogicalType::BIGINT, ColumnBinding(10, 0))); From 7a095307ccb8d6749d3d03395be99c9ef10833ff Mon Sep 17 00:00:00 2001 From: dtenwolde Date: Thu, 15 Aug 2024 16:29:05 +0200 Subject: [PATCH 179/249] Now getting the columnBinding not hardcoded --- .../logical_path_finding_operator.cpp | 4 +- .../optimizer/path_finding_optimizer_rule.cpp | 233 +++++++++++------- .../logical_path_finding_operator.hpp | 7 +- 3 files changed, 150 insertions(+), 94 deletions(-) diff --git a/src/core/operator/logical_path_finding_operator.cpp b/src/core/operator/logical_path_finding_operator.cpp index d8e8ee72..8e403f5d 100644 --- a/src/core/operator/logical_path_finding_operator.cpp +++ b/src/core/operator/logical_path_finding_operator.cpp @@ -18,7 +18,9 @@ unique_ptr LogicalPathFindingOperator::CreatePlan( vector LogicalPathFindingOperator::GetColumnBindings() { auto left_bindings = children[0]->GetColumnBindings(); - left_bindings.push_back(ColumnBinding(10, 0)); + for (const auto &offset : offsets) { + left_bindings.push_back(ColumnBinding(table_index, offset)); + } return left_bindings; } diff --git a/src/core/optimizer/path_finding_optimizer_rule.cpp b/src/core/optimizer/path_finding_optimizer_rule.cpp index f62596e1..32d2d97d 100644 --- a/src/core/optimizer/path_finding_optimizer_rule.cpp +++ b/src/core/optimizer/path_finding_optimizer_rule.cpp @@ -13,6 +13,7 @@ #include #include #include +#include #include namespace duckpgq { @@ -24,96 +25,68 @@ bool DuckpgqOptimizerExtension::GetPathFindingOption(ClientContext &context) { return value.GetValue(); } -bool DuckpgqOptimizerExtension::InsertPathFindingOperator(LogicalOperator &op, ClientContext &context) { - unique_ptr function_expression; - string mode; - for (int64_t i = op.expressions.size() - 1; i >= 0; --i) { - const auto &expr = op.expressions[i]; - if (expr->expression_class != ExpressionClass::BOUND_FUNCTION) { - continue; - } - auto &bound_function_expression = expr->Cast(); - if (bound_function_expression.function.name == "iterativelengthoperator") { - op.expressions.emplace_back(make_uniq( - expr->alias, LogicalType::BIGINT, ColumnBinding(10, 0))); - function_expression = expr->Copy(); - op.expressions.erase(op.expressions.begin() + i); - mode = "iterativelength"; - } else if (bound_function_expression.function.name == "shortestpathoperator") { - op.expressions.emplace_back(make_uniq( - expr->alias, LogicalType::LIST(LogicalType::BIGINT), ColumnBinding(10, 0))); - function_expression = expr->Copy(); - op.expressions.erase(op.expressions.begin() + i); - mode = "shortestpath"; - } - } - for (const auto &child : op.children) { - vector> path_finding_children; - if (child->type != LogicalOperatorType::LOGICAL_COMPARISON_JOIN) { - continue; - } - auto &get_join = child->Cast(); - //! For now we assume this is enough to detect we have found a - //! path-finding query. Should be improved in the future - if (get_join.children.size() != 2) { - continue; - } - if (get_join.children[1]->type != - LogicalOperatorType::LOGICAL_AGGREGATE_AND_GROUP_BY) { - continue; - } - auto &get_aggregate = get_join.children[1]->Cast(); - auto &get_limit = get_aggregate.children[0]->Cast(); - auto &get_projection = get_limit.children[0]->Cast(); - auto &get_function_expression = - get_projection.expressions[0]->Cast(); - if (get_function_expression.function.name != "csr_operator") { - continue; - } - vector> path_finding_expressions = - std::move(get_function_expression.children); - if (get_join.children[0]->type == LogicalOperatorType::LOGICAL_FILTER) { - auto &get_filter = get_join.children[0]->Cast(); - if (get_filter.children[0]->type != LogicalOperatorType::LOGICAL_GET) { - continue; +// Helper function to create the required BoundColumnRefExpression +unique_ptr CreateReplacementExpression(const string &alias, const string &functionName, idx_t tableIndex, idx_t position) { + if (functionName == "iterativelengthoperator") { + return make_uniq(alias, LogicalType::BIGINT, ColumnBinding(tableIndex, position)); + } + if (functionName == "shortestpathoperator") { + return make_uniq(alias, LogicalType::LIST(LogicalType::BIGINT), ColumnBinding(tableIndex, position)); + } + return nullptr; +} + +void ReplaceExpressions(LogicalProjection &op, unique_ptr &function_expression, string &mode, vector &offsets) { + // Create a temporary vector to hold the new expressions + vector> new_expressions; + new_expressions.reserve(op.expressions.size()); // Reserve space to avoid multiple reallocations + + for (size_t offset = 0; offset < op.expressions.size(); ++offset) { + const auto &expr = op.expressions[offset]; + if (expr->expression_class != ExpressionClass::BOUND_FUNCTION) { + // Directly transfer the expression to the new vector if no replacement is needed + new_expressions.push_back(std::move(op.expressions[offset])); + continue; } - path_finding_children.push_back(std::move(get_filter.children[0])); - } else if (get_join.children[0]->type == - LogicalOperatorType::LOGICAL_EMPTY_RESULT) { - auto default_database = DatabaseManager::GetDefaultDatabase(context); - auto &catalog = Catalog::GetCatalog(context, default_database); - auto &bound_function_expression = function_expression->Cast(); - auto &bind_info = bound_function_expression.bind_info->Cast(); - auto &duckdb_table = catalog.GetEntry(context, DEFAULT_SCHEMA, bind_info.table_to_scan); - auto &get_empty_result = - get_join.children[0]->Cast(); - vector returned_names = {"src", "dst"}; - unique_ptr bind_data; - auto scan_function = duckdb_table.GetScanFunction(context, bind_data); - auto logical_get = make_uniq( - get_empty_result.bindings[0].table_index, scan_function, - std::move(bind_data), get_empty_result.return_types, - returned_names); - vector column_ids_vector; - for (const auto &binding : get_empty_result.bindings) { - column_ids_vector.push_back(binding.column_index); + + auto &bound_function_expression = expr->Cast(); + const auto &function_name = bound_function_expression.function.name; + + if (function_name == "iterativelengthoperator" || function_name == "shortestpathoperator") { + // Create the replacement expression + auto replacement_expr = CreateReplacementExpression(expr->alias, function_name, op.table_index, offset); + if (replacement_expr) { + // store the offsets of the expressions that need to be replaced + offsets.push_back(offset); + // Push the replacement into the new vector + new_expressions.push_back(std::move(replacement_expr)); + // Optionally, copy the original expression if it's needed elsewhere + function_expression = expr->Copy(); + mode = function_name == "iterativelengthoperator" ? "iterativelength" : "shortestpath"; + } else { + // If no replacement is created, throw an internal exception + throw InternalException("Found a bound path-finding function that should be replaced but could not be replaced."); + } + } else { + // If the expression is a bound function but not one we are interested in replacing + new_expressions.push_back(std::move(op.expressions[offset])); } - logical_get->column_ids = std::move(column_ids_vector); - path_finding_children.push_back(std::move(logical_get)); - } - path_finding_children.push_back(std::move(get_projection.children[0])); - // Iterate in reverse to not influence the upcoming iterations when - // erasing an element from the list. Does not work if both iterativelength - // and shortestpath are called in the same query for now. To be improved - // in the future. - - - auto path_finding_operator = make_uniq( - path_finding_children, path_finding_expressions, mode); - op.children.clear(); - op.children.push_back(std::move(path_finding_operator)); - return true; // We have found the path-finding operator, no need to continue } + + // Replace the old expressions vector with the new one + op.expressions = std::move(new_expressions); +} + +bool DuckpgqOptimizerExtension::InsertPathFindingOperator( + LogicalOperator &op, ClientContext &context) { + unique_ptr function_expression; + string mode; + vector offsets; + // Iterate in reverse to not influence the upcoming iterations when + // erasing an element from the list. Does not work if both iterativelength + // and shortestpath are called in the same query for now. To be improved + // in the future. + if (op.type != LogicalOperatorType::LOGICAL_PROJECTION) { for (auto &child : op.children) { if (InsertPathFindingOperator(*child, context)) { return true; @@ -121,21 +94,97 @@ bool DuckpgqOptimizerExtension::InsertPathFindingOperator(LogicalOperator &op, C } return false; } + auto &op_proj = op.Cast(); + ReplaceExpressions(op_proj, function_expression, mode, offsets); + + for (const auto &child : op.children) { + vector> path_finding_children; + if (child->type != LogicalOperatorType::LOGICAL_COMPARISON_JOIN) { + continue; + } + auto &get_join = child->Cast(); + //! For now we assume this is enough to detect we have found a + //! path-finding query. Should be improved in the future + if (get_join.children.size() != 2) { + continue; + } + /*TODO Check both options: + Left is aggregate and right is filter + Right is aggregate, left is filter + */ -void DuckpgqOptimizerExtension::DuckpgqOptimizeFunction(OptimizerExtensionInput &input, - duckdb::unique_ptr &plan) { + if (get_join.children[1]->type != + LogicalOperatorType::LOGICAL_AGGREGATE_AND_GROUP_BY) { + continue; + } + auto &get_aggregate = get_join.children[1]->Cast(); + auto &get_limit = get_aggregate.children[0]->Cast(); + auto &get_projection = get_limit.children[0]->Cast(); + auto &get_function_expression = + get_projection.expressions[0]->Cast(); + if (get_function_expression.function.name != "csr_operator") { + continue; + } + vector> path_finding_expressions = + std::move(get_function_expression.children); + if (get_join.children[0]->type == LogicalOperatorType::LOGICAL_FILTER) { + auto &get_filter = get_join.children[0]->Cast(); + if (get_filter.children[0]->type != LogicalOperatorType::LOGICAL_GET) { + continue; + } + path_finding_children.push_back(std::move(get_filter.children[0])); + } else if (get_join.children[0]->type == + LogicalOperatorType::LOGICAL_EMPTY_RESULT) { + auto default_database = DatabaseManager::GetDefaultDatabase(context); + auto &catalog = Catalog::GetCatalog(context, default_database); + auto &bound_function_expression = + function_expression->Cast(); + auto &bind_info = bound_function_expression.bind_info + ->Cast(); + auto &duckdb_table = catalog.GetEntry( + context, DEFAULT_SCHEMA, bind_info.table_to_scan); + auto &get_empty_result = get_join.children[0]->Cast(); + vector returned_names = {"src", "dst"}; + unique_ptr bind_data; + auto scan_function = duckdb_table.GetScanFunction(context, bind_data); + auto logical_get = make_uniq( + get_empty_result.bindings[0].table_index, scan_function, + std::move(bind_data), get_empty_result.return_types, returned_names); + vector column_ids_vector; + for (const auto &binding : get_empty_result.bindings) { + column_ids_vector.push_back(binding.column_index); + } + logical_get->column_ids = std::move(column_ids_vector); + path_finding_children.push_back(std::move(logical_get)); + } + path_finding_children.push_back(std::move(get_projection.children[0])); + + auto path_finding_operator = make_uniq( + path_finding_children, path_finding_expressions, mode, op_proj.table_index, offsets); + op.children.clear(); + op.children.push_back(std::move(path_finding_operator)); + return true; // We have found the path-finding operator, no need to continue + } + for (auto &child : op.children) { + if (InsertPathFindingOperator(*child, context)) { + return true; + } + } + return false; +} + +void DuckpgqOptimizerExtension::DuckpgqOptimizeFunction( + OptimizerExtensionInput &input, unique_ptr &plan) { if (!GetPathFindingOption(input.context)) { return; } InsertPathFindingOperator(*plan, input.context); } - //------------------------------------------------------------------------------ // Register optimizer //------------------------------------------------------------------------------ -void CorePGQOptimizer::RegisterPathFindingOptimizerRule( - DatabaseInstance &db) { +void CorePGQOptimizer::RegisterPathFindingOptimizerRule(DatabaseInstance &db) { auto &config = DBConfig::GetConfig(db); config.optimizer_extensions.push_back(DuckpgqOptimizerExtension()); diff --git a/src/include/duckpgq/core/operator/logical_path_finding_operator.hpp b/src/include/duckpgq/core/operator/logical_path_finding_operator.hpp index 83598a14..06517107 100644 --- a/src/include/duckpgq/core/operator/logical_path_finding_operator.hpp +++ b/src/include/duckpgq/core/operator/logical_path_finding_operator.hpp @@ -9,10 +9,13 @@ namespace core { class LogicalPathFindingOperator : public LogicalExtensionOperator { public: - explicit LogicalPathFindingOperator(vector> &children_, vector> &expressions_, const string& mode_) + explicit LogicalPathFindingOperator(vector> &children_, vector> &expressions_, + const string& mode_, idx_t table_index_, vector &offsets_) : LogicalExtensionOperator(std::move(expressions_)) { children = std::move(children_); mode = mode_; + table_index = table_index_; + offsets = offsets_; } void Serialize(Serializer &serializer) const override { @@ -32,6 +35,8 @@ class LogicalPathFindingOperator : public LogicalExtensionOperator { public: string mode; + idx_t table_index; + vector offsets; }; } // namespace core From 3b1f192ac3bff582df969a6aaf4b1a3014d8245e Mon Sep 17 00:00:00 2001 From: dtenwolde Date: Thu, 15 Aug 2024 16:46:34 +0200 Subject: [PATCH 180/249] Add tests --- .../path_finding/parallel_path_finding.test | 28 +++++++++++++++++++ 1 file changed, 28 insertions(+) diff --git a/test/sql/path_finding/parallel_path_finding.test b/test/sql/path_finding/parallel_path_finding.test index c90731e1..c1c83cab 100644 --- a/test/sql/path_finding/parallel_path_finding.test +++ b/test/sql/path_finding/parallel_path_finding.test @@ -24,6 +24,31 @@ create table student(id INT); INSERT INTO student(id) VALUES (10), (20), (30), ( statement ok create table know(src INT, dst INT); INSERT INTO know(src, dst) VALUES (40, 20), (10,30), (10,10), (20,10), (30,10); +query III +WITH shortestpath_cte as ( +SELECT src, dst, shortestpathoperator(src, dst, 'pair') as path +FROM pair p +WHERE p.src between (SELECT CSR_OPERATOR( + (SELECT count(a.id) as v_size FROM Student a), + (SELECT count(k.src) as e_size from know k), + a.rowid, + c.rowid, + k.rowid, + t.cnt) FROM Know k + JOIN student a on a.id = k.src + JOIN student c on c.id = k.dst + JOIN (SELECT count(k.src) cnt, a.rowid as a_rowid + FROM student a + LEFT JOIN know k ON k.src = a.id + GROUP BY a.rowid) t + ON t.a_rowid = a.rowid) and p.dst) + SELECT * FROM shortestpath_cte; +---- +0 1 NULL +1 2 [1, 3, 0, 1, 2] +2 0 [2, 4, 0] + + query III SELECT *, shortestpathoperator(src, dst, 'pair') as path FROM pair AS p @@ -46,6 +71,9 @@ WHERE p.src BETWEEN (SELECT CSR_OPERATOR( 1 2 [1, 3, 0, 1, 2] 2 0 [2, 4, 0] + + + statement ok import database 'duckdb-pgq/data/SNB0.003'; From 117da8af69a8d872bf53a123487d51bb1f752d52 Mon Sep 17 00:00:00 2001 From: dtenwolde Date: Mon, 19 Aug 2024 14:40:29 +0200 Subject: [PATCH 181/249] Update header of test --- test/sql/path_finding/parallel_path_finding.test | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/test/sql/path_finding/parallel_path_finding.test b/test/sql/path_finding/parallel_path_finding.test index c1c83cab..a1002ce9 100644 --- a/test/sql/path_finding/parallel_path_finding.test +++ b/test/sql/path_finding/parallel_path_finding.test @@ -1,6 +1,6 @@ -# name: test/sql/join/pathfinding/test_path_finding.test +# name: test/sql/path_finding/parallel_path_finding.test # description: Prototype the path finding operator -# group: [path-finding] +# group: [duckpgq_sql_path_finding] #statement ok #PRAGMA enable_verification From 1cab56173a638d483ea8b9614a2da2e8ec218c64 Mon Sep 17 00:00:00 2001 From: dtenwolde Date: Mon, 19 Aug 2024 15:13:33 +0200 Subject: [PATCH 182/249] Adding error handling --- src/core/optimizer/path_finding_optimizer_rule.cpp | 6 +++++- 1 file changed, 5 insertions(+), 1 deletion(-) diff --git a/src/core/optimizer/path_finding_optimizer_rule.cpp b/src/core/optimizer/path_finding_optimizer_rule.cpp index 32d2d97d..e2a2e970 100644 --- a/src/core/optimizer/path_finding_optimizer_rule.cpp +++ b/src/core/optimizer/path_finding_optimizer_rule.cpp @@ -156,9 +156,13 @@ bool DuckpgqOptimizerExtension::InsertPathFindingOperator( } logical_get->column_ids = std::move(column_ids_vector); path_finding_children.push_back(std::move(logical_get)); + } else { + throw InternalException("Did not find pairs for path-finding operator. The left child was of type " + LogicalOperatorToString(get_join.children[0]->type)); } path_finding_children.push_back(std::move(get_projection.children[0])); - + if (path_finding_children.size() != 2) { + throw InternalException("Path-findig operator should have 2 children"); + } auto path_finding_operator = make_uniq( path_finding_children, path_finding_expressions, mode, op_proj.table_index, offsets); op.children.clear(); From d6eb5e7c316dc5b965876f150d9003ea407266b4 Mon Sep 17 00:00:00 2001 From: dtenwolde Date: Mon, 19 Aug 2024 16:50:39 +0200 Subject: [PATCH 183/249] Rename left and right child --- .../optimizer/path_finding_optimizer_rule.cpp | 44 ++++++++++++------- 1 file changed, 27 insertions(+), 17 deletions(-) diff --git a/src/core/optimizer/path_finding_optimizer_rule.cpp b/src/core/optimizer/path_finding_optimizer_rule.cpp index e2a2e970..e2ef6e2a 100644 --- a/src/core/optimizer/path_finding_optimizer_rule.cpp +++ b/src/core/optimizer/path_finding_optimizer_rule.cpp @@ -82,10 +82,6 @@ bool DuckpgqOptimizerExtension::InsertPathFindingOperator( unique_ptr function_expression; string mode; vector offsets; - // Iterate in reverse to not influence the upcoming iterations when - // erasing an element from the list. Does not work if both iterativelength - // and shortestpath are called in the same query for now. To be improved - // in the future. if (op.type != LogicalOperatorType::LOGICAL_PROJECTION) { for (auto &child : op.children) { if (InsertPathFindingOperator(*child, context)) { @@ -112,12 +108,13 @@ bool DuckpgqOptimizerExtension::InsertPathFindingOperator( Left is aggregate and right is filter Right is aggregate, left is filter */ - - if (get_join.children[1]->type != + auto &left_child = get_join.children[0]; + auto &right_child = get_join.children[1]; + if (right_child->type != LogicalOperatorType::LOGICAL_AGGREGATE_AND_GROUP_BY) { continue; } - auto &get_aggregate = get_join.children[1]->Cast(); + auto &get_aggregate = right_child->Cast(); auto &get_limit = get_aggregate.children[0]->Cast(); auto &get_projection = get_limit.children[0]->Cast(); auto &get_function_expression = @@ -127,37 +124,50 @@ bool DuckpgqOptimizerExtension::InsertPathFindingOperator( } vector> path_finding_expressions = std::move(get_function_expression.children); - if (get_join.children[0]->type == LogicalOperatorType::LOGICAL_FILTER) { - auto &get_filter = get_join.children[0]->Cast(); + + if (left_child->type == LogicalOperatorType::LOGICAL_FILTER) { + auto &get_filter = left_child->Cast(); + if (get_filter.children[0]->type != LogicalOperatorType::LOGICAL_GET) { continue; } + path_finding_children.push_back(std::move(get_filter.children[0])); - } else if (get_join.children[0]->type == - LogicalOperatorType::LOGICAL_EMPTY_RESULT) { + } else if (left_child->type == LogicalOperatorType::LOGICAL_EMPTY_RESULT) { auto default_database = DatabaseManager::GetDefaultDatabase(context); auto &catalog = Catalog::GetCatalog(context, default_database); auto &bound_function_expression = function_expression->Cast(); - auto &bind_info = bound_function_expression.bind_info - ->Cast(); + auto &bind_info = + bound_function_expression.bind_info->Cast(); auto &duckdb_table = catalog.GetEntry( context, DEFAULT_SCHEMA, bind_info.table_to_scan); - auto &get_empty_result = get_join.children[0]->Cast(); + auto &get_empty_result = left_child->Cast(); + vector returned_names = {"src", "dst"}; unique_ptr bind_data; auto scan_function = duckdb_table.GetScanFunction(context, bind_data); + auto logical_get = make_uniq( get_empty_result.bindings[0].table_index, scan_function, - std::move(bind_data), get_empty_result.return_types, returned_names); + std::move(bind_data), get_empty_result.return_types, returned_names + ); + vector column_ids_vector; for (const auto &binding : get_empty_result.bindings) { column_ids_vector.push_back(binding.column_index); } logical_get->column_ids = std::move(column_ids_vector); + path_finding_children.push_back(std::move(logical_get)); - } else { - throw InternalException("Did not find pairs for path-finding operator. The left child was of type " + LogicalOperatorToString(get_join.children[0]->type)); + } else if (left_child->type == LogicalOperatorType::LOGICAL_PROJECTION) { + path_finding_children.push_back(std::move(left_child)); + } + + else { + throw InternalException("Did not find pairs for path-finding operator. " + "The left child was of type " + + LogicalOperatorToString(left_child->type)); } path_finding_children.push_back(std::move(get_projection.children[0])); if (path_finding_children.size() != 2) { From 653340441bcf16ec262cebb76b2b7681687644f8 Mon Sep 17 00:00:00 2001 From: dtenwolde Date: Mon, 19 Aug 2024 16:51:04 +0200 Subject: [PATCH 184/249] Get task value size properly --- src/core/operator/physical_path_finding_operator.cpp | 5 ++++- 1 file changed, 4 insertions(+), 1 deletion(-) diff --git a/src/core/operator/physical_path_finding_operator.cpp b/src/core/operator/physical_path_finding_operator.cpp index 0041d6ed..f2f67826 100644 --- a/src/core/operator/physical_path_finding_operator.cpp +++ b/src/core/operator/physical_path_finding_operator.cpp @@ -894,6 +894,9 @@ PhysicalPathFinding::Finalize(Pipeline &pipeline, Event &event, } else { gstate.global_bfs_state = make_uniq(csr, all_pairs, csr->v_size - 2, num_threads, mode, context, true); } + Value task_size_value; + context.TryGetCurrentSetting("experimental_path_finding_operator_task_size", task_size_value); + gstate.global_bfs_state->split_size = task_size_value.GetValue();; auto const task_size = client_config.set_variables.find("experimental_path_finding_operator_task_size"); gstate.global_bfs_state->split_size = task_size != client_config.set_variables.end() ? task_size->second.GetValue() : 256; @@ -923,7 +926,7 @@ void PhysicalPathFinding::ScheduleBFSTasks(Pipeline &pipeline, Event &event, if (bfs_state->started_searches < gstate.global_tasks->Count()) { auto result_data = FlatVector::GetData(bfs_state->result.data[0]); - auto& result_validity = FlatVector::Validity(bfs_state->result.data[0]); + auto &result_validity = FlatVector::Validity(bfs_state->result.data[0]); std::bitset seen_mask; seen_mask.set(); From c27b73ae54e1de5af93ef8aac8cc13564e8d9fc0 Mon Sep 17 00:00:00 2001 From: dtenwolde Date: Tue, 20 Aug 2024 11:39:15 +0200 Subject: [PATCH 185/249] Update path --- src/include/duckpgq/core/utils/compressed_sparse_row.hpp | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/src/include/duckpgq/core/utils/compressed_sparse_row.hpp b/src/include/duckpgq/core/utils/compressed_sparse_row.hpp index 5740bde0..5d2ebf49 100644 --- a/src/include/duckpgq/core/utils/compressed_sparse_row.hpp +++ b/src/include/duckpgq/core/utils/compressed_sparse_row.hpp @@ -1,7 +1,7 @@ //===----------------------------------------------------------------------===// // DuckPGQ // -// duckpgq/utils/compressed_sparse_row.hpp +// duckpgq/core/utils/compressed_sparse_row.hpp // // //===----------------------------------------------------------------------===// From bd73dc4f8ed568ed1ef435a51b4985feb4a2579b Mon Sep 17 00:00:00 2001 From: dtenwolde Date: Tue, 20 Aug 2024 11:40:32 +0200 Subject: [PATCH 186/249] Move barrier to separate file --- .../physical_path_finding_operator.cpp | 28 ++-------------- src/core/utils/CMakeLists.txt | 1 + src/core/utils/duckpgq_barrier.cpp | 25 +++++++++++++++ .../duckpgq/core/utils/duckpgq_barrier.hpp | 32 +++++++++++++++++++ 4 files changed, 60 insertions(+), 26 deletions(-) create mode 100644 src/core/utils/duckpgq_barrier.cpp create mode 100644 src/include/duckpgq/core/utils/duckpgq_barrier.hpp diff --git a/src/core/operator/physical_path_finding_operator.cpp b/src/core/operator/physical_path_finding_operator.cpp index f2f67826..d8b94636 100644 --- a/src/core/operator/physical_path_finding_operator.cpp +++ b/src/core/operator/physical_path_finding_operator.cpp @@ -14,6 +14,8 @@ #include #include +#include "duckpgq/core/utils/duckpgq_barrier.hpp" + namespace duckpgq { namespace core { @@ -149,33 +151,7 @@ void PathFindingLocalState::CreateCSRVertex(DataChunk &input, }); } -class Barrier { -public: - explicit Barrier(std::size_t iCount) : - mThreshold(iCount), - mCount(iCount), - mGeneration(0) { - } - void Wait() { - std::unique_lock lLock{mMutex}; - auto lGen = mGeneration.load(); - if (!--mCount) { - mGeneration++; - mCount = mThreshold; - mCond.notify_all(); - } else { - mCond.wait(lLock, [this, lGen] { return lGen != mGeneration; }); - } - } - -private: - std::mutex mMutex; - std::condition_variable mCond; - std::size_t mThreshold; - std::atomic mCount; - std::atomic mGeneration; -}; class GlobalBFSState { using GlobalCompressedSparseRow = diff --git a/src/core/utils/CMakeLists.txt b/src/core/utils/CMakeLists.txt index 81e414bf..ea8e2cde 100644 --- a/src/core/utils/CMakeLists.txt +++ b/src/core/utils/CMakeLists.txt @@ -1,6 +1,7 @@ set(EXTENSION_SOURCES ${EXTENSION_SOURCES} ${CMAKE_CURRENT_SOURCE_DIR}/compressed_sparse_row.cpp + ${CMAKE_CURRENT_SOURCE_DIR}/duckpgq_barrier.cpp ${CMAKE_CURRENT_SOURCE_DIR}/duckpgq_bitmap.cpp ${CMAKE_CURRENT_SOURCE_DIR}/duckpgq_utils.cpp PARENT_SCOPE diff --git a/src/core/utils/duckpgq_barrier.cpp b/src/core/utils/duckpgq_barrier.cpp new file mode 100644 index 00000000..882bb5f6 --- /dev/null +++ b/src/core/utils/duckpgq_barrier.cpp @@ -0,0 +1,25 @@ + +#include "duckpgq/core/utils/duckpgq_barrier.hpp" + +#include + +namespace duckpgq { +namespace core { + +Barrier::Barrier(std::size_t iCount) + : mThreshold(iCount), mCount(iCount), mGeneration(0) {} + +void Barrier::Wait() { + std::unique_lock lLock{mMutex}; + auto lGen = mGeneration.load(); + if (!--mCount) { + mGeneration++; + mCount = mThreshold; + mCond.notify_all(); + } else { + mCond.wait(lLock, [this, lGen] { return lGen != mGeneration; }); + } +} + +} // namespace core +} // namespace duckpgq diff --git a/src/include/duckpgq/core/utils/duckpgq_barrier.hpp b/src/include/duckpgq/core/utils/duckpgq_barrier.hpp new file mode 100644 index 00000000..b205db37 --- /dev/null +++ b/src/include/duckpgq/core/utils/duckpgq_barrier.hpp @@ -0,0 +1,32 @@ +//===----------------------------------------------------------------------===// +// DuckPGQ +// +// duckpgq/core/utils/duckpgq_barrier.hpp +// +// +//===----------------------------------------------------------------------===// + + +#pragma once + +#include "mutex" + +namespace duckpgq { +namespace core { + +class Barrier { +public: + explicit Barrier::Barrier(std::size_t iCount); + + void Wait(); + +private: + std::mutex mMutex; + std::condition_variable mCond; + std::size_t mThreshold; + std::atomic mCount; + std::atomic mGeneration; +}; + +} // namespace core +} // namespace duckpgq \ No newline at end of file From 1bf75906f8dc61a478e2e9d622f135ab8aea000c Mon Sep 17 00:00:00 2001 From: dtenwolde Date: Tue, 20 Aug 2024 12:02:11 +0200 Subject: [PATCH 187/249] Move ve struct for path reconstruction to separate file --- .../physical_path_finding_operator.cpp | 30 ------------- .../physical_path_finding_operator.hpp | 4 ++ .../utils/duckpgq_path_reconstruction.hpp | 45 +++++++++++++++++++ 3 files changed, 49 insertions(+), 30 deletions(-) create mode 100644 src/include/duckpgq/core/utils/duckpgq_path_reconstruction.hpp diff --git a/src/core/operator/physical_path_finding_operator.cpp b/src/core/operator/physical_path_finding_operator.cpp index d8b94636..301692da 100644 --- a/src/core/operator/physical_path_finding_operator.cpp +++ b/src/core/operator/physical_path_finding_operator.cpp @@ -20,36 +20,6 @@ namespace duckpgq { namespace core { -struct ve { - // higher 30 bits for v, lower 34 bits for e - const uint8_t v_bits = 30; - const uint8_t e_bits = 34; - uint64_t value; - const uint64_t v_mask = UINT64_MAX << e_bits; - const uint64_t e_mask = UINT64_MAX >> v_bits; - ve() : value(UINT64_MAX) {} - explicit ve(uint64_t value) : value(value) {} - ve(int64_t v, int64_t e) { - uint64_t new_value = 0; - new_value |= v < 0 ? v_mask : (v << e_bits); - new_value |= e < 0 ? e_mask : (e & e_mask); - value = new_value; - } - ve& operator=(std::initializer_list list) { - uint64_t new_value = 0; - auto it = list.begin(); - new_value |= *it < 0 ? v_mask : (*it << e_bits); - new_value |= *(++it) < 0 ? e_mask : (*it & e_mask); - value = new_value; - return *this; - } - inline int64_t GetV() { - return (value & v_mask) == v_mask ? -1 : static_cast(value >> e_bits); - } - inline int64_t GetE() { - return (value & e_mask) == e_mask ? -1 : static_cast(value & e_mask); - } -}; PhysicalPathFinding::PhysicalPathFinding(LogicalExtensionOperator &op, unique_ptr left, diff --git a/src/include/duckpgq/core/operator/physical_path_finding_operator.hpp b/src/include/duckpgq/core/operator/physical_path_finding_operator.hpp index 2d6ecfd2..4fb6c21d 100644 --- a/src/include/duckpgq/core/operator/physical_path_finding_operator.hpp +++ b/src/include/duckpgq/core/operator/physical_path_finding_operator.hpp @@ -12,11 +12,15 @@ #include "duckdb/execution/operator/join/physical_comparison_join.hpp" #include "duckdb/execution/physical_operator.hpp" #include "duckdb/planner/operator/logical_extension_operator.hpp" +#include "duckpgq/core/utils/duckpgq_barrier.hpp" +#include "duckpgq/core/utils/duckpgq_path_reconstruction.hpp" namespace duckpgq { namespace core { + + class PhysicalPathFinding : public PhysicalComparisonJoin { #define LANE_LIMIT 512 diff --git a/src/include/duckpgq/core/utils/duckpgq_path_reconstruction.hpp b/src/include/duckpgq/core/utils/duckpgq_path_reconstruction.hpp new file mode 100644 index 00000000..62d0db3e --- /dev/null +++ b/src/include/duckpgq/core/utils/duckpgq_path_reconstruction.hpp @@ -0,0 +1,45 @@ +#pragma once + +#include "duckpgq/common.hpp" + +namespace duckpgq { + +namespace core { + +struct ve { + // higher 30 bits for v, lower 34 bits for e + const uint8_t v_bits = 30; + const uint8_t e_bits = 34; + uint64_t value; + const uint64_t v_mask = UINT64_MAX << e_bits; + const uint64_t e_mask = UINT64_MAX >> v_bits; + ve() : value(UINT64_MAX) {} + explicit ve(uint64_t value) : value(value) {} + ve(int64_t v, int64_t e) { + uint64_t new_value = 0; + new_value |= v < 0 ? v_mask : (v << e_bits); + new_value |= e < 0 ? e_mask : (e & e_mask); + value = new_value; + } + ve &operator=(std::initializer_list list) { + uint64_t new_value = 0; + auto it = list.begin(); + new_value |= *it < 0 ? v_mask : (*it << e_bits); + new_value |= *(++it) < 0 ? e_mask : (*it & e_mask); + value = new_value; + return *this; + } + inline int64_t GetV() { + return (value & v_mask) == v_mask ? -1 + : static_cast(value >> e_bits); + } + inline int64_t GetE() { + return (value & e_mask) == e_mask ? -1 + : static_cast(value & e_mask); + } +}; + + +} // namespace core + +} // namespace duckpgq From 51f8809827c65e1e6c0d43da1834933da6b2b332 Mon Sep 17 00:00:00 2001 From: dtenwolde Date: Tue, 20 Aug 2024 13:30:50 +0200 Subject: [PATCH 188/249] Remove extra qualification --- src/include/duckpgq/core/utils/duckpgq_barrier.hpp | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/src/include/duckpgq/core/utils/duckpgq_barrier.hpp b/src/include/duckpgq/core/utils/duckpgq_barrier.hpp index b205db37..75eb24bb 100644 --- a/src/include/duckpgq/core/utils/duckpgq_barrier.hpp +++ b/src/include/duckpgq/core/utils/duckpgq_barrier.hpp @@ -16,7 +16,7 @@ namespace core { class Barrier { public: - explicit Barrier::Barrier(std::size_t iCount); + explicit Barrier(std::size_t iCount); void Wait(); From bbf8faaf474396d01b8cb538a7f0f3bd9f7ffce1 Mon Sep 17 00:00:00 2001 From: dtenwolde Date: Tue, 20 Aug 2024 13:31:43 +0200 Subject: [PATCH 189/249] Move shortet path event to separate file --- src/core/operator/event/CMakeLists.txt | 5 +++ .../operator/event/shortest_path_event.cpp | 39 +++++++++++++++++++ .../operator/event/shortest_path_event.hpp | 30 ++++++++++++++ 3 files changed, 74 insertions(+) create mode 100644 src/core/operator/event/CMakeLists.txt create mode 100644 src/core/operator/event/shortest_path_event.cpp create mode 100644 src/include/duckpgq/core/operator/event/shortest_path_event.hpp diff --git a/src/core/operator/event/CMakeLists.txt b/src/core/operator/event/CMakeLists.txt new file mode 100644 index 00000000..72e4b6af --- /dev/null +++ b/src/core/operator/event/CMakeLists.txt @@ -0,0 +1,5 @@ +set(EXTENSION_SOURCES + ${CMAKE_CURRENT_SOURCE_DIR}/shortest_path_event.cpp + ${EXTENSION_SOURCES} + PARENT_SCOPE +) \ No newline at end of file diff --git a/src/core/operator/event/shortest_path_event.cpp b/src/core/operator/event/shortest_path_event.cpp new file mode 100644 index 00000000..3d8d96df --- /dev/null +++ b/src/core/operator/event/shortest_path_event.cpp @@ -0,0 +1,39 @@ +#include "duckpgq/core/operator/event/shortest_path_event.hpp" +#include "duckpgq/core/operator/physical_path_finding_operator.hpp" + +#include + +namespace duckpgq { +namespace core { + +ParallelShortestPathEvent::ParallelShortestPathEvent(PathFindingGlobalState &gstate_p, + Pipeline &pipeline_p) + : BasePipelineEvent(pipeline_p), gstate(gstate_p) { + +} + +void ParallelShortestPathEvent::Schedule() { + auto &bfs_state = gstate.global_bfs_state; + auto &context = pipeline->GetClientContext(); + + vector> bfs_tasks; + for (idx_t tnum = 0; tnum < bfs_state->num_threads; tnum++) { + bfs_tasks.push_back(make_uniq( + shared_from_this(), context, gstate, tnum)); + } + SetTasks(std::move(bfs_tasks)); +} + +void ParallelShortestPathEvent::FinishEvent() { + auto &bfs_state = gstate.global_bfs_state; + + // if remaining pairs, schedule the BFS for the next batch + if (bfs_state->started_searches < gstate.global_tasks->Count()) { + PhysicalPathFinding::ScheduleBFSEvent(*pipeline, *this, gstate); + } +}; + + + +} // namespace core +} // namespace duckpgq \ No newline at end of file diff --git a/src/include/duckpgq/core/operator/event/shortest_path_event.hpp b/src/include/duckpgq/core/operator/event/shortest_path_event.hpp new file mode 100644 index 00000000..f86884e1 --- /dev/null +++ b/src/include/duckpgq/core/operator/event/shortest_path_event.hpp @@ -0,0 +1,30 @@ +//===----------------------------------------------------------------------===// +// DuckDB +// +// duckpgq/core/operators/event/parallel_shortest_path_event.hpp +// +// +//===----------------------------------------------------------------------===// + + +#pragma once + +#include "duckpgq/common.hpp" +#include "duckdb/parallel/base_pipeline_event.hpp" +namespace duckpgq { +namespace core { +class PathFindingGlobalState; + +class ParallelShortestPathEvent : public BasePipelineEvent { +public: + explicit ParallelShortestPathEvent(PathFindingGlobalState &gstate_p, Pipeline &pipeline_p); + + void Schedule() override; + void FinishEvent() override; + +private: + PathFindingGlobalState &gstate; +}; + +} // namespace core +} // namespace duckpgq \ No newline at end of file From b350b235a518f43fa4dbedf8f95619221d8eaf3d Mon Sep 17 00:00:00 2001 From: dtenwolde Date: Tue, 20 Aug 2024 13:31:56 +0200 Subject: [PATCH 190/249] Move task to separate file --- src/core/operator/CMakeLists.txt | 3 + src/core/operator/task/CMakeLists.txt | 5 + src/core/operator/task/shortest_path_task.cpp | 196 ++++++++++++++++++ .../core/operator/task/shortest_path_task.hpp | 33 +++ 4 files changed, 237 insertions(+) create mode 100644 src/core/operator/task/CMakeLists.txt create mode 100644 src/core/operator/task/shortest_path_task.cpp create mode 100644 src/include/duckpgq/core/operator/task/shortest_path_task.hpp diff --git a/src/core/operator/CMakeLists.txt b/src/core/operator/CMakeLists.txt index 02c1c60f..c46a28ab 100644 --- a/src/core/operator/CMakeLists.txt +++ b/src/core/operator/CMakeLists.txt @@ -1,3 +1,6 @@ +add_subdirectory(event) +add_subdirectory(task) + set(EXTENSION_SOURCES ${CMAKE_CURRENT_SOURCE_DIR}/duckpgq_bind.cpp ${CMAKE_CURRENT_SOURCE_DIR}/logical_path_finding_operator.cpp diff --git a/src/core/operator/task/CMakeLists.txt b/src/core/operator/task/CMakeLists.txt new file mode 100644 index 00000000..f7c19616 --- /dev/null +++ b/src/core/operator/task/CMakeLists.txt @@ -0,0 +1,5 @@ +set(EXTENSION_SOURCES + ${CMAKE_CURRENT_SOURCE_DIR}/shortest_path_task.cpp + ${EXTENSION_SOURCES} + PARENT_SCOPE +) \ No newline at end of file diff --git a/src/core/operator/task/shortest_path_task.cpp b/src/core/operator/task/shortest_path_task.cpp new file mode 100644 index 00000000..9656872a --- /dev/null +++ b/src/core/operator/task/shortest_path_task.cpp @@ -0,0 +1,196 @@ +#include "duckpgq/core/operator/task/shortest_path_task.hpp" +#include + +namespace duckpgq { +namespace core { + +PhysicalShortestPathTask::PhysicalShortestPathTask(shared_ptr event_p, ClientContext &context, + PathFindingGlobalState &state, idx_t worker_id) + : ExecutorTask(context, std::move(event_p)), context(context), + state(state), worker_id(worker_id) {} + + TaskExecutionResult PhysicalShortestPathTask::ExecuteTask(TaskExecutionMode mode) override { + auto &bfs_state = state.global_bfs_state; + auto &change = bfs_state->change; + auto &barrier = bfs_state->barrier; + + auto bound = bfs_state->BoundaryCalculation(worker_id); + left = bound.first; + right = bound.second; + + do { + bfs_state->InitTask(worker_id); + + IterativePath(); + barrier.Wait(); + + if (worker_id == 0) { + ReachDetect(); + } + + barrier.Wait(); + } while (change); + + if (worker_id == 0) { + PathConstruction(); + } + + event->FinishTask(); + return TaskExecutionResult::TASK_FINISHED; + } + + + void PhysicalShortestPathTask::IterativePath() { + auto &bfs_state = state.global_bfs_state; + auto &seen = bfs_state->seen; + auto &visit = bfs_state->iter & 1 ? bfs_state->visit1 : bfs_state->visit2; + auto &next = bfs_state->iter & 1 ? bfs_state->visit2 : bfs_state->visit1; + auto &barrier = bfs_state->barrier; + int64_t *v = (int64_t *)state.global_csr->v; + vector &e = state.global_csr->e; + auto &edge_ids = state.global_csr->edge_ids; + auto &parents_ve = bfs_state->parents_ve; + auto &change = bfs_state->change; + + // clear next before each iteration + for (auto i = left; i < right; i++) { + next[i] = 0; + } + + barrier.Wait(); + + while (true) { + auto task = bfs_state->FetchTask(worker_id); + if (task.first == task.second) { + break; + } + auto start = task.first; + auto end = task.second; + + for (auto i = start; i < end; i++) { + if (visit[i].any()) { + for (auto offset = v[i]; offset < v[i + 1]; offset++) { + auto n = e[offset]; + auto edge_id = edge_ids[offset]; + { + std::lock_guard lock(bfs_state->element_locks[n]); + next[n] |= visit[i]; + } + for (auto l = 0; l < LANE_LIMIT; l++) { + if (parents_ve[n][l].GetV() == -1 && visit[i][l]) { + parents_ve[n][l] = {static_cast(i), edge_id}; + } + } + } + } + } + } + + barrier.Wait(); + + for (auto i = left; i < right; i++) { + if (next[i].any()) { + next[i] &= ~seen[i]; + seen[i] |= next[i]; + change |= next[i].any(); + } + } + } + + void PhysicalShortestPathTask::ReachDetect() { + auto &bfs_state = state.global_bfs_state; + auto &change = bfs_state->change; + // detect lanes that finished + for (int64_t lane = 0; lane < LANE_LIMIT; lane++) { + int64_t search_num = bfs_state->lane_to_num[lane]; + + if (search_num >= 0) { // active lane + //! Check if dst for a source has been seen + int64_t dst_pos = bfs_state->vdata_dst.sel->get_index(search_num); + if (bfs_state->seen[bfs_state->dst[dst_pos]][lane]) { + bfs_state->active--; + } + } + } + if (bfs_state->active == 0) { + change = false; + } + // into the next iteration + bfs_state->iter++; + } + + void PhysicalShortestPathTask::PathConstruction() { + auto &bfs_state = state.global_bfs_state; + auto &result = bfs_state->result.data[1]; + auto result_data = FlatVector::GetData(result); + auto &result_validity = FlatVector::Validity(result); + //! Reconstruct the paths + for (int64_t lane = 0; lane < LANE_LIMIT; lane++) { + int64_t search_num = bfs_state->lane_to_num[lane]; + if (search_num == -1) { // empty lanes + continue; + } + + //! Searches that have stopped have found a path + int64_t src_pos = bfs_state->vdata_src.sel->get_index(search_num); + int64_t dst_pos = bfs_state->vdata_dst.sel->get_index(search_num); + if (bfs_state->src[src_pos] == + bfs_state->dst[dst_pos]) { // Source == destination + unique_ptr output = + make_uniq(LogicalType::LIST(LogicalType::BIGINT)); + ListVector::PushBack(*output, bfs_state->src[src_pos]); + ListVector::Append(result, ListVector::GetEntry(*output), + ListVector::GetListSize(*output)); + result_data[search_num].length = ListVector::GetListSize(*output); + result_data[search_num].offset = bfs_state->total_len; + bfs_state->total_len += result_data[search_num].length; + continue; + } + std::vector output_vector; + std::vector output_edge; + auto source_v = bfs_state->src[src_pos]; // Take the source + + auto parent_vertex = + bfs_state->parents_ve[bfs_state->dst[dst_pos]][lane].GetV(); + auto parent_edge = + bfs_state->parents_ve[bfs_state->dst[dst_pos]][lane].GetE(); + + output_vector.push_back( + bfs_state->dst[dst_pos]); // Add destination vertex + output_vector.push_back(parent_edge); + while (parent_vertex != source_v) { // Continue adding vertices until we + // have reached the source vertex + //! -1 is used to signify no parent + if (parent_vertex == -1 || + parent_vertex == + bfs_state->parents_ve[parent_vertex][lane].GetV()) { + result_validity.SetInvalid(search_num); + break; + } + output_vector.push_back(parent_vertex); + parent_edge = bfs_state->parents_ve[parent_vertex][lane].GetE(); + parent_vertex = bfs_state->parents_ve[parent_vertex][lane].GetV(); + output_vector.push_back(parent_edge); + } + + if (!result_validity.RowIsValid(search_num)) { + continue; + } + output_vector.push_back(source_v); + std::reverse(output_vector.begin(), output_vector.end()); + auto output = make_uniq(LogicalType::LIST(LogicalType::BIGINT)); + for (auto val : output_vector) { + Value value_to_insert = val; + ListVector::PushBack(*output, value_to_insert); + } + + result_data[search_num].length = ListVector::GetListSize(*output); + result_data[search_num].offset = bfs_state->total_len; + ListVector::Append(result, ListVector::GetEntry(*output), + ListVector::GetListSize(*output)); + bfs_state->total_len += result_data[search_num].length; + } + } + +} // namespace core +} // namespace duckpgq \ No newline at end of file diff --git a/src/include/duckpgq/core/operator/task/shortest_path_task.hpp b/src/include/duckpgq/core/operator/task/shortest_path_task.hpp new file mode 100644 index 00000000..99c43e35 --- /dev/null +++ b/src/include/duckpgq/core/operator/task/shortest_path_task.hpp @@ -0,0 +1,33 @@ +#pragma once + +#include "duckpgq/common.hpp" +#include + +namespace duckpgq { +namespace core { + +class PhysicalShortestPathTask : public ExecutorTask { +public: + PhysicalShortestPathTask(shared_ptr event_p, ClientContext &context, + PathFindingGlobalState &state, idx_t worker_id); + + TaskExecutionResult ExecuteTask(TaskExecutionMode mode) override; + +private: + void IterativePath(); + + void ReachDetect(); + + void PathConstruction(); + + ClientContext &context; + PathFindingGlobalState &state; + // [left, right) + idx_t left; + idx_t right; + idx_t worker_id; +}; + + +} // namespace core +} // namespace duckpgq \ No newline at end of file From 027a86470e36003ec65b5f47ad1b37a6980e6c14 Mon Sep 17 00:00:00 2001 From: dtenwolde Date: Tue, 20 Aug 2024 13:32:00 +0200 Subject: [PATCH 191/249] Move task to separate file --- .../physical_path_finding_operator.cpp | 689 +++++------------- .../physical_path_finding_operator.hpp | 125 +++- 2 files changed, 304 insertions(+), 510 deletions(-) diff --git a/src/core/operator/physical_path_finding_operator.cpp b/src/core/operator/physical_path_finding_operator.cpp index 301692da..9495e832 100644 --- a/src/core/operator/physical_path_finding_operator.cpp +++ b/src/core/operator/physical_path_finding_operator.cpp @@ -1,6 +1,6 @@ #include "duckpgq/core/operator/physical_path_finding_operator.hpp" -#include #include "duckpgq/common.hpp" +#include #include "duckdb/common/sort/sort.hpp" #include "duckdb/common/types/column/column_data_collection.hpp" @@ -11,7 +11,7 @@ #include "duckdb/parallel/meta_pipeline.hpp" #include "duckdb/parallel/thread_context.hpp" #include -#include +#include #include #include "duckpgq/core/utils/duckpgq_barrier.hpp" @@ -35,7 +35,6 @@ PhysicalPathFinding::PhysicalPathFinding(LogicalExtensionOperator &op, void PhysicalPathFinding::GlobalCompressedSparseRow::InitializeVertex( int64_t v_size_) { lock_guard csr_init_lock(csr_lock); - if (initialized_v) { return; } @@ -71,41 +70,123 @@ void PhysicalPathFinding::GlobalCompressedSparseRow::InitializeEdge( initialized_e = true; } -//===--------------------------------------------------------------------===// -// Sink -//===--------------------------------------------------------------------===// -class PathFindingLocalState : public LocalSinkState { -public: - using GlobalCompressedSparseRow = - PhysicalPathFinding::GlobalCompressedSparseRow; - PathFindingLocalState(ClientContext &context, const PhysicalPathFinding &op, - const idx_t child) : local_tasks(context, op.children[0]->GetTypes()), - local_inputs(context, op.children[1]->GetTypes()) { +GlobalBFSState::GlobalBFSState(shared_ptr csr_, + shared_ptr pairs_, int64_t v_size_, + idx_t num_threads_, idx_t mode_, ClientContext &context_) + : csr(csr_), pairs(pairs_), iter(1), v_size(v_size_), change(false), + started_searches(0), total_len(0), context(context_), seen(v_size_), + visit1(v_size_), visit2(v_size_), num_threads(num_threads_), + task_queues(num_threads_), barrier(num_threads_), + element_locks(v_size_), mode(mode_) { + result.Initialize( + context, {LogicalType::BIGINT, LogicalType::LIST(LogicalType::BIGINT)}, + pairs_->size()); + auto &src_data = pairs->data[0]; + auto &dst_data = pairs->data[1]; + src_data.ToUnifiedFormat(pairs->size(), vdata_src); + dst_data.ToUnifiedFormat(pairs->size(), vdata_dst); + src = FlatVector::GetData(src_data); + dst = FlatVector::GetData(dst_data); + + CreateTasks(); +} + +void GlobalBFSState::Clear() { + iter = 1; + active = 0; + change = false; + // empty visit vectors + for (auto i = 0; i < v_size; i++) { + visit1[i] = 0; + if (mode == 1) { + for (auto j = 0; j < LANE_LIMIT; j++) { + parents_ve[i][j] = {-1, -1}; + } + } } +} - void Sink(DataChunk &input, GlobalCompressedSparseRow &global_csr, idx_t child) { - if (child == 1) { - // Add the tasks (src, dst) to sink - // Optimizations: Eliminate duplicate sources/destinations - // input.Print(); - local_tasks.Append(input); - // local_tasks.Print(); - } else { - // Create CSR - local_inputs.Append(input); - CreateCSRVertex(input, global_csr); +void GlobalBFSState::CreateTasks() { + // workerTasks[workerId] = [task1, task2, ...] + auto queues = &task_queues; + vector>> worker_tasks(num_threads); + auto cur_worker = 0; + int64_t *v = (int64_t *)csr->v; + int64_t current_task_edges = 0; + idx_t current_task_start = 0; + for (idx_t i = 0; i < (idx_t)v_size; i++) { + auto vertex_edges = v[i + 1] - v[i]; + if (current_task_edges + vertex_edges > split_size && + i != current_task_start) { + auto worker_id = cur_worker % num_threads; + pair range = {current_task_start, i}; + worker_tasks[worker_id].push_back(range); + current_task_start = i; + current_task_edges = 0; + cur_worker++; } + current_task_edges += vertex_edges; } + if (current_task_start < (idx_t)v_size) { + auto worker_id = cur_worker % num_threads; + pair range = {current_task_start, v_size}; - static void CreateCSRVertex(DataChunk &input, - GlobalCompressedSparseRow &global_csr); + worker_tasks[worker_id].push_back(range); + } + for (idx_t worker_id = 0; worker_id < num_threads; worker_id++) { + queues->at(worker_id).first.store(0); + queues->at(worker_id).second = worker_tasks[worker_id]; + } +} - ColumnDataCollection local_tasks; - ColumnDataCollection local_inputs; -}; +void GlobalBFSState::InitTask(idx_t worker_id) { task_queues[worker_id].first.store(0); } + +pair GlobalBFSState::FetchTask(idx_t worker_id) { + auto &task_queue = task_queues; + idx_t offset = 0; + do { + auto worker_idx = (worker_id + offset) % task_queue.size(); + auto cur_task_ix = task_queue[worker_idx].first.fetch_add(1); + if (cur_task_ix < task_queue[worker_idx].second.size()) { + return task_queue[worker_idx].second[cur_task_ix]; + } + offset++; + } while (offset < task_queue.size()); + return {0, 0}; +} + +pair GlobalBFSState::BoundaryCalculation(idx_t worker_id) const { + idx_t block_size = ceil((double)v_size / num_threads); + block_size = block_size == 0 ? 1 : block_size; + idx_t left = block_size * worker_id; + idx_t right = std::min(block_size * (worker_id + 1), (idx_t)v_size); + return {left, right}; +} + +//===--------------------------------------------------------------------===// +// Sink +//===--------------------------------------------------------------------===// +PathFindingLocalState::PathFindingLocalState(ClientContext &context, const PhysicalPathFinding &op, + const idx_t child) + : local_tasks(context, op.children[0]->GetTypes()), + local_inputs(context, op.children[1]->GetTypes()) {} + +void PathFindingLocalState::Sink(DataChunk &input, GlobalCompressedSparseRow &global_csr, + idx_t child) { + if (child == 1) { + // Add the tasks (src, dst) to sink + // Optimizations: Eliminate duplicate sources/destinations + local_tasks.Append(input); + local_tasks.Print(); + } else { + // Create CSR + local_inputs.Append(input); + CreateCSRVertex(input, global_csr); + } +} -void PathFindingLocalState::CreateCSRVertex(DataChunk &input, - GlobalCompressedSparseRow &global_csr) { +void PathFindingLocalState::CreateCSRVertex( + DataChunk &input, GlobalCompressedSparseRow &global_csr) { if (!global_csr.initialized_v) { const auto v_size = input.data[8].GetValue(0).GetValue(); global_csr.InitializeVertex(v_size); @@ -121,192 +202,27 @@ void PathFindingLocalState::CreateCSRVertex(DataChunk &input, }); } +PathFindingGlobalState::PathFindingGlobalState(ClientContext &context, + const PhysicalPathFinding &op) { + global_tasks = + make_uniq(context, op.children[0]->GetTypes()); + global_inputs = + make_uniq(context, op.children[1]->GetTypes()); + global_csr = make_uniq(context); + child = 0; + mode = op.mode; +} - -class GlobalBFSState { - using GlobalCompressedSparseRow = - PhysicalPathFinding::GlobalCompressedSparseRow; -public: - GlobalBFSState(shared_ptr csr_, shared_ptr pairs_, int64_t v_size_, - idx_t num_threads_, idx_t mode_, ClientContext &context_) - : csr(csr_), pairs(pairs_), iter(1), v_size(v_size_), change(false), - started_searches(0), total_len(0), context(context_), seen(v_size_), visit1(v_size_), visit2(v_size_), - num_threads(num_threads_), task_queues(num_threads_), barrier(num_threads_), element_locks(v_size_), mode(mode_) { - result.Initialize(context, {LogicalType::BIGINT, LogicalType::LIST(LogicalType::BIGINT)}, pairs_->size()); - auto &src_data = pairs->data[0]; - auto &dst_data = pairs->data[1]; - src_data.ToUnifiedFormat(pairs->size(), vdata_src); - dst_data.ToUnifiedFormat(pairs->size(), vdata_dst); - src = FlatVector::GetData(src_data); - dst = FlatVector::GetData(dst_data); - - CreateTasks(); - } - - GlobalBFSState(shared_ptr csr_, shared_ptr pairs_, int64_t v_size_, - idx_t num_threads_, idx_t mode_, ClientContext &context_, bool is_path_) - : csr(csr_), pairs(pairs_), iter(1), v_size(v_size_), change(false), - started_searches(0), total_len(0), context(context_), seen(v_size_), visit1(v_size_), visit2(v_size_), - parents_ve(v_size_), num_threads(num_threads_), task_queues(num_threads_), barrier(num_threads_), - element_locks(v_size_), mode(mode_) { - result.Initialize(context, {LogicalType::BIGINT, LogicalType::LIST(LogicalType::BIGINT)}, pairs_->size()); - auto &src_data = pairs->data[0]; - auto &dst_data = pairs->data[1]; - src_data.ToUnifiedFormat(pairs->size(), vdata_src); - dst_data.ToUnifiedFormat(pairs->size(), vdata_dst); - src = FlatVector::GetData(src_data); - dst = FlatVector::GetData(dst_data); - - CreateTasks(); - } - - void Clear() { - iter = 1; - active = 0; - change = false; - // empty visit vectors - for (auto i = 0; i < v_size; i++) { - visit1[i] = 0; - if (mode == 1) { - for (auto j = 0; j < LANE_LIMIT; j++) { - parents_ve[i][j] = {-1, -1}; - } - } - } - } - - - void CreateTasks() { - // workerTasks[workerId] = [task1, task2, ...] - auto queues = &task_queues; - vector>> worker_tasks(num_threads); - auto cur_worker = 0; - int64_t *v = (int64_t*)csr->v; - int64_t current_task_edges = 0; - idx_t current_task_start = 0; - for (idx_t i = 0; i < (idx_t)v_size; i++) { - auto vertex_edges = v[i + 1] - v[i]; - if (current_task_edges + vertex_edges > split_size && i != current_task_start) { - auto worker_id = cur_worker % num_threads; - pair range = {current_task_start, i}; - worker_tasks[worker_id].push_back(range); - current_task_start = i; - current_task_edges = 0; - cur_worker++; - } - current_task_edges += vertex_edges; - } - if (current_task_start < (idx_t)v_size) { - auto worker_id = cur_worker % num_threads; - pair range = {current_task_start, v_size}; - worker_tasks[worker_id].push_back(range); - } - for (idx_t worker_id = 0; worker_id < num_threads; worker_id++) { - queues->at(worker_id).first.store(0); - queues->at(worker_id).second = worker_tasks[worker_id]; - } - } - - void InitTask(idx_t worker_id) { - task_queues[worker_id].first.store(0); - } - - pair FetchTask(idx_t worker_id) { - auto& task_queue = task_queues; - idx_t offset = 0; - do { - auto worker_idx = (worker_id + offset) % task_queue.size(); - auto cur_task_ix = task_queue[worker_idx].first.fetch_add(1); - if (cur_task_ix < task_queue[worker_idx].second.size()) { - return task_queue[worker_idx].second[cur_task_ix]; - } else { - offset++; - } - } while (offset < task_queue.size()); - return {0, 0}; - } - - pair BoundaryCalculation(idx_t worker_id) { - idx_t block_size = ceil((double)v_size / num_threads); - block_size = block_size == 0 ? 1 : block_size; - idx_t left = block_size * worker_id; - idx_t right = std::min(block_size * (worker_id + 1), (idx_t)v_size); - return {left, right}; - } - -public: - shared_ptr csr; - shared_ptr pairs; - int64_t iter; - int64_t v_size; - bool change; - idx_t started_searches; - int64_t total_len; - int64_t *src; - int64_t *dst; - UnifiedVectorFormat vdata_src; - UnifiedVectorFormat vdata_dst; - int64_t lane_to_num[LANE_LIMIT]; - idx_t active = 0; - DataChunk result; // 0 for length, 1 for path - ClientContext& context; - vector> seen; - vector> visit1; - vector> visit2; - vector> parents_ve; - - idx_t num_threads; - // task_queues[workerId] = {curTaskIx, queuedTasks} - // queuedTasks[curTaskIx] = {start, end} - vector, vector>>> task_queues; - int64_t split_size = 256; - - Barrier barrier; - - // lock for next - mutable vector element_locks; - - idx_t mode; -}; - -class PathFindingGlobalState : public GlobalSinkState { -public: - using GlobalCompressedSparseRow = - PhysicalPathFinding::GlobalCompressedSparseRow; - PathFindingGlobalState(ClientContext &context, - const PhysicalPathFinding &op) { - global_tasks = make_uniq(context, op.children[0]->GetTypes()); - global_inputs = make_uniq(context, op.children[1]->GetTypes()); - global_csr = make_uniq(context); - child = 0; - mode = op.mode; - } - - PathFindingGlobalState(PathFindingGlobalState &prev) - : GlobalSinkState(prev), global_tasks(std::move(prev.global_tasks)), +PathFindingGlobalState::PathFindingGlobalState(PathFindingGlobalState &prev) + : GlobalSinkState(prev), global_tasks(std::move(prev.global_tasks)), global_inputs(std::move(prev.global_inputs)), - global_csr(std::move(prev.global_csr)), - global_bfs_state(std::move(prev.global_bfs_state)), child(prev.child + 1), mode(prev.mode) { - } + global_csr(std::move(prev.global_csr)), + global_bfs_state(std::move(prev.global_bfs_state)), + child(prev.child + 1), mode(prev.mode) {} - void Sink(DataChunk &input, PathFindingLocalState &lstate) { - lstate.Sink(input, *global_csr, child); - } - - // pairs is a 2-column table with src and dst - unique_ptr global_tasks; - unique_ptr global_inputs; - // pairs with path exists - // ColumnDataCollection global_results; - ColumnDataScanState scan_state; - ColumnDataAppendState append_state; - - shared_ptr global_csr; - // state for BFS - unique_ptr global_bfs_state; - size_t child; - string mode; -}; +void PathFindingGlobalState::Sink(DataChunk &input, PathFindingLocalState &lstate) { + lstate.Sink(input, *global_csr, child); +} unique_ptr PhysicalPathFinding::GetGlobalSinkState(ClientContext &context) const { @@ -352,14 +268,15 @@ PhysicalPathFinding::Combine(ExecutionContext &context, class PhysicalIterativeTask : public ExecutorTask { public: - PhysicalIterativeTask(shared_ptr event_p, ClientContext &context, PathFindingGlobalState &state, idx_t worker_id) - : ExecutorTask(context, std::move(event_p)), context(context), state(state), worker_id(worker_id) { - } + PhysicalIterativeTask(shared_ptr event_p, ClientContext &context, + PathFindingGlobalState &state, idx_t worker_id) + : ExecutorTask(context, std::move(event_p)), context(context), + state(state), worker_id(worker_id) {} - TaskExecutionResult ExecuteTask(TaskExecutionMode mode) override { - auto& bfs_state = state.global_bfs_state; - auto& change = bfs_state->change; - auto& barrier = bfs_state->barrier; + TaskExecutionResult ExecuteTask(TaskExecutionMode mode) override { + auto &bfs_state = state.global_bfs_state; + auto &change = bfs_state->change; + auto &barrier = bfs_state->barrier; auto bound = bfs_state->BoundaryCalculation(worker_id); left = bound.first; @@ -382,21 +299,21 @@ class PhysicalIterativeTask : public ExecutorTask { UnReachableSet(); } - event->FinishTask(); - return TaskExecutionResult::TASK_FINISHED; - } + event->FinishTask(); + return TaskExecutionResult::TASK_FINISHED; + } private: void IterativeLength() { - auto& bfs_state = state.global_bfs_state; - auto& seen = bfs_state->seen; - auto& visit = bfs_state->iter & 1 ? bfs_state->visit1 : bfs_state->visit2; - auto& next = bfs_state->iter & 1 ? bfs_state->visit2 : bfs_state->visit1; - auto& barrier = bfs_state->barrier; + auto &bfs_state = state.global_bfs_state; + auto &seen = bfs_state->seen; + auto &visit = bfs_state->iter & 1 ? bfs_state->visit1 : bfs_state->visit2; + auto &next = bfs_state->iter & 1 ? bfs_state->visit2 : bfs_state->visit1; + auto &barrier = bfs_state->barrier; int64_t *v = (int64_t *)state.global_csr->v; vector &e = state.global_csr->e; - auto& lane_to_num = bfs_state->lane_to_num; - auto& change = bfs_state->change; + auto &lane_to_num = bfs_state->lane_to_num; + auto &change = bfs_state->change; // clear next before each iteration for (auto i = left; i < right; i++) { @@ -447,7 +364,7 @@ class PhysicalIterativeTask : public ExecutorTask { int64_t dst_pos = bfs_state->vdata_dst.sel->get_index(search_num); if (bfs_state->seen[bfs_state->dst[dst_pos]][lane]) { result_data[search_num] = - bfs_state->iter; /* found at iter => iter = path length */ + bfs_state->iter; /* found at iter => iter = path length */ bfs_state->lane_to_num[lane] = -1; // mark inactive bfs_state->active--; } @@ -474,9 +391,10 @@ class PhysicalIterativeTask : public ExecutorTask { } } } + private: - ClientContext &context; - PathFindingGlobalState &state; + ClientContext &context; + PathFindingGlobalState &state; // [left, right) idx_t left; idx_t right; @@ -485,14 +403,15 @@ class PhysicalIterativeTask : public ExecutorTask { class PhysicalCSREdgeCreationTask : public ExecutorTask { public: - PhysicalCSREdgeCreationTask(shared_ptr event_p, ClientContext &context, PathFindingGlobalState &state) - : ExecutorTask(context, std::move(event_p)), context(context), state(state) { - } + PhysicalCSREdgeCreationTask(shared_ptr event_p, ClientContext &context, + PathFindingGlobalState &state) + : ExecutorTask(context, std::move(event_p)), context(context), + state(state) {} TaskExecutionResult ExecuteTask(TaskExecutionMode mode) override { - auto& global_inputs = state.global_inputs; - auto& global_csr = state.global_csr; - auto& scan_state = state.scan_state; + auto &global_inputs = state.global_inputs; + auto &global_csr = state.global_csr; + auto &scan_state = state.scan_state; DataChunk input; global_inputs->InitializeScanChunk(input); @@ -518,6 +437,7 @@ class PhysicalCSREdgeCreationTask : public ExecutorTask { }); } event->FinishTask(); + global_csr->Print(); return TaskExecutionResult::TASK_FINISHED; } @@ -529,8 +449,7 @@ class PhysicalCSREdgeCreationTask : public ExecutorTask { class CSREdgeCreationEvent : public BasePipelineEvent { public: CSREdgeCreationEvent(PathFindingGlobalState &gstate_p, Pipeline &pipeline_p) - : BasePipelineEvent(pipeline_p), gstate(gstate_p) { - } + : BasePipelineEvent(pipeline_p), gstate(gstate_p) {} PathFindingGlobalState &gstate; @@ -538,14 +457,15 @@ class CSREdgeCreationEvent : public BasePipelineEvent { auto &context = pipeline->GetClientContext(); auto &ts = TaskScheduler::GetScheduler(context); idx_t num_threads = ts.NumberOfThreads(); - auto& scan_state = gstate.scan_state; - auto& global_inputs = gstate.global_inputs; + auto &scan_state = gstate.scan_state; + auto &global_inputs = gstate.global_inputs; global_inputs->InitializeScan(scan_state); vector> tasks; for (idx_t tnum = 0; tnum < num_threads; tnum++) { - tasks.push_back(make_uniq(shared_from_this(), context, gstate)); + tasks.push_back(make_uniq(shared_from_this(), + context, gstate)); } SetTasks(std::move(tasks)); } @@ -559,244 +479,22 @@ class CSREdgeCreationEvent : public BasePipelineEvent { class ParallelIterativeEvent : public BasePipelineEvent { public: - ParallelIterativeEvent(PathFindingGlobalState &gstate_p, Pipeline &pipeline_p) - : BasePipelineEvent(pipeline_p), gstate(gstate_p) { - } - - PathFindingGlobalState &gstate; - -public: - void Schedule() override { - auto &bfs_state = gstate.global_bfs_state; - auto &context = pipeline->GetClientContext(); - - vector> bfs_tasks; - for (idx_t tnum = 0; tnum < bfs_state->num_threads; tnum++) { - bfs_tasks.push_back(make_uniq(shared_from_this(), context, gstate, tnum)); - } - SetTasks(std::move(bfs_tasks)); - } - - void FinishEvent() override { - auto& bfs_state = gstate.global_bfs_state; - - // if remaining pairs, schedule the BFS for the next batch - if (bfs_state->started_searches < gstate.global_tasks->Count()) { - PhysicalPathFinding::ScheduleBFSTasks(*pipeline, *this, gstate); - } - } -}; - -class PhysicalShortestPathTask : public ExecutorTask { -public: - PhysicalShortestPathTask(shared_ptr event_p, ClientContext &context, PathFindingGlobalState &state, idx_t worker_id) - : ExecutorTask(context, std::move(event_p)), context(context), state(state), worker_id(worker_id) { - } - - TaskExecutionResult ExecuteTask(TaskExecutionMode mode) override { - auto& bfs_state = state.global_bfs_state; - auto& change = bfs_state->change; - auto& barrier = bfs_state->barrier; - - auto bound = bfs_state->BoundaryCalculation(worker_id); - left = bound.first; - right = bound.second; - - do { - bfs_state->InitTask(worker_id); - - IterativePath(); - barrier.Wait(); - - if (worker_id == 0) { - ReachDetect(); - } - - barrier.Wait(); - } while (change); - - if (worker_id == 0) { - PathConstruction(); - } - - event->FinishTask(); - return TaskExecutionResult::TASK_FINISHED; - } - -private: - void IterativePath() { - auto& bfs_state = state.global_bfs_state; - auto& seen = bfs_state->seen; - auto& visit = bfs_state->iter & 1 ? bfs_state->visit1 : bfs_state->visit2; - auto& next = bfs_state->iter & 1 ? bfs_state->visit2 : bfs_state->visit1; - auto& barrier = bfs_state->barrier; - int64_t *v = (int64_t *)state.global_csr->v; - vector &e = state.global_csr->e; - auto& edge_ids = state.global_csr->edge_ids; - auto& parents_ve = bfs_state->parents_ve; - auto& change = bfs_state->change; - - // clear next before each iteration - for (auto i = left; i < right; i++) { - next[i] = 0; - } - - barrier.Wait(); - - while (true) { - auto task = bfs_state->FetchTask(worker_id); - if (task.first == task.second) { - break; - } - auto start = task.first; - auto end = task.second; - - for (auto i = start; i < end; i++) { - if (visit[i].any()) { - for (auto offset = v[i]; offset < v[i + 1]; offset++) { - auto n = e[offset]; - auto edge_id = edge_ids[offset]; - { - std::lock_guard lock(bfs_state->element_locks[n]); - next[n] |= visit[i]; - } - for (auto l = 0; l < LANE_LIMIT; l++) { - if (parents_ve[n][l].GetV() == -1 && visit[i][l]) { - parents_ve[n][l] = {static_cast(i), edge_id}; - } - } - } - } - } - } - - barrier.Wait(); - - for (auto i = left; i < right; i++) { - if (next[i].any()) { - next[i] &= ~seen[i]; - seen[i] |= next[i]; - change |= next[i].any(); - } - } - } - - void ReachDetect() { - auto &bfs_state = state.global_bfs_state; - auto &change = bfs_state->change; - // detect lanes that finished - for (int64_t lane = 0; lane < LANE_LIMIT; lane++) { - int64_t search_num = bfs_state->lane_to_num[lane]; - - if (search_num >= 0) { // active lane - //! Check if dst for a source has been seen - int64_t dst_pos = bfs_state->vdata_dst.sel->get_index(search_num); - if (bfs_state->seen[bfs_state->dst[dst_pos]][lane]) { - bfs_state->active--; - } - } - } - if (bfs_state->active == 0) { - change = false; - } - // into the next iteration - bfs_state->iter++; - } - - void PathConstruction() { - auto &bfs_state = state.global_bfs_state; - auto &result = bfs_state->result.data[1]; - auto result_data = FlatVector::GetData(result); - auto &result_validity = FlatVector::Validity(result); - //! Reconstruct the paths - for (int64_t lane = 0; lane < LANE_LIMIT; lane++) { - int64_t search_num = bfs_state->lane_to_num[lane]; - if (search_num == -1) { // empty lanes - continue; - } - - //! Searches that have stopped have found a path - int64_t src_pos = bfs_state->vdata_src.sel->get_index(search_num); - int64_t dst_pos = bfs_state->vdata_dst.sel->get_index(search_num); - if (bfs_state->src[src_pos] == bfs_state->dst[dst_pos]) { // Source == destination - unique_ptr output = - make_uniq(LogicalType::LIST(LogicalType::BIGINT)); - ListVector::PushBack(*output, bfs_state->src[src_pos]); - ListVector::Append(result, ListVector::GetEntry(*output), - ListVector::GetListSize(*output)); - result_data[search_num].length = ListVector::GetListSize(*output); - result_data[search_num].offset = bfs_state->total_len; - bfs_state->total_len += result_data[search_num].length; - continue; - } - std::vector output_vector; - std::vector output_edge; - auto source_v = bfs_state->src[src_pos]; // Take the source - - auto parent_vertex = bfs_state->parents_ve[bfs_state->dst[dst_pos]][lane].GetV(); - auto parent_edge = bfs_state->parents_ve[bfs_state->dst[dst_pos]][lane].GetE(); - - output_vector.push_back(bfs_state->dst[dst_pos]); // Add destination vertex - output_vector.push_back(parent_edge); - while (parent_vertex != source_v) { // Continue adding vertices until we - // have reached the source vertex - //! -1 is used to signify no parent - if (parent_vertex == -1 || - parent_vertex == bfs_state->parents_ve[parent_vertex][lane].GetV()) { - result_validity.SetInvalid(search_num); - break; - } - output_vector.push_back(parent_vertex); - parent_edge = bfs_state->parents_ve[parent_vertex][lane].GetE(); - parent_vertex = bfs_state->parents_ve[parent_vertex][lane].GetV(); - output_vector.push_back(parent_edge); - } - - if (!result_validity.RowIsValid(search_num)) { - continue; - } - output_vector.push_back(source_v); - std::reverse(output_vector.begin(), output_vector.end()); - auto output = make_uniq(LogicalType::LIST(LogicalType::BIGINT)); - for (auto val : output_vector) { - Value value_to_insert = val; - ListVector::PushBack(*output, value_to_insert); - } - - result_data[search_num].length = ListVector::GetListSize(*output); - result_data[search_num].offset = bfs_state->total_len; - ListVector::Append(result, ListVector::GetEntry(*output), - ListVector::GetListSize(*output)); - bfs_state->total_len += result_data[search_num].length; - } - } - - ClientContext &context; - PathFindingGlobalState &state; - // [left, right) - idx_t left; - idx_t right; - idx_t worker_id; -}; - -class ParallelShortestPathEvent : public BasePipelineEvent { -public: - ParallelShortestPathEvent(PathFindingGlobalState &gstate_p, Pipeline &pipeline_p) - : BasePipelineEvent(pipeline_p), gstate(gstate_p) { - } + ParallelIterativeEvent(PathFindingGlobalState &gstate_p, Pipeline &pipeline_p) + : BasePipelineEvent(pipeline_p), gstate(gstate_p) {} PathFindingGlobalState &gstate; public: void Schedule() override { auto &bfs_state = gstate.global_bfs_state; - auto &context = pipeline->GetClientContext(); + auto &context = pipeline->GetClientContext(); - vector> bfs_tasks; + vector> bfs_tasks; for (idx_t tnum = 0; tnum < bfs_state->num_threads; tnum++) { - bfs_tasks.push_back(make_uniq(shared_from_this(), context, gstate, tnum)); + bfs_tasks.push_back(make_uniq( + shared_from_this(), context, gstate, tnum)); } - SetTasks(std::move(bfs_tasks)); + SetTasks(std::move(bfs_tasks)); } void FinishEvent() override { @@ -804,7 +502,7 @@ class ParallelShortestPathEvent : public BasePipelineEvent { // if remaining pairs, schedule the BFS for the next batch if (bfs_state->started_searches < gstate.global_tasks->Count()) { - PhysicalPathFinding::ScheduleBFSTasks(*pipeline, *this, gstate); + PhysicalPathFinding::ScheduleBFSEvent(*pipeline, *this, gstate); } } }; @@ -832,24 +530,25 @@ PhysicalPathFinding::Finalize(Pipeline &pipeline, Event &event, } auto &ts = TaskScheduler::GetScheduler(context); - idx_t num_threads = ts.NumberOfThreads(); - auto& client_config = ClientConfig::GetConfig(context); + idx_t num_threads = ts.NumberOfThreads(); + auto &client_config = ClientConfig::GetConfig(context); idx_t mode = this->mode == "iterativelength" ? 0 : 1; if (mode == 0) { - gstate.global_bfs_state = make_uniq(csr, all_pairs, csr->v_size - 2, num_threads, mode, context); + gstate.global_bfs_state = make_uniq( + csr, all_pairs, csr->v_size - 2, num_threads, mode, context); } else { - gstate.global_bfs_state = make_uniq(csr, all_pairs, csr->v_size - 2, num_threads, mode, context, true); + gstate.global_bfs_state = make_uniq( + csr, all_pairs, csr->v_size - 2, num_threads, mode, context, true); } Value task_size_value; - context.TryGetCurrentSetting("experimental_path_finding_operator_task_size", task_size_value); - gstate.global_bfs_state->split_size = task_size_value.GetValue();; - - auto const task_size = client_config.set_variables.find("experimental_path_finding_operator_task_size"); - gstate.global_bfs_state->split_size = task_size != client_config.set_variables.end() ? task_size->second.GetValue() : 256; + context.TryGetCurrentSetting("experimental_path_finding_operator_task_size", + task_size_value); + gstate.global_bfs_state->split_size = task_size_value.GetValue(); + ; // Schedule the first round of BFS tasks if (all_pairs->size() > 0) { - ScheduleBFSTasks(pipeline, event, gstate); + ScheduleBFSEvent(pipeline, event, gstate); } } @@ -859,18 +558,16 @@ PhysicalPathFinding::Finalize(Pipeline &pipeline, Event &event, return SinkFinalizeType::READY; } -void PhysicalPathFinding::ScheduleBFSTasks(Pipeline &pipeline, Event &event, +void PhysicalPathFinding::ScheduleBFSEvent(Pipeline &pipeline, Event &event, GlobalSinkState &state) { auto &gstate = state.Cast(); auto &bfs_state = gstate.global_bfs_state; - int64_t *v = (int64_t *)gstate.global_csr->v; // for every batch of pairs, schedule a BFS task bfs_state->Clear(); // remaining pairs if (bfs_state->started_searches < gstate.global_tasks->Count()) { - auto result_data = FlatVector::GetData(bfs_state->result.data[0]); auto &result_validity = FlatVector::Validity(bfs_state->result.data[0]); std::bitset seen_mask; @@ -902,13 +599,14 @@ void PhysicalPathFinding::ScheduleBFSTasks(Pipeline &pipeline, Event &event, } if (gstate.mode == "iterativelength") { - auto bfs_event = make_shared_ptr(gstate, pipeline); + auto bfs_event = + make_shared_ptr(gstate, pipeline); event.InsertEvent(std::move(bfs_event)); } else if (gstate.mode == "shortestpath") { - auto bfs_event = make_shared_ptr(gstate, pipeline); + auto bfs_event = + make_shared_ptr(gstate, pipeline); event.InsertEvent(std::move(bfs_event)); } - } } @@ -993,13 +691,12 @@ PhysicalPathFinding::GetData(ExecutionContext &context, DataChunk &result, } else if (pf_sink.mode == "shortestpath") { result.Fuse(*result_path); } else { - throw NotImplementedException("Unrecognized mode for Path Finding"); + throw NotImplementedException("Unrecognized mode for Path Finding"); } // result.Print(); - return result.size() == 0 - ? SourceResultType::FINISHED - : SourceResultType::HAVE_MORE_OUTPUT; + return result.size() == 0 ? SourceResultType::FINISHED + : SourceResultType::HAVE_MORE_OUTPUT; } //===--------------------------------------------------------------------===// @@ -1034,4 +731,4 @@ void PhysicalPathFinding::BuildPipelines(Pipeline ¤t, } } // namespace core -} // namespace duckdb +} // namespace duckpgq diff --git a/src/include/duckpgq/core/operator/physical_path_finding_operator.hpp b/src/include/duckpgq/core/operator/physical_path_finding_operator.hpp index 4fb6c21d..06ba43cb 100644 --- a/src/include/duckpgq/core/operator/physical_path_finding_operator.hpp +++ b/src/include/duckpgq/core/operator/physical_path_finding_operator.hpp @@ -1,7 +1,7 @@ //===----------------------------------------------------------------------===// // DuckDB // -// duckpgq/operators/physical_path_finding_operator.hpp +// duckpgq/core/operator/physical_path_finding_operator.hpp // // //===----------------------------------------------------------------------===// @@ -19,15 +19,13 @@ namespace duckpgq { namespace core { - - class PhysicalPathFinding : public PhysicalComparisonJoin { #define LANE_LIMIT 512 public: class GlobalCompressedSparseRow { public: - GlobalCompressedSparseRow(ClientContext &context){ + explicit GlobalCompressedSparseRow(ClientContext &context){ }; ~GlobalCompressedSparseRow() { if (v) { @@ -104,22 +102,16 @@ class PhysicalPathFinding : public PhysicalComparisonJoin { const PhysicalPathFinding &op; //! Local copy of the expression executor ExpressionExecutor executor; - //! Final result for the path-finding pairs DataChunk local_results; - }; -public: - static constexpr const PhysicalOperatorType TYPE = - PhysicalOperatorType::EXTENSION; - - -public: PhysicalPathFinding(LogicalExtensionOperator &op, unique_ptr left, unique_ptr right); -public: + + static constexpr PhysicalOperatorType TYPE = + PhysicalOperatorType::EXTENSION; vector> expressions; string mode; // "iterativelength" or "shortestpath" @@ -164,7 +156,112 @@ class PhysicalPathFinding : public PhysicalComparisonJoin { void BuildPipelines(Pipeline ¤t, MetaPipeline &meta_pipeline) override; // //! Schedules tasks to calculate the next iteration of the path-finding - static void ScheduleBFSTasks(Pipeline &pipeline, Event &event, GlobalSinkState &state); + static void ScheduleBFSEvent(Pipeline &pipeline, Event &event, GlobalSinkState &state); +}; + +//===--------------------------------------------------------------------===// +// Sink +//===--------------------------------------------------------------------===// +class PathFindingLocalState : public LocalSinkState { +public: + using GlobalCompressedSparseRow = + PhysicalPathFinding::GlobalCompressedSparseRow; + PathFindingLocalState(ClientContext &context, const PhysicalPathFinding &op, + idx_t child); + + void Sink(DataChunk &input, GlobalCompressedSparseRow &global_csr, + idx_t child); + + static void CreateCSRVertex(DataChunk &input, + GlobalCompressedSparseRow &global_csr); + + ColumnDataCollection local_tasks; + ColumnDataCollection local_inputs; +}; + +class GlobalBFSState { + using GlobalCompressedSparseRow = + PhysicalPathFinding::GlobalCompressedSparseRow; + +public: + GlobalBFSState(shared_ptr csr_, + shared_ptr pairs_, int64_t v_size_, + idx_t num_threads_, idx_t mode_, ClientContext &context_); + + GlobalBFSState(shared_ptr csr_, + shared_ptr pairs_, int64_t v_size_, + idx_t num_threads_, idx_t mode_, ClientContext &context_, + bool is_path_); + + void Clear(); + + void CreateTasks(); + + void InitTask(idx_t worker_id); + + pair FetchTask(idx_t worker_id); + + pair BoundaryCalculation(idx_t worker_id) const; + +public: + shared_ptr csr; + shared_ptr pairs; + int64_t iter; + int64_t v_size; + bool change; + idx_t started_searches; + int64_t total_len; + int64_t *src; + int64_t *dst; + UnifiedVectorFormat vdata_src; + UnifiedVectorFormat vdata_dst; + int64_t lane_to_num[LANE_LIMIT]; + idx_t active = 0; + DataChunk result; // 0 for length, 1 for path + ClientContext &context; + vector> seen; + vector> visit1; + vector> visit2; + vector> parents_ve; + + idx_t num_threads; + // task_queues[workerId] = {curTaskIx, queuedTasks} + // queuedTasks[curTaskIx] = {start, end} + vector, vector>>> task_queues; + int64_t split_size = 256; + + Barrier barrier; + + // lock for next + mutable vector element_locks; + + idx_t mode; +}; + +class PathFindingGlobalState : public GlobalSinkState { +public: + using GlobalCompressedSparseRow = + PhysicalPathFinding::GlobalCompressedSparseRow; + PathFindingGlobalState(ClientContext &context, + const PhysicalPathFinding &op); + + PathFindingGlobalState(PathFindingGlobalState &prev); + + void Sink(DataChunk &input, PathFindingLocalState &lstate); + + // pairs is a 2-column table with src and dst + unique_ptr global_tasks; + unique_ptr global_inputs; + // pairs with path exists + // ColumnDataCollection global_results; + ColumnDataScanState scan_state; + ColumnDataAppendState append_state; + + shared_ptr global_csr; + // state for BFS + unique_ptr global_bfs_state; + size_t child; + string mode; }; } // namespace core From 5b81cb507f62ee561a79a1a18ba1ea20638ec900 Mon Sep 17 00:00:00 2001 From: dtenwolde Date: Tue, 20 Aug 2024 13:32:18 +0200 Subject: [PATCH 192/249] Remove override --- src/core/operator/task/shortest_path_task.cpp | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/src/core/operator/task/shortest_path_task.cpp b/src/core/operator/task/shortest_path_task.cpp index 9656872a..4e0b3c6e 100644 --- a/src/core/operator/task/shortest_path_task.cpp +++ b/src/core/operator/task/shortest_path_task.cpp @@ -9,7 +9,7 @@ PhysicalShortestPathTask::PhysicalShortestPathTask(shared_ptr event_p, Cl : ExecutorTask(context, std::move(event_p)), context(context), state(state), worker_id(worker_id) {} - TaskExecutionResult PhysicalShortestPathTask::ExecuteTask(TaskExecutionMode mode) override { + TaskExecutionResult PhysicalShortestPathTask::ExecuteTask(TaskExecutionMode mode) { auto &bfs_state = state.global_bfs_state; auto &change = bfs_state->change; auto &barrier = bfs_state->barrier; From 5836a80f8640bf1edfb6804c2d56abaeeb932812 Mon Sep 17 00:00:00 2001 From: dtenwolde Date: Tue, 20 Aug 2024 13:36:09 +0200 Subject: [PATCH 193/249] Remove second constructor for globalbfsstate --- src/core/operator/physical_path_finding_operator.cpp | 12 ++++-------- .../core/operator/physical_path_finding_operator.hpp | 5 ----- 2 files changed, 4 insertions(+), 13 deletions(-) diff --git a/src/core/operator/physical_path_finding_operator.cpp b/src/core/operator/physical_path_finding_operator.cpp index 9495e832..1f007a42 100644 --- a/src/core/operator/physical_path_finding_operator.cpp +++ b/src/core/operator/physical_path_finding_operator.cpp @@ -13,6 +13,7 @@ #include #include #include +#include #include "duckpgq/core/utils/duckpgq_barrier.hpp" @@ -73,7 +74,7 @@ void PhysicalPathFinding::GlobalCompressedSparseRow::InitializeEdge( GlobalBFSState::GlobalBFSState(shared_ptr csr_, shared_ptr pairs_, int64_t v_size_, idx_t num_threads_, idx_t mode_, ClientContext &context_) - : csr(csr_), pairs(pairs_), iter(1), v_size(v_size_), change(false), + : csr(std::move(csr_)), pairs(pairs_), iter(1), v_size(v_size_), change(false), started_searches(0), total_len(0), context(context_), seen(v_size_), visit1(v_size_), visit2(v_size_), num_threads(num_threads_), task_queues(num_threads_), barrier(num_threads_), @@ -531,15 +532,10 @@ PhysicalPathFinding::Finalize(Pipeline &pipeline, Event &event, auto &ts = TaskScheduler::GetScheduler(context); idx_t num_threads = ts.NumberOfThreads(); - auto &client_config = ClientConfig::GetConfig(context); idx_t mode = this->mode == "iterativelength" ? 0 : 1; - if (mode == 0) { - gstate.global_bfs_state = make_uniq( + gstate.global_bfs_state = make_uniq( csr, all_pairs, csr->v_size - 2, num_threads, mode, context); - } else { - gstate.global_bfs_state = make_uniq( - csr, all_pairs, csr->v_size - 2, num_threads, mode, context, true); - } + Value task_size_value; context.TryGetCurrentSetting("experimental_path_finding_operator_task_size", task_size_value); diff --git a/src/include/duckpgq/core/operator/physical_path_finding_operator.hpp b/src/include/duckpgq/core/operator/physical_path_finding_operator.hpp index 06ba43cb..086bfb20 100644 --- a/src/include/duckpgq/core/operator/physical_path_finding_operator.hpp +++ b/src/include/duckpgq/core/operator/physical_path_finding_operator.hpp @@ -188,11 +188,6 @@ class GlobalBFSState { shared_ptr pairs_, int64_t v_size_, idx_t num_threads_, idx_t mode_, ClientContext &context_); - GlobalBFSState(shared_ptr csr_, - shared_ptr pairs_, int64_t v_size_, - idx_t num_threads_, idx_t mode_, ClientContext &context_, - bool is_path_); - void Clear(); void CreateTasks(); From 0bf5bc78f0aa4f69443f924aa4cf5ba76bf0b74f Mon Sep 17 00:00:00 2001 From: dtenwolde Date: Tue, 20 Aug 2024 13:49:22 +0200 Subject: [PATCH 194/249] Move iterative length event and tasks to separate files --- src/core/operator/event/CMakeLists.txt | 1 + .../operator/event/iterative_length_event.cpp | 35 ++++ .../physical_path_finding_operator.cpp | 167 +----------------- src/core/operator/task/CMakeLists.txt | 1 + .../operator/task/iterative_length_task.cpp | 131 ++++++++++++++ .../operator/event/iterative_length_event.hpp | 24 +++ .../operator/task/iterative_length_task.hpp | 32 ++++ 7 files changed, 226 insertions(+), 165 deletions(-) create mode 100644 src/core/operator/event/iterative_length_event.cpp create mode 100644 src/core/operator/task/iterative_length_task.cpp create mode 100644 src/include/duckpgq/core/operator/event/iterative_length_event.hpp create mode 100644 src/include/duckpgq/core/operator/task/iterative_length_task.hpp diff --git a/src/core/operator/event/CMakeLists.txt b/src/core/operator/event/CMakeLists.txt index 72e4b6af..533e767d 100644 --- a/src/core/operator/event/CMakeLists.txt +++ b/src/core/operator/event/CMakeLists.txt @@ -1,4 +1,5 @@ set(EXTENSION_SOURCES + ${CMAKE_CURRENT_SOURCE_DIR}/iterative_length_event.cpp ${CMAKE_CURRENT_SOURCE_DIR}/shortest_path_event.cpp ${EXTENSION_SOURCES} PARENT_SCOPE diff --git a/src/core/operator/event/iterative_length_event.cpp b/src/core/operator/event/iterative_length_event.cpp new file mode 100644 index 00000000..f68009f8 --- /dev/null +++ b/src/core/operator/event/iterative_length_event.cpp @@ -0,0 +1,35 @@ +#include "duckpgq/core/operator/event/iterative_length_event.hpp" + +#include + +namespace duckpgq { +namespace core { + +ParallelIterativeEvent::ParallelIterativeEvent(PathFindingGlobalState &gstate_p, Pipeline &pipeline_p) + : BasePipelineEvent(pipeline_p), gstate(gstate_p) {} + + +void ParallelIterativeEvent::Schedule() { + auto &bfs_state = gstate.global_bfs_state; + auto &context = pipeline->GetClientContext(); + + vector> bfs_tasks; + for (idx_t tnum = 0; tnum < bfs_state->num_threads; tnum++) { + bfs_tasks.push_back(make_uniq( + shared_from_this(), context, gstate, tnum)); + } + SetTasks(std::move(bfs_tasks)); +} + +void ParallelIterativeEvent::FinishEvent() override { + auto &bfs_state = gstate.global_bfs_state; + + // if remaining pairs, schedule the BFS for the next batch + if (bfs_state->started_searches < gstate.global_tasks->Count()) { + PhysicalPathFinding::ScheduleBFSEvent(*pipeline, *this, gstate); + } +} + +} // namespace core +} // namespace duckpgq + diff --git a/src/core/operator/physical_path_finding_operator.cpp b/src/core/operator/physical_path_finding_operator.cpp index 1f007a42..a8048309 100644 --- a/src/core/operator/physical_path_finding_operator.cpp +++ b/src/core/operator/physical_path_finding_operator.cpp @@ -10,12 +10,12 @@ #include "duckdb/parallel/event.hpp" #include "duckdb/parallel/meta_pipeline.hpp" #include "duckdb/parallel/thread_context.hpp" +#include "duckpgq/core/utils/duckpgq_barrier.hpp" #include #include #include -#include -#include "duckpgq/core/utils/duckpgq_barrier.hpp" +#include namespace duckpgq { @@ -267,141 +267,6 @@ PhysicalPathFinding::Combine(ExecutionContext &context, // Finalize //===--------------------------------------------------------------------===// -class PhysicalIterativeTask : public ExecutorTask { -public: - PhysicalIterativeTask(shared_ptr event_p, ClientContext &context, - PathFindingGlobalState &state, idx_t worker_id) - : ExecutorTask(context, std::move(event_p)), context(context), - state(state), worker_id(worker_id) {} - - TaskExecutionResult ExecuteTask(TaskExecutionMode mode) override { - auto &bfs_state = state.global_bfs_state; - auto &change = bfs_state->change; - auto &barrier = bfs_state->barrier; - - auto bound = bfs_state->BoundaryCalculation(worker_id); - left = bound.first; - right = bound.second; - - do { - bfs_state->InitTask(worker_id); - - IterativeLength(); - - barrier.Wait(); - - if (worker_id == 0) { - ReachDetect(); - } - barrier.Wait(); - } while (change); - - if (worker_id == 0) { - UnReachableSet(); - } - - event->FinishTask(); - return TaskExecutionResult::TASK_FINISHED; - } - -private: - void IterativeLength() { - auto &bfs_state = state.global_bfs_state; - auto &seen = bfs_state->seen; - auto &visit = bfs_state->iter & 1 ? bfs_state->visit1 : bfs_state->visit2; - auto &next = bfs_state->iter & 1 ? bfs_state->visit2 : bfs_state->visit1; - auto &barrier = bfs_state->barrier; - int64_t *v = (int64_t *)state.global_csr->v; - vector &e = state.global_csr->e; - auto &lane_to_num = bfs_state->lane_to_num; - auto &change = bfs_state->change; - - // clear next before each iteration - for (auto i = left; i < right; i++) { - next[i] = 0; - } - - barrier.Wait(); - - while (true) { - auto task = bfs_state->FetchTask(worker_id); - if (task.first == task.second) { - break; - } - auto start = task.first; - auto end = task.second; - - for (auto i = start; i < end; i++) { - if (visit[i].any()) { - for (auto offset = v[i]; offset < v[i + 1]; offset++) { - auto n = e[offset]; - std::lock_guard lock(bfs_state->element_locks[n]); - next[n] |= visit[i]; - } - } - } - } - - change = false; - barrier.Wait(); - - for (auto i = left; i < right; i++) { - if (next[i].any()) { - next[i] &= ~seen[i]; - seen[i] |= next[i]; - change |= next[i].any(); - } - } - } - - void ReachDetect() { - auto &bfs_state = state.global_bfs_state; - auto result_data = FlatVector::GetData(bfs_state->result.data[0]); - - // detect lanes that finished - for (int64_t lane = 0; lane < LANE_LIMIT; lane++) { - int64_t search_num = bfs_state->lane_to_num[lane]; - if (search_num >= 0) { // active lane - int64_t dst_pos = bfs_state->vdata_dst.sel->get_index(search_num); - if (bfs_state->seen[bfs_state->dst[dst_pos]][lane]) { - result_data[search_num] = - bfs_state->iter; /* found at iter => iter = path length */ - bfs_state->lane_to_num[lane] = -1; // mark inactive - bfs_state->active--; - } - } - } - if (bfs_state->active == 0) { - bfs_state->change = false; - } - // into the next iteration - bfs_state->iter++; - } - - void UnReachableSet() { - auto &bfs_state = state.global_bfs_state; - auto result_data = FlatVector::GetData(bfs_state->result.data[0]); - auto &result_validity = FlatVector::Validity(bfs_state->result.data[0]); - - for (int64_t lane = 0; lane < LANE_LIMIT; lane++) { - int64_t search_num = bfs_state->lane_to_num[lane]; - if (search_num >= 0) { // active lane - result_validity.SetInvalid(search_num); - result_data[search_num] = (int64_t)-1; /* no path */ - bfs_state->lane_to_num[lane] = -1; // mark inactive - } - } - } - -private: - ClientContext &context; - PathFindingGlobalState &state; - // [left, right) - idx_t left; - idx_t right; - idx_t worker_id; -}; - class PhysicalCSREdgeCreationTask : public ExecutorTask { public: PhysicalCSREdgeCreationTask(shared_ptr event_p, ClientContext &context, @@ -478,35 +343,7 @@ class CSREdgeCreationEvent : public BasePipelineEvent { } }; -class ParallelIterativeEvent : public BasePipelineEvent { -public: - ParallelIterativeEvent(PathFindingGlobalState &gstate_p, Pipeline &pipeline_p) - : BasePipelineEvent(pipeline_p), gstate(gstate_p) {} - - PathFindingGlobalState &gstate; - -public: - void Schedule() override { - auto &bfs_state = gstate.global_bfs_state; - auto &context = pipeline->GetClientContext(); - - vector> bfs_tasks; - for (idx_t tnum = 0; tnum < bfs_state->num_threads; tnum++) { - bfs_tasks.push_back(make_uniq( - shared_from_this(), context, gstate, tnum)); - } - SetTasks(std::move(bfs_tasks)); - } - void FinishEvent() override { - auto &bfs_state = gstate.global_bfs_state; - - // if remaining pairs, schedule the BFS for the next batch - if (bfs_state->started_searches < gstate.global_tasks->Count()) { - PhysicalPathFinding::ScheduleBFSEvent(*pipeline, *this, gstate); - } - } -}; SinkFinalizeType PhysicalPathFinding::Finalize(Pipeline &pipeline, Event &event, diff --git a/src/core/operator/task/CMakeLists.txt b/src/core/operator/task/CMakeLists.txt index f7c19616..54522b97 100644 --- a/src/core/operator/task/CMakeLists.txt +++ b/src/core/operator/task/CMakeLists.txt @@ -1,4 +1,5 @@ set(EXTENSION_SOURCES + ${CMAKE_CURRENT_SOURCE_DIR}/iterative_length_task.cpp ${CMAKE_CURRENT_SOURCE_DIR}/shortest_path_task.cpp ${EXTENSION_SOURCES} PARENT_SCOPE diff --git a/src/core/operator/task/iterative_length_task.cpp b/src/core/operator/task/iterative_length_task.cpp new file mode 100644 index 00000000..dd6aec64 --- /dev/null +++ b/src/core/operator/task/iterative_length_task.cpp @@ -0,0 +1,131 @@ +#include "duckpgq/core/operator/task/iterative_length_task.hpp" +#include + +namespace duckpgq { +namespace core { + +PhysicalIterativeTask::PhysicalIterativeTask(shared_ptr event_p, ClientContext &context, + PathFindingGlobalState &state, idx_t worker_id) + : ExecutorTask(context, std::move(event_p)), context(context), + state(state), worker_id(worker_id) {} + + TaskExecutionResult PhysicalIterativeTask::ExecuteTask(TaskExecutionMode mode) override { + auto &bfs_state = state.global_bfs_state; + auto &change = bfs_state->change; + auto &barrier = bfs_state->barrier; + + auto bound = bfs_state->BoundaryCalculation(worker_id); + left = bound.first; + right = bound.second; + + do { + bfs_state->InitTask(worker_id); + + IterativeLength(); + + barrier.Wait(); + + if (worker_id == 0) { + ReachDetect(); + } + barrier.Wait(); + } while (change); + + if (worker_id == 0) { + UnReachableSet(); + } + + event->FinishTask(); + return TaskExecutionResult::TASK_FINISHED; + } + + void PhysicalIterativeTask::IterativeLength() const { + auto &bfs_state = state.global_bfs_state; + auto &seen = bfs_state->seen; + auto &visit = bfs_state->iter & 1 ? bfs_state->visit1 : bfs_state->visit2; + auto &next = bfs_state->iter & 1 ? bfs_state->visit2 : bfs_state->visit1; + auto &barrier = bfs_state->barrier; + int64_t *v = (int64_t *)state.global_csr->v; + vector &e = state.global_csr->e; + auto &lane_to_num = bfs_state->lane_to_num; + auto &change = bfs_state->change; + + // clear next before each iteration + for (auto i = left; i < right; i++) { + next[i] = 0; + } + + barrier.Wait(); + + while (true) { + auto task = bfs_state->FetchTask(worker_id); + if (task.first == task.second) { + break; + } + auto start = task.first; + auto end = task.second; + + for (auto i = start; i < end; i++) { + if (visit[i].any()) { + for (auto offset = v[i]; offset < v[i + 1]; offset++) { + auto n = e[offset]; + std::lock_guard lock(bfs_state->element_locks[n]); + next[n] |= visit[i]; + } + } + } + } + + change = false; + barrier.Wait(); + + for (auto i = left; i < right; i++) { + if (next[i].any()) { + next[i] &= ~seen[i]; + seen[i] |= next[i]; + change |= next[i].any(); + } + } + } + + void PhysicalIterativeTask::ReachDetect() const { + auto &bfs_state = state.global_bfs_state; + auto result_data = FlatVector::GetData(bfs_state->result.data[0]); + + // detect lanes that finished + for (int64_t lane = 0; lane < LANE_LIMIT; lane++) { + int64_t search_num = bfs_state->lane_to_num[lane]; + if (search_num >= 0) { // active lane + int64_t dst_pos = bfs_state->vdata_dst.sel->get_index(search_num); + if (bfs_state->seen[bfs_state->dst[dst_pos]][lane]) { + result_data[search_num] = + bfs_state->iter; /* found at iter => iter = path length */ + bfs_state->lane_to_num[lane] = -1; // mark inactive + bfs_state->active--; + } + } + } + if (bfs_state->active == 0) { + bfs_state->change = false; + } + // into the next iteration + bfs_state->iter++; + } + + void PhysicalIterativeTask::UnReachableSet() const { + auto &bfs_state = state.global_bfs_state; + auto result_data = FlatVector::GetData(bfs_state->result.data[0]); + auto &result_validity = FlatVector::Validity(bfs_state->result.data[0]); + + for (int64_t lane = 0; lane < LANE_LIMIT; lane++) { + int64_t search_num = bfs_state->lane_to_num[lane]; + if (search_num >= 0) { // active lane + result_validity.SetInvalid(search_num); + result_data[search_num] = (int64_t)-1; /* no path */ + bfs_state->lane_to_num[lane] = -1; // mark inactive + } + } + } + +} // namespace core +} // namespace duckpgq \ No newline at end of file diff --git a/src/include/duckpgq/core/operator/event/iterative_length_event.hpp b/src/include/duckpgq/core/operator/event/iterative_length_event.hpp new file mode 100644 index 00000000..5e7f7275 --- /dev/null +++ b/src/include/duckpgq/core/operator/event/iterative_length_event.hpp @@ -0,0 +1,24 @@ +#pragma once + +#include "duckpgq/common.hpp" + +#include +#include +#include + +namespace duckpgq { +namespace core { + +class ParallelIterativeEvent : public BasePipelineEvent { +public: + ParallelIterativeEvent(PathFindingGlobalState &gstate_p, Pipeline &pipeline_p); + + PathFindingGlobalState &gstate; + + void Schedule() override; + + void FinishEvent() override; +}; + +} // namespace core +} // namespace duckpgq diff --git a/src/include/duckpgq/core/operator/task/iterative_length_task.hpp b/src/include/duckpgq/core/operator/task/iterative_length_task.hpp new file mode 100644 index 00000000..77871cf0 --- /dev/null +++ b/src/include/duckpgq/core/operator/task/iterative_length_task.hpp @@ -0,0 +1,32 @@ +#pragma once + +#include "duckpgq/common.hpp" + +#include + +namespace duckpgq { +namespace core { + +class PhysicalIterativeTask : public ExecutorTask { +public: + PhysicalIterativeTask(shared_ptr event_p, ClientContext &context, + PathFindingGlobalState &state, idx_t worker_id); + + TaskExecutionResult ExecuteTask(TaskExecutionMode mode) override; +private: + void IterativeLength() const; + void ReachDetect() const; + void UnReachableSet() const; + +private: + ClientContext &context; + PathFindingGlobalState &state; + // [left, right) + idx_t left; + idx_t right; + idx_t worker_id; +}; + + +} // namespace core +} // namespace duckpgq \ No newline at end of file From 5f777493fc9aeb3de59c61f8c3422b0da1ea5113 Mon Sep 17 00:00:00 2001 From: dtenwolde Date: Tue, 20 Aug 2024 14:05:02 +0200 Subject: [PATCH 195/249] Remove overrides --- src/core/operator/event/iterative_length_event.cpp | 2 +- src/core/operator/task/iterative_length_task.cpp | 2 +- 2 files changed, 2 insertions(+), 2 deletions(-) diff --git a/src/core/operator/event/iterative_length_event.cpp b/src/core/operator/event/iterative_length_event.cpp index f68009f8..a67781fc 100644 --- a/src/core/operator/event/iterative_length_event.cpp +++ b/src/core/operator/event/iterative_length_event.cpp @@ -21,7 +21,7 @@ void ParallelIterativeEvent::Schedule() { SetTasks(std::move(bfs_tasks)); } -void ParallelIterativeEvent::FinishEvent() override { +void ParallelIterativeEvent::FinishEvent() { auto &bfs_state = gstate.global_bfs_state; // if remaining pairs, schedule the BFS for the next batch diff --git a/src/core/operator/task/iterative_length_task.cpp b/src/core/operator/task/iterative_length_task.cpp index dd6aec64..fa618a15 100644 --- a/src/core/operator/task/iterative_length_task.cpp +++ b/src/core/operator/task/iterative_length_task.cpp @@ -9,7 +9,7 @@ PhysicalIterativeTask::PhysicalIterativeTask(shared_ptr event_p, ClientCo : ExecutorTask(context, std::move(event_p)), context(context), state(state), worker_id(worker_id) {} - TaskExecutionResult PhysicalIterativeTask::ExecuteTask(TaskExecutionMode mode) override { + TaskExecutionResult PhysicalIterativeTask::ExecuteTask(TaskExecutionMode mode) { auto &bfs_state = state.global_bfs_state; auto &change = bfs_state->change; auto &barrier = bfs_state->barrier; From 338cdbb04f68e8648c9a80122c281f912a3320e9 Mon Sep 17 00:00:00 2001 From: dtenwolde Date: Tue, 20 Aug 2024 14:11:51 +0200 Subject: [PATCH 196/249] Mode csr edge creation task and event to separate files --- src/core/operator/event/CMakeLists.txt | 1 + .../event/csr_edge_creation_event.cpp | 34 ++++++++ .../physical_path_finding_operator.cpp | 79 +------------------ src/core/operator/task/CMakeLists.txt | 1 + .../operator/task/csr_edge_creation_task.cpp | 46 +++++++++++ .../event/csr_edge_creation_event.hpp | 22 ++++++ .../operator/task/csr_edge_creation_task.hpp | 24 ++++++ 7 files changed, 129 insertions(+), 78 deletions(-) create mode 100644 src/core/operator/event/csr_edge_creation_event.cpp create mode 100644 src/core/operator/task/csr_edge_creation_task.cpp create mode 100644 src/include/duckpgq/core/operator/event/csr_edge_creation_event.hpp create mode 100644 src/include/duckpgq/core/operator/task/csr_edge_creation_task.hpp diff --git a/src/core/operator/event/CMakeLists.txt b/src/core/operator/event/CMakeLists.txt index 533e767d..e9059699 100644 --- a/src/core/operator/event/CMakeLists.txt +++ b/src/core/operator/event/CMakeLists.txt @@ -1,4 +1,5 @@ set(EXTENSION_SOURCES + ${CMAKE_CURRENT_SOURCE_DIR}/csr_edge_creation_event.cpp ${CMAKE_CURRENT_SOURCE_DIR}/iterative_length_event.cpp ${CMAKE_CURRENT_SOURCE_DIR}/shortest_path_event.cpp ${EXTENSION_SOURCES} diff --git a/src/core/operator/event/csr_edge_creation_event.cpp b/src/core/operator/event/csr_edge_creation_event.cpp new file mode 100644 index 00000000..4ea2180e --- /dev/null +++ b/src/core/operator/event/csr_edge_creation_event.cpp @@ -0,0 +1,34 @@ +#include "duckpgq/core/operator/event/csr_edge_creation_event.hpp" +#include + +namespace duckpgq { +namespace core { + +CSREdgeCreationEvent::CSREdgeCreationEvent(PathFindingGlobalState &gstate_p, Pipeline &pipeline_p) + : BasePipelineEvent(pipeline_p), gstate(gstate_p) {} + +void CSREdgeCreationEvent::Schedule() { + auto &context = pipeline->GetClientContext(); + auto &ts = TaskScheduler::GetScheduler(context); + idx_t num_threads = ts.NumberOfThreads(); + auto &scan_state = gstate.scan_state; + auto &global_inputs = gstate.global_inputs; + + global_inputs->InitializeScan(scan_state); + + vector> tasks; + for (idx_t tnum = 0; tnum < num_threads; tnum++) { + tasks.push_back(make_uniq(shared_from_this(), + context, gstate)); + } + SetTasks(std::move(tasks)); +} + +void CSREdgeCreationEvent::FinishEvent() { + auto &gstate = this->gstate; + auto &global_csr = gstate.global_csr; + global_csr->is_ready = true; +} + +} // namespace core +} // namespace duckpgq diff --git a/src/core/operator/physical_path_finding_operator.cpp b/src/core/operator/physical_path_finding_operator.cpp index a8048309..0bfba6a9 100644 --- a/src/core/operator/physical_path_finding_operator.cpp +++ b/src/core/operator/physical_path_finding_operator.cpp @@ -12,6 +12,7 @@ #include "duckdb/parallel/thread_context.hpp" #include "duckpgq/core/utils/duckpgq_barrier.hpp" #include +#include #include #include @@ -267,84 +268,6 @@ PhysicalPathFinding::Combine(ExecutionContext &context, // Finalize //===--------------------------------------------------------------------===// -class PhysicalCSREdgeCreationTask : public ExecutorTask { -public: - PhysicalCSREdgeCreationTask(shared_ptr event_p, ClientContext &context, - PathFindingGlobalState &state) - : ExecutorTask(context, std::move(event_p)), context(context), - state(state) {} - - TaskExecutionResult ExecuteTask(TaskExecutionMode mode) override { - auto &global_inputs = state.global_inputs; - auto &global_csr = state.global_csr; - auto &scan_state = state.scan_state; - - DataChunk input; - global_inputs->InitializeScanChunk(input); - auto result = Vector(LogicalTypeId::BIGINT); - while (true) { - { - lock_guard lock(global_csr->csr_lock); - if (!global_inputs->Scan(scan_state, input)) { - break; - } - } - if (!global_csr->initialized_e) { - const auto e_size = input.data[7].GetValue(0).GetValue(); - global_csr->InitializeEdge(e_size); - } - TernaryExecutor::Execute( - input.data[6], input.data[4], input.data[2], result, input.size(), - [&](int64_t src, int64_t dst, int64_t edge_id) { - const auto pos = ++global_csr->v[src + 1]; - global_csr->e[static_cast(pos) - 1] = dst; - global_csr->edge_ids[static_cast(pos) - 1] = edge_id; - return 1; - }); - } - event->FinishTask(); - global_csr->Print(); - return TaskExecutionResult::TASK_FINISHED; - } - -private: - ClientContext &context; - PathFindingGlobalState &state; -}; - -class CSREdgeCreationEvent : public BasePipelineEvent { -public: - CSREdgeCreationEvent(PathFindingGlobalState &gstate_p, Pipeline &pipeline_p) - : BasePipelineEvent(pipeline_p), gstate(gstate_p) {} - - PathFindingGlobalState &gstate; - - void Schedule() override { - auto &context = pipeline->GetClientContext(); - auto &ts = TaskScheduler::GetScheduler(context); - idx_t num_threads = ts.NumberOfThreads(); - auto &scan_state = gstate.scan_state; - auto &global_inputs = gstate.global_inputs; - - global_inputs->InitializeScan(scan_state); - - vector> tasks; - for (idx_t tnum = 0; tnum < num_threads; tnum++) { - tasks.push_back(make_uniq(shared_from_this(), - context, gstate)); - } - SetTasks(std::move(tasks)); - } - - void FinishEvent() override { - auto &gstate = this->gstate; - auto &global_csr = gstate.global_csr; - global_csr->is_ready = true; - } -}; - - - SinkFinalizeType PhysicalPathFinding::Finalize(Pipeline &pipeline, Event &event, ClientContext &context, diff --git a/src/core/operator/task/CMakeLists.txt b/src/core/operator/task/CMakeLists.txt index 54522b97..637b9b5c 100644 --- a/src/core/operator/task/CMakeLists.txt +++ b/src/core/operator/task/CMakeLists.txt @@ -1,4 +1,5 @@ set(EXTENSION_SOURCES + ${CMAKE_CURRENT_SOURCE_DIR}/csr_edge_creation_task.cpp ${CMAKE_CURRENT_SOURCE_DIR}/iterative_length_task.cpp ${CMAKE_CURRENT_SOURCE_DIR}/shortest_path_task.cpp ${EXTENSION_SOURCES} diff --git a/src/core/operator/task/csr_edge_creation_task.cpp b/src/core/operator/task/csr_edge_creation_task.cpp new file mode 100644 index 00000000..2593491b --- /dev/null +++ b/src/core/operator/task/csr_edge_creation_task.cpp @@ -0,0 +1,46 @@ +#include "duckpgq/core/operator/task/csr_edge_creation_task.hpp" + +namespace duckpgq { +namespace core { + +PhysicalCSREdgeCreationTask::PhysicalCSREdgeCreationTask(shared_ptr event_p, ClientContext &context, + PathFindingGlobalState &state) + : ExecutorTask(context, std::move(event_p)), context(context), + state(state) {} + +TaskExecutionResult PhysicalCSREdgeCreationTask::ExecuteTask(TaskExecutionMode mode) { + auto &global_inputs = state.global_inputs; + auto &global_csr = state.global_csr; + auto &scan_state = state.scan_state; + + DataChunk input; + global_inputs->InitializeScanChunk(input); + auto result = Vector(LogicalTypeId::BIGINT); + while (true) { + { + lock_guard lock(global_csr->csr_lock); + if (!global_inputs->Scan(scan_state, input)) { + break; + } + } + if (!global_csr->initialized_e) { + const auto e_size = input.data[7].GetValue(0).GetValue(); + global_csr->InitializeEdge(e_size); + } + TernaryExecutor::Execute( + input.data[6], input.data[4], input.data[2], result, input.size(), + [&](int64_t src, int64_t dst, int64_t edge_id) { + const auto pos = ++global_csr->v[src + 1]; + global_csr->e[static_cast(pos) - 1] = dst; + global_csr->edge_ids[static_cast(pos) - 1] = edge_id; + return 1; + }); + } + event->FinishTask(); + global_csr->Print(); + return TaskExecutionResult::TASK_FINISHED; +} + + +} // namespace core +} // namespace duckpgq diff --git a/src/include/duckpgq/core/operator/event/csr_edge_creation_event.hpp b/src/include/duckpgq/core/operator/event/csr_edge_creation_event.hpp new file mode 100644 index 00000000..5bebaa0f --- /dev/null +++ b/src/include/duckpgq/core/operator/event/csr_edge_creation_event.hpp @@ -0,0 +1,22 @@ +#pragma once + +#include "duckpgq/common.hpp" + +#include +#include + +namespace duckpgq { +namespace core { +class CSREdgeCreationEvent : public BasePipelineEvent { +public: + CSREdgeCreationEvent(PathFindingGlobalState &gstate_p, Pipeline &pipeline_p); + PathFindingGlobalState &gstate; + + void Schedule() override; + void FinishEvent() override; +}; + + + +} // namespace core +} // namespace duckpgq \ No newline at end of file diff --git a/src/include/duckpgq/core/operator/task/csr_edge_creation_task.hpp b/src/include/duckpgq/core/operator/task/csr_edge_creation_task.hpp new file mode 100644 index 00000000..b0d96ba8 --- /dev/null +++ b/src/include/duckpgq/core/operator/task/csr_edge_creation_task.hpp @@ -0,0 +1,24 @@ +#pragma once + +#include "duckpgq/common.hpp" + +#include + +namespace duckpgq { +namespace core { + +class PhysicalCSREdgeCreationTask : public ExecutorTask { +public: + PhysicalCSREdgeCreationTask(shared_ptr event_p, ClientContext &context, + PathFindingGlobalState &state); + + TaskExecutionResult ExecuteTask(TaskExecutionMode mode) override; + +private: + ClientContext &context; + PathFindingGlobalState &state; +}; + + +} // namespace core +} // namespace duckpgq \ No newline at end of file From 03e696e61b30925d2e277ff2ff359abe8214246e Mon Sep 17 00:00:00 2001 From: dtenwolde Date: Tue, 20 Aug 2024 14:30:42 +0200 Subject: [PATCH 197/249] Fix incorrect initialization of parents_ve --- src/core/operator/physical_path_finding_operator.cpp | 2 +- .../duckpgq/core/operator/physical_path_finding_operator.hpp | 1 - 2 files changed, 1 insertion(+), 2 deletions(-) diff --git a/src/core/operator/physical_path_finding_operator.cpp b/src/core/operator/physical_path_finding_operator.cpp index 0bfba6a9..22569950 100644 --- a/src/core/operator/physical_path_finding_operator.cpp +++ b/src/core/operator/physical_path_finding_operator.cpp @@ -79,7 +79,7 @@ GlobalBFSState::GlobalBFSState(shared_ptr csr_, started_searches(0), total_len(0), context(context_), seen(v_size_), visit1(v_size_), visit2(v_size_), num_threads(num_threads_), task_queues(num_threads_), barrier(num_threads_), - element_locks(v_size_), mode(mode_) { + element_locks(v_size_), mode(mode_), parents_ve(v_size_) { result.Initialize( context, {LogicalType::BIGINT, LogicalType::LIST(LogicalType::BIGINT)}, pairs_->size()); diff --git a/src/include/duckpgq/core/operator/physical_path_finding_operator.hpp b/src/include/duckpgq/core/operator/physical_path_finding_operator.hpp index 086bfb20..43163d63 100644 --- a/src/include/duckpgq/core/operator/physical_path_finding_operator.hpp +++ b/src/include/duckpgq/core/operator/physical_path_finding_operator.hpp @@ -198,7 +198,6 @@ class GlobalBFSState { pair BoundaryCalculation(idx_t worker_id) const; -public: shared_ptr csr; shared_ptr pairs; int64_t iter; From c69b1573f31c59612a7cee6c0366256cbc158e13 Mon Sep 17 00:00:00 2001 From: dtenwolde Date: Wed, 21 Aug 2024 13:17:13 +0200 Subject: [PATCH 198/249] Adding a new test --- .../path_finding/parallel_path_finding.test | 30 +++++++++++++++++++ 1 file changed, 30 insertions(+) diff --git a/test/sql/path_finding/parallel_path_finding.test b/test/sql/path_finding/parallel_path_finding.test index a1002ce9..d0043d4e 100644 --- a/test/sql/path_finding/parallel_path_finding.test +++ b/test/sql/path_finding/parallel_path_finding.test @@ -162,3 +162,33 @@ WHERE p.src BETWEEN (SELECT CSR_OPERATOR( ON t.a_rowid = a.rowid) AND p.dst; ---- 10 2 NULL + +statement ok +CREATE OR REPLACE TABLE Student(id BIGINT, name VARCHAR); INSERT INTO Student VALUES (0, 'Daniel'), (1, 'Tavneet'), (2, 'Gabor'), (3, 'Peter'), (4, 'David'); + +statement ok +CREATE OR REPLACE TABLE know(src BIGINT, dst BIGINT, createDate BIGINT); INSERT INTO know VALUES (0,1, 10), (0,2, 11), (0,3, 12), (3,0, 13), (1,2, 14), (1,3, 15), (2,3, 16), (4,3, 17); + +query III +WITH pairs as ( + SELECT src, dst + FROM (SELECT a.rowid AS src FROM Student a), + (SELECT b.rowid AS dst FROM Student b) +) +SELECT src, dst, shortestpathoperator(src, dst, 'pairs') as path +FROM pairs p +WHERE p.src BETWEEN (SELECT CSR_OPERATOR( + (SELECT count(a.id) as v_size FROM Student a), + (SELECT count(k.src) as e_size from know k), + a.rowid, + c.rowid, + k.rowid, + t.cnt) FROM Know k + JOIN student a on a.id = k.src + JOIN student c on c.id = k.dst + JOIN (SELECT count(k.src) cnt, a.rowid as a_rowid + FROM student a + LEFT JOIN know k ON k.src = a.id + GROUP BY a.rowid) t + ON t.a_rowid = a.rowid) AND p.dst; +---- From b3dd0dc5913815f83db64932f0e7c2a49bed4b88 Mon Sep 17 00:00:00 2001 From: dtenwolde Date: Tue, 27 Aug 2024 11:54:21 +0200 Subject: [PATCH 199/249] Initialize boundaries --- src/core/operator/task/shortest_path_task.cpp | 4 +++- 1 file changed, 3 insertions(+), 1 deletion(-) diff --git a/src/core/operator/task/shortest_path_task.cpp b/src/core/operator/task/shortest_path_task.cpp index 4e0b3c6e..ae4c1b96 100644 --- a/src/core/operator/task/shortest_path_task.cpp +++ b/src/core/operator/task/shortest_path_task.cpp @@ -7,7 +7,9 @@ namespace core { PhysicalShortestPathTask::PhysicalShortestPathTask(shared_ptr event_p, ClientContext &context, PathFindingGlobalState &state, idx_t worker_id) : ExecutorTask(context, std::move(event_p)), context(context), - state(state), worker_id(worker_id) {} + state(state), worker_id(worker_id) { + left = right = UINT64_MAX; // NOLINT +} TaskExecutionResult PhysicalShortestPathTask::ExecuteTask(TaskExecutionMode mode) { auto &bfs_state = state.global_bfs_state; From b786377b3156d82e147d7ceedf80aea0cd12f844 Mon Sep 17 00:00:00 2001 From: dtenwolde Date: Tue, 27 Aug 2024 11:54:41 +0200 Subject: [PATCH 200/249] Rework task creation, instead have a single global queue of tasks --- .../physical_path_finding_operator.cpp | 41 ++++++++----------- .../physical_path_finding_operator.hpp | 7 ++-- 2 files changed, 19 insertions(+), 29 deletions(-) diff --git a/src/core/operator/physical_path_finding_operator.cpp b/src/core/operator/physical_path_finding_operator.cpp index 22569950..ce0b2319 100644 --- a/src/core/operator/physical_path_finding_operator.cpp +++ b/src/core/operator/physical_path_finding_operator.cpp @@ -78,8 +78,8 @@ GlobalBFSState::GlobalBFSState(shared_ptr csr_, : csr(std::move(csr_)), pairs(pairs_), iter(1), v_size(v_size_), change(false), started_searches(0), total_len(0), context(context_), seen(v_size_), visit1(v_size_), visit2(v_size_), num_threads(num_threads_), - task_queues(num_threads_), barrier(num_threads_), - element_locks(v_size_), mode(mode_), parents_ve(v_size_) { + barrier(num_threads_), element_locks(v_size_), + mode(mode_), parents_ve(v_size_) { result.Initialize( context, {LogicalType::BIGINT, LogicalType::LIST(LogicalType::BIGINT)}, pairs_->size()); @@ -110,38 +110,29 @@ void GlobalBFSState::Clear() { void GlobalBFSState::CreateTasks() { // workerTasks[workerId] = [task1, task2, ...] - auto queues = &task_queues; - vector>> worker_tasks(num_threads); - auto cur_worker = 0; + // vector>> worker_tasks(num_threads); + // auto cur_worker = 0; int64_t *v = (int64_t *)csr->v; int64_t current_task_edges = 0; idx_t current_task_start = 0; - for (idx_t i = 0; i < (idx_t)v_size; i++) { - auto vertex_edges = v[i + 1] - v[i]; - if (current_task_edges + vertex_edges > split_size && - i != current_task_start) { - auto worker_id = cur_worker % num_threads; - pair range = {current_task_start, i}; - worker_tasks[worker_id].push_back(range); - current_task_start = i; - current_task_edges = 0; - cur_worker++; + for (idx_t v_idx = 0; v_idx < (idx_t)v_size; v_idx++) { + auto number_of_edges = v[v_idx + 1] - v[v_idx]; + if (current_task_edges + number_of_edges > split_size) { + global_task_queue.push_back({current_task_start, v_idx}); + current_task_start = v_idx; + current_task_edges = 0; // reset } - current_task_edges += vertex_edges; - } - if (current_task_start < (idx_t)v_size) { - auto worker_id = cur_worker % num_threads; - pair range = {current_task_start, v_size}; - worker_tasks[worker_id].push_back(range); + current_task_edges += number_of_edges; } - for (idx_t worker_id = 0; worker_id < num_threads; worker_id++) { - queues->at(worker_id).first.store(0); - queues->at(worker_id).second = worker_tasks[worker_id]; + + // Final task if there are any remaining edges + if (current_task_start < (idx_t)v_size) { + global_task_queue.push_back({current_task_start, v_size}); } } -void GlobalBFSState::InitTask(idx_t worker_id) { task_queues[worker_id].first.store(0); } +// void GlobalBFSState::InitTask(idx_t worker_id) { task_queues[worker_id].first.store(0); } pair GlobalBFSState::FetchTask(idx_t worker_id) { auto &task_queue = task_queues; diff --git a/src/include/duckpgq/core/operator/physical_path_finding_operator.hpp b/src/include/duckpgq/core/operator/physical_path_finding_operator.hpp index 43163d63..64b79075 100644 --- a/src/include/duckpgq/core/operator/physical_path_finding_operator.hpp +++ b/src/include/duckpgq/core/operator/physical_path_finding_operator.hpp @@ -192,8 +192,6 @@ class GlobalBFSState { void CreateTasks(); - void InitTask(idx_t worker_id); - pair FetchTask(idx_t worker_id); pair BoundaryCalculation(idx_t worker_id) const; @@ -219,9 +217,10 @@ class GlobalBFSState { vector> parents_ve; idx_t num_threads; - // task_queues[workerId] = {curTaskIx, queuedTasks} + // task_queues[workerId] = {curTaskIdx, queuedTasks} // queuedTasks[curTaskIx] = {start, end} - vector, vector>>> task_queues; + vector> global_task_queue; + // vector, vector>>> task_queues; int64_t split_size = 256; Barrier barrier; From 0f155e20af3955abe8a4cc3b8ffce5db9f89a05d Mon Sep 17 00:00:00 2001 From: dtenwolde Date: Tue, 27 Aug 2024 12:02:42 +0200 Subject: [PATCH 201/249] Fetch task now fetches from global queue --- .../physical_path_finding_operator.cpp | 31 ++++++++++--------- src/core/operator/task/shortest_path_task.cpp | 2 +- .../physical_path_finding_operator.hpp | 5 +-- 3 files changed, 21 insertions(+), 17 deletions(-) diff --git a/src/core/operator/physical_path_finding_operator.cpp b/src/core/operator/physical_path_finding_operator.cpp index ce0b2319..47bbf248 100644 --- a/src/core/operator/physical_path_finding_operator.cpp +++ b/src/core/operator/physical_path_finding_operator.cpp @@ -132,20 +132,23 @@ void GlobalBFSState::CreateTasks() { } } -// void GlobalBFSState::InitTask(idx_t worker_id) { task_queues[worker_id].first.store(0); } - -pair GlobalBFSState::FetchTask(idx_t worker_id) { - auto &task_queue = task_queues; - idx_t offset = 0; - do { - auto worker_idx = (worker_id + offset) % task_queue.size(); - auto cur_task_ix = task_queue[worker_idx].first.fetch_add(1); - if (cur_task_ix < task_queue[worker_idx].second.size()) { - return task_queue[worker_idx].second[cur_task_ix]; - } - offset++; - } while (offset < task_queue.size()); - return {0, 0}; + +optional_ptr> GlobalBFSState::FetchTask() { + std::unique_lock lock(queue_mutex); // Lock the mutex to access the queue + + // Wait until the queue is not empty or some other condition to continue + queue_cv.wait(lock, [this]() { return !global_task_queue.empty(); }); + + // If the queue is empty, return an empty optional (though this shouldn't happen due to wait condition) + if (global_task_queue.empty()) { + return nullptr; + } + + // Fetch the task from the front of the queue + auto task = global_task_queue.back(); + global_task_queue.pop_back(); + + return task; } pair GlobalBFSState::BoundaryCalculation(idx_t worker_id) const { diff --git a/src/core/operator/task/shortest_path_task.cpp b/src/core/operator/task/shortest_path_task.cpp index ae4c1b96..64e7b9c1 100644 --- a/src/core/operator/task/shortest_path_task.cpp +++ b/src/core/operator/task/shortest_path_task.cpp @@ -21,7 +21,7 @@ PhysicalShortestPathTask::PhysicalShortestPathTask(shared_ptr event_p, Cl right = bound.second; do { - bfs_state->InitTask(worker_id); + // bfs_state->InitTask(worker_id); IterativePath(); barrier.Wait(); diff --git a/src/include/duckpgq/core/operator/physical_path_finding_operator.hpp b/src/include/duckpgq/core/operator/physical_path_finding_operator.hpp index 64b79075..eb34117d 100644 --- a/src/include/duckpgq/core/operator/physical_path_finding_operator.hpp +++ b/src/include/duckpgq/core/operator/physical_path_finding_operator.hpp @@ -191,8 +191,7 @@ class GlobalBFSState { void Clear(); void CreateTasks(); - - pair FetchTask(idx_t worker_id); + optional_ptr> FetchTask(); // Function to fetch a task pair BoundaryCalculation(idx_t worker_id) const; @@ -220,6 +219,8 @@ class GlobalBFSState { // task_queues[workerId] = {curTaskIdx, queuedTasks} // queuedTasks[curTaskIx] = {start, end} vector> global_task_queue; + std::mutex queue_mutex; // Mutex for synchronizing access + std::condition_variable queue_cv; // Condition variable for task availability // vector, vector>>> task_queues; int64_t split_size = 256; From 5020657b4c3ba539a6c2a0fdc878aa626a6c830e Mon Sep 17 00:00:00 2001 From: dtenwolde Date: Tue, 27 Aug 2024 12:40:03 +0200 Subject: [PATCH 202/249] Use index to keep track of task queue --- src/core/operator/physical_path_finding_operator.cpp | 10 +++++----- .../core/operator/physical_path_finding_operator.hpp | 2 +- 2 files changed, 6 insertions(+), 6 deletions(-) diff --git a/src/core/operator/physical_path_finding_operator.cpp b/src/core/operator/physical_path_finding_operator.cpp index 47bbf248..4c53c42f 100644 --- a/src/core/operator/physical_path_finding_operator.cpp +++ b/src/core/operator/physical_path_finding_operator.cpp @@ -139,14 +139,14 @@ optional_ptr> GlobalBFSState::FetchTask() { // Wait until the queue is not empty or some other condition to continue queue_cv.wait(lock, [this]() { return !global_task_queue.empty(); }); - // If the queue is empty, return an empty optional (though this shouldn't happen due to wait condition) - if (global_task_queue.empty()) { + // If all tasks are processed, return an empty optional + if (current_task_index >= global_task_queue.size()) { return nullptr; } - // Fetch the task from the front of the queue - auto task = global_task_queue.back(); - global_task_queue.pop_back(); + // Fetch the task using the current index + auto task = global_task_queue[current_task_index]; + current_task_index++; // Move to the next task return task; } diff --git a/src/include/duckpgq/core/operator/physical_path_finding_operator.hpp b/src/include/duckpgq/core/operator/physical_path_finding_operator.hpp index eb34117d..3f548e91 100644 --- a/src/include/duckpgq/core/operator/physical_path_finding_operator.hpp +++ b/src/include/duckpgq/core/operator/physical_path_finding_operator.hpp @@ -221,7 +221,7 @@ class GlobalBFSState { vector> global_task_queue; std::mutex queue_mutex; // Mutex for synchronizing access std::condition_variable queue_cv; // Condition variable for task availability - // vector, vector>>> task_queues; + size_t current_task_index = 0; // Index to track the current task int64_t split_size = 256; Barrier barrier; From 12a2ff8f958e7e232086655fe2d6f03d71b9d1c0 Mon Sep 17 00:00:00 2001 From: dtenwolde Date: Tue, 27 Aug 2024 12:40:14 +0200 Subject: [PATCH 203/249] Set the task range in a method --- src/core/operator/task/shortest_path_task.cpp | 33 +++++++++++-------- .../core/operator/task/shortest_path_task.hpp | 2 ++ 2 files changed, 21 insertions(+), 14 deletions(-) diff --git a/src/core/operator/task/shortest_path_task.cpp b/src/core/operator/task/shortest_path_task.cpp index 64e7b9c1..a5f681cd 100644 --- a/src/core/operator/task/shortest_path_task.cpp +++ b/src/core/operator/task/shortest_path_task.cpp @@ -16,13 +16,7 @@ PhysicalShortestPathTask::PhysicalShortestPathTask(shared_ptr event_p, Cl auto &change = bfs_state->change; auto &barrier = bfs_state->barrier; - auto bound = bfs_state->BoundaryCalculation(worker_id); - left = bound.first; - right = bound.second; - do { - // bfs_state->InitTask(worker_id); - IterativePath(); barrier.Wait(); @@ -41,6 +35,16 @@ PhysicalShortestPathTask::PhysicalShortestPathTask(shared_ptr event_p, Cl return TaskExecutionResult::TASK_FINISHED; } + void PhysicalShortestPathTask::SetTaskRange() { + auto task = state.global_bfs_state->FetchTask(); + if (task == nullptr) { + return; + } + left = task->first; + right = task->second; +} + + void PhysicalShortestPathTask::IterativePath() { auto &bfs_state = state.global_bfs_state; @@ -54,6 +58,8 @@ PhysicalShortestPathTask::PhysicalShortestPathTask(shared_ptr event_p, Cl auto &parents_ve = bfs_state->parents_ve; auto &change = bfs_state->change; + + // clear next before each iteration for (auto i = left; i < right; i++) { next[i] = 0; @@ -62,14 +68,7 @@ PhysicalShortestPathTask::PhysicalShortestPathTask(shared_ptr event_p, Cl barrier.Wait(); while (true) { - auto task = bfs_state->FetchTask(worker_id); - if (task.first == task.second) { - break; - } - auto start = task.first; - auto end = task.second; - - for (auto i = start; i < end; i++) { + for (auto i = left; i < right; i++) { if (visit[i].any()) { for (auto offset = v[i]; offset < v[i + 1]; offset++) { auto n = e[offset]; @@ -86,6 +85,12 @@ PhysicalShortestPathTask::PhysicalShortestPathTask(shared_ptr event_p, Cl } } } + auto task = bfs_state->FetchTask(); + if (!task) { + break; + } + left = task->first; + right = task->second; } barrier.Wait(); diff --git a/src/include/duckpgq/core/operator/task/shortest_path_task.hpp b/src/include/duckpgq/core/operator/task/shortest_path_task.hpp index 99c43e35..47855632 100644 --- a/src/include/duckpgq/core/operator/task/shortest_path_task.hpp +++ b/src/include/duckpgq/core/operator/task/shortest_path_task.hpp @@ -20,6 +20,8 @@ class PhysicalShortestPathTask : public ExecutorTask { void PathConstruction(); + void SetTaskRange(); + ClientContext &context; PathFindingGlobalState &state; // [left, right) From 42ebe4e637af344c37adb7d8a03b2abd3cd5745d Mon Sep 17 00:00:00 2001 From: dtenwolde Date: Wed, 28 Aug 2024 11:08:51 +0200 Subject: [PATCH 204/249] Add possibility to reset counter inside barrier --- src/core/utils/duckpgq_barrier.cpp | 13 +++++++++++-- src/include/duckpgq/core/utils/duckpgq_barrier.hpp | 2 +- 2 files changed, 12 insertions(+), 3 deletions(-) diff --git a/src/core/utils/duckpgq_barrier.cpp b/src/core/utils/duckpgq_barrier.cpp index 882bb5f6..e4fd708a 100644 --- a/src/core/utils/duckpgq_barrier.cpp +++ b/src/core/utils/duckpgq_barrier.cpp @@ -9,14 +9,23 @@ namespace core { Barrier::Barrier(std::size_t iCount) : mThreshold(iCount), mCount(iCount), mGeneration(0) {} -void Barrier::Wait() { +void Barrier::Wait(std::function resetAction) { std::unique_lock lLock{mMutex}; auto lGen = mGeneration.load(); + if (!--mCount) { + // Last thread to reach the barrier mGeneration++; mCount = mThreshold; - mCond.notify_all(); + + // If a reset action is provided, execute it + if (resetAction) { + resetAction(); // Perform the reset action + } + + mCond.notify_all(); // Wake up all waiting threads } else { + // Other threads wait for the generation to change mCond.wait(lLock, [this, lGen] { return lGen != mGeneration; }); } } diff --git a/src/include/duckpgq/core/utils/duckpgq_barrier.hpp b/src/include/duckpgq/core/utils/duckpgq_barrier.hpp index 75eb24bb..331aa445 100644 --- a/src/include/duckpgq/core/utils/duckpgq_barrier.hpp +++ b/src/include/duckpgq/core/utils/duckpgq_barrier.hpp @@ -18,7 +18,7 @@ class Barrier { public: explicit Barrier(std::size_t iCount); - void Wait(); + void Wait(std::function resetAction = nullptr); private: std::mutex mMutex; From 75f94e0a38745bbd2d45422900981c3936f55ac1 Mon Sep 17 00:00:00 2001 From: dtenwolde Date: Wed, 28 Aug 2024 11:10:25 +0200 Subject: [PATCH 205/249] Set the number of threads to schedule to min of tasks and number of tasks --- src/core/operator/event/shortest_path_event.cpp | 3 ++- src/core/operator/physical_path_finding_operator.cpp | 4 +++- 2 files changed, 5 insertions(+), 2 deletions(-) diff --git a/src/core/operator/event/shortest_path_event.cpp b/src/core/operator/event/shortest_path_event.cpp index 3d8d96df..75212e4f 100644 --- a/src/core/operator/event/shortest_path_event.cpp +++ b/src/core/operator/event/shortest_path_event.cpp @@ -17,7 +17,8 @@ void ParallelShortestPathEvent::Schedule() { auto &context = pipeline->GetClientContext(); vector> bfs_tasks; - for (idx_t tnum = 0; tnum < bfs_state->num_threads; tnum++) { + size_t threads_to_schedule = std::min(bfs_state->num_threads, (idx_t)bfs_state->global_task_queue.size()); + for (idx_t tnum = 0; tnum < threads_to_schedule; tnum++) { bfs_tasks.push_back(make_uniq( shared_from_this(), context, gstate, tnum)); } diff --git a/src/core/operator/physical_path_finding_operator.cpp b/src/core/operator/physical_path_finding_operator.cpp index 4c53c42f..714d239b 100644 --- a/src/core/operator/physical_path_finding_operator.cpp +++ b/src/core/operator/physical_path_finding_operator.cpp @@ -78,7 +78,7 @@ GlobalBFSState::GlobalBFSState(shared_ptr csr_, : csr(std::move(csr_)), pairs(pairs_), iter(1), v_size(v_size_), change(false), started_searches(0), total_len(0), context(context_), seen(v_size_), visit1(v_size_), visit2(v_size_), num_threads(num_threads_), - barrier(num_threads_), element_locks(v_size_), + element_locks(v_size_), mode(mode_), parents_ve(v_size_) { result.Initialize( context, {LogicalType::BIGINT, LogicalType::LIST(LogicalType::BIGINT)}, @@ -91,6 +91,8 @@ GlobalBFSState::GlobalBFSState(shared_ptr csr_, dst = FlatVector::GetData(dst_data); CreateTasks(); + size_t number_of_threads_to_schedule = std::min(num_threads, (idx_t)global_task_queue.size()); + barrier = make_uniq(number_of_threads_to_schedule); } void GlobalBFSState::Clear() { From f5e2479c7622adce95b43792e6179a4f6d5e2955 Mon Sep 17 00:00:00 2001 From: dtenwolde Date: Wed, 28 Aug 2024 11:10:50 +0200 Subject: [PATCH 206/249] Make barrier unique ptr --- .../duckpgq/core/operator/physical_path_finding_operator.hpp | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/src/include/duckpgq/core/operator/physical_path_finding_operator.hpp b/src/include/duckpgq/core/operator/physical_path_finding_operator.hpp index 3f548e91..54dde67e 100644 --- a/src/include/duckpgq/core/operator/physical_path_finding_operator.hpp +++ b/src/include/duckpgq/core/operator/physical_path_finding_operator.hpp @@ -224,7 +224,7 @@ class GlobalBFSState { size_t current_task_index = 0; // Index to track the current task int64_t split_size = 256; - Barrier barrier; + unique_ptr barrier; // lock for next mutable vector element_locks; From 71d95be1fa8e69b3f983d5585339df0f0a0321cd Mon Sep 17 00:00:00 2001 From: dtenwolde Date: Wed, 28 Aug 2024 11:13:35 +0200 Subject: [PATCH 207/249] Fix barrier being a pointer now --- src/core/operator/task/iterative_length_task.cpp | 9 ++++----- 1 file changed, 4 insertions(+), 5 deletions(-) diff --git a/src/core/operator/task/iterative_length_task.cpp b/src/core/operator/task/iterative_length_task.cpp index fa618a15..90e5f0e3 100644 --- a/src/core/operator/task/iterative_length_task.cpp +++ b/src/core/operator/task/iterative_length_task.cpp @@ -19,16 +19,15 @@ PhysicalIterativeTask::PhysicalIterativeTask(shared_ptr event_p, ClientCo right = bound.second; do { - bfs_state->InitTask(worker_id); IterativeLength(); - barrier.Wait(); + barrier->Wait(); if (worker_id == 0) { ReachDetect(); } - barrier.Wait(); + barrier->Wait(); } while (change); if (worker_id == 0) { @@ -55,7 +54,7 @@ PhysicalIterativeTask::PhysicalIterativeTask(shared_ptr event_p, ClientCo next[i] = 0; } - barrier.Wait(); + barrier->Wait(); while (true) { auto task = bfs_state->FetchTask(worker_id); @@ -77,7 +76,7 @@ PhysicalIterativeTask::PhysicalIterativeTask(shared_ptr event_p, ClientCo } change = false; - barrier.Wait(); + barrier->Wait(); for (auto i = left; i < right; i++) { if (next[i].any()) { From 0ef6b20a84796d1a53bf9637031949fa7cca1464 Mon Sep 17 00:00:00 2001 From: dtenwolde Date: Wed, 28 Aug 2024 11:14:08 +0200 Subject: [PATCH 208/249] Fetch task and set range now work properly --- .../physical_path_finding_operator.cpp | 21 +++- src/core/operator/task/shortest_path_task.cpp | 115 +++++++++++------- .../physical_path_finding_operator.hpp | 4 +- .../core/operator/task/shortest_path_task.hpp | 2 +- 4 files changed, 90 insertions(+), 52 deletions(-) diff --git a/src/core/operator/physical_path_finding_operator.cpp b/src/core/operator/physical_path_finding_operator.cpp index 714d239b..23df6a66 100644 --- a/src/core/operator/physical_path_finding_operator.cpp +++ b/src/core/operator/physical_path_finding_operator.cpp @@ -132,14 +132,22 @@ void GlobalBFSState::CreateTasks() { if (current_task_start < (idx_t)v_size) { global_task_queue.push_back({current_task_start, v_size}); } + // for (const auto& task : global_task_queue) { + // std::cout << "Task: " << task.first << " " << task.second << std::endl; // debug + // } } -optional_ptr> GlobalBFSState::FetchTask() { +shared_ptr> GlobalBFSState::FetchTask() { std::unique_lock lock(queue_mutex); // Lock the mutex to access the queue + // Check if there are no more tasks to process + if (current_task_index >= global_task_queue.size()) { + return nullptr; // No more tasks, return immediately + } + // Wait until the queue is not empty or some other condition to continue - queue_cv.wait(lock, [this]() { return !global_task_queue.empty(); }); + queue_cv.wait(lock, [this]() { return current_task_index < global_task_queue.size(); }); // If all tasks are processed, return an empty optional if (current_task_index >= global_task_queue.size()) { @@ -147,12 +155,18 @@ optional_ptr> GlobalBFSState::FetchTask() { } // Fetch the task using the current index - auto task = global_task_queue[current_task_index]; + auto task = make_shared_ptr>(global_task_queue[current_task_index]); current_task_index++; // Move to the next task return task; } +void GlobalBFSState::ResetTaskIndex() { + std::lock_guard lock(queue_mutex); // Lock to reset index safely + current_task_index = 0; // Reset the task index for the next stage + queue_cv.notify_all(); // Notify all threads that tasks are available +} + pair GlobalBFSState::BoundaryCalculation(idx_t worker_id) const { idx_t block_size = ceil((double)v_size / num_threads); block_size = block_size == 0 ? 1 : block_size; @@ -339,6 +353,7 @@ void PhysicalPathFinding::ScheduleBFSEvent(Pipeline &pipeline, Event &event, (uint64_t)0; // path of length 0 does not require a search } else { bfs_state->visit1[bfs_state->src[src_pos]][lane] = true; + // bfs_state->seen[bfs_state->src[src_pos]][lane] = true; bfs_state->lane_to_num[lane] = search_num; // active lane bfs_state->active++; seen_mask[lane] = false; diff --git a/src/core/operator/task/shortest_path_task.cpp b/src/core/operator/task/shortest_path_task.cpp index a5f681cd..0be16202 100644 --- a/src/core/operator/task/shortest_path_task.cpp +++ b/src/core/operator/task/shortest_path_task.cpp @@ -18,13 +18,21 @@ PhysicalShortestPathTask::PhysicalShortestPathTask(shared_ptr event_p, Cl do { IterativePath(); - barrier.Wait(); + barrier->Wait([&]() { + bfs_state->ResetTaskIndex(); // Reset task index safely + }); + + barrier->Wait(); if (worker_id == 0) { ReachDetect(); } - barrier.Wait(); + barrier->Wait([&]() { + bfs_state->ResetTaskIndex(); // Reset task index safely + }); + + barrier->Wait(); } while (change); if (worker_id == 0) { @@ -35,66 +43,71 @@ PhysicalShortestPathTask::PhysicalShortestPathTask(shared_ptr event_p, Cl return TaskExecutionResult::TASK_FINISHED; } - void PhysicalShortestPathTask::SetTaskRange() { + bool PhysicalShortestPathTask::SetTaskRange() { auto task = state.global_bfs_state->FetchTask(); if (task == nullptr) { - return; + return false; } left = task->first; right = task->second; + return true; } - - void PhysicalShortestPathTask::IterativePath() { - auto &bfs_state = state.global_bfs_state; - auto &seen = bfs_state->seen; - auto &visit = bfs_state->iter & 1 ? bfs_state->visit1 : bfs_state->visit2; - auto &next = bfs_state->iter & 1 ? bfs_state->visit2 : bfs_state->visit1; - auto &barrier = bfs_state->barrier; - int64_t *v = (int64_t *)state.global_csr->v; - vector &e = state.global_csr->e; - auto &edge_ids = state.global_csr->edge_ids; - auto &parents_ve = bfs_state->parents_ve; - auto &change = bfs_state->change; - + auto &bfs_state = state.global_bfs_state; + auto &seen = bfs_state->seen; + auto &visit = bfs_state->iter & 1 ? bfs_state->visit1 : bfs_state->visit2; + auto &next = bfs_state->iter & 1 ? bfs_state->visit2 : bfs_state->visit1; + auto &barrier = bfs_state->barrier; + int64_t *v = (int64_t *)state.global_csr->v; + vector &e = state.global_csr->e; + auto &edge_ids = state.global_csr->edge_ids; + auto &parents_ve = bfs_state->parents_ve; + auto &change = bfs_state->change; + + if (!SetTaskRange()) { + return; // no more tasks + } + // clear next before each iteration + for (auto i = left; i < right; i++) { + next[i] = 0; + } + barrier->Wait(); - // clear next before each iteration + while (true) { for (auto i = left; i < right; i++) { - next[i] = 0; - } - - barrier.Wait(); - - while (true) { - for (auto i = left; i < right; i++) { - if (visit[i].any()) { - for (auto offset = v[i]; offset < v[i + 1]; offset++) { - auto n = e[offset]; - auto edge_id = edge_ids[offset]; - { - std::lock_guard lock(bfs_state->element_locks[n]); - next[n] |= visit[i]; - } - for (auto l = 0; l < LANE_LIMIT; l++) { - if (parents_ve[n][l].GetV() == -1 && visit[i][l]) { - parents_ve[n][l] = {static_cast(i), edge_id}; - } + if (visit[i].any()) { + for (auto offset = v[i]; offset < v[i + 1]; offset++) { + auto n = e[offset]; + auto edge_id = edge_ids[offset]; + { + std::lock_guard lock(bfs_state->element_locks[n]); + next[n] |= visit[i]; + } + for (auto l = 0; l < LANE_LIMIT; l++) { + if (parents_ve[n][l].GetV() == -1 && visit[i][l]) { + parents_ve[n][l] = {static_cast(i), edge_id}; } } } } - auto task = bfs_state->FetchTask(); - if (!task) { - break; - } - left = task->first; - right = task->second; } + if (!SetTaskRange()) { + break; // no more tasks + } + } + + barrier->Wait([&]() { + bfs_state->ResetTaskIndex(); // Reset task index safely + }); - barrier.Wait(); + if (!SetTaskRange()) { + return; // no more tasks + } + barrier->Wait(); + while (true) { for (auto i = left; i < right; i++) { if (next[i].any()) { next[i] &= ~seen[i]; @@ -102,15 +115,23 @@ PhysicalShortestPathTask::PhysicalShortestPathTask(shared_ptr event_p, Cl change |= next[i].any(); } } + + if (!SetTaskRange()) { + break; // no more tasks + } } + barrier->Wait([&]() { + bfs_state->ResetTaskIndex(); // Reset task index safely + }); + + barrier->Wait(); +} void PhysicalShortestPathTask::ReachDetect() { auto &bfs_state = state.global_bfs_state; - auto &change = bfs_state->change; // detect lanes that finished for (int64_t lane = 0; lane < LANE_LIMIT; lane++) { int64_t search_num = bfs_state->lane_to_num[lane]; - if (search_num >= 0) { // active lane //! Check if dst for a source has been seen int64_t dst_pos = bfs_state->vdata_dst.sel->get_index(search_num); @@ -120,7 +141,7 @@ PhysicalShortestPathTask::PhysicalShortestPathTask(shared_ptr event_p, Cl } } if (bfs_state->active == 0) { - change = false; + bfs_state->change = false; } // into the next iteration bfs_state->iter++; diff --git a/src/include/duckpgq/core/operator/physical_path_finding_operator.hpp b/src/include/duckpgq/core/operator/physical_path_finding_operator.hpp index 54dde67e..3c84385a 100644 --- a/src/include/duckpgq/core/operator/physical_path_finding_operator.hpp +++ b/src/include/duckpgq/core/operator/physical_path_finding_operator.hpp @@ -191,7 +191,9 @@ class GlobalBFSState { void Clear(); void CreateTasks(); - optional_ptr> FetchTask(); // Function to fetch a task + shared_ptr> FetchTask(); // Function to fetch a task + void ResetTaskIndex(); + pair BoundaryCalculation(idx_t worker_id) const; diff --git a/src/include/duckpgq/core/operator/task/shortest_path_task.hpp b/src/include/duckpgq/core/operator/task/shortest_path_task.hpp index 47855632..f1ff6747 100644 --- a/src/include/duckpgq/core/operator/task/shortest_path_task.hpp +++ b/src/include/duckpgq/core/operator/task/shortest_path_task.hpp @@ -20,7 +20,7 @@ class PhysicalShortestPathTask : public ExecutorTask { void PathConstruction(); - void SetTaskRange(); + bool SetTaskRange(); ClientContext &context; PathFindingGlobalState &state; From 9a74c05038de60789876f1d2999c375b5db23b7c Mon Sep 17 00:00:00 2001 From: dtenwolde Date: Wed, 28 Aug 2024 14:40:42 +0200 Subject: [PATCH 209/249] Add more elaborate test case with search where src == dst. Now also starting search in that case --- .../physical_path_finding_operator.cpp | 3 -- .../path_finding/parallel_path_finding.test | 29 +++++++++++++++++++ 2 files changed, 29 insertions(+), 3 deletions(-) diff --git a/src/core/operator/physical_path_finding_operator.cpp b/src/core/operator/physical_path_finding_operator.cpp index 23df6a66..aa5b2142 100644 --- a/src/core/operator/physical_path_finding_operator.cpp +++ b/src/core/operator/physical_path_finding_operator.cpp @@ -348,9 +348,6 @@ void PhysicalPathFinding::ScheduleBFSEvent(Pipeline &pipeline, Event &event, if (!bfs_state->vdata_src.validity.RowIsValid(src_pos)) { result_validity.SetInvalid(search_num); result_data[search_num] = (uint64_t)-1; /* no path */ - } else if (bfs_state->src[src_pos] == bfs_state->dst[dst_pos]) { - result_data[search_num] = - (uint64_t)0; // path of length 0 does not require a search } else { bfs_state->visit1[bfs_state->src[src_pos]][lane] = true; // bfs_state->seen[bfs_state->src[src_pos]][lane] = true; diff --git a/test/sql/path_finding/parallel_path_finding.test b/test/sql/path_finding/parallel_path_finding.test index d0043d4e..4a09acb6 100644 --- a/test/sql/path_finding/parallel_path_finding.test +++ b/test/sql/path_finding/parallel_path_finding.test @@ -48,6 +48,35 @@ WHERE p.src between (SELECT CSR_OPERATOR( 1 2 [1, 3, 0, 1, 2] 2 0 [2, 4, 0] +query III +WITH pairs as ( + SELECT src, dst + FROM (SELECT a.rowid AS src FROM Student a where id = 10), + (SELECT b.rowid AS dst FROM Student b) +), shortestpath_cte as ( +SELECT src, dst, shortestpathoperator(src, dst, 'pairs') as path +FROM pairs p +WHERE p.src between (SELECT CSR_OPERATOR( + (SELECT count(a.id) as v_size FROM Student a), + (SELECT count(k.src) as e_size from know k), + a.rowid, + c.rowid, + k.rowid, + t.cnt) FROM Know k + JOIN student a on a.id = k.src + JOIN student c on c.id = k.dst + JOIN (SELECT count(k.src) cnt, a.rowid as a_rowid + FROM student a + LEFT JOIN know k ON k.src = a.id + GROUP BY a.rowid) t + ON t.a_rowid = a.rowid) and p.dst) + SELECT * FROM shortestpath_cte; +---- +0 0 [0] +0 1 NULL +0 2 [0, 1, 2] +0 3 NULL + query III SELECT *, shortestpathoperator(src, dst, 'pair') as path From 520d9d25c7d1e8f2a094d932099e318e5b57a049 Mon Sep 17 00:00:00 2001 From: dtenwolde Date: Wed, 28 Aug 2024 15:13:38 +0200 Subject: [PATCH 210/249] Update the test cases --- test/sql/path_finding/parallel_path_finding.test | 14 ++++++++------ 1 file changed, 8 insertions(+), 6 deletions(-) diff --git a/test/sql/path_finding/parallel_path_finding.test b/test/sql/path_finding/parallel_path_finding.test index 4a09acb6..cad1fc42 100644 --- a/test/sql/path_finding/parallel_path_finding.test +++ b/test/sql/path_finding/parallel_path_finding.test @@ -48,7 +48,7 @@ WHERE p.src between (SELECT CSR_OPERATOR( 1 2 [1, 3, 0, 1, 2] 2 0 [2, 4, 0] -query III +query IIII WITH pairs as ( SELECT src, dst FROM (SELECT a.rowid AS src FROM Student a where id = 10), @@ -70,12 +70,14 @@ WHERE p.src between (SELECT CSR_OPERATOR( LEFT JOIN know k ON k.src = a.id GROUP BY a.rowid) t ON t.a_rowid = a.rowid) and p.dst) - SELECT * FROM shortestpath_cte; +SELECT src, dst, src.id as src_id, path FROM shortestpath_cte +JOIN student src on src = rowid +ORDER BY dst; ---- -0 0 [0] -0 1 NULL -0 2 [0, 1, 2] -0 3 NULL +0 0 10 [0] +0 1 10 NULL +0 2 10 [0, 1, 2] +0 3 10 NULL query III From 7a06c6bf5c5bebbd37ff10d22ab9f331c08cb178 Mon Sep 17 00:00:00 2001 From: dtenwolde Date: Wed, 28 Aug 2024 15:23:12 +0200 Subject: [PATCH 211/249] Remove print and unused var --- src/core/operator/physical_path_finding_operator.cpp | 6 ------ 1 file changed, 6 deletions(-) diff --git a/src/core/operator/physical_path_finding_operator.cpp b/src/core/operator/physical_path_finding_operator.cpp index aa5b2142..c2b80308 100644 --- a/src/core/operator/physical_path_finding_operator.cpp +++ b/src/core/operator/physical_path_finding_operator.cpp @@ -132,9 +132,6 @@ void GlobalBFSState::CreateTasks() { if (current_task_start < (idx_t)v_size) { global_task_queue.push_back({current_task_start, v_size}); } - // for (const auto& task : global_task_queue) { - // std::cout << "Task: " << task.first << " " << task.second << std::endl; // debug - // } } @@ -189,7 +186,6 @@ void PathFindingLocalState::Sink(DataChunk &input, GlobalCompressedSparseRow &gl // Add the tasks (src, dst) to sink // Optimizations: Eliminate duplicate sources/destinations local_tasks.Append(input); - local_tasks.Print(); } else { // Create CSR local_inputs.Append(input); @@ -344,7 +340,6 @@ void PhysicalPathFinding::ScheduleBFSEvent(Pipeline &pipeline, Event &event, while (bfs_state->started_searches < gstate.global_tasks->Count()) { int64_t search_num = bfs_state->started_searches++; int64_t src_pos = bfs_state->vdata_src.sel->get_index(search_num); - int64_t dst_pos = bfs_state->vdata_dst.sel->get_index(search_num); if (!bfs_state->vdata_src.validity.RowIsValid(src_pos)) { result_validity.SetInvalid(search_num); result_data[search_num] = (uint64_t)-1; /* no path */ @@ -458,7 +453,6 @@ PhysicalPathFinding::GetData(ExecutionContext &context, DataChunk &result, throw NotImplementedException("Unrecognized mode for Path Finding"); } - // result.Print(); return result.size() == 0 ? SourceResultType::FINISHED : SourceResultType::HAVE_MORE_OUTPUT; } From 1f7af7757d89ff415cf088fb8230b466ab41f975 Mon Sep 17 00:00:00 2001 From: dtenwolde Date: Wed, 28 Aug 2024 15:26:47 +0200 Subject: [PATCH 212/249] remove print --- src/core/operator/task/csr_edge_creation_task.cpp | 1 - 1 file changed, 1 deletion(-) diff --git a/src/core/operator/task/csr_edge_creation_task.cpp b/src/core/operator/task/csr_edge_creation_task.cpp index 2593491b..b9333a41 100644 --- a/src/core/operator/task/csr_edge_creation_task.cpp +++ b/src/core/operator/task/csr_edge_creation_task.cpp @@ -37,7 +37,6 @@ TaskExecutionResult PhysicalCSREdgeCreationTask::ExecuteTask(TaskExecutionMode m }); } event->FinishTask(); - global_csr->Print(); return TaskExecutionResult::TASK_FINISHED; } From 93832f666c067490509e7c235ea5a4eb134991ce Mon Sep 17 00:00:00 2001 From: dtenwolde Date: Wed, 28 Aug 2024 16:07:33 +0200 Subject: [PATCH 213/249] Iterative length test now passing --- .../operator/event/iterative_length_event.cpp | 4 +- .../operator/task/iterative_length_task.cpp | 77 +++++++++++++------ src/core/operator/task/shortest_path_task.cpp | 6 +- .../operator/task/iterative_length_task.hpp | 3 +- 4 files changed, 62 insertions(+), 28 deletions(-) diff --git a/src/core/operator/event/iterative_length_event.cpp b/src/core/operator/event/iterative_length_event.cpp index a67781fc..9efa494e 100644 --- a/src/core/operator/event/iterative_length_event.cpp +++ b/src/core/operator/event/iterative_length_event.cpp @@ -14,7 +14,9 @@ void ParallelIterativeEvent::Schedule() { auto &context = pipeline->GetClientContext(); vector> bfs_tasks; - for (idx_t tnum = 0; tnum < bfs_state->num_threads; tnum++) { + size_t threads_to_schedule = std::min(bfs_state->num_threads, (idx_t)bfs_state->global_task_queue.size()); + + for (idx_t tnum = 0; tnum < threads_to_schedule; tnum++) { bfs_tasks.push_back(make_uniq( shared_from_this(), context, gstate, tnum)); } diff --git a/src/core/operator/task/iterative_length_task.cpp b/src/core/operator/task/iterative_length_task.cpp index 90e5f0e3..2385651e 100644 --- a/src/core/operator/task/iterative_length_task.cpp +++ b/src/core/operator/task/iterative_length_task.cpp @@ -9,26 +9,39 @@ PhysicalIterativeTask::PhysicalIterativeTask(shared_ptr event_p, ClientCo : ExecutorTask(context, std::move(event_p)), context(context), state(state), worker_id(worker_id) {} +bool PhysicalIterativeTask::SetTaskRange() { + auto task = state.global_bfs_state->FetchTask(); + if (task == nullptr) { + return false; + } + left = task->first; + right = task->second; + return true; +} + + TaskExecutionResult PhysicalIterativeTask::ExecuteTask(TaskExecutionMode mode) { auto &bfs_state = state.global_bfs_state; - auto &change = bfs_state->change; auto &barrier = bfs_state->barrier; - - auto bound = bfs_state->BoundaryCalculation(worker_id); - left = bound.first; - right = bound.second; - do { IterativeLength(); + barrier->Wait([&]() { + bfs_state->ResetTaskIndex(); // Reset task index safely + }); barrier->Wait(); if (worker_id == 0) { ReachDetect(); } + + barrier->Wait([&]() { + bfs_state->ResetTaskIndex(); // Reset task index safely + }); + barrier->Wait(); - } while (change); + } while (bfs_state->change); if (worker_id == 0) { UnReachableSet(); @@ -38,7 +51,7 @@ PhysicalIterativeTask::PhysicalIterativeTask(shared_ptr event_p, ClientCo return TaskExecutionResult::TASK_FINISHED; } - void PhysicalIterativeTask::IterativeLength() const { + void PhysicalIterativeTask::IterativeLength() { auto &bfs_state = state.global_bfs_state; auto &seen = bfs_state->seen; auto &visit = bfs_state->iter & 1 ? bfs_state->visit1 : bfs_state->visit2; @@ -46,9 +59,12 @@ PhysicalIterativeTask::PhysicalIterativeTask(shared_ptr event_p, ClientCo auto &barrier = bfs_state->barrier; int64_t *v = (int64_t *)state.global_csr->v; vector &e = state.global_csr->e; - auto &lane_to_num = bfs_state->lane_to_num; auto &change = bfs_state->change; + if (!SetTaskRange()) { + return; + } + // clear next before each iteration for (auto i = left; i < right; i++) { next[i] = 0; @@ -57,14 +73,7 @@ PhysicalIterativeTask::PhysicalIterativeTask(shared_ptr event_p, ClientCo barrier->Wait(); while (true) { - auto task = bfs_state->FetchTask(worker_id); - if (task.first == task.second) { - break; - } - auto start = task.first; - auto end = task.second; - - for (auto i = start; i < end; i++) { + for (auto i = left; i < right; i++) { if (visit[i].any()) { for (auto offset = v[i]; offset < v[i + 1]; offset++) { auto n = e[offset]; @@ -73,18 +82,40 @@ PhysicalIterativeTask::PhysicalIterativeTask(shared_ptr event_p, ClientCo } } } + if (!SetTaskRange()) { + break; // no more tasks + } } - change = false; + barrier->Wait([&]() { + bfs_state->ResetTaskIndex(); // Reset task index safely + }); + + + barrier->Wait(); - for (auto i = left; i < right; i++) { - if (next[i].any()) { - next[i] &= ~seen[i]; - seen[i] |= next[i]; - change |= next[i].any(); + if (!SetTaskRange()) { + return; // no more tasks + } + while (true) { + for (auto i = left; i < right; i++) { + if (next[i].any()) { + next[i] &= ~seen[i]; + seen[i] |= next[i]; + change |= next[i].any(); + } + } + if (!SetTaskRange()) { + break; // no more tasks } } + barrier->Wait([&]() { + bfs_state->ResetTaskIndex(); // Reset task index safely + }); + + barrier->Wait(); + } void PhysicalIterativeTask::ReachDetect() const { diff --git a/src/core/operator/task/shortest_path_task.cpp b/src/core/operator/task/shortest_path_task.cpp index 0be16202..f3ee8325 100644 --- a/src/core/operator/task/shortest_path_task.cpp +++ b/src/core/operator/task/shortest_path_task.cpp @@ -13,7 +13,6 @@ PhysicalShortestPathTask::PhysicalShortestPathTask(shared_ptr event_p, Cl TaskExecutionResult PhysicalShortestPathTask::ExecuteTask(TaskExecutionMode mode) { auto &bfs_state = state.global_bfs_state; - auto &change = bfs_state->change; auto &barrier = bfs_state->barrier; do { @@ -33,7 +32,7 @@ PhysicalShortestPathTask::PhysicalShortestPathTask(shared_ptr event_p, Cl }); barrier->Wait(); - } while (change); + } while (bfs_state->change); if (worker_id == 0) { PathConstruction(); @@ -102,11 +101,12 @@ PhysicalShortestPathTask::PhysicalShortestPathTask(shared_ptr event_p, Cl bfs_state->ResetTaskIndex(); // Reset task index safely }); + barrier->Wait(); + if (!SetTaskRange()) { return; // no more tasks } - barrier->Wait(); while (true) { for (auto i = left; i < right; i++) { if (next[i].any()) { diff --git a/src/include/duckpgq/core/operator/task/iterative_length_task.hpp b/src/include/duckpgq/core/operator/task/iterative_length_task.hpp index 77871cf0..8d1995be 100644 --- a/src/include/duckpgq/core/operator/task/iterative_length_task.hpp +++ b/src/include/duckpgq/core/operator/task/iterative_length_task.hpp @@ -14,9 +14,10 @@ class PhysicalIterativeTask : public ExecutorTask { TaskExecutionResult ExecuteTask(TaskExecutionMode mode) override; private: - void IterativeLength() const; + void IterativeLength(); void ReachDetect() const; void UnReachableSet() const; + bool SetTaskRange(); private: ClientContext &context; From a77b9df03a1c5eb967888db16727774947505e93 Mon Sep 17 00:00:00 2001 From: dtenwolde Date: Thu, 29 Aug 2024 13:30:16 +0200 Subject: [PATCH 214/249] Setting change to false and adding correct test result --- src/core/operator/task/shortest_path_task.cpp | 2 +- test/sql/path_finding/parallel_path_finding.test | 15 +++++++++++++++ 2 files changed, 16 insertions(+), 1 deletion(-) diff --git a/src/core/operator/task/shortest_path_task.cpp b/src/core/operator/task/shortest_path_task.cpp index f3ee8325..b3d5fe62 100644 --- a/src/core/operator/task/shortest_path_task.cpp +++ b/src/core/operator/task/shortest_path_task.cpp @@ -96,7 +96,7 @@ PhysicalShortestPathTask::PhysicalShortestPathTask(shared_ptr event_p, Cl break; // no more tasks } } - + change = false; barrier->Wait([&]() { bfs_state->ResetTaskIndex(); // Reset task index safely }); diff --git a/test/sql/path_finding/parallel_path_finding.test b/test/sql/path_finding/parallel_path_finding.test index cad1fc42..b2e9357f 100644 --- a/test/sql/path_finding/parallel_path_finding.test +++ b/test/sql/path_finding/parallel_path_finding.test @@ -223,3 +223,18 @@ WHERE p.src BETWEEN (SELECT CSR_OPERATOR( GROUP BY a.rowid) t ON t.a_rowid = a.rowid) AND p.dst; ---- +0 0 [0] +0 1 [0, 0, 1] +0 2 [0, 1, 2] +0 3 [0, 2, 3] +0 4 NULL +1 1 [1] +1 2 [1, 4, 2] +1 3 [1, 5, 3] +1 4 NULL +2 2 [2] +2 3 [2, 6, 3] +2 4 NULL +3 3 [3] +3 4 NULL +4 4 [4] \ No newline at end of file From e323d122ad555d0df7d3b560993237f66d488d48 Mon Sep 17 00:00:00 2001 From: dtenwolde Date: Thu, 29 Aug 2024 16:00:31 +0200 Subject: [PATCH 215/249] Adding basic cardinality estimation and paramsToString method --- src/core/operator/logical_path_finding_operator.cpp | 3 ++- src/core/operator/physical_path_finding_operator.cpp | 12 ++++++++++-- .../core/operator/physical_path_finding_operator.hpp | 2 ++ 3 files changed, 14 insertions(+), 3 deletions(-) diff --git a/src/core/operator/logical_path_finding_operator.cpp b/src/core/operator/logical_path_finding_operator.cpp index 8e403f5d..a7bdb689 100644 --- a/src/core/operator/logical_path_finding_operator.cpp +++ b/src/core/operator/logical_path_finding_operator.cpp @@ -8,8 +8,9 @@ namespace duckpgq { namespace core { unique_ptr LogicalPathFindingOperator::CreatePlan( - ClientContext &, duckdb::PhysicalPlanGenerator &generator) { + ClientContext &context, duckdb::PhysicalPlanGenerator &generator) { D_ASSERT(children.size() == 2); + estimated_cardinality = children[0]->EstimateCardinality(context); auto left = generator.CreatePlan(std::move(children[0])); auto right = generator.CreatePlan(std::move(children[1])); return make_uniq(*this, std::move(left), diff --git a/src/core/operator/physical_path_finding_operator.cpp b/src/core/operator/physical_path_finding_operator.cpp index c2b80308..8a4584b6 100644 --- a/src/core/operator/physical_path_finding_operator.cpp +++ b/src/core/operator/physical_path_finding_operator.cpp @@ -26,10 +26,11 @@ namespace core { PhysicalPathFinding::PhysicalPathFinding(LogicalExtensionOperator &op, unique_ptr left, unique_ptr right) - : PhysicalComparisonJoin(op, TYPE, {}, JoinType::INNER, 0) { + : PhysicalComparisonJoin(op, TYPE, {}, JoinType::INNER, op.estimated_cardinality) { children.push_back(std::move(left)); children.push_back(std::move(right)); expressions = std::move(op.expressions); + estimated_cardinality = op.estimated_cardinality; auto &path_finding_op = op.Cast(); mode = path_finding_op.mode; } @@ -306,7 +307,6 @@ PhysicalPathFinding::Finalize(Pipeline &pipeline, Event &event, context.TryGetCurrentSetting("experimental_path_finding_operator_task_size", task_size_value); gstate.global_bfs_state->split_size = task_size_value.GetValue(); - ; // Schedule the first round of BFS tasks if (all_pairs->size() > 0) { @@ -369,6 +369,14 @@ void PhysicalPathFinding::ScheduleBFSEvent(Pipeline &pipeline, Event &event, } } +string PhysicalPathFinding::ParamsToString() const { + std::cout << "ParamsToString" << std::endl; + auto result = mode; + result += "\n[INFOSEPARATOR]\n"; + result += StringUtil::Format("EC: %llu", estimated_cardinality); + return result; +} + //===--------------------------------------------------------------------===// // Operator //===--------------------------------------------------------------------===// diff --git a/src/include/duckpgq/core/operator/physical_path_finding_operator.hpp b/src/include/duckpgq/core/operator/physical_path_finding_operator.hpp index 3c84385a..458fa05a 100644 --- a/src/include/duckpgq/core/operator/physical_path_finding_operator.hpp +++ b/src/include/duckpgq/core/operator/physical_path_finding_operator.hpp @@ -117,6 +117,8 @@ class PhysicalPathFinding : public PhysicalComparisonJoin { public: + string ParamsToString() const override; + // CachingOperator Interface OperatorResultType ExecuteInternal(ExecutionContext &context, DataChunk &input, DataChunk &chunk, From 15a837268099f0498e5783aea956427f8e2ec6d7 Mon Sep 17 00:00:00 2001 From: dtenwolde Date: Thu, 29 Aug 2024 16:36:54 +0200 Subject: [PATCH 216/249] Move the getPathFindingOption method to the option file --- src/core/optimizer/path_finding_optimizer_rule.cpp | 7 +------ src/core/option/duckpgq_option.cpp | 5 +++++ .../duckpgq/core/optimizer/path_finding_optimizer_rule.hpp | 1 - src/include/duckpgq/core/option/duckpgq_option.hpp | 2 ++ 4 files changed, 8 insertions(+), 7 deletions(-) diff --git a/src/core/optimizer/path_finding_optimizer_rule.cpp b/src/core/optimizer/path_finding_optimizer_rule.cpp index e2ef6e2a..270d4778 100644 --- a/src/core/optimizer/path_finding_optimizer_rule.cpp +++ b/src/core/optimizer/path_finding_optimizer_rule.cpp @@ -15,16 +15,11 @@ #include #include #include +#include namespace duckpgq { namespace core { -bool DuckpgqOptimizerExtension::GetPathFindingOption(ClientContext &context) { - Value value; - context.TryGetCurrentSetting("experimental_path_finding_operator", value); - return value.GetValue(); -} - // Helper function to create the required BoundColumnRefExpression unique_ptr CreateReplacementExpression(const string &alias, const string &functionName, idx_t tableIndex, idx_t position) { if (functionName == "iterativelengthoperator") { diff --git a/src/core/option/duckpgq_option.cpp b/src/core/option/duckpgq_option.cpp index 4ad21f10..b476bcd2 100644 --- a/src/core/option/duckpgq_option.cpp +++ b/src/core/option/duckpgq_option.cpp @@ -4,6 +4,11 @@ namespace duckpgq { namespace core { +bool GetPathFindingOption(ClientContext &context) { + Value value; + context.TryGetCurrentSetting("experimental_path_finding_operator", value); + return value.GetValue(); +} //------------------------------------------------------------------------------ // Register option diff --git a/src/include/duckpgq/core/optimizer/path_finding_optimizer_rule.hpp b/src/include/duckpgq/core/optimizer/path_finding_optimizer_rule.hpp index 4c12397e..6a3ddd14 100644 --- a/src/include/duckpgq/core/optimizer/path_finding_optimizer_rule.hpp +++ b/src/include/duckpgq/core/optimizer/path_finding_optimizer_rule.hpp @@ -11,7 +11,6 @@ class DuckpgqOptimizerExtension : public OptimizerExtension { optimize_function = DuckpgqOptimizeFunction; } - static bool GetPathFindingOption(ClientContext &context); static bool InsertPathFindingOperator(LogicalOperator &op, ClientContext &context); static void DuckpgqOptimizeFunction(OptimizerExtensionInput &input, diff --git a/src/include/duckpgq/core/option/duckpgq_option.hpp b/src/include/duckpgq/core/option/duckpgq_option.hpp index 1eadc6b5..25dd4020 100644 --- a/src/include/duckpgq/core/option/duckpgq_option.hpp +++ b/src/include/duckpgq/core/option/duckpgq_option.hpp @@ -6,6 +6,8 @@ namespace duckpgq { namespace core { +bool GetPathFindingOption(ClientContext &context); + struct CorePGQOptions { static void Register(DatabaseInstance &db) { RegisterExperimentalPathFindingOperator(db); From b10eeb13a8e4a2d7a3b53591facf7372d3348cf7 Mon Sep 17 00:00:00 2001 From: dtenwolde Date: Fri, 30 Aug 2024 10:13:19 +0200 Subject: [PATCH 217/249] Adding test case with slightly more complex pattern --- .../path_finding/parallel_path_finding.test | 35 +++++++++++++++++++ 1 file changed, 35 insertions(+) diff --git a/test/sql/path_finding/parallel_path_finding.test b/test/sql/path_finding/parallel_path_finding.test index b2e9357f..44565505 100644 --- a/test/sql/path_finding/parallel_path_finding.test +++ b/test/sql/path_finding/parallel_path_finding.test @@ -48,6 +48,41 @@ WHERE p.src between (SELECT CSR_OPERATOR( 1 2 [1, 3, 0, 1, 2] 2 0 [2, 4, 0] +# MATCH ANY SHORTEST p = (a:Person)-[k2:knows]->(src:Person WHERE src.id = 10)-[k:knows]->*(dst:Person) +# COLUMNS (element_id(p)) AS path +query IIII +WITH pairs as ( + SELECT src, dst + FROM (SELECT a.rowid AS src FROM Student a where id = 10), + (SELECT b.rowid AS dst FROM Student b) +), p_cte as ( +SELECT src, dst, shortestpathoperator(src, dst, 'pairs') as path +FROM pairs p +WHERE p.src between (SELECT CSR_OPERATOR( + (SELECT count(a.id) as v_size FROM Student a), + (SELECT count(k.src) as e_size from know k), + a.rowid, + c.rowid, + k.rowid, + t.cnt) FROM Know k + JOIN student a on a.id = k.src + JOIN student c on c.id = k.dst + JOIN (SELECT count(k.src) cnt, a.rowid as a_rowid + FROM student a + LEFT JOIN know k ON k.src = a.id + GROUP BY a.rowid) t + ON t.a_rowid = a.rowid) and p.dst) +SELECT a.id, src.id, p_cte.src, p_cte.dst, src.id as src_id, path FROM p_cte +JOIN student src on p_cte.src = rowid +JOIN know k2 on k2.dst = src.id +JOIN student a on a.id = k2.src +ORDER BY p_cte.dst; +---- +0 0 10 [0] +0 1 10 NULL +0 2 10 [0, 1, 2] +0 3 10 NULL + query IIII WITH pairs as ( SELECT src, dst From 3332a06c5f0976d40f0a0e0592205ea9ac643d88 Mon Sep 17 00:00:00 2001 From: dtenwolde Date: Fri, 30 Aug 2024 10:13:32 +0200 Subject: [PATCH 218/249] Add ToString method for PGQMatchType --- .../duckpgq/core/utils/duckpgq_utils.hpp | 22 +++++++++++++++++++ 1 file changed, 22 insertions(+) diff --git a/src/include/duckpgq/core/utils/duckpgq_utils.hpp b/src/include/duckpgq/core/utils/duckpgq_utils.hpp index 6e831b39..61ed5d9d 100644 --- a/src/include/duckpgq/core/utils/duckpgq_utils.hpp +++ b/src/include/duckpgq/core/utils/duckpgq_utils.hpp @@ -1,6 +1,8 @@ #pragma once #include "duckpgq_state.hpp" +#include + namespace duckpgq { namespace core { @@ -11,6 +13,26 @@ namespace core { // Function to get DuckPGQState from ClientContext DuckPGQState *GetDuckPGQState(ClientContext &context); +class GraphUtils { +public: + static std::string ToString(PGQMatchType matchType) { + switch (matchType) { + case PGQMatchType::MATCH_VERTEX: + return "MATCH_VERTEX"; + case PGQMatchType::MATCH_EDGE_ANY: + return "MATCH_EDGE_ANY"; + case PGQMatchType::MATCH_EDGE_LEFT: + return "MATCH_EDGE_LEFT"; + case PGQMatchType::MATCH_EDGE_RIGHT: + return "MATCH_EDGE_RIGHT"; + case PGQMatchType::MATCH_EDGE_LEFT_RIGHT: + return "MATCH_EDGE_LEFT_RIGHT"; + default: + return "UNKNOWN_MATCH_TYPE"; + } + } +}; + } // namespace core } // namespace duckpgq From 55e27fa22e2a83e62c3867936cf7b9fc5b403b48 Mon Sep 17 00:00:00 2001 From: dtenwolde Date: Fri, 30 Aug 2024 10:13:50 +0200 Subject: [PATCH 219/249] Passing clientContext to methods to get the PathFindingOption --- src/core/functions/table/match.cpp | 58 +++++++++++++------ .../duckpgq/core/functions/table/match.hpp | 11 +++- 2 files changed, 47 insertions(+), 22 deletions(-) diff --git a/src/core/functions/table/match.cpp b/src/core/functions/table/match.cpp index 4e190bad..92e5b0d8 100644 --- a/src/core/functions/table/match.cpp +++ b/src/core/functions/table/match.cpp @@ -28,6 +28,8 @@ #include #include #include +#include + namespace duckpgq { @@ -368,6 +370,12 @@ PGQMatchFunction::CreateWhereClause(vector> &condit return where_clause; } +unique_ptr PGQMatchFunction::GenerateShortestPathOperatorCTE(CreatePropertyGraphInfo &pg_table, SubPath *edge_subpath) { + auto cte_info = make_uniq(); + + return cte_info; +} + unique_ptr PGQMatchFunction::GenerateShortestPathCTE(CreatePropertyGraphInfo &pg_table, SubPath *edge_subpath, PathElement * previous_vertex_element, PathElement * next_vertex_element, vector> &path_finding_conditions) { auto cte_info = make_uniq(); @@ -438,7 +446,8 @@ unique_ptr PGQMatchFunction::CreatePathFindingFunction( vector> &path_list, CreatePropertyGraphInfo &pg_table, const string &path_variable, unique_ptr &final_select_node, - vector> &conditions) { + vector> &conditions, + ClientContext &context) { // This method will return a SubqueryRef of a list of rowids // For every vertex and edge element, we add the rowid to the list using // list_append, or list_prepend The difficulty is that there may be a @@ -483,19 +492,30 @@ unique_ptr PGQMatchFunction::CreatePathFindingFunction( if (next_vertex_subpath) { path_finding_conditions.push_back(std::move(next_vertex_subpath->where_clause)); } - if (final_select_node->cte_map.map.find("cte1") == final_select_node->cte_map.map.end()) { + string shortest_path_cte_name = "shortest_path_cte" ; + if (GetPathFindingOption(context)) { edge_element = reinterpret_cast(edge_subpath->path_list[0].get()); - if (edge_element->match_type == PGQMatchType::MATCH_EDGE_RIGHT) { - final_select_node->cte_map.map["cte1"] = - CreateDirectedCSRCTE(FindGraphTable(edge_element->label, pg_table), previous_vertex_element->variable_binding, edge_element->variable_binding, next_vertex_element->variable_binding); - } else if (edge_element->match_type == PGQMatchType::MATCH_EDGE_ANY) { - final_select_node->cte_map.map["cte1"] = - CreateUndirectedCSRCTE(FindGraphTable(edge_element->label, pg_table), final_select_node); - } else { - throw NotImplementedException("Cannot do shortest path for edge type %s", edge_element->match_type == PGQMatchType::MATCH_EDGE_LEFT ? "MATCH_EDGE_LEFT" : "MATCH_EDGE_LEFT_RIGHT"); + if (edge_element->match_type != PGQMatchType::MATCH_EDGE_RIGHT) { + throw NotImplementedException("Cannot do shortest path for edge type %s", GraphUtils::ToString(edge_element->match_type)); + } + // Do the new path finding operator + CSR creation here + final_select_node->cte_map.map[shortest_path_cte_name] = + GenerateShortestPathOperatorCTE(pg_table, edge_subpath); + } else { + // Create UDF based CSR if not exists + if (final_select_node->cte_map.map.find("cte1") == final_select_node->cte_map.map.end()) { + edge_element = reinterpret_cast(edge_subpath->path_list[0].get()); + if (edge_element->match_type == PGQMatchType::MATCH_EDGE_RIGHT) { + final_select_node->cte_map.map["cte1"] = + CreateDirectedCSRCTE(FindGraphTable(edge_element->label, pg_table), previous_vertex_element->variable_binding, edge_element->variable_binding, next_vertex_element->variable_binding); + } else if (edge_element->match_type == PGQMatchType::MATCH_EDGE_ANY) { + final_select_node->cte_map.map["cte1"] = + CreateUndirectedCSRCTE(FindGraphTable(edge_element->label, pg_table), final_select_node); + } else { + throw NotImplementedException("Cannot do shortest path for edge type %s", edge_element->match_type == PGQMatchType::MATCH_EDGE_LEFT ? "MATCH_EDGE_LEFT" : "MATCH_EDGE_LEFT_RIGHT"); + } } } - string shortest_path_cte_name = "shortest_path_cte" ; if (final_select_node->cte_map.map.find(shortest_path_cte_name) == final_select_node->cte_map.map.end()) { final_select_node->cte_map.map[shortest_path_cte_name] = GenerateShortestPathCTE(pg_table, edge_subpath, previous_vertex_element, @@ -715,7 +735,7 @@ void PGQMatchFunction::AddPathFinding( void PGQMatchFunction::CheckNamedSubpath( SubPath &subpath, MatchExpression &original_ref, CreatePropertyGraphInfo &pg_table, unique_ptr &final_select_node, - vector> &conditions) { + vector> &conditions, ClientContext &context) { for (idx_t idx_i = 0; idx_i < original_ref.column_list.size(); idx_i++) { auto parsed_ref = dynamic_cast(original_ref.column_list[idx_i].get()); @@ -737,7 +757,7 @@ void PGQMatchFunction::CheckNamedSubpath( // Check subpath name matches the column referenced in the function --> // element_id(named_subpath) auto shortest_path_function = - CreatePathFindingFunction(subpath.path_list, pg_table, subpath.path_variable, final_select_node, conditions); + CreatePathFindingFunction(subpath.path_list, pg_table, subpath.path_variable, final_select_node, conditions, context); if (column_alias.empty()) { shortest_path_function->alias = @@ -750,7 +770,7 @@ void PGQMatchFunction::CheckNamedSubpath( std::move(shortest_path_function)); } else if (parsed_ref->function_name == "path_length") { auto shortest_path_function = - CreatePathFindingFunction(subpath.path_list, pg_table, subpath.path_variable, final_select_node, conditions); + CreatePathFindingFunction(subpath.path_list, pg_table, subpath.path_variable, final_select_node, conditions, context); auto path_len_children = vector>(); path_len_children.push_back(std::move(shortest_path_function)); auto path_len = @@ -771,7 +791,7 @@ void PGQMatchFunction::CheckNamedSubpath( parsed_ref->function_name == "edges") { auto list_slice_children = vector>(); auto shortest_path_function = - CreatePathFindingFunction(subpath.path_list, pg_table, subpath.path_variable, final_select_node, conditions); + CreatePathFindingFunction(subpath.path_list, pg_table, subpath.path_variable, final_select_node, conditions, context); list_slice_children.push_back(std::move(shortest_path_function)); if (parsed_ref->function_name == "vertices") { @@ -809,7 +829,7 @@ void PGQMatchFunction::ProcessPathList( unique_ptr &final_select_node, unordered_map &alias_map, CreatePropertyGraphInfo &pg_table, int32_t &extra_alias_counter, - MatchExpression &original_ref) { + MatchExpression &original_ref, ClientContext &context) { PathElement *previous_vertex_element = GetPathElement(path_list[0]); if (!previous_vertex_element) { const auto previous_vertex_subpath = @@ -818,7 +838,7 @@ void PGQMatchFunction::ProcessPathList( conditions.push_back(std::move(previous_vertex_subpath->where_clause)); } if (!previous_vertex_subpath->path_variable.empty() && previous_vertex_subpath->path_list.size() > 1) { - CheckNamedSubpath(*previous_vertex_subpath, original_ref, pg_table, final_select_node, conditions); + CheckNamedSubpath(*previous_vertex_subpath, original_ref, pg_table, final_select_node, conditions, context); } if (previous_vertex_subpath->path_list.size() == 1) { previous_vertex_element = @@ -827,7 +847,7 @@ void PGQMatchFunction::ProcessPathList( // Add the shortest path if the name is found in the column_list ProcessPathList(previous_vertex_subpath->path_list, conditions, final_select_node, alias_map, pg_table, - extra_alias_counter, original_ref); + extra_alias_counter, original_ref, context); return; } } @@ -935,7 +955,7 @@ PGQMatchFunction::MatchBindReplace(ClientContext &context, // items ProcessPathList(path_pattern->path_elements, conditions, final_select_node, alias_map, *pg_table, extra_alias_counter, - *ref); + *ref, context); } // Go through all aliases encountered diff --git a/src/include/duckpgq/core/functions/table/match.hpp b/src/include/duckpgq/core/functions/table/match.hpp index 53f476b2..c606fb0e 100644 --- a/src/include/duckpgq/core/functions/table/match.hpp +++ b/src/include/duckpgq/core/functions/table/match.hpp @@ -117,6 +117,9 @@ struct PGQMatchFunction : public TableFunction { vector> &column_list, unordered_set &named_subpaths); + static unique_ptr GenerateShortestPathOperatorCTE( + CreatePropertyGraphInfo &pg_table, SubPath *edge_subpath); + static unique_ptr GenerateShortestPathCTE( CreatePropertyGraphInfo &pg_table, SubPath *edge_subpath, PathElement *path_element, PathElement *next_vertex_element, @@ -127,7 +130,8 @@ struct PGQMatchFunction : public TableFunction { CreatePropertyGraphInfo &pg_table, const string &path_variable, unique_ptr &final_select_node, - vector> &conditions); + vector> &conditions, + ClientContext &context); static void AddPathFinding( unique_ptr &select_node, @@ -153,13 +157,14 @@ struct PGQMatchFunction : public TableFunction { unordered_map &alias_map, CreatePropertyGraphInfo &pg_table, int32_t &extra_alias_counter, - MatchExpression &original_ref); + MatchExpression &original_ref, ClientContext &context); static void CheckNamedSubpath(SubPath &subpath, MatchExpression &original_ref, CreatePropertyGraphInfo &pg_table, unique_ptr &final_select_node, - vector> &conditions); + vector> &conditions, + ClientContext &context); }; } // namespace core From e1e9a2a289ca0395c4ed12c729c55fddd1b51e96 Mon Sep 17 00:00:00 2001 From: dtenwolde Date: Wed, 4 Sep 2024 16:58:39 +0200 Subject: [PATCH 220/249] Register the new operators, got lost in merge --- src/include/duckpgq/core/functions/scalar.hpp | 6 ++++++ 1 file changed, 6 insertions(+) diff --git a/src/include/duckpgq/core/functions/scalar.hpp b/src/include/duckpgq/core/functions/scalar.hpp index 25d063a9..5a2f60db 100644 --- a/src/include/duckpgq/core/functions/scalar.hpp +++ b/src/include/duckpgq/core/functions/scalar.hpp @@ -13,6 +13,9 @@ struct CoreScalarFunctions { RegisterGetCSRWTypeScalarFunction(db); RegisterIterativeLengthScalarFunction(db); RegisterIterativeLength2ScalarFunction(db); + RegisterCSROperatorCreationScalarFunctions(db); + RegisterShortestPathOperatorScalarFunction(db); + RegisterIterativeLengthOperatorScalarFunction(db); RegisterIterativeLengthBidirectionalScalarFunction(db); RegisterLocalClusteringCoefficientScalarFunction(db); RegisterReachabilityScalarFunction(db); @@ -34,6 +37,9 @@ struct CoreScalarFunctions { static void RegisterShortestPathScalarFunction(DatabaseInstance &db); static void RegisterWeaklyConnectedComponentScalarFunction(DatabaseInstance &db); static void RegisterPageRankScalarFunction(DatabaseInstance &db); + static void RegisterCSROperatorCreationScalarFunctions(DatabaseInstance &db); + static void RegisterShortestPathOperatorScalarFunction(DatabaseInstance &db); + static void RegisterIterativeLengthOperatorScalarFunction(DatabaseInstance &db); }; From 092ef27929d85ce30bf186956e12c11251d271e9 Mon Sep 17 00:00:00 2001 From: dtenwolde Date: Wed, 4 Sep 2024 17:17:35 +0200 Subject: [PATCH 221/249] Fix csr index --- src/core/functions/function_data/csr_function_data.cpp | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/src/core/functions/function_data/csr_function_data.cpp b/src/core/functions/function_data/csr_function_data.cpp index 437b0a12..cbf4e7ba 100644 --- a/src/core/functions/function_data/csr_function_data.cpp +++ b/src/core/functions/function_data/csr_function_data.cpp @@ -36,9 +36,9 @@ CSRFunctionData::CSREdgeBind(ClientContext &context, ScalarFunction &bound_function, vector> &arguments) { auto csr_id = GetCSRId(arguments[0], context); - if (arguments.size() == 7) { + if (arguments.size() == 8) { return make_uniq(context, csr_id, - arguments[6]->return_type); + arguments[7]->return_type); } auto logical_type = LogicalType::SQLNULL; return make_uniq(context, csr_id, logical_type); From 451297a7bb4a0588fd2c78993f6606483d6b917d Mon Sep 17 00:00:00 2001 From: dtenwolde Date: Thu, 5 Sep 2024 10:45:56 +0200 Subject: [PATCH 222/249] Adding test with PGQ syntax --- .../path_finding/parallel_path_finding.test | 42 +++++++++++++++---- 1 file changed, 33 insertions(+), 9 deletions(-) diff --git a/test/sql/path_finding/parallel_path_finding.test b/test/sql/path_finding/parallel_path_finding.test index 44565505..4607a7f7 100644 --- a/test/sql/path_finding/parallel_path_finding.test +++ b/test/sql/path_finding/parallel_path_finding.test @@ -24,6 +24,21 @@ create table student(id INT); INSERT INTO student(id) VALUES (10), (20), (30), ( statement ok create table know(src INT, dst INT); INSERT INTO know(src, dst) VALUES (40, 20), (10,30), (10,10), (20,10), (30,10); +statement ok +-CREATE PROPERTY GRAPH g +VERTEX TABLES (student) +EDGE TABLES (know SOURCE KEY ( src ) REFERENCES Student ( id ) + DESTINATION KEY ( dst ) REFERENCES Student ( id ) + ); + + +query III +-FROM GRAPH_TABLE (g + MATCH p = ANY SHORTEST (src:Student WHERE src.id = 10)-[k:know]-> *(dst:Student) + COLUMNS (element_id(p) AS path)); +---- + + query III WITH shortestpath_cte as ( SELECT src, dst, shortestpathoperator(src, dst, 'pair') as path @@ -49,13 +64,13 @@ WHERE p.src between (SELECT CSR_OPERATOR( 2 0 [2, 4, 0] # MATCH ANY SHORTEST p = (a:Person)-[k2:knows]->(src:Person WHERE src.id = 10)-[k:knows]->*(dst:Person) -# COLUMNS (element_id(p)) AS path +# COLUMNS (element_id(p) AS path) query IIII WITH pairs as ( SELECT src, dst FROM (SELECT a.rowid AS src FROM Student a where id = 10), (SELECT b.rowid AS dst FROM Student b) -), p_cte as ( +), shortestpath as ( SELECT src, dst, shortestpathoperator(src, dst, 'pairs') as path FROM pairs p WHERE p.src between (SELECT CSR_OPERATOR( @@ -72,16 +87,25 @@ WHERE p.src between (SELECT CSR_OPERATOR( LEFT JOIN know k ON k.src = a.id GROUP BY a.rowid) t ON t.a_rowid = a.rowid) and p.dst) -SELECT a.id, src.id, p_cte.src, p_cte.dst, src.id as src_id, path FROM p_cte -JOIN student src on p_cte.src = rowid +SELECT a.id as a_id, src.id src_id, shortestpath.src as sp_src, shortestpath.dst as sp_dst, dst.id, path FROM shortestpath +JOIN student src on shortestpath.src = src.rowid +JOIN student dst on shortestpath.dst = dst.rowid JOIN know k2 on k2.dst = src.id JOIN student a on a.id = k2.src -ORDER BY p_cte.dst; +ORDER BY shortestpath.dst; ---- -0 0 10 [0] -0 1 10 NULL -0 2 10 [0, 1, 2] -0 3 10 NULL +10 10 0 0 10 [0] +20 10 0 0 10 [0] +30 10 0 0 10 [0] +10 10 0 1 20 NULL +20 10 0 1 20 NULL +30 10 0 1 20 NULL +10 10 0 2 30 [0, 1, 2] +20 10 0 2 30 [0, 1, 2] +30 10 0 2 30 [0, 1, 2] +10 10 0 3 40 NULL +20 10 0 3 40 NULL +30 10 0 3 40 NULL query IIII WITH pairs as ( From 8ee8970beadf477944c770fa1545abc895db8c93 Mon Sep 17 00:00:00 2001 From: dtenwolde Date: Thu, 5 Sep 2024 11:02:55 +0200 Subject: [PATCH 223/249] Select and from clauses --- src/core/functions/table/match.cpp | 21 +++++++++++++++++++++ 1 file changed, 21 insertions(+) diff --git a/src/core/functions/table/match.cpp b/src/core/functions/table/match.cpp index 46ac0a59..619952d7 100644 --- a/src/core/functions/table/match.cpp +++ b/src/core/functions/table/match.cpp @@ -347,6 +347,27 @@ PGQMatchFunction::CreateWhereClause(vector> &condit unique_ptr PGQMatchFunction::GenerateShortestPathOperatorCTE(CreatePropertyGraphInfo &pg_table, SubPath *edge_subpath) { auto cte_info = make_uniq(); + cte_info->materialized = CTEMaterialize::CTE_MATERIALIZE_ALWAYS; + auto select_statement = make_uniq(); + auto select_node = make_uniq(); + + auto edge_element = GetPathElement(edge_subpath->path_list[0]); + auto edge_table = FindGraphTable(edge_element->label, pg_table); + + select_node->select_list.emplace_back(CreateColumnRefExpression("src")); + select_node->select_list.emplace_back(CreateColumnRefExpression("dst")); + + vector> shortest_path_operator_children; + shortest_path_operator_children.emplace_back(CreateColumnRefExpression("src")); + shortest_path_operator_children.emplace_back(CreateColumnRefExpression("dst")); + auto pairs_cte_ref = make_uniq(Value('pairs_cte')); + pairs_cte_ref->alias = "path"; + shortest_path_operator_children.emplace_back(std::move(pairs_cte_ref)); + + select_node->from_table = CreateBaseTableRef("pairs_cte", "p"); + + + return cte_info; } From 790de5d3bbe6e50a347544bea07bdad2d5f24a86 Mon Sep 17 00:00:00 2001 From: dtenwolde Date: Thu, 5 Sep 2024 11:03:52 +0200 Subject: [PATCH 224/249] Fixed shortest path function expression --- src/core/functions/table/match.cpp | 4 +++- 1 file changed, 3 insertions(+), 1 deletion(-) diff --git a/src/core/functions/table/match.cpp b/src/core/functions/table/match.cpp index 619952d7..2fca1e35 100644 --- a/src/core/functions/table/match.cpp +++ b/src/core/functions/table/match.cpp @@ -362,7 +362,9 @@ unique_ptr PGQMatchFunction::GenerateShortestPathOper shortest_path_operator_children.emplace_back(CreateColumnRefExpression("dst")); auto pairs_cte_ref = make_uniq(Value('pairs_cte')); pairs_cte_ref->alias = "path"; - shortest_path_operator_children.emplace_back(std::move(pairs_cte_ref)); + + auto shortest_path_function = make_uniq("shortestpathoperator", std::move(shortest_path_operator_children)); + shortest_path_operator_children.emplace_back(std::move(shortest_path_function)); select_node->from_table = CreateBaseTableRef("pairs_cte", "p"); From 447978d3401fe503c599d351be1486f4124366cd Mon Sep 17 00:00:00 2001 From: dtenwolde Date: Thu, 5 Sep 2024 11:26:25 +0200 Subject: [PATCH 225/249] Starting on pairs cte --- src/core/functions/table/match.cpp | 38 +++++++++++++------ .../duckpgq/core/functions/table/match.hpp | 10 ++++- 2 files changed, 34 insertions(+), 14 deletions(-) diff --git a/src/core/functions/table/match.cpp b/src/core/functions/table/match.cpp index 2fca1e35..5531cfbd 100644 --- a/src/core/functions/table/match.cpp +++ b/src/core/functions/table/match.cpp @@ -345,7 +345,16 @@ PGQMatchFunction::CreateWhereClause(vector> &condit return where_clause; } -unique_ptr PGQMatchFunction::GenerateShortestPathOperatorCTE(CreatePropertyGraphInfo &pg_table, SubPath *edge_subpath) { +void PGQMatchFunction::CreatePairsCTE( + shared_ptr &edge_table, const string &pairs_cte_name, + unique_ptr &final_select_node, + vector> &conditions) {} + +void PGQMatchFunction::GenerateShortestPathOperatorCTE( + CreatePropertyGraphInfo &pg_table, SubPath *edge_subpath, + const unique_ptr &final_select_node, + vector> &path_finding_conditions) { + string pairs_cte_name = "pairs_cte"; auto cte_info = make_uniq(); cte_info->materialized = CTEMaterialize::CTE_MATERIALIZE_ALWAYS; auto select_statement = make_uniq(); @@ -354,24 +363,28 @@ unique_ptr PGQMatchFunction::GenerateShortestPathOper auto edge_element = GetPathElement(edge_subpath->path_list[0]); auto edge_table = FindGraphTable(edge_element->label, pg_table); + CreatePairsCTE(edge_table, pairs_cte_name, select_node, + path_finding_conditions); + select_node->select_list.emplace_back(CreateColumnRefExpression("src")); select_node->select_list.emplace_back(CreateColumnRefExpression("dst")); vector> shortest_path_operator_children; shortest_path_operator_children.emplace_back(CreateColumnRefExpression("src")); - shortest_path_operator_children.emplace_back(CreateColumnRefExpression("dst")); - auto pairs_cte_ref = make_uniq(Value('pairs_cte')); + shortest_path_operator_children.emplace_back( + CreateColumnRefExpression("dst")); + auto pairs_cte_ref = make_uniq(Value(pairs_cte_name)); pairs_cte_ref->alias = "path"; auto shortest_path_function = make_uniq("shortestpathoperator", std::move(shortest_path_operator_children)); - shortest_path_operator_children.emplace_back(std::move(shortest_path_function)); - - select_node->from_table = CreateBaseTableRef("pairs_cte", "p"); - + shortest_path_operator_children.emplace_back( + std::move(shortest_path_function)); + select_node->from_table = CreateBaseTableRef(pairs_cte_name, "p"); - - return cte_info; + select_statement->node = std::move(select_node); + cte_info->query = std::move(select_statement); + final_select_node->cte_map.map["shortestpath_cte"] = std::move(cte_info); } unique_ptr PGQMatchFunction::GenerateShortestPathCTE(CreatePropertyGraphInfo &pg_table, SubPath *edge_subpath, @@ -494,11 +507,12 @@ unique_ptr PGQMatchFunction::CreatePathFindingFunction( if (GetPathFindingOption(context)) { edge_element = reinterpret_cast(edge_subpath->path_list[0].get()); if (edge_element->match_type != PGQMatchType::MATCH_EDGE_RIGHT) { - throw NotImplementedException("Cannot do shortest path for edge type %s", GraphUtils::ToString(edge_element->match_type)); + throw NotImplementedException( + "Cannot do shortest path using the operator for edge type %s", + GraphUtils::ToString(edge_element->match_type)); } // Do the new path finding operator + CSR creation here - final_select_node->cte_map.map[shortest_path_cte_name] = - GenerateShortestPathOperatorCTE(pg_table, edge_subpath); + GenerateShortestPathOperatorCTE(pg_table, edge_subpath, final_select_node, path_finding_conditions); } else { // Create UDF based CSR if not exists if (final_select_node->cte_map.map.find("cte1") == final_select_node->cte_map.map.end()) { diff --git a/src/include/duckpgq/core/functions/table/match.hpp b/src/include/duckpgq/core/functions/table/match.hpp index 59038ec0..85550b85 100644 --- a/src/include/duckpgq/core/functions/table/match.hpp +++ b/src/include/duckpgq/core/functions/table/match.hpp @@ -113,8 +113,14 @@ struct PGQMatchFunction : public TableFunction { vector> &column_list, unordered_set &named_subpaths); - static unique_ptr GenerateShortestPathOperatorCTE( - CreatePropertyGraphInfo &pg_table, SubPath *edge_subpath); + static void CreatePairsCTE(shared_ptr &edge_table, + const string& pairs_cte_name, unique_ptr &final_select_node, + vector> &conditions); + + static void GenerateShortestPathOperatorCTE( + CreatePropertyGraphInfo &pg_table, SubPath *edge_subpath, + const unique_ptr &final_select_node, + vector> &path_finding_conditions); static unique_ptr GenerateShortestPathCTE( CreatePropertyGraphInfo &pg_table, SubPath *edge_subpath, From 1b157f1891cbfdb999518aeb3e136bb3f75ac067 Mon Sep 17 00:00:00 2001 From: dtenwolde Date: Thu, 5 Sep 2024 11:37:27 +0200 Subject: [PATCH 226/249] Finished pairs cte statement --- src/core/functions/table/match.cpp | 59 +++++++++++++++---- .../duckpgq/core/functions/table/match.hpp | 6 +- 2 files changed, 53 insertions(+), 12 deletions(-) diff --git a/src/core/functions/table/match.cpp b/src/core/functions/table/match.cpp index 5531cfbd..38390695 100644 --- a/src/core/functions/table/match.cpp +++ b/src/core/functions/table/match.cpp @@ -348,12 +348,50 @@ PGQMatchFunction::CreateWhereClause(vector> &condit void PGQMatchFunction::CreatePairsCTE( shared_ptr &edge_table, const string &pairs_cte_name, unique_ptr &final_select_node, - vector> &conditions) {} + unique_ptr &src_conditions, + unique_ptr &dst_conditions) { + auto pairs_cte_select_node = make_uniq(); + + pairs_cte_select_node->select_list.emplace_back(CreateColumnRefExpression("src")); + pairs_cte_select_node->select_list.emplace_back(CreateColumnRefExpression("dst")); + + auto src_subquery = make_uniq(); + auto src_select_statement = make_uniq(); + auto src_select_node = make_uniq(); + + src_select_node->select_list.emplace_back(CreateColumnRefExpression("rowid", "", "src")); + src_select_node->from_table = CreateBaseTableRef(edge_table->source_reference); + src_select_node->where_clause = std::move(src_conditions); + src_select_statement->node = std::move(src_select_node); + src_subquery->subquery = std::move(src_select_statement); + + auto dst_subquery = make_uniq(); + auto dst_select_statement = make_uniq(); + auto dst_select_node = make_uniq(); + + dst_select_node->select_list.emplace_back(CreateColumnRefExpression("rowid", "", "dst")); + dst_select_node->from_table = CreateBaseTableRef(edge_table->destination_reference); + dst_select_node->where_clause = std::move(dst_conditions); + dst_select_statement->node = std::move(dst_select_node); + dst_subquery->subquery = std::move(dst_select_statement); + + auto cross_join = make_uniq(JoinRefType::CROSS); + cross_join->left = std::move(src_subquery); + cross_join->right = std::move(dst_subquery); + + pairs_cte_select_node->from_table = std::move(cross_join); + auto select_statement = make_uniq(); + auto cte_info = make_uniq(); + select_statement->node = std::move(pairs_cte_select_node); + cte_info->query = std::move(select_statement); + final_select_node->cte_map.map[pairs_cte_name] = std::move(cte_info); +} void PGQMatchFunction::GenerateShortestPathOperatorCTE( CreatePropertyGraphInfo &pg_table, SubPath *edge_subpath, const unique_ptr &final_select_node, - vector> &path_finding_conditions) { + unique_ptr &src_conditions, + unique_ptr &dst_conditions) { string pairs_cte_name = "pairs_cte"; auto cte_info = make_uniq(); cte_info->materialized = CTEMaterialize::CTE_MATERIALIZE_ALWAYS; @@ -364,7 +402,7 @@ void PGQMatchFunction::GenerateShortestPathOperatorCTE( auto edge_table = FindGraphTable(edge_element->label, pg_table); CreatePairsCTE(edge_table, pairs_cte_name, select_node, - path_finding_conditions); + src_conditions, dst_conditions); select_node->select_list.emplace_back(CreateColumnRefExpression("src")); select_node->select_list.emplace_back(CreateColumnRefExpression("dst")); @@ -497,12 +535,7 @@ unique_ptr PGQMatchFunction::CreatePathFindingFunction( if (edge_subpath->upper > 1) { // (un)bounded shortest path // Add the shortest path UDF as a CTE - if (previous_vertex_subpath) { - path_finding_conditions.push_back(std::move(previous_vertex_subpath->where_clause)); - } - if (next_vertex_subpath) { - path_finding_conditions.push_back(std::move(next_vertex_subpath->where_clause)); - } + string shortest_path_cte_name = "shortest_path_cte" ; if (GetPathFindingOption(context)) { edge_element = reinterpret_cast(edge_subpath->path_list[0].get()); @@ -512,8 +545,14 @@ unique_ptr PGQMatchFunction::CreatePathFindingFunction( GraphUtils::ToString(edge_element->match_type)); } // Do the new path finding operator + CSR creation here - GenerateShortestPathOperatorCTE(pg_table, edge_subpath, final_select_node, path_finding_conditions); + GenerateShortestPathOperatorCTE(pg_table, edge_subpath, final_select_node, previous_vertex_subpath->where_clause, next_vertex_subpath->where_clause); } else { + if (previous_vertex_subpath) { + path_finding_conditions.push_back(std::move(previous_vertex_subpath->where_clause)); + } + if (next_vertex_subpath) { + path_finding_conditions.push_back(std::move(next_vertex_subpath->where_clause)); + } // Create UDF based CSR if not exists if (final_select_node->cte_map.map.find("cte1") == final_select_node->cte_map.map.end()) { edge_element = reinterpret_cast(edge_subpath->path_list[0].get()); diff --git a/src/include/duckpgq/core/functions/table/match.hpp b/src/include/duckpgq/core/functions/table/match.hpp index 85550b85..7d764bb1 100644 --- a/src/include/duckpgq/core/functions/table/match.hpp +++ b/src/include/duckpgq/core/functions/table/match.hpp @@ -115,12 +115,14 @@ struct PGQMatchFunction : public TableFunction { static void CreatePairsCTE(shared_ptr &edge_table, const string& pairs_cte_name, unique_ptr &final_select_node, - vector> &conditions); + unique_ptr &src_conditions, + unique_ptr &dst_conditions); static void GenerateShortestPathOperatorCTE( CreatePropertyGraphInfo &pg_table, SubPath *edge_subpath, const unique_ptr &final_select_node, - vector> &path_finding_conditions); + unique_ptr &src_conditions, + unique_ptr &dst_conditions); static unique_ptr GenerateShortestPathCTE( CreatePropertyGraphInfo &pg_table, SubPath *edge_subpath, From 3921baa90264dcb02539fb4b43dbd95b065754ea Mon Sep 17 00:00:00 2001 From: dtenwolde Date: Thu, 5 Sep 2024 15:14:15 +0200 Subject: [PATCH 227/249] Starting on csr operator subquery --- src/core/functions/table/match.cpp | 17 +++++++++++++++-- .../duckpgq/core/functions/table/match.hpp | 6 +++++- 2 files changed, 20 insertions(+), 3 deletions(-) diff --git a/src/core/functions/table/match.cpp b/src/core/functions/table/match.cpp index 38390695..a6f965d8 100644 --- a/src/core/functions/table/match.cpp +++ b/src/core/functions/table/match.cpp @@ -387,11 +387,19 @@ void PGQMatchFunction::CreatePairsCTE( final_select_node->cte_map.map[pairs_cte_name] = std::move(cte_info); } +unique_ptr PGQMatchFunction::GenerateCSROperatorSubquery(shared_ptr &edge_table, const string& src_table_alias, const string& dst_table_alias) { + auto result = make_uniq(); + +} + + void PGQMatchFunction::GenerateShortestPathOperatorCTE( CreatePropertyGraphInfo &pg_table, SubPath *edge_subpath, const unique_ptr &final_select_node, unique_ptr &src_conditions, - unique_ptr &dst_conditions) { + unique_ptr &dst_conditions, + const string& src_table_alias, + const string& dst_table_alias) { string pairs_cte_name = "pairs_cte"; auto cte_info = make_uniq(); cte_info->materialized = CTEMaterialize::CTE_MATERIALIZE_ALWAYS; @@ -420,6 +428,9 @@ void PGQMatchFunction::GenerateShortestPathOperatorCTE( select_node->from_table = CreateBaseTableRef(pairs_cte_name, "p"); + GenerateCSROperatorSubquery(edge_table, src_table_alias, dst_table_alias); + + select_statement->node = std::move(select_node); cte_info->query = std::move(select_statement); final_select_node->cte_map.map["shortestpath_cte"] = std::move(cte_info); @@ -545,7 +556,9 @@ unique_ptr PGQMatchFunction::CreatePathFindingFunction( GraphUtils::ToString(edge_element->match_type)); } // Do the new path finding operator + CSR creation here - GenerateShortestPathOperatorCTE(pg_table, edge_subpath, final_select_node, previous_vertex_subpath->where_clause, next_vertex_subpath->where_clause); + GenerateShortestPathOperatorCTE(pg_table, edge_subpath, final_select_node, + previous_vertex_subpath->where_clause, next_vertex_subpath->where_clause, + previous_vertex_element->variable_binding, next_vertex_element->variable_binding); } else { if (previous_vertex_subpath) { path_finding_conditions.push_back(std::move(previous_vertex_subpath->where_clause)); diff --git a/src/include/duckpgq/core/functions/table/match.hpp b/src/include/duckpgq/core/functions/table/match.hpp index 7d764bb1..c947df7e 100644 --- a/src/include/duckpgq/core/functions/table/match.hpp +++ b/src/include/duckpgq/core/functions/table/match.hpp @@ -113,6 +113,8 @@ struct PGQMatchFunction : public TableFunction { vector> &column_list, unordered_set &named_subpaths); + static unique_ptr GenerateCSROperatorSubquery(shared_ptr &edge_table, const string& src_table_alias, const string& dst_table_alias) + static void CreatePairsCTE(shared_ptr &edge_table, const string& pairs_cte_name, unique_ptr &final_select_node, unique_ptr &src_conditions, @@ -122,7 +124,9 @@ struct PGQMatchFunction : public TableFunction { CreatePropertyGraphInfo &pg_table, SubPath *edge_subpath, const unique_ptr &final_select_node, unique_ptr &src_conditions, - unique_ptr &dst_conditions); + unique_ptr &dst_conditions, + const string& src_table_alias, + const string& dst_table_alias); static unique_ptr GenerateShortestPathCTE( CreatePropertyGraphInfo &pg_table, SubPath *edge_subpath, From f9962310f08dfc8b65c593742c0c782c059f07e0 Mon Sep 17 00:00:00 2001 From: dtenwolde Date: Thu, 5 Sep 2024 15:55:37 +0200 Subject: [PATCH 228/249] remove unused method --- src/core/functions/table/match.cpp | 7 ------- src/include/duckpgq/core/functions/table/match.hpp | 5 ----- 2 files changed, 12 deletions(-) diff --git a/src/core/functions/table/match.cpp b/src/core/functions/table/match.cpp index a6f965d8..2d1d89dc 100644 --- a/src/core/functions/table/match.cpp +++ b/src/core/functions/table/match.cpp @@ -323,13 +323,6 @@ void PGQMatchFunction::EdgeTypeLeftRight( conditions.push_back(std::move(combined_expr)); } -PathElement *PGQMatchFunction::HandleNestedSubPath( - unique_ptr &path_reference, - vector> &conditions, idx_t element_idx) { - auto subpath = reinterpret_cast(path_reference.get()); - return GetPathElement(subpath->path_list[element_idx]); -} - unique_ptr PGQMatchFunction::CreateWhereClause(vector> &conditions) { unique_ptr where_clause; diff --git a/src/include/duckpgq/core/functions/table/match.hpp b/src/include/duckpgq/core/functions/table/match.hpp index c947df7e..d6c34dcb 100644 --- a/src/include/duckpgq/core/functions/table/match.hpp +++ b/src/include/duckpgq/core/functions/table/match.hpp @@ -96,11 +96,6 @@ struct PGQMatchFunction : public TableFunction { unordered_map &alias_map, int32_t &extra_alias_counter); - static PathElement * - HandleNestedSubPath(unique_ptr &path_reference, - vector> &conditions, - idx_t element_idx); - static unique_ptr AddPathQuantifierCondition( const string &prev_binding, const string &next_binding, const shared_ptr &edge_table, const SubPath *subpath); From 990936d74ed1f085580142e3feeefb82ae6e622c Mon Sep 17 00:00:00 2001 From: dtenwolde Date: Thu, 5 Sep 2024 15:55:54 +0200 Subject: [PATCH 229/249] Only missing group by for csr operator subquery --- src/core/functions/table/match.cpp | 53 ++++++++++++++++++- .../duckpgq/core/functions/table/match.hpp | 6 ++- 2 files changed, 56 insertions(+), 3 deletions(-) diff --git a/src/core/functions/table/match.cpp b/src/core/functions/table/match.cpp index 2d1d89dc..7f9952ec 100644 --- a/src/core/functions/table/match.cpp +++ b/src/core/functions/table/match.cpp @@ -380,9 +380,56 @@ void PGQMatchFunction::CreatePairsCTE( final_select_node->cte_map.map[pairs_cte_name] = std::move(cte_info); } -unique_ptr PGQMatchFunction::GenerateCSROperatorSubquery(shared_ptr &edge_table, const string& src_table_alias, const string& dst_table_alias) { +unique_ptr PGQMatchFunction::GenerateCSROperatorSubquery(shared_ptr &edge_table, const string& src_table_alias, const string& edge_table_alias, const string& dst_table_alias) { auto result = make_uniq(); + auto select_statement = make_uniq(); + auto select_node = make_uniq(); + + vector> csr_operator_children; + csr_operator_children.emplace_back(GetCountTable(edge_table->source_reference, src_table_alias, edge_table->source_pk[0])); + csr_operator_children.emplace_back(GetCountTable(edge_table->table_name, edge_table_alias, edge_table->source_fk[0])); + + csr_operator_children.emplace_back(CreateColumnRefExpression("rowid", src_table_alias)); + csr_operator_children.emplace_back(CreateColumnRefExpression("rowid", dst_table_alias)); + csr_operator_children.emplace_back(CreateColumnRefExpression("rowid", edge_table_alias)); + + csr_operator_children.emplace_back(CreateColumnRefExpression("cnt", "count_per_vertex")); + + select_node->select_list.emplace_back(make_uniq("csr_operator", std::move(csr_operator_children))); + auto edge_src_joinref = make_uniq(JoinRefType::REGULAR); + edge_src_joinref->left = CreateBaseTableRef(edge_table->table_name, edge_table_alias); + edge_src_joinref->right = CreateBaseTableRef(edge_table->source_reference, src_table_alias); + edge_src_joinref->condition = make_uniq(ExpressionType::COMPARE_EQUAL, CreateColumnRefExpression(edge_table->source_fk[0], edge_table_alias), CreateColumnRefExpression(edge_table->source_pk[0], src_table_alias)); + + auto edge_dst_joinref = make_uniq(JoinRefType::REGULAR); + edge_dst_joinref->left = std::move(edge_src_joinref); + edge_dst_joinref->right = CreateBaseTableRef(edge_table->destination_reference, dst_table_alias); + edge_dst_joinref->condition = make_uniq(ExpressionType::COMPARE_EQUAL, CreateColumnRefExpression(edge_table->destination_fk[0], edge_table_alias), CreateColumnRefExpression(edge_table->destination_pk[0], dst_table_alias)); + auto edges_per_vertex_subquery = make_uniq(); + auto edges_per_vertex_select_statement = make_uniq(); + auto edges_per_vertex_select_node = make_uniq(); + + vector> count_children; + count_children.emplace_back(CreateColumnRefExpression(edge_table->source_fk[0], edge_table_alias)); + auto count_function_expression = make_uniq("count", std::move(count_children)); + count_function_expression->alias = "cnt"; + edges_per_vertex_select_node->select_list.emplace_back(std::move(count_function_expression)); + edges_per_vertex_select_node->select_list.emplace_back(CreateColumnRefExpression("rowid", src_table_alias)); + + auto left_join = make_uniq(JoinRefType::REGULAR); + left_join->type = JoinType::LEFT; + left_join->left = CreateBaseTableRef(edge_table->source_reference, src_table_alias); + left_join->right = CreateBaseTableRef(edge_table->table_name, edge_table_alias); + left_join->condition = make_uniq(ExpressionType::COMPARE_EQUAL, CreateColumnRefExpression(edge_table->source_pk[0], src_table_alias), CreateColumnRefExpression(edge_table->source_fk[0], edge_table_alias)); + + edges_per_vertex_select_node->from_table = std::move(left_join); + // edges_per_vertex_select_node->groups = make_uniq(); + + + select_statement->node = std::move(select_node); + result->subquery = std::move(select_statement); + return result; } @@ -392,6 +439,7 @@ void PGQMatchFunction::GenerateShortestPathOperatorCTE( unique_ptr &src_conditions, unique_ptr &dst_conditions, const string& src_table_alias, + const string& edge_table_alias, const string& dst_table_alias) { string pairs_cte_name = "pairs_cte"; auto cte_info = make_uniq(); @@ -551,7 +599,8 @@ unique_ptr PGQMatchFunction::CreatePathFindingFunction( // Do the new path finding operator + CSR creation here GenerateShortestPathOperatorCTE(pg_table, edge_subpath, final_select_node, previous_vertex_subpath->where_clause, next_vertex_subpath->where_clause, - previous_vertex_element->variable_binding, next_vertex_element->variable_binding); + previous_vertex_element->variable_binding, edge_element->variable_binding, + next_vertex_element->variable_binding); } else { if (previous_vertex_subpath) { path_finding_conditions.push_back(std::move(previous_vertex_subpath->where_clause)); diff --git a/src/include/duckpgq/core/functions/table/match.hpp b/src/include/duckpgq/core/functions/table/match.hpp index d6c34dcb..c9211825 100644 --- a/src/include/duckpgq/core/functions/table/match.hpp +++ b/src/include/duckpgq/core/functions/table/match.hpp @@ -108,7 +108,10 @@ struct PGQMatchFunction : public TableFunction { vector> &column_list, unordered_set &named_subpaths); - static unique_ptr GenerateCSROperatorSubquery(shared_ptr &edge_table, const string& src_table_alias, const string& dst_table_alias) + static unique_ptr GenerateCSROperatorSubquery( + shared_ptr &edge_table, + const string& src_table_alias, const string& edge_table_alias, + const string& dst_table_alias); static void CreatePairsCTE(shared_ptr &edge_table, const string& pairs_cte_name, unique_ptr &final_select_node, @@ -121,6 +124,7 @@ struct PGQMatchFunction : public TableFunction { unique_ptr &src_conditions, unique_ptr &dst_conditions, const string& src_table_alias, + const string& edge_table_alias, const string& dst_table_alias); static unique_ptr GenerateShortestPathCTE( From 4a39ec39e4e15545a6898fa9de5be20023d88cc2 Mon Sep 17 00:00:00 2001 From: dtenwolde Date: Thu, 5 Sep 2024 15:56:24 +0200 Subject: [PATCH 230/249] Add missing argument --- src/core/functions/table/match.cpp | 5 +++-- 1 file changed, 3 insertions(+), 2 deletions(-) diff --git a/src/core/functions/table/match.cpp b/src/core/functions/table/match.cpp index 7f9952ec..30b48b9d 100644 --- a/src/core/functions/table/match.cpp +++ b/src/core/functions/table/match.cpp @@ -380,7 +380,8 @@ void PGQMatchFunction::CreatePairsCTE( final_select_node->cte_map.map[pairs_cte_name] = std::move(cte_info); } -unique_ptr PGQMatchFunction::GenerateCSROperatorSubquery(shared_ptr &edge_table, const string& src_table_alias, const string& edge_table_alias, const string& dst_table_alias) { +unique_ptr PGQMatchFunction::GenerateCSROperatorSubquery(shared_ptr &edge_table, + const string& src_table_alias, const string& edge_table_alias, const string& dst_table_alias) { auto result = make_uniq(); auto select_statement = make_uniq(); auto select_node = make_uniq(); @@ -469,7 +470,7 @@ void PGQMatchFunction::GenerateShortestPathOperatorCTE( select_node->from_table = CreateBaseTableRef(pairs_cte_name, "p"); - GenerateCSROperatorSubquery(edge_table, src_table_alias, dst_table_alias); + GenerateCSROperatorSubquery(edge_table, src_table_alias, edge_table_alias, dst_table_alias); select_statement->node = std::move(select_node); From c2aa5c32a0e2697b0e63cd82f9b54889e28b3a18 Mon Sep 17 00:00:00 2001 From: dtenwolde Date: Thu, 5 Sep 2024 15:59:32 +0200 Subject: [PATCH 231/249] Remove left ofter utils folder --- src/utils/compressed_sparse_row.cpp | 0 1 file changed, 0 insertions(+), 0 deletions(-) delete mode 100644 src/utils/compressed_sparse_row.cpp diff --git a/src/utils/compressed_sparse_row.cpp b/src/utils/compressed_sparse_row.cpp deleted file mode 100644 index e69de29b..00000000 From 7802fc4b740c49d82cf65f69f2377857a3fc4daa Mon Sep 17 00:00:00 2001 From: dtenwolde Date: Thu, 5 Sep 2024 16:05:08 +0200 Subject: [PATCH 232/249] Completed csr operator subquery --- src/core/functions/table/match.cpp | 18 ++++++++++++++---- 1 file changed, 14 insertions(+), 4 deletions(-) diff --git a/src/core/functions/table/match.cpp b/src/core/functions/table/match.cpp index 30b48b9d..80a092c2 100644 --- a/src/core/functions/table/match.cpp +++ b/src/core/functions/table/match.cpp @@ -394,7 +394,7 @@ unique_ptr PGQMatchFunction::GenerateCSROperatorSubquery(sha csr_operator_children.emplace_back(CreateColumnRefExpression("rowid", dst_table_alias)); csr_operator_children.emplace_back(CreateColumnRefExpression("rowid", edge_table_alias)); - csr_operator_children.emplace_back(CreateColumnRefExpression("cnt", "count_per_vertex")); + csr_operator_children.emplace_back(CreateColumnRefExpression("cnt", "__count_per_vertex")); select_node->select_list.emplace_back(make_uniq("csr_operator", std::move(csr_operator_children))); auto edge_src_joinref = make_uniq(JoinRefType::REGULAR); @@ -425,9 +425,19 @@ unique_ptr PGQMatchFunction::GenerateCSROperatorSubquery(sha left_join->condition = make_uniq(ExpressionType::COMPARE_EQUAL, CreateColumnRefExpression(edge_table->source_pk[0], src_table_alias), CreateColumnRefExpression(edge_table->source_fk[0], edge_table_alias)); edges_per_vertex_select_node->from_table = std::move(left_join); - // edges_per_vertex_select_node->groups = make_uniq(); - - + edges_per_vertex_select_node->groups.group_expressions.emplace_back(CreateColumnRefExpression("rowid", src_table_alias, src_table_alias + "_rowid")); + GroupingSet grouping_set = {0}; + edges_per_vertex_select_node->groups.grouping_sets.push_back(grouping_set); + + edges_per_vertex_select_statement->node = std::move(edges_per_vertex_select_node); + edges_per_vertex_subquery->subquery = std::move(edges_per_vertex_select_statement); + edges_per_vertex_subquery->alias = "__count_per_vertex"; + auto final_join = make_uniq(JoinRefType::REGULAR); + final_join->left = std::move(edge_dst_joinref); + final_join->right = std::move(edges_per_vertex_subquery); + final_join->condition = make_uniq(ExpressionType::COMPARE_EQUAL, CreateColumnRefExpression(src_table_alias + "_rowid", "__count_per_vertex"), CreateColumnRefExpression("rowid", src_table_alias)); + + select_node->from_table = std::move(final_join); select_statement->node = std::move(select_node); result->subquery = std::move(select_statement); return result; From 1c5a57eb9abad6952bd9375aab7337b4dd288710 Mon Sep 17 00:00:00 2001 From: dtenwolde Date: Thu, 5 Sep 2024 16:08:34 +0200 Subject: [PATCH 233/249] Fix incorrect subqueryRef --- src/core/functions/table/match.cpp | 17 +++++++---------- 1 file changed, 7 insertions(+), 10 deletions(-) diff --git a/src/core/functions/table/match.cpp b/src/core/functions/table/match.cpp index 80a092c2..8378bb36 100644 --- a/src/core/functions/table/match.cpp +++ b/src/core/functions/table/match.cpp @@ -348,7 +348,6 @@ void PGQMatchFunction::CreatePairsCTE( pairs_cte_select_node->select_list.emplace_back(CreateColumnRefExpression("src")); pairs_cte_select_node->select_list.emplace_back(CreateColumnRefExpression("dst")); - auto src_subquery = make_uniq(); auto src_select_statement = make_uniq(); auto src_select_node = make_uniq(); @@ -356,9 +355,8 @@ void PGQMatchFunction::CreatePairsCTE( src_select_node->from_table = CreateBaseTableRef(edge_table->source_reference); src_select_node->where_clause = std::move(src_conditions); src_select_statement->node = std::move(src_select_node); - src_subquery->subquery = std::move(src_select_statement); + auto src_subquery = make_uniq(std::move(src_select_statement)); - auto dst_subquery = make_uniq(); auto dst_select_statement = make_uniq(); auto dst_select_node = make_uniq(); @@ -366,7 +364,7 @@ void PGQMatchFunction::CreatePairsCTE( dst_select_node->from_table = CreateBaseTableRef(edge_table->destination_reference); dst_select_node->where_clause = std::move(dst_conditions); dst_select_statement->node = std::move(dst_select_node); - dst_subquery->subquery = std::move(dst_select_statement); + auto dst_subquery = make_uniq(std::move(dst_select_statement)); auto cross_join = make_uniq(JoinRefType::CROSS); cross_join->left = std::move(src_subquery); @@ -407,7 +405,6 @@ unique_ptr PGQMatchFunction::GenerateCSROperatorSubquery(sha edge_dst_joinref->right = CreateBaseTableRef(edge_table->destination_reference, dst_table_alias); edge_dst_joinref->condition = make_uniq(ExpressionType::COMPARE_EQUAL, CreateColumnRefExpression(edge_table->destination_fk[0], edge_table_alias), CreateColumnRefExpression(edge_table->destination_pk[0], dst_table_alias)); - auto edges_per_vertex_subquery = make_uniq(); auto edges_per_vertex_select_statement = make_uniq(); auto edges_per_vertex_select_node = make_uniq(); @@ -430,7 +427,7 @@ unique_ptr PGQMatchFunction::GenerateCSROperatorSubquery(sha edges_per_vertex_select_node->groups.grouping_sets.push_back(grouping_set); edges_per_vertex_select_statement->node = std::move(edges_per_vertex_select_node); - edges_per_vertex_subquery->subquery = std::move(edges_per_vertex_select_statement); + auto edges_per_vertex_subquery = make_uniq(std::move(edges_per_vertex_select_statement)); edges_per_vertex_subquery->alias = "__count_per_vertex"; auto final_join = make_uniq(JoinRefType::REGULAR); final_join->left = std::move(edge_dst_joinref); @@ -478,11 +475,11 @@ void PGQMatchFunction::GenerateShortestPathOperatorCTE( shortest_path_operator_children.emplace_back( std::move(shortest_path_function)); - select_node->from_table = CreateBaseTableRef(pairs_cte_name, "p"); - - GenerateCSROperatorSubquery(edge_table, src_table_alias, edge_table_alias, dst_table_alias); - + select_node->from_table = CreateBaseTableRef(pairs_cte_name, "__p"); + auto csr_subquery = GenerateCSROperatorSubquery(edge_table, src_table_alias, edge_table_alias, dst_table_alias); + auto between_expression = make_uniq(std::move(csr_subquery), CreateColumnRefExpression("src", "__p"), CreateColumnRefExpression("dst, __p")); + select_node->where_clause = std::move(between_expression); select_statement->node = std::move(select_node); cte_info->query = std::move(select_statement); final_select_node->cte_map.map["shortestpath_cte"] = std::move(cte_info); From 5fdf509a1da79a9b72cb2cbe8b7f7c835452f3ab Mon Sep 17 00:00:00 2001 From: dtenwolde Date: Thu, 5 Sep 2024 16:27:35 +0200 Subject: [PATCH 234/249] Add check for where_clause --- src/core/functions/table/match.cpp | 10 +++++++--- 1 file changed, 7 insertions(+), 3 deletions(-) diff --git a/src/core/functions/table/match.cpp b/src/core/functions/table/match.cpp index 8378bb36..33649cbb 100644 --- a/src/core/functions/table/match.cpp +++ b/src/core/functions/table/match.cpp @@ -605,10 +605,14 @@ unique_ptr PGQMatchFunction::CreatePathFindingFunction( GraphUtils::ToString(edge_element->match_type)); } // Do the new path finding operator + CSR creation here + unique_ptr previous_where_clause = (previous_vertex_subpath && previous_vertex_subpath->where_clause) ? previous_vertex_subpath->where_clause->Copy() : nullptr; + unique_ptr next_where_clause = (next_vertex_subpath && next_vertex_subpath->where_clause) ? next_vertex_subpath->where_clause->Copy() : nullptr; + GenerateShortestPathOperatorCTE(pg_table, edge_subpath, final_select_node, - previous_vertex_subpath->where_clause, next_vertex_subpath->where_clause, - previous_vertex_element->variable_binding, edge_element->variable_binding, - next_vertex_element->variable_binding); + previous_where_clause, next_where_clause, + previous_vertex_element->variable_binding, edge_element->variable_binding, + next_vertex_element->variable_binding); + } else { if (previous_vertex_subpath) { path_finding_conditions.push_back(std::move(previous_vertex_subpath->where_clause)); From a576e60fa9997b2054916c42848a0039aea24e46 Mon Sep 17 00:00:00 2001 From: dtenwolde Date: Thu, 5 Sep 2024 16:32:58 +0200 Subject: [PATCH 235/249] Remove print and add src and dst column to test --- src/core/operator/physical_path_finding_operator.cpp | 1 - test/sql/path_finding/parallel_path_finding.test | 7 ++++--- 2 files changed, 4 insertions(+), 4 deletions(-) diff --git a/src/core/operator/physical_path_finding_operator.cpp b/src/core/operator/physical_path_finding_operator.cpp index 8a4584b6..8bdeaf32 100644 --- a/src/core/operator/physical_path_finding_operator.cpp +++ b/src/core/operator/physical_path_finding_operator.cpp @@ -370,7 +370,6 @@ void PhysicalPathFinding::ScheduleBFSEvent(Pipeline &pipeline, Event &event, } string PhysicalPathFinding::ParamsToString() const { - std::cout << "ParamsToString" << std::endl; auto result = mode; result += "\n[INFOSEPARATOR]\n"; result += StringUtil::Format("EC: %llu", estimated_cardinality); diff --git a/test/sql/path_finding/parallel_path_finding.test b/test/sql/path_finding/parallel_path_finding.test index 4607a7f7..4fa5b53b 100644 --- a/test/sql/path_finding/parallel_path_finding.test +++ b/test/sql/path_finding/parallel_path_finding.test @@ -35,9 +35,10 @@ EDGE TABLES (know SOURCE KEY ( src ) REFERENCES Student ( id ) query III -FROM GRAPH_TABLE (g MATCH p = ANY SHORTEST (src:Student WHERE src.id = 10)-[k:know]-> *(dst:Student) - COLUMNS (element_id(p) AS path)); + COLUMNS (element_id(p) AS path, src.id, dst.id)); ---- - +[0] 10 10 +[0, 1, 2] 10 30 query III WITH shortestpath_cte as ( @@ -65,7 +66,7 @@ WHERE p.src between (SELECT CSR_OPERATOR( # MATCH ANY SHORTEST p = (a:Person)-[k2:knows]->(src:Person WHERE src.id = 10)-[k:knows]->*(dst:Person) # COLUMNS (element_id(p) AS path) -query IIII +query IIIIII WITH pairs as ( SELECT src, dst FROM (SELECT a.rowid AS src FROM Student a where id = 10), From 287ae40883a3379f047fb7573570b257f6e4ed92 Mon Sep 17 00:00:00 2001 From: dtenwolde Date: Thu, 5 Sep 2024 16:44:41 +0200 Subject: [PATCH 236/249] Add slightly more complex pattern --- .../path_finding/parallel_path_finding.test | 31 +++++++++++++++++++ 1 file changed, 31 insertions(+) diff --git a/test/sql/path_finding/parallel_path_finding.test b/test/sql/path_finding/parallel_path_finding.test index 4fa5b53b..0e319a38 100644 --- a/test/sql/path_finding/parallel_path_finding.test +++ b/test/sql/path_finding/parallel_path_finding.test @@ -40,6 +40,37 @@ query III [0] 10 10 [0, 1, 2] 10 30 +query III +-FROM GRAPH_TABLE (g + MATCH p = ANY SHORTEST (src:Student)-[k:know]-> *(dst:Student where dst.id = 30) + COLUMNS (element_id(p) AS path, src.id, dst.id)); +---- +[0, 1, 2] 10 30 +[1, 3, 0, 1, 2] 20 30 +[2] 30 30 +[3, 0, 1, 3, 0, 1, 2] 40 30 + +query III +-FROM GRAPH_TABLE (g + MATCH p = ANY SHORTEST (src:Student)-[k:know]-> *(dst:Student where dst.id = 30) + COLUMNS (path_length(p) AS path, src.id, dst.id)); +---- +1 10 30 +2 20 30 +0 30 30 +3 40 30 + +query IIIII +-FROM GRAPH_TABLE (g + MATCH p = ANY SHORTEST (a:Student)-[k2:know]->(src:Student)-[k:know]-> *(dst:Student where dst.id = 30) + COLUMNS (a.id, element_id(p), path_length(p) AS path, src.id, dst.id)); +---- +40 [3, 0, 1, 3, 0, 1, 2] 3 20 30 +10 [0, 1, 2] 1 30 30 +10 [0, 2, 0, 1, 2] 2 10 30 +20 [1, 3, 0, 1, 2] 2 10 30 +30 [2, 4, 0, 1, 2] 2 10 30 + query III WITH shortestpath_cte as ( SELECT src, dst, shortestpathoperator(src, dst, 'pair') as path From 18d6b6699dda5e4b22ada9a296e50a43feee285b Mon Sep 17 00:00:00 2001 From: dtenwolde Date: Thu, 5 Sep 2024 16:44:55 +0200 Subject: [PATCH 237/249] Fix case insensitive issue --- src/core/functions/table/match.cpp | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/src/core/functions/table/match.cpp b/src/core/functions/table/match.cpp index 33649cbb..1bb87d57 100644 --- a/src/core/functions/table/match.cpp +++ b/src/core/functions/table/match.cpp @@ -92,12 +92,12 @@ void PGQMatchFunction::CheckInheritance( void PGQMatchFunction::CheckEdgeTableConstraints( const string &src_reference, const string &dst_reference, const shared_ptr &edge_table) { - if (src_reference != edge_table->source_reference) { + if (StringUtil::Lower(src_reference) != StringUtil::Lower(edge_table->source_reference)) { throw BinderException("Label %s is not registered as a source reference " "for edge pattern of table %s", src_reference, edge_table->table_name); } - if (dst_reference != edge_table->destination_reference) { + if (StringUtil::Lower(dst_reference) != StringUtil::Lower(edge_table->destination_reference)) { throw BinderException("Label %s is not registered as a destination " "reference for edge pattern of table %s", src_reference, edge_table->table_name); From 3f6ae184fc07da1d4368a71d9ef99ac2743b95d1 Mon Sep 17 00:00:00 2001 From: dtenwolde Date: Thu, 5 Sep 2024 16:50:41 +0200 Subject: [PATCH 238/249] Include functional --- src/core/utils/duckpgq_barrier.cpp | 1 - src/include/duckpgq/core/utils/duckpgq_barrier.hpp | 2 +- 2 files changed, 1 insertion(+), 2 deletions(-) diff --git a/src/core/utils/duckpgq_barrier.cpp b/src/core/utils/duckpgq_barrier.cpp index e4fd708a..058cf0ba 100644 --- a/src/core/utils/duckpgq_barrier.cpp +++ b/src/core/utils/duckpgq_barrier.cpp @@ -1,6 +1,5 @@ #include "duckpgq/core/utils/duckpgq_barrier.hpp" - #include namespace duckpgq { diff --git a/src/include/duckpgq/core/utils/duckpgq_barrier.hpp b/src/include/duckpgq/core/utils/duckpgq_barrier.hpp index 331aa445..9363aebd 100644 --- a/src/include/duckpgq/core/utils/duckpgq_barrier.hpp +++ b/src/include/duckpgq/core/utils/duckpgq_barrier.hpp @@ -8,7 +8,7 @@ #pragma once - +#include #include "mutex" namespace duckpgq { From 8bcad5e574974329e0c12e69b1d4bd98abe262c3 Mon Sep 17 00:00:00 2001 From: dtenwolde Date: Thu, 5 Sep 2024 17:00:02 +0200 Subject: [PATCH 239/249] Include condition_variables and atomic --- src/include/duckpgq/core/utils/duckpgq_barrier.hpp | 2 ++ 1 file changed, 2 insertions(+) diff --git a/src/include/duckpgq/core/utils/duckpgq_barrier.hpp b/src/include/duckpgq/core/utils/duckpgq_barrier.hpp index 9363aebd..84e26587 100644 --- a/src/include/duckpgq/core/utils/duckpgq_barrier.hpp +++ b/src/include/duckpgq/core/utils/duckpgq_barrier.hpp @@ -9,6 +9,8 @@ #pragma once #include +#include +#include #include "mutex" namespace duckpgq { From 41484d5a8279075951a4c60514cb4863b015a167 Mon Sep 17 00:00:00 2001 From: dtenwolde Date: Mon, 9 Sep 2024 11:57:05 +0200 Subject: [PATCH 240/249] Adding debug print --- src/core/optimizer/path_finding_optimizer_rule.cpp | 3 +++ 1 file changed, 3 insertions(+) diff --git a/src/core/optimizer/path_finding_optimizer_rule.cpp b/src/core/optimizer/path_finding_optimizer_rule.cpp index 270d4778..7f43834d 100644 --- a/src/core/optimizer/path_finding_optimizer_rule.cpp +++ b/src/core/optimizer/path_finding_optimizer_rule.cpp @@ -185,8 +185,11 @@ bool DuckpgqOptimizerExtension::InsertPathFindingOperator( void DuckpgqOptimizerExtension::DuckpgqOptimizeFunction( OptimizerExtensionInput &input, unique_ptr &plan) { if (!GetPathFindingOption(input.context)) { + std::cout << "Disabled path finding operator, skipping optimizer rule" << std::endl; return; } + std::cout << "Enabled path finding operator, running optimizer rule" << std::endl; + InsertPathFindingOperator(*plan, input.context); } From de3bd84678ddc6d3c66bd6000a91af68501d2cda Mon Sep 17 00:00:00 2001 From: dtenwolde Date: Mon, 9 Sep 2024 12:05:03 +0200 Subject: [PATCH 241/249] Add more debug print --- src/core/optimizer/path_finding_optimizer_rule.cpp | 1 + 1 file changed, 1 insertion(+) diff --git a/src/core/optimizer/path_finding_optimizer_rule.cpp b/src/core/optimizer/path_finding_optimizer_rule.cpp index 7f43834d..d95f6aa5 100644 --- a/src/core/optimizer/path_finding_optimizer_rule.cpp +++ b/src/core/optimizer/path_finding_optimizer_rule.cpp @@ -172,6 +172,7 @@ bool DuckpgqOptimizerExtension::InsertPathFindingOperator( path_finding_children, path_finding_expressions, mode, op_proj.table_index, offsets); op.children.clear(); op.children.push_back(std::move(path_finding_operator)); + std::cout << "Inserted path-finding operator" << std::endl; return true; // We have found the path-finding operator, no need to continue } for (auto &child : op.children) { From f2789db7cc47a9e684010b135fe1b1ea90bc92c1 Mon Sep 17 00:00:00 2001 From: dtenwolde Date: Tue, 10 Sep 2024 13:54:22 +0200 Subject: [PATCH 242/249] Making physical operator compatible with v1.1.0 changes. Passing operator to scheduler --- src/core/functions/table/match.cpp | 2 +- .../event/csr_edge_creation_event.cpp | 6 ++--- .../operator/event/iterative_length_event.cpp | 6 ++--- .../operator/event/shortest_path_event.cpp | 6 ++--- .../logical_path_finding_operator.cpp | 12 ++++++---- .../physical_path_finding_operator.cpp | 23 +++++++------------ .../operator/task/csr_edge_creation_task.cpp | 5 ++-- .../operator/task/iterative_length_task.cpp | 5 ++-- src/core/operator/task/shortest_path_task.cpp | 4 ++-- .../optimizer/path_finding_optimizer_rule.cpp | 2 +- .../event/csr_edge_creation_event.hpp | 3 ++- .../operator/event/iterative_length_event.hpp | 3 ++- .../operator/event/shortest_path_event.hpp | 3 ++- .../logical_path_finding_operator.hpp | 2 +- .../physical_path_finding_operator.hpp | 6 ++--- .../operator/task/csr_edge_creation_task.hpp | 3 +-- .../operator/task/iterative_length_task.hpp | 3 ++- .../core/operator/task/shortest_path_task.hpp | 2 +- .../duckpgq/core/utils/duckpgq_utils.hpp | 3 --- 19 files changed, 46 insertions(+), 53 deletions(-) diff --git a/src/core/functions/table/match.cpp b/src/core/functions/table/match.cpp index 5881f6cf..e9c4444e 100644 --- a/src/core/functions/table/match.cpp +++ b/src/core/functions/table/match.cpp @@ -611,7 +611,7 @@ unique_ptr PGQMatchFunction::CreatePathFindingFunction( previous_where_clause, next_where_clause, previous_vertex_element->variable_binding, edge_element->variable_binding, next_vertex_element->variable_binding); - + std::cout << "Inserted shortest path operator cte" << std::endl; } else { if (previous_vertex_subpath) { path_finding_conditions.push_back(std::move(previous_vertex_subpath->where_clause)); diff --git a/src/core/operator/event/csr_edge_creation_event.cpp b/src/core/operator/event/csr_edge_creation_event.cpp index 4ea2180e..155cfd25 100644 --- a/src/core/operator/event/csr_edge_creation_event.cpp +++ b/src/core/operator/event/csr_edge_creation_event.cpp @@ -4,8 +4,8 @@ namespace duckpgq { namespace core { -CSREdgeCreationEvent::CSREdgeCreationEvent(PathFindingGlobalState &gstate_p, Pipeline &pipeline_p) - : BasePipelineEvent(pipeline_p), gstate(gstate_p) {} +CSREdgeCreationEvent::CSREdgeCreationEvent(PathFindingGlobalState &gstate_p, Pipeline &pipeline_p, const PhysicalOperator &op_p) + : BasePipelineEvent(pipeline_p), gstate(gstate_p), op(op_p) {} void CSREdgeCreationEvent::Schedule() { auto &context = pipeline->GetClientContext(); @@ -19,7 +19,7 @@ void CSREdgeCreationEvent::Schedule() { vector> tasks; for (idx_t tnum = 0; tnum < num_threads; tnum++) { tasks.push_back(make_uniq(shared_from_this(), - context, gstate)); + context, gstate, op)); } SetTasks(std::move(tasks)); } diff --git a/src/core/operator/event/iterative_length_event.cpp b/src/core/operator/event/iterative_length_event.cpp index 9efa494e..1df8777b 100644 --- a/src/core/operator/event/iterative_length_event.cpp +++ b/src/core/operator/event/iterative_length_event.cpp @@ -5,8 +5,8 @@ namespace duckpgq { namespace core { -ParallelIterativeEvent::ParallelIterativeEvent(PathFindingGlobalState &gstate_p, Pipeline &pipeline_p) - : BasePipelineEvent(pipeline_p), gstate(gstate_p) {} +ParallelIterativeEvent::ParallelIterativeEvent(PathFindingGlobalState &gstate_p, Pipeline &pipeline_p, const PhysicalOperator &op_p) + : BasePipelineEvent(pipeline_p), gstate(gstate_p), op(op_p) {} void ParallelIterativeEvent::Schedule() { @@ -18,7 +18,7 @@ void ParallelIterativeEvent::Schedule() { for (idx_t tnum = 0; tnum < threads_to_schedule; tnum++) { bfs_tasks.push_back(make_uniq( - shared_from_this(), context, gstate, tnum)); + shared_from_this(), context, gstate, tnum, op)); } SetTasks(std::move(bfs_tasks)); } diff --git a/src/core/operator/event/shortest_path_event.cpp b/src/core/operator/event/shortest_path_event.cpp index 75212e4f..bbffc1b7 100644 --- a/src/core/operator/event/shortest_path_event.cpp +++ b/src/core/operator/event/shortest_path_event.cpp @@ -7,8 +7,8 @@ namespace duckpgq { namespace core { ParallelShortestPathEvent::ParallelShortestPathEvent(PathFindingGlobalState &gstate_p, - Pipeline &pipeline_p) - : BasePipelineEvent(pipeline_p), gstate(gstate_p) { + Pipeline &pipeline_p, const PhysicalOperator &op_p) + : BasePipelineEvent(pipeline_p), gstate(gstate_p), op(op_p) { } @@ -20,7 +20,7 @@ void ParallelShortestPathEvent::Schedule() { size_t threads_to_schedule = std::min(bfs_state->num_threads, (idx_t)bfs_state->global_task_queue.size()); for (idx_t tnum = 0; tnum < threads_to_schedule; tnum++) { bfs_tasks.push_back(make_uniq( - shared_from_this(), context, gstate, tnum)); + shared_from_this(), context, gstate, tnum, op)); } SetTasks(std::move(bfs_tasks)); } diff --git a/src/core/operator/logical_path_finding_operator.cpp b/src/core/operator/logical_path_finding_operator.cpp index a7bdb689..d6f1e3e8 100644 --- a/src/core/operator/logical_path_finding_operator.cpp +++ b/src/core/operator/logical_path_finding_operator.cpp @@ -36,13 +36,15 @@ void LogicalPathFindingOperator::ResolveTypes() { } } -string LogicalPathFindingOperator::ParamsToString() const { - string extra_info; +InsertionOrderPreservingMap LogicalPathFindingOperator::ParamsToString() const { + InsertionOrderPreservingMap result; + string expression_info; for (auto &expr : expressions) { - extra_info += "\n"; - extra_info += expr->ToString(); + expression_info += "\n"; + expression_info += expr->GetName(); } - return extra_info; + result["Expressions"] = expression_info; + return result; } diff --git a/src/core/operator/physical_path_finding_operator.cpp b/src/core/operator/physical_path_finding_operator.cpp index 8bdeaf32..6376ffba 100644 --- a/src/core/operator/physical_path_finding_operator.cpp +++ b/src/core/operator/physical_path_finding_operator.cpp @@ -222,13 +222,6 @@ PathFindingGlobalState::PathFindingGlobalState(ClientContext &context, mode = op.mode; } -PathFindingGlobalState::PathFindingGlobalState(PathFindingGlobalState &prev) - : GlobalSinkState(prev), global_tasks(std::move(prev.global_tasks)), - global_inputs(std::move(prev.global_inputs)), - global_csr(std::move(prev.global_csr)), - global_bfs_state(std::move(prev.global_bfs_state)), - child(prev.child + 1), mode(prev.mode) {} - void PathFindingGlobalState::Sink(DataChunk &input, PathFindingLocalState &lstate) { lstate.Sink(input, *global_csr, child); } @@ -284,7 +277,7 @@ PhysicalPathFinding::Finalize(Pipeline &pipeline, Event &event, auto &global_tasks = gstate.global_tasks; if (gstate.child == 0) { - auto csr_event = make_shared_ptr(gstate, pipeline); + auto csr_event = make_shared_ptr(gstate, pipeline, *this); event.InsertEvent(std::move(csr_event)); } else if (gstate.child == 1 && global_tasks->Count() > 0) { auto all_pairs = make_shared_ptr(); @@ -321,7 +314,7 @@ PhysicalPathFinding::Finalize(Pipeline &pipeline, Event &event, } void PhysicalPathFinding::ScheduleBFSEvent(Pipeline &pipeline, Event &event, - GlobalSinkState &state) { + GlobalSinkState &state) const { auto &gstate = state.Cast(); auto &bfs_state = gstate.global_bfs_state; @@ -359,20 +352,20 @@ void PhysicalPathFinding::ScheduleBFSEvent(Pipeline &pipeline, Event &event, if (gstate.mode == "iterativelength") { auto bfs_event = - make_shared_ptr(gstate, pipeline); + make_shared_ptr(gstate, pipeline, *this); event.InsertEvent(std::move(bfs_event)); } else if (gstate.mode == "shortestpath") { auto bfs_event = - make_shared_ptr(gstate, pipeline); + make_shared_ptr(gstate, pipeline, *this); event.InsertEvent(std::move(bfs_event)); } } } -string PhysicalPathFinding::ParamsToString() const { - auto result = mode; - result += "\n[INFOSEPARATOR]\n"; - result += StringUtil::Format("EC: %llu", estimated_cardinality); +InsertionOrderPreservingMap PhysicalPathFinding::ParamsToString() const { + InsertionOrderPreservingMap result; + result["Mode"] = mode; + SetEstimatedCardinality(result, estimated_cardinality); return result; } diff --git a/src/core/operator/task/csr_edge_creation_task.cpp b/src/core/operator/task/csr_edge_creation_task.cpp index b9333a41..533d1f17 100644 --- a/src/core/operator/task/csr_edge_creation_task.cpp +++ b/src/core/operator/task/csr_edge_creation_task.cpp @@ -4,9 +4,8 @@ namespace duckpgq { namespace core { PhysicalCSREdgeCreationTask::PhysicalCSREdgeCreationTask(shared_ptr event_p, ClientContext &context, - PathFindingGlobalState &state) - : ExecutorTask(context, std::move(event_p)), context(context), - state(state) {} + PathFindingGlobalState &state, const PhysicalOperator &op_p) + : ExecutorTask(context, std::move(event_p), op_p), state(state) {} TaskExecutionResult PhysicalCSREdgeCreationTask::ExecuteTask(TaskExecutionMode mode) { auto &global_inputs = state.global_inputs; diff --git a/src/core/operator/task/iterative_length_task.cpp b/src/core/operator/task/iterative_length_task.cpp index 2385651e..a5227c3d 100644 --- a/src/core/operator/task/iterative_length_task.cpp +++ b/src/core/operator/task/iterative_length_task.cpp @@ -5,8 +5,9 @@ namespace duckpgq { namespace core { PhysicalIterativeTask::PhysicalIterativeTask(shared_ptr event_p, ClientContext &context, - PathFindingGlobalState &state, idx_t worker_id) - : ExecutorTask(context, std::move(event_p)), context(context), + PathFindingGlobalState &state, idx_t worker_id, + const PhysicalOperator &op_p) + : ExecutorTask(context, std::move(event_p), op_p), context(context), state(state), worker_id(worker_id) {} bool PhysicalIterativeTask::SetTaskRange() { diff --git a/src/core/operator/task/shortest_path_task.cpp b/src/core/operator/task/shortest_path_task.cpp index b3d5fe62..9395c77c 100644 --- a/src/core/operator/task/shortest_path_task.cpp +++ b/src/core/operator/task/shortest_path_task.cpp @@ -5,8 +5,8 @@ namespace duckpgq { namespace core { PhysicalShortestPathTask::PhysicalShortestPathTask(shared_ptr event_p, ClientContext &context, - PathFindingGlobalState &state, idx_t worker_id) - : ExecutorTask(context, std::move(event_p)), context(context), + PathFindingGlobalState &state, idx_t worker_id, const PhysicalOperator &op_p) + : ExecutorTask(context, std::move(event_p), op_p), context(context), state(state), worker_id(worker_id) { left = right = UINT64_MAX; // NOLINT } diff --git a/src/core/optimizer/path_finding_optimizer_rule.cpp b/src/core/optimizer/path_finding_optimizer_rule.cpp index d95f6aa5..e69fa504 100644 --- a/src/core/optimizer/path_finding_optimizer_rule.cpp +++ b/src/core/optimizer/path_finding_optimizer_rule.cpp @@ -152,7 +152,7 @@ bool DuckpgqOptimizerExtension::InsertPathFindingOperator( for (const auto &binding : get_empty_result.bindings) { column_ids_vector.push_back(binding.column_index); } - logical_get->column_ids = std::move(column_ids_vector); + logical_get->SetColumnIds(std::move(column_ids_vector)); path_finding_children.push_back(std::move(logical_get)); } else if (left_child->type == LogicalOperatorType::LOGICAL_PROJECTION) { diff --git a/src/include/duckpgq/core/operator/event/csr_edge_creation_event.hpp b/src/include/duckpgq/core/operator/event/csr_edge_creation_event.hpp index 5bebaa0f..c2e48d97 100644 --- a/src/include/duckpgq/core/operator/event/csr_edge_creation_event.hpp +++ b/src/include/duckpgq/core/operator/event/csr_edge_creation_event.hpp @@ -9,8 +9,9 @@ namespace duckpgq { namespace core { class CSREdgeCreationEvent : public BasePipelineEvent { public: - CSREdgeCreationEvent(PathFindingGlobalState &gstate_p, Pipeline &pipeline_p); + CSREdgeCreationEvent(PathFindingGlobalState &gstate_p, Pipeline &pipeline_p, const PhysicalOperator &op_p); PathFindingGlobalState &gstate; + const PhysicalOperator &op; void Schedule() override; void FinishEvent() override; diff --git a/src/include/duckpgq/core/operator/event/iterative_length_event.hpp b/src/include/duckpgq/core/operator/event/iterative_length_event.hpp index 5e7f7275..14a6c078 100644 --- a/src/include/duckpgq/core/operator/event/iterative_length_event.hpp +++ b/src/include/duckpgq/core/operator/event/iterative_length_event.hpp @@ -11,9 +11,10 @@ namespace core { class ParallelIterativeEvent : public BasePipelineEvent { public: - ParallelIterativeEvent(PathFindingGlobalState &gstate_p, Pipeline &pipeline_p); + ParallelIterativeEvent(PathFindingGlobalState &gstate_p, Pipeline &pipeline_p, const PhysicalOperator &op_p); PathFindingGlobalState &gstate; + const PhysicalOperator &op; void Schedule() override; diff --git a/src/include/duckpgq/core/operator/event/shortest_path_event.hpp b/src/include/duckpgq/core/operator/event/shortest_path_event.hpp index f86884e1..508fbaa6 100644 --- a/src/include/duckpgq/core/operator/event/shortest_path_event.hpp +++ b/src/include/duckpgq/core/operator/event/shortest_path_event.hpp @@ -17,13 +17,14 @@ class PathFindingGlobalState; class ParallelShortestPathEvent : public BasePipelineEvent { public: - explicit ParallelShortestPathEvent(PathFindingGlobalState &gstate_p, Pipeline &pipeline_p); + explicit ParallelShortestPathEvent(PathFindingGlobalState &gstate_p, Pipeline &pipeline_p, const PhysicalOperator& op_p); void Schedule() override; void FinishEvent() override; private: PathFindingGlobalState &gstate; + const PhysicalOperator &op; }; } // namespace core diff --git a/src/include/duckpgq/core/operator/logical_path_finding_operator.hpp b/src/include/duckpgq/core/operator/logical_path_finding_operator.hpp index 06517107..bfced962 100644 --- a/src/include/duckpgq/core/operator/logical_path_finding_operator.hpp +++ b/src/include/duckpgq/core/operator/logical_path_finding_operator.hpp @@ -31,7 +31,7 @@ class LogicalPathFindingOperator : public LogicalExtensionOperator { void ResolveTypes() override; - string ParamsToString() const override; + InsertionOrderPreservingMap ParamsToString() const override; public: string mode; diff --git a/src/include/duckpgq/core/operator/physical_path_finding_operator.hpp b/src/include/duckpgq/core/operator/physical_path_finding_operator.hpp index 458fa05a..74e23d47 100644 --- a/src/include/duckpgq/core/operator/physical_path_finding_operator.hpp +++ b/src/include/duckpgq/core/operator/physical_path_finding_operator.hpp @@ -117,7 +117,7 @@ class PhysicalPathFinding : public PhysicalComparisonJoin { public: - string ParamsToString() const override; + InsertionOrderPreservingMap ParamsToString() const override; // CachingOperator Interface OperatorResultType ExecuteInternal(ExecutionContext &context, @@ -158,7 +158,7 @@ class PhysicalPathFinding : public PhysicalComparisonJoin { void BuildPipelines(Pipeline ¤t, MetaPipeline &meta_pipeline) override; // //! Schedules tasks to calculate the next iteration of the path-finding - static void ScheduleBFSEvent(Pipeline &pipeline, Event &event, GlobalSinkState &state); + void ScheduleBFSEvent(Pipeline &pipeline, Event &event, GlobalSinkState &state) const; }; //===--------------------------------------------------------------------===// @@ -243,8 +243,6 @@ class PathFindingGlobalState : public GlobalSinkState { PathFindingGlobalState(ClientContext &context, const PhysicalPathFinding &op); - PathFindingGlobalState(PathFindingGlobalState &prev); - void Sink(DataChunk &input, PathFindingLocalState &lstate); // pairs is a 2-column table with src and dst diff --git a/src/include/duckpgq/core/operator/task/csr_edge_creation_task.hpp b/src/include/duckpgq/core/operator/task/csr_edge_creation_task.hpp index b0d96ba8..60ca6f6e 100644 --- a/src/include/duckpgq/core/operator/task/csr_edge_creation_task.hpp +++ b/src/include/duckpgq/core/operator/task/csr_edge_creation_task.hpp @@ -10,12 +10,11 @@ namespace core { class PhysicalCSREdgeCreationTask : public ExecutorTask { public: PhysicalCSREdgeCreationTask(shared_ptr event_p, ClientContext &context, - PathFindingGlobalState &state); + PathFindingGlobalState &state, const PhysicalOperator &op_p); TaskExecutionResult ExecuteTask(TaskExecutionMode mode) override; private: - ClientContext &context; PathFindingGlobalState &state; }; diff --git a/src/include/duckpgq/core/operator/task/iterative_length_task.hpp b/src/include/duckpgq/core/operator/task/iterative_length_task.hpp index 8d1995be..7aaec282 100644 --- a/src/include/duckpgq/core/operator/task/iterative_length_task.hpp +++ b/src/include/duckpgq/core/operator/task/iterative_length_task.hpp @@ -10,7 +10,8 @@ namespace core { class PhysicalIterativeTask : public ExecutorTask { public: PhysicalIterativeTask(shared_ptr event_p, ClientContext &context, - PathFindingGlobalState &state, idx_t worker_id); + PathFindingGlobalState &state, idx_t worker_id, + const PhysicalOperator &op_p); TaskExecutionResult ExecuteTask(TaskExecutionMode mode) override; private: diff --git a/src/include/duckpgq/core/operator/task/shortest_path_task.hpp b/src/include/duckpgq/core/operator/task/shortest_path_task.hpp index f1ff6747..e6ff6a3d 100644 --- a/src/include/duckpgq/core/operator/task/shortest_path_task.hpp +++ b/src/include/duckpgq/core/operator/task/shortest_path_task.hpp @@ -9,7 +9,7 @@ namespace core { class PhysicalShortestPathTask : public ExecutorTask { public: PhysicalShortestPathTask(shared_ptr event_p, ClientContext &context, - PathFindingGlobalState &state, idx_t worker_id); + PathFindingGlobalState &state, idx_t worker_id, const PhysicalOperator &op_p); TaskExecutionResult ExecuteTask(TaskExecutionMode mode) override; diff --git a/src/include/duckpgq/core/utils/duckpgq_utils.hpp b/src/include/duckpgq/core/utils/duckpgq_utils.hpp index 33e77da6..99237b58 100644 --- a/src/include/duckpgq/core/utils/duckpgq_utils.hpp +++ b/src/include/duckpgq/core/utils/duckpgq_utils.hpp @@ -10,9 +10,6 @@ namespace core { #define LANE_LIMIT 512 #define VISIT_SIZE_DIVISOR 2 -// Function to get DuckPGQState from ClientContext -DuckPGQState *GetDuckPGQState(ClientContext &context); - class GraphUtils { public: static std::string ToString(PGQMatchType matchType) { From 3a25303d4fc760754787870057b91649a5d257a1 Mon Sep 17 00:00:00 2001 From: dtenwolde Date: Tue, 10 Sep 2024 13:59:48 +0200 Subject: [PATCH 243/249] Bump ci tools --- extension-ci-tools | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/extension-ci-tools b/extension-ci-tools index fdf2be8c..d0d6ef1c 160000 --- a/extension-ci-tools +++ b/extension-ci-tools @@ -1 +1 @@ -Subproject commit fdf2be8c08fafa209a403b1867e9d386a76ca5f6 +Subproject commit d0d6ef1cba7cefd81b6ea8f18b2f5d4e50aa8a6b From 12dff28f949fc45925b14c52b314846814940ef1 Mon Sep 17 00:00:00 2001 From: dtenwolde Date: Tue, 10 Sep 2024 14:15:11 +0200 Subject: [PATCH 244/249] Update type to PhysicalPathFinding to schedule new iterations --- src/core/operator/event/iterative_length_event.cpp | 4 ++-- src/core/operator/event/shortest_path_event.cpp | 4 ++-- .../duckpgq/core/operator/event/iterative_length_event.hpp | 4 ++-- .../duckpgq/core/operator/event/shortest_path_event.hpp | 6 ++++-- 4 files changed, 10 insertions(+), 8 deletions(-) diff --git a/src/core/operator/event/iterative_length_event.cpp b/src/core/operator/event/iterative_length_event.cpp index 1df8777b..6ea6ccfd 100644 --- a/src/core/operator/event/iterative_length_event.cpp +++ b/src/core/operator/event/iterative_length_event.cpp @@ -5,7 +5,7 @@ namespace duckpgq { namespace core { -ParallelIterativeEvent::ParallelIterativeEvent(PathFindingGlobalState &gstate_p, Pipeline &pipeline_p, const PhysicalOperator &op_p) +ParallelIterativeEvent::ParallelIterativeEvent(PathFindingGlobalState &gstate_p, Pipeline &pipeline_p, const PhysicalPathFinding &op_p) : BasePipelineEvent(pipeline_p), gstate(gstate_p), op(op_p) {} @@ -28,7 +28,7 @@ void ParallelIterativeEvent::FinishEvent() { // if remaining pairs, schedule the BFS for the next batch if (bfs_state->started_searches < gstate.global_tasks->Count()) { - PhysicalPathFinding::ScheduleBFSEvent(*pipeline, *this, gstate); + op.ScheduleBFSEvent(*pipeline, *this, gstate); } } diff --git a/src/core/operator/event/shortest_path_event.cpp b/src/core/operator/event/shortest_path_event.cpp index bbffc1b7..d980d9d2 100644 --- a/src/core/operator/event/shortest_path_event.cpp +++ b/src/core/operator/event/shortest_path_event.cpp @@ -7,7 +7,7 @@ namespace duckpgq { namespace core { ParallelShortestPathEvent::ParallelShortestPathEvent(PathFindingGlobalState &gstate_p, - Pipeline &pipeline_p, const PhysicalOperator &op_p) + Pipeline &pipeline_p, const PhysicalPathFinding &op_p) : BasePipelineEvent(pipeline_p), gstate(gstate_p), op(op_p) { } @@ -30,7 +30,7 @@ void ParallelShortestPathEvent::FinishEvent() { // if remaining pairs, schedule the BFS for the next batch if (bfs_state->started_searches < gstate.global_tasks->Count()) { - PhysicalPathFinding::ScheduleBFSEvent(*pipeline, *this, gstate); + op.ScheduleBFSEvent(*pipeline, *this, gstate); } }; diff --git a/src/include/duckpgq/core/operator/event/iterative_length_event.hpp b/src/include/duckpgq/core/operator/event/iterative_length_event.hpp index 14a6c078..c48be675 100644 --- a/src/include/duckpgq/core/operator/event/iterative_length_event.hpp +++ b/src/include/duckpgq/core/operator/event/iterative_length_event.hpp @@ -11,10 +11,10 @@ namespace core { class ParallelIterativeEvent : public BasePipelineEvent { public: - ParallelIterativeEvent(PathFindingGlobalState &gstate_p, Pipeline &pipeline_p, const PhysicalOperator &op_p); + ParallelIterativeEvent(PathFindingGlobalState &gstate_p, Pipeline &pipeline_p, const PhysicalPathFinding &op_p); PathFindingGlobalState &gstate; - const PhysicalOperator &op; + const PhysicalPathFinding &op; void Schedule() override; diff --git a/src/include/duckpgq/core/operator/event/shortest_path_event.hpp b/src/include/duckpgq/core/operator/event/shortest_path_event.hpp index 508fbaa6..97b3fafc 100644 --- a/src/include/duckpgq/core/operator/event/shortest_path_event.hpp +++ b/src/include/duckpgq/core/operator/event/shortest_path_event.hpp @@ -11,20 +11,22 @@ #include "duckpgq/common.hpp" #include "duckdb/parallel/base_pipeline_event.hpp" +#include "duckpgq/core/operator/physical_path_finding_operator.hpp" + namespace duckpgq { namespace core { class PathFindingGlobalState; class ParallelShortestPathEvent : public BasePipelineEvent { public: - explicit ParallelShortestPathEvent(PathFindingGlobalState &gstate_p, Pipeline &pipeline_p, const PhysicalOperator& op_p); + explicit ParallelShortestPathEvent(PathFindingGlobalState &gstate_p, Pipeline &pipeline_p, const PhysicalPathFinding& op_p); void Schedule() override; void FinishEvent() override; private: PathFindingGlobalState &gstate; - const PhysicalOperator &op; + const PhysicalPathFinding &op; }; } // namespace core From 0beae90610aedb143472305feeed08c355a4ba1f Mon Sep 17 00:00:00 2001 From: dtenwolde Date: Tue, 10 Sep 2024 14:16:12 +0200 Subject: [PATCH 245/249] Fix result --- test/sql/path_finding/parallel_path_finding.test | 5 +++-- 1 file changed, 3 insertions(+), 2 deletions(-) diff --git a/test/sql/path_finding/parallel_path_finding.test b/test/sql/path_finding/parallel_path_finding.test index 0e319a38..4d1551dc 100644 --- a/test/sql/path_finding/parallel_path_finding.test +++ b/test/sql/path_finding/parallel_path_finding.test @@ -63,13 +63,14 @@ query III query IIIII -FROM GRAPH_TABLE (g MATCH p = ANY SHORTEST (a:Student)-[k2:know]->(src:Student)-[k:know]-> *(dst:Student where dst.id = 30) - COLUMNS (a.id, element_id(p), path_length(p) AS path, src.id, dst.id)); + COLUMNS (a.id as a_id, element_id(p), path_length(p) AS path_length, src.id, dst.id)) + ORDER BY a_id, path_length ---- -40 [3, 0, 1, 3, 0, 1, 2] 3 20 30 10 [0, 1, 2] 1 30 30 10 [0, 2, 0, 1, 2] 2 10 30 20 [1, 3, 0, 1, 2] 2 10 30 30 [2, 4, 0, 1, 2] 2 10 30 +40 [3, 0, 1, 3, 0, 1, 2] 3 20 30 query III WITH shortestpath_cte as ( From 765c0782ebc83ad087bcc15e3b62a4f2fd61ed2b Mon Sep 17 00:00:00 2001 From: dtenwolde Date: Tue, 10 Sep 2024 14:53:45 +0200 Subject: [PATCH 246/249] Format fix --- .../optimizer/path_finding_optimizer_rule.cpp | 4 +- .../path_finding/parallel_path_finding.test | 78 +++++++++---------- 2 files changed, 40 insertions(+), 42 deletions(-) diff --git a/src/core/optimizer/path_finding_optimizer_rule.cpp b/src/core/optimizer/path_finding_optimizer_rule.cpp index e69fa504..e092a868 100644 --- a/src/core/optimizer/path_finding_optimizer_rule.cpp +++ b/src/core/optimizer/path_finding_optimizer_rule.cpp @@ -157,9 +157,7 @@ bool DuckpgqOptimizerExtension::InsertPathFindingOperator( path_finding_children.push_back(std::move(logical_get)); } else if (left_child->type == LogicalOperatorType::LOGICAL_PROJECTION) { path_finding_children.push_back(std::move(left_child)); - } - - else { + } else { throw InternalException("Did not find pairs for path-finding operator. " "The left child was of type " + LogicalOperatorToString(left_child->type)); diff --git a/test/sql/path_finding/parallel_path_finding.test b/test/sql/path_finding/parallel_path_finding.test index 4d1551dc..d560b58c 100644 --- a/test/sql/path_finding/parallel_path_finding.test +++ b/test/sql/path_finding/parallel_path_finding.test @@ -32,45 +32,45 @@ EDGE TABLES (know SOURCE KEY ( src ) REFERENCES Student ( id ) ); -query III --FROM GRAPH_TABLE (g - MATCH p = ANY SHORTEST (src:Student WHERE src.id = 10)-[k:know]-> *(dst:Student) - COLUMNS (element_id(p) AS path, src.id, dst.id)); ----- -[0] 10 10 -[0, 1, 2] 10 30 - -query III --FROM GRAPH_TABLE (g - MATCH p = ANY SHORTEST (src:Student)-[k:know]-> *(dst:Student where dst.id = 30) - COLUMNS (element_id(p) AS path, src.id, dst.id)); ----- -[0, 1, 2] 10 30 -[1, 3, 0, 1, 2] 20 30 -[2] 30 30 -[3, 0, 1, 3, 0, 1, 2] 40 30 - -query III --FROM GRAPH_TABLE (g - MATCH p = ANY SHORTEST (src:Student)-[k:know]-> *(dst:Student where dst.id = 30) - COLUMNS (path_length(p) AS path, src.id, dst.id)); ----- -1 10 30 -2 20 30 -0 30 30 -3 40 30 - -query IIIII --FROM GRAPH_TABLE (g - MATCH p = ANY SHORTEST (a:Student)-[k2:know]->(src:Student)-[k:know]-> *(dst:Student where dst.id = 30) - COLUMNS (a.id as a_id, element_id(p), path_length(p) AS path_length, src.id, dst.id)) - ORDER BY a_id, path_length ----- -10 [0, 1, 2] 1 30 30 -10 [0, 2, 0, 1, 2] 2 10 30 -20 [1, 3, 0, 1, 2] 2 10 30 -30 [2, 4, 0, 1, 2] 2 10 30 -40 [3, 0, 1, 3, 0, 1, 2] 3 20 30 +#query III +#-FROM GRAPH_TABLE (g +# MATCH p = ANY SHORTEST (src:Student WHERE src.id = 10)-[k:know]-> *(dst:Student) +# COLUMNS (element_id(p) AS path, src.id, dst.id)); +#---- +#[0] 10 10 +#[0, 1, 2] 10 30 +# +#query III +#-FROM GRAPH_TABLE (g +# MATCH p = ANY SHORTEST (src:Student)-[k:know]-> *(dst:Student where dst.id = 30) +# COLUMNS (element_id(p) AS path, src.id, dst.id)); +#---- +#[0, 1, 2] 10 30 +#[1, 3, 0, 1, 2] 20 30 +#[2] 30 30 +#[3, 0, 1, 3, 0, 1, 2] 40 30 +# +#query III +#-FROM GRAPH_TABLE (g +# MATCH p = ANY SHORTEST (src:Student)-[k:know]-> *(dst:Student where dst.id = 30) +# COLUMNS (path_length(p) AS path, src.id, dst.id)); +#---- +#1 10 30 +#2 20 30 +#0 30 30 +#3 40 30 +# +#query IIIII +#-FROM GRAPH_TABLE (g +# MATCH p = ANY SHORTEST (a:Student)-[k2:know]->(src:Student)-[k:know]-> *(dst:Student where dst.id = 30) +# COLUMNS (a.id as a_id, element_id(p), path_length(p) AS path_length, src.id, dst.id)) +# ORDER BY a_id, path_length +#---- +#10 [0, 1, 2] 1 30 30 +#10 [0, 2, 0, 1, 2] 2 10 30 +#20 [1, 3, 0, 1, 2] 2 10 30 +#30 [2, 4, 0, 1, 2] 2 10 30 +#40 [3, 0, 1, 3, 0, 1, 2] 3 20 30 query III WITH shortestpath_cte as ( From e4800dbcc6a77afbc6350de94c8598d3372f6a4f Mon Sep 17 00:00:00 2001 From: dtenwolde Date: Thu, 12 Sep 2024 10:15:25 +0200 Subject: [PATCH 247/249] Fix correct index --- src/core/operator/physical_path_finding_operator.cpp | 5 +++-- src/core/operator/task/csr_edge_creation_task.cpp | 4 ++-- 2 files changed, 5 insertions(+), 4 deletions(-) diff --git a/src/core/operator/physical_path_finding_operator.cpp b/src/core/operator/physical_path_finding_operator.cpp index 6376ffba..385f132d 100644 --- a/src/core/operator/physical_path_finding_operator.cpp +++ b/src/core/operator/physical_path_finding_operator.cpp @@ -197,12 +197,12 @@ void PathFindingLocalState::Sink(DataChunk &input, GlobalCompressedSparseRow &gl void PathFindingLocalState::CreateCSRVertex( DataChunk &input, GlobalCompressedSparseRow &global_csr) { if (!global_csr.initialized_v) { - const auto v_size = input.data[8].GetValue(0).GetValue(); + const auto v_size = input.data[7].GetValue(0).GetValue(); global_csr.InitializeVertex(v_size); } auto result = Vector(LogicalTypeId::BIGINT); BinaryExecutor::Execute( - input.data[6], input.data[5], result, input.size(), + input.data[1], input.data[2], result, input.size(), [&](const int64_t src, const int64_t cnt) { int64_t edge_count = 0; global_csr.v[src + 2] = cnt; @@ -247,6 +247,7 @@ SinkResultType PhysicalPathFinding::Sink(ExecutionContext &context, OperatorSinkInput &input) const { auto &gstate = input.global_state.Cast(); auto &lstate = input.local_state.Cast(); + chunk.Print(); gstate.Sink(chunk, lstate); return SinkResultType::NEED_MORE_INPUT; } diff --git a/src/core/operator/task/csr_edge_creation_task.cpp b/src/core/operator/task/csr_edge_creation_task.cpp index 533d1f17..fbe81516 100644 --- a/src/core/operator/task/csr_edge_creation_task.cpp +++ b/src/core/operator/task/csr_edge_creation_task.cpp @@ -23,11 +23,11 @@ TaskExecutionResult PhysicalCSREdgeCreationTask::ExecuteTask(TaskExecutionMode m } } if (!global_csr->initialized_e) { - const auto e_size = input.data[7].GetValue(0).GetValue(); + const auto e_size = input.data[6].GetValue(0).GetValue(); global_csr->InitializeEdge(e_size); } TernaryExecutor::Execute( - input.data[6], input.data[4], input.data[2], result, input.size(), + input.data[3], input.data[5], input.data[10], result, input.size(), [&](int64_t src, int64_t dst, int64_t edge_id) { const auto pos = ++global_csr->v[src + 1]; global_csr->e[static_cast(pos) - 1] = dst; From 6ee3878f148925bc851bf64328bdac387086bfbf Mon Sep 17 00:00:00 2001 From: dtenwolde Date: Thu, 12 Sep 2024 10:15:39 +0200 Subject: [PATCH 248/249] Debug --- .../optimizer/path_finding_optimizer_rule.cpp | 56 ++- .../path_finding/parallel_path_finding.test | 385 +++++++++--------- 2 files changed, 237 insertions(+), 204 deletions(-) diff --git a/src/core/optimizer/path_finding_optimizer_rule.cpp b/src/core/optimizer/path_finding_optimizer_rule.cpp index e092a868..fcd5602b 100644 --- a/src/core/optimizer/path_finding_optimizer_rule.cpp +++ b/src/core/optimizer/path_finding_optimizer_rule.cpp @@ -38,6 +38,7 @@ void ReplaceExpressions(LogicalProjection &op, unique_ptr &function_ for (size_t offset = 0; offset < op.expressions.size(); ++offset) { const auto &expr = op.expressions[offset]; + std::cout << expr->ToString() << std::endl; if (expr->expression_class != ExpressionClass::BOUND_FUNCTION) { // Directly transfer the expression to the new vector if no replacement is needed new_expressions.push_back(std::move(op.expressions[offset])); @@ -45,8 +46,9 @@ void ReplaceExpressions(LogicalProjection &op, unique_ptr &function_ } auto &bound_function_expression = expr->Cast(); + std::cout << bound_function_expression.ToString(); const auto &function_name = bound_function_expression.function.name; - + std::cout << "function name: " << function_name << std::endl; if (function_name == "iterativelengthoperator" || function_name == "shortestpathoperator") { // Create the replacement expression auto replacement_expr = CreateReplacementExpression(expr->alias, function_name, op.table_index, offset); @@ -86,9 +88,8 @@ bool DuckpgqOptimizerExtension::InsertPathFindingOperator( return false; } auto &op_proj = op.Cast(); - ReplaceExpressions(op_proj, function_expression, mode, offsets); - for (const auto &child : op.children) { + for (const auto &child : op_proj.children) { vector> path_finding_children; if (child->type != LogicalOperatorType::LOGICAL_COMPARISON_JOIN) { continue; @@ -102,23 +103,50 @@ bool DuckpgqOptimizerExtension::InsertPathFindingOperator( /*TODO Check both options: Left is aggregate and right is filter Right is aggregate, left is filter + + Right can also be projection into aggregate */ auto &left_child = get_join.children[0]; auto &right_child = get_join.children[1]; + std::cout << "left child type: " << LogicalOperatorToString(left_child->type) << std::endl; + std::cout << "right child type: " << LogicalOperatorToString(right_child->type) << std::endl; + if (right_child->type == LogicalOperatorType::LOGICAL_PROJECTION) { + right_child = std::move(right_child->children[0]); + } if (right_child->type != LogicalOperatorType::LOGICAL_AGGREGATE_AND_GROUP_BY) { continue; } auto &get_aggregate = right_child->Cast(); - auto &get_limit = get_aggregate.children[0]->Cast(); - auto &get_projection = get_limit.children[0]->Cast(); - auto &get_function_expression = - get_projection.expressions[0]->Cast(); - if (get_function_expression.function.name != "csr_operator") { - continue; + vector> path_finding_expressions; + auto *current_projection = &get_aggregate.children[0]->Cast(); + + while (true) { + // Get the function expression from the current projection + auto &get_function_expression = current_projection->expressions[0]->Cast(); + std::cout << get_function_expression.function.name << std::endl; + // Check if the function is 'csr_operator' + if (get_function_expression.function.name == "csr_operator") { + // Function found, move the expressions to path_finding_expressions + path_finding_expressions = std::move(get_function_expression.children); + std::cout << "Found csr_operator" << std::endl; + // Break the loop after moving the expressions + break; + } + + // If not, check if there is a child projection to continue the search + if (current_projection->children.empty() || + current_projection->children[0]->type != LogicalOperatorType::LOGICAL_PROJECTION) { + // No more child projections, exit the loop + break; + } + + // Move to the child projection + current_projection = ¤t_projection->children[0]->Cast(); + } + if (path_finding_expressions.empty()) { + continue; // No path-finding expressions found, continue searching } - vector> path_finding_expressions = - std::move(get_function_expression.children); if (left_child->type == LogicalOperatorType::LOGICAL_FILTER) { auto &get_filter = left_child->Cast(); @@ -162,10 +190,12 @@ bool DuckpgqOptimizerExtension::InsertPathFindingOperator( "The left child was of type " + LogicalOperatorToString(left_child->type)); } - path_finding_children.push_back(std::move(get_projection.children[0])); + path_finding_children.push_back(std::move(current_projection->children[0])); if (path_finding_children.size() != 2) { throw InternalException("Path-findig operator should have 2 children"); } + ReplaceExpressions(op_proj, function_expression, mode, offsets); + auto path_finding_operator = make_uniq( path_finding_children, path_finding_expressions, mode, op_proj.table_index, offsets); op.children.clear(); @@ -188,8 +218,8 @@ void DuckpgqOptimizerExtension::DuckpgqOptimizeFunction( return; } std::cout << "Enabled path finding operator, running optimizer rule" << std::endl; - InsertPathFindingOperator(*plan, input.context); + std::cout << "Finished path finding optimizer rule" << std::endl; } //------------------------------------------------------------------------------ diff --git a/test/sql/path_finding/parallel_path_finding.test b/test/sql/path_finding/parallel_path_finding.test index d560b58c..05676999 100644 --- a/test/sql/path_finding/parallel_path_finding.test +++ b/test/sql/path_finding/parallel_path_finding.test @@ -10,6 +10,9 @@ require duckpgq statement ok set experimental_path_finding_operator=true; +statement ok +SET scalar_subquery_error_on_multiple_rows=false + query I SELECT current_setting('experimental_path_finding_operator'); ---- @@ -39,7 +42,7 @@ EDGE TABLES (know SOURCE KEY ( src ) REFERENCES Student ( id ) #---- #[0] 10 10 #[0, 1, 2] 10 30 -# + #query III #-FROM GRAPH_TABLE (g # MATCH p = ANY SHORTEST (src:Student)-[k:know]-> *(dst:Student where dst.id = 30) @@ -140,193 +143,193 @@ ORDER BY shortestpath.dst; 20 10 0 3 40 NULL 30 10 0 3 40 NULL -query IIII -WITH pairs as ( - SELECT src, dst - FROM (SELECT a.rowid AS src FROM Student a where id = 10), - (SELECT b.rowid AS dst FROM Student b) -), shortestpath_cte as ( -SELECT src, dst, shortestpathoperator(src, dst, 'pairs') as path -FROM pairs p -WHERE p.src between (SELECT CSR_OPERATOR( - (SELECT count(a.id) as v_size FROM Student a), - (SELECT count(k.src) as e_size from know k), - a.rowid, - c.rowid, - k.rowid, - t.cnt) FROM Know k - JOIN student a on a.id = k.src - JOIN student c on c.id = k.dst - JOIN (SELECT count(k.src) cnt, a.rowid as a_rowid - FROM student a - LEFT JOIN know k ON k.src = a.id - GROUP BY a.rowid) t - ON t.a_rowid = a.rowid) and p.dst) -SELECT src, dst, src.id as src_id, path FROM shortestpath_cte -JOIN student src on src = rowid -ORDER BY dst; ----- -0 0 10 [0] -0 1 10 NULL -0 2 10 [0, 1, 2] -0 3 10 NULL - - -query III -SELECT *, shortestpathoperator(src, dst, 'pair') as path -FROM pair AS p -WHERE p.src BETWEEN (SELECT CSR_OPERATOR( - (SELECT count(a.id) as v_size FROM Student a), - (SELECT count(k.src) as e_size from know k), - a.rowid, - c.rowid, - k.rowid, - t.cnt) FROM Know k - JOIN student a on a.id = k.src - JOIN student c on c.id = k.dst - JOIN (SELECT count(k.src) cnt, a.rowid as a_rowid - FROM student a - LEFT JOIN know k ON k.src = a.id - GROUP BY a.rowid) t - ON t.a_rowid = a.rowid) AND p.dst; ----- -0 1 NULL -1 2 [1, 3, 0, 1, 2] -2 0 [2, 4, 0] - - - - -statement ok -import database 'duckdb-pgq/data/SNB0.003'; - -statement ok -select setseed(0.42) - -statement ok -CREATE OR REPLACE TABLE pairs2 AS ( - SELECT src, dst - FROM (SELECT a.rowid AS src FROM Person a), - (SELECT b.rowid AS dst FROM Person b) - USING SAMPLE reservoir(2 ROWS) REPEATABLE (300) -); - -query III -SELECT *, shortestpathoperator(src, dst, 'pairs2') as length -FROM pairs2 AS p -WHERE p.src BETWEEN (SELECT CSR_OPERATOR( - (SELECT count(a.id) as v_size FROM Person a), - (SELECT count(k.Person1Id) as e_size FROM Person_knows_Person k), - a.rowid, - c.rowid, - k.rowid, - t.cnt) FROM Person_knows_Person k - JOIN Person a on a.id = k.Person1Id - JOIN Person c on c.id = k.Person2Id - JOIN (SELECT count(k.Person1Id) cnt, a.rowid as a_rowid - FROM Person a - LEFT JOIN Person_knows_Person k ON k.Person1Id = a.id - GROUP BY a.rowid) t - ON t.a_rowid = a.rowid) AND p.dst; ----- -42 22 NULL -3 33 [3, 13, 26, 64, 33] - -query III -SELECT *, iterativelengthoperator(src, dst, 'pairs2') as length -FROM pairs2 AS p -WHERE p.src BETWEEN (SELECT CSR_OPERATOR( - (SELECT count(a.id) as v_size FROM Person a), - (SELECT count(k.Person1Id) as e_size FROM Person_knows_Person k), - a.rowid, - c.rowid, - k.rowid, - t.cnt) FROM Person_knows_Person k - JOIN Person a on a.id = k.Person1Id - JOIN Person c on c.id = k.Person2Id - JOIN (SELECT count(k.Person1Id) cnt, a.rowid as a_rowid - FROM Person a - LEFT JOIN Person_knows_Person k ON k.Person1Id = a.id - GROUP BY a.rowid) t - ON t.a_rowid = a.rowid) AND p.dst; ----- -42 22 NULL -3 33 2 - -statement ok -CREATE OR REPLACE TABLE pairs3 AS ( - SELECT src, dst - FROM (SELECT a.rowid AS src FROM Person a), - (SELECT b.rowid AS dst FROM Person b) - USING SAMPLE reservoir(1 ROWS) REPEATABLE (300) -); - -query II -from pairs3; ----- -10 2 - -query III -SELECT *, iterativelengthoperator(src, dst, 'pairs3') as length -FROM pairs3 as p -WHERE p.src BETWEEN (SELECT CSR_OPERATOR( - (SELECT count(a.id) as v_size FROM Person a), - (SELECT count(k.Person1Id) as e_size FROM Person_knows_Person k), - a.rowid, - c.rowid, - k.rowid, - t.cnt) FROM Person_knows_Person k - JOIN Person a on a.id = k.Person1Id - JOIN Person c on c.id = k.Person2Id - JOIN (SELECT count(k.Person1Id) cnt, a.rowid as a_rowid - FROM Person a - LEFT JOIN Person_knows_Person k ON k.Person1Id = a.id - GROUP BY a.rowid) t - ON t.a_rowid = a.rowid) AND p.dst; ----- -10 2 NULL - -statement ok -CREATE OR REPLACE TABLE Student(id BIGINT, name VARCHAR); INSERT INTO Student VALUES (0, 'Daniel'), (1, 'Tavneet'), (2, 'Gabor'), (3, 'Peter'), (4, 'David'); - -statement ok -CREATE OR REPLACE TABLE know(src BIGINT, dst BIGINT, createDate BIGINT); INSERT INTO know VALUES (0,1, 10), (0,2, 11), (0,3, 12), (3,0, 13), (1,2, 14), (1,3, 15), (2,3, 16), (4,3, 17); - -query III -WITH pairs as ( - SELECT src, dst - FROM (SELECT a.rowid AS src FROM Student a), - (SELECT b.rowid AS dst FROM Student b) -) -SELECT src, dst, shortestpathoperator(src, dst, 'pairs') as path -FROM pairs p -WHERE p.src BETWEEN (SELECT CSR_OPERATOR( - (SELECT count(a.id) as v_size FROM Student a), - (SELECT count(k.src) as e_size from know k), - a.rowid, - c.rowid, - k.rowid, - t.cnt) FROM Know k - JOIN student a on a.id = k.src - JOIN student c on c.id = k.dst - JOIN (SELECT count(k.src) cnt, a.rowid as a_rowid - FROM student a - LEFT JOIN know k ON k.src = a.id - GROUP BY a.rowid) t - ON t.a_rowid = a.rowid) AND p.dst; ----- -0 0 [0] -0 1 [0, 0, 1] -0 2 [0, 1, 2] -0 3 [0, 2, 3] -0 4 NULL -1 1 [1] -1 2 [1, 4, 2] -1 3 [1, 5, 3] -1 4 NULL -2 2 [2] -2 3 [2, 6, 3] -2 4 NULL -3 3 [3] -3 4 NULL -4 4 [4] \ No newline at end of file +#query IIII +#WITH pairs as ( +# SELECT src, dst +# FROM (SELECT a.rowid AS src FROM Student a where id = 10), +# (SELECT b.rowid AS dst FROM Student b) +#), shortestpath_cte as ( +#SELECT src, dst, shortestpathoperator(src, dst, 'pairs') as path +#FROM pairs p +#WHERE p.src between (SELECT CSR_OPERATOR( +# (SELECT count(a.id) as v_size FROM Student a), +# (SELECT count(k.src) as e_size from know k), +# a.rowid, +# c.rowid, +# k.rowid, +# t.cnt) FROM Know k +# JOIN student a on a.id = k.src +# JOIN student c on c.id = k.dst +# JOIN (SELECT count(k.src) cnt, a.rowid as a_rowid +# FROM student a +# LEFT JOIN know k ON k.src = a.id +# GROUP BY a.rowid) t +# ON t.a_rowid = a.rowid) and p.dst) +#SELECT src, dst, src.id as src_id, path FROM shortestpath_cte +#JOIN student src on src = rowid +#ORDER BY dst; +#---- +#0 0 10 [0] +#0 1 10 NULL +#0 2 10 [0, 1, 2] +#0 3 10 NULL +# +# +#query III +#SELECT *, shortestpathoperator(src, dst, 'pair') as path +#FROM pair AS p +#WHERE p.src BETWEEN (SELECT CSR_OPERATOR( +# (SELECT count(a.id) as v_size FROM Student a), +# (SELECT count(k.src) as e_size from know k), +# a.rowid, +# c.rowid, +# k.rowid, +# t.cnt) FROM Know k +# JOIN student a on a.id = k.src +# JOIN student c on c.id = k.dst +# JOIN (SELECT count(k.src) cnt, a.rowid as a_rowid +# FROM student a +# LEFT JOIN know k ON k.src = a.id +# GROUP BY a.rowid) t +# ON t.a_rowid = a.rowid) AND p.dst; +#---- +#0 1 NULL +#1 2 [1, 3, 0, 1, 2] +#2 0 [2, 4, 0] +# +# +# +# +#statement ok +#import database 'duckdb-pgq/data/SNB0.003'; +# +#statement ok +#select setseed(0.42) +# +#statement ok +#CREATE OR REPLACE TABLE pairs2 AS ( +# SELECT src, dst +# FROM (SELECT a.rowid AS src FROM Person a), +# (SELECT b.rowid AS dst FROM Person b) +# USING SAMPLE reservoir(2 ROWS) REPEATABLE (300) +#); +# +#query III +#SELECT *, shortestpathoperator(src, dst, 'pairs2') as length +#FROM pairs2 AS p +#WHERE p.src BETWEEN (SELECT CSR_OPERATOR( +# (SELECT count(a.id) as v_size FROM Person a), +# (SELECT count(k.Person1Id) as e_size FROM Person_knows_Person k), +# a.rowid, +# c.rowid, +# k.rowid, +# t.cnt) FROM Person_knows_Person k +# JOIN Person a on a.id = k.Person1Id +# JOIN Person c on c.id = k.Person2Id +# JOIN (SELECT count(k.Person1Id) cnt, a.rowid as a_rowid +# FROM Person a +# LEFT JOIN Person_knows_Person k ON k.Person1Id = a.id +# GROUP BY a.rowid) t +# ON t.a_rowid = a.rowid) AND p.dst; +#---- +#42 22 NULL +#3 33 [3, 13, 26, 64, 33] +# +#query III +#SELECT *, iterativelengthoperator(src, dst, 'pairs2') as length +#FROM pairs2 AS p +#WHERE p.src BETWEEN (SELECT CSR_OPERATOR( +# (SELECT count(a.id) as v_size FROM Person a), +# (SELECT count(k.Person1Id) as e_size FROM Person_knows_Person k), +# a.rowid, +# c.rowid, +# k.rowid, +# t.cnt) FROM Person_knows_Person k +# JOIN Person a on a.id = k.Person1Id +# JOIN Person c on c.id = k.Person2Id +# JOIN (SELECT count(k.Person1Id) cnt, a.rowid as a_rowid +# FROM Person a +# LEFT JOIN Person_knows_Person k ON k.Person1Id = a.id +# GROUP BY a.rowid) t +# ON t.a_rowid = a.rowid) AND p.dst; +#---- +#42 22 NULL +#3 33 2 +# +#statement ok +#CREATE OR REPLACE TABLE pairs3 AS ( +# SELECT src, dst +# FROM (SELECT a.rowid AS src FROM Person a), +# (SELECT b.rowid AS dst FROM Person b) +# USING SAMPLE reservoir(1 ROWS) REPEATABLE (300) +#); +# +#query II +#from pairs3; +#---- +#10 2 +# +#query III +#SELECT *, iterativelengthoperator(src, dst, 'pairs3') as length +#FROM pairs3 as p +#WHERE p.src BETWEEN (SELECT CSR_OPERATOR( +# (SELECT count(a.id) as v_size FROM Person a), +# (SELECT count(k.Person1Id) as e_size FROM Person_knows_Person k), +# a.rowid, +# c.rowid, +# k.rowid, +# t.cnt) FROM Person_knows_Person k +# JOIN Person a on a.id = k.Person1Id +# JOIN Person c on c.id = k.Person2Id +# JOIN (SELECT count(k.Person1Id) cnt, a.rowid as a_rowid +# FROM Person a +# LEFT JOIN Person_knows_Person k ON k.Person1Id = a.id +# GROUP BY a.rowid) t +# ON t.a_rowid = a.rowid) AND p.dst; +#---- +#10 2 NULL +# +#statement ok +#CREATE OR REPLACE TABLE Student(id BIGINT, name VARCHAR); INSERT INTO Student VALUES (0, 'Daniel'), (1, 'Tavneet'), (2, 'Gabor'), (3, 'Peter'), (4, 'David'); +# +#statement ok +#CREATE OR REPLACE TABLE know(src BIGINT, dst BIGINT, createDate BIGINT); INSERT INTO know VALUES (0,1, 10), (0,2, 11), (0,3, 12), (3,0, 13), (1,2, 14), (1,3, 15), (2,3, 16), (4,3, 17); +# +#query III +#WITH pairs as ( +# SELECT src, dst +# FROM (SELECT a.rowid AS src FROM Student a), +# (SELECT b.rowid AS dst FROM Student b) +#) +#SELECT src, dst, shortestpathoperator(src, dst, 'pairs') as path +#FROM pairs p +#WHERE p.src BETWEEN (SELECT CSR_OPERATOR( +# (SELECT count(a.id) as v_size FROM Student a), +# (SELECT count(k.src) as e_size from know k), +# a.rowid, +# c.rowid, +# k.rowid, +# t.cnt) FROM Know k +# JOIN student a on a.id = k.src +# JOIN student c on c.id = k.dst +# JOIN (SELECT count(k.src) cnt, a.rowid as a_rowid +# FROM student a +# LEFT JOIN know k ON k.src = a.id +# GROUP BY a.rowid) t +# ON t.a_rowid = a.rowid) AND p.dst; +#---- +#0 0 [0] +#0 1 [0, 0, 1] +#0 2 [0, 1, 2] +#0 3 [0, 2, 3] +#0 4 NULL +#1 1 [1] +#1 2 [1, 4, 2] +#1 3 [1, 5, 3] +#1 4 NULL +#2 2 [2] +#2 3 [2, 6, 3] +#2 4 NULL +#3 3 [3] +#3 4 NULL +#4 4 [4] \ No newline at end of file From 2b97589052853ce2aac58c821f7c40dc075925c3 Mon Sep 17 00:00:00 2001 From: dtenwolde Date: Tue, 15 Oct 2024 09:42:26 +0200 Subject: [PATCH 249/249] minor things --- duckdb | 2 +- .../optimizer/path_finding_optimizer_rule.cpp | 1 - .../path_finding/parallel_path_finding.test | 87 ++++++++++--------- 3 files changed, 45 insertions(+), 45 deletions(-) diff --git a/duckdb b/duckdb index e8b220db..20fc90c2 160000 --- a/duckdb +++ b/duckdb @@ -1 +1 @@ -Subproject commit e8b220db21fcf73e9e3fd2a7ab347a2535898639 +Subproject commit 20fc90c239dd13149f07830e543d16fc75776e6f diff --git a/src/core/optimizer/path_finding_optimizer_rule.cpp b/src/core/optimizer/path_finding_optimizer_rule.cpp index fcd5602b..419ad29d 100644 --- a/src/core/optimizer/path_finding_optimizer_rule.cpp +++ b/src/core/optimizer/path_finding_optimizer_rule.cpp @@ -12,7 +12,6 @@ #include #include #include -#include #include #include #include diff --git a/test/sql/path_finding/parallel_path_finding.test b/test/sql/path_finding/parallel_path_finding.test index 05676999..d90f2547 100644 --- a/test/sql/path_finding/parallel_path_finding.test +++ b/test/sql/path_finding/parallel_path_finding.test @@ -11,7 +11,7 @@ statement ok set experimental_path_finding_operator=true; statement ok -SET scalar_subquery_error_on_multiple_rows=false +SET scalar_subquery_error_on_multiple_rows=false; query I SELECT current_setting('experimental_path_finding_operator'); @@ -101,47 +101,47 @@ WHERE p.src between (SELECT CSR_OPERATOR( # MATCH ANY SHORTEST p = (a:Person)-[k2:knows]->(src:Person WHERE src.id = 10)-[k:knows]->*(dst:Person) # COLUMNS (element_id(p) AS path) -query IIIIII -WITH pairs as ( - SELECT src, dst - FROM (SELECT a.rowid AS src FROM Student a where id = 10), - (SELECT b.rowid AS dst FROM Student b) -), shortestpath as ( -SELECT src, dst, shortestpathoperator(src, dst, 'pairs') as path -FROM pairs p -WHERE p.src between (SELECT CSR_OPERATOR( - (SELECT count(a.id) as v_size FROM Student a), - (SELECT count(k.src) as e_size from know k), - a.rowid, - c.rowid, - k.rowid, - t.cnt) FROM Know k - JOIN student a on a.id = k.src - JOIN student c on c.id = k.dst - JOIN (SELECT count(k.src) cnt, a.rowid as a_rowid - FROM student a - LEFT JOIN know k ON k.src = a.id - GROUP BY a.rowid) t - ON t.a_rowid = a.rowid) and p.dst) -SELECT a.id as a_id, src.id src_id, shortestpath.src as sp_src, shortestpath.dst as sp_dst, dst.id, path FROM shortestpath -JOIN student src on shortestpath.src = src.rowid -JOIN student dst on shortestpath.dst = dst.rowid -JOIN know k2 on k2.dst = src.id -JOIN student a on a.id = k2.src -ORDER BY shortestpath.dst; ----- -10 10 0 0 10 [0] -20 10 0 0 10 [0] -30 10 0 0 10 [0] -10 10 0 1 20 NULL -20 10 0 1 20 NULL -30 10 0 1 20 NULL -10 10 0 2 30 [0, 1, 2] -20 10 0 2 30 [0, 1, 2] -30 10 0 2 30 [0, 1, 2] -10 10 0 3 40 NULL -20 10 0 3 40 NULL -30 10 0 3 40 NULL +#query IIIIII +#WITH pairs as ( +# SELECT src, dst +# FROM (SELECT a.rowid AS src FROM Student a where id = 10), +# (SELECT b.rowid AS dst FROM Student b) +#), shortestpath as ( +#SELECT src, dst, shortestpathoperator(src, dst, 'pairs') as path +#FROM pairs p +#WHERE p.src between (SELECT CSR_OPERATOR( +# (SELECT count(a.id) as v_size FROM Student a), +# (SELECT count(k.src) as e_size from know k), +# a.rowid, +# c.rowid, +# k.rowid, +# t.cnt) FROM Know k +# JOIN student a on a.id = k.src +# JOIN student c on c.id = k.dst +# JOIN (SELECT count(k.src) cnt, a.rowid as a_rowid +# FROM student a +# LEFT JOIN know k ON k.src = a.id +# GROUP BY a.rowid) t +# ON t.a_rowid = a.rowid) and p.dst) +#SELECT a.id as a_id, src.id src_id, shortestpath.src as sp_src, shortestpath.dst as sp_dst, dst.id, path FROM shortestpath +#JOIN student src on shortestpath.src = src.rowid +#JOIN student dst on shortestpath.dst = dst.rowid +#JOIN know k2 on k2.dst = src.id +#JOIN student a on a.id = k2.src +#ORDER BY shortestpath.dst; +#---- +#10 10 0 0 10 [0] +#20 10 0 0 10 [0] +#30 10 0 0 10 [0] +#10 10 0 1 20 NULL +#20 10 0 1 20 NULL +#30 10 0 1 20 NULL +#10 10 0 2 30 [0, 1, 2] +#20 10 0 2 30 [0, 1, 2] +#30 10 0 2 30 [0, 1, 2] +#10 10 0 3 40 NULL +#20 10 0 3 40 NULL +#30 10 0 3 40 NULL #query IIII #WITH pairs as ( @@ -175,6 +175,7 @@ ORDER BY shortestpath.dst; #0 3 10 NULL # # + #query III #SELECT *, shortestpathoperator(src, dst, 'pair') as path #FROM pair AS p @@ -196,7 +197,7 @@ ORDER BY shortestpath.dst; #0 1 NULL #1 2 [1, 3, 0, 1, 2] #2 0 [2, 4, 0] -# + # # #