diff --git a/src/Storages/MergeTree/MergeList.cpp b/src/Storages/MergeTree/MergeList.cpp index 5afe6e25136..8fbb163384e 100644 --- a/src/Storages/MergeTree/MergeList.cpp +++ b/src/Storages/MergeTree/MergeList.cpp @@ -37,7 +37,6 @@ MergeListElement::MergeListElement(const StorageID & table_id_, FutureMergedMuta source_data_version = future_part->parts[0]->info.getDataVersion(); is_mutation = (result_part_info.getDataVersion() != source_data_version); - // partition = "jianfei-debugging-sigsev"; WriteBufferFromString out(partition); const auto & part = future_part->parts[0]; part->partition.serializeText(part->storage, out, {}); diff --git a/src/Storages/MergeTree/MergeTreePartition.cpp b/src/Storages/MergeTree/MergeTreePartition.cpp index bce33438229..b8d1879cd67 100644 --- a/src/Storages/MergeTree/MergeTreePartition.cpp +++ b/src/Storages/MergeTree/MergeTreePartition.cpp @@ -1,20 +1,20 @@ -#include -#include -#include -#include -#include -#include +#include +#include #include #include #include -#include -#include -#include -#include -#include +#include +#include +#include +#include +#include #include -#include - +#include "Common/logger_useful.h" +#include +#include +#include +#include +#include namespace DB { @@ -38,51 +38,52 @@ namespace { private: SipHash & hash; - public: - explicit LegacyFieldVisitorHash(SipHash & hash_) : hash(hash_) {} - void operator() (const Null &) const + public: + explicit LegacyFieldVisitorHash(SipHash & hash_) : hash(hash_) { } + + void operator()(const Null &) const { UInt8 type = Field::Types::Null; hash.update(type); } - void operator() (const UInt64 & x) const + void operator()(const UInt64 & x) const { UInt8 type = Field::Types::UInt64; hash.update(type); hash.update(x); } - void operator() (const UInt128 & x) const + void operator()(const UInt128 & x) const { UInt8 type = Field::Types::UInt128; hash.update(type); hash.update(x); } - void operator() (const UInt256 & x) const + void operator()(const UInt256 & x) const { UInt8 type = Field::Types::UInt256; hash.update(type); hash.update(x); } - void operator() (const Int64 & x) const + void operator()(const Int64 & x) const { UInt8 type = Field::Types::Int64; hash.update(type); hash.update(x); } - void operator() (const Int128 & x) const + void operator()(const Int128 & x) const { UInt8 type = Field::Types::Int128; hash.update(type); hash.update(x); } - void operator() (const Int256 & x) const + void operator()(const Int256 & x) const { UInt8 type = Field::Types::Int256; hash.update(type); hash.update(x); } - void operator() (const UUID & x) const + void operator()(const UUID & x) const { #if __BYTE_ORDER__ == __ORDER_BIG_ENDIAN__ auto tmp_x = x.toUnderType(); @@ -94,30 +95,27 @@ namespace operator()(x.toUnderType()); #endif } - void operator() (const IPv4 & x) const + void operator()(const IPv4 & x) const { UInt8 type = Field::Types::IPv4; hash.update(type); hash.update(x); } - void operator() (const IPv6 & x) const - { - return operator()(String(reinterpret_cast(&x), 16)); - } - void operator() (const Float64 & x) const + void operator()(const IPv6 & x) const { return operator()(String(reinterpret_cast(&x), 16)); } + void operator()(const Float64 & x) const { UInt8 type = Field::Types::Float64; hash.update(type); hash.update(x); } - void operator() (const String & x) const + void operator()(const String & x) const { UInt8 type = Field::Types::String; hash.update(type); hash.update(x.size()); hash.update(x.data(), x.size()); } - void operator() (const Array & x) const + void operator()(const Array & x) const { UInt8 type = Field::Types::Array; hash.update(type); @@ -126,7 +124,7 @@ namespace for (const auto & elem : x) applyVisitor(*this, elem); } - void operator() (const Tuple & x) const + void operator()(const Tuple & x) const { UInt8 type = Field::Types::Tuple; hash.update(type); @@ -135,7 +133,7 @@ namespace for (const auto & elem : x) applyVisitor(*this, elem); } - void operator() (const Map & x) const + void operator()(const Map & x) const { UInt8 type = Field::Types::Map; hash.update(type); @@ -144,43 +142,43 @@ namespace for (const auto & elem : x) applyVisitor(*this, elem); } - void operator() (const Object & x) const + void operator()(const Object & x) const { UInt8 type = Field::Types::Object; hash.update(type); hash.update(x.size()); - for (const auto & [key, value]: x) + for (const auto & [key, value] : x) { hash.update(key); applyVisitor(*this, value); } } - void operator() (const DecimalField & x) const + void operator()(const DecimalField & x) const { UInt8 type = Field::Types::Decimal32; hash.update(type); hash.update(x.getValue().value); } - void operator() (const DecimalField & x) const + void operator()(const DecimalField & x) const { UInt8 type = Field::Types::Decimal64; hash.update(type); hash.update(x.getValue().value); } - void operator() (const DecimalField & x) const + void operator()(const DecimalField & x) const { UInt8 type = Field::Types::Decimal128; hash.update(type); hash.update(x.getValue().value); } - void operator() (const DecimalField & x) const + void operator()(const DecimalField & x) const { UInt8 type = Field::Types::Decimal256; hash.update(type); hash.update(x.getValue().value); } - void operator() (const AggregateFunctionStateData & x) const + void operator()(const AggregateFunctionStateData & x) const { UInt8 type = Field::Types::AggregateFunctionState; hash.update(type); @@ -189,7 +187,7 @@ namespace hash.update(x.data.size()); hash.update(x.data.data(), x.data.size()); } - void operator() (const CustomType & x) const + void operator()(const CustomType & x) const { UInt8 type = Field::Types::CustomType; hash.update(type); @@ -198,7 +196,7 @@ namespace hash.update(result.size()); hash.update(result.data(), result.size()); } - void operator() (const bool & x) const + void operator()(const bool & x) const { UInt8 type = Field::Types::Bool; hash.update(type); @@ -291,7 +289,12 @@ std::optional MergeTreePartition::tryParseValueFromID(const String & partit return res; } - enum KeyType { DATE, UNSIGNED, SIGNED }; + enum KeyType + { + DATE, + UNSIGNED, + SIGNED + }; std::vector key_types; key_types.reserve(num_keys); @@ -321,8 +324,7 @@ std::optional MergeTreePartition::tryParseValueFromID(const String & partit { switch (key_types[i]) { - case DATE: - { + case DATE: { UInt32 date_yyyymmdd; readText(date_yyyymmdd, buf); constexpr UInt32 min_yyyymmdd = 10000000; @@ -335,15 +337,13 @@ std::optional MergeTreePartition::tryParseValueFromID(const String & partit res.emplace_back(date); break; } - case UNSIGNED: - { + case UNSIGNED: { UInt64 value; readText(value, buf); res.emplace_back(value); break; } - case SIGNED: - { + case SIGNED: { Int64 value; readText(value, buf); res.emplace_back(value); @@ -359,8 +359,8 @@ std::optional MergeTreePartition::tryParseValueFromID(const String & partit String expected_partition_id = MergeTreePartition{res}.getID(partition_key_sample); if (expected_partition_id != partition_id) - throw Exception(ErrorCodes::LOGICAL_ERROR, "Partition ID was parsed incorrectly: expected {}, got {}", - expected_partition_id, partition_id); + throw Exception( + ErrorCodes::LOGICAL_ERROR, "Partition ID was parsed incorrectly: expected {}, got {}", expected_partition_id, partition_id); return res; } @@ -379,8 +379,38 @@ void MergeTreePartition::serializeText(const MergeTreeData & storage, WriteBuffe { const DataTypePtr & type = partition_key_sample.getByPosition(0).type; auto column = type->createColumn(); - column->insert(value[0]); - type->getDefaultSerialization()->serializeText(*column, 0, out, format_settings); + // metadata_snapshot-> + // TODO: here print out the type that causing the segfault. + const auto & all_column_names = metadata_snapshot->getColumns().getAll(); + // if (type.get() != nullptr) + // { + // LOG_INFO(&Poco::Logger::get("DNSCacheUpdater"), "jianfeih debug type is not nil, name {}.", type->getName()); + // } + if (value.empty()) + { + LOG_INFO(&Poco::Logger::get("DNSCacheUpdater"), "jianfeih debug: find the root cause, value is empty"); + writeCString("tuple()", out); + LOG_INFO( + &Poco::Logger::get("DNSCacheUpdater"), + "jianfeih debug: type is nil: {}, colum is nil: {}, storage has partition key: {}, column size: {}, " + "column names: {} ", + type.get() == nullptr, + column == nullptr, + metadata_snapshot->hasPartitionKey(), + metadata_snapshot->columns.size(), + all_column_names.toString()); + return; + } + else + { + // LOG_INFO(&Poco::Logger::get("DNSCacheUpdater"), "jianfeih debug: value[0] to be inserted: {}", value[0]); + // should be here causing the segfault. column might be empty. some type created column is not okay. + // let's check the table definition; type; parts info at this point. + // logging is not right, need to figure out include header properly. + column->insert(value[0]); + type->getDefaultSerialization()->serializeText(*column, 0, out, format_settings); + // LOG_INFO(&Poco::Logger::get("DNSCacheUpdater"), "jianfeih debug: finish the serialization no error."); + } } else { @@ -415,7 +445,8 @@ void MergeTreePartition::load(const MergeTreeData & storage, const PartMetadataM partition_key_sample.getByPosition(i).type->getDefaultSerialization()->deserializeBinary(value[i], *file, {}); } -std::unique_ptr MergeTreePartition::store(const MergeTreeData & storage, IDataPartStorage & data_part_storage, MergeTreeDataPartChecksums & checksums) const +std::unique_ptr +MergeTreePartition::store(const MergeTreeData & storage, IDataPartStorage & data_part_storage, MergeTreeDataPartChecksums & checksums) const { auto metadata_snapshot = storage.getInMemoryMetadataPtr(); const auto & context = storage.getContext(); @@ -423,7 +454,11 @@ std::unique_ptr MergeTreePartition::store(const MergeTr return store(partition_key_sample, data_part_storage, checksums, context->getWriteSettings()); } -std::unique_ptr MergeTreePartition::store(const Block & partition_key_sample, IDataPartStorage & data_part_storage, MergeTreeDataPartChecksums & checksums, const WriteSettings & settings) const +std::unique_ptr MergeTreePartition::store( + const Block & partition_key_sample, + IDataPartStorage & data_part_storage, + MergeTreeDataPartChecksums & checksums, + const WriteSettings & settings) const { if (!partition_key_sample) return nullptr; @@ -469,7 +504,8 @@ void MergeTreePartition::create(const StorageMetadataPtr & metadata_snapshot, Bl } } -NamesAndTypesList MergeTreePartition::executePartitionByExpression(const StorageMetadataPtr & metadata_snapshot, Block & block, ContextPtr context) +NamesAndTypesList +MergeTreePartition::executePartitionByExpression(const StorageMetadataPtr & metadata_snapshot, Block & block, ContextPtr context) { auto adjusted_partition_key = adjustPartitionKey(metadata_snapshot, context); adjusted_partition_key.expression->execute(block); @@ -496,7 +532,7 @@ KeyDescription MergeTreePartition::adjustPartitionKey(const StorageMetadataPtr & } -void MergeTreePartition::appendFiles(const MergeTreeData & storage, Strings& files) +void MergeTreePartition::appendFiles(const MergeTreeData & storage, Strings & files) { auto metadata_snapshot = storage.getInMemoryMetadataPtr(); if (!metadata_snapshot->hasPartitionKey()) diff --git a/src/Storages/MergeTree/MergeTreePartition.h b/src/Storages/MergeTree/MergeTreePartition.h index 78b141f26ec..855f912877f 100644 --- a/src/Storages/MergeTree/MergeTreePartition.h +++ b/src/Storages/MergeTree/MergeTreePartition.h @@ -1,11 +1,12 @@ #pragma once -#include +#include #include #include #include #include -#include +#include +#include namespace DB { @@ -28,10 +29,10 @@ struct MergeTreePartition public: MergeTreePartition() = default; - explicit MergeTreePartition(Row value_) : value(std::move(value_)) {} + explicit MergeTreePartition(Row value_) : value(std::move(value_)) { } /// For month-based partitioning. - explicit MergeTreePartition(UInt32 yyyymm) : value(1, yyyymm) {} + explicit MergeTreePartition(UInt32 yyyymm) : value(1, yyyymm) { } String getID(const MergeTreeData & storage) const; String getID(const Block & partition_key_sample) const; @@ -44,8 +45,13 @@ public: /// Store functions return write buffer with written but not finalized data. /// User must call finish() for returned object. - [[nodiscard]] std::unique_ptr store(const MergeTreeData & storage, IDataPartStorage & data_part_storage, MergeTreeDataPartChecksums & checksums) const; - [[nodiscard]] std::unique_ptr store(const Block & partition_key_sample, IDataPartStorage & data_part_storage, MergeTreeDataPartChecksums & checksums, const WriteSettings & settings) const; + [[nodiscard]] std::unique_ptr + store(const MergeTreeData & storage, IDataPartStorage & data_part_storage, MergeTreeDataPartChecksums & checksums) const; + [[nodiscard]] std::unique_ptr store( + const Block & partition_key_sample, + IDataPartStorage & data_part_storage, + MergeTreeDataPartChecksums & checksums, + const WriteSettings & settings) const; void assign(const MergeTreePartition & other) { value = other.value; }