From 27e015fb3bd696fb013872f15e065c559655795c Mon Sep 17 00:00:00 2001 From: DuckDB Labs GitHub Bot Date: Thu, 24 Jul 2025 06:02:36 +0000 Subject: [PATCH 1/9] Update vendored DuckDB sources to 9162313f76 --- .../core_functions/scalar/date/date_part.cpp | 20 - .../core_functions/scalar/string/printf.cpp | 16 + .../reader/variant/variant_binary_decoder.hpp | 26 +- .../variant/variant_shredded_conversion.hpp | 23 + .../include/reader/variant/variant_value.hpp | 54 + .../writer/templated_column_writer.hpp | 5 +- .../extension/parquet/parquet_metadata.cpp | 8 + .../extension/parquet/parquet_reader.cpp | 11 +- .../extension/parquet/parquet_writer.cpp | 7 +- .../reader/variant/variant_binary_decoder.cpp | 132 +- .../variant/variant_shredded_conversion.cpp | 559 + .../parquet/reader/variant/variant_value.cpp | 85 + .../parquet/reader/variant_column_reader.cpp | 59 +- src/duckdb/src/catalog/catalog.cpp | 10 + .../catalog_entry/duck_table_entry.cpp | 3 +- src/duckdb/src/common/adbc/adbc.cpp | 287 +- .../src/common/arrow/appender/append_data.cpp | 29 + .../src/common/arrow/appender/bool_data.cpp | 13 +- .../arrow/appender/fixed_size_list_data.cpp | 2 +- .../src/common/arrow/appender/struct_data.cpp | 2 +- .../src/common/arrow/appender/union_data.cpp | 1 - .../src/common/arrow/arrow_converter.cpp | 2 + src/duckdb/src/common/enum_util.cpp | 26 +- .../src/common/exception_format_value.cpp | 7 + src/duckdb/src/common/local_file_system.cpp | 2 +- src/duckdb/src/common/types/vector.cpp | 9 +- .../comparison_operations.cpp | 27 - .../vector_operations/is_distinct_from.cpp | 55 - src/duckdb/src/execution/index/art/art.cpp | 38 +- .../src/execution/index/art/iterator.cpp | 6 +- src/duckdb/src/execution/index/art/leaf.cpp | 13 +- .../operator/join/physical_iejoin.cpp | 4 +- .../join/physical_piecewise_merge_join.cpp | 4 +- .../operator/join/physical_range_join.cpp | 15 +- .../persistent/physical_merge_into.cpp | 115 +- .../operator/schema/physical_attach.cpp | 5 +- .../physical_plan/plan_merge_into.cpp | 33 +- src/duckdb/src/function/function_list.cpp | 1 + .../src/function/pragma/pragma_queries.cpp | 23 +- .../compress_integral.cpp | 5 + .../compress_string.cpp | 18 +- src/duckdb/src/function/table/arrow.cpp | 30 +- .../table/arrow/arrow_array_scan_state.cpp | 5 +- .../table/arrow/arrow_duck_schema.cpp | 26 +- .../src/function/table/arrow_conversion.cpp | 342 +- src/duckdb/src/function/table/table_scan.cpp | 50 +- .../function/table/version/pragma_version.cpp | 6 +- src/duckdb/src/include/duckdb.h | 169 +- .../src/include/duckdb/catalog/catalog.hpp | 2 + .../src/include/duckdb/common/adbc/adbc.hpp | 69 +- .../common/arrow/appender/append_data.hpp | 86 +- .../common/arrow/appender/enum_data.hpp | 2 +- .../common/arrow/appender/list_data.hpp | 2 +- .../common/arrow/appender/list_view_data.hpp | 2 +- .../duckdb/common/arrow/appender/map_data.hpp | 2 +- .../common/arrow/appender/scalar_data.hpp | 2 +- .../common/arrow/appender/varchar_data.hpp | 12 +- .../duckdb/common/arrow/arrow_wrapper.hpp | 10 + .../src/include/duckdb/common/enum_util.hpp | 8 + .../duckdb/common/enums/checkpoint_type.hpp | 4 +- .../duckdb/common/exception_format_value.hpp | 5 +- .../common/multi_file/multi_file_function.hpp | 6 +- .../duckdb/execution/index/art/art.hpp | 10 +- .../duckdb/execution/index/art/iterator.hpp | 2 +- .../duckdb/execution/index/art/leaf.hpp | 2 +- .../operator/join/physical_range_join.hpp | 2 +- .../persistent/physical_merge_into.hpp | 5 +- .../include/duckdb/function/copy_function.hpp | 10 +- .../function/pragma/pragma_functions.hpp | 2 +- .../compressed_materialization_functions.hpp | 10 + .../include/duckdb/function/table/arrow.hpp | 35 +- .../table/arrow/arrow_duck_schema.hpp | 10 +- .../function/table/arrow/arrow_type_info.hpp | 2 + .../include/duckdb/main/attached_database.hpp | 5 +- .../duckdb/main/capi/capi_internal.hpp | 9 + .../duckdb/main/capi/extension_api.hpp | 40 + .../include/duckdb/main/database_manager.hpp | 4 +- .../parser/parsed_data/alter_table_info.hpp | 26 +- .../duckdb/parser/parsed_data/attach_info.hpp | 4 - .../parser/statement/merge_into_statement.hpp | 3 + .../duckdb/parser/tableref/showref.hpp | 6 +- .../include/duckdb/planner/bind_context.hpp | 3 + .../src/include/duckdb/planner/binder.hpp | 3 +- .../planner/operator/logical_merge_into.hpp | 3 + .../storage/checkpoint/row_group_writer.hpp | 9 +- .../duckdb/storage/checkpoint_manager.hpp | 5 +- .../chimp/algorithm/packed_data.hpp | 3 +- .../storage/compression/patas/patas_scan.hpp | 3 +- .../duckdb/storage/partial_block_manager.hpp | 6 +- .../duckdb/storage/storage_extension.hpp | 8 +- .../duckdb/storage/storage_manager.hpp | 31 +- .../duckdb/storage/storage_options.hpp | 7 + .../storage/table/column_checkpoint_state.hpp | 2 +- .../duckdb/storage/table/column_data.hpp | 1 + .../table/column_data_checkpointer.hpp | 7 +- .../duckdb/storage/table/column_segment.hpp | 5 +- .../storage/table/in_memory_checkpoint.hpp | 90 + src/duckdb/src/include/duckdb_extension.h | 64 +- src/duckdb/src/main/attached_database.cpp | 43 +- src/duckdb/src/main/capi/arrow-c.cpp | 142 + src/duckdb/src/main/capi/duckdb-c.cpp | 19 + src/duckdb/src/main/capi/duckdb_value-c.cpp | 2 +- src/duckdb/src/main/capi/expression-c.cpp | 57 + src/duckdb/src/main/capi/result-c.cpp | 12 + .../src/main/capi/scalar_function-c.cpp | 19 + src/duckdb/src/main/database_manager.cpp | 68 +- .../src/main/extension/extension_helper.cpp | 17 - .../join_order/cardinality_estimator.cpp | 20 +- .../optimizer/join_order/plan_enumerator.cpp | 2 +- src/duckdb/src/optimizer/optimizer.cpp | 1 + src/duckdb/src/parallel/task_scheduler.cpp | 2 +- .../src/parser/parsed_data/attach_info.cpp | 45 - .../parser/statement/merge_into_statement.cpp | 17 + src/duckdb/src/parser/tableref/showref.cpp | 14 + .../statement/transform_merge_into.cpp | 3 + .../transform/statement/transform_show.cpp | 26 +- src/duckdb/src/planner/bind_context.cpp | 10 +- src/duckdb/src/planner/binder.cpp | 8 +- .../planner/binder/statement/bind_copy.cpp | 4 +- .../binder/statement/bind_create_table.cpp | 16 +- .../planner/binder/statement/bind_delete.cpp | 5 +- .../planner/binder/statement/bind_insert.cpp | 4 +- .../binder/statement/bind_merge_into.cpp | 21 +- .../planner/binder/statement/bind_update.cpp | 4 +- .../planner/binder/tableref/bind_showref.cpp | 30 + .../planner/operator/logical_merge_into.cpp | 16 +- .../storage/checkpoint/row_group_writer.cpp | 10 +- src/duckdb/src/storage/checkpoint_manager.cpp | 1 + .../compression/dict_fsst/decompression.cpp | 1 - .../compression/fixed_size_uncompressed.cpp | 15 +- src/duckdb/src/storage/data_table.cpp | 3 +- .../storage/open_file_storage_extension.cpp | 6 +- .../src/storage/partial_block_manager.cpp | 15 + .../serialize_logical_operator.cpp | 2 + .../serialization/serialize_tableref.cpp | 4 + src/duckdb/src/storage/storage_manager.cpp | 100 +- .../storage/table/column_checkpoint_state.cpp | 32 +- src/duckdb/src/storage/table/column_data.cpp | 11 +- .../table/column_data_checkpointer.cpp | 8 +- .../src/storage/table/column_segment.cpp | 42 +- .../storage/table/in_memory_checkpoint.cpp | 129 + src/duckdb/src/storage/table/row_group.cpp | 40 +- .../storage/table/row_group_collection.cpp | 18 +- .../storage/table/standard_column_data.cpp | 11 +- .../src/transaction/duck_transaction.cpp | 6 +- .../transaction/duck_transaction_manager.cpp | 23 +- .../src/transaction/meta_transaction.cpp | 15 +- src/duckdb/third_party/fmt/include/fmt/core.h | 18 +- .../third_party/fmt/include/fmt/format-inl.h | 6 +- .../third_party/fmt/include/fmt/format.h | 48 +- .../third_party/fmt/include/fmt/printf.h | 70 +- .../libpg_query/include/nodes/parsenodes.hpp | 1 + .../libpg_query/src_backend_parser_gram.cpp | 27171 ++++++++-------- .../ub_extension_parquet_reader_variant.cpp | 4 + src/duckdb/ub_src_common_arrow_appender.cpp | 2 + src/duckdb/ub_src_main_capi.cpp | 2 + src/duckdb/ub_src_storage_table.cpp | 2 + 157 files changed, 16823 insertions(+), 14657 deletions(-) create mode 100644 src/duckdb/extension/parquet/include/reader/variant/variant_shredded_conversion.hpp create mode 100644 src/duckdb/extension/parquet/include/reader/variant/variant_value.hpp create mode 100644 src/duckdb/extension/parquet/reader/variant/variant_shredded_conversion.cpp create mode 100644 src/duckdb/extension/parquet/reader/variant/variant_value.cpp create mode 100644 src/duckdb/src/common/arrow/appender/append_data.cpp create mode 100644 src/duckdb/src/include/duckdb/storage/table/in_memory_checkpoint.hpp create mode 100644 src/duckdb/src/main/capi/expression-c.cpp create mode 100644 src/duckdb/src/storage/table/in_memory_checkpoint.cpp diff --git a/src/duckdb/extension/core_functions/scalar/date/date_part.cpp b/src/duckdb/extension/core_functions/scalar/date/date_part.cpp index 1484155c8..7ced59dcb 100644 --- a/src/duckdb/extension/core_functions/scalar/date/date_part.cpp +++ b/src/duckdb/extension/core_functions/scalar/date/date_part.cpp @@ -1438,26 +1438,6 @@ double DatePart::JulianDayOperator::Operation(date_t input) { return double(Date::ExtractJulianDay(input)); } -template <> -double DatePart::JulianDayOperator::Operation(interval_t input) { - throw NotImplementedException("interval units \"julian\" not recognized"); -} - -template <> -double DatePart::JulianDayOperator::Operation(dtime_t input) { - throw NotImplementedException("\"time\" units \"julian\" not recognized"); -} - -template <> -double DatePart::JulianDayOperator::Operation(dtime_ns_t input) { - return JulianDayOperator::Operation(input.time()); -} - -template <> -double DatePart::JulianDayOperator::Operation(dtime_tz_t input) { - return JulianDayOperator::Operation(input.time()); -} - template <> void DatePart::StructOperator::Operation(bigint_vec &bigint_values, double_vec &double_values, const dtime_t &input, const idx_t idx, const part_mask_t mask) { diff --git a/src/duckdb/extension/core_functions/scalar/string/printf.cpp b/src/duckdb/extension/core_functions/scalar/string/printf.cpp index 1db25b0df..1ec8ae2cd 100644 --- a/src/duckdb/extension/core_functions/scalar/string/printf.cpp +++ b/src/duckdb/extension/core_functions/scalar/string/printf.cpp @@ -41,6 +41,12 @@ unique_ptr BindPrintfFunction(ClientContext &context, ScalarFuncti case LogicalTypeId::UBIGINT: bound_function.arguments.emplace_back(LogicalType::UBIGINT); break; + case LogicalTypeId::HUGEINT: + bound_function.arguments.emplace_back(LogicalType::HUGEINT); + break; + case LogicalTypeId::UHUGEINT: + bound_function.arguments.emplace_back(LogicalType::UHUGEINT); + break; case LogicalTypeId::FLOAT: case LogicalTypeId::DOUBLE: bound_function.arguments.emplace_back(LogicalType::DOUBLE); @@ -146,6 +152,16 @@ static void PrintfFunction(DataChunk &args, ExpressionState &state, Vector &resu format_args.emplace_back(duckdb_fmt::internal::make_arg(arg_data[arg_idx])); break; } + case LogicalTypeId::HUGEINT: { + auto arg_data = FlatVector::GetData(col); + format_args.emplace_back(duckdb_fmt::internal::make_arg(arg_data[arg_idx])); + break; + } + case LogicalTypeId::UHUGEINT: { + auto arg_data = FlatVector::GetData(col); + format_args.emplace_back(duckdb_fmt::internal::make_arg(arg_data[arg_idx])); + break; + } case LogicalTypeId::DOUBLE: { auto arg_data = FlatVector::GetData(col); format_args.emplace_back(duckdb_fmt::internal::make_arg(arg_data[arg_idx])); diff --git a/src/duckdb/extension/parquet/include/reader/variant/variant_binary_decoder.hpp b/src/duckdb/extension/parquet/include/reader/variant/variant_binary_decoder.hpp index 796d7696c..a7c717709 100644 --- a/src/duckdb/extension/parquet/include/reader/variant/variant_binary_decoder.hpp +++ b/src/duckdb/extension/parquet/include/reader/variant/variant_binary_decoder.hpp @@ -1,7 +1,8 @@ #pragma once #include "duckdb/common/types/string_type.hpp" -#include "yyjson.hpp" +#include "duckdb/common/types/value.hpp" +#include "reader/variant/variant_value.hpp" using namespace duckdb_yyjson; @@ -130,23 +131,20 @@ struct VariantDecodeResult { class VariantBinaryDecoder { public: - explicit VariantBinaryDecoder(ClientContext &context); + VariantBinaryDecoder() = delete; public: - yyjson_mut_val *Decode(yyjson_mut_doc *doc, const VariantMetadata &metadata, const_data_ptr_t data); + static VariantValue Decode(const VariantMetadata &metadata, const_data_ptr_t data); public: - yyjson_mut_val *PrimitiveTypeDecode(yyjson_mut_doc *doc, const VariantMetadata &metadata, - const VariantValueMetadata &value_metadata, const_data_ptr_t data); - yyjson_mut_val *ShortStringDecode(yyjson_mut_doc *doc, const VariantMetadata &metadata, - const VariantValueMetadata &value_metadata, const_data_ptr_t data); - yyjson_mut_val *ObjectDecode(yyjson_mut_doc *doc, const VariantMetadata &metadata, - const VariantValueMetadata &value_metadata, const_data_ptr_t data); - yyjson_mut_val *ArrayDecode(yyjson_mut_doc *doc, const VariantMetadata &metadata, - const VariantValueMetadata &value_metadata, const_data_ptr_t data); - -public: - ClientContext &context; + static VariantValue PrimitiveTypeDecode(const VariantMetadata &metadata, const VariantValueMetadata &value_metadata, + const_data_ptr_t data); + static VariantValue ShortStringDecode(const VariantMetadata &metadata, const VariantValueMetadata &value_metadata, + const_data_ptr_t data); + static VariantValue ObjectDecode(const VariantMetadata &metadata, const VariantValueMetadata &value_metadata, + const_data_ptr_t data); + static VariantValue ArrayDecode(const VariantMetadata &metadata, const VariantValueMetadata &value_metadata, + const_data_ptr_t data); }; } // namespace duckdb diff --git a/src/duckdb/extension/parquet/include/reader/variant/variant_shredded_conversion.hpp b/src/duckdb/extension/parquet/include/reader/variant/variant_shredded_conversion.hpp new file mode 100644 index 000000000..27ece7d70 --- /dev/null +++ b/src/duckdb/extension/parquet/include/reader/variant/variant_shredded_conversion.hpp @@ -0,0 +1,23 @@ +#pragma once + +#include "reader/variant/variant_value.hpp" +#include "reader/variant/variant_binary_decoder.hpp" + +namespace duckdb { + +class VariantShreddedConversion { +public: + VariantShreddedConversion() = delete; + +public: + static vector Convert(Vector &metadata, Vector &group, idx_t offset, idx_t length, idx_t total_size, + bool is_field = false); + static vector ConvertShreddedLeaf(Vector &metadata, Vector &value, Vector &typed_value, idx_t offset, + idx_t length, idx_t total_size); + static vector ConvertShreddedArray(Vector &metadata, Vector &value, Vector &typed_value, idx_t offset, + idx_t length, idx_t total_size); + static vector ConvertShreddedObject(Vector &metadata, Vector &value, Vector &typed_value, + idx_t offset, idx_t length, idx_t total_size); +}; + +} // namespace duckdb diff --git a/src/duckdb/extension/parquet/include/reader/variant/variant_value.hpp b/src/duckdb/extension/parquet/include/reader/variant/variant_value.hpp new file mode 100644 index 000000000..a4c38ede7 --- /dev/null +++ b/src/duckdb/extension/parquet/include/reader/variant/variant_value.hpp @@ -0,0 +1,54 @@ +#pragma once + +#include "duckdb/common/map.hpp" +#include "duckdb/common/vector.hpp" +#include "duckdb/common/types/value.hpp" + +#include "yyjson.hpp" + +using namespace duckdb_yyjson; + +namespace duckdb { + +enum class VariantValueType : uint8_t { PRIMITIVE, OBJECT, ARRAY, MISSING }; + +struct VariantValue { +public: + VariantValue() : value_type(VariantValueType::MISSING) { + } + explicit VariantValue(VariantValueType type) : value_type(type) { + } + explicit VariantValue(Value &&val) : value_type(VariantValueType::PRIMITIVE), primitive_value(std::move(val)) { + } + // Delete copy constructor and copy assignment operator + VariantValue(const VariantValue &) = delete; + VariantValue &operator=(const VariantValue &) = delete; + + // Default move constructor and move assignment operator + VariantValue(VariantValue &&) noexcept = default; + VariantValue &operator=(VariantValue &&) noexcept = default; + +public: + bool IsNull() const { + return value_type == VariantValueType::PRIMITIVE && primitive_value.IsNull(); + } + bool IsMissing() const { + return value_type == VariantValueType::MISSING; + } + +public: + void AddChild(const string &key, VariantValue &&val); + void AddItem(VariantValue &&val); + +public: + yyjson_mut_val *ToJSON(ClientContext &context, yyjson_mut_doc *doc) const; + +public: + VariantValueType value_type; + //! FIXME: how can we get a deterministic child order for a partially shredded object? + map object_children; + vector array_items; + Value primitive_value; +}; + +} // namespace duckdb diff --git a/src/duckdb/extension/parquet/include/writer/templated_column_writer.hpp b/src/duckdb/extension/parquet/include/writer/templated_column_writer.hpp index 392c2d815..522ded3c9 100644 --- a/src/duckdb/extension/parquet/include/writer/templated_column_writer.hpp +++ b/src/duckdb/extension/parquet/include/writer/templated_column_writer.hpp @@ -20,8 +20,9 @@ namespace duckdb { template static void TemplatedWritePlain(Vector &col, ColumnWriterStatistics *stats, const idx_t chunk_start, const idx_t chunk_end, const ValidityMask &mask, WriteStream &ser) { - static constexpr bool COPY_DIRECTLY_FROM_VECTOR = - ALL_VALID && std::is_same::value && std::is_arithmetic::value; + static constexpr bool COPY_DIRECTLY_FROM_VECTOR = ALL_VALID && std::is_same::value && + std::is_arithmetic::value && + std::is_same::value; const auto *const ptr = FlatVector::GetData(col); diff --git a/src/duckdb/extension/parquet/parquet_metadata.cpp b/src/duckdb/extension/parquet/parquet_metadata.cpp index 0e4dd5ab7..59eaf2dac 100644 --- a/src/duckdb/extension/parquet/parquet_metadata.cpp +++ b/src/duckdb/extension/parquet/parquet_metadata.cpp @@ -200,6 +200,9 @@ void ParquetMetaDataOperatorData::BindMetaData(vector &return_types names.emplace_back("max_is_exact"); return_types.emplace_back(LogicalType::BOOLEAN); + + names.emplace_back("row_group_compressed_bytes"); + return_types.emplace_back(LogicalType::BIGINT); } static Value ConvertParquetStats(const LogicalType &type, const ParquetColumnSchema &schema_ele, bool stats_is_set, @@ -350,6 +353,11 @@ void ParquetMetaDataOperatorData::LoadRowGroupMetadata(ClientContext &context, c current_chunk.SetValue(27, count, ParquetElementBoolean(stats.is_max_value_exact, stats.__isset.is_max_value_exact)); + // row_group_compressed_bytes + current_chunk.SetValue( + 28, count, + ParquetElementBigint(row_group.__isset.total_compressed_size, row_group.__isset.total_compressed_size)); + count++; if (count >= STANDARD_VECTOR_SIZE) { current_chunk.SetCardinality(count); diff --git a/src/duckdb/extension/parquet/parquet_reader.cpp b/src/duckdb/extension/parquet/parquet_reader.cpp index ca7f88588..d42bd9e94 100644 --- a/src/duckdb/extension/parquet/parquet_reader.cpp +++ b/src/duckdb/extension/parquet/parquet_reader.cpp @@ -540,7 +540,6 @@ static bool IsVariantType(const SchemaElement &root, const vector 0) { // check if the parent node of this is a map auto &p_ele = file_meta_data->schema[this_idx - 1]; @@ -986,8 +989,8 @@ const ParquetRowGroup &ParquetReader::GetGroup(ParquetReaderScanState &state) { } uint64_t ParquetReader::GetGroupCompressedSize(ParquetReaderScanState &state) { - auto &group = GetGroup(state); - auto total_compressed_size = group.total_compressed_size; + const auto &group = GetGroup(state); + int64_t total_compressed_size = group.__isset.total_compressed_size ? group.total_compressed_size : 0; idx_t calc_compressed_size = 0; diff --git a/src/duckdb/extension/parquet/parquet_writer.cpp b/src/duckdb/extension/parquet/parquet_writer.cpp index 3771e2c59..41f186822 100644 --- a/src/duckdb/extension/parquet/parquet_writer.cpp +++ b/src/duckdb/extension/parquet/parquet_writer.cpp @@ -543,12 +543,15 @@ void ParquetWriter::FlushRowGroup(PreparedRowGroup &prepared) { // let's make sure all offsets are ay-okay ValidateColumnOffsets(file_name, writer->GetTotalWritten(), row_group); - // append the row group to the file meta data + row_group.total_compressed_size = NumericCast(writer->GetTotalWritten()) - row_group.file_offset; + row_group.__isset.total_compressed_size = true; + + // append the row group to the file metadata file_meta_data.row_groups.push_back(row_group); file_meta_data.num_rows += row_group.num_rows; total_written = writer->GetTotalWritten(); - num_row_groups++; + ++num_row_groups; } void ParquetWriter::Flush(ColumnDataCollection &buffer) { diff --git a/src/duckdb/extension/parquet/reader/variant/variant_binary_decoder.cpp b/src/duckdb/extension/parquet/reader/variant/variant_binary_decoder.cpp index 74b6057aa..eacff5501 100644 --- a/src/duckdb/extension/parquet/reader/variant/variant_binary_decoder.cpp +++ b/src/duckdb/extension/parquet/reader/variant/variant_binary_decoder.cpp @@ -114,9 +114,6 @@ VariantValueMetadata VariantValueMetadata::FromHeaderByte(uint8_t byte) { return result; } -VariantBinaryDecoder::VariantBinaryDecoder(ClientContext &context) : context(context) { -} - template static T DecodeDecimal(const_data_ptr_t data, uint8_t &scale, uint8_t &width) { scale = Load(data); @@ -143,39 +140,42 @@ hugeint_t DecodeDecimal(const_data_ptr_t data, uint8_t &scale, uint8_t &width) { return result; } -yyjson_mut_val *VariantBinaryDecoder::PrimitiveTypeDecode(yyjson_mut_doc *doc, const VariantMetadata &metadata, - const VariantValueMetadata &value_metadata, - const_data_ptr_t data) { +VariantValue VariantBinaryDecoder::PrimitiveTypeDecode(const VariantMetadata &metadata, + const VariantValueMetadata &value_metadata, + const_data_ptr_t data) { switch (value_metadata.primitive_type) { case VariantPrimitiveType::NULL_TYPE: { - return yyjson_mut_null(doc); + return VariantValue(Value()); } case VariantPrimitiveType::BOOLEAN_TRUE: { - return yyjson_mut_true(doc); + return VariantValue(Value::BOOLEAN(true)); } case VariantPrimitiveType::BOOLEAN_FALSE: { - return yyjson_mut_false(doc); + return VariantValue(Value::BOOLEAN(false)); } case VariantPrimitiveType::INT8: { auto value = Load(data); - return yyjson_mut_int(doc, value); + return VariantValue(Value::TINYINT(value)); } case VariantPrimitiveType::INT16: { auto value = Load(data); - return yyjson_mut_int(doc, value); + return VariantValue(Value::SMALLINT(value)); } case VariantPrimitiveType::INT32: { auto value = Load(data); - return yyjson_mut_int(doc, value); + return VariantValue(Value::INTEGER(value)); } case VariantPrimitiveType::INT64: { auto value = Load(data); - return yyjson_mut_int(doc, value); + return VariantValue(Value::BIGINT(value)); } case VariantPrimitiveType::DOUBLE: { - double value; - memcpy(&value, data, sizeof(double)); - return yyjson_mut_real(doc, value); + double value = Load(data); + return VariantValue(Value::DOUBLE(value)); + } + case VariantPrimitiveType::FLOAT: { + float value = Load(data); + return VariantValue(Value::FLOAT(value)); } case VariantPrimitiveType::DECIMAL4: { uint8_t scale; @@ -183,7 +183,7 @@ yyjson_mut_val *VariantBinaryDecoder::PrimitiveTypeDecode(yyjson_mut_doc *doc, c auto value = DecodeDecimal(data, scale, width); auto value_str = Decimal::ToString(value, width, scale); - return yyjson_mut_strcpy(doc, value_str.c_str()); + return VariantValue(Value(value_str)); } case VariantPrimitiveType::DECIMAL8: { uint8_t scale; @@ -191,7 +191,7 @@ yyjson_mut_val *VariantBinaryDecoder::PrimitiveTypeDecode(yyjson_mut_doc *doc, c auto value = DecodeDecimal(data, scale, width); auto value_str = Decimal::ToString(value, width, scale); - return yyjson_mut_strcpy(doc, value_str.c_str()); + return VariantValue(Value(value_str)); } case VariantPrimitiveType::DECIMAL16: { uint8_t scale; @@ -199,21 +199,17 @@ yyjson_mut_val *VariantBinaryDecoder::PrimitiveTypeDecode(yyjson_mut_doc *doc, c auto value = DecodeDecimal(data, scale, width); auto value_str = Decimal::ToString(value, width, scale); - return yyjson_mut_strcpy(doc, value_str.c_str()); + return VariantValue(Value(value_str)); } case VariantPrimitiveType::DATE: { date_t value; value.days = Load(data); - auto value_str = Date::ToString(value); - return yyjson_mut_strcpy(doc, value_str.c_str()); + return VariantValue(Value::DATE(value)); } case VariantPrimitiveType::TIMESTAMP_MICROS: { - timestamp_tz_t micros_tz_ts; - micros_tz_ts.value = Load(data); - - auto value = Value::TIMESTAMPTZ(micros_tz_ts); - auto value_str = value.CastAs(context, LogicalType::VARCHAR).GetValue(); - return yyjson_mut_strcpy(doc, value_str.c_str()); + timestamp_tz_t micros_ts_tz; + micros_ts_tz.value = Load(data); + return VariantValue(Value::TIMESTAMPTZ(micros_ts_tz)); } case VariantPrimitiveType::TIMESTAMP_NTZ_MICROS: { timestamp_t micros_ts; @@ -221,12 +217,7 @@ yyjson_mut_val *VariantBinaryDecoder::PrimitiveTypeDecode(yyjson_mut_doc *doc, c auto value = Value::TIMESTAMP(micros_ts); auto value_str = value.ToString(); - return yyjson_mut_strcpy(doc, value_str.c_str()); - } - case VariantPrimitiveType::FLOAT: { - float value; - memcpy(&value, data, sizeof(float)); - return yyjson_mut_real(doc, value); + return VariantValue(Value(value_str)); } case VariantPrimitiveType::BINARY: { //! Follow the JSON serialization guide by converting BINARY to Base64: @@ -234,7 +225,7 @@ yyjson_mut_val *VariantBinaryDecoder::PrimitiveTypeDecode(yyjson_mut_doc *doc, c auto size = Load(data); auto string_data = reinterpret_cast(data + sizeof(uint32_t)); auto base64_string = Blob::ToBase64(string_t(string_data, size)); - return yyjson_mut_strncpy(doc, base64_string.c_str(), base64_string.size()); + return VariantValue(Value(base64_string)); } case VariantPrimitiveType::STRING: { auto size = Load(data); @@ -242,36 +233,20 @@ yyjson_mut_val *VariantBinaryDecoder::PrimitiveTypeDecode(yyjson_mut_doc *doc, c if (!Utf8Proc::IsValid(string_data, size)) { throw InternalException("Can't decode Variant short-string, string isn't valid UTF8"); } - return yyjson_mut_strncpy(doc, string_data, size); + return VariantValue(Value(string(string_data, size))); } case VariantPrimitiveType::TIME_NTZ_MICROS: { dtime_t micros_time; micros_time.micros = Load(data); - auto value_str = Time::ToString(micros_time); - return yyjson_mut_strcpy(doc, value_str.c_str()); + return VariantValue(Value::TIME(micros_time)); } case VariantPrimitiveType::TIMESTAMP_NANOS: { timestamp_ns_t nanos_ts; nanos_ts.value = Load(data); - //! Convert the nanos timestamp to a micros timestamp - date_t out_date; - dtime_t out_time; - int32_t out_nanos; - Timestamp::Convert(nanos_ts, out_date, out_time, out_nanos); - auto micros_ts = Timestamp::FromDatetime(out_date, out_time); - - //! Turn the micros timestamp into a micros_tz timestamp and serialize it - timestamp_tz_t micros_tz_ts(micros_ts.value); - auto value = Value::TIMESTAMPTZ(micros_tz_ts); - auto value_str = value.CastAs(context, LogicalType::VARCHAR).GetValue(); - - if (StringUtil::Contains(value_str, "+")) { - //! Don't attempt this for NaN/Inf timestamps - auto parts = StringUtil::Split(value_str, '+'); - value_str = StringUtil::Format("%s%s+%s", parts[0], to_string(out_nanos), parts[1]); - } - return yyjson_mut_strcpy(doc, value_str.c_str()); + //! Convert the nanos timestamp to a micros timestamp (not lossless) + auto micros_ts = Timestamp::FromEpochNanoSeconds(nanos_ts.value); + return VariantValue(Value::TIMESTAMPTZ(timestamp_tz_t(micros_ts))); } case VariantPrimitiveType::TIMESTAMP_NTZ_NANOS: { timestamp_ns_t nanos_ts; @@ -279,12 +254,12 @@ yyjson_mut_val *VariantBinaryDecoder::PrimitiveTypeDecode(yyjson_mut_doc *doc, c auto value = Value::TIMESTAMPNS(nanos_ts); auto value_str = value.ToString(); - return yyjson_mut_strcpy(doc, value_str.c_str()); + return VariantValue(Value(value_str)); } case VariantPrimitiveType::UUID: { auto uuid_value = UUIDValueConversion::ReadParquetUUID(data); auto value_str = UUID::ToString(uuid_value); - return yyjson_mut_strcpy(doc, value_str.c_str()); + return VariantValue(Value(value_str)); } default: throw NotImplementedException("Variant PrimitiveTypeDecode not implemented for type (%d)", @@ -292,20 +267,20 @@ yyjson_mut_val *VariantBinaryDecoder::PrimitiveTypeDecode(yyjson_mut_doc *doc, c } } -yyjson_mut_val *VariantBinaryDecoder::ShortStringDecode(yyjson_mut_doc *doc, const VariantMetadata &metadata, - const VariantValueMetadata &value_metadata, - const_data_ptr_t data) { +VariantValue VariantBinaryDecoder::ShortStringDecode(const VariantMetadata &metadata, + const VariantValueMetadata &value_metadata, + const_data_ptr_t data) { D_ASSERT(value_metadata.string_size < 64); auto string_data = reinterpret_cast(data); if (!Utf8Proc::IsValid(string_data, value_metadata.string_size)) { throw InternalException("Can't decode Variant short-string, string isn't valid UTF8"); } - return yyjson_mut_strncpy(doc, string_data, value_metadata.string_size); + return VariantValue(Value(string(string_data, value_metadata.string_size))); } -yyjson_mut_val *VariantBinaryDecoder::ObjectDecode(yyjson_mut_doc *doc, const VariantMetadata &metadata, - const VariantValueMetadata &value_metadata, const_data_ptr_t data) { - auto obj = yyjson_mut_obj(doc); +VariantValue VariantBinaryDecoder::ObjectDecode(const VariantMetadata &metadata, + const VariantValueMetadata &value_metadata, const_data_ptr_t data) { + VariantValue ret(VariantValueType::OBJECT); auto field_offset_size = value_metadata.field_offset_size; auto field_id_size = value_metadata.field_id_size; @@ -329,17 +304,18 @@ yyjson_mut_val *VariantBinaryDecoder::ObjectDecode(yyjson_mut_doc *doc, const Va auto field_id = ReadVariableLengthLittleEndian(field_id_size, field_ids); auto next_offset = ReadVariableLengthLittleEndian(field_offset_size, field_offsets); - auto value = Decode(doc, metadata, values + last_offset); + auto value = Decode(metadata, values + last_offset); auto &key = metadata.strings[field_id]; - yyjson_mut_obj_add_val(doc, obj, key.c_str(), value); + + ret.AddChild(key, std::move(value)); last_offset = next_offset; } - return obj; + return ret; } -yyjson_mut_val *VariantBinaryDecoder::ArrayDecode(yyjson_mut_doc *doc, const VariantMetadata &metadata, - const VariantValueMetadata &value_metadata, const_data_ptr_t data) { - auto arr = yyjson_mut_arr(doc); +VariantValue VariantBinaryDecoder::ArrayDecode(const VariantMetadata &metadata, + const VariantValueMetadata &value_metadata, const_data_ptr_t data) { + VariantValue ret(VariantValueType::ARRAY); auto field_offset_size = value_metadata.field_offset_size; auto is_large = value_metadata.is_large; @@ -360,30 +336,28 @@ yyjson_mut_val *VariantBinaryDecoder::ArrayDecode(yyjson_mut_doc *doc, const Var for (idx_t i = 0; i < num_elements; i++) { auto next_offset = ReadVariableLengthLittleEndian(field_offset_size, field_offsets); - auto value = Decode(doc, metadata, values + last_offset); - yyjson_mut_arr_add_val(arr, value); + ret.AddItem(Decode(metadata, values + last_offset)); last_offset = next_offset; } - return arr; + return ret; } -yyjson_mut_val *VariantBinaryDecoder::Decode(yyjson_mut_doc *doc, const VariantMetadata &variant_metadata, - const_data_ptr_t data) { +VariantValue VariantBinaryDecoder::Decode(const VariantMetadata &variant_metadata, const_data_ptr_t data) { auto value_metadata = VariantValueMetadata::FromHeaderByte(data[0]); data++; switch (value_metadata.basic_type) { case VariantBasicType::PRIMITIVE: { - return PrimitiveTypeDecode(doc, variant_metadata, value_metadata, data); + return PrimitiveTypeDecode(variant_metadata, value_metadata, data); } case VariantBasicType::SHORT_STRING: { - return ShortStringDecode(doc, variant_metadata, value_metadata, data); + return ShortStringDecode(variant_metadata, value_metadata, data); } case VariantBasicType::OBJECT: { - return ObjectDecode(doc, variant_metadata, value_metadata, data); + return ObjectDecode(variant_metadata, value_metadata, data); } case VariantBasicType::ARRAY: { - return ArrayDecode(doc, variant_metadata, value_metadata, data); + return ArrayDecode(variant_metadata, value_metadata, data); } default: throw InternalException("Unexpected value for VariantBasicType"); diff --git a/src/duckdb/extension/parquet/reader/variant/variant_shredded_conversion.cpp b/src/duckdb/extension/parquet/reader/variant/variant_shredded_conversion.cpp new file mode 100644 index 000000000..8c7473e93 --- /dev/null +++ b/src/duckdb/extension/parquet/reader/variant/variant_shredded_conversion.cpp @@ -0,0 +1,559 @@ +#include "reader/variant/variant_shredded_conversion.hpp" +#include "column_reader.hpp" +#include "utf8proc_wrapper.hpp" + +#include "duckdb/common/types/timestamp.hpp" +#include "duckdb/common/types/decimal.hpp" +#include "duckdb/common/types/uuid.hpp" +#include "duckdb/common/types/time.hpp" +#include "duckdb/common/types/date.hpp" +#include "duckdb/common/types/blob.hpp" + +namespace duckdb { + +template +struct ConvertShreddedValue { + static VariantValue Convert(T val); + static VariantValue ConvertDecimal(T val, uint8_t width, uint8_t scale) { + throw InternalException("ConvertShreddedValue::ConvertDecimal not implemented for type"); + } + static VariantValue ConvertBlob(T val) { + throw InternalException("ConvertShreddedValue::ConvertBlob not implemented for type"); + } +}; + +//! boolean +template <> +VariantValue ConvertShreddedValue::Convert(bool val) { + return VariantValue(Value::BOOLEAN(val)); +} +//! int8 +template <> +VariantValue ConvertShreddedValue::Convert(int8_t val) { + return VariantValue(Value::TINYINT(val)); +} +//! int16 +template <> +VariantValue ConvertShreddedValue::Convert(int16_t val) { + return VariantValue(Value::SMALLINT(val)); +} +//! int32 +template <> +VariantValue ConvertShreddedValue::Convert(int32_t val) { + return VariantValue(Value::INTEGER(val)); +} +//! int64 +template <> +VariantValue ConvertShreddedValue::Convert(int64_t val) { + return VariantValue(Value::BIGINT(val)); +} +//! float +template <> +VariantValue ConvertShreddedValue::Convert(float val) { + return VariantValue(Value::FLOAT(val)); +} +//! double +template <> +VariantValue ConvertShreddedValue::Convert(double val) { + return VariantValue(Value::DOUBLE(val)); +} +//! decimal4/decimal8/decimal16 +template <> +VariantValue ConvertShreddedValue::ConvertDecimal(int32_t val, uint8_t width, uint8_t scale) { + auto value_str = Decimal::ToString(val, width, scale); + return VariantValue(Value(value_str)); +} +template <> +VariantValue ConvertShreddedValue::ConvertDecimal(int64_t val, uint8_t width, uint8_t scale) { + auto value_str = Decimal::ToString(val, width, scale); + return VariantValue(Value(value_str)); +} +template <> +VariantValue ConvertShreddedValue::ConvertDecimal(hugeint_t val, uint8_t width, uint8_t scale) { + auto value_str = Decimal::ToString(val, width, scale); + return VariantValue(Value(value_str)); +} +//! date +template <> +VariantValue ConvertShreddedValue::Convert(date_t val) { + return VariantValue(Value::DATE(val)); +} +//! time +template <> +VariantValue ConvertShreddedValue::Convert(dtime_t val) { + return VariantValue(Value::TIME(val)); +} +//! timestamptz(6) +template <> +VariantValue ConvertShreddedValue::Convert(timestamp_tz_t val) { + return VariantValue(Value::TIMESTAMPTZ(val)); +} +////! timestamptz(9) +// template <> +// VariantValue ConvertShreddedValue::Convert(timestamp_ns_tz_t val) { +// return VariantValue(Value::TIMESTAMPNS_TZ(val)); +//} +//! timestampntz(6) +template <> +VariantValue ConvertShreddedValue::Convert(timestamp_t val) { + return VariantValue(Value::TIMESTAMP(val)); +} +//! timestampntz(9) +template <> +VariantValue ConvertShreddedValue::Convert(timestamp_ns_t val) { + return VariantValue(Value::TIMESTAMPNS(val)); +} +//! binary +template <> +VariantValue ConvertShreddedValue::ConvertBlob(string_t val) { + return VariantValue(Value(Blob::ToBase64(val))); +} +//! string +template <> +VariantValue ConvertShreddedValue::Convert(string_t val) { + if (!Utf8Proc::IsValid(val.GetData(), val.GetSize())) { + throw InternalException("Can't decode Variant string, it isn't valid UTF8"); + } + return VariantValue(Value(val.GetString())); +} +//! uuid +template <> +VariantValue ConvertShreddedValue::Convert(hugeint_t val) { + return VariantValue(Value(UUID::ToString(val))); +} + +template +vector ConvertTypedValues(Vector &vec, Vector &metadata, Vector &blob, idx_t offset, idx_t length, + idx_t total_size) { + UnifiedVectorFormat metadata_format; + metadata.ToUnifiedFormat(length, metadata_format); + auto metadata_data = metadata_format.GetData(metadata_format); + + UnifiedVectorFormat typed_format; + vec.ToUnifiedFormat(total_size, typed_format); + auto data = typed_format.GetData(typed_format); + + UnifiedVectorFormat value_format; + blob.ToUnifiedFormat(total_size, value_format); + auto value_data = value_format.GetData(value_format); + + auto &validity = typed_format.validity; + auto &value_validity = value_format.validity; + auto &type = vec.GetType(); + + //! Values only used for Decimal conversion + uint8_t width; + uint8_t scale; + if (TYPE_ID == LogicalTypeId::DECIMAL) { + type.GetDecimalProperties(width, scale); + } + + vector ret(length); + if (validity.AllValid()) { + for (idx_t i = 0; i < length; i++) { + auto index = typed_format.sel->get_index(i + offset); + if (TYPE_ID == LogicalTypeId::DECIMAL) { + ret[i] = OP::ConvertDecimal(data[index], width, scale); + } else if (TYPE_ID == LogicalTypeId::BLOB) { + ret[i] = OP::ConvertBlob(data[index]); + } else { + ret[i] = OP::Convert(data[index]); + } + } + } else { + for (idx_t i = 0; i < length; i++) { + auto typed_index = typed_format.sel->get_index(i + offset); + auto value_index = value_format.sel->get_index(i + offset); + if (validity.RowIsValid(typed_index)) { + //! This is a leaf, partially shredded values aren't possible here + D_ASSERT(!value_validity.RowIsValid(value_index)); + if (TYPE_ID == LogicalTypeId::DECIMAL) { + ret[i] = OP::ConvertDecimal(data[typed_index], width, scale); + } else if (TYPE_ID == LogicalTypeId::BLOB) { + ret[i] = OP::ConvertBlob(data[typed_index]); + } else { + ret[i] = OP::Convert(data[typed_index]); + } + } else if (value_validity.RowIsValid(value_index)) { + auto metadata_value = metadata_data[metadata_format.sel->get_index(i)]; + VariantMetadata variant_metadata(metadata_value); + ret[i] = VariantBinaryDecoder::Decode(variant_metadata, + const_data_ptr_cast(value_data[value_index].GetData())); + } + } + } + return ret; +} + +vector VariantShreddedConversion::ConvertShreddedLeaf(Vector &metadata, Vector &value, + Vector &typed_value, idx_t offset, idx_t length, + idx_t total_size) { + D_ASSERT(!typed_value.GetType().IsNested()); + vector result; + + auto &type = typed_value.GetType(); + switch (type.id()) { + //! boolean + case LogicalTypeId::BOOLEAN: { + return ConvertTypedValues, LogicalTypeId::BOOLEAN>( + typed_value, metadata, value, offset, length, total_size); + } + //! int8 + case LogicalTypeId::TINYINT: { + return ConvertTypedValues, LogicalTypeId::TINYINT>( + typed_value, metadata, value, offset, length, total_size); + } + //! int16 + case LogicalTypeId::SMALLINT: { + return ConvertTypedValues, LogicalTypeId::SMALLINT>( + typed_value, metadata, value, offset, length, total_size); + } + //! int32 + case LogicalTypeId::INTEGER: { + return ConvertTypedValues, LogicalTypeId::INTEGER>( + typed_value, metadata, value, offset, length, total_size); + } + //! int64 + case LogicalTypeId::BIGINT: { + return ConvertTypedValues, LogicalTypeId::BIGINT>( + typed_value, metadata, value, offset, length, total_size); + } + //! float + case LogicalTypeId::FLOAT: { + return ConvertTypedValues, LogicalTypeId::FLOAT>( + typed_value, metadata, value, offset, length, total_size); + } + //! double + case LogicalTypeId::DOUBLE: { + return ConvertTypedValues, LogicalTypeId::DOUBLE>( + typed_value, metadata, value, offset, length, total_size); + } + //! decimal4/decimal8/decimal16 + case LogicalTypeId::DECIMAL: { + auto physical_type = type.InternalType(); + switch (physical_type) { + case PhysicalType::INT32: { + return ConvertTypedValues, LogicalTypeId::DECIMAL>( + typed_value, metadata, value, offset, length, total_size); + } + case PhysicalType::INT64: { + return ConvertTypedValues, LogicalTypeId::DECIMAL>( + typed_value, metadata, value, offset, length, total_size); + } + case PhysicalType::INT128: { + return ConvertTypedValues, LogicalTypeId::DECIMAL>( + typed_value, metadata, value, offset, length, total_size); + } + default: + throw NotImplementedException("Decimal with PhysicalType (%s) not implemented for shredded Variant", + EnumUtil::ToString(physical_type)); + } + } + //! date + case LogicalTypeId::DATE: { + return ConvertTypedValues, LogicalTypeId::DATE>( + typed_value, metadata, value, offset, length, total_size); + } + //! time + case LogicalTypeId::TIME: { + return ConvertTypedValues, LogicalTypeId::TIME>( + typed_value, metadata, value, offset, length, total_size); + } + //! timestamptz(6) (timestamptz(9) not implemented in DuckDB) + case LogicalTypeId::TIMESTAMP_TZ: { + return ConvertTypedValues, LogicalTypeId::TIMESTAMP_TZ>( + typed_value, metadata, value, offset, length, total_size); + } + //! timestampntz(6) + case LogicalTypeId::TIMESTAMP: { + return ConvertTypedValues, LogicalTypeId::TIMESTAMP>( + typed_value, metadata, value, offset, length, total_size); + } + //! timestampntz(9) + case LogicalTypeId::TIMESTAMP_NS: { + return ConvertTypedValues, LogicalTypeId::TIMESTAMP_NS>( + typed_value, metadata, value, offset, length, total_size); + } + //! binary + case LogicalTypeId::BLOB: { + return ConvertTypedValues, LogicalTypeId::BLOB>( + typed_value, metadata, value, offset, length, total_size); + } + //! string + case LogicalTypeId::VARCHAR: { + return ConvertTypedValues, LogicalTypeId::VARCHAR>( + typed_value, metadata, value, offset, length, total_size); + } + //! uuid + case LogicalTypeId::UUID: { + return ConvertTypedValues, LogicalTypeId::UUID>( + typed_value, metadata, value, offset, length, total_size); + } + default: + throw NotImplementedException("Variant shredding on type: '%s' is not implemented", type.ToString()); + } +} + +namespace { + +struct ShreddedVariantField { +public: + explicit ShreddedVariantField(const string &field_name) : field_name(field_name) { + } + +public: + string field_name; + //! Values for the field, for all rows + vector values; +}; + +} // namespace + +template +static vector ConvertBinaryEncoding(Vector &metadata, Vector &value, idx_t offset, idx_t length, + idx_t total_size) { + UnifiedVectorFormat value_format; + value.ToUnifiedFormat(total_size, value_format); + auto value_data = value_format.GetData(value_format); + auto &validity = value_format.validity; + + UnifiedVectorFormat metadata_format; + metadata.ToUnifiedFormat(length, metadata_format); + auto metadata_data = metadata_format.GetData(metadata_format); + + vector ret(length); + if (IS_REQUIRED) { + D_ASSERT(validity.CountValid(length) == length); + for (idx_t i = 0; i < length; i++) { + auto index = value_format.sel->get_index(i + offset); + + D_ASSERT(validity.RowIsValid(index)); + auto &metadata_value = metadata_data[metadata_format.sel->get_index(i)]; + VariantMetadata variant_metadata(metadata_value); + auto binary_value = value_data[index].GetData(); + ret[i] = VariantBinaryDecoder::Decode(variant_metadata, const_data_ptr_cast(binary_value)); + } + } else { + //! Even though 'typed_value' is not present, 'value' is allowed to contain NULLs because we're scanning an + //! Object's shredded field. + //! When 'value' is null for a row, that means the Object does not contain this field + //! for that row. + for (idx_t i = 0; i < length; i++) { + auto index = value_format.sel->get_index(i + offset); + if (validity.RowIsValid(index)) { + auto &metadata_value = metadata_data[metadata_format.sel->get_index(i)]; + VariantMetadata variant_metadata(metadata_value); + auto binary_value = value_data[index].GetData(); + ret[i] = VariantBinaryDecoder::Decode(variant_metadata, const_data_ptr_cast(binary_value)); + } + } + } + return ret; +} + +static VariantValue ConvertPartiallyShreddedObject(vector &shredded_fields, + const UnifiedVectorFormat &metadata_format, + const UnifiedVectorFormat &value_format, idx_t i, idx_t offset) { + auto ret = VariantValue(VariantValueType::OBJECT); + auto index = value_format.sel->get_index(i + offset); + auto value_data = value_format.GetData(value_format); + auto metadata_data = metadata_format.GetData(metadata_format); + auto &value_validity = value_format.validity; + + for (idx_t field_index = 0; field_index < shredded_fields.size(); field_index++) { + auto &shredded_field = shredded_fields[field_index]; + auto &field_value = shredded_field.values[i]; + + if (field_value.IsMissing()) { + //! This field is missing from the value, skip it + continue; + } + ret.AddChild(shredded_field.field_name, std::move(field_value)); + } + + if (value_validity.RowIsValid(index)) { + //! Object is partially shredded, decode the object and merge the values + auto &metadata_value = metadata_data[metadata_format.sel->get_index(i)]; + VariantMetadata variant_metadata(metadata_value); + auto binary_value = value_data[index].GetData(); + auto unshredded = VariantBinaryDecoder::Decode(variant_metadata, const_data_ptr_cast(binary_value)); + if (unshredded.value_type != VariantValueType::OBJECT) { + throw InvalidInputException("Partially shredded objects have to encode Object Variants in the 'value'"); + } + for (auto &item : unshredded.object_children) { + ret.AddChild(item.first, std::move(item.second)); + } + } + return ret; +} + +vector VariantShreddedConversion::ConvertShreddedObject(Vector &metadata, Vector &value, + Vector &typed_value, idx_t offset, idx_t length, + idx_t total_size) { + auto &type = typed_value.GetType(); + D_ASSERT(type.id() == LogicalTypeId::STRUCT); + auto &fields = StructType::GetChildTypes(type); + auto &entries = StructVector::GetEntries(typed_value); + D_ASSERT(entries.size() == fields.size()); + + //! 'value' + UnifiedVectorFormat value_format; + value.ToUnifiedFormat(total_size, value_format); + auto value_data = value_format.GetData(value_format); + auto &validity = value_format.validity; + + //! 'metadata' + UnifiedVectorFormat metadata_format; + metadata.ToUnifiedFormat(length, metadata_format); + auto metadata_data = metadata_format.GetData(metadata_format); + + //! 'typed_value' + UnifiedVectorFormat typed_format; + typed_value.ToUnifiedFormat(total_size, typed_format); + auto &typed_validity = typed_format.validity; + + //! Process all fields to get the shredded field values + vector shredded_fields; + shredded_fields.reserve(fields.size()); + for (idx_t i = 0; i < fields.size(); i++) { + auto &field = fields[i]; + auto &field_name = field.first; + auto &field_vec = *entries[i]; + + shredded_fields.emplace_back(field_name); + auto &shredded_field = shredded_fields.back(); + shredded_field.values = Convert(metadata, field_vec, offset, length, total_size, true); + } + + vector ret(length); + if (typed_validity.AllValid()) { + for (idx_t i = 0; i < length; i++) { + ret[i] = ConvertPartiallyShreddedObject(shredded_fields, metadata_format, value_format, i, offset); + } + } else { + //! For some of the rows, the value is not an object + for (idx_t i = 0; i < length; i++) { + auto typed_index = typed_format.sel->get_index(i + offset); + auto value_index = value_format.sel->get_index(i + offset); + if (typed_validity.RowIsValid(typed_index)) { + ret[i] = ConvertPartiallyShreddedObject(shredded_fields, metadata_format, value_format, i, offset); + } else { + //! The value on this row is not an object, and guaranteed to be present + D_ASSERT(validity.RowIsValid(value_index)); + auto &metadata_value = metadata_data[metadata_format.sel->get_index(i)]; + VariantMetadata variant_metadata(metadata_value); + auto binary_value = value_data[value_index].GetData(); + ret[i] = VariantBinaryDecoder::Decode(variant_metadata, const_data_ptr_cast(binary_value)); + if (ret[i].value_type == VariantValueType::OBJECT) { + throw InvalidInputException( + "When 'typed_value' for a shredded Object is NULL, 'value' can not contain an Object value"); + } + } + } + } + return ret; +} + +vector VariantShreddedConversion::ConvertShreddedArray(Vector &metadata, Vector &value, + Vector &typed_value, idx_t offset, idx_t length, + idx_t total_size) { + auto &child = ListVector::GetEntry(typed_value); + auto list_size = ListVector::GetListSize(typed_value); + + //! 'value' + UnifiedVectorFormat value_format; + value.ToUnifiedFormat(total_size, value_format); + auto value_data = value_format.GetData(value_format); + + //! 'metadata' + UnifiedVectorFormat metadata_format; + metadata.ToUnifiedFormat(length, metadata_format); + auto metadata_data = metadata_format.GetData(metadata_format); + + //! 'typed_value' + UnifiedVectorFormat list_format; + typed_value.ToUnifiedFormat(total_size, list_format); + auto list_data = list_format.GetData(list_format); + auto &validity = list_format.validity; + auto &value_validity = value_format.validity; + + vector ret(length); + if (validity.AllValid()) { + //! We can be sure that none of the values are binary encoded + for (idx_t i = 0; i < length; i++) { + auto typed_index = list_format.sel->get_index(i + offset); + //! FIXME: next 4 lines duplicated below + auto entry = list_data[typed_index]; + Vector child_metadata(metadata.GetValue(i)); + ret[i] = VariantValue(VariantValueType::ARRAY); + ret[i].array_items = Convert(child_metadata, child, entry.offset, entry.length, list_size); + } + } else { + for (idx_t i = 0; i < length; i++) { + auto typed_index = list_format.sel->get_index(i + offset); + auto value_index = value_format.sel->get_index(i + offset); + if (validity.RowIsValid(typed_index)) { + //! FIXME: next 4 lines duplicate + auto entry = list_data[typed_index]; + Vector child_metadata(metadata.GetValue(i)); + ret[i] = VariantValue(VariantValueType::ARRAY); + ret[i].array_items = Convert(child_metadata, child, entry.offset, entry.length, list_size); + } else if (value_validity.RowIsValid(value_index)) { + auto metadata_value = metadata_data[metadata_format.sel->get_index(i)]; + VariantMetadata variant_metadata(metadata_value); + ret[i] = VariantBinaryDecoder::Decode(variant_metadata, + const_data_ptr_cast(value_data[value_index].GetData())); + } + } + } + return ret; +} + +vector VariantShreddedConversion::Convert(Vector &metadata, Vector &group, idx_t offset, idx_t length, + idx_t total_size, bool is_field) { + D_ASSERT(group.GetType().id() == LogicalTypeId::STRUCT); + + auto &group_entries = StructVector::GetEntries(group); + auto &group_type_children = StructType::GetChildTypes(group.GetType()); + D_ASSERT(group_type_children.size() == group_entries.size()); + + //! From the spec: + //! The Parquet columns used to store variant metadata and values must be accessed by name, not by position. + optional_ptr value; + optional_ptr typed_value; + for (idx_t i = 0; i < group_entries.size(); i++) { + auto &name = group_type_children[i].first; + auto &vec = group_entries[i]; + if (name == "value") { + value = vec.get(); + } else if (name == "typed_value") { + typed_value = vec.get(); + } else { + throw InvalidInputException("Variant group can only contain 'value'/'typed_value', not: %s", name); + } + } + if (!value) { + throw InvalidInputException("Required column 'value' not found in Variant group"); + } + + if (typed_value) { + auto &type = typed_value->GetType(); + vector ret; + if (type.id() == LogicalTypeId::STRUCT) { + return ConvertShreddedObject(metadata, *value, *typed_value, offset, length, total_size); + } else if (type.id() == LogicalTypeId::LIST) { + return ConvertShreddedArray(metadata, *value, *typed_value, offset, length, total_size); + } else { + return ConvertShreddedLeaf(metadata, *value, *typed_value, offset, length, total_size); + } + } else { + if (is_field) { + return ConvertBinaryEncoding(metadata, *value, offset, length, total_size); + } else { + //! Only 'value' is present, we can assume this to be 'required', so it can't contain NULLs + return ConvertBinaryEncoding(metadata, *value, offset, length, total_size); + } + } +} + +} // namespace duckdb diff --git a/src/duckdb/extension/parquet/reader/variant/variant_value.cpp b/src/duckdb/extension/parquet/reader/variant/variant_value.cpp new file mode 100644 index 000000000..0ac213469 --- /dev/null +++ b/src/duckdb/extension/parquet/reader/variant/variant_value.cpp @@ -0,0 +1,85 @@ +#include "reader/variant/variant_value.hpp" + +namespace duckdb { + +void VariantValue::AddChild(const string &key, VariantValue &&val) { + D_ASSERT(value_type == VariantValueType::OBJECT); + object_children.emplace(key, std::move(val)); +} + +void VariantValue::AddItem(VariantValue &&val) { + D_ASSERT(value_type == VariantValueType::ARRAY); + array_items.push_back(std::move(val)); +} + +yyjson_mut_val *VariantValue::ToJSON(ClientContext &context, yyjson_mut_doc *doc) const { + switch (value_type) { + case VariantValueType::PRIMITIVE: { + if (primitive_value.IsNull()) { + return yyjson_mut_null(doc); + } + switch (primitive_value.type().id()) { + case LogicalTypeId::BOOLEAN: { + if (primitive_value.GetValue()) { + return yyjson_mut_true(doc); + } else { + return yyjson_mut_false(doc); + } + } + case LogicalTypeId::TINYINT: + return yyjson_mut_int(doc, primitive_value.GetValue()); + case LogicalTypeId::SMALLINT: + return yyjson_mut_int(doc, primitive_value.GetValue()); + case LogicalTypeId::INTEGER: + return yyjson_mut_int(doc, primitive_value.GetValue()); + case LogicalTypeId::BIGINT: + return yyjson_mut_int(doc, primitive_value.GetValue()); + case LogicalTypeId::FLOAT: + return yyjson_mut_real(doc, primitive_value.GetValue()); + case LogicalTypeId::DOUBLE: + return yyjson_mut_real(doc, primitive_value.GetValue()); + case LogicalTypeId::DATE: + case LogicalTypeId::TIME: + case LogicalTypeId::VARCHAR: { + auto value_str = primitive_value.ToString(); + return yyjson_mut_strncpy(doc, value_str.c_str(), value_str.size()); + } + case LogicalTypeId::TIMESTAMP: { + auto value_str = primitive_value.ToString(); + return yyjson_mut_strncpy(doc, value_str.c_str(), value_str.size()); + } + case LogicalTypeId::TIMESTAMP_TZ: { + auto value_str = primitive_value.CastAs(context, LogicalType::VARCHAR).GetValue(); + return yyjson_mut_strncpy(doc, value_str.c_str(), value_str.size()); + } + case LogicalTypeId::TIMESTAMP_NS: { + auto value_str = primitive_value.CastAs(context, LogicalType::VARCHAR).GetValue(); + return yyjson_mut_strncpy(doc, value_str.c_str(), value_str.size()); + } + default: + throw InternalException("Unexpected primitive type: %s", primitive_value.type().ToString()); + } + } + case VariantValueType::OBJECT: { + auto obj = yyjson_mut_obj(doc); + for (const auto &it : object_children) { + auto &key = it.first; + auto value = it.second.ToJSON(context, doc); + yyjson_mut_obj_add_val(doc, obj, key.c_str(), value); + } + return obj; + } + case VariantValueType::ARRAY: { + auto arr = yyjson_mut_arr(doc); + for (auto &item : array_items) { + auto value = item.ToJSON(context, doc); + yyjson_mut_arr_add_val(arr, value); + } + return arr; + } + default: + throw InternalException("Can't serialize this VariantValue type to JSON"); + } +} + +} // namespace duckdb diff --git a/src/duckdb/extension/parquet/reader/variant_column_reader.cpp b/src/duckdb/extension/parquet/reader/variant_column_reader.cpp index 80f9dfedd..751edc9ed 100644 --- a/src/duckdb/extension/parquet/reader/variant_column_reader.cpp +++ b/src/duckdb/extension/parquet/reader/variant_column_reader.cpp @@ -1,5 +1,6 @@ #include "reader/variant_column_reader.hpp" #include "reader/variant/variant_binary_decoder.hpp" +#include "reader/variant/variant_shredded_conversion.hpp" namespace duckdb { @@ -31,52 +32,68 @@ void VariantColumnReader::InitializeRead(idx_t row_group_idx_p, const vector typed_value) { + child_list_t children; + children.emplace_back("value", LogicalType::BLOB); + if (typed_value) { + children.emplace_back("typed_value", typed_value->Type()); } + return LogicalType::STRUCT(std::move(children)); +} +idx_t VariantColumnReader::Read(uint64_t num_values, data_ptr_t define_out, data_ptr_t repeat_out, Vector &result) { if (pending_skips > 0) { throw InternalException("VariantColumnReader cannot have pending skips"); } + optional_ptr typed_value_reader = child_readers.size() == 3 ? child_readers[2].get() : nullptr; // If the child reader values are all valid, "define_out" may not be initialized at all // So, we just initialize them to all be valid beforehand std::fill_n(define_out, num_values, MaxDefine()); optional_idx read_count; - Vector value_intermediate(LogicalType::BLOB, num_values); + Vector metadata_intermediate(LogicalType::BLOB, num_values); + Vector intermediate_group(GetIntermediateGroupType(typed_value_reader), num_values); + auto &group_entries = StructVector::GetEntries(intermediate_group); + auto &value_intermediate = *group_entries[0]; + auto metadata_values = child_readers[0]->Read(num_values, define_out, repeat_out, metadata_intermediate); auto value_values = child_readers[1]->Read(num_values, define_out, repeat_out, value_intermediate); if (metadata_values != value_values) { throw InvalidInputException( - "The unshredded Variant column did not contain the same amount of values for 'metadata' and 'value'"); + "The Variant column did not contain the same amount of values for 'metadata' and 'value'"); } - VariantBinaryDecoder decoder(context); - auto result_data = FlatVector::GetData(result); - auto metadata_intermediate_data = FlatVector::GetData(metadata_intermediate); - auto value_intermediate_data = FlatVector::GetData(value_intermediate); - - auto metadata_validity = FlatVector::Validity(metadata_intermediate); - auto value_validity = FlatVector::Validity(value_intermediate); - for (idx_t i = 0; i < num_values; i++) { - if (!metadata_validity.RowIsValid(i) || !value_validity.RowIsValid(i)) { - throw InvalidInputException("The Variant 'metadata' and 'value' columns can not produce NULL values"); + auto &result_validity = FlatVector::Validity(result); + + vector conversion_result; + if (typed_value_reader) { + auto typed_values = typed_value_reader->Read(num_values, define_out, repeat_out, *group_entries[1]); + if (typed_values != value_values) { + throw InvalidInputException( + "The shredded Variant column did not contain the same amount of values for 'typed_value' and 'value'"); } - VariantMetadata variant_metadata(metadata_intermediate_data[i]); - auto value_data = reinterpret_cast(value_intermediate_data[i].GetData()); + } + conversion_result = + VariantShreddedConversion::Convert(metadata_intermediate, intermediate_group, 0, num_values, num_values); + for (idx_t i = 0; i < conversion_result.size(); i++) { + auto &variant = conversion_result[i]; + if (variant.IsNull()) { + result_validity.SetInvalid(i); + continue; + } + + //! Write the result to a string VariantDecodeResult decode_result; decode_result.doc = yyjson_mut_doc_new(nullptr); + auto json_val = variant.ToJSON(context, decode_result.doc); - auto val = decoder.Decode(decode_result.doc, variant_metadata, value_data); - - //! Write the result to a string size_t len; - decode_result.data = yyjson_mut_val_write_opts(val, YYJSON_WRITE_ALLOW_INF_AND_NAN, nullptr, &len, nullptr); + decode_result.data = + yyjson_mut_val_write_opts(json_val, YYJSON_WRITE_ALLOW_INF_AND_NAN, nullptr, &len, nullptr); if (!decode_result.data) { throw InvalidInputException("Could not serialize the JSON to string, yyjson failed"); } diff --git a/src/duckdb/src/catalog/catalog.cpp b/src/duckdb/src/catalog/catalog.cpp index bb895d4d7..687293ad3 100644 --- a/src/duckdb/src/catalog/catalog.cpp +++ b/src/duckdb/src/catalog/catalog.cpp @@ -1140,6 +1140,16 @@ vector> Catalog::GetAllSchemas(ClientContext &cont return result; } +vector> Catalog::GetAllEntries(ClientContext &context, CatalogType catalog_type) { + vector> result; + auto schemas = GetAllSchemas(context); + for (const auto &schema_ref : schemas) { + auto &schema = schema_ref.get(); + schema.Scan(context, catalog_type, [&](CatalogEntry &entry) { result.push_back(entry); }); + } + return result; +} + void Catalog::Alter(CatalogTransaction transaction, AlterInfo &info) { if (transaction.HasContext()) { CatalogEntryRetriever retriever(transaction.GetContext()); diff --git a/src/duckdb/src/catalog/catalog_entry/duck_table_entry.cpp b/src/duckdb/src/catalog/catalog_entry/duck_table_entry.cpp index b9345b529..a1f23aa7f 100644 --- a/src/duckdb/src/catalog/catalog_entry/duck_table_entry.cpp +++ b/src/duckdb/src/catalog/catalog_entry/duck_table_entry.cpp @@ -855,12 +855,12 @@ unique_ptr DuckTableEntry::RenameField(ClientContext &context, Ren if (!ColumnExists(info.column_path[0])) { throw CatalogException("Cannot rename field from column \"%s\" - it does not exist", info.column_path[0]); } + // follow the path auto &col = GetColumn(info.column_path[0]); auto res = RenameFieldFromStruct(col.Type(), info.column_path, info.new_name, 1); if (res.error.HasError()) { res.error.Throw(); - return nullptr; } // construct the struct remapping expression @@ -871,7 +871,6 @@ unique_ptr DuckTableEntry::RenameField(ClientContext &context, Ren children.push_back(make_uniq(Value())); auto function = make_uniq("remap_struct", std::move(children)); - ChangeColumnTypeInfo change_column_type(info.GetAlterEntryData(), info.column_path[0], std::move(res.new_type), std::move(function)); return ChangeColumnType(context, change_column_type); diff --git a/src/duckdb/src/common/adbc/adbc.cpp b/src/duckdb/src/common/adbc/adbc.cpp index 3da768021..197b5f3d0 100644 --- a/src/duckdb/src/common/adbc/adbc.cpp +++ b/src/duckdb/src/common/adbc/adbc.cpp @@ -8,8 +8,6 @@ #include "duckdb/common/arrow/arrow_wrapper.hpp" #include "duckdb/common/arrow/nanoarrow/nanoarrow.hpp" -#include "duckdb/main/capi/capi_internal.hpp" - #ifndef DUCKDB_AMALGAMATION #include "duckdb/main/connection.hpp" #endif @@ -21,6 +19,8 @@ #include #include +#include "duckdb/main/prepared_statement_data.hpp" + // We must leak the symbols of the init function AdbcStatusCode duckdb_adbc_init(int version, void *driver, struct AdbcError *error) { if (!driver) { @@ -62,7 +62,6 @@ enum class IngestionMode { CREATE = 0, APPEND = 1 }; struct DuckDBAdbcStatementWrapper { duckdb_connection connection; - duckdb_arrow result; duckdb_prepared_statement statement; char *ingestion_table_name; char *db_schema; @@ -72,6 +71,10 @@ struct DuckDBAdbcStatementWrapper { uint64_t plan_length; }; +struct DuckDBAdbcStreamWrapper { + duckdb_result result; +}; + static AdbcStatusCode QueryInternal(struct AdbcConnection *connection, struct ArrowArrayStream *out, const char *query, struct AdbcError *error) { AdbcStatement statement; @@ -533,8 +536,31 @@ static int get_schema(struct ArrowArrayStream *stream, struct ArrowSchema *out) if (!stream || !stream->private_data || !out) { return DuckDBError; } - return duckdb_query_arrow_schema(static_cast(stream->private_data), - reinterpret_cast(&out)); + auto result_wrapper = static_cast(stream->private_data); + auto count = duckdb_column_count(&result_wrapper->result); + std::vector types(count); + + std::vector owned_names(count); + duckdb::vector names(count); + for (idx_t i = 0; i < count; i++) { + types[i] = duckdb_column_logical_type(&result_wrapper->result, i); + auto column_name = duckdb_column_name(&result_wrapper->result, i); + owned_names.emplace_back(column_name); + names[i] = owned_names.back().c_str(); + } + + auto arrow_options = duckdb_result_get_arrow_options(&result_wrapper->result); + + auto res = duckdb_to_arrow_schema(arrow_options, &types[0], names.data(), count, out); + duckdb_destroy_arrow_options(&arrow_options); + for (auto &type : types) { + duckdb_destroy_logical_type(&type); + } + if (res) { + duckdb_destroy_error_data(&res); + return DuckDBError; + } + return DuckDBSuccess; } static int get_next(struct ArrowArrayStream *stream, struct ArrowArray *out) { @@ -542,28 +568,39 @@ static int get_next(struct ArrowArrayStream *stream, struct ArrowArray *out) { return DuckDBError; } out->release = nullptr; + auto result_wrapper = static_cast(stream->private_data); + auto duckdb_chunk = duckdb_fetch_chunk(result_wrapper->result); + if (!duckdb_chunk) { + return DuckDBSuccess; + } + auto arrow_options = duckdb_result_get_arrow_options(&result_wrapper->result); - return duckdb_query_arrow_array(static_cast(stream->private_data), - reinterpret_cast(&out)); + auto conversion_success = duckdb_data_chunk_to_arrow(arrow_options, duckdb_chunk, out); + duckdb_destroy_arrow_options(&arrow_options); + duckdb_destroy_data_chunk(&duckdb_chunk); + + if (conversion_success) { + duckdb_destroy_error_data(&conversion_success); + return DuckDBError; + } + return DuckDBSuccess; } void release(struct ArrowArrayStream *stream) { if (!stream || !stream->release) { return; } - if (stream->private_data) { - duckdb_destroy_arrow(reinterpret_cast(&stream->private_data)); - stream->private_data = nullptr; + auto result_wrapper = reinterpret_cast(stream->private_data); + if (result_wrapper) { + duckdb_destroy_result(&result_wrapper->result); } + free(stream->private_data); + stream->private_data = nullptr; stream->release = nullptr; } const char *get_last_error(struct ArrowArrayStream *stream) { - if (!stream) { - return nullptr; - } return nullptr; - // return duckdb_query_arrow_error(stream); } // this is an evil hack, normally we would need a stream factory here, but its probably much easier if the adbc clients @@ -605,45 +642,66 @@ AdbcStatusCode Ingest(duckdb_connection connection, const char *table_name, cons return ADBC_STATUS_INVALID_ARGUMENT; } - auto cconn = reinterpret_cast(connection); + duckdb::ArrowSchemaWrapper arrow_schema_wrapper; + ConvertedSchemaWrapper out_types; - auto arrow_scan = - cconn->TableFunction("arrow_scan", {duckdb::Value::POINTER(reinterpret_cast(input)), - duckdb::Value::POINTER(reinterpret_cast(stream_produce)), - duckdb::Value::POINTER(reinterpret_cast(stream_schema))}); - try { - switch (ingestion_mode) { - case IngestionMode::CREATE: - if (schema) { - arrow_scan->Create(schema, table_name, temporary); - } else { - arrow_scan->Create(table_name, temporary); - } - break; - case IngestionMode::APPEND: { - arrow_scan->CreateView("temp_adbc_view", true, true); - std::string query = "insert into "; - if (schema) { - query += duckdb::KeywordHelper::WriteOptionallyQuoted(schema) + "."; - } - query += duckdb::KeywordHelper::WriteOptionallyQuoted(table_name); - query += " select * from temp_adbc_view"; - auto result = cconn->Query(query); - break; + input->get_schema(input, &arrow_schema_wrapper.arrow_schema); + auto res = duckdb_schema_from_arrow(connection, &arrow_schema_wrapper.arrow_schema, out_types.GetPtr()); + if (res) { + SetError(error, duckdb_error_data_message(res)); + duckdb_destroy_error_data(&res); + return ADBC_STATUS_INTERNAL; + } + + auto &d_converted_schema = *reinterpret_cast(out_types.Get()); + auto types = d_converted_schema.GetTypes(); + auto names = d_converted_schema.GetNames(); + + if (ingestion_mode == IngestionMode::CREATE) { + // We must construct the create table SQL query + std::ostringstream create_table; + create_table << "CREATE TABLE "; + if (schema) { + create_table << schema << "."; } + create_table << table_name << " ("; + for (idx_t i = 0; i < types.size(); i++) { + create_table << names[i] << " "; + create_table << types[i].ToString(); + if (i + 1 < types.size()) { + create_table << ", "; + } } - // After creating a table, the arrow array stream is released. Hence we must set it as released to avoid - // double-releasing it - input->release = nullptr; - } catch (std::exception &ex) { - if (error) { - duckdb::ErrorData parsed_error(ex); - error->message = strdup(parsed_error.RawMessage().c_str()); + create_table << ");"; + duckdb_result result; + if (duckdb_query(connection, create_table.str().c_str(), &result) == DuckDBError) { + SetError(error, duckdb_result_error(&result)); + duckdb_destroy_result(&result); + return ADBC_STATUS_INTERNAL; } - return ADBC_STATUS_INTERNAL; - } catch (...) { + duckdb_destroy_result(&result); + } + AppenderWrapper appender(connection, schema, table_name); + if (!appender.Valid()) { return ADBC_STATUS_INTERNAL; } + duckdb::ArrowArrayWrapper arrow_array_wrapper; + + input->get_next(input, &arrow_array_wrapper.arrow_array); + while (arrow_array_wrapper.arrow_array.release) { + DataChunkWrapper out_chunk; + auto res = duckdb_data_chunk_from_arrow(connection, &arrow_array_wrapper.arrow_array, out_types.Get(), + &out_chunk.chunk); + if (res) { + SetError(error, duckdb_error_data_message(res)); + duckdb_destroy_error_data(&res); + } + if (duckdb_append_data_chunk(appender.Get(), out_chunk.chunk) != DuckDBSuccess) { + return ADBC_STATUS_INTERNAL; + } + arrow_array_wrapper = duckdb::ArrowArrayWrapper(); + input->get_next(input, &arrow_array_wrapper.arrow_array); + } return ADBC_STATUS_OK; } @@ -675,7 +733,6 @@ AdbcStatusCode StatementNew(struct AdbcConnection *connection, struct AdbcStatem statement_wrapper->connection = conn_wrapper->connection; statement_wrapper->statement = nullptr; - statement_wrapper->result = nullptr; statement_wrapper->ingestion_stream.release = nullptr; statement_wrapper->ingestion_table_name = nullptr; statement_wrapper->db_schema = nullptr; @@ -694,10 +751,6 @@ AdbcStatusCode StatementRelease(struct AdbcStatement *statement, struct AdbcErro duckdb_destroy_prepare(&wrapper->statement); wrapper->statement = nullptr; } - if (wrapper->result) { - duckdb_destroy_arrow(&wrapper->result); - wrapper->result = nullptr; - } if (wrapper->ingestion_stream.release) { wrapper->ingestion_stream.release(&wrapper->ingestion_stream); wrapper->ingestion_stream.release = nullptr; @@ -732,35 +785,44 @@ AdbcStatusCode StatementGetParameterSchema(struct AdbcStatement *statement, stru auto wrapper = static_cast(statement->private_data); // TODO: we might want to cache this, but then we need to return a deep copy anyways.., so I'm not sure if that // would be worth the extra management - auto res = duckdb_prepared_arrow_schema(wrapper->statement, reinterpret_cast(&schema)); - if (res != DuckDBSuccess) { + + auto prepared_wrapper = reinterpret_cast(wrapper->statement); + if (!prepared_wrapper || !prepared_wrapper->statement || !prepared_wrapper->statement->data) { + SetError(error, "Invalid prepared statement wrapper"); return ADBC_STATUS_INVALID_ARGUMENT; } - return ADBC_STATUS_OK; -} + auto count = prepared_wrapper->statement->data->properties.parameter_count; + if (count == 0) { + count = 1; + } + std::vector types(count); + std::vector owned_names(count); + duckdb::vector names(count); -AdbcStatusCode GetPreparedParameters(duckdb_connection connection, duckdb::unique_ptr &result, - ArrowArrayStream *input, AdbcError *error) { - - auto cconn = reinterpret_cast(connection); - - try { - auto arrow_scan = - cconn->TableFunction("arrow_scan", {duckdb::Value::POINTER(reinterpret_cast(input)), - duckdb::Value::POINTER(reinterpret_cast(stream_produce)), - duckdb::Value::POINTER(reinterpret_cast(stream_schema))}); - result = arrow_scan->Execute(); - // After creating a table, the arrow array stream is released. Hence we must set it as released to avoid - // double-releasing it - input->release = nullptr; - } catch (std::exception &ex) { - if (error) { - ::duckdb::ErrorData parsed_error(ex); - error->message = strdup(parsed_error.RawMessage().c_str()); - } - return ADBC_STATUS_INTERNAL; - } catch (...) { - return ADBC_STATUS_INTERNAL; + for (idx_t i = 0; i < count; i++) { + // FIXME: we don't support named parameters yet, but when we do, this needs to be updated + // Every prepared parameter type is UNKNOWN, which we need to map to NULL according to the spec of + // 'AdbcStatementGetParameterSchema' + types[i] = duckdb_create_logical_type(DUCKDB_TYPE_SQLNULL); + auto column_name = std::to_string(i); + owned_names.emplace_back(column_name); + names[i] = owned_names.back().c_str(); + } + + duckdb_arrow_options arrow_options; + duckdb_connection_get_arrow_options(wrapper->connection, &arrow_options); + + auto res = duckdb_to_arrow_schema(arrow_options, &types[0], names.data(), count, schema); + + for (auto &type : types) { + duckdb_destroy_logical_type(&type); + } + duckdb_destroy_arrow_options(&arrow_options); + + if (res) { + SetError(error, duckdb_error_data_message(res)); + duckdb_destroy_error_data(&res); + return ADBC_STATUS_INVALID_ARGUMENT; } return ADBC_STATUS_OK; } @@ -772,7 +834,6 @@ static AdbcStatusCode IngestToTableFromBoundStream(DuckDBAdbcStatementWrapper *s // Take the input stream from the statement auto stream = statement->ingestion_stream; - statement->ingestion_stream.release = nullptr; // Ingest into a table from the bound stream return Ingest(statement->connection, statement->ingestion_table_name, statement->db_schema, &stream, error, @@ -802,34 +863,61 @@ AdbcStatusCode StatementExecuteQuery(struct AdbcStatement *statement, struct Arr if (has_stream && to_table) { return IngestToTableFromBoundStream(wrapper, error); } + auto stream_wrapper = static_cast(malloc(sizeof(DuckDBAdbcStreamWrapper))); if (has_stream) { // A stream was bound to the statement, use that to bind parameters - duckdb::unique_ptr result; ArrowArrayStream stream = wrapper->ingestion_stream; - wrapper->ingestion_stream.release = nullptr; - auto adbc_res = GetPreparedParameters(wrapper->connection, result, &stream, error); - if (adbc_res != ADBC_STATUS_OK) { - return adbc_res; - } - if (!result) { - return ADBC_STATUS_INVALID_ARGUMENT; + ConvertedSchemaWrapper out_types; + duckdb::ArrowSchemaWrapper arrow_schema_wrapper; + stream.get_schema(&stream, &arrow_schema_wrapper.arrow_schema); + try { + auto res = + duckdb_schema_from_arrow(wrapper->connection, &arrow_schema_wrapper.arrow_schema, out_types.GetPtr()); + if (res) { + SetError(error, duckdb_error_data_message(res)); + duckdb_destroy_error_data(&res); + } + } catch (...) { + free(stream_wrapper); + return ADBC_STATUS_INTERNAL; } - duckdb::unique_ptr chunk; auto prepared_statement_params = reinterpret_cast(wrapper->statement)->statement->named_param_map.size(); - while ((chunk = result->Fetch()) != nullptr) { + duckdb::ArrowArrayWrapper arrow_array_wrapper; + + stream.get_next(&stream, &arrow_array_wrapper.arrow_array); + + while (arrow_array_wrapper.arrow_array.release) { + // This is a valid arrow array, let's make it into a data chunk + DataChunkWrapper out_chunk; + auto res_conv = duckdb_data_chunk_from_arrow(wrapper->connection, &arrow_array_wrapper.arrow_array, + out_types.Get(), &out_chunk.chunk); + if (res_conv) { + SetError(error, duckdb_error_data_message(res_conv)); + duckdb_destroy_error_data(&res_conv); + return ADBC_STATUS_INVALID_ARGUMENT; + } + if (!out_chunk.chunk) { + SetError(error, "Please provide a non-empty chunk to be bound"); + free(stream_wrapper); + return ADBC_STATUS_INVALID_ARGUMENT; + } + auto chunk = reinterpret_cast(out_chunk.chunk); if (chunk->size() == 0) { SetError(error, "Please provide a non-empty chunk to be bound"); + free(stream_wrapper); return ADBC_STATUS_INVALID_ARGUMENT; } if (chunk->size() != 1) { // TODO: add support for binding multiple rows SetError(error, "Binding multiple rows at once is not supported yet"); + free(stream_wrapper); return ADBC_STATUS_NOT_IMPLEMENTED; } if (chunk->ColumnCount() > prepared_statement_params) { SetError(error, "Input data has more column than prepared statement has parameters"); + free(stream_wrapper); return ADBC_STATUS_INVALID_ARGUMENT; } duckdb_clear_bindings(wrapper->statement); @@ -839,34 +927,35 @@ AdbcStatusCode StatementExecuteQuery(struct AdbcStatement *statement, struct Arr auto res = duckdb_bind_value(wrapper->statement, 1 + col_idx, duck_val); if (res != DuckDBSuccess) { SetError(error, duckdb_prepare_error(wrapper->statement)); + free(stream_wrapper); return ADBC_STATUS_INVALID_ARGUMENT; } } - - auto res = duckdb_execute_prepared_arrow(wrapper->statement, &wrapper->result); + auto res = duckdb_execute_prepared(wrapper->statement, &stream_wrapper->result); if (res != DuckDBSuccess) { - SetError(error, duckdb_query_arrow_error(wrapper->result)); + SetError(error, duckdb_result_error(&stream_wrapper->result)); + free(stream_wrapper); return ADBC_STATUS_INVALID_ARGUMENT; } + // Recreate wrappers for next iteration + arrow_array_wrapper = duckdb::ArrowArrayWrapper(); + stream.get_next(&stream, &arrow_array_wrapper.arrow_array); } } else { - auto res = duckdb_execute_prepared_arrow(wrapper->statement, &wrapper->result); + auto res = duckdb_execute_prepared(wrapper->statement, &stream_wrapper->result); if (res != DuckDBSuccess) { - SetError(error, duckdb_query_arrow_error(wrapper->result)); + SetError(error, duckdb_result_error(&stream_wrapper->result)); return ADBC_STATUS_INVALID_ARGUMENT; } } if (out) { - out->private_data = wrapper->result; + // We pass ownership of the statement private data to our stream + out->private_data = stream_wrapper; out->get_schema = get_schema; out->get_next = get_next; out->release = release; out->get_last_error = get_last_error; - - // because we handed out the stream pointer its no longer our responsibility to destroy it in - // AdbcStatementRelease, this is now done in release() - wrapper->result = nullptr; } return ADBC_STATUS_OK; diff --git a/src/duckdb/src/common/arrow/appender/append_data.cpp b/src/duckdb/src/common/arrow/appender/append_data.cpp new file mode 100644 index 000000000..06ccbc1ad --- /dev/null +++ b/src/duckdb/src/common/arrow/appender/append_data.cpp @@ -0,0 +1,29 @@ +#include "duckdb/common/arrow/appender/append_data.hpp" + +namespace duckdb { + +void ArrowAppendData::AppendValidity(UnifiedVectorFormat &format, idx_t from, idx_t to) { + // resize the buffer, filling the validity buffer with all valid values + idx_t size = to - from; + ResizeValidity(GetValidityBuffer(), row_count + size); + if (format.validity.AllValid()) { + // if all values are valid we don't need to do anything else + return; + } + + // otherwise we iterate through the validity mask + auto validity_data = (uint8_t *)GetValidityBuffer().data(); + uint8_t current_bit; + idx_t current_byte; + GetBitPosition(row_count, current_byte, current_bit); + for (idx_t i = from; i < to; i++) { + auto source_idx = format.sel->get_index(i); + // append the validity mask + if (!format.validity.RowIsValid(source_idx)) { + SetNull(validity_data, current_byte, current_bit); + } + NextBit(current_byte, current_bit); + } +} + +} // namespace duckdb diff --git a/src/duckdb/src/common/arrow/appender/bool_data.cpp b/src/duckdb/src/common/arrow/appender/bool_data.cpp index 78befb603..798b71e44 100644 --- a/src/duckdb/src/common/arrow/appender/bool_data.cpp +++ b/src/duckdb/src/common/arrow/appender/bool_data.cpp @@ -6,7 +6,6 @@ namespace duckdb { void ArrowBoolData::Initialize(ArrowAppendData &result, const LogicalType &type, idx_t capacity) { auto byte_count = (capacity + 7) / 8; result.GetMainBuffer().reserve(byte_count); - (void)AppendValidity; // silence a compiler warning about unused static function } void ArrowBoolData::Append(ArrowAppendData &append_data, Vector &input, idx_t from, idx_t to, idx_t input_size) { @@ -16,24 +15,24 @@ void ArrowBoolData::Append(ArrowAppendData &append_data, Vector &input, idx_t fr auto &main_buffer = append_data.GetMainBuffer(); auto &validity_buffer = append_data.GetValidityBuffer(); // we initialize both the validity and the bit set to 1's - ResizeValidity(validity_buffer, append_data.row_count + size); - ResizeValidity(main_buffer, append_data.row_count + size); + ArrowAppendData::ResizeValidity(validity_buffer, append_data.row_count + size); + ArrowAppendData::ResizeValidity(main_buffer, append_data.row_count + size); auto data = UnifiedVectorFormat::GetData(format); auto result_data = main_buffer.GetData(); auto validity_data = validity_buffer.GetData(); uint8_t current_bit; idx_t current_byte; - GetBitPosition(append_data.row_count, current_byte, current_bit); + ArrowAppendData::GetBitPosition(append_data.row_count, current_byte, current_bit); for (idx_t i = from; i < to; i++) { auto source_idx = format.sel->get_index(i); // append the validity mask if (!format.validity.RowIsValid(source_idx)) { - SetNull(append_data, validity_data, current_byte, current_bit); + append_data.SetNull(validity_data, current_byte, current_bit); } else if (!data[source_idx]) { - UnsetBit(result_data, current_byte, current_bit); + ArrowAppendData::UnsetBit(result_data, current_byte, current_bit); } - NextBit(current_byte, current_bit); + ArrowAppendData::NextBit(current_byte, current_bit); } append_data.row_count += size; } diff --git a/src/duckdb/src/common/arrow/appender/fixed_size_list_data.cpp b/src/duckdb/src/common/arrow/appender/fixed_size_list_data.cpp index 172144fd3..a8cbc16d9 100644 --- a/src/duckdb/src/common/arrow/appender/fixed_size_list_data.cpp +++ b/src/duckdb/src/common/arrow/appender/fixed_size_list_data.cpp @@ -18,7 +18,7 @@ void ArrowFixedSizeListData::Append(ArrowAppendData &append_data, Vector &input, UnifiedVectorFormat format; input.ToUnifiedFormat(input_size, format); idx_t size = to - from; - AppendValidity(append_data, format, from, to); + append_data.AppendValidity(format, from, to); input.Flatten(input_size); auto array_size = ArrayType::GetSize(input.GetType()); auto &child_vector = ArrayVector::GetEntry(input); diff --git a/src/duckdb/src/common/arrow/appender/struct_data.cpp b/src/duckdb/src/common/arrow/appender/struct_data.cpp index b2afa62d1..28cee72a9 100644 --- a/src/duckdb/src/common/arrow/appender/struct_data.cpp +++ b/src/duckdb/src/common/arrow/appender/struct_data.cpp @@ -18,7 +18,7 @@ void ArrowStructData::Append(ArrowAppendData &append_data, Vector &input, idx_t UnifiedVectorFormat format; input.ToUnifiedFormat(input_size, format); idx_t size = to - from; - AppendValidity(append_data, format, from, to); + append_data.AppendValidity(format, from, to); // append the children of the struct auto &children = StructVector::GetEntries(input); for (idx_t child_idx = 0; child_idx < children.size(); child_idx++) { diff --git a/src/duckdb/src/common/arrow/appender/union_data.cpp b/src/duckdb/src/common/arrow/appender/union_data.cpp index 1e9f4f432..4ca4ebf67 100644 --- a/src/duckdb/src/common/arrow/appender/union_data.cpp +++ b/src/duckdb/src/common/arrow/appender/union_data.cpp @@ -14,7 +14,6 @@ void ArrowUnionData::Initialize(ArrowAppendData &result, const LogicalType &type auto child_buffer = ArrowAppender::InitializeChild(child.second, capacity, result.options); result.child_data.push_back(std::move(child_buffer)); } - (void)AppendValidity; // silence a compiler warning about unused static functiondep } void ArrowUnionData::Append(ArrowAppendData &append_data, Vector &input, idx_t from, idx_t to, idx_t input_size) { diff --git a/src/duckdb/src/common/arrow/arrow_converter.cpp b/src/duckdb/src/common/arrow/arrow_converter.cpp index 8cded261e..e088dd6ec 100644 --- a/src/duckdb/src/common/arrow/arrow_converter.cpp +++ b/src/duckdb/src/common/arrow/arrow_converter.cpp @@ -39,6 +39,8 @@ static void ReleaseDuckDBArrowSchema(ArrowSchema *schema) { } schema->release = nullptr; auto holder = static_cast(schema->private_data); + schema->private_data = nullptr; + delete holder; } diff --git a/src/duckdb/src/common/enum_util.cpp b/src/duckdb/src/common/enum_util.cpp index 5257e9902..43161e340 100644 --- a/src/duckdb/src/common/enum_util.cpp +++ b/src/duckdb/src/common/enum_util.cpp @@ -481,6 +481,25 @@ AppenderType EnumUtil::FromString(const char *value) { return static_cast(StringUtil::StringToEnum(GetAppenderTypeValues(), 2, "AppenderType", value)); } +const StringUtil::EnumStringLiteral *GetArrowArrayPhysicalTypeValues() { + static constexpr StringUtil::EnumStringLiteral values[] { + { static_cast(ArrowArrayPhysicalType::DICTIONARY_ENCODED), "DICTIONARY_ENCODED" }, + { static_cast(ArrowArrayPhysicalType::RUN_END_ENCODED), "RUN_END_ENCODED" }, + { static_cast(ArrowArrayPhysicalType::DEFAULT), "DEFAULT" } + }; + return values; +} + +template<> +const char* EnumUtil::ToChars(ArrowArrayPhysicalType value) { + return StringUtil::EnumToString(GetArrowArrayPhysicalTypeValues(), 3, "ArrowArrayPhysicalType", static_cast(value)); +} + +template<> +ArrowArrayPhysicalType EnumUtil::FromString(const char *value) { + return static_cast(StringUtil::StringToEnum(GetArrowArrayPhysicalTypeValues(), 3, "ArrowArrayPhysicalType", value)); +} + const StringUtil::EnumStringLiteral *GetArrowDateTimeTypeValues() { static constexpr StringUtil::EnumStringLiteral values[] { { static_cast(ArrowDateTimeType::MILLISECONDS), "MILLISECONDS" }, @@ -3839,19 +3858,20 @@ SettingScope EnumUtil::FromString(const char *value) { const StringUtil::EnumStringLiteral *GetShowTypeValues() { static constexpr StringUtil::EnumStringLiteral values[] { { static_cast(ShowType::SUMMARY), "SUMMARY" }, - { static_cast(ShowType::DESCRIBE), "DESCRIBE" } + { static_cast(ShowType::DESCRIBE), "DESCRIBE" }, + { static_cast(ShowType::SHOW_FROM), "SHOW_FROM" } }; return values; } template<> const char* EnumUtil::ToChars(ShowType value) { - return StringUtil::EnumToString(GetShowTypeValues(), 2, "ShowType", static_cast(value)); + return StringUtil::EnumToString(GetShowTypeValues(), 3, "ShowType", static_cast(value)); } template<> ShowType EnumUtil::FromString(const char *value) { - return static_cast(StringUtil::StringToEnum(GetShowTypeValues(), 2, "ShowType", value)); + return static_cast(StringUtil::StringToEnum(GetShowTypeValues(), 3, "ShowType", value)); } const StringUtil::EnumStringLiteral *GetSimplifiedTokenTypeValues() { diff --git a/src/duckdb/src/common/exception_format_value.cpp b/src/duckdb/src/common/exception_format_value.cpp index ddef4e10c..a77ab7f38 100644 --- a/src/duckdb/src/common/exception_format_value.cpp +++ b/src/duckdb/src/common/exception_format_value.cpp @@ -15,6 +15,9 @@ ExceptionFormatValue::ExceptionFormatValue(double dbl_val) ExceptionFormatValue::ExceptionFormatValue(int64_t int_val) : type(ExceptionFormatValueType::FORMAT_VALUE_TYPE_INTEGER), int_val(int_val) { } +ExceptionFormatValue::ExceptionFormatValue(idx_t uint_val) + : type(ExceptionFormatValueType::FORMAT_VALUE_TYPE_INTEGER), int_val(Hugeint::Convert(uint_val)) { +} ExceptionFormatValue::ExceptionFormatValue(hugeint_t huge_val) : type(ExceptionFormatValueType::FORMAT_VALUE_TYPE_STRING), str_val(Hugeint::ToString(huge_val)) { } @@ -68,6 +71,10 @@ ExceptionFormatValue ExceptionFormatValue::CreateFormatValue(char *value) { return ExceptionFormatValue(string(value)); } template <> +ExceptionFormatValue ExceptionFormatValue::CreateFormatValue(idx_t value) { + return ExceptionFormatValue(value); +} +template <> ExceptionFormatValue ExceptionFormatValue::CreateFormatValue(hugeint_t value) { return ExceptionFormatValue(value); } diff --git a/src/duckdb/src/common/local_file_system.cpp b/src/duckdb/src/common/local_file_system.cpp index f1672aa8d..b754ae0ba 100644 --- a/src/duckdb/src/common/local_file_system.cpp +++ b/src/duckdb/src/common/local_file_system.cpp @@ -589,7 +589,7 @@ timestamp_t LocalFileSystem::GetLastModifiedTime(FileHandle &handle) { throw IOException("Failed to get last modified time for file \"%s\": %s", {{"errno", std::to_string(errno)}}, handle.path, strerror(errno)); } - return Timestamp::FromTimeT(s.st_mtime); + return Timestamp::FromEpochSeconds(s.st_mtime); } FileType LocalFileSystem::GetFileType(FileHandle &handle) { diff --git a/src/duckdb/src/common/types/vector.cpp b/src/duckdb/src/common/types/vector.cpp index 4edfab4d9..8f82d5fc6 100644 --- a/src/duckdb/src/common/types/vector.cpp +++ b/src/duckdb/src/common/types/vector.cpp @@ -147,10 +147,9 @@ void Vector::ReferenceAndSetType(const Vector &other) { void Vector::Reinterpret(const Vector &other) { vector_type = other.vector_type; -#ifdef DEBUG auto &this_type = GetType(); auto &other_type = other.GetType(); - +#ifdef DEBUG auto type_is_same = other_type == this_type; bool this_is_nested = this_type.IsNested(); bool other_is_nested = other_type.IsNested(); @@ -163,7 +162,7 @@ void Vector::Reinterpret(const Vector &other) { D_ASSERT((not_nested && type_size_equal) || type_is_same); #endif AssignSharedPointer(buffer, other.buffer); - if (vector_type == VectorType::DICTIONARY_VECTOR) { + if (vector_type == VectorType::DICTIONARY_VECTOR && other_type != this_type) { Vector new_vector(GetType(), nullptr); new_vector.Reinterpret(DictionaryVector::Child(other)); auxiliary = make_shared_ptr(std::move(new_vector)); @@ -590,7 +589,7 @@ Value Vector::GetValueInternal(const Vector &v_p, idx_t index_p) { int64_t start, increment; SequenceVector::GetSequence(*vector, start, increment); return Value::Numeric(vector->GetType(), - static_cast(start + static_cast(increment) * index)); + start + static_cast(static_cast(increment) * index)); } default: throw InternalException("Unimplemented vector type for Vector::GetValue"); @@ -840,7 +839,7 @@ string Vector::ToString(idx_t count) const { int64_t start, increment; SequenceVector::GetSequence(*this, start, increment); for (idx_t i = 0; i < count; i++) { - retval += to_string(static_cast(start + static_cast(increment) * i)) + + retval += to_string(start + static_cast(static_cast(increment) * i)) + (i == count - 1 ? "" : ", "); } break; diff --git a/src/duckdb/src/common/value_operations/comparison_operations.cpp b/src/duckdb/src/common/value_operations/comparison_operations.cpp index 78dd5244e..455b6a72a 100644 --- a/src/duckdb/src/common/value_operations/comparison_operations.cpp +++ b/src/duckdb/src/common/value_operations/comparison_operations.cpp @@ -65,38 +65,11 @@ inline bool ValuePositionComparator::Final(const Value &lhs, return ValueOperations::NotDistinctFrom(lhs, rhs); } -// Non-strict inequalities must use strict comparisons for Definite -template <> -bool ValuePositionComparator::Definite(const Value &lhs, const Value &rhs) { - return !ValuePositionComparator::Definite(lhs, rhs); -} - template <> bool ValuePositionComparator::Final(const Value &lhs, const Value &rhs) { return ValueOperations::DistinctGreaterThan(lhs, rhs); } -template <> -bool ValuePositionComparator::Final(const Value &lhs, const Value &rhs) { - return !ValuePositionComparator::Final(lhs, rhs); -} - -template <> -bool ValuePositionComparator::Definite(const Value &lhs, const Value &rhs) { - return !ValuePositionComparator::Definite(rhs, lhs); -} - -template <> -bool ValuePositionComparator::Final(const Value &lhs, const Value &rhs) { - return !ValuePositionComparator::Final(rhs, lhs); -} - -// Strict inequalities just use strict for both Definite and Final -template <> -bool ValuePositionComparator::Final(const Value &lhs, const Value &rhs) { - return ValuePositionComparator::Final(rhs, lhs); -} - template static bool TemplatedBooleanOperation(const Value &left, const Value &right) { const auto &left_type = left.type(); diff --git a/src/duckdb/src/common/vector_operations/is_distinct_from.cpp b/src/duckdb/src/common/vector_operations/is_distinct_from.cpp index 054ef2173..e57f9738d 100644 --- a/src/duckdb/src/common/vector_operations/is_distinct_from.cpp +++ b/src/duckdb/src/common/vector_operations/is_distinct_from.cpp @@ -446,61 +446,6 @@ idx_t PositionComparator::Final(Vector &left, Vector &righ return VectorOperations::NestedNotEquals(left, right, &sel, count, true_sel, false_sel, null_mask); } -// Non-strict inequalities must use strict comparisons for Definite -template <> -idx_t PositionComparator::Definite(Vector &left, Vector &right, - const SelectionVector &sel, idx_t count, - optional_ptr true_sel, - SelectionVector &false_sel, - optional_ptr null_mask) { - return VectorOperations::DistinctGreaterThan(right, left, &sel, count, true_sel, &false_sel, null_mask); -} - -template <> -idx_t PositionComparator::Final(Vector &left, Vector &right, const SelectionVector &sel, - idx_t count, optional_ptr true_sel, - optional_ptr false_sel, - optional_ptr null_mask) { - return VectorOperations::DistinctGreaterThanEquals(right, left, &sel, count, true_sel, false_sel, null_mask); -} - -template <> -idx_t PositionComparator::Definite(Vector &left, Vector &right, - const SelectionVector &sel, idx_t count, - optional_ptr true_sel, - SelectionVector &false_sel, - optional_ptr null_mask) { - return VectorOperations::DistinctGreaterThan(left, right, &sel, count, true_sel, &false_sel, null_mask); -} - -template <> -idx_t PositionComparator::Final(Vector &left, Vector &right, - const SelectionVector &sel, idx_t count, - optional_ptr true_sel, - optional_ptr false_sel, - optional_ptr null_mask) { - return VectorOperations::DistinctGreaterThanEquals(left, right, &sel, count, true_sel, false_sel, null_mask); -} - -// Strict inequalities just use strict for both Definite and Final -template <> -idx_t PositionComparator::Final(Vector &left, Vector &right, const SelectionVector &sel, - idx_t count, optional_ptr true_sel, - optional_ptr false_sel, - optional_ptr null_mask) { - return VectorOperations::DistinctGreaterThan(right, left, &sel, count, true_sel, false_sel, null_mask); -} - -template <> -idx_t PositionComparator::Final(Vector &left, Vector &right, - const SelectionVector &sel, idx_t count, - optional_ptr true_sel, - optional_ptr false_sel, - optional_ptr null_mask) { - // DistinctGreaterThan has NULLs last - return VectorOperations::DistinctGreaterThan(right, left, &sel, count, true_sel, false_sel, null_mask); -} - template <> idx_t PositionComparator::Final(Vector &left, Vector &right, const SelectionVector &sel, idx_t count, optional_ptr true_sel, diff --git a/src/duckdb/src/execution/index/art/art.cpp b/src/duckdb/src/execution/index/art/art.cpp index 6d1136b22..ac095991d 100644 --- a/src/duckdb/src/execution/index/art/art.cpp +++ b/src/duckdb/src/execution/index/art/art.cpp @@ -35,7 +35,7 @@ struct ARTIndexScanState : public IndexScanState { ExpressionType expressions[2]; bool checked = false; //! All scanned row IDs. - unsafe_vector row_ids; + set row_ids; }; //===--------------------------------------------------------------------===// @@ -207,6 +207,8 @@ unique_ptr ART::TryInitializeScan(const Expression &expr, const high_comparison_type = between.upper_inclusive ? ExpressionType::COMPARE_LESSTHANOREQUALTO : ExpressionType::COMPARE_LESSTHAN; } + // FIXME: add another if...else... to match rewritten BETWEEN, + // i.e., WHERE i BETWEEN 50 AND 1502 is rewritten to CONJUNCTION_AND. // We cannot use an index scan. if (equal_value.IsNull() && low_value.IsNull() && high_value.IsNull()) { @@ -495,7 +497,7 @@ bool ART::Construct(unsafe_vector &keys, unsafe_vector &row_ids, } #ifdef DEBUG - unsafe_vector row_ids_debug; + set row_ids_debug; Iterator it(*this); it.FindMinimum(tree); ARTKey empty_key = ARTKey(); @@ -761,7 +763,7 @@ void ART::Erase(Node &node, reference key, idx_t depth, reference< // Point and range lookups //===--------------------------------------------------------------------===// -bool ART::SearchEqual(ARTKey &key, idx_t max_count, unsafe_vector &row_ids) { +bool ART::SearchEqual(ARTKey &key, idx_t max_count, set &row_ids) { auto leaf = ARTOperator::Lookup(*this, tree, key, 0); if (!leaf) { return true; @@ -773,7 +775,7 @@ bool ART::SearchEqual(ARTKey &key, idx_t max_count, unsafe_vector &row_id return it.Scan(empty_key, max_count, row_ids, false); } -bool ART::SearchGreater(ARTKey &key, bool equal, idx_t max_count, unsafe_vector &row_ids) { +bool ART::SearchGreater(ARTKey &key, bool equal, idx_t max_count, set &row_ids) { if (!tree.HasMetadata()) { return true; } @@ -791,7 +793,7 @@ bool ART::SearchGreater(ARTKey &key, bool equal, idx_t max_count, unsafe_vector< return it.Scan(ARTKey(), max_count, row_ids, false); } -bool ART::SearchLess(ARTKey &upper_bound, bool equal, idx_t max_count, unsafe_vector &row_ids) { +bool ART::SearchLess(ARTKey &upper_bound, bool equal, idx_t max_count, set &row_ids) { if (!tree.HasMetadata()) { return true; } @@ -810,7 +812,7 @@ bool ART::SearchLess(ARTKey &upper_bound, bool equal, idx_t max_count, unsafe_ve } bool ART::SearchCloseRange(ARTKey &lower_bound, ARTKey &upper_bound, bool left_equal, bool right_equal, idx_t max_count, - unsafe_vector &row_ids) { + set &row_ids) { // Find the first node that satisfies the left predicate. Iterator it(*this); @@ -823,7 +825,7 @@ bool ART::SearchCloseRange(ARTKey &lower_bound, ARTKey &upper_bound, bool left_e return it.Scan(upper_bound, max_count, row_ids, right_equal); } -bool ART::Scan(IndexScanState &state, const idx_t max_count, unsafe_vector &row_ids) { +bool ART::Scan(IndexScanState &state, const idx_t max_count, set &row_ids) { auto &scan_state = state.Cast(); D_ASSERT(scan_state.values[0].type().InternalType() == types[0]); ArenaAllocator arena_allocator(Allocator::Get(db)); @@ -962,27 +964,27 @@ void ART::VerifyLeaf(const Node &leaf, const ARTKey &key, optional_ptr dele Iterator it(*this); it.FindMinimum(leaf); ARTKey empty_key = ARTKey(); - unsafe_vector row_ids; + set row_ids; auto success = it.Scan(empty_key, 2, row_ids, false); if (!success || row_ids.size() != 2) { throw InternalException("VerifyLeaf expects exactly two row IDs to be scanned"); } - if (!deleted_leaf) { - if (manager.AddHit(i, row_ids[0]) || manager.AddSecondHit(i, row_ids[1])) { - conflict_idx = i; + if (deleted_leaf) { + auto deleted_row_id = deleted_leaf->GetRowId(); + for (const auto row_id : row_ids) { + if (deleted_row_id == row_id) { + return; + } } - return; - } - - auto deleted_row_id = deleted_leaf->GetRowId(); - if (deleted_row_id == row_ids[0] || deleted_row_id == row_ids[1]) { - return; } - if (manager.AddHit(i, row_ids[0]) || manager.AddSecondHit(i, row_ids[1])) { + auto row_id_it = row_ids.begin(); + if (manager.AddHit(i, *row_id_it)) { conflict_idx = i; } + row_id_it++; + manager.AddSecondHit(i, *row_id_it); } void ART::VerifyConstraint(DataChunk &chunk, IndexAppendInfo &info, ConflictManager &manager) { diff --git a/src/duckdb/src/execution/index/art/iterator.cpp b/src/duckdb/src/execution/index/art/iterator.cpp index 1c138e1d3..1a88b7262 100644 --- a/src/duckdb/src/execution/index/art/iterator.cpp +++ b/src/duckdb/src/execution/index/art/iterator.cpp @@ -42,7 +42,7 @@ bool IteratorKey::GreaterThan(const ARTKey &key, const bool equal, const uint8_t // Iterator //===--------------------------------------------------------------------===// -bool Iterator::Scan(const ARTKey &upper_bound, const idx_t max_count, unsafe_vector &row_ids, const bool equal) { +bool Iterator::Scan(const ARTKey &upper_bound, const idx_t max_count, set &row_ids, const bool equal) { bool has_next; do { // An empty upper bound indicates that no upper bound exists. @@ -59,7 +59,7 @@ bool Iterator::Scan(const ARTKey &upper_bound, const idx_t max_count, unsafe_vec if (row_ids.size() + 1 > max_count) { return false; } - row_ids.push_back(last_leaf.GetRowId()); + row_ids.insert(last_leaf.GetRowId()); break; case NType::LEAF: if (!Leaf::DeprecatedGetRowIds(art, last_leaf, row_ids, max_count)) { @@ -76,7 +76,7 @@ bool Iterator::Scan(const ARTKey &upper_bound, const idx_t max_count, unsafe_vec } row_id[ROW_ID_SIZE - 1] = byte; ARTKey key(&row_id[0], ROW_ID_SIZE); - row_ids.push_back(key.GetRowId()); + row_ids.insert(key.GetRowId()); if (byte == NumericLimits::Maximum()) { break; } diff --git a/src/duckdb/src/execution/index/art/leaf.cpp b/src/duckdb/src/execution/index/art/leaf.cpp index 949318ce5..da2502c41 100644 --- a/src/duckdb/src/execution/index/art/leaf.cpp +++ b/src/duckdb/src/execution/index/art/leaf.cpp @@ -111,7 +111,7 @@ void Leaf::TransformToDeprecated(ART &art, Node &node) { } // Collect all row IDs and free the nested leaf. - unsafe_vector row_ids; + set row_ids; Iterator it(art); it.FindMinimum(node); ARTKey empty_key = ARTKey(); @@ -121,7 +121,7 @@ void Leaf::TransformToDeprecated(ART &art, Node &node) { // Create the deprecated leaves. idx_t remaining = row_ids.size(); - idx_t copy_count = 0; + auto row_ids_it = row_ids.begin(); reference ref(node); while (remaining) { ref.get() = Node::GetAllocator(art, LEAF).New(); @@ -132,10 +132,9 @@ void Leaf::TransformToDeprecated(ART &art, Node &node) { leaf.count = UnsafeNumericCast(min); for (uint8_t i = 0; i < leaf.count; i++) { - leaf.row_ids[i] = row_ids[copy_count + i]; + leaf.row_ids[i] = *row_ids_it; + row_ids_it++; } - - copy_count += leaf.count; remaining -= leaf.count; ref = leaf.ptr; @@ -159,7 +158,7 @@ void Leaf::DeprecatedFree(ART &art, Node &node) { node.Clear(); } -bool Leaf::DeprecatedGetRowIds(ART &art, const Node &node, unsafe_vector &row_ids, const idx_t max_count) { +bool Leaf::DeprecatedGetRowIds(ART &art, const Node &node, set &row_ids, const idx_t max_count) { D_ASSERT(node.GetType() == LEAF); reference ref(node); @@ -170,7 +169,7 @@ bool Leaf::DeprecatedGetRowIds(ART &art, const Node &node, unsafe_vector return false; } for (uint8_t i = 0; i < leaf.count; i++) { - row_ids.push_back(leaf.row_ids[i]); + row_ids.insert(leaf.row_ids[i]); } ref = leaf.ptr; } diff --git a/src/duckdb/src/execution/operator/join/physical_iejoin.cpp b/src/duckdb/src/execution/operator/join/physical_iejoin.cpp index ad9896f66..90aba4722 100644 --- a/src/duckdb/src/execution/operator/join/physical_iejoin.cpp +++ b/src/duckdb/src/execution/operator/join/physical_iejoin.cpp @@ -23,9 +23,7 @@ PhysicalIEJoin::PhysicalIEJoin(PhysicalPlan &physical_plan, LogicalComparisonJoi PhysicalOperator &right, vector cond, JoinType join_type, idx_t estimated_cardinality, unique_ptr pushdown_info) : PhysicalRangeJoin(physical_plan, op, PhysicalOperatorType::IE_JOIN, left, right, std::move(cond), join_type, - estimated_cardinality) { - - filter_pushdown = std::move(pushdown_info); + estimated_cardinality, std::move(pushdown_info)) { // 1. let L1 (resp. L2) be the array of column X (resp. Y) D_ASSERT(conditions.size() >= 2); diff --git a/src/duckdb/src/execution/operator/join/physical_piecewise_merge_join.cpp b/src/duckdb/src/execution/operator/join/physical_piecewise_merge_join.cpp index 437fa1175..25c5494ee 100644 --- a/src/duckdb/src/execution/operator/join/physical_piecewise_merge_join.cpp +++ b/src/duckdb/src/execution/operator/join/physical_piecewise_merge_join.cpp @@ -20,9 +20,7 @@ PhysicalPiecewiseMergeJoin::PhysicalPiecewiseMergeJoin(PhysicalPlan &physical_pl idx_t estimated_cardinality, unique_ptr pushdown_info_p) : PhysicalRangeJoin(physical_plan, op, PhysicalOperatorType::PIECEWISE_MERGE_JOIN, left, right, std::move(cond), - join_type, estimated_cardinality) { - - filter_pushdown = std::move(pushdown_info_p); + join_type, estimated_cardinality, std::move(pushdown_info_p)) { for (auto &join_cond : conditions) { D_ASSERT(join_cond.left->return_type == join_cond.right->return_type); diff --git a/src/duckdb/src/execution/operator/join/physical_range_join.cpp b/src/duckdb/src/execution/operator/join/physical_range_join.cpp index a11a402ee..0cb5c79c1 100644 --- a/src/duckdb/src/execution/operator/join/physical_range_join.cpp +++ b/src/duckdb/src/execution/operator/join/physical_range_join.cpp @@ -7,6 +7,7 @@ #include "duckdb/common/sort/sort.hpp" #include "duckdb/common/types/validity_mask.hpp" #include "duckdb/common/types/vector.hpp" +#include "duckdb/common/unordered_map.hpp" #include "duckdb/common/vector_operations/vector_operations.hpp" #include "duckdb/execution/expression_executor.hpp" #include "duckdb/main/client_context.hpp" @@ -167,12 +168,15 @@ void PhysicalRangeJoin::GlobalSortedTable::Finalize(Pipeline &pipeline, Event &e PhysicalRangeJoin::PhysicalRangeJoin(PhysicalPlan &physical_plan, LogicalComparisonJoin &op, PhysicalOperatorType type, PhysicalOperator &left, PhysicalOperator &right, vector cond, - JoinType join_type, idx_t estimated_cardinality) + JoinType join_type, idx_t estimated_cardinality, + unique_ptr pushdown_info) : PhysicalComparisonJoin(physical_plan, op, type, std::move(cond), join_type, estimated_cardinality) { + filter_pushdown = std::move(pushdown_info); // Reorder the conditions so that ranges are at the front. // TODO: use stats to improve the choice? // TODO: Prefer fixed length types? if (conditions.size() > 1) { + unordered_map cond_idx; vector conditions_p(conditions.size()); std::swap(conditions_p, conditions); idx_t range_position = 0; @@ -184,12 +188,21 @@ PhysicalRangeJoin::PhysicalRangeJoin(PhysicalPlan &physical_plan, LogicalCompari case ExpressionType::COMPARE_GREATERTHAN: case ExpressionType::COMPARE_GREATERTHANOREQUALTO: conditions[range_position++] = std::move(conditions_p[i]); + cond_idx[i] = range_position - 1; break; default: conditions[--other_position] = std::move(conditions_p[i]); + cond_idx[i] = other_position; break; } } + if (filter_pushdown) { + for (auto &idx : filter_pushdown->join_condition) { + if (cond_idx.find(idx) != cond_idx.end()) { + idx = cond_idx[idx]; + } + } + } } children.push_back(left); diff --git a/src/duckdb/src/execution/operator/persistent/physical_merge_into.cpp b/src/duckdb/src/execution/operator/persistent/physical_merge_into.cpp index 4d568ff51..04a5f3dca 100644 --- a/src/duckdb/src/execution/operator/persistent/physical_merge_into.cpp +++ b/src/duckdb/src/execution/operator/persistent/physical_merge_into.cpp @@ -6,9 +6,10 @@ namespace duckdb { PhysicalMergeInto::PhysicalMergeInto(PhysicalPlan &physical_plan, vector types, map>> actions_p, - idx_t row_id_index, optional_idx source_marker, bool parallel_p) + idx_t row_id_index, optional_idx source_marker, bool parallel_p, + bool return_chunk_p) : PhysicalOperator(physical_plan, PhysicalOperatorType::MERGE_INTO, std::move(types), 1), - row_id_index(row_id_index), source_marker(source_marker), parallel(parallel_p) { + row_id_index(row_id_index), source_marker(source_marker), parallel(parallel_p), return_chunk(return_chunk_p) { map ranges; for (auto &entry : actions_p) { @@ -396,15 +397,119 @@ SinkFinalizeType PhysicalMergeInto::Finalize(Pipeline &pipeline, Event &event, C //===--------------------------------------------------------------------===// // Source //===--------------------------------------------------------------------===// +class MergeGlobalSourceState : public GlobalSourceState { +public: + explicit MergeGlobalSourceState(ClientContext &context, const PhysicalMergeInto &op) { + if (!op.return_chunk) { + return; + } + auto &g = op.sink_state->Cast(); + for (idx_t i = 0; i < op.actions.size(); i++) { + auto &action = *op.actions[i]; + unique_ptr global_state; + if (action.op) { + // assign the global sink state + action.op->sink_state = std::move(g.sink_states[i]); + // initialize the global source state + global_state = action.op->GetGlobalSourceState(context); + } + global_states.push_back(std::move(global_state)); + } + } + + vector> global_states; +}; + +class MergeLocalSourceState : public LocalSourceState { +public: + explicit MergeLocalSourceState(ExecutionContext &context, const PhysicalMergeInto &op, + MergeGlobalSourceState &gstate) { + if (!op.return_chunk) { + return; + } + for (idx_t i = 0; i < op.actions.size(); i++) { + auto &action = *op.actions[i]; + unique_ptr local_state; + if (action.op) { + local_state = action.op->GetLocalSourceState(context, *gstate.global_states[i]); + } + local_states.push_back(std::move(local_state)); + } + vector scan_types; + for (idx_t c = 0; c < op.types.size() - 1; c++) { + scan_types.emplace_back(op.types[c]); + } + scan_chunk.Initialize(context.client, scan_types); + } + + DataChunk scan_chunk; + vector> local_states; + idx_t index = 0; +}; + unique_ptr PhysicalMergeInto::GetGlobalSourceState(ClientContext &context) const { - return make_uniq(); + return make_uniq(context, *this); +} + +unique_ptr PhysicalMergeInto::GetLocalSourceState(ExecutionContext &context, + GlobalSourceState &gstate) const { + return make_uniq(context, *this, gstate.Cast()); } SourceResultType PhysicalMergeInto::GetData(ExecutionContext &context, DataChunk &chunk, OperatorSourceInput &input) const { auto &g = sink_state->Cast(); - chunk.SetCardinality(1); - chunk.SetValue(0, 0, Value::BIGINT(NumericCast(g.merged_count.load()))); + if (!return_chunk) { + chunk.SetCardinality(1); + chunk.SetValue(0, 0, Value::BIGINT(NumericCast(g.merged_count.load()))); + return SourceResultType::FINISHED; + } + auto &gstate = input.global_state.Cast(); + auto &lstate = input.local_state.Cast(); + chunk.Reset(); + for (; lstate.index < actions.size(); lstate.index++) { + auto &action = *actions[lstate.index]; + if (!action.op) { + // no action to scan from + continue; + } + auto &child_gstate = *gstate.global_states[lstate.index]; + auto &child_lstate = *lstate.local_states[lstate.index]; + OperatorSourceInput source_input {child_gstate, child_lstate, input.interrupt_state}; + + auto result = action.op->GetData(context, lstate.scan_chunk, source_input); + if (lstate.scan_chunk.size() > 0) { + // construct the result chunk + for (idx_t c = 0; c < lstate.scan_chunk.ColumnCount(); c++) { + chunk.data[c].Reference(lstate.scan_chunk.data[c]); + } + // set the merge action + string merge_action_name; + switch (action.action_type) { + case MergeActionType::MERGE_UPDATE: + merge_action_name = "UPDATE"; + break; + case MergeActionType::MERGE_INSERT: + merge_action_name = "INSERT"; + break; + case MergeActionType::MERGE_DELETE: + merge_action_name = "DELETE"; + break; + default: + throw InternalException("Unsupported merge action for RETURNING"); + } + Value merge_action(merge_action_name); + chunk.data.back().Reference(merge_action); + chunk.SetCardinality(lstate.scan_chunk.size()); + } + + if (result != SourceResultType::FINISHED) { + return result; + } + if (chunk.size() != 0) { + return SourceResultType::HAVE_MORE_OUTPUT; + } + } return SourceResultType::FINISHED; } diff --git a/src/duckdb/src/execution/operator/schema/physical_attach.cpp b/src/duckdb/src/execution/operator/schema/physical_attach.cpp index d871f9ef5..0544aa032 100644 --- a/src/duckdb/src/execution/operator/schema/physical_attach.cpp +++ b/src/duckdb/src/execution/operator/schema/physical_attach.cpp @@ -17,7 +17,7 @@ SourceResultType PhysicalAttach::GetData(ExecutionContext &context, DataChunk &c OperatorSourceInput &input) const { // parse the options auto &config = DBConfig::GetConfig(context.client); - AttachOptions options(info, config.options.access_mode); + AttachOptions options(info->options, config.options.access_mode); // get the name and path of the database auto &name = info->name; @@ -67,8 +67,7 @@ SourceResultType PhysicalAttach::GetData(ExecutionContext &context, DataChunk &c auto attached_db = db_manager.AttachDatabase(context.client, *info, options); //! Initialize the database. - const auto storage_options = info->GetStorageOptions(); - attached_db->Initialize(context.client, storage_options); + attached_db->Initialize(context.client); if (!options.default_table.name.empty()) { attached_db->GetCatalog().SetDefaultTable(options.default_table.schema, options.default_table.name); } diff --git a/src/duckdb/src/execution/physical_plan/plan_merge_into.cpp b/src/duckdb/src/execution/physical_plan/plan_merge_into.cpp index 80edab918..40a0c6ec6 100644 --- a/src/duckdb/src/execution/physical_plan/plan_merge_into.cpp +++ b/src/duckdb/src/execution/physical_plan/plan_merge_into.cpp @@ -19,23 +19,31 @@ unique_ptr PlanMergeIntoAction(ClientContext &context, Logica for (auto &constraint : op.bound_constraints) { bound_constraints.push_back(constraint->Copy()); } + auto return_types = op.types; + if (op.return_chunk) { + // for RETURNING, the last column is the merge_action - this is added in the merge itself + return_types.pop_back(); + } + auto cardinality = op.EstimateCardinality(context); switch (action.action_type) { case MergeActionType::MERGE_UPDATE: { vector> defaults; for (auto &def : op.bound_defaults) { defaults.push_back(def->Copy()); } - result->op = planner.Make(op.types, op.table, op.table.GetStorage(), std::move(action.columns), - std::move(action.expressions), std::move(defaults), - std::move(bound_constraints), 1ULL, false); + result->op = + planner.Make(std::move(return_types), op.table, op.table.GetStorage(), + std::move(action.columns), std::move(action.expressions), std::move(defaults), + std::move(bound_constraints), cardinality, op.return_chunk); auto &cast_update = result->op->Cast(); cast_update.update_is_del_and_insert = action.update_is_del_and_insert; break; } case MergeActionType::MERGE_DELETE: { - result->op = planner.Make(op.types, op.table, op.table.GetStorage(), - std::move(bound_constraints), op.row_id_start, 1ULL, false); + result->op = + planner.Make(std::move(return_types), op.table, op.table.GetStorage(), + std::move(bound_constraints), op.row_id_start, cardinality, op.return_chunk); break; } case MergeActionType::MERGE_INSERT: { @@ -45,10 +53,11 @@ unique_ptr PlanMergeIntoAction(ClientContext &context, Logica unordered_set on_conflict_filter; vector columns_to_fetch; - result->op = planner.Make( - op.types, op.table, std::move(bound_constraints), std::move(set_expressions), std::move(set_columns), - std::move(set_types), 1ULL, false, true, OnConflictAction::THROW, nullptr, nullptr, - std::move(on_conflict_filter), std::move(columns_to_fetch), false); + result->op = planner.Make(std::move(return_types), op.table, std::move(bound_constraints), + std::move(set_expressions), std::move(set_columns), + std::move(set_types), cardinality, op.return_chunk, !op.return_chunk, + OnConflictAction::THROW, nullptr, nullptr, + std::move(on_conflict_filter), std::move(columns_to_fetch), false); // transform expressions if required if (!action.column_index_map.empty()) { vector> new_expressions; @@ -100,10 +109,10 @@ PhysicalOperator &DuckCatalog::PlanMergeInto(ClientContext &context, PhysicalPla actions.emplace(entry.first, std::move(planned_actions)); } - bool parallel = append_count <= 1; + bool parallel = append_count <= 1 && !op.return_chunk; - auto &result = - planner.Make(op.types, std::move(actions), op.row_id_start, op.source_marker, parallel); + auto &result = planner.Make(op.types, std::move(actions), op.row_id_start, op.source_marker, + parallel, op.return_chunk); result.children.push_back(plan); return result; } diff --git a/src/duckdb/src/function/function_list.cpp b/src/duckdb/src/function/function_list.cpp index 08ec07a24..ad1a3185a 100644 --- a/src/duckdb/src/function/function_list.cpp +++ b/src/duckdb/src/function/function_list.cpp @@ -53,6 +53,7 @@ static const StaticFunctionDefinition function[] = { DUCKDB_SCALAR_FUNCTION_SET(InternalCompressIntegralUintegerFun), DUCKDB_SCALAR_FUNCTION_SET(InternalCompressIntegralUsmallintFun), DUCKDB_SCALAR_FUNCTION_SET(InternalCompressIntegralUtinyintFun), + DUCKDB_SCALAR_FUNCTION(InternalCompressStringHugeintFun), DUCKDB_SCALAR_FUNCTION(InternalCompressStringUbigintFun), DUCKDB_SCALAR_FUNCTION(InternalCompressStringUhugeintFun), DUCKDB_SCALAR_FUNCTION(InternalCompressStringUintegerFun), diff --git a/src/duckdb/src/function/pragma/pragma_queries.cpp b/src/duckdb/src/function/pragma/pragma_queries.cpp index 921395a03..9107b8c01 100644 --- a/src/duckdb/src/function/pragma/pragma_queries.cpp +++ b/src/duckdb/src/function/pragma/pragma_queries.cpp @@ -19,19 +19,32 @@ static string PragmaTableInfo(ClientContext &context, const FunctionParameters & KeywordHelper::WriteQuoted(parameters.values[0].ToString(), '\'')); } -string PragmaShowTables() { +string PragmaShowTables(const string &database, const string &schema) { + string where_clause = ""; + vector where_conditions; + if (!database.empty()) { + where_conditions.push_back(StringUtil::Format("lower(database_name) = lower(%s)", SQLString(database))); + } + if (!schema.empty()) { + where_conditions.push_back(StringUtil::Format("lower(schema_name) = lower(%s)", SQLString(schema))); + } + if (where_conditions.empty()) { + where_conditions.push_back("in_search_path(database_name, schema_name)"); + } + where_clause = "WHERE " + StringUtil::Join(where_conditions, " AND "); + // clang-format off - return R"EOF( + string query = R"EOF( with "tables" as ( SELECT table_name as "name" FROM duckdb_tables - where in_search_path(database_name, schema_name) + )EOF" + where_clause + R"EOF( ), "views" as ( SELECT view_name as "name" FROM duckdb_views - where in_search_path(database_name, schema_name) + )EOF" + where_clause + R"EOF( ), db_objects as ( SELECT "name" FROM "tables" @@ -41,6 +54,8 @@ string PragmaShowTables() { SELECT "name" FROM db_objects ORDER BY "name";)EOF"; + + return query; // clang-format on } diff --git a/src/duckdb/src/function/scalar/compressed_materialization/compress_integral.cpp b/src/duckdb/src/function/scalar/compressed_materialization/compress_integral.cpp index 21c71a5e3..740924397 100644 --- a/src/duckdb/src/function/scalar/compressed_materialization/compress_integral.cpp +++ b/src/duckdb/src/function/scalar/compressed_materialization/compress_integral.cpp @@ -228,6 +228,11 @@ ScalarFunction CMIntegralCompressFun::GetFunction(const LogicalType &input_type, GetIntegralCompressFunctionInputSwitch(input_type, result_type), CMUtils::Bind); result.serialize = CMIntegralSerialize; result.deserialize = CMIntegralDeserialize; +#if defined(D_ASSERT_IS_ENABLED) + result.errors = FunctionErrors::CAN_THROW_RUNTIME_ERROR; // Can only throw runtime error when assertions are enabled +#else + result.errors = FunctionErrors::CANNOT_ERROR; +#endif return result; } diff --git a/src/duckdb/src/function/scalar/compressed_materialization/compress_string.cpp b/src/duckdb/src/function/scalar/compressed_materialization/compress_string.cpp index 36175bedb..204a36c66 100644 --- a/src/duckdb/src/function/scalar/compressed_materialization/compress_string.cpp +++ b/src/duckdb/src/function/scalar/compressed_materialization/compress_string.cpp @@ -38,8 +38,9 @@ inline RESULT_TYPE StringCompressInternal(const string_t &input) { TemplatedReverseMemCpy(result_ptr + REMAINDER, const_data_ptr_cast(input.GetPrefix())); memset(result_ptr, '\0', REMAINDER); } else { - const auto remainder = sizeof(RESULT_TYPE) - input.GetSize(); - ReverseMemCpy(result_ptr + remainder, data_ptr_cast(input.GetPointer()), input.GetSize()); + const auto size = MinValue(sizeof(RESULT_TYPE), input.GetSize()); + const auto remainder = sizeof(RESULT_TYPE) - size; + ReverseMemCpy(result_ptr + remainder, data_ptr_cast(input.GetPointer()), size); memset(result_ptr, '\0', remainder); } result_ptr[0] = UnsafeNumericCast(input.GetSize()); @@ -97,6 +98,9 @@ scalar_function_t GetStringCompressFunctionSwitch(const LogicalType &result_type return GetStringCompressFunction(result_type); case LogicalTypeId::UHUGEINT: return GetStringCompressFunction(result_type); + case LogicalTypeId::HUGEINT: + // Never generated, only for backwards compatibility + return GetStringCompressFunction(result_type); default: throw InternalException("Unexpected type in GetStringCompressFunctionSwitch"); } @@ -238,6 +242,11 @@ ScalarFunction CMStringCompressFun::GetFunction(const LogicalType &result_type) GetStringCompressFunctionSwitch(result_type), CMUtils::Bind); result.serialize = CMStringCompressSerialize; result.deserialize = CMStringCompressDeserialize; +#if defined(D_ASSERT_IS_ENABLED) + result.errors = FunctionErrors::CAN_THROW_RUNTIME_ERROR; // Can only throw runtime error when assertions are enabled +#else + result.errors = FunctionErrors::CANNOT_ERROR; +#endif return result; } @@ -266,6 +275,11 @@ ScalarFunction InternalCompressStringUbigintFun::GetFunction() { return CMStringCompressFun::GetFunction(LogicalType(LogicalTypeId::UBIGINT)); } +ScalarFunction InternalCompressStringHugeintFun::GetFunction() { + // We never generate this, but it's needed for backwards compatibility + return CMStringCompressFun::GetFunction(LogicalType(LogicalTypeId::HUGEINT)); +} + ScalarFunction InternalCompressStringUhugeintFun::GetFunction() { return CMStringCompressFun::GetFunction(LogicalType(LogicalTypeId::UHUGEINT)); } diff --git a/src/duckdb/src/function/table/arrow.cpp b/src/duckdb/src/function/table/arrow.cpp index 676787ae0..d41f63b2c 100644 --- a/src/duckdb/src/function/table/arrow.cpp +++ b/src/duckdb/src/function/table/arrow.cpp @@ -18,23 +18,32 @@ namespace duckdb { -void ArrowTableFunction::PopulateArrowTableType(DBConfig &config, ArrowTableType &arrow_table, - const ArrowSchemaWrapper &schema_p, vector &names, - vector &return_types) { - for (idx_t col_idx = 0; col_idx < static_cast(schema_p.arrow_schema.n_children); col_idx++) { - auto &schema = *schema_p.arrow_schema.children[col_idx]; +void ArrowTableFunction::PopulateArrowTableSchema(DBConfig &config, ArrowTableSchema &arrow_table, + const ArrowSchema &arrow_schema) { + vector names; + // We first gather the column names and deduplicate them + for (idx_t col_idx = 0; col_idx < static_cast(arrow_schema.n_children); col_idx++) { + const auto &schema = *arrow_schema.children[col_idx]; if (!schema.release) { throw InvalidInputException("arrow_scan: released schema passed"); } - auto arrow_type = ArrowType::GetArrowLogicalType(config, schema); - return_types.emplace_back(arrow_type->GetDuckType(true)); - arrow_table.AddColumn(col_idx, std::move(arrow_type)); auto name = string(schema.name); if (name.empty()) { name = string("v") + to_string(col_idx); } names.push_back(name); } + QueryResult::DeduplicateColumns(names); + + // We do a second iteration to figure out the arrow types and already set their deduplicated names + for (idx_t col_idx = 0; col_idx < static_cast(arrow_schema.n_children); col_idx++) { + auto &schema = *arrow_schema.children[col_idx]; + if (!schema.release) { + throw InvalidInputException("arrow_scan: released schema passed"); + } + auto arrow_type = ArrowType::GetArrowLogicalType(config, schema); + arrow_table.AddColumn(col_idx, std::move(arrow_type), names[col_idx]); + } } unique_ptr ArrowTableFunction::ArrowScanBindDumb(ClientContext &context, TableFunctionBindInput &input, @@ -69,8 +78,9 @@ unique_ptr ArrowTableFunction::ArrowScanBind(ClientContext &contex auto &data = *res; stream_factory_get_schema(reinterpret_cast(stream_factory_ptr), data.schema_root.arrow_schema); - PopulateArrowTableType(DBConfig::GetConfig(context), res->arrow_table, data.schema_root, names, return_types); - QueryResult::DeduplicateColumns(names); + PopulateArrowTableSchema(DBConfig::GetConfig(context), res->arrow_table, data.schema_root.arrow_schema); + names = res->arrow_table.GetNames(); + return_types = res->arrow_table.GetTypes(); res->all_types = return_types; if (return_types.empty()) { throw InvalidInputException("Provided table/dataframe must have at least one column"); diff --git a/src/duckdb/src/function/table/arrow/arrow_array_scan_state.cpp b/src/duckdb/src/function/table/arrow/arrow_array_scan_state.cpp index bbb1bf3e2..fd9d14dc7 100644 --- a/src/duckdb/src/function/table/arrow/arrow_array_scan_state.cpp +++ b/src/duckdb/src/function/table/arrow/arrow_array_scan_state.cpp @@ -4,15 +4,14 @@ namespace duckdb { -ArrowArrayScanState::ArrowArrayScanState(ArrowScanLocalState &state, ClientContext &context) - : state(state), context(context) { +ArrowArrayScanState::ArrowArrayScanState(ClientContext &context) : context(context) { arrow_dictionary = nullptr; } ArrowArrayScanState &ArrowArrayScanState::GetChild(idx_t child_idx) { auto it = children.find(child_idx); if (it == children.end()) { - auto child_p = make_uniq(state, context); + auto child_p = make_uniq(context); auto &child = *child_p; child.owned_data = owned_data; children.emplace(child_idx, std::move(child_p)); diff --git a/src/duckdb/src/function/table/arrow/arrow_duck_schema.cpp b/src/duckdb/src/function/table/arrow/arrow_duck_schema.cpp index 86e63ff36..ab5611c13 100644 --- a/src/duckdb/src/function/table/arrow/arrow_duck_schema.cpp +++ b/src/duckdb/src/function/table/arrow/arrow_duck_schema.cpp @@ -8,15 +8,27 @@ namespace duckdb { -void ArrowTableType::AddColumn(idx_t index, shared_ptr type) { +void ArrowTableSchema::AddColumn(idx_t index, shared_ptr type, const string &name) { D_ASSERT(arrow_convert_data.find(index) == arrow_convert_data.end()); + if (index >= types.size()) { + types.resize(index + 1); + column_names.resize(index + 1); + } + types[index] = type->GetDuckType(true); + column_names[index] = name; arrow_convert_data.emplace(std::make_pair(index, std::move(type))); } -const arrow_column_map_t &ArrowTableType::GetColumns() const { +const arrow_column_map_t &ArrowTableSchema::GetColumns() const { return arrow_convert_data; } +vector &ArrowTableSchema::GetTypes() { + return types; +} +vector &ArrowTableSchema::GetNames() { + return column_names; +} void ArrowType::SetDictionary(unique_ptr dictionary) { D_ASSERT(!this->dictionary_type); dictionary_type = std::move(dictionary); @@ -380,6 +392,16 @@ bool ArrowType::HasExtension() const { return extension_data.get() != nullptr; } +ArrowArrayPhysicalType ArrowType::GetPhysicalType() const { + if (HasDictionary()) { + return ArrowArrayPhysicalType::DICTIONARY_ENCODED; + } + if (RunEndEncoded()) { + return ArrowArrayPhysicalType::RUN_END_ENCODED; + } + return ArrowArrayPhysicalType::DEFAULT; +} + unique_ptr ArrowType::GetTypeFromSchema(DBConfig &config, ArrowSchema &schema) { auto format = string(schema.format); // Let's first figure out if this type is an extension type diff --git a/src/duckdb/src/function/table/arrow_conversion.cpp b/src/duckdb/src/function/table/arrow_conversion.cpp index 047034033..eeeba397b 100644 --- a/src/duckdb/src/function/table/arrow_conversion.cpp +++ b/src/duckdb/src/function/table/arrow_conversion.cpp @@ -13,22 +13,6 @@ namespace duckdb { -namespace { - -enum class ArrowArrayPhysicalType : uint8_t { DICTIONARY_ENCODED, RUN_END_ENCODED, DEFAULT }; - -ArrowArrayPhysicalType GetArrowArrayPhysicalType(const ArrowType &type) { - if (type.HasDictionary()) { - return ArrowArrayPhysicalType::DICTIONARY_ENCODED; - } - if (type.RunEndEncoded()) { - return ArrowArrayPhysicalType::RUN_END_ENCODED; - } - return ArrowArrayPhysicalType::DEFAULT; -} - -} // namespace - #if STANDARD_VECTOR_SIZE > 64 static void ShiftRight(unsigned char *ar, int size, int shift) { int carry = 0; @@ -42,7 +26,7 @@ static void ShiftRight(unsigned char *ar, int size, int shift) { } #endif -idx_t GetEffectiveOffset(const ArrowArray &array, int64_t parent_offset, const ArrowScanLocalState &state, +idx_t GetEffectiveOffset(const ArrowArray &array, int64_t parent_offset, idx_t chunk_offset, int64_t nested_offset = -1) { if (nested_offset != -1) { // The parent of this array is a list @@ -52,7 +36,7 @@ idx_t GetEffectiveOffset(const ArrowArray &array, int64_t parent_offset, const A // Parent offset is set in the case of a struct, it applies to all child arrays // 'chunk_offset' is how much of the chunk we've already scanned, in case the chunk size exceeds // STANDARD_VECTOR_SIZE - return UnsafeNumericCast(array.offset + parent_offset) + state.chunk_offset; + return UnsafeNumericCast(array.offset + parent_offset) + chunk_offset; } template @@ -60,7 +44,7 @@ T *ArrowBufferData(ArrowArray &array, idx_t buffer_idx) { return (T *)array.buffers[buffer_idx]; // NOLINT } -static void GetValidityMask(ValidityMask &mask, ArrowArray &array, const ArrowScanLocalState &scan_state, idx_t size, +static void GetValidityMask(ValidityMask &mask, ArrowArray &array, idx_t chunk_offset, idx_t size, int64_t parent_offset, int64_t nested_offset = -1, bool add_null = false) { // In certains we don't need to or cannot copy arrow's validity mask to duckdb. // @@ -69,7 +53,7 @@ static void GetValidityMask(ValidityMask &mask, ArrowArray &array, const ArrowSc // 2. n_buffers > 0, meaning the array's arrow type is not `null` // 3. the validity buffer (the first buffer) is not a nullptr if (array.null_count != 0 && array.n_buffers > 0 && array.buffers[0]) { - auto bit_offset = GetEffectiveOffset(array, parent_offset, scan_state, nested_offset); + auto bit_offset = GetEffectiveOffset(array, parent_offset, chunk_offset, nested_offset); mask.EnsureWritable(); #if STANDARD_VECTOR_SIZE > 64 auto n_bitmask_bytes = (size + 8 - 1) / 8; @@ -107,26 +91,13 @@ static void GetValidityMask(ValidityMask &mask, ArrowArray &array, const ArrowSc } } -static void SetValidityMask(Vector &vector, ArrowArray &array, const ArrowScanLocalState &scan_state, idx_t size, - int64_t parent_offset, int64_t nested_offset, bool add_null = false) { +void ArrowToDuckDBConversion::SetValidityMask(Vector &vector, ArrowArray &array, idx_t chunk_offset, idx_t size, + int64_t parent_offset, int64_t nested_offset, bool add_null) { D_ASSERT(vector.GetVectorType() == VectorType::FLAT_VECTOR); auto &mask = FlatVector::Validity(vector); - GetValidityMask(mask, array, scan_state, size, parent_offset, nested_offset, add_null); + GetValidityMask(mask, array, chunk_offset, size, parent_offset, nested_offset, add_null); } -static void ColumnArrowToDuckDBRunEndEncoded(Vector &vector, const ArrowArray &array, ArrowArrayScanState &array_state, - idx_t size, const ArrowType &arrow_type, int64_t nested_offset = -1, - ValidityMask *parent_mask = nullptr, uint64_t parent_offset = 0); - -static void ColumnArrowToDuckDB(Vector &vector, ArrowArray &array, ArrowArrayScanState &array_state, idx_t size, - const ArrowType &arrow_type, int64_t nested_offset = -1, - ValidityMask *parent_mask = nullptr, uint64_t parent_offset = 0, - bool ignore_extensions = false); - -static void ColumnArrowToDuckDBDictionary(Vector &vector, ArrowArray &array, ArrowArrayScanState &array_state, - idx_t size, const ArrowType &arrow_type, int64_t nested_offset = -1, - const ValidityMask *parent_mask = nullptr, uint64_t parent_offset = 0); - namespace { struct ArrowListOffsetData { @@ -223,15 +194,13 @@ static ArrowListOffsetData ConvertArrowListOffsets(Vector &vector, ArrowArray &a } } -static void ArrowToDuckDBList(Vector &vector, ArrowArray &array, ArrowArrayScanState &array_state, idx_t size, - const ArrowType &arrow_type, int64_t nested_offset, const ValidityMask *parent_mask, - int64_t parent_offset) { - auto &scan_state = array_state.state; - +static void ArrowToDuckDBList(Vector &vector, ArrowArray &array, idx_t chunk_offset, ArrowArrayScanState &array_state, + idx_t size, const ArrowType &arrow_type, int64_t nested_offset, + const ValidityMask *parent_mask, int64_t parent_offset) { auto &list_info = arrow_type.GetTypeInfo(); - SetValidityMask(vector, array, scan_state, size, parent_offset, nested_offset); + ArrowToDuckDBConversion::SetValidityMask(vector, array, chunk_offset, size, parent_offset, nested_offset); - auto effective_offset = GetEffectiveOffset(array, parent_offset, scan_state, nested_offset); + auto effective_offset = GetEffectiveOffset(array, parent_offset, chunk_offset, nested_offset); auto list_data = ConvertArrowListOffsets(vector, array, size, arrow_type, effective_offset); auto &start_offset = list_data.start_offset; auto &list_size = list_data.list_size; @@ -239,8 +208,8 @@ static void ArrowToDuckDBList(Vector &vector, ArrowArray &array, ArrowArrayScanS ListVector::Reserve(vector, list_size); ListVector::SetListSize(vector, list_size); auto &child_vector = ListVector::GetEntry(vector); - SetValidityMask(child_vector, *array.children[0], scan_state, list_size, array.offset, - NumericCast(start_offset)); + ArrowToDuckDBConversion::SetValidityMask(child_vector, *array.children[0], chunk_offset, list_size, array.offset, + NumericCast(start_offset)); auto &list_mask = FlatVector::Validity(vector); if (parent_mask) { //! Since this List is owned by a struct we must guarantee their validity map matches on Null @@ -258,45 +227,47 @@ static void ArrowToDuckDBList(Vector &vector, ArrowArray &array, ArrowArrayScanS if (list_size == 0 && start_offset == 0) { D_ASSERT(!child_array.dictionary); - ColumnArrowToDuckDB(child_vector, child_array, child_state, list_size, child_type, -1); + ArrowToDuckDBConversion::ColumnArrowToDuckDB(child_vector, child_array, chunk_offset, child_state, list_size, + child_type, -1); return; } - auto array_physical_type = GetArrowArrayPhysicalType(child_type); + auto array_physical_type = child_type.GetPhysicalType(); switch (array_physical_type) { case ArrowArrayPhysicalType::DICTIONARY_ENCODED: // TODO: add support for offsets - ColumnArrowToDuckDBDictionary(child_vector, child_array, child_state, list_size, child_type, - NumericCast(start_offset)); + ArrowToDuckDBConversion::ColumnArrowToDuckDBDictionary(child_vector, child_array, chunk_offset, child_state, + list_size, child_type, + NumericCast(start_offset)); break; case ArrowArrayPhysicalType::RUN_END_ENCODED: - ColumnArrowToDuckDBRunEndEncoded(child_vector, child_array, child_state, list_size, child_type, - NumericCast(start_offset)); + ArrowToDuckDBConversion::ColumnArrowToDuckDBRunEndEncoded(child_vector, child_array, chunk_offset, child_state, + list_size, child_type, + NumericCast(start_offset)); break; case ArrowArrayPhysicalType::DEFAULT: - ColumnArrowToDuckDB(child_vector, child_array, child_state, list_size, child_type, - NumericCast(start_offset)); + ArrowToDuckDBConversion::ColumnArrowToDuckDB(child_vector, child_array, chunk_offset, child_state, list_size, + child_type, NumericCast(start_offset)); break; default: throw NotImplementedException("ArrowArrayPhysicalType not recognized"); } } -static void ArrowToDuckDBArray(Vector &vector, ArrowArray &array, ArrowArrayScanState &array_state, idx_t size, - const ArrowType &arrow_type, int64_t nested_offset, const ValidityMask *parent_mask, - int64_t parent_offset) { +static void ArrowToDuckDBArray(Vector &vector, ArrowArray &array, idx_t chunk_offset, ArrowArrayScanState &array_state, + idx_t size, const ArrowType &arrow_type, int64_t nested_offset, + const ValidityMask *parent_mask, int64_t parent_offset) { auto &array_info = arrow_type.GetTypeInfo(); - auto &scan_state = array_state.state; auto array_size = array_info.FixedSize(); auto child_count = array_size * size; - auto child_offset = GetEffectiveOffset(array, parent_offset, scan_state, nested_offset) * array_size; + auto child_offset = GetEffectiveOffset(array, parent_offset, chunk_offset, nested_offset) * array_size; - SetValidityMask(vector, array, scan_state, size, parent_offset, nested_offset); + ArrowToDuckDBConversion::SetValidityMask(vector, array, chunk_offset, size, parent_offset, nested_offset); auto &child_vector = ArrayVector::GetEntry(vector); - SetValidityMask(child_vector, *array.children[0], scan_state, child_count, array.offset, - NumericCast(child_offset)); + ArrowToDuckDBConversion::SetValidityMask(child_vector, *array.children[0], chunk_offset, child_count, array.offset, + NumericCast(child_offset)); auto &array_mask = FlatVector::Validity(vector); if (parent_mask) { @@ -327,14 +298,16 @@ static void ArrowToDuckDBArray(Vector &vector, ArrowArray &array, ArrowArrayScan auto &child_type = array_info.GetChild(); if (child_count == 0 && child_offset == 0) { D_ASSERT(!child_array.dictionary); - ColumnArrowToDuckDB(child_vector, child_array, child_state, child_count, child_type, -1); + ArrowToDuckDBConversion::ColumnArrowToDuckDB(child_vector, child_array, chunk_offset, child_state, child_count, + child_type, -1); } else { if (child_array.dictionary) { - ColumnArrowToDuckDBDictionary(child_vector, child_array, child_state, child_count, child_type, - NumericCast(child_offset)); + ArrowToDuckDBConversion::ColumnArrowToDuckDBDictionary(child_vector, child_array, chunk_offset, child_state, + child_count, child_type, + NumericCast(child_offset)); } else { - ColumnArrowToDuckDB(child_vector, child_array, child_state, child_count, child_type, - NumericCast(child_offset)); + ArrowToDuckDBConversion::ColumnArrowToDuckDB(child_vector, child_array, chunk_offset, child_state, + child_count, child_type, NumericCast(child_offset)); } } } @@ -401,22 +374,22 @@ static void SetVectorStringView(Vector &vector, idx_t size, ArrowArray &array, i } } -static void DirectConversion(Vector &vector, ArrowArray &array, const ArrowScanLocalState &scan_state, - int64_t nested_offset, uint64_t parent_offset) { +static void DirectConversion(Vector &vector, ArrowArray &array, idx_t chunk_offset, int64_t nested_offset, + uint64_t parent_offset) { auto internal_type = GetTypeIdSize(vector.GetType().InternalType()); auto data_ptr = ArrowBufferData(array, 1) + - internal_type * GetEffectiveOffset(array, NumericCast(parent_offset), scan_state, nested_offset); + internal_type * GetEffectiveOffset(array, NumericCast(parent_offset), chunk_offset, nested_offset); FlatVector::SetData(vector, data_ptr); } template -static void TimeConversion(Vector &vector, ArrowArray &array, const ArrowScanLocalState &scan_state, - int64_t nested_offset, int64_t parent_offset, idx_t size, int64_t conversion) { +static void TimeConversion(Vector &vector, ArrowArray &array, idx_t chunk_offset, int64_t nested_offset, + int64_t parent_offset, idx_t size, int64_t conversion) { auto tgt_ptr = FlatVector::GetData(vector); auto &validity_mask = FlatVector::Validity(vector); - auto src_ptr = - static_cast(array.buffers[1]) + GetEffectiveOffset(array, parent_offset, scan_state, nested_offset); + auto src_ptr = static_cast(array.buffers[1]) + + GetEffectiveOffset(array, parent_offset, chunk_offset, nested_offset); for (idx_t row = 0; row < size; row++) { if (!validity_mask.RowIsValid(row)) { continue; @@ -427,12 +400,12 @@ static void TimeConversion(Vector &vector, ArrowArray &array, const ArrowScanLoc } } -static void UUIDConversion(Vector &vector, const ArrowArray &array, const ArrowScanLocalState &scan_state, - int64_t nested_offset, int64_t parent_offset, idx_t size) { +static void UUIDConversion(Vector &vector, const ArrowArray &array, idx_t chunk_offset, int64_t nested_offset, + int64_t parent_offset, idx_t size) { auto tgt_ptr = FlatVector::GetData(vector); auto &validity_mask = FlatVector::Validity(vector); auto src_ptr = static_cast(array.buffers[1]) + - GetEffectiveOffset(array, parent_offset, scan_state, nested_offset); + GetEffectiveOffset(array, parent_offset, chunk_offset, nested_offset); for (idx_t row = 0; row < size; row++) { if (!validity_mask.RowIsValid(row)) { continue; @@ -444,12 +417,12 @@ static void UUIDConversion(Vector &vector, const ArrowArray &array, const ArrowS } } -static void TimestampTZConversion(Vector &vector, ArrowArray &array, const ArrowScanLocalState &scan_state, - int64_t nested_offset, int64_t parent_offset, idx_t size, int64_t conversion) { +static void TimestampTZConversion(Vector &vector, ArrowArray &array, idx_t chunk_offset, int64_t nested_offset, + int64_t parent_offset, idx_t size, int64_t conversion) { auto tgt_ptr = FlatVector::GetData(vector); auto &validity_mask = FlatVector::Validity(vector); auto src_ptr = - ArrowBufferData(array, 1) + GetEffectiveOffset(array, parent_offset, scan_state, nested_offset); + ArrowBufferData(array, 1) + GetEffectiveOffset(array, parent_offset, chunk_offset, nested_offset); for (idx_t row = 0; row < size; row++) { if (!validity_mask.RowIsValid(row)) { continue; @@ -460,11 +433,11 @@ static void TimestampTZConversion(Vector &vector, ArrowArray &array, const Arrow } } -static void IntervalConversionUs(Vector &vector, ArrowArray &array, const ArrowScanLocalState &scan_state, - int64_t nested_offset, int64_t parent_offset, idx_t size, int64_t conversion) { +static void IntervalConversionUs(Vector &vector, ArrowArray &array, idx_t chunk_offset, int64_t nested_offset, + int64_t parent_offset, idx_t size, int64_t conversion) { auto tgt_ptr = FlatVector::GetData(vector); auto src_ptr = - ArrowBufferData(array, 1) + GetEffectiveOffset(array, parent_offset, scan_state, nested_offset); + ArrowBufferData(array, 1) + GetEffectiveOffset(array, parent_offset, chunk_offset, nested_offset); for (idx_t row = 0; row < size; row++) { tgt_ptr[row].days = 0; tgt_ptr[row].months = 0; @@ -474,11 +447,11 @@ static void IntervalConversionUs(Vector &vector, ArrowArray &array, const ArrowS } } -static void IntervalConversionMonths(Vector &vector, ArrowArray &array, const ArrowScanLocalState &scan_state, - int64_t nested_offset, int64_t parent_offset, idx_t size) { +static void IntervalConversionMonths(Vector &vector, ArrowArray &array, idx_t chunk_offset, int64_t nested_offset, + int64_t parent_offset, idx_t size) { auto tgt_ptr = FlatVector::GetData(vector); auto src_ptr = - ArrowBufferData(array, 1) + GetEffectiveOffset(array, parent_offset, scan_state, nested_offset); + ArrowBufferData(array, 1) + GetEffectiveOffset(array, parent_offset, chunk_offset, nested_offset); for (idx_t row = 0; row < size; row++) { tgt_ptr[row].days = 0; tgt_ptr[row].micros = 0; @@ -486,11 +459,11 @@ static void IntervalConversionMonths(Vector &vector, ArrowArray &array, const Ar } } -static void IntervalConversionMonthDayNanos(Vector &vector, ArrowArray &array, const ArrowScanLocalState &scan_state, +static void IntervalConversionMonthDayNanos(Vector &vector, ArrowArray &array, idx_t chunk_offset, int64_t nested_offset, int64_t parent_offset, idx_t size) { auto tgt_ptr = FlatVector::GetData(vector); - auto src_ptr = - ArrowBufferData(array, 1) + GetEffectiveOffset(array, parent_offset, scan_state, nested_offset); + auto src_ptr = ArrowBufferData(array, 1) + + GetEffectiveOffset(array, parent_offset, chunk_offset, nested_offset); for (idx_t row = 0; row < size; row++) { tgt_ptr[row].days = src_ptr[row].days; tgt_ptr[row].micros = src_ptr[row].nanoseconds / Interval::NANOS_PER_MICRO; @@ -664,9 +637,11 @@ static void FlattenRunEndsSwitch(Vector &result, ArrowRunEndEncodingState &run_e } } -static void ColumnArrowToDuckDBRunEndEncoded(Vector &vector, const ArrowArray &array, ArrowArrayScanState &array_state, - idx_t size, const ArrowType &arrow_type, int64_t nested_offset, - ValidityMask *parent_mask, uint64_t parent_offset) { +void ArrowToDuckDBConversion::ColumnArrowToDuckDBRunEndEncoded(Vector &vector, const ArrowArray &array, + idx_t chunk_offset, ArrowArrayScanState &array_state, + idx_t size, const ArrowType &arrow_type, + int64_t nested_offset, ValidityMask *parent_mask, + uint64_t parent_offset) { // Scan the 'run_ends' array D_ASSERT(array.n_children == 2); auto &run_ends_array = *array.children[0]; @@ -677,7 +652,6 @@ static void ColumnArrowToDuckDBRunEndEncoded(Vector &vector, const ArrowArray &a auto &values_type = struct_info.GetChild(1); D_ASSERT(vector.GetType() == values_type.GetDuckType()); - auto &scan_state = array_state.state; if (vector.GetBuffer()) { vector.GetBuffer()->SetAuxiliaryData(make_uniq(array_state.owned_data)); } @@ -692,14 +666,16 @@ static void ColumnArrowToDuckDBRunEndEncoded(Vector &vector, const ArrowArray &a run_end_encoding.run_ends = make_uniq(run_ends_type.GetDuckType(), compressed_size); run_end_encoding.values = make_uniq(values_type.GetDuckType(), compressed_size); - ColumnArrowToDuckDB(*run_end_encoding.run_ends, run_ends_array, array_state, compressed_size, run_ends_type); + ArrowToDuckDBConversion::ColumnArrowToDuckDB(*run_end_encoding.run_ends, run_ends_array, chunk_offset, + array_state, compressed_size, run_ends_type); auto &values = *run_end_encoding.values; - SetValidityMask(values, values_array, scan_state, compressed_size, NumericCast(parent_offset), - nested_offset); - ColumnArrowToDuckDB(values, values_array, array_state, compressed_size, values_type); + ArrowToDuckDBConversion::SetValidityMask(values, values_array, chunk_offset, compressed_size, + NumericCast(parent_offset), nested_offset); + ArrowToDuckDBConversion::ColumnArrowToDuckDB(values, values_array, chunk_offset, array_state, compressed_size, + values_type); } - idx_t scan_offset = GetEffectiveOffset(array, NumericCast(parent_offset), scan_state, nested_offset); + idx_t scan_offset = GetEffectiveOffset(array, NumericCast(parent_offset), chunk_offset, nested_offset); auto physical_type = run_ends_type.GetDuckType().InternalType(); switch (physical_type) { case PhysicalType::INT16: @@ -717,7 +693,7 @@ static void ColumnArrowToDuckDBRunEndEncoded(Vector &vector, const ArrowArray &a } template void ConvertDecimal(SRC src_ptr, Vector &vector, ArrowArray &array, idx_t size, int64_t nested_offset, - uint64_t parent_offset, ArrowScanLocalState &scan_state, ValidityMask &val_mask, + uint64_t parent_offset, idx_t chunk_offset, ValidityMask &val_mask, DecimalBitWidth arrow_bit_width) { switch (vector.GetType().InternalType()) { @@ -737,7 +713,7 @@ void ConvertDecimal(SRC src_ptr, Vector &vector, ArrowArray &array, idx_t size, FlatVector::SetData(vector, ArrowBufferData(array, 1) + GetTypeIdSize(vector.GetType().InternalType()) * GetEffectiveOffset(array, NumericCast(parent_offset), - scan_state, nested_offset)); + chunk_offset, nested_offset)); } else { auto tgt_ptr = FlatVector::GetData(vector); for (idx_t row = 0; row < size; row++) { @@ -755,7 +731,7 @@ void ConvertDecimal(SRC src_ptr, Vector &vector, ArrowArray &array, idx_t size, FlatVector::SetData(vector, ArrowBufferData(array, 1) + GetTypeIdSize(vector.GetType().InternalType()) * GetEffectiveOffset(array, NumericCast(parent_offset), - scan_state, nested_offset)); + chunk_offset, nested_offset)); } else { auto tgt_ptr = FlatVector::GetData(vector); for (idx_t row = 0; row < size; row++) { @@ -773,7 +749,7 @@ void ConvertDecimal(SRC src_ptr, Vector &vector, ArrowArray &array, idx_t size, FlatVector::SetData(vector, ArrowBufferData(array, 1) + GetTypeIdSize(vector.GetType().InternalType()) * GetEffectiveOffset(array, NumericCast(parent_offset), - scan_state, nested_offset)); + chunk_offset, nested_offset)); } else { auto tgt_ptr = FlatVector::GetData(vector); for (idx_t row = 0; row < size; row++) { @@ -793,17 +769,18 @@ void ConvertDecimal(SRC src_ptr, Vector &vector, ArrowArray &array, idx_t size, } } -static void ColumnArrowToDuckDB(Vector &vector, ArrowArray &array, ArrowArrayScanState &array_state, idx_t size, - const ArrowType &arrow_type, int64_t nested_offset, ValidityMask *parent_mask, - uint64_t parent_offset, bool ignore_extensions) { - auto &scan_state = array_state.state; +void ArrowToDuckDBConversion::ColumnArrowToDuckDB(Vector &vector, ArrowArray &array, idx_t chunk_offset, + ArrowArrayScanState &array_state, idx_t size, + const ArrowType &arrow_type, int64_t nested_offset, + ValidityMask *parent_mask, uint64_t parent_offset, + bool ignore_extensions) { D_ASSERT(!array.dictionary); if (!ignore_extensions && arrow_type.HasExtension()) { if (arrow_type.extension_data->arrow_to_duckdb) { // Convert the storage and then call the cast function Vector input_data(arrow_type.extension_data->GetInternalType()); - ColumnArrowToDuckDB(input_data, array, array_state, size, arrow_type, nested_offset, parent_mask, - parent_offset, /*ignore_extensions*/ true); + ColumnArrowToDuckDB(input_data, array, chunk_offset, array_state, size, arrow_type, nested_offset, + parent_mask, parent_offset, /*ignore_extensions*/ true); arrow_type.extension_data->arrow_to_duckdb(array_state.context, input_data, vector, size); return; } @@ -820,7 +797,7 @@ static void ColumnArrowToDuckDB(Vector &vector, ArrowArray &array, ArrowArraySca //! Arrow bit-packs boolean values //! Lets first figure out where we are in the source array auto effective_offset = - GetEffectiveOffset(array, NumericCast(parent_offset), scan_state, nested_offset); + GetEffectiveOffset(array, NumericCast(parent_offset), chunk_offset, nested_offset); auto src_ptr = ArrowBufferData(array, 1) + effective_offset / 8; auto tgt_ptr = (uint8_t *)FlatVector::GetData(vector); int src_pos = 0; @@ -856,11 +833,11 @@ static void ColumnArrowToDuckDB(Vector &vector, ArrowArray &array, ArrowArraySca case LogicalTypeId::TIMESTAMP_MS: case LogicalTypeId::TIMESTAMP_NS: case LogicalTypeId::TIME_TZ: { - DirectConversion(vector, array, scan_state, nested_offset, parent_offset); + DirectConversion(vector, array, chunk_offset, nested_offset, parent_offset); break; } case LogicalTypeId::UUID: - UUIDConversion(vector, array, scan_state, nested_offset, NumericCast(parent_offset), size); + UUIDConversion(vector, array, chunk_offset, nested_offset, NumericCast(parent_offset), size); break; case LogicalTypeId::BLOB: case LogicalTypeId::BIT: @@ -872,29 +849,29 @@ static void ColumnArrowToDuckDB(Vector &vector, ArrowArray &array, ArrowArraySca case ArrowVariableSizeType::SUPER_SIZE: { auto cdata = ArrowBufferData(array, 2); auto offsets = ArrowBufferData(array, 1) + - GetEffectiveOffset(array, NumericCast(parent_offset), scan_state, nested_offset); + GetEffectiveOffset(array, NumericCast(parent_offset), chunk_offset, nested_offset); SetVectorString(vector, size, cdata, offsets); break; } case ArrowVariableSizeType::NORMAL: { auto cdata = ArrowBufferData(array, 2); auto offsets = ArrowBufferData(array, 1) + - GetEffectiveOffset(array, NumericCast(parent_offset), scan_state, nested_offset); + GetEffectiveOffset(array, NumericCast(parent_offset), chunk_offset, nested_offset); SetVectorString(vector, size, cdata, offsets); break; } case ArrowVariableSizeType::VIEW: { SetVectorStringView( vector, size, array, - GetEffectiveOffset(array, NumericCast(parent_offset), scan_state, nested_offset)); + GetEffectiveOffset(array, NumericCast(parent_offset), chunk_offset, nested_offset)); break; } case ArrowVariableSizeType::FIXED_SIZE: { - SetValidityMask(vector, array, scan_state, size, NumericCast(parent_offset), nested_offset); + SetValidityMask(vector, array, chunk_offset, size, NumericCast(parent_offset), nested_offset); auto fixed_size = string_info.FixedSize(); // Have to check validity mask before setting this up - idx_t offset = - GetEffectiveOffset(array, NumericCast(parent_offset), scan_state, nested_offset) * fixed_size; + idx_t offset = GetEffectiveOffset(array, NumericCast(parent_offset), chunk_offset, nested_offset) * + fixed_size; auto cdata = ArrowBufferData(array, 1); auto blob_len = fixed_size; auto result = FlatVector::GetData(vector); @@ -916,13 +893,13 @@ static void ColumnArrowToDuckDB(Vector &vector, ArrowArray &array, ArrowArraySca auto precision = datetime_info.GetDateTimeType(); switch (precision) { case ArrowDateTimeType::DAYS: { - DirectConversion(vector, array, scan_state, nested_offset, parent_offset); + DirectConversion(vector, array, chunk_offset, nested_offset, parent_offset); break; } case ArrowDateTimeType::MILLISECONDS: { //! convert date from nanoseconds to days auto src_ptr = ArrowBufferData(array, 1) + - GetEffectiveOffset(array, NumericCast(parent_offset), scan_state, nested_offset); + GetEffectiveOffset(array, NumericCast(parent_offset), chunk_offset, nested_offset); auto tgt_ptr = FlatVector::GetData(vector); for (idx_t row = 0; row < size; row++) { tgt_ptr[row] = date_t(UnsafeNumericCast(static_cast(src_ptr[row]) / @@ -940,24 +917,24 @@ static void ColumnArrowToDuckDB(Vector &vector, ArrowArray &array, ArrowArraySca auto precision = datetime_info.GetDateTimeType(); switch (precision) { case ArrowDateTimeType::SECONDS: { - TimeConversion(vector, array, scan_state, nested_offset, NumericCast(parent_offset), size, - 1000000); + TimeConversion(vector, array, chunk_offset, nested_offset, NumericCast(parent_offset), + size, 1000000); break; } case ArrowDateTimeType::MILLISECONDS: { - TimeConversion(vector, array, scan_state, nested_offset, NumericCast(parent_offset), size, - 1000); + TimeConversion(vector, array, chunk_offset, nested_offset, NumericCast(parent_offset), + size, 1000); break; } case ArrowDateTimeType::MICROSECONDS: { - TimeConversion(vector, array, scan_state, nested_offset, NumericCast(parent_offset), size, - 1); + TimeConversion(vector, array, chunk_offset, nested_offset, NumericCast(parent_offset), + size, 1); break; } case ArrowDateTimeType::NANOSECONDS: { auto tgt_ptr = FlatVector::GetData(vector); auto src_ptr = ArrowBufferData(array, 1) + - GetEffectiveOffset(array, NumericCast(parent_offset), scan_state, nested_offset); + GetEffectiveOffset(array, NumericCast(parent_offset), chunk_offset, nested_offset); for (idx_t row = 0; row < size; row++) { tgt_ptr[row].micros = src_ptr[row] / 1000; } @@ -973,23 +950,23 @@ static void ColumnArrowToDuckDB(Vector &vector, ArrowArray &array, ArrowArraySca auto precision = datetime_info.GetDateTimeType(); switch (precision) { case ArrowDateTimeType::SECONDS: { - TimestampTZConversion(vector, array, scan_state, nested_offset, NumericCast(parent_offset), size, + TimestampTZConversion(vector, array, chunk_offset, nested_offset, NumericCast(parent_offset), size, 1000000); break; } case ArrowDateTimeType::MILLISECONDS: { - TimestampTZConversion(vector, array, scan_state, nested_offset, NumericCast(parent_offset), size, + TimestampTZConversion(vector, array, chunk_offset, nested_offset, NumericCast(parent_offset), size, 1000); break; } case ArrowDateTimeType::MICROSECONDS: { - DirectConversion(vector, array, scan_state, nested_offset, parent_offset); + DirectConversion(vector, array, chunk_offset, nested_offset, parent_offset); break; } case ArrowDateTimeType::NANOSECONDS: { auto tgt_ptr = FlatVector::GetData(vector); auto src_ptr = ArrowBufferData(array, 1) + - GetEffectiveOffset(array, NumericCast(parent_offset), scan_state, nested_offset); + GetEffectiveOffset(array, NumericCast(parent_offset), chunk_offset, nested_offset); for (idx_t row = 0; row < size; row++) { tgt_ptr[row].value = src_ptr[row] / 1000; } @@ -1005,25 +982,25 @@ static void ColumnArrowToDuckDB(Vector &vector, ArrowArray &array, ArrowArraySca auto precision = datetime_info.GetDateTimeType(); switch (precision) { case ArrowDateTimeType::SECONDS: { - IntervalConversionUs(vector, array, scan_state, nested_offset, NumericCast(parent_offset), size, + IntervalConversionUs(vector, array, chunk_offset, nested_offset, NumericCast(parent_offset), size, 1000000); break; } case ArrowDateTimeType::DAYS: case ArrowDateTimeType::MILLISECONDS: { - IntervalConversionUs(vector, array, scan_state, nested_offset, NumericCast(parent_offset), size, + IntervalConversionUs(vector, array, chunk_offset, nested_offset, NumericCast(parent_offset), size, 1000); break; } case ArrowDateTimeType::MICROSECONDS: { - IntervalConversionUs(vector, array, scan_state, nested_offset, NumericCast(parent_offset), size, + IntervalConversionUs(vector, array, chunk_offset, nested_offset, NumericCast(parent_offset), size, 1); break; } case ArrowDateTimeType::NANOSECONDS: { auto tgt_ptr = FlatVector::GetData(vector); auto src_ptr = ArrowBufferData(array, 1) + - GetEffectiveOffset(array, NumericCast(parent_offset), scan_state, nested_offset); + GetEffectiveOffset(array, NumericCast(parent_offset), chunk_offset, nested_offset); for (idx_t row = 0; row < size; row++) { tgt_ptr[row].micros = src_ptr[row] / 1000; tgt_ptr[row].days = 0; @@ -1032,12 +1009,12 @@ static void ColumnArrowToDuckDB(Vector &vector, ArrowArray &array, ArrowArraySca break; } case ArrowDateTimeType::MONTHS: { - IntervalConversionMonths(vector, array, scan_state, nested_offset, NumericCast(parent_offset), + IntervalConversionMonths(vector, array, chunk_offset, nested_offset, NumericCast(parent_offset), size); break; } case ArrowDateTimeType::MONTH_DAY_NANO: { - IntervalConversionMonthDayNanos(vector, array, scan_state, nested_offset, + IntervalConversionMonthDayNanos(vector, array, chunk_offset, nested_offset, NumericCast(parent_offset), size); break; } @@ -1054,22 +1031,25 @@ static void ColumnArrowToDuckDB(Vector &vector, ArrowArray &array, ArrowArraySca switch (bit_width) { case DecimalBitWidth::DECIMAL_32: { auto src_ptr = ArrowBufferData(array, 1) + - GetEffectiveOffset(array, NumericCast(parent_offset), scan_state, nested_offset); - ConvertDecimal(src_ptr, vector, array, size, nested_offset, parent_offset, scan_state, val_mask, bit_width); + GetEffectiveOffset(array, NumericCast(parent_offset), chunk_offset, nested_offset); + ConvertDecimal(src_ptr, vector, array, size, nested_offset, parent_offset, chunk_offset, val_mask, + bit_width); break; } case DecimalBitWidth::DECIMAL_64: { auto src_ptr = ArrowBufferData(array, 1) + - GetEffectiveOffset(array, NumericCast(parent_offset), scan_state, nested_offset); - ConvertDecimal(src_ptr, vector, array, size, nested_offset, parent_offset, scan_state, val_mask, bit_width); + GetEffectiveOffset(array, NumericCast(parent_offset), chunk_offset, nested_offset); + ConvertDecimal(src_ptr, vector, array, size, nested_offset, parent_offset, chunk_offset, val_mask, + bit_width); break; } case DecimalBitWidth::DECIMAL_128: { auto src_ptr = ArrowBufferData(array, 1) + - GetEffectiveOffset(array, NumericCast(parent_offset), scan_state, nested_offset); - ConvertDecimal(src_ptr, vector, array, size, nested_offset, parent_offset, scan_state, val_mask, bit_width); + GetEffectiveOffset(array, NumericCast(parent_offset), chunk_offset, nested_offset); + ConvertDecimal(src_ptr, vector, array, size, nested_offset, parent_offset, chunk_offset, val_mask, + bit_width); break; } default: @@ -1078,17 +1058,17 @@ static void ColumnArrowToDuckDB(Vector &vector, ArrowArray &array, ArrowArraySca break; } case LogicalTypeId::LIST: { - ArrowToDuckDBList(vector, array, array_state, size, arrow_type, nested_offset, parent_mask, + ArrowToDuckDBList(vector, array, chunk_offset, array_state, size, arrow_type, nested_offset, parent_mask, NumericCast(parent_offset)); break; } case LogicalTypeId::ARRAY: { - ArrowToDuckDBArray(vector, array, array_state, size, arrow_type, nested_offset, parent_mask, + ArrowToDuckDBArray(vector, array, chunk_offset, array_state, size, arrow_type, nested_offset, parent_mask, NumericCast(parent_offset)); break; } case LogicalTypeId::MAP: { - ArrowToDuckDBList(vector, array, array_state, size, arrow_type, nested_offset, parent_mask, + ArrowToDuckDBList(vector, array, chunk_offset, array_state, size, arrow_type, nested_offset, parent_mask, NumericCast(parent_offset)); ArrowToDuckDBMapVerify(vector, size); break; @@ -1104,7 +1084,8 @@ static void ColumnArrowToDuckDB(Vector &vector, ArrowArray &array, ArrowArraySca auto &child_type = struct_info.GetChild(child_idx); auto &child_state = array_state.GetChild(child_idx); - SetValidityMask(child_entry, child_array, scan_state, size, array.offset, nested_offset); + ArrowToDuckDBConversion::SetValidityMask(child_entry, child_array, chunk_offset, size, array.offset, + nested_offset); if (!struct_validity_mask.AllValid()) { auto &child_validity_mark = FlatVector::Validity(child_entry); for (idx_t i = 0; i < size; i++) { @@ -1114,19 +1095,21 @@ static void ColumnArrowToDuckDB(Vector &vector, ArrowArray &array, ArrowArraySca } } - auto array_physical_type = GetArrowArrayPhysicalType(child_type); + auto array_physical_type = child_type.GetPhysicalType(); switch (array_physical_type) { case ArrowArrayPhysicalType::DICTIONARY_ENCODED: - ColumnArrowToDuckDBDictionary(child_entry, child_array, child_state, size, child_type, nested_offset, - &struct_validity_mask, NumericCast(array.offset)); + ArrowToDuckDBConversion::ColumnArrowToDuckDBDictionary( + child_entry, child_array, chunk_offset, child_state, size, child_type, nested_offset, + &struct_validity_mask, NumericCast(array.offset)); break; case ArrowArrayPhysicalType::RUN_END_ENCODED: - ColumnArrowToDuckDBRunEndEncoded(child_entry, child_array, child_state, size, child_type, nested_offset, - &struct_validity_mask, NumericCast(array.offset)); + ColumnArrowToDuckDBRunEndEncoded(child_entry, child_array, chunk_offset, child_state, size, child_type, + nested_offset, &struct_validity_mask, + NumericCast(array.offset)); break; case ArrowArrayPhysicalType::DEFAULT: - ColumnArrowToDuckDB(child_entry, child_array, child_state, size, child_type, nested_offset, - &struct_validity_mask, NumericCast(array.offset), false); + ColumnArrowToDuckDB(child_entry, child_array, chunk_offset, child_state, size, child_type, + nested_offset, &struct_validity_mask, NumericCast(array.offset), false); break; default: throw NotImplementedException("ArrowArrayPhysicalType not recognized"); @@ -1148,19 +1131,22 @@ static void ColumnArrowToDuckDB(Vector &vector, ArrowArray &array, ArrowArraySca auto &child_state = array_state.GetChild(child_idx); auto &child_type = union_info.GetChild(child_idx); - SetValidityMask(child, child_array, scan_state, size, NumericCast(parent_offset), nested_offset); - auto array_physical_type = GetArrowArrayPhysicalType(child_type); + ArrowToDuckDBConversion::SetValidityMask(child, child_array, chunk_offset, size, + NumericCast(parent_offset), nested_offset); + auto array_physical_type = child_type.GetPhysicalType(); switch (array_physical_type) { case ArrowArrayPhysicalType::DICTIONARY_ENCODED: - ColumnArrowToDuckDBDictionary(child, child_array, child_state, size, child_type); + ArrowToDuckDBConversion::ColumnArrowToDuckDBDictionary(child, child_array, chunk_offset, child_state, + size, child_type); break; case ArrowArrayPhysicalType::RUN_END_ENCODED: - ColumnArrowToDuckDBRunEndEncoded(child, child_array, child_state, size, child_type); + ArrowToDuckDBConversion::ColumnArrowToDuckDBRunEndEncoded(child, child_array, chunk_offset, child_state, + size, child_type); break; case ArrowArrayPhysicalType::DEFAULT: - ColumnArrowToDuckDB(child, child_array, child_state, size, child_type, nested_offset, &validity_mask, - false); + ArrowToDuckDBConversion::ColumnArrowToDuckDB(child, child_array, chunk_offset, child_state, size, + child_type, nested_offset, &validity_mask, false); break; default: throw NotImplementedException("ArrowArrayPhysicalType not recognized"); @@ -1321,33 +1307,34 @@ static bool CanContainNull(const ArrowArray &array, const ValidityMask *parent_m return !parent_mask->AllValid(); } -static void ColumnArrowToDuckDBDictionary(Vector &vector, ArrowArray &array, ArrowArrayScanState &array_state, - idx_t size, const ArrowType &arrow_type, int64_t nested_offset, - const ValidityMask *parent_mask, uint64_t parent_offset) { +void ArrowToDuckDBConversion::ColumnArrowToDuckDBDictionary(Vector &vector, ArrowArray &array, idx_t chunk_offset, + ArrowArrayScanState &array_state, idx_t size, + const ArrowType &arrow_type, int64_t nested_offset, + const ValidityMask *parent_mask, uint64_t parent_offset) { if (vector.GetBuffer()) { vector.GetBuffer()->SetAuxiliaryData(make_uniq(array_state.owned_data)); } D_ASSERT(arrow_type.HasDictionary()); - auto &scan_state = array_state.state; const bool has_nulls = CanContainNull(array, parent_mask); if (array_state.CacheOutdated(array.dictionary)) { //! We need to set the dictionary data for this column auto base_vector = make_uniq(vector.GetType(), NumericCast(array.dictionary->length)); - SetValidityMask(*base_vector, *array.dictionary, scan_state, NumericCast(array.dictionary->length), 0, 0, - has_nulls); + ArrowToDuckDBConversion::SetValidityMask(*base_vector, *array.dictionary, chunk_offset, + NumericCast(array.dictionary->length), 0, 0, has_nulls); auto &dictionary_type = arrow_type.GetDictionary(); - auto arrow_physical_type = GetArrowArrayPhysicalType(dictionary_type); + auto arrow_physical_type = dictionary_type.GetPhysicalType(); + ; switch (arrow_physical_type) { case ArrowArrayPhysicalType::DICTIONARY_ENCODED: - ColumnArrowToDuckDBDictionary(*base_vector, *array.dictionary, array_state, + ColumnArrowToDuckDBDictionary(*base_vector, *array.dictionary, chunk_offset, array_state, NumericCast(array.dictionary->length), dictionary_type); break; case ArrowArrayPhysicalType::RUN_END_ENCODED: - ColumnArrowToDuckDBRunEndEncoded(*base_vector, *array.dictionary, array_state, + ColumnArrowToDuckDBRunEndEncoded(*base_vector, *array.dictionary, chunk_offset, array_state, NumericCast(array.dictionary->length), dictionary_type); break; case ArrowArrayPhysicalType::DEFAULT: - ColumnArrowToDuckDB(*base_vector, *array.dictionary, array_state, + ColumnArrowToDuckDB(*base_vector, *array.dictionary, chunk_offset, array_state, NumericCast(array.dictionary->length), dictionary_type); break; default: @@ -1359,12 +1346,12 @@ static void ColumnArrowToDuckDBDictionary(Vector &vector, ArrowArray &array, Arr //! Get Pointer to Indices of Dictionary auto indices = ArrowBufferData(array, 1) + GetTypeIdSize(offset_type.InternalType()) * - GetEffectiveOffset(array, NumericCast(parent_offset), scan_state, nested_offset); + GetEffectiveOffset(array, NumericCast(parent_offset), chunk_offset, nested_offset); SelectionVector sel; if (has_nulls) { ValidityMask indices_validity; - GetValidityMask(indices_validity, array, scan_state, size, NumericCast(parent_offset)); + GetValidityMask(indices_validity, array, chunk_offset, size, NumericCast(parent_offset)); if (parent_mask && !parent_mask->AllValid()) { auto &struct_validity_mask = *parent_mask; for (idx_t i = 0; i < size; i++) { @@ -1427,19 +1414,22 @@ void ArrowTableFunction::ArrowToDuckDB(ArrowScanLocalState &scan_state, const ar if (!array_state.owned_data) { array_state.owned_data = scan_state.chunk; } - - auto array_physical_type = GetArrowArrayPhysicalType(arrow_type); + auto array_physical_type = arrow_type.GetPhysicalType(); switch (array_physical_type) { case ArrowArrayPhysicalType::DICTIONARY_ENCODED: - ColumnArrowToDuckDBDictionary(output.data[idx], array, array_state, output.size(), arrow_type); + ArrowToDuckDBConversion::ColumnArrowToDuckDBDictionary(output.data[idx], array, scan_state.chunk_offset, + array_state, output.size(), arrow_type); break; case ArrowArrayPhysicalType::RUN_END_ENCODED: - ColumnArrowToDuckDBRunEndEncoded(output.data[idx], array, array_state, output.size(), arrow_type); + ArrowToDuckDBConversion::ColumnArrowToDuckDBRunEndEncoded(output.data[idx], array, scan_state.chunk_offset, + array_state, output.size(), arrow_type); break; case ArrowArrayPhysicalType::DEFAULT: - SetValidityMask(output.data[idx], array, scan_state, output.size(), parent_array.offset, -1); - ColumnArrowToDuckDB(output.data[idx], array, array_state, output.size(), arrow_type); + ArrowToDuckDBConversion::SetValidityMask(output.data[idx], array, scan_state.chunk_offset, output.size(), + parent_array.offset, -1); + ArrowToDuckDBConversion::ColumnArrowToDuckDB(output.data[idx], array, scan_state.chunk_offset, array_state, + output.size(), arrow_type); break; default: throw NotImplementedException("ArrowArrayPhysicalType not recognized"); diff --git a/src/duckdb/src/function/table/table_scan.cpp b/src/duckdb/src/function/table/table_scan.cpp index 08f8ae4c1..d91de1392 100644 --- a/src/duckdb/src/function/table/table_scan.cpp +++ b/src/duckdb/src/function/table/table_scan.cpp @@ -109,14 +109,19 @@ class TableScanGlobalState : public GlobalTableFunctionState { class DuckIndexScanState : public TableScanGlobalState { public: DuckIndexScanState(ClientContext &context, const FunctionData *bind_data_p) - : TableScanGlobalState(context, bind_data_p), next_batch_index(0), finished(false) { + : TableScanGlobalState(context, bind_data_p), next_batch_index(0), arena(Allocator::Get(context)), + row_ids(nullptr), row_id_count(0), finished(false) { } //! The batch index of the next Sink. //! Also determines the offset of the next chunk. I.e., offset = next_batch_index * STANDARD_VECTOR_SIZE. atomic next_batch_index; - //! The total scanned row IDs. - unsafe_vector row_ids; + //! The arena allocator containing the memory of the row IDs. + ArenaAllocator arena; + //! A pointer to the row IDs. + row_t *row_ids; + //! The number of scanned row IDs. + idx_t row_id_count; //! The column IDs of the to-be-scanned columns. vector column_ids; //! True, if no more row IDs must be scanned. @@ -154,7 +159,6 @@ class DuckIndexScanState : public TableScanGlobalState { auto &storage = duck_table.GetStorage(); auto &l_state = data_p.local_state->Cast(); - auto row_id_count = row_ids.size(); idx_t scan_count = 0; idx_t offset = 0; @@ -173,7 +177,7 @@ class DuckIndexScanState : public TableScanGlobalState { } if (scan_count != 0) { - auto row_id_data = (data_ptr_t)&row_ids[0 + offset]; // NOLINT - this is not pretty + auto row_id_data = reinterpret_cast(row_ids + offset); Vector local_vector(LogicalType::ROW_TYPE, row_id_data); if (CanRemoveFilterColumns()) { @@ -198,13 +202,11 @@ class DuckIndexScanState : public TableScanGlobalState { } double TableScanProgress(ClientContext &context, const FunctionData *bind_data_p) const override { - auto total_rows = row_ids.size(); - if (total_rows == 0) { + if (row_id_count == 0) { return 100; } - auto scanned_rows = next_batch_index * STANDARD_VECTOR_SIZE; - auto percentage = 100 * (static_cast(scanned_rows) / static_cast(total_rows)); + auto percentage = 100 * (static_cast(scanned_rows) / static_cast(row_id_count)); return percentage > 100 ? 100 : percentage; } @@ -339,26 +341,20 @@ unique_ptr DuckTableScanInitGlobal(ClientContext &cont } unique_ptr DuckIndexScanInitGlobal(ClientContext &context, TableFunctionInitInput &input, - const TableScanBindData &bind_data, - unsafe_vector &row_ids) { + const TableScanBindData &bind_data, set &row_ids) { auto g_state = make_uniq(context, input.bind_data.get()); + g_state->finished = row_ids.empty() ? true : false; + if (!row_ids.empty()) { - // Duplicate-eliminate row IDs. - unordered_set row_id_set; - auto it = row_ids.begin(); - while (it != row_ids.end()) { - if (row_id_set.find(*it) == row_id_set.end()) { - row_id_set.insert(*it++); - continue; - } - // Found a duplicate. - it = row_ids.erase(it); - } + auto row_id_ptr = g_state->arena.AllocateAligned(row_ids.size() * sizeof(row_t)); + g_state->row_ids = reinterpret_cast(row_id_ptr); + g_state->row_id_count = row_ids.size(); - std::sort(row_ids.begin(), row_ids.end()); - g_state->row_ids = std::move(row_ids); + idx_t row_id_count = 0; + for (const auto row_id : row_ids) { + g_state->row_ids[row_id_count++] = row_id; + } } - g_state->finished = g_state->row_ids.empty() ? true : false; auto &duck_table = bind_data.table.Cast(); if (input.CanRemoveFilterColumns()) { @@ -482,7 +478,7 @@ vector> ExtractFilterExpressions(const ColumnDefinition & } bool TryScanIndex(ART &art, const ColumnList &column_list, TableFunctionInitInput &input, TableFilterSet &filter_set, - idx_t max_count, unsafe_vector &row_ids) { + idx_t max_count, set &row_ids) { // FIXME: No support for index scans on compound ARTs. // See note above on multi-filter support. if (art.unbound_expressions.size() > 1) { @@ -579,7 +575,7 @@ unique_ptr TableScanInitGlobal(ClientContext &context, auto &column_list = duck_table.GetColumns(); bool index_scan = false; - unsafe_vector row_ids; + set row_ids; info->GetIndexes().BindAndScan(context, *info, [&](ART &art) { index_scan = TryScanIndex(art, column_list, input, filter_set, max_count, row_ids); diff --git a/src/duckdb/src/function/table/version/pragma_version.cpp b/src/duckdb/src/function/table/version/pragma_version.cpp index 82ca7e67b..7285dd244 100644 --- a/src/duckdb/src/function/table/version/pragma_version.cpp +++ b/src/duckdb/src/function/table/version/pragma_version.cpp @@ -1,5 +1,5 @@ #ifndef DUCKDB_PATCH_VERSION -#define DUCKDB_PATCH_VERSION "0-dev1655" +#define DUCKDB_PATCH_VERSION "0-dev2165" #endif #ifndef DUCKDB_MINOR_VERSION #define DUCKDB_MINOR_VERSION 4 @@ -8,10 +8,10 @@ #define DUCKDB_MAJOR_VERSION 1 #endif #ifndef DUCKDB_VERSION -#define DUCKDB_VERSION "v1.4.0-dev1655" +#define DUCKDB_VERSION "v1.4.0-dev2165" #endif #ifndef DUCKDB_SOURCE_ID -#define DUCKDB_SOURCE_ID "59382ca326" +#define DUCKDB_SOURCE_ID "9162313f76" #endif #include "duckdb/function/table/system_functions.hpp" #include "duckdb/main/database.hpp" diff --git a/src/duckdb/src/include/duckdb.h b/src/duckdb/src/include/duckdb.h index 2737575c5..0370c5e89 100644 --- a/src/duckdb/src/include/duckdb.h +++ b/src/duckdb/src/include/duckdb.h @@ -564,6 +564,12 @@ typedef struct _duckdb_error_data { void *internal_ptr; } * duckdb_error_data; +//! Holds a bound expression. +//! Must be destroyed with `duckdb_destroy_expression`. +typedef struct _duckdb_expression { + void *internal_ptr; +} * duckdb_expression; + //===--------------------------------------------------------------------===// // C API extension information //===--------------------------------------------------------------------===// @@ -702,6 +708,14 @@ typedef void (*duckdb_replacement_callback_t)(duckdb_replacement_scan_info info, // Arrow-related types //===--------------------------------------------------------------------===// +//! Forward declare Arrow structs +//! It is important to notice that these structs are not defined by DuckDB but are actually Arrow external objects. +//! They're defined by the C Data Interface Arrow spec: https://arrow.apache.org/docs/format/CDataInterface.html + +struct ArrowArray; + +struct ArrowSchema; + //! Holds an arrow query result. Must be destroyed with `duckdb_destroy_arrow`. typedef struct _duckdb_arrow { void *internal_ptr; @@ -717,11 +731,25 @@ typedef struct _duckdb_arrow_schema { void *internal_ptr; } * duckdb_arrow_schema; -//! Holds an arrow array. Remember to release the respective ArrowArray object. +//! Holds an arrow converted schema (i.e., duckdb::ArrowTableSchema). +//! In practice, this object holds the information necessary to do proper conversion between Arrow Types and DuckDB +//! Types. Check duckdb/function/table/arrow/arrow_duck_schema.hpp for more details! Must be destroyed with +//! `duckdb_destroy_arrow_converted_schema` +typedef struct _duckdb_arrow_converted_schema { + void *internal_ptr; +} * duckdb_arrow_converted_schema; + +//! Holds an arrow array. Remember to release the respective ArrowSchema object. typedef struct _duckdb_arrow_array { void *internal_ptr; } * duckdb_arrow_array; +//! The arrow options used when transforming the DuckDB schema and datachunks into Arrow schema and arrays. +//! Used in `duckdb_to_arrow_schema` and `duckdb_data_chunk_to_arrow` +typedef struct _duckdb_arrow_options { + void *internal_ptr; +} * duckdb_arrow_options; + //===--------------------------------------------------------------------===// // DuckDB extension access //===--------------------------------------------------------------------===// @@ -855,6 +883,14 @@ Retrieves the client context of the connection. DUCKDB_C_API void duckdb_connection_get_client_context(duckdb_connection connection, duckdb_client_context *out_context); +/*! +Retrieves the arrow options of the connection. + +* @param connection The connection. +*/ +DUCKDB_C_API void duckdb_connection_get_arrow_options(duckdb_connection connection, + duckdb_arrow_options *out_arrow_options); + /*! Returns the connection id of the client context. @@ -870,6 +906,13 @@ Destroys the client context and deallocates its memory. */ DUCKDB_C_API void duckdb_destroy_client_context(duckdb_client_context *context); +/*! +Destroys the arrow options and deallocates its memory. + +* @param arrow_options The arrow options to destroy. +*/ +DUCKDB_C_API void duckdb_destroy_arrow_options(duckdb_arrow_options *arrow_options); + /*! Returns the version of the linked DuckDB, with a version postfix for dev versions @@ -1070,6 +1113,15 @@ Returns `NULL` if the column is out of range. */ DUCKDB_C_API duckdb_logical_type duckdb_column_logical_type(duckdb_result *result, idx_t col); +/*! +Returns the arrow options associated with the given result. These options are definitions of how the arrow arrays/schema +should be produced. +* @param result The result object to fetch arrow options from. +* @return The arrow options associated with the given result. This must be destroyed with +`duckdb_destroy_arrow_options`. +*/ +DUCKDB_C_API duckdb_arrow_options duckdb_result_get_arrow_options(duckdb_result *result); + /*! Returns the number of columns present in a the result object. @@ -3525,6 +3577,23 @@ If the set is incomplete or a function with this name already exists DuckDBError */ DUCKDB_C_API duckdb_state duckdb_register_scalar_function_set(duckdb_connection con, duckdb_scalar_function_set set); +/*! +Returns the number of input arguments of the scalar function. + +* @param info The bind info. +* @return The number of input arguments. +*/ +DUCKDB_C_API idx_t duckdb_scalar_function_bind_get_argument_count(duckdb_bind_info info); + +/*! +Returns the input argument at index of the scalar function. + +* @param info The bind info. +* @param index The argument index. +* @return The input argument at index. Must be destroyed with `duckdb_destroy_expression`. +*/ +DUCKDB_C_API duckdb_expression duckdb_scalar_function_bind_get_argument(duckdb_bind_info info, idx_t index); + //===--------------------------------------------------------------------===// // Selection Vector Interface //===--------------------------------------------------------------------===// @@ -4485,6 +4554,66 @@ DUCKDB_C_API char *duckdb_table_description_get_column_name(duckdb_table_descrip // Arrow Interface //===--------------------------------------------------------------------===// +/*! +Transforms a DuckDB Schema into an Arrow Schema + +* @param arrow_options The Arrow settings used to produce arrow. +* @param types The DuckDB logical types for each column in the schema. +* @param names The names for each column in the schema. +* @param column_count The number of columns that exist in the schema. +* @param out_schema The resulting arrow schema. Must be destroyed with `out_schema->release(out_schema)`. +* @return The error data. Must be destroyed with `duckdb_destroy_error_data`. +*/ +DUCKDB_C_API duckdb_error_data duckdb_to_arrow_schema(duckdb_arrow_options arrow_options, duckdb_logical_type *types, + const char **names, idx_t column_count, + struct ArrowSchema *out_schema); + +/*! +Transforms a DuckDB data chunk into an Arrow array. + +* @param arrow_options The Arrow settings used to produce arrow. +* @param chunk The DuckDB data chunk to convert. +* @param out_arrow_array The output Arrow structure that will hold the converted data. Must be released with +`out_arrow_array->release(out_arrow_array)` +* @return The error data. Must be destroyed with `duckdb_destroy_error_data`. +*/ +DUCKDB_C_API duckdb_error_data duckdb_data_chunk_to_arrow(duckdb_arrow_options arrow_options, duckdb_data_chunk chunk, + struct ArrowArray *out_arrow_array); + +/*! +Transforms an Arrow Schema into a DuckDB Schema. + +* @param connection The connection to get the transformation settings from. +* @param schema The input Arrow schema. Must be released with `schema->release(schema)`. +* @param out_types The Arrow converted schema with extra information about the arrow types. Must be destroyed with +`duckdb_destroy_arrow_converted_schema`. +* @return The error data. Must be destroyed with `duckdb_destroy_error_data`. +*/ +DUCKDB_C_API duckdb_error_data duckdb_schema_from_arrow(duckdb_connection connection, struct ArrowSchema *schema, + duckdb_arrow_converted_schema *out_types); + +/*! +Transforms an Arrow array into a DuckDB data chunk. The data chunk will retain ownership of the underlying Arrow data. + +* @param connection The connection to get the transformation settings from. +* @param arrow_array The input Arrow array. Data ownership is passed on to DuckDB's DataChunk, the underlying object +does not need to be released and won't have ownership of the data. +* @param converted_schema The Arrow converted schema with extra information about the arrow types. +* @param out_chunk The resulting DuckDB data chunk. Must be destroyed by duckdb_destroy_data_chunk. +* @return The error data. Must be destroyed with `duckdb_destroy_error_data`. +*/ +DUCKDB_C_API duckdb_error_data duckdb_data_chunk_from_arrow(duckdb_connection connection, + struct ArrowArray *arrow_array, + duckdb_arrow_converted_schema converted_schema, + duckdb_data_chunk *out_chunk); + +/*! +Destroys the arrow converted schema and de-allocates all memory allocated for that arrow converted schema. + +* @param arrow_converted_schema The arrow converted schema to destroy. +*/ +DUCKDB_C_API void duckdb_destroy_arrow_converted_schema(duckdb_arrow_converted_schema *arrow_converted_schema); + #ifndef DUCKDB_API_NO_DEPRECATED /*! **DEPRECATION NOTICE**: This method is scheduled for removal in a future release. @@ -4889,6 +5018,44 @@ Destroys the cast function object. */ DUCKDB_C_API void duckdb_destroy_cast_function(duckdb_cast_function *cast_function); +//===--------------------------------------------------------------------===// +// Expression Interface +//===--------------------------------------------------------------------===// + +/*! +Destroys the expression and de-allocates its memory. + +* @param expr A pointer to the expression. +*/ +DUCKDB_C_API void duckdb_destroy_expression(duckdb_expression *expr); + +/*! +Returns the return type of an expression. + +* @param expr The expression. +* @return The return type. Must be destroyed with `duckdb_destroy_logical_type`. +*/ +DUCKDB_C_API duckdb_logical_type duckdb_expression_return_type(duckdb_expression expr); + +/*! +Returns whether the expression is foldable into a value or not. + +* @param expr The expression. +* @return True, if the expression is foldable, else false. +*/ +DUCKDB_C_API bool duckdb_expression_is_foldable(duckdb_expression expr); + +/*! +Folds an expression creating a folded value. + +* @param context The client context. +* @param expr The expression. Must be foldable. +* @param out_value The folded value, if folding was successful. Must be destroyed with `duckdb_destroy_value`. +* @return The error data. Must be destroyed with `duckdb_destroy_error_data`. +*/ +DUCKDB_C_API duckdb_error_data duckdb_expression_fold(duckdb_client_context context, duckdb_expression expr, + duckdb_value *out_value); + #endif #ifdef __cplusplus diff --git a/src/duckdb/src/include/duckdb/catalog/catalog.hpp b/src/duckdb/src/include/duckdb/catalog/catalog.hpp index c36d683e0..c44c22267 100644 --- a/src/duckdb/src/include/duckdb/catalog/catalog.hpp +++ b/src/duckdb/src/include/duckdb/catalog/catalog.hpp @@ -368,6 +368,8 @@ class Catalog { const string &catalog_name); DUCKDB_API static vector> GetAllSchemas(ClientContext &context); + static vector> GetAllEntries(ClientContext &context, CatalogType catalog_type); + virtual void Verify(); static CatalogException UnrecognizedConfigurationError(ClientContext &context, const string &name); diff --git a/src/duckdb/src/include/duckdb/common/adbc/adbc.hpp b/src/duckdb/src/include/duckdb/common/adbc/adbc.hpp index 365de229a..391744f48 100644 --- a/src/duckdb/src/include/duckdb/common/adbc/adbc.hpp +++ b/src/duckdb/src/include/duckdb/common/adbc/adbc.hpp @@ -10,10 +10,78 @@ #include "duckdb/common/adbc/adbc.h" +#include "duckdb/main/capi/capi_internal.hpp" + #include namespace duckdb_adbc { +class AppenderWrapper { +public: + AppenderWrapper(duckdb_connection conn, const char *schema, const char *table) : appender(nullptr) { + if (duckdb_appender_create(conn, schema, table, &appender) != DuckDBSuccess) { + appender = nullptr; + } + } + ~AppenderWrapper() { + if (appender) { + duckdb_appender_destroy(&appender); + } + } + + duckdb_appender Get() const { + return appender; + } + bool Valid() const { + return appender != nullptr; + } + +private: + duckdb_appender appender; +}; + +class DataChunkWrapper { +public: + DataChunkWrapper() : chunk(nullptr) { + } + + ~DataChunkWrapper() { + if (chunk) { + duckdb_destroy_data_chunk(&chunk); + } + } + + explicit operator duckdb_data_chunk() const { + return chunk; + } + + duckdb_data_chunk chunk; +}; + +class ConvertedSchemaWrapper { +public: + ConvertedSchemaWrapper() : schema(nullptr) { + } + ~ConvertedSchemaWrapper() { + if (schema) { + duckdb_destroy_arrow_converted_schema(&schema); + } + } + duckdb_arrow_converted_schema *GetPtr() { + return &schema; + } + + explicit operator duckdb_arrow_converted_schema() const { + return schema; + } + duckdb_arrow_converted_schema Get() const { + return schema; + } + +private: + duckdb_arrow_converted_schema schema; +}; + AdbcStatusCode DatabaseNew(struct AdbcDatabase *database, struct AdbcError *error); AdbcStatusCode DatabaseSetOption(struct AdbcDatabase *database, const char *key, const char *value, @@ -90,4 +158,3 @@ void InitializeADBCError(AdbcError *error); //! This method should only be called when the string is guaranteed to not be NULL void SetError(struct AdbcError *error, const std::string &message); -// void SetError(struct AdbcError *error, const char *message); diff --git a/src/duckdb/src/include/duckdb/common/arrow/appender/append_data.hpp b/src/duckdb/src/include/duckdb/common/arrow/appender/append_data.hpp index c136100ae..4a1e594d0 100644 --- a/src/duckdb/src/include/duckdb/common/arrow/appender/append_data.hpp +++ b/src/duckdb/src/include/duckdb/common/arrow/appender/append_data.hpp @@ -42,6 +42,7 @@ struct ArrowAppendData { arrow_buffers.resize(3); } +public: //! Getters for the Buffers ArrowBuffer &GetValidityBuffer() { return arrow_buffers[0]; @@ -63,6 +64,36 @@ struct ArrowAppendData { return arrow_buffers[3]; } +public: + static void GetBitPosition(idx_t row_idx, idx_t ¤t_byte, uint8_t ¤t_bit) { + current_byte = row_idx / 8; + current_bit = row_idx % 8; + } + + static void UnsetBit(uint8_t *data, idx_t current_byte, uint8_t current_bit) { + data[current_byte] &= ~((uint64_t)1 << current_bit); + } + + static void NextBit(idx_t ¤t_byte, uint8_t ¤t_bit) { + current_bit++; + if (current_bit == 8) { + current_byte++; + current_bit = 0; + } + } + + static void ResizeValidity(ArrowBuffer &buffer, idx_t row_count) { + auto byte_count = (row_count + 7) / 8; + buffer.resize(byte_count, 0xFF); + } + + void SetNull(uint8_t *validity_data, idx_t current_byte, uint8_t current_bit) { + UnsetBit(validity_data, current_byte, current_bit); + null_count++; + } + void AppendValidity(UnifiedVectorFormat &format, idx_t from, idx_t to); + +public: idx_t row_count = 0; idx_t null_count = 0; @@ -93,59 +124,4 @@ struct ArrowAppendData { vector arrow_buffers; }; -//===--------------------------------------------------------------------===// -// Append Helper Functions -//===--------------------------------------------------------------------===// - -static void GetBitPosition(idx_t row_idx, idx_t ¤t_byte, uint8_t ¤t_bit) { - current_byte = row_idx / 8; - current_bit = row_idx % 8; -} - -static void UnsetBit(uint8_t *data, idx_t current_byte, uint8_t current_bit) { - data[current_byte] &= ~((uint64_t)1 << current_bit); -} - -static void NextBit(idx_t ¤t_byte, uint8_t ¤t_bit) { - current_bit++; - if (current_bit == 8) { - current_byte++; - current_bit = 0; - } -} - -static void ResizeValidity(ArrowBuffer &buffer, idx_t row_count) { - auto byte_count = (row_count + 7) / 8; - buffer.resize(byte_count, 0xFF); -} - -static void SetNull(ArrowAppendData &append_data, uint8_t *validity_data, idx_t current_byte, uint8_t current_bit) { - UnsetBit(validity_data, current_byte, current_bit); - append_data.null_count++; -} - -static void AppendValidity(ArrowAppendData &append_data, UnifiedVectorFormat &format, idx_t from, idx_t to) { - // resize the buffer, filling the validity buffer with all valid values - idx_t size = to - from; - ResizeValidity(append_data.GetValidityBuffer(), append_data.row_count + size); - if (format.validity.AllValid()) { - // if all values are valid we don't need to do anything else - return; - } - - // otherwise we iterate through the validity mask - auto validity_data = (uint8_t *)append_data.GetValidityBuffer().data(); - uint8_t current_bit; - idx_t current_byte; - GetBitPosition(append_data.row_count, current_byte, current_bit); - for (idx_t i = from; i < to; i++) { - auto source_idx = format.sel->get_index(i); - // append the validity mask - if (!format.validity.RowIsValid(source_idx)) { - SetNull(append_data, validity_data, current_byte, current_bit); - } - NextBit(current_byte, current_bit); - } -} - } // namespace duckdb diff --git a/src/duckdb/src/include/duckdb/common/arrow/appender/enum_data.hpp b/src/duckdb/src/include/duckdb/common/arrow/appender/enum_data.hpp index a5d066abb..20cb1a1b0 100644 --- a/src/duckdb/src/include/duckdb/common/arrow/appender/enum_data.hpp +++ b/src/duckdb/src/include/duckdb/common/arrow/appender/enum_data.hpp @@ -35,7 +35,7 @@ struct ArrowEnumData : public ArrowScalarBaseData { auto &main_buffer = append_data.GetMainBuffer(); auto &aux_buffer = append_data.GetAuxBuffer(); // resize the validity mask and set up the validity buffer for iteration - ResizeValidity(append_data.GetValidityBuffer(), append_data.row_count + size); + ArrowAppendData::ResizeValidity(append_data.GetValidityBuffer(), append_data.row_count + size); // resize the offset buffer - the offset buffer holds the offsets into the child array main_buffer.resize(main_buffer.size() + sizeof(int32_t) * (size + 1)); diff --git a/src/duckdb/src/include/duckdb/common/arrow/appender/list_data.hpp b/src/duckdb/src/include/duckdb/common/arrow/appender/list_data.hpp index 21274da48..627e5fbbc 100644 --- a/src/duckdb/src/include/duckdb/common/arrow/appender/list_data.hpp +++ b/src/duckdb/src/include/duckdb/common/arrow/appender/list_data.hpp @@ -28,7 +28,7 @@ struct ArrowListData { input.ToUnifiedFormat(input_size, format); idx_t size = to - from; vector child_indices; - AppendValidity(append_data, format, from, to); + append_data.AppendValidity(format, from, to); AppendOffsets(append_data, format, from, to, child_indices); // append the child vector of the list diff --git a/src/duckdb/src/include/duckdb/common/arrow/appender/list_view_data.hpp b/src/duckdb/src/include/duckdb/common/arrow/appender/list_view_data.hpp index f46b316dd..f326b7648 100644 --- a/src/duckdb/src/include/duckdb/common/arrow/appender/list_view_data.hpp +++ b/src/duckdb/src/include/duckdb/common/arrow/appender/list_view_data.hpp @@ -30,7 +30,7 @@ struct ArrowListViewData { input.ToUnifiedFormat(input_size, format); idx_t size = to - from; vector child_indices; - AppendValidity(append_data, format, from, to); + append_data.AppendValidity(format, from, to); AppendListMetadata(append_data, format, from, to, child_indices); // append the child vector of the list diff --git a/src/duckdb/src/include/duckdb/common/arrow/appender/map_data.hpp b/src/duckdb/src/include/duckdb/common/arrow/appender/map_data.hpp index 7f82f401f..b73b0016b 100644 --- a/src/duckdb/src/include/duckdb/common/arrow/appender/map_data.hpp +++ b/src/duckdb/src/include/duckdb/common/arrow/appender/map_data.hpp @@ -39,7 +39,7 @@ struct ArrowMapData { UnifiedVectorFormat format; input.ToUnifiedFormat(input_size, format); idx_t size = to - from; - AppendValidity(append_data, format, from, to); + append_data.AppendValidity(format, from, to); vector child_indices; ArrowListData::AppendOffsets(append_data, format, from, to, child_indices); diff --git a/src/duckdb/src/include/duckdb/common/arrow/appender/scalar_data.hpp b/src/duckdb/src/include/duckdb/common/arrow/appender/scalar_data.hpp index a6729b460..e28c002ee 100644 --- a/src/duckdb/src/include/duckdb/common/arrow/appender/scalar_data.hpp +++ b/src/duckdb/src/include/duckdb/common/arrow/appender/scalar_data.hpp @@ -96,7 +96,7 @@ struct ArrowScalarBaseData { input.ToUnifiedFormat(input_size, format); // append the validity mask - AppendValidity(append_data, format, from, to); + append_data.AppendValidity(format, from, to); // append the main data auto &main_buffer = append_data.GetMainBuffer(); diff --git a/src/duckdb/src/include/duckdb/common/arrow/appender/varchar_data.hpp b/src/duckdb/src/include/duckdb/common/arrow/appender/varchar_data.hpp index 45e9c14a9..87e45f5dc 100644 --- a/src/duckdb/src/include/duckdb/common/arrow/appender/varchar_data.hpp +++ b/src/duckdb/src/include/duckdb/common/arrow/appender/varchar_data.hpp @@ -59,7 +59,7 @@ struct ArrowVarcharData { auto &aux_buffer = append_data.GetAuxBuffer(); // resize the validity mask and set up the validity buffer for iteration - ResizeValidity(validity_buffer, append_data.row_count + size); + ArrowAppendData::ResizeValidity(validity_buffer, append_data.row_count + size); auto validity_data = (uint8_t *)validity_buffer.data(); // resize the offset buffer - the offset buffer holds the offsets into the child array @@ -80,8 +80,8 @@ struct ArrowVarcharData { if (!format.validity.RowIsValid(source_idx)) { uint8_t current_bit; idx_t current_byte; - GetBitPosition(append_data.row_count + i - from, current_byte, current_bit); - SetNull(append_data, validity_data, current_byte, current_bit); + ArrowAppendData::GetBitPosition(append_data.row_count + i - from, current_byte, current_bit); + append_data.SetNull(validity_data, current_byte, current_bit); offset_data[offset_idx] = last_offset; continue; } @@ -141,7 +141,7 @@ struct ArrowVarcharToStringViewData { auto &validity_buffer = append_data.GetValidityBuffer(); auto &aux_buffer = append_data.GetAuxBuffer(); // resize the validity mask and set up the validity buffer for iteration - ResizeValidity(validity_buffer, append_data.row_count + size); + ArrowAppendData::ResizeValidity(validity_buffer, append_data.row_count + size); auto validity_data = (uint8_t *)validity_buffer.data(); main_buffer.resize(main_buffer.size() + sizeof(arrow_string_view_t) * (size)); @@ -155,8 +155,8 @@ struct ArrowVarcharToStringViewData { // Null value uint8_t current_bit; idx_t current_byte; - GetBitPosition(result_idx, current_byte, current_bit); - SetNull(append_data, validity_data, current_byte, current_bit); + ArrowAppendData::GetBitPosition(result_idx, current_byte, current_bit); + append_data.SetNull(validity_data, current_byte, current_bit); // We have to set these bytes to 0, for some reason arrow_data[result_idx] = arrow_string_view_t(0, ""); continue; diff --git a/src/duckdb/src/include/duckdb/common/arrow/arrow_wrapper.hpp b/src/duckdb/src/include/duckdb/common/arrow/arrow_wrapper.hpp index 0639da539..9d0fd49f5 100644 --- a/src/duckdb/src/include/duckdb/common/arrow/arrow_wrapper.hpp +++ b/src/duckdb/src/include/duckdb/common/arrow/arrow_wrapper.hpp @@ -33,6 +33,16 @@ class ArrowArrayWrapper { ArrowArrayWrapper(ArrowArrayWrapper &&other) noexcept : arrow_array(other.arrow_array) { other.arrow_array.release = nullptr; } + ArrowArrayWrapper &operator=(ArrowArrayWrapper &&other) noexcept { + if (this != &other) { + if (arrow_array.release) { + arrow_array.release(&arrow_array); + } + arrow_array = other.arrow_array; + other.arrow_array.release = nullptr; + } + return *this; + } ~ArrowArrayWrapper(); }; diff --git a/src/duckdb/src/include/duckdb/common/enum_util.hpp b/src/duckdb/src/include/duckdb/common/enum_util.hpp index 98eb64bbc..89ff7f054 100644 --- a/src/duckdb/src/include/duckdb/common/enum_util.hpp +++ b/src/duckdb/src/include/duckdb/common/enum_util.hpp @@ -64,6 +64,8 @@ enum class AlterViewType : uint8_t; enum class AppenderType : uint8_t; +enum class ArrowArrayPhysicalType : uint8_t; + enum class ArrowDateTimeType : uint8_t; enum class ArrowOffsetSize : uint8_t; @@ -479,6 +481,9 @@ const char* EnumUtil::ToChars(AlterViewType value); template<> const char* EnumUtil::ToChars(AppenderType value); +template<> +const char* EnumUtil::ToChars(ArrowArrayPhysicalType value); + template<> const char* EnumUtil::ToChars(ArrowDateTimeType value); @@ -1077,6 +1082,9 @@ AlterViewType EnumUtil::FromString(const char *value); template<> AppenderType EnumUtil::FromString(const char *value); +template<> +ArrowArrayPhysicalType EnumUtil::FromString(const char *value); + template<> ArrowDateTimeType EnumUtil::FromString(const char *value); diff --git a/src/duckdb/src/include/duckdb/common/enums/checkpoint_type.hpp b/src/duckdb/src/include/duckdb/common/enums/checkpoint_type.hpp index 9ad3b4529..7610b1abe 100644 --- a/src/duckdb/src/include/duckdb/common/enums/checkpoint_type.hpp +++ b/src/duckdb/src/include/duckdb/common/enums/checkpoint_type.hpp @@ -32,7 +32,9 @@ enum class CheckpointType { FULL_CHECKPOINT, //! Concurrent checkpoints write committed data to disk but do less clean-up //! They can be run even when active transactions need to read old data - CONCURRENT_CHECKPOINT + CONCURRENT_CHECKPOINT, + //! Only run vacuum - this can be triggered for in-memory tables + VACUUM_ONLY }; } // namespace duckdb diff --git a/src/duckdb/src/include/duckdb/common/exception_format_value.hpp b/src/duckdb/src/include/duckdb/common/exception_format_value.hpp index 34be78c09..3f5db0a43 100644 --- a/src/duckdb/src/include/duckdb/common/exception_format_value.hpp +++ b/src/duckdb/src/include/duckdb/common/exception_format_value.hpp @@ -49,6 +49,7 @@ enum class ExceptionFormatValueType : uint8_t { struct ExceptionFormatValue { DUCKDB_API ExceptionFormatValue(double dbl_val); // NOLINT DUCKDB_API ExceptionFormatValue(int64_t int_val); // NOLINT + DUCKDB_API ExceptionFormatValue(idx_t uint_val); // NOLINT DUCKDB_API ExceptionFormatValue(string str_val); // NOLINT DUCKDB_API ExceptionFormatValue(hugeint_t hg_val); // NOLINT DUCKDB_API ExceptionFormatValue(uhugeint_t uhg_val); // NOLINT @@ -56,7 +57,7 @@ struct ExceptionFormatValue { ExceptionFormatValueType type; double dbl_val = 0; - int64_t int_val = 0; + hugeint_t int_val = 0; string str_val; public: @@ -86,6 +87,8 @@ DUCKDB_API ExceptionFormatValue ExceptionFormatValue::CreateFormatValue(const ch template <> DUCKDB_API ExceptionFormatValue ExceptionFormatValue::CreateFormatValue(char *value); template <> +DUCKDB_API ExceptionFormatValue ExceptionFormatValue::CreateFormatValue(idx_t value); +template <> DUCKDB_API ExceptionFormatValue ExceptionFormatValue::CreateFormatValue(hugeint_t value); template <> DUCKDB_API ExceptionFormatValue ExceptionFormatValue::CreateFormatValue(uhugeint_t value); diff --git a/src/duckdb/src/include/duckdb/common/multi_file/multi_file_function.hpp b/src/duckdb/src/include/duckdb/common/multi_file/multi_file_function.hpp index 172601c71..534b979f8 100644 --- a/src/duckdb/src/include/duckdb/common/multi_file/multi_file_function.hpp +++ b/src/duckdb/src/include/duckdb/common/multi_file/multi_file_function.hpp @@ -181,11 +181,11 @@ class MultiFileFunction : public TableFunction { std::move(file_options), std::move(options), std::move(interface)); } - static unique_ptr MultiFileBindCopy(ClientContext &context, CopyInfo &info, + static unique_ptr MultiFileBindCopy(ClientContext &context, CopyFromFunctionBindInput &input, vector &expected_names, vector &expected_types) { auto multi_file_reader = MultiFileReader::CreateDefault("COPY"); - vector paths = {info.file_path}; + vector paths = {input.info.file_path}; auto file_list = multi_file_reader->CreateFileList(context, paths); auto interface = OP::InitializeInterface(context, *multi_file_reader, *file_list); @@ -193,7 +193,7 @@ class MultiFileFunction : public TableFunction { auto options = interface->InitializeOptions(context, nullptr); MultiFileOptions file_options; - for (auto &option : info.options) { + for (auto &option : input.info.options) { auto loption = StringUtil::Lower(option.first); if (interface->ParseCopyOption(context, loption, option.second, *options, expected_names, expected_types)) { continue; 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 60ad2ac71..1e6b705f4 100644 --- a/src/duckdb/src/include/duckdb/execution/index/art/art.hpp +++ b/src/duckdb/src/include/duckdb/execution/index/art/art.hpp @@ -72,7 +72,7 @@ class ART : public BoundIndex { unique_ptr TryInitializeScan(const Expression &expr, const Expression &filter_expr); //! Perform a lookup on the ART, fetching up to max_count row IDs. //! If all row IDs were fetched, it return true, else false. - bool Scan(IndexScanState &state, idx_t max_count, unsafe_vector &row_ids); + bool Scan(IndexScanState &state, idx_t max_count, set &row_ids); //! Appends data to the locked index. ErrorData Append(IndexLock &l, DataChunk &chunk, Vector &row_ids) override; @@ -124,11 +124,11 @@ class ART : public BoundIndex { void VerifyBuffers(IndexLock &l) override; private: - bool SearchEqual(ARTKey &key, idx_t max_count, unsafe_vector &row_ids); - bool SearchGreater(ARTKey &key, bool equal, idx_t max_count, unsafe_vector &row_ids); - bool SearchLess(ARTKey &upper_bound, bool equal, idx_t max_count, unsafe_vector &row_ids); + bool SearchEqual(ARTKey &key, idx_t max_count, set &row_ids); + bool SearchGreater(ARTKey &key, bool equal, idx_t max_count, set &row_ids); + bool SearchLess(ARTKey &upper_bound, bool equal, idx_t max_count, set &row_ids); bool SearchCloseRange(ARTKey &lower_bound, ARTKey &upper_bound, bool left_equal, bool right_equal, idx_t max_count, - unsafe_vector &row_ids); + set &row_ids); string GenerateErrorKeyName(DataChunk &input, idx_t row); string GenerateConstraintErrorMessage(VerifyExistenceType verify_type, const string &key_name); 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 977cc7791..c5907f820 100644 --- a/src/duckdb/src/include/duckdb/execution/index/art/iterator.hpp +++ b/src/duckdb/src/include/duckdb/execution/index/art/iterator.hpp @@ -65,7 +65,7 @@ class Iterator { public: //! Scans the tree, starting at the current top node on the stack, and ending at upper_bound. //! If upper_bound is the empty ARTKey, than there is no upper bound. - bool Scan(const ARTKey &upper_bound, const idx_t max_count, unsafe_vector &row_ids, const bool equal); + bool Scan(const ARTKey &upper_bound, const idx_t max_count, set &row_ids, const bool equal); //! Finds the minimum (leaf) of the current subtree. void FindMinimum(const Node &node); //! Finds the lower bound of the ART and adds the nodes to the stack. Returns false, if the lower 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 01196c68e..30efdba0a 100644 --- a/src/duckdb/src/include/duckdb/execution/index/art/leaf.hpp +++ b/src/duckdb/src/include/duckdb/execution/index/art/leaf.hpp @@ -54,7 +54,7 @@ class Leaf { static void DeprecatedFree(ART &art, Node &node); //! Fills the row_ids vector with the row IDs of this linked list of leaves. //! Never pushes more than max_count row IDs. - static bool DeprecatedGetRowIds(ART &art, const Node &node, unsafe_vector &row_ids, const idx_t max_count); + static bool DeprecatedGetRowIds(ART &art, const Node &node, set &row_ids, const idx_t max_count); //! Vacuums the linked list of leaves. static void DeprecatedVacuum(ART &art, Node &node); //! Returns the string representation of the linked list of leaves, if only_verify is true. diff --git a/src/duckdb/src/include/duckdb/execution/operator/join/physical_range_join.hpp b/src/duckdb/src/include/duckdb/execution/operator/join/physical_range_join.hpp index 0634641f1..4ee6ef557 100644 --- a/src/duckdb/src/include/duckdb/execution/operator/join/physical_range_join.hpp +++ b/src/duckdb/src/include/duckdb/execution/operator/join/physical_range_join.hpp @@ -94,7 +94,7 @@ class PhysicalRangeJoin : public PhysicalComparisonJoin { public: PhysicalRangeJoin(PhysicalPlan &physical_plan, LogicalComparisonJoin &op, PhysicalOperatorType type, PhysicalOperator &left, PhysicalOperator &right, vector cond, JoinType join_type, - idx_t estimated_cardinality); + idx_t estimated_cardinality, unique_ptr pushdown_info); // Projection mappings using ProjectionMapping = vector; diff --git a/src/duckdb/src/include/duckdb/execution/operator/persistent/physical_merge_into.hpp b/src/duckdb/src/include/duckdb/execution/operator/persistent/physical_merge_into.hpp index f3798d877..9cbba99c1 100644 --- a/src/duckdb/src/include/duckdb/execution/operator/persistent/physical_merge_into.hpp +++ b/src/duckdb/src/include/duckdb/execution/operator/persistent/physical_merge_into.hpp @@ -39,7 +39,7 @@ class PhysicalMergeInto : public PhysicalOperator { public: PhysicalMergeInto(PhysicalPlan &physical_plan, vector types, map>> actions, idx_t row_id_index, - optional_idx source_marker, bool parallel); + optional_idx source_marker, bool parallel, bool return_chunk); //! List of all actions vector> actions; @@ -50,10 +50,13 @@ class PhysicalMergeInto : public PhysicalOperator { idx_t row_id_index; optional_idx source_marker; bool parallel; + bool return_chunk; public: // Source interface unique_ptr GetGlobalSourceState(ClientContext &context) const override; + unique_ptr GetLocalSourceState(ExecutionContext &context, + GlobalSourceState &gstate) const override; SourceResultType GetData(ExecutionContext &context, DataChunk &chunk, OperatorSourceInput &input) const override; bool IsSource() const override { diff --git a/src/duckdb/src/include/duckdb/function/copy_function.hpp b/src/duckdb/src/include/duckdb/function/copy_function.hpp index b3244cce0..a86625050 100644 --- a/src/duckdb/src/include/duckdb/function/copy_function.hpp +++ b/src/duckdb/src/include/duckdb/function/copy_function.hpp @@ -76,6 +76,14 @@ struct CopyFunctionBindInput { string file_extension; }; +struct CopyFromFunctionBindInput { + explicit CopyFromFunctionBindInput(const CopyInfo &info_p, TableFunction &tf_p) : info(info_p), tf(tf_p) { + } + + const CopyInfo &info; + TableFunction &tf; +}; + struct CopyToSelectInput { ClientContext &context; case_insensitive_map_t> &options; @@ -102,7 +110,7 @@ typedef void (*copy_to_serialize_t)(Serializer &serializer, const FunctionData & typedef unique_ptr (*copy_to_deserialize_t)(Deserializer &deserializer, CopyFunction &function); -typedef unique_ptr (*copy_from_bind_t)(ClientContext &context, CopyInfo &info, +typedef unique_ptr (*copy_from_bind_t)(ClientContext &context, CopyFromFunctionBindInput &info, vector &expected_names, vector &expected_types); typedef CopyFunctionExecutionMode (*copy_to_execution_mode_t)(bool preserve_insertion_order, bool supports_batch_index); diff --git a/src/duckdb/src/include/duckdb/function/pragma/pragma_functions.hpp b/src/duckdb/src/include/duckdb/function/pragma/pragma_functions.hpp index 9cef9078a..a1dd91f65 100644 --- a/src/duckdb/src/include/duckdb/function/pragma/pragma_functions.hpp +++ b/src/duckdb/src/include/duckdb/function/pragma/pragma_functions.hpp @@ -21,7 +21,7 @@ struct PragmaFunctions { static void RegisterFunction(BuiltinFunctions &set); }; -string PragmaShowTables(); +string PragmaShowTables(const string &catalog = "", const string &schema = ""); string PragmaShowTablesExpanded(); string PragmaShowDatabases(); string PragmaShowVariables(); diff --git a/src/duckdb/src/include/duckdb/function/scalar/compressed_materialization_functions.hpp b/src/duckdb/src/include/duckdb/function/scalar/compressed_materialization_functions.hpp index ac042a906..f1289cc60 100644 --- a/src/duckdb/src/include/duckdb/function/scalar/compressed_materialization_functions.hpp +++ b/src/duckdb/src/include/duckdb/function/scalar/compressed_materialization_functions.hpp @@ -105,6 +105,16 @@ struct InternalCompressStringUhugeintFun { static ScalarFunction GetFunction(); }; +struct InternalCompressStringHugeintFun { + static constexpr const char *Name = "__internal_compress_string_hugeint"; + static constexpr const char *Parameters = ""; + static constexpr const char *Description = ""; + static constexpr const char *Example = ""; + static constexpr const char *Categories = ""; + + static ScalarFunction GetFunction(); +}; + struct InternalDecompressIntegralSmallintFun { static constexpr const char *Name = "__internal_decompress_integral_smallint"; static constexpr const char *Parameters = ""; diff --git a/src/duckdb/src/include/duckdb/function/table/arrow.hpp b/src/duckdb/src/include/duckdb/function/table/arrow.hpp index f286732f2..a596c86e9 100644 --- a/src/duckdb/src/include/duckdb/function/table/arrow.hpp +++ b/src/duckdb/src/include/duckdb/function/table/arrow.hpp @@ -65,7 +65,7 @@ struct ArrowScanFunctionData : public TableFunctionData { //! The (optional) dependency of this function (used in Python for example) shared_ptr dependency; //! Arrow table data - ArrowTableType arrow_table; + ArrowTableSchema arrow_table; //! Whether projection pushdown is enabled on the scan bool projection_pushdown_enabled = true; }; @@ -89,10 +89,9 @@ struct ArrowRunEndEncodingState { struct ArrowScanLocalState; struct ArrowArrayScanState { public: - explicit ArrowArrayScanState(ArrowScanLocalState &state, ClientContext &context); + explicit ArrowArrayScanState(ClientContext &context); public: - ArrowScanLocalState &state; // Hold ownership over the Arrow Arrays owned by DuckDB to allow for zero-copy shared_ptr owned_data; unordered_map> children; @@ -153,7 +152,7 @@ struct ArrowScanLocalState : public LocalTableFunctionState { ArrowArrayScanState &GetState(idx_t child_idx) { auto it = array_states.find(child_idx); if (it == array_states.end()) { - auto child_p = make_uniq(*this, context); + auto child_p = make_uniq(context); auto &child = *child_p; array_states.emplace(child_idx, std::move(child_p)); return child; @@ -181,6 +180,26 @@ struct ArrowScanGlobalState : public GlobalTableFunctionState { } }; +struct ArrowToDuckDBConversion { + static void SetValidityMask(Vector &vector, ArrowArray &array, idx_t chunk_offset, idx_t size, + int64_t parent_offset, int64_t nested_offset, bool add_null = false); + + static void ColumnArrowToDuckDBRunEndEncoded(Vector &vector, const ArrowArray &array, idx_t chunk_offset, + ArrowArrayScanState &array_state, idx_t size, + const ArrowType &arrow_type, int64_t nested_offset = -1, + ValidityMask *parent_mask = nullptr, uint64_t parent_offset = 0); + + static void ColumnArrowToDuckDB(Vector &vector, ArrowArray &array, idx_t chunk_offset, + ArrowArrayScanState &array_state, idx_t size, const ArrowType &arrow_type, + int64_t nested_offset = -1, ValidityMask *parent_mask = nullptr, + uint64_t parent_offset = 0, bool ignore_extensions = false); + + static void ColumnArrowToDuckDBDictionary(Vector &vector, ArrowArray &array, idx_t chunk_offset, + ArrowArrayScanState &array_state, idx_t size, const ArrowType &arrow_type, + int64_t nested_offset = -1, const ValidityMask *parent_mask = nullptr, + uint64_t parent_offset = 0); +}; + struct ArrowTableFunction { public: static void RegisterFunction(BuiltinFunctions &set); @@ -214,9 +233,8 @@ struct ArrowTableFunction { //! Scan Function static void ArrowScanFunction(ClientContext &context, TableFunctionInput &data, DataChunk &output); - static void PopulateArrowTableType(DBConfig &config, ArrowTableType &arrow_table, - const ArrowSchemaWrapper &schema_p, vector &names, - vector &return_types); + static void PopulateArrowTableSchema(DBConfig &config, ArrowTableSchema &arrow_table, + const ArrowSchema &arrow_schema); protected: //! Defines Maximum Number of Threads @@ -230,9 +248,6 @@ struct ArrowTableFunction { //! -----Utility Functions:----- //! Gets Arrow Table's Cardinality static unique_ptr ArrowScanCardinality(ClientContext &context, const FunctionData *bind_data); - //! Gets the progress on the table scan, used for Progress Bars - static double ArrowProgress(ClientContext &context, const FunctionData *bind_data, - const GlobalTableFunctionState *global_state); }; } // namespace duckdb diff --git a/src/duckdb/src/include/duckdb/function/table/arrow/arrow_duck_schema.hpp b/src/duckdb/src/include/duckdb/function/table/arrow/arrow_duck_schema.hpp index 7d92a3437..44472ea63 100644 --- a/src/duckdb/src/include/duckdb/function/table/arrow/arrow_duck_schema.hpp +++ b/src/duckdb/src/include/duckdb/function/table/arrow/arrow_duck_schema.hpp @@ -96,6 +96,8 @@ class ArrowType { bool HasExtension() const; + ArrowArrayPhysicalType GetPhysicalType() const; + //! The Arrow Type Extension data, if any shared_ptr extension_data; @@ -114,13 +116,17 @@ class ArrowType { using arrow_column_map_t = unordered_map>; -struct ArrowTableType { +struct ArrowTableSchema { public: - void AddColumn(idx_t index, shared_ptr type); + void AddColumn(idx_t index, shared_ptr type, const string &name); const arrow_column_map_t &GetColumns() const; + vector &GetTypes(); + vector &GetNames(); private: arrow_column_map_t arrow_convert_data; + vector types; + vector column_names; }; } // namespace duckdb diff --git a/src/duckdb/src/include/duckdb/function/table/arrow/arrow_type_info.hpp b/src/duckdb/src/include/duckdb/function/table/arrow/arrow_type_info.hpp index d2e419646..822f60343 100644 --- a/src/duckdb/src/include/duckdb/function/table/arrow/arrow_type_info.hpp +++ b/src/duckdb/src/include/duckdb/function/table/arrow/arrow_type_info.hpp @@ -21,6 +21,8 @@ namespace duckdb { class ArrowType; +enum class ArrowArrayPhysicalType : uint8_t { DICTIONARY_ENCODED, RUN_END_ENCODED, DEFAULT }; + struct ArrowTypeInfo { public: explicit ArrowTypeInfo() : type() { diff --git a/src/duckdb/src/include/duckdb/main/attached_database.hpp b/src/duckdb/src/include/duckdb/main/attached_database.hpp index b92e0dbfd..d20726365 100644 --- a/src/duckdb/src/include/duckdb/main/attached_database.hpp +++ b/src/duckdb/src/include/duckdb/main/attached_database.hpp @@ -13,7 +13,6 @@ #include "duckdb/common/mutex.hpp" #include "duckdb/main/config.hpp" #include "duckdb/catalog/catalog_entry.hpp" -#include "duckdb/storage/storage_options.hpp" namespace duckdb { class Catalog; @@ -38,7 +37,7 @@ struct AttachOptions { //! Constructor for databases we attach outside of the ATTACH DATABASE statement. explicit AttachOptions(const DBConfigOptions &options); //! Constructor for databases we attach when using ATTACH DATABASE. - AttachOptions(const unique_ptr &info, const AccessMode default_access_mode); + AttachOptions(const unordered_map &options, const AccessMode default_access_mode); //! Defaults to the access mode configured in the DBConfig, unless specified otherwise. AccessMode access_mode; @@ -63,7 +62,7 @@ class AttachedDatabase : public CatalogEntry { ~AttachedDatabase() override; //! Initializes the catalog and storage of the attached database. - void Initialize(optional_ptr context = nullptr, StorageOptions options = StorageOptions()); + void Initialize(optional_ptr context = nullptr); void FinalizeLoad(optional_ptr context); void Close(); diff --git a/src/duckdb/src/include/duckdb/main/capi/capi_internal.hpp b/src/duckdb/src/include/duckdb/main/capi/capi_internal.hpp index 941a63561..1784ce990 100644 --- a/src/duckdb/src/include/duckdb/main/capi/capi_internal.hpp +++ b/src/duckdb/src/include/duckdb/main/capi/capi_internal.hpp @@ -42,6 +42,11 @@ struct CClientContextWrapper { ClientContext &context; }; +struct CClientArrowOptionsWrapper { + explicit CClientArrowOptionsWrapper(ClientProperties &properties) : properties(properties) {}; + ClientProperties properties; +}; + struct PreparedStatementWrapper { //! Map of name -> values case_insensitive_map_t values; @@ -77,6 +82,10 @@ struct ErrorDataWrapper { ErrorData error_data; }; +struct ExpressionWrapper { + unique_ptr expr; +}; + enum class CAPIResultSetType : uint8_t { CAPI_RESULT_TYPE_NONE = 0, CAPI_RESULT_TYPE_MATERIALIZED, diff --git a/src/duckdb/src/include/duckdb/main/capi/extension_api.hpp b/src/duckdb/src/include/duckdb/main/capi/extension_api.hpp index 4ba922f42..f7eea40e1 100644 --- a/src/duckdb/src/include/duckdb/main/capi/extension_api.hpp +++ b/src/duckdb/src/include/duckdb/main/capi/extension_api.hpp @@ -472,6 +472,18 @@ typedef struct { duckdb_state (*duckdb_append_default_to_chunk)(duckdb_appender appender, duckdb_data_chunk chunk, idx_t col, idx_t row); duckdb_error_data (*duckdb_appender_error_data)(duckdb_appender appender); + // New arrow interface functions + + duckdb_error_data (*duckdb_to_arrow_schema)(duckdb_arrow_options arrow_options, duckdb_logical_type *types, + const char **names, idx_t column_count, struct ArrowSchema *out_schema); + duckdb_error_data (*duckdb_data_chunk_to_arrow)(duckdb_arrow_options arrow_options, duckdb_data_chunk chunk, + struct ArrowArray *out_arrow_array); + duckdb_error_data (*duckdb_schema_from_arrow)(duckdb_connection connection, struct ArrowSchema *schema, + duckdb_arrow_converted_schema *out_types); + duckdb_error_data (*duckdb_data_chunk_from_arrow)(duckdb_connection connection, struct ArrowArray *arrow_array, + duckdb_arrow_converted_schema converted_schema, + duckdb_data_chunk *out_chunk); + void (*duckdb_destroy_arrow_converted_schema)(duckdb_arrow_converted_schema *arrow_converted_schema); // New functions for duckdb error data duckdb_error_data (*duckdb_create_error_data)(duckdb_error_type type, const char *message); @@ -479,12 +491,24 @@ typedef struct { duckdb_error_type (*duckdb_error_data_error_type)(duckdb_error_data error_data); const char *(*duckdb_error_data_message)(duckdb_error_data error_data); bool (*duckdb_error_data_has_error)(duckdb_error_data error_data); + // API to create and manipulate expressions + + void (*duckdb_destroy_expression)(duckdb_expression *expr); + duckdb_logical_type (*duckdb_expression_return_type)(duckdb_expression expr); + bool (*duckdb_expression_is_foldable)(duckdb_expression expr); + duckdb_error_data (*duckdb_expression_fold)(duckdb_client_context context, duckdb_expression expr, + duckdb_value *out_value); // New functions around the client context idx_t (*duckdb_client_context_get_connection_id)(duckdb_client_context context); void (*duckdb_destroy_client_context)(duckdb_client_context *context); void (*duckdb_connection_get_client_context)(duckdb_connection connection, duckdb_client_context *out_context); duckdb_value (*duckdb_get_table_names)(duckdb_connection connection, const char *query, bool qualified); + void (*duckdb_connection_get_arrow_options)(duckdb_connection connection, duckdb_arrow_options *out_arrow_options); + void (*duckdb_destroy_arrow_options)(duckdb_arrow_options *arrow_options); + // New query execution functions + + duckdb_arrow_options (*duckdb_result_get_arrow_options)(duckdb_result *result); // New functions around scalar function binding void (*duckdb_scalar_function_set_bind)(duckdb_scalar_function scalar_function, duckdb_scalar_function_bind_t bind); @@ -494,6 +518,8 @@ typedef struct { duckdb_delete_callback_t destroy); void *(*duckdb_scalar_function_get_bind_data)(duckdb_function_info info); void *(*duckdb_scalar_function_bind_get_extra_info)(duckdb_bind_info info); + idx_t (*duckdb_scalar_function_bind_get_argument_count)(duckdb_bind_info info); + duckdb_expression (*duckdb_scalar_function_bind_get_argument)(duckdb_bind_info info, idx_t index); // New string functions that are added char *(*duckdb_value_to_string)(duckdb_value value); @@ -930,21 +956,35 @@ inline duckdb_ext_api_v1 CreateAPIv1() { result.duckdb_destroy_instance_cache = duckdb_destroy_instance_cache; result.duckdb_append_default_to_chunk = duckdb_append_default_to_chunk; result.duckdb_appender_error_data = duckdb_appender_error_data; + result.duckdb_to_arrow_schema = duckdb_to_arrow_schema; + result.duckdb_data_chunk_to_arrow = duckdb_data_chunk_to_arrow; + result.duckdb_schema_from_arrow = duckdb_schema_from_arrow; + result.duckdb_data_chunk_from_arrow = duckdb_data_chunk_from_arrow; + result.duckdb_destroy_arrow_converted_schema = duckdb_destroy_arrow_converted_schema; result.duckdb_create_error_data = duckdb_create_error_data; result.duckdb_destroy_error_data = duckdb_destroy_error_data; result.duckdb_error_data_error_type = duckdb_error_data_error_type; result.duckdb_error_data_message = duckdb_error_data_message; result.duckdb_error_data_has_error = duckdb_error_data_has_error; + result.duckdb_destroy_expression = duckdb_destroy_expression; + result.duckdb_expression_return_type = duckdb_expression_return_type; + result.duckdb_expression_is_foldable = duckdb_expression_is_foldable; + result.duckdb_expression_fold = duckdb_expression_fold; result.duckdb_client_context_get_connection_id = duckdb_client_context_get_connection_id; result.duckdb_destroy_client_context = duckdb_destroy_client_context; result.duckdb_connection_get_client_context = duckdb_connection_get_client_context; result.duckdb_get_table_names = duckdb_get_table_names; + result.duckdb_connection_get_arrow_options = duckdb_connection_get_arrow_options; + result.duckdb_destroy_arrow_options = duckdb_destroy_arrow_options; + result.duckdb_result_get_arrow_options = duckdb_result_get_arrow_options; result.duckdb_scalar_function_set_bind = duckdb_scalar_function_set_bind; result.duckdb_scalar_function_bind_set_error = duckdb_scalar_function_bind_set_error; result.duckdb_scalar_function_get_client_context = duckdb_scalar_function_get_client_context; result.duckdb_scalar_function_set_bind_data = duckdb_scalar_function_set_bind_data; result.duckdb_scalar_function_get_bind_data = duckdb_scalar_function_get_bind_data; result.duckdb_scalar_function_bind_get_extra_info = duckdb_scalar_function_bind_get_extra_info; + result.duckdb_scalar_function_bind_get_argument_count = duckdb_scalar_function_bind_get_argument_count; + result.duckdb_scalar_function_bind_get_argument = duckdb_scalar_function_bind_get_argument; result.duckdb_value_to_string = duckdb_value_to_string; result.duckdb_create_map_value = duckdb_create_map_value; result.duckdb_create_union_value = duckdb_create_union_value; diff --git a/src/duckdb/src/include/duckdb/main/database_manager.hpp b/src/duckdb/src/include/duckdb/main/database_manager.hpp index fe221a344..4fb8690d9 100644 --- a/src/duckdb/src/include/duckdb/main/database_manager.hpp +++ b/src/duckdb/src/include/duckdb/main/database_manager.hpp @@ -98,8 +98,6 @@ class DatabaseManager { vector GetAttachedDatabasePaths(); private: - //! Returns a database with a specified path - optional_ptr GetDatabaseFromPath(ClientContext &context, const string &path); void CheckPathConflict(ClientContext &context, const string &path); private: @@ -121,7 +119,7 @@ class DatabaseManager { //! A set containing all attached database path //! This allows to attach many databases efficiently, and to avoid attaching the //! same file path twice - case_insensitive_set_t db_paths; + case_insensitive_map_t db_paths_to_name; }; } // namespace duckdb diff --git a/src/duckdb/src/include/duckdb/parser/parsed_data/alter_table_info.hpp b/src/duckdb/src/include/duckdb/parser/parsed_data/alter_table_info.hpp index 25864c3e8..b206e9143 100644 --- a/src/duckdb/src/include/duckdb/parser/parsed_data/alter_table_info.hpp +++ b/src/duckdb/src/include/duckdb/parser/parsed_data/alter_table_info.hpp @@ -134,14 +134,17 @@ struct RenameFieldInfo : public AlterTableInfo { RenameFieldInfo(AlterEntryData data, vector column_path, string new_name_p); ~RenameFieldInfo() override; - //! Path to source field + //! Path to source field. vector column_path; - //! Column new name + //! New name of the column (field). string new_name; public: unique_ptr Copy() const override; string ToString() const override; + string GetColumnName() const override { + return column_path[0]; + } void Serialize(Serializer &serializer) const override; static unique_ptr Deserialize(Deserializer &deserializer); @@ -201,16 +204,19 @@ struct AddFieldInfo : public AlterTableInfo { AddFieldInfo(AlterEntryData data, vector column_path, ColumnDefinition new_field, bool if_field_not_exists); ~AddFieldInfo() override; - //! The path to the struct + //! Path to source field. vector column_path; - //! New field to add to the struct + //! New field to add. ColumnDefinition new_field; - //! Whether or not an error should be thrown if the field exist + //! Whether or not an error should be thrown if the field does not exist. bool if_field_not_exists; public: unique_ptr Copy() const override; string ToString() const override; + string GetColumnName() const override { + return column_path[0]; + } void Serialize(Serializer &serializer) const override; static unique_ptr Deserialize(Deserializer &deserializer); @@ -253,16 +259,20 @@ struct RemoveFieldInfo : public AlterTableInfo { RemoveFieldInfo(AlterEntryData data, vector column_path, bool if_column_exists, bool cascade); ~RemoveFieldInfo() override; - //! The path to the field to remove + //! Path to source field. vector column_path; - //! Whether or not an error should be thrown if the column does not exist + //! Whether or not an error should be thrown if the column does not exist. bool if_column_exists; - //! Whether or not the column should be removed if a dependency conflict arises (used by GENERATED columns) + //! Whether or not the column should be removed if a dependency conflict arises (used by GENERATED columns). bool cascade; public: unique_ptr Copy() const override; string ToString() const override; + string GetColumnName() const override { + return column_path[0]; + } + void Serialize(Serializer &serializer) const override; static unique_ptr Deserialize(Deserializer &deserializer); diff --git a/src/duckdb/src/include/duckdb/parser/parsed_data/attach_info.hpp b/src/duckdb/src/include/duckdb/parser/parsed_data/attach_info.hpp index 835dd9a4c..6fffbe385 100644 --- a/src/duckdb/src/include/duckdb/parser/parsed_data/attach_info.hpp +++ b/src/duckdb/src/include/duckdb/parser/parsed_data/attach_info.hpp @@ -9,11 +9,9 @@ #pragma once #include "duckdb/parser/parsed_data/parse_info.hpp" -#include "duckdb/common/vector.hpp" #include "duckdb/common/unordered_map.hpp" #include "duckdb/common/types/value.hpp" #include "duckdb/common/enums/on_create_conflict.hpp" -#include "duckdb/storage/storage_options.hpp" namespace duckdb { @@ -35,8 +33,6 @@ struct AttachInfo : public ParseInfo { OnCreateConflict on_conflict = OnCreateConflict::ERROR_ON_CONFLICT; public: - //! Returns the storage options - StorageOptions GetStorageOptions() const; //! Copies this AttachInfo and returns an unique pointer to the new AttachInfo. unique_ptr Copy() const; string ToString() const; diff --git a/src/duckdb/src/include/duckdb/parser/statement/merge_into_statement.hpp b/src/duckdb/src/include/duckdb/parser/statement/merge_into_statement.hpp index 8f69686c1..554cdb126 100644 --- a/src/duckdb/src/include/duckdb/parser/statement/merge_into_statement.hpp +++ b/src/duckdb/src/include/duckdb/parser/statement/merge_into_statement.hpp @@ -53,6 +53,9 @@ class MergeIntoStatement : public SQLStatement { map>> actions; + //! keep track of optional returningList if statement contains a RETURNING keyword + vector> returning_list; + //! CTEs CommonTableExpressionMap cte_map; diff --git a/src/duckdb/src/include/duckdb/parser/tableref/showref.hpp b/src/duckdb/src/include/duckdb/parser/tableref/showref.hpp index fdd98e21b..77a37d444 100644 --- a/src/duckdb/src/include/duckdb/parser/tableref/showref.hpp +++ b/src/duckdb/src/include/duckdb/parser/tableref/showref.hpp @@ -16,7 +16,7 @@ namespace duckdb { -enum class ShowType : uint8_t { SUMMARY, DESCRIBE }; +enum class ShowType : uint8_t { SUMMARY, DESCRIBE, SHOW_FROM }; //! Represents a SHOW/DESCRIBE/SUMMARIZE statement class ShowRef : public TableRef { @@ -28,6 +28,10 @@ class ShowRef : public TableRef { //! The table name (if any) string table_name; + //! The catalog name (if any) + string catalog_name; + //! The schema name (if any) + string schema_name; //! The QueryNode of select query (if any) unique_ptr query; //! Whether or not we are requesting a summary or a describe diff --git a/src/duckdb/src/include/duckdb/planner/bind_context.hpp b/src/duckdb/src/include/duckdb/planner/bind_context.hpp index a9a905b3e..d9c20dd1d 100644 --- a/src/duckdb/src/include/duckdb/planner/bind_context.hpp +++ b/src/duckdb/src/include/duckdb/planner/bind_context.hpp @@ -97,6 +97,9 @@ class BindContext { vector &bound_column_ids, TableCatalogEntry &entry, bool add_row_id = true); void AddBaseTable(idx_t index, const string &alias, const vector &names, const vector &types, vector &bound_column_ids, const string &table_name); + void AddBaseTable(idx_t index, const string &alias, const vector &names, const vector &types, + vector &bound_column_ids, TableCatalogEntry &entry, + virtual_column_map_t virtual_columns); //! Adds a call to a table function with the given alias to the BindContext. void AddTableFunction(idx_t index, const string &alias, const vector &names, const vector &types, vector &bound_column_ids, diff --git a/src/duckdb/src/include/duckdb/planner/binder.hpp b/src/duckdb/src/include/duckdb/planner/binder.hpp index 19bd0a290..41f3596bd 100644 --- a/src/duckdb/src/include/duckdb/planner/binder.hpp +++ b/src/duckdb/src/include/duckdb/planner/binder.hpp @@ -333,7 +333,8 @@ class Binder : public enable_shared_from_this { void BindRowIdColumns(TableCatalogEntry &table, LogicalGet &get, vector> &expressions); BoundStatement BindReturning(vector> returning_list, TableCatalogEntry &table, const string &alias, idx_t update_table_index, - unique_ptr child_operator, BoundStatement result); + unique_ptr child_operator, + virtual_column_map_t virtual_columns = virtual_column_map_t()); unique_ptr BindTableMacro(FunctionExpression &function, TableMacroCatalogEntry ¯o_func, idx_t depth); diff --git a/src/duckdb/src/include/duckdb/planner/operator/logical_merge_into.hpp b/src/duckdb/src/include/duckdb/planner/operator/logical_merge_into.hpp index 959b66ffe..657fec212 100644 --- a/src/duckdb/src/include/duckdb/planner/operator/logical_merge_into.hpp +++ b/src/duckdb/src/include/duckdb/planner/operator/logical_merge_into.hpp @@ -52,6 +52,8 @@ class LogicalMergeInto : public LogicalOperator { optional_idx source_marker; //! Bound constraints vector> bound_constraints; + //! Whether or not to return the input data + bool return_chunk = false; map>> actions; @@ -60,6 +62,7 @@ class LogicalMergeInto : public LogicalOperator { static unique_ptr Deserialize(Deserializer &deserializer); idx_t EstimateCardinality(ClientContext &context) override; + vector GetTableIndex() const override; protected: vector GetColumnBindings() override; diff --git a/src/duckdb/src/include/duckdb/storage/checkpoint/row_group_writer.hpp b/src/duckdb/src/include/duckdb/storage/checkpoint/row_group_writer.hpp index d52313902..99b8f6570 100644 --- a/src/duckdb/src/include/duckdb/storage/checkpoint/row_group_writer.hpp +++ b/src/duckdb/src/include/duckdb/storage/checkpoint/row_group_writer.hpp @@ -9,6 +9,7 @@ #pragma once #include "duckdb/storage/checkpoint_manager.hpp" +#include "duckdb/common/serializer/memory_stream.hpp" namespace duckdb { struct ColumnCheckpointState; @@ -31,7 +32,9 @@ class RowGroupWriter { CompressionType GetColumnCompressionType(idx_t i); virtual CheckpointType GetCheckpointType() const = 0; - virtual MetadataWriter &GetPayloadWriter() = 0; + virtual WriteStream &GetPayloadWriter() = 0; + virtual MetaBlockPointer GetMetaBlockPointer() = 0; + virtual optional_ptr GetMetadataManager() = 0; PartialBlockManager &GetPartialBlockManager() { return partial_block_manager; @@ -50,7 +53,9 @@ class SingleFileRowGroupWriter : public RowGroupWriter { public: CheckpointType GetCheckpointType() const override; - MetadataWriter &GetPayloadWriter() override; + WriteStream &GetPayloadWriter() override; + MetaBlockPointer GetMetaBlockPointer() override; + optional_ptr GetMetadataManager() override; private: //! Underlying writer object diff --git a/src/duckdb/src/include/duckdb/storage/checkpoint_manager.hpp b/src/duckdb/src/include/duckdb/storage/checkpoint_manager.hpp index 374547e4b..318d93abf 100644 --- a/src/duckdb/src/include/duckdb/storage/checkpoint_manager.hpp +++ b/src/duckdb/src/include/duckdb/storage/checkpoint_manager.hpp @@ -33,6 +33,7 @@ class CheckpointWriter { //! The database AttachedDatabase &db; + virtual void CreateCheckpoint() = 0; virtual MetadataManager &GetMetadataManager() = 0; virtual MetadataWriter &GetMetadataWriter() = 0; virtual unique_ptr GetTableDataWriter(TableCatalogEntry &table) = 0; @@ -100,9 +101,7 @@ class SingleFileCheckpointWriter final : public CheckpointWriter { SingleFileCheckpointWriter(QueryContext context, AttachedDatabase &db, BlockManager &block_manager, CheckpointType checkpoint_type); - //! Checkpoint the current state of the WAL and flush it to the main storage. This should be called BEFORE any - //! connection is available because right now the checkpointing cannot be done online. (TODO) - void CreateCheckpoint(); + void CreateCheckpoint() override; MetadataWriter &GetMetadataWriter() override; MetadataManager &GetMetadataManager() override; diff --git a/src/duckdb/src/include/duckdb/storage/compression/chimp/algorithm/packed_data.hpp b/src/duckdb/src/include/duckdb/storage/compression/chimp/algorithm/packed_data.hpp index 932719ded..7dea7af52 100644 --- a/src/duckdb/src/include/duckdb/storage/compression/chimp/algorithm/packed_data.hpp +++ b/src/duckdb/src/include/duckdb/storage/compression/chimp/algorithm/packed_data.hpp @@ -1,7 +1,7 @@ //===----------------------------------------------------------------------===// // DuckDB // -// duckdb/storage/compression/chimp/packed_data.hpp +// duckdb/storage/compression/chimp/algorithm/packed_data.hpp // // //===----------------------------------------------------------------------===// @@ -9,6 +9,7 @@ #pragma once #include "duckdb/storage/compression/chimp/algorithm/chimp_utils.hpp" +#include "duckdb/common/assert.hpp" #include "duckdb.h" namespace duckdb { diff --git a/src/duckdb/src/include/duckdb/storage/compression/patas/patas_scan.hpp b/src/duckdb/src/include/duckdb/storage/compression/patas/patas_scan.hpp index 05089982e..b523600e3 100644 --- a/src/duckdb/src/include/duckdb/storage/compression/patas/patas_scan.hpp +++ b/src/duckdb/src/include/duckdb/storage/compression/patas/patas_scan.hpp @@ -1,7 +1,7 @@ //===----------------------------------------------------------------------===// // DuckDB // -// duckdb/common/storage/compression/chimp/chimp_scan.hpp +// duckdb/common/storage/compression/patas/patas_scan.hpp // // //===----------------------------------------------------------------------===// @@ -10,6 +10,7 @@ #include "duckdb/storage/compression/chimp/chimp.hpp" #include "duckdb/storage/compression/chimp/algorithm/chimp_utils.hpp" +#include "duckdb/storage/compression/chimp/algorithm/packed_data.hpp" #include "duckdb/storage/compression/chimp/algorithm/byte_reader.hpp" #include "duckdb/storage/compression/patas/shared.hpp" #include "duckdb/storage/compression/patas/algorithm/patas.hpp" diff --git a/src/duckdb/src/include/duckdb/storage/partial_block_manager.hpp b/src/duckdb/src/include/duckdb/storage/partial_block_manager.hpp index 8bd67e239..4e901fb0f 100644 --- a/src/duckdb/src/include/duckdb/storage/partial_block_manager.hpp +++ b/src/duckdb/src/include/duckdb/storage/partial_block_manager.hpp @@ -60,6 +60,7 @@ struct PartialBlock { public: //! Add regions that need zero-initialization to avoid leaking memory void AddUninitializedRegion(const idx_t start, const idx_t end); + virtual void AddSegmentToTail(ColumnData &data, ColumnSegment &segment, uint32_t offset_in_block); //! Flush the block to disk and zero-initialize any free space and uninitialized regions virtual void Flush(QueryContext context, const idx_t free_space_left) = 0; void FlushInternal(const idx_t free_space_left); @@ -85,7 +86,7 @@ struct PartialBlockAllocation { unique_ptr partial_block; }; -enum class PartialBlockType { FULL_CHECKPOINT, APPEND_TO_TABLE }; +enum class PartialBlockType { FULL_CHECKPOINT, APPEND_TO_TABLE, IN_MEMORY_CHECKPOINT }; //! Enables sharing blocks across some scope. Scope is whatever we want to share //! blocks across. It may be an entire checkpoint or just a single row group. @@ -122,6 +123,9 @@ class PartialBlockManager { //! Flush any remaining partial blocks to disk void FlushPartialBlocks(); + unique_ptr CreatePartialBlock(ColumnData &data, ColumnSegment &segment, PartialBlockState state, + BlockManager &block_manager); + unique_lock GetLock() { return unique_lock(partial_block_lock); } diff --git a/src/duckdb/src/include/duckdb/storage/storage_extension.hpp b/src/duckdb/src/include/duckdb/storage/storage_extension.hpp index 62d5a8028..7770df5fc 100644 --- a/src/duckdb/src/include/duckdb/storage/storage_extension.hpp +++ b/src/duckdb/src/include/duckdb/storage/storage_extension.hpp @@ -25,10 +25,10 @@ struct StorageExtensionInfo { } }; -typedef unique_ptr (*attach_function_t)(StorageExtensionInfo *storage_info, ClientContext &context, - AttachedDatabase &db, const string &name, AttachInfo &info, - AccessMode access_mode); -typedef unique_ptr (*create_transaction_manager_t)(StorageExtensionInfo *storage_info, +typedef unique_ptr (*attach_function_t)(optional_ptr storage_info, + ClientContext &context, AttachedDatabase &db, const string &name, + AttachInfo &info, AttachOptions &options); +typedef unique_ptr (*create_transaction_manager_t)(optional_ptr storage_info, AttachedDatabase &db, Catalog &catalog); class StorageExtension { diff --git a/src/duckdb/src/include/duckdb/storage/storage_manager.hpp b/src/duckdb/src/include/duckdb/storage/storage_manager.hpp index fa9b3b40e..ea60405b5 100644 --- a/src/duckdb/src/include/duckdb/storage/storage_manager.hpp +++ b/src/duckdb/src/include/duckdb/storage/storage_manager.hpp @@ -61,7 +61,7 @@ struct CheckpointOptions { //! StorageManager is responsible for managing the physical storage of a persistent database. class StorageManager { public: - StorageManager(AttachedDatabase &db, string path, bool read_only); + StorageManager(AttachedDatabase &db, string path, const AttachOptions &options); virtual ~StorageManager(); public: @@ -71,10 +71,10 @@ class StorageManager { //! Initialize a database or load an existing database from the database file path. The block_alloc_size is //! either set, or invalid. If invalid, then DuckDB defaults to the default_block_alloc_size (DBConfig), //! or the file's block allocation size, if it is an existing database. - void Initialize(QueryContext context, StorageOptions &options); + void Initialize(QueryContext context); DatabaseInstance &GetDatabase(); - AttachedDatabase &GetAttached() { + AttachedDatabase &GetAttached() const { return db; } @@ -93,8 +93,8 @@ class StorageManager { return load_complete; } //! The path to the WAL, derived from the database file path - string GetWALPath(); - bool InMemory(); + string GetWALPath() const; + bool InMemory() const; virtual bool AutomaticCheckpoint(idx_t estimated_wal_bytes) = 0; virtual unique_ptr GenStorageCommitState(WriteAheadLog &wal) = 0; @@ -115,9 +115,18 @@ class StorageManager { D_ASSERT(HasStorageVersion()); return storage_version.GetIndex(); } + void AddInMemoryChange(idx_t size) { + in_memory_change_size += size; + } + void ResetInMemoryChange() { + in_memory_change_size = 0; + } + bool CompressionIsEnabled() const { + return storage_options.compress_in_memory == CompressInMemory::COMPRESS; + } protected: - virtual void LoadDatabase(QueryContext context, StorageOptions &options) = 0; + virtual void LoadDatabase(QueryContext context) = 0; protected: //! The attached database managed by this storage manager. @@ -133,6 +142,10 @@ class StorageManager { bool load_complete = false; //! The serialization compatibility version when reading and writing from this database optional_idx storage_version; + //! Estimated size of changes for determining automatic checkpointing on in-memory databases + atomic in_memory_change_size; + //! Storage options passed in through configuration + StorageOptions storage_options; public: template @@ -151,7 +164,7 @@ class StorageManager { class SingleFileStorageManager : public StorageManager { public: SingleFileStorageManager() = delete; - SingleFileStorageManager(AttachedDatabase &db, string path, bool read_only); + SingleFileStorageManager(AttachedDatabase &db, string path, const AttachOptions &options); //! The BlockManager to read from and write to blocks (meta data and data). unique_ptr block_manager; @@ -169,6 +182,8 @@ class SingleFileStorageManager : public StorageManager { BlockManager &GetBlockManager() override; protected: - void LoadDatabase(QueryContext context, StorageOptions &options) override; + void LoadDatabase(QueryContext context) override; + + unique_ptr CreateCheckpointWriter(QueryContext context, CheckpointOptions options); }; } // namespace duckdb diff --git a/src/duckdb/src/include/duckdb/storage/storage_options.hpp b/src/duckdb/src/include/duckdb/storage/storage_options.hpp index 61c99151b..a3ff65acc 100644 --- a/src/duckdb/src/include/duckdb/storage/storage_options.hpp +++ b/src/duckdb/src/include/duckdb/storage/storage_options.hpp @@ -10,9 +10,12 @@ #include "duckdb/common/common.hpp" #include "duckdb/common/optional_idx.hpp" +#include "duckdb/common/types/value.hpp" namespace duckdb { +enum class CompressInMemory { AUTOMATIC, COMPRESS, DO_NOT_COMPRESS }; + struct StorageOptions { //! The allocation size of blocks for this attached database file (if any) optional_idx block_alloc_size; @@ -23,6 +26,8 @@ struct StorageOptions { //! Block header size (only used for encryption) optional_idx block_header_size; + CompressInMemory compress_in_memory = CompressInMemory::AUTOMATIC; + //! Whether the database is encrypted bool encryption = false; //! Encryption algorithm (default = GCM) @@ -30,6 +35,8 @@ struct StorageOptions { //! encryption key //! FIXME: change to a unique_ptr in the future shared_ptr user_key; + + void Initialize(const unordered_map &options); }; inline void ClearUserKey(shared_ptr const &encryption_key) { diff --git a/src/duckdb/src/include/duckdb/storage/table/column_checkpoint_state.hpp b/src/duckdb/src/include/duckdb/storage/table/column_checkpoint_state.hpp index 25504a3e8..9e5f7f98b 100644 --- a/src/duckdb/src/include/duckdb/storage/table/column_checkpoint_state.hpp +++ b/src/duckdb/src/include/duckdb/storage/table/column_checkpoint_state.hpp @@ -87,7 +87,7 @@ struct PartialBlockForCheckpoint : public PartialBlock { bool IsFlushed(); void Flush(QueryContext context, const idx_t free_space_left) override; void Merge(PartialBlock &other, idx_t offset, idx_t other_size) override; - void AddSegmentToTail(ColumnData &data, ColumnSegment &segment, uint32_t offset_in_block); + void AddSegmentToTail(ColumnData &data, ColumnSegment &segment, uint32_t offset_in_block) override; void Clear() override; }; diff --git a/src/duckdb/src/include/duckdb/storage/table/column_data.hpp b/src/duckdb/src/include/duckdb/storage/table/column_data.hpp index 49a95c7ed..042ea3793 100644 --- a/src/duckdb/src/include/duckdb/storage/table/column_data.hpp +++ b/src/duckdb/src/include/duckdb/storage/table/column_data.hpp @@ -214,6 +214,7 @@ class ColumnData { void FetchUpdateRow(TransactionData transaction, row_t row_id, Vector &result, idx_t result_idx); void UpdateInternal(TransactionData transaction, idx_t column_index, Vector &update_vector, row_t *row_ids, idx_t update_count, Vector &base_vector); + idx_t FetchUpdateData(row_t *row_ids, Vector &base_vector); idx_t GetVectorCount(idx_t vector_index) const; diff --git a/src/duckdb/src/include/duckdb/storage/table/column_data_checkpointer.hpp b/src/duckdb/src/include/duckdb/storage/table/column_data_checkpointer.hpp index aec2efdba..f31e77521 100644 --- a/src/duckdb/src/include/duckdb/storage/table/column_data_checkpointer.hpp +++ b/src/duckdb/src/include/duckdb/storage/table/column_data_checkpointer.hpp @@ -22,9 +22,10 @@ struct ColumnDataCheckpointData { ColumnDataCheckpointData() { } ColumnDataCheckpointData(ColumnCheckpointState &checkpoint_state, ColumnData &col_data, DatabaseInstance &db, - RowGroup &row_group, ColumnCheckpointInfo &checkpoint_info) + RowGroup &row_group, ColumnCheckpointInfo &checkpoint_info, + StorageManager &storage_manager) : checkpoint_state(checkpoint_state), col_data(col_data), db(db), row_group(row_group), - checkpoint_info(checkpoint_info) { + checkpoint_info(checkpoint_info), storage_manager(storage_manager) { } public: @@ -34,6 +35,7 @@ struct ColumnDataCheckpointData { RowGroup &GetRowGroup(); ColumnCheckpointState &GetCheckpointState(); DatabaseInstance &GetDatabase(); + StorageManager &GetStorageManager(); private: optional_ptr checkpoint_state; @@ -41,6 +43,7 @@ struct ColumnDataCheckpointData { optional_ptr db; optional_ptr row_group; optional_ptr checkpoint_info; + optional_ptr storage_manager; }; struct CheckpointAnalyzeResult { diff --git a/src/duckdb/src/include/duckdb/storage/table/column_segment.hpp b/src/duckdb/src/include/duckdb/storage/table/column_segment.hpp index 0af6d6fd0..61b2c0d4f 100644 --- a/src/duckdb/src/include/duckdb/storage/table/column_segment.hpp +++ b/src/duckdb/src/include/duckdb/storage/table/column_segment.hpp @@ -99,12 +99,13 @@ class ColumnSegment : public SegmentBase { //! Revert an append made to this segment void RevertAppend(idx_t start_row); - //! Convert a transient in-memory segment into a persistent segment blocked by an on-disk block. + //! Convert a transient in-memory segment to a persistent segment backed by an on-disk block. //! Only used during checkpointing. - void ConvertToPersistent(QueryContext context, optional_ptr block_manager, block_id_t block_id); + void ConvertToPersistent(QueryContext context, optional_ptr block_manager, const block_id_t block_id); //! Updates pointers to refer to the given block and offset. This is only used //! when sharing a block among segments. This is invoked only AFTER the block is written. void MarkAsPersistent(shared_ptr block, uint32_t offset_in_block); + void SetBlock(shared_ptr block, uint32_t offset); //! Gets a data pointer from a persistent column segment DataPointer GetDataPointer(); diff --git a/src/duckdb/src/include/duckdb/storage/table/in_memory_checkpoint.hpp b/src/duckdb/src/include/duckdb/storage/table/in_memory_checkpoint.hpp new file mode 100644 index 000000000..59e280da5 --- /dev/null +++ b/src/duckdb/src/include/duckdb/storage/table/in_memory_checkpoint.hpp @@ -0,0 +1,90 @@ +//===----------------------------------------------------------------------===// +// DuckDB +// +// duckdb/storage/table/in_memory_checkpoint.hpp +// +// +//===----------------------------------------------------------------------===// + +#pragma once + +#include "duckdb/storage/checkpoint/row_group_writer.hpp" +#include "duckdb/storage/checkpoint/table_data_writer.hpp" +#include "duckdb/storage/checkpoint_manager.hpp" + +namespace duckdb { + +class InMemoryCheckpointer final : public CheckpointWriter { +public: + InMemoryCheckpointer(QueryContext context, AttachedDatabase &db, BlockManager &block_manager, + StorageManager &storage_manager, CheckpointType checkpoint_type); + + void CreateCheckpoint() override; + + MetadataWriter &GetMetadataWriter() override; + MetadataManager &GetMetadataManager() override; + unique_ptr GetTableDataWriter(TableCatalogEntry &table) override; + optional_ptr GetClientContext() const { + return context; + } + CheckpointType GetCheckpointType() const { + return checkpoint_type; + } + PartialBlockManager &GetPartialBlockManager() { + return partial_block_manager; + } + +public: + void WriteTable(TableCatalogEntry &table, Serializer &serializer) override; + +private: + optional_ptr context; + PartialBlockManager partial_block_manager; + StorageManager &storage_manager; + CheckpointType checkpoint_type; +}; + +class InMemoryTableDataWriter : public TableDataWriter { +public: + InMemoryTableDataWriter(InMemoryCheckpointer &checkpoint_manager, TableCatalogEntry &table); + +public: + void FinalizeTable(const TableStatistics &global_stats, DataTableInfo *info, Serializer &serializer) override; + unique_ptr GetRowGroupWriter(RowGroup &row_group) override; + CheckpointType GetCheckpointType() const override; + +private: + InMemoryCheckpointer &checkpoint_manager; +}; + +class InMemoryRowGroupWriter : public RowGroupWriter { +public: + InMemoryRowGroupWriter(TableCatalogEntry &table, PartialBlockManager &partial_block_manager, + InMemoryCheckpointer &checkpoint_manager); + +public: + CheckpointType GetCheckpointType() const override; + WriteStream &GetPayloadWriter() override; + MetaBlockPointer GetMetaBlockPointer() override; + optional_ptr GetMetadataManager() override; + +private: + //! Underlying writer object + InMemoryCheckpointer &checkpoint_manager; + // Nop metadata writer + MemoryStream metadata_writer; +}; + +struct InMemoryPartialBlock : public PartialBlock { +public: + InMemoryPartialBlock(ColumnData &data, ColumnSegment &segment, PartialBlockState state, + BlockManager &block_manager); + ~InMemoryPartialBlock() override; + +public: + void Flush(QueryContext context, const idx_t free_space_left) override; + void Merge(PartialBlock &other, idx_t offset, idx_t other_size) override; + void AddSegmentToTail(ColumnData &data, ColumnSegment &segment, uint32_t offset_in_block) override; + void Clear() override; +}; +} // namespace duckdb diff --git a/src/duckdb/src/include/duckdb_extension.h b/src/duckdb/src/include/duckdb_extension.h index 9462e1247..9c99dbc9f 100644 --- a/src/duckdb/src/include/duckdb_extension.h +++ b/src/duckdb/src/include/duckdb_extension.h @@ -543,6 +543,20 @@ typedef struct { duckdb_error_data (*duckdb_appender_error_data)(duckdb_appender appender); #endif +// New arrow interface functions +#ifdef DUCKDB_EXTENSION_API_VERSION_UNSTABLE + duckdb_error_data (*duckdb_to_arrow_schema)(duckdb_arrow_options arrow_options, duckdb_logical_type *types, + const char **names, idx_t column_count, struct ArrowSchema *out_schema); + duckdb_error_data (*duckdb_data_chunk_to_arrow)(duckdb_arrow_options arrow_options, duckdb_data_chunk chunk, + struct ArrowArray *out_arrow_array); + duckdb_error_data (*duckdb_schema_from_arrow)(duckdb_connection connection, struct ArrowSchema *schema, + duckdb_arrow_converted_schema *out_types); + duckdb_error_data (*duckdb_data_chunk_from_arrow)(duckdb_connection connection, struct ArrowArray *arrow_array, + duckdb_arrow_converted_schema converted_schema, + duckdb_data_chunk *out_chunk); + void (*duckdb_destroy_arrow_converted_schema)(duckdb_arrow_converted_schema *arrow_converted_schema); +#endif + // New functions for duckdb error data #ifdef DUCKDB_EXTENSION_API_VERSION_UNSTABLE duckdb_error_data (*duckdb_create_error_data)(duckdb_error_type type, const char *message); @@ -552,12 +566,28 @@ typedef struct { bool (*duckdb_error_data_has_error)(duckdb_error_data error_data); #endif +// API to create and manipulate expressions +#ifdef DUCKDB_EXTENSION_API_VERSION_UNSTABLE + void (*duckdb_destroy_expression)(duckdb_expression *expr); + duckdb_logical_type (*duckdb_expression_return_type)(duckdb_expression expr); + bool (*duckdb_expression_is_foldable)(duckdb_expression expr); + duckdb_error_data (*duckdb_expression_fold)(duckdb_client_context context, duckdb_expression expr, + duckdb_value *out_value); +#endif + // New functions around the client context #ifdef DUCKDB_EXTENSION_API_VERSION_UNSTABLE idx_t (*duckdb_client_context_get_connection_id)(duckdb_client_context context); void (*duckdb_destroy_client_context)(duckdb_client_context *context); void (*duckdb_connection_get_client_context)(duckdb_connection connection, duckdb_client_context *out_context); duckdb_value (*duckdb_get_table_names)(duckdb_connection connection, const char *query, bool qualified); + void (*duckdb_connection_get_arrow_options)(duckdb_connection connection, duckdb_arrow_options *out_arrow_options); + void (*duckdb_destroy_arrow_options)(duckdb_arrow_options *arrow_options); +#endif + +// New query execution functions +#ifdef DUCKDB_EXTENSION_API_VERSION_UNSTABLE + duckdb_arrow_options (*duckdb_result_get_arrow_options)(duckdb_result *result); #endif // New functions around scalar function binding @@ -569,6 +599,8 @@ typedef struct { duckdb_delete_callback_t destroy); void *(*duckdb_scalar_function_get_bind_data)(duckdb_function_info info); void *(*duckdb_scalar_function_bind_get_extra_info)(duckdb_bind_info info); + idx_t (*duckdb_scalar_function_bind_get_argument_count)(duckdb_bind_info info); + duckdb_expression (*duckdb_scalar_function_bind_get_argument)(duckdb_bind_info info, idx_t index); #endif // New string functions that are added @@ -1020,6 +1052,13 @@ typedef struct { #define duckdb_appender_error_data duckdb_ext_api.duckdb_appender_error_data #define duckdb_append_default_to_chunk duckdb_ext_api.duckdb_append_default_to_chunk +// Version unstable_new_arrow_functions +#define duckdb_to_arrow_schema duckdb_ext_api.duckdb_to_arrow_schema +#define duckdb_data_chunk_to_arrow duckdb_ext_api.duckdb_data_chunk_to_arrow +#define duckdb_schema_from_arrow duckdb_ext_api.duckdb_schema_from_arrow +#define duckdb_data_chunk_from_arrow duckdb_ext_api.duckdb_data_chunk_from_arrow +#define duckdb_destroy_arrow_converted_schema duckdb_ext_api.duckdb_destroy_arrow_converted_schema + // Version unstable_new_error_data_functions #define duckdb_create_error_data duckdb_ext_api.duckdb_create_error_data #define duckdb_destroy_error_data duckdb_ext_api.duckdb_destroy_error_data @@ -1027,19 +1066,32 @@ typedef struct { #define duckdb_error_data_message duckdb_ext_api.duckdb_error_data_message #define duckdb_error_data_has_error duckdb_ext_api.duckdb_error_data_has_error +// Version unstable_new_expression_functions +#define duckdb_destroy_expression duckdb_ext_api.duckdb_destroy_expression +#define duckdb_expression_return_type duckdb_ext_api.duckdb_expression_return_type +#define duckdb_expression_is_foldable duckdb_ext_api.duckdb_expression_is_foldable +#define duckdb_expression_fold duckdb_ext_api.duckdb_expression_fold + // Version unstable_new_open_connect_functions #define duckdb_connection_get_client_context duckdb_ext_api.duckdb_connection_get_client_context +#define duckdb_connection_get_arrow_options duckdb_ext_api.duckdb_connection_get_arrow_options #define duckdb_client_context_get_connection_id duckdb_ext_api.duckdb_client_context_get_connection_id #define duckdb_destroy_client_context duckdb_ext_api.duckdb_destroy_client_context +#define duckdb_destroy_arrow_options duckdb_ext_api.duckdb_destroy_arrow_options #define duckdb_get_table_names duckdb_ext_api.duckdb_get_table_names +// Version unstable_new_query_execution_functions +#define duckdb_result_get_arrow_options duckdb_ext_api.duckdb_result_get_arrow_options + // Version unstable_new_scalar_function_functions -#define duckdb_scalar_function_set_bind duckdb_ext_api.duckdb_scalar_function_set_bind -#define duckdb_scalar_function_set_bind_data duckdb_ext_api.duckdb_scalar_function_set_bind_data -#define duckdb_scalar_function_bind_set_error duckdb_ext_api.duckdb_scalar_function_bind_set_error -#define duckdb_scalar_function_bind_get_extra_info duckdb_ext_api.duckdb_scalar_function_bind_get_extra_info -#define duckdb_scalar_function_get_bind_data duckdb_ext_api.duckdb_scalar_function_get_bind_data -#define duckdb_scalar_function_get_client_context duckdb_ext_api.duckdb_scalar_function_get_client_context +#define duckdb_scalar_function_set_bind duckdb_ext_api.duckdb_scalar_function_set_bind +#define duckdb_scalar_function_set_bind_data duckdb_ext_api.duckdb_scalar_function_set_bind_data +#define duckdb_scalar_function_bind_set_error duckdb_ext_api.duckdb_scalar_function_bind_set_error +#define duckdb_scalar_function_bind_get_extra_info duckdb_ext_api.duckdb_scalar_function_bind_get_extra_info +#define duckdb_scalar_function_get_bind_data duckdb_ext_api.duckdb_scalar_function_get_bind_data +#define duckdb_scalar_function_get_client_context duckdb_ext_api.duckdb_scalar_function_get_client_context +#define duckdb_scalar_function_bind_get_argument_count duckdb_ext_api.duckdb_scalar_function_bind_get_argument_count +#define duckdb_scalar_function_bind_get_argument duckdb_ext_api.duckdb_scalar_function_bind_get_argument // Version unstable_new_string_functions #define duckdb_value_to_string duckdb_ext_api.duckdb_value_to_string diff --git a/src/duckdb/src/main/attached_database.cpp b/src/duckdb/src/main/attached_database.cpp index b5ee4a3a0..7d85553b5 100644 --- a/src/duckdb/src/main/attached_database.cpp +++ b/src/duckdb/src/main/attached_database.cpp @@ -21,10 +21,10 @@ AttachOptions::AttachOptions(const DBConfigOptions &options) : access_mode(options.access_mode), db_type(options.database_type) { } -AttachOptions::AttachOptions(const unique_ptr &info, const AccessMode default_access_mode) +AttachOptions::AttachOptions(const unordered_map &attach_options, const AccessMode default_access_mode) : access_mode(default_access_mode) { - for (auto &entry : info->options) { + for (auto &entry : attach_options) { if (entry.first == "readonly" || entry.first == "read_only") { // Extract the read access mode. @@ -58,8 +58,7 @@ AttachOptions::AttachOptions(const unique_ptr &info, const AccessMod default_table = QualifiedName::Parse(StringValue::Get(entry.second.DefaultCastAs(LogicalType::VARCHAR))); continue; } - - options[entry.first] = entry.second; + options.emplace(entry.first, entry.second); } } @@ -75,7 +74,9 @@ AttachedDatabase::AttachedDatabase(DatabaseInstance &db, AttachedDatabaseType ty // This database does not have storage, or uses temporary_objects for in-memory storage. D_ASSERT(type == AttachedDatabaseType::TEMP_DATABASE || type == AttachedDatabaseType::SYSTEM_DATABASE); if (type == AttachedDatabaseType::TEMP_DATABASE) { - storage = make_uniq(*this, string(IN_MEMORY_PATH), false); + unordered_map options; + AttachOptions attach_options(options, AccessMode::READ_WRITE); + storage = make_uniq(*this, string(IN_MEMORY_PATH), attach_options); } catalog = make_uniq(*this); @@ -92,28 +93,9 @@ AttachedDatabase::AttachedDatabase(DatabaseInstance &db, Catalog &catalog_p, str } else { type = AttachedDatabaseType::READ_WRITE_DATABASE; } - for (auto &entry : options.options) { - if (StringUtil::CIEquals(entry.first, "block_size")) { - continue; - } - if (StringUtil::CIEquals(entry.first, "encryption_key")) { - continue; - } - if (StringUtil::CIEquals(entry.first, "encryption_cipher")) { - continue; - } - if (StringUtil::CIEquals(entry.first, "row_group_size")) { - continue; - } - if (StringUtil::CIEquals(entry.first, "storage_version")) { - continue; - } - throw BinderException("Unrecognized option for attach \"%s\"", entry.first); - } // We create the storage after the catalog to guarantee we allow extensions to instantiate the DuckCatalog. catalog = make_uniq(*this); - auto read_only = options.access_mode == AccessMode::READ_ONLY; - storage = make_uniq(*this, std::move(file_path_p), read_only); + storage = make_uniq(*this, std::move(file_path_p), options); transaction_manager = make_uniq(*this); internal = true; } @@ -128,15 +110,14 @@ AttachedDatabase::AttachedDatabase(DatabaseInstance &db, Catalog &catalog_p, Sto type = AttachedDatabaseType::READ_WRITE_DATABASE; } - StorageExtensionInfo *storage_info = storage_extension->storage_info.get(); - catalog = storage_extension->attach(storage_info, context, *this, name, info, options.access_mode); + optional_ptr storage_info = storage_extension->storage_info.get(); + catalog = storage_extension->attach(storage_info, context, *this, name, info, options); if (!catalog) { throw InternalException("AttachedDatabase - attach function did not return a catalog"); } if (catalog->IsDuckCatalog()) { // The attached database uses the DuckCatalog. - auto read_only = options.access_mode == AccessMode::READ_ONLY; - storage = make_uniq(*this, info.path, read_only); + storage = make_uniq(*this, info.path, options); } transaction_manager = storage_extension->create_transaction_manager(storage_info, *this, *catalog); if (!transaction_manager) { @@ -177,14 +158,14 @@ string AttachedDatabase::ExtractDatabaseName(const string &dbpath, FileSystem &f return name; } -void AttachedDatabase::Initialize(optional_ptr context, StorageOptions options) { +void AttachedDatabase::Initialize(optional_ptr context) { if (IsSystem()) { catalog->Initialize(context, true); } else { catalog->Initialize(context, false); } if (storage) { - storage->Initialize(QueryContext(context), options); + storage->Initialize(QueryContext(context)); } } diff --git a/src/duckdb/src/main/capi/arrow-c.cpp b/src/duckdb/src/main/capi/arrow-c.cpp index ab567393e..204f092ef 100644 --- a/src/duckdb/src/main/capi/arrow-c.cpp +++ b/src/duckdb/src/main/capi/arrow-c.cpp @@ -3,9 +3,11 @@ #include "duckdb/function/table/arrow.hpp" #include "duckdb/main/capi/capi_internal.hpp" #include "duckdb/main/prepared_statement_data.hpp" +#include "fmt/format.h" using duckdb::ArrowConverter; using duckdb::ArrowResultWrapper; +using duckdb::CClientArrowOptionsWrapper; using duckdb::Connection; using duckdb::DataChunk; using duckdb::LogicalType; @@ -14,6 +16,146 @@ using duckdb::PreparedStatementWrapper; using duckdb::QueryResult; using duckdb::QueryResultType; +duckdb_error_data duckdb_to_arrow_schema(duckdb_arrow_options arrow_options, duckdb_logical_type *types, + const char **names, idx_t column_count, struct ArrowSchema *out_schema) { + + if (!types || !names || !arrow_options || !out_schema) { + return duckdb_create_error_data(DUCKDB_ERROR_INVALID_INPUT, "Invalid argument(s) to duckdb_to_arrow_schema"); + } + duckdb::vector schema_types; + duckdb::vector schema_names; + for (idx_t i = 0; i < column_count; i++) { + schema_names.emplace_back(names[i]); + schema_types.emplace_back(*reinterpret_cast(types[i])); + } + const auto arrow_options_wrapper = reinterpret_cast(arrow_options); + try { + ArrowConverter::ToArrowSchema(out_schema, schema_types, schema_names, arrow_options_wrapper->properties); + } catch (const duckdb::Exception &ex) { + return duckdb_create_error_data(DUCKDB_ERROR_INVALID_INPUT, ex.what()); + } catch (const std::exception &ex) { + return duckdb_create_error_data(DUCKDB_ERROR_INVALID_INPUT, ex.what()); + } catch (...) { + return duckdb_create_error_data(DUCKDB_ERROR_INVALID_INPUT, "Unknown error occurred during conversion"); + } + return nullptr; +} + +duckdb_error_data duckdb_data_chunk_to_arrow(duckdb_arrow_options arrow_options, duckdb_data_chunk chunk, + struct ArrowArray *out_arrow_array) { + if (!arrow_options || !chunk || !out_arrow_array) { + return duckdb_create_error_data(DUCKDB_ERROR_INVALID_INPUT, + "Invalid argument(s) to duckdb_data_chunk_to_arrow"); + } + auto dchunk = reinterpret_cast(chunk); + auto arrow_options_wrapper = reinterpret_cast(arrow_options); + auto extension_type_cast = duckdb::ArrowTypeExtensionData::GetExtensionTypes( + *arrow_options_wrapper->properties.client_context, dchunk->GetTypes()); + + try { + ArrowConverter::ToArrowArray(*dchunk, out_arrow_array, arrow_options_wrapper->properties, extension_type_cast); + } catch (const duckdb::Exception &ex) { + return duckdb_create_error_data(DUCKDB_ERROR_INVALID_INPUT, ex.what()); + } catch (const std::exception &ex) { + return duckdb_create_error_data(DUCKDB_ERROR_INVALID_INPUT, ex.what()); + } catch (...) { + return duckdb_create_error_data(DUCKDB_ERROR_INVALID_INPUT, "Unknown error occurred during conversion"); + } + return nullptr; +} + +duckdb_error_data duckdb_schema_from_arrow(duckdb_connection connection, struct ArrowSchema *schema, + duckdb_arrow_converted_schema *out_types) { + if (!connection || !out_types || !schema) { + return duckdb_create_error_data(DUCKDB_ERROR_INVALID_INPUT, + "Invalid argument(s) to duckdb_data_chunk_to_arrow"); + } + duckdb::vector names; + const auto conn = reinterpret_cast(connection); + auto arrow_table = duckdb::make_uniq(); + try { + duckdb::vector return_types; + duckdb::ArrowTableFunction::PopulateArrowTableSchema(duckdb::DBConfig::GetConfig(*conn->context), *arrow_table, + *schema); + } catch (const duckdb::Exception &ex) { + return duckdb_create_error_data(DUCKDB_ERROR_INVALID_INPUT, ex.what()); + } catch (const std::exception &ex) { + return duckdb_create_error_data(DUCKDB_ERROR_INVALID_INPUT, ex.what()); + } catch (...) { + return duckdb_create_error_data(DUCKDB_ERROR_INVALID_INPUT, "Unknown error occurred during conversion"); + } + *out_types = reinterpret_cast(arrow_table.release()); + return nullptr; +} + +duckdb_error_data duckdb_data_chunk_from_arrow(duckdb_connection connection, struct ArrowArray *arrow_array, + duckdb_arrow_converted_schema converted_schema, + duckdb_data_chunk *out_chunk) { + if (!connection || !converted_schema || !out_chunk || !arrow_array) { + return duckdb_create_error_data(DUCKDB_ERROR_INVALID_INPUT, + "Invalid argument(s) to duckdb_data_chunk_to_arrow"); + } + auto arrow_table = reinterpret_cast(converted_schema); + auto conn = reinterpret_cast(connection); + auto &types = arrow_table->GetTypes(); + + auto dchunk = duckdb::make_uniq(); + dchunk->Initialize(duckdb::Allocator::DefaultAllocator(), types, duckdb::NumericCast(arrow_array->length)); + + auto &arrow_types = arrow_table->GetColumns(); + dchunk->SetCardinality(duckdb::NumericCast(arrow_array->length)); + for (idx_t i = 0; i < dchunk->ColumnCount(); i++) { + auto &parent_array = *arrow_array; + auto &array = parent_array.children[i]; + auto arrow_type = arrow_types.at(i); + auto array_physical_type = arrow_type->GetPhysicalType(); + auto array_state = duckdb::make_uniq(*conn->context); + // We need to make sure that our chunk will hold the ownership + array_state->owned_data = duckdb::make_shared_ptr(); + array_state->owned_data->arrow_array = *arrow_array; + // We set it to nullptr to effectively transfer the ownership + arrow_array->release = nullptr; + try { + switch (array_physical_type) { + case duckdb::ArrowArrayPhysicalType::DICTIONARY_ENCODED: + duckdb::ArrowToDuckDBConversion::ColumnArrowToDuckDBDictionary(dchunk->data[i], *array, 0, *array_state, + dchunk->size(), *arrow_type); + break; + case duckdb::ArrowArrayPhysicalType::RUN_END_ENCODED: + duckdb::ArrowToDuckDBConversion::ColumnArrowToDuckDBRunEndEncoded( + dchunk->data[i], *array, 0, *array_state, dchunk->size(), *arrow_type); + break; + case duckdb::ArrowArrayPhysicalType::DEFAULT: + duckdb::ArrowToDuckDBConversion::SetValidityMask(dchunk->data[i], *array, 0, dchunk->size(), + parent_array.offset, -1); + + duckdb::ArrowToDuckDBConversion::ColumnArrowToDuckDB(dchunk->data[i], *array, 0, *array_state, + dchunk->size(), *arrow_type); + break; + default: + return duckdb_create_error_data(DUCKDB_ERROR_NOT_IMPLEMENTED, + "Only Default Physical Types are currently supported"); + } + } catch (const duckdb::Exception &ex) { + return duckdb_create_error_data(DUCKDB_ERROR_INVALID_INPUT, ex.what()); + } catch (const std::exception &ex) { + return duckdb_create_error_data(DUCKDB_ERROR_INVALID_INPUT, ex.what()); + } catch (...) { + return duckdb_create_error_data(DUCKDB_ERROR_INVALID_INPUT, "Unknown error occurred during conversion"); + } + } + *out_chunk = reinterpret_cast(dchunk.release()); + return nullptr; +} + +void duckdb_destroy_arrow_converted_schema(duckdb_arrow_converted_schema *arrow_converted_schema) { + if (arrow_converted_schema && *arrow_converted_schema) { + auto converted_schema = reinterpret_cast(*arrow_converted_schema); + delete converted_schema; + *arrow_converted_schema = nullptr; + } +} + duckdb_state duckdb_query_arrow(duckdb_connection connection, const char *query, duckdb_arrow *out_result) { Connection *conn = (Connection *)connection; auto wrapper = new ArrowResultWrapper(); diff --git a/src/duckdb/src/main/capi/duckdb-c.cpp b/src/duckdb/src/main/capi/duckdb-c.cpp index a1d7657a4..43205cdf0 100644 --- a/src/duckdb/src/main/capi/duckdb-c.cpp +++ b/src/duckdb/src/main/capi/duckdb-c.cpp @@ -1,5 +1,6 @@ #include "duckdb/main/capi/capi_internal.hpp" +using duckdb::CClientArrowOptionsWrapper; using duckdb::CClientContextWrapper; using duckdb::Connection; using duckdb::DatabaseWrapper; @@ -151,6 +152,16 @@ void duckdb_connection_get_client_context(duckdb_connection connection, duckdb_c *out_context = reinterpret_cast(wrapper); } +void duckdb_connection_get_arrow_options(duckdb_connection connection, duckdb_arrow_options *out_arrow_options) { + if (!connection || !out_arrow_options) { + return; + } + Connection *conn = reinterpret_cast(connection); + auto client_properties = conn->context->GetClientProperties(); + auto wrapper = new CClientArrowOptionsWrapper(client_properties); + *out_arrow_options = reinterpret_cast(wrapper); +} + idx_t duckdb_client_context_get_connection_id(duckdb_client_context context) { auto wrapper = reinterpret_cast(context); return wrapper->context.GetConnectionId(); @@ -164,6 +175,14 @@ void duckdb_destroy_client_context(duckdb_client_context *context) { } } +void duckdb_destroy_arrow_options(duckdb_arrow_options *arrow_options) { + if (arrow_options && *arrow_options) { + auto wrapper = reinterpret_cast(*arrow_options); + delete wrapper; + *arrow_options = nullptr; + } +} + duckdb_state duckdb_query(duckdb_connection connection, const char *query, duckdb_result *out) { Connection *conn = reinterpret_cast(connection); auto result = conn->Query(query); diff --git a/src/duckdb/src/main/capi/duckdb_value-c.cpp b/src/duckdb/src/main/capi/duckdb_value-c.cpp index a74bae4f6..9149b36fb 100644 --- a/src/duckdb/src/main/capi/duckdb_value-c.cpp +++ b/src/duckdb/src/main/capi/duckdb_value-c.cpp @@ -352,7 +352,7 @@ duckdb_value duckdb_create_list_value(duckdb_logical_type type, duckdb_value *va } unwrapped_values.push_back(UnwrapValue(value)); } - duckdb::Value *list_value = new duckdb::Value; + auto list_value = new duckdb::Value; try { *list_value = duckdb::Value::LIST(logical_type, std::move(unwrapped_values)); } catch (...) { diff --git a/src/duckdb/src/main/capi/expression-c.cpp b/src/duckdb/src/main/capi/expression-c.cpp new file mode 100644 index 000000000..c416e03e5 --- /dev/null +++ b/src/duckdb/src/main/capi/expression-c.cpp @@ -0,0 +1,57 @@ +#include "duckdb/main/capi/capi_internal.hpp" + +#include "duckdb/execution/expression_executor.hpp" + +using duckdb::CClientContextWrapper; +using duckdb::ExpressionWrapper; + +void duckdb_destroy_expression(duckdb_expression *expr) { + if (!expr || !*expr) { + return; + } + auto wrapper = reinterpret_cast(*expr); + delete wrapper; + *expr = nullptr; +} + +duckdb_logical_type duckdb_expression_return_type(duckdb_expression expr) { + if (!expr) { + return nullptr; + } + auto wrapper = reinterpret_cast(expr); + auto logical_type = new duckdb::LogicalType(wrapper->expr->return_type); + return reinterpret_cast(logical_type); +} + +bool duckdb_expression_is_foldable(duckdb_expression expr) { + if (!expr) { + return false; + } + auto wrapper = reinterpret_cast(expr); + return wrapper->expr->IsFoldable(); +} + +duckdb_error_data duckdb_expression_fold(duckdb_client_context context, duckdb_expression expr, + duckdb_value *out_value) { + if (!expr || !duckdb_expression_is_foldable(expr)) { + return nullptr; + } + + auto value = new duckdb::Value; + try { + auto context_wrapper = reinterpret_cast(context); + auto expr_wrapper = reinterpret_cast(expr); + *value = duckdb::ExpressionExecutor::EvaluateScalar(context_wrapper->context, *expr_wrapper->expr); + *out_value = reinterpret_cast(value); + } catch (const duckdb::Exception &ex) { + delete value; + return duckdb_create_error_data(DUCKDB_ERROR_INVALID_INPUT, ex.what()); + } catch (const std::exception &ex) { + delete value; + return duckdb_create_error_data(DUCKDB_ERROR_INVALID_INPUT, ex.what()); + } catch (...) { + delete value; + return duckdb_create_error_data(DUCKDB_ERROR_INVALID_INPUT, "unknown error occurred during folding"); + } + return nullptr; +} diff --git a/src/duckdb/src/main/capi/result-c.cpp b/src/duckdb/src/main/capi/result-c.cpp index f2ab54e37..247b9f67f 100644 --- a/src/duckdb/src/main/capi/result-c.cpp +++ b/src/duckdb/src/main/capi/result-c.cpp @@ -443,6 +443,18 @@ duckdb_logical_type duckdb_column_logical_type(duckdb_result *result, idx_t col) return reinterpret_cast(new duckdb::LogicalType(result_data.result->types[col])); } +duckdb_arrow_options duckdb_result_get_arrow_options(duckdb_result *result) { + if (!result) { + return nullptr; + } + auto &result_data = *(reinterpret_cast(result->internal_data)); + if (!result_data.result) { + return nullptr; + } + auto arrow_options_wrapper = new duckdb::CClientArrowOptionsWrapper(result_data.result->client_properties); + return reinterpret_cast(arrow_options_wrapper); +} + idx_t duckdb_column_count(duckdb_result *result) { if (!result) { return 0; diff --git a/src/duckdb/src/main/capi/scalar_function-c.cpp b/src/duckdb/src/main/capi/scalar_function-c.cpp index f1189532f..eba21dbfc 100644 --- a/src/duckdb/src/main/capi/scalar_function-c.cpp +++ b/src/duckdb/src/main/capi/scalar_function-c.cpp @@ -148,6 +148,7 @@ void CAPIScalarFunction(DataChunk &input, ExpressionState &state, Vector &result } // namespace duckdb +using duckdb::ExpressionWrapper; using duckdb::GetCScalarFunction; using duckdb::GetCScalarFunctionBindInfo; using duckdb::GetCScalarFunctionInfo; @@ -270,6 +271,24 @@ void duckdb_scalar_function_bind_set_error(duckdb_bind_info info, const char *er bind_info.success = false; } +idx_t duckdb_scalar_function_bind_get_argument_count(duckdb_bind_info info) { + if (!info) { + return 0; + } + auto &bind_info = GetCScalarFunctionBindInfo(info); + return bind_info.arguments.size(); +} + +duckdb_expression duckdb_scalar_function_bind_get_argument(duckdb_bind_info info, idx_t index) { + if (!info || index >= duckdb_scalar_function_bind_get_argument_count(info)) { + return nullptr; + } + auto &bind_info = GetCScalarFunctionBindInfo(info); + auto wrapper = new ExpressionWrapper(); + wrapper->expr = bind_info.arguments[index]->Copy(); + return reinterpret_cast(wrapper); +} + void duckdb_scalar_function_set_extra_info(duckdb_scalar_function function, void *extra_info, duckdb_delete_callback_t destroy) { if (!function || !extra_info) { diff --git a/src/duckdb/src/main/database_manager.cpp b/src/duckdb/src/main/database_manager.cpp index 88e2c6dc6..f3b1599a2 100644 --- a/src/duckdb/src/main/database_manager.cpp +++ b/src/duckdb/src/main/database_manager.cpp @@ -30,8 +30,8 @@ void DatabaseManager::InitializeSystemCatalog() { } void DatabaseManager::FinalizeStartup() { - auto databases = GetDatabases(); - for (auto &db : databases) { + auto dbs = GetDatabases(); + for (auto &db : dbs) { db.get().FinalizeLoad(nullptr); } } @@ -113,40 +113,34 @@ void DatabaseManager::DetachDatabase(ClientContext &context, const string &name, } } -optional_ptr DatabaseManager::GetDatabaseFromPath(ClientContext &context, const string &path) { - auto database_list = GetDatabases(context); - for (auto &db_ref : database_list) { - auto &db = db_ref.get(); - if (db.IsSystem()) { - continue; - } - auto &catalog = Catalog::GetCatalog(db); - if (catalog.InMemory()) { - continue; - } - auto db_path = catalog.GetDBPath(); - if (StringUtil::CIEquals(path, db_path)) { - return &db; - } - } - return nullptr; -} - void DatabaseManager::CheckPathConflict(ClientContext &context, const string &path) { - // ensure that we did not already attach a database with the same path - bool path_exists; + // Ensure that we did not already attach a database with the same path. + string db_name = ""; { lock_guard path_lock(db_paths_lock); - path_exists = db_paths.find(path) != db_paths.end(); - } - if (path_exists) { - // check that the database is actually still attached - auto entry = GetDatabaseFromPath(context, path); - if (entry) { - throw BinderException("Unique file handle conflict: Database \"%s\" is already attached with path \"%s\", ", - entry->name, path); + auto it = db_paths_to_name.find(path); + if (it != db_paths_to_name.end()) { + db_name = it->second; } } + if (db_name.empty()) { + return; + } + + // Check against the catalog set. + auto entry = GetDatabase(context, db_name); + if (!entry) { + return; + } + if (entry->IsSystem()) { + return; + } + auto &catalog = Catalog::GetCatalog(*entry); + if (catalog.InMemory()) { + return; + } + throw BinderException("Unique file handle conflict: Database \"%s\" is already attached with path \"%s\", ", + db_name, path); } void DatabaseManager::InsertDatabasePath(ClientContext &context, const string &path, const string &name) { @@ -156,7 +150,7 @@ void DatabaseManager::InsertDatabasePath(ClientContext &context, const string &p CheckPathConflict(context, path); lock_guard path_lock(db_paths_lock); - db_paths.insert(path); + db_paths_to_name[path] = name; } void DatabaseManager::EraseDatabasePath(const string &path) { @@ -164,17 +158,17 @@ void DatabaseManager::EraseDatabasePath(const string &path) { return; } lock_guard path_lock(db_paths_lock); - auto path_it = db_paths.find(path); - if (path_it != db_paths.end()) { - db_paths.erase(path_it); + auto path_it = db_paths_to_name.find(path); + if (path_it != db_paths_to_name.end()) { + db_paths_to_name.erase(path_it); } } vector DatabaseManager::GetAttachedDatabasePaths() { lock_guard path_lock(db_paths_lock); vector paths; - for (auto &path : db_paths) { - paths.push_back(path); + for (auto &entry : db_paths_to_name) { + paths.push_back(entry.first); } return paths; } diff --git a/src/duckdb/src/main/extension/extension_helper.cpp b/src/duckdb/src/main/extension/extension_helper.cpp index a852b00da..b71fbc6f9 100644 --- a/src/duckdb/src/main/extension/extension_helper.cpp +++ b/src/duckdb/src/main/extension/extension_helper.cpp @@ -433,23 +433,6 @@ ExtensionLoadResult ExtensionHelper::LoadExtension(DuckDB &db, const std::string ExtensionLoadResult ExtensionHelper::LoadExtensionInternal(DuckDB &db, const std::string &extension, bool initial_load) { -#ifdef DUCKDB_TEST_REMOTE_INSTALL - if (!initial_load && StringUtil::Contains(DUCKDB_TEST_REMOTE_INSTALL, extension)) { - Connection con(db); - auto result = con.Query("INSTALL " + extension); - if (result->HasError()) { - result->Print(); - return ExtensionLoadResult::EXTENSION_UNKNOWN; - } - result = con.Query("LOAD " + extension); - if (result->HasError()) { - result->Print(); - return ExtensionLoadResult::EXTENSION_UNKNOWN; - } - return ExtensionLoadResult::LOADED_EXTENSION; - } -#endif - #ifdef DUCKDB_EXTENSIONS_TEST_WITH_LOADABLE // Note: weird comma's are on purpose to do easy string contains on a list of extension names if (!initial_load && StringUtil::Contains(DUCKDB_EXTENSIONS_TEST_WITH_LOADABLE, "," + extension + ",")) { diff --git a/src/duckdb/src/optimizer/join_order/cardinality_estimator.cpp b/src/duckdb/src/optimizer/join_order/cardinality_estimator.cpp index 2539bf37a..6dd086ffc 100644 --- a/src/duckdb/src/optimizer/join_order/cardinality_estimator.cpp +++ b/src/duckdb/src/optimizer/join_order/cardinality_estimator.cpp @@ -355,7 +355,7 @@ DenomInfo CardinalityEstimator::GetDenominator(JoinRelationSet &set) { auto denom_multiplier = 1.0 + static_cast(unused_edge_tdoms.size()); // It's possible cross-products were added and are not present in the filters in the relation_2_tdom - // structures. When that's the case, merge all remaining subgraphs. + // structures. When that's the case, merge all remaining subgraphs as if they are connected by a cross product if (subgraphs.size() > 1) { auto final_subgraph = subgraphs.at(0); for (auto merge_with = subgraphs.begin() + 1; merge_with != subgraphs.end(); merge_with++) { @@ -367,6 +367,23 @@ DenomInfo CardinalityEstimator::GetDenominator(JoinRelationSet &set) { final_subgraph.denom *= merge_with->denom; } } + if (!subgraphs.empty()) { + // Some relations are connected by cross products and will not end up in a subgraph + // Check and make sure all relations were considered, if not, they are connected to the graph by cross products + auto &returning_subgraph = subgraphs.at(0); + if (returning_subgraph.relations->count != set.count) { + for (idx_t rel_index = 0; rel_index < set.count; rel_index++) { + auto relation_id = set.relations[rel_index]; + auto &rel = set_manager.GetJoinRelation(relation_id); + if (!JoinRelationSet::IsSubset(*returning_subgraph.relations, rel)) { + returning_subgraph.numerator_relations = + &set_manager.Union(*returning_subgraph.numerator_relations, rel); + returning_subgraph.relations = &set_manager.Union(*returning_subgraph.relations, rel); + } + } + } + } + // can happen if a table has cardinality 0, a tdom is set to 0, or if a cross product is used. if (subgraphs.empty() || subgraphs.at(0).denom == 0) { // denominator is 1 and numerators are a cross product of cardinalities. @@ -377,7 +394,6 @@ DenomInfo CardinalityEstimator::GetDenominator(JoinRelationSet &set) { template <> double CardinalityEstimator::EstimateCardinalityWithSet(JoinRelationSet &new_set) { - if (relation_set_2_cardinality.find(new_set.ToString()) != relation_set_2_cardinality.end()) { return relation_set_2_cardinality[new_set.ToString()].cardinality_before_filters; } diff --git a/src/duckdb/src/optimizer/join_order/plan_enumerator.cpp b/src/duckdb/src/optimizer/join_order/plan_enumerator.cpp index 54f846615..7181570cb 100644 --- a/src/duckdb/src/optimizer/join_order/plan_enumerator.cpp +++ b/src/duckdb/src/optimizer/join_order/plan_enumerator.cpp @@ -68,7 +68,7 @@ static vector> GetAllNeighborSets(vector neighbors) count += 1; } } - D_ASSERT(count == static_cast(std::pow(2, neighbors.size()) - 1)); + D_ASSERT(count == static_cast(std::pow(2, neighbors.size() - 1))); } #endif return ret; diff --git a/src/duckdb/src/optimizer/optimizer.cpp b/src/duckdb/src/optimizer/optimizer.cpp index 93ec543ee..34d2f44bb 100644 --- a/src/duckdb/src/optimizer/optimizer.cpp +++ b/src/duckdb/src/optimizer/optimizer.cpp @@ -104,6 +104,7 @@ void Optimizer::RunBuiltInOptimizers() { case LogicalOperatorType::LOGICAL_TRANSACTION: case LogicalOperatorType::LOGICAL_PRAGMA: case LogicalOperatorType::LOGICAL_SET: + case LogicalOperatorType::LOGICAL_ATTACH: case LogicalOperatorType::LOGICAL_UPDATE_EXTENSIONS: case LogicalOperatorType::LOGICAL_CREATE_SECRET: case LogicalOperatorType::LOGICAL_EXTENSION_OPERATOR: diff --git a/src/duckdb/src/parallel/task_scheduler.cpp b/src/duckdb/src/parallel/task_scheduler.cpp index 9fda080f4..af2951947 100644 --- a/src/duckdb/src/parallel/task_scheduler.cpp +++ b/src/duckdb/src/parallel/task_scheduler.cpp @@ -419,7 +419,7 @@ idx_t TaskScheduler::GetEstimatedCPUId() { /* Other oses most likely use tpidr_el0 instead */ uintptr_t c; asm volatile("mrs %x0, tpidrro_el0" : "=r"(c)::"memory"); - return (idx_t)(c & (1 << 3) - 1); + return (idx_t)(c & ((1 << 3) - 1)); #else #ifndef DUCKDB_NO_THREADS // fallback to thread id diff --git a/src/duckdb/src/parser/parsed_data/attach_info.cpp b/src/duckdb/src/parser/parsed_data/attach_info.cpp index c4e1548c4..336150ed7 100644 --- a/src/duckdb/src/parser/parsed_data/attach_info.cpp +++ b/src/duckdb/src/parser/parsed_data/attach_info.cpp @@ -7,51 +7,6 @@ namespace duckdb { -StorageOptions AttachInfo::GetStorageOptions() const { - StorageOptions storage_options; - string storage_version_user_provided = ""; - for (auto &entry : options) { - if (entry.first == "block_size") { - // Extract the block allocation size. This is NOT the actual memory available on a block (block_size), - // even though the corresponding option we expose to the user is called "block_size". - storage_options.block_alloc_size = entry.second.GetValue(); - } else if (entry.first == "encryption_key") { - // check the type of the key - auto type = entry.second.type(); - if (type.id() != LogicalTypeId::VARCHAR) { - throw BinderException("\"%s\" is not a valid key. A key must be of type VARCHAR", - entry.second.ToString()); - } else if (entry.second.GetValue().empty()) { - throw BinderException("Not a valid key. A key cannot be empty"); - } - storage_options.user_key = - make_shared_ptr(StringValue::Get(entry.second.DefaultCastAs(LogicalType::BLOB))); - storage_options.block_header_size = DEFAULT_ENCRYPTION_BLOCK_HEADER_SIZE; - storage_options.encryption = true; - } else if (entry.first == "encryption_cipher") { - throw BinderException("\"%s\" is not a valid cipher. Only AES GCM is supported.", entry.second.ToString()); - } else if (entry.first == "row_group_size") { - storage_options.row_group_size = entry.second.GetValue(); - } else if (entry.first == "storage_version") { - storage_version_user_provided = entry.second.ToString(); - storage_options.storage_version = - SerializationCompatibility::FromString(entry.second.ToString()).serialization_version; - } - } - if (storage_options.encryption && (!storage_options.storage_version.IsValid() || - storage_options.storage_version.GetIndex() < - SerializationCompatibility::FromString("v1.4.0").serialization_version)) { - if (!storage_version_user_provided.empty()) { - throw InvalidInputException( - "Explicit provided STORAGE_VERSION (\"%s\") and ENCRYPTION_KEY (storage >= v1.4.0) are not compatible", - storage_version_user_provided); - } - // set storage version to v1.4.0 - storage_options.storage_version = SerializationCompatibility::FromString("v1.4.0").serialization_version; - } - return storage_options; -} - unique_ptr AttachInfo::Copy() const { auto result = make_uniq(); result->name = name; diff --git a/src/duckdb/src/parser/statement/merge_into_statement.cpp b/src/duckdb/src/parser/statement/merge_into_statement.cpp index 0b6ebcd95..0943f000b 100644 --- a/src/duckdb/src/parser/statement/merge_into_statement.cpp +++ b/src/duckdb/src/parser/statement/merge_into_statement.cpp @@ -16,6 +16,9 @@ MergeIntoStatement::MergeIntoStatement(const MergeIntoStatement &other) : SQLSta action_list.push_back(action->Copy()); } } + for (auto &entry : other.returning_list) { + returning_list.push_back(entry->Copy()); + } cte_map = other.cte_map.Copy(); } @@ -60,6 +63,20 @@ string MergeIntoStatement::ToString() const { result += action->ToString(); } } + if (!returning_list.empty()) { + result += " RETURNING "; + for (idx_t i = 0; i < returning_list.size(); i++) { + if (i > 0) { + result += ", "; + } + auto column = returning_list[i]->ToString(); + if (!returning_list[i]->GetAlias().empty()) { + column += + StringUtil::Format(" AS %s", KeywordHelper::WriteOptionallyQuoted(returning_list[i]->GetAlias())); + } + result += column; + } + } return result; } diff --git a/src/duckdb/src/parser/tableref/showref.cpp b/src/duckdb/src/parser/tableref/showref.cpp index 950defb31..6a40719d3 100644 --- a/src/duckdb/src/parser/tableref/showref.cpp +++ b/src/duckdb/src/parser/tableref/showref.cpp @@ -1,4 +1,5 @@ #include "duckdb/parser/tableref/showref.hpp" +#include "duckdb/parser/keyword_helper.hpp" namespace duckdb { @@ -9,6 +10,17 @@ string ShowRef::ToString() const { string result; if (show_type == ShowType::SUMMARY) { result += "SUMMARIZE "; + } else if (show_type == ShowType::SHOW_FROM) { + result += "SHOW TABLES FROM "; + string name = ""; + if (!catalog_name.empty()) { + name += KeywordHelper::WriteOptionallyQuoted(catalog_name, '"'); + if (!schema_name.empty()) { + name += "."; + } + } + name += KeywordHelper::WriteOptionallyQuoted(schema_name, '"'); + result += name; } else { result += "DESCRIBE "; } @@ -38,6 +50,8 @@ bool ShowRef::Equals(const TableRef &other_p) const { unique_ptr ShowRef::Copy() { auto copy = make_uniq(); + copy->catalog_name = catalog_name; + copy->schema_name = schema_name; copy->table_name = table_name; copy->query = query ? query->Copy() : nullptr; copy->show_type = show_type; diff --git a/src/duckdb/src/parser/transform/statement/transform_merge_into.cpp b/src/duckdb/src/parser/transform/statement/transform_merge_into.cpp index 464c1bb51..634b58b92 100644 --- a/src/duckdb/src/parser/transform/statement/transform_merge_into.cpp +++ b/src/duckdb/src/parser/transform/statement/transform_merge_into.cpp @@ -102,6 +102,9 @@ unique_ptr Transformer::TransformMergeInto(duckdb_libpgquery::PGMe for (auto &entry : unconditional_actions) { result->actions[entry.first].push_back(std::move(entry.second)); } + if (stmt.returningList) { + TransformExpressionList(*stmt.returningList, result->returning_list); + } return std::move(result); } diff --git a/src/duckdb/src/parser/transform/statement/transform_show.cpp b/src/duckdb/src/parser/transform/statement/transform_show.cpp index 648dbb9b7..ebd0ebd8d 100644 --- a/src/duckdb/src/parser/transform/statement/transform_show.cpp +++ b/src/duckdb/src/parser/transform/statement/transform_show.cpp @@ -15,8 +15,23 @@ unique_ptr Transformer::TransformShow(duckdb_libpgquery::PGVariableSh select_node->select_list.push_back(make_uniq()); auto showref = make_uniq(); if (stmt.set) { - // describing a set (e.g. SHOW ALL TABLES) - push it in the table name - showref->table_name = stmt.set; + if (std::string(stmt.set) == "__show_tables_from_database") { + showref->show_type = ShowType::SHOW_FROM; + auto qualified_name = TransformQualifiedName(*stmt.relation); + if (!IsInvalidCatalog(qualified_name.catalog)) { + throw ParserException("Expected \"SHOW TABLES FROM database\", \"SHOW TABLES FROM schema\", or " + "\"SHOW TABLES FROM database.schema\""); + } + if (qualified_name.schema.empty()) { + showref->schema_name = qualified_name.name; + } else { + showref->catalog_name = qualified_name.schema; + showref->schema_name = qualified_name.name; + } + } else { + // describing a set (e.g. SHOW ALL TABLES) - push it in the table name + showref->table_name = stmt.set; + } } else if (!stmt.relation->schemaname) { // describing an unqualified relation - check if this is a "special" relation string table_name = StringUtil::Lower(stmt.relation->relname); @@ -24,7 +39,7 @@ unique_ptr Transformer::TransformShow(duckdb_libpgquery::PGVariableSh showref->table_name = "\"" + std::move(table_name) + "\""; } } - if (showref->table_name.empty()) { + if (showref->table_name.empty() && showref->show_type != ShowType::SHOW_FROM) { // describing a single relation // wrap the relation in a "SELECT * FROM [table_name]" query auto show_select_node = make_uniq(); @@ -34,7 +49,10 @@ unique_ptr Transformer::TransformShow(duckdb_libpgquery::PGVariableSh showref->query = std::move(show_select_node); } - showref->show_type = stmt.is_summary ? ShowType::SUMMARY : ShowType::DESCRIBE; + // If the show type is set to default, check if summary + if (showref->show_type == ShowType::DESCRIBE) { + showref->show_type = stmt.is_summary ? ShowType::SUMMARY : ShowType::DESCRIBE; + } select_node->from_table = std::move(showref); return std::move(select_node); } diff --git a/src/duckdb/src/planner/bind_context.cpp b/src/duckdb/src/planner/bind_context.cpp index 7f1382dfc..b6e5df81f 100644 --- a/src/duckdb/src/planner/bind_context.cpp +++ b/src/duckdb/src/planner/bind_context.cpp @@ -625,6 +625,13 @@ void BindContext::AddBinding(unique_ptr binding) { bindings_list.push_back(std::move(binding)); } +void BindContext::AddBaseTable(idx_t index, const string &alias, const vector &names, + const vector &types, vector &bound_column_ids, + TableCatalogEntry &entry, virtual_column_map_t virtual_columns) { + AddBinding( + make_uniq(alias, types, names, bound_column_ids, &entry, index, std::move(virtual_columns))); +} + void BindContext::AddBaseTable(idx_t index, const string &alias, const vector &names, const vector &types, vector &bound_column_ids, TableCatalogEntry &entry, bool add_virtual_columns) { @@ -632,8 +639,7 @@ void BindContext::AddBaseTable(idx_t index, const string &alias, const vector(alias, types, names, bound_column_ids, &entry, index, std::move(virtual_columns))); + AddBaseTable(index, alias, names, types, bound_column_ids, entry, std::move(virtual_columns)); } void BindContext::AddBaseTable(idx_t index, const string &alias, const vector &names, diff --git a/src/duckdb/src/planner/binder.cpp b/src/duckdb/src/planner/binder.cpp index 5006b5c2b..c443ad72f 100644 --- a/src/duckdb/src/planner/binder.cpp +++ b/src/duckdb/src/planner/binder.cpp @@ -547,10 +547,10 @@ void VerifyNotExcluded(const ParsedExpression &root_expr) { BoundStatement Binder::BindReturning(vector> returning_list, TableCatalogEntry &table, const string &alias, idx_t update_table_index, - unique_ptr child_operator, BoundStatement result) { + unique_ptr child_operator, virtual_column_map_t virtual_columns) { vector types; - vector names; + vector names; auto binder = Binder::CreateBinder(context); @@ -565,12 +565,14 @@ BoundStatement Binder::BindReturning(vector> return column_count++; } - binder->bind_context.AddBaseTable(update_table_index, alias, names, types, bound_columns, table, false); + binder->bind_context.AddBaseTable(update_table_index, alias, names, types, bound_columns, table, + std::move(virtual_columns)); ReturningBinder returning_binder(*binder, context); vector> projection_expressions; LogicalType result_type; vector> new_returning_list; + BoundStatement result; binder->ExpandStarExpressions(returning_list, new_returning_list); for (auto &returning_expr : new_returning_list) { VerifyNotExcluded(*returning_expr); diff --git a/src/duckdb/src/planner/binder/statement/bind_copy.cpp b/src/duckdb/src/planner/binder/statement/bind_copy.cpp index 21d20d8ff..f5c050839 100644 --- a/src/duckdb/src/planner/binder/statement/bind_copy.cpp +++ b/src/duckdb/src/planner/binder/statement/bind_copy.cpp @@ -383,9 +383,9 @@ BoundStatement Binder::BindCopyFrom(CopyStatement &stmt) { expected_names.push_back(col.Name()); } } - + CopyFromFunctionBindInput input(*stmt.info, copy_function.function.copy_from_function); auto function_data = - copy_function.function.copy_from_bind(context, *stmt.info, expected_names, bound_insert.expected_types); + copy_function.function.copy_from_bind(context, input, expected_names, bound_insert.expected_types); auto get = make_uniq(GenerateTableIndex(), copy_function.function.copy_from_function, std::move(function_data), bound_insert.expected_types, expected_names); for (idx_t i = 0; i < bound_insert.expected_types.size(); i++) { diff --git a/src/duckdb/src/planner/binder/statement/bind_create_table.cpp b/src/duckdb/src/planner/binder/statement/bind_create_table.cpp index 686b99676..7973de7a3 100644 --- a/src/duckdb/src/planner/binder/statement/bind_create_table.cpp +++ b/src/duckdb/src/planner/binder/statement/bind_create_table.cpp @@ -35,8 +35,8 @@ static void CreateColumnDependencyManager(BoundCreateTableInfo &info) { } } -static void VerifyCompressionType(optional_ptr storage_manager, DBConfig &config, - BoundCreateTableInfo &info) { +static void VerifyCompressionType(ClientContext &context, optional_ptr storage_manager, + DBConfig &config, BoundCreateTableInfo &info) { auto &base = info.base->Cast(); for (auto &col : base.columns.Logical()) { auto compression_type = col.CompressionType(); @@ -45,7 +45,15 @@ static void VerifyCompressionType(optional_ptr storage_manager, "and only has decompress support", CompressionTypeToString(compression_type)); } - const auto &logical_type = col.GetType(); + auto logical_type = col.GetType(); + if (logical_type.id() == LogicalTypeId::USER && logical_type.HasAlias()) { + // Resolve user type if possible + const auto type_entry = Catalog::GetEntry( + context, INVALID_CATALOG, INVALID_SCHEMA, logical_type.GetAlias(), OnEntryNotFound::RETURN_NULL); + if (type_entry) { + logical_type = type_entry->user_type; + } + } auto physical_type = logical_type.InternalType(); if (compression_type == CompressionType::COMPRESSION_AUTO) { continue; @@ -630,7 +638,7 @@ unique_ptr Binder::BindCreateTableInfo(unique_ptrtype != TableReferenceType::BASE_TABLE) { @@ -84,8 +82,9 @@ BoundStatement Binder::Bind(DeleteStatement &stmt) { unique_ptr del_as_logicaloperator = std::move(del); return BindReturning(std::move(stmt.returning_list), table, stmt.table->alias, update_table_index, - std::move(del_as_logicaloperator), std::move(result)); + std::move(del_as_logicaloperator)); } + BoundStatement result; result.plan = std::move(del); result.names = {"Count"}; result.types = {LogicalType::BIGINT}; diff --git a/src/duckdb/src/planner/binder/statement/bind_insert.cpp b/src/duckdb/src/planner/binder/statement/bind_insert.cpp index a3cc9369d..946266a45 100644 --- a/src/duckdb/src/planner/binder/statement/bind_insert.cpp +++ b/src/duckdb/src/planner/binder/statement/bind_insert.cpp @@ -625,14 +625,12 @@ BoundStatement Binder::Bind(InsertStatement &stmt) { if (!stmt.returning_list.empty()) { insert->return_chunk = true; - result.types.clear(); - result.names.clear(); auto insert_table_index = GenerateTableIndex(); insert->table_index = insert_table_index; unique_ptr index_as_logicaloperator = std::move(insert); return BindReturning(std::move(stmt.returning_list), table, stmt.table_ref ? stmt.table_ref->alias : string(), - insert_table_index, std::move(index_as_logicaloperator), std::move(result)); + insert_table_index, std::move(index_as_logicaloperator)); } D_ASSERT(result.types.size() == result.names.size()); diff --git a/src/duckdb/src/planner/binder/statement/bind_merge_into.cpp b/src/duckdb/src/planner/binder/statement/bind_merge_into.cpp index b92b00fd4..b68b4f8d7 100644 --- a/src/duckdb/src/planner/binder/statement/bind_merge_into.cpp +++ b/src/duckdb/src/planner/binder/statement/bind_merge_into.cpp @@ -74,7 +74,7 @@ unique_ptr Binder::BindMergeAction(LogicalMergeInto &merge // construct a dummy projection and update LogicalProjection proj(proj_index, std::move(expressions)); LogicalUpdate update(table); - update.return_chunk = false; + update.return_chunk = merge_into.return_chunk; update.columns = std::move(result->columns); update.expressions = std::move(result->expressions); update.bound_defaults = std::move(merge_into.bound_defaults); @@ -171,9 +171,9 @@ void RewriteMergeBindings(LogicalOperator &op, const vector &sour } BoundStatement Binder::Bind(MergeIntoStatement &stmt) { - // bind the target table auto target_binder = Binder::CreateBinder(context, this); + string table_alias = stmt.target->alias; auto bound_table = target_binder->Bind(*stmt.target); if (bound_table->type != TableReferenceType::BASE_TABLE) { throw BinderException("Can only merge into base tables!"); @@ -235,8 +235,10 @@ BoundStatement Binder::Bind(MergeIntoStatement &stmt) { auto &get = root->children[inverted ? 0 : 1]->Cast(); auto merge_into = make_uniq(table); - merge_into->table_index = GenerateTableIndex(); + if (!stmt.returning_list.empty()) { + merge_into->return_chunk = true; + } // bind table constraints/default values in case these are referenced auto &catalog_name = table.ParentCatalog().GetName(); @@ -305,10 +307,21 @@ BoundStatement Binder::Bind(MergeIntoStatement &stmt) { merge_into->AddChild(std::move(proj)); + if (!stmt.returning_list.empty()) { + auto merge_table_index = merge_into->table_index; + unique_ptr index_as_logicaloperator = std::move(merge_into); + + // add the merge_action virtual column + virtual_column_map_t virtual_columns; + virtual_columns.insert(make_pair(VIRTUAL_COLUMN_START, TableColumn("merge_action", LogicalType::VARCHAR))); + return BindReturning(std::move(stmt.returning_list), table, table_alias, merge_table_index, + std::move(index_as_logicaloperator), std::move(virtual_columns)); + } + BoundStatement result; + result.plan = std::move(merge_into); result.names = {"Count"}; result.types = {LogicalType::BIGINT}; - result.plan = std::move(merge_into); auto &properties = GetStatementProperties(); properties.allow_stream_result = false; diff --git a/src/duckdb/src/planner/binder/statement/bind_update.cpp b/src/duckdb/src/planner/binder/statement/bind_update.cpp index 03d33b5be..650b23b89 100644 --- a/src/duckdb/src/planner/binder/statement/bind_update.cpp +++ b/src/duckdb/src/planner/binder/statement/bind_update.cpp @@ -106,7 +106,6 @@ void Binder::BindRowIdColumns(TableCatalogEntry &table, LogicalGet &get, vector< } BoundStatement Binder::Bind(UpdateStatement &stmt) { - BoundStatement result; unique_ptr root; // visit the table reference @@ -184,9 +183,10 @@ BoundStatement Binder::Bind(UpdateStatement &stmt) { unique_ptr update_as_logicaloperator = std::move(update); return BindReturning(std::move(stmt.returning_list), table, stmt.table->alias, update_table_index, - std::move(update_as_logicaloperator), std::move(result)); + std::move(update_as_logicaloperator)); } + BoundStatement result; result.names = {"Count"}; result.types = {LogicalType::BIGINT}; result.plan = std::move(update); diff --git a/src/duckdb/src/planner/binder/tableref/bind_showref.cpp b/src/duckdb/src/planner/binder/tableref/bind_showref.cpp index 78803e395..b23456cab 100644 --- a/src/duckdb/src/planner/binder/tableref/bind_showref.cpp +++ b/src/duckdb/src/planner/binder/tableref/bind_showref.cpp @@ -9,6 +9,10 @@ #include "duckdb/planner/operator/logical_get.hpp" #include "duckdb/planner/operator/logical_projection.hpp" #include "duckdb/catalog/catalog_entry/table_catalog_entry.hpp" +#include "duckdb/catalog/catalog.hpp" +#include "duckdb/catalog/catalog_search_path.hpp" +#include "duckdb/main/client_data.hpp" +#include "duckdb/main/client_context.hpp" namespace duckdb { @@ -151,6 +155,32 @@ unique_ptr Binder::BindShowTable(ShowRef &ref) { sql = PragmaShowDatabases(); } else if (lname == "\"tables\"") { sql = PragmaShowTables(); + } else if (ref.show_type == ShowType::SHOW_FROM) { + auto catalog_name = ref.catalog_name; + auto schema_name = ref.schema_name; + + // Check for unqualified name, promote schema to catalog if unambiguous, and set schema_name to empty if so + Binder::BindSchemaOrCatalog(catalog_name, schema_name); + + // If fully qualified, check if the schema exists + if (!catalog_name.empty() && !schema_name.empty()) { + auto schema_entry = Catalog::GetSchema(context, catalog_name, schema_name, OnEntryNotFound::RETURN_NULL); + if (!schema_entry) { + throw CatalogException("SHOW TABLES FROM: No catalog + schema named \"%s.%s\" found.", catalog_name, + schema_name); + } + } else if (catalog_name.empty() && !schema_name.empty()) { + // We have a schema name, use default catalog + auto &client_data = ClientData::Get(context); + auto &default_entry = client_data.catalog_search_path->GetDefault(); + catalog_name = default_entry.catalog; + auto schema_entry = Catalog::GetSchema(context, catalog_name, schema_name, OnEntryNotFound::RETURN_NULL); + if (!schema_entry) { + throw CatalogException("SHOW TABLES FROM: No catalog + schema named \"%s.%s\" found.", catalog_name, + schema_name); + } + } + sql = PragmaShowTables(catalog_name, schema_name); } else if (lname == "\"variables\"") { sql = PragmaShowVariables(); } else if (lname == "__show_tables_expanded") { diff --git a/src/duckdb/src/planner/operator/logical_merge_into.cpp b/src/duckdb/src/planner/operator/logical_merge_into.cpp index f08d6cfb9..fa99f69ab 100644 --- a/src/duckdb/src/planner/operator/logical_merge_into.cpp +++ b/src/duckdb/src/planner/operator/logical_merge_into.cpp @@ -17,15 +17,27 @@ LogicalMergeInto::LogicalMergeInto(ClientContext &context, const unique_ptr LogicalMergeInto::GetTableIndex() const { + return vector {table_index}; } vector LogicalMergeInto::GetColumnBindings() { + if (return_chunk) { + return GenerateColumnBindings(table_index, table.GetTypes().size() + 1); + } return {ColumnBinding(0, 0)}; } void LogicalMergeInto::ResolveTypes() { - types.emplace_back(LogicalType::BIGINT); + if (return_chunk) { + types = table.GetTypes(); + types.push_back(LogicalType::VARCHAR); + } else { + types.emplace_back(LogicalType::BIGINT); + } } } // namespace duckdb diff --git a/src/duckdb/src/storage/checkpoint/row_group_writer.cpp b/src/duckdb/src/storage/checkpoint/row_group_writer.cpp index 14be12778..4d46c6a34 100644 --- a/src/duckdb/src/storage/checkpoint/row_group_writer.cpp +++ b/src/duckdb/src/storage/checkpoint/row_group_writer.cpp @@ -18,8 +18,16 @@ CheckpointType SingleFileRowGroupWriter::GetCheckpointType() const { return writer.GetCheckpointType(); } -MetadataWriter &SingleFileRowGroupWriter::GetPayloadWriter() { +WriteStream &SingleFileRowGroupWriter::GetPayloadWriter() { return table_data_writer; } +MetaBlockPointer SingleFileRowGroupWriter::GetMetaBlockPointer() { + return table_data_writer.GetMetaBlockPointer(); +} + +optional_ptr SingleFileRowGroupWriter::GetMetadataManager() { + return table_data_writer.GetManager(); +} + } // namespace duckdb diff --git a/src/duckdb/src/storage/checkpoint_manager.cpp b/src/duckdb/src/storage/checkpoint_manager.cpp index 9dd675773..c9f6f8ec2 100644 --- a/src/duckdb/src/storage/checkpoint_manager.cpp +++ b/src/duckdb/src/storage/checkpoint_manager.cpp @@ -31,6 +31,7 @@ #include "duckdb/transaction/meta_transaction.hpp" #include "duckdb/transaction/transaction_manager.hpp" #include "duckdb/catalog/dependency_manager.hpp" +#include "duckdb/common/serializer/memory_stream.hpp" namespace duckdb { diff --git a/src/duckdb/src/storage/compression/dict_fsst/decompression.cpp b/src/duckdb/src/storage/compression/dict_fsst/decompression.cpp index fd6640e68..0546096bb 100644 --- a/src/duckdb/src/storage/compression/dict_fsst/decompression.cpp +++ b/src/duckdb/src/storage/compression/dict_fsst/decompression.cpp @@ -219,7 +219,6 @@ bool CompressedStringScanState::AllowDictionaryScan(idx_t scan_count) { void CompressedStringScanState::ScanToDictionaryVector(ColumnSegment &segment, Vector &result, idx_t result_offset, idx_t start, idx_t scan_count) { - D_ASSERT(start % BitpackingPrimitives::BITPACKING_ALGORITHM_GROUP_SIZE == 0); D_ASSERT(scan_count == STANDARD_VECTOR_SIZE); D_ASSERT(result_offset == 0); diff --git a/src/duckdb/src/storage/compression/fixed_size_uncompressed.cpp b/src/duckdb/src/storage/compression/fixed_size_uncompressed.cpp index 24919388d..afd335dab 100644 --- a/src/duckdb/src/storage/compression/fixed_size_uncompressed.cpp +++ b/src/duckdb/src/storage/compression/fixed_size_uncompressed.cpp @@ -72,9 +72,12 @@ void UncompressedCompressState::CreateEmptySegment(idx_t row_start) { info.GetBlockManager()); if (type.InternalType() == PhysicalType::VARCHAR) { auto &state = compressed_segment->GetSegmentState()->Cast(); - auto &partial_block_manager = checkpoint_data.GetCheckpointState().GetPartialBlockManager(); - state.block_manager = partial_block_manager.GetBlockManager(); - state.overflow_writer = make_uniq(partial_block_manager); + auto &storage_manager = checkpoint_data.GetStorageManager(); + if (!storage_manager.InMemory()) { + auto &partial_block_manager = checkpoint_data.GetCheckpointState().GetPartialBlockManager(); + state.block_manager = partial_block_manager.GetBlockManager(); + state.overflow_writer = make_uniq(partial_block_manager); + } } current_segment = std::move(compressed_segment); current_segment->InitializeAppend(append_state); @@ -84,8 +87,10 @@ void UncompressedCompressState::FlushSegment(idx_t segment_size) { auto &state = checkpoint_data.GetCheckpointState(); if (current_segment->type.InternalType() == PhysicalType::VARCHAR) { auto &segment_state = current_segment->GetSegmentState()->Cast(); - segment_state.overflow_writer->Flush(); - segment_state.overflow_writer.reset(); + if (segment_state.overflow_writer) { + segment_state.overflow_writer->Flush(); + segment_state.overflow_writer.reset(); + } } append_state.child_appends.clear(); append_state.append_state.reset(); diff --git a/src/duckdb/src/storage/data_table.cpp b/src/duckdb/src/storage/data_table.cpp index cdd98cebb..c7c501216 100644 --- a/src/duckdb/src/storage/data_table.cpp +++ b/src/duckdb/src/storage/data_table.cpp @@ -163,6 +163,7 @@ DataTable::DataTable(ClientContext &context, DataTable &parent, BoundConstraint DataTable::DataTable(ClientContext &context, DataTable &parent, idx_t changed_idx, const LogicalType &target_type, const vector &bound_columns, Expression &cast_expr) : db(parent.db), info(parent.info), version(DataTableVersion::MAIN_TABLE) { + auto &local_storage = LocalStorage::Get(context, db); // prevent any tuples from being added to the parent lock_guard lock(append_lock); @@ -187,7 +188,7 @@ DataTable::DataTable(ClientContext &context, DataTable &parent, idx_t changed_id // set up the statistics for the table // the column that had its type changed will have the new statistics computed during conversion - this->row_groups = parent.row_groups->AlterType(context, changed_idx, target_type, bound_columns, cast_expr); + row_groups = parent.row_groups->AlterType(context, changed_idx, target_type, bound_columns, cast_expr); // scan the original table, and fill the new column with the transformed value local_storage.ChangeType(parent, *this, changed_idx, target_type, bound_columns, cast_expr); diff --git a/src/duckdb/src/storage/open_file_storage_extension.cpp b/src/duckdb/src/storage/open_file_storage_extension.cpp index 125f65b26..1daf5b5e6 100644 --- a/src/duckdb/src/storage/open_file_storage_extension.cpp +++ b/src/duckdb/src/storage/open_file_storage_extension.cpp @@ -42,9 +42,9 @@ class OpenFileDefaultGenerator : public DefaultGenerator { string file; }; -unique_ptr OpenFileStorageAttach(StorageExtensionInfo *storage_info, ClientContext &context, +unique_ptr OpenFileStorageAttach(optional_ptr storage_info, ClientContext &context, AttachedDatabase &db, const string &name, AttachInfo &info, - AccessMode access_mode) { + AttachOptions &attach_options) { auto file = info.path; // open an in-memory database info.path = ":memory:"; @@ -66,7 +66,7 @@ unique_ptr OpenFileStorageAttach(StorageExtensionInfo *storage_info, Cl return std::move(catalog); } -unique_ptr OpenFileStorageTransactionManager(StorageExtensionInfo *storage_info, +unique_ptr OpenFileStorageTransactionManager(optional_ptr storage_info, AttachedDatabase &db, Catalog &catalog) { return make_uniq(db); } diff --git a/src/duckdb/src/storage/partial_block_manager.cpp b/src/duckdb/src/storage/partial_block_manager.cpp index f0f330be1..27fe86cd3 100644 --- a/src/duckdb/src/storage/partial_block_manager.cpp +++ b/src/duckdb/src/storage/partial_block_manager.cpp @@ -1,4 +1,6 @@ #include "duckdb/storage/partial_block_manager.hpp" +#include "duckdb/storage/table/in_memory_checkpoint.hpp" +#include "duckdb/storage/table/column_checkpoint_state.hpp" namespace duckdb { @@ -15,6 +17,10 @@ void PartialBlock::AddUninitializedRegion(idx_t start, idx_t end) { uninitialized_regions.push_back({start, end}); } +void PartialBlock::AddSegmentToTail(ColumnData &data, ColumnSegment &segment, uint32_t offset_in_block) { + throw InternalException("PartialBlock::AddSegmentToTail not supported for this block type"); +} + void PartialBlock::FlushInternal(const idx_t free_space_left) { // ensure that we do not leak any data @@ -105,6 +111,14 @@ bool PartialBlockManager::GetPartialBlock(idx_t segment_size, unique_ptr PartialBlockManager::CreatePartialBlock(ColumnData &column_data, ColumnSegment &segment, + PartialBlockState state, BlockManager &block_manager) { + if (partial_block_type == PartialBlockType::IN_MEMORY_CHECKPOINT) { + return make_uniq(column_data, segment, state, block_manager); + } + return make_uniq(column_data, segment, state, block_manager); +} + void PartialBlockManager::RegisterPartialBlock(PartialBlockAllocation allocation) { auto &state = allocation.partial_block->state; D_ASSERT(partial_block_type != PartialBlockType::FULL_CHECKPOINT || state.block_id >= 0); @@ -120,6 +134,7 @@ void PartialBlockManager::RegisterPartialBlock(PartialBlockAllocation allocation // check if the block is STILL partially filled after adding the segment_size if (new_space_left >= block_manager.GetBlockSize() - max_partial_block_size) { // the block is still partially filled: add it to the partially_filled_blocks list + D_ASSERT(allocation.partial_block->state.offset > 0); partially_filled_blocks.insert(make_pair(new_space_left, std::move(allocation.partial_block))); } } diff --git a/src/duckdb/src/storage/serialization/serialize_logical_operator.cpp b/src/duckdb/src/storage/serialization/serialize_logical_operator.cpp index 06d526706..a33e07dfa 100644 --- a/src/duckdb/src/storage/serialization/serialize_logical_operator.cpp +++ b/src/duckdb/src/storage/serialization/serialize_logical_operator.cpp @@ -616,6 +616,7 @@ void LogicalMergeInto::Serialize(Serializer &serializer) const { serializer.WritePropertyWithDefault(203, "row_id_start", row_id_start); serializer.WriteProperty(204, "source_marker", source_marker); serializer.WritePropertyWithDefault>>>(205, "actions", actions); + serializer.WritePropertyWithDefault(206, "return_chunk", return_chunk); } unique_ptr LogicalMergeInto::Deserialize(Deserializer &deserializer) { @@ -626,6 +627,7 @@ unique_ptr LogicalMergeInto::Deserialize(Deserializer &deserial deserializer.ReadPropertyWithDefault(203, "row_id_start", result->row_id_start); deserializer.ReadProperty(204, "source_marker", result->source_marker); deserializer.ReadPropertyWithDefault>>>(205, "actions", result->actions); + deserializer.ReadPropertyWithDefault(206, "return_chunk", result->return_chunk); return std::move(result); } diff --git a/src/duckdb/src/storage/serialization/serialize_tableref.cpp b/src/duckdb/src/storage/serialization/serialize_tableref.cpp index 97e5c3fda..caad6d1d6 100644 --- a/src/duckdb/src/storage/serialization/serialize_tableref.cpp +++ b/src/duckdb/src/storage/serialization/serialize_tableref.cpp @@ -181,6 +181,8 @@ void ShowRef::Serialize(Serializer &serializer) const { serializer.WritePropertyWithDefault(200, "table_name", table_name); serializer.WritePropertyWithDefault>(201, "query", query); serializer.WriteProperty(202, "show_type", show_type); + serializer.WritePropertyWithDefault(203, "catalog_name", catalog_name); + serializer.WritePropertyWithDefault(204, "schema_name", schema_name); } unique_ptr ShowRef::Deserialize(Deserializer &deserializer) { @@ -188,6 +190,8 @@ unique_ptr ShowRef::Deserialize(Deserializer &deserializer) { deserializer.ReadPropertyWithDefault(200, "table_name", result->table_name); deserializer.ReadPropertyWithDefault>(201, "query", result->query); deserializer.ReadProperty(202, "show_type", result->show_type); + deserializer.ReadPropertyWithDefault(203, "catalog_name", result->catalog_name); + deserializer.ReadPropertyWithDefault(204, "schema_name", result->schema_name); return std::move(result); } diff --git a/src/duckdb/src/storage/storage_manager.cpp b/src/duckdb/src/storage/storage_manager.cpp index 62f25d7ff..b8bb3be39 100644 --- a/src/duckdb/src/storage/storage_manager.cpp +++ b/src/duckdb/src/storage/storage_manager.cpp @@ -12,14 +12,65 @@ #include "duckdb/storage/single_file_block_manager.hpp" #include "duckdb/storage/storage_extension.hpp" #include "duckdb/storage/table/column_data.hpp" +#include "duckdb/storage/table/in_memory_checkpoint.hpp" #include "mbedtls_wrapper.hpp" namespace duckdb { using SHA256State = duckdb_mbedtls::MbedTlsWrapper::SHA256State; -StorageManager::StorageManager(AttachedDatabase &db, string path_p, bool read_only) - : db(db), path(std::move(path_p)), read_only(read_only) { +void StorageOptions::Initialize(const unordered_map &options) { + string storage_version_user_provided = ""; + for (auto &entry : options) { + if (entry.first == "block_size") { + // Extract the block allocation size. This is NOT the actual memory available on a block (block_size), + // even though the corresponding option we expose to the user is called "block_size". + block_alloc_size = entry.second.GetValue(); + } else if (entry.first == "encryption_key") { + // check the type of the key + auto type = entry.second.type(); + if (type.id() != LogicalTypeId::VARCHAR) { + throw BinderException("\"%s\" is not a valid key. A key must be of type VARCHAR", + entry.second.ToString()); + } else if (entry.second.GetValue().empty()) { + throw BinderException("Not a valid key. A key cannot be empty"); + } + user_key = make_shared_ptr(StringValue::Get(entry.second.DefaultCastAs(LogicalType::BLOB))); + block_header_size = DEFAULT_ENCRYPTION_BLOCK_HEADER_SIZE; + encryption = true; + } else if (entry.first == "encryption_cipher") { + throw BinderException("\"%s\" is not a valid cipher. Only AES GCM is supported.", entry.second.ToString()); + } else if (entry.first == "row_group_size") { + row_group_size = entry.second.GetValue(); + } else if (entry.first == "storage_version") { + storage_version_user_provided = entry.second.ToString(); + storage_version = SerializationCompatibility::FromString(entry.second.ToString()).serialization_version; + } else if (entry.first == "compress") { + if (entry.second.DefaultCastAs(LogicalType::BOOLEAN).GetValue()) { + compress_in_memory = CompressInMemory::COMPRESS; + } else { + compress_in_memory = CompressInMemory::DO_NOT_COMPRESS; + } + } else { + throw BinderException("Unrecognized option for attach \"%s\"", entry.first); + } + } + if (encryption && + (!storage_version.IsValid() || + storage_version.GetIndex() < SerializationCompatibility::FromString("v1.4.0").serialization_version)) { + if (!storage_version_user_provided.empty()) { + throw InvalidInputException( + "Explicit provided STORAGE_VERSION (\"%s\") and ENCRYPTION_KEY (storage >= v1.4.0) are not compatible", + storage_version_user_provided); + } + // set storage version to v1.4.0 + storage_version = SerializationCompatibility::FromString("v1.4.0").serialization_version; + } +} + +StorageManager::StorageManager(AttachedDatabase &db, string path_p, const AttachOptions &options) + : db(db), path(std::move(path_p)), read_only(options.access_mode == AccessMode::READ_ONLY), + in_memory_change_size(0) { if (path.empty()) { path = IN_MEMORY_PATH; @@ -27,6 +78,8 @@ StorageManager::StorageManager(AttachedDatabase &db, string path_p, bool read_on } auto &fs = FileSystem::Get(db); path = fs.ExpandPath(path); + + storage_options.Initialize(options.options); } StorageManager::~StorageManager() { @@ -48,7 +101,7 @@ ObjectCache &ObjectCache::GetObjectCache(ClientContext &context) { } idx_t StorageManager::GetWALSize() { - return wal->GetWALSize(); + return InMemory() ? in_memory_change_size.load() : wal->GetWALSize(); } optional_ptr StorageManager::GetWAL() { @@ -62,7 +115,7 @@ void StorageManager::ResetWAL() { wal->Delete(); } -string StorageManager::GetWALPath() { +string StorageManager::GetWALPath() const { // we append the ".wal" **before** a question mark in case of GET parameters // but only if we are not in a windows long path (which starts with \\?\) std::size_t question_mark_pos = std::string::npos; @@ -78,22 +131,22 @@ string StorageManager::GetWALPath() { return wal_path; } -bool StorageManager::InMemory() { +bool StorageManager::InMemory() const { D_ASSERT(!path.empty()); return path == IN_MEMORY_PATH; } -void StorageManager::Initialize(QueryContext context, StorageOptions &options) { +void StorageManager::Initialize(QueryContext context) { bool in_memory = InMemory(); if (in_memory && read_only) { throw CatalogException("Cannot launch in-memory database in read-only mode!"); } // Create or load the database from disk, if not in-memory mode. - LoadDatabase(context, options); + LoadDatabase(context); - if (options.encryption) { - ClearUserKey(options.user_key); + if (storage_options.encryption) { + ClearUserKey(storage_options.user_key); } } @@ -121,18 +174,23 @@ class SingleFileTableIOManager : public TableIOManager { } }; -SingleFileStorageManager::SingleFileStorageManager(AttachedDatabase &db, string path, bool read_only) - : StorageManager(db, std::move(path), read_only) { +SingleFileStorageManager::SingleFileStorageManager(AttachedDatabase &db, string path, const AttachOptions &options) + : StorageManager(db, std::move(path), options) { } -void SingleFileStorageManager::LoadDatabase(QueryContext context, StorageOptions &storage_options) { - +void SingleFileStorageManager::LoadDatabase(QueryContext context) { if (InMemory()) { block_manager = make_uniq(BufferManager::GetBufferManager(db), DEFAULT_BLOCK_ALLOC_SIZE, DEFAULT_BLOCK_HEADER_STORAGE_SIZE); table_io_manager = make_uniq(*block_manager, DEFAULT_ROW_GROUP_SIZE); + // in-memory databases can always use the latest storage version + storage_version = GetSerializationVersion("latest"); + load_complete = true; return; } + if (storage_options.compress_in_memory != CompressInMemory::AUTOMATIC) { + throw InvalidInputException("COMPRESS can only be set for in-memory databases"); + } auto &fs = FileSystem::Get(db); auto &config = DBConfig::Get(db); @@ -392,8 +450,16 @@ bool SingleFileStorageManager::IsCheckpointClean(MetaBlockPointer checkpoint_id) return block_manager->IsRootBlock(checkpoint_id); } +unique_ptr SingleFileStorageManager::CreateCheckpointWriter(QueryContext context, + CheckpointOptions options) { + if (InMemory()) { + return make_uniq(context, db, *block_manager, *this, options.type); + } + return make_uniq(context, db, *block_manager, options.type); +} + void SingleFileStorageManager::CreateCheckpoint(QueryContext context, CheckpointOptions options) { - if (InMemory() || read_only || !load_complete) { + if (read_only || !load_complete) { return; } if (db.GetStorageExtension()) { @@ -403,14 +469,14 @@ void SingleFileStorageManager::CreateCheckpoint(QueryContext context, Checkpoint if (GetWALSize() > 0 || config.options.force_checkpoint || options.action == CheckpointAction::ALWAYS_CHECKPOINT) { // we only need to checkpoint if there is anything in the WAL try { - SingleFileCheckpointWriter checkpointer(context, db, *block_manager, options.type); - checkpointer.CreateCheckpoint(); + auto checkpointer = CreateCheckpointWriter(context, options); + checkpointer->CreateCheckpoint(); } catch (std::exception &ex) { ErrorData error(ex); throw FatalException("Failed to create checkpoint because of error: %s", error.RawMessage()); } } - if (options.wal_action == CheckpointWALAction::DELETE_WAL) { + if (!InMemory() && options.wal_action == CheckpointWALAction::DELETE_WAL) { ResetWAL(); } diff --git a/src/duckdb/src/storage/table/column_checkpoint_state.cpp b/src/duckdb/src/storage/table/column_checkpoint_state.cpp index 1bd809e50..213338d97 100644 --- a/src/duckdb/src/storage/table/column_checkpoint_state.cpp +++ b/src/duckdb/src/storage/table/column_checkpoint_state.cpp @@ -25,7 +25,7 @@ unique_ptr ColumnCheckpointState::GetStatistics() { PartialBlockForCheckpoint::PartialBlockForCheckpoint(ColumnData &data, ColumnSegment &segment, PartialBlockState state, BlockManager &block_manager) : PartialBlock(state, block_manager, segment.block) { - AddSegmentToTail(data, segment, 0); + PartialBlockForCheckpoint::AddSegmentToTail(data, segment, 0); } PartialBlockForCheckpoint::~PartialBlockForCheckpoint() { @@ -126,28 +126,32 @@ void ColumnCheckpointState::FlushSegmentInternal(unique_ptr segme return; } // LCOV_EXCL_STOP - // merge the segment stats into the global stats + // Merge the segment statistics into the global statistics. global_stats->Merge(segment->stats.statistics); - // get the buffer of the segment and pin it - auto &db = column_data.GetDatabase(); - auto &buffer_manager = BufferManager::GetBufferManager(db); block_id_t block_id = INVALID_BLOCK; uint32_t offset_in_block = 0; unique_lock partial_block_lock; - if (!segment->stats.statistics.IsConstant()) { + if (segment->stats.statistics.IsConstant()) { + // Constant block. + segment->ConvertToPersistent(partial_block_manager.GetClientContext(), nullptr, INVALID_BLOCK); + + } else if (segment_size != 0) { + // Non-constant block with data that has to go to disk. + auto &db = column_data.GetDatabase(); + auto &buffer_manager = BufferManager::GetBufferManager(db); partial_block_lock = partial_block_manager.GetLock(); - // non-constant block - auto allocation = partial_block_manager.GetBlockAllocation(NumericCast(segment_size)); + auto cast_segment_size = NumericCast(segment_size); + auto allocation = partial_block_manager.GetBlockAllocation(cast_segment_size); block_id = allocation.state.block_id; offset_in_block = allocation.state.offset; if (allocation.partial_block) { // Use an existing block. D_ASSERT(offset_in_block > 0); - auto &pstate = allocation.partial_block->Cast(); + auto &pstate = *allocation.partial_block; // pin the source block auto old_handle = buffer_manager.Pin(segment->block); // pin the target block @@ -164,13 +168,17 @@ void ColumnCheckpointState::FlushSegmentInternal(unique_ptr segme segment->Resize(block_size); } D_ASSERT(offset_in_block == 0); - allocation.partial_block = make_uniq(column_data, *segment, allocation.state, - *allocation.block_manager); + allocation.partial_block = partial_block_manager.CreatePartialBlock(column_data, *segment, allocation.state, + *allocation.block_manager); } // Writer will decide whether to reuse this block. partial_block_manager.RegisterPartialBlock(std::move(allocation)); + } else { - segment->ConvertToPersistent(partial_block_manager.GetClientContext(), nullptr, INVALID_BLOCK); + // Empty segment, which does not have to go to disk. + // We still need to change its type to persistent, because we need to write its metadata. + segment->segment_type = ColumnSegmentType::PERSISTENT; + segment->block.reset(); } // construct the data pointer diff --git a/src/duckdb/src/storage/table/column_data.cpp b/src/duckdb/src/storage/table/column_data.cpp index 1366aadb9..19eda42a2 100644 --- a/src/duckdb/src/storage/table/column_data.cpp +++ b/src/duckdb/src/storage/table/column_data.cpp @@ -542,13 +542,18 @@ void ColumnData::FetchRow(TransactionData transaction, ColumnFetchState &state, FetchUpdateRow(transaction, row_id, result, result_idx); } +idx_t ColumnData::FetchUpdateData(row_t *row_ids, Vector &base_vector) { + ColumnScanState state; + auto fetch_count = Fetch(state, row_ids[0], base_vector); + base_vector.Flatten(fetch_count); + return fetch_count; +} + void ColumnData::Update(TransactionData transaction, idx_t column_index, Vector &update_vector, row_t *row_ids, idx_t update_count) { Vector base_vector(type); - ColumnScanState state; - auto fetch_count = Fetch(state, row_ids[0], base_vector); + FetchUpdateData(row_ids, base_vector); - base_vector.Flatten(fetch_count); UpdateInternal(transaction, column_index, update_vector, row_ids, update_count, base_vector); } diff --git a/src/duckdb/src/storage/table/column_data_checkpointer.cpp b/src/duckdb/src/storage/table/column_data_checkpointer.cpp index eb20b13d8..85278cd18 100644 --- a/src/duckdb/src/storage/table/column_data_checkpointer.cpp +++ b/src/duckdb/src/storage/table/column_data_checkpointer.cpp @@ -38,6 +38,10 @@ ColumnCheckpointState &ColumnDataCheckpointData::GetCheckpointState() { return *checkpoint_state; } +StorageManager &ColumnDataCheckpointData::GetStorageManager() { + return *storage_manager; +} + //! ColumnDataCheckpointer static Vector CreateIntermediateVector(vector> &states) { @@ -328,8 +332,8 @@ void ColumnDataCheckpointer::WriteToDisk() { auto &checkpoint_state = checkpoint_states[i]; auto &col_data = checkpoint_state.get().column_data; - checkpoint_data[i] = - ColumnDataCheckpointData(checkpoint_state, col_data, col_data.GetDatabase(), row_group, checkpoint_info); + checkpoint_data[i] = ColumnDataCheckpointData(checkpoint_state, col_data, col_data.GetDatabase(), row_group, + checkpoint_info, storage_manager); compression_states[i] = function->init_compression(checkpoint_data[i], std::move(analyze_state)); } diff --git a/src/duckdb/src/storage/table/column_segment.cpp b/src/duckdb/src/storage/table/column_segment.cpp index cb1227854..347463fbe 100644 --- a/src/duckdb/src/storage/table/column_segment.cpp +++ b/src/duckdb/src/storage/table/column_segment.cpp @@ -35,10 +35,8 @@ unique_ptr ColumnSegment::CreatePersistentSegment(DatabaseInstanc optional_ptr function; shared_ptr block; - if (block_id == INVALID_BLOCK) { - function = config.GetCompressionFunction(CompressionType::COMPRESSION_CONSTANT, type.InternalType()); - } else { - function = config.GetCompressionFunction(compression_type, type.InternalType()); + function = config.GetCompressionFunction(compression_type, type.InternalType()); + if (block_id != INVALID_BLOCK) { block = block_manager.RegisterBlock(block_id); } @@ -224,36 +222,38 @@ void ColumnSegment::RevertAppend(idx_t start_row) { // Convert To Persistent //===--------------------------------------------------------------------===// void ColumnSegment::ConvertToPersistent(QueryContext context, optional_ptr block_manager, - block_id_t block_id_p) { + const block_id_t block_id_p) { D_ASSERT(segment_type == ColumnSegmentType::TRANSIENT); segment_type = ColumnSegmentType::PERSISTENT; - block_id = block_id_p; offset = 0; - if (block_id == INVALID_BLOCK) { - // Constant block: no need to write anything to disk besides the stats. - // Set the compression function to constant. - D_ASSERT(stats.statistics.IsConstant()); - auto &config = DBConfig::GetConfig(db); - function = *config.GetCompressionFunction(CompressionType::COMPRESSION_CONSTANT, type.InternalType()); - // Reset the block buffer. - block.reset(); + if (block_id != INVALID_BLOCK) { + D_ASSERT(!stats.statistics.IsConstant()); + // Non-constant block: write the block to disk. + // The block data already exists in memory, so we alter the metadata, + // which ensures that the buffer points to an on-disk block. + block = block_manager->ConvertToPersistent(context, block_id, std::move(block)); return; } - // Non-constant block: write the block to disk. - // The data for the block already exists in-memory of our block. - // Instead of copying the data, we alter the metadata so that the buffer points to an on-disk block. - D_ASSERT(!stats.statistics.IsConstant()); - block = block_manager->ConvertToPersistent(context, block_id, std::move(block)); + // Constant block: no need to write anything to disk besides the stats (metadata). + // I.e., we do not need to write an actual block. + // Thus, we set the compression function to constant and reset the block buffer. + D_ASSERT(stats.statistics.IsConstant()); + auto &config = DBConfig::GetConfig(db); + function = *config.GetCompressionFunction(CompressionType::COMPRESSION_CONSTANT, type.InternalType()); + block.reset(); } void ColumnSegment::MarkAsPersistent(shared_ptr block_p, uint32_t offset_p) { D_ASSERT(segment_type == ColumnSegmentType::TRANSIENT); - segment_type = ColumnSegmentType::PERSISTENT; - block_id = block_p->BlockId(); + SetBlock(std::move(block_p), offset_p); +} + +void ColumnSegment::SetBlock(shared_ptr block_p, uint32_t offset_p) { + segment_type = ColumnSegmentType::PERSISTENT; offset = offset_p; block = std::move(block_p); } diff --git a/src/duckdb/src/storage/table/in_memory_checkpoint.cpp b/src/duckdb/src/storage/table/in_memory_checkpoint.cpp new file mode 100644 index 000000000..ea30b5fa0 --- /dev/null +++ b/src/duckdb/src/storage/table/in_memory_checkpoint.cpp @@ -0,0 +1,129 @@ +#include "duckdb/storage/table/in_memory_checkpoint.hpp" +#include "duckdb/common/serializer/binary_serializer.hpp" +#include "duckdb/catalog/catalog_entry/duck_table_entry.hpp" +#include "duckdb/catalog/duck_catalog.hpp" + +namespace duckdb { + +//===--------------------------------------------------------------------===// +// In-Memory Checkpoint Writer +//===--------------------------------------------------------------------===// +InMemoryCheckpointer::InMemoryCheckpointer(QueryContext context, AttachedDatabase &db, BlockManager &block_manager, + StorageManager &storage_manager, CheckpointType checkpoint_type) + : CheckpointWriter(db), context(context.GetClientContext()), + partial_block_manager(context, block_manager, PartialBlockType::IN_MEMORY_CHECKPOINT), + storage_manager(storage_manager), checkpoint_type(checkpoint_type) { +} + +void InMemoryCheckpointer::CreateCheckpoint() { + vector> schemas; + // we scan the set of committed schemas + auto &catalog = Catalog::GetCatalog(db).Cast(); + catalog.ScanSchemas([&](SchemaCatalogEntry &entry) { schemas.push_back(entry); }); + + vector> tables; + for (const auto &schema_ref : schemas) { + auto &schema = schema_ref.get(); + schema.Scan(CatalogType::TABLE_ENTRY, [&](CatalogEntry &entry) { + if (entry.type == CatalogType::TABLE_ENTRY) { + tables.push_back(entry.Cast()); + } + }); + } + + for (auto &table : tables) { + MemoryStream write_stream; + BinarySerializer serializer(write_stream); + + WriteTable(table, serializer); + } + storage_manager.ResetInMemoryChange(); +} + +MetadataWriter &InMemoryCheckpointer::GetMetadataWriter() { + throw InternalException("Unsupported method GetMetadataWriter for InMemoryCheckpointer"); +} +MetadataManager &InMemoryCheckpointer::GetMetadataManager() { + throw InternalException("Unsupported method GetMetadataManager for InMemoryCheckpointer"); +} +unique_ptr InMemoryCheckpointer::GetTableDataWriter(TableCatalogEntry &table) { + throw InternalException("Unsupported method GetTableDataWriter for InMemoryCheckpointer"); +} + +void InMemoryCheckpointer::WriteTable(TableCatalogEntry &table, Serializer &serializer) { + InMemoryTableDataWriter data_writer(*this, table); + + // Write the table data + auto table_lock = table.GetStorage().GetCheckpointLock(); + table.GetStorage().Checkpoint(data_writer, serializer); + // flush any partial blocks BEFORE releasing the table lock + // flushing partial blocks updates where data lives and is not thread-safe + partial_block_manager.FlushPartialBlocks(); +} + +InMemoryRowGroupWriter::InMemoryRowGroupWriter(TableCatalogEntry &table, PartialBlockManager &partial_block_manager, + InMemoryCheckpointer &checkpoint_manager) + : RowGroupWriter(table, partial_block_manager), checkpoint_manager(checkpoint_manager) { +} + +CheckpointType InMemoryRowGroupWriter::GetCheckpointType() const { + return checkpoint_manager.GetCheckpointType(); +} + +WriteStream &InMemoryRowGroupWriter::GetPayloadWriter() { + return metadata_writer; +} + +MetaBlockPointer InMemoryRowGroupWriter::GetMetaBlockPointer() { + return MetaBlockPointer(); +} + +optional_ptr InMemoryRowGroupWriter::GetMetadataManager() { + return nullptr; +} + +InMemoryTableDataWriter::InMemoryTableDataWriter(InMemoryCheckpointer &checkpoint_manager, TableCatalogEntry &table) + : TableDataWriter(table, checkpoint_manager.GetClientContext()), checkpoint_manager(checkpoint_manager) { +} + +void InMemoryTableDataWriter::FinalizeTable(const TableStatistics &global_stats, DataTableInfo *info, + Serializer &serializer) { + // nop: no need to write anything +} + +unique_ptr InMemoryTableDataWriter::GetRowGroupWriter(RowGroup &row_group) { + return make_uniq(table, checkpoint_manager.GetPartialBlockManager(), checkpoint_manager); +} + +CheckpointType InMemoryTableDataWriter::GetCheckpointType() const { + return checkpoint_manager.GetCheckpointType(); +} + +InMemoryPartialBlock::InMemoryPartialBlock(ColumnData &data, ColumnSegment &segment, PartialBlockState state, + BlockManager &block_manager) + : PartialBlock(state, block_manager, segment.block) { + AddSegmentToTail(data, segment, 0); +} + +InMemoryPartialBlock::~InMemoryPartialBlock() { +} + +void InMemoryPartialBlock::Flush(QueryContext context, const idx_t free_space_left) { + Clear(); +} + +void InMemoryPartialBlock::Merge(PartialBlock &other_p, idx_t offset, idx_t other_size) { + auto &other = other_p.Cast(); + other.Clear(); +} + +void InMemoryPartialBlock::AddSegmentToTail(ColumnData &data, ColumnSegment &segment, uint32_t offset_in_block) { + segment.SetBlock(block_handle, offset_in_block); +} + +void InMemoryPartialBlock::Clear() { + uninitialized_regions.clear(); + block_handle.reset(); +} + +} // namespace duckdb diff --git a/src/duckdb/src/storage/table/row_group.cpp b/src/duckdb/src/storage/table/row_group.cpp index eef8461fe..b1be94284 100644 --- a/src/duckdb/src/storage/table/row_group.cpp +++ b/src/duckdb/src/storage/table/row_group.cpp @@ -1,28 +1,25 @@ #include "duckdb/storage/table/row_group.hpp" -#include "duckdb/common/types/vector.hpp" + #include "duckdb/common/exception.hpp" -#include "duckdb/storage/table/column_data.hpp" -#include "duckdb/storage/table/column_checkpoint_state.hpp" -#include "duckdb/storage/table/update_segment.hpp" -#include "duckdb/storage/table_storage_info.hpp" -#include "duckdb/planner/table_filter.hpp" +#include "duckdb/common/serializer/binary_serializer.hpp" +#include "duckdb/common/serializer/deserializer.hpp" +#include "duckdb/common/serializer/serializer.hpp" +#include "duckdb/common/types/vector.hpp" +#include "duckdb/execution/adaptive_filter.hpp" #include "duckdb/execution/expression_executor.hpp" +#include "duckdb/main/database.hpp" +#include "duckdb/planner/table_filter.hpp" #include "duckdb/storage/checkpoint/table_data_writer.hpp" #include "duckdb/storage/metadata/metadata_reader.hpp" -#include "duckdb/transaction/duck_transaction_manager.hpp" -#include "duckdb/main/database.hpp" -#include "duckdb/main/attached_database.hpp" -#include "duckdb/transaction/duck_transaction.hpp" #include "duckdb/storage/table/append_state.hpp" -#include "duckdb/storage/table/scan_state.hpp" +#include "duckdb/storage/table/column_checkpoint_state.hpp" +#include "duckdb/storage/table/column_data.hpp" #include "duckdb/storage/table/row_version_manager.hpp" -#include "duckdb/common/serializer/serializer.hpp" -#include "duckdb/common/serializer/deserializer.hpp" -#include "duckdb/common/serializer/binary_serializer.hpp" -#include "duckdb/planner/filter/conjunction_filter.hpp" -#include "duckdb/planner/filter/struct_filter.hpp" -#include "duckdb/planner/filter/optional_filter.hpp" -#include "duckdb/execution/adaptive_filter.hpp" +#include "duckdb/storage/table/scan_state.hpp" +#include "duckdb/storage/table/update_segment.hpp" +#include "duckdb/storage/table_storage_info.hpp" +#include "duckdb/transaction/duck_transaction.hpp" +#include "duckdb/transaction/duck_transaction_manager.hpp" namespace duckdb { @@ -1044,7 +1041,7 @@ RowGroupPointer RowGroup::Checkpoint(RowGroupWriteData write_data, RowGroupWrite for (auto &state : write_data.states) { // get the current position of the table data writer auto &data_writer = writer.GetPayloadWriter(); - auto pointer = data_writer.GetMetaBlockPointer(); + auto pointer = writer.GetMetaBlockPointer(); // store the stats and the data pointers in the row group pointers row_group_pointer.data_pointers.push_back(pointer); @@ -1059,7 +1056,10 @@ RowGroupPointer RowGroup::Checkpoint(RowGroupWriteData write_data, RowGroupWrite persistent_data.Serialize(serializer); serializer.End(); } - row_group_pointer.deletes_pointers = CheckpointDeletes(writer.GetPayloadWriter().GetManager()); + auto metadata_manager = writer.GetMetadataManager(); + if (metadata_manager) { + row_group_pointer.deletes_pointers = CheckpointDeletes(*metadata_manager); + } Verify(); return row_group_pointer; } diff --git a/src/duckdb/src/storage/table/row_group_collection.cpp b/src/duckdb/src/storage/table/row_group_collection.cpp index 57287f3b7..20ffe8a9b 100644 --- a/src/duckdb/src/storage/table/row_group_collection.cpp +++ b/src/duckdb/src/storage/table/row_group_collection.cpp @@ -940,9 +940,10 @@ class VacuumTask : public BaseCheckpointTask { void RowGroupCollection::InitializeVacuumState(CollectionCheckpointState &checkpoint_state, VacuumState &state, vector> &segments) { - bool is_full_checkpoint = checkpoint_state.writer.GetCheckpointType() == CheckpointType::FULL_CHECKPOINT; + auto checkpoint_type = checkpoint_state.writer.GetCheckpointType(); + bool vacuum_is_allowed = checkpoint_type != CheckpointType::CONCURRENT_CHECKPOINT; // currently we can only vacuum deletes if we are doing a full checkpoint and there are no indexes - state.can_vacuum_deletes = info->GetIndexes().Empty() && is_full_checkpoint; + state.can_vacuum_deletes = info->GetIndexes().Empty() && vacuum_is_allowed; if (!state.can_vacuum_deletes) { return; } @@ -1065,8 +1066,10 @@ void RowGroupCollection::Checkpoint(TableDataWriter &writer, TableStatistics &gl } // schedule a checkpoint task for this row group entry.node->MoveToCollection(*this, vacuum_state.row_start); - auto checkpoint_task = GetCheckpointTask(checkpoint_state, segment_idx); - checkpoint_state.executor->ScheduleTask(std::move(checkpoint_task)); + if (writer.GetCheckpointType() != CheckpointType::VACUUM_ONLY) { + auto checkpoint_task = GetCheckpointTask(checkpoint_state, segment_idx); + checkpoint_state.executor->ScheduleTask(std::move(checkpoint_task)); + } vacuum_state.row_start += entry.node->count; } } catch (const std::exception &e) { @@ -1087,6 +1090,13 @@ void RowGroupCollection::Checkpoint(TableDataWriter &writer, TableStatistics &gl continue; } auto &row_group = *entry.node; + if (!checkpoint_state.writers[segment_idx]) { + // row group was not checkpointed - this can happen if compressing is disabled for in-memory tables + D_ASSERT(writer.GetCheckpointType() == CheckpointType::VACUUM_ONLY); + row_groups->AppendSegment(l, std::move(entry.node)); + new_total_rows += row_group.count; + continue; + } auto row_group_writer = std::move(checkpoint_state.writers[segment_idx]); if (!row_group_writer) { throw InternalException("Missing row group writer for index %llu", segment_idx); diff --git a/src/duckdb/src/storage/table/standard_column_data.cpp b/src/duckdb/src/storage/table/standard_column_data.cpp index 50062a45c..4854d7c6c 100644 --- a/src/duckdb/src/storage/table/standard_column_data.cpp +++ b/src/duckdb/src/storage/table/standard_column_data.cpp @@ -154,8 +154,15 @@ idx_t StandardColumnData::Fetch(ColumnScanState &state, row_t row_id, Vector &re void StandardColumnData::Update(TransactionData transaction, idx_t column_index, Vector &update_vector, row_t *row_ids, idx_t update_count) { - ColumnData::Update(transaction, column_index, update_vector, row_ids, update_count); - validity.Update(transaction, column_index, update_vector, row_ids, update_count); + Vector base_vector(type); + auto standard_fetch = FetchUpdateData(row_ids, base_vector); + auto validity_fetch = validity.FetchUpdateData(row_ids, base_vector); + if (standard_fetch != validity_fetch) { + throw InternalException("Unaligned fetch in validity and main column data for update"); + } + + UpdateInternal(transaction, column_index, update_vector, row_ids, update_count, base_vector); + validity.UpdateInternal(transaction, column_index, update_vector, row_ids, update_count, base_vector); } void StandardColumnData::UpdateColumn(TransactionData transaction, const vector &column_path, diff --git a/src/duckdb/src/transaction/duck_transaction.cpp b/src/duckdb/src/transaction/duck_transaction.cpp index cb1b46b32..f27a43c83 100644 --- a/src/duckdb/src/transaction/duck_transaction.cpp +++ b/src/duckdb/src/transaction/duck_transaction.cpp @@ -161,7 +161,9 @@ bool DuckTransaction::ChangesMade() { } UndoBufferProperties DuckTransaction::GetUndoProperties() { - return undo_buffer.GetProperties(); + auto properties = undo_buffer.GetProperties(); + properties.estimated_size += storage->EstimatedSize(); + return properties; } bool DuckTransaction::AutomaticCheckpoint(AttachedDatabase &db, const UndoBufferProperties &properties) { @@ -176,7 +178,7 @@ bool DuckTransaction::AutomaticCheckpoint(AttachedDatabase &db, const UndoBuffer return false; } auto &storage_manager = db.GetStorageManager(); - return storage_manager.AutomaticCheckpoint(storage->EstimatedSize() + properties.estimated_size); + return storage_manager.AutomaticCheckpoint(properties.estimated_size); } bool DuckTransaction::ShouldWriteToWAL(AttachedDatabase &db) { diff --git a/src/duckdb/src/transaction/duck_transaction_manager.cpp b/src/duckdb/src/transaction/duck_transaction_manager.cpp index c25bc903f..c9bbd088f 100644 --- a/src/duckdb/src/transaction/duck_transaction_manager.cpp +++ b/src/duckdb/src/transaction/duck_transaction_manager.cpp @@ -102,10 +102,10 @@ DuckTransactionManager::CanCheckpoint(DuckTransaction &transaction, unique_ptr // std::FILE #include #include @@ -231,19 +234,16 @@ using std_string_view = std::experimental::basic_string_view; template struct std_string_view {}; #endif +using int128_t = duckdb::hugeint_t; +using uint128_t = duckdb::uhugeint_t; + #ifdef FMT_USE_INT128 // Do nothing. #elif defined(__SIZEOF_INT128__) # define FMT_USE_INT128 1 -using int128_t = __int128_t; -using uint128_t = __uint128_t; #else # define FMT_USE_INT128 0 #endif -#if !FMT_USE_INT128 -struct int128_t {}; -struct uint128_t {}; -#endif // Casts a nonnegative integer to unsigned. template @@ -1004,16 +1004,10 @@ FMT_CONSTEXPR auto visit_format_arg(Visitor&& vis, return vis(arg.value_.long_long_value); case internal::ulong_long_type: return vis(arg.value_.ulong_long_value); -#if FMT_USE_INT128 case internal::int128_type: return vis(arg.value_.int128_value); case internal::uint128_type: return vis(arg.value_.uint128_value); -#else - case internal::int128_type: - case internal::uint128_type: - break; -#endif case internal::bool_type: return vis(arg.value_.bool_value); case internal::char_type: diff --git a/src/duckdb/third_party/fmt/include/fmt/format-inl.h b/src/duckdb/third_party/fmt/include/fmt/format-inl.h index 1755025ae..b92c914b5 100644 --- a/src/duckdb/third_party/fmt/include/fmt/format-inl.h +++ b/src/duckdb/third_party/fmt/include/fmt/format-inl.h @@ -629,7 +629,11 @@ class bigint { int num_bigits = static_cast(bigits_.size()); int num_result_bigits = 2 * num_bigits; bigits_.resize(num_result_bigits); - using accumulator_t = conditional_t; +#if FMT_USE_INT128 + using accumulator_t = __uint128_t; +#else + using accumulator_t = accumulator; +#endif auto sum = accumulator_t(); for (int bigit_index = 0; bigit_index < num_bigits; ++bigit_index) { // Compute bigit at position bigit_index of the result by adding diff --git a/src/duckdb/third_party/fmt/include/fmt/format.h b/src/duckdb/third_party/fmt/include/fmt/format.h index 67190586d..4c5163010 100644 --- a/src/duckdb/third_party/fmt/include/fmt/format.h +++ b/src/duckdb/third_party/fmt/include/fmt/format.h @@ -34,6 +34,7 @@ #define FMT_FORMAT_H_ #include "duckdb/common/exception.hpp" +#include "duckdb/common/limits.hpp" #include "duckdb/original/std/memory.hpp" #include "fmt/core.h" @@ -254,9 +255,21 @@ inline fallback_uintptr to_uintptr(const void* p) { template constexpr T max_value() { return (std::numeric_limits::max)(); } +template <> constexpr int128_t max_value() { + return duckdb::NumericLimits::Maximum(); +} +template <> constexpr uint128_t max_value() { + return duckdb::NumericLimits::Maximum(); +} template constexpr int num_bits() { return std::numeric_limits::digits; } +template <> constexpr int num_bits() { + return 127; +} +template <> constexpr int num_bits() { + return 128; +} template <> constexpr int num_bits() { return static_cast(sizeof(void*) * std::numeric_limits::digits); @@ -691,11 +704,11 @@ namespace internal { // Returns true if value is negative, false otherwise. // Same as `value < 0` but doesn't produce warnings if T is an unsigned type. -template ::is_signed)> +template ::is_signed || std::is_same::value)> FMT_CONSTEXPR bool is_negative(T value) { return value < 0; } -template ::is_signed)> +template ::is_signed && !std::is_same::value)> FMT_CONSTEXPR bool is_negative(T) { return false; } @@ -703,9 +716,10 @@ FMT_CONSTEXPR bool is_negative(T) { // Smallest of uint32_t, uint64_t, uint128_t that is large enough to // represent all values of T. template -using uint32_or_64_or_128_t = conditional_t< - std::numeric_limits::digits <= 32, uint32_t, - conditional_t::digits <= 64, uint64_t, uint128_t>>; +using uint32_or_64_or_128_t = + conditional_t::value || std::is_same::value, uint128_t, + conditional_t::digits <= 32, uint32_t, + conditional_t::digits <= 64, uint64_t, uint128_t>>>; // Static data is placed in this class template for the header-only config. template struct FMT_EXTERN_TEMPLATE_API basic_data { @@ -755,7 +769,6 @@ inline int count_digits(uint64_t n) { } #endif -#if FMT_USE_INT128 inline int count_digits(uint128_t n) { int count = 1; for (;;) { @@ -770,7 +783,6 @@ inline int count_digits(uint128_t n) { count += 4; } } -#endif // Counts the number of digits in n. BITS = log2(radix). template inline int count_digits(UInt n) { @@ -842,7 +854,7 @@ inline Char* format_decimal(Char* buffer, UInt value, int num_digits, add_thousands_sep(buffer); } if (value < 10) { - *--buffer = static_cast('0' + value); + *--buffer = static_cast('0' + static_cast(value)); return end; } auto index = static_cast(value * 2); @@ -881,7 +893,7 @@ inline Char* format_uint(Char* buffer, UInt value, int num_digits, Char* end = buffer; do { const char* digits = upper ? "0123456789ABCDEF" : data::hex_digits; - unsigned digit = (value & ((1 << BASE_BITS) - 1)); + unsigned digit = (static_cast(value) & ((1 << BASE_BITS) - 1)); *--buffer = static_cast(BASE_BITS < 4 ? static_cast('0' + digit) : digits[digit]); } while ((value >>= BASE_BITS) != 0); @@ -1455,7 +1467,7 @@ template class basic_writer { if (is_negative(value)) { prefix[0] = '-'; ++prefix_size; - abs_value = 0 - abs_value; + abs_value = -abs_value; } else if (specs.sign != sign::none && specs.sign != sign::minus) { prefix[0] = specs.sign == sign::plus ? '+' : ' '; ++prefix_size; @@ -1651,10 +1663,8 @@ template class basic_writer { void write(unsigned long value) { write_decimal(value); } void write(unsigned long long value) { write_decimal(value); } -#if FMT_USE_INT128 void write(int128_t value) { write_decimal(value); } void write(uint128_t value) { write_decimal(value); } -#endif template void write_int(T value, const Spec& spec) { @@ -1977,13 +1987,13 @@ template class width_checker { explicit FMT_CONSTEXPR width_checker(ErrorHandler& eh) : handler_(eh) {} template ::value)> - FMT_CONSTEXPR unsigned long long operator()(T value) { + FMT_CONSTEXPR uint64_t operator()(T value) { if (is_negative(value)) handler_.on_error("negative width"); - return static_cast(value); + return static_cast(value); } template ::value)> - FMT_CONSTEXPR unsigned long long operator()(T) { + FMT_CONSTEXPR uint64_t operator()(T) { handler_.on_error("width is not integer"); return 0; } @@ -1997,13 +2007,13 @@ template class precision_checker { explicit FMT_CONSTEXPR precision_checker(ErrorHandler& eh) : handler_(eh) {} template ::value)> - FMT_CONSTEXPR unsigned long long operator()(T value) { + FMT_CONSTEXPR uint64_t operator()(T value) { if (is_negative(value)) handler_.on_error("negative precision"); - return static_cast(value); + return static_cast(value); } template ::value)> - FMT_CONSTEXPR unsigned long long operator()(T) { + FMT_CONSTEXPR uint64_t operator()(T) { handler_.on_error("precision is not integer"); return 0; } @@ -2128,7 +2138,7 @@ template class specs_checker : public Handler { template