diff --git a/src/duckdb/src/common/enum_util.cpp b/src/duckdb/src/common/enum_util.cpp index 59efdcfdd..2336ddbaa 100644 --- a/src/duckdb/src/common/enum_util.cpp +++ b/src/duckdb/src/common/enum_util.cpp @@ -57,6 +57,7 @@ #include "duckdb/common/types/column/column_data_scan_states.hpp" #include "duckdb/common/types/column/partitioned_column_data.hpp" #include "duckdb/common/types/conflict_manager.hpp" +#include "duckdb/common/types/hyperloglog.hpp" #include "duckdb/common/types/row/partitioned_tuple_data.hpp" #include "duckdb/common/types/row/tuple_data_states.hpp" #include "duckdb/common/types/timestamp.hpp" @@ -64,8 +65,10 @@ #include "duckdb/common/types/vector_buffer.hpp" #include "duckdb/execution/index/art/art.hpp" #include "duckdb/execution/index/art/node.hpp" -#include "duckdb/execution/operator/persistent/base_csv_reader.hpp" -#include "duckdb/execution/operator/persistent/csv_reader_options.hpp" +#include "duckdb/execution/operator/scan/csv/base_csv_reader.hpp" +#include "duckdb/execution/operator/scan/csv/csv_reader_options.hpp" +#include "duckdb/execution/operator/scan/csv/csv_state_machine.hpp" +#include "duckdb/execution/operator/scan/csv/quote_rules.hpp" #include "duckdb/function/aggregate_state.hpp" #include "duckdb/function/function.hpp" #include "duckdb/function/macro_function.hpp" @@ -639,6 +642,64 @@ CAPIResultSetType EnumUtil::FromString(const char *value) { throw NotImplementedException(StringUtil::Format("Enum value: '%s' not implemented", value)); } +template<> +const char* EnumUtil::ToChars(CSVState value) { + switch(value) { + case CSVState::STANDARD: + return "STANDARD"; + case CSVState::DELIMITER: + return "DELIMITER"; + case CSVState::RECORD_SEPARATOR: + return "RECORD_SEPARATOR"; + case CSVState::CARRIAGE_RETURN: + return "CARRIAGE_RETURN"; + case CSVState::QUOTED: + return "QUOTED"; + case CSVState::UNQUOTED: + return "UNQUOTED"; + case CSVState::ESCAPE: + return "ESCAPE"; + case CSVState::EMPTY_LINE: + return "EMPTY_LINE"; + case CSVState::INVALID: + return "INVALID"; + default: + throw NotImplementedException(StringUtil::Format("Enum value: '%d' not implemented", value)); + } +} + +template<> +CSVState EnumUtil::FromString(const char *value) { + if (StringUtil::Equals(value, "STANDARD")) { + return CSVState::STANDARD; + } + if (StringUtil::Equals(value, "DELIMITER")) { + return CSVState::DELIMITER; + } + if (StringUtil::Equals(value, "RECORD_SEPARATOR")) { + return CSVState::RECORD_SEPARATOR; + } + if (StringUtil::Equals(value, "CARRIAGE_RETURN")) { + return CSVState::CARRIAGE_RETURN; + } + if (StringUtil::Equals(value, "QUOTED")) { + return CSVState::QUOTED; + } + if (StringUtil::Equals(value, "UNQUOTED")) { + return CSVState::UNQUOTED; + } + if (StringUtil::Equals(value, "ESCAPE")) { + return CSVState::ESCAPE; + } + if (StringUtil::Equals(value, "EMPTY_LINE")) { + return CSVState::EMPTY_LINE; + } + if (StringUtil::Equals(value, "INVALID")) { + return CSVState::INVALID; + } + throw NotImplementedException(StringUtil::Format("Enum value: '%s' not implemented", value)); +} + template<> const char* EnumUtil::ToChars(CTEMaterialize value) { switch(value) { @@ -2305,6 +2366,24 @@ FunctionSideEffects EnumUtil::FromString(const char *value) throw NotImplementedException(StringUtil::Format("Enum value: '%s' not implemented", value)); } +template<> +const char* EnumUtil::ToChars(HLLStorageType value) { + switch(value) { + case HLLStorageType::UNCOMPRESSED: + return "UNCOMPRESSED"; + default: + throw NotImplementedException(StringUtil::Format("Enum value: '%d' not implemented", value)); + } +} + +template<> +HLLStorageType EnumUtil::FromString(const char *value) { + if (StringUtil::Equals(value, "UNCOMPRESSED")) { + return HLLStorageType::UNCOMPRESSED; + } + throw NotImplementedException(StringUtil::Format("Enum value: '%s' not implemented", value)); +} + template<> const char* EnumUtil::ToChars(IndexConstraintType value) { switch(value) { @@ -3761,8 +3840,6 @@ const char* EnumUtil::ToChars(ParserMode value) { switch(value) { case ParserMode::PARSING: return "PARSING"; - case ParserMode::SNIFFING_DIALECT: - return "SNIFFING_DIALECT"; case ParserMode::SNIFFING_DATATYPES: return "SNIFFING_DATATYPES"; case ParserMode::PARSING_HEADER: @@ -3777,9 +3854,6 @@ ParserMode EnumUtil::FromString(const char *value) { if (StringUtil::Equals(value, "PARSING")) { return ParserMode::PARSING; } - if (StringUtil::Equals(value, "SNIFFING_DIALECT")) { - return ParserMode::SNIFFING_DIALECT; - } if (StringUtil::Equals(value, "SNIFFING_DATATYPES")) { return ParserMode::SNIFFING_DATATYPES; } @@ -3794,6 +3868,8 @@ const char* EnumUtil::ToChars(PartitionSortStage value) { switch(value) { case PartitionSortStage::INIT: return "INIT"; + case PartitionSortStage::SCAN: + return "SCAN"; case PartitionSortStage::PREPARE: return "PREPARE"; case PartitionSortStage::MERGE: @@ -3810,6 +3886,9 @@ PartitionSortStage EnumUtil::FromString(const char *value) { if (StringUtil::Equals(value, "INIT")) { return PartitionSortStage::INIT; } + if (StringUtil::Equals(value, "SCAN")) { + return PartitionSortStage::SCAN; + } if (StringUtil::Equals(value, "PREPARE")) { return PartitionSortStage::PREPARE; } @@ -4547,6 +4626,34 @@ QueryResultType EnumUtil::FromString(const char *value) { throw NotImplementedException(StringUtil::Format("Enum value: '%s' not implemented", value)); } +template<> +const char* EnumUtil::ToChars(QuoteRule value) { + switch(value) { + case QuoteRule::QUOTES_RFC: + return "QUOTES_RFC"; + case QuoteRule::QUOTES_OTHER: + return "QUOTES_OTHER"; + case QuoteRule::NO_QUOTES: + return "NO_QUOTES"; + default: + throw NotImplementedException(StringUtil::Format("Enum value: '%d' not implemented", value)); + } +} + +template<> +QuoteRule EnumUtil::FromString(const char *value) { + if (StringUtil::Equals(value, "QUOTES_RFC")) { + return QuoteRule::QUOTES_RFC; + } + if (StringUtil::Equals(value, "QUOTES_OTHER")) { + return QuoteRule::QUOTES_OTHER; + } + if (StringUtil::Equals(value, "NO_QUOTES")) { + return QuoteRule::NO_QUOTES; + } + throw NotImplementedException(StringUtil::Format("Enum value: '%s' not implemented", value)); +} + template<> const char* EnumUtil::ToChars(RelationType value) { switch(value) { @@ -6009,6 +6116,8 @@ const char* EnumUtil::ToChars(VerificationType value) { return "DESERIALIZED"; case VerificationType::DESERIALIZED_V2: return "DESERIALIZED_V2"; + case VerificationType::DESERIALIZED_V2_NO_DEFAULT: + return "DESERIALIZED_V2_NO_DEFAULT"; case VerificationType::PARSED: return "PARSED"; case VerificationType::UNOPTIMIZED: @@ -6040,6 +6149,9 @@ VerificationType EnumUtil::FromString(const char *value) { if (StringUtil::Equals(value, "DESERIALIZED_V2")) { return VerificationType::DESERIALIZED_V2; } + if (StringUtil::Equals(value, "DESERIALIZED_V2_NO_DEFAULT")) { + return VerificationType::DESERIALIZED_V2_NO_DEFAULT; + } if (StringUtil::Equals(value, "PARSED")) { return VerificationType::PARSED; } diff --git a/src/duckdb/src/common/extra_type_info.cpp b/src/duckdb/src/common/extra_type_info.cpp index 450bfc8c2..3fdd94f7c 100644 --- a/src/duckdb/src/common/extra_type_info.cpp +++ b/src/duckdb/src/common/extra_type_info.cpp @@ -326,9 +326,11 @@ struct EnumTypeInfoTemplated : public EnumTypeInfo { return make_shared(values_insert_order, size); } - static shared_ptr FormatDeserialize(FormatDeserializer &source, uint32_t size) { + static shared_ptr FormatDeserialize(FormatDeserializer &deserializer, uint32_t size) { Vector values_insert_order(LogicalType::VARCHAR, size); - values_insert_order.FormatDeserialize(source, size); + deserializer.ReadObject(201, "values_insert_order", [&](FormatDeserializer &source) { + values_insert_order.FormatDeserialize(source, size); + }); return make_shared(values_insert_order, size); } @@ -477,7 +479,9 @@ void EnumTypeInfo::Serialize(FieldWriter &writer) const { void EnumTypeInfo::FormatSerialize(FormatSerializer &serializer) const { ExtraTypeInfo::FormatSerialize(serializer); serializer.WriteProperty(200, "dict_size", dict_size); - ((Vector &)values_insert_order).FormatSerialize(serializer, dict_size); // NOLINT - FIXME + serializer.WriteObject(201, "values_insert_order", [&](FormatSerializer &serializer) { + ((Vector &)GetValuesInsertOrder()).FormatSerialize(serializer, dict_size); // NOLINT - FIXME + }); } } // namespace duckdb diff --git a/src/duckdb/src/common/radix_partitioning.cpp b/src/duckdb/src/common/radix_partitioning.cpp index f4371b428..8f42c2492 100644 --- a/src/duckdb/src/common/radix_partitioning.cpp +++ b/src/duckdb/src/common/radix_partitioning.cpp @@ -26,9 +26,11 @@ struct RadixPartitioningConstants { }; template -RETURN_TYPE RadixBitsSwitch(idx_t radix_bits, ARGS &&... args) { +RETURN_TYPE RadixBitsSwitch(idx_t radix_bits, ARGS &&...args) { D_ASSERT(radix_bits <= RadixPartitioning::MAX_RADIX_BITS); switch (radix_bits) { + case 0: + return OP::template Operation<0>(std::forward(args)...); case 1: return OP::template Operation<1>(std::forward(args)...); case 2: @@ -82,36 +84,6 @@ idx_t RadixPartitioning::Select(Vector &hashes, const SelectionVector *sel, idx_ return RadixBitsSwitch(radix_bits, hashes, sel, count, cutoff, true_sel, false_sel); } -struct HashsToBinsFunctor { - template - static void Operation(Vector &hashes, Vector &bins, idx_t count) { - using CONSTANTS = RadixPartitioningConstants; - UnaryExecutor::Execute(hashes, bins, count, - [&](hash_t hash) { return CONSTANTS::ApplyMask(hash); }); - } -}; - -//===--------------------------------------------------------------------===// -// Row Data Partitioning -//===--------------------------------------------------------------------===// -template -static void InitPartitions(BufferManager &buffer_manager, vector> &partition_collections, - RowDataBlock *partition_blocks[], vector &partition_handles, - data_ptr_t partition_ptrs[], idx_t block_capacity, idx_t row_width) { - using CONSTANTS = RadixPartitioningConstants; - - partition_collections.reserve(CONSTANTS::NUM_PARTITIONS); - partition_handles.reserve(CONSTANTS::NUM_PARTITIONS); - for (idx_t i = 0; i < CONSTANTS::NUM_PARTITIONS; i++) { - partition_collections.push_back(make_uniq(buffer_manager, block_capacity, row_width)); - partition_blocks[i] = &partition_collections[i]->CreateBlock(); - partition_handles.push_back(buffer_manager.Pin(partition_blocks[i]->block)); - if (partition_ptrs) { - partition_ptrs[i] = partition_handles[i].Ptr(); - } - } -} - struct ComputePartitionIndicesFunctor { template static void Operation(Vector &hashes, Vector &partition_indices, idx_t count) { @@ -129,6 +101,7 @@ RadixPartitionedColumnData::RadixPartitionedColumnData(ClientContext &context_p, idx_t radix_bits_p, idx_t hash_col_idx_p) : PartitionedColumnData(PartitionedColumnDataType::RADIX, context_p, std::move(types_p)), radix_bits(radix_bits_p), hash_col_idx(hash_col_idx_p) { + D_ASSERT(radix_bits <= RadixPartitioning::MAX_RADIX_BITS); D_ASSERT(hash_col_idx < types.size()); const auto num_partitions = RadixPartitioning::NumberOfPartitions(radix_bits); allocators->allocators.reserve(num_partitions); @@ -173,6 +146,7 @@ RadixPartitionedTupleData::RadixPartitionedTupleData(BufferManager &buffer_manag idx_t radix_bits_p, idx_t hash_col_idx_p) : PartitionedTupleData(PartitionedTupleDataType::RADIX, buffer_manager, layout_p.Copy()), radix_bits(radix_bits_p), hash_col_idx(hash_col_idx_p) { + D_ASSERT(radix_bits <= RadixPartitioning::MAX_RADIX_BITS); D_ASSERT(hash_col_idx < layout.GetTypes().size()); const auto num_partitions = RadixPartitioning::NumberOfPartitions(radix_bits); allocators->allocators.reserve(num_partitions); @@ -215,6 +189,9 @@ void RadixPartitionedTupleData::InitializeAppendStateInternal(PartitionedTupleDa column_ids.emplace_back(col_idx); } partitions[0]->InitializeAppend(state.chunk_state, std::move(column_ids)); + + // Initialize fixed-size map + state.fixed_partition_entries.resize(RadixPartitioning::NumberOfPartitions(radix_bits)); } void RadixPartitionedTupleData::ComputePartitionIndices(PartitionedTupleDataAppendState &state, DataChunk &input) { diff --git a/src/duckdb/src/common/row_operations/row_aggregate.cpp b/src/duckdb/src/common/row_operations/row_aggregate.cpp index ed0c68749..6c89d8870 100644 --- a/src/duckdb/src/common/row_operations/row_aggregate.cpp +++ b/src/duckdb/src/common/row_operations/row_aggregate.cpp @@ -76,6 +76,10 @@ void RowOperations::CombineStates(RowOperationsState &state, TupleDataLayout &la // Move to the first aggregate states VectorOperations::AddInPlace(sources, layout.GetAggrOffset(), count); VectorOperations::AddInPlace(targets, layout.GetAggrOffset(), count); + + // Keep track of the offset + idx_t offset = layout.GetAggrOffset(); + for (auto &aggr : layout.GetAggregates()) { D_ASSERT(aggr.function.combine); AggregateInputData aggr_input_data(aggr.GetFunctionData(), state.allocator); @@ -84,23 +88,34 @@ void RowOperations::CombineStates(RowOperationsState &state, TupleDataLayout &la // Move to the next aggregate states VectorOperations::AddInPlace(sources, aggr.payload_size, count); VectorOperations::AddInPlace(targets, aggr.payload_size, count); + + // Increment the offset + offset += aggr.payload_size; } + + // Now subtract the offset to get back to the original position + VectorOperations::AddInPlace(sources, -offset, count); + VectorOperations::AddInPlace(targets, -offset, count); } void RowOperations::FinalizeStates(RowOperationsState &state, TupleDataLayout &layout, Vector &addresses, DataChunk &result, idx_t aggr_idx) { + // Copy the addresses + Vector addresses_copy(LogicalType::POINTER); + VectorOperations::Copy(addresses, addresses_copy, result.size(), 0, 0); + // Move to the first aggregate state - VectorOperations::AddInPlace(addresses, layout.GetAggrOffset(), result.size()); + VectorOperations::AddInPlace(addresses_copy, layout.GetAggrOffset(), result.size()); auto &aggregates = layout.GetAggregates(); for (idx_t i = 0; i < aggregates.size(); i++) { auto &target = result.data[aggr_idx + i]; auto &aggr = aggregates[i]; AggregateInputData aggr_input_data(aggr.GetFunctionData(), state.allocator); - aggr.function.finalize(addresses, aggr_input_data, target, result.size(), 0); + aggr.function.finalize(addresses_copy, aggr_input_data, target, result.size(), 0); // Move to the next aggregate state - VectorOperations::AddInPlace(addresses, aggr.payload_size, result.size()); + VectorOperations::AddInPlace(addresses_copy, aggr.payload_size, result.size()); } } diff --git a/src/duckdb/src/common/serializer/binary_deserializer.cpp b/src/duckdb/src/common/serializer/binary_deserializer.cpp index 4a5db0787..86d9638bb 100644 --- a/src/duckdb/src/common/serializer/binary_deserializer.cpp +++ b/src/duckdb/src/common/serializer/binary_deserializer.cpp @@ -2,147 +2,132 @@ namespace duckdb { -void BinaryDeserializer::SetTag(const field_id_t field_id, const char *tag) { - current_field_id = field_id; - current_tag = tag; - stack.back().read_field_count++; - if (stack.back().read_field_count > stack.back().expected_field_count) { - throw SerializationException("Attempting to read a required field, but field is missing"); +//------------------------------------------------------------------------- +// Nested Type Hooks +//------------------------------------------------------------------------- +void BinaryDeserializer::OnPropertyBegin(const field_id_t field_id, const char *) { + auto field = NextField(); + if (field != field_id) { + throw InternalException("Failed to deserialize: field id mismatch, expected: %d, got: %d", field_id, field); } } -//===--------------------------------------------------------------------===// -// Nested Types Hooks -//===--------------------------------------------------------------------===// -void BinaryDeserializer::OnObjectBegin() { - auto expected_field_id = ReadPrimitive(); - auto expected_field_count = ReadPrimitive(); - auto expected_size = ReadPrimitive(); - D_ASSERT(expected_field_count > 0); - D_ASSERT(expected_size > 0); - D_ASSERT(expected_field_id == current_field_id); - stack.emplace_back(expected_field_count, expected_size, expected_field_id); +void BinaryDeserializer::OnPropertyEnd() { } -void BinaryDeserializer::OnObjectEnd() { - auto &frame = stack.back(); - if (frame.read_field_count < frame.expected_field_count) { - throw SerializationException("Not all fields were read. This file might have been written with a newer version " - "of DuckDB and is incompatible with this version of DuckDB."); +bool BinaryDeserializer::OnOptionalPropertyBegin(const field_id_t field_id, const char *s) { + auto next_field = PeekField(); + auto present = next_field == field_id; + if (present) { + ConsumeField(); } - stack.pop_back(); -} - -idx_t BinaryDeserializer::OnListBegin() { - return ReadPrimitive(); -} - -void BinaryDeserializer::OnListEnd() { -} - -// Deserialize maps as [ { key: ..., value: ... } ] -idx_t BinaryDeserializer::OnMapBegin() { - return ReadPrimitive(); -} - -void BinaryDeserializer::OnMapEntryBegin() { -} - -void BinaryDeserializer::OnMapKeyBegin() { -} - -void BinaryDeserializer::OnMapValueBegin() { + return present; } -void BinaryDeserializer::OnMapEntryEnd() { +void BinaryDeserializer::OnOptionalPropertyEnd(bool present) { } -void BinaryDeserializer::OnMapEnd() { +void BinaryDeserializer::OnObjectBegin() { + nesting_level++; } -void BinaryDeserializer::OnPairBegin() { +void BinaryDeserializer::OnObjectEnd() { + auto next_field = NextField(); + if (next_field != MESSAGE_TERMINATOR_FIELD_ID) { + throw InternalException("Failed to deserialize: expected end of object, but found field id: %d", next_field); + } + nesting_level--; } -void BinaryDeserializer::OnPairKeyBegin() { +idx_t BinaryDeserializer::OnListBegin() { + return VarIntDecode(); } -void BinaryDeserializer::OnPairValueBegin() { +void BinaryDeserializer::OnListEnd() { } -void BinaryDeserializer::OnPairEnd() { +bool BinaryDeserializer::OnNullableBegin() { + return ReadBool(); } -bool BinaryDeserializer::OnOptionalBegin() { - return ReadPrimitive(); +void BinaryDeserializer::OnNullableEnd() { } -//===--------------------------------------------------------------------===// +//------------------------------------------------------------------------- // Primitive Types -//===--------------------------------------------------------------------===// +//------------------------------------------------------------------------- bool BinaryDeserializer::ReadBool() { - return ReadPrimitive(); + return static_cast(ReadPrimitive()); +} + +char BinaryDeserializer::ReadChar() { + return ReadPrimitive(); } int8_t BinaryDeserializer::ReadSignedInt8() { - return ReadPrimitive(); + return VarIntDecode(); } uint8_t BinaryDeserializer::ReadUnsignedInt8() { - return ReadPrimitive(); + return VarIntDecode(); } int16_t BinaryDeserializer::ReadSignedInt16() { - return ReadPrimitive(); + return VarIntDecode(); } uint16_t BinaryDeserializer::ReadUnsignedInt16() { - return ReadPrimitive(); + return VarIntDecode(); } int32_t BinaryDeserializer::ReadSignedInt32() { - return ReadPrimitive(); + return VarIntDecode(); } uint32_t BinaryDeserializer::ReadUnsignedInt32() { - return ReadPrimitive(); + return VarIntDecode(); } int64_t BinaryDeserializer::ReadSignedInt64() { - return ReadPrimitive(); + return VarIntDecode(); } uint64_t BinaryDeserializer::ReadUnsignedInt64() { - return ReadPrimitive(); + return VarIntDecode(); } float BinaryDeserializer::ReadFloat() { - return ReadPrimitive(); + auto value = ReadPrimitive(); + return value; } double BinaryDeserializer::ReadDouble() { - return ReadPrimitive(); + auto value = ReadPrimitive(); + return value; } string BinaryDeserializer::ReadString() { - uint32_t size = ReadPrimitive(); - if (size == 0) { + auto len = VarIntDecode(); + if (len == 0) { return string(); } - auto buffer = make_unsafe_uniq_array(size); - ReadData(buffer.get(), size); - return string(const_char_ptr_cast(buffer.get()), size); -} - -interval_t BinaryDeserializer::ReadInterval() { - return ReadPrimitive(); + auto buffer = make_unsafe_uniq_array(len); + ReadData(buffer.get(), len); + return string(const_char_ptr_cast(buffer.get()), len); } hugeint_t BinaryDeserializer::ReadHugeInt() { - return ReadPrimitive(); + auto upper = VarIntDecode(); + auto lower = VarIntDecode(); + return hugeint_t(upper, lower); } -void BinaryDeserializer::ReadDataPtr(data_ptr_t &ptr, idx_t count) { - ReadData(ptr, count); +void BinaryDeserializer::ReadDataPtr(data_ptr_t &ptr_p, idx_t count) { + auto len = VarIntDecode(); + if (len != count) { + throw SerializationException("Tried to read blob of %d size, but only %d elements are available", count, len); + } + ReadData(ptr_p, count); } } // namespace duckdb diff --git a/src/duckdb/src/common/serializer/binary_serializer.cpp b/src/duckdb/src/common/serializer/binary_serializer.cpp index 069096768..1119c8372 100644 --- a/src/duckdb/src/common/serializer/binary_serializer.cpp +++ b/src/duckdb/src/common/serializer/binary_serializer.cpp @@ -1,121 +1,134 @@ #include "duckdb/common/serializer/binary_serializer.hpp" -namespace duckdb { - -void BinarySerializer::SetTag(const field_id_t field_id, const char *tag) { - current_field_id = field_id; - current_tag = tag; - // Increment the number of fields - stack.back().field_count++; -} - -//===--------------------------------------------------------------------===// -// Nested types -//===--------------------------------------------------------------------===// -void BinarySerializer::OnOptionalBegin(bool present) { - Write(present); -} - -void BinarySerializer::OnListBegin(idx_t count) { - Write(count); -} - -void BinarySerializer::OnListEnd(idx_t count) { -} +#ifdef DEBUG +#include "duckdb/common/string_util.hpp" +#endif -// Serialize maps as arrays of objects with "key" and "value" properties. -void BinarySerializer::OnMapBegin(idx_t count) { - Write(count); -} +namespace duckdb { -void BinarySerializer::OnMapEntryBegin() { -} +void BinarySerializer::OnPropertyBegin(const field_id_t field_id, const char *tag) { + // Just write the field id straight up + Write(field_id); +#ifdef DEBUG + // Check that the tag is unique + auto &state = debug_stack.back(); + auto &seen_field_ids = state.seen_field_ids; + auto &seen_field_tags = state.seen_field_tags; + auto &seen_fields = state.seen_fields; + + if (seen_field_ids.find(field_id) != seen_field_ids.end() || seen_field_tags.find(tag) != seen_field_tags.end()) { + string all_fields; + for (auto &field : seen_fields) { + all_fields += StringUtil::Format("\"%s\":%d ", field.first, field.second); + } + throw InternalException("Duplicate field id/tag in field: \"%s\":%d, other fields: %s", tag, field_id, + all_fields); + } -void BinarySerializer::OnMapKeyBegin() { + seen_field_ids.insert(field_id); + seen_field_tags.insert(tag); + seen_fields.emplace_back(tag, field_id); +#else + (void)tag; +#endif } -void BinarySerializer::OnMapValueBegin() { +void BinarySerializer::OnPropertyEnd() { + // Nothing to do here } -void BinarySerializer::OnMapEntryEnd() { +void BinarySerializer::OnOptionalPropertyBegin(const field_id_t field_id, const char *tag, bool present) { + // Dont write anything at all if the property is not present + if (present) { + OnPropertyBegin(field_id, tag); + } } -void BinarySerializer::OnMapEnd(idx_t count) { +void BinarySerializer::OnOptionalPropertyEnd(bool present) { + // Nothing to do here } +//------------------------------------------------------------------------- +// Nested Type Hooks +//------------------------------------------------------------------------- void BinarySerializer::OnObjectBegin() { - stack.push_back(State({0, 0, data.size()})); - // Store the field id - Write(current_field_id); - // Store the offset so we can patch the field count and size later - Write(0); // Placeholder for the field count - Write(0); // Placeholder for the size +#ifdef DEBUG + debug_stack.emplace_back(); +#endif } void BinarySerializer::OnObjectEnd() { - auto &frame = stack.back(); - // Patch the field count and size - auto ptr = &data[frame.offset]; - ptr += sizeof(field_id_t); // Skip the field id - Store(frame.field_count, ptr); - ptr += sizeof(uint32_t); // Skip the field count - Store(frame.size, ptr); - stack.pop_back(); +#ifdef DEBUG + debug_stack.pop_back(); +#endif + // Write object terminator + Write(MESSAGE_TERMINATOR_FIELD_ID); } -void BinarySerializer::OnPairBegin() { +void BinarySerializer::OnListBegin(idx_t count) { + VarIntEncode(count); } -void BinarySerializer::OnPairKeyBegin() { +void BinarySerializer::OnListEnd() { } -void BinarySerializer::OnPairValueBegin() { +void BinarySerializer::OnNullableBegin(bool present) { + WriteValue(present); } -void BinarySerializer::OnPairEnd() { +void BinarySerializer::OnNullableEnd() { } -//===--------------------------------------------------------------------===// -// Primitive types -//===--------------------------------------------------------------------===// +//------------------------------------------------------------------------- +// Primitive Types +//------------------------------------------------------------------------- void BinarySerializer::WriteNull() { // This should never be called, optional writes should be handled by OnOptionalBegin } +void BinarySerializer::WriteValue(bool value) { + Write(value); +} + void BinarySerializer::WriteValue(uint8_t value) { + VarIntEncode(value); +} + +void BinarySerializer::WriteValue(char value) { Write(value); } void BinarySerializer::WriteValue(int8_t value) { - Write(value); + VarIntEncode(value); } void BinarySerializer::WriteValue(uint16_t value) { - Write(value); + VarIntEncode(value); } void BinarySerializer::WriteValue(int16_t value) { - Write(value); + VarIntEncode(value); } void BinarySerializer::WriteValue(uint32_t value) { - Write(value); + VarIntEncode(value); } void BinarySerializer::WriteValue(int32_t value) { - Write(value); + VarIntEncode(value); } void BinarySerializer::WriteValue(uint64_t value) { - Write(value); + VarIntEncode(value); } void BinarySerializer::WriteValue(int64_t value) { - Write(value); + VarIntEncode(value); } void BinarySerializer::WriteValue(hugeint_t value) { - Write(value); + VarIntEncode(value.upper); + VarIntEncode(value.lower); } void BinarySerializer::WriteValue(float value) { @@ -126,39 +139,26 @@ void BinarySerializer::WriteValue(double value) { Write(value); } -void BinarySerializer::WriteValue(interval_t value) { - Write(value); -} - void BinarySerializer::WriteValue(const string &value) { - auto len = value.length(); - Write((uint32_t)len); - if (len > 0) { - WriteDataInternal(value.c_str(), len); - } + uint32_t len = value.length(); + VarIntEncode(len); + WriteDataInternal(value.c_str(), len); } void BinarySerializer::WriteValue(const string_t value) { - auto len = value.GetSize(); - Write((uint32_t)len); - if (len > 0) { - WriteDataInternal(value.GetDataUnsafe(), len); - } + uint32_t len = value.GetSize(); + VarIntEncode(len); + WriteDataInternal(value.GetDataUnsafe(), len); } void BinarySerializer::WriteValue(const char *value) { - auto len = strlen(value); - Write((uint32_t)len); - if (len > 0) { - WriteDataInternal(value, len); - } -} - -void BinarySerializer::WriteValue(bool value) { - Write(value); + uint32_t len = strlen(value); + VarIntEncode(len); + WriteDataInternal(value, len); } void BinarySerializer::WriteDataPtr(const_data_ptr_t ptr, idx_t count) { + VarIntEncode(static_cast(count)); WriteDataInternal(ptr, count); } diff --git a/src/duckdb/src/common/serializer/format_serializer.cpp b/src/duckdb/src/common/serializer/format_serializer.cpp index 76415a81e..e4fb0a5eb 100644 --- a/src/duckdb/src/common/serializer/format_serializer.cpp +++ b/src/duckdb/src/common/serializer/format_serializer.cpp @@ -9,7 +9,7 @@ void FormatSerializer::WriteValue(const vector &vec) { for (auto item : vec) { WriteValue(item); } - OnListEnd(count); + OnListEnd(); } } // namespace duckdb diff --git a/src/duckdb/src/common/sort/partition_state.cpp b/src/duckdb/src/common/sort/partition_state.cpp index 873303a15..c84b81ed0 100644 --- a/src/duckdb/src/common/sort/partition_state.cpp +++ b/src/duckdb/src/common/sort/partition_state.cpp @@ -11,7 +11,7 @@ namespace duckdb { PartitionGlobalHashGroup::PartitionGlobalHashGroup(BufferManager &buffer_manager, const Orders &partitions, const Orders &orders, const Types &payload_types, bool external) - : count(0) { + : count(0), batch_base(0) { RowLayout payload_layout; payload_layout.Initialize(payload_types); @@ -191,52 +191,45 @@ void PartitionGlobalSinkState::CombineLocalPartition(GroupingPartition &local_pa grouping_data->Combine(*local_partition); } -void PartitionGlobalSinkState::BuildSortState(TupleDataCollection &group_data, GlobalSortState &global_sort) const { +PartitionLocalMergeState::PartitionLocalMergeState(PartitionGlobalSinkState &gstate) + : merge_state(nullptr), stage(PartitionSortStage::INIT), finished(true), executor(gstate.context) { + // Set up the sort expression computation. vector sort_types; - ExpressionExecutor executor(context); - for (auto &order : orders) { + for (auto &order : gstate.orders) { auto &oexpr = order.expression; sort_types.emplace_back(oexpr->return_type); executor.AddExpression(*oexpr); } - DataChunk sort_chunk; - sort_chunk.Initialize(allocator, sort_types); + sort_chunk.Initialize(gstate.allocator, sort_types); + payload_chunk.Initialize(gstate.allocator, gstate.payload_types); +} +void PartitionLocalMergeState::Scan() { + auto &group_data = *merge_state->group_data; + auto &hash_group = *merge_state->hash_group; + auto &chunk_state = merge_state->chunk_state; // Copy the data from the group into the sort code. + auto &global_sort = *hash_group.global_sort; LocalSortState local_sort; local_sort.Initialize(global_sort, global_sort.buffer_manager); - // Strip hash column - DataChunk payload_chunk; - payload_chunk.Initialize(allocator, payload_types); - - vector column_ids; - column_ids.reserve(payload_types.size()); - for (column_t i = 0; i < payload_types.size(); ++i) { - column_ids.emplace_back(i); - } - TupleDataScanState chunk_state; - group_data.InitializeScan(chunk_state, column_ids); - while (group_data.Scan(chunk_state, payload_chunk)) { + TupleDataScanState local_scan; + group_data.InitializeScan(local_scan, merge_state->column_ids); + while (group_data.Scan(chunk_state, local_scan, payload_chunk)) { sort_chunk.Reset(); executor.Execute(payload_chunk, sort_chunk); local_sort.SinkChunk(sort_chunk, payload_chunk); - if (local_sort.SizeInBytes() > memory_per_thread) { + if (local_sort.SizeInBytes() > merge_state->memory_per_thread) { local_sort.Sort(global_sort, true); } + hash_group.count += payload_chunk.size(); } global_sort.AddLocalState(local_sort); } -void PartitionGlobalSinkState::BuildSortState(TupleDataCollection &group_data, PartitionGlobalHashGroup &hash_group) { - BuildSortState(group_data, *hash_group.global_sort); - - hash_group.count += group_data.Count(); -} - // Per-thread sink state PartitionLocalSinkState::PartitionLocalSinkState(ClientContext &context, PartitionGlobalSinkState &gstate_p) : gstate(gstate_p), allocator(Allocator::Get(context)), executor(context) { @@ -349,10 +342,11 @@ void PartitionLocalSinkState::Combine() { gstate.CombineLocalPartition(local_partition, local_append); } -PartitionGlobalMergeState::PartitionGlobalMergeState(PartitionGlobalSinkState &sink, GroupDataPtr group_data, +PartitionGlobalMergeState::PartitionGlobalMergeState(PartitionGlobalSinkState &sink, GroupDataPtr group_data_p, hash_t hash_bin) - : sink(sink), group_data(std::move(group_data)), stage(PartitionSortStage::INIT), total_tasks(0), tasks_assigned(0), - tasks_completed(0) { + : sink(sink), group_data(std::move(group_data_p)), memory_per_thread(sink.memory_per_thread), + num_threads(TaskScheduler::GetScheduler(sink.context).NumberOfThreads()), stage(PartitionSortStage::INIT), + total_tasks(0), tasks_assigned(0), tasks_completed(0) { const auto group_idx = sink.hash_groups.size(); auto new_group = make_uniq(sink.buffer_manager, sink.partitions, sink.orders, @@ -363,13 +357,18 @@ PartitionGlobalMergeState::PartitionGlobalMergeState(PartitionGlobalSinkState &s global_sort = sink.hash_groups[group_idx]->global_sort.get(); sink.bin_groups[hash_bin] = group_idx; + + column_ids.reserve(sink.payload_types.size()); + for (column_t i = 0; i < sink.payload_types.size(); ++i) { + column_ids.emplace_back(i); + } + group_data->InitializeScan(chunk_state, column_ids); } void PartitionLocalMergeState::Prepare() { - auto &global_sort = *merge_state->global_sort; - merge_state->sink.BuildSortState(*merge_state->group_data, *merge_state->hash_group); merge_state->group_data.reset(); + auto &global_sort = *merge_state->global_sort; global_sort.PrepareMergePhase(); } @@ -381,6 +380,9 @@ void PartitionLocalMergeState::Merge() { void PartitionLocalMergeState::ExecuteTask() { switch (stage) { + case PartitionSortStage::SCAN: + Scan(); + break; case PartitionSortStage::PREPARE: Prepare(); break; @@ -427,6 +429,11 @@ bool PartitionGlobalMergeState::TryPrepareNextStage() { switch (stage) { case PartitionSortStage::INIT: + total_tasks = num_threads; + stage = PartitionSortStage::SCAN; + return true; + + case PartitionSortStage::SCAN: total_tasks = 1; stage = PartitionSortStage::PREPARE; return true; @@ -474,8 +481,9 @@ PartitionGlobalMergeStates::PartitionGlobalMergeStates(PartitionGlobalSinkState class PartitionMergeTask : public ExecutorTask { public: - PartitionMergeTask(shared_ptr event_p, ClientContext &context_p, PartitionGlobalMergeStates &hash_groups_p) - : ExecutorTask(context_p), event(std::move(event_p)), hash_groups(hash_groups_p) { + PartitionMergeTask(shared_ptr event_p, ClientContext &context_p, PartitionGlobalMergeStates &hash_groups_p, + PartitionGlobalSinkState &gstate) + : ExecutorTask(context_p), event(std::move(event_p)), local_state(gstate), hash_groups(hash_groups_p) { } TaskExecutionResult ExecuteTask(TaskExecutionMode mode) override; @@ -576,7 +584,7 @@ void PartitionMergeEvent::Schedule() { vector> merge_tasks; for (idx_t tnum = 0; tnum < num_threads; tnum++) { - merge_tasks.emplace_back(make_uniq(shared_from_this(), context, merge_states)); + merge_tasks.emplace_back(make_uniq(shared_from_this(), context, merge_states, gstate)); } SetTasks(std::move(merge_tasks)); } diff --git a/src/duckdb/src/common/types/data_chunk.cpp b/src/duckdb/src/common/types/data_chunk.cpp index d9430d805..d2e5034dd 100644 --- a/src/duckdb/src/common/types/data_chunk.cpp +++ b/src/duckdb/src/common/types/data_chunk.cpp @@ -1,21 +1,15 @@ #include "duckdb/common/types/data_chunk.hpp" #include "duckdb/common/array.hpp" -#include "duckdb/common/arrow/arrow.hpp" #include "duckdb/common/exception.hpp" #include "duckdb/common/helper.hpp" #include "duckdb/common/printer.hpp" #include "duckdb/common/serializer.hpp" -#include "duckdb/common/to_string.hpp" -#include "duckdb/common/types/arrow_aux_data.hpp" -#include "duckdb/common/types/date.hpp" +#include "duckdb/common/serializer/format_serializer.hpp" +#include "duckdb/common/serializer/format_deserializer.hpp" #include "duckdb/common/types/interval.hpp" -#include "duckdb/common/types/null_value.hpp" #include "duckdb/common/types/sel_cache.hpp" -#include "duckdb/common/types/timestamp.hpp" -#include "duckdb/common/types/uuid.hpp" #include "duckdb/common/types/vector_cache.hpp" -#include "duckdb/common/unordered_map.hpp" #include "duckdb/common/vector.hpp" #include "duckdb/common/vector_operations/vector_operations.hpp" #include "duckdb/execution/execution_context.hpp" @@ -268,6 +262,48 @@ void DataChunk::Deserialize(Deserializer &source) { Verify(); } +void DataChunk::FormatSerialize(FormatSerializer &serializer) const { + // write the count + auto row_count = size(); + serializer.WriteProperty(100, "rows", row_count); + auto column_count = ColumnCount(); + + // Write the types + serializer.WriteList(101, "types", column_count, + [&](FormatSerializer::List &list, idx_t i) { list.WriteElement(data[i].GetType()); }); + + // Write the data + serializer.WriteList(102, "columns", column_count, [&](FormatSerializer::List &list, idx_t i) { + list.WriteObject([&](FormatSerializer &object) { + // Reference the vector to avoid potentially mutating it during serialization + Vector serialized_vector(data[i].GetType()); + serialized_vector.Reference(data[i]); + serialized_vector.FormatSerialize(object, row_count); + }); + }); +} + +void DataChunk::FormatDeserialize(FormatDeserializer &deserializer) { + // read the count + auto row_count = deserializer.ReadProperty(100, "rows"); + + // Read the types + vector types; + deserializer.ReadList(101, "types", [&](FormatDeserializer::List &list, idx_t i) { + auto type = list.ReadElement(); + types.push_back(type); + }); + Initialize(Allocator::DefaultAllocator(), types); + + // now load the column data + SetCardinality(row_count); + + // Read the data + deserializer.ReadList(102, "columns", [&](FormatDeserializer::List &list, idx_t i) { + list.ReadObject([&](FormatDeserializer &object) { data[i].FormatDeserialize(object, row_count); }); + }); +} + void DataChunk::Slice(const SelectionVector &sel_vector, idx_t count_p) { this->count = count_p; SelCache merge_cache; diff --git a/src/duckdb/src/common/types/hyperloglog.cpp b/src/duckdb/src/common/types/hyperloglog.cpp index c0455298e..2586e46ec 100644 --- a/src/duckdb/src/common/types/hyperloglog.cpp +++ b/src/duckdb/src/common/types/hyperloglog.cpp @@ -2,6 +2,9 @@ #include "duckdb/common/exception.hpp" #include "duckdb/common/field_writer.hpp" +#include "duckdb/common/serializer/format_serializer.hpp" +#include "duckdb/common/serializer/format_deserializer.hpp" + #include "hyperloglog.hpp" namespace duckdb { @@ -106,6 +109,24 @@ unique_ptr HyperLogLog::Deserialize(FieldReader &reader) { return result; } +void HyperLogLog::FormatSerialize(FormatSerializer &serializer) const { + serializer.WriteProperty(100, "type", HLLStorageType::UNCOMPRESSED); + serializer.WriteProperty(101, "data", GetPtr(), GetSize()); +} + +unique_ptr HyperLogLog::FormatDeserialize(FormatDeserializer &deserializer) { + auto result = make_uniq(); + auto storage_type = deserializer.ReadProperty(100, "type"); + switch (storage_type) { + case HLLStorageType::UNCOMPRESSED: + deserializer.ReadProperty(101, "data", result->GetPtr(), GetSize()); + break; + default: + throw SerializationException("Unknown HyperLogLog storage type!"); + } + return result; +} + //===--------------------------------------------------------------------===// // Vectorized HLL implementation //===--------------------------------------------------------------------===// diff --git a/src/duckdb/src/common/types/interval.cpp b/src/duckdb/src/common/types/interval.cpp index 14b26861c..e28f7258b 100644 --- a/src/duckdb/src/common/types/interval.cpp +++ b/src/duckdb/src/common/types/interval.cpp @@ -11,6 +11,9 @@ #include "duckdb/common/operator/subtract.hpp" #include "duckdb/common/string_util.hpp" +#include "duckdb/common/serializer/format_serializer.hpp" +#include "duckdb/common/serializer/format_deserializer.hpp" + namespace duckdb { bool Interval::FromString(const string &str, interval_t &result) { diff --git a/src/duckdb/src/common/types/row/partitioned_tuple_data.cpp b/src/duckdb/src/common/types/row/partitioned_tuple_data.cpp index 4b08c222c..03fded4f7 100644 --- a/src/duckdb/src/common/types/row/partitioned_tuple_data.cpp +++ b/src/duckdb/src/common/types/row/partitioned_tuple_data.cpp @@ -8,7 +8,7 @@ namespace duckdb { PartitionedTupleData::PartitionedTupleData(PartitionedTupleDataType type_p, BufferManager &buffer_manager_p, const TupleDataLayout &layout_p) - : type(type_p), buffer_manager(buffer_manager_p), layout(layout_p.Copy()), + : type(type_p), buffer_manager(buffer_manager_p), layout(layout_p.Copy()), count(0), data_size(0), allocators(make_shared()) { } @@ -19,6 +19,10 @@ PartitionedTupleData::PartitionedTupleData(const PartitionedTupleData &other) PartitionedTupleData::~PartitionedTupleData() { } +const TupleDataLayout &PartitionedTupleData::GetLayout() const { + return layout; +} + PartitionedTupleDataType PartitionedTupleData::GetType() const { return type; } @@ -26,6 +30,7 @@ PartitionedTupleDataType PartitionedTupleData::GetType() const { void PartitionedTupleData::InitializeAppendState(PartitionedTupleDataAppendState &state, TupleDataPinProperties properties) const { state.partition_sel.Initialize(); + state.reverse_partition_sel.Initialize(); vector column_ids; column_ids.reserve(layout.ColumnCount()); @@ -36,97 +41,178 @@ void PartitionedTupleData::InitializeAppendState(PartitionedTupleDataAppendState InitializeAppendStateInternal(state, properties); } -void PartitionedTupleData::Append(PartitionedTupleDataAppendState &state, DataChunk &input) { +void PartitionedTupleData::Append(PartitionedTupleDataAppendState &state, DataChunk &input, + const SelectionVector &append_sel, const idx_t append_count) { + TupleDataCollection::ToUnifiedFormat(state.chunk_state, input); + AppendUnified(state, input, append_sel, append_count); +} + +bool PartitionedTupleData::UseFixedSizeMap() const { + return MaxPartitionIndex() < PartitionedTupleDataAppendState::MAP_THRESHOLD; +} + +void PartitionedTupleData::AppendUnified(PartitionedTupleDataAppendState &state, DataChunk &input, + const SelectionVector &append_sel, const idx_t append_count) { + const idx_t actual_append_count = append_count == DConstants::INVALID_INDEX ? input.size() : append_count; + // Compute partition indices and store them in state.partition_indices ComputePartitionIndices(state, input); // Build the selection vector for the partitions - BuildPartitionSel(state, input.size()); + BuildPartitionSel(state, append_sel, actual_append_count); // Early out: check if everything belongs to a single partition - const auto &partition_entries = state.partition_entries; - if (partition_entries.size() == 1) { - const auto &partition_index = partition_entries.begin()->first; - auto &partition = *partitions[partition_index]; - auto &partition_pin_state = *state.partition_pin_states[partition_index]; - partition.Append(partition_pin_state, state.chunk_state, input); - return; + optional_idx partition_index; + if (UseFixedSizeMap()) { + if (state.fixed_partition_entries.size() == 1) { + partition_index = state.fixed_partition_entries.begin().GetKey(); + } + } else { + if (state.partition_entries.size() == 1) { + partition_index = state.partition_entries.begin()->first; + } } + if (partition_index.IsValid()) { + auto &partition = *partitions[partition_index.GetIndex()]; + auto &partition_pin_state = *state.partition_pin_states[partition_index.GetIndex()]; - TupleDataCollection::ToUnifiedFormat(state.chunk_state, input); + const auto size_before = partition.SizeInBytes(); + partition.AppendUnified(partition_pin_state, state.chunk_state, input, append_sel, actual_append_count); + data_size += partition.SizeInBytes() - size_before; + } else { + // Compute the heap sizes for the whole chunk + if (!layout.AllConstant()) { + TupleDataCollection::ComputeHeapSizes(state.chunk_state, input, state.partition_sel, actual_append_count); + } - // Compute the heap sizes for the whole chunk - if (!layout.AllConstant()) { - TupleDataCollection::ComputeHeapSizes(state.chunk_state, input, state.partition_sel, input.size()); - } + // Build the buffer space + BuildBufferSpace(state); - // Build the buffer space - BuildBufferSpace(state); + // Now scatter everything in one go + partitions[0]->Scatter(state.chunk_state, input, state.partition_sel, actual_append_count); + } - // Now scatter everything in one go - partitions[0]->Scatter(state.chunk_state, input, state.partition_sel, input.size()); + count += actual_append_count; + Verify(); } -void PartitionedTupleData::Append(PartitionedTupleDataAppendState &state, TupleDataChunkState &input, idx_t count) { +void PartitionedTupleData::Append(PartitionedTupleDataAppendState &state, TupleDataChunkState &input, + const idx_t append_count) { // Compute partition indices and store them in state.partition_indices - ComputePartitionIndices(input.row_locations, count, state.partition_indices); + ComputePartitionIndices(input.row_locations, append_count, state.partition_indices); // Build the selection vector for the partitions - BuildPartitionSel(state, count); + BuildPartitionSel(state, *FlatVector::IncrementalSelectionVector(), append_count); // Early out: check if everything belongs to a single partition - auto &partition_entries = state.partition_entries; - if (partition_entries.size() == 1) { - const auto &partition_index = partition_entries.begin()->first; - auto &partition = *partitions[partition_index]; - auto &partition_pin_state = *state.partition_pin_states[partition_index]; + optional_idx partition_index; + if (UseFixedSizeMap()) { + if (state.fixed_partition_entries.size() == 1) { + partition_index = state.fixed_partition_entries.begin().GetKey(); + } + } else { + if (state.partition_entries.size() == 1) { + partition_index = state.partition_entries.begin()->first; + } + } + + if (partition_index.IsValid()) { + auto &partition = *partitions[partition_index.GetIndex()]; + auto &partition_pin_state = *state.partition_pin_states[partition_index.GetIndex()]; state.chunk_state.heap_sizes.Reference(input.heap_sizes); - partition.Build(partition_pin_state, state.chunk_state, 0, count); - partition.CopyRows(state.chunk_state, input, *FlatVector::IncrementalSelectionVector(), count); - return; + + const auto size_before = partition.SizeInBytes(); + partition.Build(partition_pin_state, state.chunk_state, 0, append_count); + data_size += partition.SizeInBytes() - size_before; + + partition.CopyRows(state.chunk_state, input, *FlatVector::IncrementalSelectionVector(), append_count); + } else { + // Build the buffer space + state.chunk_state.heap_sizes.Slice(input.heap_sizes, state.partition_sel, append_count); + state.chunk_state.heap_sizes.Flatten(append_count); + BuildBufferSpace(state); + + // Copy the rows + partitions[0]->CopyRows(state.chunk_state, input, state.partition_sel, append_count); + } + + count += append_count; + Verify(); +} + +// LCOV_EXCL_START +template +struct UnorderedMapGetter { + static inline const typename MAP_TYPE::key_type &GetKey(typename MAP_TYPE::iterator &iterator) { + return iterator->first; } - // Build the buffer space - state.chunk_state.heap_sizes.Slice(input.heap_sizes, state.partition_sel, count); - state.chunk_state.heap_sizes.Flatten(count); - BuildBufferSpace(state); + static inline const typename MAP_TYPE::key_type &GetKey(const typename MAP_TYPE::const_iterator &iterator) { + return iterator->first; + } + + static inline typename MAP_TYPE::mapped_type &GetValue(typename MAP_TYPE::iterator &iterator) { + return iterator->second; + } + + static inline const typename MAP_TYPE::mapped_type &GetValue(const typename MAP_TYPE::const_iterator &iterator) { + return iterator->second; + } +}; + +template +struct FixedSizeMapGetter { + static inline const idx_t &GetKey(fixed_size_map_iterator_t &iterator) { + return iterator.GetKey(); + } - // Copy the rows - partitions[0]->CopyRows(state.chunk_state, input, state.partition_sel, count); + static inline const idx_t &GetKey(const const_fixed_size_map_iterator_t &iterator) { + return iterator.GetKey(); + } + + static inline T &GetValue(fixed_size_map_iterator_t &iterator) { + return iterator.GetValue(); + } + + static inline const T &GetValue(const const_fixed_size_map_iterator_t &iterator) { + return iterator.GetValue(); + } +}; +// LCOV_EXCL_STOP + +void PartitionedTupleData::BuildPartitionSel(PartitionedTupleDataAppendState &state, const SelectionVector &append_sel, + const idx_t append_count) { + if (UseFixedSizeMap()) { + BuildPartitionSel, FixedSizeMapGetter>( + state, state.fixed_partition_entries, append_sel, append_count); + } else { + BuildPartitionSel, UnorderedMapGetter>>( + state, state.partition_entries, append_sel, append_count); + } } -void PartitionedTupleData::BuildPartitionSel(PartitionedTupleDataAppendState &state, idx_t count) { +template +void PartitionedTupleData::BuildPartitionSel(PartitionedTupleDataAppendState &state, MAP_TYPE &partition_entries, + const SelectionVector &append_sel, const idx_t append_count) { const auto partition_indices = FlatVector::GetData(state.partition_indices); - auto &partition_entries = state.partition_entries; - auto &partition_entries_arr = state.partition_entries_arr; partition_entries.clear(); - const auto max_partition_index = MaxPartitionIndex(); - const auto use_arr = max_partition_index < PartitionedTupleDataAppendState::MAP_THRESHOLD; - switch (state.partition_indices.GetVectorType()) { case VectorType::FLAT_VECTOR: - if (use_arr) { - std::fill_n(partition_entries_arr, max_partition_index + 1, list_entry_t(0, 0)); - for (idx_t i = 0; i < count; i++) { - const auto &partition_index = partition_indices[i]; - partition_entries_arr[partition_index].length++; - } - } else { - for (idx_t i = 0; i < count; i++) { - const auto &partition_index = partition_indices[i]; - auto partition_entry = partition_entries.find(partition_index); - if (partition_entry == partition_entries.end()) { - partition_entries.emplace(partition_index, list_entry_t(0, 1)); - } else { - partition_entry->second.length++; - } + for (idx_t i = 0; i < append_count; i++) { + const auto index = append_sel.get_index(i); + const auto &partition_index = partition_indices[index]; + auto partition_entry = partition_entries.find(partition_index); + if (partition_entry == partition_entries.end()) { + partition_entries[partition_index] = list_entry_t(0, 1); + } else { + GETTER::GetValue(partition_entry).length++; } } break; case VectorType::CONSTANT_VECTOR: - partition_entries[partition_indices[0]] = list_entry_t(0, count); + partition_entries[partition_indices[0]] = list_entry_t(0, append_count); break; default: throw InternalException("Unexpected VectorType in PartitionedTupleData::Append"); @@ -134,64 +220,62 @@ void PartitionedTupleData::BuildPartitionSel(PartitionedTupleDataAppendState &st // Early out: check if everything belongs to a single partition if (partition_entries.size() == 1) { + // This needs to be initialized, even if we go the short path here + for (idx_t i = 0; i < append_count; i++) { + const auto index = append_sel.get_index(i); + state.reverse_partition_sel[index] = i; + } return; } // Compute offsets from the counts idx_t offset = 0; - if (use_arr) { - for (idx_t partition_index = 0; partition_index <= max_partition_index; partition_index++) { - auto &partition_entry = partition_entries_arr[partition_index]; - partition_entry.offset = offset; - offset += partition_entry.length; - } - } else { - for (auto &pc : partition_entries) { - auto &partition_entry = pc.second; - partition_entry.offset = offset; - offset += partition_entry.length; - } + for (auto it = partition_entries.begin(); it != partition_entries.end(); ++it) { + auto &partition_entry = GETTER::GetValue(it); + partition_entry.offset = offset; + offset += partition_entry.length; } // Now initialize a single selection vector that acts as a selection vector for every partition - auto &all_partitions_sel = state.partition_sel; - if (use_arr) { - for (idx_t i = 0; i < count; i++) { - const auto &partition_index = partition_indices[i]; - auto &partition_offset = partition_entries_arr[partition_index].offset; - all_partitions_sel[partition_offset++] = i; - } - // Now just add it to the map anyway so the rest of the functionality is shared - for (idx_t partition_index = 0; partition_index <= max_partition_index; partition_index++) { - const auto &partition_entry = partition_entries_arr[partition_index]; - if (partition_entry.length != 0) { - partition_entries.emplace(partition_index, partition_entry); - } - } - } else { - for (idx_t i = 0; i < count; i++) { - const auto &partition_index = partition_indices[i]; - auto &partition_offset = partition_entries[partition_index].offset; - all_partitions_sel[partition_offset++] = i; - } + auto &partition_sel = state.partition_sel; + auto &reverse_partition_sel = state.reverse_partition_sel; + for (idx_t i = 0; i < append_count; i++) { + const auto index = append_sel.get_index(i); + const auto &partition_index = partition_indices[index]; + auto &partition_offset = partition_entries[partition_index].offset; + reverse_partition_sel[index] = partition_offset; + partition_sel[partition_offset++] = index; } } void PartitionedTupleData::BuildBufferSpace(PartitionedTupleDataAppendState &state) { - for (auto &pc : state.partition_entries) { - const auto &partition_index = pc.first; + if (UseFixedSizeMap()) { + BuildBufferSpace, FixedSizeMapGetter>( + state, state.fixed_partition_entries); + } else { + BuildBufferSpace, UnorderedMapGetter>>( + state, state.partition_entries); + } +} + +template +void PartitionedTupleData::BuildBufferSpace(PartitionedTupleDataAppendState &state, const MAP_TYPE &partition_entries) { + for (auto it = partition_entries.begin(); it != partition_entries.end(); ++it) { + const auto &partition_index = GETTER::GetKey(it); // Partition, pin state for this partition index auto &partition = *partitions[partition_index]; auto &partition_pin_state = *state.partition_pin_states[partition_index]; // Length and offset for this partition - const auto &partition_entry = pc.second; + const auto &partition_entry = GETTER::GetValue(it); const auto &partition_length = partition_entry.length; const auto partition_offset = partition_entry.offset - partition_length; // Build out the buffer space for this partition + const auto size_before = partition.SizeInBytes(); partition.Build(partition_pin_state, state.chunk_state, partition_offset, partition_length); + data_size += partition.SizeInBytes() - size_before; } } @@ -210,7 +294,6 @@ void PartitionedTupleData::Combine(PartitionedTupleData &other) { // Now combine the state's partitions into this lock_guard guard(lock); - if (partitions.empty()) { // This is the first merge, we just copy them over partitions = std::move(other.partitions); @@ -221,40 +304,28 @@ void PartitionedTupleData::Combine(PartitionedTupleData &other) { partitions[i]->Combine(*other.partitions[i]); } } + this->count += other.count; + this->data_size += other.data_size; + Verify(); } -void PartitionedTupleData::Partition(TupleDataCollection &source, TupleDataPinProperties properties) { - if (source.Count() == 0) { - return; - } -#ifdef DEBUG - const auto count_before = source.Count(); -#endif - - PartitionedTupleDataAppendState append_state; - InitializeAppendState(append_state, properties); - - TupleDataChunkIterator iterator(source, TupleDataPinProperties::DESTROY_AFTER_DONE, true); - auto &chunk_state = iterator.GetChunkState(); - do { - Append(append_state, chunk_state, iterator.GetCurrentChunkCount()); - } while (iterator.Next()); - - FlushAppendState(append_state); - source.Reset(); - -#ifdef DEBUG - idx_t count_after = 0; - for (const auto &partition : partitions) { - count_after += partition->Count(); +void PartitionedTupleData::Reset() { + for (auto &partition : partitions) { + partition->Reset(); } - D_ASSERT(count_before == count_after); -#endif + this->count = 0; + this->data_size = 0; + Verify(); } void PartitionedTupleData::Repartition(PartitionedTupleData &new_partitioned_data) { D_ASSERT(layout.GetTypes() == new_partitioned_data.layout.GetTypes()); + if (partitions.size() == new_partitioned_data.partitions.size()) { + new_partitioned_data.Combine(*this); + return; + } + PartitionedTupleDataAppendState append_state; new_partitioned_data.InitializeAppendState(append_state); @@ -279,20 +350,42 @@ void PartitionedTupleData::Repartition(PartitionedTupleData &new_partitioned_dat } partitions[actual_partition_idx]->Reset(); } - new_partitioned_data.FlushAppendState(append_state); + + count = 0; + data_size = 0; + + Verify(); +} + +void PartitionedTupleData::Unpin() { + for (auto &partition : partitions) { + partition->Unpin(); + } } vector> &PartitionedTupleData::GetPartitions() { return partitions; } -idx_t PartitionedTupleData::Count() const { - idx_t total_count = 0; - for (auto &partition : partitions) { - total_count += partition->Count(); +unique_ptr PartitionedTupleData::GetUnpartitioned() { + auto data_collection = std::move(partitions[0]); + partitions[0] = make_uniq(buffer_manager, layout); + + for (idx_t i = 1; i < partitions.size(); i++) { + data_collection->Combine(*partitions[i]); } - return total_count; + count = 0; + data_size = 0; + + data_collection->Verify(); + Verify(); + + return data_collection; +} + +idx_t PartitionedTupleData::Count() const { + return count; } idx_t PartitionedTupleData::SizeInBytes() const { @@ -303,6 +396,39 @@ idx_t PartitionedTupleData::SizeInBytes() const { return total_size; } +idx_t PartitionedTupleData::PartitionCount() const { + return partitions.size(); +} + +void PartitionedTupleData::Verify() const { +#ifdef DEBUG + idx_t total_count = 0; + idx_t total_size = 0; + for (auto &partition : partitions) { + partition->Verify(); + total_count += partition->Count(); + total_size += partition->SizeInBytes(); + } + D_ASSERT(total_count == this->count); + D_ASSERT(total_size == this->data_size); +#endif +} + +// LCOV_EXCL_START +string PartitionedTupleData::ToString() { + string result = + StringUtil::Format("PartitionedTupleData - [%llu Partitions, %llu Rows]\n", partitions.size(), Count()); + for (idx_t partition_idx = 0; partition_idx < partitions.size(); partition_idx++) { + result += StringUtil::Format("Partition %llu: ", partition_idx) + partitions[partition_idx]->ToString(); + } + return result; +} + +void PartitionedTupleData::Print() { + Printer::Print(ToString()); +} +// LCOV_EXCL_STOP + void PartitionedTupleData::CreateAllocator() { allocators->allocators.emplace_back(make_shared(buffer_manager, layout)); } diff --git a/src/duckdb/src/common/types/row/row_layout.cpp b/src/duckdb/src/common/types/row/row_layout.cpp index 7e19b342a..3add8e425 100644 --- a/src/duckdb/src/common/types/row/row_layout.cpp +++ b/src/duckdb/src/common/types/row/row_layout.cpp @@ -12,11 +12,10 @@ namespace duckdb { -RowLayout::RowLayout() - : flag_width(0), data_width(0), aggr_width(0), row_width(0), all_constant(true), heap_pointer_offset(0) { +RowLayout::RowLayout() : flag_width(0), data_width(0), row_width(0), all_constant(true), heap_pointer_offset(0) { } -void RowLayout::Initialize(vector types_p, Aggregates aggregates_p, bool align) { +void RowLayout::Initialize(vector types_p, bool align) { offsets.clear(); types = std::move(types_p); @@ -31,7 +30,7 @@ void RowLayout::Initialize(vector types_p, Aggregates aggregates_p, // This enables pointer swizzling for out-of-core computation. if (!all_constant) { - // When unswizzled the pointer lives here. + // When unswizzled, the pointer lives here. // When swizzled, the pointer is replaced by an offset. heap_pointer_offset = row_width; // The 8 byte pointer will be replaced with an 8 byte idx_t when swizzled. @@ -52,39 +51,12 @@ void RowLayout::Initialize(vector types_p, Aggregates aggregates_p, } } - // Alignment padding for aggregates -#ifndef DUCKDB_ALLOW_UNDEFINED - if (align) { - row_width = AlignValue(row_width); - } -#endif data_width = row_width - flag_width; - // Aggregate fields. - aggregates = std::move(aggregates_p); - for (auto &aggregate : aggregates) { - offsets.push_back(row_width); - row_width += aggregate.payload_size; -#ifndef DUCKDB_ALLOW_UNDEFINED - D_ASSERT(aggregate.payload_size == AlignValue(aggregate.payload_size)); -#endif - } - aggr_width = row_width - data_width - flag_width; - // Alignment padding for the next row -#ifndef DUCKDB_ALLOW_UNDEFINED if (align) { row_width = AlignValue(row_width); } -#endif -} - -void RowLayout::Initialize(vector types_p, bool align) { - Initialize(std::move(types_p), Aggregates(), align); -} - -void RowLayout::Initialize(Aggregates aggregates_p, bool align) { - Initialize(vector(), std::move(aggregates_p), align); } } // namespace duckdb diff --git a/src/duckdb/src/common/types/row/tuple_data_allocator.cpp b/src/duckdb/src/common/types/row/tuple_data_allocator.cpp index 17ca9057c..e9569372c 100644 --- a/src/duckdb/src/common/types/row/tuple_data_allocator.cpp +++ b/src/duckdb/src/common/types/row/tuple_data_allocator.cpp @@ -33,6 +33,10 @@ TupleDataAllocator::TupleDataAllocator(TupleDataAllocator &allocator) : buffer_manager(allocator.buffer_manager), layout(allocator.layout.Copy()) { } +BufferManager &TupleDataAllocator::GetBufferManager() { + return buffer_manager; +} + Allocator &TupleDataAllocator::GetAllocator() { return buffer_manager.GetBufferAllocator(); } @@ -58,7 +62,7 @@ void TupleDataAllocator::Build(TupleDataSegment &segment, TupleDataPinState &pin } // Build the chunk parts for the incoming data - vector> chunk_part_indices; + chunk_part_indices.clear(); idx_t offset = 0; while (offset != append_count) { if (chunks.empty() || chunks.back().count == STANDARD_VECTOR_SIZE) { @@ -68,23 +72,26 @@ void TupleDataAllocator::Build(TupleDataSegment &segment, TupleDataPinState &pin // Build the next part auto next = MinValue(append_count - offset, STANDARD_VECTOR_SIZE - chunk.count); - chunk.AddPart(BuildChunkPart(pin_state, chunk_state, append_offset + offset, next), layout); - chunk_part_indices.emplace_back(chunks.size() - 1, chunk.parts.size() - 1); - + chunk.AddPart(BuildChunkPart(pin_state, chunk_state, append_offset + offset, next, chunk), layout); auto &chunk_part = chunk.parts.back(); next = chunk_part.count; + segment.count += next; + segment.data_size += chunk_part.count * layout.GetRowWidth(); + if (!layout.AllConstant()) { + segment.data_size += chunk_part.total_heap_size; + } offset += next; + chunk_part_indices.emplace_back(chunks.size() - 1, chunk.parts.size() - 1); } // Now initialize the pointers to write the data to - vector parts; - parts.reserve(chunk_part_indices.size()); + chunk_parts.clear(); for (auto &indices : chunk_part_indices) { - parts.emplace_back(&segment.chunks[indices.first].parts[indices.second]); + chunk_parts.emplace_back(segment.chunks[indices.first].parts[indices.second]); } - InitializeChunkStateInternal(pin_state, chunk_state, append_offset, false, true, false, parts); + InitializeChunkStateInternal(pin_state, chunk_state, append_offset, false, true, false, chunk_parts); // To reduce metadata, we try to merge chunk parts where possible // Due to the way chunk parts are constructed, only the last part of the first chunk is eligible for merging @@ -94,9 +101,10 @@ void TupleDataAllocator::Build(TupleDataSegment &segment, TupleDataPinState &pin } TupleDataChunkPart TupleDataAllocator::BuildChunkPart(TupleDataPinState &pin_state, TupleDataChunkState &chunk_state, - const idx_t append_offset, const idx_t append_count) { + const idx_t append_offset, const idx_t append_count, + TupleDataChunk &chunk) { D_ASSERT(append_count != 0); - TupleDataChunkPart result; + TupleDataChunkPart result(*chunk.lock); // Allocate row block (if needed) if (row_blocks.empty() || row_blocks.back().RemainingCapacity() < layout.GetRowWidth()) { @@ -176,10 +184,10 @@ void TupleDataAllocator::InitializeChunkState(TupleDataSegment &segment, TupleDa // when chunk 0 needs heap block 0, chunk 1 does not need any heap blocks, and chunk 2 needs heap block 0 again ReleaseOrStoreHandles(pin_state, segment, chunk, !chunk.heap_block_ids.empty()); - vector parts; + unsafe_vector> parts; parts.reserve(chunk.parts.size()); for (auto &part : chunk.parts) { - parts.emplace_back(&part); + parts.emplace_back(part); } InitializeChunkStateInternal(pin_state, chunk_state, 0, true, init_heap, init_heap, parts); @@ -206,17 +214,19 @@ static inline void InitializeHeapSizes(const data_ptr_t row_locations[], idx_t h void TupleDataAllocator::InitializeChunkStateInternal(TupleDataPinState &pin_state, TupleDataChunkState &chunk_state, idx_t offset, bool recompute, bool init_heap_pointers, - bool init_heap_sizes, vector &parts) { + bool init_heap_sizes, + unsafe_vector> &parts) { auto row_locations = FlatVector::GetData(chunk_state.row_locations); auto heap_sizes = FlatVector::GetData(chunk_state.heap_sizes); auto heap_locations = FlatVector::GetData(chunk_state.heap_locations); - for (auto &part : parts) { - const auto next = part->count; + for (auto &part_ref : parts) { + auto &part = part_ref.get(); + const auto next = part.count; // Set up row locations for the scan const auto row_width = layout.GetRowWidth(); - const auto base_row_ptr = GetRowPointer(pin_state, *part); + const auto base_row_ptr = GetRowPointer(pin_state, part); for (idx_t i = 0; i < next; i++) { row_locations[offset + i] = base_row_ptr + i * row_width; } @@ -226,9 +236,9 @@ void TupleDataAllocator::InitializeChunkStateInternal(TupleDataPinState &pin_sta continue; } - if (part->total_heap_size == 0) { + if (part.total_heap_size == 0) { if (init_heap_sizes) { // No heap, but we need the heap sizes - InitializeHeapSizes(row_locations, heap_sizes, offset, next, *part, layout.GetHeapSizeOffset()); + InitializeHeapSizes(row_locations, heap_sizes, offset, next, part, layout.GetHeapSizeOffset()); } offset += next; continue; @@ -236,29 +246,29 @@ void TupleDataAllocator::InitializeChunkStateInternal(TupleDataPinState &pin_sta // Check if heap block has changed - re-compute the pointers within each row if so if (recompute && pin_state.properties != TupleDataPinProperties::ALREADY_PINNED) { - const auto new_base_heap_ptr = GetBaseHeapPointer(pin_state, *part); - if (part->base_heap_ptr != new_base_heap_ptr) { - lock_guard guard(part->lock); - const auto old_base_heap_ptr = part->base_heap_ptr; + const auto new_base_heap_ptr = GetBaseHeapPointer(pin_state, part); + if (part.base_heap_ptr != new_base_heap_ptr) { + lock_guard guard(part.lock); + const auto old_base_heap_ptr = part.base_heap_ptr; if (old_base_heap_ptr != new_base_heap_ptr) { Vector old_heap_ptrs( - Value::POINTER(CastPointerToValue(old_base_heap_ptr + part->heap_block_offset))); + Value::POINTER(CastPointerToValue(old_base_heap_ptr + part.heap_block_offset))); Vector new_heap_ptrs( - Value::POINTER(CastPointerToValue(new_base_heap_ptr + part->heap_block_offset))); + Value::POINTER(CastPointerToValue(new_base_heap_ptr + part.heap_block_offset))); RecomputeHeapPointers(old_heap_ptrs, *ConstantVector::ZeroSelectionVector(), row_locations, new_heap_ptrs, offset, next, layout, 0); - part->base_heap_ptr = new_base_heap_ptr; + part.base_heap_ptr = new_base_heap_ptr; } } } if (init_heap_sizes) { - InitializeHeapSizes(row_locations, heap_sizes, offset, next, *part, layout.GetHeapSizeOffset()); + InitializeHeapSizes(row_locations, heap_sizes, offset, next, part, layout.GetHeapSizeOffset()); } if (init_heap_pointers) { // Set the pointers where the heap data will be written (if needed) - heap_locations[offset] = part->base_heap_ptr + part->heap_block_offset; + heap_locations[offset] = part.base_heap_ptr + part.heap_block_offset; for (idx_t i = 1; i < next; i++) { auto idx = offset + i; heap_locations[idx] = heap_locations[idx - 1] + heap_sizes[idx - 1]; @@ -387,11 +397,9 @@ void TupleDataAllocator::ReleaseOrStoreHandles(TupleDataPinState &pin_state, Tup ReleaseOrStoreHandles(pin_state, segment, DUMMY_CHUNK, true); } -void TupleDataAllocator::ReleaseOrStoreHandlesInternal(TupleDataSegment &segment, vector &pinned_handles, - unordered_map &handles, - const unordered_set &block_ids, - vector &blocks, - TupleDataPinProperties properties) { +void TupleDataAllocator::ReleaseOrStoreHandlesInternal( + TupleDataSegment &segment, unsafe_vector &pinned_handles, perfect_map_t &handles, + const perfect_set_t &block_ids, unsafe_vector &blocks, TupleDataPinProperties properties) { bool found_handle; do { found_handle = false; diff --git a/src/duckdb/src/common/types/row/tuple_data_collection.cpp b/src/duckdb/src/common/types/row/tuple_data_collection.cpp index 1e1cb607c..a556a24c1 100644 --- a/src/duckdb/src/common/types/row/tuple_data_collection.cpp +++ b/src/duckdb/src/common/types/row/tuple_data_collection.cpp @@ -27,6 +27,7 @@ TupleDataCollection::~TupleDataCollection() { void TupleDataCollection::Initialize() { D_ASSERT(!layout.GetTypes().empty()); this->count = 0; + this->data_size = 0; scatter_functions.reserve(layout.ColumnCount()); gather_functions.reserve(layout.ColumnCount()); for (idx_t col_idx = 0; col_idx < layout.ColumnCount(); col_idx++) { @@ -67,23 +68,13 @@ idx_t TupleDataCollection::SizeInBytes() const { return total_size; } -void TupleDataCollection::GetBlockPointers(vector &block_pointers) const { - D_ASSERT(segments.size() == 1); - const auto &segment = segments[0]; - const auto block_count = segment.allocator->RowBlockCount(); - D_ASSERT(segment.pinned_row_handles.size() == block_count); - block_pointers.resize(block_count); - for (idx_t block_idx = 0; block_idx < block_count; block_idx++) { - block_pointers[block_idx] = segment.pinned_row_handles[block_idx].Ptr(); - } -} - void TupleDataCollection::Unpin() { for (auto &segment : segments) { segment.Unpin(); } } +// LCOV_EXCL_START void VerifyAppendColumns(const TupleDataLayout &layout, const vector &column_ids) { #ifdef DEBUG for (idx_t col_idx = 0; col_idx < layout.ColumnCount(); col_idx++) { @@ -105,6 +96,7 @@ void VerifyAppendColumns(const TupleDataLayout &layout, const vector & } #endif } +// LCOV_EXCL_STOP void TupleDataCollection::InitializeAppend(TupleDataAppendState &append_state, TupleDataPinProperties properties) { vector column_ids; @@ -260,11 +252,15 @@ void TupleDataCollection::GetVectorData(const TupleDataChunkState &chunk_state, void TupleDataCollection::Build(TupleDataPinState &pin_state, TupleDataChunkState &chunk_state, const idx_t append_offset, const idx_t append_count) { - segments.back().allocator->Build(segments.back(), pin_state, chunk_state, append_offset, append_count); + auto &segment = segments.back(); + const auto size_before = segment.SizeInBytes(); + segment.allocator->Build(segment, pin_state, chunk_state, append_offset, append_count); + data_size += segment.SizeInBytes() - size_before; count += append_count; Verify(); } +// LCOV_EXCL_START void VerifyHeapSizes(const data_ptr_t source_locations[], const idx_t heap_sizes[], const SelectionVector &append_sel, const idx_t append_count, const idx_t heap_size_offset) { #ifdef DEBUG @@ -275,6 +271,7 @@ void VerifyHeapSizes(const data_ptr_t source_locations[], const idx_t heap_sizes } #endif } +// LCOV_EXCL_STOP void TupleDataCollection::CopyRows(TupleDataChunkState &chunk_state, TupleDataChunkState &input, const SelectionVector &append_sel, const idx_t append_count) const { @@ -324,12 +321,17 @@ void TupleDataCollection::Combine(TupleDataCollection &other) { if (this->layout.GetTypes() != other.GetLayout().GetTypes()) { throw InternalException("Attempting to combine TupleDataCollection with mismatching types"); } - this->count += other.count; this->segments.reserve(this->segments.size() + other.segments.size()); for (auto &other_seg : other.segments) { - this->segments.emplace_back(std::move(other_seg)); + AddSegment(std::move(other_seg)); } other.Reset(); +} + +void TupleDataCollection::AddSegment(TupleDataSegment &&segment) { + count += segment.count; + data_size += segment.data_size; + segments.emplace_back(std::move(segment)); Verify(); } @@ -339,6 +341,7 @@ void TupleDataCollection::Combine(unique_ptr other) { void TupleDataCollection::Reset() { count = 0; + data_size = 0; segments.clear(); // Refreshes the TupleDataAllocator to prevent holding on to allocated data unnecessarily @@ -395,6 +398,10 @@ bool TupleDataCollection::Scan(TupleDataScanState &state, DataChunk &result) { idx_t segment_index; idx_t chunk_index; if (!NextScanIndex(state, segment_index, chunk_index)) { + if (!segments.empty()) { + FinalizePinState(state.pin_state, segments[segment_index_before]); + } + result.SetCardinality(0); return false; } if (segment_index_before != DConstants::INVALID_INDEX && segment_index != segment_index_before) { @@ -408,20 +415,21 @@ bool TupleDataCollection::Scan(TupleDataParallelScanState &gstate, TupleDataLoca lstate.pin_state.properties = gstate.scan_state.pin_state.properties; const auto segment_index_before = lstate.segment_index; - idx_t segment_index; - idx_t chunk_index; { lock_guard guard(gstate.lock); - if (!NextScanIndex(gstate.scan_state, segment_index, chunk_index)) { + if (!NextScanIndex(gstate.scan_state, lstate.segment_index, lstate.chunk_index)) { + if (!segments.empty()) { + FinalizePinState(lstate.pin_state, segments[segment_index_before]); + } + result.SetCardinality(0); return false; } } - if (segment_index_before != DConstants::INVALID_INDEX && segment_index_before != segment_index) { + if (segment_index_before != DConstants::INVALID_INDEX && segment_index_before != lstate.segment_index) { FinalizePinState(lstate.pin_state, segments[lstate.segment_index]); - lstate.segment_index = segment_index; } - ScanAtIndex(lstate.pin_state, lstate.chunk_state, gstate.scan_state.chunk_state.column_ids, segment_index, - chunk_index, result); + ScanAtIndex(lstate.pin_state, lstate.chunk_state, gstate.scan_state.chunk_state.column_ids, lstate.segment_index, + lstate.chunk_index, result); return true; } @@ -430,8 +438,8 @@ void TupleDataCollection::FinalizePinState(TupleDataPinState &pin_state, TupleDa } void TupleDataCollection::FinalizePinState(TupleDataPinState &pin_state) { - D_ASSERT(segments.size() == 1); - allocator->ReleaseOrStoreHandles(pin_state, segments.back()); + D_ASSERT(!segments.empty()); + FinalizePinState(pin_state, segments.back()); } bool TupleDataCollection::NextScanIndex(TupleDataScanState &state, idx_t &segment_index, idx_t &chunk_index) { @@ -466,6 +474,7 @@ void TupleDataCollection::ScanAtIndex(TupleDataPinState &pin_state, TupleDataChu result.SetCardinality(chunk.count); } +// LCOV_EXCL_START string TupleDataCollection::ToString() { DataChunk chunk; InitializeChunk(chunk); @@ -493,12 +502,15 @@ void TupleDataCollection::Print() { void TupleDataCollection::Verify() const { #ifdef DEBUG - idx_t total_segment_count = 0; + idx_t total_count = 0; + idx_t total_size = 0; for (const auto &segment : segments) { segment.Verify(); - total_segment_count += segment.count; + total_count += segment.count; + total_size += segment.data_size; } - D_ASSERT(total_segment_count == this->count); + D_ASSERT(total_count == this->count); + D_ASSERT(total_size == this->data_size); #endif } @@ -509,5 +521,6 @@ void TupleDataCollection::VerifyEverythingPinned() const { } #endif } +// LCOV_EXCL_STOP } // namespace duckdb diff --git a/src/duckdb/src/common/types/row/tuple_data_layout.cpp b/src/duckdb/src/common/types/row/tuple_data_layout.cpp index bd9f67aed..3caa365b0 100644 --- a/src/duckdb/src/common/types/row/tuple_data_layout.cpp +++ b/src/duckdb/src/common/types/row/tuple_data_layout.cpp @@ -5,7 +5,8 @@ namespace duckdb { TupleDataLayout::TupleDataLayout() - : flag_width(0), data_width(0), aggr_width(0), row_width(0), all_constant(true), heap_size_offset(0) { + : flag_width(0), data_width(0), aggr_width(0), row_width(0), all_constant(true), heap_size_offset(0), + has_destructor(false) { } TupleDataLayout TupleDataLayout::Copy() const { @@ -25,6 +26,7 @@ TupleDataLayout TupleDataLayout::Copy() const { result.offsets = this->offsets; result.all_constant = this->all_constant; result.heap_size_offset = this->heap_size_offset; + result.has_destructor = this->has_destructor; return result; } @@ -106,6 +108,14 @@ void TupleDataLayout::Initialize(vector types_p, Aggregates aggrega row_width = AlignValue(row_width); } #endif + + has_destructor = false; + for (auto &aggr : GetAggregates()) { + if (aggr.function.destructor) { + has_destructor = true; + break; + } + } } void TupleDataLayout::Initialize(vector types_p, bool align, bool heap_offset_p) { diff --git a/src/duckdb/src/common/types/row/tuple_data_segment.cpp b/src/duckdb/src/common/types/row/tuple_data_segment.cpp index 6056f3041..d84da30fd 100644 --- a/src/duckdb/src/common/types/row/tuple_data_segment.cpp +++ b/src/duckdb/src/common/types/row/tuple_data_segment.cpp @@ -4,7 +4,7 @@ namespace duckdb { -TupleDataChunkPart::TupleDataChunkPart() { +TupleDataChunkPart::TupleDataChunkPart(mutex &lock_p) : lock(lock_p) { } void SwapTupleDataChunkPart(TupleDataChunkPart &a, TupleDataChunkPart &b) { @@ -15,10 +15,10 @@ void SwapTupleDataChunkPart(TupleDataChunkPart &a, TupleDataChunkPart &b) { std::swap(a.base_heap_ptr, b.base_heap_ptr); std::swap(a.total_heap_size, b.total_heap_size); std::swap(a.count, b.count); - // Cannot swap the lock, but not needed as move constructor only happens during append, lock only needed for scans + std::swap(a.lock, b.lock); } -TupleDataChunkPart::TupleDataChunkPart(TupleDataChunkPart &&other) noexcept { +TupleDataChunkPart::TupleDataChunkPart(TupleDataChunkPart &&other) noexcept : lock((other.lock)) { SwapTupleDataChunkPart(*this, other); } @@ -27,7 +27,8 @@ TupleDataChunkPart &TupleDataChunkPart::operator=(TupleDataChunkPart &&other) no return *this; } -TupleDataChunk::TupleDataChunk() : count(0) { +TupleDataChunk::TupleDataChunk() : count(0), lock(make_unsafe_uniq()) { + parts.reserve(2); } static inline void SwapTupleDataChunk(TupleDataChunk &a, TupleDataChunk &b) noexcept { @@ -35,6 +36,7 @@ static inline void SwapTupleDataChunk(TupleDataChunk &a, TupleDataChunk &b) noex std::swap(a.row_block_ids, b.row_block_ids); std::swap(a.heap_block_ids, b.heap_block_ids); std::swap(a.count, b.count); + std::swap(a.lock, b.lock); } TupleDataChunk::TupleDataChunk(TupleDataChunk &&other) noexcept { @@ -52,6 +54,7 @@ void TupleDataChunk::AddPart(TupleDataChunkPart &&part, const TupleDataLayout &l if (!layout.AllConstant() && part.total_heap_size > 0) { heap_block_ids.insert(part.heap_block_index); } + part.lock = *lock; parts.emplace_back(std::move(part)); } @@ -98,7 +101,7 @@ void TupleDataChunk::MergeLastChunkPart(const TupleDataLayout &layout) { } TupleDataSegment::TupleDataSegment(shared_ptr allocator_p) - : allocator(std::move(allocator_p)), count(0) { + : allocator(std::move(allocator_p)), count(0), data_size(0) { } TupleDataSegment::~TupleDataSegment() { @@ -112,6 +115,7 @@ void SwapTupleDataSegment(TupleDataSegment &a, TupleDataSegment &b) { std::swap(a.allocator, b.allocator); std::swap(a.chunks, b.chunks); std::swap(a.count, b.count); + std::swap(a.data_size, b.data_size); std::swap(a.pinned_row_handles, b.pinned_row_handles); std::swap(a.pinned_heap_handles, b.pinned_heap_handles); } @@ -130,17 +134,7 @@ idx_t TupleDataSegment::ChunkCount() const { } idx_t TupleDataSegment::SizeInBytes() const { - const auto &layout = allocator->GetLayout(); - idx_t total_size = 0; - for (const auto &chunk : chunks) { - total_size += chunk.count * layout.GetRowWidth(); - if (!layout.AllConstant()) { - for (const auto &part : chunk.parts) { - total_size += part.total_heap_size; - } - } - } - return total_size; + return data_size; } void TupleDataSegment::Unpin() { @@ -151,12 +145,23 @@ void TupleDataSegment::Unpin() { void TupleDataSegment::Verify() const { #ifdef DEBUG + const auto &layout = allocator->GetLayout(); + idx_t total_count = 0; + idx_t total_size = 0; for (const auto &chunk : chunks) { chunk.Verify(); total_count += chunk.count; + + total_size += chunk.count * layout.GetRowWidth(); + if (!layout.AllConstant()) { + for (const auto &part : chunk.parts) { + total_size += part.total_heap_size; + } + } } D_ASSERT(total_count == this->count); + D_ASSERT(total_size == this->data_size); #endif } diff --git a/src/duckdb/src/common/types/value.cpp b/src/duckdb/src/common/types/value.cpp index a264bbbce..2535886ac 100644 --- a/src/duckdb/src/common/types/value.cpp +++ b/src/duckdb/src/common/types/value.cpp @@ -1780,58 +1780,69 @@ void Value::FormatSerialize(FormatSerializer &serializer) const { serializer.WriteProperty(101, "is_null", is_null); if (!IsNull()) { switch (type_.InternalType()) { + case PhysicalType::BIT: + throw InternalException("BIT type should not be serialized"); case PhysicalType::BOOL: - serializer.WriteProperty(100, "value", value_.boolean); + serializer.WriteProperty(102, "value", value_.boolean); break; case PhysicalType::INT8: - serializer.WriteProperty(100, "value", value_.tinyint); + serializer.WriteProperty(102, "value", value_.tinyint); break; case PhysicalType::INT16: - serializer.WriteProperty(100, "value", value_.smallint); + serializer.WriteProperty(102, "value", value_.smallint); break; case PhysicalType::INT32: - serializer.WriteProperty(100, "value", value_.integer); + serializer.WriteProperty(102, "value", value_.integer); break; case PhysicalType::INT64: - serializer.WriteProperty(100, "value", value_.bigint); + serializer.WriteProperty(102, "value", value_.bigint); break; case PhysicalType::UINT8: - serializer.WriteProperty(100, "value", value_.utinyint); + serializer.WriteProperty(102, "value", value_.utinyint); break; case PhysicalType::UINT16: - serializer.WriteProperty(100, "value", value_.usmallint); + serializer.WriteProperty(102, "value", value_.usmallint); break; case PhysicalType::UINT32: - serializer.WriteProperty(100, "value", value_.uinteger); + serializer.WriteProperty(102, "value", value_.uinteger); break; case PhysicalType::UINT64: - serializer.WriteProperty(100, "value", value_.ubigint); + serializer.WriteProperty(102, "value", value_.ubigint); break; case PhysicalType::INT128: - serializer.WriteProperty(100, "value", value_.hugeint); + serializer.WriteProperty(102, "value", value_.hugeint); break; case PhysicalType::FLOAT: - serializer.WriteProperty(100, "value", value_.float_); + serializer.WriteProperty(102, "value", value_.float_); break; case PhysicalType::DOUBLE: - serializer.WriteProperty(100, "value", value_.double_); + serializer.WriteProperty(102, "value", value_.double_); break; case PhysicalType::INTERVAL: - serializer.WriteProperty(100, "value", value_.interval); + serializer.WriteProperty(102, "value", value_.interval); break; - case PhysicalType::VARCHAR: + case PhysicalType::VARCHAR: { if (type_.id() == LogicalTypeId::BLOB) { auto blob_str = Blob::ToString(StringValue::Get(*this)); - serializer.WriteProperty(100, "value", blob_str); + serializer.WriteProperty(102, "value", blob_str); } else { - serializer.WriteProperty(100, "value", StringValue::Get(*this)); + serializer.WriteProperty(102, "value", StringValue::Get(*this)); } - break; - default: { - Vector v(*this); - v.FormatSerialize(serializer, 1); - break; - } + } break; + case PhysicalType::LIST: { + serializer.WriteObject(102, "value", [&](FormatSerializer &serializer) { + auto &children = ListValue::GetChildren(*this); + serializer.WriteProperty(100, "children", children); + }); + } break; + case PhysicalType::STRUCT: { + serializer.WriteObject(102, "value", [&](FormatSerializer &serializer) { + auto &children = StructValue::GetChildren(*this); + serializer.WriteProperty(100, "children", children); + }); + } break; + default: + throw NotImplementedException("Unimplemented type for FormatSerialize"); } } } @@ -1845,59 +1856,69 @@ Value Value::FormatDeserialize(FormatDeserializer &deserializer) { } new_value.is_null = false; switch (type.InternalType()) { + case PhysicalType::BIT: + throw InternalException("BIT type should not be deserialized"); case PhysicalType::BOOL: - new_value.value_.boolean = deserializer.ReadProperty(100, "value"); + new_value.value_.boolean = deserializer.ReadProperty(102, "value"); break; case PhysicalType::UINT8: - new_value.value_.utinyint = deserializer.ReadProperty(100, "value"); + new_value.value_.utinyint = deserializer.ReadProperty(102, "value"); break; case PhysicalType::INT8: - new_value.value_.tinyint = deserializer.ReadProperty(100, "value"); + new_value.value_.tinyint = deserializer.ReadProperty(102, "value"); break; case PhysicalType::UINT16: - new_value.value_.usmallint = deserializer.ReadProperty(100, "value"); + new_value.value_.usmallint = deserializer.ReadProperty(102, "value"); break; case PhysicalType::INT16: - new_value.value_.smallint = deserializer.ReadProperty(100, "value"); + new_value.value_.smallint = deserializer.ReadProperty(102, "value"); break; case PhysicalType::UINT32: - new_value.value_.uinteger = deserializer.ReadProperty(100, "value"); + new_value.value_.uinteger = deserializer.ReadProperty(102, "value"); break; case PhysicalType::INT32: - new_value.value_.integer = deserializer.ReadProperty(100, "value"); + new_value.value_.integer = deserializer.ReadProperty(102, "value"); break; case PhysicalType::UINT64: - new_value.value_.ubigint = deserializer.ReadProperty(100, "value"); + new_value.value_.ubigint = deserializer.ReadProperty(102, "value"); break; case PhysicalType::INT64: - new_value.value_.bigint = deserializer.ReadProperty(100, "value"); + new_value.value_.bigint = deserializer.ReadProperty(102, "value"); break; case PhysicalType::INT128: - new_value.value_.hugeint = deserializer.ReadProperty(100, "value"); + new_value.value_.hugeint = deserializer.ReadProperty(102, "value"); break; case PhysicalType::FLOAT: - new_value.value_.float_ = deserializer.ReadProperty(100, "value"); + new_value.value_.float_ = deserializer.ReadProperty(102, "value"); break; case PhysicalType::DOUBLE: - new_value.value_.double_ = deserializer.ReadProperty(100, "value"); + new_value.value_.double_ = deserializer.ReadProperty(102, "value"); break; case PhysicalType::INTERVAL: - new_value.value_.interval = deserializer.ReadProperty(100, "value"); + new_value.value_.interval = deserializer.ReadProperty(102, "value"); break; case PhysicalType::VARCHAR: { - auto str = deserializer.ReadProperty(100, "value"); + auto str = deserializer.ReadProperty(102, "value"); if (type.id() == LogicalTypeId::BLOB) { new_value.value_info_ = make_shared(Blob::ToBlob(str)); } else { new_value.value_info_ = make_shared(str); } } break; - default: { - Vector v(type); - v.FormatDeserialize(deserializer, 1); - new_value = v.GetValue(0); - break; - } + case PhysicalType::LIST: { + deserializer.ReadObject(102, "value", [&](FormatDeserializer &obj) { + auto children = obj.ReadProperty>(100, "children"); + new_value.value_info_ = make_shared(children); + }); + } break; + case PhysicalType::STRUCT: { + deserializer.ReadObject(102, "value", [&](FormatDeserializer &obj) { + auto children = obj.ReadProperty>(100, "children"); + new_value.value_info_ = make_shared(children); + }); + } break; + default: + throw NotImplementedException("Unimplemented type for FormatDeserialize"); } return new_value; } diff --git a/src/duckdb/src/common/types/vector.cpp b/src/duckdb/src/common/types/vector.cpp index 2bca1bff5..6ffdb2394 100644 --- a/src/duckdb/src/common/types/vector.cpp +++ b/src/duckdb/src/common/types/vector.cpp @@ -99,7 +99,7 @@ void Vector::Reference(const Value &value) { } } -void Vector::Reference(Vector &other) { +void Vector::Reference(const Vector &other) { if (other.GetType().id() != GetType().id()) { throw InternalException("Vector::Reference used on vector of different type"); } @@ -107,12 +107,12 @@ void Vector::Reference(Vector &other) { Reinterpret(other); } -void Vector::ReferenceAndSetType(Vector &other) { +void Vector::ReferenceAndSetType(const Vector &other) { type = other.GetType(); Reference(other); } -void Vector::Reinterpret(Vector &other) { +void Vector::Reinterpret(const Vector &other) { vector_type = other.vector_type; AssignSharedPointer(buffer, other.buffer); AssignSharedPointer(auxiliary, other.auxiliary); @@ -1021,14 +1021,11 @@ void Vector::FormatSerialize(FormatSerializer &serializer, idx_t count) { auto strings = UnifiedVectorFormat::GetData(vdata); // Serialize data as a list - serializer.SetTag(102, "data"); - serializer.OnListBegin(count); - for (idx_t i = 0; i < count; i++) { + serializer.WriteList(102, "data", count, [&](FormatSerializer::List &list, idx_t i) { auto idx = vdata.sel->get_index(i); auto str = !vdata.validity.RowIsValid(idx) ? NullValue() : strings[idx]; - serializer.WriteValue(str); - } - serializer.OnListEnd(count); + list.WriteElement(str); + }); break; } case PhysicalType::STRUCT: { @@ -1036,14 +1033,9 @@ void Vector::FormatSerialize(FormatSerializer &serializer, idx_t count) { auto &entries = StructVector::GetEntries(*this); // Serialize entries as a list - serializer.SetTag(103, "children"); - serializer.OnListBegin(entries.size()); - for (auto &entry : entries) { - serializer.OnObjectBegin(); - entry->FormatSerialize(serializer, count); - serializer.OnObjectEnd(); - } - serializer.OnListEnd(entries.size()); + serializer.WriteList(103, "children", count, [&](FormatSerializer::List &list, idx_t i) { + list.WriteObject([&](FormatSerializer &object) { entries[i]->FormatSerialize(object, count); }); + }); break; } case PhysicalType::LIST: { @@ -1060,19 +1052,14 @@ void Vector::FormatSerialize(FormatSerializer &serializer, idx_t count) { entries[i].length = source.length; } serializer.WriteProperty(104, "list_size", list_size); - serializer.SetTag(105, "entries"); - serializer.OnListBegin(count); - for (idx_t i = 0; i < count; i++) { - serializer.OnObjectBegin(); - serializer.WriteProperty(100, "offset", entries[i].offset); - serializer.WriteProperty(101, "length", entries[i].length); - serializer.OnObjectEnd(); - } - serializer.OnListEnd(count); - serializer.SetTag(106, "child"); - serializer.OnObjectBegin(); - child.FormatSerialize(serializer, list_size); - serializer.OnObjectEnd(); + serializer.WriteList(105, "entries", count, [&](FormatSerializer::List &list, idx_t i) { + list.WriteObject([&](FormatSerializer &object) { + object.WriteProperty(100, "offset", entries[i].offset); + object.WriteProperty(101, "length", entries[i].length); + }); + }); + serializer.WriteObject(106, "child", + [&](FormatSerializer &object) { child.FormatSerialize(object, list_size); }); break; } default: @@ -1103,35 +1090,20 @@ void Vector::FormatDeserialize(FormatDeserializer &deserializer, idx_t count) { switch (logical_type.InternalType()) { case PhysicalType::VARCHAR: { auto strings = FlatVector::GetData(*this); - deserializer.SetTag(102, "data"); - auto read_count = deserializer.OnListBegin(); - D_ASSERT(read_count == count); - (void)read_count; // otherwise unused variable error in release mode - for (idx_t i = 0; i < count; i++) { - // read the strings - auto str = deserializer.ReadString(); - // now add the string to the StringHeap of the vector - // and write the pointer into the vector + deserializer.ReadList(102, "data", [&](FormatDeserializer::List &list, idx_t i) { + auto str = list.ReadElement(); if (validity.RowIsValid(i)) { strings[i] = StringVector::AddStringOrBlob(*this, str); } - } - deserializer.OnListEnd(); + }); break; } case PhysicalType::STRUCT: { auto &entries = StructVector::GetEntries(*this); // Deserialize entries as a list - deserializer.SetTag(103, "children"); - auto read_size = deserializer.OnListBegin(); - D_ASSERT(read_size == entries.size()); - (void)read_size; - for (auto &entry : entries) { - deserializer.OnObjectBegin(); - entry->FormatDeserialize(deserializer, count); - deserializer.OnObjectEnd(); - } - deserializer.OnListEnd(); + deserializer.ReadList(103, "children", [&](FormatDeserializer::List &list, idx_t i) { + list.ReadObject([&](FormatDeserializer &obj) { entries[i]->FormatDeserialize(obj, count); }); + }); break; } case PhysicalType::LIST: { @@ -1142,24 +1114,18 @@ void Vector::FormatDeserialize(FormatDeserializer &deserializer, idx_t count) { // Read the entries auto list_entries = FlatVector::GetData(*this); - deserializer.SetTag(105, "entries"); - auto entries_count = deserializer.OnListBegin(); - D_ASSERT(entries_count == count); - (void)entries_count; - for (idx_t i = 0; i < count; i++) { - deserializer.OnObjectBegin(); - deserializer.ReadProperty(100, "offset", list_entries[i].offset); - deserializer.ReadProperty(101, "length", list_entries[i].length); - deserializer.OnObjectEnd(); - } - deserializer.OnListEnd(); + deserializer.ReadList(105, "entries", [&](FormatDeserializer::List &list, idx_t i) { + list.ReadObject([&](FormatDeserializer &obj) { + list_entries[i].offset = obj.ReadProperty(100, "offset"); + list_entries[i].length = obj.ReadProperty(101, "length"); + }); + }); // Read the child vector - deserializer.SetTag(106, "child"); - auto &child = ListVector::GetEntry(*this); - deserializer.OnObjectBegin(); - child.FormatDeserialize(deserializer, list_size); - deserializer.OnObjectEnd(); + deserializer.ReadObject(106, "child", [&](FormatDeserializer &obj) { + auto &child = ListVector::GetEntry(*this); + child.FormatDeserialize(obj, list_size); + }); break; } default: diff --git a/src/duckdb/src/core_functions/scalar/list/list_lambdas.cpp b/src/duckdb/src/core_functions/scalar/list/list_lambdas.cpp index 36de67ef0..13bc60b35 100644 --- a/src/duckdb/src/core_functions/scalar/list/list_lambdas.cpp +++ b/src/duckdb/src/core_functions/scalar/list/list_lambdas.cpp @@ -35,12 +35,13 @@ struct ListLambdaBindData : public FunctionData { const ScalarFunction &function) { auto &bind_data = bind_data_p->Cast(); serializer.WriteProperty(100, "stype", bind_data.stype); - serializer.WriteOptionalProperty(101, "lambda_expr", bind_data.lambda_expr); + serializer.WritePropertyWithDefault(101, "lambda_expr", bind_data.lambda_expr, unique_ptr()); } static unique_ptr FormatDeserialize(FormatDeserializer &deserializer, ScalarFunction &function) { auto stype = deserializer.ReadProperty(100, "stype"); - auto lambda_expr = deserializer.ReadOptionalProperty>(101, "lambda_expr"); + auto lambda_expr = + deserializer.ReadPropertyWithDefault>(101, "lambda_expr", unique_ptr()); return make_uniq(stype, std::move(lambda_expr)); } }; diff --git a/src/duckdb/src/execution/aggregate_hashtable.cpp b/src/duckdb/src/execution/aggregate_hashtable.cpp index 9c23cca4a..1aeb08c90 100644 --- a/src/duckdb/src/execution/aggregate_hashtable.cpp +++ b/src/duckdb/src/execution/aggregate_hashtable.cpp @@ -7,13 +7,9 @@ #include "duckdb/common/row_operations/row_operations.hpp" #include "duckdb/common/types/null_value.hpp" #include "duckdb/common/types/row/tuple_data_iterator.hpp" -#include "duckdb/common/vector_operations/unary_executor.hpp" #include "duckdb/common/vector_operations/vector_operations.hpp" #include "duckdb/execution/expression_executor.hpp" #include "duckdb/planner/expression/bound_aggregate_expression.hpp" -#include "duckdb/storage/buffer_manager.hpp" - -#include namespace duckdb { @@ -22,9 +18,9 @@ using ValidityBytes = TupleDataLayout::ValidityBytes; GroupedAggregateHashTable::GroupedAggregateHashTable(ClientContext &context, Allocator &allocator, vector group_types, vector payload_types, const vector &bindings, - HtEntryType entry_type, idx_t initial_capacity) + idx_t initial_capacity, idx_t radix_bits) : GroupedAggregateHashTable(context, allocator, std::move(group_types), std::move(payload_types), - AggregateObject::CreateAggregateObjects(bindings), entry_type, initial_capacity) { + AggregateObject::CreateAggregateObjects(bindings), initial_capacity, radix_bits) { } GroupedAggregateHashTable::GroupedAggregateHashTable(ClientContext &context, Allocator &allocator, @@ -32,205 +28,189 @@ GroupedAggregateHashTable::GroupedAggregateHashTable(ClientContext &context, All : GroupedAggregateHashTable(context, allocator, std::move(group_types), {}, vector()) { } -AggregateHTAppendState::AggregateHTAppendState() - : ht_offsets(LogicalTypeId::BIGINT), hash_salts(LogicalTypeId::SMALLINT), - group_compare_vector(STANDARD_VECTOR_SIZE), no_match_vector(STANDARD_VECTOR_SIZE), - empty_vector(STANDARD_VECTOR_SIZE), new_groups(STANDARD_VECTOR_SIZE), addresses(LogicalType::POINTER), - chunk_state_initialized(false) { +GroupedAggregateHashTable::AggregateHTAppendState::AggregateHTAppendState() + : ht_offsets(LogicalType::UBIGINT), hash_salts(LogicalType::HASH), group_compare_vector(STANDARD_VECTOR_SIZE), + no_match_vector(STANDARD_VECTOR_SIZE), empty_vector(STANDARD_VECTOR_SIZE), new_groups(STANDARD_VECTOR_SIZE), + addresses(LogicalType::POINTER) { } GroupedAggregateHashTable::GroupedAggregateHashTable(ClientContext &context, Allocator &allocator, vector group_types_p, vector payload_types_p, vector aggregate_objects_p, - HtEntryType entry_type, idx_t initial_capacity) + idx_t initial_capacity, idx_t radix_bits) : BaseAggregateHashTable(context, allocator, aggregate_objects_p, std::move(payload_types_p)), - entry_type(entry_type), capacity(0), is_finalized(false), - aggregate_allocator(make_shared(allocator)) { + radix_bits(radix_bits), count(0), capacity(0), aggregate_allocator(make_shared(allocator)) { + // Append hash column to the end and initialise the row layout group_types_p.emplace_back(LogicalType::HASH); layout.Initialize(std::move(group_types_p), std::move(aggregate_objects_p)); - tuple_size = layout.GetRowWidth(); - tuples_per_block = Storage::BLOCK_SIZE / tuple_size; - - // HT layout hash_offset = layout.GetOffsets()[layout.ColumnCount() - 1]; - data_collection = make_uniq(buffer_manager, layout); - data_collection->InitializeAppend(td_pin_state, TupleDataPinProperties::KEEP_EVERYTHING_PINNED); - - switch (entry_type) { - case HtEntryType::HT_WIDTH_64: { - hash_prefix_shift = (HASH_WIDTH - sizeof(aggr_ht_entry_64::salt)) * 8; - Resize(initial_capacity); - break; - } - case HtEntryType::HT_WIDTH_32: { - hash_prefix_shift = (HASH_WIDTH - sizeof(aggr_ht_entry_32::salt)) * 8; - Resize(initial_capacity); - break; - } - default: - throw InternalException("Unknown HT entry width"); - } + // Partitioned data and pointer table + InitializePartitionedData(); + Resize(initial_capacity); + + // Predicates predicates.resize(layout.ColumnCount() - 1, ExpressionType::COMPARE_EQUAL); } +void GroupedAggregateHashTable::InitializePartitionedData() { + if (!partitioned_data || RadixPartitioning::RadixBits(partitioned_data->PartitionCount()) != radix_bits) { + D_ASSERT(!partitioned_data || partitioned_data->Count() == 0); + partitioned_data = + make_uniq(buffer_manager, layout, radix_bits, layout.ColumnCount() - 1); + } else { + partitioned_data->Reset(); + } + + D_ASSERT(GetLayout().GetAggrWidth() == layout.GetAggrWidth()); + D_ASSERT(GetLayout().GetDataWidth() == layout.GetDataWidth()); + D_ASSERT(GetLayout().GetRowWidth() == layout.GetRowWidth()); + + partitioned_data->InitializeAppendState(state.append_state, TupleDataPinProperties::KEEP_EVERYTHING_PINNED); +} + +unique_ptr &GroupedAggregateHashTable::GetPartitionedData() { + return partitioned_data; +} + +shared_ptr GroupedAggregateHashTable::GetAggregateAllocator() { + return aggregate_allocator; +} + GroupedAggregateHashTable::~GroupedAggregateHashTable() { Destroy(); } void GroupedAggregateHashTable::Destroy() { - if (data_collection->Count() == 0) { - return; - } - - // Check if there is an aggregate with a destructor - bool has_destructor = false; - for (auto &aggr : layout.GetAggregates()) { - if (aggr.function.destructor) { - has_destructor = true; - } - } - if (!has_destructor) { + if (!partitioned_data || partitioned_data->Count() == 0 || !layout.HasDestructor()) { return; } // There are aggregates with destructors: Call the destructor for each of the aggregates - RowOperationsState state(*aggregate_allocator); - TupleDataChunkIterator iterator(*data_collection, TupleDataPinProperties::DESTROY_AFTER_DONE, false); - auto &row_locations = iterator.GetChunkState().row_locations; - do { - RowOperations::DestroyStates(state, layout, row_locations, iterator.GetCurrentChunkCount()); - } while (iterator.Next()); - data_collection->Reset(); -} - -template -void GroupedAggregateHashTable::VerifyInternal() { - auto hashes_ptr = (ENTRY *)hashes_hdl_ptr; - idx_t count = 0; - for (idx_t i = 0; i < capacity; i++) { - if (hashes_ptr[i].page_nr > 0) { - D_ASSERT(hashes_ptr[i].page_offset < tuples_per_block); - D_ASSERT(hashes_ptr[i].page_nr <= payload_hds_ptrs.size()); - auto ptr = payload_hds_ptrs[hashes_ptr[i].page_nr - 1] + ((hashes_ptr[i].page_offset) * tuple_size); - auto hash = Load(ptr + hash_offset); - D_ASSERT((hashes_ptr[i].salt) == (hash >> hash_prefix_shift)); - - count++; + // Currently does not happen because aggregate destructors are called while scanning in RadixPartitionedHashTable + // LCOV_EXCL_START + RowOperationsState row_state(*aggregate_allocator); + for (auto &data_collection : partitioned_data->GetPartitions()) { + if (data_collection->Count() == 0) { + continue; } + TupleDataChunkIterator iterator(*data_collection, TupleDataPinProperties::DESTROY_AFTER_DONE, false); + auto &row_locations = iterator.GetChunkState().row_locations; + do { + RowOperations::DestroyStates(row_state, layout, row_locations, iterator.GetCurrentChunkCount()); + } while (iterator.Next()); + data_collection->Reset(); } - (void)count; - D_ASSERT(count == Count()); + // LCOV_EXCL_STOP +} + +const TupleDataLayout &GroupedAggregateHashTable::GetLayout() const { + return partitioned_data->GetLayout(); +} + +idx_t GroupedAggregateHashTable::Count() const { + return count; } idx_t GroupedAggregateHashTable::InitialCapacity() { return STANDARD_VECTOR_SIZE * 2ULL; } -idx_t GroupedAggregateHashTable::GetMaxCapacity(HtEntryType entry_type, idx_t tuple_size) { - idx_t max_pages; - idx_t max_tuples; - - switch (entry_type) { - case HtEntryType::HT_WIDTH_32: - max_pages = NumericLimits::Maximum(); - max_tuples = NumericLimits::Maximum(); - break; - case HtEntryType::HT_WIDTH_64: - max_pages = NumericLimits::Maximum(); - max_tuples = NumericLimits::Maximum(); - break; - default: - throw InternalException("Unsupported hash table width"); - } +idx_t GroupedAggregateHashTable::GetCapacityForCount(idx_t count) { + count = MaxValue(InitialCapacity(), count); + return NextPowerOfTwo(count * LOAD_FACTOR); +} + +idx_t GroupedAggregateHashTable::Capacity() const { + return capacity; +} - return max_pages * MinValue(max_tuples, (idx_t)Storage::BLOCK_SIZE / tuple_size); +idx_t GroupedAggregateHashTable::ResizeThreshold() const { + return Capacity() / LOAD_FACTOR; } -idx_t GroupedAggregateHashTable::MaxCapacity() { - return GetMaxCapacity(entry_type, tuple_size); +idx_t GroupedAggregateHashTable::ApplyBitMask(hash_t hash) const { + return hash & bitmask; } void GroupedAggregateHashTable::Verify() { #ifdef DEBUG - switch (entry_type) { - case HtEntryType::HT_WIDTH_32: - VerifyInternal(); - break; - case HtEntryType::HT_WIDTH_64: - VerifyInternal(); - break; + idx_t total_count = 0; + for (idx_t i = 0; i < capacity; i++) { + const auto &entry = entries[i]; + if (!entry.IsOccupied()) { + continue; + } + auto hash = Load(entry.GetPointer() + hash_offset); + D_ASSERT(entry.GetSalt() == aggr_ht_entry_t::ExtractSalt(hash)); + total_count++; } + D_ASSERT(total_count == Count()); #endif } -template +void GroupedAggregateHashTable::ClearPointerTable() { + std::fill_n(entries, capacity, aggr_ht_entry_t(0)); +} + +void GroupedAggregateHashTable::ResetCount() { + count = 0; +} + +void GroupedAggregateHashTable::SetRadixBits(idx_t radix_bits_p) { + radix_bits = radix_bits_p; +} + void GroupedAggregateHashTable::Resize(idx_t size) { - D_ASSERT(!is_finalized); D_ASSERT(size >= STANDARD_VECTOR_SIZE); D_ASSERT(IsPowerOfTwo(size)); - if (size < capacity) { throw InternalException("Cannot downsize a hash table!"); } - capacity = size; + capacity = size; + hash_map = buffer_manager.GetBufferAllocator().Allocate(capacity * sizeof(aggr_ht_entry_t)); + entries = reinterpret_cast(hash_map.get()); + ClearPointerTable(); bitmask = capacity - 1; - const auto byte_size = capacity * sizeof(ENTRY); - hashes_hdl = buffer_manager.GetBufferAllocator().Allocate(byte_size); - hashes_hdl_ptr = hashes_hdl.get(); - memset(hashes_hdl_ptr, 0, byte_size); if (Count() != 0) { - D_ASSERT(!payload_hds_ptrs.empty()); - auto hashes_arr = (ENTRY *)hashes_hdl_ptr; - - idx_t block_id = 0; - auto block_pointer = payload_hds_ptrs[block_id]; - auto block_end = block_pointer + tuples_per_block * tuple_size; - - TupleDataChunkIterator iterator(*data_collection, TupleDataPinProperties::ALREADY_PINNED, false); - const auto row_locations = iterator.GetRowLocations(); - do { - for (idx_t i = 0; i < iterator.GetCurrentChunkCount(); i++) { - const auto &row_location = row_locations[i]; - if (row_location > block_end || row_location < block_pointer) { - block_id++; - D_ASSERT(block_id < payload_hds_ptrs.size()); - block_pointer = payload_hds_ptrs[block_id]; - block_end = block_pointer + tuples_per_block * tuple_size; - } - D_ASSERT(row_location >= block_pointer && row_location < block_end); - D_ASSERT((row_location - block_pointer) % tuple_size == 0); - - const auto hash = Load(row_location + hash_offset); - D_ASSERT((hash & bitmask) == (hash % capacity)); - D_ASSERT(hash >> hash_prefix_shift <= NumericLimits::Maximum()); - - auto entry_idx = (idx_t)hash & bitmask; - while (hashes_arr[entry_idx].page_nr > 0) { - entry_idx++; - if (entry_idx >= capacity) { - entry_idx = 0; + for (auto &data_collection : partitioned_data->GetPartitions()) { + if (data_collection->Count() == 0) { + continue; + } + TupleDataChunkIterator iterator(*data_collection, TupleDataPinProperties::ALREADY_PINNED, false); + const auto row_locations = iterator.GetRowLocations(); + do { + for (idx_t i = 0; i < iterator.GetCurrentChunkCount(); i++) { + const auto &row_location = row_locations[i]; + const auto hash = Load(row_location + hash_offset); + + // Find an empty entry + auto entry_idx = ApplyBitMask(hash); + D_ASSERT(entry_idx == hash % capacity); + while (entries[entry_idx].IsOccupied() > 0) { + entry_idx++; + if (entry_idx >= capacity) { + entry_idx = 0; + } } + auto &entry = entries[entry_idx]; + D_ASSERT(!entry.IsOccupied()); + entry.SetSalt(aggr_ht_entry_t::ExtractSalt(hash)); + entry.SetPointer(row_location); + D_ASSERT(entry.IsOccupied()); } - - auto &ht_entry = hashes_arr[entry_idx]; - D_ASSERT(!ht_entry.page_nr); - ht_entry.salt = hash >> hash_prefix_shift; - ht_entry.page_nr = block_id + 1; - ht_entry.page_offset = (row_location - block_pointer) / tuple_size; - } - } while (iterator.Next()); + } while (iterator.Next()); + } } Verify(); } -idx_t GroupedAggregateHashTable::AddChunk(AggregateHTAppendState &state, DataChunk &groups, DataChunk &payload, - AggregateType filter) { +idx_t GroupedAggregateHashTable::AddChunk(DataChunk &groups, DataChunk &payload, AggregateType filter) { unsafe_vector aggregate_filter; auto &aggregates = layout.GetAggregates(); @@ -240,20 +220,18 @@ idx_t GroupedAggregateHashTable::AddChunk(AggregateHTAppendState &state, DataChu aggregate_filter.push_back(i); } } - return AddChunk(state, groups, payload, aggregate_filter); + return AddChunk(groups, payload, aggregate_filter); } -idx_t GroupedAggregateHashTable::AddChunk(AggregateHTAppendState &state, DataChunk &groups, DataChunk &payload, - const unsafe_vector &filter) { +idx_t GroupedAggregateHashTable::AddChunk(DataChunk &groups, DataChunk &payload, const unsafe_vector &filter) { Vector hashes(LogicalType::HASH); groups.Hash(hashes); - return AddChunk(state, groups, hashes, payload, filter); + return AddChunk(groups, hashes, payload, filter); } -idx_t GroupedAggregateHashTable::AddChunk(AggregateHTAppendState &state, DataChunk &groups, Vector &group_hashes, - DataChunk &payload, const unsafe_vector &filter) { - D_ASSERT(!is_finalized); +idx_t GroupedAggregateHashTable::AddChunk(DataChunk &groups, Vector &group_hashes, DataChunk &payload, + const unsafe_vector &filter) { if (groups.size() == 0) { return 0; } @@ -265,7 +243,7 @@ idx_t GroupedAggregateHashTable::AddChunk(AggregateHTAppendState &state, DataChu } #endif - auto new_group_count = FindOrCreateGroups(state, groups, group_hashes, state.addresses, state.new_groups); + const auto new_group_count = FindOrCreateGroups(groups, group_hashes, state.addresses, state.new_groups); VectorOperations::AddInPlace(state.addresses, layout.GetAggrOffset(), payload.size()); // Now every cell has an entry, update the aggregates @@ -301,11 +279,14 @@ idx_t GroupedAggregateHashTable::AddChunk(AggregateHTAppendState &state, DataChu } void GroupedAggregateHashTable::FetchAggregates(DataChunk &groups, DataChunk &result) { +#ifdef DEBUG groups.Verify(); D_ASSERT(groups.ColumnCount() + 1 == layout.ColumnCount()); for (idx_t i = 0; i < result.ColumnCount(); i++) { D_ASSERT(result.data[i].GetType() == payload_types[i]); } +#endif + result.SetCardinality(groups); if (groups.size() == 0) { return; @@ -313,57 +294,46 @@ void GroupedAggregateHashTable::FetchAggregates(DataChunk &groups, DataChunk &re // find the groups associated with the addresses // FIXME: this should not use the FindOrCreateGroups, creating them is unnecessary - AggregateHTAppendState append_state; Vector addresses(LogicalType::POINTER); - FindOrCreateGroups(append_state, groups, addresses); + FindOrCreateGroups(groups, addresses); // now fetch the aggregates RowOperationsState row_state(*aggregate_allocator); RowOperations::FinalizeStates(row_state, layout, addresses, result, 0); } -idx_t GroupedAggregateHashTable::ResizeThreshold() { - return capacity / LOAD_FACTOR; -} - -template -idx_t GroupedAggregateHashTable::FindOrCreateGroupsInternal(AggregateHTAppendState &state, DataChunk &groups, - Vector &group_hashes_v, Vector &addresses_v, - SelectionVector &new_groups_out) { - D_ASSERT(!is_finalized); +idx_t GroupedAggregateHashTable::FindOrCreateGroupsInternal(DataChunk &groups, Vector &group_hashes_v, + Vector &addresses_v, SelectionVector &new_groups_out) { D_ASSERT(groups.ColumnCount() + 1 == layout.ColumnCount()); D_ASSERT(group_hashes_v.GetType() == LogicalType::HASH); D_ASSERT(state.ht_offsets.GetVectorType() == VectorType::FLAT_VECTOR); - D_ASSERT(state.ht_offsets.GetType() == LogicalType::BIGINT); + D_ASSERT(state.ht_offsets.GetType() == LogicalType::UBIGINT); D_ASSERT(addresses_v.GetType() == LogicalType::POINTER); - D_ASSERT(state.hash_salts.GetType() == LogicalType::SMALLINT); - - if (Count() + groups.size() > MaxCapacity()) { - throw InternalException("Hash table capacity reached"); - } + D_ASSERT(state.hash_salts.GetType() == LogicalType::HASH); - // Resize at 50% capacity, also need to fit the entire vector - if (capacity - Count() <= groups.size() || Count() > ResizeThreshold()) { + // Need to fit the entire vector, and resize at threshold + if (Count() + groups.size() > capacity || Count() + groups.size() > ResizeThreshold()) { Verify(); - Resize(capacity * 2); + Resize(capacity * 2); } D_ASSERT(capacity - Count() >= groups.size()); // we need to be able to fit at least one vector of data group_hashes_v.Flatten(groups.size()); - auto group_hashes = FlatVector::GetData(group_hashes_v); + auto hashes = FlatVector::GetData(group_hashes_v); addresses_v.Flatten(groups.size()); auto addresses = FlatVector::GetData(addresses_v); // Compute the entry in the table based on the hash using a modulo, // and precompute the hash salts for faster comparison below - auto ht_offsets_ptr = FlatVector::GetData(state.ht_offsets); - auto hash_salts_ptr = FlatVector::GetData(state.hash_salts); + auto ht_offsets = FlatVector::GetData(state.ht_offsets); + auto hash_salts = FlatVector::GetData(state.hash_salts); for (idx_t r = 0; r < groups.size(); r++) { - auto element = group_hashes[r]; - D_ASSERT((element & bitmask) == (element % capacity)); - ht_offsets_ptr[r] = element & bitmask; - hash_salts_ptr[r] = element >> hash_prefix_shift; + const auto &hash = hashes[r]; + ht_offsets[r] = ApplyBitMask(hash); + D_ASSERT(ht_offsets[r] == hash % capacity); + hash_salts[r] = aggr_ht_entry_t::ExtractSalt(hash); } + // we start out with all entries [0, 1, 2, ..., groups.size()] const SelectionVector *sel_vector = FlatVector::IncrementalSelectionVector(); @@ -379,15 +349,12 @@ idx_t GroupedAggregateHashTable::FindOrCreateGroupsInternal(AggregateHTAppendSta state.group_chunk.SetCardinality(groups); // convert all vectors to unified format - if (!state.chunk_state_initialized) { - data_collection->InitializeAppend(state.chunk_state); - state.chunk_state_initialized = true; - } - TupleDataCollection::ToUnifiedFormat(state.chunk_state, state.group_chunk); + auto &chunk_state = state.append_state.chunk_state; + TupleDataCollection::ToUnifiedFormat(chunk_state, state.group_chunk); if (!state.group_data) { state.group_data = make_unsafe_uniq_array(state.group_chunk.ColumnCount()); } - TupleDataCollection::GetVectorData(state.chunk_state, state.group_data.get()); + TupleDataCollection::GetVectorData(chunk_state, state.group_data.get()); idx_t new_group_count = 0; idx_t remaining_entries = groups.size(); @@ -398,57 +365,42 @@ idx_t GroupedAggregateHashTable::FindOrCreateGroupsInternal(AggregateHTAppendSta // For each remaining entry, figure out whether or not it belongs to a full or empty group for (idx_t i = 0; i < remaining_entries; i++) { - const idx_t index = sel_vector->get_index(i); - auto &ht_entry = *(((ENTRY *)this->hashes_hdl_ptr) + ht_offsets_ptr[index]); - if (ht_entry.page_nr == 0) { // Cell is unoccupied (we use page number 0 as a "unused marker") - D_ASSERT(group_hashes[index] >> hash_prefix_shift <= NumericLimits::Maximum()); - D_ASSERT(payload_hds_ptrs.size() < NumericLimits::Maximum()); - - // Set page nr to 1 for now to mark it as occupied (will be corrected later) and set the salt - ht_entry.page_nr = 1; - ht_entry.salt = group_hashes[index] >> hash_prefix_shift; - - // Update selection lists for outer loops - state.empty_vector.set_index(new_entry_count++, index); - new_groups_out.set_index(new_group_count++, index); - } else { // Cell is occupied: Compare salts - if (ht_entry.salt == hash_salts_ptr[index]) { + const auto index = sel_vector->get_index(i); + const auto &salt = hash_salts[index]; + auto &entry = entries[ht_offsets[index]]; + if (entry.IsOccupied()) { // Cell is occupied: Compare salts + if (entry.GetSalt() == salt) { state.group_compare_vector.set_index(need_compare_count++, index); } else { state.no_match_vector.set_index(no_match_count++, index); } + } else { // Cell is unoccupied + // Set salt (also marks as occupied) + entry.SetSalt(salt); + + // Update selection lists for outer loops + state.empty_vector.set_index(new_entry_count++, index); + new_groups_out.set_index(new_group_count++, index); } } if (new_entry_count != 0) { // Append everything that belongs to an empty group - data_collection->AppendUnified(td_pin_state, state.chunk_state, state.group_chunk, state.empty_vector, - new_entry_count); - RowOperations::InitializeStates(layout, state.chunk_state.row_locations, + partitioned_data->AppendUnified(state.append_state, state.group_chunk, state.empty_vector, new_entry_count); + RowOperations::InitializeStates(layout, chunk_state.row_locations, *FlatVector::IncrementalSelectionVector(), new_entry_count); - // Get the pointers to the (possibly) newly created blocks of the data collection - idx_t block_id = payload_hds_ptrs.empty() ? 0 : payload_hds_ptrs.size() - 1; - UpdateBlockPointers(); - auto block_pointer = payload_hds_ptrs[block_id]; - auto block_end = block_pointer + tuples_per_block * tuple_size; - - // Set the page nrs/offsets in the 1st part of the HT now that the data has been appended - const auto row_locations = FlatVector::GetData(state.chunk_state.row_locations); + // Set the entry pointers in the 1st part of the HT now that the data has been appended + const auto row_locations = FlatVector::GetData(chunk_state.row_locations); + const auto &row_sel = state.append_state.reverse_partition_sel; for (idx_t new_entry_idx = 0; new_entry_idx < new_entry_count; new_entry_idx++) { - const auto &row_location = row_locations[new_entry_idx]; - if (row_location > block_end || row_location < block_pointer) { - block_id++; - D_ASSERT(block_id < payload_hds_ptrs.size()); - block_pointer = payload_hds_ptrs[block_id]; - block_end = block_pointer + tuples_per_block * tuple_size; - } - D_ASSERT(row_location >= block_pointer && row_location < block_end); - D_ASSERT((row_location - block_pointer) % tuple_size == 0); const auto index = state.empty_vector.get_index(new_entry_idx); - auto &ht_entry = *(((ENTRY *)this->hashes_hdl_ptr) + ht_offsets_ptr[index]); - ht_entry.page_nr = block_id + 1; - ht_entry.page_offset = (row_location - block_pointer) / tuple_size; + const auto row_idx = row_sel.get_index(index); + const auto &row_location = row_locations[row_idx]; + + auto &entry = entries[ht_offsets[index]]; + + entry.SetPointer(row_location); addresses[index] = row_location; } } @@ -457,10 +409,8 @@ idx_t GroupedAggregateHashTable::FindOrCreateGroupsInternal(AggregateHTAppendSta // Get the pointers to the rows that need to be compared for (idx_t need_compare_idx = 0; need_compare_idx < need_compare_count; need_compare_idx++) { const auto index = state.group_compare_vector.get_index(need_compare_idx); - const auto &ht_entry = *(((ENTRY *)this->hashes_hdl_ptr) + ht_offsets_ptr[index]); - auto page_ptr = payload_hds_ptrs[ht_entry.page_nr - 1]; - auto page_offset = ht_entry.page_offset * tuple_size; - addresses[index] = page_ptr + page_offset; + const auto &entry = entries[ht_offsets[index]]; + addresses[index] = entry.GetPointer(); } // Perform group comparisons @@ -472,55 +422,36 @@ idx_t GroupedAggregateHashTable::FindOrCreateGroupsInternal(AggregateHTAppendSta // Linear probing: each of the entries that do not match move to the next entry in the HT for (idx_t i = 0; i < no_match_count; i++) { idx_t index = state.no_match_vector.get_index(i); - ht_offsets_ptr[index]++; - if (ht_offsets_ptr[index] >= capacity) { - ht_offsets_ptr[index] = 0; + ht_offsets[index]++; + if (ht_offsets[index] >= capacity) { + ht_offsets[index] = 0; } } sel_vector = &state.no_match_vector; remaining_entries = no_match_count; } + count += new_group_count; return new_group_count; } -void GroupedAggregateHashTable::UpdateBlockPointers() { - for (const auto &id_and_handle : td_pin_state.row_handles) { - const auto &id = id_and_handle.first; - const auto &handle = id_and_handle.second; - if (payload_hds_ptrs.empty() || id > payload_hds_ptrs.size() - 1) { - payload_hds_ptrs.resize(id + 1); - } - payload_hds_ptrs[id] = handle.Ptr(); - } -} - // this is to support distinct aggregations where we need to record whether we // have already seen a value for a group -idx_t GroupedAggregateHashTable::FindOrCreateGroups(AggregateHTAppendState &state, DataChunk &groups, - Vector &group_hashes, Vector &addresses_out, +idx_t GroupedAggregateHashTable::FindOrCreateGroups(DataChunk &groups, Vector &group_hashes, Vector &addresses_out, SelectionVector &new_groups_out) { - switch (entry_type) { - case HtEntryType::HT_WIDTH_64: - return FindOrCreateGroupsInternal(state, groups, group_hashes, addresses_out, new_groups_out); - case HtEntryType::HT_WIDTH_32: - return FindOrCreateGroupsInternal(state, groups, group_hashes, addresses_out, new_groups_out); - default: - throw InternalException("Unknown HT entry width"); - } + return FindOrCreateGroupsInternal(groups, group_hashes, addresses_out, new_groups_out); } -void GroupedAggregateHashTable::FindOrCreateGroups(AggregateHTAppendState &state, DataChunk &groups, - Vector &addresses) { +void GroupedAggregateHashTable::FindOrCreateGroups(DataChunk &groups, Vector &addresses) { // create a dummy new_groups sel vector - FindOrCreateGroups(state, groups, addresses, state.new_groups); + FindOrCreateGroups(groups, addresses, state.new_groups); } -idx_t GroupedAggregateHashTable::FindOrCreateGroups(AggregateHTAppendState &state, DataChunk &groups, - Vector &addresses_out, SelectionVector &new_groups_out) { +idx_t GroupedAggregateHashTable::FindOrCreateGroups(DataChunk &groups, Vector &addresses_out, + SelectionVector &new_groups_out) { Vector hashes(LogicalType::HASH); groups.Hash(hashes); - return FindOrCreateGroups(state, groups, hashes, addresses_out, new_groups_out); + return FindOrCreateGroups(groups, hashes, addresses_out, new_groups_out); } struct FlushMoveState { @@ -533,13 +464,21 @@ struct FlushMoveState { for (idx_t col_idx = 0; col_idx < layout.ColumnCount() - 1; col_idx++) { column_ids.emplace_back(col_idx); } - // FIXME DESTROY_AFTER_DONE if we make it possible to pass a selection vector to RowOperations::DestroyStates? - collection.InitializeScan(scan_state, column_ids, TupleDataPinProperties::UNPIN_AFTER_DONE); + collection.InitializeScan(scan_state, column_ids, TupleDataPinProperties::DESTROY_AFTER_DONE); collection.InitializeScanChunk(scan_state, groups); hash_col_idx = layout.ColumnCount() - 1; } - bool Scan(); + bool Scan() { + if (collection.Scan(scan_state, groups)) { + collection.Gather(scan_state.chunk_state.row_locations, *FlatVector::IncrementalSelectionVector(), + groups.size(), hash_col_idx, hashes, *FlatVector::IncrementalSelectionVector()); + return true; + } + + collection.FinalizePinState(scan_state.pin_state); + return false; + } TupleDataCollection &collection; TupleDataScanState scan_state; @@ -548,52 +487,13 @@ struct FlushMoveState { idx_t hash_col_idx; Vector hashes; - AggregateHTAppendState append_state; Vector group_addresses; SelectionVector new_groups_sel; }; -bool FlushMoveState::Scan() { - if (collection.Scan(scan_state, groups)) { - collection.Gather(scan_state.chunk_state.row_locations, *FlatVector::IncrementalSelectionVector(), - groups.size(), hash_col_idx, hashes, *FlatVector::IncrementalSelectionVector()); - return true; - } - - collection.FinalizePinState(scan_state.pin_state); - return false; -} - void GroupedAggregateHashTable::Combine(GroupedAggregateHashTable &other) { - D_ASSERT(!is_finalized); - - D_ASSERT(other.layout.GetAggrWidth() == layout.GetAggrWidth()); - D_ASSERT(other.layout.GetDataWidth() == layout.GetDataWidth()); - D_ASSERT(other.layout.GetRowWidth() == layout.GetRowWidth()); - - if (other.Count() == 0) { - return; - } - - FlushMoveState state(*other.data_collection); - RowOperationsState row_state(*aggregate_allocator); - while (state.Scan()) { - FindOrCreateGroups(state.append_state, state.groups, state.hashes, state.group_addresses, state.new_groups_sel); - RowOperations::CombineStates(row_state, layout, state.scan_state.chunk_state.row_locations, - state.group_addresses, state.groups.size()); - } - - Verify(); - - // if we combine states, then we also need to combine the arena allocators - for (auto &stored_allocator : other.stored_allocators) { - stored_allocators.push_back(stored_allocator); - } - stored_allocators.push_back(other.aggregate_allocator); -} - -void GroupedAggregateHashTable::Append(GroupedAggregateHashTable &other) { - data_collection->Combine(other.GetDataCollection()); + auto other_data = other.partitioned_data->GetUnpartitioned(); + Combine(*other_data); // Inherit ownership to all stored aggregate allocators stored_allocators.emplace_back(other.aggregate_allocator); @@ -602,75 +502,33 @@ void GroupedAggregateHashTable::Append(GroupedAggregateHashTable &other) { } } -void GroupedAggregateHashTable::Partition(vector &partition_hts, idx_t radix_bits, - bool sink_done) { - const auto num_partitions = RadixPartitioning::NumberOfPartitions(radix_bits); - D_ASSERT(partition_hts.size() == num_partitions); - - // Partition the data - auto pin_properties = - sink_done ? TupleDataPinProperties::UNPIN_AFTER_DONE : TupleDataPinProperties::KEEP_EVERYTHING_PINNED; - auto partitioned_data = - make_uniq(buffer_manager, layout, radix_bits, layout.ColumnCount() - 1); - partitioned_data->Partition(*data_collection, pin_properties); - D_ASSERT(partitioned_data->GetPartitions().size() == num_partitions); - - // Move the partitioned data collections to the partitioned hash tables and initialize the 1st part of the HT - auto &partitions = partitioned_data->GetPartitions(); - for (idx_t partition_idx = 0; partition_idx < num_partitions; partition_idx++) { - auto &partition_ht = *partition_hts[partition_idx]; - partition_ht.data_collection = std::move(partitions[partition_idx]); - - // Inherit ownership to all stored aggregate allocators - partition_ht.stored_allocators.emplace_back(aggregate_allocator); - for (const auto &stored_allocator : stored_allocators) { - partition_ht.stored_allocators.emplace_back(stored_allocator); - } - - if (!sink_done) { - partition_ht.InitializeFirstPart(); - partition_ht.Verify(); - } - } -} +void GroupedAggregateHashTable::Combine(TupleDataCollection &other_data) { + D_ASSERT(other_data.GetLayout().GetAggrWidth() == layout.GetAggrWidth()); + D_ASSERT(other_data.GetLayout().GetDataWidth() == layout.GetDataWidth()); + D_ASSERT(other_data.GetLayout().GetRowWidth() == layout.GetRowWidth()); -void GroupedAggregateHashTable::InitializeFirstPart() { - data_collection->GetBlockPointers(payload_hds_ptrs); - auto size = MaxValue(NextPowerOfTwo(Count() * 2L), capacity); - switch (entry_type) { - case HtEntryType::HT_WIDTH_64: - Resize(size); - break; - case HtEntryType::HT_WIDTH_32: - Resize(size); - break; - default: - throw InternalException("Unknown HT entry width"); + if (other_data.Count() == 0) { + return; } -} - -idx_t GroupedAggregateHashTable::Scan(TupleDataParallelScanState &gstate, TupleDataLocalScanState &lstate, - DataChunk &result) { - data_collection->Scan(gstate, lstate, result); + FlushMoveState fm_state(other_data); RowOperationsState row_state(*aggregate_allocator); - const auto group_cols = layout.ColumnCount() - 1; - RowOperations::FinalizeStates(row_state, layout, lstate.chunk_state.row_locations, result, group_cols); - - return result.size(); -} - -void GroupedAggregateHashTable::Finalize() { - if (is_finalized) { - return; + while (fm_state.Scan()) { + FindOrCreateGroups(fm_state.groups, fm_state.hashes, fm_state.group_addresses, fm_state.new_groups_sel); + RowOperations::CombineStates(row_state, layout, fm_state.scan_state.chunk_state.row_locations, + fm_state.group_addresses, fm_state.groups.size()); + if (layout.HasDestructor()) { + RowOperations::DestroyStates(row_state, layout, fm_state.scan_state.chunk_state.row_locations, + fm_state.groups.size()); + } } - // Early release hashes (not needed for partition/scan) and data collection (will be pinned again when scanning) - hashes_hdl.Reset(); - data_collection->FinalizePinState(td_pin_state); - data_collection->Unpin(); + Verify(); +} - is_finalized = true; +void GroupedAggregateHashTable::UnpinData() { + partitioned_data->FlushAppendState(state.append_state); + partitioned_data->Unpin(); } } // namespace duckdb diff --git a/src/duckdb/src/execution/join_hashtable.cpp b/src/duckdb/src/execution/join_hashtable.cpp index 8e9ecddfb..1cfb66b93 100644 --- a/src/duckdb/src/execution/join_hashtable.cpp +++ b/src/duckdb/src/execution/join_hashtable.cpp @@ -191,9 +191,7 @@ void JoinHashTable::Build(PartitionedTupleDataAppendState &append_state, DataChu } info.correlated_payload.SetCardinality(keys); info.correlated_payload.data[0].Reference(keys.data[info.correlated_types.size()]); - AggregateHTAppendState append_state; - info.correlated_counts->AddChunk(append_state, info.group_chunk, info.correlated_payload, - AggregateType::NON_DISTINCT); + info.correlated_counts->AddChunk(info.group_chunk, info.correlated_payload, AggregateType::NON_DISTINCT); } // prepare the keys for processing @@ -851,9 +849,10 @@ bool JoinHashTable::RequiresExternalJoin(ClientConfig &config, vector(); + auto &gstate = state.Cast(); for (auto &grouping_state : gstate.grouping_states) { - auto &radix_state = grouping_state.table_state; - RadixPartitionedHashTable::SetMultiScan(*radix_state); + RadixPartitionedHashTable::SetMultiScan(*grouping_state.table_state); if (!grouping_state.distinct_state) { continue; } } } +//===--------------------------------------------------------------------===// +// Sink +//===--------------------------------------------------------------------===// unique_ptr PhysicalHashAggregate::GetGlobalSinkState(ClientContext &context) const { - return make_uniq(*this, context); + return make_uniq(*this, context); } unique_ptr PhysicalHashAggregate::GetLocalSinkState(ExecutionContext &context) const { - return make_uniq(*this, context); + return make_uniq(*this, context); } void PhysicalHashAggregate::SinkDistinctGrouping(ExecutionContext &context, DataChunk &chunk, OperatorSinkInput &input, idx_t grouping_idx) const { - auto &sink = input.local_state.Cast(); - auto &global_sink = input.global_state.Cast(); + auto &sink = input.local_state.Cast(); + auto &global_sink = input.global_state.Cast(); auto &grouping_gstate = global_sink.grouping_states[grouping_idx]; auto &grouping_lstate = sink.grouping_states[grouping_idx]; @@ -341,8 +343,8 @@ void PhysicalHashAggregate::SinkDistinct(ExecutionContext &context, DataChunk &c SinkResultType PhysicalHashAggregate::Sink(ExecutionContext &context, DataChunk &chunk, OperatorSinkInput &input) const { - auto &llstate = input.local_state.Cast(); - auto &gstate = input.global_state.Cast(); + auto &llstate = input.local_state.Cast(); + auto &gstate = input.global_state.Cast(); if (distinct_collection_info) { SinkDistinct(context, chunk, input); @@ -396,10 +398,13 @@ SinkResultType PhysicalHashAggregate::Sink(ExecutionContext &context, DataChunk return SinkResultType::NEED_MORE_INPUT; } +//===--------------------------------------------------------------------===// +// Combine +//===--------------------------------------------------------------------===// void PhysicalHashAggregate::CombineDistinct(ExecutionContext &context, OperatorSinkCombineInput &input) const { - auto &global_sink = input.global_state.Cast(); - auto &sink = input.local_state.Cast(); + auto &global_sink = input.global_state.Cast(); + auto &sink = input.local_state.Cast(); if (!distinct_collection_info) { return; @@ -426,8 +431,8 @@ void PhysicalHashAggregate::CombineDistinct(ExecutionContext &context, OperatorS } SinkCombineResultType PhysicalHashAggregate::Combine(ExecutionContext &context, OperatorSinkCombineInput &input) const { - auto &gstate = input.global_state.Cast(); - auto &llstate = input.local_state.Cast(); + auto &gstate = input.global_state.Cast(); + auto &llstate = input.local_state.Cast(); OperatorSinkCombineInput combine_distinct_input {gstate, llstate, input.interrupt_state}; CombineDistinct(context, combine_distinct_input); @@ -447,321 +452,267 @@ SinkCombineResultType PhysicalHashAggregate::Combine(ExecutionContext &context, return SinkCombineResultType::FINISHED; } -//! REGULAR FINALIZE EVENT - -class HashAggregateMergeEvent : public BasePipelineEvent { +//===--------------------------------------------------------------------===// +// Finalize +//===--------------------------------------------------------------------===// +class HashAggregateFinalizeEvent : public BasePipelineEvent { public: - HashAggregateMergeEvent(const PhysicalHashAggregate &op_p, HashAggregateGlobalState &gstate_p, Pipeline *pipeline_p) - : BasePipelineEvent(*pipeline_p), op(op_p), gstate(gstate_p) { + //! "Regular" Finalize Event that is scheduled after combining the thread-local distinct HTs + HashAggregateFinalizeEvent(ClientContext &context, Pipeline *pipeline_p, const PhysicalHashAggregate &op_p, + HashAggregateGlobalSinkState &gstate_p) + : BasePipelineEvent(*pipeline_p), context(context), op(op_p), gstate(gstate_p) { } - const PhysicalHashAggregate &op; - HashAggregateGlobalState &gstate; - public: - void Schedule() override { - vector> tasks; - for (idx_t i = 0; i < op.groupings.size(); i++) { - auto &grouping_gstate = gstate.grouping_states[i]; + void Schedule() override; - auto &grouping = op.groupings[i]; - auto &table = grouping.table_data; - table.ScheduleTasks(pipeline->executor, shared_from_this(), *grouping_gstate.table_state, tasks); - } - D_ASSERT(!tasks.empty()); - SetTasks(std::move(tasks)); - } -}; +private: + ClientContext &context; -//! REGULAR FINALIZE FROM DISTINCT FINALIZE + const PhysicalHashAggregate &op; + HashAggregateGlobalSinkState &gstate; +}; class HashAggregateFinalizeTask : public ExecutorTask { public: - HashAggregateFinalizeTask(Pipeline &pipeline, shared_ptr event_p, HashAggregateGlobalState &state_p, - ClientContext &context, const PhysicalHashAggregate &op) - : ExecutorTask(pipeline.executor), pipeline(pipeline), event(std::move(event_p)), gstate(state_p), - context(context), op(op) { + HashAggregateFinalizeTask(ClientContext &context, Pipeline &pipeline, shared_ptr event_p, + const PhysicalHashAggregate &op, HashAggregateGlobalSinkState &state_p) + : ExecutorTask(pipeline.executor), context(context), pipeline(pipeline), event(std::move(event_p)), op(op), + gstate(state_p) { } - TaskExecutionResult ExecuteTask(TaskExecutionMode mode) override { - op.FinalizeInternal(pipeline, *event, context, gstate, false); - D_ASSERT(!gstate.finished); - gstate.finished = true; - event->FinishTask(); - return TaskExecutionResult::TASK_FINISHED; - } +public: + TaskExecutionResult ExecuteTask(TaskExecutionMode mode) override; private: + ClientContext &context; Pipeline &pipeline; shared_ptr event; - HashAggregateGlobalState &gstate; - ClientContext &context; + const PhysicalHashAggregate &op; + HashAggregateGlobalSinkState &gstate; }; -class HashAggregateFinalizeEvent : public BasePipelineEvent { +void HashAggregateFinalizeEvent::Schedule() { + vector> tasks; + tasks.push_back(make_uniq(context, *pipeline, shared_from_this(), op, gstate)); + D_ASSERT(!tasks.empty()); + SetTasks(std::move(tasks)); +} + +TaskExecutionResult HashAggregateFinalizeTask::ExecuteTask(TaskExecutionMode mode) { + op.FinalizeInternal(pipeline, *event, context, gstate, false); + D_ASSERT(!gstate.finished); + gstate.finished = true; + event->FinishTask(); + return TaskExecutionResult::TASK_FINISHED; +} + +class HashAggregateDistinctFinalizeEvent : public BasePipelineEvent { public: - HashAggregateFinalizeEvent(const PhysicalHashAggregate &op_p, HashAggregateGlobalState &gstate_p, - Pipeline *pipeline_p, ClientContext &context) - : BasePipelineEvent(*pipeline_p), op(op_p), gstate(gstate_p), context(context) { + //! Distinct Finalize Event that is scheduled if we have distinct aggregates + HashAggregateDistinctFinalizeEvent(ClientContext &context, Pipeline &pipeline_p, const PhysicalHashAggregate &op_p, + HashAggregateGlobalSinkState &gstate_p) + : BasePipelineEvent(pipeline_p), context(context), op(op_p), gstate(gstate_p) { } - const PhysicalHashAggregate &op; - HashAggregateGlobalState &gstate; +public: + void Schedule() override; + void FinishEvent() override; + +private: + void CreateGlobalSources(); + +private: ClientContext &context; + const PhysicalHashAggregate &op; + HashAggregateGlobalSinkState &gstate; + public: - void Schedule() override { - vector> tasks; - tasks.push_back(make_uniq(*pipeline, shared_from_this(), gstate, context, op)); - D_ASSERT(!tasks.empty()); - SetTasks(std::move(tasks)); - } + //! The GlobalSourceStates for all the radix tables of the distinct aggregates + vector>> global_source_states; }; -//! DISTINCT FINALIZE TASK +class HashAggregateDistinctFinalizeTask : public ExecutorTask { +public: + HashAggregateDistinctFinalizeTask(Pipeline &pipeline, shared_ptr event_p, const PhysicalHashAggregate &op, + HashAggregateGlobalSinkState &state_p) + : ExecutorTask(pipeline.executor), pipeline(pipeline), event(std::move(event_p)), op(op), gstate(state_p) { + } -class HashDistinctAggregateFinalizeTask : public ExecutorTask { public: - HashDistinctAggregateFinalizeTask(Pipeline &pipeline, shared_ptr event_p, HashAggregateGlobalState &state_p, - ClientContext &context, const PhysicalHashAggregate &op, - vector>> &global_sources_p) - : ExecutorTask(pipeline.executor), pipeline(pipeline), event(std::move(event_p)), gstate(state_p), - context(context), op(op), global_sources(global_sources_p) { - } - - void AggregateDistinctGrouping(DistinctAggregateCollectionInfo &info, - const HashAggregateGroupingData &grouping_data, - HashAggregateGroupingGlobalState &grouping_state, idx_t grouping_idx) { - auto &aggregates = info.aggregates; - auto &data = *grouping_data.distinct_data; - auto &state = *grouping_state.distinct_state; - auto &table_state = *grouping_state.table_state; - - ThreadContext temp_thread_context(context); - ExecutionContext temp_exec_context(context, temp_thread_context, &pipeline); - - auto temp_local_state = grouping_data.table_data.GetLocalSinkState(temp_exec_context); - - // Create a chunk that mimics the 'input' chunk in Sink, for storing the group vectors - DataChunk group_chunk; - if (!op.input_group_types.empty()) { - group_chunk.Initialize(context, op.input_group_types); - } + TaskExecutionResult ExecuteTask(TaskExecutionMode mode) override; - auto &groups = op.grouped_aggregate_data.groups; - const idx_t group_by_size = groups.size(); +private: + void AggregateDistinctGrouping(const idx_t grouping_idx); - DataChunk aggregate_input_chunk; - if (!gstate.payload_types.empty()) { - aggregate_input_chunk.Initialize(context, gstate.payload_types); - } +private: + Pipeline &pipeline; + shared_ptr event; - idx_t payload_idx; - idx_t next_payload_idx = 0; + const PhysicalHashAggregate &op; + HashAggregateGlobalSinkState &gstate; +}; + +void HashAggregateDistinctFinalizeEvent::Schedule() { + CreateGlobalSources(); + + const idx_t n_threads = TaskScheduler::GetScheduler(context).NumberOfThreads(); + vector> tasks; + for (idx_t i = 0; i < n_threads; i++) { + tasks.push_back(make_uniq(*pipeline, shared_from_this(), op, gstate)); + } + SetTasks(std::move(tasks)); +} - for (idx_t i = 0; i < op.grouped_aggregate_data.aggregates.size(); i++) { - auto &aggregate = aggregates[i]->Cast(); +void HashAggregateDistinctFinalizeEvent::CreateGlobalSources() { + auto &aggregates = op.grouped_aggregate_data.aggregates; + global_source_states.reserve(op.groupings.size()); + for (idx_t grouping_idx = 0; grouping_idx < op.groupings.size(); grouping_idx++) { + auto &grouping = op.groupings[grouping_idx]; + auto &distinct_data = *grouping.distinct_data; - // Forward the payload idx - payload_idx = next_payload_idx; - next_payload_idx = payload_idx + aggregate.children.size(); + vector> aggregate_sources; + aggregate_sources.reserve(aggregates.size()); + for (idx_t agg_idx = 0; agg_idx < aggregates.size(); agg_idx++) { + auto &aggregate = aggregates[agg_idx]; + auto &aggr = aggregate->Cast(); - // If aggregate is not distinct, skip it - if (!data.IsDistinct(i)) { + if (!aggr.IsDistinct()) { + aggregate_sources.push_back(nullptr); continue; } - D_ASSERT(data.info.table_map.count(i)); - auto table_idx = data.info.table_map.at(i); - auto &radix_table_p = data.radix_tables[table_idx]; - - // Create a duplicate of the output_chunk, because of multi-threading we cant alter the original - DataChunk output_chunk; - output_chunk.Initialize(context, state.distinct_output_chunks[table_idx]->GetTypes()); - - auto &global_source = global_sources[grouping_idx][i]; - auto local_source = radix_table_p->GetLocalSourceState(temp_exec_context); - - // Fetch all the data from the aggregate ht, and Sink it into the main ht - while (true) { - output_chunk.Reset(); - group_chunk.Reset(); - aggregate_input_chunk.Reset(); - - InterruptState interrupt_state; - OperatorSourceInput source_input {*global_source, *local_source, interrupt_state}; - auto res = radix_table_p->GetData(temp_exec_context, output_chunk, *state.radix_states[table_idx], - source_input); - - if (res == SourceResultType::FINISHED) { - D_ASSERT(output_chunk.size() == 0); - break; - } else if (res == SourceResultType::BLOCKED) { - throw InternalException( - "Unexpected interrupt from radix table GetData in HashDistinctAggregateFinalizeTask"); - } - - auto &grouped_aggregate_data = *data.grouped_aggregate_data[table_idx]; - - for (idx_t group_idx = 0; group_idx < group_by_size; group_idx++) { - auto &group = grouped_aggregate_data.groups[group_idx]; - auto &bound_ref_expr = group->Cast(); - group_chunk.data[bound_ref_expr.index].Reference(output_chunk.data[group_idx]); - } - group_chunk.SetCardinality(output_chunk); - - for (idx_t child_idx = 0; child_idx < grouped_aggregate_data.groups.size() - group_by_size; - child_idx++) { - aggregate_input_chunk.data[payload_idx + child_idx].Reference( - output_chunk.data[group_by_size + child_idx]); - } - aggregate_input_chunk.SetCardinality(output_chunk); - - // Sink it into the main ht - OperatorSinkInput sink_input {table_state, *temp_local_state, interrupt_state}; - grouping_data.table_data.Sink(temp_exec_context, group_chunk, sink_input, aggregate_input_chunk, {i}); - } + D_ASSERT(distinct_data.info.table_map.count(agg_idx)); + + auto table_idx = distinct_data.info.table_map.at(agg_idx); + auto &radix_table_p = distinct_data.radix_tables[table_idx]; + aggregate_sources.push_back(radix_table_p->GetGlobalSourceState(context)); } - grouping_data.table_data.Combine(temp_exec_context, table_state, *temp_local_state); + global_source_states.push_back(std::move(aggregate_sources)); } +} - TaskExecutionResult ExecuteTask(TaskExecutionMode mode) override { - D_ASSERT(op.distinct_collection_info); - auto &info = *op.distinct_collection_info; - for (idx_t i = 0; i < op.groupings.size(); i++) { - auto &grouping = op.groupings[i]; - auto &grouping_state = gstate.grouping_states[i]; - AggregateDistinctGrouping(info, grouping, grouping_state, i); - } - event->FinishTask(); - return TaskExecutionResult::TASK_FINISHED; +void HashAggregateDistinctFinalizeEvent::FinishEvent() { + // Now that everything is added to the main ht, we can actually finalize + auto new_event = make_shared(context, pipeline.get(), op, gstate); + this->InsertEvent(std::move(new_event)); +} + +TaskExecutionResult HashAggregateDistinctFinalizeTask::ExecuteTask(TaskExecutionMode mode) { + for (idx_t grouping_idx = 0; grouping_idx < op.groupings.size(); grouping_idx++) { + AggregateDistinctGrouping(grouping_idx); } + event->FinishTask(); + return TaskExecutionResult::TASK_FINISHED; +} -private: - Pipeline &pipeline; - shared_ptr event; - HashAggregateGlobalState &gstate; - ClientContext &context; - const PhysicalHashAggregate &op; - vector>> &global_sources; -}; +void HashAggregateDistinctFinalizeTask::AggregateDistinctGrouping(const idx_t grouping_idx) { + D_ASSERT(op.distinct_collection_info); + auto &info = *op.distinct_collection_info; -//! DISTINCT FINALIZE EVENT + auto &grouping_data = op.groupings[grouping_idx]; + auto &grouping_state = gstate.grouping_states[grouping_idx]; + D_ASSERT(grouping_state.distinct_state); + auto &distinct_state = *grouping_state.distinct_state; + auto &distinct_data = *grouping_data.distinct_data; -// TODO: Create tasks and run these in parallel instead of doing this all in Schedule, single threaded -class HashDistinctAggregateFinalizeEvent : public BasePipelineEvent { -public: - HashDistinctAggregateFinalizeEvent(const PhysicalHashAggregate &op_p, HashAggregateGlobalState &gstate_p, - Pipeline &pipeline_p, ClientContext &context) - : BasePipelineEvent(pipeline_p), op(op_p), gstate(gstate_p), context(context) { - } - const PhysicalHashAggregate &op; - HashAggregateGlobalState &gstate; - ClientContext &context; - //! The GlobalSourceStates for all the radix tables of the distinct aggregates - vector>> global_sources; + auto &aggregates = info.aggregates; -public: - void Schedule() override { - global_sources = CreateGlobalSources(); - - vector> tasks; - auto &scheduler = TaskScheduler::GetScheduler(context); - auto number_of_threads = scheduler.NumberOfThreads(); - tasks.reserve(number_of_threads); - for (int32_t i = 0; i < number_of_threads; i++) { - tasks.push_back(make_uniq(*pipeline, shared_from_this(), gstate, context, - op, global_sources)); - } - D_ASSERT(!tasks.empty()); - SetTasks(std::move(tasks)); + // Thread-local contexts + ThreadContext thread_context(executor.context); + ExecutionContext execution_context(executor.context, thread_context, &pipeline); + + // Sink state to sink into global HTs + InterruptState interrupt_state; + auto &global_sink_state = *grouping_state.table_state; + auto local_sink_state = grouping_data.table_data.GetLocalSinkState(execution_context); + OperatorSinkInput sink_input {global_sink_state, *local_sink_state, interrupt_state}; + + // Create a chunk that mimics the 'input' chunk in Sink, for storing the group vectors + DataChunk group_chunk; + if (!op.input_group_types.empty()) { + group_chunk.Initialize(executor.context, op.input_group_types); } - void FinishEvent() override { - //! Now that everything is added to the main ht, we can actually finalize - auto new_event = make_shared(op, gstate, pipeline.get(), context); - this->InsertEvent(std::move(new_event)); + auto &groups = op.grouped_aggregate_data.groups; + const idx_t group_by_size = groups.size(); + + DataChunk aggregate_input_chunk; + if (!gstate.payload_types.empty()) { + aggregate_input_chunk.Initialize(executor.context, gstate.payload_types); } -private: - vector>> CreateGlobalSources() { - vector>> grouping_sources; - grouping_sources.reserve(op.groupings.size()); - for (idx_t grouping_idx = 0; grouping_idx < op.groupings.size(); grouping_idx++) { - auto &grouping = op.groupings[grouping_idx]; - auto &data = *grouping.distinct_data; - - vector> aggregate_sources; - aggregate_sources.reserve(op.grouped_aggregate_data.aggregates.size()); - - for (idx_t i = 0; i < op.grouped_aggregate_data.aggregates.size(); i++) { - auto &aggregate = op.grouped_aggregate_data.aggregates[i]; - auto &aggr = aggregate->Cast(); - - if (!aggr.IsDistinct()) { - aggregate_sources.push_back(nullptr); - continue; - } - - D_ASSERT(data.info.table_map.count(i)); - auto table_idx = data.info.table_map.at(i); - auto &radix_table_p = data.radix_tables[table_idx]; - aggregate_sources.push_back(radix_table_p->GetGlobalSourceState(context)); - } - grouping_sources.push_back(std::move(aggregate_sources)); + auto &finalize_event = event->Cast(); + + idx_t payload_idx; + idx_t next_payload_idx = 0; + for (idx_t agg_idx = 0; agg_idx < op.grouped_aggregate_data.aggregates.size(); agg_idx++) { + auto &aggregate = aggregates[agg_idx]->Cast(); + + // Forward the payload idx + payload_idx = next_payload_idx; + next_payload_idx = payload_idx + aggregate.children.size(); + + // If aggregate is not distinct, skip it + if (!distinct_data.IsDistinct(agg_idx)) { + continue; } - return grouping_sources; - } -}; -//! DISTINCT COMBINE EVENT + D_ASSERT(distinct_data.info.table_map.count(agg_idx)); + const auto &table_idx = distinct_data.info.table_map.at(agg_idx); + auto &radix_table = distinct_data.radix_tables[table_idx]; -class HashDistinctCombineFinalizeEvent : public BasePipelineEvent { -public: - HashDistinctCombineFinalizeEvent(const PhysicalHashAggregate &op_p, HashAggregateGlobalState &gstate_p, - Pipeline &pipeline_p, ClientContext &client) - : BasePipelineEvent(pipeline_p), op(op_p), gstate(gstate_p), client(client) { - } + auto &sink = *distinct_state.radix_states[table_idx]; + auto local_source = radix_table->GetLocalSourceState(execution_context); + OperatorSourceInput source_input {*finalize_event.global_source_states[grouping_idx][agg_idx], *local_source, + interrupt_state}; - const PhysicalHashAggregate &op; - HashAggregateGlobalState &gstate; - ClientContext &client; + // Create a duplicate of the output_chunk, because of multi-threading we cant alter the original + DataChunk output_chunk; + output_chunk.Initialize(executor.context, distinct_state.distinct_output_chunks[table_idx]->GetTypes()); + + // Fetch all the data from the aggregate ht, and Sink it into the main ht + while (true) { + output_chunk.Reset(); + group_chunk.Reset(); + aggregate_input_chunk.Reset(); + + auto res = radix_table->GetData(execution_context, output_chunk, sink, source_input); + if (res == SourceResultType::FINISHED) { + D_ASSERT(output_chunk.size() == 0); + break; + } else if (res == SourceResultType::BLOCKED) { + throw InternalException( + "Unexpected interrupt from radix table GetData in HashAggregateDistinctFinalizeTask"); + } -public: - void Schedule() override { - vector> tasks; - for (idx_t i = 0; i < op.groupings.size(); i++) { - auto &grouping = op.groupings[i]; - auto &distinct_data = *grouping.distinct_data; - auto &distinct_state = *gstate.grouping_states[i].distinct_state; - for (idx_t table_idx = 0; table_idx < distinct_data.radix_tables.size(); table_idx++) { - if (!distinct_data.radix_tables[table_idx]) { - continue; - } - distinct_data.radix_tables[table_idx]->ScheduleTasks(pipeline->executor, shared_from_this(), - *distinct_state.radix_states[table_idx], tasks); + auto &grouped_aggregate_data = *distinct_data.grouped_aggregate_data[table_idx]; + for (idx_t group_idx = 0; group_idx < group_by_size; group_idx++) { + auto &group = grouped_aggregate_data.groups[group_idx]; + auto &bound_ref_expr = group->Cast(); + group_chunk.data[bound_ref_expr.index].Reference(output_chunk.data[group_idx]); } - } + group_chunk.SetCardinality(output_chunk); - D_ASSERT(!tasks.empty()); - SetTasks(std::move(tasks)); - } + for (idx_t child_idx = 0; child_idx < grouped_aggregate_data.groups.size() - group_by_size; child_idx++) { + aggregate_input_chunk.data[payload_idx + child_idx].Reference( + output_chunk.data[group_by_size + child_idx]); + } + aggregate_input_chunk.SetCardinality(output_chunk); - void FinishEvent() override { - //! Now that all tables are combined, it's time to do the distinct aggregations - auto new_event = make_shared(op, gstate, *pipeline, client); - this->InsertEvent(std::move(new_event)); + // Sink it into the main ht + grouping_data.table_data.Sink(execution_context, group_chunk, sink_input, aggregate_input_chunk, {agg_idx}); + } } -}; - -//! FINALIZE + grouping_data.table_data.Combine(execution_context, global_sink_state, *local_sink_state); +} SinkFinalizeType PhysicalHashAggregate::FinalizeDistinct(Pipeline &pipeline, Event &event, ClientContext &context, GlobalSinkState &gstate_p) const { - auto &gstate = gstate_p.Cast(); + auto &gstate = gstate_p.Cast(); D_ASSERT(distinct_collection_info); - bool any_partitioned = false; for (idx_t i = 0; i < groupings.size(); i++) { auto &grouping = groupings[i]; auto &distinct_data = *grouping.distinct_data; @@ -773,28 +724,17 @@ SinkFinalizeType PhysicalHashAggregate::FinalizeDistinct(Pipeline &pipeline, Eve } auto &radix_table = distinct_data.radix_tables[table_idx]; auto &radix_state = *distinct_state.radix_states[table_idx]; - bool partitioned = radix_table->Finalize(context, radix_state); - if (partitioned) { - any_partitioned = true; - } + radix_table->Finalize(context, radix_state); } } - if (any_partitioned) { - // If any of the groupings are partitioned then we first need to combine those, then aggregate - auto new_event = make_shared(*this, gstate, pipeline, context); - event.InsertEvent(std::move(new_event)); - } else { - // Hashtables aren't partitioned, they dont need to be joined first - // so we can already compute the aggregate - auto new_event = make_shared(*this, gstate, pipeline, context); - event.InsertEvent(std::move(new_event)); - } + auto new_event = make_shared(context, pipeline, *this, gstate); + event.InsertEvent(std::move(new_event)); return SinkFinalizeType::READY; } SinkFinalizeType PhysicalHashAggregate::FinalizeInternal(Pipeline &pipeline, Event &event, ClientContext &context, GlobalSinkState &gstate_p, bool check_distinct) const { - auto &gstate = gstate_p.Cast(); + auto &gstate = gstate_p.Cast(); if (check_distinct && distinct_collection_info) { // There are distinct aggregates @@ -803,19 +743,10 @@ SinkFinalizeType PhysicalHashAggregate::FinalizeInternal(Pipeline &pipeline, Eve return FinalizeDistinct(pipeline, event, context, gstate_p); } - bool any_partitioned = false; for (idx_t i = 0; i < groupings.size(); i++) { auto &grouping = groupings[i]; auto &grouping_gstate = gstate.grouping_states[i]; - - bool is_partitioned = grouping.table_data.Finalize(context, *grouping_gstate.table_state); - if (is_partitioned) { - any_partitioned = true; - } - } - if (any_partitioned) { - auto new_event = make_shared(*this, gstate, &pipeline); - event.InsertEvent(std::move(new_event)); + grouping.table_data.Finalize(context, *grouping_gstate.table_state); } return SinkFinalizeType::READY; } @@ -828,10 +759,9 @@ SinkFinalizeType PhysicalHashAggregate::Finalize(Pipeline &pipeline, Event &even //===--------------------------------------------------------------------===// // Source //===--------------------------------------------------------------------===// -class PhysicalHashAggregateGlobalSourceState : public GlobalSourceState { +class HashAggregateGlobalSourceState : public GlobalSourceState { public: - PhysicalHashAggregateGlobalSourceState(ClientContext &context, const PhysicalHashAggregate &op) - : op(op), state_index(0) { + HashAggregateGlobalSourceState(ClientContext &context, const PhysicalHashAggregate &op) : op(op), state_index(0) { for (auto &grouping : op.groupings) { auto &rt = grouping.table_data; radix_states.push_back(rt.GetGlobalSourceState(context)); @@ -851,24 +781,24 @@ class PhysicalHashAggregateGlobalSourceState : public GlobalSourceState { return 1; } - auto &ht_state = op.sink_state->Cast(); + auto &ht_state = op.sink_state->Cast(); idx_t count = 0; for (size_t sidx = 0; sidx < op.groupings.size(); ++sidx) { auto &grouping = op.groupings[sidx]; auto &grouping_gstate = ht_state.grouping_states[sidx]; - count += grouping.table_data.Size(*grouping_gstate.table_state); + count += grouping.table_data.Count(*grouping_gstate.table_state); } return MaxValue(1, count / STANDARD_VECTOR_SIZE); } }; unique_ptr PhysicalHashAggregate::GetGlobalSourceState(ClientContext &context) const { - return make_uniq(context, *this); + return make_uniq(context, *this); } -class PhysicalHashAggregateLocalSourceState : public LocalSourceState { +class HashAggregateLocalSourceState : public LocalSourceState { public: - explicit PhysicalHashAggregateLocalSourceState(ExecutionContext &context, const PhysicalHashAggregate &op) { + explicit HashAggregateLocalSourceState(ExecutionContext &context, const PhysicalHashAggregate &op) { for (auto &grouping : op.groupings) { auto &rt = grouping.table_data; radix_states.push_back(rt.GetLocalSourceState(context)); @@ -880,14 +810,14 @@ class PhysicalHashAggregateLocalSourceState : public LocalSourceState { unique_ptr PhysicalHashAggregate::GetLocalSourceState(ExecutionContext &context, GlobalSourceState &gstate) const { - return make_uniq(context, *this); + return make_uniq(context, *this); } SourceResultType PhysicalHashAggregate::GetData(ExecutionContext &context, DataChunk &chunk, OperatorSourceInput &input) const { - auto &sink_gstate = sink_state->Cast(); - auto &gstate = input.global_state.Cast(); - auto &lstate = input.local_state.Cast(); + auto &sink_gstate = sink_state->Cast(); + auto &gstate = input.global_state.Cast(); + auto &lstate = input.local_state.Cast(); while (true) { idx_t radix_idx = gstate.state_index; if (radix_idx >= groupings.size()) { diff --git a/src/duckdb/src/execution/operator/aggregate/physical_ungrouped_aggregate.cpp b/src/duckdb/src/execution/operator/aggregate/physical_ungrouped_aggregate.cpp index 0f9e4b766..8db0ede31 100644 --- a/src/duckdb/src/execution/operator/aggregate/physical_ungrouped_aggregate.cpp +++ b/src/duckdb/src/execution/operator/aggregate/physical_ungrouped_aggregate.cpp @@ -37,7 +37,9 @@ PhysicalUngroupedAggregate::PhysicalUngroupedAggregate(vector types //===--------------------------------------------------------------------===// struct AggregateState { explicit AggregateState(const vector> &aggregate_expressions) { - for (auto &aggregate : aggregate_expressions) { + counts = make_uniq_array>(aggregate_expressions.size()); + for (idx_t i = 0; i < aggregate_expressions.size(); i++) { + auto &aggregate = aggregate_expressions[i]; D_ASSERT(aggregate->GetExpressionClass() == ExpressionClass::BOUND_AGGREGATE); auto &aggr = aggregate->Cast(); auto state = make_unsafe_uniq_array(aggr.function.state_size()); @@ -46,7 +48,7 @@ struct AggregateState { bind_data.push_back(aggr.bind_info.get()); destructors.push_back(aggr.function.destructor); #ifdef DEBUG - counts.push_back(0); + counts[i] = 0; #endif } } @@ -77,12 +79,12 @@ struct AggregateState { //! The destructors vector destructors; //! Counts (used for verification) - vector counts; + unique_array> counts; }; -class UngroupedAggregateGlobalState : public GlobalSinkState { +class UngroupedAggregateGlobalSinkState : public GlobalSinkState { public: - UngroupedAggregateGlobalState(const PhysicalUngroupedAggregate &op, ClientContext &client) + UngroupedAggregateGlobalSinkState(const PhysicalUngroupedAggregate &op, ClientContext &client) : state(op.aggregates), finished(false), allocator(BufferAllocator::Get(client)) { if (op.distinct_data) { distinct_state = make_uniq(*op.distinct_data, client); @@ -101,13 +103,13 @@ class UngroupedAggregateGlobalState : public GlobalSinkState { ArenaAllocator allocator; }; -class UngroupedAggregateLocalState : public LocalSinkState { +class UngroupedAggregateLocalSinkState : public LocalSinkState { public: - UngroupedAggregateLocalState(const PhysicalUngroupedAggregate &op, const vector &child_types, - GlobalSinkState &gstate_p, ExecutionContext &context) + UngroupedAggregateLocalSinkState(const PhysicalUngroupedAggregate &op, const vector &child_types, + GlobalSinkState &gstate_p, ExecutionContext &context) : allocator(BufferAllocator::Get(context.client)), state(op.aggregates), child_executor(context.client), aggregate_input_chunk(), filter_set() { - auto &gstate = gstate_p.Cast(); + auto &gstate = gstate_p.Cast(); auto &allocator = BufferAllocator::Get(context.client); InitializeDistinctAggregates(op, gstate, context); @@ -147,8 +149,8 @@ class UngroupedAggregateLocalState : public LocalSinkState { void Reset() { aggregate_input_chunk.Reset(); } - void InitializeDistinctAggregates(const PhysicalUngroupedAggregate &op, const UngroupedAggregateGlobalState &gstate, - ExecutionContext &context) { + void InitializeDistinctAggregates(const PhysicalUngroupedAggregate &op, + const UngroupedAggregateGlobalSinkState &gstate, ExecutionContext &context) { if (!op.distinct_data) { return; @@ -185,19 +187,19 @@ bool PhysicalUngroupedAggregate::SinkOrderDependent() const { } unique_ptr PhysicalUngroupedAggregate::GetGlobalSinkState(ClientContext &context) const { - return make_uniq(*this, context); + return make_uniq(*this, context); } unique_ptr PhysicalUngroupedAggregate::GetLocalSinkState(ExecutionContext &context) const { D_ASSERT(sink_state); auto &gstate = *sink_state; - return make_uniq(*this, children[0]->GetTypes(), gstate, context); + return make_uniq(*this, children[0]->GetTypes(), gstate, context); } void PhysicalUngroupedAggregate::SinkDistinct(ExecutionContext &context, DataChunk &chunk, OperatorSinkInput &input) const { - auto &sink = input.local_state.Cast(); - auto &global_sink = input.global_state.Cast(); + auto &sink = input.local_state.Cast(); + auto &global_sink = input.global_state.Cast(); D_ASSERT(distinct_data); auto &distinct_state = *global_sink.distinct_state; auto &distinct_info = *distinct_collection_info; @@ -239,7 +241,7 @@ void PhysicalUngroupedAggregate::SinkDistinct(ExecutionContext &context, DataChu SinkResultType PhysicalUngroupedAggregate::Sink(ExecutionContext &context, DataChunk &chunk, OperatorSinkInput &input) const { - auto &sink = input.local_state.Cast(); + auto &sink = input.local_state.Cast(); // perform the aggregation inside the local state sink.Reset(); @@ -296,23 +298,22 @@ SinkResultType PhysicalUngroupedAggregate::Sink(ExecutionContext &context, DataC } //===--------------------------------------------------------------------===// -// Finalize +// Combine //===--------------------------------------------------------------------===// - void PhysicalUngroupedAggregate::CombineDistinct(ExecutionContext &context, OperatorSinkCombineInput &input) const { - auto &global_sink = input.global_state.Cast(); - auto &source = input.local_state.Cast(); + auto &gstate = input.global_state.Cast(); + auto &lstate = input.local_state.Cast(); if (!distinct_data) { return; } - auto &distinct_state = global_sink.distinct_state; + auto &distinct_state = gstate.distinct_state; auto table_count = distinct_data->radix_tables.size(); for (idx_t table_idx = 0; table_idx < table_count; table_idx++) { D_ASSERT(distinct_data->radix_tables[table_idx]); auto &radix_table = *distinct_data->radix_tables[table_idx]; auto &radix_global_sink = *distinct_state->radix_states[table_idx]; - auto &radix_local_sink = *source.radix_states[table_idx]; + auto &radix_local_sink = *lstate.radix_states[table_idx]; radix_table.Combine(context, radix_global_sink, radix_local_sink); } @@ -320,18 +321,17 @@ void PhysicalUngroupedAggregate::CombineDistinct(ExecutionContext &context, Oper SinkCombineResultType PhysicalUngroupedAggregate::Combine(ExecutionContext &context, OperatorSinkCombineInput &input) const { - auto &gstate = input.global_state.Cast(); - auto &source = input.local_state.Cast(); + auto &gstate = input.global_state.Cast(); + auto &lstate = input.local_state.Cast(); D_ASSERT(!gstate.finished); // finalize: combine the local state into the global state // all aggregates are combinable: we might be doing a parallel aggregate // use the combine method to combine the partial aggregates - lock_guard glock(gstate.lock); - - OperatorSinkCombineInput distinct_input {gstate, source, input.interrupt_state}; + OperatorSinkCombineInput distinct_input {gstate, lstate, input.interrupt_state}; CombineDistinct(context, distinct_input); + lock_guard glock(gstate.lock); for (idx_t aggr_idx = 0; aggr_idx < aggregates.size(); aggr_idx++) { auto &aggregate = aggregates[aggr_idx]->Cast(); @@ -339,207 +339,236 @@ SinkCombineResultType PhysicalUngroupedAggregate::Combine(ExecutionContext &cont continue; } - Vector source_state(Value::POINTER(CastPointerToValue(source.state.aggregates[aggr_idx].get()))); + Vector source_state(Value::POINTER(CastPointerToValue(lstate.state.aggregates[aggr_idx].get()))); Vector dest_state(Value::POINTER(CastPointerToValue(gstate.state.aggregates[aggr_idx].get()))); AggregateInputData aggr_input_data(aggregate.bind_info.get(), gstate.allocator); aggregate.function.combine(source_state, dest_state, aggr_input_data, 1); #ifdef DEBUG - gstate.state.counts[aggr_idx] += source.state.counts[aggr_idx]; + gstate.state.counts[aggr_idx] += lstate.state.counts[aggr_idx]; #endif } - source.allocator.Destroy(); + lstate.allocator.Destroy(); auto &client_profiler = QueryProfiler::Get(context.client); - context.thread.profiler.Flush(*this, source.child_executor, "child_executor", 0); + context.thread.profiler.Flush(*this, lstate.child_executor, "child_executor", 0); client_profiler.Flush(context.thread.profiler); return SinkCombineResultType::FINISHED; } -class UngroupedDistinctAggregateFinalizeTask : public ExecutorTask { +//===--------------------------------------------------------------------===// +// Finalize +//===--------------------------------------------------------------------===// +class UngroupedDistinctAggregateFinalizeEvent : public BasePipelineEvent { public: - UngroupedDistinctAggregateFinalizeTask(Executor &executor, shared_ptr event_p, - UngroupedAggregateGlobalState &state_p, ClientContext &context, - const PhysicalUngroupedAggregate &op) - : ExecutorTask(executor), event(std::move(event_p)), gstate(state_p), context(context), op(op), - allocator(BufferAllocator::Get(context)) { + UngroupedDistinctAggregateFinalizeEvent(ClientContext &context, const PhysicalUngroupedAggregate &op_p, + UngroupedAggregateGlobalSinkState &gstate_p, Pipeline &pipeline_p) + : BasePipelineEvent(pipeline_p), context(context), op(op_p), gstate(gstate_p), tasks_scheduled(0), + tasks_done(0) { } - void AggregateDistinct() { - D_ASSERT(gstate.distinct_state); - auto &aggregates = op.aggregates; - auto &distinct_state = *gstate.distinct_state; - auto &distinct_data = *op.distinct_data; - - ThreadContext temp_thread_context(context); - ExecutionContext temp_exec_context(context, temp_thread_context, nullptr); - - idx_t payload_idx = 0; - idx_t next_payload_idx = 0; +public: + void Schedule() override; - for (idx_t agg_idx = 0; agg_idx < aggregates.size(); agg_idx++) { - auto &aggregate = aggregates[agg_idx]->Cast(); +private: + ClientContext &context; - // Forward the payload idx - payload_idx = next_payload_idx; - next_payload_idx = payload_idx + aggregate.children.size(); + const PhysicalUngroupedAggregate &op; + UngroupedAggregateGlobalSinkState &gstate; - // If aggregate is not distinct, skip it - if (!distinct_data.IsDistinct(agg_idx)) { - continue; - } +public: + mutex lock; + idx_t tasks_scheduled; + idx_t tasks_done; - DataChunk payload_chunk; - - D_ASSERT(distinct_data.info.table_map.count(agg_idx)); - auto table_idx = distinct_data.info.table_map.at(agg_idx); - auto &radix_table_p = distinct_data.radix_tables[table_idx]; - auto &output_chunk = *distinct_state.distinct_output_chunks[table_idx]; - auto &grouped_aggregate_data = *distinct_data.grouped_aggregate_data[table_idx]; - - payload_chunk.InitializeEmpty(grouped_aggregate_data.group_types); - payload_chunk.SetCardinality(0); - - //! Create global and local state for the hashtable - auto global_source_state = radix_table_p->GetGlobalSourceState(context); - auto local_source_state = radix_table_p->GetLocalSourceState(temp_exec_context); - - //! Retrieve the stored data from the hashtable - while (true) { - output_chunk.Reset(); - - InterruptState interrupt_state; - OperatorSourceInput source_input {*global_source_state, *local_source_state, interrupt_state}; - auto res = radix_table_p->GetData(temp_exec_context, output_chunk, - *distinct_state.radix_states[table_idx], source_input); - if (res == SourceResultType::FINISHED) { - D_ASSERT(output_chunk.size() == 0); - break; - } else if (res == SourceResultType::BLOCKED) { - throw InternalException( - "Unexpected interrupt from radix table GetData in UngroupedDistinctAggregateFinalizeTask"); - } - - // We dont need to resolve the filter, we already did this in Sink - idx_t payload_cnt = aggregate.children.size(); - for (idx_t i = 0; i < payload_cnt; i++) { - payload_chunk.data[i].Reference(output_chunk.data[i]); - } - payload_chunk.SetCardinality(output_chunk); -#ifdef DEBUG - gstate.state.counts[agg_idx] += payload_chunk.size(); -#endif + vector> global_source_states; +}; - auto start_of_input = payload_cnt ? &payload_chunk.data[0] : nullptr; - //! Update the aggregate state - AggregateInputData aggr_input_data(aggregate.bind_info.get(), allocator); - aggregate.function.simple_update(start_of_input, aggr_input_data, payload_cnt, - gstate.state.aggregates[agg_idx].get(), payload_chunk.size()); - } - } - D_ASSERT(!gstate.finished); - gstate.finished = true; +class UngroupedDistinctAggregateFinalizeTask : public ExecutorTask { +public: + UngroupedDistinctAggregateFinalizeTask(Executor &executor, shared_ptr event_p, + const PhysicalUngroupedAggregate &op, + UngroupedAggregateGlobalSinkState &state_p) + : ExecutorTask(executor), event(std::move(event_p)), op(op), gstate(state_p), + allocator(BufferAllocator::Get(executor.context)) { } - TaskExecutionResult ExecuteTask(TaskExecutionMode mode) override { - AggregateDistinct(); - event->FinishTask(); - return TaskExecutionResult::TASK_FINISHED; - } + TaskExecutionResult ExecuteTask(TaskExecutionMode mode) override; + +private: + void AggregateDistinct(); private: shared_ptr event; - UngroupedAggregateGlobalState &gstate; - ClientContext &context; + const PhysicalUngroupedAggregate &op; + UngroupedAggregateGlobalSinkState &gstate; + ArenaAllocator allocator; }; -// TODO: Create tasks and run these in parallel instead of doing this all in Schedule, single threaded -class UngroupedDistinctAggregateFinalizeEvent : public BasePipelineEvent { -public: - UngroupedDistinctAggregateFinalizeEvent(const PhysicalUngroupedAggregate &op_p, - UngroupedAggregateGlobalState &gstate_p, Pipeline &pipeline_p, - ClientContext &context) - : BasePipelineEvent(pipeline_p), op(op_p), gstate(gstate_p), context(context) { - } - const PhysicalUngroupedAggregate &op; - UngroupedAggregateGlobalState &gstate; - ClientContext &context; +void UngroupedDistinctAggregateFinalizeEvent::Schedule() { + D_ASSERT(gstate.distinct_state); + auto &aggregates = op.aggregates; + auto &distinct_data = *op.distinct_data; -public: - void Schedule() override { - vector> tasks; - tasks.push_back(make_uniq(pipeline->executor, shared_from_this(), - gstate, context, op)); - D_ASSERT(!tasks.empty()); - SetTasks(std::move(tasks)); + idx_t payload_idx = 0; + idx_t next_payload_idx = 0; + for (idx_t agg_idx = 0; agg_idx < aggregates.size(); agg_idx++) { + auto &aggregate = aggregates[agg_idx]->Cast(); + + // Forward the payload idx + payload_idx = next_payload_idx; + next_payload_idx = payload_idx + aggregate.children.size(); + + // If aggregate is not distinct, skip it + if (!distinct_data.IsDistinct(agg_idx)) { + global_source_states.push_back(nullptr); + continue; + } + D_ASSERT(distinct_data.info.table_map.count(agg_idx)); + + // Create global state for scanning + auto table_idx = distinct_data.info.table_map.at(agg_idx); + auto &radix_table_p = *distinct_data.radix_tables[table_idx]; + global_source_states.push_back(radix_table_p.GetGlobalSourceState(context)); } -}; -class UngroupedDistinctCombineFinalizeEvent : public BasePipelineEvent { -public: - UngroupedDistinctCombineFinalizeEvent(const PhysicalUngroupedAggregate &op_p, - UngroupedAggregateGlobalState &gstate_p, Pipeline &pipeline_p, - ClientContext &client) - : BasePipelineEvent(pipeline_p), op(op_p), gstate(gstate_p), client(client) { + const idx_t n_threads = TaskScheduler::GetScheduler(context).NumberOfThreads(); + vector> tasks; + for (idx_t i = 0; i < n_threads; i++) { + tasks.push_back( + make_uniq(pipeline->executor, shared_from_this(), op, gstate)); + tasks_scheduled++; } + SetTasks(std::move(tasks)); +} - const PhysicalUngroupedAggregate &op; - UngroupedAggregateGlobalState &gstate; - ClientContext &client; +TaskExecutionResult UngroupedDistinctAggregateFinalizeTask::ExecuteTask(TaskExecutionMode mode) { + AggregateDistinct(); + event->FinishTask(); + return TaskExecutionResult::TASK_FINISHED; +} -public: - void Schedule() override { - auto &distinct_state = *gstate.distinct_state; - auto &distinct_data = *op.distinct_data; - vector> tasks; - for (idx_t table_idx = 0; table_idx < distinct_data.radix_tables.size(); table_idx++) { - distinct_data.radix_tables[table_idx]->ScheduleTasks(pipeline->executor, shared_from_this(), - *distinct_state.radix_states[table_idx], tasks); +void UngroupedDistinctAggregateFinalizeTask::AggregateDistinct() { + D_ASSERT(gstate.distinct_state); + auto &distinct_state = *gstate.distinct_state; + auto &distinct_data = *op.distinct_data; + + // Create thread-local copy of aggregate state + auto &aggregates = op.aggregates; + AggregateState state(aggregates); + + // Thread-local contexts + ThreadContext thread_context(executor.context); + ExecutionContext execution_context(executor.context, thread_context, nullptr); + + auto &finalize_event = event->Cast(); + + // Now loop through the distinct aggregates, scanning the distinct HTs + idx_t payload_idx = 0; + idx_t next_payload_idx = 0; + for (idx_t agg_idx = 0; agg_idx < aggregates.size(); agg_idx++) { + auto &aggregate = aggregates[agg_idx]->Cast(); + + // Forward the payload idx + payload_idx = next_payload_idx; + next_payload_idx = payload_idx + aggregate.children.size(); + + // If aggregate is not distinct, skip it + if (!distinct_data.IsDistinct(agg_idx)) { + continue; + } + + const auto table_idx = distinct_data.info.table_map.at(agg_idx); + auto &radix_table = *distinct_data.radix_tables[table_idx]; + auto lstate = radix_table.GetLocalSourceState(execution_context); + + auto &sink = *distinct_state.radix_states[table_idx]; + InterruptState interrupt_state; + OperatorSourceInput source_input {*finalize_event.global_source_states[agg_idx], *lstate, interrupt_state}; + + DataChunk output_chunk; + output_chunk.Initialize(executor.context, distinct_state.distinct_output_chunks[table_idx]->GetTypes()); + + DataChunk payload_chunk; + payload_chunk.InitializeEmpty(distinct_data.grouped_aggregate_data[table_idx]->group_types); + payload_chunk.SetCardinality(0); + + AggregateInputData aggr_input_data(aggregate.bind_info.get(), allocator); + while (true) { + output_chunk.Reset(); + + auto res = radix_table.GetData(execution_context, output_chunk, sink, source_input); + if (res == SourceResultType::FINISHED) { + D_ASSERT(output_chunk.size() == 0); + break; + } else if (res == SourceResultType::BLOCKED) { + throw InternalException( + "Unexpected interrupt from radix table GetData in UngroupedDistinctAggregateFinalizeTask"); + } + + // We dont need to resolve the filter, we already did this in Sink + idx_t payload_cnt = aggregate.children.size(); + for (idx_t i = 0; i < payload_cnt; i++) { + payload_chunk.data[i].Reference(output_chunk.data[i]); + } + payload_chunk.SetCardinality(output_chunk); + +#ifdef DEBUG + gstate.state.counts[agg_idx] += payload_chunk.size(); +#endif + + // Update the aggregate state + auto start_of_input = payload_cnt ? &payload_chunk.data[0] : nullptr; + aggregate.function.simple_update(start_of_input, aggr_input_data, payload_cnt, + state.aggregates[agg_idx].get(), payload_chunk.size()); } - D_ASSERT(!tasks.empty()); - SetTasks(std::move(tasks)); } - void FinishEvent() override { - //! Now that all tables are combined, it's time to do the distinct aggregations - auto new_event = make_shared(op, gstate, *pipeline, client); - this->InsertEvent(std::move(new_event)); + // After scanning the distinct HTs, we can combine the thread-local agg states with the thread-global + lock_guard guard(finalize_event.lock); + payload_idx = 0; + next_payload_idx = 0; + for (idx_t agg_idx = 0; agg_idx < aggregates.size(); agg_idx++) { + if (!distinct_data.IsDistinct(agg_idx)) { + continue; + } + + auto &aggregate = aggregates[agg_idx]->Cast(); + AggregateInputData aggr_input_data(aggregate.bind_info.get(), allocator); + + Vector state_vec(Value::POINTER(CastPointerToValue(state.aggregates[agg_idx].get()))); + Vector combined_vec(Value::POINTER(CastPointerToValue(gstate.state.aggregates[agg_idx].get()))); + aggregate.function.combine(state_vec, combined_vec, aggr_input_data, 1); } -}; + + D_ASSERT(!gstate.finished); + if (++finalize_event.tasks_done == finalize_event.tasks_scheduled) { + gstate.finished = true; + } +} SinkFinalizeType PhysicalUngroupedAggregate::FinalizeDistinct(Pipeline &pipeline, Event &event, ClientContext &context, GlobalSinkState &gstate_p) const { - auto &gstate = gstate_p.Cast(); + auto &gstate = gstate_p.Cast(); D_ASSERT(distinct_data); auto &distinct_state = *gstate.distinct_state; - bool any_partitioned = false; for (idx_t table_idx = 0; table_idx < distinct_data->radix_tables.size(); table_idx++) { auto &radix_table_p = distinct_data->radix_tables[table_idx]; auto &radix_state = *distinct_state.radix_states[table_idx]; - bool partitioned = radix_table_p->Finalize(context, radix_state); - if (partitioned) { - any_partitioned = true; - } - } - if (any_partitioned) { - auto new_event = make_shared(*this, gstate, pipeline, context); - event.InsertEvent(std::move(new_event)); - } else { - //! Hashtables aren't partitioned, they dont need to be joined first - //! So we can compute the aggregate already - auto new_event = make_shared(*this, gstate, pipeline, context); - event.InsertEvent(std::move(new_event)); + radix_table_p->Finalize(context, radix_state); } + auto new_event = make_shared(context, *this, gstate, pipeline); + event.InsertEvent(std::move(new_event)); return SinkFinalizeType::READY; } SinkFinalizeType PhysicalUngroupedAggregate::Finalize(Pipeline &pipeline, Event &event, ClientContext &context, OperatorSinkFinalizeInput &input) const { - auto &gstate = input.global_state.Cast(); + auto &gstate = input.global_state.Cast(); if (distinct_data) { return FinalizeDistinct(pipeline, event, context, input.global_state); @@ -569,7 +598,7 @@ void VerifyNullHandling(DataChunk &chunk, AggregateState &state, const vectorCast(); + auto &gstate = sink_state->Cast(); D_ASSERT(gstate.finished); // initialize the result chunk with the aggregate values diff --git a/src/duckdb/src/execution/operator/aggregate/physical_window.cpp b/src/duckdb/src/execution/operator/aggregate/physical_window.cpp index 66e5dc666..cba3e5dcc 100644 --- a/src/duckdb/src/execution/operator/aggregate/physical_window.cpp +++ b/src/duckdb/src/execution/operator/aggregate/physical_window.cpp @@ -14,7 +14,6 @@ #include "duckdb/common/vector_operations/vector_operations.hpp" #include "duckdb/common/windows_undefs.hpp" #include "duckdb/execution/expression_executor.hpp" -#include "duckdb/execution/partitionable_hashtable.hpp" #include "duckdb/execution/window_executor.hpp" #include "duckdb/execution/window_segment_tree.hpp" #include "duckdb/main/client_config.hpp" @@ -222,6 +221,7 @@ WindowGlobalSourceState::WindowGlobalSourceState(ClientContext &context_p, Windo } } else { built.resize(hash_groups.size()); + idx_t batch_base = 0; for (auto &hash_group : hash_groups) { if (!hash_group) { continue; @@ -235,6 +235,9 @@ WindowGlobalSourceState::WindowGlobalSourceState(ClientContext &context_p, Windo auto &sb = *global_sort_state.sorted_blocks[0]; auto &sd = *sb.payload_data; tasks_remaining += sd.data_blocks.size(); + + hash_group->batch_base = batch_base; + batch_base += sd.data_blocks.size(); } } } @@ -436,6 +439,7 @@ class WindowLocalSourceState : public LocalSourceState { using ReadStates = vector; explicit WindowLocalSourceState(WindowGlobalSourceState &gsource); + void UpdateBatchIndex(); bool NextPartition(); void Scan(DataChunk &chunk); @@ -443,6 +447,8 @@ class WindowLocalSourceState : public LocalSourceState { WindowGlobalSourceState &gsource; //! The current bin being processed idx_t hash_bin; + //! The current batch index (for output reordering) + idx_t batch_index; //! The current source being processed optional_ptr partition_source; //! The read cursor @@ -456,7 +462,7 @@ class WindowLocalSourceState : public LocalSourceState { }; WindowLocalSourceState::WindowLocalSourceState(WindowGlobalSourceState &gsource) - : gsource(gsource), hash_bin(gsource.built.size()) { + : gsource(gsource), hash_bin(gsource.built.size()), batch_index(0) { auto &gsink = *gsource.gsink.global_partition; auto &op = gsource.gsink.op; @@ -564,6 +570,14 @@ WindowGlobalSourceState::Task WindowGlobalSourceState::NextTask(idx_t hash_bin) return Task(); } +void WindowLocalSourceState::UpdateBatchIndex() { + D_ASSERT(partition_source); + D_ASSERT(scanner.get()); + + batch_index = partition_source->hash_group ? partition_source->hash_group->batch_base : 0; + batch_index += scanner->BlockIndex(); +} + bool WindowLocalSourceState::NextPartition() { // Release old states before the source scanner.reset(); @@ -578,6 +592,7 @@ bool WindowLocalSourceState::NextPartition() { partition_source = task.first; scanner = std::move(task.second); hash_bin = partition_source->hash_bin; + UpdateBatchIndex(); } for (auto &wexec : partition_source->executors) { @@ -599,6 +614,8 @@ void WindowLocalSourceState::Scan(DataChunk &result) { read_states.clear(); return; } + + UpdateBatchIndex(); } const auto position = scanner->Scanned(); @@ -638,6 +655,23 @@ unique_ptr PhysicalWindow::GetGlobalSourceState(ClientContext return make_uniq(context, gsink); } +bool PhysicalWindow::SupportsBatchIndex() const { + // We can only preserve order for single partitioning + // or work stealing causes out of order batch numbers + auto &wexpr = select_list[0]->Cast(); + return wexpr.partitions.empty() && !wexpr.orders.empty(); +} + +OrderPreservationType PhysicalWindow::SourceOrder() const { + return SupportsBatchIndex() ? OrderPreservationType::FIXED_ORDER : OrderPreservationType::NO_ORDER; +} + +idx_t PhysicalWindow::GetBatchIndex(ExecutionContext &context, DataChunk &chunk, GlobalSourceState &gstate_p, + LocalSourceState &lstate_p) const { + auto &lstate = lstate_p.Cast(); + return lstate.batch_index; +} + SourceResultType PhysicalWindow::GetData(ExecutionContext &context, DataChunk &chunk, OperatorSourceInput &input) const { auto &lsource = input.local_state.Cast(); diff --git a/src/duckdb/src/execution/operator/csv_scanner/base_csv_reader.cpp b/src/duckdb/src/execution/operator/csv_scanner/base_csv_reader.cpp new file mode 100644 index 000000000..025486495 --- /dev/null +++ b/src/duckdb/src/execution/operator/csv_scanner/base_csv_reader.cpp @@ -0,0 +1,591 @@ +#include "duckdb/execution/operator/scan/csv/base_csv_reader.hpp" + +#include "duckdb/catalog/catalog_entry/table_catalog_entry.hpp" +#include "duckdb/common/file_system.hpp" +#include "duckdb/common/string_util.hpp" +#include "duckdb/common/to_string.hpp" +#include "duckdb/common/types/cast_helpers.hpp" +#include "duckdb/common/operator/cast_operators.hpp" +#include "duckdb/common/operator/decimal_cast_operators.hpp" +#include "duckdb/common/vector_operations/unary_executor.hpp" +#include "duckdb/common/vector_operations/vector_operations.hpp" +#include "duckdb/function/scalar/strftime_format.hpp" +#include "duckdb/main/appender.hpp" +#include "duckdb/main/database.hpp" +#include "duckdb/parser/column_definition.hpp" +#include "duckdb/storage/data_table.hpp" +#include "utf8proc_wrapper.hpp" +#include "utf8proc.hpp" +#include "duckdb/parser/keyword_helper.hpp" +#include "duckdb/main/error_manager.hpp" +#include "duckdb/execution/operator/scan/csv/parallel_csv_reader.hpp" +#include "duckdb/execution/operator/persistent/csv_rejects_table.hpp" +#include "duckdb/main/client_data.hpp" +#include +#include +#include +#include + +namespace duckdb { + +string BaseCSVReader::GetLineNumberStr(idx_t line_error, bool is_line_estimated, idx_t buffer_idx) { + // If an error happens during auto-detect it is an estimated line + string estimated = (is_line_estimated ? string(" (estimated)") : string("")); + return to_string(GetLineError(line_error, buffer_idx)) + estimated; +} + +BaseCSVReader::BaseCSVReader(ClientContext &context_p, CSVReaderOptions options_p, + const vector &requested_types) + : context(context_p), fs(FileSystem::GetFileSystem(context)), allocator(BufferAllocator::Get(context)), + options(std::move(options_p)) { +} + +BaseCSVReader::~BaseCSVReader() { +} + +unique_ptr BaseCSVReader::OpenCSV(ClientContext &context, const CSVReaderOptions &options_p) { + return CSVFileHandle::OpenFile(FileSystem::GetFileSystem(context), BufferAllocator::Get(context), + options_p.file_path, options_p.compression); +} + +void BaseCSVReader::InitParseChunk(idx_t num_cols) { + // adapt not null info + if (options.force_not_null.size() != num_cols) { + options.force_not_null.resize(num_cols, false); + } + if (num_cols == parse_chunk.ColumnCount()) { + parse_chunk.Reset(); + } else { + parse_chunk.Destroy(); + + // initialize the parse_chunk with a set of VARCHAR types + vector varchar_types(num_cols, LogicalType::VARCHAR); + parse_chunk.Initialize(allocator, varchar_types); + } +} + +void BaseCSVReader::InitializeProjection() { + for (idx_t i = 0; i < GetTypes().size(); i++) { + reader_data.column_ids.push_back(i); + reader_data.column_mapping.push_back(i); + } +} + +template +static bool TemplatedTryCastDateVector(map &options, Vector &input_vector, + Vector &result_vector, idx_t count, string &error_message, idx_t &line_error) { + D_ASSERT(input_vector.GetType().id() == LogicalTypeId::VARCHAR); + bool all_converted = true; + idx_t cur_line = 0; + UnaryExecutor::Execute(input_vector, result_vector, count, [&](string_t input) { + T result; + if (!OP::Operation(options, input, result, error_message)) { + line_error = cur_line; + all_converted = false; + } + cur_line++; + return result; + }); + return all_converted; +} + +struct TryCastDateOperator { + static bool Operation(map &options, string_t input, date_t &result, + string &error_message) { + return options[LogicalTypeId::DATE].TryParseDate(input, result, error_message); + } +}; + +struct TryCastTimestampOperator { + static bool Operation(map &options, string_t input, timestamp_t &result, + string &error_message) { + return options[LogicalTypeId::TIMESTAMP].TryParseTimestamp(input, result, error_message); + } +}; + +bool BaseCSVReader::TryCastDateVector(map &options, Vector &input_vector, + Vector &result_vector, idx_t count, string &error_message, idx_t &line_error) { + return TemplatedTryCastDateVector(options, input_vector, result_vector, count, + error_message, line_error); +} + +bool BaseCSVReader::TryCastTimestampVector(map &options, Vector &input_vector, + Vector &result_vector, idx_t count, string &error_message) { + idx_t line_error; + return TemplatedTryCastDateVector(options, input_vector, result_vector, + count, error_message, line_error); +} + +void BaseCSVReader::VerifyLineLength(idx_t line_size, idx_t buffer_idx) { + if (line_size > options.maximum_line_size) { + throw InvalidInputException( + "Error in file \"%s\" on line %s: Maximum line size of %llu bytes exceeded!", options.file_path, + GetLineNumberStr(parse_chunk.size(), linenr_estimated, buffer_idx).c_str(), options.maximum_line_size); + } +} + +template +bool TemplatedTryCastFloatingVector(CSVReaderOptions &options, Vector &input_vector, Vector &result_vector, idx_t count, + string &error_message, idx_t &line_error) { + D_ASSERT(input_vector.GetType().id() == LogicalTypeId::VARCHAR); + bool all_converted = true; + idx_t row = 0; + UnaryExecutor::Execute(input_vector, result_vector, count, [&](string_t input) { + T result; + if (!OP::Operation(input, result, &error_message)) { + line_error = row; + all_converted = false; + } else { + row++; + } + return result; + }); + return all_converted; +} + +template +bool TemplatedTryCastDecimalVector(CSVReaderOptions &options, Vector &input_vector, Vector &result_vector, idx_t count, + string &error_message, uint8_t width, uint8_t scale) { + D_ASSERT(input_vector.GetType().id() == LogicalTypeId::VARCHAR); + bool all_converted = true; + UnaryExecutor::Execute(input_vector, result_vector, count, [&](string_t input) { + T result; + if (!OP::Operation(input, result, &error_message, width, scale)) { + all_converted = false; + } + return result; + }); + return all_converted; +} + +void BaseCSVReader::AddValue(string_t str_val, idx_t &column, vector &escape_positions, bool has_quotes, + idx_t buffer_idx) { + auto length = str_val.GetSize(); + if (length == 0 && column == 0) { + row_empty = true; + } else { + row_empty = false; + } + if (!return_types.empty() && column == return_types.size() && length == 0) { + // skip a single trailing delimiter in last column + return; + } + if (column >= return_types.size()) { + if (options.ignore_errors) { + error_column_overflow = true; + return; + } else { + throw InvalidInputException( + "Error in file \"%s\", on line %s: expected %lld values per row, but got more. (%s)", options.file_path, + GetLineNumberStr(linenr, linenr_estimated, buffer_idx).c_str(), return_types.size(), + options.ToString()); + } + } + + // insert the line number into the chunk + idx_t row_entry = parse_chunk.size(); + + // test against null string, but only if the value was not quoted + if ((!(has_quotes && !options.allow_quoted_nulls) || return_types[column].id() != LogicalTypeId::VARCHAR) && + !options.force_not_null[column] && Equals::Operation(str_val, string_t(options.null_str))) { + FlatVector::SetNull(parse_chunk.data[column], row_entry, true); + } else { + auto &v = parse_chunk.data[column]; + auto parse_data = FlatVector::GetData(v); + if (!escape_positions.empty()) { + // remove escape characters (if any) + string old_val = str_val.GetString(); + string new_val = ""; + idx_t prev_pos = 0; + for (idx_t i = 0; i < escape_positions.size(); i++) { + idx_t next_pos = escape_positions[i]; + new_val += old_val.substr(prev_pos, next_pos - prev_pos); + prev_pos = ++next_pos; + } + new_val += old_val.substr(prev_pos, old_val.size() - prev_pos); + escape_positions.clear(); + parse_data[row_entry] = StringVector::AddStringOrBlob(v, string_t(new_val)); + } else { + parse_data[row_entry] = str_val; + } + } + + // move to the next column + column++; +} + +bool BaseCSVReader::AddRow(DataChunk &insert_chunk, idx_t &column, string &error_message, idx_t buffer_idx) { + linenr++; + + if (row_empty) { + row_empty = false; + if (return_types.size() != 1) { + if (mode == ParserMode::PARSING) { + FlatVector::SetNull(parse_chunk.data[0], parse_chunk.size(), false); + } + column = 0; + return false; + } + } + + // Error forwarded by 'ignore_errors' - originally encountered in 'AddValue' + if (error_column_overflow) { + D_ASSERT(options.ignore_errors); + error_column_overflow = false; + column = 0; + return false; + } + + if (column < return_types.size()) { + if (options.null_padding) { + for (; column < return_types.size(); column++) { + FlatVector::SetNull(parse_chunk.data[column], parse_chunk.size(), true); + } + } else if (options.ignore_errors) { + column = 0; + return false; + } else { + if (mode == ParserMode::SNIFFING_DATATYPES) { + error_message = "Error when adding line"; + return false; + } else { + throw InvalidInputException( + "Error in file \"%s\" on line %s: expected %lld values per row, but got %d.\nParser options:\n%s", + options.file_path, GetLineNumberStr(linenr, linenr_estimated, buffer_idx).c_str(), + return_types.size(), column, options.ToString()); + } + } + } + + parse_chunk.SetCardinality(parse_chunk.size() + 1); + + if (mode == ParserMode::PARSING_HEADER) { + return true; + } + + if (mode == ParserMode::SNIFFING_DATATYPES && parse_chunk.size() == options.sample_chunk_size) { + return true; + } + + if (mode == ParserMode::PARSING && parse_chunk.size() == STANDARD_VECTOR_SIZE) { + Flush(insert_chunk, buffer_idx); + return true; + } + + column = 0; + return false; +} + +void BaseCSVReader::VerifyUTF8(idx_t col_idx, idx_t row_idx, DataChunk &chunk, int64_t offset) { + D_ASSERT(col_idx < chunk.data.size()); + D_ASSERT(row_idx < chunk.size()); + auto &v = chunk.data[col_idx]; + if (FlatVector::IsNull(v, row_idx)) { + return; + } + + auto parse_data = FlatVector::GetData(chunk.data[col_idx]); + auto s = parse_data[row_idx]; + auto utf_type = Utf8Proc::Analyze(s.GetData(), s.GetSize()); + if (utf_type == UnicodeType::INVALID) { + string col_name = to_string(col_idx); + if (col_idx < names.size()) { + col_name = "\"" + names[col_idx] + "\""; + } + int64_t error_line = linenr - (chunk.size() - row_idx) + 1 + offset; + D_ASSERT(error_line >= 0); + throw InvalidInputException("Error in file \"%s\" at line %llu in column \"%s\": " + "%s. Parser options:\n%s", + options.file_path, error_line, col_name, + ErrorManager::InvalidUnicodeError(s.GetString(), "CSV file"), options.ToString()); + } +} + +void BaseCSVReader::VerifyUTF8(idx_t col_idx) { + D_ASSERT(col_idx < parse_chunk.data.size()); + for (idx_t i = 0; i < parse_chunk.size(); i++) { + VerifyUTF8(col_idx, i, parse_chunk); + } +} + +bool TryCastDecimalVectorCommaSeparated(CSVReaderOptions &options, Vector &input_vector, Vector &result_vector, + idx_t count, string &error_message, const LogicalType &result_type) { + auto width = DecimalType::GetWidth(result_type); + auto scale = DecimalType::GetScale(result_type); + switch (result_type.InternalType()) { + case PhysicalType::INT16: + return TemplatedTryCastDecimalVector( + options, input_vector, result_vector, count, error_message, width, scale); + case PhysicalType::INT32: + return TemplatedTryCastDecimalVector( + options, input_vector, result_vector, count, error_message, width, scale); + case PhysicalType::INT64: + return TemplatedTryCastDecimalVector( + options, input_vector, result_vector, count, error_message, width, scale); + case PhysicalType::INT128: + return TemplatedTryCastDecimalVector( + options, input_vector, result_vector, count, error_message, width, scale); + default: + throw InternalException("Unimplemented physical type for decimal"); + } +} + +bool TryCastFloatingVectorCommaSeparated(CSVReaderOptions &options, Vector &input_vector, Vector &result_vector, + idx_t count, string &error_message, const LogicalType &result_type, + idx_t &line_error) { + switch (result_type.InternalType()) { + case PhysicalType::DOUBLE: + return TemplatedTryCastFloatingVector( + options, input_vector, result_vector, count, error_message, line_error); + case PhysicalType::FLOAT: + return TemplatedTryCastFloatingVector( + options, input_vector, result_vector, count, error_message, line_error); + default: + throw InternalException("Unimplemented physical type for floating"); + } +} + +// Location of erroneous value in the current parse chunk +struct ErrorLocation { + idx_t row_idx; + idx_t col_idx; + idx_t row_line; + + ErrorLocation(idx_t row_idx, idx_t col_idx, idx_t row_line) + : row_idx(row_idx), col_idx(col_idx), row_line(row_line) { + } +}; + +bool BaseCSVReader::Flush(DataChunk &insert_chunk, idx_t buffer_idx, bool try_add_line) { + if (parse_chunk.size() == 0) { + return true; + } + + bool conversion_error_ignored = false; + + // convert the columns in the parsed chunk to the types of the table + insert_chunk.SetCardinality(parse_chunk); + if (reader_data.column_ids.empty() && !reader_data.empty_columns) { + throw InternalException("BaseCSVReader::Flush called on a CSV reader that was not correctly initialized. Call " + "MultiFileReader::InitializeReader or InitializeProjection"); + } + D_ASSERT(reader_data.column_ids.size() == reader_data.column_mapping.size()); + for (idx_t c = 0; c < reader_data.column_ids.size(); c++) { + auto col_idx = reader_data.column_ids[c]; + auto result_idx = reader_data.column_mapping[c]; + auto &parse_vector = parse_chunk.data[col_idx]; + auto &result_vector = insert_chunk.data[result_idx]; + auto &type = result_vector.GetType(); + if (type.id() == LogicalTypeId::VARCHAR) { + // target type is varchar: no need to convert + // just test that all strings are valid utf-8 strings + VerifyUTF8(col_idx); + // reinterpret rather than reference so we can deal with user-defined types + result_vector.Reinterpret(parse_vector); + } else { + string error_message; + bool success; + idx_t line_error = 0; + bool target_type_not_varchar = false; + if (options.dialect_options.has_format[LogicalTypeId::DATE] && type.id() == LogicalTypeId::DATE) { + // use the date format to cast the chunk + success = TryCastDateVector(options.dialect_options.date_format, parse_vector, result_vector, + parse_chunk.size(), error_message, line_error); + } else if (options.dialect_options.has_format[LogicalTypeId::TIMESTAMP] && + type.id() == LogicalTypeId::TIMESTAMP) { + // use the date format to cast the chunk + success = TryCastTimestampVector(options.dialect_options.date_format, parse_vector, result_vector, + parse_chunk.size(), error_message); + } else if (options.decimal_separator != "." && + (type.id() == LogicalTypeId::FLOAT || type.id() == LogicalTypeId::DOUBLE)) { + success = TryCastFloatingVectorCommaSeparated(options, parse_vector, result_vector, parse_chunk.size(), + error_message, type, line_error); + } else if (options.decimal_separator != "." && type.id() == LogicalTypeId::DECIMAL) { + success = TryCastDecimalVectorCommaSeparated(options, parse_vector, result_vector, parse_chunk.size(), + error_message, type); + } else { + // target type is not varchar: perform a cast + target_type_not_varchar = true; + success = + VectorOperations::TryCast(context, parse_vector, result_vector, parse_chunk.size(), &error_message); + } + if (success) { + continue; + } + if (try_add_line) { + return false; + } + + string col_name = to_string(col_idx); + if (col_idx < names.size()) { + col_name = "\"" + names[col_idx] + "\""; + } + + // figure out the exact line number + if (target_type_not_varchar) { + UnifiedVectorFormat inserted_column_data; + result_vector.ToUnifiedFormat(parse_chunk.size(), inserted_column_data); + for (; line_error < parse_chunk.size(); line_error++) { + if (!inserted_column_data.validity.RowIsValid(line_error) && + !FlatVector::IsNull(parse_vector, line_error)) { + break; + } + } + } + + // The line_error must be summed with linenr (All lines emmited from this batch) + // But subtracted from the parse_chunk + D_ASSERT(line_error + linenr >= parse_chunk.size()); + line_error += linenr; + line_error -= parse_chunk.size(); + + auto error_line = GetLineError(line_error, buffer_idx); + + if (options.ignore_errors) { + conversion_error_ignored = true; + + } else if (options.auto_detect) { + throw InvalidInputException("%s in column %s, at line %llu.\n\nParser " + "options:\n%s.\n\nConsider either increasing the sample size " + "(SAMPLE_SIZE=X [X rows] or SAMPLE_SIZE=-1 [all rows]), " + "or skipping column conversion (ALL_VARCHAR=1)", + error_message, col_name, error_line, options.ToString()); + } else { + throw InvalidInputException("%s at line %llu in column %s. Parser options:\n%s ", error_message, + error_line, col_name, options.ToString()); + } + } + } + if (conversion_error_ignored) { + D_ASSERT(options.ignore_errors); + + SelectionVector succesful_rows(parse_chunk.size()); + idx_t sel_size = 0; + + // Keep track of failed cells + vector failed_cells; + + for (idx_t row_idx = 0; row_idx < parse_chunk.size(); row_idx++) { + + auto global_row_idx = row_idx + linenr - parse_chunk.size(); + auto row_line = GetLineError(global_row_idx, buffer_idx, false); + + bool row_failed = false; + for (idx_t c = 0; c < reader_data.column_ids.size(); c++) { + auto col_idx = reader_data.column_ids[c]; + auto result_idx = reader_data.column_mapping[c]; + + auto &parse_vector = parse_chunk.data[col_idx]; + auto &result_vector = insert_chunk.data[result_idx]; + + bool was_already_null = FlatVector::IsNull(parse_vector, row_idx); + if (!was_already_null && FlatVector::IsNull(result_vector, row_idx)) { + row_failed = true; + failed_cells.emplace_back(row_idx, col_idx, row_line); + } + } + if (!row_failed) { + succesful_rows.set_index(sel_size++, row_idx); + } + } + + // Now do a second pass to produce the reject table entries + if (!failed_cells.empty() && !options.rejects_table_name.empty()) { + auto limit = options.rejects_limit; + + auto rejects = CSVRejectsTable::GetOrCreate(context, options.rejects_table_name); + lock_guard lock(rejects->write_lock); + + // short circuit if we already have too many rejects + if (limit == 0 || rejects->count < limit) { + auto &table = rejects->GetTable(context); + InternalAppender appender(context, table); + auto file_name = GetFileName(); + + for (auto &cell : failed_cells) { + if (limit != 0 && rejects->count >= limit) { + break; + } + rejects->count++; + + auto row_idx = cell.row_idx; + auto col_idx = cell.col_idx; + auto row_line = cell.row_line; + + auto col_name = to_string(col_idx); + if (col_idx < names.size()) { + col_name = "\"" + names[col_idx] + "\""; + } + + auto &parse_vector = parse_chunk.data[col_idx]; + auto parsed_str = FlatVector::GetData(parse_vector)[row_idx]; + auto &type = insert_chunk.data[col_idx].GetType(); + auto row_error_msg = StringUtil::Format("Could not convert string '%s' to '%s'", + parsed_str.GetString(), type.ToString()); + + // Add the row to the rejects table + appender.BeginRow(); + appender.Append(string_t(file_name)); + appender.Append(row_line); + appender.Append(col_idx); + appender.Append(string_t(col_name)); + appender.Append(parsed_str); + + if (!options.rejects_recovery_columns.empty()) { + child_list_t recovery_key; + for (auto &key_idx : options.rejects_recovery_column_ids) { + // Figure out if the recovery key is valid. + // If not, error out for real. + auto &component_vector = parse_chunk.data[key_idx]; + if (FlatVector::IsNull(component_vector, row_idx)) { + throw InvalidInputException("%s at line %llu in column %s. Parser options:\n%s ", + "Could not parse recovery column", row_line, col_name, + options.ToString()); + } + auto component = Value(FlatVector::GetData(component_vector)[row_idx]); + recovery_key.emplace_back(names[key_idx], component); + } + appender.Append(Value::STRUCT(recovery_key)); + } + + appender.Append(string_t(row_error_msg)); + appender.EndRow(); + } + appender.Close(); + } + } + + // Now slice the insert chunk to only include the succesful rows + insert_chunk.Slice(succesful_rows, sel_size); + } + parse_chunk.Reset(); + return true; +} + +void BaseCSVReader::SetNewLineDelimiter(bool carry, bool carry_followed_by_nl) { + if (options.dialect_options.new_line == NewLineIdentifier::NOT_SET) { + if (options.dialect_options.new_line == NewLineIdentifier::MIX) { + return; + } + NewLineIdentifier this_line_identifier; + if (carry) { + if (carry_followed_by_nl) { + this_line_identifier = NewLineIdentifier::CARRY_ON; + } else { + this_line_identifier = NewLineIdentifier::SINGLE; + } + } else { + this_line_identifier = NewLineIdentifier::SINGLE; + } + if (options.dialect_options.new_line == NewLineIdentifier::NOT_SET) { + options.dialect_options.new_line = this_line_identifier; + return; + } + if (options.dialect_options.new_line != this_line_identifier) { + options.dialect_options.new_line = NewLineIdentifier::MIX; + return; + } + options.dialect_options.new_line = this_line_identifier; + } +} +} // namespace duckdb diff --git a/src/duckdb/src/execution/operator/csv_scanner/buffered_csv_reader.cpp b/src/duckdb/src/execution/operator/csv_scanner/buffered_csv_reader.cpp new file mode 100644 index 000000000..55c9494cd --- /dev/null +++ b/src/duckdb/src/execution/operator/csv_scanner/buffered_csv_reader.cpp @@ -0,0 +1,434 @@ +#include "duckdb/execution/operator/scan/csv/buffered_csv_reader.hpp" + +#include "duckdb/catalog/catalog_entry/table_catalog_entry.hpp" +#include "duckdb/common/file_system.hpp" +#include "duckdb/common/string_util.hpp" +#include "duckdb/common/to_string.hpp" +#include "duckdb/common/types/cast_helpers.hpp" +#include "duckdb/common/vector_operations/unary_executor.hpp" +#include "duckdb/common/vector_operations/vector_operations.hpp" +#include "duckdb/execution/operator/scan/csv/csv_sniffer.hpp" +#include "duckdb/execution/operator/scan/csv/csv_state_machine.hpp" +#include "duckdb/function/scalar/strftime_format.hpp" +#include "duckdb/main/client_data.hpp" +#include "duckdb/main/database.hpp" +#include "duckdb/main/error_manager.hpp" +#include "duckdb/parser/column_definition.hpp" +#include "duckdb/parser/keyword_helper.hpp" +#include "duckdb/storage/data_table.hpp" +#include "utf8proc.hpp" +#include "utf8proc_wrapper.hpp" + +#include +#include +#include +#include + +namespace duckdb { + +BufferedCSVReader::BufferedCSVReader(ClientContext &context, CSVReaderOptions options_p, + const vector &requested_types) + : BaseCSVReader(context, std::move(options_p), requested_types), buffer_size(0), position(0), start(0) { + file_handle = OpenCSV(context, options); + Initialize(requested_types); +} + +BufferedCSVReader::BufferedCSVReader(ClientContext &context, string filename, CSVReaderOptions options_p, + const vector &requested_types) + : BaseCSVReader(context, std::move(options_p), requested_types), buffer_size(0), position(0), start(0) { + options.file_path = std::move(filename); + file_handle = OpenCSV(context, options); + Initialize(requested_types); +} + +void BufferedCSVReader::Initialize(const vector &requested_types) { + if (options.auto_detect && options.file_options.union_by_name) { + // This is required for the sniffer to work on Union By Name + D_ASSERT(options.file_path == file_handle->GetFilePath()); + auto bm_file_handle = BaseCSVReader::OpenCSV(context, options); + auto csv_buffer_manager = make_shared(context, std::move(bm_file_handle), options); + CSVSniffer sniffer(options, csv_buffer_manager, state_machine_cache); + auto sniffer_result = sniffer.SniffCSV(); + return_types = sniffer_result.return_types; + names = sniffer_result.names; + if (return_types.empty()) { + throw InvalidInputException("Failed to detect column types from CSV: is the file a valid CSV file?"); + } + } else { + return_types = requested_types; + ResetBuffer(); + } + SkipRowsAndReadHeader(options.dialect_options.skip_rows, options.dialect_options.header); + InitParseChunk(return_types.size()); +} + +void BufferedCSVReader::ResetBuffer() { + buffer.reset(); + buffer_size = 0; + position = 0; + start = 0; + cached_buffers.clear(); +} + +void BufferedCSVReader::SkipRowsAndReadHeader(idx_t skip_rows, bool skip_header) { + for (idx_t i = 0; i < skip_rows; i++) { + // ignore skip rows + string read_line = file_handle->ReadLine(); + linenr++; + } + + if (skip_header) { + // ignore the first line as a header line + InitParseChunk(return_types.size()); + ParseCSV(ParserMode::PARSING_HEADER); + } +} + +string BufferedCSVReader::ColumnTypesError(case_insensitive_map_t sql_types_per_column, + const vector &names) { + for (idx_t i = 0; i < names.size(); i++) { + auto it = sql_types_per_column.find(names[i]); + if (it != sql_types_per_column.end()) { + sql_types_per_column.erase(names[i]); + continue; + } + } + if (sql_types_per_column.empty()) { + return string(); + } + string exception = "COLUMN_TYPES error: Columns with names: "; + for (auto &col : sql_types_per_column) { + exception += "\"" + col.first + "\","; + } + exception.pop_back(); + exception += " do not exist in the CSV File"; + return exception; +} + +void BufferedCSVReader::SkipEmptyLines() { + if (parse_chunk.data.size() == 1) { + // Empty lines are null data. + return; + } + for (; position < buffer_size; position++) { + if (!StringUtil::CharacterIsNewline(buffer[position])) { + return; + } + } +} + +void UpdateMaxLineLength(ClientContext &context, idx_t line_length) { + if (!context.client_data->debug_set_max_line_length) { + return; + } + if (line_length < context.client_data->debug_max_line_length) { + return; + } + context.client_data->debug_max_line_length = line_length; +} + +bool BufferedCSVReader::ReadBuffer(idx_t &start, idx_t &line_start) { + if (start > buffer_size) { + return false; + } + auto old_buffer = std::move(buffer); + + // the remaining part of the last buffer + idx_t remaining = buffer_size - start; + + idx_t buffer_read_size = INITIAL_BUFFER_SIZE_LARGE; + + while (remaining > buffer_read_size) { + buffer_read_size *= 2; + } + + // Check line length + if (remaining > options.maximum_line_size) { + throw InvalidInputException("Maximum line size of %llu bytes exceeded on line %s!", options.maximum_line_size, + GetLineNumberStr(linenr, linenr_estimated)); + } + + buffer = make_unsafe_uniq_array(buffer_read_size + remaining + 1); + buffer_size = remaining + buffer_read_size; + if (remaining > 0) { + // remaining from last buffer: copy it here + memcpy(buffer.get(), old_buffer.get() + start, remaining); + } + idx_t read_count = file_handle->Read(buffer.get() + remaining, buffer_read_size); + + bytes_in_chunk += read_count; + buffer_size = remaining + read_count; + buffer[buffer_size] = '\0'; + if (old_buffer) { + cached_buffers.push_back(std::move(old_buffer)); + } + start = 0; + position = remaining; + if (!bom_checked) { + bom_checked = true; + if (read_count >= 3 && buffer[0] == '\xEF' && buffer[1] == '\xBB' && buffer[2] == '\xBF') { + start += 3; + position += 3; + } + } + line_start = start; + + return read_count > 0; +} + +void BufferedCSVReader::ParseCSV(DataChunk &insert_chunk) { + string error_message; + if (!TryParseCSV(ParserMode::PARSING, insert_chunk, error_message)) { + throw InvalidInputException(error_message); + } +} + +void BufferedCSVReader::ParseCSV(ParserMode mode) { + DataChunk dummy_chunk; + string error_message; + if (!TryParseCSV(mode, dummy_chunk, error_message)) { + throw InvalidInputException(error_message); + } +} + +bool BufferedCSVReader::TryParseCSV(ParserMode parser_mode, DataChunk &insert_chunk, string &error_message) { + mode = parser_mode; + // used for parsing algorithm + bool finished_chunk = false; + idx_t column = 0; + idx_t offset = 0; + bool has_quotes = false; + vector escape_positions; + + idx_t line_start = position; + idx_t line_size = 0; + // read values into the buffer (if any) + if (position >= buffer_size) { + if (!ReadBuffer(start, line_start)) { + return true; + } + } + + // start parsing the first value + goto value_start; +value_start: + offset = 0; + /* state: value_start */ + // this state parses the first character of a value + if (buffer[position] == options.dialect_options.state_machine_options.quote) { + // quote: actual value starts in the next position + // move to in_quotes state + start = position + 1; + line_size++; + goto in_quotes; + } else { + // no quote, move to normal parsing state + start = position; + goto normal; + } +normal: + /* state: normal parsing state */ + // this state parses the remainder of a non-quoted value until we reach a delimiter or newline + do { + for (; position < buffer_size; position++) { + line_size++; + if (buffer[position] == options.dialect_options.state_machine_options.delimiter) { + // delimiter: end the value and add it to the chunk + goto add_value; + } else if (StringUtil::CharacterIsNewline(buffer[position])) { + // newline: add row + goto add_row; + } + } + } while (ReadBuffer(start, line_start)); + // file ends during normal scan: go to end state + goto final_state; +add_value: + AddValue(string_t(buffer.get() + start, position - start - offset), column, escape_positions, has_quotes); + // increase position by 1 and move start to the new position + offset = 0; + has_quotes = false; + start = ++position; + line_size++; + if (position >= buffer_size && !ReadBuffer(start, line_start)) { + // file ends right after delimiter, go to final state + goto final_state; + } + goto value_start; +add_row : { + // check type of newline (\r or \n) + bool carriage_return = buffer[position] == '\r'; + AddValue(string_t(buffer.get() + start, position - start - offset), column, escape_positions, has_quotes); + if (!error_message.empty()) { + return false; + } + VerifyLineLength(position - line_start); + + finished_chunk = AddRow(insert_chunk, column, error_message); + UpdateMaxLineLength(context, position - line_start); + if (!error_message.empty()) { + return false; + } + // increase position by 1 and move start to the new position + offset = 0; + has_quotes = false; + position++; + line_size = 0; + start = position; + line_start = position; + if (position >= buffer_size && !ReadBuffer(start, line_start)) { + // file ends right after delimiter, go to final state + goto final_state; + } + if (carriage_return) { + // \r newline, go to special state that parses an optional \n afterwards + goto carriage_return; + } else { + SetNewLineDelimiter(); + SkipEmptyLines(); + + start = position; + line_start = position; + if (position >= buffer_size && !ReadBuffer(start, line_start)) { + // file ends right after delimiter, go to final state + goto final_state; + } + // \n newline, move to value start + if (finished_chunk) { + return true; + } + goto value_start; + } +} +in_quotes: + /* state: in_quotes */ + // this state parses the remainder of a quoted value + has_quotes = true; + position++; + line_size++; + do { + for (; position < buffer_size; position++) { + line_size++; + if (buffer[position] == options.dialect_options.state_machine_options.quote) { + // quote: move to unquoted state + goto unquote; + } else if (buffer[position] == options.dialect_options.state_machine_options.escape) { + // escape: store the escaped position and move to handle_escape state + escape_positions.push_back(position - start); + goto handle_escape; + } + } + } while (ReadBuffer(start, line_start)); + // still in quoted state at the end of the file, error: + throw InvalidInputException("Error in file \"%s\" on line %s: unterminated quotes. (%s)", options.file_path, + GetLineNumberStr(linenr, linenr_estimated).c_str(), options.ToString()); +unquote: + /* state: unquote */ + // this state handles the state directly after we unquote + // in this state we expect either another quote (entering the quoted state again, and escaping the quote) + // or a delimiter/newline, ending the current value and moving on to the next value + position++; + line_size++; + if (position >= buffer_size && !ReadBuffer(start, line_start)) { + // file ends right after unquote, go to final state + offset = 1; + goto final_state; + } + if (buffer[position] == options.dialect_options.state_machine_options.quote && + (options.dialect_options.state_machine_options.escape == '\0' || + options.dialect_options.state_machine_options.escape == options.dialect_options.state_machine_options.quote)) { + // escaped quote, return to quoted state and store escape position + escape_positions.push_back(position - start); + goto in_quotes; + } else if (buffer[position] == options.dialect_options.state_machine_options.delimiter) { + // delimiter, add value + offset = 1; + goto add_value; + } else if (StringUtil::CharacterIsNewline(buffer[position])) { + offset = 1; + goto add_row; + } else { + error_message = StringUtil::Format( + "Error in file \"%s\" on line %s: quote should be followed by end of value, end of " + "row or another quote. (%s)", + options.file_path, GetLineNumberStr(linenr, linenr_estimated).c_str(), options.ToString()); + return false; + } +handle_escape: + /* state: handle_escape */ + // escape should be followed by a quote or another escape character + position++; + line_size++; + if (position >= buffer_size && !ReadBuffer(start, line_start)) { + error_message = StringUtil::Format( + "Error in file \"%s\" on line %s: neither QUOTE nor ESCAPE is proceeded by ESCAPE. (%s)", options.file_path, + GetLineNumberStr(linenr, linenr_estimated).c_str(), options.ToString()); + return false; + } + if (buffer[position] != options.dialect_options.state_machine_options.quote && + buffer[position] != options.dialect_options.state_machine_options.escape) { + error_message = StringUtil::Format( + "Error in file \"%s\" on line %s: neither QUOTE nor ESCAPE is proceeded by ESCAPE. (%s)", options.file_path, + GetLineNumberStr(linenr, linenr_estimated).c_str(), options.ToString()); + return false; + } + // escape was followed by quote or escape, go back to quoted state + goto in_quotes; +carriage_return: + /* state: carriage_return */ + // this stage optionally skips a newline (\n) character, which allows \r\n to be interpreted as a single line + if (buffer[position] == '\n') { + SetNewLineDelimiter(true, true); + // newline after carriage return: skip + // increase position by 1 and move start to the new position + start = ++position; + line_size++; + + if (position >= buffer_size && !ReadBuffer(start, line_start)) { + // file ends right after delimiter, go to final state + goto final_state; + } + } else { + SetNewLineDelimiter(true, false); + } + if (finished_chunk) { + return true; + } + SkipEmptyLines(); + start = position; + line_start = position; + if (position >= buffer_size && !ReadBuffer(start, line_start)) { + // file ends right after delimiter, go to final state + goto final_state; + } + + goto value_start; +final_state: + if (finished_chunk) { + return true; + } + + if (column > 0 || position > start) { + // remaining values to be added to the chunk + AddValue(string_t(buffer.get() + start, position - start - offset), column, escape_positions, has_quotes); + VerifyLineLength(position - line_start); + + finished_chunk = AddRow(insert_chunk, column, error_message); + SkipEmptyLines(); + UpdateMaxLineLength(context, line_size); + if (!error_message.empty()) { + return false; + } + } + + // final stage, only reached after parsing the file is finished + // flush the parsed chunk and finalize parsing + if (mode == ParserMode::PARSING) { + Flush(insert_chunk); + } + + end_of_file_reached = true; + return true; +} + +} // namespace duckdb diff --git a/src/duckdb/src/execution/operator/csv_scanner/csv_buffer.cpp b/src/duckdb/src/execution/operator/csv_scanner/csv_buffer.cpp new file mode 100644 index 000000000..3f01667d1 --- /dev/null +++ b/src/duckdb/src/execution/operator/csv_scanner/csv_buffer.cpp @@ -0,0 +1,80 @@ +#include "duckdb/execution/operator/scan/csv/csv_buffer.hpp" +#include "duckdb/common/string_util.hpp" + +namespace duckdb { + +CSVBuffer::CSVBuffer(ClientContext &context, idx_t buffer_size_p, CSVFileHandle &file_handle, + idx_t &global_csv_current_position, idx_t file_number_p) + : context(context), first_buffer(true), file_number(file_number_p), can_seek(file_handle.CanSeek()) { + AllocateBuffer(buffer_size_p); + auto buffer = Ptr(); + file_size = file_handle.Read(buffer, buffer_size_p); + global_csv_start = global_csv_current_position; + // BOM check (https://en.wikipedia.org/wiki/Byte_order_mark) + if (file_size >= 3 && buffer[0] == '\xEF' && buffer[1] == '\xBB' && buffer[2] == '\xBF') { + start_position += 3; + } + last_buffer = file_handle.FinishedReading(); +} + +CSVBuffer::CSVBuffer(CSVFileHandle &file_handle, ClientContext &context, idx_t buffer_size, + idx_t global_csv_current_position, idx_t file_number_p) + : context(context), global_csv_start(global_csv_current_position), file_number(file_number_p), + can_seek(file_handle.CanSeek()) { + AllocateBuffer(buffer_size); + file_size = file_handle.Read(handle.Ptr(), buffer_size); + last_buffer = file_handle.FinishedReading(); +} + +shared_ptr CSVBuffer::Next(CSVFileHandle &file_handle, idx_t buffer_size, idx_t file_number_p) { + auto next_csv_buffer = + make_shared(file_handle, context, buffer_size, global_csv_start + file_size, file_number_p); + if (next_csv_buffer->GetBufferSize() == 0) { + // We are done reading + return nullptr; + } + return next_csv_buffer; +} + +void CSVBuffer::AllocateBuffer(idx_t buffer_size) { + auto &buffer_manager = BufferManager::GetBufferManager(context); + bool can_destroy = can_seek; + handle = buffer_manager.Allocate(MaxValue(Storage::BLOCK_SIZE, buffer_size), can_destroy, &block); +} + +idx_t CSVBuffer::GetBufferSize() { + return file_size; +} + +void CSVBuffer::Reload(CSVFileHandle &file_handle) { + AllocateBuffer(file_size); + file_handle.Seek(global_csv_start); + file_handle.Read(handle.Ptr(), file_size); +} + +unique_ptr CSVBuffer::Pin(CSVFileHandle &file_handle) { + auto &buffer_manager = BufferManager::GetBufferManager(context); + if (can_seek && block->IsUnloaded()) { + // We have to reload it from disk + block = nullptr; + Reload(file_handle); + } + return make_uniq(buffer_manager.Pin(block), file_size, first_buffer, last_buffer, global_csv_start, + start_position, file_number); +} + +void CSVBuffer::Unpin() { + if (handle.IsValid()) { + handle.Destroy(); + } +} + +idx_t CSVBuffer::GetStart() { + return start_position; +} + +bool CSVBuffer::IsCSVFileLastBuffer() { + return last_buffer; +} + +} // namespace duckdb diff --git a/src/duckdb/src/execution/operator/csv_scanner/csv_buffer_manager.cpp b/src/duckdb/src/execution/operator/csv_scanner/csv_buffer_manager.cpp new file mode 100644 index 000000000..408e7c855 --- /dev/null +++ b/src/duckdb/src/execution/operator/csv_scanner/csv_buffer_manager.cpp @@ -0,0 +1,90 @@ +#include "duckdb/execution/operator/scan/csv/csv_buffer_manager.hpp" +#include "duckdb/execution/operator/scan/csv/csv_buffer.hpp" +namespace duckdb { + +CSVBufferManager::CSVBufferManager(ClientContext &context_p, unique_ptr file_handle_p, + const CSVReaderOptions &options, idx_t file_idx_p) + : file_handle(std::move(file_handle_p)), context(context_p), file_idx(file_idx_p), + buffer_size(CSVBuffer::CSV_BUFFER_SIZE) { + if (options.skip_rows_set) { + // Skip rows if they are set + skip_rows = options.dialect_options.skip_rows; + } + auto file_size = file_handle->FileSize(); + if (file_size > 0 && file_size < buffer_size) { + buffer_size = CSVBuffer::CSV_MINIMUM_BUFFER_SIZE; + } + if (options.buffer_size < buffer_size) { + buffer_size = options.buffer_size; + } + for (idx_t i = 0; i < skip_rows; i++) { + file_handle->ReadLine(); + } + Initialize(); +} + +void CSVBufferManager::UnpinBuffer(idx_t cache_idx) { + if (cache_idx < cached_buffers.size()) { + cached_buffers[cache_idx]->Unpin(); + } +} + +void CSVBufferManager::Initialize() { + if (cached_buffers.empty()) { + cached_buffers.emplace_back( + make_shared(context, buffer_size, *file_handle, global_csv_pos, file_idx)); + last_buffer = cached_buffers.front(); + } + start_pos = last_buffer->GetStart(); +} + +idx_t CSVBufferManager::GetStartPos() { + return start_pos; +} +bool CSVBufferManager::ReadNextAndCacheIt() { + D_ASSERT(last_buffer); + if (!last_buffer->IsCSVFileLastBuffer()) { + auto maybe_last_buffer = last_buffer->Next(*file_handle, buffer_size, file_idx); + if (!maybe_last_buffer) { + last_buffer->last_buffer = true; + return false; + } + last_buffer = std::move(maybe_last_buffer); + cached_buffers.emplace_back(last_buffer); + return true; + } + return false; +} + +unique_ptr CSVBufferManager::GetBuffer(const idx_t pos) { + while (pos >= cached_buffers.size()) { + if (done) { + return nullptr; + } + if (!ReadNextAndCacheIt()) { + done = true; + } + } + if (pos != 0) { + cached_buffers[pos - 1]->Unpin(); + } + return cached_buffers[pos]->Pin(*file_handle); +} + +bool CSVBufferIterator::Finished() { + return !cur_buffer_handle; +} + +void CSVBufferIterator::Reset() { + if (cur_buffer_handle) { + cur_buffer_handle.reset(); + } + if (cur_buffer_idx > 0) { + buffer_manager->UnpinBuffer(cur_buffer_idx - 1); + } + cur_buffer_idx = 0; + buffer_manager->Initialize(); + cur_pos = buffer_manager->GetStartPos(); +} + +} // namespace duckdb diff --git a/src/duckdb/src/execution/operator/csv_scanner/csv_file_handle.cpp b/src/duckdb/src/execution/operator/csv_scanner/csv_file_handle.cpp new file mode 100644 index 000000000..6462db94d --- /dev/null +++ b/src/duckdb/src/execution/operator/csv_scanner/csv_file_handle.cpp @@ -0,0 +1,95 @@ +#include "duckdb/execution/operator/scan/csv/csv_file_handle.hpp" + +namespace duckdb { + +CSVFileHandle::CSVFileHandle(FileSystem &fs, Allocator &allocator, unique_ptr file_handle_p, + const string &path_p, FileCompressionType compression) + : file_handle(std::move(file_handle_p)), path(path_p) { + can_seek = file_handle->CanSeek(); + on_disk_file = file_handle->OnDiskFile(); + file_size = file_handle->GetFileSize(); +} + +unique_ptr CSVFileHandle::OpenFileHandle(FileSystem &fs, Allocator &allocator, const string &path, + FileCompressionType compression) { + auto file_handle = fs.OpenFile(path, FileFlags::FILE_FLAGS_READ, FileLockType::NO_LOCK, compression); + if (file_handle->CanSeek()) { + file_handle->Reset(); + } + return file_handle; +} + +unique_ptr CSVFileHandle::OpenFile(FileSystem &fs, Allocator &allocator, const string &path, + FileCompressionType compression) { + auto file_handle = CSVFileHandle::OpenFileHandle(fs, allocator, path, compression); + return make_uniq(fs, allocator, std::move(file_handle), path, compression); +} + +bool CSVFileHandle::CanSeek() { + return can_seek; +} + +void CSVFileHandle::Seek(idx_t position) { + if (!can_seek) { + throw InternalException("Cannot seek in this file"); + } + file_handle->Seek(position); +} + +bool CSVFileHandle::OnDiskFile() { + return on_disk_file; +} + +idx_t CSVFileHandle::FileSize() { + return file_size; +} + +bool CSVFileHandle::FinishedReading() { + return finished; +} + +idx_t CSVFileHandle::Read(void *buffer, idx_t nr_bytes) { + requested_bytes += nr_bytes; + // if this is a plain file source OR we can seek we are not caching anything + auto bytes_read = file_handle->Read(buffer, nr_bytes); + if (!finished) { + finished = bytes_read == 0; + } + return bytes_read; +} + +string CSVFileHandle::ReadLine() { + bool carriage_return = false; + string result; + char buffer[1]; + while (true) { + idx_t bytes_read = Read(buffer, 1); + if (bytes_read == 0) { + return result; + } + if (carriage_return) { + if (buffer[0] != '\n') { + if (!file_handle->CanSeek()) { + throw BinderException( + "Carriage return newlines not supported when reading CSV files in which we cannot seek"); + } + file_handle->Seek(file_handle->SeekPosition() - 1); + return result; + } + } + if (buffer[0] == '\n') { + return result; + } + if (buffer[0] != '\r') { + result += buffer[0]; + } else { + carriage_return = true; + } + } +} + +string CSVFileHandle::GetFilePath() { + return path; +} + +} // namespace duckdb diff --git a/src/duckdb/src/execution/operator/csv_scanner/csv_reader_options.cpp b/src/duckdb/src/execution/operator/csv_scanner/csv_reader_options.cpp new file mode 100644 index 000000000..524f3a500 --- /dev/null +++ b/src/duckdb/src/execution/operator/csv_scanner/csv_reader_options.cpp @@ -0,0 +1,299 @@ +#include "duckdb/execution/operator/scan/csv/csv_reader_options.hpp" +#include "duckdb/common/bind_helpers.hpp" +#include "duckdb/common/vector_size.hpp" +#include "duckdb/common/string_util.hpp" + +namespace duckdb { + +static bool ParseBoolean(const Value &value, const string &loption); + +static bool ParseBoolean(const vector &set, const string &loption) { + if (set.empty()) { + // no option specified: default to true + return true; + } + if (set.size() > 1) { + throw BinderException("\"%s\" expects a single argument as a boolean value (e.g. TRUE or 1)", loption); + } + return ParseBoolean(set[0], loption); +} + +static bool ParseBoolean(const Value &value, const string &loption) { + + if (value.type().id() == LogicalTypeId::LIST) { + auto &children = ListValue::GetChildren(value); + return ParseBoolean(children, loption); + } + if (value.type() == LogicalType::FLOAT || value.type() == LogicalType::DOUBLE || + value.type().id() == LogicalTypeId::DECIMAL) { + throw BinderException("\"%s\" expects a boolean value (e.g. TRUE or 1)", loption); + } + return BooleanValue::Get(value.DefaultCastAs(LogicalType::BOOLEAN)); +} + +static string ParseString(const Value &value, const string &loption) { + if (value.IsNull()) { + return string(); + } + if (value.type().id() == LogicalTypeId::LIST) { + auto &children = ListValue::GetChildren(value); + if (children.size() != 1) { + throw BinderException("\"%s\" expects a single argument as a string value", loption); + } + return ParseString(children[0], loption); + } + if (value.type().id() != LogicalTypeId::VARCHAR) { + throw BinderException("\"%s\" expects a string argument!", loption); + } + return value.GetValue(); +} + +static int64_t ParseInteger(const Value &value, const string &loption) { + if (value.type().id() == LogicalTypeId::LIST) { + auto &children = ListValue::GetChildren(value); + if (children.size() != 1) { + // no option specified or multiple options specified + throw BinderException("\"%s\" expects a single argument as an integer value", loption); + } + return ParseInteger(children[0], loption); + } + return value.GetValue(); +} + +void CSVReaderOptions::SetHeader(bool input) { + this->dialect_options.header = input; + this->has_header = true; +} + +void CSVReaderOptions::SetCompression(const string &compression_p) { + this->compression = FileCompressionTypeFromString(compression_p); +} + +void CSVReaderOptions::SetEscape(const string &input) { + auto escape_str = input; + if (escape_str.size() > 1) { + throw InvalidInputException("The escape option cannot exceed a size of 1 byte."); + } + if (escape_str.empty()) { + escape_str = string("\0", 1); + } + this->dialect_options.state_machine_options.escape = escape_str[0]; + this->has_escape = true; +} + +void CSVReaderOptions::SetDelimiter(const string &input) { + auto delim_str = StringUtil::Replace(input, "\\t", "\t"); + if (delim_str.size() > 1) { + throw InvalidInputException("The delimiter option cannot exceed a size of 1 byte."); + } + this->has_delimiter = true; + if (input.empty()) { + delim_str = string("\0", 1); + } + this->dialect_options.state_machine_options.delimiter = delim_str[0]; +} + +void CSVReaderOptions::SetQuote(const string "e_p) { + auto quote_str = quote_p; + if (quote_str.size() > 1) { + throw InvalidInputException("The quote option cannot exceed a size of 1 byte."); + } + if (quote_str.empty()) { + quote_str = string("\0", 1); + } + this->dialect_options.state_machine_options.quote = quote_str[0]; + this->has_quote = true; +} + +void CSVReaderOptions::SetNewline(const string &input) { + if (input == "\\n" || input == "\\r") { + dialect_options.new_line = NewLineIdentifier::SINGLE; + } else if (input == "\\r\\n") { + dialect_options.new_line = NewLineIdentifier::CARRY_ON; + } else { + throw InvalidInputException("This is not accepted as a newline: " + input); + } + has_newline = true; +} + +void CSVReaderOptions::SetDateFormat(LogicalTypeId type, const string &format, bool read_format) { + string error; + if (read_format) { + error = StrTimeFormat::ParseFormatSpecifier(format, dialect_options.date_format[type]); + dialect_options.date_format[type].format_specifier = format; + } else { + error = StrTimeFormat::ParseFormatSpecifier(format, write_date_format[type]); + } + if (!error.empty()) { + throw InvalidInputException("Could not parse DATEFORMAT: %s", error.c_str()); + } + dialect_options.has_format[type] = true; +} + +void CSVReaderOptions::SetReadOption(const string &loption, const Value &value, vector &expected_names) { + if (SetBaseOption(loption, value)) { + return; + } + if (loption == "auto_detect") { + auto_detect = ParseBoolean(value, loption); + } else if (loption == "sample_size") { + int64_t sample_size = ParseInteger(value, loption); + if (sample_size < 1 && sample_size != -1) { + throw BinderException("Unsupported parameter for SAMPLE_SIZE: cannot be smaller than 1"); + } + if (sample_size == -1) { + sample_chunks = std::numeric_limits::max(); + sample_chunk_size = STANDARD_VECTOR_SIZE; + } else if (sample_size <= STANDARD_VECTOR_SIZE) { + sample_chunk_size = sample_size; + sample_chunks = 1; + } else { + sample_chunk_size = STANDARD_VECTOR_SIZE; + sample_chunks = sample_size / STANDARD_VECTOR_SIZE + 1; + } + } else if (loption == "skip") { + dialect_options.skip_rows = ParseInteger(value, loption); + skip_rows_set = true; + } else if (loption == "max_line_size" || loption == "maximum_line_size") { + maximum_line_size = ParseInteger(value, loption); + } else if (loption == "sample_chunk_size") { + sample_chunk_size = ParseInteger(value, loption); + if (sample_chunk_size > STANDARD_VECTOR_SIZE) { + throw BinderException( + "Unsupported parameter for SAMPLE_CHUNK_SIZE: cannot be bigger than STANDARD_VECTOR_SIZE %d", + STANDARD_VECTOR_SIZE); + } else if (sample_chunk_size < 1) { + throw BinderException("Unsupported parameter for SAMPLE_CHUNK_SIZE: cannot be smaller than 1"); + } + } else if (loption == "sample_chunks") { + sample_chunks = ParseInteger(value, loption); + if (sample_chunks < 1) { + throw BinderException("Unsupported parameter for SAMPLE_CHUNKS: cannot be smaller than 1"); + } + } else if (loption == "force_not_null") { + force_not_null = ParseColumnList(value, expected_names, loption); + } else if (loption == "date_format" || loption == "dateformat") { + string format = ParseString(value, loption); + SetDateFormat(LogicalTypeId::DATE, format, true); + } else if (loption == "timestamp_format" || loption == "timestampformat") { + string format = ParseString(value, loption); + SetDateFormat(LogicalTypeId::TIMESTAMP, format, true); + } else if (loption == "ignore_errors") { + ignore_errors = ParseBoolean(value, loption); + } else if (loption == "buffer_size") { + buffer_size = ParseInteger(value, loption); + if (buffer_size == 0) { + throw InvalidInputException("Buffer Size option must be higher than 0"); + } + } else if (loption == "decimal_separator") { + decimal_separator = ParseString(value, loption); + if (decimal_separator != "." && decimal_separator != ",") { + throw BinderException("Unsupported parameter for DECIMAL_SEPARATOR: should be '.' or ','"); + } + } else if (loption == "null_padding") { + null_padding = ParseBoolean(value, loption); + } else if (loption == "allow_quoted_nulls") { + allow_quoted_nulls = ParseBoolean(value, loption); + } else if (loption == "parallel") { + parallel_mode = ParseBoolean(value, loption) ? ParallelMode::PARALLEL : ParallelMode::SINGLE_THREADED; + } else if (loption == "rejects_table") { + // skip, handled in SetRejectsOptions + auto table_name = ParseString(value, loption); + if (table_name.empty()) { + throw BinderException("REJECTS_TABLE option cannot be empty"); + } + rejects_table_name = table_name; + } else if (loption == "rejects_recovery_columns") { + // Get the list of columns to use as a recovery key + auto &children = ListValue::GetChildren(value); + for (auto &child : children) { + auto col_name = child.GetValue(); + rejects_recovery_columns.push_back(col_name); + } + } else if (loption == "rejects_limit") { + int64_t limit = ParseInteger(value, loption); + if (limit < 0) { + throw BinderException("Unsupported parameter for REJECTS_LIMIT: cannot be negative"); + } + rejects_limit = limit; + } else { + throw BinderException("Unrecognized option for CSV reader \"%s\"", loption); + } +} + +void CSVReaderOptions::SetWriteOption(const string &loption, const Value &value) { + if (loption == "new_line") { + // Steal this from SetBaseOption so we can write different newlines (e.g., format JSON ARRAY) + write_newline = ParseString(value, loption); + return; + } + + if (SetBaseOption(loption, value)) { + return; + } + + if (loption == "force_quote") { + force_quote = ParseColumnList(value, name_list, loption); + } else if (loption == "date_format" || loption == "dateformat") { + string format = ParseString(value, loption); + SetDateFormat(LogicalTypeId::DATE, format, false); + } else if (loption == "timestamp_format" || loption == "timestampformat") { + string format = ParseString(value, loption); + if (StringUtil::Lower(format) == "iso") { + format = "%Y-%m-%dT%H:%M:%S.%fZ"; + } + SetDateFormat(LogicalTypeId::TIMESTAMP, format, false); + SetDateFormat(LogicalTypeId::TIMESTAMP_TZ, format, false); + } else if (loption == "prefix") { + prefix = ParseString(value, loption); + } else if (loption == "suffix") { + suffix = ParseString(value, loption); + } else { + throw BinderException("Unrecognized option CSV writer \"%s\"", loption); + } +} + +bool CSVReaderOptions::SetBaseOption(const string &loption, const Value &value) { + // Make sure this function was only called after the option was turned into lowercase + D_ASSERT(!std::any_of(loption.begin(), loption.end(), ::isupper)); + + if (StringUtil::StartsWith(loption, "delim") || StringUtil::StartsWith(loption, "sep")) { + SetDelimiter(ParseString(value, loption)); + } else if (loption == "quote") { + SetQuote(ParseString(value, loption)); + } else if (loption == "new_line") { + SetNewline(ParseString(value, loption)); + } else if (loption == "escape") { + SetEscape(ParseString(value, loption)); + } else if (loption == "header") { + SetHeader(ParseBoolean(value, loption)); + } else if (loption == "null" || loption == "nullstr") { + null_str = ParseString(value, loption); + } else if (loption == "encoding") { + auto encoding = StringUtil::Lower(ParseString(value, loption)); + if (encoding != "utf8" && encoding != "utf-8") { + throw BinderException("Copy is only supported for UTF-8 encoded files, ENCODING 'UTF-8'"); + } + } else if (loption == "compression") { + SetCompression(ParseString(value, loption)); + } else { + // unrecognized option in base CSV + return false; + } + return true; +} + +string CSVReaderOptions::ToString() const { + return " file=" + file_path + "\n delimiter='" + dialect_options.state_machine_options.delimiter + + (has_delimiter ? "'" : (auto_detect ? "' (auto detected)" : "' (default)")) + "\n quote='" + + dialect_options.state_machine_options.quote + + (has_quote ? "'" : (auto_detect ? "' (auto detected)" : "' (default)")) + "\n escape='" + + dialect_options.state_machine_options.escape + + (has_escape ? "'" : (auto_detect ? "' (auto detected)" : "' (default)")) + + "\n header=" + std::to_string(dialect_options.header) + + (has_header ? "" : (auto_detect ? " (auto detected)" : "' (default)")) + + "\n sample_size=" + std::to_string(sample_chunk_size * sample_chunks) + + "\n ignore_errors=" + std::to_string(ignore_errors) + "\n all_varchar=" + std::to_string(all_varchar); +} + +} // namespace duckdb diff --git a/src/duckdb/src/execution/operator/csv_scanner/csv_state_machine.cpp b/src/duckdb/src/execution/operator/csv_scanner/csv_state_machine.cpp new file mode 100644 index 000000000..785b74f41 --- /dev/null +++ b/src/duckdb/src/execution/operator/csv_scanner/csv_state_machine.cpp @@ -0,0 +1,35 @@ +#include "duckdb/execution/operator/scan/csv/csv_state_machine.hpp" +#include "duckdb/execution/operator/scan/csv/csv_sniffer.hpp" +#include "utf8proc_wrapper.hpp" +#include "duckdb/main/error_manager.hpp" +#include "duckdb/execution/operator/scan/csv/csv_state_machine_cache.hpp" + +namespace duckdb { + +CSVStateMachine::CSVStateMachine(CSVReaderOptions &options_p, const CSVStateMachineOptions &state_machine_options, + shared_ptr buffer_manager_p, + CSVStateMachineCache &csv_state_machine_cache_p) + : csv_state_machine_cache(csv_state_machine_cache_p), options(options_p), + csv_buffer_iterator(std::move(buffer_manager_p)), + transition_array(csv_state_machine_cache.Get(state_machine_options)) { + dialect_options.state_machine_options = state_machine_options; + dialect_options.has_format = options.dialect_options.has_format; + dialect_options.date_format = options.dialect_options.date_format; + dialect_options.skip_rows = options.dialect_options.skip_rows; +} + +void CSVStateMachine::Reset() { + csv_buffer_iterator.Reset(); +} + +void CSVStateMachine::VerifyUTF8() { + auto utf_type = Utf8Proc::Analyze(value.c_str(), value.size()); + if (utf_type == UnicodeType::INVALID) { + int64_t error_line = cur_rows; + throw InvalidInputException("Error in file \"%s\" at line %llu: " + "%s. Parser options:\n%s", + options.file_path, error_line, ErrorManager::InvalidUnicodeError(value, "CSV file"), + options.ToString()); + } +} +} // namespace duckdb diff --git a/src/duckdb/src/execution/operator/csv_scanner/csv_state_machine_cache.cpp b/src/duckdb/src/execution/operator/csv_scanner/csv_state_machine_cache.cpp new file mode 100644 index 000000000..42c441aab --- /dev/null +++ b/src/duckdb/src/execution/operator/csv_scanner/csv_state_machine_cache.cpp @@ -0,0 +1,107 @@ +#include "duckdb/execution/operator/scan/csv/csv_state_machine.hpp" +#include "duckdb/execution/operator/scan/csv/csv_state_machine_cache.hpp" + +namespace duckdb { + +void InitializeTransitionArray(unsigned char *transition_array, const uint8_t state) { + for (uint32_t i = 0; i < NUM_TRANSITIONS; i++) { + transition_array[i] = state; + } +} + +void CSVStateMachineCache::Insert(const CSVStateMachineOptions &state_machine_options) { + D_ASSERT(state_machine_cache.find(state_machine_options) == state_machine_cache.end()); + // Initialize transition array with default values to the Standard option + auto &transition_array = state_machine_cache[state_machine_options]; + const uint8_t standard_state = static_cast(CSVState::STANDARD); + const uint8_t field_separator_state = static_cast(CSVState::DELIMITER); + const uint8_t record_separator_state = static_cast(CSVState::RECORD_SEPARATOR); + const uint8_t carriage_return_state = static_cast(CSVState::CARRIAGE_RETURN); + const uint8_t quoted_state = static_cast(CSVState::QUOTED); + const uint8_t unquoted_state = static_cast(CSVState::UNQUOTED); + const uint8_t escape_state = static_cast(CSVState::ESCAPE); + const uint8_t empty_line_state = static_cast(CSVState::EMPTY_LINE); + const uint8_t invalid_state = static_cast(CSVState::INVALID); + + for (uint32_t i = 0; i < NUM_STATES; i++) { + switch (i) { + case quoted_state: + InitializeTransitionArray(transition_array[i], quoted_state); + break; + case unquoted_state: + InitializeTransitionArray(transition_array[i], invalid_state); + break; + case escape_state: + InitializeTransitionArray(transition_array[i], invalid_state); + break; + default: + InitializeTransitionArray(transition_array[i], standard_state); + break; + } + } + + // Now set values depending on configuration + // 1) Standard State + transition_array[standard_state][static_cast(state_machine_options.delimiter)] = field_separator_state; + transition_array[standard_state][static_cast('\n')] = record_separator_state; + transition_array[standard_state][static_cast('\r')] = carriage_return_state; + transition_array[standard_state][static_cast(state_machine_options.quote)] = quoted_state; + // 2) Field Separator State + transition_array[field_separator_state][static_cast(state_machine_options.delimiter)] = + field_separator_state; + transition_array[field_separator_state][static_cast('\n')] = record_separator_state; + transition_array[field_separator_state][static_cast('\r')] = carriage_return_state; + transition_array[field_separator_state][static_cast(state_machine_options.quote)] = quoted_state; + // 3) Record Separator State + transition_array[record_separator_state][static_cast(state_machine_options.delimiter)] = + field_separator_state; + transition_array[record_separator_state][static_cast('\n')] = empty_line_state; + transition_array[record_separator_state][static_cast('\r')] = empty_line_state; + transition_array[record_separator_state][static_cast(state_machine_options.quote)] = quoted_state; + // 4) Carriage Return State + transition_array[carriage_return_state][static_cast('\n')] = record_separator_state; + transition_array[carriage_return_state][static_cast('\r')] = empty_line_state; + transition_array[carriage_return_state][static_cast(state_machine_options.escape)] = escape_state; + // 5) Quoted State + transition_array[quoted_state][static_cast(state_machine_options.quote)] = unquoted_state; + if (state_machine_options.quote != state_machine_options.escape) { + transition_array[quoted_state][static_cast(state_machine_options.escape)] = escape_state; + } + // 6) Unquoted State + transition_array[unquoted_state][static_cast('\n')] = record_separator_state; + transition_array[unquoted_state][static_cast('\r')] = carriage_return_state; + transition_array[unquoted_state][static_cast(state_machine_options.delimiter)] = field_separator_state; + if (state_machine_options.quote == state_machine_options.escape) { + transition_array[unquoted_state][static_cast(state_machine_options.escape)] = quoted_state; + } + // 7) Escaped State + transition_array[escape_state][static_cast(state_machine_options.quote)] = quoted_state; + transition_array[escape_state][static_cast(state_machine_options.escape)] = quoted_state; + // 8) Empty Line State + transition_array[empty_line_state][static_cast('\r')] = empty_line_state; + transition_array[empty_line_state][static_cast('\n')] = empty_line_state; +} + +CSVStateMachineCache::CSVStateMachineCache() { + for (auto quoterule : default_quote_rule) { + const auto "e_candidates = default_quote[static_cast(quoterule)]; + for (const auto "e : quote_candidates) { + for (const auto &delimiter : default_delimiter) { + const auto &escape_candidates = default_escape[static_cast(quoterule)]; + for (const auto &escape : escape_candidates) { + Insert({delimiter, quote, escape}); + } + } + } + } +} + +const state_machine_t &CSVStateMachineCache::Get(const CSVStateMachineOptions &state_machine_options) { + //! Custom State Machine, we need to create it and cache it first + if (state_machine_cache.find(state_machine_options) == state_machine_cache.end()) { + Insert(state_machine_options); + } + const auto &transition_array = state_machine_cache[state_machine_options]; + return transition_array; +} +} // namespace duckdb diff --git a/src/duckdb/src/execution/operator/csv_scanner/parallel_csv_reader.cpp b/src/duckdb/src/execution/operator/csv_scanner/parallel_csv_reader.cpp new file mode 100644 index 000000000..4758c5cea --- /dev/null +++ b/src/duckdb/src/execution/operator/csv_scanner/parallel_csv_reader.cpp @@ -0,0 +1,666 @@ +#include "duckdb/execution/operator/scan/csv/parallel_csv_reader.hpp" + +#include "duckdb/catalog/catalog_entry/table_catalog_entry.hpp" +#include "duckdb/common/file_system.hpp" +#include "duckdb/common/string_util.hpp" +#include "duckdb/common/to_string.hpp" +#include "duckdb/common/types/cast_helpers.hpp" +#include "duckdb/common/vector_operations/unary_executor.hpp" +#include "duckdb/common/vector_operations/vector_operations.hpp" +#include "duckdb/function/scalar/strftime_format.hpp" +#include "duckdb/main/database.hpp" +#include "duckdb/parser/column_definition.hpp" +#include "duckdb/storage/data_table.hpp" +#include "utf8proc_wrapper.hpp" +#include "utf8proc.hpp" +#include "duckdb/parser/keyword_helper.hpp" +#include "duckdb/function/table/read_csv.hpp" +#include "duckdb/execution/operator/scan/csv/csv_line_info.hpp" + +#include +#include +#include +#include + +namespace duckdb { + +ParallelCSVReader::ParallelCSVReader(ClientContext &context, CSVReaderOptions options_p, + unique_ptr buffer_p, idx_t first_pos_first_buffer_p, + const vector &requested_types, idx_t file_idx_p) + : BaseCSVReader(context, std::move(options_p), requested_types), file_idx(file_idx_p), + first_pos_first_buffer(first_pos_first_buffer_p) { + Initialize(requested_types); + SetBufferRead(std::move(buffer_p)); +} + +void ParallelCSVReader::Initialize(const vector &requested_types) { + return_types = requested_types; + InitParseChunk(return_types.size()); +} + +bool ParallelCSVReader::NewLineDelimiter(bool carry, bool carry_followed_by_nl, bool first_char) { + // Set the delimiter if not set yet. + SetNewLineDelimiter(carry, carry_followed_by_nl); + D_ASSERT(options.dialect_options.new_line == NewLineIdentifier::SINGLE || + options.dialect_options.new_line == NewLineIdentifier::CARRY_ON); + if (options.dialect_options.new_line == NewLineIdentifier::SINGLE) { + return (!carry) || (carry && !carry_followed_by_nl); + } + return (carry && carry_followed_by_nl) || (!carry && first_char); +} + +void ParallelCSVReader::SkipEmptyLines() { + idx_t new_pos_buffer = position_buffer; + if (parse_chunk.data.size() == 1) { + // Empty lines are null data. + return; + } + for (; new_pos_buffer < end_buffer; new_pos_buffer++) { + if (StringUtil::CharacterIsNewline((*buffer)[new_pos_buffer])) { + bool carrier_return = (*buffer)[new_pos_buffer] == '\r'; + new_pos_buffer++; + if (carrier_return && new_pos_buffer < buffer_size && (*buffer)[new_pos_buffer] == '\n') { + position_buffer++; + } + if (new_pos_buffer > end_buffer) { + return; + } + position_buffer = new_pos_buffer; + } else if ((*buffer)[new_pos_buffer] != ' ') { + return; + } + } +} + +bool ParallelCSVReader::SetPosition() { + if (buffer->buffer->is_first_buffer && start_buffer == position_buffer && start_buffer == first_pos_first_buffer) { + start_buffer = buffer->buffer->start_position; + position_buffer = start_buffer; + verification_positions.beginning_of_first_line = position_buffer; + verification_positions.end_of_last_line = position_buffer; + // First buffer doesn't need any setting + + if (options.dialect_options.header) { + for (; position_buffer < end_buffer; position_buffer++) { + if (StringUtil::CharacterIsNewline((*buffer)[position_buffer])) { + bool carrier_return = (*buffer)[position_buffer] == '\r'; + position_buffer++; + if (carrier_return && position_buffer < buffer_size && (*buffer)[position_buffer] == '\n') { + position_buffer++; + } + if (position_buffer > end_buffer) { + return false; + } + SkipEmptyLines(); + if (verification_positions.beginning_of_first_line == 0) { + verification_positions.beginning_of_first_line = position_buffer; + } + + verification_positions.end_of_last_line = position_buffer; + return true; + } + } + return false; + } + SkipEmptyLines(); + if (verification_positions.beginning_of_first_line == 0) { + verification_positions.beginning_of_first_line = position_buffer; + } + + verification_positions.end_of_last_line = position_buffer; + return true; + } + + // We have to move position up to next new line + idx_t end_buffer_real = end_buffer; + // Check if we already start in a valid line + string error_message; + bool successfully_read_first_line = false; + while (!successfully_read_first_line) { + DataChunk first_line_chunk; + first_line_chunk.Initialize(allocator, return_types); + // Ensure that parse_chunk has no gunk when trying to figure new line + parse_chunk.Reset(); + for (; position_buffer < end_buffer; position_buffer++) { + if (StringUtil::CharacterIsNewline((*buffer)[position_buffer])) { + bool carriage_return = (*buffer)[position_buffer] == '\r'; + bool carriage_return_followed = false; + position_buffer++; + if (position_buffer < end_buffer) { + if (carriage_return && (*buffer)[position_buffer] == '\n') { + carriage_return_followed = true; + position_buffer++; + } + } + if (NewLineDelimiter(carriage_return, carriage_return_followed, position_buffer - 1 == start_buffer)) { + break; + } + } + } + SkipEmptyLines(); + + if (position_buffer > buffer_size) { + break; + } + + if (position_buffer >= end_buffer && !StringUtil::CharacterIsNewline((*buffer)[position_buffer - 1])) { + break; + } + + if (position_buffer > end_buffer && options.dialect_options.new_line == NewLineIdentifier::CARRY_ON && + (*buffer)[position_buffer - 1] == '\n') { + break; + } + idx_t position_set = position_buffer; + start_buffer = position_buffer; + // We check if we can add this line + // disable the projection pushdown while reading the first line + // otherwise the first line parsing can be influenced by which columns we are reading + auto column_ids = std::move(reader_data.column_ids); + auto column_mapping = std::move(reader_data.column_mapping); + InitializeProjection(); + try { + successfully_read_first_line = TryParseSimpleCSV(first_line_chunk, error_message, true); + } catch (...) { + successfully_read_first_line = false; + } + // restore the projection pushdown + reader_data.column_ids = std::move(column_ids); + reader_data.column_mapping = std::move(column_mapping); + end_buffer = end_buffer_real; + start_buffer = position_set; + if (position_buffer >= end_buffer) { + if (successfully_read_first_line) { + position_buffer = position_set; + } + break; + } + position_buffer = position_set; + } + if (verification_positions.beginning_of_first_line == 0) { + verification_positions.beginning_of_first_line = position_buffer; + } + // Ensure that parse_chunk has no gunk when trying to figure new line + parse_chunk.Reset(); + + verification_positions.end_of_last_line = position_buffer; + finished = false; + return successfully_read_first_line; +} + +void ParallelCSVReader::SetBufferRead(unique_ptr buffer_read_p) { + if (!buffer_read_p->buffer) { + throw InternalException("ParallelCSVReader::SetBufferRead - CSVBufferRead does not have a buffer to read"); + } + position_buffer = buffer_read_p->buffer_start; + start_buffer = buffer_read_p->buffer_start; + end_buffer = buffer_read_p->buffer_end; + if (buffer_read_p->next_buffer) { + buffer_size = buffer_read_p->buffer->actual_size + buffer_read_p->next_buffer->actual_size; + } else { + buffer_size = buffer_read_p->buffer->actual_size; + } + buffer = std::move(buffer_read_p); + + reached_remainder_state = false; + verification_positions.beginning_of_first_line = 0; + verification_positions.end_of_last_line = 0; + finished = false; + D_ASSERT(end_buffer <= buffer_size); +} + +VerificationPositions ParallelCSVReader::GetVerificationPositions() { + verification_positions.beginning_of_first_line += buffer->buffer->csv_global_start; + verification_positions.end_of_last_line += buffer->buffer->csv_global_start; + return verification_positions; +} + +// If BufferRemainder returns false, it means we are done scanning this buffer and should go to the end_state +bool ParallelCSVReader::BufferRemainder() { + if (position_buffer >= end_buffer && !reached_remainder_state) { + // First time we finish the buffer piece we should scan here, we set the variables + // to allow this piece to be scanned up to the end of the buffer or the next new line + reached_remainder_state = true; + // end_buffer is allowed to go to buffer size to finish its last line + end_buffer = buffer_size; + } + if (position_buffer >= end_buffer) { + // buffer ends, return false + return false; + } + // we can still scan stuff, return true + return true; +} + +bool AllNewLine(string_t value, idx_t column_amount) { + auto value_str = value.GetString(); + if (value_str.empty() && column_amount == 1) { + // This is a one column (empty) + return false; + } + for (idx_t i = 0; i < value.GetSize(); i++) { + if (!StringUtil::CharacterIsNewline(value_str[i])) { + return false; + } + } + return true; +} + +bool ParallelCSVReader::TryParseSimpleCSV(DataChunk &insert_chunk, string &error_message, bool try_add_line) { + // If line is not set, we have to figure it out, we assume whatever is in the first line + if (options.dialect_options.new_line == NewLineIdentifier::NOT_SET) { + idx_t cur_pos = position_buffer; + // we can start in the middle of a new line, so move a bit forward. + while (cur_pos < end_buffer) { + if (StringUtil::CharacterIsNewline((*buffer)[cur_pos])) { + cur_pos++; + } else { + break; + } + } + for (; cur_pos < end_buffer; cur_pos++) { + if (StringUtil::CharacterIsNewline((*buffer)[cur_pos])) { + bool carriage_return = (*buffer)[cur_pos] == '\r'; + bool carriage_return_followed = false; + cur_pos++; + if (cur_pos < end_buffer) { + if (carriage_return && (*buffer)[cur_pos] == '\n') { + carriage_return_followed = true; + cur_pos++; + } + } + SetNewLineDelimiter(carriage_return, carriage_return_followed); + break; + } + } + } + // used for parsing algorithm + if (start_buffer == buffer_size) { + // Nothing to read + finished = true; + return true; + } + D_ASSERT(end_buffer <= buffer_size); + bool finished_chunk = false; + idx_t column = 0; + idx_t offset = 0; + bool has_quotes = false; + + vector escape_positions; + if ((start_buffer == buffer->buffer_start || start_buffer == buffer->buffer_end) && !try_add_line) { + // First time reading this buffer piece + if (!SetPosition()) { + finished = true; + return true; + } + } + if (position_buffer == buffer_size) { + // Nothing to read + finished = true; + return true; + } + // Keep track of line size + idx_t line_start = position_buffer; + // start parsing the first value + goto value_start; + +value_start : { + /* state: value_start */ + if (!BufferRemainder()) { + goto final_state; + } + offset = 0; + + // this state parses the first character of a value + if ((*buffer)[position_buffer] == options.dialect_options.state_machine_options.quote) { + // quote: actual value starts in the next position + // move to in_quotes state + start_buffer = position_buffer + 1; + goto in_quotes; + } else { + // no quote, move to normal parsing state + start_buffer = position_buffer; + goto normal; + } +}; + +normal : { + /* state: normal parsing state */ + // this state parses the remainder of a non-quoted value until we reach a delimiter or newline + for (; position_buffer < end_buffer; position_buffer++) { + auto c = (*buffer)[position_buffer]; + if (c == options.dialect_options.state_machine_options.delimiter) { + // delimiter: end the value and add it to the chunk + goto add_value; + } else if (c == options.dialect_options.state_machine_options.quote && try_add_line) { + return false; + } else if (StringUtil::CharacterIsNewline(c)) { + // newline: add row + if (column > 0 || try_add_line || parse_chunk.data.size() == 1) { + goto add_row; + } + if (column == 0 && position_buffer == start_buffer) { + start_buffer++; + } + } + } + if (!BufferRemainder()) { + goto final_state; + } else { + goto normal; + } +}; + +add_value : { + /* state: Add value to string vector */ + AddValue(buffer->GetValue(start_buffer, position_buffer, offset), column, escape_positions, has_quotes, + buffer->local_batch_index); + // increase position by 1 and move start to the new position + offset = 0; + has_quotes = false; + start_buffer = ++position_buffer; + if (!BufferRemainder()) { + goto final_state; + } + goto value_start; +}; + +add_row : { + /* state: Add Row to Parse chunk */ + // check type of newline (\r or \n) + bool carriage_return = (*buffer)[position_buffer] == '\r'; + + AddValue(buffer->GetValue(start_buffer, position_buffer, offset), column, escape_positions, has_quotes, + buffer->local_batch_index); + if (try_add_line) { + bool success = column == insert_chunk.ColumnCount(); + if (success) { + idx_t cur_linenr = linenr; + AddRow(insert_chunk, column, error_message, buffer->local_batch_index); + success = Flush(insert_chunk, buffer->local_batch_index, true); + linenr = cur_linenr; + } + reached_remainder_state = false; + parse_chunk.Reset(); + return success; + } else { + VerifyLineLength(position_buffer - line_start, buffer->batch_index); + line_start = position_buffer; + finished_chunk = AddRow(insert_chunk, column, error_message, buffer->local_batch_index); + } + // increase position by 1 and move start to the new position + offset = 0; + has_quotes = false; + position_buffer++; + start_buffer = position_buffer; + verification_positions.end_of_last_line = position_buffer; + if (carriage_return) { + // \r newline, go to special state that parses an optional \n afterwards + // optionally skips a newline (\n) character, which allows \r\n to be interpreted as a single line + if (!BufferRemainder()) { + goto final_state; + } + if ((*buffer)[position_buffer] == '\n') { + if (options.dialect_options.new_line == NewLineIdentifier::SINGLE) { + error_message = "Wrong NewLine Identifier. Expecting \\r\\n"; + return false; + } + // newline after carriage return: skip + // increase position by 1 and move start to the new position + start_buffer = ++position_buffer; + + SkipEmptyLines(); + verification_positions.end_of_last_line = position_buffer; + start_buffer = position_buffer; + if (reached_remainder_state) { + goto final_state; + } + } else { + if (options.dialect_options.new_line == NewLineIdentifier::CARRY_ON) { + error_message = "Wrong NewLine Identifier. Expecting \\r or \\n"; + return false; + } + } + if (!BufferRemainder()) { + goto final_state; + } + if (reached_remainder_state || finished_chunk) { + goto final_state; + } + goto value_start; + } else { + if (options.dialect_options.new_line == NewLineIdentifier::CARRY_ON) { + error_message = "Wrong NewLine Identifier. Expecting \\r or \\n"; + return false; + } + if (reached_remainder_state) { + goto final_state; + } + if (!BufferRemainder()) { + goto final_state; + } + SkipEmptyLines(); + if (position_buffer - verification_positions.end_of_last_line > options.buffer_size) { + error_message = "Line does not fit in one buffer. Increase the buffer size."; + return false; + } + verification_positions.end_of_last_line = position_buffer; + start_buffer = position_buffer; + // \n newline, move to value start + if (finished_chunk) { + goto final_state; + } + goto value_start; + } +} +in_quotes: + /* state: in_quotes this state parses the remainder of a quoted value*/ + has_quotes = true; + position_buffer++; + for (; position_buffer < end_buffer; position_buffer++) { + auto c = (*buffer)[position_buffer]; + if (c == options.dialect_options.state_machine_options.quote) { + // quote: move to unquoted state + goto unquote; + } else if (c == options.dialect_options.state_machine_options.escape) { + // escape: store the escaped position and move to handle_escape state + escape_positions.push_back(position_buffer - start_buffer); + goto handle_escape; + } + } + if (!BufferRemainder()) { + if (buffer->buffer->is_last_buffer) { + if (try_add_line) { + return false; + } + // still in quoted state at the end of the file or at the end of a buffer when running multithreaded, error: + throw InvalidInputException("Error in file \"%s\" on line %s: unterminated quotes. (%s)", options.file_path, + GetLineNumberStr(linenr, linenr_estimated, buffer->local_batch_index).c_str(), + options.ToString()); + } else { + goto final_state; + } + } else { + position_buffer--; + goto in_quotes; + } + +unquote : { + /* state: unquote: this state handles the state directly after we unquote*/ + // + // in this state we expect either another quote (entering the quoted state again, and escaping the quote) + // or a delimiter/newline, ending the current value and moving on to the next value + position_buffer++; + if (!BufferRemainder()) { + offset = 1; + goto final_state; + } + auto c = (*buffer)[position_buffer]; + if (c == options.dialect_options.state_machine_options.quote && + (options.dialect_options.state_machine_options.escape == '\0' || + options.dialect_options.state_machine_options.escape == options.dialect_options.state_machine_options.quote)) { + // escaped quote, return to quoted state and store escape position + escape_positions.push_back(position_buffer - start_buffer); + goto in_quotes; + } else if (c == options.dialect_options.state_machine_options.delimiter) { + // delimiter, add value + offset = 1; + goto add_value; + } else if (StringUtil::CharacterIsNewline(c)) { + offset = 1; + // FIXME: should this be an assertion? + D_ASSERT(try_add_line || (!try_add_line && column == parse_chunk.ColumnCount() - 1)); + goto add_row; + } else if (position_buffer >= end_buffer) { + // reached end of buffer + offset = 1; + goto final_state; + } else { + error_message = StringUtil::Format( + "Error in file \"%s\" on line %s: quote should be followed by end of value, end of " + "row or another quote. (%s). ", + options.file_path, GetLineNumberStr(linenr, linenr_estimated, buffer->local_batch_index).c_str(), + options.ToString()); + return false; + } +} +handle_escape : { + /* state: handle_escape */ + // escape should be followed by a quote or another escape character + position_buffer++; + if (!BufferRemainder()) { + goto final_state; + } + if (position_buffer >= buffer_size && buffer->buffer->is_last_buffer) { + error_message = StringUtil::Format( + "Error in file \"%s\" on line %s: neither QUOTE nor ESCAPE is proceeded by ESCAPE. (%s)", options.file_path, + GetLineNumberStr(linenr, linenr_estimated, buffer->local_batch_index).c_str(), options.ToString()); + return false; + } + if ((*buffer)[position_buffer] != options.dialect_options.state_machine_options.quote && + (*buffer)[position_buffer] != options.dialect_options.state_machine_options.escape) { + error_message = StringUtil::Format( + "Error in file \"%s\" on line %s: neither QUOTE nor ESCAPE is proceeded by ESCAPE. (%s)", options.file_path, + GetLineNumberStr(linenr, linenr_estimated, buffer->local_batch_index).c_str(), options.ToString()); + return false; + } + // escape was followed by quote or escape, go back to quoted state + goto in_quotes; +} +final_state : { + /* state: final_stage reached after we finished reading the end_buffer of the csv buffer */ + // reset end buffer + end_buffer = buffer->buffer_end; + if (position_buffer == end_buffer) { + reached_remainder_state = false; + } + if (finished_chunk) { + if (position_buffer >= end_buffer) { + if (position_buffer == end_buffer && StringUtil::CharacterIsNewline((*buffer)[position_buffer - 1]) && + position_buffer < buffer_size) { + // last position is a new line, we still have to go through one more line of this buffer + finished = false; + } else { + finished = true; + } + } + buffer->lines_read += insert_chunk.size(); + return true; + } + // If this is the last buffer, we have to read the last value + if (buffer->buffer->is_last_buffer || !buffer->next_buffer || + (buffer->next_buffer && buffer->next_buffer->is_last_buffer)) { + if (column > 0 || start_buffer != position_buffer || try_add_line || + (insert_chunk.data.size() == 1 && start_buffer != position_buffer)) { + // remaining values to be added to the chunk + auto str_value = buffer->GetValue(start_buffer, position_buffer, offset); + if (!AllNewLine(str_value, insert_chunk.data.size()) || offset == 0) { + AddValue(str_value, column, escape_positions, has_quotes, buffer->local_batch_index); + if (try_add_line) { + bool success = column == return_types.size(); + if (success) { + auto cur_linenr = linenr; + AddRow(insert_chunk, column, error_message, buffer->local_batch_index); + success = Flush(insert_chunk, buffer->local_batch_index); + linenr = cur_linenr; + } + parse_chunk.Reset(); + reached_remainder_state = false; + return success; + } else { + VerifyLineLength(position_buffer - line_start, buffer->batch_index); + line_start = position_buffer; + AddRow(insert_chunk, column, error_message, buffer->local_batch_index); + if (position_buffer - verification_positions.end_of_last_line > options.buffer_size) { + error_message = "Line does not fit in one buffer. Increase the buffer size."; + return false; + } + verification_positions.end_of_last_line = position_buffer; + } + } + } + } + // flush the parsed chunk and finalize parsing + if (mode == ParserMode::PARSING) { + Flush(insert_chunk, buffer->local_batch_index); + buffer->lines_read += insert_chunk.size(); + } + if (position_buffer - verification_positions.end_of_last_line > options.buffer_size) { + error_message = "Line does not fit in one buffer. Increase the buffer size."; + return false; + } + end_buffer = buffer_size; + SkipEmptyLines(); + end_buffer = buffer->buffer_end; + verification_positions.end_of_last_line = position_buffer; + if (position_buffer >= end_buffer) { + if (position_buffer >= end_buffer) { + if (position_buffer == end_buffer && StringUtil::CharacterIsNewline((*buffer)[position_buffer - 1]) && + position_buffer < buffer_size) { + // last position is a new line, we still have to go through one more line of this buffer + finished = false; + } else { + finished = true; + } + } + } + return true; +}; +} + +void ParallelCSVReader::ParseCSV(DataChunk &insert_chunk) { + string error_message; + if (!TryParseCSV(ParserMode::PARSING, insert_chunk, error_message)) { + throw InvalidInputException(error_message); + } +} + +idx_t ParallelCSVReader::GetLineError(idx_t line_error, idx_t buffer_idx, bool stop_at_first) { + while (true) { + if (buffer->line_info->CanItGetLine(file_idx, buffer_idx)) { + auto cur_start = verification_positions.beginning_of_first_line + buffer->buffer->csv_global_start; + return buffer->line_info->GetLine(buffer_idx, line_error, file_idx, cur_start, false, stop_at_first); + } + } +} + +bool ParallelCSVReader::TryParseCSV(ParserMode mode) { + DataChunk dummy_chunk; + string error_message; + return TryParseCSV(mode, dummy_chunk, error_message); +} + +void ParallelCSVReader::ParseCSV(ParserMode mode) { + DataChunk dummy_chunk; + string error_message; + if (!TryParseCSV(mode, dummy_chunk, error_message)) { + throw InvalidInputException(error_message); + } +} + +bool ParallelCSVReader::TryParseCSV(ParserMode parser_mode, DataChunk &insert_chunk, string &error_message) { + mode = parser_mode; + return TryParseSimpleCSV(insert_chunk, error_message); +} + +} // namespace duckdb diff --git a/src/duckdb/src/execution/operator/csv_scanner/sniffer/csv_sniffer.cpp b/src/duckdb/src/execution/operator/csv_scanner/sniffer/csv_sniffer.cpp new file mode 100644 index 000000000..d6c63f7e5 --- /dev/null +++ b/src/duckdb/src/execution/operator/csv_scanner/sniffer/csv_sniffer.cpp @@ -0,0 +1,52 @@ +#include "duckdb/execution/operator/scan/csv/csv_sniffer.hpp" + +namespace duckdb { + +CSVSniffer::CSVSniffer(CSVReaderOptions &options_p, shared_ptr buffer_manager_p, + CSVStateMachineCache &state_machine_cache_p) + : state_machine_cache(state_machine_cache_p), options(options_p), buffer_manager(std::move(buffer_manager_p)) { + + // Check if any type is BLOB + for (auto &type : options.sql_type_list) { + if (type.id() == LogicalTypeId::BLOB) { + throw InvalidInputException( + "CSV auto-detect for blobs not supported: there may be invalid UTF-8 in the file"); + } + } + + // Initialize Format Candidates + for (const auto &format_template : format_template_candidates) { + auto &logical_type = format_template.first; + best_format_candidates[logical_type].clear(); + } +} + +SnifferResult CSVSniffer::SniffCSV() { + // 1. Dialect Detection + DetectDialect(); + // 2. Type Detection + DetectTypes(); + // 3. Header Detection + DetectHeader(); + D_ASSERT(best_sql_types_candidates_per_column_idx.size() == names.size()); + // 4. Type Replacement + ReplaceTypes(); + // 5. Type Refinement + RefineTypes(); + // We are done, construct and return the result. + + // Set the CSV Options in the reference + options.dialect_options = best_candidate->dialect_options; + options.has_header = best_candidate->dialect_options.header; + options.skip_rows_set = options.dialect_options.skip_rows > 0; + if (options.has_header) { + options.dialect_options.true_start = best_start_with_header; + } else { + options.dialect_options.true_start = best_start_without_header; + } + + // Return the types and names + return SnifferResult(detected_types, names); +} + +} // namespace duckdb diff --git a/src/duckdb/src/execution/operator/csv_scanner/sniffer/dialect_detection.cpp b/src/duckdb/src/execution/operator/csv_scanner/sniffer/dialect_detection.cpp new file mode 100644 index 000000000..e600dc359 --- /dev/null +++ b/src/duckdb/src/execution/operator/csv_scanner/sniffer/dialect_detection.cpp @@ -0,0 +1,336 @@ +#include "duckdb/execution/operator/scan/csv/csv_sniffer.hpp" +#include "duckdb/main/client_data.hpp" + +namespace duckdb { + +struct SniffDialect { + inline static void Initialize(CSVStateMachine &machine) { + machine.state = CSVState::STANDARD; + machine.previous_state = CSVState::STANDARD; + machine.pre_previous_state = CSVState::STANDARD; + machine.cur_rows = 0; + machine.column_count = 1; + } + + inline static bool Process(CSVStateMachine &machine, vector &sniffed_column_counts, char current_char, + idx_t current_pos) { + + D_ASSERT(sniffed_column_counts.size() == machine.options.sample_chunk_size); + + if (machine.state == CSVState::INVALID) { + sniffed_column_counts.clear(); + return true; + } + machine.pre_previous_state = machine.previous_state; + machine.previous_state = machine.state; + + machine.state = static_cast( + machine.transition_array[static_cast(machine.state)][static_cast(current_char)]); + + bool carriage_return = machine.previous_state == CSVState::CARRIAGE_RETURN; + machine.column_count += machine.previous_state == CSVState::DELIMITER; + sniffed_column_counts[machine.cur_rows] = machine.column_count; + machine.cur_rows += + machine.previous_state == CSVState::RECORD_SEPARATOR && machine.state != CSVState::EMPTY_LINE; + machine.column_count -= (machine.column_count - 1) * (machine.previous_state == CSVState::RECORD_SEPARATOR); + + // It means our carriage return is actually a record separator + machine.cur_rows += machine.state != CSVState::RECORD_SEPARATOR && carriage_return; + machine.column_count -= + (machine.column_count - 1) * (machine.state != CSVState::RECORD_SEPARATOR && carriage_return); + + // Identify what is our line separator + machine.carry_on_separator = + (machine.state == CSVState::RECORD_SEPARATOR && carriage_return) || machine.carry_on_separator; + machine.single_record_separator = ((machine.state != CSVState::RECORD_SEPARATOR && carriage_return) || + (machine.state == CSVState::RECORD_SEPARATOR && !carriage_return)) || + machine.single_record_separator; + if (machine.cur_rows >= machine.options.sample_chunk_size) { + // We sniffed enough rows + return true; + } + return false; + } + inline static void Finalize(CSVStateMachine &machine, vector &sniffed_column_counts) { + if (machine.state == CSVState::INVALID) { + return; + } + if (machine.cur_rows < machine.options.sample_chunk_size && machine.state != CSVState::EMPTY_LINE) { + sniffed_column_counts[machine.cur_rows++] = machine.column_count; + } + NewLineIdentifier suggested_newline; + if (machine.carry_on_separator) { + if (machine.single_record_separator) { + suggested_newline = NewLineIdentifier::MIX; + } else { + suggested_newline = NewLineIdentifier::CARRY_ON; + } + } else { + suggested_newline = NewLineIdentifier::SINGLE; + } + if (machine.options.dialect_options.new_line == NewLineIdentifier::NOT_SET) { + machine.dialect_options.new_line = suggested_newline; + } else { + if (machine.options.dialect_options.new_line != suggested_newline) { + // Invalidate this whole detection + machine.cur_rows = 0; + } + } + sniffed_column_counts.erase(sniffed_column_counts.begin() + machine.cur_rows, sniffed_column_counts.end()); + } +}; + +void CSVSniffer::GenerateCandidateDetectionSearchSpace(vector &delim_candidates, + vector "erule_candidates, + unordered_map> "e_candidates_map, + unordered_map> &escape_candidates_map) { + if (options.has_delimiter) { + // user provided a delimiter: use that delimiter + delim_candidates = {options.dialect_options.state_machine_options.delimiter}; + } else { + // no delimiter provided: try standard/common delimiters + delim_candidates = {',', '|', ';', '\t'}; + } + if (options.has_quote) { + // user provided quote: use that quote rule + quote_candidates_map[(uint8_t)QuoteRule::QUOTES_RFC] = {options.dialect_options.state_machine_options.quote}; + quote_candidates_map[(uint8_t)QuoteRule::QUOTES_OTHER] = {options.dialect_options.state_machine_options.quote}; + quote_candidates_map[(uint8_t)QuoteRule::NO_QUOTES] = {options.dialect_options.state_machine_options.quote}; + } else { + // no quote rule provided: use standard/common quotes + quote_candidates_map[(uint8_t)QuoteRule::QUOTES_RFC] = {'\"'}; + quote_candidates_map[(uint8_t)QuoteRule::QUOTES_OTHER] = {'\"', '\''}; + quote_candidates_map[(uint8_t)QuoteRule::NO_QUOTES] = {'\0'}; + } + if (options.has_escape) { + // user provided escape: use that escape rule + if (options.dialect_options.state_machine_options.escape == '\0') { + quoterule_candidates = {QuoteRule::QUOTES_RFC}; + } else { + quoterule_candidates = {QuoteRule::QUOTES_OTHER}; + } + escape_candidates_map[(uint8_t)quoterule_candidates[0]] = { + options.dialect_options.state_machine_options.escape}; + } else { + // no escape provided: try standard/common escapes + quoterule_candidates = {QuoteRule::QUOTES_RFC, QuoteRule::QUOTES_OTHER, QuoteRule::NO_QUOTES}; + } +} + +void CSVSniffer::GenerateStateMachineSearchSpace(vector> &csv_state_machines, + const vector &delimiter_candidates, + const vector "erule_candidates, + const unordered_map> "e_candidates_map, + const unordered_map> &escape_candidates_map) { + // Generate state machines for all option combinations + for (const auto quoterule : quoterule_candidates) { + const auto "e_candidates = quote_candidates_map.at((uint8_t)quoterule); + for (const auto "e : quote_candidates) { + for (const auto &delimiter : delimiter_candidates) { + const auto &escape_candidates = escape_candidates_map.at((uint8_t)quoterule); + for (const auto &escape : escape_candidates) { + D_ASSERT(buffer_manager); + CSVStateMachineOptions state_machine_options(delimiter, quote, escape); + csv_state_machines.emplace_back(make_uniq(options, state_machine_options, + buffer_manager, state_machine_cache)); + } + } + } + } +} + +void CSVSniffer::AnalyzeDialectCandidate(unique_ptr state_machine, idx_t &rows_read, + idx_t &best_consistent_rows, idx_t &prev_padding_count) { + // The sniffed_column_counts variable keeps track of the number of columns found for each row + vector sniffed_column_counts(options.sample_chunk_size); + + state_machine->csv_buffer_iterator.Process(*state_machine, sniffed_column_counts); + idx_t start_row = options.dialect_options.skip_rows; + idx_t consistent_rows = 0; + idx_t num_cols = sniffed_column_counts.empty() ? 0 : sniffed_column_counts[0]; + idx_t padding_count = 0; + bool allow_padding = options.null_padding; + if (sniffed_column_counts.size() > rows_read) { + rows_read = sniffed_column_counts.size(); + } + for (idx_t row = 0; row < sniffed_column_counts.size(); row++) { + if (sniffed_column_counts[row] == num_cols) { + consistent_rows++; + } else if (num_cols < sniffed_column_counts[row] && !options.skip_rows_set) { + // all rows up to this point will need padding + padding_count = 0; + // we use the maximum amount of num_cols that we find + num_cols = sniffed_column_counts[row]; + start_row = row + options.dialect_options.skip_rows; + consistent_rows = 1; + + } else if (num_cols >= sniffed_column_counts[row]) { + // we are missing some columns, we can parse this as long as we add padding + padding_count++; + } + } + + // Calculate the total number of consistent rows after adding padding. + consistent_rows += padding_count; + + // Whether there are more values (rows) available that are consistent, exceeding the current best. + bool more_values = (consistent_rows > best_consistent_rows && num_cols >= max_columns_found); + + // If additional padding is required when compared to the previous padding count. + bool require_more_padding = padding_count > prev_padding_count; + + // If less padding is now required when compared to the previous padding count. + bool require_less_padding = padding_count < prev_padding_count; + + // If there was only a single column before, and the new number of columns exceeds that. + bool single_column_before = max_columns_found < 2 && num_cols > max_columns_found; + + // If the number of rows is consistent with the calculated value after accounting for skipped rows and the + // start row. + bool rows_consistent = + start_row + consistent_rows - options.dialect_options.skip_rows == sniffed_column_counts.size(); + + // If there are more than one consistent row. + bool more_than_one_row = (consistent_rows > 1); + + // If there are more than one column. + bool more_than_one_column = (num_cols > 1); + + // If the start position is valid. + bool start_good = !candidates.empty() && (start_row <= candidates.front()->start_row); + + // If padding happened but it is not allowed. + bool invalid_padding = !allow_padding && padding_count > 0; + + // If rows are consistent and no invalid padding happens, this is the best suitable candidate if one of the + // following is valid: + // - There's a single column before. + // - There are more values and no additional padding is required. + // - There's more than one column and less padding is required. + if (rows_consistent && + (single_column_before || (more_values && !require_more_padding) || + (more_than_one_column && require_less_padding)) && + !invalid_padding) { + best_consistent_rows = consistent_rows; + max_columns_found = num_cols; + prev_padding_count = padding_count; + state_machine->start_row = start_row; + candidates.clear(); + state_machine->dialect_options.num_cols = num_cols; + candidates.emplace_back(std::move(state_machine)); + return; + } + // If there's more than one row and column, the start is good, rows are consistent, + // no additional padding is required, and there is no invalid padding, and there is not yet a candidate + // with the same quote, we add this state_machine as a suitable candidate. + if (more_than_one_row && more_than_one_column && start_good && rows_consistent && !require_more_padding && + !invalid_padding) { + bool same_quote_is_candidate = false; + for (auto &candidate : candidates) { + if (state_machine->dialect_options.state_machine_options.quote == + candidate->dialect_options.state_machine_options.quote) { + same_quote_is_candidate = true; + } + } + if (!same_quote_is_candidate) { + state_machine->start_row = start_row; + state_machine->dialect_options.num_cols = num_cols; + candidates.emplace_back(std::move(state_machine)); + } + } +} + +bool CSVSniffer::RefineCandidateNextChunk(CSVStateMachine &candidate) { + vector sniffed_column_counts(options.sample_chunk_size); + candidate.csv_buffer_iterator.Process(candidate, sniffed_column_counts); + bool allow_padding = options.null_padding; + + for (idx_t row = 0; row < sniffed_column_counts.size(); row++) { + if (max_columns_found != sniffed_column_counts[row] && !allow_padding) { + return false; + } + } + return true; +} + +void CSVSniffer::RefineCandidates() { + // It's very frequent that more than one dialect can parse a csv file, hence here we run one state machine + // fully on the whole sample dataset, when/if it fails we go to the next one. + if (candidates.empty()) { + // No candidates to refine + return; + } + if (candidates.size() == 1 || candidates[0]->csv_buffer_iterator.Finished()) { + // Only one candidate nothing to refine or all candidates already checked + return; + } + for (auto &cur_candidate : candidates) { + for (idx_t i = 1; i <= options.sample_chunks; i++) { + bool finished_file = cur_candidate->csv_buffer_iterator.Finished(); + if (finished_file || i == options.sample_chunks) { + // we finished the file or our chunk sample successfully: stop + auto successful_candidate = std::move(cur_candidate); + candidates.clear(); + candidates.emplace_back(std::move(successful_candidate)); + return; + } + cur_candidate->cur_rows = 0; + cur_candidate->column_count = 1; + if (!RefineCandidateNextChunk(*cur_candidate)) { + // This candidate failed, move to the next one + break; + } + } + } + candidates.clear(); + return; +} + +// Dialect Detection consists of five steps: +// 1. Generate a search space of all possible dialects +// 2. Generate a state machine for each dialect +// 3. Analyze the first chunk of the file and find the best dialect candidates +// 4. Analyze the remaining chunks of the file and find the best dialect candidate +void CSVSniffer::DetectDialect() { + // Variables for Dialect Detection + // Candidates for the delimiter + vector delim_candidates; + // Quote-Rule Candidates + vector quoterule_candidates; + // Candidates for the quote option + unordered_map> quote_candidates_map; + // Candidates for the escape option + unordered_map> escape_candidates_map; + escape_candidates_map[(uint8_t)QuoteRule::QUOTES_RFC] = {'\0', '\"', '\''}; + escape_candidates_map[(uint8_t)QuoteRule::QUOTES_OTHER] = {'\\'}; + escape_candidates_map[(uint8_t)QuoteRule::NO_QUOTES] = {'\0'}; + // Number of rows read + idx_t rows_read = 0; + // Best Number of consistent rows (i.e., presenting all columns) + idx_t best_consistent_rows = 0; + // If padding was necessary (i.e., rows are missing some columns, how many) + idx_t prev_padding_count = 0; + // Vector of CSV State Machines + vector> csv_state_machines; + + // Step 1: Generate search space + GenerateCandidateDetectionSearchSpace(delim_candidates, quoterule_candidates, quote_candidates_map, + escape_candidates_map); + // Step 2: Generate state machines + GenerateStateMachineSearchSpace(csv_state_machines, delim_candidates, quoterule_candidates, quote_candidates_map, + escape_candidates_map); + // Step 3: Analyze all candidates on the first chunk + for (auto &state_machine : csv_state_machines) { + state_machine->Reset(); + AnalyzeDialectCandidate(std::move(state_machine), rows_read, best_consistent_rows, prev_padding_count); + } + // Step 4: Loop over candidates and find if they can still produce good results for the remaining chunks + RefineCandidates(); + // if no dialect candidate was found, we throw an exception + if (candidates.empty()) { + throw InvalidInputException( + "Error in file \"%s\": CSV options could not be auto-detected. Consider setting parser options manually.", + options.file_path); + } +} +} // namespace duckdb diff --git a/src/duckdb/src/execution/operator/csv_scanner/sniffer/header_detection.cpp b/src/duckdb/src/execution/operator/csv_scanner/sniffer/header_detection.cpp new file mode 100644 index 000000000..d43a23a4b --- /dev/null +++ b/src/duckdb/src/execution/operator/csv_scanner/sniffer/header_detection.cpp @@ -0,0 +1,165 @@ +#include "duckdb/common/types/cast_helpers.hpp" +#include "duckdb/execution/operator/scan/csv/csv_sniffer.hpp" +#include "utf8proc.hpp" + +namespace duckdb { + +// Helper function to generate column names +static string GenerateColumnName(const idx_t total_cols, const idx_t col_number, const string &prefix = "column") { + int max_digits = NumericHelper::UnsignedLength(total_cols - 1); + int digits = NumericHelper::UnsignedLength(col_number); + string leading_zeros = string(max_digits - digits, '0'); + string value = to_string(col_number); + return string(prefix + leading_zeros + value); +} + +// Helper function for UTF-8 aware space trimming +static string TrimWhitespace(const string &col_name) { + utf8proc_int32_t codepoint; + auto str = reinterpret_cast(col_name.c_str()); + idx_t size = col_name.size(); + // Find the first character that is not left trimmed + idx_t begin = 0; + while (begin < size) { + auto bytes = utf8proc_iterate(str + begin, size - begin, &codepoint); + D_ASSERT(bytes > 0); + if (utf8proc_category(codepoint) != UTF8PROC_CATEGORY_ZS) { + break; + } + begin += bytes; + } + + // Find the last character that is not right trimmed + idx_t end; + end = begin; + for (auto next = begin; next < col_name.size();) { + auto bytes = utf8proc_iterate(str + next, size - next, &codepoint); + D_ASSERT(bytes > 0); + next += bytes; + if (utf8proc_category(codepoint) != UTF8PROC_CATEGORY_ZS) { + end = next; + } + } + + // return the trimmed string + return col_name.substr(begin, end - begin); +} + +static string NormalizeColumnName(const string &col_name) { + // normalize UTF8 characters to NFKD + auto nfkd = utf8proc_NFKD(reinterpret_cast(col_name.c_str()), col_name.size()); + const string col_name_nfkd = string(const_char_ptr_cast(nfkd), strlen(const_char_ptr_cast(nfkd))); + free(nfkd); + + // only keep ASCII characters 0-9 a-z A-Z and replace spaces with regular whitespace + string col_name_ascii = ""; + for (idx_t i = 0; i < col_name_nfkd.size(); i++) { + if (col_name_nfkd[i] == '_' || (col_name_nfkd[i] >= '0' && col_name_nfkd[i] <= '9') || + (col_name_nfkd[i] >= 'A' && col_name_nfkd[i] <= 'Z') || + (col_name_nfkd[i] >= 'a' && col_name_nfkd[i] <= 'z')) { + col_name_ascii += col_name_nfkd[i]; + } else if (StringUtil::CharacterIsSpace(col_name_nfkd[i])) { + col_name_ascii += " "; + } + } + + // trim whitespace and replace remaining whitespace by _ + string col_name_trimmed = TrimWhitespace(col_name_ascii); + string col_name_cleaned = ""; + bool in_whitespace = false; + for (idx_t i = 0; i < col_name_trimmed.size(); i++) { + if (col_name_trimmed[i] == ' ') { + if (!in_whitespace) { + col_name_cleaned += "_"; + in_whitespace = true; + } + } else { + col_name_cleaned += col_name_trimmed[i]; + in_whitespace = false; + } + } + + // don't leave string empty; if not empty, make lowercase + if (col_name_cleaned.empty()) { + col_name_cleaned = "_"; + } else { + col_name_cleaned = StringUtil::Lower(col_name_cleaned); + } + + // prepend _ if name starts with a digit or is a reserved keyword + if (KeywordHelper::IsKeyword(col_name_cleaned) || (col_name_cleaned[0] >= '0' && col_name_cleaned[0] <= '9')) { + col_name_cleaned = "_" + col_name_cleaned; + } + return col_name_cleaned; +} +void CSVSniffer::DetectHeader() { + // information for header detection + bool first_row_consistent = true; + // check if header row is all null and/or consistent with detected column data types + bool first_row_nulls = true; + // This case will fail in dialect detection, so we assert here just for sanity + D_ASSERT(best_candidate->options.null_padding || + best_sql_types_candidates_per_column_idx.size() == best_header_row.size()); + for (idx_t col = 0; col < best_header_row.size(); col++) { + auto dummy_val = best_header_row[col]; + if (!dummy_val.IsNull()) { + first_row_nulls = false; + } + + // try cast to sql_type of column + const auto &sql_type = best_sql_types_candidates_per_column_idx[col].back(); + if (!TryCastValue(*best_candidate, dummy_val, sql_type)) { + first_row_consistent = false; + } + } + bool has_header; + if (!best_candidate->options.has_header) { + has_header = !first_row_consistent || first_row_nulls; + } else { + has_header = best_candidate->options.dialect_options.header; + } + // update parser info, and read, generate & set col_names based on previous findings + if (has_header) { + best_candidate->dialect_options.header = true; + case_insensitive_map_t name_collision_count; + + // get header names from CSV + for (idx_t col = 0; col < best_header_row.size(); col++) { + const auto &val = best_header_row[col]; + string col_name = val.ToString(); + + // generate name if field is empty + if (col_name.empty() || val.IsNull()) { + col_name = GenerateColumnName(best_candidate->dialect_options.num_cols, col); + } + + // normalize names or at least trim whitespace + if (best_candidate->options.normalize_names) { + col_name = NormalizeColumnName(col_name); + } else { + col_name = TrimWhitespace(col_name); + } + + // avoid duplicate header names + while (name_collision_count.find(col_name) != name_collision_count.end()) { + name_collision_count[col_name] += 1; + col_name = col_name + "_" + to_string(name_collision_count[col_name]); + } + names.push_back(col_name); + name_collision_count[col_name] = 0; + } + + } else { + best_candidate->dialect_options.header = false; + for (idx_t col = 0; col < best_candidate->dialect_options.num_cols; col++) { + string column_name = GenerateColumnName(best_candidate->dialect_options.num_cols, col); + names.push_back(column_name); + } + } + + // If the user provided names, we must replace our header with the user provided names + for (idx_t i = 0; i < MinValue(names.size(), best_candidate->options.name_list.size()); i++) { + names[i] = best_candidate->options.name_list[i]; + } +} +} // namespace duckdb diff --git a/src/duckdb/src/execution/operator/csv_scanner/sniffer/type_detection.cpp b/src/duckdb/src/execution/operator/csv_scanner/sniffer/type_detection.cpp new file mode 100644 index 000000000..eade5d6c2 --- /dev/null +++ b/src/duckdb/src/execution/operator/csv_scanner/sniffer/type_detection.cpp @@ -0,0 +1,398 @@ +#include "duckdb/common/operator/decimal_cast_operators.hpp" +#include "duckdb/execution/operator/scan/csv/csv_sniffer.hpp" +#include "duckdb/common/algorithm.hpp" +#include "duckdb/common/string.hpp" + +namespace duckdb { +struct TryCastFloatingOperator { + template + static bool Operation(string_t input) { + T result; + string error_message; + return OP::Operation(input, result, &error_message); + } +}; + +struct TupleSniffing { + idx_t line_number; + idx_t position; + bool set = false; + vector values; +}; + +static bool StartsWithNumericDate(string &separator, const string &value) { + auto begin = value.c_str(); + auto end = begin + value.size(); + + // StrpTimeFormat::Parse will skip whitespace, so we can too + auto field1 = std::find_if_not(begin, end, StringUtil::CharacterIsSpace); + if (field1 == end) { + return false; + } + + // first numeric field must start immediately + if (!StringUtil::CharacterIsDigit(*field1)) { + return false; + } + auto literal1 = std::find_if_not(field1, end, StringUtil::CharacterIsDigit); + if (literal1 == end) { + return false; + } + + // second numeric field must exist + auto field2 = std::find_if(literal1, end, StringUtil::CharacterIsDigit); + if (field2 == end) { + return false; + } + auto literal2 = std::find_if_not(field2, end, StringUtil::CharacterIsDigit); + if (literal2 == end) { + return false; + } + + // third numeric field must exist + auto field3 = std::find_if(literal2, end, StringUtil::CharacterIsDigit); + if (field3 == end) { + return false; + } + + // second literal must match first + if (((field3 - literal2) != (field2 - literal1)) || strncmp(literal1, literal2, (field2 - literal1)) != 0) { + return false; + } + + // copy the literal as the separator, escaping percent signs + separator.clear(); + while (literal1 < field2) { + const auto literal_char = *literal1++; + if (literal_char == '%') { + separator.push_back(literal_char); + } + separator.push_back(literal_char); + } + + return true; +} + +string GenerateDateFormat(const string &separator, const char *format_template) { + string format_specifier = format_template; + auto amount_of_dashes = std::count(format_specifier.begin(), format_specifier.end(), '-'); + // All our date formats must have at least one - + D_ASSERT(amount_of_dashes); + string result; + result.reserve(format_specifier.size() - amount_of_dashes + (amount_of_dashes * separator.size())); + for (auto &character : format_specifier) { + if (character == '-') { + result += separator; + } else { + result += character; + } + } + return result; +} + +bool CSVSniffer::TryCastValue(CSVStateMachine &candidate, const Value &value, const LogicalType &sql_type) { + if (value.IsNull()) { + return true; + } + if (candidate.dialect_options.has_format.find(LogicalTypeId::DATE)->second && + sql_type.id() == LogicalTypeId::DATE) { + date_t result; + string error_message; + return candidate.dialect_options.date_format.find(LogicalTypeId::DATE) + ->second.TryParseDate(string_t(StringValue::Get(value)), result, error_message); + } + if (candidate.dialect_options.has_format.find(LogicalTypeId::TIMESTAMP)->second && + sql_type.id() == LogicalTypeId::TIMESTAMP) { + timestamp_t result; + string error_message; + return candidate.dialect_options.date_format.find(LogicalTypeId::TIMESTAMP) + ->second.TryParseTimestamp(string_t(StringValue::Get(value)), result, error_message); + } + if (candidate.options.decimal_separator != "." && (sql_type.id() == LogicalTypeId::DOUBLE)) { + return TryCastFloatingOperator::Operation(StringValue::Get(value)); + } + Value new_value; + string error_message; + return value.TryCastAs(buffer_manager->context, sql_type, new_value, &error_message, true); +} + +void CSVSniffer::SetDateFormat(CSVStateMachine &candidate, const string &format_specifier, + const LogicalTypeId &sql_type) { + candidate.dialect_options.has_format[sql_type] = true; + auto &date_format = candidate.dialect_options.date_format[sql_type]; + date_format.format_specifier = format_specifier; + StrTimeFormat::ParseFormatSpecifier(date_format.format_specifier, date_format); +} + +struct SniffValue { + inline static void Initialize(CSVStateMachine &machine) { + machine.state = CSVState::STANDARD; + machine.previous_state = CSVState::STANDARD; + machine.pre_previous_state = CSVState::STANDARD; + machine.cur_rows = 0; + machine.value = ""; + machine.rows_read = 0; + } + + inline static bool Process(CSVStateMachine &machine, vector &sniffed_values, char current_char, + idx_t current_pos) { + + if ((machine.dialect_options.new_line == NewLineIdentifier::SINGLE && + (current_char == '\r' || current_char == '\n')) || + (machine.dialect_options.new_line == NewLineIdentifier::CARRY_ON && current_char == '\n')) { + machine.rows_read++; + sniffed_values[machine.cur_rows].position = machine.line_start_pos; + sniffed_values[machine.cur_rows].set = true; + machine.line_start_pos = current_pos; + } + machine.pre_previous_state = machine.previous_state; + machine.previous_state = machine.state; + machine.state = static_cast( + machine.transition_array[static_cast(machine.state)][static_cast(current_char)]); + + bool carriage_return = machine.previous_state == CSVState::CARRIAGE_RETURN; + if (machine.previous_state == CSVState::DELIMITER || + (machine.previous_state == CSVState::RECORD_SEPARATOR && machine.state != CSVState::EMPTY_LINE) || + (machine.state != CSVState::RECORD_SEPARATOR && carriage_return)) { + // Started a new value + // Check if it's UTF-8 + machine.VerifyUTF8(); + if (machine.value.empty() || machine.value == machine.options.null_str) { + // We set empty == null value + sniffed_values[machine.cur_rows].values.push_back(Value(LogicalType::VARCHAR)); + } else { + sniffed_values[machine.cur_rows].values.push_back(Value(machine.value)); + } + sniffed_values[machine.cur_rows].line_number = machine.rows_read; + + machine.value = ""; + } + if (machine.state == CSVState::STANDARD || + (machine.state == CSVState::QUOTED && machine.previous_state == CSVState::QUOTED)) { + machine.value += current_char; + } + machine.cur_rows += + machine.previous_state == CSVState::RECORD_SEPARATOR && machine.state != CSVState::EMPTY_LINE; + // It means our carriage return is actually a record separator + machine.cur_rows += machine.state != CSVState::RECORD_SEPARATOR && carriage_return; + if (machine.cur_rows >= sniffed_values.size()) { + // We sniffed enough rows + return true; + } + return false; + } + + inline static void Finalize(CSVStateMachine &machine, vector &sniffed_values) { + if (machine.cur_rows < sniffed_values.size() && machine.state != CSVState::EMPTY_LINE) { + machine.VerifyUTF8(); + sniffed_values[machine.cur_rows].line_number = machine.rows_read; + if (!sniffed_values[machine.cur_rows].set) { + sniffed_values[machine.cur_rows].position = machine.line_start_pos; + sniffed_values[machine.cur_rows].set = true; + } + + sniffed_values[machine.cur_rows++].values.push_back(Value(machine.value)); + } + sniffed_values.erase(sniffed_values.end() - (sniffed_values.size() - machine.cur_rows), sniffed_values.end()); + } +}; + +void CSVSniffer::DetectDateAndTimeStampFormats(CSVStateMachine &candidate, + map &has_format_candidates, + map> &format_candidates, + const LogicalType &sql_type, const string &separator, Value &dummy_val) { + // generate date format candidates the first time through + auto &type_format_candidates = format_candidates[sql_type.id()]; + const auto had_format_candidates = has_format_candidates[sql_type.id()]; + if (!has_format_candidates[sql_type.id()]) { + has_format_candidates[sql_type.id()] = true; + // order by preference + auto entry = format_template_candidates.find(sql_type.id()); + if (entry != format_template_candidates.end()) { + const auto &format_template_list = entry->second; + for (const auto &t : format_template_list) { + const auto format_string = GenerateDateFormat(separator, t); + // don't parse ISO 8601 + if (format_string.find("%Y-%m-%d") == string::npos) { + type_format_candidates.emplace_back(format_string); + } + } + } + // initialise the first candidate + candidate.dialect_options.has_format[sql_type.id()] = true; + // all formats are constructed to be valid + SetDateFormat(candidate, type_format_candidates.back(), sql_type.id()); + } + // check all formats and keep the first one that works + StrpTimeFormat::ParseResult result; + auto save_format_candidates = type_format_candidates; + while (!type_format_candidates.empty()) { + // avoid using exceptions for flow control... + auto ¤t_format = candidate.dialect_options.date_format[sql_type.id()]; + if (current_format.Parse(StringValue::Get(dummy_val), result)) { + break; + } + // doesn't work - move to the next one + type_format_candidates.pop_back(); + candidate.dialect_options.has_format[sql_type.id()] = (!type_format_candidates.empty()); + if (!type_format_candidates.empty()) { + SetDateFormat(candidate, type_format_candidates.back(), sql_type.id()); + } + } + // if none match, then this is not a value of type sql_type, + if (type_format_candidates.empty()) { + // so restore the candidates that did work. + // or throw them out if they were generated by this value. + if (had_format_candidates) { + type_format_candidates.swap(save_format_candidates); + if (!type_format_candidates.empty()) { + SetDateFormat(candidate, type_format_candidates.back(), sql_type.id()); + } + } else { + has_format_candidates[sql_type.id()] = false; + } + } +} + +void CSVSniffer::DetectTypes() { + idx_t min_varchar_cols = max_columns_found + 1; + vector return_types; + // check which info candidate leads to minimum amount of non-varchar columns... + for (auto &candidate : candidates) { + unordered_map> info_sql_types_candidates; + for (idx_t i = 0; i < candidate->dialect_options.num_cols; i++) { + info_sql_types_candidates[i] = candidate->options.auto_type_candidates; + } + map has_format_candidates; + map> format_candidates; + for (const auto &t : format_template_candidates) { + has_format_candidates[t.first] = false; + format_candidates[t.first].clear(); + } + D_ASSERT(candidate->dialect_options.num_cols > 0); + + // Set all return_types to VARCHAR so we can do datatype detection based on VARCHAR values + return_types.clear(); + return_types.assign(candidate->dialect_options.num_cols, LogicalType::VARCHAR); + + // Reset candidate for parsing + candidate->Reset(); + + // Parse chunk and read csv with info candidate + idx_t sample_size = options.sample_chunk_size; + if (options.sample_chunk_size == 1) { + sample_size++; + } + vector tuples(sample_size); + candidate->csv_buffer_iterator.Process(*candidate, tuples); + // Potentially Skip empty rows (I find this dirty, but it is what the original code does) + idx_t true_start = 0; + idx_t values_start = 0; + while (true_start < tuples.size()) { + if (tuples[true_start].values.empty() || + (tuples[true_start].values.size() == 1 && tuples[true_start].values[0].IsNull())) { + true_start = tuples[true_start].line_number; + values_start++; + } else { + break; + } + } + + // Potentially Skip Notes (I also find this dirty, but it is what the original code does) + while (true_start < tuples.size()) { + if (tuples[true_start].values.size() < max_columns_found) { + true_start = tuples[true_start].line_number; + values_start++; + } else { + break; + } + } + + tuples.erase(tuples.begin(), tuples.begin() + values_start); + idx_t row_idx = 0; + if (tuples.size() > 1 && (!options.has_header || (options.has_header && options.dialect_options.header))) { + // This means we have more than one row, hence we can use the first row to detect if we have a header + row_idx = 1; + } + if (!tuples.empty()) { + best_start_without_header = tuples[0].position; + } + + // First line where we start our type detection + const idx_t start_idx_detection = row_idx; + for (; row_idx < tuples.size(); row_idx++) { + for (idx_t col = 0; col < tuples[row_idx].values.size(); col++) { + auto &col_type_candidates = info_sql_types_candidates[col]; + auto cur_top_candidate = col_type_candidates.back(); + auto dummy_val = tuples[row_idx].values[col]; + // try cast from string to sql_type + while (col_type_candidates.size() > 1) { + const auto &sql_type = col_type_candidates.back(); + // try formatting for date types if the user did not specify one and it starts with numeric values. + string separator; + bool has_format_is_set = false; + auto format_iterator = candidate->dialect_options.has_format.find(sql_type.id()); + if (format_iterator != candidate->dialect_options.has_format.end()) { + has_format_is_set = format_iterator->second; + } + if (has_format_candidates.count(sql_type.id()) && + (!has_format_is_set || format_candidates[sql_type.id()].size() > 1) && !dummy_val.IsNull() && + StartsWithNumericDate(separator, StringValue::Get(dummy_val))) { + DetectDateAndTimeStampFormats(*candidate, has_format_candidates, format_candidates, sql_type, + separator, dummy_val); + } + // try cast from string to sql_type + if (TryCastValue(*candidate, dummy_val, sql_type)) { + break; + } else { + if (row_idx != start_idx_detection && cur_top_candidate == LogicalType::BOOLEAN) { + // If we thought this was a boolean value (i.e., T,F, True, False) and it is not, we + // immediately pop to varchar. + while (col_type_candidates.back() != LogicalType::VARCHAR) { + col_type_candidates.pop_back(); + } + break; + } + col_type_candidates.pop_back(); + } + } + } + } + + idx_t varchar_cols = 0; + + for (idx_t col = 0; col < info_sql_types_candidates.size(); col++) { + auto &col_type_candidates = info_sql_types_candidates[col]; + // check number of varchar columns + const auto &col_type = col_type_candidates.back(); + if (col_type == LogicalType::VARCHAR) { + varchar_cols++; + } + } + + // it's good if the dialect creates more non-varchar columns, but only if we sacrifice < 30% of best_num_cols. + if (varchar_cols < min_varchar_cols && info_sql_types_candidates.size() > (max_columns_found * 0.7)) { + // we have a new best_options candidate + if (true_start > 0) { + // Add empty rows to skip_rows + candidate->dialect_options.skip_rows += true_start; + } + best_candidate = std::move(candidate); + min_varchar_cols = varchar_cols; + best_sql_types_candidates_per_column_idx = info_sql_types_candidates; + best_format_candidates = format_candidates; + best_header_row = tuples[0].values; + best_start_with_header = tuples[0].position; + } + } + // Assert that it's all good at this point. + D_ASSERT(best_candidate && !best_format_candidates.empty() && !best_header_row.empty()); + + for (const auto &best : best_format_candidates) { + if (!best.second.empty()) { + SetDateFormat(*best_candidate, best.second.back(), best.first); + } + } +} + +} // namespace duckdb diff --git a/src/duckdb/src/execution/operator/csv_scanner/sniffer/type_refinement.cpp b/src/duckdb/src/execution/operator/csv_scanner/sniffer/type_refinement.cpp new file mode 100644 index 000000000..300479679 --- /dev/null +++ b/src/duckdb/src/execution/operator/csv_scanner/sniffer/type_refinement.cpp @@ -0,0 +1,175 @@ +#include "duckdb/execution/operator/scan/csv/csv_sniffer.hpp" +#include "duckdb/execution/operator/scan/csv/base_csv_reader.hpp" +namespace duckdb { +struct Parse { + inline static void Initialize(CSVStateMachine &machine) { + machine.state = CSVState::STANDARD; + machine.previous_state = CSVState::STANDARD; + machine.pre_previous_state = CSVState::STANDARD; + + machine.cur_rows = 0; + machine.column_count = 0; + machine.value = ""; + } + + inline static bool Process(CSVStateMachine &machine, DataChunk &parse_chunk, char current_char, idx_t current_pos) { + + machine.pre_previous_state = machine.previous_state; + machine.previous_state = machine.state; + machine.state = static_cast( + machine.transition_array[static_cast(machine.state)][static_cast(current_char)]); + + bool carriage_return = machine.previous_state == CSVState::CARRIAGE_RETURN; + if (machine.previous_state == CSVState::DELIMITER || + (machine.previous_state == CSVState::RECORD_SEPARATOR && machine.state != CSVState::EMPTY_LINE) || + (machine.state != CSVState::RECORD_SEPARATOR && carriage_return)) { + // Started a new value + // Check if it's UTF-8 (Or not?) + machine.VerifyUTF8(); + auto &v = parse_chunk.data[machine.column_count++]; + auto parse_data = FlatVector::GetData(v); + auto &validity_mask = FlatVector::Validity(v); + if (machine.value.empty()) { + validity_mask.SetInvalid(machine.cur_rows); + } else { + parse_data[machine.cur_rows] = StringVector::AddStringOrBlob(v, string_t(machine.value)); + } + machine.value = ""; + } + if (((machine.previous_state == CSVState::RECORD_SEPARATOR && machine.state != CSVState::EMPTY_LINE) || + (machine.state != CSVState::RECORD_SEPARATOR && carriage_return)) && + machine.options.null_padding && machine.column_count < parse_chunk.ColumnCount()) { + // It's a new row, check if we need to pad stuff + while (machine.column_count < parse_chunk.ColumnCount()) { + auto &v = parse_chunk.data[machine.column_count++]; + auto &validity_mask = FlatVector::Validity(v); + validity_mask.SetInvalid(machine.cur_rows); + } + } + if (machine.state == CSVState::STANDARD) { + machine.value += current_char; + } + machine.cur_rows += + machine.previous_state == CSVState::RECORD_SEPARATOR && machine.state != CSVState::EMPTY_LINE; + machine.column_count -= machine.column_count * (machine.previous_state == CSVState::RECORD_SEPARATOR); + + // It means our carriage return is actually a record separator + machine.cur_rows += machine.state != CSVState::RECORD_SEPARATOR && carriage_return; + machine.column_count -= machine.column_count * (machine.state != CSVState::RECORD_SEPARATOR && carriage_return); + + if (machine.cur_rows >= machine.options.sample_chunk_size) { + // We sniffed enough rows + return true; + } + return false; + } + + inline static void Finalize(CSVStateMachine &machine, DataChunk &parse_chunk) { + if (machine.cur_rows < machine.options.sample_chunk_size && machine.state != CSVState::EMPTY_LINE) { + machine.VerifyUTF8(); + auto &v = parse_chunk.data[machine.column_count++]; + auto parse_data = FlatVector::GetData(v); + parse_data[machine.cur_rows] = StringVector::AddStringOrBlob(v, string_t(machine.value)); + } + parse_chunk.SetCardinality(machine.cur_rows); + } +}; + +bool CSVSniffer::TryCastVector(Vector &parse_chunk_col, idx_t size, const LogicalType &sql_type) { + // try vector-cast from string to sql_type + Vector dummy_result(sql_type); + if (best_candidate->dialect_options.has_format[LogicalTypeId::DATE] && sql_type == LogicalTypeId::DATE) { + // use the date format to cast the chunk + string error_message; + idx_t line_error; + return BaseCSVReader::TryCastDateVector(best_candidate->dialect_options.date_format, parse_chunk_col, + dummy_result, size, error_message, line_error); + } + if (best_candidate->dialect_options.has_format[LogicalTypeId::TIMESTAMP] && sql_type == LogicalTypeId::TIMESTAMP) { + // use the timestamp format to cast the chunk + string error_message; + return BaseCSVReader::TryCastTimestampVector(best_candidate->dialect_options.date_format, parse_chunk_col, + dummy_result, size, error_message); + } + // target type is not varchar: perform a cast + string error_message; + return VectorOperations::DefaultTryCast(parse_chunk_col, dummy_result, size, &error_message, true); +} + +void CSVSniffer::RefineTypes() { + // if data types were provided, exit here if number of columns does not match + detected_types.assign(best_candidate->dialect_options.num_cols, LogicalType::VARCHAR); + if (best_candidate->options.all_varchar) { + // return all types varchar + return; + } + DataChunk parse_chunk; + parse_chunk.Initialize(BufferAllocator::Get(buffer_manager->context), detected_types, options.sample_chunk_size); + for (idx_t i = 1; i < best_candidate->options.sample_chunks; i++) { + bool finished_file = best_candidate->csv_buffer_iterator.Finished(); + if (finished_file) { + // we finished the file: stop + // set sql types + detected_types.clear(); + for (idx_t column_idx = 0; column_idx < best_sql_types_candidates_per_column_idx.size(); column_idx++) { + LogicalType d_type = best_sql_types_candidates_per_column_idx[column_idx].back(); + if (best_sql_types_candidates_per_column_idx[column_idx].size() == + best_candidate->options.auto_type_candidates.size()) { + d_type = LogicalType::VARCHAR; + } + detected_types.push_back(d_type); + } + return; + } + best_candidate->csv_buffer_iterator.Process(*best_candidate, parse_chunk); + for (idx_t col = 0; col < parse_chunk.ColumnCount(); col++) { + vector &col_type_candidates = best_sql_types_candidates_per_column_idx[col]; + while (col_type_candidates.size() > 1) { + const auto &sql_type = col_type_candidates.back(); + // narrow down the date formats + if (best_format_candidates.count(sql_type.id())) { + auto &best_type_format_candidates = best_format_candidates[sql_type.id()]; + auto save_format_candidates = best_type_format_candidates; + while (!best_type_format_candidates.empty()) { + if (TryCastVector(parse_chunk.data[col], parse_chunk.size(), sql_type)) { + break; + } + // doesn't work - move to the next one + best_type_format_candidates.pop_back(); + best_candidate->dialect_options.has_format[sql_type.id()] = + (!best_type_format_candidates.empty()); + if (!best_type_format_candidates.empty()) { + SetDateFormat(*best_candidate, best_type_format_candidates.back(), sql_type.id()); + } + } + // if none match, then this is not a column of type sql_type, + if (best_type_format_candidates.empty()) { + // so restore the candidates that did work. + best_type_format_candidates.swap(save_format_candidates); + if (!best_type_format_candidates.empty()) { + SetDateFormat(*best_candidate, best_type_format_candidates.back(), sql_type.id()); + } + } + } + if (TryCastVector(parse_chunk.data[col], parse_chunk.size(), sql_type)) { + break; + } else { + col_type_candidates.pop_back(); + } + } + } + // reset parse chunk for the next iteration + parse_chunk.Reset(); + } + detected_types.clear(); + // set sql types + for (idx_t column_idx = 0; column_idx < best_sql_types_candidates_per_column_idx.size(); column_idx++) { + LogicalType d_type = best_sql_types_candidates_per_column_idx[column_idx].back(); + if (best_sql_types_candidates_per_column_idx[column_idx].size() == + best_candidate->options.auto_type_candidates.size()) { + d_type = LogicalType::VARCHAR; + } + detected_types.push_back(d_type); + } +} +} // namespace duckdb diff --git a/src/duckdb/src/execution/operator/csv_scanner/sniffer/type_replacement.cpp b/src/duckdb/src/execution/operator/csv_scanner/sniffer/type_replacement.cpp new file mode 100644 index 000000000..41988082a --- /dev/null +++ b/src/duckdb/src/execution/operator/csv_scanner/sniffer/type_replacement.cpp @@ -0,0 +1,39 @@ +#include "duckdb/execution/operator/scan/csv/csv_sniffer.hpp" +#include "duckdb/execution/operator/scan/csv/buffered_csv_reader.hpp" + +namespace duckdb { +void CSVSniffer::ReplaceTypes() { + if (best_candidate->options.sql_type_list.empty()) { + return; + } + // user-defined types were supplied for certain columns + // override the types + if (!best_candidate->options.sql_types_per_column.empty()) { + // types supplied as name -> value map + idx_t found = 0; + for (idx_t i = 0; i < names.size(); i++) { + auto it = best_candidate->options.sql_types_per_column.find(names[i]); + if (it != best_candidate->options.sql_types_per_column.end()) { + best_sql_types_candidates_per_column_idx[i] = {best_candidate->options.sql_type_list[it->second]}; + found++; + } + } + if (!best_candidate->options.file_options.union_by_name && + found < best_candidate->options.sql_types_per_column.size()) { + string error_msg = BufferedCSVReader::ColumnTypesError(options.sql_types_per_column, names); + if (!error_msg.empty()) { + throw BinderException(error_msg); + } + } + return; + } + // types supplied as list + if (names.size() < best_candidate->options.sql_type_list.size()) { + throw BinderException("read_csv: %d types were provided, but CSV file only has %d columns", + best_candidate->options.sql_type_list.size(), names.size()); + } + for (idx_t i = 0; i < best_candidate->options.sql_type_list.size(); i++) { + best_sql_types_candidates_per_column_idx[i] = {best_candidate->options.sql_type_list[i]}; + } +} +} // namespace duckdb diff --git a/src/duckdb/src/execution/operator/join/physical_asof_join.cpp b/src/duckdb/src/execution/operator/join/physical_asof_join.cpp index 7844e53db..439f7fcd6 100644 --- a/src/duckdb/src/execution/operator/join/physical_asof_join.cpp +++ b/src/duckdb/src/execution/operator/join/physical_asof_join.cpp @@ -722,7 +722,7 @@ void AsOfLocalSourceState::CombineLeftPartitions() { void AsOfLocalSourceState::MergeLeftPartitions() { PartitionGlobalMergeStates::Callback local_callback; - PartitionLocalMergeState local_merge; + PartitionLocalMergeState local_merge(*gsource.gsink.lhs_sink); gsource.GetMergeStates().ExecuteTask(local_merge, local_callback); gsource.merged++; while (gsource.merged < gsource.mergers) { diff --git a/src/duckdb/src/execution/operator/set/physical_recursive_cte.cpp b/src/duckdb/src/execution/operator/set/physical_recursive_cte.cpp index 8f713afda..ea367ed47 100644 --- a/src/duckdb/src/execution/operator/set/physical_recursive_cte.cpp +++ b/src/duckdb/src/execution/operator/set/physical_recursive_cte.cpp @@ -43,7 +43,6 @@ class RecursiveCTEState : public GlobalSinkState { bool initialized = false; bool finished_scan = false; SelectionVector new_groups; - AggregateHTAppendState append_state; }; unique_ptr PhysicalRecursiveCTE::GetGlobalSinkState(ClientContext &context) const { @@ -54,7 +53,7 @@ idx_t PhysicalRecursiveCTE::ProbeHT(DataChunk &chunk, RecursiveCTEState &state) Vector dummy_addresses(LogicalType::POINTER); // Use the HT to eliminate duplicate rows - idx_t new_group_count = state.ht->FindOrCreateGroups(state.append_state, chunk, dummy_addresses, state.new_groups); + idx_t new_group_count = state.ht->FindOrCreateGroups(chunk, dummy_addresses, state.new_groups); // we only return entries we have not seen before (i.e. new groups) chunk.Slice(state.new_groups, new_group_count); diff --git a/src/duckdb/src/execution/radix_partitioned_hashtable.cpp b/src/duckdb/src/execution/radix_partitioned_hashtable.cpp index fd30b6330..56c03d4db 100644 --- a/src/duckdb/src/execution/radix_partitioned_hashtable.cpp +++ b/src/duckdb/src/execution/radix_partitioned_hashtable.cpp @@ -1,49 +1,28 @@ #include "duckdb/execution/radix_partitioned_hashtable.hpp" #include "duckdb/common/radix_partitioning.hpp" +#include "duckdb/common/row_operations/row_operations.hpp" #include "duckdb/common/types/row/tuple_data_collection.hpp" +#include "duckdb/common/types/row/tuple_data_iterator.hpp" +#include "duckdb/execution/aggregate_hashtable.hpp" #include "duckdb/execution/executor.hpp" #include "duckdb/execution/operator/aggregate/physical_hash_aggregate.hpp" +#include "duckdb/main/config.hpp" #include "duckdb/parallel/event.hpp" -#include "duckdb/parallel/task_scheduler.hpp" #include "duckdb/planner/expression/bound_reference_expression.hpp" namespace duckdb { -// compute the GROUPING values -// for each parameter to the GROUPING clause, we check if the hash table groups on this particular group -// if it does, we return 0, otherwise we return 1 -// we then use bitshifts to combine these values -void RadixPartitionedHashTable::SetGroupingValues() { - auto &grouping_functions = op.GetGroupingFunctions(); - for (auto &grouping : grouping_functions) { - int64_t grouping_value = 0; - D_ASSERT(grouping.size() < sizeof(int64_t) * 8); - for (idx_t i = 0; i < grouping.size(); i++) { - if (grouping_set.find(grouping[i]) == grouping_set.end()) { - // we don't group on this value! - grouping_value += (int64_t)1 << (grouping.size() - (i + 1)); - } - } - grouping_values.push_back(Value::BIGINT(grouping_value)); - } -} - RadixPartitionedHashTable::RadixPartitionedHashTable(GroupingSet &grouping_set_p, const GroupedAggregateData &op_p) : grouping_set(grouping_set_p), op(op_p) { - auto groups_count = op.GroupCount(); for (idx_t i = 0; i < groups_count; i++) { if (grouping_set.find(i) == grouping_set.end()) { null_groups.push_back(i); } } - - // 10000 seems like a good compromise here - radix_limit = 10000; - if (grouping_set.empty()) { - // fake a single group with a constant value for aggregation without groups + // Fake a single group with a constant value for aggregation without groups group_types.emplace_back(LogicalType::TINYINT); } for (auto &entry : grouping_set) { @@ -51,79 +30,279 @@ RadixPartitionedHashTable::RadixPartitionedHashTable(GroupingSet &grouping_set_p group_types.push_back(op.group_types[entry]); } SetGroupingValues(); + + auto group_types_copy = group_types; + group_types_copy.emplace_back(LogicalType::HASH); + layout.Initialize(std::move(group_types_copy), AggregateObject::CreateAggregateObjects(op.bindings)); +} + +void RadixPartitionedHashTable::SetGroupingValues() { + // Compute the GROUPING values: + // For each parameter to the GROUPING clause, we check if the hash table groups on this particular group + // If it does, we return 0, otherwise we return 1 + // We then use bitshifts to combine these values + auto &grouping_functions = op.GetGroupingFunctions(); + for (auto &grouping : grouping_functions) { + int64_t grouping_value = 0; + D_ASSERT(grouping.size() < sizeof(int64_t) * 8); + for (idx_t i = 0; i < grouping.size(); i++) { + if (grouping_set.find(grouping[i]) == grouping_set.end()) { + // We don't group on this value! + grouping_value += (int64_t)1 << (grouping.size() - (i + 1)); + } + } + grouping_values.push_back(Value::BIGINT(grouping_value)); + } +} + +const TupleDataLayout &RadixPartitionedHashTable::GetLayout() const { + return layout; +} + +unique_ptr RadixPartitionedHashTable::CreateHT(ClientContext &context, const idx_t capacity, + const idx_t radix_bits) const { + return make_uniq(context, BufferAllocator::Get(context), group_types, op.payload_types, + op.bindings, capacity, radix_bits); } //===--------------------------------------------------------------------===// // Sink //===--------------------------------------------------------------------===// -class RadixHTGlobalState : public GlobalSinkState { - constexpr const static idx_t MAX_RADIX_PARTITIONS = 32; +struct AggregatePartition { + explicit AggregatePartition(unique_ptr data_p) : data(std::move(data_p)), finalized(false) { + } + unique_ptr data; + atomic finalized; +}; +class RadixHTGlobalSinkState; + +struct RadixHTConfig { public: - explicit RadixHTGlobalState(ClientContext &context) - : is_empty(true), multi_scan(true), partitioned(false), - partition_info(make_uniq( - MinValue(MAX_RADIX_PARTITIONS, TaskScheduler::GetScheduler(context).NumberOfThreads()))) { - } + explicit RadixHTConfig(ClientContext &context, RadixHTGlobalSinkState &sink); - vector> intermediate_hts; - vector> finalized_hts; + void SetRadixBits(idx_t radix_bits_p); + bool SetRadixBitsToExternal(); + idx_t GetRadixBits() const; - //! Whether or not any tuples were added to the HT - bool is_empty; - //! Whether or not the hash table should be scannable multiple times - bool multi_scan; - //! The lock for updating the global aggregate state - mutex lock; - //! Whether or not any thread has crossed the partitioning threshold - atomic partitioned; - - bool is_finalized = false; - bool is_partitioned = false; - - unique_ptr partition_info; - AggregateHTAppendState append_state; - - //! Repartitioned HT info - bool repartitioned = false; - idx_t repartition_tasks_per_partition; - vector>> repartition_tasks; - unique_array> repartition_tasks_assigned; - unique_array> repartition_tasks_done; - unique_array> finalize_assigned; +private: + void SetRadixBitsInternal(const idx_t radix_bits_p, bool external); + static idx_t InitialSinkRadixBits(ClientContext &context); + static idx_t MaximumSinkRadixBits(ClientContext &context); + static idx_t ExternalRadixBits(const idx_t &maximum_sink_radix_bits_p); + static idx_t SinkCapacity(ClientContext &context); + +private: + //! Assume (1 << 15) = 32KB L1 cache per core, divided by two because hyperthreading + static constexpr const idx_t L1_CACHE_SIZE = 32768 / 2; + //! Assume (1 << 20) = 1MB L2 cache per core, divided by two because hyperthreading + static constexpr const idx_t L2_CACHE_SIZE = 1048576 / 2; + //! Assume (1 << 20) + (1 << 19) = 1.5MB L3 cache per core (shared), divided by two because hyperthreading + static constexpr const idx_t L3_CACHE_SIZE = 1572864 / 2; + + //! Sink radix bits to initialize with + static constexpr const idx_t MAXIMUM_INITIAL_SINK_RADIX_BITS = 3; + //! Maximum Sink radix bits (independent of threads) + static constexpr const idx_t MAXIMUM_FINAL_SINK_RADIX_BITS = 7; + //! By how many radix bits to increment if we go external + static constexpr const idx_t EXTERNAL_RADIX_BITS_INCREMENT = 3; + + //! The global sink state + RadixHTGlobalSinkState &sink; + //! Current thread-global sink radix bits + atomic sink_radix_bits; + //! Maximum Sink radix bits (set based on number of threads) + const idx_t maximum_sink_radix_bits; + //! Radix bits if we go external + const idx_t external_radix_bits; + +public: + //! Capacity of HTs during the Sink + const idx_t sink_capacity; + + //! If we fill this many blocks per partition, we trigger a repartition + static constexpr const double BLOCK_FILL_FACTOR = 1.8; + //! By how many bits to repartition if a repartition is triggered + static constexpr const idx_t REPARTITION_RADIX_BITS = 2; }; -class RadixHTLocalState : public LocalSinkState { +class RadixHTGlobalSinkState : public GlobalSinkState { +public: + RadixHTGlobalSinkState(ClientContext &context, const RadixPartitionedHashTable &radix_ht); + + //! Destroys aggregate states (if multi-scan) + ~RadixHTGlobalSinkState() override; + void Destroy(); + public: - explicit RadixHTLocalState(const RadixPartitionedHashTable &ht) : total_groups(0), is_empty(true) { - // if there are no groups we create a fake group so everything has the same group - group_chunk.InitializeEmpty(ht.group_types); - if (ht.grouping_set.empty()) { - group_chunk.data[0].Reference(Value::TINYINT(42)); + //! The radix HT + const RadixPartitionedHashTable &radix_ht; + //! Config for partitioning + RadixHTConfig config; + + //! Whether we've called Finalize + bool finalized; + //! Whether we are doing an external aggregation + atomic external; + //! Threads that have called Sink + atomic active_threads; + //! If any thread has called combine + atomic any_combined; + + //! Lock for uncombined_data/stored_allocators + mutex lock; + //! Uncombined partitioned data that will be put into the AggregatePartitions + unique_ptr uncombined_data; + //! Allocators used during the Sink/Finalize + vector> stored_allocators; + + //! Partitions that are finalized during GetData + vector> partitions; + + //! For synchronizing finalize tasks + atomic finalize_idx; + + //! Pin properties when scanning + TupleDataPinProperties scan_pin_properties; + //! Total count before combining + idx_t count_before_combining; +}; + +RadixHTGlobalSinkState::RadixHTGlobalSinkState(ClientContext &context, const RadixPartitionedHashTable &radix_ht_p) + : radix_ht(radix_ht_p), config(context, *this), finalized(false), external(false), active_threads(0), + any_combined(false), finalize_idx(0), scan_pin_properties(TupleDataPinProperties::DESTROY_AFTER_DONE), + count_before_combining(0) { +} + +RadixHTGlobalSinkState::~RadixHTGlobalSinkState() { + Destroy(); +} + +// LCOV_EXCL_START +void RadixHTGlobalSinkState::Destroy() { + if (scan_pin_properties == TupleDataPinProperties::DESTROY_AFTER_DONE || count_before_combining == 0 || + partitions.empty()) { + // Already destroyed / empty + return; + } + + TupleDataLayout layout = partitions[0]->data->GetLayout().Copy(); + if (!layout.HasDestructor()) { + return; // No destructors, exit + } + + // There are aggregates with destructors: Call the destructor for each of the aggregates + RowOperationsState row_state(*stored_allocators.back()); + for (auto &partition : partitions) { + auto &data_collection = *partition->data; + if (data_collection.Count() == 0) { + continue; } + TupleDataChunkIterator iterator(data_collection, TupleDataPinProperties::DESTROY_AFTER_DONE, false); + auto &row_locations = iterator.GetChunkState().row_locations; + do { + RowOperations::DestroyStates(row_state, layout, row_locations, iterator.GetCurrentChunkCount()); + } while (iterator.Next()); + data_collection.Reset(); } +} +// LCOV_EXCL_STOP +RadixHTConfig::RadixHTConfig(ClientContext &context, RadixHTGlobalSinkState &sink_p) + : sink(sink_p), sink_radix_bits(InitialSinkRadixBits(context)), + maximum_sink_radix_bits(MaximumSinkRadixBits(context)), + external_radix_bits(ExternalRadixBits(maximum_sink_radix_bits)), sink_capacity(SinkCapacity(context)) { +} + +void RadixHTConfig::SetRadixBits(idx_t radix_bits_p) { + SetRadixBitsInternal(MinValue(radix_bits_p, maximum_sink_radix_bits), false); +} + +bool RadixHTConfig::SetRadixBitsToExternal() { + SetRadixBitsInternal(external_radix_bits, true); + return sink.external; +} + +idx_t RadixHTConfig::GetRadixBits() const { + return sink_radix_bits; +} + +void RadixHTConfig::SetRadixBitsInternal(const idx_t radix_bits_p, bool external) { + if (sink_radix_bits >= radix_bits_p || sink.any_combined) { + return; + } + + lock_guard guard(sink.lock); + if (sink_radix_bits >= radix_bits_p || sink.any_combined) { + return; + } + + if (external) { + sink.external = true; + } + sink_radix_bits = radix_bits_p; + return; +} + +idx_t RadixHTConfig::InitialSinkRadixBits(ClientContext &context) { + const idx_t active_threads = TaskScheduler::GetScheduler(context).NumberOfThreads(); + return MinValue(RadixPartitioning::RadixBits(NextPowerOfTwo(active_threads)), MAXIMUM_INITIAL_SINK_RADIX_BITS); +} + +idx_t RadixHTConfig::MaximumSinkRadixBits(ClientContext &context) { + const idx_t active_threads = TaskScheduler::GetScheduler(context).NumberOfThreads(); + return MinValue(RadixPartitioning::RadixBits(NextPowerOfTwo(active_threads)), MAXIMUM_FINAL_SINK_RADIX_BITS); +} + +idx_t RadixHTConfig::ExternalRadixBits(const idx_t &maximum_sink_radix_bits_p) { + return MinValue(maximum_sink_radix_bits_p + EXTERNAL_RADIX_BITS_INCREMENT, MAXIMUM_FINAL_SINK_RADIX_BITS); +} + +idx_t RadixHTConfig::SinkCapacity(ClientContext &context) { + // Get active and maximum number of threads + const idx_t active_threads = TaskScheduler::GetScheduler(context).NumberOfThreads(); + const auto max_threads = DBConfig::GetSystemMaxThreads(FileSystem::GetFileSystem(context)); + + // Compute cache size per active thread (assuming cache is shared) + const auto total_shared_cache_size = max_threads * L3_CACHE_SIZE; + const auto cache_per_active_thread = L1_CACHE_SIZE + L2_CACHE_SIZE + total_shared_cache_size / active_threads; + + // Divide cache per active thread by entry size, round up to next power of two, to get capacity + const auto size_per_entry = sizeof(aggr_ht_entry_t) * GroupedAggregateHashTable::LOAD_FACTOR; + const auto capacity = NextPowerOfTwo(cache_per_active_thread / size_per_entry); + + // Capacity must be at least the minimum capacity + return MaxValue(capacity, GroupedAggregateHashTable::InitialCapacity()); +} + +class RadixHTLocalSinkState : public LocalSinkState { +public: + RadixHTLocalSinkState(ClientContext &context, const RadixPartitionedHashTable &radix_ht); + +public: + //! Thread-local HT that is re-used after abandoning + unique_ptr ht; + //! Chunk with group columns DataChunk group_chunk; - //! The aggregate HT - unique_ptr ht; - //! The total number of groups found by this thread - idx_t total_groups; - //! Whether or not any tuples were added to the HT - bool is_empty; + //! Data that is abandoned ends up here (only if we're doing external aggregation) + unique_ptr abandoned_data; }; -void RadixPartitionedHashTable::SetMultiScan(GlobalSinkState &state) { - auto &gstate = state.Cast(); - gstate.multi_scan = true; +RadixHTLocalSinkState::RadixHTLocalSinkState(ClientContext &, const RadixPartitionedHashTable &radix_ht) { + // If there are no groups we create a fake group so everything has the same group + group_chunk.InitializeEmpty(radix_ht.group_types); + if (radix_ht.grouping_set.empty()) { + group_chunk.data[0].Reference(Value::TINYINT(42)); + } } unique_ptr RadixPartitionedHashTable::GetGlobalSinkState(ClientContext &context) const { - return make_uniq(context); + return make_uniq(context, *this); } unique_ptr RadixPartitionedHashTable::GetLocalSinkState(ExecutionContext &context) const { - return make_uniq(*this); + return make_uniq(context.client, *this); } void RadixPartitionedHashTable::PopulateGroupChunk(DataChunk &group_chunk, DataChunk &input_chunk) const { @@ -141,507 +320,448 @@ void RadixPartitionedHashTable::PopulateGroupChunk(DataChunk &group_chunk, DataC group_chunk.Verify(); } -void RadixPartitionedHashTable::Sink(ExecutionContext &context, DataChunk &chunk, OperatorSinkInput &input, - DataChunk &payload_input, const unsafe_vector &filter) const { - auto &llstate = input.local_state.Cast(); - auto &gstate = input.global_state.Cast(); - D_ASSERT(!gstate.is_finalized); - - DataChunk &group_chunk = llstate.group_chunk; - PopulateGroupChunk(group_chunk, chunk); +bool MaybeRepartition(ClientContext &context, RadixHTGlobalSinkState &gstate, RadixHTLocalSinkState &lstate) { + auto &config = gstate.config; + auto &ht = *lstate.ht; + auto &partitioned_data = ht.GetPartitionedData(); + + // Check if we're approaching the memory limit + const idx_t n_threads = TaskScheduler::GetScheduler(context).NumberOfThreads(); + const idx_t limit = BufferManager::GetBufferManager(context).GetMaxMemory(); + const idx_t thread_limit = 0.6 * limit / n_threads; + if (ht.GetPartitionedData()->SizeInBytes() > thread_limit || context.config.force_external) { + if (gstate.config.SetRadixBitsToExternal()) { + // We're approaching the memory limit, unpin the data + if (!lstate.abandoned_data) { + lstate.abandoned_data = make_uniq( + BufferManager::GetBufferManager(context), gstate.radix_ht.GetLayout(), config.GetRadixBits(), + gstate.radix_ht.GetLayout().ColumnCount() - 1); + } - // if we have non-combinable aggregates (e.g. string_agg) we cannot keep parallel hash - // tables - if (ForceSingleHT(input.global_state)) { - lock_guard glock(gstate.lock); - gstate.is_empty = gstate.is_empty && group_chunk.size() == 0; - if (gstate.finalized_hts.empty()) { - // Create a finalized ht in the global state, that we can populate - gstate.finalized_hts.push_back(make_shared( - context.client, BufferAllocator::Get(context.client), group_types, op.payload_types, op.bindings, - HtEntryType::HT_WIDTH_64)); + ht.UnpinData(); + partitioned_data->Repartition(*lstate.abandoned_data); + ht.SetRadixBits(gstate.config.GetRadixBits()); + ht.InitializePartitionedData(); + return true; } - D_ASSERT(gstate.finalized_hts.size() == 1); - D_ASSERT(gstate.finalized_hts[0]); - llstate.total_groups += - gstate.finalized_hts[0]->AddChunk(gstate.append_state, group_chunk, payload_input, filter); - return; } - if (group_chunk.size() > 0) { - llstate.is_empty = false; - } + const auto partition_count = partitioned_data->PartitionCount(); + const auto current_radix_bits = RadixPartitioning::RadixBits(partition_count); + D_ASSERT(current_radix_bits <= config.GetRadixBits()); - if (!llstate.ht) { - llstate.ht = - make_uniq(context.client, BufferAllocator::Get(context.client), - *gstate.partition_info, group_types, op.payload_types, op.bindings); - if (context.client.config.force_external) { - gstate.partitioned = true; - } + const auto row_size_per_partition = + partitioned_data->Count() * partitioned_data->GetLayout().GetRowWidth() / partition_count; + if (row_size_per_partition > config.BLOCK_FILL_FACTOR * Storage::BLOCK_SIZE) { + // We crossed our block filling threshold, try to increment radix bits + config.SetRadixBits(current_radix_bits + config.REPARTITION_RADIX_BITS); } - llstate.total_groups += llstate.ht->AddChunk(group_chunk, payload_input, - gstate.partitioned && gstate.partition_info->n_partitions > 1, filter); - if (llstate.total_groups >= radix_limit) { - gstate.partitioned = true; + const auto global_radix_bits = config.GetRadixBits(); + if (current_radix_bits == global_radix_bits) { + return false; // We're already on the right number of radix bits } + + // We're out-of-sync with the global radix bits, repartition + ht.UnpinData(); + auto old_partitioned_data = std::move(partitioned_data); + ht.SetRadixBits(global_radix_bits); + ht.InitializePartitionedData(); + old_partitioned_data->Repartition(*ht.GetPartitionedData()); + return true; } -void RadixPartitionedHashTable::Combine(ExecutionContext &context, GlobalSinkState &state, - LocalSinkState &lstate) const { - auto &llstate = lstate.Cast(); - auto &gstate = state.Cast(); - D_ASSERT(!gstate.is_finalized); +void RadixPartitionedHashTable::Sink(ExecutionContext &context, DataChunk &chunk, OperatorSinkInput &input, + DataChunk &payload_input, const unsafe_vector &filter) const { + auto &gstate = input.global_state.Cast(); + auto &lstate = input.local_state.Cast(); + if (!lstate.ht) { + lstate.ht = CreateHT(context.client, gstate.config.sink_capacity, gstate.config.GetRadixBits()); + gstate.active_threads++; + } - // this actually does not do a lot but just pushes the local HTs into the global state so we can later combine them - // in parallel + auto &group_chunk = lstate.group_chunk; + PopulateGroupChunk(group_chunk, chunk); - if (ForceSingleHT(state)) { - D_ASSERT(gstate.finalized_hts.size() <= 1); - return; - } + auto &ht = *lstate.ht; + ht.AddChunk(group_chunk, payload_input, filter); - if (!llstate.ht) { - return; // no data + if (ht.Count() + STANDARD_VECTOR_SIZE < ht.ResizeThreshold()) { + return; // We can fit another chunk } - if (!llstate.ht->IsPartitioned() && gstate.partition_info->n_partitions > 1 && gstate.partitioned) { - llstate.ht->Partition(true); + if (gstate.active_threads > 2) { + // 'Reset' the HT without taking its data, we can just keep appending to the same collection + // This only works because we never resize the HT + ht.ClearPointerTable(); + ht.ResetCount(); + // We don't do this when running with 1 or 2 threads, it only makes sense when there's many threads } - // we will never add new values to these HTs so we can drop the first part of the HT - llstate.ht->Finalize(); + // Check if we need to repartition + auto repartitioned = MaybeRepartition(context.client, gstate, lstate); - lock_guard glock(gstate.lock); - if (!llstate.is_empty) { - gstate.is_empty = false; + if (repartitioned && ht.Count() != 0) { + // We repartitioned, but we didn't clear the pointer table / reset the count because we're on 1 or 2 threads + ht.ClearPointerTable(); + ht.ResetCount(); } - // at this point we just collect them the PhysicalHashAggregateFinalizeTask (below) will merge them in parallel - gstate.intermediate_hts.push_back(std::move(llstate.ht)); + + // TODO: combine early and often } -void RadixPartitionedHashTable::InitializeFinalizedHTs(ClientContext &context, GlobalSinkState &gstate_p) const { - auto &gstate = gstate_p.Cast(); - auto &allocator = BufferAllocator::Get(context); - gstate.finalized_hts.resize(gstate.partition_info->n_partitions); - for (idx_t r = 0; r < gstate.partition_info->n_partitions; r++) { - gstate.finalized_hts[r] = make_shared( - context, allocator, group_types, op.payload_types, op.bindings, HtEntryType::HT_WIDTH_64); +void RadixPartitionedHashTable::Combine(ExecutionContext &context, GlobalSinkState &gstate_p, + LocalSinkState &lstate_p) const { + auto &gstate = gstate_p.Cast(); + auto &lstate = lstate_p.Cast(); + if (!lstate.ht) { + return; } -} -bool RadixPartitionedHashTable::Finalize(ClientContext &context, GlobalSinkState &gstate_p) const { - auto &gstate = gstate_p.Cast(); - D_ASSERT(!gstate.is_finalized); - gstate.is_finalized = true; + // Set any_combined, then check one last time whether we need to repartition + gstate.any_combined = true; + MaybeRepartition(context.client, gstate, lstate); - // special case if we have non-combinable aggregates - // we have already aggregated into a global shared HT that does not require any additional finalization steps - if (ForceSingleHT(gstate)) { - D_ASSERT(gstate.finalized_hts.size() <= 1); - D_ASSERT(gstate.finalized_hts.empty() || gstate.finalized_hts[0]); - return false; + auto &ht = *lstate.ht; + ht.UnpinData(); + + if (lstate.abandoned_data) { + D_ASSERT(gstate.external); + D_ASSERT(lstate.abandoned_data->PartitionCount() == lstate.ht->GetPartitionedData()->PartitionCount()); + D_ASSERT(lstate.abandoned_data->PartitionCount() == + RadixPartitioning::NumberOfPartitions(gstate.config.GetRadixBits())); + lstate.abandoned_data->Combine(*lstate.ht->GetPartitionedData()); + } else { + lstate.abandoned_data = std::move(ht.GetPartitionedData()); } - // we can have two cases now, non-partitioned for few groups and radix-partitioned for very many groups. - auto &allocator = BufferAllocator::Get(context); - if (AnyPartitioned(gstate_p)) { - // if one is partitioned, all have to be - // this should mostly have already happened in Combine, but if not we do it here - for (auto &pht : gstate.intermediate_hts) { - if (!pht->IsPartitioned()) { - pht->Partition(true); - } - } - // schedule additional tasks to combine the partial HTs - InitializeFinalizedHTs(context, gstate_p); - gstate.is_partitioned = true; - return true; - } else { // in the non-partitioned case we immediately combine all the unpartitioned hts created by the threads. - // TODO possible optimization, if total count < limit for 32 bit ht, use that one - // create this ht here so finalize needs no lock on gstate - - gstate.finalized_hts.push_back(make_shared( - context, allocator, group_types, op.payload_types, op.bindings, HtEntryType::HT_WIDTH_64)); - for (auto &pht : gstate.intermediate_hts) { - auto unpartitioned = pht->GetUnpartitioned(); - for (auto &unpartitioned_ht : unpartitioned) { - D_ASSERT(unpartitioned_ht); - gstate.finalized_hts[0]->Combine(*unpartitioned_ht); - unpartitioned_ht.reset(); - } - unpartitioned.clear(); - } - D_ASSERT(gstate.finalized_hts[0]); - gstate.finalized_hts[0]->Finalize(); - return false; + lock_guard guard(gstate.lock); + if (gstate.uncombined_data) { + gstate.uncombined_data->Combine(*lstate.abandoned_data); + } else { + gstate.uncombined_data = std::move(lstate.abandoned_data); } + gstate.stored_allocators.emplace_back(ht.GetAggregateAllocator()); } -// this task is run in multiple threads and combines the radix-partitioned hash tables into a single one and then -// folds them into the global ht finally. -class RadixAggregateFinalizeTask : public ExecutorTask { -public: - RadixAggregateFinalizeTask(Executor &executor, shared_ptr event_p, RadixHTGlobalState &state_p, - idx_t radix_p) - : ExecutorTask(executor), event(std::move(event_p)), state(state_p), radix(radix_p) { - } - - static void FinalizeHT(RadixHTGlobalState &gstate, idx_t radix) { - D_ASSERT(gstate.partition_info->n_partitions <= gstate.finalized_hts.size()); - D_ASSERT(gstate.finalized_hts[radix]); - - idx_t pht_idx_from = 0; - idx_t pht_idx_to = gstate.intermediate_hts.size(); - if (gstate.repartitioned) { - const auto num_partitions_before = gstate.repartition_tasks.size(); - const auto multiplier = gstate.partition_info->n_partitions / num_partitions_before; - const auto radix_before = radix / multiplier; - pht_idx_from = radix_before * gstate.repartition_tasks_per_partition; - pht_idx_to = pht_idx_from + gstate.repartition_tasks_per_partition; - } +void RadixPartitionedHashTable::Finalize(ClientContext &, GlobalSinkState &gstate_p) const { + auto &gstate = gstate_p.Cast(); + + if (gstate.uncombined_data) { + auto &uncombined_data = *gstate.uncombined_data; + gstate.count_before_combining = uncombined_data.Count(); + + // If true there is no need to combine, it was all done by a single thread in a single HT + const auto single_ht = !gstate.external && gstate.active_threads == 1; - for (idx_t i = pht_idx_from; i < pht_idx_to; i++) { - for (auto &ht : gstate.intermediate_hts[i]->GetPartition(radix)) { - gstate.finalized_hts[radix]->Combine(*ht); - ht.reset(); + auto &uncombined_partition_data = uncombined_data.GetPartitions(); + const auto n_partitions = uncombined_partition_data.size(); + gstate.partitions.reserve(n_partitions); + for (idx_t i = 0; i < n_partitions; i++) { + gstate.partitions.emplace_back(make_uniq(std::move(uncombined_partition_data[i]))); + if (single_ht) { + gstate.finalize_idx++; + gstate.partitions.back()->finalized = true; } } - gstate.finalized_hts[radix]->Finalize(); + } else { + gstate.count_before_combining = 0; } - TaskExecutionResult ExecuteTask(TaskExecutionMode mode) override { - FinalizeHT(state, radix); - event->FinishTask(); - return TaskExecutionResult::TASK_FINISHED; - } + gstate.finalized = true; +} -private: - shared_ptr event; - RadixHTGlobalState &state; - idx_t radix; +//===--------------------------------------------------------------------===// +// Source +//===--------------------------------------------------------------------===// +idx_t RadixPartitionedHashTable::Count(GlobalSinkState &sink_p) const { + const auto count = CountInternal(sink_p); + return count == 0 && grouping_set.empty() ? 1 : count; +} + +idx_t RadixPartitionedHashTable::CountInternal(GlobalSinkState &sink_p) const { + auto &sink = sink_p.Cast(); + return sink.count_before_combining; +} + +void RadixPartitionedHashTable::SetMultiScan(GlobalSinkState &sink_p) { + auto &sink = sink_p.Cast(); + sink.scan_pin_properties = TupleDataPinProperties::UNPIN_AFTER_DONE; +} + +enum class RadixHTSourceTaskType : uint8_t { NO_TASK, FINALIZE, SCAN }; + +class RadixHTLocalSourceState; + +class RadixHTGlobalSourceState : public GlobalSourceState { +public: + RadixHTGlobalSourceState(ClientContext &context, const RadixPartitionedHashTable &radix_ht); + + //! Assigns a task to a local source state + bool AssignTask(RadixHTGlobalSinkState &sink, RadixHTLocalSourceState &lstate); + +public: + //! The client context + ClientContext &context; + //! For synchronizing the source phase + atomic finished; + + //! Column ids for scanning + vector column_ids; + + //! For synchronizing scan tasks + atomic scan_idx; + atomic scan_done; }; -class RadixAggregateRepartitionTask : public ExecutorTask { +enum class RadixHTScanStatus : uint8_t { INIT, IN_PROGRESS, DONE }; + +class RadixHTLocalSourceState : public LocalSourceState { public: - RadixAggregateRepartitionTask(Executor &executor, shared_ptr event_p, RadixHTGlobalState &state_p, - idx_t num_partitions_before_p) - : ExecutorTask(executor), event(std::move(event_p)), state(state_p), - num_partitions_before(num_partitions_before_p) { - } - - TaskExecutionResult ExecuteTask(TaskExecutionMode mode) override { - const auto multiplier = state.partition_info->n_partitions / num_partitions_before; - - idx_t repartition_radix = 0; - idx_t finalize_radix = 0; - while (repartition_radix < num_partitions_before && finalize_radix < state.partition_info->n_partitions) { - // Loop over original partitions until we find one that we can repartition - for (; repartition_radix < num_partitions_before; repartition_radix++) { - auto task_idx = state.repartition_tasks_assigned[repartition_radix]++; - if (task_idx >= state.repartition_tasks_per_partition) { - continue; - } - auto &ht = state.repartition_tasks[repartition_radix][task_idx]; - ht->Partition(true); - state.intermediate_hts[repartition_radix * state.repartition_tasks_per_partition + task_idx] = - std::move(ht); - state.repartition_tasks_done[repartition_radix]++; - break; - } + explicit RadixHTLocalSourceState(ExecutionContext &context, const RadixPartitionedHashTable &radix_ht); - // Loop over repartitioned partitions - for (; finalize_radix < state.partition_info->n_partitions; finalize_radix++) { - const auto original_radix = finalize_radix / multiplier; - if (state.repartition_tasks_done[original_radix] != state.repartition_tasks_per_partition) { - break; // Needs more repartitioning - } - - if (state.finalize_assigned[finalize_radix]) { - continue; // Already assigned - } - - { - lock_guard guard(state.lock); - if (state.finalize_assigned[finalize_radix]) { - // LCOV_EXCL_START - continue; // Check again with lock, but already assigned - // LCOV_EXCL_STOP - } - state.finalize_assigned[finalize_radix] = true; - } - - // We can finalize! - RadixAggregateFinalizeTask::FinalizeHT(state, finalize_radix); - } - } - event->FinishTask(); - return TaskExecutionResult::TASK_FINISHED; - } +public: + //! Do the work this thread has been assigned + void ExecuteTask(RadixHTGlobalSinkState &sink, RadixHTGlobalSourceState &gstate, DataChunk &chunk); + //! Whether this thread has finished the work it has been assigned + bool TaskFinished(); private: - shared_ptr event; - RadixHTGlobalState &state; - const idx_t num_partitions_before; + //! Execute the finalize or scan task + void Finalize(RadixHTGlobalSinkState &sink, RadixHTGlobalSourceState &gstate); + void Scan(RadixHTGlobalSinkState &sink, RadixHTGlobalSourceState &gstate, DataChunk &chunk); + +public: + //! Current task and index + RadixHTSourceTaskType task; + idx_t task_idx; + + //! Thread-local HT that is re-used to Finalize + unique_ptr ht; + //! Current status of a Scan + RadixHTScanStatus scan_status; + +private: + //! Allocator and layout for finalizing state + TupleDataLayout layout; + ArenaAllocator aggregate_allocator; + + //! State and chunk for scanning + TupleDataScanState scan_state; + DataChunk scan_chunk; }; -void RadixPartitionedHashTable::ScheduleTasks(Executor &executor, const shared_ptr &event, - GlobalSinkState &state, vector> &tasks) const { - auto &gstate = state.Cast(); - if (!gstate.is_partitioned) { - return; +unique_ptr RadixPartitionedHashTable::GetGlobalSourceState(ClientContext &context) const { + return make_uniq(context, *this); +} + +unique_ptr RadixPartitionedHashTable::GetLocalSourceState(ExecutionContext &context) const { + return make_uniq(context, *this); +} + +RadixHTGlobalSourceState::RadixHTGlobalSourceState(ClientContext &context_p, const RadixPartitionedHashTable &radix_ht) + : context(context_p), finished(false), scan_idx(0), scan_done(0) { + for (column_t column_id = 0; column_id < radix_ht.group_types.size(); column_id++) { + column_ids.push_back(column_id); } +} - idx_t repartition_radix_bits; - idx_t concurrent_repartitions; - idx_t tasks_per_partition; - GetRepartitionInfo(executor.context, state, repartition_radix_bits, concurrent_repartitions, tasks_per_partition); - if (repartition_radix_bits == gstate.partition_info->radix_bits) { - // No repartitioning necessary - for (idx_t r = 0; r < gstate.partition_info->n_partitions; r++) { - D_ASSERT(gstate.partition_info->n_partitions <= gstate.finalized_hts.size()); - D_ASSERT(gstate.finalized_hts[r]); - tasks.push_back(make_uniq(executor, event, gstate, r)); - } - } else { - // Schedule repartition / finalize tasks - ScheduleRepartitionTasks(executor, event, state, tasks, repartition_radix_bits, concurrent_repartitions, - tasks_per_partition); - } -} - -void RadixPartitionedHashTable::ScheduleRepartitionTasks(Executor &executor, const shared_ptr &event, - GlobalSinkState &state, vector> &tasks, - const idx_t repartition_radix_bits, - const idx_t concurrent_repartitions, - const idx_t tasks_per_partition) const { - auto &gstate = state.Cast(); - D_ASSERT(repartition_radix_bits > gstate.partition_info->radix_bits); - const auto num_partitions_before = gstate.partition_info->n_partitions; - const auto multiplier = RadixPartitioning::NumberOfPartitions(repartition_radix_bits) / num_partitions_before; - - // Inititialize gstate - auto new_partition_info = - make_uniq(RadixPartitioning::NumberOfPartitions(repartition_radix_bits)); - gstate.repartitioned = true; - gstate.repartition_tasks_per_partition = tasks_per_partition; - gstate.repartition_tasks.resize(num_partitions_before); - gstate.repartition_tasks_assigned = make_uniq_array>(num_partitions_before); - gstate.repartition_tasks_done = make_uniq_array>(num_partitions_before); - gstate.finalize_assigned = make_uniq_array>(new_partition_info->n_partitions); - for (idx_t partition_idx = 0; partition_idx < num_partitions_before; partition_idx++) { - gstate.repartition_tasks_assigned[partition_idx] = 0; - gstate.repartition_tasks_done[partition_idx] = 0; - - // Grab intermediate data from gstate - HashTableList partition_list; - for (auto &pht : gstate.intermediate_hts) { - for (auto &ht : pht->GetPartition(partition_idx)) { - partition_list.push_back(std::move(ht)); - } - } +bool RadixHTGlobalSourceState::AssignTask(RadixHTGlobalSinkState &sink, RadixHTLocalSourceState &lstate) { + D_ASSERT(lstate.scan_status != RadixHTScanStatus::IN_PROGRESS); - // Spread the data across the tasks - const idx_t hts_per_task = (partition_list.size() + tasks_per_partition - 1) / tasks_per_partition; - idx_t ht_idx = 0; - for (idx_t task_idx = 0; task_idx < tasks_per_partition; task_idx++) { - auto task_ht = - make_uniq(executor.context, BufferAllocator::Get(executor.context), - *new_partition_info, group_types, op.payload_types, op.bindings); - auto ht_idx_to = MinValue(ht_idx + hts_per_task, partition_list.size()); - for (; ht_idx < ht_idx_to; ht_idx++) { - auto &ht = partition_list[ht_idx]; - task_ht->Append(*ht); - ht.reset(); - } - gstate.repartition_tasks[partition_idx].push_back(std::move(task_ht)); - } + const auto n_partitions = sink.partitions.size(); + if (scan_done == n_partitions) { + finished = true; + return false; + } + // We first try to assign a Scan task, then a Finalize task if that didn't work, without using any locks - for (idx_t i = 0; i < multiplier; i++) { - gstate.finalize_assigned[partition_idx * multiplier + i] = false; + // We need an atomic compare-and-swap to assign a Scan task, because we need to only increment + // the 'scan_idx' atomic if the 'finalize' of that partition is true, i.e., ready to be scanned + bool scan_assigned = true; + do { + lstate.task_idx = scan_idx.load(); + if (lstate.task_idx >= n_partitions || !sink.partitions[lstate.task_idx]->finalized) { + scan_assigned = false; + break; } - } + } while (!std::atomic_compare_exchange_weak(&scan_idx, &lstate.task_idx, lstate.task_idx + 1)); - // Schedule tasks equal to number of therads - const idx_t num_threads = TaskScheduler::GetScheduler(executor.context).NumberOfThreads(); - for (idx_t i = 0; i < num_threads; i++) { - tasks.emplace_back(make_shared(executor, event, gstate, num_partitions_before)); + if (scan_assigned) { + // We successfully assigned a Scan task + D_ASSERT(lstate.task_idx < n_partitions && sink.partitions[lstate.task_idx]->finalized); + lstate.task = RadixHTSourceTaskType::SCAN; + lstate.scan_status = RadixHTScanStatus::INIT; + return true; } - gstate.intermediate_hts.clear(); - gstate.intermediate_hts.resize(num_partitions_before * tasks_per_partition); + // We can just increment the atomic here, much simpler than assigning the scan task + lstate.task_idx = sink.finalize_idx++; + if (lstate.task_idx < n_partitions) { + // We successfully assigned a Finalize task + lstate.task = RadixHTSourceTaskType::FINALIZE; + return true; + } - gstate.partition_info = std::move(new_partition_info); - InitializeFinalizedHTs(executor.context, state); + // We didn't manage to assign a finalize task + return false; } -bool RadixPartitionedHashTable::ForceSingleHT(GlobalSinkState &state) { - auto &gstate = state.Cast(); - return gstate.partition_info->n_partitions < 2; +RadixHTLocalSourceState::RadixHTLocalSourceState(ExecutionContext &context, const RadixPartitionedHashTable &radix_ht) + : task(RadixHTSourceTaskType::NO_TASK), scan_status(RadixHTScanStatus::DONE), layout(radix_ht.GetLayout().Copy()), + aggregate_allocator(BufferAllocator::Get(context.client)) { + auto &allocator = BufferAllocator::Get(context.client); + auto scan_chunk_types = radix_ht.group_types; + for (auto &aggr_type : radix_ht.op.aggregate_return_types) { + scan_chunk_types.push_back(aggr_type); + } + scan_chunk.Initialize(allocator, scan_chunk_types); } -bool RadixPartitionedHashTable::AnyPartitioned(GlobalSinkState &state) { - auto &gstate = state.Cast(); - for (auto &pht : gstate.intermediate_hts) { - if (pht->IsPartitioned()) { - return true; - } +void RadixHTLocalSourceState::ExecuteTask(RadixHTGlobalSinkState &sink, RadixHTGlobalSourceState &gstate, + DataChunk &chunk) { + switch (task) { + case RadixHTSourceTaskType::FINALIZE: + Finalize(sink, gstate); + break; + case RadixHTSourceTaskType::SCAN: + Scan(sink, gstate, chunk); + break; + default: + throw InternalException("Unexpected RadixHTSourceTaskType in ExecuteTask!"); } - return false; } -void RadixPartitionedHashTable::GetRepartitionInfo(ClientContext &context, GlobalSinkState &state, - idx_t &repartition_radix_bits, idx_t &concurrent_repartitions, - idx_t &tasks_per_partition) { - auto &gstate = state.Cast(); - const auto num_partitions = gstate.partition_info->n_partitions; - const auto radix_bits = gstate.partition_info->radix_bits; - D_ASSERT(IsPowerOfTwo(num_partitions)); - - vector partition_counts(num_partitions, 0); - vector partition_sizes(num_partitions, 0); - for (const auto &ht : gstate.intermediate_hts) { - for (idx_t partition_idx = 0; partition_idx < num_partitions; partition_idx++) { - partition_counts[partition_idx] += ht->GetPartitionCount(partition_idx); - partition_sizes[partition_idx] += ht->GetPartitionSize(partition_idx); - } - } +void RadixHTLocalSourceState::Finalize(RadixHTGlobalSinkState &sink, RadixHTGlobalSourceState &gstate) { + D_ASSERT(task == RadixHTSourceTaskType::FINALIZE); + D_ASSERT(scan_status != RadixHTScanStatus::IN_PROGRESS); - idx_t total_size = 0; - idx_t max_partition_idx = 0; - idx_t max_partition_size = 0; - for (idx_t partition_idx = 0; partition_idx < num_partitions; partition_idx++) { - const auto &partition_count = partition_counts[partition_idx]; - const auto &partition_size = partition_sizes[partition_idx]; - auto partition_ht_size = - partition_size + GroupedAggregateHashTable::FirstPartSize(partition_count, HtEntryType::HT_WIDTH_64); - if (partition_ht_size > max_partition_size) { - max_partition_idx = partition_idx; - max_partition_size = partition_ht_size; - } - total_size += partition_ht_size; - } - - // Switch to out-of-core finalize at ~60% - const auto max_ht_size = double(0.6) * BufferManager::GetBufferManager(context).GetMaxMemory(); - const idx_t n_threads = PreviousPowerOfTwo(TaskScheduler::GetScheduler(context).NumberOfThreads()); - D_ASSERT(IsPowerOfTwo(n_threads)); - if (!context.config.force_external && total_size < max_ht_size) { - // In-memory finalize - if (num_partitions >= n_threads) { // Can already keep all threads busy - repartition_radix_bits = radix_bits; - tasks_per_partition = 1; - } else { // Repartition to keep all threads busy - // Can't have coverage because RadixHTGlobalState::MAX_RADIX_PARTITIONS > threads on github actions - // LCOV_EXCL_START - repartition_radix_bits = RadixPartitioning::RadixBits(NextPowerOfTwo(n_threads)); - tasks_per_partition = n_threads / num_partitions; - // LCOV_EXCL_STOP - } - concurrent_repartitions = num_partitions; + auto &partition = *sink.partitions[task_idx]; + if (partition.data->Count() == 0) { + partition.finalized = true; return; } - // Out-of-core finalize - const auto partition_count = partition_counts[max_partition_idx]; - const auto partition_size = partition_sizes[max_partition_idx]; + if (!ht) { + // Create a HT with sufficient capacity + const auto capacity = GroupedAggregateHashTable::GetCapacityForCount(partition.data->Count()); + ht = sink.radix_ht.CreateHT(gstate.context, capacity, 0); + } else { + // We may want to resize here to the size of this partition, but for now we just assume uniform partition sizes + ht->InitializePartitionedData(); + ht->ClearPointerTable(); + ht->ResetCount(); + } - const auto max_added_bits = RadixPartitioning::MAX_RADIX_BITS - radix_bits; - idx_t added_bits; - for (added_bits = 1; added_bits < max_added_bits; added_bits++) { - double partition_multiplier = RadixPartitioning::NumberOfPartitions(added_bits); + // Now combine the uncombined data using this thread's HT + ht->Combine(*partition.data); + ht->UnpinData(); - auto new_estimated_count = double(partition_count) / partition_multiplier; - auto new_estimated_size = double(partition_size) / partition_multiplier; - auto new_estimated_ht_size = new_estimated_size + GroupedAggregateHashTable::FirstPartSize( - new_estimated_count, HtEntryType::HT_WIDTH_64); + // Move the combined data back to the partition + partition.data = + make_uniq(BufferManager::GetBufferManager(gstate.context), sink.radix_ht.GetLayout()); + partition.data->Combine(*ht->GetPartitionedData()->GetPartitions()[0]); - if (new_estimated_ht_size <= max_ht_size / n_threads) { - break; // Max HT size is safe - } - } - repartition_radix_bits = radix_bits + added_bits; - concurrent_repartitions = MinValue(MaxValue(1, max_ht_size / max_partition_size), n_threads); - tasks_per_partition = NextPowerOfTwo(n_threads / concurrent_repartitions); + // Mark partition as ready to scan + partition.finalized = true; + + // Make sure this thread's aggregate allocator does not get lost + lock_guard guard(sink.lock); + sink.stored_allocators.emplace_back(ht->GetAggregateAllocator()); } -//===--------------------------------------------------------------------===// -// Source -//===--------------------------------------------------------------------===// -class RadixHTGlobalSourceState : public GlobalSourceState { -public: - explicit RadixHTGlobalSourceState(Allocator &allocator, const RadixPartitionedHashTable &ht) - : ht_index(0), initialized(false), finished(false) { - } +void RadixHTLocalSourceState::Scan(RadixHTGlobalSinkState &sink, RadixHTGlobalSourceState &gstate, DataChunk &chunk) { + D_ASSERT(task == RadixHTSourceTaskType::SCAN); + D_ASSERT(scan_status != RadixHTScanStatus::DONE); - //! Heavy handed for now. - mutex lock; - //! The current position to scan the HT for output tuples - idx_t ht_index; - //! The set of aggregate scan states - unsafe_unique_array ht_scan_states; - atomic initialized; - atomic finished; -}; + auto &partition = *sink.partitions[task_idx]; + D_ASSERT(partition.finalized); + auto &data_collection = *partition.data; -class RadixHTLocalSourceState : public LocalSourceState { -public: - explicit RadixHTLocalSourceState(ExecutionContext &context, const RadixPartitionedHashTable &ht) { - auto &allocator = BufferAllocator::Get(context.client); - auto scan_chunk_types = ht.group_types; - for (auto &aggr_type : ht.op.aggregate_return_types) { - scan_chunk_types.push_back(aggr_type); + if (data_collection.Count() == 0) { + scan_status = RadixHTScanStatus::DONE; + if (++gstate.scan_done == sink.partitions.size()) { + gstate.finished = true; } - scan_chunk.Initialize(allocator, scan_chunk_types); + return; } - //! Materialized GROUP BY expressions & aggregates - DataChunk scan_chunk; - //! HT index - idx_t ht_index = DConstants::INVALID_INDEX; - //! A reference to the current HT that we are scanning - shared_ptr ht; - //! Scan state for the current HT - TupleDataLocalScanState scan_state; -}; + if (scan_status == RadixHTScanStatus::INIT) { + data_collection.InitializeScan(scan_state, gstate.column_ids, sink.scan_pin_properties); + scan_status = RadixHTScanStatus::IN_PROGRESS; + } -unique_ptr RadixPartitionedHashTable::GetGlobalSourceState(ClientContext &context) const { - return make_uniq(BufferAllocator::Get(context), *this); -} + if (!data_collection.Scan(scan_state, scan_chunk)) { + scan_status = RadixHTScanStatus::DONE; + if (++gstate.scan_done == sink.partitions.size()) { + gstate.finished = true; + } + if (sink.scan_pin_properties == TupleDataPinProperties::DESTROY_AFTER_DONE) { + data_collection.Reset(); + } + return; + } -unique_ptr RadixPartitionedHashTable::GetLocalSourceState(ExecutionContext &context) const { - return make_uniq(context, *this); -} + RowOperationsState row_state(aggregate_allocator); + const auto group_cols = layout.ColumnCount() - 1; + RowOperations::FinalizeStates(row_state, layout, scan_state.chunk_state.row_locations, scan_chunk, group_cols); -idx_t RadixPartitionedHashTable::Size(GlobalSinkState &sink_state) const { - auto &gstate = sink_state.Cast(); - if (gstate.is_empty && grouping_set.empty()) { - return 1; + if (sink.scan_pin_properties == TupleDataPinProperties::DESTROY_AFTER_DONE && layout.HasDestructor()) { + RowOperations::DestroyStates(row_state, layout, scan_state.chunk_state.row_locations, scan_chunk.size()); } - idx_t count = 0; - for (const auto &ht : gstate.finalized_hts) { - count += ht->Count(); + auto &radix_ht = sink.radix_ht; + idx_t chunk_index = 0; + for (auto &entry : radix_ht.grouping_set) { + chunk.data[entry].Reference(scan_chunk.data[chunk_index++]); + } + for (auto null_group : radix_ht.null_groups) { + chunk.data[null_group].SetVectorType(VectorType::CONSTANT_VECTOR); + ConstantVector::SetNull(chunk.data[null_group], true); + } + D_ASSERT(radix_ht.grouping_set.size() + radix_ht.null_groups.size() == radix_ht.op.GroupCount()); + for (idx_t col_idx = 0; col_idx < radix_ht.op.aggregates.size(); col_idx++) { + chunk.data[radix_ht.op.GroupCount() + col_idx].Reference( + scan_chunk.data[radix_ht.group_types.size() + col_idx]); + } + D_ASSERT(radix_ht.op.grouping_functions.size() == radix_ht.grouping_values.size()); + for (idx_t i = 0; i < radix_ht.op.grouping_functions.size(); i++) { + chunk.data[radix_ht.op.GroupCount() + radix_ht.op.aggregates.size() + i].Reference(radix_ht.grouping_values[i]); + } + chunk.SetCardinality(scan_chunk); + D_ASSERT(chunk.size() != 0); +} + +bool RadixHTLocalSourceState::TaskFinished() { + switch (task) { + case RadixHTSourceTaskType::FINALIZE: + return true; + case RadixHTSourceTaskType::SCAN: + return scan_status == RadixHTScanStatus::DONE; + default: + D_ASSERT(task == RadixHTSourceTaskType::NO_TASK); + return true; } - return count; } SourceResultType RadixPartitionedHashTable::GetData(ExecutionContext &context, DataChunk &chunk, - GlobalSinkState &sink_state, OperatorSourceInput &input) const { - auto &gstate = sink_state.Cast(); - auto &state = input.global_state.Cast(); + GlobalSinkState &sink_p, OperatorSourceInput &input) const { + auto &sink = sink_p.Cast(); + D_ASSERT(sink.finalized); + + auto &gstate = input.global_state.Cast(); auto &lstate = input.local_state.Cast(); - D_ASSERT(gstate.is_finalized); - if (state.finished) { + D_ASSERT(sink.scan_pin_properties == TupleDataPinProperties::UNPIN_AFTER_DONE || + sink.scan_pin_properties == TupleDataPinProperties::DESTROY_AFTER_DONE); + + if (gstate.finished) { return SourceResultType::FINISHED; } - // special case hack to sort out aggregating from empty intermediates - // for aggregations without groups - if (gstate.is_empty && grouping_set.empty()) { + // Special case hack to sort out aggregating from empty intermediates for aggregations without groups + if (CountInternal(sink_p) == 0 && grouping_set.empty()) { D_ASSERT(chunk.ColumnCount() == null_groups.size() + op.aggregates.size() + op.grouping_functions.size()); - // for each column in the aggregates, set to initial state + // For each column in the aggregates, set to initial state chunk.SetCardinality(1); for (auto null_group : null_groups) { chunk.data[null_group].SetVectorType(VectorType::CONSTANT_VECTOR); @@ -666,97 +786,17 @@ SourceResultType RadixPartitionedHashTable::GetData(ExecutionContext &context, D for (idx_t i = 0; i < op.grouping_functions.size(); i++) { chunk.data[null_groups.size() + op.aggregates.size() + i].Reference(grouping_values[i]); } - state.finished = true; - return chunk.size() == 0 ? SourceResultType::FINISHED : SourceResultType::HAVE_MORE_OUTPUT; - } - if (gstate.is_empty) { - state.finished = true; - return chunk.size() == 0 ? SourceResultType::FINISHED : SourceResultType::HAVE_MORE_OUTPUT; - } - idx_t elements_found = 0; - - lstate.scan_chunk.Reset(); - if (!state.initialized) { - lock_guard l(state.lock); - if (!state.initialized) { - auto &finalized_hts = gstate.finalized_hts; - state.ht_scan_states = make_unsafe_uniq_array(finalized_hts.size()); - - const auto &layout = gstate.finalized_hts[0]->GetDataCollection().GetLayout(); - vector column_ids; - column_ids.reserve(layout.ColumnCount() - 1); - for (idx_t col_idx = 0; col_idx < layout.ColumnCount() - 1; col_idx++) { - column_ids.emplace_back(col_idx); - } - - for (idx_t ht_idx = 0; ht_idx < finalized_hts.size(); ht_idx++) { - gstate.finalized_hts[ht_idx]->GetDataCollection().InitializeScan( - state.ht_scan_states.get()[ht_idx].scan_state, column_ids); - } - state.initialized = true; - } + gstate.finished = true; + return SourceResultType::HAVE_MORE_OUTPUT; } - auto &local_scan_state = lstate.scan_state; - while (true) { - D_ASSERT(state.ht_scan_states); - idx_t ht_index; - { - lock_guard l(state.lock); - ht_index = state.ht_index; - if (ht_index >= gstate.finalized_hts.size()) { - state.finished = true; - return chunk.size() == 0 ? SourceResultType::FINISHED : SourceResultType::HAVE_MORE_OUTPUT; - } - } - D_ASSERT(ht_index < gstate.finalized_hts.size()); - if (lstate.ht_index != DConstants::INVALID_INDEX && ht_index != lstate.ht_index) { - lstate.ht->GetDataCollection().FinalizePinState(local_scan_state.pin_state); - } - lstate.ht_index = ht_index; - lstate.ht = gstate.finalized_hts[ht_index]; - D_ASSERT(lstate.ht); - - auto &global_scan_state = state.ht_scan_states[ht_index]; - elements_found = lstate.ht->Scan(global_scan_state, local_scan_state, lstate.scan_chunk); - if (elements_found > 0) { - break; - } - lstate.ht->GetDataCollection().FinalizePinState(local_scan_state.pin_state); - - // move to the next hash table - lock_guard l(state.lock); - ht_index++; - if (ht_index > state.ht_index) { - // we have not yet worked on the table - // move the global index forwards - if (!gstate.multi_scan) { - gstate.finalized_hts[state.ht_index].reset(); - } - state.ht_index = ht_index; + while (!gstate.finished && chunk.size() == 0) { + if (!lstate.TaskFinished() || gstate.AssignTask(sink, lstate)) { + lstate.ExecuteTask(sink, gstate, chunk); } } - // compute the final projection list - chunk.SetCardinality(elements_found); - - idx_t chunk_index = 0; - for (auto &entry : grouping_set) { - chunk.data[entry].Reference(lstate.scan_chunk.data[chunk_index++]); - } - for (auto null_group : null_groups) { - chunk.data[null_group].SetVectorType(VectorType::CONSTANT_VECTOR); - ConstantVector::SetNull(chunk.data[null_group], true); - } - D_ASSERT(grouping_set.size() + null_groups.size() == op.GroupCount()); - for (idx_t col_idx = 0; col_idx < op.aggregates.size(); col_idx++) { - chunk.data[op.GroupCount() + col_idx].Reference(lstate.scan_chunk.data[group_types.size() + col_idx]); - } - D_ASSERT(op.grouping_functions.size() == grouping_values.size()); - for (idx_t i = 0; i < op.grouping_functions.size(); i++) { - chunk.data[op.GroupCount() + op.aggregates.size() + i].Reference(grouping_values[i]); - } - return chunk.size() == 0 ? SourceResultType::FINISHED : SourceResultType::HAVE_MORE_OUTPUT; + return SourceResultType::HAVE_MORE_OUTPUT; } } // namespace duckdb diff --git a/src/duckdb/src/execution/window_executor.cpp b/src/duckdb/src/execution/window_executor.cpp index fc1181d18..fb094e8da 100644 --- a/src/duckdb/src/execution/window_executor.cpp +++ b/src/duckdb/src/execution/window_executor.cpp @@ -344,13 +344,14 @@ void WindowBoundariesState::Update(const idx_t row_idx, const WindowInputColumn // when the partition changes, recompute the boundaries if (!is_same_partition || is_jump) { - partition_start = row_idx; - peer_start = row_idx; - if (is_jump) { - // Go back as far as the previous partition start idx_t n = 1; - partition_start = FindPrevStart(partition_mask, partition_start, row_idx + 1, n); + partition_start = FindPrevStart(partition_mask, 0, row_idx + 1, n); + n = 1; + peer_start = FindPrevStart(order_mask, 0, row_idx + 1, n); + } else { + partition_start = row_idx; + peer_start = row_idx; } // find end of partition diff --git a/src/duckdb/src/function/cast/cast_function_set.cpp b/src/duckdb/src/function/cast/cast_function_set.cpp index 7608736c3..cc152e623 100644 --- a/src/duckdb/src/function/cast/cast_function_set.cpp +++ b/src/duckdb/src/function/cast/cast_function_set.cpp @@ -1,3 +1,4 @@ + #include "duckdb/function/cast/cast_function_set.hpp" #include "duckdb/common/pair.hpp" diff --git a/src/duckdb/src/function/scalar/strftime_format.cpp b/src/duckdb/src/function/scalar/strftime_format.cpp index 7245d4ca6..4d65b1b71 100644 --- a/src/duckdb/src/function/scalar/strftime_format.cpp +++ b/src/duckdb/src/function/scalar/strftime_format.cpp @@ -668,7 +668,7 @@ int StrpTimeFormat::NumericSpecifierWidth(StrTimeSpecifier specifier) { enum class TimeSpecifierAMOrPM : uint8_t { TIME_SPECIFIER_NONE = 0, TIME_SPECIFIER_AM = 1, TIME_SPECIFIER_PM = 2 }; int32_t StrpTimeFormat::TryParseCollection(const char *data, idx_t &pos, idx_t size, const string_t collection[], - idx_t collection_count) { + idx_t collection_count) const { for (idx_t c = 0; c < collection_count; c++) { auto &entry = collection[c]; auto entry_data = entry.GetData(); @@ -695,7 +695,7 @@ int32_t StrpTimeFormat::TryParseCollection(const char *data, idx_t &pos, idx_t s } //! Parses a timestamp using the given specifier -bool StrpTimeFormat::Parse(string_t str, ParseResult &result) { +bool StrpTimeFormat::Parse(string_t str, ParseResult &result) const { auto &result_data = result.data; auto &error_message = result.error_message; auto &error_position = result.error_position; @@ -1146,7 +1146,7 @@ string StrpTimeFormat::ParseResult::FormatError(string_t input, const string &fo FormatStrpTimeError(input.GetString(), error_position), error_message); } -bool StrpTimeFormat::TryParseDate(string_t input, date_t &result, string &error_message) { +bool StrpTimeFormat::TryParseDate(string_t input, date_t &result, string &error_message) const { ParseResult parse_result; if (!Parse(input, parse_result)) { error_message = parse_result.FormatError(input, format_specifier); @@ -1155,7 +1155,7 @@ bool StrpTimeFormat::TryParseDate(string_t input, date_t &result, string &error_ return parse_result.TryToDate(result); } -bool StrpTimeFormat::TryParseTimestamp(string_t input, timestamp_t &result, string &error_message) { +bool StrpTimeFormat::TryParseTimestamp(string_t input, timestamp_t &result, string &error_message) const { ParseResult parse_result; if (!Parse(input, parse_result)) { error_message = parse_result.FormatError(input, format_specifier); diff --git a/src/duckdb/src/function/table/copy_csv.cpp b/src/duckdb/src/function/table/copy_csv.cpp index b43833eae..f5989727d 100644 --- a/src/duckdb/src/function/table/copy_csv.cpp +++ b/src/duckdb/src/function/table/copy_csv.cpp @@ -6,6 +6,7 @@ #include "duckdb/common/types/column/column_data_collection.hpp" #include "duckdb/common/types/string_type.hpp" #include "duckdb/common/vector_operations/vector_operations.hpp" +#include "duckdb/execution/operator/scan/csv/csv_sniffer.hpp" #include "duckdb/function/copy_function.hpp" #include "duckdb/function/scalar/string_functions.hpp" #include "duckdb/function/table/read_csv.hpp" @@ -15,11 +16,20 @@ namespace duckdb { -void SubstringDetection(string &str_1, string &str_2, const string &name_str_1, const string &name_str_2) { - if (str_1.empty() || str_2.empty()) { +void AreOptionsEqual(char &str_1, char &str_2, const string &name_str_1, const string &name_str_2) { + if (str_1 == '\0' || str_2 == '\0') { return; } - if ((str_1.find(str_2) != string::npos || str_2.find(str_1) != std::string::npos)) { + if (str_1 == str_2) { + throw BinderException("%s must not appear in the %s specification and vice versa", name_str_1, name_str_2); + } +} + +void SubstringDetection(char &str_1, string &str_2, const string &name_str_1, const string &name_str_2) { + if (str_1 == '\0' || str_2.empty()) { + return; + } + if (str_2.find(str_1) != string::npos) { throw BinderException("%s must not appear in the %s specification and vice versa", name_str_1, name_str_2); } } @@ -28,34 +38,46 @@ void SubstringDetection(string &str_1, string &str_2, const string &name_str_1, // Bind //===--------------------------------------------------------------------===// +void WriteQuoteOrEscape(Serializer &serializer, char quote_or_escape) { + if (quote_or_escape != '\0') { + serializer.Write(quote_or_escape); + } +} + void BaseCSVData::Finalize() { // verify that the options are correct in the final pass - if (options.escape.empty()) { - options.escape = options.quote; + if (options.dialect_options.state_machine_options.escape == '\0') { + options.dialect_options.state_machine_options.escape = options.dialect_options.state_machine_options.quote; } // escape and delimiter must not be substrings of each other if (options.has_delimiter && options.has_escape) { - SubstringDetection(options.delimiter, options.escape, "DELIMITER", "ESCAPE"); + AreOptionsEqual(options.dialect_options.state_machine_options.delimiter, + options.dialect_options.state_machine_options.escape, "DELIMITER", "ESCAPE"); } // delimiter and quote must not be substrings of each other if (options.has_quote && options.has_delimiter) { - SubstringDetection(options.quote, options.delimiter, "DELIMITER", "QUOTE"); + AreOptionsEqual(options.dialect_options.state_machine_options.quote, + options.dialect_options.state_machine_options.delimiter, "DELIMITER", "QUOTE"); } // escape and quote must not be substrings of each other (but can be the same) - if (options.quote != options.escape && options.has_quote && options.has_escape) { - SubstringDetection(options.quote, options.escape, "QUOTE", "ESCAPE"); + if (options.dialect_options.state_machine_options.quote != options.dialect_options.state_machine_options.escape && + options.has_quote && options.has_escape) { + AreOptionsEqual(options.dialect_options.state_machine_options.quote, + options.dialect_options.state_machine_options.escape, "QUOTE", "ESCAPE"); } if (!options.null_str.empty()) { // null string and delimiter must not be substrings of each other if (options.has_delimiter) { - SubstringDetection(options.delimiter, options.null_str, "DELIMITER", "NULL"); + SubstringDetection(options.dialect_options.state_machine_options.delimiter, options.null_str, "DELIMITER", + "NULL"); } // quote/escape and nullstr must not be substrings of each other if (options.has_quote) { - SubstringDetection(options.quote, options.null_str, "QUOTE", "NULL"); + SubstringDetection(options.dialect_options.state_machine_options.quote, options.null_str, "QUOTE", "NULL"); } if (options.has_escape) { - SubstringDetection(options.escape, options.null_str, "ESCAPE", "NULL"); + SubstringDetection(options.dialect_options.state_machine_options.escape, options.null_str, "ESCAPE", + "NULL"); } } @@ -63,7 +85,7 @@ void BaseCSVData::Finalize() { if (options.prefix.empty() || options.suffix.empty()) { throw BinderException("COPY ... (FORMAT CSV) must have both PREFIX and SUFFIX, or none at all"); } - if (options.header) { + if (options.dialect_options.header) { throw BinderException("COPY ... (FORMAT CSV)'s HEADER cannot be combined with PREFIX/SUFFIX"); } } @@ -85,16 +107,14 @@ static unique_ptr WriteCSVBind(ClientContext &context, CopyInfo &i bind_data->options.force_quote.resize(names.size(), false); } bind_data->Finalize(); - bind_data->is_simple = bind_data->options.delimiter.size() == 1 && bind_data->options.escape.size() == 1 && - bind_data->options.quote.size() == 1; - if (bind_data->is_simple) { - bind_data->requires_quotes = make_unsafe_uniq_array(256); - memset(bind_data->requires_quotes.get(), 0, sizeof(bool) * 256); - bind_data->requires_quotes['\n'] = true; - bind_data->requires_quotes['\r'] = true; - bind_data->requires_quotes[bind_data->options.delimiter[0]] = true; - bind_data->requires_quotes[bind_data->options.quote[0]] = true; - } + + bind_data->requires_quotes = make_unsafe_uniq_array(256); + memset(bind_data->requires_quotes.get(), 0, sizeof(bool) * 256); + bind_data->requires_quotes['\n'] = true; + bind_data->requires_quotes['\r'] = true; + bind_data->requires_quotes[bind_data->options.dialect_options.state_machine_options.delimiter] = true; + bind_data->requires_quotes[bind_data->options.dialect_options.state_machine_options.quote] = true; + if (!bind_data->options.write_newline.empty()) { bind_data->newline = bind_data->options.write_newline; } @@ -129,13 +149,24 @@ static unique_ptr ReadCSVBind(ClientContext &context, CopyInfo &in for (auto &option : info.options) { options_map[option.first] = ConvertVectorToValue(std::move(option.second)); } + options.file_path = bind_data->files[0]; + options.name_list = expected_names; + options.sql_type_list = expected_types; + for (idx_t i = 0; i < expected_types.size(); i++) { + options.sql_types_per_column[expected_names[i]] = i; + } bind_data->FinalizeRead(context); - if (!bind_data->single_threaded && options.auto_detect) { - options.file_path = bind_data->files[0]; - options.name_list = expected_names; - auto initial_reader = make_uniq(context, options, expected_types); - options = initial_reader->options; + if (options.auto_detect) { + // We must run the sniffer. + auto file_handle = BaseCSVReader::OpenCSV(context, options); + auto buffer_manager = make_shared(context, std::move(file_handle), options); + CSVSniffer sniffer(options, buffer_manager, bind_data->state_machine_cache); + auto sniffer_result = sniffer.SniffCSV(); + bind_data->csv_types = sniffer_result.return_types; + bind_data->csv_names = sniffer_result.names; + bind_data->return_types = sniffer_result.return_types; + bind_data->return_names = sniffer_result.names; } return std::move(bind_data); } @@ -143,7 +174,7 @@ static unique_ptr ReadCSVBind(ClientContext &context, CopyInfo &in //===--------------------------------------------------------------------===// // Helper writing functions //===--------------------------------------------------------------------===// -static string AddEscapes(string &to_be_escaped, const string &escape, const string &val) { +static string AddEscapes(char &to_be_escaped, const char &escape, const string &val) { idx_t i = 0; string new_val = ""; idx_t found = val.find(to_be_escaped); @@ -153,8 +184,10 @@ static string AddEscapes(string &to_be_escaped, const string &escape, const stri new_val += val[i]; i++; } - new_val += escape; - found = val.find(to_be_escaped, found + escape.length()); + if (escape != '\0') { + new_val += escape; + found = val.find(to_be_escaped, found + 1); + } } while (i < val.length()) { new_val += val[i]; @@ -169,43 +202,16 @@ static bool RequiresQuotes(WriteCSVData &csv_data, const char *str, idx_t len) { if (len == options.null_str.size() && memcmp(str, options.null_str.c_str(), len) == 0) { return true; } - if (csv_data.is_simple) { - // simple CSV: check for newlines, quotes and delimiter all at once - auto str_data = reinterpret_cast(str); - for (idx_t i = 0; i < len; i++) { - if (csv_data.requires_quotes[str_data[i]]) { - // this byte requires quotes - write a quoted string - return true; - } - } - // no newline, quote or delimiter in the string - // no quoting or escaping necessary - return false; - } else { - // CSV with complex quotes/delimiter (multiple bytes) - - // first check for \n, \r, \n\r in string - for (idx_t i = 0; i < len; i++) { - if (str[i] == '\n' || str[i] == '\r') { - // newline, write a quoted string - return true; - } - } - - // check for delimiter - if (options.delimiter.length() != 0 && - ContainsFun::Find(const_uchar_ptr_cast(str), len, const_uchar_ptr_cast(options.delimiter.c_str()), - options.delimiter.size()) != DConstants::INVALID_INDEX) { + auto str_data = reinterpret_cast(str); + for (idx_t i = 0; i < len; i++) { + if (csv_data.requires_quotes[str_data[i]]) { + // this byte requires quotes - write a quoted string return true; } - // check for quote - if (options.quote.length() != 0 && - ContainsFun::Find(const_uchar_ptr_cast(str), len, const_uchar_ptr_cast(options.quote.c_str()), - options.quote.size()) != DConstants::INVALID_INDEX) { - return true; - } - return false; } + // no newline, quote or delimiter in the string + // no quoting or escaping necessary + return false; } static void WriteQuotedString(Serializer &serializer, WriteCSVData &csv_data, const char *str, idx_t len, @@ -218,46 +224,37 @@ static void WriteQuotedString(Serializer &serializer, WriteCSVData &csv_data, co if (force_quote) { // quoting is enabled: we might need to escape things in the string bool requires_escape = false; - if (csv_data.is_simple) { - // simple CSV - // do a single loop to check for a quote or escape value - for (idx_t i = 0; i < len; i++) { - if (str[i] == options.quote[0] || str[i] == options.escape[0]) { - requires_escape = true; - break; - } - } - } else { - // complex CSV - // check for quote or escape separately - if (options.quote.length() != 0 && - ContainsFun::Find(const_uchar_ptr_cast(str), len, const_uchar_ptr_cast(options.quote.c_str()), - options.quote.size()) != DConstants::INVALID_INDEX) { - requires_escape = true; - } else if (options.escape.length() != 0 && - ContainsFun::Find(const_uchar_ptr_cast(str), len, const_uchar_ptr_cast(options.escape.c_str()), - options.escape.size()) != DConstants::INVALID_INDEX) { + // simple CSV + // do a single loop to check for a quote or escape value + for (idx_t i = 0; i < len; i++) { + if (str[i] == options.dialect_options.state_machine_options.quote || + str[i] == options.dialect_options.state_machine_options.escape) { requires_escape = true; + break; } } + if (!requires_escape) { // fast path: no need to escape anything - serializer.WriteBufferData(options.quote); + WriteQuoteOrEscape(serializer, options.dialect_options.state_machine_options.quote); serializer.WriteData(const_data_ptr_cast(str), len); - serializer.WriteBufferData(options.quote); + WriteQuoteOrEscape(serializer, options.dialect_options.state_machine_options.quote); return; } // slow path: need to add escapes string new_val(str, len); - new_val = AddEscapes(options.escape, options.escape, new_val); - if (options.escape != options.quote) { + new_val = AddEscapes(options.dialect_options.state_machine_options.escape, + options.dialect_options.state_machine_options.escape, new_val); + if (options.dialect_options.state_machine_options.escape != + options.dialect_options.state_machine_options.quote) { // need to escape quotes separately - new_val = AddEscapes(options.quote, options.escape, new_val); + new_val = AddEscapes(options.dialect_options.state_machine_options.quote, + options.dialect_options.state_machine_options.escape, new_val); } - serializer.WriteBufferData(options.quote); + WriteQuoteOrEscape(serializer, options.dialect_options.state_machine_options.quote); serializer.WriteBufferData(new_val); - serializer.WriteBufferData(options.quote); + WriteQuoteOrEscape(serializer, options.dialect_options.state_machine_options.quote); } else { serializer.WriteData(const_data_ptr_cast(str), len); } @@ -335,12 +332,12 @@ static unique_ptr WriteCSVInitializeGlobal(ClientContext &co global_data->WriteData(options.prefix.c_str(), options.prefix.size()); } - if (options.header) { + if (options.dialect_options.header) { BufferedSerializer serializer; // write the header line to the file for (idx_t i = 0; i < csv_data.options.name_list.size(); i++) { if (i != 0) { - serializer.WriteBufferData(options.delimiter); + WriteQuoteOrEscape(serializer, options.dialect_options.state_machine_options.delimiter); } WriteQuotedString(serializer, csv_data, csv_data.options.name_list[i].c_str(), csv_data.options.name_list[i].size(), false); @@ -365,11 +362,12 @@ static void WriteCSVChunkInternal(ClientContext &context, FunctionData &bind_dat if (csv_data.sql_types[col_idx].id() == LogicalTypeId::VARCHAR) { // VARCHAR, just reinterpret (cannot reference, because LogicalTypeId::VARCHAR is used by the JSON type too) cast_chunk.data[col_idx].Reinterpret(input.data[col_idx]); - } else if (options.has_format[LogicalTypeId::DATE] && csv_data.sql_types[col_idx].id() == LogicalTypeId::DATE) { + } else if (options.dialect_options.has_format[LogicalTypeId::DATE] && + csv_data.sql_types[col_idx].id() == LogicalTypeId::DATE) { // use the date format to cast the chunk csv_data.options.write_date_format[LogicalTypeId::DATE].ConvertDateVector( input.data[col_idx], cast_chunk.data[col_idx], input.size()); - } else if (options.has_format[LogicalTypeId::TIMESTAMP] && + } else if (options.dialect_options.has_format[LogicalTypeId::TIMESTAMP] && (csv_data.sql_types[col_idx].id() == LogicalTypeId::TIMESTAMP || csv_data.sql_types[col_idx].id() == LogicalTypeId::TIMESTAMP_TZ)) { // use the timestamp format to cast the chunk @@ -392,7 +390,7 @@ static void WriteCSVChunkInternal(ClientContext &context, FunctionData &bind_dat // write values for (idx_t col_idx = 0; col_idx < cast_chunk.ColumnCount(); col_idx++) { if (col_idx != 0) { - writer.WriteBufferData(options.delimiter); + WriteQuoteOrEscape(writer, options.dialect_options.state_machine_options.delimiter); } if (FlatVector::IsNull(cast_chunk.data[col_idx], row_idx)) { // write null value diff --git a/src/duckdb/src/function/table/read_csv.cpp b/src/duckdb/src/function/table/read_csv.cpp index fcb6d8601..2ba4f876e 100644 --- a/src/duckdb/src/function/table/read_csv.cpp +++ b/src/duckdb/src/function/table/read_csv.cpp @@ -1,20 +1,21 @@ #include "duckdb/function/table/read_csv.hpp" -#include "duckdb/function/function_set.hpp" -#include "duckdb/main/client_context.hpp" -#include "duckdb/main/database.hpp" -#include "duckdb/common/string_util.hpp" #include "duckdb/common/enum_util.hpp" +#include "duckdb/common/multi_file_reader.hpp" +#include "duckdb/common/string_util.hpp" #include "duckdb/common/union_by_name.hpp" +#include "duckdb/execution/operator/persistent/csv_rejects_table.hpp" +#include "duckdb/execution/operator/scan/csv/csv_line_info.hpp" +#include "duckdb/execution/operator/scan/csv/csv_sniffer.hpp" +#include "duckdb/function/function_set.hpp" +#include "duckdb/main/client_context.hpp" +#include "duckdb/main/client_data.hpp" #include "duckdb/main/config.hpp" +#include "duckdb/main/database.hpp" #include "duckdb/parser/expression/constant_expression.hpp" #include "duckdb/parser/expression/function_expression.hpp" #include "duckdb/parser/tableref/table_function_ref.hpp" #include "duckdb/planner/operator/logical_get.hpp" #include "duckdb/main/extension_helper.hpp" -#include "duckdb/common/multi_file_reader.hpp" -#include "duckdb/main/client_data.hpp" -#include "duckdb/execution/operator/persistent/csv_line_info.hpp" -#include "duckdb/execution/operator/persistent/csv_rejects_table.hpp" #include "duckdb/common/serializer/format_serializer.hpp" #include "duckdb/common/serializer/format_deserializer.hpp" @@ -26,23 +27,22 @@ unique_ptr ReadCSV::OpenCSV(const string &file_path, FileCompress ClientContext &context) { auto &fs = FileSystem::GetFileSystem(context); auto &allocator = BufferAllocator::Get(context); - return CSVFileHandle::OpenFile(fs, allocator, file_path, compression, false); + return CSVFileHandle::OpenFile(fs, allocator, file_path, compression); } void ReadCSVData::FinalizeRead(ClientContext &context) { BaseCSVData::Finalize(); // Here we identify if we can run this CSV file on parallel or not. - bool null_or_empty = options.delimiter.empty() || options.escape.empty() || options.quote.empty() || - options.delimiter[0] == '\0' || options.escape[0] == '\0' || options.quote[0] == '\0'; - bool complex_options = options.delimiter.size() > 1 || options.escape.size() > 1 || options.quote.size() > 1; bool not_supported_options = options.null_padding; auto number_of_threads = TaskScheduler::GetScheduler(context).NumberOfThreads(); - if (options.parallel_mode != ParallelMode::PARALLEL && int64_t(files.size() * 2) >= number_of_threads) { + //! If we have many csv files, we run single-threaded on each file and parallelize on the number of files + bool many_csv_files = files.size() > 1 && int64_t(files.size() * 2) >= number_of_threads; + if (options.parallel_mode != ParallelMode::PARALLEL && many_csv_files) { single_threaded = true; } - if (options.parallel_mode == ParallelMode::SINGLE_THREADED || null_or_empty || not_supported_options || - complex_options || options.new_line == NewLineIdentifier::MIX) { + if (options.parallel_mode == ParallelMode::SINGLE_THREADED || not_supported_options || + options.dialect_options.new_line == NewLineIdentifier::MIX) { // not supported for parallel CSV reading single_threaded = true; } @@ -231,10 +231,14 @@ static unique_ptr ReadCSVBind(ClientContext &context, TableFunctio } if (options.auto_detect) { options.file_path = result->files[0]; - auto initial_reader = make_uniq(context, options); - return_types.assign(initial_reader->return_types.begin(), initial_reader->return_types.end()); + // Initialize Buffer Manager and Sniffer + auto file_handle = BaseCSVReader::OpenCSV(context, options); + result->buffer_manager = make_shared(context, std::move(file_handle), options); + CSVSniffer sniffer(options, result->buffer_manager, result->state_machine_cache); + auto sniffer_result = sniffer.SniffCSV(); + return_types = sniffer_result.return_types; if (names.empty()) { - names.assign(initial_reader->names.begin(), initial_reader->names.end()); + names = sniffer_result.names; } else { if (explicitly_set_columns) { // The user has influenced the names, can't assume they are valid anymore @@ -246,10 +250,8 @@ static unique_ptr ReadCSVBind(ClientContext &context, TableFunctio } else { D_ASSERT(return_types.size() == names.size()); } - initial_reader->names = names; } - options = initial_reader->options; - result->initial_reader = std::move(initial_reader); + } else { D_ASSERT(return_types.size() == names.size()); } @@ -275,15 +277,10 @@ static unique_ptr ReadCSVBind(ClientContext &context, TableFunctio } else { result->reader_bind = MultiFileReader::BindOptions(options.file_options, result->files, return_types, names); } - result->return_types = return_types; result->return_names = names; result->FinalizeRead(context); - if (options.auto_detect) { - result->initial_reader->options = options; - } - return std::move(result); } @@ -299,17 +296,25 @@ static unique_ptr ReadCSVAutoBind(ClientContext &context, TableFun struct ParallelCSVGlobalState : public GlobalTableFunctionState { public: - ParallelCSVGlobalState(ClientContext &context, unique_ptr file_handle_p, - const vector &files_path_p, idx_t system_threads_p, idx_t buffer_size_p, - idx_t rows_to_skip, bool force_parallelism_p, vector column_ids_p, bool has_header) - : file_handle(std::move(file_handle_p)), system_threads(system_threads_p), buffer_size(buffer_size_p), - force_parallelism(force_parallelism_p), column_ids(std::move(column_ids_p)), + ParallelCSVGlobalState(ClientContext &context, shared_ptr buffer_manager_p, + const CSVReaderOptions &options, idx_t system_threads_p, const vector &files_path_p, + bool force_parallelism_p, vector column_ids_p) + : buffer_manager(std::move(buffer_manager_p)), system_threads(system_threads_p), + buffer_size(options.buffer_size), force_parallelism(force_parallelism_p), column_ids(std::move(column_ids_p)), line_info(main_mutex, batch_to_tuple_end, tuple_start, tuple_end) { - file_handle->DisableReset(); current_file_path = files_path_p[0]; - file_size = file_handle->FileSize(); + CSVFileHandle *file_handle_ptr; + + if (!buffer_manager) { + file_handle = ReadCSV::OpenCSV(current_file_path, options.compression, context); + file_handle_ptr = file_handle.get(); + } else { + file_handle_ptr = buffer_manager->file_handle.get(); + } + + file_size = file_handle_ptr->FileSize(); first_file_size = file_size; - on_disk_file = file_handle->OnDiskFile(); + on_disk_file = file_handle_ptr->OnDiskFile(); bytes_read = 0; if (buffer_size < file_size || file_size == 0) { bytes_per_local_state = buffer_size / ParallelCSVGlobalState::MaxThreads(); @@ -321,10 +326,6 @@ struct ParallelCSVGlobalState : public GlobalTableFunctionState { // this boy needs to be at least one. bytes_per_local_state = 1; } - for (idx_t i = 0; i < rows_to_skip; i++) { - file_handle->ReadLine(); - } - first_position = current_csv_position; running_threads = MaxThreads(); // Initialize all the book-keeping variables @@ -337,10 +338,11 @@ struct ParallelCSVGlobalState : public GlobalTableFunctionState { batch_to_tuple_end.resize(file_count); // Initialize the lines read - line_info.lines_read[0][0] = rows_to_skip; - if (has_header) { + line_info.lines_read[0][0] = options.dialect_options.skip_rows; + if (options.has_header && options.dialect_options.header) { line_info.lines_read[0][0]++; } + first_position = options.dialect_options.true_start; } explicit ParallelCSVGlobalState(idx_t system_threads_p) : system_threads(system_threads_p), line_info(main_mutex, batch_to_tuple_end, tuple_start, tuple_end) { @@ -390,9 +392,7 @@ struct ParallelCSVGlobalState : public GlobalTableFunctionState { private: //! File Handle for current file - unique_ptr file_handle; - shared_ptr current_buffer; - shared_ptr next_buffer; + shared_ptr buffer_manager; //! The index of the next file to read (i.e. current file + 1) idx_t file_index = 1; @@ -418,12 +418,9 @@ struct ParallelCSVGlobalState : public GlobalTableFunctionState { //! Forces parallelism for small CSV Files, should only be used for testing. bool force_parallelism = false; - //! Current (Global) position of CSV - idx_t current_csv_position = 0; //! First Position of First Buffer idx_t first_position = 0; //! Current File Number - idx_t file_number = 0; idx_t max_tuple_end = 0; //! The vector stores positions where threads ended the last line they read in the CSV File, and the set stores //! Positions where they started reading the first line. @@ -438,8 +435,10 @@ struct ParallelCSVGlobalState : public GlobalTableFunctionState { vector column_ids; //! Line Info used in error messages LineInfo line_info; - //! Have we initialized our reading - bool initialized = false; + //! Current Buffer index + idx_t cur_buffer_idx = 0; + //! Only used if we don't run auto_detection first + unique_ptr file_handle; }; idx_t ParallelCSVGlobalState::MaxThreads() const { @@ -538,31 +537,33 @@ void LineInfo::Verify(idx_t file_idx, idx_t batch_idx, idx_t cur_first_pos) { problematic_line); } } - bool ParallelCSVGlobalState::Next(ClientContext &context, const ReadCSVData &bind_data, unique_ptr &reader) { lock_guard parallel_lock(main_mutex); - if (!initialized && file_handle) { - current_buffer = make_shared(context, buffer_size, *file_handle, current_csv_position, file_number); - next_buffer = shared_ptr( - current_buffer->Next(*file_handle, buffer_size, current_csv_position, file_number).release()); - initialized = true; + if (!buffer_manager && file_handle) { + buffer_manager = make_shared(context, std::move(file_handle), bind_data.options); + } + if (!buffer_manager) { + return false; } + auto current_buffer = buffer_manager->GetBuffer(cur_buffer_idx); + auto next_buffer = buffer_manager->GetBuffer(cur_buffer_idx + 1); + if (!current_buffer) { // This means we are done with the current file, we need to go to the next one (if exists). if (file_index < bind_data.files.size()) { - current_file_path = bind_data.files[file_index++]; + current_file_path = bind_data.files[file_index]; file_handle = ReadCSV::OpenCSV(current_file_path, bind_data.options.compression, context); - current_csv_position = 0; - file_number++; + buffer_manager = + make_shared(context, std::move(file_handle), bind_data.options, file_index); + cur_buffer_idx = 0; + first_position = 0; local_batch_index = 0; - line_info.lines_read[file_number][local_batch_index] = (bind_data.options.has_header ? 1 : 0); + line_info.lines_read[file_index++][local_batch_index] = (bind_data.options.has_header ? 1 : 0); - current_buffer = - make_shared(context, buffer_size, *file_handle, current_csv_position, file_number); - next_buffer = shared_ptr( - current_buffer->Next(*file_handle, buffer_size, current_csv_position, file_number).release()); + current_buffer = buffer_manager->GetBuffer(cur_buffer_idx); + next_buffer = buffer_manager->GetBuffer(cur_buffer_idx + 1); } else { // We are done scanning. reader.reset(); @@ -570,20 +571,21 @@ bool ParallelCSVGlobalState::Next(ClientContext &context, const ReadCSVData &bin } } // set up the current buffer - line_info.current_batches[file_number].insert(local_batch_index); - auto result = make_uniq(current_buffer, next_buffer, next_byte, next_byte + bytes_per_local_state, - batch_index++, local_batch_index++, &line_info); + line_info.current_batches[file_index - 1].insert(local_batch_index); + auto result = make_uniq( + buffer_manager->GetBuffer(cur_buffer_idx), buffer_manager->GetBuffer(cur_buffer_idx + 1), next_byte, + next_byte + bytes_per_local_state, batch_index++, local_batch_index++, &line_info); // move the byte index of the CSV reader to the next buffer next_byte += bytes_per_local_state; - if (next_byte >= current_buffer->GetBufferSize()) { + if (next_byte >= current_buffer->actual_size) { // We replace the current buffer with the next buffer next_byte = 0; - bytes_read += current_buffer->GetBufferSize(); - current_buffer = next_buffer; - if (next_buffer) { + bytes_read += current_buffer->actual_size; + current_buffer = std::move(next_buffer); + cur_buffer_idx++; + if (current_buffer) { // Next buffer gets the next-next buffer - next_buffer = shared_ptr( - next_buffer->Next(*file_handle, buffer_size, current_csv_position, file_number).release()); + next_buffer = buffer_manager->GetBuffer(cur_buffer_idx + 1); } } if (!reader || reader->options.file_path != current_file_path) { @@ -602,6 +604,9 @@ bool ParallelCSVGlobalState::Next(ClientContext &context, const ReadCSVData &bin reader->names = bind_data.column_info[file_index - 1].names; } else { // regular file - use the standard options + if (!result) { + return false; + } reader = make_uniq(context, bind_data.options, std::move(result), first_position, bind_data.csv_types, file_index - 1); reader->names = bind_data.csv_names; @@ -701,22 +706,11 @@ static unique_ptr ParallelCSVInitGlobal(ClientContext // This can happen when a filename based filter pushdown has eliminated all possible files for this scan. return make_uniq(context.db->NumberOfThreads()); } - unique_ptr file_handle; - bind_data.options.file_path = bind_data.files[0]; - - if (bind_data.initial_reader) { - file_handle = std::move(bind_data.initial_reader->file_handle); - file_handle->Reset(); - file_handle->DisableReset(); - bind_data.initial_reader.reset(); - } else { - file_handle = ReadCSV::OpenCSV(bind_data.options.file_path, bind_data.options.compression, context); - } - return make_uniq( - context, std::move(file_handle), bind_data.files, context.db->NumberOfThreads(), bind_data.options.buffer_size, - bind_data.options.skip_rows, ClientConfig::GetConfig(context).verify_parallelism, input.column_ids, - bind_data.options.header && bind_data.options.has_header); + auto buffer_manager = bind_data.buffer_manager; + return make_uniq(context, buffer_manager, bind_data.options, context.db->NumberOfThreads(), + bind_data.files, ClientConfig::GetConfig(context).verify_parallelism, + input.column_ids); } //===--------------------------------------------------------------------===// @@ -764,7 +758,7 @@ static void ParallelReadCSVFunction(ClientContext &context, TableFunctionInput & if (csv_local_state.csv_reader->finished) { auto verification_updates = csv_local_state.csv_reader->GetVerificationPositions(); csv_global_state.UpdateVerification(verification_updates, - csv_local_state.csv_reader->buffer->buffer->GetFileNumber(), + csv_local_state.csv_reader->buffer->buffer->file_idx, csv_local_state.csv_reader->buffer->local_batch_index); csv_global_state.UpdateLinesRead(*csv_local_state.csv_reader->buffer, csv_local_state.csv_reader->file_idx); auto has_next = csv_global_state.Next(context, bind_data, csv_local_state.csv_reader); @@ -819,17 +813,13 @@ struct SingleThreadedCSVState : public GlobalTableFunctionState { unique_ptr GetCSVReader(ClientContext &context, ReadCSVData &bind_data, idx_t &file_index, idx_t &total_size) { - auto reader = GetCSVReaderInternal(context, bind_data, file_index, total_size); - if (reader) { - reader->file_handle->DisableReset(); - } - return reader; + return GetCSVReaderInternal(context, bind_data, file_index, total_size); } private: unique_ptr GetCSVReaderInternal(ClientContext &context, ReadCSVData &bind_data, idx_t &file_index, idx_t &total_size) { - BufferedCSVReaderOptions options; + CSVReaderOptions options; { lock_guard l(csv_lock); if (initial_reader) { @@ -889,13 +879,7 @@ static unique_ptr SingleThreadedCSVInit(ClientContext return std::move(result); } else { bind_data.options.file_path = bind_data.files[0]; - if (bind_data.initial_reader) { - // If this is a pipe and an initial reader already exists due to read_csv_auto - // We must re-use it, since we can't restart the reader due for it being a pipe. - result->initial_reader = std::move(bind_data.initial_reader); - } else { - result->initial_reader = make_uniq(context, bind_data.options, bind_data.csv_types); - } + result->initial_reader = make_uniq(context, bind_data.options, bind_data.csv_types); if (!bind_data.options.file_options.union_by_name) { result->initial_reader->names = bind_data.csv_names; } @@ -1095,35 +1079,79 @@ void CSVComplexFilterPushdown(ClientContext &context, LogicalGet &get, FunctionD unique_ptr CSVReaderCardinality(ClientContext &context, const FunctionData *bind_data_p) { auto &bind_data = bind_data_p->Cast(); idx_t per_file_cardinality = 0; - if (bind_data.initial_reader && bind_data.initial_reader->file_handle) { + if (bind_data.buffer_manager && bind_data.buffer_manager->file_handle) { auto estimated_row_width = (bind_data.csv_types.size() * 5); - per_file_cardinality = bind_data.initial_reader->file_handle->FileSize() / estimated_row_width; + per_file_cardinality = bind_data.buffer_manager->file_handle->FileSize() / estimated_row_width; } else { // determined through the scientific method as the average amount of rows in a CSV file per_file_cardinality = 42; } return make_uniq(bind_data.files.size() * per_file_cardinality); } +void CSVStateMachineOptions::Serialize(FieldWriter &writer) const { + writer.WriteField(delimiter); + writer.WriteField(quote); + writer.WriteField(escape); +} -void BufferedCSVReaderOptions::Serialize(FieldWriter &writer) const { +void DialectOptions::Serialize(FieldWriter &writer) const { + state_machine_options.Serialize(writer); + writer.WriteField(header); + writer.WriteField(num_cols); + writer.WriteField(new_line); + writer.WriteField(skip_rows); + vector csv_formats; + for (auto &format : date_format) { + writer.WriteField(has_format.find(format.first)->second); + csv_formats.push_back(format.second.format_specifier); + } + writer.WriteList(csv_formats); +} +void CSVStateMachineOptions::Deserialize(FieldReader &reader) { + delimiter = reader.ReadRequired(); + quote = reader.ReadRequired(); + escape = reader.ReadRequired(); +} +void DialectOptions::Deserialize(FieldReader &reader) { + state_machine_options.Deserialize(reader); + header = reader.ReadRequired(); + num_cols = reader.ReadRequired(); + new_line = reader.ReadRequired(); + skip_rows = reader.ReadRequired(); + + bool has_date = reader.ReadRequired(); + bool has_timestamp = reader.ReadRequired(); + auto formats = reader.ReadRequiredList(); + + vector format_types {LogicalTypeId::DATE, LogicalTypeId::TIMESTAMP}; + if (has_date) { + has_format[LogicalTypeId::DATE] = true; + } + if (has_timestamp) { + has_format[LogicalTypeId::TIMESTAMP] = true; + } + for (idx_t f_idx = 0; f_idx < formats.size(); f_idx++) { + auto &format = formats[f_idx]; + auto &type = format_types[f_idx]; + if (format.empty()) { + continue; + } + StrTimeFormat::ParseFormatSpecifier(format, date_format[type]); + } +} + +void CSVReaderOptions::Serialize(FieldWriter &writer) const { // common options writer.WriteField(has_delimiter); - writer.WriteString(delimiter); writer.WriteField(has_quote); - writer.WriteString(quote); writer.WriteField(has_escape); - writer.WriteString(escape); writer.WriteField(has_header); - writer.WriteField(header); writer.WriteField(ignore_errors); - writer.WriteField(num_cols); writer.WriteField(buffer_sample_size); writer.WriteString(null_str); writer.WriteField(compression); - writer.WriteField(new_line); writer.WriteField(allow_quoted_nulls); // read options - writer.WriteField(skip_rows); writer.WriteField(skip_rows_set); writer.WriteField(maximum_line_size); writer.WriteField(normalize_names); @@ -1139,37 +1167,29 @@ void BufferedCSVReaderOptions::Serialize(FieldWriter &writer) const { writer.WriteSerializable(file_options); // write options writer.WriteListNoReference(force_quote); - // FIXME: serialize date_format / has_format - vector csv_formats; - for (auto &format : date_format) { - csv_formats.push_back(format.second.format_specifier); - } - writer.WriteList(csv_formats); + + // reject options writer.WriteString(rejects_table_name); writer.WriteField(rejects_limit); writer.WriteList(rejects_recovery_columns); writer.WriteList(rejects_recovery_column_ids); + + // Serialize Dialect Options + dialect_options.Serialize(writer); } -void BufferedCSVReaderOptions::Deserialize(FieldReader &reader) { +void CSVReaderOptions::Deserialize(FieldReader &reader) { // common options has_delimiter = reader.ReadRequired(); - delimiter = reader.ReadRequired(); has_quote = reader.ReadRequired(); - quote = reader.ReadRequired(); has_escape = reader.ReadRequired(); - escape = reader.ReadRequired(); has_header = reader.ReadRequired(); - header = reader.ReadRequired(); ignore_errors = reader.ReadRequired(); - num_cols = reader.ReadRequired(); buffer_sample_size = reader.ReadRequired(); null_str = reader.ReadRequired(); compression = reader.ReadRequired(); - new_line = reader.ReadRequired(); allow_quoted_nulls = reader.ReadRequired(); // read options - skip_rows = reader.ReadRequired(); skip_rows_set = reader.ReadRequired(); maximum_line_size = reader.ReadRequired(); normalize_names = reader.ReadRequired(); @@ -1185,21 +1205,15 @@ void BufferedCSVReaderOptions::Deserialize(FieldReader &reader) { file_options = reader.ReadRequiredSerializable(); // write options force_quote = reader.ReadRequiredList(); - auto formats = reader.ReadRequiredList(); - vector format_types {LogicalTypeId::DATE, LogicalTypeId::TIMESTAMP}; - for (idx_t f_idx = 0; f_idx < formats.size(); f_idx++) { - auto &format = formats[f_idx]; - auto &type = format_types[f_idx]; - if (format.empty()) { - continue; - } - has_format[type] = true; - StrTimeFormat::ParseFormatSpecifier(format, date_format[type]); - } + + // rejects options rejects_table_name = reader.ReadRequired(); rejects_limit = reader.ReadRequired(); rejects_recovery_columns = reader.ReadRequiredList(); rejects_recovery_column_ids = reader.ReadRequiredList(); + + // dialect options + dialect_options.Deserialize(reader); } static void CSVReaderSerialize(FieldWriter &writer, const FunctionData *bind_data_p, const TableFunction &function) { @@ -1246,7 +1260,7 @@ static void CSVReaderFormatSerialize(FormatSerializer &serializer, const optiona const TableFunction &function) { auto &bind_data = bind_data_p->Cast(); serializer.WriteProperty(100, "extra_info", function.extra_info); - serializer.WriteProperty(101, "csv_data", bind_data); + serializer.WriteProperty(101, "csv_data", &bind_data); } static unique_ptr CSVReaderFormatDeserialize(FormatDeserializer &deserializer, TableFunction &function) { diff --git a/src/duckdb/src/function/table/table_scan.cpp b/src/duckdb/src/function/table/table_scan.cpp index ca310e62e..55ce442f8 100644 --- a/src/duckdb/src/function/table/table_scan.cpp +++ b/src/duckdb/src/function/table/table_scan.cpp @@ -458,7 +458,6 @@ static void TableScanFormatSerialize(FormatSerializer &serializer, const optiona serializer.WriteProperty(103, "is_index_scan", bind_data.is_index_scan); serializer.WriteProperty(104, "is_create_index", bind_data.is_create_index); serializer.WriteProperty(105, "result_ids", bind_data.result_ids); - serializer.WriteProperty(106, "result_ids", bind_data.result_ids); } static unique_ptr TableScanFormatDeserialize(FormatDeserializer &deserializer, TableFunction &function) { @@ -474,7 +473,6 @@ static unique_ptr TableScanFormatDeserialize(FormatDeserializer &d deserializer.ReadProperty(103, "is_index_scan", result->is_index_scan); deserializer.ReadProperty(104, "is_create_index", result->is_create_index); deserializer.ReadProperty(105, "result_ids", result->result_ids); - deserializer.ReadProperty(106, "result_ids", result->result_ids); return std::move(result); } diff --git a/src/duckdb/src/function/table/version/pragma_version.cpp b/src/duckdb/src/function/table/version/pragma_version.cpp index c304a329e..12b08e6a2 100644 --- a/src/duckdb/src/function/table/version/pragma_version.cpp +++ b/src/duckdb/src/function/table/version/pragma_version.cpp @@ -1,8 +1,8 @@ #ifndef DUCKDB_VERSION -#define DUCKDB_VERSION "0.8.2-dev3456" +#define DUCKDB_VERSION "0.8.2-dev3949" #endif #ifndef DUCKDB_SOURCE_ID -#define DUCKDB_SOURCE_ID "44fec4a812" +#define DUCKDB_SOURCE_ID "c21a9cb87c" #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 645f5e79b..511463523 100644 --- a/src/duckdb/src/include/duckdb.h +++ b/src/duckdb/src/include/duckdb.h @@ -1372,6 +1372,18 @@ The resulting type should be destroyed with `duckdb_destroy_logical_type`. DUCKDB_API duckdb_logical_type duckdb_create_union_type(duckdb_logical_type member_types, const char **member_names, idx_t member_count); +/*! +Creates a STRUCT type from the passed member name and type arrays. +The resulting type should be destroyed with `duckdb_destroy_logical_type`. + +* member_types: The array of types that the struct should consist of. +* member_names: The array of names that the struct should consist of. +* member_count: The number of members that were specified for both arrays. +* returns: The logical type. +*/ +DUCKDB_API duckdb_logical_type duckdb_create_struct_type(duckdb_logical_type *member_types, const char **member_names, + idx_t member_count); + /*! Creates a `duckdb_logical_type` of type decimal with the specified width and scale The resulting type should be destroyed with `duckdb_destroy_logical_type`. diff --git a/src/duckdb/src/include/duckdb/common/enum_util.hpp b/src/duckdb/src/include/duckdb/common/enum_util.hpp index 609f0ecfb..a210572e5 100644 --- a/src/duckdb/src/include/duckdb/common/enum_util.hpp +++ b/src/duckdb/src/include/duckdb/common/enum_util.hpp @@ -66,6 +66,8 @@ enum class BlockState : uint8_t; enum class CAPIResultSetType : uint8_t; +enum class CSVState : uint8_t; + enum class CTEMaterialize : uint8_t; enum class CatalogType : uint8_t; @@ -130,6 +132,8 @@ enum class FunctionNullHandling : uint8_t; enum class FunctionSideEffects : uint8_t; +enum class HLLStorageType : uint8_t; + enum class IndexConstraintType : uint8_t; enum class IndexType : uint8_t; @@ -208,6 +212,8 @@ enum class QueryNodeType : uint8_t; enum class QueryResultType : uint8_t; +enum class QuoteRule : uint8_t; + enum class RelationType : uint8_t; enum class ResultModifierType : uint8_t; @@ -334,6 +340,9 @@ const char* EnumUtil::ToChars(BlockState value); template<> const char* EnumUtil::ToChars(CAPIResultSetType value); +template<> +const char* EnumUtil::ToChars(CSVState value); + template<> const char* EnumUtil::ToChars(CTEMaterialize value); @@ -430,6 +439,9 @@ const char* EnumUtil::ToChars(FunctionNullHandling value); template<> const char* EnumUtil::ToChars(FunctionSideEffects value); +template<> +const char* EnumUtil::ToChars(HLLStorageType value); + template<> const char* EnumUtil::ToChars(IndexConstraintType value); @@ -547,6 +559,9 @@ const char* EnumUtil::ToChars(QueryNodeType value); template<> const char* EnumUtil::ToChars(QueryResultType value); +template<> +const char* EnumUtil::ToChars(QuoteRule value); + template<> const char* EnumUtil::ToChars(RelationType value); @@ -710,6 +725,9 @@ BlockState EnumUtil::FromString(const char *value); template<> CAPIResultSetType EnumUtil::FromString(const char *value); +template<> +CSVState EnumUtil::FromString(const char *value); + template<> CTEMaterialize EnumUtil::FromString(const char *value); @@ -806,6 +824,9 @@ FunctionNullHandling EnumUtil::FromString(const char *valu template<> FunctionSideEffects EnumUtil::FromString(const char *value); +template<> +HLLStorageType EnumUtil::FromString(const char *value); + template<> IndexConstraintType EnumUtil::FromString(const char *value); @@ -923,6 +944,9 @@ QueryNodeType EnumUtil::FromString(const char *value); template<> QueryResultType EnumUtil::FromString(const char *value); +template<> +QuoteRule EnumUtil::FromString(const char *value); + template<> RelationType EnumUtil::FromString(const char *value); diff --git a/src/duckdb/src/include/duckdb/common/file_opener.hpp b/src/duckdb/src/include/duckdb/common/file_opener.hpp index b7c23090b..7268599a4 100644 --- a/src/duckdb/src/include/duckdb/common/file_opener.hpp +++ b/src/duckdb/src/include/duckdb/common/file_opener.hpp @@ -16,16 +16,25 @@ namespace duckdb { class ClientContext; class Value; +struct FileOpenerInfo { + string file_path; +}; + //! Abstract type that provide client-specific context to FileSystem. class FileOpener { public: + FileOpener() { + } virtual ~FileOpener() {}; + virtual bool TryGetCurrentSetting(const string &key, Value &result, FileOpenerInfo &info); virtual bool TryGetCurrentSetting(const string &key, Value &result) = 0; virtual ClientContext *TryGetClientContext() = 0; DUCKDB_API static ClientContext *TryGetClientContext(FileOpener *opener); DUCKDB_API static bool TryGetCurrentSetting(FileOpener *opener, const string &key, Value &result); + DUCKDB_API static bool TryGetCurrentSetting(FileOpener *opener, const string &key, Value &result, + FileOpenerInfo &info); }; } // namespace duckdb diff --git a/src/duckdb/src/include/duckdb/common/fixed_size_map.hpp b/src/duckdb/src/include/duckdb/common/fixed_size_map.hpp new file mode 100644 index 000000000..02950e099 --- /dev/null +++ b/src/duckdb/src/include/duckdb/common/fixed_size_map.hpp @@ -0,0 +1,208 @@ +//===----------------------------------------------------------------------===// +// DuckDB +// +// duckdb/common/fixed_size_map.hpp +// +// +//===----------------------------------------------------------------------===// + +#pragma once + +#include "duckdb/common/pair.hpp" +#include "duckdb/common/types.hpp" +#include "duckdb/common/types/validity_mask.hpp" + +namespace duckdb { + +template +struct fixed_size_map_iterator_t; + +template +struct const_fixed_size_map_iterator_t; + +template +class fixed_size_map_t { + friend struct fixed_size_map_iterator_t; + friend struct const_fixed_size_map_iterator_t; + +public: + using key_type = idx_t; + using mapped_type = T; + +public: + explicit fixed_size_map_t(idx_t capacity_p = 0) : capacity(capacity_p) { + resize(capacity); + } + + idx_t size() const { + return count; + } + + void resize(idx_t capacity_p) { + capacity = capacity_p; + occupied = ValidityMask(capacity); + values = make_unsafe_uniq_array(capacity + 1); + clear(); + } + + void clear() { + count = 0; + occupied.SetAllInvalid(capacity); + } + + T &operator[](const idx_t &key) { + D_ASSERT(key < capacity); + count += 1 - occupied.RowIsValid(key); + occupied.SetValidUnsafe(key); + return values[key]; + } + + const T &operator[](const idx_t &key) const { + D_ASSERT(key < capacity); + return values[key]; + } + + fixed_size_map_iterator_t begin() { + return fixed_size_map_iterator_t(begin_internal(), *this); + } + + const_fixed_size_map_iterator_t begin() const { + return const_fixed_size_map_iterator_t(begin_internal(), *this); + } + + fixed_size_map_iterator_t end() { + return fixed_size_map_iterator_t(capacity, *this); + } + + const_fixed_size_map_iterator_t end() const { + return const_fixed_size_map_iterator_t(capacity, *this); + } + + fixed_size_map_iterator_t find(const idx_t &index) { + if (occupied.RowIsValid(index)) { + return fixed_size_map_iterator_t(index, *this); + } else { + return end(); + } + } + + const_fixed_size_map_iterator_t find(const idx_t &index) const { + if (occupied.RowIsValid(index)) { + return const_fixed_size_map_iterator_t(index, *this); + } else { + return end(); + } + } + +private: + idx_t begin_internal() const { + idx_t index; + for (index = 0; index < capacity; index++) { + if (occupied.RowIsValid(index)) { + break; + } + } + return index; + } + +private: + idx_t capacity; + idx_t count; + + ValidityMask occupied; + unsafe_unique_array values; +}; + +template +struct fixed_size_map_iterator_t { +public: + fixed_size_map_iterator_t(idx_t index_p, fixed_size_map_t &map_p) : map(map_p), current(index_p) { + } + + fixed_size_map_iterator_t &operator++() { + for (current++; current < map.capacity; current++) { + if (map.occupied.RowIsValidUnsafe(current)) { + break; + } + } + return *this; + } + + fixed_size_map_iterator_t operator++(int) { + fixed_size_map_iterator_t tmp = *this; + ++(*this); + return tmp; + } + + idx_t &GetKey() { + return current; + } + + const idx_t &GetKey() const { + return current; + } + + T &GetValue() { + return map.values[current]; + } + + const T &GetValue() const { + return map.values[current]; + } + + friend bool operator==(const fixed_size_map_iterator_t &a, const fixed_size_map_iterator_t &b) { + return a.current == b.current; + } + + friend bool operator!=(const fixed_size_map_iterator_t &a, const fixed_size_map_iterator_t &b) { + return !(a == b); + } + +private: + fixed_size_map_t ↦ + idx_t current; +}; + +template +struct const_fixed_size_map_iterator_t { +public: + const_fixed_size_map_iterator_t(idx_t index_p, const fixed_size_map_t &map_p) : map(map_p), current(index_p) { + } + + const_fixed_size_map_iterator_t &operator++() { + for (current++; current < map.capacity; current++) { + if (map.occupied.RowIsValidUnsafe(current)) { + break; + } + } + return *this; + } + + const_fixed_size_map_iterator_t operator++(int) { + const_fixed_size_map_iterator_t tmp = *this; + ++(*this); + return tmp; + } + + const idx_t &GetKey() const { + return current; + } + + const T &GetValue() const { + return map.values[current]; + } + + friend bool operator==(const const_fixed_size_map_iterator_t &a, const const_fixed_size_map_iterator_t &b) { + return a.current == b.current; + } + + friend bool operator!=(const const_fixed_size_map_iterator_t &a, const const_fixed_size_map_iterator_t &b) { + return !(a == b); + } + +private: + const fixed_size_map_t ↦ + idx_t current; +}; + +} // namespace duckdb diff --git a/src/duckdb/src/include/duckdb/common/optional_idx.hpp b/src/duckdb/src/include/duckdb/common/optional_idx.hpp index 652bb9c0c..43e899f56 100644 --- a/src/duckdb/src/include/duckdb/common/optional_idx.hpp +++ b/src/duckdb/src/include/duckdb/common/optional_idx.hpp @@ -31,6 +31,9 @@ class optional_idx { bool IsValid() const { return index != DConstants::INVALID_INDEX; } + void Invalidate() { + index = INVALID_INDEX; + } idx_t GetIndex() { if (index == INVALID_INDEX) { throw InternalException("Attempting to get the index of an optional_idx that is not set"); diff --git a/src/duckdb/src/include/duckdb/common/perfect_map_set.hpp b/src/duckdb/src/include/duckdb/common/perfect_map_set.hpp index 9a1fc8134..1e735a638 100644 --- a/src/duckdb/src/include/duckdb/common/perfect_map_set.hpp +++ b/src/duckdb/src/include/duckdb/common/perfect_map_set.hpp @@ -1,7 +1,7 @@ //===----------------------------------------------------------------------===// // DuckDB // -// duckdb/common/string_map_set.hpp +// duckdb/common/perfect_map_set.hpp // // //===----------------------------------------------------------------------===// @@ -9,6 +9,7 @@ #pragma once #include "duckdb/common/types.hpp" +#include "duckdb/common/types/validity_mask.hpp" #include "duckdb/common/unordered_map.hpp" #include "duckdb/common/unordered_set.hpp" diff --git a/src/duckdb/src/include/duckdb/common/printer.hpp b/src/duckdb/src/include/duckdb/common/printer.hpp index 1d1e809d7..8ecc9f58c 100644 --- a/src/duckdb/src/include/duckdb/common/printer.hpp +++ b/src/duckdb/src/include/duckdb/common/printer.hpp @@ -9,6 +9,7 @@ #pragma once #include "duckdb/common/common.hpp" +#include "duckdb/common/string_util.hpp" namespace duckdb { @@ -21,6 +22,16 @@ class Printer { DUCKDB_API static void Print(OutputStream stream, const string &str); //! Print the object to stderr DUCKDB_API static void Print(const string &str); + //! Print the formatted object to the stream + template + static void PrintF(OutputStream stream, const string &str, Args... params) { + Printer::Print(stream, StringUtil::Format(str, params...)); + } + //! Print the formatted object to stderr + template + static void PrintF(const string &str, Args... params) { + Printer::PrintF(OutputStream::STREAM_STDERR, str, std::forward(params)...); + } //! Directly prints the string to stdout without a newline DUCKDB_API static void RawPrint(OutputStream stream, const string &str); //! Flush an output stream diff --git a/src/duckdb/src/include/duckdb/common/serializer/binary_deserializer.hpp b/src/duckdb/src/include/duckdb/common/serializer/binary_deserializer.hpp index 3da9d8412..565a75e19 100644 --- a/src/duckdb/src/include/duckdb/common/serializer/binary_deserializer.hpp +++ b/src/duckdb/src/include/duckdb/common/serializer/binary_deserializer.hpp @@ -9,6 +9,7 @@ #pragma once #include "duckdb/common/serializer/format_deserializer.hpp" +#include "duckdb/common/serializer/encoding_util.hpp" namespace duckdb { class ClientContext; @@ -20,6 +21,7 @@ class BinaryDeserializer : public FormatDeserializer { OnObjectBegin(); auto result = T::FormatDeserialize(*this); OnObjectEnd(); + D_ASSERT(nesting_level == 0); // make sure we are at the root level return result; } @@ -42,23 +44,35 @@ class BinaryDeserializer : public FormatDeserializer { explicit BinaryDeserializer(data_ptr_t ptr, idx_t length) : ptr(ptr), end_ptr(ptr + length) { deserialize_enum_from_string = false; } - struct State { - uint32_t expected_field_count; - idx_t expected_size; - field_id_t expected_field_id; - uint32_t read_field_count; - State(uint32_t expected_field_count, idx_t expected_size, field_id_t expected_field_id) - : expected_field_count(expected_field_count), expected_size(expected_size), - expected_field_id(expected_field_id), read_field_count(0) { - } - }; - - const char *current_tag = nullptr; - field_id_t current_field_id = 0; data_ptr_t ptr; data_ptr_t end_ptr; - vector stack; + idx_t nesting_level = 0; + + // Allow peeking 1 field ahead + bool has_buffered_field = false; + field_id_t buffered_field = 0; + field_id_t PeekField() { + if (!has_buffered_field) { + buffered_field = ReadPrimitive(); + has_buffered_field = true; + } + return buffered_field; + } + void ConsumeField() { + if (!has_buffered_field) { + buffered_field = ReadPrimitive(); + } else { + has_buffered_field = false; + } + } + field_id_t NextField() { + if (has_buffered_field) { + has_buffered_field = false; + return buffered_field; + } + return ReadPrimitive(); + } template T ReadPrimitive() { @@ -69,39 +83,39 @@ class BinaryDeserializer : public FormatDeserializer { void ReadData(data_ptr_t buffer, idx_t read_size) { if (ptr + read_size > end_ptr) { - throw SerializationException("Failed to deserialize: not enough data in buffer to fulfill read request"); + throw InternalException("Failed to deserialize: not enough data in buffer to fulfill read request"); } memcpy(buffer, ptr, read_size); ptr += read_size; } - // Set the 'tag' of the property to read - void SetTag(const field_id_t field_id, const char *tag) final; + template + T VarIntDecode() { + T value; + auto read_size = EncodingUtil::DecodeLEB128(ptr, value); + ptr += read_size; + return value; + } //===--------------------------------------------------------------------===// // Nested Types Hooks //===--------------------------------------------------------------------===// + void OnPropertyBegin(const field_id_t field_id, const char *tag) final; + void OnPropertyEnd() final; + bool OnOptionalPropertyBegin(const field_id_t field_id, const char *tag) final; + void OnOptionalPropertyEnd(bool present) final; void OnObjectBegin() final; void OnObjectEnd() final; idx_t OnListBegin() final; void OnListEnd() final; - idx_t OnMapBegin() final; - void OnMapEnd() final; - void OnMapEntryBegin() final; - void OnMapEntryEnd() final; - void OnMapKeyBegin() final; - void OnMapValueBegin() final; - bool OnOptionalBegin() final; - - void OnPairBegin() final; - void OnPairKeyBegin() final; - void OnPairValueBegin() final; - void OnPairEnd() final; + bool OnNullableBegin() final; + void OnNullableEnd() final; //===--------------------------------------------------------------------===// // Primitive Types //===--------------------------------------------------------------------===// bool ReadBool() final; + char ReadChar() final; int8_t ReadSignedInt8() final; uint8_t ReadUnsignedInt8() final; int16_t ReadSignedInt16() final; @@ -113,7 +127,6 @@ class BinaryDeserializer : public FormatDeserializer { float ReadFloat() final; double ReadDouble() final; string ReadString() final; - interval_t ReadInterval() final; hugeint_t ReadHugeInt() final; void ReadDataPtr(data_ptr_t &ptr, idx_t count) final; }; diff --git a/src/duckdb/src/include/duckdb/common/serializer/binary_serializer.hpp b/src/duckdb/src/include/duckdb/common/serializer/binary_serializer.hpp index 17dc61a6a..2598c6572 100644 --- a/src/duckdb/src/include/duckdb/common/serializer/binary_serializer.hpp +++ b/src/duckdb/src/include/duckdb/common/serializer/binary_serializer.hpp @@ -9,25 +9,21 @@ #pragma once #include "duckdb/common/serializer/format_serializer.hpp" +#include "duckdb/common/pair.hpp" +#include "duckdb/common/serializer/encoding_util.hpp" namespace duckdb { struct BinarySerializer : public FormatSerializer { private: - struct State { - // how many fields are present in the object - uint32_t field_count; - // the size of the object - uint64_t size; - // the offset of the object start in the buffer - uint64_t offset; + struct DebugState { + unordered_set seen_field_tags; + unordered_set seen_field_ids; + vector> seen_fields; }; - const char *current_tag; - field_id_t current_field_id = 0; - + vector debug_stack; vector data; - vector stack; template void Write(T element) { @@ -36,51 +32,57 @@ struct BinarySerializer : public FormatSerializer { } void WriteDataInternal(const_data_ptr_t buffer, idx_t write_size) { data.insert(data.end(), buffer, buffer + write_size); - stack.back().size += write_size; } void WriteDataInternal(const char *ptr, idx_t write_size) { WriteDataInternal(const_data_ptr_cast(ptr), write_size); } - explicit BinarySerializer() { + template + void VarIntEncode(T value) { + uint8_t buffer[16]; + auto write_size = EncodingUtil::EncodeLEB128(buffer, value); + D_ASSERT(write_size <= sizeof(buffer)); + WriteDataInternal(buffer, write_size); + } + + explicit BinarySerializer(bool serialize_default_values_p) { + serialize_default_values = serialize_default_values_p; serialize_enum_as_string = false; } public: + //! Serializes the given object into a binary blob, optionally serializing default values if + //! serialize_default_values is set to true, otherwise properties set to their provided default value + //! will not be serialized template - static vector Serialize(T &obj) { - BinarySerializer serializer; + static vector Serialize(T &obj, bool serialize_default_values) { + BinarySerializer serializer(serialize_default_values); serializer.OnObjectBegin(); obj.FormatSerialize(serializer); serializer.OnObjectEnd(); return std::move(serializer.data); } - void SetTag(const field_id_t field_id, const char *tag) final; - - //===--------------------------------------------------------------------===// - // Nested Types Hooks - //===--------------------------------------------------------------------===// - void OnOptionalBegin(bool present) final; + //------------------------------------------------------------------------- + // Nested Type Hooks + //------------------------------------------------------------------------- + // We serialize optional values as a message with a "present" flag, followed by the value. + void OnPropertyBegin(const field_id_t field_id, const char *tag) final; + void OnPropertyEnd() final; + void OnOptionalPropertyBegin(const field_id_t field_id, const char *tag, bool present) final; + void OnOptionalPropertyEnd(bool present) final; void OnListBegin(idx_t count) final; - void OnListEnd(idx_t count) final; - void OnMapBegin(idx_t count) final; - void OnMapEntryBegin() final; - void OnMapEntryEnd() final; - void OnMapKeyBegin() final; - void OnMapValueBegin() final; - void OnMapEnd(idx_t count) final; + void OnListEnd() final; void OnObjectBegin() final; void OnObjectEnd() final; - void OnPairBegin() final; - void OnPairKeyBegin() final; - void OnPairValueBegin() final; - void OnPairEnd() final; + void OnNullableBegin(bool present) final; + void OnNullableEnd() final; - //===--------------------------------------------------------------------===// + //------------------------------------------------------------------------- // Primitive Types - //===--------------------------------------------------------------------===// + //------------------------------------------------------------------------- void WriteNull() final; + void WriteValue(char value) final; void WriteValue(uint8_t value) final; void WriteValue(int8_t value) final; void WriteValue(uint16_t value) final; @@ -92,7 +94,6 @@ struct BinarySerializer : public FormatSerializer { void WriteValue(hugeint_t value) final; void WriteValue(float value) final; void WriteValue(double value) final; - void WriteValue(interval_t value) final; void WriteValue(const string_t value) final; void WriteValue(const string &value) final; void WriteValue(const char *value) final; diff --git a/src/duckdb/src/include/duckdb/common/serializer/deserialization_data.hpp b/src/duckdb/src/include/duckdb/common/serializer/deserialization_data.hpp index 0f4629689..af7499241 100644 --- a/src/duckdb/src/include/duckdb/common/serializer/deserialization_data.hpp +++ b/src/duckdb/src/include/duckdb/common/serializer/deserialization_data.hpp @@ -21,6 +21,7 @@ struct DeserializationData { stack> contexts; stack enums; stack> parameter_data; + stack> types; template void Set(T entry) = delete; @@ -107,4 +108,21 @@ inline void DeserializationData::Unset() { parameter_data.pop(); } +template <> +inline void DeserializationData::Set(LogicalType &type) { + types.emplace(type); +} + +template <> +inline LogicalType &DeserializationData::Get() { + AssertNotEmpty(types); + return types.top(); +} + +template <> +inline void DeserializationData::Unset() { + AssertNotEmpty(types); + types.pop(); +} + } // namespace duckdb diff --git a/src/duckdb/src/include/duckdb/common/serializer/encoding_util.hpp b/src/duckdb/src/include/duckdb/common/serializer/encoding_util.hpp new file mode 100644 index 000000000..f30cf5790 --- /dev/null +++ b/src/duckdb/src/include/duckdb/common/serializer/encoding_util.hpp @@ -0,0 +1,132 @@ +//===----------------------------------------------------------------------===// +// DuckDB +// +// duckdb/common/serializer/encoding_util.hpp +// +// +//===----------------------------------------------------------------------===// + +#pragma once + +#include "duckdb/common/typedefs.hpp" +#include + +namespace duckdb { + +struct EncodingUtil { + + // Encode unsigned integer, returns the number of bytes written + template + static idx_t EncodeUnsignedLEB128(data_ptr_t target, T value) { + static_assert(std::is_integral::value, "Must be integral"); + static_assert(std::is_unsigned::value, "Must be unsigned"); + static_assert(sizeof(T) <= sizeof(uint64_t), "Must be uint64_t or smaller"); + + idx_t offset = 0; + do { + uint8_t byte = value & 0x7F; + value >>= 7; + if (value != 0) { + byte |= 0x80; + } + target[offset++] = byte; + } while (value != 0); + return offset; + } + + // Decode unsigned integer, returns the number of bytes read + template + static idx_t DecodeUnsignedLEB128(const_data_ptr_t source, T &result) { + static_assert(std::is_integral::value, "Must be integral"); + static_assert(std::is_unsigned::value, "Must be unsigned"); + static_assert(sizeof(T) <= sizeof(uint64_t), "Must be uint64_t or smaller"); + + result = 0; + idx_t shift = 0; + idx_t offset = 0; + uint8_t byte; + do { + byte = source[offset++]; + result |= static_cast(byte & 0x7F) << shift; + shift += 7; + } while (byte & 0x80); + + return offset; + } + + // Encode signed integer, returns the number of bytes written + template + static idx_t EncodeSignedLEB128(data_ptr_t target, T value) { + static_assert(std::is_integral::value, "Must be integral"); + static_assert(std::is_signed::value, "Must be signed"); + static_assert(sizeof(T) <= sizeof(int64_t), "Must be int64_t or smaller"); + + idx_t offset = 0; + do { + uint8_t byte = value & 0x7F; + value >>= 7; + + // Determine whether more bytes are needed + if ((value == 0 && (byte & 0x40) == 0) || (value == -1 && (byte & 0x40))) { + target[offset++] = byte; + break; + } else { + byte |= 0x80; + target[offset++] = byte; + } + } while (true); + return offset; + } + + // Decode signed integer, returns the number of bytes read + template + static idx_t DecodeSignedLEB128(const_data_ptr_t source, T &result) { + static_assert(std::is_integral::value, "Must be integral"); + static_assert(std::is_signed::value, "Must be signed"); + static_assert(sizeof(T) <= sizeof(int64_t), "Must be int64_t or smaller"); + + // This is used to avoid undefined behavior when shifting into the sign bit + using unsigned_type = typename std::make_unsigned::type; + + result = 0; + idx_t shift = 0; + idx_t offset = 0; + + uint8_t byte; + do { + byte = source[offset++]; + result |= static_cast(byte & 0x7F) << shift; + shift += 7; + } while (byte & 0x80); + + // Sign-extend if the most significant bit of the last byte is set + if (shift < sizeof(T) * 8 && (byte & 0x40)) { + result |= -(static_cast(1) << shift); + } + return offset; + } + + template + static typename std::enable_if::value, idx_t>::type DecodeLEB128(const_data_ptr_t source, + T &result) { + return DecodeSignedLEB128(source, result); + } + + template + static typename std::enable_if::value, idx_t>::type DecodeLEB128(const_data_ptr_t source, + T &result) { + return DecodeUnsignedLEB128(source, result); + } + + template + static typename std::enable_if::value, idx_t>::type EncodeLEB128(data_ptr_t target, T value) { + return EncodeSignedLEB128(target, value); + } + + template + static typename std::enable_if::value, idx_t>::type EncodeLEB128(data_ptr_t target, T value) { + return EncodeUnsignedLEB128(target, value); + } +}; + +} // namespace duckdb diff --git a/src/duckdb/src/include/duckdb/common/serializer/format_deserializer.hpp b/src/duckdb/src/include/duckdb/common/serializer/format_deserializer.hpp index f0e6face3..1bad83418 100644 --- a/src/duckdb/src/include/duckdb/common/serializer/format_deserializer.hpp +++ b/src/duckdb/src/include/duckdb/common/serializer/format_deserializer.hpp @@ -13,7 +13,6 @@ #include "duckdb/common/enum_util.hpp" #include "duckdb/common/serializer/serialization_traits.hpp" #include "duckdb/common/serializer/deserialization_data.hpp" -#include "duckdb/common/types/interval.hpp" #include "duckdb/common/types/string_type.hpp" #include "duckdb/common/unordered_map.hpp" #include "duckdb/common/unordered_set.hpp" @@ -21,92 +20,78 @@ namespace duckdb { class FormatDeserializer { - friend Vector; - protected: bool deserialize_enum_from_string = false; DeserializationData data; +public: + class List { + friend FormatDeserializer; + + private: + FormatDeserializer &deserializer; + explicit List(FormatDeserializer &deserializer) : deserializer(deserializer) { + } + + public: + // Deserialize an element + template + T ReadElement(); + + // Deserialize an object + template + void ReadObject(FUNC f); + }; + public: // Read into an existing value template inline void ReadProperty(const field_id_t field_id, const char *tag, T &ret) { - SetTag(field_id, tag); + OnPropertyBegin(field_id, tag); ret = Read(); + OnPropertyEnd(); } // Read and return a value template inline T ReadProperty(const field_id_t field_id, const char *tag) { - SetTag(field_id, tag); - return Read(); + OnPropertyBegin(field_id, tag); + auto ret = Read(); + OnPropertyEnd(); + ; + return ret; } - // Read optional property and return a value, or forward a default value + // Default Value return template - inline T ReadOptionalPropertyOrDefault(const field_id_t field_id, const char *tag, T &&default_value) { - SetTag(field_id, tag); - auto present = OnOptionalBegin(); - if (present) { - auto item = Read(); - OnOptionalEnd(); - return item; - } else { - OnOptionalEnd(); + inline T ReadPropertyWithDefault(const field_id_t field_id, const char *tag, T &&default_value) { + if (!OnOptionalPropertyBegin(field_id, tag)) { + OnOptionalPropertyEnd(false); return std::forward(default_value); } + auto ret = Read(); + OnOptionalPropertyEnd(true); + return ret; } - // Read optional property into an existing value, or use a default value + // Default value in place template - inline void ReadOptionalPropertyOrDefault(const field_id_t field_id, const char *tag, T &ret, T &&default_value) { - SetTag(field_id, tag); - auto present = OnOptionalBegin(); - if (present) { - ret = Read(); - OnOptionalEnd(); - } else { + inline void ReadPropertyWithDefault(const field_id_t field_id, const char *tag, T &ret, T &&default_value) { + if (!OnOptionalPropertyBegin(field_id, tag)) { ret = std::forward(default_value); - OnOptionalEnd(); - } - } - - // Read optional property and return a value, or default construct it - template - inline typename std::enable_if::value, T>::type - ReadOptionalProperty(const field_id_t field_id, const char *tag) { - SetTag(field_id, tag); - auto present = OnOptionalBegin(); - if (present) { - auto item = Read(); - OnOptionalEnd(); - return item; - } else { - OnOptionalEnd(); - return T(); - } - } - - // Read optional property into an existing value, or default construct it - template - inline typename std::enable_if::value, void>::type - ReadOptionalProperty(const field_id_t field_id, const char *tag, T &ret) { - SetTag(field_id, tag); - auto present = OnOptionalBegin(); - if (present) { - ret = Read(); - OnOptionalEnd(); - } else { - ret = T(); - OnOptionalEnd(); + OnOptionalPropertyEnd(false); + return; } + ret = Read(); + OnOptionalPropertyEnd(true); } // Special case: // Read into an existing data_ptr_t inline void ReadProperty(const field_id_t field_id, const char *tag, data_ptr_t ret, idx_t count) { - SetTag(field_id, tag); + OnPropertyBegin(field_id, tag); ReadDataPtr(ret, count); + OnPropertyEnd(); } //! Set a serialization property @@ -127,14 +112,25 @@ class FormatDeserializer { return data.Unset(); } - // Manually begin an object - should be followed by EndObject - void BeginObject(const field_id_t field_id, const char *tag) { - SetTag(field_id, tag); - OnObjectBegin(); + template + void ReadList(const field_id_t field_id, const char *tag, FUNC func) { + OnPropertyBegin(field_id, tag); + auto size = OnListBegin(); + List list {*this}; + for (idx_t i = 0; i < size; i++) { + func(list, i); + } + OnListEnd(); + OnPropertyEnd(); } - void EndObject() { + template + void ReadObject(const field_id_t field_id, const char *tag, FUNC func) { + OnPropertyBegin(field_id, tag); + OnObjectBegin(); + func(*this); OnObjectEnd(); + OnPropertyEnd(); } private: @@ -147,25 +143,33 @@ class FormatDeserializer { return val; } - // Structural Types - // Deserialize a unique_ptr template inline typename std::enable_if::value, T>::type Read() { using ELEMENT_TYPE = typename is_unique_ptr::ELEMENT_TYPE; - OnObjectBegin(); - auto val = ELEMENT_TYPE::FormatDeserialize(*this); - OnObjectEnd(); - return val; + unique_ptr ptr = nullptr; + auto is_present = OnNullableBegin(); + if (is_present) { + OnObjectBegin(); + ptr = ELEMENT_TYPE::FormatDeserialize(*this); + OnObjectEnd(); + } + OnNullableEnd(); + return ptr; } // Deserialize shared_ptr template inline typename std::enable_if::value, T>::type Read() { using ELEMENT_TYPE = typename is_shared_ptr::ELEMENT_TYPE; - OnObjectBegin(); - auto val = ELEMENT_TYPE::FormatDeserialize(*this); - OnObjectEnd(); - return val; + shared_ptr ptr = nullptr; + auto is_present = OnNullableBegin(); + if (is_present) { + OnObjectBegin(); + ptr = ELEMENT_TYPE::FormatDeserialize(*this); + OnObjectEnd(); + } + OnNullableEnd(); + return ptr; } // Deserialize a vector @@ -178,7 +182,6 @@ class FormatDeserializer { vec.push_back(Read()); } OnListEnd(); - return vec; } @@ -202,19 +205,15 @@ class FormatDeserializer { using VALUE_TYPE = typename is_unordered_map::VALUE_TYPE; T map; - auto size = OnMapBegin(); + auto size = OnListBegin(); for (idx_t i = 0; i < size; i++) { - OnMapEntryBegin(); - OnMapKeyBegin(); - auto key = Read(); - OnMapKeyEnd(); - OnMapValueBegin(); - auto value = Read(); - OnMapValueEnd(); - OnMapEntryEnd(); + OnObjectBegin(); + auto key = ReadProperty(0, "key"); + auto value = ReadProperty(1, "value"); + OnObjectEnd(); map[std::move(key)] = std::move(value); } - OnMapEnd(); + OnListEnd(); return map; } @@ -224,19 +223,15 @@ class FormatDeserializer { using VALUE_TYPE = typename is_map::VALUE_TYPE; T map; - auto size = OnMapBegin(); + auto size = OnListBegin(); for (idx_t i = 0; i < size; i++) { - OnMapEntryBegin(); - OnMapKeyBegin(); - auto key = Read(); - OnMapKeyEnd(); - OnMapValueBegin(); - auto value = Read(); - OnMapValueEnd(); - OnMapEntryEnd(); + OnObjectBegin(); + auto key = ReadProperty(0, "key"); + auto value = ReadProperty(1, "value"); + OnObjectEnd(); map[std::move(key)] = std::move(value); } - OnMapEnd(); + OnListEnd(); return map; } @@ -271,15 +266,10 @@ class FormatDeserializer { inline typename std::enable_if::value, T>::type Read() { using FIRST_TYPE = typename is_pair::FIRST_TYPE; using SECOND_TYPE = typename is_pair::SECOND_TYPE; - - OnPairBegin(); - OnPairKeyBegin(); - FIRST_TYPE first = Read(); - OnPairKeyEnd(); - OnPairValueBegin(); - SECOND_TYPE second = Read(); - OnPairValueEnd(); - OnPairEnd(); + OnObjectBegin(); + auto first = ReadProperty(0, "first"); + auto second = ReadProperty(1, "second"); + OnObjectEnd(); return std::make_pair(first, second); } @@ -290,6 +280,12 @@ class FormatDeserializer { return ReadBool(); } + // Deserialize a char + template + inline typename std::enable_if::value, T>::type Read() { + return ReadChar(); + } + // Deserialize a int8_t template inline typename std::enable_if::value, T>::type Read() { @@ -367,12 +363,6 @@ class FormatDeserializer { } } - // Deserialize a interval_t - template - inline typename std::enable_if::value, T>::type Read() { - return ReadInterval(); - } - // Deserialize a hugeint_t template inline typename std::enable_if::value, T>::type Read() { @@ -392,50 +382,24 @@ class FormatDeserializer { } protected: - virtual void SetTag(const field_id_t field_id, const char *tag) { - (void)field_id; - (void)tag; - } - + // Hooks for subclasses to override to implement custom behavior + virtual void OnPropertyBegin(const field_id_t field_id, const char *tag) = 0; + virtual void OnPropertyEnd() = 0; + virtual bool OnOptionalPropertyBegin(const field_id_t field_id, const char *tag) = 0; + virtual void OnOptionalPropertyEnd(bool present) = 0; + + virtual void OnObjectBegin() = 0; + virtual void OnObjectEnd() = 0; virtual idx_t OnListBegin() = 0; - virtual void OnListEnd() { - } - virtual idx_t OnMapBegin() = 0; - virtual void OnMapEnd() { - } - virtual void OnMapEntryBegin() { - } - virtual void OnMapEntryEnd() { - } - virtual void OnMapKeyBegin() { - } - virtual void OnMapKeyEnd() { - } - virtual void OnMapValueBegin() { - } - virtual void OnMapValueEnd() { - } - virtual bool OnOptionalBegin() = 0; - virtual void OnOptionalEnd() { - } - virtual void OnObjectBegin() { - } - virtual void OnObjectEnd() { - } - virtual void OnPairBegin() { - } - virtual void OnPairKeyBegin() { - } - virtual void OnPairKeyEnd() { - } - virtual void OnPairValueBegin() { - } - virtual void OnPairValueEnd() { - } - virtual void OnPairEnd() { - } + virtual void OnListEnd() = 0; + virtual bool OnNullableBegin() = 0; + virtual void OnNullableEnd() = 0; + // Handle primitive types, a serializer needs to implement these. virtual bool ReadBool() = 0; + virtual char ReadChar() { + throw NotImplementedException("ReadChar not implemented"); + } virtual int8_t ReadSignedInt8() = 0; virtual uint8_t ReadUnsignedInt8() = 0; virtual int16_t ReadSignedInt16() = 0; @@ -448,8 +412,19 @@ class FormatDeserializer { virtual float ReadFloat() = 0; virtual double ReadDouble() = 0; virtual string ReadString() = 0; - virtual interval_t ReadInterval() = 0; virtual void ReadDataPtr(data_ptr_t &ptr, idx_t count) = 0; }; +template +void FormatDeserializer::List::ReadObject(FUNC f) { + deserializer.OnObjectBegin(); + f(deserializer); + deserializer.OnObjectEnd(); +} + +template +T FormatDeserializer::List::ReadElement() { + return deserializer.Read(); +} + } // namespace duckdb diff --git a/src/duckdb/src/include/duckdb/common/serializer/format_serializer.hpp b/src/duckdb/src/include/duckdb/common/serializer/format_serializer.hpp index 88b24b72b..59b68ae98 100644 --- a/src/duckdb/src/include/duckdb/common/serializer/format_serializer.hpp +++ b/src/duckdb/src/include/duckdb/common/serializer/format_serializer.hpp @@ -8,9 +8,9 @@ #pragma once +#include "duckdb/common/enum_util.hpp" #include "duckdb/common/field_writer.hpp" #include "duckdb/common/serializer.hpp" -#include "duckdb/common/enum_util.hpp" #include "duckdb/common/serializer/serialization_traits.hpp" #include "duckdb/common/types/interval.hpp" #include "duckdb/common/types/string_type.hpp" @@ -20,47 +20,79 @@ namespace duckdb { class FormatSerializer { - friend Vector; - protected: bool serialize_enum_as_string = false; + bool serialize_default_values = false; + +public: + class List { + friend FormatSerializer; + + private: + FormatSerializer &serializer; + explicit List(FormatSerializer &serializer) : serializer(serializer) { + } + + public: + // Serialize an element + template + void WriteElement(const T &value); + + // Serialize an object + template + void WriteObject(FUNC f); + }; public: // Serialize a value template void WriteProperty(const field_id_t field_id, const char *tag, const T &value) { - SetTag(field_id, tag); + OnPropertyBegin(field_id, tag); WriteValue(value); + OnPropertyEnd(); } - // Optional pointer - template - void WriteOptionalProperty(const field_id_t field_id, const char *tag, POINTER &&ptr) { - SetTag(field_id, tag); - if (ptr == nullptr) { - OnOptionalBegin(false); - OnOptionalEnd(false); - } else { - OnOptionalBegin(true); - WriteValue(*ptr); - OnOptionalEnd(true); + // Default value + template + void WritePropertyWithDefault(const field_id_t field_id, const char *tag, const T &value, const T &&default_value) { + // If current value is default, don't write it + if (!serialize_default_values && (value == default_value)) { + OnOptionalPropertyBegin(field_id, tag, false); + OnOptionalPropertyEnd(false); + return; } + OnOptionalPropertyBegin(field_id, tag, true); + WriteValue(value); + OnOptionalPropertyEnd(true); } // Special case: data_ptr_T void WriteProperty(const field_id_t field_id, const char *tag, const_data_ptr_t ptr, idx_t count) { - SetTag(field_id, tag); + OnPropertyBegin(field_id, tag); WriteDataPtr(ptr, count); + OnPropertyEnd(); } - // Manually begin an object - should be followed by EndObject - void BeginObject(const field_id_t field_id, const char *tag) { - SetTag(field_id, tag); + // Manually begin an object + template + void WriteObject(const field_id_t field_id, const char *tag, FUNC f) { + OnPropertyBegin(field_id, tag); OnObjectBegin(); + f(*this); + OnObjectEnd(); + OnPropertyEnd(); } - void EndObject() { - OnObjectEnd(); + template + void WriteList(const field_id_t field_id, const char *tag, idx_t count, FUNC func) { + OnPropertyBegin(field_id, tag); + OnListBegin(count); + List list {*this}; + for (idx_t i = 0; i < count; i++) { + func(list, i); + } + OnListEnd(); + OnPropertyEnd(); } protected: @@ -82,27 +114,38 @@ class FormatSerializer { WriteValue(ptr.get()); } + // Shared Pointer Ref + template + void WriteValue(const shared_ptr &ptr) { + WriteValue(ptr.get()); + } + // Pointer template - typename std::enable_if::value, void>::type WriteValue(const T ptr) { + void WriteValue(const T *ptr) { if (ptr == nullptr) { - WriteNull(); + OnNullableBegin(false); + OnNullableEnd(); } else { + OnNullableBegin(true); WriteValue(*ptr); + OnNullableEnd(); } } // Pair template void WriteValue(const std::pair &pair) { - OnPairBegin(); - OnPairKeyBegin(); - WriteValue(pair.first); - OnPairKeyEnd(); - OnPairValueBegin(); - WriteValue(pair.second); - OnPairValueEnd(); - OnPairEnd(); + OnObjectBegin(); + WriteProperty(0, "first", pair.first); + WriteProperty(1, "second", pair.second); + OnObjectEnd(); + } + + // Reference Wrapper + template + void WriteValue(const reference ref) { + WriteValue(ref.get()); } // Vector @@ -113,7 +156,7 @@ class FormatSerializer { for (auto &item : vec) { WriteValue(item); } - OnListEnd(count); + OnListEnd(); } template @@ -123,7 +166,7 @@ class FormatSerializer { for (auto &item : vec) { WriteValue(item); } - OnListEnd(count); + OnListEnd(); } // UnorderedSet @@ -135,7 +178,7 @@ class FormatSerializer { for (auto &item : set) { WriteValue(item); } - OnListEnd(count); + OnListEnd(); } // Set @@ -147,108 +190,65 @@ class FormatSerializer { for (auto &item : set) { WriteValue(item); } - OnListEnd(count); + OnListEnd(); } // Map + // serialized as a list of pairs template void WriteValue(const duckdb::unordered_map &map) { auto count = map.size(); - OnMapBegin(count); + OnListBegin(count); for (auto &item : map) { - OnMapEntryBegin(); - OnMapKeyBegin(); - WriteValue(item.first); - OnMapKeyEnd(); - OnMapValueBegin(); - WriteValue(item.second); - OnMapValueEnd(); - OnMapEntryEnd(); + OnObjectBegin(); + WriteProperty(0, "key", item.first); + WriteProperty(1, "value", item.second); + OnObjectEnd(); } - OnMapEnd(count); + OnListEnd(); } // Map + // serialized as a list of pairs template void WriteValue(const duckdb::map &map) { auto count = map.size(); - OnMapBegin(count); + OnListBegin(count); for (auto &item : map) { - OnMapEntryBegin(); - OnMapKeyBegin(); - WriteValue(item.first); - OnMapKeyEnd(); - OnMapValueBegin(); - WriteValue(item.second); - OnMapValueEnd(); - OnMapEntryEnd(); + OnObjectBegin(); + WriteProperty(0, "key", item.first); + WriteProperty(1, "value", item.second); + OnObjectEnd(); } - OnMapEnd(count); + OnListEnd(); } // class or struct implementing `FormatSerialize(FormatSerializer& FormatSerializer)`; template typename std::enable_if::value>::type WriteValue(const T &value) { - // Else, we defer to the .FormatSerialize method OnObjectBegin(); value.FormatSerialize(*this); OnObjectEnd(); } - // Handle setting a "tag" (optional) - virtual void SetTag(const field_id_t field_id, const char *tag) { - (void)field_id; - (void)tag; - } - +protected: // Hooks for subclasses to override to implement custom behavior - virtual void OnListBegin(idx_t count) { - (void)count; - } - virtual void OnListEnd(idx_t count) { - (void)count; - } - virtual void OnMapBegin(idx_t count) { - (void)count; - } - virtual void OnMapEnd(idx_t count) { - (void)count; - } - virtual void OnMapEntryBegin() { - } - virtual void OnMapEntryEnd() { - } - virtual void OnMapKeyBegin() { - } - virtual void OnMapKeyEnd() { - } - virtual void OnMapValueBegin() { - } - virtual void OnMapValueEnd() { - } - virtual void OnOptionalBegin(bool present) { - } - virtual void OnOptionalEnd(bool present) { - } - virtual void OnObjectBegin() { - } - virtual void OnObjectEnd() { - } - virtual void OnPairBegin() { - } - virtual void OnPairKeyBegin() { - } - virtual void OnPairKeyEnd() { - } - virtual void OnPairValueBegin() { - } - virtual void OnPairValueEnd() { - } - virtual void OnPairEnd() { - } + virtual void OnPropertyBegin(const field_id_t field_id, const char *tag) = 0; + virtual void OnPropertyEnd() = 0; + virtual void OnOptionalPropertyBegin(const field_id_t field_id, const char *tag, bool present) = 0; + virtual void OnOptionalPropertyEnd(bool present) = 0; + virtual void OnObjectBegin() = 0; + virtual void OnObjectEnd() = 0; + virtual void OnListBegin(idx_t count) = 0; + virtual void OnListEnd() = 0; + virtual void OnNullableBegin(bool present) = 0; + virtual void OnNullableEnd() = 0; // Handle primitive types, a serializer needs to implement these. virtual void WriteNull() = 0; + virtual void WriteValue(char value) { + throw NotImplementedException("Write char value not implemented"); + } virtual void WriteValue(bool value) = 0; virtual void WriteValue(uint8_t value) = 0; virtual void WriteValue(int8_t value) = 0; @@ -264,7 +264,6 @@ class FormatSerializer { virtual void WriteValue(const string_t value) = 0; virtual void WriteValue(const string &value) = 0; virtual void WriteValue(const char *str) = 0; - virtual void WriteValue(interval_t value) = 0; virtual void WriteDataPtr(const_data_ptr_t ptr, idx_t count) = 0; void WriteValue(LogicalIndex value) { WriteValue(value.index); @@ -278,4 +277,17 @@ class FormatSerializer { template <> void FormatSerializer::WriteValue(const vector &vec); +// List Impl +template +void FormatSerializer::List::WriteObject(FUNC f) { + serializer.OnObjectBegin(); + f(serializer); + serializer.OnObjectEnd(); +} + +template +void FormatSerializer::List::WriteElement(const T &value) { + serializer.WriteValue(value); +} + } // namespace duckdb diff --git a/src/duckdb/src/include/duckdb/common/serializer/serialization_traits.hpp b/src/duckdb/src/include/duckdb/common/serializer/serialization_traits.hpp index 588b8fc23..31b2d7de7 100644 --- a/src/duckdb/src/include/duckdb/common/serializer/serialization_traits.hpp +++ b/src/duckdb/src/include/duckdb/common/serializer/serialization_traits.hpp @@ -12,7 +12,8 @@ namespace duckdb { class FormatSerializer; // Forward declare class FormatDeserializer; // Forward declare -typedef uint32_t field_id_t; +typedef uint16_t field_id_t; +const field_id_t MESSAGE_TERMINATOR_FIELD_ID = 0xFFFF; // Backport to c++11 template diff --git a/src/duckdb/src/include/duckdb/common/shared_ptr.hpp b/src/duckdb/src/include/duckdb/common/shared_ptr.hpp index f849e0700..4d97075eb 100644 --- a/src/duckdb/src/include/duckdb/common/shared_ptr.hpp +++ b/src/duckdb/src/include/duckdb/common/shared_ptr.hpp @@ -1,3 +1,11 @@ +//===----------------------------------------------------------------------===// +// DuckDB +// +// duckdb/common/shared_ptr.hpp +// +// +//===----------------------------------------------------------------------===// + #pragma once #include diff --git a/src/duckdb/src/include/duckdb/common/sort/partition_state.hpp b/src/duckdb/src/include/duckdb/common/sort/partition_state.hpp index 21d953a32..5c7be445e 100644 --- a/src/duckdb/src/include/duckdb/common/sort/partition_state.hpp +++ b/src/duckdb/src/include/duckdb/common/sort/partition_state.hpp @@ -31,6 +31,7 @@ class PartitionGlobalHashGroup { GlobalSortStatePtr global_sort; atomic count; + idx_t batch_base; // Mask computation SortLayout partition_layout; @@ -59,9 +60,6 @@ class PartitionGlobalSinkState { void UpdateLocalPartition(GroupingPartition &local_partition, GroupingAppend &local_append); void CombineLocalPartition(GroupingPartition &local_partition, GroupingAppend &local_append); - void BuildSortState(TupleDataCollection &group_data, GlobalSortState &global_sort) const; - void BuildSortState(TupleDataCollection &group_data, PartitionGlobalHashGroup &global_sort); - ClientContext &context; BufferManager &buffer_manager; Allocator &allocator; @@ -128,7 +126,7 @@ class PartitionLocalSinkState { void Combine(); }; -enum class PartitionSortStage : uint8_t { INIT, PREPARE, MERGE, SORTED }; +enum class PartitionSortStage : uint8_t { INIT, SCAN, PREPARE, MERGE, SORTED }; class PartitionLocalMergeState; @@ -150,7 +148,11 @@ class PartitionGlobalMergeState { PartitionGlobalSinkState &sink; GroupDataPtr group_data; PartitionGlobalHashGroup *hash_group; + vector column_ids; + TupleDataParallelScanState chunk_state; GlobalSortState *global_sort; + const idx_t memory_per_thread; + const idx_t num_threads; private: mutable mutex lock; @@ -162,15 +164,14 @@ class PartitionGlobalMergeState { class PartitionLocalMergeState { public: - PartitionLocalMergeState() : merge_state(nullptr), stage(PartitionSortStage::INIT) { - finished = true; - } + explicit PartitionLocalMergeState(PartitionGlobalSinkState &gstate); bool TaskFinished() { return finished; } void Prepare(); + void Scan(); void Merge(); void ExecuteTask(); @@ -178,6 +179,11 @@ class PartitionLocalMergeState { PartitionGlobalMergeState *merge_state; PartitionSortStage stage; atomic finished; + + // Sorting buffers + ExpressionExecutor executor; + DataChunk sort_chunk; + DataChunk payload_chunk; }; class PartitionGlobalMergeStates { diff --git a/src/duckdb/src/include/duckdb/common/types/data_chunk.hpp b/src/duckdb/src/include/duckdb/common/types/data_chunk.hpp index 71b00b17d..5bd07aee9 100644 --- a/src/duckdb/src/include/duckdb/common/types/data_chunk.hpp +++ b/src/duckdb/src/include/duckdb/common/types/data_chunk.hpp @@ -19,6 +19,8 @@ class Allocator; class ClientContext; class ExecutionContext; class VectorCache; +class FormatSerializer; +class FormatDeserializer; //! A Data Chunk represents a set of vectors. /*! @@ -141,6 +143,9 @@ class DataChunk { //! Deserializes a blob back into a DataChunk DUCKDB_API void Deserialize(Deserializer &source); + DUCKDB_API void FormatSerialize(FormatSerializer &serializer) const; + DUCKDB_API void FormatDeserialize(FormatDeserializer &source); + //! Hashes the DataChunk to the target vector DUCKDB_API void Hash(Vector &result); //! Hashes specific vectors of the DataChunk to the target vector diff --git a/src/duckdb/src/include/duckdb/common/types/hyperloglog.hpp b/src/duckdb/src/include/duckdb/common/types/hyperloglog.hpp index 2a84bbf14..eb61a0899 100644 --- a/src/duckdb/src/include/duckdb/common/types/hyperloglog.hpp +++ b/src/duckdb/src/include/duckdb/common/types/hyperloglog.hpp @@ -18,11 +18,14 @@ struct robj; namespace duckdb { -enum class HLLStorageType { UNCOMPRESSED = 1 }; +enum class HLLStorageType : uint8_t { UNCOMPRESSED = 1 }; class FieldWriter; class FieldReader; +class FormatSerializer; +class FormatDeserializer; + //! The HyperLogLog class holds a HyperLogLog counter for approximate cardinality counting class HyperLogLog { public: @@ -50,6 +53,9 @@ class HyperLogLog { void Serialize(FieldWriter &writer) const; static unique_ptr Deserialize(FieldReader &reader); + void FormatSerialize(FormatSerializer &serializer) const; + static unique_ptr FormatDeserialize(FormatDeserializer &deserializer); + public: //! Compute HLL hashes over vdata, and store them in 'hashes' //! Then, compute register indices and prefix lengths, and also store them in 'hashes' as a pair of uint32_t diff --git a/src/duckdb/src/include/duckdb/common/types/interval.hpp b/src/duckdb/src/include/duckdb/common/types/interval.hpp index 2b1966f3b..590554557 100644 --- a/src/duckdb/src/include/duckdb/common/types/interval.hpp +++ b/src/duckdb/src/include/duckdb/common/types/interval.hpp @@ -16,6 +16,9 @@ struct dtime_t; struct date_t; struct timestamp_t; +class FormatSerializer; +class FormatDeserializer; + struct interval_t { int32_t months; int32_t days; @@ -24,6 +27,10 @@ struct interval_t { inline bool operator==(const interval_t &rhs) const { return this->days == rhs.days && this->months == rhs.months && this->micros == rhs.micros; } + + // Serialization + void FormatSerialize(FormatSerializer &serializer) const; + static interval_t FormatDeserialize(FormatDeserializer &source); }; //! The Interval class is a static class that holds helper functions for the Interval diff --git a/src/duckdb/src/include/duckdb/common/types/row/partitioned_tuple_data.hpp b/src/duckdb/src/include/duckdb/common/types/row/partitioned_tuple_data.hpp index 896fadfc2..f31187232 100644 --- a/src/duckdb/src/include/duckdb/common/types/row/partitioned_tuple_data.hpp +++ b/src/duckdb/src/include/duckdb/common/types/row/partitioned_tuple_data.hpp @@ -8,6 +8,7 @@ #pragma once +#include "duckdb/common/fixed_size_map.hpp" #include "duckdb/common/perfect_map_set.hpp" #include "duckdb/common/types/row/tuple_data_allocator.hpp" #include "duckdb/common/types/row/tuple_data_collection.hpp" @@ -23,10 +24,11 @@ struct PartitionedTupleDataAppendState { public: Vector partition_indices; SelectionVector partition_sel; + SelectionVector reverse_partition_sel; - static constexpr idx_t MAP_THRESHOLD = 32; + static constexpr idx_t MAP_THRESHOLD = 256; perfect_map_t partition_entries; - list_entry_t partition_entries_arr[MAP_THRESHOLD]; + fixed_size_map_t fixed_partition_entries; vector> partition_pin_states; TupleDataChunkState chunk_state; @@ -51,30 +53,48 @@ class PartitionedTupleData { virtual ~PartitionedTupleData(); public: + //! Get the layout of this PartitionedTupleData + const TupleDataLayout &GetLayout() const; //! Get the partitioning type of this PartitionedTupleData PartitionedTupleDataType GetType() const; //! Initializes a local state for parallel partitioning that can be merged into this PartitionedTupleData void InitializeAppendState(PartitionedTupleDataAppendState &state, TupleDataPinProperties properties = TupleDataPinProperties::UNPIN_AFTER_DONE) const; //! Appends a DataChunk to this PartitionedTupleData - void Append(PartitionedTupleDataAppendState &state, DataChunk &input); + void Append(PartitionedTupleDataAppendState &state, DataChunk &input, + const SelectionVector &append_sel = *FlatVector::IncrementalSelectionVector(), + const idx_t append_count = DConstants::INVALID_INDEX); + //! Appends a DataChunk to this PartitionedTupleData + //! - ToUnifiedFormat has already been called + void AppendUnified(PartitionedTupleDataAppendState &state, DataChunk &input, + const SelectionVector &append_sel = *FlatVector::IncrementalSelectionVector(), + const idx_t append_count = DConstants::INVALID_INDEX); //! Appends rows to this PartitionedTupleData - void Append(PartitionedTupleDataAppendState &state, TupleDataChunkState &input, idx_t count); + void Append(PartitionedTupleDataAppendState &state, TupleDataChunkState &input, const idx_t count); //! Flushes any remaining data in the append state into this PartitionedTupleData void FlushAppendState(PartitionedTupleDataAppendState &state); //! Combine another PartitionedTupleData into this PartitionedTupleData void Combine(PartitionedTupleData &other); - //! Partition a TupleDataCollection - void Partition(TupleDataCollection &source, - TupleDataPinProperties properties = TupleDataPinProperties::UNPIN_AFTER_DONE); + //! Resets this PartitionedTupleData + void Reset(); //! Repartition this PartitionedTupleData into the new PartitionedTupleData void Repartition(PartitionedTupleData &new_partitioned_data); + //! Unpins the data + void Unpin(); //! Get the partitions in this PartitionedTupleData vector> &GetPartitions(); + //! Get the data of this PartitionedTupleData as a single unpartitioned TupleDataCollection + unique_ptr GetUnpartitioned(); //! Get the count of this PartitionedTupleData idx_t Count() const; //! Get the size (in bytes) of this PartitionedTupleData idx_t SizeInBytes() const; + //! Get the number of partitions of this PartitionedTupleData + idx_t PartitionCount() const; + //! Converts this PartitionedTupleData to a string representation + string ToString(); + //! Prints the string representation of this PartitionedTupleData + void Print(); protected: //===--------------------------------------------------------------------===// @@ -91,7 +111,7 @@ class PartitionedTupleData { throw NotImplementedException("ComputePartitionIndices for this type of PartitionedTupleData"); } //! Compute partition indices from rows (similar to function above) - virtual void ComputePartitionIndices(Vector &row_locations, idx_t count, Vector &partition_indices) const { + virtual void ComputePartitionIndices(Vector &row_locations, idx_t append_count, Vector &partition_indices) const { throw NotImplementedException("ComputePartitionIndices for this type of PartitionedTupleData"); } //! Maximum partition index (optional) @@ -116,11 +136,19 @@ class PartitionedTupleData { //! Create a new shared allocator void CreateAllocator(); + //! Whether to use fixed size map or regular marp + bool UseFixedSizeMap() const; //! Builds a selection vector in the Append state for the partitions //! - returns true if everything belongs to the same partition - stores partition index in single_partition_idx - void BuildPartitionSel(PartitionedTupleDataAppendState &state, idx_t count); + void BuildPartitionSel(PartitionedTupleDataAppendState &state, const SelectionVector &append_sel, + const idx_t append_count); + template + void BuildPartitionSel(PartitionedTupleDataAppendState &state, MAP_TYPE &partition_entries, + const SelectionVector &append_sel, const idx_t append_count); //! Builds out the buffer space in the partitions void BuildBufferSpace(PartitionedTupleDataAppendState &state); + template + void BuildBufferSpace(PartitionedTupleDataAppendState &state, const MAP_TYPE &partition_entries); //! Create a collection for a specific a partition unique_ptr CreatePartitionCollection(idx_t partition_index) const { if (allocators) { @@ -129,11 +157,15 @@ class PartitionedTupleData { return make_uniq(buffer_manager, layout); } } + //! Verify count/data size of this PartitionedTupleData + void Verify() const; protected: PartitionedTupleDataType type; BufferManager &buffer_manager; const TupleDataLayout layout; + idx_t count; + idx_t data_size; mutex lock; shared_ptr allocators; diff --git a/src/duckdb/src/include/duckdb/common/types/row/row_data_collection_scanner.hpp b/src/duckdb/src/include/duckdb/common/types/row/row_data_collection_scanner.hpp index 0304bb2b1..697531814 100644 --- a/src/duckdb/src/include/duckdb/common/types/row/row_data_collection_scanner.hpp +++ b/src/duckdb/src/include/duckdb/common/types/row/row_data_collection_scanner.hpp @@ -74,6 +74,11 @@ struct RowDataCollectionScanner { return total_count - total_scanned; } + //! The number of remaining rows + inline idx_t BlockIndex() const { + return read_state.block_idx; + } + //! Swizzle the blocks for external scanning //! Swizzling is all or nothing, so if we have scanned previously, //! we need to re-swizzle. diff --git a/src/duckdb/src/include/duckdb/common/types/row/row_layout.hpp b/src/duckdb/src/include/duckdb/common/types/row/row_layout.hpp index 20fb46891..702cba4d2 100644 --- a/src/duckdb/src/include/duckdb/common/types/row/row_layout.hpp +++ b/src/duckdb/src/include/duckdb/common/types/row/row_layout.hpp @@ -10,28 +10,22 @@ #include "duckdb/common/common.hpp" #include "duckdb/common/types/validity_mask.hpp" -#include "duckdb/planner/expression.hpp" #include "duckdb/execution/operator/aggregate/aggregate_object.hpp" +#include "duckdb/planner/expression.hpp" namespace duckdb { class RowLayout { public: friend class TupleDataLayout; - - using Aggregates = vector; using ValidityBytes = TemplatedValidityMask; //! Creates an empty RowLayout RowLayout(); public: - //! Initializes the RowLayout with the specified types and aggregates to an empty RowLayout - void Initialize(vector types_p, Aggregates aggregates_p, bool align = true); //! Initializes the RowLayout with the specified types to an empty RowLayout void Initialize(vector types, bool align = true); - //! Initializes the RowLayout with the specified aggregates to an empty RowLayout - void Initialize(Aggregates aggregates_p, bool align = true); //! Returns the number of data columns inline idx_t ColumnCount() const { return types.size(); @@ -40,14 +34,6 @@ class RowLayout { inline const vector &GetTypes() const { return types; } - //! Returns the number of aggregates - inline idx_t AggregateCount() const { - return aggregates.size(); - } - //! Returns a list of the aggregates for this data chunk - inline Aggregates &GetAggregates() { - return aggregates; - } //! Returns the total width required for each row, including padding inline idx_t GetRowWidth() const { return row_width; @@ -64,10 +50,6 @@ class RowLayout { inline idx_t GetAggrOffset() const { return flag_width + data_width; } - //! Returns the total width required for the aggregates, including padding - inline idx_t GetAggrWidth() const { - return aggr_width; - } //! Returns the column offsets into each row inline const vector &GetOffsets() const { return offsets; @@ -83,14 +65,10 @@ class RowLayout { private: //! The types of the data columns vector types; - //! The aggregate functions - Aggregates aggregates; //! The width of the validity header idx_t flag_width; //! The width of the data portion idx_t data_width; - //! The width of the aggregate state portion - idx_t aggr_width; //! The width of the entire row idx_t row_width; //! The offsets to the columns and aggregate data in each row diff --git a/src/duckdb/src/include/duckdb/common/types/row/tuple_data_allocator.hpp b/src/duckdb/src/include/duckdb/common/types/row/tuple_data_allocator.hpp index 8c1d3db30..93eedfa09 100644 --- a/src/duckdb/src/include/duckdb/common/types/row/tuple_data_allocator.hpp +++ b/src/duckdb/src/include/duckdb/common/types/row/tuple_data_allocator.hpp @@ -55,6 +55,8 @@ class TupleDataAllocator { TupleDataAllocator(BufferManager &buffer_manager, const TupleDataLayout &layout); TupleDataAllocator(TupleDataAllocator &allocator); + //! Get the buffer manager + BufferManager &GetBufferManager(); //! Get the buffer allocator Allocator &GetAllocator(); //! Get the layout @@ -83,16 +85,16 @@ class TupleDataAllocator { private: //! Builds out a single part (grabs the lock) TupleDataChunkPart BuildChunkPart(TupleDataPinState &pin_state, TupleDataChunkState &chunk_state, - const idx_t append_offset, const idx_t append_count); + const idx_t append_offset, const idx_t append_count, TupleDataChunk &chunk); //! Internal function for InitializeChunkState void InitializeChunkStateInternal(TupleDataPinState &pin_state, TupleDataChunkState &chunk_state, idx_t offset, bool recompute, bool init_heap_pointers, bool init_heap_sizes, - vector &parts); + unsafe_vector> &parts); //! Internal function for ReleaseOrStoreHandles - static void ReleaseOrStoreHandlesInternal(TupleDataSegment &segment, vector &pinned_row_handles, - unordered_map &handles, - const unordered_set &block_ids, vector &blocks, - TupleDataPinProperties properties); + static void ReleaseOrStoreHandlesInternal(TupleDataSegment &segment, + unsafe_vector &pinned_row_handles, + perfect_map_t &handles, const perfect_set_t &block_ids, + unsafe_vector &blocks, TupleDataPinProperties properties); //! Pins the given row block BufferHandle &PinRowBlock(TupleDataPinState &state, const TupleDataChunkPart &part); //! Pins the given heap block @@ -108,9 +110,13 @@ class TupleDataAllocator { //! The layout of the data const TupleDataLayout layout; //! Blocks storing the fixed-size rows - vector row_blocks; + unsafe_vector row_blocks; //! Blocks storing the variable-size data of the fixed-size rows (e.g., string, list) - vector heap_blocks; + unsafe_vector heap_blocks; + + //! Re-usable arrays used while building buffer space + unsafe_vector> chunk_parts; + unsafe_vector> chunk_part_indices; }; } // namespace duckdb diff --git a/src/duckdb/src/include/duckdb/common/types/row/tuple_data_collection.hpp b/src/duckdb/src/include/duckdb/common/types/row/tuple_data_collection.hpp index 9148bf2c6..6950cb514 100644 --- a/src/duckdb/src/include/duckdb/common/types/row/tuple_data_collection.hpp +++ b/src/duckdb/src/include/duckdb/common/types/row/tuple_data_collection.hpp @@ -45,6 +45,7 @@ struct TupleDataGatherFunction { //! FIXME: rename to RowDataCollection after we phase it out class TupleDataCollection { friend class TupleDataChunkIterator; + friend class PartitionedTupleData; public: //! Constructs a TupleDataCollection with the specified layout @@ -63,8 +64,6 @@ class TupleDataCollection { idx_t ChunkCount() const; //! The size (in bytes) of the blocks held by this tuple data collection idx_t SizeInBytes() const; - //! Get pointers to the pinned blocks - void GetBlockPointers(vector &block_pointers) const; //! Unpins all held pins void Unpin(); @@ -186,6 +185,8 @@ class TupleDataCollection { void Initialize(); //! Gets all column ids void GetAllColumnIDs(vector &column_ids); + //! Adds a segment to this TupleDataCollection + void AddSegment(TupleDataSegment &&segment); //! Computes the heap sizes for the specific Vector that will be appended static void ComputeHeapSizes(Vector &heap_sizes_v, const Vector &source_v, TupleDataVectorFormat &source, @@ -219,7 +220,7 @@ class TupleDataCollection { void ScanAtIndex(TupleDataPinState &pin_state, TupleDataChunkState &chunk_state, const vector &column_ids, idx_t segment_index, idx_t chunk_index, DataChunk &result); - //! Verify counts of the segments in this collection + //! Verify count/data size of this collection void Verify() const; private: @@ -229,6 +230,8 @@ class TupleDataCollection { shared_ptr allocator; //! The number of entries stored in the TupleDataCollection idx_t count; + //! The size (in bytes) of this TupleDataCollection + idx_t data_size; //! The data segments of the TupleDataCollection unsafe_vector segments; //! The set of scatter functions diff --git a/src/duckdb/src/include/duckdb/common/types/row/tuple_data_layout.hpp b/src/duckdb/src/include/duckdb/common/types/row/tuple_data_layout.hpp index 95da61b42..f0d90e472 100644 --- a/src/duckdb/src/include/duckdb/common/types/row/tuple_data_layout.hpp +++ b/src/duckdb/src/include/duckdb/common/types/row/tuple_data_layout.hpp @@ -83,9 +83,14 @@ class TupleDataLayout { inline bool AllConstant() const { return all_constant; } + //! Gets offset to where heap size is stored inline idx_t GetHeapSizeOffset() const { return heap_size_offset; } + //! Returns whether any of the aggregates have a destructor + inline bool HasDestructor() const { + return has_destructor; + } private: //! The types of the data columns @@ -108,6 +113,8 @@ class TupleDataLayout { bool all_constant; //! Offset to the heap size of every row idx_t heap_size_offset; + //! Whether any of the aggregates have a destructor + bool has_destructor; }; } // namespace duckdb diff --git a/src/duckdb/src/include/duckdb/common/types/row/tuple_data_segment.hpp b/src/duckdb/src/include/duckdb/common/types/row/tuple_data_segment.hpp index 75559f7c4..7b8489f2d 100644 --- a/src/duckdb/src/include/duckdb/common/types/row/tuple_data_segment.hpp +++ b/src/duckdb/src/include/duckdb/common/types/row/tuple_data_segment.hpp @@ -10,6 +10,7 @@ #include "duckdb/common/common.hpp" #include "duckdb/common/mutex.hpp" +#include "duckdb/common/perfect_map_set.hpp" #include "duckdb/common/unordered_set.hpp" #include "duckdb/common/vector.hpp" #include "duckdb/storage/buffer_manager.hpp" @@ -21,7 +22,7 @@ class TupleDataLayout; struct TupleDataChunkPart { public: - TupleDataChunkPart(); + TupleDataChunkPart(mutex &lock); //! Disable copy constructors TupleDataChunkPart(const TupleDataChunkPart &other) = delete; @@ -45,8 +46,8 @@ struct TupleDataChunkPart { uint32_t total_heap_size; //! Tuple count for this chunk part uint32_t count; - //! Lock for recomputing heap pointers - mutex lock; + //! Lock for recomputing heap pointers (owned by TupleDataChunk) + reference lock; }; struct TupleDataChunk { @@ -70,13 +71,15 @@ struct TupleDataChunk { public: //! The parts of this chunk - vector parts; + unsafe_vector parts; //! The row block ids referenced by the chunk - unordered_set row_block_ids; + perfect_set_t row_block_ids; //! The heap block ids referenced by the chunk - unordered_set heap_block_ids; + perfect_set_t heap_block_ids; //! Tuple count for this chunk idx_t count; + //! Lock for recomputing heap pointers + unsafe_unique_ptr lock; }; struct TupleDataSegment { @@ -112,13 +115,15 @@ struct TupleDataSegment { unsafe_vector chunks; //! The tuple count of this segment idx_t count; + //! The data size of this segment + idx_t data_size; //! Lock for modifying pinned_handles mutex pinned_handles_lock; //! Where handles to row blocks will be stored with TupleDataPinProperties::KEEP_EVERYTHING_PINNED - vector pinned_row_handles; + unsafe_vector pinned_row_handles; //! Where handles to heap blocks will be stored with TupleDataPinProperties::KEEP_EVERYTHING_PINNED - vector pinned_heap_handles; + unsafe_vector pinned_heap_handles; }; } // namespace duckdb diff --git a/src/duckdb/src/include/duckdb/common/types/row/tuple_data_states.hpp b/src/duckdb/src/include/duckdb/common/types/row/tuple_data_states.hpp index 0d22bdadd..1bd5cf333 100644 --- a/src/duckdb/src/include/duckdb/common/types/row/tuple_data_states.hpp +++ b/src/duckdb/src/include/duckdb/common/types/row/tuple_data_states.hpp @@ -9,6 +9,7 @@ #pragma once #include "duckdb/common/mutex.hpp" +#include "duckdb/common/perfect_map_set.hpp" #include "duckdb/common/types.hpp" namespace duckdb { @@ -26,8 +27,8 @@ enum class TupleDataPinProperties : uint8_t { }; struct TupleDataPinState { - unordered_map row_handles; - unordered_map heap_handles; + perfect_map_t row_handles; + perfect_map_t heap_handles; TupleDataPinProperties properties = TupleDataPinProperties::INVALID; }; diff --git a/src/duckdb/src/include/duckdb/common/types/vector.hpp b/src/duckdb/src/include/duckdb/common/types/vector.hpp index b3091cdc4..690bd69f9 100644 --- a/src/duckdb/src/include/duckdb/common/types/vector.hpp +++ b/src/duckdb/src/include/duckdb/common/types/vector.hpp @@ -101,14 +101,14 @@ class Vector { DUCKDB_API void Reference(const Value &value); //! Causes this vector to reference the data held by the other vector. //! The type of the "other" vector should match the type of this vector - DUCKDB_API void Reference(Vector &other); + DUCKDB_API void Reference(const Vector &other); //! Reinterpret the data of the other vector as the type of this vector //! Note that this takes the data of the other vector as-is and places it in this vector //! Without changing the type of this vector - DUCKDB_API void Reinterpret(Vector &other); + DUCKDB_API void Reinterpret(const Vector &other); //! Causes this vector to reference the data held by the other vector, changes the type if required. - DUCKDB_API void ReferenceAndSetType(Vector &other); + DUCKDB_API void ReferenceAndSetType(const Vector &other); //! Resets a vector from a vector cache. //! This turns the vector back into an empty FlatVector with STANDARD_VECTOR_SIZE entries. diff --git a/src/duckdb/src/include/duckdb/common/vector.hpp b/src/duckdb/src/include/duckdb/common/vector.hpp index 61690173b..66a6bc731 100644 --- a/src/duckdb/src/include/duckdb/common/vector.hpp +++ b/src/duckdb/src/include/duckdb/common/vector.hpp @@ -88,14 +88,14 @@ class vector : public std::vector<_Tp, std::allocator<_Tp>> { } typename original::reference back() { - if (original::empty()) { + if (MemorySafety::enabled && original::empty()) { throw InternalException("'back' called on an empty vector!"); } return get(original::size() - 1); } typename original::const_reference back() const { - if (original::empty()) { + if (MemorySafety::enabled && original::empty()) { throw InternalException("'back' called on an empty vector!"); } return get(original::size() - 1); diff --git a/src/duckdb/src/include/duckdb/execution/aggregate_hashtable.hpp b/src/duckdb/src/include/duckdb/execution/aggregate_hashtable.hpp index 144a9d015..d5fabe518 100644 --- a/src/duckdb/src/include/duckdb/execution/aggregate_hashtable.hpp +++ b/src/duckdb/src/include/duckdb/execution/aggregate_hashtable.hpp @@ -8,12 +8,13 @@ #pragma once -#include "duckdb/common/types/row/tuple_data_collection.hpp" +#include "duckdb/common/types/row/partitioned_tuple_data.hpp" #include "duckdb/execution/base_aggregate_hashtable.hpp" #include "duckdb/storage/arena_allocator.hpp" #include "duckdb/storage/buffer/buffer_handle.hpp" namespace duckdb { + class BlockHandle; class BufferHandle; @@ -27,91 +28,87 @@ struct FlushMoveState; stores them in the HT. It uses linear probing for collision resolution. */ -// two part hash table -// hashes and payload -// hashes layout: -// [SALT][PAGE_NR][PAGE_OFFSET] -// [SALT] are the high bits of the hash value, e.g. 16 for 64 bit hashes -// [PAGE_NR] is the buffer managed payload page index -// [PAGE_OFFSET] is the logical entry offset into said payload page - -// NOTE: PAGE_NR and PAGE_OFFSET are reversed for 64 bit HTs because struct packing - -// payload layout -// [VALIDITY][GROUPS][HASH][PADDING][PAYLOAD] -// [VALIDITY] is the validity bits of the data columns (including the HASH) -// [GROUPS] is the group data, could be multiple values, fixed size, strings are elsewhere -// [HASH] is the hash data of the groups -// [PADDING] is gunk data to align payload properly -// [PAYLOAD] is the payload (i.e. the aggregate states) -struct aggr_ht_entry_64 { - uint16_t salt; - uint16_t page_offset; - uint32_t page_nr; // this has to come last because alignment -}; +struct aggr_ht_entry_t { +public: + explicit aggr_ht_entry_t(hash_t value_p) : value(value_p) { + } -struct aggr_ht_entry_32 { - uint8_t salt; - uint8_t page_nr; - uint16_t page_offset; -}; + inline bool IsOccupied() const { + return value != 0; + } + + inline data_ptr_t GetPointer() const { + D_ASSERT(IsOccupied()); + return reinterpret_cast(value & POINTER_MASK); + } + inline void SetPointer(const data_ptr_t &pointer) { + // Pointer shouldn't use upper bits + D_ASSERT((reinterpret_cast(pointer) & SALT_MASK) == 0); + // Value should have all 1's in the pointer area + D_ASSERT((value & POINTER_MASK) == POINTER_MASK); + // Set upper bits to 1 in pointer so the salt stays intact + value &= reinterpret_cast(pointer) | SALT_MASK; + } -enum HtEntryType { HT_WIDTH_32, HT_WIDTH_64 }; + static inline hash_t ExtractSalt(const hash_t &hash) { + // Leaves upper bits intact, sets lower bits to all 1's + return hash | POINTER_MASK; + } + inline hash_t GetSalt() const { + return ExtractSalt(value); + } + inline void SetSalt(const hash_t &salt) { + // Shouldn't be occupied when we set this + D_ASSERT(!IsOccupied()); + // Salt should have all 1's in the pointer field + D_ASSERT((salt & POINTER_MASK) == POINTER_MASK); + // No need to mask, just put the whole thing there + value = salt; + } -struct AggregateHTScanState { - mutex lock; - TupleDataScanState scan_state; -}; +private: + //! Upper 16 bits are salt + static constexpr const hash_t SALT_MASK = 0xFFFF000000000000; + //! Lower 48 bits are the pointer + static constexpr const hash_t POINTER_MASK = 0x0000FFFFFFFFFFFF; -struct AggregateHTAppendState { - AggregateHTAppendState(); - - Vector ht_offsets; - Vector hash_salts; - SelectionVector group_compare_vector; - SelectionVector no_match_vector; - SelectionVector empty_vector; - SelectionVector new_groups; - Vector addresses; - unsafe_unique_array group_data; - DataChunk group_chunk; - - TupleDataChunkState chunk_state; - bool chunk_state_initialized; + hash_t value; }; class GroupedAggregateHashTable : public BaseAggregateHashTable { -public: - //! The hash table load factor, when a resize is triggered - constexpr static float LOAD_FACTOR = 1.5; - constexpr static uint8_t HASH_WIDTH = sizeof(hash_t); - public: GroupedAggregateHashTable(ClientContext &context, Allocator &allocator, vector group_types, vector payload_types, const vector &aggregates, - HtEntryType entry_type = HtEntryType::HT_WIDTH_64, - idx_t initial_capacity = InitialCapacity()); + idx_t initial_capacity = InitialCapacity(), idx_t radix_bits = 0); GroupedAggregateHashTable(ClientContext &context, Allocator &allocator, vector group_types, vector payload_types, vector aggregates, - HtEntryType entry_type = HtEntryType::HT_WIDTH_64, - idx_t initial_capacity = InitialCapacity()); + idx_t initial_capacity = InitialCapacity(), idx_t radix_bits = 0); GroupedAggregateHashTable(ClientContext &context, Allocator &allocator, vector group_types); ~GroupedAggregateHashTable() override; public: + //! The hash table load factor, when a resize is triggered + constexpr static float LOAD_FACTOR = 1.5; + + //! Get the layout of this HT + const TupleDataLayout &GetLayout() const; + //! Number of groups in the HT + idx_t Count() const; + //! Initial capacity of the HT + static idx_t InitialCapacity(); + //! Capacity that can hold 'count' entries without resizing + static idx_t GetCapacityForCount(idx_t count); + //! Current capacity of the HT + idx_t Capacity() const; + //! Threshold at which to resize the HT + idx_t ResizeThreshold() const; + //! Add the given data to the HT, computing the aggregates grouped by the //! data in the group chunk. When resize = true, aggregates will not be //! computed but instead just assigned. - idx_t AddChunk(AggregateHTAppendState &state, DataChunk &groups, DataChunk &payload, - const unsafe_vector &filter); - idx_t AddChunk(AggregateHTAppendState &state, DataChunk &groups, Vector &group_hashes, DataChunk &payload, - const unsafe_vector &filter); - idx_t AddChunk(AggregateHTAppendState &state, DataChunk &groups, DataChunk &payload, AggregateType filter); - - //! Scan the HT starting from the scan_position until the result and group - //! chunks are filled. scan_position will be updated by this function. - //! Returns the amount of elements found. - idx_t Scan(TupleDataParallelScanState &gstate, TupleDataLocalScanState &lstate, DataChunk &result); + idx_t AddChunk(DataChunk &groups, DataChunk &payload, const unsafe_vector &filter); + idx_t AddChunk(DataChunk &groups, Vector &group_hashes, DataChunk &payload, const unsafe_vector &filter); + idx_t AddChunk(DataChunk &groups, DataChunk &payload, AggregateType filter); //! Fetch the aggregates for specific groups from the HT and place them in the result void FetchAggregates(DataChunk &groups, DataChunk &result); @@ -119,108 +116,90 @@ class GroupedAggregateHashTable : public BaseAggregateHashTable { //! Finds or creates groups in the hashtable using the specified group keys. The addresses vector will be filled //! with pointers to the groups in the hash table, and the new_groups selection vector will point to the newly //! created groups. The return value is the amount of newly created groups. - idx_t FindOrCreateGroups(AggregateHTAppendState &state, DataChunk &groups, Vector &group_hashes, - Vector &addresses_out, SelectionVector &new_groups_out); - idx_t FindOrCreateGroups(AggregateHTAppendState &state, DataChunk &groups, Vector &addresses_out, + idx_t FindOrCreateGroups(DataChunk &groups, Vector &group_hashes, Vector &addresses_out, SelectionVector &new_groups_out); - void FindOrCreateGroups(AggregateHTAppendState &state, DataChunk &groups, Vector &addresses_out); - - //! Executes the filter(if any) and update the aggregates - void Combine(GroupedAggregateHashTable &other); - - //! Appends the data in the other HT to this one - void Append(GroupedAggregateHashTable &other); - - TupleDataCollection &GetDataCollection() { - return *data_collection; - } - - idx_t Count() const { - return data_collection->Count(); - } - - idx_t DataSize() const { - return data_collection->SizeInBytes(); - } - - static idx_t InitialCapacity(); - idx_t Capacity() { - return capacity; - } - - static idx_t FirstPartSize(idx_t count, HtEntryType entry_type) { - idx_t entry_size = entry_type == HT_WIDTH_32 ? sizeof(aggr_ht_entry_32) : sizeof(aggr_ht_entry_64); - return NextPowerOfTwo(count * 2L) * entry_size; - } + idx_t FindOrCreateGroups(DataChunk &groups, Vector &addresses_out, SelectionVector &new_groups_out); + void FindOrCreateGroups(DataChunk &groups, Vector &addresses_out); - idx_t TotalSize() const { - return DataSize() + FirstPartSize(Count(), entry_type); - } + unique_ptr &GetPartitionedData(); + shared_ptr GetAggregateAllocator(); - idx_t ResizeThreshold(); - idx_t MaxCapacity(); - static idx_t GetMaxCapacity(HtEntryType entry_type, idx_t tuple_size); + //! Resize the HT to the specified size. Must be larger than the current size. + void Resize(idx_t size); + //! Resets the pointer table of the HT to all 0's + void ClearPointerTable(); + //! Resets the group count to 0 + void ResetCount(); + //! Set the radix bits for this HT + void SetRadixBits(idx_t radix_bits); + //! Initializes the PartitionedTupleData + void InitializePartitionedData(); - void Partition(vector &partition_hts, idx_t radix_bits, bool sink_done); - void InitializeFirstPart(); + //! Executes the filter(if any) and update the aggregates + void Combine(GroupedAggregateHashTable &other); + void Combine(TupleDataCollection &other_data); - void Finalize(); + //! Unpins the data blocks + void UnpinData(); private: - HtEntryType entry_type; - - //! The capacity of the HT. This can be increased using GroupedAggregateHashTable::Resize - idx_t capacity; - //! Tuple width - idx_t tuple_size; - //! Tuples per block - idx_t tuples_per_block; + //! Append state + struct AggregateHTAppendState { + AggregateHTAppendState(); + + PartitionedTupleDataAppendState append_state; + + Vector ht_offsets; + Vector hash_salts; + SelectionVector group_compare_vector; + SelectionVector no_match_vector; + SelectionVector empty_vector; + SelectionVector new_groups; + Vector addresses; + unsafe_unique_array group_data; + DataChunk group_chunk; + } state; + + //! The number of radix bits to partition by + idx_t radix_bits; //! The data of the HT - unique_ptr data_collection; - TupleDataPinState td_pin_state; - vector payload_hds_ptrs; + unique_ptr partitioned_data; - //! The hashes of the HT - AllocatedData hashes_hdl; - data_ptr_t hashes_hdl_ptr; - idx_t hash_offset; // Offset into the layout of the hash column - - hash_t hash_prefix_shift; + //! Predicates for matching groups (always ExpressionType::COMPARE_EQUAL) + vector predicates; + //! The number of groups in the HT + idx_t count; + //! The capacity of the HT. This can be increased using GroupedAggregateHashTable::Resize + idx_t capacity; + //! The hash map (pointer table) of the HT: allocated data and pointer into it + AllocatedData hash_map; + aggr_ht_entry_t *entries; + //! Offset of the hash column in the rows + idx_t hash_offset; //! Bitmask for getting relevant bits from the hashes to determine the position hash_t bitmask; - bool is_finalized; - - vector predicates; - //! The active arena allocator used by the aggregates for their internal state shared_ptr aggregate_allocator; //! Owning arena allocators that this HT has data from vector> stored_allocators; private: + //! Disabled the copy constructor GroupedAggregateHashTable(const GroupedAggregateHashTable &) = delete; - + //! Destroy the HT void Destroy(); - void Verify(); - template - void VerifyInternal(); - //! Resize the HT to the specified size. Must be larger than the current size. - template - void Resize(idx_t size); - //! Initializes the first part of the HT - template - void InitializeHashes(); + + //! Apply bitmask to get the entry in the HT + inline idx_t ApplyBitMask(hash_t hash) const; + //! Does the actual group matching / creation - template - idx_t FindOrCreateGroupsInternal(DataChunk &groups, Vector &group_hashes_v, Vector &addresses_v, + idx_t FindOrCreateGroupsInternal(DataChunk &groups, Vector &group_hashes, Vector &addresses, SelectionVector &new_groups); - //! Updates payload_hds_ptrs with the new pointers (after appending to data_collection) - void UpdateBlockPointers(); - template - idx_t FindOrCreateGroupsInternal(AggregateHTAppendState &state, DataChunk &groups, Vector &group_hashes, - Vector &addresses, SelectionVector &new_groups); + + //! Verify the pointer table of the HT + void Verify(); }; } // namespace duckdb diff --git a/src/duckdb/src/include/duckdb/execution/operator/aggregate/physical_hash_aggregate.hpp b/src/duckdb/src/include/duckdb/execution/operator/aggregate/physical_hash_aggregate.hpp index 4a0f5f6d6..d36b17c0a 100644 --- a/src/duckdb/src/include/duckdb/execution/operator/aggregate/physical_hash_aggregate.hpp +++ b/src/duckdb/src/include/duckdb/execution/operator/aggregate/physical_hash_aggregate.hpp @@ -8,17 +8,18 @@ #pragma once +#include "duckdb/execution/operator/aggregate/distinct_aggregate_data.hpp" +#include "duckdb/execution/operator/aggregate/grouped_aggregate_data.hpp" #include "duckdb/execution/physical_operator.hpp" -#include "duckdb/storage/data_table.hpp" -#include "duckdb/parser/group_by_node.hpp" #include "duckdb/execution/radix_partitioned_hashtable.hpp" -#include "duckdb/execution/operator/aggregate/grouped_aggregate_data.hpp" -#include "duckdb/execution/operator/aggregate/distinct_aggregate_data.hpp" +#include "duckdb/parser/group_by_node.hpp" +#include "duckdb/storage/data_table.hpp" namespace duckdb { class ClientContext; class BufferManager; +class PhysicalHashAggregate; struct HashAggregateGroupingData { public: diff --git a/src/duckdb/src/include/duckdb/execution/operator/aggregate/physical_window.hpp b/src/duckdb/src/include/duckdb/execution/operator/aggregate/physical_window.hpp index 6992aab70..5bf8fdcc8 100644 --- a/src/duckdb/src/include/duckdb/execution/operator/aggregate/physical_window.hpp +++ b/src/duckdb/src/include/duckdb/execution/operator/aggregate/physical_window.hpp @@ -36,6 +36,8 @@ class PhysicalWindow : public PhysicalOperator { GlobalSourceState &gstate) const override; unique_ptr GetGlobalSourceState(ClientContext &context) const override; SourceResultType GetData(ExecutionContext &context, DataChunk &chunk, OperatorSourceInput &input) const override; + idx_t GetBatchIndex(ExecutionContext &context, DataChunk &chunk, GlobalSourceState &gstate, + LocalSourceState &lstate) const override; bool IsSource() const override { return true; @@ -44,9 +46,8 @@ class PhysicalWindow : public PhysicalOperator { return true; } - OrderPreservationType SourceOrder() const override { - return OrderPreservationType::NO_ORDER; - } + bool SupportsBatchIndex() const override; + OrderPreservationType SourceOrder() const override; public: // Sink interface diff --git a/src/duckdb/src/include/duckdb/execution/operator/scan/csv/base_csv_reader.hpp b/src/duckdb/src/include/duckdb/execution/operator/scan/csv/base_csv_reader.hpp new file mode 100644 index 000000000..23a1c69db --- /dev/null +++ b/src/duckdb/src/include/duckdb/execution/operator/scan/csv/base_csv_reader.hpp @@ -0,0 +1,119 @@ +//===----------------------------------------------------------------------===// +// DuckDB +// +// duckdb/execution/operator/scan/csv/base_csv_reader.hpp +// +// +//===----------------------------------------------------------------------===// + +#pragma once + +#include "duckdb/execution/physical_operator.hpp" +#include "duckdb/parser/parsed_data/copy_info.hpp" +#include "duckdb/function/scalar/strftime_format.hpp" +#include "duckdb/common/types/chunk_collection.hpp" +#include "duckdb/common/enums/file_compression_type.hpp" +#include "duckdb/common/map.hpp" +#include "duckdb/common/queue.hpp" +#include "duckdb/execution/operator/scan/csv/csv_reader_options.hpp" +#include "duckdb/common/multi_file_reader.hpp" +#include "duckdb/execution/operator/scan/csv/csv_line_info.hpp" + +#include + +namespace duckdb { +struct CopyInfo; +struct CSVFileHandle; +struct FileHandle; +struct StrpTimeFormat; + +class FileOpener; +class FileSystem; + +enum class ParserMode : uint8_t { PARSING = 0, SNIFFING_DATATYPES = 1, PARSING_HEADER = 2 }; + +//! Buffered CSV reader is a class that reads values from a stream and parses them as a CSV file +class BaseCSVReader { +public: + BaseCSVReader(ClientContext &context, CSVReaderOptions options, + const vector &requested_types = vector()); + virtual ~BaseCSVReader(); + + ClientContext &context; + FileSystem &fs; + Allocator &allocator; + CSVReaderOptions options; + vector return_types; + vector names; + MultiFileReaderData reader_data; + + idx_t linenr = 0; + bool linenr_estimated = false; + + bool row_empty = false; + idx_t sample_chunk_idx = 0; + bool jumping_samples = false; + bool end_of_file_reached = false; + bool bom_checked = false; + + idx_t bytes_in_chunk = 0; + double bytes_per_line_avg = 0; + + DataChunk parse_chunk; + + ParserMode mode; + +public: + const string &GetFileName() { + return options.file_path; + } + const vector &GetNames() { + return names; + } + const vector &GetTypes() { + return return_types; + } + //! Get the 1-indexed global line number for the given local error line + virtual idx_t GetLineError(idx_t line_error, idx_t buffer_idx, bool stop_at_first = true) { + return line_error + 1; + }; + + //! Initialize projection indices to select all columns + void InitializeProjection(); + + static unique_ptr OpenCSV(ClientContext &context, const CSVReaderOptions &options); + + static bool TryCastDateVector(map &options, Vector &input_vector, + Vector &result_vector, idx_t count, string &error_message, idx_t &line_error); + + static bool TryCastTimestampVector(map &options, Vector &input_vector, + Vector &result_vector, idx_t count, string &error_message); + +protected: + //! Initializes the parse_chunk with varchar columns and aligns info with new number of cols + void InitParseChunk(idx_t num_cols); + //! Adds a value to the current row + void AddValue(string_t str_val, idx_t &column, vector &escape_positions, bool has_quotes, + idx_t buffer_idx = 0); + //! Adds a row to the insert_chunk, returns true if the chunk is filled as a result of this row being added + bool AddRow(DataChunk &insert_chunk, idx_t &column, string &error_message, idx_t buffer_idx = 0); + //! Finalizes a chunk, parsing all values that have been added so far and adding them to the insert_chunk + bool Flush(DataChunk &insert_chunk, idx_t buffer_idx = 0, bool try_add_line = false); + + void VerifyUTF8(idx_t col_idx); + void VerifyUTF8(idx_t col_idx, idx_t row_idx, DataChunk &chunk, int64_t offset = 0); + string GetLineNumberStr(idx_t linenr, bool linenr_estimated, idx_t buffer_idx = 0); + + //! Sets the newline delimiter + void SetNewLineDelimiter(bool carry = false, bool carry_followed_by_nl = false); + + //! Verifies that the line length did not go over a pre-defined limit. + void VerifyLineLength(idx_t line_size, idx_t buffer_idx = 0); + +protected: + //! Whether or not the current row's columns have overflown return_types.size() + bool error_column_overflow = false; + //! Number of sniffed columns - only used when auto-detecting +}; + +} // namespace duckdb diff --git a/src/duckdb/src/include/duckdb/execution/operator/scan/csv/buffered_csv_reader.hpp b/src/duckdb/src/include/duckdb/execution/operator/scan/csv/buffered_csv_reader.hpp new file mode 100644 index 000000000..0dc156771 --- /dev/null +++ b/src/duckdb/src/include/duckdb/execution/operator/scan/csv/buffered_csv_reader.hpp @@ -0,0 +1,72 @@ +//===----------------------------------------------------------------------===// +// DuckDB +// +// duckdb/execution/operator/scan/csv/buffered_csv_reader.hpp +// +// +//===----------------------------------------------------------------------===// + +#pragma once + +#include "duckdb/execution/operator/scan/csv/csv_buffer_manager.hpp" +#include "duckdb/execution/operator/scan/csv/base_csv_reader.hpp" +#include "duckdb/execution/operator/scan/csv/csv_state_machine_cache.hpp" + +namespace duckdb { +struct CopyInfo; +struct CSVFileHandle; +struct FileHandle; +struct StrpTimeFormat; + +class FileOpener; +class FileSystem; + +//! Buffered CSV reader is a class that reads values from a stream and parses them as a CSV file +class BufferedCSVReader : public BaseCSVReader { + //! Initial buffer read size; can be extended for long lines + static constexpr idx_t INITIAL_BUFFER_SIZE = 16384; + //! Larger buffer size for non disk files + static constexpr idx_t INITIAL_BUFFER_SIZE_LARGE = 10000000; // 10MB + +public: + BufferedCSVReader(ClientContext &context, CSVReaderOptions options, + const vector &requested_types = vector()); + BufferedCSVReader(ClientContext &context, string filename, CSVReaderOptions options, + const vector &requested_types = vector()); + virtual ~BufferedCSVReader() { + } + + unsafe_unique_array buffer; + idx_t buffer_size; + idx_t position; + idx_t start = 0; + + vector> cached_buffers; + + unique_ptr file_handle; + //! CSV State Machine Cache + CSVStateMachineCache state_machine_cache; + +public: + //! Extract a single DataChunk from the CSV file and stores it in insert_chunk + void ParseCSV(DataChunk &insert_chunk); + static string ColumnTypesError(case_insensitive_map_t sql_types_per_column, const vector &names); + +private: + //! Initialize Parser + void Initialize(const vector &requested_types); + //! Skips skip_rows, reads header row from input stream + void SkipRowsAndReadHeader(idx_t skip_rows, bool skip_header); + //! Resets the buffer + void ResetBuffer(); + //! Reads a new buffer from the CSV file if the current one has been exhausted + bool ReadBuffer(idx_t &start, idx_t &line_start); + //! Try to parse a single datachunk from the file. Throws an exception if anything goes wrong. + void ParseCSV(ParserMode mode); + //! Extract a single DataChunk from the CSV file and stores it in insert_chunk + bool TryParseCSV(ParserMode mode, DataChunk &insert_chunk, string &error_message); + //! Skip Empty lines for tables with over one column + void SkipEmptyLines(); +}; + +} // namespace duckdb diff --git a/src/duckdb/src/include/duckdb/execution/operator/scan/csv/csv_buffer.hpp b/src/duckdb/src/include/duckdb/execution/operator/scan/csv/csv_buffer.hpp new file mode 100644 index 000000000..756a11233 --- /dev/null +++ b/src/duckdb/src/include/duckdb/execution/operator/scan/csv/csv_buffer.hpp @@ -0,0 +1,110 @@ +//===----------------------------------------------------------------------===// +// DuckDB +// +// duckdb/execution/operator/scan/csv/csv_buffer.hpp +// +// +//===----------------------------------------------------------------------===// + +#pragma once + +#include "duckdb/common/constants.hpp" +#include "duckdb/execution/operator/scan/csv/csv_file_handle.hpp" +#include "duckdb/storage/buffer_manager.hpp" +#include "duckdb/storage/block_manager.hpp" +#include "duckdb/storage/buffer/block_handle.hpp" + +namespace duckdb { + +class CSVBufferHandle { +public: + CSVBufferHandle(BufferHandle handle_p, idx_t actual_size_p, const bool is_first_buffer_p, + const bool is_final_buffer_p, idx_t csv_global_state_p, idx_t start_position_p, idx_t file_idx_p) + : handle(std::move(handle_p)), actual_size(actual_size_p), is_first_buffer(is_first_buffer_p), + is_last_buffer(is_final_buffer_p), csv_global_start(csv_global_state_p), start_position(start_position_p), + file_idx(file_idx_p) {}; + CSVBufferHandle() + : actual_size(0), is_first_buffer(false), is_last_buffer(false), csv_global_start(0), start_position(0), + file_idx(0) {}; + //! Handle created during allocation + BufferHandle handle; + const idx_t actual_size; + const bool is_first_buffer; + const bool is_last_buffer; + const idx_t csv_global_start; + const idx_t start_position; + const idx_t file_idx; + inline char *Ptr() { + return char_ptr_cast(handle.Ptr()); + } +}; + +//! CSV Buffers are parts of a decompressed CSV File. +//! For a decompressed file of 100Mb. With our Buffer size set to 32Mb, we would generate 4 buffers. +//! One for the first 32Mb, second and third for the other 32Mb, and the last one with 4 Mb +//! These buffers are actually used for sniffing and parsing! +class CSVBuffer { +public: + //! Constructor for Initial Buffer + CSVBuffer(ClientContext &context, idx_t buffer_size_p, CSVFileHandle &file_handle, + idx_t &global_csv_current_position, idx_t file_number); + + //! Constructor for `Next()` Buffers + CSVBuffer(CSVFileHandle &file_handle, ClientContext &context, idx_t buffer_size, idx_t global_csv_current_position, + idx_t file_number_p); + + //! Creates a new buffer with the next part of the CSV File + shared_ptr Next(CSVFileHandle &file_handle, idx_t buffer_size, idx_t file_number); + + //! Gets the buffer actual size + idx_t GetBufferSize(); + + //! Gets the start position of the buffer, only relevant for the first time it's scanned + idx_t GetStart(); + + //! If this buffer is the last buffer of the CSV File + bool IsCSVFileLastBuffer(); + + //! Allocates internal buffer, sets 'block' and 'handle' variables. + void AllocateBuffer(idx_t buffer_size); + + void Reload(CSVFileHandle &file_handle); + //! Wrapper for the Pin Function, if it can seek, it means that the buffer might have been destroyed, hence we must + //! Scan it from the disk file again. + unique_ptr Pin(CSVFileHandle &file_handle); + //! Wrapper for the unpin + void Unpin(); + char *Ptr() { + return char_ptr_cast(handle.Ptr()); + } + + static constexpr idx_t CSV_BUFFER_SIZE = 32000000; // 32MB + //! In case the file has a size < 32MB, we will use this size instead + //! This is to avoid mallocing a lot of memory for a small file + //! And if it's a compressed file we can't use the actual size of the file + static constexpr idx_t CSV_MINIMUM_BUFFER_SIZE = 10000000; // 10MB + //! If this is the last buffer of the CSV File + bool last_buffer = false; + +private: + ClientContext &context; + //! Actual size can be smaller than the buffer size in case we allocate it too optimistically. + idx_t file_size; + //! We need to check for Byte Order Mark, to define the start position of this buffer + //! https://en.wikipedia.org/wiki/Byte_order_mark#UTF-8 + idx_t start_position = 0; + //! If this is the first buffer of the CSV File + bool first_buffer = false; + //! Global position from the CSV File where this buffer starts + idx_t global_csv_start = 0; + //! Number of the file that is in this buffer + idx_t file_number = 0; + //! If we can seek in the file or not. + //! If we can't seek, this means we can't destroy the buffers + bool can_seek; + //! -------- Allocated Block ---------// + //! Block created in allocation + shared_ptr block; + BufferHandle handle; +}; +} // namespace duckdb diff --git a/src/duckdb/src/include/duckdb/execution/operator/scan/csv/csv_buffer_manager.hpp b/src/duckdb/src/include/duckdb/execution/operator/scan/csv/csv_buffer_manager.hpp new file mode 100644 index 000000000..169ac0be9 --- /dev/null +++ b/src/duckdb/src/include/duckdb/execution/operator/scan/csv/csv_buffer_manager.hpp @@ -0,0 +1,103 @@ +//===----------------------------------------------------------------------===// +// DuckDB +// +// duckdb/execution/operator/scan/csv/csv_buffer_manager.hpp +// +// +//===----------------------------------------------------------------------===// + +#pragma once + +#include "duckdb/main/client_context.hpp" +#include "duckdb/execution/operator/scan/csv/csv_file_handle.hpp" +#include "duckdb/execution/operator/scan/csv/csv_reader_options.hpp" + +namespace duckdb { +class CSVBuffer; +class CSVStateMachine; + +//! This class is used to manage the CSV buffers. Buffers are cached when used for auto detection. +//! When parsing, buffer are not cached and just returned. +class CSVBufferManager { +public: + CSVBufferManager(ClientContext &context, unique_ptr file_handle, const CSVReaderOptions &options, + idx_t file_idx = 0); + //! Returns a buffer from a buffer id (starting from 0). If it's in the auto-detection then we cache new buffers + //! Otherwise we remove them from the cache if they are already there, or just return them bypassing the cache. + unique_ptr GetBuffer(const idx_t pos); + //! Returns the starting position of the first buffer + idx_t GetStartPos(); + //! unique_ptr to the file handle, gets stolen after sniffing + unique_ptr file_handle; + //! Initializes the buffer manager, during it's construction/reset + void Initialize(); + + void UnpinBuffer(idx_t cache_idx); + + ClientContext &context; + idx_t skip_rows = 0; + idx_t file_idx; + bool done = false; + +private: + //! Reads next buffer in reference to cached_buffers.front() + bool ReadNextAndCacheIt(); + vector> cached_buffers; + shared_ptr last_buffer; + idx_t global_csv_pos = 0; + //! The size of the buffer, if the csv file has a smaller size than this, we will use that instead to malloc less + idx_t buffer_size; + //! Starting position of first buffer + idx_t start_pos = 0; +}; + +class CSVBufferIterator { +public: + explicit CSVBufferIterator(shared_ptr buffer_manager_p) + : buffer_manager(std::move(buffer_manager_p)) { + cur_pos = buffer_manager->GetStartPos(); + }; + + //! This functions templates an operation over the CSV File + template + inline bool Process(CSVStateMachine &machine, T &result) { + + OP::Initialize(machine); + //! If current buffer is not set we try to get a new one + if (!cur_buffer_handle) { + cur_pos = 0; + if (cur_buffer_idx == 0) { + cur_pos = buffer_manager->GetStartPos(); + } + cur_buffer_handle = buffer_manager->GetBuffer(cur_buffer_idx++); + D_ASSERT(cur_buffer_handle); + } + while (cur_buffer_handle) { + char *buffer_handle_ptr = cur_buffer_handle->Ptr(); + while (cur_pos < cur_buffer_handle->actual_size) { + if (OP::Process(machine, result, buffer_handle_ptr[cur_pos], cur_pos)) { + //! Not-Done Processing the File, but the Operator is happy! + OP::Finalize(machine, result); + return false; + } + cur_pos++; + } + cur_buffer_handle = buffer_manager->GetBuffer(cur_buffer_idx++); + cur_pos = 0; + } + //! Done Processing the File + OP::Finalize(machine, result); + return true; + } + //! Returns true if the iterator is finished + bool Finished(); + //! Resets the iterator + void Reset(); + +private: + idx_t cur_pos = 0; + idx_t cur_buffer_idx = 0; + shared_ptr buffer_manager; + unique_ptr cur_buffer_handle; +}; +} // namespace duckdb diff --git a/src/duckdb/src/include/duckdb/execution/operator/scan/csv/csv_file_handle.hpp b/src/duckdb/src/include/duckdb/execution/operator/scan/csv/csv_file_handle.hpp new file mode 100644 index 000000000..c27538fe2 --- /dev/null +++ b/src/duckdb/src/include/duckdb/execution/operator/scan/csv/csv_file_handle.hpp @@ -0,0 +1,59 @@ +//===----------------------------------------------------------------------===// +// DuckDB +// +// duckdb/execution/operator/scan/csv/csv_file_handle.hpp +// +// +//===----------------------------------------------------------------------===// + +#pragma once + +#include "duckdb/common/file_system.hpp" +#include "duckdb/common/mutex.hpp" +#include "duckdb/common/helper.hpp" +#include "duckdb/common/allocator.hpp" + +namespace duckdb { +class Allocator; +class FileSystem; + +struct CSVFileHandle { +public: + CSVFileHandle(FileSystem &fs, Allocator &allocator, unique_ptr file_handle_p, const string &path_p, + FileCompressionType compression); + + mutex main_mutex; + +public: + bool CanSeek(); + void Seek(idx_t position); + bool OnDiskFile(); + + idx_t FileSize(); + + bool FinishedReading(); + + idx_t Read(void *buffer, idx_t nr_bytes); + + string ReadLine(); + + string GetFilePath(); + + static unique_ptr OpenFileHandle(FileSystem &fs, Allocator &allocator, const string &path, + FileCompressionType compression); + static unique_ptr OpenFile(FileSystem &fs, Allocator &allocator, const string &path, + FileCompressionType compression); + +private: + unique_ptr file_handle; + string path; + bool can_seek = false; + bool on_disk_file = false; + idx_t file_size = 0; + + idx_t requested_bytes = 0; + //! If we finished reading the file + bool finished = false; +}; + +} // namespace duckdb diff --git a/src/duckdb/src/include/duckdb/execution/operator/scan/csv/csv_line_info.hpp b/src/duckdb/src/include/duckdb/execution/operator/scan/csv/csv_line_info.hpp new file mode 100644 index 000000000..13617acd7 --- /dev/null +++ b/src/duckdb/src/include/duckdb/execution/operator/scan/csv/csv_line_info.hpp @@ -0,0 +1,42 @@ +//===----------------------------------------------------------------------===// +// DuckDB +// +// duckdb/execution/operator/scan/csv/csv_line_info.hpp +// +// +//===----------------------------------------------------------------------===// + +#pragma once + +namespace duckdb { +struct LineInfo { +public: + explicit LineInfo(mutex &main_mutex_p, vector> &batch_to_tuple_end_p, + vector> &tuple_start_p, vector> &tuple_end_p) + : main_mutex(main_mutex_p), batch_to_tuple_end(batch_to_tuple_end_p), tuple_start(tuple_start_p), + tuple_end(tuple_end_p) {}; + bool CanItGetLine(idx_t file_idx, idx_t batch_idx); + + //! Return the 1-indexed line number + idx_t GetLine(idx_t batch_idx, idx_t line_error = 0, idx_t file_idx = 0, idx_t cur_start = 0, bool verify = true, + bool stop_at_first = true); + //! Verify if the CSV File was read correctly from [0,batch_idx] batches. + void Verify(idx_t file_idx, idx_t batch_idx, idx_t cur_first_pos); + //! Lines read per batch, > + vector> lines_read; + //! Set of batches that have been initialized but are not yet finished. + vector> current_batches; + //! Pointer to CSV Reader Mutex + mutex &main_mutex; + //! Pointer Batch to Tuple End + vector> &batch_to_tuple_end; + //! Pointer Batch to Tuple Start + vector> &tuple_start; + //! Pointer Batch to Tuple End + vector> &tuple_end; + //! If we already threw an exception on a previous thread. + bool done = false; + idx_t first_line = 0; +}; + +} // namespace duckdb diff --git a/src/duckdb/src/include/duckdb/execution/operator/scan/csv/csv_reader_options.hpp b/src/duckdb/src/include/duckdb/execution/operator/scan/csv/csv_reader_options.hpp new file mode 100644 index 000000000..4154def3c --- /dev/null +++ b/src/duckdb/src/include/duckdb/execution/operator/scan/csv/csv_reader_options.hpp @@ -0,0 +1,197 @@ +//===----------------------------------------------------------------------===// +// DuckDB +// +// duckdb/execution/operator/scan/csv/csv_reader_options.hpp +// +// +//===----------------------------------------------------------------------===// + +#pragma once + +#include "duckdb/execution/operator/scan/csv/csv_buffer.hpp" +#include "duckdb/common/map.hpp" +#include "duckdb/function/scalar/strftime_format.hpp" +#include "duckdb/common/types/value.hpp" +#include "duckdb/common/field_writer.hpp" +#include "duckdb/common/case_insensitive_map.hpp" +#include "duckdb/common/types.hpp" +#include "duckdb/common/multi_file_reader_options.hpp" + +namespace duckdb { + +enum class NewLineIdentifier : uint8_t { + SINGLE = 1, // Either \r or \n + CARRY_ON = 2, // \r\n + MIX = 3, // Hippie-Land, can't run it multithreaded + NOT_SET = 4 +}; + +enum class ParallelMode { AUTOMATIC = 0, PARALLEL = 1, SINGLE_THREADED = 2 }; + +//! Struct that holds the configuration of a CSV State Machine +//! Basically which char, quote and escape were used to generate it. +struct CSVStateMachineOptions { + CSVStateMachineOptions() {}; + CSVStateMachineOptions(char delimiter_p, char quote_p, char escape_p) + : delimiter(delimiter_p), quote(quote_p), escape(escape_p) {}; + + //! Delimiter to separate columns within each line + char delimiter = ','; + //! Quote used for columns that contain reserved characters, e.g ' + char quote = '\"'; + //! Escape character to escape quote character + char escape = '\0'; + + bool operator==(const CSVStateMachineOptions &other) const { + return delimiter == other.delimiter && quote == other.quote && escape == other.escape; + } + + void Serialize(FieldWriter &writer) const; + void Deserialize(FieldReader &reader); +}; + +struct DialectOptions { + CSVStateMachineOptions state_machine_options; + //! New Line separator + NewLineIdentifier new_line = NewLineIdentifier::NOT_SET; + //! Expected number of columns + idx_t num_cols = 0; + //! Whether or not the file has a header line + bool header = false; + //! The date format to use (if any is specified) + map date_format = {{LogicalTypeId::DATE, {}}, {LogicalTypeId::TIMESTAMP, {}}}; + //! Whether or not a type format is specified + map has_format = {{LogicalTypeId::DATE, false}, {LogicalTypeId::TIMESTAMP, false}}; + //! How many leading rows to skip + idx_t skip_rows = 0; + //! True start of the first CSV Buffer (After skipping empty lines, headers, notes and so on) + idx_t true_start = 0; + + void Serialize(FieldWriter &writer) const; + void Deserialize(FieldReader &reader); +}; + +struct CSVReaderOptions { + //===--------------------------------------------------------------------===// + // CommonCSVOptions + //===--------------------------------------------------------------------===// + //! See struct above. + DialectOptions dialect_options; + //! Whether or not a delimiter was defined by the user + bool has_delimiter = false; + //! Whether or not a new_line was defined by the user + bool has_newline = false; + //! Whether or not a quote was defined by the user + bool has_quote = false; + //! Whether or not an escape character was defined by the user + bool has_escape = false; + //! Whether or not a header information was given by the user + bool has_header = false; + //! Whether or not we should ignore InvalidInput errors + bool ignore_errors = false; + //! Rejects table name + string rejects_table_name; + //! Rejects table entry limit (0 = no limit) + idx_t rejects_limit = 0; + //! Columns to use as recovery key for rejected rows when reading with ignore_errors = true + vector rejects_recovery_columns; + //! Index of the recovery columns + vector rejects_recovery_column_ids; + //! Number of samples to buffer + idx_t buffer_sample_size = STANDARD_VECTOR_SIZE * 50; + //! Specifies the string that represents a null value + string null_str; + //! Whether file is compressed or not, and if so which compression type + //! AUTO_DETECT (default; infer from file extension) + FileCompressionType compression = FileCompressionType::AUTO_DETECT; + //! Option to convert quoted values to NULL values + bool allow_quoted_nulls = true; + + //===--------------------------------------------------------------------===// + // CSVAutoOptions + //===--------------------------------------------------------------------===// + //! SQL Type list mapping of name to SQL type index in sql_type_list + case_insensitive_map_t sql_types_per_column; + //! User-defined SQL type list + vector sql_type_list; + //! User-defined name list + vector name_list; + //! Types considered as candidates for auto detection ordered by descending specificity (~ from high to low) + vector auto_type_candidates = {LogicalType::VARCHAR, LogicalType::TIMESTAMP, LogicalType::DATE, + LogicalType::TIME, LogicalType::DOUBLE, LogicalType::BIGINT, + LogicalType::BOOLEAN, LogicalType::SQLNULL}; + + //===--------------------------------------------------------------------===// + // ReadCSVOptions + //===--------------------------------------------------------------------===// + //! Whether or not the skip_rows is set by the user + bool skip_rows_set = false; + //! Maximum CSV line size: specified because if we reach this amount, we likely have wrong delimiters (default: 2MB) + //! note that this is the guaranteed line length that will succeed, longer lines may be accepted if slightly above + idx_t maximum_line_size = 2097152; + //! Whether or not header names shall be normalized + bool normalize_names = false; + //! True, if column with that index must skip null check + vector force_not_null; + //! Consider all columns to be of type varchar + bool all_varchar = false; + //! Size of sample chunk used for dialect and type detection + idx_t sample_chunk_size = STANDARD_VECTOR_SIZE; + //! Number of sample chunks used for type detection + idx_t sample_chunks = 10; + //! Whether or not to automatically detect dialect and datatypes + bool auto_detect = false; + //! The file path of the CSV file to read + string file_path; + //! Multi-file reader options + MultiFileReaderOptions file_options; + //! Buffer Size (Parallel Scan) + idx_t buffer_size = CSVBuffer::CSV_BUFFER_SIZE; + //! Decimal separator when reading as numeric + string decimal_separator = "."; + //! Whether or not to pad rows that do not have enough columns with NULL values + bool null_padding = false; + + //! If we are running the parallel version of the CSV Reader. In general, the system should always auto-detect + //! When it can't execute a parallel run before execution. However, there are (rather specific) situations where + //! setting up this manually might be important + ParallelMode parallel_mode; + //===--------------------------------------------------------------------===// + // WriteCSVOptions + //===--------------------------------------------------------------------===// + //! True, if column with that index must be quoted + vector force_quote; + //! Prefix/suffix/custom newline the entire file once (enables writing of files as JSON arrays) + string prefix; + string suffix; + string write_newline; + + //! The date format to use for writing (if any is specified) + map write_date_format = {{LogicalTypeId::DATE, {}}, {LogicalTypeId::TIMESTAMP, {}}}; + + void Serialize(FieldWriter &writer) const; + void Deserialize(FieldReader &reader); + void FormatSerialize(FormatSerializer &serializer) const; + static CSVReaderOptions FormatDeserialize(FormatDeserializer &deserializer); + + void SetCompression(const string &compression); + void SetHeader(bool has_header); + void SetEscape(const string &escape); + void SetQuote(const string "e); + void SetDelimiter(const string &delimiter); + + void SetNewline(const string &input); + //! Set an option that is supported by both reading and writing functions, called by + //! the SetReadOption and SetWriteOption methods + bool SetBaseOption(const string &loption, const Value &value); + + //! loption - lowercase string + //! set - argument(s) to the option + //! expected_names - names expected if the option is "columns" + void SetReadOption(const string &loption, const Value &value, vector &expected_names); + void SetWriteOption(const string &loption, const Value &value); + void SetDateFormat(LogicalTypeId type, const string &format, bool read_format); + + string ToString() const; +}; +} // namespace duckdb diff --git a/src/duckdb/src/include/duckdb/execution/operator/scan/csv/csv_sniffer.hpp b/src/duckdb/src/include/duckdb/execution/operator/scan/csv/csv_sniffer.hpp new file mode 100644 index 000000000..8a2542ae3 --- /dev/null +++ b/src/duckdb/src/include/duckdb/execution/operator/scan/csv/csv_sniffer.hpp @@ -0,0 +1,127 @@ +//===----------------------------------------------------------------------===// +// DuckDB +// +// duckdb/execution/operator/scan/csv/csv_sniffer.hpp +// +// +//===----------------------------------------------------------------------===// + +#pragma once + +#include "duckdb/execution/operator/scan/csv/csv_state_machine.hpp" +#include "duckdb/common/vector.hpp" +#include "duckdb/execution/operator/scan/csv/quote_rules.hpp" + +namespace duckdb { +//! Struct to store the result of the Sniffer +struct SnifferResult { + SnifferResult(vector return_types_p, vector names_p) + : return_types(std::move(return_types_p)), names(std::move(names_p)) { + } + //! Return Types that were detected + vector return_types; + //! Column Names that were detected + vector names; +}; + +//! Sniffer that detects Header, Dialect and Types of CSV Files +class CSVSniffer { +public: + explicit CSVSniffer(CSVReaderOptions &options_p, shared_ptr buffer_manager_p, + CSVStateMachineCache &state_machine_cache); + + //! Main method that sniffs the CSV file, returns the types, names and options as a result + //! CSV Sniffing consists of five steps: + //! 1. Dialect Detection: Generate the CSV Options (delimiter, quote, escape, etc.) + //! 2. Type Detection: Figures out the types of the columns (For one chunk) + //! 3. Header Detection: Figures out if the CSV file has a header and produces the names of the columns + //! 4. Type Replacement: Replaces the types of the columns if the user specified them + //! 5. Type Refinement: Refines the types of the columns for the remaining chunks + SnifferResult SniffCSV(); + +private: + //! CSV State Machine Cache + CSVStateMachineCache &state_machine_cache; + //! Highest number of columns found + idx_t max_columns_found = 0; + //! Current Candidates being considered + vector> candidates; + //! Reference to original CSV Options, it will be modified as a result of the sniffer. + CSVReaderOptions &options; + //! Buffer being used on sniffer + shared_ptr buffer_manager; + + //! ------------------------------------------------------// + //! ----------------- Dialect Detection ----------------- // + //! ------------------------------------------------------// + //! First phase of auto detection: detect CSV dialect (i.e. delimiter, quote rules, etc) + void DetectDialect(); + //! Functions called in the main DetectDialect(); function + //! 1. Generates the search space candidates for the dialect + void GenerateCandidateDetectionSearchSpace(vector &delim_candidates, vector "erule_candidates, + unordered_map> "e_candidates_map, + unordered_map> &escape_candidates_map); + //! 2. Generates the search space candidates for the state machines + void GenerateStateMachineSearchSpace(vector> &csv_state_machines, + const vector &delimiter_candidates, + const vector "erule_candidates, + const unordered_map> "e_candidates_map, + const unordered_map> &escape_candidates_map); + //! 3. Analyzes if dialect candidate is a good candidate to be considered, if so, it adds it to the candidates + void AnalyzeDialectCandidate(unique_ptr, idx_t &rows_read, idx_t &best_consistent_rows, + idx_t &prev_padding_count); + //! 4. Refine Candidates over remaining chunks + void RefineCandidates(); + //! Checks if candidate still produces good values for the next chunk + bool RefineCandidateNextChunk(CSVStateMachine &candidate); + + //! ------------------------------------------------------// + //! ------------------- Type Detection ------------------ // + //! ------------------------------------------------------// + //! Second phase of auto detection: detect types, format template candidates + //! ordered by descending specificity (~ from high to low) + void DetectTypes(); + //! Change the date format for the type to the string + //! Try to cast a string value to the specified sql type + bool TryCastValue(CSVStateMachine &candidate, const Value &value, const LogicalType &sql_type); + void SetDateFormat(CSVStateMachine &candidate, const string &format_specifier, const LogicalTypeId &sql_type); + //! Functions that performs detection for date and timestamp formats + void DetectDateAndTimeStampFormats(CSVStateMachine &candidate, map &has_format_candidates, + map> &format_candidates, + const LogicalType &sql_type, const string &separator, Value &dummy_val); + + //! Variables for Type Detection + //! Format Candidates for Date and Timestamp Types + const map> format_template_candidates = { + {LogicalTypeId::DATE, {"%m-%d-%Y", "%m-%d-%y", "%d-%m-%Y", "%d-%m-%y", "%Y-%m-%d", "%y-%m-%d"}}, + {LogicalTypeId::TIMESTAMP, + {"%Y-%m-%d %H:%M:%S.%f", "%m-%d-%Y %I:%M:%S %p", "%m-%d-%y %I:%M:%S %p", "%d-%m-%Y %H:%M:%S", + "%d-%m-%y %H:%M:%S", "%Y-%m-%d %H:%M:%S", "%y-%m-%d %H:%M:%S"}}, + }; + unordered_map> best_sql_types_candidates_per_column_idx; + map> best_format_candidates; + unique_ptr best_candidate; + idx_t best_start_with_header = 0; + idx_t best_start_without_header = 0; + vector best_header_row; + + //! ------------------------------------------------------// + //! ------------------ Header Detection ----------------- // + //! ------------------------------------------------------// + void DetectHeader(); + vector names; + + //! ------------------------------------------------------// + //! ------------------ Type Replacement ----------------- // + //! ------------------------------------------------------// + void ReplaceTypes(); + + //! ------------------------------------------------------// + //! ------------------ Type Refinement ------------------ // + //! ------------------------------------------------------// + void RefineTypes(); + bool TryCastVector(Vector &parse_chunk_col, idx_t size, const LogicalType &sql_type); + vector detected_types; +}; + +} // namespace duckdb diff --git a/src/duckdb/src/include/duckdb/execution/operator/scan/csv/csv_state_machine.hpp b/src/duckdb/src/include/duckdb/execution/operator/scan/csv/csv_state_machine.hpp new file mode 100644 index 000000000..b4d82c96b --- /dev/null +++ b/src/duckdb/src/include/duckdb/execution/operator/scan/csv/csv_state_machine.hpp @@ -0,0 +1,75 @@ +//===----------------------------------------------------------------------===// +// DuckDB +// +// duckdb/execution/operator/scan/csv/csv_state_machine.hpp +// +// +//===----------------------------------------------------------------------===// + +#pragma once + +#include "duckdb/execution/operator/scan/csv/csv_reader_options.hpp" +#include "duckdb/execution/operator/scan/csv/csv_buffer_manager.hpp" +#include "duckdb/execution/operator/scan/csv/csv_state_machine_cache.hpp" + +namespace duckdb { + +//! All States of CSV Parsing +enum class CSVState : uint8_t { + STANDARD = 0, //! Regular unquoted field state + DELIMITER = 1, //! State after encountering a field separator (e.g., ;) + RECORD_SEPARATOR = 2, //! State after encountering a record separator (i.e., \n) + CARRIAGE_RETURN = 3, //! State after encountering a carriage return(i.e., \r) + QUOTED = 4, //! State when inside a quoted field + UNQUOTED = 5, //! State when leaving a quoted field + ESCAPE = 6, //! State when encountering an escape character (e.g., \) + EMPTY_LINE = 7, //! State when encountering an empty line (i.e., \r\r \n\n, \n\r) + INVALID = 8 //! Got to an Invalid State, this should error. +}; + +//! The CSV State Machine comprises a state transition array (STA). +//! The STA indicates the current state of parsing based on both the current and preceding characters. +//! This reveals whether we are dealing with a Field, a New Line, a Delimiter, and so forth. +//! The STA's creation depends on the provided quote, character, and delimiter options for that state machine. +//! The motivation behind implementing an STA is to remove branching in regular CSV Parsing by predicting and detecting +//! the states. Note: The State Machine is currently utilized solely in the CSV Sniffer. +class CSVStateMachine { +public: + explicit CSVStateMachine(CSVReaderOptions &options_p, const CSVStateMachineOptions &state_machine_options, + shared_ptr buffer_manager_p, + CSVStateMachineCache &csv_state_machine_cache_p); + //! Resets the state machine, so it can be used again + void Reset(); + + //! Aux Function for string UTF8 Verification + void VerifyUTF8(); + + CSVStateMachineCache &csv_state_machine_cache; + + const CSVReaderOptions &options; + CSVBufferIterator csv_buffer_iterator; + //! Stores identified start row for this file (e.g., a file can start with garbage like notes, before the header) + idx_t start_row = 0; + //! The Transition Array is a Finite State Machine + //! It holds the transitions of all states, on all 256 possible different characters + const state_machine_t &transition_array; + + //! Both these variables are used for new line identifier detection + bool single_record_separator = false; + bool carry_on_separator = false; + + //! Variables Used for Sniffing + CSVState state; + CSVState previous_state; + CSVState pre_previous_state; + idx_t cur_rows; + idx_t column_count; + string value; + idx_t rows_read; + idx_t line_start_pos = 0; + + //! Dialect options resulting from sniffing + DialectOptions dialect_options; +}; + +} // namespace duckdb diff --git a/src/duckdb/src/include/duckdb/execution/operator/scan/csv/csv_state_machine_cache.hpp b/src/duckdb/src/include/duckdb/execution/operator/scan/csv/csv_state_machine_cache.hpp new file mode 100644 index 000000000..8870dd4c7 --- /dev/null +++ b/src/duckdb/src/include/duckdb/execution/operator/scan/csv/csv_state_machine_cache.hpp @@ -0,0 +1,51 @@ +//===----------------------------------------------------------------------===// +// DuckDB +// +// duckdb/execution/operator/scan/csv/csv_state_machine_cache.hpp +// +// +//===----------------------------------------------------------------------===// + +#pragma once + +#include "duckdb/execution/operator/scan/csv/csv_reader_options.hpp" +#include "duckdb/execution/operator/scan/csv/csv_buffer_manager.hpp" +#include "duckdb/execution/operator/scan/csv/quote_rules.hpp" + +namespace duckdb { +static constexpr uint32_t NUM_STATES = 8; +static constexpr uint32_t NUM_TRANSITIONS = 256; +typedef uint8_t state_machine_t[NUM_STATES][NUM_TRANSITIONS]; + +//! Hash function used in out state machine cache, it hashes and combines all options used to generate a state machine +struct HashCSVStateMachineConfig { + size_t operator()(CSVStateMachineOptions const &config) const noexcept { + auto h_delimiter = Hash(config.delimiter); + auto h_quote = Hash(config.quote); + auto h_escape = Hash(config.escape); + return CombineHash(h_delimiter, CombineHash(h_quote, h_escape)); + } +}; + +//! The CSVStateMachineCache caches state machines, although small ~2kb, the actual creation of multiple State Machines +//! can become a bottleneck on sniffing, when reading very small csv files. +//! Hence the cache stores State Machines based on their different delimiter|quote|escape options. +class CSVStateMachineCache { +public: + CSVStateMachineCache(); + ~CSVStateMachineCache() {}; + //! Gets a state machine from the cache, if it's not from one the default options + //! It first caches it, then returns it. + const state_machine_t &Get(const CSVStateMachineOptions &state_machine_options); + +private: + void Insert(const CSVStateMachineOptions &state_machine_options); + //! Cache on delimiter|quote|escape + unordered_map state_machine_cache; + //! Default value for options used to intialize CSV State Machine Cache + const vector default_delimiter = {',', '|', ';', '\t'}; + const vector> default_quote = {{'\"'}, {'\"', '\''}, {'\0'}}; + const vector default_quote_rule = {QuoteRule::QUOTES_RFC, QuoteRule::QUOTES_OTHER, QuoteRule::NO_QUOTES}; + const vector> default_escape = {{'\0', '\"', '\''}, {'\\'}, {'\0'}}; +}; +} // namespace duckdb diff --git a/src/duckdb/src/include/duckdb/execution/operator/scan/csv/parallel_csv_reader.hpp b/src/duckdb/src/include/duckdb/execution/operator/scan/csv/parallel_csv_reader.hpp new file mode 100644 index 000000000..5da7b4cc6 --- /dev/null +++ b/src/duckdb/src/include/duckdb/execution/operator/scan/csv/parallel_csv_reader.hpp @@ -0,0 +1,166 @@ +//===----------------------------------------------------------------------===// +// DuckDB +// +// duckdb/execution/operator/scan/csv/parallel_csv_reader.hpp +// +// +//===----------------------------------------------------------------------===// + +#pragma once + +#include "duckdb/execution/operator/scan/csv/base_csv_reader.hpp" +#include "duckdb/execution/operator/scan/csv/csv_reader_options.hpp" +#include "duckdb/execution/operator/scan/csv/csv_file_handle.hpp" +#include "duckdb/execution/operator/scan/csv/csv_buffer.hpp" +#include "duckdb/execution/operator/scan/csv/csv_line_info.hpp" + +#include +#include + +namespace duckdb { + +struct CSVBufferRead { + CSVBufferRead(unique_ptr buffer_p, idx_t buffer_start_p, idx_t buffer_end_p, idx_t batch_index, + idx_t local_batch_index_p, optional_ptr line_info_p) + : buffer(std::move(buffer_p)), line_info(line_info_p), buffer_start(buffer_start_p), buffer_end(buffer_end_p), + batch_index(batch_index), local_batch_index(local_batch_index_p) { + D_ASSERT(buffer); + if (buffer_end > buffer->actual_size) { + buffer_end = buffer->actual_size; + } + } + + CSVBufferRead(unique_ptr buffer_p, unique_ptr nxt_buffer_p, idx_t buffer_start_p, + idx_t buffer_end_p, idx_t batch_index, idx_t local_batch_index, optional_ptr line_info_p) + : CSVBufferRead(std::move(buffer_p), buffer_start_p, buffer_end_p, batch_index, local_batch_index, + line_info_p) { + next_buffer = std::move(nxt_buffer_p); + } + + CSVBufferRead() : buffer_start(0), buffer_end(NumericLimits::Maximum()) {}; + + const char &operator[](size_t i) const { + if (i < buffer->actual_size) { + auto buffer_ptr = buffer->Ptr(); + return buffer_ptr[i]; + } + auto next_ptr = next_buffer->Ptr(); + return next_ptr[i - buffer->actual_size]; + } + + string_t GetValue(idx_t start_buffer, idx_t position_buffer, idx_t offset) { + idx_t length = position_buffer - start_buffer - offset; + // 1) It's all in the current buffer + if (start_buffer + length <= buffer->actual_size) { + auto buffer_ptr = buffer->Ptr(); + return string_t(buffer_ptr + start_buffer, length); + } else if (start_buffer >= buffer->actual_size) { + // 2) It's all in the next buffer + D_ASSERT(next_buffer); + D_ASSERT(next_buffer->actual_size >= length + (start_buffer - buffer->actual_size)); + auto buffer_ptr = next_buffer->Ptr(); + return string_t(buffer_ptr + (start_buffer - buffer->actual_size), length); + } else { + // 3) It starts in the current buffer and ends in the next buffer + D_ASSERT(next_buffer); + auto intersection = make_unsafe_uniq_array(length); + idx_t cur_pos = 0; + auto buffer_ptr = buffer->Ptr(); + for (idx_t i = start_buffer; i < buffer->actual_size; i++) { + intersection[cur_pos++] = buffer_ptr[i]; + } + idx_t nxt_buffer_pos = 0; + auto next_buffer_ptr = next_buffer->Ptr(); + for (; cur_pos < length; cur_pos++) { + intersection[cur_pos] = next_buffer_ptr[nxt_buffer_pos++]; + } + intersections.emplace_back(std::move(intersection)); + return string_t(intersections.back().get(), length); + } + } + + unique_ptr buffer; + unique_ptr next_buffer; + vector> intersections; + optional_ptr line_info; + + idx_t buffer_start; + idx_t buffer_end; + idx_t batch_index; + idx_t local_batch_index; + idx_t lines_read = 0; +}; + +struct VerificationPositions { + idx_t beginning_of_first_line = 0; + idx_t end_of_last_line = 0; +}; + +//! CSV Reader for Parallel Reading +class ParallelCSVReader : public BaseCSVReader { +public: + ParallelCSVReader(ClientContext &context, CSVReaderOptions options, unique_ptr buffer, + idx_t first_pos_first_buffer, const vector &requested_types, idx_t file_idx_p); + virtual ~ParallelCSVReader() { + } + + //! Current Position (Relative to the Buffer) + idx_t position_buffer = 0; + + //! Start of the piece of the buffer this thread should read + idx_t start_buffer = 0; + //! End of the piece of this buffer this thread should read + idx_t end_buffer = NumericLimits::Maximum(); + //! The actual buffer size + idx_t buffer_size = 0; + + //! If this flag is set, it means we are about to try to read our last row. + bool reached_remainder_state = false; + + bool finished = false; + + unique_ptr buffer; + + idx_t file_idx; + + VerificationPositions GetVerificationPositions(); + + //! Position of the first read line and last read line for verification purposes + VerificationPositions verification_positions; + +public: + void SetBufferRead(unique_ptr buffer); + //! Extract a single DataChunk from the CSV file and stores it in insert_chunk + void ParseCSV(DataChunk &insert_chunk); + + idx_t GetLineError(idx_t line_error, idx_t buffer_idx, bool stop_at_first = true) override; + +private: + //! Initialize Parser + void Initialize(const vector &requested_types); + //! Try to parse a single datachunk from the file. Throws an exception if anything goes wrong. + void ParseCSV(ParserMode mode); + //! Try to parse a single datachunk from the file. Returns whether or not the parsing is successful + bool TryParseCSV(ParserMode mode); + //! Extract a single DataChunk from the CSV file and stores it in insert_chunk + bool TryParseCSV(ParserMode mode, DataChunk &insert_chunk, string &error_message); + //! Sets Position depending on the byte_start of this thread + bool SetPosition(); + //! Called when scanning the 1st buffer, skips empty lines + void SkipEmptyLines(); + //! When a buffer finishes reading its piece, it still can try to scan up to the real end of the buffer + //! Up to finding a new line. This function sets the buffer_end and marks a boolean variable + //! when changing the buffer end the first time. + //! It returns FALSE if the parser should jump to the final state of parsing or not + bool BufferRemainder(); + + bool NewLineDelimiter(bool carry, bool carry_followed_by_nl, bool first_char); + + //! Parses a CSV file with a one-byte delimiter, escape and quote character + bool TryParseSimpleCSV(DataChunk &insert_chunk, string &error_message, bool try_add_line = false); + + //! First Position of First Buffer + idx_t first_pos_first_buffer = 0; +}; + +} // namespace duckdb diff --git a/src/duckdb/src/include/duckdb/execution/operator/scan/csv/quote_rules.hpp b/src/duckdb/src/include/duckdb/execution/operator/scan/csv/quote_rules.hpp new file mode 100644 index 000000000..4dc767170 --- /dev/null +++ b/src/duckdb/src/include/duckdb/execution/operator/scan/csv/quote_rules.hpp @@ -0,0 +1,21 @@ +//===----------------------------------------------------------------------===// +// DuckDB +// +// duckdb/execution/operator/scan/csv/quote_rules.hpp +// +// +//===----------------------------------------------------------------------===// + +#pragma once + +#include "duckdb/common/vector.hpp" + +namespace duckdb { +//! Different Rules regarding possible combinations of Quote and Escape Values for CSV Dialects. +//! Each rule has a comment on the possible combinations. +enum class QuoteRule : uint8_t { + QUOTES_RFC = 0, //! quote = " escape = (\0 || " || ') + QUOTES_OTHER = 1, //! quote = ( " || ' ) escape = '\\' + NO_QUOTES = 2 //! quote = \0 escape = \0 +}; +} // namespace duckdb diff --git a/src/duckdb/src/include/duckdb/execution/radix_partitioned_hashtable.hpp b/src/duckdb/src/include/duckdb/execution/radix_partitioned_hashtable.hpp index 6f2c1d629..04f2fbb8e 100644 --- a/src/duckdb/src/include/duckdb/execution/radix_partitioned_hashtable.hpp +++ b/src/duckdb/src/include/duckdb/execution/radix_partitioned_hashtable.hpp @@ -8,31 +8,27 @@ #pragma once +#include "duckdb/common/types/row/tuple_data_layout.hpp" #include "duckdb/execution/operator/aggregate/grouped_aggregate_data.hpp" -#include "duckdb/execution/partitionable_hashtable.hpp" -#include "duckdb/execution/physical_operator.hpp" #include "duckdb/parser/group_by_node.hpp" namespace duckdb { -class BufferManager; -class Executor; -class PhysicalHashAggregate; -class Pipeline; -class Task; + +class GroupedAggregateHashTable; +struct AggregatePartition; class RadixPartitionedHashTable { public: RadixPartitionedHashTable(GroupingSet &grouping_set, const GroupedAggregateData &op); + unique_ptr CreateHT(ClientContext &context, const idx_t capacity, + const idx_t radix_bits) const; +public: GroupingSet &grouping_set; //! The indices specified in the groups_count that do not appear in the grouping_set unsafe_vector null_groups; const GroupedAggregateData &op; - vector group_types; - //! how many groups can we have in the operator before we switch to radix partitioning - idx_t radix_limit; - //! The GROUPING values that belong to this hash table vector grouping_values; @@ -43,32 +39,27 @@ class RadixPartitionedHashTable { void Sink(ExecutionContext &context, DataChunk &chunk, OperatorSinkInput &input, DataChunk &aggregate_input_chunk, const unsafe_vector &filter) const; - void Combine(ExecutionContext &context, GlobalSinkState &state, LocalSinkState &lstate) const; - bool Finalize(ClientContext &context, GlobalSinkState &gstate_p) const; - - void ScheduleTasks(Executor &executor, const shared_ptr &event, GlobalSinkState &state, - vector> &tasks) const; + void Combine(ExecutionContext &context, GlobalSinkState &gstate, LocalSinkState &lstate) const; + void Finalize(ClientContext &context, GlobalSinkState &gstate) const; +public: //! Source interface - idx_t Size(GlobalSinkState &sink_state) const; unique_ptr GetGlobalSourceState(ClientContext &context) const; unique_ptr GetLocalSourceState(ExecutionContext &context) const; - SourceResultType GetData(ExecutionContext &context, DataChunk &chunk, GlobalSinkState &sink_state, + + SourceResultType GetData(ExecutionContext &context, DataChunk &chunk, GlobalSinkState &sink, OperatorSourceInput &input) const; - static void SetMultiScan(GlobalSinkState &state); - static bool ForceSingleHT(GlobalSinkState &state); - static bool AnyPartitioned(GlobalSinkState &state); - static void GetRepartitionInfo(ClientContext &context, GlobalSinkState &state, idx_t &repartition_radix_bits, - idx_t &concurrent_repartitions, idx_t &tasks_per_partition); + const TupleDataLayout &GetLayout() const; + idx_t Count(GlobalSinkState &sink) const; + static void SetMultiScan(GlobalSinkState &sink); private: void SetGroupingValues(); void PopulateGroupChunk(DataChunk &group_chunk, DataChunk &input_chunk) const; - void InitializeFinalizedHTs(ClientContext &context, GlobalSinkState &state) const; - void ScheduleRepartitionTasks(Executor &executor, const shared_ptr &event, GlobalSinkState &state, - vector> &tasks, const idx_t repartition_radix_bits, - const idx_t concurrent_repartitions, const idx_t tasks_per_partition) const; + idx_t CountInternal(GlobalSinkState &sink) const; + + TupleDataLayout layout; }; } // namespace duckdb diff --git a/src/duckdb/src/include/duckdb/function/function_serialization.hpp b/src/duckdb/src/include/duckdb/function/function_serialization.hpp index 542af1ea3..948c07e41 100644 --- a/src/duckdb/src/include/duckdb/function/function_serialization.hpp +++ b/src/duckdb/src/include/duckdb/function/function_serialization.hpp @@ -110,9 +110,8 @@ class FunctionSerializer { bool has_serialize = function.format_serialize; serializer.WriteProperty(503, "has_serialize", has_serialize); if (has_serialize) { - serializer.BeginObject(504, "function_data"); - function.format_serialize(serializer, bind_info, function); - serializer.EndObject(); + serializer.WriteObject(504, "function_data", + [&](FormatSerializer &obj) { function.format_serialize(obj, bind_info, function); }); D_ASSERT(function.format_deserialize); } } @@ -150,9 +149,9 @@ class FunctionSerializer { throw SerializationException("Function requires deserialization but no deserialization function for %s", function.name); } - deserializer.BeginObject(504, "function_data"); - auto result = function.format_deserialize(deserializer, function); - deserializer.EndObject(); + unique_ptr result; + deserializer.ReadObject(504, "function_data", + [&](FormatDeserializer &obj) { result = function.format_deserialize(obj, function); }); return result; } diff --git a/src/duckdb/src/include/duckdb/function/scalar/strftime_format.hpp b/src/duckdb/src/include/duckdb/function/scalar/strftime_format.hpp index ba4597c1d..ea44a4d43 100644 --- a/src/duckdb/src/include/duckdb/function/scalar/strftime_format.hpp +++ b/src/duckdb/src/include/duckdb/function/scalar/strftime_format.hpp @@ -142,10 +142,10 @@ struct StrpTimeFormat : public StrTimeFormat { public: DUCKDB_API static ParseResult Parse(const string &format, const string &text); - DUCKDB_API bool Parse(string_t str, ParseResult &result); + DUCKDB_API bool Parse(string_t str, ParseResult &result) const; - DUCKDB_API bool TryParseDate(string_t str, date_t &result, string &error_message); - DUCKDB_API bool TryParseTimestamp(string_t str, timestamp_t &result, string &error_message); + DUCKDB_API bool TryParseDate(string_t str, date_t &result, string &error_message) const; + DUCKDB_API bool TryParseTimestamp(string_t str, timestamp_t &result, string &error_message) const; date_t ParseDate(string_t str); timestamp_t ParseTimestamp(string_t str); @@ -158,7 +158,7 @@ struct StrpTimeFormat : public StrTimeFormat { DUCKDB_API void AddFormatSpecifier(string preceding_literal, StrTimeSpecifier specifier) override; int NumericSpecifierWidth(StrTimeSpecifier specifier); int32_t TryParseCollection(const char *data, idx_t &pos, idx_t size, const string_t collection[], - idx_t collection_count); + idx_t collection_count) const; private: explicit StrpTimeFormat(const string &format_string); diff --git a/src/duckdb/src/include/duckdb/function/table/read_csv.hpp b/src/duckdb/src/include/duckdb/function/table/read_csv.hpp index 380da758e..07bec45e2 100644 --- a/src/duckdb/src/include/duckdb/function/table/read_csv.hpp +++ b/src/duckdb/src/include/duckdb/function/table/read_csv.hpp @@ -8,14 +8,16 @@ #pragma once -#include "duckdb/function/table_function.hpp" -#include "duckdb/function/scalar/strftime_format.hpp" -#include "duckdb/execution/operator/persistent/csv_reader_options.hpp" -#include "duckdb/execution/operator/persistent/buffered_csv_reader.hpp" -#include "duckdb/execution/operator/persistent/parallel_csv_reader.hpp" -#include "duckdb/execution/operator/persistent/csv_file_handle.hpp" -#include "duckdb/execution/operator/persistent/csv_buffer.hpp" +#include "duckdb/execution/operator/scan/csv/buffered_csv_reader.hpp" +#include "duckdb/execution/operator/scan/csv/csv_buffer.hpp" +#include "duckdb/execution/operator/scan/csv/csv_buffer_manager.hpp" +#include "duckdb/execution/operator/scan/csv/csv_file_handle.hpp" +#include "duckdb/execution/operator/scan/csv/csv_reader_options.hpp" +#include "duckdb/execution/operator/scan/csv/parallel_csv_reader.hpp" #include "duckdb/function/built_in_functions.hpp" +#include "duckdb/function/scalar/strftime_format.hpp" +#include "duckdb/function/table_function.hpp" +#include "duckdb/execution/operator/scan/csv/csv_state_machine_cache.hpp" namespace duckdb { @@ -31,7 +33,7 @@ struct BaseCSVData : public TableFunctionData { //! The file path of the CSV file to read or write vector files; //! The CSV reader options - BufferedCSVReaderOptions options; + CSVReaderOptions options; //! Offsets for generated columns idx_t filename_col_idx; idx_t hive_partition_col_idx; @@ -50,8 +52,6 @@ struct WriteCSVData : public BaseCSVData { vector sql_types; //! The newline string to write string newline = "\n"; - //! Whether or not we are writing a simple CSV (delimiter, quote and escape are all 1 byte in length) - bool is_simple; //! The size of the CSV file (in bytes) that we buffer before we flush it to disk idx_t flush_size = 4096 * 8; //! For each byte whether or not the CSV file requires quotes when containing the byte @@ -93,8 +93,9 @@ struct ReadCSVData : public BaseCSVData { vector return_types; //! The expected SQL names to be returned from the read - including added constants (e.g. filename, hive partitions) vector return_names; - //! The initial reader (if any): this is used when automatic detection is used during binding. - //! In this case, the CSV reader is already created and might as well be re-used. + //! The buffer manager (if any): this is used when automatic detection is used during binding. + //! In this case, some CSV buffers have already been read and can be reused. + shared_ptr buffer_manager; unique_ptr initial_reader; //! The union readers are created (when csv union_by_name option is on) during binding //! Those readers can be re-used during ReadCSVFunction @@ -104,6 +105,10 @@ struct ReadCSVData : public BaseCSVData { //! Reader bind data MultiFileReaderBindData reader_bind; vector column_info; + //! The CSVStateMachineCache caches state machines created for sniffing and parsing csv files + //! We cache them because when reading very small csv files, the cost of creating all the possible + //! State machines for sniffing becomes a major bottleneck. + CSVStateMachineCache state_machine_cache; void Initialize(unique_ptr &reader) { this->initial_reader = std::move(reader); diff --git a/src/duckdb/src/include/duckdb/main/client_context_file_opener.hpp b/src/duckdb/src/include/duckdb/main/client_context_file_opener.hpp index 9f55f3260..d956825d0 100644 --- a/src/duckdb/src/include/duckdb/main/client_context_file_opener.hpp +++ b/src/duckdb/src/include/duckdb/main/client_context_file_opener.hpp @@ -21,6 +21,7 @@ class ClientContextFileOpener : public FileOpener { explicit ClientContextFileOpener(ClientContext &context_p) : context(context_p) { } + bool TryGetCurrentSetting(const string &key, Value &result, FileOpenerInfo &info) override; bool TryGetCurrentSetting(const string &key, Value &result) override; ClientContext *TryGetClientContext() override { diff --git a/src/duckdb/src/include/duckdb/main/client_data.hpp b/src/duckdb/src/include/duckdb/main/client_data.hpp index 21a89676f..4eb385784 100644 --- a/src/duckdb/src/include/duckdb/main/client_data.hpp +++ b/src/duckdb/src/include/duckdb/main/client_data.hpp @@ -13,6 +13,7 @@ #include "duckdb/common/types/value.hpp" #include "duckdb/common/case_insensitive_map.hpp" #include "duckdb/common/atomic.hpp" +#include "duckdb/execution/operator/scan/csv/csv_state_machine_cache.hpp" namespace duckdb { class AttachedDatabase; @@ -29,7 +30,7 @@ class SchemaCatalogEntry; struct RandomEngine; struct ClientData { - ClientData(ClientContext &context); + explicit ClientData(ClientContext &context); ~ClientData(); //! Query profiler diff --git a/src/duckdb/src/include/duckdb/main/config.hpp b/src/duckdb/src/include/duckdb/main/config.hpp index c9eced8e4..d54e3e35f 100644 --- a/src/duckdb/src/include/duckdb/main/config.hpp +++ b/src/duckdb/src/include/duckdb/main/config.hpp @@ -248,6 +248,7 @@ struct DBConfig { bool operator!=(const DBConfig &other); DUCKDB_API CastFunctionSet &GetCastFunctions(); + static idx_t GetSystemMaxThreads(FileSystem &fs); void SetDefaultMaxThreads(); void SetDefaultMaxMemory(); diff --git a/src/duckdb/src/include/duckdb/main/connection.hpp b/src/duckdb/src/include/duckdb/main/connection.hpp index b8cd4d14e..5df85444b 100644 --- a/src/duckdb/src/include/duckdb/main/connection.hpp +++ b/src/duckdb/src/include/duckdb/main/connection.hpp @@ -30,7 +30,7 @@ class DatabaseInstance; class DuckDB; class LogicalOperator; class SelectStatement; -struct BufferedCSVReaderOptions; +struct CSVReaderOptions; typedef void (*warning_callback)(std::string); @@ -131,7 +131,7 @@ class Connection { //! Reads CSV file DUCKDB_API shared_ptr ReadCSV(const string &csv_file); - DUCKDB_API shared_ptr ReadCSV(const string &csv_file, BufferedCSVReaderOptions &options); + DUCKDB_API shared_ptr ReadCSV(const string &csv_file, CSVReaderOptions &options); DUCKDB_API shared_ptr ReadCSV(const string &csv_file, const vector &columns); //! Reads Parquet file diff --git a/src/duckdb/src/include/duckdb/main/relation/read_csv_relation.hpp b/src/duckdb/src/include/duckdb/main/relation/read_csv_relation.hpp index f561e0a62..938f24339 100644 --- a/src/duckdb/src/include/duckdb/main/relation/read_csv_relation.hpp +++ b/src/duckdb/src/include/duckdb/main/relation/read_csv_relation.hpp @@ -8,19 +8,19 @@ #pragma once -#include "duckdb/execution/operator/persistent/csv_reader_options.hpp" +#include "duckdb/execution/operator/scan/csv/csv_reader_options.hpp" #include "duckdb/main/relation/table_function_relation.hpp" namespace duckdb { -struct BufferedCSVReaderOptions; +struct CSVReaderOptions; class ReadCSVRelation : public TableFunctionRelation { public: - ReadCSVRelation(const std::shared_ptr &context, const string &csv_file, - vector columns, string alias = string()); - ReadCSVRelation(const std::shared_ptr &context, const string &csv_file, - BufferedCSVReaderOptions options, string alias = string()); + ReadCSVRelation(const shared_ptr &context, const string &csv_file, vector columns, + string alias = string()); + ReadCSVRelation(const shared_ptr &context, const string &csv_file, CSVReaderOptions options, + string alias = string()); string alias; bool auto_detect; diff --git a/src/duckdb/src/include/duckdb/parallel/event.hpp b/src/duckdb/src/include/duckdb/parallel/event.hpp index 99ea705f6..1cfee6917 100644 --- a/src/duckdb/src/include/duckdb/parallel/event.hpp +++ b/src/duckdb/src/include/duckdb/parallel/event.hpp @@ -18,7 +18,7 @@ class Task; class Event : public std::enable_shared_from_this { public: - Event(Executor &executor); + explicit Event(Executor &executor); virtual ~Event() = default; public: @@ -52,6 +52,17 @@ class Event : public std::enable_shared_from_this { virtual void PrintPipeline() { } + template + TARGET &Cast() { + D_ASSERT(dynamic_cast(this)); + return reinterpret_cast(*this); + } + template + const TARGET &Cast() const { + D_ASSERT(dynamic_cast(this)); + return reinterpret_cast(*this); + } + protected: Executor &executor; //! The current threads working on the event diff --git a/src/duckdb/src/include/duckdb/storage/block.hpp b/src/duckdb/src/include/duckdb/storage/block.hpp index ebcd89fbe..ac839636b 100644 --- a/src/duckdb/src/include/duckdb/storage/block.hpp +++ b/src/duckdb/src/include/duckdb/storage/block.hpp @@ -14,6 +14,9 @@ namespace duckdb { +class FormatSerializer; +class FormatDeserializer; + class Block : public FileBuffer { public: Block(Allocator &allocator, block_id_t id); @@ -51,6 +54,9 @@ struct MetaBlockPointer { } block_id_t GetBlockId(); uint32_t GetBlockIndex(); + + void FormatSerialize(FormatSerializer &serializer) const; + static MetaBlockPointer FormatDeserialize(FormatDeserializer &source); }; } // namespace duckdb diff --git a/src/duckdb/src/include/duckdb/storage/buffer/block_handle.hpp b/src/duckdb/src/include/duckdb/storage/buffer/block_handle.hpp index dbd46b8b7..e797003bb 100644 --- a/src/duckdb/src/include/duckdb/storage/buffer/block_handle.hpp +++ b/src/duckdb/src/include/duckdb/storage/buffer/block_handle.hpp @@ -97,6 +97,9 @@ class BlockHandle { inline const idx_t &GetMemoryUsage() const { return memory_usage; } + bool IsUnloaded() { + return state == BlockState::BLOCK_UNLOADED; + } private: static BufferHandle Load(shared_ptr &handle, unique_ptr buffer = nullptr); diff --git a/src/duckdb/src/include/duckdb/storage/statistics/base_statistics.hpp b/src/duckdb/src/include/duckdb/storage/statistics/base_statistics.hpp index 2be977e33..6c9fef702 100644 --- a/src/duckdb/src/include/duckdb/storage/statistics/base_statistics.hpp +++ b/src/duckdb/src/include/duckdb/storage/statistics/base_statistics.hpp @@ -19,6 +19,9 @@ namespace duckdb { struct SelectionVector; +class FormatSerializer; +class FormatDeserializer; + class Serializer; class Deserializer; class FieldWriter; @@ -94,17 +97,18 @@ class BaseStatistics { void Serialize(Serializer &serializer) const; void Serialize(FieldWriter &writer) const; - - idx_t GetDistinctCount(); - static BaseStatistics Deserialize(Deserializer &source, LogicalType type); + void FormatSerialize(FormatSerializer &serializer) const; + static BaseStatistics FormatDeserialize(FormatDeserializer &deserializer); + //! Verify that a vector does not violate the statistics void Verify(Vector &vector, const SelectionVector &sel, idx_t count) const; void Verify(Vector &vector, idx_t count) const; string ToString() const; + idx_t GetDistinctCount(); static BaseStatistics FromConstant(const Value &input); private: diff --git a/src/duckdb/src/include/duckdb/storage/statistics/column_statistics.hpp b/src/duckdb/src/include/duckdb/storage/statistics/column_statistics.hpp index ca0faaa34..7f21d6b44 100644 --- a/src/duckdb/src/include/duckdb/storage/statistics/column_statistics.hpp +++ b/src/duckdb/src/include/duckdb/storage/statistics/column_statistics.hpp @@ -12,6 +12,7 @@ #include "duckdb/storage/statistics/distinct_statistics.hpp" namespace duckdb { +class FormatSerializer; class ColumnStatistics { public: @@ -35,6 +36,9 @@ class ColumnStatistics { void Serialize(Serializer &serializer) const; static shared_ptr Deserialize(Deserializer &source, const LogicalType &type); + void FormatSerialize(FormatSerializer &serializer) const; + static shared_ptr FormatDeserialize(FormatDeserializer &source); + private: BaseStatistics stats; //! The approximate count distinct stats of the column diff --git a/src/duckdb/src/include/duckdb/storage/statistics/distinct_statistics.hpp b/src/duckdb/src/include/duckdb/storage/statistics/distinct_statistics.hpp index 40798c7e3..40e624be7 100644 --- a/src/duckdb/src/include/duckdb/storage/statistics/distinct_statistics.hpp +++ b/src/duckdb/src/include/duckdb/storage/statistics/distinct_statistics.hpp @@ -16,6 +16,8 @@ namespace duckdb { class Serializer; class Deserializer; class Vector; +class FormatSerializer; +class FormatDeserializer; class DistinctStatistics { public: @@ -48,6 +50,9 @@ class DistinctStatistics { static bool TypeIsSupported(const LogicalType &type); + void FormatSerialize(FormatSerializer &serializer) const; + static unique_ptr FormatDeserialize(FormatDeserializer &deserializer); + private: //! For distinct statistics we sample the input to speed up insertions static constexpr const double SAMPLE_RATE = 0.1; diff --git a/src/duckdb/src/include/duckdb/storage/statistics/list_stats.hpp b/src/duckdb/src/include/duckdb/storage/statistics/list_stats.hpp index 0a87bc7f6..3d06c7655 100644 --- a/src/duckdb/src/include/duckdb/storage/statistics/list_stats.hpp +++ b/src/duckdb/src/include/duckdb/storage/statistics/list_stats.hpp @@ -31,6 +31,9 @@ struct ListStats { DUCKDB_API static void Serialize(const BaseStatistics &stats, FieldWriter &writer); DUCKDB_API static BaseStatistics Deserialize(FieldReader &reader, LogicalType type); + DUCKDB_API static void FormatSerialize(const BaseStatistics &stats, FormatSerializer &serializer); + DUCKDB_API static BaseStatistics FormatDeserialize(FormatDeserializer &deserializer, LogicalType type); + DUCKDB_API static string ToString(const BaseStatistics &stats); DUCKDB_API static void Merge(BaseStatistics &stats, const BaseStatistics &other); diff --git a/src/duckdb/src/include/duckdb/storage/statistics/numeric_stats.hpp b/src/duckdb/src/include/duckdb/storage/statistics/numeric_stats.hpp index 6d0eecae2..c3413864a 100644 --- a/src/duckdb/src/include/duckdb/storage/statistics/numeric_stats.hpp +++ b/src/duckdb/src/include/duckdb/storage/statistics/numeric_stats.hpp @@ -64,6 +64,9 @@ struct NumericStats { DUCKDB_API static void Serialize(const BaseStatistics &stats, FieldWriter &writer); DUCKDB_API static BaseStatistics Deserialize(FieldReader &reader, LogicalType type); + DUCKDB_API static void FormatSerialize(const BaseStatistics &stats, FormatSerializer &serializer); + DUCKDB_API static BaseStatistics FormatDeserialize(FormatDeserializer &deserializer, LogicalType type); + DUCKDB_API static string ToString(const BaseStatistics &stats); template diff --git a/src/duckdb/src/include/duckdb/storage/statistics/string_stats.hpp b/src/duckdb/src/include/duckdb/storage/statistics/string_stats.hpp index f55e04858..c5452047d 100644 --- a/src/duckdb/src/include/duckdb/storage/statistics/string_stats.hpp +++ b/src/duckdb/src/include/duckdb/storage/statistics/string_stats.hpp @@ -61,6 +61,9 @@ struct StringStats { DUCKDB_API static void Serialize(const BaseStatistics &stats, FieldWriter &writer); DUCKDB_API static BaseStatistics Deserialize(FieldReader &reader, LogicalType type); + DUCKDB_API static void FormatSerialize(const BaseStatistics &stats, FormatSerializer &serializer); + DUCKDB_API static BaseStatistics FormatDeserialize(FormatDeserializer &deserializer, LogicalType type); + DUCKDB_API static string ToString(const BaseStatistics &stats); DUCKDB_API static FilterPropagateResult CheckZonemap(const BaseStatistics &stats, ExpressionType comparison_type, diff --git a/src/duckdb/src/include/duckdb/storage/statistics/struct_stats.hpp b/src/duckdb/src/include/duckdb/storage/statistics/struct_stats.hpp index 26f870975..981ee2345 100644 --- a/src/duckdb/src/include/duckdb/storage/statistics/struct_stats.hpp +++ b/src/duckdb/src/include/duckdb/storage/statistics/struct_stats.hpp @@ -32,6 +32,9 @@ struct StructStats { DUCKDB_API static void Serialize(const BaseStatistics &stats, FieldWriter &writer); DUCKDB_API static BaseStatistics Deserialize(FieldReader &reader, LogicalType type); + DUCKDB_API static void FormatSerialize(const BaseStatistics &stats, FormatSerializer &serializer); + DUCKDB_API static BaseStatistics FormatDeserialize(FormatDeserializer &deserializer, LogicalType type); + DUCKDB_API static string ToString(const BaseStatistics &stats); DUCKDB_API static void Merge(BaseStatistics &stats, const BaseStatistics &other); diff --git a/src/duckdb/src/include/duckdb/storage/table/chunk_info.hpp b/src/duckdb/src/include/duckdb/storage/table/chunk_info.hpp index 24bfbb1d8..ae437b197 100644 --- a/src/duckdb/src/include/duckdb/storage/table/chunk_info.hpp +++ b/src/duckdb/src/include/duckdb/storage/table/chunk_info.hpp @@ -18,6 +18,9 @@ struct SelectionVector; class Transaction; struct TransactionData; +class FormatSerializer; +class FormatDeserializer; + enum class ChunkInfoType : uint8_t { CONSTANT_INFO, VECTOR_INFO, EMPTY_INFO }; class ChunkInfo { @@ -46,6 +49,9 @@ class ChunkInfo { virtual void Serialize(Serializer &serialize) = 0; static unique_ptr Deserialize(Deserializer &source); + virtual void FormatSerialize(FormatSerializer &serializer) const = 0; + static unique_ptr FormatDeserialize(FormatDeserializer &deserializer); + public: template TARGET &Cast() { @@ -85,10 +91,13 @@ class ChunkConstantInfo : public ChunkInfo { void Serialize(Serializer &serialize) override; static unique_ptr Deserialize(Deserializer &source); + void FormatSerialize(FormatSerializer &serializer) const override; + static unique_ptr FormatDeserialize(FormatDeserializer &deserializer); + private: template idx_t TemplatedGetSelVector(transaction_t start_time, transaction_t transaction_id, SelectionVector &sel_vector, - idx_t max_count); + idx_t max_count) const; }; class ChunkVectorInfo : public ChunkInfo { @@ -109,7 +118,7 @@ class ChunkVectorInfo : public ChunkInfo { public: idx_t GetSelVector(transaction_t start_time, transaction_t transaction_id, SelectionVector &sel_vector, - idx_t max_count); + idx_t max_count) const; idx_t GetSelVector(TransactionData transaction, SelectionVector &sel_vector, idx_t max_count) override; idx_t GetCommittedSelVector(transaction_t min_start_id, transaction_t min_transaction_id, SelectionVector &sel_vector, idx_t max_count) override; @@ -130,10 +139,13 @@ class ChunkVectorInfo : public ChunkInfo { void Serialize(Serializer &serialize) override; static unique_ptr Deserialize(Deserializer &source); + void FormatSerialize(FormatSerializer &serializer) const override; + static unique_ptr FormatDeserialize(FormatDeserializer &deserializer); + private: template idx_t TemplatedGetSelVector(transaction_t start_time, transaction_t transaction_id, SelectionVector &sel_vector, - idx_t max_count); + idx_t max_count) const; }; } // namespace duckdb diff --git a/src/duckdb/src/include/duckdb/storage/table/row_group.hpp b/src/duckdb/src/include/duckdb/storage/table/row_group.hpp index c9df3d359..3595a34d7 100644 --- a/src/duckdb/src/include/duckdb/storage/table/row_group.hpp +++ b/src/duckdb/src/include/duckdb/storage/table/row_group.hpp @@ -147,6 +147,10 @@ class RowGroup : public SegmentBase { void NextVector(CollectionScanState &state); + // Serialization + static void FormatSerialize(RowGroupPointer &pointer, FormatSerializer &serializer); + static RowGroupPointer FormatDeserialize(FormatDeserializer &deserializer); + private: ChunkInfo *GetChunkInfo(idx_t vector_idx); ColumnData &GetColumn(storage_t c); diff --git a/src/duckdb/src/include/duckdb/storage/table/table_statistics.hpp b/src/duckdb/src/include/duckdb/storage/table/table_statistics.hpp index d9c682f3e..248acd37d 100644 --- a/src/duckdb/src/include/duckdb/storage/table/table_statistics.hpp +++ b/src/duckdb/src/include/duckdb/storage/table/table_statistics.hpp @@ -16,6 +16,8 @@ namespace duckdb { class ColumnList; class PersistentTableData; +class FormatSerializer; +class FormatDeserializer; class TableStatisticsLock { public: @@ -50,6 +52,9 @@ class TableStatistics { void Serialize(Serializer &serializer); void Deserialize(Deserializer &source, ColumnList &columns); + void FormatSerialize(FormatSerializer &serializer); + void FormatDeserialize(FormatDeserializer &deserializer, ColumnList &columns); + private: //! The statistics lock mutex stats_lock; diff --git a/src/duckdb/src/include/duckdb/verification/deserialized_statement_verifier_v2.hpp b/src/duckdb/src/include/duckdb/verification/deserialized_statement_verifier_v2.hpp index 903b030ff..391ae9ff9 100644 --- a/src/duckdb/src/include/duckdb/verification/deserialized_statement_verifier_v2.hpp +++ b/src/duckdb/src/include/duckdb/verification/deserialized_statement_verifier_v2.hpp @@ -23,4 +23,10 @@ class DeserializedStatementVerifierV2 : public StatementVerifier { static unique_ptr Create(const SQLStatement &statement); }; +class DeserializedStatementVerifierNoDefaultV2 : public StatementVerifier { +public: + explicit DeserializedStatementVerifierNoDefaultV2(unique_ptr statement_p); + static unique_ptr Create(const SQLStatement &statement); +}; + } // namespace duckdb diff --git a/src/duckdb/src/include/duckdb/verification/statement_verifier.hpp b/src/duckdb/src/include/duckdb/verification/statement_verifier.hpp index 2a915d765..006553b2b 100644 --- a/src/duckdb/src/include/duckdb/verification/statement_verifier.hpp +++ b/src/duckdb/src/include/duckdb/verification/statement_verifier.hpp @@ -19,6 +19,7 @@ enum class VerificationType : uint8_t { COPIED, DESERIALIZED, DESERIALIZED_V2, + DESERIALIZED_V2_NO_DEFAULT, PARSED, UNOPTIMIZED, NO_OPERATOR_CACHING, diff --git a/src/duckdb/src/main/capi/logical_types-c.cpp b/src/duckdb/src/main/capi/logical_types-c.cpp index 66f8bb3bd..83906f68c 100644 --- a/src/duckdb/src/main/capi/logical_types-c.cpp +++ b/src/duckdb/src/main/capi/logical_types-c.cpp @@ -51,6 +51,28 @@ duckdb_logical_type duckdb_create_union_type(duckdb_logical_type member_types_p, return reinterpret_cast(mtype); } +duckdb_logical_type duckdb_create_struct_type(duckdb_logical_type *member_types_p, const char **member_names, + idx_t member_count) { + if (!member_types_p || !member_names) { + return nullptr; + } + duckdb::LogicalType **member_types = (duckdb::LogicalType **)member_types_p; + for (idx_t i = 0; i < member_count; i++) { + if (!member_names[i] || !member_types[i]) { + return nullptr; + } + } + + duckdb::LogicalType *mtype = new duckdb::LogicalType; + duckdb::child_list_t members; + + for (idx_t i = 0; i < member_count; i++) { + members.push_back(make_pair(member_names[i], *member_types[i])); + } + *mtype = duckdb::LogicalType::STRUCT(members); + return reinterpret_cast(mtype); +} + duckdb_logical_type duckdb_create_map_type(duckdb_logical_type key_type, duckdb_logical_type value_type) { if (!key_type || !value_type) { return nullptr; diff --git a/src/duckdb/src/main/client_context_file_opener.cpp b/src/duckdb/src/main/client_context_file_opener.cpp index 0ad9cc1b7..20de7959f 100644 --- a/src/duckdb/src/main/client_context_file_opener.cpp +++ b/src/duckdb/src/main/client_context_file_opener.cpp @@ -1,5 +1,6 @@ #include "duckdb/main/client_context_file_opener.hpp" +#include "duckdb/common/file_opener.hpp" #include "duckdb/main/client_context.hpp" namespace duckdb { @@ -8,6 +9,11 @@ bool ClientContextFileOpener::TryGetCurrentSetting(const string &key, Value &res return context.TryGetCurrentSetting(key, result); } +// LCOV_EXCL_START +bool ClientContextFileOpener::TryGetCurrentSetting(const string &key, Value &result, FileOpenerInfo &) { + return context.TryGetCurrentSetting(key, result); +} + ClientContext *FileOpener::TryGetClientContext(FileOpener *opener) { if (!opener) { return nullptr; @@ -22,4 +28,15 @@ bool FileOpener::TryGetCurrentSetting(FileOpener *opener, const string &key, Val return opener->TryGetCurrentSetting(key, result); } +bool FileOpener::TryGetCurrentSetting(FileOpener *opener, const string &key, Value &result, FileOpenerInfo &info) { + if (!opener) { + return false; + } + return opener->TryGetCurrentSetting(key, result, info); +} + +bool FileOpener::TryGetCurrentSetting(const string &key, Value &result, FileOpenerInfo &info) { + return this->TryGetCurrentSetting(key, result); +} +// LCOV_EXCL_STOP } // namespace duckdb diff --git a/src/duckdb/src/main/client_verify.cpp b/src/duckdb/src/main/client_verify.cpp index 389c15ecc..936f14b31 100644 --- a/src/duckdb/src/main/client_verify.cpp +++ b/src/duckdb/src/main/client_verify.cpp @@ -41,6 +41,7 @@ PreservedError ClientContext::VerifyQuery(ClientContextLock &lock, const string statement_verifiers.emplace_back(StatementVerifier::Create(VerificationType::COPIED, stmt)); statement_verifiers.emplace_back(StatementVerifier::Create(VerificationType::DESERIALIZED, stmt)); statement_verifiers.emplace_back(StatementVerifier::Create(VerificationType::DESERIALIZED_V2, stmt)); + statement_verifiers.emplace_back(StatementVerifier::Create(VerificationType::DESERIALIZED_V2_NO_DEFAULT, stmt)); statement_verifiers.emplace_back(StatementVerifier::Create(VerificationType::UNOPTIMIZED, stmt)); prepared_statement_verifier = StatementVerifier::Create(VerificationType::PREPARED, stmt); #ifdef DUCKDB_DEBUG_ASYNC_SINK_SOURCE diff --git a/src/duckdb/src/main/config.cpp b/src/duckdb/src/main/config.cpp index d8ad33b20..904df41d5 100644 --- a/src/duckdb/src/main/config.cpp +++ b/src/duckdb/src/main/config.cpp @@ -285,7 +285,7 @@ idx_t CGroupBandwidthQuota(idx_t physical_cores, FileSystem &fs) { } } -idx_t GetSystemMaxThreadsInternal(FileSystem &fs) { +idx_t DBConfig::GetSystemMaxThreads(FileSystem &fs) { #ifndef DUCKDB_NO_THREADS idx_t physical_cores = std::thread::hardware_concurrency(); #ifdef __linux__ @@ -301,7 +301,7 @@ idx_t GetSystemMaxThreadsInternal(FileSystem &fs) { void DBConfig::SetDefaultMaxThreads() { #ifndef DUCKDB_NO_THREADS - options.maximum_threads = GetSystemMaxThreadsInternal(*file_system); + options.maximum_threads = GetSystemMaxThreads(*file_system); #else options.maximum_threads = 1; #endif diff --git a/src/duckdb/src/main/connection.cpp b/src/duckdb/src/main/connection.cpp index 5c2f6e5be..12a222639 100644 --- a/src/duckdb/src/main/connection.cpp +++ b/src/duckdb/src/main/connection.cpp @@ -1,7 +1,7 @@ #include "duckdb/main/connection.hpp" #include "duckdb/common/types/column/column_data_collection.hpp" -#include "duckdb/execution/operator/persistent/parallel_csv_reader.hpp" +#include "duckdb/execution/operator/scan/csv/parallel_csv_reader.hpp" #include "duckdb/function/table/read_csv.hpp" #include "duckdb/main/appender.hpp" #include "duckdb/main/client_context.hpp" @@ -219,11 +219,11 @@ shared_ptr Connection::Values(const string &values, const vector Connection::ReadCSV(const string &csv_file) { - BufferedCSVReaderOptions options; + CSVReaderOptions options; return ReadCSV(csv_file, options); } -shared_ptr Connection::ReadCSV(const string &csv_file, BufferedCSVReaderOptions &options) { +shared_ptr Connection::ReadCSV(const string &csv_file, CSVReaderOptions &options) { options.file_path = csv_file; options.auto_detect = true; return make_shared(context, csv_file, options); diff --git a/src/duckdb/src/main/relation/read_csv_relation.cpp b/src/duckdb/src/main/relation/read_csv_relation.cpp index 8cda897ee..bde933bf8 100644 --- a/src/duckdb/src/main/relation/read_csv_relation.cpp +++ b/src/duckdb/src/main/relation/read_csv_relation.cpp @@ -1,18 +1,21 @@ #include "duckdb/main/relation/read_csv_relation.hpp" -#include "duckdb/parser/tableref/table_function_ref.hpp" -#include "duckdb/parser/tableref/basetableref.hpp" -#include "duckdb/parser/query_node/select_node.hpp" -#include "duckdb/parser/expression/star_expression.hpp" + +#include "duckdb/common/string_util.hpp" +#include "duckdb/execution/operator/scan/csv/buffered_csv_reader.hpp" +#include "duckdb/execution/operator/scan/csv/csv_buffer_manager.hpp" +#include "duckdb/execution/operator/scan/csv/csv_sniffer.hpp" #include "duckdb/parser/expression/columnref_expression.hpp" #include "duckdb/parser/expression/comparison_expression.hpp" #include "duckdb/parser/expression/constant_expression.hpp" #include "duckdb/parser/expression/function_expression.hpp" -#include "duckdb/common/string_util.hpp" -#include "duckdb/execution/operator/persistent/buffered_csv_reader.hpp" +#include "duckdb/parser/expression/star_expression.hpp" +#include "duckdb/parser/query_node/select_node.hpp" +#include "duckdb/parser/tableref/basetableref.hpp" +#include "duckdb/parser/tableref/table_function_ref.hpp" namespace duckdb { -ReadCSVRelation::ReadCSVRelation(const std::shared_ptr &context, const string &csv_file, +ReadCSVRelation::ReadCSVRelation(const shared_ptr &context, const string &csv_file, vector columns_p, string alias_p) : TableFunctionRelation(context, "read_csv", {Value(csv_file)}, nullptr, false), alias(std::move(alias_p)), auto_detect(false) { @@ -31,8 +34,8 @@ ReadCSVRelation::ReadCSVRelation(const std::shared_ptr &context, AddNamedParameter("columns", Value::STRUCT(std::move(column_names))); } -ReadCSVRelation::ReadCSVRelation(const std::shared_ptr &context, const string &csv_file, - BufferedCSVReaderOptions options, string alias_p) +ReadCSVRelation::ReadCSVRelation(const shared_ptr &context, const string &csv_file, + CSVReaderOptions options, string alias_p) : TableFunctionRelation(context, "read_csv_auto", {Value(csv_file)}, nullptr, false), alias(std::move(alias_p)), auto_detect(true) { @@ -42,10 +45,13 @@ ReadCSVRelation::ReadCSVRelation(const std::shared_ptr &context, // Force auto_detect for this constructor options.auto_detect = true; - BufferedCSVReader reader(*context, std::move(options)); - - auto &types = reader.GetTypes(); - auto &names = reader.GetNames(); + auto bm_file_handle = BaseCSVReader::OpenCSV(*context, options); + auto buffer_manager = make_shared(*context, std::move(bm_file_handle), options); + CSVStateMachineCache state_machine_cache; + CSVSniffer sniffer(options, buffer_manager, state_machine_cache); + auto sniffer_result = sniffer.SniffCSV(); + auto &types = sniffer_result.return_types; + auto &names = sniffer_result.names; for (idx_t i = 0; i < types.size(); i++) { columns.emplace_back(names[i], types[i]); } diff --git a/src/duckdb/src/parallel/pipeline_finish_event.cpp b/src/duckdb/src/parallel/pipeline_finish_event.cpp index 056bf2f07..a3858a2dd 100644 --- a/src/duckdb/src/parallel/pipeline_finish_event.cpp +++ b/src/duckdb/src/parallel/pipeline_finish_event.cpp @@ -51,7 +51,7 @@ class PipelineFinishTask : public ExecutorTask { //! Debugging state: number of times blocked int debug_blocked_count = 0; //! Number of times the Finalize will block before actually returning data - int debug_blocked_target_count = 1; + int debug_blocked_target_count = 10; #endif }; diff --git a/src/duckdb/src/parser/tableref/pivotref.cpp b/src/duckdb/src/parser/tableref/pivotref.cpp index a6d08177a..0dce90f1f 100644 --- a/src/duckdb/src/parser/tableref/pivotref.cpp +++ b/src/duckdb/src/parser/tableref/pivotref.cpp @@ -2,8 +2,6 @@ #include "duckdb/common/limits.hpp" #include "duckdb/common/field_writer.hpp" -#include "duckdb/common/serializer/format_serializer.hpp" -#include "duckdb/common/serializer/format_deserializer.hpp" namespace duckdb { @@ -160,12 +158,6 @@ void PivotColumnEntry::Serialize(Serializer &serializer) const { writer.Finalize(); } -void PivotColumnEntry::FormatSerialize(FormatSerializer &serializer) const { - serializer.WriteProperty(100, "values", values); - serializer.WriteOptionalProperty(101, "star_expr", star_expr); - serializer.WriteProperty(102, "alias", alias); -} - PivotColumnEntry PivotColumnEntry::Deserialize(Deserializer &source) { PivotColumnEntry result; FieldReader reader(source); @@ -176,14 +168,6 @@ PivotColumnEntry PivotColumnEntry::Deserialize(Deserializer &source) { return result; } -PivotColumnEntry PivotColumnEntry::FormatDeserialize(FormatDeserializer &source) { - PivotColumnEntry result; - source.ReadProperty(100, "values", result.values); - source.ReadOptionalProperty(101, "star_expr", result.star_expr); - source.ReadProperty(102, "alias", result.alias); - return result; -} - //===--------------------------------------------------------------------===// // PivotRef //===--------------------------------------------------------------------===// diff --git a/src/duckdb/src/planner/binder/statement/bind_copy.cpp b/src/duckdb/src/planner/binder/statement/bind_copy.cpp index fc32ffc1a..35cc798e9 100644 --- a/src/duckdb/src/planner/binder/statement/bind_copy.cpp +++ b/src/duckdb/src/planner/binder/statement/bind_copy.cpp @@ -5,7 +5,7 @@ #include "duckdb/common/bind_helpers.hpp" #include "duckdb/common/filename_pattern.hpp" #include "duckdb/common/local_file_system.hpp" -#include "duckdb/execution/operator/persistent/parallel_csv_reader.hpp" +#include "duckdb/execution/operator/scan/csv/parallel_csv_reader.hpp" #include "duckdb/function/table/read_csv.hpp" #include "duckdb/main/client_context.hpp" #include "duckdb/main/database.hpp" diff --git a/src/duckdb/src/planner/binder/statement/bind_export.cpp b/src/duckdb/src/planner/binder/statement/bind_export.cpp index eb466f2df..a69e19e0a 100644 --- a/src/duckdb/src/planner/binder/statement/bind_export.cpp +++ b/src/duckdb/src/planner/binder/statement/bind_export.cpp @@ -44,47 +44,63 @@ string SanitizeExportIdentifier(const string &str) { return result; } -bool IsExistMainKeyTable(string &table_name, vector> &unordered) { - for (idx_t i = 0; i < unordered.size(); i++) { - if (unordered[i].get().name == table_name) { +bool ReferencedTableIsOrdered(string &referenced_table, catalog_entry_vector_t &ordered) { + for (auto &entry : ordered) { + auto &table_entry = entry.get().Cast(); + if (StringUtil::CIEquals(table_entry.name, referenced_table)) { + // The referenced table is already ordered return true; } } return false; } -void ScanForeignKeyTable(vector> &ordered, vector> &unordered, - bool move_only_pk_table) { - for (auto i = unordered.begin(); i != unordered.end();) { - auto table_entry = *i; +void ScanForeignKeyTable(catalog_entry_vector_t &ordered, catalog_entry_vector_t &unordered, + bool move_primary_keys_only) { + catalog_entry_vector_t remaining; + + for (auto &entry : unordered) { + auto &table_entry = entry.get().Cast(); bool move_to_ordered = true; - auto &constraints = table_entry.get().GetConstraints(); - for (idx_t j = 0; j < constraints.size(); j++) { - auto &cond = constraints[j]; - if (cond->type == ConstraintType::FOREIGN_KEY) { - auto &fk = cond->Cast(); - if ((move_only_pk_table && fk.info.type == ForeignKeyType::FK_TYPE_FOREIGN_KEY_TABLE) || - (!move_only_pk_table && fk.info.type == ForeignKeyType::FK_TYPE_FOREIGN_KEY_TABLE && - IsExistMainKeyTable(fk.info.table, unordered))) { - move_to_ordered = false; - break; - } + auto &constraints = table_entry.GetConstraints(); + + for (auto &cond : constraints) { + if (cond->type != ConstraintType::FOREIGN_KEY) { + continue; + } + auto &fk = cond->Cast(); + if (fk.info.type != ForeignKeyType::FK_TYPE_FOREIGN_KEY_TABLE) { + continue; + } + + if (move_primary_keys_only) { + // This table references a table, don't move it yet + move_to_ordered = false; + break; + } else if (!ReferencedTableIsOrdered(fk.info.table, ordered)) { + // The table that it references isn't ordered yet + move_to_ordered = false; + break; } } + if (move_to_ordered) { ordered.push_back(table_entry); - i = unordered.erase(i); } else { - i++; + remaining.push_back(table_entry); } } + unordered = remaining; } -void ReorderTableEntries(vector> &tables) { - vector> ordered; - vector> unordered = tables; +void ReorderTableEntries(catalog_entry_vector_t &tables) { + catalog_entry_vector_t ordered; + catalog_entry_vector_t unordered = tables; + // First only move the tables that don't have any dependencies ScanForeignKeyTable(ordered, unordered, true); while (!unordered.empty()) { + // Now we will start moving tables that have foreign key constraints + // if the tables they reference are already moved ScanForeignKeyTable(ordered, unordered, false); } tables = ordered; @@ -130,7 +146,7 @@ BoundStatement Binder::Bind(ExportStatement &stmt) { // gather a list of all the tables string catalog = stmt.database.empty() ? INVALID_CATALOG : stmt.database; - vector> tables; + catalog_entry_vector_t tables; auto schemas = Catalog::GetSchemas(context, catalog); for (auto &schema : schemas) { schema.get().Scan(context, CatalogType::TABLE_ENTRY, [&](CatalogEntry &entry) { @@ -151,7 +167,7 @@ BoundStatement Binder::Bind(ExportStatement &stmt) { unordered_set table_name_index; for (auto &t : tables) { - auto &table = t.get(); + auto &table = t.get().Cast(); auto info = make_uniq(); // we copy the options supplied to the EXPORT info->format = stmt.info->format; diff --git a/src/duckdb/src/planner/expression/bound_aggregate_expression.cpp b/src/duckdb/src/planner/expression/bound_aggregate_expression.cpp index 6c5f2d192..1777e658f 100644 --- a/src/duckdb/src/planner/expression/bound_aggregate_expression.cpp +++ b/src/duckdb/src/planner/expression/bound_aggregate_expression.cpp @@ -110,8 +110,8 @@ void BoundAggregateExpression::FormatSerialize(FormatSerializer &serializer) con serializer.WriteProperty(201, "children", children); FunctionSerializer::FormatSerialize(serializer, function, bind_info.get()); serializer.WriteProperty(203, "aggregate_type", aggr_type); - serializer.WriteOptionalProperty(204, "filter", filter); - serializer.WriteOptionalProperty(205, "order_bys", order_bys); + serializer.WritePropertyWithDefault(204, "filter", filter, unique_ptr()); + serializer.WritePropertyWithDefault(205, "order_bys", order_bys, unique_ptr()); } unique_ptr BoundAggregateExpression::FormatDeserialize(FormatDeserializer &deserializer) { @@ -120,10 +120,10 @@ unique_ptr BoundAggregateExpression::FormatDeserialize(FormatDeseria auto entry = FunctionSerializer::FormatDeserialize( deserializer, CatalogType::AGGREGATE_FUNCTION_ENTRY, children); auto aggregate_type = deserializer.ReadProperty(203, "aggregate_type"); - auto filter = deserializer.ReadOptionalProperty>(204, "filter"); + auto filter = deserializer.ReadPropertyWithDefault>(204, "filter", unique_ptr()); auto result = make_uniq(std::move(entry.first), std::move(children), std::move(filter), std::move(entry.second), aggregate_type); - deserializer.ReadOptionalProperty(205, "order_bys", result->order_bys); + deserializer.ReadPropertyWithDefault(205, "order_bys", result->order_bys, unique_ptr()); return std::move(result); } diff --git a/src/duckdb/src/planner/expression/bound_window_expression.cpp b/src/duckdb/src/planner/expression/bound_window_expression.cpp index 05a04bfcb..8235fb579 100644 --- a/src/duckdb/src/planner/expression/bound_window_expression.cpp +++ b/src/duckdb/src/planner/expression/bound_window_expression.cpp @@ -172,14 +172,14 @@ void BoundWindowExpression::FormatSerialize(FormatSerializer &serializer) const } serializer.WriteProperty(202, "partitions", partitions); serializer.WriteProperty(203, "orders", orders); - serializer.WriteOptionalProperty(204, "filters", filter_expr); + serializer.WritePropertyWithDefault(204, "filters", filter_expr, unique_ptr()); serializer.WriteProperty(205, "ignore_nulls", ignore_nulls); serializer.WriteProperty(206, "start", start); serializer.WriteProperty(207, "end", end); - serializer.WriteOptionalProperty(208, "start_expr", start_expr); - serializer.WriteOptionalProperty(209, "end_expr", end_expr); - serializer.WriteOptionalProperty(210, "offset_expr", offset_expr); - serializer.WriteOptionalProperty(211, "default_expr", default_expr); + serializer.WritePropertyWithDefault(208, "start_expr", start_expr, unique_ptr()); + serializer.WritePropertyWithDefault(209, "end_expr", end_expr, unique_ptr()); + serializer.WritePropertyWithDefault(210, "offset_expr", offset_expr, unique_ptr()); + serializer.WritePropertyWithDefault(211, "default_expr", default_expr, unique_ptr()); } unique_ptr BoundWindowExpression::FormatDeserialize(FormatDeserializer &deserializer) { @@ -198,14 +198,14 @@ unique_ptr BoundWindowExpression::FormatDeserialize(FormatDeserializ make_uniq(expression_type, return_type, std::move(aggregate), std::move(bind_info)); deserializer.ReadProperty(202, "partitions", result->partitions); deserializer.ReadProperty(203, "orders", result->orders); - deserializer.ReadOptionalProperty(204, "filters", result->filter_expr); + deserializer.ReadPropertyWithDefault(204, "filters", result->filter_expr, unique_ptr()); deserializer.ReadProperty(205, "ignore_nulls", result->ignore_nulls); deserializer.ReadProperty(206, "start", result->start); deserializer.ReadProperty(207, "end", result->end); - deserializer.ReadOptionalProperty(208, "start_expr", result->start_expr); - deserializer.ReadOptionalProperty(209, "end_expr", result->end_expr); - deserializer.ReadOptionalProperty(210, "offset_expr", result->offset_expr); - deserializer.ReadOptionalProperty(211, "default_expr", result->default_expr); + deserializer.ReadPropertyWithDefault(208, "start_expr", result->start_expr, unique_ptr()); + deserializer.ReadPropertyWithDefault(209, "end_expr", result->end_expr, unique_ptr()); + deserializer.ReadPropertyWithDefault(210, "offset_expr", result->offset_expr, unique_ptr()); + deserializer.ReadPropertyWithDefault(211, "default_expr", result->default_expr, unique_ptr()); return std::move(result); } diff --git a/src/duckdb/src/planner/logical_operator.cpp b/src/duckdb/src/planner/logical_operator.cpp index 2e96f2a30..75501f715 100644 --- a/src/duckdb/src/planner/logical_operator.cpp +++ b/src/duckdb/src/planner/logical_operator.cpp @@ -146,7 +146,7 @@ void LogicalOperator::Verify(ClientContext &context) { // format (de)serialization of expressions try { - auto blob = BinarySerializer::Serialize(*expressions[expr_idx]); + auto blob = BinarySerializer::Serialize(*expressions[expr_idx], true); bound_parameter_map_t parameters; auto result = BinaryDeserializer::Deserialize(context, parameters, blob.data(), blob.size()); result->Hash(); diff --git a/src/duckdb/src/planner/planner.cpp b/src/duckdb/src/planner/planner.cpp index 2f6d05800..018b28fec 100644 --- a/src/duckdb/src/planner/planner.cpp +++ b/src/duckdb/src/planner/planner.cpp @@ -169,7 +169,7 @@ void Planner::VerifyPlan(ClientContext &context, unique_ptr &op // format (de)serialization of this operator try { - auto blob = BinarySerializer::Serialize(*op); + auto blob = BinarySerializer::Serialize(*op, true); bound_parameter_map_t parameters; auto result = BinaryDeserializer::Deserialize(context, parameters, blob.data(), blob.size()); } catch (SerializationException &ex) { diff --git a/src/duckdb/src/storage/checkpoint_manager.cpp b/src/duckdb/src/storage/checkpoint_manager.cpp index 38d77a57e..c8fd8c992 100644 --- a/src/duckdb/src/storage/checkpoint_manager.cpp +++ b/src/duckdb/src/storage/checkpoint_manager.cpp @@ -34,7 +34,7 @@ namespace duckdb { -void ReorderTableEntries(vector> &tables); +void ReorderTableEntries(catalog_entry_vector_t &tables); SingleFileCheckpointWriter::SingleFileCheckpointWriter(AttachedDatabase &db, BlockManager &block_manager) : CheckpointWriter(db), partial_block_manager(block_manager, CheckpointType::FULL_CHECKPOINT) { @@ -157,7 +157,7 @@ void CheckpointWriter::WriteSchema(SchemaCatalogEntry &schema) { // write the schema data schema.Serialize(GetMetadataWriter()); // then, we fetch the tables/views/sequences information - vector> tables; + catalog_entry_vector_t tables; vector> views; schema.Scan(CatalogType::TABLE_ENTRY, [&](CatalogEntry &entry) { if (entry.internal) { @@ -235,7 +235,8 @@ void CheckpointWriter::WriteSchema(SchemaCatalogEntry &schema) { // reorder tables because of foreign key constraint ReorderTableEntries(tables); // Write the tables - for (auto &table : tables) { + for (auto &entry : tables) { + auto &table = entry.get().Cast(); WriteTable(table); } // Write the views diff --git a/src/duckdb/src/storage/serialization/serialize_constraint.cpp b/src/duckdb/src/storage/serialization/serialize_constraint.cpp index 20a878ed9..6013b2cb1 100644 --- a/src/duckdb/src/storage/serialization/serialize_constraint.cpp +++ b/src/duckdb/src/storage/serialization/serialize_constraint.cpp @@ -37,7 +37,7 @@ unique_ptr Constraint::FormatDeserialize(FormatDeserializer &deseria void CheckConstraint::FormatSerialize(FormatSerializer &serializer) const { Constraint::FormatSerialize(serializer); - serializer.WriteProperty(200, "expression", *expression); + serializer.WriteProperty(200, "expression", expression); } unique_ptr CheckConstraint::FormatDeserialize(FormatDeserializer &deserializer) { diff --git a/src/duckdb/src/storage/serialization/serialize_create_info.cpp b/src/duckdb/src/storage/serialization/serialize_create_info.cpp index 62859cb79..d7c1e9697 100644 --- a/src/duckdb/src/storage/serialization/serialize_create_info.cpp +++ b/src/duckdb/src/storage/serialization/serialize_create_info.cpp @@ -102,7 +102,7 @@ unique_ptr CreateIndexInfo::FormatDeserialize(FormatDeserializer &de void CreateMacroInfo::FormatSerialize(FormatSerializer &serializer) const { CreateInfo::FormatSerialize(serializer); serializer.WriteProperty(200, "name", name); - serializer.WriteProperty(201, "function", *function); + serializer.WriteProperty(201, "function", function); } unique_ptr CreateMacroInfo::FormatDeserialize(FormatDeserializer &deserializer) { @@ -149,7 +149,7 @@ void CreateTableInfo::FormatSerialize(FormatSerializer &serializer) const { serializer.WriteProperty(200, "table", table); serializer.WriteProperty(201, "columns", columns); serializer.WriteProperty(202, "constraints", constraints); - serializer.WriteOptionalProperty(203, "query", query); + serializer.WritePropertyWithDefault(203, "query", query, unique_ptr()); } unique_ptr CreateTableInfo::FormatDeserialize(FormatDeserializer &deserializer) { @@ -157,7 +157,7 @@ unique_ptr CreateTableInfo::FormatDeserialize(FormatDeserializer &de deserializer.ReadProperty(200, "table", result->table); deserializer.ReadProperty(201, "columns", result->columns); deserializer.ReadProperty(202, "constraints", result->constraints); - deserializer.ReadOptionalProperty(203, "query", result->query); + deserializer.ReadPropertyWithDefault(203, "query", result->query, unique_ptr()); return std::move(result); } @@ -179,7 +179,7 @@ void CreateViewInfo::FormatSerialize(FormatSerializer &serializer) const { serializer.WriteProperty(200, "view_name", view_name); serializer.WriteProperty(201, "aliases", aliases); serializer.WriteProperty(202, "types", types); - serializer.WriteOptionalProperty(203, "query", query); + serializer.WritePropertyWithDefault(203, "query", query, unique_ptr()); } unique_ptr CreateViewInfo::FormatDeserialize(FormatDeserializer &deserializer) { @@ -187,7 +187,7 @@ unique_ptr CreateViewInfo::FormatDeserialize(FormatDeserializer &des deserializer.ReadProperty(200, "view_name", result->view_name); deserializer.ReadProperty(201, "aliases", result->aliases); deserializer.ReadProperty(202, "types", result->types); - deserializer.ReadOptionalProperty(203, "query", result->query); + deserializer.ReadPropertyWithDefault(203, "query", result->query, unique_ptr()); return std::move(result); } diff --git a/src/duckdb/src/storage/serialization/serialize_expression.cpp b/src/duckdb/src/storage/serialization/serialize_expression.cpp index c28f11d44..cb8c72863 100644 --- a/src/duckdb/src/storage/serialization/serialize_expression.cpp +++ b/src/duckdb/src/storage/serialization/serialize_expression.cpp @@ -83,9 +83,9 @@ unique_ptr Expression::FormatDeserialize(FormatDeserializer &deseria void BoundBetweenExpression::FormatSerialize(FormatSerializer &serializer) const { Expression::FormatSerialize(serializer); - serializer.WriteProperty(200, "input", *input); - serializer.WriteProperty(201, "lower", *lower); - serializer.WriteProperty(202, "upper", *upper); + serializer.WriteProperty(200, "input", input); + serializer.WriteProperty(201, "lower", lower); + serializer.WriteProperty(202, "upper", upper); serializer.WriteProperty(203, "lower_inclusive", lower_inclusive); serializer.WriteProperty(204, "upper_inclusive", upper_inclusive); } @@ -104,7 +104,7 @@ void BoundCaseExpression::FormatSerialize(FormatSerializer &serializer) const { Expression::FormatSerialize(serializer); serializer.WriteProperty(200, "return_type", return_type); serializer.WriteProperty(201, "case_checks", case_checks); - serializer.WriteProperty(202, "else_expr", *else_expr); + serializer.WriteProperty(202, "else_expr", else_expr); } unique_ptr BoundCaseExpression::FormatDeserialize(FormatDeserializer &deserializer) { @@ -117,7 +117,7 @@ unique_ptr BoundCaseExpression::FormatDeserialize(FormatDeserializer void BoundCastExpression::FormatSerialize(FormatSerializer &serializer) const { Expression::FormatSerialize(serializer); - serializer.WriteProperty(200, "child", *child); + serializer.WriteProperty(200, "child", child); serializer.WriteProperty(201, "return_type", return_type); serializer.WriteProperty(202, "try_cast", try_cast); } @@ -147,8 +147,8 @@ unique_ptr BoundColumnRefExpression::FormatDeserialize(FormatDeseria void BoundComparisonExpression::FormatSerialize(FormatSerializer &serializer) const { Expression::FormatSerialize(serializer); - serializer.WriteProperty(200, "left", *left); - serializer.WriteProperty(201, "right", *right); + serializer.WriteProperty(200, "left", left); + serializer.WriteProperty(201, "right", right); } unique_ptr BoundComparisonExpression::FormatDeserialize(FormatDeserializer &deserializer) { @@ -194,7 +194,7 @@ unique_ptr BoundDefaultExpression::FormatDeserialize(FormatDeseriali void BoundLambdaExpression::FormatSerialize(FormatSerializer &serializer) const { Expression::FormatSerialize(serializer); serializer.WriteProperty(200, "return_type", return_type); - serializer.WriteProperty(201, "lambda_expr", *lambda_expr); + serializer.WriteProperty(201, "lambda_expr", lambda_expr); serializer.WriteProperty(202, "captures", captures); serializer.WriteProperty(203, "parameter_count", parameter_count); } @@ -243,7 +243,7 @@ void BoundParameterExpression::FormatSerialize(FormatSerializer &serializer) con Expression::FormatSerialize(serializer); serializer.WriteProperty(200, "identifier", identifier); serializer.WriteProperty(201, "return_type", return_type); - serializer.WriteProperty(202, "parameter_data", *parameter_data); + serializer.WriteProperty(202, "parameter_data", parameter_data); } unique_ptr BoundParameterExpression::FormatDeserialize(FormatDeserializer &deserializer) { @@ -270,7 +270,7 @@ unique_ptr BoundReferenceExpression::FormatDeserialize(FormatDeseria void BoundUnnestExpression::FormatSerialize(FormatSerializer &serializer) const { Expression::FormatSerialize(serializer); serializer.WriteProperty(200, "return_type", return_type); - serializer.WriteProperty(201, "child", *child); + serializer.WriteProperty(201, "child", child); } unique_ptr BoundUnnestExpression::FormatDeserialize(FormatDeserializer &deserializer) { diff --git a/src/duckdb/src/storage/serialization/serialize_logical_operator.cpp b/src/duckdb/src/storage/serialization/serialize_logical_operator.cpp index ea0bcd2b5..4ca8bc918 100644 --- a/src/duckdb/src/storage/serialization/serialize_logical_operator.cpp +++ b/src/duckdb/src/storage/serialization/serialize_logical_operator.cpp @@ -218,7 +218,7 @@ void LogicalAnyJoin::FormatSerialize(FormatSerializer &serializer) const { serializer.WriteProperty(201, "mark_index", mark_index); serializer.WriteProperty(202, "left_projection_map", left_projection_map); serializer.WriteProperty(203, "right_projection_map", right_projection_map); - serializer.WriteProperty(204, "condition", *condition); + serializer.WriteProperty(204, "condition", condition); } unique_ptr LogicalAnyJoin::FormatDeserialize(FormatDeserializer &deserializer) { @@ -254,7 +254,7 @@ void LogicalColumnDataGet::FormatSerialize(FormatSerializer &serializer) const { LogicalOperator::FormatSerialize(serializer); serializer.WriteProperty(200, "table_index", table_index); serializer.WriteProperty(201, "chunk_types", chunk_types); - serializer.WriteProperty(202, "collection", *collection); + serializer.WriteProperty(202, "collection", collection); } unique_ptr LogicalColumnDataGet::FormatDeserialize(FormatDeserializer &deserializer) { @@ -290,7 +290,7 @@ unique_ptr LogicalComparisonJoin::FormatDeserialize(FormatDeser void LogicalCreate::FormatSerialize(FormatSerializer &serializer) const { LogicalOperator::FormatSerialize(serializer); - serializer.WriteProperty(200, "info", *info); + serializer.WriteProperty(200, "info", info); } unique_ptr LogicalCreate::FormatDeserialize(FormatDeserializer &deserializer) { @@ -301,7 +301,7 @@ unique_ptr LogicalCreate::FormatDeserialize(FormatDeserializer void LogicalCreateIndex::FormatSerialize(FormatSerializer &serializer) const { LogicalOperator::FormatSerialize(serializer); - serializer.WriteProperty(200, "info", *info); + serializer.WriteProperty(200, "info", info); serializer.WriteProperty(201, "unbound_expressions", unbound_expressions); } @@ -316,7 +316,7 @@ void LogicalCreateTable::FormatSerialize(FormatSerializer &serializer) const { LogicalOperator::FormatSerialize(serializer); serializer.WriteProperty(200, "catalog", schema.ParentCatalog().GetName()); serializer.WriteProperty(201, "schema", schema.name); - serializer.WriteProperty(202, "info", *info->base); + serializer.WriteProperty(202, "info", info->base); } unique_ptr LogicalCreateTable::FormatDeserialize(FormatDeserializer &deserializer) { @@ -374,14 +374,14 @@ void LogicalDistinct::FormatSerialize(FormatSerializer &serializer) const { LogicalOperator::FormatSerialize(serializer); serializer.WriteProperty(200, "distinct_type", distinct_type); serializer.WriteProperty(201, "distinct_targets", distinct_targets); - serializer.WriteOptionalProperty(202, "order_by", order_by); + serializer.WritePropertyWithDefault(202, "order_by", order_by, unique_ptr()); } unique_ptr LogicalDistinct::FormatDeserialize(FormatDeserializer &deserializer) { auto distinct_type = deserializer.ReadProperty(200, "distinct_type"); auto distinct_targets = deserializer.ReadProperty>>(201, "distinct_targets"); auto result = duckdb::unique_ptr(new LogicalDistinct(std::move(distinct_targets), distinct_type)); - deserializer.ReadOptionalProperty(202, "order_by", result->order_by); + deserializer.ReadPropertyWithDefault(202, "order_by", result->order_by, unique_ptr()); return std::move(result); } @@ -468,8 +468,8 @@ void LogicalInsert::FormatSerialize(FormatSerializer &serializer) const { serializer.WriteProperty(209, "action_type", action_type); serializer.WriteProperty(210, "expected_set_types", expected_set_types); serializer.WriteProperty(211, "on_conflict_filter", on_conflict_filter); - serializer.WriteOptionalProperty(212, "on_conflict_condition", on_conflict_condition); - serializer.WriteOptionalProperty(213, "do_update_condition", do_update_condition); + serializer.WritePropertyWithDefault(212, "on_conflict_condition", on_conflict_condition, unique_ptr()); + serializer.WritePropertyWithDefault(213, "do_update_condition", do_update_condition, unique_ptr()); serializer.WriteProperty(214, "set_columns", set_columns); serializer.WriteProperty(215, "set_types", set_types); serializer.WriteProperty(216, "excluded_table_index", excluded_table_index); @@ -491,8 +491,8 @@ unique_ptr LogicalInsert::FormatDeserialize(FormatDeserializer deserializer.ReadProperty(209, "action_type", result->action_type); deserializer.ReadProperty(210, "expected_set_types", result->expected_set_types); deserializer.ReadProperty(211, "on_conflict_filter", result->on_conflict_filter); - deserializer.ReadOptionalProperty(212, "on_conflict_condition", result->on_conflict_condition); - deserializer.ReadOptionalProperty(213, "do_update_condition", result->do_update_condition); + deserializer.ReadPropertyWithDefault(212, "on_conflict_condition", result->on_conflict_condition, unique_ptr()); + deserializer.ReadPropertyWithDefault(213, "do_update_condition", result->do_update_condition, unique_ptr()); deserializer.ReadProperty(214, "set_columns", result->set_columns); deserializer.ReadProperty(215, "set_types", result->set_types); deserializer.ReadProperty(216, "excluded_table_index", result->excluded_table_index); @@ -505,15 +505,15 @@ void LogicalLimit::FormatSerialize(FormatSerializer &serializer) const { LogicalOperator::FormatSerialize(serializer); serializer.WriteProperty(200, "limit_val", limit_val); serializer.WriteProperty(201, "offset_val", offset_val); - serializer.WriteOptionalProperty(202, "limit", limit); - serializer.WriteOptionalProperty(203, "offset", offset); + serializer.WritePropertyWithDefault(202, "limit", limit, unique_ptr()); + serializer.WritePropertyWithDefault(203, "offset", offset, unique_ptr()); } unique_ptr LogicalLimit::FormatDeserialize(FormatDeserializer &deserializer) { auto limit_val = deserializer.ReadProperty(200, "limit_val"); auto offset_val = deserializer.ReadProperty(201, "offset_val"); - auto limit = deserializer.ReadOptionalProperty>(202, "limit"); - auto offset = deserializer.ReadOptionalProperty>(203, "offset"); + auto limit = deserializer.ReadPropertyWithDefault>(202, "limit", unique_ptr()); + auto offset = deserializer.ReadPropertyWithDefault>(203, "offset", unique_ptr()); auto result = duckdb::unique_ptr(new LogicalLimit(limit_val, offset_val, std::move(limit), std::move(offset))); return std::move(result); } @@ -522,15 +522,15 @@ void LogicalLimitPercent::FormatSerialize(FormatSerializer &serializer) const { LogicalOperator::FormatSerialize(serializer); serializer.WriteProperty(200, "limit_percent", limit_percent); serializer.WriteProperty(201, "offset_val", offset_val); - serializer.WriteOptionalProperty(202, "limit", limit); - serializer.WriteOptionalProperty(203, "offset", offset); + serializer.WritePropertyWithDefault(202, "limit", limit, unique_ptr()); + serializer.WritePropertyWithDefault(203, "offset", offset, unique_ptr()); } unique_ptr LogicalLimitPercent::FormatDeserialize(FormatDeserializer &deserializer) { auto limit_percent = deserializer.ReadProperty(200, "limit_percent"); auto offset_val = deserializer.ReadProperty(201, "offset_val"); - auto limit = deserializer.ReadOptionalProperty>(202, "limit"); - auto offset = deserializer.ReadOptionalProperty>(203, "offset"); + auto limit = deserializer.ReadPropertyWithDefault>(202, "limit", unique_ptr()); + auto offset = deserializer.ReadPropertyWithDefault>(203, "offset", unique_ptr()); auto result = duckdb::unique_ptr(new LogicalLimitPercent(limit_percent, offset_val, std::move(limit), std::move(offset))); return std::move(result); } @@ -682,7 +682,7 @@ unique_ptr LogicalShow::FormatDeserialize(FormatDeserializer &d void LogicalSimple::FormatSerialize(FormatSerializer &serializer) const { LogicalOperator::FormatSerialize(serializer); - serializer.WriteProperty(200, "info", *info); + serializer.WriteProperty(200, "info", info); } unique_ptr LogicalSimple::FormatDeserialize(FormatDeserializer &deserializer) { diff --git a/src/duckdb/src/storage/serialization/serialize_macro_function.cpp b/src/duckdb/src/storage/serialization/serialize_macro_function.cpp index 01bcce712..adad524c5 100644 --- a/src/duckdb/src/storage/serialization/serialize_macro_function.cpp +++ b/src/duckdb/src/storage/serialization/serialize_macro_function.cpp @@ -39,7 +39,7 @@ unique_ptr MacroFunction::FormatDeserialize(FormatDeserializer &d void ScalarMacroFunction::FormatSerialize(FormatSerializer &serializer) const { MacroFunction::FormatSerialize(serializer); - serializer.WriteProperty(200, "expression", *expression); + serializer.WriteProperty(200, "expression", expression); } unique_ptr ScalarMacroFunction::FormatDeserialize(FormatDeserializer &deserializer) { @@ -50,7 +50,7 @@ unique_ptr ScalarMacroFunction::FormatDeserialize(FormatDeseriali void TableMacroFunction::FormatSerialize(FormatSerializer &serializer) const { MacroFunction::FormatSerialize(serializer); - serializer.WriteProperty(200, "query_node", *query_node); + serializer.WriteProperty(200, "query_node", query_node); } unique_ptr TableMacroFunction::FormatDeserialize(FormatDeserializer &deserializer) { diff --git a/src/duckdb/src/storage/serialization/serialize_nodes.cpp b/src/duckdb/src/storage/serialization/serialize_nodes.cpp index f51be9e30..885461848 100644 --- a/src/duckdb/src/storage/serialization/serialize_nodes.cpp +++ b/src/duckdb/src/storage/serialization/serialize_nodes.cpp @@ -25,15 +25,16 @@ #include "duckdb/planner/table_filter.hpp" #include "duckdb/common/multi_file_reader_options.hpp" #include "duckdb/common/multi_file_reader.hpp" -#include "duckdb/execution/operator/persistent/csv_reader_options.hpp" +#include "duckdb/execution/operator/scan/csv/csv_reader_options.hpp" #include "duckdb/function/scalar/strftime_format.hpp" #include "duckdb/function/table/read_csv.hpp" +#include "duckdb/common/types/interval.hpp" namespace duckdb { void BoundCaseCheck::FormatSerialize(FormatSerializer &serializer) const { - serializer.WriteProperty(100, "when_expr", *when_expr); - serializer.WriteProperty(101, "then_expr", *then_expr); + serializer.WriteProperty(100, "when_expr", when_expr); + serializer.WriteProperty(101, "then_expr", then_expr); } BoundCaseCheck BoundCaseCheck::FormatDeserialize(FormatDeserializer &deserializer) { @@ -46,7 +47,7 @@ BoundCaseCheck BoundCaseCheck::FormatDeserialize(FormatDeserializer &deserialize void BoundOrderByNode::FormatSerialize(FormatSerializer &serializer) const { serializer.WriteProperty(100, "type", type); serializer.WriteProperty(101, "null_order", null_order); - serializer.WriteProperty(102, "expression", *expression); + serializer.WriteProperty(102, "expression", expression); } BoundOrderByNode BoundOrderByNode::FormatDeserialize(FormatDeserializer &deserializer) { @@ -85,89 +86,89 @@ BoundPivotInfo BoundPivotInfo::FormatDeserialize(FormatDeserializer &deserialize return result; } -void BufferedCSVReaderOptions::FormatSerialize(FormatSerializer &serializer) const { +void CSVReaderOptions::FormatSerialize(FormatSerializer &serializer) const { serializer.WriteProperty(100, "has_delimiter", has_delimiter); - serializer.WriteProperty(101, "delimiter", delimiter); - serializer.WriteProperty(102, "has_quote", has_quote); - serializer.WriteProperty(103, "quote", quote); - serializer.WriteProperty(104, "has_escape", has_escape); - serializer.WriteProperty(105, "escape", escape); - serializer.WriteProperty(106, "has_header", has_header); - serializer.WriteProperty(107, "header", header); - serializer.WriteProperty(108, "ignore_errors", ignore_errors); - serializer.WriteProperty(109, "num_cols", num_cols); - serializer.WriteProperty(110, "buffer_sample_size", buffer_sample_size); - serializer.WriteProperty(111, "null_str", null_str); - serializer.WriteProperty(112, "compression", compression); - serializer.WriteProperty(113, "new_line", new_line); - serializer.WriteProperty(114, "allow_quoted_nulls", allow_quoted_nulls); - serializer.WriteProperty(115, "skip_rows", skip_rows); - serializer.WriteProperty(116, "skip_rows_set", skip_rows_set); - serializer.WriteProperty(117, "maximum_line_size", maximum_line_size); - serializer.WriteProperty(118, "normalize_names", normalize_names); - serializer.WriteProperty(119, "force_not_null", force_not_null); - serializer.WriteProperty(120, "all_varchar", all_varchar); - serializer.WriteProperty(121, "sample_chunk_size", sample_chunk_size); - serializer.WriteProperty(122, "sample_chunks", sample_chunks); - serializer.WriteProperty(123, "auto_detect", auto_detect); - serializer.WriteProperty(124, "file_path", file_path); - serializer.WriteProperty(125, "decimal_separator", decimal_separator); - serializer.WriteProperty(126, "null_padding", null_padding); - serializer.WriteProperty(127, "buffer_size", buffer_size); - serializer.WriteProperty(128, "file_options", file_options); - serializer.WriteProperty(129, "force_quote", force_quote); - serializer.WriteProperty(130, "date_format", date_format); - serializer.WriteProperty(131, "has_format", has_format); - serializer.WriteProperty(132, "rejects_table_name", rejects_table_name); - serializer.WriteProperty(133, "rejects_limit", rejects_limit); - serializer.WriteProperty(134, "rejects_recovery_columns", rejects_recovery_columns); - serializer.WriteProperty(135, "rejects_recovery_column_ids", rejects_recovery_column_ids); -} - -BufferedCSVReaderOptions BufferedCSVReaderOptions::FormatDeserialize(FormatDeserializer &deserializer) { - BufferedCSVReaderOptions result; + serializer.WriteProperty(101, "has_quote", has_quote); + serializer.WriteProperty(102, "has_escape", has_escape); + serializer.WriteProperty(103, "has_header", has_header); + serializer.WriteProperty(104, "ignore_errors", ignore_errors); + serializer.WriteProperty(105, "buffer_sample_size", buffer_sample_size); + serializer.WriteProperty(106, "null_str", null_str); + serializer.WriteProperty(107, "compression", compression); + serializer.WriteProperty(108, "allow_quoted_nulls", allow_quoted_nulls); + serializer.WriteProperty(109, "skip_rows_set", skip_rows_set); + serializer.WriteProperty(110, "maximum_line_size", maximum_line_size); + serializer.WriteProperty(111, "normalize_names", normalize_names); + serializer.WriteProperty(112, "force_not_null", force_not_null); + serializer.WriteProperty(113, "all_varchar", all_varchar); + serializer.WriteProperty(114, "sample_chunk_size", sample_chunk_size); + serializer.WriteProperty(115, "sample_chunks", sample_chunks); + serializer.WriteProperty(116, "auto_detect", auto_detect); + serializer.WriteProperty(117, "file_path", file_path); + serializer.WriteProperty(118, "decimal_separator", decimal_separator); + serializer.WriteProperty(119, "null_padding", null_padding); + serializer.WriteProperty(120, "buffer_size", buffer_size); + serializer.WriteProperty(121, "file_options", file_options); + serializer.WriteProperty(122, "force_quote", force_quote); + serializer.WriteProperty(123, "rejects_table_name", rejects_table_name); + serializer.WriteProperty(124, "rejects_limit", rejects_limit); + serializer.WriteProperty(125, "rejects_recovery_columns", rejects_recovery_columns); + serializer.WriteProperty(126, "rejects_recovery_column_ids", rejects_recovery_column_ids); + serializer.WriteProperty(127, "dialect_options.state_machine_options.delimiter", dialect_options.state_machine_options.delimiter); + serializer.WriteProperty(128, "dialect_options.state_machine_options.quote", dialect_options.state_machine_options.quote); + serializer.WriteProperty(129, "dialect_options.state_machine_options.escape", dialect_options.state_machine_options.escape); + serializer.WriteProperty(130, "dialect_options.header", dialect_options.header); + serializer.WriteProperty(131, "dialect_options.num_cols", dialect_options.num_cols); + serializer.WriteProperty(132, "dialect_options.new_line", dialect_options.new_line); + serializer.WriteProperty(133, "dialect_options.skip_rows", dialect_options.skip_rows); + serializer.WriteProperty(134, "dialect_options.date_format", dialect_options.date_format); + serializer.WriteProperty(135, "dialect_options.has_format", dialect_options.has_format); +} + +CSVReaderOptions CSVReaderOptions::FormatDeserialize(FormatDeserializer &deserializer) { + CSVReaderOptions result; deserializer.ReadProperty(100, "has_delimiter", result.has_delimiter); - deserializer.ReadProperty(101, "delimiter", result.delimiter); - deserializer.ReadProperty(102, "has_quote", result.has_quote); - deserializer.ReadProperty(103, "quote", result.quote); - deserializer.ReadProperty(104, "has_escape", result.has_escape); - deserializer.ReadProperty(105, "escape", result.escape); - deserializer.ReadProperty(106, "has_header", result.has_header); - deserializer.ReadProperty(107, "header", result.header); - deserializer.ReadProperty(108, "ignore_errors", result.ignore_errors); - deserializer.ReadProperty(109, "num_cols", result.num_cols); - deserializer.ReadProperty(110, "buffer_sample_size", result.buffer_sample_size); - deserializer.ReadProperty(111, "null_str", result.null_str); - deserializer.ReadProperty(112, "compression", result.compression); - deserializer.ReadProperty(113, "new_line", result.new_line); - deserializer.ReadProperty(114, "allow_quoted_nulls", result.allow_quoted_nulls); - deserializer.ReadProperty(115, "skip_rows", result.skip_rows); - deserializer.ReadProperty(116, "skip_rows_set", result.skip_rows_set); - deserializer.ReadProperty(117, "maximum_line_size", result.maximum_line_size); - deserializer.ReadProperty(118, "normalize_names", result.normalize_names); - deserializer.ReadProperty(119, "force_not_null", result.force_not_null); - deserializer.ReadProperty(120, "all_varchar", result.all_varchar); - deserializer.ReadProperty(121, "sample_chunk_size", result.sample_chunk_size); - deserializer.ReadProperty(122, "sample_chunks", result.sample_chunks); - deserializer.ReadProperty(123, "auto_detect", result.auto_detect); - deserializer.ReadProperty(124, "file_path", result.file_path); - deserializer.ReadProperty(125, "decimal_separator", result.decimal_separator); - deserializer.ReadProperty(126, "null_padding", result.null_padding); - deserializer.ReadProperty(127, "buffer_size", result.buffer_size); - deserializer.ReadProperty(128, "file_options", result.file_options); - deserializer.ReadProperty(129, "force_quote", result.force_quote); - deserializer.ReadProperty(130, "date_format", result.date_format); - deserializer.ReadProperty(131, "has_format", result.has_format); - deserializer.ReadProperty(132, "rejects_table_name", result.rejects_table_name); - deserializer.ReadProperty(133, "rejects_limit", result.rejects_limit); - deserializer.ReadProperty(134, "rejects_recovery_columns", result.rejects_recovery_columns); - deserializer.ReadProperty(135, "rejects_recovery_column_ids", result.rejects_recovery_column_ids); + deserializer.ReadProperty(101, "has_quote", result.has_quote); + deserializer.ReadProperty(102, "has_escape", result.has_escape); + deserializer.ReadProperty(103, "has_header", result.has_header); + deserializer.ReadProperty(104, "ignore_errors", result.ignore_errors); + deserializer.ReadProperty(105, "buffer_sample_size", result.buffer_sample_size); + deserializer.ReadProperty(106, "null_str", result.null_str); + deserializer.ReadProperty(107, "compression", result.compression); + deserializer.ReadProperty(108, "allow_quoted_nulls", result.allow_quoted_nulls); + deserializer.ReadProperty(109, "skip_rows_set", result.skip_rows_set); + deserializer.ReadProperty(110, "maximum_line_size", result.maximum_line_size); + deserializer.ReadProperty(111, "normalize_names", result.normalize_names); + deserializer.ReadProperty(112, "force_not_null", result.force_not_null); + deserializer.ReadProperty(113, "all_varchar", result.all_varchar); + deserializer.ReadProperty(114, "sample_chunk_size", result.sample_chunk_size); + deserializer.ReadProperty(115, "sample_chunks", result.sample_chunks); + deserializer.ReadProperty(116, "auto_detect", result.auto_detect); + deserializer.ReadProperty(117, "file_path", result.file_path); + deserializer.ReadProperty(118, "decimal_separator", result.decimal_separator); + deserializer.ReadProperty(119, "null_padding", result.null_padding); + deserializer.ReadProperty(120, "buffer_size", result.buffer_size); + deserializer.ReadProperty(121, "file_options", result.file_options); + deserializer.ReadProperty(122, "force_quote", result.force_quote); + deserializer.ReadProperty(123, "rejects_table_name", result.rejects_table_name); + deserializer.ReadProperty(124, "rejects_limit", result.rejects_limit); + deserializer.ReadProperty(125, "rejects_recovery_columns", result.rejects_recovery_columns); + deserializer.ReadProperty(126, "rejects_recovery_column_ids", result.rejects_recovery_column_ids); + deserializer.ReadProperty(127, "dialect_options.state_machine_options.delimiter", result.dialect_options.state_machine_options.delimiter); + deserializer.ReadProperty(128, "dialect_options.state_machine_options.quote", result.dialect_options.state_machine_options.quote); + deserializer.ReadProperty(129, "dialect_options.state_machine_options.escape", result.dialect_options.state_machine_options.escape); + deserializer.ReadProperty(130, "dialect_options.header", result.dialect_options.header); + deserializer.ReadProperty(131, "dialect_options.num_cols", result.dialect_options.num_cols); + deserializer.ReadProperty(132, "dialect_options.new_line", result.dialect_options.new_line); + deserializer.ReadProperty(133, "dialect_options.skip_rows", result.dialect_options.skip_rows); + deserializer.ReadProperty(134, "dialect_options.date_format", result.dialect_options.date_format); + deserializer.ReadProperty(135, "dialect_options.has_format", result.dialect_options.has_format); return result; } void CaseCheck::FormatSerialize(FormatSerializer &serializer) const { - serializer.WriteProperty(100, "when_expr", *when_expr); - serializer.WriteProperty(101, "then_expr", *then_expr); + serializer.WriteProperty(100, "when_expr", when_expr); + serializer.WriteProperty(101, "then_expr", then_expr); } CaseCheck CaseCheck::FormatDeserialize(FormatDeserializer &deserializer) { @@ -192,7 +193,7 @@ ColumnBinding ColumnBinding::FormatDeserialize(FormatDeserializer &deserializer) void ColumnDefinition::FormatSerialize(FormatSerializer &serializer) const { serializer.WriteProperty(100, "name", name); serializer.WriteProperty(101, "type", type); - serializer.WriteOptionalProperty(102, "expression", expression); + serializer.WritePropertyWithDefault(102, "expression", expression, unique_ptr()); serializer.WriteProperty(103, "category", category); serializer.WriteProperty(104, "compression_type", compression_type); } @@ -200,7 +201,7 @@ void ColumnDefinition::FormatSerialize(FormatSerializer &serializer) const { ColumnDefinition ColumnDefinition::FormatDeserialize(FormatDeserializer &deserializer) { auto name = deserializer.ReadProperty(100, "name"); auto type = deserializer.ReadProperty(101, "type"); - auto expression = deserializer.ReadOptionalProperty>(102, "expression"); + auto expression = deserializer.ReadPropertyWithDefault>(102, "expression", unique_ptr()); auto category = deserializer.ReadProperty(103, "category"); ColumnDefinition result(std::move(name), std::move(type), std::move(expression), category); deserializer.ReadProperty(104, "compression_type", result.compression_type); @@ -231,7 +232,7 @@ ColumnList ColumnList::FormatDeserialize(FormatDeserializer &deserializer) { void CommonTableExpressionInfo::FormatSerialize(FormatSerializer &serializer) const { serializer.WriteProperty(100, "aliases", aliases); - serializer.WriteProperty(101, "query", *query); + serializer.WriteProperty(101, "query", query); serializer.WriteProperty(102, "materialized", materialized); } @@ -266,8 +267,8 @@ HivePartitioningIndex HivePartitioningIndex::FormatDeserialize(FormatDeserialize } void JoinCondition::FormatSerialize(FormatSerializer &serializer) const { - serializer.WriteProperty(100, "left", *left); - serializer.WriteProperty(101, "right", *right); + serializer.WriteProperty(100, "left", left); + serializer.WriteProperty(101, "right", right); serializer.WriteProperty(102, "comparison", comparison); } @@ -281,12 +282,12 @@ JoinCondition JoinCondition::FormatDeserialize(FormatDeserializer &deserializer) void LogicalType::FormatSerialize(FormatSerializer &serializer) const { serializer.WriteProperty(100, "id", id_); - serializer.WriteOptionalProperty(101, "type_info", type_info_); + serializer.WritePropertyWithDefault(101, "type_info", type_info_, shared_ptr()); } LogicalType LogicalType::FormatDeserialize(FormatDeserializer &deserializer) { auto id = deserializer.ReadProperty(100, "id"); - auto type_info = deserializer.ReadOptionalProperty>(101, "type_info"); + auto type_info = deserializer.ReadPropertyWithDefault>(101, "type_info", shared_ptr()); LogicalType result(id, std::move(type_info)); return result; } @@ -326,7 +327,7 @@ MultiFileReaderOptions MultiFileReaderOptions::FormatDeserialize(FormatDeseriali void OrderByNode::FormatSerialize(FormatSerializer &serializer) const { serializer.WriteProperty(100, "type", type); serializer.WriteProperty(101, "null_order", null_order); - serializer.WriteProperty(102, "expression", *expression); + serializer.WriteProperty(102, "expression", expression); } OrderByNode OrderByNode::FormatDeserialize(FormatDeserializer &deserializer) { @@ -353,6 +354,20 @@ PivotColumn PivotColumn::FormatDeserialize(FormatDeserializer &deserializer) { return result; } +void PivotColumnEntry::FormatSerialize(FormatSerializer &serializer) const { + serializer.WriteProperty(100, "values", values); + serializer.WritePropertyWithDefault(101, "star_expr", star_expr, unique_ptr()); + serializer.WriteProperty(102, "alias", alias); +} + +PivotColumnEntry PivotColumnEntry::FormatDeserialize(FormatDeserializer &deserializer) { + PivotColumnEntry result; + deserializer.ReadProperty(100, "values", result.values); + deserializer.ReadPropertyWithDefault(101, "star_expr", result.star_expr, unique_ptr()); + deserializer.ReadProperty(102, "alias", result.alias); + return result; +} + void ReadCSVData::FormatSerialize(FormatSerializer &serializer) const { serializer.WriteProperty(100, "files", files); serializer.WriteProperty(101, "csv_types", csv_types); @@ -429,4 +444,18 @@ VacuumOptions VacuumOptions::FormatDeserialize(FormatDeserializer &deserializer) return result; } +void interval_t::FormatSerialize(FormatSerializer &serializer) const { + serializer.WriteProperty(1, "months", months); + serializer.WriteProperty(2, "days", days); + serializer.WriteProperty(3, "micros", micros); +} + +interval_t interval_t::FormatDeserialize(FormatDeserializer &deserializer) { + interval_t result; + deserializer.ReadProperty(1, "months", result.months); + deserializer.ReadProperty(2, "days", result.days); + deserializer.ReadProperty(3, "micros", result.micros); + return result; +} + } // namespace duckdb diff --git a/src/duckdb/src/storage/serialization/serialize_parse_info.cpp b/src/duckdb/src/storage/serialization/serialize_parse_info.cpp index 97e269432..9322dd19b 100644 --- a/src/duckdb/src/storage/serialization/serialize_parse_info.cpp +++ b/src/duckdb/src/storage/serialization/serialize_parse_info.cpp @@ -209,7 +209,7 @@ void ChangeColumnTypeInfo::FormatSerialize(FormatSerializer &serializer) const { AlterTableInfo::FormatSerialize(serializer); serializer.WriteProperty(400, "column_name", column_name); serializer.WriteProperty(401, "target_type", target_type); - serializer.WriteProperty(402, "expression", *expression); + serializer.WriteProperty(402, "expression", expression); } unique_ptr ChangeColumnTypeInfo::FormatDeserialize(FormatDeserializer &deserializer) { @@ -375,13 +375,13 @@ unique_ptr RenameViewInfo::FormatDeserialize(FormatDeserializer & void SetDefaultInfo::FormatSerialize(FormatSerializer &serializer) const { AlterTableInfo::FormatSerialize(serializer); serializer.WriteProperty(400, "column_name", column_name); - serializer.WriteOptionalProperty(401, "expression", expression); + serializer.WritePropertyWithDefault(401, "expression", expression, unique_ptr()); } unique_ptr SetDefaultInfo::FormatDeserialize(FormatDeserializer &deserializer) { auto result = duckdb::unique_ptr(new SetDefaultInfo()); deserializer.ReadProperty(400, "column_name", result->column_name); - deserializer.ReadOptionalProperty(401, "expression", result->expression); + deserializer.ReadPropertyWithDefault(401, "expression", result->expression, unique_ptr()); return std::move(result); } diff --git a/src/duckdb/src/storage/serialization/serialize_parsed_expression.cpp b/src/duckdb/src/storage/serialization/serialize_parsed_expression.cpp index 1f943e583..8e92a7ffb 100644 --- a/src/duckdb/src/storage/serialization/serialize_parsed_expression.cpp +++ b/src/duckdb/src/storage/serialization/serialize_parsed_expression.cpp @@ -83,9 +83,9 @@ unique_ptr ParsedExpression::FormatDeserialize(FormatDeseriali void BetweenExpression::FormatSerialize(FormatSerializer &serializer) const { ParsedExpression::FormatSerialize(serializer); - serializer.WriteProperty(200, "input", *input); - serializer.WriteProperty(201, "lower", *lower); - serializer.WriteProperty(202, "upper", *upper); + serializer.WriteProperty(200, "input", input); + serializer.WriteProperty(201, "lower", lower); + serializer.WriteProperty(202, "upper", upper); } unique_ptr BetweenExpression::FormatDeserialize(FormatDeserializer &deserializer) { @@ -99,7 +99,7 @@ unique_ptr BetweenExpression::FormatDeserialize(FormatDeserial void CaseExpression::FormatSerialize(FormatSerializer &serializer) const { ParsedExpression::FormatSerialize(serializer); serializer.WriteProperty(200, "case_checks", case_checks); - serializer.WriteProperty(201, "else_expr", *else_expr); + serializer.WriteProperty(201, "else_expr", else_expr); } unique_ptr CaseExpression::FormatDeserialize(FormatDeserializer &deserializer) { @@ -111,7 +111,7 @@ unique_ptr CaseExpression::FormatDeserialize(FormatDeserialize void CastExpression::FormatSerialize(FormatSerializer &serializer) const { ParsedExpression::FormatSerialize(serializer); - serializer.WriteProperty(200, "child", *child); + serializer.WriteProperty(200, "child", child); serializer.WriteProperty(201, "cast_type", cast_type); serializer.WriteProperty(202, "try_cast", try_cast); } @@ -126,7 +126,7 @@ unique_ptr CastExpression::FormatDeserialize(FormatDeserialize void CollateExpression::FormatSerialize(FormatSerializer &serializer) const { ParsedExpression::FormatSerialize(serializer); - serializer.WriteProperty(200, "child", *child); + serializer.WriteProperty(200, "child", child); serializer.WriteProperty(201, "collation", collation); } @@ -150,8 +150,8 @@ unique_ptr ColumnRefExpression::FormatDeserialize(FormatDeseri void ComparisonExpression::FormatSerialize(FormatSerializer &serializer) const { ParsedExpression::FormatSerialize(serializer); - serializer.WriteProperty(200, "left", *left); - serializer.WriteProperty(201, "right", *right); + serializer.WriteProperty(200, "left", left); + serializer.WriteProperty(201, "right", right); } unique_ptr ComparisonExpression::FormatDeserialize(FormatDeserializer &deserializer) { @@ -197,8 +197,8 @@ void FunctionExpression::FormatSerialize(FormatSerializer &serializer) const { serializer.WriteProperty(200, "function_name", function_name); serializer.WriteProperty(201, "schema", schema); serializer.WriteProperty(202, "children", children); - serializer.WriteOptionalProperty(203, "filter", filter); - serializer.WriteProperty(204, "order_bys", (ResultModifier &)*order_bys); + serializer.WritePropertyWithDefault(203, "filter", filter, unique_ptr()); + serializer.WriteProperty(204, "order_bys", (ResultModifier *)order_bys.get()); serializer.WriteProperty(205, "distinct", distinct); serializer.WriteProperty(206, "is_operator", is_operator); serializer.WriteProperty(207, "export_state", export_state); @@ -210,7 +210,7 @@ unique_ptr FunctionExpression::FormatDeserialize(FormatDeseria deserializer.ReadProperty(200, "function_name", result->function_name); deserializer.ReadProperty(201, "schema", result->schema); deserializer.ReadProperty(202, "children", result->children); - deserializer.ReadOptionalProperty(203, "filter", result->filter); + deserializer.ReadPropertyWithDefault(203, "filter", result->filter, unique_ptr()); auto order_bys = deserializer.ReadProperty>(204, "order_bys"); result->order_bys = unique_ptr_cast(std::move(order_bys)); deserializer.ReadProperty(205, "distinct", result->distinct); @@ -222,8 +222,8 @@ unique_ptr FunctionExpression::FormatDeserialize(FormatDeseria void LambdaExpression::FormatSerialize(FormatSerializer &serializer) const { ParsedExpression::FormatSerialize(serializer); - serializer.WriteProperty(200, "lhs", *lhs); - serializer.WriteProperty(201, "expr", *expr); + serializer.WriteProperty(200, "lhs", lhs); + serializer.WriteProperty(201, "expr", expr); } unique_ptr LambdaExpression::FormatDeserialize(FormatDeserializer &deserializer) { @@ -272,7 +272,7 @@ void StarExpression::FormatSerialize(FormatSerializer &serializer) const { serializer.WriteProperty(201, "exclude_list", exclude_list); serializer.WriteProperty(202, "replace_list", replace_list); serializer.WriteProperty(203, "columns", columns); - serializer.WriteOptionalProperty(204, "expr", expr); + serializer.WritePropertyWithDefault(204, "expr", expr, unique_ptr()); } unique_ptr StarExpression::FormatDeserialize(FormatDeserializer &deserializer) { @@ -281,7 +281,7 @@ unique_ptr StarExpression::FormatDeserialize(FormatDeserialize deserializer.ReadProperty(201, "exclude_list", result->exclude_list); deserializer.ReadProperty(202, "replace_list", result->replace_list); deserializer.ReadProperty(203, "columns", result->columns); - deserializer.ReadOptionalProperty(204, "expr", result->expr); + deserializer.ReadPropertyWithDefault(204, "expr", result->expr, unique_ptr()); return std::move(result); } @@ -289,7 +289,7 @@ void SubqueryExpression::FormatSerialize(FormatSerializer &serializer) const { ParsedExpression::FormatSerialize(serializer); serializer.WriteProperty(200, "subquery_type", subquery_type); serializer.WriteProperty(201, "subquery", subquery); - serializer.WriteOptionalProperty(202, "child", child); + serializer.WritePropertyWithDefault(202, "child", child, unique_ptr()); serializer.WriteProperty(203, "comparison_type", comparison_type); } @@ -297,7 +297,7 @@ unique_ptr SubqueryExpression::FormatDeserialize(FormatDeseria auto result = duckdb::unique_ptr(new SubqueryExpression()); deserializer.ReadProperty(200, "subquery_type", result->subquery_type); deserializer.ReadProperty(201, "subquery", result->subquery); - deserializer.ReadOptionalProperty(202, "child", result->child); + deserializer.ReadPropertyWithDefault(202, "child", result->child, unique_ptr()); deserializer.ReadProperty(203, "comparison_type", result->comparison_type); return std::move(result); } @@ -312,12 +312,12 @@ void WindowExpression::FormatSerialize(FormatSerializer &serializer) const { serializer.WriteProperty(205, "orders", orders); serializer.WriteProperty(206, "start", start); serializer.WriteProperty(207, "end", end); - serializer.WriteOptionalProperty(208, "start_expr", start_expr); - serializer.WriteOptionalProperty(209, "end_expr", end_expr); - serializer.WriteOptionalProperty(210, "offset_expr", offset_expr); - serializer.WriteOptionalProperty(211, "default_expr", default_expr); + serializer.WritePropertyWithDefault(208, "start_expr", start_expr, unique_ptr()); + serializer.WritePropertyWithDefault(209, "end_expr", end_expr, unique_ptr()); + serializer.WritePropertyWithDefault(210, "offset_expr", offset_expr, unique_ptr()); + serializer.WritePropertyWithDefault(211, "default_expr", default_expr, unique_ptr()); serializer.WriteProperty(212, "ignore_nulls", ignore_nulls); - serializer.WriteOptionalProperty(213, "filter_expr", filter_expr); + serializer.WritePropertyWithDefault(213, "filter_expr", filter_expr, unique_ptr()); } unique_ptr WindowExpression::FormatDeserialize(FormatDeserializer &deserializer) { @@ -330,12 +330,12 @@ unique_ptr WindowExpression::FormatDeserialize(FormatDeseriali deserializer.ReadProperty(205, "orders", result->orders); deserializer.ReadProperty(206, "start", result->start); deserializer.ReadProperty(207, "end", result->end); - deserializer.ReadOptionalProperty(208, "start_expr", result->start_expr); - deserializer.ReadOptionalProperty(209, "end_expr", result->end_expr); - deserializer.ReadOptionalProperty(210, "offset_expr", result->offset_expr); - deserializer.ReadOptionalProperty(211, "default_expr", result->default_expr); + deserializer.ReadPropertyWithDefault(208, "start_expr", result->start_expr, unique_ptr()); + deserializer.ReadPropertyWithDefault(209, "end_expr", result->end_expr, unique_ptr()); + deserializer.ReadPropertyWithDefault(210, "offset_expr", result->offset_expr, unique_ptr()); + deserializer.ReadPropertyWithDefault(211, "default_expr", result->default_expr, unique_ptr()); deserializer.ReadProperty(212, "ignore_nulls", result->ignore_nulls); - deserializer.ReadOptionalProperty(213, "filter_expr", result->filter_expr); + deserializer.ReadPropertyWithDefault(213, "filter_expr", result->filter_expr, unique_ptr()); return std::move(result); } diff --git a/src/duckdb/src/storage/serialization/serialize_query_node.cpp b/src/duckdb/src/storage/serialization/serialize_query_node.cpp index cfe0707d0..745000d30 100644 --- a/src/duckdb/src/storage/serialization/serialize_query_node.cpp +++ b/src/duckdb/src/storage/serialization/serialize_query_node.cpp @@ -44,8 +44,8 @@ unique_ptr QueryNode::FormatDeserialize(FormatDeserializer &deseriali void CTENode::FormatSerialize(FormatSerializer &serializer) const { QueryNode::FormatSerialize(serializer); serializer.WriteProperty(200, "cte_name", ctename); - serializer.WriteProperty(201, "query", *query); - serializer.WriteProperty(202, "child", *child); + serializer.WriteProperty(201, "query", query); + serializer.WriteProperty(202, "child", child); serializer.WriteProperty(203, "aliases", aliases); } @@ -62,8 +62,8 @@ void RecursiveCTENode::FormatSerialize(FormatSerializer &serializer) const { QueryNode::FormatSerialize(serializer); serializer.WriteProperty(200, "cte_name", ctename); serializer.WriteProperty(201, "union_all", union_all); - serializer.WriteProperty(202, "left", *left); - serializer.WriteProperty(203, "right", *right); + serializer.WriteProperty(202, "left", left); + serializer.WriteProperty(203, "right", right); serializer.WriteProperty(204, "aliases", aliases); } @@ -80,35 +80,35 @@ unique_ptr RecursiveCTENode::FormatDeserialize(FormatDeserializer &de void SelectNode::FormatSerialize(FormatSerializer &serializer) const { QueryNode::FormatSerialize(serializer); serializer.WriteProperty(200, "select_list", select_list); - serializer.WriteOptionalProperty(201, "from_table", from_table); - serializer.WriteOptionalProperty(202, "where_clause", where_clause); + serializer.WritePropertyWithDefault(201, "from_table", from_table, unique_ptr()); + serializer.WritePropertyWithDefault(202, "where_clause", where_clause, unique_ptr()); serializer.WriteProperty(203, "group_expressions", groups.group_expressions); serializer.WriteProperty(204, "group_sets", groups.grouping_sets); serializer.WriteProperty(205, "aggregate_handling", aggregate_handling); - serializer.WriteOptionalProperty(206, "having", having); - serializer.WriteOptionalProperty(207, "sample", sample); - serializer.WriteOptionalProperty(208, "qualify", qualify); + serializer.WritePropertyWithDefault(206, "having", having, unique_ptr()); + serializer.WritePropertyWithDefault(207, "sample", sample, unique_ptr()); + serializer.WritePropertyWithDefault(208, "qualify", qualify, unique_ptr()); } unique_ptr SelectNode::FormatDeserialize(FormatDeserializer &deserializer) { auto result = duckdb::unique_ptr(new SelectNode()); deserializer.ReadProperty(200, "select_list", result->select_list); - deserializer.ReadOptionalProperty(201, "from_table", result->from_table); - deserializer.ReadOptionalProperty(202, "where_clause", result->where_clause); + deserializer.ReadPropertyWithDefault(201, "from_table", result->from_table, unique_ptr()); + deserializer.ReadPropertyWithDefault(202, "where_clause", result->where_clause, unique_ptr()); deserializer.ReadProperty(203, "group_expressions", result->groups.group_expressions); deserializer.ReadProperty(204, "group_sets", result->groups.grouping_sets); deserializer.ReadProperty(205, "aggregate_handling", result->aggregate_handling); - deserializer.ReadOptionalProperty(206, "having", result->having); - deserializer.ReadOptionalProperty(207, "sample", result->sample); - deserializer.ReadOptionalProperty(208, "qualify", result->qualify); + deserializer.ReadPropertyWithDefault(206, "having", result->having, unique_ptr()); + deserializer.ReadPropertyWithDefault(207, "sample", result->sample, unique_ptr()); + deserializer.ReadPropertyWithDefault(208, "qualify", result->qualify, unique_ptr()); return std::move(result); } void SetOperationNode::FormatSerialize(FormatSerializer &serializer) const { QueryNode::FormatSerialize(serializer); serializer.WriteProperty(200, "setop_type", setop_type); - serializer.WriteProperty(201, "left", *left); - serializer.WriteProperty(202, "right", *right); + serializer.WriteProperty(201, "left", left); + serializer.WriteProperty(202, "right", right); } unique_ptr SetOperationNode::FormatDeserialize(FormatDeserializer &deserializer) { diff --git a/src/duckdb/src/storage/serialization/serialize_result_modifier.cpp b/src/duckdb/src/storage/serialization/serialize_result_modifier.cpp index 5da474884..9b53faa4f 100644 --- a/src/duckdb/src/storage/serialization/serialize_result_modifier.cpp +++ b/src/duckdb/src/storage/serialization/serialize_result_modifier.cpp @@ -59,27 +59,27 @@ unique_ptr DistinctModifier::FormatDeserialize(FormatDeserialize void LimitModifier::FormatSerialize(FormatSerializer &serializer) const { ResultModifier::FormatSerialize(serializer); - serializer.WriteOptionalProperty(200, "limit", limit); - serializer.WriteOptionalProperty(201, "offset", offset); + serializer.WritePropertyWithDefault(200, "limit", limit, unique_ptr()); + serializer.WritePropertyWithDefault(201, "offset", offset, unique_ptr()); } unique_ptr LimitModifier::FormatDeserialize(FormatDeserializer &deserializer) { auto result = duckdb::unique_ptr(new LimitModifier()); - deserializer.ReadOptionalProperty(200, "limit", result->limit); - deserializer.ReadOptionalProperty(201, "offset", result->offset); + deserializer.ReadPropertyWithDefault(200, "limit", result->limit, unique_ptr()); + deserializer.ReadPropertyWithDefault(201, "offset", result->offset, unique_ptr()); return std::move(result); } void LimitPercentModifier::FormatSerialize(FormatSerializer &serializer) const { ResultModifier::FormatSerialize(serializer); - serializer.WriteOptionalProperty(200, "limit", limit); - serializer.WriteOptionalProperty(201, "offset", offset); + serializer.WritePropertyWithDefault(200, "limit", limit, unique_ptr()); + serializer.WritePropertyWithDefault(201, "offset", offset, unique_ptr()); } unique_ptr LimitPercentModifier::FormatDeserialize(FormatDeserializer &deserializer) { auto result = duckdb::unique_ptr(new LimitPercentModifier()); - deserializer.ReadOptionalProperty(200, "limit", result->limit); - deserializer.ReadOptionalProperty(201, "offset", result->offset); + deserializer.ReadPropertyWithDefault(200, "limit", result->limit, unique_ptr()); + deserializer.ReadPropertyWithDefault(201, "offset", result->offset, unique_ptr()); return std::move(result); } diff --git a/src/duckdb/src/storage/serialization/serialize_statement.cpp b/src/duckdb/src/storage/serialization/serialize_statement.cpp index 48f1d24d3..713b5eef9 100644 --- a/src/duckdb/src/storage/serialization/serialize_statement.cpp +++ b/src/duckdb/src/storage/serialization/serialize_statement.cpp @@ -10,7 +10,7 @@ namespace duckdb { void SelectStatement::FormatSerialize(FormatSerializer &serializer) const { - serializer.WriteProperty(100, "node", *node); + serializer.WriteProperty(100, "node", node); } unique_ptr SelectStatement::FormatDeserialize(FormatDeserializer &deserializer) { diff --git a/src/duckdb/src/storage/serialization/serialize_storage.cpp b/src/duckdb/src/storage/serialization/serialize_storage.cpp new file mode 100644 index 000000000..6bb2b4749 --- /dev/null +++ b/src/duckdb/src/storage/serialization/serialize_storage.cpp @@ -0,0 +1,39 @@ +//===----------------------------------------------------------------------===// +// This file is automatically generated by scripts/generate_serialization.py +// Do not edit this file manually, your changes will be overwritten +//===----------------------------------------------------------------------===// + +#include "duckdb/common/serializer/format_serializer.hpp" +#include "duckdb/common/serializer/format_deserializer.hpp" +#include "duckdb/storage/block.hpp" +#include "duckdb/storage/statistics/distinct_statistics.hpp" + +namespace duckdb { + +void DistinctStatistics::FormatSerialize(FormatSerializer &serializer) const { + serializer.WriteProperty(100, "sample_count", sample_count); + serializer.WriteProperty(101, "total_count", total_count); + serializer.WriteProperty(102, "log", log); +} + +unique_ptr DistinctStatistics::FormatDeserialize(FormatDeserializer &deserializer) { + auto sample_count = deserializer.ReadProperty(100, "sample_count"); + auto total_count = deserializer.ReadProperty(101, "total_count"); + auto log = deserializer.ReadProperty>(102, "log"); + auto result = duckdb::unique_ptr(new DistinctStatistics(std::move(log), sample_count, total_count)); + return result; +} + +void MetaBlockPointer::FormatSerialize(FormatSerializer &serializer) const { + serializer.WriteProperty(100, "block_pointer", block_pointer); + serializer.WriteProperty(101, "offset", offset); +} + +MetaBlockPointer MetaBlockPointer::FormatDeserialize(FormatDeserializer &deserializer) { + auto block_pointer = deserializer.ReadProperty(100, "block_pointer"); + auto offset = deserializer.ReadProperty(101, "offset"); + MetaBlockPointer result(block_pointer, offset); + return result; +} + +} // namespace duckdb diff --git a/src/duckdb/src/storage/serialization/serialize_tableref.cpp b/src/duckdb/src/storage/serialization/serialize_tableref.cpp index 8cbd98673..917c07f51 100644 --- a/src/duckdb/src/storage/serialization/serialize_tableref.cpp +++ b/src/duckdb/src/storage/serialization/serialize_tableref.cpp @@ -12,13 +12,13 @@ namespace duckdb { void TableRef::FormatSerialize(FormatSerializer &serializer) const { serializer.WriteProperty(100, "type", type); serializer.WriteProperty(101, "alias", alias); - serializer.WriteOptionalProperty(102, "sample", sample); + serializer.WritePropertyWithDefault(102, "sample", sample, unique_ptr()); } unique_ptr TableRef::FormatDeserialize(FormatDeserializer &deserializer) { auto type = deserializer.ReadProperty(100, "type"); auto alias = deserializer.ReadProperty(101, "alias"); - auto sample = deserializer.ReadOptionalProperty>(102, "sample"); + auto sample = deserializer.ReadPropertyWithDefault>(102, "sample", unique_ptr()); unique_ptr result; switch (type) { case TableReferenceType::BASE_TABLE: @@ -93,9 +93,9 @@ unique_ptr ExpressionListRef::FormatDeserialize(FormatDeserializer &de void JoinRef::FormatSerialize(FormatSerializer &serializer) const { TableRef::FormatSerialize(serializer); - serializer.WriteProperty(200, "left", *left); - serializer.WriteProperty(201, "right", *right); - serializer.WriteOptionalProperty(202, "condition", condition); + serializer.WriteProperty(200, "left", left); + serializer.WriteProperty(201, "right", right); + serializer.WritePropertyWithDefault(202, "condition", condition, unique_ptr()); serializer.WriteProperty(203, "join_type", type); serializer.WriteProperty(204, "ref_type", ref_type); serializer.WriteProperty(205, "using_columns", using_columns); @@ -105,7 +105,7 @@ unique_ptr JoinRef::FormatDeserialize(FormatDeserializer &deserializer auto result = duckdb::unique_ptr(new JoinRef()); deserializer.ReadProperty(200, "left", result->left); deserializer.ReadProperty(201, "right", result->right); - deserializer.ReadOptionalProperty(202, "condition", result->condition); + deserializer.ReadPropertyWithDefault(202, "condition", result->condition, unique_ptr()); deserializer.ReadProperty(203, "join_type", result->type); deserializer.ReadProperty(204, "ref_type", result->ref_type); deserializer.ReadProperty(205, "using_columns", result->using_columns); @@ -114,7 +114,7 @@ unique_ptr JoinRef::FormatDeserialize(FormatDeserializer &deserializer void PivotRef::FormatSerialize(FormatSerializer &serializer) const { TableRef::FormatSerialize(serializer); - serializer.WriteProperty(200, "source", *source); + serializer.WriteProperty(200, "source", source); serializer.WriteProperty(201, "aggregates", aggregates); serializer.WriteProperty(202, "unpivot_names", unpivot_names); serializer.WriteProperty(203, "pivots", pivots); @@ -137,7 +137,7 @@ unique_ptr PivotRef::FormatDeserialize(FormatDeserializer &deserialize void SubqueryRef::FormatSerialize(FormatSerializer &serializer) const { TableRef::FormatSerialize(serializer); - serializer.WriteProperty(200, "subquery", *subquery); + serializer.WriteProperty(200, "subquery", subquery); serializer.WriteProperty(201, "column_name_alias", column_name_alias); } @@ -150,7 +150,7 @@ unique_ptr SubqueryRef::FormatDeserialize(FormatDeserializer &deserial void TableFunctionRef::FormatSerialize(FormatSerializer &serializer) const { TableRef::FormatSerialize(serializer); - serializer.WriteProperty(200, "function", *function); + serializer.WriteProperty(200, "function", function); serializer.WriteProperty(201, "column_name_alias", column_name_alias); } diff --git a/src/duckdb/src/storage/statistics/base_statistics.cpp b/src/duckdb/src/storage/statistics/base_statistics.cpp index 08ce54340..8e115063d 100644 --- a/src/duckdb/src/storage/statistics/base_statistics.cpp +++ b/src/duckdb/src/storage/statistics/base_statistics.cpp @@ -6,6 +6,9 @@ #include "duckdb/storage/statistics/list_stats.hpp" #include "duckdb/storage/statistics/struct_stats.hpp" +#include "duckdb/common/serializer/format_serializer.hpp" +#include "duckdb/common/serializer/format_deserializer.hpp" + namespace duckdb { BaseStatistics::BaseStatistics() : type(LogicalType::INVALID) { @@ -267,6 +270,10 @@ void BaseStatistics::CopyValidity(BaseStatistics &stats) { has_no_null = stats.has_no_null; } +void BaseStatistics::SetDistinctCount(idx_t count) { + this->distinct_count = count; +} + void BaseStatistics::Serialize(Serializer &serializer) const { FieldWriter writer(serializer); writer.WriteField(has_null); @@ -276,10 +283,6 @@ void BaseStatistics::Serialize(Serializer &serializer) const { writer.Finalize(); } -void BaseStatistics::SetDistinctCount(idx_t count) { - this->distinct_count = count; -} - void BaseStatistics::Serialize(FieldWriter &writer) const { switch (GetStatsType()) { case StatisticsType::NUMERIC_STATS: @@ -298,6 +301,66 @@ void BaseStatistics::Serialize(FieldWriter &writer) const { break; } } + +void BaseStatistics::FormatSerialize(FormatSerializer &serializer) const { + serializer.WriteProperty(100, "has_null", has_null); + serializer.WriteProperty(101, "has_no_null", has_no_null); + serializer.WriteProperty(102, "distinct_count", distinct_count); + serializer.WriteProperty(103, "stats_type", GetStatsType()); + serializer.WriteObject(104, "stats", [this](FormatSerializer &serializer) { + switch (GetStatsType()) { + case StatisticsType::NUMERIC_STATS: + NumericStats::FormatSerialize(*this, serializer); + break; + case StatisticsType::STRING_STATS: + StringStats::FormatSerialize(*this, serializer); + break; + case StatisticsType::LIST_STATS: + ListStats::FormatSerialize(*this, serializer); + break; + case StatisticsType::STRUCT_STATS: + StructStats::FormatSerialize(*this, serializer); + break; + default: + throw NotImplementedException("Unrecognized StatisticsType for BaseStatistics::FormatSerialize"); + } + }); +} + +BaseStatistics BaseStatistics::FormatDeserialize(FormatDeserializer &deserializer) { + auto has_null = deserializer.ReadProperty(100, "has_null"); + auto has_no_null = deserializer.ReadProperty(101, "has_no_null"); + auto distinct_count = deserializer.ReadProperty(102, "distinct_count"); + + // Get the logical type from the deserializer context. + auto type = deserializer.Get(); + + BaseStatistics stats; + switch (GetStatsType(type)) { + case StatisticsType::NUMERIC_STATS: + stats = NumericStats::FormatDeserialize(deserializer, type); + break; + case StatisticsType::STRING_STATS: + stats = StringStats::FormatDeserialize(deserializer, type); + break; + case StatisticsType::LIST_STATS: + stats = ListStats::FormatDeserialize(deserializer, type); + break; + case StatisticsType::STRUCT_STATS: + stats = StructStats::FormatDeserialize(deserializer, type); + break; + default: + stats = BaseStatistics(std::move(type)); + break; + } + + stats.has_null = has_null; + stats.has_no_null = has_no_null; + stats.distinct_count = distinct_count; + + return stats; +} + BaseStatistics BaseStatistics::DeserializeType(FieldReader &reader, LogicalType type) { switch (GetStatsType(type)) { case StatisticsType::NUMERIC_STATS: diff --git a/src/duckdb/src/storage/statistics/column_statistics.cpp b/src/duckdb/src/storage/statistics/column_statistics.cpp index b557d8207..7d5baebc4 100644 --- a/src/duckdb/src/storage/statistics/column_statistics.cpp +++ b/src/duckdb/src/storage/statistics/column_statistics.cpp @@ -1,5 +1,7 @@ #include "duckdb/storage/statistics/column_statistics.hpp" #include "duckdb/common/serializer.hpp" +#include "duckdb/common/serializer/format_deserializer.hpp" +#include "duckdb/common/serializer/format_serializer.hpp" namespace duckdb { @@ -64,4 +66,18 @@ shared_ptr ColumnStatistics::Deserialize(Deserializer &source, return make_shared(stats.Copy(), std::move(distinct_stats)); } +void ColumnStatistics::FormatSerialize(FormatSerializer &serializer) const { + serializer.WriteProperty(100, "statistics", stats); + serializer.WritePropertyWithDefault(101, "distinct", distinct_stats, unique_ptr()); +} + +shared_ptr ColumnStatistics::FormatDeserialize(FormatDeserializer &deserializer) { + // TODO: do we read this as an property or into the object itself? + // we have this sort of pseudo inheritance going on here which is annoying + auto stats = BaseStatistics::FormatDeserialize(deserializer); + auto distinct_stats = deserializer.ReadPropertyWithDefault>( + 101, "distinct", unique_ptr()); + return make_shared(stats.Copy(), std::move(distinct_stats)); +} + } // namespace duckdb diff --git a/src/duckdb/src/storage/statistics/list_stats.cpp b/src/duckdb/src/storage/statistics/list_stats.cpp index 29f34c345..a9abcdf5b 100644 --- a/src/duckdb/src/storage/statistics/list_stats.cpp +++ b/src/duckdb/src/storage/statistics/list_stats.cpp @@ -4,6 +4,9 @@ #include "duckdb/common/string_util.hpp" #include "duckdb/common/types/vector.hpp" +#include "duckdb/common/serializer/format_serializer.hpp" +#include "duckdb/common/serializer/format_deserializer.hpp" + namespace duckdb { void ListStats::Construct(BaseStatistics &stats) { @@ -79,6 +82,24 @@ BaseStatistics ListStats::Deserialize(FieldReader &reader, LogicalType type) { return result; } +void ListStats::FormatSerialize(const BaseStatistics &stats, FormatSerializer &serializer) { + auto &child_stats = ListStats::GetChildStats(stats); + serializer.WriteProperty(200, "child_stats", child_stats); +} + +BaseStatistics ListStats::FormatDeserialize(FormatDeserializer &deserializer, LogicalType type) { + D_ASSERT(type.InternalType() == PhysicalType::LIST); + auto &child_type = ListType::GetChildType(type); + BaseStatistics result(std::move(type)); + + // Push the logical type of the child type to the deserialization context + deserializer.Set(const_cast(child_type)); + result.child_stats[0].Copy(deserializer.ReadProperty(200, "child_stats")); + deserializer.Unset(); + + return result; +} + string ListStats::ToString(const BaseStatistics &stats) { auto &child_stats = ListStats::GetChildStats(stats); return StringUtil::Format("[%s]", child_stats.ToString()); diff --git a/src/duckdb/src/storage/statistics/numeric_stats.cpp b/src/duckdb/src/storage/statistics/numeric_stats.cpp index a37821bd7..f5dd477db 100644 --- a/src/duckdb/src/storage/statistics/numeric_stats.cpp +++ b/src/duckdb/src/storage/statistics/numeric_stats.cpp @@ -4,6 +4,9 @@ #include "duckdb/common/types/vector.hpp" #include "duckdb/common/operator/comparison_operators.hpp" +#include "duckdb/common/serializer/format_serializer.hpp" +#include "duckdb/common/serializer/format_deserializer.hpp" + namespace duckdb { template <> @@ -403,7 +406,8 @@ Value NumericStats::MaxOrNull(const BaseStatistics &stats) { return NumericStats::Max(stats); } -void SerializeNumericStatsValue(const LogicalType &type, NumericValueUnion val, bool has_value, FieldWriter &writer) { +static void SerializeNumericStatsValue(const LogicalType &type, NumericValueUnion val, bool has_value, + FieldWriter &writer) { writer.WriteField(!has_value); if (!has_value) { return; @@ -514,6 +518,127 @@ BaseStatistics NumericStats::Deserialize(FieldReader &reader, LogicalType type) return result; } +static void FormatSerializeNumericStatsValue(const LogicalType &type, NumericValueUnion val, bool has_value, + FormatSerializer &serializer) { + serializer.WriteProperty(100, "has_value", has_value); + if (!has_value) { + return; + } + switch (type.InternalType()) { + case PhysicalType::BOOL: + serializer.WriteProperty(101, "value", val.value_.boolean); + break; + case PhysicalType::INT8: + serializer.WriteProperty(101, "value", val.value_.tinyint); + break; + case PhysicalType::INT16: + serializer.WriteProperty(101, "value", val.value_.smallint); + break; + case PhysicalType::INT32: + serializer.WriteProperty(101, "value", val.value_.integer); + break; + case PhysicalType::INT64: + serializer.WriteProperty(101, "value", val.value_.bigint); + break; + case PhysicalType::UINT8: + serializer.WriteProperty(101, "value", val.value_.utinyint); + break; + case PhysicalType::UINT16: + serializer.WriteProperty(101, "value", val.value_.usmallint); + break; + case PhysicalType::UINT32: + serializer.WriteProperty(101, "value", val.value_.uinteger); + break; + case PhysicalType::UINT64: + serializer.WriteProperty(101, "value", val.value_.ubigint); + break; + case PhysicalType::INT128: + serializer.WriteProperty(101, "value", val.value_.hugeint); + break; + case PhysicalType::FLOAT: + serializer.WriteProperty(101, "value", val.value_.float_); + break; + case PhysicalType::DOUBLE: + serializer.WriteProperty(101, "value", val.value_.double_); + break; + default: + throw InternalException("Unsupported type for serializing numeric statistics"); + } +} + +void NumericStats::FormatSerialize(const BaseStatistics &stats, FormatSerializer &serializer) { + auto &numeric_stats = NumericStats::GetDataUnsafe(stats); + serializer.WriteObject(200, "max", [&](FormatSerializer &object) { + FormatSerializeNumericStatsValue(stats.GetType(), numeric_stats.min, numeric_stats.has_min, object); + }); + serializer.WriteObject(201, "min", [&](FormatSerializer &object) { + FormatSerializeNumericStatsValue(stats.GetType(), numeric_stats.max, numeric_stats.has_max, object); + }); +} + +static void FormatDeserializeNumericStatsValue(const LogicalType &type, NumericValueUnion &result, bool &has_stats, + FormatDeserializer &deserializer) { + auto has_value = deserializer.ReadProperty(100, "has_value"); + if (!has_value) { + has_stats = false; + return; + } + has_stats = true; + switch (type.InternalType()) { + case PhysicalType::BOOL: + result.value_.boolean = deserializer.ReadProperty(101, "value"); + break; + case PhysicalType::INT8: + result.value_.tinyint = deserializer.ReadProperty(101, "value"); + break; + case PhysicalType::INT16: + result.value_.smallint = deserializer.ReadProperty(101, "value"); + break; + case PhysicalType::INT32: + result.value_.integer = deserializer.ReadProperty(101, "value"); + break; + case PhysicalType::INT64: + result.value_.bigint = deserializer.ReadProperty(101, "value"); + break; + case PhysicalType::UINT8: + result.value_.utinyint = deserializer.ReadProperty(101, "value"); + break; + case PhysicalType::UINT16: + result.value_.usmallint = deserializer.ReadProperty(101, "value"); + break; + case PhysicalType::UINT32: + result.value_.uinteger = deserializer.ReadProperty(101, "value"); + break; + case PhysicalType::UINT64: + result.value_.ubigint = deserializer.ReadProperty(101, "value"); + break; + case PhysicalType::INT128: + result.value_.hugeint = deserializer.ReadProperty(101, "value"); + break; + case PhysicalType::FLOAT: + result.value_.float_ = deserializer.ReadProperty(101, "value"); + break; + case PhysicalType::DOUBLE: + result.value_.double_ = deserializer.ReadProperty(101, "value"); + break; + default: + throw InternalException("Unsupported type for serializing numeric statistics"); + } +} + +BaseStatistics NumericStats::FormatDeserialize(FormatDeserializer &deserializer, LogicalType type) { + BaseStatistics result(std::move(type)); + auto &numeric_stats = NumericStats::GetDataUnsafe(result); + + deserializer.ReadObject(200, "max", [&](FormatDeserializer &object) { + FormatDeserializeNumericStatsValue(result.GetType(), numeric_stats.min, numeric_stats.has_min, object); + }); + deserializer.ReadObject(201, "min", [&](FormatDeserializer &object) { + FormatDeserializeNumericStatsValue(result.GetType(), numeric_stats.max, numeric_stats.has_max, object); + }); + return result; +} + string NumericStats::ToString(const BaseStatistics &stats) { return StringUtil::Format("[Min: %s, Max: %s]", NumericStats::MinOrNull(stats).ToString(), NumericStats::MaxOrNull(stats).ToString()); diff --git a/src/duckdb/src/storage/statistics/string_stats.cpp b/src/duckdb/src/storage/statistics/string_stats.cpp index 0cf227d89..68bfe9106 100644 --- a/src/duckdb/src/storage/statistics/string_stats.cpp +++ b/src/duckdb/src/storage/statistics/string_stats.cpp @@ -7,6 +7,9 @@ #include "duckdb/storage/statistics/base_statistics.hpp" #include "utf8proc_wrapper.hpp" +#include "duckdb/common/serializer/format_serializer.hpp" +#include "duckdb/common/serializer/format_deserializer.hpp" + namespace duckdb { BaseStatistics StringStats::CreateUnknown(LogicalType type) { @@ -103,6 +106,15 @@ void StringStats::Serialize(const BaseStatistics &stats, FieldWriter &writer) { writer.WriteField(string_data.max_string_length); } +void StringStats::FormatSerialize(const BaseStatistics &stats, FormatSerializer &serializer) { + auto &string_data = StringStats::GetDataUnsafe(stats); + serializer.WriteProperty(200, "min", string_data.min, StringStatsData::MAX_STRING_MINMAX_SIZE); + serializer.WriteProperty(201, "max", string_data.max, StringStatsData::MAX_STRING_MINMAX_SIZE); + serializer.WriteProperty(202, "has_unicode", string_data.has_unicode); + serializer.WriteProperty(203, "has_max_string_length", string_data.has_max_string_length); + serializer.WriteProperty(204, "max_string_length", string_data.max_string_length); +} + BaseStatistics StringStats::Deserialize(FieldReader &reader, LogicalType type) { BaseStatistics result(std::move(type)); auto &string_data = StringStats::GetDataUnsafe(result); @@ -114,6 +126,17 @@ BaseStatistics StringStats::Deserialize(FieldReader &reader, LogicalType type) { return result; } +BaseStatistics StringStats::FormatDeserialize(FormatDeserializer &deserializer, LogicalType type) { + BaseStatistics result(std::move(type)); + auto &string_data = StringStats::GetDataUnsafe(result); + deserializer.ReadProperty(200, "min", string_data.min, StringStatsData::MAX_STRING_MINMAX_SIZE); + deserializer.ReadProperty(201, "max", string_data.max, StringStatsData::MAX_STRING_MINMAX_SIZE); + deserializer.ReadProperty(202, "has_unicode", string_data.has_unicode); + deserializer.ReadProperty(203, "has_max_string_length", string_data.has_max_string_length); + deserializer.ReadProperty(204, "max_string_length", string_data.max_string_length); + return result; +} + static int StringValueComparison(const_data_ptr_t data, idx_t len, const_data_ptr_t comparison) { D_ASSERT(len <= StringStatsData::MAX_STRING_MINMAX_SIZE); for (idx_t i = 0; i < len; i++) { diff --git a/src/duckdb/src/storage/statistics/struct_stats.cpp b/src/duckdb/src/storage/statistics/struct_stats.cpp index 7c13df2cf..2f0ef0eb0 100644 --- a/src/duckdb/src/storage/statistics/struct_stats.cpp +++ b/src/duckdb/src/storage/statistics/struct_stats.cpp @@ -3,6 +3,9 @@ #include "duckdb/common/field_writer.hpp" #include "duckdb/common/types/vector.hpp" +#include "duckdb/common/serializer/format_serializer.hpp" +#include "duckdb/common/serializer/format_deserializer.hpp" + namespace duckdb { void StructStats::Construct(BaseStatistics &stats) { @@ -109,6 +112,30 @@ BaseStatistics StructStats::Deserialize(FieldReader &reader, LogicalType type) { return result; } +void StructStats::FormatSerialize(const BaseStatistics &stats, FormatSerializer &serializer) { + auto child_stats = StructStats::GetChildStats(stats); + auto child_count = StructType::GetChildCount(stats.GetType()); + + serializer.WriteList(200, "child_stats", child_count, + [&](FormatSerializer::List &list, idx_t i) { list.WriteElement(child_stats[i]); }); +} + +BaseStatistics StructStats::FormatDeserialize(FormatDeserializer &deserializer, LogicalType type) { + D_ASSERT(type.InternalType() == PhysicalType::STRUCT); + + auto &child_types = StructType::GetChildTypes(type); + BaseStatistics result(std::move(type)); + + deserializer.ReadList(200, "child_stats", [&](FormatDeserializer::List &list, idx_t i) { + deserializer.Set(const_cast(child_types[i].second)); + auto stat = list.ReadElement(); + result.child_stats[i].Copy(stat); + deserializer.Unset(); + }); + + return result; +} + string StructStats::ToString(const BaseStatistics &stats) { string result; result += " {"; diff --git a/src/duckdb/src/storage/storage_info.cpp b/src/duckdb/src/storage/storage_info.cpp index dea9567b5..f920245ad 100644 --- a/src/duckdb/src/storage/storage_info.cpp +++ b/src/duckdb/src/storage/storage_info.cpp @@ -2,7 +2,7 @@ namespace duckdb { -const uint64_t VERSION_NUMBER = 56; +const uint64_t VERSION_NUMBER = 57; struct StorageVersionInfo { const char *version_name; diff --git a/src/duckdb/src/storage/table/chunk_info.cpp b/src/duckdb/src/storage/table/chunk_info.cpp index cf67e5c92..c9b05411f 100644 --- a/src/duckdb/src/storage/table/chunk_info.cpp +++ b/src/duckdb/src/storage/table/chunk_info.cpp @@ -1,6 +1,8 @@ #include "duckdb/storage/table/chunk_info.hpp" #include "duckdb/transaction/transaction.hpp" #include "duckdb/common/serializer.hpp" +#include "duckdb/common/serializer/format_serializer.hpp" +#include "duckdb/common/serializer/format_deserializer.hpp" namespace duckdb { @@ -42,6 +44,20 @@ unique_ptr ChunkInfo::Deserialize(Deserializer &source) { } } +unique_ptr ChunkInfo::FormatDeserialize(FormatDeserializer &deserializer) { + auto type = deserializer.ReadProperty(100, "type"); + switch (type) { + case ChunkInfoType::EMPTY_INFO: + return nullptr; + case ChunkInfoType::CONSTANT_INFO: + return ChunkConstantInfo::FormatDeserialize(deserializer); + case ChunkInfoType::VECTOR_INFO: + return ChunkVectorInfo::FormatDeserialize(deserializer); + default: + throw SerializationException("Could not deserialize Chunk Info Type: unrecognized type"); + } +} + //===--------------------------------------------------------------------===// // Constant info //===--------------------------------------------------------------------===// @@ -51,7 +67,7 @@ ChunkConstantInfo::ChunkConstantInfo(idx_t start) template idx_t ChunkConstantInfo::TemplatedGetSelVector(transaction_t start_time, transaction_t transaction_id, - SelectionVector &sel_vector, idx_t max_count) { + SelectionVector &sel_vector, idx_t max_count) const { if (OP::UseInsertedVersion(start_time, transaction_id, insert_id) && OP::UseDeletedVersion(start_time, transaction_id, delete_id)) { return max_count; @@ -102,6 +118,24 @@ unique_ptr ChunkConstantInfo::Deserialize(Deserializer &source) { return std::move(info); } +void ChunkConstantInfo::FormatSerialize(FormatSerializer &serializer) const { + bool is_deleted = insert_id >= TRANSACTION_ID_START || delete_id < TRANSACTION_ID_START; + if (!is_deleted) { + serializer.WriteProperty(100, "type", ChunkInfoType::EMPTY_INFO); + return; + } + serializer.WriteProperty(100, "type", type); + serializer.WriteProperty(200, "start", start); +} + +unique_ptr ChunkConstantInfo::FormatDeserialize(FormatDeserializer &deserializer) { + auto start = deserializer.ReadProperty(200, "start"); + auto info = make_uniq(start); + info->insert_id = 0; + info->delete_id = 0; + return std::move(info); +} + //===--------------------------------------------------------------------===// // Vector info //===--------------------------------------------------------------------===// @@ -115,7 +149,7 @@ ChunkVectorInfo::ChunkVectorInfo(idx_t start) template idx_t ChunkVectorInfo::TemplatedGetSelVector(transaction_t start_time, transaction_t transaction_id, - SelectionVector &sel_vector, idx_t max_count) { + SelectionVector &sel_vector, idx_t max_count) const { idx_t count = 0; if (same_inserted_id && !any_deleted) { // all tuples have the same inserted id: and no tuples were deleted @@ -154,7 +188,7 @@ idx_t ChunkVectorInfo::TemplatedGetSelVector(transaction_t start_time, transacti } idx_t ChunkVectorInfo::GetSelVector(transaction_t start_time, transaction_t transaction_id, SelectionVector &sel_vector, - idx_t max_count) { + idx_t max_count) const { return TemplatedGetSelVector(start_time, transaction_id, sel_vector, max_count); } @@ -276,4 +310,49 @@ unique_ptr ChunkVectorInfo::Deserialize(Deserializer &source) { return std::move(result); } +void ChunkVectorInfo::FormatSerialize(FormatSerializer &serializer) const { + SelectionVector sel(STANDARD_VECTOR_SIZE); + transaction_t start_time = TRANSACTION_ID_START - 1; + transaction_t transaction_id = DConstants::INVALID_INDEX; + idx_t count = GetSelVector(start_time, transaction_id, sel, STANDARD_VECTOR_SIZE); + if (count == STANDARD_VECTOR_SIZE) { + // nothing is deleted: skip writing anything + serializer.WriteProperty(100, "type", ChunkInfoType::EMPTY_INFO); + return; + } + if (count == 0) { + // everything is deleted: write a constant vector + serializer.WriteProperty(100, "type", ChunkInfoType::CONSTANT_INFO); + serializer.WriteProperty(200, "start", start); + return; + } + // write a boolean vector + serializer.WriteProperty(100, "type", ChunkInfoType::VECTOR_INFO); + serializer.WriteProperty(200, "start", start); + bool deleted_tuples[STANDARD_VECTOR_SIZE]; + for (idx_t i = 0; i < STANDARD_VECTOR_SIZE; i++) { + deleted_tuples[i] = true; + } + for (idx_t i = 0; i < count; i++) { + deleted_tuples[sel.get_index(i)] = false; + } + serializer.WriteProperty(201, "deleted_tuples", data_ptr_cast(deleted_tuples), sizeof(bool) * STANDARD_VECTOR_SIZE); +} + +unique_ptr ChunkVectorInfo::FormatDeserialize(FormatDeserializer &deserializer) { + auto start = deserializer.ReadProperty(200, "start"); + + auto result = make_uniq(start); + result->any_deleted = true; + bool deleted_tuples[STANDARD_VECTOR_SIZE]; + deserializer.ReadProperty(201, "deleted_tuples", data_ptr_cast(deleted_tuples), + sizeof(bool) * STANDARD_VECTOR_SIZE); + for (idx_t i = 0; i < STANDARD_VECTOR_SIZE; i++) { + if (deleted_tuples[i]) { + result->deleted[i] = 0; + } + } + return std::move(result); +} + } // namespace duckdb diff --git a/src/duckdb/src/storage/table/row_group.cpp b/src/duckdb/src/storage/table/row_group.cpp index fe74f0610..83f2fadf6 100644 --- a/src/duckdb/src/storage/table/row_group.cpp +++ b/src/duckdb/src/storage/table/row_group.cpp @@ -1,6 +1,5 @@ #include "duckdb/storage/table/row_group.hpp" #include "duckdb/common/types/vector.hpp" -#include "duckdb/transaction/transaction.hpp" #include "duckdb/common/exception.hpp" #include "duckdb/common/field_writer.hpp" #include "duckdb/storage/table/column_data.hpp" @@ -18,6 +17,8 @@ #include "duckdb/transaction/duck_transaction.hpp" #include "duckdb/storage/table/append_state.hpp" #include "duckdb/storage/table/scan_state.hpp" +#include "duckdb/common/serializer/format_serializer.hpp" +#include "duckdb/common/serializer/format_deserializer.hpp" namespace duckdb { @@ -927,6 +928,72 @@ RowGroupPointer RowGroup::Deserialize(Deserializer &main_source, const vectorinfo[vector_idx].get(); + if (!chunk_info) { + continue; + } + idx_map[chunk_info_count++] = vector_idx; + } + + // now serialize the actual version information + serializer.WriteProperty(103, "versions_count", chunk_info_count); + serializer.WriteList(104, "versions", chunk_info_count, [&](FormatSerializer::List &list, idx_t i) { + auto vector_idx = idx_map[i]; + auto chunk_info = versions->info[vector_idx].get(); + list.WriteObject([&](FormatSerializer &obj) { + obj.WriteProperty(100, "vector_index", vector_idx); + obj.WriteProperty(101, "chunk_info", const_cast(chunk_info)); + }); + }); +} + +RowGroupPointer RowGroup::FormatDeserialize(FormatDeserializer &deserializer) { + RowGroupPointer result; + result.row_start = deserializer.ReadProperty(100, "row_start"); + result.tuple_count = deserializer.ReadProperty(101, "tuple_count"); + result.data_pointers = deserializer.ReadProperty>(102, "data_pointers"); + result.versions = nullptr; + // Deserialize Deletes + auto chunk_count = deserializer.ReadProperty(103, "versions_count"); + if (chunk_count == 0) { + // no deletes + return result; + } + + auto version_info = make_shared(); + deserializer.ReadList(104, "versions", [&](FormatDeserializer::List &list, idx_t i) { + list.ReadObject([&](FormatDeserializer &obj) { + auto vector_index = obj.ReadProperty(100, "vector_index"); + if (vector_index >= RowGroup::ROW_GROUP_VECTOR_COUNT) { + throw Exception("In DeserializeDeletes, vector_index is out of range for the row group. Corrupted " + "file?"); + } + version_info->info[vector_index] = obj.ReadProperty>(101, "chunk_info"); + }); + }); + + result.versions = version_info; + + return result; +} + //===--------------------------------------------------------------------===// // GetColumnSegmentInfo //===--------------------------------------------------------------------===// diff --git a/src/duckdb/src/storage/table/table_statistics.cpp b/src/duckdb/src/storage/table/table_statistics.cpp index edbfe0e82..e0a35c32b 100644 --- a/src/duckdb/src/storage/table/table_statistics.cpp +++ b/src/duckdb/src/storage/table/table_statistics.cpp @@ -1,5 +1,7 @@ #include "duckdb/storage/table/table_statistics.hpp" #include "duckdb/storage/table/persistent_table_data.hpp" +#include "duckdb/common/serializer/format_serializer.hpp" +#include "duckdb/common/serializer/format_deserializer.hpp" namespace duckdb { @@ -112,6 +114,25 @@ void TableStatistics::Deserialize(Deserializer &source, ColumnList &columns) { } } +void TableStatistics::FormatSerialize(FormatSerializer &serializer) { + auto column_count = column_stats.size(); + serializer.WriteList(100, "column_stats", column_count, + [&](FormatSerializer::List &list, idx_t i) { list.WriteElement(column_stats[i]); }); +} + +void TableStatistics::FormatDeserialize(FormatDeserializer &deserializer, ColumnList &columns) { + auto physical_columns = columns.Physical(); + auto iter = physical_columns.begin(); + deserializer.ReadList(100, "column_stats", [&](FormatDeserializer::List &list, idx_t i) { + auto &col = *iter.operator++(); + auto type = col.GetType(); + deserializer.Set(type); + auto stats = ColumnStatistics::FormatDeserialize(deserializer); + deserializer.Unset(); + column_stats.push_back(std::move(stats)); + }); +} + unique_ptr TableStatistics::GetLock() { return make_uniq(stats_lock); } diff --git a/src/duckdb/src/storage/wal_replay.cpp b/src/duckdb/src/storage/wal_replay.cpp index 02465820b..15fdc865e 100644 --- a/src/duckdb/src/storage/wal_replay.cpp +++ b/src/duckdb/src/storage/wal_replay.cpp @@ -53,7 +53,7 @@ bool WriteAheadLog::Replay(AttachedDatabase &database, string &path) { } } } catch (std::exception &ex) { // LCOV_EXCL_START - Printer::Print(StringUtil::Format("Exception in WAL playback during initial read: %s\n", ex.what())); + Printer::PrintF("Exception in WAL playback during initial read: %s\n", ex.what()); return false; } catch (...) { Printer::Print("Unknown Exception in WAL playback during initial read"); @@ -99,7 +99,7 @@ bool WriteAheadLog::Replay(AttachedDatabase &database, string &path) { } } catch (std::exception &ex) { // LCOV_EXCL_START // FIXME: this should report a proper warning in the connection - Printer::Print(StringUtil::Format("Exception in WAL playback: %s\n", ex.what())); + Printer::PrintF("Exception in WAL playback: %s\n", ex.what()); // exception thrown in WAL replay: rollback con.Rollback(); } catch (...) { diff --git a/src/duckdb/src/verification/deserialized_statement_verifier_v2.cpp b/src/duckdb/src/verification/deserialized_statement_verifier_v2.cpp index ad793f9eb..f3d349e70 100644 --- a/src/duckdb/src/verification/deserialized_statement_verifier_v2.cpp +++ b/src/duckdb/src/verification/deserialized_statement_verifier_v2.cpp @@ -11,10 +11,24 @@ DeserializedStatementVerifierV2::DeserializedStatementVerifierV2(unique_ptr DeserializedStatementVerifierV2::Create(const SQLStatement &statement) { auto &select_stmt = statement.Cast(); - auto blob = BinarySerializer::Serialize(select_stmt); + auto blob = BinarySerializer::Serialize(select_stmt, true); auto result = BinaryDeserializer::Deserialize(blob.data(), blob.size()); return make_uniq(std::move(result)); } +DeserializedStatementVerifierNoDefaultV2::DeserializedStatementVerifierNoDefaultV2(unique_ptr statement_p) + : StatementVerifier(VerificationType::DESERIALIZED_V2_NO_DEFAULT, "Deserialized V2 without default values", + std::move(statement_p)) { +} + +unique_ptr DeserializedStatementVerifierNoDefaultV2::Create(const SQLStatement &statement) { + auto &select_stmt = statement.Cast(); + + auto blob = BinarySerializer::Serialize(select_stmt, false); + auto result = BinaryDeserializer::Deserialize(blob.data(), blob.size()); + + return make_uniq(std::move(result)); +} + } // namespace duckdb diff --git a/src/duckdb/src/verification/statement_verifier.cpp b/src/duckdb/src/verification/statement_verifier.cpp index 4330c746f..44fd3c3dc 100644 --- a/src/duckdb/src/verification/statement_verifier.cpp +++ b/src/duckdb/src/verification/statement_verifier.cpp @@ -35,6 +35,8 @@ unique_ptr StatementVerifier::Create(VerificationType type, c return DeserializedStatementVerifier::Create(statement_p); case VerificationType::DESERIALIZED_V2: return DeserializedStatementVerifierV2::Create(statement_p); + case VerificationType::DESERIALIZED_V2_NO_DEFAULT: + return DeserializedStatementVerifierNoDefaultV2::Create(statement_p); case VerificationType::PARSED: return ParsedStatementVerifier::Create(statement_p); case VerificationType::UNOPTIMIZED: diff --git a/src/duckdb/third_party/utf8proc/include/utf8proc_wrapper.hpp b/src/duckdb/third_party/utf8proc/include/utf8proc_wrapper.hpp index 5614fd996..fb988b254 100644 --- a/src/duckdb/third_party/utf8proc/include/utf8proc_wrapper.hpp +++ b/src/duckdb/third_party/utf8proc/include/utf8proc_wrapper.hpp @@ -1,3 +1,11 @@ +//===----------------------------------------------------------------------===// +// DuckDB +// +// utf8proc_wrapper.hpp +// +// +//===----------------------------------------------------------------------===// + #pragma once #include diff --git a/src/duckdb/ub_src_execution.cpp b/src/duckdb/ub_src_execution.cpp index 3747a66b6..d845738bc 100644 --- a/src/duckdb/ub_src_execution.cpp +++ b/src/duckdb/ub_src_execution.cpp @@ -12,8 +12,6 @@ #include "src/execution/join_hashtable.cpp" -#include "src/execution/partitionable_hashtable.cpp" - #include "src/execution/perfect_aggregate_hashtable.cpp" #include "src/execution/physical_operator.cpp" diff --git a/src/duckdb/ub_src_execution_operator_csv_scanner.cpp b/src/duckdb/ub_src_execution_operator_csv_scanner.cpp new file mode 100644 index 000000000..117072ef2 --- /dev/null +++ b/src/duckdb/ub_src_execution_operator_csv_scanner.cpp @@ -0,0 +1,18 @@ +#include "src/execution/operator/csv_scanner/base_csv_reader.cpp" + +#include "src/execution/operator/csv_scanner/buffered_csv_reader.cpp" + +#include "src/execution/operator/csv_scanner/csv_buffer.cpp" + +#include "src/execution/operator/csv_scanner/csv_buffer_manager.cpp" + +#include "src/execution/operator/csv_scanner/csv_file_handle.cpp" + +#include "src/execution/operator/csv_scanner/csv_reader_options.cpp" + +#include "src/execution/operator/csv_scanner/csv_state_machine.cpp" + +#include "src/execution/operator/csv_scanner/csv_state_machine_cache.cpp" + +#include "src/execution/operator/csv_scanner/parallel_csv_reader.cpp" + diff --git a/src/duckdb/ub_src_execution_operator_csv_scanner_sniffer.cpp b/src/duckdb/ub_src_execution_operator_csv_scanner_sniffer.cpp new file mode 100644 index 000000000..61cfd4912 --- /dev/null +++ b/src/duckdb/ub_src_execution_operator_csv_scanner_sniffer.cpp @@ -0,0 +1,12 @@ +#include "src/execution/operator/csv_scanner/sniffer/csv_sniffer.cpp" + +#include "src/execution/operator/csv_scanner/sniffer/dialect_detection.cpp" + +#include "src/execution/operator/csv_scanner/sniffer/header_detection.cpp" + +#include "src/execution/operator/csv_scanner/sniffer/type_detection.cpp" + +#include "src/execution/operator/csv_scanner/sniffer/type_refinement.cpp" + +#include "src/execution/operator/csv_scanner/sniffer/type_replacement.cpp" + diff --git a/src/duckdb/ub_src_execution_operator_persistent.cpp b/src/duckdb/ub_src_execution_operator_persistent.cpp index fd19de68d..be135ea0b 100644 --- a/src/duckdb/ub_src_execution_operator_persistent.cpp +++ b/src/duckdb/ub_src_execution_operator_persistent.cpp @@ -1,15 +1,3 @@ -#include "src/execution/operator/persistent/base_csv_reader.cpp" - -#include "src/execution/operator/persistent/buffered_csv_reader.cpp" - -#include "src/execution/operator/persistent/parallel_csv_reader.cpp" - -#include "src/execution/operator/persistent/csv_buffer.cpp" - -#include "src/execution/operator/persistent/csv_file_handle.cpp" - -#include "src/execution/operator/persistent/csv_reader_options.cpp" - #include "src/execution/operator/persistent/csv_rejects_table.cpp" #include "src/execution/operator/persistent/physical_fixed_batch_copy.cpp" diff --git a/src/duckdb/ub_src_storage_serialization.cpp b/src/duckdb/ub_src_storage_serialization.cpp index aa187b2c4..2465830ff 100644 --- a/src/duckdb/ub_src_storage_serialization.cpp +++ b/src/duckdb/ub_src_storage_serialization.cpp @@ -26,3 +26,5 @@ #include "src/storage/serialization/serialize_types.cpp" +#include "src/storage/serialization/serialize_storage.cpp" + diff --git a/src/sources.mk b/src/sources.mk index 9e249de53..411696555 100644 --- a/src/sources.mk +++ b/src/sources.mk @@ -1 +1 @@ -SOURCES=duckdb/ub_src_catalog.o duckdb/ub_src_catalog_catalog_entry.o duckdb/ub_src_catalog_default.o duckdb/ub_src_common_adbc.o duckdb/ub_src_common_adbc_nanoarrow.o duckdb/ub_src_common.o duckdb/ub_src_common_arrow_appender.o duckdb/ub_src_common_arrow.o duckdb/ub_src_common_crypto.o duckdb/ub_src_common_enums.o duckdb/ub_src_common_operator.o duckdb/ub_src_common_progress_bar.o duckdb/ub_src_common_row_operations.o duckdb/ub_src_common_serializer.o duckdb/ub_src_common_sort.o duckdb/ub_src_common_types.o duckdb/ub_src_common_types_column.o duckdb/ub_src_common_types_row.o duckdb/ub_src_common_value_operations.o duckdb/src/common/vector_operations/boolean_operators.o duckdb/src/common/vector_operations/comparison_operators.o duckdb/src/common/vector_operations/generators.o duckdb/src/common/vector_operations/is_distinct_from.o duckdb/src/common/vector_operations/null_operations.o duckdb/src/common/vector_operations/numeric_inplace_operators.o duckdb/src/common/vector_operations/vector_cast.o duckdb/src/common/vector_operations/vector_copy.o duckdb/src/common/vector_operations/vector_hash.o duckdb/src/common/vector_operations/vector_storage.o duckdb/ub_src_core_functions_aggregate_algebraic.o duckdb/ub_src_core_functions_aggregate_distributive.o duckdb/ub_src_core_functions_aggregate_holistic.o duckdb/ub_src_core_functions_aggregate_nested.o duckdb/ub_src_core_functions_aggregate_regression.o duckdb/ub_src_core_functions.o duckdb/ub_src_core_functions_scalar_bit.o duckdb/ub_src_core_functions_scalar_blob.o duckdb/ub_src_core_functions_scalar_date.o duckdb/ub_src_core_functions_scalar_debug.o duckdb/ub_src_core_functions_scalar_enum.o duckdb/ub_src_core_functions_scalar_generic.o duckdb/ub_src_core_functions_scalar_list.o duckdb/ub_src_core_functions_scalar_map.o duckdb/ub_src_core_functions_scalar_math.o duckdb/ub_src_core_functions_scalar_operators.o duckdb/ub_src_core_functions_scalar_random.o duckdb/ub_src_core_functions_scalar_string.o duckdb/ub_src_core_functions_scalar_struct.o duckdb/ub_src_core_functions_scalar_union.o duckdb/ub_src_execution.o duckdb/ub_src_execution_expression_executor.o duckdb/ub_src_execution_index_art.o duckdb/ub_src_execution_nested_loop_join.o duckdb/ub_src_execution_operator_aggregate.o duckdb/ub_src_execution_operator_filter.o duckdb/ub_src_execution_operator_helper.o duckdb/ub_src_execution_operator_join.o duckdb/ub_src_execution_operator_order.o duckdb/ub_src_execution_operator_persistent.o duckdb/ub_src_execution_operator_projection.o duckdb/ub_src_execution_operator_scan.o duckdb/ub_src_execution_operator_schema.o duckdb/ub_src_execution_operator_set.o duckdb/ub_src_execution_physical_plan.o duckdb/ub_src_function_aggregate_distributive.o duckdb/ub_src_function_aggregate.o duckdb/ub_src_function.o duckdb/ub_src_function_cast.o duckdb/ub_src_function_pragma.o duckdb/ub_src_function_scalar_compressed_materialization.o duckdb/ub_src_function_scalar.o duckdb/ub_src_function_scalar_generic.o duckdb/ub_src_function_scalar_list.o duckdb/ub_src_function_scalar_operators.o duckdb/ub_src_function_scalar_sequence.o duckdb/ub_src_function_scalar_string.o duckdb/ub_src_function_scalar_string_regexp.o duckdb/ub_src_function_scalar_struct.o duckdb/ub_src_function_scalar_system.o duckdb/ub_src_function_table_arrow.o duckdb/ub_src_function_table.o duckdb/ub_src_function_table_system.o duckdb/ub_src_function_table_version.o duckdb/ub_src_main.o duckdb/ub_src_main_capi.o duckdb/ub_src_main_capi_cast.o duckdb/ub_src_main_chunk_scan_state.o duckdb/ub_src_main_extension.o duckdb/ub_src_main_relation.o duckdb/ub_src_main_settings.o duckdb/ub_src_optimizer.o duckdb/ub_src_optimizer_compressed_materialization.o duckdb/ub_src_optimizer_join_order.o duckdb/ub_src_optimizer_matcher.o duckdb/ub_src_optimizer_pullup.o duckdb/ub_src_optimizer_pushdown.o duckdb/ub_src_optimizer_rule.o duckdb/ub_src_optimizer_statistics_expression.o duckdb/ub_src_optimizer_statistics_operator.o duckdb/ub_src_parallel.o duckdb/ub_src_parser.o duckdb/ub_src_parser_constraints.o duckdb/ub_src_parser_expression.o duckdb/ub_src_parser_parsed_data.o duckdb/ub_src_parser_query_node.o duckdb/ub_src_parser_statement.o duckdb/ub_src_parser_tableref.o duckdb/ub_src_parser_transform_constraint.o duckdb/ub_src_parser_transform_expression.o duckdb/ub_src_parser_transform_helpers.o duckdb/ub_src_parser_transform_statement.o duckdb/ub_src_parser_transform_tableref.o duckdb/ub_src_planner.o duckdb/ub_src_planner_binder_expression.o duckdb/ub_src_planner_binder_query_node.o duckdb/ub_src_planner_binder_statement.o duckdb/ub_src_planner_binder_tableref.o duckdb/ub_src_planner_expression.o duckdb/ub_src_planner_expression_binder.o duckdb/ub_src_planner_filter.o duckdb/ub_src_planner_operator.o duckdb/ub_src_planner_parsed_data.o duckdb/ub_src_planner_subquery.o duckdb/ub_src_storage.o duckdb/ub_src_storage_buffer.o duckdb/ub_src_storage_checkpoint.o duckdb/ub_src_storage_compression.o duckdb/ub_src_storage_compression_chimp.o duckdb/ub_src_storage_metadata.o duckdb/ub_src_storage_serialization.o duckdb/ub_src_storage_statistics.o duckdb/ub_src_storage_table.o duckdb/ub_src_transaction.o duckdb/src/verification/copied_statement_verifier.o duckdb/src/verification/deserialized_statement_verifier.o duckdb/src/verification/deserialized_statement_verifier_v2.o duckdb/src/verification/external_statement_verifier.o duckdb/src/verification/no_operator_caching_verifier.o duckdb/src/verification/parsed_statement_verifier.o duckdb/src/verification/prepared_statement_verifier.o duckdb/src/verification/statement_verifier.o duckdb/src/verification/unoptimized_statement_verifier.o duckdb/third_party/fmt/format.o duckdb/third_party/fsst/fsst_avx512.o duckdb/third_party/fsst/libfsst.o duckdb/third_party/miniz/miniz.o duckdb/third_party/re2/re2/bitstate.o duckdb/third_party/re2/re2/compile.o duckdb/third_party/re2/re2/dfa.o duckdb/third_party/re2/re2/filtered_re2.o duckdb/third_party/re2/re2/mimics_pcre.o duckdb/third_party/re2/re2/nfa.o duckdb/third_party/re2/re2/onepass.o duckdb/third_party/re2/re2/parse.o duckdb/third_party/re2/re2/perl_groups.o duckdb/third_party/re2/re2/prefilter.o duckdb/third_party/re2/re2/prefilter_tree.o duckdb/third_party/re2/re2/prog.o duckdb/third_party/re2/re2/re2.o duckdb/third_party/re2/re2/regexp.o duckdb/third_party/re2/re2/set.o duckdb/third_party/re2/re2/simplify.o duckdb/third_party/re2/re2/stringpiece.o duckdb/third_party/re2/re2/tostring.o duckdb/third_party/re2/re2/unicode_casefold.o duckdb/third_party/re2/re2/unicode_groups.o duckdb/third_party/re2/util/rune.o duckdb/third_party/re2/util/strutil.o duckdb/third_party/hyperloglog/hyperloglog.o duckdb/third_party/hyperloglog/sds.o duckdb/third_party/fastpforlib/bitpacking.o duckdb/third_party/utf8proc/utf8proc.o duckdb/third_party/utf8proc/utf8proc_wrapper.o duckdb/third_party/libpg_query/pg_functions.o duckdb/third_party/libpg_query/postgres_parser.o duckdb/third_party/libpg_query/src_backend_nodes_list.o duckdb/third_party/libpg_query/src_backend_nodes_makefuncs.o duckdb/third_party/libpg_query/src_backend_nodes_value.o duckdb/third_party/libpg_query/src_backend_parser_gram.o duckdb/third_party/libpg_query/src_backend_parser_parser.o duckdb/third_party/libpg_query/src_backend_parser_scan.o duckdb/third_party/libpg_query/src_backend_parser_scansup.o duckdb/third_party/libpg_query/src_common_keywords.o duckdb/third_party/mbedtls/library/asn1parse.o duckdb/third_party/mbedtls/library/base64.o duckdb/third_party/mbedtls/library/bignum.o duckdb/third_party/mbedtls/library/constant_time.o duckdb/third_party/mbedtls/library/md.o duckdb/third_party/mbedtls/library/oid.o duckdb/third_party/mbedtls/library/pem.o duckdb/third_party/mbedtls/library/pk.o duckdb/third_party/mbedtls/library/pk_wrap.o duckdb/third_party/mbedtls/library/pkparse.o duckdb/third_party/mbedtls/library/platform_util.o duckdb/third_party/mbedtls/library/rsa.o duckdb/third_party/mbedtls/library/rsa_alt_helpers.o duckdb/third_party/mbedtls/library/sha1.o duckdb/third_party/mbedtls/library/sha256.o duckdb/third_party/mbedtls/library/sha512.o duckdb/third_party/mbedtls/mbedtls_wrapper.o duckdb/extension/parquet/parquet_extension.o duckdb/extension/parquet/column_writer.o duckdb/extension/parquet/serialize_parquet.o duckdb/extension/parquet/parquet_reader.o duckdb/extension/parquet/parquet_timestamp.o duckdb/extension/parquet/parquet_writer.o duckdb/extension/parquet/column_reader.o duckdb/extension/parquet/parquet_statistics.o duckdb/extension/parquet/parquet_metadata.o duckdb/extension/parquet/zstd_file_system.o duckdb/third_party/parquet/parquet_constants.o duckdb/third_party/parquet/parquet_types.o duckdb/third_party/thrift/thrift/protocol/TProtocol.o duckdb/third_party/thrift/thrift/transport/TTransportException.o duckdb/third_party/thrift/thrift/transport/TBufferTransports.o duckdb/third_party/snappy/snappy.o duckdb/third_party/snappy/snappy-sinksource.o duckdb/third_party/zstd/decompress/zstd_ddict.o duckdb/third_party/zstd/decompress/huf_decompress.o duckdb/third_party/zstd/decompress/zstd_decompress.o duckdb/third_party/zstd/decompress/zstd_decompress_block.o duckdb/third_party/zstd/common/entropy_common.o duckdb/third_party/zstd/common/fse_decompress.o duckdb/third_party/zstd/common/zstd_common.o duckdb/third_party/zstd/common/error_private.o duckdb/third_party/zstd/common/xxhash.o duckdb/third_party/zstd/compress/fse_compress.o duckdb/third_party/zstd/compress/hist.o duckdb/third_party/zstd/compress/huf_compress.o duckdb/third_party/zstd/compress/zstd_compress.o duckdb/third_party/zstd/compress/zstd_compress_literals.o duckdb/third_party/zstd/compress/zstd_compress_sequences.o duckdb/third_party/zstd/compress/zstd_compress_superblock.o duckdb/third_party/zstd/compress/zstd_double_fast.o duckdb/third_party/zstd/compress/zstd_fast.o duckdb/third_party/zstd/compress/zstd_lazy.o duckdb/third_party/zstd/compress/zstd_ldm.o duckdb/third_party/zstd/compress/zstd_opt.o +SOURCES=duckdb/ub_src_catalog.o duckdb/ub_src_catalog_catalog_entry.o duckdb/ub_src_catalog_default.o duckdb/ub_src_common_adbc.o duckdb/ub_src_common_adbc_nanoarrow.o duckdb/ub_src_common.o duckdb/ub_src_common_arrow_appender.o duckdb/ub_src_common_arrow.o duckdb/ub_src_common_crypto.o duckdb/ub_src_common_enums.o duckdb/ub_src_common_operator.o duckdb/ub_src_common_progress_bar.o duckdb/ub_src_common_row_operations.o duckdb/ub_src_common_serializer.o duckdb/ub_src_common_sort.o duckdb/ub_src_common_types.o duckdb/ub_src_common_types_column.o duckdb/ub_src_common_types_row.o duckdb/ub_src_common_value_operations.o duckdb/src/common/vector_operations/boolean_operators.o duckdb/src/common/vector_operations/comparison_operators.o duckdb/src/common/vector_operations/generators.o duckdb/src/common/vector_operations/is_distinct_from.o duckdb/src/common/vector_operations/null_operations.o duckdb/src/common/vector_operations/numeric_inplace_operators.o duckdb/src/common/vector_operations/vector_cast.o duckdb/src/common/vector_operations/vector_copy.o duckdb/src/common/vector_operations/vector_hash.o duckdb/src/common/vector_operations/vector_storage.o duckdb/ub_src_core_functions_aggregate_algebraic.o duckdb/ub_src_core_functions_aggregate_distributive.o duckdb/ub_src_core_functions_aggregate_holistic.o duckdb/ub_src_core_functions_aggregate_nested.o duckdb/ub_src_core_functions_aggregate_regression.o duckdb/ub_src_core_functions.o duckdb/ub_src_core_functions_scalar_bit.o duckdb/ub_src_core_functions_scalar_blob.o duckdb/ub_src_core_functions_scalar_date.o duckdb/ub_src_core_functions_scalar_debug.o duckdb/ub_src_core_functions_scalar_enum.o duckdb/ub_src_core_functions_scalar_generic.o duckdb/ub_src_core_functions_scalar_list.o duckdb/ub_src_core_functions_scalar_map.o duckdb/ub_src_core_functions_scalar_math.o duckdb/ub_src_core_functions_scalar_operators.o duckdb/ub_src_core_functions_scalar_random.o duckdb/ub_src_core_functions_scalar_string.o duckdb/ub_src_core_functions_scalar_struct.o duckdb/ub_src_core_functions_scalar_union.o duckdb/ub_src_execution.o duckdb/ub_src_execution_expression_executor.o duckdb/ub_src_execution_index_art.o duckdb/ub_src_execution_nested_loop_join.o duckdb/ub_src_execution_operator_aggregate.o duckdb/ub_src_execution_operator_csv_scanner.o duckdb/ub_src_execution_operator_csv_scanner_sniffer.o duckdb/ub_src_execution_operator_filter.o duckdb/ub_src_execution_operator_helper.o duckdb/ub_src_execution_operator_join.o duckdb/ub_src_execution_operator_order.o duckdb/ub_src_execution_operator_persistent.o duckdb/ub_src_execution_operator_projection.o duckdb/ub_src_execution_operator_scan.o duckdb/ub_src_execution_operator_schema.o duckdb/ub_src_execution_operator_set.o duckdb/ub_src_execution_physical_plan.o duckdb/ub_src_function_aggregate_distributive.o duckdb/ub_src_function_aggregate.o duckdb/ub_src_function.o duckdb/ub_src_function_cast.o duckdb/ub_src_function_pragma.o duckdb/ub_src_function_scalar_compressed_materialization.o duckdb/ub_src_function_scalar.o duckdb/ub_src_function_scalar_generic.o duckdb/ub_src_function_scalar_list.o duckdb/ub_src_function_scalar_operators.o duckdb/ub_src_function_scalar_sequence.o duckdb/ub_src_function_scalar_string.o duckdb/ub_src_function_scalar_string_regexp.o duckdb/ub_src_function_scalar_struct.o duckdb/ub_src_function_scalar_system.o duckdb/ub_src_function_table_arrow.o duckdb/ub_src_function_table.o duckdb/ub_src_function_table_system.o duckdb/ub_src_function_table_version.o duckdb/ub_src_main.o duckdb/ub_src_main_capi.o duckdb/ub_src_main_capi_cast.o duckdb/ub_src_main_chunk_scan_state.o duckdb/ub_src_main_extension.o duckdb/ub_src_main_relation.o duckdb/ub_src_main_settings.o duckdb/ub_src_optimizer.o duckdb/ub_src_optimizer_compressed_materialization.o duckdb/ub_src_optimizer_join_order.o duckdb/ub_src_optimizer_matcher.o duckdb/ub_src_optimizer_pullup.o duckdb/ub_src_optimizer_pushdown.o duckdb/ub_src_optimizer_rule.o duckdb/ub_src_optimizer_statistics_expression.o duckdb/ub_src_optimizer_statistics_operator.o duckdb/ub_src_parallel.o duckdb/ub_src_parser.o duckdb/ub_src_parser_constraints.o duckdb/ub_src_parser_expression.o duckdb/ub_src_parser_parsed_data.o duckdb/ub_src_parser_query_node.o duckdb/ub_src_parser_statement.o duckdb/ub_src_parser_tableref.o duckdb/ub_src_parser_transform_constraint.o duckdb/ub_src_parser_transform_expression.o duckdb/ub_src_parser_transform_helpers.o duckdb/ub_src_parser_transform_statement.o duckdb/ub_src_parser_transform_tableref.o duckdb/ub_src_planner.o duckdb/ub_src_planner_binder_expression.o duckdb/ub_src_planner_binder_query_node.o duckdb/ub_src_planner_binder_statement.o duckdb/ub_src_planner_binder_tableref.o duckdb/ub_src_planner_expression.o duckdb/ub_src_planner_expression_binder.o duckdb/ub_src_planner_filter.o duckdb/ub_src_planner_operator.o duckdb/ub_src_planner_parsed_data.o duckdb/ub_src_planner_subquery.o duckdb/ub_src_storage.o duckdb/ub_src_storage_buffer.o duckdb/ub_src_storage_checkpoint.o duckdb/ub_src_storage_compression.o duckdb/ub_src_storage_compression_chimp.o duckdb/ub_src_storage_metadata.o duckdb/ub_src_storage_serialization.o duckdb/ub_src_storage_statistics.o duckdb/ub_src_storage_table.o duckdb/ub_src_transaction.o duckdb/src/verification/copied_statement_verifier.o duckdb/src/verification/deserialized_statement_verifier.o duckdb/src/verification/deserialized_statement_verifier_v2.o duckdb/src/verification/external_statement_verifier.o duckdb/src/verification/no_operator_caching_verifier.o duckdb/src/verification/parsed_statement_verifier.o duckdb/src/verification/prepared_statement_verifier.o duckdb/src/verification/statement_verifier.o duckdb/src/verification/unoptimized_statement_verifier.o duckdb/third_party/fmt/format.o duckdb/third_party/fsst/fsst_avx512.o duckdb/third_party/fsst/libfsst.o duckdb/third_party/miniz/miniz.o duckdb/third_party/re2/re2/bitstate.o duckdb/third_party/re2/re2/compile.o duckdb/third_party/re2/re2/dfa.o duckdb/third_party/re2/re2/filtered_re2.o duckdb/third_party/re2/re2/mimics_pcre.o duckdb/third_party/re2/re2/nfa.o duckdb/third_party/re2/re2/onepass.o duckdb/third_party/re2/re2/parse.o duckdb/third_party/re2/re2/perl_groups.o duckdb/third_party/re2/re2/prefilter.o duckdb/third_party/re2/re2/prefilter_tree.o duckdb/third_party/re2/re2/prog.o duckdb/third_party/re2/re2/re2.o duckdb/third_party/re2/re2/regexp.o duckdb/third_party/re2/re2/set.o duckdb/third_party/re2/re2/simplify.o duckdb/third_party/re2/re2/stringpiece.o duckdb/third_party/re2/re2/tostring.o duckdb/third_party/re2/re2/unicode_casefold.o duckdb/third_party/re2/re2/unicode_groups.o duckdb/third_party/re2/util/rune.o duckdb/third_party/re2/util/strutil.o duckdb/third_party/hyperloglog/hyperloglog.o duckdb/third_party/hyperloglog/sds.o duckdb/third_party/fastpforlib/bitpacking.o duckdb/third_party/utf8proc/utf8proc.o duckdb/third_party/utf8proc/utf8proc_wrapper.o duckdb/third_party/libpg_query/pg_functions.o duckdb/third_party/libpg_query/postgres_parser.o duckdb/third_party/libpg_query/src_backend_nodes_list.o duckdb/third_party/libpg_query/src_backend_nodes_makefuncs.o duckdb/third_party/libpg_query/src_backend_nodes_value.o duckdb/third_party/libpg_query/src_backend_parser_gram.o duckdb/third_party/libpg_query/src_backend_parser_parser.o duckdb/third_party/libpg_query/src_backend_parser_scan.o duckdb/third_party/libpg_query/src_backend_parser_scansup.o duckdb/third_party/libpg_query/src_common_keywords.o duckdb/third_party/mbedtls/library/asn1parse.o duckdb/third_party/mbedtls/library/base64.o duckdb/third_party/mbedtls/library/bignum.o duckdb/third_party/mbedtls/library/constant_time.o duckdb/third_party/mbedtls/library/md.o duckdb/third_party/mbedtls/library/oid.o duckdb/third_party/mbedtls/library/pem.o duckdb/third_party/mbedtls/library/pk.o duckdb/third_party/mbedtls/library/pk_wrap.o duckdb/third_party/mbedtls/library/pkparse.o duckdb/third_party/mbedtls/library/platform_util.o duckdb/third_party/mbedtls/library/rsa.o duckdb/third_party/mbedtls/library/rsa_alt_helpers.o duckdb/third_party/mbedtls/library/sha1.o duckdb/third_party/mbedtls/library/sha256.o duckdb/third_party/mbedtls/library/sha512.o duckdb/third_party/mbedtls/mbedtls_wrapper.o duckdb/extension/parquet/parquet_extension.o duckdb/extension/parquet/column_writer.o duckdb/extension/parquet/serialize_parquet.o duckdb/extension/parquet/parquet_reader.o duckdb/extension/parquet/parquet_timestamp.o duckdb/extension/parquet/parquet_writer.o duckdb/extension/parquet/column_reader.o duckdb/extension/parquet/parquet_statistics.o duckdb/extension/parquet/parquet_metadata.o duckdb/extension/parquet/zstd_file_system.o duckdb/third_party/parquet/parquet_constants.o duckdb/third_party/parquet/parquet_types.o duckdb/third_party/thrift/thrift/protocol/TProtocol.o duckdb/third_party/thrift/thrift/transport/TTransportException.o duckdb/third_party/thrift/thrift/transport/TBufferTransports.o duckdb/third_party/snappy/snappy.o duckdb/third_party/snappy/snappy-sinksource.o duckdb/third_party/zstd/decompress/zstd_ddict.o duckdb/third_party/zstd/decompress/huf_decompress.o duckdb/third_party/zstd/decompress/zstd_decompress.o duckdb/third_party/zstd/decompress/zstd_decompress_block.o duckdb/third_party/zstd/common/entropy_common.o duckdb/third_party/zstd/common/fse_decompress.o duckdb/third_party/zstd/common/zstd_common.o duckdb/third_party/zstd/common/error_private.o duckdb/third_party/zstd/common/xxhash.o duckdb/third_party/zstd/compress/fse_compress.o duckdb/third_party/zstd/compress/hist.o duckdb/third_party/zstd/compress/huf_compress.o duckdb/third_party/zstd/compress/zstd_compress.o duckdb/third_party/zstd/compress/zstd_compress_literals.o duckdb/third_party/zstd/compress/zstd_compress_sequences.o duckdb/third_party/zstd/compress/zstd_compress_superblock.o duckdb/third_party/zstd/compress/zstd_double_fast.o duckdb/third_party/zstd/compress/zstd_fast.o duckdb/third_party/zstd/compress/zstd_lazy.o duckdb/third_party/zstd/compress/zstd_ldm.o duckdb/third_party/zstd/compress/zstd_opt.o