diff --git a/src/duckdb/src/catalog/catalog.cpp b/src/duckdb/src/catalog/catalog.cpp index 50f50dead..d4b937d56 100644 --- a/src/duckdb/src/catalog/catalog.cpp +++ b/src/duckdb/src/catalog/catalog.cpp @@ -431,6 +431,26 @@ void FindMinimalQualification(ClientContext &context, const string &catalog_name qualify_schema = true; } +bool Catalog::TryAutoLoad(ClientContext &context, const string &extension_name) noexcept { + if (context.db->ExtensionIsLoaded(extension_name)) { + return true; + } +#ifndef DUCKDB_DISABLE_EXTENSION_LOAD + auto &dbconfig = DBConfig::GetConfig(context); + if (!dbconfig.options.autoload_known_extensions) { + return false; + } + try { + if (ExtensionHelper::CanAutoloadExtension(extension_name)) { + return ExtensionHelper::TryAutoLoadExtension(context, extension_name); + } + } catch (...) { + return false; + } +#endif + return false; +} + void Catalog::AutoloadExtensionByConfigName(ClientContext &context, const string &configuration_name) { #ifndef DUCKDB_DISABLE_EXTENSION_LOAD auto &dbconfig = DBConfig::GetConfig(context); diff --git a/src/duckdb/src/catalog/catalog_entry/duck_index_entry.cpp b/src/duckdb/src/catalog/catalog_entry/duck_index_entry.cpp index d6fc040b7..0848bc166 100644 --- a/src/duckdb/src/catalog/catalog_entry/duck_index_entry.cpp +++ b/src/duckdb/src/catalog/catalog_entry/duck_index_entry.cpp @@ -24,4 +24,9 @@ string DuckIndexEntry::GetTableName() const { return info->table; } +void DuckIndexEntry::CommitDrop() { + D_ASSERT(info && index); + index->CommitDrop(); +} + } // namespace duckdb diff --git a/src/duckdb/src/common/arrow/arrow_converter.cpp b/src/duckdb/src/common/arrow/arrow_converter.cpp index 3e9539aeb..b0e17f52f 100644 --- a/src/duckdb/src/common/arrow/arrow_converter.cpp +++ b/src/duckdb/src/common/arrow/arrow_converter.cpp @@ -139,6 +139,9 @@ void SetArrowFormat(DuckDBArrowSchemaHolder &root_holder, ArrowSchema &child, co case LogicalTypeId::DATE: child.format = "tdD"; break; +#ifdef DUCKDB_WASM + case LogicalTypeId::TIME_TZ: +#endif case LogicalTypeId::TIME: child.format = "ttu"; break; diff --git a/src/duckdb/src/common/radix_partitioning.cpp b/src/duckdb/src/common/radix_partitioning.cpp index 8f42c2492..6fae9a154 100644 --- a/src/duckdb/src/common/radix_partitioning.cpp +++ b/src/duckdb/src/common/radix_partitioning.cpp @@ -26,7 +26,7 @@ struct RadixPartitioningConstants { }; template -RETURN_TYPE RadixBitsSwitch(idx_t radix_bits, ARGS &&...args) { +RETURN_TYPE RadixBitsSwitch(idx_t radix_bits, ARGS &&... args) { D_ASSERT(radix_bits <= RadixPartitioning::MAX_RADIX_BITS); switch (radix_bits) { case 0: diff --git a/src/duckdb/src/common/sort/partition_state.cpp b/src/duckdb/src/common/sort/partition_state.cpp index c84b81ed0..4333c2d3c 100644 --- a/src/duckdb/src/common/sort/partition_state.cpp +++ b/src/duckdb/src/common/sort/partition_state.cpp @@ -429,7 +429,11 @@ bool PartitionGlobalMergeState::TryPrepareNextStage() { switch (stage) { case PartitionSortStage::INIT: - total_tasks = num_threads; + // If the partitions are unordered, don't scan in parallel + // because it produces non-deterministic orderings. + // This can theoretically happen with ORDER BY, + // but that is something the query should be explicit about. + total_tasks = sink.orders.size() > sink.partitions.size() ? num_threads : 1; stage = PartitionSortStage::SCAN; return true; diff --git a/src/duckdb/src/core_functions/aggregate/holistic/mode.cpp b/src/duckdb/src/core_functions/aggregate/holistic/mode.cpp index ea5be9113..0f81b88c9 100644 --- a/src/duckdb/src/core_functions/aggregate/holistic/mode.cpp +++ b/src/duckdb/src/core_functions/aggregate/holistic/mode.cpp @@ -220,7 +220,7 @@ struct ModeFunction { state.frequency_map = new typename STATE::Counts; } const double tau = .25; - if (state.nonzero <= tau * state.frequency_map->size()) { + if (state.nonzero <= tau * state.frequency_map->size() || prev.end <= frame.start || frame.end <= prev.start) { state.Reset(); // for f ∈ F do for (auto f = frame.start; f < frame.end; ++f) { diff --git a/src/duckdb/src/core_functions/function_list.cpp b/src/duckdb/src/core_functions/function_list.cpp index 01e101f74..550e44f54 100644 --- a/src/duckdb/src/core_functions/function_list.cpp +++ b/src/duckdb/src/core_functions/function_list.cpp @@ -49,7 +49,10 @@ static StaticFunctionDefinition internal_functions[] = { DUCKDB_SCALAR_FUNCTION(FactorialOperatorFun), DUCKDB_SCALAR_FUNCTION_SET(BitwiseAndFun), DUCKDB_SCALAR_FUNCTION(PowOperatorFun), + DUCKDB_SCALAR_FUNCTION_SET_ALIAS(ListInnerProductFunAlias), + DUCKDB_SCALAR_FUNCTION_SET_ALIAS(ListDistanceFunAlias), DUCKDB_SCALAR_FUNCTION_SET(LeftShiftFun), + DUCKDB_SCALAR_FUNCTION_SET_ALIAS(ListCosineSimilarityFunAlias), DUCKDB_SCALAR_FUNCTION_SET(RightShiftFun), DUCKDB_SCALAR_FUNCTION_SET(AbsOperatorFun), DUCKDB_SCALAR_FUNCTION_ALIAS(PowOperatorFunAlias), @@ -197,8 +200,12 @@ static StaticFunctionDefinition internal_functions[] = { DUCKDB_SCALAR_FUNCTION_ALIAS(ListAggrFun), DUCKDB_SCALAR_FUNCTION(ListAggregateFun), DUCKDB_SCALAR_FUNCTION_ALIAS(ListApplyFun), + DUCKDB_SCALAR_FUNCTION_SET(ListCosineSimilarityFun), + DUCKDB_SCALAR_FUNCTION_SET(ListDistanceFun), DUCKDB_SCALAR_FUNCTION(ListDistinctFun), + DUCKDB_SCALAR_FUNCTION_SET_ALIAS(ListDotProductFun), DUCKDB_SCALAR_FUNCTION(ListFilterFun), + DUCKDB_SCALAR_FUNCTION_SET(ListInnerProductFun), DUCKDB_SCALAR_FUNCTION_ALIAS(ListPackFun), DUCKDB_SCALAR_FUNCTION_SET(ListReverseSortFun), DUCKDB_SCALAR_FUNCTION_SET(ListSliceFun), diff --git a/src/duckdb/src/core_functions/scalar/list/list_cosine_similarity.cpp b/src/duckdb/src/core_functions/scalar/list/list_cosine_similarity.cpp new file mode 100644 index 000000000..726077526 --- /dev/null +++ b/src/duckdb/src/core_functions/scalar/list/list_cosine_similarity.cpp @@ -0,0 +1,78 @@ +#include "duckdb/core_functions/scalar/list_functions.hpp" +#include +#include + +namespace duckdb { + +template +static void ListCosineSimilarity(DataChunk &args, ExpressionState &, Vector &result) { + D_ASSERT(args.ColumnCount() == 2); + + auto count = args.size(); + auto &left = args.data[0]; + auto &right = args.data[1]; + auto left_count = ListVector::GetListSize(left); + auto right_count = ListVector::GetListSize(right); + + auto &left_child = ListVector::GetEntry(left); + auto &right_child = ListVector::GetEntry(right); + + D_ASSERT(left_child.GetVectorType() == VectorType::FLAT_VECTOR); + D_ASSERT(right_child.GetVectorType() == VectorType::FLAT_VECTOR); + + if (!FlatVector::Validity(left_child).CheckAllValid(left_count)) { + throw InvalidInputException("list_cosine_similarity: left argument can not contain NULL values"); + } + + if (!FlatVector::Validity(right_child).CheckAllValid(right_count)) { + throw InvalidInputException("list_cosine_similarity: right argument can not contain NULL values"); + } + + auto left_data = FlatVector::GetData(left_child); + auto right_data = FlatVector::GetData(right_child); + + BinaryExecutor::Execute( + left, right, result, count, [&](list_entry_t left, list_entry_t right) { + if (left.length != right.length) { + throw InvalidInputException(StringUtil::Format( + "list_cosine_similarity: list dimensions must be equal, got left length %d and right length %d", + left.length, right.length)); + } + + auto dimensions = left.length; + + NUMERIC_TYPE distance = 0; + NUMERIC_TYPE norm_l = 0; + NUMERIC_TYPE norm_r = 0; + + auto l_ptr = left_data + left.offset; + auto r_ptr = right_data + right.offset; + for (idx_t i = 0; i < dimensions; i++) { + auto x = *l_ptr++; + auto y = *r_ptr++; + distance += x * y; + norm_l += x * x; + norm_r += y * y; + } + + auto similarity = distance / (std::sqrt(norm_l) * std::sqrt(norm_r)); + + // clamp to [-1, 1] to avoid floating point errors + return std::max(static_cast(-1), std::min(similarity, static_cast(1))); + }); + + if (args.AllConstant()) { + result.SetVectorType(VectorType::CONSTANT_VECTOR); + } +} + +ScalarFunctionSet ListCosineSimilarityFun::GetFunctions() { + ScalarFunctionSet set("list_cosine_similarity"); + set.AddFunction(ScalarFunction({LogicalType::LIST(LogicalType::FLOAT), LogicalType::LIST(LogicalType::FLOAT)}, + LogicalType::FLOAT, ListCosineSimilarity)); + set.AddFunction(ScalarFunction({LogicalType::LIST(LogicalType::DOUBLE), LogicalType::LIST(LogicalType::DOUBLE)}, + LogicalType::DOUBLE, ListCosineSimilarity)); + return set; +} + +} // namespace duckdb diff --git a/src/duckdb/src/core_functions/scalar/list/list_distance.cpp b/src/duckdb/src/core_functions/scalar/list/list_distance.cpp new file mode 100644 index 000000000..aa70e4a13 --- /dev/null +++ b/src/duckdb/src/core_functions/scalar/list/list_distance.cpp @@ -0,0 +1,72 @@ +#include "duckdb/core_functions/scalar/list_functions.hpp" +#include + +namespace duckdb { + +template +static void ListDistance(DataChunk &args, ExpressionState &, Vector &result) { + D_ASSERT(args.ColumnCount() == 2); + + auto count = args.size(); + auto &left = args.data[0]; + auto &right = args.data[1]; + auto left_count = ListVector::GetListSize(left); + auto right_count = ListVector::GetListSize(right); + + auto &left_child = ListVector::GetEntry(left); + auto &right_child = ListVector::GetEntry(right); + + D_ASSERT(left_child.GetVectorType() == VectorType::FLAT_VECTOR); + D_ASSERT(right_child.GetVectorType() == VectorType::FLAT_VECTOR); + + if (!FlatVector::Validity(left_child).CheckAllValid(left_count)) { + throw InvalidInputException("list_distance: left argument can not contain NULL values"); + } + + if (!FlatVector::Validity(right_child).CheckAllValid(right_count)) { + throw InvalidInputException("list_distance: right argument can not contain NULL values"); + } + + auto left_data = FlatVector::GetData(left_child); + auto right_data = FlatVector::GetData(right_child); + + BinaryExecutor::Execute( + left, right, result, count, [&](list_entry_t left, list_entry_t right) { + if (left.length != right.length) { + throw InvalidInputException(StringUtil::Format( + "list_distance: list dimensions must be equal, got left length %d and right length %d", left.length, + right.length)); + } + + auto dimensions = left.length; + + NUMERIC_TYPE distance = 0; + + auto l_ptr = left_data + left.offset; + auto r_ptr = right_data + right.offset; + + for (idx_t i = 0; i < dimensions; i++) { + auto x = *l_ptr++; + auto y = *r_ptr++; + auto diff = x - y; + distance += diff * diff; + } + + return std::sqrt(distance); + }); + + if (args.AllConstant()) { + result.SetVectorType(VectorType::CONSTANT_VECTOR); + } +} + +ScalarFunctionSet ListDistanceFun::GetFunctions() { + ScalarFunctionSet set("list_distance"); + set.AddFunction(ScalarFunction({LogicalType::LIST(LogicalType::FLOAT), LogicalType::LIST(LogicalType::FLOAT)}, + LogicalType::FLOAT, ListDistance)); + set.AddFunction(ScalarFunction({LogicalType::LIST(LogicalType::DOUBLE), LogicalType::LIST(LogicalType::DOUBLE)}, + LogicalType::DOUBLE, ListDistance)); + return set; +} + +} // namespace duckdb diff --git a/src/duckdb/src/core_functions/scalar/list/list_inner_product.cpp b/src/duckdb/src/core_functions/scalar/list/list_inner_product.cpp new file mode 100644 index 000000000..45293e0c4 --- /dev/null +++ b/src/duckdb/src/core_functions/scalar/list/list_inner_product.cpp @@ -0,0 +1,70 @@ +#include "duckdb/core_functions/scalar/list_functions.hpp" + +namespace duckdb { + +template +static void ListInnerProduct(DataChunk &args, ExpressionState &, Vector &result) { + D_ASSERT(args.ColumnCount() == 2); + + auto count = args.size(); + auto &left = args.data[0]; + auto &right = args.data[1]; + auto left_count = ListVector::GetListSize(left); + auto right_count = ListVector::GetListSize(right); + + auto &left_child = ListVector::GetEntry(left); + auto &right_child = ListVector::GetEntry(right); + + D_ASSERT(left_child.GetVectorType() == VectorType::FLAT_VECTOR); + D_ASSERT(right_child.GetVectorType() == VectorType::FLAT_VECTOR); + + if (!FlatVector::Validity(left_child).CheckAllValid(left_count)) { + throw InvalidInputException("list_inner_product: left argument can not contain NULL values"); + } + + if (!FlatVector::Validity(right_child).CheckAllValid(right_count)) { + throw InvalidInputException("list_inner_product: right argument can not contain NULL values"); + } + + auto left_data = FlatVector::GetData(left_child); + auto right_data = FlatVector::GetData(right_child); + + BinaryExecutor::Execute( + left, right, result, count, [&](list_entry_t left, list_entry_t right) { + if (left.length != right.length) { + throw InvalidInputException(StringUtil::Format( + "list_inner_product: list dimensions must be equal, got left length %d and right length %d", + left.length, right.length)); + } + + auto dimensions = left.length; + + NUMERIC_TYPE distance = 0; + + auto l_ptr = left_data + left.offset; + auto r_ptr = right_data + right.offset; + + for (idx_t i = 0; i < dimensions; i++) { + auto x = *l_ptr++; + auto y = *r_ptr++; + distance += x * y; + } + + return distance; + }); + + if (args.AllConstant()) { + result.SetVectorType(VectorType::CONSTANT_VECTOR); + } +} + +ScalarFunctionSet ListInnerProductFun::GetFunctions() { + ScalarFunctionSet set("list_inner_product"); + set.AddFunction(ScalarFunction({LogicalType::LIST(LogicalType::FLOAT), LogicalType::LIST(LogicalType::FLOAT)}, + LogicalType::FLOAT, ListInnerProduct)); + set.AddFunction(ScalarFunction({LogicalType::LIST(LogicalType::DOUBLE), LogicalType::LIST(LogicalType::DOUBLE)}, + LogicalType::DOUBLE, ListInnerProduct)); + return set; +} + +} // namespace duckdb diff --git a/src/duckdb/src/execution/index/art/art.cpp b/src/duckdb/src/execution/index/art/art.cpp index d4d100988..cdc365c91 100644 --- a/src/duckdb/src/execution/index/art/art.cpp +++ b/src/duckdb/src/execution/index/art/art.cpp @@ -14,6 +14,8 @@ #include "duckdb/execution/index/art/iterator.hpp" #include "duckdb/common/types/conflict_manager.hpp" #include "duckdb/storage/table/scan_state.hpp" +#include "duckdb/storage/metadata/metadata_reader.hpp" +#include "duckdb/storage/table_io_manager.hpp" #include @@ -33,7 +35,8 @@ struct ARTIndexScanState : public IndexScanState { ART::ART(const vector &column_ids, TableIOManager &table_io_manager, const vector> &unbound_expressions, const IndexConstraintType constraint_type, - AttachedDatabase &db, const shared_ptr> &allocators_ptr, BlockPointer pointer) + AttachedDatabase &db, const shared_ptr, ALLOCATOR_COUNT>> &allocators_ptr, + const BlockPointer &pointer) : Index(db, IndexType::ART, table_io_manager, column_ids, unbound_expressions, constraint_type), allocators(allocators_ptr), owns_data(false) { if (!Radix::IsLittleEndian()) { @@ -43,22 +46,20 @@ ART::ART(const vector &column_ids, TableIOManager &table_io_manager, // initialize all allocators if (!allocators) { owns_data = true; - allocators = make_shared>(); - allocators->emplace_back(FixedSizeAllocator(sizeof(Prefix), buffer_manager.GetBufferAllocator())); - allocators->emplace_back(FixedSizeAllocator(sizeof(Leaf), buffer_manager.GetBufferAllocator())); - allocators->emplace_back(FixedSizeAllocator(sizeof(Node4), buffer_manager.GetBufferAllocator())); - allocators->emplace_back(FixedSizeAllocator(sizeof(Node16), buffer_manager.GetBufferAllocator())); - allocators->emplace_back(FixedSizeAllocator(sizeof(Node48), buffer_manager.GetBufferAllocator())); - allocators->emplace_back(FixedSizeAllocator(sizeof(Node256), buffer_manager.GetBufferAllocator())); - } - - // set the root node of the tree - tree = make_uniq(); - serialized_data_pointer = pointer; + auto &block_manager = table_io_manager.GetIndexBlockManager(); + + array, ALLOCATOR_COUNT> allocator_array = { + make_uniq(sizeof(Prefix), block_manager), + make_uniq(sizeof(Leaf), block_manager), + make_uniq(sizeof(Node4), block_manager), + make_uniq(sizeof(Node16), block_manager), + make_uniq(sizeof(Node48), block_manager), + make_uniq(sizeof(Node256), block_manager)}; + allocators = make_shared, ALLOCATOR_COUNT>>(std::move(allocator_array)); + } + if (pointer.IsValid()) { - tree->SetSerialized(); - tree->SetPtr(pointer.block_id, pointer.offset); - tree->Deserialize(*this); + Deserialize(pointer); } // validate the types of the key columns @@ -84,10 +85,6 @@ ART::ART(const vector &column_ids, TableIOManager &table_io_manager, } } -ART::~ART() { - tree->Reset(); -} - //===--------------------------------------------------------------------===// // Initialize Predicate Scans //===--------------------------------------------------------------------===// @@ -351,7 +348,7 @@ bool ART::ConstructFromSorted(idx_t count, vector &keys, Vector &row_ide auto key_section = KeySection(0, count - 1, 0, 0); auto has_constraint = IsUnique(); - if (!Construct(*this, keys, row_ids, *this->tree, key_section, has_constraint)) { + if (!Construct(*this, keys, row_ids, tree, key_section, has_constraint)) { return false; } @@ -359,9 +356,8 @@ bool ART::ConstructFromSorted(idx_t count, vector &keys, Vector &row_ide D_ASSERT(!VerifyAndToStringInternal(true).empty()); for (idx_t i = 0; i < count; i++) { D_ASSERT(!keys[i].Empty()); - auto leaf = Lookup(*tree, keys[i], 0); - D_ASSERT(leaf.IsSet()); - D_ASSERT(Leaf::ContainsRowId(*this, leaf, row_ids[i])); + auto leaf = Lookup(tree, keys[i], 0); + D_ASSERT(Leaf::ContainsRowId(*this, *leaf, row_ids[i])); } #endif @@ -393,7 +389,7 @@ PreservedError ART::Insert(IndexLock &lock, DataChunk &input, Vector &row_ids) { } row_t row_id = row_identifiers[i]; - if (!Insert(*tree, keys[i], 0, row_id)) { + if (!Insert(tree, keys[i], 0, row_id)) { // failed to insert because of constraint violation failed_index = i; break; @@ -407,7 +403,7 @@ PreservedError ART::Insert(IndexLock &lock, DataChunk &input, Vector &row_ids) { continue; } row_t row_id = row_identifiers[i]; - Erase(*tree, keys[i], 0, row_id); + Erase(tree, keys[i], 0, row_id); } } @@ -422,9 +418,8 @@ PreservedError ART::Insert(IndexLock &lock, DataChunk &input, Vector &row_ids) { continue; } - auto leaf = Lookup(*tree, keys[i], 0); - D_ASSERT(leaf.IsSet()); - D_ASSERT(Leaf::ContainsRowId(*this, leaf, row_identifiers[i])); + auto leaf = Lookup(tree, keys[i], 0); + D_ASSERT(Leaf::ContainsRowId(*this, *leaf, row_identifiers[i])); } #endif @@ -465,7 +460,7 @@ bool ART::InsertToLeaf(Node &leaf, const row_t &row_id) { bool ART::Insert(Node &node, const ARTKey &key, idx_t depth, const row_t &row_id) { // node is currently empty, create a leaf here with the key - if (!node.IsSet()) { + if (!node.HasMetadata()) { D_ASSERT(depth <= key.len); reference ref_node(node); Prefix::New(*this, ref_node, key, depth, key.len - depth); @@ -482,7 +477,7 @@ bool ART::Insert(Node &node, const ARTKey &key, idx_t depth, const row_t &row_id if (node_type != NType::PREFIX) { D_ASSERT(depth < key.len); - auto child = node.GetChild(*this, key[depth]); + auto child = node.GetChildMutable(*this, key[depth]); // recurse, if a child exists at key[depth] if (child) { @@ -504,7 +499,7 @@ bool ART::Insert(Node &node, const ARTKey &key, idx_t depth, const row_t &row_id // this is a prefix node, traverse reference next_node(node); - auto mismatch_position = Prefix::Traverse(*this, next_node, key, depth); + auto mismatch_position = Prefix::TraverseMutable(*this, next_node, key, depth); // prefix matches key if (next_node.get().GetType() != NType::PREFIX) { @@ -533,9 +528,16 @@ bool ART::Insert(Node &node, const ARTKey &key, idx_t depth, const row_t &row_id } //===--------------------------------------------------------------------===// -// Delete +// Drop and Delete //===--------------------------------------------------------------------===// +void ART::CommitDrop(IndexLock &index_lock) { + for (auto &allocator : *allocators) { + allocator->Reset(); + } + tree.Clear(); +} + void ART::Delete(IndexLock &state, DataChunk &input, Vector &row_ids) { DataChunk expression; @@ -557,7 +559,7 @@ void ART::Delete(IndexLock &state, DataChunk &input, Vector &row_ids) { if (keys[i].Empty()) { continue; } - Erase(*tree, keys[i], 0, row_identifiers[i]); + Erase(tree, keys[i], 0, row_identifiers[i]); } #ifdef DEBUG @@ -567,9 +569,9 @@ void ART::Delete(IndexLock &state, DataChunk &input, Vector &row_ids) { continue; } - auto leaf = Lookup(*tree, keys[i], 0); - if (leaf.IsSet()) { - D_ASSERT(!Leaf::ContainsRowId(*this, leaf, row_identifiers[i])); + auto leaf = Lookup(tree, keys[i], 0); + if (leaf) { + D_ASSERT(!Leaf::ContainsRowId(*this, *leaf, row_identifiers[i])); } } #endif @@ -577,14 +579,14 @@ void ART::Delete(IndexLock &state, DataChunk &input, Vector &row_ids) { void ART::Erase(Node &node, const ARTKey &key, idx_t depth, const row_t &row_id) { - if (!node.IsSet()) { + if (!node.HasMetadata()) { return; } // handle prefix reference next_node(node); if (next_node.get().GetType() == NType::PREFIX) { - Prefix::Traverse(*this, next_node, key, depth); + Prefix::TraverseMutable(*this, next_node, key, depth); if (next_node.get().GetType() == NType::PREFIX) { return; } @@ -599,14 +601,14 @@ void ART::Erase(Node &node, const ARTKey &key, idx_t depth, const row_t &row_id) } D_ASSERT(depth < key.len); - auto child = next_node.get().GetChild(*this, key[depth]); + auto child = next_node.get().GetChildMutable(*this, key[depth]); if (child) { - D_ASSERT(child->IsSet()); + D_ASSERT(child->HasMetadata()); auto temp_depth = depth + 1; reference child_node(*child); if (child_node.get().GetType() == NType::PREFIX) { - Prefix::Traverse(*this, child_node, key, temp_depth); + Prefix::TraverseMutable(*this, child_node, key, temp_depth); if (child_node.get().GetType() == NType::PREFIX) { return; } @@ -666,24 +668,24 @@ static ARTKey CreateKey(ArenaAllocator &allocator, PhysicalType type, Value &val bool ART::SearchEqual(ARTKey &key, idx_t max_count, vector &result_ids) { - auto leaf = Lookup(*tree, key, 0); - if (!leaf.IsSet()) { + auto leaf = Lookup(tree, key, 0); + if (!leaf) { return true; } - return Leaf::GetRowIds(*this, leaf, result_ids, max_count); + return Leaf::GetRowIds(*this, *leaf, result_ids, max_count); } void ART::SearchEqualJoinNoFetch(ARTKey &key, idx_t &result_size) { // we need to look for a leaf - auto leaf_node = Lookup(*tree, key, 0); - if (!leaf_node.IsSet()) { + auto leaf_node = Lookup(tree, key, 0); + if (!leaf_node) { result_size = 0; return; } // we only perform index joins on PK/FK columns - D_ASSERT(leaf_node.GetType() == NType::LEAF_INLINED); + D_ASSERT(leaf_node->GetType() == NType::LEAF_INLINED); result_size = 1; return; } @@ -692,37 +694,38 @@ void ART::SearchEqualJoinNoFetch(ARTKey &key, idx_t &result_size) { // Lookup //===--------------------------------------------------------------------===// -Node ART::Lookup(Node node, const ARTKey &key, idx_t depth) { +optional_ptr ART::Lookup(const Node &node, const ARTKey &key, idx_t depth) { - while (node.IsSet()) { + reference node_ref(node); + while (node_ref.get().HasMetadata()) { // traverse prefix, if exists - reference next_node(node); + reference next_node(node_ref.get()); if (next_node.get().GetType() == NType::PREFIX) { Prefix::Traverse(*this, next_node, key, depth); if (next_node.get().GetType() == NType::PREFIX) { - return Node(); + return nullptr; } } if (next_node.get().GetType() == NType::LEAF || next_node.get().GetType() == NType::LEAF_INLINED) { - return next_node.get(); + return &next_node.get(); } D_ASSERT(depth < key.len); auto child = next_node.get().GetChild(*this, key[depth]); if (!child) { // prefix matches key, but no child at byte, ART/subtree does not contain key - return Node(); + return nullptr; } // lookup in child node - node = *child; - D_ASSERT(node.IsSet()); + node_ref = *child; + D_ASSERT(node_ref.get().HasMetadata()); depth++; } - return Node(); + return nullptr; } //===--------------------------------------------------------------------===// @@ -731,7 +734,7 @@ Node ART::Lookup(Node node, const ARTKey &key, idx_t depth) { bool ART::SearchGreater(ARTIndexScanState &state, ARTKey &key, bool equal, idx_t max_count, vector &result_ids) { - if (!tree->IsSet()) { + if (!tree.HasMetadata()) { return true; } Iterator &it = state.iterator; @@ -739,7 +742,7 @@ bool ART::SearchGreater(ARTIndexScanState &state, ARTKey &key, bool equal, idx_t // find the lowest value that satisfies the predicate if (!it.art) { it.art = this; - if (!it.LowerBound(*tree, key, equal, 0)) { + if (!it.LowerBound(tree, key, equal, 0)) { // early-out, if the maximum value in the ART is lower than the lower bound return true; } @@ -754,7 +757,7 @@ bool ART::SearchGreater(ARTIndexScanState &state, ARTKey &key, bool equal, idx_t bool ART::SearchLess(ARTIndexScanState &state, ARTKey &upper_bound, bool equal, idx_t max_count, vector &result_ids) { - if (!tree->IsSet()) { + if (!tree.HasMetadata()) { return true; } Iterator &it = state.iterator; @@ -762,7 +765,7 @@ bool ART::SearchLess(ARTIndexScanState &state, ARTKey &upper_bound, bool equal, if (!it.art) { it.art = this; // find the minimum value in the ART: we start scanning from this value - it.FindMinimum(*tree); + it.FindMinimum(tree); // early-out, if the minimum value is higher than the upper bound if (it.current_key > upper_bound) { return true; @@ -785,7 +788,7 @@ bool ART::SearchCloseRange(ARTIndexScanState &state, ARTKey &lower_bound, ARTKey // find the first node that satisfies the left predicate if (!it.art) { it.art = this; - if (!it.LowerBound(*tree, lower_bound, left_equal, 0)) { + if (!it.LowerBound(tree, lower_bound, left_equal, 0)) { // early-out, if the maximum value in the ART is lower than the lower bound return true; } @@ -940,8 +943,8 @@ void ART::CheckConstraintsForChunk(DataChunk &input, ConflictManager &conflict_m continue; } - auto leaf = Lookup(*tree, keys[i], 0); - if (!leaf.IsSet()) { + auto leaf = Lookup(tree, keys[i], 0); + if (!leaf) { if (conflict_manager.AddMiss(i)) { found_conflict = i; } @@ -950,8 +953,8 @@ void ART::CheckConstraintsForChunk(DataChunk &input, ConflictManager &conflict_m // when we find a node, we need to update the 'matches' and 'row_ids' // NOTE: leaves can have more than one row_id, but for UNIQUE/PRIMARY KEY they will only have one - D_ASSERT(leaf.GetType() == NType::LEAF_INLINED); - if (conflict_manager.AddHit(i, leaf.GetRowId())) { + D_ASSERT(leaf->GetType() == NType::LEAF_INLINED); + if (conflict_manager.AddHit(i, leaf->GetRowId())) { found_conflict = i; } } @@ -973,14 +976,38 @@ void ART::CheckConstraintsForChunk(DataChunk &input, ConflictManager &conflict_m BlockPointer ART::Serialize(MetadataWriter &writer) { + D_ASSERT(owns_data); + + // early-out, if all allocators are empty + if (!tree.HasMetadata()) { + root_block_pointer = BlockPointer(); + return root_block_pointer; + } + lock_guard l(lock); - if (tree->IsSet()) { - serialized_data_pointer = tree->Serialize(*this, writer); - } else { - serialized_data_pointer = BlockPointer(); + vector allocator_pointers; + for (auto &allocator : *allocators) { + allocator_pointers.push_back(allocator->Serialize(writer)); } - return serialized_data_pointer; + root_block_pointer = writer.GetBlockPointer(); + writer.Write(tree); + for (auto &allocator_pointer : allocator_pointers) { + writer.Write(allocator_pointer); + } + + return root_block_pointer; +} + +void ART::Deserialize(const BlockPointer &pointer) { + + D_ASSERT(pointer.IsValid()); + MetadataReader reader(table_io_manager.GetMetadataManager(), pointer); + tree = reader.Read(); + + for (idx_t i = 0; i < ALLOCATOR_COUNT; i++) { + (*allocators)[i]->Deserialize(reader.Read()); + } } //===--------------------------------------------------------------------===// @@ -991,7 +1018,7 @@ void ART::InitializeVacuum(ARTFlags &flags) { flags.vacuum_flags.reserve(allocators->size()); for (auto &allocator : *allocators) { - flags.vacuum_flags.push_back(allocator.InitializeVacuum()); + flags.vacuum_flags.push_back(allocator->InitializeVacuum()); } } @@ -999,7 +1026,7 @@ void ART::FinalizeVacuum(const ARTFlags &flags) { for (idx_t i = 0; i < allocators->size(); i++) { if (flags.vacuum_flags[i]) { - (*allocators)[i].FinalizeVacuum(); + (*allocators)[i]->FinalizeVacuum(); } } } @@ -1008,9 +1035,9 @@ void ART::Vacuum(IndexLock &state) { D_ASSERT(owns_data); - if (!tree->IsSet()) { + if (!tree.HasMetadata()) { for (auto &allocator : *allocators) { - allocator.Reset(); + allocator->Reset(); } return; } @@ -1032,14 +1059,10 @@ void ART::Vacuum(IndexLock &state) { } // traverse the allocated memory of the tree to perform a vacuum - tree->Vacuum(*this, flags); + tree.Vacuum(*this, flags); // finalize the vacuum operation FinalizeVacuum(flags); - - for (auto &allocator : *allocators) { - allocator.Verify(); - } } //===--------------------------------------------------------------------===// @@ -1052,39 +1075,35 @@ void ART::InitializeMerge(ARTFlags &flags) { flags.merge_buffer_counts.reserve(allocators->size()); for (auto &allocator : *allocators) { - flags.merge_buffer_counts.emplace_back(allocator.buffers.size()); + flags.merge_buffer_counts.emplace_back(allocator->GetUpperBoundBufferId()); } } bool ART::MergeIndexes(IndexLock &state, Index &other_index) { auto &other_art = other_index.Cast(); - if (!other_art.tree->IsSet()) { + if (!other_art.tree.HasMetadata()) { return true; } if (other_art.owns_data) { - if (tree->IsSet()) { + if (tree.HasMetadata()) { // fully deserialize other_index, and traverse it to increment its buffer IDs ARTFlags flags; InitializeMerge(flags); - other_art.tree->InitializeMerge(other_art, flags); + other_art.tree.InitializeMerge(other_art, flags); } // merge the node storage for (idx_t i = 0; i < allocators->size(); i++) { - (*allocators)[i].Merge((*other_art.allocators)[i]); + (*allocators)[i]->Merge(*(*other_art.allocators)[i]); } } // merge the ARTs - if (!tree->Merge(*this, *other_art.tree)) { + if (!tree.Merge(*this, other_art.tree)) { return false; } - - for (auto &allocator : *allocators) { - allocator.Verify(); - } return true; } @@ -1100,8 +1119,8 @@ string ART::VerifyAndToString(IndexLock &state, const bool only_verify) { } string ART::VerifyAndToStringInternal(const bool only_verify) { - if (tree->IsSet()) { - return "ART: " + tree->VerifyAndToString(*this, only_verify); + if (tree.HasMetadata()) { + return "ART: " + tree.VerifyAndToString(*this, only_verify); } return "[empty]"; } diff --git a/src/duckdb/src/execution/index/art/fixed_size_allocator.cpp b/src/duckdb/src/execution/index/art/fixed_size_allocator.cpp deleted file mode 100644 index ac1526e22..000000000 --- a/src/duckdb/src/execution/index/art/fixed_size_allocator.cpp +++ /dev/null @@ -1,238 +0,0 @@ -#include "duckdb/execution/index/art/fixed_size_allocator.hpp" - -namespace duckdb { - -constexpr idx_t FixedSizeAllocator::BASE[]; -constexpr uint8_t FixedSizeAllocator::SHIFT[]; - -FixedSizeAllocator::FixedSizeAllocator(const idx_t allocation_size, Allocator &allocator) - : allocation_size(allocation_size), total_allocations(0), allocator(allocator) { - - // calculate how many allocations fit into one buffer - - idx_t bits_per_value = sizeof(validity_t) * 8; - idx_t curr_alloc_size = 0; - - bitmask_count = 0; - allocations_per_buffer = 0; - - while (curr_alloc_size < BUFFER_ALLOC_SIZE) { - if (!bitmask_count || (bitmask_count * bits_per_value) % allocations_per_buffer == 0) { - bitmask_count++; - curr_alloc_size += sizeof(validity_t); - } - - auto remaining_alloc_size = BUFFER_ALLOC_SIZE - curr_alloc_size; - auto remaining_allocations = MinValue(remaining_alloc_size / allocation_size, bits_per_value); - - if (remaining_allocations == 0) { - break; - } - - allocations_per_buffer += remaining_allocations; - curr_alloc_size += remaining_allocations * allocation_size; - } - - allocation_offset = bitmask_count * sizeof(validity_t); -} - -FixedSizeAllocator::~FixedSizeAllocator() { - for (auto &buffer : buffers) { - allocator.FreeData(buffer.ptr, BUFFER_ALLOC_SIZE); - } -} - -Node FixedSizeAllocator::New() { - - // no more free pointers - if (buffers_with_free_space.empty()) { - - // add a new buffer - idx_t buffer_id = buffers.size(); - D_ASSERT(buffer_id <= (uint32_t)DConstants::INVALID_INDEX); - auto buffer = allocator.AllocateData(BUFFER_ALLOC_SIZE); - buffers.emplace_back(buffer, 0); - buffers_with_free_space.insert(buffer_id); - - // set the bitmask - ValidityMask mask(reinterpret_cast(buffer)); - mask.SetAllValid(allocations_per_buffer); - } - - // return a pointer - D_ASSERT(!buffers_with_free_space.empty()); - auto buffer_id = (uint32_t)*buffers_with_free_space.begin(); - - auto bitmask_ptr = reinterpret_cast(buffers[buffer_id].ptr); - ValidityMask mask(bitmask_ptr); - auto offset = GetOffset(mask, buffers[buffer_id].allocation_count); - - buffers[buffer_id].allocation_count++; - total_allocations++; - if (buffers[buffer_id].allocation_count == allocations_per_buffer) { - buffers_with_free_space.erase(buffer_id); - } - - return Node(buffer_id, offset); -} - -void FixedSizeAllocator::Free(const Node ptr) { - auto bitmask_ptr = reinterpret_cast(buffers[ptr.GetBufferId()].ptr); - ValidityMask mask(bitmask_ptr); - D_ASSERT(!mask.RowIsValid(ptr.GetOffset())); - mask.SetValid(ptr.GetOffset()); - buffers_with_free_space.insert(ptr.GetBufferId()); - - D_ASSERT(total_allocations > 0); - D_ASSERT(buffers[ptr.GetBufferId()].allocation_count > 0); - buffers[ptr.GetBufferId()].allocation_count--; - total_allocations--; -} - -void FixedSizeAllocator::Reset() { - - for (auto &buffer : buffers) { - allocator.FreeData(buffer.ptr, BUFFER_ALLOC_SIZE); - } - buffers.clear(); - buffers_with_free_space.clear(); - total_allocations = 0; -} - -void FixedSizeAllocator::Merge(FixedSizeAllocator &other) { - - D_ASSERT(allocation_size == other.allocation_size); - - // remember the buffer count and merge the buffers - idx_t buffer_count = buffers.size(); - for (auto &buffer : other.buffers) { - buffers.push_back(buffer); - } - other.buffers.clear(); - - // merge the buffers with free spaces - for (auto &buffer_id : other.buffers_with_free_space) { - buffers_with_free_space.insert(buffer_id + buffer_count); - } - other.buffers_with_free_space.clear(); - - // add the total allocations - total_allocations += other.total_allocations; -} - -bool FixedSizeAllocator::InitializeVacuum() { - - if (total_allocations == 0) { - Reset(); - return false; - } - - auto total_available_allocations = allocations_per_buffer * buffers.size(); - D_ASSERT(total_available_allocations >= total_allocations); - auto total_free_positions = total_available_allocations - total_allocations; - - // vacuum_count buffers can be freed - auto vacuum_count = total_free_positions / allocations_per_buffer; - - // calculate the vacuum threshold adaptively - D_ASSERT(vacuum_count < buffers.size()); - idx_t memory_usage = GetMemoryUsage(); - idx_t excess_memory_usage = vacuum_count * BUFFER_ALLOC_SIZE; - auto excess_percentage = (double)excess_memory_usage / (double)memory_usage; - auto threshold = (double)VACUUM_THRESHOLD / 100.0; - if (excess_percentage < threshold) { - return false; - } - - min_vacuum_buffer_id = buffers.size() - vacuum_count; - - // remove all invalid buffers from the available buffer list to ensure that we do not reuse them - auto it = buffers_with_free_space.begin(); - while (it != buffers_with_free_space.end()) { - if (*it >= min_vacuum_buffer_id) { - it = buffers_with_free_space.erase(it); - } else { - it++; - } - } - - return true; -} - -void FixedSizeAllocator::FinalizeVacuum() { - - // free all (now unused) buffers - while (min_vacuum_buffer_id < buffers.size()) { - allocator.FreeData(buffers.back().ptr, BUFFER_ALLOC_SIZE); - buffers.pop_back(); - } -} - -Node FixedSizeAllocator::VacuumPointer(const Node ptr) { - - // we do not need to adjust the bitmask of the old buffer, because we will free the entire - // buffer after the vacuum operation - - auto new_ptr = New(); - - // new increases the allocation count - total_allocations--; - - memcpy(Get(new_ptr), Get(ptr), allocation_size); - return new_ptr; -} - -void FixedSizeAllocator::Verify() const { -#ifdef DEBUG - auto total_available_allocations = allocations_per_buffer * buffers.size(); - D_ASSERT(total_available_allocations >= total_allocations); - D_ASSERT(buffers.size() >= buffers_with_free_space.size()); -#endif -} - -uint32_t FixedSizeAllocator::GetOffset(ValidityMask &mask, const idx_t allocation_count) { - - auto data = mask.GetData(); - - // fills up a buffer sequentially before searching for free bits - if (mask.RowIsValid(allocation_count)) { - mask.SetInvalid(allocation_count); - return allocation_count; - } - - // get an entry with free bits - for (idx_t entry_idx = 0; entry_idx < bitmask_count; entry_idx++) { - if (data[entry_idx] != 0) { - - // find the position of the free bit - auto entry = data[entry_idx]; - idx_t first_valid_bit = 0; - - // this loop finds the position of the rightmost set bit in entry and stores it - // in first_valid_bit - for (idx_t i = 0; i < 6; i++) { - // set the left half of the bits of this level to zero and test if the entry is still not zero - if (entry & BASE[i]) { - // first valid bit is in the rightmost s[i] bits - // permanently set the left half of the bits to zero - entry &= BASE[i]; - } else { - // first valid bit is in the leftmost s[i] bits - // shift by s[i] for the next iteration and add s[i] to the position of the rightmost set bit - entry >>= SHIFT[i]; - first_valid_bit += SHIFT[i]; - } - } - D_ASSERT(entry); - - auto prev_bits = entry_idx * sizeof(validity_t) * 8; - D_ASSERT(mask.RowIsValid(prev_bits + first_valid_bit)); - mask.SetInvalid(prev_bits + first_valid_bit); - return (prev_bits + first_valid_bit); - } - } - - throw InternalException("Invalid bitmask of FixedSizeAllocator"); -} - -} // namespace duckdb diff --git a/src/duckdb/src/execution/index/art/iterator.cpp b/src/duckdb/src/execution/index/art/iterator.cpp index 68e71c906..0d0290eb9 100644 --- a/src/duckdb/src/execution/index/art/iterator.cpp +++ b/src/duckdb/src/execution/index/art/iterator.cpp @@ -7,34 +7,34 @@ namespace duckdb { -bool IteratorKey::operator>(const ARTKey &k) const { - for (idx_t i = 0; i < MinValue(key_bytes.size(), k.len); i++) { - if (key_bytes[i] > k.data[i]) { +bool IteratorKey::operator>(const ARTKey &key) const { + for (idx_t i = 0; i < MinValue(key_bytes.size(), key.len); i++) { + if (key_bytes[i] > key.data[i]) { return true; - } else if (key_bytes[i] < k.data[i]) { + } else if (key_bytes[i] < key.data[i]) { return false; } } - return key_bytes.size() > k.len; + return key_bytes.size() > key.len; } -bool IteratorKey::operator>=(const ARTKey &k) const { - for (idx_t i = 0; i < MinValue(key_bytes.size(), k.len); i++) { - if (key_bytes[i] > k.data[i]) { +bool IteratorKey::operator>=(const ARTKey &key) const { + for (idx_t i = 0; i < MinValue(key_bytes.size(), key.len); i++) { + if (key_bytes[i] > key.data[i]) { return true; - } else if (key_bytes[i] < k.data[i]) { + } else if (key_bytes[i] < key.data[i]) { return false; } } - return key_bytes.size() >= k.len; + return key_bytes.size() >= key.len; } -bool IteratorKey::operator==(const ARTKey &k) const { +bool IteratorKey::operator==(const ARTKey &key) const { // NOTE: we only use this for finding the LowerBound, in which case the length // has to be equal - D_ASSERT(key_bytes.size() == k.len); + D_ASSERT(key_bytes.size() == key.len); for (idx_t i = 0; i < key_bytes.size(); i++) { - if (key_bytes[i] != k.data[i]) { + if (key_bytes[i] != key.data[i]) { return false; } } @@ -71,12 +71,9 @@ bool Iterator::Scan(const ARTKey &upper_bound, const idx_t max_count, vector(*art, node, NType::PREFIX); for (idx_t i = 0; i < prefix.data[Node::PREFIX_SIZE]; i++) { current_key.Push(prefix.data[i]); } @@ -103,16 +100,12 @@ void Iterator::FindMinimum(Node &node) { FindMinimum(*next); } -bool Iterator::LowerBound(Node &node, const ARTKey &key, const bool equal, idx_t depth) { +bool Iterator::LowerBound(const Node &node, const ARTKey &key, const bool equal, idx_t depth) { - if (!node.IsSet()) { + if (!node.HasMetadata()) { return false; } - if (node.IsSerialized()) { - node.Deserialize(*art); - } - // we found the lower bound if (node.GetType() == NType::LEAF || node.GetType() == NType::LEAF_INLINED) { if (!equal && current_key == key) { @@ -145,7 +138,7 @@ bool Iterator::LowerBound(Node &node, const ARTKey &key, const bool equal, idx_t } // resolve the prefix - auto &prefix = Prefix::Get(*art, node); + auto &prefix = Node::Ref(*art, node, NType::PREFIX); for (idx_t i = 0; i < prefix.data[Node::PREFIX_SIZE]; i++) { current_key.Push(prefix.data[i]); } @@ -206,7 +199,8 @@ bool Iterator::Next() { void Iterator::PopNode() { if (nodes.top().node.GetType() == NType::PREFIX) { - auto prefix_byte_count = Prefix::Get(*art, nodes.top().node).data[Node::PREFIX_SIZE]; + auto &prefix = Node::Ref(*art, nodes.top().node, NType::PREFIX); + auto prefix_byte_count = prefix.data[Node::PREFIX_SIZE]; current_key.Pop(prefix_byte_count); } else { current_key.Pop(1); diff --git a/src/duckdb/src/execution/index/art/leaf.cpp b/src/duckdb/src/execution/index/art/leaf.cpp index 107e7b8a2..fbc217337 100644 --- a/src/duckdb/src/execution/index/art/leaf.cpp +++ b/src/duckdb/src/execution/index/art/leaf.cpp @@ -2,8 +2,6 @@ #include "duckdb/execution/index/art/art.hpp" #include "duckdb/execution/index/art/node.hpp" -#include "duckdb/storage/metadata/metadata_reader.hpp" -#include "duckdb/storage/metadata/metadata_writer.hpp" namespace duckdb { @@ -11,8 +9,8 @@ void Leaf::New(Node &node, const row_t row_id) { // we directly inline this row ID into the node pointer D_ASSERT(row_id < MAX_ROW_ID_LOCAL); - node.Reset(); - node.SetType((uint8_t)NType::LEAF_INLINED); + node.Clear(); + node.SetMetadata(static_cast(NType::LEAF_INLINED)); node.SetRowId(row_id); } @@ -23,9 +21,9 @@ void Leaf::New(ART &art, reference &node, const row_t *row_ids, idx_t coun idx_t copy_count = 0; while (count) { node.get() = Node::GetAllocator(art, NType::LEAF).New(); - node.get().SetType((uint8_t)NType::LEAF); + node.get().SetMetadata(static_cast(NType::LEAF)); - auto &leaf = Leaf::Get(art, node); + auto &leaf = Node::RefMutable(art, node, NType::LEAF); leaf.count = MinValue((idx_t)Node::LEAF_SIZE, count); for (idx_t i = 0; i < leaf.count; i++) { @@ -36,7 +34,7 @@ void Leaf::New(ART &art, reference &node, const row_t *row_ids, idx_t coun count -= leaf.count; node = leaf.ptr; - leaf.ptr.Reset(); + leaf.ptr.Clear(); } } @@ -44,40 +42,39 @@ void Leaf::Free(ART &art, Node &node) { Node current_node = node; Node next_node; - while (current_node.IsSet() && !current_node.IsSerialized()) { - next_node = Leaf::Get(art, current_node).ptr; + while (current_node.HasMetadata()) { + next_node = Node::RefMutable(art, current_node, NType::LEAF).ptr; Node::GetAllocator(art, NType::LEAF).Free(current_node); current_node = next_node; } - node.Reset(); + node.Clear(); } void Leaf::InitializeMerge(ART &art, Node &node, const ARTFlags &flags) { - auto merge_buffer_count = flags.merge_buffer_counts[(uint8_t)NType::LEAF - 1]; + auto merge_buffer_count = flags.merge_buffer_counts[static_cast(NType::LEAF) - 1]; Node next_node = node; - node.AddToBufferID(merge_buffer_count); + node.IncreaseBufferId(merge_buffer_count); - while (next_node.IsSet()) { - auto &leaf = Leaf::Get(art, next_node); + while (next_node.HasMetadata()) { + auto &leaf = Node::RefMutable(art, next_node, NType::LEAF); next_node = leaf.ptr; - if (leaf.ptr.IsSet()) { - leaf.ptr.AddToBufferID(merge_buffer_count); + if (leaf.ptr.HasMetadata()) { + leaf.ptr.IncreaseBufferId(merge_buffer_count); } } } void Leaf::Merge(ART &art, Node &l_node, Node &r_node) { - D_ASSERT(l_node.IsSet() && !l_node.IsSerialized()); - D_ASSERT(r_node.IsSet() && !r_node.IsSerialized()); + D_ASSERT(l_node.HasMetadata() && r_node.HasMetadata()); // copy inlined row ID of r_node if (r_node.GetType() == NType::LEAF_INLINED) { - Leaf::Insert(art, l_node, r_node.GetRowId()); - r_node.Reset(); + Insert(art, l_node, r_node.GetRowId()); + r_node.Clear(); return; } @@ -85,8 +82,8 @@ void Leaf::Merge(ART &art, Node &l_node, Node &r_node) { if (l_node.GetType() == NType::LEAF_INLINED) { auto row_id = l_node.GetRowId(); l_node = r_node; - Leaf::Insert(art, l_node, row_id); - r_node.Reset(); + Insert(art, l_node, row_id); + r_node.Clear(); return; } @@ -94,33 +91,33 @@ void Leaf::Merge(ART &art, Node &l_node, Node &r_node) { D_ASSERT(r_node.GetType() != NType::LEAF_INLINED); reference l_node_ref(l_node); - reference l_leaf = Leaf::Get(art, l_node_ref); + reference l_leaf = Node::RefMutable(art, l_node_ref, NType::LEAF); // find a non-full node while (l_leaf.get().count == Node::LEAF_SIZE) { l_node_ref = l_leaf.get().ptr; // the last leaf is full - if (!l_leaf.get().ptr.IsSet()) { + if (!l_leaf.get().ptr.HasMetadata()) { break; } - l_leaf = Leaf::Get(art, l_node_ref); + l_leaf = Node::RefMutable(art, l_node_ref, NType::LEAF); } // store the last leaf and then append r_node auto last_leaf_node = l_node_ref.get(); l_node_ref.get() = r_node; - r_node.Reset(); + r_node.Clear(); // append the remaining row IDs of the last leaf node - if (last_leaf_node.IsSet()) { + if (last_leaf_node.HasMetadata()) { // find the tail - l_leaf = Leaf::Get(art, l_node_ref); - while (l_leaf.get().ptr.IsSet()) { - l_leaf = Leaf::Get(art, l_leaf.get().ptr); + l_leaf = Node::RefMutable(art, l_node_ref, NType::LEAF); + while (l_leaf.get().ptr.HasMetadata()) { + l_leaf = Node::RefMutable(art, l_leaf.get().ptr, NType::LEAF); } // append the row IDs - auto &last_leaf = Leaf::Get(art, last_leaf_node); + auto &last_leaf = Node::RefMutable(art, last_leaf_node, NType::LEAF); for (idx_t i = 0; i < last_leaf.count; i++) { l_leaf = l_leaf.get().Append(art, last_leaf.row_ids[i]); } @@ -130,28 +127,25 @@ void Leaf::Merge(ART &art, Node &l_node, Node &r_node) { void Leaf::Insert(ART &art, Node &node, const row_t row_id) { - D_ASSERT(node.IsSet() && !node.IsSerialized()); + D_ASSERT(node.HasMetadata()); if (node.GetType() == NType::LEAF_INLINED) { - Leaf::MoveInlinedToLeaf(art, node); - Leaf::Insert(art, node, row_id); + MoveInlinedToLeaf(art, node); + Insert(art, node, row_id); return; } // append to the tail - reference leaf = Leaf::Get(art, node); - while (leaf.get().ptr.IsSet()) { - if (leaf.get().ptr.IsSerialized()) { - leaf.get().ptr.Deserialize(art); - } - leaf = Leaf::Get(art, leaf.get().ptr); + reference leaf = Node::RefMutable(art, node, NType::LEAF); + while (leaf.get().ptr.HasMetadata()) { + leaf = Node::RefMutable(art, leaf.get().ptr, NType::LEAF); } leaf.get().Append(art, row_id); } bool Leaf::Remove(ART &art, reference &node, const row_t row_id) { - D_ASSERT(node.get().IsSet() && !node.get().IsSerialized()); + D_ASSERT(node.get().HasMetadata()); if (node.get().GetType() == NType::LEAF_INLINED) { if (node.get().GetRowId() == row_id) { @@ -160,14 +154,14 @@ bool Leaf::Remove(ART &art, reference &node, const row_t row_id) { return false; } - reference leaf = Leaf::Get(art, node); + reference leaf = Node::RefMutable(art, node, NType::LEAF); // inline the remaining row ID if (leaf.get().count == 2) { if (leaf.get().row_ids[0] == row_id || leaf.get().row_ids[1] == row_id) { auto remaining_row_id = leaf.get().row_ids[0] == row_id ? leaf.get().row_ids[1] : leaf.get().row_ids[0]; Node::Free(art, node); - Leaf::New(node, remaining_row_id); + New(node, remaining_row_id); } return false; } @@ -177,12 +171,9 @@ bool Leaf::Remove(ART &art, reference &node, const row_t row_id) { // go to the tail and keep track of the previous leaf node reference prev_leaf(leaf); - while (leaf.get().ptr.IsSet()) { + while (leaf.get().ptr.HasMetadata()) { prev_leaf = leaf; - if (leaf.get().ptr.IsSerialized()) { - leaf.get().ptr.Deserialize(art); - } - leaf = Leaf::Get(art, leaf.get().ptr); + leaf = Node::RefMutable(art, leaf.get().ptr, NType::LEAF); } auto last_idx = leaf.get().count; @@ -199,9 +190,8 @@ bool Leaf::Remove(ART &art, reference &node, const row_t row_id) { } // find the row ID and copy the last row ID to that position - while (node.get().IsSet()) { - D_ASSERT(!node.get().IsSerialized()); - leaf = Leaf::Get(art, node); + while (node.get().HasMetadata()) { + leaf = Node::RefMutable(art, node, NType::LEAF); for (idx_t i = 0; i < leaf.get().count; i++) { if (leaf.get().row_ids[i] == row_id) { leaf.get().row_ids[i] = last_row_id; @@ -213,54 +203,43 @@ bool Leaf::Remove(ART &art, reference &node, const row_t row_id) { return false; } -idx_t Leaf::TotalCount(ART &art, Node &node) { - - // NOTE: first leaf in the leaf chain is already deserialized - D_ASSERT(node.IsSet() && !node.IsSerialized()); +idx_t Leaf::TotalCount(ART &art, const Node &node) { + D_ASSERT(node.HasMetadata()); if (node.GetType() == NType::LEAF_INLINED) { return 1; } idx_t count = 0; - reference node_ref(node); - while (node_ref.get().IsSet()) { - auto &leaf = Leaf::Get(art, node_ref); + reference node_ref(node); + while (node_ref.get().HasMetadata()) { + auto &leaf = Node::Ref(art, node_ref, NType::LEAF); count += leaf.count; - - if (leaf.ptr.IsSerialized()) { - leaf.ptr.Deserialize(art); - } node_ref = leaf.ptr; } return count; } -bool Leaf::GetRowIds(ART &art, Node &node, vector &result_ids, idx_t max_count) { +bool Leaf::GetRowIds(ART &art, const Node &node, vector &result_ids, idx_t max_count) { // adding more elements would exceed the maximum count - D_ASSERT(node.IsSet()); - if (result_ids.size() + Leaf::TotalCount(art, node) > max_count) { + D_ASSERT(node.HasMetadata()); + if (result_ids.size() + TotalCount(art, node) > max_count) { return false; } - // NOTE: Leaf::TotalCount fully deserializes the leaf - D_ASSERT(!node.IsSerialized()); - if (node.GetType() == NType::LEAF_INLINED) { // push back the inlined row ID of this leaf result_ids.push_back(node.GetRowId()); } else { // push back all the row IDs of this leaf - reference last_leaf_ref(node); - while (last_leaf_ref.get().IsSet()) { - auto &leaf = Leaf::Get(art, last_leaf_ref); + reference last_leaf_ref(node); + while (last_leaf_ref.get().HasMetadata()) { + auto &leaf = Node::Ref(art, last_leaf_ref, NType::LEAF); for (idx_t i = 0; i < leaf.count; i++) { result_ids.push_back(leaf.row_ids[i]); } - - D_ASSERT(!leaf.ptr.IsSerialized()); last_leaf_ref = leaf.ptr; } } @@ -268,45 +247,40 @@ bool Leaf::GetRowIds(ART &art, Node &node, vector &result_ids, idx_t max_ return true; } -bool Leaf::ContainsRowId(ART &art, Node &node, const row_t row_id) { +bool Leaf::ContainsRowId(ART &art, const Node &node, const row_t row_id) { - // NOTE: we either just removed a row ID from this leaf (by copying the - // last row ID at a different position) or inserted a row ID into this leaf - // (at the end), so the whole leaf is deserialized - D_ASSERT(node.IsSet() && !node.IsSerialized()); + D_ASSERT(node.HasMetadata()); if (node.GetType() == NType::LEAF_INLINED) { return node.GetRowId() == row_id; } - reference ref_node(node); - while (ref_node.get().IsSet()) { - auto &leaf = Leaf::Get(art, ref_node); + reference ref_node(node); + while (ref_node.get().HasMetadata()) { + auto &leaf = Node::Ref(art, ref_node, NType::LEAF); for (idx_t i = 0; i < leaf.count; i++) { if (leaf.row_ids[i] == row_id) { return true; } } - - D_ASSERT(!leaf.ptr.IsSerialized()); ref_node = leaf.ptr; } return false; } -string Leaf::VerifyAndToString(ART &art, Node &node) { +string Leaf::VerifyAndToString(ART &art, const Node &node, const bool only_verify) { if (node.GetType() == NType::LEAF_INLINED) { - return "Leaf [count: 1, row ID: " + to_string(node.GetRowId()) + "]"; + return only_verify ? "" : "Leaf [count: 1, row ID: " + to_string(node.GetRowId()) + "]"; } string str = ""; - reference node_ref(node); - while (node_ref.get().IsSet()) { + reference node_ref(node); + while (node_ref.get().HasMetadata()) { - auto &leaf = Leaf::Get(art, node_ref); + auto &leaf = Node::Ref(art, node_ref, NType::LEAF); D_ASSERT(leaf.count <= Node::LEAF_SIZE); str += "Leaf [count: " + to_string(leaf.count) + ", row IDs: "; @@ -315,64 +289,9 @@ string Leaf::VerifyAndToString(ART &art, Node &node) { } str += "] "; - // NOTE: we are currently only calling this function during CREATE INDEX - // statements (and debugging), so the index is never serialized - D_ASSERT(!leaf.ptr.IsSerialized()); node_ref = leaf.ptr; } - return str; -} - -BlockPointer Leaf::Serialize(ART &art, Node &node, MetadataWriter &writer) { - - if (node.GetType() == NType::LEAF_INLINED) { - auto block_pointer = writer.GetBlockPointer(); - writer.Write(NType::LEAF_INLINED); - writer.Write(node.GetRowId()); - return block_pointer; - } - - auto block_pointer = writer.GetBlockPointer(); - writer.Write(NType::LEAF); - idx_t total_count = Leaf::TotalCount(art, node); - writer.Write(total_count); - - // iterate all leaves and write their row IDs - reference ref_node(node); - while (ref_node.get().IsSet()) { - D_ASSERT(!ref_node.get().IsSerialized()); - auto &leaf = Leaf::Get(art, ref_node); - - // write row IDs - for (idx_t i = 0; i < leaf.count; i++) { - writer.Write(leaf.row_ids[i]); - } - ref_node = leaf.ptr; - } - - return block_pointer; -} - -void Leaf::Deserialize(ART &art, Node &node, MetadataReader &reader) { - - auto total_count = reader.Read(); - reference ref_node(node); - - while (total_count) { - ref_node.get() = Node::GetAllocator(art, NType::LEAF).New(); - ref_node.get().SetType((uint8_t)NType::LEAF); - - auto &leaf = Leaf::Get(art, ref_node); - - leaf.count = MinValue((idx_t)Node::LEAF_SIZE, total_count); - for (idx_t i = 0; i < leaf.count; i++) { - leaf.row_ids[i] = reader.Read(); - } - - total_count -= leaf.count; - ref_node = leaf.ptr; - leaf.ptr.Reset(); - } + return only_verify ? "" : str; } void Leaf::Vacuum(ART &art, Node &node) { @@ -380,12 +299,12 @@ void Leaf::Vacuum(ART &art, Node &node) { auto &allocator = Node::GetAllocator(art, NType::LEAF); reference node_ref(node); - while (node_ref.get().IsSet() && !node_ref.get().IsSerialized()) { + while (node_ref.get().HasMetadata()) { if (allocator.NeedsVacuum(node_ref)) { node_ref.get() = allocator.VacuumPointer(node_ref); - node_ref.get().SetType((uint8_t)NType::LEAF); + node_ref.get().SetMetadata(static_cast(NType::LEAF)); } - auto &leaf = Leaf::Get(art, node_ref); + auto &leaf = Node::RefMutable(art, node_ref, NType::LEAF); node_ref = leaf.ptr; } } @@ -395,12 +314,12 @@ void Leaf::MoveInlinedToLeaf(ART &art, Node &node) { D_ASSERT(node.GetType() == NType::LEAF_INLINED); auto row_id = node.GetRowId(); node = Node::GetAllocator(art, NType::LEAF).New(); - node.SetType((uint8_t)NType::LEAF); + node.SetMetadata(static_cast(NType::LEAF)); - auto &leaf = Leaf::Get(art, node); + auto &leaf = Node::RefMutable(art, node, NType::LEAF); leaf.count = 1; leaf.row_ids[0] = row_id; - leaf.ptr.Reset(); + leaf.ptr.Clear(); } Leaf &Leaf::Append(ART &art, const row_t row_id) { @@ -410,11 +329,11 @@ Leaf &Leaf::Append(ART &art, const row_t row_id) { // we need a new leaf node if (leaf.get().count == Node::LEAF_SIZE) { leaf.get().ptr = Node::GetAllocator(art, NType::LEAF).New(); - leaf.get().ptr.SetType((uint8_t)NType::LEAF); + leaf.get().ptr.SetMetadata(static_cast(NType::LEAF)); - leaf = Leaf::Get(art, leaf.get().ptr); + leaf = Node::RefMutable(art, leaf.get().ptr, NType::LEAF); leaf.get().count = 0; - leaf.get().ptr.Reset(); + leaf.get().ptr.Clear(); } leaf.get().row_ids[leaf.get().count] = row_id; diff --git a/src/duckdb/src/execution/index/art/node.cpp b/src/duckdb/src/execution/index/art/node.cpp index d6dc005c8..5c82b7483 100644 --- a/src/duckdb/src/execution/index/art/node.cpp +++ b/src/duckdb/src/execution/index/art/node.cpp @@ -9,29 +9,10 @@ #include "duckdb/execution/index/art/node4.hpp" #include "duckdb/execution/index/art/leaf.hpp" #include "duckdb/execution/index/art/prefix.hpp" -#include "duckdb/storage/metadata/metadata_reader.hpp" -#include "duckdb/storage/metadata/metadata_writer.hpp" #include "duckdb/storage/table_io_manager.hpp" namespace duckdb { -//===--------------------------------------------------------------------===// -// Constructors / Destructors -//===--------------------------------------------------------------------===// - -Node::Node(MetadataReader &reader) { - block_id_t block_id = reader.Read(); - auto offset = reader.Read(); - Reset(); - - if (block_id == INVALID_BLOCK) { - return; - } - - SetSerialized(); - SetPtr(block_id, offset); -} - //===--------------------------------------------------------------------===// // New / Free //===--------------------------------------------------------------------===// @@ -60,61 +41,62 @@ void Node::New(ART &art, Node &node, const NType type) { void Node::Free(ART &art, Node &node) { - if (!node.IsSet()) { - return; + if (!node.HasMetadata()) { + return node.Clear(); } - if (!node.IsSerialized()) { - - // free the children of the nodes - auto type = node.GetType(); - switch (type) { - case NType::PREFIX: - // iterative - return Prefix::Free(art, node); - case NType::LEAF: - // iterative - return Leaf::Free(art, node); - case NType::NODE_4: - Node4::Free(art, node); - break; - case NType::NODE_16: - Node16::Free(art, node); - break; - case NType::NODE_48: - Node48::Free(art, node); - break; - case NType::NODE_256: - Node256::Free(art, node); - break; - case NType::LEAF_INLINED: - return node.Reset(); - } - - Node::GetAllocator(art, type).Free(node); + // free the children of the nodes + auto type = node.GetType(); + switch (type) { + case NType::PREFIX: + // iterative + return Prefix::Free(art, node); + case NType::LEAF: + // iterative + return Leaf::Free(art, node); + case NType::NODE_4: + Node4::Free(art, node); + break; + case NType::NODE_16: + Node16::Free(art, node); + break; + case NType::NODE_48: + Node48::Free(art, node); + break; + case NType::NODE_256: + Node256::Free(art, node); + break; + case NType::LEAF_INLINED: + return node.Clear(); } - // overwrite with an empty ART node - node.Reset(); + GetAllocator(art, type).Free(node); + node.Clear(); } //===--------------------------------------------------------------------===// -// Inserts +// Get Allocators //===--------------------------------------------------------------------===// -void Node::ReplaceChild(const ART &art, const uint8_t byte, const Node child) { +FixedSizeAllocator &Node::GetAllocator(const ART &art, const NType type) { + return *(*art.allocators)[static_cast(type) - 1]; +} + +//===--------------------------------------------------------------------===// +// Inserts +//===--------------------------------------------------------------------===// - D_ASSERT(!IsSerialized()); +void Node::ReplaceChild(const ART &art, const uint8_t byte, const Node child) const { switch (GetType()) { case NType::NODE_4: - return Node4::Get(art, *this).ReplaceChild(byte, child); + return RefMutable(art, *this, NType::NODE_4).ReplaceChild(byte, child); case NType::NODE_16: - return Node16::Get(art, *this).ReplaceChild(byte, child); + return RefMutable(art, *this, NType::NODE_16).ReplaceChild(byte, child); case NType::NODE_48: - return Node48::Get(art, *this).ReplaceChild(byte, child); + return RefMutable(art, *this, NType::NODE_48).ReplaceChild(byte, child); case NType::NODE_256: - return Node256::Get(art, *this).ReplaceChild(byte, child); + return RefMutable(art, *this, NType::NODE_256).ReplaceChild(byte, child); default: throw InternalException("Invalid node type for ReplaceChild."); } @@ -160,133 +142,75 @@ void Node::DeleteChild(ART &art, Node &node, Node &prefix, const uint8_t byte) { // Get functions //===--------------------------------------------------------------------===// -optional_ptr Node::GetChild(ART &art, const uint8_t byte) const { +optional_ptr Node::GetChild(ART &art, const uint8_t byte) const { - D_ASSERT(IsSet() && !IsSerialized()); + D_ASSERT(HasMetadata()); - optional_ptr child; switch (GetType()) { case NType::NODE_4: - child = Node4::Get(art, *this).GetChild(byte); - break; + return Ref(art, *this, NType::NODE_4).GetChild(byte); case NType::NODE_16: - child = Node16::Get(art, *this).GetChild(byte); - break; + return Ref(art, *this, NType::NODE_16).GetChild(byte); case NType::NODE_48: - child = Node48::Get(art, *this).GetChild(byte); - break; + return Ref(art, *this, NType::NODE_48).GetChild(byte); case NType::NODE_256: - child = Node256::Get(art, *this).GetChild(byte); - break; + return Ref(art, *this, NType::NODE_256).GetChild(byte); default: throw InternalException("Invalid node type for GetChild."); } - - // deserialize the ART node before returning it - if (child && child->IsSerialized()) { - child->Deserialize(art); - } - return child; } -optional_ptr Node::GetNextChild(ART &art, uint8_t &byte, const bool deserialize) const { +optional_ptr Node::GetChildMutable(ART &art, const uint8_t byte) const { - D_ASSERT(IsSet() && !IsSerialized()); + D_ASSERT(HasMetadata()); - optional_ptr child; switch (GetType()) { case NType::NODE_4: - child = Node4::Get(art, *this).GetNextChild(byte); - break; + return RefMutable(art, *this, NType::NODE_4).GetChildMutable(byte); case NType::NODE_16: - child = Node16::Get(art, *this).GetNextChild(byte); - break; + return RefMutable(art, *this, NType::NODE_16).GetChildMutable(byte); case NType::NODE_48: - child = Node48::Get(art, *this).GetNextChild(byte); - break; + return RefMutable(art, *this, NType::NODE_48).GetChildMutable(byte); case NType::NODE_256: - child = Node256::Get(art, *this).GetNextChild(byte); - break; + return RefMutable(art, *this, NType::NODE_256).GetChildMutable(byte); default: - throw InternalException("Invalid node type for GetNextChild."); + throw InternalException("Invalid node type for GetChildMutable."); } - - // deserialize the ART node before returning it - if (child && deserialize && child->IsSerialized()) { - child->Deserialize(art); - } - return child; } -//===--------------------------------------------------------------------===// -// (De)serialization -//===--------------------------------------------------------------------===// +optional_ptr Node::GetNextChild(ART &art, uint8_t &byte) const { -BlockPointer Node::Serialize(ART &art, MetadataWriter &writer) { - if (!IsSet()) { - return BlockPointer(); - } - if (IsSerialized()) { - Deserialize(art); - } + D_ASSERT(HasMetadata()); switch (GetType()) { - case NType::PREFIX: - // iterative - return Prefix::Serialize(art, *this, writer); - case NType::LEAF: - // iterative - return Leaf::Serialize(art, *this, writer); case NType::NODE_4: - return Node4::Get(art, *this).Serialize(art, writer); + return Ref(art, *this, NType::NODE_4).GetNextChild(byte); case NType::NODE_16: - return Node16::Get(art, *this).Serialize(art, writer); + return Ref(art, *this, NType::NODE_16).GetNextChild(byte); case NType::NODE_48: - return Node48::Get(art, *this).Serialize(art, writer); + return Ref(art, *this, NType::NODE_48).GetNextChild(byte); case NType::NODE_256: - return Node256::Get(art, *this).Serialize(art, writer); - case NType::LEAF_INLINED: - return Leaf::Serialize(art, *this, writer); + return Ref(art, *this, NType::NODE_256).GetNextChild(byte); + default: + throw InternalException("Invalid node type for GetNextChild."); } - throw InternalException("Invalid node type for Serialize."); } -void Node::Deserialize(ART &art) { - D_ASSERT(IsSet() && IsSerialized()); +optional_ptr Node::GetNextChildMutable(ART &art, uint8_t &byte) const { - BlockPointer pointer(GetBufferId(), GetOffset()); - MetadataReader reader(art.table_io_manager.GetMetadataManager(), pointer); - Reset(); - SetType(reader.Read()); + D_ASSERT(HasMetadata()); - auto decoded_type = GetType(); - - // iterative functions - if (decoded_type == NType::PREFIX) { - return Prefix::Deserialize(art, *this, reader); - } - if (decoded_type == NType::LEAF_INLINED) { - return SetRowId(reader.Read()); - } - if (decoded_type == NType::LEAF) { - return Leaf::Deserialize(art, *this, reader); - } - - *this = Node::GetAllocator(art, decoded_type).New(); - SetType((uint8_t)decoded_type); - - // recursive functions - switch (decoded_type) { + switch (GetType()) { case NType::NODE_4: - return Node4::Get(art, *this).Deserialize(reader); + return RefMutable(art, *this, NType::NODE_4).GetNextChildMutable(byte); case NType::NODE_16: - return Node16::Get(art, *this).Deserialize(reader); + return RefMutable(art, *this, NType::NODE_16).GetNextChildMutable(byte); case NType::NODE_48: - return Node48::Get(art, *this).Deserialize(reader); + return RefMutable(art, *this, NType::NODE_48).GetNextChildMutable(byte); case NType::NODE_256: - return Node256::Get(art, *this).Deserialize(reader); + return RefMutable(art, *this, NType::NODE_256).GetNextChildMutable(byte); default: - throw InternalException("Invalid node type for Deserialize."); + throw InternalException("Invalid node type for GetNextChildMutable."); } } @@ -294,40 +218,31 @@ void Node::Deserialize(ART &art) { // Utility //===--------------------------------------------------------------------===// -string Node::VerifyAndToString(ART &art, const bool only_verify) { +string Node::VerifyAndToString(ART &art, const bool only_verify) const { - D_ASSERT(IsSet()); - if (IsSerialized()) { - return only_verify ? "" : "serialized"; - } + D_ASSERT(HasMetadata()); if (GetType() == NType::LEAF || GetType() == NType::LEAF_INLINED) { - auto str = Leaf::VerifyAndToString(art, *this); + auto str = Leaf::VerifyAndToString(art, *this, only_verify); return only_verify ? "" : "\n" + str; } if (GetType() == NType::PREFIX) { - auto str = Prefix::Get(art, *this).VerifyAndToString(art, *this, only_verify); + auto str = Prefix::VerifyAndToString(art, *this, only_verify); return only_verify ? "" : "\n" + str; } string str = "Node" + to_string(GetCapacity()) + ": ["; uint8_t byte = 0; - auto child = GetNextChild(art, byte, false); + auto child = GetNextChild(art, byte); while (child) { - if (child->IsSerialized()) { - if (!only_verify) { - str += "(serialized)"; - } - } else { - str += "(" + to_string(byte) + ", " + child->VerifyAndToString(art, only_verify) + ")"; - if (byte == NumericLimits::Maximum()) { - break; - } + str += "(" + to_string(byte) + ", " + child->VerifyAndToString(art, only_verify) + ")"; + if (byte == NumericLimits::Maximum()) { + break; } byte++; - child = GetNextChild(art, byte, false); + child = GetNextChild(art, byte); } return only_verify ? "" : "\n" + str + "]"; @@ -335,17 +250,15 @@ string Node::VerifyAndToString(ART &art, const bool only_verify) { idx_t Node::GetCapacity() const { - D_ASSERT(!IsSerialized()); - switch (GetType()) { case NType::NODE_4: - return Node::NODE_4_CAPACITY; + return NODE_4_CAPACITY; case NType::NODE_16: - return Node::NODE_16_CAPACITY; + return NODE_16_CAPACITY; case NType::NODE_48: - return Node::NODE_48_CAPACITY; + return NODE_48_CAPACITY; case NType::NODE_256: - return Node::NODE_256_CAPACITY; + return NODE_256_CAPACITY; default: throw InternalException("Invalid node type for GetCapacity."); } @@ -363,18 +276,13 @@ NType Node::GetARTNodeTypeByCount(const idx_t count) { return NType::NODE_256; } -FixedSizeAllocator &Node::GetAllocator(const ART &art, NType type) { - return (*art.allocators)[(uint8_t)type - 1]; -} - //===--------------------------------------------------------------------===// // Merging //===--------------------------------------------------------------------===// void Node::InitializeMerge(ART &art, const ARTFlags &flags) { - // the index is fully in memory during CREATE [UNIQUE] INDEX statements - D_ASSERT(IsSet() && !IsSerialized()); + D_ASSERT(HasMetadata()); switch (GetType()) { case NType::PREFIX: @@ -384,27 +292,27 @@ void Node::InitializeMerge(ART &art, const ARTFlags &flags) { // iterative return Leaf::InitializeMerge(art, *this, flags); case NType::NODE_4: - Node4::Get(art, *this).InitializeMerge(art, flags); + RefMutable(art, *this, NType::NODE_4).InitializeMerge(art, flags); break; case NType::NODE_16: - Node16::Get(art, *this).InitializeMerge(art, flags); + RefMutable(art, *this, NType::NODE_16).InitializeMerge(art, flags); break; case NType::NODE_48: - Node48::Get(art, *this).InitializeMerge(art, flags); + RefMutable(art, *this, NType::NODE_48).InitializeMerge(art, flags); break; case NType::NODE_256: - Node256::Get(art, *this).InitializeMerge(art, flags); + RefMutable(art, *this, NType::NODE_256).InitializeMerge(art, flags); break; case NType::LEAF_INLINED: return; } - AddToBufferID(flags.merge_buffer_counts[(uint8_t)GetType() - 1]); + IncreaseBufferId(flags.merge_buffer_counts[static_cast(GetType()) - 1]); } bool Node::Merge(ART &art, Node &other) { - if (!IsSet()) { + if (!HasMetadata()) { *this = other; other = Node(); return true; @@ -423,7 +331,7 @@ bool MergePrefixContainsOtherPrefix(ART &art, reference &l_node, reference // test if the next byte (mismatch_position) in r_node (prefix) exists in l_node auto mismatch_byte = Prefix::GetByte(art, r_node, mismatch_position); - auto child_node = l_node.get().GetChild(art, mismatch_byte); + auto child_node = l_node.get().GetChildMutable(art, mismatch_byte); // update the prefix of r_node to only consist of the bytes after mismatch_position Prefix::Reduce(art, r_node, mismatch_position); @@ -431,7 +339,7 @@ bool MergePrefixContainsOtherPrefix(ART &art, reference &l_node, reference if (!child_node) { // insert r_node as a child of l_node at the empty position Node::InsertChild(art, l_node, mismatch_byte, r_node); - r_node.get().Reset(); + r_node.get().Clear(); return true; } @@ -454,14 +362,14 @@ void MergePrefixesDiffer(ART &art, reference &l_node, reference &r_n Prefix::Reduce(art, r_node, mismatch_position); Node4::InsertChild(art, l_node, r_byte, r_node); - r_node.get().Reset(); + r_node.get().Clear(); } bool Node::ResolvePrefixes(ART &art, Node &other) { // NOTE: we always merge into the left ART - D_ASSERT(IsSet() && other.IsSet()); + D_ASSERT(HasMetadata() && other.HasMetadata()); // case 1: both nodes have no prefix if (GetType() != NType::PREFIX && other.GetType() != NType::PREFIX) { @@ -506,7 +414,7 @@ bool Node::ResolvePrefixes(ART &art, Node &other) { bool Node::MergeInternal(ART &art, Node &other) { - D_ASSERT(IsSet() && other.IsSet()); + D_ASSERT(HasMetadata() && other.HasMetadata()); D_ASSERT(GetType() != NType::PREFIX && other.GetType() != NType::PREFIX); // always try to merge the smaller node into the bigger node @@ -532,14 +440,14 @@ bool Node::MergeInternal(ART &art, Node &other) { } uint8_t byte = 0; - auto r_child = r_node.GetNextChild(art, byte); + auto r_child = r_node.GetNextChildMutable(art, byte); // while r_node still has children to merge while (r_child) { - auto l_child = l_node.GetChild(art, byte); + auto l_child = l_node.GetChildMutable(art, byte); if (!l_child) { // insert child at empty byte - Node::InsertChild(art, l_node, byte, *r_child); + InsertChild(art, l_node, byte, *r_child); r_node.ReplaceChild(art, byte, empty_node); } else { @@ -553,10 +461,10 @@ bool Node::MergeInternal(ART &art, Node &other) { break; } byte++; - r_child = r_node.GetNextChild(art, byte); + r_child = r_node.GetNextChildMutable(art, byte); } - Node::Free(art, r_node); + Free(art, r_node); return true; } @@ -566,12 +474,10 @@ bool Node::MergeInternal(ART &art, Node &other) { void Node::Vacuum(ART &art, const ARTFlags &flags) { - D_ASSERT(IsSet()); - if (IsSerialized()) { - return; - } + D_ASSERT(HasMetadata()); auto node_type = GetType(); + auto node_type_idx = static_cast(node_type); // iterative functions if (node_type == NType::PREFIX) { @@ -581,29 +487,29 @@ void Node::Vacuum(ART &art, const ARTFlags &flags) { return; } if (node_type == NType::LEAF) { - if (flags.vacuum_flags[(uint8_t)node_type - 1]) { + if (flags.vacuum_flags[node_type_idx - 1]) { Leaf::Vacuum(art, *this); } return; } - auto &allocator = Node::GetAllocator(art, node_type); - auto needs_vacuum = flags.vacuum_flags[(uint8_t)GetType() - 1] && allocator.NeedsVacuum(*this); + auto &allocator = GetAllocator(art, node_type); + auto needs_vacuum = flags.vacuum_flags[node_type_idx - 1] && allocator.NeedsVacuum(*this); if (needs_vacuum) { *this = allocator.VacuumPointer(*this); - SetType((uint8_t)node_type); + SetMetadata(node_type_idx); } // recursive functions switch (node_type) { case NType::NODE_4: - return Node4::Get(art, *this).Vacuum(art, flags); + return RefMutable(art, *this, NType::NODE_4).Vacuum(art, flags); case NType::NODE_16: - return Node16::Get(art, *this).Vacuum(art, flags); + return RefMutable(art, *this, NType::NODE_16).Vacuum(art, flags); case NType::NODE_48: - return Node48::Get(art, *this).Vacuum(art, flags); + return RefMutable(art, *this, NType::NODE_48).Vacuum(art, flags); case NType::NODE_256: - return Node256::Get(art, *this).Vacuum(art, flags); + return RefMutable(art, *this, NType::NODE_256).Vacuum(art, flags); default: throw InternalException("Invalid node type for Vacuum."); } diff --git a/src/duckdb/src/execution/index/art/node16.cpp b/src/duckdb/src/execution/index/art/node16.cpp index e710bbdd1..4541c1244 100644 --- a/src/duckdb/src/execution/index/art/node16.cpp +++ b/src/duckdb/src/execution/index/art/node16.cpp @@ -2,16 +2,14 @@ #include "duckdb/execution/index/art/node4.hpp" #include "duckdb/execution/index/art/node48.hpp" -#include "duckdb/storage/metadata/metadata_reader.hpp" -#include "duckdb/storage/metadata/metadata_writer.hpp" namespace duckdb { Node16 &Node16::New(ART &art, Node &node) { node = Node::GetAllocator(art, NType::NODE_16).New(); - node.SetType((uint8_t)NType::NODE_16); - auto &n16 = Node16::Get(art, node); + node.SetMetadata(static_cast(NType::NODE_16)); + auto &n16 = Node::RefMutable(art, node, NType::NODE_16); n16.count = 0; return n16; @@ -19,10 +17,8 @@ Node16 &Node16::New(ART &art, Node &node) { void Node16::Free(ART &art, Node &node) { - D_ASSERT(node.IsSet()); - D_ASSERT(!node.IsSerialized()); - - auto &n16 = Node16::Get(art, node); + D_ASSERT(node.HasMetadata()); + auto &n16 = Node::RefMutable(art, node, NType::NODE_16); // free all children for (idx_t i = 0; i < n16.count; i++) { @@ -32,8 +28,8 @@ void Node16::Free(ART &art, Node &node) { Node16 &Node16::GrowNode4(ART &art, Node &node16, Node &node4) { - auto &n4 = Node4::Get(art, node4); - auto &n16 = Node16::New(art, node16); + auto &n4 = Node::RefMutable(art, node4, NType::NODE_4); + auto &n16 = New(art, node16); n16.count = n4.count; for (idx_t i = 0; i < n4.count; i++) { @@ -48,8 +44,8 @@ Node16 &Node16::GrowNode4(ART &art, Node &node16, Node &node4) { Node16 &Node16::ShrinkNode48(ART &art, Node &node16, Node &node48) { - auto &n16 = Node16::New(art, node16); - auto &n48 = Node48::Get(art, node48); + auto &n16 = New(art, node16); + auto &n48 = Node::RefMutable(art, node48, NType::NODE_48); n16.count = 0; for (idx_t i = 0; i < Node::NODE_256_CAPACITY; i++) { @@ -75,9 +71,8 @@ void Node16::InitializeMerge(ART &art, const ARTFlags &flags) { void Node16::InsertChild(ART &art, Node &node, const uint8_t byte, const Node child) { - D_ASSERT(node.IsSet()); - D_ASSERT(!node.IsSerialized()); - auto &n16 = Node16::Get(art, node); + D_ASSERT(node.HasMetadata()); + auto &n16 = Node::RefMutable(art, node, NType::NODE_16); // ensure that there is no other child at the same byte for (idx_t i = 0; i < n16.count; i++) { @@ -111,9 +106,8 @@ void Node16::InsertChild(ART &art, Node &node, const uint8_t byte, const Node ch void Node16::DeleteChild(ART &art, Node &node, const uint8_t byte) { - D_ASSERT(node.IsSet()); - D_ASSERT(!node.IsSerialized()); - auto &n16 = Node16::Get(art, node); + D_ASSERT(node.HasMetadata()); + auto &n16 = Node::RefMutable(art, node, NType::NODE_16); idx_t child_pos = 0; for (; child_pos < n16.count; child_pos++) { @@ -150,72 +144,46 @@ void Node16::ReplaceChild(const uint8_t byte, const Node child) { } } -optional_ptr Node16::GetChild(const uint8_t byte) { - +optional_ptr Node16::GetChild(const uint8_t byte) const { for (idx_t i = 0; i < count; i++) { if (key[i] == byte) { - D_ASSERT(children[i].IsSet()); + D_ASSERT(children[i].HasMetadata()); return &children[i]; } } return nullptr; } -optional_ptr Node16::GetNextChild(uint8_t &byte) { - +optional_ptr Node16::GetChildMutable(const uint8_t byte) { for (idx_t i = 0; i < count; i++) { - if (key[i] >= byte) { - byte = key[i]; - D_ASSERT(children[i].IsSet()); + if (key[i] == byte) { + D_ASSERT(children[i].HasMetadata()); return &children[i]; } } return nullptr; } -BlockPointer Node16::Serialize(ART &art, MetadataWriter &writer) { - - // recurse into children and retrieve child block pointers - vector child_block_pointers; +optional_ptr Node16::GetNextChild(uint8_t &byte) const { for (idx_t i = 0; i < count; i++) { - child_block_pointers.push_back(children[i].Serialize(art, writer)); - } - for (idx_t i = count; i < Node::NODE_16_CAPACITY; i++) { - child_block_pointers.emplace_back((block_id_t)DConstants::INVALID_INDEX, 0); - } - - // get pointer and write fields - auto block_pointer = writer.GetBlockPointer(); - writer.Write(NType::NODE_16); - writer.Write(count); - - // write key values - for (idx_t i = 0; i < Node::NODE_16_CAPACITY; i++) { - writer.Write(key[i]); - } - - // write child block pointers - for (auto &child_block_pointer : child_block_pointers) { - writer.Write(child_block_pointer.block_id); - writer.Write(child_block_pointer.offset); + if (key[i] >= byte) { + byte = key[i]; + D_ASSERT(children[i].HasMetadata()); + return &children[i]; + } } - - return block_pointer; + return nullptr; } -void Node16::Deserialize(MetadataReader &reader) { - - count = reader.Read(); - - // read key values - for (idx_t i = 0; i < Node::NODE_16_CAPACITY; i++) { - key[i] = reader.Read(); - } - - // read child block pointers - for (idx_t i = 0; i < Node::NODE_16_CAPACITY; i++) { - children[i] = Node(reader); +optional_ptr Node16::GetNextChildMutable(uint8_t &byte) { + for (idx_t i = 0; i < count; i++) { + if (key[i] >= byte) { + byte = key[i]; + D_ASSERT(children[i].HasMetadata()); + return &children[i]; + } } + return nullptr; } void Node16::Vacuum(ART &art, const ARTFlags &flags) { diff --git a/src/duckdb/src/execution/index/art/node256.cpp b/src/duckdb/src/execution/index/art/node256.cpp index 208ddc0b9..b4d82fece 100644 --- a/src/duckdb/src/execution/index/art/node256.cpp +++ b/src/duckdb/src/execution/index/art/node256.cpp @@ -1,20 +1,18 @@ #include "duckdb/execution/index/art/node256.hpp" #include "duckdb/execution/index/art/node48.hpp" -#include "duckdb/storage/metadata/metadata_reader.hpp" -#include "duckdb/storage/metadata/metadata_writer.hpp" namespace duckdb { Node256 &Node256::New(ART &art, Node &node) { node = Node::GetAllocator(art, NType::NODE_256).New(); - node.SetType((uint8_t)NType::NODE_256); - auto &n256 = Node256::Get(art, node); + node.SetMetadata(static_cast(NType::NODE_256)); + auto &n256 = Node::RefMutable(art, node, NType::NODE_256); n256.count = 0; for (idx_t i = 0; i < Node::NODE_256_CAPACITY; i++) { - n256.children[i].Reset(); + n256.children[i].Clear(); } return n256; @@ -22,10 +20,8 @@ Node256 &Node256::New(ART &art, Node &node) { void Node256::Free(ART &art, Node &node) { - D_ASSERT(node.IsSet()); - D_ASSERT(!node.IsSerialized()); - - auto &n256 = Node256::Get(art, node); + D_ASSERT(node.HasMetadata()); + auto &n256 = Node::RefMutable(art, node, NType::NODE_256); if (!n256.count) { return; @@ -33,7 +29,7 @@ void Node256::Free(ART &art, Node &node) { // free all children for (idx_t i = 0; i < Node::NODE_256_CAPACITY; i++) { - if (n256.children[i].IsSet()) { + if (n256.children[i].HasMetadata()) { Node::Free(art, n256.children[i]); } } @@ -41,15 +37,15 @@ void Node256::Free(ART &art, Node &node) { Node256 &Node256::GrowNode48(ART &art, Node &node256, Node &node48) { - auto &n48 = Node48::Get(art, node48); - auto &n256 = Node256::New(art, node256); + auto &n48 = Node::RefMutable(art, node48, NType::NODE_48); + auto &n256 = New(art, node256); n256.count = n48.count; for (idx_t i = 0; i < Node::NODE_256_CAPACITY; i++) { if (n48.child_index[i] != Node::EMPTY_MARKER) { n256.children[i] = n48.children[n48.child_index[i]]; } else { - n256.children[i].Reset(); + n256.children[i].Clear(); } } @@ -61,7 +57,7 @@ Node256 &Node256::GrowNode48(ART &art, Node &node256, Node &node48) { void Node256::InitializeMerge(ART &art, const ARTFlags &flags) { for (idx_t i = 0; i < Node::NODE_256_CAPACITY; i++) { - if (children[i].IsSet()) { + if (children[i].HasMetadata()) { children[i].InitializeMerge(art, flags); } } @@ -69,12 +65,11 @@ void Node256::InitializeMerge(ART &art, const ARTFlags &flags) { void Node256::InsertChild(ART &art, Node &node, const uint8_t byte, const Node child) { - D_ASSERT(node.IsSet()); - D_ASSERT(!node.IsSerialized()); - auto &n256 = Node256::Get(art, node); + D_ASSERT(node.HasMetadata()); + auto &n256 = Node::RefMutable(art, node, NType::NODE_256); // ensure that there is no other child at the same byte - D_ASSERT(!n256.children[byte].IsSet()); + D_ASSERT(!n256.children[byte].HasMetadata()); n256.count++; D_ASSERT(n256.count <= Node::NODE_256_CAPACITY); @@ -83,9 +78,8 @@ void Node256::InsertChild(ART &art, Node &node, const uint8_t byte, const Node c void Node256::DeleteChild(ART &art, Node &node, const uint8_t byte) { - D_ASSERT(node.IsSet()); - D_ASSERT(!node.IsSerialized()); - auto &n256 = Node256::Get(art, node); + D_ASSERT(node.HasMetadata()); + auto &n256 = Node::RefMutable(art, node, NType::NODE_256); // free the child and decrease the count Node::Free(art, n256.children[byte]); @@ -98,53 +92,44 @@ void Node256::DeleteChild(ART &art, Node &node, const uint8_t byte) { } } -optional_ptr Node256::GetNextChild(uint8_t &byte) { - - for (idx_t i = byte; i < Node::NODE_256_CAPACITY; i++) { - if (children[i].IsSet()) { - byte = i; - return &children[i]; - } +optional_ptr Node256::GetChild(const uint8_t byte) const { + if (children[byte].HasMetadata()) { + return &children[byte]; } return nullptr; } -BlockPointer Node256::Serialize(ART &art, MetadataWriter &writer) { - - // recurse into children and retrieve child block pointers - vector child_block_pointers; - for (idx_t i = 0; i < Node::NODE_256_CAPACITY; i++) { - child_block_pointers.push_back(children[i].Serialize(art, writer)); +optional_ptr Node256::GetChildMutable(const uint8_t byte) { + if (children[byte].HasMetadata()) { + return &children[byte]; } + return nullptr; +} - // get pointer and write fields - auto block_pointer = writer.GetBlockPointer(); - writer.Write(NType::NODE_256); - writer.Write(count); - - // write child block pointers - for (auto &child_block_pointer : child_block_pointers) { - writer.Write(child_block_pointer.block_id); - writer.Write(child_block_pointer.offset); +optional_ptr Node256::GetNextChild(uint8_t &byte) const { + for (idx_t i = byte; i < Node::NODE_256_CAPACITY; i++) { + if (children[i].HasMetadata()) { + byte = i; + return &children[i]; + } } - - return block_pointer; + return nullptr; } -void Node256::Deserialize(MetadataReader &reader) { - - count = reader.Read(); - - // read child block pointers - for (idx_t i = 0; i < Node::NODE_256_CAPACITY; i++) { - children[i] = Node(reader); +optional_ptr Node256::GetNextChildMutable(uint8_t &byte) { + for (idx_t i = byte; i < Node::NODE_256_CAPACITY; i++) { + if (children[i].HasMetadata()) { + byte = i; + return &children[i]; + } } + return nullptr; } void Node256::Vacuum(ART &art, const ARTFlags &flags) { for (idx_t i = 0; i < Node::NODE_256_CAPACITY; i++) { - if (children[i].IsSet()) { + if (children[i].HasMetadata()) { children[i].Vacuum(art, flags); } } diff --git a/src/duckdb/src/execution/index/art/node4.cpp b/src/duckdb/src/execution/index/art/node4.cpp index 4c2b771cb..6438b7448 100644 --- a/src/duckdb/src/execution/index/art/node4.cpp +++ b/src/duckdb/src/execution/index/art/node4.cpp @@ -2,16 +2,14 @@ #include "duckdb/execution/index/art/prefix.hpp" #include "duckdb/execution/index/art/node16.hpp" -#include "duckdb/storage/metadata/metadata_reader.hpp" -#include "duckdb/storage/metadata/metadata_writer.hpp" namespace duckdb { Node4 &Node4::New(ART &art, Node &node) { node = Node::GetAllocator(art, NType::NODE_4).New(); - node.SetType((uint8_t)NType::NODE_4); - auto &n4 = Node4::Get(art, node); + node.SetMetadata(static_cast(NType::NODE_4)); + auto &n4 = Node::RefMutable(art, node, NType::NODE_4); n4.count = 0; return n4; @@ -19,10 +17,8 @@ Node4 &Node4::New(ART &art, Node &node) { void Node4::Free(ART &art, Node &node) { - D_ASSERT(node.IsSet()); - D_ASSERT(!node.IsSerialized()); - - auto &n4 = Node4::Get(art, node); + D_ASSERT(node.HasMetadata()); + auto &n4 = Node::RefMutable(art, node, NType::NODE_4); // free all children for (idx_t i = 0; i < n4.count; i++) { @@ -32,8 +28,8 @@ void Node4::Free(ART &art, Node &node) { Node4 &Node4::ShrinkNode16(ART &art, Node &node4, Node &node16) { - auto &n4 = Node4::New(art, node4); - auto &n16 = Node16::Get(art, node16); + auto &n4 = New(art, node4); + auto &n16 = Node::RefMutable(art, node16, NType::NODE_16); D_ASSERT(n16.count <= Node::NODE_4_CAPACITY); n4.count = n16.count; @@ -56,9 +52,8 @@ void Node4::InitializeMerge(ART &art, const ARTFlags &flags) { void Node4::InsertChild(ART &art, Node &node, const uint8_t byte, const Node child) { - D_ASSERT(node.IsSet()); - D_ASSERT(!node.IsSerialized()); - auto &n4 = Node4::Get(art, node); + D_ASSERT(node.HasMetadata()); + auto &n4 = Node::RefMutable(art, node, NType::NODE_4); // ensure that there is no other child at the same byte for (idx_t i = 0; i < n4.count; i++) { @@ -92,9 +87,8 @@ void Node4::InsertChild(ART &art, Node &node, const uint8_t byte, const Node chi void Node4::DeleteChild(ART &art, Node &node, Node &prefix, const uint8_t byte) { - D_ASSERT(node.IsSet()); - D_ASSERT(!node.IsSerialized()); - auto &n4 = Node4::Get(art, node); + D_ASSERT(node.HasMetadata()); + auto &n4 = Node::RefMutable(art, node, NType::NODE_4); idx_t child_pos = 0; for (; child_pos < n4.count; child_pos++) { @@ -126,7 +120,7 @@ void Node4::DeleteChild(ART &art, Node &node, Node &prefix, const uint8_t byte) auto old_n4_node = node; // get only child and concatenate prefixes - auto child = *n4.GetChild(n4.key[0]); + auto child = *n4.GetChildMutable(n4.key[0]); Prefix::Concatenate(art, prefix, n4.key[0], child); n4.count--; @@ -143,71 +137,46 @@ void Node4::ReplaceChild(const uint8_t byte, const Node child) { } } -optional_ptr Node4::GetChild(const uint8_t byte) { - +optional_ptr Node4::GetChild(const uint8_t byte) const { for (idx_t i = 0; i < count; i++) { if (key[i] == byte) { - D_ASSERT(children[i].IsSet()); + D_ASSERT(children[i].HasMetadata()); return &children[i]; } } return nullptr; } -optional_ptr Node4::GetNextChild(uint8_t &byte) { - +optional_ptr Node4::GetChildMutable(const uint8_t byte) { for (idx_t i = 0; i < count; i++) { - if (key[i] >= byte) { - byte = key[i]; - D_ASSERT(children[i].IsSet()); + if (key[i] == byte) { + D_ASSERT(children[i].HasMetadata()); return &children[i]; } } return nullptr; } -BlockPointer Node4::Serialize(ART &art, MetadataWriter &writer) { - - // recurse into children and retrieve child block pointers - vector child_block_pointers; +optional_ptr Node4::GetNextChild(uint8_t &byte) const { for (idx_t i = 0; i < count; i++) { - child_block_pointers.push_back(children[i].Serialize(art, writer)); - } - for (idx_t i = count; i < Node::NODE_4_CAPACITY; i++) { - child_block_pointers.emplace_back((block_id_t)DConstants::INVALID_INDEX, 0); - } - - // get pointer and write fields - auto block_pointer = writer.GetBlockPointer(); - writer.Write(NType::NODE_4); - writer.Write(count); - - // write key values - for (idx_t i = 0; i < Node::NODE_4_CAPACITY; i++) { - writer.Write(key[i]); - } - - // write child block pointers - for (auto &child_block_pointer : child_block_pointers) { - writer.Write(child_block_pointer.block_id); - writer.Write(child_block_pointer.offset); + if (key[i] >= byte) { + byte = key[i]; + D_ASSERT(children[i].HasMetadata()); + return &children[i]; + } } - - return block_pointer; + return nullptr; } -void Node4::Deserialize(MetadataReader &reader) { - count = reader.Read(); - - // read key values - for (idx_t i = 0; i < Node::NODE_4_CAPACITY; i++) { - key[i] = reader.Read(); - } - - // read child block pointers - for (idx_t i = 0; i < Node::NODE_4_CAPACITY; i++) { - children[i] = Node(reader); +optional_ptr Node4::GetNextChildMutable(uint8_t &byte) { + for (idx_t i = 0; i < count; i++) { + if (key[i] >= byte) { + byte = key[i]; + D_ASSERT(children[i].HasMetadata()); + return &children[i]; + } } + return nullptr; } void Node4::Vacuum(ART &art, const ARTFlags &flags) { diff --git a/src/duckdb/src/execution/index/art/node48.cpp b/src/duckdb/src/execution/index/art/node48.cpp index 761554d6f..6c3d9816f 100644 --- a/src/duckdb/src/execution/index/art/node48.cpp +++ b/src/duckdb/src/execution/index/art/node48.cpp @@ -2,16 +2,14 @@ #include "duckdb/execution/index/art/node16.hpp" #include "duckdb/execution/index/art/node256.hpp" -#include "duckdb/storage/metadata/metadata_reader.hpp" -#include "duckdb/storage/metadata/metadata_writer.hpp" namespace duckdb { Node48 &Node48::New(ART &art, Node &node) { node = Node::GetAllocator(art, NType::NODE_48).New(); - node.SetType((uint8_t)NType::NODE_48); - auto &n48 = Node48::Get(art, node); + node.SetMetadata(static_cast(NType::NODE_48)); + auto &n48 = Node::RefMutable(art, node, NType::NODE_48); n48.count = 0; for (idx_t i = 0; i < Node::NODE_256_CAPACITY; i++) { @@ -20,7 +18,7 @@ Node48 &Node48::New(ART &art, Node &node) { // necessary for faster child insertion/deletion for (idx_t i = 0; i < Node::NODE_48_CAPACITY; i++) { - n48.children[i].Reset(); + n48.children[i].Clear(); } return n48; @@ -28,10 +26,8 @@ Node48 &Node48::New(ART &art, Node &node) { void Node48::Free(ART &art, Node &node) { - D_ASSERT(node.IsSet()); - D_ASSERT(!node.IsSerialized()); - - auto &n48 = Node48::Get(art, node); + D_ASSERT(node.HasMetadata()); + auto &n48 = Node::RefMutable(art, node, NType::NODE_48); if (!n48.count) { return; @@ -47,8 +43,8 @@ void Node48::Free(ART &art, Node &node) { Node48 &Node48::GrowNode16(ART &art, Node &node48, Node &node16) { - auto &n16 = Node16::Get(art, node16); - auto &n48 = Node48::New(art, node48); + auto &n16 = Node::RefMutable(art, node16, NType::NODE_16); + auto &n48 = New(art, node48); n48.count = n16.count; for (idx_t i = 0; i < Node::NODE_256_CAPACITY; i++) { @@ -62,7 +58,7 @@ Node48 &Node48::GrowNode16(ART &art, Node &node48, Node &node16) { // necessary for faster child insertion/deletion for (idx_t i = n16.count; i < Node::NODE_48_CAPACITY; i++) { - n48.children[i].Reset(); + n48.children[i].Clear(); } n16.count = 0; @@ -72,13 +68,13 @@ Node48 &Node48::GrowNode16(ART &art, Node &node48, Node &node16) { Node48 &Node48::ShrinkNode256(ART &art, Node &node48, Node &node256) { - auto &n48 = Node48::New(art, node48); - auto &n256 = Node256::Get(art, node256); + auto &n48 = New(art, node48); + auto &n256 = Node::RefMutable(art, node256, NType::NODE_256); n48.count = 0; for (idx_t i = 0; i < Node::NODE_256_CAPACITY; i++) { D_ASSERT(n48.count <= Node::NODE_48_CAPACITY); - if (n256.children[i].IsSet()) { + if (n256.children[i].HasMetadata()) { n48.child_index[i] = n48.count; n48.children[n48.count] = n256.children[i]; n48.count++; @@ -89,7 +85,7 @@ Node48 &Node48::ShrinkNode256(ART &art, Node &node48, Node &node256) { // necessary for faster child insertion/deletion for (idx_t i = n48.count; i < Node::NODE_48_CAPACITY; i++) { - n48.children[i].Reset(); + n48.children[i].Clear(); } n256.count = 0; @@ -108,9 +104,8 @@ void Node48::InitializeMerge(ART &art, const ARTFlags &flags) { void Node48::InsertChild(ART &art, Node &node, const uint8_t byte, const Node child) { - D_ASSERT(node.IsSet()); - D_ASSERT(!node.IsSerialized()); - auto &n48 = Node48::Get(art, node); + D_ASSERT(node.HasMetadata()); + auto &n48 = Node::RefMutable(art, node, NType::NODE_48); // ensure that there is no other child at the same byte D_ASSERT(n48.child_index[byte] == Node::EMPTY_MARKER); @@ -119,10 +114,10 @@ void Node48::InsertChild(ART &art, Node &node, const uint8_t byte, const Node ch if (n48.count < Node::NODE_48_CAPACITY) { // still space, just insert the child idx_t child_pos = n48.count; - if (n48.children[child_pos].IsSet()) { + if (n48.children[child_pos].HasMetadata()) { // find an empty position in the node list if the current position is occupied child_pos = 0; - while (n48.children[child_pos].IsSet()) { + while (n48.children[child_pos].HasMetadata()) { child_pos++; } } @@ -140,9 +135,8 @@ void Node48::InsertChild(ART &art, Node &node, const uint8_t byte, const Node ch void Node48::DeleteChild(ART &art, Node &node, const uint8_t byte) { - D_ASSERT(node.IsSet()); - D_ASSERT(!node.IsSerialized()); - auto &n48 = Node48::Get(art, node); + D_ASSERT(node.HasMetadata()); + auto &n48 = Node::RefMutable(art, node, NType::NODE_48); // free the child and decrease the count Node::Free(art, n48.children[n48.child_index[byte]]); @@ -156,58 +150,42 @@ void Node48::DeleteChild(ART &art, Node &node, const uint8_t byte) { } } -optional_ptr Node48::GetNextChild(uint8_t &byte) { +optional_ptr Node48::GetChild(const uint8_t byte) const { + if (child_index[byte] != Node::EMPTY_MARKER) { + D_ASSERT(children[child_index[byte]].HasMetadata()); + return &children[child_index[byte]]; + } + return nullptr; +} + +optional_ptr Node48::GetChildMutable(const uint8_t byte) { + if (child_index[byte] != Node::EMPTY_MARKER) { + D_ASSERT(children[child_index[byte]].HasMetadata()); + return &children[child_index[byte]]; + } + return nullptr; +} +optional_ptr Node48::GetNextChild(uint8_t &byte) const { for (idx_t i = byte; i < Node::NODE_256_CAPACITY; i++) { if (child_index[i] != Node::EMPTY_MARKER) { byte = i; - D_ASSERT(children[child_index[i]].IsSet()); + D_ASSERT(children[child_index[i]].HasMetadata()); return &children[child_index[i]]; } } return nullptr; } -BlockPointer Node48::Serialize(ART &art, MetadataWriter &writer) { - - // recurse into children and retrieve child block pointers - vector child_block_pointers; - for (idx_t i = 0; i < Node::NODE_48_CAPACITY; i++) { - child_block_pointers.push_back(children[i].Serialize(art, writer)); - } - - // get pointer and write fields - auto block_pointer = writer.GetBlockPointer(); - writer.Write(NType::NODE_48); - writer.Write(count); - - // write key values - for (idx_t i = 0; i < Node::NODE_256_CAPACITY; i++) { - writer.Write(child_index[i]); - } - - // write child block pointers - for (auto &child_block_pointer : child_block_pointers) { - writer.Write(child_block_pointer.block_id); - writer.Write(child_block_pointer.offset); - } - - return block_pointer; -} - -void Node48::Deserialize(MetadataReader &reader) { - - count = reader.Read(); - - // read key values - for (idx_t i = 0; i < Node::NODE_256_CAPACITY; i++) { - child_index[i] = reader.Read(); - } - - // read child block pointers - for (idx_t i = 0; i < Node::NODE_48_CAPACITY; i++) { - children[i] = Node(reader); +optional_ptr Node48::GetNextChildMutable(uint8_t &byte) { + for (idx_t i = byte; i < Node::NODE_256_CAPACITY; i++) { + if (child_index[i] != Node::EMPTY_MARKER) { + byte = i; + D_ASSERT(children[child_index[i]].HasMetadata()); + return &children[child_index[i]]; + } } + return nullptr; } void Node48::Vacuum(ART &art, const ARTFlags &flags) { diff --git a/src/duckdb/src/execution/index/art/prefix.cpp b/src/duckdb/src/execution/index/art/prefix.cpp index b7ae67149..a26ec85f6 100644 --- a/src/duckdb/src/execution/index/art/prefix.cpp +++ b/src/duckdb/src/execution/index/art/prefix.cpp @@ -3,8 +3,6 @@ #include "duckdb/execution/index/art/art.hpp" #include "duckdb/execution/index/art/art_key.hpp" #include "duckdb/execution/index/art/node.hpp" -#include "duckdb/storage/metadata/metadata_reader.hpp" -#include "duckdb/storage/metadata/metadata_writer.hpp" #include "duckdb/common/swap.hpp" namespace duckdb { @@ -12,19 +10,19 @@ namespace duckdb { Prefix &Prefix::New(ART &art, Node &node) { node = Node::GetAllocator(art, NType::PREFIX).New(); - node.SetType((uint8_t)NType::PREFIX); + node.SetMetadata(static_cast(NType::PREFIX)); - auto &prefix = Prefix::Get(art, node); + auto &prefix = Node::RefMutable(art, node, NType::PREFIX); prefix.data[Node::PREFIX_SIZE] = 0; return prefix; } -Prefix &Prefix::New(ART &art, Node &node, uint8_t byte, Node next) { +Prefix &Prefix::New(ART &art, Node &node, uint8_t byte, const Node &next) { node = Node::GetAllocator(art, NType::PREFIX).New(); - node.SetType((uint8_t)NType::PREFIX); + node.SetMetadata(static_cast(NType::PREFIX)); - auto &prefix = Prefix::Get(art, node); + auto &prefix = Node::RefMutable(art, node, NType::PREFIX); prefix.data[Node::PREFIX_SIZE] = 1; prefix.data[0] = byte; prefix.ptr = next; @@ -40,8 +38,8 @@ void Prefix::New(ART &art, reference &node, const ARTKey &key, const uint3 while (count) { node.get() = Node::GetAllocator(art, NType::PREFIX).New(); - node.get().SetType((uint8_t)NType::PREFIX); - auto &prefix = Prefix::Get(art, node); + node.get().SetMetadata(static_cast(NType::PREFIX)); + auto &prefix = Node::RefMutable(art, node, NType::PREFIX); auto this_count = MinValue((uint32_t)Node::PREFIX_SIZE, count); prefix.data[Node::PREFIX_SIZE] = (uint8_t)this_count; @@ -57,54 +55,49 @@ void Prefix::Free(ART &art, Node &node) { Node current_node = node; Node next_node; - while (current_node.IsSet() && !current_node.IsSerialized() && current_node.GetType() == NType::PREFIX) { - next_node = Prefix::Get(art, current_node).ptr; + while (current_node.HasMetadata() && current_node.GetType() == NType::PREFIX) { + next_node = Node::RefMutable(art, current_node, NType::PREFIX).ptr; Node::GetAllocator(art, NType::PREFIX).Free(current_node); current_node = next_node; } Node::Free(art, current_node); - node.Reset(); + node.Clear(); } void Prefix::InitializeMerge(ART &art, Node &node, const ARTFlags &flags) { - auto merge_buffer_count = flags.merge_buffer_counts[(uint8_t)NType::PREFIX - 1]; + auto merge_buffer_count = flags.merge_buffer_counts[static_cast(NType::PREFIX) - 1]; Node next_node = node; - reference prefix = Prefix::Get(art, next_node); + reference prefix = Node::RefMutable(art, next_node, NType::PREFIX); while (next_node.GetType() == NType::PREFIX) { next_node = prefix.get().ptr; if (prefix.get().ptr.GetType() == NType::PREFIX) { - prefix.get().ptr.AddToBufferID(merge_buffer_count); - prefix = Prefix::Get(art, next_node); + prefix.get().ptr.IncreaseBufferId(merge_buffer_count); + prefix = Node::RefMutable(art, next_node, NType::PREFIX); } } - node.AddToBufferID(merge_buffer_count); + node.IncreaseBufferId(merge_buffer_count); prefix.get().ptr.InitializeMerge(art, flags); } void Prefix::Concatenate(ART &art, Node &prefix_node, const uint8_t byte, Node &child_prefix_node) { - D_ASSERT(prefix_node.IsSet() && !prefix_node.IsSerialized()); - D_ASSERT(child_prefix_node.IsSet()); - - if (child_prefix_node.IsSerialized()) { - child_prefix_node.Deserialize(art); - } + D_ASSERT(prefix_node.HasMetadata() && child_prefix_node.HasMetadata()); // append a byte and a child_prefix to prefix if (prefix_node.GetType() == NType::PREFIX) { // get the tail - reference prefix = Prefix::Get(art, prefix_node); - D_ASSERT(prefix.get().ptr.IsSet() && !prefix.get().ptr.IsSerialized()); + reference prefix = Node::RefMutable(art, prefix_node, NType::PREFIX); + D_ASSERT(prefix.get().ptr.HasMetadata()); while (prefix.get().ptr.GetType() == NType::PREFIX) { - prefix = Prefix::Get(art, prefix.get().ptr); - D_ASSERT(prefix.get().ptr.IsSet() && !prefix.get().ptr.IsSerialized()); + prefix = Node::RefMutable(art, prefix.get().ptr, NType::PREFIX); + D_ASSERT(prefix.get().ptr.HasMetadata()); } // append the byte @@ -124,24 +117,24 @@ void Prefix::Concatenate(ART &art, Node &prefix_node, const uint8_t byte, Node & if (prefix_node.GetType() != NType::PREFIX && child_prefix_node.GetType() == NType::PREFIX) { auto child_prefix = child_prefix_node; - auto &prefix = Prefix::New(art, prefix_node, byte, Node()); + auto &prefix = New(art, prefix_node, byte); prefix.Append(art, child_prefix); return; } // neither prefix nor child_prefix are prefix nodes // create a new prefix containing the byte - Prefix::New(art, prefix_node, byte, child_prefix_node); + New(art, prefix_node, byte, child_prefix_node); } -idx_t Prefix::Traverse(ART &art, reference &prefix_node, const ARTKey &key, idx_t &depth) { +idx_t Prefix::Traverse(ART &art, reference &prefix_node, const ARTKey &key, idx_t &depth) { - D_ASSERT(prefix_node.get().IsSet() && !prefix_node.get().IsSerialized()); + D_ASSERT(prefix_node.get().HasMetadata()); D_ASSERT(prefix_node.get().GetType() == NType::PREFIX); // compare prefix nodes to key bytes while (prefix_node.get().GetType() == NType::PREFIX) { - auto &prefix = Prefix::Get(art, prefix_node); + auto &prefix = Node::Ref(art, prefix_node, NType::PREFIX); for (idx_t i = 0; i < prefix.data[Node::PREFIX_SIZE]; i++) { if (prefix.data[i] != key[depth]) { return i; @@ -149,10 +142,28 @@ idx_t Prefix::Traverse(ART &art, reference &prefix_node, const ARTKey &key depth++; } prefix_node = prefix.ptr; - D_ASSERT(prefix_node.get().IsSet()); - if (prefix_node.get().IsSerialized()) { - prefix_node.get().Deserialize(art); + D_ASSERT(prefix_node.get().HasMetadata()); + } + + return DConstants::INVALID_INDEX; +} + +idx_t Prefix::TraverseMutable(ART &art, reference &prefix_node, const ARTKey &key, idx_t &depth) { + + D_ASSERT(prefix_node.get().HasMetadata()); + D_ASSERT(prefix_node.get().GetType() == NType::PREFIX); + + // compare prefix nodes to key bytes + while (prefix_node.get().GetType() == NType::PREFIX) { + auto &prefix = Node::RefMutable(art, prefix_node, NType::PREFIX); + for (idx_t i = 0; i < prefix.data[Node::PREFIX_SIZE]; i++) { + if (prefix.data[i] != key[depth]) { + return i; + } + depth++; } + prefix_node = prefix.ptr; + D_ASSERT(prefix_node.get().HasMetadata()); } return DConstants::INVALID_INDEX; @@ -160,8 +171,8 @@ idx_t Prefix::Traverse(ART &art, reference &prefix_node, const ARTKey &key bool Prefix::Traverse(ART &art, reference &l_node, reference &r_node, idx_t &mismatch_position) { - auto &l_prefix = Prefix::Get(art, l_node.get()); - auto &r_prefix = Prefix::Get(art, r_node.get()); + auto &l_prefix = Node::RefMutable(art, l_node.get(), NType::PREFIX); + auto &r_prefix = Node::RefMutable(art, r_node.get(), NType::PREFIX); // compare prefix bytes idx_t max_count = MinValue(l_prefix.data[Node::PREFIX_SIZE], r_prefix.data[Node::PREFIX_SIZE]); @@ -197,16 +208,16 @@ bool Prefix::Traverse(ART &art, reference &l_node, reference &r_node void Prefix::Reduce(ART &art, Node &prefix_node, const idx_t n) { - D_ASSERT(prefix_node.IsSet() && !prefix_node.IsSerialized()); + D_ASSERT(prefix_node.HasMetadata()); D_ASSERT(n < Node::PREFIX_SIZE); - reference prefix = Prefix::Get(art, prefix_node); + reference prefix = Node::RefMutable(art, prefix_node, NType::PREFIX); // free this prefix node if (n == (idx_t)(prefix.get().data[Node::PREFIX_SIZE] - 1)) { auto next_ptr = prefix.get().ptr; - D_ASSERT(next_ptr.IsSet()); - prefix.get().ptr.Reset(); + D_ASSERT(next_ptr.HasMetadata()); + prefix.get().ptr.Clear(); Node::Free(art, prefix_node); prefix_node = next_ptr; return; @@ -225,9 +236,9 @@ void Prefix::Reduce(ART &art, Node &prefix_node, const idx_t n) { void Prefix::Split(ART &art, reference &prefix_node, Node &child_node, idx_t position) { - D_ASSERT(prefix_node.get().IsSet() && !prefix_node.get().IsSerialized()); + D_ASSERT(prefix_node.get().HasMetadata()); - auto &prefix = Prefix::Get(art, prefix_node); + auto &prefix = Node::RefMutable(art, prefix_node, NType::PREFIX); // the split is at the last byte of this prefix, so the child_node contains all subsequent // prefix nodes (prefix.ptr) (if any), and the count of this prefix decreases by one, @@ -241,15 +252,12 @@ void Prefix::Split(ART &art, reference &prefix_node, Node &child_node, idx // append the remaining bytes after the split if (position + 1 < prefix.data[Node::PREFIX_SIZE]) { - reference child_prefix = Prefix::New(art, child_node); + reference child_prefix = New(art, child_node); for (idx_t i = position + 1; i < prefix.data[Node::PREFIX_SIZE]; i++) { child_prefix = child_prefix.get().Append(art, prefix.data[i]); } - D_ASSERT(prefix.ptr.IsSet()); - if (prefix.ptr.IsSerialized()) { - prefix.ptr.Deserialize(art); - } + D_ASSERT(prefix.ptr.HasMetadata()); if (prefix.ptr.GetType() == NType::PREFIX) { child_prefix.get().Append(art, prefix.ptr); @@ -269,7 +277,7 @@ void Prefix::Split(ART &art, reference &prefix_node, Node &child_node, idx // no bytes left before the split, free this node if (position == 0) { - prefix.ptr.Reset(); + prefix.ptr.Clear(); Node::Free(art, prefix_node.get()); return; } @@ -279,15 +287,15 @@ void Prefix::Split(ART &art, reference &prefix_node, Node &child_node, idx return; } -string Prefix::VerifyAndToString(ART &art, Node &node, const bool only_verify) { +string Prefix::VerifyAndToString(ART &art, const Node &node, const bool only_verify) { // NOTE: we could do this recursively, but the function-call overhead can become kinda crazy string str = ""; - reference node_ref(node); + reference node_ref(node); while (node_ref.get().GetType() == NType::PREFIX) { - auto &prefix = Prefix::Get(art, node_ref); + auto &prefix = Node::Ref(art, node_ref, NType::PREFIX); D_ASSERT(prefix.data[Node::PREFIX_SIZE] != 0); D_ASSERT(prefix.data[Node::PREFIX_SIZE] <= Node::PREFIX_SIZE); @@ -297,82 +305,25 @@ string Prefix::VerifyAndToString(ART &art, Node &node, const bool only_verify) { } str += "] "; - if (prefix.ptr.IsSerialized()) { - return str + " serialized"; - } node_ref = prefix.ptr; } - return str + node_ref.get().VerifyAndToString(art, only_verify); -} - -BlockPointer Prefix::Serialize(ART &art, Node &node, MetadataWriter &writer) { - reference first_non_prefix(node); - idx_t total_count = Prefix::TotalCount(art, first_non_prefix); - auto child_block_pointer = first_non_prefix.get().Serialize(art, writer); - - // get pointer and write fields - auto block_pointer = writer.GetBlockPointer(); - writer.Write(NType::PREFIX); - writer.Write(total_count); - - reference current_node(node); - while (current_node.get().GetType() == NType::PREFIX) { - - // write prefix bytes - D_ASSERT(!current_node.get().IsSerialized()); - auto &prefix = Prefix::Get(art, current_node); - for (idx_t i = 0; i < prefix.data[Node::PREFIX_SIZE]; i++) { - writer.Write(prefix.data[i]); - } - - current_node = prefix.ptr; - } - - // write child block pointer - writer.Write(child_block_pointer.block_id); - writer.Write(child_block_pointer.offset); - - return block_pointer; -} - -void Prefix::Deserialize(ART &art, Node &node, MetadataReader &reader) { - auto total_count = reader.Read(); - reference current_node(node); - - while (total_count) { - current_node.get() = Node::GetAllocator(art, NType::PREFIX).New(); - current_node.get().SetType((uint8_t)NType::PREFIX); - - auto &prefix = Prefix::Get(art, current_node); - prefix.data[Node::PREFIX_SIZE] = MinValue((idx_t)Node::PREFIX_SIZE, total_count); - - // read bytes - for (idx_t i = 0; i < prefix.data[Node::PREFIX_SIZE]; i++) { - prefix.data[i] = reader.Read(); - } - - total_count -= prefix.data[Node::PREFIX_SIZE]; - current_node = prefix.ptr; - prefix.ptr.Reset(); - } - - // read child block pointer - current_node.get() = Node(reader); + auto subtree = node_ref.get().VerifyAndToString(art, only_verify); + return only_verify ? "" : str + subtree; } void Prefix::Vacuum(ART &art, Node &node, const ARTFlags &flags) { - bool flag_set = flags.vacuum_flags[(uint8_t)NType::PREFIX - 1]; + bool flag_set = flags.vacuum_flags[static_cast(NType::PREFIX) - 1]; auto &allocator = Node::GetAllocator(art, NType::PREFIX); reference node_ref(node); - while (!node_ref.get().IsSerialized() && node_ref.get().GetType() == NType::PREFIX) { + while (node_ref.get().GetType() == NType::PREFIX) { if (flag_set && allocator.NeedsVacuum(node_ref)) { node_ref.get() = allocator.VacuumPointer(node_ref); - node_ref.get().SetType((uint8_t)NType::PREFIX); + node_ref.get().SetMetadata(static_cast(NType::PREFIX)); } - auto &prefix = Prefix::Get(art, node_ref); + auto &prefix = Node::RefMutable(art, node_ref, NType::PREFIX); node_ref = prefix.ptr; } @@ -385,7 +336,7 @@ Prefix &Prefix::Append(ART &art, const uint8_t byte) { // we need a new prefix node if (prefix.get().data[Node::PREFIX_SIZE] == Node::PREFIX_SIZE) { - prefix = Prefix::New(art, prefix.get().ptr); + prefix = New(art, prefix.get().ptr); } prefix.get().data[prefix.get().data[Node::PREFIX_SIZE]] = byte; @@ -395,22 +346,18 @@ Prefix &Prefix::Append(ART &art, const uint8_t byte) { void Prefix::Append(ART &art, Node other_prefix) { - // NOTE: all usages of this function already deserialize the other prefix - D_ASSERT(other_prefix.IsSet() && !other_prefix.IsSerialized()); + D_ASSERT(other_prefix.HasMetadata()); reference prefix(*this); while (other_prefix.GetType() == NType::PREFIX) { // copy prefix bytes - auto &other = Prefix::Get(art, other_prefix); + auto &other = Node::RefMutable(art, other_prefix, NType::PREFIX); for (idx_t i = 0; i < other.data[Node::PREFIX_SIZE]; i++) { prefix = prefix.get().Append(art, other.data[i]); } - D_ASSERT(other.ptr.IsSet()); - if (other.ptr.IsSerialized()) { - other.ptr.Deserialize(art); - } + D_ASSERT(other.ptr.HasMetadata()); prefix.get().ptr = other.ptr; Node::GetAllocator(art, NType::PREFIX).Free(other_prefix); @@ -420,22 +367,4 @@ void Prefix::Append(ART &art, Node other_prefix) { D_ASSERT(prefix.get().ptr.GetType() != NType::PREFIX); } -idx_t Prefix::TotalCount(ART &art, reference &node) { - - // NOTE: first prefix in the prefix chain is already deserialized - D_ASSERT(node.get().IsSet() && !node.get().IsSerialized()); - - idx_t count = 0; - while (node.get().GetType() == NType::PREFIX) { - auto &prefix = Prefix::Get(art, node); - count += prefix.data[Node::PREFIX_SIZE]; - - if (prefix.ptr.IsSerialized()) { - prefix.ptr.Deserialize(art); - } - node = prefix.ptr; - } - return count; -} - } // namespace duckdb diff --git a/src/duckdb/src/execution/index/fixed_size_allocator.cpp b/src/duckdb/src/execution/index/fixed_size_allocator.cpp new file mode 100644 index 000000000..b199e429f --- /dev/null +++ b/src/duckdb/src/execution/index/fixed_size_allocator.cpp @@ -0,0 +1,345 @@ +#include "duckdb/execution/index/fixed_size_allocator.hpp" + +#include "duckdb/storage/metadata/metadata_reader.hpp" + +namespace duckdb { + +constexpr idx_t FixedSizeAllocator::BASE[]; +constexpr uint8_t FixedSizeAllocator::SHIFT[]; + +FixedSizeAllocator::FixedSizeAllocator(const idx_t segment_size, BlockManager &block_manager) + : block_manager(block_manager), buffer_manager(block_manager.buffer_manager), + metadata_manager(block_manager.GetMetadataManager()), segment_size(segment_size), total_segment_count(0) { + + if (segment_size > Storage::BLOCK_SIZE - sizeof(validity_t)) { + throw InternalException("The maximum segment size of fixed-size allocators is " + + to_string(Storage::BLOCK_SIZE - sizeof(validity_t))); + } + + // calculate how many segments fit into one buffer (available_segments_per_buffer) + + idx_t bits_per_value = sizeof(validity_t) * 8; + idx_t byte_count = 0; + + bitmask_count = 0; + available_segments_per_buffer = 0; + + while (byte_count < Storage::BLOCK_SIZE) { + if (!bitmask_count || (bitmask_count * bits_per_value) % available_segments_per_buffer == 0) { + // we need to add another validity_t value to the bitmask, to allow storing another + // bits_per_value segments on a buffer + bitmask_count++; + byte_count += sizeof(validity_t); + } + + auto remaining_bytes = Storage::BLOCK_SIZE - byte_count; + auto remaining_segments = MinValue(remaining_bytes / segment_size, bits_per_value); + + if (remaining_segments == 0) { + break; + } + + available_segments_per_buffer += remaining_segments; + byte_count += remaining_segments * segment_size; + } + + bitmask_offset = bitmask_count * sizeof(validity_t); +} + +IndexPointer FixedSizeAllocator::New() { + + // no more segments available + if (buffers_with_free_space.empty()) { + + // add a new buffer + auto buffer_id = GetAvailableBufferId(); + FixedSizeBuffer new_buffer(block_manager); + buffers.insert(make_pair(buffer_id, std::move(new_buffer))); + buffers_with_free_space.insert(buffer_id); + + // set the bitmask + D_ASSERT(buffers.find(buffer_id) != buffers.end()); + auto &buffer = buffers.find(buffer_id)->second; + ValidityMask mask(reinterpret_cast(buffer.Get())); + mask.SetAllValid(available_segments_per_buffer); + } + + // return a pointer + D_ASSERT(!buffers_with_free_space.empty()); + auto buffer_id = uint32_t(*buffers_with_free_space.begin()); + + D_ASSERT(buffers.find(buffer_id) != buffers.end()); + auto &buffer = buffers.find(buffer_id)->second; + auto bitmask_ptr = reinterpret_cast(buffer.Get()); + ValidityMask mask(bitmask_ptr); + auto offset = GetOffset(mask, buffer.segment_count); + + buffer.segment_count++; + total_segment_count++; + if (buffer.segment_count == available_segments_per_buffer) { + buffers_with_free_space.erase(buffer_id); + } + + return IndexPointer(buffer_id, offset); +} + +void FixedSizeAllocator::Free(const IndexPointer ptr) { + + auto buffer_id = ptr.GetBufferId(); + auto offset = ptr.GetOffset(); + + D_ASSERT(buffers.find(buffer_id) != buffers.end()); + auto &buffer = buffers.find(buffer_id)->second; + + auto bitmask_ptr = reinterpret_cast(buffer.Get()); + ValidityMask mask(bitmask_ptr); + D_ASSERT(!mask.RowIsValid(offset)); + mask.SetValid(offset); + buffers_with_free_space.insert(buffer_id); + + D_ASSERT(total_segment_count > 0); + D_ASSERT(buffer.segment_count > 0); + + buffer.segment_count--; + total_segment_count--; +} + +void FixedSizeAllocator::Reset() { + for (auto &buffer : buffers) { + buffer.second.Destroy(); + } + buffers.clear(); + buffers_with_free_space.clear(); + total_segment_count = 0; +} + +idx_t FixedSizeAllocator::GetMemoryUsage() const { + idx_t memory_usage = 0; + for (auto &buffer : buffers) { + if (buffer.second.InMemory()) { + memory_usage += Storage::BLOCK_SIZE; + } + } + return memory_usage; +} + +void FixedSizeAllocator::Merge(FixedSizeAllocator &other) { + + D_ASSERT(segment_size == other.segment_size); + + // remember the buffer count and merge the buffers + idx_t upper_bound_id = GetUpperBoundBufferId(); + for (auto &buffer : other.buffers) { + buffers.insert(make_pair(buffer.first + upper_bound_id, std::move(buffer.second))); + } + other.buffers.clear(); + + // merge the buffers with free spaces + for (auto &buffer_id : other.buffers_with_free_space) { + buffers_with_free_space.insert(buffer_id + upper_bound_id); + } + other.buffers_with_free_space.clear(); + + // add the total allocations + total_segment_count += other.total_segment_count; +} + +bool FixedSizeAllocator::InitializeVacuum() { + + // NOTE: we do not vacuum buffers that are not in memory. We might consider changing this + // in the future, although buffers on disk should almost never be eligible for a vacuum + + if (total_segment_count == 0) { + Reset(); + return false; + } + + multimap temporary_vacuum_buffers; + D_ASSERT(vacuum_buffers.empty()); + idx_t available_segments_in_memory = 0; + + for (auto &buffer : buffers) { + buffer.second.vacuum = false; + if (buffer.second.InMemory()) { + auto available_segments_in_buffer = available_segments_per_buffer - buffer.second.segment_count; + available_segments_in_memory += available_segments_in_buffer; + temporary_vacuum_buffers.emplace(available_segments_in_buffer, buffer.first); + } + } + + // no buffers in memory + if (temporary_vacuum_buffers.empty()) { + return false; + } + + auto excess_buffer_count = available_segments_in_memory / available_segments_per_buffer; + + // calculate the vacuum threshold adaptively + D_ASSERT(excess_buffer_count < temporary_vacuum_buffers.size()); + idx_t memory_usage = GetMemoryUsage(); + idx_t excess_memory_usage = excess_buffer_count * Storage::BLOCK_SIZE; + auto excess_percentage = double(excess_memory_usage) / double(memory_usage); + auto threshold = double(VACUUM_THRESHOLD) / 100.0; + if (excess_percentage < threshold) { + return false; + } + + D_ASSERT(excess_buffer_count <= temporary_vacuum_buffers.size()); + D_ASSERT(temporary_vacuum_buffers.size() <= buffers.size()); + + // erasing from a multimap, we vacuum the buffers with the most free spaces (least full) + while (temporary_vacuum_buffers.size() != excess_buffer_count) { + temporary_vacuum_buffers.erase(temporary_vacuum_buffers.begin()); + } + + // adjust the buffers, and erase all to-be-vacuumed buffers from the available buffer list + for (auto &vacuum_buffer : temporary_vacuum_buffers) { + auto buffer_id = vacuum_buffer.second; + D_ASSERT(buffers.find(buffer_id) != buffers.end()); + buffers.find(buffer_id)->second.vacuum = true; + buffers_with_free_space.erase(buffer_id); + } + + for (auto &vacuum_buffer : temporary_vacuum_buffers) { + vacuum_buffers.insert(vacuum_buffer.second); + } + + return true; +} + +void FixedSizeAllocator::FinalizeVacuum() { + + for (auto &buffer_id : vacuum_buffers) { + D_ASSERT(buffers.find(buffer_id) != buffers.end()); + auto &buffer = buffers.find(buffer_id)->second; + D_ASSERT(buffer.InMemory()); + buffer.Destroy(); + buffers.erase(buffer_id); + } + vacuum_buffers.clear(); +} + +IndexPointer FixedSizeAllocator::VacuumPointer(const IndexPointer ptr) { + + // we do not need to adjust the bitmask of the old buffer, because we will free the entire + // buffer after the vacuum operation + + auto new_ptr = New(); + // new increases the allocation count, we need to counter that here + total_segment_count--; + + memcpy(Get(new_ptr), Get(ptr), segment_size); + return new_ptr; +} + +BlockPointer FixedSizeAllocator::Serialize(MetadataWriter &writer) { + + for (auto &buffer : buffers) { + buffer.second.Serialize(); + } + + auto block_pointer = writer.GetBlockPointer(); + writer.Write(segment_size); + writer.Write(static_cast(buffers.size())); + writer.Write(static_cast(buffers_with_free_space.size())); + + for (auto &buffer : buffers) { + writer.Write(buffer.first); + writer.Write(buffer.second.BlockId()); + writer.Write(buffer.second.segment_count); + } + for (auto &buffer_id : buffers_with_free_space) { + writer.Write(buffer_id); + } + + return block_pointer; +} + +void FixedSizeAllocator::Deserialize(const BlockPointer &block_pointer) { + + MetadataReader reader(metadata_manager, block_pointer); + segment_size = reader.Read(); + auto buffer_count = reader.Read(); + auto buffers_with_free_space_count = reader.Read(); + + total_segment_count = 0; + + for (idx_t i = 0; i < buffer_count; i++) { + auto buffer_id = reader.Read(); + auto block_id = reader.Read(); + auto buffer_segment_count = reader.Read(); + FixedSizeBuffer new_buffer(block_manager, buffer_segment_count, block_id); + buffers.insert(make_pair(buffer_id, std::move(new_buffer))); + total_segment_count += buffer_segment_count; + } + for (idx_t i = 0; i < buffers_with_free_space_count; i++) { + buffers_with_free_space.insert(reader.Read()); + } +} + +uint32_t FixedSizeAllocator::GetOffset(ValidityMask &mask, const idx_t segment_count) { + + auto data = mask.GetData(); + + // fills up a buffer sequentially before searching for free bits + if (mask.RowIsValid(segment_count)) { + mask.SetInvalid(segment_count); + return segment_count; + } + + for (idx_t entry_idx = 0; entry_idx < bitmask_count; entry_idx++) { + // get an entry with free bits + if (data[entry_idx] == 0) { + continue; + } + + // find the position of the free bit + auto entry = data[entry_idx]; + idx_t first_valid_bit = 0; + + // this loop finds the position of the rightmost set bit in entry and stores it + // in first_valid_bit + for (idx_t i = 0; i < 6; i++) { + // set the left half of the bits of this level to zero and test if the entry is still not zero + if (entry & BASE[i]) { + // first valid bit is in the rightmost s[i] bits + // permanently set the left half of the bits to zero + entry &= BASE[i]; + } else { + // first valid bit is in the leftmost s[i] bits + // shift by s[i] for the next iteration and add s[i] to the position of the rightmost set bit + entry >>= SHIFT[i]; + first_valid_bit += SHIFT[i]; + } + } + D_ASSERT(entry); + + auto prev_bits = entry_idx * sizeof(validity_t) * 8; + D_ASSERT(mask.RowIsValid(prev_bits + first_valid_bit)); + mask.SetInvalid(prev_bits + first_valid_bit); + return (prev_bits + first_valid_bit); + } + + throw InternalException("Invalid bitmask for FixedSizeAllocator"); +} + +idx_t FixedSizeAllocator::GetAvailableBufferId() const { + idx_t buffer_id = buffers.size(); + while (buffers.find(buffer_id) != buffers.end()) { + D_ASSERT(buffer_id > 0); + buffer_id--; + } + return buffer_id; +} + +idx_t FixedSizeAllocator::GetUpperBoundBufferId() const { + idx_t upper_bound_id = 0; + for (auto &buffer : buffers) { + if (buffer.first >= upper_bound_id) { + upper_bound_id = buffer.first + 1; + } + } + return upper_bound_id; +} + +} // namespace duckdb diff --git a/src/duckdb/src/execution/index/fixed_size_buffer.cpp b/src/duckdb/src/execution/index/fixed_size_buffer.cpp new file mode 100644 index 000000000..d45364447 --- /dev/null +++ b/src/duckdb/src/execution/index/fixed_size_buffer.cpp @@ -0,0 +1,74 @@ +#include "duckdb/execution/index/fixed_size_buffer.hpp" + +#include "duckdb/storage/block_manager.hpp" +#include "duckdb/storage/buffer_manager.hpp" + +namespace duckdb { + +FixedSizeBuffer::FixedSizeBuffer(BlockManager &block_manager) + : block_manager(block_manager), segment_count(0), dirty(false), vacuum(false), block_handle(nullptr) { + + auto &buffer_manager = block_manager.buffer_manager; + buffer_handle = buffer_manager.Allocate(Storage::BLOCK_SIZE, false, &block_handle); +} + +FixedSizeBuffer::FixedSizeBuffer(BlockManager &block_manager, const idx_t segment_count, const block_id_t &block_id) + : block_manager(block_manager), segment_count(segment_count), dirty(false), vacuum(false) { + + D_ASSERT(block_id < MAXIMUM_BLOCK); + block_handle = block_manager.RegisterBlock(block_id); + D_ASSERT(BlockId() < MAXIMUM_BLOCK); +} + +void FixedSizeBuffer::Destroy() { + if (InMemory()) { + buffer_handle.Destroy(); + } + if (OnDisk()) { + block_manager.MarkBlockAsFree(BlockId()); + } +} + +void FixedSizeBuffer::Serialize() { + + if (!InMemory()) { + if (!OnDisk() || dirty) { + throw InternalException("invalid/missing buffer in FixedSizeAllocator"); + } + return; + } + if (!dirty && OnDisk()) { + return; + } + + // the buffer is in memory + D_ASSERT(InMemory()); + // the buffer never was on disk, or there were changes to it after loading it from disk + D_ASSERT(!OnDisk() || dirty); + + // we persist any changes, so the buffer is no longer dirty + dirty = false; + + if (!OnDisk()) { + // temporary block - convert to persistent + auto block_id = block_manager.GetFreeBlockId(); + D_ASSERT(block_id < MAXIMUM_BLOCK); + block_handle = block_manager.ConvertToPersistent(block_id, std::move(block_handle)); + buffer_handle.Destroy(); + + } else { + // already a persistent block - only need to write it + auto block_id = block_handle->BlockId(); + D_ASSERT(block_id < MAXIMUM_BLOCK); + block_manager.Write(buffer_handle.GetFileBuffer(), block_id); + } +} + +void FixedSizeBuffer::Pin() { + + auto &buffer_manager = block_manager.buffer_manager; + D_ASSERT(BlockId() < MAXIMUM_BLOCK); + buffer_handle = BufferHandle(buffer_manager.Pin(block_handle)); +} + +} // namespace duckdb diff --git a/src/duckdb/src/execution/operator/aggregate/physical_window.cpp b/src/duckdb/src/execution/operator/aggregate/physical_window.cpp index cba3e5dcc..18cef8316 100644 --- a/src/duckdb/src/execution/operator/aggregate/physical_window.cpp +++ b/src/duckdb/src/execution/operator/aggregate/physical_window.cpp @@ -564,7 +564,7 @@ WindowGlobalSourceState::Task WindowGlobalSourceState::NextTask(idx_t hash_bin) // If there is nothing to steal but there are unfinished partitions, // yield until any pending builds are done. - TaskScheduler::GetScheduler(context).YieldThread(); + TaskScheduler::YieldThread(); } return Task(); diff --git a/src/duckdb/src/execution/operator/schema/physical_create_art_index.cpp b/src/duckdb/src/execution/operator/schema/physical_create_art_index.cpp index 690ec9ba1..7c598dc83 100644 --- a/src/duckdb/src/execution/operator/schema/physical_create_art_index.cpp +++ b/src/duckdb/src/execution/operator/schema/physical_create_art_index.cpp @@ -89,7 +89,7 @@ SinkResultType PhysicalCreateARTIndex::SinkUnsorted(Vector &row_identifiers, Ope // insert the row IDs auto &art = l_state.local_index->Cast(); for (idx_t i = 0; i < count; i++) { - if (!art.Insert(*art.tree, l_state.keys[i], 0, row_ids[i])) { + if (!art.Insert(art.tree, l_state.keys[i], 0, row_ids[i])) { throw ConstraintException("Data contains duplicates on indexed column(s)"); } } diff --git a/src/duckdb/src/function/table/system/duckdb_columns.cpp b/src/duckdb/src/function/table/system/duckdb_columns.cpp index 72e1cf7c2..42d8203e7 100644 --- a/src/duckdb/src/function/table/system/duckdb_columns.cpp +++ b/src/duckdb/src/function/table/system/duckdb_columns.cpp @@ -131,7 +131,9 @@ class TableColumnHelper : public ColumnHelper { } const Value ColumnDefault(idx_t col) override { auto &column = entry.GetColumn(LogicalIndex(col)); - if (column.DefaultValue()) { + if (column.Generated()) { + return Value(column.GeneratedExpression().ToString()); + } else if (column.DefaultValue()) { return Value(column.DefaultValue()->ToString()); } return Value(); diff --git a/src/duckdb/src/function/table/version/pragma_version.cpp b/src/duckdb/src/function/table/version/pragma_version.cpp index afa2b3c02..de0f955b0 100644 --- a/src/duckdb/src/function/table/version/pragma_version.cpp +++ b/src/duckdb/src/function/table/version/pragma_version.cpp @@ -1,8 +1,8 @@ #ifndef DUCKDB_VERSION -#define DUCKDB_VERSION "0.8.2-dev4025" +#define DUCKDB_VERSION "0.8.2-dev4142" #endif #ifndef DUCKDB_SOURCE_ID -#define DUCKDB_SOURCE_ID "9698e9e6a8" +#define DUCKDB_SOURCE_ID "d5c4422f72" #endif #include "duckdb/function/table/system_functions.hpp" #include "duckdb/main/database.hpp" diff --git a/src/duckdb/src/include/duckdb/catalog/catalog.hpp b/src/duckdb/src/include/duckdb/catalog/catalog.hpp index 070ef2fc4..ff6e5ca16 100644 --- a/src/duckdb/src/include/duckdb/catalog/catalog.hpp +++ b/src/duckdb/src/include/duckdb/catalog/catalog.hpp @@ -305,6 +305,7 @@ class Catalog { static void AutoloadExtensionByConfigName(ClientContext &context, const string &configuration_name); //! Autoload the extension required for `function_name` or throw a CatalogException static bool AutoLoadExtensionByCatalogEntry(ClientContext &context, CatalogType type, const string &entry_name); + DUCKDB_API static bool TryAutoLoad(ClientContext &context, const string &extension_name) noexcept; protected: //! Reference to the database diff --git a/src/duckdb/src/include/duckdb/catalog/catalog_entry/duck_index_entry.hpp b/src/duckdb/src/include/duckdb/catalog/catalog_entry/duck_index_entry.hpp index 1d8b3681d..270c0748b 100644 --- a/src/duckdb/src/include/duckdb/catalog/catalog_entry/duck_index_entry.hpp +++ b/src/duckdb/src/include/duckdb/catalog/catalog_entry/duck_index_entry.hpp @@ -24,6 +24,8 @@ class DuckIndexEntry : public IndexCatalogEntry { public: string GetSchemaName() const override; string GetTableName() const override; + //! This drops in-memory index data and marks all blocks on disk as free blocks, allowing to reclaim them + void CommitDrop(); }; } // namespace duckdb diff --git a/src/duckdb/src/include/duckdb/common/optional_idx.hpp b/src/duckdb/src/include/duckdb/common/optional_idx.hpp index 43e899f56..28c618f2a 100644 --- a/src/duckdb/src/include/duckdb/common/optional_idx.hpp +++ b/src/duckdb/src/include/duckdb/common/optional_idx.hpp @@ -34,7 +34,7 @@ class optional_idx { void Invalidate() { index = INVALID_INDEX; } - idx_t GetIndex() { + idx_t GetIndex() const { if (index == INVALID_INDEX) { throw InternalException("Attempting to get the index of an optional_idx that is not set"); } diff --git a/src/duckdb/src/include/duckdb/core_functions/scalar/list_functions.hpp b/src/duckdb/src/include/duckdb/core_functions/scalar/list_functions.hpp index fee9ab6cd..3f129ee20 100644 --- a/src/duckdb/src/include/duckdb/core_functions/scalar/list_functions.hpp +++ b/src/duckdb/src/include/duckdb/core_functions/scalar/list_functions.hpp @@ -219,4 +219,55 @@ struct ListRangeFun { static ScalarFunctionSet GetFunctions(); }; +struct ListCosineSimilarityFun { + static constexpr const char *Name = "list_cosine_similarity"; + static constexpr const char *Parameters = "list1,list2"; + static constexpr const char *Description = "Compute the cosine similarity between two lists."; + static constexpr const char *Example = "list_cosine_similarity([1, 2, 3], [1, 2, 3])"; + + static ScalarFunctionSet GetFunctions(); +}; + +struct ListCosineSimilarityFunAlias { + using ALIAS = ListCosineSimilarityFun; + + static constexpr const char *Name = "<=>"; +}; + +struct ListDistanceFun { + static constexpr const char *Name = "list_distance"; + static constexpr const char *Parameters = "list1,list2"; + static constexpr const char *Description = "Compute the distance between two lists."; + static constexpr const char *Example = "list_distance([1, 2, 3], [1, 2, 3])"; + + static ScalarFunctionSet GetFunctions(); +}; + +struct ListDistanceFunAlias { + using ALIAS = ListDistanceFun; + + static constexpr const char *Name = "<->"; +}; + +struct ListInnerProductFun { + static constexpr const char *Name = "list_inner_product"; + static constexpr const char *Parameters = "list1,list2"; + static constexpr const char *Description = "Compute the inner product between two lists."; + static constexpr const char *Example = "list_inner_product([1, 2, 3], [1, 2, 3])"; + + static ScalarFunctionSet GetFunctions(); +}; + +struct ListDotProductFun { + using ALIAS = ListInnerProductFun; + + static constexpr const char *Name = "list_dot_product"; +}; + +struct ListInnerProductFunAlias { + using ALIAS = ListInnerProductFun; + + static constexpr const char *Name = "<#>"; +}; + } // namespace duckdb diff --git a/src/duckdb/src/include/duckdb/execution/index/art/art.hpp b/src/duckdb/src/include/duckdb/execution/index/art/art.hpp index a127f6d50..c3fc090dc 100644 --- a/src/duckdb/src/include/duckdb/execution/index/art/art.hpp +++ b/src/duckdb/src/include/duckdb/execution/index/art/art.hpp @@ -9,6 +9,8 @@ #pragma once #include "duckdb/storage/index.hpp" +#include "duckdb/execution/index/art/node.hpp" +#include "duckdb/common/array.hpp" namespace duckdb { @@ -19,7 +21,6 @@ enum class VerifyExistenceType : uint8_t { DELETE_FK = 2 // delete from a table that has a foreign key }; class ConflictManager; -class Node; class ARTKey; class FixedSizeAllocator; @@ -31,18 +32,22 @@ struct ARTFlags { }; class ART : public Index { +public: + //! FixedSizeAllocator count of the ART + static constexpr uint8_t ALLOCATOR_COUNT = 6; + public: //! Constructs an ART ART(const vector &column_ids, TableIOManager &table_io_manager, const vector> &unbound_expressions, const IndexConstraintType constraint_type, - AttachedDatabase &db, const shared_ptr> &allocators_ptr = nullptr, - BlockPointer block = BlockPointer()); - ~ART() override; + AttachedDatabase &db, + const shared_ptr, ALLOCATOR_COUNT>> &allocators_ptr = nullptr, + const BlockPointer &block = BlockPointer()); //! Root of the tree - unique_ptr tree; + Node tree = Node(); //! Fixed-size allocators holding the ART nodes - shared_ptr> allocators; + shared_ptr, ALLOCATOR_COUNT>> allocators; //! True, if the ART owns its data bool owns_data; @@ -66,6 +71,8 @@ class ART : public Index { void VerifyAppend(DataChunk &chunk) override; //! Verify that data can be appended to the index without a constraint violation using the conflict manager void VerifyAppend(DataChunk &chunk, ConflictManager &conflict_manager) override; + //! Deletes all data from the index. The lock obtained from InitializeLock must be held + void CommitDrop(IndexLock &index_lock) override; //! Delete a chunk of entries from the index. The lock obtained from InitializeLock must be held void Delete(IndexLock &lock, DataChunk &entries, Vector &row_identifiers) override; //! Insert a chunk of entries into the index @@ -103,7 +110,7 @@ class ART : public Index { string VerifyAndToString(IndexLock &state, const bool only_verify) override; //! Find the node with a matching key, or return nullptr if not found - Node Lookup(Node node, const ARTKey &key, idx_t depth); + optional_ptr Lookup(const Node &node, const ARTKey &key, idx_t depth); //! Insert a key into the tree bool Insert(Node &node, const ARTKey &key, idx_t depth, const row_t &row_id); @@ -136,6 +143,9 @@ class ART : public Index { //! Internal function to return the string representation of the ART, //! or only traverses and verifies the index string VerifyAndToStringInternal(const bool only_verify); + + //! Deserialize the allocators of the ART + void Deserialize(const BlockPointer &pointer); }; } // namespace duckdb diff --git a/src/duckdb/src/include/duckdb/execution/index/art/fixed_size_allocator.hpp b/src/duckdb/src/include/duckdb/execution/index/art/fixed_size_allocator.hpp deleted file mode 100644 index dccf7b652..000000000 --- a/src/duckdb/src/include/duckdb/execution/index/art/fixed_size_allocator.hpp +++ /dev/null @@ -1,115 +0,0 @@ -//===----------------------------------------------------------------------===// -// DuckDB -// -// duckdb/execution/index/art/fixed_size_allocator.hpp -// -// -//===----------------------------------------------------------------------===// - -#pragma once - -#include "duckdb/common/types/validity_mask.hpp" -#include "duckdb/common/unordered_set.hpp" -#include "duckdb/storage/buffer_manager.hpp" -#include "duckdb/execution/index/art/node.hpp" - -namespace duckdb { - -// structs -struct BufferEntry { - BufferEntry(const data_ptr_t &ptr, const idx_t &allocation_count) : ptr(ptr), allocation_count(allocation_count) { - } - data_ptr_t ptr; - idx_t allocation_count; -}; - -//! The FixedSizeAllocator provides pointers to fixed-size sections of pre-allocated memory buffers. -//! The pointers are Node pointers, and the leftmost byte (serialize flag and type) must always be zero. -class FixedSizeAllocator { -public: - //! Fixed size of the buffers - static constexpr idx_t BUFFER_ALLOC_SIZE = Storage::BLOCK_ALLOC_SIZE; - //! We can vacuum 10% or more of the total memory usage of the allocator - static constexpr uint8_t VACUUM_THRESHOLD = 10; - - //! Constants for fast offset calculations in the bitmask - static constexpr idx_t BASE[] = {0x00000000FFFFFFFF, 0x0000FFFF, 0x00FF, 0x0F, 0x3, 0x1}; - static constexpr uint8_t SHIFT[] = {32, 16, 8, 4, 2, 1}; - -public: - explicit FixedSizeAllocator(const idx_t allocation_size, Allocator &allocator); - ~FixedSizeAllocator(); - - //! Allocation size of one element in a buffer - idx_t allocation_size; - //! Total number of allocations - idx_t total_allocations; - //! Number of validity_t values in the bitmask - idx_t bitmask_count; - //! First starting byte of the payload - idx_t allocation_offset; - //! Number of possible allocations per buffer - idx_t allocations_per_buffer; - - //! Buffers containing the data - vector buffers; - //! Buffers with free space - unordered_set buffers_with_free_space; - - //! Minimum buffer ID of buffers that can be vacuumed - idx_t min_vacuum_buffer_id; - - //! Buffer manager of the database instance - Allocator &allocator; - -public: - //! Get a new Node pointer to data, might cause a new buffer allocation - Node New(); - //! Free the data of the Node pointer - void Free(const Node ptr); - //! Get the data of the Node pointer - template - inline T *Get(const Node ptr) const { - return (T *)Get(ptr); - } - - //! Resets the allocator, e.g., becomes necessary during DELETE FROM table - void Reset(); - - //! Returns the allocated memory size in bytes - inline idx_t GetMemoryUsage() const { - return buffers.size() * BUFFER_ALLOC_SIZE; - } - - //! Merge another FixedSizeAllocator into this allocator. Both must have the same allocation size - void Merge(FixedSizeAllocator &other); - - //! Initialize a vacuum operation, and return true, if the allocator needs a vacuum - bool InitializeVacuum(); - //! Finalize a vacuum operation by freeing all buffers exceeding the min_vacuum_buffer_id - void FinalizeVacuum(); - //! Returns true, if a Node pointer qualifies for a vacuum operation, and false otherwise - inline bool NeedsVacuum(const Node ptr) const { - if (ptr.GetBufferId() >= min_vacuum_buffer_id) { - return true; - } - return false; - } - //! Vacuums a Node pointer - Node VacuumPointer(const Node ptr); - - //! Verify that the allocation counts match the existing positions on the buffers - void Verify() const; - -private: - //! Returns the data_ptr_t of a Node pointer - inline data_ptr_t Get(const Node ptr) const { - D_ASSERT(ptr.GetBufferId() < buffers.size()); - D_ASSERT(ptr.GetOffset() < allocations_per_buffer); - return buffers[ptr.GetBufferId()].ptr + ptr.GetOffset() * allocation_size + allocation_offset; - } - //! Returns the first free offset in a bitmask - uint32_t GetOffset(ValidityMask &mask, const idx_t allocation_count); -}; - -} // namespace duckdb diff --git a/src/duckdb/src/include/duckdb/execution/index/art/iterator.hpp b/src/duckdb/src/include/duckdb/execution/index/art/iterator.hpp index 52be15cd9..437a818ba 100644 --- a/src/duckdb/src/include/duckdb/execution/index/art/iterator.hpp +++ b/src/duckdb/src/include/duckdb/execution/index/art/iterator.hpp @@ -41,11 +41,11 @@ class IteratorKey { return key_bytes[idx]; } //! Greater than operator - bool operator>(const ARTKey &k) const; + bool operator>(const ARTKey &key) const; //! Greater than or equal to operator - bool operator>=(const ARTKey &k) const; + bool operator>=(const ARTKey &key) const; //! Equal to operator - bool operator==(const ARTKey &k) const; + bool operator==(const ARTKey &key) const; private: vector key_bytes; @@ -62,10 +62,10 @@ class Iterator { //! If upper_bound is the empty ARTKey, than there is no upper bound bool Scan(const ARTKey &upper_bound, const idx_t max_count, vector &result_ids, const bool equal); //! Finds the minimum (leaf) of the current subtree - void FindMinimum(Node &node); + void FindMinimum(const Node &node); //! Finds the lower bound of the ART and adds the nodes to the stack. Returns false, if the lower //! bound exceeds the maximum value of the ART - bool LowerBound(Node &node, const ARTKey &key, const bool equal, idx_t depth); + bool LowerBound(const Node &node, const ARTKey &key, const bool equal, idx_t depth); private: //! Stack of nodes from the root to the currently active node diff --git a/src/duckdb/src/include/duckdb/execution/index/art/leaf.hpp b/src/duckdb/src/include/duckdb/execution/index/art/leaf.hpp index a8109ea27..8dc87cc85 100644 --- a/src/duckdb/src/include/duckdb/execution/index/art/leaf.hpp +++ b/src/duckdb/src/include/duckdb/execution/index/art/leaf.hpp @@ -8,7 +8,7 @@ #pragma once -#include "duckdb/execution/index/art/fixed_size_allocator.hpp" +#include "duckdb/execution/index/fixed_size_allocator.hpp" #include "duckdb/execution/index/art/art.hpp" #include "duckdb/execution/index/art/node.hpp" @@ -28,6 +28,10 @@ struct BlockPointer; //! row ID directly in the node pointer. class Leaf { public: + //! Delete copy constructors, as any Leaf can never own its memory + Leaf(const Leaf &) = delete; + Leaf &operator=(const Leaf &) = delete; + //! The number of row IDs in this leaf uint8_t count; //! Up to LEAF_SIZE row IDs @@ -43,15 +47,10 @@ class Leaf { static void New(ART &art, reference &node, const row_t *row_ids, idx_t count); //! Free the leaf (chain) static void Free(ART &art, Node &node); - //! Get a reference to the leaf - static inline Leaf &Get(const ART &art, const Node ptr) { - D_ASSERT(!ptr.IsSerialized()); - return *Node::GetAllocator(art, NType::LEAF).Get(ptr); - } //! Initializes a merge by incrementing the buffer IDs of the leaf (chain) static void InitializeMerge(ART &art, Node &node, const ARTFlags &flags); - //! Merge leaves (chains) and free all copied leaf nodes + //! Merge leaf (chains) and free all copied leaf nodes static void Merge(ART &art, Node &l_node, Node &r_node); //! Insert a row ID into a leaf @@ -60,19 +59,14 @@ class Leaf { static bool Remove(ART &art, reference &node, const row_t row_id); //! Get the total count of row IDs in the chain of leaves - static idx_t TotalCount(ART &art, Node &node); + static idx_t TotalCount(ART &art, const Node &node); //! Fill the result_ids vector with the row IDs of this leaf chain, if the total count does not exceed max_count - static bool GetRowIds(ART &art, Node &node, vector &result_ids, idx_t max_count); + static bool GetRowIds(ART &art, const Node &node, vector &result_ids, const idx_t max_count); //! Returns whether the leaf contains the row ID - static bool ContainsRowId(ART &art, Node &node, const row_t row_id); + static bool ContainsRowId(ART &art, const Node &node, const row_t row_id); //! Returns the string representation of the leaf (chain), or only traverses and verifies the leaf (chain) - static string VerifyAndToString(ART &art, Node &node); - - //! Serialize the leaf (chain) - static BlockPointer Serialize(ART &art, Node &node, MetadataWriter &writer); - //! Deserialize the leaf (chain) - static void Deserialize(ART &art, Node &node, MetadataReader &reader); + static string VerifyAndToString(ART &art, const Node &node, const bool only_verify); //! Vacuum the leaf (chain) static void Vacuum(ART &art, Node &node); diff --git a/src/duckdb/src/include/duckdb/execution/index/art/node.hpp b/src/duckdb/src/include/duckdb/execution/index/art/node.hpp index d2f2056f9..2e170d410 100644 --- a/src/duckdb/src/include/duckdb/execution/index/art/node.hpp +++ b/src/duckdb/src/include/duckdb/execution/index/art/node.hpp @@ -13,6 +13,8 @@ #include "duckdb/common/to_string.hpp" #include "duckdb/common/typedefs.hpp" #include "duckdb/common/limits.hpp" +#include "duckdb/execution/index/index_pointer.hpp" +#include "duckdb/execution/index/fixed_size_allocator.hpp" namespace duckdb { @@ -26,7 +28,7 @@ enum class NType : uint8_t { NODE_256 = 6, LEAF_INLINED = 7, }; -class FixedSizeAllocator; + class ART; class Prefix; class MetadataReader; @@ -38,9 +40,8 @@ struct ARTFlags; struct MetaBlockPointer; //! The Node is the pointer class of the ART index. -//! If the node is serialized, then the pointer points to a storage address (and has no type), -//! otherwise the pointer has a type and stores other information (e.g., a buffer location or a row ID). -class Node { +//! It inherits from the IndexPointer, and adds ART-specific functionality +class Node : public IndexPointer { public: //! Node thresholds static constexpr uint8_t NODE_48_SHRINK_THRESHOLD = 12; @@ -50,33 +51,11 @@ class Node { static constexpr uint8_t NODE_16_CAPACITY = 16; static constexpr uint8_t NODE_48_CAPACITY = 48; static constexpr uint16_t NODE_256_CAPACITY = 256; - //! Bit-shifting - static constexpr uint64_t SHIFT_OFFSET = 32; - static constexpr uint64_t SHIFT_TYPE = 56; - static constexpr uint64_t SHIFT_SERIALIZED_FLAG = 63; - //! AND operations - static constexpr uint64_t AND_OFFSET = 0x0000000000FFFFFF; - static constexpr uint64_t AND_BUFFER_ID = 0x00000000FFFFFFFF; - static constexpr uint64_t AND_IS_SET = 0xFF00000000000000; - static constexpr uint64_t AND_RESET = 0x00FFFFFFFFFFFFFF; - //! OR operations - static constexpr uint64_t SET_SERIALIZED_FLAG = 0x8000000000000000; //! Other constants static constexpr uint8_t EMPTY_MARKER = 48; static constexpr uint8_t LEAF_SIZE = 4; static constexpr uint8_t PREFIX_SIZE = 15; - -public: - //! Constructors - - //! Constructs an empty Node - Node() : data(0) {}; - //! Constructs a serialized Node pointer from a block ID and an offset - explicit Node(MetadataReader &reader); - //! Constructs an in-memory Node from a buffer ID and an offset - Node(const uint32_t buffer_id, const uint32_t offset) : data(0) { - SetPtr(buffer_id, offset); - }; + static constexpr idx_t AND_ROW_ID = 0x00FFFFFFFFFFFFFF; public: //! Get a new pointer to a node, might cause a new buffer allocation, and initialize it @@ -84,33 +63,43 @@ class Node { //! Free the node (and its subtree) static void Free(ART &art, Node &node); - //! Replace the child node at the respective byte - void ReplaceChild(const ART &art, const uint8_t byte, const Node child); + //! Get references to the allocator + static FixedSizeAllocator &GetAllocator(const ART &art, const NType type); + //! Get a (immutable) reference to the node. If dirty is false, then T should be a const class + template + static inline const NODE &Ref(const ART &art, const Node ptr, const NType type) { + return *(GetAllocator(art, type).Get(ptr, false)); + } + //! Get a (const) reference to the node. If dirty is false, then T should be a const class + template + static inline NODE &RefMutable(const ART &art, const Node ptr, const NType type) { + return *(GetAllocator(art, type).Get(ptr)); + } + + //! Replace the child node at byte + void ReplaceChild(const ART &art, const uint8_t byte, const Node child) const; //! Insert the child node at byte static void InsertChild(ART &art, Node &node, const uint8_t byte, const Node child); - //! Delete the child node at the respective byte + //! Delete the child node at byte static void DeleteChild(ART &art, Node &node, Node &prefix, const uint8_t byte); + //! Get the child (immutable) for the respective byte in the node + optional_ptr GetChild(ART &art, const uint8_t byte) const; //! Get the child for the respective byte in the node - optional_ptr GetChild(ART &art, const uint8_t byte) const; + optional_ptr GetChildMutable(ART &art, const uint8_t byte) const; + //! Get the first child (immutable) that is greater or equal to the specific byte + optional_ptr GetNextChild(ART &art, uint8_t &byte) const; //! Get the first child that is greater or equal to the specific byte - optional_ptr GetNextChild(ART &art, uint8_t &byte, const bool deserialize = true) const; - - //! Serialize the node - BlockPointer Serialize(ART &art, MetadataWriter &writer); - //! Deserialize the node - void Deserialize(ART &art); + optional_ptr GetNextChildMutable(ART &art, uint8_t &byte) const; //! Returns the string representation of the node, or only traverses and verifies the node and its subtree - string VerifyAndToString(ART &art, const bool only_verify); + string VerifyAndToString(ART &art, const bool only_verify) const; //! Returns the capacity of the node idx_t GetCapacity() const; //! Returns the matching node type for a given count static NType GetARTNodeTypeByCount(const idx_t count); - //! Get references to the different allocators - static FixedSizeAllocator &GetAllocator(const ART &art, NType type); - //! Initializes a merge by fully deserializing the subtree of the node and incrementing its buffer IDs + //! Initializes a merge by incrementing the buffer IDs of a node and its subtree void InitializeMerge(ART &art, const ARTFlags &flags); //! Merge another node into this node bool Merge(ART &art, Node &other); @@ -122,90 +111,23 @@ class Node { //! Vacuum all nodes that exceed their respective vacuum thresholds void Vacuum(ART &art, const ARTFlags &flags); - // Getters and Setters - - //! Returns whether the node is serialized or not (zero bit) - inline bool IsSerialized() const { - return data >> Node::SHIFT_SERIALIZED_FLAG; - } - //! Get the type (1st to 7th bit) - inline NType GetType() const { - D_ASSERT(!IsSerialized()); - auto type = data >> Node::SHIFT_TYPE; - D_ASSERT(type >= (uint8_t)NType::PREFIX); - D_ASSERT(type <= (uint8_t)NType::LEAF_INLINED); - return NType(type); - } - //! Get the offset (8th to 23rd bit) - inline idx_t GetOffset() const { - auto offset = data >> Node::SHIFT_OFFSET; - return offset & Node::AND_OFFSET; - } - //! Get the block/buffer ID (24th to 63rd bit) - inline idx_t GetBufferId() const { - return data & Node::AND_BUFFER_ID; - } //! Get the row ID (8th to 63rd bit) inline row_t GetRowId() const { - return data & Node::AND_RESET; - } - - //! Set the serialized flag (zero bit) - inline void SetSerialized() { - data &= Node::AND_RESET; - data |= Node::SET_SERIALIZED_FLAG; - } - //! Set the type (1st to 7th bit) - inline void SetType(const uint8_t type) { - D_ASSERT(!IsSerialized()); - data += (uint64_t)type << Node::SHIFT_TYPE; - } - //! Set the block/buffer ID (24th to 63rd bit) and offset (8th to 23rd bit) - inline void SetPtr(const uint32_t buffer_id, const uint32_t offset) { - D_ASSERT(!(data & Node::AND_RESET)); - auto shifted_offset = ((uint64_t)offset) << Node::SHIFT_OFFSET; - data += shifted_offset; - data += buffer_id; + return Get() & AND_ROW_ID; } //! Set the row ID (8th to 63rd bit) inline void SetRowId(const row_t row_id) { - D_ASSERT(!(data & Node::AND_RESET)); - data += row_id; - } - - //! Returns true, if neither the serialized flag is set nor the type - inline bool IsSet() const { - return data & Node::AND_IS_SET; - } - //! Reset the Node pointer by setting the node info to zero - inline void Reset() { - data = 0; + Set((Get() & AND_METADATA) | row_id); } - //! Adds an idx_t to a buffer ID, the rightmost 32 bits contain the buffer ID - inline void AddToBufferID(const idx_t summand) { - D_ASSERT(summand < NumericLimits().Maximum()); - data += summand; + //! Returns the type of the node, which is held in the metadata + inline NType GetType() const { + return NType(GetMetadata()); } - //! Comparison operator - inline bool operator==(const Node &node) const { - return data == node.data; + //! Assign operator + inline void operator=(const IndexPointer &ptr) { + Set(ptr.Get()); } - -private: - //! Data holds all the information contained in a Node pointer - //! [0: serialized flag, 1 - 7: type, - //! 8 - 23: offset, 24 - 63: buffer/block ID OR - //! 8 - 63: row ID] - //! NOTE: a Node pointer can be either serialized OR have a type - //! NOTE: we do not use bit fields because when using bit fields Windows compiles - //! the Node class into 16 bytes instead of the intended 8 bytes, doubling the - //! space requirements - //! https://learn.microsoft.com/en-us/cpp/cpp/cpp-bit-fields?view=msvc-170 - uint64_t data; }; - -static_assert(sizeof(Node) == sizeof(uint64_t), "Invalid size for Node type."); - } // namespace duckdb diff --git a/src/duckdb/src/include/duckdb/execution/index/art/node16.hpp b/src/duckdb/src/include/duckdb/execution/index/art/node16.hpp index b83a45968..36d85e839 100644 --- a/src/duckdb/src/include/duckdb/execution/index/art/node16.hpp +++ b/src/duckdb/src/include/duckdb/execution/index/art/node16.hpp @@ -8,32 +8,32 @@ #pragma once -#include "duckdb/execution/index/art/fixed_size_allocator.hpp" +#include "duckdb/execution/index/fixed_size_allocator.hpp" #include "duckdb/execution/index/art/art.hpp" #include "duckdb/execution/index/art/node.hpp" namespace duckdb { -//! Node16 holds up to 16 ARTNode children sorted by their key byte +//! Node16 holds up to 16 Node children sorted by their key byte class Node16 { public: + //! Delete copy constructors, as any Node16 can never own its memory + Node16(const Node16 &) = delete; + Node16 &operator=(const Node16 &) = delete; + //! Number of non-null children uint8_t count; //! Array containing all partial key bytes uint8_t key[Node::NODE_16_CAPACITY]; - //! ART node pointers to the child nodes + //! Node pointers to the child nodes Node children[Node::NODE_16_CAPACITY]; public: - //! Get a new Node16 node, might cause a new buffer allocation, and initialize it + //! Get a new Node16, might cause a new buffer allocation, and initialize it static Node16 &New(ART &art, Node &node); //! Free the node (and its subtree) static void Free(ART &art, Node &node); - //! Get a reference to the node - static inline Node16 &Get(const ART &art, const Node ptr) { - D_ASSERT(!ptr.IsSerialized()); - return *Node::GetAllocator(art, NType::NODE_16).Get(ptr); - } + //! Initializes all the fields of the node while growing a Node4 to a Node16 static Node16 &GrowNode4(ART &art, Node &node16, Node &node4); //! Initializes all fields of the node while shrinking a Node48 to a Node16 @@ -44,21 +44,20 @@ class Node16 { //! Insert a child node at byte static void InsertChild(ART &art, Node &node, const uint8_t byte, const Node child); - //! Delete the child node at the respective byte + //! Delete the child node at byte static void DeleteChild(ART &art, Node &node, const uint8_t byte); - //! Replace the child node at the respective byte + //! Replace the child node at byte void ReplaceChild(const uint8_t byte, const Node child); + //! Get the (immutable) child for the respective byte in the node + optional_ptr GetChild(const uint8_t byte) const; //! Get the child for the respective byte in the node - optional_ptr GetChild(const uint8_t byte); + optional_ptr GetChildMutable(const uint8_t byte); + //! Get the first (immutable) child that is greater or equal to the specific byte + optional_ptr GetNextChild(uint8_t &byte) const; //! Get the first child that is greater or equal to the specific byte - optional_ptr GetNextChild(uint8_t &byte); - - //! Serialize this node - BlockPointer Serialize(ART &art, MetadataWriter &writer); - //! Deserialize this node - void Deserialize(MetadataReader &reader); + optional_ptr GetNextChildMutable(uint8_t &byte); //! Vacuum the children of the node void Vacuum(ART &art, const ARTFlags &flags); diff --git a/src/duckdb/src/include/duckdb/execution/index/art/node256.hpp b/src/duckdb/src/include/duckdb/execution/index/art/node256.hpp index faa105009..bd6e9c3dd 100644 --- a/src/duckdb/src/include/duckdb/execution/index/art/node256.hpp +++ b/src/duckdb/src/include/duckdb/execution/index/art/node256.hpp @@ -8,30 +8,30 @@ #pragma once -#include "duckdb/execution/index/art/fixed_size_allocator.hpp" +#include "duckdb/execution/index/fixed_size_allocator.hpp" #include "duckdb/execution/index/art/art.hpp" #include "duckdb/execution/index/art/node.hpp" namespace duckdb { -//! Node256 holds up to 256 ARTNode children which can be directly indexed by the key byte +//! Node256 holds up to 256 Node children which can be directly indexed by the key byte class Node256 { public: + //! Delete copy constructors, as any Node256 can never own its memory + Node256(const Node256 &) = delete; + Node256 &operator=(const Node256 &) = delete; + //! Number of non-null children uint16_t count; - //! ART node pointers to the child nodes + //! Node pointers to the child nodes Node children[Node::NODE_256_CAPACITY]; public: - //! Get a new Node256 node, might cause a new buffer allocation, and initialize it + //! Get a new Node256, might cause a new buffer allocation, and initialize it static Node256 &New(ART &art, Node &node); //! Free the node (and its subtree) static void Free(ART &art, Node &node); - //! Get a reference to the node - static inline Node256 &Get(const ART &art, const Node ptr) { - D_ASSERT(!ptr.IsSerialized()); - return *Node::GetAllocator(art, NType::NODE_256).Get(ptr); - } + //! Initializes all the fields of the node while growing a Node48 to a Node256 static Node256 &GrowNode48(ART &art, Node &node256, Node &node48); @@ -40,28 +40,22 @@ class Node256 { //! Insert a child node at byte static void InsertChild(ART &art, Node &node, const uint8_t byte, const Node child); - //! Delete the child node at the respective byte + //! Delete the child node at byte static void DeleteChild(ART &art, Node &node, const uint8_t byte); - //! Replace the child node at the respective byte + //! Replace the child node at byte inline void ReplaceChild(const uint8_t byte, const Node child) { children[byte] = child; } + //! Get the (immutable) child for the respective byte in the node + optional_ptr GetChild(const uint8_t byte) const; //! Get the child for the respective byte in the node - inline optional_ptr GetChild(const uint8_t byte) { - if (children[byte].IsSet()) { - return &children[byte]; - } - return nullptr; - } + optional_ptr GetChildMutable(const uint8_t byte); + //! Get the first (immutable) child that is greater or equal to the specific byte + optional_ptr GetNextChild(uint8_t &byte) const; //! Get the first child that is greater or equal to the specific byte - optional_ptr GetNextChild(uint8_t &byte); - - //! Serialize this node - BlockPointer Serialize(ART &art, MetadataWriter &writer); - //! Deserialize this node - void Deserialize(MetadataReader &reader); + optional_ptr GetNextChildMutable(uint8_t &byte); //! Vacuum the children of the node void Vacuum(ART &art, const ARTFlags &flags); diff --git a/src/duckdb/src/include/duckdb/execution/index/art/node4.hpp b/src/duckdb/src/include/duckdb/execution/index/art/node4.hpp index 7470a882d..86952b854 100644 --- a/src/duckdb/src/include/duckdb/execution/index/art/node4.hpp +++ b/src/duckdb/src/include/duckdb/execution/index/art/node4.hpp @@ -8,32 +8,32 @@ #pragma once -#include "duckdb/execution/index/art/fixed_size_allocator.hpp" +#include "duckdb/execution/index/fixed_size_allocator.hpp" #include "duckdb/execution/index/art/art.hpp" #include "duckdb/execution/index/art/node.hpp" namespace duckdb { -//! Node4 holds up to four ARTNode children sorted by their key byte +//! Node4 holds up to four Node children sorted by their key byte class Node4 { public: + //! Delete copy constructors, as any Node4 can never own its memory + Node4(const Node4 &) = delete; + Node4 &operator=(const Node4 &) = delete; + //! Number of non-null children uint8_t count; //! Array containing all partial key bytes uint8_t key[Node::NODE_4_CAPACITY]; - //! ART node pointers to the child nodes + //! Node pointers to the child nodes Node children[Node::NODE_4_CAPACITY]; public: - //! Get a new Node4 node, might cause a new buffer allocation, and initialize it + //! Get a new Node4, might cause a new buffer allocation, and initialize it static Node4 &New(ART &art, Node &node); //! Free the node (and its subtree) static void Free(ART &art, Node &node); - //! Get a reference to the node - static inline Node4 &Get(const ART &art, const Node ptr) { - D_ASSERT(!ptr.IsSerialized()); - return *Node::GetAllocator(art, NType::NODE_4).Get(ptr); - } + //! Initializes all fields of the node while shrinking a Node16 to a Node4 static Node4 &ShrinkNode16(ART &art, Node &node4, Node &node16); @@ -42,21 +42,20 @@ class Node4 { //! Insert a child node at byte static void InsertChild(ART &art, Node &node, const uint8_t byte, const Node child); - //! Delete the child node at the respective byte + //! Delete the child node at byte static void DeleteChild(ART &art, Node &node, Node &prefix, const uint8_t byte); - //! Replace the child node at the respective byte + //! Replace the child node at byte void ReplaceChild(const uint8_t byte, const Node child); + //! Get the (immutable) child for the respective byte in the node + optional_ptr GetChild(const uint8_t byte) const; //! Get the child for the respective byte in the node - optional_ptr GetChild(const uint8_t byte); + optional_ptr GetChildMutable(const uint8_t byte); + //! Get the first (immutable) child that is greater or equal to the specific byte + optional_ptr GetNextChild(uint8_t &byte) const; //! Get the first child that is greater or equal to the specific byte - optional_ptr GetNextChild(uint8_t &byte); - - //! Serialize this node - BlockPointer Serialize(ART &art, MetadataWriter &writer); - //! Deserialize this node - void Deserialize(MetadataReader &reader); + optional_ptr GetNextChildMutable(uint8_t &byte); //! Vacuum the children of the node void Vacuum(ART &art, const ARTFlags &flags); diff --git a/src/duckdb/src/include/duckdb/execution/index/art/node48.hpp b/src/duckdb/src/include/duckdb/execution/index/art/node48.hpp index 718b06dba..f57eea2fa 100644 --- a/src/duckdb/src/include/duckdb/execution/index/art/node48.hpp +++ b/src/duckdb/src/include/duckdb/execution/index/art/node48.hpp @@ -8,33 +8,33 @@ #pragma once -#include "duckdb/execution/index/art/fixed_size_allocator.hpp" +#include "duckdb/execution/index/fixed_size_allocator.hpp" #include "duckdb/execution/index/art/art.hpp" #include "duckdb/execution/index/art/node.hpp" namespace duckdb { -//! Node48 holds up to 48 ARTNode children. It contains a child_index array which can be directly indexed by the key +//! Node48 holds up to 48 Node children. It contains a child_index array which can be directly indexed by the key //! byte, and which contains the position of the child node in the children array class Node48 { public: + //! Delete copy constructors, as any Node48 can never own its memory + Node48(const Node48 &) = delete; + Node48 &operator=(const Node48 &) = delete; + //! Number of non-null children uint8_t count; //! Array containing all possible partial key bytes, those not set have an EMPTY_MARKER uint8_t child_index[Node::NODE_256_CAPACITY]; - //! ART node pointers to the child nodes + //! Node pointers to the child nodes Node children[Node::NODE_48_CAPACITY]; public: - //! Get a new Node48 node, might cause a new buffer allocation, and initialize it + //! Get a new Node48, might cause a new buffer allocation, and initialize it static Node48 &New(ART &art, Node &node); //! Free the node (and its subtree) static void Free(ART &art, Node &node); - //! Get a reference to the node - static inline Node48 &Get(const ART &art, const Node ptr) { - D_ASSERT(!ptr.IsSerialized()); - return *Node::GetAllocator(art, NType::NODE_48).Get(ptr); - } + //! Initializes all the fields of the node while growing a Node16 to a Node48 static Node48 &GrowNode16(ART &art, Node &node48, Node &node16); //! Initializes all fields of the node while shrinking a Node256 to a Node48 @@ -45,30 +45,23 @@ class Node48 { //! Insert a child node at byte static void InsertChild(ART &art, Node &node, const uint8_t byte, const Node child); - //! Delete the child node at the respective byte + //! Delete the child node at byte static void DeleteChild(ART &art, Node &node, const uint8_t byte); - //! Replace the child node at the respective byte + //! Replace the child node at byte inline void ReplaceChild(const uint8_t byte, const Node child) { D_ASSERT(child_index[byte] != Node::EMPTY_MARKER); children[child_index[byte]] = child; } + //! Get the (immutable) child for the respective byte in the node + optional_ptr GetChild(const uint8_t byte) const; //! Get the child for the respective byte in the node - inline optional_ptr GetChild(const uint8_t byte) { - if (child_index[byte] != Node::EMPTY_MARKER) { - D_ASSERT(children[child_index[byte]].IsSet()); - return &children[child_index[byte]]; - } - return nullptr; - } + optional_ptr GetChildMutable(const uint8_t byte); + //! Get the first (immutable) child that is greater or equal to the specific byte + optional_ptr GetNextChild(uint8_t &byte) const; //! Get the first child that is greater or equal to the specific byte - optional_ptr GetNextChild(uint8_t &byte); - - //! Serialize this node - BlockPointer Serialize(ART &art, MetadataWriter &writer); - //! Deserialize this node - void Deserialize(MetadataReader &reader); + optional_ptr GetNextChildMutable(uint8_t &byte); //! Vacuum the children of the node void Vacuum(ART &art, const ARTFlags &flags); diff --git a/src/duckdb/src/include/duckdb/execution/index/art/prefix.hpp b/src/duckdb/src/include/duckdb/execution/index/art/prefix.hpp index 9d9d88013..7c3068fb3 100644 --- a/src/duckdb/src/include/duckdb/execution/index/art/prefix.hpp +++ b/src/duckdb/src/include/duckdb/execution/index/art/prefix.hpp @@ -7,8 +7,8 @@ //===----------------------------------------------------------------------===// #pragma once +#include "duckdb/execution/index/fixed_size_allocator.hpp" #include "duckdb/execution/index/art/art.hpp" -#include "duckdb/execution/index/art/fixed_size_allocator.hpp" #include "duckdb/execution/index/art/node.hpp" namespace duckdb { @@ -17,30 +17,28 @@ namespace duckdb { class ARTKey; //! The Prefix is a special node type that contains up to PREFIX_SIZE bytes, and one byte for the count, -//! and a Node pointer. This pointer either points to another prefix -//! node or the 'actual' ART node. +//! and a Node pointer. This pointer either points to a prefix node or another Node. class Prefix { public: + //! Delete copy constructors, as any Prefix can never own its memory + Prefix(const Prefix &) = delete; + Prefix &operator=(const Prefix &) = delete; + //! Up to PREFIX_SIZE bytes of prefix data and the count uint8_t data[Node::PREFIX_SIZE + 1]; - //! A pointer to the next ART node + //! A pointer to the next Node Node ptr; public: //! Get a new empty prefix node, might cause a new buffer allocation static Prefix &New(ART &art, Node &node); //! Create a new prefix node containing a single byte and a pointer to a next node - static Prefix &New(ART &art, Node &node, uint8_t byte, Node next); + static Prefix &New(ART &art, Node &node, uint8_t byte, const Node &next = Node()); //! Get a new chain of prefix nodes, might cause new buffer allocations, //! with the node parameter holding the tail of the chain static void New(ART &art, reference &node, const ARTKey &key, const uint32_t depth, uint32_t count); //! Free the node (and its subtree) static void Free(ART &art, Node &node); - //! Get a reference to the prefix - static inline Prefix &Get(const ART &art, const Node ptr) { - D_ASSERT(!ptr.IsSerialized()); - return *Node::GetAllocator(art, NType::PREFIX).Get(ptr); - } //! Initializes a merge by incrementing the buffer ID of the prefix and its child node(s) static void InitializeMerge(ART &art, Node &node, const ARTFlags &flags); @@ -51,13 +49,17 @@ class Prefix { static void Concatenate(ART &art, Node &prefix_node, const uint8_t byte, Node &child_prefix_node); //! Traverse a prefix and a key until (1) encountering a non-prefix node, or (2) encountering //! a mismatching byte, in which case depth indexes the mismatching byte in the key - static idx_t Traverse(ART &art, reference &prefix_node, const ARTKey &key, idx_t &depth); + static idx_t Traverse(ART &art, reference &prefix_node, const ARTKey &key, idx_t &depth); + //! Traverse a prefix and a key until (1) encountering a non-prefix node, or (2) encountering + //! a mismatching byte, in which case depth indexes the mismatching byte in the key + static idx_t TraverseMutable(ART &art, reference &prefix_node, const ARTKey &key, idx_t &depth); //! Traverse two prefixes to find (1) that they match (so far), or (2) that they have a mismatching position, - //! or (3) that one prefix contains the other prefix + //! or (3) that one prefix contains the other prefix. This function aids in merging Nodes, and, therefore, + //! the nodes are not const static bool Traverse(ART &art, reference &l_node, reference &r_node, idx_t &mismatch_position); //! Returns the byte at position static inline uint8_t GetByte(const ART &art, const Node &prefix_node, const idx_t position) { - auto prefix = Prefix::Get(art, prefix_node); + auto &prefix = Node::Ref(art, prefix_node, NType::PREFIX); D_ASSERT(position < Node::PREFIX_SIZE); D_ASSERT(position < prefix.data[Node::PREFIX_SIZE]); return prefix.data[position]; @@ -71,12 +73,7 @@ class Prefix { static void Split(ART &art, reference &prefix_node, Node &child_node, idx_t position); //! Returns the string representation of the node, or only traverses and verifies the node and its subtree - static string VerifyAndToString(ART &art, Node &node, const bool only_verify); - - //! Serialize this node and all subsequent nodes - static BlockPointer Serialize(ART &art, Node &node, MetadataWriter &writer); - //! Deserialize this node and all subsequent prefix nodes - static void Deserialize(ART &art, Node &node, MetadataReader &reader); + static string VerifyAndToString(ART &art, const Node &node, const bool only_verify); //! Vacuum the child of the node static void Vacuum(ART &art, Node &node, const ARTFlags &flags); @@ -88,8 +85,5 @@ class Prefix { //! Appends the other_prefix and all its subsequent prefix nodes to this prefix node. //! Also frees all copied/appended nodes void Append(ART &art, Node other_prefix); - //! Get the total count of bytes in the chain of prefixes, with the node reference pointing to first non-prefix node - static idx_t TotalCount(ART &art, reference &node); }; - } // namespace duckdb diff --git a/src/duckdb/src/include/duckdb/execution/index/fixed_size_allocator.hpp b/src/duckdb/src/include/duckdb/execution/index/fixed_size_allocator.hpp new file mode 100644 index 000000000..74199fcc8 --- /dev/null +++ b/src/duckdb/src/include/duckdb/execution/index/fixed_size_allocator.hpp @@ -0,0 +1,126 @@ +//===----------------------------------------------------------------------===// +// DuckDB +// +// duckdb/execution/index/fixed_size_allocator.hpp +// +// +//===----------------------------------------------------------------------===// + +#pragma once + +#include "duckdb/common/types/validity_mask.hpp" +#include "duckdb/common/unordered_set.hpp" +#include "duckdb/storage/buffer_manager.hpp" +#include "duckdb/storage/metadata/metadata_manager.hpp" +#include "duckdb/storage/metadata/metadata_writer.hpp" +#include "duckdb/execution/index/fixed_size_buffer.hpp" +#include "duckdb/execution/index/index_pointer.hpp" +#include "duckdb/common/unordered_map.hpp" +#include "duckdb/common/constants.hpp" +#include "duckdb/common/map.hpp" + +namespace duckdb { + +//! The FixedSizeAllocator provides pointers to fixed-size memory segments of pre-allocated memory buffers. +//! The pointers are IndexPointers, and the leftmost byte (metadata) must always be zero. +//! It is also possible to directly request a C++ pointer to the underlying segment of an index pointer. +class FixedSizeAllocator { +public: + //! We can vacuum 10% or more of the total in-memory footprint + static constexpr uint8_t VACUUM_THRESHOLD = 10; + + //! Constants for fast offset calculations in the bitmask + static constexpr idx_t BASE[] = {0x00000000FFFFFFFF, 0x0000FFFF, 0x00FF, 0x0F, 0x3, 0x1}; + static constexpr uint8_t SHIFT[] = {32, 16, 8, 4, 2, 1}; + +public: + FixedSizeAllocator(const idx_t segment_size, BlockManager &block_manager); + + //! Block manager of the database instance + BlockManager &block_manager; + //! Buffer manager of the database instance + BufferManager &buffer_manager; + //! Metadata manager for (de)serialization + MetadataManager &metadata_manager; + +public: + //! Get a new IndexPointer to a segment, might cause a new buffer allocation + IndexPointer New(); + //! Free the segment of the IndexPointer + void Free(const IndexPointer ptr); + //! Returns a pointer of type T to a segment. If dirty is false, then T should be a const class + template + inline T *Get(const IndexPointer ptr, const bool dirty = true) { + return (T *)Get(ptr, dirty); + } + + //! Resets the allocator, e.g., during 'DELETE FROM table' + void Reset(); + + //! Returns the in-memory usage in bytes + inline idx_t GetMemoryUsage() const; + + //! Returns the upper bound of the available buffer IDs, i.e., upper_bound > max_buffer_id + idx_t GetUpperBoundBufferId() const; + //! Merge another FixedSizeAllocator into this allocator. Both must have the same segment size + void Merge(FixedSizeAllocator &other); + + //! Initialize a vacuum operation, and return true, if the allocator needs a vacuum + bool InitializeVacuum(); + //! Finalize a vacuum operation by freeing all vacuumed buffers + void FinalizeVacuum(); + //! Returns true, if an IndexPointer qualifies for a vacuum operation, and false otherwise + inline bool NeedsVacuum(const IndexPointer ptr) const { + if (vacuum_buffers.find(ptr.GetBufferId()) != vacuum_buffers.end()) { + return true; + } + return false; + } + //! Vacuums an IndexPointer + IndexPointer VacuumPointer(const IndexPointer ptr); + + //! Serializes all in-memory buffers and the metadata + BlockPointer Serialize(MetadataWriter &writer); + //! Deserializes all metadata + void Deserialize(const BlockPointer &block_pointer); + +private: + //! Allocation size of one segment in a buffer + //! We only need this value to calculate bitmask_count, bitmask_offset, and + //! available_segments_per_buffer + idx_t segment_size; + + //! Number of validity_t values in the bitmask + idx_t bitmask_count; + //! First starting byte of the payload (segments) + idx_t bitmask_offset; + //! Number of possible segment allocations per buffer + idx_t available_segments_per_buffer; + + //! Total number of allocated segments in all buffers + //! We can recalculate this by iterating over all buffers + idx_t total_segment_count; + + //! Buffers containing the segments + unordered_map buffers; + //! Buffers with free space + unordered_set buffers_with_free_space; + //! Buffers qualifying for a vacuum (helper field to allow for fast NeedsVacuum checks) + unordered_set vacuum_buffers; + +private: + //! Returns the data_ptr_t to a segment, and sets the dirty flag of the buffer containing that segment + inline data_ptr_t Get(const IndexPointer ptr, const bool dirty = true) { + D_ASSERT(ptr.GetOffset() < available_segments_per_buffer); + D_ASSERT(buffers.find(ptr.GetBufferId()) != buffers.end()); + auto &buffer = buffers.find(ptr.GetBufferId())->second; + auto buffer_ptr = buffer.Get(dirty); + return buffer_ptr + ptr.GetOffset() * segment_size + bitmask_offset; + } + //! Returns the first free offset in a bitmask + uint32_t GetOffset(ValidityMask &mask, const idx_t segment_count); + //! Returns an available buffer id + idx_t GetAvailableBufferId() const; +}; + +} // namespace duckdb diff --git a/src/duckdb/src/include/duckdb/execution/index/fixed_size_buffer.hpp b/src/duckdb/src/include/duckdb/execution/index/fixed_size_buffer.hpp new file mode 100644 index 000000000..47823757b --- /dev/null +++ b/src/duckdb/src/include/duckdb/execution/index/fixed_size_buffer.hpp @@ -0,0 +1,79 @@ +//===----------------------------------------------------------------------===// +// DuckDB +// +// duckdb/execution/index/fixed_size_buffer.hpp +// +// +//===----------------------------------------------------------------------===// + +#pragma once + +#include "duckdb/common/typedefs.hpp" +#include "duckdb/storage/buffer/block_handle.hpp" +#include "duckdb/storage/buffer/buffer_handle.hpp" + +namespace duckdb { + +class FixedSizeAllocator; +class MetadataWriter; + +//! A fixed-size buffer holds fixed-size segments of data. It lazily deserializes a buffer, if on-disk and not +//! yet in memory, and it only serializes dirty and non-written buffers to disk during +//! serialization. +class FixedSizeBuffer { +public: + //! Constructor for a new in-memory buffer + explicit FixedSizeBuffer(BlockManager &block_manager); + //! Constructor for deserializing buffer metadata from disk + FixedSizeBuffer(BlockManager &block_manager, const idx_t segment_count, const block_id_t &block_id); + + //! Block manager of the database instance + BlockManager &block_manager; + + //! The number of allocated segments + idx_t segment_count; + + //! True: the in-memory buffer is no longer consistent with a (possibly existing) copy on disk + bool dirty; + //! True: can be vacuumed after the vacuum operation + bool vacuum; + +public: + //! Returns true, if the buffer is in-memory + inline bool InMemory() const { + return buffer_handle.IsValid(); + } + //! Returns true, if the block is on-disk + inline bool OnDisk() const { + return (block_handle != nullptr) && (block_handle->BlockId() < MAXIMUM_BLOCK); + } + //! Returns the block ID + inline block_id_t BlockId() const { + D_ASSERT(OnDisk()); + return block_handle->BlockId(); + } + //! Returns a pointer to the buffer in memory, and calls Deserialize, if the buffer is not in memory + inline data_ptr_t Get(const bool dirty_p = true) { + if (!InMemory()) { + Pin(); + } + if (dirty_p) { + dirty = dirty_p; + } + return buffer_handle.Ptr(); + } + //! Destroys the in-memory buffer and the on-disk block + void Destroy(); + //! Serializes a buffer (if dirty or not on disk) + void Serialize(); + //! Pin a buffer (if not in-memory) + void Pin(); + +private: + //! The buffer handle of the in-memory buffer + BufferHandle buffer_handle; + //! The block handle of the on-disk buffer + shared_ptr block_handle; +}; + +} // namespace duckdb diff --git a/src/duckdb/src/include/duckdb/execution/index/index_pointer.hpp b/src/duckdb/src/include/duckdb/execution/index/index_pointer.hpp new file mode 100644 index 000000000..3b7f0c754 --- /dev/null +++ b/src/duckdb/src/include/duckdb/execution/index/index_pointer.hpp @@ -0,0 +1,96 @@ +//===----------------------------------------------------------------------===// +// DuckDB +// +// duckdb/execution/index/index_pointer.hpp +// +// +//===----------------------------------------------------------------------===// + +#pragma once + +#include "duckdb/common/typedefs.hpp" + +namespace duckdb { + +class IndexPointer { +public: + //! Bit-shifting + static constexpr idx_t SHIFT_OFFSET = 32; + static constexpr idx_t SHIFT_METADATA = 56; + //! AND operations + static constexpr idx_t AND_OFFSET = 0x0000000000FFFFFF; + static constexpr idx_t AND_BUFFER_ID = 0x00000000FFFFFFFF; + static constexpr idx_t AND_METADATA = 0xFF00000000000000; + +public: + //! Constructs an empty IndexPointer + IndexPointer() : data(0) {}; + //! Constructs an in-memory IndexPointer with a buffer ID and an offset + IndexPointer(const uint32_t buffer_id, const uint32_t offset) : data(0) { + auto shifted_offset = ((idx_t)offset) << SHIFT_OFFSET; + data += shifted_offset; + data += buffer_id; + }; + +public: + //! Get data (all 64 bits) + inline idx_t Get() const { + return data; + } + //! Set data (all 64 bits) + inline void Set(const idx_t data_p) { + data = data_p; + } + + //! Returns false, if the metadata is empty + inline bool HasMetadata() const { + return data & AND_METADATA; + } + //! Get metadata (zero to 7th bit) + inline uint8_t GetMetadata() const { + return data >> SHIFT_METADATA; + } + //! Set metadata (zero to 7th bit) + inline void SetMetadata(const uint8_t metadata) { + data += (idx_t)metadata << SHIFT_METADATA; + } + + //! Get the offset (8th to 23rd bit) + inline idx_t GetOffset() const { + auto offset = data >> SHIFT_OFFSET; + return offset & AND_OFFSET; + } + //! Get the buffer ID (24th to 63rd bit) + inline idx_t GetBufferId() const { + return data & AND_BUFFER_ID; + } + + //! Resets the IndexPointer + inline void Clear() { + data = 0; + } + + //! Adds an idx_t to a buffer ID, the rightmost 32 bits of data contain the buffer ID + inline void IncreaseBufferId(const idx_t summand) { + data += summand; + } + + //! Comparison operator + inline bool operator==(const IndexPointer &ptr) const { + return data == ptr.data; + } + +private: + //! Data holds all the information contained in an IndexPointer + //! [0 - 7: metadata, + //! 8 - 23: offset, 24 - 63: buffer ID] + //! NOTE: we do not use bit fields because when using bit fields Windows compiles + //! the IndexPointer class into 16 bytes instead of the intended 8 bytes, doubling the + //! space requirements + //! https://learn.microsoft.com/en-us/cpp/cpp/cpp-bit-fields?view=msvc-170 + idx_t data; +}; + +static_assert(sizeof(IndexPointer) == sizeof(idx_t), "Invalid size for IndexPointer."); + +} // namespace duckdb diff --git a/src/duckdb/src/include/duckdb/main/extension_entries.hpp b/src/duckdb/src/include/duckdb/main/extension_entries.hpp index 7c576ea36..72432d19e 100644 --- a/src/duckdb/src/include/duckdb/main/extension_entries.hpp +++ b/src/duckdb/src/include/duckdb/main/extension_entries.hpp @@ -234,6 +234,7 @@ static constexpr ExtensionEntry EXTENSION_FILE_CONTAINS[] = {{".parquet?", "parq static constexpr const char *AUTOLOADABLE_EXTENSIONS[] = { // "azure", + "aws", "autocomplete", "excel", "fts", diff --git a/src/duckdb/src/include/duckdb/main/extension_helper.hpp b/src/duckdb/src/include/duckdb/main/extension_helper.hpp index a56928c36..1d2c5463b 100644 --- a/src/duckdb/src/include/duckdb/main/extension_helper.hpp +++ b/src/duckdb/src/include/duckdb/main/extension_helper.hpp @@ -51,6 +51,7 @@ class ExtensionHelper { //! Autoload an extension by name. Depending on the current settings, this will either load or install+load static void AutoLoadExtension(ClientContext &context, const string &extension_name); + DUCKDB_API static bool TryAutoLoadExtension(ClientContext &context, const string &extension_name) noexcept; static string ExtensionDirectory(ClientContext &context); static string ExtensionDirectory(DBConfig &config, FileSystem &fs); diff --git a/src/duckdb/src/include/duckdb/parallel/task_scheduler.hpp b/src/duckdb/src/include/duckdb/parallel/task_scheduler.hpp index d3171c55d..b54da51ce 100644 --- a/src/duckdb/src/include/duckdb/parallel/task_scheduler.hpp +++ b/src/duckdb/src/include/duckdb/parallel/task_scheduler.hpp @@ -68,7 +68,7 @@ class TaskScheduler { void Signal(idx_t n); //! Yield to other threads - void YieldThread(); + static void YieldThread(); //! Set the allocator flush threshold void SetAllocatorFlushTreshold(idx_t threshold); diff --git a/src/duckdb/src/include/duckdb/storage/block.hpp b/src/duckdb/src/include/duckdb/storage/block.hpp index ac839636b..d1b13011c 100644 --- a/src/duckdb/src/include/duckdb/storage/block.hpp +++ b/src/duckdb/src/include/duckdb/storage/block.hpp @@ -35,7 +35,7 @@ struct BlockPointer { block_id_t block_id; uint32_t offset; - bool IsValid() { + bool IsValid() const { return block_id != INVALID_BLOCK; } }; diff --git a/src/duckdb/src/include/duckdb/storage/index.hpp b/src/duckdb/src/include/duckdb/storage/index.hpp index 32173a1a5..474421d4c 100644 --- a/src/duckdb/src/include/duckdb/storage/index.hpp +++ b/src/duckdb/src/include/duckdb/storage/index.hpp @@ -54,8 +54,6 @@ class Index { //! Attached database instance AttachedDatabase &db; - //! Buffer manager of the database instance - BufferManager &buffer_manager; public: //! Initialize a single predicate scan on the index with the given expression and column IDs @@ -85,6 +83,10 @@ class Index { //! Performs constraint checking for a chunk of input data virtual void CheckConstraintsForChunk(DataChunk &input, ConflictManager &conflict_manager) = 0; + //! Deletes all data from the index. The lock obtained from InitializeLock must be held + virtual void CommitDrop(IndexLock &index_lock) = 0; + //! Deletes all data from the index + void CommitDrop(); //! Delete a chunk of entries from the index. The lock obtained from InitializeLock must be held virtual void Delete(IndexLock &state, DataChunk &entries, Vector &row_identifiers) = 0; //! Obtains a lock and calls Delete while holding that lock @@ -125,11 +127,11 @@ class Index { return (constraint_type == IndexConstraintType::FOREIGN); } - //! Serializes the index and returns the pair of block_id offset positions + //! Serializes the index to disk virtual BlockPointer Serialize(MetadataWriter &writer); - //! Returns the serialized data pointer to the block and offset of the serialized index - BlockPointer GetSerializedDataPointer() const { - return serialized_data_pointer; + //! Returns the serialized root block pointer + BlockPointer GetRootBlockPointer() const { + return root_block_pointer; } //! Execute the index expressions on an input chunk @@ -139,8 +141,8 @@ class Index { protected: //! Lock used for any changes to the index mutex lock; - //! Pointer to serialized index data - BlockPointer serialized_data_pointer; + //! Pointer to the index on disk + BlockPointer root_block_pointer; private: //! Bound expressions used during expression execution diff --git a/src/duckdb/src/include/duckdb/storage/metadata/metadata_writer.hpp b/src/duckdb/src/include/duckdb/storage/metadata/metadata_writer.hpp index dbed32041..1d4d0c8ae 100644 --- a/src/duckdb/src/include/duckdb/storage/metadata/metadata_writer.hpp +++ b/src/duckdb/src/include/duckdb/storage/metadata/metadata_writer.hpp @@ -14,6 +14,9 @@ namespace duckdb { class MetadataWriter : public Serializer { public: + MetadataWriter(const MetadataWriter &) = delete; + MetadataWriter &operator=(const MetadataWriter &) = delete; + explicit MetadataWriter(MetadataManager &manager); ~MetadataWriter() override; diff --git a/src/duckdb/src/main/extension/extension_helper.cpp b/src/duckdb/src/main/extension/extension_helper.cpp index c02e945a0..99f544250 100644 --- a/src/duckdb/src/main/extension/extension_helper.cpp +++ b/src/duckdb/src/main/extension/extension_helper.cpp @@ -194,13 +194,30 @@ string ExtensionHelper::AddExtensionInstallHintToErrorMsg(ClientContext &context return base_error; } +bool ExtensionHelper::TryAutoLoadExtension(ClientContext &context, const string &extension_name) noexcept { + auto &dbconfig = DBConfig::GetConfig(context); + try { + if (dbconfig.options.autoinstall_known_extensions) { + ExtensionHelper::InstallExtension(context, extension_name, false, + context.config.autoinstall_extension_repo); + } + ExtensionHelper::LoadExternalExtension(context, extension_name); + return true; + } catch (...) { + return false; + } + return false; +} + void ExtensionHelper::AutoLoadExtension(ClientContext &context, const string &extension_name) { auto &dbconfig = DBConfig::GetConfig(context); try { +#ifndef DUCKDB_WASM if (dbconfig.options.autoinstall_known_extensions) { ExtensionHelper::InstallExtension(context, extension_name, false, context.config.autoinstall_extension_repo); } +#endif ExtensionHelper::LoadExternalExtension(context, extension_name); } catch (Exception &e) { throw AutoloadException(extension_name, e); diff --git a/src/duckdb/src/main/extension/extension_install.cpp b/src/duckdb/src/main/extension/extension_install.cpp index 2f0907068..9a453196d 100644 --- a/src/duckdb/src/main/extension/extension_install.cpp +++ b/src/duckdb/src/main/extension/extension_install.cpp @@ -45,7 +45,7 @@ const vector ExtensionHelper::PathComponents() { string ExtensionHelper::ExtensionDirectory(DBConfig &config, FileSystem &fs) { #ifdef WASM_LOADABLE_EXTENSIONS - static_assert(0, "ExtensionDirectory functionality is not supported in duckdb-wasm"); + throw PermissionException("ExtensionDirectory functionality is not supported in duckdb-wasm"); #endif string extension_directory; if (!config.options.extension_directory.empty()) { // create the extension directory if not present @@ -159,9 +159,11 @@ void WriteExtensionFileToDisk(FileSystem &fs, const string &path, void *data, id string ExtensionHelper::ExtensionUrlTemplate(optional_ptr client_config, const string &repository) { string default_endpoint = "http://extensions.duckdb.org"; - string versioned_path = "/${REVISION}/${PLATFORM}/${NAME}.duckdb_extension.gz"; + string versioned_path = "/${REVISION}/${PLATFORM}/${NAME}.duckdb_extension"; #ifdef WASM_LOADABLE_EXTENSIONS - versioned_path = "/duckdb-wasm" + versioned_path; + versioned_path = "/duckdb-wasm" + versioned_path + ".wasm"; +#else + versioned_path = versioned_path + ".gz"; #endif string custom_endpoint = client_config ? client_config->custom_extension_repo : string(); string endpoint; diff --git a/src/duckdb/src/main/extension/extension_load.cpp b/src/duckdb/src/main/extension/extension_load.cpp index bbd3349ef..80d24c298 100644 --- a/src/duckdb/src/main/extension/extension_load.cpp +++ b/src/duckdb/src/main/extension/extension_load.cpp @@ -70,14 +70,15 @@ bool ExtensionHelper::TryInitialLoad(DBConfig &config, FileSystem &fs, const str // shorthand case if (!ExtensionHelper::IsFullPath(extension)) { + string extension_name = ApplyExtensionAlias(extension); #ifdef WASM_LOADABLE_EXTENSIONS string url_template = ExtensionUrlTemplate(client_config, ""); string url = ExtensionFinalizeUrlTemplate(url_template, extension_name); char *str = (char *)EM_ASM_PTR( { - var jsString = ((typeof runtime == = 'object') && runtime && - (typeof runtime.whereToLoad == = 'function') && runtime.whereToLoad) + var jsString = ((typeof runtime == 'object') && runtime && (typeof runtime.whereToLoad == 'function') && + runtime.whereToLoad) ? runtime.whereToLoad(UTF8ToString($0)) : (UTF8ToString($1)); var lengthBytes = lengthBytesUTF8(jsString) + 1; @@ -105,7 +106,6 @@ bool ExtensionHelper::TryInitialLoad(DBConfig &config, FileSystem &fs, const str for (auto &path_ele : path_components) { local_path = fs.JoinPath(local_path, path_ele); } - string extension_name = ApplyExtensionAlias(extension); filename = fs.JoinPath(local_path, extension_name + ".duckdb_extension"); #endif } diff --git a/src/duckdb/src/planner/binder/expression/bind_function_expression.cpp b/src/duckdb/src/planner/binder/expression/bind_function_expression.cpp index 01e568b3f..c2a6179ed 100644 --- a/src/duckdb/src/planner/binder/expression/bind_function_expression.cpp +++ b/src/duckdb/src/planner/binder/expression/bind_function_expression.cpp @@ -69,21 +69,30 @@ BindResult ExpressionBinder::BindExpression(FunctionExpression &function, idx_t } switch (func->type) { - case CatalogType::SCALAR_FUNCTION_ENTRY: + case CatalogType::SCALAR_FUNCTION_ENTRY: { // scalar function // check for lambda parameters, ignore ->> operator (JSON extension) + bool try_bind_lambda = false; if (function.function_name != "->>") { for (auto &child : function.children) { if (child->expression_class == ExpressionClass::LAMBDA) { - return BindLambdaFunction(function, func->Cast(), depth); + try_bind_lambda = true; } } } + if (try_bind_lambda) { + auto result = BindLambdaFunction(function, func->Cast(), depth); + if (!result.HasError()) { + // Lambda bind successful + return result; + } + } + // other scalar function return BindFunction(function, func->Cast(), depth); - + } case CatalogType::MACRO_ENTRY: // macro function return BindMacro(function, func->Cast(), depth, expr_ptr); @@ -134,7 +143,7 @@ BindResult ExpressionBinder::BindLambdaFunction(FunctionExpression &function, Sc string error; if (function.children.size() != 2) { - throw BinderException("Invalid function arguments!"); + return BindResult("Invalid function arguments!"); } D_ASSERT(function.children[1]->GetExpressionClass() == ExpressionClass::LAMBDA); @@ -148,7 +157,7 @@ BindResult ExpressionBinder::BindLambdaFunction(FunctionExpression &function, Sc auto &list_child = BoundExpression::GetExpression(*function.children[0]); if (list_child->return_type.id() != LogicalTypeId::LIST && list_child->return_type.id() != LogicalTypeId::SQLNULL && list_child->return_type.id() != LogicalTypeId::UNKNOWN) { - throw BinderException(" Invalid LIST argument to " + function.function_name + "!"); + return BindResult(" Invalid LIST argument to " + function.function_name + "!"); } LogicalType list_child_type = list_child->return_type.id(); diff --git a/src/duckdb/src/storage/checkpoint/table_data_writer.cpp b/src/duckdb/src/storage/checkpoint/table_data_writer.cpp index 55a6ff6e7..cd75bfcce 100644 --- a/src/duckdb/src/storage/checkpoint/table_data_writer.cpp +++ b/src/duckdb/src/storage/checkpoint/table_data_writer.cpp @@ -69,9 +69,8 @@ void SingleFileTableDataWriter::FinalizeTable(TableStatistics &&global_stats, Da // Write-off to metadata block ids and offsets of indexes meta_data_writer.Write(index_pointers.size()); - for (auto &block_info : index_pointers) { - meta_data_writer.Write(block_info.block_id); - meta_data_writer.Write(block_info.offset); + for (const auto &index_pointer : index_pointers) { + meta_data_writer.Write(index_pointer); } } diff --git a/src/duckdb/src/storage/checkpoint_manager.cpp b/src/duckdb/src/storage/checkpoint_manager.cpp index c8fd8c992..b822dd806 100644 --- a/src/duckdb/src/storage/checkpoint_manager.cpp +++ b/src/duckdb/src/storage/checkpoint_manager.cpp @@ -262,7 +262,8 @@ void CheckpointWriter::WriteSchema(SchemaCatalogEntry &schema) { void CheckpointReader::ReadSchema(ClientContext &context, MetadataReader &reader) { // read the schema and create it in the catalog auto info = CatalogEntry::Deserialize(reader); - // we set create conflict to ignore to ignore the failure of recreating the main schema + + // we set create conflict to IGNORE_ON_CONFLICT, so that we can ignore a failure when recreating the main schema info->on_conflict = OnCreateConflict::IGNORE_ON_CONFLICT; catalog.CreateSchema(context, info->Cast()); @@ -336,14 +337,12 @@ void CheckpointReader::ReadSequence(ClientContext &context, MetadataReader &read // Indexes //===--------------------------------------------------------------------===// void CheckpointWriter::WriteIndex(IndexCatalogEntry &index_catalog) { - // The index data should already have been written as part of WriteTableData. - // Here, we need only serialize the pointer to that data. - auto root_offset = index_catalog.index->GetSerializedDataPointer(); + // we write the index data in WriteTableData + // here, we only write the root pointer + const auto root_block_pointer = index_catalog.index->GetRootBlockPointer(); auto &metadata_writer = GetMetadataWriter(); index_catalog.Serialize(metadata_writer); - // Serialize the Block id and offset of root node - metadata_writer.Write(root_offset.block_id); - metadata_writer.Write(root_offset.offset); + metadata_writer.Write(root_block_pointer); } void CheckpointReader::ReadIndex(ClientContext &context, MetadataReader &reader) { @@ -358,10 +357,8 @@ void CheckpointReader::ReadIndex(ClientContext &context, MetadataReader &reader) auto &index_catalog = schema_catalog.CreateIndex(context, index_info, table_catalog)->Cast(); index_catalog.info = table_catalog.GetStorage().info; - // we deserialize the index lazily, i.e., we do not need to load any node information - // except the root block id and offset - auto root_block_id = reader.Read(); - auto root_offset = reader.Read(); + // we deserialize the index lazily, i.e., we only load the root block pointer + const auto index_block_pointer = reader.Read(); // obtain the expressions of the ART from the index metadata vector> unbound_expressions; @@ -401,9 +398,8 @@ void CheckpointReader::ReadIndex(ClientContext &context, MetadataReader &reader) switch (index_info.index_type) { case IndexType::ART: { auto &storage = table_catalog.GetStorage(); - auto art = - make_uniq(index_info.column_ids, TableIOManager::Get(storage), std::move(unbound_expressions), - index_info.constraint_type, storage.db, nullptr, BlockPointer(root_block_id, root_offset)); + auto art = make_uniq(index_info.column_ids, TableIOManager::Get(storage), std::move(unbound_expressions), + index_info.constraint_type, storage.db, nullptr, index_block_pointer); index_catalog.index = art.get(); storage.info->indexes.AddIndex(std::move(art)); break; @@ -450,7 +446,7 @@ void CheckpointReader::ReadTableMacro(ClientContext &context, MetadataReader &re // Table Metadata //===--------------------------------------------------------------------===// void CheckpointWriter::WriteTable(TableCatalogEntry &table) { - // write the table meta data + // write the table metadata table.Serialize(GetMetadataWriter()); // now we need to write the table data. if (auto writer = GetTableDataWriter(table)) { @@ -483,12 +479,11 @@ void CheckpointReader::ReadTableData(ClientContext &context, MetadataReader &rea data_reader.ReadTableData(); bound_info.data->total_rows = reader.Read(); - // Get any indexes block info - idx_t num_indexes = reader.Read(); - for (idx_t i = 0; i < num_indexes; i++) { - auto idx_block_id = reader.Read(); - auto idx_offset = reader.Read(); - bound_info.indexes.emplace_back(idx_block_id, idx_offset); + // get the root block pointers of each index + idx_t index_count = reader.Read(); + for (idx_t i = 0; i < index_count; i++) { + const auto index_pointer = reader.Read(); + bound_info.indexes.emplace_back(index_pointer); } } diff --git a/src/duckdb/src/storage/data_table.cpp b/src/duckdb/src/storage/data_table.cpp index 346a6ff8f..2b5fbc703 100644 --- a/src/duckdb/src/storage/data_table.cpp +++ b/src/duckdb/src/storage/data_table.cpp @@ -429,13 +429,13 @@ void DataTable::VerifyForeignKeyConstraint(const BoundForeignKeyConstraint &bfk, data_table.info->indexes.VerifyForeignKey(*dst_keys_ptr, dst_chunk, regular_conflicts); regular_conflicts.Finalize(); auto ®ular_matches = regular_conflicts.Conflicts(); - // check whether or not the chunk can be inserted or deleted into the referenced table' transaction local storage - auto &local_storage = LocalStorage::Get(context, db); + // check if we can insert the chunk into the reference table's local storage + auto &local_storage = LocalStorage::Get(context, db); bool error = IsForeignKeyConstraintError(is_append, count, regular_matches); bool transaction_error = false; - bool transaction_check = local_storage.Find(data_table); + if (transaction_check) { auto &transact_index = local_storage.GetIndexes(data_table); transact_index.VerifyForeignKey(*dst_keys_ptr, dst_chunk, transaction_conflicts); diff --git a/src/duckdb/src/storage/index.cpp b/src/duckdb/src/storage/index.cpp index 3acc818d0..74b19c94e 100644 --- a/src/duckdb/src/storage/index.cpp +++ b/src/duckdb/src/storage/index.cpp @@ -12,7 +12,7 @@ Index::Index(AttachedDatabase &db, IndexType type, TableIOManager &table_io_mana IndexConstraintType constraint_type_p) : type(type), table_io_manager(table_io_manager), column_ids(column_ids_p), constraint_type(constraint_type_p), - db(db), buffer_manager(BufferManager::GetBufferManager(db)) { + db(db) { for (auto &expr : unbound_expressions) { types.push_back(expr->return_type.InternalType()); @@ -39,6 +39,12 @@ PreservedError Index::Append(DataChunk &entries, Vector &row_identifiers) { return Append(state, entries, row_identifiers); } +void Index::CommitDrop() { + IndexLock index_lock; + InitializeLock(index_lock); + CommitDrop(index_lock); +} + void Index::Delete(DataChunk &entries, Vector &row_identifiers) { IndexLock state; InitializeLock(state); diff --git a/src/duckdb/src/storage/metadata/metadata_manager.cpp b/src/duckdb/src/storage/metadata/metadata_manager.cpp index 4634e36d6..7aa49ecba 100644 --- a/src/duckdb/src/storage/metadata/metadata_manager.cpp +++ b/src/duckdb/src/storage/metadata/metadata_manager.cpp @@ -13,7 +13,7 @@ MetadataManager::~MetadataManager() { MetadataHandle MetadataManager::AllocateHandle() { // check if there is any free space left in an existing block - // if not allocate a new bloc + // if not allocate a new block block_id_t free_block = INVALID_BLOCK; for (auto &kv : blocks) { auto &block = kv.second; @@ -230,27 +230,27 @@ void MetadataBlock::FreeBlocksFromInteger(idx_t free_list) { } void MetadataManager::MarkBlocksAsModified() { - if (!modified_blocks.empty()) { - // for any blocks that were modified in the last checkpoint - set them to free blocks currently - for (auto &kv : modified_blocks) { - auto block_id = kv.first; - idx_t modified_list = kv.second; - auto entry = blocks.find(block_id); - D_ASSERT(entry != blocks.end()); - auto &block = entry->second; - idx_t current_free_blocks = block.FreeBlocksToInteger(); - // merge the current set of free blocks with the modified blocks - idx_t new_free_blocks = current_free_blocks | modified_list; - // if (new_free_blocks == NumericLimits::Maximum()) { - // // if new free_blocks is all blocks - mark entire block as modified - // blocks.erase(entry); - // block_manager.MarkBlockAsModified(block_id); - // } else { - // set the new set of free blocks - block.FreeBlocksFromInteger(new_free_blocks); - // } - } + + // for any blocks that were modified in the last checkpoint - set them to free blocks currently + for (auto &kv : modified_blocks) { + auto block_id = kv.first; + idx_t modified_list = kv.second; + auto entry = blocks.find(block_id); + D_ASSERT(entry != blocks.end()); + auto &block = entry->second; + idx_t current_free_blocks = block.FreeBlocksToInteger(); + // merge the current set of free blocks with the modified blocks + idx_t new_free_blocks = current_free_blocks | modified_list; + // if (new_free_blocks == NumericLimits::Maximum()) { + // // if new free_blocks is all blocks - mark entire block as modified + // blocks.erase(entry); + // block_manager.MarkBlockAsModified(block_id); + // } else { + // set the new set of free blocks + block.FreeBlocksFromInteger(new_free_blocks); + // } } + modified_blocks.clear(); for (auto &kv : blocks) { auto &block = kv.second; diff --git a/src/duckdb/src/storage/standard_buffer_manager.cpp b/src/duckdb/src/storage/standard_buffer_manager.cpp index 1637d78f7..87505a80d 100644 --- a/src/duckdb/src/storage/standard_buffer_manager.cpp +++ b/src/duckdb/src/storage/standard_buffer_manager.cpp @@ -77,14 +77,6 @@ idx_t StandardBufferManager::GetMaxMemory() const { return buffer_pool.GetMaxMemory(); } -// POTENTIALLY PROBLEMATIC -// void StandardBufferManager::IncreaseUsedMemory(idx_t size, bool unsafe) { -// if (!unsafe && buffer_pool.GetUsedMemory() + size > buffer_pool.GetMaxMemory()) { -// throw OutOfMemoryException("Failed to allocate data of size %lld%s", size, InMemoryWarning()); -// } -// buffer_pool.IncreaseUsedMemory(size); -//} - template TempBufferPoolReservation StandardBufferManager::EvictBlocksOrThrow(idx_t memory_delta, unique_ptr *buffer, ARGS... args) { diff --git a/src/duckdb/src/storage/storage_info.cpp b/src/duckdb/src/storage/storage_info.cpp index f920245ad..537c2502a 100644 --- a/src/duckdb/src/storage/storage_info.cpp +++ b/src/duckdb/src/storage/storage_info.cpp @@ -2,7 +2,7 @@ namespace duckdb { -const uint64_t VERSION_NUMBER = 57; +const uint64_t VERSION_NUMBER = 58; struct StorageVersionInfo { const char *version_name; diff --git a/src/duckdb/src/storage/table_index_list.cpp b/src/duckdb/src/storage/table_index_list.cpp index d6cb4deef..3223f5e30 100644 --- a/src/duckdb/src/storage/table_index_list.cpp +++ b/src/duckdb/src/storage/table_index_list.cpp @@ -54,7 +54,7 @@ void TableIndexList::VerifyForeignKey(const vector &fk_keys, Data ? ForeignKeyType::FK_TYPE_PRIMARY_KEY_TABLE : ForeignKeyType::FK_TYPE_FOREIGN_KEY_TABLE; - // check whether or not the chunk can be inserted or deleted into the referenced table' storage + // check whether the chunk can be inserted or deleted into the referenced table storage auto index = FindForeignKeyIndex(fk_keys, fk_type); if (!index) { throw InternalException("Internal Foreign Key error: could not find index to verify..."); diff --git a/src/duckdb/src/transaction/commit_state.cpp b/src/duckdb/src/transaction/commit_state.cpp index 5034e4620..af784c091 100644 --- a/src/duckdb/src/transaction/commit_state.cpp +++ b/src/duckdb/src/transaction/commit_state.cpp @@ -1,5 +1,6 @@ #include "duckdb/transaction/commit_state.hpp" +#include "duckdb/catalog/catalog_entry/duck_index_entry.hpp" #include "duckdb/catalog/catalog_entry/duck_table_entry.hpp" #include "duckdb/catalog/catalog_entry/type_catalog_entry.hpp" #include "duckdb/catalog/catalog_set.hpp" @@ -124,9 +125,12 @@ void CommitState::WriteCatalogEntry(CatalogEntry &entry, data_ptr_t dataptr) { case CatalogType::TYPE_ENTRY: log->WriteDropType(entry.Cast()); break; - case CatalogType::INDEX_ENTRY: + case CatalogType::INDEX_ENTRY: { + auto &index_entry = entry.Cast(); + index_entry.CommitDrop(); log->WriteDropIndex(entry.Cast()); break; + } case CatalogType::PREPARED_STATEMENT: case CatalogType::SCALAR_FUNCTION_ENTRY: // do nothing, indexes/prepared statements/functions aren't persisted to disk diff --git a/src/duckdb/ub_src_core_functions_scalar_list.cpp b/src/duckdb/ub_src_core_functions_scalar_list.cpp index 62ff96d4b..0ac5bed76 100644 --- a/src/duckdb/ub_src_core_functions_scalar_list.cpp +++ b/src/duckdb/ub_src_core_functions_scalar_list.cpp @@ -10,5 +10,11 @@ #include "src/core_functions/scalar/list/list_sort.cpp" +#include "src/core_functions/scalar/list/list_distance.cpp" + +#include "src/core_functions/scalar/list/list_cosine_similarity.cpp" + +#include "src/core_functions/scalar/list/list_inner_product.cpp" + #include "src/core_functions/scalar/list/range.cpp" diff --git a/src/duckdb/ub_src_execution_index.cpp b/src/duckdb/ub_src_execution_index.cpp new file mode 100644 index 000000000..32dd0bf7b --- /dev/null +++ b/src/duckdb/ub_src_execution_index.cpp @@ -0,0 +1,4 @@ +#include "src/execution/index/fixed_size_allocator.cpp" + +#include "src/execution/index/fixed_size_buffer.cpp" + diff --git a/src/duckdb/ub_src_execution_index_art.cpp b/src/duckdb/ub_src_execution_index_art.cpp index f860a01a0..eb3d40304 100644 --- a/src/duckdb/ub_src_execution_index_art.cpp +++ b/src/duckdb/ub_src_execution_index_art.cpp @@ -2,8 +2,6 @@ #include "src/execution/index/art/node.cpp" -#include "src/execution/index/art/fixed_size_allocator.cpp" - #include "src/execution/index/art/iterator.cpp" #include "src/execution/index/art/leaf.cpp" diff --git a/src/include/sources.mk b/src/include/sources.mk index 411696555..841478edf 100644 --- a/src/include/sources.mk +++ b/src/include/sources.mk @@ -1 +1 @@ -SOURCES=duckdb/ub_src_catalog.o duckdb/ub_src_catalog_catalog_entry.o duckdb/ub_src_catalog_default.o duckdb/ub_src_common_adbc.o duckdb/ub_src_common_adbc_nanoarrow.o duckdb/ub_src_common.o duckdb/ub_src_common_arrow_appender.o duckdb/ub_src_common_arrow.o duckdb/ub_src_common_crypto.o duckdb/ub_src_common_enums.o duckdb/ub_src_common_operator.o duckdb/ub_src_common_progress_bar.o duckdb/ub_src_common_row_operations.o duckdb/ub_src_common_serializer.o duckdb/ub_src_common_sort.o duckdb/ub_src_common_types.o duckdb/ub_src_common_types_column.o duckdb/ub_src_common_types_row.o duckdb/ub_src_common_value_operations.o duckdb/src/common/vector_operations/boolean_operators.o duckdb/src/common/vector_operations/comparison_operators.o duckdb/src/common/vector_operations/generators.o duckdb/src/common/vector_operations/is_distinct_from.o duckdb/src/common/vector_operations/null_operations.o duckdb/src/common/vector_operations/numeric_inplace_operators.o duckdb/src/common/vector_operations/vector_cast.o duckdb/src/common/vector_operations/vector_copy.o duckdb/src/common/vector_operations/vector_hash.o duckdb/src/common/vector_operations/vector_storage.o duckdb/ub_src_core_functions_aggregate_algebraic.o duckdb/ub_src_core_functions_aggregate_distributive.o duckdb/ub_src_core_functions_aggregate_holistic.o duckdb/ub_src_core_functions_aggregate_nested.o duckdb/ub_src_core_functions_aggregate_regression.o duckdb/ub_src_core_functions.o duckdb/ub_src_core_functions_scalar_bit.o duckdb/ub_src_core_functions_scalar_blob.o duckdb/ub_src_core_functions_scalar_date.o duckdb/ub_src_core_functions_scalar_debug.o duckdb/ub_src_core_functions_scalar_enum.o duckdb/ub_src_core_functions_scalar_generic.o duckdb/ub_src_core_functions_scalar_list.o duckdb/ub_src_core_functions_scalar_map.o duckdb/ub_src_core_functions_scalar_math.o duckdb/ub_src_core_functions_scalar_operators.o duckdb/ub_src_core_functions_scalar_random.o duckdb/ub_src_core_functions_scalar_string.o duckdb/ub_src_core_functions_scalar_struct.o duckdb/ub_src_core_functions_scalar_union.o duckdb/ub_src_execution.o duckdb/ub_src_execution_expression_executor.o duckdb/ub_src_execution_index_art.o duckdb/ub_src_execution_nested_loop_join.o duckdb/ub_src_execution_operator_aggregate.o duckdb/ub_src_execution_operator_csv_scanner.o duckdb/ub_src_execution_operator_csv_scanner_sniffer.o duckdb/ub_src_execution_operator_filter.o duckdb/ub_src_execution_operator_helper.o duckdb/ub_src_execution_operator_join.o duckdb/ub_src_execution_operator_order.o duckdb/ub_src_execution_operator_persistent.o duckdb/ub_src_execution_operator_projection.o duckdb/ub_src_execution_operator_scan.o duckdb/ub_src_execution_operator_schema.o duckdb/ub_src_execution_operator_set.o duckdb/ub_src_execution_physical_plan.o duckdb/ub_src_function_aggregate_distributive.o duckdb/ub_src_function_aggregate.o duckdb/ub_src_function.o duckdb/ub_src_function_cast.o duckdb/ub_src_function_pragma.o duckdb/ub_src_function_scalar_compressed_materialization.o duckdb/ub_src_function_scalar.o duckdb/ub_src_function_scalar_generic.o duckdb/ub_src_function_scalar_list.o duckdb/ub_src_function_scalar_operators.o duckdb/ub_src_function_scalar_sequence.o duckdb/ub_src_function_scalar_string.o duckdb/ub_src_function_scalar_string_regexp.o duckdb/ub_src_function_scalar_struct.o duckdb/ub_src_function_scalar_system.o duckdb/ub_src_function_table_arrow.o duckdb/ub_src_function_table.o duckdb/ub_src_function_table_system.o duckdb/ub_src_function_table_version.o duckdb/ub_src_main.o duckdb/ub_src_main_capi.o duckdb/ub_src_main_capi_cast.o duckdb/ub_src_main_chunk_scan_state.o duckdb/ub_src_main_extension.o duckdb/ub_src_main_relation.o duckdb/ub_src_main_settings.o duckdb/ub_src_optimizer.o duckdb/ub_src_optimizer_compressed_materialization.o duckdb/ub_src_optimizer_join_order.o duckdb/ub_src_optimizer_matcher.o duckdb/ub_src_optimizer_pullup.o duckdb/ub_src_optimizer_pushdown.o duckdb/ub_src_optimizer_rule.o duckdb/ub_src_optimizer_statistics_expression.o duckdb/ub_src_optimizer_statistics_operator.o duckdb/ub_src_parallel.o duckdb/ub_src_parser.o duckdb/ub_src_parser_constraints.o duckdb/ub_src_parser_expression.o duckdb/ub_src_parser_parsed_data.o duckdb/ub_src_parser_query_node.o duckdb/ub_src_parser_statement.o duckdb/ub_src_parser_tableref.o duckdb/ub_src_parser_transform_constraint.o duckdb/ub_src_parser_transform_expression.o duckdb/ub_src_parser_transform_helpers.o duckdb/ub_src_parser_transform_statement.o duckdb/ub_src_parser_transform_tableref.o duckdb/ub_src_planner.o duckdb/ub_src_planner_binder_expression.o duckdb/ub_src_planner_binder_query_node.o duckdb/ub_src_planner_binder_statement.o duckdb/ub_src_planner_binder_tableref.o duckdb/ub_src_planner_expression.o duckdb/ub_src_planner_expression_binder.o duckdb/ub_src_planner_filter.o duckdb/ub_src_planner_operator.o duckdb/ub_src_planner_parsed_data.o duckdb/ub_src_planner_subquery.o duckdb/ub_src_storage.o duckdb/ub_src_storage_buffer.o duckdb/ub_src_storage_checkpoint.o duckdb/ub_src_storage_compression.o duckdb/ub_src_storage_compression_chimp.o duckdb/ub_src_storage_metadata.o duckdb/ub_src_storage_serialization.o duckdb/ub_src_storage_statistics.o duckdb/ub_src_storage_table.o duckdb/ub_src_transaction.o duckdb/src/verification/copied_statement_verifier.o duckdb/src/verification/deserialized_statement_verifier.o duckdb/src/verification/deserialized_statement_verifier_v2.o duckdb/src/verification/external_statement_verifier.o duckdb/src/verification/no_operator_caching_verifier.o duckdb/src/verification/parsed_statement_verifier.o duckdb/src/verification/prepared_statement_verifier.o duckdb/src/verification/statement_verifier.o duckdb/src/verification/unoptimized_statement_verifier.o duckdb/third_party/fmt/format.o duckdb/third_party/fsst/fsst_avx512.o duckdb/third_party/fsst/libfsst.o duckdb/third_party/miniz/miniz.o duckdb/third_party/re2/re2/bitstate.o duckdb/third_party/re2/re2/compile.o duckdb/third_party/re2/re2/dfa.o duckdb/third_party/re2/re2/filtered_re2.o duckdb/third_party/re2/re2/mimics_pcre.o duckdb/third_party/re2/re2/nfa.o duckdb/third_party/re2/re2/onepass.o duckdb/third_party/re2/re2/parse.o duckdb/third_party/re2/re2/perl_groups.o duckdb/third_party/re2/re2/prefilter.o duckdb/third_party/re2/re2/prefilter_tree.o duckdb/third_party/re2/re2/prog.o duckdb/third_party/re2/re2/re2.o duckdb/third_party/re2/re2/regexp.o duckdb/third_party/re2/re2/set.o duckdb/third_party/re2/re2/simplify.o duckdb/third_party/re2/re2/stringpiece.o duckdb/third_party/re2/re2/tostring.o duckdb/third_party/re2/re2/unicode_casefold.o duckdb/third_party/re2/re2/unicode_groups.o duckdb/third_party/re2/util/rune.o duckdb/third_party/re2/util/strutil.o duckdb/third_party/hyperloglog/hyperloglog.o duckdb/third_party/hyperloglog/sds.o duckdb/third_party/fastpforlib/bitpacking.o duckdb/third_party/utf8proc/utf8proc.o duckdb/third_party/utf8proc/utf8proc_wrapper.o duckdb/third_party/libpg_query/pg_functions.o duckdb/third_party/libpg_query/postgres_parser.o duckdb/third_party/libpg_query/src_backend_nodes_list.o duckdb/third_party/libpg_query/src_backend_nodes_makefuncs.o duckdb/third_party/libpg_query/src_backend_nodes_value.o duckdb/third_party/libpg_query/src_backend_parser_gram.o duckdb/third_party/libpg_query/src_backend_parser_parser.o duckdb/third_party/libpg_query/src_backend_parser_scan.o duckdb/third_party/libpg_query/src_backend_parser_scansup.o duckdb/third_party/libpg_query/src_common_keywords.o duckdb/third_party/mbedtls/library/asn1parse.o duckdb/third_party/mbedtls/library/base64.o duckdb/third_party/mbedtls/library/bignum.o duckdb/third_party/mbedtls/library/constant_time.o duckdb/third_party/mbedtls/library/md.o duckdb/third_party/mbedtls/library/oid.o duckdb/third_party/mbedtls/library/pem.o duckdb/third_party/mbedtls/library/pk.o duckdb/third_party/mbedtls/library/pk_wrap.o duckdb/third_party/mbedtls/library/pkparse.o duckdb/third_party/mbedtls/library/platform_util.o duckdb/third_party/mbedtls/library/rsa.o duckdb/third_party/mbedtls/library/rsa_alt_helpers.o duckdb/third_party/mbedtls/library/sha1.o duckdb/third_party/mbedtls/library/sha256.o duckdb/third_party/mbedtls/library/sha512.o duckdb/third_party/mbedtls/mbedtls_wrapper.o duckdb/extension/parquet/parquet_extension.o duckdb/extension/parquet/column_writer.o duckdb/extension/parquet/serialize_parquet.o duckdb/extension/parquet/parquet_reader.o duckdb/extension/parquet/parquet_timestamp.o duckdb/extension/parquet/parquet_writer.o duckdb/extension/parquet/column_reader.o duckdb/extension/parquet/parquet_statistics.o duckdb/extension/parquet/parquet_metadata.o duckdb/extension/parquet/zstd_file_system.o duckdb/third_party/parquet/parquet_constants.o duckdb/third_party/parquet/parquet_types.o duckdb/third_party/thrift/thrift/protocol/TProtocol.o duckdb/third_party/thrift/thrift/transport/TTransportException.o duckdb/third_party/thrift/thrift/transport/TBufferTransports.o duckdb/third_party/snappy/snappy.o duckdb/third_party/snappy/snappy-sinksource.o duckdb/third_party/zstd/decompress/zstd_ddict.o duckdb/third_party/zstd/decompress/huf_decompress.o duckdb/third_party/zstd/decompress/zstd_decompress.o duckdb/third_party/zstd/decompress/zstd_decompress_block.o duckdb/third_party/zstd/common/entropy_common.o duckdb/third_party/zstd/common/fse_decompress.o duckdb/third_party/zstd/common/zstd_common.o duckdb/third_party/zstd/common/error_private.o duckdb/third_party/zstd/common/xxhash.o duckdb/third_party/zstd/compress/fse_compress.o duckdb/third_party/zstd/compress/hist.o duckdb/third_party/zstd/compress/huf_compress.o duckdb/third_party/zstd/compress/zstd_compress.o duckdb/third_party/zstd/compress/zstd_compress_literals.o duckdb/third_party/zstd/compress/zstd_compress_sequences.o duckdb/third_party/zstd/compress/zstd_compress_superblock.o duckdb/third_party/zstd/compress/zstd_double_fast.o duckdb/third_party/zstd/compress/zstd_fast.o duckdb/third_party/zstd/compress/zstd_lazy.o duckdb/third_party/zstd/compress/zstd_ldm.o duckdb/third_party/zstd/compress/zstd_opt.o +SOURCES=duckdb/ub_src_catalog.o duckdb/ub_src_catalog_catalog_entry.o duckdb/ub_src_catalog_default.o duckdb/ub_src_common_adbc.o duckdb/ub_src_common_adbc_nanoarrow.o duckdb/ub_src_common.o duckdb/ub_src_common_arrow_appender.o duckdb/ub_src_common_arrow.o duckdb/ub_src_common_crypto.o duckdb/ub_src_common_enums.o duckdb/ub_src_common_operator.o duckdb/ub_src_common_progress_bar.o duckdb/ub_src_common_row_operations.o duckdb/ub_src_common_serializer.o duckdb/ub_src_common_sort.o duckdb/ub_src_common_types.o duckdb/ub_src_common_types_column.o duckdb/ub_src_common_types_row.o duckdb/ub_src_common_value_operations.o duckdb/src/common/vector_operations/boolean_operators.o duckdb/src/common/vector_operations/comparison_operators.o duckdb/src/common/vector_operations/generators.o duckdb/src/common/vector_operations/is_distinct_from.o duckdb/src/common/vector_operations/null_operations.o duckdb/src/common/vector_operations/numeric_inplace_operators.o duckdb/src/common/vector_operations/vector_cast.o duckdb/src/common/vector_operations/vector_copy.o duckdb/src/common/vector_operations/vector_hash.o duckdb/src/common/vector_operations/vector_storage.o duckdb/ub_src_core_functions_aggregate_algebraic.o duckdb/ub_src_core_functions_aggregate_distributive.o duckdb/ub_src_core_functions_aggregate_holistic.o duckdb/ub_src_core_functions_aggregate_nested.o duckdb/ub_src_core_functions_aggregate_regression.o duckdb/ub_src_core_functions.o duckdb/ub_src_core_functions_scalar_bit.o duckdb/ub_src_core_functions_scalar_blob.o duckdb/ub_src_core_functions_scalar_date.o duckdb/ub_src_core_functions_scalar_debug.o duckdb/ub_src_core_functions_scalar_enum.o duckdb/ub_src_core_functions_scalar_generic.o duckdb/ub_src_core_functions_scalar_list.o duckdb/ub_src_core_functions_scalar_map.o duckdb/ub_src_core_functions_scalar_math.o duckdb/ub_src_core_functions_scalar_operators.o duckdb/ub_src_core_functions_scalar_random.o duckdb/ub_src_core_functions_scalar_string.o duckdb/ub_src_core_functions_scalar_struct.o duckdb/ub_src_core_functions_scalar_union.o duckdb/ub_src_execution.o duckdb/ub_src_execution_expression_executor.o duckdb/ub_src_execution_index_art.o duckdb/ub_src_execution_index.o duckdb/ub_src_execution_nested_loop_join.o duckdb/ub_src_execution_operator_aggregate.o duckdb/ub_src_execution_operator_csv_scanner.o duckdb/ub_src_execution_operator_csv_scanner_sniffer.o duckdb/ub_src_execution_operator_filter.o duckdb/ub_src_execution_operator_helper.o duckdb/ub_src_execution_operator_join.o duckdb/ub_src_execution_operator_order.o duckdb/ub_src_execution_operator_persistent.o duckdb/ub_src_execution_operator_projection.o duckdb/ub_src_execution_operator_scan.o duckdb/ub_src_execution_operator_schema.o duckdb/ub_src_execution_operator_set.o duckdb/ub_src_execution_physical_plan.o duckdb/ub_src_function_aggregate_distributive.o duckdb/ub_src_function_aggregate.o duckdb/ub_src_function.o duckdb/ub_src_function_cast.o duckdb/ub_src_function_pragma.o duckdb/ub_src_function_scalar_compressed_materialization.o duckdb/ub_src_function_scalar.o duckdb/ub_src_function_scalar_generic.o duckdb/ub_src_function_scalar_list.o duckdb/ub_src_function_scalar_operators.o duckdb/ub_src_function_scalar_sequence.o duckdb/ub_src_function_scalar_string.o duckdb/ub_src_function_scalar_string_regexp.o duckdb/ub_src_function_scalar_struct.o duckdb/ub_src_function_scalar_system.o duckdb/ub_src_function_table_arrow.o duckdb/ub_src_function_table.o duckdb/ub_src_function_table_system.o duckdb/ub_src_function_table_version.o duckdb/ub_src_main.o duckdb/ub_src_main_capi.o duckdb/ub_src_main_capi_cast.o duckdb/ub_src_main_chunk_scan_state.o duckdb/ub_src_main_extension.o duckdb/ub_src_main_relation.o duckdb/ub_src_main_settings.o duckdb/ub_src_optimizer.o duckdb/ub_src_optimizer_compressed_materialization.o duckdb/ub_src_optimizer_join_order.o duckdb/ub_src_optimizer_matcher.o duckdb/ub_src_optimizer_pullup.o duckdb/ub_src_optimizer_pushdown.o duckdb/ub_src_optimizer_rule.o duckdb/ub_src_optimizer_statistics_expression.o duckdb/ub_src_optimizer_statistics_operator.o duckdb/ub_src_parallel.o duckdb/ub_src_parser.o duckdb/ub_src_parser_constraints.o duckdb/ub_src_parser_expression.o duckdb/ub_src_parser_parsed_data.o duckdb/ub_src_parser_query_node.o duckdb/ub_src_parser_statement.o duckdb/ub_src_parser_tableref.o duckdb/ub_src_parser_transform_constraint.o duckdb/ub_src_parser_transform_expression.o duckdb/ub_src_parser_transform_helpers.o duckdb/ub_src_parser_transform_statement.o duckdb/ub_src_parser_transform_tableref.o duckdb/ub_src_planner.o duckdb/ub_src_planner_binder_expression.o duckdb/ub_src_planner_binder_query_node.o duckdb/ub_src_planner_binder_statement.o duckdb/ub_src_planner_binder_tableref.o duckdb/ub_src_planner_expression.o duckdb/ub_src_planner_expression_binder.o duckdb/ub_src_planner_filter.o duckdb/ub_src_planner_operator.o duckdb/ub_src_planner_parsed_data.o duckdb/ub_src_planner_subquery.o duckdb/ub_src_storage.o duckdb/ub_src_storage_buffer.o duckdb/ub_src_storage_checkpoint.o duckdb/ub_src_storage_compression.o duckdb/ub_src_storage_compression_chimp.o duckdb/ub_src_storage_metadata.o duckdb/ub_src_storage_serialization.o duckdb/ub_src_storage_statistics.o duckdb/ub_src_storage_table.o duckdb/ub_src_transaction.o duckdb/src/verification/copied_statement_verifier.o duckdb/src/verification/deserialized_statement_verifier.o duckdb/src/verification/deserialized_statement_verifier_v2.o duckdb/src/verification/external_statement_verifier.o duckdb/src/verification/no_operator_caching_verifier.o duckdb/src/verification/parsed_statement_verifier.o duckdb/src/verification/prepared_statement_verifier.o duckdb/src/verification/statement_verifier.o duckdb/src/verification/unoptimized_statement_verifier.o duckdb/third_party/fmt/format.o duckdb/third_party/fsst/fsst_avx512.o duckdb/third_party/fsst/libfsst.o duckdb/third_party/miniz/miniz.o duckdb/third_party/re2/re2/bitstate.o duckdb/third_party/re2/re2/compile.o duckdb/third_party/re2/re2/dfa.o duckdb/third_party/re2/re2/filtered_re2.o duckdb/third_party/re2/re2/mimics_pcre.o duckdb/third_party/re2/re2/nfa.o duckdb/third_party/re2/re2/onepass.o duckdb/third_party/re2/re2/parse.o duckdb/third_party/re2/re2/perl_groups.o duckdb/third_party/re2/re2/prefilter.o duckdb/third_party/re2/re2/prefilter_tree.o duckdb/third_party/re2/re2/prog.o duckdb/third_party/re2/re2/re2.o duckdb/third_party/re2/re2/regexp.o duckdb/third_party/re2/re2/set.o duckdb/third_party/re2/re2/simplify.o duckdb/third_party/re2/re2/stringpiece.o duckdb/third_party/re2/re2/tostring.o duckdb/third_party/re2/re2/unicode_casefold.o duckdb/third_party/re2/re2/unicode_groups.o duckdb/third_party/re2/util/rune.o duckdb/third_party/re2/util/strutil.o duckdb/third_party/hyperloglog/hyperloglog.o duckdb/third_party/hyperloglog/sds.o duckdb/third_party/fastpforlib/bitpacking.o duckdb/third_party/utf8proc/utf8proc.o duckdb/third_party/utf8proc/utf8proc_wrapper.o duckdb/third_party/libpg_query/pg_functions.o duckdb/third_party/libpg_query/postgres_parser.o duckdb/third_party/libpg_query/src_backend_nodes_list.o duckdb/third_party/libpg_query/src_backend_nodes_makefuncs.o duckdb/third_party/libpg_query/src_backend_nodes_value.o duckdb/third_party/libpg_query/src_backend_parser_gram.o duckdb/third_party/libpg_query/src_backend_parser_parser.o duckdb/third_party/libpg_query/src_backend_parser_scan.o duckdb/third_party/libpg_query/src_backend_parser_scansup.o duckdb/third_party/libpg_query/src_common_keywords.o duckdb/third_party/mbedtls/library/asn1parse.o duckdb/third_party/mbedtls/library/base64.o duckdb/third_party/mbedtls/library/bignum.o duckdb/third_party/mbedtls/library/constant_time.o duckdb/third_party/mbedtls/library/md.o duckdb/third_party/mbedtls/library/oid.o duckdb/third_party/mbedtls/library/pem.o duckdb/third_party/mbedtls/library/pk.o duckdb/third_party/mbedtls/library/pk_wrap.o duckdb/third_party/mbedtls/library/pkparse.o duckdb/third_party/mbedtls/library/platform_util.o duckdb/third_party/mbedtls/library/rsa.o duckdb/third_party/mbedtls/library/rsa_alt_helpers.o duckdb/third_party/mbedtls/library/sha1.o duckdb/third_party/mbedtls/library/sha256.o duckdb/third_party/mbedtls/library/sha512.o duckdb/third_party/mbedtls/mbedtls_wrapper.o duckdb/extension/parquet/parquet_extension.o duckdb/extension/parquet/column_writer.o duckdb/extension/parquet/serialize_parquet.o duckdb/extension/parquet/parquet_reader.o duckdb/extension/parquet/parquet_timestamp.o duckdb/extension/parquet/parquet_writer.o duckdb/extension/parquet/column_reader.o duckdb/extension/parquet/parquet_statistics.o duckdb/extension/parquet/parquet_metadata.o duckdb/extension/parquet/zstd_file_system.o duckdb/third_party/parquet/parquet_constants.o duckdb/third_party/parquet/parquet_types.o duckdb/third_party/thrift/thrift/protocol/TProtocol.o duckdb/third_party/thrift/thrift/transport/TTransportException.o duckdb/third_party/thrift/thrift/transport/TBufferTransports.o duckdb/third_party/snappy/snappy.o duckdb/third_party/snappy/snappy-sinksource.o duckdb/third_party/zstd/decompress/zstd_ddict.o duckdb/third_party/zstd/decompress/huf_decompress.o duckdb/third_party/zstd/decompress/zstd_decompress.o duckdb/third_party/zstd/decompress/zstd_decompress_block.o duckdb/third_party/zstd/common/entropy_common.o duckdb/third_party/zstd/common/fse_decompress.o duckdb/third_party/zstd/common/zstd_common.o duckdb/third_party/zstd/common/error_private.o duckdb/third_party/zstd/common/xxhash.o duckdb/third_party/zstd/compress/fse_compress.o duckdb/third_party/zstd/compress/hist.o duckdb/third_party/zstd/compress/huf_compress.o duckdb/third_party/zstd/compress/zstd_compress.o duckdb/third_party/zstd/compress/zstd_compress_literals.o duckdb/third_party/zstd/compress/zstd_compress_sequences.o duckdb/third_party/zstd/compress/zstd_compress_superblock.o duckdb/third_party/zstd/compress/zstd_double_fast.o duckdb/third_party/zstd/compress/zstd_fast.o duckdb/third_party/zstd/compress/zstd_lazy.o duckdb/third_party/zstd/compress/zstd_ldm.o duckdb/third_party/zstd/compress/zstd_opt.o