From bda40b5a98974a9d78ca055855215b01bba546dc Mon Sep 17 00:00:00 2001 From: Filipp Bakanov Date: Thu, 14 Sep 2023 17:36:21 +0000 Subject: [PATCH 001/417] Fixed crash while using override + null field with MaterializedMySQL, #54647 --- src/Databases/MySQL/MaterializedMySQLSyncThread.cpp | 10 +++++++++- 1 file changed, 9 insertions(+), 1 deletion(-) diff --git a/src/Databases/MySQL/MaterializedMySQLSyncThread.cpp b/src/Databases/MySQL/MaterializedMySQLSyncThread.cpp index 3578362b8dd..a0edbfd6ce2 100644 --- a/src/Databases/MySQL/MaterializedMySQLSyncThread.cpp +++ b/src/Databases/MySQL/MaterializedMySQLSyncThread.cpp @@ -577,8 +577,16 @@ static void writeFieldsToColumn( } null_map_column->insertValue(0); + } else { + // Column is not null but field is null. It's possible due to overrides + if (field.isNull()) + { + column_to.insertDefault(); + return false; + } } + return true; }; @@ -653,7 +661,7 @@ static void writeFieldsToColumn( if (write_data_to_null_map(value, index)) { - const String & data = value.get(); + const String & data = value.safeGet(); casted_string_column->insertData(data.data(), data.size()); } } From a46f2826ecd933dc5aa9f6243f24c1b35a5a04ca Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Thu, 21 Dec 2023 06:49:36 +0300 Subject: [PATCH 002/417] Update MaterializedMySQLSyncThread.cpp --- src/Databases/MySQL/MaterializedMySQLSyncThread.cpp | 4 +++- 1 file changed, 3 insertions(+), 1 deletion(-) diff --git a/src/Databases/MySQL/MaterializedMySQLSyncThread.cpp b/src/Databases/MySQL/MaterializedMySQLSyncThread.cpp index a0edbfd6ce2..772ce09f0eb 100644 --- a/src/Databases/MySQL/MaterializedMySQLSyncThread.cpp +++ b/src/Databases/MySQL/MaterializedMySQLSyncThread.cpp @@ -577,7 +577,9 @@ static void writeFieldsToColumn( } null_map_column->insertValue(0); - } else { + } + else + { // Column is not null but field is null. It's possible due to overrides if (field.isNull()) { From 517191218fbc8c780fc218ac059b9095d1670752 Mon Sep 17 00:00:00 2001 From: avogar Date: Thu, 18 Apr 2024 22:33:09 +0000 Subject: [PATCH 003/417] Add compact mode in Variant discriminators serialization --- src/DataTypes/DataTypeVariant.h | 1 + src/DataTypes/Serializations/ISerialization.h | 2 + .../Serializations/SerializationVariant.cpp | 323 +++++++++++++++--- .../Serializations/SerializationVariant.h | 65 ++++ .../SerializationVariantElement.cpp | 135 ++++++-- .../SerializationVariantElement.h | 9 + .../MergeTreeDataPartWriterCompact.cpp | 6 +- .../MergeTree/MergeTreeDataPartWriterWide.cpp | 5 +- .../MergeTree/MergeTreeReaderWide.cpp | 23 +- src/Storages/MergeTree/MergeTreeReaderWide.h | 5 +- src/Storages/MergeTree/MergeTreeSettings.h | 1 + 11 files changed, 483 insertions(+), 92 deletions(-) diff --git a/src/DataTypes/DataTypeVariant.h b/src/DataTypes/DataTypeVariant.h index dadc85ac3b3..ab471d37b2f 100644 --- a/src/DataTypes/DataTypeVariant.h +++ b/src/DataTypes/DataTypeVariant.h @@ -42,6 +42,7 @@ public: bool equals(const IDataType & rhs) const override; bool isParametric() const override { return true; } + bool isComparable() const override { return true; } bool haveSubtypes() const override { return true; } bool textCanContainOnlyValidUTF8() const override; bool haveMaximumSizeOfValue() const override; diff --git a/src/DataTypes/Serializations/ISerialization.h b/src/DataTypes/Serializations/ISerialization.h index ebaa26d19a6..934a92ecfba 100644 --- a/src/DataTypes/Serializations/ISerialization.h +++ b/src/DataTypes/Serializations/ISerialization.h @@ -240,6 +240,8 @@ public: bool low_cardinality_use_single_dictionary_for_part = true; bool position_independent_encoding = true; + + bool use_compact_variant_discriminators_serialization = false; }; struct DeserializeBinaryBulkSettings diff --git a/src/DataTypes/Serializations/SerializationVariant.cpp b/src/DataTypes/Serializations/SerializationVariant.cpp index 8ca86c63bf6..0490ed62c3b 100644 --- a/src/DataTypes/Serializations/SerializationVariant.cpp +++ b/src/DataTypes/Serializations/SerializationVariant.cpp @@ -70,14 +70,25 @@ void SerializationVariant::enumerateStreams( settings.path.pop_back(); } -struct SerializeBinaryBulkStateVariant : public ISerialization::SerializeBinaryBulkState +struct SerializationVariant::SerializeBinaryBulkStateVariant : public ISerialization::SerializeBinaryBulkState { + SerializeBinaryBulkStateVariant(UInt64 mode) : discriminators_mode(mode) + { + } + + SerializationVariant::DiscriminatorsSerializationMode discriminators_mode; std::vector states; }; -struct DeserializeBinaryBulkStateVariant : public ISerialization::DeserializeBinaryBulkState +struct SerializationVariant::DeserializeBinaryBulkStateVariant : public ISerialization::DeserializeBinaryBulkState { + DeserializeBinaryBulkStateVariant(UInt64 mode) : discriminators_mode(mode) + { + } + + SerializationVariant::DiscriminatorsSerializationMode discriminators_mode; std::vector states; + SerializationVariant::DiscriminatorsDeserializationState discriminators_state; }; void SerializationVariant::serializeBinaryBulkStatePrefix( @@ -85,9 +96,18 @@ void SerializationVariant::serializeBinaryBulkStatePrefix( SerializeBinaryBulkSettings & settings, SerializeBinaryBulkStatePtr & state) const { - const ColumnVariant & col = assert_cast(column); + settings.path.push_back(Substream::VariantDiscriminators); + auto * discriminators_stream = settings.getter(settings.path); + settings.path.pop_back(); - auto variant_state = std::make_shared(); + if (!discriminators_stream) + throw Exception(ErrorCodes::LOGICAL_ERROR, "Got empty stream for VariantDiscriminators in SerializationVariant::serializeBinaryBulkStatePrefix"); + + UInt64 mode = settings.use_compact_variant_discriminators_serialization ? DiscriminatorsSerializationMode::COMPACT : DiscriminatorsSerializationMode::BASIC; + writeBinaryLittleEndian(mode, *discriminators_stream); + + const ColumnVariant & col = assert_cast(column); + auto variant_state = std::make_shared(mode); variant_state->states.resize(variants.size()); settings.path.push_back(Substream::VariantElements); @@ -125,7 +145,16 @@ void SerializationVariant::deserializeBinaryBulkStatePrefix( DeserializeBinaryBulkSettings & settings, DeserializeBinaryBulkStatePtr & state) const { - auto variant_state = std::make_shared(); + settings.path.push_back(Substream::VariantDiscriminators); + auto * discriminators_stream = settings.getter(settings.path); + settings.path.pop_back(); + + if (!discriminators_stream) + return; + + UInt64 mode; + readBinaryLittleEndian(mode, *discriminators_stream); + auto variant_state = std::make_shared(mode); variant_state->states.resize(variants.size()); settings.path.push_back(Substream::VariantElements); @@ -161,13 +190,66 @@ void SerializationVariant::serializeBinaryBulkWithMultipleStreams( auto * variant_state = checkAndGetState(state); - /// If offset = 0 and limit == col.size() or we have only NULLs, we don't need to calculate + /// Write number of rows in this granule in compact mode. + if (variant_state->discriminators_mode.value == DiscriminatorsSerializationMode::COMPACT) + writeVarUInt(UInt64(limit), *discriminators_stream); + + /// If column has only one none empty discriminators and no NULLs we don't need to + /// calculate limits for variants and use provided offset/limit. + if (auto non_empty_local_discr = col.getLocalDiscriminatorOfOneNoneEmptyVariantNoNulls()) + { + auto non_empty_global_discr = col.globalDiscriminatorByLocal(*non_empty_local_discr); + + /// In compact mode write the format of the granule and single non-empty discriminator. + if (variant_state->discriminators_mode.value == DiscriminatorsSerializationMode::COMPACT) + { + writeBinaryLittleEndian(UInt8(CompactDiscriminatorsGranuleFormat::COMPACT), *discriminators_stream); + writeBinaryLittleEndian(non_empty_global_discr, *discriminators_stream); + } + /// For basic mode just serialize this discriminator limit times. + else + { + for (size_t i = 0; i < limit; ++i) + writeBinaryLittleEndian(non_empty_global_discr, *discriminators_stream); + } + + settings.path.push_back(Substream::VariantElements); + addVariantElementToPath(settings.path, non_empty_global_discr); + /// We can use the same offset/limit as for whole Variant column + variants[non_empty_global_discr]->serializeBinaryBulkWithMultipleStreams(col.getVariantByGlobalDiscriminator(non_empty_global_discr), offset, limit, settings, variant_state->states[non_empty_global_discr]); + settings.path.pop_back(); + settings.path.pop_back(); + return; + } + /// If column has only NULLs, just serialize NULL discriminators. + else if (col.hasOnlyNulls()) + { + /// In compact mode write single NULL_DISCRIMINATOR. + if (variant_state->discriminators_mode.value == DiscriminatorsSerializationMode::COMPACT) + { + writeBinaryLittleEndian(UInt8(CompactDiscriminatorsGranuleFormat::COMPACT), *discriminators_stream); + writeBinaryLittleEndian(ColumnVariant::NULL_DISCRIMINATOR, *discriminators_stream); + } + /// In basic mode write NULL_DISCRIMINATOR limit times. + else + { + for (size_t i = 0; i < limit; ++i) + writeBinaryLittleEndian(ColumnVariant::NULL_DISCRIMINATOR, *discriminators_stream); + } + return; + } + + /// If offset = 0 and limit == col.size() we don't need to calculate /// offsets and limits for variants and need to just serialize whole columns. - if ((offset == 0 && limit == col.size()) || col.hasOnlyNulls()) + if ((offset == 0 && limit == col.size())) { /// First, serialize discriminators. - /// If we have only NULLs or local and global discriminators are the same, just serialize the column as is. - if (col.hasOnlyNulls() || col.hasGlobalVariantsOrder()) + /// Here we are sure that column contains different discriminators, use plain granule format in compact mode. + if (variant_state->discriminators_mode.value == DiscriminatorsSerializationMode::COMPACT) + writeBinaryLittleEndian(UInt8(CompactDiscriminatorsGranuleFormat::PLAIN), *discriminators_stream); + + /// If local and global discriminators are the same, just serialize the column as is. + if (col.hasGlobalVariantsOrder()) { SerializationNumber().serializeBinaryBulk(col.getLocalDiscriminatorsColumn(), *discriminators_stream, offset, limit); } @@ -191,35 +273,16 @@ void SerializationVariant::serializeBinaryBulkWithMultipleStreams( return; } - /// If we have only one non empty variant and no NULLs, we can use the same limit offset for this variant. - if (auto non_empty_local_discr = col.getLocalDiscriminatorOfOneNoneEmptyVariantNoNulls()) - { - /// First, serialize discriminators. - /// We know that all discriminators are the same, so we just need to serialize this discriminator limit times. - auto non_empty_global_discr = col.globalDiscriminatorByLocal(*non_empty_local_discr); - for (size_t i = 0; i != limit; ++i) - writeBinaryLittleEndian(non_empty_global_discr, *discriminators_stream); - - /// Second, serialize non-empty variant (other variants are empty and we can skip their serialization). - settings.path.push_back(Substream::VariantElements); - addVariantElementToPath(settings.path, non_empty_global_discr); - /// We can use the same offset/limit as for whole Variant column - variants[non_empty_global_discr]->serializeBinaryBulkWithMultipleStreams(col.getVariantByGlobalDiscriminator(non_empty_global_discr), offset, limit, settings, variant_state->states[non_empty_global_discr]); - settings.path.pop_back(); - settings.path.pop_back(); - return; - } - /// In general case we should iterate through local discriminators in range [offset, offset + limit] to serialize global discriminators and calculate offset/limit pair for each variant. const auto & local_discriminators = col.getLocalDiscriminators(); const auto & offsets = col.getOffsets(); std::vector> variant_offsets_and_limits(variants.size(), {0, 0}); size_t end = offset + limit; + size_t num_non_empty_variants_in_range = 0; + ColumnVariant::Discriminator last_non_empty_variant_discr = 0; for (size_t i = offset; i < end; ++i) { auto global_discr = col.globalDiscriminatorByLocal(local_discriminators[i]); - writeBinaryLittleEndian(global_discr, *discriminators_stream); - if (global_discr != ColumnVariant::NULL_DISCRIMINATOR) { /// If we see this discriminator for the first time, update offset @@ -227,9 +290,38 @@ void SerializationVariant::serializeBinaryBulkWithMultipleStreams( variant_offsets_and_limits[global_discr].first = offsets[i]; /// Update limit for this discriminator. ++variant_offsets_and_limits[global_discr].second; + ++num_non_empty_variants_in_range; + last_non_empty_variant_discr = global_discr; } } + /// In basic mode just serialize discriminators as is row by row. + if (variant_state->discriminators_mode.value == DiscriminatorsSerializationMode::BASIC) + { + for (size_t i = offset; i < end; ++i) + writeBinaryLittleEndian(col.globalDiscriminatorByLocal(local_discriminators[i]), *discriminators_stream); + } + /// In compact mode check if we have the same discriminator for all rows in this granule. + /// First, check if all values in granule are NULLs. + else if (num_non_empty_variants_in_range == 0) + { + writeBinaryLittleEndian(UInt8(CompactDiscriminatorsGranuleFormat::COMPACT), *discriminators_stream); + writeBinaryLittleEndian(ColumnVariant::NULL_DISCRIMINATOR, *discriminators_stream); + } + /// Then, check if there is only 1 variant and no NULLs in this granule. + else if (num_non_empty_variants_in_range == 1 && variant_offsets_and_limits[last_non_empty_variant_discr].second == limit) + { + writeBinaryLittleEndian(UInt8(CompactDiscriminatorsGranuleFormat::COMPACT), *discriminators_stream); + writeBinaryLittleEndian(last_non_empty_variant_discr, *discriminators_stream); + } + /// Otherwise there are different discriminators in this granule. + else + { + writeBinaryLittleEndian(UInt8(CompactDiscriminatorsGranuleFormat::PLAIN), *discriminators_stream); + for (size_t i = offset; i < end; ++i) + writeBinaryLittleEndian(col.globalDiscriminatorByLocal(local_discriminators[i]), *discriminators_stream); + } + /// Serialize variants in global order. settings.path.push_back(Substream::VariantElements); for (size_t i = 0; i != variants.size(); ++i) @@ -267,34 +359,62 @@ void SerializationVariant::deserializeBinaryBulkWithMultipleStreams( /// First, deserialize discriminators. settings.path.push_back(Substream::VariantDiscriminators); + + DeserializeBinaryBulkStateVariant * variant_state = nullptr; + std::vector variant_limits; if (auto cached_discriminators = getFromSubstreamsCache(cache, settings.path)) { + variant_state = checkAndGetState(state); col.getLocalDiscriminatorsPtr() = cached_discriminators; } - else + else if (auto * discriminators_stream = settings.getter(settings.path)) { - auto * discriminators_stream = settings.getter(settings.path); - if (!discriminators_stream) - return; + variant_state = checkAndGetState(state); + + /// Deserialize discriminators according to serialization mode. + if (variant_state->discriminators_mode.value == DiscriminatorsSerializationMode::BASIC) + SerializationNumber().deserializeBinaryBulk(*col.getLocalDiscriminatorsPtr()->assumeMutable(), *discriminators_stream, limit, 0); + else + variant_limits = deserializeCompactDiscriminators(col.getLocalDiscriminatorsPtr(), limit, discriminators_stream, settings.continuous_reading, variant_state->discriminators_state); - SerializationNumber().deserializeBinaryBulk(*col.getLocalDiscriminatorsPtr()->assumeMutable(), *discriminators_stream, limit, 0); addToSubstreamsCache(cache, settings.path, col.getLocalDiscriminatorsPtr()); } + /// It may happen that there is no such stream, in this case just do nothing. + else + { + settings.path.pop_back(); + return; + } + settings.path.pop_back(); - /// Second, calculate limits for each variant by iterating through new discriminators. - std::vector variant_limits(variants.size(), 0); - auto & discriminators_data = col.getLocalDiscriminators(); - size_t discriminators_offset = discriminators_data.size() - limit; - for (size_t i = discriminators_offset; i != discriminators_data.size(); ++i) + /// Second, calculate limits for each variant by iterating through new discriminators + /// if we didn't do it during discriminators deserialization. + if (variant_limits.empty()) { - ColumnVariant::Discriminator discr = discriminators_data[i]; - if (discr != ColumnVariant::NULL_DISCRIMINATOR) - ++variant_limits[discr]; + variant_limits.resize(variants.size(), 0); + auto & discriminators_data = col.getLocalDiscriminators(); + + /// We can actually read less than limit discriminators and we cannot determine the actual number of read rows + /// by discriminators column as it could be taken from the substreams cache. And we need actual number of read + /// rows to fill offsets correctly later if they are not in the cache. We can determine if offsets column is in cache + /// or not by comparing it with discriminators column size (they should be the same when offsets are in cache). + /// If offsets are not in the cache, we can use it's size to determine the actual number of read rows. + size_t num_new_discriminators = limit; + size_t offsets_size = col.getOffsetsPtr()->size(); + if (discriminators_data.size() > offsets_size) + num_new_discriminators = discriminators_data.size() - offsets_size; + size_t discriminators_offset = discriminators_data.size() - num_new_discriminators; + + for (size_t i = discriminators_offset; i != discriminators_data.size(); ++i) + { + ColumnVariant::Discriminator discr = discriminators_data[i]; + if (discr != ColumnVariant::NULL_DISCRIMINATOR) + ++variant_limits[discr]; + } } /// Now we can deserialize variants according to their limits. - auto * variant_state = checkAndGetState(state); settings.path.push_back(Substream::VariantElements); for (size_t i = 0; i != variants.size(); ++i) { @@ -319,20 +439,49 @@ void SerializationVariant::deserializeBinaryBulkWithMultipleStreams( } else { - auto & offsets = col.getOffsets(); - offsets.reserve(offsets.size() + limit); std::vector variant_offsets; variant_offsets.reserve(variants.size()); + size_t num_non_empty_variants = 0; + ColumnVariant::Discriminator last_non_empty_discr = 0; for (size_t i = 0; i != variants.size(); ++i) - variant_offsets.push_back(col.getVariantByLocalDiscriminator(i).size() - variant_limits[i]); - - for (size_t i = discriminators_offset; i != discriminators_data.size(); ++i) { - ColumnVariant::Discriminator discr = discriminators_data[i]; - if (discr == ColumnVariant::NULL_DISCRIMINATOR) - offsets.emplace_back(); - else - offsets.push_back(variant_offsets[discr]++); + if (variant_limits[i]) + { + ++num_non_empty_variants; + last_non_empty_discr = i; + } + + variant_offsets.push_back(col.getVariantByLocalDiscriminator(i).size() - variant_limits[i]); + } + + auto & discriminators_data = col.getLocalDiscriminators(); + auto & offsets = col.getOffsets(); + size_t num_new_offsets = discriminators_data.size() - offsets.size(); + offsets.reserve(offsets.size() + num_new_offsets); + /// If there are only NULLs were read, fill offsets with 0. + if (num_non_empty_variants == 0) + { + offsets.resize_fill(discriminators_data.size(), 0); + } + /// If there is only 1 variant and no NULLs was read, fill offsets with sequential offsets of this variant. + else if (num_non_empty_variants == 1 && variant_limits[last_non_empty_discr] == num_new_offsets) + { + size_t first_offset = col.getVariantByLocalDiscriminator(last_non_empty_discr).size() - num_new_offsets; + for (size_t i = 0; i != num_new_offsets; ++i) + offsets.push_back(first_offset + i); + } + /// Otherwise iterate through discriminators and fill offsets accordingly. + else + { + size_t start = offsets.size(); + for (size_t i = start; i != discriminators_data.size(); ++i) + { + ColumnVariant::Discriminator discr = discriminators_data[i]; + if (discr == ColumnVariant::NULL_DISCRIMINATOR) + offsets.emplace_back(); + else + offsets.push_back(variant_offsets[discr]++); + } } addToSubstreamsCache(cache, settings.path, col.getOffsetsPtr()); @@ -340,6 +489,72 @@ void SerializationVariant::deserializeBinaryBulkWithMultipleStreams( settings.path.pop_back(); } +std::vector SerializationVariant::deserializeCompactDiscriminators( + DB::ColumnPtr & discriminators_column, + size_t limit, + ReadBuffer * stream, + bool continuous_reading, + DiscriminatorsDeserializationState & state) const +{ + auto & discriminators = assert_cast(*discriminators_column->assumeMutable()); + auto & discriminators_data = discriminators.getData(); + + /// Reset state if we are reading from the start of the granule and not from the previous position in the file. + if (!continuous_reading) + state.remaining_rows_in_granule = 0; + + /// Calculate limits for variants during discriminators deserialization. + std::vector variant_limits(variants.size(), 0); + while (limit) + { + /// If we read all rows from current granule, start reading the next one. + if (state.remaining_rows_in_granule == 0) + { + if (stream->eof()) + return variant_limits; + + readDiscriminatorsGranuleStart(state, stream); + } + + size_t limit_in_granule = std::min(limit, state.remaining_rows_in_granule); + if (state.granule_format == CompactDiscriminatorsGranuleFormat::COMPACT) + { + auto & data = discriminators.getData(); + data.resize_fill(data.size() + limit_in_granule, state.compact_discr); + if (state.compact_discr != ColumnVariant::NULL_DISCRIMINATOR) + variant_limits[state.compact_discr] += limit_in_granule; + } + else + { + SerializationNumber().deserializeBinaryBulk(discriminators, *stream, limit_in_granule, 0); + size_t start = discriminators_data.size() - limit_in_granule; + for (size_t i = start; i != discriminators_data.size(); ++i) + { + ColumnVariant::Discriminator discr = discriminators_data[i]; + if (discr != ColumnVariant::NULL_DISCRIMINATOR) + ++variant_limits[discr]; + } + } + + state.remaining_rows_in_granule -= limit_in_granule; + limit -= limit_in_granule; + } + + return variant_limits; +} + +void SerializationVariant::readDiscriminatorsGranuleStart(DB::SerializationVariant::DiscriminatorsDeserializationState & state, DB::ReadBuffer * stream) +{ + UInt64 granule_size; + readVarUInt(granule_size, *stream); + state.remaining_rows_in_granule = granule_size; + UInt8 granule_format; + readBinaryLittleEndian(granule_format, *stream); + state.granule_format = static_cast(granule_format); + if (granule_format == CompactDiscriminatorsGranuleFormat::COMPACT) + readBinaryLittleEndian(state.compact_discr, *stream); +} + void SerializationVariant::addVariantElementToPath(DB::ISerialization::SubstreamPath & path, size_t i) const { path.push_back(Substream::VariantElement); diff --git a/src/DataTypes/Serializations/SerializationVariant.h b/src/DataTypes/Serializations/SerializationVariant.h index 3f53dcf1339..e6b067dc68a 100644 --- a/src/DataTypes/Serializations/SerializationVariant.h +++ b/src/DataTypes/Serializations/SerializationVariant.h @@ -6,6 +6,13 @@ namespace DB { + +namespace ErrorCodes +{ + extern const int INCORRECT_DATA; +} + + /// Class for serializing/deserializing column with Variant type. /// It supports both text and binary bulk serializations/deserializations. /// @@ -18,6 +25,17 @@ namespace DB /// /// During binary bulk serialization it transforms local discriminators /// to global and serializes them into a separate stream VariantDiscriminators. +/// There are 2 modes of serialising discriminators: +/// Basic mode, when all discriminators are serialized as is row by row. +/// Compact mode, when we avoid writing the same discriminators in granules when there is +/// only one variant (or only NULLs) in the granule. +/// In compact mode we serialize granules in the following format: +/// +/// There are 2 different formats of granule - plain and compact. +/// Plain format is used when there are different discriminators in this granule, +/// in this format all discriminators are serialized as is row by row. +/// Compact format is used when all discriminators are the same in this granule, +/// in this case only this single discriminator is serialized. /// Each variant is serialized into a separate stream with path VariantElements/VariantElement /// (VariantElements stream is needed for correct sub-columns creation). We store and serialize /// variants in a sparse form (the size of a variant column equals to the number of its discriminator @@ -32,6 +50,25 @@ namespace DB class SerializationVariant : public ISerialization { public: + struct DiscriminatorsSerializationMode + { + enum Value + { + BASIC = 0, /// Store the whole discriminators column. + COMPACT = 1, /// Don't write discriminators in granule if all of them are the same. + }; + + static void checkMode(UInt64 mode) + { + if (mode > Value::COMPACT) + throw Exception(ErrorCodes::INCORRECT_DATA, "Invalid version for SerializationVariant discriminators column."); + } + + explicit DiscriminatorsSerializationMode(UInt64 mode) : value(static_cast(mode)) { checkMode(mode); } + + Value value; + }; + using VariantSerializations = std::vector; explicit SerializationVariant( @@ -114,8 +151,36 @@ public: static std::vector getVariantsDeserializeTextOrder(const DataTypes & variant_types); private: + friend SerializationVariantElement; + void addVariantElementToPath(SubstreamPath & path, size_t i) const; + enum CompactDiscriminatorsGranuleFormat + { + PLAIN = 0, /// Granule has different discriminators and they are serialized as is row by row. + COMPACT = 1, /// Granule has single discriminator for all rows and it is serialized as single value. + }; + + /// State of currently deserialized granule. + struct DiscriminatorsDeserializationState + { + CompactDiscriminatorsGranuleFormat granule_format = CompactDiscriminatorsGranuleFormat::PLAIN; + size_t remaining_rows_in_granule = 0; + ColumnVariant::Discriminator compact_discr = 0; + }; + + struct SerializeBinaryBulkStateVariant; + struct DeserializeBinaryBulkStateVariant; + + std::vector deserializeCompactDiscriminators( + ColumnPtr & discriminators_column, + size_t limit, + ReadBuffer * stream, + bool continuous_reading, + DiscriminatorsDeserializationState & state) const; + + static void readDiscriminatorsGranuleStart(DiscriminatorsDeserializationState & state, ReadBuffer * stream); + bool tryDeserializeTextEscapedImpl(IColumn & column, const String & field, const FormatSettings & settings) const; bool tryDeserializeTextQuotedImpl(IColumn & column, const String & field, const FormatSettings & settings) const; bool tryDeserializeWholeTextImpl(IColumn & column, const String & field, const FormatSettings & settings) const; diff --git a/src/DataTypes/Serializations/SerializationVariantElement.cpp b/src/DataTypes/Serializations/SerializationVariantElement.cpp index 7d4487fe6da..27166dd0704 100644 --- a/src/DataTypes/Serializations/SerializationVariantElement.cpp +++ b/src/DataTypes/Serializations/SerializationVariantElement.cpp @@ -1,7 +1,9 @@ #include #include +#include #include #include +#include namespace DB { @@ -39,8 +41,12 @@ void SerializationVariantElement::serializeBinaryBulkStateSuffix(SerializeBinary ErrorCodes::NOT_IMPLEMENTED, "Method serializeBinaryBulkStateSuffix is not implemented for SerializationVariantElement"); } -struct DeserializeBinaryBulkStateVariantElement : public ISerialization::DeserializeBinaryBulkState +struct SerializationVariantElement::DeserializeBinaryBulkStateVariantElement : public ISerialization::DeserializeBinaryBulkState { + DeserializeBinaryBulkStateVariantElement(UInt64 discriminators_version_) : discriminators_version(discriminators_version_) + { + } + /// During deserialization discriminators and variant streams can be shared. /// For example we can read several variant elements together: "select v.UInt32, v.String from table", /// or we can read the whole variant and some of variant elements: "select v, v.UInt32 from table". @@ -51,13 +57,24 @@ struct DeserializeBinaryBulkStateVariantElement : public ISerialization::Deseria /// substream cache correctly. ColumnPtr discriminators; ColumnPtr variant; - + SerializationVariant::DiscriminatorsSerializationMode discriminators_version; + SerializationVariant::DiscriminatorsDeserializationState discriminators_state; ISerialization::DeserializeBinaryBulkStatePtr variant_element_state; }; void SerializationVariantElement::deserializeBinaryBulkStatePrefix(DeserializeBinaryBulkSettings & settings, DeserializeBinaryBulkStatePtr & state) const { - auto variant_element_state = std::make_shared(); + settings.path.push_back(Substream::VariantDiscriminators); + auto * discriminators_stream = settings.getter(settings.path); + settings.path.pop_back(); + + if (!discriminators_stream) + return; + + UInt64 version; + readBinaryLittleEndian(version, *discriminators_stream); + + auto variant_element_state = std::make_shared(version); addVariantToPath(settings.path); nested_serialization->deserializeBinaryBulkStatePrefix(settings, variant_element_state->variant_element_state); @@ -78,35 +95,53 @@ void SerializationVariantElement::deserializeBinaryBulkWithMultipleStreams( DeserializeBinaryBulkStatePtr & state, SubstreamsCache * cache) const { - auto * variant_element_state = checkAndGetState(state); - /// First, deserialize discriminators from Variant column. settings.path.push_back(Substream::VariantDiscriminators); + + DeserializeBinaryBulkStateVariantElement * variant_element_state = nullptr; + std::optional variant_limit; if (auto cached_discriminators = getFromSubstreamsCache(cache, settings.path)) { + variant_element_state = checkAndGetState(state); variant_element_state->discriminators = cached_discriminators; } - else + else if (auto * discriminators_stream = settings.getter(settings.path)) { - auto * discriminators_stream = settings.getter(settings.path); - if (!discriminators_stream) - return; + variant_element_state = checkAndGetState(state); /// If we started to read a new column, reinitialize discriminators column in deserialization state. if (!variant_element_state->discriminators || result_column->empty()) variant_element_state->discriminators = ColumnVariant::ColumnDiscriminators::create(); - SerializationNumber().deserializeBinaryBulk(*variant_element_state->discriminators->assumeMutable(), *discriminators_stream, limit, 0); + /// Deserialize discriminators according to serialization mode. + if (variant_element_state->discriminators_version.value == SerializationVariant::DiscriminatorsSerializationMode::BASIC) + SerializationNumber().deserializeBinaryBulk(*variant_element_state->discriminators->assumeMutable(), *discriminators_stream, limit, 0); + else + variant_limit = deserializeCompactDiscriminators(variant_element_state->discriminators, limit, discriminators_stream, settings.continuous_reading, *variant_element_state); + addToSubstreamsCache(cache, settings.path, variant_element_state->discriminators); } + else + { + settings.path.pop_back(); + return; + } + settings.path.pop_back(); - /// Iterate through new discriminators to calculate the limit for our variant. + /// We could read less than limit discriminators, but we will need actual number of read rows later. + size_t num_new_discriminators = variant_element_state->discriminators->size() - result_column->size(); + + /// Iterate through new discriminators to calculate the limit for our variant + /// if we didn't do it during discriminators deserialization. const auto & discriminators_data = assert_cast(*variant_element_state->discriminators).getData(); - size_t discriminators_offset = variant_element_state->discriminators->size() - limit; - size_t variant_limit = 0; - for (size_t i = discriminators_offset; i != discriminators_data.size(); ++i) - variant_limit += (discriminators_data[i] == variant_discriminator); + size_t discriminators_offset = variant_element_state->discriminators->size() - num_new_discriminators; + if (!variant_limit) + { + variant_limit = 0; + for (size_t i = discriminators_offset; i != discriminators_data.size(); ++i) + *variant_limit += (discriminators_data[i] == variant_discriminator); + } /// Now we know the limit for our variant and can deserialize it. @@ -117,19 +152,19 @@ void SerializationVariantElement::deserializeBinaryBulkWithMultipleStreams( auto & nullable_column = assert_cast(*mutable_column); NullMap & null_map = nullable_column.getNullMapData(); /// If we have only our discriminator in range, fill null map with 0. - if (variant_limit == limit) + if (variant_limit == num_new_discriminators) { - null_map.resize_fill(null_map.size() + limit, 0); + null_map.resize_fill(null_map.size() + num_new_discriminators, 0); } /// If no our discriminator in current range, fill null map with 1. else if (variant_limit == 0) { - null_map.resize_fill(null_map.size() + limit, 1); + null_map.resize_fill(null_map.size() + num_new_discriminators, 1); } /// Otherwise we should iterate through discriminators to fill null map. else { - null_map.reserve(null_map.size() + limit); + null_map.reserve(null_map.size() + num_new_discriminators); for (size_t i = discriminators_offset; i != discriminators_data.size(); ++i) null_map.push_back(discriminators_data[i] != variant_discriminator); } @@ -151,12 +186,12 @@ void SerializationVariantElement::deserializeBinaryBulkWithMultipleStreams( /// If nothing to deserialize, just insert defaults. if (variant_limit == 0) { - mutable_column->insertManyDefaults(limit); + mutable_column->insertManyDefaults(num_new_discriminators); return; } addVariantToPath(settings.path); - nested_serialization->deserializeBinaryBulkWithMultipleStreams(variant_element_state->variant, variant_limit, settings, variant_element_state->variant_element_state, cache); + nested_serialization->deserializeBinaryBulkWithMultipleStreams(variant_element_state->variant, *variant_limit, settings, variant_element_state->variant_element_state, cache); removeVariantFromPath(settings.path); /// If nothing was deserialized when variant_limit > 0 @@ -165,16 +200,16 @@ void SerializationVariantElement::deserializeBinaryBulkWithMultipleStreams( /// In this case we should just insert default values. if (variant_element_state->variant->empty()) { - mutable_column->insertManyDefaults(limit); + mutable_column->insertManyDefaults(num_new_discriminators); return; } - size_t variant_offset = variant_element_state->variant->size() - variant_limit; + size_t variant_offset = variant_element_state->variant->size() - *variant_limit; /// If we have only our discriminator in range, insert the whole range to result column. - if (variant_limit == limit) + if (variant_limit == num_new_discriminators) { - mutable_column->insertRangeFrom(*variant_element_state->variant, variant_offset, variant_limit); + mutable_column->insertRangeFrom(*variant_element_state->variant, variant_offset, *variant_limit); } /// Otherwise iterate through discriminators and insert value from variant or default value depending on the discriminator. else @@ -189,6 +224,56 @@ void SerializationVariantElement::deserializeBinaryBulkWithMultipleStreams( } } +size_t SerializationVariantElement::deserializeCompactDiscriminators( + DB::ColumnPtr & discriminators_column, + size_t limit, + DB::ReadBuffer * stream, + bool continuous_reading, + DB::SerializationVariantElement::DeserializeBinaryBulkStateVariantElement & state) const +{ + auto & discriminators = assert_cast(*discriminators_column->assumeMutable()); + auto & discriminators_data = discriminators.getData(); + + /// Reset state if we are reading from the start of the granule and not from the previous position in the file. + if (!continuous_reading) + state.discriminators_state.remaining_rows_in_granule = 0; + + /// Calculate our variant limit during discriminators deserialization. + size_t variant_limit = 0; + while (limit) + { + /// If we read all rows from current granule, start reading the next one. + if (state.discriminators_state.remaining_rows_in_granule == 0) + { + if (stream->eof()) + return variant_limit; + + SerializationVariant::readDiscriminatorsGranuleStart(state.discriminators_state, stream); + } + + size_t limit_in_granule = std::min(limit, state.discriminators_state.remaining_rows_in_granule); + if (state.discriminators_state.granule_format == SerializationVariant::CompactDiscriminatorsGranuleFormat::COMPACT) + { + auto & data = discriminators.getData(); + data.resize_fill(data.size() + limit_in_granule, state.discriminators_state.compact_discr); + if (state.discriminators_state.compact_discr == variant_discriminator) + variant_limit += limit_in_granule; + } + else + { + SerializationNumber().deserializeBinaryBulk(discriminators, *stream, limit_in_granule, 0); + size_t start = discriminators_data.size() - limit_in_granule; + for (size_t i = start; i != discriminators_data.size(); ++i) + variant_limit += (discriminators_data[i] == variant_discriminator); + } + + state.discriminators_state.remaining_rows_in_granule -= limit_in_granule; + limit -= limit_in_granule; + } + + return variant_limit; +} + void SerializationVariantElement::addVariantToPath(DB::ISerialization::SubstreamPath & path) const { path.push_back(Substream::VariantElements); diff --git a/src/DataTypes/Serializations/SerializationVariantElement.h b/src/DataTypes/Serializations/SerializationVariantElement.h index aafecf43d39..e86d7ecefbe 100644 --- a/src/DataTypes/Serializations/SerializationVariantElement.h +++ b/src/DataTypes/Serializations/SerializationVariantElement.h @@ -62,6 +62,15 @@ public: private: friend SerializationVariant; + struct DeserializeBinaryBulkStateVariantElement; + + size_t deserializeCompactDiscriminators( + ColumnPtr & discriminators_column, + size_t limit, + ReadBuffer * stream, + bool continuous_reading, + DeserializeBinaryBulkStateVariantElement & state) const; + void addVariantToPath(SubstreamPath & path) const; void removeVariantFromPath(SubstreamPath & path) const; diff --git a/src/Storages/MergeTree/MergeTreeDataPartWriterCompact.cpp b/src/Storages/MergeTree/MergeTreeDataPartWriterCompact.cpp index 1605e5cdb9a..b25b4042b5f 100644 --- a/src/Storages/MergeTree/MergeTreeDataPartWriterCompact.cpp +++ b/src/Storages/MergeTree/MergeTreeDataPartWriterCompact.cpp @@ -130,7 +130,8 @@ void writeColumnSingleGranule( const SerializationPtr & serialization, ISerialization::OutputStreamGetter stream_getter, size_t from_row, - size_t number_of_rows) + size_t number_of_rows, + const MergeTreeSettingsPtr & settings) { ISerialization::SerializeBinaryBulkStatePtr state; ISerialization::SerializeBinaryBulkSettings serialize_settings; @@ -138,6 +139,7 @@ void writeColumnSingleGranule( serialize_settings.getter = stream_getter; serialize_settings.position_independent_encoding = true; serialize_settings.low_cardinality_max_dictionary_size = 0; + serialize_settings.use_compact_variant_discriminators_serialization = settings->use_compact_variant_discriminators_serialization; serialization->serializeBinaryBulkStatePrefix(*column.column, serialize_settings, state); serialization->serializeBinaryBulkWithMultipleStreams(*column.column, from_row, number_of_rows, serialize_settings, state); @@ -231,7 +233,7 @@ void MergeTreeDataPartWriterCompact::writeDataBlock(const Block & block, const G writeColumnSingleGranule( block.getByName(name_and_type->name), data_part->getSerialization(name_and_type->name), - stream_getter, granule.start_row, granule.rows_to_write); + stream_getter, granule.start_row, granule.rows_to_write, storage.getSettings()); /// Each type always have at least one substream prev_stream->hashing_buf.next(); diff --git a/src/Storages/MergeTree/MergeTreeDataPartWriterWide.cpp b/src/Storages/MergeTree/MergeTreeDataPartWriterWide.cpp index 6a3b08d4d65..89abb1f242f 100644 --- a/src/Storages/MergeTree/MergeTreeDataPartWriterWide.cpp +++ b/src/Storages/MergeTree/MergeTreeDataPartWriterWide.cpp @@ -399,19 +399,21 @@ void MergeTreeDataPartWriterWide::writeColumn( const auto & [name, type] = name_and_type; auto [it, inserted] = serialization_states.emplace(name, nullptr); auto serialization = data_part->getSerialization(name_and_type.name); + const auto & global_settings = storage.getContext()->getSettingsRef(); if (inserted) { ISerialization::SerializeBinaryBulkSettings serialize_settings; + serialize_settings.use_compact_variant_discriminators_serialization = storage.getSettings()->use_compact_variant_discriminators_serialization; serialize_settings.getter = createStreamGetter(name_and_type, offset_columns); serialization->serializeBinaryBulkStatePrefix(column, serialize_settings, it->second); } - const auto & global_settings = storage.getContext()->getSettingsRef(); ISerialization::SerializeBinaryBulkSettings serialize_settings; serialize_settings.getter = createStreamGetter(name_and_type, offset_columns); serialize_settings.low_cardinality_max_dictionary_size = global_settings.low_cardinality_max_dictionary_size; serialize_settings.low_cardinality_use_single_dictionary_for_part = global_settings.low_cardinality_use_single_dictionary_for_part != 0; + serialize_settings.use_compact_variant_discriminators_serialization = storage.getSettings()->use_compact_variant_discriminators_serialization; for (const auto & granule : granules) { @@ -600,6 +602,7 @@ void MergeTreeDataPartWriterWide::fillDataChecksums(IMergeTreeDataPart::Checksum ISerialization::SerializeBinaryBulkSettings serialize_settings; serialize_settings.low_cardinality_max_dictionary_size = global_settings.low_cardinality_max_dictionary_size; serialize_settings.low_cardinality_use_single_dictionary_for_part = global_settings.low_cardinality_use_single_dictionary_for_part != 0; + serialize_settings.use_compact_variant_discriminators_serialization = storage.getSettings()->use_compact_variant_discriminators_serialization; WrittenOffsetColumns offset_columns; if (rows_written_in_last_mark > 0) { diff --git a/src/Storages/MergeTree/MergeTreeReaderWide.cpp b/src/Storages/MergeTree/MergeTreeReaderWide.cpp index 394a22835f1..1fd886a6216 100644 --- a/src/Storages/MergeTree/MergeTreeReaderWide.cpp +++ b/src/Storages/MergeTree/MergeTreeReaderWide.cpp @@ -128,6 +128,9 @@ size_t MergeTreeReaderWide::readRows( size_t num_columns = res_columns.size(); checkNumberOfColumns(num_columns); + if (deserialize_binary_bulk_state_map.empty()) + readPrefixes(num_columns, current_task_last_mark); + if (num_columns == 0) return max_rows_to_read; @@ -290,19 +293,28 @@ static ReadBuffer * getStream( return stream.getDataBuffer(); } +void MergeTreeReaderWide::readPrefixes(size_t num_columns, size_t current_task_last_mark) +{ + for (size_t pos = 0; pos < num_columns; ++pos) + { + const auto & column_to_read = columns_to_read[pos]; + deserializePrefix(serializations[pos], column_to_read, current_task_last_mark); + } +} + void MergeTreeReaderWide::deserializePrefix( const SerializationPtr & serialization, const NameAndTypePair & name_and_type, - size_t current_task_last_mark, - ISerialization::SubstreamsCache & cache) + size_t current_task_last_mark) { const auto & name = name_and_type.name; if (!deserialize_binary_bulk_state_map.contains(name)) { ISerialization::DeserializeBinaryBulkSettings deserialize_settings; + ISerialization::SubstreamsCache empty_cache; deserialize_settings.getter = [&](const ISerialization::SubstreamPath & substream_path) { - return getStream(/* seek_to_start = */true, substream_path, data_part_info_for_read->getChecksums(), streams, name_and_type, 0, /* seek_to_mark = */false, current_task_last_mark, cache); + return getStream(/* seek_to_start = */true, substream_path, data_part_info_for_read->getChecksums(), streams, name_and_type, 0, /* seek_to_mark = */false, current_task_last_mark, empty_cache); }; serialization->deserializeBinaryBulkStatePrefix(deserialize_settings, deserialize_binary_bulk_state_map[name]); } @@ -317,8 +329,6 @@ void MergeTreeReaderWide::prefetchForColumn( size_t current_task_last_mark, ISerialization::SubstreamsCache & cache) { - deserializePrefix(serialization, name_and_type, current_task_last_mark, cache); - serialization->enumerateStreams([&](const ISerialization::SubstreamPath & substream_path) { auto stream_name = IMergeTreeDataPart::getStreamNameForColumn(name_and_type, substream_path, data_part_info_for_read->getChecksums()); @@ -344,9 +354,6 @@ void MergeTreeReaderWide::readData( double & avg_value_size_hint = avg_value_size_hints[name_and_type.name]; ISerialization::DeserializeBinaryBulkSettings deserialize_settings; deserialize_settings.avg_value_size_hint = avg_value_size_hint; - - deserializePrefix(serialization, name_and_type, current_task_last_mark, cache); - deserialize_settings.getter = [&](const ISerialization::SubstreamPath & substream_path) { bool seek_to_mark = !was_prefetched && !continue_reading; diff --git a/src/Storages/MergeTree/MergeTreeReaderWide.h b/src/Storages/MergeTree/MergeTreeReaderWide.h index a9a5526dd65..617e26864a9 100644 --- a/src/Storages/MergeTree/MergeTreeReaderWide.h +++ b/src/Storages/MergeTree/MergeTreeReaderWide.h @@ -64,11 +64,12 @@ private: size_t current_task_last_mark, ISerialization::SubstreamsCache & cache); + void readPrefixes(size_t num_columns, size_t current_task_last_mark); + void deserializePrefix( const SerializationPtr & serialization, const NameAndTypePair & name_and_type, - size_t current_task_last_mark, - ISerialization::SubstreamsCache & cache); + size_t current_task_last_mark); std::unordered_map caches; std::unordered_set prefetched_streams; diff --git a/src/Storages/MergeTree/MergeTreeSettings.h b/src/Storages/MergeTree/MergeTreeSettings.h index a00508fd1c1..da8fa4875dd 100644 --- a/src/Storages/MergeTree/MergeTreeSettings.h +++ b/src/Storages/MergeTree/MergeTreeSettings.h @@ -43,6 +43,7 @@ struct Settings; M(UInt64, compact_parts_max_granules_to_buffer, 128, "Only available in ClickHouse Cloud", 0) \ M(UInt64, compact_parts_merge_max_bytes_to_prefetch_part, 16 * 1024 * 1024, "Only available in ClickHouse Cloud", 0) \ M(Bool, load_existing_rows_count_for_old_parts, false, "Whether to load existing_rows_count for existing parts. If false, existing_rows_count will be equal to rows_count for existing parts.", 0) \ + M(Bool, use_compact_variant_discriminators_serialization, true, "Use compact version of Variant discriminators serialization.", 0) \ \ /** Merge settings. */ \ M(UInt64, merge_max_block_size, 8192, "How many rows in blocks should be formed for merge operations. By default has the same value as `index_granularity`.", 0) \ From f70851f5f492b9a372d0715e90fa7753383038f3 Mon Sep 17 00:00:00 2001 From: avogar Date: Thu, 18 Apr 2024 22:39:26 +0000 Subject: [PATCH 004/417] Add new MergeTree setting in docs --- docs/en/operations/settings/merge-tree-settings.md | 5 +++++ 1 file changed, 5 insertions(+) diff --git a/docs/en/operations/settings/merge-tree-settings.md b/docs/en/operations/settings/merge-tree-settings.md index 76250b80476..ff21d792aee 100644 --- a/docs/en/operations/settings/merge-tree-settings.md +++ b/docs/en/operations/settings/merge-tree-settings.md @@ -885,3 +885,8 @@ Default value: false **See Also** - [exclude_deleted_rows_for_part_size_in_merge](#exclude_deleted_rows_for_part_size_in_merge) setting + +## use_compact_variant_discriminators_serialization {#use_compact_variant_discriminators_serialization} + +Enables compact mode for binary serialization of discriminators in Variant data type. +This mode allows to use significantly less memory for storing discriminators in parts when there is mostly one variant or a lot of NULL values. From e524f3817254488c2e43b3aa9ea350e236910493 Mon Sep 17 00:00:00 2001 From: avogar Date: Thu, 18 Apr 2024 22:40:30 +0000 Subject: [PATCH 005/417] Add default value of new setting in docs --- docs/en/operations/settings/merge-tree-settings.md | 2 ++ 1 file changed, 2 insertions(+) diff --git a/docs/en/operations/settings/merge-tree-settings.md b/docs/en/operations/settings/merge-tree-settings.md index ff21d792aee..0f54adb5c7b 100644 --- a/docs/en/operations/settings/merge-tree-settings.md +++ b/docs/en/operations/settings/merge-tree-settings.md @@ -890,3 +890,5 @@ Default value: false Enables compact mode for binary serialization of discriminators in Variant data type. This mode allows to use significantly less memory for storing discriminators in parts when there is mostly one variant or a lot of NULL values. + +Default value: true From 7b6a8ed28e4f985fc1866d316a59bc205fd919ca Mon Sep 17 00:00:00 2001 From: avogar Date: Fri, 19 Apr 2024 16:17:25 +0000 Subject: [PATCH 006/417] Introduce deserialization states cache --- .../Serializations/ISerialization.cpp | 17 +++++ src/DataTypes/Serializations/ISerialization.h | 8 ++- .../Serializations/SerializationArray.cpp | 5 +- .../Serializations/SerializationArray.h | 3 +- .../Serializations/SerializationInterval.cpp | 4 +- .../Serializations/SerializationInterval.h | 2 +- .../SerializationLowCardinality.cpp | 3 +- .../SerializationLowCardinality.h | 3 +- .../Serializations/SerializationMap.cpp | 5 +- .../Serializations/SerializationMap.h | 3 +- .../Serializations/SerializationNamed.cpp | 5 +- .../Serializations/SerializationNamed.h | 3 +- .../Serializations/SerializationNullable.cpp | 5 +- .../Serializations/SerializationNullable.h | 3 +- .../Serializations/SerializationObject.cpp | 5 +- .../Serializations/SerializationObject.h | 3 +- .../Serializations/SerializationSparse.cpp | 5 +- .../Serializations/SerializationSparse.h | 3 +- .../Serializations/SerializationTuple.cpp | 5 +- .../Serializations/SerializationTuple.h | 3 +- .../Serializations/SerializationVariant.cpp | 72 +++++++++++-------- .../Serializations/SerializationVariant.h | 20 +++--- .../SerializationVariantElement.cpp | 64 ++++++++++------- .../SerializationVariantElement.h | 5 +- .../Serializations/SerializationWrapper.cpp | 5 +- .../Serializations/SerializationWrapper.h | 3 +- .../tests/gtest_object_serialization.cpp | 2 +- src/Formats/NativeReader.cpp | 2 +- src/Storages/MergeTree/MergeTreeIndexSet.cpp | 2 +- .../MergeTree/MergeTreeReaderCompact.cpp | 4 +- .../MergeTree/MergeTreeReaderWide.cpp | 38 +++++----- src/Storages/MergeTree/MergeTreeReaderWide.h | 12 ++-- src/Storages/StorageLog.cpp | 2 +- 33 files changed, 195 insertions(+), 129 deletions(-) diff --git a/src/DataTypes/Serializations/ISerialization.cpp b/src/DataTypes/Serializations/ISerialization.cpp index a3a28f8091c..5c8a0d49038 100644 --- a/src/DataTypes/Serializations/ISerialization.cpp +++ b/src/DataTypes/Serializations/ISerialization.cpp @@ -271,6 +271,23 @@ ColumnPtr ISerialization::getFromSubstreamsCache(SubstreamsCache * cache, const return it == cache->end() ? nullptr : it->second; } +void ISerialization::addToSubstreamsDeserializeStatesCache(SubstreamsDeserializeStatesCache * cache, const SubstreamPath & path, DeserializeBinaryBulkStatePtr state) +{ + if (!cache || path.empty()) + return; + + cache->emplace(getSubcolumnNameForStream(path), state); +} + +ISerialization::DeserializeBinaryBulkStatePtr ISerialization::getFromSubstreamsDeserializeStatesCache(SubstreamsDeserializeStatesCache * cache, const SubstreamPath & path) +{ + if (!cache || path.empty()) + return nullptr; + + auto it = cache->find(getSubcolumnNameForStream(path)); + return it == cache->end() ? nullptr : it->second; +} + bool ISerialization::isSpecialCompressionAllowed(const SubstreamPath & path) { for (const auto & elem : path) diff --git a/src/DataTypes/Serializations/ISerialization.h b/src/DataTypes/Serializations/ISerialization.h index 934a92ecfba..408b5fe2133 100644 --- a/src/DataTypes/Serializations/ISerialization.h +++ b/src/DataTypes/Serializations/ISerialization.h @@ -231,6 +231,9 @@ public: using SerializeBinaryBulkStatePtr = std::shared_ptr; using DeserializeBinaryBulkStatePtr = std::shared_ptr; + using SubstreamsDeserializeStatesCache = std::unordered_map; + + struct SerializeBinaryBulkSettings { OutputStreamGetter getter; @@ -275,7 +278,8 @@ public: /// Call before before deserializeBinaryBulkWithMultipleStreams chain to get DeserializeBinaryBulkStatePtr. virtual void deserializeBinaryBulkStatePrefix( DeserializeBinaryBulkSettings & /*settings*/, - DeserializeBinaryBulkStatePtr & /*state*/) const {} + DeserializeBinaryBulkStatePtr & /*state*/, + SubstreamsDeserializeStatesCache * /*cache*/) const {} /** 'offset' and 'limit' are used to specify range. * limit = 0 - means no limit. @@ -394,6 +398,8 @@ public: static void addToSubstreamsCache(SubstreamsCache * cache, const SubstreamPath & path, ColumnPtr column); static ColumnPtr getFromSubstreamsCache(SubstreamsCache * cache, const SubstreamPath & path); + static void addToSubstreamsDeserializeStatesCache(SubstreamsDeserializeStatesCache * cache, const SubstreamPath & path, DeserializeBinaryBulkStatePtr state); + static DeserializeBinaryBulkStatePtr getFromSubstreamsDeserializeStatesCache(SubstreamsDeserializeStatesCache * cache, const SubstreamPath & path); static bool isSpecialCompressionAllowed(const SubstreamPath & path); diff --git a/src/DataTypes/Serializations/SerializationArray.cpp b/src/DataTypes/Serializations/SerializationArray.cpp index e8aab615849..d6546b338b5 100644 --- a/src/DataTypes/Serializations/SerializationArray.cpp +++ b/src/DataTypes/Serializations/SerializationArray.cpp @@ -284,10 +284,11 @@ void SerializationArray::serializeBinaryBulkStateSuffix( void SerializationArray::deserializeBinaryBulkStatePrefix( DeserializeBinaryBulkSettings & settings, - DeserializeBinaryBulkStatePtr & state) const + DeserializeBinaryBulkStatePtr & state, + SubstreamsDeserializeStatesCache * cache) const { settings.path.push_back(Substream::ArrayElements); - nested->deserializeBinaryBulkStatePrefix(settings, state); + nested->deserializeBinaryBulkStatePrefix(settings, state, cache); settings.path.pop_back(); } diff --git a/src/DataTypes/Serializations/SerializationArray.h b/src/DataTypes/Serializations/SerializationArray.h index 82f5e8bce45..429e8fa64cc 100644 --- a/src/DataTypes/Serializations/SerializationArray.h +++ b/src/DataTypes/Serializations/SerializationArray.h @@ -55,7 +55,8 @@ public: void deserializeBinaryBulkStatePrefix( DeserializeBinaryBulkSettings & settings, - DeserializeBinaryBulkStatePtr & state) const override; + DeserializeBinaryBulkStatePtr & state, + SubstreamsDeserializeStatesCache * cache) const override; void serializeBinaryBulkWithMultipleStreams( const IColumn & column, diff --git a/src/DataTypes/Serializations/SerializationInterval.cpp b/src/DataTypes/Serializations/SerializationInterval.cpp index 59086d8aef3..2157566895d 100644 --- a/src/DataTypes/Serializations/SerializationInterval.cpp +++ b/src/DataTypes/Serializations/SerializationInterval.cpp @@ -68,9 +68,9 @@ void SerializationInterval::deserializeBinaryBulk(IColumn & column, ReadBuffer & } void SerializationInterval::deserializeBinaryBulkStatePrefix( - DeserializeBinaryBulkSettings & settings, DeserializeBinaryBulkStatePtr & state) const + DeserializeBinaryBulkSettings & settings, DeserializeBinaryBulkStatePtr & state, SubstreamsDeserializeStatesCache * cache) const { - dispatch(&ISerialization::deserializeBinaryBulkStatePrefix, FormatSettings::IntervalOutputFormat::Numeric, settings, state); + dispatch(&ISerialization::deserializeBinaryBulkStatePrefix, FormatSettings::IntervalOutputFormat::Numeric, settings, state, cache); } diff --git a/src/DataTypes/Serializations/SerializationInterval.h b/src/DataTypes/Serializations/SerializationInterval.h index a4e6c204e4f..05dfdb00afc 100644 --- a/src/DataTypes/Serializations/SerializationInterval.h +++ b/src/DataTypes/Serializations/SerializationInterval.h @@ -34,7 +34,7 @@ public: void deserializeBinary(Field & field, ReadBuffer & istr, const FormatSettings & settings) const override; void deserializeBinary(IColumn & column, ReadBuffer & istr, const FormatSettings & settings) const override; void deserializeBinaryBulk(IColumn & column, ReadBuffer & istr, size_t limit, double avg_value_size_hint) const override; - void deserializeBinaryBulkStatePrefix(DeserializeBinaryBulkSettings & settings, DeserializeBinaryBulkStatePtr & state) const override; + void deserializeBinaryBulkStatePrefix(DeserializeBinaryBulkSettings & settings, DeserializeBinaryBulkStatePtr & state, SubstreamsDeserializeStatesCache * cache) const override; void deserializeBinaryBulkWithMultipleStreams( ColumnPtr & column, size_t limit, diff --git a/src/DataTypes/Serializations/SerializationLowCardinality.cpp b/src/DataTypes/Serializations/SerializationLowCardinality.cpp index 9efe05042ed..802da263d89 100644 --- a/src/DataTypes/Serializations/SerializationLowCardinality.cpp +++ b/src/DataTypes/Serializations/SerializationLowCardinality.cpp @@ -267,7 +267,8 @@ void SerializationLowCardinality::serializeBinaryBulkStateSuffix( void SerializationLowCardinality::deserializeBinaryBulkStatePrefix( DeserializeBinaryBulkSettings & settings, - DeserializeBinaryBulkStatePtr & state) const + DeserializeBinaryBulkStatePtr & state, + SubstreamsDeserializeStatesCache * /*cache*/) const { settings.path.push_back(Substream::DictionaryKeys); auto * stream = settings.getter(settings.path); diff --git a/src/DataTypes/Serializations/SerializationLowCardinality.h b/src/DataTypes/Serializations/SerializationLowCardinality.h index d2c3a95c702..aa64e956a64 100644 --- a/src/DataTypes/Serializations/SerializationLowCardinality.h +++ b/src/DataTypes/Serializations/SerializationLowCardinality.h @@ -33,7 +33,8 @@ public: void deserializeBinaryBulkStatePrefix( DeserializeBinaryBulkSettings & settings, - DeserializeBinaryBulkStatePtr & state) const override; + DeserializeBinaryBulkStatePtr & state, + SubstreamsDeserializeStatesCache * cache) const override; void serializeBinaryBulkWithMultipleStreams( const IColumn & column, diff --git a/src/DataTypes/Serializations/SerializationMap.cpp b/src/DataTypes/Serializations/SerializationMap.cpp index 7b6f87baf2e..dac4fbe88e0 100644 --- a/src/DataTypes/Serializations/SerializationMap.cpp +++ b/src/DataTypes/Serializations/SerializationMap.cpp @@ -420,9 +420,10 @@ void SerializationMap::serializeBinaryBulkStateSuffix( void SerializationMap::deserializeBinaryBulkStatePrefix( DeserializeBinaryBulkSettings & settings, - DeserializeBinaryBulkStatePtr & state) const + DeserializeBinaryBulkStatePtr & state, + SubstreamsDeserializeStatesCache * cache) const { - nested->deserializeBinaryBulkStatePrefix(settings, state); + nested->deserializeBinaryBulkStatePrefix(settings, state, cache); } diff --git a/src/DataTypes/Serializations/SerializationMap.h b/src/DataTypes/Serializations/SerializationMap.h index 3e27ef1b04a..cfcde445c1f 100644 --- a/src/DataTypes/Serializations/SerializationMap.h +++ b/src/DataTypes/Serializations/SerializationMap.h @@ -51,7 +51,8 @@ public: void deserializeBinaryBulkStatePrefix( DeserializeBinaryBulkSettings & settings, - DeserializeBinaryBulkStatePtr & state) const override; + DeserializeBinaryBulkStatePtr & state, + SubstreamsDeserializeStatesCache * cache) const override; void serializeBinaryBulkWithMultipleStreams( const IColumn & column, diff --git a/src/DataTypes/Serializations/SerializationNamed.cpp b/src/DataTypes/Serializations/SerializationNamed.cpp index 2792827e690..07f5f9ea7ed 100644 --- a/src/DataTypes/Serializations/SerializationNamed.cpp +++ b/src/DataTypes/Serializations/SerializationNamed.cpp @@ -54,10 +54,11 @@ void SerializationNamed::serializeBinaryBulkStateSuffix( void SerializationNamed::deserializeBinaryBulkStatePrefix( DeserializeBinaryBulkSettings & settings, - DeserializeBinaryBulkStatePtr & state) const + DeserializeBinaryBulkStatePtr & state, + SubstreamsDeserializeStatesCache * cache) const { addToPath(settings.path); - nested_serialization->deserializeBinaryBulkStatePrefix(settings, state); + nested_serialization->deserializeBinaryBulkStatePrefix(settings, state, cache); settings.path.pop_back(); } diff --git a/src/DataTypes/Serializations/SerializationNamed.h b/src/DataTypes/Serializations/SerializationNamed.h index 0633ba2ea6f..bb2161e40e6 100644 --- a/src/DataTypes/Serializations/SerializationNamed.h +++ b/src/DataTypes/Serializations/SerializationNamed.h @@ -36,7 +36,8 @@ public: void deserializeBinaryBulkStatePrefix( DeserializeBinaryBulkSettings & settings, - DeserializeBinaryBulkStatePtr & state) const override; + DeserializeBinaryBulkStatePtr & state, + SubstreamsDeserializeStatesCache * cache) const override; void serializeBinaryBulkWithMultipleStreams( const IColumn & column, diff --git a/src/DataTypes/Serializations/SerializationNullable.cpp b/src/DataTypes/Serializations/SerializationNullable.cpp index 4d31451f92d..477349f955d 100644 --- a/src/DataTypes/Serializations/SerializationNullable.cpp +++ b/src/DataTypes/Serializations/SerializationNullable.cpp @@ -95,10 +95,11 @@ void SerializationNullable::serializeBinaryBulkStateSuffix( void SerializationNullable::deserializeBinaryBulkStatePrefix( DeserializeBinaryBulkSettings & settings, - DeserializeBinaryBulkStatePtr & state) const + DeserializeBinaryBulkStatePtr & state, + SubstreamsDeserializeStatesCache * cache) const { settings.path.push_back(Substream::NullableElements); - nested->deserializeBinaryBulkStatePrefix(settings, state); + nested->deserializeBinaryBulkStatePrefix(settings, state, cache); settings.path.pop_back(); } diff --git a/src/DataTypes/Serializations/SerializationNullable.h b/src/DataTypes/Serializations/SerializationNullable.h index 37858ccdefd..f7d2d2eadf0 100644 --- a/src/DataTypes/Serializations/SerializationNullable.h +++ b/src/DataTypes/Serializations/SerializationNullable.h @@ -29,7 +29,8 @@ public: void deserializeBinaryBulkStatePrefix( DeserializeBinaryBulkSettings & settings, - DeserializeBinaryBulkStatePtr & state) const override; + DeserializeBinaryBulkStatePtr & state, + SubstreamsDeserializeStatesCache * cache) const override; void serializeBinaryBulkWithMultipleStreams( const IColumn & column, diff --git a/src/DataTypes/Serializations/SerializationObject.cpp b/src/DataTypes/Serializations/SerializationObject.cpp index 67bf7af7799..88244a89204 100644 --- a/src/DataTypes/Serializations/SerializationObject.cpp +++ b/src/DataTypes/Serializations/SerializationObject.cpp @@ -210,7 +210,8 @@ void SerializationObject::serializeBinaryBulkStateSuffix( template void SerializationObject::deserializeBinaryBulkStatePrefix( DeserializeBinaryBulkSettings & settings, - DeserializeBinaryBulkStatePtr & state) const + DeserializeBinaryBulkStatePtr & state, + SubstreamsDeserializeStatesCache * cache) const { checkSerializationIsSupported(settings); if (state) @@ -258,7 +259,7 @@ void SerializationObject::deserializeBinaryBulkStatePrefix( } settings.path.push_back(Substream::ObjectData); - state_object->nested_serialization->deserializeBinaryBulkStatePrefix(settings, state_object->nested_state); + state_object->nested_serialization->deserializeBinaryBulkStatePrefix(settings, state_object->nested_state, cache); settings.path.pop_back(); state = std::move(state_object); diff --git a/src/DataTypes/Serializations/SerializationObject.h b/src/DataTypes/Serializations/SerializationObject.h index 39e1c514640..bdc5a9d7e2f 100644 --- a/src/DataTypes/Serializations/SerializationObject.h +++ b/src/DataTypes/Serializations/SerializationObject.h @@ -41,7 +41,8 @@ public: void deserializeBinaryBulkStatePrefix( DeserializeBinaryBulkSettings & settings, - DeserializeBinaryBulkStatePtr & state) const override; + DeserializeBinaryBulkStatePtr & state, + SubstreamsDeserializeStatesCache * /*cache*/) const override; void serializeBinaryBulkWithMultipleStreams( const IColumn & column, diff --git a/src/DataTypes/Serializations/SerializationSparse.cpp b/src/DataTypes/Serializations/SerializationSparse.cpp index 4d7514271ad..b125df564b3 100644 --- a/src/DataTypes/Serializations/SerializationSparse.cpp +++ b/src/DataTypes/Serializations/SerializationSparse.cpp @@ -242,12 +242,13 @@ void SerializationSparse::serializeBinaryBulkStateSuffix( void SerializationSparse::deserializeBinaryBulkStatePrefix( DeserializeBinaryBulkSettings & settings, - DeserializeBinaryBulkStatePtr & state) const + DeserializeBinaryBulkStatePtr & state, + SubstreamsDeserializeStatesCache * cache) const { auto state_sparse = std::make_shared(); settings.path.push_back(Substream::SparseElements); - nested->deserializeBinaryBulkStatePrefix(settings, state_sparse->nested); + nested->deserializeBinaryBulkStatePrefix(settings, state_sparse->nested, cache); settings.path.pop_back(); state = std::move(state_sparse); diff --git a/src/DataTypes/Serializations/SerializationSparse.h b/src/DataTypes/Serializations/SerializationSparse.h index b1ed7b613f0..a55856bacf0 100644 --- a/src/DataTypes/Serializations/SerializationSparse.h +++ b/src/DataTypes/Serializations/SerializationSparse.h @@ -43,7 +43,8 @@ public: void deserializeBinaryBulkStatePrefix( DeserializeBinaryBulkSettings & settings, - DeserializeBinaryBulkStatePtr & state) const override; + DeserializeBinaryBulkStatePtr & state, + SubstreamsDeserializeStatesCache * cache) const override; /// Allows to write ColumnSparse and other columns in sparse serialization. void serializeBinaryBulkWithMultipleStreams( diff --git a/src/DataTypes/Serializations/SerializationTuple.cpp b/src/DataTypes/Serializations/SerializationTuple.cpp index 632a019d2d9..bb7c19aa78d 100644 --- a/src/DataTypes/Serializations/SerializationTuple.cpp +++ b/src/DataTypes/Serializations/SerializationTuple.cpp @@ -606,13 +606,14 @@ void SerializationTuple::serializeBinaryBulkStateSuffix( void SerializationTuple::deserializeBinaryBulkStatePrefix( DeserializeBinaryBulkSettings & settings, - DeserializeBinaryBulkStatePtr & state) const + DeserializeBinaryBulkStatePtr & state, + SubstreamsDeserializeStatesCache * cache) const { auto tuple_state = std::make_shared(); tuple_state->states.resize(elems.size()); for (size_t i = 0; i < elems.size(); ++i) - elems[i]->deserializeBinaryBulkStatePrefix(settings, tuple_state->states[i]); + elems[i]->deserializeBinaryBulkStatePrefix(settings, tuple_state->states[i], cache); state = std::move(tuple_state); } diff --git a/src/DataTypes/Serializations/SerializationTuple.h b/src/DataTypes/Serializations/SerializationTuple.h index d9c63a05217..810673d8b21 100644 --- a/src/DataTypes/Serializations/SerializationTuple.h +++ b/src/DataTypes/Serializations/SerializationTuple.h @@ -53,7 +53,8 @@ public: void deserializeBinaryBulkStatePrefix( DeserializeBinaryBulkSettings & settings, - DeserializeBinaryBulkStatePtr & state) const override; + DeserializeBinaryBulkStatePtr & state, + SubstreamsDeserializeStatesCache * cache) const override; void serializeBinaryBulkWithMultipleStreams( const IColumn & column, diff --git a/src/DataTypes/Serializations/SerializationVariant.cpp b/src/DataTypes/Serializations/SerializationVariant.cpp index 0490ed62c3b..81c8a1f9ffa 100644 --- a/src/DataTypes/Serializations/SerializationVariant.cpp +++ b/src/DataTypes/Serializations/SerializationVariant.cpp @@ -70,25 +70,20 @@ void SerializationVariant::enumerateStreams( settings.path.pop_back(); } -struct SerializationVariant::SerializeBinaryBulkStateVariant : public ISerialization::SerializeBinaryBulkState +struct SerializeBinaryBulkStateVariant : public ISerialization::SerializeBinaryBulkState { SerializeBinaryBulkStateVariant(UInt64 mode) : discriminators_mode(mode) { } SerializationVariant::DiscriminatorsSerializationMode discriminators_mode; - std::vector states; + std::vector variant_states; }; -struct SerializationVariant::DeserializeBinaryBulkStateVariant : public ISerialization::DeserializeBinaryBulkState +struct DeserializeBinaryBulkStateVariant : public ISerialization::DeserializeBinaryBulkState { - DeserializeBinaryBulkStateVariant(UInt64 mode) : discriminators_mode(mode) - { - } - - SerializationVariant::DiscriminatorsSerializationMode discriminators_mode; - std::vector states; - SerializationVariant::DiscriminatorsDeserializationState discriminators_state; + ISerialization::DeserializeBinaryBulkStatePtr discriminators_state; + std::vector variant_states; }; void SerializationVariant::serializeBinaryBulkStatePrefix( @@ -108,14 +103,14 @@ void SerializationVariant::serializeBinaryBulkStatePrefix( const ColumnVariant & col = assert_cast(column); auto variant_state = std::make_shared(mode); - variant_state->states.resize(variants.size()); + variant_state->variant_states.resize(variants.size()); settings.path.push_back(Substream::VariantElements); for (size_t i = 0; i < variants.size(); ++i) { addVariantElementToPath(settings.path, i); - variants[i]->serializeBinaryBulkStatePrefix(col.getVariantByGlobalDiscriminator(i), settings, variant_state->states[i]); + variants[i]->serializeBinaryBulkStatePrefix(col.getVariantByGlobalDiscriminator(i), settings, variant_state->variant_states[i]); settings.path.pop_back(); } @@ -134,7 +129,7 @@ void SerializationVariant::serializeBinaryBulkStateSuffix( for (size_t i = 0; i < variants.size(); ++i) { addVariantElementToPath(settings.path, i); - variants[i]->serializeBinaryBulkStateSuffix(settings, variant_state->states[i]); + variants[i]->serializeBinaryBulkStateSuffix(settings, variant_state->variant_states[i]); settings.path.pop_back(); } settings.path.pop_back(); @@ -143,25 +138,39 @@ void SerializationVariant::serializeBinaryBulkStateSuffix( void SerializationVariant::deserializeBinaryBulkStatePrefix( DeserializeBinaryBulkSettings & settings, - DeserializeBinaryBulkStatePtr & state) const + DeserializeBinaryBulkStatePtr & state, + SubstreamsDeserializeStatesCache * cache) const { settings.path.push_back(Substream::VariantDiscriminators); - auto * discriminators_stream = settings.getter(settings.path); + + DeserializeBinaryBulkStatePtr discriminators_state; + if (auto cached_state = getFromSubstreamsDeserializeStatesCache(cache, settings.path)) + { + discriminators_state = cached_state; + } + else if (auto * discriminators_stream = settings.getter(settings.path)) + { + UInt64 mode; + readBinaryLittleEndian(mode, *discriminators_stream); + discriminators_state = std::make_shared(mode); + } + else + { + settings.path.pop_back(); + return; + } + settings.path.pop_back(); - if (!discriminators_stream) - return; - - UInt64 mode; - readBinaryLittleEndian(mode, *discriminators_stream); - auto variant_state = std::make_shared(mode); - variant_state->states.resize(variants.size()); + auto variant_state = std::make_shared(); + variant_state->discriminators_state = discriminators_state; + variant_state->variant_states.resize(variants.size()); settings.path.push_back(Substream::VariantElements); for (size_t i = 0; i < variants.size(); ++i) { addVariantElementToPath(settings.path, i); - variants[i]->deserializeBinaryBulkStatePrefix(settings, variant_state->states[i]); + variants[i]->deserializeBinaryBulkStatePrefix(settings, variant_state->variant_states[i], cache); settings.path.pop_back(); } @@ -216,7 +225,7 @@ void SerializationVariant::serializeBinaryBulkWithMultipleStreams( settings.path.push_back(Substream::VariantElements); addVariantElementToPath(settings.path, non_empty_global_discr); /// We can use the same offset/limit as for whole Variant column - variants[non_empty_global_discr]->serializeBinaryBulkWithMultipleStreams(col.getVariantByGlobalDiscriminator(non_empty_global_discr), offset, limit, settings, variant_state->states[non_empty_global_discr]); + variants[non_empty_global_discr]->serializeBinaryBulkWithMultipleStreams(col.getVariantByGlobalDiscriminator(non_empty_global_discr), offset, limit, settings, variant_state->variant_states[non_empty_global_discr]); settings.path.pop_back(); settings.path.pop_back(); return; @@ -266,7 +275,7 @@ void SerializationVariant::serializeBinaryBulkWithMultipleStreams( for (size_t i = 0; i != variants.size(); ++i) { addVariantElementToPath(settings.path, i); - variants[i]->serializeBinaryBulkWithMultipleStreams(col.getVariantByGlobalDiscriminator(i), 0, 0, settings, variant_state->states[i]); + variants[i]->serializeBinaryBulkWithMultipleStreams(col.getVariantByGlobalDiscriminator(i), 0, 0, settings, variant_state->variant_states[i]); settings.path.pop_back(); } settings.path.pop_back(); @@ -335,7 +344,7 @@ void SerializationVariant::serializeBinaryBulkWithMultipleStreams( variant_offsets_and_limits[i].first, variant_offsets_and_limits[i].second, settings, - variant_state->states[i]); + variant_state->variant_states[i]); settings.path.pop_back(); } } @@ -370,12 +379,13 @@ void SerializationVariant::deserializeBinaryBulkWithMultipleStreams( else if (auto * discriminators_stream = settings.getter(settings.path)) { variant_state = checkAndGetState(state); + auto discriminators_state = checkAndGetState(variant_state->discriminators_state); /// Deserialize discriminators according to serialization mode. - if (variant_state->discriminators_mode.value == DiscriminatorsSerializationMode::BASIC) + if (discriminators_state->mode.value == DiscriminatorsSerializationMode::BASIC) SerializationNumber().deserializeBinaryBulk(*col.getLocalDiscriminatorsPtr()->assumeMutable(), *discriminators_stream, limit, 0); else - variant_limits = deserializeCompactDiscriminators(col.getLocalDiscriminatorsPtr(), limit, discriminators_stream, settings.continuous_reading, variant_state->discriminators_state); + variant_limits = deserializeCompactDiscriminators(col.getLocalDiscriminatorsPtr(), limit, discriminators_stream, settings.continuous_reading, *discriminators_state); addToSubstreamsCache(cache, settings.path, col.getLocalDiscriminatorsPtr()); } @@ -419,7 +429,7 @@ void SerializationVariant::deserializeBinaryBulkWithMultipleStreams( for (size_t i = 0; i != variants.size(); ++i) { addVariantElementToPath(settings.path, i); - variants[i]->deserializeBinaryBulkWithMultipleStreams(col.getVariantPtrByLocalDiscriminator(i), variant_limits[i], settings, variant_state->states[i], cache); + variants[i]->deserializeBinaryBulkWithMultipleStreams(col.getVariantPtrByLocalDiscriminator(i), variant_limits[i], settings, variant_state->variant_states[i], cache); settings.path.pop_back(); } settings.path.pop_back(); @@ -494,7 +504,7 @@ std::vector SerializationVariant::deserializeCompactDiscriminators( size_t limit, ReadBuffer * stream, bool continuous_reading, - DiscriminatorsDeserializationState & state) const + DeserializeBinaryBulkStateVariantDiscriminators & state) const { auto & discriminators = assert_cast(*discriminators_column->assumeMutable()); auto & discriminators_data = discriminators.getData(); @@ -543,7 +553,7 @@ std::vector SerializationVariant::deserializeCompactDiscriminators( return variant_limits; } -void SerializationVariant::readDiscriminatorsGranuleStart(DB::SerializationVariant::DiscriminatorsDeserializationState & state, DB::ReadBuffer * stream) +void SerializationVariant::readDiscriminatorsGranuleStart(DeserializeBinaryBulkStateVariantDiscriminators & state, DB::ReadBuffer * stream) { UInt64 granule_size; readVarUInt(granule_size, *stream); diff --git a/src/DataTypes/Serializations/SerializationVariant.h b/src/DataTypes/Serializations/SerializationVariant.h index e6b067dc68a..a2f077964e6 100644 --- a/src/DataTypes/Serializations/SerializationVariant.h +++ b/src/DataTypes/Serializations/SerializationVariant.h @@ -96,7 +96,8 @@ public: void deserializeBinaryBulkStatePrefix( DeserializeBinaryBulkSettings & settings, - DeserializeBinaryBulkStatePtr & state) const override; + DeserializeBinaryBulkStatePtr & state, + SubstreamsDeserializeStatesCache * cache) const override; void serializeBinaryBulkWithMultipleStreams( const IColumn & column, @@ -161,25 +162,28 @@ private: COMPACT = 1, /// Granule has single discriminator for all rows and it is serialized as single value. }; - /// State of currently deserialized granule. - struct DiscriminatorsDeserializationState + struct DeserializeBinaryBulkStateVariantDiscriminators : public ISerialization::DeserializeBinaryBulkState { + DeserializeBinaryBulkStateVariantDiscriminators(UInt64 mode_) : mode(mode_) + { + } + + DiscriminatorsSerializationMode mode; + + /// Deserialize state of currently read granule in compact mode. CompactDiscriminatorsGranuleFormat granule_format = CompactDiscriminatorsGranuleFormat::PLAIN; size_t remaining_rows_in_granule = 0; ColumnVariant::Discriminator compact_discr = 0; }; - struct SerializeBinaryBulkStateVariant; - struct DeserializeBinaryBulkStateVariant; - std::vector deserializeCompactDiscriminators( ColumnPtr & discriminators_column, size_t limit, ReadBuffer * stream, bool continuous_reading, - DiscriminatorsDeserializationState & state) const; + DeserializeBinaryBulkStateVariantDiscriminators & state) const; - static void readDiscriminatorsGranuleStart(DiscriminatorsDeserializationState & state, ReadBuffer * stream); + static void readDiscriminatorsGranuleStart(DeserializeBinaryBulkStateVariantDiscriminators & state, ReadBuffer * stream); bool tryDeserializeTextEscapedImpl(IColumn & column, const String & field, const FormatSettings & settings) const; bool tryDeserializeTextQuotedImpl(IColumn & column, const String & field, const FormatSettings & settings) const; diff --git a/src/DataTypes/Serializations/SerializationVariantElement.cpp b/src/DataTypes/Serializations/SerializationVariantElement.cpp index 27166dd0704..19ac268268e 100644 --- a/src/DataTypes/Serializations/SerializationVariantElement.cpp +++ b/src/DataTypes/Serializations/SerializationVariantElement.cpp @@ -43,10 +43,6 @@ void SerializationVariantElement::serializeBinaryBulkStateSuffix(SerializeBinary struct SerializationVariantElement::DeserializeBinaryBulkStateVariantElement : public ISerialization::DeserializeBinaryBulkState { - DeserializeBinaryBulkStateVariantElement(UInt64 discriminators_version_) : discriminators_version(discriminators_version_) - { - } - /// During deserialization discriminators and variant streams can be shared. /// For example we can read several variant elements together: "select v.UInt32, v.String from table", /// or we can read the whole variant and some of variant elements: "select v, v.UInt32 from table". @@ -57,27 +53,39 @@ struct SerializationVariantElement::DeserializeBinaryBulkStateVariantElement : p /// substream cache correctly. ColumnPtr discriminators; ColumnPtr variant; - SerializationVariant::DiscriminatorsSerializationMode discriminators_version; - SerializationVariant::DiscriminatorsDeserializationState discriminators_state; + ISerialization::DeserializeBinaryBulkStatePtr discriminators_state; ISerialization::DeserializeBinaryBulkStatePtr variant_element_state; }; -void SerializationVariantElement::deserializeBinaryBulkStatePrefix(DeserializeBinaryBulkSettings & settings, DeserializeBinaryBulkStatePtr & state) const +void SerializationVariantElement::deserializeBinaryBulkStatePrefix( + DeserializeBinaryBulkSettings & settings, DeserializeBinaryBulkStatePtr & state, SubstreamsDeserializeStatesCache * cache) const { settings.path.push_back(Substream::VariantDiscriminators); - auto * discriminators_stream = settings.getter(settings.path); + + DeserializeBinaryBulkStatePtr discriminators_state; + if (auto cached_state = getFromSubstreamsDeserializeStatesCache(cache, settings.path)) + { + discriminators_state = cached_state; + } + else if (auto * discriminators_stream = settings.getter(settings.path)) + { + UInt64 mode; + readBinaryLittleEndian(mode, *discriminators_stream); + discriminators_state = std::make_shared(mode); + } + else + { + settings.path.pop_back(); + return; + } + settings.path.pop_back(); - if (!discriminators_stream) - return; - - UInt64 version; - readBinaryLittleEndian(version, *discriminators_stream); - - auto variant_element_state = std::make_shared(version); + auto variant_element_state = std::make_shared(); + variant_element_state->discriminators_state = discriminators_state; addVariantToPath(settings.path); - nested_serialization->deserializeBinaryBulkStatePrefix(settings, variant_element_state->variant_element_state); + nested_serialization->deserializeBinaryBulkStatePrefix(settings, variant_element_state->variant_element_state, cache); removeVariantFromPath(settings.path); state = std::move(variant_element_state); @@ -108,13 +116,14 @@ void SerializationVariantElement::deserializeBinaryBulkWithMultipleStreams( else if (auto * discriminators_stream = settings.getter(settings.path)) { variant_element_state = checkAndGetState(state); + auto discriminators_state = checkAndGetState(variant_element_state->discriminators_state); /// If we started to read a new column, reinitialize discriminators column in deserialization state. if (!variant_element_state->discriminators || result_column->empty()) variant_element_state->discriminators = ColumnVariant::ColumnDiscriminators::create(); /// Deserialize discriminators according to serialization mode. - if (variant_element_state->discriminators_version.value == SerializationVariant::DiscriminatorsSerializationMode::BASIC) + if (discriminators_state->mode.value == SerializationVariant::DiscriminatorsSerializationMode::BASIC) SerializationNumber().deserializeBinaryBulk(*variant_element_state->discriminators->assumeMutable(), *discriminators_stream, limit, 0); else variant_limit = deserializeCompactDiscriminators(variant_element_state->discriminators, limit, discriminators_stream, settings.continuous_reading, *variant_element_state); @@ -229,34 +238,35 @@ size_t SerializationVariantElement::deserializeCompactDiscriminators( size_t limit, DB::ReadBuffer * stream, bool continuous_reading, - DB::SerializationVariantElement::DeserializeBinaryBulkStateVariantElement & state) const + DeserializeBinaryBulkStateVariantElement & variant_element_state) const { + auto discriminators_state = checkAndGetState(variant_element_state.discriminators_state); auto & discriminators = assert_cast(*discriminators_column->assumeMutable()); auto & discriminators_data = discriminators.getData(); /// Reset state if we are reading from the start of the granule and not from the previous position in the file. if (!continuous_reading) - state.discriminators_state.remaining_rows_in_granule = 0; + discriminators_state->remaining_rows_in_granule = 0; /// Calculate our variant limit during discriminators deserialization. size_t variant_limit = 0; while (limit) { /// If we read all rows from current granule, start reading the next one. - if (state.discriminators_state.remaining_rows_in_granule == 0) + if (discriminators_state->remaining_rows_in_granule == 0) { if (stream->eof()) return variant_limit; - SerializationVariant::readDiscriminatorsGranuleStart(state.discriminators_state, stream); + SerializationVariant::readDiscriminatorsGranuleStart(*discriminators_state, stream); } - size_t limit_in_granule = std::min(limit, state.discriminators_state.remaining_rows_in_granule); - if (state.discriminators_state.granule_format == SerializationVariant::CompactDiscriminatorsGranuleFormat::COMPACT) + size_t limit_in_granule = std::min(limit, discriminators_state->remaining_rows_in_granule); + if (discriminators_state->granule_format == SerializationVariant::CompactDiscriminatorsGranuleFormat::COMPACT) { auto & data = discriminators.getData(); - data.resize_fill(data.size() + limit_in_granule, state.discriminators_state.compact_discr); - if (state.discriminators_state.compact_discr == variant_discriminator) + data.resize_fill(data.size() + limit_in_granule, discriminators_state->compact_discr); + if (discriminators_state->compact_discr == variant_discriminator) variant_limit += limit_in_granule; } else @@ -267,7 +277,7 @@ size_t SerializationVariantElement::deserializeCompactDiscriminators( variant_limit += (discriminators_data[i] == variant_discriminator); } - state.discriminators_state.remaining_rows_in_granule -= limit_in_granule; + discriminators_state->remaining_rows_in_granule -= limit_in_granule; limit -= limit_in_granule; } @@ -336,7 +346,7 @@ ColumnPtr SerializationVariantElement::VariantSubcolumnCreator::create(const DB: /// Special case for LowCardinality. We want the result to be LowCardinality(Nullable), /// but we don't have a good way to apply null-mask for LowCardinality(), so, we first - /// convert our column to LowCardinality(Nullable()) and then use expand which will + /// cont our column to LowCardinality(Nullable()) and then use expand which will /// fill rows with 0 in mask with default value (that is NULL). if (prev->lowCardinality()) res_column = assert_cast(*res_column).cloneNullable(); diff --git a/src/DataTypes/Serializations/SerializationVariantElement.h b/src/DataTypes/Serializations/SerializationVariantElement.h index e86d7ecefbe..f7779fb7b2d 100644 --- a/src/DataTypes/Serializations/SerializationVariantElement.h +++ b/src/DataTypes/Serializations/SerializationVariantElement.h @@ -43,7 +43,8 @@ public: void deserializeBinaryBulkStatePrefix( DeserializeBinaryBulkSettings & settings, - DeserializeBinaryBulkStatePtr & state) const override; + DeserializeBinaryBulkStatePtr & state, + SubstreamsDeserializeStatesCache * cache) const override; void serializeBinaryBulkWithMultipleStreams( const IColumn & column, @@ -69,7 +70,7 @@ private: size_t limit, ReadBuffer * stream, bool continuous_reading, - DeserializeBinaryBulkStateVariantElement & state) const; + DeserializeBinaryBulkStateVariantElement & variant_element_state) const; void addVariantToPath(SubstreamPath & path) const; void removeVariantFromPath(SubstreamPath & path) const; diff --git a/src/DataTypes/Serializations/SerializationWrapper.cpp b/src/DataTypes/Serializations/SerializationWrapper.cpp index bde52bb8096..ecef533d7e0 100644 --- a/src/DataTypes/Serializations/SerializationWrapper.cpp +++ b/src/DataTypes/Serializations/SerializationWrapper.cpp @@ -29,9 +29,10 @@ void SerializationWrapper::serializeBinaryBulkStateSuffix( void SerializationWrapper::deserializeBinaryBulkStatePrefix( DeserializeBinaryBulkSettings & settings, - DeserializeBinaryBulkStatePtr & state) const + DeserializeBinaryBulkStatePtr & state, + SubstreamsDeserializeStatesCache * cache) const { - nested_serialization->deserializeBinaryBulkStatePrefix(settings, state); + nested_serialization->deserializeBinaryBulkStatePrefix(settings, state, cache); } void SerializationWrapper::serializeBinaryBulkWithMultipleStreams( diff --git a/src/DataTypes/Serializations/SerializationWrapper.h b/src/DataTypes/Serializations/SerializationWrapper.h index 6c5e2046062..882f17bba0a 100644 --- a/src/DataTypes/Serializations/SerializationWrapper.h +++ b/src/DataTypes/Serializations/SerializationWrapper.h @@ -36,7 +36,8 @@ public: void deserializeBinaryBulkStatePrefix( DeserializeBinaryBulkSettings & settings, - DeserializeBinaryBulkStatePtr & state) const override; + DeserializeBinaryBulkStatePtr & state, + SubstreamsDeserializeStatesCache * cache) const override; void serializeBinaryBulkWithMultipleStreams( const IColumn & column, diff --git a/src/DataTypes/Serializations/tests/gtest_object_serialization.cpp b/src/DataTypes/Serializations/tests/gtest_object_serialization.cpp index fc7432d5bf6..c6337a31fce 100644 --- a/src/DataTypes/Serializations/tests/gtest_object_serialization.cpp +++ b/src/DataTypes/Serializations/tests/gtest_object_serialization.cpp @@ -49,7 +49,7 @@ TEST(SerializationObject, FromString) settings.position_independent_encoding = false; settings.getter = [&in](const auto &) { return ∈ }; - serialization->deserializeBinaryBulkStatePrefix(settings, state); + serialization->deserializeBinaryBulkStatePrefix(settings, state, nullptr); serialization->deserializeBinaryBulkWithMultipleStreams(result_column, column_string->size(), settings, state, nullptr); } diff --git a/src/Formats/NativeReader.cpp b/src/Formats/NativeReader.cpp index 8286b24d0a6..39915b0735e 100644 --- a/src/Formats/NativeReader.cpp +++ b/src/Formats/NativeReader.cpp @@ -93,7 +93,7 @@ void NativeReader::readData(const ISerialization & serialization, ColumnPtr & co ISerialization::DeserializeBinaryBulkStatePtr state; - serialization.deserializeBinaryBulkStatePrefix(settings, state); + serialization.deserializeBinaryBulkStatePrefix(settings, state, nullptr); serialization.deserializeBinaryBulkWithMultipleStreams(column, rows, settings, state, nullptr); if (column->size() != rows) diff --git a/src/Storages/MergeTree/MergeTreeIndexSet.cpp b/src/Storages/MergeTree/MergeTreeIndexSet.cpp index dba2bc1e56c..02a3f1b1165 100644 --- a/src/Storages/MergeTree/MergeTreeIndexSet.cpp +++ b/src/Storages/MergeTree/MergeTreeIndexSet.cpp @@ -116,7 +116,7 @@ void MergeTreeIndexGranuleSet::deserializeBinary(ReadBuffer & istr, MergeTreeInd ISerialization::DeserializeBinaryBulkStatePtr state; auto serialization = type->getDefaultSerialization(); - serialization->deserializeBinaryBulkStatePrefix(settings, state); + serialization->deserializeBinaryBulkStatePrefix(settings, state, nullptr); serialization->deserializeBinaryBulkWithMultipleStreams(new_column, rows_to_read, settings, state, nullptr); block.insert(ColumnWithTypeAndName(new_column, type, column.name)); diff --git a/src/Storages/MergeTree/MergeTreeReaderCompact.cpp b/src/Storages/MergeTree/MergeTreeReaderCompact.cpp index a22bff6b8d2..49548940dc2 100644 --- a/src/Storages/MergeTree/MergeTreeReaderCompact.cpp +++ b/src/Storages/MergeTree/MergeTreeReaderCompact.cpp @@ -195,7 +195,7 @@ void MergeTreeReaderCompact::readPrefix( deserialize_settings.getter = buffer_getter_for_prefix; ISerialization::DeserializeBinaryBulkStatePtr state_for_prefix; - serialization_for_prefix->deserializeBinaryBulkStatePrefix(deserialize_settings, state_for_prefix); + serialization_for_prefix->deserializeBinaryBulkStatePrefix(deserialize_settings, state_for_prefix, nullptr); } SerializationPtr serialization; @@ -206,7 +206,7 @@ void MergeTreeReaderCompact::readPrefix( deserialize_settings.getter = buffer_getter; - serialization->deserializeBinaryBulkStatePrefix(deserialize_settings, deserialize_binary_bulk_state_map[name_and_type.name]); + serialization->deserializeBinaryBulkStatePrefix(deserialize_settings, deserialize_binary_bulk_state_map[name_and_type.name], nullptr); } catch (Exception & e) { diff --git a/src/Storages/MergeTree/MergeTreeReaderWide.cpp b/src/Storages/MergeTree/MergeTreeReaderWide.cpp index 1fd886a6216..e9ff44306d2 100644 --- a/src/Storages/MergeTree/MergeTreeReaderWide.cpp +++ b/src/Storages/MergeTree/MergeTreeReaderWide.cpp @@ -100,9 +100,10 @@ void MergeTreeReaderWide::prefetchForAllColumns( try { auto & cache = caches[columns_to_read[pos].getNameInStorage()]; + auto & deserialize_states_cache = deserialize_states_caches[columns_to_read[pos].getNameInStorage()]; prefetchForColumn( priority, columns_to_read[pos], serializations[pos], from_mark, continue_reading, - current_task_last_mark, cache); + current_task_last_mark, cache, deserialize_states_cache); } catch (Exception & e) { @@ -128,9 +129,6 @@ size_t MergeTreeReaderWide::readRows( size_t num_columns = res_columns.size(); checkNumberOfColumns(num_columns); - if (deserialize_binary_bulk_state_map.empty()) - readPrefixes(num_columns, current_task_last_mark); - if (num_columns == 0) return max_rows_to_read; @@ -150,11 +148,12 @@ size_t MergeTreeReaderWide::readRows( { size_t column_size_before_reading = column->size(); auto & cache = caches[column_to_read.getNameInStorage()]; + auto & deserialize_states_cache = deserialize_states_caches[column_to_read.getNameInStorage()]; readData( column_to_read, serializations[pos], column, from_mark, continue_reading, current_task_last_mark, - max_rows_to_read, cache, /* was_prefetched =*/ !prefetched_streams.empty()); + max_rows_to_read, cache, deserialize_states_cache, /* was_prefetched =*/ !prefetched_streams.empty()); /// For elements of Nested, column_size_before_reading may be greater than column size /// if offsets are not empty and were already read, but elements are empty. @@ -293,30 +292,22 @@ static ReadBuffer * getStream( return stream.getDataBuffer(); } -void MergeTreeReaderWide::readPrefixes(size_t num_columns, size_t current_task_last_mark) -{ - for (size_t pos = 0; pos < num_columns; ++pos) - { - const auto & column_to_read = columns_to_read[pos]; - deserializePrefix(serializations[pos], column_to_read, current_task_last_mark); - } -} - void MergeTreeReaderWide::deserializePrefix( const SerializationPtr & serialization, const NameAndTypePair & name_and_type, - size_t current_task_last_mark) + size_t current_task_last_mark, + ISerialization::SubstreamsCache & cache, + ISerialization::SubstreamsDeserializeStatesCache & deserialize_states_cache) { const auto & name = name_and_type.name; if (!deserialize_binary_bulk_state_map.contains(name)) { ISerialization::DeserializeBinaryBulkSettings deserialize_settings; - ISerialization::SubstreamsCache empty_cache; deserialize_settings.getter = [&](const ISerialization::SubstreamPath & substream_path) { - return getStream(/* seek_to_start = */true, substream_path, data_part_info_for_read->getChecksums(), streams, name_and_type, 0, /* seek_to_mark = */false, current_task_last_mark, empty_cache); + return getStream(/* seek_to_start = */true, substream_path, data_part_info_for_read->getChecksums(), streams, name_and_type, 0, /* seek_to_mark = */false, current_task_last_mark, cache); }; - serialization->deserializeBinaryBulkStatePrefix(deserialize_settings, deserialize_binary_bulk_state_map[name]); + serialization->deserializeBinaryBulkStatePrefix(deserialize_settings, deserialize_binary_bulk_state_map[name], &deserialize_states_cache); } } @@ -327,8 +318,11 @@ void MergeTreeReaderWide::prefetchForColumn( size_t from_mark, bool continue_reading, size_t current_task_last_mark, - ISerialization::SubstreamsCache & cache) + ISerialization::SubstreamsCache & cache, + ISerialization::SubstreamsDeserializeStatesCache & deserialize_states_cache) { + deserializePrefix(serialization, name_and_type, current_task_last_mark, cache, deserialize_states_cache); + serialization->enumerateStreams([&](const ISerialization::SubstreamPath & substream_path) { auto stream_name = IMergeTreeDataPart::getStreamNameForColumn(name_and_type, substream_path, data_part_info_for_read->getChecksums()); @@ -349,11 +343,15 @@ void MergeTreeReaderWide::prefetchForColumn( void MergeTreeReaderWide::readData( const NameAndTypePair & name_and_type, const SerializationPtr & serialization, ColumnPtr & column, size_t from_mark, bool continue_reading, size_t current_task_last_mark, - size_t max_rows_to_read, ISerialization::SubstreamsCache & cache, bool was_prefetched) + size_t max_rows_to_read, ISerialization::SubstreamsCache & cache, + ISerialization::SubstreamsDeserializeStatesCache & deserialize_states_cache, bool was_prefetched) { double & avg_value_size_hint = avg_value_size_hints[name_and_type.name]; ISerialization::DeserializeBinaryBulkSettings deserialize_settings; deserialize_settings.avg_value_size_hint = avg_value_size_hint; + + deserializePrefix(serialization, name_and_type, current_task_last_mark, cache, deserialize_states_cache); + deserialize_settings.getter = [&](const ISerialization::SubstreamPath & substream_path) { bool seek_to_mark = !was_prefetched && !continue_reading; diff --git a/src/Storages/MergeTree/MergeTreeReaderWide.h b/src/Storages/MergeTree/MergeTreeReaderWide.h index 617e26864a9..edfd74ae3bc 100644 --- a/src/Storages/MergeTree/MergeTreeReaderWide.h +++ b/src/Storages/MergeTree/MergeTreeReaderWide.h @@ -52,7 +52,7 @@ private: void readData( const NameAndTypePair & name_and_type, const SerializationPtr & serialization, ColumnPtr & column, size_t from_mark, bool continue_reading, size_t current_task_last_mark, size_t max_rows_to_read, - ISerialization::SubstreamsCache & cache, bool was_prefetched); + ISerialization::SubstreamsCache & cache, ISerialization::SubstreamsDeserializeStatesCache & deserialize_states_cache, bool was_prefetched); /// Make next readData more simple by calling 'prefetch' of all related ReadBuffers (column streams). void prefetchForColumn( @@ -62,16 +62,18 @@ private: size_t from_mark, bool continue_reading, size_t current_task_last_mark, - ISerialization::SubstreamsCache & cache); - - void readPrefixes(size_t num_columns, size_t current_task_last_mark); + ISerialization::SubstreamsCache & cache, + ISerialization::SubstreamsDeserializeStatesCache & deserialize_states_cache); void deserializePrefix( const SerializationPtr & serialization, const NameAndTypePair & name_and_type, - size_t current_task_last_mark); + size_t current_task_last_mark, + ISerialization::SubstreamsCache & cache, + ISerialization::SubstreamsDeserializeStatesCache & deserialize_states_cache); std::unordered_map caches; + std::unordered_map deserialize_states_caches; std::unordered_set prefetched_streams; ssize_t prefetched_from_mark = -1; }; diff --git a/src/Storages/StorageLog.cpp b/src/Storages/StorageLog.cpp index b652750346f..3bdbddd61d6 100644 --- a/src/Storages/StorageLog.cpp +++ b/src/Storages/StorageLog.cpp @@ -254,7 +254,7 @@ void LogSource::readData(const NameAndTypePair & name_and_type, ColumnPtr & colu if (!deserialize_states.contains(name)) { settings.getter = create_stream_getter(true); - serialization->deserializeBinaryBulkStatePrefix(settings, deserialize_states[name]); + serialization->deserializeBinaryBulkStatePrefix(settings, deserialize_states[name], nullptr); } settings.getter = create_stream_getter(false); From b5a66167034f93248e2d420a37ad2fc8a40c87a4 Mon Sep 17 00:00:00 2001 From: avogar Date: Fri, 19 Apr 2024 19:19:49 +0000 Subject: [PATCH 007/417] Add state to cache --- src/DataTypes/Serializations/SerializationVariant.cpp | 1 + src/DataTypes/Serializations/SerializationVariantElement.cpp | 1 + 2 files changed, 2 insertions(+) diff --git a/src/DataTypes/Serializations/SerializationVariant.cpp b/src/DataTypes/Serializations/SerializationVariant.cpp index 81c8a1f9ffa..af41fb18cb4 100644 --- a/src/DataTypes/Serializations/SerializationVariant.cpp +++ b/src/DataTypes/Serializations/SerializationVariant.cpp @@ -153,6 +153,7 @@ void SerializationVariant::deserializeBinaryBulkStatePrefix( UInt64 mode; readBinaryLittleEndian(mode, *discriminators_stream); discriminators_state = std::make_shared(mode); + addToSubstreamsDeserializeStatesCache(cache, settings.path, discriminators_state); } else { diff --git a/src/DataTypes/Serializations/SerializationVariantElement.cpp b/src/DataTypes/Serializations/SerializationVariantElement.cpp index 19ac268268e..6ec41b3a3ef 100644 --- a/src/DataTypes/Serializations/SerializationVariantElement.cpp +++ b/src/DataTypes/Serializations/SerializationVariantElement.cpp @@ -72,6 +72,7 @@ void SerializationVariantElement::deserializeBinaryBulkStatePrefix( UInt64 mode; readBinaryLittleEndian(mode, *discriminators_stream); discriminators_state = std::make_shared(mode); + addToSubstreamsDeserializeStatesCache(cache, settings.path, discriminators_state); } else { From 77e5e8f3555067a8d7324ca16725601e18b87164 Mon Sep 17 00:00:00 2001 From: Kruglov Pavel <48961922+Avogar@users.noreply.github.com> Date: Fri, 19 Apr 2024 23:00:19 +0100 Subject: [PATCH 008/417] Fix special builds --- src/DataTypes/Serializations/SerializationVariant.h | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/src/DataTypes/Serializations/SerializationVariant.h b/src/DataTypes/Serializations/SerializationVariant.h index a2f077964e6..5bcb89b9367 100644 --- a/src/DataTypes/Serializations/SerializationVariant.h +++ b/src/DataTypes/Serializations/SerializationVariant.h @@ -164,7 +164,7 @@ private: struct DeserializeBinaryBulkStateVariantDiscriminators : public ISerialization::DeserializeBinaryBulkState { - DeserializeBinaryBulkStateVariantDiscriminators(UInt64 mode_) : mode(mode_) + explicit DeserializeBinaryBulkStateVariantDiscriminators(UInt64 mode_) : mode(mode_) { } From 8cbe3a61caa49e3e59d7e91b2fa74d975ead0c2e Mon Sep 17 00:00:00 2001 From: avogar Date: Sat, 20 Apr 2024 09:10:36 +0000 Subject: [PATCH 009/417] Fix special builds --- src/DataTypes/Serializations/SerializationVariant.cpp | 4 ++-- src/DataTypes/Serializations/SerializationVariantElement.cpp | 4 ++-- 2 files changed, 4 insertions(+), 4 deletions(-) diff --git a/src/DataTypes/Serializations/SerializationVariant.cpp b/src/DataTypes/Serializations/SerializationVariant.cpp index af41fb18cb4..f2b547cdd89 100644 --- a/src/DataTypes/Serializations/SerializationVariant.cpp +++ b/src/DataTypes/Serializations/SerializationVariant.cpp @@ -72,7 +72,7 @@ void SerializationVariant::enumerateStreams( struct SerializeBinaryBulkStateVariant : public ISerialization::SerializeBinaryBulkState { - SerializeBinaryBulkStateVariant(UInt64 mode) : discriminators_mode(mode) + explicit SerializeBinaryBulkStateVariant(UInt64 mode) : discriminators_mode(mode) { } @@ -380,7 +380,7 @@ void SerializationVariant::deserializeBinaryBulkWithMultipleStreams( else if (auto * discriminators_stream = settings.getter(settings.path)) { variant_state = checkAndGetState(state); - auto discriminators_state = checkAndGetState(variant_state->discriminators_state); + auto * discriminators_state = checkAndGetState(variant_state->discriminators_state); /// Deserialize discriminators according to serialization mode. if (discriminators_state->mode.value == DiscriminatorsSerializationMode::BASIC) diff --git a/src/DataTypes/Serializations/SerializationVariantElement.cpp b/src/DataTypes/Serializations/SerializationVariantElement.cpp index 6ec41b3a3ef..2104d2bc53e 100644 --- a/src/DataTypes/Serializations/SerializationVariantElement.cpp +++ b/src/DataTypes/Serializations/SerializationVariantElement.cpp @@ -117,7 +117,7 @@ void SerializationVariantElement::deserializeBinaryBulkWithMultipleStreams( else if (auto * discriminators_stream = settings.getter(settings.path)) { variant_element_state = checkAndGetState(state); - auto discriminators_state = checkAndGetState(variant_element_state->discriminators_state); + auto * discriminators_state = checkAndGetState(variant_element_state->discriminators_state); /// If we started to read a new column, reinitialize discriminators column in deserialization state. if (!variant_element_state->discriminators || result_column->empty()) @@ -241,7 +241,7 @@ size_t SerializationVariantElement::deserializeCompactDiscriminators( bool continuous_reading, DeserializeBinaryBulkStateVariantElement & variant_element_state) const { - auto discriminators_state = checkAndGetState(variant_element_state.discriminators_state); + auto * discriminators_state = checkAndGetState(variant_element_state.discriminators_state); auto & discriminators = assert_cast(*discriminators_column->assumeMutable()); auto & discriminators_data = discriminators.getData(); From e56cf30995063f2930a941555cc05a5e54aad77a Mon Sep 17 00:00:00 2001 From: avogar Date: Sat, 20 Apr 2024 09:11:19 +0000 Subject: [PATCH 010/417] Fix typo in comment --- src/DataTypes/Serializations/SerializationVariantElement.cpp | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/src/DataTypes/Serializations/SerializationVariantElement.cpp b/src/DataTypes/Serializations/SerializationVariantElement.cpp index 2104d2bc53e..b56e4125612 100644 --- a/src/DataTypes/Serializations/SerializationVariantElement.cpp +++ b/src/DataTypes/Serializations/SerializationVariantElement.cpp @@ -347,7 +347,7 @@ ColumnPtr SerializationVariantElement::VariantSubcolumnCreator::create(const DB: /// Special case for LowCardinality. We want the result to be LowCardinality(Nullable), /// but we don't have a good way to apply null-mask for LowCardinality(), so, we first - /// cont our column to LowCardinality(Nullable()) and then use expand which will + /// convert our column to LowCardinality(Nullable()) and then use expand which will /// fill rows with 0 in mask with default value (that is NULL). if (prev->lowCardinality()) res_column = assert_cast(*res_column).cloneNullable(); From 87f8c680531cbccc3aff039507153c09c6c0b0e1 Mon Sep 17 00:00:00 2001 From: avogar Date: Sat, 20 Apr 2024 10:24:02 +0000 Subject: [PATCH 011/417] Reduce code duplication --- .../Serializations/SerializationVariant.cpp | 45 ++++++++++--------- .../Serializations/SerializationVariant.h | 4 ++ .../SerializationVariantElement.cpp | 22 +-------- 3 files changed, 31 insertions(+), 40 deletions(-) diff --git a/src/DataTypes/Serializations/SerializationVariant.cpp b/src/DataTypes/Serializations/SerializationVariant.cpp index f2b547cdd89..1297ce15eae 100644 --- a/src/DataTypes/Serializations/SerializationVariant.cpp +++ b/src/DataTypes/Serializations/SerializationVariant.cpp @@ -141,27 +141,9 @@ void SerializationVariant::deserializeBinaryBulkStatePrefix( DeserializeBinaryBulkStatePtr & state, SubstreamsDeserializeStatesCache * cache) const { - settings.path.push_back(Substream::VariantDiscriminators); - - DeserializeBinaryBulkStatePtr discriminators_state; - if (auto cached_state = getFromSubstreamsDeserializeStatesCache(cache, settings.path)) - { - discriminators_state = cached_state; - } - else if (auto * discriminators_stream = settings.getter(settings.path)) - { - UInt64 mode; - readBinaryLittleEndian(mode, *discriminators_stream); - discriminators_state = std::make_shared(mode); - addToSubstreamsDeserializeStatesCache(cache, settings.path, discriminators_state); - } - else - { - settings.path.pop_back(); + DeserializeBinaryBulkStatePtr discriminators_state = deserializeDiscriminatorsStatePrefix(settings, cache); + if (!discriminators_state) return; - } - - settings.path.pop_back(); auto variant_state = std::make_shared(); variant_state->discriminators_state = discriminators_state; @@ -179,6 +161,29 @@ void SerializationVariant::deserializeBinaryBulkStatePrefix( state = std::move(variant_state); } +ISerialization::DeserializeBinaryBulkStatePtr SerializationVariant::deserializeDiscriminatorsStatePrefix( + DeserializeBinaryBulkSettings & settings, + SubstreamsDeserializeStatesCache * cache) const +{ + settings.path.push_back(Substream::VariantDiscriminators); + + DeserializeBinaryBulkStatePtr discriminators_state = nullptr; + if (auto cached_state = getFromSubstreamsDeserializeStatesCache(cache, settings.path)) + { + discriminators_state = cached_state; + } + else if (auto * discriminators_stream = settings.getter(settings.path)) + { + UInt64 mode; + readBinaryLittleEndian(mode, *discriminators_stream); + discriminators_state = std::make_shared(mode); + addToSubstreamsDeserializeStatesCache(cache, settings.path, discriminators_state); + } + + settings.path.pop_back(); + return discriminators_state; +} + void SerializationVariant::serializeBinaryBulkWithMultipleStreams( const IColumn & column, diff --git a/src/DataTypes/Serializations/SerializationVariant.h b/src/DataTypes/Serializations/SerializationVariant.h index 5bcb89b9367..ae7216e2223 100644 --- a/src/DataTypes/Serializations/SerializationVariant.h +++ b/src/DataTypes/Serializations/SerializationVariant.h @@ -176,6 +176,10 @@ private: ColumnVariant::Discriminator compact_discr = 0; }; + static DeserializeBinaryBulkStatePtr deserializeDiscriminatorsStatePrefix( + DeserializeBinaryBulkSettings & settings, + SubstreamsDeserializeStatesCache * cache) const; + std::vector deserializeCompactDiscriminators( ColumnPtr & discriminators_column, size_t limit, diff --git a/src/DataTypes/Serializations/SerializationVariantElement.cpp b/src/DataTypes/Serializations/SerializationVariantElement.cpp index b56e4125612..86c6cb4982d 100644 --- a/src/DataTypes/Serializations/SerializationVariantElement.cpp +++ b/src/DataTypes/Serializations/SerializationVariantElement.cpp @@ -60,27 +60,9 @@ struct SerializationVariantElement::DeserializeBinaryBulkStateVariantElement : p void SerializationVariantElement::deserializeBinaryBulkStatePrefix( DeserializeBinaryBulkSettings & settings, DeserializeBinaryBulkStatePtr & state, SubstreamsDeserializeStatesCache * cache) const { - settings.path.push_back(Substream::VariantDiscriminators); - - DeserializeBinaryBulkStatePtr discriminators_state; - if (auto cached_state = getFromSubstreamsDeserializeStatesCache(cache, settings.path)) - { - discriminators_state = cached_state; - } - else if (auto * discriminators_stream = settings.getter(settings.path)) - { - UInt64 mode; - readBinaryLittleEndian(mode, *discriminators_stream); - discriminators_state = std::make_shared(mode); - addToSubstreamsDeserializeStatesCache(cache, settings.path, discriminators_state); - } - else - { - settings.path.pop_back(); + DeserializeBinaryBulkStatePtr discriminators_state = SerializationVariant::deserializeDiscriminatorsStatePrefix(settings, cache); + if (!discriminators_state) return; - } - - settings.path.pop_back(); auto variant_element_state = std::make_shared(); variant_element_state->discriminators_state = discriminators_state; From 7445e51f7394fa48328098ecc9ce9f1aea239841 Mon Sep 17 00:00:00 2001 From: Kruglov Pavel <48961922+Avogar@users.noreply.github.com> Date: Sat, 20 Apr 2024 11:56:25 +0100 Subject: [PATCH 012/417] Fix build --- src/DataTypes/Serializations/SerializationVariant.h | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/src/DataTypes/Serializations/SerializationVariant.h b/src/DataTypes/Serializations/SerializationVariant.h index ae7216e2223..7c9c3042078 100644 --- a/src/DataTypes/Serializations/SerializationVariant.h +++ b/src/DataTypes/Serializations/SerializationVariant.h @@ -178,7 +178,7 @@ private: static DeserializeBinaryBulkStatePtr deserializeDiscriminatorsStatePrefix( DeserializeBinaryBulkSettings & settings, - SubstreamsDeserializeStatesCache * cache) const; + SubstreamsDeserializeStatesCache * cache); std::vector deserializeCompactDiscriminators( ColumnPtr & discriminators_column, From 8c5c54ba71f973e012aa59087805eab2364afa8e Mon Sep 17 00:00:00 2001 From: Kruglov Pavel <48961922+Avogar@users.noreply.github.com> Date: Sat, 20 Apr 2024 13:09:54 +0100 Subject: [PATCH 013/417] Don't mark static method as const --- src/DataTypes/Serializations/SerializationVariant.cpp | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/src/DataTypes/Serializations/SerializationVariant.cpp b/src/DataTypes/Serializations/SerializationVariant.cpp index 1297ce15eae..ea8285ac226 100644 --- a/src/DataTypes/Serializations/SerializationVariant.cpp +++ b/src/DataTypes/Serializations/SerializationVariant.cpp @@ -163,7 +163,7 @@ void SerializationVariant::deserializeBinaryBulkStatePrefix( ISerialization::DeserializeBinaryBulkStatePtr SerializationVariant::deserializeDiscriminatorsStatePrefix( DeserializeBinaryBulkSettings & settings, - SubstreamsDeserializeStatesCache * cache) const + SubstreamsDeserializeStatesCache * cache) { settings.path.push_back(Substream::VariantDiscriminators); From 72787fc0fc53c85a6f58808874c7b167554f3ebb Mon Sep 17 00:00:00 2001 From: Kruglov Pavel <48961922+Avogar@users.noreply.github.com> Date: Thu, 23 May 2024 20:19:38 +0200 Subject: [PATCH 014/417] Fix bad conflict resolution --- src/DataTypes/Serializations/ISerialization.h | 2 -- 1 file changed, 2 deletions(-) diff --git a/src/DataTypes/Serializations/ISerialization.h b/src/DataTypes/Serializations/ISerialization.h index 2714dfbee1f..89e2079490e 100644 --- a/src/DataTypes/Serializations/ISerialization.h +++ b/src/DataTypes/Serializations/ISerialization.h @@ -422,8 +422,6 @@ public: static void addToSubstreamsCache(SubstreamsCache * cache, const SubstreamPath & path, ColumnPtr column); static ColumnPtr getFromSubstreamsCache(SubstreamsCache * cache, const SubstreamPath & path); - static void addToSubstreamsDeserializeStatesCache(SubstreamsDeserializeStatesCache * cache, const SubstreamPath & path, DeserializeBinaryBulkStatePtr state); - static DeserializeBinaryBulkStatePtr getFromSubstreamsDeserializeStatesCache(SubstreamsDeserializeStatesCache * cache, const SubstreamPath & path); static void addToSubstreamsDeserializeStatesCache(SubstreamsDeserializeStatesCache * cache, const SubstreamPath & path, DeserializeBinaryBulkStatePtr state); static DeserializeBinaryBulkStatePtr getFromSubstreamsDeserializeStatesCache(SubstreamsDeserializeStatesCache * cache, const SubstreamPath & path); From bc164f842cf3eda28dd3a3d156d5c8c629a51b24 Mon Sep 17 00:00:00 2001 From: avogar Date: Fri, 24 May 2024 12:31:36 +0000 Subject: [PATCH 015/417] Fix tests --- src/DataTypes/Serializations/SerializationVariant.cpp | 4 ++++ 1 file changed, 4 insertions(+) diff --git a/src/DataTypes/Serializations/SerializationVariant.cpp b/src/DataTypes/Serializations/SerializationVariant.cpp index a202cdb71ce..8cdd312a707 100644 --- a/src/DataTypes/Serializations/SerializationVariant.cpp +++ b/src/DataTypes/Serializations/SerializationVariant.cpp @@ -208,6 +208,10 @@ void SerializationVariant::serializeBinaryBulkWithMultipleStreamsAndUpdateVarian auto * variant_state = checkAndGetState(state); + /// Don't write anything if column is empty. + if (limit == 0) + return; + /// Write number of rows in this granule in compact mode. if (variant_state->discriminators_mode.value == DiscriminatorsSerializationMode::COMPACT) writeVarUInt(UInt64(limit), *discriminators_stream); From f7238356d1530925032d923dfbe3467a4e9c46be Mon Sep 17 00:00:00 2001 From: divanik Date: Mon, 27 May 2024 11:44:45 +0000 Subject: [PATCH 016/417] Commit from private --- programs/disks/CMakeLists.txt | 13 +- programs/disks/CommandChangeDirectory.cpp | 43 +++ programs/disks/CommandCopy.cpp | 61 +-- programs/disks/CommandLink.cpp | 45 +-- programs/disks/CommandList.cpp | 98 +++-- programs/disks/CommandListDisks.cpp | 63 +-- programs/disks/CommandMkDir.cpp | 47 +-- programs/disks/CommandMove.cpp | 47 +-- programs/disks/CommandRead.cpp | 58 +-- programs/disks/CommandRemove.cpp | 43 +-- programs/disks/CommandSwitchDisk.cpp | 47 +++ programs/disks/CommandWrite.cpp | 71 ++-- programs/disks/DisksApp.cpp | 448 +++++++++++++++------- programs/disks/DisksApp.h | 91 +++-- programs/disks/DisksClient.cpp | 42 ++ programs/disks/DisksClient.h | 316 +++++++++++++++ programs/disks/ICommand.cpp | 68 ++-- programs/disks/ICommand.h | 115 ++++-- programs/disks/ICommand_fwd.h | 10 + src/Disks/DiskSelector.cpp | 20 +- 20 files changed, 1166 insertions(+), 580 deletions(-) create mode 100644 programs/disks/CommandChangeDirectory.cpp create mode 100644 programs/disks/CommandSwitchDisk.cpp create mode 100644 programs/disks/DisksClient.cpp create mode 100644 programs/disks/DisksClient.h create mode 100644 programs/disks/ICommand_fwd.h diff --git a/programs/disks/CMakeLists.txt b/programs/disks/CMakeLists.txt index f0949fcfceb..c5b30d61706 100644 --- a/programs/disks/CMakeLists.txt +++ b/programs/disks/CMakeLists.txt @@ -1,18 +1,21 @@ set (CLICKHOUSE_DISKS_SOURCES - DisksApp.cpp ICommand.cpp + DisksClient.cpp + DisksApp.cpp CommandCopy.cpp - CommandLink.cpp - CommandList.cpp CommandListDisks.cpp + CommandList.cpp + CommandLink.cpp + CommandChangeDirectory.cpp CommandMkDir.cpp CommandMove.cpp CommandRead.cpp CommandRemove.cpp - CommandWrite.cpp) + CommandWrite.cpp + CommandSwitchDisk.cpp) if (CLICKHOUSE_CLOUD) - set (CLICKHOUSE_DISKS_SOURCES ${CLICKHOUSE_DISKS_SOURCES} CommandPackedIO.cpp) + set (CLICKHOUSE_DISKS_SOURCES ${CLICKHOUSE_DISKS_SOURCES} CommandPackedIO.cpp) endif () set (CLICKHOUSE_DISKS_LINK diff --git a/programs/disks/CommandChangeDirectory.cpp b/programs/disks/CommandChangeDirectory.cpp new file mode 100644 index 00000000000..9932d918099 --- /dev/null +++ b/programs/disks/CommandChangeDirectory.cpp @@ -0,0 +1,43 @@ +#include +#include "Common/Exception.h" +#include +#include "DisksApp.h" +#include "DisksClient.h" +#include "ICommand.h" + +namespace DB +{ + +namespace ErrorCodes +{ +extern const int BAD_ARGUMENTS; +} + +class CommandChangeDirectory final : public ICommand +{ +public: + explicit CommandChangeDirectory() : ICommand() + { + command_name = "cd"; + description = "Change directory"; + options_description.add_options()("path", po::value(), "the path of listing (mandatory, positional)")( + "disk", po::value(), "A disk where the path is changed"); + positional_options_description.add("path", 1); + } + + void executeImpl(const CommandLineOptions & options, DisksClient & client) override + { + DiskWithPath & disk = getDiskWithPath(client, options, "disk"); + // std::cerr << "Disk name: " << disk.getDisk()->getName() << std::endl; + String path = getValueFromCommandLineOptionsThrow(options, "path"); + // std::cerr << "Disk path: " << path << std::endl; + disk.setPath(path); + } +}; + +CommandPtr makeCommandChangeDirectory() +{ + return std::make_unique(); +} + +} diff --git a/programs/disks/CommandCopy.cpp b/programs/disks/CommandCopy.cpp index f176fa277d7..e853e054f97 100644 --- a/programs/disks/CommandCopy.cpp +++ b/programs/disks/CommandCopy.cpp @@ -1,68 +1,45 @@ -#include "ICommand.h" #include #include +#include "DisksClient.h" +#include "ICommand.h" namespace DB { namespace ErrorCodes { - extern const int BAD_ARGUMENTS; +extern const int BAD_ARGUMENTS; } class CommandCopy final : public ICommand { public: - CommandCopy() + explicit CommandCopy() : ICommand() { command_name = "copy"; - command_option_description.emplace(createOptionsDescription("Allowed options", getTerminalWidth())); description = "Recursively copy data from `FROM_PATH` to `TO_PATH`"; - usage = "copy [OPTION]... "; - command_option_description->add_options() - ("disk-from", po::value(), "disk from which we copy") - ("disk-to", po::value(), "disk to which we copy"); + options_description.add_options()("disk-from", po::value(), "disk from which we copy")( + "disk-to", po::value(), "disk to which we copy")( + "path-from", po::value(), "path from which we copy (mandatory, positional)")( + "path-to", po::value(), "path to which we copy (mandatory, positional)"); + positional_options_description.add("path-from", 1); + positional_options_description.add("path-to", 1); } - void processOptions( - Poco::Util::LayeredConfiguration & config, - po::variables_map & options) const override + void executeImpl(const CommandLineOptions & options, DisksClient & client) override { - if (options.count("disk-from")) - config.setString("disk-from", options["disk-from"].as()); - if (options.count("disk-to")) - config.setString("disk-to", options["disk-to"].as()); - } + auto disk_from = getDiskWithPath(client, options, "disk-from"); + auto disk_to = getDiskWithPath(client, options, "disk-to"); + String path_from = disk_from.getRelativeFromRoot(getValueFromCommandLineOptionsThrow(options, "path-from")); + String path_to = disk_to.getRelativeFromRoot(getValueFromCommandLineOptionsThrow(options, "path-to")); - void execute( - const std::vector & command_arguments, - std::shared_ptr & disk_selector, - Poco::Util::LayeredConfiguration & config) override - { - if (command_arguments.size() != 2) - { - printHelpMessage(); - throw DB::Exception(DB::ErrorCodes::BAD_ARGUMENTS, "Bad Arguments"); - } - - String disk_name_from = config.getString("disk-from", config.getString("disk", "default")); - String disk_name_to = config.getString("disk-to", config.getString("disk", "default")); - - const String & path_from = command_arguments[0]; - const String & path_to = command_arguments[1]; - - DiskPtr disk_from = disk_selector->get(disk_name_from); - DiskPtr disk_to = disk_selector->get(disk_name_to); - - String relative_path_from = validatePathAndGetAsRelative(path_from); - String relative_path_to = validatePathAndGetAsRelative(path_to); - - disk_from->copyDirectoryContent(relative_path_from, disk_to, relative_path_to, /* read_settings= */ {}, /* write_settings= */ {}, /* cancellation_hook= */ {}); + disk_from.getDisk()->copyDirectoryContent( + path_from, disk_to.getDisk(), path_to, /* read_settings= */ {}, /* write_settings= */ {}, /* cancellation_hook= */ {}); } }; -} -std::unique_ptr makeCommandCopy() +CommandPtr makeCommandCopy() { return std::make_unique(); } +} diff --git a/programs/disks/CommandLink.cpp b/programs/disks/CommandLink.cpp index dbaa3162f82..8b467891d18 100644 --- a/programs/disks/CommandLink.cpp +++ b/programs/disks/CommandLink.cpp @@ -1,12 +1,12 @@ -#include "ICommand.h" #include +#include "ICommand.h" namespace DB { namespace ErrorCodes { - extern const int BAD_ARGUMENTS; +extern const int BAD_ARGUMENTS; } class CommandLink final : public ICommand @@ -16,42 +16,27 @@ public: { command_name = "link"; description = "Create hardlink from `from_path` to `to_path`"; - usage = "link [OPTION]... "; + options_description.add_options()( + "path-to", po::value(), "the path from which a hard link will be created (mandatory, positional)")( + "path-from", po::value(), "the path where a hard link will be created (mandatory, positional)"); + positional_options_description.add("path-from", 1); + positional_options_description.add("path-to", 1); } - void processOptions( - Poco::Util::LayeredConfiguration &, - po::variables_map &) const override + void executeImpl(const CommandLineOptions & options, DisksClient & client) override { - } + auto disk = client.getCurrentDiskWithPath(); - void execute( - const std::vector & command_arguments, - std::shared_ptr & disk_selector, - Poco::Util::LayeredConfiguration & config) override - { - if (command_arguments.size() != 2) - { - printHelpMessage(); - throw DB::Exception(DB::ErrorCodes::BAD_ARGUMENTS, "Bad Arguments"); - } + const String & path_from = disk.getRelativeFromRoot(getValueFromCommandLineOptionsThrow(options, "path-from")); + const String & path_to = disk.getRelativeFromRoot(getValueFromCommandLineOptionsThrow(options, "path-to")); - String disk_name = config.getString("disk", "default"); - - const String & path_from = command_arguments[0]; - const String & path_to = command_arguments[1]; - - DiskPtr disk = disk_selector->get(disk_name); - - String relative_path_from = validatePathAndGetAsRelative(path_from); - String relative_path_to = validatePathAndGetAsRelative(path_to); - - disk->createHardLink(relative_path_from, relative_path_to); + disk.getDisk()->createHardLink(path_from, path_to); } }; -} -std::unique_ptr makeCommandLink() +CommandPtr makeCommandLink() { return std::make_unique(); } + +} diff --git a/programs/disks/CommandList.cpp b/programs/disks/CommandList.cpp index 7213802ea86..26a576abc7d 100644 --- a/programs/disks/CommandList.cpp +++ b/programs/disks/CommandList.cpp @@ -1,98 +1,92 @@ -#include "ICommand.h" #include #include +#include "DisksApp.h" +#include "DisksClient.h" +#include "ICommand.h" namespace DB { namespace ErrorCodes { - extern const int BAD_ARGUMENTS; +extern const int BAD_ARGUMENTS; } class CommandList final : public ICommand { public: - CommandList() + explicit CommandList() : ICommand() { command_name = "list"; - command_option_description.emplace(createOptionsDescription("Allowed options", getTerminalWidth())); description = "List files at path[s]"; - usage = "list [OPTION]... ..."; - command_option_description->add_options() - ("recursive", "recursively list all directories"); + options_description.add_options()("recursive", "recursively list the directory")("all", "show hidden files")( + "path", po::value(), "the path of listing (mandatory, positional)"); + positional_options_description.add("path", 1); } - void processOptions( - Poco::Util::LayeredConfiguration & config, - po::variables_map & options) const override + void executeImpl(const CommandLineOptions & options, DisksClient & client) override { - if (options.count("recursive")) - config.setBool("recursive", true); - } - - void execute( - const std::vector & command_arguments, - std::shared_ptr & disk_selector, - Poco::Util::LayeredConfiguration & config) override - { - if (command_arguments.size() != 1) - { - printHelpMessage(); - throw DB::Exception(DB::ErrorCodes::BAD_ARGUMENTS, "Bad Arguments"); - } - - String disk_name = config.getString("disk", "default"); - - const String & path = command_arguments[0]; - - DiskPtr disk = disk_selector->get(disk_name); - - String relative_path = validatePathAndGetAsRelative(path); - - bool recursive = config.getBool("recursive", false); + bool recursive = options.count("recursive"); + bool show_hidden = options.count("all"); + auto disk = client.getCurrentDiskWithPath(); + String path = getValueFromCommandLineOptionsWithDefault(options, "path", ""); if (recursive) - listRecursive(disk, relative_path); + listRecursive(disk, disk.getAbsolutePath(path), show_hidden); else - list(disk, relative_path); + list(disk, path, show_hidden); } private: - static void list(const DiskPtr & disk, const std::string & relative_path) + static void list(const DiskWithPath & disk, const std::string & path, bool show_hidden) { - std::vector file_names; - disk->listFiles(relative_path, file_names); + std::vector file_names = disk.listAllFilesByPath(path); + std::vector selected_and_sorted_file_names{}; for (const auto & file_name : file_names) - std::cout << file_name << '\n'; + if (show_hidden || (!file_name.starts_with('.'))) + selected_and_sorted_file_names.push_back(file_name); + + std::sort(selected_and_sorted_file_names.begin(), selected_and_sorted_file_names.end()); + for (const auto & file_name : selected_and_sorted_file_names) + { + std::cout << file_name << "\n"; + } } - static void listRecursive(const DiskPtr & disk, const std::string & relative_path) + static void listRecursive(const DiskWithPath & disk, const std::string & absolute_path, bool show_hidden) { - std::vector file_names; - disk->listFiles(relative_path, file_names); + std::vector file_names = disk.listAllFilesByPath(absolute_path); + std::vector selected_and_sorted_file_names{}; - std::cout << relative_path << ":\n"; + std::cout << absolute_path << ":\n"; if (!file_names.empty()) { for (const auto & file_name : file_names) - std::cout << file_name << '\n'; - std::cout << "\n"; + if (show_hidden || (!file_name.starts_with('.'))) + selected_and_sorted_file_names.push_back(file_name); } + std::sort(selected_and_sorted_file_names.begin(), selected_and_sorted_file_names.end()); + for (const auto & file_name : selected_and_sorted_file_names) + { + std::cout << file_name << "\n"; + } + std::cout << "\n"; + for (const auto & file_name : file_names) { - auto path = relative_path.empty() ? file_name : (relative_path + "/" + file_name); - if (disk->isDirectory(path)) - listRecursive(disk, path); + auto path = absolute_path + "/" + file_name; + if (disk.isDirectory(path)) + if (show_hidden || (!file_name.starts_with('.'))) + listRecursive(disk, path, show_hidden); } } }; -} -std::unique_ptr makeCommandList() +CommandPtr makeCommandList() { - return std::make_unique(); + return std::make_shared(); +} } diff --git a/programs/disks/CommandListDisks.cpp b/programs/disks/CommandListDisks.cpp index 79da021fd00..16779b0fdae 100644 --- a/programs/disks/CommandListDisks.cpp +++ b/programs/disks/CommandListDisks.cpp @@ -1,68 +1,45 @@ -#include "ICommand.h" +#include #include +#include +#include "DisksClient.h" +#include "ICommand.h" namespace DB { namespace ErrorCodes { - extern const int BAD_ARGUMENTS; +extern const int BAD_ARGUMENTS; } class CommandListDisks final : public ICommand { public: - CommandListDisks() + explicit CommandListDisks() : ICommand() { command_name = "list-disks"; - description = "List disks names"; - usage = "list-disks [OPTION]"; + description = "Lists all available disks"; } - void processOptions( - Poco::Util::LayeredConfiguration &, - po::variables_map &) const override - {} - - void execute( - const std::vector & command_arguments, - std::shared_ptr &, - Poco::Util::LayeredConfiguration & config) override + void executeImpl(const CommandLineOptions &, DisksClient & client) override { - if (!command_arguments.empty()) + std::vector sorted_and_selected{}; + for (const auto & disk_name : client.getAllDiskNames()) { - printHelpMessage(); - throw DB::Exception(DB::ErrorCodes::BAD_ARGUMENTS, "Bad Arguments"); + sorted_and_selected.push_back(disk_name + ":" + client.getDiskWithPath(disk_name).getAbsolutePath("")); } - - constexpr auto config_prefix = "storage_configuration.disks"; - constexpr auto default_disk_name = "default"; - - Poco::Util::AbstractConfiguration::Keys keys; - config.keys(config_prefix, keys); - - bool has_default_disk = false; - - /// For the output to be ordered - std::set disks; - - for (const auto & disk_name : keys) + std::sort(sorted_and_selected.begin(), sorted_and_selected.end()); + for (const auto & disk_name : sorted_and_selected) { - if (disk_name == default_disk_name) - has_default_disk = true; - disks.insert(disk_name); + std::cout << disk_name << "\n"; } - - if (!has_default_disk) - disks.insert(default_disk_name); - - for (const auto & disk : disks) - std::cout << disk << '\n'; } -}; -} -std::unique_ptr makeCommandListDisks() +private: +}; + +CommandPtr makeCommandListDisks() { - return std::make_unique(); + return std::make_shared(); +} } diff --git a/programs/disks/CommandMkDir.cpp b/programs/disks/CommandMkDir.cpp index 6d33bdec498..23312435d4e 100644 --- a/programs/disks/CommandMkDir.cpp +++ b/programs/disks/CommandMkDir.cpp @@ -8,7 +8,7 @@ namespace DB namespace ErrorCodes { - extern const int BAD_ARGUMENTS; +extern const int BAD_ARGUMENTS; } class CommandMkDir final : public ICommand @@ -17,50 +17,29 @@ public: CommandMkDir() { command_name = "mkdir"; - command_option_description.emplace(createOptionsDescription("Allowed options", getTerminalWidth())); description = "Create a directory"; - usage = "mkdir [OPTION]... "; - command_option_description->add_options() - ("recursive", "recursively create directories"); + options_description.add_options()("recursive", "recursively create directories")( + "path", po::value(), "the path of listing (mandatory, positional)"); + positional_options_description.add("path", 1); } - void processOptions( - Poco::Util::LayeredConfiguration & config, - po::variables_map & options) const override + void executeImpl(const CommandLineOptions & options, DisksClient & client) override { - if (options.count("recursive")) - config.setBool("recursive", true); - } + bool recursive = options.count("recursive"); + auto disk = client.getCurrentDiskWithPath(); - void execute( - const std::vector & command_arguments, - std::shared_ptr & disk_selector, - Poco::Util::LayeredConfiguration & config) override - { - if (command_arguments.size() != 1) - { - printHelpMessage(); - throw DB::Exception(DB::ErrorCodes::BAD_ARGUMENTS, "Bad Arguments"); - } - - String disk_name = config.getString("disk", "default"); - - const String & path = command_arguments[0]; - - DiskPtr disk = disk_selector->get(disk_name); - - String relative_path = validatePathAndGetAsRelative(path); - bool recursive = config.getBool("recursive", false); + String path = disk.getRelativeFromRoot(getValueFromCommandLineOptionsThrow(options, "path")); if (recursive) - disk->createDirectories(relative_path); + disk.getDisk()->createDirectories(path); else - disk->createDirectory(relative_path); + disk.getDisk()->createDirectory(path); } }; -} -std::unique_ptr makeCommandMkDir() +CommandPtr makeCommandMkDir() { return std::make_unique(); } + +} diff --git a/programs/disks/CommandMove.cpp b/programs/disks/CommandMove.cpp index 75cf96252ed..25620de448e 100644 --- a/programs/disks/CommandMove.cpp +++ b/programs/disks/CommandMove.cpp @@ -1,12 +1,12 @@ -#include "ICommand.h" #include +#include "ICommand.h" namespace DB { namespace ErrorCodes { - extern const int BAD_ARGUMENTS; +extern const int BAD_ARGUMENTS; } class CommandMove final : public ICommand @@ -16,44 +16,29 @@ public: { command_name = "move"; description = "Move file or directory from `from_path` to `to_path`"; - usage = "move [OPTION]... "; + options_description.add_options()("path-from", po::value(), "path from which we copy (mandatory, positional)")( + "path-to", po::value(), "path to which we copy (mandatory, positional)"); + positional_options_description.add("path-from", 1); + positional_options_description.add("path-to", 1); } - void processOptions( - Poco::Util::LayeredConfiguration &, - po::variables_map &) const override - {} - - void execute( - const std::vector & command_arguments, - std::shared_ptr & disk_selector, - Poco::Util::LayeredConfiguration & config) override + void executeImpl(const CommandLineOptions & options, DisksClient & client) override { - if (command_arguments.size() != 2) - { - printHelpMessage(); - throw DB::Exception(DB::ErrorCodes::BAD_ARGUMENTS, "Bad Arguments"); - } + auto disk = client.getCurrentDiskWithPath(); - String disk_name = config.getString("disk", "default"); + String path_from = disk.getRelativeFromRoot(getValueFromCommandLineOptionsThrow(options, "path-from")); + String path_to = disk.getRelativeFromRoot(getValueFromCommandLineOptionsThrow(options, "path-to")); - const String & path_from = command_arguments[0]; - const String & path_to = command_arguments[1]; - - DiskPtr disk = disk_selector->get(disk_name); - - String relative_path_from = validatePathAndGetAsRelative(path_from); - String relative_path_to = validatePathAndGetAsRelative(path_to); - - if (disk->isFile(relative_path_from)) - disk->moveFile(relative_path_from, relative_path_to); + if (disk.getDisk()->isFile(path_from)) + disk.getDisk()->moveFile(path_from, path_from); else - disk->moveDirectory(relative_path_from, relative_path_to); + disk.getDisk()->moveDirectory(path_from, path_from); } }; -} -std::unique_ptr makeCommandMove() +CommandPtr makeCommandMove() { return std::make_unique(); } + +} diff --git a/programs/disks/CommandRead.cpp b/programs/disks/CommandRead.cpp index 0f3ac7ab98c..82ff90b6e02 100644 --- a/programs/disks/CommandRead.cpp +++ b/programs/disks/CommandRead.cpp @@ -1,16 +1,16 @@ -#include "ICommand.h" -#include #include #include #include +#include #include +#include "ICommand.h" namespace DB { namespace ErrorCodes { - extern const int BAD_ARGUMENTS; +extern const int BAD_ARGUMENTS; } class CommandRead final : public ICommand @@ -19,60 +19,44 @@ public: CommandRead() { command_name = "read"; - command_option_description.emplace(createOptionsDescription("Allowed options", getTerminalWidth())); + // command_option_description.emplace(createOptionsDescription("Allowed options", getTerminalWidth())); description = "Read a file from `FROM_PATH` to `TO_PATH`"; - usage = "read [OPTION]... []"; - command_option_description->add_options() - ("output", po::value(), "file to which we are reading, defaults to `stdout`"); + options_description.add_options()( + "path-from", po::value(), "file from which we are reading, defaults to `stdin` (mandatory, positional)")( + "path-to", po::value(), "file to which we are writing"); + positional_options_description.add("path-from", 1); } - void processOptions( - Poco::Util::LayeredConfiguration & config, - po::variables_map & options) const override + void executeImpl(const CommandLineOptions & options, DisksClient & client) override { - if (options.count("output")) - config.setString("output", options["output"].as()); - } - - void execute( - const std::vector & command_arguments, - std::shared_ptr & disk_selector, - Poco::Util::LayeredConfiguration & config) override - { - if (command_arguments.size() != 1) + auto disk = client.getCurrentDiskWithPath(); + String path_from = disk.getRelativeFromRoot(getValueFromCommandLineOptionsThrow(options, "path-from")); + std::optional path_to = getValueFromCommandLineOptionsWithOptional(options, "path-to"); + if (path_to.has_value()) { - printHelpMessage(); - throw DB::Exception(DB::ErrorCodes::BAD_ARGUMENTS, "Bad Arguments"); + path_to = std::optional{disk.getRelativeFromRoot(path_to.value())}; } - String disk_name = config.getString("disk", "default"); - - DiskPtr disk = disk_selector->get(disk_name); - - String relative_path = validatePathAndGetAsRelative(command_arguments[0]); - - String path_output = config.getString("output", ""); - - if (!path_output.empty()) + auto in = disk.getDisk()->readFile(path_from); + if (path_to.has_value()) { - String relative_path_output = validatePathAndGetAsRelative(path_output); + String relative_path_to = disk.getRelativeFromRoot(path_to.value()); - auto in = disk->readFile(relative_path); - auto out = disk->writeFile(relative_path_output); + auto out = disk.getDisk()->writeFile(relative_path_to); copyData(*in, *out); out->finalize(); } else { - auto in = disk->readFile(relative_path); std::unique_ptr out = std::make_unique(STDOUT_FILENO); copyData(*in, *out); } } }; -} -std::unique_ptr makeCommandRead() +CommandPtr makeCommandRead() { return std::make_unique(); } + +} diff --git a/programs/disks/CommandRemove.cpp b/programs/disks/CommandRemove.cpp index 0c631eacff3..0344a09d156 100644 --- a/programs/disks/CommandRemove.cpp +++ b/programs/disks/CommandRemove.cpp @@ -1,12 +1,12 @@ -#include "ICommand.h" #include +#include "ICommand.h" namespace DB { namespace ErrorCodes { - extern const int BAD_ARGUMENTS; +extern const int BAD_ARGUMENTS; } class CommandRemove final : public ICommand @@ -15,40 +15,23 @@ public: CommandRemove() { command_name = "remove"; - description = "Remove file or directory with all children. Throws exception if file doesn't exists.\nPath should be in format './' or './path' or 'path'"; - usage = "remove [OPTION]... "; + description = "Remove file or directory with all children. Throws exception if file doesn't exists.\nPath should be in format './' " + "or './path' or 'path'"; + options_description.add_options()("path", po::value(), "path from which we copy (mandatory, positional)"); + positional_options_description.add("path", 1); } - void processOptions( - Poco::Util::LayeredConfiguration &, - po::variables_map &) const override - {} - - void execute( - const std::vector & command_arguments, - std::shared_ptr & disk_selector, - Poco::Util::LayeredConfiguration & config) override + void executeImpl(const CommandLineOptions & options, DisksClient & client) override { - if (command_arguments.size() != 1) - { - printHelpMessage(); - throw DB::Exception(DB::ErrorCodes::BAD_ARGUMENTS, "Bad Arguments"); - } - - String disk_name = config.getString("disk", "default"); - - const String & path = command_arguments[0]; - - DiskPtr disk = disk_selector->get(disk_name); - - String relative_path = validatePathAndGetAsRelative(path); - - disk->removeRecursive(relative_path); + auto disk = client.getCurrentDiskWithPath(); + const String & path = disk.getRelativeFromRoot(getValueFromCommandLineOptionsThrow(options, "path")); + disk.getDisk()->removeRecursive(path); } }; -} -std::unique_ptr makeCommandRemove() +CommandPtr makeCommandRemove() { return std::make_unique(); } + +} diff --git a/programs/disks/CommandSwitchDisk.cpp b/programs/disks/CommandSwitchDisk.cpp new file mode 100644 index 00000000000..6c1fbaa0623 --- /dev/null +++ b/programs/disks/CommandSwitchDisk.cpp @@ -0,0 +1,47 @@ +#include +#include +#include "Common/Exception.h" +#include +#include "DisksApp.h" +#include "ICommand.h" + +namespace DB +{ + +namespace ErrorCodes +{ +extern const int BAD_ARGUMENTS; +} + +class CommandSwitchDisk final : public ICommand +{ +public: + explicit CommandSwitchDisk() : ICommand() + { + command_name = "switch-disk"; + // options_description.emplace(createOptionsDescription("Allowed options", getTerminalWidth())); + description = "Change disk"; + // options_description->add_options()("recursive", "recursively list all directories"); + options_description.add_options()("disk", po::value(), "the disk to switch to (mandatory, positional)")( + "path", po::value(), "the path to switch on the disk"); + positional_options_description.add("disk", 1); + } + + void executeImpl(const CommandLineOptions & options, DisksClient & client) override + { + String disk = getValueFromCommandLineOptions(options, "disk"); + std::optional path = getValueFromCommandLineOptionsWithOptional(options, "path"); + + if (!client.switchToDisk(disk, path)) + { + throw Exception( + ErrorCodes::BAD_ARGUMENTS, "Unable to switch to disk: {}, path: {}", disk, path.has_value() ? path.value() : "NO PATH"); + } + } +}; + +CommandPtr makeCommandSwitchDisk() +{ + return std::make_unique(); +} +} diff --git a/programs/disks/CommandWrite.cpp b/programs/disks/CommandWrite.cpp index 7ded37e067a..42999572443 100644 --- a/programs/disks/CommandWrite.cpp +++ b/programs/disks/CommandWrite.cpp @@ -1,17 +1,17 @@ -#include "ICommand.h" #include +#include "ICommand.h" -#include #include #include #include +#include namespace DB { namespace ErrorCodes { - extern const int BAD_ARGUMENTS; +extern const int BAD_ARGUMENTS; } class CommandWrite final : public ICommand @@ -20,60 +20,47 @@ public: CommandWrite() { command_name = "write"; - command_option_description.emplace(createOptionsDescription("Allowed options", getTerminalWidth())); + // command_option_description.emplace(createOptionsDescription("Allowed options", getTerminalWidth())); description = "Write a file from `FROM_PATH` to `TO_PATH`"; - usage = "write [OPTION]... [] "; - command_option_description->add_options() - ("input", po::value(), "file from which we are reading, defaults to `stdin`"); + options_description.add_options()("path-from", po::value(), "file from which we are reading, defaults to `stdin`")( + "path-to", po::value(), "file to which we are writing (mandatory, positional)"); + positional_options_description.add("path-to", 1); } - void processOptions( - Poco::Util::LayeredConfiguration & config, - po::variables_map & options) const override + void executeImpl(const CommandLineOptions & options, DisksClient & client) override { - if (options.count("input")) - config.setString("input", options["input"].as()); - } + auto disk = client.getCurrentDiskWithPath(); - void execute( - const std::vector & command_arguments, - std::shared_ptr & disk_selector, - Poco::Util::LayeredConfiguration & config) override - { - if (command_arguments.size() != 1) + std::optional path_from = getValueFromCommandLineOptionsWithOptional(options, "path-from"); + if (path_from.has_value()) { - printHelpMessage(); - throw DB::Exception(DB::ErrorCodes::BAD_ARGUMENTS, "Bad Arguments"); + path_from = std::optional{disk.getRelativeFromRoot(path_from.value())}; } - String disk_name = config.getString("disk", "default"); + String path_to = disk.getRelativeFromRoot(getValueFromCommandLineOptionsThrow(options, "path-to")); - const String & path = command_arguments[0]; - - DiskPtr disk = disk_selector->get(disk_name); - - String relative_path = validatePathAndGetAsRelative(path); - - String path_input = config.getString("input", ""); - std::unique_ptr in; - if (path_input.empty()) + auto in = [&]() -> std::unique_ptr { - in = std::make_unique(STDIN_FILENO); - } - else - { - String relative_path_input = validatePathAndGetAsRelative(path_input); - in = disk->readFile(relative_path_input); - } + if (!path_from.has_value()) + { + return std::make_unique(STDIN_FILENO); + } + else + { + String relative_path_from = disk.getRelativeFromRoot(path_from.value()); + return disk.getDisk()->readFile(relative_path_from); + } + }(); - auto out = disk->writeFile(relative_path); + auto out = disk.getDisk()->writeFile(path_to); copyData(*in, *out); out->finalize(); } }; + +CommandPtr makeCommandWrite() +{ + return std::make_shared(); } -std::unique_ptr makeCommandWrite() -{ - return std::make_unique(); } diff --git a/programs/disks/DisksApp.cpp b/programs/disks/DisksApp.cpp index 5da5ab4bae9..02e8b74b889 100644 --- a/programs/disks/DisksApp.cpp +++ b/programs/disks/DisksApp.cpp @@ -1,85 +1,263 @@ #include "DisksApp.h" +#include +#include +#include +#include +#include "Common/Exception.h" +#include +#include +#include +#include +#include "DisksClient.h" #include "ICommand.h" +#include +#include +#include +#include + #include -#include #include - +#include namespace DB { -namespace ErrorCodes +CommandPtr DisksApp::getCommandByName(String command) const { - extern const int BAD_ARGUMENTS; + auto it = aliases.find(command); + if (it != aliases.end()) + { + command = it->second; + } + try + { + return command_descriptions.at(command); + } + catch (...) + { + throw Exception(ErrorCodes::BAD_ARGUMENTS, "The command {} is unknown", command); + } } -size_t DisksApp::findCommandPos(std::vector & common_arguments) +std::vector DisksApp::getEmptyCompletion(CommandPtr command_) const { - for (size_t i = 0; i < common_arguments.size(); i++) - if (supported_commands.contains(common_arguments[i])) - return i + 1; - return common_arguments.size(); + auto answer = [&]() -> std::vector + { + if (multidisk_commands.contains(command_->command_name)) + { + return client->getAllFilesByPatternFromAllDisks(""); + } + else + { + return client->getCurrentDiskWithPath().getAllFilesByPattern(""); + } + }(); + for (const auto & disk_name : client->getAllDiskNames()) + { + answer.push_back(disk_name); + } + for (const auto & option : command_->options_description.options()) + { + answer.push_back("--" + option->long_name()); + } + std::sort(answer.begin(), answer.end()); + return answer; } -void DisksApp::printHelpMessage(ProgramOptionsDescription & command_option_description) +std::vector DisksApp::getCompletions(const String & prefix) const { - std::optional help_description = - createOptionsDescription("Help Message for clickhouse-disks", getTerminalWidth()); + auto arguments = split(prefix, word_break_characters); + if (arguments.empty()) + { + return {}; + } + if (word_break_characters.contains(prefix.back())) + { + CommandPtr command; + try + { + command = getCommandByName(arguments[0]); + } + catch (...) + { + return {arguments.back()}; + } + return getEmptyCompletion(command); + } + else if (arguments.size() == 1) + { + String command_prefix = arguments[0]; + std::vector answer{}; + for (const auto & [word, _] : command_descriptions) + { + if (word.starts_with(command_prefix)) + { + answer.push_back(word); + } + } + if (!answer.empty()) + { + return answer; + } + for (const auto & [word, _] : aliases) + { + if (word.starts_with(command_prefix)) + { + answer.push_back(word); + } + } + if (!answer.empty()) + { + return answer; + } + return {command_prefix}; + } + else + { + String last_token = arguments.back(); + CommandPtr command; + try + { + command = getCommandByName(arguments[0]); + } + catch (...) + { + return {last_token}; + } + auto answer = [&]() -> std::vector + { + if (multidisk_commands.contains(command->command_name)) + { + return client->getAllFilesByPatternFromAllDisks(last_token); + } + else + { + return client->getCurrentDiskWithPath().getAllFilesByPattern(last_token); + } + }(); - help_description->add(command_option_description); - - std::cout << "ClickHouse disk management tool\n"; - std::cout << "Usage: ./clickhouse-disks [OPTION]\n"; - std::cout << "clickhouse-disks\n\n"; - - for (const auto & current_command : supported_commands) - std::cout << command_descriptions[current_command]->command_name - << "\t" - << command_descriptions[current_command]->description - << "\n\n"; - - std::cout << command_option_description << '\n'; + for (const auto & disk_name : client->getAllDiskNames()) + { + if (disk_name.starts_with(last_token)) + { + answer.push_back(disk_name); + } + } + for (const auto & option : command->options_description.options()) + { + String option_sign = "--" + option->long_name(); + if (option_sign.starts_with(last_token)) + { + answer.push_back(option_sign); + } + } + if (!answer.empty()) + { + return answer; + } + else + { + return {last_token}; + } + } } -String DisksApp::getDefaultConfigFileName() +bool DisksApp::processQueryText(String text) { - return "/etc/clickhouse-server/config.xml"; + if (exit_strings.find(text) != exit_strings.end()) + return false; + CommandPtr command; + try + { + auto arguments = split(text, word_break_characters); + command = getCommandByName(arguments[0]); + arguments.erase(arguments.begin()); + command->execute(arguments, *client); + } + catch (DB::Exception & err) + { + int code = getCurrentExceptionCode(); + if (code == ErrorCodes::LOGICAL_ERROR) + { + throw std::move(err); + } + else if (code == ErrorCodes::BAD_ARGUMENTS) + { + std::cerr << err.message() << "\n" + << "\n"; + if (command.get()) + { + std::cerr << "COMMAND: " << command->command_name << "\n"; + std::cerr << command->options_description << "\n"; + } + } + else + { + std::cerr << err.message() << "\n"; + } + } + catch (std::exception & err) + { + std::cerr << err.what() << "\n"; + } + + return true; } -void DisksApp::addOptions( - ProgramOptionsDescription & options_description_, - boost::program_options::positional_options_description & positional_options_description -) +void DisksApp::runInteractiveReplxx() { - options_description_.add_options() - ("help,h", "Print common help message") - ("config-file,C", po::value(), "Set config file") - ("disk", po::value(), "Set disk name") - ("command_name", po::value(), "Name for command to do") - ("save-logs", "Save logs to a file") - ("log-level", po::value(), "Logging level") - ; + ReplxxLineReader lr( + suggest, + history_file, + /* multiline= */ false, + query_extenders, + query_delimiters, + word_break_characters.c_str(), + /* highlighter_= */ {}); + lr.enableBracketedPaste(); - positional_options_description.add("command_name", 1); + while (true) + { + String prompt = client->getCurrentDiskWithPath().getPrompt(); - supported_commands = {"list-disks", "list", "move", "remove", "link", "copy", "write", "read", "mkdir"}; -#ifdef CLICKHOUSE_CLOUD - supported_commands.insert("packed-io"); -#endif + auto input = lr.readLine(prompt, ":-] "); + if (input.empty()) + break; + + if (!processQueryText(input)) + break; + } +} + +void DisksApp::parseAndCheckOptions( + const std::vector & arguments, const ProgramOptionsDescription & options_description, CommandLineOptions & options) +{ + auto parser = po::command_line_parser(arguments).options(options_description).allow_unregistered(); + po::parsed_options parsed = parser.run(); + po::store(parsed, options); +} + +void DisksApp::addOptions() +{ + options_description.add_options()("help,h", "Print common help message")("config-file,C", po::value(), "Set config file")( + "disk", po::value(), "Set disk name")("save-logs", "Save logs to a file")( + "log-level", po::value(), "Logging level"); command_descriptions.emplace("list-disks", makeCommandListDisks()); + command_descriptions.emplace("copy", makeCommandCopy()); command_descriptions.emplace("list", makeCommandList()); + command_descriptions.emplace("cd", makeCommandChangeDirectory()); command_descriptions.emplace("move", makeCommandMove()); command_descriptions.emplace("remove", makeCommandRemove()); command_descriptions.emplace("link", makeCommandLink()); - command_descriptions.emplace("copy", makeCommandCopy()); command_descriptions.emplace("write", makeCommandWrite()); command_descriptions.emplace("read", makeCommandRead()); command_descriptions.emplace("mkdir", makeCommandMkDir()); + command_descriptions.emplace("switch-disk", makeCommandSwitchDisk()); #ifdef CLICKHOUSE_CLOUD - command_descriptions.emplace("packed-io", makeCommandPackedIO()); + // command_descriptions.emplace("packed-io", makeCommandPackedIO()); #endif } @@ -95,29 +273,66 @@ void DisksApp::processOptions() config().setString("log-level", options["log-level"].as()); } -DisksApp::~DisksApp() +void DisksApp::printHelpMessage(const ProgramOptionsDescription &) { - if (global_context) - global_context->shutdown(); + std::optional help_description + = createOptionsDescription("Help Message for clickhouse-disks", getTerminalWidth()); + + help_description->add(options_description); + + std::cout << "ClickHouse disk management tool\n"; + std::cout << "Usage: ./clickhouse-disks [OPTION]\n"; + std::cout << "clickhouse-disks\n\n"; + + for (const auto & [current_command, _] : command_descriptions) + { + std::cout << command_descriptions[current_command]->command_name; + bool was = false; + for (const auto & [alias_name, alias_command_name] : aliases) + { + if (alias_command_name == current_command) + { + if (was) + std::cout << ","; + else + std::cout << "("; + std::cout << alias_name; + was = true; + } + } + std::cout << (was ? ")" : "") << " \t" << command_descriptions[current_command]->description << "\n\n"; + } } -void DisksApp::init(std::vector & common_arguments) +void DisksApp::initializeHistoryFile() { - stopOptionsProcessing(); + String home_path; + const char * home_path_cstr = getenv("HOME"); // NOLINT(concurrency-mt-unsafe) + if (home_path_cstr) + home_path = home_path_cstr; + if (config().has("history-file")) + history_file = config().getString("history-file"); + else + history_file = home_path + "/.disks-file-history"; - ProgramOptionsDescription options_description{createOptionsDescription("clickhouse-disks", getTerminalWidth())}; + if (!history_file.empty() && !fs::exists(history_file)) + { + try + { + FS::createFile(history_file); + } + catch (const ErrnoException & e) + { + if (e.getErrno() != EEXIST) + throw; + } + } +} - po::positional_options_description positional_options_description; - - addOptions(options_description, positional_options_description); - - size_t command_pos = findCommandPos(common_arguments); - std::vector global_flags(command_pos); - command_arguments.resize(common_arguments.size() - command_pos); - copy(common_arguments.begin(), common_arguments.begin() + command_pos, global_flags.begin()); - copy(common_arguments.begin() + command_pos, common_arguments.end(), command_arguments.begin()); - - parseAndCheckOptions(options_description, positional_options_description, global_flags); +void DisksApp::init(const std::vector & common_arguments) +{ + addOptions(); + parseAndCheckOptions(common_arguments, options_description, options); po::notify(options); @@ -127,42 +342,18 @@ void DisksApp::init(std::vector & common_arguments) exit(0); // NOLINT(concurrency-mt-unsafe) } - if (!supported_commands.contains(command_name)) - { - std::cerr << "Unknown command name: " << command_name << "\n"; - printHelpMessage(options_description); - throw DB::Exception(DB::ErrorCodes::BAD_ARGUMENTS, "Bad Arguments"); - } - processOptions(); } -void DisksApp::parseAndCheckOptions( - ProgramOptionsDescription & options_description_, - boost::program_options::positional_options_description & positional_options_description, - std::vector & arguments) +String DisksApp::getDefaultConfigFileName() { - auto parser = po::command_line_parser(arguments) - .options(options_description_) - .positional(positional_options_description) - .allow_unregistered(); - - po::parsed_options parsed = parser.run(); - po::store(parsed, options); - - auto positional_arguments = po::collect_unrecognized(parsed.options, po::collect_unrecognized_mode::include_positional); - for (const auto & arg : positional_arguments) - { - if (command_descriptions.contains(arg)) - { - command_name = arg; - break; - } - } + return "/etc/clickhouse-server/config.xml"; } int DisksApp::main(const std::vector & /*args*/) { + std::vector keys; + config().keys(keys); if (config().has("config-file") || fs::exists(getDefaultConfigFileName())) { String config_path = config().getString("config-file", getDefaultConfigFileName()); @@ -176,6 +367,9 @@ int DisksApp::main(const std::vector & /*args*/) throw Exception(ErrorCodes::BAD_ARGUMENTS, "No config-file specified"); } + config().keys(keys); + initializeHistoryFile(); + if (config().has("save-logs")) { auto log_level = config().getString("log-level", "trace"); @@ -193,7 +387,7 @@ int DisksApp::main(const std::vector & /*args*/) registerDisks(/* global_skip_access_check= */ true); registerFormats(); - shared_context = Context::createShared(); + auto shared_context = Context::createShared(); global_context = Context::createGlobal(shared_context.get()); global_context->makeGlobalContext(); @@ -202,59 +396,37 @@ int DisksApp::main(const std::vector & /*args*/) String path = config().getString("path", DBMS_DEFAULT_PATH); global_context->setPath(path); - auto & command = command_descriptions[command_name]; + String main_disk = config().getString("disk", "default"); - auto command_options = command->getCommandOptions(); - std::vector args; - if (command_options) - { - auto parser = po::command_line_parser(command_arguments).options(*command_options).allow_unregistered(); - po::parsed_options parsed = parser.run(); - po::store(parsed, options); - po::notify(options); - - args = po::collect_unrecognized(parsed.options, po::collect_unrecognized_mode::include_positional); - command->processOptions(config(), options); - } - else - { - auto parser = po::command_line_parser(command_arguments).options({}).allow_unregistered(); - po::parsed_options parsed = parser.run(); - args = po::collect_unrecognized(parsed.options, po::collect_unrecognized_mode::include_positional); - } - - std::unordered_set disks - { - config().getString("disk", "default"), - config().getString("disk-from", config().getString("disk", "default")), - config().getString("disk-to", config().getString("disk", "default")), - }; - - auto validator = [&disks]( - const Poco::Util::AbstractConfiguration & config, - const std::string & disk_config_prefix, - const std::string & disk_name) - { - if (!disks.contains(disk_name)) - return false; - - const auto disk_type = config.getString(disk_config_prefix + ".type", "local"); - - if (disk_type == "cache") - throw Exception(ErrorCodes::BAD_ARGUMENTS, "Disk type 'cache' of disk {} is not supported by clickhouse-disks", disk_name); - - return true; - }; + auto validator = [](const Poco::Util::AbstractConfiguration &, const std::string &, const std::string &) { return true; }; constexpr auto config_prefix = "storage_configuration.disks"; auto disk_selector = std::make_shared(); disk_selector->initialize(config(), config_prefix, global_context, validator); - command->execute(args, disk_selector, config()); + std::vector>> disks_with_path; + + for (const auto & [_, disk_ptr] : disk_selector->getDisksMap()) + { + disks_with_path.emplace_back( + disk_ptr, (disk_ptr->getName() == "local") ? std::optional{fs::current_path().string()} : std::nullopt); + } + + + client = std::make_unique(std::move(disks_with_path), main_disk); + + suggest.setCompletionsCallback([&](const String & prefix, size_t /* prefix_length */) { return getCompletions(prefix); }); + + runInteractiveReplxx(); return Application::EXIT_OK; } +DisksApp::~DisksApp() +{ + if (global_context) + global_context->shutdown(); +} } int mainEntryClickHouseDisks(int argc, char ** argv) @@ -269,16 +441,16 @@ int mainEntryClickHouseDisks(int argc, char ** argv) catch (const DB::Exception & e) { std::cerr << DB::getExceptionMessage(e, false) << std::endl; - return 1; + return 0; } catch (const boost::program_options::error & e) { std::cerr << "Bad arguments: " << e.what() << std::endl; - return DB::ErrorCodes::BAD_ARGUMENTS; + return 0; } catch (...) { std::cerr << DB::getCurrentExceptionMessage(true) << std::endl; - return 1; + return 0; } } diff --git a/programs/disks/DisksApp.h b/programs/disks/DisksApp.h index 51bc3f58dc4..a0ce98b51d0 100644 --- a/programs/disks/DisksApp.h +++ b/programs/disks/DisksApp.h @@ -1,61 +1,86 @@ #pragma once +#include +#include +#include #include +#include "DisksClient.h" +#include "ICommand_fwd.h" #include +#include +#include #include -#include - namespace DB { -class ICommand; -using CommandPtr = std::unique_ptr; - -namespace po = boost::program_options; using ProgramOptionsDescription = boost::program_options::options_description; using CommandLineOptions = boost::program_options::variables_map; -class DisksApp : public Poco::Util::Application, public Loggers +class DisksApp : public Poco::Util::Application { public: - DisksApp() = default; - ~DisksApp() override; + void addOptions(); - void init(std::vector & common_arguments); - - int main(const std::vector & args) override; - -protected: - static String getDefaultConfigFileName(); - - void addOptions( - ProgramOptionsDescription & options_description, - boost::program_options::positional_options_description & positional_options_description); void processOptions(); - void printHelpMessage(ProgramOptionsDescription & command_option_description); + bool processQueryText(String text); - size_t findCommandPos(std::vector & common_arguments); + void init(const std::vector & common_arguments); + + int main(const std::vector & /*args*/) override; + + CommandPtr getCommandByName(String command) const; + + void initializeHistoryFile(); + + static void parseAndCheckOptions( + const std::vector & arguments, const ProgramOptionsDescription & options_description, CommandLineOptions & options); + + void printHelpMessage(const ProgramOptionsDescription &); + + std::vector getCompletions(const String & prefix) const; + + std::vector getEmptyCompletion(CommandPtr command_) const; + + ~DisksApp() override; private: - void parseAndCheckOptions( - ProgramOptionsDescription & options_description, - boost::program_options::positional_options_description & positional_options_description, - std::vector & arguments); + void runInteractiveReplxx(); -protected: + String getDefaultConfigFileName(); + + // Fields responsible for the REPL work + String history_file; + LineReader::Suggest suggest; + LineReader::Patterns query_extenders = {"\\"}; + LineReader::Patterns query_delimiters = {}; + String word_break_characters{" \t\v\f\a\b\r\n"}; + + // General commang line arguments parsing fields ContextMutablePtr global_context; - SharedContextHolder shared_context; - - String command_name; - std::vector command_arguments; - - std::unordered_set supported_commands; + ProgramOptionsDescription options_description; + CommandLineOptions options; std::unordered_map command_descriptions; - po::variables_map options; + const std::unordered_map aliases + = {{"cp", "copy"}, + {"mv", "move"}, + {"ls", "list"}, + {"list_disks", "list-disks"}, + {"ln", "link"}, + {"rm", "remove"}, + {"r", "read"}, + {"w", "write"}, + {"delete", "remove"}, + {"ls-disks", "list-disks"}, + {"ls_disks", "list-disks"}, + {"packed_io", "packed-io"}}; + + std::set multidisk_commands = {"copy", "packed-io", "switch-disk", "cd"}; + + std::unique_ptr client{}; }; } diff --git a/programs/disks/DisksClient.cpp b/programs/disks/DisksClient.cpp new file mode 100644 index 00000000000..3c258b5aa6e --- /dev/null +++ b/programs/disks/DisksClient.cpp @@ -0,0 +1,42 @@ +#include "DisksClient.h" +#include +#include +#include +#include +#include +#include +#include +#include +#include "ICommand.h" + +#include +#include + +#include + +#include +#include + +namespace DB +{ +std::vector split(const String & text, const String & delimiters) +{ + std::vector arguments; + auto prev = text.begin(); + auto pos = std::find_if(text.begin(), text.end(), [&](char x) { return delimiters.contains(x); }); + while (pos != text.end()) + { + if (pos > prev) + { + arguments.push_back({prev, pos}); + } + prev = ++pos; + pos = std::find_if(prev, text.end(), [&](char x) { return delimiters.contains(x); }); + } + if (pos > prev) + { + arguments.push_back({prev, text.end()}); + } + return arguments; + } +} diff --git a/programs/disks/DisksClient.h b/programs/disks/DisksClient.h new file mode 100644 index 00000000000..e3b8cf7c8a9 --- /dev/null +++ b/programs/disks/DisksClient.h @@ -0,0 +1,316 @@ +#pragma once + +#include <__tuple> +#include +#include +#include +#include +#include +#include +#include +#include "Disks/DiskSelector.h" +#include "Disks/IDisk.h" +#include "ICommand_fwd.h" +#include "IO/ReadHelpers.h" + +#include +#include +#include +#include "Common/Exception.h" + +// #include +namespace fs = std::filesystem; + +namespace DB +{ + + +std::vector split(const String & text, const String & delimiters); + +using ProgramOptionsDescription = boost::program_options::options_description; +using CommandLineOptions = boost::program_options::variables_map; + + +namespace ErrorCodes +{ +extern const int BAD_ARGUMENTS; +}; + +class DiskWithPath +{ +public: + explicit DiskWithPath(DiskPtr disk_, std::optional path_ = std::nullopt) + : disk(disk_) + , path( + [&]() + { + if (path_.has_value()) + { + if (!fs::path{path_.value()}.is_absolute()) + { + throw Exception(ErrorCodes::BAD_ARGUMENTS, "Initializing path {} is not absolute", path_.value()); + } + return path_.value(); + } + else + { + return String{"/"}; + } + }()) + { + if (!disk->isDirectory(normalizePathAndGetAsRelative(path))) + { + throw Exception(ErrorCodes::BAD_ARGUMENTS, "Initializing path {} at disk {} is not a directory", path, disk->getName()); + } + } + + String getPrompt() { return disk->getName() + ":" + path + "$ "; } + + String getAbsolutePath(const String & any_path) const { return normalizePath(fs::path(path) / any_path); } + + String getCurrentPath() const { return path; } + + bool isDirectory(const String & any_path) const { return disk->isDirectory(getRelativeFromRoot(any_path)); } + + std::vector listAllFilesByPath(const String & any_path) const + { + if (isDirectory(any_path)) + { + std::vector file_names; + disk->listFiles(getRelativeFromRoot(any_path), file_names); + return file_names; + } + else + { + return {}; + } + } + + std::vector getAllFilesByPattern(std::string pattern) const + { + auto [path_before, path_after] = [&]() -> std::pair + { + auto slash_pos = pattern.find_last_of('/'); + if (slash_pos >= pattern.size()) + { + return {"", pattern}; + } + else + { + return {pattern.substr(0, slash_pos + 1), pattern.substr(slash_pos + 1, pattern.size() - slash_pos - 1)}; + } + }(); + + if (!isDirectory(path_before)) + { + return {}; + } + else + { + std::vector file_names = listAllFilesByPath(path_before); + + std::vector answer; + + for (const auto & file_name : file_names) + { + if (file_name.starts_with(path_after)) + { + String file_pattern = path_before + file_name; + if (isDirectory(file_pattern)) + { + file_pattern = file_pattern + "/"; + } + answer.push_back(file_pattern); + } + } + return answer; + } + } + + DiskPtr getDisk() const { return disk; } + + void setPath(const String & any_path) + { + if (isDirectory(any_path)) + { + path = getAbsolutePath(any_path); + } + else + { + throw Exception(ErrorCodes::BAD_ARGUMENTS, "Path {} at disk {} is not a directory", any_path, disk->getName()); + } + } + + String getRelativeFromRoot(const String & any_path) const { return normalizePathAndGetAsRelative(getAbsolutePath(any_path)); } + +private: + static String validatePathAndGetAsRelative(const String & path) + { + String lexically_normal_path = fs::path(path).lexically_normal(); + if (lexically_normal_path.find("..") != std::string::npos) + throw DB::Exception(DB::ErrorCodes::BAD_ARGUMENTS, "Path {} is not normalized", path); + + /// If path is absolute we should keep it as relative inside disk, so disk will look like + /// an ordinary filesystem with root. + if (fs::path(lexically_normal_path).is_absolute()) + return lexically_normal_path.substr(1); + + return lexically_normal_path; + } + + static std::string normalizePathAndGetAsRelative(const std::string & messyPath) + { + std::filesystem::path path(messyPath); + std::filesystem::path canonical_path = std::filesystem::weakly_canonical(path); + std::string npath = canonical_path.make_preferred().string(); + return validatePathAndGetAsRelative(npath); + } + + static std::string normalizePath(const std::string & messyPath) + { + std::filesystem::path path(messyPath); + std::filesystem::path canonical_path = std::filesystem::weakly_canonical(path); + return canonical_path.make_preferred().string(); + } + + const DiskPtr disk; + String path; +}; + +class DisksClient +{ +public: + explicit DisksClient(std::vector>> && disks_with_paths, std::optional begin_disk) + { + if (disks_with_paths.empty()) + { + throw Exception(ErrorCodes::BAD_ARGUMENTS, "Initializing array of disks is empty"); + } + if (!begin_disk.has_value()) + { + begin_disk = disks_with_paths[0].first->getName(); + } + bool has_begin_disk = true; + for (auto & [disk, path] : disks_with_paths) + { + addDisk(disk, path); + if (disk->getName() == begin_disk.value()) + { + has_begin_disk = true; + } + } + if (!has_begin_disk) + { + throw Exception(ErrorCodes::BAD_ARGUMENTS, "There is no begin_disk '{}' in initializing array", begin_disk.value()); + } + current_disk = std::move(begin_disk.value()); + } + + const DiskWithPath & getDiskWithPath(const String & disk) const + { + try + { + return disks.at(disk); + } + catch (...) + { + throw Exception(ErrorCodes::BAD_ARGUMENTS, "The disk '{}' is unknown", disk); + } + } + + DiskWithPath & getDiskWithPath(const String & disk) + { + try + { + return disks.at(disk); + } + catch (...) + { + throw Exception(ErrorCodes::BAD_ARGUMENTS, "The disk '{}' is unknown", disk); + } + } + + const DiskWithPath & getCurrentDiskWithPath() const + { + try + { + return disks.at(current_disk); + } + catch (...) + { + throw Exception(ErrorCodes::LOGICAL_ERROR, "There is no current disk in client"); + } + } + + DiskWithPath & getCurrentDiskWithPath() + { + try + { + return disks.at(current_disk); + } + catch (...) + { + throw Exception(ErrorCodes::LOGICAL_ERROR, "There is no current disk in client"); + } + } + + DiskPtr getCurrentDisk() const { return getCurrentDiskWithPath().getDisk(); } + + DiskPtr getDisk(const String & disk) const { return getDiskWithPath(disk).getDisk(); } + + bool switchToDisk(const String & disk_, const std::optional & path_) + { + if (disks.contains(disk_)) + { + if (path_.has_value()) + { + disks.at(disk_).setPath(path_.value()); + } + current_disk = disk_; + return true; + } + else + { + throw Exception(ErrorCodes::BAD_ARGUMENTS, "The disk '{}' is unknown", disk_); + } + } + + std::vector getAllDiskNames() const + { + std::vector answer{}; + answer.reserve(disks.size()); + for (const auto & [disk_name, _] : disks) + { + answer.push_back(disk_name); + } + return answer; + } + + std::vector getAllFilesByPatternFromAllDisks(std::string pattern) const + { + std::vector answer{}; + for (const auto & [_, disk] : disks) + { + for (auto & word : disk.getAllFilesByPattern(pattern)) + { + answer.push_back(word); + } + } + return answer; + } + +private: + void addDisk(DiskPtr disk_, const std::optional & path_) + { + String disk_name = disk_->getName(); + if (disks.contains(disk_->getName())) + { + throw Exception(ErrorCodes::BAD_ARGUMENTS, "The disk '{}' already exists", disk_name); + } + disks.emplace(disk_name, DiskWithPath{disk_, path_}); + } + + String current_disk; + std::unordered_map disks; +}; +} diff --git a/programs/disks/ICommand.cpp b/programs/disks/ICommand.cpp index 86188fb6db1..41610f1086f 100644 --- a/programs/disks/ICommand.cpp +++ b/programs/disks/ICommand.cpp @@ -1,5 +1,6 @@ #include "ICommand.h" #include +#include "DisksClient.h" namespace DB @@ -7,46 +8,45 @@ namespace DB namespace ErrorCodes { - extern const int BAD_ARGUMENTS; +extern const int BAD_ARGUMENTS; } -void ICommand::printHelpMessage() const +CommandLineOptions ICommand::processCommandLineArguments(const Strings & commands) { - std::cout << "Command: " << command_name << '\n'; - std::cout << "Description: " << description << '\n'; - std::cout << "Usage: " << usage << '\n'; + CommandLineOptions options; + auto parser = po::command_line_parser(commands); + parser.options(options_description).positional(positional_options_description).allow_unregistered(); - if (command_option_description) + po::parsed_options parsed = parser.run(); + po::store(parsed, options); + + return options; +} + +void ICommand::execute(const Strings & commands, DisksClient & client) +{ + try { - auto options = *command_option_description; - if (!options.options().empty()) - std::cout << options << '\n'; + processCommandLineArguments(commands); + } + catch (std::exception & exc) + { + throw Exception(ErrorCodes::BAD_ARGUMENTS, "{}", exc.what()); + } + return executeImpl(processCommandLineArguments(commands), client); +} + +DiskWithPath & ICommand::getDiskWithPath(DisksClient & client, const CommandLineOptions & options, const String & name) +{ + auto disk_name = getValueFromCommandLineOptionsWithOptional(options, name); + if (disk_name.has_value()) + { + return client.getDiskWithPath(disk_name.value()); + } + else + { + return client.getCurrentDiskWithPath(); } } -void ICommand::addOptions(ProgramOptionsDescription & options_description) -{ - if (!command_option_description || command_option_description->options().empty()) - return; - - options_description.add(*command_option_description); -} - -String ICommand::validatePathAndGetAsRelative(const String & path) -{ - /// If path contain non-normalized symbols like . we will normalized them. If the resulting normalized path - /// still contain '..' it can be dangerous, disallow such paths. Also since clickhouse-disks - /// is not an interactive program (don't track you current path) it's OK to disallow .. paths. - String lexically_normal_path = fs::path(path).lexically_normal(); - if (lexically_normal_path.find("..") != std::string::npos) - throw DB::Exception(DB::ErrorCodes::BAD_ARGUMENTS, "Path {} is not normalized", path); - - /// If path is absolute we should keep it as relative inside disk, so disk will look like - /// an ordinary filesystem with root. - if (fs::path(lexically_normal_path).is_absolute()) - return lexically_normal_path.substr(1); - - return lexically_normal_path; -} - } diff --git a/programs/disks/ICommand.h b/programs/disks/ICommand.h index efe350fe87b..bf10841b636 100644 --- a/programs/disks/ICommand.h +++ b/programs/disks/ICommand.h @@ -1,66 +1,133 @@ #pragma once -#include +#include #include +#include #include -#include +#include #include +#include "Common/Exception.h" +#include -#include +#include +#include + +#include "DisksApp.h" + +#include "DisksClient.h" + +#include "ICommand_fwd.h" namespace DB { +// namespace po = boost::program_options; namespace po = boost::program_options; -using ProgramOptionsDescription = boost::program_options::options_description; -using CommandLineOptions = boost::program_options::variables_map; +using ProgramOptionsDescription = po::options_description; +using PositionalProgramOptionsDescription = po::positional_options_description; +using CommandLineOptions = po::variables_map; + +namespace ErrorCodes +{ +extern const int BAD_ARGUMENTS; +} class ICommand { public: - ICommand() = default; + explicit ICommand() = default; virtual ~ICommand() = default; - virtual void execute( - const std::vector & command_arguments, - std::shared_ptr & disk_selector, - Poco::Util::LayeredConfiguration & config) = 0; + void execute(const Strings & commands, DisksClient & client); - const std::optional & getCommandOptions() const { return command_option_description; } + virtual void executeImpl(const CommandLineOptions & options, DisksClient & client) = 0; - void addOptions(ProgramOptionsDescription & options_description); + CommandLineOptions processCommandLineArguments(const Strings & commands); - virtual void processOptions(Poco::Util::LayeredConfiguration & config, po::variables_map & options) const = 0; + void exit() { options_parsed = false; } protected: - void printHelpMessage() const; + template + static T getValueFromCommandLineOptions(const CommandLineOptions & options, const String & name) + { + try + { + return options[name].as(); + } + catch (...) + { + throw DB::Exception(ErrorCodes::BAD_ARGUMENTS, "Argument {} has wrong type and can't be parsed", name); + } + } + + template + static T getValueFromCommandLineOptionsThrow(const CommandLineOptions & options, const String & name) + { + if (options.count(name)) + { + return getValueFromCommandLineOptions(options, name); + } + else + { + throw DB::Exception(ErrorCodes::BAD_ARGUMENTS, "Mandatory argument {} is missing", name); + } + } + + template + static T getValueFromCommandLineOptionsWithDefault(const CommandLineOptions & options, const String & name, const T & default_value) + { + if (options.count(name)) + { + return getValueFromCommandLineOptions(options, name); + } + else + { + return default_value; + } + } + + template + static std::optional getValueFromCommandLineOptionsWithOptional(const CommandLineOptions & options, const String & name) + { + if (options.count(name)) + { + return std::optional{getValueFromCommandLineOptions(options, name)}; + } + else + { + return std::nullopt; + } + } + + DiskWithPath & getDiskWithPath(DisksClient & client, const CommandLineOptions & options, const String & name); - static String validatePathAndGetAsRelative(const String & path); public: String command_name; String description; + ProgramOptionsDescription options_description; protected: - std::optional command_option_description; - String usage; - po::positional_options_description positional_options_description; + PositionalProgramOptionsDescription positional_options_description; + +private: + bool options_parsed{}; }; -using CommandPtr = std::unique_ptr; - -} - DB::CommandPtr makeCommandCopy(); -DB::CommandPtr makeCommandLink(); -DB::CommandPtr makeCommandList(); DB::CommandPtr makeCommandListDisks(); +DB::CommandPtr makeCommandList(); +DB::CommandPtr makeCommandChangeDirectory(); +DB::CommandPtr makeCommandLink(); DB::CommandPtr makeCommandMove(); DB::CommandPtr makeCommandRead(); DB::CommandPtr makeCommandRemove(); DB::CommandPtr makeCommandWrite(); DB::CommandPtr makeCommandMkDir(); DB::CommandPtr makeCommandPackedIO(); +DB::CommandPtr makeCommandSwitchDisk(); + +} diff --git a/programs/disks/ICommand_fwd.h b/programs/disks/ICommand_fwd.h new file mode 100644 index 00000000000..f45b6c8d17c --- /dev/null +++ b/programs/disks/ICommand_fwd.h @@ -0,0 +1,10 @@ +#include +#include + + +namespace DB +{ +class ICommand; + +using CommandPtr = std::shared_ptr; +} diff --git a/src/Disks/DiskSelector.cpp b/src/Disks/DiskSelector.cpp index a9260a249dd..6f10f5f701e 100644 --- a/src/Disks/DiskSelector.cpp +++ b/src/Disks/DiskSelector.cpp @@ -7,7 +7,6 @@ #include #include -#include namespace DB { @@ -27,15 +26,22 @@ void DiskSelector::assertInitialized() const } -void DiskSelector::initialize(const Poco::Util::AbstractConfiguration & config, const String & config_prefix, ContextPtr context, DiskValidator disk_validator) +void DiskSelector::initialize( + const Poco::Util::AbstractConfiguration & config, const String & config_prefix, ContextPtr context, DiskValidator disk_validator) { Poco::Util::AbstractConfiguration::Keys keys; config.keys(config_prefix, keys); + std::cerr << "Config Prefix: " << config_prefix << std::endl; + for (auto & key : keys) + std::cerr << "Key inside disk selector initialize: " << key; + std::cerr << std::endl; auto & factory = DiskFactory::instance(); constexpr auto default_disk_name = "default"; bool has_default_disk = false; + constexpr auto local_disk_name = "local"; + bool has_local_disk = false; for (const auto & disk_name : keys) { if (!std::all_of(disk_name.begin(), disk_name.end(), isWordCharASCII)) @@ -44,6 +50,9 @@ void DiskSelector::initialize(const Poco::Util::AbstractConfiguration & config, if (disk_name == default_disk_name) has_default_disk = true; + if (disk_name == local_disk_name) + has_local_disk = true; + const auto disk_config_prefix = config_prefix + "." + disk_name; if (disk_validator && !disk_validator(config, disk_config_prefix, disk_name)) @@ -54,11 +63,12 @@ void DiskSelector::initialize(const Poco::Util::AbstractConfiguration & config, if (!has_default_disk) { disks.emplace( - default_disk_name, - std::make_shared( - default_disk_name, context->getPath(), 0, context, config, config_prefix)); + default_disk_name, std::make_shared(default_disk_name, context->getPath(), 0, context, config, config_prefix)); } + if (!has_local_disk) + disks.emplace(local_disk_name, std::make_shared(local_disk_name, "/", 0, context, config, config_prefix)); + is_initialized = true; } From bc21d0fb27bdfff61f7c29a2682d836e336394a5 Mon Sep 17 00:00:00 2001 From: divanik Date: Mon, 27 May 2024 11:49:59 +0000 Subject: [PATCH 017/417] Minor fix --- programs/disks/CommandList.cpp | 5 ++--- 1 file changed, 2 insertions(+), 3 deletions(-) diff --git a/programs/disks/CommandList.cpp b/programs/disks/CommandList.cpp index 26a576abc7d..16d249e299d 100644 --- a/programs/disks/CommandList.cpp +++ b/programs/disks/CommandList.cpp @@ -75,12 +75,11 @@ private: } std::cout << "\n"; - for (const auto & file_name : file_names) + for (const auto & file_name : selected_and_sorted_file_names) { auto path = absolute_path + "/" + file_name; if (disk.isDirectory(path)) - if (show_hidden || (!file_name.starts_with('.'))) - listRecursive(disk, path, show_hidden); + listRecursive(disk, path, show_hidden); } } }; From 90365ef0a6b7408a7ca62830dcb50bdf011a7584 Mon Sep 17 00:00:00 2001 From: divanik Date: Mon, 27 May 2024 11:57:20 +0000 Subject: [PATCH 018/417] Remove garbage logs --- src/Disks/DiskSelector.cpp | 4 ---- 1 file changed, 4 deletions(-) diff --git a/src/Disks/DiskSelector.cpp b/src/Disks/DiskSelector.cpp index 6f10f5f701e..77e2299ed65 100644 --- a/src/Disks/DiskSelector.cpp +++ b/src/Disks/DiskSelector.cpp @@ -31,10 +31,6 @@ void DiskSelector::initialize( { Poco::Util::AbstractConfiguration::Keys keys; config.keys(config_prefix, keys); - std::cerr << "Config Prefix: " << config_prefix << std::endl; - for (auto & key : keys) - std::cerr << "Key inside disk selector initialize: " << key; - std::cerr << std::endl; auto & factory = DiskFactory::instance(); From 846d46075d1c42666be7b090828cc7bca64f64c9 Mon Sep 17 00:00:00 2001 From: divanik Date: Mon, 27 May 2024 12:17:04 +0000 Subject: [PATCH 019/417] Remove comments --- programs/disks/CommandChangeDirectory.cpp | 7 ------- programs/disks/CommandCopy.cpp | 5 ----- programs/disks/CommandLink.cpp | 5 ----- programs/disks/CommandList.cpp | 5 ----- programs/disks/CommandListDisks.cpp | 5 ----- programs/disks/CommandMkDir.cpp | 5 ----- programs/disks/CommandMove.cpp | 5 ----- programs/disks/CommandRead.cpp | 6 ------ programs/disks/CommandRemove.cpp | 5 ----- programs/disks/CommandSwitchDisk.cpp | 7 ------- programs/disks/CommandWrite.cpp | 6 ------ programs/disks/DisksApp.cpp | 2 +- programs/disks/DisksApp.h | 6 ++++++ programs/disks/DisksClient.h | 2 +- 14 files changed, 8 insertions(+), 63 deletions(-) diff --git a/programs/disks/CommandChangeDirectory.cpp b/programs/disks/CommandChangeDirectory.cpp index 9932d918099..3baf69f8be0 100644 --- a/programs/disks/CommandChangeDirectory.cpp +++ b/programs/disks/CommandChangeDirectory.cpp @@ -8,11 +8,6 @@ namespace DB { -namespace ErrorCodes -{ -extern const int BAD_ARGUMENTS; -} - class CommandChangeDirectory final : public ICommand { public: @@ -28,9 +23,7 @@ public: void executeImpl(const CommandLineOptions & options, DisksClient & client) override { DiskWithPath & disk = getDiskWithPath(client, options, "disk"); - // std::cerr << "Disk name: " << disk.getDisk()->getName() << std::endl; String path = getValueFromCommandLineOptionsThrow(options, "path"); - // std::cerr << "Disk path: " << path << std::endl; disk.setPath(path); } }; diff --git a/programs/disks/CommandCopy.cpp b/programs/disks/CommandCopy.cpp index e853e054f97..ae749f7448a 100644 --- a/programs/disks/CommandCopy.cpp +++ b/programs/disks/CommandCopy.cpp @@ -6,11 +6,6 @@ namespace DB { -namespace ErrorCodes -{ -extern const int BAD_ARGUMENTS; -} - class CommandCopy final : public ICommand { public: diff --git a/programs/disks/CommandLink.cpp b/programs/disks/CommandLink.cpp index 8b467891d18..7e80faf9fc6 100644 --- a/programs/disks/CommandLink.cpp +++ b/programs/disks/CommandLink.cpp @@ -4,11 +4,6 @@ namespace DB { -namespace ErrorCodes -{ -extern const int BAD_ARGUMENTS; -} - class CommandLink final : public ICommand { public: diff --git a/programs/disks/CommandList.cpp b/programs/disks/CommandList.cpp index 16d249e299d..f91f0c6455c 100644 --- a/programs/disks/CommandList.cpp +++ b/programs/disks/CommandList.cpp @@ -7,11 +7,6 @@ namespace DB { -namespace ErrorCodes -{ -extern const int BAD_ARGUMENTS; -} - class CommandList final : public ICommand { public: diff --git a/programs/disks/CommandListDisks.cpp b/programs/disks/CommandListDisks.cpp index 16779b0fdae..9fb67fed5e0 100644 --- a/programs/disks/CommandListDisks.cpp +++ b/programs/disks/CommandListDisks.cpp @@ -7,11 +7,6 @@ namespace DB { -namespace ErrorCodes -{ -extern const int BAD_ARGUMENTS; -} - class CommandListDisks final : public ICommand { public: diff --git a/programs/disks/CommandMkDir.cpp b/programs/disks/CommandMkDir.cpp index 23312435d4e..895602adf72 100644 --- a/programs/disks/CommandMkDir.cpp +++ b/programs/disks/CommandMkDir.cpp @@ -6,11 +6,6 @@ namespace DB { -namespace ErrorCodes -{ -extern const int BAD_ARGUMENTS; -} - class CommandMkDir final : public ICommand { public: diff --git a/programs/disks/CommandMove.cpp b/programs/disks/CommandMove.cpp index 25620de448e..fb2fce2fa61 100644 --- a/programs/disks/CommandMove.cpp +++ b/programs/disks/CommandMove.cpp @@ -4,11 +4,6 @@ namespace DB { -namespace ErrorCodes -{ -extern const int BAD_ARGUMENTS; -} - class CommandMove final : public ICommand { public: diff --git a/programs/disks/CommandRead.cpp b/programs/disks/CommandRead.cpp index 82ff90b6e02..6963824b5cc 100644 --- a/programs/disks/CommandRead.cpp +++ b/programs/disks/CommandRead.cpp @@ -8,18 +8,12 @@ namespace DB { -namespace ErrorCodes -{ -extern const int BAD_ARGUMENTS; -} - class CommandRead final : public ICommand { public: CommandRead() { command_name = "read"; - // command_option_description.emplace(createOptionsDescription("Allowed options", getTerminalWidth())); description = "Read a file from `FROM_PATH` to `TO_PATH`"; options_description.add_options()( "path-from", po::value(), "file from which we are reading, defaults to `stdin` (mandatory, positional)")( diff --git a/programs/disks/CommandRemove.cpp b/programs/disks/CommandRemove.cpp index 0344a09d156..f332267c780 100644 --- a/programs/disks/CommandRemove.cpp +++ b/programs/disks/CommandRemove.cpp @@ -4,11 +4,6 @@ namespace DB { -namespace ErrorCodes -{ -extern const int BAD_ARGUMENTS; -} - class CommandRemove final : public ICommand { public: diff --git a/programs/disks/CommandSwitchDisk.cpp b/programs/disks/CommandSwitchDisk.cpp index 6c1fbaa0623..e59a1fc8e87 100644 --- a/programs/disks/CommandSwitchDisk.cpp +++ b/programs/disks/CommandSwitchDisk.cpp @@ -8,20 +8,13 @@ namespace DB { -namespace ErrorCodes -{ -extern const int BAD_ARGUMENTS; -} - class CommandSwitchDisk final : public ICommand { public: explicit CommandSwitchDisk() : ICommand() { command_name = "switch-disk"; - // options_description.emplace(createOptionsDescription("Allowed options", getTerminalWidth())); description = "Change disk"; - // options_description->add_options()("recursive", "recursively list all directories"); options_description.add_options()("disk", po::value(), "the disk to switch to (mandatory, positional)")( "path", po::value(), "the path to switch on the disk"); positional_options_description.add("disk", 1); diff --git a/programs/disks/CommandWrite.cpp b/programs/disks/CommandWrite.cpp index 42999572443..e8b3a0741ba 100644 --- a/programs/disks/CommandWrite.cpp +++ b/programs/disks/CommandWrite.cpp @@ -9,18 +9,12 @@ namespace DB { -namespace ErrorCodes -{ -extern const int BAD_ARGUMENTS; -} - class CommandWrite final : public ICommand { public: CommandWrite() { command_name = "write"; - // command_option_description.emplace(createOptionsDescription("Allowed options", getTerminalWidth())); description = "Write a file from `FROM_PATH` to `TO_PATH`"; options_description.add_options()("path-from", po::value(), "file from which we are reading, defaults to `stdin`")( "path-to", po::value(), "file to which we are writing (mandatory, positional)"); diff --git a/programs/disks/DisksApp.cpp b/programs/disks/DisksApp.cpp index 02e8b74b889..10eb3f986b9 100644 --- a/programs/disks/DisksApp.cpp +++ b/programs/disks/DisksApp.cpp @@ -257,7 +257,7 @@ void DisksApp::addOptions() command_descriptions.emplace("mkdir", makeCommandMkDir()); command_descriptions.emplace("switch-disk", makeCommandSwitchDisk()); #ifdef CLICKHOUSE_CLOUD - // command_descriptions.emplace("packed-io", makeCommandPackedIO()); + command_descriptions.emplace("packed-io", makeCommandPackedIO()); #endif } diff --git a/programs/disks/DisksApp.h b/programs/disks/DisksApp.h index a0ce98b51d0..7c9150cd1ce 100644 --- a/programs/disks/DisksApp.h +++ b/programs/disks/DisksApp.h @@ -15,6 +15,12 @@ namespace DB { +namespace ErrorCodes +{ +extern const int BAD_ARGUMENTS; +extern const int LOGICAL_ERROR; +}; + using ProgramOptionsDescription = boost::program_options::options_description; using CommandLineOptions = boost::program_options::variables_map; diff --git a/programs/disks/DisksClient.h b/programs/disks/DisksClient.h index e3b8cf7c8a9..89d5ecce666 100644 --- a/programs/disks/DisksClient.h +++ b/programs/disks/DisksClient.h @@ -24,7 +24,6 @@ namespace fs = std::filesystem; namespace DB { - std::vector split(const String & text, const String & delimiters); using ProgramOptionsDescription = boost::program_options::options_description; @@ -34,6 +33,7 @@ using CommandLineOptions = boost::program_options::variables_map; namespace ErrorCodes { extern const int BAD_ARGUMENTS; +extern const int LOGICAL_ERROR; }; class DiskWithPath From 4574ee75041ad945b276cf9cb5f3aa9f635e73a0 Mon Sep 17 00:00:00 2001 From: divanik Date: Mon, 27 May 2024 12:45:05 +0000 Subject: [PATCH 020/417] Correct style check --- programs/disks/CommandSwitchDisk.cpp | 5 +++++ programs/disks/DisksApp.cpp | 7 +++++++ programs/disks/DisksApp.h | 6 ------ programs/disks/ICommand_fwd.h | 4 ++-- 4 files changed, 14 insertions(+), 8 deletions(-) diff --git a/programs/disks/CommandSwitchDisk.cpp b/programs/disks/CommandSwitchDisk.cpp index e59a1fc8e87..0eb7ced7abf 100644 --- a/programs/disks/CommandSwitchDisk.cpp +++ b/programs/disks/CommandSwitchDisk.cpp @@ -8,6 +8,11 @@ namespace DB { +namespace ErrorCodes +{ +extern const int BAD_ARGUMENTS; +}; + class CommandSwitchDisk final : public ICommand { public: diff --git a/programs/disks/DisksApp.cpp b/programs/disks/DisksApp.cpp index 10eb3f986b9..dd3c60c7630 100644 --- a/programs/disks/DisksApp.cpp +++ b/programs/disks/DisksApp.cpp @@ -24,6 +24,13 @@ namespace DB { +namespace ErrorCodes +{ +extern const int BAD_ARGUMENTS; +extern const int LOGICAL_ERROR; +}; + + CommandPtr DisksApp::getCommandByName(String command) const { auto it = aliases.find(command); diff --git a/programs/disks/DisksApp.h b/programs/disks/DisksApp.h index 7c9150cd1ce..a0ce98b51d0 100644 --- a/programs/disks/DisksApp.h +++ b/programs/disks/DisksApp.h @@ -15,12 +15,6 @@ namespace DB { -namespace ErrorCodes -{ -extern const int BAD_ARGUMENTS; -extern const int LOGICAL_ERROR; -}; - using ProgramOptionsDescription = boost::program_options::options_description; using CommandLineOptions = boost::program_options::variables_map; diff --git a/programs/disks/ICommand_fwd.h b/programs/disks/ICommand_fwd.h index f45b6c8d17c..84310b4a18d 100644 --- a/programs/disks/ICommand_fwd.h +++ b/programs/disks/ICommand_fwd.h @@ -1,6 +1,6 @@ -#include -#include +#pragma once +#include namespace DB { From 4c2d8a1378f775b94eed7fdde47df8a03407288f Mon Sep 17 00:00:00 2001 From: divanik Date: Mon, 27 May 2024 14:26:20 +0000 Subject: [PATCH 021/417] Add highlighting --- programs/disks/CommandChangeDirectory.cpp | 1 - programs/disks/DisksApp.cpp | 6 ++++-- programs/disks/DisksClient.cpp | 4 ---- programs/disks/DisksClient.h | 6 ------ programs/disks/ICommand.cpp | 1 - programs/disks/ICommand.h | 6 +++--- 6 files changed, 7 insertions(+), 17 deletions(-) diff --git a/programs/disks/CommandChangeDirectory.cpp b/programs/disks/CommandChangeDirectory.cpp index 3baf69f8be0..ce79d43db30 100644 --- a/programs/disks/CommandChangeDirectory.cpp +++ b/programs/disks/CommandChangeDirectory.cpp @@ -1,5 +1,4 @@ #include -#include "Common/Exception.h" #include #include "DisksApp.h" #include "DisksClient.h" diff --git a/programs/disks/DisksApp.cpp b/programs/disks/DisksApp.cpp index dd3c60c7630..a990d85a9d1 100644 --- a/programs/disks/DisksApp.cpp +++ b/programs/disks/DisksApp.cpp @@ -227,9 +227,11 @@ void DisksApp::runInteractiveReplxx() while (true) { - String prompt = client->getCurrentDiskWithPath().getPrompt(); + DiskWithPath disk_with_path = client->getCurrentDiskWithPath(); + String prompt = "\x1b[1;34m" + disk_with_path.getDisk()->getName() + "\x1b[0m:" + "\x1b[1;31m" + disk_with_path.getCurrentPath() + + "\x1b[0m$ "; - auto input = lr.readLine(prompt, ":-] "); + auto input = lr.readLine(prompt, "\x1b[1;31m:-] \x1b[0m"); if (input.empty()) break; diff --git a/programs/disks/DisksClient.cpp b/programs/disks/DisksClient.cpp index 3c258b5aa6e..8e90f0a82c1 100644 --- a/programs/disks/DisksClient.cpp +++ b/programs/disks/DisksClient.cpp @@ -7,10 +7,6 @@ #include #include #include -#include "ICommand.h" - -#include -#include #include diff --git a/programs/disks/DisksClient.h b/programs/disks/DisksClient.h index 89d5ecce666..cc9b1015bad 100644 --- a/programs/disks/DisksClient.h +++ b/programs/disks/DisksClient.h @@ -1,6 +1,5 @@ #pragma once -#include <__tuple> #include #include #include @@ -8,10 +7,7 @@ #include #include #include -#include "Disks/DiskSelector.h" #include "Disks/IDisk.h" -#include "ICommand_fwd.h" -#include "IO/ReadHelpers.h" #include #include @@ -64,8 +60,6 @@ public: } } - String getPrompt() { return disk->getName() + ":" + path + "$ "; } - String getAbsolutePath(const String & any_path) const { return normalizePath(fs::path(path) / any_path); } String getCurrentPath() const { return path; } diff --git a/programs/disks/ICommand.cpp b/programs/disks/ICommand.cpp index 41610f1086f..7a70a61bf6c 100644 --- a/programs/disks/ICommand.cpp +++ b/programs/disks/ICommand.cpp @@ -1,5 +1,4 @@ #include "ICommand.h" -#include #include "DisksClient.h" diff --git a/programs/disks/ICommand.h b/programs/disks/ICommand.h index bf10841b636..480b42d4f7a 100644 --- a/programs/disks/ICommand.h +++ b/programs/disks/ICommand.h @@ -23,7 +23,6 @@ namespace DB { -// namespace po = boost::program_options; namespace po = boost::program_options; using ProgramOptionsDescription = po::options_description; using PositionalProgramOptionsDescription = po::positional_options_description; @@ -127,7 +126,8 @@ DB::CommandPtr makeCommandRead(); DB::CommandPtr makeCommandRemove(); DB::CommandPtr makeCommandWrite(); DB::CommandPtr makeCommandMkDir(); -DB::CommandPtr makeCommandPackedIO(); DB::CommandPtr makeCommandSwitchDisk(); - +#ifdef CLICKHOUSE_CLOUD +DB::CommandPtr makeCommandPackedIO(); +#endif } From a9000bd82173b2a57290ec75fee4bb80b51f9fd1 Mon Sep 17 00:00:00 2001 From: divanik Date: Tue, 28 May 2024 13:17:49 +0000 Subject: [PATCH 022/417] Fix bugs --- programs/disks/CMakeLists.txt | 16 ++++++++-------- programs/disks/CommandChangeDirectory.cpp | 2 +- programs/disks/CommandLink.cpp | 4 ++-- programs/disks/CommandMove.cpp | 4 ++-- programs/disks/CommandRead.cpp | 20 +++++++++----------- programs/disks/CommandSwitchDisk.cpp | 6 +----- programs/disks/CommandWrite.cpp | 5 +---- programs/disks/DisksClient.h | 3 +-- programs/disks/ICommand.h | 4 ++-- 9 files changed, 27 insertions(+), 37 deletions(-) diff --git a/programs/disks/CMakeLists.txt b/programs/disks/CMakeLists.txt index c5b30d61706..0f3cb601750 100644 --- a/programs/disks/CMakeLists.txt +++ b/programs/disks/CMakeLists.txt @@ -1,18 +1,18 @@ set (CLICKHOUSE_DISKS_SOURCES - ICommand.cpp - DisksClient.cpp DisksApp.cpp - CommandCopy.cpp - CommandListDisks.cpp - CommandList.cpp - CommandLink.cpp + DisksClient.cpp + ICommand.cpp CommandChangeDirectory.cpp + CommandCopy.cpp + CommandLink.cpp + CommandList.cpp + CommandListDisks.cpp CommandMkDir.cpp CommandMove.cpp CommandRead.cpp CommandRemove.cpp - CommandWrite.cpp - CommandSwitchDisk.cpp) + CommandSwitchDisk.cpp + CommandWrite.cpp) if (CLICKHOUSE_CLOUD) set (CLICKHOUSE_DISKS_SOURCES ${CLICKHOUSE_DISKS_SOURCES} CommandPackedIO.cpp) diff --git a/programs/disks/CommandChangeDirectory.cpp b/programs/disks/CommandChangeDirectory.cpp index ce79d43db30..5e6a08cd3fd 100644 --- a/programs/disks/CommandChangeDirectory.cpp +++ b/programs/disks/CommandChangeDirectory.cpp @@ -14,7 +14,7 @@ public: { command_name = "cd"; description = "Change directory"; - options_description.add_options()("path", po::value(), "the path of listing (mandatory, positional)")( + options_description.add_options()("path", po::value(), "the path we want to get to (mandatory, positional)")( "disk", po::value(), "A disk where the path is changed"); positional_options_description.add("path", 1); } diff --git a/programs/disks/CommandLink.cpp b/programs/disks/CommandLink.cpp index 7e80faf9fc6..74707160f67 100644 --- a/programs/disks/CommandLink.cpp +++ b/programs/disks/CommandLink.cpp @@ -12,8 +12,8 @@ public: command_name = "link"; description = "Create hardlink from `from_path` to `to_path`"; options_description.add_options()( - "path-to", po::value(), "the path from which a hard link will be created (mandatory, positional)")( - "path-from", po::value(), "the path where a hard link will be created (mandatory, positional)"); + "path-from", po::value(), "the path where a hard link will be created (mandatory, positional)")( + "path-to", po::value(), "the path from which a hard link will be created (mandatory, positional)"); positional_options_description.add("path-from", 1); positional_options_description.add("path-to", 1); } diff --git a/programs/disks/CommandMove.cpp b/programs/disks/CommandMove.cpp index fb2fce2fa61..23144df3d35 100644 --- a/programs/disks/CommandMove.cpp +++ b/programs/disks/CommandMove.cpp @@ -25,9 +25,9 @@ public: String path_to = disk.getRelativeFromRoot(getValueFromCommandLineOptionsThrow(options, "path-to")); if (disk.getDisk()->isFile(path_from)) - disk.getDisk()->moveFile(path_from, path_from); + disk.getDisk()->moveFile(path_from, path_to); else - disk.getDisk()->moveDirectory(path_from, path_from); + disk.getDisk()->moveDirectory(path_from, path_to); } }; diff --git a/programs/disks/CommandRead.cpp b/programs/disks/CommandRead.cpp index 6963824b5cc..5c7daa14bf4 100644 --- a/programs/disks/CommandRead.cpp +++ b/programs/disks/CommandRead.cpp @@ -2,6 +2,7 @@ #include #include #include +#include "Common/Exception.h" #include #include "ICommand.h" @@ -15,9 +16,8 @@ public: { command_name = "read"; description = "Read a file from `FROM_PATH` to `TO_PATH`"; - options_description.add_options()( - "path-from", po::value(), "file from which we are reading, defaults to `stdin` (mandatory, positional)")( - "path-to", po::value(), "file to which we are writing"); + options_description.add_options()("path-from", po::value(), "file from which we are reading (mandatory, positional)")( + "path-to", po::value(), "file to which we are writing, , defaults to `stdout`"); positional_options_description.add("path-from", 1); } @@ -25,26 +25,24 @@ public: { auto disk = client.getCurrentDiskWithPath(); String path_from = disk.getRelativeFromRoot(getValueFromCommandLineOptionsThrow(options, "path-from")); + std::cerr << path_from << std::endl; std::optional path_to = getValueFromCommandLineOptionsWithOptional(options, "path-to"); - if (path_to.has_value()) - { - path_to = std::optional{disk.getRelativeFromRoot(path_to.value())}; - } auto in = disk.getDisk()->readFile(path_from); + std::unique_ptr out = {}; if (path_to.has_value()) { String relative_path_to = disk.getRelativeFromRoot(path_to.value()); - - auto out = disk.getDisk()->writeFile(relative_path_to); + out = disk.getDisk()->writeFile(relative_path_to); copyData(*in, *out); - out->finalize(); } else { - std::unique_ptr out = std::make_unique(STDOUT_FILENO); + out = std::make_unique(STDOUT_FILENO); copyData(*in, *out); + out->write('\n'); } + out->finalize(); } }; diff --git a/programs/disks/CommandSwitchDisk.cpp b/programs/disks/CommandSwitchDisk.cpp index 0eb7ced7abf..285afe0b7ce 100644 --- a/programs/disks/CommandSwitchDisk.cpp +++ b/programs/disks/CommandSwitchDisk.cpp @@ -30,11 +30,7 @@ public: String disk = getValueFromCommandLineOptions(options, "disk"); std::optional path = getValueFromCommandLineOptionsWithOptional(options, "path"); - if (!client.switchToDisk(disk, path)) - { - throw Exception( - ErrorCodes::BAD_ARGUMENTS, "Unable to switch to disk: {}, path: {}", disk, path.has_value() ? path.value() : "NO PATH"); - } + client.switchToDisk(disk, path); } }; diff --git a/programs/disks/CommandWrite.cpp b/programs/disks/CommandWrite.cpp index e8b3a0741ba..433ebb3d5cf 100644 --- a/programs/disks/CommandWrite.cpp +++ b/programs/disks/CommandWrite.cpp @@ -21,15 +21,12 @@ public: positional_options_description.add("path-to", 1); } + void executeImpl(const CommandLineOptions & options, DisksClient & client) override { auto disk = client.getCurrentDiskWithPath(); std::optional path_from = getValueFromCommandLineOptionsWithOptional(options, "path-from"); - if (path_from.has_value()) - { - path_from = std::optional{disk.getRelativeFromRoot(path_from.value())}; - } String path_to = disk.getRelativeFromRoot(getValueFromCommandLineOptionsThrow(options, "path-to")); diff --git a/programs/disks/DisksClient.h b/programs/disks/DisksClient.h index cc9b1015bad..0fc20125e21 100644 --- a/programs/disks/DisksClient.h +++ b/programs/disks/DisksClient.h @@ -252,7 +252,7 @@ public: DiskPtr getDisk(const String & disk) const { return getDiskWithPath(disk).getDisk(); } - bool switchToDisk(const String & disk_, const std::optional & path_) + void switchToDisk(const String & disk_, const std::optional & path_) { if (disks.contains(disk_)) { @@ -261,7 +261,6 @@ public: disks.at(disk_).setPath(path_.value()); } current_disk = disk_; - return true; } else { diff --git a/programs/disks/ICommand.h b/programs/disks/ICommand.h index 480b42d4f7a..1e05aefd28b 100644 --- a/programs/disks/ICommand.h +++ b/programs/disks/ICommand.h @@ -58,7 +58,7 @@ protected: } catch (...) { - throw DB::Exception(ErrorCodes::BAD_ARGUMENTS, "Argument {} has wrong type and can't be parsed", name); + throw DB::Exception(ErrorCodes::BAD_ARGUMENTS, "Argument '{}' has wrong type and can't be parsed", name); } } @@ -71,7 +71,7 @@ protected: } else { - throw DB::Exception(ErrorCodes::BAD_ARGUMENTS, "Mandatory argument {} is missing", name); + throw DB::Exception(ErrorCodes::BAD_ARGUMENTS, "Mandatory argument '{}' is missing", name); } } From 856a0e35f25f8733334c4b5aa86fa2deb1da590a Mon Sep 17 00:00:00 2001 From: divanik Date: Tue, 28 May 2024 13:30:34 +0000 Subject: [PATCH 023/417] Remove redundant includes --- programs/disks/CommandRead.cpp | 1 - programs/disks/CommandSwitchDisk.cpp | 1 - 2 files changed, 2 deletions(-) diff --git a/programs/disks/CommandRead.cpp b/programs/disks/CommandRead.cpp index 5c7daa14bf4..ea05d25fb44 100644 --- a/programs/disks/CommandRead.cpp +++ b/programs/disks/CommandRead.cpp @@ -2,7 +2,6 @@ #include #include #include -#include "Common/Exception.h" #include #include "ICommand.h" diff --git a/programs/disks/CommandSwitchDisk.cpp b/programs/disks/CommandSwitchDisk.cpp index 285afe0b7ce..22d56673832 100644 --- a/programs/disks/CommandSwitchDisk.cpp +++ b/programs/disks/CommandSwitchDisk.cpp @@ -1,6 +1,5 @@ #include #include -#include "Common/Exception.h" #include #include "DisksApp.h" #include "ICommand.h" From 0a2d922d2324e4b3887ccb663f5611079f5dcd6a Mon Sep 17 00:00:00 2001 From: divanik Date: Wed, 29 May 2024 13:57:29 +0000 Subject: [PATCH 024/417] Fix help message printing --- programs/disks/CommandRemove.cpp | 3 +- programs/disks/CommandSwitchDisk.cpp | 7 +- programs/disks/DisksApp.cpp | 72 ++++++-- programs/disks/DisksApp.h | 5 +- programs/disks/DisksClient.cpp | 252 ++++++++++++++++++++++++++ programs/disks/DisksClient.h | 257 ++------------------------- 6 files changed, 332 insertions(+), 264 deletions(-) diff --git a/programs/disks/CommandRemove.cpp b/programs/disks/CommandRemove.cpp index f332267c780..b322fb2701f 100644 --- a/programs/disks/CommandRemove.cpp +++ b/programs/disks/CommandRemove.cpp @@ -10,8 +10,7 @@ public: CommandRemove() { command_name = "remove"; - description = "Remove file or directory with all children. Throws exception if file doesn't exists.\nPath should be in format './' " - "or './path' or 'path'"; + description = "Remove file or directory with all children. Throws exception if file doesn't exists"; options_description.add_options()("path", po::value(), "path from which we copy (mandatory, positional)"); positional_options_description.add("path", 1); } diff --git a/programs/disks/CommandSwitchDisk.cpp b/programs/disks/CommandSwitchDisk.cpp index 22d56673832..cbcf8e93bfc 100644 --- a/programs/disks/CommandSwitchDisk.cpp +++ b/programs/disks/CommandSwitchDisk.cpp @@ -7,18 +7,13 @@ namespace DB { -namespace ErrorCodes -{ -extern const int BAD_ARGUMENTS; -}; - class CommandSwitchDisk final : public ICommand { public: explicit CommandSwitchDisk() : ICommand() { command_name = "switch-disk"; - description = "Change disk"; + description = "Switch disk"; options_description.add_options()("disk", po::value(), "the disk to switch to (mandatory, positional)")( "path", po::value(), "the path to switch on the disk"); positional_options_description.add("disk", 1); diff --git a/programs/disks/DisksApp.cpp b/programs/disks/DisksApp.cpp index a990d85a9d1..3df6776f4ec 100644 --- a/programs/disks/DisksApp.cpp +++ b/programs/disks/DisksApp.cpp @@ -44,7 +44,7 @@ CommandPtr DisksApp::getCommandByName(String command) const } catch (...) { - throw Exception(ErrorCodes::BAD_ARGUMENTS, "The command {} is unknown", command); + throw Exception(ErrorCodes::BAD_ARGUMENTS, "The command `{}` is unknown", command); } } @@ -199,6 +199,10 @@ bool DisksApp::processQueryText(String text) std::cerr << "COMMAND: " << command->command_name << "\n"; std::cerr << command->options_description << "\n"; } + else + { + printAvailableCommandsHelpMessage(); + } } else { @@ -282,34 +286,70 @@ void DisksApp::processOptions() config().setString("log-level", options["log-level"].as()); } -void DisksApp::printHelpMessage(const ProgramOptionsDescription &) + +void DisksApp::printEntryHelpMessage() { - std::optional help_description - = createOptionsDescription("Help Message for clickhouse-disks", getTerminalWidth()); - - help_description->add(options_description); - std::cout << "ClickHouse disk management tool\n"; - std::cout << "Usage: ./clickhouse-disks [OPTION]\n"; - std::cout << "clickhouse-disks\n\n"; + std::cout << options_description << '\n'; +} +size_t DisksApp::getMagicConstant() +{ + size_t magic_constant = 0; for (const auto & [current_command, _] : command_descriptions) { - std::cout << command_descriptions[current_command]->command_name; + std::string command_string{}; + command_string += command_descriptions[current_command]->command_name; bool was = false; for (const auto & [alias_name, alias_command_name] : aliases) { if (alias_command_name == current_command) { if (was) - std::cout << ","; + command_string += ","; else - std::cout << "("; - std::cout << alias_name; + command_string += "("; + command_string += alias_name; was = true; } } - std::cout << (was ? ")" : "") << " \t" << command_descriptions[current_command]->description << "\n\n"; + command_string += (was ? ")" : ""); + + magic_constant = std::max(magic_constant, command_string.size()); + } + return magic_constant + 2; +} + +void DisksApp::printAvailableCommandsHelpMessage() +{ + size_t magic_constant = getMagicConstant(); + + std::cout << "\x1b[1;33mAvailable commands:\x1b[0m\n"; + for (const auto & [current_command, _] : command_descriptions) + { + std::string command_string{}; + command_string += command_descriptions[current_command]->command_name; + bool was = false; + for (const auto & [alias_name, alias_command_name] : aliases) + { + if (alias_command_name == current_command) + { + if (was) + command_string += ","; + else + command_string += "("; + command_string += alias_name; + was = true; + } + } + command_string += (was ? ")" : ""); + std::cout << "\x1b[1;32m" << command_string << "\x1b[0m"; + for (size_t i = command_string.size(); i < magic_constant; ++i) + { + std::cout << " "; + } + + std::cout << command_descriptions[current_command]->description << "\n"; } } @@ -347,7 +387,8 @@ void DisksApp::init(const std::vector & common_arguments) if (options.count("help")) { - printHelpMessage(options_description); + printEntryHelpMessage(); + printAvailableCommandsHelpMessage(); exit(0); // NOLINT(concurrency-mt-unsafe) } @@ -373,6 +414,7 @@ int DisksApp::main(const std::vector & /*args*/) } else { + printEntryHelpMessage(); throw Exception(ErrorCodes::BAD_ARGUMENTS, "No config-file specified"); } diff --git a/programs/disks/DisksApp.h b/programs/disks/DisksApp.h index a0ce98b51d0..7333e5804ae 100644 --- a/programs/disks/DisksApp.h +++ b/programs/disks/DisksApp.h @@ -38,12 +38,15 @@ public: static void parseAndCheckOptions( const std::vector & arguments, const ProgramOptionsDescription & options_description, CommandLineOptions & options); - void printHelpMessage(const ProgramOptionsDescription &); + void printEntryHelpMessage(); + void printAvailableCommandsHelpMessage(); std::vector getCompletions(const String & prefix) const; std::vector getEmptyCompletion(CommandPtr command_) const; + size_t getMagicConstant(); + ~DisksApp() override; private: diff --git a/programs/disks/DisksClient.cpp b/programs/disks/DisksClient.cpp index 8e90f0a82c1..509892a39a2 100644 --- a/programs/disks/DisksClient.cpp +++ b/programs/disks/DisksClient.cpp @@ -13,8 +13,15 @@ #include #include +namespace ErrorCodes +{ +extern const int BAD_ARGUMENTS; +extern const int LOGICAL_ERROR; +}; + namespace DB { + std::vector split(const String & text, const String & delimiters) { std::vector arguments; @@ -34,5 +41,250 @@ std::vector split(const String & text, const String & delimiters) arguments.push_back({prev, text.end()}); } return arguments; +} + +DiskWithPath::DiskWithPath(DiskPtr disk_, std::optional path_) + : disk(disk_) + , path( + [&]() + { + if (path_.has_value()) + { + if (!fs::path{path_.value()}.is_absolute()) + { + throw Exception(ErrorCodes::BAD_ARGUMENTS, "Initializing path {} is not absolute", path_.value()); + } + return path_.value(); + } + else + { + return String{"/"}; + } + }()) +{ + if (!disk->isDirectory(normalizePathAndGetAsRelative(path))) + { + throw Exception(ErrorCodes::BAD_ARGUMENTS, "Initializing path {} at disk {} is not a directory", path, disk->getName()); } } + +std::vector DiskWithPath::listAllFilesByPath(const String & any_path) const +{ + if (isDirectory(any_path)) + { + std::vector file_names; + disk->listFiles(getRelativeFromRoot(any_path), file_names); + return file_names; + } + else + { + return {}; + } +} + +std::vector DiskWithPath::getAllFilesByPattern(std::string pattern) const +{ + auto [path_before, path_after] = [&]() -> std::pair + { + auto slash_pos = pattern.find_last_of('/'); + if (slash_pos >= pattern.size()) + { + return {"", pattern}; + } + else + { + return {pattern.substr(0, slash_pos + 1), pattern.substr(slash_pos + 1, pattern.size() - slash_pos - 1)}; + } + }(); + + if (!isDirectory(path_before)) + { + return {}; + } + else + { + std::vector file_names = listAllFilesByPath(path_before); + + std::vector answer; + + for (const auto & file_name : file_names) + { + if (file_name.starts_with(path_after)) + { + String file_pattern = path_before + file_name; + if (isDirectory(file_pattern)) + { + file_pattern = file_pattern + "/"; + } + answer.push_back(file_pattern); + } + } + return answer; + } +}; + +void DiskWithPath::setPath(const String & any_path) +{ + if (isDirectory(any_path)) + { + path = getAbsolutePath(any_path); + } + else + { + throw Exception(ErrorCodes::BAD_ARGUMENTS, "Path {} at disk {} is not a directory", any_path, disk->getName()); + } +} + +String DiskWithPath::validatePathAndGetAsRelative(const String & path) +{ + String lexically_normal_path = fs::path(path).lexically_normal(); + if (lexically_normal_path.find("..") != std::string::npos) + throw DB::Exception(DB::ErrorCodes::BAD_ARGUMENTS, "Path {} is not normalized", path); + + /// If path is absolute we should keep it as relative inside disk, so disk will look like + /// an ordinary filesystem with root. + if (fs::path(lexically_normal_path).is_absolute()) + return lexically_normal_path.substr(1); + + return lexically_normal_path; +} + +std::string DiskWithPath::normalizePathAndGetAsRelative(const std::string & messyPath) +{ + std::filesystem::path path(messyPath); + std::filesystem::path canonical_path = std::filesystem::weakly_canonical(path); + std::string npath = canonical_path.make_preferred().string(); + return validatePathAndGetAsRelative(npath); +} + +std::string DiskWithPath::normalizePath(const std::string & messyPath) +{ + std::filesystem::path path(messyPath); + std::filesystem::path canonical_path = std::filesystem::weakly_canonical(path); + return canonical_path.make_preferred().string(); +} + +DisksClient::DisksClient(std::vector>> && disks_with_paths, std::optional begin_disk) +{ + if (disks_with_paths.empty()) + { + throw Exception(ErrorCodes::BAD_ARGUMENTS, "Initializing array of disks is empty"); + } + if (!begin_disk.has_value()) + { + begin_disk = disks_with_paths[0].first->getName(); + } + bool has_begin_disk = true; + for (auto & [disk, path] : disks_with_paths) + { + addDisk(disk, path); + if (disk->getName() == begin_disk.value()) + { + has_begin_disk = true; + } + } + if (!has_begin_disk) + { + throw Exception(ErrorCodes::BAD_ARGUMENTS, "There is no begin_disk '{}' in initializing array", begin_disk.value()); + } + current_disk = std::move(begin_disk.value()); +} + +const DiskWithPath & DisksClient::getDiskWithPath(const String & disk) const +{ + try + { + return disks.at(disk); + } + catch (...) + { + throw Exception(ErrorCodes::BAD_ARGUMENTS, "The disk '{}' is unknown", disk); + } +} + +DiskWithPath & DisksClient::getDiskWithPath(const String & disk) +{ + try + { + return disks.at(disk); + } + catch (...) + { + throw Exception(ErrorCodes::BAD_ARGUMENTS, "The disk '{}' is unknown", disk); + } +} + +const DiskWithPath & DisksClient::getCurrentDiskWithPath() const +{ + try + { + return disks.at(current_disk); + } + catch (...) + { + throw Exception(ErrorCodes::LOGICAL_ERROR, "There is no current disk in client"); + } +} + +DiskWithPath & DisksClient::getCurrentDiskWithPath() +{ + try + { + return disks.at(current_disk); + } + catch (...) + { + throw Exception(ErrorCodes::LOGICAL_ERROR, "There is no current disk in client"); + } +} + +void DisksClient::switchToDisk(const String & disk_, const std::optional & path_) +{ + if (disks.contains(disk_)) + { + if (path_.has_value()) + { + disks.at(disk_).setPath(path_.value()); + } + current_disk = disk_; + } + else + { + throw Exception(ErrorCodes::BAD_ARGUMENTS, "The disk '{}' is unknown", disk_); + } +} + +std::vector DisksClient::getAllDiskNames() const +{ + std::vector answer{}; + answer.reserve(disks.size()); + for (const auto & [disk_name, _] : disks) + { + answer.push_back(disk_name); + } + return answer; +} + +std::vector DisksClient::getAllFilesByPatternFromAllDisks(std::string pattern) const +{ + std::vector answer{}; + for (const auto & [_, disk] : disks) + { + for (auto & word : disk.getAllFilesByPattern(pattern)) + { + answer.push_back(word); + } + } + return answer; +} + +void DisksClient::addDisk(DiskPtr disk_, const std::optional & path_) +{ + String disk_name = disk_->getName(); + if (disks.contains(disk_->getName())) + { + throw Exception(ErrorCodes::BAD_ARGUMENTS, "The disk '{}' already exists", disk_name); + } + disks.emplace(disk_name, DiskWithPath{disk_, path_}); +} +} diff --git a/programs/disks/DisksClient.h b/programs/disks/DisksClient.h index 0fc20125e21..5c2de7aa529 100644 --- a/programs/disks/DisksClient.h +++ b/programs/disks/DisksClient.h @@ -12,9 +12,7 @@ #include #include #include -#include "Common/Exception.h" -// #include namespace fs = std::filesystem; namespace DB @@ -25,40 +23,10 @@ std::vector split(const String & text, const String & delimiters); using ProgramOptionsDescription = boost::program_options::options_description; using CommandLineOptions = boost::program_options::variables_map; - -namespace ErrorCodes -{ -extern const int BAD_ARGUMENTS; -extern const int LOGICAL_ERROR; -}; - class DiskWithPath { public: - explicit DiskWithPath(DiskPtr disk_, std::optional path_ = std::nullopt) - : disk(disk_) - , path( - [&]() - { - if (path_.has_value()) - { - if (!fs::path{path_.value()}.is_absolute()) - { - throw Exception(ErrorCodes::BAD_ARGUMENTS, "Initializing path {} is not absolute", path_.value()); - } - return path_.value(); - } - else - { - return String{"/"}; - } - }()) - { - if (!disk->isDirectory(normalizePathAndGetAsRelative(path))) - { - throw Exception(ErrorCodes::BAD_ARGUMENTS, "Initializing path {} at disk {} is not a directory", path, disk->getName()); - } - } + explicit DiskWithPath(DiskPtr disk_, std::optional path_ = std::nullopt); String getAbsolutePath(const String & any_path) const { return normalizePath(fs::path(path) / any_path); } @@ -66,106 +34,20 @@ public: bool isDirectory(const String & any_path) const { return disk->isDirectory(getRelativeFromRoot(any_path)); } - std::vector listAllFilesByPath(const String & any_path) const - { - if (isDirectory(any_path)) - { - std::vector file_names; - disk->listFiles(getRelativeFromRoot(any_path), file_names); - return file_names; - } - else - { - return {}; - } - } + std::vector listAllFilesByPath(const String & any_path) const; - std::vector getAllFilesByPattern(std::string pattern) const - { - auto [path_before, path_after] = [&]() -> std::pair - { - auto slash_pos = pattern.find_last_of('/'); - if (slash_pos >= pattern.size()) - { - return {"", pattern}; - } - else - { - return {pattern.substr(0, slash_pos + 1), pattern.substr(slash_pos + 1, pattern.size() - slash_pos - 1)}; - } - }(); - - if (!isDirectory(path_before)) - { - return {}; - } - else - { - std::vector file_names = listAllFilesByPath(path_before); - - std::vector answer; - - for (const auto & file_name : file_names) - { - if (file_name.starts_with(path_after)) - { - String file_pattern = path_before + file_name; - if (isDirectory(file_pattern)) - { - file_pattern = file_pattern + "/"; - } - answer.push_back(file_pattern); - } - } - return answer; - } - } + std::vector getAllFilesByPattern(std::string pattern) const; DiskPtr getDisk() const { return disk; } - void setPath(const String & any_path) - { - if (isDirectory(any_path)) - { - path = getAbsolutePath(any_path); - } - else - { - throw Exception(ErrorCodes::BAD_ARGUMENTS, "Path {} at disk {} is not a directory", any_path, disk->getName()); - } - } + void setPath(const String & any_path); String getRelativeFromRoot(const String & any_path) const { return normalizePathAndGetAsRelative(getAbsolutePath(any_path)); } private: - static String validatePathAndGetAsRelative(const String & path) - { - String lexically_normal_path = fs::path(path).lexically_normal(); - if (lexically_normal_path.find("..") != std::string::npos) - throw DB::Exception(DB::ErrorCodes::BAD_ARGUMENTS, "Path {} is not normalized", path); - - /// If path is absolute we should keep it as relative inside disk, so disk will look like - /// an ordinary filesystem with root. - if (fs::path(lexically_normal_path).is_absolute()) - return lexically_normal_path.substr(1); - - return lexically_normal_path; - } - - static std::string normalizePathAndGetAsRelative(const std::string & messyPath) - { - std::filesystem::path path(messyPath); - std::filesystem::path canonical_path = std::filesystem::weakly_canonical(path); - std::string npath = canonical_path.make_preferred().string(); - return validatePathAndGetAsRelative(npath); - } - - static std::string normalizePath(const std::string & messyPath) - { - std::filesystem::path path(messyPath); - std::filesystem::path canonical_path = std::filesystem::weakly_canonical(path); - return canonical_path.make_preferred().string(); - } + static String validatePathAndGetAsRelative(const String & path); + static std::string normalizePathAndGetAsRelative(const std::string & messyPath); + static std::string normalizePath(const std::string & messyPath); const DiskPtr disk; String path; @@ -174,134 +56,29 @@ private: class DisksClient { public: - explicit DisksClient(std::vector>> && disks_with_paths, std::optional begin_disk) - { - if (disks_with_paths.empty()) - { - throw Exception(ErrorCodes::BAD_ARGUMENTS, "Initializing array of disks is empty"); - } - if (!begin_disk.has_value()) - { - begin_disk = disks_with_paths[0].first->getName(); - } - bool has_begin_disk = true; - for (auto & [disk, path] : disks_with_paths) - { - addDisk(disk, path); - if (disk->getName() == begin_disk.value()) - { - has_begin_disk = true; - } - } - if (!has_begin_disk) - { - throw Exception(ErrorCodes::BAD_ARGUMENTS, "There is no begin_disk '{}' in initializing array", begin_disk.value()); - } - current_disk = std::move(begin_disk.value()); - } + explicit DisksClient(std::vector>> && disks_with_paths, std::optional begin_disk); - const DiskWithPath & getDiskWithPath(const String & disk) const - { - try - { - return disks.at(disk); - } - catch (...) - { - throw Exception(ErrorCodes::BAD_ARGUMENTS, "The disk '{}' is unknown", disk); - } - } + const DiskWithPath & getDiskWithPath(const String & disk) const; - DiskWithPath & getDiskWithPath(const String & disk) - { - try - { - return disks.at(disk); - } - catch (...) - { - throw Exception(ErrorCodes::BAD_ARGUMENTS, "The disk '{}' is unknown", disk); - } - } + DiskWithPath & getDiskWithPath(const String & disk); - const DiskWithPath & getCurrentDiskWithPath() const - { - try - { - return disks.at(current_disk); - } - catch (...) - { - throw Exception(ErrorCodes::LOGICAL_ERROR, "There is no current disk in client"); - } - } + const DiskWithPath & getCurrentDiskWithPath() const; - DiskWithPath & getCurrentDiskWithPath() - { - try - { - return disks.at(current_disk); - } - catch (...) - { - throw Exception(ErrorCodes::LOGICAL_ERROR, "There is no current disk in client"); - } - } + DiskWithPath & getCurrentDiskWithPath(); DiskPtr getCurrentDisk() const { return getCurrentDiskWithPath().getDisk(); } DiskPtr getDisk(const String & disk) const { return getDiskWithPath(disk).getDisk(); } - void switchToDisk(const String & disk_, const std::optional & path_) - { - if (disks.contains(disk_)) - { - if (path_.has_value()) - { - disks.at(disk_).setPath(path_.value()); - } - current_disk = disk_; - } - else - { - throw Exception(ErrorCodes::BAD_ARGUMENTS, "The disk '{}' is unknown", disk_); - } - } + void switchToDisk(const String & disk_, const std::optional & path_); - std::vector getAllDiskNames() const - { - std::vector answer{}; - answer.reserve(disks.size()); - for (const auto & [disk_name, _] : disks) - { - answer.push_back(disk_name); - } - return answer; - } + std::vector getAllDiskNames() const; + + std::vector getAllFilesByPatternFromAllDisks(std::string pattern) const; - std::vector getAllFilesByPatternFromAllDisks(std::string pattern) const - { - std::vector answer{}; - for (const auto & [_, disk] : disks) - { - for (auto & word : disk.getAllFilesByPattern(pattern)) - { - answer.push_back(word); - } - } - return answer; - } private: - void addDisk(DiskPtr disk_, const std::optional & path_) - { - String disk_name = disk_->getName(); - if (disks.contains(disk_->getName())) - { - throw Exception(ErrorCodes::BAD_ARGUMENTS, "The disk '{}' already exists", disk_name); - } - disks.emplace(disk_name, DiskWithPath{disk_, path_}); - } + void addDisk(DiskPtr disk_, const std::optional & path_); String current_disk; std::unordered_map disks; From 3133b757d797c32fb52be89e7299c5a17a6237c7 Mon Sep 17 00:00:00 2001 From: divanik Date: Wed, 29 May 2024 14:59:50 +0000 Subject: [PATCH 025/417] Add query for a non-interactive mode --- programs/disks/DisksApp.cpp | 13 +++++++++++-- programs/disks/DisksApp.h | 2 ++ 2 files changed, 13 insertions(+), 2 deletions(-) diff --git a/programs/disks/DisksApp.cpp b/programs/disks/DisksApp.cpp index 3df6776f4ec..bd4c5293c70 100644 --- a/programs/disks/DisksApp.cpp +++ b/programs/disks/DisksApp.cpp @@ -256,7 +256,7 @@ void DisksApp::addOptions() { options_description.add_options()("help,h", "Print common help message")("config-file,C", po::value(), "Set config file")( "disk", po::value(), "Set disk name")("save-logs", "Save logs to a file")( - "log-level", po::value(), "Logging level"); + "log-level", po::value(), "Logging level")("query,q", po::value(), "Query for a non-interactive mode"); command_descriptions.emplace("list-disks", makeCommandListDisks()); command_descriptions.emplace("copy", makeCommandCopy()); @@ -284,6 +284,8 @@ void DisksApp::processOptions() config().setBool("save-logs", true); if (options.count("log-level")) config().setString("log-level", options["log-level"].as()); + if (options.count("query")) + query = std::optional{options["query"].as()}; } @@ -468,7 +470,14 @@ int DisksApp::main(const std::vector & /*args*/) suggest.setCompletionsCallback([&](const String & prefix, size_t /* prefix_length */) { return getCompletions(prefix); }); - runInteractiveReplxx(); + if (!query.has_value()) + { + runInteractiveReplxx(); + } + else + { + processQueryText(query.value()); + } return Application::EXIT_OK; } diff --git a/programs/disks/DisksApp.h b/programs/disks/DisksApp.h index 7333e5804ae..fcb4b0b52f5 100644 --- a/programs/disks/DisksApp.h +++ b/programs/disks/DisksApp.h @@ -67,6 +67,8 @@ private: CommandLineOptions options; std::unordered_map command_descriptions; + std::optional query; + const std::unordered_map aliases = {{"cp", "copy"}, {"mv", "move"}, From 26fce1f59d668f2a080bf4d8d1e97bc99cf0b305 Mon Sep 17 00:00:00 2001 From: Anton Popov Date: Wed, 29 May 2024 22:43:50 +0000 Subject: [PATCH 026/417] use buffering before merging sorted --- src/Core/Settings.h | 1 + .../QueryPlan/BufferChunksTransform.cpp | 158 ++++++++++++++++++ .../QueryPlan/BufferChunksTransform.h | 48 ++++++ .../Optimizations/optimizeReadInOrder.cpp | 36 ++-- src/Processors/QueryPlan/SortingStep.cpp | 20 ++- src/Processors/QueryPlan/SortingStep.h | 4 +- 6 files changed, 251 insertions(+), 16 deletions(-) create mode 100644 src/Processors/QueryPlan/BufferChunksTransform.cpp create mode 100644 src/Processors/QueryPlan/BufferChunksTransform.h diff --git a/src/Core/Settings.h b/src/Core/Settings.h index dc61a049de8..c3091656884 100644 --- a/src/Core/Settings.h +++ b/src/Core/Settings.h @@ -542,6 +542,7 @@ class IColumn; M(Bool, optimize_read_in_order, true, "Enable ORDER BY optimization for reading data in corresponding order in MergeTree tables.", 0) \ M(Bool, optimize_read_in_window_order, true, "Enable ORDER BY optimization in window clause for reading data in corresponding order in MergeTree tables.", 0) \ M(Bool, optimize_aggregation_in_order, false, "Enable GROUP BY optimization for aggregating data in corresponding order in MergeTree tables.", 0) \ + M(UInt64, read_in_order_max_bytes_to_buffer, 128 * 1024 * 1024, "Max bytes to buffer before merging while reading in order of primary key. The higher value increases parallelism of query execution", 0) \ M(UInt64, aggregation_in_order_max_block_bytes, 50000000, "Maximal size of block in bytes accumulated during aggregation in order of primary key. Lower block size allows to parallelize more final merge stage of aggregation.", 0) \ M(UInt64, read_in_order_two_level_merge_threshold, 100, "Minimal number of parts to read to run preliminary merge step during multithread reading in order of primary key.", 0) \ M(Bool, low_cardinality_allow_in_native_format, true, "Use LowCardinality type in Native format. Otherwise, convert LowCardinality columns to ordinary for select query, and convert ordinary columns to required LowCardinality for insert query.", 0) \ diff --git a/src/Processors/QueryPlan/BufferChunksTransform.cpp b/src/Processors/QueryPlan/BufferChunksTransform.cpp new file mode 100644 index 00000000000..872f3090259 --- /dev/null +++ b/src/Processors/QueryPlan/BufferChunksTransform.cpp @@ -0,0 +1,158 @@ +#include +#include "Common/Logger.h" + +namespace DB +{ + +BufferChunksTransform::BufferChunksTransform(const Block & header_, size_t num_ports_, size_t max_bytes_to_buffer_, size_t limit_) + : IProcessor(InputPorts(num_ports_, header_), OutputPorts(num_ports_, header_)) + , max_bytes_to_buffer(max_bytes_to_buffer_) + , limit(limit_) + , chunks(num_ports_) + , num_processed_rows(num_ports_) +{ + for (auto & input : inputs) + input_ports.push_back({.port = &input, .is_finished = false}); + + for (auto & output : outputs) + output_ports.push_back({.port = &output, .is_finished = false}); +} + +IProcessor::Status BufferChunksTransform::prepare(const PortNumbers & updated_inputs, const PortNumbers & updated_outputs) +{ + if (!is_reading_started) + { + for (auto & input : inputs) + input.setNeeded(); + + is_reading_started = true; + } + + for (const auto & idx : updated_outputs) + { + auto & input = input_ports[idx]; + auto & output = output_ports[idx]; + + if (output.port->isFinished()) + { + if (!output.is_finished) + { + output.is_finished = true; + ++num_finished_outputs; + } + } + else if (output.port->canPush()) + { + available_outputs.push(idx); + } + else if (num_buffered_bytes >= max_bytes_to_buffer) + { + input.port->setNotNeeded(); + } + } + + for (const auto & idx : updated_inputs) + { + auto & input = input_ports[idx]; + + if (input.port->isFinished()) + { + if (!input.is_finished) + { + input.is_finished = true; + ++num_finished_inputs; + } + } + else if (input.port->hasData() && num_buffered_bytes < max_bytes_to_buffer) + { + auto chunk = pullChunk(idx); + num_buffered_bytes += chunk.bytes(); + chunks[idx].push(std::move(chunk)); + } + } + + std::queue next_available_outputs; + bool pushed_directly = false; + + while (!available_outputs.empty()) + { + UInt64 idx = available_outputs.front(); + available_outputs.pop(); + + auto & input = input_ports[idx]; + auto & output = output_ports[idx]; + chassert(output.port->canPush()); + + if (!chunks[idx].empty()) + { + auto & chunk = chunks[idx].front(); + num_buffered_bytes -= chunk.bytes(); + output.port->push(std::move(chunk)); + chunks[idx].pop(); + } + else if (input.port->hasData()) + { + /// Process chunk without buffering if possible. + auto chunk = pullChunk(idx); + output.port->push(std::move(chunk)); + pushed_directly = true; + } + else if (input.is_finished) + { + output.port->finish(); + output.is_finished = true; + ++num_finished_outputs; + } + else + { + input.port->setNeeded(); + next_available_outputs.push(idx); + } + } + + available_outputs = std::move(next_available_outputs); + + if (num_finished_outputs == outputs.size()) + { + for (auto & input : inputs) + input.close(); + + return Status::Finished; + } + + if (num_finished_inputs == inputs.size()) + { + if (num_buffered_bytes == 0) + { + for (auto & output : outputs) + output.finish(); + + return Status::Finished; + } + + return Status::PortFull; + } + + bool need_data = pushed_directly || num_buffered_bytes < max_bytes_to_buffer; + return need_data ? Status::NeedData : Status::PortFull; +} + +Chunk BufferChunksTransform::pullChunk(size_t input_idx) +{ + auto & input = input_ports[input_idx]; + input.port->setNeeded(); + + auto chunk = input.port->pull(); + num_processed_rows[input_idx] += chunk.getNumRows(); + + if (limit && num_processed_rows[input_idx] >= limit) + { + input.port->close(); + input.is_finished = true; + ++num_finished_inputs; + } + + return chunk; +} + +} diff --git a/src/Processors/QueryPlan/BufferChunksTransform.h b/src/Processors/QueryPlan/BufferChunksTransform.h new file mode 100644 index 00000000000..d741fb992c9 --- /dev/null +++ b/src/Processors/QueryPlan/BufferChunksTransform.h @@ -0,0 +1,48 @@ +#pragma once +#include +#include + +namespace DB +{ + +class BufferChunksTransform : public IProcessor +{ +public: + BufferChunksTransform(const Block & header_, size_t num_ports_, size_t max_bytes_to_buffer_, size_t limit_); + + String getName() const override { return "BufferChunks"; } + Status prepare(const PortNumbers & updated_inputs, const PortNumbers & updated_outputs) override; + +private: + Chunk pullChunk(size_t input_idx); + + size_t max_bytes_to_buffer; + size_t limit; + + struct InputPortWithStatus + { + InputPort * port; + bool is_finished; + }; + + struct OutputPortWithStatus + { + OutputPort * port; + bool is_finished; + }; + + std::vector> chunks; + std::vector num_processed_rows; + + std::vector input_ports; + std::vector output_ports; + std::queue available_outputs; + + bool is_reading_started = false; + + size_t num_finished_inputs = 0; + size_t num_finished_outputs = 0; + size_t num_buffered_bytes = 0; +}; + +} diff --git a/src/Processors/QueryPlan/Optimizations/optimizeReadInOrder.cpp b/src/Processors/QueryPlan/Optimizations/optimizeReadInOrder.cpp index bc1b3695d88..e87b6f7cf35 100644 --- a/src/Processors/QueryPlan/Optimizations/optimizeReadInOrder.cpp +++ b/src/Processors/QueryPlan/Optimizations/optimizeReadInOrder.cpp @@ -915,18 +915,30 @@ void optimizeReadInOrder(QueryPlan::Node & node, QueryPlan::Nodes & nodes) { auto & union_node = node.children.front(); + const SortDescription * best_sort_descr = nullptr; + StepStack best_steps_to_update; + bool use_buffering = false; + std::vector infos; - const SortDescription * max_sort_descr = nullptr; infos.reserve(node.children.size()); + for (auto * child : union_node->children) { infos.push_back(buildInputOrderInfo(*sorting, *child, steps_to_update)); - if (infos.back() && (!max_sort_descr || max_sort_descr->size() < infos.back()->sort_description_for_merging.size())) - max_sort_descr = &infos.back()->sort_description_for_merging; + if (infos.back()) + { + if (!best_sort_descr || best_sort_descr->size() < infos.back()->sort_description_for_merging.size()) + { + best_sort_descr = &infos.back()->sort_description_for_merging; + best_steps_to_update = steps_to_update; + } + + use_buffering |= infos.back()->limit == 0; + } } - if (!max_sort_descr || max_sort_descr->empty()) + if (!best_sort_descr || best_sort_descr->empty()) return; for (size_t i = 0; i < infos.size(); ++i) @@ -941,7 +953,7 @@ void optimizeReadInOrder(QueryPlan::Node & node, QueryPlan::Nodes & nodes) auto limit = sorting->getLimit(); /// If we have limit, it's better to sort up to full description and apply limit. /// We cannot sort up to partial read-in-order description with limit cause result set can be wrong. - const auto & descr = limit ? sorting->getSortDescription() : *max_sort_descr; + const auto & descr = limit ? sorting->getSortDescription() : *best_sort_descr; additional_sorting = std::make_unique( child->step->getOutputStream(), descr, @@ -949,12 +961,12 @@ void optimizeReadInOrder(QueryPlan::Node & node, QueryPlan::Nodes & nodes) sorting->getSettings(), false); } - else if (info->sort_description_for_merging.size() < max_sort_descr->size()) + else if (info->sort_description_for_merging.size() < best_sort_descr->size()) { additional_sorting = std::make_unique( child->step->getOutputStream(), info->sort_description_for_merging, - *max_sort_descr, + *best_sort_descr, sorting->getSettings().max_block_size, 0); /// TODO: support limit with ties } @@ -968,12 +980,14 @@ void optimizeReadInOrder(QueryPlan::Node & node, QueryPlan::Nodes & nodes) } } - sorting->convertToFinishSorting(*max_sort_descr); + sorting->convertToFinishSorting(*best_sort_descr, use_buffering); + updateStepsDataStreams(best_steps_to_update); } else if (auto order_info = buildInputOrderInfo(*sorting, *node.children.front(), steps_to_update)) { - sorting->convertToFinishSorting(order_info->sort_description_for_merging); - /// update data stream's sorting properties + /// Use buffering only if have filter or don't have limit. + bool use_buffering = order_info->limit == 0; + sorting->convertToFinishSorting(order_info->sort_description_for_merging, use_buffering); updateStepsDataStreams(steps_to_update); } } @@ -1087,7 +1101,7 @@ size_t tryReuseStorageOrderingForWindowFunctions(QueryPlan::Node * parent_node, bool can_read = read_from_merge_tree->requestReadingInOrder(order_info->used_prefix_of_sorting_key_size, order_info->direction, order_info->limit); if (!can_read) return 0; - sorting->convertToFinishSorting(order_info->sort_description_for_merging); + sorting->convertToFinishSorting(order_info->sort_description_for_merging, false); } return 0; diff --git a/src/Processors/QueryPlan/SortingStep.cpp b/src/Processors/QueryPlan/SortingStep.cpp index 8f40e523b42..b5b3443deca 100644 --- a/src/Processors/QueryPlan/SortingStep.cpp +++ b/src/Processors/QueryPlan/SortingStep.cpp @@ -1,5 +1,4 @@ #include -#include #include #include #include @@ -8,6 +7,7 @@ #include #include #include +#include #include #include @@ -38,6 +38,7 @@ SortingStep::Settings::Settings(const Context & context) tmp_data = context.getTempDataOnDisk(); min_free_disk_space = settings.min_free_disk_space_for_temporary_data; max_block_bytes = settings.prefer_external_sort_block_bytes; + read_in_order_max_bytes_to_buffer = settings.read_in_order_max_bytes_to_buffer; } SortingStep::Settings::Settings(size_t max_block_size_) @@ -153,10 +154,11 @@ void SortingStep::updateLimit(size_t limit_) } } -void SortingStep::convertToFinishSorting(SortDescription prefix_description_) +void SortingStep::convertToFinishSorting(SortDescription prefix_description_, bool use_buffering_) { type = Type::FinishSorting; prefix_description = std::move(prefix_description_); + use_buffering = use_buffering_; } void SortingStep::scatterByPartitionIfNeeded(QueryPipelineBuilder& pipeline) @@ -244,6 +246,17 @@ void SortingStep::mergingSorted(QueryPipelineBuilder & pipeline, const SortDescr /// If there are several streams, then we merge them into one if (pipeline.getNumStreams() > 1) { + if (use_buffering && sort_settings.read_in_order_max_bytes_to_buffer) + { + auto transform = std::make_shared( + pipeline.getHeader(), + pipeline.getNumStreams(), + sort_settings.read_in_order_max_bytes_to_buffer, + limit_); + + pipeline.addTransform(std::move(transform)); + } + auto transform = std::make_shared( pipeline.getHeader(), pipeline.getNumStreams(), @@ -373,9 +386,8 @@ void SortingStep::transformPipeline(QueryPipelineBuilder & pipeline, const Build mergingSorted(pipeline, prefix_description, (need_finish_sorting ? 0 : limit)); if (need_finish_sorting) - { finishSorting(pipeline, prefix_description, result_description, limit); - } + return; } diff --git a/src/Processors/QueryPlan/SortingStep.h b/src/Processors/QueryPlan/SortingStep.h index 49dcf9f3121..57658b6dafb 100644 --- a/src/Processors/QueryPlan/SortingStep.h +++ b/src/Processors/QueryPlan/SortingStep.h @@ -28,6 +28,7 @@ public: TemporaryDataOnDiskScopePtr tmp_data = nullptr; size_t min_free_disk_space = 0; size_t max_block_bytes = 0; + size_t read_in_order_max_bytes_to_buffer = 0; explicit Settings(const Context & context); explicit Settings(size_t max_block_size_); @@ -80,7 +81,7 @@ public: const SortDescription & getSortDescription() const { return result_description; } - void convertToFinishSorting(SortDescription prefix_description); + void convertToFinishSorting(SortDescription prefix_description, bool use_buffering_); Type getType() const { return type; } const Settings & getSettings() const { return sort_settings; } @@ -126,6 +127,7 @@ private: UInt64 limit; bool always_read_till_end = false; + bool use_buffering = false; Settings sort_settings; From e561f5cc7b30c41b7b44e0d7280cf343ac981758 Mon Sep 17 00:00:00 2001 From: Anton Popov Date: Thu, 30 May 2024 12:41:22 +0000 Subject: [PATCH 027/417] use buffering before merging sorted --- .../QueryPlan/BufferChunksTransform.cpp | 170 +++++------------- .../QueryPlan/BufferChunksTransform.h | 34 +--- .../Optimizations/optimizeReadInOrder.cpp | 21 +-- src/Processors/QueryPlan/SortingStep.cpp | 11 +- 4 files changed, 65 insertions(+), 171 deletions(-) diff --git a/src/Processors/QueryPlan/BufferChunksTransform.cpp b/src/Processors/QueryPlan/BufferChunksTransform.cpp index 872f3090259..ac24a0816ec 100644 --- a/src/Processors/QueryPlan/BufferChunksTransform.cpp +++ b/src/Processors/QueryPlan/BufferChunksTransform.cpp @@ -1,156 +1,74 @@ #include -#include "Common/Logger.h" namespace DB { -BufferChunksTransform::BufferChunksTransform(const Block & header_, size_t num_ports_, size_t max_bytes_to_buffer_, size_t limit_) - : IProcessor(InputPorts(num_ports_, header_), OutputPorts(num_ports_, header_)) +BufferChunksTransform::BufferChunksTransform(const Block & header_, size_t max_bytes_to_buffer_, size_t limit_) + : IProcessor({header_}, {header_}) + , input(inputs.front()) + , output(outputs.front()) , max_bytes_to_buffer(max_bytes_to_buffer_) , limit(limit_) - , chunks(num_ports_) - , num_processed_rows(num_ports_) { - for (auto & input : inputs) - input_ports.push_back({.port = &input, .is_finished = false}); - - for (auto & output : outputs) - output_ports.push_back({.port = &output, .is_finished = false}); } -IProcessor::Status BufferChunksTransform::prepare(const PortNumbers & updated_inputs, const PortNumbers & updated_outputs) +IProcessor::Status BufferChunksTransform::prepare() { - if (!is_reading_started) + if (output.isFinished()) { - for (auto & input : inputs) - input.setNeeded(); - - is_reading_started = true; - } - - for (const auto & idx : updated_outputs) - { - auto & input = input_ports[idx]; - auto & output = output_ports[idx]; - - if (output.port->isFinished()) - { - if (!output.is_finished) - { - output.is_finished = true; - ++num_finished_outputs; - } - } - else if (output.port->canPush()) - { - available_outputs.push(idx); - } - else if (num_buffered_bytes >= max_bytes_to_buffer) - { - input.port->setNotNeeded(); - } - } - - for (const auto & idx : updated_inputs) - { - auto & input = input_ports[idx]; - - if (input.port->isFinished()) - { - if (!input.is_finished) - { - input.is_finished = true; - ++num_finished_inputs; - } - } - else if (input.port->hasData() && num_buffered_bytes < max_bytes_to_buffer) - { - auto chunk = pullChunk(idx); - num_buffered_bytes += chunk.bytes(); - chunks[idx].push(std::move(chunk)); - } - } - - std::queue next_available_outputs; - bool pushed_directly = false; - - while (!available_outputs.empty()) - { - UInt64 idx = available_outputs.front(); - available_outputs.pop(); - - auto & input = input_ports[idx]; - auto & output = output_ports[idx]; - chassert(output.port->canPush()); - - if (!chunks[idx].empty()) - { - auto & chunk = chunks[idx].front(); - num_buffered_bytes -= chunk.bytes(); - output.port->push(std::move(chunk)); - chunks[idx].pop(); - } - else if (input.port->hasData()) - { - /// Process chunk without buffering if possible. - auto chunk = pullChunk(idx); - output.port->push(std::move(chunk)); - pushed_directly = true; - } - else if (input.is_finished) - { - output.port->finish(); - output.is_finished = true; - ++num_finished_outputs; - } - else - { - input.port->setNeeded(); - next_available_outputs.push(idx); - } - } - - available_outputs = std::move(next_available_outputs); - - if (num_finished_outputs == outputs.size()) - { - for (auto & input : inputs) - input.close(); - + chunks = {}; + input.close(); return Status::Finished; } - if (num_finished_inputs == inputs.size()) + if (output.canPush()) { - if (num_buffered_bytes == 0) - { - for (auto & output : outputs) - output.finish(); + input.setNeeded(); + if (!chunks.empty()) + { + auto chunk = std::move(chunks.front()); + chunks.pop(); + + num_buffered_bytes -= chunk.bytes(); + output.push(std::move(chunk)); + } + else if (input.hasData()) + { + auto chunk = pullChunk(); + output.push(std::move(chunk)); + } + else if (input.isFinished()) + { + output.finish(); return Status::Finished; } + } + if (input.hasData() && num_buffered_bytes < max_bytes_to_buffer) + { + auto chunk = pullChunk(); + num_buffered_bytes += chunk.bytes(); + chunks.push(std::move(chunk)); + } + + if (num_buffered_bytes >= max_bytes_to_buffer) + { + input.setNotNeeded(); return Status::PortFull; } - bool need_data = pushed_directly || num_buffered_bytes < max_bytes_to_buffer; - return need_data ? Status::NeedData : Status::PortFull; + input.setNeeded(); + return Status::NeedData; } -Chunk BufferChunksTransform::pullChunk(size_t input_idx) +Chunk BufferChunksTransform::pullChunk() { - auto & input = input_ports[input_idx]; - input.port->setNeeded(); + auto chunk = input.pull(); + num_processed_rows += chunk.getNumRows(); - auto chunk = input.port->pull(); - num_processed_rows[input_idx] += chunk.getNumRows(); - - if (limit && num_processed_rows[input_idx] >= limit) - { - input.port->close(); - input.is_finished = true; - ++num_finished_inputs; - } + if (limit && num_processed_rows >= limit) + input.close(); return chunk; } diff --git a/src/Processors/QueryPlan/BufferChunksTransform.h b/src/Processors/QueryPlan/BufferChunksTransform.h index d741fb992c9..c912f280a8a 100644 --- a/src/Processors/QueryPlan/BufferChunksTransform.h +++ b/src/Processors/QueryPlan/BufferChunksTransform.h @@ -8,41 +8,23 @@ namespace DB class BufferChunksTransform : public IProcessor { public: - BufferChunksTransform(const Block & header_, size_t num_ports_, size_t max_bytes_to_buffer_, size_t limit_); + BufferChunksTransform(const Block & header_, size_t max_bytes_to_buffer_, size_t limit_); + Status prepare() override; String getName() const override { return "BufferChunks"; } - Status prepare(const PortNumbers & updated_inputs, const PortNumbers & updated_outputs) override; private: - Chunk pullChunk(size_t input_idx); + Chunk pullChunk(); + + InputPort & input; + OutputPort & output; size_t max_bytes_to_buffer; size_t limit; - struct InputPortWithStatus - { - InputPort * port; - bool is_finished; - }; - - struct OutputPortWithStatus - { - OutputPort * port; - bool is_finished; - }; - - std::vector> chunks; - std::vector num_processed_rows; - - std::vector input_ports; - std::vector output_ports; - std::queue available_outputs; - - bool is_reading_started = false; - - size_t num_finished_inputs = 0; - size_t num_finished_outputs = 0; + std::queue chunks; size_t num_buffered_bytes = 0; + size_t num_processed_rows = 0; }; } diff --git a/src/Processors/QueryPlan/Optimizations/optimizeReadInOrder.cpp b/src/Processors/QueryPlan/Optimizations/optimizeReadInOrder.cpp index e87b6f7cf35..cb8023f603f 100644 --- a/src/Processors/QueryPlan/Optimizations/optimizeReadInOrder.cpp +++ b/src/Processors/QueryPlan/Optimizations/optimizeReadInOrder.cpp @@ -915,9 +915,8 @@ void optimizeReadInOrder(QueryPlan::Node & node, QueryPlan::Nodes & nodes) { auto & union_node = node.children.front(); - const SortDescription * best_sort_descr = nullptr; - StepStack best_steps_to_update; bool use_buffering = false; + const SortDescription * max_sort_descr = nullptr; std::vector infos; infos.reserve(node.children.size()); @@ -928,17 +927,14 @@ void optimizeReadInOrder(QueryPlan::Node & node, QueryPlan::Nodes & nodes) if (infos.back()) { - if (!best_sort_descr || best_sort_descr->size() < infos.back()->sort_description_for_merging.size()) - { - best_sort_descr = &infos.back()->sort_description_for_merging; - best_steps_to_update = steps_to_update; - } + if (!max_sort_descr || max_sort_descr->size() < infos.back()->sort_description_for_merging.size()) + max_sort_descr = &infos.back()->sort_description_for_merging; use_buffering |= infos.back()->limit == 0; } } - if (!best_sort_descr || best_sort_descr->empty()) + if (!max_sort_descr || max_sort_descr->empty()) return; for (size_t i = 0; i < infos.size(); ++i) @@ -953,7 +949,7 @@ void optimizeReadInOrder(QueryPlan::Node & node, QueryPlan::Nodes & nodes) auto limit = sorting->getLimit(); /// If we have limit, it's better to sort up to full description and apply limit. /// We cannot sort up to partial read-in-order description with limit cause result set can be wrong. - const auto & descr = limit ? sorting->getSortDescription() : *best_sort_descr; + const auto & descr = limit ? sorting->getSortDescription() : *max_sort_descr; additional_sorting = std::make_unique( child->step->getOutputStream(), descr, @@ -961,12 +957,12 @@ void optimizeReadInOrder(QueryPlan::Node & node, QueryPlan::Nodes & nodes) sorting->getSettings(), false); } - else if (info->sort_description_for_merging.size() < best_sort_descr->size()) + else if (info->sort_description_for_merging.size() < max_sort_descr->size()) { additional_sorting = std::make_unique( child->step->getOutputStream(), info->sort_description_for_merging, - *best_sort_descr, + *max_sort_descr, sorting->getSettings().max_block_size, 0); /// TODO: support limit with ties } @@ -980,8 +976,7 @@ void optimizeReadInOrder(QueryPlan::Node & node, QueryPlan::Nodes & nodes) } } - sorting->convertToFinishSorting(*best_sort_descr, use_buffering); - updateStepsDataStreams(best_steps_to_update); + sorting->convertToFinishSorting(*max_sort_descr, use_buffering); } else if (auto order_info = buildInputOrderInfo(*sorting, *node.children.front(), steps_to_update)) { diff --git a/src/Processors/QueryPlan/SortingStep.cpp b/src/Processors/QueryPlan/SortingStep.cpp index b5b3443deca..a853c908317 100644 --- a/src/Processors/QueryPlan/SortingStep.cpp +++ b/src/Processors/QueryPlan/SortingStep.cpp @@ -248,13 +248,12 @@ void SortingStep::mergingSorted(QueryPipelineBuilder & pipeline, const SortDescr { if (use_buffering && sort_settings.read_in_order_max_bytes_to_buffer) { - auto transform = std::make_shared( - pipeline.getHeader(), - pipeline.getNumStreams(), - sort_settings.read_in_order_max_bytes_to_buffer, - limit_); + size_t bytes_to_buffer = sort_settings.read_in_order_max_bytes_to_buffer / pipeline.getNumStreams(); - pipeline.addTransform(std::move(transform)); + pipeline.addSimpleTransform([&](const Block & header) + { + return std::make_shared(header, bytes_to_buffer, limit_); + }); } auto transform = std::make_shared( From 75c484a73b582308b5bb0f9f1fac338ccdfa2ee3 Mon Sep 17 00:00:00 2001 From: divanik Date: Thu, 30 May 2024 14:07:25 +0000 Subject: [PATCH 028/417] Corrected build and some integrations tests --- programs/disks/CommandChangeDirectory.cpp | 2 +- programs/disks/CommandList.cpp | 23 +++-- programs/disks/CommandSwitchDisk.cpp | 2 +- programs/disks/ICommand.cpp | 2 +- .../test_backup_restore_s3/test.py | 1 + tests/integration/test_disk_types/test.py | 5 +- tests/integration/test_disks_app_func/test.py | 84 +++++++++---------- .../test_endpoint_macro_substitution/test.py | 5 +- tests/integration/test_multiple_disks/test.py | 3 + 9 files changed, 68 insertions(+), 59 deletions(-) diff --git a/programs/disks/CommandChangeDirectory.cpp b/programs/disks/CommandChangeDirectory.cpp index 5e6a08cd3fd..71cdae904e5 100644 --- a/programs/disks/CommandChangeDirectory.cpp +++ b/programs/disks/CommandChangeDirectory.cpp @@ -13,7 +13,7 @@ public: explicit CommandChangeDirectory() : ICommand() { command_name = "cd"; - description = "Change directory"; + description = "Change directory (makes sense only in interactive mode)"; options_description.add_options()("path", po::value(), "the path we want to get to (mandatory, positional)")( "disk", po::value(), "A disk where the path is changed"); positional_options_description.add("path", 1); diff --git a/programs/disks/CommandList.cpp b/programs/disks/CommandList.cpp index f91f0c6455c..b2361532ceb 100644 --- a/programs/disks/CommandList.cpp +++ b/programs/disks/CommandList.cpp @@ -24,10 +24,10 @@ public: bool recursive = options.count("recursive"); bool show_hidden = options.count("all"); auto disk = client.getCurrentDiskWithPath(); - String path = getValueFromCommandLineOptionsWithDefault(options, "path", ""); + String path = getValueFromCommandLineOptionsWithDefault(options, "path", "."); if (recursive) - listRecursive(disk, disk.getAbsolutePath(path), show_hidden); + listRecursive(disk, path, show_hidden); else list(disk, path, show_hidden); } @@ -49,12 +49,13 @@ private: } } - static void listRecursive(const DiskWithPath & disk, const std::string & absolute_path, bool show_hidden) + static void listRecursive(const DiskWithPath & disk, const std::string & relative_path, bool show_hidden) { - std::vector file_names = disk.listAllFilesByPath(absolute_path); + // std::cerr << absolute_path << std::endl; + std::vector file_names = disk.listAllFilesByPath(relative_path); std::vector selected_and_sorted_file_names{}; - std::cout << absolute_path << ":\n"; + std::cout << relative_path << ":\n"; if (!file_names.empty()) { @@ -72,7 +73,17 @@ private: for (const auto & file_name : selected_and_sorted_file_names) { - auto path = absolute_path + "/" + file_name; + auto path = [&]() -> String + { + if (relative_path.ends_with("/")) + { + return relative_path + file_name; + } + else + { + return relative_path + "/" + file_name; + } + }(); if (disk.isDirectory(path)) listRecursive(disk, path, show_hidden); } diff --git a/programs/disks/CommandSwitchDisk.cpp b/programs/disks/CommandSwitchDisk.cpp index cbcf8e93bfc..9e5d443ebeb 100644 --- a/programs/disks/CommandSwitchDisk.cpp +++ b/programs/disks/CommandSwitchDisk.cpp @@ -13,7 +13,7 @@ public: explicit CommandSwitchDisk() : ICommand() { command_name = "switch-disk"; - description = "Switch disk"; + description = "Switch disk (makes sense only in interactive mode)"; options_description.add_options()("disk", po::value(), "the disk to switch to (mandatory, positional)")( "path", po::value(), "the path to switch on the disk"); positional_options_description.add("disk", 1); diff --git a/programs/disks/ICommand.cpp b/programs/disks/ICommand.cpp index 7a70a61bf6c..41fa281794e 100644 --- a/programs/disks/ICommand.cpp +++ b/programs/disks/ICommand.cpp @@ -32,7 +32,7 @@ void ICommand::execute(const Strings & commands, DisksClient & client) { throw Exception(ErrorCodes::BAD_ARGUMENTS, "{}", exc.what()); } - return executeImpl(processCommandLineArguments(commands), client); + executeImpl(processCommandLineArguments(commands), client); } DiskWithPath & ICommand::getDiskWithPath(DisksClient & client, const CommandLineOptions & options, const String & name) diff --git a/tests/integration/test_backup_restore_s3/test.py b/tests/integration/test_backup_restore_s3/test.py index 967ed6a221c..8a07041ced9 100644 --- a/tests/integration/test_backup_restore_s3/test.py +++ b/tests/integration/test_backup_restore_s3/test.py @@ -255,6 +255,7 @@ def check_system_tables(backup_query_id=None): ("disk_s3_other_bucket", "ObjectStorage", "S3", "Local"), ("disk_s3_plain", "ObjectStorage", "S3", "Plain"), ("disk_s3_restricted_user", "ObjectStorage", "S3", "Local"), + ("local", "Local", "None", "None"), ) assert len(expected_disks) == len(disks) for expected_disk in expected_disks: diff --git a/tests/integration/test_disk_types/test.py b/tests/integration/test_disk_types/test.py index 1cc5048eb69..609a3168e77 100644 --- a/tests/integration/test_disk_types/test.py +++ b/tests/integration/test_disk_types/test.py @@ -6,6 +6,7 @@ disk_types = { "default": "Local", "disk_s3": "S3", "disk_encrypted": "S3", + "local": "Local" } # do not test HDFS on ARM @@ -95,9 +96,9 @@ def test_select_by_type(cluster): if disk_type == "Local": assert ( node.query( - "SELECT name FROM system.disks WHERE type='" + disk_type + "'" + "SELECT name FROM system.disks WHERE type='" + disk_type + "' ORDER BY name" ) - == name + "\n" + == "default\nlocal\n" ) elif disk_type == "S3": assert ( diff --git a/tests/integration/test_disks_app_func/test.py b/tests/integration/test_disks_app_func/test.py index 97d5da787cd..dc2a538f64a 100644 --- a/tests/integration/test_disks_app_func/test.py +++ b/tests/integration/test_disks_app_func/test.py @@ -47,12 +47,12 @@ def test_disks_app_func_ld(started_cluster): source = cluster.instances["disks_app_test"] out = source.exec_in_container( - ["/usr/bin/clickhouse", "disks", "--save-logs", "list-disks"] + ["/usr/bin/clickhouse", "disks", "--save-logs", "--query", "list-disks"] ) - disks = out.split("\n") + disks = list(sorted(map(lambda x : x.split(':')[0], filter(lambda x : len(x) > 1, out.split("\n"))))) - assert disks[0] == "default" and disks[1] == "test1" and disks[2] == "test2" + assert disks[:4] == ["default", "local", "test1", "test2"] def test_disks_app_func_ls(started_cluster): @@ -61,7 +61,7 @@ def test_disks_app_func_ls(started_cluster): init_data(source) out = source.exec_in_container( - ["/usr/bin/clickhouse", "disks", "--save-logs", "--disk", "test1", "list", "."] + ["/usr/bin/clickhouse", "disks", "--save-logs", "--disk", "test1", "--query", "list ."] ) files = out.split("\n") @@ -75,9 +75,8 @@ def test_disks_app_func_ls(started_cluster): "--save-logs", "--disk", "test1", - "list", - ".", - "--recursive", + "--query", + "list . --recursive", ] ) @@ -102,8 +101,8 @@ def test_disks_app_func_cp(started_cluster): "--save-logs", "--disk", "test1", - "write", - "path1", + "--query", + "'write path1'", ] ), ] @@ -113,18 +112,13 @@ def test_disks_app_func_cp(started_cluster): [ "/usr/bin/clickhouse", "disks", - "copy", - "--disk-from", - "test1", - "--disk-to", - "test2", - ".", - ".", + "--query", + "copy --disk-from test1 --disk-to test2 . .", ] ) out = source.exec_in_container( - ["/usr/bin/clickhouse", "disks", "--save-logs", "--disk", "test2", "list", "."] + ["/usr/bin/clickhouse", "disks", "--save-logs", "--disk", "test2", "--query", "list ."] ) assert "path1" in out @@ -136,8 +130,8 @@ def test_disks_app_func_cp(started_cluster): "--save-logs", "--disk", "test2", - "remove", - "path1", + "--query", + "remove path1", ] ) @@ -148,21 +142,21 @@ def test_disks_app_func_cp(started_cluster): "--save-logs", "--disk", "test1", - "remove", - "path1", + "--query", + "remove path1", ] ) # alesapin: Why we need list one more time? # kssenii: it is an assertion that the file is indeed deleted out = source.exec_in_container( - ["/usr/bin/clickhouse", "disks", "--save-logs", "--disk", "test2", "list", "."] + ["/usr/bin/clickhouse", "disks", "--save-logs", "--disk", "test2", "--query", "list ."] ) assert "path1" not in out out = source.exec_in_container( - ["/usr/bin/clickhouse", "disks", "--save-logs", "--disk", "test1", "list", "."] + ["/usr/bin/clickhouse", "disks", "--save-logs", "--disk", "test1", "--query", "list ."] ) assert "path1" not in out @@ -177,14 +171,13 @@ def test_disks_app_func_ln(started_cluster): [ "/usr/bin/clickhouse", "disks", - "link", - "data/default/test_table", - "data/default/z_tester", + "--query", + "link data/default/test_table data/default/z_tester", ] ) out = source.exec_in_container( - ["/usr/bin/clickhouse", "disks", "--save-logs", "list", "data/default/"] + ["/usr/bin/clickhouse", "disks", "--save-logs", "--query", "list data/default/"] ) files = out.split("\n") @@ -209,15 +202,15 @@ def test_disks_app_func_rm(started_cluster): "--save-logs", "--disk", "test2", - "write", - "path3", + "--query", + "'write path3'", ] ), ] ) out = source.exec_in_container( - ["/usr/bin/clickhouse", "disks", "--save-logs", "--disk", "test2", "list", "."] + ["/usr/bin/clickhouse", "disks", "--save-logs", "--disk", "test2", "--query", "list ."] ) assert "path3" in out @@ -229,13 +222,13 @@ def test_disks_app_func_rm(started_cluster): "--save-logs", "--disk", "test2", - "remove", - "path3", + "--query", + "remove path3", ] ) out = source.exec_in_container( - ["/usr/bin/clickhouse", "disks", "--save-logs", "--disk", "test2", "list", "."] + ["/usr/bin/clickhouse", "disks", "--save-logs", "--disk", "test2", "--query", "list ."] ) assert "path3" not in out @@ -247,7 +240,8 @@ def test_disks_app_func_mv(started_cluster): init_data(source) out = source.exec_in_container( - ["/usr/bin/clickhouse", "disks", "--save-logs", "--disk", "test1", "list", "."] + ["/usr/bin/clickhouse", "disks", "--save-logs", "--disk", "test1", + "--query", "list ."] ) files = out.split("\n") @@ -260,14 +254,13 @@ def test_disks_app_func_mv(started_cluster): "disks", "--disk", "test1", - "move", - "store", - "old_store", + "--query", + "move store old_store", ] ) out = source.exec_in_container( - ["/usr/bin/clickhouse", "disks", "--save-logs", "--disk", "test1", "list", "."] + ["/usr/bin/clickhouse", "disks", "--save-logs", "--disk", "test1", "--query", "list ."] ) files = out.split("\n") @@ -290,8 +283,8 @@ def test_disks_app_func_read_write(started_cluster): "--save-logs", "--disk", "test1", - "write", - "5.txt", + "--query", + "'write 5.txt'", ] ), ] @@ -304,8 +297,8 @@ def test_disks_app_func_read_write(started_cluster): "--save-logs", "--disk", "test1", - "read", - "5.txt", + "--query", + "read 5.txt", ] ) @@ -319,7 +312,7 @@ def test_remote_disk_list(started_cluster): init_data_s3(source) out = source.exec_in_container( - ["/usr/bin/clickhouse", "disks", "--save-logs", "--disk", "test3", "list", "."] + ["/usr/bin/clickhouse", "disks", "--save-logs", "--disk", "test3", "--query", "list ."] ) files = out.split("\n") @@ -333,9 +326,8 @@ def test_remote_disk_list(started_cluster): "--save-logs", "--disk", "test3", - "list", - ".", - "--recursive", + "--query", + "list . --recursive" ] ) diff --git a/tests/integration/test_endpoint_macro_substitution/test.py b/tests/integration/test_endpoint_macro_substitution/test.py index 7dc282a980f..46d7aad4672 100644 --- a/tests/integration/test_endpoint_macro_substitution/test.py +++ b/tests/integration/test_endpoint_macro_substitution/test.py @@ -8,6 +8,7 @@ disk_types = { "disk_s3": "S3", "disk_hdfs": "HDFS", "disk_encrypted": "S3", + "local": "Local" } if is_arm(): @@ -74,9 +75,9 @@ def test_select_by_type(cluster): if disk_type == "Local": assert ( node.query( - "SELECT name FROM system.disks WHERE type='" + disk_type + "'" + "SELECT name FROM system.disks WHERE type='" + disk_type + "' ORDER BY name" ) - == name + "\n" + == "default\nlocal\n" ) elif disk_type == "S3": assert ( diff --git a/tests/integration/test_multiple_disks/test.py b/tests/integration/test_multiple_disks/test.py index fdd81284b2a..83318c5efb0 100644 --- a/tests/integration/test_multiple_disks/test.py +++ b/tests/integration/test_multiple_disks/test.py @@ -83,6 +83,9 @@ def test_system_tables(start_cluster): "path": "/external/", "keep_free_space": "0", }, + { + "name": "local", "path": "/", "keep_free_space": "0" + } ] click_disk_data = json.loads( From 71cf78a91148a34df06ef3d16c27c9ca78ce4683 Mon Sep 17 00:00:00 2001 From: Anton Popov Date: Thu, 30 May 2024 14:30:15 +0000 Subject: [PATCH 029/417] add setting to changes --- src/Core/SettingsChangesHistory.h | 1 + 1 file changed, 1 insertion(+) diff --git a/src/Core/SettingsChangesHistory.h b/src/Core/SettingsChangesHistory.h index 3a0f2ca1e27..ec5a50b44b8 100644 --- a/src/Core/SettingsChangesHistory.h +++ b/src/Core/SettingsChangesHistory.h @@ -93,6 +93,7 @@ static std::map sett {"hdfs_ignore_file_doesnt_exist", false, false, "Allow to return 0 rows when the requested files don't exist instead of throwing an exception in HDFS table engine"}, {"azure_ignore_file_doesnt_exist", false, false, "Allow to return 0 rows when the requested files don't exist instead of throwing an exception in AzureBlobStorage table engine"}, {"s3_ignore_file_doesnt_exist", false, false, "Allow to return 0 rows when the requested files don't exist instead of throwing an exception in S3 table engine"}, + {"read_in_order_max_bytes_to_buffer", 0, 128 * 1024 * 1024, "Max bytes to buffer before merging while reading in order of primary key"}, }}, {"24.5", {{"allow_deprecated_error_prone_window_functions", true, false, "Allow usage of deprecated error prone window functions (neighbor, runningAccumulate, runningDifferenceStartingWithFirstValue, runningDifference)"}, {"allow_experimental_join_condition", false, false, "Support join with inequal conditions which involve columns from both left and right table. e.g. t1.y < t2.y."}, From e2f9ecbf69e023da2dd7b187c257385f1a39ee3e Mon Sep 17 00:00:00 2001 From: Nikita Taranov Date: Fri, 24 May 2024 16:26:54 +0100 Subject: [PATCH 030/417] move HT cache to a separate file --- src/Interpreters/Aggregator.cpp | 135 +---------------- src/Interpreters/Aggregator.h | 30 +--- src/Interpreters/HashTablesStatistics.h | 157 ++++++++++++++++++++ src/Interpreters/InterpreterSelectQuery.cpp | 2 +- src/Planner/Planner.cpp | 2 +- 5 files changed, 164 insertions(+), 162 deletions(-) create mode 100644 src/Interpreters/HashTablesStatistics.h diff --git a/src/Interpreters/Aggregator.cpp b/src/Interpreters/Aggregator.cpp index 45b43ae2d3a..b7143731576 100644 --- a/src/Interpreters/Aggregator.cpp +++ b/src/Interpreters/Aggregator.cpp @@ -24,9 +24,7 @@ #include #include #include -#include #include -#include #include #include #include @@ -78,115 +76,6 @@ namespace ErrorCodes namespace { -/** Collects observed HashMap-s sizes to avoid redundant intermediate resizes. - */ -class HashTablesStatistics -{ -public: - struct Entry - { - size_t sum_of_sizes; // used to determine if it's better to convert aggregation to two-level from the beginning - size_t median_size; // roughly the size we're going to preallocate on each thread - }; - - using Cache = DB::CacheBase; - using CachePtr = std::shared_ptr; - using Params = DB::Aggregator::Params::StatsCollectingParams; - - /// Collection and use of the statistics should be enabled. - std::optional getSizeHint(const Params & params) - { - if (!params.isCollectionAndUseEnabled()) - throw DB::Exception(DB::ErrorCodes::LOGICAL_ERROR, "Collection and use of the statistics should be enabled."); - - std::lock_guard lock(mutex); - const auto cache = getHashTableStatsCache(params, lock); - if (const auto hint = cache->get(params.key)) - { - LOG_TRACE( - getLogger("Aggregator"), - "An entry for key={} found in cache: sum_of_sizes={}, median_size={}", - params.key, - hint->sum_of_sizes, - hint->median_size); - return *hint; - } - return std::nullopt; - } - - /// Collection and use of the statistics should be enabled. - void update(size_t sum_of_sizes, size_t median_size, const Params & params) - { - if (!params.isCollectionAndUseEnabled()) - throw DB::Exception(DB::ErrorCodes::LOGICAL_ERROR, "Collection and use of the statistics should be enabled."); - - std::lock_guard lock(mutex); - const auto cache = getHashTableStatsCache(params, lock); - const auto hint = cache->get(params.key); - // We'll maintain the maximum among all the observed values until the next prediction turns out to be too wrong. - if (!hint || sum_of_sizes < hint->sum_of_sizes / 2 || hint->sum_of_sizes < sum_of_sizes || median_size < hint->median_size / 2 - || hint->median_size < median_size) - { - LOG_TRACE( - getLogger("Aggregator"), - "Statistics updated for key={}: new sum_of_sizes={}, median_size={}", - params.key, - sum_of_sizes, - median_size); - cache->set(params.key, std::make_shared(Entry{.sum_of_sizes = sum_of_sizes, .median_size = median_size})); - } - } - - std::optional getCacheStats() const - { - std::lock_guard lock(mutex); - if (hash_table_stats) - { - size_t hits = 0, misses = 0; - hash_table_stats->getStats(hits, misses); - return DB::HashTablesCacheStatistics{.entries = hash_table_stats->count(), .hits = hits, .misses = misses}; - } - return std::nullopt; - } - - static size_t calculateCacheKey(const DB::ASTPtr & select_query) - { - if (!select_query) - throw DB::Exception(DB::ErrorCodes::LOGICAL_ERROR, "Query ptr cannot be null"); - - const auto & select = select_query->as(); - - // It may happen in some corner cases like `select 1 as num group by num`. - if (!select.tables()) - return 0; - - SipHash hash; - hash.update(select.tables()->getTreeHash(/*ignore_aliases=*/ true)); - if (const auto where = select.where()) - hash.update(where->getTreeHash(/*ignore_aliases=*/ true)); - if (const auto group_by = select.groupBy()) - hash.update(group_by->getTreeHash(/*ignore_aliases=*/ true)); - return hash.get64(); - } - -private: - CachePtr getHashTableStatsCache(const Params & params, const std::lock_guard &) - { - if (!hash_table_stats || hash_table_stats->maxSizeInBytes() != params.max_entries_for_hash_table_stats) - hash_table_stats = std::make_shared(params.max_entries_for_hash_table_stats); - return hash_table_stats; - } - - mutable std::mutex mutex; - CachePtr hash_table_stats; -}; - -HashTablesStatistics & getHashTablesStatistics() -{ - static HashTablesStatistics hash_tables_stats; - return hash_tables_stats; -} - bool worthConvertToTwoLevel( size_t group_by_two_level_threshold, size_t result_size, size_t group_by_two_level_threshold_bytes, auto result_size_bytes) { @@ -217,7 +106,7 @@ void initDataVariantsWithSizeHint( const auto & stats_collecting_params = params.stats_collecting_params; if (stats_collecting_params.isCollectionAndUseEnabled()) { - if (auto hint = getHashTablesStatistics().getSizeHint(stats_collecting_params)) + if (auto hint = DB::getHashTablesStatistics().getSizeHint(stats_collecting_params)) { const auto max_threads = params.group_by_two_level_threshold != 0 ? std::max(params.max_threads, 1ul) : 1; const auto lower_limit = hint->sum_of_sizes / max_threads; @@ -254,7 +143,7 @@ void initDataVariantsWithSizeHint( } /// Collection and use of the statistics should be enabled. -void updateStatistics(const DB::ManyAggregatedDataVariants & data_variants, const DB::Aggregator::Params::StatsCollectingParams & params) +void updateStatistics(const DB::ManyAggregatedDataVariants & data_variants, const DB::StatsCollectingParams & params) { if (!params.isCollectionAndUseEnabled()) throw DB::Exception(DB::ErrorCodes::LOGICAL_ERROR, "Collection and use of the statistics should be enabled."); @@ -265,7 +154,7 @@ void updateStatistics(const DB::ManyAggregatedDataVariants & data_variants, cons const auto median_size = sizes.begin() + sizes.size() / 2; // not precisely though... std::nth_element(sizes.begin(), median_size, sizes.end()); const auto sum_of_sizes = std::accumulate(sizes.begin(), sizes.end(), 0ull); - getHashTablesStatistics().update(sum_of_sizes, *median_size, params); + DB::getHashTablesStatistics().update(sum_of_sizes, *median_size, params); } DB::ColumnNumbers calculateKeysPositions(const DB::Block & header, const DB::Aggregator::Params & params) @@ -300,24 +189,6 @@ size_t getMinBytesForPrefetch() namespace DB { -std::optional getHashTablesCacheStatistics() -{ - return getHashTablesStatistics().getCacheStats(); -} - -Aggregator::Params::StatsCollectingParams::StatsCollectingParams() = default; - -Aggregator::Params::StatsCollectingParams::StatsCollectingParams( - const ASTPtr & select_query_, - bool collect_hash_table_stats_during_aggregation_, - size_t max_entries_for_hash_table_stats_, - size_t max_size_to_preallocate_for_aggregation_) - : key(collect_hash_table_stats_during_aggregation_ ? HashTablesStatistics::calculateCacheKey(select_query_) : 0) - , max_entries_for_hash_table_stats(max_entries_for_hash_table_stats_) - , max_size_to_preallocate_for_aggregation(max_size_to_preallocate_for_aggregation_) -{ -} - Block Aggregator::getHeader(bool final) const { return params.getHeader(header, final); diff --git a/src/Interpreters/Aggregator.h b/src/Interpreters/Aggregator.h index 406d28597cf..9c301d29a27 100644 --- a/src/Interpreters/Aggregator.h +++ b/src/Interpreters/Aggregator.h @@ -39,9 +39,10 @@ #include -#include #include #include +#include +#include namespace DB { @@ -128,24 +129,6 @@ public: const double min_hit_rate_to_use_consecutive_keys_optimization; - struct StatsCollectingParams - { - StatsCollectingParams(); - - StatsCollectingParams( - const ASTPtr & select_query_, - bool collect_hash_table_stats_during_aggregation_, - size_t max_entries_for_hash_table_stats_, - size_t max_size_to_preallocate_for_aggregation_); - - bool isCollectionAndUseEnabled() const { return key != 0; } - void disable() { key = 0; } - - UInt64 key = 0; - const size_t max_entries_for_hash_table_stats = 0; - const size_t max_size_to_preallocate_for_aggregation = 0; - }; - StatsCollectingParams stats_collecting_params; Params( @@ -685,13 +668,4 @@ APPLY_FOR_AGGREGATED_VARIANTS(M) #undef M - -struct HashTablesCacheStatistics -{ - size_t entries = 0; - size_t hits = 0; - size_t misses = 0; -}; - -std::optional getHashTablesCacheStatistics(); } diff --git a/src/Interpreters/HashTablesStatistics.h b/src/Interpreters/HashTablesStatistics.h new file mode 100644 index 00000000000..4af9cf68817 --- /dev/null +++ b/src/Interpreters/HashTablesStatistics.h @@ -0,0 +1,157 @@ +#pragma once + +#include +#include +#include +#include +#include + + +namespace DB +{ + +struct HashTablesCacheStatistics +{ + size_t entries = 0; + size_t hits = 0; + size_t misses = 0; +}; + +inline size_t calculateCacheKey(const DB::ASTPtr & select_query) +{ + if (!select_query) + throw DB::Exception(DB::ErrorCodes::LOGICAL_ERROR, "Query ptr cannot be null"); + + const auto & select = select_query->as(); + + // It may happen in some corner cases like `select 1 as num group by num`. + if (!select.tables()) + return 0; + + SipHash hash; + hash.update(select.tables()->getTreeHash(/*ignore_aliases=*/true)); + if (const auto where = select.where()) + hash.update(where->getTreeHash(/*ignore_aliases=*/true)); + if (const auto group_by = select.groupBy()) + hash.update(group_by->getTreeHash(/*ignore_aliases=*/true)); + return hash.get64(); +} + +struct StatsCollectingParams +{ + StatsCollectingParams() = default; + + StatsCollectingParams( + const ASTPtr & select_query_, + bool collect_hash_table_stats_during_aggregation_, + size_t max_entries_for_hash_table_stats_, + size_t max_size_to_preallocate_for_aggregation_) + : key(collect_hash_table_stats_during_aggregation_ ? calculateCacheKey(select_query_) : 0) + , max_entries_for_hash_table_stats(max_entries_for_hash_table_stats_) + , max_size_to_preallocate_for_aggregation(max_size_to_preallocate_for_aggregation_) + { + } + + bool isCollectionAndUseEnabled() const { return key != 0; } + void disable() { key = 0; } + + UInt64 key = 0; + const size_t max_entries_for_hash_table_stats = 0; + const size_t max_size_to_preallocate_for_aggregation = 0; +}; + +/** Collects observed HashMap-s sizes to avoid redundant intermediate resizes. + */ +class HashTablesStatistics +{ +public: + struct Entry + { + size_t sum_of_sizes; // used to determine if it's better to convert aggregation to two-level from the beginning + size_t median_size; // roughly the size we're going to preallocate on each thread + }; + + using Cache = DB::CacheBase; + using CachePtr = std::shared_ptr; + using Params = StatsCollectingParams; + + /// Collection and use of the statistics should be enabled. + std::optional getSizeHint(const Params & params) + { + if (!params.isCollectionAndUseEnabled()) + throw DB::Exception(DB::ErrorCodes::LOGICAL_ERROR, "Collection and use of the statistics should be enabled."); + + std::lock_guard lock(mutex); + const auto cache = getHashTableStatsCache(params, lock); + if (const auto hint = cache->get(params.key)) + { + LOG_TRACE( + getLogger("Aggregator"), + "An entry for key={} found in cache: sum_of_sizes={}, median_size={}", + params.key, + hint->sum_of_sizes, + hint->median_size); + return *hint; + } + return std::nullopt; + } + + /// Collection and use of the statistics should be enabled. + void update(size_t sum_of_sizes, size_t median_size, const Params & params) + { + if (!params.isCollectionAndUseEnabled()) + throw DB::Exception(DB::ErrorCodes::LOGICAL_ERROR, "Collection and use of the statistics should be enabled."); + + std::lock_guard lock(mutex); + const auto cache = getHashTableStatsCache(params, lock); + const auto hint = cache->get(params.key); + // We'll maintain the maximum among all the observed values until the next prediction turns out to be too wrong. + if (!hint || sum_of_sizes < hint->sum_of_sizes / 2 || hint->sum_of_sizes < sum_of_sizes || median_size < hint->median_size / 2 + || hint->median_size < median_size) + { + LOG_TRACE( + getLogger("Aggregator"), + "Statistics updated for key={}: new sum_of_sizes={}, median_size={}", + params.key, + sum_of_sizes, + median_size); + cache->set(params.key, std::make_shared(Entry{.sum_of_sizes = sum_of_sizes, .median_size = median_size})); + } + } + + std::optional getCacheStats() const + { + std::lock_guard lock(mutex); + if (hash_table_stats) + { + size_t hits = 0, misses = 0; + hash_table_stats->getStats(hits, misses); + return DB::HashTablesCacheStatistics{.entries = hash_table_stats->count(), .hits = hits, .misses = misses}; + } + return std::nullopt; + } + +private: + CachePtr getHashTableStatsCache(const Params & params, const std::lock_guard &) + { + if (!hash_table_stats || hash_table_stats->maxSizeInBytes() != params.max_entries_for_hash_table_stats) + hash_table_stats = std::make_shared(params.max_entries_for_hash_table_stats); + return hash_table_stats; + } + + mutable std::mutex mutex; + CachePtr hash_table_stats; +}; + +inline HashTablesStatistics & getHashTablesStatistics() +{ + static HashTablesStatistics hash_tables_stats; + return hash_tables_stats; +} + +inline std::optional getHashTablesCacheStatistics() +{ + return getHashTablesStatistics().getCacheStats(); +} + +} diff --git a/src/Interpreters/InterpreterSelectQuery.cpp b/src/Interpreters/InterpreterSelectQuery.cpp index ffe45d55643..ec9341cf9d5 100644 --- a/src/Interpreters/InterpreterSelectQuery.cpp +++ b/src/Interpreters/InterpreterSelectQuery.cpp @@ -2664,7 +2664,7 @@ static Aggregator::Params getAggregatorParams( size_t group_by_two_level_threshold, size_t group_by_two_level_threshold_bytes) { - const auto stats_collecting_params = Aggregator::Params::StatsCollectingParams( + const auto stats_collecting_params = StatsCollectingParams( query_ptr, settings.collect_hash_table_stats_during_aggregation, settings.max_entries_for_hash_table_stats, diff --git a/src/Planner/Planner.cpp b/src/Planner/Planner.cpp index b40e23a9553..8ed7004cd42 100644 --- a/src/Planner/Planner.cpp +++ b/src/Planner/Planner.cpp @@ -362,7 +362,7 @@ Aggregator::Params getAggregatorParams(const PlannerContextPtr & planner_context const auto & query_context = planner_context->getQueryContext(); const Settings & settings = query_context->getSettingsRef(); - const auto stats_collecting_params = Aggregator::Params::StatsCollectingParams( + const auto stats_collecting_params = StatsCollectingParams( select_query_info.query, settings.collect_hash_table_stats_during_aggregation, settings.max_entries_for_hash_table_stats, From 1a6f89dd02514eeeaf4cae43538d6a512d85220d Mon Sep 17 00:00:00 2001 From: Nikita Taranov Date: Sun, 26 May 2024 18:17:18 +0100 Subject: [PATCH 031/417] cache hash table sizes in ConcurrentHJ --- src/Interpreters/Aggregator.cpp | 67 ++++++++++----------- src/Interpreters/Aggregator.h | 1 + src/Interpreters/ConcurrentHashJoin.cpp | 47 ++++++++++++++- src/Interpreters/ConcurrentHashJoin.h | 6 ++ src/Interpreters/ExpressionAnalyzer.cpp | 3 +- src/Interpreters/HashTablesStatistics.h | 65 +++++++++++--------- src/Interpreters/InterpreterSelectQuery.cpp | 7 ++- src/Planner/Planner.cpp | 3 +- src/Planner/PlannerJoins.cpp | 11 +++- 9 files changed, 137 insertions(+), 73 deletions(-) diff --git a/src/Interpreters/Aggregator.cpp b/src/Interpreters/Aggregator.cpp index b7143731576..46c2d3c09f0 100644 --- a/src/Interpreters/Aggregator.cpp +++ b/src/Interpreters/Aggregator.cpp @@ -104,42 +104,22 @@ void initDataVariantsWithSizeHint( DB::AggregatedDataVariants & result, DB::AggregatedDataVariants::Type method_chosen, const DB::Aggregator::Params & params) { const auto & stats_collecting_params = params.stats_collecting_params; - if (stats_collecting_params.isCollectionAndUseEnabled()) + const auto max_threads = params.group_by_two_level_threshold != 0 ? std::max(params.max_threads, 1ul) : 1; + if (auto hint = findSizeHint(stats_collecting_params, max_threads)) { - if (auto hint = DB::getHashTablesStatistics().getSizeHint(stats_collecting_params)) - { - const auto max_threads = params.group_by_two_level_threshold != 0 ? std::max(params.max_threads, 1ul) : 1; - const auto lower_limit = hint->sum_of_sizes / max_threads; - const auto upper_limit = stats_collecting_params.max_size_to_preallocate_for_aggregation / max_threads; - if (hint->median_size > upper_limit) - { - /// Since we cannot afford to preallocate as much as we want, we will likely need to do resize anyway. - /// But we will also work with the big (i.e. not so cache friendly) HT from the beginning which may result in a slight slowdown. - /// So let's just do nothing. - LOG_TRACE( - getLogger("Aggregator"), - "No space were preallocated in hash tables because 'max_size_to_preallocate_for_aggregation' has too small value: {}, " - "should be at least {}", - stats_collecting_params.max_size_to_preallocate_for_aggregation, - hint->median_size * max_threads); - } - /// https://github.com/ClickHouse/ClickHouse/issues/44402#issuecomment-1359920703 - else if ((max_threads > 1 && hint->sum_of_sizes > 100'000) || hint->sum_of_sizes > 500'000) - { - const auto adjusted = std::max(lower_limit, hint->median_size); - if (worthConvertToTwoLevel( - params.group_by_two_level_threshold, - hint->sum_of_sizes, - /*group_by_two_level_threshold_bytes*/ 0, - /*result_size_bytes*/ 0)) - method_chosen = convertToTwoLevelTypeIfPossible(method_chosen); - result.init(method_chosen, adjusted); - ProfileEvents::increment(ProfileEvents::AggregationHashTablesInitializedAsTwoLevel, result.isTwoLevel()); - return; - } - } + if (worthConvertToTwoLevel( + params.group_by_two_level_threshold, + hint->sum_of_sizes, + /*group_by_two_level_threshold_bytes*/ 0, + /*result_size_bytes*/ 0)) + method_chosen = convertToTwoLevelTypeIfPossible(method_chosen); + result.init(method_chosen, hint->median_size); + ProfileEvents::increment(ProfileEvents::AggregationHashTablesInitializedAsTwoLevel, result.isTwoLevel()); + } + else + { + result.init(method_chosen); } - result.init(method_chosen); } /// Collection and use of the statistics should be enabled. @@ -3357,4 +3337,23 @@ void Aggregator::destroyAllAggregateStates(AggregatedDataVariants & result) cons throw Exception(ErrorCodes::UNKNOWN_AGGREGATED_DATA_VARIANT, "Unknown aggregated data variant."); } +UInt64 calculateCacheKey(const DB::ASTPtr & select_query) +{ + if (!select_query) + throw DB::Exception(DB::ErrorCodes::LOGICAL_ERROR, "Query ptr cannot be null"); + + const auto & select = select_query->as(); + + // It may happen in some corner cases like `select 1 as num group by num`. + if (!select.tables()) + return 0; + + SipHash hash; + hash.update(select.tables()->getTreeHash(/*ignore_aliases=*/true)); + if (const auto where = select.where()) + hash.update(where->getTreeHash(/*ignore_aliases=*/true)); + if (const auto group_by = select.groupBy()) + hash.update(group_by->getTreeHash(/*ignore_aliases=*/true)); + return hash.get64(); +} } diff --git a/src/Interpreters/Aggregator.h b/src/Interpreters/Aggregator.h index 9c301d29a27..f4f1e9a1df3 100644 --- a/src/Interpreters/Aggregator.h +++ b/src/Interpreters/Aggregator.h @@ -657,6 +657,7 @@ private: Arena * arena); }; +UInt64 calculateCacheKey(const DB::ASTPtr & select_query); /** Get the aggregation variant by its type. */ template Method & getDataVariant(AggregatedDataVariants & variants); diff --git a/src/Interpreters/ConcurrentHashJoin.cpp b/src/Interpreters/ConcurrentHashJoin.cpp index 53987694e46..85e67898569 100644 --- a/src/Interpreters/ConcurrentHashJoin.cpp +++ b/src/Interpreters/ConcurrentHashJoin.cpp @@ -29,6 +29,25 @@ extern const Metric ConcurrentHashJoinPoolThreadsActive; extern const Metric ConcurrentHashJoinPoolThreadsScheduled; } +namespace +{ + +void updateStatistics(const auto & hash_joins, const DB::StatsCollectingParams & params) +{ + if (!params.isCollectionAndUseEnabled()) + throw DB::Exception(DB::ErrorCodes::LOGICAL_ERROR, "Collection and use of the statistics should be enabled."); + + std::vector sizes(hash_joins.size()); + for (size_t i = 0; i < hash_joins.size(); ++i) + sizes[i] = hash_joins[i]->data->getTotalRowCount(); + const auto median_size = sizes.begin() + sizes.size() / 2; // not precisely though... + std::nth_element(sizes.begin(), median_size, sizes.end()); + if (auto sum_of_sizes = std::accumulate(sizes.begin(), sizes.end(), 0ull)) + DB::getHashTablesStatistics().update(sum_of_sizes, *median_size, params); +} + +} + namespace DB { @@ -46,7 +65,12 @@ static UInt32 toPowerOfTwo(UInt32 x) } ConcurrentHashJoin::ConcurrentHashJoin( - ContextPtr context_, std::shared_ptr table_join_, size_t slots_, const Block & right_sample_block, bool any_take_last_row_) + ContextPtr context_, + std::shared_ptr table_join_, + size_t slots_, + const Block & right_sample_block, + const StatsCollectingParams & stats_collecting_params_, + bool any_take_last_row_) : context(context_) , table_join(table_join_) , slots(toPowerOfTwo(std::min(static_cast(slots_), 256))) @@ -55,6 +79,7 @@ ConcurrentHashJoin::ConcurrentHashJoin( CurrentMetrics::ConcurrentHashJoinPoolThreadsActive, CurrentMetrics::ConcurrentHashJoinPoolThreadsScheduled, slots)) + , stats_collecting_params(stats_collecting_params_) { hash_joins.resize(slots); @@ -74,9 +99,13 @@ ConcurrentHashJoin::ConcurrentHashJoin( CurrentThread::attachToGroupIfDetached(thread_group); setThreadName("ConcurrentJoin"); + size_t reserve_size = 0; + if (auto hint = findSizeHint(stats_collecting_params, slots)) + reserve_size = hint->median_size; + auto inner_hash_join = std::make_shared(); inner_hash_join->data = std::make_unique( - table_join_, right_sample_block, any_take_last_row_, 0, fmt::format("concurrent{}", idx)); + table_join_, right_sample_block, any_take_last_row_, reserve_size, fmt::format("concurrent{}", idx)); /// Non zero `max_joined_block_rows` allows to process block partially and return not processed part. /// TODO: It's not handled properly in ConcurrentHashJoin case, so we set it to 0 to disable this feature. inner_hash_join->data->setMaxJoinedBlockRows(0); @@ -97,6 +126,8 @@ ConcurrentHashJoin::~ConcurrentHashJoin() { try { + updateStatistics(hash_joins, stats_collecting_params); + for (size_t i = 0; i < slots; ++i) { // Hash tables destruction may be very time-consuming. @@ -300,4 +331,16 @@ Blocks ConcurrentHashJoin::dispatchBlock(const Strings & key_columns_names, cons return result; } +UInt64 calculateCacheKey(std::shared_ptr & table_join, const QueryTreeNodePtr & right_table_expression) +{ + IQueryTreeNode::HashState hash; + hash.update(right_table_expression->getTreeHash()); + chassert(table_join->oneDisjunct()); + for (const auto & name : table_join->getClauses().at(0).key_names_right) + hash.update(name); + LOG_DEBUG(&Poco::Logger::get("debug"), "table_join->getClauses()={}", fmt::join(table_join->getClauses()[0].key_names_right, ", ")); + LOG_DEBUG(&Poco::Logger::get("debug"), "right_table_expression->dumpTree()={}", right_table_expression->dumpTree()); + LOG_DEBUG(&Poco::Logger::get("debug"), "hash.get64()={}", hash.get64()); + return hash.get64(); +} } diff --git a/src/Interpreters/ConcurrentHashJoin.h b/src/Interpreters/ConcurrentHashJoin.h index c797ff27ece..9f01117df3a 100644 --- a/src/Interpreters/ConcurrentHashJoin.h +++ b/src/Interpreters/ConcurrentHashJoin.h @@ -3,9 +3,11 @@ #include #include #include +#include #include #include #include +#include #include #include #include @@ -38,6 +40,7 @@ public: std::shared_ptr table_join_, size_t slots_, const Block & right_sample_block, + const StatsCollectingParams & stats_collecting_params_, bool any_take_last_row_ = false); ~ConcurrentHashJoin() override; @@ -70,6 +73,8 @@ private: std::unique_ptr pool; std::vector> hash_joins; + StatsCollectingParams stats_collecting_params; + std::mutex totals_mutex; Block totals; @@ -77,4 +82,5 @@ private: Blocks dispatchBlock(const Strings & key_columns_names, const Block & from_block); }; +UInt64 calculateCacheKey(std::shared_ptr & table_join, const QueryTreeNodePtr & right_table_expression); } diff --git a/src/Interpreters/ExpressionAnalyzer.cpp b/src/Interpreters/ExpressionAnalyzer.cpp index d80d5cd5b93..477f66f241c 100644 --- a/src/Interpreters/ExpressionAnalyzer.cpp +++ b/src/Interpreters/ExpressionAnalyzer.cpp @@ -982,7 +982,8 @@ static std::shared_ptr tryCreateJoin( const auto & settings = context->getSettings(); if (analyzed_join->allowParallelHashJoin()) - return std::make_shared(context, analyzed_join, settings.max_threads, right_sample_block); + return std::make_shared( + context, analyzed_join, settings.max_threads, right_sample_block, StatsCollectingParams{}); return std::make_shared(analyzed_join, right_sample_block); } diff --git a/src/Interpreters/HashTablesStatistics.h b/src/Interpreters/HashTablesStatistics.h index 4af9cf68817..4a13a0d3747 100644 --- a/src/Interpreters/HashTablesStatistics.h +++ b/src/Interpreters/HashTablesStatistics.h @@ -17,38 +17,14 @@ struct HashTablesCacheStatistics size_t misses = 0; }; -inline size_t calculateCacheKey(const DB::ASTPtr & select_query) -{ - if (!select_query) - throw DB::Exception(DB::ErrorCodes::LOGICAL_ERROR, "Query ptr cannot be null"); - - const auto & select = select_query->as(); - - // It may happen in some corner cases like `select 1 as num group by num`. - if (!select.tables()) - return 0; - - SipHash hash; - hash.update(select.tables()->getTreeHash(/*ignore_aliases=*/true)); - if (const auto where = select.where()) - hash.update(where->getTreeHash(/*ignore_aliases=*/true)); - if (const auto group_by = select.groupBy()) - hash.update(group_by->getTreeHash(/*ignore_aliases=*/true)); - return hash.get64(); -} - struct StatsCollectingParams { StatsCollectingParams() = default; - StatsCollectingParams( - const ASTPtr & select_query_, - bool collect_hash_table_stats_during_aggregation_, - size_t max_entries_for_hash_table_stats_, - size_t max_size_to_preallocate_for_aggregation_) - : key(collect_hash_table_stats_during_aggregation_ ? calculateCacheKey(select_query_) : 0) + StatsCollectingParams(UInt64 key_, bool enable_, size_t max_entries_for_hash_table_stats_, size_t max_size_to_preallocate_) + : key(enable_ ? key_ : 0) , max_entries_for_hash_table_stats(max_entries_for_hash_table_stats_) - , max_size_to_preallocate_for_aggregation(max_size_to_preallocate_for_aggregation_) + , max_size_to_preallocate(max_size_to_preallocate_) { } @@ -56,8 +32,8 @@ struct StatsCollectingParams void disable() { key = 0; } UInt64 key = 0; - const size_t max_entries_for_hash_table_stats = 0; - const size_t max_size_to_preallocate_for_aggregation = 0; + const size_t max_entries_for_hash_table_stats = 0; /// TODO: move to server settings + const size_t max_size_to_preallocate = 0; }; /** Collects observed HashMap-s sizes to avoid redundant intermediate resizes. @@ -154,4 +130,35 @@ inline std::optional getHashTablesCacheStatistics() return getHashTablesStatistics().getCacheStats(); } +inline std::optional +findSizeHint(const DB::StatsCollectingParams & stats_collecting_params, size_t max_threads) +{ + if (stats_collecting_params.isCollectionAndUseEnabled()) + { + if (auto hint = DB::getHashTablesStatistics().getSizeHint(stats_collecting_params)) + { + const auto lower_limit = hint->sum_of_sizes / max_threads; + const auto upper_limit = stats_collecting_params.max_size_to_preallocate / max_threads; + if (hint->median_size > upper_limit) + { + /// Since we cannot afford to preallocate as much as we want, we will likely need to do resize anyway. + /// But we will also work with the big (i.e. not so cache friendly) HT from the beginning which may result in a slight slowdown. + /// So let's just do nothing. + LOG_TRACE( + getLogger("HashTablesStatistics"), + "No space were preallocated in hash tables because 'max_size_to_preallocate' has too small value: {}, " + "should be at least {}", + stats_collecting_params.max_size_to_preallocate, + hint->median_size * max_threads); + } + /// https://github.com/ClickHouse/ClickHouse/issues/44402#issuecomment-1359920703 + else if ((max_threads > 1 && hint->sum_of_sizes > 100'000) || hint->sum_of_sizes > 500'000) + { + const auto adjusted = std::max(lower_limit, hint->median_size); + return HashTablesStatistics::Entry{hint->sum_of_sizes, adjusted}; + } + } + } + return std::nullopt; +} } diff --git a/src/Interpreters/InterpreterSelectQuery.cpp b/src/Interpreters/InterpreterSelectQuery.cpp index ec9341cf9d5..6476e3ab419 100644 --- a/src/Interpreters/InterpreterSelectQuery.cpp +++ b/src/Interpreters/InterpreterSelectQuery.cpp @@ -85,16 +85,17 @@ #include #include #include +#include #include #include #include #include #include +#include +#include #include #include #include -#include -#include namespace ProfileEvents @@ -2665,7 +2666,7 @@ static Aggregator::Params getAggregatorParams( size_t group_by_two_level_threshold_bytes) { const auto stats_collecting_params = StatsCollectingParams( - query_ptr, + calculateCacheKey(query_ptr), settings.collect_hash_table_stats_during_aggregation, settings.max_entries_for_hash_table_stats, settings.max_size_to_preallocate_for_aggregation); diff --git a/src/Planner/Planner.cpp b/src/Planner/Planner.cpp index 8ed7004cd42..d18b4fec52e 100644 --- a/src/Planner/Planner.cpp +++ b/src/Planner/Planner.cpp @@ -3,6 +3,7 @@ #include #include #include +#include #include #include @@ -363,7 +364,7 @@ Aggregator::Params getAggregatorParams(const PlannerContextPtr & planner_context const Settings & settings = query_context->getSettingsRef(); const auto stats_collecting_params = StatsCollectingParams( - select_query_info.query, + calculateCacheKey(select_query_info.query), settings.collect_hash_table_stats_during_aggregation, settings.max_entries_for_hash_table_stats, settings.max_size_to_preallocate_for_aggregation); diff --git a/src/Planner/PlannerJoins.cpp b/src/Planner/PlannerJoins.cpp index c410b04f209..e88d0d95341 100644 --- a/src/Planner/PlannerJoins.cpp +++ b/src/Planner/PlannerJoins.cpp @@ -768,10 +768,8 @@ std::shared_ptr tryDirectJoin(const std::shared_ptr(table_join, right_table_expression_header, storage, right_table_expression_header_with_storage_column_names); } - } - static std::shared_ptr tryCreateJoin(JoinAlgorithm algorithm, std::shared_ptr & table_join, const QueryTreeNodePtr & right_table_expression, @@ -805,7 +803,14 @@ static std::shared_ptr tryCreateJoin(JoinAlgorithm algorithm, if (table_join->allowParallelHashJoin()) { auto query_context = planner_context->getQueryContext(); - return std::make_shared(query_context, table_join, query_context->getSettings().max_threads, right_table_expression_header); + const auto & settings = query_context->getSettingsRef(); + StatsCollectingParams params{ + calculateCacheKey(table_join, right_table_expression), + settings.collect_hash_table_stats_during_aggregation, + settings.max_entries_for_hash_table_stats, + settings.max_size_to_preallocate_for_aggregation}; + return std::make_shared( + query_context, table_join, query_context->getSettings().max_threads, right_table_expression_header, params); } return std::make_shared(table_join, right_table_expression_header); From 439ac99897a8920c6d28c51318f8417a9dba445f Mon Sep 17 00:00:00 2001 From: divanik Date: Fri, 31 May 2024 13:10:42 +0000 Subject: [PATCH 032/417] Resolve several issues --- programs/disks/CommandList.cpp | 1 - programs/disks/DisksApp.cpp | 90 +++++++++++++--------------------- programs/disks/DisksApp.h | 15 +++--- programs/disks/DisksClient.cpp | 62 ++++++----------------- programs/disks/DisksClient.h | 4 +- programs/disks/ICommand.h | 10 +--- 6 files changed, 57 insertions(+), 125 deletions(-) diff --git a/programs/disks/CommandList.cpp b/programs/disks/CommandList.cpp index b2361532ceb..c21941c42ca 100644 --- a/programs/disks/CommandList.cpp +++ b/programs/disks/CommandList.cpp @@ -51,7 +51,6 @@ private: static void listRecursive(const DiskWithPath & disk, const std::string & relative_path, bool show_hidden) { - // std::cerr << absolute_path << std::endl; std::vector file_names = disk.listAllFilesByPath(relative_path); std::vector selected_and_sorted_file_names{}; diff --git a/programs/disks/DisksApp.cpp b/programs/disks/DisksApp.cpp index bd4c5293c70..4c1d98ec791 100644 --- a/programs/disks/DisksApp.cpp +++ b/programs/disks/DisksApp.cpp @@ -1,13 +1,9 @@ #include "DisksApp.h" #include #include -#include -#include #include "Common/Exception.h" +#include "Common/filesystemHelpers.h" #include -#include -#include -#include #include "DisksClient.h" #include "ICommand.h" @@ -30,19 +26,20 @@ extern const int BAD_ARGUMENTS; extern const int LOGICAL_ERROR; }; +LineReader::Patterns DisksApp::query_extenders = {"\\"}; +LineReader::Patterns DisksApp::query_delimiters = {""}; +String DisksApp::word_break_characters = " \t\v\f\a\b\r\n"; -CommandPtr DisksApp::getCommandByName(String command) const +CommandPtr DisksApp::getCommandByName(const String & command) const { - auto it = aliases.find(command); - if (it != aliases.end()) - { - command = it->second; - } try { + if (auto it = aliases.find(command); it != aliases.end()) + return command_descriptions.at(it->second); + return command_descriptions.at(command); } - catch (...) + catch (std::out_of_range &) { throw Exception(ErrorCodes::BAD_ARGUMENTS, "The command `{}` is unknown", command); } @@ -75,7 +72,7 @@ std::vector DisksApp::getEmptyCompletion(CommandPtr command_) const std::vector DisksApp::getCompletions(const String & prefix) const { - auto arguments = split(prefix, word_break_characters); + auto arguments = po::split_unix(prefix, word_break_characters); if (arguments.empty()) { return {}; @@ -171,14 +168,14 @@ std::vector DisksApp::getCompletions(const String & prefix) const } } -bool DisksApp::processQueryText(String text) +bool DisksApp::processQueryText(const String & text) { if (exit_strings.find(text) != exit_strings.end()) return false; CommandPtr command; try { - auto arguments = split(text, word_break_characters); + auto arguments = po::split_unix(text, word_break_characters); command = getCommandByName(arguments[0]); arguments.erase(arguments.begin()); command->execute(arguments, *client); @@ -188,7 +185,7 @@ bool DisksApp::processQueryText(String text) int code = getCurrentExceptionCode(); if (code == ErrorCodes::LOGICAL_ERROR) { - throw std::move(err); + throw err; } else if (code == ErrorCodes::BAD_ARGUMENTS) { @@ -272,6 +269,13 @@ void DisksApp::addOptions() #ifdef CLICKHOUSE_CLOUD command_descriptions.emplace("packed-io", makeCommandPackedIO()); #endif + for (const auto & [command_name, command_ptr] : command_descriptions) + { + if (command_name != command_ptr->command_name) + { + throw Exception(ErrorCodes::LOGICAL_ERROR, "Command name inside map doesn't coincide with actual command name"); + } + } } void DisksApp::processOptions() @@ -295,63 +299,35 @@ void DisksApp::printEntryHelpMessage() std::cout << options_description << '\n'; } -size_t DisksApp::getMagicConstant() -{ - size_t magic_constant = 0; - for (const auto & [current_command, _] : command_descriptions) - { - std::string command_string{}; - command_string += command_descriptions[current_command]->command_name; - bool was = false; - for (const auto & [alias_name, alias_command_name] : aliases) - { - if (alias_command_name == current_command) - { - if (was) - command_string += ","; - else - command_string += "("; - command_string += alias_name; - was = true; - } - } - command_string += (was ? ")" : ""); - - magic_constant = std::max(magic_constant, command_string.size()); - } - return magic_constant + 2; -} void DisksApp::printAvailableCommandsHelpMessage() { - size_t magic_constant = getMagicConstant(); - std::cout << "\x1b[1;33mAvailable commands:\x1b[0m\n"; + std::vector> commands_with_aliases_and_descrtiptions{}; + size_t maximal_command_length = 0; for (const auto & [current_command, _] : command_descriptions) { - std::string command_string{}; - command_string += command_descriptions[current_command]->command_name; - bool was = false; + std::string command_string = command_descriptions[current_command]->command_name; + bool need_comma = false; for (const auto & [alias_name, alias_command_name] : aliases) { if (alias_command_name == current_command) { - if (was) + if (std::exchange(need_comma, true)) command_string += ","; else command_string += "("; command_string += alias_name; - was = true; } } - command_string += (was ? ")" : ""); - std::cout << "\x1b[1;32m" << command_string << "\x1b[0m"; - for (size_t i = command_string.size(); i < magic_constant; ++i) - { - std::cout << " "; - } - - std::cout << command_descriptions[current_command]->description << "\n"; + command_string += (need_comma ? ")" : ""); + maximal_command_length = std::max(maximal_command_length, command_string.size()); + commands_with_aliases_and_descrtiptions.push_back({std::move(command_string), command_descriptions[current_command]->command_name}); + } + for (const auto & [command_with_aliases, description] : commands_with_aliases_and_descrtiptions) + { + std::cout << "\x1b[1;32m" << command_with_aliases << "\x1b[0m" + << std::string(maximal_command_length + 2 - command_with_aliases.size(), ' ') << description << "\n"; } } diff --git a/programs/disks/DisksApp.h b/programs/disks/DisksApp.h index fcb4b0b52f5..fad597335f0 100644 --- a/programs/disks/DisksApp.h +++ b/programs/disks/DisksApp.h @@ -25,13 +25,13 @@ public: void processOptions(); - bool processQueryText(String text); + bool processQueryText(const String & text); void init(const std::vector & common_arguments); int main(const std::vector & /*args*/) override; - CommandPtr getCommandByName(String command) const; + CommandPtr getCommandByName(const String & command) const; void initializeHistoryFile(); @@ -45,8 +45,6 @@ public: std::vector getEmptyCompletion(CommandPtr command_) const; - size_t getMagicConstant(); - ~DisksApp() override; private: @@ -57,11 +55,11 @@ private: // Fields responsible for the REPL work String history_file; LineReader::Suggest suggest; - LineReader::Patterns query_extenders = {"\\"}; - LineReader::Patterns query_delimiters = {}; - String word_break_characters{" \t\v\f\a\b\r\n"}; + static LineReader::Patterns query_extenders; + static LineReader::Patterns query_delimiters; + static String word_break_characters; - // General commang line arguments parsing fields + // General command line arguments parsing fields ContextMutablePtr global_context; ProgramOptionsDescription options_description; CommandLineOptions options; @@ -87,5 +85,4 @@ private: std::unique_ptr client{}; }; - } diff --git a/programs/disks/DisksClient.cpp b/programs/disks/DisksClient.cpp index 509892a39a2..1ebfaf40096 100644 --- a/programs/disks/DisksClient.cpp +++ b/programs/disks/DisksClient.cpp @@ -1,17 +1,10 @@ #include "DisksClient.h" #include #include -#include -#include -#include -#include -#include -#include - #include +#include #include -#include namespace ErrorCodes { @@ -22,46 +15,20 @@ extern const int LOGICAL_ERROR; namespace DB { -std::vector split(const String & text, const String & delimiters) +DiskWithPath::DiskWithPath(DiskPtr disk_, std::optional path_) : disk(disk_) { - std::vector arguments; - auto prev = text.begin(); - auto pos = std::find_if(text.begin(), text.end(), [&](char x) { return delimiters.contains(x); }); - while (pos != text.end()) + if (path_.has_value()) { - if (pos > prev) + if (!fs::path{path_.value()}.is_absolute()) { - arguments.push_back({prev, pos}); + throw Exception(ErrorCodes::BAD_ARGUMENTS, "Initializing path {} is not absolute", path_.value()); } - prev = ++pos; - pos = std::find_if(prev, text.end(), [&](char x) { return delimiters.contains(x); }); + path = path_.value(); } - if (pos > prev) + else { - arguments.push_back({prev, text.end()}); + path = String{"/"}; } - return arguments; -} - -DiskWithPath::DiskWithPath(DiskPtr disk_, std::optional path_) - : disk(disk_) - , path( - [&]() - { - if (path_.has_value()) - { - if (!fs::path{path_.value()}.is_absolute()) - { - throw Exception(ErrorCodes::BAD_ARGUMENTS, "Initializing path {} is not absolute", path_.value()); - } - return path_.value(); - } - else - { - return String{"/"}; - } - }()) -{ if (!disk->isDirectory(normalizePathAndGetAsRelative(path))) { throw Exception(ErrorCodes::BAD_ARGUMENTS, "Initializing path {} at disk {} is not a directory", path, disk->getName()); @@ -82,7 +49,7 @@ std::vector DiskWithPath::listAllFilesByPath(const String & any_path) co } } -std::vector DiskWithPath::getAllFilesByPattern(std::string pattern) const +std::vector DiskWithPath::getAllFilesByPattern(const String & pattern) const { auto [path_before, path_after] = [&]() -> std::pair { @@ -149,17 +116,16 @@ String DiskWithPath::validatePathAndGetAsRelative(const String & path) return lexically_normal_path; } -std::string DiskWithPath::normalizePathAndGetAsRelative(const std::string & messyPath) +String DiskWithPath::normalizePathAndGetAsRelative(const String & messyPath) { std::filesystem::path path(messyPath); std::filesystem::path canonical_path = std::filesystem::weakly_canonical(path); - std::string npath = canonical_path.make_preferred().string(); + String npath = canonical_path.make_preferred().string(); return validatePathAndGetAsRelative(npath); } -std::string DiskWithPath::normalizePath(const std::string & messyPath) +String DiskWithPath::normalizePath(const String & path) { - std::filesystem::path path(messyPath); std::filesystem::path canonical_path = std::filesystem::weakly_canonical(path); return canonical_path.make_preferred().string(); } @@ -174,7 +140,7 @@ DisksClient::DisksClient(std::vector>> { begin_disk = disks_with_paths[0].first->getName(); } - bool has_begin_disk = true; + bool has_begin_disk = false; for (auto & [disk, path] : disks_with_paths) { addDisk(disk, path); @@ -265,7 +231,7 @@ std::vector DisksClient::getAllDiskNames() const return answer; } -std::vector DisksClient::getAllFilesByPatternFromAllDisks(std::string pattern) const +std::vector DisksClient::getAllFilesByPatternFromAllDisks(const String & pattern) const { std::vector answer{}; for (const auto & [_, disk] : disks) diff --git a/programs/disks/DisksClient.h b/programs/disks/DisksClient.h index 5c2de7aa529..3320c5f7cef 100644 --- a/programs/disks/DisksClient.h +++ b/programs/disks/DisksClient.h @@ -36,7 +36,7 @@ public: std::vector listAllFilesByPath(const String & any_path) const; - std::vector getAllFilesByPattern(std::string pattern) const; + std::vector getAllFilesByPattern(const String & pattern) const; DiskPtr getDisk() const { return disk; } @@ -74,7 +74,7 @@ public: std::vector getAllDiskNames() const; - std::vector getAllFilesByPatternFromAllDisks(std::string pattern) const; + std::vector getAllFilesByPatternFromAllDisks(const String & pattern) const; private: diff --git a/programs/disks/ICommand.h b/programs/disks/ICommand.h index 1e05aefd28b..d726d50ba13 100644 --- a/programs/disks/ICommand.h +++ b/programs/disks/ICommand.h @@ -4,14 +4,13 @@ #include #include +#include #include -#include #include #include "Common/Exception.h" #include -#include #include #include "DisksApp.h" @@ -46,8 +45,6 @@ public: CommandLineOptions processCommandLineArguments(const Strings & commands); - void exit() { options_parsed = false; } - protected: template static T getValueFromCommandLineOptions(const CommandLineOptions & options, const String & name) @@ -56,7 +53,7 @@ protected: { return options[name].as(); } - catch (...) + catch (boost::bad_any_cast) { throw DB::Exception(ErrorCodes::BAD_ARGUMENTS, "Argument '{}' has wrong type and can't be parsed", name); } @@ -111,9 +108,6 @@ public: protected: PositionalProgramOptionsDescription positional_options_description; - -private: - bool options_parsed{}; }; DB::CommandPtr makeCommandCopy(); From 4b99a16b881c2c9e48c4f537cf96047a56e8d142 Mon Sep 17 00:00:00 2001 From: divanik Date: Fri, 31 May 2024 13:17:28 +0000 Subject: [PATCH 033/417] ../../tests --- tests/integration/test_disk_types/test.py | 12 ++- tests/integration/test_disks_app_func/test.py | 101 +++++++++++++++--- .../test_endpoint_macro_substitution/test.py | 6 +- tests/integration/test_multiple_disks/test.py | 4 +- 4 files changed, 101 insertions(+), 22 deletions(-) diff --git a/tests/integration/test_disk_types/test.py b/tests/integration/test_disk_types/test.py index 609a3168e77..f8893ef1dbc 100644 --- a/tests/integration/test_disk_types/test.py +++ b/tests/integration/test_disk_types/test.py @@ -6,7 +6,7 @@ disk_types = { "default": "Local", "disk_s3": "S3", "disk_encrypted": "S3", - "local": "Local" + "local": "Local", } # do not test HDFS on ARM @@ -20,9 +20,9 @@ def cluster(): cluster = ClickHouseCluster(__file__) cluster.add_instance( "node", - main_configs=["configs/storage_arm.xml"] - if is_arm() - else ["configs/storage_amd.xml"], + main_configs=( + ["configs/storage_arm.xml"] if is_arm() else ["configs/storage_amd.xml"] + ), with_minio=True, with_hdfs=not is_arm(), ) @@ -96,7 +96,9 @@ def test_select_by_type(cluster): if disk_type == "Local": assert ( node.query( - "SELECT name FROM system.disks WHERE type='" + disk_type + "' ORDER BY name" + "SELECT name FROM system.disks WHERE type='" + + disk_type + + "' ORDER BY name" ) == "default\nlocal\n" ) diff --git a/tests/integration/test_disks_app_func/test.py b/tests/integration/test_disks_app_func/test.py index dc2a538f64a..d643230d198 100644 --- a/tests/integration/test_disks_app_func/test.py +++ b/tests/integration/test_disks_app_func/test.py @@ -50,7 +50,13 @@ def test_disks_app_func_ld(started_cluster): ["/usr/bin/clickhouse", "disks", "--save-logs", "--query", "list-disks"] ) - disks = list(sorted(map(lambda x : x.split(':')[0], filter(lambda x : len(x) > 1, out.split("\n"))))) + disks = list( + sorted( + map( + lambda x: x.split(":")[0], filter(lambda x: len(x) > 1, out.split("\n")) + ) + ) + ) assert disks[:4] == ["default", "local", "test1", "test2"] @@ -61,7 +67,15 @@ def test_disks_app_func_ls(started_cluster): init_data(source) out = source.exec_in_container( - ["/usr/bin/clickhouse", "disks", "--save-logs", "--disk", "test1", "--query", "list ."] + [ + "/usr/bin/clickhouse", + "disks", + "--save-logs", + "--disk", + "test1", + "--query", + "list .", + ] ) files = out.split("\n") @@ -118,7 +132,15 @@ def test_disks_app_func_cp(started_cluster): ) out = source.exec_in_container( - ["/usr/bin/clickhouse", "disks", "--save-logs", "--disk", "test2", "--query", "list ."] + [ + "/usr/bin/clickhouse", + "disks", + "--save-logs", + "--disk", + "test2", + "--query", + "list .", + ] ) assert "path1" in out @@ -150,13 +172,29 @@ def test_disks_app_func_cp(started_cluster): # alesapin: Why we need list one more time? # kssenii: it is an assertion that the file is indeed deleted out = source.exec_in_container( - ["/usr/bin/clickhouse", "disks", "--save-logs", "--disk", "test2", "--query", "list ."] + [ + "/usr/bin/clickhouse", + "disks", + "--save-logs", + "--disk", + "test2", + "--query", + "list .", + ] ) assert "path1" not in out out = source.exec_in_container( - ["/usr/bin/clickhouse", "disks", "--save-logs", "--disk", "test1", "--query", "list ."] + [ + "/usr/bin/clickhouse", + "disks", + "--save-logs", + "--disk", + "test1", + "--query", + "list .", + ] ) assert "path1" not in out @@ -210,7 +248,15 @@ def test_disks_app_func_rm(started_cluster): ) out = source.exec_in_container( - ["/usr/bin/clickhouse", "disks", "--save-logs", "--disk", "test2", "--query", "list ."] + [ + "/usr/bin/clickhouse", + "disks", + "--save-logs", + "--disk", + "test2", + "--query", + "list .", + ] ) assert "path3" in out @@ -228,7 +274,15 @@ def test_disks_app_func_rm(started_cluster): ) out = source.exec_in_container( - ["/usr/bin/clickhouse", "disks", "--save-logs", "--disk", "test2", "--query", "list ."] + [ + "/usr/bin/clickhouse", + "disks", + "--save-logs", + "--disk", + "test2", + "--query", + "list .", + ] ) assert "path3" not in out @@ -240,8 +294,15 @@ def test_disks_app_func_mv(started_cluster): init_data(source) out = source.exec_in_container( - ["/usr/bin/clickhouse", "disks", "--save-logs", "--disk", "test1", - "--query", "list ."] + [ + "/usr/bin/clickhouse", + "disks", + "--save-logs", + "--disk", + "test1", + "--query", + "list .", + ] ) files = out.split("\n") @@ -260,7 +321,15 @@ def test_disks_app_func_mv(started_cluster): ) out = source.exec_in_container( - ["/usr/bin/clickhouse", "disks", "--save-logs", "--disk", "test1", "--query", "list ."] + [ + "/usr/bin/clickhouse", + "disks", + "--save-logs", + "--disk", + "test1", + "--query", + "list .", + ] ) files = out.split("\n") @@ -312,7 +381,15 @@ def test_remote_disk_list(started_cluster): init_data_s3(source) out = source.exec_in_container( - ["/usr/bin/clickhouse", "disks", "--save-logs", "--disk", "test3", "--query", "list ."] + [ + "/usr/bin/clickhouse", + "disks", + "--save-logs", + "--disk", + "test3", + "--query", + "list .", + ] ) files = out.split("\n") @@ -327,7 +404,7 @@ def test_remote_disk_list(started_cluster): "--disk", "test3", "--query", - "list . --recursive" + "list . --recursive", ] ) diff --git a/tests/integration/test_endpoint_macro_substitution/test.py b/tests/integration/test_endpoint_macro_substitution/test.py index 46d7aad4672..16da53c03ed 100644 --- a/tests/integration/test_endpoint_macro_substitution/test.py +++ b/tests/integration/test_endpoint_macro_substitution/test.py @@ -8,7 +8,7 @@ disk_types = { "disk_s3": "S3", "disk_hdfs": "HDFS", "disk_encrypted": "S3", - "local": "Local" + "local": "Local", } if is_arm(): @@ -75,7 +75,9 @@ def test_select_by_type(cluster): if disk_type == "Local": assert ( node.query( - "SELECT name FROM system.disks WHERE type='" + disk_type + "' ORDER BY name" + "SELECT name FROM system.disks WHERE type='" + + disk_type + + "' ORDER BY name" ) == "default\nlocal\n" ) diff --git a/tests/integration/test_multiple_disks/test.py b/tests/integration/test_multiple_disks/test.py index 83318c5efb0..ed29d4f9728 100644 --- a/tests/integration/test_multiple_disks/test.py +++ b/tests/integration/test_multiple_disks/test.py @@ -83,9 +83,7 @@ def test_system_tables(start_cluster): "path": "/external/", "keep_free_space": "0", }, - { - "name": "local", "path": "/", "keep_free_space": "0" - } + {"name": "local", "path": "/", "keep_free_space": "0"}, ] click_disk_data = json.loads( From 6a8bd46d3688544ed05c3d98a2e0042258cab88c Mon Sep 17 00:00:00 2001 From: Anton Popov Date: Fri, 31 May 2024 15:51:49 +0000 Subject: [PATCH 034/417] add tests --- .../03168_read_in_order_buffering_1.reference | 6 +++ .../03168_read_in_order_buffering_1.sql | 45 +++++++++++++++++++ .../03168_read_in_order_buffering_2.reference | 0 .../03168_read_in_order_buffering_2.sql | 16 +++++++ 4 files changed, 67 insertions(+) create mode 100644 tests/queries/0_stateless/03168_read_in_order_buffering_1.reference create mode 100644 tests/queries/0_stateless/03168_read_in_order_buffering_1.sql create mode 100644 tests/queries/0_stateless/03168_read_in_order_buffering_2.reference create mode 100644 tests/queries/0_stateless/03168_read_in_order_buffering_2.sql diff --git a/tests/queries/0_stateless/03168_read_in_order_buffering_1.reference b/tests/queries/0_stateless/03168_read_in_order_buffering_1.reference new file mode 100644 index 00000000000..306885a0974 --- /dev/null +++ b/tests/queries/0_stateless/03168_read_in_order_buffering_1.reference @@ -0,0 +1,6 @@ +1 +0 +1 +0 +0 +0 diff --git a/tests/queries/0_stateless/03168_read_in_order_buffering_1.sql b/tests/queries/0_stateless/03168_read_in_order_buffering_1.sql new file mode 100644 index 00000000000..02ffc9ecb7d --- /dev/null +++ b/tests/queries/0_stateless/03168_read_in_order_buffering_1.sql @@ -0,0 +1,45 @@ +DROP TABLE IF EXISTS t_read_in_order_1; + +CREATE TABLE t_read_in_order_1 (id UInt64, v UInt64) +ENGINE = MergeTree ORDER BY id +SETTINGS index_granularity = 1024, index_granularity_bytes = '10M'; + +INSERT INTO t_read_in_order_1 SELECT number, number FROM numbers(1000000); + +SET max_threads = 8; +SET optimize_read_in_order = 1; +SET read_in_order_max_bytes_to_buffer = '128M'; + +SELECT count() FROM +( + EXPLAIN PIPELINE SELECT * FROM t_read_in_order_1 ORDER BY id +) WHERE explain LIKE '%BufferChunks%'; + +SELECT count() FROM +( + EXPLAIN PIPELINE SELECT * FROM t_read_in_order_1 ORDER BY id LIMIT 10 +) WHERE explain LIKE '%BufferChunks%'; + +SELECT count() FROM +( + EXPLAIN PIPELINE SELECT * FROM t_read_in_order_1 WHERE v % 10 = 0 ORDER BY id LIMIT 10 +) WHERE explain LIKE '%BufferChunks%'; + +SET read_in_order_max_bytes_to_buffer = 0; + +SELECT count() FROM +( + EXPLAIN PIPELINE SELECT * FROM t_read_in_order_1 ORDER BY id +) WHERE explain LIKE '%BufferChunks%'; + +SELECT count() FROM +( + EXPLAIN PIPELINE SELECT * FROM t_read_in_order_1 ORDER BY id LIMIT 10 +) WHERE explain LIKE '%BufferChunks%'; + +SELECT count() FROM +( + EXPLAIN PIPELINE SELECT * FROM t_read_in_order_1 WHERE v % 10 = 0 ORDER BY id LIMIT 10 +) WHERE explain LIKE '%BufferChunks%'; + +DROP TABLE t_read_in_order_1; diff --git a/tests/queries/0_stateless/03168_read_in_order_buffering_2.reference b/tests/queries/0_stateless/03168_read_in_order_buffering_2.reference new file mode 100644 index 00000000000..e69de29bb2d diff --git a/tests/queries/0_stateless/03168_read_in_order_buffering_2.sql b/tests/queries/0_stateless/03168_read_in_order_buffering_2.sql new file mode 100644 index 00000000000..0631a2798dd --- /dev/null +++ b/tests/queries/0_stateless/03168_read_in_order_buffering_2.sql @@ -0,0 +1,16 @@ +-- Tags: long, no-random-settings + +DROP TABLE IF EXISTS t_read_in_order_2; + +CREATE TABLE t_read_in_order_2 (id UInt64, v UInt64) ENGINE = MergeTree ORDER BY id; + +INSERT INTO t_read_in_order_2 SELECT number, number FROM numbers(100000000); + +SET optimize_read_in_order = 1; +SET max_threads = 8; +SET read_in_order_max_bytes_to_buffer = '100M'; +SET max_memory_usage = '250M'; + +SELECT * FROM t_read_in_order_2 ORDER BY id FORMAT Null; + +DROP TABLE t_read_in_order_2; From 410c19de06194881825a271bbc2eaee8d957228e Mon Sep 17 00:00:00 2001 From: Anton Popov Date: Mon, 3 Jun 2024 14:43:38 +0000 Subject: [PATCH 035/417] fix test --- tests/queries/0_stateless/03168_read_in_order_buffering_2.sql | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/tests/queries/0_stateless/03168_read_in_order_buffering_2.sql b/tests/queries/0_stateless/03168_read_in_order_buffering_2.sql index 0631a2798dd..7781871390c 100644 --- a/tests/queries/0_stateless/03168_read_in_order_buffering_2.sql +++ b/tests/queries/0_stateless/03168_read_in_order_buffering_2.sql @@ -8,7 +8,7 @@ INSERT INTO t_read_in_order_2 SELECT number, number FROM numbers(100000000); SET optimize_read_in_order = 1; SET max_threads = 8; -SET read_in_order_max_bytes_to_buffer = '100M'; +SET read_in_order_max_bytes_to_buffer = '80M'; SET max_memory_usage = '250M'; SELECT * FROM t_read_in_order_2 ORDER BY id FORMAT Null; From 8a1984c245ff15740910c8c127a6a69640ca4088 Mon Sep 17 00:00:00 2001 From: Nikita Taranov Date: Tue, 28 May 2024 22:29:44 +0100 Subject: [PATCH 036/417] add test fix tests --- src/Common/ProfileEvents.cpp | 1 + src/Interpreters/ConcurrentHashJoin.cpp | 7 ++ ...151_hash_table_sizes_stats_joins.reference | 10 +++ .../02151_hash_table_sizes_stats_joins.sh | 74 +++++++++++++++++++ 4 files changed, 92 insertions(+) create mode 100644 tests/queries/0_stateless/02151_hash_table_sizes_stats_joins.reference create mode 100755 tests/queries/0_stateless/02151_hash_table_sizes_stats_joins.sh diff --git a/src/Common/ProfileEvents.cpp b/src/Common/ProfileEvents.cpp index 9bb7bece0f0..bf741272a62 100644 --- a/src/Common/ProfileEvents.cpp +++ b/src/Common/ProfileEvents.cpp @@ -555,6 +555,7 @@ The server successfully detected this situation and will download merged part fr M(AggregationPreallocatedElementsInHashTables, "How many elements were preallocated in hash tables for aggregation.") \ M(AggregationHashTablesInitializedAsTwoLevel, "How many hash tables were inited as two-level for aggregation.") \ M(AggregationOptimizedEqualRangesOfKeys, "For how many blocks optimization of equal ranges of keys was applied") \ + M(HashJoinPreallocatedElementsInHashTables, "How many elements were preallocated in hash tables for hash join.") \ \ M(MetadataFromKeeperCacheHit, "Number of times an object storage metadata request was answered from cache without making request to Keeper") \ M(MetadataFromKeeperCacheMiss, "Number of times an object storage metadata request had to be answered from Keeper") \ diff --git a/src/Interpreters/ConcurrentHashJoin.cpp b/src/Interpreters/ConcurrentHashJoin.cpp index 85e67898569..bde24941f1c 100644 --- a/src/Interpreters/ConcurrentHashJoin.cpp +++ b/src/Interpreters/ConcurrentHashJoin.cpp @@ -16,12 +16,18 @@ #include #include #include +#include #include #include #include #include #include +namespace ProfileEvents +{ +extern const Event HashJoinPreallocatedElementsInHashTables; +} + namespace CurrentMetrics { extern const Metric ConcurrentHashJoinPoolThreads; @@ -102,6 +108,7 @@ ConcurrentHashJoin::ConcurrentHashJoin( size_t reserve_size = 0; if (auto hint = findSizeHint(stats_collecting_params, slots)) reserve_size = hint->median_size; + ProfileEvents::increment(ProfileEvents::HashJoinPreallocatedElementsInHashTables, reserve_size); auto inner_hash_join = std::make_shared(); inner_hash_join->data = std::make_unique( diff --git a/tests/queries/0_stateless/02151_hash_table_sizes_stats_joins.reference b/tests/queries/0_stateless/02151_hash_table_sizes_stats_joins.reference new file mode 100644 index 00000000000..d3d171221e8 --- /dev/null +++ b/tests/queries/0_stateless/02151_hash_table_sizes_stats_joins.reference @@ -0,0 +1,10 @@ +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 diff --git a/tests/queries/0_stateless/02151_hash_table_sizes_stats_joins.sh b/tests/queries/0_stateless/02151_hash_table_sizes_stats_joins.sh new file mode 100755 index 00000000000..4ce32e51615 --- /dev/null +++ b/tests/queries/0_stateless/02151_hash_table_sizes_stats_joins.sh @@ -0,0 +1,74 @@ +#!/usr/bin/env bash +# Tags: long, distributed, no-debug, no-tsan, no-msan, no-ubsan, no-asan, no-random-settings, no-random-merge-tree-settings + +# shellcheck disable=SC2154 + +CURDIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd) +# shellcheck source=../shell_config.sh +. "$CURDIR"/../shell_config.sh + + +opts=( + --join_algorithm='parallel_hash' +) + +$CLICKHOUSE_CLIENT -nq " + CREATE TABLE t1(a UInt32, b UInt32) ENGINE=MergeTree ORDER BY (); + INSERT INTO t1 SELECT number, number FROM numbers_mt(1e6); + + CREATE TABLE t2(a UInt32, b UInt32) ENGINE=MergeTree ORDER BY (); + INSERT INTO t2 SELECT number, number FROM numbers_mt(1e6); +" + +queries_without_preallocation=() +queries_with_preallocation=() + +run_new_query() { + query_id1="hash_table_sizes_stats_joins_$RANDOM$RANDOM" + # when we see a query for the first time we only collect it stats when execution ends. preallocation will happen only on the next run + queries_without_preallocation+=("$query_id1") + $CLICKHOUSE_CLIENT "${opts[@]}" --query_id="$query_id1" -q "$1" --format Null + + query_id2="hash_table_sizes_stats_joins_$RANDOM$RANDOM" + queries_with_preallocation+=("$query_id2") + $CLICKHOUSE_CLIENT "${opts[@]}" --query_id="$query_id2" -q "$1" --format Null +} + +run_new_query "SELECT * FROM t1 AS x INNER JOIN t2 AS y ON x.a = y.a" +# it only matters what columns from the right table are part of the join key, as soon as we change them - it is a new cache entry +run_new_query "SELECT * FROM t1 AS x INNER JOIN t2 AS y ON x.a = y.b" +run_new_query "SELECT * FROM t1 AS x INNER JOIN t2 AS y USING (a, b)" + +# we already had a join on t2.a, so cache should be populated +query_id="hash_table_sizes_stats_joins_$RANDOM$RANDOM" +queries_with_preallocation+=("$query_id") +$CLICKHOUSE_CLIENT "${opts[@]}" --query_id="$query_id" -q "SELECT * FROM t1 AS x INNER JOIN t2 AS y ON x.b = y.a" --format Null +# the same query with a different alias for the t2 +query_id="hash_table_sizes_stats_joins_$RANDOM$RANDOM" +queries_with_preallocation+=("$query_id") +$CLICKHOUSE_CLIENT "${opts[@]}" --query_id="$query_id" -q "SELECT * FROM t1 AS x INNER JOIN t2 AS z ON x.b = z.a" --format Null + +# now t1 is the right table +run_new_query "SELECT * FROM t2 AS x INNER JOIN t1 AS y ON x.a = y.a" + +################################## + +$CLICKHOUSE_CLIENT -q "SYSTEM FLUSH LOGS" + +for i in "${!queries_without_preallocation[@]}"; do + $CLICKHOUSE_CLIENT --param_query_id="${queries_without_preallocation[$i]}" -q " + SELECT COUNT(*) + FROM system.query_log + WHERE event_date >= yesterday() AND query_id = {query_id:String} AND current_database = currentDatabase() + AND ProfileEvents['HashJoinPreallocatedElementsInHashTables'] = 0 AND type = 'QueryFinish' + " +done + +for i in "${!queries_with_preallocation[@]}"; do + $CLICKHOUSE_CLIENT --param_query_id="${queries_with_preallocation[$i]}" -q " + SELECT COUNT(*) + FROM system.query_log + WHERE event_date >= yesterday() AND query_id = {query_id:String} AND current_database = currentDatabase() + AND ProfileEvents['HashJoinPreallocatedElementsInHashTables'] > 0 AND type = 'QueryFinish' + " +done From 03352063e0361a07b24e5567b4024c4e89f080a9 Mon Sep 17 00:00:00 2001 From: Nikita Taranov Date: Tue, 28 May 2024 23:00:19 +0100 Subject: [PATCH 037/417] better refactor --- src/Interpreters/Aggregator.cpp | 10 +- src/Interpreters/ConcurrentHashJoin.cpp | 15 +-- src/Interpreters/HashTablesStatistics.cpp | 112 ++++++++++++++++++++++ src/Interpreters/HashTablesStatistics.h | 110 ++------------------- 4 files changed, 133 insertions(+), 114 deletions(-) create mode 100644 src/Interpreters/HashTablesStatistics.cpp diff --git a/src/Interpreters/Aggregator.cpp b/src/Interpreters/Aggregator.cpp index 46c2d3c09f0..e073b7a49b6 100644 --- a/src/Interpreters/Aggregator.cpp +++ b/src/Interpreters/Aggregator.cpp @@ -24,6 +24,7 @@ #include #include #include +#include #include #include #include @@ -105,7 +106,7 @@ void initDataVariantsWithSizeHint( { const auto & stats_collecting_params = params.stats_collecting_params; const auto max_threads = params.group_by_two_level_threshold != 0 ? std::max(params.max_threads, 1ul) : 1; - if (auto hint = findSizeHint(stats_collecting_params, max_threads)) + if (auto hint = getSizeHint(stats_collecting_params, /*tables_cnt=*/max_threads)) { if (worthConvertToTwoLevel( params.group_by_two_level_threshold, @@ -114,19 +115,19 @@ void initDataVariantsWithSizeHint( /*result_size_bytes*/ 0)) method_chosen = convertToTwoLevelTypeIfPossible(method_chosen); result.init(method_chosen, hint->median_size); - ProfileEvents::increment(ProfileEvents::AggregationHashTablesInitializedAsTwoLevel, result.isTwoLevel()); } else { result.init(method_chosen); } + ProfileEvents::increment(ProfileEvents::AggregationHashTablesInitializedAsTwoLevel, result.isTwoLevel()); } /// Collection and use of the statistics should be enabled. void updateStatistics(const DB::ManyAggregatedDataVariants & data_variants, const DB::StatsCollectingParams & params) { if (!params.isCollectionAndUseEnabled()) - throw DB::Exception(DB::ErrorCodes::LOGICAL_ERROR, "Collection and use of the statistics should be enabled."); + return; std::vector sizes(data_variants.size()); for (size_t i = 0; i < data_variants.size(); ++i) @@ -2634,8 +2635,7 @@ ManyAggregatedDataVariants Aggregator::prepareVariantsToMerge(ManyAggregatedData LOG_TRACE(log, "Merging aggregated data"); - if (params.stats_collecting_params.isCollectionAndUseEnabled()) - updateStatistics(data_variants, params.stats_collecting_params); + updateStatistics(data_variants, params.stats_collecting_params); ManyAggregatedDataVariants non_empty_data; non_empty_data.reserve(data_variants.size()); diff --git a/src/Interpreters/ConcurrentHashJoin.cpp b/src/Interpreters/ConcurrentHashJoin.cpp index bde24941f1c..4493a9f4dbd 100644 --- a/src/Interpreters/ConcurrentHashJoin.cpp +++ b/src/Interpreters/ConcurrentHashJoin.cpp @@ -2,6 +2,7 @@ #include #include #include +#include #include #include #include @@ -41,7 +42,7 @@ namespace void updateStatistics(const auto & hash_joins, const DB::StatsCollectingParams & params) { if (!params.isCollectionAndUseEnabled()) - throw DB::Exception(DB::ErrorCodes::LOGICAL_ERROR, "Collection and use of the statistics should be enabled."); + return; std::vector sizes(hash_joins.size()); for (size_t i = 0; i < hash_joins.size(); ++i) @@ -106,7 +107,7 @@ ConcurrentHashJoin::ConcurrentHashJoin( setThreadName("ConcurrentJoin"); size_t reserve_size = 0; - if (auto hint = findSizeHint(stats_collecting_params, slots)) + if (auto hint = getSizeHint(stats_collecting_params, slots)) reserve_size = hint->median_size; ProfileEvents::increment(ProfileEvents::HashJoinPreallocatedElementsInHashTables, reserve_size); @@ -341,13 +342,13 @@ Blocks ConcurrentHashJoin::dispatchBlock(const Strings & key_columns_names, cons UInt64 calculateCacheKey(std::shared_ptr & table_join, const QueryTreeNodePtr & right_table_expression) { IQueryTreeNode::HashState hash; + chassert(right_table_expression); hash.update(right_table_expression->getTreeHash()); - chassert(table_join->oneDisjunct()); - for (const auto & name : table_join->getClauses().at(0).key_names_right) + chassert(table_join && table_join->oneDisjunct()); + const auto keys + = NameOrderedSet{table_join->getClauses().at(0).key_names_right.begin(), table_join->getClauses().at(0).key_names_right.end()}; + for (const auto & name : keys) hash.update(name); - LOG_DEBUG(&Poco::Logger::get("debug"), "table_join->getClauses()={}", fmt::join(table_join->getClauses()[0].key_names_right, ", ")); - LOG_DEBUG(&Poco::Logger::get("debug"), "right_table_expression->dumpTree()={}", right_table_expression->dumpTree()); - LOG_DEBUG(&Poco::Logger::get("debug"), "hash.get64()={}", hash.get64()); return hash.get64(); } } diff --git a/src/Interpreters/HashTablesStatistics.cpp b/src/Interpreters/HashTablesStatistics.cpp new file mode 100644 index 00000000000..5ad06cdcea0 --- /dev/null +++ b/src/Interpreters/HashTablesStatistics.cpp @@ -0,0 +1,112 @@ +#include + +#include +#include + +namespace DB +{ + +std::optional HashTablesStatistics::getSizeHint(const Params & params) +{ + if (!params.isCollectionAndUseEnabled()) + throw DB::Exception(DB::ErrorCodes::LOGICAL_ERROR, "Collection and use of the statistics should be enabled."); + + std::lock_guard lock(mutex); + const auto cache = getHashTableStatsCache(params, lock); + if (const auto hint = cache->get(params.key)) + { + LOG_TRACE( + getLogger("HashTablesStatistics"), + "An entry for key={} found in cache: sum_of_sizes={}, median_size={}", + params.key, + hint->sum_of_sizes, + hint->median_size); + return *hint; + } + return std::nullopt; +} + +/// Collection and use of the statistics should be enabled. +void HashTablesStatistics::update(size_t sum_of_sizes, size_t median_size, const Params & params) +{ + if (!params.isCollectionAndUseEnabled()) + throw DB::Exception(DB::ErrorCodes::LOGICAL_ERROR, "Collection and use of the statistics should be enabled."); + + std::lock_guard lock(mutex); + const auto cache = getHashTableStatsCache(params, lock); + const auto hint = cache->get(params.key); + // We'll maintain the maximum among all the observed values until another prediction is much lower (that should indicate some change) + if (!hint || sum_of_sizes < hint->sum_of_sizes / 2 || hint->sum_of_sizes < sum_of_sizes || median_size < hint->median_size / 2 + || hint->median_size < median_size) + { + LOG_TRACE( + getLogger("HashTablesStatistics"), + "Statistics updated for key={}: new sum_of_sizes={}, median_size={}", + params.key, + sum_of_sizes, + median_size); + cache->set(params.key, std::make_shared(Entry{.sum_of_sizes = sum_of_sizes, .median_size = median_size})); + } +} + +std::optional HashTablesStatistics::getCacheStats() const +{ + std::lock_guard lock(mutex); + if (hash_table_stats) + { + size_t hits = 0, misses = 0; + hash_table_stats->getStats(hits, misses); + return DB::HashTablesCacheStatistics{.entries = hash_table_stats->count(), .hits = hits, .misses = misses}; + } + return std::nullopt; +} + +HashTablesStatistics::CachePtr HashTablesStatistics::getHashTableStatsCache(const Params & params, const std::lock_guard &) +{ + if (!hash_table_stats || hash_table_stats->maxSizeInBytes() != params.max_entries_for_hash_table_stats) + hash_table_stats = std::make_shared(params.max_entries_for_hash_table_stats); + return hash_table_stats; +} + +HashTablesStatistics & getHashTablesStatistics() +{ + static HashTablesStatistics hash_tables_stats; + return hash_tables_stats; +} + +std::optional getHashTablesCacheStatistics() +{ + return getHashTablesStatistics().getCacheStats(); +} + +std::optional getSizeHint(const DB::StatsCollectingParams & stats_collecting_params, size_t tables_cnt) +{ + if (stats_collecting_params.isCollectionAndUseEnabled()) + { + if (auto hint = DB::getHashTablesStatistics().getSizeHint(stats_collecting_params)) + { + const auto lower_limit = hint->sum_of_sizes / tables_cnt; + const auto upper_limit = stats_collecting_params.max_size_to_preallocate / tables_cnt; + if (hint->median_size > upper_limit) + { + /// Since we cannot afford to preallocate as much as needed, we would likely have to do at least one resize anyway. + /// Though it still sounds better than N resizes, but in actuality we saw that one big resize (remember, HT-s grow exponentially) + /// plus worse cache locality since we're dealing with big HT-s from the beginning yields worse performance. + /// So let's just do nothing. + LOG_TRACE( + getLogger("HashTablesStatistics"), + "No space were preallocated in hash tables because 'max_size_to_preallocate' has too small value: {}, " + "should be at least {}", + stats_collecting_params.max_size_to_preallocate, + hint->median_size * tables_cnt); + } + /// https://github.com/ClickHouse/ClickHouse/issues/44402#issuecomment-1359920703 + else if ((tables_cnt > 1 && hint->sum_of_sizes > 100'000) || hint->sum_of_sizes > 500'000) + { + return HashTablesStatistics::Entry{hint->sum_of_sizes, std::max(lower_limit, hint->median_size)}; + } + } + } + return std::nullopt; +} +} diff --git a/src/Interpreters/HashTablesStatistics.h b/src/Interpreters/HashTablesStatistics.h index 4a13a0d3747..f4357c6e8cb 100644 --- a/src/Interpreters/HashTablesStatistics.h +++ b/src/Interpreters/HashTablesStatistics.h @@ -1,10 +1,6 @@ #pragma once -#include -#include #include -#include -#include namespace DB @@ -36,7 +32,7 @@ struct StatsCollectingParams const size_t max_size_to_preallocate = 0; }; -/** Collects observed HashMap-s sizes to avoid redundant intermediate resizes. +/** Collects observed HashTable-s sizes to avoid redundant intermediate resizes. */ class HashTablesStatistics { @@ -52,113 +48,23 @@ public: using Params = StatsCollectingParams; /// Collection and use of the statistics should be enabled. - std::optional getSizeHint(const Params & params) - { - if (!params.isCollectionAndUseEnabled()) - throw DB::Exception(DB::ErrorCodes::LOGICAL_ERROR, "Collection and use of the statistics should be enabled."); - - std::lock_guard lock(mutex); - const auto cache = getHashTableStatsCache(params, lock); - if (const auto hint = cache->get(params.key)) - { - LOG_TRACE( - getLogger("Aggregator"), - "An entry for key={} found in cache: sum_of_sizes={}, median_size={}", - params.key, - hint->sum_of_sizes, - hint->median_size); - return *hint; - } - return std::nullopt; - } + std::optional getSizeHint(const Params & params); /// Collection and use of the statistics should be enabled. - void update(size_t sum_of_sizes, size_t median_size, const Params & params) - { - if (!params.isCollectionAndUseEnabled()) - throw DB::Exception(DB::ErrorCodes::LOGICAL_ERROR, "Collection and use of the statistics should be enabled."); + void update(size_t sum_of_sizes, size_t median_size, const Params & params); - std::lock_guard lock(mutex); - const auto cache = getHashTableStatsCache(params, lock); - const auto hint = cache->get(params.key); - // We'll maintain the maximum among all the observed values until the next prediction turns out to be too wrong. - if (!hint || sum_of_sizes < hint->sum_of_sizes / 2 || hint->sum_of_sizes < sum_of_sizes || median_size < hint->median_size / 2 - || hint->median_size < median_size) - { - LOG_TRACE( - getLogger("Aggregator"), - "Statistics updated for key={}: new sum_of_sizes={}, median_size={}", - params.key, - sum_of_sizes, - median_size); - cache->set(params.key, std::make_shared(Entry{.sum_of_sizes = sum_of_sizes, .median_size = median_size})); - } - } - - std::optional getCacheStats() const - { - std::lock_guard lock(mutex); - if (hash_table_stats) - { - size_t hits = 0, misses = 0; - hash_table_stats->getStats(hits, misses); - return DB::HashTablesCacheStatistics{.entries = hash_table_stats->count(), .hits = hits, .misses = misses}; - } - return std::nullopt; - } + std::optional getCacheStats() const; private: - CachePtr getHashTableStatsCache(const Params & params, const std::lock_guard &) - { - if (!hash_table_stats || hash_table_stats->maxSizeInBytes() != params.max_entries_for_hash_table_stats) - hash_table_stats = std::make_shared(params.max_entries_for_hash_table_stats); - return hash_table_stats; - } + CachePtr getHashTableStatsCache(const Params & params, const std::lock_guard &); mutable std::mutex mutex; CachePtr hash_table_stats; }; -inline HashTablesStatistics & getHashTablesStatistics() -{ - static HashTablesStatistics hash_tables_stats; - return hash_tables_stats; -} +HashTablesStatistics & getHashTablesStatistics(); -inline std::optional getHashTablesCacheStatistics() -{ - return getHashTablesStatistics().getCacheStats(); -} +std::optional getHashTablesCacheStatistics(); -inline std::optional -findSizeHint(const DB::StatsCollectingParams & stats_collecting_params, size_t max_threads) -{ - if (stats_collecting_params.isCollectionAndUseEnabled()) - { - if (auto hint = DB::getHashTablesStatistics().getSizeHint(stats_collecting_params)) - { - const auto lower_limit = hint->sum_of_sizes / max_threads; - const auto upper_limit = stats_collecting_params.max_size_to_preallocate / max_threads; - if (hint->median_size > upper_limit) - { - /// Since we cannot afford to preallocate as much as we want, we will likely need to do resize anyway. - /// But we will also work with the big (i.e. not so cache friendly) HT from the beginning which may result in a slight slowdown. - /// So let's just do nothing. - LOG_TRACE( - getLogger("HashTablesStatistics"), - "No space were preallocated in hash tables because 'max_size_to_preallocate' has too small value: {}, " - "should be at least {}", - stats_collecting_params.max_size_to_preallocate, - hint->median_size * max_threads); - } - /// https://github.com/ClickHouse/ClickHouse/issues/44402#issuecomment-1359920703 - else if ((max_threads > 1 && hint->sum_of_sizes > 100'000) || hint->sum_of_sizes > 500'000) - { - const auto adjusted = std::max(lower_limit, hint->median_size); - return HashTablesStatistics::Entry{hint->sum_of_sizes, adjusted}; - } - } - } - return std::nullopt; -} +std::optional getSizeHint(const DB::StatsCollectingParams & stats_collecting_params, size_t tables_cnt); } From 985a4badd871ace80c26423240b064bb7e658c44 Mon Sep 17 00:00:00 2001 From: Nikita Taranov Date: Mon, 3 Jun 2024 22:40:19 +0100 Subject: [PATCH 038/417] fix style fix style fix style --- src/Interpreters/HashTablesStatistics.cpp | 5 +++++ src/Interpreters/HashTablesStatistics.h | 1 - 2 files changed, 5 insertions(+), 1 deletion(-) diff --git a/src/Interpreters/HashTablesStatistics.cpp b/src/Interpreters/HashTablesStatistics.cpp index 5ad06cdcea0..91473ef7bbb 100644 --- a/src/Interpreters/HashTablesStatistics.cpp +++ b/src/Interpreters/HashTablesStatistics.cpp @@ -6,6 +6,11 @@ namespace DB { +namespace ErrorCodes +{ +extern const int LOGICAL_ERROR; +} + std::optional HashTablesStatistics::getSizeHint(const Params & params) { if (!params.isCollectionAndUseEnabled()) diff --git a/src/Interpreters/HashTablesStatistics.h b/src/Interpreters/HashTablesStatistics.h index f4357c6e8cb..73dd3c097d4 100644 --- a/src/Interpreters/HashTablesStatistics.h +++ b/src/Interpreters/HashTablesStatistics.h @@ -2,7 +2,6 @@ #include - namespace DB { From 44e4495e5bddbe044a47cf26f3996dd344a12e6e Mon Sep 17 00:00:00 2001 From: Nikita Taranov Date: Mon, 3 Jun 2024 22:20:49 +0100 Subject: [PATCH 039/417] move max_entries_for_hash_table_stats to server settings --- src/Core/ServerSettings.h | 1 + src/Core/Settings.h | 2 +- src/Interpreters/HashTablesStatistics.cpp | 4 ++-- src/Interpreters/InterpreterSelectQuery.cpp | 2 +- src/Planner/Planner.cpp | 2 +- src/Planner/PlannerJoins.cpp | 2 +- 6 files changed, 7 insertions(+), 6 deletions(-) diff --git a/src/Core/ServerSettings.h b/src/Core/ServerSettings.h index 45f235116ab..dd840e9bda3 100644 --- a/src/Core/ServerSettings.h +++ b/src/Core/ServerSettings.h @@ -146,6 +146,7 @@ namespace DB M(UInt64, global_profiler_real_time_period_ns, 0, "Period for real clock timer of global profiler (in nanoseconds). Set 0 value to turn off the real clock global profiler. Recommended value is at least 10000000 (100 times a second) for single queries or 1000000000 (once a second) for cluster-wide profiling.", 0) \ M(UInt64, global_profiler_cpu_time_period_ns, 0, "Period for CPU clock timer of global profiler (in nanoseconds). Set 0 value to turn off the CPU clock global profiler. Recommended value is at least 10000000 (100 times a second) for single queries or 1000000000 (once a second) for cluster-wide profiling.", 0) \ M(Bool, enable_azure_sdk_logging, false, "Enables logging from Azure sdk", 0) \ + M(UInt64, max_entries_for_hash_table_stats, 10'000, "How many entries hash table statistics collected during aggregation is allowed to have", 0) \ /// If you add a setting which can be updated at runtime, please update 'changeable_settings' map in StorageSystemServerSettings.cpp diff --git a/src/Core/Settings.h b/src/Core/Settings.h index dc61a049de8..a9c96597fa1 100644 --- a/src/Core/Settings.h +++ b/src/Core/Settings.h @@ -699,7 +699,6 @@ class IColumn; M(UInt64, insert_shard_id, 0, "If non zero, when insert into a distributed table, the data will be inserted into the shard `insert_shard_id` synchronously. Possible values range from 1 to `shards_number` of corresponding distributed table", 0) \ \ M(Bool, collect_hash_table_stats_during_aggregation, true, "Enable collecting hash table statistics to optimize memory allocation", 0) \ - M(UInt64, max_entries_for_hash_table_stats, 10'000, "How many entries hash table statistics collected during aggregation is allowed to have", 0) \ M(UInt64, max_size_to_preallocate_for_aggregation, 100'000'000, "For how many elements it is allowed to preallocate space in all hash tables in total before aggregation", 0) \ \ M(Bool, kafka_disable_num_consumers_limit, false, "Disable limit on kafka_num_consumers that depends on the number of available CPU cores", 0) \ @@ -976,6 +975,7 @@ class IColumn; MAKE_DEPRECATED_BY_SERVER_CONFIG(M, UInt64, async_insert_threads, 16) \ MAKE_DEPRECATED_BY_SERVER_CONFIG(M, UInt64, max_replicated_fetches_network_bandwidth_for_server, 0) \ MAKE_DEPRECATED_BY_SERVER_CONFIG(M, UInt64, max_replicated_sends_network_bandwidth_for_server, 0) \ + MAKE_DEPRECATED_BY_SERVER_CONFIG(M, UInt64, max_entries_for_hash_table_stats, 10'000) \ /* ---- */ \ MAKE_OBSOLETE(M, DefaultDatabaseEngine, default_database_engine, DefaultDatabaseEngine::Atomic) \ MAKE_OBSOLETE(M, UInt64, max_pipeline_depth, 0) \ diff --git a/src/Interpreters/HashTablesStatistics.cpp b/src/Interpreters/HashTablesStatistics.cpp index 91473ef7bbb..d66f1bbd1d3 100644 --- a/src/Interpreters/HashTablesStatistics.cpp +++ b/src/Interpreters/HashTablesStatistics.cpp @@ -68,8 +68,8 @@ std::optional HashTablesStatistics::getCacheStats() c HashTablesStatistics::CachePtr HashTablesStatistics::getHashTableStatsCache(const Params & params, const std::lock_guard &) { - if (!hash_table_stats || hash_table_stats->maxSizeInBytes() != params.max_entries_for_hash_table_stats) - hash_table_stats = std::make_shared(params.max_entries_for_hash_table_stats); + if (!hash_table_stats) + hash_table_stats = std::make_shared(params.max_entries_for_hash_table_stats * sizeof(Entry)); return hash_table_stats; } diff --git a/src/Interpreters/InterpreterSelectQuery.cpp b/src/Interpreters/InterpreterSelectQuery.cpp index 6476e3ab419..5f1af2f24bf 100644 --- a/src/Interpreters/InterpreterSelectQuery.cpp +++ b/src/Interpreters/InterpreterSelectQuery.cpp @@ -2668,7 +2668,7 @@ static Aggregator::Params getAggregatorParams( const auto stats_collecting_params = StatsCollectingParams( calculateCacheKey(query_ptr), settings.collect_hash_table_stats_during_aggregation, - settings.max_entries_for_hash_table_stats, + context.getServerSettings().max_entries_for_hash_table_stats, settings.max_size_to_preallocate_for_aggregation); return Aggregator::Params diff --git a/src/Planner/Planner.cpp b/src/Planner/Planner.cpp index d18b4fec52e..5f7a8fdbdde 100644 --- a/src/Planner/Planner.cpp +++ b/src/Planner/Planner.cpp @@ -366,7 +366,7 @@ Aggregator::Params getAggregatorParams(const PlannerContextPtr & planner_context const auto stats_collecting_params = StatsCollectingParams( calculateCacheKey(select_query_info.query), settings.collect_hash_table_stats_during_aggregation, - settings.max_entries_for_hash_table_stats, + query_context->getServerSettings().max_entries_for_hash_table_stats, settings.max_size_to_preallocate_for_aggregation); auto aggregate_descriptions = aggregation_analysis_result.aggregate_descriptions; diff --git a/src/Planner/PlannerJoins.cpp b/src/Planner/PlannerJoins.cpp index e88d0d95341..b98d2ec4da7 100644 --- a/src/Planner/PlannerJoins.cpp +++ b/src/Planner/PlannerJoins.cpp @@ -807,7 +807,7 @@ static std::shared_ptr tryCreateJoin(JoinAlgorithm algorithm, StatsCollectingParams params{ calculateCacheKey(table_join, right_table_expression), settings.collect_hash_table_stats_during_aggregation, - settings.max_entries_for_hash_table_stats, + query_context->getServerSettings().max_entries_for_hash_table_stats, settings.max_size_to_preallocate_for_aggregation}; return std::make_shared( query_context, table_join, query_context->getSettings().max_threads, right_table_expression_header, params); From 4d49be0145fa642dabdd62a5730938f63f3b6d7a Mon Sep 17 00:00:00 2001 From: Nikita Taranov Date: Tue, 4 Jun 2024 22:36:00 +0100 Subject: [PATCH 040/417] fix --- src/Planner/PlannerJoins.cpp | 2 ++ 1 file changed, 2 insertions(+) diff --git a/src/Planner/PlannerJoins.cpp b/src/Planner/PlannerJoins.cpp index b98d2ec4da7..cfcb43cb98d 100644 --- a/src/Planner/PlannerJoins.cpp +++ b/src/Planner/PlannerJoins.cpp @@ -43,6 +43,8 @@ #include #include +#include + namespace DB { From 86bbcb0037302ac699230a4d180b7eb98e1b4ee6 Mon Sep 17 00:00:00 2001 From: pufit Date: Thu, 6 Jun 2024 01:42:07 -0400 Subject: [PATCH 041/417] Startup scripts --- programs/server/Server.cpp | 46 +++++++++++++++++++ src/Core/ServerSettings.h | 1 + src/Interpreters/SystemLog.cpp | 6 +++ src/Interpreters/SystemLog.h | 12 ++--- .../test_startup_scripts/__init__.py | 0 .../test_startup_scripts/configs/config.xml | 14 ++++++ .../test_startup_scripts/configs/users.xml | 41 +++++++++++++++++ .../integration/test_startup_scripts/test.py | 18 ++++++++ 8 files changed, 132 insertions(+), 6 deletions(-) create mode 100644 tests/integration/test_startup_scripts/__init__.py create mode 100644 tests/integration/test_startup_scripts/configs/config.xml create mode 100644 tests/integration/test_startup_scripts/configs/users.xml create mode 100644 tests/integration/test_startup_scripts/test.py diff --git a/programs/server/Server.cpp b/programs/server/Server.cpp index 8fcb9d87a93..9654f90a928 100644 --- a/programs/server/Server.cpp +++ b/programs/server/Server.cpp @@ -613,6 +613,49 @@ static void sanityChecks(Server & server) } } +void loadStartupScripts(const Poco::Util::AbstractConfiguration & config, ContextMutablePtr context, Poco::Logger * log) +{ + try + { + Poco::Util::AbstractConfiguration::Keys keys; + config.keys("startup_scripts", keys); + + SetResultDetailsFunc callback; + for (const auto & key : keys) + { + std::string full_prefix = "startup_scripts." + key; + + if (config.has(full_prefix + ".condition")) + { + auto condition = config.getString(full_prefix + ".condition"); + auto condition_read_buffer = ReadBufferFromString(condition); + auto condition_write_buffer = WriteBufferFromOwnString(); + + LOG_DEBUG(log, "Checking startup query condition `{}`", condition); + executeQuery(condition_read_buffer, condition_write_buffer, true, context, callback, QueryFlags{ .internal = true }, std::nullopt, {}); + + auto result = condition_write_buffer.str(); + + if (result != "1\n" && result != "true\n") + continue; + + LOG_DEBUG(log, "Condition is true, will execute the query next"); + } + + auto query = config.getString(full_prefix + ".query"); + auto read_buffer = ReadBufferFromString(query); + auto write_buffer = WriteBufferFromOwnString(); + + LOG_DEBUG(log, "Executing query `{}`", query); + executeQuery(read_buffer, write_buffer, true, context, callback, QueryFlags{ .internal = true }, std::nullopt, {}); + } + } + catch (const std::exception & e) + { + LOG_ERROR(log, "Failed to parse startup scripts file {}", e.what()); + } +} + static void initializeAzureSDKLogger( [[ maybe_unused ]] const ServerSettings & server_settings, [[ maybe_unused ]] int server_logs_level) @@ -2107,6 +2150,9 @@ try load_metadata_tasks.clear(); load_metadata_tasks.shrink_to_fit(); + if (config().has("startup_scripts")) + loadStartupScripts(config(), global_context, log); + { std::lock_guard lock(servers_lock); for (auto & server : servers) diff --git a/src/Core/ServerSettings.h b/src/Core/ServerSettings.h index 45f235116ab..74bd6cdf0e5 100644 --- a/src/Core/ServerSettings.h +++ b/src/Core/ServerSettings.h @@ -146,6 +146,7 @@ namespace DB M(UInt64, global_profiler_real_time_period_ns, 0, "Period for real clock timer of global profiler (in nanoseconds). Set 0 value to turn off the real clock global profiler. Recommended value is at least 10000000 (100 times a second) for single queries or 1000000000 (once a second) for cluster-wide profiling.", 0) \ M(UInt64, global_profiler_cpu_time_period_ns, 0, "Period for CPU clock timer of global profiler (in nanoseconds). Set 0 value to turn off the CPU clock global profiler. Recommended value is at least 10000000 (100 times a second) for single queries or 1000000000 (once a second) for cluster-wide profiling.", 0) \ M(Bool, enable_azure_sdk_logging, false, "Enables logging from Azure sdk", 0) \ + M(Bool, prepare_system_log_tables_on_startup, false, "If true, ClickHouse creates all configured `_log` tables before the startup. It can be helpful if some startup scripts depend on `_log` tables.", 0) \ /// If you add a setting which can be updated at runtime, please update 'changeable_settings' map in StorageSystemServerSettings.cpp diff --git a/src/Interpreters/SystemLog.cpp b/src/Interpreters/SystemLog.cpp index 5e0ce2cb0de..952abd309ce 100644 --- a/src/Interpreters/SystemLog.cpp +++ b/src/Interpreters/SystemLog.cpp @@ -5,6 +5,7 @@ #include #include #include +#include #include #include #include @@ -350,10 +351,15 @@ SystemLogs::SystemLogs(ContextPtr global_context, const Poco::Util::AbstractConf if (blob_storage_log) logs.emplace_back(blob_storage_log.get()); + bool should_prepare = global_context->getServerSettings().prepare_system_log_tables_on_startup; try { for (auto & log : logs) + { log->startup(); + if (should_prepare) + log->prepareTable(); + } } catch (...) { diff --git a/src/Interpreters/SystemLog.h b/src/Interpreters/SystemLog.h index af635ca1bdb..b7becf5030c 100644 --- a/src/Interpreters/SystemLog.h +++ b/src/Interpreters/SystemLog.h @@ -132,6 +132,12 @@ public: void stopFlushThread() override; + /** Creates new table if it does not exist. + * Renames old table if its structure is not suitable. + * This cannot be done in constructor to avoid deadlock while renaming a table under locked Context when SystemLog object is created. + */ + void prepareTable() override; + protected: LoggerPtr log; @@ -142,12 +148,6 @@ protected: StoragePtr getStorage() const; - /** Creates new table if it does not exist. - * Renames old table if its structure is not suitable. - * This cannot be done in constructor to avoid deadlock while renaming a table under locked Context when SystemLog object is created. - */ - void prepareTable() override; - /// Some tables can override settings for internal queries virtual void addSettingsForQuery(ContextMutablePtr & mutable_context, IAST::QueryKind query_kind) const; diff --git a/tests/integration/test_startup_scripts/__init__.py b/tests/integration/test_startup_scripts/__init__.py new file mode 100644 index 00000000000..e69de29bb2d diff --git a/tests/integration/test_startup_scripts/configs/config.xml b/tests/integration/test_startup_scripts/configs/config.xml new file mode 100644 index 00000000000..42c1965c66d --- /dev/null +++ b/tests/integration/test_startup_scripts/configs/config.xml @@ -0,0 +1,14 @@ + + + + CREATE ROLE OR REPLACE testrole + + + GRANT CREATE USER, ALTER USER, DROP USER, SHOW USERS, SHOW CREATE USER ON *.* TO 'testrole' WITH GRANT OPTION; + + + CREATE TABLE TestTable (id UInt64) ENGINE=TinyLog + SELECT 1; + + + diff --git a/tests/integration/test_startup_scripts/configs/users.xml b/tests/integration/test_startup_scripts/configs/users.xml new file mode 100644 index 00000000000..f9917b034b2 --- /dev/null +++ b/tests/integration/test_startup_scripts/configs/users.xml @@ -0,0 +1,41 @@ + + + + + + + + 1 + + + + + + + + + + ::/0 + + + default + + default + + + + + + + + 3600 + + 0 + 0 + 0 + 0 + 0 + + + + diff --git a/tests/integration/test_startup_scripts/test.py b/tests/integration/test_startup_scripts/test.py new file mode 100644 index 00000000000..ee61994f830 --- /dev/null +++ b/tests/integration/test_startup_scripts/test.py @@ -0,0 +1,18 @@ +from helpers.cluster import ClickHouseCluster + + +def test_startup_scripts(): + cluster = ClickHouseCluster(__file__) + + node = cluster.add_instance( + "node", + main_configs=["configs/config.xml"], + with_zookeeper=False, + ) + + try: + cluster.start() + assert node.query("SHOW TABLES") == "TestTable\n" + + finally: + cluster.shutdown() From 3a4592c6285a6014a2c01010a5641274adfda7d2 Mon Sep 17 00:00:00 2001 From: Nikita Taranov Date: Thu, 6 Jun 2024 13:52:41 +0100 Subject: [PATCH 042/417] fix test --- .../0_stateless/02151_hash_table_sizes_stats_joins.sh | 6 ++++-- 1 file changed, 4 insertions(+), 2 deletions(-) diff --git a/tests/queries/0_stateless/02151_hash_table_sizes_stats_joins.sh b/tests/queries/0_stateless/02151_hash_table_sizes_stats_joins.sh index 4ce32e51615..fb0e4315652 100755 --- a/tests/queries/0_stateless/02151_hash_table_sizes_stats_joins.sh +++ b/tests/queries/0_stateless/02151_hash_table_sizes_stats_joins.sh @@ -57,7 +57,8 @@ $CLICKHOUSE_CLIENT -q "SYSTEM FLUSH LOGS" for i in "${!queries_without_preallocation[@]}"; do $CLICKHOUSE_CLIENT --param_query_id="${queries_without_preallocation[$i]}" -q " - SELECT COUNT(*) + -- the old analyzer is not supported + SELECT COUNT(*) * getSetting('allow_experimental_analyzer') FROM system.query_log WHERE event_date >= yesterday() AND query_id = {query_id:String} AND current_database = currentDatabase() AND ProfileEvents['HashJoinPreallocatedElementsInHashTables'] = 0 AND type = 'QueryFinish' @@ -66,7 +67,8 @@ done for i in "${!queries_with_preallocation[@]}"; do $CLICKHOUSE_CLIENT --param_query_id="${queries_with_preallocation[$i]}" -q " - SELECT COUNT(*) + -- the old analyzer is not supported + SELECT COUNT(*) * getSetting('allow_experimental_analyzer') FROM system.query_log WHERE event_date >= yesterday() AND query_id = {query_id:String} AND current_database = currentDatabase() AND ProfileEvents['HashJoinPreallocatedElementsInHashTables'] > 0 AND type = 'QueryFinish' From b17feefc580de21115d84e4fdec7952e6c2de3c2 Mon Sep 17 00:00:00 2001 From: Anton Popov Date: Thu, 6 Jun 2024 16:07:09 +0000 Subject: [PATCH 043/417] fix test --- tests/queries/0_stateless/03168_read_in_order_buffering_2.sql | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/tests/queries/0_stateless/03168_read_in_order_buffering_2.sql b/tests/queries/0_stateless/03168_read_in_order_buffering_2.sql index 7781871390c..a706965f3ff 100644 --- a/tests/queries/0_stateless/03168_read_in_order_buffering_2.sql +++ b/tests/queries/0_stateless/03168_read_in_order_buffering_2.sql @@ -1,4 +1,4 @@ --- Tags: long, no-random-settings +-- Tags: long, no-random-settings, no-tsan, no-asan, no-msan DROP TABLE IF EXISTS t_read_in_order_2; From d7c70d029e82666f677b3fea5007cb589204b8fe Mon Sep 17 00:00:00 2001 From: divanik Date: Thu, 6 Jun 2024 16:56:30 +0000 Subject: [PATCH 044/417] Ignore caches and encrypted --- programs/disks/DisksApp.cpp | 2 +- programs/disks/DisksClient.cpp | 4 ++-- src/Disks/DiskFactory.cpp | 8 +++++++- src/Disks/DiskFactory.h | 3 ++- src/Disks/DiskSelector.cpp | 8 ++++++-- src/Disks/DiskSelector.h | 4 +++- 6 files changed, 21 insertions(+), 8 deletions(-) diff --git a/programs/disks/DisksApp.cpp b/programs/disks/DisksApp.cpp index 4c1d98ec791..8d1c4b24fc1 100644 --- a/programs/disks/DisksApp.cpp +++ b/programs/disks/DisksApp.cpp @@ -430,7 +430,7 @@ int DisksApp::main(const std::vector & /*args*/) auto validator = [](const Poco::Util::AbstractConfiguration &, const std::string &, const std::string &) { return true; }; constexpr auto config_prefix = "storage_configuration.disks"; - auto disk_selector = std::make_shared(); + auto disk_selector = std::make_shared(std::unordered_set{"cache", "encrypted"}); disk_selector->initialize(config(), config_prefix, global_context, validator); std::vector>> disks_with_path; diff --git a/programs/disks/DisksClient.cpp b/programs/disks/DisksClient.cpp index 1ebfaf40096..40b458fd7b3 100644 --- a/programs/disks/DisksClient.cpp +++ b/programs/disks/DisksClient.cpp @@ -8,8 +8,8 @@ namespace ErrorCodes { -extern const int BAD_ARGUMENTS; -extern const int LOGICAL_ERROR; + extern const int BAD_ARGUMENTS; + extern const int LOGICAL_ERROR; }; namespace DB diff --git a/src/Disks/DiskFactory.cpp b/src/Disks/DiskFactory.cpp index de7ee5a74f4..4aa7f6ff564 100644 --- a/src/Disks/DiskFactory.cpp +++ b/src/Disks/DiskFactory.cpp @@ -27,7 +27,8 @@ DiskPtr DiskFactory::create( ContextPtr context, const DisksMap & map, bool attach, - bool custom_disk) const + bool custom_disk, + const std::unordered_set & skip_types) const { const auto disk_type = config.getString(config_prefix + ".type", "local"); @@ -38,6 +39,11 @@ DiskPtr DiskFactory::create( "DiskFactory: the disk '{}' has unknown disk type: {}", name, disk_type); } + if (skip_types.contains(found->first)) + { + return nullptr; + } + const auto & disk_creator = found->second; return disk_creator(name, config, config_prefix, context, map, attach, custom_disk); } diff --git a/src/Disks/DiskFactory.h b/src/Disks/DiskFactory.h index d03ffa6a40f..044ce81dbae 100644 --- a/src/Disks/DiskFactory.h +++ b/src/Disks/DiskFactory.h @@ -42,7 +42,8 @@ public: ContextPtr context, const DisksMap & map, bool attach = false, - bool custom_disk = false) const; + bool custom_disk = false, + const std::unordered_set & skip_types = {}) const; private: using DiskTypeRegistry = std::unordered_map; diff --git a/src/Disks/DiskSelector.cpp b/src/Disks/DiskSelector.cpp index 77e2299ed65..b187b491dc0 100644 --- a/src/Disks/DiskSelector.cpp +++ b/src/Disks/DiskSelector.cpp @@ -53,8 +53,12 @@ void DiskSelector::initialize( if (disk_validator && !disk_validator(config, disk_config_prefix, disk_name)) continue; - - disks.emplace(disk_name, factory.create(disk_name, config, disk_config_prefix, context, disks)); + auto created_disk + = factory.create(disk_name, config, disk_config_prefix, context, disks, /*attach*/ false, /*custom_disk*/ false, skip_types); + if (created_disk.get()) + { + disks.emplace(disk_name, std::move(created_disk)); + } } if (!has_default_disk) { diff --git a/src/Disks/DiskSelector.h b/src/Disks/DiskSelector.h index 6669b428158..fb3cb4a0177 100644 --- a/src/Disks/DiskSelector.h +++ b/src/Disks/DiskSelector.h @@ -20,7 +20,7 @@ class DiskSelector public: static constexpr auto TMP_INTERNAL_DISK_PREFIX = "__tmp_internal_"; - DiskSelector() = default; + explicit DiskSelector(std::unordered_set skip_types_ = {}) : skip_types(skip_types_) { } DiskSelector(const DiskSelector & from) = default; using DiskValidator = std::function; @@ -48,6 +48,8 @@ private: bool is_initialized = false; void assertInitialized() const; + + const std::unordered_set skip_types; }; } From 3f7a9738fee0f7bd79a4161b6087bdcb19f577cc Mon Sep 17 00:00:00 2001 From: divanik Date: Fri, 7 Jun 2024 09:04:28 +0000 Subject: [PATCH 045/417] Fix compilation bug --- programs/disks/DisksApp.cpp | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/programs/disks/DisksApp.cpp b/programs/disks/DisksApp.cpp index 8d1c4b24fc1..d50e3082e23 100644 --- a/programs/disks/DisksApp.cpp +++ b/programs/disks/DisksApp.cpp @@ -430,7 +430,7 @@ int DisksApp::main(const std::vector & /*args*/) auto validator = [](const Poco::Util::AbstractConfiguration &, const std::string &, const std::string &) { return true; }; constexpr auto config_prefix = "storage_configuration.disks"; - auto disk_selector = std::make_shared(std::unordered_set{"cache", "encrypted"}); + auto disk_selector = std::make_shared(std::unordered_set{"cache", "encrypted"}); disk_selector->initialize(config(), config_prefix, global_context, validator); std::vector>> disks_with_path; From bdd8bcc0d9b68474ca10df52772babc5aa1a20d4 Mon Sep 17 00:00:00 2001 From: Konstantin Morozov Date: Fri, 7 Jun 2024 15:51:13 +0000 Subject: [PATCH 046/417] add some log --- src/Databases/DatabaseAtomic.cpp | 7 +++++++ src/Storages/StorageReplicatedMergeTree.cpp | 19 ++++++++++++++++--- 2 files changed, 23 insertions(+), 3 deletions(-) diff --git a/src/Databases/DatabaseAtomic.cpp b/src/Databases/DatabaseAtomic.cpp index ccab72cfbae..d431eb5c1b7 100644 --- a/src/Databases/DatabaseAtomic.cpp +++ b/src/Databases/DatabaseAtomic.cpp @@ -112,6 +112,13 @@ StoragePtr DatabaseAtomic::detachTable(ContextPtr /* context */, const String & table_name_to_path.erase(name); detached_tables.emplace(table->getStorageID().uuid, table); not_in_use = cleanupDetachedTables(); + + if (!not_in_use.empty()) + { + not_in_use.clear(); + LOG_DEBUG(log, "Finish removing non using detached tables"); + } + return table; } diff --git a/src/Storages/StorageReplicatedMergeTree.cpp b/src/Storages/StorageReplicatedMergeTree.cpp index e18e66d7af9..68bb5916d7c 100644 --- a/src/Storages/StorageReplicatedMergeTree.cpp +++ b/src/Storages/StorageReplicatedMergeTree.cpp @@ -5,20 +5,21 @@ #include #include +#include +#include #include #include #include #include +#include #include #include #include #include #include +#include #include #include -#include -#include -#include #include @@ -5272,6 +5273,8 @@ void StorageReplicatedMergeTree::flushAndPrepareForShutdown() if (shutdown_prepared_called.exchange(true)) return; + LOG_TRACE(log, "Start preparing for shutdown"); + try { auto settings_ptr = getSettings(); @@ -5282,7 +5285,11 @@ void StorageReplicatedMergeTree::flushAndPrepareForShutdown() stopBeingLeader(); if (attach_thread) + { attach_thread->shutdown(); + LOG_TRACE(log, "Attach thread shutdowned"); + } + restarting_thread.shutdown(/* part_of_full_shutdown */true); /// Explicitly set the event, because the restarting thread will not set it again @@ -5295,6 +5302,8 @@ void StorageReplicatedMergeTree::flushAndPrepareForShutdown() shutdown_deadline.emplace(std::chrono::system_clock::now()); throw; } + + LOG_TRACE(log, "Finish preparing for shutdown"); } void StorageReplicatedMergeTree::partialShutdown() @@ -5332,6 +5341,9 @@ void StorageReplicatedMergeTree::shutdown(bool) if (shutdown_called.exchange(true)) return; + const auto storage_name = getStorageID().getNameForLogs(); + LOG_TRACE(log, "Shutdown started, table={}", storage_name); + flushAndPrepareForShutdown(); if (!shutdown_deadline.has_value()) @@ -5374,6 +5386,7 @@ void StorageReplicatedMergeTree::shutdown(bool) /// Wait for all of them std::lock_guard lock(data_parts_exchange_ptr->rwlock); } + LOG_TRACE(log, "Shutdown finished, table={}", storage_name); } From 78325c89cccbb034a3cdcaf606eb6e2919096f8b Mon Sep 17 00:00:00 2001 From: divanik Date: Mon, 10 Jun 2024 16:17:36 +0000 Subject: [PATCH 047/417] Change help message --- programs/disks/CMakeLists.txt | 3 +- programs/disks/CommandChangeDirectory.cpp | 2 +- programs/disks/CommandHelp.cpp | 42 +++++ programs/disks/CommandMkDir.cpp | 2 +- programs/disks/DisksApp.cpp | 190 +++++++++++++--------- programs/disks/DisksApp.h | 17 +- programs/disks/ICommand.h | 1 + 7 files changed, 177 insertions(+), 80 deletions(-) create mode 100644 programs/disks/CommandHelp.cpp diff --git a/programs/disks/CMakeLists.txt b/programs/disks/CMakeLists.txt index 0f3cb601750..2bf17a352e6 100644 --- a/programs/disks/CMakeLists.txt +++ b/programs/disks/CMakeLists.txt @@ -12,7 +12,8 @@ set (CLICKHOUSE_DISKS_SOURCES CommandRead.cpp CommandRemove.cpp CommandSwitchDisk.cpp - CommandWrite.cpp) + CommandWrite.cpp + CommandHelp.cpp) if (CLICKHOUSE_CLOUD) set (CLICKHOUSE_DISKS_SOURCES ${CLICKHOUSE_DISKS_SOURCES} CommandPackedIO.cpp) diff --git a/programs/disks/CommandChangeDirectory.cpp b/programs/disks/CommandChangeDirectory.cpp index 71cdae904e5..5e6a08cd3fd 100644 --- a/programs/disks/CommandChangeDirectory.cpp +++ b/programs/disks/CommandChangeDirectory.cpp @@ -13,7 +13,7 @@ public: explicit CommandChangeDirectory() : ICommand() { command_name = "cd"; - description = "Change directory (makes sense only in interactive mode)"; + description = "Change directory"; options_description.add_options()("path", po::value(), "the path we want to get to (mandatory, positional)")( "disk", po::value(), "A disk where the path is changed"); positional_options_description.add("path", 1); diff --git a/programs/disks/CommandHelp.cpp b/programs/disks/CommandHelp.cpp new file mode 100644 index 00000000000..becdae324b3 --- /dev/null +++ b/programs/disks/CommandHelp.cpp @@ -0,0 +1,42 @@ +#include "DisksApp.h" +#include "ICommand.h" + +#include +#include + +namespace DB +{ + +class CommandHelp final : public ICommand +{ +public: + explicit CommandHelp(const DisksApp & disks_app_) : disks_app(disks_app_) + { + command_name = "help"; + description = "Print help message about available commands (all or only required)"; + options_description.add_options()("command", po::value(), "A command to help with"); + positional_options_description.add("command", 1); + } + + void executeImpl(const CommandLineOptions & options, DisksClient & /*client*/) override + { + std::optional command = getValueFromCommandLineOptionsWithOptional(options, "command"); + if (command.has_value()) + { + disks_app.printCommandHelpMessage(command.value()); + } + else + { + disks_app.printAvailableCommandsHelpMessage(); + } + } + + const DisksApp & disks_app; +}; + +CommandPtr makeCommandHelp(const DisksApp & disks_app) +{ + return std::make_shared(disks_app); +} + +} diff --git a/programs/disks/CommandMkDir.cpp b/programs/disks/CommandMkDir.cpp index 895602adf72..3ea6df5622d 100644 --- a/programs/disks/CommandMkDir.cpp +++ b/programs/disks/CommandMkDir.cpp @@ -12,7 +12,7 @@ public: CommandMkDir() { command_name = "mkdir"; - description = "Create a directory"; + description = "Creates a directory"; options_description.add_options()("recursive", "recursively create directories")( "path", po::value(), "the path of listing (mandatory, positional)"); positional_options_description.add("path", 1); diff --git a/programs/disks/DisksApp.cpp b/programs/disks/DisksApp.cpp index d50e3082e23..296567c4b35 100644 --- a/programs/disks/DisksApp.cpp +++ b/programs/disks/DisksApp.cpp @@ -6,6 +6,7 @@ #include #include "DisksClient.h" #include "ICommand.h" +#include "ICommand_fwd.h" #include #include @@ -45,11 +46,12 @@ CommandPtr DisksApp::getCommandByName(const String & command) const } } -std::vector DisksApp::getEmptyCompletion(CommandPtr command_) const +std::vector DisksApp::getEmptyCompletion(String command_name) const { + auto command_ptr = command_descriptions.at(command_name); auto answer = [&]() -> std::vector { - if (multidisk_commands.contains(command_->command_name)) + if (multidisk_commands.contains(command_ptr->command_name)) { return client->getAllFilesByPatternFromAllDisks(""); } @@ -62,14 +64,49 @@ std::vector DisksApp::getEmptyCompletion(CommandPtr command_) const { answer.push_back(disk_name); } - for (const auto & option : command_->options_description.options()) + for (const auto & option : command_ptr->options_description.options()) { answer.push_back("--" + option->long_name()); } + if (command_name == "help") + { + for (const auto & [current_command_name, description] : command_descriptions) + { + answer.push_back(current_command_name); + } + } std::sort(answer.begin(), answer.end()); return answer; } +std::vector DisksApp::getCommandsToComplete(const String & command_prefix) const +{ + std::vector answer{}; + for (const auto & [word, _] : command_descriptions) + { + if (word.starts_with(command_prefix)) + { + answer.push_back(word); + } + } + if (!answer.empty()) + { + return answer; + } + for (const auto & [word, _] : aliases) + { + if (word.starts_with(command_prefix)) + { + answer.push_back(word); + } + } + if (!answer.empty()) + { + return answer; + } + return {command_prefix}; +} + std::vector DisksApp::getCompletions(const String & prefix) const { auto arguments = po::split_unix(prefix, word_break_characters); @@ -88,35 +125,12 @@ std::vector DisksApp::getCompletions(const String & prefix) const { return {arguments.back()}; } - return getEmptyCompletion(command); + return getEmptyCompletion(command->command_name); } else if (arguments.size() == 1) { String command_prefix = arguments[0]; - std::vector answer{}; - for (const auto & [word, _] : command_descriptions) - { - if (word.starts_with(command_prefix)) - { - answer.push_back(word); - } - } - if (!answer.empty()) - { - return answer; - } - for (const auto & [word, _] : aliases) - { - if (word.starts_with(command_prefix)) - { - answer.push_back(word); - } - } - if (!answer.empty()) - { - return answer; - } - return {command_prefix}; + return getCommandsToComplete(command_prefix); } else { @@ -130,31 +144,39 @@ std::vector DisksApp::getCompletions(const String & prefix) const { return {last_token}; } - auto answer = [&]() -> std::vector + std::vector answer = {}; + if (command->command_name == "help") { - if (multidisk_commands.contains(command->command_name)) - { - return client->getAllFilesByPatternFromAllDisks(last_token); - } - else - { - return client->getCurrentDiskWithPath().getAllFilesByPattern(last_token); - } - }(); - - for (const auto & disk_name : client->getAllDiskNames()) - { - if (disk_name.starts_with(last_token)) - { - answer.push_back(disk_name); - } + return getCommandsToComplete(last_token); } - for (const auto & option : command->options_description.options()) + else { - String option_sign = "--" + option->long_name(); - if (option_sign.starts_with(last_token)) + answer = [&]() -> std::vector { - answer.push_back(option_sign); + if (multidisk_commands.contains(command->command_name)) + { + return client->getAllFilesByPatternFromAllDisks(last_token); + } + else + { + return client->getCurrentDiskWithPath().getAllFilesByPattern(last_token); + } + }(); + + for (const auto & disk_name : client->getAllDiskNames()) + { + if (disk_name.starts_with(last_token)) + { + answer.push_back(disk_name); + } + } + for (const auto & option : command->options_description.options()) + { + String option_sign = "--" + option->long_name(); + if (option_sign.starts_with(last_token)) + { + answer.push_back(option_sign); + } } } if (!answer.empty()) @@ -266,6 +288,7 @@ void DisksApp::addOptions() command_descriptions.emplace("read", makeCommandRead()); command_descriptions.emplace("mkdir", makeCommandMkDir()); command_descriptions.emplace("switch-disk", makeCommandSwitchDisk()); + command_descriptions.emplace("help", makeCommandHelp(*this)); #ifdef CLICKHOUSE_CLOUD command_descriptions.emplace("packed-io", makeCommandPackedIO()); #endif @@ -293,44 +316,64 @@ void DisksApp::processOptions() } -void DisksApp::printEntryHelpMessage() +void DisksApp::printEntryHelpMessage() const { - std::cout << "ClickHouse disk management tool\n"; + std::cout << "\x1b[1;33m ClickHouse disk management tool \x1b[0m \n"; std::cout << options_description << '\n'; } -void DisksApp::printAvailableCommandsHelpMessage() +void DisksApp::printAvailableCommandsHelpMessage() const { - std::cout << "\x1b[1;33mAvailable commands:\x1b[0m\n"; - std::vector> commands_with_aliases_and_descrtiptions{}; + std::cout << "\x1b[1;32mAvailable commands:\x1b[0m\n"; + std::vector> commands_with_aliases_and_descrtiptions{}; size_t maximal_command_length = 0; - for (const auto & [current_command, _] : command_descriptions) + for (const auto & [command_name, command_ptr] : command_descriptions) { - std::string command_string = command_descriptions[current_command]->command_name; - bool need_comma = false; - for (const auto & [alias_name, alias_command_name] : aliases) - { - if (alias_command_name == current_command) - { - if (std::exchange(need_comma, true)) - command_string += ","; - else - command_string += "("; - command_string += alias_name; - } - } - command_string += (need_comma ? ")" : ""); + std::string command_string = getCommandLineWithAliases(command_ptr); maximal_command_length = std::max(maximal_command_length, command_string.size()); - commands_with_aliases_and_descrtiptions.push_back({std::move(command_string), command_descriptions[current_command]->command_name}); + commands_with_aliases_and_descrtiptions.push_back({std::move(command_string), command_descriptions.at(command_name)}); } - for (const auto & [command_with_aliases, description] : commands_with_aliases_and_descrtiptions) + for (const auto & [command_with_aliases, command_ptr] : commands_with_aliases_and_descrtiptions) { - std::cout << "\x1b[1;32m" << command_with_aliases << "\x1b[0m" - << std::string(maximal_command_length + 2 - command_with_aliases.size(), ' ') << description << "\n"; + std::cout << "\x1b[1;33m" << command_with_aliases << "\x1b[0m" << std::string(5, ' ') << "\x1b[1;33m" << command_ptr->description + << "\x1b[0m \n"; + std::cout << command_ptr->options_description; + std::cout << std::endl; } } +void DisksApp::printCommandHelpMessage(CommandPtr command) const +{ + String command_name_with_aliases = getCommandLineWithAliases(command); + std::cout << "\x1b[1;32m" << command_name_with_aliases << "\x1b[0m" << std::string(2, ' ') << command->description << "\n"; + std::cout << command->options_description; +} + +void DisksApp::printCommandHelpMessage(String command_name) const +{ + printCommandHelpMessage(getCommandByName(command_name)); +} + +String DisksApp::getCommandLineWithAliases(CommandPtr command) const +{ + String command_string = command->command_name; + bool need_comma = false; + for (const auto & [alias_name, alias_command_name] : aliases) + { + if (alias_command_name == command->command_name) + { + if (std::exchange(need_comma, true)) + command_string += ","; + else + command_string += "("; + command_string += alias_name; + } + } + command_string += (need_comma ? ")" : ""); + return command_string; +} + void DisksApp::initializeHistoryFile() { String home_path; @@ -423,6 +466,7 @@ int DisksApp::main(const std::vector & /*args*/) global_context->setApplicationType(Context::ApplicationType::DISKS); String path = config().getString("path", DBMS_DEFAULT_PATH); + global_context->setPath(path); String main_disk = config().getString("disk", "default"); diff --git a/programs/disks/DisksApp.h b/programs/disks/DisksApp.h index fad597335f0..ff05a5002e4 100644 --- a/programs/disks/DisksApp.h +++ b/programs/disks/DisksApp.h @@ -38,12 +38,16 @@ public: static void parseAndCheckOptions( const std::vector & arguments, const ProgramOptionsDescription & options_description, CommandLineOptions & options); - void printEntryHelpMessage(); - void printAvailableCommandsHelpMessage(); + void printEntryHelpMessage() const; + void printAvailableCommandsHelpMessage() const; + void printCommandHelpMessage(String command_name) const; + void printCommandHelpMessage(CommandPtr command) const; + String getCommandLineWithAliases(CommandPtr command) const; + std::vector getCompletions(const String & prefix) const; - std::vector getEmptyCompletion(CommandPtr command_) const; + std::vector getEmptyCompletion(String command_name) const; ~DisksApp() override; @@ -52,6 +56,8 @@ private: String getDefaultConfigFileName(); + std::vector getCommandsToComplete(const String & command_prefix) const; + // Fields responsible for the REPL work String history_file; LineReader::Suggest suggest; @@ -79,7 +85,10 @@ private: {"delete", "remove"}, {"ls-disks", "list-disks"}, {"ls_disks", "list-disks"}, - {"packed_io", "packed-io"}}; + {"packed_io", "packed-io"}, + {"change-dir", "cd"}, + {"change_dir", "cd"}, + {"switch_disk", "switch-disk"}}; std::set multidisk_commands = {"copy", "packed-io", "switch-disk", "cd"}; diff --git a/programs/disks/ICommand.h b/programs/disks/ICommand.h index d726d50ba13..97013717784 100644 --- a/programs/disks/ICommand.h +++ b/programs/disks/ICommand.h @@ -121,6 +121,7 @@ DB::CommandPtr makeCommandRemove(); DB::CommandPtr makeCommandWrite(); DB::CommandPtr makeCommandMkDir(); DB::CommandPtr makeCommandSwitchDisk(); +DB::CommandPtr makeCommandHelp(const DisksApp & disks_app); #ifdef CLICKHOUSE_CLOUD DB::CommandPtr makeCommandPackedIO(); #endif From a357e228a28d988d4781fcee0c7c9683721df40d Mon Sep 17 00:00:00 2001 From: Nikita Taranov Date: Tue, 11 Jun 2024 00:39:09 +0100 Subject: [PATCH 048/417] fix test --- .../0_stateless/02151_hash_table_sizes_stats_joins.sh | 10 ++++------ 1 file changed, 4 insertions(+), 6 deletions(-) diff --git a/tests/queries/0_stateless/02151_hash_table_sizes_stats_joins.sh b/tests/queries/0_stateless/02151_hash_table_sizes_stats_joins.sh index fb0e4315652..6d715604d93 100755 --- a/tests/queries/0_stateless/02151_hash_table_sizes_stats_joins.sh +++ b/tests/queries/0_stateless/02151_hash_table_sizes_stats_joins.sh @@ -58,19 +58,17 @@ $CLICKHOUSE_CLIENT -q "SYSTEM FLUSH LOGS" for i in "${!queries_without_preallocation[@]}"; do $CLICKHOUSE_CLIENT --param_query_id="${queries_without_preallocation[$i]}" -q " -- the old analyzer is not supported - SELECT COUNT(*) * getSetting('allow_experimental_analyzer') + SELECT sum(if(getSetting('allow_experimental_analyzer'), ProfileEvents['HashJoinPreallocatedElementsInHashTables'] = 0, 1)) FROM system.query_log - WHERE event_date >= yesterday() AND query_id = {query_id:String} AND current_database = currentDatabase() - AND ProfileEvents['HashJoinPreallocatedElementsInHashTables'] = 0 AND type = 'QueryFinish' + WHERE event_date >= yesterday() AND query_id = {query_id:String} AND current_database = currentDatabase() AND type = 'QueryFinish' " done for i in "${!queries_with_preallocation[@]}"; do $CLICKHOUSE_CLIENT --param_query_id="${queries_with_preallocation[$i]}" -q " -- the old analyzer is not supported - SELECT COUNT(*) * getSetting('allow_experimental_analyzer') + SELECT sum(if(getSetting('allow_experimental_analyzer'), ProfileEvents['HashJoinPreallocatedElementsInHashTables'] > 0, 1)) FROM system.query_log - WHERE event_date >= yesterday() AND query_id = {query_id:String} AND current_database = currentDatabase() - AND ProfileEvents['HashJoinPreallocatedElementsInHashTables'] > 0 AND type = 'QueryFinish' + WHERE event_date >= yesterday() AND query_id = {query_id:String} AND current_database = currentDatabase() AND type = 'QueryFinish' " done From ada56bdeeb46a903dbb4756bd61cc213b2d4b5dd Mon Sep 17 00:00:00 2001 From: Anton Popov Date: Tue, 11 Jun 2024 13:13:56 +0000 Subject: [PATCH 049/417] fix test --- tests/queries/0_stateless/03168_read_in_order_buffering_2.sql | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/tests/queries/0_stateless/03168_read_in_order_buffering_2.sql b/tests/queries/0_stateless/03168_read_in_order_buffering_2.sql index a706965f3ff..7ce07a55d5d 100644 --- a/tests/queries/0_stateless/03168_read_in_order_buffering_2.sql +++ b/tests/queries/0_stateless/03168_read_in_order_buffering_2.sql @@ -1,4 +1,4 @@ --- Tags: long, no-random-settings, no-tsan, no-asan, no-msan +-- Tags: long, no-random-settings, no-tsan, no-asan, no-msan, no-s3-storage DROP TABLE IF EXISTS t_read_in_order_2; From 8af89e6e6d919cf4f0c1eb4a5372ab49dfd9b144 Mon Sep 17 00:00:00 2001 From: Konstantin Morozov Date: Thu, 13 Jun 2024 13:22:25 +0000 Subject: [PATCH 050/417] apply comments --- src/Databases/DatabaseAtomic.cpp | 2 +- src/Storages/StorageReplicatedMergeTree.cpp | 12 +++++------- 2 files changed, 6 insertions(+), 8 deletions(-) diff --git a/src/Databases/DatabaseAtomic.cpp b/src/Databases/DatabaseAtomic.cpp index d431eb5c1b7..b30b05bb7a7 100644 --- a/src/Databases/DatabaseAtomic.cpp +++ b/src/Databases/DatabaseAtomic.cpp @@ -116,7 +116,7 @@ StoragePtr DatabaseAtomic::detachTable(ContextPtr /* context */, const String & if (!not_in_use.empty()) { not_in_use.clear(); - LOG_DEBUG(log, "Finish removing non using detached tables"); + LOG_DEBUG(log, "Finished removing not used detached tables"); } return table; diff --git a/src/Storages/StorageReplicatedMergeTree.cpp b/src/Storages/StorageReplicatedMergeTree.cpp index 68bb5916d7c..9b914e3de8f 100644 --- a/src/Storages/StorageReplicatedMergeTree.cpp +++ b/src/Storages/StorageReplicatedMergeTree.cpp @@ -5,17 +5,15 @@ #include #include -#include -#include #include #include #include #include -#include #include #include #include #include +#include #include #include #include @@ -5287,7 +5285,7 @@ void StorageReplicatedMergeTree::flushAndPrepareForShutdown() if (attach_thread) { attach_thread->shutdown(); - LOG_TRACE(log, "Attach thread shutdowned"); + LOG_TRACE(log, "The attach thread is shutdown"); } @@ -5303,7 +5301,7 @@ void StorageReplicatedMergeTree::flushAndPrepareForShutdown() throw; } - LOG_TRACE(log, "Finish preparing for shutdown"); + LOG_TRACE(log, "Finished preparing for shutdown"); } void StorageReplicatedMergeTree::partialShutdown() @@ -5342,7 +5340,7 @@ void StorageReplicatedMergeTree::shutdown(bool) return; const auto storage_name = getStorageID().getNameForLogs(); - LOG_TRACE(log, "Shutdown started, table={}", storage_name); + LOG_TRACE(log, "Shutdown started"); flushAndPrepareForShutdown(); @@ -5386,7 +5384,7 @@ void StorageReplicatedMergeTree::shutdown(bool) /// Wait for all of them std::lock_guard lock(data_parts_exchange_ptr->rwlock); } - LOG_TRACE(log, "Shutdown finished, table={}", storage_name); + LOG_TRACE(log, "Shutdown finished"); } From f7eac01b822c94184a16dfda1685d95f05c5cc8a Mon Sep 17 00:00:00 2001 From: Konstantin Morozov Date: Thu, 13 Jun 2024 13:31:52 +0000 Subject: [PATCH 051/417] up includes --- src/Storages/StorageReplicatedMergeTree.cpp | 2 ++ 1 file changed, 2 insertions(+) diff --git a/src/Storages/StorageReplicatedMergeTree.cpp b/src/Storages/StorageReplicatedMergeTree.cpp index b33514907f9..a1f4a40a0ab 100644 --- a/src/Storages/StorageReplicatedMergeTree.cpp +++ b/src/Storages/StorageReplicatedMergeTree.cpp @@ -5,10 +5,12 @@ #include #include +#include #include #include #include #include +#include #include #include #include From b8992f039786c822876c075262d0a4ac9e3962df Mon Sep 17 00:00:00 2001 From: divanik Date: Thu, 13 Jun 2024 19:14:16 +0000 Subject: [PATCH 052/417] Create integration test --- programs/disks/CMakeLists.txt | 3 +- programs/disks/CommandChangeDirectory.cpp | 2 +- .../disks/CommandGetCurrentDiskAndPath.cpp | 30 + programs/disks/CommandHelp.cpp | 3 +- programs/disks/CommandRead.cpp | 1 - programs/disks/DisksApp.cpp | 32 +- programs/disks/DisksApp.h | 8 +- programs/disks/DisksClient.cpp | 8 +- programs/disks/ICommand.h | 1 + src/Disks/DiskLocal.cpp | 1 + .../configs/config.xml | 1659 +++++++++++++++++ .../configs/users.xml | 120 ++ .../test_disks_app_interactive/test.py | 321 ++++ 13 files changed, 2181 insertions(+), 8 deletions(-) create mode 100644 programs/disks/CommandGetCurrentDiskAndPath.cpp create mode 100644 tests/integration/test_disks_app_interactive/configs/config.xml create mode 100644 tests/integration/test_disks_app_interactive/configs/users.xml create mode 100644 tests/integration/test_disks_app_interactive/test.py diff --git a/programs/disks/CMakeLists.txt b/programs/disks/CMakeLists.txt index 2bf17a352e6..40f9cf3401c 100644 --- a/programs/disks/CMakeLists.txt +++ b/programs/disks/CMakeLists.txt @@ -13,7 +13,8 @@ set (CLICKHOUSE_DISKS_SOURCES CommandRemove.cpp CommandSwitchDisk.cpp CommandWrite.cpp - CommandHelp.cpp) + CommandHelp.cpp + CommandGetCurrentDiskAndPath.cpp) if (CLICKHOUSE_CLOUD) set (CLICKHOUSE_DISKS_SOURCES ${CLICKHOUSE_DISKS_SOURCES} CommandPackedIO.cpp) diff --git a/programs/disks/CommandChangeDirectory.cpp b/programs/disks/CommandChangeDirectory.cpp index 5e6a08cd3fd..5c4ce737375 100644 --- a/programs/disks/CommandChangeDirectory.cpp +++ b/programs/disks/CommandChangeDirectory.cpp @@ -15,7 +15,7 @@ public: command_name = "cd"; description = "Change directory"; options_description.add_options()("path", po::value(), "the path we want to get to (mandatory, positional)")( - "disk", po::value(), "A disk where the path is changed"); + "disk", po::value(), "A disk where the path is changed (without disk switching)"); positional_options_description.add("path", 1); } diff --git a/programs/disks/CommandGetCurrentDiskAndPath.cpp b/programs/disks/CommandGetCurrentDiskAndPath.cpp new file mode 100644 index 00000000000..2ad3525eb19 --- /dev/null +++ b/programs/disks/CommandGetCurrentDiskAndPath.cpp @@ -0,0 +1,30 @@ +#include +#include +#include "DisksApp.h" +#include "DisksClient.h" +#include "ICommand.h" + +namespace DB +{ + +class CommandGetCurrentDiskAndPath final : public ICommand +{ +public: + explicit CommandGetCurrentDiskAndPath() : ICommand() + { + command_name = "current_disk_with_path"; + description = "Prints current disk and path (which coincide with prompt)"; + } + + void executeImpl(const CommandLineOptions &, DisksClient & client) override + { + auto disk = client.getCurrentDiskWithPath(); + std::cout << "Disk: " << disk.getDisk()->getName() << "\nPath: " << disk.getCurrentPath() << std::endl; + } +}; + +CommandPtr makeCommandGetCurrentDiskAndPath() +{ + return std::make_shared(); +} +} diff --git a/programs/disks/CommandHelp.cpp b/programs/disks/CommandHelp.cpp index becdae324b3..6f7e79a352e 100644 --- a/programs/disks/CommandHelp.cpp +++ b/programs/disks/CommandHelp.cpp @@ -14,7 +14,8 @@ public: { command_name = "help"; description = "Print help message about available commands (all or only required)"; - options_description.add_options()("command", po::value(), "A command to help with"); + options_description.add_options()( + "command", po::value(), "A command to help with (optional, positional), if not specified, help lists all the commands"); positional_options_description.add("command", 1); } diff --git a/programs/disks/CommandRead.cpp b/programs/disks/CommandRead.cpp index ea05d25fb44..9f60cca2873 100644 --- a/programs/disks/CommandRead.cpp +++ b/programs/disks/CommandRead.cpp @@ -24,7 +24,6 @@ public: { auto disk = client.getCurrentDiskWithPath(); String path_from = disk.getRelativeFromRoot(getValueFromCommandLineOptionsThrow(options, "path-from")); - std::cerr << path_from << std::endl; std::optional path_to = getValueFromCommandLineOptionsWithOptional(options, "path-to"); auto in = disk.getDisk()->readFile(path_from); diff --git a/programs/disks/DisksApp.cpp b/programs/disks/DisksApp.cpp index 296567c4b35..2fe490e22ff 100644 --- a/programs/disks/DisksApp.cpp +++ b/programs/disks/DisksApp.cpp @@ -192,6 +192,10 @@ std::vector DisksApp::getCompletions(const String & prefix) const bool DisksApp::processQueryText(const String & text) { + if (text.find_first_not_of(word_break_characters) == std::string::npos) + { + return true; + } if (exit_strings.find(text) != exit_strings.end()) return false; CommandPtr command; @@ -275,7 +279,8 @@ void DisksApp::addOptions() { options_description.add_options()("help,h", "Print common help message")("config-file,C", po::value(), "Set config file")( "disk", po::value(), "Set disk name")("save-logs", "Save logs to a file")( - "log-level", po::value(), "Logging level")("query,q", po::value(), "Query for a non-interactive mode"); + "log-level", po::value(), "Logging level")("query,q", po::value(), "Query for a non-interactive mode")( + "test-mode", "Interface in test regyme"); command_descriptions.emplace("list-disks", makeCommandListDisks()); command_descriptions.emplace("copy", makeCommandCopy()); @@ -288,6 +293,7 @@ void DisksApp::addOptions() command_descriptions.emplace("read", makeCommandRead()); command_descriptions.emplace("mkdir", makeCommandMkDir()); command_descriptions.emplace("switch-disk", makeCommandSwitchDisk()); + command_descriptions.emplace("current_disk_with_path", makeCommandGetCurrentDiskAndPath()); command_descriptions.emplace("help", makeCommandHelp(*this)); #ifdef CLICKHOUSE_CLOUD command_descriptions.emplace("packed-io", makeCommandPackedIO()); @@ -311,6 +317,8 @@ void DisksApp::processOptions() config().setBool("save-logs", true); if (options.count("log-level")) config().setString("log-level", options["log-level"].as()); + if (options.count("test-mode")) + config().setBool("test-mode", true); if (options.count("query")) query = std::optional{options["query"].as()}; } @@ -492,7 +500,7 @@ int DisksApp::main(const std::vector & /*args*/) if (!query.has_value()) { - runInteractiveReplxx(); + runInteractive(); } else { @@ -507,6 +515,26 @@ DisksApp::~DisksApp() if (global_context) global_context->shutdown(); } + +void DisksApp::runInteractiveTestMode() +{ + for (String input; std::getline(std::cin, input);) + { + if (!processQueryText(input)) + break; + + std::cout << "\a\a\a\a" << std::endl; + std::cerr << std::flush; + } +} + +void DisksApp::runInteractive() +{ + if (config().hasOption("test-mode")) + runInteractiveTestMode(); + else + runInteractiveReplxx(); +} } int mainEntryClickHouseDisks(int argc, char ** argv) diff --git a/programs/disks/DisksApp.h b/programs/disks/DisksApp.h index ff05a5002e4..75d604bf63c 100644 --- a/programs/disks/DisksApp.h +++ b/programs/disks/DisksApp.h @@ -52,7 +52,9 @@ public: ~DisksApp() override; private: + void runInteractive(); void runInteractiveReplxx(); + void runInteractiveTestMode(); String getDefaultConfigFileName(); @@ -88,7 +90,11 @@ private: {"packed_io", "packed-io"}, {"change-dir", "cd"}, {"change_dir", "cd"}, - {"switch_disk", "switch-disk"}}; + {"switch_disk", "switch-disk"}, + {"current", "current_disk_with_path"}, + {"current_disk", "current_disk_with_path"}, + {"current_path", "current_disk_with_path"}, + {"cur", "current_disk_with_path"}}; std::set multidisk_commands = {"copy", "packed-io", "switch-disk", "cd"}; diff --git a/programs/disks/DisksClient.cpp b/programs/disks/DisksClient.cpp index 40b458fd7b3..4f808f85ab6 100644 --- a/programs/disks/DisksClient.cpp +++ b/programs/disks/DisksClient.cpp @@ -29,9 +29,15 @@ DiskWithPath::DiskWithPath(DiskPtr disk_, std::optional path_) : disk(di { path = String{"/"}; } + if (!disk->isDirectory(normalizePathAndGetAsRelative(path))) { - throw Exception(ErrorCodes::BAD_ARGUMENTS, "Initializing path {} at disk {} is not a directory", path, disk->getName()); + throw Exception( + ErrorCodes::BAD_ARGUMENTS, + "Initializing path {} (normalized path: {}) at disk {} is not a directory", + path, + normalizePathAndGetAsRelative(path), + disk->getName()); } } diff --git a/programs/disks/ICommand.h b/programs/disks/ICommand.h index 97013717784..dac614808d0 100644 --- a/programs/disks/ICommand.h +++ b/programs/disks/ICommand.h @@ -121,6 +121,7 @@ DB::CommandPtr makeCommandRemove(); DB::CommandPtr makeCommandWrite(); DB::CommandPtr makeCommandMkDir(); DB::CommandPtr makeCommandSwitchDisk(); +DB::CommandPtr makeCommandGetCurrentDiskAndPath(); DB::CommandPtr makeCommandHelp(const DisksApp & disks_app); #ifdef CLICKHOUSE_CLOUD DB::CommandPtr makeCommandPackedIO(); diff --git a/src/Disks/DiskLocal.cpp b/src/Disks/DiskLocal.cpp index d1f0a928b1d..6cb2599b82a 100644 --- a/src/Disks/DiskLocal.cpp +++ b/src/Disks/DiskLocal.cpp @@ -276,6 +276,7 @@ bool DiskLocal::isFile(const String & path) const bool DiskLocal::isDirectory(const String & path) const { + // std::cerr << fs::path(disk_path) / path << std::endl; return fs::is_directory(fs::path(disk_path) / path); } diff --git a/tests/integration/test_disks_app_interactive/configs/config.xml b/tests/integration/test_disks_app_interactive/configs/config.xml new file mode 100644 index 00000000000..5db40531f13 --- /dev/null +++ b/tests/integration/test_disks_app_interactive/configs/config.xml @@ -0,0 +1,1659 @@ + + + + + trace + /var/log/clickhouse-server/clickhouse-server.log + /var/log/clickhouse-server/clickhouse-server.err.log + + 1000M + 10 + + + + + + + + + + + + + + https://{bucket}.s3.amazonaws.com + + + https://storage.googleapis.com/{bucket} + + + https://{bucket}.oss.aliyuncs.com + + + + + +
+ Access-Control-Allow-Origin + * +
+
+ Access-Control-Allow-Headers + origin, x-requested-with, x-clickhouse-format, x-clickhouse-user, x-clickhouse-key, Authorization +
+
+ Access-Control-Allow-Methods + POST, GET, OPTIONS +
+
+ Access-Control-Max-Age + 86400 +
+
+ + + + + + 8123 + + + 9000 + + + 9004 + + + 9005 + + + + + + + + + + + + 9009 + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + 4096 + + + 10 + + + + + false + + + /path/to/ssl_cert_file + /path/to/ssl_key_file + + + false + + + /path/to/ssl_ca_cert_file + + + none + + + 0 + + + -1 + -1 + + + false + + + + + + + + + + none + true + true + sslv2,sslv3 + true + + + + RejectCertificateHandler + + + + + true + true + sslv2,sslv3 + true + + + + RejectCertificateHandler + + + + + + + + + 0 + 2 + + + 1000 + + + 0 + + + + 10000 + + + + + + + + + 0.9 + + + 4194304 + + + 0 + + + + + + 8589934592 + + + 5368709120 + + + 5368709120 + + + 1000 + + + 134217728 + + + 10000 + + + /home/ubuntu/work/clickdb/cache/ + + false + + + /var/lib/clickhouse/ + + + + + + + /var/lib/clickhouse/tmp/ + + + 1 + 1 + 1 + + + sha256_password + + + 12 + + + + + + + + + /var/lib/clickhouse/user_files/ + + + + + + + + + + + + + users.xml + + + + /var/lib/clickhouse/access/ + + + + + + + + true + + + true + + + true + + + true + + + true + + + false + + + 600 + + + + default + + + SQL_ + + + + + + + + + default + + + + + + + + + true + + + false + + ' | sed -e 's|.*>\(.*\)<.*|\1|') + wget https://github.com/ClickHouse/clickhouse-jdbc-bridge/releases/download/v$PKG_VER/clickhouse-jdbc-bridge_$PKG_VER-1_all.deb + apt install --no-install-recommends -f ./clickhouse-jdbc-bridge_$PKG_VER-1_all.deb + clickhouse-jdbc-bridge & + + * [CentOS/RHEL] + export MVN_URL=https://repo1.maven.org/maven2/com/clickhouse/clickhouse-jdbc-bridge/ + export PKG_VER=$(curl -sL $MVN_URL/maven-metadata.xml | grep '' | sed -e 's|.*>\(.*\)<.*|\1|') + wget https://github.com/ClickHouse/clickhouse-jdbc-bridge/releases/download/v$PKG_VER/clickhouse-jdbc-bridge-$PKG_VER-1.noarch.rpm + yum localinstall -y clickhouse-jdbc-bridge-$PKG_VER-1.noarch.rpm + clickhouse-jdbc-bridge & + + Please refer to https://github.com/ClickHouse/clickhouse-jdbc-bridge#usage for more information. + ]]> + + + + + + + + + + + + + + + + localhost + 9000 + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + 3600 + + + + 3600 + + + 60 + + + + + + + + + + + + + system + query_log
+ + toYYYYMM(event_date) + + + + + + + + 7500 + + 1048576 + + 8192 + + 524288 + + false + + + +
+ + + + system + trace_log
+ + toYYYYMM(event_date) + 7500 + 1048576 + 8192 + 524288 + + false +
+ + + + system + query_thread_log
+ toYYYYMM(event_date) + 7500 + 1048576 + 8192 + 524288 + false +
+ + + + system + query_views_log
+ toYYYYMM(event_date) + 7500 +
+ + + + system + part_log
+ toYYYYMM(event_date) + 7500 + 1048576 + 8192 + 524288 + false +
+ + + + + + system + metric_log
+ 7500 + 1048576 + 8192 + 524288 + 1000 + false +
+ + + + system + asynchronous_metric_log
+ 7000 + 1048576 + 8192 + 524288 + false +
+ + + + + + engine MergeTree + partition by toYYYYMM(finish_date) + order by (finish_date, finish_time_us, trace_id) + + system + opentelemetry_span_log
+ 7500 + 1048576 + 8192 + 524288 + false +
+ + + + + system + crash_log
+ + + 1000 + 1024 + 1024 + 512 + true +
+ + + + + + + system + processors_profile_log
+ + toYYYYMM(event_date) + 7500 + 1048576 + 8192 + 524288 + false +
+ + + + system + asynchronous_insert_log
+ + 7500 + 1048576 + 8192 + 524288 + false + event_date + event_date + INTERVAL 3 DAY +
+ + + + system + backup_log
+ toYYYYMM(event_date) + 7500 +
+ + + + system + s3queue_log
+ toYYYYMM(event_date) + 7500 +
+ + + + system + blob_storage_log
+ toYYYYMM(event_date) + 7500 + event_date + INTERVAL 30 DAY +
+ + + + + + + + + *_dictionary.*ml + + + true + + + true + + + *_function.*ml + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + /clickhouse/task_queue/ddl + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + click_cost + any + + 0 + 3600 + + + 86400 + 60 + + + + max + + 0 + 60 + + + 3600 + 300 + + + 86400 + 3600 + + + + + + /var/lib/clickhouse/format_schemas/ + + + /usr/share/clickhouse/protos/ + + + + + + + + + + false + + false + + + https://6f33034cfe684dd7a3ab9875e57b1c8d@o388870.ingest.sentry.io/5226277 + + false + + + + + + + + + + + + + + + + + + + + + + + + + + 1073741824 + 1024 + 1048576 + 30000000 + + + + backups + + + true + + + + + + + + + + + +
\ No newline at end of file diff --git a/tests/integration/test_disks_app_interactive/configs/users.xml b/tests/integration/test_disks_app_interactive/configs/users.xml new file mode 100644 index 00000000000..57bc6309a54 --- /dev/null +++ b/tests/integration/test_disks_app_interactive/configs/users.xml @@ -0,0 +1,120 @@ + + + + + + + + + + + + 1 + + + + + + + + + + + + + ::/0 + + + + default + + + default + + + 1 + + + 1 + + + + + + + + + + + + + + 3600 + + + 0 + 0 + 0 + 0 + 0 + + + + diff --git a/tests/integration/test_disks_app_interactive/test.py b/tests/integration/test_disks_app_interactive/test.py new file mode 100644 index 00000000000..4ecd8639104 --- /dev/null +++ b/tests/integration/test_disks_app_interactive/test.py @@ -0,0 +1,321 @@ +from helpers.cluster import ClickHouseCluster + +import pytest + +import pathlib + +# import os + +# import grpc +# import pymysql.connections +# import psycopg2 as py_psql +# import sys +# import threading + +# from helpers.cluster import ClickHouseCluster, run_and_check +# from helpers.test_tools import assert_logs_contain_with_retry +import subprocess +import select +import io +from typing import List, Tuple, Dict, Union, Optional + +import os + + +class ClickHouseDisksException(Exception): + pass + + +class LocalDisksClient(object): + SEPARATOR = b"\a\a\a\a\n" + client: Optional["LocalDisksClient"] = None # static variable + default_disk_root_directory: str = "/var/lib/clickhouse" + + def __init__(self, bin_path: str, config_path: str, working_path: str): + self.bin_path = bin_path + self.working_path = working_path + + self.proc = subprocess.Popen( + [bin_path, "disks", "--test-mode", "--config", config_path], + stdin=subprocess.PIPE, + stdout=subprocess.PIPE, + stderr=subprocess.PIPE, + ) + + self.poller = select.epoll() + self.poller.register(self.proc.stdout) + self.poller.register(self.proc.stderr) + + self.stopped = False + + self._fd_nums = { + self.proc.stdout.fileno(): self.proc.stdout, + self.proc.stderr.fileno(): self.proc.stderr, + } + + def execute_query(self, query: str, timeout: float = 5.0) -> str: + output = io.BytesIO() + + self.proc.stdin.write(query.encode() + b"\n") + self.proc.stdin.flush() + + events = self.poller.poll(timeout) + if not events: + raise TimeoutError(f"Disks client returned no output") + + for fd_num, event in events: + if event & (select.EPOLLIN | select.EPOLLPRI): + file = self._fd_nums[fd_num] + + if file == self.proc.stdout: + while True: + chunk = file.readline() + if chunk.endswith(self.SEPARATOR): + break + + output.write(chunk) + + elif file == self.proc.stderr: + error_line = self.proc.stderr.readline() + print(error_line) + raise ClickHouseDisksException(error_line.strip().decode()) + + else: + raise ValueError(f"Failed to read from pipe. Flag {event}") + + data = output.getvalue().strip().decode() + return data + + def list_disks(self) -> List[Tuple[str, str]]: + output = self.execute_query("list-disks") + return list( + sorted( + map( + lambda x: (x.split(":")[0], ":".join(x.split(":")[1:])), + output.split("\n"), + ) + ) + ) + + def current_disk_with_path(self) -> Tuple[str, str]: + output = self.execute_query("current_disk_with_path") + disk_line = output.split("\n")[0] + path_line = output.split("\n")[1] + assert disk_line.startswith("Disk: ") + assert path_line.startswith("Path: ") + return disk_line[6:], path_line[6:] + + def ls( + self, path: str, recursive: bool = False, show_hidden: bool = False + ) -> Union[List[str], Dict[str, List[str]]]: + recursive_adding = "--recursive " if recursive else "" + show_hidden_adding = "--all " if show_hidden else "" + output = self.execute_query( + f"list {path} {recursive_adding} {show_hidden_adding}" + ) + if recursive: + answer: Dict[str, List[str]] = dict() + blocks = output.split("\n\n") + for block in blocks: + directory = block.split("\n")[0][:-1] + files = block.split("\n")[1:] + answer[directory] = files + return answer + else: + return output.split("\n") + + def switch_disk(self, disk: str, directory: Optional[str] = None): + directory_addition = f"--path {directory} " if directory is not None else "" + self.execute_query(f"switch-disk {disk} {directory_addition}") + + def cd(self, directory: str, disk: Optional[str] = None): + disk_addition = f"--disk {disk} " if disk is not None else "" + self.execute_query(f"cd {directory} {disk_addition}") + + def copy( + self, + path_from, + path_to, + disk_from: Optional[str] = None, + disk_to: Optional[str] = None, + ): + disk_from_option = f"--disk-from {disk_from} " if disk_from is not None else "" + disk_to_option = f"--disk-to {disk_to} " if disk_to is not None else "" + self.execute_query( + f"copy {path_from} {path_to} {disk_from_option} {disk_to_option}" + ) + + def move(self, path_from: str, path_to: str): + self.execute_query(f"move {path_from} {path_to}") + + def rm(self, path: str): + self.execute_query(f"rm {path}") + + def mkdir(self, path: str, recursive: bool = False): + recursive_adding = "--recursive " if recursive else "" + self.execute_query(f"mkdir {path} {recursive_adding}") + + def ln(self, path_from: str, path_to: str): + self.execute_query(f"link {path_from} {path_to}") + + def read(self, path_from: str, path_to: Optional[str] = None): + path_to_adding = f"--path-to {path_to} " if path_to is not None else "" + output = self.execute_query(f"read {path_from} {path_to_adding}") + return output + + def write( + self, path_from: str, path_to: str + ): # Writing from stdin is difficult to test (do not know how to do this in python) + path_from_adding = f"--path-from {path_from}" + self.execute_query(f"write {path_from_adding} {path_to}") + + @staticmethod + def getClient(refresh: bool): + if (LocalDisksClient.client is None) or refresh: + binary_file = os.environ.get("CLICKHOUSE_TESTS_SERVER_BIN_PATH") + current_working_directory = str(pathlib.Path().resolve()) + config_file = f"{current_working_directory}/test_disks_app_interactive/configs/config.xml" + if not os.path.exists(LocalDisksClient.default_disk_root_directory): + os.mkdir(LocalDisksClient.default_disk_root_directory) + + LocalDisksClient.client = LocalDisksClient( + binary_file, config_file, current_working_directory + ) + return LocalDisksClient.client + else: + return LocalDisksClient.client + + +def test_disks_app_interactive_list_disks(): + client = LocalDisksClient.getClient(True) + expected_disks_with_path = [ + ("default", "/"), + ("local", client.working_path), + ] + assert expected_disks_with_path == client.list_disks() + assert client.current_disk_with_path() == ("default", "/") + client.switch_disk("local") + assert client.current_disk_with_path() == ( + "local", + client.working_path, + ) + + +def test_disks_app_interactive_list_files_local(): + client = LocalDisksClient.getClient(True) + client.switch_disk("local") + excepted_listed_files = sorted(os.listdir("test_disks_app_interactive/")) + listed_files = sorted(client.ls("test_disks_app_interactive/")) + assert excepted_listed_files == listed_files + + +def test_disks_app_interactive_list_directories_default(): + client = LocalDisksClient.getClient(True) + traversed_dir = client.ls(".", recursive=True) + client.mkdir("dir1") + client.mkdir("dir2") + client.mkdir(".dir3") + client.cd("dir1") + client.mkdir("dir11") + client.mkdir(".dir12") + client.mkdir("dir13") + client.cd("../dir2") + client.mkdir("dir21") + client.mkdir("dir22") + client.mkdir(".dir23") + client.cd("../.dir3") + client.mkdir("dir31") + client.mkdir(".dir32") + client.cd("..") + traversed_dir = client.ls(".", recursive=True) + assert traversed_dir == { + ".": ["dir1", "dir2"], + "./dir1": ["dir11", "dir13"], + "./dir2": ["dir21", "dir22"], + "./dir1/dir11": [], + "./dir1/dir13": [], + "./dir2/dir21": [], + "./dir2/dir22": [], + } + traversed_dir = client.ls(".", recursive=True, show_hidden=True) + assert traversed_dir == { + ".": [".dir3", "dir1", "dir2"], + "./dir1": [".dir12", "dir11", "dir13"], + "./dir2": [".dir23", "dir21", "dir22"], + "./.dir3": [".dir32", "dir31"], + "./dir1/dir11": [], + "./dir1/.dir12": [], + "./dir1/dir13": [], + "./dir2/dir21": [], + "./dir2/dir22": [], + "./dir2/.dir23": [], + "./.dir3/dir31": [], + "./.dir3/.dir32": [], + } + client.rm("dir2") + traversed_dir = client.ls(".", recursive=True, show_hidden=True) + assert traversed_dir == { + ".": [".dir3", "dir1"], + "./dir1": [".dir12", "dir11", "dir13"], + "./.dir3": [".dir32", "dir31"], + "./dir1/dir11": [], + "./dir1/.dir12": [], + "./dir1/dir13": [], + "./.dir3/dir31": [], + "./.dir3/.dir32": [], + } + traversed_dir = client.ls(".", recursive=True, show_hidden=False) + assert traversed_dir == { + ".": ["dir1"], + "./dir1": ["dir11", "dir13"], + "./dir1/dir11": [], + "./dir1/dir13": [], + } + client.rm("dir1") + client.rm(".dir3") + assert client.ls(".", recursive=True, show_hidden=False) == {".": []} + + +def test_disks_app_interactive_cp_and_read(): + initial_text = "File content" + with open("a.txt", "w") as file: + file.write(initial_text) + client = LocalDisksClient.getClient(True) + client.switch_disk("default") + client.copy("a.txt", "/a.txt", disk_from="local", disk_to="default") + read_text = client.read("a.txt") + assert initial_text == read_text + client.mkdir("dir1") + client.copy("a.txt", "/dir1/b.txt", disk_from="local", disk_to="default") + read_text = client.read("a.txt", path_to="dir1/b.txt") + assert "" == read_text + read_text = client.read("/dir1/b.txt") + assert read_text == initial_text + with open( + f"{LocalDisksClient.default_disk_root_directory}/dir1/b.txt", "r" + ) as file: + read_text = file.read() + assert read_text == initial_text + os.remove("a.txt") + client.rm("a.txt") + client.rm("/dir1") + + +def test_disks_app_interactive_test_move_and_write(): + initial_text = "File content" + with open("a.txt", "w") as file: + file.write(initial_text) + client = LocalDisksClient.getClient(True) + client.switch_disk("default") + client.copy("a.txt", "/a.txt", disk_from="local", disk_to="default") + files = client.ls(".") + assert files == ["a.txt"] + client.move("a.txt", "b.txt") + files = client.ls(".") + assert files == ["b.txt"] + read_text = client.read("/b.txt") + assert read_text == initial_text + client.write("b.txt", "c.txt") + read_text = client.read("c.txt") + assert read_text == initial_text + os.remove("a.txt") From 2824ca64e0d41c7401c72d84c4a001d680ca78fd Mon Sep 17 00:00:00 2001 From: divanik Date: Fri, 14 Jun 2024 11:06:56 +0000 Subject: [PATCH 053/417] Tests and code style --- programs/disks/DisksApp.cpp | 4 +- programs/disks/ICommand.cpp | 2 +- programs/disks/ICommand.h | 2 +- tests/integration/test_disks_app_func/test.py | 4 +- .../configs/config.xml | 1656 ----------------- .../configs/users.xml | 120 -- .../test_disks_app_interactive/test.py | 60 +- 7 files changed, 40 insertions(+), 1808 deletions(-) delete mode 100644 tests/integration/test_disks_app_interactive/configs/users.xml diff --git a/programs/disks/DisksApp.cpp b/programs/disks/DisksApp.cpp index 2fe490e22ff..7f657dd32a3 100644 --- a/programs/disks/DisksApp.cpp +++ b/programs/disks/DisksApp.cpp @@ -23,8 +23,8 @@ namespace DB namespace ErrorCodes { -extern const int BAD_ARGUMENTS; -extern const int LOGICAL_ERROR; + extern const int BAD_ARGUMENTS; + extern const int LOGICAL_ERROR; }; LineReader::Patterns DisksApp::query_extenders = {"\\"}; diff --git a/programs/disks/ICommand.cpp b/programs/disks/ICommand.cpp index 41fa281794e..0c149a8f9df 100644 --- a/programs/disks/ICommand.cpp +++ b/programs/disks/ICommand.cpp @@ -7,7 +7,7 @@ namespace DB namespace ErrorCodes { -extern const int BAD_ARGUMENTS; + extern const int BAD_ARGUMENTS; } CommandLineOptions ICommand::processCommandLineArguments(const Strings & commands) diff --git a/programs/disks/ICommand.h b/programs/disks/ICommand.h index dac614808d0..b1e594066af 100644 --- a/programs/disks/ICommand.h +++ b/programs/disks/ICommand.h @@ -29,7 +29,7 @@ using CommandLineOptions = po::variables_map; namespace ErrorCodes { -extern const int BAD_ARGUMENTS; + extern const int BAD_ARGUMENTS; } class ICommand diff --git a/tests/integration/test_disks_app_func/test.py b/tests/integration/test_disks_app_func/test.py index d643230d198..ac1edae4199 100644 --- a/tests/integration/test_disks_app_func/test.py +++ b/tests/integration/test_disks_app_func/test.py @@ -9,7 +9,9 @@ def started_cluster(): try: cluster = ClickHouseCluster(__file__) cluster.add_instance( - "disks_app_test", main_configs=["config.xml"], with_minio=True + "disks_app_test", + main_configs=["server_configs/config.xml"], + with_minio=True, ) cluster.start() diff --git a/tests/integration/test_disks_app_interactive/configs/config.xml b/tests/integration/test_disks_app_interactive/configs/config.xml index 5db40531f13..bcbb107f0a2 100644 --- a/tests/integration/test_disks_app_interactive/configs/config.xml +++ b/tests/integration/test_disks_app_interactive/configs/config.xml @@ -1,1659 +1,3 @@ - - - - trace - /var/log/clickhouse-server/clickhouse-server.log - /var/log/clickhouse-server/clickhouse-server.err.log - - 1000M - 10 - - - - - - - - - - - - - - https://{bucket}.s3.amazonaws.com - - - https://storage.googleapis.com/{bucket} - - - https://{bucket}.oss.aliyuncs.com - - - - - -
- Access-Control-Allow-Origin - * -
-
- Access-Control-Allow-Headers - origin, x-requested-with, x-clickhouse-format, x-clickhouse-user, x-clickhouse-key, Authorization -
-
- Access-Control-Allow-Methods - POST, GET, OPTIONS -
-
- Access-Control-Max-Age - 86400 -
-
- - - - - - 8123 - - - 9000 - - - 9004 - - - 9005 - - - - - - - - - - - - 9009 - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - 4096 - - - 10 - - - - - false - - - /path/to/ssl_cert_file - /path/to/ssl_key_file - - - false - - - /path/to/ssl_ca_cert_file - - - none - - - 0 - - - -1 - -1 - - - false - - - - - - - - - - none - true - true - sslv2,sslv3 - true - - - - RejectCertificateHandler - - - - - true - true - sslv2,sslv3 - true - - - - RejectCertificateHandler - - - - - - - - - 0 - 2 - - - 1000 - - - 0 - - - - 10000 - - - - - - - - - 0.9 - - - 4194304 - - - 0 - - - - - - 8589934592 - - - 5368709120 - - - 5368709120 - - - 1000 - - - 134217728 - - - 10000 - - - /home/ubuntu/work/clickdb/cache/ - - false - - /var/lib/clickhouse/ - - - - - - - /var/lib/clickhouse/tmp/ - - - 1 - 1 - 1 - - - sha256_password - - - 12 - - - - - - - - - /var/lib/clickhouse/user_files/ - - - - - - - - - - - - - users.xml - - - - /var/lib/clickhouse/access/ - - - - - - - - true - - - true - - - true - - - true - - - true - - - false - - - 600 - - - - default - - - SQL_ - - - - - - - - - default - - - - - - - - - true - - - false - - ' | sed -e 's|.*>\(.*\)<.*|\1|') - wget https://github.com/ClickHouse/clickhouse-jdbc-bridge/releases/download/v$PKG_VER/clickhouse-jdbc-bridge_$PKG_VER-1_all.deb - apt install --no-install-recommends -f ./clickhouse-jdbc-bridge_$PKG_VER-1_all.deb - clickhouse-jdbc-bridge & - - * [CentOS/RHEL] - export MVN_URL=https://repo1.maven.org/maven2/com/clickhouse/clickhouse-jdbc-bridge/ - export PKG_VER=$(curl -sL $MVN_URL/maven-metadata.xml | grep '' | sed -e 's|.*>\(.*\)<.*|\1|') - wget https://github.com/ClickHouse/clickhouse-jdbc-bridge/releases/download/v$PKG_VER/clickhouse-jdbc-bridge-$PKG_VER-1.noarch.rpm - yum localinstall -y clickhouse-jdbc-bridge-$PKG_VER-1.noarch.rpm - clickhouse-jdbc-bridge & - - Please refer to https://github.com/ClickHouse/clickhouse-jdbc-bridge#usage for more information. - ]]> - - - - - - - - - - - - - - - - localhost - 9000 - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - 3600 - - - - 3600 - - - 60 - - - - - - - - - - - - - system - query_log
- - toYYYYMM(event_date) - - - - - - - - 7500 - - 1048576 - - 8192 - - 524288 - - false - - - -
- - - - system - trace_log
- - toYYYYMM(event_date) - 7500 - 1048576 - 8192 - 524288 - - false -
- - - - system - query_thread_log
- toYYYYMM(event_date) - 7500 - 1048576 - 8192 - 524288 - false -
- - - - system - query_views_log
- toYYYYMM(event_date) - 7500 -
- - - - system - part_log
- toYYYYMM(event_date) - 7500 - 1048576 - 8192 - 524288 - false -
- - - - - - system - metric_log
- 7500 - 1048576 - 8192 - 524288 - 1000 - false -
- - - - system - asynchronous_metric_log
- 7000 - 1048576 - 8192 - 524288 - false -
- - - - - - engine MergeTree - partition by toYYYYMM(finish_date) - order by (finish_date, finish_time_us, trace_id) - - system - opentelemetry_span_log
- 7500 - 1048576 - 8192 - 524288 - false -
- - - - - system - crash_log
- - - 1000 - 1024 - 1024 - 512 - true -
- - - - - - - system - processors_profile_log
- - toYYYYMM(event_date) - 7500 - 1048576 - 8192 - 524288 - false -
- - - - system - asynchronous_insert_log
- - 7500 - 1048576 - 8192 - 524288 - false - event_date - event_date + INTERVAL 3 DAY -
- - - - system - backup_log
- toYYYYMM(event_date) - 7500 -
- - - - system - s3queue_log
- toYYYYMM(event_date) - 7500 -
- - - - system - blob_storage_log
- toYYYYMM(event_date) - 7500 - event_date + INTERVAL 30 DAY -
- - - - - - - - - *_dictionary.*ml - - - true - - - true - - - *_function.*ml - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - /clickhouse/task_queue/ddl - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - click_cost - any - - 0 - 3600 - - - 86400 - 60 - - - - max - - 0 - 60 - - - 3600 - 300 - - - 86400 - 3600 - - - - - - /var/lib/clickhouse/format_schemas/ - - - /usr/share/clickhouse/protos/ - - - - - - - - - - false - - false - - - https://6f33034cfe684dd7a3ab9875e57b1c8d@o388870.ingest.sentry.io/5226277 - - false - - - - - - - - - - - - - - - - - - - - - - - - - - 1073741824 - 1024 - 1048576 - 30000000 - - - - backups - - - true - - - - - - - - - - -
\ No newline at end of file diff --git a/tests/integration/test_disks_app_interactive/configs/users.xml b/tests/integration/test_disks_app_interactive/configs/users.xml deleted file mode 100644 index 57bc6309a54..00000000000 --- a/tests/integration/test_disks_app_interactive/configs/users.xml +++ /dev/null @@ -1,120 +0,0 @@ - - - - - - - - - - - - 1 - - - - - - - - - - - - - ::/0 - - - - default - - - default - - - 1 - - - 1 - - - - - - - - - - - - - - 3600 - - - 0 - 0 - 0 - 0 - 0 - - - - diff --git a/tests/integration/test_disks_app_interactive/test.py b/tests/integration/test_disks_app_interactive/test.py index 4ecd8639104..79ffc3001a5 100644 --- a/tests/integration/test_disks_app_interactive/test.py +++ b/tests/integration/test_disks_app_interactive/test.py @@ -4,16 +4,6 @@ import pytest import pathlib -# import os - -# import grpc -# import pymysql.connections -# import psycopg2 as py_psql -# import sys -# import threading - -# from helpers.cluster import ClickHouseCluster, run_and_check -# from helpers.test_tools import assert_logs_contain_with_retry import subprocess import select import io @@ -26,9 +16,27 @@ class ClickHouseDisksException(Exception): pass -class LocalDisksClient(object): +@pytest.fixture(scope="module") +def started_cluster(): + global cluster + try: + cluster = ClickHouseCluster(__file__) + cluster.add_instance( + "disks_app_test", + main_configs=["server_configs/config.xml"], + with_minio=True, + ) + + cluster.start() + yield cluster + + finally: + cluster.shutdown() + + +class DisksClient(object): SEPARATOR = b"\a\a\a\a\n" - client: Optional["LocalDisksClient"] = None # static variable + local_client: Optional["DisksClient"] = None # static variable default_disk_root_directory: str = "/var/lib/clickhouse" def __init__(self, bin_path: str, config_path: str, working_path: str): @@ -170,24 +178,24 @@ class LocalDisksClient(object): self.execute_query(f"write {path_from_adding} {path_to}") @staticmethod - def getClient(refresh: bool): - if (LocalDisksClient.client is None) or refresh: + def getLocalDisksClient(refresh: bool): + if (DisksClient.local_client is None) or refresh: binary_file = os.environ.get("CLICKHOUSE_TESTS_SERVER_BIN_PATH") current_working_directory = str(pathlib.Path().resolve()) config_file = f"{current_working_directory}/test_disks_app_interactive/configs/config.xml" - if not os.path.exists(LocalDisksClient.default_disk_root_directory): - os.mkdir(LocalDisksClient.default_disk_root_directory) + if not os.path.exists(DisksClient.default_disk_root_directory): + os.mkdir(DisksClient.default_disk_root_directory) - LocalDisksClient.client = LocalDisksClient( + DisksClient.local_client = DisksClient( binary_file, config_file, current_working_directory ) - return LocalDisksClient.client + return DisksClient.local_client else: - return LocalDisksClient.client + return DisksClient.local_client def test_disks_app_interactive_list_disks(): - client = LocalDisksClient.getClient(True) + client = DisksClient.getLocalDisksClient(True) expected_disks_with_path = [ ("default", "/"), ("local", client.working_path), @@ -202,7 +210,7 @@ def test_disks_app_interactive_list_disks(): def test_disks_app_interactive_list_files_local(): - client = LocalDisksClient.getClient(True) + client = DisksClient.getLocalDisksClient(True) client.switch_disk("local") excepted_listed_files = sorted(os.listdir("test_disks_app_interactive/")) listed_files = sorted(client.ls("test_disks_app_interactive/")) @@ -210,7 +218,7 @@ def test_disks_app_interactive_list_files_local(): def test_disks_app_interactive_list_directories_default(): - client = LocalDisksClient.getClient(True) + client = DisksClient.getLocalDisksClient(True) traversed_dir = client.ls(".", recursive=True) client.mkdir("dir1") client.mkdir("dir2") @@ -280,7 +288,7 @@ def test_disks_app_interactive_cp_and_read(): initial_text = "File content" with open("a.txt", "w") as file: file.write(initial_text) - client = LocalDisksClient.getClient(True) + client = DisksClient.getLocalDisksClient(True) client.switch_disk("default") client.copy("a.txt", "/a.txt", disk_from="local", disk_to="default") read_text = client.read("a.txt") @@ -291,9 +299,7 @@ def test_disks_app_interactive_cp_and_read(): assert "" == read_text read_text = client.read("/dir1/b.txt") assert read_text == initial_text - with open( - f"{LocalDisksClient.default_disk_root_directory}/dir1/b.txt", "r" - ) as file: + with open(f"{DisksClient.default_disk_root_directory}/dir1/b.txt", "r") as file: read_text = file.read() assert read_text == initial_text os.remove("a.txt") @@ -305,7 +311,7 @@ def test_disks_app_interactive_test_move_and_write(): initial_text = "File content" with open("a.txt", "w") as file: file.write(initial_text) - client = LocalDisksClient.getClient(True) + client = DisksClient.getLocalDisksClient(True) client.switch_disk("default") client.copy("a.txt", "/a.txt", disk_from="local", disk_to="default") files = client.ls(".") From 4ace4006d2f1be82fe299cdda73d5561b7103110 Mon Sep 17 00:00:00 2001 From: divanik Date: Fri, 14 Jun 2024 13:02:41 +0000 Subject: [PATCH 054/417] Add documentation for new features in interactive client --- .../operations/utilities/clickhouse-disks.md | 58 ++++++++++++------- programs/disks/CommandCopy.cpp | 11 ++-- programs/disks/CommandMove.cpp | 4 +- programs/disks/CommandRemove.cpp | 2 +- programs/disks/DisksApp.cpp | 2 +- 5 files changed, 48 insertions(+), 29 deletions(-) diff --git a/docs/en/operations/utilities/clickhouse-disks.md b/docs/en/operations/utilities/clickhouse-disks.md index 76db9e41836..a2a5035600b 100644 --- a/docs/en/operations/utilities/clickhouse-disks.md +++ b/docs/en/operations/utilities/clickhouse-disks.md @@ -4,35 +4,53 @@ sidebar_position: 59 sidebar_label: clickhouse-disks --- -# clickhouse-disks +# Clickhouse-disks -A utility providing filesystem-like operations for ClickHouse disks. +A utility providing filesystem-like operations for ClickHouse disks. It can work in both interactive and not interactive modes. -Program-wide options: +## Program-wide options * `--config-file, -C` -- path to ClickHouse config, defaults to `/etc/clickhouse-server/config.xml`. * `--save-logs` -- Log progress of invoked commands to `/var/log/clickhouse-server/clickhouse-disks.log`. * `--log-level` -- What [type](../server-configuration-parameters/settings#server_configuration_parameters-logger) of events to log, defaults to `none`. * `--disk` -- what disk to use for `mkdir, move, read, write, remove` commands. Defaults to `default`. +* `--query, -q` -- single query that can be executed without launching interactive mode + +## Default Disks +After the launch two disks are initialized. The first one is a disk `local` that is supposed to imitate local file system from which clickhouse-disks utility was launched. The second one is a disk `default` that is mounted to the local filesystem in the directory that can be found in config as a parameter `clickhouse/path` (default value is `/var/lib/clickhouse`). + +## Clickhouse-disks state +For each disk that was added the utility stores current directory (as in a usual filesystem). User can change current directory and switch between disks. + +State is reflected in a prompt "`disk_name`:`path_name`" ## Commands -* `copy [--disk-from d1] [--disk-to d2] `. - Recursively copy data from `FROM_PATH` at disk `d1` (defaults to `disk` value if not provided) - to `TO_PATH` at disk `d2` (defaults to `disk` value if not provided). -* `move `. - Move file or directory from `FROM_PATH` to `TO_PATH`. -* `remove `. - Remove `PATH` recursively. -* `link `. - Create a hardlink from `FROM_PATH` to `TO_PATH`. -* `list [--recursive] ...` - List files at `PATH`s. Non-recursive by default. -* `list-disks`. +In these documentation file all mandatory positional arguments are referred as ``, named arguments are referred as `[--parameter value]`. All positional parameters could be mentioned as a named parameter with a corresponding name. + +* `cd (change-dir, change_dir) [--disk disk] ` + Change directory to path `path` on disk `disk` (default value is a current disk). No disk switching happens. +* `copy (cp) [--disk-from disk_1] [--disk-to disk_2] `. + Recursively copy data from `path-from` at disk `disk_1` (default value is a current disk (parameter `disk` in a non-interactive mode)) + to `path-to` at disk `disk_2` (default value is a current disk (parameter `disk` in a non-interactive mode)). +* `current_disk_with_path (current, current_disk, current_path)` + Print current state in format: + `Disk: "current_disk" Path: "current path on current disk"` +* `move (mv) `. + Move file or directory from `path-from` to `path-to` within current disk. +* `remove (rm, delete) `. + Remove `path` recursively on a current disk. +* `link (ln) `. + Create a hardlink from `path-from` to `path-to` on a current disk. +* `list (ls) [--recursive] ` + List files at `path`s on a current disk. Non-recursive by default. +* `list-disks (list_disks, ls-disks, ls_disks)`. List disks names. -* `mkdir [--recursive] `. +* `mkdir [--recursive] ` on a current disk. Create a directory. Non-recursive by default. -* `read: []` - Read a file from `FROM_PATH` to `TO_PATH` (`stdout` if not supplied). -* `write [FROM_PATH] `. - Write a file from `FROM_PATH` (`stdin` if not supplied) to `TO_PATH`. +* `read (r) [--path-to path]` + Read a file from `path-from` to `path` (`stdout` if not supplied). +* `switch-disk [--path path] ` + Switch to disk `disk` on path `path` (if `path` is not specified default value is a previous path on disk `disk`). +* `write (w) [--path-from path] `. + Write a file from `path` (`stdin` if not supplied) to `path-to`. diff --git a/programs/disks/CommandCopy.cpp b/programs/disks/CommandCopy.cpp index ae749f7448a..4ba8a9ecbc2 100644 --- a/programs/disks/CommandCopy.cpp +++ b/programs/disks/CommandCopy.cpp @@ -12,11 +12,12 @@ public: explicit CommandCopy() : ICommand() { command_name = "copy"; - description = "Recursively copy data from `FROM_PATH` to `TO_PATH`"; - options_description.add_options()("disk-from", po::value(), "disk from which we copy")( - "disk-to", po::value(), "disk to which we copy")( - "path-from", po::value(), "path from which we copy (mandatory, positional)")( - "path-to", po::value(), "path to which we copy (mandatory, positional)"); + description = "Recursively copy data from `path-from` to `path-to`"; + options_description.add_options()( + "disk-from", po::value(), "disk from which we copy is executed (default value is a current disk)")( + "disk-to", po::value(), "disk to which copy is executed (default value is a current disk)")( + "path-from", po::value(), "path from which copy is executed (mandatory, positional)")( + "path-to", po::value(), "path to which copy is executed (mandatory, positional)"); positional_options_description.add("path-from", 1); positional_options_description.add("path-to", 1); } diff --git a/programs/disks/CommandMove.cpp b/programs/disks/CommandMove.cpp index 23144df3d35..d762e8023d9 100644 --- a/programs/disks/CommandMove.cpp +++ b/programs/disks/CommandMove.cpp @@ -12,14 +12,14 @@ public: command_name = "move"; description = "Move file or directory from `from_path` to `to_path`"; options_description.add_options()("path-from", po::value(), "path from which we copy (mandatory, positional)")( - "path-to", po::value(), "path to which we copy (mandatory, positional)"); + "path-to", po::value(), "path to which we copy (mandatory, positional)")s; positional_options_description.add("path-from", 1); positional_options_description.add("path-to", 1); } void executeImpl(const CommandLineOptions & options, DisksClient & client) override { - auto disk = client.getCurrentDiskWithPath(); + auto disk = getDiskWithPath(client, options, "disk"); String path_from = disk.getRelativeFromRoot(getValueFromCommandLineOptionsThrow(options, "path-from")); String path_to = disk.getRelativeFromRoot(getValueFromCommandLineOptionsThrow(options, "path-to")); diff --git a/programs/disks/CommandRemove.cpp b/programs/disks/CommandRemove.cpp index b322fb2701f..1576777a4cd 100644 --- a/programs/disks/CommandRemove.cpp +++ b/programs/disks/CommandRemove.cpp @@ -17,7 +17,7 @@ public: void executeImpl(const CommandLineOptions & options, DisksClient & client) override { - auto disk = client.getCurrentDiskWithPath(); + auto disk = getDiskWithPath(client, options, "disk"); const String & path = disk.getRelativeFromRoot(getValueFromCommandLineOptionsThrow(options, "path")); disk.getDisk()->removeRecursive(path); } diff --git a/programs/disks/DisksApp.cpp b/programs/disks/DisksApp.cpp index 7f657dd32a3..d2adf5bac7a 100644 --- a/programs/disks/DisksApp.cpp +++ b/programs/disks/DisksApp.cpp @@ -280,7 +280,7 @@ void DisksApp::addOptions() options_description.add_options()("help,h", "Print common help message")("config-file,C", po::value(), "Set config file")( "disk", po::value(), "Set disk name")("save-logs", "Save logs to a file")( "log-level", po::value(), "Logging level")("query,q", po::value(), "Query for a non-interactive mode")( - "test-mode", "Interface in test regyme"); + "test-mode", "Interactive interface in test regyme"); command_descriptions.emplace("list-disks", makeCommandListDisks()); command_descriptions.emplace("copy", makeCommandCopy()); From 752bd0078450ad6b9255c860eec9933fb74abb8a Mon Sep 17 00:00:00 2001 From: divanik Date: Fri, 14 Jun 2024 13:54:12 +0000 Subject: [PATCH 055/417] Corrected problems and tests --- docs/en/operations/utilities/clickhouse-disks.md | 3 +++ programs/disks/CommandMove.cpp | 4 ++-- programs/disks/CommandRemove.cpp | 2 +- tests/integration/test_disks_app_func/test.py | 2 +- tests/integration/test_disks_app_interactive/__init__.py | 0 tests/integration/test_disks_app_interactive/test.py | 1 - 6 files changed, 7 insertions(+), 5 deletions(-) create mode 100644 tests/integration/test_disks_app_interactive/__init__.py diff --git a/docs/en/operations/utilities/clickhouse-disks.md b/docs/en/operations/utilities/clickhouse-disks.md index a2a5035600b..5363449a960 100644 --- a/docs/en/operations/utilities/clickhouse-disks.md +++ b/docs/en/operations/utilities/clickhouse-disks.md @@ -15,6 +15,7 @@ A utility providing filesystem-like operations for ClickHouse disks. It can work * `--log-level` -- What [type](../server-configuration-parameters/settings#server_configuration_parameters-logger) of events to log, defaults to `none`. * `--disk` -- what disk to use for `mkdir, move, read, write, remove` commands. Defaults to `default`. * `--query, -q` -- single query that can be executed without launching interactive mode +* `--help, -h` -- print all the options and commamds with description ## Default Disks After the launch two disks are initialized. The first one is a disk `local` that is supposed to imitate local file system from which clickhouse-disks utility was launched. The second one is a disk `default` that is mounted to the local filesystem in the directory that can be found in config as a parameter `clickhouse/path` (default value is `/var/lib/clickhouse`). @@ -36,6 +37,8 @@ In these documentation file all mandatory positional arguments are referred as ` * `current_disk_with_path (current, current_disk, current_path)` Print current state in format: `Disk: "current_disk" Path: "current path on current disk"` +* `help []` + Print help message about command `command`. If `command` is not specified print information about all commands. * `move (mv) `. Move file or directory from `path-from` to `path-to` within current disk. * `remove (rm, delete) `. diff --git a/programs/disks/CommandMove.cpp b/programs/disks/CommandMove.cpp index d762e8023d9..23144df3d35 100644 --- a/programs/disks/CommandMove.cpp +++ b/programs/disks/CommandMove.cpp @@ -12,14 +12,14 @@ public: command_name = "move"; description = "Move file or directory from `from_path` to `to_path`"; options_description.add_options()("path-from", po::value(), "path from which we copy (mandatory, positional)")( - "path-to", po::value(), "path to which we copy (mandatory, positional)")s; + "path-to", po::value(), "path to which we copy (mandatory, positional)"); positional_options_description.add("path-from", 1); positional_options_description.add("path-to", 1); } void executeImpl(const CommandLineOptions & options, DisksClient & client) override { - auto disk = getDiskWithPath(client, options, "disk"); + auto disk = client.getCurrentDiskWithPath(); String path_from = disk.getRelativeFromRoot(getValueFromCommandLineOptionsThrow(options, "path-from")); String path_to = disk.getRelativeFromRoot(getValueFromCommandLineOptionsThrow(options, "path-to")); diff --git a/programs/disks/CommandRemove.cpp b/programs/disks/CommandRemove.cpp index 1576777a4cd..b322fb2701f 100644 --- a/programs/disks/CommandRemove.cpp +++ b/programs/disks/CommandRemove.cpp @@ -17,7 +17,7 @@ public: void executeImpl(const CommandLineOptions & options, DisksClient & client) override { - auto disk = getDiskWithPath(client, options, "disk"); + auto disk = client.getCurrentDiskWithPath(); const String & path = disk.getRelativeFromRoot(getValueFromCommandLineOptionsThrow(options, "path")); disk.getDisk()->removeRecursive(path); } diff --git a/tests/integration/test_disks_app_func/test.py b/tests/integration/test_disks_app_func/test.py index ac1edae4199..34e45a9d626 100644 --- a/tests/integration/test_disks_app_func/test.py +++ b/tests/integration/test_disks_app_func/test.py @@ -10,7 +10,7 @@ def started_cluster(): cluster = ClickHouseCluster(__file__) cluster.add_instance( "disks_app_test", - main_configs=["server_configs/config.xml"], + main_configs=["config.xml"], with_minio=True, ) diff --git a/tests/integration/test_disks_app_interactive/__init__.py b/tests/integration/test_disks_app_interactive/__init__.py new file mode 100644 index 00000000000..e69de29bb2d diff --git a/tests/integration/test_disks_app_interactive/test.py b/tests/integration/test_disks_app_interactive/test.py index 79ffc3001a5..35bd49485e4 100644 --- a/tests/integration/test_disks_app_interactive/test.py +++ b/tests/integration/test_disks_app_interactive/test.py @@ -11,7 +11,6 @@ from typing import List, Tuple, Dict, Union, Optional import os - class ClickHouseDisksException(Exception): pass From a3cab6853e187d8dfd1894ce2be332529cffb24c Mon Sep 17 00:00:00 2001 From: divanik Date: Fri, 14 Jun 2024 14:10:00 +0000 Subject: [PATCH 056/417] python test reformat --- tests/integration/test_disks_app_interactive/test.py | 1 + 1 file changed, 1 insertion(+) diff --git a/tests/integration/test_disks_app_interactive/test.py b/tests/integration/test_disks_app_interactive/test.py index 35bd49485e4..79ffc3001a5 100644 --- a/tests/integration/test_disks_app_interactive/test.py +++ b/tests/integration/test_disks_app_interactive/test.py @@ -11,6 +11,7 @@ from typing import List, Tuple, Dict, Union, Optional import os + class ClickHouseDisksException(Exception): pass From 55c218b4a5cec3c50fd485b29f14a27cac75b572 Mon Sep 17 00:00:00 2001 From: divanik Date: Fri, 14 Jun 2024 14:20:49 +0000 Subject: [PATCH 057/417] Fix typo --- docs/en/operations/utilities/clickhouse-disks.md | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/docs/en/operations/utilities/clickhouse-disks.md b/docs/en/operations/utilities/clickhouse-disks.md index 5363449a960..aeca49c0e1e 100644 --- a/docs/en/operations/utilities/clickhouse-disks.md +++ b/docs/en/operations/utilities/clickhouse-disks.md @@ -15,7 +15,7 @@ A utility providing filesystem-like operations for ClickHouse disks. It can work * `--log-level` -- What [type](../server-configuration-parameters/settings#server_configuration_parameters-logger) of events to log, defaults to `none`. * `--disk` -- what disk to use for `mkdir, move, read, write, remove` commands. Defaults to `default`. * `--query, -q` -- single query that can be executed without launching interactive mode -* `--help, -h` -- print all the options and commamds with description +* `--help, -h` -- print all the options and commands with description ## Default Disks After the launch two disks are initialized. The first one is a disk `local` that is supposed to imitate local file system from which clickhouse-disks utility was launched. The second one is a disk `default` that is mounted to the local filesystem in the directory that can be found in config as a parameter `clickhouse/path` (default value is `/var/lib/clickhouse`). From 0d0fe2ab00132065d3438712cfbd3b257c7c3cb4 Mon Sep 17 00:00:00 2001 From: avogar Date: Mon, 17 Jun 2024 13:30:51 +0000 Subject: [PATCH 058/417] Fix bad conflict resolution --- src/Storages/MergeTree/MergeTreeDataPartWriterWide.cpp | 7 +------ src/Storages/MergeTree/MergeTreeIOSettings.h | 2 +- 2 files changed, 2 insertions(+), 7 deletions(-) diff --git a/src/Storages/MergeTree/MergeTreeDataPartWriterWide.cpp b/src/Storages/MergeTree/MergeTreeDataPartWriterWide.cpp index 522161c458e..89e0db83073 100644 --- a/src/Storages/MergeTree/MergeTreeDataPartWriterWide.cpp +++ b/src/Storages/MergeTree/MergeTreeDataPartWriterWide.cpp @@ -627,14 +627,9 @@ void MergeTreeDataPartWriterWide::validateColumnOfFixedSize(const NameAndTypePai void MergeTreeDataPartWriterWide::fillDataChecksums(MergeTreeDataPartChecksums & checksums, NameSet & checksums_to_remove) { ISerialization::SerializeBinaryBulkSettings serialize_settings; -<<<<<<< HEAD - serialize_settings.low_cardinality_max_dictionary_size = global_settings.low_cardinality_max_dictionary_size; - serialize_settings.low_cardinality_use_single_dictionary_for_part = global_settings.low_cardinality_use_single_dictionary_for_part != 0; - serialize_settings.use_compact_variant_discriminators_serialization = storage.getSettings()->use_compact_variant_discriminators_serialization; -======= serialize_settings.low_cardinality_max_dictionary_size = settings.low_cardinality_max_dictionary_size; serialize_settings.low_cardinality_use_single_dictionary_for_part = settings.low_cardinality_use_single_dictionary_for_part; ->>>>>>> d9a11faf4a3c02aaac2681aa9f7ee126123040b2 + serialize_settings.use_compact_variant_discriminators_serialization = settings.use_compact_variant_discriminators_serialization; WrittenOffsetColumns offset_columns; if (rows_written_in_last_mark > 0) { diff --git a/src/Storages/MergeTree/MergeTreeIOSettings.h b/src/Storages/MergeTree/MergeTreeIOSettings.h index 50ffdb8aa1f..04171656fcf 100644 --- a/src/Storages/MergeTree/MergeTreeIOSettings.h +++ b/src/Storages/MergeTree/MergeTreeIOSettings.h @@ -1,6 +1,6 @@ #pragma once #include -//#include +#include #include #include #include From 318a099d9023342a2741bfc51850a569ae6b6b46 Mon Sep 17 00:00:00 2001 From: Kruglov Pavel <48961922+Avogar@users.noreply.github.com> Date: Mon, 17 Jun 2024 19:12:35 +0200 Subject: [PATCH 059/417] Fix docs --- docs/en/operations/settings/merge-tree-settings.md | 3 +-- 1 file changed, 1 insertion(+), 2 deletions(-) diff --git a/docs/en/operations/settings/merge-tree-settings.md b/docs/en/operations/settings/merge-tree-settings.md index 1dd18606af3..01f9bf5a6ed 100644 --- a/docs/en/operations/settings/merge-tree-settings.md +++ b/docs/en/operations/settings/merge-tree-settings.md @@ -974,14 +974,13 @@ Default value: false - [exclude_deleted_rows_for_part_size_in_merge](#exclude_deleted_rows_for_part_size_in_merge) setting -<<<<<<< HEAD ## use_compact_variant_discriminators_serialization {#use_compact_variant_discriminators_serialization} Enables compact mode for binary serialization of discriminators in Variant data type. This mode allows to use significantly less memory for storing discriminators in parts when there is mostly one variant or a lot of NULL values. Default value: true -======= + ### optimize_row_order Controls if the row order should be optimized during inserts to improve the compressability of the newly inserted table part. From 9be2ec65118a7b9d4847073ac759c042768581e8 Mon Sep 17 00:00:00 2001 From: Kruglov Pavel <48961922+Avogar@users.noreply.github.com> Date: Mon, 17 Jun 2024 19:13:01 +0200 Subject: [PATCH 060/417] Fix docs --- docs/en/operations/settings/merge-tree-settings.md | 1 - 1 file changed, 1 deletion(-) diff --git a/docs/en/operations/settings/merge-tree-settings.md b/docs/en/operations/settings/merge-tree-settings.md index 01f9bf5a6ed..84bd3ba64a4 100644 --- a/docs/en/operations/settings/merge-tree-settings.md +++ b/docs/en/operations/settings/merge-tree-settings.md @@ -1026,4 +1026,3 @@ Compression rates of LZ4 or ZSTD improve on average by 20-40%. This setting works best for tables with no primary key or a low-cardinality primary key, i.e. a table with only few distinct primary key values. High-cardinality primary keys, e.g. involving timestamp columns of type `DateTime64`, are not expected to benefit from this setting. ->>>>>>> d9a11faf4a3c02aaac2681aa9f7ee126123040b2 From 6f841d89e73f804e4610bb60ab324d1b7b0bb805 Mon Sep 17 00:00:00 2001 From: divanik Date: Mon, 17 Jun 2024 17:40:52 +0000 Subject: [PATCH 061/417] Fix tests --- programs/disks/CommandList.cpp | 11 +++---- programs/disks/DisksApp.cpp | 22 ++++++------- programs/disks/DisksApp.h | 2 ++ programs/disks/DisksClient.cpp | 1 - programs/disks/DisksClient.h | 5 ++- programs/disks/ICommand.h | 2 +- .../02802_clickhouse_disks_s3_copy.sh | 18 +++++------ ...80_s3_plain_DROP_TABLE_MergeTree.reference | 32 +++++++++---------- .../02980_s3_plain_DROP_TABLE_MergeTree.sh | 4 +-- 9 files changed, 49 insertions(+), 48 deletions(-) diff --git a/programs/disks/CommandList.cpp b/programs/disks/CommandList.cpp index c21941c42ca..77479b1d217 100644 --- a/programs/disks/CommandList.cpp +++ b/programs/disks/CommandList.cpp @@ -56,12 +56,9 @@ private: std::cout << relative_path << ":\n"; - if (!file_names.empty()) - { - for (const auto & file_name : file_names) - if (show_hidden || (!file_name.starts_with('.'))) - selected_and_sorted_file_names.push_back(file_name); - } + for (const auto & file_name : file_names) + if (show_hidden || (!file_name.starts_with('.'))) + selected_and_sorted_file_names.push_back(file_name); std::sort(selected_and_sorted_file_names.begin(), selected_and_sorted_file_names.end()); for (const auto & file_name : selected_and_sorted_file_names) @@ -84,7 +81,9 @@ private: } }(); if (disk.isDirectory(path)) + { listRecursive(disk, path, show_hidden); + } } } }; diff --git a/programs/disks/DisksApp.cpp b/programs/disks/DisksApp.cpp index d2adf5bac7a..3b09feecc3b 100644 --- a/programs/disks/DisksApp.cpp +++ b/programs/disks/DisksApp.cpp @@ -49,17 +49,15 @@ CommandPtr DisksApp::getCommandByName(const String & command) const std::vector DisksApp::getEmptyCompletion(String command_name) const { auto command_ptr = command_descriptions.at(command_name); - auto answer = [&]() -> std::vector + std::vector answer{}; + if (multidisk_commands.contains(command_ptr->command_name)) { - if (multidisk_commands.contains(command_ptr->command_name)) - { - return client->getAllFilesByPatternFromAllDisks(""); - } - else - { - return client->getCurrentDiskWithPath().getAllFilesByPattern(""); - } - }(); + answer = client->getAllFilesByPatternFromAllDisks(""); + } + else + { + answer = client->getCurrentDiskWithPath().getAllFilesByPattern(""); + } for (const auto & disk_name : client->getAllDiskNames()) { answer.push_back(disk_name); @@ -211,7 +209,7 @@ bool DisksApp::processQueryText(const String & text) int code = getCurrentExceptionCode(); if (code == ErrorCodes::LOGICAL_ERROR) { - throw err; + throw std::move(err); } else if (code == ErrorCodes::BAD_ARGUMENTS) { @@ -467,7 +465,7 @@ int DisksApp::main(const std::vector & /*args*/) registerDisks(/* global_skip_access_check= */ true); registerFormats(); - auto shared_context = Context::createShared(); + shared_context = Context::createShared(); global_context = Context::createGlobal(shared_context.get()); global_context->makeGlobalContext(); diff --git a/programs/disks/DisksApp.h b/programs/disks/DisksApp.h index 75d604bf63c..1ecd9944fb8 100644 --- a/programs/disks/DisksApp.h +++ b/programs/disks/DisksApp.h @@ -68,6 +68,8 @@ private: static String word_break_characters; // General command line arguments parsing fields + + SharedContextHolder shared_context; ContextMutablePtr global_context; ProgramOptionsDescription options_description; CommandLineOptions options; diff --git a/programs/disks/DisksClient.cpp b/programs/disks/DisksClient.cpp index 4f808f85ab6..e38f7ec99b8 100644 --- a/programs/disks/DisksClient.cpp +++ b/programs/disks/DisksClient.cpp @@ -14,7 +14,6 @@ namespace ErrorCodes namespace DB { - DiskWithPath::DiskWithPath(DiskPtr disk_, std::optional path_) : disk(disk_) { if (path_.has_value()) diff --git a/programs/disks/DisksClient.h b/programs/disks/DisksClient.h index 3320c5f7cef..ab99d2f6590 100644 --- a/programs/disks/DisksClient.h +++ b/programs/disks/DisksClient.h @@ -32,7 +32,10 @@ public: String getCurrentPath() const { return path; } - bool isDirectory(const String & any_path) const { return disk->isDirectory(getRelativeFromRoot(any_path)); } + bool isDirectory(const String & any_path) const + { + return disk->isDirectory(getRelativeFromRoot(any_path)) || disk->isDirectory(getAbsolutePath(any_path)); + } std::vector listAllFilesByPath(const String & any_path) const; diff --git a/programs/disks/ICommand.h b/programs/disks/ICommand.h index b1e594066af..2b409d4ade6 100644 --- a/programs/disks/ICommand.h +++ b/programs/disks/ICommand.h @@ -53,7 +53,7 @@ protected: { return options[name].as(); } - catch (boost::bad_any_cast) + catch (boost::bad_any_cast &) { throw DB::Exception(ErrorCodes::BAD_ARGUMENTS, "Argument '{}' has wrong type and can't be parsed", name); } diff --git a/tests/queries/0_stateless/02802_clickhouse_disks_s3_copy.sh b/tests/queries/0_stateless/02802_clickhouse_disks_s3_copy.sh index 2b9e5296a05..d317b2e8a1e 100755 --- a/tests/queries/0_stateless/02802_clickhouse_disks_s3_copy.sh +++ b/tests/queries/0_stateless/02802_clickhouse_disks_s3_copy.sh @@ -14,14 +14,14 @@ function run_test_for_disk() echo "$disk" - clickhouse-disks -C "$config" --disk "$disk" write --input "$config" $CLICKHOUSE_DATABASE/test - clickhouse-disks -C "$config" --log-level test --disk "$disk" copy $CLICKHOUSE_DATABASE/test $CLICKHOUSE_DATABASE/test.copy |& { + clickhouse-disks -C "$config" --disk "$disk" --query "write --path-from "$config" $CLICKHOUSE_DATABASE/test" + clickhouse-disks -C "$config" --log-level test --disk "$disk" --query "copy $CLICKHOUSE_DATABASE/test $CLICKHOUSE_DATABASE/test.copy" |& { grep -o -e "Single part upload has completed." -e "Single operation copy has completed." } - clickhouse-disks -C "$config" --disk "$disk" remove $CLICKHOUSE_DATABASE/test + clickhouse-disks -C "$config" --disk "$disk" --query "remove $CLICKHOUSE_DATABASE/test" # NOTE: this is due to "copy" does works like "cp -R from to/" instead of "cp from to" - clickhouse-disks -C "$config" --disk "$disk" remove $CLICKHOUSE_DATABASE/test.copy/test - clickhouse-disks -C "$config" --disk "$disk" remove $CLICKHOUSE_DATABASE/test.copy + clickhouse-disks -C "$config" --disk "$disk" --query "remove $CLICKHOUSE_DATABASE/test.copy/test" + clickhouse-disks -C "$config" --disk "$disk" --query "remove $CLICKHOUSE_DATABASE/test.copy" } function run_test_copy_from_s3_to_s3(){ @@ -29,13 +29,13 @@ function run_test_copy_from_s3_to_s3(){ local disk_dest=$1 && shift echo "copy from $disk_src to $disk_dest" - clickhouse-disks -C "$config" --disk "$disk_src" write --input "$config" $CLICKHOUSE_DATABASE/test + clickhouse-disks -C "$config" --disk "$disk_src" --query "write --path-from "$config" $CLICKHOUSE_DATABASE/test" - clickhouse-disks -C "$config" --log-level test copy --disk-from "$disk_src" --disk-to "$disk_dest" $CLICKHOUSE_DATABASE/test $CLICKHOUSE_DATABASE/test.copy |& { + clickhouse-disks -C "$config" --log-level test --query "copy --disk-from "$disk_src" --disk-to "$disk_dest" $CLICKHOUSE_DATABASE/test $CLICKHOUSE_DATABASE/test.copy" |& { grep -o -e "Single part upload has completed." -e "Single operation copy has completed." } - clickhouse-disks -C "$config" --disk "$disk_dest" remove $CLICKHOUSE_DATABASE/test.copy/test - clickhouse-disks -C "$config" --disk "$disk_dest" remove $CLICKHOUSE_DATABASE/test.copy + clickhouse-disks -C "$config" --disk "$disk_dest" --query "remove $CLICKHOUSE_DATABASE/test.copy/test" + clickhouse-disks -C "$config" --disk "$disk_dest" --query "remove $CLICKHOUSE_DATABASE/test.copy" } run_test_for_disk s3_plain_native_copy diff --git a/tests/queries/0_stateless/02980_s3_plain_DROP_TABLE_MergeTree.reference b/tests/queries/0_stateless/02980_s3_plain_DROP_TABLE_MergeTree.reference index 531163e1d84..3135f2d01e1 100644 --- a/tests/queries/0_stateless/02980_s3_plain_DROP_TABLE_MergeTree.reference +++ b/tests/queries/0_stateless/02980_s3_plain_DROP_TABLE_MergeTree.reference @@ -3,28 +3,28 @@ data after ATTACH 1 Files before DETACH TABLE all_1_1_0 -backups/ordinary_default/data/ordinary_default/data/all_1_1_0: -primary.cidx -serialization.json -metadata_version.txt -default_compression_codec.txt +/backups/ordinary_default/data/ordinary_default/data/all_1_1_0: +checksums.txt +columns.txt +count.txt data.bin data.cmrk3 -count.txt -columns.txt -checksums.txt +default_compression_codec.txt +metadata_version.txt +primary.cidx +serialization.json Files after DETACH TABLE all_1_1_0 -backups/ordinary_default/data/ordinary_default/data/all_1_1_0: -primary.cidx -serialization.json -metadata_version.txt -default_compression_codec.txt +/backups/ordinary_default/data/ordinary_default/data/all_1_1_0: +checksums.txt +columns.txt +count.txt data.bin data.cmrk3 -count.txt -columns.txt -checksums.txt +default_compression_codec.txt +metadata_version.txt +primary.cidx +serialization.json diff --git a/tests/queries/0_stateless/02980_s3_plain_DROP_TABLE_MergeTree.sh b/tests/queries/0_stateless/02980_s3_plain_DROP_TABLE_MergeTree.sh index 12d08159012..e6427ab26f8 100755 --- a/tests/queries/0_stateless/02980_s3_plain_DROP_TABLE_MergeTree.sh +++ b/tests/queries/0_stateless/02980_s3_plain_DROP_TABLE_MergeTree.sh @@ -49,11 +49,11 @@ path=$($CLICKHOUSE_CLIENT -q "SELECT replace(data_paths[1], 's3_plain', '') FROM path=${path%/} echo "Files before DETACH TABLE" -clickhouse-disks -C "$config" --disk s3_plain_disk list --recursive "${path:?}" | tail -n+2 +clickhouse-disks -C "$config" --disk s3_plain_disk --query "list --recursive "${path:?}"" | tail -n+2 $CLICKHOUSE_CLIENT -q "detach table data" echo "Files after DETACH TABLE" -clickhouse-disks -C "$config" --disk s3_plain_disk list --recursive "$path" | tail -n+2 +clickhouse-disks -C "$config" --disk s3_plain_disk --query "list --recursive "$path"" | tail -n+2 # metadata file is left $CLICKHOUSE_CLIENT --force_remove_data_recursively_on_drop=1 -q "drop database if exists $CLICKHOUSE_DATABASE" From c5fdc87c1e6dd8c1d1216e5599042fee682c23f3 Mon Sep 17 00:00:00 2001 From: Anton Popov Date: Mon, 17 Jun 2024 18:15:29 +0000 Subject: [PATCH 062/417] use rows threshold --- src/Core/Settings.h | 2 +- src/Core/SettingsChangesHistory.h | 2 +- src/Processors/QueryPlan/BufferChunksTransform.cpp | 12 ++++++------ src/Processors/QueryPlan/BufferChunksTransform.h | 6 +++--- src/Processors/QueryPlan/SortingStep.cpp | 8 +++----- src/Processors/QueryPlan/SortingStep.h | 2 +- .../0_stateless/03168_read_in_order_buffering_1.sql | 4 ++-- .../0_stateless/03168_read_in_order_buffering_2.sql | 9 +++++---- 8 files changed, 22 insertions(+), 23 deletions(-) diff --git a/src/Core/Settings.h b/src/Core/Settings.h index 121be813e31..b3e0ecd9e9c 100644 --- a/src/Core/Settings.h +++ b/src/Core/Settings.h @@ -547,7 +547,7 @@ class IColumn; M(Bool, optimize_read_in_order, true, "Enable ORDER BY optimization for reading data in corresponding order in MergeTree tables.", 0) \ M(Bool, optimize_read_in_window_order, true, "Enable ORDER BY optimization in window clause for reading data in corresponding order in MergeTree tables.", 0) \ M(Bool, optimize_aggregation_in_order, false, "Enable GROUP BY optimization for aggregating data in corresponding order in MergeTree tables.", 0) \ - M(UInt64, read_in_order_max_bytes_to_buffer, 128 * 1024 * 1024, "Max bytes to buffer before merging while reading in order of primary key. The higher value increases parallelism of query execution", 0) \ + M(Bool, read_in_order_use_buffering, true, "Use buffering before merging while reading in order of primary key. It increases the parallelism of query execution", 0) \ M(UInt64, aggregation_in_order_max_block_bytes, 50000000, "Maximal size of block in bytes accumulated during aggregation in order of primary key. Lower block size allows to parallelize more final merge stage of aggregation.", 0) \ M(UInt64, read_in_order_two_level_merge_threshold, 100, "Minimal number of parts to read to run preliminary merge step during multithread reading in order of primary key.", 0) \ M(Bool, low_cardinality_allow_in_native_format, true, "Use LowCardinality type in Native format. Otherwise, convert LowCardinality columns to ordinary for select query, and convert ordinary columns to required LowCardinality for insert query.", 0) \ diff --git a/src/Core/SettingsChangesHistory.h b/src/Core/SettingsChangesHistory.h index f94959fbd74..328ffe01a02 100644 --- a/src/Core/SettingsChangesHistory.h +++ b/src/Core/SettingsChangesHistory.h @@ -96,7 +96,7 @@ static const std::map= max_bytes_to_buffer) + if (num_buffered_rows >= max_rows_to_buffer) { input.setNotNeeded(); return Status::PortFull; diff --git a/src/Processors/QueryPlan/BufferChunksTransform.h b/src/Processors/QueryPlan/BufferChunksTransform.h index c912f280a8a..a7e9d413c64 100644 --- a/src/Processors/QueryPlan/BufferChunksTransform.h +++ b/src/Processors/QueryPlan/BufferChunksTransform.h @@ -8,7 +8,7 @@ namespace DB class BufferChunksTransform : public IProcessor { public: - BufferChunksTransform(const Block & header_, size_t max_bytes_to_buffer_, size_t limit_); + BufferChunksTransform(const Block & header_, size_t max_rows_to_buffer_, size_t limit_); Status prepare() override; String getName() const override { return "BufferChunks"; } @@ -19,11 +19,11 @@ private: InputPort & input; OutputPort & output; - size_t max_bytes_to_buffer; + size_t max_rows_to_buffer; size_t limit; std::queue chunks; - size_t num_buffered_bytes = 0; + size_t num_buffered_rows = 0; size_t num_processed_rows = 0; }; diff --git a/src/Processors/QueryPlan/SortingStep.cpp b/src/Processors/QueryPlan/SortingStep.cpp index a853c908317..a06910bef7f 100644 --- a/src/Processors/QueryPlan/SortingStep.cpp +++ b/src/Processors/QueryPlan/SortingStep.cpp @@ -38,7 +38,7 @@ SortingStep::Settings::Settings(const Context & context) tmp_data = context.getTempDataOnDisk(); min_free_disk_space = settings.min_free_disk_space_for_temporary_data; max_block_bytes = settings.prefer_external_sort_block_bytes; - read_in_order_max_bytes_to_buffer = settings.read_in_order_max_bytes_to_buffer; + read_in_order_use_buffering = settings.read_in_order_use_buffering; } SortingStep::Settings::Settings(size_t max_block_size_) @@ -246,13 +246,11 @@ void SortingStep::mergingSorted(QueryPipelineBuilder & pipeline, const SortDescr /// If there are several streams, then we merge them into one if (pipeline.getNumStreams() > 1) { - if (use_buffering && sort_settings.read_in_order_max_bytes_to_buffer) + if (use_buffering && sort_settings.read_in_order_use_buffering) { - size_t bytes_to_buffer = sort_settings.read_in_order_max_bytes_to_buffer / pipeline.getNumStreams(); - pipeline.addSimpleTransform([&](const Block & header) { - return std::make_shared(header, bytes_to_buffer, limit_); + return std::make_shared(header, sort_settings.max_block_bytes, limit_); }); } diff --git a/src/Processors/QueryPlan/SortingStep.h b/src/Processors/QueryPlan/SortingStep.h index 57658b6dafb..b4a49394a13 100644 --- a/src/Processors/QueryPlan/SortingStep.h +++ b/src/Processors/QueryPlan/SortingStep.h @@ -28,7 +28,7 @@ public: TemporaryDataOnDiskScopePtr tmp_data = nullptr; size_t min_free_disk_space = 0; size_t max_block_bytes = 0; - size_t read_in_order_max_bytes_to_buffer = 0; + size_t read_in_order_use_buffering = 0; explicit Settings(const Context & context); explicit Settings(size_t max_block_size_); diff --git a/tests/queries/0_stateless/03168_read_in_order_buffering_1.sql b/tests/queries/0_stateless/03168_read_in_order_buffering_1.sql index 02ffc9ecb7d..75025dcadc8 100644 --- a/tests/queries/0_stateless/03168_read_in_order_buffering_1.sql +++ b/tests/queries/0_stateless/03168_read_in_order_buffering_1.sql @@ -8,7 +8,7 @@ INSERT INTO t_read_in_order_1 SELECT number, number FROM numbers(1000000); SET max_threads = 8; SET optimize_read_in_order = 1; -SET read_in_order_max_bytes_to_buffer = '128M'; +SET read_in_order_use_buffering = 1; SELECT count() FROM ( @@ -25,7 +25,7 @@ SELECT count() FROM EXPLAIN PIPELINE SELECT * FROM t_read_in_order_1 WHERE v % 10 = 0 ORDER BY id LIMIT 10 ) WHERE explain LIKE '%BufferChunks%'; -SET read_in_order_max_bytes_to_buffer = 0; +SET read_in_order_use_buffering = 0; SELECT count() FROM ( diff --git a/tests/queries/0_stateless/03168_read_in_order_buffering_2.sql b/tests/queries/0_stateless/03168_read_in_order_buffering_2.sql index 7ce07a55d5d..1d3a75412e0 100644 --- a/tests/queries/0_stateless/03168_read_in_order_buffering_2.sql +++ b/tests/queries/0_stateless/03168_read_in_order_buffering_2.sql @@ -4,12 +4,13 @@ DROP TABLE IF EXISTS t_read_in_order_2; CREATE TABLE t_read_in_order_2 (id UInt64, v UInt64) ENGINE = MergeTree ORDER BY id; -INSERT INTO t_read_in_order_2 SELECT number, number FROM numbers(100000000); +INSERT INTO t_read_in_order_2 SELECT number, number FROM numbers(10000000); +OPTIMIZE TABLE t_read_in_order_2 FINAL; SET optimize_read_in_order = 1; -SET max_threads = 8; -SET read_in_order_max_bytes_to_buffer = '80M'; -SET max_memory_usage = '250M'; +SET max_threads = 4; +SET read_in_order_use_buffering = 1; +SET max_memory_usage = '100M'; SELECT * FROM t_read_in_order_2 ORDER BY id FORMAT Null; From 148a3c80cbeaafe52834f931b3bcc627a3f60888 Mon Sep 17 00:00:00 2001 From: Anton Popov Date: Mon, 17 Jun 2024 20:44:53 +0000 Subject: [PATCH 063/417] better thresholds --- .../QueryPlan/BufferChunksTransform.cpp | 18 +++++++++++++----- .../QueryPlan/BufferChunksTransform.h | 8 +++++++- src/Processors/QueryPlan/SortingStep.cpp | 2 +- 3 files changed, 21 insertions(+), 7 deletions(-) diff --git a/src/Processors/QueryPlan/BufferChunksTransform.cpp b/src/Processors/QueryPlan/BufferChunksTransform.cpp index a34f0be16ef..283a0acf172 100644 --- a/src/Processors/QueryPlan/BufferChunksTransform.cpp +++ b/src/Processors/QueryPlan/BufferChunksTransform.cpp @@ -3,11 +3,16 @@ namespace DB { -BufferChunksTransform::BufferChunksTransform(const Block & header_, size_t max_rows_to_buffer_, size_t limit_) +BufferChunksTransform::BufferChunksTransform( + const Block & header_, + size_t max_rows_to_buffer_, + size_t max_bytes_to_buffer_, + size_t limit_) : IProcessor({header_}, {header_}) , input(inputs.front()) , output(outputs.front()) , max_rows_to_buffer(max_rows_to_buffer_) + , max_bytes_to_buffer(max_bytes_to_buffer_) , limit(limit_) { } @@ -30,7 +35,9 @@ IProcessor::Status BufferChunksTransform::prepare() auto chunk = std::move(chunks.front()); chunks.pop(); - num_buffered_rows -= chunk.bytes(); + num_buffered_rows -= chunk.getNumRows(); + num_buffered_bytes -= chunk.bytes(); + output.push(std::move(chunk)); } else if (input.hasData()) @@ -45,14 +52,15 @@ IProcessor::Status BufferChunksTransform::prepare() } } - if (input.hasData() && num_buffered_rows < max_rows_to_buffer) + if (input.hasData() && (num_buffered_rows < max_rows_to_buffer || num_buffered_bytes < max_bytes_to_buffer)) { auto chunk = pullChunk(); - num_buffered_rows += chunk.bytes(); + num_buffered_rows += chunk.getNumRows(); + num_buffered_bytes += chunk.bytes(); chunks.push(std::move(chunk)); } - if (num_buffered_rows >= max_rows_to_buffer) + if (num_buffered_rows >= max_rows_to_buffer && num_buffered_bytes >= max_bytes_to_buffer) { input.setNotNeeded(); return Status::PortFull; diff --git a/src/Processors/QueryPlan/BufferChunksTransform.h b/src/Processors/QueryPlan/BufferChunksTransform.h index a7e9d413c64..84c35431364 100644 --- a/src/Processors/QueryPlan/BufferChunksTransform.h +++ b/src/Processors/QueryPlan/BufferChunksTransform.h @@ -8,7 +8,11 @@ namespace DB class BufferChunksTransform : public IProcessor { public: - BufferChunksTransform(const Block & header_, size_t max_rows_to_buffer_, size_t limit_); + BufferChunksTransform( + const Block & header_, + size_t max_rows_to_buffer_, + size_t max_bytes_to_buffer_, + size_t limit_); Status prepare() override; String getName() const override { return "BufferChunks"; } @@ -20,10 +24,12 @@ private: OutputPort & output; size_t max_rows_to_buffer; + size_t max_bytes_to_buffer; size_t limit; std::queue chunks; size_t num_buffered_rows = 0; + size_t num_buffered_bytes = 0; size_t num_processed_rows = 0; }; diff --git a/src/Processors/QueryPlan/SortingStep.cpp b/src/Processors/QueryPlan/SortingStep.cpp index a06910bef7f..1c40f84d23d 100644 --- a/src/Processors/QueryPlan/SortingStep.cpp +++ b/src/Processors/QueryPlan/SortingStep.cpp @@ -250,7 +250,7 @@ void SortingStep::mergingSorted(QueryPipelineBuilder & pipeline, const SortDescr { pipeline.addSimpleTransform([&](const Block & header) { - return std::make_shared(header, sort_settings.max_block_bytes, limit_); + return std::make_shared(header, sort_settings.max_block_size, sort_settings.max_block_bytes, limit_); }); } From cc45847b9f75ecf0c19c32861a48d8e50f833f78 Mon Sep 17 00:00:00 2001 From: Nikita Taranov Date: Tue, 18 Jun 2024 19:14:28 +0100 Subject: [PATCH 064/417] new check --- src/Storages/MergeTree/ParallelReplicasReadingCoordinator.cpp | 3 +++ 1 file changed, 3 insertions(+) diff --git a/src/Storages/MergeTree/ParallelReplicasReadingCoordinator.cpp b/src/Storages/MergeTree/ParallelReplicasReadingCoordinator.cpp index 79c0e6ad262..326b4455596 100644 --- a/src/Storages/MergeTree/ParallelReplicasReadingCoordinator.cpp +++ b/src/Storages/MergeTree/ParallelReplicasReadingCoordinator.cpp @@ -443,6 +443,9 @@ void DefaultCoordinator::doHandleInitialAllRangesAnnouncement(InitialAllRangesAn ErrorCodes::LOGICAL_ERROR, "Replica number ({}) is bigger than total replicas count ({})", replica_num, stats.size()); ++stats[replica_num].number_of_requests; + + if (replica_status[replica_num].is_announcement_received) + throw Exception(ErrorCodes::LOGICAL_ERROR, "Duplicate announcement received for replica number {}", replica_num); replica_status[replica_num].is_announcement_received = true; LOG_DEBUG(log, "Sent initial requests: {} Replicas count: {}", sent_initial_requests, replicas_count); From a7ffb0e8148b0e1cd3abe05f5320af62728f65b1 Mon Sep 17 00:00:00 2001 From: Nikita Taranov Date: Tue, 18 Jun 2024 19:14:46 +0100 Subject: [PATCH 065/417] add test --- ...03173_parallel_replicas_join_bug.reference | 7 ++ .../03173_parallel_replicas_join_bug.sql | 67 +++++++++++++++++++ 2 files changed, 74 insertions(+) create mode 100644 tests/queries/0_stateless/03173_parallel_replicas_join_bug.reference create mode 100644 tests/queries/0_stateless/03173_parallel_replicas_join_bug.sql diff --git a/tests/queries/0_stateless/03173_parallel_replicas_join_bug.reference b/tests/queries/0_stateless/03173_parallel_replicas_join_bug.reference new file mode 100644 index 00000000000..b23d6b02bc1 --- /dev/null +++ b/tests/queries/0_stateless/03173_parallel_replicas_join_bug.reference @@ -0,0 +1,7 @@ +a1451105-722e-4fe7-bfaa-65ad2ae249c2 whatever +a1451105-722e-4fe7-bfaa-65ad2ae249c2 whatever +a1451105-722e-4fe7-bfaa-65ad2ae249c2 +a1451105-722e-4fe7-bfaa-65ad2ae249c2 +a1451105-722e-4fe7-bfaa-65ad2ae249c2 +a1451105-722e-4fe7-bfaa-65ad2ae249c2 +a1451105-722e-4fe7-bfaa-65ad2ae249c2 diff --git a/tests/queries/0_stateless/03173_parallel_replicas_join_bug.sql b/tests/queries/0_stateless/03173_parallel_replicas_join_bug.sql new file mode 100644 index 00000000000..3dee67fbf3f --- /dev/null +++ b/tests/queries/0_stateless/03173_parallel_replicas_join_bug.sql @@ -0,0 +1,67 @@ +CREATE TABLE ids (id UUID, whatever String) Engine=MergeTree ORDER BY tuple(); +INSERT INTO ids VALUES ('a1451105-722e-4fe7-bfaa-65ad2ae249c2', 'whatever'); + +CREATE TABLE data (id UUID, event_time DateTime, status String) Engine=MergeTree ORDER BY tuple(); +INSERT INTO data VALUES ('a1451105-722e-4fe7-bfaa-65ad2ae249c2', '2000-01-01', 'CREATED'); + +CREATE TABLE data2 (id UUID, event_time DateTime, status String) Engine=MergeTree ORDER BY tuple(); +INSERT INTO data2 VALUES ('a1451105-722e-4fe7-bfaa-65ad2ae249c2', '2000-01-02', 'CREATED'); + +SET allow_experimental_analyzer = 1, cluster_for_parallel_replicas = 'parallel_replicas', max_parallel_replicas = 10, allow_experimental_parallel_reading_from_replicas = 2, parallel_replicas_for_non_replicated_merge_tree = 1, max_threads = 1; + +SELECT + id, + whatever +FROM ids AS l +INNER JOIN view( + SELECT * + FROM merge(currentDatabase(), 'data*') +) AS s ON l.id = s.id +WHERE status IN ['CREATED', 'CREATING'] +ORDER BY event_time DESC; + +with +results1 as ( + SELECT id + FROM data t1 + inner join ids t2 + on t1.id = t2.id +), +results2 as ( + SELECT id + FROM ids t1 + inner join data t2 + on t1.id = t2.id +) +select * from results1 union all select * from results2; + +with +results1 as ( + SELECT id + FROM data t1 + inner join ids t2 + on t1.id = t2.id +), +results2 as ( + SELECT id + FROM ids t1 + inner join data t2 + on t1.id = t2.id +) +select * from results1 t1 inner join results2 t2 using (id); + +with +results1 as ( + SELECT t1.id + FROM data t1 + inner join ids t2 on t1.id = t2.id + left join data t3 on t2.id = t3.id +), +results2 as ( + SELECT id + FROM ids t1 + inner join data t2 + on t1.id = t2.id +) +select * from results1 union all select * from results2; + From ff9c64fa4e83f48c343ea920ec88ccb770b113d1 Mon Sep 17 00:00:00 2001 From: Nikita Taranov Date: Tue, 18 Jun 2024 21:00:44 +0100 Subject: [PATCH 066/417] fix --- src/Planner/PlannerJoinTree.cpp | 13 ++++++++++++- 1 file changed, 12 insertions(+), 1 deletion(-) diff --git a/src/Planner/PlannerJoinTree.cpp b/src/Planner/PlannerJoinTree.cpp index 6ec460b0894..86faec29760 100644 --- a/src/Planner/PlannerJoinTree.cpp +++ b/src/Planner/PlannerJoinTree.cpp @@ -857,12 +857,23 @@ JoinTreeQueryPlan buildQueryPlanForTableExpression(QueryTreeNodePtr table_expres from_stage = storage->getQueryProcessingStage( query_context, select_query_options.to_stage, storage_snapshot, table_expression_query_info); + auto context_for_read = Context::createCopy(query_context); + + /// It is just a safety check needed until we have a proper sending plan to replicas. + /// If we have a non-trivial storage like View it might create its own Planner inside read(), run findTableForParallelReplicas() + /// and find some other table that might be used for reading with parallel replicas. It will lead to errors. + const bool other_table_already_chosen_for_reading_with_parallel_replicas + = planner_context->getGlobalPlannerContext()->parallel_replicas_table + && !table_expression_query_info.analyzer_can_use_parallel_replicas_on_follower; + if (other_table_already_chosen_for_reading_with_parallel_replicas) + context_for_read->setSetting("allow_experimental_parallel_reading_from_replicas", Field(0)); + storage->read( query_plan, columns_names, storage_snapshot, table_expression_query_info, - query_context, + context_for_read, from_stage, max_block_size, max_streams); From 0791677268446b6b9aeb42fd4d7e01db91bc42c8 Mon Sep 17 00:00:00 2001 From: Nikita Taranov Date: Tue, 18 Jun 2024 22:35:46 +0100 Subject: [PATCH 067/417] fix? --- src/Planner/PlannerJoinTree.cpp | 6 ++---- 1 file changed, 2 insertions(+), 4 deletions(-) diff --git a/src/Planner/PlannerJoinTree.cpp b/src/Planner/PlannerJoinTree.cpp index 86faec29760..0af7bfea0b0 100644 --- a/src/Planner/PlannerJoinTree.cpp +++ b/src/Planner/PlannerJoinTree.cpp @@ -857,8 +857,6 @@ JoinTreeQueryPlan buildQueryPlanForTableExpression(QueryTreeNodePtr table_expres from_stage = storage->getQueryProcessingStage( query_context, select_query_options.to_stage, storage_snapshot, table_expression_query_info); - auto context_for_read = Context::createCopy(query_context); - /// It is just a safety check needed until we have a proper sending plan to replicas. /// If we have a non-trivial storage like View it might create its own Planner inside read(), run findTableForParallelReplicas() /// and find some other table that might be used for reading with parallel replicas. It will lead to errors. @@ -866,14 +864,14 @@ JoinTreeQueryPlan buildQueryPlanForTableExpression(QueryTreeNodePtr table_expres = planner_context->getGlobalPlannerContext()->parallel_replicas_table && !table_expression_query_info.analyzer_can_use_parallel_replicas_on_follower; if (other_table_already_chosen_for_reading_with_parallel_replicas) - context_for_read->setSetting("allow_experimental_parallel_reading_from_replicas", Field(0)); + planner_context->getMutableQueryContext()->setSetting("allow_experimental_parallel_reading_from_replicas", Field(0)); storage->read( query_plan, columns_names, storage_snapshot, table_expression_query_info, - context_for_read, + query_context, from_stage, max_block_size, max_streams); From ed44e4cf6b9f66fed04a4267b72cb375c9eae295 Mon Sep 17 00:00:00 2001 From: Nikita Taranov Date: Tue, 18 Jun 2024 23:17:50 +0100 Subject: [PATCH 068/417] fix test --- ...ql => 03173_parallel_replicas_join_bug.sh} | 26 +++++++++++++------ 1 file changed, 18 insertions(+), 8 deletions(-) rename tests/queries/0_stateless/{03173_parallel_replicas_join_bug.sql => 03173_parallel_replicas_join_bug.sh} (59%) mode change 100644 => 100755 diff --git a/tests/queries/0_stateless/03173_parallel_replicas_join_bug.sql b/tests/queries/0_stateless/03173_parallel_replicas_join_bug.sh old mode 100644 new mode 100755 similarity index 59% rename from tests/queries/0_stateless/03173_parallel_replicas_join_bug.sql rename to tests/queries/0_stateless/03173_parallel_replicas_join_bug.sh index 3dee67fbf3f..4638609b00c --- a/tests/queries/0_stateless/03173_parallel_replicas_join_bug.sql +++ b/tests/queries/0_stateless/03173_parallel_replicas_join_bug.sh @@ -1,12 +1,22 @@ -CREATE TABLE ids (id UUID, whatever String) Engine=MergeTree ORDER BY tuple(); -INSERT INTO ids VALUES ('a1451105-722e-4fe7-bfaa-65ad2ae249c2', 'whatever'); +#!/usr/bin/env bash -CREATE TABLE data (id UUID, event_time DateTime, status String) Engine=MergeTree ORDER BY tuple(); -INSERT INTO data VALUES ('a1451105-722e-4fe7-bfaa-65ad2ae249c2', '2000-01-01', 'CREATED'); +CURDIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd) +# shellcheck source=../shell_config.sh +. "$CURDIR"/../shell_config.sh -CREATE TABLE data2 (id UUID, event_time DateTime, status String) Engine=MergeTree ORDER BY tuple(); -INSERT INTO data2 VALUES ('a1451105-722e-4fe7-bfaa-65ad2ae249c2', '2000-01-02', 'CREATED'); +$CLICKHOUSE_CLIENT -nq " + CREATE TABLE ids (id UUID, whatever String) Engine=MergeTree ORDER BY tuple(); + INSERT INTO ids VALUES ('a1451105-722e-4fe7-bfaa-65ad2ae249c2', 'whatever'); + + CREATE TABLE data (id UUID, event_time DateTime, status String) Engine=MergeTree ORDER BY tuple(); + INSERT INTO data VALUES ('a1451105-722e-4fe7-bfaa-65ad2ae249c2', '2000-01-01', 'CREATED'); + + CREATE TABLE data2 (id UUID, event_time DateTime, status String) Engine=MergeTree ORDER BY tuple(); + INSERT INTO data2 VALUES ('a1451105-722e-4fe7-bfaa-65ad2ae249c2', '2000-01-02', 'CREATED'); +" + +$CLICKHOUSE_CLIENT -nq " SET allow_experimental_analyzer = 1, cluster_for_parallel_replicas = 'parallel_replicas', max_parallel_replicas = 10, allow_experimental_parallel_reading_from_replicas = 2, parallel_replicas_for_non_replicated_merge_tree = 1, max_threads = 1; SELECT @@ -15,7 +25,7 @@ SELECT FROM ids AS l INNER JOIN view( SELECT * - FROM merge(currentDatabase(), 'data*') + FROM merge($CLICKHOUSE_DATABASE, 'data.*') ) AS s ON l.id = s.id WHERE status IN ['CREATED', 'CREATING'] ORDER BY event_time DESC; @@ -64,4 +74,4 @@ results2 as ( on t1.id = t2.id ) select * from results1 union all select * from results2; - +" From 96fd928bced6c14e9de98ad14b77b370ed14de8e Mon Sep 17 00:00:00 2001 From: Konstantin Morozov Date: Wed, 19 Jun 2024 08:59:48 +0000 Subject: [PATCH 069/417] remove unused var --- src/Storages/StorageReplicatedMergeTree.cpp | 1 - 1 file changed, 1 deletion(-) diff --git a/src/Storages/StorageReplicatedMergeTree.cpp b/src/Storages/StorageReplicatedMergeTree.cpp index a1f4a40a0ab..61a492c1f63 100644 --- a/src/Storages/StorageReplicatedMergeTree.cpp +++ b/src/Storages/StorageReplicatedMergeTree.cpp @@ -5341,7 +5341,6 @@ void StorageReplicatedMergeTree::shutdown(bool) if (shutdown_called.exchange(true)) return; - const auto storage_name = getStorageID().getNameForLogs(); LOG_TRACE(log, "Shutdown started"); flushAndPrepareForShutdown(); From 1ef9bad76fadd1aa22c047760012c7644e1394b8 Mon Sep 17 00:00:00 2001 From: Antonio Andelic Date: Wed, 19 Jun 2024 14:44:26 +0200 Subject: [PATCH 070/417] Custom key support for cluster_for_parallel_replicas --- src/Client/HedgedConnections.cpp | 7 +- src/Client/MultiplexedConnections.cpp | 22 +-- src/Client/MultiplexedConnections.h | 10 +- .../ClusterProxy/executeQuery.cpp | 136 ++++++++++++-- src/Interpreters/ClusterProxy/executeQuery.h | 39 +++- src/Interpreters/Context.cpp | 33 +++- src/Interpreters/Context.h | 12 +- src/Interpreters/InterpreterSelectQuery.cpp | 19 +- src/Planner/PlannerJoinTree.cpp | 119 +++++++----- src/QueryPipeline/RemoteQueryExecutor.cpp | 10 +- src/Storages/MergeTree/MergeTreeData.cpp | 17 +- src/Storages/StorageDistributed.cpp | 40 +--- src/Storages/StorageMergeTree.cpp | 50 +++-- src/Storages/StorageReplicatedMergeTree.cpp | 34 +++- .../test_parallel_replicas_custom_key/test.py | 150 ++++++++++++--- ...max_parallel_replicas_custom_key.reference | 173 ----------------- .../02535_max_parallel_replicas_custom_key.sh | 46 ----- ..._parallel_replicas_custom_key_mt.reference | 177 ++++++++++++++++++ ...535_max_parallel_replicas_custom_key_mt.sh | 54 ++++++ ...parallel_replicas_custom_key_rmt.reference | 177 ++++++++++++++++++ ...35_max_parallel_replicas_custom_key_rmt.sh | 54 ++++++ 21 files changed, 977 insertions(+), 402 deletions(-) delete mode 100644 tests/queries/0_stateless/02535_max_parallel_replicas_custom_key.reference delete mode 100755 tests/queries/0_stateless/02535_max_parallel_replicas_custom_key.sh create mode 100644 tests/queries/0_stateless/02535_max_parallel_replicas_custom_key_mt.reference create mode 100755 tests/queries/0_stateless/02535_max_parallel_replicas_custom_key_mt.sh create mode 100644 tests/queries/0_stateless/02535_max_parallel_replicas_custom_key_rmt.reference create mode 100755 tests/queries/0_stateless/02535_max_parallel_replicas_custom_key_rmt.sh diff --git a/src/Client/HedgedConnections.cpp b/src/Client/HedgedConnections.cpp index fb4d9a6bdcc..cd662f13ce3 100644 --- a/src/Client/HedgedConnections.cpp +++ b/src/Client/HedgedConnections.cpp @@ -187,15 +187,16 @@ void HedgedConnections::sendQuery( modified_settings.group_by_two_level_threshold_bytes = 0; } - const bool enable_sample_offset_parallel_processing = settings.max_parallel_replicas > 1 && settings.allow_experimental_parallel_reading_from_replicas == 0; + const bool enable_offset_parallel_processing = context->canUseOffsetParallelReplicas(); - if (offset_states.size() > 1 && enable_sample_offset_parallel_processing) + if (offset_states.size() > 1 && enable_offset_parallel_processing) { modified_settings.parallel_replicas_count = offset_states.size(); modified_settings.parallel_replica_offset = fd_to_replica_location[replica.packet_receiver->getFileDescriptor()].offset; } - replica.connection->sendQuery(timeouts, query, /* query_parameters */ {}, query_id, stage, &modified_settings, &client_info, with_pending_data, {}); + replica.connection->sendQuery( + timeouts, query, /* query_parameters */ {}, query_id, stage, &modified_settings, &client_info, with_pending_data, {}); replica.change_replica_timeout.setRelative(timeouts.receive_data_timeout); replica.packet_receiver->setTimeout(hedged_connections_factory.getConnectionTimeouts().receive_timeout); }; diff --git a/src/Client/MultiplexedConnections.cpp b/src/Client/MultiplexedConnections.cpp index 5d0fc8fd39e..8cafad5106e 100644 --- a/src/Client/MultiplexedConnections.cpp +++ b/src/Client/MultiplexedConnections.cpp @@ -2,6 +2,7 @@ #include #include +#include #include #include #include @@ -23,8 +24,8 @@ namespace ErrorCodes } -MultiplexedConnections::MultiplexedConnections(Connection & connection, const Settings & settings_, const ThrottlerPtr & throttler) - : settings(settings_) +MultiplexedConnections::MultiplexedConnections(Connection & connection, ContextPtr context_, const ThrottlerPtr & throttler) + : context(std::move(context_)), settings(context->getSettingsRef()) { connection.setThrottler(throttler); @@ -36,9 +37,9 @@ MultiplexedConnections::MultiplexedConnections(Connection & connection, const Se } -MultiplexedConnections::MultiplexedConnections(std::shared_ptr connection_ptr_, const Settings & settings_, const ThrottlerPtr & throttler) - : settings(settings_) - , connection_ptr(connection_ptr_) +MultiplexedConnections::MultiplexedConnections( + std::shared_ptr connection_ptr_, ContextPtr context_, const ThrottlerPtr & throttler) + : context(std::move(context_)), settings(context->getSettingsRef()), connection_ptr(connection_ptr_) { connection_ptr->setThrottler(throttler); @@ -50,9 +51,8 @@ MultiplexedConnections::MultiplexedConnections(std::shared_ptr conne } MultiplexedConnections::MultiplexedConnections( - std::vector && connections, - const Settings & settings_, const ThrottlerPtr & throttler) - : settings(settings_) + std::vector && connections, ContextPtr context_, const ThrottlerPtr & throttler) + : context(std::move(context_)), settings(context->getSettingsRef()) { /// If we didn't get any connections from pool and getMany() did not throw exceptions, this means that /// `skip_unavailable_shards` was set. Then just return. @@ -150,18 +150,18 @@ void MultiplexedConnections::sendQuery( } } - const bool enable_sample_offset_parallel_processing = settings.max_parallel_replicas > 1 && settings.allow_experimental_parallel_reading_from_replicas == 0; + const bool enable_offset_parallel_processing = context->canUseOffsetParallelReplicas(); size_t num_replicas = replica_states.size(); if (num_replicas > 1) { - if (enable_sample_offset_parallel_processing) + if (enable_offset_parallel_processing) /// Use multiple replicas for parallel query processing. modified_settings.parallel_replicas_count = num_replicas; for (size_t i = 0; i < num_replicas; ++i) { - if (enable_sample_offset_parallel_processing) + if (enable_offset_parallel_processing) modified_settings.parallel_replica_offset = i; replica_states[i].connection->sendQuery( diff --git a/src/Client/MultiplexedConnections.h b/src/Client/MultiplexedConnections.h index 9f7b47e0562..dec32e52d4f 100644 --- a/src/Client/MultiplexedConnections.h +++ b/src/Client/MultiplexedConnections.h @@ -10,7 +10,6 @@ namespace DB { - /** To retrieve data directly from multiple replicas (connections) from one shard * within a single thread. As a degenerate case, it can also work with one connection. * It is assumed that all functions except sendCancel are always executed in one thread. @@ -21,14 +20,12 @@ class MultiplexedConnections final : public IConnections { public: /// Accepts ready connection. - MultiplexedConnections(Connection & connection, const Settings & settings_, const ThrottlerPtr & throttler_); + MultiplexedConnections(Connection & connection, ContextPtr context_, const ThrottlerPtr & throttler_); /// Accepts ready connection and keep it alive before drain - MultiplexedConnections(std::shared_ptr connection_, const Settings & settings_, const ThrottlerPtr & throttler_); + MultiplexedConnections(std::shared_ptr connection_, ContextPtr context_, const ThrottlerPtr & throttler_); /// Accepts a vector of connections to replicas of one shard already taken from pool. - MultiplexedConnections( - std::vector && connections, - const Settings & settings_, const ThrottlerPtr & throttler_); + MultiplexedConnections(std::vector && connections, ContextPtr context_, const ThrottlerPtr & throttler_); void sendScalarsData(Scalars & data) override; void sendExternalTablesData(std::vector & data) override; @@ -86,6 +83,7 @@ private: /// Mark the replica as invalid. void invalidateReplica(ReplicaState & replica_state); + ContextPtr context; const Settings & settings; /// The current number of valid connections to the replicas of this shard. diff --git a/src/Interpreters/ClusterProxy/executeQuery.cpp b/src/Interpreters/ClusterProxy/executeQuery.cpp index 91c0c592f28..337eb21dade 100644 --- a/src/Interpreters/ClusterProxy/executeQuery.cpp +++ b/src/Interpreters/ClusterProxy/executeQuery.cpp @@ -8,23 +8,28 @@ #include #include #include +#include #include -#include -#include #include +#include +#include +#include +#include +#include #include +#include #include #include -#include #include +#include #include +#include #include #include #include -#include +#include #include -#include -#include +#include namespace DB { @@ -172,7 +177,7 @@ ContextMutablePtr updateSettingsAndClientInfoForCluster(const Cluster & cluster, /// in case of parallel replicas custom key use round robing load balancing /// so custom key partitions will be spread over nodes in round-robin fashion - if (context->canUseParallelReplicasCustomKey(cluster) && !settings.load_balancing.changed) + if (context->canUseParallelReplicasCustomKeyForCluster(cluster) && !settings.load_balancing.changed) { new_settings.load_balancing = LoadBalancing::ROUND_ROBIN; } @@ -180,6 +185,10 @@ ContextMutablePtr updateSettingsAndClientInfoForCluster(const Cluster & cluster, auto new_context = Context::createCopy(context); new_context->setSettings(new_settings); new_context->setClientInfo(new_client_info); + + if (context->canUseParallelReplicasCustomKeyForCluster(cluster)) + new_context->disableOffsetParallelReplicas(); + return new_context; } @@ -231,17 +240,56 @@ void executeQuery( LoggerPtr log, ContextPtr context, const SelectQueryInfo & query_info, + const ColumnsDescription & columns, const ExpressionActionsPtr & sharding_key_expr, const std::string & sharding_key_column_name, const DistributedSettings & distributed_settings, - AdditionalShardFilterGenerator shard_filter_generator, bool is_remote_function) { const Settings & settings = context->getSettingsRef(); + if (settings.max_distributed_depth && context->getClientInfo().distributed_depth >= settings.max_distributed_depth) throw Exception(ErrorCodes::TOO_LARGE_DISTRIBUTED_DEPTH, "Maximum distributed depth exceeded"); + /// Return directly (with correct header) if no shard to query. + if (query_info.getCluster()->getShardsInfo().empty()) + { + if (settings.allow_experimental_analyzer) + return; + + Pipe pipe(std::make_shared(header)); + auto read_from_pipe = std::make_unique(std::move(pipe)); + read_from_pipe->setStepDescription("Read from NullSource (Distributed)"); + query_plan.addStep(std::move(read_from_pipe)); + return; + } + + ClusterProxy::AdditionalShardFilterGenerator shard_filter_generator; + if (context->canUseParallelReplicasCustomKeyForCluster(*query_info.getCluster())) + { + if (auto custom_key_ast = parseCustomKeyForTable(settings.parallel_replicas_custom_key, *context)) + { + shard_filter_generator = + [my_custom_key_ast = std::move(custom_key_ast), + column_description = columns, + custom_key_type = settings.parallel_replicas_custom_key_filter_type.value, + custom_key_range_lower = settings.parallel_replicas_custom_key_range_lower.value, + custom_key_range_upper = settings.parallel_replicas_custom_key_range_upper.value, + query_context = context, + replica_count = query_info.getCluster()->getShardsInfo().front().per_replica_pools.size()](uint64_t replica_num) -> ASTPtr + { + return getCustomKeyFilterForParallelReplica( + replica_count, + replica_num - 1, + my_custom_key_ast, + {custom_key_type, custom_key_range_lower, custom_key_range_upper}, + column_description, + query_context); + }; + } + } + const ClusterPtr & not_optimized_cluster = query_info.cluster; std::vector plans; @@ -412,14 +460,7 @@ void executeQueryWithParallelReplicas( const auto & settings = context->getSettingsRef(); /// check cluster for parallel replicas - if (settings.cluster_for_parallel_replicas.value.empty()) - { - throw Exception( - ErrorCodes::CLUSTER_DOESNT_EXIST, - "Reading in parallel from replicas is enabled but cluster to execute query is not provided. Please set " - "'cluster_for_parallel_replicas' setting"); - } - auto not_optimized_cluster = context->getCluster(settings.cluster_for_parallel_replicas); + auto not_optimized_cluster = context->getClusterForParallelReplicas(); auto new_context = Context::createCopy(context); @@ -542,6 +583,69 @@ void executeQueryWithParallelReplicas( executeQueryWithParallelReplicas(query_plan, storage_id, header, processed_stage, modified_query_ast, context, storage_limits); } +void executeQueryWithParallelReplicasCustomKey( + QueryPlan & query_plan, + const StorageID & storage_id, + const SelectQueryInfo & query_info, + const ColumnsDescription & columns, + const StorageSnapshotPtr & snapshot, + QueryProcessingStage::Enum processed_stage, + const Block & header, + ContextPtr context) +{ + ColumnsDescriptionByShardNum columns_object; + if (hasDynamicSubcolumns(columns)) + columns_object = getExtendedObjectsOfRemoteTables(*query_info.cluster, storage_id, columns, context); + + ClusterProxy::SelectStreamFactory select_stream_factory + = ClusterProxy::SelectStreamFactory(header, columns_object, snapshot, processed_stage); + + ClusterProxy::executeQuery( + query_plan, + header, + processed_stage, + storage_id, + /*table_func_ptr=*/nullptr, + select_stream_factory, + getLogger("executeQueryWithParallelReplicasCustomKey"), + context, + query_info, + columns, + /*sharding_key_expr=*/nullptr, + /*sharding_key_column_name=*/{}, + /*distributed_settings=*/{}, + /*is_remote_function= */ false); +} + +void executeQueryWithParallelReplicasCustomKey( + QueryPlan & query_plan, + const StorageID & storage_id, + const SelectQueryInfo & query_info, + const ColumnsDescription & columns, + const StorageSnapshotPtr & snapshot, + QueryProcessingStage::Enum processed_stage, + const QueryTreeNodePtr & query_tree, + ContextPtr context) +{ + auto header = InterpreterSelectQueryAnalyzer::getSampleBlock(query_tree, context, SelectQueryOptions(processed_stage).analyze()); + executeQueryWithParallelReplicasCustomKey(query_plan, storage_id, query_info, columns, snapshot, processed_stage, header, context); +} + +void executeQueryWithParallelReplicasCustomKey( + QueryPlan & query_plan, + const StorageID & storage_id, + SelectQueryInfo query_info, + const ColumnsDescription & columns, + const StorageSnapshotPtr & snapshot, + QueryProcessingStage::Enum processed_stage, + const ASTPtr & query_ast, + ContextPtr context) +{ + auto header = InterpreterSelectQuery(query_ast, context, SelectQueryOptions(processed_stage).analyze()).getSampleBlock(); + query_info.query = ClusterProxy::rewriteSelectQuery( + context, query_info.query, storage_id.getDatabaseName(), storage_id.getTableName(), /*table_function_ptr=*/nullptr); + executeQueryWithParallelReplicasCustomKey(query_plan, storage_id, query_info, columns, snapshot, processed_stage, header, context); +} } } diff --git a/src/Interpreters/ClusterProxy/executeQuery.h b/src/Interpreters/ClusterProxy/executeQuery.h index 6548edf8939..cf60fc3f168 100644 --- a/src/Interpreters/ClusterProxy/executeQuery.h +++ b/src/Interpreters/ClusterProxy/executeQuery.h @@ -1,7 +1,7 @@ #pragma once -#include #include +#include #include namespace DB @@ -13,6 +13,11 @@ class Cluster; using ClusterPtr = std::shared_ptr; struct SelectQueryInfo; +class ColumnsDescription; +struct StorageSnapshot; + +using StorageSnapshotPtr = std::shared_ptr; + class Pipe; class QueryPlan; @@ -60,10 +65,10 @@ void executeQuery( LoggerPtr log, ContextPtr context, const SelectQueryInfo & query_info, + const ColumnsDescription & columns, const ExpressionActionsPtr & sharding_key_expr, const std::string & sharding_key_column_name, const DistributedSettings & distributed_settings, - AdditionalShardFilterGenerator shard_filter_generator, bool is_remote_function); void executeQueryWithParallelReplicas( @@ -91,6 +96,36 @@ void executeQueryWithParallelReplicas( const PlannerContextPtr & planner_context, ContextPtr context, std::shared_ptr storage_limits); + +void executeQueryWithParallelReplicasCustomKey( + QueryPlan & query_plan, + const StorageID & storage_id, + const SelectQueryInfo & query_info, + const ColumnsDescription & columns, + const StorageSnapshotPtr & snapshot, + QueryProcessingStage::Enum processed_stage, + const Block & header, + ContextPtr context); + +void executeQueryWithParallelReplicasCustomKey( + QueryPlan & query_plan, + const StorageID & storage_id, + const SelectQueryInfo & query_info, + const ColumnsDescription & columns, + const StorageSnapshotPtr & snapshot, + QueryProcessingStage::Enum processed_stage, + const QueryTreeNodePtr & query_tree, + ContextPtr context); + +void executeQueryWithParallelReplicasCustomKey( + QueryPlan & query_plan, + const StorageID & storage_id, + SelectQueryInfo query_info, + const ColumnsDescription & columns, + const StorageSnapshotPtr & snapshot, + QueryProcessingStage::Enum processed_stage, + const ASTPtr & query_ast, + ContextPtr context); } } diff --git a/src/Interpreters/Context.cpp b/src/Interpreters/Context.cpp index f4433cd8288..b091e73436b 100644 --- a/src/Interpreters/Context.cpp +++ b/src/Interpreters/Context.cpp @@ -5394,10 +5394,37 @@ bool Context::canUseParallelReplicasOnFollower() const return canUseTaskBasedParallelReplicas() && getClientInfo().collaborate_with_initiator; } -bool Context::canUseParallelReplicasCustomKey(const Cluster & cluster) const +bool Context::canUseParallelReplicasCustomKey() const { - return settings.max_parallel_replicas > 1 && getParallelReplicasMode() == Context::ParallelReplicasMode::CUSTOM_KEY - && cluster.getShardCount() == 1 && cluster.getShardsInfo()[0].getAllNodeCount() > 1; + return settings.max_parallel_replicas > 1 && getParallelReplicasMode() == Context::ParallelReplicasMode::CUSTOM_KEY; +} + +bool Context::canUseParallelReplicasCustomKeyForCluster(const Cluster & cluster) const +{ + return canUseParallelReplicasCustomKey() && cluster.getShardCount() == 1 && cluster.getShardsInfo()[0].getAllNodeCount() > 1; +} + +bool Context::canUseOffsetParallelReplicas() const +{ + return offset_parallel_replicas_enabled && settings.max_parallel_replicas > 1 + && getParallelReplicasMode() != Context::ParallelReplicasMode::READ_TASKS; +} + +void Context::disableOffsetParallelReplicas() +{ + offset_parallel_replicas_enabled = false; +} + +ClusterPtr Context::getClusterForParallelReplicas() const +{ + /// check cluster for parallel replicas + if (settings.cluster_for_parallel_replicas.value.empty()) + throw Exception( + ErrorCodes::CLUSTER_DOESNT_EXIST, + "Reading in parallel from replicas is enabled but cluster to execute query is not provided. Please set " + "'cluster_for_parallel_replicas' setting"); + + return getCluster(settings.cluster_for_parallel_replicas); } void Context::setPreparedSetsCache(const PreparedSetsCachePtr & cache) diff --git a/src/Interpreters/Context.h b/src/Interpreters/Context.h index 7c7b2e4ea64..5fec7b1c2c5 100644 --- a/src/Interpreters/Context.h +++ b/src/Interpreters/Context.h @@ -151,6 +151,8 @@ class AsyncLoader; struct TemporaryTableHolder; using TemporaryTablesMapping = std::map>; +using ClusterPtr = std::shared_ptr; + class LoadTask; using LoadTaskPtr = std::shared_ptr; using LoadTaskPtrs = std::vector; @@ -436,6 +438,8 @@ protected: /// mutation tasks of one mutation executed against different parts of the same table. PreparedSetsCachePtr prepared_sets_cache; + bool offset_parallel_replicas_enabled = true; + public: /// Some counters for current query execution. /// Most of them are workarounds and should be removed in the future. @@ -1273,7 +1277,13 @@ public: bool canUseTaskBasedParallelReplicas() const; bool canUseParallelReplicasOnInitiator() const; bool canUseParallelReplicasOnFollower() const; - bool canUseParallelReplicasCustomKey(const Cluster & cluster) const; + bool canUseParallelReplicasCustomKey() const; + bool canUseParallelReplicasCustomKeyForCluster(const Cluster & cluster) const; + bool canUseOffsetParallelReplicas() const; + + void disableOffsetParallelReplicas(); + + ClusterPtr getClusterForParallelReplicas() const; enum class ParallelReplicasMode : uint8_t { diff --git a/src/Interpreters/InterpreterSelectQuery.cpp b/src/Interpreters/InterpreterSelectQuery.cpp index 8e072779b53..c7688b3471d 100644 --- a/src/Interpreters/InterpreterSelectQuery.cpp +++ b/src/Interpreters/InterpreterSelectQuery.cpp @@ -566,7 +566,7 @@ InterpreterSelectQuery::InterpreterSelectQuery( settings.additional_table_filters, joined_tables.tablesWithColumns().front().table, *context); ASTPtr parallel_replicas_custom_filter_ast = nullptr; - if (storage && context->getParallelReplicasMode() == Context::ParallelReplicasMode::CUSTOM_KEY && !joined_tables.tablesWithColumns().empty()) + if (storage && context->canUseParallelReplicasCustomKey() && !joined_tables.tablesWithColumns().empty()) { if (settings.parallel_replicas_count > 1) { @@ -587,16 +587,23 @@ InterpreterSelectQuery::InterpreterSelectQuery( else if (settings.parallel_replica_offset > 0) { throw Exception( - ErrorCodes::BAD_ARGUMENTS, - "Parallel replicas processing with custom_key has been requested " - "(setting 'max_parallel_replicas') but the table does not have custom_key defined for it " - "or it's invalid (settings `parallel_replicas_custom_key`)"); + ErrorCodes::BAD_ARGUMENTS, + "Parallel replicas processing with custom_key has been requested " + "(setting 'max_parallel_replicas') but the table does not have custom_key defined for it " + "or it's invalid (settings `parallel_replicas_custom_key`)"); } } else if (auto * distributed = dynamic_cast(storage.get()); - distributed && context->canUseParallelReplicasCustomKey(*distributed->getCluster())) + distributed && context->canUseParallelReplicasCustomKeyForCluster(*distributed->getCluster())) { context->setSetting("distributed_group_by_no_merge", 2); + context->setSetting("prefer_localhost_replica", Field(0)); + } + else if ( + storage->isMergeTree() && (storage->supportsReplication() || settings.parallel_replicas_for_non_replicated_merge_tree) + && context->canUseParallelReplicasCustomKeyForCluster(*context->getClusterForParallelReplicas())) + { + context->setSetting("prefer_localhost_replica", Field(0)); } } diff --git a/src/Planner/PlannerJoinTree.cpp b/src/Planner/PlannerJoinTree.cpp index 6ec460b0894..1dcbd87f495 100644 --- a/src/Planner/PlannerJoinTree.cpp +++ b/src/Planner/PlannerJoinTree.cpp @@ -81,6 +81,7 @@ namespace ErrorCodes extern const int TOO_MANY_COLUMNS; extern const int UNSUPPORTED_METHOD; extern const int BAD_ARGUMENTS; + extern const int CLUSTER_DOESNT_EXIST; } namespace @@ -834,7 +835,7 @@ JoinTreeQueryPlan buildQueryPlanForTableExpression(QueryTreeNodePtr table_expres if (row_policy_filter_info.actions) table_expression_data.setRowLevelFilterActions(row_policy_filter_info.actions); - if (query_context->getParallelReplicasMode() == Context::ParallelReplicasMode::CUSTOM_KEY) + if (query_context->canUseParallelReplicasCustomKey()) { if (settings.parallel_replicas_count > 1) { @@ -843,9 +844,10 @@ JoinTreeQueryPlan buildQueryPlanForTableExpression(QueryTreeNodePtr table_expres add_filter(parallel_replicas_custom_key_filter_info, "Parallel replicas custom key filter"); } else if (auto * distributed = typeid_cast(storage.get()); - distributed && query_context->canUseParallelReplicasCustomKey(*distributed->getCluster())) + distributed && query_context->canUseParallelReplicasCustomKeyForCluster(*distributed->getCluster())) { planner_context->getMutableQueryContext()->setSetting("distributed_group_by_no_merge", 2); + planner_context->getMutableQueryContext()->setSetting("prefer_localhost_replica", Field{0}); } } @@ -879,7 +881,7 @@ JoinTreeQueryPlan buildQueryPlanForTableExpression(QueryTreeNodePtr table_expres }; /// query_plan can be empty if there is nothing to read - if (query_plan.isInitialized() && parallel_replicas_enabled_for_storage(storage, settings) && query_context->canUseParallelReplicasOnInitiator()) + if (query_plan.isInitialized() && parallel_replicas_enabled_for_storage(storage, settings)) { // (1) find read step QueryPlan::Node * node = query_plan.getRootNode(); @@ -906,54 +908,79 @@ JoinTreeQueryPlan buildQueryPlanForTableExpression(QueryTreeNodePtr table_expres } chassert(reading); - - // (2) if it's ReadFromMergeTree - run index analysis and check number of rows to read - if (settings.parallel_replicas_min_number_of_rows_per_replica > 0) + if (query_context->canUseParallelReplicasCustomKey()) { - auto result_ptr = reading->selectRangesToRead(); - - UInt64 rows_to_read = result_ptr->selected_rows; - if (table_expression_query_info.limit > 0 && table_expression_query_info.limit < rows_to_read) - rows_to_read = table_expression_query_info.limit; - - if (max_block_size_limited && (max_block_size_limited < rows_to_read)) - rows_to_read = max_block_size_limited; - - const size_t number_of_replicas_to_use = rows_to_read / settings.parallel_replicas_min_number_of_rows_per_replica; - LOG_TRACE( - getLogger("Planner"), - "Estimated {} rows to read. It is enough work for {} parallel replicas", - rows_to_read, - number_of_replicas_to_use); - - if (number_of_replicas_to_use <= 1) + auto cluster = query_context->getClusterForParallelReplicas(); + if (query_context->canUseParallelReplicasCustomKeyForCluster(*cluster) + && query_context->getClientInfo().distributed_depth == 0) { - planner_context->getMutableQueryContext()->setSetting( - "allow_experimental_parallel_reading_from_replicas", Field(0)); - planner_context->getMutableQueryContext()->setSetting("max_parallel_replicas", UInt64{1}); - LOG_DEBUG(getLogger("Planner"), "Disabling parallel replicas because there aren't enough rows to read"); - } - else if (number_of_replicas_to_use < settings.max_parallel_replicas) - { - planner_context->getMutableQueryContext()->setSetting("max_parallel_replicas", number_of_replicas_to_use); - LOG_DEBUG(getLogger("Planner"), "Reducing the number of replicas to use to {}", number_of_replicas_to_use); + planner_context->getMutableQueryContext()->setSetting("prefer_localhost_replica", Field{0}); + auto modified_query_info = select_query_info; + modified_query_info.cluster = std::move(cluster); + from_stage = QueryProcessingStage::WithMergeableStateAfterAggregationAndLimit; + QueryPlan query_plan_parallel_replicas; + ClusterProxy::executeQueryWithParallelReplicasCustomKey( + query_plan_parallel_replicas, + storage->getStorageID(), + modified_query_info, + storage->getInMemoryMetadataPtr()->getColumns(), + storage_snapshot, + from_stage, + table_expression_query_info.query_tree, + query_context); + query_plan = std::move(query_plan_parallel_replicas); } } - - // (3) if parallel replicas still enabled - replace reading step - if (planner_context->getQueryContext()->canUseParallelReplicasOnInitiator()) + else if (query_context->canUseParallelReplicasOnInitiator()) { - from_stage = QueryProcessingStage::WithMergeableState; - QueryPlan query_plan_parallel_replicas; - ClusterProxy::executeQueryWithParallelReplicas( - query_plan_parallel_replicas, - storage->getStorageID(), - from_stage, - table_expression_query_info.query_tree, - table_expression_query_info.planner_context, - query_context, - table_expression_query_info.storage_limits); - query_plan = std::move(query_plan_parallel_replicas); + // (2) if it's ReadFromMergeTree - run index analysis and check number of rows to read + if (settings.parallel_replicas_min_number_of_rows_per_replica > 0) + { + auto result_ptr = reading->selectRangesToRead(); + + UInt64 rows_to_read = result_ptr->selected_rows; + if (table_expression_query_info.limit > 0 && table_expression_query_info.limit < rows_to_read) + rows_to_read = table_expression_query_info.limit; + + if (max_block_size_limited && (max_block_size_limited < rows_to_read)) + rows_to_read = max_block_size_limited; + + const size_t number_of_replicas_to_use = rows_to_read / settings.parallel_replicas_min_number_of_rows_per_replica; + LOG_TRACE( + getLogger("Planner"), + "Estimated {} rows to read. It is enough work for {} parallel replicas", + rows_to_read, + number_of_replicas_to_use); + + if (number_of_replicas_to_use <= 1) + { + planner_context->getMutableQueryContext()->setSetting( + "allow_experimental_parallel_reading_from_replicas", Field(0)); + planner_context->getMutableQueryContext()->setSetting("max_parallel_replicas", UInt64{1}); + LOG_DEBUG(getLogger("Planner"), "Disabling parallel replicas because there aren't enough rows to read"); + } + else if (number_of_replicas_to_use < settings.max_parallel_replicas) + { + planner_context->getMutableQueryContext()->setSetting("max_parallel_replicas", number_of_replicas_to_use); + LOG_DEBUG(getLogger("Planner"), "Reducing the number of replicas to use to {}", number_of_replicas_to_use); + } + } + + // (3) if parallel replicas still enabled - replace reading step + if (planner_context->getQueryContext()->canUseParallelReplicasOnInitiator()) + { + from_stage = QueryProcessingStage::WithMergeableState; + QueryPlan query_plan_parallel_replicas; + ClusterProxy::executeQueryWithParallelReplicas( + query_plan_parallel_replicas, + storage->getStorageID(), + from_stage, + table_expression_query_info.query_tree, + table_expression_query_info.planner_context, + query_context, + table_expression_query_info.storage_limits); + query_plan = std::move(query_plan_parallel_replicas); + } } } diff --git a/src/QueryPipeline/RemoteQueryExecutor.cpp b/src/QueryPipeline/RemoteQueryExecutor.cpp index 1686a101bde..bde8ce78f55 100644 --- a/src/QueryPipeline/RemoteQueryExecutor.cpp +++ b/src/QueryPipeline/RemoteQueryExecutor.cpp @@ -105,7 +105,7 @@ RemoteQueryExecutor::RemoteQueryExecutor( connection_entries.emplace_back(std::move(result.entry)); } - auto res = std::make_unique(std::move(connection_entries), current_settings, throttler); + auto res = std::make_unique(std::move(connection_entries), context, throttler); if (extension_ && extension_->replica_info) res->setReplicaInfo(*extension_->replica_info); @@ -127,7 +127,7 @@ RemoteQueryExecutor::RemoteQueryExecutor( { create_connections = [this, &connection, throttler, extension_](AsyncCallback) { - auto res = std::make_unique(connection, context->getSettingsRef(), throttler); + auto res = std::make_unique(connection, context, throttler); if (extension_ && extension_->replica_info) res->setReplicaInfo(*extension_->replica_info); return res; @@ -148,7 +148,7 @@ RemoteQueryExecutor::RemoteQueryExecutor( { create_connections = [this, connection_ptr, throttler, extension_](AsyncCallback) { - auto res = std::make_unique(connection_ptr, context->getSettingsRef(), throttler); + auto res = std::make_unique(connection_ptr, context, throttler); if (extension_ && extension_->replica_info) res->setReplicaInfo(*extension_->replica_info); return res; @@ -169,7 +169,7 @@ RemoteQueryExecutor::RemoteQueryExecutor( { create_connections = [this, connections_, throttler, extension_](AsyncCallback) mutable { - auto res = std::make_unique(std::move(connections_), context->getSettingsRef(), throttler); + auto res = std::make_unique(std::move(connections_), context, throttler); if (extension_ && extension_->replica_info) res->setReplicaInfo(*extension_->replica_info); return res; @@ -234,7 +234,7 @@ RemoteQueryExecutor::RemoteQueryExecutor( timeouts, current_settings, pool_mode, std::move(async_callback), skip_unavailable_endpoints, priority_func); } - auto res = std::make_unique(std::move(connection_entries), current_settings, throttler); + auto res = std::make_unique(std::move(connection_entries), context, throttler); if (extension && extension->replica_info) res->setReplicaInfo(*extension->replica_info); return res; diff --git a/src/Storages/MergeTree/MergeTreeData.cpp b/src/Storages/MergeTree/MergeTreeData.cpp index 89f39c65517..85e6020ff72 100644 --- a/src/Storages/MergeTree/MergeTreeData.cpp +++ b/src/Storages/MergeTree/MergeTreeData.cpp @@ -190,6 +190,7 @@ namespace ErrorCodes extern const int LIMIT_EXCEEDED; extern const int CANNOT_FORGET_PARTITION; extern const int DATA_TYPE_CANNOT_BE_USED_IN_KEY; + extern const int CLUSTER_DOESNT_EXIST; } static void checkSuspiciousIndices(const ASTFunction * index_function) @@ -7075,6 +7076,20 @@ QueryProcessingStage::Enum MergeTreeData::getQueryProcessingStage( /// with new analyzer, Planner make decision regarding parallel replicas usage, and so about processing stage on reading if (!query_context->getSettingsRef().allow_experimental_analyzer) { + const auto & settings = query_context->getSettingsRef(); + if (query_context->canUseParallelReplicasCustomKey()) + { + if (query_context->getClientInfo().distributed_depth > 0) + return QueryProcessingStage::FetchColumns; + + if (!settings.parallel_replicas_for_non_replicated_merge_tree) + return QueryProcessingStage::Enum::FetchColumns; + + if (to_stage >= QueryProcessingStage::WithMergeableState + && query_context->canUseParallelReplicasCustomKeyForCluster(*query_context->getClusterForParallelReplicas())) + return QueryProcessingStage::WithMergeableStateAfterAggregationAndLimit; + } + if (query_context->getClientInfo().collaborate_with_initiator) return QueryProcessingStage::Enum::FetchColumns; @@ -7086,7 +7101,7 @@ QueryProcessingStage::Enum MergeTreeData::getQueryProcessingStage( return QueryProcessingStage::Enum::WithMergeableState; /// For non-replicated MergeTree we allow them only if parallel_replicas_for_non_replicated_merge_tree is enabled - if (query_context->getSettingsRef().parallel_replicas_for_non_replicated_merge_tree) + if (settings.parallel_replicas_for_non_replicated_merge_tree) return QueryProcessingStage::Enum::WithMergeableState; } } diff --git a/src/Storages/StorageDistributed.cpp b/src/Storages/StorageDistributed.cpp index 5048ef4788e..c5721456f1a 100644 --- a/src/Storages/StorageDistributed.cpp +++ b/src/Storages/StorageDistributed.cpp @@ -426,7 +426,7 @@ QueryProcessingStage::Enum StorageDistributed::getQueryProcessingStage( query_info.cluster = cluster; - if (!local_context->canUseParallelReplicasCustomKey(*cluster)) + if (!local_context->canUseParallelReplicasCustomKeyForCluster(*cluster)) { if (nodes > 1 && settings.optimize_skip_unused_shards) { @@ -871,20 +871,6 @@ void StorageDistributed::read( remote_database, remote_table, remote_table_function_ptr); } - /// Return directly (with correct header) if no shard to query. - if (modified_query_info.getCluster()->getShardsInfo().empty()) - { - if (local_context->getSettingsRef().allow_experimental_analyzer) - return; - - Pipe pipe(std::make_shared(header)); - auto read_from_pipe = std::make_unique(std::move(pipe)); - read_from_pipe->setStepDescription("Read from NullSource (Distributed)"); - query_plan.addStep(std::move(read_from_pipe)); - - return; - } - const auto & snapshot_data = assert_cast(*storage_snapshot->data); ClusterProxy::SelectStreamFactory select_stream_factory = ClusterProxy::SelectStreamFactory( @@ -893,28 +879,6 @@ void StorageDistributed::read( storage_snapshot, processed_stage); - const auto & settings = local_context->getSettingsRef(); - - ClusterProxy::AdditionalShardFilterGenerator additional_shard_filter_generator; - if (local_context->canUseParallelReplicasCustomKey(*modified_query_info.getCluster())) - { - if (auto custom_key_ast = parseCustomKeyForTable(settings.parallel_replicas_custom_key, *local_context)) - { - additional_shard_filter_generator = - [my_custom_key_ast = std::move(custom_key_ast), - column_description = this->getInMemoryMetadataPtr()->columns, - custom_key_type = settings.parallel_replicas_custom_key_filter_type.value, - custom_key_range_lower = settings.parallel_replicas_custom_key_range_lower.value, - custom_key_range_upper = settings.parallel_replicas_custom_key_range_upper.value, - context = local_context, - replica_count = modified_query_info.getCluster()->getShardsInfo().front().per_replica_pools.size()](uint64_t replica_num) -> ASTPtr - { - return getCustomKeyFilterForParallelReplica( - replica_count, replica_num - 1, my_custom_key_ast, {custom_key_type, custom_key_range_lower, custom_key_range_upper}, column_description, context); - }; - } - } - ClusterProxy::executeQuery( query_plan, header, @@ -925,10 +889,10 @@ void StorageDistributed::read( log, local_context, modified_query_info, + getInMemoryMetadataPtr()->columns, sharding_key_expr, sharding_key_column_name, distributed_settings, - additional_shard_filter_generator, /* is_remote_function= */ static_cast(owned_cluster)); /// This is a bug, it is possible only when there is no shards to query, and this is handled earlier. diff --git a/src/Storages/StorageMergeTree.cpp b/src/Storages/StorageMergeTree.cpp index 9352f772ce1..95a0fe13567 100644 --- a/src/Storages/StorageMergeTree.cpp +++ b/src/Storages/StorageMergeTree.cpp @@ -1,5 +1,6 @@ #include "StorageMergeTree.h" #include "Core/QueryProcessingStage.h" +#include "Interpreters/ClientInfo.h" #include "Storages/MergeTree/IMergeTreeDataPart.h" #include @@ -66,6 +67,7 @@ namespace ErrorCodes extern const int ABORTED; extern const int SUPPORT_IS_DISABLED; extern const int TABLE_IS_READ_ONLY; + extern const int CLUSTER_DOESNT_EXIST; } namespace ActionLocks @@ -220,24 +222,44 @@ void StorageMergeTree::read( { ClusterProxy::executeQueryWithParallelReplicas( query_plan, getStorageID(), processed_stage, query_info.query, local_context, query_info.storage_limits); + return; } - else - { - const bool enable_parallel_reading = local_context->canUseParallelReplicasOnFollower() - && local_context->getSettingsRef().parallel_replicas_for_non_replicated_merge_tree - && (!local_context->getSettingsRef().allow_experimental_analyzer || query_info.analyzer_can_use_parallel_replicas_on_follower); - if (auto plan = reader.read( - column_names, + if (local_context->canUseParallelReplicasCustomKey() && settings.parallel_replicas_for_non_replicated_merge_tree + && !settings.allow_experimental_analyzer && local_context->getClientInfo().distributed_depth == 0) + { + if (auto cluster = local_context->getClusterForParallelReplicas(); + local_context->canUseParallelReplicasCustomKeyForCluster(*cluster)) + { + auto modified_query_info = query_info; + modified_query_info.cluster = std::move(cluster); + ClusterProxy::executeQueryWithParallelReplicasCustomKey( + query_plan, + getStorageID(), + std::move(modified_query_info), + getInMemoryMetadataPtr()->getColumns(), storage_snapshot, - query_info, - local_context, - max_block_size, - num_streams, - nullptr, - enable_parallel_reading)) - query_plan = std::move(*plan); + processed_stage, + query_info.query, + local_context); + return; + } } + + const bool enable_parallel_reading = local_context->canUseParallelReplicasOnFollower() + && local_context->getSettingsRef().parallel_replicas_for_non_replicated_merge_tree + && (!local_context->getSettingsRef().allow_experimental_analyzer || query_info.analyzer_can_use_parallel_replicas_on_follower); + + if (auto plan = reader.read( + column_names, + storage_snapshot, + query_info, + local_context, + max_block_size, + num_streams, + nullptr, + enable_parallel_reading)) + query_plan = std::move(*plan); } std::optional StorageMergeTree::totalRows(const Settings &) const diff --git a/src/Storages/StorageReplicatedMergeTree.cpp b/src/Storages/StorageReplicatedMergeTree.cpp index db58d0081c6..f82f4b7bb30 100644 --- a/src/Storages/StorageReplicatedMergeTree.cpp +++ b/src/Storages/StorageReplicatedMergeTree.cpp @@ -5460,13 +5460,39 @@ void StorageReplicatedMergeTree::read( /// 2. Do not read parts that have not yet been written to the quorum of the replicas. /// For this you have to synchronously go to ZooKeeper. if (settings.select_sequential_consistency) + { readLocalSequentialConsistencyImpl(query_plan, column_names, storage_snapshot, query_info, local_context, max_block_size, num_streams); + return; + } /// reading step for parallel replicas with new analyzer is built in Planner, so don't do it here - else if (local_context->canUseParallelReplicasOnInitiator() && !settings.allow_experimental_analyzer) + if (local_context->canUseParallelReplicasOnInitiator() && !settings.allow_experimental_analyzer) + { readParallelReplicasImpl(query_plan, column_names, query_info, local_context, processed_stage); - else - readLocalImpl(query_plan, column_names, storage_snapshot, query_info, local_context, max_block_size, num_streams); -} + return; + } + + if (local_context->canUseParallelReplicasCustomKey() && !settings.allow_experimental_analyzer + && local_context->getClientInfo().distributed_depth == 0) + { + if (auto cluster = local_context->getClusterForParallelReplicas(); + local_context->canUseParallelReplicasCustomKeyForCluster(*cluster)) + { + auto modified_query_info = query_info; + modified_query_info.cluster = std::move(cluster); + ClusterProxy::executeQueryWithParallelReplicasCustomKey( + query_plan, + getStorageID(), + std::move(modified_query_info), + getInMemoryMetadataPtr()->getColumns(), + storage_snapshot, + processed_stage, + query_info.query, + local_context); + return; + } + } + + readLocalImpl(query_plan, column_names, storage_snapshot, query_info, local_context, max_block_size, num_streams); } void StorageReplicatedMergeTree::readLocalSequentialConsistencyImpl( QueryPlan & query_plan, diff --git a/tests/integration/test_parallel_replicas_custom_key/test.py b/tests/integration/test_parallel_replicas_custom_key/test.py index 07a9e2badff..cb2c002f237 100644 --- a/tests/integration/test_parallel_replicas_custom_key/test.py +++ b/tests/integration/test_parallel_replicas_custom_key/test.py @@ -5,7 +5,10 @@ cluster = ClickHouseCluster(__file__) nodes = [ cluster.add_instance( - f"n{i}", main_configs=["configs/remote_servers.xml"], with_zookeeper=True + f"n{i}", + main_configs=["configs/remote_servers.xml"], + with_zookeeper=True, + macros={"replica": f"r{i}"}, ) for i in range(1, 5) ] @@ -22,32 +25,21 @@ def start_cluster(): def create_tables(cluster): n1 = nodes[0] - n1.query("DROP TABLE IF EXISTS dist_table") - n1.query(f"DROP TABLE IF EXISTS test_table ON CLUSTER {cluster}") - - n1.query( - f"CREATE TABLE test_table ON CLUSTER {cluster} (key UInt32, value String) Engine=MergeTree ORDER BY (key, sipHash64(value))" - ) - n1.query( - f""" - CREATE TABLE dist_table AS test_table - Engine=Distributed( - {cluster}, - currentDatabase(), - test_table, - rand() - ) - """ - ) + n1.query("DROP TABLE IF EXISTS dist_table SYNC") + n1.query(f"DROP TABLE IF EXISTS test_table ON CLUSTER {cluster} SYNC") -def insert_data(cluster, row_num): - create_tables(cluster) - n1 = nodes[0] - n1.query( - f"INSERT INTO dist_table SELECT number % 4, number FROM numbers({row_num})" +def insert_data(table_name, row_num, all_nodes=False): + query = ( + f"INSERT INTO {table_name} SELECT number % 4, number FROM numbers({row_num})" ) - n1.query("SYSTEM FLUSH DISTRIBUTED dist_table") + + if all_nodes: + for n in nodes: + n.query(query) + else: + n1 = nodes[0] + n1.query(query) @pytest.mark.parametrize("custom_key", ["sipHash64(key)", "key"]) @@ -56,12 +48,36 @@ def insert_data(cluster, row_num): "cluster", ["test_multiple_shards_multiple_replicas", "test_single_shard_multiple_replicas"], ) -def test_parallel_replicas_custom_key(start_cluster, cluster, custom_key, filter_type): +def test_parallel_replicas_custom_key_distributed( + start_cluster, cluster, custom_key, filter_type +): for node in nodes: node.rotate_logs() row_num = 1000 - insert_data(cluster, row_num) + + n1 = nodes[0] + n1.query(f"DROP TABLE IF EXISTS dist_table ON CLUSTER {cluster} SYNC") + n1.query(f"DROP TABLE IF EXISTS test_table_for_dist ON CLUSTER {cluster} SYNC") + n1.query( + f"CREATE TABLE test_table_for_dist ON CLUSTER {cluster} (key UInt32, value String) Engine=MergeTree ORDER BY (key, sipHash64(value))" + ) + + n1.query( + f""" + CREATE TABLE dist_table AS test_table_for_dist + Engine=Distributed( + {cluster}, + currentDatabase(), + test_table_for_dist, + rand() + ) + """ + ) + + insert_data("dist_table", row_num) + + n1.query("SYSTEM FLUSH DISTRIBUTED dist_table") expected_result = "" for i in range(4): @@ -72,7 +88,6 @@ def test_parallel_replicas_custom_key(start_cluster, cluster, custom_key, filter n1.query( "SELECT key, count() FROM dist_table GROUP BY key ORDER BY key", settings={ - "prefer_localhost_replica": 0, "max_parallel_replicas": 4, "parallel_replicas_custom_key": custom_key, "parallel_replicas_custom_key_filter_type": filter_type, @@ -87,3 +102,84 @@ def test_parallel_replicas_custom_key(start_cluster, cluster, custom_key, filter node.contains_in_log("Processing query on a replica using custom_key") for node in nodes ) + + +@pytest.mark.parametrize("custom_key", ["sipHash64(key)", "key"]) +@pytest.mark.parametrize("filter_type", ["default", "range"]) +@pytest.mark.parametrize( + "cluster", + ["test_single_shard_multiple_replicas"], +) +def test_parallel_replicas_custom_key_mergetree( + start_cluster, cluster, custom_key, filter_type +): + for node in nodes: + node.rotate_logs() + + row_num = 1000 + n1 = nodes[0] + n1.query(f"DROP TABLE IF EXISTS test_table_for_mt ON CLUSTER {cluster} SYNC") + n1.query( + f"CREATE TABLE test_table_for_mt ON CLUSTER {cluster} (key UInt32, value String) Engine=MergeTree ORDER BY (key, sipHash64(value))" + ) + + insert_data("test_table_for_mt", row_num, all_nodes=True) + + expected_result = "" + for i in range(4): + expected_result += f"{i}\t250\n" + + n1 = nodes[0] + assert ( + n1.query( + "SELECT key, count() FROM test_table_for_mt GROUP BY key ORDER BY key", + settings={ + "max_parallel_replicas": 4, + "parallel_replicas_custom_key": custom_key, + "parallel_replicas_custom_key_filter_type": filter_type, + "parallel_replicas_for_non_replicated_merge_tree": 1, + "cluster_for_parallel_replicas": cluster, + }, + ) + == expected_result + ) + + +@pytest.mark.parametrize("custom_key", ["sipHash64(key)", "key"]) +@pytest.mark.parametrize("filter_type", ["default", "range"]) +@pytest.mark.parametrize( + "cluster", + ["test_single_shard_multiple_replicas"], +) +def test_parallel_replicas_custom_key_replicatedmergetree( + start_cluster, cluster, custom_key, filter_type +): + for node in nodes: + node.rotate_logs() + + row_num = 1000 + n1 = nodes[0] + n1.query(f"DROP TABLE IF EXISTS test_table_for_rmt ON CLUSTER {cluster} SYNC") + n1.query( + f"CREATE TABLE test_table_for_rmt ON CLUSTER {cluster} (key UInt32, value String) Engine=ReplicatedMergeTree('/clickhouse/tables', '{{replica}}') ORDER BY (key, sipHash64(value))" + ) + + insert_data("test_table_for_rmt", row_num, all_nodes=False) + + expected_result = "" + for i in range(4): + expected_result += f"{i}\t250\n" + + n1 = nodes[0] + assert ( + n1.query( + "SELECT key, count() FROM test_table_for_rmt GROUP BY key ORDER BY key", + settings={ + "max_parallel_replicas": 4, + "parallel_replicas_custom_key": custom_key, + "parallel_replicas_custom_key_filter_type": filter_type, + "cluster_for_parallel_replicas": cluster, + }, + ) + == expected_result + ) diff --git a/tests/queries/0_stateless/02535_max_parallel_replicas_custom_key.reference b/tests/queries/0_stateless/02535_max_parallel_replicas_custom_key.reference deleted file mode 100644 index 8d0f56ba185..00000000000 --- a/tests/queries/0_stateless/02535_max_parallel_replicas_custom_key.reference +++ /dev/null @@ -1,173 +0,0 @@ -query='SELECT * FROM cluster(test_cluster_one_shard_three_replicas_localhost, currentDatabase(), 02535_custom_key)' with custom_key='sipHash64(x)' -filter_type='default' max_replicas=1 prefer_localhost_replica=0 -Hello -filter_type='default' max_replicas=2 prefer_localhost_replica=0 -Hello -filter_type='default' max_replicas=3 prefer_localhost_replica=0 -Hello -filter_type='range' max_replicas=1 prefer_localhost_replica=0 -Hello -filter_type='range' max_replicas=2 prefer_localhost_replica=0 -Hello -filter_type='range' max_replicas=3 prefer_localhost_replica=0 -Hello -filter_type='default' max_replicas=1 prefer_localhost_replica=1 -Hello -filter_type='default' max_replicas=2 prefer_localhost_replica=1 -Hello -filter_type='default' max_replicas=3 prefer_localhost_replica=1 -Hello -filter_type='range' max_replicas=1 prefer_localhost_replica=1 -Hello -filter_type='range' max_replicas=2 prefer_localhost_replica=1 -Hello -filter_type='range' max_replicas=3 prefer_localhost_replica=1 -Hello -query='SELECT y, count() FROM cluster(test_cluster_one_shard_three_replicas_localhost, currentDatabase(), 02535_custom_key) GROUP BY y ORDER BY y' with custom_key='y' -filter_type='default' max_replicas=1 prefer_localhost_replica=0 -0 334 -1 333 -2 333 -filter_type='default' max_replicas=2 prefer_localhost_replica=0 -0 334 -1 333 -2 333 -filter_type='default' max_replicas=3 prefer_localhost_replica=0 -0 334 -1 333 -2 333 -filter_type='range' max_replicas=1 prefer_localhost_replica=0 -0 334 -1 333 -2 333 -filter_type='range' max_replicas=2 prefer_localhost_replica=0 -0 334 -1 333 -2 333 -filter_type='range' max_replicas=3 prefer_localhost_replica=0 -0 334 -1 333 -2 333 -filter_type='default' max_replicas=1 prefer_localhost_replica=1 -0 334 -1 333 -2 333 -filter_type='default' max_replicas=2 prefer_localhost_replica=1 -0 334 -1 333 -2 333 -filter_type='default' max_replicas=3 prefer_localhost_replica=1 -0 334 -1 333 -2 333 -filter_type='range' max_replicas=1 prefer_localhost_replica=1 -0 334 -1 333 -2 333 -filter_type='range' max_replicas=2 prefer_localhost_replica=1 -0 334 -1 333 -2 333 -filter_type='range' max_replicas=3 prefer_localhost_replica=1 -0 334 -1 333 -2 333 -query='SELECT y, count() FROM cluster(test_cluster_one_shard_three_replicas_localhost, currentDatabase(), 02535_custom_key) GROUP BY y ORDER BY y' with custom_key='cityHash64(y)' -filter_type='default' max_replicas=1 prefer_localhost_replica=0 -0 334 -1 333 -2 333 -filter_type='default' max_replicas=2 prefer_localhost_replica=0 -0 334 -1 333 -2 333 -filter_type='default' max_replicas=3 prefer_localhost_replica=0 -0 334 -1 333 -2 333 -filter_type='range' max_replicas=1 prefer_localhost_replica=0 -0 334 -1 333 -2 333 -filter_type='range' max_replicas=2 prefer_localhost_replica=0 -0 334 -1 333 -2 333 -filter_type='range' max_replicas=3 prefer_localhost_replica=0 -0 334 -1 333 -2 333 -filter_type='default' max_replicas=1 prefer_localhost_replica=1 -0 334 -1 333 -2 333 -filter_type='default' max_replicas=2 prefer_localhost_replica=1 -0 334 -1 333 -2 333 -filter_type='default' max_replicas=3 prefer_localhost_replica=1 -0 334 -1 333 -2 333 -filter_type='range' max_replicas=1 prefer_localhost_replica=1 -0 334 -1 333 -2 333 -filter_type='range' max_replicas=2 prefer_localhost_replica=1 -0 334 -1 333 -2 333 -filter_type='range' max_replicas=3 prefer_localhost_replica=1 -0 334 -1 333 -2 333 -query='SELECT y, count() FROM cluster(test_cluster_one_shard_three_replicas_localhost, currentDatabase(), 02535_custom_key) GROUP BY y ORDER BY y' with custom_key='cityHash64(y) + 1' -filter_type='default' max_replicas=1 prefer_localhost_replica=0 -0 334 -1 333 -2 333 -filter_type='default' max_replicas=2 prefer_localhost_replica=0 -0 334 -1 333 -2 333 -filter_type='default' max_replicas=3 prefer_localhost_replica=0 -0 334 -1 333 -2 333 -filter_type='range' max_replicas=1 prefer_localhost_replica=0 -0 334 -1 333 -2 333 -filter_type='range' max_replicas=2 prefer_localhost_replica=0 -0 334 -1 333 -2 333 -filter_type='range' max_replicas=3 prefer_localhost_replica=0 -0 334 -1 333 -2 333 -filter_type='default' max_replicas=1 prefer_localhost_replica=1 -0 334 -1 333 -2 333 -filter_type='default' max_replicas=2 prefer_localhost_replica=1 -0 334 -1 333 -2 333 -filter_type='default' max_replicas=3 prefer_localhost_replica=1 -0 334 -1 333 -2 333 -filter_type='range' max_replicas=1 prefer_localhost_replica=1 -0 334 -1 333 -2 333 -filter_type='range' max_replicas=2 prefer_localhost_replica=1 -0 334 -1 333 -2 333 -filter_type='range' max_replicas=3 prefer_localhost_replica=1 -0 334 -1 333 -2 333 -1 diff --git a/tests/queries/0_stateless/02535_max_parallel_replicas_custom_key.sh b/tests/queries/0_stateless/02535_max_parallel_replicas_custom_key.sh deleted file mode 100755 index dccb680be42..00000000000 --- a/tests/queries/0_stateless/02535_max_parallel_replicas_custom_key.sh +++ /dev/null @@ -1,46 +0,0 @@ -#!/usr/bin/env bash -# Tags: no-parallel, long - -CURDIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd) -# shellcheck source=../shell_config.sh -. "$CURDIR"/../shell_config.sh - -function run_with_custom_key { - echo "query='$1' with custom_key='$2'" - for prefer_localhost_replica in 0 1; do - for filter_type in 'default' 'range'; do - for max_replicas in {1..3}; do - echo "filter_type='$filter_type' max_replicas=$max_replicas prefer_localhost_replica=$prefer_localhost_replica" - query="$1 SETTINGS max_parallel_replicas=$max_replicas\ - , parallel_replicas_custom_key='$2'\ - , parallel_replicas_custom_key_filter_type='$filter_type'\ - , prefer_localhost_replica=$prefer_localhost_replica" - $CLICKHOUSE_CLIENT --query="$query" - done - done - done -} - -$CLICKHOUSE_CLIENT --query="DROP TABLE IF EXISTS 02535_custom_key"; - -$CLICKHOUSE_CLIENT --query="CREATE TABLE 02535_custom_key (x String) ENGINE = MergeTree ORDER BY x"; -$CLICKHOUSE_CLIENT --query="INSERT INTO 02535_custom_key VALUES ('Hello')"; - -run_with_custom_key "SELECT * FROM cluster(test_cluster_one_shard_three_replicas_localhost, currentDatabase(), 02535_custom_key)" "sipHash64(x)" - -$CLICKHOUSE_CLIENT --query="DROP TABLE 02535_custom_key" - -$CLICKHOUSE_CLIENT --query="CREATE TABLE 02535_custom_key (x String, y UInt32) ENGINE = MergeTree ORDER BY cityHash64(x)" -$CLICKHOUSE_CLIENT --query="INSERT INTO 02535_custom_key SELECT toString(number), number % 3 FROM numbers(1000)" - -function run_count_with_custom_key { - run_with_custom_key "SELECT y, count() FROM cluster(test_cluster_one_shard_three_replicas_localhost, currentDatabase(), 02535_custom_key) GROUP BY y ORDER BY y" "$1" -} - -run_count_with_custom_key "y" -run_count_with_custom_key "cityHash64(y)" -run_count_with_custom_key "cityHash64(y) + 1" - -$CLICKHOUSE_CLIENT --query="SELECT count() FROM cluster(test_cluster_one_shard_three_replicas_localhost, currentDatabase(), 02535_custom_key) as t1 JOIN 02535_custom_key USING y" --allow_repeated_settings --parallel_replicas_custom_key="y" --send_logs_level="trace" 2>&1 | grep -Fac "JOINs are not supported with" - -$CLICKHOUSE_CLIENT --query="DROP TABLE 02535_custom_key" diff --git a/tests/queries/0_stateless/02535_max_parallel_replicas_custom_key_mt.reference b/tests/queries/0_stateless/02535_max_parallel_replicas_custom_key_mt.reference new file mode 100644 index 00000000000..1bb07f0d916 --- /dev/null +++ b/tests/queries/0_stateless/02535_max_parallel_replicas_custom_key_mt.reference @@ -0,0 +1,177 @@ +query='SELECT * FROM cluster(test_cluster_one_shard_three_replicas_localhost, currentDatabase(), 02535_custom_key_mt)' with custom_key='sipHash64(x)' +filter_type='default' max_replicas=1 +Hello +filter_type='default' max_replicas=2 +Hello +filter_type='default' max_replicas=3 +Hello +filter_type='range' max_replicas=1 +Hello +filter_type='range' max_replicas=2 +Hello +filter_type='range' max_replicas=3 +Hello +query='SELECT * FROM 02535_custom_key_mt' with custom_key='sipHash64(x)' +filter_type='default' max_replicas=1 +Hello +filter_type='default' max_replicas=2 +Hello +filter_type='default' max_replicas=3 +Hello +filter_type='range' max_replicas=1 +Hello +filter_type='range' max_replicas=2 +Hello +filter_type='range' max_replicas=3 +Hello +query='SELECT y, count() FROM cluster(test_cluster_one_shard_three_replicas_localhost, currentDatabase(), 02535_custom_key_mt) GROUP BY y ORDER BY y' with custom_key='y' +filter_type='default' max_replicas=1 +0 334 +1 333 +2 333 +filter_type='default' max_replicas=2 +0 334 +1 333 +2 333 +filter_type='default' max_replicas=3 +0 334 +1 333 +2 333 +filter_type='range' max_replicas=1 +0 334 +1 333 +2 333 +filter_type='range' max_replicas=2 +0 334 +1 333 +2 333 +filter_type='range' max_replicas=3 +0 334 +1 333 +2 333 +query='SELECT y, count() FROM cluster(test_cluster_one_shard_three_replicas_localhost, currentDatabase(), 02535_custom_key_mt) GROUP BY y ORDER BY y' with custom_key='cityHash64(y)' +filter_type='default' max_replicas=1 +0 334 +1 333 +2 333 +filter_type='default' max_replicas=2 +0 334 +1 333 +2 333 +filter_type='default' max_replicas=3 +0 334 +1 333 +2 333 +filter_type='range' max_replicas=1 +0 334 +1 333 +2 333 +filter_type='range' max_replicas=2 +0 334 +1 333 +2 333 +filter_type='range' max_replicas=3 +0 334 +1 333 +2 333 +query='SELECT y, count() FROM cluster(test_cluster_one_shard_three_replicas_localhost, currentDatabase(), 02535_custom_key_mt) GROUP BY y ORDER BY y' with custom_key='cityHash64(y) + 1' +filter_type='default' max_replicas=1 +0 334 +1 333 +2 333 +filter_type='default' max_replicas=2 +0 334 +1 333 +2 333 +filter_type='default' max_replicas=3 +0 334 +1 333 +2 333 +filter_type='range' max_replicas=1 +0 334 +1 333 +2 333 +filter_type='range' max_replicas=2 +0 334 +1 333 +2 333 +filter_type='range' max_replicas=3 +0 334 +1 333 +2 333 +query='SELECT y, count() FROM 02535_custom_key_mt GROUP BY y ORDER BY y' with custom_key='y' +filter_type='default' max_replicas=1 +0 334 +1 333 +2 333 +filter_type='default' max_replicas=2 +0 334 +1 333 +2 333 +filter_type='default' max_replicas=3 +0 334 +1 333 +2 333 +filter_type='range' max_replicas=1 +0 334 +1 333 +2 333 +filter_type='range' max_replicas=2 +0 334 +1 333 +2 333 +filter_type='range' max_replicas=3 +0 334 +1 333 +2 333 +query='SELECT y, count() FROM 02535_custom_key_mt GROUP BY y ORDER BY y' with custom_key='cityHash64(y)' +filter_type='default' max_replicas=1 +0 334 +1 333 +2 333 +filter_type='default' max_replicas=2 +0 334 +1 333 +2 333 +filter_type='default' max_replicas=3 +0 334 +1 333 +2 333 +filter_type='range' max_replicas=1 +0 334 +1 333 +2 333 +filter_type='range' max_replicas=2 +0 334 +1 333 +2 333 +filter_type='range' max_replicas=3 +0 334 +1 333 +2 333 +query='SELECT y, count() FROM 02535_custom_key_mt GROUP BY y ORDER BY y' with custom_key='cityHash64(y) + 1' +filter_type='default' max_replicas=1 +0 334 +1 333 +2 333 +filter_type='default' max_replicas=2 +0 334 +1 333 +2 333 +filter_type='default' max_replicas=3 +0 334 +1 333 +2 333 +filter_type='range' max_replicas=1 +0 334 +1 333 +2 333 +filter_type='range' max_replicas=2 +0 334 +1 333 +2 333 +filter_type='range' max_replicas=3 +0 334 +1 333 +2 333 +1 diff --git a/tests/queries/0_stateless/02535_max_parallel_replicas_custom_key_mt.sh b/tests/queries/0_stateless/02535_max_parallel_replicas_custom_key_mt.sh new file mode 100755 index 00000000000..fad43ea9070 --- /dev/null +++ b/tests/queries/0_stateless/02535_max_parallel_replicas_custom_key_mt.sh @@ -0,0 +1,54 @@ +#!/usr/bin/env bash +# Tags: no-parallel, long + +CURDIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd) +# shellcheck source=../shell_config.sh +. "$CURDIR"/../shell_config.sh + +function run_with_custom_key { + echo "query='$1' with custom_key='$2'" + for filter_type in 'default' 'range'; do + for max_replicas in {1..3}; do + echo "filter_type='$filter_type' max_replicas=$max_replicas" + query="$1 SETTINGS max_parallel_replicas=$max_replicas\ +, parallel_replicas_custom_key='$2'\ +, parallel_replicas_custom_key_filter_type='$filter_type'\ +, parallel_replicas_for_non_replicated_merge_tree=1 \ +, cluster_for_parallel_replicas='test_cluster_one_shard_three_replicas_localhost'" + $CLICKHOUSE_CLIENT --query="$query" + done + done +} + +$CLICKHOUSE_CLIENT --query="DROP TABLE IF EXISTS 02535_custom_key_mt"; + +$CLICKHOUSE_CLIENT --query="CREATE TABLE 02535_custom_key_mt (x String) ENGINE = MergeTree ORDER BY x"; +$CLICKHOUSE_CLIENT --query="INSERT INTO 02535_custom_key_mt VALUES ('Hello')"; + +run_with_custom_key "SELECT * FROM cluster(test_cluster_one_shard_three_replicas_localhost, currentDatabase(), 02535_custom_key_mt)" "sipHash64(x)" +run_with_custom_key "SELECT * FROM 02535_custom_key_mt" "sipHash64(x)" + +$CLICKHOUSE_CLIENT --query="DROP TABLE 02535_custom_key_mt" + +$CLICKHOUSE_CLIENT --query="CREATE TABLE 02535_custom_key_mt (x String, y UInt32) ENGINE = MergeTree ORDER BY cityHash64(x)" +$CLICKHOUSE_CLIENT --query="INSERT INTO 02535_custom_key_mt SELECT toString(number), number % 3 FROM numbers(1000)" + +function run_count_with_custom_key_distributed { + run_with_custom_key "SELECT y, count() FROM cluster(test_cluster_one_shard_three_replicas_localhost, currentDatabase(), 02535_custom_key_mt) GROUP BY y ORDER BY y" "$1" +} + +run_count_with_custom_key_distributed "y" +run_count_with_custom_key_distributed "cityHash64(y)" +run_count_with_custom_key_distributed "cityHash64(y) + 1" + +function run_count_with_custom_key_merge_tree { + run_with_custom_key "SELECT y, count() FROM 02535_custom_key_mt GROUP BY y ORDER BY y" "$1" +} + +run_count_with_custom_key_merge_tree "y" +run_count_with_custom_key_merge_tree "cityHash64(y)" +run_count_with_custom_key_merge_tree "cityHash64(y) + 1" + +$CLICKHOUSE_CLIENT --query="SELECT count() FROM cluster(test_cluster_one_shard_three_replicas_localhost, currentDatabase(), 02535_custom_key_mt) as t1 JOIN 02535_custom_key_mt USING y" --allow_repeated_settings --parallel_replicas_custom_key="y" --send_logs_level="trace" 2>&1 | grep -Fac "JOINs are not supported with" + +$CLICKHOUSE_CLIENT --query="DROP TABLE 02535_custom_key_mt" diff --git a/tests/queries/0_stateless/02535_max_parallel_replicas_custom_key_rmt.reference b/tests/queries/0_stateless/02535_max_parallel_replicas_custom_key_rmt.reference new file mode 100644 index 00000000000..c6526b506d3 --- /dev/null +++ b/tests/queries/0_stateless/02535_max_parallel_replicas_custom_key_rmt.reference @@ -0,0 +1,177 @@ +query='SELECT * FROM cluster(test_cluster_one_shard_three_replicas_localhost, currentDatabase(), 02535_custom_key_rmt)' with custom_key='sipHash64(x)' +filter_type='default' max_replicas=1 +Hello +filter_type='default' max_replicas=2 +Hello +filter_type='default' max_replicas=3 +Hello +filter_type='range' max_replicas=1 +Hello +filter_type='range' max_replicas=2 +Hello +filter_type='range' max_replicas=3 +Hello +query='SELECT * FROM 02535_custom_key_rmt' with custom_key='sipHash64(x)' +filter_type='default' max_replicas=1 +Hello +filter_type='default' max_replicas=2 +Hello +filter_type='default' max_replicas=3 +Hello +filter_type='range' max_replicas=1 +Hello +filter_type='range' max_replicas=2 +Hello +filter_type='range' max_replicas=3 +Hello +query='SELECT y, count() FROM cluster(test_cluster_one_shard_three_replicas_localhost, currentDatabase(), 02535_custom_key_rmt_hash) GROUP BY y ORDER BY y' with custom_key='y' +filter_type='default' max_replicas=1 +0 334 +1 333 +2 333 +filter_type='default' max_replicas=2 +0 334 +1 333 +2 333 +filter_type='default' max_replicas=3 +0 334 +1 333 +2 333 +filter_type='range' max_replicas=1 +0 334 +1 333 +2 333 +filter_type='range' max_replicas=2 +0 334 +1 333 +2 333 +filter_type='range' max_replicas=3 +0 334 +1 333 +2 333 +query='SELECT y, count() FROM cluster(test_cluster_one_shard_three_replicas_localhost, currentDatabase(), 02535_custom_key_rmt_hash) GROUP BY y ORDER BY y' with custom_key='cityHash64(y)' +filter_type='default' max_replicas=1 +0 334 +1 333 +2 333 +filter_type='default' max_replicas=2 +0 334 +1 333 +2 333 +filter_type='default' max_replicas=3 +0 334 +1 333 +2 333 +filter_type='range' max_replicas=1 +0 334 +1 333 +2 333 +filter_type='range' max_replicas=2 +0 334 +1 333 +2 333 +filter_type='range' max_replicas=3 +0 334 +1 333 +2 333 +query='SELECT y, count() FROM cluster(test_cluster_one_shard_three_replicas_localhost, currentDatabase(), 02535_custom_key_rmt_hash) GROUP BY y ORDER BY y' with custom_key='cityHash64(y) + 1' +filter_type='default' max_replicas=1 +0 334 +1 333 +2 333 +filter_type='default' max_replicas=2 +0 334 +1 333 +2 333 +filter_type='default' max_replicas=3 +0 334 +1 333 +2 333 +filter_type='range' max_replicas=1 +0 334 +1 333 +2 333 +filter_type='range' max_replicas=2 +0 334 +1 333 +2 333 +filter_type='range' max_replicas=3 +0 334 +1 333 +2 333 +query='SELECT y, count() FROM 02535_custom_key_rmt_hash GROUP BY y ORDER BY y' with custom_key='y' +filter_type='default' max_replicas=1 +0 334 +1 333 +2 333 +filter_type='default' max_replicas=2 +0 334 +1 333 +2 333 +filter_type='default' max_replicas=3 +0 334 +1 333 +2 333 +filter_type='range' max_replicas=1 +0 334 +1 333 +2 333 +filter_type='range' max_replicas=2 +0 334 +1 333 +2 333 +filter_type='range' max_replicas=3 +0 334 +1 333 +2 333 +query='SELECT y, count() FROM 02535_custom_key_rmt_hash GROUP BY y ORDER BY y' with custom_key='cityHash64(y)' +filter_type='default' max_replicas=1 +0 334 +1 333 +2 333 +filter_type='default' max_replicas=2 +0 334 +1 333 +2 333 +filter_type='default' max_replicas=3 +0 334 +1 333 +2 333 +filter_type='range' max_replicas=1 +0 334 +1 333 +2 333 +filter_type='range' max_replicas=2 +0 334 +1 333 +2 333 +filter_type='range' max_replicas=3 +0 334 +1 333 +2 333 +query='SELECT y, count() FROM 02535_custom_key_rmt_hash GROUP BY y ORDER BY y' with custom_key='cityHash64(y) + 1' +filter_type='default' max_replicas=1 +0 334 +1 333 +2 333 +filter_type='default' max_replicas=2 +0 334 +1 333 +2 333 +filter_type='default' max_replicas=3 +0 334 +1 333 +2 333 +filter_type='range' max_replicas=1 +0 334 +1 333 +2 333 +filter_type='range' max_replicas=2 +0 334 +1 333 +2 333 +filter_type='range' max_replicas=3 +0 334 +1 333 +2 333 +1 diff --git a/tests/queries/0_stateless/02535_max_parallel_replicas_custom_key_rmt.sh b/tests/queries/0_stateless/02535_max_parallel_replicas_custom_key_rmt.sh new file mode 100755 index 00000000000..6350f5027f9 --- /dev/null +++ b/tests/queries/0_stateless/02535_max_parallel_replicas_custom_key_rmt.sh @@ -0,0 +1,54 @@ +#!/usr/bin/env bash +# Tags: no-parallel, long + +CURDIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd) +# shellcheck source=../shell_config.sh +. "$CURDIR"/../shell_config.sh + +function run_with_custom_key { + echo "query='$1' with custom_key='$2'" + for filter_type in 'default' 'range'; do + for max_replicas in {1..3}; do + echo "filter_type='$filter_type' max_replicas=$max_replicas" + query="$1 SETTINGS max_parallel_replicas=$max_replicas\ +, parallel_replicas_custom_key='$2'\ +, parallel_replicas_custom_key_filter_type='$filter_type'\ +, parallel_replicas_for_non_replicated_merge_tree=1 \ +, cluster_for_parallel_replicas='test_cluster_one_shard_three_replicas_localhost'" + $CLICKHOUSE_CLIENT --query="$query" + done + done +} + +$CLICKHOUSE_CLIENT --query="DROP TABLE IF EXISTS 02535_custom_key_rmt"; + +$CLICKHOUSE_CLIENT --query="CREATE TABLE 02535_custom_key_rmt (x String) ENGINE = ReplicatedMergeTree('/clickhouse/tables/$CLICKHOUSE_TEST_ZOOKEEPER_PREFIX/test_02535', 'r1') ORDER BY x"; +$CLICKHOUSE_CLIENT --query="INSERT INTO 02535_custom_key_rmt VALUES ('Hello')"; + +run_with_custom_key "SELECT * FROM cluster(test_cluster_one_shard_three_replicas_localhost, currentDatabase(), 02535_custom_key_rmt)" "sipHash64(x)" +run_with_custom_key "SELECT * FROM 02535_custom_key_rmt" "sipHash64(x)" + +$CLICKHOUSE_CLIENT --query="DROP TABLE IF EXISTS 02535_custom_key_rmt_hash"; + +$CLICKHOUSE_CLIENT --query="CREATE TABLE 02535_custom_key_rmt_hash (x String, y UInt32) ENGINE = ReplicatedMergeTree('/clickhouse/tables/$CLICKHOUSE_TEST_ZOOKEEPER_PREFIX/test_02535_hash', 'r1') ORDER BY cityHash64(x)" +$CLICKHOUSE_CLIENT --query="INSERT INTO 02535_custom_key_rmt_hash SELECT toString(number), number % 3 FROM numbers(1000)" + +function run_count_with_custom_key { + run_with_custom_key "SELECT y, count() FROM cluster(test_cluster_one_shard_three_replicas_localhost, currentDatabase(), 02535_custom_key_rmt_hash) GROUP BY y ORDER BY y" "$1" +} + +run_count_with_custom_key "y" +run_count_with_custom_key "cityHash64(y)" +run_count_with_custom_key "cityHash64(y) + 1" + +function run_count_with_custom_key_merge_tree { + run_with_custom_key "SELECT y, count() FROM 02535_custom_key_rmt_hash GROUP BY y ORDER BY y" "$1" +} + +run_count_with_custom_key_merge_tree "y" +run_count_with_custom_key_merge_tree "cityHash64(y)" +run_count_with_custom_key_merge_tree "cityHash64(y) + 1" + +$CLICKHOUSE_CLIENT --query="SELECT count() FROM cluster(test_cluster_one_shard_three_replicas_localhost, currentDatabase(), 02535_custom_key_rmt_hash) as t1 JOIN 02535_custom_key_rmt_hash USING y" --allow_repeated_settings --parallel_replicas_custom_key="y" --send_logs_level="trace" 2>&1 | grep -Fac "JOINs are not supported with" + +$CLICKHOUSE_CLIENT --query="DROP TABLE 02535_custom_key_rmt_hash" From de747a66b8e84ba350e6ed092b8ae1bef0ac7748 Mon Sep 17 00:00:00 2001 From: divanik Date: Wed, 19 Jun 2024 12:56:24 +0000 Subject: [PATCH 071/417] Fix style check --- tests/queries/0_stateless/02802_clickhouse_disks_s3_copy.sh | 6 +++--- .../0_stateless/02980_s3_plain_DROP_TABLE_MergeTree.sh | 4 ++-- 2 files changed, 5 insertions(+), 5 deletions(-) diff --git a/tests/queries/0_stateless/02802_clickhouse_disks_s3_copy.sh b/tests/queries/0_stateless/02802_clickhouse_disks_s3_copy.sh index d317b2e8a1e..1638a3ff9c3 100755 --- a/tests/queries/0_stateless/02802_clickhouse_disks_s3_copy.sh +++ b/tests/queries/0_stateless/02802_clickhouse_disks_s3_copy.sh @@ -14,7 +14,7 @@ function run_test_for_disk() echo "$disk" - clickhouse-disks -C "$config" --disk "$disk" --query "write --path-from "$config" $CLICKHOUSE_DATABASE/test" + clickhouse-disks -C "$config" --disk "$disk" --query "write --path-from $config $CLICKHOUSE_DATABASE/test" clickhouse-disks -C "$config" --log-level test --disk "$disk" --query "copy $CLICKHOUSE_DATABASE/test $CLICKHOUSE_DATABASE/test.copy" |& { grep -o -e "Single part upload has completed." -e "Single operation copy has completed." } @@ -29,9 +29,9 @@ function run_test_copy_from_s3_to_s3(){ local disk_dest=$1 && shift echo "copy from $disk_src to $disk_dest" - clickhouse-disks -C "$config" --disk "$disk_src" --query "write --path-from "$config" $CLICKHOUSE_DATABASE/test" + clickhouse-disks -C "$config" --disk "$disk_src" --query "write --path-from $config $CLICKHOUSE_DATABASE/test" - clickhouse-disks -C "$config" --log-level test --query "copy --disk-from "$disk_src" --disk-to "$disk_dest" $CLICKHOUSE_DATABASE/test $CLICKHOUSE_DATABASE/test.copy" |& { + clickhouse-disks -C "$config" --log-level test --query "copy --disk-from $disk_src --disk-to $disk_dest $CLICKHOUSE_DATABASE/test $CLICKHOUSE_DATABASE/test.copy" |& { grep -o -e "Single part upload has completed." -e "Single operation copy has completed." } clickhouse-disks -C "$config" --disk "$disk_dest" --query "remove $CLICKHOUSE_DATABASE/test.copy/test" diff --git a/tests/queries/0_stateless/02980_s3_plain_DROP_TABLE_MergeTree.sh b/tests/queries/0_stateless/02980_s3_plain_DROP_TABLE_MergeTree.sh index e6427ab26f8..d543f7195a9 100755 --- a/tests/queries/0_stateless/02980_s3_plain_DROP_TABLE_MergeTree.sh +++ b/tests/queries/0_stateless/02980_s3_plain_DROP_TABLE_MergeTree.sh @@ -49,11 +49,11 @@ path=$($CLICKHOUSE_CLIENT -q "SELECT replace(data_paths[1], 's3_plain', '') FROM path=${path%/} echo "Files before DETACH TABLE" -clickhouse-disks -C "$config" --disk s3_plain_disk --query "list --recursive "${path:?}"" | tail -n+2 +clickhouse-disks -C "$config" --disk s3_plain_disk --query "list --recursive $path" | tail -n+2 $CLICKHOUSE_CLIENT -q "detach table data" echo "Files after DETACH TABLE" -clickhouse-disks -C "$config" --disk s3_plain_disk --query "list --recursive "$path"" | tail -n+2 +clickhouse-disks -C "$config" --disk s3_plain_disk --query "list --recursive $path" | tail -n+2 # metadata file is left $CLICKHOUSE_CLIENT --force_remove_data_recursively_on_drop=1 -q "drop database if exists $CLICKHOUSE_DATABASE" From 8fee4799b6fe004f637ae64a3d9f8a8fda6b4e36 Mon Sep 17 00:00:00 2001 From: Antonio Andelic Date: Wed, 19 Jun 2024 17:10:27 +0200 Subject: [PATCH 072/417] Fix style --- src/Interpreters/ClusterProxy/executeQuery.cpp | 1 - src/Planner/PlannerJoinTree.cpp | 1 - src/Storages/MergeTree/MergeTreeData.cpp | 1 - src/Storages/StorageMergeTree.cpp | 1 - 4 files changed, 4 deletions(-) diff --git a/src/Interpreters/ClusterProxy/executeQuery.cpp b/src/Interpreters/ClusterProxy/executeQuery.cpp index 337eb21dade..58517a09554 100644 --- a/src/Interpreters/ClusterProxy/executeQuery.cpp +++ b/src/Interpreters/ClusterProxy/executeQuery.cpp @@ -38,7 +38,6 @@ namespace ErrorCodes { extern const int TOO_LARGE_DISTRIBUTED_DEPTH; extern const int LOGICAL_ERROR; - extern const int CLUSTER_DOESNT_EXIST; extern const int UNEXPECTED_CLUSTER; } diff --git a/src/Planner/PlannerJoinTree.cpp b/src/Planner/PlannerJoinTree.cpp index 1dcbd87f495..2f7b9e9efaa 100644 --- a/src/Planner/PlannerJoinTree.cpp +++ b/src/Planner/PlannerJoinTree.cpp @@ -81,7 +81,6 @@ namespace ErrorCodes extern const int TOO_MANY_COLUMNS; extern const int UNSUPPORTED_METHOD; extern const int BAD_ARGUMENTS; - extern const int CLUSTER_DOESNT_EXIST; } namespace diff --git a/src/Storages/MergeTree/MergeTreeData.cpp b/src/Storages/MergeTree/MergeTreeData.cpp index 85e6020ff72..85f436a89c6 100644 --- a/src/Storages/MergeTree/MergeTreeData.cpp +++ b/src/Storages/MergeTree/MergeTreeData.cpp @@ -190,7 +190,6 @@ namespace ErrorCodes extern const int LIMIT_EXCEEDED; extern const int CANNOT_FORGET_PARTITION; extern const int DATA_TYPE_CANNOT_BE_USED_IN_KEY; - extern const int CLUSTER_DOESNT_EXIST; } static void checkSuspiciousIndices(const ASTFunction * index_function) diff --git a/src/Storages/StorageMergeTree.cpp b/src/Storages/StorageMergeTree.cpp index 95a0fe13567..05de77cae70 100644 --- a/src/Storages/StorageMergeTree.cpp +++ b/src/Storages/StorageMergeTree.cpp @@ -67,7 +67,6 @@ namespace ErrorCodes extern const int ABORTED; extern const int SUPPORT_IS_DISABLED; extern const int TABLE_IS_READ_ONLY; - extern const int CLUSTER_DOESNT_EXIST; } namespace ActionLocks From 6ebf8ab45b44f80a85194682f14750cbcb5885f2 Mon Sep 17 00:00:00 2001 From: Antonio Andelic Date: Thu, 20 Jun 2024 10:02:22 +0200 Subject: [PATCH 073/417] Fix no shards case --- .../ClusterProxy/executeQuery.cpp | 26 ++++---- src/Storages/StorageDistributed.cpp | 17 ++++- src/Storages/StorageMergeTree.cpp | 63 +++++++++---------- 3 files changed, 60 insertions(+), 46 deletions(-) diff --git a/src/Interpreters/ClusterProxy/executeQuery.cpp b/src/Interpreters/ClusterProxy/executeQuery.cpp index 58517a09554..47fbf35233a 100644 --- a/src/Interpreters/ClusterProxy/executeQuery.cpp +++ b/src/Interpreters/ClusterProxy/executeQuery.cpp @@ -251,19 +251,6 @@ void executeQuery( if (settings.max_distributed_depth && context->getClientInfo().distributed_depth >= settings.max_distributed_depth) throw Exception(ErrorCodes::TOO_LARGE_DISTRIBUTED_DEPTH, "Maximum distributed depth exceeded"); - /// Return directly (with correct header) if no shard to query. - if (query_info.getCluster()->getShardsInfo().empty()) - { - if (settings.allow_experimental_analyzer) - return; - - Pipe pipe(std::make_shared(header)); - auto read_from_pipe = std::make_unique(std::move(pipe)); - read_from_pipe->setStepDescription("Read from NullSource (Distributed)"); - query_plan.addStep(std::move(read_from_pipe)); - return; - } - ClusterProxy::AdditionalShardFilterGenerator shard_filter_generator; if (context->canUseParallelReplicasCustomKeyForCluster(*query_info.getCluster())) { @@ -592,6 +579,19 @@ void executeQueryWithParallelReplicasCustomKey( const Block & header, ContextPtr context) { + /// Return directly (with correct header) if no shard to query. + if (query_info.getCluster()->getShardsInfo().empty()) + { + if (context->getSettingsRef().allow_experimental_analyzer) + return; + + Pipe pipe(std::make_shared(header)); + auto read_from_pipe = std::make_unique(std::move(pipe)); + read_from_pipe->setStepDescription("Read from NullSource (Distributed)"); + query_plan.addStep(std::move(read_from_pipe)); + return; + } + ColumnsDescriptionByShardNum columns_object; if (hasDynamicSubcolumns(columns)) columns_object = getExtendedObjectsOfRemoteTables(*query_info.cluster, storage_id, columns, context); diff --git a/src/Storages/StorageDistributed.cpp b/src/Storages/StorageDistributed.cpp index ea087230a8e..1dd42d79d88 100644 --- a/src/Storages/StorageDistributed.cpp +++ b/src/Storages/StorageDistributed.cpp @@ -864,13 +864,28 @@ void StorageDistributed::read( header = InterpreterSelectQuery(modified_query_info.query, local_context, SelectQueryOptions(processed_stage).analyze()).getSampleBlock(); } - if (!local_context->getSettingsRef().allow_experimental_analyzer) + const auto & settings = local_context->getSettingsRef(); + if (!settings.allow_experimental_analyzer) { modified_query_info.query = ClusterProxy::rewriteSelectQuery( local_context, modified_query_info.query, remote_database, remote_table, remote_table_function_ptr); } + /// Return directly (with correct header) if no shard to query. + if (modified_query_info.getCluster()->getShardsInfo().empty()) + { + if (settings.allow_experimental_analyzer) + return; + + Pipe pipe(std::make_shared(header)); + auto read_from_pipe = std::make_unique(std::move(pipe)); + read_from_pipe->setStepDescription("Read from NullSource (Distributed)"); + query_plan.addStep(std::move(read_from_pipe)); + + return; + } + const auto & snapshot_data = assert_cast(*storage_snapshot->data); ClusterProxy::SelectStreamFactory select_stream_factory = ClusterProxy::SelectStreamFactory( diff --git a/src/Storages/StorageMergeTree.cpp b/src/Storages/StorageMergeTree.cpp index 05de77cae70..868c70499bc 100644 --- a/src/Storages/StorageMergeTree.cpp +++ b/src/Storages/StorageMergeTree.cpp @@ -1,52 +1,51 @@ -#include "StorageMergeTree.h" -#include "Core/QueryProcessingStage.h" -#include "Interpreters/ClientInfo.h" -#include "Storages/MergeTree/IMergeTreeDataPart.h" +#include #include #include -#include -#include #include +#include #include -#include "Common/Exception.h" -#include -#include -#include -#include -#include -#include -#include -#include -#include -#include -#include -#include #include +#include +#include +#include +#include +#include +#include +#include #include #include #include #include #include -#include #include +#include #include -#include -#include -#include -#include -#include -#include -#include -#include -#include -#include -#include -#include #include #include +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include #include +#include +#include +#include +#include +#include +#include +#include namespace DB From 6cce73bd99d1091a8bfd978e55a7e6d956836bc1 Mon Sep 17 00:00:00 2001 From: divanik Date: Thu, 20 Jun 2024 15:22:38 +0000 Subject: [PATCH 074/417] Fixed some tests --- src/Disks/DiskSelector.cpp | 3 ++ tests/integration/test_store_cleanup/test.py | 5 +++- ...n_DROP_TABLE_ReplicatedMergeTree.reference | 28 +++++++++---------- ...s3_plain_DROP_TABLE_ReplicatedMergeTree.sh | 4 +-- 4 files changed, 23 insertions(+), 17 deletions(-) diff --git a/src/Disks/DiskSelector.cpp b/src/Disks/DiskSelector.cpp index b187b491dc0..c29a24132c4 100644 --- a/src/Disks/DiskSelector.cpp +++ b/src/Disks/DiskSelector.cpp @@ -86,6 +86,7 @@ DiskSelectorPtr DiskSelector::updateFromConfig( std::shared_ptr result = std::make_shared(*this); constexpr auto default_disk_name = "default"; + constexpr auto local_disk_name = "local"; DisksMap old_disks_minus_new_disks(result->getDisksMap()); for (const auto & disk_name : keys) @@ -109,6 +110,8 @@ DiskSelectorPtr DiskSelector::updateFromConfig( } old_disks_minus_new_disks.erase(default_disk_name); + old_disks_minus_new_disks.erase(local_disk_name); + if (!old_disks_minus_new_disks.empty()) { diff --git a/tests/integration/test_store_cleanup/test.py b/tests/integration/test_store_cleanup/test.py index 6c5a20a758a..aebfde694b3 100644 --- a/tests/integration/test_store_cleanup/test.py +++ b/tests/integration/test_store_cleanup/test.py @@ -1,3 +1,4 @@ +from time import sleep import pytest from helpers.cluster import ClickHouseCluster @@ -153,7 +154,9 @@ def test_store_cleanup(started_cluster): "directories from store", timeout=90, look_behind_lines=1000000 ) node1.wait_for_log_line( - "Nothing to clean up from store/", timeout=90, look_behind_lines=1000000 + "Nothing to clean up from store/ on disk default", + timeout=90, + look_behind_lines=1000000, ) store = node1.exec_in_container(["ls", f"{path_to_data}/store"]) diff --git a/tests/queries/0_stateless/02980_s3_plain_DROP_TABLE_ReplicatedMergeTree.reference b/tests/queries/0_stateless/02980_s3_plain_DROP_TABLE_ReplicatedMergeTree.reference index 1e191b719a5..21b38a94cee 100644 --- a/tests/queries/0_stateless/02980_s3_plain_DROP_TABLE_ReplicatedMergeTree.reference +++ b/tests/queries/0_stateless/02980_s3_plain_DROP_TABLE_ReplicatedMergeTree.reference @@ -4,27 +4,27 @@ Files before DETACH TABLE all_X_X_X backups/ordinary_default/data/ordinary_default/data_read/all_X_X_X: -primary.cidx -serialization.json -metadata_version.txt -default_compression_codec.txt +checksums.txt +columns.txt +count.txt data.bin data.cmrk3 -count.txt -columns.txt -checksums.txt +default_compression_codec.txt +metadata_version.txt +primary.cidx +serialization.json Files after DETACH TABLE all_X_X_X backups/ordinary_default/data/ordinary_default/data_read/all_X_X_X: -primary.cidx -serialization.json -metadata_version.txt -default_compression_codec.txt +checksums.txt +columns.txt +count.txt data.bin data.cmrk3 -count.txt -columns.txt -checksums.txt +default_compression_codec.txt +metadata_version.txt +primary.cidx +serialization.json diff --git a/tests/queries/0_stateless/02980_s3_plain_DROP_TABLE_ReplicatedMergeTree.sh b/tests/queries/0_stateless/02980_s3_plain_DROP_TABLE_ReplicatedMergeTree.sh index b079e67a000..2500529186e 100755 --- a/tests/queries/0_stateless/02980_s3_plain_DROP_TABLE_ReplicatedMergeTree.sh +++ b/tests/queries/0_stateless/02980_s3_plain_DROP_TABLE_ReplicatedMergeTree.sh @@ -55,14 +55,14 @@ path=${path%/} echo "Files before DETACH TABLE" # sed to match any part, since in case of fault injection part name may not be all_0_0_0 but all_1_1_0 -clickhouse-disks -C "$config" --disk s3_plain_disk list --recursive "${path:?}" | tail -n+2 | sed 's/all_[^_]*_[^_]*_0/all_X_X_X/g' +clickhouse-disks -C "$config" --disk s3_plain_disk -query "list --recursive $path" | tail -n+2 | sed 's/all_[^_]*_[^_]*_0/all_X_X_X/g' $CLICKHOUSE_CLIENT -nm -q " detach table data_read; detach table data_write; " echo "Files after DETACH TABLE" -clickhouse-disks -C "$config" --disk s3_plain_disk list --recursive "$path" | tail -n+2 | sed 's/all_[^_]*_[^_]*_0/all_X_X_X/g' +clickhouse-disks -C "$config" --disk s3_plain_disk --query "list --recursive $path" | tail -n+2 | sed 's/all_[^_]*_[^_]*_0/all_X_X_X/g' # metadata file is left $CLICKHOUSE_CLIENT --force_remove_data_recursively_on_drop=1 -q "drop database if exists $CLICKHOUSE_DATABASE" From 3786ca72d0f1e0d8ae4d84afc31e8fe061698ed0 Mon Sep 17 00:00:00 2001 From: pufit Date: Fri, 21 Jun 2024 02:02:30 -0400 Subject: [PATCH 075/417] docs, review fixes --- docs/en/operations/startup-scripts.md | 31 +++++++++++++++++++ programs/server/Server.cpp | 14 +++++++-- src/Core/ServerSettings.h | 2 +- src/Interpreters/Context.cpp | 6 ++++ src/Interpreters/Context.h | 1 + .../test_startup_scripts/configs/config.xml | 10 ++++++ 6 files changed, 61 insertions(+), 3 deletions(-) create mode 100644 docs/en/operations/startup-scripts.md diff --git a/docs/en/operations/startup-scripts.md b/docs/en/operations/startup-scripts.md new file mode 100644 index 00000000000..c7842c1472b --- /dev/null +++ b/docs/en/operations/startup-scripts.md @@ -0,0 +1,31 @@ +--- +slug: /en/operations/startup-scripts.md +sidebar_position: 70 +sidebar_label: Startup Scripts +--- + +# Startup Scripts + +ClickHouse can run arbitrary SQL queries from the server configuration during startup. This can be useful for migrations or automatic schema creation. + +```xml + + + + CREATE ROLE OR REPLACE test_role + + + CREATE TABLE TestTable (id UInt64) ENGINE=TinyLog + SELECT 1; + + + +``` + +ClickHouse executes all queries from the `startup_scripts` sequentially in the specified order. If any of the queries fail, the execution of the following queries won't be interrupted. + +You can specify a conditional query in the config. In that case, the corresponding query executes only when the condition query returns the value `1` or `true`. + +:::note +If the condition query returns any other value than `1` or `true`, the result will be interpreted as `false`, and the corresponding won't be executed. +::: diff --git a/programs/server/Server.cpp b/programs/server/Server.cpp index 11113ce0c0f..02cad419fff 100644 --- a/programs/server/Server.cpp +++ b/programs/server/Server.cpp @@ -638,7 +638,12 @@ void loadStartupScripts(const Poco::Util::AbstractConfiguration & config, Contex auto result = condition_write_buffer.str(); if (result != "1\n" && result != "true\n") + { + if (result != "0\n" && result != "false\n") + context->addWarningMessage(fmt::format("The condition query returned `{}`, which can't be interpreted as a boolean (`0`, `false`, `1`, `true`). Will skip this query.", result)); + continue; + } LOG_DEBUG(log, "Condition is true, will execute the query next"); } @@ -651,9 +656,9 @@ void loadStartupScripts(const Poco::Util::AbstractConfiguration & config, Contex executeQuery(read_buffer, write_buffer, true, context, callback, QueryFlags{ .internal = true }, std::nullopt, {}); } } - catch (const std::exception & e) + catch (...) { - LOG_ERROR(log, "Failed to parse startup scripts file {}", e.what()); + tryLogCurrentException(log, "Failed to parse startup scripts file"); } } @@ -2014,6 +2019,11 @@ try /// otherwise there is a race condition between the system database initialization /// and creation of new tables in the database. waitLoad(TablesLoaderForegroundPoolId, system_startup_tasks); + + /// Startup scripts can depend on the system log tables. + if (config().has("startup_scripts") && !server_settings.prepare_system_log_tables_on_startup.changed) + global_context->setServerSetting("prepare_system_log_tables_on_startup", true); + /// After attaching system databases we can initialize system log. global_context->initializeSystemLogs(); global_context->setSystemZooKeeperLogAfterInitializationIfNeeded(); diff --git a/src/Core/ServerSettings.h b/src/Core/ServerSettings.h index cf220457c51..8ce3bb5394b 100644 --- a/src/Core/ServerSettings.h +++ b/src/Core/ServerSettings.h @@ -152,7 +152,7 @@ namespace DB M(String, merge_workload, "default", "Name of workload to be used to access resources for all merges (may be overridden by a merge tree setting)", 0) \ M(String, mutation_workload, "default", "Name of workload to be used to access resources for all mutations (may be overridden by a merge tree setting)", 0) \ M(Double, gwp_asan_force_sample_probability, 0, "Probability that an allocation from specific places will be sampled by GWP Asan (i.e. PODArray allocations)", 0) \ - M(Bool, prepare_system_log_tables_on_startup, false, "If true, ClickHouse creates all configured `_log` tables before the startup. It can be helpful if some startup scripts depend on `_log` tables.", 0) \ + M(Bool, prepare_system_log_tables_on_startup, false, "If true, ClickHouse creates all configured `system.*_log` tables before the startup. It can be helpful if some startup scripts depend on these tables.", 0) \ /// If you add a setting which can be updated at runtime, please update 'changeable_settings' map in StorageSystemServerSettings.cpp diff --git a/src/Interpreters/Context.cpp b/src/Interpreters/Context.cpp index 2807807b294..367b5c32e4f 100644 --- a/src/Interpreters/Context.cpp +++ b/src/Interpreters/Context.cpp @@ -2232,6 +2232,12 @@ void Context::setSetting(std::string_view name, const Field & value) contextSanityClampSettingsWithLock(*this, settings, lock); } +void Context::setServerSetting(std::string_view name, const Field & value) +{ + std::lock_guard lock(mutex); + shared->server_settings.set(name, value); +} + void Context::applySettingChange(const SettingChange & change) { try diff --git a/src/Interpreters/Context.h b/src/Interpreters/Context.h index b3ade94ccdc..9f2c600e9fb 100644 --- a/src/Interpreters/Context.h +++ b/src/Interpreters/Context.h @@ -825,6 +825,7 @@ public: /// Set settings by name. void setSetting(std::string_view name, const String & value); void setSetting(std::string_view name, const Field & value); + void setServerSetting(std::string_view name, const Field & value); void applySettingChange(const SettingChange & change); void applySettingsChanges(const SettingsChanges & changes); diff --git a/tests/integration/test_startup_scripts/configs/config.xml b/tests/integration/test_startup_scripts/configs/config.xml index 42c1965c66d..98cce305a25 100644 --- a/tests/integration/test_startup_scripts/configs/config.xml +++ b/tests/integration/test_startup_scripts/configs/config.xml @@ -10,5 +10,15 @@ CREATE TABLE TestTable (id UInt64) ENGINE=TinyLog SELECT 1; + + SELECT * FROM system.query_log LIMIT 1 + + + + system + query_log
+ ENGINE = MergeTree ORDER BY (event_date, event_time) SETTINGS index_granularity = 1024 + 7500 +
From b83941c42759e417611c77c19c102b17db7f067b Mon Sep 17 00:00:00 2001 From: avogar Date: Fri, 21 Jun 2024 11:37:14 +0000 Subject: [PATCH 076/417] Force push to sync From b936d77eb159e1532cb15c6f40befcc15dacdf2a Mon Sep 17 00:00:00 2001 From: avogar Date: Fri, 21 Jun 2024 14:17:13 +0000 Subject: [PATCH 077/417] Try to bump sync From 9f3024cfd610d34ed7262dd6d019300ca314dd52 Mon Sep 17 00:00:00 2001 From: pufit Date: Fri, 21 Jun 2024 12:10:49 -0400 Subject: [PATCH 078/417] fix tests --- .../test_startup_scripts/configs/config.d/query_log.xml | 8 ++++++++ .../configs/{config.xml => config.d/startup_scripts.xml} | 7 ------- tests/integration/test_startup_scripts/test.py | 5 ++++- 3 files changed, 12 insertions(+), 8 deletions(-) create mode 100644 tests/integration/test_startup_scripts/configs/config.d/query_log.xml rename tests/integration/test_startup_scripts/configs/{config.xml => config.d/startup_scripts.xml} (67%) diff --git a/tests/integration/test_startup_scripts/configs/config.d/query_log.xml b/tests/integration/test_startup_scripts/configs/config.d/query_log.xml new file mode 100644 index 00000000000..24d66fc674e --- /dev/null +++ b/tests/integration/test_startup_scripts/configs/config.d/query_log.xml @@ -0,0 +1,8 @@ + + + system + query_log
+ toYYYYMM(event_date) + 1000 +
+
diff --git a/tests/integration/test_startup_scripts/configs/config.xml b/tests/integration/test_startup_scripts/configs/config.d/startup_scripts.xml similarity index 67% rename from tests/integration/test_startup_scripts/configs/config.xml rename to tests/integration/test_startup_scripts/configs/config.d/startup_scripts.xml index 98cce305a25..e8a711a926a 100644 --- a/tests/integration/test_startup_scripts/configs/config.xml +++ b/tests/integration/test_startup_scripts/configs/config.d/startup_scripts.xml @@ -14,11 +14,4 @@ SELECT * FROM system.query_log LIMIT 1 - - - system - query_log
- ENGINE = MergeTree ORDER BY (event_date, event_time) SETTINGS index_granularity = 1024 - 7500 -
diff --git a/tests/integration/test_startup_scripts/test.py b/tests/integration/test_startup_scripts/test.py index ee61994f830..43a871a6fc5 100644 --- a/tests/integration/test_startup_scripts/test.py +++ b/tests/integration/test_startup_scripts/test.py @@ -6,7 +6,10 @@ def test_startup_scripts(): node = cluster.add_instance( "node", - main_configs=["configs/config.xml"], + main_configs=[ + "configs/config.d/query_log.xml", + "configs/config.d/startup_scripts.xml", + ], with_zookeeper=False, ) From 504cc3b0901091fd64f83f652478f078f1c79c50 Mon Sep 17 00:00:00 2001 From: divanik Date: Fri, 21 Jun 2024 16:27:21 +0000 Subject: [PATCH 079/417] Fix some tests --- programs/disks/DisksApp.cpp | 2 +- src/Disks/DiskLocal.cpp | 1 - src/Disks/DiskSelector.cpp | 17 ++++++++++++----- src/Disks/DiskSelector.h | 6 +++++- ...0_s3_plain_DROP_TABLE_ReplicatedMergeTree.sh | 4 +++- 5 files changed, 21 insertions(+), 9 deletions(-) diff --git a/programs/disks/DisksApp.cpp b/programs/disks/DisksApp.cpp index 3b09feecc3b..9ef051a2ece 100644 --- a/programs/disks/DisksApp.cpp +++ b/programs/disks/DisksApp.cpp @@ -480,7 +480,7 @@ int DisksApp::main(const std::vector & /*args*/) auto validator = [](const Poco::Util::AbstractConfiguration &, const std::string &, const std::string &) { return true; }; constexpr auto config_prefix = "storage_configuration.disks"; - auto disk_selector = std::make_shared(std::unordered_set{"cache", "encrypted"}); + auto disk_selector = std::make_shared(std::unordered_set{"cache", "encrypted"}, /*create_local=*/true); disk_selector->initialize(config(), config_prefix, global_context, validator); std::vector>> disks_with_path; diff --git a/src/Disks/DiskLocal.cpp b/src/Disks/DiskLocal.cpp index 6cb2599b82a..d1f0a928b1d 100644 --- a/src/Disks/DiskLocal.cpp +++ b/src/Disks/DiskLocal.cpp @@ -276,7 +276,6 @@ bool DiskLocal::isFile(const String & path) const bool DiskLocal::isDirectory(const String & path) const { - // std::cerr << fs::path(disk_path) / path << std::endl; return fs::is_directory(fs::path(disk_path) / path); } diff --git a/src/Disks/DiskSelector.cpp b/src/Disks/DiskSelector.cpp index c29a24132c4..6e57bdad77e 100644 --- a/src/Disks/DiskSelector.cpp +++ b/src/Disks/DiskSelector.cpp @@ -26,7 +26,7 @@ void DiskSelector::assertInitialized() const } -void DiskSelector::initialize( +void DiskSelector::( const Poco::Util::AbstractConfiguration & config, const String & config_prefix, ContextPtr context, DiskValidator disk_validator) { Poco::Util::AbstractConfiguration::Keys keys; @@ -66,7 +66,7 @@ void DiskSelector::initialize( default_disk_name, std::make_shared(default_disk_name, context->getPath(), 0, context, config, config_prefix)); } - if (!has_local_disk) + if (!has_local_disk && create_local) disks.emplace(local_disk_name, std::make_shared(local_disk_name, "/", 0, context, config, config_prefix)); is_initialized = true; @@ -97,7 +97,12 @@ DiskSelectorPtr DiskSelector::updateFromConfig( auto disk_config_prefix = config_prefix + "." + disk_name; if (!result->getDisksMap().contains(disk_name)) { - result->addToDiskMap(disk_name, factory.create(disk_name, config, disk_config_prefix, context, result->getDisksMap())); + auto created_disk = factory.create( + disk_name, config, disk_config_prefix, context, result->getDisksMap(), /*attach*/ false, /*custom_disk*/ false, skip_types); + if (created_disk) + { + result->addToDiskMap(disk_name, created_disk); + } } else { @@ -110,8 +115,10 @@ DiskSelectorPtr DiskSelector::updateFromConfig( } old_disks_minus_new_disks.erase(default_disk_name); - old_disks_minus_new_disks.erase(local_disk_name); - + if (create_local) + { + old_disks_minus_new_disks.erase(local_disk_name); + } if (!old_disks_minus_new_disks.empty()) { diff --git a/src/Disks/DiskSelector.h b/src/Disks/DiskSelector.h index fb3cb4a0177..8ceb4a58c15 100644 --- a/src/Disks/DiskSelector.h +++ b/src/Disks/DiskSelector.h @@ -20,7 +20,10 @@ class DiskSelector public: static constexpr auto TMP_INTERNAL_DISK_PREFIX = "__tmp_internal_"; - explicit DiskSelector(std::unordered_set skip_types_ = {}) : skip_types(skip_types_) { } + explicit DiskSelector(std::unordered_set skip_types_ = {}, bool create_local_ = false) + : skip_types(skip_types_), create_local(create_local_) + { + } DiskSelector(const DiskSelector & from) = default; using DiskValidator = std::function; @@ -50,6 +53,7 @@ private: void assertInitialized() const; const std::unordered_set skip_types; + const bool create_local; }; } diff --git a/tests/queries/0_stateless/02980_s3_plain_DROP_TABLE_ReplicatedMergeTree.sh b/tests/queries/0_stateless/02980_s3_plain_DROP_TABLE_ReplicatedMergeTree.sh index 2500529186e..7725688d225 100755 --- a/tests/queries/0_stateless/02980_s3_plain_DROP_TABLE_ReplicatedMergeTree.sh +++ b/tests/queries/0_stateless/02980_s3_plain_DROP_TABLE_ReplicatedMergeTree.sh @@ -55,13 +55,15 @@ path=${path%/} echo "Files before DETACH TABLE" # sed to match any part, since in case of fault injection part name may not be all_0_0_0 but all_1_1_0 -clickhouse-disks -C "$config" --disk s3_plain_disk -query "list --recursive $path" | tail -n+2 | sed 's/all_[^_]*_[^_]*_0/all_X_X_X/g' +echo "Path $path" +clickhouse-disks -C "$config" --disk s3_plain_disk --query "list --recursive $path" | tail -n+2 | sed 's/all_[^_]*_[^_]*_0/all_X_X_X/g' $CLICKHOUSE_CLIENT -nm -q " detach table data_read; detach table data_write; " echo "Files after DETACH TABLE" +echo "Path $path" clickhouse-disks -C "$config" --disk s3_plain_disk --query "list --recursive $path" | tail -n+2 | sed 's/all_[^_]*_[^_]*_0/all_X_X_X/g' # metadata file is left From b1f45da3eb72f0aae9976c64919bdbc7a4353bc6 Mon Sep 17 00:00:00 2001 From: avogar Date: Fri, 21 Jun 2024 18:20:35 +0000 Subject: [PATCH 080/417] Implement binary encoding for ClickHouse data types --- docs/en/interfaces/formats.md | 4 + .../operations/settings/settings-formats.md | 24 + .../data-types/data-types-binary-encoding.md | 104 +++ .../domains/data-types-binary-encoding.md | 0 src/Columns/ColumnDynamic.cpp | 48 +- src/Common/FieldBinaryEncoding.cpp | 388 ++++++++++ src/Common/FieldBinaryEncoding.h | 43 ++ src/Common/IntervalKind.h | 23 +- src/Core/Field.cpp | 16 +- src/Core/Settings.h | 4 + .../tests/gtest_fields_binary_enciding.cpp | 65 ++ src/DataTypes/DataTypeAggregateFunction.h | 3 +- .../DataTypeCustomSimpleAggregateFunction.cpp | 13 + .../DataTypeCustomSimpleAggregateFunction.h | 5 + src/DataTypes/DataTypeNested.h | 2 + src/DataTypes/DataTypesBinaryEncoding.cpp | 677 ++++++++++++++++++ src/DataTypes/DataTypesBinaryEncoding.h | 117 +++ src/DataTypes/Serializations/ISerialization.h | 6 + .../Serializations/SerializationArray.cpp | 8 +- .../Serializations/SerializationDynamic.cpp | 67 +- .../Serializations/SerializationMap.cpp | 4 +- .../Serializations/SerializationString.cpp | 16 +- .../SerializationVariantElement.cpp | 2 +- .../data_type_deserialization_fuzzer.cpp | 4 +- .../gtest_data_types_binary_encoding.cpp | 123 ++++ src/Formats/FormatFactory.cpp | 8 +- src/Formats/FormatSettings.h | 12 +- src/Formats/NativeReader.cpp | 36 +- src/Formats/NativeReader.h | 12 +- src/Formats/NativeWriter.cpp | 45 +- src/Formats/NativeWriter.h | 4 +- src/Formats/SchemaInferenceUtils.cpp | 92 ++- src/Interpreters/Context.cpp | 3 - .../Formats/Impl/BinaryRowInputFormat.cpp | 24 +- .../Formats/Impl/BinaryRowOutputFormat.cpp | 11 +- src/Processors/Formats/Impl/NativeFormat.cpp | 21 +- src/Server/TCPHandler.cpp | 3 + src/Storages/StorageMemory.cpp | 2 +- src/Storages/StorageStripeLog.cpp | 2 +- ...172_dynamic_binary_serialization.reference | 48 ++ .../03172_dynamic_binary_serialization.sh | 61 ++ ...native_with_binary_encoded_types.reference | 102 +++ ...ry_and_native_with_binary_encoded_types.sh | 63 ++ 43 files changed, 2128 insertions(+), 187 deletions(-) create mode 100644 docs/en/sql-reference/data-types/data-types-binary-encoding.md create mode 100644 docs/en/sql-reference/data-types/domains/data-types-binary-encoding.md create mode 100644 src/Common/FieldBinaryEncoding.cpp create mode 100644 src/Common/FieldBinaryEncoding.h create mode 100644 src/Core/tests/gtest_fields_binary_enciding.cpp create mode 100644 src/DataTypes/DataTypesBinaryEncoding.cpp create mode 100644 src/DataTypes/DataTypesBinaryEncoding.h create mode 100644 src/DataTypes/tests/gtest_data_types_binary_encoding.cpp create mode 100644 tests/queries/0_stateless/03172_dynamic_binary_serialization.reference create mode 100755 tests/queries/0_stateless/03172_dynamic_binary_serialization.sh create mode 100644 tests/queries/0_stateless/03173_row_binary_and_native_with_binary_encoded_types.reference create mode 100755 tests/queries/0_stateless/03173_row_binary_and_native_with_binary_encoded_types.sh diff --git a/docs/en/interfaces/formats.md b/docs/en/interfaces/formats.md index fdbfb742a10..b71e58b8b0e 100644 --- a/docs/en/interfaces/formats.md +++ b/docs/en/interfaces/formats.md @@ -1533,6 +1533,10 @@ the columns from input data will be mapped to the columns from the table by thei Otherwise, the first row will be skipped. If setting [input_format_with_types_use_header](/docs/en/operations/settings/settings-formats.md/#input_format_with_types_use_header) is set to 1, the types from input data will be compared with the types of the corresponding columns from the table. Otherwise, the second row will be skipped. +If setting [output_format_binary_encode_types_in_binary_format](/docs/en/operations/settings/settings-formats.md/#output_format_binary_encode_types_in_binary_format) is set to 1, +the types in header will be written using [binary encoding](../../sql-reference/data-types/data-types-binary-encoding.md) instead of strings with type names in RowBinaryWithNamesAndTypes output format. +If setting [input_format_binary_encode_types_in_binary_format](/docs/en/operations/settings/settings-formats.md/#input_format_binary_encode_types_in_binary_format) is set to 1, +the types in header will be read using [binary encoding](../../sql-reference/data-types/data-types-binary-encoding.md) instead of strings with type names in RowBinaryWithNamesAndTypes input format. ::: ## RowBinaryWithDefaults {#rowbinarywithdefaults} diff --git a/docs/en/operations/settings/settings-formats.md b/docs/en/operations/settings/settings-formats.md index 6aae1ea62e5..6709d5a8ab9 100644 --- a/docs/en/operations/settings/settings-formats.md +++ b/docs/en/operations/settings/settings-formats.md @@ -1907,6 +1907,18 @@ The maximum allowed size for String in RowBinary format. It prevents allocating Default value: `1GiB`. +### output_format_binary_encode_types_in_binary_format {#output_format_binary_encode_types_in_binary_format} + +Write data types in [binary format](../../sql-reference/data-types/data-types-binary-encoding.md) instead of type names in RowBinaryWithNamesAndTypes output format. + +Disabled by default. + +### input_format_binary_decode_types_in_binary_format {#input_format_binary_decode_types_in_binary_format} + +Read data types in [binary format](../../sql-reference/data-types/data-types-binary-encoding.md) instead of type names in RowBinaryWithNamesAndTypes input format. + +Disabled by default. + ## Native format settings {#native-format-settings} ### input_format_native_allow_types_conversion {#input_format_native_allow_types_conversion} @@ -1914,3 +1926,15 @@ Default value: `1GiB`. Allow types conversion in Native input format between columns from input data and requested columns. Enabled by default. + +### output_format_native_encode_types_in_binary_format {#output_format_native_encode_types_in_binary_format} + +Write data types in [binary format](../../sql-reference/data-types/data-types-binary-encoding.md) instead of type names in Native output format. + +Disabled by default. + +### input_format_native_decode_types_in_binary_format {#input_format_native_decode_types_in_binary_format} + +Read data types in [binary format](../../sql-reference/data-types/data-types-binary-encoding.md) instead of type names in Native input format. + +Disabled by default. \ No newline at end of file diff --git a/docs/en/sql-reference/data-types/data-types-binary-encoding.md b/docs/en/sql-reference/data-types/data-types-binary-encoding.md new file mode 100644 index 00000000000..ebcb480ea0a --- /dev/null +++ b/docs/en/sql-reference/data-types/data-types-binary-encoding.md @@ -0,0 +1,104 @@ +--- +slug: /en/sql-reference/data-types/dynamic +sidebar_position: 56 +sidebar_label: Data types binary encoding specification. +--- + + +# Data types binary encoding specification + +| ClickHouse data type | Binary encoding | +|--------------------------------------------------------------------------------------|------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------| +| `Nothing` | `0x00` | +| `UInt8` | `0x01` | +| `UInt16` | `0x02` | +| `UInt32` | `0x03` | +| `UInt64` | `0x04` | +| `UInt128` | `0x05` | +| `UInt256` | `0x06` | +| `Int8` | `0x07` | +| `Int16` | `0x08` | +| `Int32` | `0x09` | +| `Int64` | `0x0A` | +| `Int128` | `0x0B` | +| `Int256` | `0x0C` | +| `Float32` | `0x0D` | +| `Float64` | `0x0E` | +| `Date` | `0x0F` | +| `Date32` | `0x10` | +| `DateTime` | `0x11` | +| `DateTime64(P)` | `0x12` | +| `String` | `0x13` | +| `FixedString(N)` | `0x14` | +| `Enum8` | `0x15...` | +| `Enum16` | `0x16...>` | +| `Decimal32(P, S)` | `0x17` | +| `Decimal64(P, S)` | `0x18` | +| `Decimal128(P, S)` | `0x19` | +| `Decimal256(P, S)` | `0x1A` | +| `UUID` | `0x1B` | +| `Array(T)` | `0x1C` | +| `Tuple(T1, ..., TN)` | `0x1D...` | +| `Tuple(name1 T1, ..., nameN TN)` | `0x1E...` | +| `Set` | `0x1F` | +| `Interval` | `0x20` (see [interval kind binary encoding](#interval-kind-binary-encoding)) | +| `Nullable(T)` | `0x21` | +| `Function` | `0x22...` | +| `AggregateFunction(function_name(param_1, ..., param_N), arg_T1, ..., arg_TN)` | `0x23......` (see [aggregate function parameter binary encoding](#aggregate-function-parameter-binary-encoding)) | +| `LowCardinality(T)` | `0x24` | +| `Map(K, V)` | `0x25` | +| `Object('schema_format')` | `0x26` | +| `IPv4` | `0x27` | +| `IPv6` | `0x28` | +| `Variant(T1, ..., TN)` | `0x29...` | +| `Dynamic` | `0x2A` | +| `Custom type` (`Ring`, `Polygon`, etc) | `0x2B` | +| `Bool` | `0x2C` | +| `SimpleAggregateFunction(function_name(param_1, ..., param_N), arg_T1, ..., arg_TN)` | `0x2D......` (see [aggregate function parameter binary encoding](#aggregate-function-parameter-binary-encoding)) | +| `Nested(name1 T1, ..., nameN TN)` | `0x2E...` | + + +### Interval kind binary encoding + +| Interval kind | Binary encoding | +|---------------|-----------------| +| `Nanosecond` | `0x00` | +| `Microsecond` | `0x01` | +| `Millisecond` | `0x02` | +| `Second` | `0x03` | +| `Minute` | `0x04` | +| `Hour` | `0x05` | +| `Day` | `0x06` | +| `Week` | `0x07` | +| `Month` | `0x08` | +| `Quarter` | `0x09` | +| `Year` | `0x1A` | + +### Aggregate function parameter binary encoding + +| Parameter type | Binary encoding | +|--------------------------|--------------------------------------------------------------------------------------------------------------------------------| +| `Null` | `0x00` | +| `UInt64` | `0x01` | +| `Int64` | `0x02` | +| `UInt128` | `0x03` | +| `Int128` | `0x04` | +| `UInt128` | `0x05` | +| `Int128` | `0x06` | +| `Float64` | `0x07` | +| `Decimal32` | `0x08` | +| `Decimal64` | `0x09` | +| `Decimal128` | `0x0A` | +| `Decimal256` | `0x0B` | +| `String` | `0x0C` | +| `Array` | `0x0D...` | +| `Tuple` | `0x0E...` | +| `Map` | `0x0F...` | +| `IPv4` | `0x10` | +| `IPv6` | `0x11` | +| `UUID` | `0x12` | +| `Bool` | `0x13` | +| `Object` | `0x14...` | +| `AggregateFunctionState` | `0x15` | +| `Negative infinity` | `0xFE` | +| `Positive infinity` | `0xFF` | diff --git a/docs/en/sql-reference/data-types/domains/data-types-binary-encoding.md b/docs/en/sql-reference/data-types/domains/data-types-binary-encoding.md new file mode 100644 index 00000000000..e69de29bb2d diff --git a/src/Columns/ColumnDynamic.cpp b/src/Columns/ColumnDynamic.cpp index 3c147b6f123..d56999ce5a2 100644 --- a/src/Columns/ColumnDynamic.cpp +++ b/src/Columns/ColumnDynamic.cpp @@ -4,7 +4,9 @@ #include #include #include +#include #include +#include #include #include #include @@ -481,7 +483,7 @@ StringRef ColumnDynamic::serializeValueIntoArena(size_t n, DB::Arena & arena, co /// We cannot use Variant serialization here as it serializes discriminator + value, /// but Dynamic doesn't have fixed mapping discriminator <-> variant type /// as different Dynamic column can have different Variants. - /// Instead, we serialize null bit + variant type name (size + bytes) + value. + /// Instead, we serialize null bit + variant type in binary format (size + bytes) + value. const auto & variant_col = assert_cast(*variant_column); auto discr = variant_col.globalDiscriminatorAt(n); StringRef res; @@ -495,14 +497,15 @@ StringRef ColumnDynamic::serializeValueIntoArena(size_t n, DB::Arena & arena, co return res; } - const auto & variant_name = variant_info.variant_names[discr]; - size_t variant_name_size = variant_name.size(); - char * pos = arena.allocContinue(sizeof(UInt8) + sizeof(size_t) + variant_name.size(), begin); + const auto & variant_type = assert_cast(*variant_info.variant_type).getVariant(discr); + String variant_type_binary_data = encodeDataType(variant_type); + size_t variant_type_binary_data_size = variant_type_binary_data.size(); + char * pos = arena.allocContinue(sizeof(UInt8) + sizeof(size_t) + variant_type_binary_data.size(), begin); memcpy(pos, &null_bit, sizeof(UInt8)); - memcpy(pos + sizeof(UInt8), &variant_name_size, sizeof(size_t)); - memcpy(pos + sizeof(UInt8) + sizeof(size_t), variant_name.data(), variant_name.size()); + memcpy(pos + sizeof(UInt8), &variant_type_binary_data_size, sizeof(size_t)); + memcpy(pos + sizeof(UInt8) + sizeof(size_t), variant_type_binary_data.data(), variant_type_binary_data.size()); res.data = pos; - res.size = sizeof(UInt8) + sizeof(size_t) + variant_name.size(); + res.size = sizeof(UInt8) + sizeof(size_t) + variant_type_binary_data.size(); auto value_ref = variant_col.getVariantByGlobalDiscriminator(discr).serializeValueIntoArena(variant_col.offsetAt(n), arena, begin); res.data = value_ref.data - res.size; @@ -521,13 +524,15 @@ const char * ColumnDynamic::deserializeAndInsertFromArena(const char * pos) return pos; } - /// Read variant type name. - const size_t variant_name_size = unalignedLoad(pos); - pos += sizeof(variant_name_size); - String variant_name; - variant_name.resize(variant_name_size); - memcpy(variant_name.data(), pos, variant_name_size); - pos += variant_name_size; + /// Read variant type in binary format. + const size_t variant_type_binary_data_size = unalignedLoad(pos); + pos += sizeof(variant_type_binary_data_size); + String variant_type_binary_data; + variant_type_binary_data.resize(variant_type_binary_data_size); + memcpy(variant_type_binary_data.data(), pos, variant_type_binary_data_size); + pos += variant_type_binary_data_size; + auto variant_type = decodeDataType(variant_type_binary_data); + auto variant_name = variant_type->getName(); /// If we already have such variant, just deserialize it into corresponding variant column. auto it = variant_info.variant_name_to_discriminator.find(variant_name); if (it != variant_info.variant_name_to_discriminator.end()) @@ -537,7 +542,6 @@ const char * ColumnDynamic::deserializeAndInsertFromArena(const char * pos) } /// If we don't have such variant, add it. - auto variant_type = DataTypeFactory::instance().get(variant_name); if (likely(addNewVariant(variant_type))) { auto discr = variant_info.variant_name_to_discriminator[variant_name]; @@ -563,13 +567,13 @@ const char * ColumnDynamic::skipSerializedInArena(const char * pos) const if (null_bit) return pos; - const size_t variant_name_size = unalignedLoad(pos); - pos += sizeof(variant_name_size); - String variant_name; - variant_name.resize(variant_name_size); - memcpy(variant_name.data(), pos, variant_name_size); - pos += variant_name_size; - auto tmp_variant_column = DataTypeFactory::instance().get(variant_name)->createColumn(); + const size_t variant_type_binary_data_size = unalignedLoad(pos); + pos += sizeof(variant_type_binary_data_size); + String variant_type_binary_data; + variant_type_binary_data.resize(variant_type_binary_data_size); + memcpy(variant_type_binary_data.data(), pos, variant_type_binary_data_size); + pos += variant_type_binary_data_size; + auto tmp_variant_column = decodeDataType(variant_type_binary_data)->createColumn(); return tmp_variant_column->skipSerializedInArena(pos); } diff --git a/src/Common/FieldBinaryEncoding.cpp b/src/Common/FieldBinaryEncoding.cpp new file mode 100644 index 00000000000..7e0d815368c --- /dev/null +++ b/src/Common/FieldBinaryEncoding.cpp @@ -0,0 +1,388 @@ +#include +#include +#include + + +namespace DB +{ + +namespace ErrorCodes +{ + extern const int UNSUPPORTED_METHOD; +} + +namespace +{ + +enum class FieldBinaryTypeIndex: uint8_t +{ + Null = 0x00, + UInt64 = 0x01, + Int64 = 0x02, + UInt128 = 0x03, + Int128 = 0x04, + UInt256 = 0x05, + Int256 = 0x06, + Float64 = 0x07, + Decimal32 = 0x08, + Decimal64 = 0x09, + Decimal128 = 0x0A, + Decimal256 = 0x0B, + String = 0x0C, + Array = 0x0D, + Tuple = 0x0E, + Map = 0x0F, + IPv4 = 0x10, + IPv6 = 0x11, + UUID = 0x12, + Bool = 0x13, + Object = 0x14, + AggregateFunctionState = 0x15, + + NegativeInfinity = 0xFE, + PositiveInfinity = 0xFF, +}; + +class FieldVisitorEncodeBinary +{ +public: + void operator() (const Null & x, WriteBuffer & buf) const; + void operator() (const UInt64 & x, WriteBuffer & buf) const; + void operator() (const UInt128 & x, WriteBuffer & buf) const; + void operator() (const UInt256 & x, WriteBuffer & buf) const; + void operator() (const Int64 & x, WriteBuffer & buf) const; + void operator() (const Int128 & x, WriteBuffer & buf) const; + void operator() (const Int256 & x, WriteBuffer & buf) const; + void operator() (const UUID & x, WriteBuffer & buf) const; + void operator() (const IPv4 & x, WriteBuffer & buf) const; + void operator() (const IPv6 & x, WriteBuffer & buf) const; + void operator() (const Float64 & x, WriteBuffer & buf) const; + void operator() (const String & x, WriteBuffer & buf) const; + void operator() (const Array & x, WriteBuffer & buf) const; + void operator() (const Tuple & x, WriteBuffer & buf) const; + void operator() (const Map & x, WriteBuffer & buf) const; + void operator() (const Object & x, WriteBuffer & buf) const; + void operator() (const DecimalField & x, WriteBuffer & buf) const; + void operator() (const DecimalField & x, WriteBuffer & buf) const; + void operator() (const DecimalField & x, WriteBuffer & buf) const; + void operator() (const DecimalField & x, WriteBuffer & buf) const; + void operator() (const AggregateFunctionStateData & x, WriteBuffer & buf) const; + [[noreturn]] void operator() (const CustomType & x, WriteBuffer & buf) const; + void operator() (const bool & x, WriteBuffer & buf) const; +}; + +void FieldVisitorEncodeBinary::operator() (const Null & x, WriteBuffer & buf) const +{ + if (x.isNull()) + writeBinary(UInt8(FieldBinaryTypeIndex::Null), buf); + else if (x.isPositiveInfinity()) + writeBinary(UInt8(FieldBinaryTypeIndex::PositiveInfinity), buf); + else if (x.isNegativeInfinity()) + writeBinary(UInt8(FieldBinaryTypeIndex::NegativeInfinity), buf); +} + +void FieldVisitorEncodeBinary::operator() (const UInt64 & x, WriteBuffer & buf) const +{ + writeBinary(UInt8(FieldBinaryTypeIndex::UInt64), buf); + writeVarUInt(x, buf); +} + +void FieldVisitorEncodeBinary::operator() (const Int64 & x, WriteBuffer & buf) const +{ + writeBinary(UInt8(FieldBinaryTypeIndex::Int64), buf); + writeVarInt(x, buf); +} + +void FieldVisitorEncodeBinary::operator() (const Float64 & x, WriteBuffer & buf) const +{ + writeBinary(UInt8(FieldBinaryTypeIndex::Float64), buf); + writeBinaryLittleEndian(x, buf); +} + +void FieldVisitorEncodeBinary::operator() (const String & x, WriteBuffer & buf) const +{ + writeBinary(UInt8(FieldBinaryTypeIndex::String), buf); + writeStringBinary(x, buf); +} + +void FieldVisitorEncodeBinary::operator() (const UInt128 & x, WriteBuffer & buf) const +{ + writeBinary(UInt8(FieldBinaryTypeIndex::UInt128), buf); + writeBinaryLittleEndian(x, buf); +} + +void FieldVisitorEncodeBinary::operator() (const Int128 & x, WriteBuffer & buf) const +{ + writeBinary(UInt8(FieldBinaryTypeIndex::Int128), buf); + writeBinaryLittleEndian(x, buf); +} + +void FieldVisitorEncodeBinary::operator() (const UInt256 & x, WriteBuffer & buf) const +{ + writeBinary(UInt8(FieldBinaryTypeIndex::UInt256), buf); + writeBinaryLittleEndian(x, buf); +} + +void FieldVisitorEncodeBinary::operator() (const Int256 & x, WriteBuffer & buf) const +{ + writeBinary(UInt8(FieldBinaryTypeIndex::Int256), buf); + writeBinaryLittleEndian(x, buf); +} + +void FieldVisitorEncodeBinary::operator() (const UUID & x, WriteBuffer & buf) const +{ + writeBinary(UInt8(FieldBinaryTypeIndex::UUID), buf); + writeBinaryLittleEndian(x, buf); +} + +void FieldVisitorEncodeBinary::operator() (const IPv4 & x, WriteBuffer & buf) const +{ + writeBinary(UInt8(FieldBinaryTypeIndex::IPv4), buf); + writeBinaryLittleEndian(x, buf); +} + +void FieldVisitorEncodeBinary::operator() (const IPv6 & x, WriteBuffer & buf) const +{ + writeBinary(UInt8(FieldBinaryTypeIndex::IPv6), buf); + writeBinaryLittleEndian(x, buf); +} + +void FieldVisitorEncodeBinary::operator() (const DecimalField & x, WriteBuffer & buf) const +{ + writeBinary(UInt8(FieldBinaryTypeIndex::Decimal32), buf); + writeVarUInt(x.getScale(), buf); + writeBinaryLittleEndian(x.getValue(), buf); +} + +void FieldVisitorEncodeBinary::operator() (const DecimalField & x, WriteBuffer & buf) const +{ + writeBinary(UInt8(FieldBinaryTypeIndex::Decimal64), buf); + writeVarUInt(x.getScale(), buf); + writeBinaryLittleEndian(x.getValue(), buf); +} + +void FieldVisitorEncodeBinary::operator() (const DecimalField & x, WriteBuffer & buf) const +{ + writeBinary(UInt8(FieldBinaryTypeIndex::Decimal128), buf); + writeVarUInt(x.getScale(), buf); + writeBinaryLittleEndian(x.getValue(), buf); +} + +void FieldVisitorEncodeBinary::operator() (const DecimalField & x, WriteBuffer & buf) const +{ + writeBinary(UInt8(FieldBinaryTypeIndex::Decimal256), buf); + writeVarUInt(x.getScale(), buf); + writeBinaryLittleEndian(x.getValue(), buf); +} + +void FieldVisitorEncodeBinary::operator() (const AggregateFunctionStateData & x, WriteBuffer & buf) const +{ + writeBinary(UInt8(FieldBinaryTypeIndex::AggregateFunctionState), buf); + writeStringBinary(x.name, buf); + writeStringBinary(x.data, buf); +} + +void FieldVisitorEncodeBinary::operator() (const Array & x, WriteBuffer & buf) const +{ + writeBinary(UInt8(FieldBinaryTypeIndex::Array), buf); + size_t size = x.size(); + writeVarUInt(size, buf); + for (size_t i = 0; i < size; ++i) + Field::dispatch([&buf] (const auto & value) { FieldVisitorEncodeBinary()(value, buf); }, x[i]); +} + +void FieldVisitorEncodeBinary::operator() (const Tuple & x, WriteBuffer & buf) const +{ + writeBinary(UInt8(FieldBinaryTypeIndex::Tuple), buf); + size_t size = x.size(); + writeVarUInt(size, buf); + for (size_t i = 0; i < size; ++i) + Field::dispatch([&buf] (const auto & value) { FieldVisitorEncodeBinary()(value, buf); }, x[i]); +} + +void FieldVisitorEncodeBinary::operator() (const Map & x, WriteBuffer & buf) const +{ + writeBinary(UInt8(FieldBinaryTypeIndex::Map), buf); + size_t size = x.size(); + writeVarUInt(size, buf); + for (size_t i = 0; i < size; ++i) + { + const Tuple & key_and_value = x[i].get(); + Field::dispatch([&buf] (const auto & value) { FieldVisitorEncodeBinary()(value, buf); }, key_and_value[0]); + Field::dispatch([&buf] (const auto & value) { FieldVisitorEncodeBinary()(value, buf); }, key_and_value[1]); + } +} + +void FieldVisitorEncodeBinary::operator() (const Object & x, WriteBuffer & buf) const +{ + writeBinary(UInt8(FieldBinaryTypeIndex::Object), buf); + + size_t size = x.size(); + writeVarUInt(size, buf); + for (const auto & [key, value] : x) + { + writeStringBinary(key, buf); + Field::dispatch([&buf] (const auto & val) { FieldVisitorEncodeBinary()(val, buf); }, value); + } +} + +void FieldVisitorEncodeBinary::operator()(const bool & x, WriteBuffer & buf) const +{ + writeBinary(UInt8(FieldBinaryTypeIndex::Bool), buf); + writeBinary(static_cast(x), buf); +} + +[[noreturn]] void FieldVisitorEncodeBinary::operator()(const CustomType &, WriteBuffer &) const +{ + /// TODO: Support binary encoding/decoding for custom types somehow. + throw Exception(ErrorCodes::UNSUPPORTED_METHOD, "Binary encoding of Field with custom type is not supported"); +} + +template +Field decodeBigInteger(ReadBuffer & buf) +{ + T value; + readBinaryLittleEndian(value, buf); + return value; +} + +template +DecimalField decodeDecimal(ReadBuffer & buf) +{ + UInt32 scale; + readVarUInt(scale, buf); + T value; + readBinaryLittleEndian(value, buf); + return DecimalField(value, scale); +} + +template +T decodeValueLittleEndian(ReadBuffer & buf) +{ + T value; + readBinaryLittleEndian(value, buf); + return value; +} + +template +T decodeArrayLikeField(ReadBuffer & buf) +{ + size_t size; + readVarUInt(size, buf); + T value; + for (size_t i = 0; i != size; ++i) + value.push_back(decodeField(buf)); + return value; +} + +} +void encodeField(const Field & x, WriteBuffer & buf) +{ + Field::dispatch([&buf] (const auto & val) { FieldVisitorEncodeBinary()(val, buf); }, x); +} + +Field decodeField(ReadBuffer & buf) +{ + UInt8 type; + readBinary(type, buf); + switch (FieldBinaryTypeIndex(type)) + { + case FieldBinaryTypeIndex::Null: + return Null(); + case FieldBinaryTypeIndex::PositiveInfinity: + return POSITIVE_INFINITY; + case FieldBinaryTypeIndex::NegativeInfinity: + return NEGATIVE_INFINITY; + case FieldBinaryTypeIndex::Int64: + { + Int64 value; + readVarInt(value, buf); + return value; + } + case FieldBinaryTypeIndex::UInt64: + { + UInt64 value; + readVarUInt(value, buf); + return value; + } + case FieldBinaryTypeIndex::Int128: + return decodeBigInteger(buf); + case FieldBinaryTypeIndex::UInt128: + return decodeBigInteger(buf); + case FieldBinaryTypeIndex::Int256: + return decodeBigInteger(buf); + case FieldBinaryTypeIndex::UInt256: + return decodeBigInteger(buf); + case FieldBinaryTypeIndex::Float64: + return decodeValueLittleEndian(buf); + case FieldBinaryTypeIndex::Decimal32: + return decodeDecimal(buf); + case FieldBinaryTypeIndex::Decimal64: + return decodeDecimal(buf); + case FieldBinaryTypeIndex::Decimal128: + return decodeDecimal(buf); + case FieldBinaryTypeIndex::Decimal256: + return decodeDecimal(buf); + case FieldBinaryTypeIndex::String: + { + String value; + readStringBinary(value, buf); + return value; + } + case FieldBinaryTypeIndex::UUID: + return decodeValueLittleEndian(buf); + case FieldBinaryTypeIndex::IPv4: + return decodeValueLittleEndian(buf); + case FieldBinaryTypeIndex::IPv6: + return decodeValueLittleEndian(buf); + case FieldBinaryTypeIndex::Bool: + { + bool value; + readBinary(value, buf); + return value; + } + case FieldBinaryTypeIndex::Array: + return decodeArrayLikeField(buf); + case FieldBinaryTypeIndex::Tuple: + return decodeArrayLikeField(buf); + case FieldBinaryTypeIndex::Map: + { + size_t size; + readVarUInt(size, buf); + Map map; + for (size_t i = 0; i != size; ++i) + { + Tuple key_and_value; + key_and_value.push_back(decodeField(buf)); + key_and_value.push_back(decodeField(buf)); + map.push_back(key_and_value); + } + return map; + } + case FieldBinaryTypeIndex::Object: + { + size_t size; + readVarUInt(size, buf); + Object value; + for (size_t i = 0; i != size; ++i) + { + String name; + readStringBinary(name, buf); + value[name] = decodeField(buf); + } + return value; + } + case FieldBinaryTypeIndex::AggregateFunctionState: + { + String name; + readStringBinary(name, buf); + String data; + readStringBinary(data, buf); + return AggregateFunctionStateData{.name = name, .data = data}; + } + } + + throw Exception(ErrorCodes::INCORRECT_DATA, "Unknown Field type: {0:#04x}", UInt64(type)); +} + +} diff --git a/src/Common/FieldBinaryEncoding.h b/src/Common/FieldBinaryEncoding.h new file mode 100644 index 00000000000..aa6694cb03e --- /dev/null +++ b/src/Common/FieldBinaryEncoding.h @@ -0,0 +1,43 @@ +#pragma once + +#include + +namespace DB +{ + +/** +Binary encoding for Fields: +|--------------------------|--------------------------------------------------------------------------------------------------------------------------------| +| Field type | Binary encoding | +|--------------------------|--------------------------------------------------------------------------------------------------------------------------------| +| `Null` | `0x00` | +| `UInt64` | `0x01` | +| `Int64` | `0x02` | +| `UInt128` | `0x03` | +| `Int128` | `0x04` | +| `UInt128` | `0x05` | +| `Int128` | `0x06` | +| `Float64` | `0x07` | +| `Decimal32` | `0x08` | +| `Decimal64` | `0x09` | +| `Decimal128` | `0x0A` | +| `Decimal256` | `0x0B` | +| `String` | `0x0C` | +| `Array` | `0x0D...` | +| `Tuple` | `0x0E...` | +| `Map` | `0x0F...` | +| `IPv4` | `0x10` | +| `IPv6` | `0x11` | +| `UUID` | `0x12` | +| `Bool` | `0x13` | +| `Object` | `0x14...` | +| `AggregateFunctionState` | `0x15` | +| `Negative infinity` | `0xFE` | +| `Positive infinity` | `0xFF` | +|--------------------------|--------------------------------------------------------------------------------------------------------------------------------| +*/ + +void encodeField(const Field &, WriteBuffer & buf); +Field decodeField(ReadBuffer & buf); + +} diff --git a/src/Common/IntervalKind.h b/src/Common/IntervalKind.h index f8e1fe87276..d66ea6018d4 100644 --- a/src/Common/IntervalKind.h +++ b/src/Common/IntervalKind.h @@ -7,19 +7,20 @@ namespace DB /// Kind of a temporal interval. struct IntervalKind { + /// note: The order and numbers are important and used in binary encoding, append new interval kinds to the end of list. enum class Kind : uint8_t { - Nanosecond, - Microsecond, - Millisecond, - Second, - Minute, - Hour, - Day, - Week, - Month, - Quarter, - Year, + Nanosecond = 0x00, + Microsecond = 0x01, + Millisecond = 0x02, + Second = 0x03, + Minute = 0x04, + Hour = 0x05, + Day = 0x06, + Week = 0x07, + Month = 0x08, + Quarter = 0x09, + Year = 0x0A, }; Kind kind = Kind::Second; diff --git a/src/Core/Field.cpp b/src/Core/Field.cpp index 0e5b1bac000..fb820ad2b56 100644 --- a/src/Core/Field.cpp +++ b/src/Core/Field.cpp @@ -1,11 +1,12 @@ -#include -#include -#include -#include -#include -#include -#include #include +#include +#include +#include +#include +#include +#include +#include +#include #include #include #include @@ -21,6 +22,7 @@ namespace ErrorCodes extern const int CANNOT_RESTORE_FROM_FIELD_DUMP; extern const int DECIMAL_OVERFLOW; extern const int INCORRECT_DATA; + extern const int UNSUPPORTED_METHOD; } template diff --git a/src/Core/Settings.h b/src/Core/Settings.h index bda403b1b40..23c5d7fc1a2 100644 --- a/src/Core/Settings.h +++ b/src/Core/Settings.h @@ -1105,6 +1105,8 @@ class IColumn; M(Bool, input_format_tsv_crlf_end_of_line, false, "If it is set true, file function will read TSV format with \\r\\n instead of \\n.", 0) \ \ M(Bool, input_format_native_allow_types_conversion, true, "Allow data types conversion in Native input format", 0) \ + M(Bool, input_format_native_decode_types_in_binary_format, false, "Read data types in binary format instead of type names in Native input format", 0) \ + M(Bool, output_format_native_encode_types_in_binary_format, false, "Write data types in binary format instead of type names in Native output format", 0) \ \ M(DateTimeInputFormat, date_time_input_format, FormatSettings::DateTimeInputFormat::Basic, "Method to read DateTime from text input formats. Possible values: 'basic', 'best_effort' and 'best_effort_us'.", 0) \ M(DateTimeOutputFormat, date_time_output_format, FormatSettings::DateTimeOutputFormat::Simple, "Method to write DateTime to text output. Possible values: 'simple', 'iso', 'unix_timestamp'.", 0) \ @@ -1124,6 +1126,8 @@ class IColumn; M(Bool, input_format_avro_null_as_default, false, "For Avro/AvroConfluent format: insert default in case of null and non Nullable column", 0) \ M(UInt64, format_binary_max_string_size, 1_GiB, "The maximum allowed size for String in RowBinary format. It prevents allocating large amount of memory in case of corrupted data. 0 means there is no limit", 0) \ M(UInt64, format_binary_max_array_size, 1_GiB, "The maximum allowed size for Array in RowBinary format. It prevents allocating large amount of memory in case of corrupted data. 0 means there is no limit", 0) \ + M(Bool, input_format_binary_decode_types_in_binary_format, false, "Read data types in binary format instead of type names in RowBinaryWithNamesAndTypes input format", 0) \ + M(Bool, output_format_binary_encode_types_in_binary_format, false, "Write data types in binary format instead of type names in RowBinaryWithNamesAndTypes output format ", 0) \ M(URI, format_avro_schema_registry_url, "", "For AvroConfluent format: Confluent Schema Registry URL.", 0) \ \ M(Bool, output_format_json_quote_64bit_integers, true, "Controls quoting of 64-bit integers in JSON output format.", 0) \ diff --git a/src/Core/tests/gtest_fields_binary_enciding.cpp b/src/Core/tests/gtest_fields_binary_enciding.cpp new file mode 100644 index 00000000000..087caf746bb --- /dev/null +++ b/src/Core/tests/gtest_fields_binary_enciding.cpp @@ -0,0 +1,65 @@ +#include +#include +#include +#include + +using namespace DB; + +namespace DB::ErrorCodes +{ + extern const int UNSUPPORTED_METHOD; +} + + +void check(const Field & field) +{ +// std::cerr << "Check " << toString(field) << "\n"; + WriteBufferFromOwnString ostr; + encodeField(field, ostr); + ReadBufferFromString istr(ostr.str()); + Field decoded_field = decodeField(istr); + ASSERT_TRUE(istr.eof()); + ASSERT_EQ(field, decoded_field); +} + +GTEST_TEST(FieldBinaryEncoding, EncodeAndDecode) +{ + check(Null()); + check(POSITIVE_INFINITY); + check(NEGATIVE_INFINITY); + check(true); + check(UInt64(42)); + check(Int64(-42)); + check(UInt128(42)); + check(Int128(-42)); + check(UInt256(42)); + check(Int256(-42)); + check(UUID(42)); + check(IPv4(42)); + check(IPv6(42)); + check(Float64(42.42)); + check(String("Hello, World!")); + check(Array({Field(UInt64(42)), Field(UInt64(43))})); + check(Tuple({Field(UInt64(42)), Field(Null()), Field(UUID(42)), Field(String("Hello, World!"))})); + check(Map({Tuple{Field(UInt64(42)), Field(String("str_42"))}, Tuple{Field(UInt64(43)), Field(String("str_43"))}})); + check(Object({{String("key_1"), Field(UInt64(42))}, {String("key_2"), Field(UInt64(43))}})); + check(DecimalField(4242, 3)); + check(DecimalField(4242, 3)); + check(DecimalField(Int128(4242), 3)); + check(DecimalField(Int256(4242), 3)); + check(AggregateFunctionStateData{.name="some_name", .data="some_data"}); + try + { + check(CustomType()); + } + catch (const Exception & e) + { + ASSERT_EQ(e.code(), ErrorCodes::UNSUPPORTED_METHOD); + } + + check(Array({ + Tuple({Field(UInt64(42)), Map({Tuple{Field(UInt64(42)), Field(String("str_42"))}, Tuple{Field(UInt64(43)), Field(String("str_43"))}}), Field(UUID(42)), Field(String("Hello, World!"))}), + Tuple({Field(UInt64(43)), Map({Tuple{Field(UInt64(43)), Field(String("str_43"))}, Tuple{Field(UInt64(44)), Field(String("str_44"))}}), Field(UUID(43)), Field(String("Hello, World 2!"))}) + })); +} + diff --git a/src/DataTypes/DataTypeAggregateFunction.h b/src/DataTypes/DataTypeAggregateFunction.h index 8b4b3d6ee4c..52ed151107e 100644 --- a/src/DataTypes/DataTypeAggregateFunction.h +++ b/src/DataTypes/DataTypeAggregateFunction.h @@ -25,7 +25,6 @@ private: mutable std::optional version; String getNameImpl(bool with_version) const; - size_t getVersion() const; public: static constexpr bool is_parametric = true; @@ -39,6 +38,8 @@ public: { } + size_t getVersion() const; + String getFunctionName() const; AggregateFunctionPtr getFunction() const { return function; } diff --git a/src/DataTypes/DataTypeCustomSimpleAggregateFunction.cpp b/src/DataTypes/DataTypeCustomSimpleAggregateFunction.cpp index cae9622bcb9..d3b3adc4965 100644 --- a/src/DataTypes/DataTypeCustomSimpleAggregateFunction.cpp +++ b/src/DataTypes/DataTypeCustomSimpleAggregateFunction.cpp @@ -165,6 +165,19 @@ static std::pair create(const ASTPtr & argum return std::make_pair(storage_type, std::make_unique(std::move(custom_name), nullptr)); } +String DataTypeCustomSimpleAggregateFunction::getFunctionName() const +{ + return function->getName(); +} + +DataTypePtr createSimpleAggregateFunctionType(const AggregateFunctionPtr & function, const DataTypes & argument_types, const Array & parameters) +{ + auto custom_desc = std::make_unique( + std::make_unique(function, argument_types, parameters)); + + return DataTypeFactory::instance().getCustom(std::move(custom_desc)); +} + void registerDataTypeDomainSimpleAggregateFunction(DataTypeFactory & factory) { factory.registerDataTypeCustom("SimpleAggregateFunction", create); diff --git a/src/DataTypes/DataTypeCustomSimpleAggregateFunction.h b/src/DataTypes/DataTypeCustomSimpleAggregateFunction.h index bdabb465fe5..303da86979a 100644 --- a/src/DataTypes/DataTypeCustomSimpleAggregateFunction.h +++ b/src/DataTypes/DataTypeCustomSimpleAggregateFunction.h @@ -40,8 +40,13 @@ public: : function(function_), argument_types(argument_types_), parameters(parameters_) {} AggregateFunctionPtr getFunction() const { return function; } + String getFunctionName() const; + const DataTypes & getArgumentsDataTypes() const { return argument_types; } + const Array & getParameters() const { return parameters; } String getName() const override; static void checkSupportedFunctions(const AggregateFunctionPtr & function); }; +DataTypePtr createSimpleAggregateFunctionType(const AggregateFunctionPtr & function, const DataTypes & argument_types, const Array & parameters); + } diff --git a/src/DataTypes/DataTypeNested.h b/src/DataTypes/DataTypeNested.h index 1ad06477a6e..102e6c293cc 100644 --- a/src/DataTypes/DataTypeNested.h +++ b/src/DataTypes/DataTypeNested.h @@ -19,6 +19,8 @@ public: } String getName() const override; + const DataTypes & getElements() const { return elems; } + const Names & getNames() const { return names; } }; DataTypePtr createNested(const DataTypes & types, const Names & names); diff --git a/src/DataTypes/DataTypesBinaryEncoding.cpp b/src/DataTypes/DataTypesBinaryEncoding.cpp new file mode 100644 index 00000000000..4c42f650798 --- /dev/null +++ b/src/DataTypes/DataTypesBinaryEncoding.cpp @@ -0,0 +1,677 @@ +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include + +namespace DB +{ + +namespace ErrorCodes +{ + extern const int UNSUPPORTED_METHOD; + extern const int UNKNOWN_TYPE; +} + +namespace +{ + +enum class BinaryTypeIndex : uint8_t +{ + Nothing = 0x00, + UInt8 = 0x01, + UInt16 = 0x02, + UInt32 = 0x03, + UInt64 = 0x04, + UInt128 = 0x05, + UInt256 = 0x06, + Int8 = 0x07, + Int16 = 0x08, + Int32 = 0x09, + Int64 = 0x0A, + Int128 = 0x0B, + Int256 = 0x0C, + Float32 = 0x0D, + Float64 = 0x0E, + Date = 0x0F, + Date32 = 0x10, + DateTime = 0x11, + DateTime64 = 0x12, + String = 0x13, + FixedString = 0x14, + Enum8 = 0x15, + Enum16 = 0x16, + Decimal32 = 0x17, + Decimal64 = 0x18, + Decimal128 = 0x19, + Decimal256 = 0x1A, + UUID = 0x1B, + Array = 0x1C, + UnnamedTuple = 0x1D, + NamedTuple = 0x1E, + Set = 0x1F, + Interval = 0x20, + Nullable = 0x21, + Function = 0x22, + AggregateFunction = 0x23, + LowCardinality = 0x24, + Map = 0x25, + Object = 0x26, + IPv4 = 0x27, + IPv6 = 0x28, + Variant = 0x29, + Dynamic = 0x2A, + Custom = 0x2B, + Bool = 0x2C, + SimpleAggregateFunction = 0x2D, + Nested = 0x2E, +}; + +BinaryTypeIndex getBinaryTypeIndex(const DataTypePtr & type) +{ + /// By default custom types don't have their own BinaryTypeIndex. + if (type->hasCustomName()) + { + /// Some widely used custom types have separate BinaryTypeIndex for better serialization. + /// Right now it's Bool, SimpleAggregateFunction and Nested types. + /// TODO: Consider adding BinaryTypeIndex for more custom types. + + if (isBool(type)) + return BinaryTypeIndex::Bool; + + if (typeid_cast(type->getCustomName())) + return BinaryTypeIndex::SimpleAggregateFunction; + + if (isNested(type)) + return BinaryTypeIndex::Nested; + + return BinaryTypeIndex::Custom; + } + + switch (type->getTypeId()) + { + case TypeIndex::Nothing: + return BinaryTypeIndex::Nothing; + case TypeIndex::UInt8: + return BinaryTypeIndex::UInt8; + case TypeIndex::UInt16: + return BinaryTypeIndex::UInt16; + case TypeIndex::UInt32: + return BinaryTypeIndex::UInt32; + case TypeIndex::UInt64: + return BinaryTypeIndex::UInt64; + case TypeIndex::UInt128: + return BinaryTypeIndex::UInt128; + case TypeIndex::UInt256: + return BinaryTypeIndex::UInt256; + case TypeIndex::Int8: + return BinaryTypeIndex::Int8; + case TypeIndex::Int16: + return BinaryTypeIndex::Int16; + case TypeIndex::Int32: + return BinaryTypeIndex::Int32; + case TypeIndex::Int64: + return BinaryTypeIndex::Int64; + case TypeIndex::Int128: + return BinaryTypeIndex::Int128; + case TypeIndex::Int256: + return BinaryTypeIndex::Int256; + case TypeIndex::Float32: + return BinaryTypeIndex::Float32; + case TypeIndex::Float64: + return BinaryTypeIndex::Float64; + case TypeIndex::Date: + return BinaryTypeIndex::Date; + case TypeIndex::Date32: + return BinaryTypeIndex::Date32; + case TypeIndex::DateTime: + return BinaryTypeIndex::DateTime; + case TypeIndex::DateTime64: + return BinaryTypeIndex::DateTime64; + case TypeIndex::String: + return BinaryTypeIndex::String; + case TypeIndex::FixedString: + return BinaryTypeIndex::FixedString; + case TypeIndex::Enum8: + return BinaryTypeIndex::Enum8; + case TypeIndex::Enum16: + return BinaryTypeIndex::Enum16; + case TypeIndex::Decimal32: + return BinaryTypeIndex::Decimal32; + case TypeIndex::Decimal64: + return BinaryTypeIndex::Decimal64; + case TypeIndex::Decimal128: + return BinaryTypeIndex::Decimal128; + case TypeIndex::Decimal256: + return BinaryTypeIndex::Decimal256; + case TypeIndex::UUID: + return BinaryTypeIndex::UUID; + case TypeIndex::Array: + return BinaryTypeIndex::Array; + case TypeIndex::Tuple: + { + const auto & tuple_type = assert_cast(*type); + if (tuple_type.haveExplicitNames()) + return BinaryTypeIndex::NamedTuple; + return BinaryTypeIndex::UnnamedTuple; + } + case TypeIndex::Set: + return BinaryTypeIndex::Set; + case TypeIndex::Interval: + return BinaryTypeIndex::Interval; + case TypeIndex::Nullable: + return BinaryTypeIndex::Nullable; + case TypeIndex::Function: + return BinaryTypeIndex::Function; + case TypeIndex::AggregateFunction: + return BinaryTypeIndex::AggregateFunction; + case TypeIndex::LowCardinality: + return BinaryTypeIndex::LowCardinality; + case TypeIndex::Map: + return BinaryTypeIndex::Map; + case TypeIndex::Object: + return BinaryTypeIndex::Object; + case TypeIndex::IPv4: + return BinaryTypeIndex::IPv4; + case TypeIndex::IPv6: + return BinaryTypeIndex::IPv6; + case TypeIndex::Variant: + return BinaryTypeIndex::Variant; + case TypeIndex::Dynamic: + return BinaryTypeIndex::Dynamic; + /// JSONPaths is used only during schema inference and cannot be used anywhere else. + case TypeIndex::JSONPaths: + throw Exception(ErrorCodes::UNSUPPORTED_METHOD, "Binary encoding of type JSONPaths is not supported"); + } +} + +template +void encodeEnumValues(const DataTypePtr & type, WriteBuffer & buf) +{ + const auto & enum_type = assert_cast &>(*type); + const auto & values = enum_type.getValues(); + writeVarUInt(values.size(), buf); + for (const auto & [name, value] : values) + { + writeStringBinary(name, buf); + writeBinaryLittleEndian(value, buf); + } +} + +template +DataTypePtr decodeEnum(ReadBuffer & buf) +{ + typename DataTypeEnum::Values values; + size_t size; + readVarUInt(size, buf); + for (size_t i = 0; i != size; ++i) + { + String name; + readStringBinary(name, buf); + T value; + readBinaryLittleEndian(value, buf); + values.emplace_back(name, value); + } + + return std::make_shared>(values); +} + +template +void encodeDecimal(const DataTypePtr & type, WriteBuffer & buf) +{ + const auto & decimal_type = assert_cast &>(*type); + /// Both precision and scale should be less than 76, so we can decode it in 1 byte. + writeBinary(UInt8(decimal_type.getPrecision()), buf); + writeBinary(UInt8(decimal_type.getScale()), buf); +} + +template +DataTypePtr decodeDecimal(ReadBuffer & buf) +{ + UInt8 precision; + readBinary(precision, buf); + UInt8 scale; + readBinary(scale, buf); + return std::make_shared>(precision, scale); +} + +void encodeAggregateFunction(const String & function_name, const Array & parameters, const DataTypes & arguments_types, WriteBuffer & buf) +{ + writeStringBinary(function_name, buf); + writeVarUInt(parameters.size(), buf); + for (const auto & param : parameters) + encodeField(param, buf); + writeVarUInt(arguments_types.size(), buf); + for (const auto & argument_type : arguments_types) + encodeDataType(argument_type, buf); +} + +std::tuple decodeAggregateFunction(ReadBuffer & buf) +{ + String function_name; + readStringBinary(function_name, buf); + size_t num_parameters; + readVarUInt(num_parameters, buf); + Array parameters; + parameters.reserve(num_parameters); + for (size_t i = 0; i != num_parameters; ++i) + parameters.push_back(decodeField(buf)); + size_t num_arguments; + readVarUInt(num_arguments, buf); + DataTypes arguments_types; + arguments_types.reserve(num_arguments); + for (size_t i = 0; i != num_arguments; ++i) + arguments_types.push_back(decodeDataType(buf)); + AggregateFunctionProperties properties; + auto action = NullsAction::EMPTY; + auto function = AggregateFunctionFactory::instance().get(function_name, action, arguments_types, parameters, properties); + return {function, parameters, arguments_types}; +} + +} + +void encodeDataType(const DataTypePtr & type, WriteBuffer & buf) +{ + /// First, write the BinaryTypeIndex byte. + auto binary_type_index = getBinaryTypeIndex(type); + buf.write(UInt8(binary_type_index)); + /// Then, write additional information depending on the data type. + switch (binary_type_index) + { + case BinaryTypeIndex::DateTime64: + { + const auto & datetime64_type = assert_cast(*type); + /// Maximum scale for DateTime64 is 9, so we can write it as 1 byte. + buf.write(UInt8(datetime64_type.getScale())); + break; + } + case BinaryTypeIndex::FixedString: + { + const auto & fixed_string_type = assert_cast(*type); + writeVarUInt(fixed_string_type.getN(), buf); + break; + } + case BinaryTypeIndex::Enum8: + { + encodeEnumValues(type, buf); + break; + } + case BinaryTypeIndex::Enum16: + { + encodeEnumValues(type, buf); + break; + } + case BinaryTypeIndex::Decimal32: + { + encodeDecimal(type, buf); + break; + } + case BinaryTypeIndex::Decimal64: + { + encodeDecimal(type, buf); + break; + } + case BinaryTypeIndex::Decimal128: + { + encodeDecimal(type, buf); + break; + } + case BinaryTypeIndex::Decimal256: + { + encodeDecimal(type, buf); + break; + } + case BinaryTypeIndex::Array: + { + const auto & array_type = assert_cast(*type); + encodeDataType(array_type.getNestedType(), buf); + break; + } + case BinaryTypeIndex::NamedTuple: + { + const auto & tuple_type = assert_cast(*type); + const auto & types = tuple_type.getElements(); + const auto & names = tuple_type.getElementNames(); + writeVarUInt(types.size(), buf); + for (size_t i = 0; i != types.size(); ++i) + { + writeStringBinary(names[i], buf); + encodeDataType(types[i], buf); + } + break; + } + case BinaryTypeIndex::UnnamedTuple: + { + const auto & tuple_type = assert_cast(*type); + const auto & types = tuple_type.getElements(); + writeVarUInt(types.size(), buf); + for (size_t i = 0; i != types.size(); ++i) + encodeDataType(types[i], buf); + break; + } + case BinaryTypeIndex::Interval: + { + const auto & interval_type = assert_cast(*type); + writeBinary(UInt8(interval_type.getKind().kind), buf); + break; + } + case BinaryTypeIndex::Nullable: + { + const auto & nullable_type = assert_cast(*type); + encodeDataType(nullable_type.getNestedType(), buf); + break; + } + case BinaryTypeIndex::Function: + { + const auto & function_type = assert_cast(*type); + const auto & arguments_types = function_type.getArgumentTypes(); + const auto & return_type = function_type.getReturnType(); + writeVarUInt(arguments_types.size(), buf); + for (const auto & argument_type : arguments_types) + encodeDataType(argument_type, buf); + encodeDataType(return_type, buf); + break; + } + case BinaryTypeIndex::LowCardinality: + { + const auto & low_cardinality_type = assert_cast(*type); + encodeDataType(low_cardinality_type.getDictionaryType(), buf); + break; + } + case BinaryTypeIndex::Map: + { + const auto & map_type = assert_cast(*type); + encodeDataType(map_type.getKeyType(), buf); + encodeDataType(map_type.getValueType(), buf); + break; + } + case BinaryTypeIndex::Object: + { + const auto & object_deprecated_type = assert_cast(*type); + writeBinary(object_deprecated_type.hasNullableSubcolumns(), buf); + writeStringBinary(object_deprecated_type.getSchemaFormat(), buf); + break; + } + case BinaryTypeIndex::Variant: + { + const auto & variant_type = assert_cast(*type); + const auto & variants = variant_type.getVariants(); + writeVarUInt(variants.size(), buf); + for (const auto & variant : variants) + encodeDataType(variant, buf); + break; + } + case BinaryTypeIndex::AggregateFunction: + { + const auto & aggregate_function_type = assert_cast(*type); + writeVarUInt(aggregate_function_type.getVersion(), buf); + encodeAggregateFunction(aggregate_function_type.getFunctionName(), aggregate_function_type.getParameters(), aggregate_function_type.getArgumentsDataTypes(), buf); + break; + } + case BinaryTypeIndex::SimpleAggregateFunction: + { + const auto & simple_aggregate_function_type = assert_cast(*type->getCustomName()); + encodeAggregateFunction(simple_aggregate_function_type.getFunctionName(), simple_aggregate_function_type.getParameters(), simple_aggregate_function_type.getArgumentsDataTypes(), buf); + break; + } + case BinaryTypeIndex::Nested: + { + const auto & nested_type = assert_cast(*type->getCustomName()); + const auto & elements = nested_type.getElements(); + const auto & names = nested_type.getNames(); + writeVarUInt(elements.size(), buf); + for (size_t i = 0; i != elements.size(); ++i) + { + writeStringBinary(names[i], buf); + encodeDataType(elements[i], buf); + } + break; + } + case BinaryTypeIndex::Custom: + { + const auto & type_name = type->getName(); + writeStringBinary(type_name, buf); + break; + } + default: + break; + } +} + +String encodeDataType(const DataTypePtr & type) +{ + WriteBufferFromOwnString buf; + encodeDataType(type, buf); + return buf.str(); +} + +DataTypePtr decodeDataType(ReadBuffer & buf) +{ + UInt8 type; + readBinary(type, buf); + switch (BinaryTypeIndex(type)) + { + case BinaryTypeIndex::Nothing: + return std::make_shared(); + case BinaryTypeIndex::UInt8: + return std::make_shared(); + case BinaryTypeIndex::Bool: + return DataTypeFactory::instance().get("Bool"); + case BinaryTypeIndex::UInt16: + return std::make_shared(); + case BinaryTypeIndex::UInt32: + return std::make_shared(); + case BinaryTypeIndex::UInt64: + return std::make_shared(); + case BinaryTypeIndex::UInt128: + return std::make_shared(); + case BinaryTypeIndex::UInt256: + return std::make_shared(); + case BinaryTypeIndex::Int8: + return std::make_shared(); + case BinaryTypeIndex::Int16: + return std::make_shared(); + case BinaryTypeIndex::Int32: + return std::make_shared(); + case BinaryTypeIndex::Int64: + return std::make_shared(); + case BinaryTypeIndex::Int128: + return std::make_shared(); + case BinaryTypeIndex::Int256: + return std::make_shared(); + case BinaryTypeIndex::Float32: + return std::make_shared(); + case BinaryTypeIndex::Float64: + return std::make_shared(); + case BinaryTypeIndex::Date: + return std::make_shared(); + case BinaryTypeIndex::Date32: + return std::make_shared(); + case BinaryTypeIndex::DateTime: + return std::make_shared(); + case BinaryTypeIndex::DateTime64: + { + UInt8 scale; + readBinary(scale, buf); + return std::make_shared(scale); + } + case BinaryTypeIndex::String: + return std::make_shared(); + case BinaryTypeIndex::FixedString: + { + UInt64 size; + readVarUInt(size, buf); + return std::make_shared(size); + } + case BinaryTypeIndex::Enum8: + return decodeEnum(buf); + case BinaryTypeIndex::Enum16: + return decodeEnum(buf); + case BinaryTypeIndex::Decimal32: + return decodeDecimal(buf); + case BinaryTypeIndex::Decimal64: + return decodeDecimal(buf); + case BinaryTypeIndex::Decimal128: + return decodeDecimal(buf); + case BinaryTypeIndex::Decimal256: + return decodeDecimal(buf); + case BinaryTypeIndex::UUID: + return std::make_shared(); + case BinaryTypeIndex::Array: + return std::make_shared(decodeDataType(buf)); + case BinaryTypeIndex::NamedTuple: + { + size_t size; + readVarUInt(size, buf); + DataTypes elements; + elements.reserve(size); + Names names; + names.reserve(size); + for (size_t i = 0; i != size; ++i) + { + names.emplace_back(); + readStringBinary(names.back(), buf); + elements.push_back(decodeDataType(buf)); + } + + return std::make_shared(elements, names); + } + case BinaryTypeIndex::UnnamedTuple: + { + size_t size; + readVarUInt(size, buf); + DataTypes elements; + elements.reserve(size); + for (size_t i = 0; i != size; ++i) + elements.push_back(decodeDataType(buf)); + return std::make_shared(elements); + } + case BinaryTypeIndex::Set: + return std::make_shared(); + case BinaryTypeIndex::Interval: + { + UInt8 kind; + readBinary(kind, buf); + return std::make_shared(IntervalKind(IntervalKind::Kind(kind))); + } + case BinaryTypeIndex::Nullable: + return std::make_shared(decodeDataType(buf)); + case BinaryTypeIndex::Function: + { + size_t arguments_size; + readVarUInt(arguments_size, buf); + DataTypes arguments; + arguments.reserve(arguments_size); + for (size_t i = 0; i != arguments_size; ++i) + arguments.push_back(decodeDataType(buf)); + auto return_type = decodeDataType(buf); + return std::make_shared(arguments, return_type); + } + case BinaryTypeIndex::LowCardinality: + return std::make_shared(decodeDataType(buf)); + case BinaryTypeIndex::Map: + { + auto key_type = decodeDataType(buf); + auto value_type = decodeDataType(buf); + return std::make_shared(key_type, value_type); + } + case BinaryTypeIndex::Object: + { + bool has_nullable_subcolumns; + readBinary(has_nullable_subcolumns, buf); + String schema_format; + readStringBinary(schema_format, buf); + return std::make_shared(schema_format, has_nullable_subcolumns); + } + case BinaryTypeIndex::IPv4: + return std::make_shared(); + case BinaryTypeIndex::IPv6: + return std::make_shared(); + case BinaryTypeIndex::Variant: + { + size_t size; + readVarUInt(size, buf); + DataTypes variants; + variants.reserve(size); + for (size_t i = 0; i != size; ++i) + variants.push_back(decodeDataType(buf)); + return std::make_shared(variants); + } + case BinaryTypeIndex::Dynamic: + return std::make_shared(); + case BinaryTypeIndex::AggregateFunction: + { + size_t version; + readVarUInt(version, buf); + const auto & [function, parameters, arguments_types] = decodeAggregateFunction(buf); + return std::make_shared(function, arguments_types, parameters, version); + } + case BinaryTypeIndex::SimpleAggregateFunction: + { + const auto & [function, parameters, arguments_types] = decodeAggregateFunction(buf); + return createSimpleAggregateFunctionType(function, arguments_types, parameters); + } + case BinaryTypeIndex::Nested: + { + size_t size; + readVarUInt(size, buf); + Names names; + names.reserve(size); + DataTypes elements; + elements.reserve(size); + for (size_t i = 0; i != size; ++i) + { + names.emplace_back(); + readStringBinary(names.back(), buf); + elements.push_back(decodeDataType(buf)); + } + + return createNested(elements, names); + } + case BinaryTypeIndex::Custom: + { + String type_name; + readStringBinary(type_name, buf); + return DataTypeFactory::instance().get(type_name); + } + } + + throw Exception(ErrorCodes::UNKNOWN_TYPE, "Unknown type code: {0:#04x}", UInt64(type)); +} + +DataTypePtr decodeDataType(const String & data) +{ + ReadBufferFromString buf(data); + return decodeDataType(buf); +} + +} diff --git a/src/DataTypes/DataTypesBinaryEncoding.h b/src/DataTypes/DataTypesBinaryEncoding.h new file mode 100644 index 00000000000..b4ed500f185 --- /dev/null +++ b/src/DataTypes/DataTypesBinaryEncoding.h @@ -0,0 +1,117 @@ +#pragma once + +#include + +namespace DB +{ + +/** + +Binary encoding for ClickHouse data types: +|--------------------------------------------------------------------------------------|--------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------| +| ClickHouse data type | Binary encoding | +|--------------------------------------------------------------------------------------|--------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------| +| `Nothing` | `0x00` | +| `UInt8` | `0x01` | +| `UInt16` | `0x02` | +| `UInt32` | `0x03` | +| `UInt64` | `0x04` | +| `UInt128` | `0x05` | +| `UInt256` | `0x06` | +| `Int8` | `0x07` | +| `Int16` | `0x08` | +| `Int32` | `0x09` | +| `Int64` | `0x0A` | +| `Int128` | `0x0B` | +| `Int256` | `0x0C` | +| `Float32` | `0x0D` | +| `Float64` | `0x0E` | +| `Date` | `0x0F` | +| `Date32` | `0x10` | +| `DateTime` | `0x11` | +| `DateTime64(P)` | `0x12` | +| `String` | `0x13` | +| `FixedString(N)` | `0x14` | +| `Enum8` | `0x15...` | +| `Enum16` | `0x16...>` | +| `Decimal32(P, S)` | `0x17` | +| `Decimal64(P, S)` | `0x18` | +| `Decimal128(P, S)` | `0x19` | +| `Decimal256(P, S)` | `0x1A` | +| `UUID` | `0x1B` | +| `Array(T)` | `0x1C` | +| `Tuple(T1, ..., TN)` | `0x1D...` | +| `Tuple(name1 T1, ..., nameN TN)` | `0x1E...` | +| `Set` | `0x1F` | +| `Interval` | `0x20` | +| `Nullable(T)` | `0x21` | +| `Function` | `0x22...` | +| `AggregateFunction(function_name(param_1, ..., param_N), arg_T1, ..., arg_TN)` | `0x23......` | +| `LowCardinality(T)` | `0x24` | +| `Map(K, V)` | `0x25` | +| `Object('schema_format')` | `0x26` | +| `IPv4` | `0x27` | +| `IPv6` | `0x28` | +| `Variant(T1, ..., TN)` | `0x29...` | +| `Dynamic` | `0x2A` | +| `Custom type` (`Ring`, `Polygon`, etc) | `0x2B` | +| `Bool` | `0x2C` | +| `SimpleAggregateFunction(function_name(param_1, ..., param_N), arg_T1, ..., arg_TN)` | `0x2D......` | +| `Nested(name1 T1, ..., nameN TN)` | `0x2E...` | +|--------------------------------------------------------------------------------------|--------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------| + +Interval kind binary encoding: +|---------------|-----------------| +| Interval kind | Binary encoding | +|---------------|-----------------| +| `Nanosecond` | `0x00` | +| `Microsecond` | `0x01` | +| `Millisecond` | `0x02` | +| `Second` | `0x03` | +| `Minute` | `0x04` | +| `Hour` | `0x05` | +| `Day` | `0x06` | +| `Week` | `0x07` | +| `Month` | `0x08` | +| `Quarter` | `0x09` | +| `Year` | `0x1A` | +|---------------|-----------------| + +Aggregate function parameter binary encoding (binary encoding of a Field, see src/Common/FieldBinaryEncoding.h): +|--------------------------|--------------------------------------------------------------------------------------------------------------------------------| +| Parameter type | Binary encoding | +|--------------------------|--------------------------------------------------------------------------------------------------------------------------------| +| `Null` | `0x00` | +| `UInt64` | `0x01` | +| `Int64` | `0x02` | +| `UInt128` | `0x03` | +| `Int128` | `0x04` | +| `UInt128` | `0x05` | +| `Int128` | `0x06` | +| `Float64` | `0x07` | +| `Decimal32` | `0x08` | +| `Decimal64` | `0x09` | +| `Decimal128` | `0x0A` | +| `Decimal256` | `0x0B` | +| `String` | `0x0C` | +| `Array` | `0x0D...` | +| `Tuple` | `0x0E...` | +| `Map` | `0x0F...` | +| `IPv4` | `0x10` | +| `IPv6` | `0x11` | +| `UUID` | `0x12` | +| `Bool` | `0x13` | +| `Object` | `0x14...` | +| `AggregateFunctionState` | `0x15` | +| `Negative infinity` | `0xFE` | +| `Positive infinity` | `0xFF` | +|--------------------------|--------------------------------------------------------------------------------------------------------------------------------| +*/ + +String encodeDataType(const DataTypePtr & type); +void encodeDataType(const DataTypePtr & type, WriteBuffer & buf); + +DataTypePtr decodeDataType(const String & data); +DataTypePtr decodeDataType(ReadBuffer & buf); + +} diff --git a/src/DataTypes/Serializations/ISerialization.h b/src/DataTypes/Serializations/ISerialization.h index 914ff9cf4a2..f1ccb13f97f 100644 --- a/src/DataTypes/Serializations/ISerialization.h +++ b/src/DataTypes/Serializations/ISerialization.h @@ -256,6 +256,9 @@ public: bool position_independent_encoding = true; + /// True if data type names should be serialized in binary encoding. + bool data_types_binary_encoding = false; + enum class DynamicStatisticsMode { NONE, /// Don't write statistics. @@ -275,6 +278,9 @@ public: bool position_independent_encoding = true; + /// True if data type names should be deserialized in binary encoding. + bool data_types_binary_encoding = false; + bool native_format = false; /// If not zero, may be used to avoid reallocations while reading column of String type. diff --git a/src/DataTypes/Serializations/SerializationArray.cpp b/src/DataTypes/Serializations/SerializationArray.cpp index ac7b8f4d084..b7d43332085 100644 --- a/src/DataTypes/Serializations/SerializationArray.cpp +++ b/src/DataTypes/Serializations/SerializationArray.cpp @@ -42,13 +42,13 @@ void SerializationArray::deserializeBinary(Field & field, ReadBuffer & istr, con { size_t size; readVarUInt(size, istr); - if (settings.max_binary_array_size && size > settings.max_binary_array_size) + if (settings.binary.max_binary_string_size && size > settings.binary.max_binary_string_size) throw Exception( ErrorCodes::TOO_LARGE_ARRAY_SIZE, "Too large array size: {}. The maximum is: {}. To increase the maximum, use setting " "format_binary_max_array_size", size, - settings.max_binary_array_size); + settings.binary.max_binary_string_size); field = Array(); Array & arr = field.get(); @@ -82,13 +82,13 @@ void SerializationArray::deserializeBinary(IColumn & column, ReadBuffer & istr, size_t size; readVarUInt(size, istr); - if (settings.max_binary_array_size && size > settings.max_binary_array_size) + if (settings.binary.max_binary_string_size && size > settings.binary.max_binary_string_size) throw Exception( ErrorCodes::TOO_LARGE_ARRAY_SIZE, "Too large array size: {}. The maximum is: {}. To increase the maximum, use setting " "format_binary_max_array_size", size, - settings.max_binary_array_size); + settings.binary.max_binary_string_size); IColumn & nested_column = column_array.getData(); diff --git a/src/DataTypes/Serializations/SerializationDynamic.cpp b/src/DataTypes/Serializations/SerializationDynamic.cpp index 6351ff0ca0b..7609ffc91ca 100644 --- a/src/DataTypes/Serializations/SerializationDynamic.cpp +++ b/src/DataTypes/Serializations/SerializationDynamic.cpp @@ -4,6 +4,8 @@ #include #include #include +#include +#include #include #include @@ -109,7 +111,10 @@ void SerializationDynamic::serializeBinaryBulkStatePrefix( const auto & variant_column = column_dynamic.getVariantColumn(); /// Write internal Variant type name. - writeStringBinary(dynamic_state->variant_type->getName(), *stream); + if (settings.data_types_binary_encoding) + encodeDataType(dynamic_state->variant_type, *stream); + else + writeStringBinary(dynamic_state->variant_type->getName(), *stream); /// Write statistics in prefix if needed. if (settings.dynamic_write_statistics == SerializeBinaryBulkSettings::DynamicStatisticsMode::PREFIX) @@ -178,9 +183,16 @@ ISerialization::DeserializeBinaryBulkStatePtr SerializationDynamic::deserializeD readBinaryLittleEndian(structure_version, *structure_stream); auto structure_state = std::make_shared(structure_version); /// Read internal Variant type name. - String data_type_name; - readStringBinary(data_type_name, *structure_stream); - structure_state->variant_type = DataTypeFactory::instance().get(data_type_name); + if (settings.data_types_binary_encoding) + { + structure_state->variant_type = decodeDataType(*structure_stream); + } + else + { + String data_type_name; + readStringBinary(data_type_name, *structure_stream); + structure_state->variant_type = DataTypeFactory::instance().get(data_type_name); + } const auto * variant_type = typeid_cast(structure_state->variant_type.get()); if (!variant_type) throw Exception(ErrorCodes::INCORRECT_DATA, "Incorrect type of Dynamic nested column, expected Variant, got {}", structure_state->variant_type->getName()); @@ -280,33 +292,27 @@ void SerializationDynamic::deserializeBinaryBulkWithMultipleStreams( void SerializationDynamic::serializeBinary(const Field & field, WriteBuffer & ostr, const FormatSettings & settings) const { - UInt8 null_bit = field.isNull(); - writeBinary(null_bit, ostr); - if (null_bit) + /// Serialize NULL as Nothing type with no value. + if (field.isNull()) + { + encodeDataType(std::make_shared(), ostr); return; + } auto field_type = applyVisitor(FieldToDataType(), field); - auto field_type_name = field_type->getName(); - writeVarUInt(field_type_name.size(), ostr); - writeString(field_type_name, ostr); + encodeDataType(field_type, ostr); field_type->getDefaultSerialization()->serializeBinary(field, ostr, settings); } void SerializationDynamic::deserializeBinary(Field & field, ReadBuffer & istr, const FormatSettings & settings) const { - UInt8 null_bit; - readBinary(null_bit, istr); - if (null_bit) + auto field_type = decodeDataType(istr); + if (isNothing(field_type)) { field = Null(); return; } - size_t field_type_name_size; - readVarUInt(field_type_name_size, istr); - String field_type_name(field_type_name_size, 0); - istr.readStrict(field_type_name.data(), field_type_name_size); - auto field_type = DataTypeFactory::instance().get(field_type_name); field_type->getDefaultSerialization()->deserializeBinary(field, istr, settings); } @@ -317,15 +323,15 @@ void SerializationDynamic::serializeBinary(const IColumn & column, size_t row_nu const auto & variant_column = dynamic_column.getVariantColumn(); auto global_discr = variant_column.globalDiscriminatorAt(row_num); - UInt8 null_bit = global_discr == ColumnVariant::NULL_DISCRIMINATOR; - writeBinary(null_bit, ostr); - if (null_bit) + /// Serialize NULL as Nothing type with no value. + if (global_discr == ColumnVariant::NULL_DISCRIMINATOR) + { + encodeDataType(std::make_shared(), ostr); return; + } const auto & variant_type = assert_cast(*variant_info.variant_type).getVariant(global_discr); - const auto & variant_type_name = variant_info.variant_names[global_discr]; - writeVarUInt(variant_type_name.size(), ostr); - writeString(variant_type_name, ostr); + encodeDataType(variant_type, ostr); variant_type->getDefaultSerialization()->serializeBinary(variant_column.getVariantByGlobalDiscriminator(global_discr), variant_column.offsetAt(row_num), ostr, settings); } @@ -346,30 +352,23 @@ static void deserializeVariant( void SerializationDynamic::deserializeBinary(IColumn & column, ReadBuffer & istr, const FormatSettings & settings) const { auto & dynamic_column = assert_cast(column); - UInt8 null_bit; - readBinary(null_bit, istr); - if (null_bit) + auto variant_type = decodeDataType(istr); + if (isNothing(variant_type)) { dynamic_column.insertDefault(); return; } - size_t variant_type_name_size; - readVarUInt(variant_type_name_size, istr); - String variant_type_name(variant_type_name_size, 0); - istr.readStrict(variant_type_name.data(), variant_type_name_size); - + auto variant_type_name = variant_type->getName(); const auto & variant_info = dynamic_column.getVariantInfo(); auto it = variant_info.variant_name_to_discriminator.find(variant_type_name); if (it != variant_info.variant_name_to_discriminator.end()) { - const auto & variant_type = assert_cast(*variant_info.variant_type).getVariant(it->second); deserializeVariant(dynamic_column.getVariantColumn(), variant_type, it->second, istr, [&settings](const ISerialization & serialization, IColumn & variant, ReadBuffer & buf){ serialization.deserializeBinary(variant, buf, settings); }); return; } /// We don't have this variant yet. Let's try to add it. - auto variant_type = DataTypeFactory::instance().get(variant_type_name); if (dynamic_column.addNewVariant(variant_type)) { auto discr = variant_info.variant_name_to_discriminator.at(variant_type_name); diff --git a/src/DataTypes/Serializations/SerializationMap.cpp b/src/DataTypes/Serializations/SerializationMap.cpp index 70fe5182ade..0bef3c7d79d 100644 --- a/src/DataTypes/Serializations/SerializationMap.cpp +++ b/src/DataTypes/Serializations/SerializationMap.cpp @@ -55,13 +55,13 @@ void SerializationMap::deserializeBinary(Field & field, ReadBuffer & istr, const { size_t size; readVarUInt(size, istr); - if (settings.max_binary_array_size && size > settings.max_binary_array_size) + if (settings.binary.max_binary_string_size && size > settings.binary.max_binary_string_size) throw Exception( ErrorCodes::TOO_LARGE_ARRAY_SIZE, "Too large map size: {}. The maximum is: {}. To increase the maximum, use setting " "format_binary_max_array_size", size, - settings.max_binary_array_size); + settings.binary.max_binary_string_size); field = Map(); Map & map = field.get(); map.reserve(size); diff --git a/src/DataTypes/Serializations/SerializationString.cpp b/src/DataTypes/Serializations/SerializationString.cpp index 9e39ab23709..9e523d0d745 100644 --- a/src/DataTypes/Serializations/SerializationString.cpp +++ b/src/DataTypes/Serializations/SerializationString.cpp @@ -33,13 +33,13 @@ namespace ErrorCodes void SerializationString::serializeBinary(const Field & field, WriteBuffer & ostr, const FormatSettings & settings) const { const String & s = field.get(); - if (settings.max_binary_string_size && s.size() > settings.max_binary_string_size) + if (settings.binary.max_binary_string_size && s.size() > settings.binary.max_binary_string_size) throw Exception( ErrorCodes::TOO_LARGE_STRING_SIZE, "Too large string size: {}. The maximum is: {}. To increase the maximum, use setting " "format_binary_max_string_size", s.size(), - settings.max_binary_string_size); + settings.binary.max_binary_string_size); writeVarUInt(s.size(), ostr); writeString(s, ostr); @@ -50,13 +50,13 @@ void SerializationString::deserializeBinary(Field & field, ReadBuffer & istr, co { UInt64 size; readVarUInt(size, istr); - if (settings.max_binary_string_size && size > settings.max_binary_string_size) + if (settings.binary.max_binary_string_size && size > settings.binary.max_binary_string_size) throw Exception( ErrorCodes::TOO_LARGE_STRING_SIZE, "Too large string size: {}. The maximum is: {}. To increase the maximum, use setting " "format_binary_max_string_size", size, - settings.max_binary_string_size); + settings.binary.max_binary_string_size); field = String(); String & s = field.get(); @@ -68,13 +68,13 @@ void SerializationString::deserializeBinary(Field & field, ReadBuffer & istr, co void SerializationString::serializeBinary(const IColumn & column, size_t row_num, WriteBuffer & ostr, const FormatSettings & settings) const { const StringRef & s = assert_cast(column).getDataAt(row_num); - if (settings.max_binary_string_size && s.size > settings.max_binary_string_size) + if (settings.binary.max_binary_string_size && s.size > settings.binary.max_binary_string_size) throw Exception( ErrorCodes::TOO_LARGE_STRING_SIZE, "Too large string size: {}. The maximum is: {}. To increase the maximum, use setting " "format_binary_max_string_size", s.size, - settings.max_binary_string_size); + settings.binary.max_binary_string_size); writeVarUInt(s.size, ostr); writeString(s, ostr); @@ -89,13 +89,13 @@ void SerializationString::deserializeBinary(IColumn & column, ReadBuffer & istr, UInt64 size; readVarUInt(size, istr); - if (settings.max_binary_string_size && size > settings.max_binary_string_size) + if (settings.binary.max_binary_string_size && size > settings.binary.max_binary_string_size) throw Exception( ErrorCodes::TOO_LARGE_STRING_SIZE, "Too large string size: {}. The maximum is: {}. To increase the maximum, use setting " "format_binary_max_string_size", size, - settings.max_binary_string_size); + settings.binary.max_binary_string_size); size_t old_chars_size = data.size(); size_t offset = old_chars_size + size + 1; diff --git a/src/DataTypes/Serializations/SerializationVariantElement.cpp b/src/DataTypes/Serializations/SerializationVariantElement.cpp index 1f9a81ac671..ec0b4019c2f 100644 --- a/src/DataTypes/Serializations/SerializationVariantElement.cpp +++ b/src/DataTypes/Serializations/SerializationVariantElement.cpp @@ -146,7 +146,7 @@ void SerializationVariantElement::deserializeBinaryBulkWithMultipleStreams( } /// If we started to read a new column, reinitialize variant column in deserialization state. - if (!variant_element_state->variant || result_column->empty()) + if (!variant_element_state->variant || mutable_column->empty()) { variant_element_state->variant = mutable_column->cloneEmpty(); diff --git a/src/DataTypes/fuzzers/data_type_deserialization_fuzzer.cpp b/src/DataTypes/fuzzers/data_type_deserialization_fuzzer.cpp index 0ae325871fb..033a6ea8a4a 100644 --- a/src/DataTypes/fuzzers/data_type_deserialization_fuzzer.cpp +++ b/src/DataTypes/fuzzers/data_type_deserialization_fuzzer.cpp @@ -72,8 +72,8 @@ extern "C" int LLVMFuzzerTestOneInput(const uint8_t * data, size_t size) DataTypePtr type = DataTypeFactory::instance().get(data_type); FormatSettings settings; - settings.max_binary_string_size = 100; - settings.max_binary_array_size = 100; + settings.binary.max_binary_string_size = 100; + settings.binary.max_binary_string_size = 100; Field field; type->getDefaultSerialization()->deserializeBinary(field, in, settings); diff --git a/src/DataTypes/tests/gtest_data_types_binary_encoding.cpp b/src/DataTypes/tests/gtest_data_types_binary_encoding.cpp new file mode 100644 index 00000000000..4459e2558b6 --- /dev/null +++ b/src/DataTypes/tests/gtest_data_types_binary_encoding.cpp @@ -0,0 +1,123 @@ +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include + +using namespace DB; + +namespace DB::ErrorCodes +{ +extern const int UNSUPPORTED_METHOD; +} + + +void check(const DataTypePtr & type) +{ +// std::cerr << "Check " << type->getName() << "\n"; + WriteBufferFromOwnString ostr; + encodeDataType(type, ostr); + ReadBufferFromString istr(ostr.str()); + DataTypePtr decoded_type = decodeDataType(istr); + ASSERT_TRUE(istr.eof()); + ASSERT_EQ(type->getName(), decoded_type->getName()); + ASSERT_TRUE(type->equals(*decoded_type)); +} + +GTEST_TEST(DataTypesBinaryEncoding, EncodeAndDecode) +{ + registerAggregateFunctions(); + check(std::make_shared()); + check(std::make_shared()); + check(std::make_shared()); + check(std::make_shared()); + check(std::make_shared()); + check(std::make_shared()); + check(std::make_shared()); + check(std::make_shared()); + check(std::make_shared()); + check(std::make_shared()); + check(std::make_shared()); + check(std::make_shared()); + check(std::make_shared()); + check(std::make_shared()); + check(std::make_shared()); + check(std::make_shared()); + check(std::make_shared()); + check(std::make_shared()); + check(std::make_shared(3)); + check(std::make_shared()); + check(std::make_shared(10)); + check(DataTypeFactory::instance().get("Enum8('a' = 1, 'b' = 2, 'c' = 3, 'd' = -128)")); + check(DataTypeFactory::instance().get("Enum16('a' = 1, 'b' = 2, 'c' = 3, 'd' = -1000)")); + check(std::make_shared(3, 6)); + check(std::make_shared(3, 6)); + check(std::make_shared(3, 6)); + check(std::make_shared(3, 6)); + check(std::make_shared()); + check(DataTypeFactory::instance().get("Array(UInt32)")); + check(DataTypeFactory::instance().get("Array(Array(Array(UInt32)))")); + check(DataTypeFactory::instance().get("Tuple(UInt32, String, UUID)")); + check(DataTypeFactory::instance().get("Tuple(UInt32, String, Tuple(UUID, Date, IPv4))")); + check(DataTypeFactory::instance().get("Tuple(c1 UInt32, c2 String, c3 UUID)")); + check(DataTypeFactory::instance().get("Tuple(c1 UInt32, c2 String, c3 Tuple(c4 UUID, c5 Date, c6 IPv4))")); + check(std::make_shared()); + check(std::make_shared(IntervalKind::Kind::Nanosecond)); + check(std::make_shared(IntervalKind::Kind::Microsecond)); + check(DataTypeFactory::instance().get("Nullable(UInt32)")); + check(DataTypeFactory::instance().get("Nullable(Nothing)")); + check(DataTypeFactory::instance().get("Nullable(UUID)")); + check(std::make_shared( + DataTypes{ + std::make_shared(), + std::make_shared(), + DataTypeFactory::instance().get("Array(Array(Array(UInt32)))")}, + DataTypeFactory::instance().get("Tuple(c1 UInt32, c2 String, c3 UUID)"))); + DataTypes argument_types = {std::make_shared()}; + Array parameters = {Field(0.1), Field(0.2)}; + AggregateFunctionProperties properties; + AggregateFunctionPtr function = AggregateFunctionFactory::instance().get("quantiles", NullsAction::EMPTY, argument_types, parameters, properties); + check(std::make_shared(function, argument_types, parameters)); + check(std::make_shared(function, argument_types, parameters, 2)); + check(DataTypeFactory::instance().get("AggregateFunction(sum, UInt64)")); + check(DataTypeFactory::instance().get("AggregateFunction(quantiles(0.5, 0.9), UInt64)")); + check(DataTypeFactory::instance().get("AggregateFunction(sequenceMatch('(?1)(?2)'), Date, UInt8, UInt8)")); + check(DataTypeFactory::instance().get("AggregateFunction(sumMapFiltered([1, 4, 8]), Array(UInt64), Array(UInt64))")); + check(DataTypeFactory::instance().get("LowCardinality(UInt32)")); + check(DataTypeFactory::instance().get("LowCardinality(Nullable(String))")); + check(DataTypeFactory::instance().get("Map(String, UInt32)")); + check(DataTypeFactory::instance().get("Map(String, Map(String, Map(String, UInt32)))")); + check(std::make_shared()); + check(std::make_shared()); + check(DataTypeFactory::instance().get("Variant(String, UInt32, Date32)")); + check(std::make_shared()); + check(DataTypeFactory::instance().get("Bool")); + check(DataTypeFactory::instance().get("SimpleAggregateFunction(sum, UInt64)")); + check(DataTypeFactory::instance().get("SimpleAggregateFunction(maxMap, Tuple(Array(UInt32), Array(UInt32)))")); + check(DataTypeFactory::instance().get("SimpleAggregateFunction(groupArrayArray(19), Array(UInt64))")); + check(DataTypeFactory::instance().get("Nested(a UInt32, b UInt32)")); + check(DataTypeFactory::instance().get("Nested(a UInt32, b Nested(c String, d Nested(e Date)))")); + check(DataTypeFactory::instance().get("Ring")); + check(DataTypeFactory::instance().get("Point")); + check(DataTypeFactory::instance().get("Polygon")); + check(DataTypeFactory::instance().get("MultiPolygon")); + check(DataTypeFactory::instance().get("Tuple(Map(LowCardinality(String), Array(AggregateFunction(2, quantiles(0.1, 0.2), Float32))), Array(Array(Tuple(UInt32, Tuple(a Map(String, String), b Nullable(Date), c Variant(Tuple(g String, d Array(UInt32)), Date, Map(String, String)))))))")); +} diff --git a/src/Formats/FormatFactory.cpp b/src/Formats/FormatFactory.cpp index e9a405aa796..a51eb975180 100644 --- a/src/Formats/FormatFactory.cpp +++ b/src/Formats/FormatFactory.cpp @@ -265,9 +265,13 @@ FormatSettings getFormatSettings(const ContextPtr & context, const Settings & se format_settings.markdown.escape_special_characters = settings.output_format_markdown_escape_special_characters; format_settings.bson.output_string_as_string = settings.output_format_bson_string_as_string; format_settings.bson.skip_fields_with_unsupported_types_in_schema_inference = settings.input_format_bson_skip_fields_with_unsupported_types_in_schema_inference; - format_settings.max_binary_string_size = settings.format_binary_max_string_size; - format_settings.max_binary_array_size = settings.format_binary_max_array_size; + format_settings.binary.max_binary_string_size = settings.format_binary_max_string_size; + format_settings.binary.max_binary_array_size = settings.format_binary_max_array_size; + format_settings.binary.encode_types_in_binary_format = settings.output_format_binary_encode_types_in_binary_format; + format_settings.binary.decode_types_in_binary_format = settings.input_format_binary_decode_types_in_binary_format; format_settings.native.allow_types_conversion = settings.input_format_native_allow_types_conversion; + format_settings.native.encode_types_in_binary_format = settings.output_format_native_encode_types_in_binary_format; + format_settings.native.decode_types_in_binary_format = settings.input_format_native_decode_types_in_binary_format; format_settings.max_parser_depth = context->getSettingsRef().max_parser_depth; format_settings.client_protocol_version = context->getClientProtocolVersion(); format_settings.date_time_overflow_behavior = settings.date_time_overflow_behavior; diff --git a/src/Formats/FormatSettings.h b/src/Formats/FormatSettings.h index 421ed4d112d..69a0c64b3b0 100644 --- a/src/Formats/FormatSettings.h +++ b/src/Formats/FormatSettings.h @@ -106,8 +106,6 @@ struct FormatSettings UInt64 input_allow_errors_num = 0; Float32 input_allow_errors_ratio = 0; - UInt64 max_binary_string_size = 1_GiB; - UInt64 max_binary_array_size = 1_GiB; UInt64 client_protocol_version = 0; UInt64 max_parser_depth = DBMS_DEFAULT_MAX_PARSER_DEPTH; @@ -121,6 +119,14 @@ struct FormatSettings ZSTD }; + struct + { + UInt64 max_binary_string_size = 1_GiB; + UInt64 max_binary_array_size = 1_GiB; + bool encode_types_in_binary_format = false; + bool decode_types_in_binary_format = false; + } binary{}; + struct { UInt64 row_group_size = 1000000; @@ -454,6 +460,8 @@ struct FormatSettings struct { bool allow_types_conversion = true; + bool encode_types_in_binary_format = false; + bool decode_types_in_binary_format = false; } native{}; struct diff --git a/src/Formats/NativeReader.cpp b/src/Formats/NativeReader.cpp index fa5d41d6536..45be0402dc4 100644 --- a/src/Formats/NativeReader.cpp +++ b/src/Formats/NativeReader.cpp @@ -6,6 +6,7 @@ #include #include +#include #include #include @@ -31,8 +32,8 @@ namespace ErrorCodes } -NativeReader::NativeReader(ReadBuffer & istr_, UInt64 server_revision_) - : istr(istr_), server_revision(server_revision_) +NativeReader::NativeReader(ReadBuffer & istr_, UInt64 server_revision_, std::optional format_settings_) + : istr(istr_), server_revision(server_revision_), format_settings(format_settings_) { } @@ -40,16 +41,12 @@ NativeReader::NativeReader( ReadBuffer & istr_, const Block & header_, UInt64 server_revision_, - bool skip_unknown_columns_, - bool null_as_default_, - bool allow_types_conversion_, + std::optionalformat_settings_, BlockMissingValues * block_missing_values_) : istr(istr_) , header(header_) , server_revision(server_revision_) - , skip_unknown_columns(skip_unknown_columns_) - , null_as_default(null_as_default_) - , allow_types_conversion(allow_types_conversion_) + , format_settings(std::move(format_settings_)) , block_missing_values(block_missing_values_) { } @@ -83,13 +80,14 @@ void NativeReader::resetParser() use_index = false; } -void NativeReader::readData(const ISerialization & serialization, ColumnPtr & column, ReadBuffer & istr, size_t rows, double avg_value_size_hint) +static void readData(const ISerialization & serialization, ColumnPtr & column, ReadBuffer & istr, const std::optional & format_settings, size_t rows, double avg_value_size_hint) { ISerialization::DeserializeBinaryBulkSettings settings; settings.getter = [&](ISerialization::SubstreamPath) -> ReadBuffer * { return &istr; }; settings.avg_value_size_hint = avg_value_size_hint; settings.position_independent_encoding = false; settings.native_format = true; + settings.data_types_binary_encoding = format_settings && format_settings->native.decode_types_in_binary_format; ISerialization::DeserializeBinaryBulkStatePtr state; @@ -167,8 +165,16 @@ Block NativeReader::read() /// Type String type_name; - readBinary(type_name, istr); - column.type = data_type_factory.get(type_name); + if (format_settings && format_settings->native.decode_types_in_binary_format) + { + column.type = decodeDataType(istr); + type_name = column.type->getName(); + } + else + { + readBinary(type_name, istr); + column.type = data_type_factory.get(type_name); + } setVersionToAggregateFunctions(column.type, true, server_revision); @@ -203,7 +209,7 @@ Block NativeReader::read() double avg_value_size_hint = avg_value_size_hints.empty() ? 0 : avg_value_size_hints[i]; if (rows) /// If no rows, nothing to read. - readData(*serialization, read_column, istr, rows, avg_value_size_hint); + readData(*serialization, read_column, istr, format_settings, rows, avg_value_size_hint); column.column = std::move(read_column); @@ -214,12 +220,12 @@ Block NativeReader::read() { auto & header_column = header.getByName(column.name); - if (null_as_default) + if (format_settings && format_settings->null_as_default) insertNullAsDefaultIfNeeded(column, header_column, header.getPositionByName(column.name), block_missing_values); if (!header_column.type->equals(*column.type)) { - if (allow_types_conversion) + if (format_settings && format_settings->native.allow_types_conversion) { try { @@ -246,7 +252,7 @@ Block NativeReader::read() } else { - if (!skip_unknown_columns) + if (format_settings && !format_settings->skip_unknown_fields) throw Exception(ErrorCodes::INCORRECT_DATA, "Unknown column with name {} found while reading data in Native format", column.name); use_in_result = false; } diff --git a/src/Formats/NativeReader.h b/src/Formats/NativeReader.h index 3cec4afd997..97b6ea22b15 100644 --- a/src/Formats/NativeReader.h +++ b/src/Formats/NativeReader.h @@ -20,7 +20,7 @@ class NativeReader { public: /// If a non-zero server_revision is specified, additional block information may be expected and read. - NativeReader(ReadBuffer & istr_, UInt64 server_revision_); + NativeReader(ReadBuffer & istr_, UInt64 server_revision_, std::optional format_settings_ = std::nullopt); /// For cases when data structure (header) is known in advance. /// NOTE We may use header for data validation and/or type conversions. It is not implemented. @@ -28,9 +28,7 @@ public: ReadBuffer & istr_, const Block & header_, UInt64 server_revision_, - bool skip_unknown_columns_ = false, - bool null_as_default_ = false, - bool allow_types_conversion_ = false, + std::optional format_settings_ = std::nullopt, BlockMissingValues * block_missing_values_ = nullptr); /// For cases when we have an index. It allows to skip columns. Only columns specified in the index will be read. @@ -38,8 +36,6 @@ public: IndexForNativeFormat::Blocks::const_iterator index_block_it_, IndexForNativeFormat::Blocks::const_iterator index_block_end_); - static void readData(const ISerialization & serialization, ColumnPtr & column, ReadBuffer & istr, size_t rows, double avg_value_size_hint); - Block getHeader() const; void resetParser(); @@ -50,9 +46,7 @@ private: ReadBuffer & istr; Block header; UInt64 server_revision; - bool skip_unknown_columns = false; - bool null_as_default = false; - bool allow_types_conversion = false; + std::optional format_settings = std::nullopt; BlockMissingValues * block_missing_values = nullptr; bool use_index = false; diff --git a/src/Formats/NativeWriter.cpp b/src/Formats/NativeWriter.cpp index b150561a5fc..3c87e489b1c 100644 --- a/src/Formats/NativeWriter.cpp +++ b/src/Formats/NativeWriter.cpp @@ -14,6 +14,7 @@ #include #include #include +#include namespace DB { @@ -25,10 +26,20 @@ namespace ErrorCodes NativeWriter::NativeWriter( - WriteBuffer & ostr_, UInt64 client_revision_, const Block & header_, bool remove_low_cardinality_, - IndexForNativeFormat * index_, size_t initial_size_of_file_) - : ostr(ostr_), client_revision(client_revision_), header(header_), - index(index_), initial_size_of_file(initial_size_of_file_), remove_low_cardinality(remove_low_cardinality_) + WriteBuffer & ostr_, + UInt64 client_revision_, + const Block & header_, + std::optional format_settings_, + bool remove_low_cardinality_, + IndexForNativeFormat * index_, + size_t initial_size_of_file_) + : ostr(ostr_) + , client_revision(client_revision_) + , header(header_) + , index(index_) + , initial_size_of_file(initial_size_of_file_) + , remove_low_cardinality(remove_low_cardinality_) + , format_settings(std::move(format_settings_)) { if (index) { @@ -45,7 +56,7 @@ void NativeWriter::flush() } -static void writeData(const ISerialization & serialization, const ColumnPtr & column, WriteBuffer & ostr, UInt64 offset, UInt64 limit) +static void writeData(const ISerialization & serialization, const ColumnPtr & column, WriteBuffer & ostr, const std::optional & format_settings, UInt64 offset, UInt64 limit) { /** If there are columns-constants - then we materialize them. * (Since the data type does not know how to serialize / deserialize constants.) @@ -57,6 +68,7 @@ static void writeData(const ISerialization & serialization, const ColumnPtr & co settings.getter = [&ostr](ISerialization::SubstreamPath) -> WriteBuffer * { return &ostr; }; settings.position_independent_encoding = false; settings.low_cardinality_max_dictionary_size = 0; + settings.data_types_binary_encoding = format_settings && format_settings->native.encode_types_in_binary_format; ISerialization::SerializeBinaryBulkStatePtr state; serialization.serializeBinaryBulkStatePrefix(*full_column, settings, state); @@ -121,15 +133,22 @@ size_t NativeWriter::write(const Block & block) setVersionToAggregateFunctions(column.type, include_version, include_version ? std::optional(client_revision) : std::nullopt); /// Type - String type_name = column.type->getName(); + if (format_settings && format_settings->native.encode_types_in_binary_format) + { + encodeDataType(column.type, ostr); + } + else + { + String type_name = column.type->getName(); - /// For compatibility, we will not send explicit timezone parameter in DateTime data type - /// to older clients, that cannot understand it. - if (client_revision < DBMS_MIN_REVISION_WITH_TIME_ZONE_PARAMETER_IN_DATETIME_DATA_TYPE - && startsWith(type_name, "DateTime(")) - type_name = "DateTime"; + /// For compatibility, we will not send explicit timezone parameter in DateTime data type + /// to older clients, that cannot understand it. + if (client_revision < DBMS_MIN_REVISION_WITH_TIME_ZONE_PARAMETER_IN_DATETIME_DATA_TYPE + && startsWith(type_name, "DateTime(")) + type_name = "DateTime"; - writeStringBinary(type_name, ostr); + writeStringBinary(type_name, ostr); + } /// Serialization. Dynamic, if client supports it. SerializationPtr serialization; @@ -161,7 +180,7 @@ size_t NativeWriter::write(const Block & block) /// Data if (rows) /// Zero items of data is always represented as zero number of bytes. - writeData(*serialization, column.column, ostr, 0, 0); + writeData(*serialization, column.column, ostr, format_settings, 0, 0); if (index) { diff --git a/src/Formats/NativeWriter.h b/src/Formats/NativeWriter.h index 7bb377d2e4a..b4903243d45 100644 --- a/src/Formats/NativeWriter.h +++ b/src/Formats/NativeWriter.h @@ -3,6 +3,7 @@ #include #include #include +#include namespace DB { @@ -23,7 +24,7 @@ public: /** If non-zero client_revision is specified, additional block information can be written. */ NativeWriter( - WriteBuffer & ostr_, UInt64 client_revision_, const Block & header_, bool remove_low_cardinality_ = false, + WriteBuffer & ostr_, UInt64 client_revision_, const Block & header_, std::optional format_settings_ = std::nullopt, bool remove_low_cardinality_ = false, IndexForNativeFormat * index_ = nullptr, size_t initial_size_of_file_ = 0); Block getHeader() const { return header; } @@ -44,6 +45,7 @@ private: CompressedWriteBuffer * ostr_concrete = nullptr; bool remove_low_cardinality; + std::optional format_settings; }; } diff --git a/src/Formats/SchemaInferenceUtils.cpp b/src/Formats/SchemaInferenceUtils.cpp index 31faea2e13e..6cbcae2bebe 100644 --- a/src/Formats/SchemaInferenceUtils.cpp +++ b/src/Formats/SchemaInferenceUtils.cpp @@ -879,11 +879,11 @@ namespace } template - bool tryReadFloat(Float64 & value, ReadBuffer & buf, const FormatSettings & settings, bool & has_fractional) + bool tryReadFloat(Float64 & value, ReadBuffer & buf, const FormatSettings & settings) { if (is_json || settings.try_infer_exponent_floats) - return tryReadFloatTextExt(value, buf, has_fractional); - return tryReadFloatTextExtNoExponent(value, buf, has_fractional); + return tryReadFloatText(value, buf); + return tryReadFloatTextNoExponent(value, buf); } template @@ -893,31 +893,46 @@ namespace return nullptr; Float64 tmp_float; - bool has_fractional; if (settings.try_infer_integers) { /// If we read from String, we can do it in a more efficient way. if (auto * string_buf = dynamic_cast(&buf)) { /// Remember the pointer to the start of the number to rollback to it. - /// We can safely get back to the start of the number, because we read from a string and we didn't reach eof. char * number_start = buf.position(); - - /// NOTE: it may break parsing of tryReadFloat() != tryReadIntText() + parsing of '.'/'e' - /// But, for now it is true - if (tryReadFloat(tmp_float, buf, settings, has_fractional) && has_fractional) - return std::make_shared(); - Int64 tmp_int; - buf.position() = number_start; - if (tryReadIntText(tmp_int, buf)) - return std::make_shared(); + bool read_int = tryReadIntText(tmp_int, buf); + /// If we reached eof, it cannot be float (it requires no less data than integer) + if (buf.eof()) + return read_int ? std::make_shared() : nullptr; - /// In case of Int64 overflow we can try to infer UInt64. - UInt64 tmp_uint; + char * int_end = buf.position(); + /// We can safely get back to the start of the number, because we read from a string and we didn't reach eof. buf.position() = number_start; - if (tryReadIntText(tmp_uint, buf)) - return std::make_shared(); + + bool read_uint = false; + char * uint_end = nullptr; + /// In case of Int64 overflow we can try to infer UInt64. + if (!read_int) + { + UInt64 tmp_uint; + read_uint = tryReadIntText(tmp_uint, buf); + /// If we reached eof, it cannot be float (it requires no less data than integer) + if (buf.eof()) + return read_uint ? std::make_shared() : nullptr; + + uint_end = buf.position(); + buf.position() = number_start; + } + + if (tryReadFloat(tmp_float, buf, settings)) + { + if (read_int && buf.position() == int_end) + return std::make_shared(); + if (read_uint && buf.position() == uint_end) + return std::make_shared(); + return std::make_shared(); + } return nullptr; } @@ -927,22 +942,36 @@ namespace /// and then as float. PeekableReadBuffer peekable_buf(buf); PeekableReadBufferCheckpoint checkpoint(peekable_buf); - - if (tryReadFloat(tmp_float, peekable_buf, settings, has_fractional) && has_fractional) - return std::make_shared(); - peekable_buf.rollbackToCheckpoint(/* drop= */ false); - Int64 tmp_int; - if (tryReadIntText(tmp_int, peekable_buf)) - return std::make_shared(); - peekable_buf.rollbackToCheckpoint(/* drop= */ true); + bool read_int = tryReadIntText(tmp_int, peekable_buf); + auto * int_end = peekable_buf.position(); + peekable_buf.rollbackToCheckpoint(true); + bool read_uint = false; + char * uint_end = nullptr; /// In case of Int64 overflow we can try to infer UInt64. - UInt64 tmp_uint; - if (tryReadIntText(tmp_uint, peekable_buf)) - return std::make_shared(); + if (!read_int) + { + PeekableReadBufferCheckpoint new_checkpoint(peekable_buf); + UInt64 tmp_uint; + read_uint = tryReadIntText(tmp_uint, peekable_buf); + uint_end = peekable_buf.position(); + peekable_buf.rollbackToCheckpoint(true); + } + + if (tryReadFloat(tmp_float, peekable_buf, settings)) + { + /// Float parsing reads no fewer bytes than integer parsing, + /// so position of the buffer is either the same, or further. + /// If it's the same, then it's integer. + if (read_int && peekable_buf.position() == int_end) + return std::make_shared(); + if (read_uint && peekable_buf.position() == uint_end) + return std::make_shared(); + return std::make_shared(); + } } - else if (tryReadFloat(tmp_float, buf, settings, has_fractional)) + else if (tryReadFloat(tmp_float, buf, settings)) { return std::make_shared(); } @@ -975,8 +1004,7 @@ namespace buf.position() = buf.buffer().begin(); Float64 tmp; - bool has_fractional; - if (tryReadFloat(tmp, buf, settings, has_fractional) && buf.eof()) + if (tryReadFloat(tmp, buf, settings) && buf.eof()) return std::make_shared(); return nullptr; diff --git a/src/Interpreters/Context.cpp b/src/Interpreters/Context.cpp index f4433cd8288..db5c5a37125 100644 --- a/src/Interpreters/Context.cpp +++ b/src/Interpreters/Context.cpp @@ -18,7 +18,6 @@ #include #include #include -#include #include #include #include @@ -611,8 +610,6 @@ struct ContextSharedPart : boost::noncopyable LOG_TRACE(log, "Shutting down database catalog"); DatabaseCatalog::shutdown(); - NamedCollectionFactory::instance().shutdown(); - delete_async_insert_queue.reset(); SHUTDOWN(log, "merges executor", merge_mutate_executor, wait()); diff --git a/src/Processors/Formats/Impl/BinaryRowInputFormat.cpp b/src/Processors/Formats/Impl/BinaryRowInputFormat.cpp index ac5da172210..c5336f3bcc7 100644 --- a/src/Processors/Formats/Impl/BinaryRowInputFormat.cpp +++ b/src/Processors/Formats/Impl/BinaryRowInputFormat.cpp @@ -4,6 +4,7 @@ #include #include #include +#include namespace DB { @@ -55,10 +56,25 @@ std::vector BinaryFormatReader::readNames() template std::vector BinaryFormatReader::readTypes() { - auto types = readHeaderRow(); - for (const auto & type_name : types) - read_data_types.push_back(DataTypeFactory::instance().get(type_name)); - return types; + read_data_types.reserve(read_columns); + Names type_names; + if (format_settings.binary.decode_types_in_binary_format) + { + type_names.reserve(read_columns); + for (size_t i = 0; i < read_columns; ++i) + { + read_data_types.push_back(decodeDataType(*in)); + type_names.push_back(read_data_types.back()->getName()); + } + } + else + { + type_names = readHeaderRow(); + for (const auto & type_name : type_names) + read_data_types.push_back(DataTypeFactory::instance().get(type_name)); + } + + return type_names; } template diff --git a/src/Processors/Formats/Impl/BinaryRowOutputFormat.cpp b/src/Processors/Formats/Impl/BinaryRowOutputFormat.cpp index ff904f61d22..d4c2348d080 100644 --- a/src/Processors/Formats/Impl/BinaryRowOutputFormat.cpp +++ b/src/Processors/Formats/Impl/BinaryRowOutputFormat.cpp @@ -2,6 +2,7 @@ #include #include #include +#include #include #include #include @@ -35,9 +36,15 @@ void BinaryRowOutputFormat::writePrefix() if (with_types) { - for (size_t i = 0; i < columns; ++i) + if (format_settings.binary.encode_types_in_binary_format) { - writeStringBinary(header.safeGetByPosition(i).type->getName(), out); + for (size_t i = 0; i < columns; ++i) + encodeDataType(header.safeGetByPosition(i).type, out); + } + else + { + for (size_t i = 0; i < columns; ++i) + writeStringBinary(header.safeGetByPosition(i).type->getName(), out); } } } diff --git a/src/Processors/Formats/Impl/NativeFormat.cpp b/src/Processors/Formats/Impl/NativeFormat.cpp index a7a49ab6a8c..38fac60eef6 100644 --- a/src/Processors/Formats/Impl/NativeFormat.cpp +++ b/src/Processors/Formats/Impl/NativeFormat.cpp @@ -21,9 +21,7 @@ public: buf, header_, 0, - settings.skip_unknown_fields, - settings.null_as_default, - settings.native.allow_types_conversion, + settings, settings.defaults_for_omitted_fields ? &block_missing_values : nullptr)) , header(header_) {} @@ -72,9 +70,9 @@ private: class NativeOutputFormat final : public IOutputFormat { public: - NativeOutputFormat(WriteBuffer & buf, const Block & header, UInt64 client_protocol_version = 0) + NativeOutputFormat(WriteBuffer & buf, const Block & header, const FormatSettings & settings, UInt64 client_protocol_version = 0) : IOutputFormat(header, buf) - , writer(buf, client_protocol_version, header) + , writer(buf, client_protocol_version, header, settings) { } @@ -103,14 +101,17 @@ private: class NativeSchemaReader : public ISchemaReader { public: - explicit NativeSchemaReader(ReadBuffer & in_) : ISchemaReader(in_) {} + explicit NativeSchemaReader(ReadBuffer & in_, const FormatSettings & settings_) : ISchemaReader(in_), settings(settings_) {} NamesAndTypesList readSchema() override { - auto reader = NativeReader(in, 0); + auto reader = NativeReader(in, 0, settings); auto block = reader.read(); return block.getNamesAndTypesList(); } + +private: + const FormatSettings settings; }; @@ -134,16 +135,16 @@ void registerOutputFormatNative(FormatFactory & factory) const Block & sample, const FormatSettings & settings) { - return std::make_shared(buf, sample, settings.client_protocol_version); + return std::make_shared(buf, sample, settings, settings.client_protocol_version); }); } void registerNativeSchemaReader(FormatFactory & factory) { - factory.registerSchemaReader("Native", [](ReadBuffer & buf, const FormatSettings &) + factory.registerSchemaReader("Native", [](ReadBuffer & buf, const FormatSettings & settings) { - return std::make_shared(buf); + return std::make_shared(buf, settings); }); } diff --git a/src/Server/TCPHandler.cpp b/src/Server/TCPHandler.cpp index e3a820340ad..c8f86b1c2c9 100644 --- a/src/Server/TCPHandler.cpp +++ b/src/Server/TCPHandler.cpp @@ -2086,6 +2086,7 @@ void TCPHandler::initBlockOutput(const Block & block) *state.maybe_compressed_out, client_tcp_protocol_version, block.cloneEmpty(), + std::nullopt, !query_settings.low_cardinality_allow_in_native_format); } } @@ -2100,6 +2101,7 @@ void TCPHandler::initLogsBlockOutput(const Block & block) *out, client_tcp_protocol_version, block.cloneEmpty(), + std::nullopt, !query_settings.low_cardinality_allow_in_native_format); } } @@ -2114,6 +2116,7 @@ void TCPHandler::initProfileEventsBlockOutput(const Block & block) *out, client_tcp_protocol_version, block.cloneEmpty(), + std::nullopt, !query_settings.low_cardinality_allow_in_native_format); } } diff --git a/src/Storages/StorageMemory.cpp b/src/Storages/StorageMemory.cpp index f69c4adb552..57da72d06ed 100644 --- a/src/Storages/StorageMemory.cpp +++ b/src/Storages/StorageMemory.cpp @@ -408,7 +408,7 @@ namespace auto data_file_path = temp_dir / fs::path{file_paths[data_bin_pos]}.filename(); auto data_out_compressed = temp_disk->writeFile(data_file_path); auto data_out = std::make_unique(*data_out_compressed, CompressionCodecFactory::instance().getDefaultCodec(), max_compress_block_size); - NativeWriter block_out{*data_out, 0, metadata_snapshot->getSampleBlock(), false, &index}; + NativeWriter block_out{*data_out, 0, metadata_snapshot->getSampleBlock(), std::nullopt, false, &index}; for (const auto & block : *blocks) block_out.write(block); data_out->finalize(); diff --git a/src/Storages/StorageStripeLog.cpp b/src/Storages/StorageStripeLog.cpp index f0c5103d657..e456665a615 100644 --- a/src/Storages/StorageStripeLog.cpp +++ b/src/Storages/StorageStripeLog.cpp @@ -193,7 +193,7 @@ public: storage.saveFileSizes(lock); size_t initial_data_size = storage.file_checker.getFileSize(storage.data_file_path); - block_out = std::make_unique(*data_out, 0, metadata_snapshot->getSampleBlock(), false, &storage.indices, initial_data_size); + block_out = std::make_unique(*data_out, 0, metadata_snapshot->getSampleBlock(), std::nullopt, false, &storage.indices, initial_data_size); } String getName() const override { return "StripeLogSink"; } diff --git a/tests/queries/0_stateless/03172_dynamic_binary_serialization.reference b/tests/queries/0_stateless/03172_dynamic_binary_serialization.reference new file mode 100644 index 00000000000..26bd3326d5c --- /dev/null +++ b/tests/queries/0_stateless/03172_dynamic_binary_serialization.reference @@ -0,0 +1,48 @@ +\N None +42 UInt8 +-42 Int8 +42 UInt16 +-42 Int16 +42 UInt32 +-42 Int32 +42 UInt64 +-42 Int64 +42 UInt128 +-42 Int128 +42 UInt256 +-42 Int256 +42.42 Float32 +42.42 Float64 +2020-01-01 Date +2020-01-01 Date32 +2020-01-01 00:00:00 DateTime +2020-01-01 00:00:00.000000 DateTime64(6) +Hello, World! String +aaaaa FixedString(5) +a Enum8(\'c\' = -128, \'a\' = 1, \'b\' = 2) +a Enum16(\'c\' = -1280, \'a\' = 1, \'b\' = 2) +42.42 Decimal(9, 3) +42.42 Decimal(18, 3) +42.42 Decimal(38, 3) +42.42 Decimal(76, 3) +984ac60f-4d08-4ef1-9c62-d82f343fbc90 UUID +[1,2,3] Array(UInt64) +[[[1],[2]],[[3,4,5]]] Array(Array(Array(UInt64))) +(1,'str',42.42) Tuple(UInt32, String, Float32) +(1,'str',42.42) Tuple(a UInt32, b String, c Float32) +(1,('str',(42.42,-30))) Tuple(UInt32, Tuple(String, Tuple(Float32, Int8))) +(1,('str',(42.42,-30))) Tuple(a UInt32, b Tuple(c String, d Tuple(e Float32, f Int8))) +\0 \0\0\0\0\0\0\0\0\0\0\0\0\06364136223846793005 0 123459*\0\0\0\0\0\0\0 AggregateFunction(quantile(0.5), UInt64) +42 SimpleAggregateFunction(sum, UInt64) +Hello, World! LowCardinality(String) +{1:'str1',2:'str2'} Map(UInt64, String) +{1:{1:{1:'str1'}},2:{2:{2:'str2'}}} Map(UInt64, Map(UInt64, Map(UInt64, String))) +127.0.0.0 IPv4 +2001:db8:cafe:1::1 IPv6 +true Bool +[(1,2),(3,4)] Nested(a UInt32, b UInt32) +[(0,0),(10,0),(10,10),(0,10)] Ring +(0,0) Point +[[(20,20),(50,20),(50,50),(20,50)],[(30,30),(50,50),(50,30)]] Polygon +[[[(0,0),(10,0),(10,10),(0,10)]],[[(20,20),(50,20),(50,50),(20,50)],[(30,30),(50,50),(50,30)]]] MultiPolygon +[{42:(1,[(2,{1:2})])}] Array(Map(UInt8, Tuple(UInt8, Array(Tuple(UInt8, Map(UInt8, UInt8)))))) diff --git a/tests/queries/0_stateless/03172_dynamic_binary_serialization.sh b/tests/queries/0_stateless/03172_dynamic_binary_serialization.sh new file mode 100755 index 00000000000..b078211f088 --- /dev/null +++ b/tests/queries/0_stateless/03172_dynamic_binary_serialization.sh @@ -0,0 +1,61 @@ +#!/usr/bin/env bash + +CURDIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd) +# shellcheck source=../shell_config.sh +. "$CURDIR"/../shell_config.sh + +$CLICKHOUSE_CLIENT -q "drop table if exists test" +$CLICKHOUSE_CLIENT --allow_experimental_dynamic_type=1 -q "create table test (id UInt64, d Dynamic(max_types=255)) engine=Memory" + +$CLICKHOUSE_CLIENT -q "insert into test select 0, NULL" +$CLICKHOUSE_CLIENT -q "insert into test select 1, materialize(42)::UInt8" +$CLICKHOUSE_CLIENT -q "insert into test select 2, materialize(-42)::Int8" +$CLICKHOUSE_CLIENT -q "insert into test select 3, materialize(42)::UInt16" +$CLICKHOUSE_CLIENT -q "insert into test select 4, materialize(-42)::Int16" +$CLICKHOUSE_CLIENT -q "insert into test select 5, materialize(42)::UInt32" +$CLICKHOUSE_CLIENT -q "insert into test select 6, materialize(-42)::Int32" +$CLICKHOUSE_CLIENT -q "insert into test select 7, materialize(42)::UInt64" +$CLICKHOUSE_CLIENT -q "insert into test select 8, materialize(-42)::Int64" +$CLICKHOUSE_CLIENT -q "insert into test select 9, materialize(42)::UInt128" +$CLICKHOUSE_CLIENT -q "insert into test select 10, materialize(-42)::Int128" +$CLICKHOUSE_CLIENT -q "insert into test select 11, materialize(42)::UInt256" +$CLICKHOUSE_CLIENT -q "insert into test select 12, materialize(-42)::Int256" +$CLICKHOUSE_CLIENT -q "insert into test select 13, materialize(42.42)::Float32" +$CLICKHOUSE_CLIENT -q "insert into test select 14, materialize(42.42)::Float64" +$CLICKHOUSE_CLIENT -q "insert into test select 15, materialize('2020-01-01')::Date" +$CLICKHOUSE_CLIENT -q "insert into test select 16, materialize('2020-01-01')::Date32" +$CLICKHOUSE_CLIENT -q "insert into test select 17, materialize('2020-01-01 00:00:00')::DateTime" +$CLICKHOUSE_CLIENT -q "insert into test select 18, materialize('2020-01-01 00:00:00.000000')::DateTime64(6)" +$CLICKHOUSE_CLIENT -q "insert into test select 19, materialize('Hello, World!')" +$CLICKHOUSE_CLIENT -q "insert into test select 20, materialize('aaaaa')::FixedString(5)" +$CLICKHOUSE_CLIENT -q "insert into test select 21, materialize('a')::Enum8('a' = 1, 'b' = 2, 'c' = -128)" +$CLICKHOUSE_CLIENT -q "insert into test select 22, materialize('a')::Enum16('a' = 1, 'b' = 2, 'c' = -1280)" +$CLICKHOUSE_CLIENT -q "insert into test select 23, materialize(42.42)::Decimal32(3)" +$CLICKHOUSE_CLIENT -q "insert into test select 24, materialize(42.42)::Decimal64(3)" +$CLICKHOUSE_CLIENT -q "insert into test select 25, materialize(42.42)::Decimal128(3)" +$CLICKHOUSE_CLIENT -q "insert into test select 26, materialize(42.42)::Decimal256(3)" +$CLICKHOUSE_CLIENT -q "insert into test select 27, materialize('984ac60f-4d08-4ef1-9c62-d82f343fbc90')::UUID" +$CLICKHOUSE_CLIENT -q "insert into test select 28, materialize([1, 2, 3])::Array(UInt64)" +$CLICKHOUSE_CLIENT -q "insert into test select 29, materialize([[[1], [2]], [[3, 4, 5]]])::Array(Array(Array(UInt64)))" +$CLICKHOUSE_CLIENT -q "insert into test select 30, materialize(tuple(1, 'str', 42.42))::Tuple(UInt32, String, Float32)" +$CLICKHOUSE_CLIENT -q "insert into test select 31, materialize(tuple(1, 'str', 42.42))::Tuple(a UInt32, b String, c Float32)" +$CLICKHOUSE_CLIENT -q "insert into test select 32, materialize(tuple(1, tuple('str', tuple(42.42, -30))))::Tuple(UInt32, Tuple(String, Tuple(Float32, Int8)))" +$CLICKHOUSE_CLIENT -q "insert into test select 33, materialize(tuple(1, tuple('str', tuple(42.42, -30))))::Tuple(a UInt32, b Tuple(c String, d Tuple(e Float32, f Int8)))" +$CLICKHOUSE_CLIENT -q "insert into test select 34, quantileState(0.5)(42::UInt64)" +$CLICKHOUSE_CLIENT -q "insert into test select 35, sumSimpleState(42::UInt64)" +$CLICKHOUSE_CLIENT -q "insert into test select 36, toLowCardinality('Hello, World!')" +$CLICKHOUSE_CLIENT -q "insert into test select 37, materialize(map(1, 'str1', 2, 'str2'))::Map(UInt64, String)" +$CLICKHOUSE_CLIENT -q "insert into test select 38, materialize(map(1, map(1, map(1, 'str1')), 2, map(2, map(2, 'str2'))))::Map(UInt64, Map(UInt64, Map(UInt64, String)))" +$CLICKHOUSE_CLIENT -q "insert into test select 39, materialize('127.0.0.0')::IPv4" +$CLICKHOUSE_CLIENT -q "insert into test select 40, materialize('2001:db8:cafe:1:0:0:0:1')::IPv6" +$CLICKHOUSE_CLIENT -q "insert into test select 41, materialize(true)::Bool" +$CLICKHOUSE_CLIENT -q "insert into test select 42, materialize([tuple(1, 2), tuple(3, 4)])::Nested(a UInt32, b UInt32)" +$CLICKHOUSE_CLIENT -q "insert into test select 43, materialize([(0, 0), (10, 0), (10, 10), (0, 10)])::Ring" +$CLICKHOUSE_CLIENT -q "insert into test select 44, materialize((0, 0))::Point" +$CLICKHOUSE_CLIENT -q "insert into test select 45, materialize([[(20, 20), (50, 20), (50, 50), (20, 50)], [(30, 30), (50, 50), (50, 30)]])::Polygon" +$CLICKHOUSE_CLIENT -q "insert into test select 46, materialize([[[(0, 0), (10, 0), (10, 10), (0, 10)]], [[(20, 20), (50, 20), (50, 50), (20, 50)],[(30, 30), (50, 50), (50, 30)]]])::MultiPolygon" +$CLICKHOUSE_CLIENT -q "insert into test select 47, materialize([map(42, tuple(1, [tuple(2, map(1, 2))]))])" + +$CLICKHOUSE_CLIENT -q "select * from test format RowBinary" | $CLICKHOUSE_LOCAL --allow_experimental_dynamic_type=1 --input-format RowBinary --structure 'id UInt64, d Dynamic(max_types=255)' -q "select d, dynamicType(d) from table order by id" +$CLICKHOUSE_CLIENT -q "drop table test" + diff --git a/tests/queries/0_stateless/03173_row_binary_and_native_with_binary_encoded_types.reference b/tests/queries/0_stateless/03173_row_binary_and_native_with_binary_encoded_types.reference new file mode 100644 index 00000000000..0bc257adf23 --- /dev/null +++ b/tests/queries/0_stateless/03173_row_binary_and_native_with_binary_encoded_types.reference @@ -0,0 +1,102 @@ +42 UInt8 +42 UInt8 +\N Nullable(Nothing) +\N Nullable(Nothing) +42 UInt8 +42 UInt8 +-42 Int8 +-42 Int8 +42 UInt16 +42 UInt16 +-42 Int16 +-42 Int16 +42 UInt32 +42 UInt32 +-42 Int32 +-42 Int32 +42 UInt64 +42 UInt64 +-42 Int64 +-42 Int64 +42 UInt128 +42 UInt128 +-42 Int128 +-42 Int128 +42 UInt256 +42 UInt256 +-42 Int256 +-42 Int256 +42.42 Float32 +42.42 Float32 +42.42 Float64 +42.42 Float64 +2020-01-01 Date +2020-01-01 Date +2020-01-01 Date32 +2020-01-01 Date32 +2020-01-01 00:00:00 DateTime +2020-01-01 00:00:00 DateTime +2020-01-01 00:00:00.000000 DateTime64(6) +2020-01-01 00:00:00.000000 DateTime64(6) +Hello, World! String +Hello, World! String +aaaaa FixedString(5) +aaaaa FixedString(5) +a Enum8(\'c\' = -128, \'a\' = 1, \'b\' = 2) +a Enum8(\'c\' = -128, \'a\' = 1, \'b\' = 2) +a Enum16(\'c\' = -1280, \'a\' = 1, \'b\' = 2) +a Enum16(\'c\' = -1280, \'a\' = 1, \'b\' = 2) +42.42 Decimal(9, 3) +42.42 Decimal(9, 3) +42.42 Decimal(18, 3) +42.42 Decimal(18, 3) +42.42 Decimal(38, 3) +42.42 Decimal(38, 3) +42.42 Decimal(76, 3) +42.42 Decimal(76, 3) +984ac60f-4d08-4ef1-9c62-d82f343fbc90 UUID +984ac60f-4d08-4ef1-9c62-d82f343fbc90 UUID +[1,2,3] Array(UInt64) +[1,2,3] Array(UInt64) +[[[1],[2]],[[3,4,5]]] Array(Array(Array(UInt64))) +[[[1],[2]],[[3,4,5]]] Array(Array(Array(UInt64))) +(1,'str',42.42) Tuple(UInt32, String, Float32) +(1,'str',42.42) Tuple(UInt32, String, Float32) +(1,'str',42.42) Tuple(\n a UInt32,\n b String,\n c Float32) +(1,'str',42.42) Tuple(\n a UInt32,\n b String,\n c Float32) +(1,('str',(42.42,-30))) Tuple(UInt32, Tuple(String, Tuple(Float32, Int8))) +(1,('str',(42.42,-30))) Tuple(UInt32, Tuple(String, Tuple(Float32, Int8))) +(1,('str',(42.42,-30))) Tuple(\n a UInt32,\n b Tuple(\n c String,\n d Tuple(\n e Float32,\n f Int8))) +(1,('str',(42.42,-30))) Tuple(\n a UInt32,\n b Tuple(\n c String,\n d Tuple(\n e Float32,\n f Int8))) +\0 \0\0\0\0\0\0\0\0\0\0\0\0\06364136223846793005 0 123459*\0\0\0\0\0\0\0 AggregateFunction(quantile(0.5), UInt64) +\0 \0\0\0\0\0\0\0\0\0\0\0\0\06364136223846793005 0 123459*\0\0\0\0\0\0\0 AggregateFunction(quantile(0.5), UInt64) +42 SimpleAggregateFunction(sum, UInt64) +42 SimpleAggregateFunction(sum, UInt64) +Hello, World! LowCardinality(String) +Hello, World! LowCardinality(String) +{1:'str1',2:'str2'} Map(UInt64, String) +{1:'str1',2:'str2'} Map(UInt64, String) +{1:{1:{1:'str1'}},2:{2:{2:'str2'}}} Map(UInt64, Map(UInt64, Map(UInt64, String))) +{1:{1:{1:'str1'}},2:{2:{2:'str2'}}} Map(UInt64, Map(UInt64, Map(UInt64, String))) +127.0.0.0 IPv4 +127.0.0.0 IPv4 +2001:db8:cafe:1::1 IPv6 +2001:db8:cafe:1::1 IPv6 +true Bool +true Bool +[(1,2),(3,4)] Nested(a UInt32, b UInt32) +[(1,2),(3,4)] Nested(a UInt32, b UInt32) +[(0,0),(10,0),(10,10),(0,10)] Ring +[(0,0),(10,0),(10,10),(0,10)] Ring +(0,0) Point +(0,0) Point +[[(20,20),(50,20),(50,50),(20,50)],[(30,30),(50,50),(50,30)]] Polygon +[[(20,20),(50,20),(50,50),(20,50)],[(30,30),(50,50),(50,30)]] Polygon +[[[(0,0),(10,0),(10,10),(0,10)]],[[(20,20),(50,20),(50,50),(20,50)],[(30,30),(50,50),(50,30)]]] MultiPolygon +[[[(0,0),(10,0),(10,10),(0,10)]],[[(20,20),(50,20),(50,50),(20,50)],[(30,30),(50,50),(50,30)]]] MultiPolygon +[{42:(1,[(2,{1:2})])}] Array(Map(UInt8, Tuple(UInt8, Array(Tuple(UInt8, Map(UInt8, UInt8)))))) +[{42:(1,[(2,{1:2})])}] Array(Map(UInt8, Tuple(UInt8, Array(Tuple(UInt8, Map(UInt8, UInt8)))))) +42 Variant(String, Tuple(\n a UInt32,\n b Array(Map(String, String))), UInt32) +42 Variant(String, Tuple(\n a UInt32,\n b Array(Map(String, String))), UInt32) +[{42:(1,[(2,{1:2})])}] Dynamic +[{42:(1,[(2,{1:2})])}] Dynamic diff --git a/tests/queries/0_stateless/03173_row_binary_and_native_with_binary_encoded_types.sh b/tests/queries/0_stateless/03173_row_binary_and_native_with_binary_encoded_types.sh new file mode 100755 index 00000000000..a1e8aa99548 --- /dev/null +++ b/tests/queries/0_stateless/03173_row_binary_and_native_with_binary_encoded_types.sh @@ -0,0 +1,63 @@ +#!/usr/bin/env bash + +CURDIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd) +# shellcheck source=../shell_config.sh +. "$CURDIR"/../shell_config.sh + +function test +{ + $CLICKHOUSE_LOCAL --allow_experimental_dynamic_type=1 --allow_experimental_variant_type=1 --output_format_binary_encode_types_in_binary_format=1 -q "select $1 as value format RowBinaryWithNamesAndTypes" | $CLICKHOUSE_LOCAL --input-format RowBinaryWithNamesAndTypes --input_format_binary_decode_types_in_binary_format=1 -q "select value, toTypeName(value) from table" + $CLICKHOUSE_LOCAL --allow_experimental_dynamic_type=1 --allow_experimental_variant_type=1 --output_format_native_encode_types_in_binary_format=1 -q "select $1 as value format Native" | $CLICKHOUSE_LOCAL --input-format Native --input_format_native_decode_types_in_binary_format=1 -q "select value, toTypeName(value) from table" +} + +test "materialize(42)::UInt8" +test "NULL" +test "materialize(42)::UInt8" +test "materialize(-42)::Int8" +test "materialize(42)::UInt16" +test "materialize(-42)::Int16" +test "materialize(42)::UInt32" +test "materialize(-42)::Int32" +test "materialize(42)::UInt64" +test "materialize(-42)::Int64" +test "materialize(42)::UInt128" +test "materialize(-42)::Int128" +test "materialize(42)::UInt256" +test "materialize(-42)::Int256" +test "materialize(42.42)::Float32" +test "materialize(42.42)::Float64" +test "materialize('2020-01-01')::Date" +test "materialize('2020-01-01')::Date32" +test "materialize('2020-01-01 00:00:00')::DateTime" +test "materialize('2020-01-01 00:00:00.000000')::DateTime64(6)" +test "materialize('Hello, World!')" +test "materialize('aaaaa')::FixedString(5)" +test "materialize('a')::Enum8('a' = 1, 'b' = 2, 'c' = -128)" +test "materialize('a')::Enum16('a' = 1, 'b' = 2, 'c' = -1280)" +test "materialize(42.42)::Decimal32(3)" +test "materialize(42.42)::Decimal64(3)" +test "materialize(42.42)::Decimal128(3)" +test "materialize(42.42)::Decimal256(3)" +test "materialize('984ac60f-4d08-4ef1-9c62-d82f343fbc90')::UUID" +test "materialize([1, 2, 3])::Array(UInt64)" +test "materialize([[[1], [2]], [[3, 4, 5]]])::Array(Array(Array(UInt64)))" +test "materialize(tuple(1, 'str', 42.42))::Tuple(UInt32, String, Float32)" +test "materialize(tuple(1, 'str', 42.42))::Tuple(a UInt32, b String, c Float32)" +test "materialize(tuple(1, tuple('str', tuple(42.42, -30))))::Tuple(UInt32, Tuple(String, Tuple(Float32, Int8)))" +test "materialize(tuple(1, tuple('str', tuple(42.42, -30))))::Tuple(a UInt32, b Tuple(c String, d Tuple(e Float32, f Int8)))" +test "quantileState(0.5)(42::UInt64)" +test "sumSimpleState(42::UInt64)" +test "toLowCardinality('Hello, World!')" +test "materialize(map(1, 'str1', 2, 'str2'))::Map(UInt64, String)" +test "materialize(map(1, map(1, map(1, 'str1')), 2, map(2, map(2, 'str2'))))::Map(UInt64, Map(UInt64, Map(UInt64, String)))" +test "materialize('127.0.0.0')::IPv4" +test "materialize('2001:db8:cafe:1:0:0:0:1')::IPv6" +test "materialize(true)::Bool" +test "materialize([tuple(1, 2), tuple(3, 4)])::Nested(a UInt32, b UInt32)" +test "materialize([(0, 0), (10, 0), (10, 10), (0, 10)])::Ring" +test "materialize((0, 0))::Point" +test "materialize([[(20, 20), (50, 20), (50, 50), (20, 50)], [(30, 30), (50, 50), (50, 30)]])::Polygon" +test "materialize([[[(0, 0), (10, 0), (10, 10), (0, 10)]], [[(20, 20), (50, 20), (50, 50), (20, 50)],[(30, 30), (50, 50), (50, 30)]]])::MultiPolygon" +test "materialize([map(42, tuple(1, [tuple(2, map(1, 2))]))])" +test "materialize(42::UInt32)::Variant(UInt32, String, Tuple(a UInt32, b Array(Map(String, String))))" +test "materialize([map(42, tuple(1, [tuple(2, map(1, 2))]))])::Dynamic" From b9fbbbb28496692a44b77528f552b81097110a8d Mon Sep 17 00:00:00 2001 From: avogar Date: Fri, 21 Jun 2024 18:23:52 +0000 Subject: [PATCH 081/417] Update settings changes history --- src/Core/Settings.h | 6 +--- src/Core/SettingsChangesHistory.h | 9 ++--- src/Formats/FormatFactory.cpp | 2 -- src/Formats/FormatSettings.h | 2 -- src/Interpreters/Context.cpp | 60 +++---------------------------- src/Server/TCPHandler.cpp | 24 +++++-------- 6 files changed, 19 insertions(+), 84 deletions(-) diff --git a/src/Core/Settings.h b/src/Core/Settings.h index f718acf9b25..23c5d7fc1a2 100644 --- a/src/Core/Settings.h +++ b/src/Core/Settings.h @@ -31,7 +31,6 @@ class IColumn; * for tracking settings changes in different versions and for special `compatibility` setting to work correctly. */ -// clang-format off #define COMMON_SETTINGS(M, ALIAS) \ M(Dialect, dialect, Dialect::clickhouse, "Which dialect will be used to parse query", 0)\ M(UInt64, min_compress_block_size, 65536, "The actual size of the block to compress, if the uncompressed data less than max_compress_block_size is no less than this value and no less than the volume of data for one mark.", 0) \ @@ -934,7 +933,6 @@ class IColumn; M(Int64, prefer_warmed_unmerged_parts_seconds, 0, "Only available in ClickHouse Cloud. If a merged part is less than this many seconds old and is not pre-warmed (see cache_populated_by_fetch), but all its source parts are available and pre-warmed, SELECT queries will read from those parts instead. Only for ReplicatedMergeTree. Note that this only checks whether CacheWarmer processed the part; if the part was fetched into cache by something else, it'll still be considered cold until CacheWarmer gets to it; if it was warmed, then evicted from cache, it'll still be considered warm.", 0) \ M(Bool, iceberg_engine_ignore_schema_evolution, false, "Ignore schema evolution in Iceberg table engine and read all data using latest schema saved on table creation. Note that it can lead to incorrect result", 0) \ M(Bool, allow_deprecated_error_prone_window_functions, false, "Allow usage of deprecated error prone window functions (neighbor, runningAccumulate, runningDifferenceStartingWithFirstValue, runningDifference)", 0) \ - M(Bool, uniform_snowflake_conversion_functions, true, "Enables functions snowflakeIDToDateTime[64] and dateTime[64]ToSnowflakeID while disabling functions snowflakeToDateTime[64] and dateTime[64]ToSnowflake.", 0) \ // End of COMMON_SETTINGS // Please add settings related to formats into the FORMAT_FACTORY_SETTINGS, move obsolete settings to OBSOLETE_SETTINGS and obsolete format settings to OBSOLETE_FORMAT_SETTINGS. @@ -1150,9 +1148,7 @@ class IColumn; M(UInt64, output_format_pretty_max_value_width, 10000, "Maximum width of value to display in Pretty formats. If greater - it will be cut.", 0) \ M(UInt64, output_format_pretty_max_value_width_apply_for_single_value, false, "Only cut values (see the `output_format_pretty_max_value_width` setting) when it is not a single value in a block. Otherwise output it entirely, which is useful for the `SHOW CREATE TABLE` query.", 0) \ M(UInt64Auto, output_format_pretty_color, "auto", "Use ANSI escape sequences in Pretty formats. 0 - disabled, 1 - enabled, 'auto' - enabled if a terminal.", 0) \ - M(String, output_format_pretty_grid_charset, "UTF-8", "Charset for printing grid borders. Available charsets: ASCII, UTF-8 (default one).", 0) \ - M(UInt64, output_format_pretty_display_footer_column_names, true, "Display column names in the footer if there are 999 or more rows.", 0) \ - M(UInt64, output_format_pretty_display_footer_column_names_min_rows, 50, "Sets the minimum threshold value of rows for which to enable displaying column names in the footer. 50 (default)", 0) \ + M(String, output_format_pretty_grid_charset, "UTF-8", "Charset for printing grid borders. Available charsets: ASCII, UTF-8 (default one).", 0) \ M(UInt64, output_format_parquet_row_group_size, 1000000, "Target row group size in rows.", 0) \ M(UInt64, output_format_parquet_row_group_size_bytes, 512 * 1024 * 1024, "Target row group size in bytes, before compression.", 0) \ M(Bool, output_format_parquet_string_as_string, true, "Use Parquet String type instead of Binary for String columns.", 0) \ diff --git a/src/Core/SettingsChangesHistory.h b/src/Core/SettingsChangesHistory.h index 895db9c7ca0..e9da55e66c5 100644 --- a/src/Core/SettingsChangesHistory.h +++ b/src/Core/SettingsChangesHistory.h @@ -75,7 +75,6 @@ namespace SettingsChangesHistory using SettingsChanges = std::vector; } -// clang-format off /// History of settings changes that controls some backward incompatible changes /// across all ClickHouse versions. It maps ClickHouse version to settings changes that were done /// in this version. This history contains both changes to existing settings and newly added settings. @@ -86,6 +85,11 @@ namespace SettingsChangesHistory /// It's used to implement `compatibility` setting (see https://github.com/ClickHouse/ClickHouse/issues/35972) static const std::map settings_changes_history = { + {"24.7", {{"output_format_binary_encode_types_in_binary_format", false, false, "Added new setting to allow to write type names in binary format in RowBinaryWithNamesAndTypes output format"}, + {"input_format_binary_encode_types_in_binary_format", false, false, "Added new setting to allow to read type names in binary format in RowBinaryWithNamesAndTypes input format"}, + {"output_format_native_encode_types_in_binary_format", false, false, "Added new setting to allow to write type names in binary format in Native output format"}, + {"input_format_binary_encode_types_in_binary_format", false, false, "Added new setting to allow to read type names in binary format in Native output format"} + }}, {"24.6", {{"materialize_skip_indexes_on_insert", true, true, "Added new setting to allow to disable materialization of skip indexes on insert"}, {"materialize_statistics_on_insert", true, true, "Added new setting to allow to disable materialization of statistics on insert"}, {"input_format_parquet_use_native_reader", false, false, "When reading Parquet files, to use native reader instead of arrow reader."}, @@ -102,7 +106,6 @@ static const std::map access_control TSA_GUARDED_BY(mutex); mutable OnceFlag resource_manager_initialized; mutable ResourceManagerPtr resource_manager; @@ -832,7 +830,6 @@ ContextMutablePtr Context::createGlobal(ContextSharedPart * shared_part) auto res = std::shared_ptr(new Context); res->shared = shared_part; res->query_access_info = std::make_shared(); - res->query_privileges_info = std::make_shared(); return res; } @@ -1425,7 +1422,7 @@ void Context::checkAccess(const AccessFlags & flags, const StorageID & table_id, void Context::checkAccess(const AccessRightsElement & element) const { checkAccessImpl(element); } void Context::checkAccess(const AccessRightsElements & elements) const { checkAccessImpl(elements); } -std::shared_ptr Context::getAccess() const +std::shared_ptr Context::getAccess() const { /// A helper function to collect parameters for calculating access rights, called with Context::getLocalSharedLock() acquired. auto get_params = [this]() @@ -1442,14 +1439,14 @@ std::shared_ptr Context::getAccess() const { SharedLockGuard lock(mutex); if (access && !need_recalculate_access) - return std::make_shared(access, shared_from_this()); /// No need to recalculate access rights. + return access; /// No need to recalculate access rights. params.emplace(get_params()); if (access && (access->getParams() == *params)) { need_recalculate_access = false; - return std::make_shared(access, shared_from_this()); /// No need to recalculate access rights. + return access; /// No need to recalculate access rights. } } @@ -1469,7 +1466,7 @@ std::shared_ptr Context::getAccess() const } } - return std::make_shared(res, shared_from_this()); + return res; } RowPolicyFilterPtr Context::getRowPolicyFilter(const String & database, const String & table_name, RowPolicyFilterType filter_type) const @@ -1561,36 +1558,11 @@ ResourceManagerPtr Context::getResourceManager() const ClassifierPtr Context::getWorkloadClassifier() const { std::lock_guard lock(mutex); - // NOTE: Workload cannot be changed after query start, and getWorkloadClassifier() should not be called before proper `workload` is set if (!classifier) classifier = getResourceManager()->acquire(getSettingsRef().workload); return classifier; } -String Context::getMergeWorkload() const -{ - SharedLockGuard lock(shared->mutex); - return shared->merge_workload; -} - -void Context::setMergeWorkload(const String & value) -{ - std::lock_guard lock(shared->mutex); - shared->merge_workload = value; -} - -String Context::getMutationWorkload() const -{ - SharedLockGuard lock(shared->mutex); - return shared->mutation_workload; -} - -void Context::setMutationWorkload(const String & value) -{ - std::lock_guard lock(shared->mutex); - shared->mutation_workload = value; -} - Scalars Context::getScalars() const { @@ -1855,15 +1827,6 @@ void Context::addQueryFactoriesInfo(QueryLogFactories factory_type, const String } } -void Context::addQueryPrivilegesInfo(const String & privilege, bool granted) const -{ - std::lock_guard lock(query_privileges_info->mutex); - if (granted) - query_privileges_info->used_privileges.emplace(privilege); - else - query_privileges_info->missing_privileges.emplace(privilege); -} - static bool findIdentifier(const ASTFunction * function) { if (!function || !function->arguments) @@ -2545,21 +2508,6 @@ void Context::makeQueryContext() local_read_query_throttler.reset(); local_write_query_throttler.reset(); backups_query_throttler.reset(); - query_privileges_info = std::make_shared(*query_privileges_info); -} - -void Context::makeQueryContextForMerge(const MergeTreeSettings & merge_tree_settings) -{ - makeQueryContext(); - classifier.reset(); // It is assumed that there are no active queries running using this classifier, otherwise this will lead to crashes - settings.workload = merge_tree_settings.merge_workload.value.empty() ? getMergeWorkload() : merge_tree_settings.merge_workload; -} - -void Context::makeQueryContextForMutate(const MergeTreeSettings & merge_tree_settings) -{ - makeQueryContext(); - classifier.reset(); // It is assumed that there are no active queries running using this classifier, otherwise this will lead to crashes - settings.workload = merge_tree_settings.mutation_workload.value.empty() ? getMutationWorkload() : merge_tree_settings.mutation_workload; } void Context::makeSessionContext() diff --git a/src/Server/TCPHandler.cpp b/src/Server/TCPHandler.cpp index d340fbe7e77..c8f86b1c2c9 100644 --- a/src/Server/TCPHandler.cpp +++ b/src/Server/TCPHandler.cpp @@ -1,8 +1,9 @@ -#include -#include -#include +#include "Interpreters/AsynchronousInsertQueue.h" +#include "Interpreters/SquashingTransform.h" +#include "Parsers/ASTInsertQuery.h" #include #include +#include #include #include #include @@ -245,6 +246,7 @@ TCPHandler::~TCPHandler() void TCPHandler::runImpl() { setThreadName("TCPHandler"); + ThreadStatus thread_status; extractConnectionSettingsFromContext(server.context()); @@ -884,16 +886,13 @@ AsynchronousInsertQueue::PushResult TCPHandler::processAsyncInsertQuery(Asynchro using PushResult = AsynchronousInsertQueue::PushResult; startInsertQuery(); - Squashing squashing(state.input_header, 0, query_context->getSettingsRef().async_insert_max_data_size); + SquashingTransform squashing(0, query_context->getSettingsRef().async_insert_max_data_size); while (readDataNext()) { - squashing.header = state.block_for_insert; - auto planned_chunk = squashing.add({state.block_for_insert.getColumns(), state.block_for_insert.rows()}); - if (planned_chunk.hasChunkInfo()) + auto result = squashing.add(std::move(state.block_for_insert)); + if (result) { - Chunk result_chunk = DB::Squashing::squash(std::move(planned_chunk)); - auto result = state.block_for_insert.cloneWithColumns(result_chunk.getColumns()); return PushResult { .status = PushResult::TOO_MUCH_DATA, @@ -902,12 +901,7 @@ AsynchronousInsertQueue::PushResult TCPHandler::processAsyncInsertQuery(Asynchro } } - auto planned_chunk = squashing.flush(); - Chunk result_chunk; - if (planned_chunk.hasChunkInfo()) - result_chunk = DB::Squashing::squash(std::move(planned_chunk)); - - auto result = squashing.header.cloneWithColumns(result_chunk.getColumns()); + auto result = squashing.add({}); return insert_queue.pushQueryWithBlock(state.parsed_query, std::move(result), query_context); } From 7e7dd78844fb3000e46742bf4884f5166f2abe30 Mon Sep 17 00:00:00 2001 From: avogar Date: Fri, 21 Jun 2024 18:31:41 +0000 Subject: [PATCH 082/417] Revert unrelated changes --- src/Core/Field.cpp | 10 ++- src/Formats/SchemaInferenceUtils.cpp | 96 ++++++++++------------------ 2 files changed, 38 insertions(+), 68 deletions(-) diff --git a/src/Core/Field.cpp b/src/Core/Field.cpp index fb820ad2b56..0e5b1bac000 100644 --- a/src/Core/Field.cpp +++ b/src/Core/Field.cpp @@ -1,12 +1,11 @@ -#include -#include #include -#include -#include #include +#include #include +#include #include -#include +#include +#include #include #include #include @@ -22,7 +21,6 @@ namespace ErrorCodes extern const int CANNOT_RESTORE_FROM_FIELD_DUMP; extern const int DECIMAL_OVERFLOW; extern const int INCORRECT_DATA; - extern const int UNSUPPORTED_METHOD; } template diff --git a/src/Formats/SchemaInferenceUtils.cpp b/src/Formats/SchemaInferenceUtils.cpp index 6cbcae2bebe..31faea2e13e 100644 --- a/src/Formats/SchemaInferenceUtils.cpp +++ b/src/Formats/SchemaInferenceUtils.cpp @@ -879,11 +879,11 @@ namespace } template - bool tryReadFloat(Float64 & value, ReadBuffer & buf, const FormatSettings & settings) + bool tryReadFloat(Float64 & value, ReadBuffer & buf, const FormatSettings & settings, bool & has_fractional) { if (is_json || settings.try_infer_exponent_floats) - return tryReadFloatText(value, buf); - return tryReadFloatTextNoExponent(value, buf); + return tryReadFloatTextExt(value, buf, has_fractional); + return tryReadFloatTextExtNoExponent(value, buf, has_fractional); } template @@ -893,46 +893,31 @@ namespace return nullptr; Float64 tmp_float; + bool has_fractional; if (settings.try_infer_integers) { /// If we read from String, we can do it in a more efficient way. if (auto * string_buf = dynamic_cast(&buf)) { /// Remember the pointer to the start of the number to rollback to it. - char * number_start = buf.position(); - Int64 tmp_int; - bool read_int = tryReadIntText(tmp_int, buf); - /// If we reached eof, it cannot be float (it requires no less data than integer) - if (buf.eof()) - return read_int ? std::make_shared() : nullptr; - - char * int_end = buf.position(); /// We can safely get back to the start of the number, because we read from a string and we didn't reach eof. - buf.position() = number_start; + char * number_start = buf.position(); - bool read_uint = false; - char * uint_end = nullptr; - /// In case of Int64 overflow we can try to infer UInt64. - if (!read_int) - { - UInt64 tmp_uint; - read_uint = tryReadIntText(tmp_uint, buf); - /// If we reached eof, it cannot be float (it requires no less data than integer) - if (buf.eof()) - return read_uint ? std::make_shared() : nullptr; - - uint_end = buf.position(); - buf.position() = number_start; - } - - if (tryReadFloat(tmp_float, buf, settings)) - { - if (read_int && buf.position() == int_end) - return std::make_shared(); - if (read_uint && buf.position() == uint_end) - return std::make_shared(); + /// NOTE: it may break parsing of tryReadFloat() != tryReadIntText() + parsing of '.'/'e' + /// But, for now it is true + if (tryReadFloat(tmp_float, buf, settings, has_fractional) && has_fractional) return std::make_shared(); - } + + Int64 tmp_int; + buf.position() = number_start; + if (tryReadIntText(tmp_int, buf)) + return std::make_shared(); + + /// In case of Int64 overflow we can try to infer UInt64. + UInt64 tmp_uint; + buf.position() = number_start; + if (tryReadIntText(tmp_uint, buf)) + return std::make_shared(); return nullptr; } @@ -942,36 +927,22 @@ namespace /// and then as float. PeekableReadBuffer peekable_buf(buf); PeekableReadBufferCheckpoint checkpoint(peekable_buf); - Int64 tmp_int; - bool read_int = tryReadIntText(tmp_int, peekable_buf); - auto * int_end = peekable_buf.position(); - peekable_buf.rollbackToCheckpoint(true); - bool read_uint = false; - char * uint_end = nullptr; - /// In case of Int64 overflow we can try to infer UInt64. - if (!read_int) - { - PeekableReadBufferCheckpoint new_checkpoint(peekable_buf); - UInt64 tmp_uint; - read_uint = tryReadIntText(tmp_uint, peekable_buf); - uint_end = peekable_buf.position(); - peekable_buf.rollbackToCheckpoint(true); - } - - if (tryReadFloat(tmp_float, peekable_buf, settings)) - { - /// Float parsing reads no fewer bytes than integer parsing, - /// so position of the buffer is either the same, or further. - /// If it's the same, then it's integer. - if (read_int && peekable_buf.position() == int_end) - return std::make_shared(); - if (read_uint && peekable_buf.position() == uint_end) - return std::make_shared(); + if (tryReadFloat(tmp_float, peekable_buf, settings, has_fractional) && has_fractional) return std::make_shared(); - } + peekable_buf.rollbackToCheckpoint(/* drop= */ false); + + Int64 tmp_int; + if (tryReadIntText(tmp_int, peekable_buf)) + return std::make_shared(); + peekable_buf.rollbackToCheckpoint(/* drop= */ true); + + /// In case of Int64 overflow we can try to infer UInt64. + UInt64 tmp_uint; + if (tryReadIntText(tmp_uint, peekable_buf)) + return std::make_shared(); } - else if (tryReadFloat(tmp_float, buf, settings)) + else if (tryReadFloat(tmp_float, buf, settings, has_fractional)) { return std::make_shared(); } @@ -1004,7 +975,8 @@ namespace buf.position() = buf.buffer().begin(); Float64 tmp; - if (tryReadFloat(tmp, buf, settings) && buf.eof()) + bool has_fractional; + if (tryReadFloat(tmp, buf, settings, has_fractional) && buf.eof()) return std::make_shared(); return nullptr; From 785f3ac5c9297dc70cfd5b0dbb9790b869551436 Mon Sep 17 00:00:00 2001 From: avogar Date: Fri, 21 Jun 2024 18:40:49 +0000 Subject: [PATCH 083/417] Revert unrelated changes --- .../domains/data-types-binary-encoding.md | 0 src/Core/Settings.h | 4 ++ src/Core/SettingsChangesHistory.h | 4 ++ src/Formats/FormatFactory.cpp | 2 + src/Formats/FormatSettings.h | 2 + src/Formats/NativeReader.cpp | 2 +- src/Interpreters/Context.cpp | 64 +++++++++++++++++-- src/Server/TCPHandler.cpp | 24 ++++--- 8 files changed, 87 insertions(+), 15 deletions(-) delete mode 100644 docs/en/sql-reference/data-types/domains/data-types-binary-encoding.md diff --git a/docs/en/sql-reference/data-types/domains/data-types-binary-encoding.md b/docs/en/sql-reference/data-types/domains/data-types-binary-encoding.md deleted file mode 100644 index e69de29bb2d..00000000000 diff --git a/src/Core/Settings.h b/src/Core/Settings.h index 23c5d7fc1a2..65146a65a0f 100644 --- a/src/Core/Settings.h +++ b/src/Core/Settings.h @@ -31,6 +31,7 @@ class IColumn; * for tracking settings changes in different versions and for special `compatibility` setting to work correctly. */ +// clang-format off #define COMMON_SETTINGS(M, ALIAS) \ M(Dialect, dialect, Dialect::clickhouse, "Which dialect will be used to parse query", 0)\ M(UInt64, min_compress_block_size, 65536, "The actual size of the block to compress, if the uncompressed data less than max_compress_block_size is no less than this value and no less than the volume of data for one mark.", 0) \ @@ -933,6 +934,7 @@ class IColumn; M(Int64, prefer_warmed_unmerged_parts_seconds, 0, "Only available in ClickHouse Cloud. If a merged part is less than this many seconds old and is not pre-warmed (see cache_populated_by_fetch), but all its source parts are available and pre-warmed, SELECT queries will read from those parts instead. Only for ReplicatedMergeTree. Note that this only checks whether CacheWarmer processed the part; if the part was fetched into cache by something else, it'll still be considered cold until CacheWarmer gets to it; if it was warmed, then evicted from cache, it'll still be considered warm.", 0) \ M(Bool, iceberg_engine_ignore_schema_evolution, false, "Ignore schema evolution in Iceberg table engine and read all data using latest schema saved on table creation. Note that it can lead to incorrect result", 0) \ M(Bool, allow_deprecated_error_prone_window_functions, false, "Allow usage of deprecated error prone window functions (neighbor, runningAccumulate, runningDifferenceStartingWithFirstValue, runningDifference)", 0) \ + M(Bool, uniform_snowflake_conversion_functions, true, "Enables functions snowflakeIDToDateTime[64] and dateTime[64]ToSnowflakeID while disabling functions snowflakeToDateTime[64] and dateTime[64]ToSnowflake.", 0) \ // End of COMMON_SETTINGS // Please add settings related to formats into the FORMAT_FACTORY_SETTINGS, move obsolete settings to OBSOLETE_SETTINGS and obsolete format settings to OBSOLETE_FORMAT_SETTINGS. @@ -1149,6 +1151,8 @@ class IColumn; M(UInt64, output_format_pretty_max_value_width_apply_for_single_value, false, "Only cut values (see the `output_format_pretty_max_value_width` setting) when it is not a single value in a block. Otherwise output it entirely, which is useful for the `SHOW CREATE TABLE` query.", 0) \ M(UInt64Auto, output_format_pretty_color, "auto", "Use ANSI escape sequences in Pretty formats. 0 - disabled, 1 - enabled, 'auto' - enabled if a terminal.", 0) \ M(String, output_format_pretty_grid_charset, "UTF-8", "Charset for printing grid borders. Available charsets: ASCII, UTF-8 (default one).", 0) \ + M(UInt64, output_format_pretty_display_footer_column_names, true, "Display column names in the footer if there are 999 or more rows.", 0) \ + M(UInt64, output_format_pretty_display_footer_column_names_min_rows, 50, "Sets the minimum threshold value of rows for which to enable displaying column names in the footer. 50 (default)", 0) \ M(UInt64, output_format_parquet_row_group_size, 1000000, "Target row group size in rows.", 0) \ M(UInt64, output_format_parquet_row_group_size_bytes, 512 * 1024 * 1024, "Target row group size in bytes, before compression.", 0) \ M(Bool, output_format_parquet_string_as_string, true, "Use Parquet String type instead of Binary for String columns.", 0) \ diff --git a/src/Core/SettingsChangesHistory.h b/src/Core/SettingsChangesHistory.h index e9da55e66c5..e48503bb705 100644 --- a/src/Core/SettingsChangesHistory.h +++ b/src/Core/SettingsChangesHistory.h @@ -75,6 +75,7 @@ namespace SettingsChangesHistory using SettingsChanges = std::vector; } +// clang-format off /// History of settings changes that controls some backward incompatible changes /// across all ClickHouse versions. It maps ClickHouse version to settings changes that were done /// in this version. This history contains both changes to existing settings and newly added settings. @@ -106,6 +107,7 @@ static const std::mapformat_settings_, + std::optional format_settings_, BlockMissingValues * block_missing_values_) : istr(istr_) , header(header_) diff --git a/src/Interpreters/Context.cpp b/src/Interpreters/Context.cpp index db5c5a37125..90c52d683c2 100644 --- a/src/Interpreters/Context.cpp +++ b/src/Interpreters/Context.cpp @@ -18,6 +18,7 @@ #include #include #include +#include #include #include #include @@ -280,6 +281,8 @@ struct ContextSharedPart : boost::noncopyable String default_profile_name; /// Default profile name used for default values. String system_profile_name; /// Profile used by system processes String buffer_profile_name; /// Profile used by Buffer engine for flushing to the underlying + String merge_workload TSA_GUARDED_BY(mutex); /// Workload setting value that is used by all merges + String mutation_workload TSA_GUARDED_BY(mutex); /// Workload setting value that is used by all mutations std::unique_ptr access_control TSA_GUARDED_BY(mutex); mutable OnceFlag resource_manager_initialized; mutable ResourceManagerPtr resource_manager; @@ -610,6 +613,8 @@ struct ContextSharedPart : boost::noncopyable LOG_TRACE(log, "Shutting down database catalog"); DatabaseCatalog::shutdown(); + NamedCollectionFactory::instance().shutdown(); + delete_async_insert_queue.reset(); SHUTDOWN(log, "merges executor", merge_mutate_executor, wait()); @@ -830,6 +835,7 @@ ContextMutablePtr Context::createGlobal(ContextSharedPart * shared_part) auto res = std::shared_ptr(new Context); res->shared = shared_part; res->query_access_info = std::make_shared(); + res->query_privileges_info = std::make_shared(); return res; } @@ -1422,7 +1428,7 @@ void Context::checkAccess(const AccessFlags & flags, const StorageID & table_id, void Context::checkAccess(const AccessRightsElement & element) const { checkAccessImpl(element); } void Context::checkAccess(const AccessRightsElements & elements) const { checkAccessImpl(elements); } -std::shared_ptr Context::getAccess() const +std::shared_ptr Context::getAccess() const { /// A helper function to collect parameters for calculating access rights, called with Context::getLocalSharedLock() acquired. auto get_params = [this]() @@ -1439,14 +1445,14 @@ std::shared_ptr Context::getAccess() const { SharedLockGuard lock(mutex); if (access && !need_recalculate_access) - return access; /// No need to recalculate access rights. + return std::make_shared(access, shared_from_this()); /// No need to recalculate access rights. params.emplace(get_params()); if (access && (access->getParams() == *params)) { need_recalculate_access = false; - return access; /// No need to recalculate access rights. + return std::make_shared(access, shared_from_this()); /// No need to recalculate access rights. } } @@ -1466,7 +1472,7 @@ std::shared_ptr Context::getAccess() const } } - return res; + return std::make_shared(res, shared_from_this()); } RowPolicyFilterPtr Context::getRowPolicyFilter(const String & database, const String & table_name, RowPolicyFilterType filter_type) const @@ -1558,11 +1564,36 @@ ResourceManagerPtr Context::getResourceManager() const ClassifierPtr Context::getWorkloadClassifier() const { std::lock_guard lock(mutex); + // NOTE: Workload cannot be changed after query start, and getWorkloadClassifier() should not be called before proper `workload` is set if (!classifier) classifier = getResourceManager()->acquire(getSettingsRef().workload); return classifier; } +String Context::getMergeWorkload() const +{ + SharedLockGuard lock(shared->mutex); + return shared->merge_workload; +} + +void Context::setMergeWorkload(const String & value) +{ + std::lock_guard lock(shared->mutex); + shared->merge_workload = value; +} + +String Context::getMutationWorkload() const +{ + SharedLockGuard lock(shared->mutex); + return shared->mutation_workload; +} + +void Context::setMutationWorkload(const String & value) +{ + std::lock_guard lock(shared->mutex); + shared->mutation_workload = value; +} + Scalars Context::getScalars() const { @@ -1827,6 +1858,15 @@ void Context::addQueryFactoriesInfo(QueryLogFactories factory_type, const String } } +void Context::addQueryPrivilegesInfo(const String & privilege, bool granted) const +{ + std::lock_guard lock(query_privileges_info->mutex); + if (granted) + query_privileges_info->used_privileges.emplace(privilege); + else + query_privileges_info->missing_privileges.emplace(privilege); +} + static bool findIdentifier(const ASTFunction * function) { if (!function || !function->arguments) @@ -2508,6 +2548,21 @@ void Context::makeQueryContext() local_read_query_throttler.reset(); local_write_query_throttler.reset(); backups_query_throttler.reset(); + query_privileges_info = std::make_shared(*query_privileges_info); +} + +void Context::makeQueryContextForMerge(const MergeTreeSettings & merge_tree_settings) +{ + makeQueryContext(); + classifier.reset(); // It is assumed that there are no active queries running using this classifier, otherwise this will lead to crashes + settings.workload = merge_tree_settings.merge_workload.value.empty() ? getMergeWorkload() : merge_tree_settings.merge_workload; +} + +void Context::makeQueryContextForMutate(const MergeTreeSettings & merge_tree_settings) +{ + makeQueryContext(); + classifier.reset(); // It is assumed that there are no active queries running using this classifier, otherwise this will lead to crashes + settings.workload = merge_tree_settings.mutation_workload.value.empty() ? getMutationWorkload() : merge_tree_settings.mutation_workload; } void Context::makeSessionContext() @@ -3943,7 +3998,6 @@ std::shared_ptr Context::getQueryThreadLog() const std::shared_ptr Context::getQueryViewsLog() const { SharedLockGuard lock(shared->mutex); - if (!shared->system_logs) return {}; diff --git a/src/Server/TCPHandler.cpp b/src/Server/TCPHandler.cpp index c8f86b1c2c9..d340fbe7e77 100644 --- a/src/Server/TCPHandler.cpp +++ b/src/Server/TCPHandler.cpp @@ -1,9 +1,8 @@ -#include "Interpreters/AsynchronousInsertQueue.h" -#include "Interpreters/SquashingTransform.h" -#include "Parsers/ASTInsertQuery.h" +#include +#include +#include #include #include -#include #include #include #include @@ -246,7 +245,6 @@ TCPHandler::~TCPHandler() void TCPHandler::runImpl() { setThreadName("TCPHandler"); - ThreadStatus thread_status; extractConnectionSettingsFromContext(server.context()); @@ -886,13 +884,16 @@ AsynchronousInsertQueue::PushResult TCPHandler::processAsyncInsertQuery(Asynchro using PushResult = AsynchronousInsertQueue::PushResult; startInsertQuery(); - SquashingTransform squashing(0, query_context->getSettingsRef().async_insert_max_data_size); + Squashing squashing(state.input_header, 0, query_context->getSettingsRef().async_insert_max_data_size); while (readDataNext()) { - auto result = squashing.add(std::move(state.block_for_insert)); - if (result) + squashing.header = state.block_for_insert; + auto planned_chunk = squashing.add({state.block_for_insert.getColumns(), state.block_for_insert.rows()}); + if (planned_chunk.hasChunkInfo()) { + Chunk result_chunk = DB::Squashing::squash(std::move(planned_chunk)); + auto result = state.block_for_insert.cloneWithColumns(result_chunk.getColumns()); return PushResult { .status = PushResult::TOO_MUCH_DATA, @@ -901,7 +902,12 @@ AsynchronousInsertQueue::PushResult TCPHandler::processAsyncInsertQuery(Asynchro } } - auto result = squashing.add({}); + auto planned_chunk = squashing.flush(); + Chunk result_chunk; + if (planned_chunk.hasChunkInfo()) + result_chunk = DB::Squashing::squash(std::move(planned_chunk)); + + auto result = squashing.header.cloneWithColumns(result_chunk.getColumns()); return insert_queue.pushQueryWithBlock(state.parsed_query, std::move(result), query_context); } From ae072260bcc016a2a38f28aaaced8e690f85ed36 Mon Sep 17 00:00:00 2001 From: avogar Date: Fri, 21 Jun 2024 19:22:10 +0000 Subject: [PATCH 084/417] Fix style --- src/Common/FieldBinaryEncoding.cpp | 3 ++- src/DataTypes/DataTypesBinaryEncoding.cpp | 6 +++--- 2 files changed, 5 insertions(+), 4 deletions(-) diff --git a/src/Common/FieldBinaryEncoding.cpp b/src/Common/FieldBinaryEncoding.cpp index 7e0d815368c..6c1a8496fe6 100644 --- a/src/Common/FieldBinaryEncoding.cpp +++ b/src/Common/FieldBinaryEncoding.cpp @@ -9,9 +9,10 @@ namespace DB namespace ErrorCodes { extern const int UNSUPPORTED_METHOD; + extern const int INCORRECT_DATA; } -namespace +namespace { enum class FieldBinaryTypeIndex: uint8_t diff --git a/src/DataTypes/DataTypesBinaryEncoding.cpp b/src/DataTypes/DataTypesBinaryEncoding.cpp index 4c42f650798..ea62c7b1286 100644 --- a/src/DataTypes/DataTypesBinaryEncoding.cpp +++ b/src/DataTypes/DataTypesBinaryEncoding.cpp @@ -39,7 +39,7 @@ namespace DB namespace ErrorCodes { extern const int UNSUPPORTED_METHOD; - extern const int UNKNOWN_TYPE; + extern const int INCORRECT_DATA; } namespace @@ -275,7 +275,7 @@ void encodeAggregateFunction(const String & function_name, const Array & paramet encodeDataType(argument_type, buf); } -std::tuple decodeAggregateFunction(ReadBuffer & buf) +std::tuple decodeAggregateFunction(ReadBuffer & buf) { String function_name; readStringBinary(function_name, buf); @@ -665,7 +665,7 @@ DataTypePtr decodeDataType(ReadBuffer & buf) } } - throw Exception(ErrorCodes::UNKNOWN_TYPE, "Unknown type code: {0:#04x}", UInt64(type)); + throw Exception(ErrorCodes::INCORRECT_DATA, "Unknown type code: {0:#04x}", UInt64(type)); } DataTypePtr decodeDataType(const String & data) From b2c22b07c5b191527e6384d17a662cb35d23e58b Mon Sep 17 00:00:00 2001 From: avogar Date: Fri, 21 Jun 2024 19:27:48 +0000 Subject: [PATCH 085/417] Update docs --- docs/en/sql-reference/data-types/data-types-binary-encoding.md | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/docs/en/sql-reference/data-types/data-types-binary-encoding.md b/docs/en/sql-reference/data-types/data-types-binary-encoding.md index ebcb480ea0a..58bc0ed8428 100644 --- a/docs/en/sql-reference/data-types/data-types-binary-encoding.md +++ b/docs/en/sql-reference/data-types/data-types-binary-encoding.md @@ -1,5 +1,5 @@ --- -slug: /en/sql-reference/data-types/dynamic +slug: /en/sql-reference/data-types/data-types-binary-encoding sidebar_position: 56 sidebar_label: Data types binary encoding specification. --- From a0c8106a36515dd0073d7d738720b00b8ac910da Mon Sep 17 00:00:00 2001 From: Kruglov Pavel <48961922+Avogar@users.noreply.github.com> Date: Fri, 21 Jun 2024 22:15:26 +0200 Subject: [PATCH 086/417] Fix settings names --- src/Core/SettingsChangesHistory.h | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/src/Core/SettingsChangesHistory.h b/src/Core/SettingsChangesHistory.h index e48503bb705..a18f442140e 100644 --- a/src/Core/SettingsChangesHistory.h +++ b/src/Core/SettingsChangesHistory.h @@ -87,9 +87,9 @@ namespace SettingsChangesHistory static const std::map settings_changes_history = { {"24.7", {{"output_format_binary_encode_types_in_binary_format", false, false, "Added new setting to allow to write type names in binary format in RowBinaryWithNamesAndTypes output format"}, - {"input_format_binary_encode_types_in_binary_format", false, false, "Added new setting to allow to read type names in binary format in RowBinaryWithNamesAndTypes input format"}, + {"input_format_binary_decode_types_in_binary_format", false, false, "Added new setting to allow to read type names in binary format in RowBinaryWithNamesAndTypes input format"}, {"output_format_native_encode_types_in_binary_format", false, false, "Added new setting to allow to write type names in binary format in Native output format"}, - {"input_format_binary_encode_types_in_binary_format", false, false, "Added new setting to allow to read type names in binary format in Native output format"} + {"input_format_binary_decode_types_in_binary_format", false, false, "Added new setting to allow to read type names in binary format in Native output format"} }}, {"24.6", {{"materialize_skip_indexes_on_insert", true, true, "Added new setting to allow to disable materialization of skip indexes on insert"}, {"materialize_statistics_on_insert", true, true, "Added new setting to allow to disable materialization of statistics on insert"}, From 5044a89f3f08e68ea2c8589b75b9c93cd2a71be2 Mon Sep 17 00:00:00 2001 From: Kruglov Pavel <48961922+Avogar@users.noreply.github.com> Date: Fri, 21 Jun 2024 22:48:47 +0200 Subject: [PATCH 087/417] Update SettingsChangesHistory.h --- src/Core/SettingsChangesHistory.h | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/src/Core/SettingsChangesHistory.h b/src/Core/SettingsChangesHistory.h index a18f442140e..4a19baf2492 100644 --- a/src/Core/SettingsChangesHistory.h +++ b/src/Core/SettingsChangesHistory.h @@ -89,7 +89,7 @@ static const std::map Date: Sat, 22 Jun 2024 18:17:32 +0200 Subject: [PATCH 088/417] Provide keeper override for local development (rewrite path to current directory) The same way like clickhouse-server has Signed-off-by: Azat Khuzhin --- programs/keeper/conf.d/local.yaml | 9 +++++++++ 1 file changed, 9 insertions(+) create mode 100644 programs/keeper/conf.d/local.yaml diff --git a/programs/keeper/conf.d/local.yaml b/programs/keeper/conf.d/local.yaml new file mode 100644 index 00000000000..722e90e374a --- /dev/null +++ b/programs/keeper/conf.d/local.yaml @@ -0,0 +1,9 @@ +logger: + log: + "@remove": remove + errorlog: + "@remove": remove + console: 1 +keeper_server: + log_storage_path: ./logs + snapshot_storage_path: ./snapshots From 6dc68983e787a7d91d814148514937ed29887279 Mon Sep 17 00:00:00 2001 From: Azat Khuzhin Date: Sat, 22 Jun 2024 18:17:02 +0200 Subject: [PATCH 089/417] Fix keeper with non-system-wide directories Otherwise it still tries to access default system-wide directory on config reloading: 2024.06.22 20:36:19.860615 [ 31600 ] {} Application: std::exception. Code: 1001, type: std::__1::__fs::filesystem::filesystem_error, e.what() = filesystem error: in create_directories: Permission denied ["/var/lib/clickhouse-keeper"], Stack trace (when copying this message, always include the lines below): 0. /src/ch/clickhouse/contrib/llvm-project/libcxx/include/exception:141: std::runtime_error::runtime_error(String const&) @ 0x0000000016f16a17 1. /src/ch/clickhouse/contrib/llvm-project/libcxx/include/string:1499: std::system_error::system_error(std::error_code, String const&) @ 0x0000000016f1d09f 2. /src/ch/clickhouse/contrib/llvm-project/libcxx/include/__filesystem/filesystem_error.h:42: std::__fs::filesystem::filesystem_error::filesystem_error[abi:v15000](String const&, std::__fs::filesystem::path const&, std::error_code) @ 0x000000000b639ed2 3. /src/ch/clickhouse/contrib/llvm-project/libcxx/include/__filesystem/filesystem_error.h:90: void std::__fs::filesystem::__throw_filesystem_error[abi:v15000](String&, std::__fs::filesystem::path const&, std::error_code const&) @ 0x0000000016ebaf96 4. /src/ch/clickhouse/contrib/llvm-project/libcxx/src/filesystem/filesystem_common.h:173: std::__fs::filesystem::detail::(anonymous namespace)::ErrorHandler::report(std::error_code const&) const @ 0x0000000016ebe416 5. /src/ch/clickhouse/contrib/llvm-project/libcxx/src/filesystem/operations.cpp:1030: std::__fs::filesystem::__create_directories(std::__fs::filesystem::path const&, std::error_code*) @ 0x0000000016ebec3d 6. /src/ch/clickhouse/contrib/llvm-project/libcxx/src/filesystem/filesystem_common.h:161: std::__fs::filesystem::__create_directories(std::__fs::filesystem::path const&, std::error_code*) @ 0x0000000016ebed0e 7. /src/ch/clickhouse/contrib/llvm-project/libcxx/include/string:1499: DB::ConfigProcessor::savePreprocessedConfig(DB::ConfigProcessor::LoadedConfig&, String) @ 0x00000000128362b3 8. /src/ch/clickhouse/contrib/llvm-project/libcxx/include/string:1499: DB::ConfigReloader::reloadIfNewer(bool, bool, bool, bool) @ 0x000000001283c085 9. /src/ch/clickhouse/src/Common/Config/ConfigReloader.cpp:33: DB::ConfigReloader::ConfigReloader(std::basic_string_view>, std::vector> const&, String const&, zkutil::ZooKeeperNodeCache&&, std::shared_ptr const&, std::function, bool)>&&, bool) @ 0x000000001283b457 10. /src/ch/clickhouse/contrib/llvm-project/libcxx/include/__functional/function.h:818: ? @ 0x000000000b686ecd 11. /src/ch/clickhouse/base/poco/Util/src/Application.cpp:0: Poco::Util::Application::run() @ 0x0000000014afb156 12. /src/ch/clickhouse/programs/keeper/Keeper.cpp:165: DB::Keeper::run() @ 0x000000000b68317e 13. /src/ch/clickhouse/base/poco/Util/src/ServerApplication.cpp:132: Poco::Util::ServerApplication::run(int, char**) @ 0x0000000014b0faf2 14. /src/ch/clickhouse/programs/keeper/Keeper.cpp:0: mainEntryClickHouseKeeper(int, char**) @ 0x000000000b68227e 15. /src/ch/clickhouse/programs/main.cpp:0: main @ 0x00000000061d6204 16. ? @ 0x00007ffff7dc2c88 17. ? @ 0x00007ffff7dc2d4c 18. _start @ 0x00000000061d502e Signed-off-by: Azat Khuzhin --- programs/keeper/Keeper.cpp | 57 +++++++++++++++++++++----------------- 1 file changed, 31 insertions(+), 26 deletions(-) diff --git a/programs/keeper/Keeper.cpp b/programs/keeper/Keeper.cpp index 0d3c1f10894..ef215911e80 100644 --- a/programs/keeper/Keeper.cpp +++ b/programs/keeper/Keeper.cpp @@ -272,6 +272,35 @@ HTTPContextPtr httpContext() return std::make_shared(Context::getGlobalContextInstance()); } +String getKeeperPath(Poco::Util::LayeredConfiguration & config) +{ + String path; + if (config.has("keeper_server.storage_path")) + { + path = config.getString("keeper_server.storage_path"); + } + else if (config.has("keeper_server.log_storage_path")) + { + path = std::filesystem::path(config.getString("keeper_server.log_storage_path")).parent_path(); + } + else if (config.has("keeper_server.snapshot_storage_path")) + { + path = std::filesystem::path(config.getString("keeper_server.snapshot_storage_path")).parent_path(); + } + else if (std::filesystem::is_directory(std::filesystem::path{config.getString("path", DBMS_DEFAULT_PATH)} / "coordination")) + { + throw Exception(ErrorCodes::NO_ELEMENTS_IN_CONFIG, + "By default 'keeper_server.storage_path' could be assigned to {}, but the directory {} already exists. Please specify 'keeper_server.storage_path' in the keeper configuration explicitly", + KEEPER_DEFAULT_PATH, String{std::filesystem::path{config.getString("path", DBMS_DEFAULT_PATH)} / "coordination"}); + } + else + { + path = KEEPER_DEFAULT_PATH; + } + return path; +} + + } int Keeper::main(const std::vector & /*args*/) @@ -321,31 +350,7 @@ try updateMemorySoftLimitInConfig(config()); - std::string path; - - if (config().has("keeper_server.storage_path")) - { - path = config().getString("keeper_server.storage_path"); - } - else if (config().has("keeper_server.log_storage_path")) - { - path = std::filesystem::path(config().getString("keeper_server.log_storage_path")).parent_path(); - } - else if (config().has("keeper_server.snapshot_storage_path")) - { - path = std::filesystem::path(config().getString("keeper_server.snapshot_storage_path")).parent_path(); - } - else if (std::filesystem::is_directory(std::filesystem::path{config().getString("path", DBMS_DEFAULT_PATH)} / "coordination")) - { - throw Exception(ErrorCodes::NO_ELEMENTS_IN_CONFIG, - "By default 'keeper_server.storage_path' could be assigned to {}, but the directory {} already exists. Please specify 'keeper_server.storage_path' in the keeper configuration explicitly", - KEEPER_DEFAULT_PATH, String{std::filesystem::path{config().getString("path", DBMS_DEFAULT_PATH)} / "coordination"}); - } - else - { - path = KEEPER_DEFAULT_PATH; - } - + std::string path = getKeeperPath(config()); std::filesystem::create_directories(path); /// Check that the process user id matches the owner of the data. @@ -562,7 +567,7 @@ try auto main_config_reloader = std::make_unique( config_path, extra_paths, - config().getString("path", KEEPER_DEFAULT_PATH), + getKeeperPath(config()), std::move(unused_cache), unused_event, [&](ConfigurationPtr config, bool /* initial_loading */) From e74ab64b0c1b6bfd3d3df4a6ce9c95eee7b6a716 Mon Sep 17 00:00:00 2001 From: Yakov Olkhovskiy Date: Sun, 23 Jun 2024 05:02:58 +0000 Subject: [PATCH 090/417] filter for columns in projection --- src/Storages/MergeTree/MergeTask.cpp | 7 +++++- .../03174_projection_deduplicate.reference | 1 + .../03174_projection_deduplicate.sql | 24 +++++++++++++++++++ 3 files changed, 31 insertions(+), 1 deletion(-) create mode 100644 tests/queries/0_stateless/03174_projection_deduplicate.reference create mode 100644 tests/queries/0_stateless/03174_projection_deduplicate.sql diff --git a/src/Storages/MergeTree/MergeTask.cpp b/src/Storages/MergeTree/MergeTask.cpp index 56bd1181fef..9fe16ad8650 100644 --- a/src/Storages/MergeTree/MergeTask.cpp +++ b/src/Storages/MergeTree/MergeTask.cpp @@ -818,6 +818,11 @@ bool MergeTask::MergeProjectionsStage::mergeMinMaxIndexAndPrepareProjections() c // projection_future_part->path = global_ctx->future_part->path + "/" + projection.name + ".proj/"; projection_future_part->part_info = {"all", 0, 0, 0}; + Names deduplicate_by_columns; + for (const auto & column : global_ctx->deduplicate_by_columns) + if (projection.metadata->getColumns().has(column)) + deduplicate_by_columns.emplace_back(column); + MergeTreeData::MergingParams projection_merging_params; projection_merging_params.mode = MergeTreeData::MergingParams::Ordinary; if (projection.type == ProjectionDescription::Type::Aggregate) @@ -832,7 +837,7 @@ bool MergeTask::MergeProjectionsStage::mergeMinMaxIndexAndPrepareProjections() c global_ctx->context, global_ctx->space_reservation, global_ctx->deduplicate, - global_ctx->deduplicate_by_columns, + deduplicate_by_columns, global_ctx->cleanup, projection_merging_params, global_ctx->need_prefix, diff --git a/tests/queries/0_stateless/03174_projection_deduplicate.reference b/tests/queries/0_stateless/03174_projection_deduplicate.reference new file mode 100644 index 00000000000..d2ee9d25154 --- /dev/null +++ b/tests/queries/0_stateless/03174_projection_deduplicate.reference @@ -0,0 +1 @@ +1 one diff --git a/tests/queries/0_stateless/03174_projection_deduplicate.sql b/tests/queries/0_stateless/03174_projection_deduplicate.sql new file mode 100644 index 00000000000..529c3260baa --- /dev/null +++ b/tests/queries/0_stateless/03174_projection_deduplicate.sql @@ -0,0 +1,24 @@ +-- https://github.com/ClickHouse/ClickHouse/issues/65548 +DROP TABLE IF EXISTS test_projection_deduplicate; + +CREATE TABLE test_projection_deduplicate +( + `id` Int32, + `string` String, + PROJECTION test_projection + ( + SELECT id + GROUP BY id + ) +) +ENGINE = MergeTree +PRIMARY KEY id; + +INSERT INTO test_projection_deduplicate VALUES (1, 'one'); +INSERT INTO test_projection_deduplicate VALUES (1, 'one'); + +OPTIMIZE TABLE test_projection_deduplicate DEDUPLICATE; + +SELECT * FROM test_projection_deduplicate; + +DROP TABLE test_projection_deduplicate; From bdac9d6c24164adc85b5c90f41b58b4d6416daed Mon Sep 17 00:00:00 2001 From: divanik Date: Mon, 24 Jun 2024 09:58:00 +0000 Subject: [PATCH 091/417] Fix build bug --- src/Disks/DiskSelector.cpp | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/src/Disks/DiskSelector.cpp b/src/Disks/DiskSelector.cpp index 6e57bdad77e..f3b4893e820 100644 --- a/src/Disks/DiskSelector.cpp +++ b/src/Disks/DiskSelector.cpp @@ -26,7 +26,7 @@ void DiskSelector::assertInitialized() const } -void DiskSelector::( +void DiskSelector::initialize( const Poco::Util::AbstractConfiguration & config, const String & config_prefix, ContextPtr context, DiskValidator disk_validator) { Poco::Util::AbstractConfiguration::Keys keys; From 6f71e04b15078581e728fe40fd9fd6e583c5e06b Mon Sep 17 00:00:00 2001 From: avogar Date: Mon, 24 Jun 2024 13:25:25 +0000 Subject: [PATCH 092/417] Add separate encoding for datetimes with timezones, fix encoding for Dynamic type --- .../data-types/data-types-binary-encoding.md | 60 +++--- src/DataTypes/DataTypeDynamic.cpp | 2 +- src/DataTypes/DataTypesBinaryEncoding.cpp | 124 ++++++++---- src/DataTypes/DataTypesBinaryEncoding.h | 182 +++++++++--------- .../gtest_data_types_binary_encoding.cpp | 9 +- ...172_dynamic_binary_serialization.reference | 6 +- .../03172_dynamic_binary_serialization.sh | 6 +- ...native_with_binary_encoded_types.reference | 12 ++ ...ry_and_native_with_binary_encoded_types.sh | 6 + 9 files changed, 242 insertions(+), 165 deletions(-) diff --git a/docs/en/sql-reference/data-types/data-types-binary-encoding.md b/docs/en/sql-reference/data-types/data-types-binary-encoding.md index 58bc0ed8428..ba1a4fa44c5 100644 --- a/docs/en/sql-reference/data-types/data-types-binary-encoding.md +++ b/docs/en/sql-reference/data-types/data-types-binary-encoding.md @@ -27,35 +27,37 @@ sidebar_label: Data types binary encoding specification. | `Date` | `0x0F` | | `Date32` | `0x10` | | `DateTime` | `0x11` | -| `DateTime64(P)` | `0x12` | -| `String` | `0x13` | -| `FixedString(N)` | `0x14` | -| `Enum8` | `0x15...` | -| `Enum16` | `0x16...>` | -| `Decimal32(P, S)` | `0x17` | -| `Decimal64(P, S)` | `0x18` | -| `Decimal128(P, S)` | `0x19` | -| `Decimal256(P, S)` | `0x1A` | -| `UUID` | `0x1B` | -| `Array(T)` | `0x1C` | -| `Tuple(T1, ..., TN)` | `0x1D...` | -| `Tuple(name1 T1, ..., nameN TN)` | `0x1E...` | -| `Set` | `0x1F` | -| `Interval` | `0x20` (see [interval kind binary encoding](#interval-kind-binary-encoding)) | -| `Nullable(T)` | `0x21` | -| `Function` | `0x22...` | -| `AggregateFunction(function_name(param_1, ..., param_N), arg_T1, ..., arg_TN)` | `0x23......` (see [aggregate function parameter binary encoding](#aggregate-function-parameter-binary-encoding)) | -| `LowCardinality(T)` | `0x24` | -| `Map(K, V)` | `0x25` | -| `Object('schema_format')` | `0x26` | -| `IPv4` | `0x27` | -| `IPv6` | `0x28` | -| `Variant(T1, ..., TN)` | `0x29...` | -| `Dynamic` | `0x2A` | -| `Custom type` (`Ring`, `Polygon`, etc) | `0x2B` | -| `Bool` | `0x2C` | -| `SimpleAggregateFunction(function_name(param_1, ..., param_N), arg_T1, ..., arg_TN)` | `0x2D......` (see [aggregate function parameter binary encoding](#aggregate-function-parameter-binary-encoding)) | -| `Nested(name1 T1, ..., nameN TN)` | `0x2E...` | +| `DateTime(time_zone)` | `0x12` | +| `DateTime64(P)` | `0x13` | +| `DateTime64(P, time_zone)` | `0x14` | +| `String` | `0x15` | +| `FixedString(N)` | `0x16` | +| `Enum8` | `0x17...` | +| `Enum16` | `0x18...>` | +| `Decimal32(P, S)` | `0x19` | +| `Decimal64(P, S)` | `0x1A` | +| `Decimal128(P, S)` | `0x1B` | +| `Decimal256(P, S)` | `0x1C` | +| `UUID` | `0x1D` | +| `Array(T)` | `0x1E` | +| `Tuple(T1, ..., TN)` | `0x1F...` | +| `Tuple(name1 T1, ..., nameN TN)` | `0x20...` | +| `Set` | `0x21` | +| `Interval` | `0x22` (see [interval kind binary encoding](#interval-kind-binary-encoding)) | +| `Nullable(T)` | `0x23` | +| `Function` | `0x24...` | +| `AggregateFunction(function_name(param_1, ..., param_N), arg_T1, ..., arg_TN)` | `0x25......` (see [aggregate function parameter binary encoding](#aggregate-function-parameter-binary-encoding)) | +| `LowCardinality(T)` | `0x26` | +| `Map(K, V)` | `0x27` | +| `Object('schema_format')` | `0x28` | +| `IPv4` | `0x29` | +| `IPv6` | `0x2A` | +| `Variant(T1, ..., TN)` | `0x2B...` | +| `Dynamic(max_types=N)` | `0x2C` | +| `Custom type` (`Ring`, `Polygon`, etc) | `0x2D` | +| `Bool` | `0x2E` | +| `SimpleAggregateFunction(function_name(param_1, ..., param_N), arg_T1, ..., arg_TN)` | `0x2F......` (see [aggregate function parameter binary encoding](#aggregate-function-parameter-binary-encoding)) | +| `Nested(name1 T1, ..., nameN TN)` | `0x30...` | ### Interval kind binary encoding diff --git a/src/DataTypes/DataTypeDynamic.cpp b/src/DataTypes/DataTypeDynamic.cpp index c920e69c13b..ba263c2f7fa 100644 --- a/src/DataTypes/DataTypeDynamic.cpp +++ b/src/DataTypes/DataTypeDynamic.cpp @@ -69,7 +69,7 @@ static DataTypePtr create(const ASTPtr & arguments) auto * literal = argument->arguments->children[1]->as(); - if (!literal || literal->value.getType() != Field::Types::UInt64 || literal->value.get() == 0 || literal->value.get() > 255) + if (!literal || literal->value.getType() != Field::Types::UInt64 || literal->value.get() == 0 || literal->value.get() > ColumnVariant::MAX_NESTED_COLUMNS) throw Exception(ErrorCodes::UNEXPECTED_AST_STRUCTURE, "'max_types' argument for Dynamic type should be a positive integer between 1 and 255"); return std::make_shared(literal->value.get()); diff --git a/src/DataTypes/DataTypesBinaryEncoding.cpp b/src/DataTypes/DataTypesBinaryEncoding.cpp index ea62c7b1286..02a58897529 100644 --- a/src/DataTypes/DataTypesBinaryEncoding.cpp +++ b/src/DataTypes/DataTypesBinaryEncoding.cpp @@ -64,36 +64,38 @@ enum class BinaryTypeIndex : uint8_t Float64 = 0x0E, Date = 0x0F, Date32 = 0x10, - DateTime = 0x11, - DateTime64 = 0x12, - String = 0x13, - FixedString = 0x14, - Enum8 = 0x15, - Enum16 = 0x16, - Decimal32 = 0x17, - Decimal64 = 0x18, - Decimal128 = 0x19, - Decimal256 = 0x1A, - UUID = 0x1B, - Array = 0x1C, - UnnamedTuple = 0x1D, - NamedTuple = 0x1E, - Set = 0x1F, - Interval = 0x20, - Nullable = 0x21, - Function = 0x22, - AggregateFunction = 0x23, - LowCardinality = 0x24, - Map = 0x25, - Object = 0x26, - IPv4 = 0x27, - IPv6 = 0x28, - Variant = 0x29, - Dynamic = 0x2A, - Custom = 0x2B, - Bool = 0x2C, - SimpleAggregateFunction = 0x2D, - Nested = 0x2E, + DateTimeUTC = 0x11, + DateTimeWithTimezone = 0x12, + DateTime64UTC = 0x13, + DateTime64WithTimezone = 0x14, + String = 0x15, + FixedString = 0x16, + Enum8 = 0x17, + Enum16 = 0x18, + Decimal32 = 0x19, + Decimal64 = 0x1A, + Decimal128 = 0x1B, + Decimal256 = 0x1C, + UUID = 0x1D, + Array = 0x1E, + UnnamedTuple = 0x1F, + NamedTuple = 0x20, + Set = 0x21, + Interval = 0x22, + Nullable = 0x23, + Function = 0x24, + AggregateFunction = 0x25, + LowCardinality = 0x26, + Map = 0x27, + Object = 0x28, + IPv4 = 0x29, + IPv6 = 0x2A, + Variant = 0x2B, + Dynamic = 0x2C, + Custom = 0x2D, + Bool = 0x2E, + SimpleAggregateFunction = 0x2F, + Nested = 0x30, }; BinaryTypeIndex getBinaryTypeIndex(const DataTypePtr & type) @@ -154,9 +156,13 @@ BinaryTypeIndex getBinaryTypeIndex(const DataTypePtr & type) case TypeIndex::Date32: return BinaryTypeIndex::Date32; case TypeIndex::DateTime: - return BinaryTypeIndex::DateTime; + if (assert_cast(*type).hasExplicitTimeZone()) + return BinaryTypeIndex::DateTimeWithTimezone; + return BinaryTypeIndex::DateTimeUTC; case TypeIndex::DateTime64: - return BinaryTypeIndex::DateTime64; + if (assert_cast(*type).hasExplicitTimeZone()) + return BinaryTypeIndex::DateTime64WithTimezone; + return BinaryTypeIndex::DateTime64UTC; case TypeIndex::String: return BinaryTypeIndex::String; case TypeIndex::FixedString: @@ -307,13 +313,26 @@ void encodeDataType(const DataTypePtr & type, WriteBuffer & buf) /// Then, write additional information depending on the data type. switch (binary_type_index) { - case BinaryTypeIndex::DateTime64: + case BinaryTypeIndex::DateTimeWithTimezone: + { + const auto & datetime_type = assert_cast(*type); + writeStringBinary(datetime_type.getTimeZone().getTimeZone(), buf); + break; + } + case BinaryTypeIndex::DateTime64UTC: { const auto & datetime64_type = assert_cast(*type); /// Maximum scale for DateTime64 is 9, so we can write it as 1 byte. buf.write(UInt8(datetime64_type.getScale())); break; } + case BinaryTypeIndex::DateTime64WithTimezone: + { + const auto & datetime64_type = assert_cast(*type); + buf.write(UInt8(datetime64_type.getScale())); + writeStringBinary(datetime64_type.getTimeZone().getTimeZone(), buf); + break; + } case BinaryTypeIndex::FixedString: { const auto & fixed_string_type = assert_cast(*type); @@ -372,10 +391,10 @@ void encodeDataType(const DataTypePtr & type, WriteBuffer & buf) case BinaryTypeIndex::UnnamedTuple: { const auto & tuple_type = assert_cast(*type); - const auto & types = tuple_type.getElements(); - writeVarUInt(types.size(), buf); - for (size_t i = 0; i != types.size(); ++i) - encodeDataType(types[i], buf); + const auto & element_types = tuple_type.getElements(); + writeVarUInt(element_types.size(), buf); + for (const auto & element_type : element_types) + encodeDataType(element_type, buf); break; } case BinaryTypeIndex::Interval: @@ -430,6 +449,13 @@ void encodeDataType(const DataTypePtr & type, WriteBuffer & buf) encodeDataType(variant, buf); break; } + case BinaryTypeIndex::Dynamic: + { + const auto & dynamic_type = assert_cast(*type); + /// Maximum number of dynamic types is 255, we can write it as 1 byte. + writeBinary(UInt8(dynamic_type.getMaxDynamicTypes()), buf); + break; + } case BinaryTypeIndex::AggregateFunction: { const auto & aggregate_function_type = assert_cast(*type); @@ -516,14 +542,28 @@ DataTypePtr decodeDataType(ReadBuffer & buf) return std::make_shared(); case BinaryTypeIndex::Date32: return std::make_shared(); - case BinaryTypeIndex::DateTime: + case BinaryTypeIndex::DateTimeUTC: return std::make_shared(); - case BinaryTypeIndex::DateTime64: + case BinaryTypeIndex::DateTimeWithTimezone: + { + String time_zone; + readStringBinary(time_zone, buf); + return std::make_shared(time_zone); + } + case BinaryTypeIndex::DateTime64UTC: { UInt8 scale; readBinary(scale, buf); return std::make_shared(scale); } + case BinaryTypeIndex::DateTime64WithTimezone: + { + UInt8 scale; + readBinary(scale, buf); + String time_zone; + readStringBinary(time_zone, buf); + return std::make_shared(scale, time_zone); + } case BinaryTypeIndex::String: return std::make_shared(); case BinaryTypeIndex::FixedString: @@ -627,7 +667,11 @@ DataTypePtr decodeDataType(ReadBuffer & buf) return std::make_shared(variants); } case BinaryTypeIndex::Dynamic: - return std::make_shared(); + { + UInt8 max_dynamic_types; + readBinary(max_dynamic_types, buf); + return std::make_shared(max_dynamic_types); + } case BinaryTypeIndex::AggregateFunction: { size_t version; diff --git a/src/DataTypes/DataTypesBinaryEncoding.h b/src/DataTypes/DataTypesBinaryEncoding.h index b4ed500f185..d735565f636 100644 --- a/src/DataTypes/DataTypesBinaryEncoding.h +++ b/src/DataTypes/DataTypesBinaryEncoding.h @@ -8,104 +8,106 @@ namespace DB /** Binary encoding for ClickHouse data types: -|--------------------------------------------------------------------------------------|--------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------| -| ClickHouse data type | Binary encoding | -|--------------------------------------------------------------------------------------|--------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------| -| `Nothing` | `0x00` | -| `UInt8` | `0x01` | -| `UInt16` | `0x02` | -| `UInt32` | `0x03` | -| `UInt64` | `0x04` | -| `UInt128` | `0x05` | -| `UInt256` | `0x06` | -| `Int8` | `0x07` | -| `Int16` | `0x08` | -| `Int32` | `0x09` | -| `Int64` | `0x0A` | -| `Int128` | `0x0B` | -| `Int256` | `0x0C` | -| `Float32` | `0x0D` | -| `Float64` | `0x0E` | -| `Date` | `0x0F` | -| `Date32` | `0x10` | -| `DateTime` | `0x11` | -| `DateTime64(P)` | `0x12` | -| `String` | `0x13` | -| `FixedString(N)` | `0x14` | -| `Enum8` | `0x15...` | -| `Enum16` | `0x16...>` | -| `Decimal32(P, S)` | `0x17` | -| `Decimal64(P, S)` | `0x18` | -| `Decimal128(P, S)` | `0x19` | -| `Decimal256(P, S)` | `0x1A` | -| `UUID` | `0x1B` | -| `Array(T)` | `0x1C` | -| `Tuple(T1, ..., TN)` | `0x1D...` | -| `Tuple(name1 T1, ..., nameN TN)` | `0x1E...` | -| `Set` | `0x1F` | -| `Interval` | `0x20` | -| `Nullable(T)` | `0x21` | -| `Function` | `0x22...` | -| `AggregateFunction(function_name(param_1, ..., param_N), arg_T1, ..., arg_TN)` | `0x23......` | -| `LowCardinality(T)` | `0x24` | -| `Map(K, V)` | `0x25` | -| `Object('schema_format')` | `0x26` | -| `IPv4` | `0x27` | -| `IPv6` | `0x28` | -| `Variant(T1, ..., TN)` | `0x29...` | -| `Dynamic` | `0x2A` | -| `Custom type` (`Ring`, `Polygon`, etc) | `0x2B` | -| `Bool` | `0x2C` | -| `SimpleAggregateFunction(function_name(param_1, ..., param_N), arg_T1, ..., arg_TN)` | `0x2D......` | -| `Nested(name1 T1, ..., nameN TN)` | `0x2E...` | -|--------------------------------------------------------------------------------------|--------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------| +|------------------------------------------------------------------------------------|------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------| +| ClickHouse data type | Binary encoding | +|------------------------------------------------------------------------------------|------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------| +| Nothing | 0x00 | +| UInt8 | 0x01 | +| UInt16 | 0x02 | +| UInt32 | 0x03 | +| UInt64 | 0x04 | +| UInt128 | 0x05 | +| UInt256 | 0x06 | +| Int8 | 0x07 | +| Int16 | 0x08 | +| Int32 | 0x09 | +| Int64 | 0x0A | +| Int128 | 0x0B | +| Int256 | 0x0C | +| Float32 | 0x0D | +| Float64 | 0x0E | +| Date | 0x0F | +| Date32 | 0x10 | +| DateTime | 0x11 | +| DateTime(time_zone) | 0x12 | +| DateTime64(P) | 0x13 | +| DateTime64(P, time_zone) | 0x14 | +| String | 0x15 | +| FixedString(N) | 0x16 | +| Enum8 | 0x17... | +| Enum16 | 0x18...> | +| Decimal32(P, S) | 0x19 | +| Decimal64(P, S) | 0x1A | +| Decimal128(P, S) | 0x1B | +| Decimal256(P, S) | 0x1C | +| UUID | 0x1D | +| Array(T) | 0x1E | +| Tuple(T1, ..., TN) | 0x1F... | +| Tuple(name1 T1, ..., nameN TN) | 0x20... | +| Set | 0x21 | +| Interval | 0x22 | +| Nullable(T) | 0x23 | +| Function | 0x24... | +| AggregateFunction(function_name(param_1, ..., param_N), arg_T1, ..., arg_TN) | 0x25...... | +| LowCardinality(T) | 0x26 | +| Map(K, V) | 0x27 | +| Object('schema_format') | 0x28 | +| IPv4 | 0x29 | +| IPv6 | 0x2A | +| Variant(T1, ..., TN) | 0x2B... | +| Dynamic(max_types=N) | 0x2C | +| Custom type (Ring, Polygon, etc) | 0x2D | +| Bool | 0x2E | +| SimpleAggregateFunction(function_name(param_1, ..., param_N), arg_T1, ..., arg_TN) | 0x2F...... | +| Nested(name1 T1, ..., nameN TN) | 0x30... | +|------------------------------------------------------------------------------------|------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------| Interval kind binary encoding: |---------------|-----------------| | Interval kind | Binary encoding | |---------------|-----------------| -| `Nanosecond` | `0x00` | -| `Microsecond` | `0x01` | -| `Millisecond` | `0x02` | -| `Second` | `0x03` | -| `Minute` | `0x04` | -| `Hour` | `0x05` | -| `Day` | `0x06` | -| `Week` | `0x07` | -| `Month` | `0x08` | -| `Quarter` | `0x09` | -| `Year` | `0x1A` | +| Nanosecond | 0x00 | +| Microsecond | 0x01 | +| Millisecond | 0x02 | +| Second | 0x03 | +| Minute | 0x04 | +| Hour | 0x05 | +| Day | 0x06 | +| Week | 0x07 | +| Month | 0x08 | +| Quarter | 0x09 | +| Year | 0x1A | |---------------|-----------------| Aggregate function parameter binary encoding (binary encoding of a Field, see src/Common/FieldBinaryEncoding.h): -|--------------------------|--------------------------------------------------------------------------------------------------------------------------------| -| Parameter type | Binary encoding | -|--------------------------|--------------------------------------------------------------------------------------------------------------------------------| -| `Null` | `0x00` | -| `UInt64` | `0x01` | -| `Int64` | `0x02` | -| `UInt128` | `0x03` | -| `Int128` | `0x04` | -| `UInt128` | `0x05` | -| `Int128` | `0x06` | -| `Float64` | `0x07` | -| `Decimal32` | `0x08` | -| `Decimal64` | `0x09` | -| `Decimal128` | `0x0A` | -| `Decimal256` | `0x0B` | -| `String` | `0x0C` | -| `Array` | `0x0D...` | -| `Tuple` | `0x0E...` | -| `Map` | `0x0F...` | -| `IPv4` | `0x10` | -| `IPv6` | `0x11` | -| `UUID` | `0x12` | -| `Bool` | `0x13` | -| `Object` | `0x14...` | -| `AggregateFunctionState` | `0x15` | -| `Negative infinity` | `0xFE` | -| `Positive infinity` | `0xFF` | -|--------------------------|--------------------------------------------------------------------------------------------------------------------------------| +|------------------------|------------------------------------------------------------------------------------------------------------------------------| +| Parameter type | Binary encoding | +|------------------------|------------------------------------------------------------------------------------------------------------------------------| +| Null | 0x00 | +| UInt64 | 0x01 | +| Int64 | 0x02 | +| UInt128 | 0x03 | +| Int128 | 0x04 | +| UInt128 | 0x05 | +| Int128 | 0x06 | +| Float64 | 0x07 | +| Decimal32 | 0x08 | +| Decimal64 | 0x09 | +| Decimal128 | 0x0A | +| Decimal256 | 0x0B | +| String | 0x0C | +| Array | 0x0D... | +| Tuple | 0x0E... | +| Map | 0x0F... | +| IPv4 | 0x10 | +| IPv6 | 0x11 | +| UUID | 0x12 | +| Bool | 0x13 | +| Object | 0x14... | +| AggregateFunctionState | 0x15 | +| Negative infinity | 0xFE | +| Positive infinity | 0xFF | +|------------------------|------------------------------------------------------------------------------------------------------------------------------| */ String encodeDataType(const DataTypePtr & type); diff --git a/src/DataTypes/tests/gtest_data_types_binary_encoding.cpp b/src/DataTypes/tests/gtest_data_types_binary_encoding.cpp index 4459e2558b6..6dbb9451922 100644 --- a/src/DataTypes/tests/gtest_data_types_binary_encoding.cpp +++ b/src/DataTypes/tests/gtest_data_types_binary_encoding.cpp @@ -21,6 +21,7 @@ #include #include #include +#include using namespace DB; @@ -44,7 +45,7 @@ void check(const DataTypePtr & type) GTEST_TEST(DataTypesBinaryEncoding, EncodeAndDecode) { - registerAggregateFunctions(); + tryRegisterAggregateFunctions(); check(std::make_shared()); check(std::make_shared()); check(std::make_shared()); @@ -63,7 +64,11 @@ GTEST_TEST(DataTypesBinaryEncoding, EncodeAndDecode) check(std::make_shared()); check(std::make_shared()); check(std::make_shared()); + check(std::make_shared("EST")); + check(std::make_shared("CET")); check(std::make_shared(3)); + check(std::make_shared(3, "EST")); + check(std::make_shared(3, "CET")); check(std::make_shared()); check(std::make_shared(10)); check(DataTypeFactory::instance().get("Enum8('a' = 1, 'b' = 2, 'c' = 3, 'd' = -128)")); @@ -109,6 +114,8 @@ GTEST_TEST(DataTypesBinaryEncoding, EncodeAndDecode) check(std::make_shared()); check(DataTypeFactory::instance().get("Variant(String, UInt32, Date32)")); check(std::make_shared()); + check(std::make_shared(10)); + check(std::make_shared(255)); check(DataTypeFactory::instance().get("Bool")); check(DataTypeFactory::instance().get("SimpleAggregateFunction(sum, UInt64)")); check(DataTypeFactory::instance().get("SimpleAggregateFunction(maxMap, Tuple(Array(UInt32), Array(UInt32)))")); diff --git a/tests/queries/0_stateless/03172_dynamic_binary_serialization.reference b/tests/queries/0_stateless/03172_dynamic_binary_serialization.reference index 26bd3326d5c..1670a7e9634 100644 --- a/tests/queries/0_stateless/03172_dynamic_binary_serialization.reference +++ b/tests/queries/0_stateless/03172_dynamic_binary_serialization.reference @@ -15,8 +15,10 @@ 42.42 Float64 2020-01-01 Date 2020-01-01 Date32 -2020-01-01 00:00:00 DateTime -2020-01-01 00:00:00.000000 DateTime64(6) +2020-01-01 00:00:00 DateTime(\'CET\') +2020-01-01 00:00:00 DateTime(\'EST\') +2020-01-01 00:00:00.000000 DateTime64(6, \'EST\') +2020-01-01 00:00:00.000000 DateTime64(6, \'CET\') Hello, World! String aaaaa FixedString(5) a Enum8(\'c\' = -128, \'a\' = 1, \'b\' = 2) diff --git a/tests/queries/0_stateless/03172_dynamic_binary_serialization.sh b/tests/queries/0_stateless/03172_dynamic_binary_serialization.sh index b078211f088..9e6d78adba5 100755 --- a/tests/queries/0_stateless/03172_dynamic_binary_serialization.sh +++ b/tests/queries/0_stateless/03172_dynamic_binary_serialization.sh @@ -24,8 +24,10 @@ $CLICKHOUSE_CLIENT -q "insert into test select 13, materialize(42.42)::Float32" $CLICKHOUSE_CLIENT -q "insert into test select 14, materialize(42.42)::Float64" $CLICKHOUSE_CLIENT -q "insert into test select 15, materialize('2020-01-01')::Date" $CLICKHOUSE_CLIENT -q "insert into test select 16, materialize('2020-01-01')::Date32" -$CLICKHOUSE_CLIENT -q "insert into test select 17, materialize('2020-01-01 00:00:00')::DateTime" -$CLICKHOUSE_CLIENT -q "insert into test select 18, materialize('2020-01-01 00:00:00.000000')::DateTime64(6)" +$CLICKHOUSE_CLIENT -q "insert into test select 17, materialize('2020-01-01 00:00:00')::DateTime('EST')" +$CLICKHOUSE_CLIENT -q "insert into test select 17, materialize('2020-01-01 00:00:00')::DateTime('CET')" +$CLICKHOUSE_CLIENT -q "insert into test select 18, materialize('2020-01-01 00:00:00.000000')::DateTime64(6, 'EST')" +$CLICKHOUSE_CLIENT -q "insert into test select 18, materialize('2020-01-01 00:00:00.000000')::DateTime64(6, 'CET')" $CLICKHOUSE_CLIENT -q "insert into test select 19, materialize('Hello, World!')" $CLICKHOUSE_CLIENT -q "insert into test select 20, materialize('aaaaa')::FixedString(5)" $CLICKHOUSE_CLIENT -q "insert into test select 21, materialize('a')::Enum8('a' = 1, 'b' = 2, 'c' = -128)" diff --git a/tests/queries/0_stateless/03173_row_binary_and_native_with_binary_encoded_types.reference b/tests/queries/0_stateless/03173_row_binary_and_native_with_binary_encoded_types.reference index 0bc257adf23..1ba147f9627 100644 --- a/tests/queries/0_stateless/03173_row_binary_and_native_with_binary_encoded_types.reference +++ b/tests/queries/0_stateless/03173_row_binary_and_native_with_binary_encoded_types.reference @@ -36,8 +36,16 @@ 2020-01-01 Date32 2020-01-01 00:00:00 DateTime 2020-01-01 00:00:00 DateTime +2020-01-01 00:00:00 DateTime(\'EST\') +2020-01-01 00:00:00 DateTime(\'EST\') +2020-01-01 00:00:00 DateTime(\'CET\') +2020-01-01 00:00:00 DateTime(\'CET\') 2020-01-01 00:00:00.000000 DateTime64(6) 2020-01-01 00:00:00.000000 DateTime64(6) +2020-01-01 00:00:00.000000 DateTime64(6, \'EST\') +2020-01-01 00:00:00.000000 DateTime64(6, \'EST\') +2020-01-01 00:00:00.000000 DateTime64(6, \'CET\') +2020-01-01 00:00:00.000000 DateTime64(6, \'CET\') Hello, World! String Hello, World! String aaaaa FixedString(5) @@ -100,3 +108,7 @@ true Bool 42 Variant(String, Tuple(\n a UInt32,\n b Array(Map(String, String))), UInt32) [{42:(1,[(2,{1:2})])}] Dynamic [{42:(1,[(2,{1:2})])}] Dynamic +[{42:(1,[(2,{1:2})])}] Dynamic(max_types=10) +[{42:(1,[(2,{1:2})])}] Dynamic(max_types=10) +[{42:(1,[(2,{1:2})])}] Dynamic(max_types=255) +[{42:(1,[(2,{1:2})])}] Dynamic(max_types=255) diff --git a/tests/queries/0_stateless/03173_row_binary_and_native_with_binary_encoded_types.sh b/tests/queries/0_stateless/03173_row_binary_and_native_with_binary_encoded_types.sh index a1e8aa99548..723b11ad620 100755 --- a/tests/queries/0_stateless/03173_row_binary_and_native_with_binary_encoded_types.sh +++ b/tests/queries/0_stateless/03173_row_binary_and_native_with_binary_encoded_types.sh @@ -29,7 +29,11 @@ test "materialize(42.42)::Float64" test "materialize('2020-01-01')::Date" test "materialize('2020-01-01')::Date32" test "materialize('2020-01-01 00:00:00')::DateTime" +test "materialize('2020-01-01 00:00:00')::DateTime('EST')" +test "materialize('2020-01-01 00:00:00')::DateTime('CET')" test "materialize('2020-01-01 00:00:00.000000')::DateTime64(6)" +test "materialize('2020-01-01 00:00:00.000000')::DateTime64(6, 'EST')" +test "materialize('2020-01-01 00:00:00.000000')::DateTime64(6, 'CET')" test "materialize('Hello, World!')" test "materialize('aaaaa')::FixedString(5)" test "materialize('a')::Enum8('a' = 1, 'b' = 2, 'c' = -128)" @@ -61,3 +65,5 @@ test "materialize([[[(0, 0), (10, 0), (10, 10), (0, 10)]], [[(20, 20), (50, 20), test "materialize([map(42, tuple(1, [tuple(2, map(1, 2))]))])" test "materialize(42::UInt32)::Variant(UInt32, String, Tuple(a UInt32, b Array(Map(String, String))))" test "materialize([map(42, tuple(1, [tuple(2, map(1, 2))]))])::Dynamic" +test "materialize([map(42, tuple(1, [tuple(2, map(1, 2))]))])::Dynamic(max_types=10)" +test "materialize([map(42, tuple(1, [tuple(2, map(1, 2))]))])::Dynamic(max_types=255)" From bbafb7adb22d6edaf304ea68f6b50efa71744d84 Mon Sep 17 00:00:00 2001 From: Michael Kolupaev Date: Tue, 25 Jun 2024 01:58:28 +0000 Subject: [PATCH 093/417] Fix 02834_apache_arrow_abort flakiness with MSAN --- tests/queries/0_stateless/02834_apache_arrow_abort.sql | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/tests/queries/0_stateless/02834_apache_arrow_abort.sql b/tests/queries/0_stateless/02834_apache_arrow_abort.sql index 47db46f1e43..bd29e95db9a 100644 --- a/tests/queries/0_stateless/02834_apache_arrow_abort.sql +++ b/tests/queries/0_stateless/02834_apache_arrow_abort.sql @@ -1,4 +1,4 @@ -- Tags: no-fasttest -- This tests depends on internet access, but it does not matter, because it only has to check that there is no abort due to a bug in Apache Arrow library. -INSERT INTO TABLE FUNCTION url('https://clickhouse-public-datasets.s3.amazonaws.com/hits_compatible/hits.parquet') SELECT * FROM url('https://clickhouse-public-datasets.s3.amazonaws.com/hits_compatible/hits.parquet'); -- { serverError CANNOT_WRITE_TO_OSTREAM, RECEIVED_ERROR_FROM_REMOTE_IO_SERVER, POCO_EXCEPTION } +INSERT INTO TABLE FUNCTION url('https://clickhouse-public-datasets.s3.amazonaws.com/hits_compatible/athena_partitioned/hits_9.parquet') SELECT * FROM url('https://clickhouse-public-datasets.s3.amazonaws.com/hits_compatible/athena_partitioned/hits_9.parquet'); -- { serverError CANNOT_WRITE_TO_OSTREAM, RECEIVED_ERROR_FROM_REMOTE_IO_SERVER, POCO_EXCEPTION } From 9c25a1f69b0680136dccbc29eab82fb4f1d3b652 Mon Sep 17 00:00:00 2001 From: Michael Kolupaev Date: Tue, 25 Jun 2024 02:12:08 +0000 Subject: [PATCH 094/417] Exclude some more HTTP codes in ReadWriteBufferFromHTTP::getFileInfo() --- src/IO/ReadWriteBufferFromHTTP.cpp | 10 +++++++++- 1 file changed, 9 insertions(+), 1 deletion(-) diff --git a/src/IO/ReadWriteBufferFromHTTP.cpp b/src/IO/ReadWriteBufferFromHTTP.cpp index 303ffb744b5..f17c19f3cc2 100644 --- a/src/IO/ReadWriteBufferFromHTTP.cpp +++ b/src/IO/ReadWriteBufferFromHTTP.cpp @@ -713,8 +713,16 @@ ReadWriteBufferFromHTTP::HTTPFileInfo ReadWriteBufferFromHTTP::getFileInfo() /// fall back to slow whole-file reads when HEAD is actually supported; that sounds /// like a nightmare to debug.) if (e.getHTTPStatus() >= 400 && e.getHTTPStatus() <= 499 && - e.getHTTPStatus() != Poco::Net::HTTPResponse::HTTP_TOO_MANY_REQUESTS) + e.getHTTPStatus() != Poco::Net::HTTPResponse::HTTP_TOO_MANY_REQUESTS && + e.getHTTPStatus() != Poco::Net::HTTPResponse::HTTP_REQUEST_TIMEOUT && + e.getHTTPStatus() != Poco::Net::HTTPResponse::HTTP_MISDIRECTED_REQUEST) + { + LOG_DEBUG(log, + "HEAD request to '{}'{} failed with HTTP status {}", + initial_uri.toString(), current_uri == initial_uri ? String() : fmt::format(" redirect to '{}'", current_uri.toString()), + e.getHTTPStatus()); return HTTPFileInfo{}; + } throw; } From 8d518151242d4ff385a97d35946b971803df4e16 Mon Sep 17 00:00:00 2001 From: Antonio Andelic Date: Tue, 25 Jun 2024 12:13:44 +0200 Subject: [PATCH 095/417] Fix --- src/Interpreters/InterpreterSelectQuery.cpp | 1 + src/Planner/PlannerJoinTree.cpp | 7 ++--- src/Storages/MergeTree/MergeTreeData.cpp | 2 +- .../test_parallel_replicas_custom_key/test.py | 1 + .../test.py | 28 ++++++++----------- 5 files changed, 18 insertions(+), 21 deletions(-) diff --git a/src/Interpreters/InterpreterSelectQuery.cpp b/src/Interpreters/InterpreterSelectQuery.cpp index 9b51b3a82f6..bc0635affe9 100644 --- a/src/Interpreters/InterpreterSelectQuery.cpp +++ b/src/Interpreters/InterpreterSelectQuery.cpp @@ -600,6 +600,7 @@ InterpreterSelectQuery::InterpreterSelectQuery( } else if ( storage->isMergeTree() && (storage->supportsReplication() || settings.parallel_replicas_for_non_replicated_merge_tree) + && context->getClientInfo().distributed_depth == 0 && context->canUseParallelReplicasCustomKeyForCluster(*context->getClusterForParallelReplicas())) { context->setSetting("prefer_localhost_replica", Field(0)); diff --git a/src/Planner/PlannerJoinTree.cpp b/src/Planner/PlannerJoinTree.cpp index a2c6a478ac4..c85e6cd0cd1 100644 --- a/src/Planner/PlannerJoinTree.cpp +++ b/src/Planner/PlannerJoinTree.cpp @@ -907,11 +907,10 @@ JoinTreeQueryPlan buildQueryPlanForTableExpression(QueryTreeNodePtr table_expres } chassert(reading); - if (query_context->canUseParallelReplicasCustomKey()) + if (query_context->canUseParallelReplicasCustomKey() && query_context->getClientInfo().distributed_depth == 0) { - auto cluster = query_context->getClusterForParallelReplicas(); - if (query_context->canUseParallelReplicasCustomKeyForCluster(*cluster) - && query_context->getClientInfo().distributed_depth == 0) + if (auto cluster = query_context->getClusterForParallelReplicas(); + query_context->canUseParallelReplicasCustomKeyForCluster(*cluster)) { planner_context->getMutableQueryContext()->setSetting("prefer_localhost_replica", Field{0}); auto modified_query_info = select_query_info; diff --git a/src/Storages/MergeTree/MergeTreeData.cpp b/src/Storages/MergeTree/MergeTreeData.cpp index 37f8de60eca..70d6041f0e2 100644 --- a/src/Storages/MergeTree/MergeTreeData.cpp +++ b/src/Storages/MergeTree/MergeTreeData.cpp @@ -7081,7 +7081,7 @@ QueryProcessingStage::Enum MergeTreeData::getQueryProcessingStage( if (query_context->getClientInfo().distributed_depth > 0) return QueryProcessingStage::FetchColumns; - if (!settings.parallel_replicas_for_non_replicated_merge_tree) + if (!supportsReplication() && !settings.parallel_replicas_for_non_replicated_merge_tree) return QueryProcessingStage::Enum::FetchColumns; if (to_stage >= QueryProcessingStage::WithMergeableState diff --git a/tests/integration/test_parallel_replicas_custom_key/test.py b/tests/integration/test_parallel_replicas_custom_key/test.py index cb2c002f237..9a2480a77c3 100644 --- a/tests/integration/test_parallel_replicas_custom_key/test.py +++ b/tests/integration/test_parallel_replicas_custom_key/test.py @@ -91,6 +91,7 @@ def test_parallel_replicas_custom_key_distributed( "max_parallel_replicas": 4, "parallel_replicas_custom_key": custom_key, "parallel_replicas_custom_key_filter_type": filter_type, + "prefer_localhost_replica": 0, }, ) == expected_result diff --git a/tests/integration/test_parallel_replicas_custom_key_failover/test.py b/tests/integration/test_parallel_replicas_custom_key_failover/test.py index 3ba3ce092c3..5c1c8ef71ab 100644 --- a/tests/integration/test_parallel_replicas_custom_key_failover/test.py +++ b/tests/integration/test_parallel_replicas_custom_key_failover/test.py @@ -53,13 +53,11 @@ def create_tables(cluster, table_name): @pytest.mark.parametrize("use_hedged_requests", [1, 0]) @pytest.mark.parametrize("custom_key", ["sipHash64(key)", "key"]) @pytest.mark.parametrize("filter_type", ["default", "range"]) -@pytest.mark.parametrize("prefer_localhost_replica", [0, 1]) def test_parallel_replicas_custom_key_failover( start_cluster, use_hedged_requests, custom_key, filter_type, - prefer_localhost_replica, ): cluster_name = "test_single_shard_multiple_replicas" table = "test_table" @@ -76,7 +74,6 @@ def test_parallel_replicas_custom_key_failover( f"SELECT key, count() FROM cluster('{cluster_name}', currentDatabase(), test_table) GROUP BY key ORDER BY key", settings={ "log_comment": log_comment, - "prefer_localhost_replica": prefer_localhost_replica, "max_parallel_replicas": 4, "parallel_replicas_custom_key": custom_key, "parallel_replicas_custom_key_filter_type": filter_type, @@ -100,20 +97,19 @@ def test_parallel_replicas_custom_key_failover( assert query_id != "" query_id = query_id[:-1] - if prefer_localhost_replica == 0: + assert ( + node1.query( + f"SELECT 'subqueries', count() FROM clusterAllReplicas({cluster_name}, system.query_log) WHERE initial_query_id = '{query_id}' AND type ='QueryFinish' AND query_id != initial_query_id SETTINGS skip_unavailable_shards=1" + ) + == "subqueries\t4\n" + ) + + # With enabled hedged requests, we can't guarantee exact query distribution among nodes + # In case of a replica being slow in terms of responsiveness, hedged connection can change initial replicas choice + if use_hedged_requests == 0: assert ( node1.query( - f"SELECT 'subqueries', count() FROM clusterAllReplicas({cluster_name}, system.query_log) WHERE initial_query_id = '{query_id}' AND type ='QueryFinish' AND query_id != initial_query_id SETTINGS skip_unavailable_shards=1" + f"SELECT h, count() FROM clusterAllReplicas({cluster_name}, system.query_log) WHERE initial_query_id = '{query_id}' AND type ='QueryFinish' GROUP BY hostname() as h ORDER BY h SETTINGS skip_unavailable_shards=1" ) - == "subqueries\t4\n" + == "n1\t3\nn3\t2\n" ) - - # With enabled hedged requests, we can't guarantee exact query distribution among nodes - # In case of a replica being slow in terms of responsiveness, hedged connection can change initial replicas choice - if use_hedged_requests == 0: - assert ( - node1.query( - f"SELECT h, count() FROM clusterAllReplicas({cluster_name}, system.query_log) WHERE initial_query_id = '{query_id}' AND type ='QueryFinish' GROUP BY hostname() as h ORDER BY h SETTINGS skip_unavailable_shards=1" - ) - == "n1\t3\nn3\t2\n" - ) From 1c667723a374854fb6a8249cf3f7b62458f65ba3 Mon Sep 17 00:00:00 2001 From: avogar Date: Tue, 25 Jun 2024 12:12:51 +0000 Subject: [PATCH 096/417] Fix test flakiness --- ...172_dynamic_binary_serialization.reference | 2 +- .../03172_dynamic_binary_serialization.sh | 64 +++++++++---------- 2 files changed, 33 insertions(+), 33 deletions(-) diff --git a/tests/queries/0_stateless/03172_dynamic_binary_serialization.reference b/tests/queries/0_stateless/03172_dynamic_binary_serialization.reference index 1670a7e9634..f5668ed935b 100644 --- a/tests/queries/0_stateless/03172_dynamic_binary_serialization.reference +++ b/tests/queries/0_stateless/03172_dynamic_binary_serialization.reference @@ -15,8 +15,8 @@ 42.42 Float64 2020-01-01 Date 2020-01-01 Date32 -2020-01-01 00:00:00 DateTime(\'CET\') 2020-01-01 00:00:00 DateTime(\'EST\') +2020-01-01 00:00:00 DateTime(\'CET\') 2020-01-01 00:00:00.000000 DateTime64(6, \'EST\') 2020-01-01 00:00:00.000000 DateTime64(6, \'CET\') Hello, World! String diff --git a/tests/queries/0_stateless/03172_dynamic_binary_serialization.sh b/tests/queries/0_stateless/03172_dynamic_binary_serialization.sh index 9e6d78adba5..9b57e5c8718 100755 --- a/tests/queries/0_stateless/03172_dynamic_binary_serialization.sh +++ b/tests/queries/0_stateless/03172_dynamic_binary_serialization.sh @@ -25,38 +25,38 @@ $CLICKHOUSE_CLIENT -q "insert into test select 14, materialize(42.42)::Float64" $CLICKHOUSE_CLIENT -q "insert into test select 15, materialize('2020-01-01')::Date" $CLICKHOUSE_CLIENT -q "insert into test select 16, materialize('2020-01-01')::Date32" $CLICKHOUSE_CLIENT -q "insert into test select 17, materialize('2020-01-01 00:00:00')::DateTime('EST')" -$CLICKHOUSE_CLIENT -q "insert into test select 17, materialize('2020-01-01 00:00:00')::DateTime('CET')" -$CLICKHOUSE_CLIENT -q "insert into test select 18, materialize('2020-01-01 00:00:00.000000')::DateTime64(6, 'EST')" -$CLICKHOUSE_CLIENT -q "insert into test select 18, materialize('2020-01-01 00:00:00.000000')::DateTime64(6, 'CET')" -$CLICKHOUSE_CLIENT -q "insert into test select 19, materialize('Hello, World!')" -$CLICKHOUSE_CLIENT -q "insert into test select 20, materialize('aaaaa')::FixedString(5)" -$CLICKHOUSE_CLIENT -q "insert into test select 21, materialize('a')::Enum8('a' = 1, 'b' = 2, 'c' = -128)" -$CLICKHOUSE_CLIENT -q "insert into test select 22, materialize('a')::Enum16('a' = 1, 'b' = 2, 'c' = -1280)" -$CLICKHOUSE_CLIENT -q "insert into test select 23, materialize(42.42)::Decimal32(3)" -$CLICKHOUSE_CLIENT -q "insert into test select 24, materialize(42.42)::Decimal64(3)" -$CLICKHOUSE_CLIENT -q "insert into test select 25, materialize(42.42)::Decimal128(3)" -$CLICKHOUSE_CLIENT -q "insert into test select 26, materialize(42.42)::Decimal256(3)" -$CLICKHOUSE_CLIENT -q "insert into test select 27, materialize('984ac60f-4d08-4ef1-9c62-d82f343fbc90')::UUID" -$CLICKHOUSE_CLIENT -q "insert into test select 28, materialize([1, 2, 3])::Array(UInt64)" -$CLICKHOUSE_CLIENT -q "insert into test select 29, materialize([[[1], [2]], [[3, 4, 5]]])::Array(Array(Array(UInt64)))" -$CLICKHOUSE_CLIENT -q "insert into test select 30, materialize(tuple(1, 'str', 42.42))::Tuple(UInt32, String, Float32)" -$CLICKHOUSE_CLIENT -q "insert into test select 31, materialize(tuple(1, 'str', 42.42))::Tuple(a UInt32, b String, c Float32)" -$CLICKHOUSE_CLIENT -q "insert into test select 32, materialize(tuple(1, tuple('str', tuple(42.42, -30))))::Tuple(UInt32, Tuple(String, Tuple(Float32, Int8)))" -$CLICKHOUSE_CLIENT -q "insert into test select 33, materialize(tuple(1, tuple('str', tuple(42.42, -30))))::Tuple(a UInt32, b Tuple(c String, d Tuple(e Float32, f Int8)))" -$CLICKHOUSE_CLIENT -q "insert into test select 34, quantileState(0.5)(42::UInt64)" -$CLICKHOUSE_CLIENT -q "insert into test select 35, sumSimpleState(42::UInt64)" -$CLICKHOUSE_CLIENT -q "insert into test select 36, toLowCardinality('Hello, World!')" -$CLICKHOUSE_CLIENT -q "insert into test select 37, materialize(map(1, 'str1', 2, 'str2'))::Map(UInt64, String)" -$CLICKHOUSE_CLIENT -q "insert into test select 38, materialize(map(1, map(1, map(1, 'str1')), 2, map(2, map(2, 'str2'))))::Map(UInt64, Map(UInt64, Map(UInt64, String)))" -$CLICKHOUSE_CLIENT -q "insert into test select 39, materialize('127.0.0.0')::IPv4" -$CLICKHOUSE_CLIENT -q "insert into test select 40, materialize('2001:db8:cafe:1:0:0:0:1')::IPv6" -$CLICKHOUSE_CLIENT -q "insert into test select 41, materialize(true)::Bool" -$CLICKHOUSE_CLIENT -q "insert into test select 42, materialize([tuple(1, 2), tuple(3, 4)])::Nested(a UInt32, b UInt32)" -$CLICKHOUSE_CLIENT -q "insert into test select 43, materialize([(0, 0), (10, 0), (10, 10), (0, 10)])::Ring" -$CLICKHOUSE_CLIENT -q "insert into test select 44, materialize((0, 0))::Point" -$CLICKHOUSE_CLIENT -q "insert into test select 45, materialize([[(20, 20), (50, 20), (50, 50), (20, 50)], [(30, 30), (50, 50), (50, 30)]])::Polygon" -$CLICKHOUSE_CLIENT -q "insert into test select 46, materialize([[[(0, 0), (10, 0), (10, 10), (0, 10)]], [[(20, 20), (50, 20), (50, 50), (20, 50)],[(30, 30), (50, 50), (50, 30)]]])::MultiPolygon" -$CLICKHOUSE_CLIENT -q "insert into test select 47, materialize([map(42, tuple(1, [tuple(2, map(1, 2))]))])" +$CLICKHOUSE_CLIENT -q "insert into test select 18, materialize('2020-01-01 00:00:00')::DateTime('CET')" +$CLICKHOUSE_CLIENT -q "insert into test select 19, materialize('2020-01-01 00:00:00.000000')::DateTime64(6, 'EST')" +$CLICKHOUSE_CLIENT -q "insert into test select 20, materialize('2020-01-01 00:00:00.000000')::DateTime64(6, 'CET')" +$CLICKHOUSE_CLIENT -q "insert into test select 21, materialize('Hello, World!')" +$CLICKHOUSE_CLIENT -q "insert into test select 22, materialize('aaaaa')::FixedString(5)" +$CLICKHOUSE_CLIENT -q "insert into test select 23, materialize('a')::Enum8('a' = 1, 'b' = 2, 'c' = -128)" +$CLICKHOUSE_CLIENT -q "insert into test select 24, materialize('a')::Enum16('a' = 1, 'b' = 2, 'c' = -1280)" +$CLICKHOUSE_CLIENT -q "insert into test select 25, materialize(42.42)::Decimal32(3)" +$CLICKHOUSE_CLIENT -q "insert into test select 26, materialize(42.42)::Decimal64(3)" +$CLICKHOUSE_CLIENT -q "insert into test select 27, materialize(42.42)::Decimal128(3)" +$CLICKHOUSE_CLIENT -q "insert into test select 28, materialize(42.42)::Decimal256(3)" +$CLICKHOUSE_CLIENT -q "insert into test select 29, materialize('984ac60f-4d08-4ef1-9c62-d82f343fbc90')::UUID" +$CLICKHOUSE_CLIENT -q "insert into test select 30, materialize([1, 2, 3])::Array(UInt64)" +$CLICKHOUSE_CLIENT -q "insert into test select 31, materialize([[[1], [2]], [[3, 4, 5]]])::Array(Array(Array(UInt64)))" +$CLICKHOUSE_CLIENT -q "insert into test select 32, materialize(tuple(1, 'str', 42.42))::Tuple(UInt32, String, Float32)" +$CLICKHOUSE_CLIENT -q "insert into test select 33, materialize(tuple(1, 'str', 42.42))::Tuple(a UInt32, b String, c Float32)" +$CLICKHOUSE_CLIENT -q "insert into test select 34, materialize(tuple(1, tuple('str', tuple(42.42, -30))))::Tuple(UInt32, Tuple(String, Tuple(Float32, Int8)))" +$CLICKHOUSE_CLIENT -q "insert into test select 35, materialize(tuple(1, tuple('str', tuple(42.42, -30))))::Tuple(a UInt32, b Tuple(c String, d Tuple(e Float32, f Int8)))" +$CLICKHOUSE_CLIENT -q "insert into test select 36, quantileState(0.5)(42::UInt64)" +$CLICKHOUSE_CLIENT -q "insert into test select 37, sumSimpleState(42::UInt64)" +$CLICKHOUSE_CLIENT -q "insert into test select 38, toLowCardinality('Hello, World!')" +$CLICKHOUSE_CLIENT -q "insert into test select 39, materialize(map(1, 'str1', 2, 'str2'))::Map(UInt64, String)" +$CLICKHOUSE_CLIENT -q "insert into test select 40, materialize(map(1, map(1, map(1, 'str1')), 2, map(2, map(2, 'str2'))))::Map(UInt64, Map(UInt64, Map(UInt64, String)))" +$CLICKHOUSE_CLIENT -q "insert into test select 41, materialize('127.0.0.0')::IPv4" +$CLICKHOUSE_CLIENT -q "insert into test select 42, materialize('2001:db8:cafe:1:0:0:0:1')::IPv6" +$CLICKHOUSE_CLIENT -q "insert into test select 43, materialize(true)::Bool" +$CLICKHOUSE_CLIENT -q "insert into test select 44, materialize([tuple(1, 2), tuple(3, 4)])::Nested(a UInt32, b UInt32)" +$CLICKHOUSE_CLIENT -q "insert into test select 45, materialize([(0, 0), (10, 0), (10, 10), (0, 10)])::Ring" +$CLICKHOUSE_CLIENT -q "insert into test select 46, materialize((0, 0))::Point" +$CLICKHOUSE_CLIENT -q "insert into test select 47, materialize([[(20, 20), (50, 20), (50, 50), (20, 50)], [(30, 30), (50, 50), (50, 30)]])::Polygon" +$CLICKHOUSE_CLIENT -q "insert into test select 48, materialize([[[(0, 0), (10, 0), (10, 10), (0, 10)]], [[(20, 20), (50, 20), (50, 50), (20, 50)],[(30, 30), (50, 50), (50, 30)]]])::MultiPolygon" +$CLICKHOUSE_CLIENT -q "insert into test select 49, materialize([map(42, tuple(1, [tuple(2, map(1, 2))]))])" $CLICKHOUSE_CLIENT -q "select * from test format RowBinary" | $CLICKHOUSE_LOCAL --allow_experimental_dynamic_type=1 --input-format RowBinary --structure 'id UInt64, d Dynamic(max_types=255)' -q "select d, dynamicType(d) from table order by id" $CLICKHOUSE_CLIENT -q "drop table test" From 1525dff4c3ba7c5c31d677864014ae75bdee8102 Mon Sep 17 00:00:00 2001 From: Kruglov Pavel <48961922+Avogar@users.noreply.github.com> Date: Tue, 25 Jun 2024 18:01:42 +0200 Subject: [PATCH 097/417] Fix links in docs --- docs/en/interfaces/formats.md | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/docs/en/interfaces/formats.md b/docs/en/interfaces/formats.md index dcf87c13786..d3eb37b536f 100644 --- a/docs/en/interfaces/formats.md +++ b/docs/en/interfaces/formats.md @@ -1536,9 +1536,9 @@ Otherwise, the first row will be skipped. If setting [input_format_with_types_use_header](/docs/en/operations/settings/settings-formats.md/#input_format_with_types_use_header) is set to 1, the types from input data will be compared with the types of the corresponding columns from the table. Otherwise, the second row will be skipped. If setting [output_format_binary_encode_types_in_binary_format](/docs/en/operations/settings/settings-formats.md/#output_format_binary_encode_types_in_binary_format) is set to 1, -the types in header will be written using [binary encoding](../../sql-reference/data-types/data-types-binary-encoding.md) instead of strings with type names in RowBinaryWithNamesAndTypes output format. +the types in header will be written using [binary encoding](/docs/en/sql-reference/data-types/data-types-binary-encoding.md) instead of strings with type names in RowBinaryWithNamesAndTypes output format. If setting [input_format_binary_encode_types_in_binary_format](/docs/en/operations/settings/settings-formats.md/#input_format_binary_encode_types_in_binary_format) is set to 1, -the types in header will be read using [binary encoding](../../sql-reference/data-types/data-types-binary-encoding.md) instead of strings with type names in RowBinaryWithNamesAndTypes input format. +the types in header will be read using [binary encoding](/docs/en/sql-reference/data-types/data-types-binary-encoding.md) instead of strings with type names in RowBinaryWithNamesAndTypes input format. ::: ## RowBinaryWithDefaults {#rowbinarywithdefaults} From 1b2a0036da6f20a74f5ad6a700559f0573c77136 Mon Sep 17 00:00:00 2001 From: divanik Date: Tue, 25 Jun 2024 16:29:57 +0000 Subject: [PATCH 098/417] Fixed tests --- .../02980_s3_plain_DROP_TABLE_ReplicatedMergeTree.reference | 4 ++-- .../02980_s3_plain_DROP_TABLE_ReplicatedMergeTree.sh | 2 -- 2 files changed, 2 insertions(+), 4 deletions(-) diff --git a/tests/queries/0_stateless/02980_s3_plain_DROP_TABLE_ReplicatedMergeTree.reference b/tests/queries/0_stateless/02980_s3_plain_DROP_TABLE_ReplicatedMergeTree.reference index 21b38a94cee..a2dd196083e 100644 --- a/tests/queries/0_stateless/02980_s3_plain_DROP_TABLE_ReplicatedMergeTree.reference +++ b/tests/queries/0_stateless/02980_s3_plain_DROP_TABLE_ReplicatedMergeTree.reference @@ -3,7 +3,7 @@ data after ATTACH 1 Files before DETACH TABLE all_X_X_X -backups/ordinary_default/data/ordinary_default/data_read/all_X_X_X: +/backups/ordinary_default/data/ordinary_default/data_read/all_X_X_X: checksums.txt columns.txt count.txt @@ -17,7 +17,7 @@ serialization.json Files after DETACH TABLE all_X_X_X -backups/ordinary_default/data/ordinary_default/data_read/all_X_X_X: +/backups/ordinary_default/data/ordinary_default/data_read/all_X_X_X: checksums.txt columns.txt count.txt diff --git a/tests/queries/0_stateless/02980_s3_plain_DROP_TABLE_ReplicatedMergeTree.sh b/tests/queries/0_stateless/02980_s3_plain_DROP_TABLE_ReplicatedMergeTree.sh index 7725688d225..eec05c81344 100755 --- a/tests/queries/0_stateless/02980_s3_plain_DROP_TABLE_ReplicatedMergeTree.sh +++ b/tests/queries/0_stateless/02980_s3_plain_DROP_TABLE_ReplicatedMergeTree.sh @@ -55,7 +55,6 @@ path=${path%/} echo "Files before DETACH TABLE" # sed to match any part, since in case of fault injection part name may not be all_0_0_0 but all_1_1_0 -echo "Path $path" clickhouse-disks -C "$config" --disk s3_plain_disk --query "list --recursive $path" | tail -n+2 | sed 's/all_[^_]*_[^_]*_0/all_X_X_X/g' $CLICKHOUSE_CLIENT -nm -q " @@ -63,7 +62,6 @@ $CLICKHOUSE_CLIENT -nm -q " detach table data_write; " echo "Files after DETACH TABLE" -echo "Path $path" clickhouse-disks -C "$config" --disk s3_plain_disk --query "list --recursive $path" | tail -n+2 | sed 's/all_[^_]*_[^_]*_0/all_X_X_X/g' # metadata file is left From 5da4b69d5745d7187e6a21a14793d7355590423b Mon Sep 17 00:00:00 2001 From: divanik Date: Tue, 25 Jun 2024 17:20:42 +0000 Subject: [PATCH 099/417] Tried to fix tests --- programs/disks/DisksClient.cpp | 16 +++++++++------- programs/disks/DisksClient.h | 5 +---- 2 files changed, 10 insertions(+), 11 deletions(-) diff --git a/programs/disks/DisksClient.cpp b/programs/disks/DisksClient.cpp index e38f7ec99b8..379c87e4f2f 100644 --- a/programs/disks/DisksClient.cpp +++ b/programs/disks/DisksClient.cpp @@ -29,15 +29,17 @@ DiskWithPath::DiskWithPath(DiskPtr disk_, std::optional path_) : disk(di path = String{"/"}; } - if (!disk->isDirectory(normalizePathAndGetAsRelative(path))) + String relative_path = normalizePathAndGetAsRelative(path); + if (disk->isDirectory(relative_path) || (relative_path.empty() && (disk->isDirectory("/")))) { - throw Exception( - ErrorCodes::BAD_ARGUMENTS, - "Initializing path {} (normalized path: {}) at disk {} is not a directory", - path, - normalizePathAndGetAsRelative(path), - disk->getName()); + return; } + throw Exception( + ErrorCodes::BAD_ARGUMENTS, + "Initializing path {} (normalized path: {}) at disk {} is not a directory", + path, + relative_path, + disk->getName()); } std::vector DiskWithPath::listAllFilesByPath(const String & any_path) const diff --git a/programs/disks/DisksClient.h b/programs/disks/DisksClient.h index ab99d2f6590..3320c5f7cef 100644 --- a/programs/disks/DisksClient.h +++ b/programs/disks/DisksClient.h @@ -32,10 +32,7 @@ public: String getCurrentPath() const { return path; } - bool isDirectory(const String & any_path) const - { - return disk->isDirectory(getRelativeFromRoot(any_path)) || disk->isDirectory(getAbsolutePath(any_path)); - } + bool isDirectory(const String & any_path) const { return disk->isDirectory(getRelativeFromRoot(any_path)); } std::vector listAllFilesByPath(const String & any_path) const; From e8e1cd79e0325a6ab1a73b59fe9ab830bb8561ac Mon Sep 17 00:00:00 2001 From: divanik Date: Wed, 26 Jun 2024 13:17:39 +0000 Subject: [PATCH 100/417] Corrected integration tests --- .../test_backup_restore_s3/test.py | 273 +++--------------- tests/integration/test_disk_types/test.py | 13 +- .../test_endpoint_macro_substitution/test.py | 7 +- 3 files changed, 45 insertions(+), 248 deletions(-) diff --git a/tests/integration/test_backup_restore_s3/test.py b/tests/integration/test_backup_restore_s3/test.py index bd600a06c7f..05424887736 100644 --- a/tests/integration/test_backup_restore_s3/test.py +++ b/tests/integration/test_backup_restore_s3/test.py @@ -3,11 +3,8 @@ import pytest from helpers.cluster import ClickHouseCluster from helpers.test_tools import TSV import uuid -import os -CONFIG_DIR = os.path.join(os.path.dirname(os.path.realpath(__file__)), "configs") - cluster = ClickHouseCluster(__file__) node = cluster.add_instance( "node", @@ -23,127 +20,13 @@ node = cluster.add_instance( ], with_minio=True, with_zookeeper=True, - stay_alive=True, ) -def setup_minio_users(): - # create 2 extra users with restricted access - # miniorestricted1 - full access to bucket 'root', no access to other buckets - # miniorestricted2 - full access to bucket 'root2', no access to other buckets - # storage policy 'policy_s3_restricted' defines a policy for storing files inside bucket 'root' using 'miniorestricted1' user - for user, bucket in [("miniorestricted1", "root"), ("miniorestricted2", "root2")]: - print( - cluster.exec_in_container( - cluster.minio_docker_id, - [ - "mc", - "alias", - "set", - "root", - "http://minio1:9001", - "minio", - "minio123", - ], - ) - ) - policy = f""" -{{ - "Version": "2012-10-17", - "Statement": [ - {{ - "Effect": "Allow", - "Principal": {{ - "AWS": [ - "*" - ] - }}, - "Action": [ - "s3:GetBucketLocation", - "s3:ListBucket", - "s3:ListBucketMultipartUploads" - ], - "Resource": [ - "arn:aws:s3:::{bucket}" - ] - }}, - {{ - "Effect": "Allow", - "Principal": {{ - "AWS": [ - "*" - ] - }}, - "Action": [ - "s3:AbortMultipartUpload", - "s3:DeleteObject", - "s3:GetObject", - "s3:ListMultipartUploadParts", - "s3:PutObject" - ], - "Resource": [ - "arn:aws:s3:::{bucket}/*" - ] - }} - ] -}}""" - - cluster.exec_in_container( - cluster.minio_docker_id, - ["bash", "-c", f"cat >/tmp/{bucket}_policy.json < Date: Wed, 26 Jun 2024 13:25:34 +0000 Subject: [PATCH 101/417] Revert some tests --- tests/integration/test_multiple_disks/test.py | 16 +++++++++------- tests/integration/test_store_cleanup/test.py | 5 +---- 2 files changed, 10 insertions(+), 11 deletions(-) diff --git a/tests/integration/test_multiple_disks/test.py b/tests/integration/test_multiple_disks/test.py index 32013dd0757..fdd81284b2a 100644 --- a/tests/integration/test_multiple_disks/test.py +++ b/tests/integration/test_multiple_disks/test.py @@ -83,7 +83,6 @@ def test_system_tables(start_cluster): "path": "/external/", "keep_free_space": "0", }, - {"name": "local", "path": "/", "keep_free_space": "0"}, ] click_disk_data = json.loads( @@ -1784,12 +1783,15 @@ def test_move_across_policies_does_not_work(start_cluster): except QueryRuntimeException: """All parts of partition 'all' are already on disk 'jbod2'.""" - # works when attach - node1.query( - """ALTER TABLE {name}2 ATTACH PARTITION tuple() FROM {name}""".format( - name=name + with pytest.raises( + QueryRuntimeException, + match=".*because disk does not belong to storage policy.*", + ): + node1.query( + """ALTER TABLE {name}2 ATTACH PARTITION tuple() FROM {name}""".format( + name=name + ) ) - ) with pytest.raises( QueryRuntimeException, @@ -1812,7 +1814,7 @@ def test_move_across_policies_does_not_work(start_cluster): ) assert node1.query( - """SELECT * FROM {name}2""".format(name=name) + """SELECT * FROM {name}""".format(name=name) ).splitlines() == ["1"] finally: diff --git a/tests/integration/test_store_cleanup/test.py b/tests/integration/test_store_cleanup/test.py index aebfde694b3..6c5a20a758a 100644 --- a/tests/integration/test_store_cleanup/test.py +++ b/tests/integration/test_store_cleanup/test.py @@ -1,4 +1,3 @@ -from time import sleep import pytest from helpers.cluster import ClickHouseCluster @@ -154,9 +153,7 @@ def test_store_cleanup(started_cluster): "directories from store", timeout=90, look_behind_lines=1000000 ) node1.wait_for_log_line( - "Nothing to clean up from store/ on disk default", - timeout=90, - look_behind_lines=1000000, + "Nothing to clean up from store/", timeout=90, look_behind_lines=1000000 ) store = node1.exec_in_container(["ls", f"{path_to_data}/store"]) From cef503cb4c6b27cc580d948e8b0e3688997e99ff Mon Sep 17 00:00:00 2001 From: divanik Date: Wed, 26 Jun 2024 13:47:15 +0000 Subject: [PATCH 102/417] Revert test --- .../test_backup_restore_s3/test.py | 248 ++++++++++++++++-- 1 file changed, 222 insertions(+), 26 deletions(-) diff --git a/tests/integration/test_backup_restore_s3/test.py b/tests/integration/test_backup_restore_s3/test.py index 05424887736..d53335000a6 100644 --- a/tests/integration/test_backup_restore_s3/test.py +++ b/tests/integration/test_backup_restore_s3/test.py @@ -3,8 +3,11 @@ import pytest from helpers.cluster import ClickHouseCluster from helpers.test_tools import TSV import uuid +import os +CONFIG_DIR = os.path.join(os.path.dirname(os.path.realpath(__file__)), "configs") + cluster = ClickHouseCluster(__file__) node = cluster.add_instance( "node", @@ -20,13 +23,127 @@ node = cluster.add_instance( ], with_minio=True, with_zookeeper=True, + stay_alive=True, ) +def setup_minio_users(): + # create 2 extra users with restricted access + # miniorestricted1 - full access to bucket 'root', no access to other buckets + # miniorestricted2 - full access to bucket 'root2', no access to other buckets + # storage policy 'policy_s3_restricted' defines a policy for storing files inside bucket 'root' using 'miniorestricted1' user + for user, bucket in [("miniorestricted1", "root"), ("miniorestricted2", "root2")]: + print( + cluster.exec_in_container( + cluster.minio_docker_id, + [ + "mc", + "alias", + "set", + "root", + "http://minio1:9001", + "minio", + "minio123", + ], + ) + ) + policy = f""" +{{ + "Version": "2012-10-17", + "Statement": [ + {{ + "Effect": "Allow", + "Principal": {{ + "AWS": [ + "*" + ] + }}, + "Action": [ + "s3:GetBucketLocation", + "s3:ListBucket", + "s3:ListBucketMultipartUploads" + ], + "Resource": [ + "arn:aws:s3:::{bucket}" + ] + }}, + {{ + "Effect": "Allow", + "Principal": {{ + "AWS": [ + "*" + ] + }}, + "Action": [ + "s3:AbortMultipartUpload", + "s3:DeleteObject", + "s3:GetObject", + "s3:ListMultipartUploadParts", + "s3:PutObject" + ], + "Resource": [ + "arn:aws:s3:::{bucket}/*" + ] + }} + ] +}}""" + + cluster.exec_in_container( + cluster.minio_docker_id, + ["bash", "-c", f"cat >/tmp/{bucket}_policy.json < Date: Wed, 26 Jun 2024 15:59:15 +0000 Subject: [PATCH 103/417] fix some issues --- .../operations/utilities/clickhouse-disks.md | 2 +- programs/disks/CMakeLists.txt | 1 + programs/disks/CommandTouch.cpp | 34 +++++++++++++++++++ programs/disks/CommandWrite.cpp | 2 +- programs/disks/DisksApp.cpp | 4 +++ programs/disks/DisksApp.h | 2 ++ programs/disks/DisksClient.h | 5 ++- programs/disks/ICommand.h | 1 + 8 files changed, 48 insertions(+), 3 deletions(-) create mode 100644 programs/disks/CommandTouch.cpp diff --git a/docs/en/operations/utilities/clickhouse-disks.md b/docs/en/operations/utilities/clickhouse-disks.md index aeca49c0e1e..e22bc06b641 100644 --- a/docs/en/operations/utilities/clickhouse-disks.md +++ b/docs/en/operations/utilities/clickhouse-disks.md @@ -56,4 +56,4 @@ In these documentation file all mandatory positional arguments are referred as ` * `switch-disk [--path path] ` Switch to disk `disk` on path `path` (if `path` is not specified default value is a previous path on disk `disk`). * `write (w) [--path-from path] `. - Write a file from `path` (`stdin` if not supplied) to `path-to`. + Write a file from `path` (`stdin` if `path` is not supplied, input must finish by Ctrl+D) to `path-to`. diff --git a/programs/disks/CMakeLists.txt b/programs/disks/CMakeLists.txt index 40f9cf3401c..7e8afe084fb 100644 --- a/programs/disks/CMakeLists.txt +++ b/programs/disks/CMakeLists.txt @@ -14,6 +14,7 @@ set (CLICKHOUSE_DISKS_SOURCES CommandSwitchDisk.cpp CommandWrite.cpp CommandHelp.cpp + CommandTouch.cpp CommandGetCurrentDiskAndPath.cpp) if (CLICKHOUSE_CLOUD) diff --git a/programs/disks/CommandTouch.cpp b/programs/disks/CommandTouch.cpp new file mode 100644 index 00000000000..c0bdb64cf9e --- /dev/null +++ b/programs/disks/CommandTouch.cpp @@ -0,0 +1,34 @@ +#include +#include +#include "DisksApp.h" +#include "DisksClient.h" +#include "ICommand.h" + +namespace DB +{ + +class CommandTouch final : public ICommand +{ +public: + explicit CommandTouch() : ICommand() + { + command_name = "touch"; + description = "Create a file by path"; + options_description.add_options()("path", po::value(), "the path of listing (mandatory, positional)"); + positional_options_description.add("path", 1); + } + + void executeImpl(const CommandLineOptions & options, DisksClient & client) override + { + auto disk = client.getCurrentDiskWithPath(); + String path = getValueFromCommandLineOptionsThrow(options, "path"); + + disk.getDisk()->createFile(disk.getRelativeFromRoot(path)); + } +}; + +CommandPtr makeCommandTouch() +{ + return std::make_shared(); +} +} diff --git a/programs/disks/CommandWrite.cpp b/programs/disks/CommandWrite.cpp index 433ebb3d5cf..ffe27f37c0e 100644 --- a/programs/disks/CommandWrite.cpp +++ b/programs/disks/CommandWrite.cpp @@ -16,7 +16,7 @@ public: { command_name = "write"; description = "Write a file from `FROM_PATH` to `TO_PATH`"; - options_description.add_options()("path-from", po::value(), "file from which we are reading, defaults to `stdin`")( + options_description.add_options()("path-from", po::value(), "file from which we are reading, defaults to `stdin` (input from `stdin` is finished by Ctrl+D)")( "path-to", po::value(), "file to which we are writing (mandatory, positional)"); positional_options_description.add("path-to", 1); } diff --git a/programs/disks/DisksApp.cpp b/programs/disks/DisksApp.cpp index 9ef051a2ece..392fca8e035 100644 --- a/programs/disks/DisksApp.cpp +++ b/programs/disks/DisksApp.cpp @@ -89,6 +89,7 @@ std::vector DisksApp::getCommandsToComplete(const String & command_prefi } if (!answer.empty()) { + std::sort(answer.begin(), answer.end()); return answer; } for (const auto & [word, _] : aliases) @@ -100,6 +101,7 @@ std::vector DisksApp::getCommandsToComplete(const String & command_prefi } if (!answer.empty()) { + std::sort(answer.begin(), answer.end()); return answer; } return {command_prefix}; @@ -179,6 +181,7 @@ std::vector DisksApp::getCompletions(const String & prefix) const } if (!answer.empty()) { + std::sort(answer.begin(), answer.end()); return answer; } else @@ -292,6 +295,7 @@ void DisksApp::addOptions() command_descriptions.emplace("mkdir", makeCommandMkDir()); command_descriptions.emplace("switch-disk", makeCommandSwitchDisk()); command_descriptions.emplace("current_disk_with_path", makeCommandGetCurrentDiskAndPath()); + command_descriptions.emplace("touch", makeCommandTouch()); command_descriptions.emplace("help", makeCommandHelp(*this)); #ifdef CLICKHOUSE_CLOUD command_descriptions.emplace("packed-io", makeCommandPackedIO()); diff --git a/programs/disks/DisksApp.h b/programs/disks/DisksApp.h index 1ecd9944fb8..f8167884c62 100644 --- a/programs/disks/DisksApp.h +++ b/programs/disks/DisksApp.h @@ -84,8 +84,10 @@ private: {"list_disks", "list-disks"}, {"ln", "link"}, {"rm", "remove"}, + {"cat", "read"}, {"r", "read"}, {"w", "write"}, + {"create", "touch"}, {"delete", "remove"}, {"ls-disks", "list-disks"}, {"ls_disks", "list-disks"}, diff --git a/programs/disks/DisksClient.h b/programs/disks/DisksClient.h index 3320c5f7cef..8a55d22af93 100644 --- a/programs/disks/DisksClient.h +++ b/programs/disks/DisksClient.h @@ -32,7 +32,10 @@ public: String getCurrentPath() const { return path; } - bool isDirectory(const String & any_path) const { return disk->isDirectory(getRelativeFromRoot(any_path)); } + bool isDirectory(const String & any_path) const + { + return disk->isDirectory(getRelativeFromRoot(any_path)) || (getRelativeFromRoot(any_path).empty() && (disk->isDirectory("/"))); + } std::vector listAllFilesByPath(const String & any_path) const; diff --git a/programs/disks/ICommand.h b/programs/disks/ICommand.h index 2b409d4ade6..4b0ec731966 100644 --- a/programs/disks/ICommand.h +++ b/programs/disks/ICommand.h @@ -123,6 +123,7 @@ DB::CommandPtr makeCommandMkDir(); DB::CommandPtr makeCommandSwitchDisk(); DB::CommandPtr makeCommandGetCurrentDiskAndPath(); DB::CommandPtr makeCommandHelp(const DisksApp & disks_app); +DB::CommandPtr makeCommandTouch(); #ifdef CLICKHOUSE_CLOUD DB::CommandPtr makeCommandPackedIO(); #endif From 4377a1f4edc6642583110c1597d0c72e30613aa2 Mon Sep 17 00:00:00 2001 From: divanik Date: Wed, 26 Jun 2024 16:12:55 +0000 Subject: [PATCH 104/417] Take test from master --- tests/integration/test_multiple_disks/test.py | 15 ++++++--------- 1 file changed, 6 insertions(+), 9 deletions(-) diff --git a/tests/integration/test_multiple_disks/test.py b/tests/integration/test_multiple_disks/test.py index fdd81284b2a..e97ffeb4cc3 100644 --- a/tests/integration/test_multiple_disks/test.py +++ b/tests/integration/test_multiple_disks/test.py @@ -1783,15 +1783,12 @@ def test_move_across_policies_does_not_work(start_cluster): except QueryRuntimeException: """All parts of partition 'all' are already on disk 'jbod2'.""" - with pytest.raises( - QueryRuntimeException, - match=".*because disk does not belong to storage policy.*", - ): - node1.query( - """ALTER TABLE {name}2 ATTACH PARTITION tuple() FROM {name}""".format( - name=name - ) + # works when attach + node1.query( + """ALTER TABLE {name}2 ATTACH PARTITION tuple() FROM {name}""".format( + name=name ) + ) with pytest.raises( QueryRuntimeException, @@ -1814,7 +1811,7 @@ def test_move_across_policies_does_not_work(start_cluster): ) assert node1.query( - """SELECT * FROM {name}""".format(name=name) + """SELECT * FROM {name}2""".format(name=name) ).splitlines() == ["1"] finally: From a67d468eaa33bba0a82073f37a0d17eefd5a3e86 Mon Sep 17 00:00:00 2001 From: divanik Date: Wed, 26 Jun 2024 17:25:43 +0000 Subject: [PATCH 105/417] Forbid unregistered options --- programs/disks/ICommand.cpp | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/programs/disks/ICommand.cpp b/programs/disks/ICommand.cpp index 0c149a8f9df..f622bcad3c6 100644 --- a/programs/disks/ICommand.cpp +++ b/programs/disks/ICommand.cpp @@ -14,7 +14,7 @@ CommandLineOptions ICommand::processCommandLineArguments(const Strings & command { CommandLineOptions options; auto parser = po::command_line_parser(commands); - parser.options(options_description).positional(positional_options_description).allow_unregistered(); + parser.options(options_description).positional(positional_options_description); po::parsed_options parsed = parser.run(); po::store(parsed, options); From cbb850517f71607a9b7f5b8d645b9ac8c90b387b Mon Sep 17 00:00:00 2001 From: avogar Date: Thu, 27 Jun 2024 15:18:14 +0000 Subject: [PATCH 106/417] Remove support for depricated Object type in the encoding --- .../data-types/data-types-binary-encoding.md | 17 +++++---- src/DataTypes/DataTypesBinaryEncoding.cpp | 36 ++++++------------- src/DataTypes/DataTypesBinaryEncoding.h | 17 +++++---- .../gtest_data_types_binary_encoding.cpp | 1 - 4 files changed, 26 insertions(+), 45 deletions(-) diff --git a/docs/en/sql-reference/data-types/data-types-binary-encoding.md b/docs/en/sql-reference/data-types/data-types-binary-encoding.md index ba1a4fa44c5..d7eddf848d6 100644 --- a/docs/en/sql-reference/data-types/data-types-binary-encoding.md +++ b/docs/en/sql-reference/data-types/data-types-binary-encoding.md @@ -49,15 +49,14 @@ sidebar_label: Data types binary encoding specification. | `AggregateFunction(function_name(param_1, ..., param_N), arg_T1, ..., arg_TN)` | `0x25......` (see [aggregate function parameter binary encoding](#aggregate-function-parameter-binary-encoding)) | | `LowCardinality(T)` | `0x26` | | `Map(K, V)` | `0x27` | -| `Object('schema_format')` | `0x28` | -| `IPv4` | `0x29` | -| `IPv6` | `0x2A` | -| `Variant(T1, ..., TN)` | `0x2B...` | -| `Dynamic(max_types=N)` | `0x2C` | -| `Custom type` (`Ring`, `Polygon`, etc) | `0x2D` | -| `Bool` | `0x2E` | -| `SimpleAggregateFunction(function_name(param_1, ..., param_N), arg_T1, ..., arg_TN)` | `0x2F......` (see [aggregate function parameter binary encoding](#aggregate-function-parameter-binary-encoding)) | -| `Nested(name1 T1, ..., nameN TN)` | `0x30...` | +| `IPv4` | `0x28` | +| `IPv6` | `0x29` | +| `Variant(T1, ..., TN)` | `0x2A...` | +| `Dynamic(max_types=N)` | `0x2B` | +| `Custom type` (`Ring`, `Polygon`, etc) | `0x2C` | +| `Bool` | `0x2D` | +| `SimpleAggregateFunction(function_name(param_1, ..., param_N), arg_T1, ..., arg_TN)` | `0x2E......` (see [aggregate function parameter binary encoding](#aggregate-function-parameter-binary-encoding)) | +| `Nested(name1 T1, ..., nameN TN)` | `0x2F...` | ### Interval kind binary encoding diff --git a/src/DataTypes/DataTypesBinaryEncoding.cpp b/src/DataTypes/DataTypesBinaryEncoding.cpp index 02a58897529..bd994e313ba 100644 --- a/src/DataTypes/DataTypesBinaryEncoding.cpp +++ b/src/DataTypes/DataTypesBinaryEncoding.cpp @@ -9,7 +9,6 @@ #include #include #include -#include #include #include #include @@ -87,15 +86,14 @@ enum class BinaryTypeIndex : uint8_t AggregateFunction = 0x25, LowCardinality = 0x26, Map = 0x27, - Object = 0x28, - IPv4 = 0x29, - IPv6 = 0x2A, - Variant = 0x2B, - Dynamic = 0x2C, - Custom = 0x2D, - Bool = 0x2E, - SimpleAggregateFunction = 0x2F, - Nested = 0x30, + IPv4 = 0x28, + IPv6 = 0x29, + Variant = 0x2A, + Dynamic = 0x2B, + Custom = 0x2C, + Bool = 0x2D, + SimpleAggregateFunction = 0x2E, + Nested = 0x2F, }; BinaryTypeIndex getBinaryTypeIndex(const DataTypePtr & type) @@ -205,7 +203,8 @@ BinaryTypeIndex getBinaryTypeIndex(const DataTypePtr & type) case TypeIndex::Map: return BinaryTypeIndex::Map; case TypeIndex::Object: - return BinaryTypeIndex::Object; + /// Object type will be deprecated and replaced by new implementation. No need to support it here. + throw Exception(ErrorCodes::UNSUPPORTED_METHOD, "Binary encoding of type Object is not supported"); case TypeIndex::IPv4: return BinaryTypeIndex::IPv4; case TypeIndex::IPv6: @@ -433,13 +432,6 @@ void encodeDataType(const DataTypePtr & type, WriteBuffer & buf) encodeDataType(map_type.getValueType(), buf); break; } - case BinaryTypeIndex::Object: - { - const auto & object_deprecated_type = assert_cast(*type); - writeBinary(object_deprecated_type.hasNullableSubcolumns(), buf); - writeStringBinary(object_deprecated_type.getSchemaFormat(), buf); - break; - } case BinaryTypeIndex::Variant: { const auto & variant_type = assert_cast(*type); @@ -644,14 +636,6 @@ DataTypePtr decodeDataType(ReadBuffer & buf) auto value_type = decodeDataType(buf); return std::make_shared(key_type, value_type); } - case BinaryTypeIndex::Object: - { - bool has_nullable_subcolumns; - readBinary(has_nullable_subcolumns, buf); - String schema_format; - readStringBinary(schema_format, buf); - return std::make_shared(schema_format, has_nullable_subcolumns); - } case BinaryTypeIndex::IPv4: return std::make_shared(); case BinaryTypeIndex::IPv6: diff --git a/src/DataTypes/DataTypesBinaryEncoding.h b/src/DataTypes/DataTypesBinaryEncoding.h index d735565f636..d02e7f85942 100644 --- a/src/DataTypes/DataTypesBinaryEncoding.h +++ b/src/DataTypes/DataTypesBinaryEncoding.h @@ -51,15 +51,14 @@ Binary encoding for ClickHouse data types: | AggregateFunction(function_name(param_1, ..., param_N), arg_T1, ..., arg_TN) | 0x25...... | | LowCardinality(T) | 0x26 | | Map(K, V) | 0x27 | -| Object('schema_format') | 0x28 | -| IPv4 | 0x29 | -| IPv6 | 0x2A | -| Variant(T1, ..., TN) | 0x2B... | -| Dynamic(max_types=N) | 0x2C | -| Custom type (Ring, Polygon, etc) | 0x2D | -| Bool | 0x2E | -| SimpleAggregateFunction(function_name(param_1, ..., param_N), arg_T1, ..., arg_TN) | 0x2F...... | -| Nested(name1 T1, ..., nameN TN) | 0x30... | +| IPv4 | 0x28 | +| IPv6 | 0x29 | +| Variant(T1, ..., TN) | 0x2A... | +| Dynamic(max_types=N) | 0x2B | +| Custom type (Ring, Polygon, etc) | 0x2C | +| Bool | 0x2D | +| SimpleAggregateFunction(function_name(param_1, ..., param_N), arg_T1, ..., arg_TN) | 0x2E...... | +| Nested(name1 T1, ..., nameN TN) | 0x2F... | |------------------------------------------------------------------------------------|------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------| Interval kind binary encoding: diff --git a/src/DataTypes/tests/gtest_data_types_binary_encoding.cpp b/src/DataTypes/tests/gtest_data_types_binary_encoding.cpp index 6dbb9451922..4d0bfc67183 100644 --- a/src/DataTypes/tests/gtest_data_types_binary_encoding.cpp +++ b/src/DataTypes/tests/gtest_data_types_binary_encoding.cpp @@ -6,7 +6,6 @@ #include #include #include -#include #include #include #include From 1402a00c8c741ceb2daa798e533ab95600c0e7a2 Mon Sep 17 00:00:00 2001 From: Yakov Olkhovskiy Date: Thu, 27 Jun 2024 23:15:15 +0000 Subject: [PATCH 107/417] optimize deduplicate for tables with projections is not supported --- src/Storages/MergeTree/MergeTask.cpp | 7 +------ src/Storages/StorageMergeTree.cpp | 6 ++++++ src/Storages/StorageReplicatedMergeTree.cpp | 6 ++++++ .../0_stateless/03174_projection_deduplicate.reference | 1 + tests/queries/0_stateless/03174_projection_deduplicate.sql | 2 +- 5 files changed, 15 insertions(+), 7 deletions(-) diff --git a/src/Storages/MergeTree/MergeTask.cpp b/src/Storages/MergeTree/MergeTask.cpp index 9fe16ad8650..56bd1181fef 100644 --- a/src/Storages/MergeTree/MergeTask.cpp +++ b/src/Storages/MergeTree/MergeTask.cpp @@ -818,11 +818,6 @@ bool MergeTask::MergeProjectionsStage::mergeMinMaxIndexAndPrepareProjections() c // projection_future_part->path = global_ctx->future_part->path + "/" + projection.name + ".proj/"; projection_future_part->part_info = {"all", 0, 0, 0}; - Names deduplicate_by_columns; - for (const auto & column : global_ctx->deduplicate_by_columns) - if (projection.metadata->getColumns().has(column)) - deduplicate_by_columns.emplace_back(column); - MergeTreeData::MergingParams projection_merging_params; projection_merging_params.mode = MergeTreeData::MergingParams::Ordinary; if (projection.type == ProjectionDescription::Type::Aggregate) @@ -837,7 +832,7 @@ bool MergeTask::MergeProjectionsStage::mergeMinMaxIndexAndPrepareProjections() c global_ctx->context, global_ctx->space_reservation, global_ctx->deduplicate, - deduplicate_by_columns, + global_ctx->deduplicate_by_columns, global_ctx->cleanup, projection_merging_params, global_ctx->need_prefix, diff --git a/src/Storages/StorageMergeTree.cpp b/src/Storages/StorageMergeTree.cpp index 9352f772ce1..0dd0f051244 100644 --- a/src/Storages/StorageMergeTree.cpp +++ b/src/Storages/StorageMergeTree.cpp @@ -1549,6 +1549,12 @@ bool StorageMergeTree::optimize( { assertNotReadonly(); + if (deduplicate && getInMemoryMetadataPtr()->hasProjections()) + throw Exception(ErrorCodes::NOT_IMPLEMENTED, + "OPTIMIZE DEDUPLICATE query is not supported for table {} as it has projections. " + "User should drop all the projections manually before running the query", + getStorageID().getTableName()); + if (deduplicate) { if (deduplicate_by_columns.empty()) diff --git a/src/Storages/StorageReplicatedMergeTree.cpp b/src/Storages/StorageReplicatedMergeTree.cpp index db58d0081c6..403313a7177 100644 --- a/src/Storages/StorageReplicatedMergeTree.cpp +++ b/src/Storages/StorageReplicatedMergeTree.cpp @@ -5746,6 +5746,12 @@ bool StorageReplicatedMergeTree::optimize( if (!is_leader) throw Exception(ErrorCodes::NOT_A_LEADER, "OPTIMIZE cannot be done on this replica because it is not a leader"); + if (deduplicate && getInMemoryMetadataPtr()->hasProjections()) + throw Exception(ErrorCodes::NOT_IMPLEMENTED, + "OPTIMIZE DEDUPLICATE query is not supported for table {} as it has projections. " + "User should drop all the projections manually before running the query", + getStorageID().getTableName()); + if (cleanup) { if (!getSettings()->allow_experimental_replacing_merge_with_cleanup) diff --git a/tests/queries/0_stateless/03174_projection_deduplicate.reference b/tests/queries/0_stateless/03174_projection_deduplicate.reference index d2ee9d25154..586b63a392e 100644 --- a/tests/queries/0_stateless/03174_projection_deduplicate.reference +++ b/tests/queries/0_stateless/03174_projection_deduplicate.reference @@ -1 +1,2 @@ 1 one +1 one diff --git a/tests/queries/0_stateless/03174_projection_deduplicate.sql b/tests/queries/0_stateless/03174_projection_deduplicate.sql index 529c3260baa..ed6da5d73fc 100644 --- a/tests/queries/0_stateless/03174_projection_deduplicate.sql +++ b/tests/queries/0_stateless/03174_projection_deduplicate.sql @@ -17,7 +17,7 @@ PRIMARY KEY id; INSERT INTO test_projection_deduplicate VALUES (1, 'one'); INSERT INTO test_projection_deduplicate VALUES (1, 'one'); -OPTIMIZE TABLE test_projection_deduplicate DEDUPLICATE; +OPTIMIZE TABLE test_projection_deduplicate DEDUPLICATE; -- { serverError NOT_IMPLEMENTED } SELECT * FROM test_projection_deduplicate; From 7d4293f6f8bcd7fcba45b703a83bf44d103395c9 Mon Sep 17 00:00:00 2001 From: divanik Date: Fri, 28 Jun 2024 00:21:19 +0000 Subject: [PATCH 108/417] Make commands more profound --- programs/disks/CommandCopy.cpp | 109 ++++++++++++++++++++------------- programs/disks/CommandMove.cpp | 81 +++++++++++++----------- 2 files changed, 110 insertions(+), 80 deletions(-) diff --git a/programs/disks/CommandCopy.cpp b/programs/disks/CommandCopy.cpp index f176fa277d7..0938e88a7f5 100644 --- a/programs/disks/CommandCopy.cpp +++ b/programs/disks/CommandCopy.cpp @@ -1,68 +1,89 @@ -#include "ICommand.h" #include +#include "Common/Exception.h" #include +#include "DisksClient.h" +#include "ICommand.h" namespace DB { -namespace ErrorCodes -{ - extern const int BAD_ARGUMENTS; -} - class CommandCopy final : public ICommand { public: - CommandCopy() + explicit CommandCopy() : ICommand() { command_name = "copy"; - command_option_description.emplace(createOptionsDescription("Allowed options", getTerminalWidth())); - description = "Recursively copy data from `FROM_PATH` to `TO_PATH`"; - usage = "copy [OPTION]... "; - command_option_description->add_options() - ("disk-from", po::value(), "disk from which we copy") - ("disk-to", po::value(), "disk to which we copy"); + description = "Recursively copy data from `path-from` to `path-to`"; + options_description.add_options()( + "disk-from", po::value(), "disk from which we copy is executed (default value is a current disk)")( + "disk-to", po::value(), "disk to which copy is executed (default value is a current disk)")( + "path-from", po::value(), "path from which copy is executed (mandatory, positional)")( + "path-to", po::value(), "path to which copy is executed (mandatory, positional)")( + "recursive", "recursively copy the directory"); + positional_options_description.add("path-from", 1); + positional_options_description.add("path-to", 1); } - void processOptions( - Poco::Util::LayeredConfiguration & config, - po::variables_map & options) const override + void executeImpl(const CommandLineOptions & options, DisksClient & client) override { - if (options.count("disk-from")) - config.setString("disk-from", options["disk-from"].as()); - if (options.count("disk-to")) - config.setString("disk-to", options["disk-to"].as()); - } + auto disk_from = getDiskWithPath(client, options, "disk-from"); + auto disk_to = getDiskWithPath(client, options, "disk-to"); + String path_from = disk_from.getRelativeFromRoot(getValueFromCommandLineOptionsThrow(options, "path-from")); + String path_to = disk_to.getRelativeFromRoot(getValueFromCommandLineOptionsThrow(options, "path-to")); + bool recursive = options.count("recursive"); - void execute( - const std::vector & command_arguments, - std::shared_ptr & disk_selector, - Poco::Util::LayeredConfiguration & config) override - { - if (command_arguments.size() != 2) + if (!disk_from.getDisk()->exists(path_from)) { - printHelpMessage(); - throw DB::Exception(DB::ErrorCodes::BAD_ARGUMENTS, "Bad Arguments"); + throw Exception(ErrorCodes::BAD_ARGUMENTS, "cannot stat '{}': No such file or directory", path_from); } + else if (disk_from.getDisk()->isFile(path_from)) + { + auto target_location = getTargetLocation(path_from, disk_to, path_to); + if (!disk_to.getDisk()->exists(target_location) || disk_to.getDisk()->isFile(target_location)) + { + disk_from.getDisk()->copyFile( + path_from, + *disk_to.getDisk(), + target_location, + /* read_settings= */ {}, + /* write_settings= */ {}, + /* cancellation_hook= */ {}); + } + else + { + throw Exception( + ErrorCodes::BAD_ARGUMENTS, "cannot overwrite directory {} with non-directory {}", target_location, path_from); + } + } + else if (disk_from.getDisk()->isDirectory(path_from)) + { + if (!recursive) + { + throw Exception(ErrorCodes::BAD_ARGUMENTS, "--recursive not specified; omitting directory {}", path_from); + } + auto target_location = getTargetLocation(path_from, disk_to, path_to); - String disk_name_from = config.getString("disk-from", config.getString("disk", "default")); - String disk_name_to = config.getString("disk-to", config.getString("disk", "default")); - - const String & path_from = command_arguments[0]; - const String & path_to = command_arguments[1]; - - DiskPtr disk_from = disk_selector->get(disk_name_from); - DiskPtr disk_to = disk_selector->get(disk_name_to); - - String relative_path_from = validatePathAndGetAsRelative(path_from); - String relative_path_to = validatePathAndGetAsRelative(path_to); - - disk_from->copyDirectoryContent(relative_path_from, disk_to, relative_path_to, /* read_settings= */ {}, /* write_settings= */ {}, /* cancellation_hook= */ {}); + if (disk_to.getDisk()->isFile(target_location)) + { + throw Exception(ErrorCodes::BAD_ARGUMENTS, "cannot overwrite non-directory {} with directory {}", path_to, target_location); + } + else if (!disk_to.getDisk()->exists(target_location)) + { + disk_to.getDisk()->createDirectory(target_location); + } + disk_from.getDisk()->copyDirectoryContent( + path_from, + disk_to.getDisk(), + target_location, + /* read_settings= */ {}, + /* write_settings= */ {}, + /* cancellation_hook= */ {}); + } } }; -} -std::unique_ptr makeCommandCopy() +CommandPtr makeCommandCopy() { return std::make_unique(); } +} diff --git a/programs/disks/CommandMove.cpp b/programs/disks/CommandMove.cpp index 75cf96252ed..6080fcf6811 100644 --- a/programs/disks/CommandMove.cpp +++ b/programs/disks/CommandMove.cpp @@ -1,14 +1,9 @@ -#include "ICommand.h" #include +#include "ICommand.h" namespace DB { -namespace ErrorCodes -{ - extern const int BAD_ARGUMENTS; -} - class CommandMove final : public ICommand { public: @@ -16,44 +11,58 @@ public: { command_name = "move"; description = "Move file or directory from `from_path` to `to_path`"; - usage = "move [OPTION]... "; + options_description.add_options()("path-from", po::value(), "path from which we copy (mandatory, positional)")( + "path-to", po::value(), "path to which we copy (mandatory, positional)"); + positional_options_description.add("path-from", 1); + positional_options_description.add("path-to", 1); } - void processOptions( - Poco::Util::LayeredConfiguration &, - po::variables_map &) const override - {} - - void execute( - const std::vector & command_arguments, - std::shared_ptr & disk_selector, - Poco::Util::LayeredConfiguration & config) override + void executeImpl(const CommandLineOptions & options, DisksClient & client) override { - if (command_arguments.size() != 2) + auto disk = client.getCurrentDiskWithPath(); + + String path_from = disk.getRelativeFromRoot(getValueFromCommandLineOptionsThrow(options, "path-from")); + String path_to = disk.getRelativeFromRoot(getValueFromCommandLineOptionsThrow(options, "path-to")); + + if (disk.getDisk()->isFile(path_from)) { - printHelpMessage(); - throw DB::Exception(DB::ErrorCodes::BAD_ARGUMENTS, "Bad Arguments"); + disk.getDisk()->moveFile(path_from, path_to); + } + else if (disk.getDisk()->isDirectory(path_from)) + { + auto target_location = getTargetLocation(path_from, disk, path_to); + if (!disk.getDisk()->exists(target_location)) + { + disk.getDisk()->createDirectory(target_location); + disk.getDisk()->moveDirectory(path_from, target_location); + } + else + { + if (disk.getDisk()->isFile(target_location)) + { + throw Exception( + ErrorCodes::BAD_ARGUMENTS, "cannot overwrite non-directory '{}' with directory '{}'", target_location, path_from); + } + if (!disk.getDisk()->isDirectoryEmpty(target_location)) + { + throw Exception(ErrorCodes::BAD_ARGUMENTS, "cannot move '{}' to '{}': Directory not empty", path_from, target_location); + } + else + { + disk.getDisk()->moveDirectory(path_from, target_location); + } + } + } + else if (!disk.getDisk()->exists(path_from)) + { + throw Exception(ErrorCodes::BAD_ARGUMENTS, "cannot stat '{}': No such file or directory", path_from); } - - String disk_name = config.getString("disk", "default"); - - const String & path_from = command_arguments[0]; - const String & path_to = command_arguments[1]; - - DiskPtr disk = disk_selector->get(disk_name); - - String relative_path_from = validatePathAndGetAsRelative(path_from); - String relative_path_to = validatePathAndGetAsRelative(path_to); - - if (disk->isFile(relative_path_from)) - disk->moveFile(relative_path_from, relative_path_to); - else - disk->moveDirectory(relative_path_from, relative_path_to); } }; -} -std::unique_ptr makeCommandMove() +CommandPtr makeCommandMove() { return std::make_unique(); } + +} From f95434f4c001ce2fdce6066de241181f771a48f0 Mon Sep 17 00:00:00 2001 From: vdimir Date: Fri, 28 Jun 2024 14:31:11 +0000 Subject: [PATCH 109/417] Better handling join on nulls --- .../Passes/LogicalExpressionOptimizerPass.cpp | 191 +++++++++++++----- src/Analyzer/QueryTreeBuilder.cpp | 1 + src/Parsers/ASTFunction.cpp | 39 ++-- src/Parsers/ASTTablesInSelectQuery.cpp | 2 +- src/Planner/PlannerJoins.cpp | 2 +- ...11_join_on_nullsafe_optimization.reference | 6 +- .../02911_join_on_nullsafe_optimization.sql | 6 +- 7 files changed, 171 insertions(+), 76 deletions(-) diff --git a/src/Analyzer/Passes/LogicalExpressionOptimizerPass.cpp b/src/Analyzer/Passes/LogicalExpressionOptimizerPass.cpp index ac221bd66e7..3cbc44315c9 100644 --- a/src/Analyzer/Passes/LogicalExpressionOptimizerPass.cpp +++ b/src/Analyzer/Passes/LogicalExpressionOptimizerPass.cpp @@ -10,6 +10,7 @@ #include #include +#include namespace DB { @@ -26,12 +27,100 @@ static constexpr std::array boolean_functions{ "like"sv, "notLike"sv, "ilike"sv, "notILike"sv, "empty"sv, "notEmpty"sv, "not"sv, "and"sv, "or"sv}; -static bool isBooleanFunction(const String & func_name) + +bool isBooleanFunction(const String & func_name) { return std::any_of( boolean_functions.begin(), boolean_functions.end(), [&](const auto boolean_func) { return func_name == boolean_func; }); } +bool isNodeFunction(const QueryTreeNodePtr & node, const String & func_name) +{ + if (const auto * function_node = node->as()) + return function_node->getFunctionName() == func_name; + return false; +} + +QueryTreeNodePtr getFunctionArgument(const QueryTreeNodePtr & node, size_t idx) +{ + if (const auto * function_node = node->as()) + { + const auto & args = function_node->getArguments().getNodes(); + if (idx < args.size()) + return args[idx]; + } + throw Exception(ErrorCodes::LOGICAL_ERROR, "Expected '{}' to be a function with at least {} arguments", node->formatASTForErrorMessage(), idx + 1); +} + +QueryTreeNodePtr findEqualsFunction(const QueryTreeNodes & nodes) +{ + for (const auto & node : nodes) + { + const auto * function_node = node->as(); + if (function_node && function_node->getFunctionName() == "equals" && + function_node->getArguments().getNodes().size() == 2) + { + return node; + } + } + return nullptr; +} + +bool isNodeBooleanConstant(const QueryTreeNodePtr & node, bool expected_value) +{ + const auto * constant_node = node->as(); + if (!constant_node || !constant_node->getResultType()->equals(DataTypeUInt8())) + return false; + + UInt64 constant_value; + return (constant_node->getValue().tryGet(constant_value) && constant_value == expected_value); +} + +/// Returns true if expression consists of only conjunctions of functions with the specified name or true constants +bool isOnlyConjunctionOfFunctions( + const QueryTreeNodePtr & node, + const String & func_name, + const QueryTreeNodePtrWithHashSet & allowed_arguments) +{ + if (isNodeBooleanConstant(node, true)) + return true; + + const auto * node_function = node->as(); + if (node_function + && node_function->getFunctionName() == "isNotNull" + && allowed_arguments.contains(node_function->getArgumentsNode())) + return true; + + if (node_function && node_function->getFunctionName() == "and") + { + for (const auto & and_argument : node_function->getArguments().getNodes()) + { + if (!isOnlyConjunctionOfFunctions(and_argument, func_name, allowed_arguments)) + return false; + } + } + return false; +} + +/// We can rewrite to a <=> b only if we are joining on a and b, +/// because the function is not yet implemented for other cases. +bool isTwoArgumentsFromDifferentSides(const FunctionNode & node_function, const JoinNode & join_node) +{ + const auto & argument_nodes = node_function.getArguments().getNodes(); + if (argument_nodes.size() != 2) + return false; + + auto first_src = getExpressionSource(argument_nodes[0]); + auto second_src = getExpressionSource(argument_nodes[1]); + if (!first_src || !second_src) + return false; + + const auto & lhs_join = *join_node.getLeftTableExpression(); + const auto & rhs_join = *join_node.getRightTableExpression(); + return (first_src->isEqual(lhs_join) && second_src->isEqual(rhs_join)) || + (first_src->isEqual(rhs_join) && second_src->isEqual(lhs_join)); +} + /// Visitor that optimizes logical expressions _only_ in JOIN ON section class JoinOnLogicalExpressionOptimizerVisitor : public InDepthQueryTreeVisitorWithContext { @@ -78,7 +167,6 @@ private: auto & function_node = node->as(); chassert(function_node.getFunctionName() == "or"); - QueryTreeNodes or_operands; or_operands.reserve(function_node.getArguments().getNodes().size()); @@ -93,14 +181,22 @@ private: * b => [(a IS NULL AND b IS NULL)] * c => [(a IS NULL AND c IS NULL)] * } - * Then for each a <=> b we can find all operands that contains both a IS NULL and b IS NULL + * Then for each equality a = b we can check if we have operand (a IS NULL AND b IS NULL) */ QueryTreeNodePtrWithHashMap> is_null_argument_to_indices; + bool is_anything_changed = false; + for (const auto & argument : function_node.getArguments()) { - or_operands.push_back(argument); + if (isNodeBooleanConstant(argument, false)) + { + /// Remove false constants from OR + is_anything_changed = true; + continue; + } + or_operands.push_back(argument); auto * argument_function = argument->as(); if (!argument_function) continue; @@ -108,32 +204,48 @@ private: const auto & func_name = argument_function->getFunctionName(); if (func_name == "equals" || func_name == "isNotDistinctFrom") { - const auto & argument_nodes = argument_function->getArguments().getNodes(); - if (argument_nodes.size() != 2) - continue; - /// We can rewrite to a <=> b only if we are joining on a and b, - /// because the function is not yet implemented for other cases. - auto first_src = getExpressionSource(argument_nodes[0]); - auto second_src = getExpressionSource(argument_nodes[1]); - if (!first_src || !second_src) - continue; - const auto & lhs_join = *join_node->getLeftTableExpression(); - const auto & rhs_join = *join_node->getRightTableExpression(); - bool arguments_from_both_sides = (first_src->isEqual(lhs_join) && second_src->isEqual(rhs_join)) || - (first_src->isEqual(rhs_join) && second_src->isEqual(lhs_join)); - if (!arguments_from_both_sides) - continue; - equals_functions_indices.push_back(or_operands.size() - 1); + if (isTwoArgumentsFromDifferentSides(*argument_function, *join_node)) + equals_functions_indices.push_back(or_operands.size() - 1); } else if (func_name == "and") { - for (const auto & and_argument : argument_function->getArguments().getNodes()) + const auto & and_arguments = argument_function->getArguments().getNodes(); + bool all_are_is_null = and_arguments.size() == 2 && isNodeFunction(and_arguments[0], "isNull") && isNodeFunction(and_arguments[1], "isNull"); + if (all_are_is_null) { - auto * and_argument_function = and_argument->as(); - if (and_argument_function && and_argument_function->getFunctionName() == "isNull") + is_null_argument_to_indices[getFunctionArgument(and_arguments.front(), 0)].push_back(or_operands.size() - 1); + is_null_argument_to_indices[getFunctionArgument(and_arguments.back(), 0)].push_back(or_operands.size() - 1); + } + + /// Expression `a = b AND (a IS NOT NULL) AND true AND (b IS NOT NULL)` we can be replaced with `a = b` + if (const auto & equals_function = findEqualsFunction(and_arguments)) + { + const auto & equals_arguments = equals_function->as()->getArguments().getNodes(); + /// Expected isNotNull arguments + QueryTreeNodePtrWithHashSet allowed_arguments; + allowed_arguments.insert(QueryTreeNodePtrWithHash(std::make_shared(QueryTreeNodes{equals_arguments[0]}))); + allowed_arguments.insert(QueryTreeNodePtrWithHash(std::make_shared(QueryTreeNodes{equals_arguments[1]}))); + + bool can_be_optimized = true; + for (const auto & and_argument : and_arguments) { - const auto & is_null_argument = and_argument_function->getArguments().getNodes()[0]; - is_null_argument_to_indices[is_null_argument].push_back(or_operands.size() - 1); + if (and_argument.get() == equals_function.get()) + continue; + + if (isOnlyConjunctionOfFunctions(and_argument, "isNotNull", allowed_arguments)) + continue; + + can_be_optimized = false; + break; + } + + if (can_be_optimized) + { + is_anything_changed = true; + or_operands.pop_back(); + or_operands.push_back(equals_function); + if (isTwoArgumentsFromDifferentSides(equals_function->as(), *join_node)) + equals_functions_indices.push_back(or_operands.size() - 1); } } } @@ -146,7 +258,7 @@ private: { auto * equals_function = or_operands[equals_function_idx]->as(); - /// For a <=> b we are looking for expressions containing both `a IS NULL` and `b IS NULL` combined with AND + /// For a = b we are looking for all expressions `a IS NULL AND b IS NULL` const auto & argument_nodes = equals_function->getArguments().getNodes(); const auto & lhs_is_null_parents = is_null_argument_to_indices[argument_nodes[0]]; const auto & rhs_is_null_parents = is_null_argument_to_indices[argument_nodes[1]]; @@ -161,33 +273,14 @@ private: for (size_t to_optimize_idx : operands_to_optimize) { - /// We are looking for operand `a IS NULL AND b IS NULL AND ...` + /// Remove `a IS NULL AND b IS NULL` auto * operand_to_optimize = or_operands[to_optimize_idx]->as(); - - /// Remove `a IS NULL` and `b IS NULL` arguments from AND - QueryTreeNodes new_arguments; - for (const auto & and_argument : operand_to_optimize->getArguments().getNodes()) - { - bool to_eliminate = false; - - const auto * and_argument_function = and_argument->as(); - if (and_argument_function && and_argument_function->getFunctionName() == "isNull") - { - const auto & is_null_argument = and_argument_function->getArguments().getNodes()[0]; - to_eliminate = (is_null_argument->isEqual(*argument_nodes[0]) || is_null_argument->isEqual(*argument_nodes[1])); - } - - if (to_eliminate) - arguments_to_reresolve.insert(to_optimize_idx); - else - new_arguments.emplace_back(and_argument); - } - /// If less than two arguments left, we will remove or replace the whole AND below - operand_to_optimize->getArguments().getNodes() = std::move(new_arguments); + operand_to_optimize->getArguments().getNodes() = {}; + arguments_to_reresolve.insert(to_optimize_idx); } } - if (arguments_to_reresolve.empty()) + if (arguments_to_reresolve.empty() && !is_anything_changed) /// Nothing have been changed return false; diff --git a/src/Analyzer/QueryTreeBuilder.cpp b/src/Analyzer/QueryTreeBuilder.cpp index 6a5db4bc1de..dd083dd5df6 100644 --- a/src/Analyzer/QueryTreeBuilder.cpp +++ b/src/Analyzer/QueryTreeBuilder.cpp @@ -940,6 +940,7 @@ QueryTreeNodePtr QueryTreeBuilder::buildJoinTree(const ASTPtr & tables_in_select table_join.locality, result_join_strictness, result_join_kind); + join_node->setOriginalAST(table_element.table_join); /** Original AST is not set because it will contain only join part and does * not include left table expression. diff --git a/src/Parsers/ASTFunction.cpp b/src/Parsers/ASTFunction.cpp index 602ef8c232b..f39229d7566 100644 --- a/src/Parsers/ASTFunction.cpp +++ b/src/Parsers/ASTFunction.cpp @@ -408,25 +408,26 @@ void ASTFunction::formatImplWithoutAlias(const FormatSettings & settings, Format { const char * operators[] = { - "multiply", " * ", - "divide", " / ", - "modulo", " % ", - "plus", " + ", - "minus", " - ", - "notEquals", " != ", - "lessOrEquals", " <= ", - "greaterOrEquals", " >= ", - "less", " < ", - "greater", " > ", - "equals", " = ", - "like", " LIKE ", - "ilike", " ILIKE ", - "notLike", " NOT LIKE ", - "notILike", " NOT ILIKE ", - "in", " IN ", - "notIn", " NOT IN ", - "globalIn", " GLOBAL IN ", - "globalNotIn", " GLOBAL NOT IN ", + "multiply", " * ", + "divide", " / ", + "modulo", " % ", + "plus", " + ", + "minus", " - ", + "notEquals", " != ", + "lessOrEquals", " <= ", + "greaterOrEquals", " >= ", + "less", " < ", + "greater", " > ", + "equals", " = ", + "isNotDistinctFrom", " <=> ", + "like", " LIKE ", + "ilike", " ILIKE ", + "notLike", " NOT LIKE ", + "notILike", " NOT ILIKE ", + "in", " IN ", + "notIn", " NOT IN ", + "globalIn", " GLOBAL IN ", + "globalNotIn", " GLOBAL NOT IN ", nullptr }; diff --git a/src/Parsers/ASTTablesInSelectQuery.cpp b/src/Parsers/ASTTablesInSelectQuery.cpp index e782bad797e..d22a4eca0fc 100644 --- a/src/Parsers/ASTTablesInSelectQuery.cpp +++ b/src/Parsers/ASTTablesInSelectQuery.cpp @@ -243,7 +243,7 @@ void ASTTableJoin::formatImplAfterTable(const FormatSettings & settings, FormatS void ASTTableJoin::formatImpl(const FormatSettings & settings, FormatState & state, FormatStateStacked frame) const { formatImplBeforeTable(settings, state, frame); - settings.ostr << " ... "; + settings.ostr << " ..."; formatImplAfterTable(settings, state, frame); } diff --git a/src/Planner/PlannerJoins.cpp b/src/Planner/PlannerJoins.cpp index 84efdd21336..58332df62a0 100644 --- a/src/Planner/PlannerJoins.cpp +++ b/src/Planner/PlannerJoins.cpp @@ -528,7 +528,7 @@ JoinClausesAndActions buildJoinClausesAndActions( size_t join_clause_key_nodes_size = join_clause.getLeftKeyNodes().size(); if (join_clause_key_nodes_size == 0) - throw Exception(ErrorCodes::INVALID_JOIN_ON_EXPRESSION, "JOIN {} cannot get JOIN keys", + throw Exception(ErrorCodes::INVALID_JOIN_ON_EXPRESSION, "Cannot determine join keys in {}", join_node.formatASTForErrorMessage()); for (size_t i = 0; i < join_clause_key_nodes_size; ++i) diff --git a/tests/queries/0_stateless/02911_join_on_nullsafe_optimization.reference b/tests/queries/0_stateless/02911_join_on_nullsafe_optimization.reference index 5b6c14ca24f..1ab16084cb4 100644 --- a/tests/queries/0_stateless/02911_join_on_nullsafe_optimization.reference +++ b/tests/queries/0_stateless/02911_join_on_nullsafe_optimization.reference @@ -3,7 +3,7 @@ SELECT * FROM t1 JOIN t2 ON (t1.x <=> t2.x OR (t1.x IS NULL AND t2.x IS NULL)) O 2 2 2 2 3 3 3 33 \N \N \N \N -SELECT * FROM t1 JOIN t2 ON (t1.x <=> t2.x OR t1.x IS NULL AND t1.y <=> t2.y AND t2.x IS NULL) ORDER BY t1.x NULLS LAST; +SELECT * FROM t1 JOIN t2 ON (t1.x <=> t2.x OR t1.x IS NULL AND t2.x IS NULL) OR t1.y <=> t2.y ORDER BY t1.x NULLS LAST; 1 42 4 42 2 2 2 2 3 3 3 33 @@ -12,14 +12,14 @@ SELECT * FROM t1 JOIN t2 ON (t1.x = t2.x OR t1.x IS NULL AND t2.x IS NULL) ORDER 2 2 2 2 3 3 3 33 \N \N \N \N -SELECT * FROM t1 JOIN t2 ON t1.x <=> t2.x AND (t1.x = t1.y OR t1.x IS NULL AND t1.y IS NULL) ORDER BY t1.x; +SELECT * FROM t1 JOIN t2 ON t1.x <=> t2.x AND ((t1.x = t1.y) OR t1.x IS NULL AND t1.y IS NULL) ORDER BY t1.x; 2 2 2 2 3 3 3 33 \N \N \N \N SELECT * FROM t1 JOIN t2 ON (t1.x = t2.x OR t1.x IS NULL AND t2.x IS NULL) AND t1.y <=> t2.y ORDER BY t1.x NULLS LAST; 2 2 2 2 \N \N \N \N -SELECT * FROM t1 JOIN t2 ON (t1.x <=> t2.x OR t1.y <=> t2.y OR (t1.x IS NULL AND t1.y IS NULL AND t2.x IS NULL AND t2.y IS NULL)) ORDER BY t1.x NULLS LAST; +SELECT * FROM t1 JOIN t2 ON (t1.x <=> t2.x OR t1.y <=> t2.y OR (t1.x IS NULL AND t2.x IS NULL) OR (t1.y IS NULL AND t2.y IS NULL)) ORDER BY t1.x NULLS LAST; 1 42 4 42 2 2 2 2 3 3 3 33 diff --git a/tests/queries/0_stateless/02911_join_on_nullsafe_optimization.sql b/tests/queries/0_stateless/02911_join_on_nullsafe_optimization.sql index 5458370db8c..6a163482d68 100644 --- a/tests/queries/0_stateless/02911_join_on_nullsafe_optimization.sql +++ b/tests/queries/0_stateless/02911_join_on_nullsafe_optimization.sql @@ -12,14 +12,14 @@ SET allow_experimental_analyzer = 1; -- { echoOn } SELECT * FROM t1 JOIN t2 ON (t1.x <=> t2.x OR (t1.x IS NULL AND t2.x IS NULL)) ORDER BY t1.x NULLS LAST; -SELECT * FROM t1 JOIN t2 ON (t1.x <=> t2.x OR t1.x IS NULL AND t1.y <=> t2.y AND t2.x IS NULL) ORDER BY t1.x NULLS LAST; +SELECT * FROM t1 JOIN t2 ON (t1.x <=> t2.x OR t1.x IS NULL AND t2.x IS NULL) OR t1.y <=> t2.y ORDER BY t1.x NULLS LAST; SELECT * FROM t1 JOIN t2 ON (t1.x = t2.x OR t1.x IS NULL AND t2.x IS NULL) ORDER BY t1.x; -SELECT * FROM t1 JOIN t2 ON t1.x <=> t2.x AND (t1.x = t1.y OR t1.x IS NULL AND t1.y IS NULL) ORDER BY t1.x; +SELECT * FROM t1 JOIN t2 ON t1.x <=> t2.x AND ((t1.x = t1.y) OR t1.x IS NULL AND t1.y IS NULL) ORDER BY t1.x; SELECT * FROM t1 JOIN t2 ON (t1.x = t2.x OR t1.x IS NULL AND t2.x IS NULL) AND t1.y <=> t2.y ORDER BY t1.x NULLS LAST; -SELECT * FROM t1 JOIN t2 ON (t1.x <=> t2.x OR t1.y <=> t2.y OR (t1.x IS NULL AND t1.y IS NULL AND t2.x IS NULL AND t2.y IS NULL)) ORDER BY t1.x NULLS LAST; +SELECT * FROM t1 JOIN t2 ON (t1.x <=> t2.x OR t1.y <=> t2.y OR (t1.x IS NULL AND t2.x IS NULL) OR (t1.y IS NULL AND t2.y IS NULL)) ORDER BY t1.x NULLS LAST; SELECT * FROM t1 JOIN t2 ON (t1.x <=> t2.x OR (t1.x IS NULL AND t2.x IS NULL)) AND (t1.y == t2.y OR (t1.y IS NULL AND t2.y IS NULL)) AND COALESCE(t1.x, 0) != 2 ORDER BY t1.x NULLS LAST; From 8e770c7046679ac36d72c5b1142f14a94851711e Mon Sep 17 00:00:00 2001 From: vdimir Date: Fri, 28 Jun 2024 15:19:33 +0000 Subject: [PATCH 110/417] fix --- .../Passes/LogicalExpressionOptimizerPass.cpp | 11 ++++++++--- ...02911_join_on_nullsafe_optimization.reference | 7 +++++++ .../02911_join_on_nullsafe_optimization.sql | 16 ++++++++++++++++ 3 files changed, 31 insertions(+), 3 deletions(-) diff --git a/src/Analyzer/Passes/LogicalExpressionOptimizerPass.cpp b/src/Analyzer/Passes/LogicalExpressionOptimizerPass.cpp index 3cbc44315c9..c74148a7252 100644 --- a/src/Analyzer/Passes/LogicalExpressionOptimizerPass.cpp +++ b/src/Analyzer/Passes/LogicalExpressionOptimizerPass.cpp @@ -83,21 +83,26 @@ bool isOnlyConjunctionOfFunctions( const QueryTreeNodePtrWithHashSet & allowed_arguments) { if (isNodeBooleanConstant(node, true)) + { return true; + } const auto * node_function = node->as(); - if (node_function - && node_function->getFunctionName() == "isNotNull" + if (!node_function) + return false; + + if (node_function->getFunctionName() == func_name && allowed_arguments.contains(node_function->getArgumentsNode())) return true; - if (node_function && node_function->getFunctionName() == "and") + if (node_function->getFunctionName() == "and") { for (const auto & and_argument : node_function->getArguments().getNodes()) { if (!isOnlyConjunctionOfFunctions(and_argument, func_name, allowed_arguments)) return false; } + return true; } return false; } diff --git a/tests/queries/0_stateless/02911_join_on_nullsafe_optimization.reference b/tests/queries/0_stateless/02911_join_on_nullsafe_optimization.reference index 1ab16084cb4..1df3606592c 100644 --- a/tests/queries/0_stateless/02911_join_on_nullsafe_optimization.reference +++ b/tests/queries/0_stateless/02911_join_on_nullsafe_optimization.reference @@ -31,3 +31,10 @@ SELECT x = y OR (x IS NULL AND y IS NULL) FROM t1 ORDER BY x NULLS LAST; 1 1 1 +SELECT * FROM t1 JOIN t2 ON (t1.x == t2.x AND ((t2.x IS NOT NULL) AND (t1.x IS NOT NULL)) ) OR ( (t2.x IS NULL) AND (t1.x IS NULL) ) ORDER BY t1.x NULLS LAST; +2 2 2 2 +3 3 3 33 +\N \N \N \N +SELECT * FROM t1n as t1 JOIN t2n as t2 ON (t1.x == t2.x AND ((t2.x IS NOT NULL) AND (t1.x IS NOT NULL)) ) OR ( (t2.x IS NULL) AND (t1.x IS NULL) ) ORDER BY t1.x NULLS LAST; +2 2 2 2 +3 3 3 33 diff --git a/tests/queries/0_stateless/02911_join_on_nullsafe_optimization.sql b/tests/queries/0_stateless/02911_join_on_nullsafe_optimization.sql index 6a163482d68..3b150c2f284 100644 --- a/tests/queries/0_stateless/02911_join_on_nullsafe_optimization.sql +++ b/tests/queries/0_stateless/02911_join_on_nullsafe_optimization.sql @@ -1,5 +1,7 @@ DROP TABLE IF EXISTS t1; +DROP TABLE IF EXISTS t1n; DROP TABLE IF EXISTS t2; +DROP TABLE IF EXISTS t2n; CREATE TABLE t1 (x Nullable(Int64), y Nullable(UInt64)) ENGINE = TinyLog; CREATE TABLE t2 (x Nullable(Int64), y Nullable(UInt64)) ENGINE = TinyLog; @@ -7,6 +9,13 @@ CREATE TABLE t2 (x Nullable(Int64), y Nullable(UInt64)) ENGINE = TinyLog; INSERT INTO t1 VALUES (1,42), (2,2), (3,3), (NULL,NULL); INSERT INTO t2 VALUES (NULL,NULL), (2,2), (3,33), (4,42); + +CREATE TABLE t1n (x Int64, y UInt64) ENGINE = TinyLog; +CREATE TABLE t2n (x Int64, y UInt64) ENGINE = TinyLog; + +INSERT INTO t1n VALUES (1,42), (2,2), (3,3); +INSERT INTO t2n VALUES (2,2), (3,33), (4,42); + SET allow_experimental_analyzer = 1; -- { echoOn } @@ -24,7 +33,14 @@ SELECT * FROM t1 JOIN t2 ON (t1.x <=> t2.x OR t1.y <=> t2.y OR (t1.x IS NULL AND SELECT * FROM t1 JOIN t2 ON (t1.x <=> t2.x OR (t1.x IS NULL AND t2.x IS NULL)) AND (t1.y == t2.y OR (t1.y IS NULL AND t2.y IS NULL)) AND COALESCE(t1.x, 0) != 2 ORDER BY t1.x NULLS LAST; SELECT x = y OR (x IS NULL AND y IS NULL) FROM t1 ORDER BY x NULLS LAST; + +SELECT * FROM t1 JOIN t2 ON (t1.x == t2.x AND ((t2.x IS NOT NULL) AND (t1.x IS NOT NULL)) ) OR ( (t2.x IS NULL) AND (t1.x IS NULL) ) ORDER BY t1.x NULLS LAST; + +SELECT * FROM t1n as t1 JOIN t2n as t2 ON (t1.x == t2.x AND ((t2.x IS NOT NULL) AND (t1.x IS NOT NULL)) ) OR ( (t2.x IS NULL) AND (t1.x IS NULL) ) ORDER BY t1.x NULLS LAST; + -- { echoOff } DROP TABLE IF EXISTS t1; DROP TABLE IF EXISTS t2; +DROP TABLE IF EXISTS t1n; +DROP TABLE IF EXISTS t2n; From aa7017a7fb1dcc09f6d7f948d3adb2d65a7b5201 Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Mon, 1 Jul 2024 00:32:39 +0200 Subject: [PATCH 111/417] Add a test for #43003 --- .../03199_join_with_materialized_column.reference | 0 .../0_stateless/03199_join_with_materialized_column.sql | 6 ++++++ 2 files changed, 6 insertions(+) create mode 100644 tests/queries/0_stateless/03199_join_with_materialized_column.reference create mode 100644 tests/queries/0_stateless/03199_join_with_materialized_column.sql diff --git a/tests/queries/0_stateless/03199_join_with_materialized_column.reference b/tests/queries/0_stateless/03199_join_with_materialized_column.reference new file mode 100644 index 00000000000..e69de29bb2d diff --git a/tests/queries/0_stateless/03199_join_with_materialized_column.sql b/tests/queries/0_stateless/03199_join_with_materialized_column.sql new file mode 100644 index 00000000000..8c53c5b3e66 --- /dev/null +++ b/tests/queries/0_stateless/03199_join_with_materialized_column.sql @@ -0,0 +1,6 @@ +SET allow_experimental_analyzer = 1; + +DROP TABLE IF EXISTS table_with_materialized; +CREATE TABLE table_with_materialized (col String MATERIALIZED 'A') ENGINE = Memory; +SELECT number FROM numbers(1) AS n, table_with_materialized; +DROP TABLE table_with_materialized; From d627709179db7b3af9b6e2f1b1c30b3e87da0cb1 Mon Sep 17 00:00:00 2001 From: vdimir Date: Mon, 1 Jul 2024 09:17:28 +0000 Subject: [PATCH 112/417] upd test --- tests/queries/0_stateless/01881_join_on_conditions_hash.sql.j2 | 2 +- .../queries/0_stateless/01881_join_on_conditions_merge.sql.j2 | 2 +- .../0_stateless/02911_join_on_nullsafe_optimization.sql | 3 +-- 3 files changed, 3 insertions(+), 4 deletions(-) diff --git a/tests/queries/0_stateless/01881_join_on_conditions_hash.sql.j2 b/tests/queries/0_stateless/01881_join_on_conditions_hash.sql.j2 index bd20d34b684..c2d85cefb18 100644 --- a/tests/queries/0_stateless/01881_join_on_conditions_hash.sql.j2 +++ b/tests/queries/0_stateless/01881_join_on_conditions_hash.sql.j2 @@ -72,7 +72,7 @@ SELECT * FROM t1 INNER ALL JOIN t2 ON t1.id == t2.id AND t1.id + 2; -- { serverE SELECT * FROM t1 INNER ALL JOIN t2 ON t1.id == t2.id AND t2.id + 2; -- { serverError 403 } SELECT * FROM t1 INNER ALL JOIN t2 ON t1.id == t2.id AND t1.key; -- { serverError 43, 403 } SELECT * FROM t1 INNER ALL JOIN t2 ON t1.id == t2.id AND t2.key; -- { serverError 43, 403 } -SELECT * FROM t1 JOIN t2 ON t2.key == t2.key2 AND (t1.id == t2.id OR isNull(t2.key2)); -- { serverError 403 } +SELECT * FROM t1 JOIN t2_nullable as t2 ON t2.key == t2.key2 AND (t1.id == t2.id OR isNull(t2.key2)); -- { serverError 403 } SELECT * FROM t1 JOIN t2 ON t2.key == t2.key2 OR t1.id == t2.id; -- { serverError 403 } SELECT * FROM t1 JOIN t2 ON (t2.key == t2.key2 AND (t1.key == t1.key2 AND t1.key != 'XXX' OR t1.id == t2.id)) AND t1.id == t2.id; -- { serverError 403 } SELECT * FROM t1 JOIN t2 ON t2.key == t2.key2 AND t1.key == t1.key2 AND t1.key != 'XXX' AND t1.id == t2.id OR t2.key == t2.key2 AND t1.id == t2.id AND t1.id == t2.id; diff --git a/tests/queries/0_stateless/01881_join_on_conditions_merge.sql.j2 b/tests/queries/0_stateless/01881_join_on_conditions_merge.sql.j2 index e4b704247b2..13703771ac8 100644 --- a/tests/queries/0_stateless/01881_join_on_conditions_merge.sql.j2 +++ b/tests/queries/0_stateless/01881_join_on_conditions_merge.sql.j2 @@ -70,7 +70,7 @@ SELECT * FROM t1 INNER ALL JOIN t2 ON t1.id == t2.id AND t1.id + 2; -- { serverE SELECT * FROM t1 INNER ALL JOIN t2 ON t1.id == t2.id AND t2.id + 2; -- { serverError 403 } SELECT * FROM t1 INNER ALL JOIN t2 ON t1.id == t2.id AND t1.key; -- { serverError 43, 403 } SELECT * FROM t1 INNER ALL JOIN t2 ON t1.id == t2.id AND t2.key; -- { serverError 43, 403 } -SELECT * FROM t1 JOIN t2 ON t2.key == t2.key2 AND (t1.id == t2.id OR isNull(t2.key2)); -- { serverError 403 } +SELECT * FROM t1 JOIN t2_nullable as t2 ON t2.key == t2.key2 AND (t1.id == t2.id OR isNull(t2.key2)); -- { serverError 403 } SELECT * FROM t1 JOIN t2 ON t2.key == t2.key2 OR t1.id == t2.id; -- { serverError 403 } SELECT * FROM t1 JOIN t2 ON (t2.key == t2.key2 AND (t1.key == t1.key2 AND t1.key != 'XXX' OR t1.id == t2.id)) AND t1.id == t2.id; -- { serverError 403 } SELECT * FROM t1 JOIN t2 ON t2.key == t2.key2 AND t1.key == t1.key2 AND t1.key != 'XXX' AND t1.id == t2.id OR t2.key == t2.key2 AND t1.id == t2.id AND t1.id == t2.id; -- { serverError 48 } diff --git a/tests/queries/0_stateless/02911_join_on_nullsafe_optimization.sql b/tests/queries/0_stateless/02911_join_on_nullsafe_optimization.sql index 3b150c2f284..0a642a716a4 100644 --- a/tests/queries/0_stateless/02911_join_on_nullsafe_optimization.sql +++ b/tests/queries/0_stateless/02911_join_on_nullsafe_optimization.sql @@ -1,6 +1,6 @@ DROP TABLE IF EXISTS t1; -DROP TABLE IF EXISTS t1n; DROP TABLE IF EXISTS t2; +DROP TABLE IF EXISTS t1n; DROP TABLE IF EXISTS t2n; CREATE TABLE t1 (x Nullable(Int64), y Nullable(UInt64)) ENGINE = TinyLog; @@ -9,7 +9,6 @@ CREATE TABLE t2 (x Nullable(Int64), y Nullable(UInt64)) ENGINE = TinyLog; INSERT INTO t1 VALUES (1,42), (2,2), (3,3), (NULL,NULL); INSERT INTO t2 VALUES (NULL,NULL), (2,2), (3,33), (4,42); - CREATE TABLE t1n (x Int64, y UInt64) ENGINE = TinyLog; CREATE TABLE t2n (x Int64, y UInt64) ENGINE = TinyLog; From bb0b93f77db0a3e3618523b55766a7e61700bc55 Mon Sep 17 00:00:00 2001 From: divanik Date: Mon, 1 Jul 2024 11:11:16 +0000 Subject: [PATCH 113/417] Change wrong implementations of copy and move --- programs/disks/CommandCopy.cpp | 54 ++++++++++++++++++++++++++++++-- programs/disks/CommandMkDir.cpp | 6 ++-- programs/disks/CommandMove.cpp | 33 +++++++++++++++++-- programs/disks/CommandRemove.cpp | 27 ++++++++++++++-- programs/disks/DisksClient.cpp | 2 +- programs/disks/ICommand.h | 16 ++++++++++ 6 files changed, 126 insertions(+), 12 deletions(-) diff --git a/programs/disks/CommandCopy.cpp b/programs/disks/CommandCopy.cpp index 4ba8a9ecbc2..0938e88a7f5 100644 --- a/programs/disks/CommandCopy.cpp +++ b/programs/disks/CommandCopy.cpp @@ -1,4 +1,5 @@ #include +#include "Common/Exception.h" #include #include "DisksClient.h" #include "ICommand.h" @@ -17,7 +18,8 @@ public: "disk-from", po::value(), "disk from which we copy is executed (default value is a current disk)")( "disk-to", po::value(), "disk to which copy is executed (default value is a current disk)")( "path-from", po::value(), "path from which copy is executed (mandatory, positional)")( - "path-to", po::value(), "path to which copy is executed (mandatory, positional)"); + "path-to", po::value(), "path to which copy is executed (mandatory, positional)")( + "recursive", "recursively copy the directory"); positional_options_description.add("path-from", 1); positional_options_description.add("path-to", 1); } @@ -28,9 +30,55 @@ public: auto disk_to = getDiskWithPath(client, options, "disk-to"); String path_from = disk_from.getRelativeFromRoot(getValueFromCommandLineOptionsThrow(options, "path-from")); String path_to = disk_to.getRelativeFromRoot(getValueFromCommandLineOptionsThrow(options, "path-to")); + bool recursive = options.count("recursive"); - disk_from.getDisk()->copyDirectoryContent( - path_from, disk_to.getDisk(), path_to, /* read_settings= */ {}, /* write_settings= */ {}, /* cancellation_hook= */ {}); + if (!disk_from.getDisk()->exists(path_from)) + { + throw Exception(ErrorCodes::BAD_ARGUMENTS, "cannot stat '{}': No such file or directory", path_from); + } + else if (disk_from.getDisk()->isFile(path_from)) + { + auto target_location = getTargetLocation(path_from, disk_to, path_to); + if (!disk_to.getDisk()->exists(target_location) || disk_to.getDisk()->isFile(target_location)) + { + disk_from.getDisk()->copyFile( + path_from, + *disk_to.getDisk(), + target_location, + /* read_settings= */ {}, + /* write_settings= */ {}, + /* cancellation_hook= */ {}); + } + else + { + throw Exception( + ErrorCodes::BAD_ARGUMENTS, "cannot overwrite directory {} with non-directory {}", target_location, path_from); + } + } + else if (disk_from.getDisk()->isDirectory(path_from)) + { + if (!recursive) + { + throw Exception(ErrorCodes::BAD_ARGUMENTS, "--recursive not specified; omitting directory {}", path_from); + } + auto target_location = getTargetLocation(path_from, disk_to, path_to); + + if (disk_to.getDisk()->isFile(target_location)) + { + throw Exception(ErrorCodes::BAD_ARGUMENTS, "cannot overwrite non-directory {} with directory {}", path_to, target_location); + } + else if (!disk_to.getDisk()->exists(target_location)) + { + disk_to.getDisk()->createDirectory(target_location); + } + disk_from.getDisk()->copyDirectoryContent( + path_from, + disk_to.getDisk(), + target_location, + /* read_settings= */ {}, + /* write_settings= */ {}, + /* cancellation_hook= */ {}); + } } }; diff --git a/programs/disks/CommandMkDir.cpp b/programs/disks/CommandMkDir.cpp index 3ea6df5622d..535936480d9 100644 --- a/programs/disks/CommandMkDir.cpp +++ b/programs/disks/CommandMkDir.cpp @@ -13,14 +13,14 @@ public: { command_name = "mkdir"; description = "Creates a directory"; - options_description.add_options()("recursive", "recursively create directories")( - "path", po::value(), "the path of listing (mandatory, positional)"); + options_description.add_options()("parents", "recursively create directories")( + "path", po::value(), "the path on which directory should be created (mandatory, positional)"); positional_options_description.add("path", 1); } void executeImpl(const CommandLineOptions & options, DisksClient & client) override { - bool recursive = options.count("recursive"); + bool recursive = options.count("parents"); auto disk = client.getCurrentDiskWithPath(); String path = disk.getRelativeFromRoot(getValueFromCommandLineOptionsThrow(options, "path")); diff --git a/programs/disks/CommandMove.cpp b/programs/disks/CommandMove.cpp index 23144df3d35..6080fcf6811 100644 --- a/programs/disks/CommandMove.cpp +++ b/programs/disks/CommandMove.cpp @@ -25,9 +25,38 @@ public: String path_to = disk.getRelativeFromRoot(getValueFromCommandLineOptionsThrow(options, "path-to")); if (disk.getDisk()->isFile(path_from)) + { disk.getDisk()->moveFile(path_from, path_to); - else - disk.getDisk()->moveDirectory(path_from, path_to); + } + else if (disk.getDisk()->isDirectory(path_from)) + { + auto target_location = getTargetLocation(path_from, disk, path_to); + if (!disk.getDisk()->exists(target_location)) + { + disk.getDisk()->createDirectory(target_location); + disk.getDisk()->moveDirectory(path_from, target_location); + } + else + { + if (disk.getDisk()->isFile(target_location)) + { + throw Exception( + ErrorCodes::BAD_ARGUMENTS, "cannot overwrite non-directory '{}' with directory '{}'", target_location, path_from); + } + if (!disk.getDisk()->isDirectoryEmpty(target_location)) + { + throw Exception(ErrorCodes::BAD_ARGUMENTS, "cannot move '{}' to '{}': Directory not empty", path_from, target_location); + } + else + { + disk.getDisk()->moveDirectory(path_from, target_location); + } + } + } + else if (!disk.getDisk()->exists(path_from)) + { + throw Exception(ErrorCodes::BAD_ARGUMENTS, "cannot stat '{}': No such file or directory", path_from); + } } }; diff --git a/programs/disks/CommandRemove.cpp b/programs/disks/CommandRemove.cpp index b322fb2701f..d508645fc65 100644 --- a/programs/disks/CommandRemove.cpp +++ b/programs/disks/CommandRemove.cpp @@ -1,4 +1,5 @@ #include +#include "Common/Exception.h" #include "ICommand.h" namespace DB @@ -10,8 +11,9 @@ public: CommandRemove() { command_name = "remove"; - description = "Remove file or directory with all children. Throws exception if file doesn't exists"; - options_description.add_options()("path", po::value(), "path from which we copy (mandatory, positional)"); + description = "Remove file or directory. Throws exception if file doesn't exists"; + options_description.add_options()("path", po::value(), "path from which we copy (mandatory, positional)")( + "recursive", "recursively removes the directory (required to remove a directory)"); positional_options_description.add("path", 1); } @@ -19,7 +21,26 @@ public: { auto disk = client.getCurrentDiskWithPath(); const String & path = disk.getRelativeFromRoot(getValueFromCommandLineOptionsThrow(options, "path")); - disk.getDisk()->removeRecursive(path); + bool recursive = options.count("recursive"); + if (!disk.getDisk()->exists(path)) + { + throw Exception(ErrorCodes::BAD_ARGUMENTS, "Path {} on disk {} doesn't exist", path, disk.getDisk()->getName()); + } + else if (disk.getDisk()->isDirectory(path)) + { + if (!recursive) + { + throw Exception(ErrorCodes::BAD_ARGUMENTS, "cannot remove '{}': Is a directory", path); + } + else + { + disk.getDisk()->removeRecursive(path); + } + } + else + { + disk.getDisk()->removeFileIfExists(path); + } } }; diff --git a/programs/disks/DisksClient.cpp b/programs/disks/DisksClient.cpp index 379c87e4f2f..7e36c7911ab 100644 --- a/programs/disks/DisksClient.cpp +++ b/programs/disks/DisksClient.cpp @@ -20,7 +20,7 @@ DiskWithPath::DiskWithPath(DiskPtr disk_, std::optional path_) : disk(di { if (!fs::path{path_.value()}.is_absolute()) { - throw Exception(ErrorCodes::BAD_ARGUMENTS, "Initializing path {} is not absolute", path_.value()); + throw Exception(ErrorCodes::BAD_ARGUMENTS, "Initializing path {} is not absolute", path_.value()); } path = path_.value(); } diff --git a/programs/disks/ICommand.h b/programs/disks/ICommand.h index 4b0ec731966..6faf90e2b52 100644 --- a/programs/disks/ICommand.h +++ b/programs/disks/ICommand.h @@ -100,6 +100,22 @@ protected: DiskWithPath & getDiskWithPath(DisksClient & client, const CommandLineOptions & options, const String & name); + String getTargetLocation(const String & path_from, DiskWithPath & disk_to, const String & path_to) + { + if (!disk_to.getDisk()->isDirectory(path_to)) + { + return path_to; + } + String copied_path_from = path_from; + if (copied_path_from.ends_with('/')) + { + copied_path_from.pop_back(); + } + String plain_filename = fs::path(copied_path_from).filename(); + + return fs::path{path_to} / plain_filename; + } + public: String command_name; From fee7da5ed3aec7b852d15f13485789f7deeda50f Mon Sep 17 00:00:00 2001 From: divanik Date: Mon, 1 Jul 2024 12:46:17 +0000 Subject: [PATCH 114/417] Corrected tests --- programs/disks/CommandCopy.cpp | 8 ++++++-- programs/disks/CommandMove.cpp | 6 +++++- programs/disks/CommandRemove.cpp | 2 +- tests/integration/test_disks_app_func/test.py | 2 +- .../test_disks_app_interactive/test.py | 18 +++++++++++------- 5 files changed, 24 insertions(+), 12 deletions(-) diff --git a/programs/disks/CommandCopy.cpp b/programs/disks/CommandCopy.cpp index 0938e88a7f5..eef87535e51 100644 --- a/programs/disks/CommandCopy.cpp +++ b/programs/disks/CommandCopy.cpp @@ -19,7 +19,7 @@ public: "disk-to", po::value(), "disk to which copy is executed (default value is a current disk)")( "path-from", po::value(), "path from which copy is executed (mandatory, positional)")( "path-to", po::value(), "path to which copy is executed (mandatory, positional)")( - "recursive", "recursively copy the directory"); + "recursive,r", "recursively copy the directory"); positional_options_description.add("path-from", 1); positional_options_description.add("path-to", 1); } @@ -34,7 +34,11 @@ public: if (!disk_from.getDisk()->exists(path_from)) { - throw Exception(ErrorCodes::BAD_ARGUMENTS, "cannot stat '{}': No such file or directory", path_from); + throw Exception( + ErrorCodes::BAD_ARGUMENTS, + "cannot stat '{}' on disk '{}': No such file or directory", + path_from, + disk_from.getDisk()->getName()); } else if (disk_from.getDisk()->isFile(path_from)) { diff --git a/programs/disks/CommandMove.cpp b/programs/disks/CommandMove.cpp index 6080fcf6811..22c1f851174 100644 --- a/programs/disks/CommandMove.cpp +++ b/programs/disks/CommandMove.cpp @@ -55,7 +55,11 @@ public: } else if (!disk.getDisk()->exists(path_from)) { - throw Exception(ErrorCodes::BAD_ARGUMENTS, "cannot stat '{}': No such file or directory", path_from); + throw Exception( + ErrorCodes::BAD_ARGUMENTS, + "cannot stat '{}' on disk: '{}': No such file or directory", + path_from, + disk.getDisk()->getName()); } } }; diff --git a/programs/disks/CommandRemove.cpp b/programs/disks/CommandRemove.cpp index d508645fc65..caa70905bef 100644 --- a/programs/disks/CommandRemove.cpp +++ b/programs/disks/CommandRemove.cpp @@ -13,7 +13,7 @@ public: command_name = "remove"; description = "Remove file or directory. Throws exception if file doesn't exists"; options_description.add_options()("path", po::value(), "path from which we copy (mandatory, positional)")( - "recursive", "recursively removes the directory (required to remove a directory)"); + "recursive,r", "recursively removes the directory (required to remove a directory)"); positional_options_description.add("path", 1); } diff --git a/tests/integration/test_disks_app_func/test.py b/tests/integration/test_disks_app_func/test.py index 34e45a9d626..56ea5c8846a 100644 --- a/tests/integration/test_disks_app_func/test.py +++ b/tests/integration/test_disks_app_func/test.py @@ -129,7 +129,7 @@ def test_disks_app_func_cp(started_cluster): "/usr/bin/clickhouse", "disks", "--query", - "copy --disk-from test1 --disk-to test2 . .", + "copy --recursive --disk-from test1 --disk-to test2 . .", ] ) diff --git a/tests/integration/test_disks_app_interactive/test.py b/tests/integration/test_disks_app_interactive/test.py index 79ffc3001a5..ca4ba5d9065 100644 --- a/tests/integration/test_disks_app_interactive/test.py +++ b/tests/integration/test_disks_app_interactive/test.py @@ -146,18 +146,22 @@ class DisksClient(object): path_to, disk_from: Optional[str] = None, disk_to: Optional[str] = None, + recursive: bool = False, ): disk_from_option = f"--disk-from {disk_from} " if disk_from is not None else "" disk_to_option = f"--disk-to {disk_to} " if disk_to is not None else "" + recursive_tag = "--recursive" if recursive else "" + self.execute_query( - f"copy {path_from} {path_to} {disk_from_option} {disk_to_option}" + f"copy {recursive_tag} {path_from} {path_to} {disk_from_option} {disk_to_option}" ) def move(self, path_from: str, path_to: str): self.execute_query(f"move {path_from} {path_to}") - def rm(self, path: str): - self.execute_query(f"rm {path}") + def rm(self, path: str, recursive: bool = False): + recursive_tag = "--recursive" if recursive else "" + self.execute_query(f"rm {recursive_tag} {path}") def mkdir(self, path: str, recursive: bool = False): recursive_adding = "--recursive " if recursive else "" @@ -260,7 +264,7 @@ def test_disks_app_interactive_list_directories_default(): "./.dir3/dir31": [], "./.dir3/.dir32": [], } - client.rm("dir2") + client.rm("dir2", recursive=True) traversed_dir = client.ls(".", recursive=True, show_hidden=True) assert traversed_dir == { ".": [".dir3", "dir1"], @@ -279,8 +283,8 @@ def test_disks_app_interactive_list_directories_default(): "./dir1/dir11": [], "./dir1/dir13": [], } - client.rm("dir1") - client.rm(".dir3") + client.rm("dir1", recursive=True) + client.rm(".dir3", recursive=True) assert client.ls(".", recursive=True, show_hidden=False) == {".": []} @@ -304,7 +308,7 @@ def test_disks_app_interactive_cp_and_read(): assert read_text == initial_text os.remove("a.txt") client.rm("a.txt") - client.rm("/dir1") + client.rm("/dir1", recursive=True) def test_disks_app_interactive_test_move_and_write(): From 9e92aed2de6594524070bb6dac8e11d07616c954 Mon Sep 17 00:00:00 2001 From: divanik Date: Mon, 1 Jul 2024 13:24:13 +0000 Subject: [PATCH 115/417] Correct style check --- programs/disks/CommandCopy.cpp | 6 ++++++ programs/disks/CommandMove.cpp | 6 ++++++ programs/disks/CommandRemove.cpp | 6 ++++++ 3 files changed, 18 insertions(+) diff --git a/programs/disks/CommandCopy.cpp b/programs/disks/CommandCopy.cpp index eef87535e51..62eb1cad6ab 100644 --- a/programs/disks/CommandCopy.cpp +++ b/programs/disks/CommandCopy.cpp @@ -7,6 +7,12 @@ namespace DB { +namespace ErrorCodes +{ + extern const int BAD_ARGUMENTS; +} + + class CommandCopy final : public ICommand { public: diff --git a/programs/disks/CommandMove.cpp b/programs/disks/CommandMove.cpp index 22c1f851174..40b698c9340 100644 --- a/programs/disks/CommandMove.cpp +++ b/programs/disks/CommandMove.cpp @@ -4,6 +4,12 @@ namespace DB { +namespace ErrorCodes +{ + extern const int BAD_ARGUMENTS; +} + + class CommandMove final : public ICommand { public: diff --git a/programs/disks/CommandRemove.cpp b/programs/disks/CommandRemove.cpp index caa70905bef..3b6ad018fb8 100644 --- a/programs/disks/CommandRemove.cpp +++ b/programs/disks/CommandRemove.cpp @@ -5,6 +5,12 @@ namespace DB { +namespace ErrorCodes +{ + extern const int BAD_ARGUMENTS; +} + + class CommandRemove final : public ICommand { public: From c43ce89a6254bf790467129bd911e9a6e32631d9 Mon Sep 17 00:00:00 2001 From: Maksim Kita Date: Mon, 1 Jul 2024 16:27:10 +0300 Subject: [PATCH 116/417] PostgreSQL source cancel query comments --- src/Processors/Sources/PostgreSQLSource.cpp | 8 ++++++++ 1 file changed, 8 insertions(+) diff --git a/src/Processors/Sources/PostgreSQLSource.cpp b/src/Processors/Sources/PostgreSQLSource.cpp index 37a84d9fe96..a3d6fd691d8 100644 --- a/src/Processors/Sources/PostgreSQLSource.cpp +++ b/src/Processors/Sources/PostgreSQLSource.cpp @@ -193,7 +193,15 @@ PostgreSQLSource::~PostgreSQLSource() { if (stream) { + /** Internally libpqxx::stream_from runs PostgreSQL copy query `COPY query TO STDOUT`. + * During transaction abort we try to execute PostgreSQL `ROLLBACK` command and if + * copy query is not cancelled, we wait until it finishes. + */ tx->conn().cancel_query(); + + /** If stream is not closed, libpqxx::stream_from closes stream in destructor, but that way + * exception is added into transaction pending error and we can potentially ignore exception message. + */ stream->close(); } From 6573b5436edc23bc94ee7207a5479bfd0c32064c Mon Sep 17 00:00:00 2001 From: avogar Date: Mon, 1 Jul 2024 10:29:23 +0000 Subject: [PATCH 117/417] Add docs about dynamic binary format --- docs/en/sql-reference/data-types/dynamic.md | 11 + src/Formats/JSONExtractTree.cpp | 666 ++++++++++++++++++++ src/Formats/JSONExtractTree.h | 37 ++ 3 files changed, 714 insertions(+) create mode 100644 src/Formats/JSONExtractTree.cpp create mode 100644 src/Formats/JSONExtractTree.h diff --git a/docs/en/sql-reference/data-types/dynamic.md b/docs/en/sql-reference/data-types/dynamic.md index 955fd54e641..f45feb9fe5e 100644 --- a/docs/en/sql-reference/data-types/dynamic.md +++ b/docs/en/sql-reference/data-types/dynamic.md @@ -493,3 +493,14 @@ SELECT count(), dynamicType(d), _part FROM test GROUP BY _part, dynamicType(d) O ``` As we can see, ClickHouse kept the most frequent types `UInt64` and `Array(UInt64)` and casted all other types to `String`. + + +### Binary output format + +In [RowBinary](../../interfaces/formats.md#rowbinary-rowbinary) format values of `Dynamic` type are serialized in the following format: + +```text + +``` + +See the [data types binary encoding specification](../../sql-reference/data-types/data-types-binary-encoding.md) diff --git a/src/Formats/JSONExtractTree.cpp b/src/Formats/JSONExtractTree.cpp new file mode 100644 index 00000000000..6a4241003f1 --- /dev/null +++ b/src/Formats/JSONExtractTree.cpp @@ -0,0 +1,666 @@ +//#include +// +//#include +//#include +//#include +//#include +// +//#include +//#include +//#include +// +//#include +//#include +//#include +// +//namespace DB +//{ +// +//namespace +//{ +// +//const FormatSettings & getFormatSettings() +//{ +// static const FormatSettings instance = [] +// { +// FormatSettings settings; +// settings.json.escape_forward_slashes = false; +// return settings; +// }(); +// return instance; +//} +// +//template +//void elementToString(const Element & element, WriteBuffer & buf) +//{ +// if (element.isInt64()) +// { +// writeIntText(element.getInt64(), buf); +// return; +// } +// if (element.isUInt64()) +// { +// writeIntText(element.getUInt64(), buf); +// return; +// } +// if (element.isDouble()) +// { +// writeFloatText(element.getDouble(), buf); +// return; +// } +// if (element.isBool()) +// { +// if (element.getBool()) +// writeCString("true", buf); +// else +// writeCString("false", buf); +// return; +// } +// if (element.isString()) +// { +// writeJSONString(element.getString(), buf, getFormatSettings()); +// return; +// } +// if (element.isArray()) +// { +// writeChar('[', buf); +// bool need_comma = false; +// for (auto value : element.getArray()) +// { +// if (std::exchange(need_comma, true)) +// writeChar(',', buf); +// elementToString(value, buf); +// } +// writeChar(']', buf); +// return; +// } +// if (element.isObject()) +// { +// writeChar('{', buf); +// bool need_comma = false; +// for (auto [key, value] : element.getObject()) +// { +// if (std::exchange(need_comma, true)) +// writeChar(',', buf); +// writeJSONString(key, buf, getFormatSettings()); +// writeChar(':', buf); +// elementToString(value, buf); +// } +// writeChar('}', buf); +// return; +// } +// if (element.isNull()) +// { +// writeCString("null", buf); +// return; +// } +//} +// +//template +//class NumericNode : public JSONExtractTree::Node +//{ +//public: +// NumericNode(bool convert_bool_to_integer_) : convert_bool_to_integer(convert_bool_to_integer_) {} +// +// bool insertResultToColumn(IColumn & dest, const Element & element) override +// { +// NumberType value; +// if (!tryGetValue(element, value)) +// return false; +// +// auto & col_vec = assert_cast &>(dest); +// col_vec.insertValue(value); +// return true; +// } +// +// bool tryGetValue(const Element & element, NumberType & value) +// { +// switch (element.type()) +// { +// case ElementType::DOUBLE: +// if constexpr (std::is_floating_point_v) +// { +// /// We permit inaccurate conversion of double to float. +// /// Example: double 0.1 from JSON is not representable in float. +// /// But it will be more convenient for user to perform conversion. +// value = static_cast(element.getDouble()); +// } +// else if (!accurate::convertNumeric(element.getDouble(), value)) +// return false; +// break; +// case ElementType::UINT64: +// if (!accurate::convertNumeric(element.getUInt64(), value)) +// return false; +// break; +// case ElementType::INT64: +// if (!accurate::convertNumeric(element.getInt64(), value)) +// return false; +// break; +// case ElementType::BOOL: +// if constexpr (is_integer) +// { +// if (convert_bool_to_integer) +// { +// value = static_cast(element.getBool()); +// break; +// } +// } +// return false; +// case ElementType::STRING: +// { +// auto rb = ReadBufferFromMemory{element.getString()}; +// if constexpr (std::is_floating_point_v) +// { +// if (!tryReadFloatText(value, rb) || !rb.eof()) +// return false; +// } +// else +// { +// if (tryReadIntText(value, rb) && rb.eof()) +// break; +// +// /// Try to parse float and convert it to integer. +// Float64 tmp_float; +// rb.position() = rb.buffer().begin(); +// if (!tryReadFloatText(tmp_float, rb) || !rb.eof()) +// return false; +// +// if (!accurate::convertNumeric(tmp_float, value)) +// return false; +// } +// break; +// } +// case ElementType::NULL_VALUE: +// { +// if () +// } +// default: +// return false; +// } +// +// return true; +// } +// +//private: +// bool convert_bool_to_integer; +//}; +// +//template +//class LowCardinalityNumericNode : public NumericNode +//{ +// bool insertResultToColumn(IColumn & dest, const Element & element) override +// { +// NumberType value; +// if (!tryGetValue(element, value)) +// return false; +// +// auto & col_lc = assert_cast(dest); +// col_lc.insertData(reinterpret_cast(&value), sizeof(value)); +// return true; +// } +//}; +// +//template +//class StringNode : public JSONExtractTree::Node +//{ +//public: +// bool insertResultToColumn(IColumn & dest, const Element & element) override +// { +// if (element.isNull()) +// return false; +// +// if (!element.isString()) +// { +// ColumnString & col_str = assert_cast(dest); +// auto & chars = col_str.getChars(); +// WriteBufferFromVector buf(chars, AppendModeTag()); +// elementToString(element, buf); +// buf.finalize(); +// chars.push_back(0); +// col_str.getOffsets().push_back(chars.size()); +// return true; +// } +// else +// { +// auto str = element.getString(); +// ColumnString & col_str = assert_cast(dest); +// col_str.insertData(str.data(), str.size()); +// } +// return true; +// } +//}; +// +//template +//class LowCardinalityStringNode : public JSONExtractTree::Node +//{ +// bool insertResultToColumn(IColumn & dest, const Element & element) override +// { +// if (element.isNull()) +// return false; +// +// if (!element.isString()) +// { +// ColumnString & col_str = assert_cast(dest); +// auto & chars = col_str.getChars(); +// WriteBufferFromVector buf(chars, AppendModeTag()); +// elementToString(element, buf); +// buf.finalize(); +// chars.push_back(0); +// col_str.getOffsets().push_back(chars.size()); +// return true; +// } +// else +// { +// auto str = element.getString(); +// ColumnString & col_str = assert_cast(dest); +// col_str.insertData(str.data(), str.size()); +// } +// return true; +// } +//}; +// +// +// +// +// +// +//class LowCardinalityFixedStringNode : public Node +//{ +//public: +// explicit LowCardinalityFixedStringNode(const size_t fixed_length_) : fixed_length(fixed_length_) { } +// bool insertResultToColumn(IColumn & dest, const Element & element) override +// { +// // If element is an object we delegate the insertion to JSONExtractRawImpl +// if (element.isObject()) +// return JSONExtractRawImpl::insertResultToLowCardinalityFixedStringColumn(dest, element, fixed_length); +// else if (!element.isString()) +// return false; +// +// auto str = element.getString(); +// if (str.size() > fixed_length) +// return false; +// +// // For the non low cardinality case of FixedString, the padding is done in the FixedString Column implementation. +// // In order to avoid having to pass the data to a FixedString Column and read it back (which would slow down the execution) +// // the data is padded here and written directly to the Low Cardinality Column +// if (str.size() == fixed_length) +// { +// assert_cast(dest).insertData(str.data(), str.size()); +// } +// else +// { +// String padded_str(str); +// padded_str.resize(fixed_length, '\0'); +// +// assert_cast(dest).insertData(padded_str.data(), padded_str.size()); +// } +// return true; +// } +// +//private: +// const size_t fixed_length; +//}; +// +//class UUIDNode : public Node +//{ +//public: +// bool insertResultToColumn(IColumn & dest, const Element & element) override +// { +// if (!element.isString()) +// return false; +// +// auto uuid = parseFromString(element.getString()); +// if (dest.getDataType() == TypeIndex::LowCardinality) +// { +// ColumnLowCardinality & col_low = assert_cast(dest); +// col_low.insertData(reinterpret_cast(&uuid), sizeof(uuid)); +// } +// else +// { +// assert_cast(dest).insert(uuid); +// } +// return true; +// } +//}; +// +//template +//class DecimalNode : public Node +//{ +//public: +// explicit DecimalNode(DataTypePtr data_type_) : data_type(data_type_) {} +// bool insertResultToColumn(IColumn & dest, const Element & element) override +// { +// const auto * type = assert_cast *>(data_type.get()); +// +// DecimalType value{}; +// +// switch (element.type()) +// { +// case ElementType::DOUBLE: +// value = convertToDecimal, DataTypeDecimal>( +// element.getDouble(), type->getScale()); +// break; +// case ElementType::UINT64: +// value = convertToDecimal, DataTypeDecimal>( +// element.getUInt64(), type->getScale()); +// break; +// case ElementType::INT64: +// value = convertToDecimal, DataTypeDecimal>( +// element.getInt64(), type->getScale()); +// break; +// case ElementType::STRING: { +// auto rb = ReadBufferFromMemory{element.getString()}; +// if (!SerializationDecimal::tryReadText(value, rb, DecimalUtils::max_precision, type->getScale())) +// return false; +// break; +// } +// default: +// return false; +// } +// +// assert_cast &>(dest).insertValue(value); +// return true; +// } +// +//private: +// DataTypePtr data_type; +//}; +// +//class FixedStringNode : public Node +//{ +//public: +// bool insertResultToColumn(IColumn & dest, const Element & element) override +// { +// if (element.isNull()) +// return false; +// +// if (!element.isString()) +// return JSONExtractRawImpl::insertResultToFixedStringColumn(dest, element, {}); +// +// auto str = element.getString(); +// auto & col_str = assert_cast(dest); +// if (str.size() > col_str.getN()) +// return false; +// col_str.insertData(str.data(), str.size()); +// +// return true; +// } +//}; +// +//template +//class EnumNode : public Node +//{ +//public: +// explicit EnumNode(const std::vector> & name_value_pairs_) : name_value_pairs(name_value_pairs_) +// { +// for (const auto & name_value_pair : name_value_pairs) +// { +// name_to_value_map.emplace(name_value_pair.first, name_value_pair.second); +// only_values.emplace(name_value_pair.second); +// } +// } +// +// bool insertResultToColumn(IColumn & dest, const Element & element) override +// { +// auto & col_vec = assert_cast &>(dest); +// +// if (element.isInt64()) +// { +// Type value; +// if (!accurate::convertNumeric(element.getInt64(), value) || !only_values.contains(value)) +// return false; +// col_vec.insertValue(value); +// return true; +// } +// +// if (element.isUInt64()) +// { +// Type value; +// if (!accurate::convertNumeric(element.getUInt64(), value) || !only_values.contains(value)) +// return false; +// col_vec.insertValue(value); +// return true; +// } +// +// if (element.isString()) +// { +// auto value = name_to_value_map.find(element.getString()); +// if (value == name_to_value_map.end()) +// return false; +// col_vec.insertValue(value->second); +// return true; +// } +// +// return false; +// } +// +//private: +// std::vector> name_value_pairs; +// std::unordered_map name_to_value_map; +// std::unordered_set only_values; +//}; +// +//class NullableNode : public Node +//{ +//public: +// explicit NullableNode(std::unique_ptr nested_) : nested(std::move(nested_)) {} +// +// bool insertResultToColumn(IColumn & dest, const Element & element) override +// { +// if (dest.getDataType() == TypeIndex::LowCardinality) +// { +// /// We do not need to handle nullability in that case +// /// because nested node handles LowCardinality columns and will call proper overload of `insertData` +// return nested->insertResultToColumn(dest, element); +// } +// +// ColumnNullable & col_null = assert_cast(dest); +// if (!nested->insertResultToColumn(col_null.getNestedColumn(), element)) +// return false; +// col_null.getNullMapColumn().insertValue(0); +// return true; +// } +// +//private: +// std::unique_ptr nested; +//}; +// +//class ArrayNode : public Node +//{ +//public: +// explicit ArrayNode(std::unique_ptr nested_) : nested(std::move(nested_)) {} +// +// bool insertResultToColumn(IColumn & dest, const Element & element) override +// { +// if (!element.isArray()) +// return false; +// +// auto array = element.getArray(); +// +// ColumnArray & col_arr = assert_cast(dest); +// auto & data = col_arr.getData(); +// size_t old_size = data.size(); +// bool were_valid_elements = false; +// +// for (auto value : array) +// { +// if (nested->insertResultToColumn(data, value)) +// were_valid_elements = true; +// else +// data.insertDefault(); +// } +// +// if (!were_valid_elements) +// { +// data.popBack(data.size() - old_size); +// return false; +// } +// +// col_arr.getOffsets().push_back(data.size()); +// return true; +// } +// +//private: +// std::unique_ptr nested; +//}; +// +//class TupleNode : public Node +//{ +//public: +// TupleNode(std::vector> nested_, const std::vector & explicit_names_) : nested(std::move(nested_)), explicit_names(explicit_names_) +// { +// for (size_t i = 0; i != explicit_names.size(); ++i) +// name_to_index_map.emplace(explicit_names[i], i); +// } +// +// bool insertResultToColumn(IColumn & dest, const Element & element) override +// { +// ColumnTuple & tuple = assert_cast(dest); +// size_t old_size = dest.size(); +// bool were_valid_elements = false; +// +// auto set_size = [&](size_t size) +// { +// for (size_t i = 0; i != tuple.tupleSize(); ++i) +// { +// auto & col = tuple.getColumn(i); +// if (col.size() != size) +// { +// if (col.size() > size) +// col.popBack(col.size() - size); +// else +// while (col.size() < size) +// col.insertDefault(); +// } +// } +// }; +// +// if (element.isArray()) +// { +// auto array = element.getArray(); +// auto it = array.begin(); +// +// for (size_t index = 0; (index != nested.size()) && (it != array.end()); ++index) +// { +// if (nested[index]->insertResultToColumn(tuple.getColumn(index), *it++)) +// were_valid_elements = true; +// else +// tuple.getColumn(index).insertDefault(); +// } +// +// set_size(old_size + static_cast(were_valid_elements)); +// return were_valid_elements; +// } +// +// if (element.isObject()) +// { +// auto object = element.getObject(); +// if (name_to_index_map.empty()) +// { +// auto it = object.begin(); +// for (size_t index = 0; (index != nested.size()) && (it != object.end()); ++index) +// { +// if (nested[index]->insertResultToColumn(tuple.getColumn(index), (*it++).second)) +// were_valid_elements = true; +// else +// tuple.getColumn(index).insertDefault(); +// } +// } +// else +// { +// for (const auto & [key, value] : object) +// { +// auto index = name_to_index_map.find(key); +// if (index != name_to_index_map.end()) +// { +// if (nested[index->second]->insertResultToColumn(tuple.getColumn(index->second), value)) +// were_valid_elements = true; +// } +// } +// } +// +// set_size(old_size + static_cast(were_valid_elements)); +// return were_valid_elements; +// } +// +// return false; +// } +// +//private: +// std::vector> nested; +// std::vector explicit_names; +// std::unordered_map name_to_index_map; +//}; +// +//class MapNode : public Node +//{ +//public: +// MapNode(std::unique_ptr key_, std::unique_ptr value_) : key(std::move(key_)), value(std::move(value_)) { } +// +// bool insertResultToColumn(IColumn & dest, const Element & element) override +// { +// if (!element.isObject()) +// return false; +// +// ColumnMap & map_col = assert_cast(dest); +// auto & offsets = map_col.getNestedColumn().getOffsets(); +// auto & tuple_col = map_col.getNestedData(); +// auto & key_col = tuple_col.getColumn(0); +// auto & value_col = tuple_col.getColumn(1); +// size_t old_size = tuple_col.size(); +// +// auto object = element.getObject(); +// auto it = object.begin(); +// for (; it != object.end(); ++it) +// { +// auto pair = *it; +// +// /// Insert key +// key_col.insertData(pair.first.data(), pair.first.size()); +// +// /// Insert value +// if (!value->insertResultToColumn(value_col, pair.second)) +// value_col.insertDefault(); +// } +// +// offsets.push_back(old_size + object.size()); +// return true; +// } +// +//private: +// std::unique_ptr key; +// std::unique_ptr value; +//}; +// +//class VariantNode : public Node +//{ +//public: +// VariantNode(std::vector> variant_nodes_, std::vector order_) : variant_nodes(std::move(variant_nodes_)), order(std::move(order_)) { } +// +// bool insertResultToColumn(IColumn & dest, const Element & element) override +// { +// auto & column_variant = assert_cast(dest); +// for (size_t i : order) +// { +// auto & variant = column_variant.getVariantByGlobalDiscriminator(i); +// if (variant_nodes[i]->insertResultToColumn(variant, element)) +// { +// column_variant.getLocalDiscriminators().push_back(column_variant.localDiscriminatorByGlobal(i)); +// column_variant.getOffsets().push_back(variant.size() - 1); +// return true; +// } +// } +// +// return false; +// } +// +//private: +// std::vector> variant_nodes; +// /// Order in which we should try variants nodes. +// /// For example, String should be always the last one. +// std::vector order; +//}; +// +//} +// +//} diff --git a/src/Formats/JSONExtractTree.h b/src/Formats/JSONExtractTree.h new file mode 100644 index 00000000000..f07c974f595 --- /dev/null +++ b/src/Formats/JSONExtractTree.h @@ -0,0 +1,37 @@ +#pragma once +#include +#include + +namespace DB +{ + +template +struct JSONExtractTree +{ + class Node + { + public: + Node() = default; + virtual ~Node() = default; + virtual bool insertResultToColumn(IColumn &, const Element &) = 0; + }; + + struct Settings + { + bool convert_bool_to_integer = true; + bool type_json_infer_numbers_from_strings = true; + bool type_json_infer_date = true; + bool type_json_infer_datetime = true; + bool type_json_infer_ipv4 = true; + bool type_json_infer_ipv6 = true; + bool type_json_infer_uuid = true; + bool insert_null_as_default = true; + }; + + static std::unique_ptr build(const DataTypePtr & type, const Settings & settings, const char * source_for_exception_message); +}; + +template +void elementToString(const Element & element, WriteBuffer & buf); + +} From a1630073258c1790608aa726e428745b08d464e2 Mon Sep 17 00:00:00 2001 From: divanik Date: Mon, 1 Jul 2024 15:33:34 +0000 Subject: [PATCH 118/417] Change creation criteria of a local disk --- programs/disks/DisksApp.cpp | 2 +- src/Disks/DiskSelector.cpp | 8 +++++--- src/Disks/DiskSelector.h | 8 +++----- 3 files changed, 9 insertions(+), 9 deletions(-) diff --git a/programs/disks/DisksApp.cpp b/programs/disks/DisksApp.cpp index 392fca8e035..0898b692095 100644 --- a/programs/disks/DisksApp.cpp +++ b/programs/disks/DisksApp.cpp @@ -484,7 +484,7 @@ int DisksApp::main(const std::vector & /*args*/) auto validator = [](const Poco::Util::AbstractConfiguration &, const std::string &, const std::string &) { return true; }; constexpr auto config_prefix = "storage_configuration.disks"; - auto disk_selector = std::make_shared(std::unordered_set{"cache", "encrypted"}, /*create_local=*/true); + auto disk_selector = std::make_shared(std::unordered_set{"cache", "encrypted"}); disk_selector->initialize(config(), config_prefix, global_context, validator); std::vector>> disks_with_path; diff --git a/src/Disks/DiskSelector.cpp b/src/Disks/DiskSelector.cpp index f3b4893e820..f45d12618bf 100644 --- a/src/Disks/DiskSelector.cpp +++ b/src/Disks/DiskSelector.cpp @@ -66,9 +66,11 @@ void DiskSelector::initialize( default_disk_name, std::make_shared(default_disk_name, context->getPath(), 0, context, config, config_prefix)); } - if (!has_local_disk && create_local) + if (!has_local_disk && (context->getApplicationType() == Context::ApplicationType::DISKS)) + { + throw_away_local_on_update = true; disks.emplace(local_disk_name, std::make_shared(local_disk_name, "/", 0, context, config, config_prefix)); - + } is_initialized = true; } @@ -115,7 +117,7 @@ DiskSelectorPtr DiskSelector::updateFromConfig( } old_disks_minus_new_disks.erase(default_disk_name); - if (create_local) + if (throw_away_local_on_update) { old_disks_minus_new_disks.erase(local_disk_name); } diff --git a/src/Disks/DiskSelector.h b/src/Disks/DiskSelector.h index 8ceb4a58c15..49a1be5cf50 100644 --- a/src/Disks/DiskSelector.h +++ b/src/Disks/DiskSelector.h @@ -20,10 +20,7 @@ class DiskSelector public: static constexpr auto TMP_INTERNAL_DISK_PREFIX = "__tmp_internal_"; - explicit DiskSelector(std::unordered_set skip_types_ = {}, bool create_local_ = false) - : skip_types(skip_types_), create_local(create_local_) - { - } + explicit DiskSelector(std::unordered_set skip_types_ = {}) : skip_types(skip_types_) { } DiskSelector(const DiskSelector & from) = default; using DiskValidator = std::function; @@ -53,7 +50,8 @@ private: void assertInitialized() const; const std::unordered_set skip_types; - const bool create_local; + + bool throw_away_local_on_update = false; }; } From ad3170accae51d8b9ad93da6d4e889e162a576ac Mon Sep 17 00:00:00 2001 From: divanik Date: Mon, 1 Jul 2024 16:06:17 +0000 Subject: [PATCH 119/417] Correct tests --- .../0_stateless/02802_clickhouse_disks_s3_copy.sh | 12 ++++-------- 1 file changed, 4 insertions(+), 8 deletions(-) diff --git a/tests/queries/0_stateless/02802_clickhouse_disks_s3_copy.sh b/tests/queries/0_stateless/02802_clickhouse_disks_s3_copy.sh index 1638a3ff9c3..20b02bcba32 100755 --- a/tests/queries/0_stateless/02802_clickhouse_disks_s3_copy.sh +++ b/tests/queries/0_stateless/02802_clickhouse_disks_s3_copy.sh @@ -15,13 +15,10 @@ function run_test_for_disk() echo "$disk" clickhouse-disks -C "$config" --disk "$disk" --query "write --path-from $config $CLICKHOUSE_DATABASE/test" - clickhouse-disks -C "$config" --log-level test --disk "$disk" --query "copy $CLICKHOUSE_DATABASE/test $CLICKHOUSE_DATABASE/test.copy" |& { + clickhouse-disks -C "$config" --log-level test --disk "$disk" --query "copy -r $CLICKHOUSE_DATABASE/test $CLICKHOUSE_DATABASE/test.copy" |& { grep -o -e "Single part upload has completed." -e "Single operation copy has completed." } - clickhouse-disks -C "$config" --disk "$disk" --query "remove $CLICKHOUSE_DATABASE/test" - # NOTE: this is due to "copy" does works like "cp -R from to/" instead of "cp from to" - clickhouse-disks -C "$config" --disk "$disk" --query "remove $CLICKHOUSE_DATABASE/test.copy/test" - clickhouse-disks -C "$config" --disk "$disk" --query "remove $CLICKHOUSE_DATABASE/test.copy" + clickhouse-disks -C "$config" --disk "$disk" --query "remove -r $CLICKHOUSE_DATABASE/test" } function run_test_copy_from_s3_to_s3(){ @@ -31,11 +28,10 @@ function run_test_copy_from_s3_to_s3(){ echo "copy from $disk_src to $disk_dest" clickhouse-disks -C "$config" --disk "$disk_src" --query "write --path-from $config $CLICKHOUSE_DATABASE/test" - clickhouse-disks -C "$config" --log-level test --query "copy --disk-from $disk_src --disk-to $disk_dest $CLICKHOUSE_DATABASE/test $CLICKHOUSE_DATABASE/test.copy" |& { + clickhouse-disks -C "$config" --log-level test --query "copy -r --disk-from $disk_src --disk-to $disk_dest $CLICKHOUSE_DATABASE/test $CLICKHOUSE_DATABASE/test.copy" |& { grep -o -e "Single part upload has completed." -e "Single operation copy has completed." } - clickhouse-disks -C "$config" --disk "$disk_dest" --query "remove $CLICKHOUSE_DATABASE/test.copy/test" - clickhouse-disks -C "$config" --disk "$disk_dest" --query "remove $CLICKHOUSE_DATABASE/test.copy" + clickhouse-disks -C "$config" --disk "$disk_dest" --query "remove -r $CLICKHOUSE_DATABASE/test.copy" } run_test_for_disk s3_plain_native_copy From 242b553253b9437c44533986e3655f4ff770670f Mon Sep 17 00:00:00 2001 From: divanik Date: Mon, 1 Jul 2024 17:35:07 +0000 Subject: [PATCH 120/417] Change unique_ptr creation to shared_ptr --- programs/disks/CommandChangeDirectory.cpp | 2 +- programs/disks/CommandCopy.cpp | 2 +- programs/disks/CommandLink.cpp | 2 +- programs/disks/CommandMkDir.cpp | 2 +- programs/disks/CommandMove.cpp | 2 +- programs/disks/CommandRead.cpp | 2 +- programs/disks/CommandRemove.cpp | 2 +- programs/disks/CommandSwitchDisk.cpp | 2 +- 8 files changed, 8 insertions(+), 8 deletions(-) diff --git a/programs/disks/CommandChangeDirectory.cpp b/programs/disks/CommandChangeDirectory.cpp index 5c4ce737375..11596f88e4b 100644 --- a/programs/disks/CommandChangeDirectory.cpp +++ b/programs/disks/CommandChangeDirectory.cpp @@ -29,7 +29,7 @@ public: CommandPtr makeCommandChangeDirectory() { - return std::make_unique(); + return std::make_shared(); } } diff --git a/programs/disks/CommandCopy.cpp b/programs/disks/CommandCopy.cpp index 62eb1cad6ab..c7b2f7cdcac 100644 --- a/programs/disks/CommandCopy.cpp +++ b/programs/disks/CommandCopy.cpp @@ -94,6 +94,6 @@ public: CommandPtr makeCommandCopy() { - return std::make_unique(); + return std::make_shared(); } } diff --git a/programs/disks/CommandLink.cpp b/programs/disks/CommandLink.cpp index 74707160f67..51aff2ef0af 100644 --- a/programs/disks/CommandLink.cpp +++ b/programs/disks/CommandLink.cpp @@ -31,7 +31,7 @@ public: CommandPtr makeCommandLink() { - return std::make_unique(); + return std::make_shared(); } } diff --git a/programs/disks/CommandMkDir.cpp b/programs/disks/CommandMkDir.cpp index 535936480d9..c6222f326d4 100644 --- a/programs/disks/CommandMkDir.cpp +++ b/programs/disks/CommandMkDir.cpp @@ -34,7 +34,7 @@ public: CommandPtr makeCommandMkDir() { - return std::make_unique(); + return std::make_shared(); } } diff --git a/programs/disks/CommandMove.cpp b/programs/disks/CommandMove.cpp index 40b698c9340..e3d485032e0 100644 --- a/programs/disks/CommandMove.cpp +++ b/programs/disks/CommandMove.cpp @@ -72,7 +72,7 @@ public: CommandPtr makeCommandMove() { - return std::make_unique(); + return std::make_shared(); } } diff --git a/programs/disks/CommandRead.cpp b/programs/disks/CommandRead.cpp index 9f60cca2873..11207578150 100644 --- a/programs/disks/CommandRead.cpp +++ b/programs/disks/CommandRead.cpp @@ -46,7 +46,7 @@ public: CommandPtr makeCommandRead() { - return std::make_unique(); + return std::make_shared(); } } diff --git a/programs/disks/CommandRemove.cpp b/programs/disks/CommandRemove.cpp index 3b6ad018fb8..ad859dd0310 100644 --- a/programs/disks/CommandRemove.cpp +++ b/programs/disks/CommandRemove.cpp @@ -52,7 +52,7 @@ public: CommandPtr makeCommandRemove() { - return std::make_unique(); + return std::make_shared(); } } diff --git a/programs/disks/CommandSwitchDisk.cpp b/programs/disks/CommandSwitchDisk.cpp index 9e5d443ebeb..fa02d991365 100644 --- a/programs/disks/CommandSwitchDisk.cpp +++ b/programs/disks/CommandSwitchDisk.cpp @@ -30,6 +30,6 @@ public: CommandPtr makeCommandSwitchDisk() { - return std::make_unique(); + return std::make_shared(); } } From 4d85603e13ba16bdf0eac1b8d45bcf78a88514c7 Mon Sep 17 00:00:00 2001 From: divanik Date: Mon, 1 Jul 2024 17:56:26 +0000 Subject: [PATCH 121/417] Fix build problems --- src/Coordination/Standalone/Context.h | 1 + 1 file changed, 1 insertion(+) diff --git a/src/Coordination/Standalone/Context.h b/src/Coordination/Standalone/Context.h index d3bbfececed..971689a9215 100644 --- a/src/Coordination/Standalone/Context.h +++ b/src/Coordination/Standalone/Context.h @@ -132,6 +132,7 @@ public: { KEEPER, SERVER, + DISKS }; void setApplicationType(ApplicationType) {} From d81e8f3d2d897d4dd364610cc56e8a95e546fb4a Mon Sep 17 00:00:00 2001 From: avogar Date: Mon, 1 Jul 2024 18:54:28 +0000 Subject: [PATCH 122/417] Bump CI From 37489dc31ff66bb561162016ecd1b43151d5c9be Mon Sep 17 00:00:00 2001 From: Nikita Fomichev Date: Tue, 2 Jul 2024 12:32:38 +0200 Subject: [PATCH 123/417] Add tests for base64URLEncode and base64URLDecode --- .../03167_base64_url_functions_sh.reference | 0 .../03167_base64_url_functions_sh.sh | 191 ++++++++++++++++++ 2 files changed, 191 insertions(+) create mode 100755 tests/queries/0_stateless/03167_base64_url_functions_sh.reference create mode 100755 tests/queries/0_stateless/03167_base64_url_functions_sh.sh diff --git a/tests/queries/0_stateless/03167_base64_url_functions_sh.reference b/tests/queries/0_stateless/03167_base64_url_functions_sh.reference new file mode 100755 index 00000000000..e69de29bb2d diff --git a/tests/queries/0_stateless/03167_base64_url_functions_sh.sh b/tests/queries/0_stateless/03167_base64_url_functions_sh.sh new file mode 100755 index 00000000000..d4f56d8e18c --- /dev/null +++ b/tests/queries/0_stateless/03167_base64_url_functions_sh.sh @@ -0,0 +1,191 @@ +#!/usr/bin/env bash +# Tags: no-fastest, no-debug +# shellcheck disable=SC2155 + +set -e + +CURDIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd) +# shellcheck source=../shell_config.sh +. "$CURDIR"/../shell_config.sh + + +urls=( + "http://www.example.com" + "https://secure.example.com" + "http://example.com" + "https://www.example.org" + "https://subdomain.example.com" + "http://sub.sub.example.com" + "http://192.168.1.1" + "https://[2001:db8:85a3:8d3:1319:8a2e:370:7348]" + "http://example.com:8080" + "https://example.com:443" + "http://example.com/path/to/page.html" + "https://example.com/path/with/trailing/slash/" + "http://example.com/search?q=query&lang=en" + "https://example.com/path?param1=value1¶m2=value2" + "http://example.com/page.html#section1" + "https://example.com/document.pdf#page=10" + "http://user:password@example.com" + "https://user@example.com" + "https://user:pass@sub.example.com:8080/path/page.html?query=123#fragment" + "http://example.com/path%20with%20spaces" + "https://example.com/search?q=encode+this" + "http://例子.测试" + "https://mañana.com" + "http://example.com/%E2%82%AC" + "data:text/plain;base64,SGVsbG8sIFdvcmxkIQ==" + "file:///C:/path/to/file.txt" + "file:///home/user/document.pdf" + "ftp://ftp.example.com/pub/file.zip" + "ftps://secure-ftp.example.com/private/doc.pdf" + "mailto:user@example.com" + "mailto:user@example.com?subject=Hello&body=How%20are%20you" + "git://github.com/user/repo.git" + "ssh://user@host.xz:port/path/to/repo.git" + "https://example.com/path(1)/[2]/{3}" + "http://example.com/path;param?query,value" + "" + "http://" + "example.com" + "http:" + "//" + "?query=value" + "#fragment" + "http://?#" + "http://xn--bcher-kva.ch" + "https://xn--bcher-kva.xn--tckwe/xn--8ws00zhy3a/%E6%B8%AC%E8%A9%A6.php?xn--o39an51a5phao35a=xn--mgbh0fb&xn--fiq228c5hs=test" + "https://xn--3e0b707e.xn--79-8kcre8v3a/%ED%85%8C%EC%8A%A4%ED%8A%B8/%ED%8C%8C%EC%9D%BC.jsp?xn--i1b6b1a6a2e=xn--9t4b11yi5a&xn--3e0b707e=xn--80aaa1cbgbm" + "https://example.com/path?param=value&special=!@#$%^&*()" + + "http://example.com/path/with/~tilde" + "https://example.com/path/with/\`backtick\`" + + "https://example.com/path?param1=value1¶m2=value2¶m3=value3#section1#section2" + "http://example.com/page?q1=v1&q2=v2#frag1#frag2#frag3" + + "https://example.com/☃/snowman" + "http://example.com/path/⽇本語" + "https://example.com/ü/ñ/path?q=ç" + + "https://example.com/path/to/very/long/url/that/exceeds/two/hundred/and/fifty/five/characters/lorem/ipsum/dolor/sit/amet/consectetur/adipiscing/elit/sed/do/eiusmod/tempor/incididunt/ut/labore/et/dolore/magna/aliqua/ut/enim/ad/minim/veniam/quis/nostrud/exercitation/ullamco/laboris/nisi/ut/aliquip/ex/ea/commodo/consequat" + + "https://example.com//path///to//file" + "http://example.com/path?param1=value1&¶m2=value2&&¶m3=value3" + + "http://example.com/%70%61%74%68?%70%61%72%61%6d=%76%61%6c%75%65#%66%72%61%67%6d%65%6e%74" + + "HtTpS://ExAmPlE.cOm/PaTh" + "http://EXAMPLE.COM/PATH" + + "http://127.0.0.1:8080/path" + "https://[::1]/path" + "http://[2001:0db8:85a3:0000:0000:8a2e:0370:7334]:8080/path" + + "http://example.com:65535/path" + "https://example.com:0/path" + + "data:image/png;base64,iVBORw0KGgoAAAANSUhEUgAAAAEAAAABCAYAAAAfFcSJAAAACklEQVR4nGMAAQAABQABDQottAAAAABJRU5ErkJggg==" + + "https://user:password@example.com:8080/path?query=value#fragment" + "ftp://anonymous:password@ftp.example.com/pub/" + + "http://example.com/path%20with%20spaces" + "https://example.com/search?q=query%20with%20spaces" + + "https://www.mañana.com/path" + "http://例子.测试/path" + "https://рм.рф/path" + + "https://user:pass@sub.example.com:8080/p/a/t/h?query=123&key=value#fragid1" + + "jdbc:mysql://localhost:3306/database" + "market://details?id=com.example.app" + "tel:+1-816-555-1212" + "sms:+18165551212" + + "http://[1080:0:0:0:8:800:200C:417A]/index.html" + "https://[2001:db8::1428:57ab]:8080/path" + + "http://.." + "http://../" + "http://??" + "http://??/" + "http:///a" + "http://example.com??" + "http://example.com??/" + "foo://example.com:8042/over/there?name=ferret#nose" + "//example.com/path" +) + + +base64URLEncode() { + echo -n "$1" | openssl base64 -e -A | tr '+/' '-_' | tr -d '=' +} + +base64URLDecode() { + local len=$((${#1} % 4)) + local result="$1" + if [ $len -eq 2 ]; then result="$1"'==' + elif [ $len -eq 3 ]; then result="$1"'=' + fi + echo "$result" | tr '_-' '/+' | openssl base64 -d -A +} + +test_compare_to_gold_encode() { + local input="$1" + local encode=$(${CLICKHOUSE_CLIENT} --query="SELECT base64URLEncode('$input')") + local encode_gold=$(base64URLEncode $input) + + if [ "$encode" != "$encode_gold" ]; then + echo "Input: $input" + echo "Expected: $encode_gold" + echo "Got: $encode" + fi +} + +test_compare_to_gold_decode() { + local input="$1" + local encode_gold=$(base64URLEncode $input) + local decode=$(${CLICKHOUSE_CLIENT} --query="SELECT base64URLDecode('$encode_gold')") + local decode_gold=$(base64URLDecode $encode_gold) + + if [ "$decode" != "$decode_gold" ]; then + echo "Input: $input" + echo "Expected: $decode_gold" + echo "Got: $decode" + fi +} + +test_compare_to_self() { + local input="$1" + local encode=$(${CLICKHOUSE_CLIENT} --query="SELECT base64URLEncode('$input')") + local decode=$(${CLICKHOUSE_CLIENT} --query="SELECT base64URLDecode('$encode')") + + if [ "$decode" != "$input" ]; then + echo "Input: $input" + echo "Encode: $encode" + echo "Got: $decode" + fi +} + +for url in "${urls[@]}"; do + test_compare_to_gold_encode "$url" +done + +for url in "${urls[@]}"; do + test_compare_to_gold_decode "$url" +done + +for url in "${urls[@]}"; do + test_compare_to_self "$url" +done + +# special case for ' +encode=$(${CLICKHOUSE_CLIENT} --query="SELECT base64URLEncode('http://example.com/!$&\'()*+,;=:@/path')") +decode=$(${CLICKHOUSE_CLIENT} --query="SELECT base64URLDecode('$encode')") +if [ "$decode" != "http://example.com/!$&\'()*+,;=:@/path" ]; then + echo "Special case fail" + echo "Encode: $encode" + echo "Got: $decode" +fi From a2f7d8e13fc597bdfa8a8ebfbc5797dcf767fe7c Mon Sep 17 00:00:00 2001 From: avogar Date: Tue, 2 Jul 2024 10:42:12 +0000 Subject: [PATCH 124/417] Add missing settings in schema inference cache --- src/Formats/EscapingRuleUtils.cpp | 6 ++-- .../Formats/Impl/CSVRowInputFormat.cpp | 2 +- .../Impl/TabSeparatedRowInputFormat.cpp | 5 +-- ...settings_in_csv_tsv_schema_cache.reference | 28 +++++++++++++++ .../03198_settings_in_csv_tsv_schema_cache.sh | 34 +++++++++++++++++++ 5 files changed, 70 insertions(+), 5 deletions(-) create mode 100644 tests/queries/0_stateless/03198_settings_in_csv_tsv_schema_cache.reference create mode 100755 tests/queries/0_stateless/03198_settings_in_csv_tsv_schema_cache.sh diff --git a/src/Formats/EscapingRuleUtils.cpp b/src/Formats/EscapingRuleUtils.cpp index 36d16d8d154..58407a810c5 100644 --- a/src/Formats/EscapingRuleUtils.cpp +++ b/src/Formats/EscapingRuleUtils.cpp @@ -439,13 +439,15 @@ String getAdditionalFormatInfoByEscapingRule(const FormatSettings & settings, Fo case FormatSettings::EscapingRule::CSV: result += fmt::format( ", use_best_effort_in_schema_inference={}, bool_true_representation={}, bool_false_representation={}," - " null_representation={}, delimiter={}, tuple_delimiter={}", + " null_representation={}, delimiter={}, tuple_delimiter={}, try_infer_numbers_from_strings={}, try_infer_strings_from_quoted_tuples={}", settings.csv.use_best_effort_in_schema_inference, settings.bool_true_representation, settings.bool_false_representation, settings.csv.null_representation, settings.csv.delimiter, - settings.csv.tuple_delimiter); + settings.csv.tuple_delimiter, + settings.csv.try_infer_numbers_from_strings, + settings.csv.try_infer_strings_from_quoted_tuples); break; case FormatSettings::EscapingRule::JSON: result += fmt::format( diff --git a/src/Processors/Formats/Impl/CSVRowInputFormat.cpp b/src/Processors/Formats/Impl/CSVRowInputFormat.cpp index dd7d6c6b024..b7f84748f61 100644 --- a/src/Processors/Formats/Impl/CSVRowInputFormat.cpp +++ b/src/Processors/Formats/Impl/CSVRowInputFormat.cpp @@ -616,7 +616,7 @@ void registerCSVSchemaReader(FormatFactory & factory) { String result = getAdditionalFormatInfoByEscapingRule(settings, FormatSettings::EscapingRule::CSV); if (!with_names) - result += fmt::format(", column_names_for_schema_inference={}, try_detect_header={}", settings.column_names_for_schema_inference, settings.csv.try_detect_header); + result += fmt::format(", column_names_for_schema_inference={}, try_detect_header={}, skip_first_lines={}", settings.column_names_for_schema_inference, settings.csv.try_detect_header, settings.csv.skip_first_lines); return result; }); } diff --git a/src/Processors/Formats/Impl/TabSeparatedRowInputFormat.cpp b/src/Processors/Formats/Impl/TabSeparatedRowInputFormat.cpp index 6d4dcba9e60..d2e17e92924 100644 --- a/src/Processors/Formats/Impl/TabSeparatedRowInputFormat.cpp +++ b/src/Processors/Formats/Impl/TabSeparatedRowInputFormat.cpp @@ -440,9 +440,10 @@ void registerTSVSchemaReader(FormatFactory & factory) settings, is_raw ? FormatSettings::EscapingRule::Raw : FormatSettings::EscapingRule::Escaped); if (!with_names) result += fmt::format( - ", column_names_for_schema_inference={}, try_detect_header={}", + ", column_names_for_schema_inference={}, try_detect_header={}, skip_first_lines={}", settings.column_names_for_schema_inference, - settings.tsv.try_detect_header); + settings.tsv.try_detect_header, + settings.tsv.skip_first_lines); return result; }); } diff --git a/tests/queries/0_stateless/03198_settings_in_csv_tsv_schema_cache.reference b/tests/queries/0_stateless/03198_settings_in_csv_tsv_schema_cache.reference new file mode 100644 index 00000000000..aecacd10e00 --- /dev/null +++ b/tests/queries/0_stateless/03198_settings_in_csv_tsv_schema_cache.reference @@ -0,0 +1,28 @@ +c1 Nullable(Int64) +c2 Nullable(Int64) +c3 Nullable(Int64) +a Nullable(Int64) +b Nullable(Int64) +c Nullable(Int64) +2 +a Nullable(Int64) +b Nullable(Int64) +c Nullable(Int64) +a Nullable(String) +b Nullable(Int64) +c Nullable(Int64) +2 +a Nullable(String) +b Nullable(Int64) +c Nullable(Int64) +a Tuple(Nullable(Int64), Nullable(Int64), Nullable(Int64)) +b Nullable(Int64) +c Nullable(Int64) +2 +c1 Nullable(Int64) +c2 Nullable(Int64) +c3 Nullable(Int64) +a Nullable(Int64) +b Nullable(Int64) +c Nullable(Int64) +2 diff --git a/tests/queries/0_stateless/03198_settings_in_csv_tsv_schema_cache.sh b/tests/queries/0_stateless/03198_settings_in_csv_tsv_schema_cache.sh new file mode 100755 index 00000000000..ce53f467823 --- /dev/null +++ b/tests/queries/0_stateless/03198_settings_in_csv_tsv_schema_cache.sh @@ -0,0 +1,34 @@ +#!/usr/bin/env bash + +CUR_DIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd) +# shellcheck source=../shell_config.sh +. "$CUR_DIR"/../shell_config.sh + +echo -e 'a,b,c\n1,2,3' > $CLICKHOUSE_TEST_UNIQUE_NAME.csv +$CLICKHOUSE_LOCAL -nm -q " +DESC file('$CLICKHOUSE_TEST_UNIQUE_NAME.csv') SETTINGS input_format_csv_skip_first_lines=1; +DESC file('$CLICKHOUSE_TEST_UNIQUE_NAME.csv') SETTINGS input_format_csv_skip_first_lines=0; +SELECT count() from system.schema_inference_cache where format = 'CSV' and additional_format_info like '%skip_first_lines%';" + +echo -e 'a,b,c\n"1",2,3' > $CLICKHOUSE_TEST_UNIQUE_NAME.csv +$CLICKHOUSE_LOCAL -nm -q " +DESC file('$CLICKHOUSE_TEST_UNIQUE_NAME.csv') SETTINGS input_format_csv_try_infer_numbers_from_strings=1; +DESC file('$CLICKHOUSE_TEST_UNIQUE_NAME.csv') SETTINGS input_format_csv_try_infer_numbers_from_strings=0; +SELECT count() from system.schema_inference_cache where format = 'CSV' and additional_format_info like '%try_infer_numbers_from_strings%';" + +echo -e 'a,b,c\n"(1,2,3)",2,3' > $CLICKHOUSE_TEST_UNIQUE_NAME.csv +$CLICKHOUSE_LOCAL -nm -q " +DESC file('$CLICKHOUSE_TEST_UNIQUE_NAME.csv') SETTINGS input_format_csv_try_infer_strings_from_quoted_tuples=1; +DESC file('$CLICKHOUSE_TEST_UNIQUE_NAME.csv') SETTINGS input_format_csv_try_infer_strings_from_quoted_tuples=0; +SELECT count() from system.schema_inference_cache where format = 'CSV' and additional_format_info like '%try_infer_strings_from_quoted_tuples%';" + +echo -e 'a\tb\tc\n1\t2\t3' > $CLICKHOUSE_TEST_UNIQUE_NAME.tsv +$CLICKHOUSE_LOCAL -nm -q " +DESC file('$CLICKHOUSE_TEST_UNIQUE_NAME.tsv') SETTINGS input_format_tsv_skip_first_lines=1; +DESC file('$CLICKHOUSE_TEST_UNIQUE_NAME.tsv') SETTINGS input_format_tsv_skip_first_lines=0; +SELECT count() from system.schema_inference_cache where format = 'TSV' and additional_format_info like '%skip_first_lines%';" + + +rm $CLICKHOUSE_TEST_UNIQUE_NAME.csv +rm $CLICKHOUSE_TEST_UNIQUE_NAME.tsv + From c4da270ddb67e0710e35e0ed022b91280d30dc70 Mon Sep 17 00:00:00 2001 From: Nikita Fomichev Date: Tue, 2 Jul 2024 14:01:01 +0200 Subject: [PATCH 125/417] remove no-fastest, no-debug --- tests/queries/0_stateless/03167_base64_url_functions_sh.sh | 1 - 1 file changed, 1 deletion(-) diff --git a/tests/queries/0_stateless/03167_base64_url_functions_sh.sh b/tests/queries/0_stateless/03167_base64_url_functions_sh.sh index d4f56d8e18c..12eea7462df 100755 --- a/tests/queries/0_stateless/03167_base64_url_functions_sh.sh +++ b/tests/queries/0_stateless/03167_base64_url_functions_sh.sh @@ -1,5 +1,4 @@ #!/usr/bin/env bash -# Tags: no-fastest, no-debug # shellcheck disable=SC2155 set -e From cfa86b54ea9977e5730d91e799856238b3f5432e Mon Sep 17 00:00:00 2001 From: Antonio Andelic Date: Tue, 2 Jul 2024 16:33:06 +0200 Subject: [PATCH 126/417] Review fixes --- .../ClusterProxy/executeQuery.cpp | 63 ++++++++++--------- src/Interpreters/ClusterProxy/executeQuery.h | 5 +- src/Interpreters/Context.h | 3 + src/Interpreters/InterpreterSelectQuery.cpp | 4 ++ src/Planner/PlannerJoinTree.cpp | 4 ++ src/Storages/StorageDistributed.cpp | 10 ++- src/Storages/StorageMergeTree.cpp | 6 ++ src/Storages/StorageReplicatedMergeTree.cpp | 6 ++ .../test_parallel_replicas_custom_key/test.py | 6 -- .../test.py | 3 + 10 files changed, 71 insertions(+), 39 deletions(-) diff --git a/src/Interpreters/ClusterProxy/executeQuery.cpp b/src/Interpreters/ClusterProxy/executeQuery.cpp index 47fbf35233a..5d56ef09127 100644 --- a/src/Interpreters/ClusterProxy/executeQuery.cpp +++ b/src/Interpreters/ClusterProxy/executeQuery.cpp @@ -228,6 +228,35 @@ static ThrottlerPtr getThrottler(const ContextPtr & context) return throttler; } +AdditionalShardFilterGenerator +getShardFilterGeneratorForCustomKey(const Cluster & cluster, ContextPtr context, const ColumnsDescription & columns) +{ + if (!context->canUseParallelReplicasCustomKeyForCluster(cluster)) + return {}; + + const auto & settings = context->getSettingsRef(); + auto custom_key_ast = parseCustomKeyForTable(settings.parallel_replicas_custom_key, *context); + if (custom_key_ast == nullptr) + return {}; + + return [my_custom_key_ast = std::move(custom_key_ast), + column_description = columns, + custom_key_type = settings.parallel_replicas_custom_key_filter_type.value, + custom_key_range_lower = settings.parallel_replicas_custom_key_range_lower.value, + custom_key_range_upper = settings.parallel_replicas_custom_key_range_upper.value, + query_context = context, + replica_count = cluster.getShardsInfo().front().per_replica_pools.size()](uint64_t replica_num) -> ASTPtr + { + return getCustomKeyFilterForParallelReplica( + replica_count, + replica_num - 1, + my_custom_key_ast, + {custom_key_type, custom_key_range_lower, custom_key_range_upper}, + column_description, + query_context); + }; +} + void executeQuery( QueryPlan & query_plan, @@ -239,43 +268,17 @@ void executeQuery( LoggerPtr log, ContextPtr context, const SelectQueryInfo & query_info, - const ColumnsDescription & columns, const ExpressionActionsPtr & sharding_key_expr, const std::string & sharding_key_column_name, const DistributedSettings & distributed_settings, + AdditionalShardFilterGenerator shard_filter_generator, bool is_remote_function) { const Settings & settings = context->getSettingsRef(); - if (settings.max_distributed_depth && context->getClientInfo().distributed_depth >= settings.max_distributed_depth) throw Exception(ErrorCodes::TOO_LARGE_DISTRIBUTED_DEPTH, "Maximum distributed depth exceeded"); - ClusterProxy::AdditionalShardFilterGenerator shard_filter_generator; - if (context->canUseParallelReplicasCustomKeyForCluster(*query_info.getCluster())) - { - if (auto custom_key_ast = parseCustomKeyForTable(settings.parallel_replicas_custom_key, *context)) - { - shard_filter_generator = - [my_custom_key_ast = std::move(custom_key_ast), - column_description = columns, - custom_key_type = settings.parallel_replicas_custom_key_filter_type.value, - custom_key_range_lower = settings.parallel_replicas_custom_key_range_lower.value, - custom_key_range_upper = settings.parallel_replicas_custom_key_range_upper.value, - query_context = context, - replica_count = query_info.getCluster()->getShardsInfo().front().per_replica_pools.size()](uint64_t replica_num) -> ASTPtr - { - return getCustomKeyFilterForParallelReplica( - replica_count, - replica_num - 1, - my_custom_key_ast, - {custom_key_type, custom_key_range_lower, custom_key_range_upper}, - column_description, - query_context); - }; - } - } - const ClusterPtr & not_optimized_cluster = query_info.cluster; std::vector plans; @@ -599,6 +602,8 @@ void executeQueryWithParallelReplicasCustomKey( ClusterProxy::SelectStreamFactory select_stream_factory = ClusterProxy::SelectStreamFactory(header, columns_object, snapshot, processed_stage); + auto shard_filter_generator = getShardFilterGeneratorForCustomKey(*query_info.getCluster(), context, columns); + ClusterProxy::executeQuery( query_plan, header, @@ -609,11 +614,11 @@ void executeQueryWithParallelReplicasCustomKey( getLogger("executeQueryWithParallelReplicasCustomKey"), context, query_info, - columns, /*sharding_key_expr=*/nullptr, /*sharding_key_column_name=*/{}, /*distributed_settings=*/{}, - /*is_remote_function= */ false); + shard_filter_generator, + /*is_remote_function=*/false); } void executeQueryWithParallelReplicasCustomKey( diff --git a/src/Interpreters/ClusterProxy/executeQuery.h b/src/Interpreters/ClusterProxy/executeQuery.h index cf60fc3f168..c22fcd24f03 100644 --- a/src/Interpreters/ClusterProxy/executeQuery.h +++ b/src/Interpreters/ClusterProxy/executeQuery.h @@ -52,6 +52,9 @@ class SelectStreamFactory; ContextMutablePtr updateSettingsForCluster(const Cluster & cluster, ContextPtr context, const Settings & settings, const StorageID & main_table); using AdditionalShardFilterGenerator = std::function; +AdditionalShardFilterGenerator +getShardFilterGeneratorForCustomKey(const Cluster & cluster, ContextPtr context, const ColumnsDescription & columns); + /// Execute a distributed query, creating a query plan, from which the query pipeline can be built. /// `stream_factory` object encapsulates the logic of creating plans for a different type of query /// (currently SELECT, DESCRIBE). @@ -65,10 +68,10 @@ void executeQuery( LoggerPtr log, ContextPtr context, const SelectQueryInfo & query_info, - const ColumnsDescription & columns, const ExpressionActionsPtr & sharding_key_expr, const std::string & sharding_key_column_name, const DistributedSettings & distributed_settings, + AdditionalShardFilterGenerator shard_filter_generator, bool is_remote_function); void executeQueryWithParallelReplicas( diff --git a/src/Interpreters/Context.h b/src/Interpreters/Context.h index 680c9ecaa1c..85b9f1d249e 100644 --- a/src/Interpreters/Context.h +++ b/src/Interpreters/Context.h @@ -461,6 +461,9 @@ protected: /// mutation tasks of one mutation executed against different parts of the same table. PreparedSetsCachePtr prepared_sets_cache; + /// this is a mode of parallel replicas where we set parallel_replicas_count and parallel_replicas_offset + /// and generate specific filters on the replicas (e.g. when using parallel replicas with sample key) + /// if we already use a different mode of parallel replicas we want to disable this mode bool offset_parallel_replicas_enabled = true; public: diff --git a/src/Interpreters/InterpreterSelectQuery.cpp b/src/Interpreters/InterpreterSelectQuery.cpp index 571a3c67415..fae204912fc 100644 --- a/src/Interpreters/InterpreterSelectQuery.cpp +++ b/src/Interpreters/InterpreterSelectQuery.cpp @@ -592,6 +592,10 @@ InterpreterSelectQuery::InterpreterSelectQuery( "or it's invalid (settings `parallel_replicas_custom_key`)"); } } + /// We disable prefer_localhost_replica because if one of the replicas is local it will create a single local plan + /// instead of executing the query with multiple replicas + /// We can enable this setting again for custom key parallel replicas when we can generate a plan that will use both a + /// local plan and remote replicas else if (auto * distributed = dynamic_cast(storage.get()); distributed && context->canUseParallelReplicasCustomKeyForCluster(*distributed->getCluster())) { diff --git a/src/Planner/PlannerJoinTree.cpp b/src/Planner/PlannerJoinTree.cpp index 19028839ea7..5c66f69638c 100644 --- a/src/Planner/PlannerJoinTree.cpp +++ b/src/Planner/PlannerJoinTree.cpp @@ -846,6 +846,10 @@ JoinTreeQueryPlan buildQueryPlanForTableExpression(QueryTreeNodePtr table_expres distributed && query_context->canUseParallelReplicasCustomKeyForCluster(*distributed->getCluster())) { planner_context->getMutableQueryContext()->setSetting("distributed_group_by_no_merge", 2); + /// We disable prefer_localhost_replica because if one of the replicas is local it will create a single local plan + /// instead of executing the query with multiple replicas + /// We can enable this setting again for custom key parallel replicas when we can generate a plan that will use both a + /// local plan and remote replicas planner_context->getMutableQueryContext()->setSetting("prefer_localhost_replica", Field{0}); } } diff --git a/src/Storages/StorageDistributed.cpp b/src/Storages/StorageDistributed.cpp index 1dd42d79d88..2cf3ced2904 100644 --- a/src/Storages/StorageDistributed.cpp +++ b/src/Storages/StorageDistributed.cpp @@ -839,7 +839,9 @@ void StorageDistributed::read( SelectQueryInfo modified_query_info = query_info; - if (local_context->getSettingsRef().allow_experimental_analyzer) + const auto & settings = local_context->getSettingsRef(); + + if (settings.allow_experimental_analyzer) { StorageID remote_storage_id = StorageID::createEmpty(); if (!remote_table_function_ptr) @@ -864,7 +866,6 @@ void StorageDistributed::read( header = InterpreterSelectQuery(modified_query_info.query, local_context, SelectQueryOptions(processed_stage).analyze()).getSampleBlock(); } - const auto & settings = local_context->getSettingsRef(); if (!settings.allow_experimental_analyzer) { modified_query_info.query = ClusterProxy::rewriteSelectQuery( @@ -894,6 +895,9 @@ void StorageDistributed::read( storage_snapshot, processed_stage); + auto shard_filter_generator = ClusterProxy::getShardFilterGeneratorForCustomKey( + *modified_query_info.getCluster(), local_context, getInMemoryMetadataPtr()->columns); + ClusterProxy::executeQuery( query_plan, header, @@ -904,10 +908,10 @@ void StorageDistributed::read( log, local_context, modified_query_info, - getInMemoryMetadataPtr()->columns, sharding_key_expr, sharding_key_column_name, distributed_settings, + shard_filter_generator, /* is_remote_function= */ static_cast(owned_cluster)); /// This is a bug, it is possible only when there is no shards to query, and this is handled earlier. diff --git a/src/Storages/StorageMergeTree.cpp b/src/Storages/StorageMergeTree.cpp index 58f64d6158b..061f611927d 100644 --- a/src/Storages/StorageMergeTree.cpp +++ b/src/Storages/StorageMergeTree.cpp @@ -242,6 +242,12 @@ void StorageMergeTree::read( local_context); return; } + else + LOG_WARNING( + log, + "Parallel replicas with custom key will not be used because cluster defined by 'cluster_for_parallel_replicas' ('{}') has " + "multiple shards", + cluster->getName()); } const bool enable_parallel_reading = local_context->canUseParallelReplicasOnFollower() diff --git a/src/Storages/StorageReplicatedMergeTree.cpp b/src/Storages/StorageReplicatedMergeTree.cpp index baa4b26102e..d1e2a9c55b7 100644 --- a/src/Storages/StorageReplicatedMergeTree.cpp +++ b/src/Storages/StorageReplicatedMergeTree.cpp @@ -5490,6 +5490,12 @@ void StorageReplicatedMergeTree::read( local_context); return; } + else + LOG_WARNING( + log, + "Parallel replicas with custom key will not be used because cluster defined by 'cluster_for_parallel_replicas' ('{}') has " + "multiple shards", + cluster->getName()); } readLocalImpl(query_plan, column_names, storage_snapshot, query_info, local_context, max_block_size, num_streams); } diff --git a/tests/integration/test_parallel_replicas_custom_key/test.py b/tests/integration/test_parallel_replicas_custom_key/test.py index 9a2480a77c3..affa3f32cbe 100644 --- a/tests/integration/test_parallel_replicas_custom_key/test.py +++ b/tests/integration/test_parallel_replicas_custom_key/test.py @@ -23,12 +23,6 @@ def start_cluster(): cluster.shutdown() -def create_tables(cluster): - n1 = nodes[0] - n1.query("DROP TABLE IF EXISTS dist_table SYNC") - n1.query(f"DROP TABLE IF EXISTS test_table ON CLUSTER {cluster} SYNC") - - def insert_data(table_name, row_num, all_nodes=False): query = ( f"INSERT INTO {table_name} SELECT number % 4, number FROM numbers({row_num})" diff --git a/tests/integration/test_parallel_replicas_custom_key_failover/test.py b/tests/integration/test_parallel_replicas_custom_key_failover/test.py index 5c1c8ef71ab..f24a24f3238 100644 --- a/tests/integration/test_parallel_replicas_custom_key_failover/test.py +++ b/tests/integration/test_parallel_replicas_custom_key_failover/test.py @@ -53,11 +53,13 @@ def create_tables(cluster, table_name): @pytest.mark.parametrize("use_hedged_requests", [1, 0]) @pytest.mark.parametrize("custom_key", ["sipHash64(key)", "key"]) @pytest.mark.parametrize("filter_type", ["default", "range"]) +@pytest.mark.parametrize("prefer_localhost_replica", [0, 1]) def test_parallel_replicas_custom_key_failover( start_cluster, use_hedged_requests, custom_key, filter_type, + prefer_localhost_replica, ): cluster_name = "test_single_shard_multiple_replicas" table = "test_table" @@ -78,6 +80,7 @@ def test_parallel_replicas_custom_key_failover( "parallel_replicas_custom_key": custom_key, "parallel_replicas_custom_key_filter_type": filter_type, "use_hedged_requests": use_hedged_requests, + "prefer_localhost_replica": prefer_localhost_replica, # avoid considering replica delay on connection choice # otherwise connection can be not distributed evenly among available nodes # and so custom key secondary queries (we check it bellow) From 9821c042f1a046b094fd7d17fb355f572c74cc9f Mon Sep 17 00:00:00 2001 From: gabrielmcg44 Date: Fri, 28 Jun 2024 10:22:26 -0300 Subject: [PATCH 127/417] fix tie handling issue --- src/Functions/array/arrayAUC.cpp | 28 ++++++++++++++++++++-------- 1 file changed, 20 insertions(+), 8 deletions(-) diff --git a/src/Functions/array/arrayAUC.cpp b/src/Functions/array/arrayAUC.cpp index 499fe4ce7b2..878a8fcae34 100644 --- a/src/Functions/array/arrayAUC.cpp +++ b/src/Functions/array/arrayAUC.cpp @@ -103,27 +103,39 @@ private: sorted_labels[i].label = label; } - /// Stable sort is required for for labels to apply in same order if score is equal - std::stable_sort(sorted_labels.begin(), sorted_labels.end(), [](const auto & lhs, const auto & rhs) { return lhs.score > rhs.score; }); + /// Sorting scores in descending order to traverse the ROC curve from left to right + std::sort(sorted_labels.begin(), sorted_labels.end(), [](const auto & lhs, const auto & rhs) { return lhs.score > rhs.score; }); /// We will first calculate non-normalized area. - size_t area = 0; - size_t count_positive = 0; + Float64 area = 0.0; + Float64 prev_score = sorted_labels[0].score; + size_t prev_fp = 0, prev_tp = 0; + size_t curr_fp = 0, curr_tp = 0; for (size_t i = 0; i < size; ++i) { + // Only change the current ROC point when the score changes + if (sorted_labels[i].score != prev_score) { + area += (curr_fp - prev_fp) * (curr_tp + prev_tp) / 2.0; // Trapezoidal area under curve (might degenerate to zero or to a rectangle) + prev_fp = curr_fp; + prev_tp = curr_tp; + prev_score = sorted_labels[i].score; + } + if (sorted_labels[i].label) - ++count_positive; /// The curve moves one step up. No area increase. + curr_tp += 1; /// The curve moves one step up. else - area += count_positive; /// The curve moves one step right. Area is increased by 1 * height = count_positive. + curr_fp += 1; /// The curve moves one step right. } + area += (curr_fp - prev_fp) * (curr_tp + prev_tp) / 2.0; + /// Then divide the area to the area of rectangle. - if (count_positive == 0 || count_positive == size) + if (curr_tp == 0 || curr_tp == size) return std::numeric_limits::quiet_NaN(); - return static_cast(area) / count_positive / (size - count_positive); + return static_cast(area) / curr_tp / (size - curr_tp); } static void vector( From 6a0d912b1c9a35aab2466c50f2e51730215f1a35 Mon Sep 17 00:00:00 2001 From: gabrielmcg44 Date: Fri, 28 Jun 2024 11:05:48 -0300 Subject: [PATCH 128/417] remove trailing spaces --- src/Functions/array/arrayAUC.cpp | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/src/Functions/array/arrayAUC.cpp b/src/Functions/array/arrayAUC.cpp index 878a8fcae34..58e2a864a7f 100644 --- a/src/Functions/array/arrayAUC.cpp +++ b/src/Functions/array/arrayAUC.cpp @@ -121,7 +121,7 @@ private: prev_tp = curr_tp; prev_score = sorted_labels[i].score; } - + if (sorted_labels[i].label) curr_tp += 1; /// The curve moves one step up. else From 7d7750e29b88bcde9b6b53efbd86f89915985747 Mon Sep 17 00:00:00 2001 From: gabrielmcg44 Date: Fri, 28 Jun 2024 11:40:56 -0300 Subject: [PATCH 129/417] remove brackets --- src/Functions/array/arrayAUC.cpp | 3 +-- 1 file changed, 1 insertion(+), 2 deletions(-) diff --git a/src/Functions/array/arrayAUC.cpp b/src/Functions/array/arrayAUC.cpp index 58e2a864a7f..940cb01a5bf 100644 --- a/src/Functions/array/arrayAUC.cpp +++ b/src/Functions/array/arrayAUC.cpp @@ -115,12 +115,11 @@ private: for (size_t i = 0; i < size; ++i) { // Only change the current ROC point when the score changes - if (sorted_labels[i].score != prev_score) { + if (sorted_labels[i].score != prev_score) area += (curr_fp - prev_fp) * (curr_tp + prev_tp) / 2.0; // Trapezoidal area under curve (might degenerate to zero or to a rectangle) prev_fp = curr_fp; prev_tp = curr_tp; prev_score = sorted_labels[i].score; - } if (sorted_labels[i].label) curr_tp += 1; /// The curve moves one step up. From 067b1474513069840646414d99b947766c939a9d Mon Sep 17 00:00:00 2001 From: gabrielmcg44 Date: Fri, 28 Jun 2024 12:17:12 -0300 Subject: [PATCH 130/417] add test --- .../03198_fix_auc_tie_handling.reference | 2 + .../03198_fix_auc_tie_handling.sql | 39 +++++++++++++++++++ 2 files changed, 41 insertions(+) create mode 100644 tests/queries/0_stateless/03198_fix_auc_tie_handling.reference create mode 100644 tests/queries/0_stateless/03198_fix_auc_tie_handling.sql diff --git a/tests/queries/0_stateless/03198_fix_auc_tie_handling.reference b/tests/queries/0_stateless/03198_fix_auc_tie_handling.reference new file mode 100644 index 00000000000..56c97b6849c --- /dev/null +++ b/tests/queries/0_stateless/03198_fix_auc_tie_handling.reference @@ -0,0 +1,2 @@ +0.58333 +0.58333 diff --git a/tests/queries/0_stateless/03198_fix_auc_tie_handling.sql b/tests/queries/0_stateless/03198_fix_auc_tie_handling.sql new file mode 100644 index 00000000000..1064668a2b5 --- /dev/null +++ b/tests/queries/0_stateless/03198_fix_auc_tie_handling.sql @@ -0,0 +1,39 @@ +CREATE TABLE labels_unordered +( + idx Int64, + score Float64, + label Int64 +) +ENGINE = MergeTree +PRIMARY KEY idx +ORDER BY idx; + +INSERT INTO labels_unordered (idx,score,label) VALUES + (1,0.1,0), + (2,0.35,1), + (3,0.4,0), + (4,0.8,1), + (5,0.8,0); + +SELECT floor(arrayAUC(array_concat_agg([score]), array_concat_agg([label])), 5) +FROM labels_unordered; + +CREATE TABLE labels_ordered +( + idx Int64, + score Float64, + label Int64 +) +ENGINE = MergeTree +PRIMARY KEY idx +ORDER BY idx; + +INSERT INTO labels_ordered (idx,score,label) VALUES + (1,0.1,0), + (2,0.35,1), + (3,0.4,0), + (4,0.8,0), + (5,0.8,1); + +SELECT floor(arrayAUC(array_concat_agg([score]), array_concat_agg([label])), 5) +FROM labels_ordered; \ No newline at end of file From dd1eccd32f09da8bd076b3adb1cc838f75338b7c Mon Sep 17 00:00:00 2001 From: gabrielmcg44 Date: Fri, 28 Jun 2024 12:21:19 -0300 Subject: [PATCH 131/417] fix brackets --- src/Functions/array/arrayAUC.cpp | 2 ++ 1 file changed, 2 insertions(+) diff --git a/src/Functions/array/arrayAUC.cpp b/src/Functions/array/arrayAUC.cpp index 940cb01a5bf..0fc05fdf996 100644 --- a/src/Functions/array/arrayAUC.cpp +++ b/src/Functions/array/arrayAUC.cpp @@ -116,10 +116,12 @@ private: { // Only change the current ROC point when the score changes if (sorted_labels[i].score != prev_score) + { area += (curr_fp - prev_fp) * (curr_tp + prev_tp) / 2.0; // Trapezoidal area under curve (might degenerate to zero or to a rectangle) prev_fp = curr_fp; prev_tp = curr_tp; prev_score = sorted_labels[i].score; + } if (sorted_labels[i].label) curr_tp += 1; /// The curve moves one step up. From dc38c863d3de80849d0f0d87117d78b1638ffa76 Mon Sep 17 00:00:00 2001 From: gabrielmcg44 Date: Fri, 28 Jun 2024 12:47:46 -0300 Subject: [PATCH 132/417] fix test syntax --- .../0_stateless/03198_fix_auc_tie_handling.sql | 14 ++------------ 1 file changed, 2 insertions(+), 12 deletions(-) diff --git a/tests/queries/0_stateless/03198_fix_auc_tie_handling.sql b/tests/queries/0_stateless/03198_fix_auc_tie_handling.sql index 1064668a2b5..394a384ec8d 100644 --- a/tests/queries/0_stateless/03198_fix_auc_tie_handling.sql +++ b/tests/queries/0_stateless/03198_fix_auc_tie_handling.sql @@ -8,12 +8,7 @@ ENGINE = MergeTree PRIMARY KEY idx ORDER BY idx; -INSERT INTO labels_unordered (idx,score,label) VALUES - (1,0.1,0), - (2,0.35,1), - (3,0.4,0), - (4,0.8,1), - (5,0.8,0); +INSERT INTO labels_unordered (idx,score,label) VALUES (1,0.1,0), (2,0.35,1), (3,0.4,0), (4,0.8,1), (5,0.8,0); SELECT floor(arrayAUC(array_concat_agg([score]), array_concat_agg([label])), 5) FROM labels_unordered; @@ -28,12 +23,7 @@ ENGINE = MergeTree PRIMARY KEY idx ORDER BY idx; -INSERT INTO labels_ordered (idx,score,label) VALUES - (1,0.1,0), - (2,0.35,1), - (3,0.4,0), - (4,0.8,0), - (5,0.8,1); +INSERT INTO labels_ordered (idx,score,label) VALUES (1,0.1,0), (2,0.35,1), (3,0.4,0), (4,0.8,0), (5,0.8,1); SELECT floor(arrayAUC(array_concat_agg([score]), array_concat_agg([label])), 5) FROM labels_ordered; \ No newline at end of file From b847ccabc8e9f6968f20a2d317e1ca5702b8e57f Mon Sep 17 00:00:00 2001 From: gabrielmcg44 Date: Fri, 28 Jun 2024 18:55:23 -0300 Subject: [PATCH 133/417] fix previous test and add test with empty arrays --- .../queries/0_stateless/01202_array_auc_special.reference | 8 ++++---- .../0_stateless/03198_fix_auc_tie_handling.reference | 1 + tests/queries/0_stateless/03198_fix_auc_tie_handling.sql | 3 +++ 3 files changed, 8 insertions(+), 4 deletions(-) diff --git a/tests/queries/0_stateless/01202_array_auc_special.reference b/tests/queries/0_stateless/01202_array_auc_special.reference index 85c230fba58..8f3f0cf1efe 100644 --- a/tests/queries/0_stateless/01202_array_auc_special.reference +++ b/tests/queries/0_stateless/01202_array_auc_special.reference @@ -1,9 +1,9 @@ nan nan nan -0 -1 -0 0.5 1 -0.5 +0 +0.75 +1 +0.75 diff --git a/tests/queries/0_stateless/03198_fix_auc_tie_handling.reference b/tests/queries/0_stateless/03198_fix_auc_tie_handling.reference index 56c97b6849c..f35b39d5972 100644 --- a/tests/queries/0_stateless/03198_fix_auc_tie_handling.reference +++ b/tests/queries/0_stateless/03198_fix_auc_tie_handling.reference @@ -1,2 +1,3 @@ +nan 0.58333 0.58333 diff --git a/tests/queries/0_stateless/03198_fix_auc_tie_handling.sql b/tests/queries/0_stateless/03198_fix_auc_tie_handling.sql index 394a384ec8d..5de0844f445 100644 --- a/tests/queries/0_stateless/03198_fix_auc_tie_handling.sql +++ b/tests/queries/0_stateless/03198_fix_auc_tie_handling.sql @@ -8,6 +8,9 @@ ENGINE = MergeTree PRIMARY KEY idx ORDER BY idx; +SELECT floor(arrayAUC(array_concat_agg([score]), array_concat_agg([label])), 5) +FROM labels_unordered; + INSERT INTO labels_unordered (idx,score,label) VALUES (1,0.1,0), (2,0.35,1), (3,0.4,0), (4,0.8,1), (5,0.8,0); SELECT floor(arrayAUC(array_concat_agg([score]), array_concat_agg([label])), 5) From df086999ff0fe471277e25ff0abc5afff618a746 Mon Sep 17 00:00:00 2001 From: gabrielmcg44 Date: Sat, 29 Jun 2024 07:13:08 -0300 Subject: [PATCH 134/417] remove redundant type cast --- src/Functions/array/arrayAUC.cpp | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/src/Functions/array/arrayAUC.cpp b/src/Functions/array/arrayAUC.cpp index 0fc05fdf996..acc1505fbb3 100644 --- a/src/Functions/array/arrayAUC.cpp +++ b/src/Functions/array/arrayAUC.cpp @@ -136,7 +136,7 @@ private: if (curr_tp == 0 || curr_tp == size) return std::numeric_limits::quiet_NaN(); - return static_cast(area) / curr_tp / (size - curr_tp); + return area / curr_tp / (size - curr_tp); } static void vector( From b9b030d0aad0060a0d6c1cde6ad3165f0ef8757d Mon Sep 17 00:00:00 2001 From: gabrielmcg44 Date: Mon, 1 Jul 2024 10:29:51 -0300 Subject: [PATCH 135/417] better comment --- src/Functions/array/arrayAUC.cpp | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/src/Functions/array/arrayAUC.cpp b/src/Functions/array/arrayAUC.cpp index acc1505fbb3..04a840e8da5 100644 --- a/src/Functions/array/arrayAUC.cpp +++ b/src/Functions/array/arrayAUC.cpp @@ -114,7 +114,7 @@ private: size_t curr_fp = 0, curr_tp = 0; for (size_t i = 0; i < size; ++i) { - // Only change the current ROC point when the score changes + // Only increment the area when the score changes if (sorted_labels[i].score != prev_score) { area += (curr_fp - prev_fp) * (curr_tp + prev_tp) / 2.0; // Trapezoidal area under curve (might degenerate to zero or to a rectangle) From ce00de19e92041bf5dcf10d40379732f33b8f1d3 Mon Sep 17 00:00:00 2001 From: gabrielmcg44 Date: Tue, 2 Jul 2024 11:38:49 -0300 Subject: [PATCH 136/417] rebase and rename test --- ...ie_handling.reference => 03199_fix_auc_tie_handling.reference} | 0 ...98_fix_auc_tie_handling.sql => 03199_fix_auc_tie_handling.sql} | 0 2 files changed, 0 insertions(+), 0 deletions(-) rename tests/queries/0_stateless/{03198_fix_auc_tie_handling.reference => 03199_fix_auc_tie_handling.reference} (100%) rename tests/queries/0_stateless/{03198_fix_auc_tie_handling.sql => 03199_fix_auc_tie_handling.sql} (100%) diff --git a/tests/queries/0_stateless/03198_fix_auc_tie_handling.reference b/tests/queries/0_stateless/03199_fix_auc_tie_handling.reference similarity index 100% rename from tests/queries/0_stateless/03198_fix_auc_tie_handling.reference rename to tests/queries/0_stateless/03199_fix_auc_tie_handling.reference diff --git a/tests/queries/0_stateless/03198_fix_auc_tie_handling.sql b/tests/queries/0_stateless/03199_fix_auc_tie_handling.sql similarity index 100% rename from tests/queries/0_stateless/03198_fix_auc_tie_handling.sql rename to tests/queries/0_stateless/03199_fix_auc_tie_handling.sql From 02de44124616de36ae2606f22a4a6a0ba212ed3c Mon Sep 17 00:00:00 2001 From: avogar Date: Tue, 2 Jul 2024 15:55:26 +0000 Subject: [PATCH 137/417] Remove unrelated files --- src/Formats/JSONExtractTree.cpp | 666 -------------------------------- src/Formats/JSONExtractTree.h | 37 -- 2 files changed, 703 deletions(-) delete mode 100644 src/Formats/JSONExtractTree.cpp delete mode 100644 src/Formats/JSONExtractTree.h diff --git a/src/Formats/JSONExtractTree.cpp b/src/Formats/JSONExtractTree.cpp deleted file mode 100644 index 6a4241003f1..00000000000 --- a/src/Formats/JSONExtractTree.cpp +++ /dev/null @@ -1,666 +0,0 @@ -//#include -// -//#include -//#include -//#include -//#include -// -//#include -//#include -//#include -// -//#include -//#include -//#include -// -//namespace DB -//{ -// -//namespace -//{ -// -//const FormatSettings & getFormatSettings() -//{ -// static const FormatSettings instance = [] -// { -// FormatSettings settings; -// settings.json.escape_forward_slashes = false; -// return settings; -// }(); -// return instance; -//} -// -//template -//void elementToString(const Element & element, WriteBuffer & buf) -//{ -// if (element.isInt64()) -// { -// writeIntText(element.getInt64(), buf); -// return; -// } -// if (element.isUInt64()) -// { -// writeIntText(element.getUInt64(), buf); -// return; -// } -// if (element.isDouble()) -// { -// writeFloatText(element.getDouble(), buf); -// return; -// } -// if (element.isBool()) -// { -// if (element.getBool()) -// writeCString("true", buf); -// else -// writeCString("false", buf); -// return; -// } -// if (element.isString()) -// { -// writeJSONString(element.getString(), buf, getFormatSettings()); -// return; -// } -// if (element.isArray()) -// { -// writeChar('[', buf); -// bool need_comma = false; -// for (auto value : element.getArray()) -// { -// if (std::exchange(need_comma, true)) -// writeChar(',', buf); -// elementToString(value, buf); -// } -// writeChar(']', buf); -// return; -// } -// if (element.isObject()) -// { -// writeChar('{', buf); -// bool need_comma = false; -// for (auto [key, value] : element.getObject()) -// { -// if (std::exchange(need_comma, true)) -// writeChar(',', buf); -// writeJSONString(key, buf, getFormatSettings()); -// writeChar(':', buf); -// elementToString(value, buf); -// } -// writeChar('}', buf); -// return; -// } -// if (element.isNull()) -// { -// writeCString("null", buf); -// return; -// } -//} -// -//template -//class NumericNode : public JSONExtractTree::Node -//{ -//public: -// NumericNode(bool convert_bool_to_integer_) : convert_bool_to_integer(convert_bool_to_integer_) {} -// -// bool insertResultToColumn(IColumn & dest, const Element & element) override -// { -// NumberType value; -// if (!tryGetValue(element, value)) -// return false; -// -// auto & col_vec = assert_cast &>(dest); -// col_vec.insertValue(value); -// return true; -// } -// -// bool tryGetValue(const Element & element, NumberType & value) -// { -// switch (element.type()) -// { -// case ElementType::DOUBLE: -// if constexpr (std::is_floating_point_v) -// { -// /// We permit inaccurate conversion of double to float. -// /// Example: double 0.1 from JSON is not representable in float. -// /// But it will be more convenient for user to perform conversion. -// value = static_cast(element.getDouble()); -// } -// else if (!accurate::convertNumeric(element.getDouble(), value)) -// return false; -// break; -// case ElementType::UINT64: -// if (!accurate::convertNumeric(element.getUInt64(), value)) -// return false; -// break; -// case ElementType::INT64: -// if (!accurate::convertNumeric(element.getInt64(), value)) -// return false; -// break; -// case ElementType::BOOL: -// if constexpr (is_integer) -// { -// if (convert_bool_to_integer) -// { -// value = static_cast(element.getBool()); -// break; -// } -// } -// return false; -// case ElementType::STRING: -// { -// auto rb = ReadBufferFromMemory{element.getString()}; -// if constexpr (std::is_floating_point_v) -// { -// if (!tryReadFloatText(value, rb) || !rb.eof()) -// return false; -// } -// else -// { -// if (tryReadIntText(value, rb) && rb.eof()) -// break; -// -// /// Try to parse float and convert it to integer. -// Float64 tmp_float; -// rb.position() = rb.buffer().begin(); -// if (!tryReadFloatText(tmp_float, rb) || !rb.eof()) -// return false; -// -// if (!accurate::convertNumeric(tmp_float, value)) -// return false; -// } -// break; -// } -// case ElementType::NULL_VALUE: -// { -// if () -// } -// default: -// return false; -// } -// -// return true; -// } -// -//private: -// bool convert_bool_to_integer; -//}; -// -//template -//class LowCardinalityNumericNode : public NumericNode -//{ -// bool insertResultToColumn(IColumn & dest, const Element & element) override -// { -// NumberType value; -// if (!tryGetValue(element, value)) -// return false; -// -// auto & col_lc = assert_cast(dest); -// col_lc.insertData(reinterpret_cast(&value), sizeof(value)); -// return true; -// } -//}; -// -//template -//class StringNode : public JSONExtractTree::Node -//{ -//public: -// bool insertResultToColumn(IColumn & dest, const Element & element) override -// { -// if (element.isNull()) -// return false; -// -// if (!element.isString()) -// { -// ColumnString & col_str = assert_cast(dest); -// auto & chars = col_str.getChars(); -// WriteBufferFromVector buf(chars, AppendModeTag()); -// elementToString(element, buf); -// buf.finalize(); -// chars.push_back(0); -// col_str.getOffsets().push_back(chars.size()); -// return true; -// } -// else -// { -// auto str = element.getString(); -// ColumnString & col_str = assert_cast(dest); -// col_str.insertData(str.data(), str.size()); -// } -// return true; -// } -//}; -// -//template -//class LowCardinalityStringNode : public JSONExtractTree::Node -//{ -// bool insertResultToColumn(IColumn & dest, const Element & element) override -// { -// if (element.isNull()) -// return false; -// -// if (!element.isString()) -// { -// ColumnString & col_str = assert_cast(dest); -// auto & chars = col_str.getChars(); -// WriteBufferFromVector buf(chars, AppendModeTag()); -// elementToString(element, buf); -// buf.finalize(); -// chars.push_back(0); -// col_str.getOffsets().push_back(chars.size()); -// return true; -// } -// else -// { -// auto str = element.getString(); -// ColumnString & col_str = assert_cast(dest); -// col_str.insertData(str.data(), str.size()); -// } -// return true; -// } -//}; -// -// -// -// -// -// -//class LowCardinalityFixedStringNode : public Node -//{ -//public: -// explicit LowCardinalityFixedStringNode(const size_t fixed_length_) : fixed_length(fixed_length_) { } -// bool insertResultToColumn(IColumn & dest, const Element & element) override -// { -// // If element is an object we delegate the insertion to JSONExtractRawImpl -// if (element.isObject()) -// return JSONExtractRawImpl::insertResultToLowCardinalityFixedStringColumn(dest, element, fixed_length); -// else if (!element.isString()) -// return false; -// -// auto str = element.getString(); -// if (str.size() > fixed_length) -// return false; -// -// // For the non low cardinality case of FixedString, the padding is done in the FixedString Column implementation. -// // In order to avoid having to pass the data to a FixedString Column and read it back (which would slow down the execution) -// // the data is padded here and written directly to the Low Cardinality Column -// if (str.size() == fixed_length) -// { -// assert_cast(dest).insertData(str.data(), str.size()); -// } -// else -// { -// String padded_str(str); -// padded_str.resize(fixed_length, '\0'); -// -// assert_cast(dest).insertData(padded_str.data(), padded_str.size()); -// } -// return true; -// } -// -//private: -// const size_t fixed_length; -//}; -// -//class UUIDNode : public Node -//{ -//public: -// bool insertResultToColumn(IColumn & dest, const Element & element) override -// { -// if (!element.isString()) -// return false; -// -// auto uuid = parseFromString(element.getString()); -// if (dest.getDataType() == TypeIndex::LowCardinality) -// { -// ColumnLowCardinality & col_low = assert_cast(dest); -// col_low.insertData(reinterpret_cast(&uuid), sizeof(uuid)); -// } -// else -// { -// assert_cast(dest).insert(uuid); -// } -// return true; -// } -//}; -// -//template -//class DecimalNode : public Node -//{ -//public: -// explicit DecimalNode(DataTypePtr data_type_) : data_type(data_type_) {} -// bool insertResultToColumn(IColumn & dest, const Element & element) override -// { -// const auto * type = assert_cast *>(data_type.get()); -// -// DecimalType value{}; -// -// switch (element.type()) -// { -// case ElementType::DOUBLE: -// value = convertToDecimal, DataTypeDecimal>( -// element.getDouble(), type->getScale()); -// break; -// case ElementType::UINT64: -// value = convertToDecimal, DataTypeDecimal>( -// element.getUInt64(), type->getScale()); -// break; -// case ElementType::INT64: -// value = convertToDecimal, DataTypeDecimal>( -// element.getInt64(), type->getScale()); -// break; -// case ElementType::STRING: { -// auto rb = ReadBufferFromMemory{element.getString()}; -// if (!SerializationDecimal::tryReadText(value, rb, DecimalUtils::max_precision, type->getScale())) -// return false; -// break; -// } -// default: -// return false; -// } -// -// assert_cast &>(dest).insertValue(value); -// return true; -// } -// -//private: -// DataTypePtr data_type; -//}; -// -//class FixedStringNode : public Node -//{ -//public: -// bool insertResultToColumn(IColumn & dest, const Element & element) override -// { -// if (element.isNull()) -// return false; -// -// if (!element.isString()) -// return JSONExtractRawImpl::insertResultToFixedStringColumn(dest, element, {}); -// -// auto str = element.getString(); -// auto & col_str = assert_cast(dest); -// if (str.size() > col_str.getN()) -// return false; -// col_str.insertData(str.data(), str.size()); -// -// return true; -// } -//}; -// -//template -//class EnumNode : public Node -//{ -//public: -// explicit EnumNode(const std::vector> & name_value_pairs_) : name_value_pairs(name_value_pairs_) -// { -// for (const auto & name_value_pair : name_value_pairs) -// { -// name_to_value_map.emplace(name_value_pair.first, name_value_pair.second); -// only_values.emplace(name_value_pair.second); -// } -// } -// -// bool insertResultToColumn(IColumn & dest, const Element & element) override -// { -// auto & col_vec = assert_cast &>(dest); -// -// if (element.isInt64()) -// { -// Type value; -// if (!accurate::convertNumeric(element.getInt64(), value) || !only_values.contains(value)) -// return false; -// col_vec.insertValue(value); -// return true; -// } -// -// if (element.isUInt64()) -// { -// Type value; -// if (!accurate::convertNumeric(element.getUInt64(), value) || !only_values.contains(value)) -// return false; -// col_vec.insertValue(value); -// return true; -// } -// -// if (element.isString()) -// { -// auto value = name_to_value_map.find(element.getString()); -// if (value == name_to_value_map.end()) -// return false; -// col_vec.insertValue(value->second); -// return true; -// } -// -// return false; -// } -// -//private: -// std::vector> name_value_pairs; -// std::unordered_map name_to_value_map; -// std::unordered_set only_values; -//}; -// -//class NullableNode : public Node -//{ -//public: -// explicit NullableNode(std::unique_ptr nested_) : nested(std::move(nested_)) {} -// -// bool insertResultToColumn(IColumn & dest, const Element & element) override -// { -// if (dest.getDataType() == TypeIndex::LowCardinality) -// { -// /// We do not need to handle nullability in that case -// /// because nested node handles LowCardinality columns and will call proper overload of `insertData` -// return nested->insertResultToColumn(dest, element); -// } -// -// ColumnNullable & col_null = assert_cast(dest); -// if (!nested->insertResultToColumn(col_null.getNestedColumn(), element)) -// return false; -// col_null.getNullMapColumn().insertValue(0); -// return true; -// } -// -//private: -// std::unique_ptr nested; -//}; -// -//class ArrayNode : public Node -//{ -//public: -// explicit ArrayNode(std::unique_ptr nested_) : nested(std::move(nested_)) {} -// -// bool insertResultToColumn(IColumn & dest, const Element & element) override -// { -// if (!element.isArray()) -// return false; -// -// auto array = element.getArray(); -// -// ColumnArray & col_arr = assert_cast(dest); -// auto & data = col_arr.getData(); -// size_t old_size = data.size(); -// bool were_valid_elements = false; -// -// for (auto value : array) -// { -// if (nested->insertResultToColumn(data, value)) -// were_valid_elements = true; -// else -// data.insertDefault(); -// } -// -// if (!were_valid_elements) -// { -// data.popBack(data.size() - old_size); -// return false; -// } -// -// col_arr.getOffsets().push_back(data.size()); -// return true; -// } -// -//private: -// std::unique_ptr nested; -//}; -// -//class TupleNode : public Node -//{ -//public: -// TupleNode(std::vector> nested_, const std::vector & explicit_names_) : nested(std::move(nested_)), explicit_names(explicit_names_) -// { -// for (size_t i = 0; i != explicit_names.size(); ++i) -// name_to_index_map.emplace(explicit_names[i], i); -// } -// -// bool insertResultToColumn(IColumn & dest, const Element & element) override -// { -// ColumnTuple & tuple = assert_cast(dest); -// size_t old_size = dest.size(); -// bool were_valid_elements = false; -// -// auto set_size = [&](size_t size) -// { -// for (size_t i = 0; i != tuple.tupleSize(); ++i) -// { -// auto & col = tuple.getColumn(i); -// if (col.size() != size) -// { -// if (col.size() > size) -// col.popBack(col.size() - size); -// else -// while (col.size() < size) -// col.insertDefault(); -// } -// } -// }; -// -// if (element.isArray()) -// { -// auto array = element.getArray(); -// auto it = array.begin(); -// -// for (size_t index = 0; (index != nested.size()) && (it != array.end()); ++index) -// { -// if (nested[index]->insertResultToColumn(tuple.getColumn(index), *it++)) -// were_valid_elements = true; -// else -// tuple.getColumn(index).insertDefault(); -// } -// -// set_size(old_size + static_cast(were_valid_elements)); -// return were_valid_elements; -// } -// -// if (element.isObject()) -// { -// auto object = element.getObject(); -// if (name_to_index_map.empty()) -// { -// auto it = object.begin(); -// for (size_t index = 0; (index != nested.size()) && (it != object.end()); ++index) -// { -// if (nested[index]->insertResultToColumn(tuple.getColumn(index), (*it++).second)) -// were_valid_elements = true; -// else -// tuple.getColumn(index).insertDefault(); -// } -// } -// else -// { -// for (const auto & [key, value] : object) -// { -// auto index = name_to_index_map.find(key); -// if (index != name_to_index_map.end()) -// { -// if (nested[index->second]->insertResultToColumn(tuple.getColumn(index->second), value)) -// were_valid_elements = true; -// } -// } -// } -// -// set_size(old_size + static_cast(were_valid_elements)); -// return were_valid_elements; -// } -// -// return false; -// } -// -//private: -// std::vector> nested; -// std::vector explicit_names; -// std::unordered_map name_to_index_map; -//}; -// -//class MapNode : public Node -//{ -//public: -// MapNode(std::unique_ptr key_, std::unique_ptr value_) : key(std::move(key_)), value(std::move(value_)) { } -// -// bool insertResultToColumn(IColumn & dest, const Element & element) override -// { -// if (!element.isObject()) -// return false; -// -// ColumnMap & map_col = assert_cast(dest); -// auto & offsets = map_col.getNestedColumn().getOffsets(); -// auto & tuple_col = map_col.getNestedData(); -// auto & key_col = tuple_col.getColumn(0); -// auto & value_col = tuple_col.getColumn(1); -// size_t old_size = tuple_col.size(); -// -// auto object = element.getObject(); -// auto it = object.begin(); -// for (; it != object.end(); ++it) -// { -// auto pair = *it; -// -// /// Insert key -// key_col.insertData(pair.first.data(), pair.first.size()); -// -// /// Insert value -// if (!value->insertResultToColumn(value_col, pair.second)) -// value_col.insertDefault(); -// } -// -// offsets.push_back(old_size + object.size()); -// return true; -// } -// -//private: -// std::unique_ptr key; -// std::unique_ptr value; -//}; -// -//class VariantNode : public Node -//{ -//public: -// VariantNode(std::vector> variant_nodes_, std::vector order_) : variant_nodes(std::move(variant_nodes_)), order(std::move(order_)) { } -// -// bool insertResultToColumn(IColumn & dest, const Element & element) override -// { -// auto & column_variant = assert_cast(dest); -// for (size_t i : order) -// { -// auto & variant = column_variant.getVariantByGlobalDiscriminator(i); -// if (variant_nodes[i]->insertResultToColumn(variant, element)) -// { -// column_variant.getLocalDiscriminators().push_back(column_variant.localDiscriminatorByGlobal(i)); -// column_variant.getOffsets().push_back(variant.size() - 1); -// return true; -// } -// } -// -// return false; -// } -// -//private: -// std::vector> variant_nodes; -// /// Order in which we should try variants nodes. -// /// For example, String should be always the last one. -// std::vector order; -//}; -// -//} -// -//} diff --git a/src/Formats/JSONExtractTree.h b/src/Formats/JSONExtractTree.h deleted file mode 100644 index f07c974f595..00000000000 --- a/src/Formats/JSONExtractTree.h +++ /dev/null @@ -1,37 +0,0 @@ -#pragma once -#include -#include - -namespace DB -{ - -template -struct JSONExtractTree -{ - class Node - { - public: - Node() = default; - virtual ~Node() = default; - virtual bool insertResultToColumn(IColumn &, const Element &) = 0; - }; - - struct Settings - { - bool convert_bool_to_integer = true; - bool type_json_infer_numbers_from_strings = true; - bool type_json_infer_date = true; - bool type_json_infer_datetime = true; - bool type_json_infer_ipv4 = true; - bool type_json_infer_ipv6 = true; - bool type_json_infer_uuid = true; - bool insert_null_as_default = true; - }; - - static std::unique_ptr build(const DataTypePtr & type, const Settings & settings, const char * source_for_exception_message); -}; - -template -void elementToString(const Element & element, WriteBuffer & buf); - -} From a020138764ff01f6a89c05824d9236bad13459ce Mon Sep 17 00:00:00 2001 From: Nikita Fomichev Date: Tue, 2 Jul 2024 18:15:57 +0200 Subject: [PATCH 138/417] Tests: add tests for new analyzer --- .../03199_queries_with_new_analyzer.reference | 27 ++++++++++++ .../03199_queries_with_new_analyzer.sql | 41 +++++++++++++++++++ 2 files changed, 68 insertions(+) create mode 100644 tests/queries/0_stateless/03199_queries_with_new_analyzer.reference create mode 100644 tests/queries/0_stateless/03199_queries_with_new_analyzer.sql diff --git a/tests/queries/0_stateless/03199_queries_with_new_analyzer.reference b/tests/queries/0_stateless/03199_queries_with_new_analyzer.reference new file mode 100644 index 00000000000..10ce589000d --- /dev/null +++ b/tests/queries/0_stateless/03199_queries_with_new_analyzer.reference @@ -0,0 +1,27 @@ +5 (4230072075578472911,4230072075578472911) 71789584853496063 +2 (4401188181514187637,4401188181514187637) 878466845199253299 +4 (4940826638032106783,4940826638032106783) 3675164899122807807 +6 (10957420562507184961,10957420562507184961) 3732623117916254211 +0 (797076400500506358,797076400500506358) 3746094338409299772 +7 (10843611042193511775,10843611042193511775) 4607251742847087615 +3 (12588286986351526898,12588286986351526898) 13889114719560662796 +8 (452995860660674674,452995860660674674) 17365664920787500812 +9 (12206106972241516904,12206106972241516904) 17567684527097330880 +1 (14558425114501132193,14558425114501132193) 18445898820068822019 +3 255 255 +0 0 0 +0 0 0 +0 0 0 +0 0 0 +0 0 0 +0 0 0 +0 0 0 +0 0 0 +0 0 0 +0 +1 +2 +3 +4 +5 +6 diff --git a/tests/queries/0_stateless/03199_queries_with_new_analyzer.sql b/tests/queries/0_stateless/03199_queries_with_new_analyzer.sql new file mode 100644 index 00000000000..c32d7524492 --- /dev/null +++ b/tests/queries/0_stateless/03199_queries_with_new_analyzer.sql @@ -0,0 +1,41 @@ +SET allow_experimental_analyzer=1; + +SELECT *, ngramMinHash(*) AS minhash, mortonEncode(untuple(ngramMinHash(*))) AS z +FROM (SELECT toString(number) FROM numbers(10)) +ORDER BY z LIMIT 100; + +CREATE TABLE test ( + idx UInt64, + coverage Array(UInt64), + test_name String +) +ENGINE = MergeTree +ORDER BY tuple(); + +INSERT INTO test VALUES (10, [0,1,2,3], 'xx'), (20, [3,4,5,6], 'xxx'), (90, [3,4,5,6,9], 'xxxx'); + +WITH + 4096 AS w, 4096 AS h, w * h AS pixels, + arrayJoin(coverage) AS num, + num DIV (32768 * 32768 DIV pixels) AS idx, + mortonDecode(2, idx) AS coord, + 255 AS b, + least(255, uniq(test_name)) AS r, + 255 * uniq(test_name) / (max(uniq(test_name)) OVER ()) AS g +SELECT r::UInt8, g::UInt8, b::UInt8 +FROM test +GROUP BY coord +ORDER BY coord.2 * w + coord.1 +WITH FILL FROM 0 TO 10; + + +CREATE TABLE seq ( + number UInt64 +) +ENGINE = MergeTree +ORDER BY tuple(); + +INSERT INTO seq VALUES (0), (6), (7); + +WITH (Select min(number), max(number) from seq) as range Select * from numbers(range.1, range.2); + From 4f66a6651afc9f655c820fedf6a6163fc4942fe9 Mon Sep 17 00:00:00 2001 From: Nikita Fomichev Date: Tue, 2 Jul 2024 18:16:22 +0200 Subject: [PATCH 139/417] Tests: remove +x flag from 03167_base64_url_functions_sh.reference --- tests/queries/0_stateless/03167_base64_url_functions_sh.reference | 0 1 file changed, 0 insertions(+), 0 deletions(-) mode change 100755 => 100644 tests/queries/0_stateless/03167_base64_url_functions_sh.reference diff --git a/tests/queries/0_stateless/03167_base64_url_functions_sh.reference b/tests/queries/0_stateless/03167_base64_url_functions_sh.reference old mode 100755 new mode 100644 From be9a17f2cca06814c68b712a6df2c18a44362220 Mon Sep 17 00:00:00 2001 From: gun9nir Date: Tue, 2 Jul 2024 21:42:59 -0700 Subject: [PATCH 140/417] feat: implicitly append wildcard if querying directory in file engine --- src/Storages/StorageFile.cpp | 16 +++++++++++----- ...03198_table_function_directory_path.reference | 3 +++ .../03198_table_function_directory_path.sql | 11 +++++++++++ 3 files changed, 25 insertions(+), 5 deletions(-) create mode 100644 tests/queries/0_stateless/03198_table_function_directory_path.reference create mode 100644 tests/queries/0_stateless/03198_table_function_directory_path.sql diff --git a/src/Storages/StorageFile.cpp b/src/Storages/StorageFile.cpp index 7f39ff615f0..9ff2a6667af 100644 --- a/src/Storages/StorageFile.cpp +++ b/src/Storages/StorageFile.cpp @@ -366,12 +366,18 @@ Strings StorageFile::getPathsList(const String & table_path, const String & user } else if (path.find_first_of("*?{") == std::string::npos) { - std::error_code error; - size_t size = fs::file_size(path, error); - if (!error) - total_bytes_to_read += size; + if (!fs::is_directory(path)) { + std::error_code error; + size_t size = fs::file_size(path, error); + if (!error) + total_bytes_to_read += size; - paths.push_back(path); + paths.push_back(path); + } else { + /// We list non-directory files under that directory. + paths = listFilesWithRegexpMatching(path / fs::path("*"), total_bytes_to_read); + can_be_directory = false; + } } else { diff --git a/tests/queries/0_stateless/03198_table_function_directory_path.reference b/tests/queries/0_stateless/03198_table_function_directory_path.reference new file mode 100644 index 00000000000..19920de3d3c --- /dev/null +++ b/tests/queries/0_stateless/03198_table_function_directory_path.reference @@ -0,0 +1,3 @@ +2 +2 +1 diff --git a/tests/queries/0_stateless/03198_table_function_directory_path.sql b/tests/queries/0_stateless/03198_table_function_directory_path.sql new file mode 100644 index 00000000000..671074ab45a --- /dev/null +++ b/tests/queries/0_stateless/03198_table_function_directory_path.sql @@ -0,0 +1,11 @@ +INSERT INTO FUNCTION file('data_03198_table_function_directory_path/1.csv', 'csv') SELECT '1.csv'; +INSERT INTO FUNCTION file('data_03198_table_function_directory_path/2.csv', 'csv') SELECT '2.csv'; +INSERT INTO FUNCTION file('data_03198_table_function_directory_path/dir/3.csv', 'csv') SELECT '3.csv'; +INSERT INTO FUNCTION file('data_03198_table_function_directory_path/dir1/dir/4.csv', 'csv') SELECT '4.csv'; +INSERT INTO FUNCTION file('data_03198_table_function_directory_path/dir2/dir/5.csv', 'csv') SELECT '5.csv'; + +SELECT COUNT(*) FROM file('data_03198_table_function_directory_path'); +SELECT COUNT(*) FROM file('data_03198_table_function_directory_path/'); +SELECT COUNT(*) FROM file('data_03198_table_function_directory_path/dir'); +SELECT COUNT(*) FROM file('data_03198_table_function_directory_path/*/dir', 'csv'); -- { serverError 74, 636 } +SELECT COUNT(*) FROM file('data_03198_table_function_directory_pat'); -- { serverError 400 } From 9341bbe19733fc2da455c6b0c51d0ee9bb13bf49 Mon Sep 17 00:00:00 2001 From: kevinyhzou Date: Wed, 3 Jul 2024 16:36:08 +0800 Subject: [PATCH 141/417] ORC read use writer time zone --- src/Core/Settings.h | 1 + src/Formats/FormatFactory.cpp | 1 + src/Formats/FormatSettings.h | 1 + .../Formats/Impl/NativeORCBlockInputFormat.cpp | 5 +++++ .../03198_orc_read_time_zone.reference | 1 + .../0_stateless/03198_orc_read_time_zone.sh | 11 +++++++++++ .../data_orc/read_time_zone.snappy.orc | Bin 0 -> 427 bytes 7 files changed, 20 insertions(+) create mode 100644 tests/queries/0_stateless/03198_orc_read_time_zone.reference create mode 100644 tests/queries/0_stateless/03198_orc_read_time_zone.sh create mode 100644 tests/queries/0_stateless/data_orc/read_time_zone.snappy.orc diff --git a/src/Core/Settings.h b/src/Core/Settings.h index d84e5b149f6..47dde4b5d14 100644 --- a/src/Core/Settings.h +++ b/src/Core/Settings.h @@ -1041,6 +1041,7 @@ class IColumn; M(Bool, input_format_orc_allow_missing_columns, true, "Allow missing columns while reading ORC input formats", 0) \ M(Bool, input_format_orc_use_fast_decoder, true, "Use a faster ORC decoder implementation.", 0) \ M(Bool, input_format_orc_filter_push_down, true, "When reading ORC files, skip whole stripes or row groups based on the WHERE/PREWHERE expressions, min/max statistics or bloom filter in the ORC metadata.", 0) \ + M(Bool, input_format_orc_read_use_writer_time_zone, false, "Whether use the writer's time zone in ORC stripe for ORC row reader, the default ORC row reader's time zone is GMT.", 0) \ M(Bool, input_format_parquet_allow_missing_columns, true, "Allow missing columns while reading Parquet input formats", 0) \ M(UInt64, input_format_parquet_local_file_min_bytes_for_seek, 8192, "Min bytes required for local read (file) to do seek, instead of read with ignore in Parquet input format", 0) \ M(Bool, input_format_arrow_allow_missing_columns, true, "Allow missing columns while reading Arrow input formats", 0) \ diff --git a/src/Formats/FormatFactory.cpp b/src/Formats/FormatFactory.cpp index 79c2e6b4890..6de106893f3 100644 --- a/src/Formats/FormatFactory.cpp +++ b/src/Formats/FormatFactory.cpp @@ -243,6 +243,7 @@ FormatSettings getFormatSettings(const ContextPtr & context, const Settings & se format_settings.orc.output_row_index_stride = settings.output_format_orc_row_index_stride; format_settings.orc.use_fast_decoder = settings.input_format_orc_use_fast_decoder; format_settings.orc.filter_push_down = settings.input_format_orc_filter_push_down; + format_settings.orc.read_use_writer_time_zone = settings.input_format_orc_read_use_writer_time_zone; format_settings.defaults_for_omitted_fields = settings.input_format_defaults_for_omitted_fields; format_settings.capn_proto.enum_comparing_mode = settings.format_capn_proto_enum_comparising_mode; format_settings.capn_proto.skip_fields_with_unsupported_types_in_schema_inference = settings.input_format_capn_proto_skip_fields_with_unsupported_types_in_schema_inference; diff --git a/src/Formats/FormatSettings.h b/src/Formats/FormatSettings.h index 8ac783a1d86..446dc17a187 100644 --- a/src/Formats/FormatSettings.h +++ b/src/Formats/FormatSettings.h @@ -403,6 +403,7 @@ struct FormatSettings bool use_fast_decoder = true; bool filter_push_down = true; UInt64 output_row_index_stride = 10'000; + bool read_use_writer_time_zone = false; } orc{}; /// For capnProto format we should determine how to diff --git a/src/Processors/Formats/Impl/NativeORCBlockInputFormat.cpp b/src/Processors/Formats/Impl/NativeORCBlockInputFormat.cpp index dcd5a531b05..c10969b02b7 100644 --- a/src/Processors/Formats/Impl/NativeORCBlockInputFormat.cpp +++ b/src/Processors/Formats/Impl/NativeORCBlockInputFormat.cpp @@ -900,6 +900,11 @@ bool NativeORCBlockInputFormat::prepareStripeReader() orc::RowReaderOptions row_reader_options; row_reader_options.includeTypes(include_indices); + if (format_settings.orc.read_use_writer_time_zone) + { + String writer_time_zone = current_stripe_info->getWriterTimezone(); + row_reader_options.setTimezoneName(writer_time_zone); + } row_reader_options.range(current_stripe_info->getOffset(), current_stripe_info->getLength()); if (format_settings.orc.filter_push_down && sarg) { diff --git a/tests/queries/0_stateless/03198_orc_read_time_zone.reference b/tests/queries/0_stateless/03198_orc_read_time_zone.reference new file mode 100644 index 00000000000..809dba44400 --- /dev/null +++ b/tests/queries/0_stateless/03198_orc_read_time_zone.reference @@ -0,0 +1 @@ +1 2024-06-30 20:00:00.000 diff --git a/tests/queries/0_stateless/03198_orc_read_time_zone.sh b/tests/queries/0_stateless/03198_orc_read_time_zone.sh new file mode 100644 index 00000000000..fec4e2bcc72 --- /dev/null +++ b/tests/queries/0_stateless/03198_orc_read_time_zone.sh @@ -0,0 +1,11 @@ +#!/usr/bin/env bash + +CURDIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd) +# shellcheck source=../shell_config.sh +. "$CURDIR"/../shell_config.sh + +$CLICKHOUSE_CLIENT -q "drop table if exists test" +$CLICKHOUSE_CLIENT -q "create table test(id UInt64, t DateTime64) Engine=MergeTree order by id" +$CLICKHOUSE_CLIENT -q "insert into test from infile '"$CURDIR"/data_orc/read_time_zone.snappy.orc' SETTINGS input_format_orc_read_use_writer_time_zone=true FORMAT ORC" +$CLICKHOUSE_CLIENT -q "select * from test" +$CLICKHOUSE_CLIENT -q "drop table test" \ No newline at end of file diff --git a/tests/queries/0_stateless/data_orc/read_time_zone.snappy.orc b/tests/queries/0_stateless/data_orc/read_time_zone.snappy.orc new file mode 100644 index 0000000000000000000000000000000000000000..ab1b785dbbfc2381e23edb45244a18b07eeb370e GIT binary patch literal 427 zcmZ{g&q~8U5XNUW&AM4zjS+PT9%@tw3Tw28C8uf;s)8*Qq}MeKZLoi06AHZrPaYI} z4e{#9ckwNJ0N3^+qKDy|?;D2sG4*D334jqRM^OWIRlCeI0I>2DS%hK}bKHPal@}RN z`S)m*qH~jH4W4`x z`|-{O Date: Wed, 3 Jul 2024 11:34:35 +0200 Subject: [PATCH 142/417] Move experimental settings to the experimental block --- src/Core/Settings.h | 139 ++++++++++++++++++++++++-------------------- 1 file changed, 77 insertions(+), 62 deletions(-) diff --git a/src/Core/Settings.h b/src/Core/Settings.h index d84e5b149f6..ee56c1133bf 100644 --- a/src/Core/Settings.h +++ b/src/Core/Settings.h @@ -164,9 +164,6 @@ class IColumn; M(Bool, enable_multiple_prewhere_read_steps, true, "Move more conditions from WHERE to PREWHERE and do reads from disk and filtering in multiple steps if there are multiple conditions combined with AND", 0) \ M(Bool, move_primary_key_columns_to_end_of_prewhere, true, "Move PREWHERE conditions containing primary key columns to the end of AND chain. It is likely that these conditions are taken into account during primary key analysis and thus will not contribute a lot to PREWHERE filtering.", 0) \ \ - M(Bool, allow_statistics_optimize, false, "Allows using statistics to optimize queries", 0) ALIAS(allow_statistic_optimize) \ - M(Bool, allow_experimental_statistics, false, "Allows using statistics", 0) ALIAS(allow_experimental_statistic) \ - \ M(UInt64, alter_sync, 1, "Wait for actions to manipulate the partitions. 0 - do not wait, 1 - wait for execution only of itself, 2 - wait for everyone.", 0) ALIAS(replication_alter_partitions_sync) \ M(Int64, replication_wait_for_inactive_replica_timeout, 120, "Wait for inactive replica to execute ALTER/OPTIMIZE. Time in seconds, 0 - do not wait, negative - wait for unlimited time.", 0) \ M(Bool, alter_move_to_space_execute_async, false, "Execute ALTER TABLE MOVE ... TO [DISK|VOLUME] asynchronously", 0) \ @@ -202,21 +199,6 @@ class IColumn; M(Bool, group_by_use_nulls, false, "Treat columns mentioned in ROLLUP, CUBE or GROUPING SETS as Nullable", 0) \ \ M(NonZeroUInt64, max_parallel_replicas, 1, "The maximum number of replicas of each shard used when the query is executed. For consistency (to get different parts of the same partition), this option only works for the specified sampling key. The lag of the replicas is not controlled. Should be always greater than 0", 0) \ - M(UInt64, parallel_replicas_count, 0, "This is internal setting that should not be used directly and represents an implementation detail of the 'parallel replicas' mode. This setting will be automatically set up by the initiator server for distributed queries to the number of parallel replicas participating in query processing.", 0) \ - M(UInt64, parallel_replica_offset, 0, "This is internal setting that should not be used directly and represents an implementation detail of the 'parallel replicas' mode. This setting will be automatically set up by the initiator server for distributed queries to the index of the replica participating in query processing among parallel replicas.", 0) \ - M(String, parallel_replicas_custom_key, "", "Custom key assigning work to replicas when parallel replicas are used.", 0) \ - M(ParallelReplicasCustomKeyFilterType, parallel_replicas_custom_key_filter_type, ParallelReplicasCustomKeyFilterType::DEFAULT, "Type of filter to use with custom key for parallel replicas. default - use modulo operation on the custom key, range - use range filter on custom key using all possible values for the value type of custom key.", 0) \ - M(UInt64, parallel_replicas_custom_key_range_lower, 0, "Lower bound for the universe that the parallel replicas custom range filter is calculated over", 0) \ - M(UInt64, parallel_replicas_custom_key_range_upper, 0, "Upper bound for the universe that the parallel replicas custom range filter is calculated over. A value of 0 disables the upper bound, setting it to the max value of the custom key expression", 0) \ - \ - M(String, cluster_for_parallel_replicas, "", "Cluster for a shard in which current server is located", 0) \ - M(UInt64, allow_experimental_parallel_reading_from_replicas, 0, "Use all the replicas from a shard for SELECT query execution. Reading is parallelized and coordinated dynamically. 0 - disabled, 1 - enabled, silently disable them in case of failure, 2 - enabled, throw an exception in case of failure", 0) \ - M(Bool, parallel_replicas_allow_in_with_subquery, true, "If true, subquery for IN will be executed on every follower replica.", 0) \ - M(Float, parallel_replicas_single_task_marks_count_multiplier, 2, "A multiplier which will be added during calculation for minimal number of marks to retrieve from coordinator. This will be applied only for remote replicas.", 0) \ - M(Bool, parallel_replicas_for_non_replicated_merge_tree, false, "If true, ClickHouse will use parallel replicas algorithm also for non-replicated MergeTree tables", 0) \ - M(UInt64, parallel_replicas_min_number_of_rows_per_replica, 0, "Limit the number of replicas used in a query to (estimated rows to read / min_number_of_rows_per_replica). The max is still limited by 'max_parallel_replicas'", 0) \ - M(Bool, parallel_replicas_prefer_local_join, true, "If true, and JOIN can be executed with parallel replicas algorithm, and all storages of right JOIN part are *MergeTree, local JOIN will be used instead of GLOBAL JOIN.", 0) \ - M(UInt64, parallel_replicas_mark_segment_size, 128, "Parts virtually divided into segments to be distributed between replicas for parallel reading. This setting controls the size of these segments. Not recommended to change until you're absolutely sure in what you're doing", 0) \ \ M(Bool, skip_unavailable_shards, false, "If true, ClickHouse silently skips unavailable shards. Shard is marked as unavailable when: 1) The shard cannot be reached due to a connection failure. 2) Shard is unresolvable through DNS. 3) Table does not exist on the shard.", 0) \ \ @@ -248,8 +230,6 @@ class IColumn; M(Bool, do_not_merge_across_partitions_select_final, false, "Merge parts only in one partition in select final", 0) \ M(Bool, split_parts_ranges_into_intersecting_and_non_intersecting_final, true, "Split parts ranges into intersecting and non intersecting during FINAL optimization", 0) \ M(Bool, split_intersecting_parts_ranges_into_layers_final, true, "Split intersecting parts ranges into layers during FINAL optimization", 0) \ - M(Bool, allow_experimental_inverted_index, false, "If it is set to true, allow to use experimental inverted index.", 0) \ - M(Bool, allow_experimental_full_text_index, false, "If it is set to true, allow to use experimental full-text index.", 0) \ \ M(UInt64, mysql_max_rows_to_insert, 65536, "The maximum number of rows in MySQL batch insertion of the MySQL storage engine", 0) \ M(Bool, mysql_map_string_to_text_in_show_columns, true, "If enabled, String type will be mapped to TEXT in SHOW [FULL] COLUMNS, BLOB otherwise. Has an effect only when the connection is made through the MySQL wire protocol.", 0) \ @@ -338,7 +318,6 @@ class IColumn; M(Bool, fsync_metadata, true, "Do fsync after changing metadata for tables and databases (.sql files). Could be disabled in case of poor latency on server with high load of DDL queries and high load of disk subsystem.", 0) \ \ M(Bool, join_use_nulls, false, "Use NULLs for non-joined rows of outer JOINs for types that can be inside Nullable. If false, use default value of corresponding columns data type.", IMPORTANT) \ - M(Bool, allow_experimental_join_condition, false, "Support join with inequal conditions which involve columns from both left and right table. e.g. t1.y < t2.y.", 0) \ \ M(JoinStrictness, join_default_strictness, JoinStrictness::All, "Set default strictness in JOIN query. Possible values: empty string, 'ANY', 'ALL'. If empty, query without strictness will throw exception.", 0) \ M(Bool, any_join_distinct_right_table_keys, false, "Enable old ANY JOIN logic with many-to-one left-to-right table keys mapping for all ANY JOINs. It leads to confusing not equal results for 't1 ANY LEFT JOIN t2' and 't2 ANY RIGHT JOIN t1'. ANY RIGHT JOIN needs one-to-many keys mapping to be consistent with LEFT one.", IMPORTANT) \ @@ -389,7 +368,6 @@ class IColumn; M(Bool, empty_result_for_aggregation_by_constant_keys_on_empty_set, true, "Return empty result when aggregating by constant keys on empty set.", 0) \ M(Bool, allow_distributed_ddl, true, "If it is set to true, then a user is allowed to executed distributed DDL queries.", 0) \ M(Bool, allow_suspicious_codecs, false, "If it is set to true, allow to specify meaningless compression codecs.", 0) \ - M(Bool, allow_experimental_codecs, false, "If it is set to true, allow to specify experimental compression codecs (but we don't have those yet and this option does nothing).", 0) \ M(Bool, enable_deflate_qpl_codec, false, "Enable/disable the DEFLATE_QPL codec.", 0) \ M(Bool, enable_zstd_qat_codec, false, "Enable/disable the ZSTD_QAT codec.", 0) \ M(UInt64, query_profiler_real_time_period_ns, QUERY_PROFILER_DEFAULT_SAMPLE_RATE_NS, "Period for real clock timer of query profiler (in nanoseconds). Set 0 value to turn off the real clock query profiler. Recommended value is at least 10000000 (100 times a second) for single queries or 1000000000 (once a second) for cluster-wide profiling.", 0) \ @@ -399,8 +377,7 @@ class IColumn; M(Float, opentelemetry_start_trace_probability, 0., "Probability to start an OpenTelemetry trace for an incoming query.", 0) \ M(Bool, opentelemetry_trace_processors, false, "Collect OpenTelemetry spans for processors.", 0) \ M(Bool, prefer_column_name_to_alias, false, "Prefer using column names instead of aliases if possible.", 0) \ - M(Bool, allow_experimental_analyzer, true, "Allow experimental analyzer.", 0) \ - M(Bool, analyzer_compatibility_join_using_top_level_identifier, false, "Force to resolve identifier in JOIN USING from projection (for example, in `SELECT a + 1 AS b FROM t1 JOIN t2 USING (b)` join will be performed by `t1.a + 1 = t2.b`, rather then `t1.b = t2.b`).", 0) \ + \ M(Bool, prefer_global_in_and_join, false, "If enabled, all IN/JOIN operators will be rewritten as GLOBAL IN/JOIN. It's useful when the to-be-joined tables are only available on the initiator and we need to always scatter their data on-the-fly during distributed processing with the GLOBAL keyword. It's also useful to reduce the need to access the external sources joining external tables.", 0) \ M(Bool, enable_vertical_final, true, "If enable, remove duplicated rows during FINAL by marking rows as deleted and filtering them later instead of merging rows", 0) \ \ @@ -590,13 +567,6 @@ class IColumn; M(UInt64, distributed_replica_error_cap, DBMS_CONNECTION_POOL_WITH_FAILOVER_MAX_ERROR_COUNT, "Max number of errors per replica, prevents piling up an incredible amount of errors if replica was offline for some time and allows it to be reconsidered in a shorter amount of time.", 0) \ M(UInt64, distributed_replica_max_ignored_errors, 0, "Number of errors that will be ignored while choosing replicas", 0) \ \ - M(Bool, allow_experimental_live_view, false, "Enable LIVE VIEW. Not mature enough.", 0) \ - M(Seconds, live_view_heartbeat_interval, 15, "The heartbeat interval in seconds to indicate live query is alive.", 0) \ - M(UInt64, max_live_view_insert_blocks_before_refresh, 64, "Limit maximum number of inserted blocks after which mergeable blocks are dropped and query is re-executed.", 0) \ - M(Bool, allow_experimental_window_view, false, "Enable WINDOW VIEW. Not mature enough.", 0) \ - M(Seconds, window_view_clean_interval, 60, "The clean interval of window view in seconds to free outdated data.", 0) \ - M(Seconds, window_view_heartbeat_interval, 15, "The heartbeat interval in seconds to indicate watch query is alive.", 0) \ - M(Seconds, wait_for_window_view_fire_signal_timeout, 10, "Timeout for waiting for window view fire signal in event time processing", 0) \ M(UInt64, min_free_disk_space_for_temporary_data, 0, "The minimum disk space to keep while writing temporary data used in external sorting and aggregation.", 0) \ \ M(DefaultTableEngine, default_temporary_table_engine, DefaultTableEngine::Memory, "Default table engine used when ENGINE is not set in CREATE TEMPORARY statement.",0) \ @@ -635,8 +605,6 @@ class IColumn; M(Bool, update_insert_deduplication_token_in_dependent_materialized_views, false, "Should update insert deduplication token with table identifier during insert in dependent materialized views.", 0) \ M(Bool, materialized_views_ignore_errors, false, "Allows to ignore errors for MATERIALIZED VIEW, and deliver original block to the table regardless of MVs", 0) \ M(Bool, ignore_materialized_views_with_dropped_target_table, false, "Ignore MVs with dropped target table during pushing to views", 0) \ - M(Bool, allow_experimental_refreshable_materialized_view, false, "Allow refreshable materialized views (CREATE MATERIALIZED VIEW REFRESH ...).", 0) \ - M(Bool, stop_refreshable_materialized_views_on_startup, false, "On server startup, prevent scheduling of refreshable materialized views, as if with SYSTEM STOP VIEWS. You can manually start them with SYSTEM START VIEWS or SYSTEM START VIEW afterwards. Also applies to newly created views. Has no effect on non-refreshable materialized views.", 0) \ M(Bool, use_compact_format_in_distributed_parts_names, true, "Changes format of directories names for distributed table insert parts.", 0) \ M(Bool, validate_polygons, true, "Throw exception if polygon is invalid in function pointInPolygon (e.g. self-tangent, self-intersecting). If the setting is false, the function will accept invalid polygons but may silently return wrong result.", 0) \ M(UInt64, max_parser_depth, DBMS_DEFAULT_MAX_PARSER_DEPTH, "Maximum parser depth (recursion depth of recursive descend parser).", 0) \ @@ -653,8 +621,6 @@ class IColumn; M(Bool, cast_keep_nullable, false, "CAST operator keep Nullable for result data type", 0) \ M(Bool, cast_ipv4_ipv6_default_on_conversion_error, false, "CAST operator into IPv4, CAST operator into IPV6 type, toIPv4, toIPv6 functions will return default value instead of throwing exception on conversion error.", 0) \ M(Bool, alter_partition_verbose_result, false, "Output information about affected parts. Currently works only for FREEZE and ATTACH commands.", 0) \ - M(Bool, allow_experimental_database_materialized_mysql, false, "Allow to create database with Engine=MaterializedMySQL(...).", 0) \ - M(Bool, allow_experimental_database_materialized_postgresql, false, "Allow to create database with Engine=MaterializedPostgreSQL(...).", 0) \ M(Bool, system_events_show_zero_values, false, "When querying system.events or system.metrics tables, include all metrics, even with zero values.", 0) \ M(MySQLDataTypesSupport, mysql_datatypes_support_level, MySQLDataTypesSupportList{}, "Defines how MySQL types are converted to corresponding ClickHouse types. A comma separated list in any combination of 'decimal', 'datetime64', 'date2Date32' or 'date2String'. decimal: convert NUMERIC and DECIMAL types to Decimal when precision allows it. datetime64: convert DATETIME and TIMESTAMP types to DateTime64 instead of DateTime when precision is not 0. date2Date32: convert DATE to Date32 instead of Date. Takes precedence over date2String. date2String: convert DATE to String instead of Date. Overridden by datetime64.", 0) \ M(Bool, optimize_trivial_insert_select, false, "Optimize trivial 'INSERT INTO table SELECT ... FROM TABLES' query", 0) \ @@ -716,9 +682,6 @@ class IColumn; M(Bool, force_aggregate_partitions_independently, false, "Force the use of optimization when it is applicable, but heuristics decided not to use it", 0) \ M(UInt64, max_number_of_partitions_for_independent_aggregation, 128, "Maximal number of partitions in table to apply optimization", 0) \ M(Float, min_hit_rate_to_use_consecutive_keys_optimization, 0.5, "Minimal hit rate of a cache which is used for consecutive keys optimization in aggregation to keep it enabled", 0) \ - /** Experimental feature for moving data between shards. */ \ - \ - M(Bool, allow_experimental_query_deduplication, false, "Experimental data deduplication for SELECT queries based on part UUIDs", 0) \ \ M(Bool, engine_file_empty_if_not_exists, false, "Allows to select data from a file engine table without file", 0) \ M(Bool, engine_file_truncate_on_insert, false, "Enables or disables truncate before insert in file engine tables", 0) \ @@ -898,34 +861,11 @@ class IColumn; M(Bool, allow_get_client_http_header, false, "Allow to use the function `getClientHTTPHeader` which lets to obtain a value of an the current HTTP request's header. It is not enabled by default for security reasons, because some headers, such as `Cookie`, could contain sensitive info. Note that the `X-ClickHouse-*` and `Authentication` headers are always restricted and cannot be obtained with this function.", 0) \ M(Bool, cast_string_to_dynamic_use_inference, false, "Use types inference during String to Dynamic conversion", 0) \ M(Bool, enable_blob_storage_log, true, "Write information about blob storage operations to system.blob_storage_log table", 0) \ - \ - /** Experimental functions */ \ - M(Bool, allow_experimental_materialized_postgresql_table, false, "Allows to use the MaterializedPostgreSQL table engine. Disabled by default, because this feature is experimental", 0) \ - M(Bool, allow_experimental_funnel_functions, false, "Enable experimental functions for funnel analysis.", 0) \ - M(Bool, allow_experimental_nlp_functions, false, "Enable experimental functions for natural language processing.", 0) \ - M(Bool, allow_experimental_hash_functions, false, "Enable experimental hash functions", 0) \ - M(Bool, allow_experimental_object_type, false, "Allow Object and JSON data types", 0) \ - M(Bool, allow_experimental_variant_type, false, "Allow Variant data type", 0) \ - M(Bool, allow_experimental_dynamic_type, false, "Allow Dynamic data type", 0) \ - M(Bool, allow_experimental_annoy_index, false, "Allows to use Annoy index. Disabled by default because this feature is experimental", 0) \ - M(Bool, allow_experimental_usearch_index, false, "Allows to use USearch index. Disabled by default because this feature is experimental", 0) \ - M(UInt64, max_limit_for_ann_queries, 1'000'000, "SELECT queries with LIMIT bigger than this setting cannot use ANN indexes. Helps to prevent memory overflows in ANN search indexes.", 0) \ - M(UInt64, max_threads_for_annoy_index_creation, 4, "Number of threads used to build Annoy indexes (0 means all cores, not recommended)", 0) \ - M(Int64, annoy_index_search_k_nodes, -1, "SELECT queries search up to this many nodes in Annoy indexes.", 0) \ - M(Bool, throw_on_unsupported_query_inside_transaction, true, "Throw exception if unsupported query is used inside transaction", 0) \ - M(TransactionsWaitCSNMode, wait_changes_become_visible_after_commit_mode, TransactionsWaitCSNMode::WAIT_UNKNOWN, "Wait for committed changes to become actually visible in the latest snapshot", 0) \ - M(Bool, implicit_transaction, false, "If enabled and not already inside a transaction, wraps the query inside a full transaction (begin + commit or rollback)", 0) \ - M(UInt64, grace_hash_join_initial_buckets, 1, "Initial number of grace hash join buckets", 0) \ - M(UInt64, grace_hash_join_max_buckets, 1024, "Limit on the number of grace hash join buckets", 0) \ - M(Bool, optimize_distinct_in_order, true, "Enable DISTINCT optimization if some columns in DISTINCT form a prefix of sorting. For example, prefix of sorting key in merge tree or ORDER BY statement", 0) \ - M(Bool, keeper_map_strict_mode, false, "Enforce additional checks during operations on KeeperMap. E.g. throw an exception on an insert for already existing key", 0) \ - M(UInt64, extract_key_value_pairs_max_pairs_per_row, 1000, "Max number of pairs that can be produced by the `extractKeyValuePairs` function. Used as a safeguard against consuming too much memory.", 0) ALIAS(extract_kvp_max_pairs_per_row) \ - M(Timezone, session_timezone, "", "This setting can be removed in the future due to potential caveats. It is experimental and is not suitable for production usage. The default timezone for current session or query. The server default timezone if empty.", 0) \ M(Bool, allow_create_index_without_type, false, "Allow CREATE INDEX query without TYPE. Query will be ignored. Made for SQL compatibility tests.", 0) \ M(Bool, create_index_ignore_unique, false, "Ignore UNIQUE keyword in CREATE UNIQUE INDEX. Made for SQL compatibility tests.", 0) \ M(Bool, print_pretty_type_names, true, "Print pretty type names in DESCRIBE query and toTypeName() function", 0) \ M(Bool, create_table_empty_primary_key_by_default, false, "Allow to create *MergeTree tables with empty primary key when ORDER BY and PRIMARY KEY not specified", 0) \ - M(Bool, allow_named_collection_override_by_default, true, "Allow named collections' fields override by default.", 0)\ + M(Bool, allow_named_collection_override_by_default, true, "Allow named collections' fields override by default.", 0) \ M(SQLSecurityType, default_normal_view_sql_security, SQLSecurityType::INVOKER, "Allows to set a default value for SQL SECURITY option when creating a normal view.", 0) \ M(SQLSecurityType, default_materialized_view_sql_security, SQLSecurityType::DEFINER, "Allows to set a default value for SQL SECURITY option when creating a materialized view.", 0) \ M(String, default_view_definer, "CURRENT_USER", "Allows to set a default value for DEFINER option when creating view.", 0) \ @@ -935,6 +875,81 @@ class IColumn; M(Bool, iceberg_engine_ignore_schema_evolution, false, "Ignore schema evolution in Iceberg table engine and read all data using latest schema saved on table creation. Note that it can lead to incorrect result", 0) \ M(Bool, allow_deprecated_error_prone_window_functions, false, "Allow usage of deprecated error prone window functions (neighbor, runningAccumulate, runningDifferenceStartingWithFirstValue, runningDifference)", 0) \ M(Bool, allow_deprecated_snowflake_conversion_functions, false, "Enables deprecated functions snowflakeToDateTime[64] and dateTime[64]ToSnowflake.", 0) \ + M(Bool, optimize_distinct_in_order, true, "Enable DISTINCT optimization if some columns in DISTINCT form a prefix of sorting. For example, prefix of sorting key in merge tree or ORDER BY statement", 0) \ + M(Bool, keeper_map_strict_mode, false, "Enforce additional checks during operations on KeeperMap. E.g. throw an exception on an insert for already existing key", 0) \ + M(UInt64, extract_key_value_pairs_max_pairs_per_row, 1000, "Max number of pairs that can be produced by the `extractKeyValuePairs` function. Used as a safeguard against consuming too much memory.", 0) ALIAS(extract_kvp_max_pairs_per_row) \ + \ + /** Experimental features */ \ + M(Bool, allow_experimental_materialized_postgresql_table, false, "Allows to use the MaterializedPostgreSQL table engine. Disabled by default, because this feature is experimental", 0) \ + M(Bool, allow_experimental_funnel_functions, false, "Enable experimental functions for funnel analysis.", 0) \ + M(Bool, allow_experimental_nlp_functions, false, "Enable experimental functions for natural language processing.", 0) \ + M(Bool, allow_experimental_hash_functions, false, "Enable experimental hash functions", 0) \ + M(Bool, allow_experimental_object_type, false, "Allow Object and JSON data types", 0) \ + M(Bool, allow_experimental_variant_type, false, "Allow Variant data type", 0) \ + M(Bool, allow_experimental_dynamic_type, false, "Allow Dynamic data type", 0) \ + M(Bool, allow_experimental_annoy_index, false, "Allows to use Annoy index. Disabled by default because this feature is experimental", 0) \ + M(Bool, allow_experimental_usearch_index, false, "Allows to use USearch index. Disabled by default because this feature is experimental", 0) \ + M(Bool, allow_experimental_codecs, false, "If it is set to true, allow to specify experimental compression codecs (but we don't have those yet and this option does nothing).", 0) \ + M(UInt64, max_limit_for_ann_queries, 1'000'000, "SELECT queries with LIMIT bigger than this setting cannot use ANN indexes. Helps to prevent memory overflows in ANN search indexes.", 0) \ + M(UInt64, max_threads_for_annoy_index_creation, 4, "Number of threads used to build Annoy indexes (0 means all cores, not recommended)", 0) \ + M(Int64, annoy_index_search_k_nodes, -1, "SELECT queries search up to this many nodes in Annoy indexes.", 0) \ + M(Bool, throw_on_unsupported_query_inside_transaction, true, "Throw exception if unsupported query is used inside transaction", 0) \ + M(TransactionsWaitCSNMode, wait_changes_become_visible_after_commit_mode, TransactionsWaitCSNMode::WAIT_UNKNOWN, "Wait for committed changes to become actually visible in the latest snapshot", 0) \ + M(Bool, implicit_transaction, false, "If enabled and not already inside a transaction, wraps the query inside a full transaction (begin + commit or rollback)", 0) \ + M(UInt64, grace_hash_join_initial_buckets, 1, "Initial number of grace hash join buckets", 0) \ + M(UInt64, grace_hash_join_max_buckets, 1024, "Limit on the number of grace hash join buckets", 0) \ + M(Timezone, session_timezone, "", "This setting can be removed in the future due to potential caveats. It is experimental and is not suitable for production usage. The default timezone for current session or query. The server default timezone if empty.", 0) \ + \ + M(Bool, allow_statistics_optimize, false, "Allows using statistics to optimize queries", 0) ALIAS(allow_statistic_optimize) \ + M(Bool, allow_experimental_statistics, false, "Allows using statistics", 0) ALIAS(allow_experimental_statistic) \ + \ + /* Parallel replicas */ \ + M(UInt64, parallel_replicas_count, 0, "This is internal setting that should not be used directly and represents an implementation detail of the 'parallel replicas' mode. This setting will be automatically set up by the initiator server for distributed queries to the number of parallel replicas participating in query processing.", 0) \ + M(UInt64, parallel_replica_offset, 0, "This is internal setting that should not be used directly and represents an implementation detail of the 'parallel replicas' mode. This setting will be automatically set up by the initiator server for distributed queries to the index of the replica participating in query processing among parallel replicas.", 0) \ + M(String, parallel_replicas_custom_key, "", "Custom key assigning work to replicas when parallel replicas are used.", 0) \ + M(ParallelReplicasCustomKeyFilterType, parallel_replicas_custom_key_filter_type, ParallelReplicasCustomKeyFilterType::DEFAULT, "Type of filter to use with custom key for parallel replicas. default - use modulo operation on the custom key, range - use range filter on custom key using all possible values for the value type of custom key.", 0) \ + M(UInt64, parallel_replicas_custom_key_range_lower, 0, "Lower bound for the universe that the parallel replicas custom range filter is calculated over", 0) \ + M(UInt64, parallel_replicas_custom_key_range_upper, 0, "Upper bound for the universe that the parallel replicas custom range filter is calculated over. A value of 0 disables the upper bound, setting it to the max value of the custom key expression", 0) \ + M(String, cluster_for_parallel_replicas, "", "Cluster for a shard in which current server is located", 0) \ + M(UInt64, allow_experimental_parallel_reading_from_replicas, 0, "Use all the replicas from a shard for SELECT query execution. Reading is parallelized and coordinated dynamically. 0 - disabled, 1 - enabled, silently disable them in case of failure, 2 - enabled, throw an exception in case of failure", 0) \ + M(Bool, parallel_replicas_allow_in_with_subquery, true, "If true, subquery for IN will be executed on every follower replica.", 0) \ + M(Float, parallel_replicas_single_task_marks_count_multiplier, 2, "A multiplier which will be added during calculation for minimal number of marks to retrieve from coordinator. This will be applied only for remote replicas.", 0) \ + M(Bool, parallel_replicas_for_non_replicated_merge_tree, false, "If true, ClickHouse will use parallel replicas algorithm also for non-replicated MergeTree tables", 0) \ + M(UInt64, parallel_replicas_min_number_of_rows_per_replica, 0, "Limit the number of replicas used in a query to (estimated rows to read / min_number_of_rows_per_replica). The max is still limited by 'max_parallel_replicas'", 0) \ + M(Bool, parallel_replicas_prefer_local_join, true, "If true, and JOIN can be executed with parallel replicas algorithm, and all storages of right JOIN part are *MergeTree, local JOIN will be used instead of GLOBAL JOIN.", 0) \ + M(UInt64, parallel_replicas_mark_segment_size, 128, "Parts virtually divided into segments to be distributed between replicas for parallel reading. This setting controls the size of these segments. Not recommended to change until you're absolutely sure in what you're doing", 0) \ + \ + M(Bool, allow_experimental_inverted_index, false, "If it is set to true, allow to use experimental inverted index.", 0) \ + M(Bool, allow_experimental_full_text_index, false, "If it is set to true, allow to use experimental full-text index.", 0) \ + \ + M(Bool, allow_experimental_join_condition, false, "Support join with inequal conditions which involve columns from both left and right table. e.g. t1.y < t2.y.", 0) \ + \ + /* Analyzer: It's not experimental anymore (WIP) */ \ + M(Bool, allow_experimental_analyzer, true, "Allow new query analyzer.", 0) \ + M(Bool, analyzer_compatibility_join_using_top_level_identifier, false, "Force to resolve identifier in JOIN USING from projection (for example, in `SELECT a + 1 AS b FROM t1 JOIN t2 USING (b)` join will be performed by `t1.a + 1 = t2.b`, rather then `t1.b = t2.b`).", 0) \ + \ + M(Bool, allow_experimental_live_view, false, "Enable LIVE VIEW. Not mature enough.", 0) \ + M(Seconds, live_view_heartbeat_interval, 15, "The heartbeat interval in seconds to indicate live query is alive.", 0) \ + M(UInt64, max_live_view_insert_blocks_before_refresh, 64, "Limit maximum number of inserted blocks after which mergeable blocks are dropped and query is re-executed.", 0) \ + \ + M(Bool, allow_experimental_window_view, false, "Enable WINDOW VIEW. Not mature enough.", 0) \ + M(Seconds, window_view_clean_interval, 60, "The clean interval of window view in seconds to free outdated data.", 0) \ + M(Seconds, window_view_heartbeat_interval, 15, "The heartbeat interval in seconds to indicate watch query is alive.", 0) \ + M(Seconds, wait_for_window_view_fire_signal_timeout, 10, "Timeout for waiting for window view fire signal in event time processing", 0) \ + \ + M(Bool, allow_experimental_refreshable_materialized_view, false, "Allow refreshable materialized views (CREATE MATERIALIZED VIEW REFRESH ...).", 0) \ + M(Bool, stop_refreshable_materialized_views_on_startup, false, "On server startup, prevent scheduling of refreshable materialized views, as if with SYSTEM STOP VIEWS. You can manually start them with SYSTEM START VIEWS or SYSTEM START VIEW afterwards. Also applies to newly created views. Has no effect on non-refreshable materialized views.", 0) \ + \ + M(Bool, allow_experimental_database_materialized_mysql, false, "Allow to create database with Engine=MaterializedMySQL(...).", 0) \ + M(Bool, allow_experimental_database_materialized_postgresql, false, "Allow to create database with Engine=MaterializedPostgreSQL(...).", 0) \ + \ + /** Experimental feature for moving data between shards. */ \ + M(Bool, allow_experimental_query_deduplication, false, "Experimental data deduplication for SELECT queries based on part UUIDs", 0) \ + + + + /** End of experimental features */ + // End of COMMON_SETTINGS // Please add settings related to formats into the FORMAT_FACTORY_SETTINGS, move obsolete settings to OBSOLETE_SETTINGS and obsolete format settings to OBSOLETE_FORMAT_SETTINGS. From 601ee4ee3e50b2967063a124ac921753e3546d6e Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?Ra=C3=BAl=20Mar=C3=ADn?= Date: Wed, 3 Jul 2024 12:08:02 +0200 Subject: [PATCH 143/417] Update Settings.h --- src/Core/Settings.h | 2 -- 1 file changed, 2 deletions(-) diff --git a/src/Core/Settings.h b/src/Core/Settings.h index ee56c1133bf..b45c1e38d1c 100644 --- a/src/Core/Settings.h +++ b/src/Core/Settings.h @@ -946,8 +946,6 @@ class IColumn; /** Experimental feature for moving data between shards. */ \ M(Bool, allow_experimental_query_deduplication, false, "Experimental data deduplication for SELECT queries based on part UUIDs", 0) \ - - /** End of experimental features */ From fe43ea27d2414a8b52d3002dad392bea5ee0b028 Mon Sep 17 00:00:00 2001 From: Nikita Fomichev Date: Wed, 3 Jul 2024 13:37:36 +0200 Subject: [PATCH 144/417] Tests: fix timeouts handling in case of github timeout termination --- docker/test/fasttest/run.sh | 15 ++++++++++++++- docker/test/stateless/run.sh | 8 ++++++-- docker/test/stateless/utils.lib | 2 +- 3 files changed, 21 insertions(+), 4 deletions(-) diff --git a/docker/test/fasttest/run.sh b/docker/test/fasttest/run.sh index 4d5159cfa9e..2bed4c5c343 100755 --- a/docker/test/fasttest/run.sh +++ b/docker/test/fasttest/run.sh @@ -254,6 +254,19 @@ function configure rm -f "$FASTTEST_DATA/config.d/secure_ports.xml" } +function timeout_with_logging() { + local exit_code=0 + + timeout -s TERM --preserve-status "${@}" || exit_code="${?}" + + if [[ "${exit_code}" -eq "124" ]] + then + echo "The command 'timeout ${*}' has been killed by timeout" + fi + + return $exit_code +} + function run_tests { clickhouse-server --version @@ -315,7 +328,7 @@ case "$stage" in configure 2>&1 | ts '%Y-%m-%d %H:%M:%S' | tee "$FASTTEST_OUTPUT/install_log.txt" ;& "run_tests") - run_tests + timeout_with_logging 35m bash -c run_tests ||: /process_functional_tests_result.py --in-results-dir "$FASTTEST_OUTPUT/" \ --out-results-file "$FASTTEST_OUTPUT/test_results.tsv" \ --out-status-file "$FASTTEST_OUTPUT/check_status.tsv" || echo -e "failure\tCannot parse results" > "$FASTTEST_OUTPUT/check_status.tsv" diff --git a/docker/test/stateless/run.sh b/docker/test/stateless/run.sh index 3ce489b9e0e..2cbc5304212 100755 --- a/docker/test/stateless/run.sh +++ b/docker/test/stateless/run.sh @@ -6,6 +6,9 @@ source /setup_export_logs.sh # fail on errors, verbose and export all env variables set -e -x -a +MAX_RUN_TIME=${MAX_RUN_TIME:-10800} +MAX_RUN_TIME=$((MAX_RUN_TIME == 0 ? 10800 : MAX_RUN_TIME)) + # Choose random timezone for this test run. # # NOTE: that clickhouse-test will randomize session_timezone by itself as well @@ -262,14 +265,15 @@ function run_tests() export -f run_tests +TIMEOUT=$((${MAX_RUN_TIME} - 200)) if [ "$NUM_TRIES" -gt "1" ]; then # We don't run tests with Ordinary database in PRs, only in master. # So run new/changed tests with Ordinary at least once in flaky check. - timeout_with_logging "$MAX_RUN_TIME" bash -c 'NUM_TRIES=1; USE_DATABASE_ORDINARY=1; run_tests' \ + timeout_with_logging "$TIMEOUT" bash -c 'NUM_TRIES=1; USE_DATABASE_ORDINARY=1; run_tests' \ | sed 's/All tests have finished//' | sed 's/No tests were run//' ||: fi -timeout_with_logging "$MAX_RUN_TIME" bash -c run_tests ||: +timeout_with_logging "$TIMEOUT" bash -c run_tests ||: echo "Files in current directory" ls -la ./ diff --git a/docker/test/stateless/utils.lib b/docker/test/stateless/utils.lib index 9b6ab535a90..833e1a05384 100644 --- a/docker/test/stateless/utils.lib +++ b/docker/test/stateless/utils.lib @@ -38,7 +38,7 @@ function fn_exists() { function timeout_with_logging() { local exit_code=0 - timeout "${@}" || exit_code="${?}" + timeout -s TERM --preserve-status "${@}" || exit_code="${?}" if [[ "${exit_code}" -eq "124" ]] then From dd28947e827ff9264b45e0a8130b3b44a685c097 Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?Ra=C3=BAl=20Mar=C3=ADn?= Date: Wed, 3 Jul 2024 13:41:39 +0200 Subject: [PATCH 145/417] Update Settings.h --- src/Core/Settings.h | 6 ++++-- 1 file changed, 4 insertions(+), 2 deletions(-) diff --git a/src/Core/Settings.h b/src/Core/Settings.h index b45c1e38d1c..1ab67a8e5b7 100644 --- a/src/Core/Settings.h +++ b/src/Core/Settings.h @@ -879,7 +879,10 @@ class IColumn; M(Bool, keeper_map_strict_mode, false, "Enforce additional checks during operations on KeeperMap. E.g. throw an exception on an insert for already existing key", 0) \ M(UInt64, extract_key_value_pairs_max_pairs_per_row, 1000, "Max number of pairs that can be produced by the `extractKeyValuePairs` function. Used as a safeguard against consuming too much memory.", 0) ALIAS(extract_kvp_max_pairs_per_row) \ \ - /** Experimental features */ \ + \ + /* ###################################### */ \ + /* ######## EXPERIMENTAL FEATURES ####### */ \ + /* ###################################### */ \ M(Bool, allow_experimental_materialized_postgresql_table, false, "Allows to use the MaterializedPostgreSQL table engine. Disabled by default, because this feature is experimental", 0) \ M(Bool, allow_experimental_funnel_functions, false, "Enable experimental functions for funnel analysis.", 0) \ M(Bool, allow_experimental_nlp_functions, false, "Enable experimental functions for natural language processing.", 0) \ @@ -948,7 +951,6 @@ class IColumn; /** End of experimental features */ - // End of COMMON_SETTINGS // Please add settings related to formats into the FORMAT_FACTORY_SETTINGS, move obsolete settings to OBSOLETE_SETTINGS and obsolete format settings to OBSOLETE_FORMAT_SETTINGS. From b88be7260f6ce1eda9b949e1aa297eea5a2a110f Mon Sep 17 00:00:00 2001 From: Nikita Fomichev Date: Wed, 3 Jul 2024 13:40:36 +0200 Subject: [PATCH 146/417] Tests: Eliminating the global tests queue to prevent clickhouse-test from hanging when a server dies --- tests/clickhouse-test | 121 +++++++++++------------------------------- 1 file changed, 30 insertions(+), 91 deletions(-) diff --git a/tests/clickhouse-test b/tests/clickhouse-test index 36870d59c3a..8e2a256fae2 100755 --- a/tests/clickhouse-test +++ b/tests/clickhouse-test @@ -34,10 +34,8 @@ import urllib.parse # for crc32 import zlib from argparse import ArgumentParser -from contextlib import closing from datetime import datetime, timedelta from errno import ESRCH -from queue import Full from subprocess import PIPE, Popen from time import sleep, time from typing import Dict, List, Optional, Set, Tuple, Union @@ -360,39 +358,6 @@ def clickhouse_execute_json( return rows -class Terminated(KeyboardInterrupt): - pass - - -def signal_handler(sig, frame): - raise Terminated(f"Terminated with {sig} signal") - - -def stop_tests(): - global stop_tests_triggered_lock - global stop_tests_triggered - global restarted_tests - - with stop_tests_triggered_lock: - print("Stopping tests") - if not stop_tests_triggered.is_set(): - stop_tests_triggered.set() - - # materialize multiprocessing.Manager().list() object before - # sending SIGTERM since this object is a proxy, that requires - # communicating with manager thread, but after SIGTERM will be - # send, this thread will die, and you will get - # ConnectionRefusedError error for any access to "restarted_tests" - # variable. - restarted_tests = [*restarted_tests] - - # send signal to all processes in group to avoid hung check triggering - # (to avoid terminating clickhouse-test itself, the signal should be ignored) - signal.signal(signal.SIGTERM, signal.SIG_IGN) - os.killpg(os.getpgid(os.getpid()), signal.SIGTERM) - signal.signal(signal.SIGTERM, signal.SIG_DFL) - - def get_db_engine(args, database_name): if args.replicated_database: return f" ON CLUSTER test_cluster_database_replicated \ @@ -2061,13 +2026,18 @@ class TestSuite: stop_time = None exit_code = None server_died = None -stop_tests_triggered_lock = None -stop_tests_triggered = None -queue = None multiprocessing_manager = None restarted_tests = None +class ServerDied(Exception): + pass + + +class GlobalTimeout(Exception): + pass + + def run_tests_array(all_tests_with_params: Tuple[List[str], int, TestSuite]): all_tests, num_tests, test_suite = all_tests_with_params global stop_time @@ -2122,24 +2092,17 @@ def run_tests_array(all_tests_with_params: Tuple[List[str], int, TestSuite]): print(f"\nRunning {about}{num_tests} {test_suite.suite} tests ({proc_name}).\n") while True: - if is_concurrent: - case = queue.get(timeout=args.timeout * 1.1) - if not case: - break + if all_tests: + case = all_tests.pop(0) else: - if all_tests: - case = all_tests.pop(0) - else: - break + break if server_died.is_set(): - stop_tests() - break + raise ServerDied("Server died") if stop_time and time() > stop_time: print("\nStop tests run because global time limit is exceeded.\n") - stop_tests() - break + raise GlobalTimeout("Stop tests run because global time limit is exceeded") test_case = TestCase(test_suite, case, args, is_concurrent) @@ -2182,18 +2145,15 @@ def run_tests_array(all_tests_with_params: Tuple[List[str], int, TestSuite]): failures_chain += 1 if test_result.reason == FailureReason.SERVER_DIED: server_died.set() - stop_tests() elif test_result.status == TestStatus.SKIPPED: skipped_total += 1 except KeyboardInterrupt as e: print(colored("Break tests execution", args, "red")) - stop_tests() raise e if failures_chain >= args.max_failures_chain: - stop_tests() - break + raise ServerDied("Max failures chain") if failures_total > 0: print( @@ -2390,7 +2350,7 @@ def extract_key(key: str) -> str: )[1] -def do_run_tests(jobs, test_suite: TestSuite, parallel): +def do_run_tests(jobs, test_suite: TestSuite): if jobs > 1 and len(test_suite.parallel_tests) > 0: print( "Found", @@ -2399,19 +2359,8 @@ def do_run_tests(jobs, test_suite: TestSuite, parallel): len(test_suite.sequential_tests), "sequential tests", ) - run_n, run_total = parallel.split("/") - run_n = float(run_n) - run_total = float(run_total) tests_n = len(test_suite.parallel_tests) - run_total = min(run_total, tests_n) - jobs = min(jobs, tests_n) - run_total = max(jobs, run_total) - - batch_size = max(1, len(test_suite.parallel_tests) // jobs) - parallel_tests_array = [] - for _ in range(jobs): - parallel_tests_array.append((None, batch_size, test_suite)) # If we don't do random shuffling then there will be always # nearly the same groups of test suites running concurrently. @@ -2424,25 +2373,21 @@ def do_run_tests(jobs, test_suite: TestSuite, parallel): # of failures will be nearly the same for all tests from the group. random.shuffle(test_suite.parallel_tests) + batch_size = max(1, len(test_suite.parallel_tests) // jobs) + parallel_tests_array = [] + for job in range(jobs): + range_ = job * batch_size, job * batch_size + batch_size + batch = test_suite.parallel_tests[range_[0] : range_[1]] + parallel_tests_array.append((batch, batch_size, test_suite)) + try: - with closing(multiprocessing.Pool(processes=jobs)) as pool: - pool.map_async(run_tests_array, parallel_tests_array) - - for suit in test_suite.parallel_tests: - queue.put(suit, timeout=args.timeout * 1.1) - - for _ in range(jobs): - queue.put(None, timeout=args.timeout * 1.1) - - queue.close() - except Full: - print( - "Couldn't put test to the queue within timeout. Server probably hung." - ) - print_stacktraces() - queue.close() - - pool.join() + with multiprocessing.Pool(processes=jobs) as pool: + future = pool.map_async(run_tests_array, parallel_tests_array) + future.wait() + finally: + pool.terminate() + pool.close() + pool.join() run_tests_array( (test_suite.sequential_tests, len(test_suite.sequential_tests), test_suite) @@ -2807,7 +2752,7 @@ def main(args): test_suite.cloud_skip_list = cloud_skip_list test_suite.private_skip_list = private_skip_list - total_tests_run += do_run_tests(args.jobs, test_suite, args.parallel) + total_tests_run += do_run_tests(args.jobs, test_suite) if server_died.is_set(): exit_code.value = 1 @@ -3268,9 +3213,6 @@ if __name__ == "__main__": stop_time = None exit_code = multiprocessing.Value("i", 0) server_died = multiprocessing.Event() - stop_tests_triggered_lock = multiprocessing.Lock() - stop_tests_triggered = multiprocessing.Event() - queue = multiprocessing.Queue(maxsize=1) multiprocessing_manager = multiprocessing.Manager() restarted_tests = multiprocessing_manager.list() @@ -3278,9 +3220,6 @@ if __name__ == "__main__": # infinite tests processes left # (new process group is required to avoid killing some parent processes) os.setpgid(0, 0) - signal.signal(signal.SIGTERM, signal_handler) - signal.signal(signal.SIGINT, signal_handler) - signal.signal(signal.SIGHUP, signal_handler) try: args = parse_args() From 7a993d737b55177a27436f98159f0458f03610fc Mon Sep 17 00:00:00 2001 From: vdimir Date: Wed, 3 Jul 2024 11:45:31 +0000 Subject: [PATCH 147/417] better --- .../Passes/LogicalExpressionOptimizerPass.cpp | 82 +++++++++---------- ...11_join_on_nullsafe_optimization.reference | 20 +++++ .../02911_join_on_nullsafe_optimization.sql | 25 ++++++ 3 files changed, 85 insertions(+), 42 deletions(-) diff --git a/src/Analyzer/Passes/LogicalExpressionOptimizerPass.cpp b/src/Analyzer/Passes/LogicalExpressionOptimizerPass.cpp index c74148a7252..698602ca5bc 100644 --- a/src/Analyzer/Passes/LogicalExpressionOptimizerPass.cpp +++ b/src/Analyzer/Passes/LogicalExpressionOptimizerPass.cpp @@ -66,7 +66,7 @@ QueryTreeNodePtr findEqualsFunction(const QueryTreeNodes & nodes) return nullptr; } -bool isNodeBooleanConstant(const QueryTreeNodePtr & node, bool expected_value) +bool isBooleanConstant(const QueryTreeNodePtr & node, bool expected_value) { const auto * constant_node = node->as(); if (!constant_node || !constant_node->getResultType()->equals(DataTypeUInt8())) @@ -82,10 +82,8 @@ bool isOnlyConjunctionOfFunctions( const String & func_name, const QueryTreeNodePtrWithHashSet & allowed_arguments) { - if (isNodeBooleanConstant(node, true)) - { + if (isBooleanConstant(node, true)) return true; - } const auto * node_function = node->as(); if (!node_function) @@ -141,15 +139,16 @@ public: { auto * function_node = node->as(); - if (!function_node) - return; + QueryTreeNodePtr new_node = nullptr; + if (function_node && function_node->getFunctionName() == "or") + new_node = tryOptimizeJoinOnNulls(function_node->getArguments().getNodes(), getContext()); + else + new_node = tryOptimizeJoinOnNulls({node}, getContext()); - if (function_node->getFunctionName() == "or") + if (new_node) { - bool is_argument_type_changed = tryOptimizeIsNotDistinctOrIsNull(node, getContext()); - if (is_argument_type_changed) - need_rerun_resolve = true; - return; + need_rerun_resolve |= !new_node->getResultType()->equals(*node->getResultType()); + node = new_node; } } @@ -166,14 +165,11 @@ private: const JoinNode * join_node; bool need_rerun_resolve = false; - /// Returns true if type of some operand is changed and parent function needs to be re-resolved - bool tryOptimizeIsNotDistinctOrIsNull(QueryTreeNodePtr & node, const ContextPtr & context) + /// Returns optimized node or nullptr if nothing have been changed + QueryTreeNodePtr tryOptimizeJoinOnNulls(const QueryTreeNodes & nodes, const ContextPtr & context) { - auto & function_node = node->as(); - chassert(function_node.getFunctionName() == "or"); - QueryTreeNodes or_operands; - or_operands.reserve(function_node.getArguments().getNodes().size()); + or_operands.reserve(nodes.size()); /// Indices of `equals` or `isNotDistinctFrom` functions in the vector above std::vector equals_functions_indices; @@ -192,17 +188,17 @@ private: bool is_anything_changed = false; - for (const auto & argument : function_node.getArguments()) + for (const auto & node : nodes) { - if (isNodeBooleanConstant(argument, false)) + if (isBooleanConstant(node, false)) { /// Remove false constants from OR is_anything_changed = true; continue; } - or_operands.push_back(argument); - auto * argument_function = argument->as(); + or_operands.push_back(node); + auto * argument_function = node->as(); if (!argument_function) continue; @@ -223,6 +219,8 @@ private: } /// Expression `a = b AND (a IS NOT NULL) AND true AND (b IS NOT NULL)` we can be replaced with `a = b` + /// Even though this expression are not equivalent (first is NULL on NULLs, while second is FALSE), + /// it is still correct since for JOIN ON condition NULL is treated as FALSE if (const auto & equals_function = findEqualsFunction(and_arguments)) { const auto & equals_arguments = equals_function->as()->getArguments().getNodes(); @@ -261,7 +259,7 @@ private: for (size_t equals_function_idx : equals_functions_indices) { - auto * equals_function = or_operands[equals_function_idx]->as(); + const auto * equals_function = or_operands[equals_function_idx]->as(); /// For a = b we are looking for all expressions `a IS NULL AND b IS NULL` const auto & argument_nodes = equals_function->getArguments().getNodes(); @@ -279,40 +277,39 @@ private: for (size_t to_optimize_idx : operands_to_optimize) { /// Remove `a IS NULL AND b IS NULL` - auto * operand_to_optimize = or_operands[to_optimize_idx]->as(); - operand_to_optimize->getArguments().getNodes() = {}; - arguments_to_reresolve.insert(to_optimize_idx); + or_operands[to_optimize_idx] = nullptr; + is_anything_changed = true; } } if (arguments_to_reresolve.empty() && !is_anything_changed) /// Nothing have been changed - return false; + return nullptr; auto and_function_resolver = FunctionFactory::instance().get("and", context); auto strict_equals_function_resolver = FunctionFactory::instance().get("isNotDistinctFrom", context); - bool need_reresolve = false; QueryTreeNodes new_or_operands; for (size_t i = 0; i < or_operands.size(); ++i) { if (arguments_to_reresolve.contains(i)) { - auto * function = or_operands[i]->as(); + const auto * function = or_operands[i]->as(); if (function->getFunctionName() == "equals") { /// We should replace `a = b` with `a <=> b` because we removed checks for IS NULL - need_reresolve |= function->getResultType()->isNullable(); - function->resolveAsFunction(strict_equals_function_resolver); - new_or_operands.emplace_back(std::move(or_operands[i])); + auto new_function = or_operands[i]->clone(); + new_function->as()->resolveAsFunction(strict_equals_function_resolver); + new_or_operands.emplace_back(std::move(new_function)); } else if (function->getFunctionName() == "and") { const auto & and_arguments = function->getArguments().getNodes(); if (and_arguments.size() > 1) { - function->resolveAsFunction(and_function_resolver); - new_or_operands.emplace_back(std::move(or_operands[i])); + auto new_function = or_operands[i]->clone(); + new_function->as()->resolveAsFunction(and_function_resolver); + new_or_operands.emplace_back(std::move(new_function)); } else if (and_arguments.size() == 1) { @@ -321,25 +318,26 @@ private: } } else - throw Exception(ErrorCodes::LOGICAL_ERROR, "Unexpected function name: '{}'", function->getFunctionName()); + throw Exception(ErrorCodes::LOGICAL_ERROR, "Unexpected function '{}'", function->getFunctionName()); } - else + else if (or_operands[i]) { new_or_operands.emplace_back(std::move(or_operands[i])); } } + if (new_or_operands.empty()) + return nullptr; + if (new_or_operands.size() == 1) - { - node = std::move(new_or_operands[0]); - return need_reresolve; - } + return new_or_operands[0]; /// Rebuild OR function auto or_function_resolver = FunctionFactory::instance().get("or", context); - function_node.getArguments().getNodes() = std::move(new_or_operands); - function_node.resolveAsFunction(or_function_resolver); - return need_reresolve; + auto function_node = std::make_shared("or"); + function_node->getArguments().getNodes() = std::move(new_or_operands); + function_node->resolveAsFunction(or_function_resolver); + return function_node; } }; diff --git a/tests/queries/0_stateless/02911_join_on_nullsafe_optimization.reference b/tests/queries/0_stateless/02911_join_on_nullsafe_optimization.reference index 1df3606592c..f0463509b80 100644 --- a/tests/queries/0_stateless/02911_join_on_nullsafe_optimization.reference +++ b/tests/queries/0_stateless/02911_join_on_nullsafe_optimization.reference @@ -35,6 +35,26 @@ SELECT * FROM t1 JOIN t2 ON (t1.x == t2.x AND ((t2.x IS NOT NULL) AND (t1.x IS N 2 2 2 2 3 3 3 33 \N \N \N \N +-- aliases defined in the join condition are valid +SELECT *, e, e2 FROM t1 FULL JOIN t2 ON ( ( ((t1.x == t2.x) AS e) AND ((t2.x IS NOT NULL) AND (t1.x IS NOT NULL)) ) OR ( (t2.x IS NULL) AND (t1.x IS NULL) ) AS e2 ) ORDER BY t1.x NULLS LAST, t2.x NULLS LAST; +1 42 \N \N \N 0 +2 2 2 2 1 1 +3 3 3 33 1 1 +\N \N 4 42 \N 0 +\N \N \N \N \N 1 +SELECT *, e, e2 FROM t1 FULL JOIN t2 ON ( ( ((t1.x == t2.x) AS e) AND ((t2.x IS NOT NULL) AND (t1.x IS NOT NULL)) ) AS e2 ) ORDER BY t1.x NULLS LAST, t2.x NULLS LAST; +1 42 \N \N \N 0 +2 2 2 2 1 1 +3 3 3 33 1 1 +\N \N 4 42 \N 0 +\N \N \N \N \N 0 +\N \N \N \N \N 0 +-- check for non-nullable columns for which `is null` is replaced with constant SELECT * FROM t1n as t1 JOIN t2n as t2 ON (t1.x == t2.x AND ((t2.x IS NOT NULL) AND (t1.x IS NOT NULL)) ) OR ( (t2.x IS NULL) AND (t1.x IS NULL) ) ORDER BY t1.x NULLS LAST; 2 2 2 2 3 3 3 33 +-- +0 +0 +2 +2 diff --git a/tests/queries/0_stateless/02911_join_on_nullsafe_optimization.sql b/tests/queries/0_stateless/02911_join_on_nullsafe_optimization.sql index 0a642a716a4..67918f4302f 100644 --- a/tests/queries/0_stateless/02911_join_on_nullsafe_optimization.sql +++ b/tests/queries/0_stateless/02911_join_on_nullsafe_optimization.sql @@ -35,10 +35,35 @@ SELECT x = y OR (x IS NULL AND y IS NULL) FROM t1 ORDER BY x NULLS LAST; SELECT * FROM t1 JOIN t2 ON (t1.x == t2.x AND ((t2.x IS NOT NULL) AND (t1.x IS NOT NULL)) ) OR ( (t2.x IS NULL) AND (t1.x IS NULL) ) ORDER BY t1.x NULLS LAST; +-- aliases defined in the join condition are valid +SELECT *, e, e2 FROM t1 FULL JOIN t2 ON ( ( ((t1.x == t2.x) AS e) AND ((t2.x IS NOT NULL) AND (t1.x IS NOT NULL)) ) OR ( (t2.x IS NULL) AND (t1.x IS NULL) ) AS e2 ) ORDER BY t1.x NULLS LAST, t2.x NULLS LAST; +SELECT *, e, e2 FROM t1 FULL JOIN t2 ON ( ( ((t1.x == t2.x) AS e) AND ((t2.x IS NOT NULL) AND (t1.x IS NOT NULL)) ) AS e2 ) ORDER BY t1.x NULLS LAST, t2.x NULLS LAST; + +-- check for non-nullable columns for which `is null` is replaced with constant SELECT * FROM t1n as t1 JOIN t2n as t2 ON (t1.x == t2.x AND ((t2.x IS NOT NULL) AND (t1.x IS NOT NULL)) ) OR ( (t2.x IS NULL) AND (t1.x IS NULL) ) ORDER BY t1.x NULLS LAST; -- { echoOff } +SELECT '--'; + +-- IS NOT NULL and constants are optimized out +SELECT count() FROM ( EXPLAIN QUERY TREE + SELECT * FROM t1 JOIN t2 ON ( (t1.x = t2.x) AND (t1.x IS NOT NULL) AND true AND (t2.x IS NOT NULL) ) +) WHERE explain like '%CONSTANT%' OR explain ilike '%is%null%'; + +SELECT count() FROM ( EXPLAIN QUERY TREE + SELECT * FROM t1 JOIN t2 ON ( (t1.x = t2.x) AND true ) +) WHERE explain like '%CONSTANT%' OR explain ilike '%is%null%'; + +-- this is not optimized out +SELECT count() FROM ( EXPLAIN QUERY TREE + SELECT * FROM t1 JOIN t2 ON (t1.x <=> t2.x OR t1.x IS NULL AND t1.y <=> t2.y AND t2.x IS NULL) +) WHERE explain like '%CONSTANT%' OR explain ilike '%is%null%'; + +SELECT count() FROM ( EXPLAIN QUERY TREE + SELECT * FROM t1 JOIN t2 ON t1.x <=> t2.x AND (t1.x = t1.y OR t1.x IS NULL AND t1.y IS NULL) +) WHERE explain like '%CONSTANT%' OR explain ilike '%is%null%'; + DROP TABLE IF EXISTS t1; DROP TABLE IF EXISTS t2; DROP TABLE IF EXISTS t1n; From 97b5c78b8bad243f69c2e27ff056873758314d99 Mon Sep 17 00:00:00 2001 From: gabrielmcg44 Date: Wed, 3 Jul 2024 10:25:37 -0300 Subject: [PATCH 148/417] clearer comment --- src/Functions/array/arrayAUC.cpp | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/src/Functions/array/arrayAUC.cpp b/src/Functions/array/arrayAUC.cpp index 04a840e8da5..3e2a3bf6863 100644 --- a/src/Functions/array/arrayAUC.cpp +++ b/src/Functions/array/arrayAUC.cpp @@ -131,7 +131,7 @@ private: area += (curr_fp - prev_fp) * (curr_tp + prev_tp) / 2.0; - /// Then divide the area to the area of rectangle. + /// Then normalize it dividing by the area to the area of rectangle. if (curr_tp == 0 || curr_tp == size) return std::numeric_limits::quiet_NaN(); From ea3b0e735de285db89cb36e2782db88c6d403ee2 Mon Sep 17 00:00:00 2001 From: avogar Date: Wed, 3 Jul 2024 13:40:41 +0000 Subject: [PATCH 149/417] Refactor JSONExtract functions and support more types and reuse its code in new JSON type --- docs/en/sql-reference/data-types/dynamic.md | 34 + src/Common/JSONParsers/SimdJSONParser.h | 1 + src/DataTypes/DataTypeDynamic.cpp | 1 + src/Formats/JSONExtractTree.cpp | 1561 +++++++++ src/Formats/JSONExtractTree.h | 35 + src/Formats/SchemaInferenceUtils.cpp | 94 +- src/Formats/SchemaInferenceUtils.h | 10 + src/Functions/FunctionsJSON.cpp | 1061 +++++- src/Functions/FunctionsJSON.h | 3054 +++++++---------- .../03198_json_extract_more_types.reference | 21 + .../03198_json_extract_more_types.sql | 29 + .../03199_json_extract_dynamic.reference | 30 + .../03199_json_extract_dynamic.sql | 37 + 13 files changed, 4158 insertions(+), 1810 deletions(-) create mode 100644 src/Formats/JSONExtractTree.cpp create mode 100644 src/Formats/JSONExtractTree.h create mode 100644 tests/queries/0_stateless/03198_json_extract_more_types.reference create mode 100644 tests/queries/0_stateless/03198_json_extract_more_types.sql create mode 100644 tests/queries/0_stateless/03199_json_extract_dynamic.reference create mode 100644 tests/queries/0_stateless/03199_json_extract_dynamic.sql diff --git a/docs/en/sql-reference/data-types/dynamic.md b/docs/en/sql-reference/data-types/dynamic.md index 955fd54e641..e063bed2de4 100644 --- a/docs/en/sql-reference/data-types/dynamic.md +++ b/docs/en/sql-reference/data-types/dynamic.md @@ -493,3 +493,37 @@ SELECT count(), dynamicType(d), _part FROM test GROUP BY _part, dynamicType(d) O ``` As we can see, ClickHouse kept the most frequent types `UInt64` and `Array(UInt64)` and casted all other types to `String`. + + +## JSONExtract functions with Dynamic + +All `JSONExtract*` functions support `Dynamic` type: + +```sql +SELECT JSONExtract('{"a" : [1, 2, 3]}', 'a', 'Dynamic') AS dynamic, dynamicType(dynamic) AS dynamic_type; +``` + +```text +┌─dynamic─┬─dynamic_type───────────┐ +│ [1,2,3] │ Array(Nullable(Int64)) │ +└─────────┴────────────────────────┘ +``` + +```sql +SELECT JSONExtract('{"obj" : {"a" : 42, "b" : "Hello", "c" : [1,2,3]}}', 'obj', 'Map(String, Variant(UInt32, String, Array(UInt32)))') AS map_of_dynamics, mapApply((k, v) -> (k, variantType(v)), map_of_dynamics) AS map_of_dynamic_types``` + +```text +┌─map_of_dynamics──────────────────┬─map_of_dynamic_types────────────────────────────┐ +│ {'a':42,'b':'Hello','c':[1,2,3]} │ {'a':'UInt32','b':'String','c':'Array(UInt32)'} │ +└──────────────────────────────────┴─────────────────────────────────────────────────┘ +``` + +```sql +SELECT JSONExtractKeysAndValues('{"a" : 42, "b" : "Hello", "c" : [1,2,3]}', 'Variant(UInt32, String, Array(UInt32))') AS dynamics, arrayMap(x -> (x.1, variantType(x.2)), dynamics) AS dynamic_types``` +``` + +```text +┌─dynamics───────────────────────────────┬─dynamic_types─────────────────────────────────────────┐ +│ [('a',42),('b','Hello'),('c',[1,2,3])] │ [('a','UInt32'),('b','String'),('c','Array(UInt32)')] │ +└────────────────────────────────────────┴───────────────────────────────────────────────────────┘ +``` diff --git a/src/Common/JSONParsers/SimdJSONParser.h b/src/Common/JSONParsers/SimdJSONParser.h index 827d142266a..db679b14f52 100644 --- a/src/Common/JSONParsers/SimdJSONParser.h +++ b/src/Common/JSONParsers/SimdJSONParser.h @@ -14,6 +14,7 @@ namespace DB { + namespace ErrorCodes { extern const int CANNOT_ALLOCATE_MEMORY; diff --git a/src/DataTypes/DataTypeDynamic.cpp b/src/DataTypes/DataTypeDynamic.cpp index c920e69c13b..6826c46a1a7 100644 --- a/src/DataTypes/DataTypeDynamic.cpp +++ b/src/DataTypes/DataTypeDynamic.cpp @@ -12,6 +12,7 @@ #include #include #include +#include namespace DB { diff --git a/src/Formats/JSONExtractTree.cpp b/src/Formats/JSONExtractTree.cpp new file mode 100644 index 00000000000..6d019f96ba6 --- /dev/null +++ b/src/Formats/JSONExtractTree.cpp @@ -0,0 +1,1561 @@ +#include +#include + +#include +#if USE_SIMDJSON +#include +#endif +#if USE_RAPIDJSON +#include +#endif + +#include + +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include + +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include + + +#include +#include +#include +#include + +namespace DB +{ + +template +void jsonElementToString(const typename JSONParser::Element & element, WriteBuffer & buf, const FormatSettings & format_settings) +{ + if (element.isInt64()) + { + writeIntText(element.getInt64(), buf); + return; + } + if (element.isUInt64()) + { + writeIntText(element.getUInt64(), buf); + return; + } + if (element.isDouble()) + { + writeFloatText(element.getDouble(), buf); + return; + } + if (element.isBool()) + { + if (element.getBool()) + writeCString("true", buf); + else + writeCString("false", buf); + return; + } + if (element.isString()) + { + writeJSONString(element.getString(), buf, format_settings); + return; + } + if (element.isArray()) + { + writeChar('[', buf); + bool need_comma = false; + for (auto value : element.getArray()) + { + if (std::exchange(need_comma, true)) + writeChar(',', buf); + jsonElementToString(value, buf, format_settings); + } + writeChar(']', buf); + return; + } + if (element.isObject()) + { + writeChar('{', buf); + bool need_comma = false; + for (auto [key, value] : element.getObject()) + { + if (std::exchange(need_comma, true)) + writeChar(',', buf); + writeJSONString(key, buf, format_settings); + writeChar(':', buf); + jsonElementToString(value, buf, format_settings); + } + writeChar('}', buf); + return; + } + if (element.isNull()) + { + writeCString("null", buf); + return; + } +} + +template +bool tryGetNumericValueFromJSONElement( + NumberType & value, + const typename JSONParser::Element & element, + bool convert_bool_to_integer, + String & error) +{ + switch (element.type()) + { + case ElementType::DOUBLE: + if constexpr (std::is_floating_point_v) + { + /// We permit inaccurate conversion of double to float. + /// Example: double 0.1 from JSON is not representable in float. + /// But it will be more convenient for user to perform conversion. + value = static_cast(element.getDouble()); + } + else if (!accurate::convertNumeric(element.getDouble(), value)) + { + error = fmt::format("cannot convert double value {} to {}", element.getDouble(), TypeName); + return false; + } + break; + case ElementType::UINT64: + if (!accurate::convertNumeric(element.getUInt64(), value)) + { + error = fmt::format("cannot convert UInt64 value {} to {}", element.getUInt64(), TypeName); + return false; + } + break; + case ElementType::INT64: + if (!accurate::convertNumeric(element.getInt64(), value)) + { + error = fmt::format("cannot convert Int64 value {} to {}", element.getInt64(), TypeName); + return false; + } + break; + case ElementType::BOOL: + if constexpr (is_integer) + { + if (convert_bool_to_integer) + { + value = static_cast(element.getBool()); + break; + } + } + error = fmt::format("cannot convert bool value to {}", TypeName); + return false; + case ElementType::STRING: { + auto rb = ReadBufferFromMemory{element.getString()}; + if constexpr (std::is_floating_point_v) + { + if (!tryReadFloatText(value, rb) || !rb.eof()) + { + error = fmt::format("cannot parse {} value here: {}", TypeName, element.getString()); + return false; + } + } + else + { + if (tryReadIntText(value, rb) && rb.eof()) + break; + + /// Try to parse float and convert it to integer. + Float64 tmp_float; + rb.position() = rb.buffer().begin(); + if (!tryReadFloatText(tmp_float, rb) || !rb.eof()) + { + error = fmt::format("cannot parse {} value here: {}", TypeName, element.getString()); + return false; + } + + if (!accurate::convertNumeric(tmp_float, value)) + { + error = fmt::format("cannot parse {} value here: {}", TypeName, element.getString()); + return false; + } + } + break; + } + default: + return false; + } + + return true; +} + +namespace +{ + +template +String jsonElementToString(const typename JSONParser::Element & element, const FormatSettings & format_settings) +{ + WriteBufferFromOwnString buf; + jsonElementToString(element, buf, format_settings); + return buf.str(); +} + +template +class NumericNode : public JSONExtractTreeNode +{ +public: + explicit NumericNode(bool is_bool_type_ = false) : is_bool_type(is_bool_type_) { } + + bool insertResultToColumn( + IColumn & column, const typename JSONParser::Element & element, const JSONExtractInsertSettings & insert_settings, const FormatSettings & format_settings, String & error) const override + { + if (element.isNull()) + { + if (format_settings.null_as_default) + { + column.insertDefault(); + return true; + } + + error = fmt::format("cannot parse {} value from null", TypeName); + return false; + } + + NumberType value; + if (!tryGetNumericValueFromJSONElement(value, element, insert_settings.convert_bool_to_integer || is_bool_type, error)) + { + if (error.empty()) + error = fmt::format("cannot read {} value from JSON element: {}", TypeName, jsonElementToString(element, format_settings)); + return false; + } + + if (is_bool_type) + value = static_cast(value); + + auto & col_vec = assert_cast &>(column); + col_vec.insertValue(value); + return true; + } + +protected: + bool is_bool_type; +}; + +template +class LowCardinalityNumericNode : public NumericNode +{ +public: + explicit LowCardinalityNumericNode(bool is_nullable_, bool is_bool_type_ = false) + : NumericNode(is_bool_type_), is_nullable(is_nullable_) + { + } + + bool insertResultToColumn( + IColumn & column, const typename JSONParser::Element & element, const JSONExtractInsertSettings & insert_settings, const FormatSettings & format_settings, String & error) const override + { + if (element.isNull()) + { + if (is_nullable || format_settings.null_as_default) + { + column.insertDefault(); + return true; + } + + error = fmt::format("cannot parse {} value from null", TypeName); + return false; + } + + NumberType value; + if (!tryGetNumericValueFromJSONElement(value, element, insert_settings.convert_bool_to_integer || this->is_bool_type, error)) + { + if (error.empty()) + error = fmt::format("cannot read {} value from JSON element: {}", TypeName, jsonElementToString(element, format_settings)); + return false; + } + + if (this->is_bool_type) + value = static_cast(value); + + auto & col_lc = assert_cast(column); + col_lc.insertData(reinterpret_cast(&value), sizeof(value)); + return true; + } + +private: + bool is_nullable; +}; + +template +class StringNode : public JSONExtractTreeNode +{ +public: + bool insertResultToColumn( + IColumn & column, const typename JSONParser::Element & element, const JSONExtractInsertSettings &, const FormatSettings & format_settings, String & error) const override + { + if (element.isNull()) + { + if (format_settings.null_as_default) + { + column.insertDefault(); + return true; + } + error = "cannot parse String value from null"; + return false; + } + + if (!element.isString()) + { + auto & col_str = assert_cast(column); + auto & chars = col_str.getChars(); + WriteBufferFromVector buf(chars, AppendModeTag()); + jsonElementToString(element, buf, format_settings); + buf.finalize(); + chars.push_back(0); + col_str.getOffsets().push_back(chars.size()); + } + else + { + auto value = element.getString(); + auto & col_str = assert_cast(column); + col_str.insertData(value.data(), value.size()); + } + return true; + } +}; + +template +class LowCardinalityStringNode : public JSONExtractTreeNode +{ +public: + explicit LowCardinalityStringNode(bool is_nullable_) : is_nullable(is_nullable_) { } + + bool insertResultToColumn( + IColumn & column, const typename JSONParser::Element & element, const JSONExtractInsertSettings &, const FormatSettings & format_settings, String & error) const override + { + if (element.isNull()) + { + if (is_nullable || format_settings.null_as_default) + { + column.insertDefault(); + return true; + } + + error = "cannot parse String value from null"; + return false; + } + + if (!element.isString()) + { + auto value = jsonElementToString(element, format_settings); + assert_cast(column).insertData(value.data(), value.size()); + } + else + { + auto value = element.getString(); + assert_cast(column).insertData(value.data(), value.size()); + } + + return true; + } + +private: + bool is_nullable; +}; + +template +class FixedStringNode : public JSONExtractTreeNode +{ +public: + explicit FixedStringNode(size_t fixed_length_) : fixed_length(fixed_length_) { } + bool insertResultToColumn( + IColumn & column, const typename JSONParser::Element & element, const JSONExtractInsertSettings &, const FormatSettings & format_settings, String & error) const override + { + if (element.isNull()) + { + if (format_settings.null_as_default) + { + column.insertDefault(); + return true; + } + + error = "cannot parse FixedString value from null"; + return false; + } + + if (!element.isString()) + return checkValueSizeAndInsert(column, jsonElementToString(element, format_settings), error); + return checkValueSizeAndInsert(column, element.getString(), error); + } + +private: + template + bool checkValueSizeAndInsert(IColumn & column, const T & value, String & error) const + { + if (value.size() > fixed_length) + { + error = fmt::format("too large string for FixedString({}): {}", fixed_length, value); + return false; + } + assert_cast(column).insertData(value.data(), value.size()); + return true; + } + + size_t fixed_length; +}; + +template +class LowCardinalityFixedStringNode : public JSONExtractTreeNode +{ +public: + explicit LowCardinalityFixedStringNode(bool is_nullable_, size_t fixed_length_) : is_nullable(is_nullable_), fixed_length(fixed_length_) + { + } + + bool insertResultToColumn( + IColumn & column, const typename JSONParser::Element & element, const JSONExtractInsertSettings &, const FormatSettings & format_settings, String & error) const override + { + if (element.isNull()) + { + if (is_nullable || format_settings.null_as_default) + { + column.insertDefault(); + return true; + } + error = "cannot parse FixedString value from null"; + return false; + } + + if (!element.isString()) + return checkValueSizeAndInsert(column, jsonElementToString(element, format_settings), error); + return checkValueSizeAndInsert(column, element.getString(), error); + } + +private: + template + bool checkValueSizeAndInsert(IColumn & column, const T & value, String & error) const + { + if (value.size() > fixed_length) + { + error = fmt::format("too large string for FixedString({}): {}", fixed_length, value); + return false; + } + + // For the non low cardinality case of FixedString, the padding is done in the FixedString Column implementation. + // In order to avoid having to pass the data to a FixedString Column and read it back (which would slow down the execution) + // the data is padded here and written directly to the Low Cardinality Column + if (value.size() == fixed_length) + { + assert_cast(column).insertData(value.data(), value.size()); + } + else + { + String padded_value(value); + padded_value.resize(fixed_length, '\0'); + assert_cast(column).insertData(padded_value.data(), padded_value.size()); + } + return true; + } + + bool is_nullable; + size_t fixed_length; +}; + +template +class UUIDNode : public JSONExtractTreeNode +{ +public: + bool insertResultToColumn( + IColumn & column, const typename JSONParser::Element & element, const JSONExtractInsertSettings &, const FormatSettings & format_settings, String & error) const override + { + if (element.isNull() && format_settings.null_as_default) + { + column.insertDefault(); + return true; + } + + if (!element.isString()) + { + error = fmt::format("cannot read UUID value from JSON element: {}", jsonElementToString(element, format_settings)); + return false; + } + + auto data = element.getString(); + UUID uuid; + if (!tryParse(uuid, data)) + { + error = fmt::format("cannot parse UUID value here: {}", data); + return false; + } + + assert_cast(column).insert(uuid); + return true; + } + + + static bool tryParse(UUID & uuid, std::string_view data) + { + ReadBufferFromMemory buf(data.data(), data.size()); + return tryReadUUIDText(uuid, buf) && buf.eof(); + } +}; + +template +class LowCardinalityUUIDNode : public JSONExtractTreeNode +{ +public: + explicit LowCardinalityUUIDNode(bool is_nullable_) : is_nullable(is_nullable_) { } + + bool insertResultToColumn( + IColumn & column, const typename JSONParser::Element & element, const JSONExtractInsertSettings &, const FormatSettings & format_settings, String & error) const override + { + if (element.isNull() && (is_nullable || format_settings.null_as_default)) + { + column.insertDefault(); + return true; + } + + if (!element.isString()) + { + error = fmt::format("cannot read UUID value from JSON element: {}", jsonElementToString(element, format_settings)); + return false; + } + + auto data = element.getString(); + ReadBufferFromMemory buf(data.data(), data.size()); + UUID uuid; + if (!tryReadUUIDText(uuid, buf) || !buf.eof()) + { + error = fmt::format("cannot parse UUID value here: {}", data); + return false; + } + assert_cast(column).insertData(reinterpret_cast(&uuid), sizeof(uuid)); + return true; + } + +private: + bool is_nullable; +}; + +template +class DateNode : public JSONExtractTreeNode +{ +public: + bool insertResultToColumn( + IColumn & column, const typename JSONParser::Element & element, const JSONExtractInsertSettings &, const FormatSettings & format_settings, String & error) const override + { + if (element.isNull() && format_settings.null_as_default) + { + column.insertDefault(); + return true; + } + + if (!element.isString()) + { + error = fmt::format("cannot read Date value from JSON element: {}", jsonElementToString(element, format_settings)); + return false; + } + + auto data = element.getString(); + ReadBufferFromMemory buf(data.data(), data.size()); + DateType date; + if (!tryReadDateText(date, buf) || !buf.eof()) + { + error = fmt::format("cannot parse Date value here: {}", data); + return false; + } + + assert_cast &>(column).insertValue(date); + return true; + } +}; + +template +class DateTimeNode : public JSONExtractTreeNode, public TimezoneMixin +{ +public: + explicit DateTimeNode(const DataTypeDateTime & datetime_type) : TimezoneMixin(datetime_type) { } + + bool insertResultToColumn( + IColumn & column, const typename JSONParser::Element & element, const JSONExtractInsertSettings &, const FormatSettings & format_settings, String & error) const override + { + if (element.isNull() && format_settings.null_as_default) + { + column.insertDefault(); + return true; + } + + time_t value; + if (element.isString()) + { + if (!tryParse(value, element.getString(), format_settings.date_time_input_format)) + { + error = fmt::format("cannot parse DateTime value here: {}", element.getString()); + return false; + } + } + else if (element.isUInt64()) + { + value = element.getUInt64(); + } + else + { + error = fmt::format("cannot read DateTime value from JSON element: {}", jsonElementToString(element, format_settings)); + return false; + } + + assert_cast(column).insert(value); + return true; + } + + bool tryParse(time_t & value, std::string_view data, FormatSettings::DateTimeInputFormat date_time_input_format) const + { + ReadBufferFromMemory buf(data.data(), data.size()); + switch (date_time_input_format) + { + case FormatSettings::DateTimeInputFormat::Basic: + if (tryReadDateTimeText(value, buf, time_zone) && buf.eof()) + return true; + break; + case FormatSettings::DateTimeInputFormat::BestEffort: + if (tryParseDateTimeBestEffort(value, buf, time_zone, utc_time_zone) && buf.eof()) + return true; + break; + case FormatSettings::DateTimeInputFormat::BestEffortUS: + if (tryParseDateTimeBestEffortUS(value, buf, time_zone, utc_time_zone) && buf.eof()) + return true; + break; + } + + return false; + } +}; + +template +class DecimalNode : public JSONExtractTreeNode +{ +public: + explicit DecimalNode(const DataTypePtr & type) : scale(assert_cast &>(*type).getScale()) { } + + bool insertResultToColumn( + IColumn & column, const typename JSONParser::Element & element, const JSONExtractInsertSettings &, const FormatSettings & format_settings, String & error) const override + { + DecimalType value{}; + + switch (element.type()) + { + case ElementType::DOUBLE: + value = convertToDecimal, DataTypeDecimal>(element.getDouble(), scale); + break; + case ElementType::UINT64: + value = convertToDecimal, DataTypeDecimal>(element.getUInt64(), scale); + break; + case ElementType::INT64: + value = convertToDecimal, DataTypeDecimal>(element.getInt64(), scale); + break; + case ElementType::STRING: { + auto rb = ReadBufferFromMemory{element.getString()}; + if (!SerializationDecimal::tryReadText(value, rb, DecimalUtils::max_precision, scale)) + { + error = fmt::format("cannot parse Decimal value here: {}", element.getString()); + return false; + } + break; + } + case ElementType::NULL_VALUE: { + if (!format_settings.null_as_default) + { + error = "cannot convert null to Decimal value"; + return false; + } + break; + } + default: { + error = fmt::format("cannot read Decimal value from JSON element: {}", jsonElementToString(element, format_settings)); + return false; + } + } + + assert_cast &>(column).insertValue(value); + return true; + } + +private: + UInt32 scale; +}; + + +template +class DateTime64Node : public JSONExtractTreeNode, public TimezoneMixin +{ +public: + explicit DateTime64Node(const DataTypeDateTime64 & datetime64_type) : TimezoneMixin(datetime64_type), scale(datetime64_type.getScale()) { } + + bool insertResultToColumn( + IColumn & column, const typename JSONParser::Element & element, const JSONExtractInsertSettings &, const FormatSettings & format_settings, String & error) const override + { + if (element.isNull() && format_settings.null_as_default) + { + column.insertDefault(); + return true; + } + + DateTime64 value; + if (element.isString()) + { + if (!tryParse(value, element.getString(), format_settings.date_time_input_format)) + { + error = fmt::format("cannot parse DateTime64 value here: {}", element.getString()); + return false; + } + } + else + { + switch (element.type()) + { + case ElementType::DOUBLE: + value = convertToDecimal, DataTypeDecimal>(element.getDouble(), scale); + break; + case ElementType::UINT64: + value = convertToDecimal, DataTypeDecimal>(element.getUInt64(), scale); + break; + case ElementType::INT64: + value = convertToDecimal, DataTypeDecimal>(element.getInt64(), scale); + break; + default: + error = fmt::format("cannot read DateTime64 value from JSON element: {}", jsonElementToString(element, format_settings)); + return false; + } + } + + assert_cast(column).insert(value); + return true; + } + + bool tryParse(DateTime64 & value, std::string_view data, FormatSettings::DateTimeInputFormat date_time_input_format) const + { + ReadBufferFromMemory buf(data.data(), data.size()); + switch (date_time_input_format) + { + case FormatSettings::DateTimeInputFormat::Basic: + if (tryReadDateTime64Text(value, scale, buf, time_zone) && buf.eof()) + return true; + break; + case FormatSettings::DateTimeInputFormat::BestEffort: + if (tryParseDateTime64BestEffort(value, scale, buf, time_zone, utc_time_zone) && buf.eof()) + return true; + break; + case FormatSettings::DateTimeInputFormat::BestEffortUS: + if (tryParseDateTime64BestEffortUS(value, scale, buf, time_zone, utc_time_zone) && buf.eof()) + return true; + break; + } + + return false; + } + +private: + UInt32 scale; +}; + +template +class EnumNode : public JSONExtractTreeNode +{ +public: + explicit EnumNode(const std::vector> & name_value_pairs_) : name_value_pairs(name_value_pairs_) + { + for (const auto & name_value_pair : name_value_pairs) + { + name_to_value_map.emplace(name_value_pair.first, name_value_pair.second); + only_values.emplace(name_value_pair.second); + } + } + + bool insertResultToColumn( + IColumn & column, const typename JSONParser::Element & element, const JSONExtractInsertSettings &, const FormatSettings & format_settings, String & error) const override + { + if (element.isNull()) + { + if (format_settings.null_as_default) + { + column.insertDefault(); + return true; + } + + error = "cannot convert null to Enum value"; + return false; + } + + auto & col_vec = assert_cast &>(column); + + if (element.isInt64()) + { + Type value; + if (!accurate::convertNumeric(element.getInt64(), value) || !only_values.contains(value)) + { + error = fmt::format("cannot convert value {} to enum: there is no such value in enum", element.getInt64()); + return false; + } + col_vec.insertValue(value); + return true; + } + + if (element.isUInt64()) + { + Type value; + if (!accurate::convertNumeric(element.getUInt64(), value) || !only_values.contains(value)) + { + error = fmt::format("cannot convert value {} to enum: there is no such value in enum", element.getUInt64()); + return false; + } + col_vec.insertValue(value); + return true; + } + + if (element.isString()) + { + auto value = name_to_value_map.find(element.getString()); + if (value == name_to_value_map.end()) + { + error = fmt::format("cannot convert value {} to enum: there is no such value in enum", element.getString()); + return false; + } + col_vec.insertValue(value->second); + return true; + } + + error = fmt::format("cannot read Enum value from JSON element: {}", jsonElementToString(element, format_settings)); + return false; + } + +private: + std::vector> name_value_pairs; + std::unordered_map name_to_value_map; + std::unordered_set only_values; +}; + +template +class IPv4Node : public JSONExtractTreeNode +{ +public: + bool insertResultToColumn( + IColumn & column, const typename JSONParser::Element & element, const JSONExtractInsertSettings &, const FormatSettings & format_settings, String & error) const override + { + if (element.isNull() && format_settings.null_as_default) + { + column.insertDefault(); + return true; + } + + if (!element.isString()) + { + error = fmt::format("cannot read IPv4 value from JSON element: {}", jsonElementToString(element, format_settings)); + return false; + } + + auto data = element.getString(); + IPv4 value; + if (!tryParse(value, data)) + { + error = fmt::format("cannot parse IPv4 value here: {}", data); + return false; + } + + assert_cast(column).insert(value); + return true; + } + + static bool tryParse(IPv4 & value, std::string_view data) + { + ReadBufferFromMemory buf(data.data(), data.size()); + return tryReadIPv4Text(value, buf) && buf.eof(); + } +}; + +template +class IPv6Node : public JSONExtractTreeNode +{ +public: + bool insertResultToColumn( + IColumn & column, const typename JSONParser::Element & element, const JSONExtractInsertSettings &, const FormatSettings & format_settings, String & error) const override + { + if (element.isNull() && format_settings.null_as_default) + { + column.insertDefault(); + return true; + } + + if (!element.isString()) + { + error = fmt::format("cannot read IPv6 value from JSON element: {}", jsonElementToString(element, format_settings)); + return false; + } + + auto data = element.getString(); + IPv6 value; + if (!tryParse(value, data)) + { + error = fmt::format("cannot parse IPv6 value here: {}", data); + return false; + } + + assert_cast(column).insert(value); + return true; + } + + + static bool tryParse(IPv6 & value, std::string_view data) + { + ReadBufferFromMemory buf(data.data(), data.size()); + return tryReadIPv6Text(value, buf) && buf.eof(); + } +}; + +template +class NullableNode : public JSONExtractTreeNode +{ +public: + explicit NullableNode(std::unique_ptr> nested_) : nested(std::move(nested_)) { } + + bool insertResultToColumn( + IColumn & column, const typename JSONParser::Element & element, const JSONExtractInsertSettings & insert_settings, const FormatSettings & format_settings, String & error) const override + { + if (element.isNull()) + { + column.insertDefault(); + return true; + } + + auto & col_null = assert_cast(column); + if (!nested-> insertResultToColumn(col_null.getNestedColumn(), element, insert_settings, format_settings, error)) + return false; + col_null.getNullMapColumn().insertValue(0); + return true; + } + +private: + std::unique_ptr> nested; +}; + +template +class LowCardinalityNode : public JSONExtractTreeNode +{ +public: + explicit LowCardinalityNode(bool is_nullable_, std::unique_ptr> nested_) + : is_nullable(is_nullable_), nested(std::move(nested_)) + { + } + + bool insertResultToColumn( + IColumn & column, const typename JSONParser::Element & element, const JSONExtractInsertSettings & insert_settings, const FormatSettings & format_settings, String & error) const override + { + if (element.isNull() && (is_nullable || format_settings.null_as_default)) + { + column.insertDefault(); + return true; + } + + auto & col_lc = assert_cast(column); + auto tmp_nested = col_lc.getDictionary().getNestedColumn()->cloneEmpty(); + if (!nested-> insertResultToColumn(*tmp_nested, element, insert_settings, format_settings, error)) + return false; + + col_lc.insertFromFullColumn(*tmp_nested, 0); + return true; + } + +private: + bool is_nullable; + std::unique_ptr> nested; +}; + +template +class ArrayNode : public JSONExtractTreeNode +{ +public: + explicit ArrayNode(std::unique_ptr> nested_) : nested(std::move(nested_)) { } + + bool insertResultToColumn( + IColumn & column, const typename JSONParser::Element & element, const JSONExtractInsertSettings & insert_settings, const FormatSettings & format_settings, String & error) const override + { + if (element.isNull() && format_settings.null_as_default) + { + column.insertDefault(); + return true; + } + + if (!element.isArray()) + { + error = fmt::format("cannot read Array value from JSON element: {}", jsonElementToString(element, format_settings)); + return false; + } + + auto array = element.getArray(); + + auto & col_arr = assert_cast(column); + auto & data = col_arr.getData(); + size_t old_size = data.size(); + bool were_valid_elements = false; + + for (auto value : array) + { + if (nested-> insertResultToColumn(data, value, insert_settings, format_settings, error)) + { + were_valid_elements = true; + } + else if (insert_settings.insert_default_on_invalid_elements_in_complex_types) + { + data.insertDefault(); + } + else + { + data.popBack(data.size() - old_size); + return false; + } + } + + if (!were_valid_elements) + { + data.popBack(data.size() - old_size); + return false; + } + + col_arr.getOffsets().push_back(data.size()); + return true; + } + +private: + std::unique_ptr> nested; +}; + +template +class TupleNode : public JSONExtractTreeNode +{ +public: + TupleNode(std::vector>> nested_, const std::vector & explicit_names_) + : nested(std::move(nested_)), explicit_names(explicit_names_) + { + for (size_t i = 0; i != explicit_names.size(); ++i) + name_to_index_map.emplace(explicit_names[i], i); + } + + bool insertResultToColumn( + IColumn & column, const typename JSONParser::Element & element, const JSONExtractInsertSettings & insert_settings, const FormatSettings & format_settings, String & error) const override + { + auto & tuple = assert_cast(column); + size_t old_size = column.size(); + bool were_valid_elements = false; + + auto set_size = [&](size_t size) + { + for (size_t i = 0; i != tuple.tupleSize(); ++i) + { + auto & col = tuple.getColumn(i); + if (col.size() != size) + { + if (col.size() > size) + col.popBack(col.size() - size); + else + while (col.size() < size) + col.insertDefault(); + } + } + }; + + if (element.isArray()) + { + auto array = element.getArray(); + auto it = array.begin(); + + for (size_t index = 0; (index != nested.size()) && (it != array.end()); ++index) + { + if (nested[index]-> insertResultToColumn(tuple.getColumn(index), *it++, insert_settings, format_settings, error)) + { + were_valid_elements = true; + } + else if (insert_settings.insert_default_on_invalid_elements_in_complex_types) + { + tuple.getColumn(index).insertDefault(); + } + else + { + set_size(old_size); + error += fmt::format("(during reading tuple {} element)", index); + return false; + } + } + + set_size(old_size + static_cast(were_valid_elements)); + return were_valid_elements; + } + + if (element.isObject()) + { + auto object = element.getObject(); + if (name_to_index_map.empty()) + { + auto it = object.begin(); + for (size_t index = 0; (index != nested.size()) && (it != object.end()); ++index) + { + if (nested[index]-> insertResultToColumn(tuple.getColumn(index), (*it++).second, insert_settings, format_settings, error)) + { + were_valid_elements = true; + } + else if (insert_settings.insert_default_on_invalid_elements_in_complex_types) + { + tuple.getColumn(index).insertDefault(); + } + else + { + set_size(old_size); + error += fmt::format("(during reading tuple {} element)", index); + return false; + } + } + } + else + { + for (const auto & [key, value] : object) + { + auto index = name_to_index_map.find(key); + if (index != name_to_index_map.end()) + { + if (nested[index->second]-> insertResultToColumn(tuple.getColumn(index->second), value, insert_settings, format_settings, error)) + { + were_valid_elements = true; + } + else if (!insert_settings.insert_default_on_invalid_elements_in_complex_types) + { + set_size(old_size); + error += fmt::format("(during reading tuple element \"{}\")", key); + return false; + } + } + } + } + + set_size(old_size + static_cast(were_valid_elements)); + return were_valid_elements; + } + + error = fmt::format("cannot read Tuple value from JSON element: {}", jsonElementToString(element, format_settings)); + return false; + } + +private: + std::vector>> nested; + std::vector explicit_names; + std::unordered_map name_to_index_map; +}; + +template +class MapNode : public JSONExtractTreeNode +{ +public: + explicit MapNode(std::unique_ptr> value_) : value(std::move(value_)) { } + + bool insertResultToColumn( + IColumn & column, const typename JSONParser::Element & element, const JSONExtractInsertSettings & insert_settings, const FormatSettings & format_settings, String & error) const override + { + if (!element.isObject()) + { + error = fmt::format("cannot read Map value from JSON element: {}", jsonElementToString(element, format_settings)); + return false; + } + + auto & map_col = assert_cast(column); + auto & offsets = map_col.getNestedColumn().getOffsets(); + auto & tuple_col = map_col.getNestedData(); + auto & key_col = tuple_col.getColumn(0); + auto & value_col = tuple_col.getColumn(1); + size_t old_size = tuple_col.size(); + + auto object = element.getObject(); + auto it = object.begin(); + for (; it != object.end(); ++it) + { + auto pair = *it; + + /// Insert key + key_col.insertData(pair.first.data(), pair.first.size()); + + /// Insert value + if (!value-> insertResultToColumn(value_col, pair.second, insert_settings, format_settings, error)) + { + if (insert_settings.insert_default_on_invalid_elements_in_complex_types) + { + value_col.insertDefault(); + } + else + { + key_col.popBack(key_col.size() - offsets.back()); + value_col.popBack(value_col.size() - offsets.back()); + error += fmt::format("(during reading value of key \"{}\")", pair.first); + return false; + } + } + } + + offsets.push_back(old_size + object.size()); + return true; + } + +private: + std::unique_ptr> value; +}; + +template +class VariantNode : public JSONExtractTreeNode +{ +public: + VariantNode(std::vector>> variant_nodes_, std::vector order_) + : variant_nodes(std::move(variant_nodes_)), order(std::move(order_)) + { + } + + bool insertResultToColumn( + IColumn & column, const typename JSONParser::Element & element, const JSONExtractInsertSettings & insert_settings, const FormatSettings & format_settings, String & error) const override + { + auto & column_variant = assert_cast(column); + for (size_t i : order) + { + auto & variant = column_variant.getVariantByGlobalDiscriminator(i); + if (variant_nodes[i]-> insertResultToColumn(variant, element, insert_settings, format_settings, error)) + { + column_variant.getLocalDiscriminators().push_back(column_variant.localDiscriminatorByGlobal(i)); + column_variant.getOffsets().push_back(variant.size() - 1); + return true; + } + } + + error = fmt::format("cannot read Map value from JSON element: {}", jsonElementToString(element, format_settings)); + return false; + } + +private: + std::vector>> variant_nodes; + /// Order in which we should try variants nodes. + /// For example, String should be always the last one. + std::vector order; +}; + + +template +class DynamicNode : public JSONExtractTreeNode +{ +public: + bool insertResultToColumn(IColumn & column, const typename JSONParser::Element & element, const JSONExtractInsertSettings & insert_settings, const FormatSettings & format_settings, String & error) const override + { + auto & column_dynamic = assert_cast(column); + auto & variant_column = column_dynamic.getVariantColumn(); + auto variant_info = column_dynamic.getVariantInfo(); + /// First, infer ClickHouse type for this element and add it as a new variant. + auto element_type = elementToDataType(element, format_settings); + if (column_dynamic.addNewVariant(element_type)) + { + auto node = buildJSONExtractTree(element_type, "Dynamic inference"); + auto global_discriminator = variant_info.variant_name_to_discriminator[element_type->getName()]; + auto & variant = variant_column.getVariantByGlobalDiscriminator(global_discriminator); + if (!node-> insertResultToColumn(variant, element, insert_settings, format_settings, error)) + return false; + variant_column.getLocalDiscriminators().push_back(variant_column.localDiscriminatorByGlobal(global_discriminator)); + variant_column.getOffsets().push_back(variant.size() - 1); + return true; + } + + /// We couldn't add new variant. Try to insert element into current variants. + auto variant_node = buildJSONExtractTree(variant_info.variant_type, "Dynamic inference"); + if (variant_node-> insertResultToColumn(variant_column, element, insert_settings, format_settings, error)) + return true; + + /// We couldn't insert element into any existing variant, add String variant and read value as String. + column_dynamic.addStringVariant(); + auto string_global_discriminator = variant_info.variant_name_to_discriminator["String"]; + auto & string_column = variant_column.getVariantByGlobalDiscriminator(string_global_discriminator); + if (!getStringNode()-> insertResultToColumn(string_column, element, insert_settings, format_settings, error)) + return false; + variant_column.getLocalDiscriminators().push_back(variant_column.localDiscriminatorByGlobal(string_global_discriminator)); + variant_column.getOffsets().push_back(string_column.size() - 1); + return true; + } + + static const std::unique_ptr> & getStringNode() + { + static const std::unique_ptr> string_node + = buildJSONExtractTree(std::make_shared(), "Dynamic inference"); + return string_node; + } + + static DataTypePtr elementToDataType(const typename JSONParser::Element & element, const FormatSettings & format_settings) + { + JSONInferenceInfo json_inference_info; + auto type = elementToDataTypeImpl(element, format_settings, json_inference_info); + transformFinalInferredJSONTypeIfNeeded(type, format_settings, &json_inference_info); + return type; + } + +private: + static DataTypePtr elementToDataTypeImpl(const typename JSONParser::Element & element, const FormatSettings & format_settings, JSONInferenceInfo & json_inference_info) + { + switch (element.type()) + { + case ElementType::NULL_VALUE: + return makeNullable(std::make_shared()); + case ElementType::BOOL: + return DataTypeFactory::instance().get("Bool"); + case ElementType::INT64: + { + auto type = std::make_shared(); + if (element.getInt64() < 0) + json_inference_info.negative_integers.insert(type.get()); + return type; + } + case ElementType::UINT64: + return std::make_shared(); + case ElementType::DOUBLE: + return std::make_shared(); + case ElementType::STRING: + { + auto data = element.getString(); + + if (auto type = tryInferDateOrDateTimeFromString(data, format_settings)) + return type; + + if (format_settings.json.try_infer_numbers_from_strings) + { + bool is_negative = false; + if (auto type = tryInferJSONNumberFromString(data, format_settings, &json_inference_info)) + { + json_inference_info.numbers_parsed_from_json_strings.insert(type.get()); + if (is_negative) + json_inference_info.negative_integers.insert(type.get()); + return type; + } + } + + return std::make_shared(); + } + case ElementType::ARRAY: + { + auto array = element.getArray(); + DataTypes types; + types.reserve(array.size()); + for (auto value : array) + types.push_back(makeNullableSafe(elementToDataTypeImpl(value, format_settings, json_inference_info))); + + if (types.empty()) + return std::make_shared(makeNullable(std::make_shared())); + + if (checkIfTypesAreEqual(types)) + return std::make_shared(types.back()); + + /// For JSON if we have not complete types, we should not try to transform them + /// and return it as a Tuple. + /// For example, if we have types [Nullable(Float64), Nullable(Nothing), Nullable(Float64)] + /// it can be Array(Nullable(Float64)) or Tuple(Nullable(Float64), , Nullable(Float64)) and + /// we can't determine which one it is right now. But we will be able to do it later + /// when we will have the final top level type. + /// For example, we can have JSON element [[42.42, null, 43.43], [44.44, "Some string", 45.45]] and we should + /// determine the type for this element as Tuple(Nullable(Float64), Nullable(String), Nullable(Float64)). + for (const auto & type : types) + { + if (!checkIfTypeIsComplete(type)) + return std::make_shared(types); + } + + auto types_copy = types; + transformInferredJSONTypesIfNeeded(types_copy, format_settings, &json_inference_info); + + if (checkIfTypesAreEqual(types_copy)) + return std::make_shared(types_copy.back()); + + return std::make_shared(types); + } + case ElementType::OBJECT: { + /// TODO: Use new JSON type here when it's ready. + return std::make_shared(std::make_shared(), makeNullable(std::make_shared())); + } + } + } +}; + +} + +template +std::unique_ptr> buildJSONExtractTree(const DataTypePtr & type, const char * source_for_exception_message) +{ + switch (type->getTypeId()) + { + case TypeIndex::UInt8: + return std::make_unique>(isBool(type)); + case TypeIndex::UInt16: + return std::make_unique>(); + case TypeIndex::UInt32: + return std::make_unique>(); + case TypeIndex::UInt64: + return std::make_unique>(); + case TypeIndex::UInt128: + return std::make_unique>(); + case TypeIndex::UInt256: + return std::make_unique>(); + case TypeIndex::Int8: + return std::make_unique>(); + case TypeIndex::Int16: + return std::make_unique>(); + case TypeIndex::Int32: + return std::make_unique>(); + case TypeIndex::Int64: + return std::make_unique>(); + case TypeIndex::Int128: + return std::make_unique>(); + case TypeIndex::Int256: + return std::make_unique>(); + case TypeIndex::Float32: + return std::make_unique>(); + case TypeIndex::Float64: + return std::make_unique>(); + case TypeIndex::String: + return std::make_unique>(); + case TypeIndex::FixedString: + return std::make_unique>(assert_cast(*type).getN()); + case TypeIndex::UUID: + return std::make_unique>(); + case TypeIndex::IPv4: + return std::make_unique>(); + case TypeIndex::IPv6: + return std::make_unique>(); + case TypeIndex::Date:; + return std::make_unique>(); + case TypeIndex::Date32: + return std::make_unique>(); + case TypeIndex::DateTime: + return std::make_unique>(assert_cast(*type)); + case TypeIndex::DateTime64: + return std::make_unique>(assert_cast(*type)); + case TypeIndex::Decimal32: + return std::make_unique>(type); + case TypeIndex::Decimal64: + return std::make_unique>(type); + case TypeIndex::Decimal128: + return std::make_unique>(type); + case TypeIndex::Decimal256: + return std::make_unique>(type); + case TypeIndex::Enum8: + return std::make_unique>(assert_cast(*type).getValues()); + case TypeIndex::Enum16: + return std::make_unique>(assert_cast(*type).getValues()); + case TypeIndex::LowCardinality: + { + /// To optimize inserting into LowCardinality we have special nodes for LowCardinality of numeric and string types. + auto lc_type = typeid_cast(type.get()); + auto dictionary_type = removeNullable(lc_type->getDictionaryType()); + bool is_nullable = lc_type->isLowCardinalityNullable(); + + switch (dictionary_type->getTypeId()) + { + case TypeIndex::UInt8: + return std::make_unique>(is_nullable, isBool(type)); + case TypeIndex::UInt16: + return std::make_unique>(is_nullable); + case TypeIndex::UInt32: + return std::make_unique>(is_nullable); + case TypeIndex::UInt64: + return std::make_unique>(is_nullable); + case TypeIndex::Int8: + return std::make_unique>(is_nullable); + case TypeIndex::Int16: + return std::make_unique>(is_nullable); + case TypeIndex::Int32: + return std::make_unique>(is_nullable); + case TypeIndex::Int64: + return std::make_unique>(is_nullable); + case TypeIndex::Float32: + return std::make_unique>(is_nullable); + case TypeIndex::Float64: + return std::make_unique>(is_nullable); + case TypeIndex::String: + return std::make_unique>(is_nullable); + case TypeIndex::FixedString: + return std::make_unique>(is_nullable, assert_cast(*dictionary_type).getN()); + case TypeIndex::UUID: + return std::make_unique>(is_nullable); + default: + return std::make_unique>(is_nullable, buildJSONExtractTree(dictionary_type, source_for_exception_message)); + } + } + case TypeIndex::Nullable: + return std::make_unique>(buildJSONExtractTree(assert_cast(*type).getNestedType(), source_for_exception_message)); + case TypeIndex::Array: + return std::make_unique>(buildJSONExtractTree(assert_cast(*type).getNestedType(), source_for_exception_message)); + case TypeIndex::Tuple: + { + const auto & tuple = assert_cast(*type); + const auto & tuple_elements = tuple.getElements(); + std::vector>> elements; + elements.reserve(tuple_elements.size()); + for (const auto & tuple_element : tuple_elements) + elements.emplace_back(buildJSONExtractTree(tuple_element, source_for_exception_message)); + return std::make_unique>(std::move(elements), tuple.haveExplicitNames() ? tuple.getElementNames() : Strings{}); + } + case TypeIndex::Map: + { + const auto & map_type = assert_cast(*type); + const auto & key_type = map_type.getKeyType(); + if (!isString(removeLowCardinality(key_type))) + throw Exception( + ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT, + "{} doesn't support the return type schema: {} with key type not String", + source_for_exception_message, + type->getName()); + + const auto & value_type = map_type.getValueType(); + return std::make_unique>(buildJSONExtractTree(value_type, source_for_exception_message)); + } + case TypeIndex::Variant: + { + const auto & variant_type = assert_cast(*type); + const auto & variants = variant_type.getVariants(); + std::vector>> variant_nodes; + variant_nodes.reserve(variants.size()); + for (const auto & variant : variants) + variant_nodes.push_back(buildJSONExtractTree(variant, source_for_exception_message)); + return std::make_unique>(std::move(variant_nodes), SerializationVariant::getVariantsDeserializeTextOrder(variants)); + } + case TypeIndex::Dynamic: + return std::make_unique>(); + default: + throw Exception( + ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT, + "{} doesn't support the return type schema: {}", + source_for_exception_message, + type->getName()); + } +} + +#if USE_SIMDJSON +template void jsonElementToString(const SimdJSONParser::Element & element, WriteBuffer & buf, const FormatSettings & format_settings); +template std::unique_ptr> buildJSONExtractTree(const DataTypePtr & type, const char * source_for_exception_message); +#endif + +#if USE_RAPIDJSON +template void jsonElementToString(const RapidJSONParser::Element & element, WriteBuffer & buf, const FormatSettings & format_settings); +template std::unique_ptr> buildJSONExtractTree(const DataTypePtr & type, const char * source_for_exception_message); +#else +template void jsonElementToString(const DummyJSONParser::Element & element, WriteBuffer & buf, const FormatSettings & format_settings); +template std::unique_ptr> buildJSONExtractTree(const DataTypePtr & type, const char * source_for_exception_message); +#endif + +} diff --git a/src/Formats/JSONExtractTree.h b/src/Formats/JSONExtractTree.h new file mode 100644 index 00000000000..4735f568b1c --- /dev/null +++ b/src/Formats/JSONExtractTree.h @@ -0,0 +1,35 @@ +#pragma once +#include +#include +#include + + +namespace DB +{ + +struct JSONExtractInsertSettings +{ + bool convert_bool_to_integer = true; + bool insert_default_on_invalid_elements_in_complex_types = false; +}; + +template +class JSONExtractTreeNode +{ +public: + JSONExtractTreeNode() = default; + virtual ~JSONExtractTreeNode() = default; + virtual bool insertResultToColumn(IColumn &, const typename JSONParser::Element &, const JSONExtractInsertSettings & insert_setting, const FormatSettings & format_settings, String & error) const = 0; +}; + +/// Build a tree for insertion JSON element into a column with provided data type. +template +std::unique_ptr> buildJSONExtractTree(const DataTypePtr & type, const char * source_for_exception_message); + +template +void jsonElementToString(const typename JSONParser::Element & element, WriteBuffer & buf, const FormatSettings & format_settings); + +template +bool tryGetNumericValueFromJSONElement(NumberType & value, const typename JSONParser::Element & element, bool convert_bool_to_integer, String & error); + +} diff --git a/src/Formats/SchemaInferenceUtils.cpp b/src/Formats/SchemaInferenceUtils.cpp index 31faea2e13e..6519d54a8c5 100644 --- a/src/Formats/SchemaInferenceUtils.cpp +++ b/src/Formats/SchemaInferenceUtils.cpp @@ -225,19 +225,6 @@ namespace Paths paths; }; - bool checkIfTypesAreEqual(const DataTypes & types) - { - if (types.empty()) - return true; - - for (size_t i = 1; i < types.size(); ++i) - { - if (!types[0]->equals(*types[i])) - return false; - } - return true; - } - void updateTypeIndexes(DataTypes & data_types, TypeIndexesSet & type_indexes) { type_indexes.clear(); @@ -272,24 +259,31 @@ namespace type_indexes.erase(TypeIndex::Nothing); } - /// If we have both Int64 and UInt64, convert all Int64 to UInt64, + /// If we have both Int64 and UInt64, convert all not-negative Int64 to UInt64, /// because UInt64 is inferred only in case of Int64 overflow. - void transformIntegers(DataTypes & data_types, TypeIndexesSet & type_indexes) + void transformIntegers(DataTypes & data_types, TypeIndexesSet & type_indexes, JSONInferenceInfo * json_info) { if (!type_indexes.contains(TypeIndex::Int64) || !type_indexes.contains(TypeIndex::UInt64)) return; + bool have_negative_integers = false; for (auto & type : data_types) { if (WhichDataType(type).isInt64()) - type = std::make_shared(); + { + bool is_negative = json_info->negative_integers.contains(type.get()); + have_negative_integers |= is_negative; + if (!is_negative) + type = std::make_shared(); + } } - type_indexes.erase(TypeIndex::Int64); + if (!have_negative_integers) + type_indexes.erase(TypeIndex::Int64); } /// If we have both Int64 and Float64 types, convert all Int64 to Float64. - void transformIntegersAndFloatsToFloats(DataTypes & data_types, TypeIndexesSet & type_indexes) + void transformIntegersAndFloatsToFloats(DataTypes & data_types, TypeIndexesSet & type_indexes, JSONInferenceInfo * json_info) { bool have_floats = type_indexes.contains(TypeIndex::Float64); bool have_integers = type_indexes.contains(TypeIndex::Int64) || type_indexes.contains(TypeIndex::UInt64); @@ -300,7 +294,12 @@ namespace { WhichDataType which(type); if (which.isInt64() || which.isUInt64()) - type = std::make_shared(); + { + auto new_type = std::make_shared(); + if (json_info->numbers_parsed_from_json_strings.erase(type.get())) + json_info->numbers_parsed_from_json_strings.insert(new_type.get()); + type = new_type; + } } type_indexes.erase(TypeIndex::Int64); @@ -635,9 +634,9 @@ namespace if (settings.try_infer_integers) { /// Transform Int64 to UInt64 if needed. - transformIntegers(data_types, type_indexes); + transformIntegers(data_types, type_indexes, json_info); /// Transform integers to floats if needed. - transformIntegersAndFloatsToFloats(data_types, type_indexes); + transformIntegersAndFloatsToFloats(data_types, type_indexes, json_info); } /// Transform Date to DateTime or both to String if needed. @@ -887,7 +886,7 @@ namespace } template - DataTypePtr tryInferNumber(ReadBuffer & buf, const FormatSettings & settings) + DataTypePtr tryInferNumber(ReadBuffer & buf, const FormatSettings & settings, JSONInferenceInfo * json_info) { if (buf.eof()) return nullptr; @@ -911,7 +910,12 @@ namespace Int64 tmp_int; buf.position() = number_start; if (tryReadIntText(tmp_int, buf)) - return std::make_shared(); + { + auto type = std::make_shared(); + if (json_info && tmp_int < 0) + json_info->negative_integers.insert(type.get()); + return type; + } /// In case of Int64 overflow we can try to infer UInt64. UInt64 tmp_uint; @@ -934,7 +938,12 @@ namespace Int64 tmp_int; if (tryReadIntText(tmp_int, peekable_buf)) - return std::make_shared(); + { + auto type = std::make_shared(); + if (json_info && tmp_int < 0) + json_info->negative_integers.insert(type.get()); + return type; + } peekable_buf.rollbackToCheckpoint(/* drop= */ true); /// In case of Int64 overflow we can try to infer UInt64. @@ -952,7 +961,7 @@ namespace } template - DataTypePtr tryInferNumberFromStringImpl(std::string_view field, const FormatSettings & settings) + DataTypePtr tryInferNumberFromStringImpl(std::string_view field, const FormatSettings & settings, JSONInferenceInfo * json_inference_info = nullptr) { ReadBufferFromString buf(field); @@ -960,7 +969,12 @@ namespace { Int64 tmp_int; if (tryReadIntText(tmp_int, buf) && buf.eof()) - return std::make_shared(); + { + auto type = std::make_shared(); + if (json_inference_info && tmp_int < 0) + json_inference_info->negative_integers.insert(type.get()); + return type; + } /// We can safely get back to the start of buffer, because we read from a string and we didn't reach eof. buf.position() = buf.buffer().begin(); @@ -1011,7 +1025,7 @@ namespace { if (settings.json.try_infer_numbers_from_strings) { - if (auto number_type = tryInferNumberFromStringImpl(field, settings)) + if (auto number_type = tryInferNumberFromStringImpl(field, settings, json_info)) { json_info->numbers_parsed_from_json_strings.insert(number_type.get()); return number_type; @@ -1254,10 +1268,23 @@ namespace } /// Number - return tryInferNumber(buf, settings); + return tryInferNumber(buf, settings, json_info); } } +bool checkIfTypesAreEqual(const DataTypes & types) +{ + if (types.empty()) + return true; + + for (size_t i = 1; i < types.size(); ++i) + { + if (!types[0]->equals(*types[i])) + return false; + } + return true; +} + void transformInferredTypesIfNeeded(DataTypePtr & first, DataTypePtr & second, const FormatSettings & settings) { DataTypes types = {first, second}; @@ -1275,6 +1302,11 @@ void transformInferredJSONTypesIfNeeded( second = std::move(types[1]); } +void transformInferredJSONTypesIfNeeded(DataTypes & types, const FormatSettings & settings, JSONInferenceInfo * json_info) +{ + transformInferredTypesIfNeededImpl(types, settings, json_info); +} + void transformInferredJSONTypesFromDifferentFilesIfNeeded(DataTypePtr & first, DataTypePtr & second, const FormatSettings & settings) { JSONInferenceInfo json_info; @@ -1396,6 +1428,12 @@ DataTypePtr tryInferNumberFromString(std::string_view field, const FormatSetting return tryInferNumberFromStringImpl(field, settings); } +DataTypePtr tryInferJSONNumberFromString(std::string_view field, const FormatSettings & settings, JSONInferenceInfo * json_info) +{ + return tryInferNumberFromStringImpl(field, settings, json_info); + +} + DataTypePtr tryInferDateOrDateTimeFromString(std::string_view field, const FormatSettings & settings) { if (settings.try_infer_dates && tryInferDate(field)) diff --git a/src/Formats/SchemaInferenceUtils.h b/src/Formats/SchemaInferenceUtils.h index bcf3d194825..06c14c0797a 100644 --- a/src/Formats/SchemaInferenceUtils.h +++ b/src/Formats/SchemaInferenceUtils.h @@ -2,6 +2,7 @@ #include #include +#include #include @@ -18,6 +19,11 @@ struct JSONInferenceInfo /// We store numbers that were parsed from strings. /// It's used in types transformation to change such numbers back to string if needed. std::unordered_set numbers_parsed_from_json_strings; + /// Store integer types that were inferred from negative numbers. + /// It's used to determine common type for Int64 and UInt64 + /// TODO: check it not only in JSON formats. + std::unordered_set negative_integers; + /// Indicates if currently we are inferring type for Map/Object key. bool is_object_key = false; /// When we transform types for the same column from different files @@ -48,6 +54,7 @@ DataTypePtr tryInferDateOrDateTimeFromString(std::string_view field, const Forma /// Try to parse a number value from a string. By default, it tries to parse Float64, /// but if setting try_infer_integers is enabled, it also tries to parse Int64. DataTypePtr tryInferNumberFromString(std::string_view field, const FormatSettings & settings); +DataTypePtr tryInferJSONNumberFromString(std::string_view field, const FormatSettings & settings, JSONInferenceInfo * json_info); /// It takes two types inferred for the same column and tries to transform them to a common type if possible. /// It's also used when we try to infer some not ordinary types from another types. @@ -77,6 +84,7 @@ void transformInferredTypesIfNeeded(DataTypePtr & first, DataTypePtr & second, c /// Example 2: /// We merge DataTypeJSONPaths types to a single DataTypeJSONPaths type with union of all JSON paths. void transformInferredJSONTypesIfNeeded(DataTypePtr & first, DataTypePtr & second, const FormatSettings & settings, JSONInferenceInfo * json_info); +void transformInferredJSONTypesIfNeeded(DataTypes & types, const FormatSettings & settings, JSONInferenceInfo * json_info); /// Make final transform for types inferred in JSON format. It does 3 types of transformation: /// 1) Checks if type is unnamed Tuple(...), tries to transform nested types to find a common type for them and if all nested types @@ -107,4 +115,6 @@ NamesAndTypesList getNamesAndRecursivelyNullableTypes(const Block & header); /// Check if type contains Nothing, like Array(Tuple(Nullable(Nothing), String)) bool checkIfTypeIsComplete(const DataTypePtr & type); +bool checkIfTypesAreEqual(const DataTypes & types); + } diff --git a/src/Functions/FunctionsJSON.cpp b/src/Functions/FunctionsJSON.cpp index fbd987577e9..c6af0674db7 100644 --- a/src/Functions/FunctionsJSON.cpp +++ b/src/Functions/FunctionsJSON.cpp @@ -1,10 +1,1069 @@ -#include +#include +#include + +#include + +#include +#include + +#include +#include + +#include + +#include +#include +#include +#include +#include +#include + +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include + +#include #include +#include +#include +#include +#include + +#include + +#include "config.h" namespace DB { +namespace ErrorCodes +{ +extern const int ILLEGAL_TYPE_OF_ARGUMENT; +extern const int ILLEGAL_COLUMN; +extern const int NUMBER_OF_ARGUMENTS_DOESNT_MATCH; +} + +template +concept HasIndexOperator = requires (T t) +{ + t[0]; +}; + +/// Functions to parse JSONs and extract values from it. +/// The first argument of all these functions gets a JSON, +/// after that there are any number of arguments specifying path to a desired part from the JSON's root. +/// For example, +/// select JSONExtractInt('{"a": "hello", "b": [-100, 200.0, 300]}', 'b', 1) = -100 + +class FunctionJSONHelpers +{ +public: + template typename Impl, class JSONParser> + class Executor + { + public: + static ColumnPtr run(const ColumnsWithTypeAndName & arguments, const DataTypePtr & result_type, size_t input_rows_count, const FormatSettings & format_settings) + { + MutableColumnPtr to{result_type->createColumn()}; + to->reserve(input_rows_count); + + if (arguments.empty()) + throw Exception(ErrorCodes::NUMBER_OF_ARGUMENTS_DOESNT_MATCH, "Function {} requires at least one argument", String(Name::name)); + + const auto & first_column = arguments[0]; + if (!isString(first_column.type)) + throw Exception(ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT, + "The first argument of function {} should be a string containing JSON, illegal type: " + "{}", String(Name::name), first_column.type->getName()); + + const ColumnPtr & arg_json = first_column.column; + const auto * col_json_const = typeid_cast(arg_json.get()); + const auto * col_json_string + = typeid_cast(col_json_const ? col_json_const->getDataColumnPtr().get() : arg_json.get()); + + if (!col_json_string) + throw Exception(ErrorCodes::ILLEGAL_COLUMN, "Illegal column {}", arg_json->getName()); + + const ColumnString::Chars & chars = col_json_string->getChars(); + const ColumnString::Offsets & offsets = col_json_string->getOffsets(); + + size_t num_index_arguments = Impl::getNumberOfIndexArguments(arguments); + std::vector moves = prepareMoves(Name::name, arguments, 1, num_index_arguments); + + /// Preallocate memory in parser if necessary. + JSONParser parser; + if constexpr (has_member_function_reserve::value) + { + size_t max_size = calculateMaxSize(offsets); + if (max_size) + parser.reserve(max_size); + } + + Impl impl; + + /// prepare() does Impl-specific preparation before handling each row. + if constexpr (has_member_function_prepare::*)(const char *, const ColumnsWithTypeAndName &, const DataTypePtr &)>::value) + impl.prepare(Name::name, arguments, result_type); + + using Element = typename JSONParser::Element; + + Element document; + bool document_ok = false; + if (col_json_const) + { + std::string_view json{reinterpret_cast(chars.data()), offsets[0] - 1}; + document_ok = parser.parse(json, document); + } + + String error; + for (const auto i : collections::range(0, input_rows_count)) + { + if (!col_json_const) + { + std::string_view json{reinterpret_cast(&chars[offsets[i - 1]]), offsets[i] - offsets[i - 1] - 1}; + document_ok = parser.parse(json, document); + } + + bool added_to_column = false; + if (document_ok) + { + /// Perform moves. + Element element; + std::string_view last_key; + bool moves_ok = performMoves(arguments, i, document, moves, element, last_key); + + if (moves_ok) + added_to_column = impl.insertResultToColumn(*to, element, last_key, format_settings, error); + } + + /// We add default value (=null or zero) if something goes wrong, we don't throw exceptions in these JSON functions. + if (!added_to_column) + to->insertDefault(); + } + return to; + } + }; + +private: + BOOST_TTI_HAS_MEMBER_FUNCTION(reserve) + BOOST_TTI_HAS_MEMBER_FUNCTION(prepare) + + /// Represents a move of a JSON iterator described by a single argument passed to a JSON function. + /// For example, the call JSONExtractInt('{"a": "hello", "b": [-100, 200.0, 300]}', 'b', 1) + /// contains two moves: {MoveType::ConstKey, "b"} and {MoveType::ConstIndex, 1}. + /// Keys and indices can be nonconst, in this case they are calculated for each row. + enum class MoveType : uint8_t + { + Key, + Index, + ConstKey, + ConstIndex, + }; + + struct Move + { + explicit Move(MoveType type_, size_t index_ = 0) : type(type_), index(index_) {} + Move(MoveType type_, const String & key_) : type(type_), key(key_) {} + MoveType type; + size_t index = 0; + String key; + }; + + static std::vector prepareMoves( + const char * function_name, + const ColumnsWithTypeAndName & columns, + size_t first_index_argument, + size_t num_index_arguments) + { + std::vector moves; + moves.reserve(num_index_arguments); + for (const auto i : collections::range(first_index_argument, first_index_argument + num_index_arguments)) + { + const auto & column = columns[i]; + if (!isString(column.type) && !isNativeInteger(column.type)) + throw Exception(ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT, + "The argument {} of function {} should be a string specifying key " + "or an integer specifying index, illegal type: {}", + std::to_string(i + 1), String(function_name), column.type->getName()); + + if (column.column && isColumnConst(*column.column)) + { + const auto & column_const = assert_cast(*column.column); + if (isString(column.type)) + moves.emplace_back(MoveType::ConstKey, column_const.getValue()); + else + moves.emplace_back(MoveType::ConstIndex, column_const.getInt(0)); + } + else + { + if (isString(column.type)) + moves.emplace_back(MoveType::Key, ""); + else + moves.emplace_back(MoveType::Index, 0); + } + } + return moves; + } + + + /// Performs moves of types MoveType::Index and MoveType::ConstIndex. + template + static bool performMoves(const ColumnsWithTypeAndName & arguments, size_t row, + const typename JSONParser::Element & document, const std::vector & moves, + typename JSONParser::Element & element, std::string_view & last_key) + { + typename JSONParser::Element res_element = document; + std::string_view key; + + for (size_t j = 0; j != moves.size(); ++j) + { + switch (moves[j].type) + { + case MoveType::ConstIndex: + { + if (!moveToElementByIndex(res_element, static_cast(moves[j].index), key)) + return false; + break; + } + case MoveType::ConstKey: + { + key = moves[j].key; + if (!moveToElementByKey(res_element, key)) + return false; + break; + } + case MoveType::Index: + { + Int64 index = (*arguments[j + 1].column)[row].get(); + if (!moveToElementByIndex(res_element, static_cast(index), key)) + return false; + break; + } + case MoveType::Key: + { + key = arguments[j + 1].column->getDataAt(row).toView(); + if (!moveToElementByKey(res_element, key)) + return false; + break; + } + } + } + + element = res_element; + last_key = key; + return true; + } + + template + static bool moveToElementByIndex(typename JSONParser::Element & element, int index, std::string_view & out_key) + { + if (element.isArray()) + { + auto array = element.getArray(); + if (index >= 0) + --index; + else + index += array.size(); + + if (static_cast(index) >= array.size()) + return false; + element = array[index]; + out_key = {}; + return true; + } + + if constexpr (HasIndexOperator) + { + if (element.isObject()) + { + auto object = element.getObject(); + if (index >= 0) + --index; + else + index += object.size(); + + if (static_cast(index) >= object.size()) + return false; + std::tie(out_key, element) = object[index]; + return true; + } + } + + return {}; + } + + /// Performs moves of types MoveType::Key and MoveType::ConstKey. + template + static bool moveToElementByKey(typename JSONParser::Element & element, std::string_view key) + { + if (!element.isObject()) + return false; + auto object = element.getObject(); + return object.find(key, element); + } + + static size_t calculateMaxSize(const ColumnString::Offsets & offsets) + { + size_t max_size = 0; + for (const auto i : collections::range(0, offsets.size())) + { + size_t size = offsets[i] - offsets[i - 1]; + max_size = std::max(max_size, size); + } + if (max_size) + --max_size; + return max_size; + } + +}; + +template +class JSONExtractImpl; + +template +class JSONExtractKeysAndValuesImpl; + +/** +* Functions JSONExtract and JSONExtractKeysAndValues force the return type - it is specified in the last argument. +* For example - `SELECT JSONExtract(materialize('{"a": 131231, "b": 1234}'), 'b', 'LowCardinality(FixedString(4))')` +* But by default ClickHouse decides on its own whether the return type will be LowCardinality based on the types of +* input arguments. +* And for these specific functions we cannot rely on this mechanism, so these functions have their own implementation - +* just convert all of the LowCardinality input columns to full ones, execute and wrap the resulting column in LowCardinality +* if needed. +*/ +template typename Impl> +constexpr bool functionForcesTheReturnType() +{ + return std::is_same_v, JSONExtractImpl> || std::is_same_v, JSONExtractKeysAndValuesImpl>; +} + +template typename Impl> +class ExecutableFunctionJSON : public IExecutableFunction +{ + +public: + explicit ExecutableFunctionJSON(const NullPresence & null_presence_, bool allow_simdjson_, const DataTypePtr & json_return_type_, const FormatSettings & format_settings_) + : null_presence(null_presence_), allow_simdjson(allow_simdjson_), json_return_type(json_return_type_), format_settings(format_settings_) + { + format_settings.json.escape_forward_slashes = false; + format_settings.null_as_default = false; + } + + String getName() const override { return Name::name; } + bool useDefaultImplementationForNulls() const override { return false; } + bool useDefaultImplementationForConstants() const override { return true; } + bool useDefaultImplementationForLowCardinalityColumns() const override + { + return !functionForcesTheReturnType(); + } + + ColumnPtr executeImpl(const ColumnsWithTypeAndName & arguments, const DataTypePtr & result_type, size_t input_rows_count) const override + { + if (null_presence.has_null_constant) + return result_type->createColumnConstWithDefaultValue(input_rows_count); + + if constexpr (functionForcesTheReturnType()) + { + ColumnsWithTypeAndName columns_without_low_cardinality = arguments; + + for (auto & column : columns_without_low_cardinality) + { + column.column = recursiveRemoveLowCardinality(column.column); + column.type = recursiveRemoveLowCardinality(column.type); + } + + ColumnsWithTypeAndName temporary_columns = null_presence.has_nullable ? createBlockWithNestedColumns(columns_without_low_cardinality) : columns_without_low_cardinality; + ColumnPtr temporary_result = chooseAndRunJSONParser(temporary_columns, json_return_type, input_rows_count); + + if (null_presence.has_nullable) + temporary_result = wrapInNullable(temporary_result, columns_without_low_cardinality, result_type, input_rows_count); + + if (result_type->lowCardinality()) + temporary_result = recursiveLowCardinalityTypeConversion(temporary_result, json_return_type, result_type); + + return temporary_result; + } + else + { + ColumnsWithTypeAndName temporary_columns = null_presence.has_nullable ? createBlockWithNestedColumns(arguments) : arguments; + ColumnPtr temporary_result = chooseAndRunJSONParser(temporary_columns, json_return_type, input_rows_count); + + if (null_presence.has_nullable) + temporary_result = wrapInNullable(temporary_result, arguments, result_type, input_rows_count); + + if (result_type->lowCardinality()) + temporary_result = recursiveLowCardinalityTypeConversion(temporary_result, json_return_type, result_type); + + return temporary_result; + } + } + +private: + + ColumnPtr + chooseAndRunJSONParser(const ColumnsWithTypeAndName & arguments, const DataTypePtr & result_type, size_t input_rows_count) const + { +#if USE_SIMDJSON + if (allow_simdjson) + return FunctionJSONHelpers::Executor::run(arguments, result_type, input_rows_count, format_settings); +#endif + +#if USE_RAPIDJSON + return FunctionJSONHelpers::Executor::run(arguments, result_type, input_rows_count, format_settings); +#else + return FunctionJSONHelpers::Executor::run(arguments, result_type, input_rows_count, format_settings); +#endif + } + + NullPresence null_presence; + bool allow_simdjson; + DataTypePtr json_return_type; + FormatSettings format_settings; +}; + + +template typename Impl> +class FunctionBaseFunctionJSON : public IFunctionBase +{ +public: + explicit FunctionBaseFunctionJSON( + const NullPresence & null_presence_, + bool allow_simdjson_, + DataTypes argument_types_, + DataTypePtr return_type_, + DataTypePtr json_return_type_, + const FormatSettings & format_settings_) + : null_presence(null_presence_) + , allow_simdjson(allow_simdjson_) + , argument_types(std::move(argument_types_)) + , return_type(std::move(return_type_)) + , json_return_type(std::move(json_return_type_)) + , format_settings(format_settings_) + { + } + + String getName() const override { return Name::name; } + + const DataTypes & getArgumentTypes() const override + { + return argument_types; + } + + const DataTypePtr & getResultType() const override + { + return return_type; + } + + bool isSuitableForShortCircuitArgumentsExecution(const DataTypesWithConstInfo & /*arguments*/) const override { return true; } + + ExecutableFunctionPtr prepare(const ColumnsWithTypeAndName &) const override + { + return std::make_unique>(null_presence, allow_simdjson, json_return_type, format_settings); + } + +private: + NullPresence null_presence; + bool allow_simdjson; + DataTypes argument_types; + DataTypePtr return_type; + DataTypePtr json_return_type; + FormatSettings format_settings; +}; + +/// We use IFunctionOverloadResolver instead of IFunction to handle non-default NULL processing. +/// Both NULL and JSON NULL should generate NULL value. If any argument is NULL, return NULL. +template typename Impl> +class JSONOverloadResolver : public IFunctionOverloadResolver, WithContext +{ +public: + static constexpr auto name = Name::name; + + String getName() const override { return name; } + + static FunctionOverloadResolverPtr create(ContextPtr context_) + { + return std::make_unique(context_); + } + + explicit JSONOverloadResolver(ContextPtr context_) : WithContext(context_) {} + + bool isVariadic() const override { return true; } + size_t getNumberOfArguments() const override { return 0; } + bool useDefaultImplementationForNulls() const override { return false; } + bool useDefaultImplementationForLowCardinalityColumns() const override + { + return !functionForcesTheReturnType(); + } + + FunctionBasePtr build(const ColumnsWithTypeAndName & arguments) const override + { + bool has_nothing_argument = false; + for (const auto & arg : arguments) + has_nothing_argument |= isNothing(arg.type); + + DataTypePtr json_return_type = Impl::getReturnType(Name::name, createBlockWithNestedColumns(arguments)); + NullPresence null_presence = getNullPresense(arguments); + DataTypePtr return_type; + if (has_nothing_argument) + return_type = std::make_shared(); + else if (null_presence.has_null_constant) + return_type = makeNullable(std::make_shared()); + else if (null_presence.has_nullable) + return_type = makeNullable(json_return_type); + else + return_type = json_return_type; + + DataTypes argument_types; + argument_types.reserve(arguments.size()); + for (const auto & argument : arguments) + argument_types.emplace_back(argument.type); + return std::make_unique>( + null_presence, getContext()->getSettingsRef().allow_simdjson, argument_types, return_type, json_return_type, getFormatSettings(getContext())); + } +}; + +struct NameJSONHas { static constexpr auto name{"JSONHas"}; }; +struct NameIsValidJSON { static constexpr auto name{"isValidJSON"}; }; +struct NameJSONLength { static constexpr auto name{"JSONLength"}; }; +struct NameJSONKey { static constexpr auto name{"JSONKey"}; }; +struct NameJSONType { static constexpr auto name{"JSONType"}; }; +struct NameJSONExtractInt { static constexpr auto name{"JSONExtractInt"}; }; +struct NameJSONExtractUInt { static constexpr auto name{"JSONExtractUInt"}; }; +struct NameJSONExtractFloat { static constexpr auto name{"JSONExtractFloat"}; }; +struct NameJSONExtractBool { static constexpr auto name{"JSONExtractBool"}; }; +struct NameJSONExtractString { static constexpr auto name{"JSONExtractString"}; }; +struct NameJSONExtract { static constexpr auto name{"JSONExtract"}; }; +struct NameJSONExtractKeysAndValues { static constexpr auto name{"JSONExtractKeysAndValues"}; }; +struct NameJSONExtractRaw { static constexpr auto name{"JSONExtractRaw"}; }; +struct NameJSONExtractArrayRaw { static constexpr auto name{"JSONExtractArrayRaw"}; }; +struct NameJSONExtractKeysAndValuesRaw { static constexpr auto name{"JSONExtractKeysAndValuesRaw"}; }; +struct NameJSONExtractKeys { static constexpr auto name{"JSONExtractKeys"}; }; + + +template +class JSONHasImpl +{ +public: + using Element = typename JSONParser::Element; + + static DataTypePtr getReturnType(const char *, const ColumnsWithTypeAndName &) { return std::make_shared(); } + + static size_t getNumberOfIndexArguments(const ColumnsWithTypeAndName & arguments) { return arguments.size() - 1; } + + static bool insertResultToColumn(IColumn & dest, const Element &, std::string_view, const FormatSettings &, String &) + { + ColumnVector & col_vec = assert_cast &>(dest); + col_vec.insertValue(1); + return true; + } +}; + + +template +class IsValidJSONImpl +{ +public: + using Element = typename JSONParser::Element; + + static DataTypePtr getReturnType(const char * function_name, const ColumnsWithTypeAndName & arguments) + { + if (arguments.size() != 1) + { + /// IsValidJSON() shouldn't get parameters other than JSON. + throw Exception(ErrorCodes::NUMBER_OF_ARGUMENTS_DOESNT_MATCH, "Function {} needs exactly one argument", + String(function_name)); + } + return std::make_shared(); + } + + static size_t getNumberOfIndexArguments(const ColumnsWithTypeAndName &) { return 0; } + + static bool insertResultToColumn(IColumn & dest, const Element &, std::string_view, const FormatSettings &, String &) + { + /// This function is called only if JSON is valid. + /// If JSON isn't valid then `FunctionJSON::Executor::run()` adds default value (=zero) to `dest` without calling this function. + ColumnVector & col_vec = assert_cast &>(dest); + col_vec.insertValue(1); + return true; + } +}; + + +template +class JSONLengthImpl +{ +public: + using Element = typename JSONParser::Element; + + static DataTypePtr getReturnType(const char *, const ColumnsWithTypeAndName &) + { + return std::make_shared(); + } + + static size_t getNumberOfIndexArguments(const ColumnsWithTypeAndName & arguments) { return arguments.size() - 1; } + + static bool insertResultToColumn(IColumn & dest, const Element & element, std::string_view, const FormatSettings &, String &) + { + size_t size; + if (element.isArray()) + size = element.getArray().size(); + else if (element.isObject()) + size = element.getObject().size(); + else + return false; + + ColumnVector & col_vec = assert_cast &>(dest); + col_vec.insertValue(size); + return true; + } +}; + + +template +class JSONKeyImpl +{ +public: + using Element = typename JSONParser::Element; + + static DataTypePtr getReturnType(const char *, const ColumnsWithTypeAndName &) + { + return std::make_shared(); + } + + static size_t getNumberOfIndexArguments(const ColumnsWithTypeAndName & arguments) { return arguments.size() - 1; } + + static bool insertResultToColumn(IColumn & dest, const Element &, std::string_view last_key, const FormatSettings &, String &) + { + if (last_key.empty()) + return false; + ColumnString & col_str = assert_cast(dest); + col_str.insertData(last_key.data(), last_key.size()); + return true; + } +}; + + +template +class JSONTypeImpl +{ +public: + using Element = typename JSONParser::Element; + + static DataTypePtr getReturnType(const char *, const ColumnsWithTypeAndName &) + { + static const std::vector> values = { + {"Array", '['}, + {"Object", '{'}, + {"String", '"'}, + {"Int64", 'i'}, + {"UInt64", 'u'}, + {"Double", 'd'}, + {"Bool", 'b'}, + {"Null", 0}, /// the default value for the column. + }; + return std::make_shared>(values); + } + + static size_t getNumberOfIndexArguments(const ColumnsWithTypeAndName & arguments) { return arguments.size() - 1; } + + static bool insertResultToColumn(IColumn & dest, const Element & element, std::string_view, const FormatSettings &, String &) + { + UInt8 type; + switch (element.type()) + { + case ElementType::INT64: + type = 'i'; + break; + case ElementType::UINT64: + type = 'u'; + break; + case ElementType::DOUBLE: + type = 'd'; + break; + case ElementType::STRING: + type = '"'; + break; + case ElementType::ARRAY: + type = '['; + break; + case ElementType::OBJECT: + type = '{'; + break; + case ElementType::BOOL: + type = 'b'; + break; + case ElementType::NULL_VALUE: + type = 0; + break; + } + + ColumnVector & col_vec = assert_cast &>(dest); + col_vec.insertValue(type); + return true; + } +}; + + +template +class JSONExtractNumericImpl +{ +public: + using Element = typename JSONParser::Element; + + static DataTypePtr getReturnType(const char *, const ColumnsWithTypeAndName &) + { + return std::make_shared>(); + } + + static size_t getNumberOfIndexArguments(const ColumnsWithTypeAndName & arguments) { return arguments.size() - 1; } + + static const std::unique_ptr> & getInsertNode() + { + static const std::unique_ptr> node = buildJSONExtractTree(std::make_shared>()); + } + + static bool insertResultToColumn(IColumn & dest, const Element & element, std::string_view, const FormatSettings &, String & error) + { + NumberType value; + + tryGetNumericValueFromJSONElement(value, element, convert_bool_to_integer, error); + + if (dest.getDataType() == TypeIndex::LowCardinality) + { + ColumnLowCardinality & col_low = assert_cast(dest); + col_low.insertData(reinterpret_cast(&value), sizeof(value)); + } + else + { + auto & col_vec = assert_cast &>(dest); + col_vec.insertValue(value); + } + return true; + } +}; + + +template +using JSONExtractInt64Impl = JSONExtractNumericImpl; +template +using JSONExtractUInt64Impl = JSONExtractNumericImpl; +template +using JSONExtractFloat64Impl = JSONExtractNumericImpl; + + +template +class JSONExtractBoolImpl +{ +public: + using Element = typename JSONParser::Element; + + static DataTypePtr getReturnType(const char *, const ColumnsWithTypeAndName &) + { + return std::make_shared(); + } + + static size_t getNumberOfIndexArguments(const ColumnsWithTypeAndName & arguments) { return arguments.size() - 1; } + + static bool insertResultToColumn(IColumn & dest, const Element & element, std::string_view, const FormatSettings &, String &) + { + bool value; + switch (element.type()) + { + case ElementType::BOOL: + value = element.getBool(); + break; + case ElementType::INT64: + value = element.getInt64() != 0; + break; + case ElementType::UINT64: + value = element.getUInt64() != 0; + break; + default: + return false; + } + + auto & col_vec = assert_cast &>(dest); + col_vec.insertValue(static_cast(value)); + return true; + } +}; + +template +class JSONExtractRawImpl; + +template +class JSONExtractStringImpl +{ +public: + using Element = typename JSONParser::Element; + + static DataTypePtr getReturnType(const char *, const ColumnsWithTypeAndName &) + { + return std::make_shared(); + } + + static size_t getNumberOfIndexArguments(const ColumnsWithTypeAndName & arguments) { return arguments.size() - 1; } + + static bool insertResultToColumn(IColumn & dest, const Element & element, std::string_view, const FormatSettings & format_settings, String & error) + { + if (element.isNull()) + return false; + + if (!element.isString()) + return JSONExtractRawImpl::insertResultToColumn(dest, element, {}, format_settings, error); + + auto str = element.getString(); + ColumnString & col_str = assert_cast(dest); + col_str.insertData(str.data(), str.size()); + return true; + } +}; + +template +class JSONExtractImpl +{ +public: + using Element = typename JSONParser::Element; + + static DataTypePtr getReturnType(const char * function_name, const ColumnsWithTypeAndName & arguments) + { + if (arguments.size() < 2) + throw Exception(ErrorCodes::NUMBER_OF_ARGUMENTS_DOESNT_MATCH, "Function {} requires at least two arguments", String(function_name)); + + const auto & col = arguments.back(); + const auto * col_type_const = typeid_cast(col.column.get()); + if (!col_type_const || !isString(col.type)) + throw Exception(ErrorCodes::ILLEGAL_COLUMN, + "The last argument of function {} should " + "be a constant string specifying the return data type, illegal value: {}", + String(function_name), col.name); + + return DataTypeFactory::instance().get(col_type_const->getValue()); + } + + static size_t getNumberOfIndexArguments(const ColumnsWithTypeAndName & arguments) { return arguments.size() - 2; } + + void prepare(const char * function_name, const ColumnsWithTypeAndName &, const DataTypePtr & result_type) + { + extract_tree = buildJSONExtractTree(result_type, function_name); + insert_settings.insert_default_on_invalid_elements_in_complex_types = true; + } + + bool insertResultToColumn(IColumn & dest, const Element & element, std::string_view, const FormatSettings & format_settings, String & error) + { + return extract_tree->insertResultToColumn(dest, element, insert_settings, format_settings, error); + } + +protected: + std::unique_ptr> extract_tree; + JSONExtractInsertSettings insert_settings; +}; + + +template +class JSONExtractKeysAndValuesImpl +{ +public: + using Element = typename JSONParser::Element; + + static DataTypePtr getReturnType(const char * function_name, const ColumnsWithTypeAndName & arguments) + { + if (arguments.size() < 2) + throw Exception(ErrorCodes::NUMBER_OF_ARGUMENTS_DOESNT_MATCH, "Function {} requires at least two arguments", String(function_name)); + + const auto & col = arguments.back(); + const auto * col_type_const = typeid_cast(col.column.get()); + if (!col_type_const || !isString(col.type)) + throw Exception(ErrorCodes::ILLEGAL_COLUMN, + "The last argument of function {} should " + "be a constant string specifying the values' data type, illegal value: {}", + String(function_name), col.name); + + DataTypePtr key_type = std::make_unique(); + DataTypePtr value_type = DataTypeFactory::instance().get(col_type_const->getValue()); + DataTypePtr tuple_type = std::make_unique(DataTypes{key_type, value_type}); + return std::make_unique(tuple_type); + } + + static size_t getNumberOfIndexArguments(const ColumnsWithTypeAndName & arguments) { return arguments.size() - 2; } + + void prepare(const char * function_name, const ColumnsWithTypeAndName &, const DataTypePtr & result_type) + { + const auto tuple_type = typeid_cast(result_type.get())->getNestedType(); + const auto value_type = typeid_cast(tuple_type.get())->getElements()[1]; + extract_tree = buildJSONExtractTree(value_type, function_name); + insert_settings.insert_default_on_invalid_elements_in_complex_types = true; + } + + bool insertResultToColumn(IColumn & dest, const Element & element, std::string_view, const FormatSettings & format_settings, String & error) + { + if (!element.isObject()) + return false; + + auto object = element.getObject(); + + auto & col_arr = assert_cast(dest); + auto & col_tuple = assert_cast(col_arr.getData()); + size_t old_size = col_tuple.size(); + auto & col_key = assert_cast(col_tuple.getColumn(0)); + auto & col_value = col_tuple.getColumn(1); + + for (const auto & [key, value] : object) + { + if (extract_tree->insertResultToColumn(col_value, value, insert_settings, format_settings, error)) + col_key.insertData(key.data(), key.size()); + } + + if (col_tuple.size() == old_size) + return false; + + col_arr.getOffsets().push_back(col_tuple.size()); + return true; + } + +private: + std::unique_ptr> extract_tree; + JSONExtractInsertSettings insert_settings; +}; + + +template +class JSONExtractRawImpl +{ +public: + using Element = typename JSONParser::Element; + + static DataTypePtr getReturnType(const char *, const ColumnsWithTypeAndName &) + { + return std::make_shared(); + } + + static size_t getNumberOfIndexArguments(const ColumnsWithTypeAndName & arguments) { return arguments.size() - 1; } + + static bool insertResultToColumn(IColumn & dest, const Element & element, std::string_view, const FormatSettings & format_settings, String &) + { + ColumnString & col_str = assert_cast(dest); + auto & chars = col_str.getChars(); + WriteBufferFromVector buf(chars, AppendModeTag()); + jsonElementToString(element, buf, format_settings); + buf.finalize(); + chars.push_back(0); + col_str.getOffsets().push_back(chars.size()); + return true; + } +}; + + +template +class JSONExtractArrayRawImpl +{ +public: + using Element = typename JSONParser::Element; + + static DataTypePtr getReturnType(const char *, const ColumnsWithTypeAndName &) + { + return std::make_shared(std::make_shared()); + } + + static size_t getNumberOfIndexArguments(const ColumnsWithTypeAndName & arguments) { return arguments.size() - 1; } + + static bool insertResultToColumn(IColumn & dest, const Element & element, std::string_view, const FormatSettings & format_settings, String & error) + { + if (!element.isArray()) + return false; + + auto array = element.getArray(); + ColumnArray & col_res = assert_cast(dest); + + for (auto value : array) + JSONExtractRawImpl::insertResultToColumn(col_res.getData(), value, {}, format_settings, error); + + col_res.getOffsets().push_back(col_res.getOffsets().back() + array.size()); + return true; + } +}; + + +template +class JSONExtractKeysAndValuesRawImpl +{ +public: + using Element = typename JSONParser::Element; + + static DataTypePtr getReturnType(const char *, const ColumnsWithTypeAndName &) + { + DataTypePtr string_type = std::make_unique(); + DataTypePtr tuple_type = std::make_unique(DataTypes{string_type, string_type}); + return std::make_unique(tuple_type); + } + + static size_t getNumberOfIndexArguments(const ColumnsWithTypeAndName & arguments) { return arguments.size() - 1; } + + bool insertResultToColumn(IColumn & dest, const Element & element, std::string_view, const FormatSettings & format_settings, String & error) + { + if (!element.isObject()) + return false; + + auto object = element.getObject(); + + auto & col_arr = assert_cast(dest); + auto & col_tuple = assert_cast(col_arr.getData()); + auto & col_key = assert_cast(col_tuple.getColumn(0)); + auto & col_value = assert_cast(col_tuple.getColumn(1)); + + for (const auto & [key, value] : object) + { + col_key.insertData(key.data(), key.size()); + JSONExtractRawImpl::insertResultToColumn(col_value, value, {}, format_settings, error); + } + + col_arr.getOffsets().push_back(col_arr.getOffsets().back() + object.size()); + return true; + } +}; + +template +class JSONExtractKeysImpl +{ +public: + using Element = typename JSONParser::Element; + + static DataTypePtr getReturnType(const char *, const ColumnsWithTypeAndName &) + { + return std::make_unique(std::make_shared()); + } + + static size_t getNumberOfIndexArguments(const ColumnsWithTypeAndName & arguments) { return arguments.size() - 1; } + + bool insertResultToColumn(IColumn & dest, const Element & element, std::string_view, const FormatSettings &, String &) + { + if (!element.isObject()) + return false; + + auto object = element.getObject(); + + ColumnArray & col_res = assert_cast(dest); + auto & col_key = assert_cast(col_res.getData()); + + for (const auto & [key, value] : object) + { + col_key.insertData(key.data(), key.size()); + } + + col_res.getOffsets().push_back(col_res.getOffsets().back() + object.size()); + return true; + } +}; + REGISTER_FUNCTION(JSON) { factory.registerFunction>(); diff --git a/src/Functions/FunctionsJSON.h b/src/Functions/FunctionsJSON.h index 8a2ad457d34..5d44e22300d 100644 --- a/src/Functions/FunctionsJSON.h +++ b/src/Functions/FunctionsJSON.h @@ -1,1781 +1,1273 @@ -#pragma once - -#include -#include - -#include - -#include -#include -#include - -#include -#include - -#include -#include -#include -#include -#include -#include -#include -#include -#include -#include - -#include -#include -#include -#include -#include -#include -#include -#include -#include -#include -#include -#include -#include -#include -#include -#include - -#include -#include -#include -#include -#include - -#include -#include - - -#include "config.h" - - -namespace DB -{ - -namespace ErrorCodes -{ - extern const int ILLEGAL_TYPE_OF_ARGUMENT; - extern const int ILLEGAL_COLUMN; - extern const int NUMBER_OF_ARGUMENTS_DOESNT_MATCH; -} - -template -concept HasIndexOperator = requires (T t) -{ - t[0]; -}; - -/// Functions to parse JSONs and extract values from it. -/// The first argument of all these functions gets a JSON, -/// after that there are any number of arguments specifying path to a desired part from the JSON's root. -/// For example, -/// select JSONExtractInt('{"a": "hello", "b": [-100, 200.0, 300]}', 'b', 1) = -100 - -class FunctionJSONHelpers -{ -public: - template typename Impl, class JSONParser> - class Executor - { - public: - static ColumnPtr run(const ColumnsWithTypeAndName & arguments, const DataTypePtr & result_type, size_t input_rows_count) - { - MutableColumnPtr to{result_type->createColumn()}; - to->reserve(input_rows_count); - - if (arguments.empty()) - throw Exception(ErrorCodes::NUMBER_OF_ARGUMENTS_DOESNT_MATCH, "Function {} requires at least one argument", String(Name::name)); - - const auto & first_column = arguments[0]; - if (!isString(first_column.type)) - throw Exception(ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT, - "The first argument of function {} should be a string containing JSON, illegal type: " - "{}", String(Name::name), first_column.type->getName()); - - const ColumnPtr & arg_json = first_column.column; - const auto * col_json_const = typeid_cast(arg_json.get()); - const auto * col_json_string - = typeid_cast(col_json_const ? col_json_const->getDataColumnPtr().get() : arg_json.get()); - - if (!col_json_string) - throw Exception(ErrorCodes::ILLEGAL_COLUMN, "Illegal column {}", arg_json->getName()); - - const ColumnString::Chars & chars = col_json_string->getChars(); - const ColumnString::Offsets & offsets = col_json_string->getOffsets(); - - size_t num_index_arguments = Impl::getNumberOfIndexArguments(arguments); - std::vector moves = prepareMoves(Name::name, arguments, 1, num_index_arguments); - - /// Preallocate memory in parser if necessary. - JSONParser parser; - if constexpr (has_member_function_reserve::value) - { - size_t max_size = calculateMaxSize(offsets); - if (max_size) - parser.reserve(max_size); - } - - Impl impl; - - /// prepare() does Impl-specific preparation before handling each row. - if constexpr (has_member_function_prepare::*)(const char *, const ColumnsWithTypeAndName &, const DataTypePtr &)>::value) - impl.prepare(Name::name, arguments, result_type); - - using Element = typename JSONParser::Element; - - Element document; - bool document_ok = false; - if (col_json_const) - { - std::string_view json{reinterpret_cast(chars.data()), offsets[0] - 1}; - document_ok = parser.parse(json, document); - } - - for (const auto i : collections::range(0, input_rows_count)) - { - if (!col_json_const) - { - std::string_view json{reinterpret_cast(&chars[offsets[i - 1]]), offsets[i] - offsets[i - 1] - 1}; - document_ok = parser.parse(json, document); - } - - bool added_to_column = false; - if (document_ok) - { - /// Perform moves. - Element element; - std::string_view last_key; - bool moves_ok = performMoves(arguments, i, document, moves, element, last_key); - - if (moves_ok) - added_to_column = impl.insertResultToColumn(*to, element, last_key); - } - - /// We add default value (=null or zero) if something goes wrong, we don't throw exceptions in these JSON functions. - if (!added_to_column) - to->insertDefault(); - } - return to; - } - }; - -private: - BOOST_TTI_HAS_MEMBER_FUNCTION(reserve) - BOOST_TTI_HAS_MEMBER_FUNCTION(prepare) - - /// Represents a move of a JSON iterator described by a single argument passed to a JSON function. - /// For example, the call JSONExtractInt('{"a": "hello", "b": [-100, 200.0, 300]}', 'b', 1) - /// contains two moves: {MoveType::ConstKey, "b"} and {MoveType::ConstIndex, 1}. - /// Keys and indices can be nonconst, in this case they are calculated for each row. - enum class MoveType : uint8_t - { - Key, - Index, - ConstKey, - ConstIndex, - }; - - struct Move - { - explicit Move(MoveType type_, size_t index_ = 0) : type(type_), index(index_) {} - Move(MoveType type_, const String & key_) : type(type_), key(key_) {} - MoveType type; - size_t index = 0; - String key; - }; - - static std::vector prepareMoves( - const char * function_name, - const ColumnsWithTypeAndName & columns, - size_t first_index_argument, - size_t num_index_arguments) - { - std::vector moves; - moves.reserve(num_index_arguments); - for (const auto i : collections::range(first_index_argument, first_index_argument + num_index_arguments)) - { - const auto & column = columns[i]; - if (!isString(column.type) && !isNativeInteger(column.type)) - throw Exception(ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT, - "The argument {} of function {} should be a string specifying key " - "or an integer specifying index, illegal type: {}", - std::to_string(i + 1), String(function_name), column.type->getName()); - - if (column.column && isColumnConst(*column.column)) - { - const auto & column_const = assert_cast(*column.column); - if (isString(column.type)) - moves.emplace_back(MoveType::ConstKey, column_const.getValue()); - else - moves.emplace_back(MoveType::ConstIndex, column_const.getInt(0)); - } - else - { - if (isString(column.type)) - moves.emplace_back(MoveType::Key, ""); - else - moves.emplace_back(MoveType::Index, 0); - } - } - return moves; - } - - - /// Performs moves of types MoveType::Index and MoveType::ConstIndex. - template - static bool performMoves(const ColumnsWithTypeAndName & arguments, size_t row, - const typename JSONParser::Element & document, const std::vector & moves, - typename JSONParser::Element & element, std::string_view & last_key) - { - typename JSONParser::Element res_element = document; - std::string_view key; - - for (size_t j = 0; j != moves.size(); ++j) - { - switch (moves[j].type) - { - case MoveType::ConstIndex: - { - if (!moveToElementByIndex(res_element, static_cast(moves[j].index), key)) - return false; - break; - } - case MoveType::ConstKey: - { - key = moves[j].key; - if (!moveToElementByKey(res_element, key)) - return false; - break; - } - case MoveType::Index: - { - Int64 index = (*arguments[j + 1].column)[row].get(); - if (!moveToElementByIndex(res_element, static_cast(index), key)) - return false; - break; - } - case MoveType::Key: - { - key = arguments[j + 1].column->getDataAt(row).toView(); - if (!moveToElementByKey(res_element, key)) - return false; - break; - } - } - } - - element = res_element; - last_key = key; - return true; - } - - template - static bool moveToElementByIndex(typename JSONParser::Element & element, int index, std::string_view & out_key) - { - if (element.isArray()) - { - auto array = element.getArray(); - if (index >= 0) - --index; - else - index += array.size(); - - if (static_cast(index) >= array.size()) - return false; - element = array[index]; - out_key = {}; - return true; - } - - if constexpr (HasIndexOperator) - { - if (element.isObject()) - { - auto object = element.getObject(); - if (index >= 0) - --index; - else - index += object.size(); - - if (static_cast(index) >= object.size()) - return false; - std::tie(out_key, element) = object[index]; - return true; - } - } - - return {}; - } - - /// Performs moves of types MoveType::Key and MoveType::ConstKey. - template - static bool moveToElementByKey(typename JSONParser::Element & element, std::string_view key) - { - if (!element.isObject()) - return false; - auto object = element.getObject(); - return object.find(key, element); - } - - static size_t calculateMaxSize(const ColumnString::Offsets & offsets) - { - size_t max_size = 0; - for (const auto i : collections::range(0, offsets.size())) - { - size_t size = offsets[i] - offsets[i - 1]; - max_size = std::max(max_size, size); - } - if (max_size) - --max_size; - return max_size; - } - -}; - -template -class JSONExtractImpl; - -template -class JSONExtractKeysAndValuesImpl; - -/** -* Functions JSONExtract and JSONExtractKeysAndValues force the return type - it is specified in the last argument. -* For example - `SELECT JSONExtract(materialize('{"a": 131231, "b": 1234}'), 'b', 'LowCardinality(FixedString(4))')` -* But by default ClickHouse decides on its own whether the return type will be LowCardinality based on the types of -* input arguments. -* And for these specific functions we cannot rely on this mechanism, so these functions have their own implementation - -* just convert all of the LowCardinality input columns to full ones, execute and wrap the resulting column in LowCardinality -* if needed. -*/ -template typename Impl> -constexpr bool functionForcesTheReturnType() -{ - return std::is_same_v, JSONExtractImpl> || std::is_same_v, JSONExtractKeysAndValuesImpl>; -} - -template typename Impl> -class ExecutableFunctionJSON : public IExecutableFunction -{ - -public: - explicit ExecutableFunctionJSON(const NullPresence & null_presence_, bool allow_simdjson_, const DataTypePtr & json_return_type_) - : null_presence(null_presence_), allow_simdjson(allow_simdjson_), json_return_type(json_return_type_) - { - } - - String getName() const override { return Name::name; } - bool useDefaultImplementationForNulls() const override { return false; } - bool useDefaultImplementationForConstants() const override { return true; } - bool useDefaultImplementationForLowCardinalityColumns() const override - { - return !functionForcesTheReturnType(); - } - - ColumnPtr executeImpl(const ColumnsWithTypeAndName & arguments, const DataTypePtr & result_type, size_t input_rows_count) const override - { - if (null_presence.has_null_constant) - return result_type->createColumnConstWithDefaultValue(input_rows_count); - - if constexpr (functionForcesTheReturnType()) - { - ColumnsWithTypeAndName columns_without_low_cardinality = arguments; - - for (auto & column : columns_without_low_cardinality) - { - column.column = recursiveRemoveLowCardinality(column.column); - column.type = recursiveRemoveLowCardinality(column.type); - } - - ColumnsWithTypeAndName temporary_columns = null_presence.has_nullable ? createBlockWithNestedColumns(columns_without_low_cardinality) : columns_without_low_cardinality; - ColumnPtr temporary_result = chooseAndRunJSONParser(temporary_columns, json_return_type, input_rows_count); - - if (null_presence.has_nullable) - temporary_result = wrapInNullable(temporary_result, columns_without_low_cardinality, result_type, input_rows_count); - - if (result_type->lowCardinality()) - temporary_result = recursiveLowCardinalityTypeConversion(temporary_result, json_return_type, result_type); - - return temporary_result; - } - else - { - ColumnsWithTypeAndName temporary_columns = null_presence.has_nullable ? createBlockWithNestedColumns(arguments) : arguments; - ColumnPtr temporary_result = chooseAndRunJSONParser(temporary_columns, json_return_type, input_rows_count); - - if (null_presence.has_nullable) - temporary_result = wrapInNullable(temporary_result, arguments, result_type, input_rows_count); - - if (result_type->lowCardinality()) - temporary_result = recursiveLowCardinalityTypeConversion(temporary_result, json_return_type, result_type); - - return temporary_result; - } - } - -private: - - ColumnPtr - chooseAndRunJSONParser(const ColumnsWithTypeAndName & arguments, const DataTypePtr & result_type, size_t input_rows_count) const - { -#if USE_SIMDJSON - if (allow_simdjson) - return FunctionJSONHelpers::Executor::run(arguments, result_type, input_rows_count); -#endif - -#if USE_RAPIDJSON - return FunctionJSONHelpers::Executor::run(arguments, result_type, input_rows_count); -#else - return FunctionJSONHelpers::Executor::run(arguments, result_type, input_rows_count); -#endif - } - - NullPresence null_presence; - bool allow_simdjson; - DataTypePtr json_return_type; -}; - - -template typename Impl> -class FunctionBaseFunctionJSON : public IFunctionBase -{ -public: - explicit FunctionBaseFunctionJSON( - const NullPresence & null_presence_, - bool allow_simdjson_, - DataTypes argument_types_, - DataTypePtr return_type_, - DataTypePtr json_return_type_) - : null_presence(null_presence_) - , allow_simdjson(allow_simdjson_) - , argument_types(std::move(argument_types_)) - , return_type(std::move(return_type_)) - , json_return_type(std::move(json_return_type_)) - { - } - - String getName() const override { return Name::name; } - - const DataTypes & getArgumentTypes() const override - { - return argument_types; - } - - const DataTypePtr & getResultType() const override - { - return return_type; - } - - bool isSuitableForShortCircuitArgumentsExecution(const DataTypesWithConstInfo & /*arguments*/) const override { return true; } - - ExecutableFunctionPtr prepare(const ColumnsWithTypeAndName &) const override - { - return std::make_unique>(null_presence, allow_simdjson, json_return_type); - } - -private: - NullPresence null_presence; - bool allow_simdjson; - DataTypes argument_types; - DataTypePtr return_type; - DataTypePtr json_return_type; -}; - -/// We use IFunctionOverloadResolver instead of IFunction to handle non-default NULL processing. -/// Both NULL and JSON NULL should generate NULL value. If any argument is NULL, return NULL. -template typename Impl> -class JSONOverloadResolver : public IFunctionOverloadResolver, WithContext -{ -public: - static constexpr auto name = Name::name; - - String getName() const override { return name; } - - static FunctionOverloadResolverPtr create(ContextPtr context_) - { - return std::make_unique(context_); - } - - explicit JSONOverloadResolver(ContextPtr context_) : WithContext(context_) {} - - bool isVariadic() const override { return true; } - size_t getNumberOfArguments() const override { return 0; } - bool useDefaultImplementationForNulls() const override { return false; } - bool useDefaultImplementationForLowCardinalityColumns() const override - { - return !functionForcesTheReturnType(); - } - - FunctionBasePtr build(const ColumnsWithTypeAndName & arguments) const override - { - bool has_nothing_argument = false; - for (const auto & arg : arguments) - has_nothing_argument |= isNothing(arg.type); - - DataTypePtr json_return_type = Impl::getReturnType(Name::name, createBlockWithNestedColumns(arguments)); - NullPresence null_presence = getNullPresense(arguments); - DataTypePtr return_type; - if (has_nothing_argument) - return_type = std::make_shared(); - else if (null_presence.has_null_constant) - return_type = makeNullable(std::make_shared()); - else if (null_presence.has_nullable) - return_type = makeNullable(json_return_type); - else - return_type = json_return_type; - - /// Top-level LowCardinality columns are processed outside JSON parser. - json_return_type = removeLowCardinality(json_return_type); - - DataTypes argument_types; - argument_types.reserve(arguments.size()); - for (const auto & argument : arguments) - argument_types.emplace_back(argument.type); - return std::make_unique>( - null_presence, getContext()->getSettingsRef().allow_simdjson, argument_types, return_type, json_return_type); - } -}; - -struct NameJSONHas { static constexpr auto name{"JSONHas"}; }; -struct NameIsValidJSON { static constexpr auto name{"isValidJSON"}; }; -struct NameJSONLength { static constexpr auto name{"JSONLength"}; }; -struct NameJSONKey { static constexpr auto name{"JSONKey"}; }; -struct NameJSONType { static constexpr auto name{"JSONType"}; }; -struct NameJSONExtractInt { static constexpr auto name{"JSONExtractInt"}; }; -struct NameJSONExtractUInt { static constexpr auto name{"JSONExtractUInt"}; }; -struct NameJSONExtractFloat { static constexpr auto name{"JSONExtractFloat"}; }; -struct NameJSONExtractBool { static constexpr auto name{"JSONExtractBool"}; }; -struct NameJSONExtractString { static constexpr auto name{"JSONExtractString"}; }; -struct NameJSONExtract { static constexpr auto name{"JSONExtract"}; }; -struct NameJSONExtractKeysAndValues { static constexpr auto name{"JSONExtractKeysAndValues"}; }; -struct NameJSONExtractRaw { static constexpr auto name{"JSONExtractRaw"}; }; -struct NameJSONExtractArrayRaw { static constexpr auto name{"JSONExtractArrayRaw"}; }; -struct NameJSONExtractKeysAndValuesRaw { static constexpr auto name{"JSONExtractKeysAndValuesRaw"}; }; -struct NameJSONExtractKeys { static constexpr auto name{"JSONExtractKeys"}; }; - - -template -class JSONHasImpl -{ -public: - using Element = typename JSONParser::Element; - - static DataTypePtr getReturnType(const char *, const ColumnsWithTypeAndName &) { return std::make_shared(); } - - static size_t getNumberOfIndexArguments(const ColumnsWithTypeAndName & arguments) { return arguments.size() - 1; } - - static bool insertResultToColumn(IColumn & dest, const Element &, std::string_view) - { - ColumnVector & col_vec = assert_cast &>(dest); - col_vec.insertValue(1); - return true; - } -}; - - -template -class IsValidJSONImpl -{ -public: - using Element = typename JSONParser::Element; - - static DataTypePtr getReturnType(const char * function_name, const ColumnsWithTypeAndName & arguments) - { - if (arguments.size() != 1) - { - /// IsValidJSON() shouldn't get parameters other than JSON. - throw Exception(ErrorCodes::NUMBER_OF_ARGUMENTS_DOESNT_MATCH, "Function {} needs exactly one argument", - String(function_name)); - } - return std::make_shared(); - } - - static size_t getNumberOfIndexArguments(const ColumnsWithTypeAndName &) { return 0; } - - static bool insertResultToColumn(IColumn & dest, const Element &, std::string_view) - { - /// This function is called only if JSON is valid. - /// If JSON isn't valid then `FunctionJSON::Executor::run()` adds default value (=zero) to `dest` without calling this function. - ColumnVector & col_vec = assert_cast &>(dest); - col_vec.insertValue(1); - return true; - } -}; - - -template -class JSONLengthImpl -{ -public: - using Element = typename JSONParser::Element; - - static DataTypePtr getReturnType(const char *, const ColumnsWithTypeAndName &) - { - return std::make_shared(); - } - - static size_t getNumberOfIndexArguments(const ColumnsWithTypeAndName & arguments) { return arguments.size() - 1; } - - static bool insertResultToColumn(IColumn & dest, const Element & element, std::string_view) - { - size_t size; - if (element.isArray()) - size = element.getArray().size(); - else if (element.isObject()) - size = element.getObject().size(); - else - return false; - - ColumnVector & col_vec = assert_cast &>(dest); - col_vec.insertValue(size); - return true; - } -}; - - -template -class JSONKeyImpl -{ -public: - using Element = typename JSONParser::Element; - - static DataTypePtr getReturnType(const char *, const ColumnsWithTypeAndName &) - { - return std::make_shared(); - } - - static size_t getNumberOfIndexArguments(const ColumnsWithTypeAndName & arguments) { return arguments.size() - 1; } - - static bool insertResultToColumn(IColumn & dest, const Element &, std::string_view last_key) - { - if (last_key.empty()) - return false; - ColumnString & col_str = assert_cast(dest); - col_str.insertData(last_key.data(), last_key.size()); - return true; - } -}; - - -template -class JSONTypeImpl -{ -public: - using Element = typename JSONParser::Element; - - static DataTypePtr getReturnType(const char *, const ColumnsWithTypeAndName &) - { - static const std::vector> values = { - {"Array", '['}, - {"Object", '{'}, - {"String", '"'}, - {"Int64", 'i'}, - {"UInt64", 'u'}, - {"Double", 'd'}, - {"Bool", 'b'}, - {"Null", 0}, /// the default value for the column. - }; - return std::make_shared>(values); - } - - static size_t getNumberOfIndexArguments(const ColumnsWithTypeAndName & arguments) { return arguments.size() - 1; } - - static bool insertResultToColumn(IColumn & dest, const Element & element, std::string_view) - { - UInt8 type; - switch (element.type()) - { - case ElementType::INT64: - type = 'i'; - break; - case ElementType::UINT64: - type = 'u'; - break; - case ElementType::DOUBLE: - type = 'd'; - break; - case ElementType::STRING: - type = '"'; - break; - case ElementType::ARRAY: - type = '['; - break; - case ElementType::OBJECT: - type = '{'; - break; - case ElementType::BOOL: - type = 'b'; - break; - case ElementType::NULL_VALUE: - type = 0; - break; - } - - ColumnVector & col_vec = assert_cast &>(dest); - col_vec.insertValue(type); - return true; - } -}; - - -template -class JSONExtractNumericImpl -{ -public: - using Element = typename JSONParser::Element; - - static DataTypePtr getReturnType(const char *, const ColumnsWithTypeAndName &) - { - return std::make_shared>(); - } - - static size_t getNumberOfIndexArguments(const ColumnsWithTypeAndName & arguments) { return arguments.size() - 1; } - - static bool insertResultToColumn(IColumn & dest, const Element & element, std::string_view) - { - NumberType value; - - switch (element.type()) - { - case ElementType::DOUBLE: - if constexpr (std::is_floating_point_v) - { - /// We permit inaccurate conversion of double to float. - /// Example: double 0.1 from JSON is not representable in float. - /// But it will be more convenient for user to perform conversion. - value = static_cast(element.getDouble()); - } - else if (!accurate::convertNumeric(element.getDouble(), value)) - return false; - break; - case ElementType::UINT64: - if (!accurate::convertNumeric(element.getUInt64(), value)) - return false; - break; - case ElementType::INT64: - if (!accurate::convertNumeric(element.getInt64(), value)) - return false; - break; - case ElementType::BOOL: - if constexpr (is_integer && convert_bool_to_integer) - { - value = static_cast(element.getBool()); - break; - } - return false; - case ElementType::STRING: - { - auto rb = ReadBufferFromMemory{element.getString()}; - if constexpr (std::is_floating_point_v) - { - if (!tryReadFloatText(value, rb) || !rb.eof()) - return false; - } - else - { - if (tryReadIntText(value, rb) && rb.eof()) - break; - - /// Try to parse float and convert it to integer. - Float64 tmp_float; - rb.position() = rb.buffer().begin(); - if (!tryReadFloatText(tmp_float, rb) || !rb.eof()) - return false; - - if (!accurate::convertNumeric(tmp_float, value)) - return false; - } - break; - } - default: - return false; - } - - if (dest.getDataType() == TypeIndex::LowCardinality) - { - ColumnLowCardinality & col_low = assert_cast(dest); - col_low.insertData(reinterpret_cast(&value), sizeof(value)); - } - else - { - auto & col_vec = assert_cast &>(dest); - col_vec.insertValue(value); - } - return true; - } -}; - - -template -using JSONExtractInt64Impl = JSONExtractNumericImpl; -template -using JSONExtractUInt64Impl = JSONExtractNumericImpl; -template -using JSONExtractFloat64Impl = JSONExtractNumericImpl; - - -template -class JSONExtractBoolImpl -{ -public: - using Element = typename JSONParser::Element; - - static DataTypePtr getReturnType(const char *, const ColumnsWithTypeAndName &) - { - return std::make_shared(); - } - - static size_t getNumberOfIndexArguments(const ColumnsWithTypeAndName & arguments) { return arguments.size() - 1; } - - static bool insertResultToColumn(IColumn & dest, const Element & element, std::string_view) - { - bool value; - switch (element.type()) - { - case ElementType::BOOL: - value = element.getBool(); - break; - case ElementType::INT64: - value = element.getInt64() != 0; - break; - case ElementType::UINT64: - value = element.getUInt64() != 0; - break; - default: - return false; - } - - auto & col_vec = assert_cast &>(dest); - col_vec.insertValue(static_cast(value)); - return true; - } -}; - -template -class JSONExtractRawImpl; - -template -class JSONExtractStringImpl -{ -public: - using Element = typename JSONParser::Element; - - static DataTypePtr getReturnType(const char *, const ColumnsWithTypeAndName &) - { - return std::make_shared(); - } - - static size_t getNumberOfIndexArguments(const ColumnsWithTypeAndName & arguments) { return arguments.size() - 1; } - - static bool insertResultToColumn(IColumn & dest, const Element & element, std::string_view) - { - if (element.isNull()) - return false; - - if (!element.isString()) - return JSONExtractRawImpl::insertResultToColumn(dest, element, {}); - - auto str = element.getString(); - - if (dest.getDataType() == TypeIndex::LowCardinality) - { - ColumnLowCardinality & col_low = assert_cast(dest); - col_low.insertData(str.data(), str.size()); - } - else - { - ColumnString & col_str = assert_cast(dest); - col_str.insertData(str.data(), str.size()); - } - return true; - } -}; - -/// Nodes of the extract tree. We need the extract tree to extract from JSON complex values containing array, tuples or nullables. -template -struct JSONExtractTree -{ - using Element = typename JSONParser::Element; - - class Node - { - public: - Node() = default; - virtual ~Node() = default; - virtual bool insertResultToColumn(IColumn &, const Element &) = 0; - }; - - template - class NumericNode : public Node - { - public: - bool insertResultToColumn(IColumn & dest, const Element & element) override - { - return JSONExtractNumericImpl::insertResultToColumn(dest, element, {}); - } - }; - - class LowCardinalityFixedStringNode : public Node - { - public: - explicit LowCardinalityFixedStringNode(const size_t fixed_length_) : fixed_length(fixed_length_) { } - bool insertResultToColumn(IColumn & dest, const Element & element) override - { - // If element is an object we delegate the insertion to JSONExtractRawImpl - if (element.isObject()) - return JSONExtractRawImpl::insertResultToLowCardinalityFixedStringColumn(dest, element, fixed_length); - else if (!element.isString()) - return false; - - auto str = element.getString(); - if (str.size() > fixed_length) - return false; - - // For the non low cardinality case of FixedString, the padding is done in the FixedString Column implementation. - // In order to avoid having to pass the data to a FixedString Column and read it back (which would slow down the execution) - // the data is padded here and written directly to the Low Cardinality Column - if (str.size() == fixed_length) - { - assert_cast(dest).insertData(str.data(), str.size()); - } - else - { - String padded_str(str); - padded_str.resize(fixed_length, '\0'); - - assert_cast(dest).insertData(padded_str.data(), padded_str.size()); - } - return true; - } - - private: - const size_t fixed_length; - }; - - class UUIDNode : public Node - { - public: - bool insertResultToColumn(IColumn & dest, const Element & element) override - { - if (!element.isString()) - return false; - - auto uuid = parseFromString(element.getString()); - if (dest.getDataType() == TypeIndex::LowCardinality) - { - ColumnLowCardinality & col_low = assert_cast(dest); - col_low.insertData(reinterpret_cast(&uuid), sizeof(uuid)); - } - else - { - assert_cast(dest).insert(uuid); - } - return true; - } - }; - - template - class DecimalNode : public Node - { - public: - explicit DecimalNode(DataTypePtr data_type_) : data_type(data_type_) {} - bool insertResultToColumn(IColumn & dest, const Element & element) override - { - const auto * type = assert_cast *>(data_type.get()); - - DecimalType value{}; - - switch (element.type()) - { - case ElementType::DOUBLE: - value = convertToDecimal, DataTypeDecimal>( - element.getDouble(), type->getScale()); - break; - case ElementType::UINT64: - value = convertToDecimal, DataTypeDecimal>( - element.getUInt64(), type->getScale()); - break; - case ElementType::INT64: - value = convertToDecimal, DataTypeDecimal>( - element.getInt64(), type->getScale()); - break; - case ElementType::STRING: { - auto rb = ReadBufferFromMemory{element.getString()}; - if (!SerializationDecimal::tryReadText(value, rb, DecimalUtils::max_precision, type->getScale())) - return false; - break; - } - default: - return false; - } - - assert_cast &>(dest).insertValue(value); - return true; - } - - private: - DataTypePtr data_type; - }; - - class StringNode : public Node - { - public: - bool insertResultToColumn(IColumn & dest, const Element & element) override - { - return JSONExtractStringImpl::insertResultToColumn(dest, element, {}); - } - }; - - class FixedStringNode : public Node - { - public: - bool insertResultToColumn(IColumn & dest, const Element & element) override - { - if (element.isNull()) - return false; - - if (!element.isString()) - return JSONExtractRawImpl::insertResultToFixedStringColumn(dest, element, {}); - - auto str = element.getString(); - auto & col_str = assert_cast(dest); - if (str.size() > col_str.getN()) - return false; - col_str.insertData(str.data(), str.size()); - - return true; - } - }; - - template - class EnumNode : public Node - { - public: - explicit EnumNode(const std::vector> & name_value_pairs_) : name_value_pairs(name_value_pairs_) - { - for (const auto & name_value_pair : name_value_pairs) - { - name_to_value_map.emplace(name_value_pair.first, name_value_pair.second); - only_values.emplace(name_value_pair.second); - } - } - - bool insertResultToColumn(IColumn & dest, const Element & element) override - { - auto & col_vec = assert_cast &>(dest); - - if (element.isInt64()) - { - Type value; - if (!accurate::convertNumeric(element.getInt64(), value) || !only_values.contains(value)) - return false; - col_vec.insertValue(value); - return true; - } - - if (element.isUInt64()) - { - Type value; - if (!accurate::convertNumeric(element.getUInt64(), value) || !only_values.contains(value)) - return false; - col_vec.insertValue(value); - return true; - } - - if (element.isString()) - { - auto value = name_to_value_map.find(element.getString()); - if (value == name_to_value_map.end()) - return false; - col_vec.insertValue(value->second); - return true; - } - - return false; - } - - private: - std::vector> name_value_pairs; - std::unordered_map name_to_value_map; - std::unordered_set only_values; - }; - - class NullableNode : public Node - { - public: - explicit NullableNode(std::unique_ptr nested_) : nested(std::move(nested_)) {} - - bool insertResultToColumn(IColumn & dest, const Element & element) override - { - if (dest.getDataType() == TypeIndex::LowCardinality) - { - /// We do not need to handle nullability in that case - /// because nested node handles LowCardinality columns and will call proper overload of `insertData` - return nested->insertResultToColumn(dest, element); - } - - ColumnNullable & col_null = assert_cast(dest); - if (!nested->insertResultToColumn(col_null.getNestedColumn(), element)) - return false; - col_null.getNullMapColumn().insertValue(0); - return true; - } - - private: - std::unique_ptr nested; - }; - - class ArrayNode : public Node - { - public: - explicit ArrayNode(std::unique_ptr nested_) : nested(std::move(nested_)) {} - - bool insertResultToColumn(IColumn & dest, const Element & element) override - { - if (!element.isArray()) - return false; - - auto array = element.getArray(); - - ColumnArray & col_arr = assert_cast(dest); - auto & data = col_arr.getData(); - size_t old_size = data.size(); - bool were_valid_elements = false; - - for (auto value : array) - { - if (nested->insertResultToColumn(data, value)) - were_valid_elements = true; - else - data.insertDefault(); - } - - if (!were_valid_elements) - { - data.popBack(data.size() - old_size); - return false; - } - - col_arr.getOffsets().push_back(data.size()); - return true; - } - - private: - std::unique_ptr nested; - }; - - class TupleNode : public Node - { - public: - TupleNode(std::vector> nested_, const std::vector & explicit_names_) : nested(std::move(nested_)), explicit_names(explicit_names_) - { - for (size_t i = 0; i != explicit_names.size(); ++i) - name_to_index_map.emplace(explicit_names[i], i); - } - - bool insertResultToColumn(IColumn & dest, const Element & element) override - { - ColumnTuple & tuple = assert_cast(dest); - size_t old_size = dest.size(); - bool were_valid_elements = false; - - auto set_size = [&](size_t size) - { - for (size_t i = 0; i != tuple.tupleSize(); ++i) - { - auto & col = tuple.getColumn(i); - if (col.size() != size) - { - if (col.size() > size) - col.popBack(col.size() - size); - else - while (col.size() < size) - col.insertDefault(); - } - } - }; - - if (element.isArray()) - { - auto array = element.getArray(); - auto it = array.begin(); - - for (size_t index = 0; (index != nested.size()) && (it != array.end()); ++index) - { - if (nested[index]->insertResultToColumn(tuple.getColumn(index), *it++)) - were_valid_elements = true; - else - tuple.getColumn(index).insertDefault(); - } - - set_size(old_size + static_cast(were_valid_elements)); - return were_valid_elements; - } - - if (element.isObject()) - { - auto object = element.getObject(); - if (name_to_index_map.empty()) - { - auto it = object.begin(); - for (size_t index = 0; (index != nested.size()) && (it != object.end()); ++index) - { - if (nested[index]->insertResultToColumn(tuple.getColumn(index), (*it++).second)) - were_valid_elements = true; - else - tuple.getColumn(index).insertDefault(); - } - } - else - { - for (const auto & [key, value] : object) - { - auto index = name_to_index_map.find(key); - if (index != name_to_index_map.end()) - { - if (nested[index->second]->insertResultToColumn(tuple.getColumn(index->second), value)) - were_valid_elements = true; - } - } - } - - set_size(old_size + static_cast(were_valid_elements)); - return were_valid_elements; - } - - return false; - } - - private: - std::vector> nested; - std::vector explicit_names; - std::unordered_map name_to_index_map; - }; - - class MapNode : public Node - { - public: - MapNode(std::unique_ptr key_, std::unique_ptr value_) : key(std::move(key_)), value(std::move(value_)) { } - - bool insertResultToColumn(IColumn & dest, const Element & element) override - { - if (!element.isObject()) - return false; - - ColumnMap & map_col = assert_cast(dest); - auto & offsets = map_col.getNestedColumn().getOffsets(); - auto & tuple_col = map_col.getNestedData(); - auto & key_col = tuple_col.getColumn(0); - auto & value_col = tuple_col.getColumn(1); - size_t old_size = tuple_col.size(); - - auto object = element.getObject(); - auto it = object.begin(); - for (; it != object.end(); ++it) - { - auto pair = *it; - - /// Insert key - key_col.insertData(pair.first.data(), pair.first.size()); - - /// Insert value - if (!value->insertResultToColumn(value_col, pair.second)) - value_col.insertDefault(); - } - - offsets.push_back(old_size + object.size()); - return true; - } - - private: - std::unique_ptr key; - std::unique_ptr value; - }; - - class VariantNode : public Node - { - public: - VariantNode(std::vector> variant_nodes_, std::vector order_) : variant_nodes(std::move(variant_nodes_)), order(std::move(order_)) { } - - bool insertResultToColumn(IColumn & dest, const Element & element) override - { - auto & column_variant = assert_cast(dest); - for (size_t i : order) - { - auto & variant = column_variant.getVariantByGlobalDiscriminator(i); - if (variant_nodes[i]->insertResultToColumn(variant, element)) - { - column_variant.getLocalDiscriminators().push_back(column_variant.localDiscriminatorByGlobal(i)); - column_variant.getOffsets().push_back(variant.size() - 1); - return true; - } - } - - return false; - } - - private: - std::vector> variant_nodes; - /// Order in which we should try variants nodes. - /// For example, String should be always the last one. - std::vector order; - }; - - static std::unique_ptr build(const char * function_name, const DataTypePtr & type) - { - switch (type->getTypeId()) - { - case TypeIndex::UInt8: return std::make_unique>(); - case TypeIndex::UInt16: return std::make_unique>(); - case TypeIndex::UInt32: return std::make_unique>(); - case TypeIndex::UInt64: return std::make_unique>(); - case TypeIndex::UInt128: return std::make_unique>(); - case TypeIndex::UInt256: return std::make_unique>(); - case TypeIndex::Int8: return std::make_unique>(); - case TypeIndex::Int16: return std::make_unique>(); - case TypeIndex::Int32: return std::make_unique>(); - case TypeIndex::Int64: return std::make_unique>(); - case TypeIndex::Int128: return std::make_unique>(); - case TypeIndex::Int256: return std::make_unique>(); - case TypeIndex::Float32: return std::make_unique>(); - case TypeIndex::Float64: return std::make_unique>(); - case TypeIndex::String: return std::make_unique(); - case TypeIndex::FixedString: return std::make_unique(); - case TypeIndex::UUID: return std::make_unique(); - case TypeIndex::LowCardinality: - { - // The low cardinality case is treated in two different ways: - // For FixedString type, an especial class is implemented for inserting the data in the destination column, - // as the string length must be passed in order to check and pad the incoming data. - // For the rest of low cardinality types, the insertion is done in their corresponding class, adapting the data - // as needed for the insertData function of the ColumnLowCardinality. - auto dictionary_type = typeid_cast(type.get())->getDictionaryType(); - if ((*dictionary_type).getTypeId() == TypeIndex::FixedString) - { - auto fixed_length = typeid_cast(dictionary_type.get())->getN(); - return std::make_unique(fixed_length); - } - return build(function_name, dictionary_type); - } - case TypeIndex::Decimal256: return std::make_unique>(type); - case TypeIndex::Decimal128: return std::make_unique>(type); - case TypeIndex::Decimal64: return std::make_unique>(type); - case TypeIndex::Decimal32: return std::make_unique>(type); - case TypeIndex::Enum8: - return std::make_unique>(static_cast(*type).getValues()); - case TypeIndex::Enum16: - return std::make_unique>(static_cast(*type).getValues()); - case TypeIndex::Nullable: - { - return std::make_unique(build(function_name, static_cast(*type).getNestedType())); - } - case TypeIndex::Array: - { - return std::make_unique(build(function_name, static_cast(*type).getNestedType())); - } - case TypeIndex::Tuple: - { - const auto & tuple = static_cast(*type); - const auto & tuple_elements = tuple.getElements(); - std::vector> elements; - elements.reserve(tuple_elements.size()); - for (const auto & tuple_element : tuple_elements) - elements.emplace_back(build(function_name, tuple_element)); - return std::make_unique(std::move(elements), tuple.haveExplicitNames() ? tuple.getElementNames() : Strings{}); - } - case TypeIndex::Map: - { - const auto & map_type = static_cast(*type); - const auto & key_type = map_type.getKeyType(); - if (!isString(removeLowCardinality(key_type))) - throw Exception( - ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT, - "Function {} doesn't support the return type schema: {} with key type not String", - String(function_name), - type->getName()); - - const auto & value_type = map_type.getValueType(); - return std::make_unique(build(function_name, key_type), build(function_name, value_type)); - } - case TypeIndex::Variant: - { - const auto & variant_type = static_cast(*type); - const auto & variants = variant_type.getVariants(); - std::vector> variant_nodes; - variant_nodes.reserve(variants.size()); - for (const auto & variant : variants) - variant_nodes.push_back(build(function_name, variant)); - return std::make_unique(std::move(variant_nodes), SerializationVariant::getVariantsDeserializeTextOrder(variants)); - } - default: - throw Exception(ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT, - "Function {} doesn't support the return type schema: {}", - String(function_name), type->getName()); - } - } -}; - - -template -class JSONExtractImpl -{ -public: - using Element = typename JSONParser::Element; - - static DataTypePtr getReturnType(const char * function_name, const ColumnsWithTypeAndName & arguments) - { - if (arguments.size() < 2) - throw Exception(ErrorCodes::NUMBER_OF_ARGUMENTS_DOESNT_MATCH, "Function {} requires at least two arguments", String(function_name)); - - const auto & col = arguments.back(); - const auto * col_type_const = typeid_cast(col.column.get()); - if (!col_type_const || !isString(col.type)) - throw Exception(ErrorCodes::ILLEGAL_COLUMN, - "The last argument of function {} should " - "be a constant string specifying the return data type, illegal value: {}", - String(function_name), col.name); - - return DataTypeFactory::instance().get(col_type_const->getValue()); - } - - static size_t getNumberOfIndexArguments(const ColumnsWithTypeAndName & arguments) { return arguments.size() - 2; } - - void prepare(const char * function_name, const ColumnsWithTypeAndName &, const DataTypePtr & result_type) - { - extract_tree = JSONExtractTree::build(function_name, result_type); - } - - bool insertResultToColumn(IColumn & dest, const Element & element, std::string_view) - { - return extract_tree->insertResultToColumn(dest, element); - } - -protected: - std::unique_ptr::Node> extract_tree; -}; - - -template -class JSONExtractKeysAndValuesImpl -{ -public: - using Element = typename JSONParser::Element; - - static DataTypePtr getReturnType(const char * function_name, const ColumnsWithTypeAndName & arguments) - { - if (arguments.size() < 2) - throw Exception(ErrorCodes::NUMBER_OF_ARGUMENTS_DOESNT_MATCH, "Function {} requires at least two arguments", String(function_name)); - - const auto & col = arguments.back(); - const auto * col_type_const = typeid_cast(col.column.get()); - if (!col_type_const || !isString(col.type)) - throw Exception(ErrorCodes::ILLEGAL_COLUMN, - "The last argument of function {} should " - "be a constant string specifying the values' data type, illegal value: {}", - String(function_name), col.name); - - DataTypePtr key_type = std::make_unique(); - DataTypePtr value_type = DataTypeFactory::instance().get(col_type_const->getValue()); - DataTypePtr tuple_type = std::make_unique(DataTypes{key_type, value_type}); - return std::make_unique(tuple_type); - } - - static size_t getNumberOfIndexArguments(const ColumnsWithTypeAndName & arguments) { return arguments.size() - 2; } - - void prepare(const char * function_name, const ColumnsWithTypeAndName &, const DataTypePtr & result_type) - { - const auto tuple_type = typeid_cast(result_type.get())->getNestedType(); - const auto value_type = typeid_cast(tuple_type.get())->getElements()[1]; - extract_tree = JSONExtractTree::build(function_name, value_type); - } - - bool insertResultToColumn(IColumn & dest, const Element & element, std::string_view) - { - if (!element.isObject()) - return false; - - auto object = element.getObject(); - - auto & col_arr = assert_cast(dest); - auto & col_tuple = assert_cast(col_arr.getData()); - size_t old_size = col_tuple.size(); - auto & col_key = assert_cast(col_tuple.getColumn(0)); - auto & col_value = col_tuple.getColumn(1); - - for (const auto & [key, value] : object) - { - if (extract_tree->insertResultToColumn(col_value, value)) - col_key.insertData(key.data(), key.size()); - } - - if (col_tuple.size() == old_size) - return false; - - col_arr.getOffsets().push_back(col_tuple.size()); - return true; - } - -private: - std::unique_ptr::Node> extract_tree; -}; - - -template -class JSONExtractRawImpl -{ -public: - using Element = typename JSONParser::Element; - - static DataTypePtr getReturnType(const char *, const ColumnsWithTypeAndName &) - { - return std::make_shared(); - } - - static size_t getNumberOfIndexArguments(const ColumnsWithTypeAndName & arguments) { return arguments.size() - 1; } - - static bool insertResultToColumn(IColumn & dest, const Element & element, std::string_view) - { - if (dest.getDataType() == TypeIndex::LowCardinality) - { - ColumnString::Chars chars; - WriteBufferFromVector buf(chars, AppendModeTag()); - traverse(element, buf); - buf.finalize(); - assert_cast(dest).insertData(reinterpret_cast(chars.data()), chars.size()); - } - else - { - ColumnString & col_str = assert_cast(dest); - auto & chars = col_str.getChars(); - WriteBufferFromVector buf(chars, AppendModeTag()); - traverse(element, buf); - buf.finalize(); - chars.push_back(0); - col_str.getOffsets().push_back(chars.size()); - } - return true; - } - - // We use insertResultToFixedStringColumn in case we are inserting raw data in a FixedString column - static bool insertResultToFixedStringColumn(IColumn & dest, const Element & element, std::string_view) - { - ColumnFixedString::Chars chars; - WriteBufferFromVector buf(chars, AppendModeTag()); - traverse(element, buf); - buf.finalize(); - - auto & col_str = assert_cast(dest); - - if (chars.size() > col_str.getN()) - return false; - - chars.resize_fill(col_str.getN()); - col_str.insertData(reinterpret_cast(chars.data()), chars.size()); - - - return true; - } - - // We use insertResultToLowCardinalityFixedStringColumn in case we are inserting raw data in a Low Cardinality FixedString column - static bool insertResultToLowCardinalityFixedStringColumn(IColumn & dest, const Element & element, size_t fixed_length) - { - if (element.getObject().size() > fixed_length) - return false; - - ColumnFixedString::Chars chars; - WriteBufferFromVector buf(chars, AppendModeTag()); - traverse(element, buf); - buf.finalize(); - - if (chars.size() > fixed_length) - return false; - chars.resize_fill(fixed_length); - assert_cast(dest).insertData(reinterpret_cast(chars.data()), chars.size()); - - return true; - } - -private: - static void traverse(const Element & element, WriteBuffer & buf) - { - if (element.isInt64()) - { - writeIntText(element.getInt64(), buf); - return; - } - if (element.isUInt64()) - { - writeIntText(element.getUInt64(), buf); - return; - } - if (element.isDouble()) - { - writeFloatText(element.getDouble(), buf); - return; - } - if (element.isBool()) - { - if (element.getBool()) - writeCString("true", buf); - else - writeCString("false", buf); - return; - } - if (element.isString()) - { - writeJSONString(element.getString(), buf, formatSettings()); - return; - } - if (element.isArray()) - { - writeChar('[', buf); - bool need_comma = false; - for (auto value : element.getArray()) - { - if (std::exchange(need_comma, true)) - writeChar(',', buf); - traverse(value, buf); - } - writeChar(']', buf); - return; - } - if (element.isObject()) - { - writeChar('{', buf); - bool need_comma = false; - for (auto [key, value] : element.getObject()) - { - if (std::exchange(need_comma, true)) - writeChar(',', buf); - writeJSONString(key, buf, formatSettings()); - writeChar(':', buf); - traverse(value, buf); - } - writeChar('}', buf); - return; - } - if (element.isNull()) - { - writeCString("null", buf); - return; - } - } - - static const FormatSettings & formatSettings() - { - static const FormatSettings the_instance = [] - { - FormatSettings settings; - settings.json.escape_forward_slashes = false; - return settings; - }(); - return the_instance; - } -}; - - -template -class JSONExtractArrayRawImpl -{ -public: - using Element = typename JSONParser::Element; - - static DataTypePtr getReturnType(const char *, const ColumnsWithTypeAndName &) - { - return std::make_shared(std::make_shared()); - } - - static size_t getNumberOfIndexArguments(const ColumnsWithTypeAndName & arguments) { return arguments.size() - 1; } - - static bool insertResultToColumn(IColumn & dest, const Element & element, std::string_view) - { - if (!element.isArray()) - return false; - - auto array = element.getArray(); - ColumnArray & col_res = assert_cast(dest); - - for (auto value : array) - JSONExtractRawImpl::insertResultToColumn(col_res.getData(), value, {}); - - col_res.getOffsets().push_back(col_res.getOffsets().back() + array.size()); - return true; - } -}; - - -template -class JSONExtractKeysAndValuesRawImpl -{ -public: - using Element = typename JSONParser::Element; - - static DataTypePtr getReturnType(const char *, const ColumnsWithTypeAndName &) - { - DataTypePtr string_type = std::make_unique(); - DataTypePtr tuple_type = std::make_unique(DataTypes{string_type, string_type}); - return std::make_unique(tuple_type); - } - - static size_t getNumberOfIndexArguments(const ColumnsWithTypeAndName & arguments) { return arguments.size() - 1; } - - bool insertResultToColumn(IColumn & dest, const Element & element, std::string_view) - { - if (!element.isObject()) - return false; - - auto object = element.getObject(); - - auto & col_arr = assert_cast(dest); - auto & col_tuple = assert_cast(col_arr.getData()); - auto & col_key = assert_cast(col_tuple.getColumn(0)); - auto & col_value = assert_cast(col_tuple.getColumn(1)); - - for (const auto & [key, value] : object) - { - col_key.insertData(key.data(), key.size()); - JSONExtractRawImpl::insertResultToColumn(col_value, value, {}); - } - - col_arr.getOffsets().push_back(col_arr.getOffsets().back() + object.size()); - return true; - } -}; - -template -class JSONExtractKeysImpl -{ -public: - using Element = typename JSONParser::Element; - - static DataTypePtr getReturnType(const char *, const ColumnsWithTypeAndName &) - { - return std::make_unique(std::make_shared()); - } - - static size_t getNumberOfIndexArguments(const ColumnsWithTypeAndName & arguments) { return arguments.size() - 1; } - - bool insertResultToColumn(IColumn & dest, const Element & element, std::string_view) - { - if (!element.isObject()) - return false; - - auto object = element.getObject(); - - ColumnArray & col_res = assert_cast(dest); - auto & col_key = assert_cast(col_res.getData()); - - for (const auto & [key, value] : object) - { - col_key.insertData(key.data(), key.size()); - } - - col_res.getOffsets().push_back(col_res.getOffsets().back() + object.size()); - return true; - } -}; - -} +//#pragma once +// +//#include +//#include +// +//#include +// +//#include +// +//#include +//#include +//#include +// +//#include +////#include +// +//#include +//#include +//#include +//#include +//#include +//#include +//#include +//#include +//#include +//#include +// +//#include +//#include +//#include +//#include +//#include +//#include +//#include +//#include +//#include +//#include +//#include +//#include +//#include +//#include +//#include +//#include +// +//#include +//#include +//#include +//#include +//#include +// +//#include +//#include +// +// +//#include "config.h" +// +// +//namespace DB +//{ +// +//namespace ErrorCodes +//{ +// extern const int ILLEGAL_TYPE_OF_ARGUMENT; +// extern const int ILLEGAL_COLUMN; +// extern const int NUMBER_OF_ARGUMENTS_DOESNT_MATCH; +//} +// +//template +//concept HasIndexOperator = requires (T t) +//{ +// t[0]; +//}; +// +///// Functions to parse JSONs and extract values from it. +///// The first argument of all these functions gets a JSON, +///// after that there are any number of arguments specifying path to a desired part from the JSON's root. +///// For example, +///// select JSONExtractInt('{"a": "hello", "b": [-100, 200.0, 300]}', 'b', 1) = -100 +// +//class FunctionJSONHelpers +//{ +//public: +// template typename Impl, class JSONParser> +// class Executor +// { +// public: +// static ColumnPtr run(const ColumnsWithTypeAndName & arguments, const DataTypePtr & result_type, size_t input_rows_count) +// { +// MutableColumnPtr to{result_type->createColumn()}; +// to->reserve(input_rows_count); +// +// if (arguments.empty()) +// throw Exception(ErrorCodes::NUMBER_OF_ARGUMENTS_DOESNT_MATCH, "Function {} requires at least one argument", String(Name::name)); +// +// const auto & first_column = arguments[0]; +// if (!isString(first_column.type)) +// throw Exception(ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT, +// "The first argument of function {} should be a string containing JSON, illegal type: " +// "{}", String(Name::name), first_column.type->getName()); +// +// const ColumnPtr & arg_json = first_column.column; +// const auto * col_json_const = typeid_cast(arg_json.get()); +// const auto * col_json_string +// = typeid_cast(col_json_const ? col_json_const->getDataColumnPtr().get() : arg_json.get()); +// +// if (!col_json_string) +// throw Exception(ErrorCodes::ILLEGAL_COLUMN, "Illegal column {}", arg_json->getName()); +// +// const ColumnString::Chars & chars = col_json_string->getChars(); +// const ColumnString::Offsets & offsets = col_json_string->getOffsets(); +// +// size_t num_index_arguments = Impl::getNumberOfIndexArguments(arguments); +// std::vector moves = prepareMoves(Name::name, arguments, 1, num_index_arguments); +// +// /// Preallocate memory in parser if necessary. +// JSONParser parser; +// if constexpr (has_member_function_reserve::value) +// { +// size_t max_size = calculateMaxSize(offsets); +// if (max_size) +// parser.reserve(max_size); +// } +// +// Impl impl; +// +// /// prepare() does Impl-specific preparation before handling each row. +// if constexpr (has_member_function_prepare::*)(const char *, const ColumnsWithTypeAndName &, const DataTypePtr &)>::value) +// impl.prepare(Name::name, arguments, result_type); +// +// using Element = typename JSONParser::Element; +// +// Element document; +// bool document_ok = false; +// if (col_json_const) +// { +// std::string_view json{reinterpret_cast(chars.data()), offsets[0] - 1}; +// document_ok = parser.parse(json, document); +// } +// +// for (const auto i : collections::range(0, input_rows_count)) +// { +// if (!col_json_const) +// { +// std::string_view json{reinterpret_cast(&chars[offsets[i - 1]]), offsets[i] - offsets[i - 1] - 1}; +// document_ok = parser.parse(json, document); +// } +// +// bool added_to_column = false; +// if (document_ok) +// { +// /// Perform moves. +// Element element; +// std::string_view last_key; +// bool moves_ok = performMoves(arguments, i, document, moves, element, last_key); +// +// if (moves_ok) +// added_to_column = impl.insertResultToColumn(*to, element, last_key); +// } +// +// /// We add default value (=null or zero) if something goes wrong, we don't throw exceptions in these JSON functions. +// if (!added_to_column) +// to->insertDefault(); +// } +// return to; +// } +// }; +// +//private: +// BOOST_TTI_HAS_MEMBER_FUNCTION(reserve) +// BOOST_TTI_HAS_MEMBER_FUNCTION(prepare) +// +// /// Represents a move of a JSON iterator described by a single argument passed to a JSON function. +// /// For example, the call JSONExtractInt('{"a": "hello", "b": [-100, 200.0, 300]}', 'b', 1) +// /// contains two moves: {MoveType::ConstKey, "b"} and {MoveType::ConstIndex, 1}. +// /// Keys and indices can be nonconst, in this case they are calculated for each row. +// enum class MoveType : uint8_t +// { +// Key, +// Index, +// ConstKey, +// ConstIndex, +// }; +// +// struct Move +// { +// explicit Move(MoveType type_, size_t index_ = 0) : type(type_), index(index_) {} +// Move(MoveType type_, const String & key_) : type(type_), key(key_) {} +// MoveType type; +// size_t index = 0; +// String key; +// }; +// +// static std::vector prepareMoves( +// const char * function_name, +// const ColumnsWithTypeAndName & columns, +// size_t first_index_argument, +// size_t num_index_arguments) +// { +// std::vector moves; +// moves.reserve(num_index_arguments); +// for (const auto i : collections::range(first_index_argument, first_index_argument + num_index_arguments)) +// { +// const auto & column = columns[i]; +// if (!isString(column.type) && !isNativeInteger(column.type)) +// throw Exception(ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT, +// "The argument {} of function {} should be a string specifying key " +// "or an integer specifying index, illegal type: {}", +// std::to_string(i + 1), String(function_name), column.type->getName()); +// +// if (column.column && isColumnConst(*column.column)) +// { +// const auto & column_const = assert_cast(*column.column); +// if (isString(column.type)) +// moves.emplace_back(MoveType::ConstKey, column_const.getValue()); +// else +// moves.emplace_back(MoveType::ConstIndex, column_const.getInt(0)); +// } +// else +// { +// if (isString(column.type)) +// moves.emplace_back(MoveType::Key, ""); +// else +// moves.emplace_back(MoveType::Index, 0); +// } +// } +// return moves; +// } +// +// +// /// Performs moves of types MoveType::Index and MoveType::ConstIndex. +// template +// static bool performMoves(const ColumnsWithTypeAndName & arguments, size_t row, +// const typename JSONParser::Element & document, const std::vector & moves, +// typename JSONParser::Element & element, std::string_view & last_key) +// { +// typename JSONParser::Element res_element = document; +// std::string_view key; +// +// for (size_t j = 0; j != moves.size(); ++j) +// { +// switch (moves[j].type) +// { +// case MoveType::ConstIndex: +// { +// if (!moveToElementByIndex(res_element, static_cast(moves[j].index), key)) +// return false; +// break; +// } +// case MoveType::ConstKey: +// { +// key = moves[j].key; +// if (!moveToElementByKey(res_element, key)) +// return false; +// break; +// } +// case MoveType::Index: +// { +// Int64 index = (*arguments[j + 1].column)[row].get(); +// if (!moveToElementByIndex(res_element, static_cast(index), key)) +// return false; +// break; +// } +// case MoveType::Key: +// { +// key = arguments[j + 1].column->getDataAt(row).toView(); +// if (!moveToElementByKey(res_element, key)) +// return false; +// break; +// } +// } +// } +// +// element = res_element; +// last_key = key; +// return true; +// } +// +// template +// static bool moveToElementByIndex(typename JSONParser::Element & element, int index, std::string_view & out_key) +// { +// if (element.isArray()) +// { +// auto array = element.getArray(); +// if (index >= 0) +// --index; +// else +// index += array.size(); +// +// if (static_cast(index) >= array.size()) +// return false; +// element = array[index]; +// out_key = {}; +// return true; +// } +// +// if constexpr (HasIndexOperator) +// { +// if (element.isObject()) +// { +// auto object = element.getObject(); +// if (index >= 0) +// --index; +// else +// index += object.size(); +// +// if (static_cast(index) >= object.size()) +// return false; +// std::tie(out_key, element) = object[index]; +// return true; +// } +// } +// +// return {}; +// } +// +// /// Performs moves of types MoveType::Key and MoveType::ConstKey. +// template +// static bool moveToElementByKey(typename JSONParser::Element & element, std::string_view key) +// { +// if (!element.isObject()) +// return false; +// auto object = element.getObject(); +// return object.find(key, element); +// } +// +// static size_t calculateMaxSize(const ColumnString::Offsets & offsets) +// { +// size_t max_size = 0; +// for (const auto i : collections::range(0, offsets.size())) +// { +// size_t size = offsets[i] - offsets[i - 1]; +// max_size = std::max(max_size, size); +// } +// if (max_size) +// --max_size; +// return max_size; +// } +// +//}; +// +//template +//class JSONExtractImpl; +// +//template +//class JSONExtractKeysAndValuesImpl; +// +///** +//* Functions JSONExtract and JSONExtractKeysAndValues force the return type - it is specified in the last argument. +//* For example - `SELECT JSONExtract(materialize('{"a": 131231, "b": 1234}'), 'b', 'LowCardinality(FixedString(4))')` +//* But by default ClickHouse decides on its own whether the return type will be LowCardinality based on the types of +//* input arguments. +//* And for these specific functions we cannot rely on this mechanism, so these functions have their own implementation - +//* just convert all of the LowCardinality input columns to full ones, execute and wrap the resulting column in LowCardinality +//* if needed. +//*/ +//template typename Impl> +//constexpr bool functionForcesTheReturnType() +//{ +// return std::is_same_v, JSONExtractImpl> || std::is_same_v, JSONExtractKeysAndValuesImpl>; +//} +// +//template typename Impl> +//class ExecutableFunctionJSON : public IExecutableFunction +//{ +// +//public: +// explicit ExecutableFunctionJSON(const NullPresence & null_presence_, bool allow_simdjson_, const DataTypePtr & json_return_type_) +// : null_presence(null_presence_), allow_simdjson(allow_simdjson_), json_return_type(json_return_type_) +// { +// } +// +// String getName() const override { return Name::name; } +// bool useDefaultImplementationForNulls() const override { return false; } +// bool useDefaultImplementationForConstants() const override { return true; } +// bool useDefaultImplementationForLowCardinalityColumns() const override +// { +// return !functionForcesTheReturnType(); +// } +// +// ColumnPtr executeImpl(const ColumnsWithTypeAndName & arguments, const DataTypePtr & result_type, size_t input_rows_count) const override +// { +// if (null_presence.has_null_constant) +// return result_type->createColumnConstWithDefaultValue(input_rows_count); +// +// if constexpr (functionForcesTheReturnType()) +// { +// ColumnsWithTypeAndName columns_without_low_cardinality = arguments; +// +// for (auto & column : columns_without_low_cardinality) +// { +// column.column = recursiveRemoveLowCardinality(column.column); +// column.type = recursiveRemoveLowCardinality(column.type); +// } +// +// ColumnsWithTypeAndName temporary_columns = null_presence.has_nullable ? createBlockWithNestedColumns(columns_without_low_cardinality) : columns_without_low_cardinality; +// ColumnPtr temporary_result = chooseAndRunJSONParser(temporary_columns, json_return_type, input_rows_count); +// +// if (null_presence.has_nullable) +// temporary_result = wrapInNullable(temporary_result, columns_without_low_cardinality, result_type, input_rows_count); +// +// if (result_type->lowCardinality()) +// temporary_result = recursiveLowCardinalityTypeConversion(temporary_result, json_return_type, result_type); +// +// return temporary_result; +// } +// else +// { +// ColumnsWithTypeAndName temporary_columns = null_presence.has_nullable ? createBlockWithNestedColumns(arguments) : arguments; +// ColumnPtr temporary_result = chooseAndRunJSONParser(temporary_columns, json_return_type, input_rows_count); +// +// if (null_presence.has_nullable) +// temporary_result = wrapInNullable(temporary_result, arguments, result_type, input_rows_count); +// +// if (result_type->lowCardinality()) +// temporary_result = recursiveLowCardinalityTypeConversion(temporary_result, json_return_type, result_type); +// +// return temporary_result; +// } +// } +// +//private: +// +// ColumnPtr +// chooseAndRunJSONParser(const ColumnsWithTypeAndName & arguments, const DataTypePtr & result_type, size_t input_rows_count) const +// { +//#if USE_SIMDJSON +// if (allow_simdjson) +// return FunctionJSONHelpers::Executor::run(arguments, result_type, input_rows_count); +//#endif +// +//#if USE_RAPIDJSON +// return FunctionJSONHelpers::Executor::run(arguments, result_type, input_rows_count); +//#else +// return FunctionJSONHelpers::Executor::run(arguments, result_type, input_rows_count); +//#endif +// } +// +// NullPresence null_presence; +// bool allow_simdjson; +// DataTypePtr json_return_type; +//}; +// +// +//template typename Impl> +//class FunctionBaseFunctionJSON : public IFunctionBase +//{ +//public: +// explicit FunctionBaseFunctionJSON( +// const NullPresence & null_presence_, +// bool allow_simdjson_, +// DataTypes argument_types_, +// DataTypePtr return_type_, +// DataTypePtr json_return_type_) +// : null_presence(null_presence_) +// , allow_simdjson(allow_simdjson_) +// , argument_types(std::move(argument_types_)) +// , return_type(std::move(return_type_)) +// , json_return_type(std::move(json_return_type_)) +// { +// } +// +// String getName() const override { return Name::name; } +// +// const DataTypes & getArgumentTypes() const override +// { +// return argument_types; +// } +// +// const DataTypePtr & getResultType() const override +// { +// return return_type; +// } +// +// bool isSuitableForShortCircuitArgumentsExecution(const DataTypesWithConstInfo & /*arguments*/) const override { return true; } +// +// ExecutableFunctionPtr prepare(const ColumnsWithTypeAndName &) const override +// { +// return std::make_unique>(null_presence, allow_simdjson, json_return_type); +// } +// +//private: +// NullPresence null_presence; +// bool allow_simdjson; +// DataTypes argument_types; +// DataTypePtr return_type; +// DataTypePtr json_return_type; +//}; +// +///// We use IFunctionOverloadResolver instead of IFunction to handle non-default NULL processing. +///// Both NULL and JSON NULL should generate NULL value. If any argument is NULL, return NULL. +//template typename Impl> +//class JSONOverloadResolver : public IFunctionOverloadResolver, WithContext +//{ +//public: +// static constexpr auto name = Name::name; +// +// String getName() const override { return name; } +// +// static FunctionOverloadResolverPtr create(ContextPtr context_) +// { +// return std::make_unique(context_); +// } +// +// explicit JSONOverloadResolver(ContextPtr context_) : WithContext(context_) {} +// +// bool isVariadic() const override { return true; } +// size_t getNumberOfArguments() const override { return 0; } +// bool useDefaultImplementationForNulls() const override { return false; } +// bool useDefaultImplementationForLowCardinalityColumns() const override +// { +// return !functionForcesTheReturnType(); +// } +// +// FunctionBasePtr build(const ColumnsWithTypeAndName & arguments) const override +// { +// bool has_nothing_argument = false; +// for (const auto & arg : arguments) +// has_nothing_argument |= isNothing(arg.type); +// +// DataTypePtr json_return_type = Impl::getReturnType(Name::name, createBlockWithNestedColumns(arguments)); +// NullPresence null_presence = getNullPresense(arguments); +// DataTypePtr return_type; +// if (has_nothing_argument) +// return_type = std::make_shared(); +// else if (null_presence.has_null_constant) +// return_type = makeNullable(std::make_shared()); +// else if (null_presence.has_nullable) +// return_type = makeNullable(json_return_type); +// else +// return_type = json_return_type; +// +// /// Top-level LowCardinality columns are processed outside JSON parser. +// json_return_type = removeLowCardinality(json_return_type); +// +// DataTypes argument_types; +// argument_types.reserve(arguments.size()); +// for (const auto & argument : arguments) +// argument_types.emplace_back(argument.type); +// return std::make_unique>( +// null_presence, getContext()->getSettingsRef().allow_simdjson, argument_types, return_type, json_return_type); +// } +//}; +// +//struct NameJSONHas { static constexpr auto name{"JSONHas"}; }; +//struct NameIsValidJSON { static constexpr auto name{"isValidJSON"}; }; +//struct NameJSONLength { static constexpr auto name{"JSONLength"}; }; +//struct NameJSONKey { static constexpr auto name{"JSONKey"}; }; +//struct NameJSONType { static constexpr auto name{"JSONType"}; }; +//struct NameJSONExtractInt { static constexpr auto name{"JSONExtractInt"}; }; +//struct NameJSONExtractUInt { static constexpr auto name{"JSONExtractUInt"}; }; +//struct NameJSONExtractFloat { static constexpr auto name{"JSONExtractFloat"}; }; +//struct NameJSONExtractBool { static constexpr auto name{"JSONExtractBool"}; }; +//struct NameJSONExtractString { static constexpr auto name{"JSONExtractString"}; }; +//struct NameJSONExtract { static constexpr auto name{"JSONExtract"}; }; +//struct NameJSONExtractKeysAndValues { static constexpr auto name{"JSONExtractKeysAndValues"}; }; +//struct NameJSONExtractRaw { static constexpr auto name{"JSONExtractRaw"}; }; +//struct NameJSONExtractArrayRaw { static constexpr auto name{"JSONExtractArrayRaw"}; }; +//struct NameJSONExtractKeysAndValuesRaw { static constexpr auto name{"JSONExtractKeysAndValuesRaw"}; }; +//struct NameJSONExtractKeys { static constexpr auto name{"JSONExtractKeys"}; }; +// +// +//template +//class JSONHasImpl +//{ +//public: +// using Element = typename JSONParser::Element; +// +// static DataTypePtr getReturnType(const char *, const ColumnsWithTypeAndName &) { return std::make_shared(); } +// +// static size_t getNumberOfIndexArguments(const ColumnsWithTypeAndName & arguments) { return arguments.size() - 1; } +// +// static bool insertResultToColumn(IColumn & dest, const Element &, std::string_view) +// { +// ColumnVector & col_vec = assert_cast &>(dest); +// col_vec.insertValue(1); +// return true; +// } +//}; +// +// +//template +//class IsValidJSONImpl +//{ +//public: +// using Element = typename JSONParser::Element; +// +// static DataTypePtr getReturnType(const char * function_name, const ColumnsWithTypeAndName & arguments) +// { +// if (arguments.size() != 1) +// { +// /// IsValidJSON() shouldn't get parameters other than JSON. +// throw Exception(ErrorCodes::NUMBER_OF_ARGUMENTS_DOESNT_MATCH, "Function {} needs exactly one argument", +// String(function_name)); +// } +// return std::make_shared(); +// } +// +// static size_t getNumberOfIndexArguments(const ColumnsWithTypeAndName &) { return 0; } +// +// static bool insertResultToColumn(IColumn & dest, const Element &, std::string_view) +// { +// /// This function is called only if JSON is valid. +// /// If JSON isn't valid then `FunctionJSON::Executor::run()` adds default value (=zero) to `dest` without calling this function. +// ColumnVector & col_vec = assert_cast &>(dest); +// col_vec.insertValue(1); +// return true; +// } +//}; +// +// +//template +//class JSONLengthImpl +//{ +//public: +// using Element = typename JSONParser::Element; +// +// static DataTypePtr getReturnType(const char *, const ColumnsWithTypeAndName &) +// { +// return std::make_shared(); +// } +// +// static size_t getNumberOfIndexArguments(const ColumnsWithTypeAndName & arguments) { return arguments.size() - 1; } +// +// static bool insertResultToColumn(IColumn & dest, const Element & element, std::string_view) +// { +// size_t size; +// if (element.isArray()) +// size = element.getArray().size(); +// else if (element.isObject()) +// size = element.getObject().size(); +// else +// return false; +// +// ColumnVector & col_vec = assert_cast &>(dest); +// col_vec.insertValue(size); +// return true; +// } +//}; +// +// +//template +//class JSONKeyImpl +//{ +//public: +// using Element = typename JSONParser::Element; +// +// static DataTypePtr getReturnType(const char *, const ColumnsWithTypeAndName &) +// { +// return std::make_shared(); +// } +// +// static size_t getNumberOfIndexArguments(const ColumnsWithTypeAndName & arguments) { return arguments.size() - 1; } +// +// static bool insertResultToColumn(IColumn & dest, const Element &, std::string_view last_key) +// { +// if (last_key.empty()) +// return false; +// ColumnString & col_str = assert_cast(dest); +// col_str.insertData(last_key.data(), last_key.size()); +// return true; +// } +//}; +// +// +//template +//class JSONTypeImpl +//{ +//public: +// using Element = typename JSONParser::Element; +// +// static DataTypePtr getReturnType(const char *, const ColumnsWithTypeAndName &) +// { +// static const std::vector> values = { +// {"Array", '['}, +// {"Object", '{'}, +// {"String", '"'}, +// {"Int64", 'i'}, +// {"UInt64", 'u'}, +// {"Double", 'd'}, +// {"Bool", 'b'}, +// {"Null", 0}, /// the default value for the column. +// }; +// return std::make_shared>(values); +// } +// +// static size_t getNumberOfIndexArguments(const ColumnsWithTypeAndName & arguments) { return arguments.size() - 1; } +// +// static bool insertResultToColumn(IColumn & dest, const Element & element, std::string_view) +// { +// UInt8 type; +// switch (element.type()) +// { +// case ElementType::INT64: +// type = 'i'; +// break; +// case ElementType::UINT64: +// type = 'u'; +// break; +// case ElementType::DOUBLE: +// type = 'd'; +// break; +// case ElementType::STRING: +// type = '"'; +// break; +// case ElementType::ARRAY: +// type = '['; +// break; +// case ElementType::OBJECT: +// type = '{'; +// break; +// case ElementType::BOOL: +// type = 'b'; +// break; +// case ElementType::NULL_VALUE: +// type = 0; +// break; +// } +// +// ColumnVector & col_vec = assert_cast &>(dest); +// col_vec.insertValue(type); +// return true; +// } +//}; +// +// +//template +//class JSONExtractNumericImpl +//{ +//public: +// using Element = typename JSONParser::Element; +// +// static DataTypePtr getReturnType(const char *, const ColumnsWithTypeAndName &) +// { +// return std::make_shared>(); +// } +// +// static size_t getNumberOfIndexArguments(const ColumnsWithTypeAndName & arguments) { return arguments.size() - 1; } +// +// static bool insertResultToColumn(IColumn & dest, const Element & element, std::string_view) +// { +// NumberType value; +// +// switch (element.type()) +// { +// case ElementType::DOUBLE: +// if constexpr (std::is_floating_point_v) +// { +// /// We permit inaccurate conversion of double to float. +// /// Example: double 0.1 from JSON is not representable in float. +// /// But it will be more convenient for user to perform conversion. +// value = static_cast(element.getDouble()); +// } +// else if (!accurate::convertNumeric(element.getDouble(), value)) +// return false; +// break; +// case ElementType::UINT64: +// if (!accurate::convertNumeric(element.getUInt64(), value)) +// return false; +// break; +// case ElementType::INT64: +// if (!accurate::convertNumeric(element.getInt64(), value)) +// return false; +// break; +// case ElementType::BOOL: +// if constexpr (is_integer && convert_bool_to_integer) +// { +// value = static_cast(element.getBool()); +// break; +// } +// return false; +// case ElementType::STRING: +// { +// auto rb = ReadBufferFromMemory{element.getString()}; +// if constexpr (std::is_floating_point_v) +// { +// if (!tryReadFloatText(value, rb) || !rb.eof()) +// return false; +// } +// else +// { +// if (tryReadIntText(value, rb) && rb.eof()) +// break; +// +// /// Try to parse float and convert it to integer. +// Float64 tmp_float; +// rb.position() = rb.buffer().begin(); +// if (!tryReadFloatText(tmp_float, rb) || !rb.eof()) +// return false; +// +// if (!accurate::convertNumeric(tmp_float, value)) +// return false; +// } +// break; +// } +// default: +// return false; +// } +// +// if (dest.getDataType() == TypeIndex::LowCardinality) +// { +// ColumnLowCardinality & col_low = assert_cast(dest); +// col_low.insertData(reinterpret_cast(&value), sizeof(value)); +// } +// else +// { +// auto & col_vec = assert_cast &>(dest); +// col_vec.insertValue(value); +// } +// return true; +// } +//}; +// +// +//template +//using JSONExtractInt64Impl = JSONExtractNumericImpl; +//template +//using JSONExtractUInt64Impl = JSONExtractNumericImpl; +//template +//using JSONExtractFloat64Impl = JSONExtractNumericImpl; +// +// +//template +//class JSONExtractBoolImpl +//{ +//public: +// using Element = typename JSONParser::Element; +// +// static DataTypePtr getReturnType(const char *, const ColumnsWithTypeAndName &) +// { +// return std::make_shared(); +// } +// +// static size_t getNumberOfIndexArguments(const ColumnsWithTypeAndName & arguments) { return arguments.size() - 1; } +// +// static bool insertResultToColumn(IColumn & dest, const Element & element, std::string_view) +// { +// bool value; +// switch (element.type()) +// { +// case ElementType::BOOL: +// value = element.getBool(); +// break; +// case ElementType::INT64: +// value = element.getInt64() != 0; +// break; +// case ElementType::UINT64: +// value = element.getUInt64() != 0; +// break; +// default: +// return false; +// } +// +// auto & col_vec = assert_cast &>(dest); +// col_vec.insertValue(static_cast(value)); +// return true; +// } +//}; +// +//template +//class JSONExtractRawImpl; +// +//template +//class JSONExtractStringImpl +//{ +//public: +// using Element = typename JSONParser::Element; +// +// static DataTypePtr getReturnType(const char *, const ColumnsWithTypeAndName &) +// { +// return std::make_shared(); +// } +// +// static size_t getNumberOfIndexArguments(const ColumnsWithTypeAndName & arguments) { return arguments.size() - 1; } +// +// static bool insertResultToColumn(IColumn & dest, const Element & element, std::string_view) +// { +// if (element.isNull()) +// return false; +// +// if (!element.isString()) +// return JSONExtractRawImpl::insertResultToColumn(dest, element, {}); +// +// auto str = element.getString(); +// +// if (dest.getDataType() == TypeIndex::LowCardinality) +// { +// ColumnLowCardinality & col_low = assert_cast(dest); +// col_low.insertData(str.data(), str.size()); +// } +// else +// { +// ColumnString & col_str = assert_cast(dest); +// col_str.insertData(str.data(), str.size()); +// } +// return true; +// } +//}; +// +// +//static const JSONExtractInsertSettings & getJSONExtractInsertSettings() +//{ +// static const JSONExtractInsertSettings instance = [] +// { +// JSONExtractInsertSettings settings; +// settings.insert_null_as_default = false; +// settings.insert_default_on_invalid_elements_in_complex_types = true; +// return settings; +// }(); +// return instance; +//} +// +//template +//class JSONExtractImpl +//{ +//public: +// using Element = typename JSONParser::Element; +// +// static DataTypePtr getReturnType(const char * function_name, const ColumnsWithTypeAndName & arguments) +// { +// if (arguments.size() < 2) +// throw Exception(ErrorCodes::NUMBER_OF_ARGUMENTS_DOESNT_MATCH, "Function {} requires at least two arguments", String(function_name)); +// +// const auto & col = arguments.back(); +// const auto * col_type_const = typeid_cast(col.column.get()); +// if (!col_type_const || !isString(col.type)) +// throw Exception(ErrorCodes::ILLEGAL_COLUMN, +// "The last argument of function {} should " +// "be a constant string specifying the return data type, illegal value: {}", +// String(function_name), col.name); +// +// return DataTypeFactory::instance().get(col_type_const->getValue()); +// } +// +// static size_t getNumberOfIndexArguments(const ColumnsWithTypeAndName & arguments) { return arguments.size() - 2; } +// +// void prepare(const char * function_name, const ColumnsWithTypeAndName &, const DataTypePtr & result_type) +// { +// extract_tree = buildJSONExtractTree(result_type, function_name); +// } +// +// bool insertResultToColumn(IColumn & dest, const Element & element, std::string_view) +// { +// String error; +// return extract_tree->insertResultToColumn(dest, element, getJSONExtractInsertSettings(), error); +// } +// +//protected: +// std::unique_ptr> extract_tree; +//}; +// +// +//template +//class JSONExtractKeysAndValuesImpl +//{ +//public: +// using Element = typename JSONParser::Element; +// +// static DataTypePtr getReturnType(const char * function_name, const ColumnsWithTypeAndName & arguments) +// { +// if (arguments.size() < 2) +// throw Exception(ErrorCodes::NUMBER_OF_ARGUMENTS_DOESNT_MATCH, "Function {} requires at least two arguments", String(function_name)); +// +// const auto & col = arguments.back(); +// const auto * col_type_const = typeid_cast(col.column.get()); +// if (!col_type_const || !isString(col.type)) +// throw Exception(ErrorCodes::ILLEGAL_COLUMN, +// "The last argument of function {} should " +// "be a constant string specifying the values' data type, illegal value: {}", +// String(function_name), col.name); +// +// DataTypePtr key_type = std::make_unique(); +// DataTypePtr value_type = DataTypeFactory::instance().get(col_type_const->getValue()); +// DataTypePtr tuple_type = std::make_unique(DataTypes{key_type, value_type}); +// return std::make_unique(tuple_type); +// } +// +// static size_t getNumberOfIndexArguments(const ColumnsWithTypeAndName & arguments) { return arguments.size() - 2; } +// +// void prepare(const char * function_name, const ColumnsWithTypeAndName &, const DataTypePtr & result_type) +// { +// const auto tuple_type = typeid_cast(result_type.get())->getNestedType(); +// const auto value_type = typeid_cast(tuple_type.get())->getElements()[1]; +// extract_tree = buildJSONExtractTree(value_type, function_name); +// } +// +// bool insertResultToColumn(IColumn & dest, const Element & element, std::string_view) +// { +// if (!element.isObject()) +// return false; +// +// auto object = element.getObject(); +// +// auto & col_arr = assert_cast(dest); +// auto & col_tuple = assert_cast(col_arr.getData()); +// size_t old_size = col_tuple.size(); +// auto & col_key = assert_cast(col_tuple.getColumn(0)); +// auto & col_value = col_tuple.getColumn(1); +// +// String error; +// for (const auto & [key, value] : object) +// { +// if (extract_tree->insertResultToColumn(col_value, value, getJSONExtractInsertSettings(), error)) +// col_key.insertData(key.data(), key.size()); +// } +// +// if (col_tuple.size() == old_size) +// return false; +// +// col_arr.getOffsets().push_back(col_tuple.size()); +// return true; +// } +// +//private: +// std::unique_ptr> extract_tree; +//}; +// +// +//template +//class JSONExtractRawImpl +//{ +//public: +// using Element = typename JSONParser::Element; +// +// static DataTypePtr getReturnType(const char *, const ColumnsWithTypeAndName &) +// { +// return std::make_shared(); +// } +// +// static size_t getNumberOfIndexArguments(const ColumnsWithTypeAndName & arguments) { return arguments.size() - 1; } +// +// static bool insertResultToColumn(IColumn & dest, const Element & element, std::string_view) +// { +// if (dest.getDataType() == TypeIndex::LowCardinality) +// { +// ColumnString::Chars chars; +// WriteBufferFromVector buf(chars, AppendModeTag()); +// traverse(element, buf); +// buf.finalize(); +// assert_cast(dest).insertData(reinterpret_cast(chars.data()), chars.size()); +// } +// else +// { +// ColumnString & col_str = assert_cast(dest); +// auto & chars = col_str.getChars(); +// WriteBufferFromVector buf(chars, AppendModeTag()); +// traverse(element, buf); +// buf.finalize(); +// chars.push_back(0); +// col_str.getOffsets().push_back(chars.size()); +// } +// return true; +// } +// +// // We use insertResultToFixedStringColumn in case we are inserting raw data in a FixedString column +// static bool insertResultToFixedStringColumn(IColumn & dest, const Element & element, std::string_view) +// { +// ColumnFixedString::Chars chars; +// WriteBufferFromVector buf(chars, AppendModeTag()); +// traverse(element, buf); +// buf.finalize(); +// +// auto & col_str = assert_cast(dest); +// +// if (chars.size() > col_str.getN()) +// return false; +// +// chars.resize_fill(col_str.getN()); +// col_str.insertData(reinterpret_cast(chars.data()), chars.size()); +// +// +// return true; +// } +// +// // We use insertResultToLowCardinalityFixedStringColumn in case we are inserting raw data in a Low Cardinality FixedString column +// static bool insertResultToLowCardinalityFixedStringColumn(IColumn & dest, const Element & element, size_t fixed_length) +// { +// if (element.getObject().size() > fixed_length) +// return false; +// +// ColumnFixedString::Chars chars; +// WriteBufferFromVector buf(chars, AppendModeTag()); +// traverse(element, buf); +// buf.finalize(); +// +// if (chars.size() > fixed_length) +// return false; +// chars.resize_fill(fixed_length); +// assert_cast(dest).insertData(reinterpret_cast(chars.data()), chars.size()); +// +// return true; +// } +// +//private: +// static void traverse(const Element & element, WriteBuffer & buf) +// { +// if (element.isInt64()) +// { +// writeIntText(element.getInt64(), buf); +// return; +// } +// if (element.isUInt64()) +// { +// writeIntText(element.getUInt64(), buf); +// return; +// } +// if (element.isDouble()) +// { +// writeFloatText(element.getDouble(), buf); +// return; +// } +// if (element.isBool()) +// { +// if (element.getBool()) +// writeCString("true", buf); +// else +// writeCString("false", buf); +// return; +// } +// if (element.isString()) +// { +// writeJSONString(element.getString(), buf, formatSettings()); +// return; +// } +// if (element.isArray()) +// { +// writeChar('[', buf); +// bool need_comma = false; +// for (auto value : element.getArray()) +// { +// if (std::exchange(need_comma, true)) +// writeChar(',', buf); +// traverse(value, buf); +// } +// writeChar(']', buf); +// return; +// } +// if (element.isObject()) +// { +// writeChar('{', buf); +// bool need_comma = false; +// for (auto [key, value] : element.getObject()) +// { +// if (std::exchange(need_comma, true)) +// writeChar(',', buf); +// writeJSONString(key, buf, formatSettings()); +// writeChar(':', buf); +// traverse(value, buf); +// } +// writeChar('}', buf); +// return; +// } +// if (element.isNull()) +// { +// writeCString("null", buf); +// return; +// } +// } +// +// static const FormatSettings & formatSettings() +// { +// static const FormatSettings the_instance = [] +// { +// FormatSettings settings; +// settings.json.escape_forward_slashes = false; +// return settings; +// }(); +// return the_instance; +// } +//}; +// +// +//template +//class JSONExtractArrayRawImpl +//{ +//public: +// using Element = typename JSONParser::Element; +// +// static DataTypePtr getReturnType(const char *, const ColumnsWithTypeAndName &) +// { +// return std::make_shared(std::make_shared()); +// } +// +// static size_t getNumberOfIndexArguments(const ColumnsWithTypeAndName & arguments) { return arguments.size() - 1; } +// +// static bool insertResultToColumn(IColumn & dest, const Element & element, std::string_view) +// { +// if (!element.isArray()) +// return false; +// +// auto array = element.getArray(); +// ColumnArray & col_res = assert_cast(dest); +// +// for (auto value : array) +// JSONExtractRawImpl::insertResultToColumn(col_res.getData(), value, {}); +// +// col_res.getOffsets().push_back(col_res.getOffsets().back() + array.size()); +// return true; +// } +//}; +// +// +//template +//class JSONExtractKeysAndValuesRawImpl +//{ +//public: +// using Element = typename JSONParser::Element; +// +// static DataTypePtr getReturnType(const char *, const ColumnsWithTypeAndName &) +// { +// DataTypePtr string_type = std::make_unique(); +// DataTypePtr tuple_type = std::make_unique(DataTypes{string_type, string_type}); +// return std::make_unique(tuple_type); +// } +// +// static size_t getNumberOfIndexArguments(const ColumnsWithTypeAndName & arguments) { return arguments.size() - 1; } +// +// bool insertResultToColumn(IColumn & dest, const Element & element, std::string_view) +// { +// if (!element.isObject()) +// return false; +// +// auto object = element.getObject(); +// +// auto & col_arr = assert_cast(dest); +// auto & col_tuple = assert_cast(col_arr.getData()); +// auto & col_key = assert_cast(col_tuple.getColumn(0)); +// auto & col_value = assert_cast(col_tuple.getColumn(1)); +// +// for (const auto & [key, value] : object) +// { +// col_key.insertData(key.data(), key.size()); +// JSONExtractRawImpl::insertResultToColumn(col_value, value, {}); +// } +// +// col_arr.getOffsets().push_back(col_arr.getOffsets().back() + object.size()); +// return true; +// } +//}; +// +//template +//class JSONExtractKeysImpl +//{ +//public: +// using Element = typename JSONParser::Element; +// +// static DataTypePtr getReturnType(const char *, const ColumnsWithTypeAndName &) +// { +// return std::make_unique(std::make_shared()); +// } +// +// static size_t getNumberOfIndexArguments(const ColumnsWithTypeAndName & arguments) { return arguments.size() - 1; } +// +// bool insertResultToColumn(IColumn & dest, const Element & element, std::string_view) +// { +// if (!element.isObject()) +// return false; +// +// auto object = element.getObject(); +// +// ColumnArray & col_res = assert_cast(dest); +// auto & col_key = assert_cast(col_res.getData()); +// +// for (const auto & [key, value] : object) +// { +// col_key.insertData(key.data(), key.size()); +// } +// +// col_res.getOffsets().push_back(col_res.getOffsets().back() + object.size()); +// return true; +// } +//}; +// +//} diff --git a/tests/queries/0_stateless/03198_json_extract_more_types.reference b/tests/queries/0_stateless/03198_json_extract_more_types.reference new file mode 100644 index 00000000000..9a6580ff81b --- /dev/null +++ b/tests/queries/0_stateless/03198_json_extract_more_types.reference @@ -0,0 +1,21 @@ +2020-01-01 +2020-01-01 +2020-01-01 00:00:00 +2020-01-01 00:00:00.000000 +127.0.0.1 +2001:db8:85a3::8a2e:370:7334 +42 +42 +42 +42 +42 +42 +42 +42 +42 +42 +Hello +Hello +\0\0\0 +Hello\0\0\0\0\0 +5801c962-1182-458a-89f8-d077da5074f9 diff --git a/tests/queries/0_stateless/03198_json_extract_more_types.sql b/tests/queries/0_stateless/03198_json_extract_more_types.sql new file mode 100644 index 00000000000..28d24bbb271 --- /dev/null +++ b/tests/queries/0_stateless/03198_json_extract_more_types.sql @@ -0,0 +1,29 @@ +set allow_suspicious_low_cardinality_types=1; + +select JSONExtract('{"a" : "2020-01-01"}', 'a', 'Date'); +select JSONExtract('{"a" : "2020-01-01"}', 'a', 'Date32'); +select JSONExtract('{"a" : "2020-01-01 00:00:00"}', 'a', 'DateTime'); +select JSONExtract('{"a" : "2020-01-01 00:00:00.000000"}', 'a', 'DateTime64(6)'); +select JSONExtract('{"a" : "127.0.0.1"}', 'a', 'IPv4'); +select JSONExtract('{"a" : "2001:0db8:85a3:0000:0000:8a2e:0370:7334"}', 'a', 'IPv6'); + + +select JSONExtract('{"a" : 42}', 'a', 'LowCardinality(UInt8)'); +select JSONExtract('{"a" : 42}', 'a', 'LowCardinality(Int8)'); +select JSONExtract('{"a" : 42}', 'a', 'LowCardinality(UInt16)'); +select JSONExtract('{"a" : 42}', 'a', 'LowCardinality(Int16)'); +select JSONExtract('{"a" : 42}', 'a', 'LowCardinality(UInt32)'); +select JSONExtract('{"a" : 42}', 'a', 'LowCardinality(Int32)'); +select JSONExtract('{"a" : 42}', 'a', 'LowCardinality(UInt64)'); +select JSONExtract('{"a" : 42}', 'a', 'LowCardinality(Int64)'); + +select JSONExtract('{"a" : 42}', 'a', 'LowCardinality(Float32)'); +select JSONExtract('{"a" : 42}', 'a', 'LowCardinality(Float32)'); + +select JSONExtract('{"a" : "Hello"}', 'a', 'LowCardinality(String)'); +select JSONExtract('{"a" : "Hello"}', 'a', 'LowCardinality(FixedString(5))'); +select JSONExtract('{"a" : "Hello"}', 'a', 'LowCardinality(FixedString(3))'); +select JSONExtract('{"a" : "Hello"}', 'a', 'LowCardinality(FixedString(10))'); + +select JSONExtract('{"a" : "5801c962-1182-458a-89f8-d077da5074f9"}', 'a', 'LowCardinality(UUID)'); + diff --git a/tests/queries/0_stateless/03199_json_extract_dynamic.reference b/tests/queries/0_stateless/03199_json_extract_dynamic.reference new file mode 100644 index 00000000000..759b7763cd1 --- /dev/null +++ b/tests/queries/0_stateless/03199_json_extract_dynamic.reference @@ -0,0 +1,30 @@ +true Bool +42 Int64 +-42 Int64 +18446744073709551615 UInt64 +42.42 Float64 +42 Int64 +-42 Int64 +18446744073709551615 UInt64 +Hello String +2020-01-01 Date +2020-01-01 00:00:00.000000000 DateTime64(9) +[1,2,3] Array(Nullable(Int64)) +['str1','str2','str3'] Array(Nullable(String)) +[[[1],[2,3,4]],[[5,6],[7]]] Array(Array(Array(Nullable(Int64)))) +['2020-01-01 00:00:00.000000000','2020-01-01 00:00:00.000000000'] Array(Nullable(DateTime64(9))) +['2020-01-01','2020-01-01 date'] Array(Nullable(String)) +['2020-01-01','2020-01-01 00:00:00','str'] Array(Nullable(String)) +['2020-01-01','2020-01-01 00:00:00','42'] Array(Nullable(String)) +['str','42'] Array(Nullable(String)) +[42,42.42] Array(Nullable(Float64)) +[42,18446744073709552000,42.42] Array(Nullable(Float64)) +[42,42.42] Array(Nullable(Float64)) +[NULL,NULL] Array(Nullable(String)) +[NULL,42] Array(Nullable(Int64)) +[[NULL],[],[42]] Array(Array(Nullable(Int64))) +[[],[NULL,NULL],[1,NULL,3],[NULL,2,NULL]] Array(Array(Nullable(Int64))) +[[],[NULL,NULL],['1',NULL,'3'],[NULL,'2',NULL],['2020-01-01']] Array(Array(Nullable(String))) +('str',42,[42]) Tuple(Nullable(String), Nullable(Int64), Array(Nullable(Int64))) +[42,18446744073709551615] Array(Nullable(UInt64)) +(-42,18446744073709551615) Tuple(Nullable(Int64), Nullable(UInt64)) diff --git a/tests/queries/0_stateless/03199_json_extract_dynamic.sql b/tests/queries/0_stateless/03199_json_extract_dynamic.sql new file mode 100644 index 00000000000..286949f4d3e --- /dev/null +++ b/tests/queries/0_stateless/03199_json_extract_dynamic.sql @@ -0,0 +1,37 @@ +set input_format_json_try_infer_numbers_from_strings=1; + +select JSONExtract(materialize('{"d" : true}'), 'd', 'Dynamic') as d, dynamicType(d); +select JSONExtract(materialize('{"d" : 42}'), 'd', 'Dynamic') as d, dynamicType(d); +select JSONExtract(materialize('{"d" : -42}'), 'd', 'Dynamic') as d, dynamicType(d); +select JSONExtract(materialize('{"d" : 18446744073709551615}'), 'd', 'Dynamic') as d, dynamicType(d); +select JSONExtract(materialize('{"d" : 42.42}'), 'd', 'Dynamic') as d, dynamicType(d); +select JSONExtract(materialize('{"d" : "42"}'), 'd', 'Dynamic') as d, dynamicType(d); +select JSONExtract(materialize('{"d" : "-42"}'), 'd', 'Dynamic') as d, dynamicType(d); +select JSONExtract(materialize('{"d" : "18446744073709551615"}'), 'd', 'Dynamic') as d, dynamicType(d); + +select JSONExtract(materialize('{"d" : "Hello"}'), 'd', 'Dynamic') as d, dynamicType(d); +select JSONExtract(materialize('{"d" : "2020-01-01"}'), 'd', 'Dynamic') as d, dynamicType(d); +select JSONExtract(materialize('{"d" : "2020-01-01 00:00:00.000"}'), 'd', 'Dynamic') as d, dynamicType(d); + +select JSONExtract(materialize('{"d" : [1, 2, 3]}'), 'd', 'Dynamic') as d, dynamicType(d); +select JSONExtract(materialize('{"d" : ["str1", "str2", "str3"]}'), 'd', 'Dynamic') as d, dynamicType(d); +select JSONExtract(materialize('{"d" : [[[1], [2, 3, 4]], [[5, 6], [7]]]}'), 'd', 'Dynamic') as d, dynamicType(d); + +select JSONExtract(materialize('{"d" : ["2020-01-01", "2020-01-01 00:00:00"]}'), 'd', 'Dynamic') as d, dynamicType(d); +select JSONExtract(materialize('{"d" : ["2020-01-01", "2020-01-01 date"]}'), 'd', 'Dynamic') as d, dynamicType(d); +select JSONExtract(materialize('{"d" : ["2020-01-01", "2020-01-01 00:00:00", "str"]}'), 'd', 'Dynamic') as d, dynamicType(d); +select JSONExtract(materialize('{"d" : ["2020-01-01", "2020-01-01 00:00:00", "42"]}'), 'd', 'Dynamic') as d, dynamicType(d); +select JSONExtract(materialize('{"d" : ["str", "42"]}'), 'd', 'Dynamic') as d, dynamicType(d); +select JSONExtract(materialize('{"d" : [42, 42.42]}'), 'd', 'Dynamic') as d, dynamicType(d); +select JSONExtract(materialize('{"d" : [42, 18446744073709551615, 42.42]}'), 'd', 'Dynamic') as d, dynamicType(d); +select JSONExtract(materialize('{"d" : [42, 42.42]}'), 'd', 'Dynamic') as d, dynamicType(d); + +select JSONExtract(materialize('{"d" : [null, null]}'), 'd', 'Dynamic') as d, dynamicType(d); +select JSONExtract(materialize('{"d" : [null, 42]}'), 'd', 'Dynamic') as d, dynamicType(d); +select JSONExtract(materialize('{"d" : [[null], [], [42]]}'), 'd', 'Dynamic') as d, dynamicType(d); +select JSONExtract(materialize('{"a" : [[], [null, null], ["1", null, "3"], [null, "2", null]]}'), 'a', 'Dynamic') as d, dynamicType(d); +select JSONExtract(materialize('{"a" : [[], [null, null], ["1", null, "3"], [null, "2", null], ["2020-01-01"]]}'), 'a', 'Dynamic') as d, dynamicType(d); + +select JSONExtract(materialize('{"d" : ["str", 42, [42]]}'), 'd', 'Dynamic') as d, dynamicType(d); +select JSONExtract(materialize('{"d" : [42, 18446744073709551615]}'), 'd', 'Dynamic') as d, dynamicType(d); +select JSONExtract(materialize('{"d" : [-42, 18446744073709551615]}'), 'd', 'Dynamic') as d, dynamicType(d); From 5fe594243a4fc281cf3ee878b2f79b09dfd83970 Mon Sep 17 00:00:00 2001 From: avogar Date: Wed, 3 Jul 2024 13:53:30 +0000 Subject: [PATCH 150/417] Remove old file --- src/Functions/FunctionsJSON.h | 1273 --------------------------------- 1 file changed, 1273 deletions(-) delete mode 100644 src/Functions/FunctionsJSON.h diff --git a/src/Functions/FunctionsJSON.h b/src/Functions/FunctionsJSON.h deleted file mode 100644 index 5d44e22300d..00000000000 --- a/src/Functions/FunctionsJSON.h +++ /dev/null @@ -1,1273 +0,0 @@ -//#pragma once -// -//#include -//#include -// -//#include -// -//#include -// -//#include -//#include -//#include -// -//#include -////#include -// -//#include -//#include -//#include -//#include -//#include -//#include -//#include -//#include -//#include -//#include -// -//#include -//#include -//#include -//#include -//#include -//#include -//#include -//#include -//#include -//#include -//#include -//#include -//#include -//#include -//#include -//#include -// -//#include -//#include -//#include -//#include -//#include -// -//#include -//#include -// -// -//#include "config.h" -// -// -//namespace DB -//{ -// -//namespace ErrorCodes -//{ -// extern const int ILLEGAL_TYPE_OF_ARGUMENT; -// extern const int ILLEGAL_COLUMN; -// extern const int NUMBER_OF_ARGUMENTS_DOESNT_MATCH; -//} -// -//template -//concept HasIndexOperator = requires (T t) -//{ -// t[0]; -//}; -// -///// Functions to parse JSONs and extract values from it. -///// The first argument of all these functions gets a JSON, -///// after that there are any number of arguments specifying path to a desired part from the JSON's root. -///// For example, -///// select JSONExtractInt('{"a": "hello", "b": [-100, 200.0, 300]}', 'b', 1) = -100 -// -//class FunctionJSONHelpers -//{ -//public: -// template typename Impl, class JSONParser> -// class Executor -// { -// public: -// static ColumnPtr run(const ColumnsWithTypeAndName & arguments, const DataTypePtr & result_type, size_t input_rows_count) -// { -// MutableColumnPtr to{result_type->createColumn()}; -// to->reserve(input_rows_count); -// -// if (arguments.empty()) -// throw Exception(ErrorCodes::NUMBER_OF_ARGUMENTS_DOESNT_MATCH, "Function {} requires at least one argument", String(Name::name)); -// -// const auto & first_column = arguments[0]; -// if (!isString(first_column.type)) -// throw Exception(ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT, -// "The first argument of function {} should be a string containing JSON, illegal type: " -// "{}", String(Name::name), first_column.type->getName()); -// -// const ColumnPtr & arg_json = first_column.column; -// const auto * col_json_const = typeid_cast(arg_json.get()); -// const auto * col_json_string -// = typeid_cast(col_json_const ? col_json_const->getDataColumnPtr().get() : arg_json.get()); -// -// if (!col_json_string) -// throw Exception(ErrorCodes::ILLEGAL_COLUMN, "Illegal column {}", arg_json->getName()); -// -// const ColumnString::Chars & chars = col_json_string->getChars(); -// const ColumnString::Offsets & offsets = col_json_string->getOffsets(); -// -// size_t num_index_arguments = Impl::getNumberOfIndexArguments(arguments); -// std::vector moves = prepareMoves(Name::name, arguments, 1, num_index_arguments); -// -// /// Preallocate memory in parser if necessary. -// JSONParser parser; -// if constexpr (has_member_function_reserve::value) -// { -// size_t max_size = calculateMaxSize(offsets); -// if (max_size) -// parser.reserve(max_size); -// } -// -// Impl impl; -// -// /// prepare() does Impl-specific preparation before handling each row. -// if constexpr (has_member_function_prepare::*)(const char *, const ColumnsWithTypeAndName &, const DataTypePtr &)>::value) -// impl.prepare(Name::name, arguments, result_type); -// -// using Element = typename JSONParser::Element; -// -// Element document; -// bool document_ok = false; -// if (col_json_const) -// { -// std::string_view json{reinterpret_cast(chars.data()), offsets[0] - 1}; -// document_ok = parser.parse(json, document); -// } -// -// for (const auto i : collections::range(0, input_rows_count)) -// { -// if (!col_json_const) -// { -// std::string_view json{reinterpret_cast(&chars[offsets[i - 1]]), offsets[i] - offsets[i - 1] - 1}; -// document_ok = parser.parse(json, document); -// } -// -// bool added_to_column = false; -// if (document_ok) -// { -// /// Perform moves. -// Element element; -// std::string_view last_key; -// bool moves_ok = performMoves(arguments, i, document, moves, element, last_key); -// -// if (moves_ok) -// added_to_column = impl.insertResultToColumn(*to, element, last_key); -// } -// -// /// We add default value (=null or zero) if something goes wrong, we don't throw exceptions in these JSON functions. -// if (!added_to_column) -// to->insertDefault(); -// } -// return to; -// } -// }; -// -//private: -// BOOST_TTI_HAS_MEMBER_FUNCTION(reserve) -// BOOST_TTI_HAS_MEMBER_FUNCTION(prepare) -// -// /// Represents a move of a JSON iterator described by a single argument passed to a JSON function. -// /// For example, the call JSONExtractInt('{"a": "hello", "b": [-100, 200.0, 300]}', 'b', 1) -// /// contains two moves: {MoveType::ConstKey, "b"} and {MoveType::ConstIndex, 1}. -// /// Keys and indices can be nonconst, in this case they are calculated for each row. -// enum class MoveType : uint8_t -// { -// Key, -// Index, -// ConstKey, -// ConstIndex, -// }; -// -// struct Move -// { -// explicit Move(MoveType type_, size_t index_ = 0) : type(type_), index(index_) {} -// Move(MoveType type_, const String & key_) : type(type_), key(key_) {} -// MoveType type; -// size_t index = 0; -// String key; -// }; -// -// static std::vector prepareMoves( -// const char * function_name, -// const ColumnsWithTypeAndName & columns, -// size_t first_index_argument, -// size_t num_index_arguments) -// { -// std::vector moves; -// moves.reserve(num_index_arguments); -// for (const auto i : collections::range(first_index_argument, first_index_argument + num_index_arguments)) -// { -// const auto & column = columns[i]; -// if (!isString(column.type) && !isNativeInteger(column.type)) -// throw Exception(ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT, -// "The argument {} of function {} should be a string specifying key " -// "or an integer specifying index, illegal type: {}", -// std::to_string(i + 1), String(function_name), column.type->getName()); -// -// if (column.column && isColumnConst(*column.column)) -// { -// const auto & column_const = assert_cast(*column.column); -// if (isString(column.type)) -// moves.emplace_back(MoveType::ConstKey, column_const.getValue()); -// else -// moves.emplace_back(MoveType::ConstIndex, column_const.getInt(0)); -// } -// else -// { -// if (isString(column.type)) -// moves.emplace_back(MoveType::Key, ""); -// else -// moves.emplace_back(MoveType::Index, 0); -// } -// } -// return moves; -// } -// -// -// /// Performs moves of types MoveType::Index and MoveType::ConstIndex. -// template -// static bool performMoves(const ColumnsWithTypeAndName & arguments, size_t row, -// const typename JSONParser::Element & document, const std::vector & moves, -// typename JSONParser::Element & element, std::string_view & last_key) -// { -// typename JSONParser::Element res_element = document; -// std::string_view key; -// -// for (size_t j = 0; j != moves.size(); ++j) -// { -// switch (moves[j].type) -// { -// case MoveType::ConstIndex: -// { -// if (!moveToElementByIndex(res_element, static_cast(moves[j].index), key)) -// return false; -// break; -// } -// case MoveType::ConstKey: -// { -// key = moves[j].key; -// if (!moveToElementByKey(res_element, key)) -// return false; -// break; -// } -// case MoveType::Index: -// { -// Int64 index = (*arguments[j + 1].column)[row].get(); -// if (!moveToElementByIndex(res_element, static_cast(index), key)) -// return false; -// break; -// } -// case MoveType::Key: -// { -// key = arguments[j + 1].column->getDataAt(row).toView(); -// if (!moveToElementByKey(res_element, key)) -// return false; -// break; -// } -// } -// } -// -// element = res_element; -// last_key = key; -// return true; -// } -// -// template -// static bool moveToElementByIndex(typename JSONParser::Element & element, int index, std::string_view & out_key) -// { -// if (element.isArray()) -// { -// auto array = element.getArray(); -// if (index >= 0) -// --index; -// else -// index += array.size(); -// -// if (static_cast(index) >= array.size()) -// return false; -// element = array[index]; -// out_key = {}; -// return true; -// } -// -// if constexpr (HasIndexOperator) -// { -// if (element.isObject()) -// { -// auto object = element.getObject(); -// if (index >= 0) -// --index; -// else -// index += object.size(); -// -// if (static_cast(index) >= object.size()) -// return false; -// std::tie(out_key, element) = object[index]; -// return true; -// } -// } -// -// return {}; -// } -// -// /// Performs moves of types MoveType::Key and MoveType::ConstKey. -// template -// static bool moveToElementByKey(typename JSONParser::Element & element, std::string_view key) -// { -// if (!element.isObject()) -// return false; -// auto object = element.getObject(); -// return object.find(key, element); -// } -// -// static size_t calculateMaxSize(const ColumnString::Offsets & offsets) -// { -// size_t max_size = 0; -// for (const auto i : collections::range(0, offsets.size())) -// { -// size_t size = offsets[i] - offsets[i - 1]; -// max_size = std::max(max_size, size); -// } -// if (max_size) -// --max_size; -// return max_size; -// } -// -//}; -// -//template -//class JSONExtractImpl; -// -//template -//class JSONExtractKeysAndValuesImpl; -// -///** -//* Functions JSONExtract and JSONExtractKeysAndValues force the return type - it is specified in the last argument. -//* For example - `SELECT JSONExtract(materialize('{"a": 131231, "b": 1234}'), 'b', 'LowCardinality(FixedString(4))')` -//* But by default ClickHouse decides on its own whether the return type will be LowCardinality based on the types of -//* input arguments. -//* And for these specific functions we cannot rely on this mechanism, so these functions have their own implementation - -//* just convert all of the LowCardinality input columns to full ones, execute and wrap the resulting column in LowCardinality -//* if needed. -//*/ -//template typename Impl> -//constexpr bool functionForcesTheReturnType() -//{ -// return std::is_same_v, JSONExtractImpl> || std::is_same_v, JSONExtractKeysAndValuesImpl>; -//} -// -//template typename Impl> -//class ExecutableFunctionJSON : public IExecutableFunction -//{ -// -//public: -// explicit ExecutableFunctionJSON(const NullPresence & null_presence_, bool allow_simdjson_, const DataTypePtr & json_return_type_) -// : null_presence(null_presence_), allow_simdjson(allow_simdjson_), json_return_type(json_return_type_) -// { -// } -// -// String getName() const override { return Name::name; } -// bool useDefaultImplementationForNulls() const override { return false; } -// bool useDefaultImplementationForConstants() const override { return true; } -// bool useDefaultImplementationForLowCardinalityColumns() const override -// { -// return !functionForcesTheReturnType(); -// } -// -// ColumnPtr executeImpl(const ColumnsWithTypeAndName & arguments, const DataTypePtr & result_type, size_t input_rows_count) const override -// { -// if (null_presence.has_null_constant) -// return result_type->createColumnConstWithDefaultValue(input_rows_count); -// -// if constexpr (functionForcesTheReturnType()) -// { -// ColumnsWithTypeAndName columns_without_low_cardinality = arguments; -// -// for (auto & column : columns_without_low_cardinality) -// { -// column.column = recursiveRemoveLowCardinality(column.column); -// column.type = recursiveRemoveLowCardinality(column.type); -// } -// -// ColumnsWithTypeAndName temporary_columns = null_presence.has_nullable ? createBlockWithNestedColumns(columns_without_low_cardinality) : columns_without_low_cardinality; -// ColumnPtr temporary_result = chooseAndRunJSONParser(temporary_columns, json_return_type, input_rows_count); -// -// if (null_presence.has_nullable) -// temporary_result = wrapInNullable(temporary_result, columns_without_low_cardinality, result_type, input_rows_count); -// -// if (result_type->lowCardinality()) -// temporary_result = recursiveLowCardinalityTypeConversion(temporary_result, json_return_type, result_type); -// -// return temporary_result; -// } -// else -// { -// ColumnsWithTypeAndName temporary_columns = null_presence.has_nullable ? createBlockWithNestedColumns(arguments) : arguments; -// ColumnPtr temporary_result = chooseAndRunJSONParser(temporary_columns, json_return_type, input_rows_count); -// -// if (null_presence.has_nullable) -// temporary_result = wrapInNullable(temporary_result, arguments, result_type, input_rows_count); -// -// if (result_type->lowCardinality()) -// temporary_result = recursiveLowCardinalityTypeConversion(temporary_result, json_return_type, result_type); -// -// return temporary_result; -// } -// } -// -//private: -// -// ColumnPtr -// chooseAndRunJSONParser(const ColumnsWithTypeAndName & arguments, const DataTypePtr & result_type, size_t input_rows_count) const -// { -//#if USE_SIMDJSON -// if (allow_simdjson) -// return FunctionJSONHelpers::Executor::run(arguments, result_type, input_rows_count); -//#endif -// -//#if USE_RAPIDJSON -// return FunctionJSONHelpers::Executor::run(arguments, result_type, input_rows_count); -//#else -// return FunctionJSONHelpers::Executor::run(arguments, result_type, input_rows_count); -//#endif -// } -// -// NullPresence null_presence; -// bool allow_simdjson; -// DataTypePtr json_return_type; -//}; -// -// -//template typename Impl> -//class FunctionBaseFunctionJSON : public IFunctionBase -//{ -//public: -// explicit FunctionBaseFunctionJSON( -// const NullPresence & null_presence_, -// bool allow_simdjson_, -// DataTypes argument_types_, -// DataTypePtr return_type_, -// DataTypePtr json_return_type_) -// : null_presence(null_presence_) -// , allow_simdjson(allow_simdjson_) -// , argument_types(std::move(argument_types_)) -// , return_type(std::move(return_type_)) -// , json_return_type(std::move(json_return_type_)) -// { -// } -// -// String getName() const override { return Name::name; } -// -// const DataTypes & getArgumentTypes() const override -// { -// return argument_types; -// } -// -// const DataTypePtr & getResultType() const override -// { -// return return_type; -// } -// -// bool isSuitableForShortCircuitArgumentsExecution(const DataTypesWithConstInfo & /*arguments*/) const override { return true; } -// -// ExecutableFunctionPtr prepare(const ColumnsWithTypeAndName &) const override -// { -// return std::make_unique>(null_presence, allow_simdjson, json_return_type); -// } -// -//private: -// NullPresence null_presence; -// bool allow_simdjson; -// DataTypes argument_types; -// DataTypePtr return_type; -// DataTypePtr json_return_type; -//}; -// -///// We use IFunctionOverloadResolver instead of IFunction to handle non-default NULL processing. -///// Both NULL and JSON NULL should generate NULL value. If any argument is NULL, return NULL. -//template typename Impl> -//class JSONOverloadResolver : public IFunctionOverloadResolver, WithContext -//{ -//public: -// static constexpr auto name = Name::name; -// -// String getName() const override { return name; } -// -// static FunctionOverloadResolverPtr create(ContextPtr context_) -// { -// return std::make_unique(context_); -// } -// -// explicit JSONOverloadResolver(ContextPtr context_) : WithContext(context_) {} -// -// bool isVariadic() const override { return true; } -// size_t getNumberOfArguments() const override { return 0; } -// bool useDefaultImplementationForNulls() const override { return false; } -// bool useDefaultImplementationForLowCardinalityColumns() const override -// { -// return !functionForcesTheReturnType(); -// } -// -// FunctionBasePtr build(const ColumnsWithTypeAndName & arguments) const override -// { -// bool has_nothing_argument = false; -// for (const auto & arg : arguments) -// has_nothing_argument |= isNothing(arg.type); -// -// DataTypePtr json_return_type = Impl::getReturnType(Name::name, createBlockWithNestedColumns(arguments)); -// NullPresence null_presence = getNullPresense(arguments); -// DataTypePtr return_type; -// if (has_nothing_argument) -// return_type = std::make_shared(); -// else if (null_presence.has_null_constant) -// return_type = makeNullable(std::make_shared()); -// else if (null_presence.has_nullable) -// return_type = makeNullable(json_return_type); -// else -// return_type = json_return_type; -// -// /// Top-level LowCardinality columns are processed outside JSON parser. -// json_return_type = removeLowCardinality(json_return_type); -// -// DataTypes argument_types; -// argument_types.reserve(arguments.size()); -// for (const auto & argument : arguments) -// argument_types.emplace_back(argument.type); -// return std::make_unique>( -// null_presence, getContext()->getSettingsRef().allow_simdjson, argument_types, return_type, json_return_type); -// } -//}; -// -//struct NameJSONHas { static constexpr auto name{"JSONHas"}; }; -//struct NameIsValidJSON { static constexpr auto name{"isValidJSON"}; }; -//struct NameJSONLength { static constexpr auto name{"JSONLength"}; }; -//struct NameJSONKey { static constexpr auto name{"JSONKey"}; }; -//struct NameJSONType { static constexpr auto name{"JSONType"}; }; -//struct NameJSONExtractInt { static constexpr auto name{"JSONExtractInt"}; }; -//struct NameJSONExtractUInt { static constexpr auto name{"JSONExtractUInt"}; }; -//struct NameJSONExtractFloat { static constexpr auto name{"JSONExtractFloat"}; }; -//struct NameJSONExtractBool { static constexpr auto name{"JSONExtractBool"}; }; -//struct NameJSONExtractString { static constexpr auto name{"JSONExtractString"}; }; -//struct NameJSONExtract { static constexpr auto name{"JSONExtract"}; }; -//struct NameJSONExtractKeysAndValues { static constexpr auto name{"JSONExtractKeysAndValues"}; }; -//struct NameJSONExtractRaw { static constexpr auto name{"JSONExtractRaw"}; }; -//struct NameJSONExtractArrayRaw { static constexpr auto name{"JSONExtractArrayRaw"}; }; -//struct NameJSONExtractKeysAndValuesRaw { static constexpr auto name{"JSONExtractKeysAndValuesRaw"}; }; -//struct NameJSONExtractKeys { static constexpr auto name{"JSONExtractKeys"}; }; -// -// -//template -//class JSONHasImpl -//{ -//public: -// using Element = typename JSONParser::Element; -// -// static DataTypePtr getReturnType(const char *, const ColumnsWithTypeAndName &) { return std::make_shared(); } -// -// static size_t getNumberOfIndexArguments(const ColumnsWithTypeAndName & arguments) { return arguments.size() - 1; } -// -// static bool insertResultToColumn(IColumn & dest, const Element &, std::string_view) -// { -// ColumnVector & col_vec = assert_cast &>(dest); -// col_vec.insertValue(1); -// return true; -// } -//}; -// -// -//template -//class IsValidJSONImpl -//{ -//public: -// using Element = typename JSONParser::Element; -// -// static DataTypePtr getReturnType(const char * function_name, const ColumnsWithTypeAndName & arguments) -// { -// if (arguments.size() != 1) -// { -// /// IsValidJSON() shouldn't get parameters other than JSON. -// throw Exception(ErrorCodes::NUMBER_OF_ARGUMENTS_DOESNT_MATCH, "Function {} needs exactly one argument", -// String(function_name)); -// } -// return std::make_shared(); -// } -// -// static size_t getNumberOfIndexArguments(const ColumnsWithTypeAndName &) { return 0; } -// -// static bool insertResultToColumn(IColumn & dest, const Element &, std::string_view) -// { -// /// This function is called only if JSON is valid. -// /// If JSON isn't valid then `FunctionJSON::Executor::run()` adds default value (=zero) to `dest` without calling this function. -// ColumnVector & col_vec = assert_cast &>(dest); -// col_vec.insertValue(1); -// return true; -// } -//}; -// -// -//template -//class JSONLengthImpl -//{ -//public: -// using Element = typename JSONParser::Element; -// -// static DataTypePtr getReturnType(const char *, const ColumnsWithTypeAndName &) -// { -// return std::make_shared(); -// } -// -// static size_t getNumberOfIndexArguments(const ColumnsWithTypeAndName & arguments) { return arguments.size() - 1; } -// -// static bool insertResultToColumn(IColumn & dest, const Element & element, std::string_view) -// { -// size_t size; -// if (element.isArray()) -// size = element.getArray().size(); -// else if (element.isObject()) -// size = element.getObject().size(); -// else -// return false; -// -// ColumnVector & col_vec = assert_cast &>(dest); -// col_vec.insertValue(size); -// return true; -// } -//}; -// -// -//template -//class JSONKeyImpl -//{ -//public: -// using Element = typename JSONParser::Element; -// -// static DataTypePtr getReturnType(const char *, const ColumnsWithTypeAndName &) -// { -// return std::make_shared(); -// } -// -// static size_t getNumberOfIndexArguments(const ColumnsWithTypeAndName & arguments) { return arguments.size() - 1; } -// -// static bool insertResultToColumn(IColumn & dest, const Element &, std::string_view last_key) -// { -// if (last_key.empty()) -// return false; -// ColumnString & col_str = assert_cast(dest); -// col_str.insertData(last_key.data(), last_key.size()); -// return true; -// } -//}; -// -// -//template -//class JSONTypeImpl -//{ -//public: -// using Element = typename JSONParser::Element; -// -// static DataTypePtr getReturnType(const char *, const ColumnsWithTypeAndName &) -// { -// static const std::vector> values = { -// {"Array", '['}, -// {"Object", '{'}, -// {"String", '"'}, -// {"Int64", 'i'}, -// {"UInt64", 'u'}, -// {"Double", 'd'}, -// {"Bool", 'b'}, -// {"Null", 0}, /// the default value for the column. -// }; -// return std::make_shared>(values); -// } -// -// static size_t getNumberOfIndexArguments(const ColumnsWithTypeAndName & arguments) { return arguments.size() - 1; } -// -// static bool insertResultToColumn(IColumn & dest, const Element & element, std::string_view) -// { -// UInt8 type; -// switch (element.type()) -// { -// case ElementType::INT64: -// type = 'i'; -// break; -// case ElementType::UINT64: -// type = 'u'; -// break; -// case ElementType::DOUBLE: -// type = 'd'; -// break; -// case ElementType::STRING: -// type = '"'; -// break; -// case ElementType::ARRAY: -// type = '['; -// break; -// case ElementType::OBJECT: -// type = '{'; -// break; -// case ElementType::BOOL: -// type = 'b'; -// break; -// case ElementType::NULL_VALUE: -// type = 0; -// break; -// } -// -// ColumnVector & col_vec = assert_cast &>(dest); -// col_vec.insertValue(type); -// return true; -// } -//}; -// -// -//template -//class JSONExtractNumericImpl -//{ -//public: -// using Element = typename JSONParser::Element; -// -// static DataTypePtr getReturnType(const char *, const ColumnsWithTypeAndName &) -// { -// return std::make_shared>(); -// } -// -// static size_t getNumberOfIndexArguments(const ColumnsWithTypeAndName & arguments) { return arguments.size() - 1; } -// -// static bool insertResultToColumn(IColumn & dest, const Element & element, std::string_view) -// { -// NumberType value; -// -// switch (element.type()) -// { -// case ElementType::DOUBLE: -// if constexpr (std::is_floating_point_v) -// { -// /// We permit inaccurate conversion of double to float. -// /// Example: double 0.1 from JSON is not representable in float. -// /// But it will be more convenient for user to perform conversion. -// value = static_cast(element.getDouble()); -// } -// else if (!accurate::convertNumeric(element.getDouble(), value)) -// return false; -// break; -// case ElementType::UINT64: -// if (!accurate::convertNumeric(element.getUInt64(), value)) -// return false; -// break; -// case ElementType::INT64: -// if (!accurate::convertNumeric(element.getInt64(), value)) -// return false; -// break; -// case ElementType::BOOL: -// if constexpr (is_integer && convert_bool_to_integer) -// { -// value = static_cast(element.getBool()); -// break; -// } -// return false; -// case ElementType::STRING: -// { -// auto rb = ReadBufferFromMemory{element.getString()}; -// if constexpr (std::is_floating_point_v) -// { -// if (!tryReadFloatText(value, rb) || !rb.eof()) -// return false; -// } -// else -// { -// if (tryReadIntText(value, rb) && rb.eof()) -// break; -// -// /// Try to parse float and convert it to integer. -// Float64 tmp_float; -// rb.position() = rb.buffer().begin(); -// if (!tryReadFloatText(tmp_float, rb) || !rb.eof()) -// return false; -// -// if (!accurate::convertNumeric(tmp_float, value)) -// return false; -// } -// break; -// } -// default: -// return false; -// } -// -// if (dest.getDataType() == TypeIndex::LowCardinality) -// { -// ColumnLowCardinality & col_low = assert_cast(dest); -// col_low.insertData(reinterpret_cast(&value), sizeof(value)); -// } -// else -// { -// auto & col_vec = assert_cast &>(dest); -// col_vec.insertValue(value); -// } -// return true; -// } -//}; -// -// -//template -//using JSONExtractInt64Impl = JSONExtractNumericImpl; -//template -//using JSONExtractUInt64Impl = JSONExtractNumericImpl; -//template -//using JSONExtractFloat64Impl = JSONExtractNumericImpl; -// -// -//template -//class JSONExtractBoolImpl -//{ -//public: -// using Element = typename JSONParser::Element; -// -// static DataTypePtr getReturnType(const char *, const ColumnsWithTypeAndName &) -// { -// return std::make_shared(); -// } -// -// static size_t getNumberOfIndexArguments(const ColumnsWithTypeAndName & arguments) { return arguments.size() - 1; } -// -// static bool insertResultToColumn(IColumn & dest, const Element & element, std::string_view) -// { -// bool value; -// switch (element.type()) -// { -// case ElementType::BOOL: -// value = element.getBool(); -// break; -// case ElementType::INT64: -// value = element.getInt64() != 0; -// break; -// case ElementType::UINT64: -// value = element.getUInt64() != 0; -// break; -// default: -// return false; -// } -// -// auto & col_vec = assert_cast &>(dest); -// col_vec.insertValue(static_cast(value)); -// return true; -// } -//}; -// -//template -//class JSONExtractRawImpl; -// -//template -//class JSONExtractStringImpl -//{ -//public: -// using Element = typename JSONParser::Element; -// -// static DataTypePtr getReturnType(const char *, const ColumnsWithTypeAndName &) -// { -// return std::make_shared(); -// } -// -// static size_t getNumberOfIndexArguments(const ColumnsWithTypeAndName & arguments) { return arguments.size() - 1; } -// -// static bool insertResultToColumn(IColumn & dest, const Element & element, std::string_view) -// { -// if (element.isNull()) -// return false; -// -// if (!element.isString()) -// return JSONExtractRawImpl::insertResultToColumn(dest, element, {}); -// -// auto str = element.getString(); -// -// if (dest.getDataType() == TypeIndex::LowCardinality) -// { -// ColumnLowCardinality & col_low = assert_cast(dest); -// col_low.insertData(str.data(), str.size()); -// } -// else -// { -// ColumnString & col_str = assert_cast(dest); -// col_str.insertData(str.data(), str.size()); -// } -// return true; -// } -//}; -// -// -//static const JSONExtractInsertSettings & getJSONExtractInsertSettings() -//{ -// static const JSONExtractInsertSettings instance = [] -// { -// JSONExtractInsertSettings settings; -// settings.insert_null_as_default = false; -// settings.insert_default_on_invalid_elements_in_complex_types = true; -// return settings; -// }(); -// return instance; -//} -// -//template -//class JSONExtractImpl -//{ -//public: -// using Element = typename JSONParser::Element; -// -// static DataTypePtr getReturnType(const char * function_name, const ColumnsWithTypeAndName & arguments) -// { -// if (arguments.size() < 2) -// throw Exception(ErrorCodes::NUMBER_OF_ARGUMENTS_DOESNT_MATCH, "Function {} requires at least two arguments", String(function_name)); -// -// const auto & col = arguments.back(); -// const auto * col_type_const = typeid_cast(col.column.get()); -// if (!col_type_const || !isString(col.type)) -// throw Exception(ErrorCodes::ILLEGAL_COLUMN, -// "The last argument of function {} should " -// "be a constant string specifying the return data type, illegal value: {}", -// String(function_name), col.name); -// -// return DataTypeFactory::instance().get(col_type_const->getValue()); -// } -// -// static size_t getNumberOfIndexArguments(const ColumnsWithTypeAndName & arguments) { return arguments.size() - 2; } -// -// void prepare(const char * function_name, const ColumnsWithTypeAndName &, const DataTypePtr & result_type) -// { -// extract_tree = buildJSONExtractTree(result_type, function_name); -// } -// -// bool insertResultToColumn(IColumn & dest, const Element & element, std::string_view) -// { -// String error; -// return extract_tree->insertResultToColumn(dest, element, getJSONExtractInsertSettings(), error); -// } -// -//protected: -// std::unique_ptr> extract_tree; -//}; -// -// -//template -//class JSONExtractKeysAndValuesImpl -//{ -//public: -// using Element = typename JSONParser::Element; -// -// static DataTypePtr getReturnType(const char * function_name, const ColumnsWithTypeAndName & arguments) -// { -// if (arguments.size() < 2) -// throw Exception(ErrorCodes::NUMBER_OF_ARGUMENTS_DOESNT_MATCH, "Function {} requires at least two arguments", String(function_name)); -// -// const auto & col = arguments.back(); -// const auto * col_type_const = typeid_cast(col.column.get()); -// if (!col_type_const || !isString(col.type)) -// throw Exception(ErrorCodes::ILLEGAL_COLUMN, -// "The last argument of function {} should " -// "be a constant string specifying the values' data type, illegal value: {}", -// String(function_name), col.name); -// -// DataTypePtr key_type = std::make_unique(); -// DataTypePtr value_type = DataTypeFactory::instance().get(col_type_const->getValue()); -// DataTypePtr tuple_type = std::make_unique(DataTypes{key_type, value_type}); -// return std::make_unique(tuple_type); -// } -// -// static size_t getNumberOfIndexArguments(const ColumnsWithTypeAndName & arguments) { return arguments.size() - 2; } -// -// void prepare(const char * function_name, const ColumnsWithTypeAndName &, const DataTypePtr & result_type) -// { -// const auto tuple_type = typeid_cast(result_type.get())->getNestedType(); -// const auto value_type = typeid_cast(tuple_type.get())->getElements()[1]; -// extract_tree = buildJSONExtractTree(value_type, function_name); -// } -// -// bool insertResultToColumn(IColumn & dest, const Element & element, std::string_view) -// { -// if (!element.isObject()) -// return false; -// -// auto object = element.getObject(); -// -// auto & col_arr = assert_cast(dest); -// auto & col_tuple = assert_cast(col_arr.getData()); -// size_t old_size = col_tuple.size(); -// auto & col_key = assert_cast(col_tuple.getColumn(0)); -// auto & col_value = col_tuple.getColumn(1); -// -// String error; -// for (const auto & [key, value] : object) -// { -// if (extract_tree->insertResultToColumn(col_value, value, getJSONExtractInsertSettings(), error)) -// col_key.insertData(key.data(), key.size()); -// } -// -// if (col_tuple.size() == old_size) -// return false; -// -// col_arr.getOffsets().push_back(col_tuple.size()); -// return true; -// } -// -//private: -// std::unique_ptr> extract_tree; -//}; -// -// -//template -//class JSONExtractRawImpl -//{ -//public: -// using Element = typename JSONParser::Element; -// -// static DataTypePtr getReturnType(const char *, const ColumnsWithTypeAndName &) -// { -// return std::make_shared(); -// } -// -// static size_t getNumberOfIndexArguments(const ColumnsWithTypeAndName & arguments) { return arguments.size() - 1; } -// -// static bool insertResultToColumn(IColumn & dest, const Element & element, std::string_view) -// { -// if (dest.getDataType() == TypeIndex::LowCardinality) -// { -// ColumnString::Chars chars; -// WriteBufferFromVector buf(chars, AppendModeTag()); -// traverse(element, buf); -// buf.finalize(); -// assert_cast(dest).insertData(reinterpret_cast(chars.data()), chars.size()); -// } -// else -// { -// ColumnString & col_str = assert_cast(dest); -// auto & chars = col_str.getChars(); -// WriteBufferFromVector buf(chars, AppendModeTag()); -// traverse(element, buf); -// buf.finalize(); -// chars.push_back(0); -// col_str.getOffsets().push_back(chars.size()); -// } -// return true; -// } -// -// // We use insertResultToFixedStringColumn in case we are inserting raw data in a FixedString column -// static bool insertResultToFixedStringColumn(IColumn & dest, const Element & element, std::string_view) -// { -// ColumnFixedString::Chars chars; -// WriteBufferFromVector buf(chars, AppendModeTag()); -// traverse(element, buf); -// buf.finalize(); -// -// auto & col_str = assert_cast(dest); -// -// if (chars.size() > col_str.getN()) -// return false; -// -// chars.resize_fill(col_str.getN()); -// col_str.insertData(reinterpret_cast(chars.data()), chars.size()); -// -// -// return true; -// } -// -// // We use insertResultToLowCardinalityFixedStringColumn in case we are inserting raw data in a Low Cardinality FixedString column -// static bool insertResultToLowCardinalityFixedStringColumn(IColumn & dest, const Element & element, size_t fixed_length) -// { -// if (element.getObject().size() > fixed_length) -// return false; -// -// ColumnFixedString::Chars chars; -// WriteBufferFromVector buf(chars, AppendModeTag()); -// traverse(element, buf); -// buf.finalize(); -// -// if (chars.size() > fixed_length) -// return false; -// chars.resize_fill(fixed_length); -// assert_cast(dest).insertData(reinterpret_cast(chars.data()), chars.size()); -// -// return true; -// } -// -//private: -// static void traverse(const Element & element, WriteBuffer & buf) -// { -// if (element.isInt64()) -// { -// writeIntText(element.getInt64(), buf); -// return; -// } -// if (element.isUInt64()) -// { -// writeIntText(element.getUInt64(), buf); -// return; -// } -// if (element.isDouble()) -// { -// writeFloatText(element.getDouble(), buf); -// return; -// } -// if (element.isBool()) -// { -// if (element.getBool()) -// writeCString("true", buf); -// else -// writeCString("false", buf); -// return; -// } -// if (element.isString()) -// { -// writeJSONString(element.getString(), buf, formatSettings()); -// return; -// } -// if (element.isArray()) -// { -// writeChar('[', buf); -// bool need_comma = false; -// for (auto value : element.getArray()) -// { -// if (std::exchange(need_comma, true)) -// writeChar(',', buf); -// traverse(value, buf); -// } -// writeChar(']', buf); -// return; -// } -// if (element.isObject()) -// { -// writeChar('{', buf); -// bool need_comma = false; -// for (auto [key, value] : element.getObject()) -// { -// if (std::exchange(need_comma, true)) -// writeChar(',', buf); -// writeJSONString(key, buf, formatSettings()); -// writeChar(':', buf); -// traverse(value, buf); -// } -// writeChar('}', buf); -// return; -// } -// if (element.isNull()) -// { -// writeCString("null", buf); -// return; -// } -// } -// -// static const FormatSettings & formatSettings() -// { -// static const FormatSettings the_instance = [] -// { -// FormatSettings settings; -// settings.json.escape_forward_slashes = false; -// return settings; -// }(); -// return the_instance; -// } -//}; -// -// -//template -//class JSONExtractArrayRawImpl -//{ -//public: -// using Element = typename JSONParser::Element; -// -// static DataTypePtr getReturnType(const char *, const ColumnsWithTypeAndName &) -// { -// return std::make_shared(std::make_shared()); -// } -// -// static size_t getNumberOfIndexArguments(const ColumnsWithTypeAndName & arguments) { return arguments.size() - 1; } -// -// static bool insertResultToColumn(IColumn & dest, const Element & element, std::string_view) -// { -// if (!element.isArray()) -// return false; -// -// auto array = element.getArray(); -// ColumnArray & col_res = assert_cast(dest); -// -// for (auto value : array) -// JSONExtractRawImpl::insertResultToColumn(col_res.getData(), value, {}); -// -// col_res.getOffsets().push_back(col_res.getOffsets().back() + array.size()); -// return true; -// } -//}; -// -// -//template -//class JSONExtractKeysAndValuesRawImpl -//{ -//public: -// using Element = typename JSONParser::Element; -// -// static DataTypePtr getReturnType(const char *, const ColumnsWithTypeAndName &) -// { -// DataTypePtr string_type = std::make_unique(); -// DataTypePtr tuple_type = std::make_unique(DataTypes{string_type, string_type}); -// return std::make_unique(tuple_type); -// } -// -// static size_t getNumberOfIndexArguments(const ColumnsWithTypeAndName & arguments) { return arguments.size() - 1; } -// -// bool insertResultToColumn(IColumn & dest, const Element & element, std::string_view) -// { -// if (!element.isObject()) -// return false; -// -// auto object = element.getObject(); -// -// auto & col_arr = assert_cast(dest); -// auto & col_tuple = assert_cast(col_arr.getData()); -// auto & col_key = assert_cast(col_tuple.getColumn(0)); -// auto & col_value = assert_cast(col_tuple.getColumn(1)); -// -// for (const auto & [key, value] : object) -// { -// col_key.insertData(key.data(), key.size()); -// JSONExtractRawImpl::insertResultToColumn(col_value, value, {}); -// } -// -// col_arr.getOffsets().push_back(col_arr.getOffsets().back() + object.size()); -// return true; -// } -//}; -// -//template -//class JSONExtractKeysImpl -//{ -//public: -// using Element = typename JSONParser::Element; -// -// static DataTypePtr getReturnType(const char *, const ColumnsWithTypeAndName &) -// { -// return std::make_unique(std::make_shared()); -// } -// -// static size_t getNumberOfIndexArguments(const ColumnsWithTypeAndName & arguments) { return arguments.size() - 1; } -// -// bool insertResultToColumn(IColumn & dest, const Element & element, std::string_view) -// { -// if (!element.isObject()) -// return false; -// -// auto object = element.getObject(); -// -// ColumnArray & col_res = assert_cast(dest); -// auto & col_key = assert_cast(col_res.getData()); -// -// for (const auto & [key, value] : object) -// { -// col_key.insertData(key.data(), key.size()); -// } -// -// col_res.getOffsets().push_back(col_res.getOffsets().back() + object.size()); -// return true; -// } -//}; -// -//} From 63303dd79893ace08ce2ed4be6bfee422287d44b Mon Sep 17 00:00:00 2001 From: avogar Date: Wed, 3 Jul 2024 14:03:04 +0000 Subject: [PATCH 151/417] Fix style --- src/Formats/JSONExtractTree.cpp | 9 +++++++-- 1 file changed, 7 insertions(+), 2 deletions(-) diff --git a/src/Formats/JSONExtractTree.cpp b/src/Formats/JSONExtractTree.cpp index 6d019f96ba6..18437c16bc9 100644 --- a/src/Formats/JSONExtractTree.cpp +++ b/src/Formats/JSONExtractTree.cpp @@ -50,6 +50,11 @@ namespace DB { +namespace ErrorCodes +{ + extern const int ILLEGAL_TYPE_OF_ARGUMENT; +} + template void jsonElementToString(const typename JSONParser::Element & element, WriteBuffer & buf, const FormatSettings & format_settings) { @@ -207,7 +212,7 @@ namespace { template -String jsonElementToString(const typename JSONParser::Element & element, const FormatSettings & format_settings) +String jsonElementToString(const typename JSONParser::Element & element, const FormatSettings & format_settings) { WriteBufferFromOwnString buf; jsonElementToString(element, buf, format_settings); @@ -1440,7 +1445,7 @@ std::unique_ptr> buildJSONExtractTree(const Data case TypeIndex::Date:; return std::make_unique>(); case TypeIndex::Date32: - return std::make_unique>(); + return std::make_unique>(); case TypeIndex::DateTime: return std::make_unique>(assert_cast(*type)); case TypeIndex::DateTime64: From 0b909ed673185cf1cc410a7b5339d264e8993cee Mon Sep 17 00:00:00 2001 From: Dale McDiarmid Date: Wed, 3 Jul 2024 15:14:55 +0100 Subject: [PATCH 152/417] Update json.md Point to guide for json --- docs/en/sql-reference/data-types/json.md | 3 +++ 1 file changed, 3 insertions(+) diff --git a/docs/en/sql-reference/data-types/json.md b/docs/en/sql-reference/data-types/json.md index c29be2cff58..9e9c3fdb489 100644 --- a/docs/en/sql-reference/data-types/json.md +++ b/docs/en/sql-reference/data-types/json.md @@ -5,6 +5,9 @@ sidebar_label: Object Data Type keywords: [object, data type] --- +:::note +**ClickHouse handles JSON documents in a variety of way, allowing you to bring structure to this data and query it quickly and efficiently. See [this guide](/docs/en/integrations/data-formats/json) for detailed guidance on working with JSON.**::: + # Object Data Type :::note From d03fcb5ff121203f9cd6bf729df98764593328fe Mon Sep 17 00:00:00 2001 From: avogar Date: Wed, 3 Jul 2024 14:23:38 +0000 Subject: [PATCH 153/417] Fix --- src/Formats/SchemaInferenceUtils.cpp | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/src/Formats/SchemaInferenceUtils.cpp b/src/Formats/SchemaInferenceUtils.cpp index 6519d54a8c5..f2ad1dc6717 100644 --- a/src/Formats/SchemaInferenceUtils.cpp +++ b/src/Formats/SchemaInferenceUtils.cpp @@ -271,7 +271,7 @@ namespace { if (WhichDataType(type).isInt64()) { - bool is_negative = json_info->negative_integers.contains(type.get()); + bool is_negative = json_info && json_info->negative_integers.contains(type.get()); have_negative_integers |= is_negative; if (!is_negative) type = std::make_shared(); From 2e5acb2bdd929928cc9e47ff7038a1775f0ab463 Mon Sep 17 00:00:00 2001 From: Nikita Fomichev Date: Wed, 3 Jul 2024 16:33:58 +0200 Subject: [PATCH 154/417] Tests: replace openssl base64 with base64 --- .../0_stateless/03167_base64_url_functions_sh.sh | 12 ++++-------- 1 file changed, 4 insertions(+), 8 deletions(-) diff --git a/tests/queries/0_stateless/03167_base64_url_functions_sh.sh b/tests/queries/0_stateless/03167_base64_url_functions_sh.sh index 12eea7462df..ec3170b165c 100755 --- a/tests/queries/0_stateless/03167_base64_url_functions_sh.sh +++ b/tests/queries/0_stateless/03167_base64_url_functions_sh.sh @@ -119,7 +119,7 @@ urls=( base64URLEncode() { - echo -n "$1" | openssl base64 -e -A | tr '+/' '-_' | tr -d '=' + echo -n "$1" | base64 -w0 | tr '+/' '-_' | tr -d '=' } base64URLDecode() { @@ -128,7 +128,7 @@ base64URLDecode() { if [ $len -eq 2 ]; then result="$1"'==' elif [ $len -eq 3 ]; then result="$1"'=' fi - echo "$result" | tr '_-' '/+' | openssl base64 -d -A + echo "$result" | tr '_-' '/+' | base64 -w0 -d } test_compare_to_gold_encode() { @@ -158,12 +158,10 @@ test_compare_to_gold_decode() { test_compare_to_self() { local input="$1" - local encode=$(${CLICKHOUSE_CLIENT} --query="SELECT base64URLEncode('$input')") - local decode=$(${CLICKHOUSE_CLIENT} --query="SELECT base64URLDecode('$encode')") + local decode=$(${CLICKHOUSE_CLIENT} --query="SELECT base64URLDecode(base64URLEncode('$input'))") if [ "$decode" != "$input" ]; then echo "Input: $input" - echo "Encode: $encode" echo "Got: $decode" fi } @@ -181,10 +179,8 @@ for url in "${urls[@]}"; do done # special case for ' -encode=$(${CLICKHOUSE_CLIENT} --query="SELECT base64URLEncode('http://example.com/!$&\'()*+,;=:@/path')") -decode=$(${CLICKHOUSE_CLIENT} --query="SELECT base64URLDecode('$encode')") +decode=$(${CLICKHOUSE_CLIENT} --query="SELECT base64URLDecode(base64URLEncode('http://example.com/!$&\'()*+,;=:@/path'))") if [ "$decode" != "http://example.com/!$&\'()*+,;=:@/path" ]; then echo "Special case fail" - echo "Encode: $encode" echo "Got: $decode" fi From 6530ae104d16ffbda51cf849b5f89e3d4080d2af Mon Sep 17 00:00:00 2001 From: avogar Date: Wed, 3 Jul 2024 15:23:01 +0000 Subject: [PATCH 155/417] Fix tests --- src/Formats/SchemaInferenceUtils.cpp | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/src/Formats/SchemaInferenceUtils.cpp b/src/Formats/SchemaInferenceUtils.cpp index f2ad1dc6717..3c374ada9e6 100644 --- a/src/Formats/SchemaInferenceUtils.cpp +++ b/src/Formats/SchemaInferenceUtils.cpp @@ -296,7 +296,7 @@ namespace if (which.isInt64() || which.isUInt64()) { auto new_type = std::make_shared(); - if (json_info->numbers_parsed_from_json_strings.erase(type.get())) + if (json_info && json_info->numbers_parsed_from_json_strings.erase(type.get())) json_info->numbers_parsed_from_json_strings.insert(new_type.get()); type = new_type; } From af845234a2666dbfe16e6877a6c6d111e26c2f44 Mon Sep 17 00:00:00 2001 From: Nikita Fomichev Date: Wed, 3 Jul 2024 17:53:05 +0200 Subject: [PATCH 156/417] Review fixes --- docker/test/stateless/run.sh | 4 +++- tests/clickhouse-test | 12 ++++++++++++ 2 files changed, 15 insertions(+), 1 deletion(-) diff --git a/docker/test/stateless/run.sh b/docker/test/stateless/run.sh index 2cbc5304212..96436beaf0c 100755 --- a/docker/test/stateless/run.sh +++ b/docker/test/stateless/run.sh @@ -265,7 +265,9 @@ function run_tests() export -f run_tests -TIMEOUT=$((${MAX_RUN_TIME} - 200)) + +# This should be enough to setup job and collect artifacts +TIMEOUT=$((MAX_RUN_TIME - 300)) if [ "$NUM_TRIES" -gt "1" ]; then # We don't run tests with Ordinary database in PRs, only in master. # So run new/changed tests with Ordinary at least once in flaky check. diff --git a/tests/clickhouse-test b/tests/clickhouse-test index 8e2a256fae2..c2acdb715ea 100755 --- a/tests/clickhouse-test +++ b/tests/clickhouse-test @@ -358,6 +358,14 @@ def clickhouse_execute_json( return rows +def stop_tests(): + # send signal to all processes in group to avoid hung check triggering + # (to avoid terminating clickhouse-test itself, the signal should be ignored) + signal.signal(signal.SIGTERM, signal.SIG_IGN) + os.killpg(os.getpgid(os.getpid()), signal.SIGTERM) + signal.signal(signal.SIGTERM, signal.SIG_DFL) + + def get_db_engine(args, database_name): if args.replicated_database: return f" ON CLUSTER test_cluster_database_replicated \ @@ -2098,10 +2106,12 @@ def run_tests_array(all_tests_with_params: Tuple[List[str], int, TestSuite]): break if server_died.is_set(): + stop_tests() raise ServerDied("Server died") if stop_time and time() > stop_time: print("\nStop tests run because global time limit is exceeded.\n") + stop_tests() raise GlobalTimeout("Stop tests run because global time limit is exceeded") test_case = TestCase(test_suite, case, args, is_concurrent) @@ -2150,9 +2160,11 @@ def run_tests_array(all_tests_with_params: Tuple[List[str], int, TestSuite]): except KeyboardInterrupt as e: print(colored("Break tests execution", args, "red")) + stop_tests() raise e if failures_chain >= args.max_failures_chain: + stop_tests() raise ServerDied("Max failures chain") if failures_total > 0: From 04eb07b453ced85f4dfeb0b12e16777e22055156 Mon Sep 17 00:00:00 2001 From: pufit Date: Wed, 3 Jul 2024 12:55:16 -0400 Subject: [PATCH 157/417] Update startup-scripts.md --- docs/en/operations/startup-scripts.md | 3 +-- 1 file changed, 1 insertion(+), 2 deletions(-) diff --git a/docs/en/operations/startup-scripts.md b/docs/en/operations/startup-scripts.md index c7842c1472b..91aa4772bcf 100644 --- a/docs/en/operations/startup-scripts.md +++ b/docs/en/operations/startup-scripts.md @@ -1,6 +1,5 @@ --- -slug: /en/operations/startup-scripts.md -sidebar_position: 70 +slug: /en/operations/startup-scripts sidebar_label: Startup Scripts --- From a5adf31b9e4dfa041150fa263ab68f32cb47122c Mon Sep 17 00:00:00 2001 From: Kruglov Pavel <48961922+Avogar@users.noreply.github.com> Date: Wed, 3 Jul 2024 19:30:36 +0200 Subject: [PATCH 158/417] Fix special build --- src/Formats/JSONExtractTree.cpp | 6 +++--- 1 file changed, 3 insertions(+), 3 deletions(-) diff --git a/src/Formats/JSONExtractTree.cpp b/src/Formats/JSONExtractTree.cpp index 18437c16bc9..b94981e7cb4 100644 --- a/src/Formats/JSONExtractTree.cpp +++ b/src/Formats/JSONExtractTree.cpp @@ -1465,9 +1465,9 @@ std::unique_ptr> buildJSONExtractTree(const Data case TypeIndex::LowCardinality: { /// To optimize inserting into LowCardinality we have special nodes for LowCardinality of numeric and string types. - auto lc_type = typeid_cast(type.get()); - auto dictionary_type = removeNullable(lc_type->getDictionaryType()); - bool is_nullable = lc_type->isLowCardinalityNullable(); + const auto & lc_type = assert_cast(*type)); + auto dictionary_type = removeNullable(lc_type.getDictionaryType()); + bool is_nullable = lc_type.isLowCardinalityNullable(); switch (dictionary_type->getTypeId()) { From a19eb8686492baee1084e44217f3e6c2f8594d54 Mon Sep 17 00:00:00 2001 From: Nikita Fomichev Date: Wed, 3 Jul 2024 19:39:00 +0200 Subject: [PATCH 159/417] Review fixes 2 --- docker/test/fasttest/run.sh | 2 ++ 1 file changed, 2 insertions(+) diff --git a/docker/test/fasttest/run.sh b/docker/test/fasttest/run.sh index 2bed4c5c343..c80ea193010 100755 --- a/docker/test/fasttest/run.sh +++ b/docker/test/fasttest/run.sh @@ -305,6 +305,8 @@ function run_tests clickhouse stop --pid-path "$FASTTEST_DATA" } +export -f run_tests + case "$stage" in "") ls -la From bbbf6cd6f9fef235a29fcb059bdd1a6833fe146a Mon Sep 17 00:00:00 2001 From: Nikita Taranov Date: Wed, 3 Jul 2024 19:41:05 +0100 Subject: [PATCH 160/417] remove todo --- src/Interpreters/HashTablesStatistics.h | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/src/Interpreters/HashTablesStatistics.h b/src/Interpreters/HashTablesStatistics.h index 73dd3c097d4..7b4c4fcbfeb 100644 --- a/src/Interpreters/HashTablesStatistics.h +++ b/src/Interpreters/HashTablesStatistics.h @@ -27,7 +27,7 @@ struct StatsCollectingParams void disable() { key = 0; } UInt64 key = 0; - const size_t max_entries_for_hash_table_stats = 0; /// TODO: move to server settings + const size_t max_entries_for_hash_table_stats = 0; const size_t max_size_to_preallocate = 0; }; From 29ed1c9bf8aec7ea41394df5fad2c28d6ed662ba Mon Sep 17 00:00:00 2001 From: Vitaly Baranov Date: Thu, 29 Feb 2024 13:00:38 +0100 Subject: [PATCH 161/417] Move function exceptionCodeToHTTPStatus() from HTTPHandler.cpp to a separate header in order to reuse it. --- src/Server/HTTP/exceptionCodeToHTTPStatus.cpp | 158 ++++++++++++++++++ src/Server/HTTP/exceptionCodeToHTTPStatus.h | 11 ++ src/Server/HTTPHandler.cpp | 139 +-------------- 3 files changed, 170 insertions(+), 138 deletions(-) create mode 100644 src/Server/HTTP/exceptionCodeToHTTPStatus.cpp create mode 100644 src/Server/HTTP/exceptionCodeToHTTPStatus.h diff --git a/src/Server/HTTP/exceptionCodeToHTTPStatus.cpp b/src/Server/HTTP/exceptionCodeToHTTPStatus.cpp new file mode 100644 index 00000000000..6de57217aac --- /dev/null +++ b/src/Server/HTTP/exceptionCodeToHTTPStatus.cpp @@ -0,0 +1,158 @@ +#include + + +namespace DB +{ + +namespace ErrorCodes +{ + extern const int BAD_ARGUMENTS; + extern const int CANNOT_PARSE_TEXT; + extern const int CANNOT_PARSE_ESCAPE_SEQUENCE; + extern const int CANNOT_PARSE_QUOTED_STRING; + extern const int CANNOT_PARSE_DATE; + extern const int CANNOT_PARSE_DATETIME; + extern const int CANNOT_PARSE_NUMBER; + extern const int CANNOT_PARSE_DOMAIN_VALUE_FROM_STRING; + extern const int CANNOT_PARSE_IPV4; + extern const int CANNOT_PARSE_IPV6; + extern const int CANNOT_PARSE_UUID; + extern const int CANNOT_PARSE_INPUT_ASSERTION_FAILED; + extern const int CANNOT_SCHEDULE_TASK; + extern const int CANNOT_OPEN_FILE; + extern const int CANNOT_COMPILE_REGEXP; + extern const int DUPLICATE_COLUMN; + extern const int ILLEGAL_COLUMN; + extern const int THERE_IS_NO_COLUMN; + extern const int UNKNOWN_ELEMENT_IN_AST; + extern const int UNKNOWN_TYPE_OF_AST_NODE; + extern const int TOO_DEEP_AST; + extern const int TOO_BIG_AST; + extern const int UNEXPECTED_AST_STRUCTURE; + extern const int VALUE_IS_OUT_OF_RANGE_OF_DATA_TYPE; + + extern const int SYNTAX_ERROR; + + extern const int INCORRECT_DATA; + extern const int TYPE_MISMATCH; + + extern const int UNKNOWN_TABLE; + extern const int UNKNOWN_FUNCTION; + extern const int UNKNOWN_IDENTIFIER; + extern const int UNKNOWN_TYPE; + extern const int UNKNOWN_STORAGE; + extern const int UNKNOWN_DATABASE; + extern const int UNKNOWN_SETTING; + extern const int UNKNOWN_DIRECTION_OF_SORTING; + extern const int UNKNOWN_AGGREGATE_FUNCTION; + extern const int UNKNOWN_FORMAT; + extern const int UNKNOWN_DATABASE_ENGINE; + extern const int UNKNOWN_TYPE_OF_QUERY; + extern const int UNKNOWN_ROLE; + + extern const int QUERY_IS_TOO_LARGE; + + extern const int NOT_IMPLEMENTED; + extern const int SOCKET_TIMEOUT; + + extern const int UNKNOWN_USER; + extern const int WRONG_PASSWORD; + extern const int REQUIRED_PASSWORD; + extern const int AUTHENTICATION_FAILED; + extern const int SET_NON_GRANTED_ROLE; + + extern const int HTTP_LENGTH_REQUIRED; + + extern const int TIMEOUT_EXCEEDED; +} + + +Poco::Net::HTTPResponse::HTTPStatus exceptionCodeToHTTPStatus(int exception_code) +{ + using namespace Poco::Net; + + if (exception_code == ErrorCodes::REQUIRED_PASSWORD) + { + return HTTPResponse::HTTP_UNAUTHORIZED; + } + else if (exception_code == ErrorCodes::UNKNOWN_USER || + exception_code == ErrorCodes::WRONG_PASSWORD || + exception_code == ErrorCodes::AUTHENTICATION_FAILED || + exception_code == ErrorCodes::SET_NON_GRANTED_ROLE) + { + return HTTPResponse::HTTP_FORBIDDEN; + } + else if (exception_code == ErrorCodes::BAD_ARGUMENTS || + exception_code == ErrorCodes::CANNOT_COMPILE_REGEXP || + exception_code == ErrorCodes::CANNOT_PARSE_TEXT || + exception_code == ErrorCodes::CANNOT_PARSE_ESCAPE_SEQUENCE || + exception_code == ErrorCodes::CANNOT_PARSE_QUOTED_STRING || + exception_code == ErrorCodes::CANNOT_PARSE_DATE || + exception_code == ErrorCodes::CANNOT_PARSE_DATETIME || + exception_code == ErrorCodes::CANNOT_PARSE_NUMBER || + exception_code == ErrorCodes::CANNOT_PARSE_DOMAIN_VALUE_FROM_STRING || + exception_code == ErrorCodes::CANNOT_PARSE_IPV4 || + exception_code == ErrorCodes::CANNOT_PARSE_IPV6 || + exception_code == ErrorCodes::CANNOT_PARSE_INPUT_ASSERTION_FAILED || + exception_code == ErrorCodes::CANNOT_PARSE_UUID || + exception_code == ErrorCodes::DUPLICATE_COLUMN || + exception_code == ErrorCodes::ILLEGAL_COLUMN || + exception_code == ErrorCodes::UNKNOWN_ELEMENT_IN_AST || + exception_code == ErrorCodes::UNKNOWN_TYPE_OF_AST_NODE || + exception_code == ErrorCodes::THERE_IS_NO_COLUMN || + exception_code == ErrorCodes::TOO_DEEP_AST || + exception_code == ErrorCodes::TOO_BIG_AST || + exception_code == ErrorCodes::UNEXPECTED_AST_STRUCTURE || + exception_code == ErrorCodes::SYNTAX_ERROR || + exception_code == ErrorCodes::INCORRECT_DATA || + exception_code == ErrorCodes::TYPE_MISMATCH || + exception_code == ErrorCodes::VALUE_IS_OUT_OF_RANGE_OF_DATA_TYPE) + { + return HTTPResponse::HTTP_BAD_REQUEST; + } + else if (exception_code == ErrorCodes::UNKNOWN_TABLE || + exception_code == ErrorCodes::UNKNOWN_FUNCTION || + exception_code == ErrorCodes::UNKNOWN_IDENTIFIER || + exception_code == ErrorCodes::UNKNOWN_TYPE || + exception_code == ErrorCodes::UNKNOWN_STORAGE || + exception_code == ErrorCodes::UNKNOWN_DATABASE || + exception_code == ErrorCodes::UNKNOWN_SETTING || + exception_code == ErrorCodes::UNKNOWN_DIRECTION_OF_SORTING || + exception_code == ErrorCodes::UNKNOWN_AGGREGATE_FUNCTION || + exception_code == ErrorCodes::UNKNOWN_FORMAT || + exception_code == ErrorCodes::UNKNOWN_DATABASE_ENGINE || + exception_code == ErrorCodes::UNKNOWN_TYPE_OF_QUERY || + exception_code == ErrorCodes::UNKNOWN_ROLE) + { + return HTTPResponse::HTTP_NOT_FOUND; + } + else if (exception_code == ErrorCodes::QUERY_IS_TOO_LARGE) + { + return HTTPResponse::HTTP_REQUESTENTITYTOOLARGE; + } + else if (exception_code == ErrorCodes::NOT_IMPLEMENTED) + { + return HTTPResponse::HTTP_NOT_IMPLEMENTED; + } + else if (exception_code == ErrorCodes::SOCKET_TIMEOUT || + exception_code == ErrorCodes::CANNOT_OPEN_FILE) + { + return HTTPResponse::HTTP_SERVICE_UNAVAILABLE; + } + else if (exception_code == ErrorCodes::HTTP_LENGTH_REQUIRED) + { + return HTTPResponse::HTTP_LENGTH_REQUIRED; + } + else if (exception_code == ErrorCodes::TIMEOUT_EXCEEDED) + { + return HTTPResponse::HTTP_REQUEST_TIMEOUT; + } + else if (exception_code == ErrorCodes::CANNOT_SCHEDULE_TASK) + { + return HTTPResponse::HTTP_SERVICE_UNAVAILABLE; + } + + return HTTPResponse::HTTP_INTERNAL_SERVER_ERROR; +} + +} diff --git a/src/Server/HTTP/exceptionCodeToHTTPStatus.h b/src/Server/HTTP/exceptionCodeToHTTPStatus.h new file mode 100644 index 00000000000..aadec7aac5a --- /dev/null +++ b/src/Server/HTTP/exceptionCodeToHTTPStatus.h @@ -0,0 +1,11 @@ +#pragma once +#include + + +namespace DB +{ + +/// Converts Exception code to HTTP status code. +Poco::Net::HTTPResponse::HTTPStatus exceptionCodeToHTTPStatus(int exception_code); + +} diff --git a/src/Server/HTTPHandler.cpp b/src/Server/HTTPHandler.cpp index a00f6fb5412..6f51a269e0e 100644 --- a/src/Server/HTTPHandler.cpp +++ b/src/Server/HTTPHandler.cpp @@ -37,6 +37,7 @@ #include #include #include +#include #include #include @@ -73,56 +74,9 @@ namespace ErrorCodes extern const int BAD_ARGUMENTS; extern const int LOGICAL_ERROR; extern const int CANNOT_COMPILE_REGEXP; - extern const int CANNOT_OPEN_FILE; - extern const int CANNOT_PARSE_TEXT; - extern const int CANNOT_PARSE_ESCAPE_SEQUENCE; - extern const int CANNOT_PARSE_QUOTED_STRING; - extern const int CANNOT_PARSE_DATE; - extern const int CANNOT_PARSE_DATETIME; - extern const int CANNOT_PARSE_NUMBER; - extern const int CANNOT_PARSE_DOMAIN_VALUE_FROM_STRING; - extern const int CANNOT_PARSE_IPV4; - extern const int CANNOT_PARSE_IPV6; - extern const int CANNOT_PARSE_UUID; - extern const int CANNOT_PARSE_INPUT_ASSERTION_FAILED; - extern const int CANNOT_SCHEDULE_TASK; - extern const int DUPLICATE_COLUMN; - extern const int ILLEGAL_COLUMN; - extern const int THERE_IS_NO_COLUMN; - extern const int UNKNOWN_ELEMENT_IN_AST; - extern const int UNKNOWN_TYPE_OF_AST_NODE; - extern const int TOO_DEEP_AST; - extern const int TOO_BIG_AST; - extern const int UNEXPECTED_AST_STRUCTURE; - extern const int VALUE_IS_OUT_OF_RANGE_OF_DATA_TYPE; - extern const int SYNTAX_ERROR; - - extern const int INCORRECT_DATA; - extern const int TYPE_MISMATCH; - - extern const int UNKNOWN_TABLE; - extern const int UNKNOWN_FUNCTION; - extern const int UNKNOWN_IDENTIFIER; - extern const int UNKNOWN_TYPE; - extern const int UNKNOWN_STORAGE; - extern const int UNKNOWN_DATABASE; - extern const int UNKNOWN_SETTING; - extern const int UNKNOWN_DIRECTION_OF_SORTING; - extern const int UNKNOWN_AGGREGATE_FUNCTION; - extern const int UNKNOWN_FORMAT; - extern const int UNKNOWN_DATABASE_ENGINE; - extern const int UNKNOWN_TYPE_OF_QUERY; - extern const int UNKNOWN_ROLE; extern const int NO_ELEMENTS_IN_CONFIG; - extern const int QUERY_IS_TOO_LARGE; - - extern const int NOT_IMPLEMENTED; - extern const int SOCKET_TIMEOUT; - - extern const int UNKNOWN_USER; - extern const int WRONG_PASSWORD; extern const int REQUIRED_PASSWORD; extern const int AUTHENTICATION_FAILED; extern const int SET_NON_GRANTED_ROLE; @@ -130,8 +84,6 @@ namespace ErrorCodes extern const int INVALID_SESSION_TIMEOUT; extern const int HTTP_LENGTH_REQUIRED; extern const int SUPPORT_IS_DISABLED; - - extern const int TIMEOUT_EXCEEDED; } namespace @@ -193,95 +145,6 @@ static String base64Encode(const String & decoded) return ostr.str(); } -static Poco::Net::HTTPResponse::HTTPStatus exceptionCodeToHTTPStatus(int exception_code) -{ - using namespace Poco::Net; - - if (exception_code == ErrorCodes::REQUIRED_PASSWORD) - { - return HTTPResponse::HTTP_UNAUTHORIZED; - } - else if (exception_code == ErrorCodes::UNKNOWN_USER || - exception_code == ErrorCodes::WRONG_PASSWORD || - exception_code == ErrorCodes::AUTHENTICATION_FAILED || - exception_code == ErrorCodes::SET_NON_GRANTED_ROLE) - { - return HTTPResponse::HTTP_FORBIDDEN; - } - else if (exception_code == ErrorCodes::BAD_ARGUMENTS || - exception_code == ErrorCodes::CANNOT_COMPILE_REGEXP || - exception_code == ErrorCodes::CANNOT_PARSE_TEXT || - exception_code == ErrorCodes::CANNOT_PARSE_ESCAPE_SEQUENCE || - exception_code == ErrorCodes::CANNOT_PARSE_QUOTED_STRING || - exception_code == ErrorCodes::CANNOT_PARSE_DATE || - exception_code == ErrorCodes::CANNOT_PARSE_DATETIME || - exception_code == ErrorCodes::CANNOT_PARSE_NUMBER || - exception_code == ErrorCodes::CANNOT_PARSE_DOMAIN_VALUE_FROM_STRING || - exception_code == ErrorCodes::CANNOT_PARSE_IPV4 || - exception_code == ErrorCodes::CANNOT_PARSE_IPV6 || - exception_code == ErrorCodes::CANNOT_PARSE_INPUT_ASSERTION_FAILED || - exception_code == ErrorCodes::CANNOT_PARSE_UUID || - exception_code == ErrorCodes::DUPLICATE_COLUMN || - exception_code == ErrorCodes::ILLEGAL_COLUMN || - exception_code == ErrorCodes::UNKNOWN_ELEMENT_IN_AST || - exception_code == ErrorCodes::UNKNOWN_TYPE_OF_AST_NODE || - exception_code == ErrorCodes::THERE_IS_NO_COLUMN || - exception_code == ErrorCodes::TOO_DEEP_AST || - exception_code == ErrorCodes::TOO_BIG_AST || - exception_code == ErrorCodes::UNEXPECTED_AST_STRUCTURE || - exception_code == ErrorCodes::SYNTAX_ERROR || - exception_code == ErrorCodes::INCORRECT_DATA || - exception_code == ErrorCodes::TYPE_MISMATCH || - exception_code == ErrorCodes::VALUE_IS_OUT_OF_RANGE_OF_DATA_TYPE) - { - return HTTPResponse::HTTP_BAD_REQUEST; - } - else if (exception_code == ErrorCodes::UNKNOWN_TABLE || - exception_code == ErrorCodes::UNKNOWN_FUNCTION || - exception_code == ErrorCodes::UNKNOWN_IDENTIFIER || - exception_code == ErrorCodes::UNKNOWN_TYPE || - exception_code == ErrorCodes::UNKNOWN_STORAGE || - exception_code == ErrorCodes::UNKNOWN_DATABASE || - exception_code == ErrorCodes::UNKNOWN_SETTING || - exception_code == ErrorCodes::UNKNOWN_DIRECTION_OF_SORTING || - exception_code == ErrorCodes::UNKNOWN_AGGREGATE_FUNCTION || - exception_code == ErrorCodes::UNKNOWN_FORMAT || - exception_code == ErrorCodes::UNKNOWN_DATABASE_ENGINE || - exception_code == ErrorCodes::UNKNOWN_TYPE_OF_QUERY || - exception_code == ErrorCodes::UNKNOWN_ROLE) - { - return HTTPResponse::HTTP_NOT_FOUND; - } - else if (exception_code == ErrorCodes::QUERY_IS_TOO_LARGE) - { - return HTTPResponse::HTTP_REQUESTENTITYTOOLARGE; - } - else if (exception_code == ErrorCodes::NOT_IMPLEMENTED) - { - return HTTPResponse::HTTP_NOT_IMPLEMENTED; - } - else if (exception_code == ErrorCodes::SOCKET_TIMEOUT || - exception_code == ErrorCodes::CANNOT_OPEN_FILE) - { - return HTTPResponse::HTTP_SERVICE_UNAVAILABLE; - } - else if (exception_code == ErrorCodes::HTTP_LENGTH_REQUIRED) - { - return HTTPResponse::HTTP_LENGTH_REQUIRED; - } - else if (exception_code == ErrorCodes::TIMEOUT_EXCEEDED) - { - return HTTPResponse::HTTP_REQUEST_TIMEOUT; - } - else if (exception_code == ErrorCodes::CANNOT_SCHEDULE_TASK) - { - return HTTPResponse::HTTP_SERVICE_UNAVAILABLE; - } - - return HTTPResponse::HTTP_INTERNAL_SERVER_ERROR; -} - - static std::chrono::steady_clock::duration parseSessionTimeout( const Poco::Util::AbstractConfiguration & config, const HTMLForm & params) From bf56ad69dde18f90b8d96e00d39e9576d4188c9e Mon Sep 17 00:00:00 2001 From: Vitaly Baranov Date: Wed, 8 May 2024 20:58:28 +0200 Subject: [PATCH 162/417] Make Context::setCurrentRoles() check that new current roles are granted in order to make HTTPHandler's implementation easier. --- .../include/Poco/Net/NameValueCollection.h | 2 +- base/poco/Net/src/NameValueCollection.cpp | 4 +- .../Access/InterpreterSetRoleQuery.cpp | 23 +------ src/Interpreters/AsynchronousInsertQueue.cpp | 5 +- src/Interpreters/Context.cpp | 65 +++++++++++++++---- src/Interpreters/Context.h | 10 ++- src/Server/HTTPHandler.cpp | 19 +----- 7 files changed, 70 insertions(+), 58 deletions(-) diff --git a/base/poco/Net/include/Poco/Net/NameValueCollection.h b/base/poco/Net/include/Poco/Net/NameValueCollection.h index be499838d0e..b66819840b1 100644 --- a/base/poco/Net/include/Poco/Net/NameValueCollection.h +++ b/base/poco/Net/include/Poco/Net/NameValueCollection.h @@ -79,7 +79,7 @@ namespace Net /// Returns the value of the first name-value pair with the given name. /// If no value with the given name has been found, the defaultValue is returned. - const std::vector> getAll(const std::string & name) const; + const std::vector getAll(const std::string & name) const; /// Returns all values of all name-value pairs with the given name. /// /// Returns an empty vector if there are no name-value pairs with the given name. diff --git a/base/poco/Net/src/NameValueCollection.cpp b/base/poco/Net/src/NameValueCollection.cpp index 783ed48cc30..0cb31924579 100644 --- a/base/poco/Net/src/NameValueCollection.cpp +++ b/base/poco/Net/src/NameValueCollection.cpp @@ -102,9 +102,9 @@ const std::string& NameValueCollection::get(const std::string& name, const std:: return defaultValue; } -const std::vector> NameValueCollection::getAll(const std::string& name) const +const std::vector NameValueCollection::getAll(const std::string& name) const { - std::vector> values; + std::vector values; for (ConstIterator it = _map.find(name); it != _map.end(); it++) if (it->first == name) values.push_back(it->second); diff --git a/src/Interpreters/Access/InterpreterSetRoleQuery.cpp b/src/Interpreters/Access/InterpreterSetRoleQuery.cpp index 24467923542..99a7a73d46c 100644 --- a/src/Interpreters/Access/InterpreterSetRoleQuery.cpp +++ b/src/Interpreters/Access/InterpreterSetRoleQuery.cpp @@ -29,33 +29,12 @@ BlockIO InterpreterSetRoleQuery::execute() void InterpreterSetRoleQuery::setRole(const ASTSetRoleQuery & query) { - auto & access_control = getContext()->getAccessControl(); auto session_context = getContext()->getSessionContext(); - auto user = session_context->getUser(); if (query.kind == ASTSetRoleQuery::Kind::SET_ROLE_DEFAULT) - { session_context->setCurrentRolesDefault(); - } else - { - RolesOrUsersSet roles_from_query{*query.roles, access_control}; - std::vector new_current_roles; - if (roles_from_query.all) - { - new_current_roles = user->granted_roles.findGranted(roles_from_query); - } - else - { - for (const auto & id : roles_from_query.getMatchingIDs()) - { - if (!user->granted_roles.isGranted(id)) - throw Exception(ErrorCodes::SET_NON_GRANTED_ROLE, "Role should be granted to set current"); - new_current_roles.emplace_back(id); - } - } - session_context->setCurrentRoles(new_current_roles); - } + session_context->setCurrentRoles(RolesOrUsersSet{*query.roles, session_context->getAccessControl()}); } diff --git a/src/Interpreters/AsynchronousInsertQueue.cpp b/src/Interpreters/AsynchronousInsertQueue.cpp index d72f3d81549..56055e7044c 100644 --- a/src/Interpreters/AsynchronousInsertQueue.cpp +++ b/src/Interpreters/AsynchronousInsertQueue.cpp @@ -726,7 +726,10 @@ try /// Access rights must be checked for the user who executed the initial INSERT query. if (key.user_id) - insert_context->setUser(*key.user_id, key.current_roles); + { + insert_context->setUser(*key.user_id); + insert_context->setCurrentRoles(key.current_roles); + } insert_context->setSettings(key.settings); diff --git a/src/Interpreters/Context.cpp b/src/Interpreters/Context.cpp index b946c2cb21e..856dac28ed1 100644 --- a/src/Interpreters/Context.cpp +++ b/src/Interpreters/Context.cpp @@ -58,6 +58,7 @@ #include #include #include +#include #include #include #include @@ -190,6 +191,7 @@ namespace ErrorCodes extern const int ILLEGAL_COLUMN; extern const int NUMBER_OF_COLUMNS_DOESNT_MATCH; extern const int CLUSTER_DOESNT_EXIST; + extern const int SET_NON_GRANTED_ROLE; } #define SHUTDOWN(log, desc, ptr, method) do \ @@ -1303,7 +1305,7 @@ ConfigurationPtr Context::getUsersConfig() return shared->users_config; } -void Context::setUser(const UUID & user_id_, const std::optional> & current_roles_) +void Context::setUser(const UUID & user_id_) { /// Prepare lists of user's profiles, constraints, settings, roles. /// NOTE: AccessControl::read() and other AccessControl's functions may require some IO work, @@ -1312,8 +1314,8 @@ void Context::setUser(const UUID & user_id_, const std::optional(user_id_); - auto new_current_roles = current_roles_ ? user->granted_roles.findGranted(*current_roles_) : user->granted_roles.findGranted(user->default_roles); - auto enabled_roles = access_control.getEnabledRolesInfo(new_current_roles, {}); + auto default_roles = user->granted_roles.findGranted(user->default_roles); + auto enabled_roles = access_control.getEnabledRolesInfo(default_roles, {}); auto enabled_profiles = access_control.getEnabledSettingsInfo(user_id_, user->settings, enabled_roles->enabled_roles, enabled_roles->settings_from_enabled_roles); const auto & database = user->default_database; @@ -1327,7 +1329,7 @@ void Context::setUser(const UUID & user_id_, const std::optional Context::getUserID() const return user_id; } -void Context::setCurrentRolesWithLock(const std::vector & current_roles_, const std::lock_guard &) +void Context::setCurrentRolesWithLock(const std::vector & new_current_roles, const std::lock_guard &) { - if (current_roles_.empty()) + if (new_current_roles.empty()) current_roles = nullptr; else - current_roles = std::make_shared>(current_roles_); + current_roles = std::make_shared>(new_current_roles); need_recalculate_access = true; } -void Context::setCurrentRoles(const std::vector & current_roles_) +void Context::setCurrentRolesImpl(const std::vector & new_current_roles, bool throw_if_not_granted, bool skip_if_not_granted, const std::shared_ptr & user) { - std::lock_guard lock(mutex); - setCurrentRolesWithLock(current_roles_, lock); + if (skip_if_not_granted) + { + auto filtered_role_ids = user->granted_roles.findGranted(new_current_roles); + std::lock_guard lock{mutex}; + setCurrentRolesWithLock(filtered_role_ids, lock); + return; + } + if (throw_if_not_granted) + { + for (const auto & role_id : new_current_roles) + { + if (!user->granted_roles.isGranted(role_id)) + { + auto role_name = getAccessControl().tryReadName(role_id); + throw Exception(ErrorCodes::SET_NON_GRANTED_ROLE, "Role {} should be granted to set as a current", role_name.value_or(toString(role_id))); + } + } + } + std::lock_guard lock2{mutex}; + setCurrentRolesWithLock(new_current_roles, lock2); +} + +void Context::setCurrentRoles(const std::vector & new_current_roles, bool check_grants) +{ + setCurrentRolesImpl(new_current_roles, /* throw_if_not_granted= */ check_grants, /* skip_if_not_granted= */ !check_grants, getUser()); +} + +void Context::setCurrentRoles(const RolesOrUsersSet & new_current_roles, bool check_grants) +{ + if (new_current_roles.all) + { + auto user = getUser(); + setCurrentRolesImpl(user->granted_roles.findGranted(new_current_roles), /* throw_if_not_granted= */ false, /* skip_if_not_granted= */ false, user); + } + else + { + setCurrentRoles(new_current_roles.getMatchingIDs(), check_grants); + } +} + +void Context::setCurrentRoles(const Strings & new_current_roles, bool check_grants) +{ + setCurrentRoles(getAccessControl().getIDs(new_current_roles), check_grants); } void Context::setCurrentRolesDefault() { auto user = getUser(); - setCurrentRoles(user->granted_roles.findGranted(user->default_roles)); + setCurrentRolesImpl(user->granted_roles.findGranted(user->default_roles), /* throw_if_not_granted= */ false, /* skip_if_not_granted= */ false, user); } std::vector Context::getCurrentRoles() const diff --git a/src/Interpreters/Context.h b/src/Interpreters/Context.h index f9b91a45978..b6de166e27a 100644 --- a/src/Interpreters/Context.h +++ b/src/Interpreters/Context.h @@ -63,6 +63,7 @@ class AccessFlags; struct AccessRightsElement; class AccessRightsElements; enum class RowPolicyFilterType : uint8_t; +struct RolesOrUsersSet; class EmbeddedDictionaries; class ExternalDictionariesLoader; class ExternalUserDefinedExecutableFunctionsLoader; @@ -602,13 +603,15 @@ public: /// Sets the current user assuming that he/she is already authenticated. /// WARNING: This function doesn't check password! - void setUser(const UUID & user_id_, const std::optional> & current_roles_ = {}); + void setUser(const UUID & user_id_); UserPtr getUser() const; std::optional getUserID() const; String getUserName() const; - void setCurrentRoles(const std::vector & current_roles_); + void setCurrentRoles(const Strings & new_current_roles, bool check_grants = true); + void setCurrentRoles(const std::vector & new_current_roles, bool check_grants = true); + void setCurrentRoles(const RolesOrUsersSet & new_current_roles, bool check_grants = true); void setCurrentRolesDefault(); std::vector getCurrentRoles() const; std::vector getEnabledRoles() const; @@ -1335,7 +1338,7 @@ private: void setCurrentProfilesWithLock(const SettingsProfilesInfo & profiles_info, bool check_constraints, const std::lock_guard & lock); - void setCurrentRolesWithLock(const std::vector & current_roles_, const std::lock_guard & lock); + void setCurrentRolesWithLock(const std::vector & new_current_roles, const std::lock_guard & lock); void setSettingWithLock(std::string_view name, const String & value, const std::lock_guard & lock); @@ -1368,6 +1371,7 @@ private: void initGlobal(); void setUserID(const UUID & user_id_); + void setCurrentRolesImpl(const std::vector & new_current_roles, bool throw_if_not_granted, bool skip_if_not_granted, const std::shared_ptr & user); template void checkAccessImpl(const Args &... args) const; diff --git a/src/Server/HTTPHandler.cpp b/src/Server/HTTPHandler.cpp index 6f51a269e0e..5d3e6aa1b05 100644 --- a/src/Server/HTTPHandler.cpp +++ b/src/Server/HTTPHandler.cpp @@ -2,10 +2,7 @@ #include #include -#include #include -#include -#include #include #include #include @@ -79,7 +76,6 @@ namespace ErrorCodes extern const int REQUIRED_PASSWORD; extern const int AUTHENTICATION_FAILED; - extern const int SET_NON_GRANTED_ROLE; extern const int INVALID_SESSION_TIMEOUT; extern const int HTTP_LENGTH_REQUIRED; @@ -615,20 +611,7 @@ void HTTPHandler::processQuery( auto roles = params.getAll("role"); if (!roles.empty()) - { - const auto & access_control = context->getAccessControl(); - const auto & user = context->getUser(); - std::vector roles_ids(roles.size()); - for (size_t i = 0; i < roles.size(); i++) - { - auto role_id = access_control.getID(roles[i]); - if (user->granted_roles.isGranted(role_id)) - roles_ids[i] = role_id; - else - throw Exception(ErrorCodes::SET_NON_GRANTED_ROLE, "Role {} should be granted to set as a current", roles[i].get()); - } - context->setCurrentRoles(roles_ids); - } + context->setCurrentRoles(roles); /// Settings can be overridden in the query. /// Some parameters (database, default_format, everything used in the code above) do not From 25dc96aaca0abd5498214bc3618f3ea9ea75e4d4 Mon Sep 17 00:00:00 2001 From: Vitaly Baranov Date: Thu, 9 May 2024 22:34:13 +0200 Subject: [PATCH 163/417] Simplify initialization of settings in HTTPHandler. --- .../setReadOnlyIfHTTPMethodIdempotent.cpp | 24 +++++ .../HTTP/setReadOnlyIfHTTPMethodIdempotent.h | 12 +++ src/Server/HTTPHandler.cpp | 99 +++++++------------ 3 files changed, 70 insertions(+), 65 deletions(-) create mode 100644 src/Server/HTTP/setReadOnlyIfHTTPMethodIdempotent.cpp create mode 100644 src/Server/HTTP/setReadOnlyIfHTTPMethodIdempotent.h diff --git a/src/Server/HTTP/setReadOnlyIfHTTPMethodIdempotent.cpp b/src/Server/HTTP/setReadOnlyIfHTTPMethodIdempotent.cpp new file mode 100644 index 00000000000..d42bd77e339 --- /dev/null +++ b/src/Server/HTTP/setReadOnlyIfHTTPMethodIdempotent.cpp @@ -0,0 +1,24 @@ +#include + +#include +#include + + +namespace DB +{ + +void setReadOnlyIfHTTPMethodIdempotent(ContextMutablePtr context, const String & http_method) +{ + /// Anything else beside HTTP POST should be readonly queries. + if (http_method != HTTPServerRequest::HTTP_POST) + { + /// 'readonly' setting values mean: + /// readonly = 0 - any query is allowed, client can change any setting. + /// readonly = 1 - only readonly queries are allowed, client can't change settings. + /// readonly = 2 - only readonly queries are allowed, client can change any setting except 'readonly'. + if (context->getSettingsRef().readonly == 0) + context->setSetting("readonly", 2); + } +} + +} diff --git a/src/Server/HTTP/setReadOnlyIfHTTPMethodIdempotent.h b/src/Server/HTTP/setReadOnlyIfHTTPMethodIdempotent.h new file mode 100644 index 00000000000..c46f2032d82 --- /dev/null +++ b/src/Server/HTTP/setReadOnlyIfHTTPMethodIdempotent.h @@ -0,0 +1,12 @@ +#pragma once + +#include + + +namespace DB +{ + +/// Sets readonly = 2 if the current HTTP method is not HTTP POST and if readonly is not set already. +void setReadOnlyIfHTTPMethodIdempotent(ContextMutablePtr context, const String & http_method); + +} diff --git a/src/Server/HTTPHandler.cpp b/src/Server/HTTPHandler.cpp index 5d3e6aa1b05..1cf710e8b8e 100644 --- a/src/Server/HTTPHandler.cpp +++ b/src/Server/HTTPHandler.cpp @@ -35,6 +35,7 @@ #include #include #include +#include #include #include @@ -586,10 +587,22 @@ void HTTPHandler::processQuery( std::unique_ptr in; - static const NameSet reserved_param_names{"compress", "decompress", "user", "password", "quota_key", "query_id", "stacktrace", "role", - "buffer_size", "wait_end_of_query", "session_id", "session_timeout", "session_check", "client_protocol_version", "close_session"}; + auto roles = params.getAll("role"); + if (!roles.empty()) + context->setCurrentRoles(roles); - Names reserved_param_suffixes; + std::string database = request.get("X-ClickHouse-Database", params.get("database", "")); + if (!database.empty()) + context->setCurrentDatabase(database); + + std::string default_format = request.get("X-ClickHouse-Format", params.get("default_format", "")); + if (!default_format.empty()) + context->setDefaultFormat(default_format); + + /// Anything else beside HTTP POST should be readonly queries. + setReadOnlyIfHTTPMethodIdempotent(context, request.getMethod()); + + bool has_external_data = startsWith(request.getContentType(), "multipart/form-data"); auto param_could_be_skipped = [&] (const String & name) { @@ -597,74 +610,36 @@ void HTTPHandler::processQuery( if (name.empty()) return true; + /// Some parameters (database, default_format, everything used in the code above) do not + /// belong to the Settings class. + static const NameSet reserved_param_names{"compress", "decompress", "user", "password", "quota_key", "query_id", "stacktrace", "role", + "buffer_size", "wait_end_of_query", "session_id", "session_timeout", "session_check", "client_protocol_version", "close_session", + "database", "default_format"}; + if (reserved_param_names.contains(name)) return true; - for (const String & suffix : reserved_param_suffixes) + /// For external data we also want settings. + if (has_external_data) { - if (endsWith(name, suffix)) - return true; + /// Skip unneeded parameters to avoid confusing them later with context settings or query parameters. + /// It is a bug and ambiguity with `date_time_input_format` and `low_cardinality_allow_in_native_format` formats/settings. + static const Names reserved_param_suffixes = {"_format", "_types", "_structure"}; + for (const String & suffix : reserved_param_suffixes) + { + if (endsWith(name, suffix)) + return true; + } } return false; }; - auto roles = params.getAll("role"); - if (!roles.empty()) - context->setCurrentRoles(roles); - /// Settings can be overridden in the query. - /// Some parameters (database, default_format, everything used in the code above) do not - /// belong to the Settings class. - - /// 'readonly' setting values mean: - /// readonly = 0 - any query is allowed, client can change any setting. - /// readonly = 1 - only readonly queries are allowed, client can't change settings. - /// readonly = 2 - only readonly queries are allowed, client can change any setting except 'readonly'. - - /// In theory if initially readonly = 0, the client can change any setting and then set readonly - /// to some other value. - const auto & settings = context->getSettingsRef(); - - /// Anything else beside HTTP POST should be readonly queries. - if (request.getMethod() != HTTPServerRequest::HTTP_POST) - { - if (settings.readonly == 0) - context->setSetting("readonly", 2); - } - - bool has_external_data = startsWith(request.getContentType(), "multipart/form-data"); - - if (has_external_data) - { - /// Skip unneeded parameters to avoid confusing them later with context settings or query parameters. - reserved_param_suffixes.reserve(3); - /// It is a bug and ambiguity with `date_time_input_format` and `low_cardinality_allow_in_native_format` formats/settings. - reserved_param_suffixes.emplace_back("_format"); - reserved_param_suffixes.emplace_back("_types"); - reserved_param_suffixes.emplace_back("_structure"); - } - - std::string database = request.get("X-ClickHouse-Database", ""); - std::string default_format = request.get("X-ClickHouse-Format", ""); - SettingsChanges settings_changes; for (const auto & [key, value] : params) { - if (key == "database") - { - if (database.empty()) - database = value; - } - else if (key == "default_format") - { - if (default_format.empty()) - default_format = value; - } - else if (param_could_be_skipped(key)) - { - } - else + if (!param_could_be_skipped(key)) { /// Other than query parameters are treated as settings. if (!customizeQueryParam(context, key, value)) @@ -672,15 +647,9 @@ void HTTPHandler::processQuery( } } - if (!database.empty()) - context->setCurrentDatabase(database); - - if (!default_format.empty()) - context->setDefaultFormat(default_format); - - /// For external data we also want settings context->checkSettingsConstraints(settings_changes, SettingSource::QUERY); context->applySettingsChanges(settings_changes); + const auto & settings = context->getSettingsRef(); /// Set the query id supplied by the user, if any, and also update the OpenTelemetry fields. context->setCurrentQueryId(params.get("query_id", request.get("X-ClickHouse-Query-Id", ""))); From 49b982747a3a772eb6f2ec417cebfa72c6c1edae Mon Sep 17 00:00:00 2001 From: Vitaly Baranov Date: Thu, 9 May 2024 23:19:01 +0200 Subject: [PATCH 164/417] Move function authenticateUserByHTTP() to a separate header in order to reuse it. --- src/Server/HTTP/authenticateUserByHTTP.cpp | 269 +++++++++++++++++++++ src/Server/HTTP/authenticateUserByHTTP.h | 31 +++ src/Server/HTTPHandler.cpp | 242 +----------------- 3 files changed, 305 insertions(+), 237 deletions(-) create mode 100644 src/Server/HTTP/authenticateUserByHTTP.cpp create mode 100644 src/Server/HTTP/authenticateUserByHTTP.h diff --git a/src/Server/HTTP/authenticateUserByHTTP.cpp b/src/Server/HTTP/authenticateUserByHTTP.cpp new file mode 100644 index 00000000000..3bcf6552e54 --- /dev/null +++ b/src/Server/HTTP/authenticateUserByHTTP.cpp @@ -0,0 +1,269 @@ +#include + +#include +#include +#include +#include +#include +#include +#include +#include +#include + +#include +#include +#include +#include +#include + +#if USE_SSL +#include +#endif + + +namespace DB +{ + +namespace ErrorCodes +{ + extern const int AUTHENTICATION_FAILED; + extern const int BAD_ARGUMENTS; + extern const int SUPPORT_IS_DISABLED; +} + + +namespace +{ + String base64Decode(const String & encoded) + { + String decoded; + Poco::MemoryInputStream istr(encoded.data(), encoded.size()); + Poco::Base64Decoder decoder(istr); + Poco::StreamCopier::copyToString(decoder, decoded); + return decoded; + } + + String base64Encode(const String & decoded) + { + std::ostringstream ostr; // STYLE_CHECK_ALLOW_STD_STRING_STREAM + ostr.exceptions(std::ios::failbit); + Poco::Base64Encoder encoder(ostr); + encoder.rdbuf()->setLineLength(0); + encoder << decoded; + encoder.close(); + return ostr.str(); + } +} + + +bool authenticateUserByHTTP( + const HTTPServerRequest & request, + const HTMLForm & params, + HTTPServerResponse & response, + Session & session, + std::unique_ptr & request_credentials, + ContextPtr global_context, + LoggerPtr log) +{ + /// Get the credentials created by the previous call of authenticateUserByHTTP() while handling the previous HTTP request. + auto current_credentials = std::move(request_credentials); + + /// The user and password can be passed by headers (similar to X-Auth-*), + /// which is used by load balancers to pass authentication information. + std::string user = request.get("X-ClickHouse-User", ""); + std::string password = request.get("X-ClickHouse-Key", ""); + std::string quota_key = request.get("X-ClickHouse-Quota", ""); + + /// The header 'X-ClickHouse-SSL-Certificate-Auth: on' enables checking the common name + /// extracted from the SSL certificate used for this connection instead of checking password. + bool has_ssl_certificate_auth = (request.get("X-ClickHouse-SSL-Certificate-Auth", "") == "on"); + bool has_auth_headers = !user.empty() || !password.empty() || has_ssl_certificate_auth; + + /// User name and password can be passed using HTTP Basic auth or query parameters + /// (both methods are insecure). + bool has_http_credentials = request.hasCredentials(); + bool has_credentials_in_query_params = params.has("user") || params.has("password"); + + std::string spnego_challenge; + SSLCertificateSubjects certificate_subjects; + + if (has_auth_headers) + { + /// It is prohibited to mix different authorization schemes. + if (has_http_credentials) + throw Exception(ErrorCodes::AUTHENTICATION_FAILED, + "Invalid authentication: it is not allowed " + "to use SSL certificate authentication and Authorization HTTP header simultaneously"); + if (has_credentials_in_query_params) + throw Exception(ErrorCodes::AUTHENTICATION_FAILED, + "Invalid authentication: it is not allowed " + "to use SSL certificate authentication and authentication via parameters simultaneously simultaneously"); + + if (has_ssl_certificate_auth) + { +#if USE_SSL + if (!password.empty()) + throw Exception(ErrorCodes::AUTHENTICATION_FAILED, + "Invalid authentication: it is not allowed " + "to use SSL certificate authentication and authentication via password simultaneously"); + + if (request.havePeerCertificate()) + certificate_subjects = extractSSLCertificateSubjects(request.peerCertificate()); + + if (certificate_subjects.empty()) + throw Exception(ErrorCodes::AUTHENTICATION_FAILED, + "Invalid authentication: SSL certificate authentication requires nonempty certificate's Common Name or Subject Alternative Name"); +#else + throw Exception(ErrorCodes::SUPPORT_IS_DISABLED, + "SSL certificate authentication disabled because ClickHouse was built without SSL library"); +#endif + } + } + else if (has_http_credentials) + { + /// It is prohibited to mix different authorization schemes. + if (has_credentials_in_query_params) + throw Exception(ErrorCodes::AUTHENTICATION_FAILED, + "Invalid authentication: it is not allowed " + "to use Authorization HTTP header and authentication via parameters simultaneously"); + + std::string scheme; + std::string auth_info; + request.getCredentials(scheme, auth_info); + + if (Poco::icompare(scheme, "Basic") == 0) + { + Poco::Net::HTTPBasicCredentials credentials(auth_info); + user = credentials.getUsername(); + password = credentials.getPassword(); + } + else if (Poco::icompare(scheme, "Negotiate") == 0) + { + spnego_challenge = auth_info; + + if (spnego_challenge.empty()) + throw Exception(ErrorCodes::AUTHENTICATION_FAILED, "Invalid authentication: SPNEGO challenge is empty"); + } + else + { + throw Exception(ErrorCodes::AUTHENTICATION_FAILED, "Invalid authentication: '{}' HTTP Authorization scheme is not supported", scheme); + } + } + else + { + /// If the user name is not set we assume it's the 'default' user. + user = params.get("user", "default"); + password = params.get("password", ""); + } + + if (!certificate_subjects.empty()) + { + if (!current_credentials) + current_credentials = std::make_unique(user, std::move(certificate_subjects)); + + auto * certificate_credentials = dynamic_cast(current_credentials.get()); + if (!certificate_credentials) + throw Exception(ErrorCodes::AUTHENTICATION_FAILED, "Invalid authentication: expected SSL certificate authorization scheme"); + } + else if (!spnego_challenge.empty()) + { + if (!current_credentials) + current_credentials = global_context->makeGSSAcceptorContext(); + + auto * gss_acceptor_context = dynamic_cast(current_credentials.get()); + if (!gss_acceptor_context) + throw Exception(ErrorCodes::AUTHENTICATION_FAILED, "Invalid authentication: unexpected 'Negotiate' HTTP Authorization scheme expected"); + +#pragma clang diagnostic push +#pragma clang diagnostic ignored "-Wunreachable-code" + const auto spnego_response = base64Encode(gss_acceptor_context->processToken(base64Decode(spnego_challenge), log)); +#pragma clang diagnostic pop + + if (!spnego_response.empty()) + response.set("WWW-Authenticate", "Negotiate " + spnego_response); + + if (!gss_acceptor_context->isFailed() && !gss_acceptor_context->isReady()) + { + if (spnego_response.empty()) + throw Exception(ErrorCodes::AUTHENTICATION_FAILED, "Invalid authentication: 'Negotiate' HTTP Authorization failure"); + + response.setStatusAndReason(HTTPResponse::HTTP_UNAUTHORIZED); + response.send(); + /// Keep the credentials for next HTTP request. A client can handle HTTP_UNAUTHORIZED and send us more credentials with the next HTTP request. + request_credentials = std::move(current_credentials); + return false; + } + } + else // I.e., now using user name and password strings ("Basic"). + { + if (!current_credentials) + current_credentials = std::make_unique(); + + auto * basic_credentials = dynamic_cast(current_credentials.get()); + if (!basic_credentials) + throw Exception(ErrorCodes::AUTHENTICATION_FAILED, "Invalid authentication: expected 'Basic' HTTP Authorization scheme"); + + basic_credentials->setUserName(user); + basic_credentials->setPassword(password); + } + + if (params.has("quota_key")) + { + if (!quota_key.empty()) + throw Exception(ErrorCodes::BAD_ARGUMENTS, + "Invalid authentication: it is not allowed " + "to use quota key as HTTP header and as parameter simultaneously"); + + quota_key = params.get("quota_key"); + } + + /// Set client info. It will be used for quota accounting parameters in 'setUser' method. + + session.setHTTPClientInfo(request); + session.setQuotaClientKey(quota_key); + + /// Extract the last entry from comma separated list of forwarded_for addresses. + /// Only the last proxy can be trusted (if any). + String forwarded_address = session.getClientInfo().getLastForwardedFor(); + try + { + if (!forwarded_address.empty() && global_context->getConfigRef().getBool("auth_use_forwarded_address", false)) + session.authenticate(*current_credentials, Poco::Net::SocketAddress(forwarded_address, request.clientAddress().port())); + else + session.authenticate(*current_credentials, request.clientAddress()); + } + catch (const Authentication::Require & required_credentials) + { + current_credentials = std::make_unique(); + + if (required_credentials.getRealm().empty()) + response.set("WWW-Authenticate", "Basic"); + else + response.set("WWW-Authenticate", "Basic realm=\"" + required_credentials.getRealm() + "\""); + + response.setStatusAndReason(HTTPResponse::HTTP_UNAUTHORIZED); + response.send(); + /// Keep the credentials for next HTTP request. A client can handle HTTP_UNAUTHORIZED and send us more credentials with the next HTTP request. + request_credentials = std::move(current_credentials); + return false; + } + catch (const Authentication::Require & required_credentials) + { + current_credentials = global_context->makeGSSAcceptorContext(); + + if (required_credentials.getRealm().empty()) + response.set("WWW-Authenticate", "Negotiate"); + else + response.set("WWW-Authenticate", "Negotiate realm=\"" + required_credentials.getRealm() + "\""); + + response.setStatusAndReason(HTTPResponse::HTTP_UNAUTHORIZED); + response.send(); + /// Keep the credentials for next HTTP request. A client can handle HTTP_UNAUTHORIZED and send us more credentials with the next HTTP request. + request_credentials = std::move(current_credentials); + return false; + } + + return true; +} + +} diff --git a/src/Server/HTTP/authenticateUserByHTTP.h b/src/Server/HTTP/authenticateUserByHTTP.h new file mode 100644 index 00000000000..3b5a04cae68 --- /dev/null +++ b/src/Server/HTTP/authenticateUserByHTTP.h @@ -0,0 +1,31 @@ +#pragma once + +#include +#include + + +namespace DB +{ +class HTTPServerRequest; +class HTMLForm; +class HTTPServerResponse; +class Session; +class Credentials; + +/// Authenticates a user via HTTP protocol and initializes a session. +/// Usually retrieves the name and the password for that user from either the request's headers or from the query parameters. +/// Returns true when the user successfully authenticated, +/// the session instance will be configured accordingly, and the request_credentials instance will be dropped. +/// Returns false when the user is not authenticated yet, and the HTTP_UNAUTHORIZED response is sent with the "WWW-Authenticate" header, +/// in this case the `request_credentials` instance must be preserved until the next request or until any exception. +/// Throws an exception if authentication failed. +bool authenticateUserByHTTP( + const HTTPServerRequest & request, + const HTMLForm & params, + HTTPServerResponse & response, + Session & session, + std::unique_ptr & request_credentials, + ContextPtr global_context, + LoggerPtr log); + +} diff --git a/src/Server/HTTPHandler.cpp b/src/Server/HTTPHandler.cpp index 1cf710e8b8e..619a18dab49 100644 --- a/src/Server/HTTPHandler.cpp +++ b/src/Server/HTTPHandler.cpp @@ -1,8 +1,6 @@ #include -#include #include -#include #include #include #include @@ -34,22 +32,14 @@ #include #include #include +#include #include #include #include -#include -#include "config.h" - -#include -#include -#include #include -#include -#include -#include -#include -#include + +#include "config.h" #include #include @@ -59,28 +49,21 @@ #include #include -#if USE_SSL -#include -#endif - namespace DB { namespace ErrorCodes { - extern const int BAD_ARGUMENTS; extern const int LOGICAL_ERROR; extern const int CANNOT_COMPILE_REGEXP; extern const int NO_ELEMENTS_IN_CONFIG; extern const int REQUIRED_PASSWORD; - extern const int AUTHENTICATION_FAILED; extern const int INVALID_SESSION_TIMEOUT; extern const int HTTP_LENGTH_REQUIRED; - extern const int SUPPORT_IS_DISABLED; } namespace @@ -122,26 +105,6 @@ void processOptionsRequest(HTTPServerResponse & response, const Poco::Util::Laye } } -static String base64Decode(const String & encoded) -{ - String decoded; - Poco::MemoryInputStream istr(encoded.data(), encoded.size()); - Poco::Base64Decoder decoder(istr); - Poco::StreamCopier::copyToString(decoder, decoded); - return decoded; -} - -static String base64Encode(const String & decoded) -{ - std::ostringstream ostr; // STYLE_CHECK_ALLOW_STD_STRING_STREAM - ostr.exceptions(std::ios::failbit); - Poco::Base64Encoder encoder(ostr); - encoder.rdbuf()->setLineLength(0); - encoder << decoded; - encoder.close(); - return ostr.str(); -} - static std::chrono::steady_clock::duration parseSessionTimeout( const Poco::Util::AbstractConfiguration & config, const HTMLForm & params) @@ -218,204 +181,9 @@ HTTPHandler::HTTPHandler(IServer & server_, const std::string & name, const HTTP HTTPHandler::~HTTPHandler() = default; -bool HTTPHandler::authenticateUser( - HTTPServerRequest & request, - HTMLForm & params, - HTTPServerResponse & response) +bool HTTPHandler::authenticateUser(HTTPServerRequest & request, HTMLForm & params, HTTPServerResponse & response) { - using namespace Poco::Net; - - /// The user and password can be passed by headers (similar to X-Auth-*), - /// which is used by load balancers to pass authentication information. - std::string user = request.get("X-ClickHouse-User", ""); - std::string password = request.get("X-ClickHouse-Key", ""); - std::string quota_key = request.get("X-ClickHouse-Quota", ""); - - /// The header 'X-ClickHouse-SSL-Certificate-Auth: on' enables checking the common name - /// extracted from the SSL certificate used for this connection instead of checking password. - bool has_ssl_certificate_auth = (request.get("X-ClickHouse-SSL-Certificate-Auth", "") == "on"); - bool has_auth_headers = !user.empty() || !password.empty() || has_ssl_certificate_auth; - - /// User name and password can be passed using HTTP Basic auth or query parameters - /// (both methods are insecure). - bool has_http_credentials = request.hasCredentials(); - bool has_credentials_in_query_params = params.has("user") || params.has("password"); - - std::string spnego_challenge; - SSLCertificateSubjects certificate_subjects; - - if (has_auth_headers) - { - /// It is prohibited to mix different authorization schemes. - if (has_http_credentials) - throw Exception(ErrorCodes::AUTHENTICATION_FAILED, - "Invalid authentication: it is not allowed " - "to use SSL certificate authentication and Authorization HTTP header simultaneously"); - if (has_credentials_in_query_params) - throw Exception(ErrorCodes::AUTHENTICATION_FAILED, - "Invalid authentication: it is not allowed " - "to use SSL certificate authentication and authentication via parameters simultaneously simultaneously"); - - if (has_ssl_certificate_auth) - { -#if USE_SSL - if (!password.empty()) - throw Exception(ErrorCodes::AUTHENTICATION_FAILED, - "Invalid authentication: it is not allowed " - "to use SSL certificate authentication and authentication via password simultaneously"); - - if (request.havePeerCertificate()) - certificate_subjects = extractSSLCertificateSubjects(request.peerCertificate()); - - if (certificate_subjects.empty()) - throw Exception(ErrorCodes::AUTHENTICATION_FAILED, - "Invalid authentication: SSL certificate authentication requires nonempty certificate's Common Name or Subject Alternative Name"); -#else - throw Exception(ErrorCodes::SUPPORT_IS_DISABLED, - "SSL certificate authentication disabled because ClickHouse was built without SSL library"); -#endif - } - } - else if (has_http_credentials) - { - /// It is prohibited to mix different authorization schemes. - if (has_credentials_in_query_params) - throw Exception(ErrorCodes::AUTHENTICATION_FAILED, - "Invalid authentication: it is not allowed " - "to use Authorization HTTP header and authentication via parameters simultaneously"); - - std::string scheme; - std::string auth_info; - request.getCredentials(scheme, auth_info); - - if (Poco::icompare(scheme, "Basic") == 0) - { - HTTPBasicCredentials credentials(auth_info); - user = credentials.getUsername(); - password = credentials.getPassword(); - } - else if (Poco::icompare(scheme, "Negotiate") == 0) - { - spnego_challenge = auth_info; - - if (spnego_challenge.empty()) - throw Exception(ErrorCodes::AUTHENTICATION_FAILED, "Invalid authentication: SPNEGO challenge is empty"); - } - else - { - throw Exception(ErrorCodes::AUTHENTICATION_FAILED, "Invalid authentication: '{}' HTTP Authorization scheme is not supported", scheme); - } - } - else - { - /// If the user name is not set we assume it's the 'default' user. - user = params.get("user", "default"); - password = params.get("password", ""); - } - - if (!certificate_subjects.empty()) - { - if (!request_credentials) - request_credentials = std::make_unique(user, std::move(certificate_subjects)); - - auto * certificate_credentials = dynamic_cast(request_credentials.get()); - if (!certificate_credentials) - throw Exception(ErrorCodes::AUTHENTICATION_FAILED, "Invalid authentication: expected SSL certificate authorization scheme"); - } - else if (!spnego_challenge.empty()) - { - if (!request_credentials) - request_credentials = server.context()->makeGSSAcceptorContext(); - - auto * gss_acceptor_context = dynamic_cast(request_credentials.get()); - if (!gss_acceptor_context) - throw Exception(ErrorCodes::AUTHENTICATION_FAILED, "Invalid authentication: unexpected 'Negotiate' HTTP Authorization scheme expected"); - -#pragma clang diagnostic push -#pragma clang diagnostic ignored "-Wunreachable-code" - const auto spnego_response = base64Encode(gss_acceptor_context->processToken(base64Decode(spnego_challenge), log)); -#pragma clang diagnostic pop - - if (!spnego_response.empty()) - response.set("WWW-Authenticate", "Negotiate " + spnego_response); - - if (!gss_acceptor_context->isFailed() && !gss_acceptor_context->isReady()) - { - if (spnego_response.empty()) - throw Exception(ErrorCodes::AUTHENTICATION_FAILED, "Invalid authentication: 'Negotiate' HTTP Authorization failure"); - - response.setStatusAndReason(HTTPResponse::HTTP_UNAUTHORIZED); - response.send(); - return false; - } - } - else // I.e., now using user name and password strings ("Basic"). - { - if (!request_credentials) - request_credentials = std::make_unique(); - - auto * basic_credentials = dynamic_cast(request_credentials.get()); - if (!basic_credentials) - throw Exception(ErrorCodes::AUTHENTICATION_FAILED, "Invalid authentication: expected 'Basic' HTTP Authorization scheme"); - - basic_credentials->setUserName(user); - basic_credentials->setPassword(password); - } - - if (params.has("quota_key")) - { - if (!quota_key.empty()) - throw Exception(ErrorCodes::BAD_ARGUMENTS, - "Invalid authentication: it is not allowed " - "to use quota key as HTTP header and as parameter simultaneously"); - - quota_key = params.get("quota_key"); - } - - /// Set client info. It will be used for quota accounting parameters in 'setUser' method. - - session->setHTTPClientInfo(request); - session->setQuotaClientKey(quota_key); - - /// Extract the last entry from comma separated list of forwarded_for addresses. - /// Only the last proxy can be trusted (if any). - String forwarded_address = session->getClientInfo().getLastForwardedFor(); - try - { - if (!forwarded_address.empty() && server.config().getBool("auth_use_forwarded_address", false)) - session->authenticate(*request_credentials, Poco::Net::SocketAddress(forwarded_address, request.clientAddress().port())); - else - session->authenticate(*request_credentials, request.clientAddress()); - } - catch (const Authentication::Require & required_credentials) - { - request_credentials = std::make_unique(); - - if (required_credentials.getRealm().empty()) - response.set("WWW-Authenticate", "Basic"); - else - response.set("WWW-Authenticate", "Basic realm=\"" + required_credentials.getRealm() + "\""); - - response.setStatusAndReason(HTTPResponse::HTTP_UNAUTHORIZED); - response.send(); - return false; - } - catch (const Authentication::Require & required_credentials) - { - request_credentials = server.context()->makeGSSAcceptorContext(); - - if (required_credentials.getRealm().empty()) - response.set("WWW-Authenticate", "Negotiate"); - else - response.set("WWW-Authenticate", "Negotiate realm=\"" + required_credentials.getRealm() + "\""); - - response.setStatusAndReason(HTTPResponse::HTTP_UNAUTHORIZED); - response.send(); - return false; - } - - request_credentials.reset(); - return true; + return authenticateUserByHTTP(request, params, response, *session, request_credentials, server.context(), log); } From ecfe6fddcfbe68477fb10fabc4b0252e08d198ef Mon Sep 17 00:00:00 2001 From: Vitaly Baranov Date: Mon, 27 May 2024 16:59:46 +0200 Subject: [PATCH 165/417] Move function HTTPHandler::formatExceptionForClient() to a separate header in order to reuse it. --- src/Server/HTTP/sendExceptionToHTTPClient.cpp | 80 +++++++++++++++++++ src/Server/HTTP/sendExceptionToHTTPClient.h | 27 +++++++ src/Server/HTTPHandler.cpp | 40 +--------- src/Server/HTTPHandler.h | 6 -- 4 files changed, 110 insertions(+), 43 deletions(-) create mode 100644 src/Server/HTTP/sendExceptionToHTTPClient.cpp create mode 100644 src/Server/HTTP/sendExceptionToHTTPClient.h diff --git a/src/Server/HTTP/sendExceptionToHTTPClient.cpp b/src/Server/HTTP/sendExceptionToHTTPClient.cpp new file mode 100644 index 00000000000..78650758e35 --- /dev/null +++ b/src/Server/HTTP/sendExceptionToHTTPClient.cpp @@ -0,0 +1,80 @@ +#include + +#include +#include +#include + + +namespace DB +{ + +namespace ErrorCodes +{ + extern const int HTTP_LENGTH_REQUIRED; + extern const int REQUIRED_PASSWORD; +} + + +void sendExceptionToHTTPClient( + const String & exception_message, + int exception_code, + HTTPServerRequest & request, + HTTPServerResponse & response, + WriteBufferFromHTTPServerResponse * out, + LoggerPtr log) +{ + setHTTPResponseStatusAndHeadersForException(exception_code, request, response, out, log); + + if (!out) + { + /// If nothing was sent yet. + WriteBufferFromHTTPServerResponse out_for_message{response, request.getMethod() == HTTPRequest::HTTP_HEAD, DEFAULT_HTTP_KEEP_ALIVE_TIMEOUT}; + + out_for_message.writeln(exception_message); + out_for_message.finalize(); + } + else + { + /// If buffer has data, and that data wasn't sent yet, then no need to send that data + bool data_sent = (out->count() != out->offset()); + + if (!data_sent) + out->position() = out->buffer().begin(); + + out->writeln(exception_message); + out->finalize(); + } +} + + +void setHTTPResponseStatusAndHeadersForException( + int exception_code, HTTPServerRequest & request, HTTPServerResponse & response, WriteBufferFromHTTPServerResponse * out, LoggerPtr log) +{ + if (out) + out->setExceptionCode(exception_code); + else + response.set("X-ClickHouse-Exception-Code", toString(exception_code)); + + /// If HTTP method is POST and Keep-Alive is turned on, we should try to read the whole request body + /// to avoid reading part of the current request body in the next request. + if (request.getMethod() == Poco::Net::HTTPRequest::HTTP_POST && response.getKeepAlive() + && exception_code != ErrorCodes::HTTP_LENGTH_REQUIRED) + { + try + { + if (!request.getStream().eof()) + request.getStream().ignoreAll(); + } + catch (...) + { + tryLogCurrentException(log, "Cannot read remaining request body during exception handling"); + response.setKeepAlive(false); + } + } + + if (exception_code == ErrorCodes::REQUIRED_PASSWORD) + response.requireAuthentication("ClickHouse server HTTP API"); + else + response.setStatusAndReason(exceptionCodeToHTTPStatus(exception_code)); +} +} diff --git a/src/Server/HTTP/sendExceptionToHTTPClient.h b/src/Server/HTTP/sendExceptionToHTTPClient.h new file mode 100644 index 00000000000..31fda88d900 --- /dev/null +++ b/src/Server/HTTP/sendExceptionToHTTPClient.h @@ -0,0 +1,27 @@ +#pragma once + +#include +#include + + +namespace DB +{ +class HTTPServerRequest; +class HTTPServerResponse; +class WriteBufferFromHTTPServerResponse; + +/// Sends an exception to HTTP client. This function doesn't handle its own exceptions so it needs to be wrapped in try-catch. +/// Argument `out` may be either created from `response` or be nullptr (if it wasn't created before the exception). +void sendExceptionToHTTPClient( + const String & exception_message, + int exception_code, + HTTPServerRequest & request, + HTTPServerResponse & response, + WriteBufferFromHTTPServerResponse * out, + LoggerPtr log); + +/// Sets "X-ClickHouse-Exception-Code" header and the correspondent HTTP status in the response for an exception. +/// This is a part of what sendExceptionToHTTPClient() does. +void setHTTPResponseStatusAndHeadersForException( + int exception_code, HTTPServerRequest & request, HTTPServerResponse & response, WriteBufferFromHTTPServerResponse * out, LoggerPtr log); +} diff --git a/src/Server/HTTPHandler.cpp b/src/Server/HTTPHandler.cpp index 619a18dab49..370af79e456 100644 --- a/src/Server/HTTPHandler.cpp +++ b/src/Server/HTTPHandler.cpp @@ -33,7 +33,7 @@ #include #include #include -#include +#include #include #include @@ -60,8 +60,6 @@ namespace ErrorCodes extern const int NO_ELEMENTS_IN_CONFIG; - extern const int REQUIRED_PASSWORD; - extern const int INVALID_SESSION_TIMEOUT; extern const int HTTP_LENGTH_REQUIRED; } @@ -519,7 +517,7 @@ void HTTPHandler::processQuery( { bool with_stacktrace = (params.getParsed("stacktrace", false) && server.config().getBool("enable_http_stacktrace", true)); ExecutionStatus status = ExecutionStatus::fromCurrentException("", with_stacktrace); - formatExceptionForClient(status.code, request, response, used_output); + setHTTPResponseStatusAndHeadersForException(status.code, request, response, used_output.out_holder.get(), log); current_output_format.setException(status.message); current_output_format.finalize(); used_output.exception_is_written = true; @@ -553,7 +551,7 @@ void HTTPHandler::trySendExceptionToClient( const std::string & s, int exception_code, HTTPServerRequest & request, HTTPServerResponse & response, Output & used_output) try { - formatExceptionForClient(exception_code, request, response, used_output); + setHTTPResponseStatusAndHeadersForException(exception_code, request, response, used_output.out_holder.get(), log); if (!used_output.out_holder && !used_output.exception_is_written) { @@ -615,38 +613,6 @@ catch (...) used_output.cancel(); } -void HTTPHandler::formatExceptionForClient(int exception_code, HTTPServerRequest & request, HTTPServerResponse & response, Output & used_output) -{ - if (used_output.out_holder) - used_output.out_holder->setExceptionCode(exception_code); - else - response.set("X-ClickHouse-Exception-Code", toString(exception_code)); - - /// FIXME: make sure that no one else is reading from the same stream at the moment. - - /// If HTTP method is POST and Keep-Alive is turned on, we should try to read the whole request body - /// to avoid reading part of the current request body in the next request. - if (request.getMethod() == Poco::Net::HTTPRequest::HTTP_POST && response.getKeepAlive() - && exception_code != ErrorCodes::HTTP_LENGTH_REQUIRED) - { - try - { - if (!request.getStream().eof()) - request.getStream().ignoreAll(); - } - catch (...) - { - tryLogCurrentException(log, "Cannot read remaining request body during exception handling"); - response.setKeepAlive(false); - } - } - - if (exception_code == ErrorCodes::REQUIRED_PASSWORD) - response.requireAuthentication("ClickHouse server HTTP API"); - else - response.setStatusAndReason(exceptionCodeToHTTPStatus(exception_code)); -} - void HTTPHandler::handleRequest(HTTPServerRequest & request, HTTPServerResponse & response, const ProfileEvents::Event & write_event) { setThreadName("HTTPHandler"); diff --git a/src/Server/HTTPHandler.h b/src/Server/HTTPHandler.h index c78c45826f0..6580b317f6e 100644 --- a/src/Server/HTTPHandler.h +++ b/src/Server/HTTPHandler.h @@ -173,12 +173,6 @@ private: HTTPServerResponse & response, Output & used_output); - void formatExceptionForClient( - int exception_code, - HTTPServerRequest & request, - HTTPServerResponse & response, - Output & used_output); - static void pushDelayedResults(Output & used_output); }; From 8cb7936838357783b702fbb461c22a7178d34bf3 Mon Sep 17 00:00:00 2001 From: Nikita Taranov Date: Wed, 3 Jul 2024 19:41:26 +0100 Subject: [PATCH 166/417] dedicated setting for joins --- src/Core/Settings.h | 3 +++ src/Core/SettingsChangesHistory.cpp | 2 ++ src/Planner/PlannerJoins.cpp | 4 ++-- 3 files changed, 7 insertions(+), 2 deletions(-) diff --git a/src/Core/Settings.h b/src/Core/Settings.h index b1bb6edfc38..898dd02c655 100644 --- a/src/Core/Settings.h +++ b/src/Core/Settings.h @@ -709,6 +709,9 @@ class IColumn; M(Bool, collect_hash_table_stats_during_aggregation, true, "Enable collecting hash table statistics to optimize memory allocation", 0) \ M(UInt64, max_size_to_preallocate_for_aggregation, 100'000'000, "For how many elements it is allowed to preallocate space in all hash tables in total before aggregation", 0) \ \ + M(Bool, collect_hash_table_stats_during_joins, true, "Enable collecting hash table statistics to optimize memory allocation", 0) \ + M(UInt64, max_size_to_preallocate_for_joins, 100'000'000, "For how many elements it is allowed to preallocate space in all hash tables in total before join", 0) \ + \ M(Bool, kafka_disable_num_consumers_limit, false, "Disable limit on kafka_num_consumers that depends on the number of available CPU cores", 0) \ M(Bool, enable_software_prefetch_in_aggregation, true, "Enable use of software prefetch in aggregation", 0) \ M(Bool, allow_aggregate_partitions_independently, false, "Enable independent aggregation of partitions on separate threads when partition key suits group by key. Beneficial when number of partitions close to number of cores and partitions have roughly the same size", 0) \ diff --git a/src/Core/SettingsChangesHistory.cpp b/src/Core/SettingsChangesHistory.cpp index b0725340f46..b3bbd9d424a 100644 --- a/src/Core/SettingsChangesHistory.cpp +++ b/src/Core/SettingsChangesHistory.cpp @@ -59,6 +59,8 @@ static std::initializer_list tryCreateJoin(JoinAlgorithm algorithm, const auto & settings = query_context->getSettingsRef(); StatsCollectingParams params{ calculateCacheKey(table_join, right_table_expression), - settings.collect_hash_table_stats_during_aggregation, + settings.collect_hash_table_stats_during_joins, query_context->getServerSettings().max_entries_for_hash_table_stats, - settings.max_size_to_preallocate_for_aggregation}; + settings.max_size_to_preallocate_for_joins}; return std::make_shared( query_context, table_join, query_context->getSettings().max_threads, right_table_expression_header, params); } From a61907e96e3c65e85406b680a8ede75f4c4a8015 Mon Sep 17 00:00:00 2001 From: Nikita Taranov Date: Wed, 3 Jul 2024 20:46:36 +0100 Subject: [PATCH 167/417] better test --- .../0_stateless/03173_parallel_replicas_join_bug.reference | 5 ++++- .../queries/0_stateless/03173_parallel_replicas_join_bug.sh | 6 ++++++ 2 files changed, 10 insertions(+), 1 deletion(-) diff --git a/tests/queries/0_stateless/03173_parallel_replicas_join_bug.reference b/tests/queries/0_stateless/03173_parallel_replicas_join_bug.reference index b23d6b02bc1..93018551e1b 100644 --- a/tests/queries/0_stateless/03173_parallel_replicas_join_bug.reference +++ b/tests/queries/0_stateless/03173_parallel_replicas_join_bug.reference @@ -1,7 +1,10 @@ a1451105-722e-4fe7-bfaa-65ad2ae249c2 whatever a1451105-722e-4fe7-bfaa-65ad2ae249c2 whatever +--------------------------- a1451105-722e-4fe7-bfaa-65ad2ae249c2 a1451105-722e-4fe7-bfaa-65ad2ae249c2 -a1451105-722e-4fe7-bfaa-65ad2ae249c2 +--------------------------- +a1451105-722e-4fe7-bfaa-65ad2ae249c2 +--------------------------- a1451105-722e-4fe7-bfaa-65ad2ae249c2 a1451105-722e-4fe7-bfaa-65ad2ae249c2 diff --git a/tests/queries/0_stateless/03173_parallel_replicas_join_bug.sh b/tests/queries/0_stateless/03173_parallel_replicas_join_bug.sh index 4638609b00c..20a29e2734e 100755 --- a/tests/queries/0_stateless/03173_parallel_replicas_join_bug.sh +++ b/tests/queries/0_stateless/03173_parallel_replicas_join_bug.sh @@ -30,6 +30,8 @@ INNER JOIN view( WHERE status IN ['CREATED', 'CREATING'] ORDER BY event_time DESC; +SELECT '---------------------------'; + with results1 as ( SELECT id @@ -45,6 +47,8 @@ results2 as ( ) select * from results1 union all select * from results2; +SELECT '---------------------------'; + with results1 as ( SELECT id @@ -60,6 +64,8 @@ results2 as ( ) select * from results1 t1 inner join results2 t2 using (id); +SELECT '---------------------------'; + with results1 as ( SELECT t1.id From 37851686ad20edf14a736e676f9d364818733c65 Mon Sep 17 00:00:00 2001 From: Nikita Taranov Date: Wed, 3 Jul 2024 20:47:07 +0100 Subject: [PATCH 168/417] better variable name --- src/Planner/PlannerJoinTree.cpp | 5 +++-- src/Storages/SelectQueryInfo.h | 2 +- src/Storages/StorageMergeTree.cpp | 3 ++- src/Storages/StorageReplicatedMergeTree.cpp | 3 ++- 4 files changed, 8 insertions(+), 5 deletions(-) diff --git a/src/Planner/PlannerJoinTree.cpp b/src/Planner/PlannerJoinTree.cpp index 0af7bfea0b0..6c4fcfa345b 100644 --- a/src/Planner/PlannerJoinTree.cpp +++ b/src/Planner/PlannerJoinTree.cpp @@ -647,7 +647,8 @@ JoinTreeQueryPlan buildQueryPlanForTableExpression(QueryTreeNodePtr table_expres auto table_expression_query_info = select_query_info; table_expression_query_info.table_expression = table_expression; table_expression_query_info.filter_actions_dag = table_expression_data.getFilterActions(); - table_expression_query_info.analyzer_can_use_parallel_replicas_on_follower = table_node == planner_context->getGlobalPlannerContext()->parallel_replicas_table; + table_expression_query_info.current_table_chosen_for_reading_with_parallel_replicas + = table_node == planner_context->getGlobalPlannerContext()->parallel_replicas_table; size_t max_streams = settings.max_threads; size_t max_threads_execute_query = settings.max_threads; @@ -862,7 +863,7 @@ JoinTreeQueryPlan buildQueryPlanForTableExpression(QueryTreeNodePtr table_expres /// and find some other table that might be used for reading with parallel replicas. It will lead to errors. const bool other_table_already_chosen_for_reading_with_parallel_replicas = planner_context->getGlobalPlannerContext()->parallel_replicas_table - && !table_expression_query_info.analyzer_can_use_parallel_replicas_on_follower; + && !table_expression_query_info.current_table_chosen_for_reading_with_parallel_replicas; if (other_table_already_chosen_for_reading_with_parallel_replicas) planner_context->getMutableQueryContext()->setSetting("allow_experimental_parallel_reading_from_replicas", Field(0)); diff --git a/src/Storages/SelectQueryInfo.h b/src/Storages/SelectQueryInfo.h index 52b6674c93d..5662f057bd4 100644 --- a/src/Storages/SelectQueryInfo.h +++ b/src/Storages/SelectQueryInfo.h @@ -163,7 +163,7 @@ struct SelectQueryInfo /// It's guaranteed to be present in JOIN TREE of `query_tree` QueryTreeNodePtr table_expression; - bool analyzer_can_use_parallel_replicas_on_follower = false; + bool current_table_chosen_for_reading_with_parallel_replicas = false; /// Table expression modifiers for storage std::optional table_expression_modifiers; diff --git a/src/Storages/StorageMergeTree.cpp b/src/Storages/StorageMergeTree.cpp index 9352f772ce1..8c24a9c191f 100644 --- a/src/Storages/StorageMergeTree.cpp +++ b/src/Storages/StorageMergeTree.cpp @@ -225,7 +225,8 @@ void StorageMergeTree::read( { const bool enable_parallel_reading = local_context->canUseParallelReplicasOnFollower() && local_context->getSettingsRef().parallel_replicas_for_non_replicated_merge_tree - && (!local_context->getSettingsRef().allow_experimental_analyzer || query_info.analyzer_can_use_parallel_replicas_on_follower); + && (!local_context->getSettingsRef().allow_experimental_analyzer + || query_info.current_table_chosen_for_reading_with_parallel_replicas); if (auto plan = reader.read( column_names, diff --git a/src/Storages/StorageReplicatedMergeTree.cpp b/src/Storages/StorageReplicatedMergeTree.cpp index db58d0081c6..480cdbb8c66 100644 --- a/src/Storages/StorageReplicatedMergeTree.cpp +++ b/src/Storages/StorageReplicatedMergeTree.cpp @@ -5508,7 +5508,8 @@ void StorageReplicatedMergeTree::readLocalImpl( const size_t num_streams) { const bool enable_parallel_reading = local_context->canUseParallelReplicasOnFollower() - && (!local_context->getSettingsRef().allow_experimental_analyzer || query_info.analyzer_can_use_parallel_replicas_on_follower); + && (!local_context->getSettingsRef().allow_experimental_analyzer + || query_info.current_table_chosen_for_reading_with_parallel_replicas); auto plan = reader.read( column_names, storage_snapshot, query_info, From 31ca631ac583b88b1ef7e083e03d53712a87ada8 Mon Sep 17 00:00:00 2001 From: alesapin Date: Wed, 3 Jul 2024 21:58:18 +0200 Subject: [PATCH 169/417] Pin all python packages in CI images --- docker/reqgenerator.py | 42 +++++++ docker/test/fasttest/Dockerfile | 6 +- docker/test/fasttest/requirements.txt | 41 +++++++ docker/test/fuzzer/Dockerfile | 3 +- docker/test/fuzzer/requirements.txt | 27 +++++ docker/test/integration/base/Dockerfile | 3 +- docker/test/integration/base/requirements.txt | 26 ++++ docker/test/integration/resolver/Dockerfile | 3 +- .../integration/resolver/requirements.txt | 6 + docker/test/integration/runner/Dockerfile | 46 +------ .../test/integration/runner/requirements.txt | 113 ++++++++++++++++++ docker/test/libfuzzer/Dockerfile | 4 +- docker/test/libfuzzer/requirements.txt | 27 +++++ docker/test/performance-comparison/Dockerfile | 5 +- .../performance-comparison/requirements.txt | 32 +++++ docker/test/sqllogic/Dockerfile | 7 +- docker/test/sqllogic/requirements.txt | 30 +++++ docker/test/sqltest/Dockerfile | 5 +- docker/test/sqltest/requirements.txt | 29 +++++ docker/test/stateful/Dockerfile | 1 - docker/test/stateless/Dockerfile | 6 +- docker/test/stateless/requirements.txt | 51 ++++++++ docker/test/style/Dockerfile | 18 +-- docker/test/style/requirements.txt | 58 +++++++++ 24 files changed, 506 insertions(+), 83 deletions(-) create mode 100644 docker/reqgenerator.py create mode 100644 docker/test/fasttest/requirements.txt create mode 100644 docker/test/fuzzer/requirements.txt create mode 100644 docker/test/integration/base/requirements.txt create mode 100644 docker/test/integration/resolver/requirements.txt create mode 100644 docker/test/integration/runner/requirements.txt create mode 100644 docker/test/libfuzzer/requirements.txt create mode 100644 docker/test/performance-comparison/requirements.txt create mode 100644 docker/test/sqllogic/requirements.txt create mode 100644 docker/test/sqltest/requirements.txt create mode 100644 docker/test/stateless/requirements.txt create mode 100644 docker/test/style/requirements.txt diff --git a/docker/reqgenerator.py b/docker/reqgenerator.py new file mode 100644 index 00000000000..89b901413d6 --- /dev/null +++ b/docker/reqgenerator.py @@ -0,0 +1,42 @@ +#!/usr/bin/env python3 +# To run this script you must install docker and piddeptree python package +# + +import subprocess +import os +import sys + +def build_docker_deps(image_name, imagedir): + cmd = f"""docker run --entrypoint "/bin/bash" {image_name} -c "pip install pipdeptree 2>/dev/null 1>/dev/null && pipdeptree --freeze --warn silence | sed 's/ \+//g' | sort | uniq" > {imagedir}/requirements.txt""" + subprocess.check_call(cmd, shell=True) + +def check_docker_file_install_with_pip(filepath): + image_name = None + with open(filepath, 'r') as f: + for line in f: + if 'docker build' in line: + arr = line.split(' ') + if len(arr) > 4: + image_name = arr[4] + if 'pip3 install' in line or 'pip install' in line: + return image_name, True + return image_name, False + +def process_affected_images(images_dir): + for root, _dirs, files in os.walk(images_dir): + for f in files: + if f == "Dockerfile": + docker_file_path = os.path.join(root, f) + print("Checking image on path", docker_file_path) + image_name, has_pip = check_docker_file_install_with_pip(docker_file_path) + if has_pip: + print("Find pip in", image_name) + try: + build_docker_deps(image_name, root) + except Exception as ex: + print(ex) + else: + print("Pip not found in", docker_file_path) + + +process_affected_images(sys.argv[1]) diff --git a/docker/test/fasttest/Dockerfile b/docker/test/fasttest/Dockerfile index e0be261d5e8..5f92db5a3ee 100644 --- a/docker/test/fasttest/Dockerfile +++ b/docker/test/fasttest/Dockerfile @@ -19,10 +19,7 @@ RUN apt-get update \ odbcinst \ psmisc \ python3 \ - python3-lxml \ python3-pip \ - python3-requests \ - python3-termcolor \ unixodbc \ pv \ jq \ @@ -31,7 +28,8 @@ RUN apt-get update \ && apt-get clean \ && rm -rf /var/lib/apt/lists/* /var/cache/debconf /tmp/* -RUN pip3 install numpy==1.26.3 scipy==1.12.0 pandas==1.5.3 Jinja2==3.1.3 +COPY requirements.txt / +RUN pip3 install --no-cache-dir -r /requirements.txt # This symlink is required by gcc to find the lld linker RUN ln -s /usr/bin/lld-${LLVM_VERSION} /usr/bin/ld.lld diff --git a/docker/test/fasttest/requirements.txt b/docker/test/fasttest/requirements.txt new file mode 100644 index 00000000000..993ea22e5ae --- /dev/null +++ b/docker/test/fasttest/requirements.txt @@ -0,0 +1,41 @@ +Jinja2==3.1.3 +MarkupSafe==2.1.5 +PyJWT==2.3.0 +PyYAML==6.0.1 +Pygments==2.11.2 +SecretStorage==3.3.1 +blinker==1.4 +certifi==2020.6.20 +chardet==4.0.0 +cryptography==3.4.8 +dbus-python==1.2.18 +distro==1.7.0 +httplib2==0.20.2 +idna==3.3 +importlib-metadata==4.6.4 +jeepney==0.7.1 +keyring==23.5.0 +launchpadlib==1.10.16 +lazr.restfulclient==0.14.4 +lazr.uri==1.0.6 +lxml==4.8.0 +more-itertools==8.10.0 +numpy==1.26.3 +oauthlib==3.2.0 +packaging==24.1 +pandas==1.5.3 +pip==24.1.1 +pipdeptree==2.23.0 +pyparsing==2.4.7 +python-apt==2.4.0+ubuntu3 +python-dateutil==2.9.0.post0 +pytz==2024.1 +requests==2.32.3 +scipy==1.12.0 +setuptools==59.6.0 +six==1.16.0 +termcolor==1.1.0 +urllib3==1.26.5 +wadllib==1.3.6 +wheel==0.37.1 +zipp==1.0.0 diff --git a/docker/test/fuzzer/Dockerfile b/docker/test/fuzzer/Dockerfile index d3f78ac1d95..e1fb09b8ed5 100644 --- a/docker/test/fuzzer/Dockerfile +++ b/docker/test/fuzzer/Dockerfile @@ -31,7 +31,8 @@ RUN apt-get update \ && apt-get clean \ && rm -rf /var/lib/apt/lists/* /var/cache/debconf /tmp/* -RUN pip3 install Jinja2 +COPY requirements.txt / +RUN pip3 install --no-cache-dir -r /requirements.txt COPY * / diff --git a/docker/test/fuzzer/requirements.txt b/docker/test/fuzzer/requirements.txt new file mode 100644 index 00000000000..3dce93e023b --- /dev/null +++ b/docker/test/fuzzer/requirements.txt @@ -0,0 +1,27 @@ +blinker==1.4 +cryptography==3.4.8 +dbus-python==1.2.18 +distro==1.7.0 +httplib2==0.20.2 +importlib-metadata==4.6.4 +jeepney==0.7.1 +Jinja2==3.1.4 +keyring==23.5.0 +launchpadlib==1.10.16 +lazr.restfulclient==0.14.4 +lazr.uri==1.0.6 +MarkupSafe==2.1.5 +more-itertools==8.10.0 +oauthlib==3.2.0 +packaging==24.1 +pip==24.1.1 +pipdeptree==2.23.0 +PyJWT==2.3.0 +pyparsing==2.4.7 +python-apt==2.4.0+ubuntu3 +SecretStorage==3.3.1 +setuptools==59.6.0 +six==1.16.0 +wadllib==1.3.6 +wheel==0.37.1 +zipp==1.0.0 diff --git a/docker/test/integration/base/Dockerfile b/docker/test/integration/base/Dockerfile index 270b40e23a6..469251f648c 100644 --- a/docker/test/integration/base/Dockerfile +++ b/docker/test/integration/base/Dockerfile @@ -33,7 +33,8 @@ RUN apt-get update \ && apt-get clean \ && rm -rf /var/lib/apt/lists/* /var/cache/debconf /tmp/* -RUN pip3 install pycurl +COPY requirements.txt / +RUN pip3 install --no-cache-dir -r requirements.txt && rm -rf /root/.cache/pip # Architecture of the image when BuildKit/buildx is used ARG TARGETARCH diff --git a/docker/test/integration/base/requirements.txt b/docker/test/integration/base/requirements.txt new file mode 100644 index 00000000000..d195d8deaf6 --- /dev/null +++ b/docker/test/integration/base/requirements.txt @@ -0,0 +1,26 @@ +blinker==1.4 +cryptography==3.4.8 +dbus-python==1.2.18 +distro==1.7.0 +httplib2==0.20.2 +importlib-metadata==4.6.4 +jeepney==0.7.1 +keyring==23.5.0 +launchpadlib==1.10.16 +lazr.restfulclient==0.14.4 +lazr.uri==1.0.6 +more-itertools==8.10.0 +oauthlib==3.2.0 +packaging==24.1 +pip==24.1.1 +pipdeptree==2.23.0 +pycurl==7.45.3 +PyJWT==2.3.0 +pyparsing==2.4.7 +python-apt==2.4.0+ubuntu3 +SecretStorage==3.3.1 +setuptools==59.6.0 +six==1.16.0 +wadllib==1.3.6 +wheel==0.37.1 +zipp==1.0.0 diff --git a/docker/test/integration/resolver/Dockerfile b/docker/test/integration/resolver/Dockerfile index 01b9b777614..b35a7262651 100644 --- a/docker/test/integration/resolver/Dockerfile +++ b/docker/test/integration/resolver/Dockerfile @@ -2,4 +2,5 @@ # Helper docker container to run python bottle apps FROM python:3 -RUN python -m pip install bottle +COPY requirements.txt / +RUN python -m pip install --no-cache-dir -r requirements.txt diff --git a/docker/test/integration/resolver/requirements.txt b/docker/test/integration/resolver/requirements.txt new file mode 100644 index 00000000000..fbf85295329 --- /dev/null +++ b/docker/test/integration/resolver/requirements.txt @@ -0,0 +1,6 @@ +bottle==0.12.25 +packaging==24.1 +pip==23.2.1 +pipdeptree==2.23.0 +setuptools==69.0.3 +wheel==0.42.0 diff --git a/docker/test/integration/runner/Dockerfile b/docker/test/integration/runner/Dockerfile index 23d8a37d822..d250b746e7d 100644 --- a/docker/test/integration/runner/Dockerfile +++ b/docker/test/integration/runner/Dockerfile @@ -26,7 +26,6 @@ RUN apt-get update \ libicu-dev \ bsdutils \ curl \ - python3-pika \ liblua5.1-dev \ luajit \ libssl-dev \ @@ -61,49 +60,8 @@ RUN curl -fsSL https://download.docker.com/linux/ubuntu/gpg | apt-key add - \ # kazoo 2.10.0 is broken # https://s3.amazonaws.com/clickhouse-test-reports/59337/524625a1d2f4cc608a3f1059e3df2c30f353a649/integration_tests__asan__analyzer__[5_6].html -RUN python3 -m pip install --no-cache-dir \ - PyMySQL==1.1.0 \ - asyncio==3.4.3 \ - avro==1.10.2 \ - azure-storage-blob==12.19.0 \ - boto3==1.34.24 \ - cassandra-driver==3.29.0 \ - confluent-kafka==2.3.0 \ - delta-spark==2.3.0 \ - dict2xml==1.7.4 \ - dicttoxml==1.7.16 \ - docker==6.1.3 \ - docker-compose==1.29.2 \ - grpcio==1.60.0 \ - grpcio-tools==1.60.0 \ - kafka-python==2.0.2 \ - lz4==4.3.3 \ - minio==7.2.3 \ - nats-py==2.6.0 \ - protobuf==4.25.2 \ - kazoo==2.9.0 \ - psycopg2-binary==2.9.6 \ - pyhdfs==0.3.1 \ - pymongo==3.11.0 \ - pyspark==3.3.2 \ - pytest==7.4.4 \ - pytest-order==1.0.0 \ - pytest-random==0.2 \ - pytest-repeat==0.9.3 \ - pytest-timeout==2.2.0 \ - pytest-xdist==3.5.0 \ - pytest-reportlog==0.4.0 \ - pytz==2023.3.post1 \ - pyyaml==5.3.1 \ - redis==5.0.1 \ - requests-kerberos==0.14.0 \ - tzlocal==2.1 \ - retry==0.9.2 \ - bs4==0.0.2 \ - lxml==5.1.0 \ - urllib3==2.0.7 \ - jwcrypto==1.5.6 -# bs4, lxml are for cloud tests, do not delete +COPY requirements.txt / +RUN python3 -m pip install --no-cache-dir -r requirements.txt # Hudi supports only spark 3.3.*, not 3.4 RUN curl -fsSL -O https://archive.apache.org/dist/spark/spark-3.3.2/spark-3.3.2-bin-hadoop3.tgz \ diff --git a/docker/test/integration/runner/requirements.txt b/docker/test/integration/runner/requirements.txt new file mode 100644 index 00000000000..8a77d8abf77 --- /dev/null +++ b/docker/test/integration/runner/requirements.txt @@ -0,0 +1,113 @@ +PyHDFS==0.3.1 +PyJWT==2.3.0 +PyMySQL==1.1.0 +PyNaCl==1.5.0 +PyYAML==5.3.1 +SecretStorage==3.3.1 +argon2-cffi-bindings==21.2.0 +argon2-cffi==23.1.0 +async-timeout==4.0.3 +asyncio==3.4.3 +attrs==23.2.0 +avro==1.10.2 +azure-core==1.30.1 +azure-storage-blob==12.19.0 +bcrypt==4.1.3 +beautifulsoup4==4.12.3 +blinker==1.4 +boto3==1.34.24 +botocore==1.34.101 +bs4==0.0.2 +cassandra-driver==3.29.0 +certifi==2024.2.2 +cffi==1.16.0 +charset-normalizer==3.3.2 +click==8.1.7 +confluent-kafka==2.3.0 +cryptography==3.4.8 +dbus-python==1.2.18 +decorator==5.1.1 +delta-spark==2.3.0 +dict2xml==1.7.4 +dicttoxml==1.7.16 +distro-info==1.1+ubuntu0.2 +distro==1.7.0 +docker-compose==1.29.2 +docker==6.1.3 +dockerpty==0.4.1 +docopt==0.6.2 +exceptiongroup==1.2.1 +execnet==2.1.1 +geomet==0.2.1.post1 +grpcio-tools==1.60.0 +grpcio==1.60.0 +gssapi==1.8.3 +httplib2==0.20.2 +idna==3.7 +importlib-metadata==4.6.4 +iniconfig==2.0.0 +isodate==0.6.1 +jeepney==0.7.1 +jmespath==1.0.1 +jsonschema==3.2.0 +jwcrypto==1.5.6 +kafka-python==2.0.2 +kazoo==2.9.0 +keyring==23.5.0 +krb5==0.5.1 +launchpadlib==1.10.16 +lazr.restfulclient==0.14.4 +lazr.uri==1.0.6 +lxml==5.1.0 +lz4==4.3.3 +minio==7.2.3 +more-itertools==8.10.0 +nats-py==2.6.0 +oauthlib==3.2.0 +packaging==24.0 +paramiko==3.4.0 +pika==1.2.0 +pip==24.1.1 +pipdeptree==2.23.0 +pluggy==1.5.0 +protobuf==4.25.2 +psycopg2-binary==2.9.6 +py4j==0.10.9.5 +py==1.11.0 +pycparser==2.22 +pycryptodome==3.20.0 +pymongo==3.11.0 +pyparsing==2.4.7 +pyrsistent==0.20.0 +pyspark==3.3.2 +pyspnego==0.10.2 +pytest-order==1.0.0 +pytest-random==0.2 +pytest-repeat==0.9.3 +pytest-reportlog==0.4.0 +pytest-timeout==2.2.0 +pytest-xdist==3.5.0 +pytest==7.4.4 +python-apt==2.4.0+ubuntu3 +python-dateutil==2.9.0.post0 +python-dotenv==0.21.1 +pytz==2023.3.post1 +redis==5.0.1 +requests-kerberos==0.14.0 +requests==2.31.0 +retry==0.9.2 +s3transfer==0.10.1 +setuptools==59.6.0 +simplejson==3.19.2 +six==1.16.0 +soupsieve==2.5 +texttable==1.7.0 +tomli==2.0.1 +typing_extensions==4.11.0 +tzlocal==2.1 +unattended-upgrades==0.1 +urllib3==2.0.7 +wadllib==1.3.6 +websocket-client==0.59.0 +wheel==0.37.1 +zipp==1.0.0 diff --git a/docker/test/libfuzzer/Dockerfile b/docker/test/libfuzzer/Dockerfile index c9802a0e44e..e6eb2ae336e 100644 --- a/docker/test/libfuzzer/Dockerfile +++ b/docker/test/libfuzzer/Dockerfile @@ -1,3 +1,4 @@ +# docker build -t clickhouse/libfuzzer . ARG FROM_TAG=latest FROM clickhouse/test-base:$FROM_TAG @@ -29,7 +30,8 @@ RUN apt-get update \ && apt-get clean \ && rm -rf /var/lib/apt/lists/* /var/cache/debconf /tmp/* -RUN pip3 install Jinja2 +COPY requirements.txt / +RUN pip3 install --no-cache-dir -r /requirements.txt COPY * / diff --git a/docker/test/libfuzzer/requirements.txt b/docker/test/libfuzzer/requirements.txt new file mode 100644 index 00000000000..3dce93e023b --- /dev/null +++ b/docker/test/libfuzzer/requirements.txt @@ -0,0 +1,27 @@ +blinker==1.4 +cryptography==3.4.8 +dbus-python==1.2.18 +distro==1.7.0 +httplib2==0.20.2 +importlib-metadata==4.6.4 +jeepney==0.7.1 +Jinja2==3.1.4 +keyring==23.5.0 +launchpadlib==1.10.16 +lazr.restfulclient==0.14.4 +lazr.uri==1.0.6 +MarkupSafe==2.1.5 +more-itertools==8.10.0 +oauthlib==3.2.0 +packaging==24.1 +pip==24.1.1 +pipdeptree==2.23.0 +PyJWT==2.3.0 +pyparsing==2.4.7 +python-apt==2.4.0+ubuntu3 +SecretStorage==3.3.1 +setuptools==59.6.0 +six==1.16.0 +wadllib==1.3.6 +wheel==0.37.1 +zipp==1.0.0 diff --git a/docker/test/performance-comparison/Dockerfile b/docker/test/performance-comparison/Dockerfile index 1835900b316..c68a39f6f70 100644 --- a/docker/test/performance-comparison/Dockerfile +++ b/docker/test/performance-comparison/Dockerfile @@ -23,7 +23,6 @@ RUN apt-get update \ python3 \ python3-dev \ python3-pip \ - python3-setuptools \ rsync \ tree \ tzdata \ @@ -33,12 +32,14 @@ RUN apt-get update \ cargo \ ripgrep \ zstd \ - && pip3 --no-cache-dir install 'clickhouse-driver==0.2.1' scipy \ && apt-get purge --yes python3-dev g++ \ && apt-get autoremove --yes \ && apt-get clean \ && rm -rf /var/lib/apt/lists/* /var/cache/debconf /tmp/* +COPY requirements.txt / +RUN pip3 --no-cache-dir install -r requirements.txt + COPY run.sh / CMD ["bash", "/run.sh"] diff --git a/docker/test/performance-comparison/requirements.txt b/docker/test/performance-comparison/requirements.txt new file mode 100644 index 00000000000..932527cc022 --- /dev/null +++ b/docker/test/performance-comparison/requirements.txt @@ -0,0 +1,32 @@ +blinker==1.4 +clickhouse-driver==0.2.7 +cryptography==3.4.8 +dbus-python==1.2.18 +distro==1.7.0 +httplib2==0.20.2 +importlib-metadata==4.6.4 +jeepney==0.7.1 +keyring==23.5.0 +launchpadlib==1.10.16 +lazr.restfulclient==0.14.4 +lazr.uri==1.0.6 +more-itertools==8.10.0 +numpy==1.26.3 +oauthlib==3.2.0 +packaging==24.1 +pip==24.1.1 +pipdeptree==2.23.0 +Pygments==2.11.2 +PyJWT==2.3.0 +pyparsing==2.4.7 +python-apt==2.4.0+ubuntu3 +pytz==2023.4 +PyYAML==6.0.1 +scipy==1.12.0 +SecretStorage==3.3.1 +setuptools==59.6.0 +six==1.16.0 +tzlocal==2.1 +wadllib==1.3.6 +wheel==0.37.1 +zipp==1.0.0 diff --git a/docker/test/sqllogic/Dockerfile b/docker/test/sqllogic/Dockerfile index 1ea1e52e6fa..1425e12cd84 100644 --- a/docker/test/sqllogic/Dockerfile +++ b/docker/test/sqllogic/Dockerfile @@ -18,11 +18,8 @@ RUN apt-get update --yes \ && apt-get clean \ && rm -rf /var/lib/apt/lists/* /var/cache/debconf /tmp/* -RUN pip3 install \ - numpy \ - pyodbc \ - deepdiff \ - sqlglot +COPY requirements.txt / +RUN pip3 install --no-cache-dir -r /requirements.txt ARG odbc_driver_url="https://github.com/ClickHouse/clickhouse-odbc/releases/download/v1.1.6.20200320/clickhouse-odbc-1.1.6-Linux.tar.gz" diff --git a/docker/test/sqllogic/requirements.txt b/docker/test/sqllogic/requirements.txt new file mode 100644 index 00000000000..abc0a368659 --- /dev/null +++ b/docker/test/sqllogic/requirements.txt @@ -0,0 +1,30 @@ +blinker==1.4 +cryptography==3.4.8 +dbus-python==1.2.18 +deepdiff==7.0.1 +distro==1.7.0 +httplib2==0.20.2 +importlib-metadata==4.6.4 +jeepney==0.7.1 +keyring==23.5.0 +launchpadlib==1.10.16 +lazr.restfulclient==0.14.4 +lazr.uri==1.0.6 +more-itertools==8.10.0 +numpy==1.26.4 +oauthlib==3.2.0 +ordered-set==4.1.0 +packaging==24.1 +pip==24.1.1 +pipdeptree==2.23.0 +PyJWT==2.3.0 +pyodbc==5.1.0 +pyparsing==2.4.7 +python-apt==2.4.0+ubuntu3 +SecretStorage==3.3.1 +setuptools==59.6.0 +six==1.16.0 +sqlglot==23.16.0 +wadllib==1.3.6 +wheel==0.37.1 +zipp==1.0.0 diff --git a/docker/test/sqltest/Dockerfile b/docker/test/sqltest/Dockerfile index 7f59f65761f..71d915b0c7a 100644 --- a/docker/test/sqltest/Dockerfile +++ b/docker/test/sqltest/Dockerfile @@ -14,9 +14,8 @@ RUN apt-get update --yes \ && apt-get clean \ && rm -rf /var/lib/apt/lists/* /var/cache/debconf /tmp/* -RUN pip3 install \ - pyyaml \ - clickhouse-driver +COPY requirements.txt / +RUN pip3 install --no-cache-dir -r /requirements.txt ARG sqltest_repo="https://github.com/elliotchance/sqltest/" diff --git a/docker/test/sqltest/requirements.txt b/docker/test/sqltest/requirements.txt new file mode 100644 index 00000000000..4a0ae3edbac --- /dev/null +++ b/docker/test/sqltest/requirements.txt @@ -0,0 +1,29 @@ +blinker==1.4 +clickhouse-driver==0.2.7 +cryptography==3.4.8 +dbus-python==1.2.18 +distro==1.7.0 +httplib2==0.20.2 +importlib-metadata==4.6.4 +jeepney==0.7.1 +keyring==23.5.0 +launchpadlib==1.10.16 +lazr.restfulclient==0.14.4 +lazr.uri==1.0.6 +more-itertools==8.10.0 +oauthlib==3.2.0 +packaging==24.1 +pip==24.1.1 +pipdeptree==2.23.0 +PyJWT==2.3.0 +pyparsing==2.4.7 +python-apt==2.4.0+ubuntu3 +pytz==2024.1 +PyYAML==6.0.1 +SecretStorage==3.3.1 +setuptools==59.6.0 +six==1.16.0 +tzlocal==5.2 +wadllib==1.3.6 +wheel==0.37.1 +zipp==1.0.0 diff --git a/docker/test/stateful/Dockerfile b/docker/test/stateful/Dockerfile index 355e70f180e..0daf88cad7e 100644 --- a/docker/test/stateful/Dockerfile +++ b/docker/test/stateful/Dockerfile @@ -6,7 +6,6 @@ FROM clickhouse/stateless-test:$FROM_TAG RUN apt-get update -y \ && env DEBIAN_FRONTEND=noninteractive \ apt-get install --yes --no-install-recommends \ - python3-requests \ nodejs \ npm \ && apt-get clean \ diff --git a/docker/test/stateless/Dockerfile b/docker/test/stateless/Dockerfile index c3d80a7334b..5a655a3fd2b 100644 --- a/docker/test/stateless/Dockerfile +++ b/docker/test/stateless/Dockerfile @@ -25,10 +25,7 @@ RUN apt-get update -y \ openssl \ postgresql-client \ python3 \ - python3-lxml \ python3-pip \ - python3-requests \ - python3-termcolor \ qemu-user-static \ sqlite3 \ sudo \ @@ -51,7 +48,8 @@ RUN curl -OL https://github.com/protocolbuffers/protobuf/releases/download/v${PR && unzip protoc-${PROTOC_VERSION}-linux-x86_64.zip -d /usr/local \ && rm protoc-${PROTOC_VERSION}-linux-x86_64.zip -RUN pip3 install numpy==1.26.3 scipy==1.12.0 pandas==1.5.3 Jinja2==3.1.3 pyarrow==15.0.0 +COPY requirements.txt / +RUN pip3 install --no-cache-dir -r /requirements.txt RUN mkdir -p /tmp/clickhouse-odbc-tmp \ && cd /tmp/clickhouse-odbc-tmp \ diff --git a/docker/test/stateless/requirements.txt b/docker/test/stateless/requirements.txt new file mode 100644 index 00000000000..3284107e24e --- /dev/null +++ b/docker/test/stateless/requirements.txt @@ -0,0 +1,51 @@ +awscli==1.22.34 +blinker==1.4 +botocore==1.23.34 +certifi==2020.6.20 +chardet==4.0.0 +colorama==0.4.4 +cryptography==3.4.8 +dbus-python==1.2.18 +distro==1.7.0 +docutils==0.17.1 +gyp==0.1 +httplib2==0.20.2 +idna==3.3 +importlib-metadata==4.6.4 +jeepney==0.7.1 +Jinja2==3.1.3 +jmespath==0.10.0 +keyring==23.5.0 +launchpadlib==1.10.16 +lazr.restfulclient==0.14.4 +lazr.uri==1.0.6 +lxml==4.8.0 +MarkupSafe==2.1.5 +more-itertools==8.10.0 +numpy==1.26.3 +oauthlib==3.2.0 +packaging==24.1 +pandas==1.5.3 +pip==24.1.1 +pipdeptree==2.23.0 +pyarrow==15.0.0 +pyasn1==0.4.8 +PyJWT==2.3.0 +pyparsing==2.4.7 +python-apt==2.4.0+ubuntu3 +python-dateutil==2.8.1 +pytz==2024.1 +PyYAML==6.0.1 +requests==2.32.3 +roman==3.3 +rsa==4.8 +s3transfer==0.5.0 +scipy==1.12.0 +SecretStorage==3.3.1 +setuptools==59.6.0 +six==1.16.0 +termcolor==1.1.0 +urllib3==1.26.5 +wadllib==1.3.6 +wheel==0.37.1 +zipp==1.0.0 diff --git a/docker/test/style/Dockerfile b/docker/test/style/Dockerfile index 7cd712b73f6..cdc1d1fa095 100644 --- a/docker/test/style/Dockerfile +++ b/docker/test/style/Dockerfile @@ -23,22 +23,8 @@ RUN apt-get update && env DEBIAN_FRONTEND=noninteractive apt-get install --yes \ && rm -rf /var/lib/apt/lists/* /var/cache/debconf /tmp/* # python-magic is the same version as in Ubuntu 22.04 -RUN pip3 install \ - PyGithub \ - black==23.12.0 \ - boto3 \ - codespell==2.2.1 \ - mypy==1.8.0 \ - pylint==3.1.0 \ - python-magic==0.4.24 \ - flake8==4.0.1 \ - requests \ - thefuzz \ - tqdm==4.66.4 \ - types-requests \ - unidiff \ - jwt \ - && rm -rf /root/.cache/pip +COPY requirements.txt / +RUN pip3 install --no-cache-dir -r requirements.txt RUN echo "en_US.UTF-8 UTF-8" > /etc/locale.gen && locale-gen en_US.UTF-8 ENV LC_ALL en_US.UTF-8 diff --git a/docker/test/style/requirements.txt b/docker/test/style/requirements.txt new file mode 100644 index 00000000000..bb0cd55dd1a --- /dev/null +++ b/docker/test/style/requirements.txt @@ -0,0 +1,58 @@ +aiohttp==3.9.5 +aiosignal==1.3.1 +astroid==3.1.0 +async-timeout==4.0.3 +attrs==23.2.0 +black==23.12.0 +boto3==1.34.131 +botocore==1.34.131 +certifi==2024.6.2 +cffi==1.16.0 +charset-normalizer==3.3.2 +click==8.1.7 +codespell==2.2.1 +cryptography==42.0.8 +Deprecated==1.2.14 +dill==0.3.8 +flake8==4.0.1 +frozenlist==1.4.1 +idna==3.7 +isort==5.13.2 +jmespath==1.0.1 +jwt==1.3.1 +mccabe==0.6.1 +multidict==6.0.5 +mypy==1.8.0 +mypy-extensions==1.0.0 +packaging==24.1 +pathspec==0.9.0 +pip==24.1.1 +pipdeptree==2.23.0 +platformdirs==4.2.2 +pycodestyle==2.8.0 +pycparser==2.22 +pyflakes==2.4.0 +PyGithub==2.3.0 +PyJWT==2.8.0 +pylint==3.1.0 +PyNaCl==1.5.0 +python-dateutil==2.9.0.post0 +python-magic==0.4.24 +PyYAML==6.0.1 +rapidfuzz==3.9.3 +requests==2.32.3 +s3transfer==0.10.1 +setuptools==59.6.0 +six==1.16.0 +thefuzz==0.22.1 +tomli==2.0.1 +tomlkit==0.12.5 +tqdm==4.66.4 +types-requests==2.32.0.20240622 +typing_extensions==4.12.2 +unidiff==0.7.5 +urllib3==2.2.2 +wheel==0.37.1 +wrapt==1.16.0 +yamllint==1.26.3 +yarl==1.9.4 From afac188ae31784520e8f093736ca8fd9427b685b Mon Sep 17 00:00:00 2001 From: robot-clickhouse Date: Wed, 3 Jul 2024 20:52:54 +0000 Subject: [PATCH 170/417] Automatic style fix --- docker/reqgenerator.py | 15 ++++++++++----- 1 file changed, 10 insertions(+), 5 deletions(-) diff --git a/docker/reqgenerator.py b/docker/reqgenerator.py index 89b901413d6..6c1d89ac0ac 100644 --- a/docker/reqgenerator.py +++ b/docker/reqgenerator.py @@ -6,29 +6,34 @@ import subprocess import os import sys + def build_docker_deps(image_name, imagedir): cmd = f"""docker run --entrypoint "/bin/bash" {image_name} -c "pip install pipdeptree 2>/dev/null 1>/dev/null && pipdeptree --freeze --warn silence | sed 's/ \+//g' | sort | uniq" > {imagedir}/requirements.txt""" subprocess.check_call(cmd, shell=True) + def check_docker_file_install_with_pip(filepath): image_name = None - with open(filepath, 'r') as f: + with open(filepath, "r") as f: for line in f: - if 'docker build' in line: - arr = line.split(' ') + if "docker build" in line: + arr = line.split(" ") if len(arr) > 4: image_name = arr[4] - if 'pip3 install' in line or 'pip install' in line: + if "pip3 install" in line or "pip install" in line: return image_name, True return image_name, False + def process_affected_images(images_dir): for root, _dirs, files in os.walk(images_dir): for f in files: if f == "Dockerfile": docker_file_path = os.path.join(root, f) print("Checking image on path", docker_file_path) - image_name, has_pip = check_docker_file_install_with_pip(docker_file_path) + image_name, has_pip = check_docker_file_install_with_pip( + docker_file_path + ) if has_pip: print("Find pip in", image_name) try: From c4005d7e06ba4f775f34db01ff9fc7b61c1fefb0 Mon Sep 17 00:00:00 2001 From: Nikita Fomichev Date: Wed, 3 Jul 2024 23:13:50 +0200 Subject: [PATCH 171/417] Fix fasttest --- docker/test/fasttest/run.sh | 2 ++ 1 file changed, 2 insertions(+) diff --git a/docker/test/fasttest/run.sh b/docker/test/fasttest/run.sh index c80ea193010..c015d3a3542 100755 --- a/docker/test/fasttest/run.sh +++ b/docker/test/fasttest/run.sh @@ -84,6 +84,8 @@ function start_server echo "ClickHouse server pid '$server_pid' started and responded" } +export -f start_server + function clone_root { [ "$UID" -eq 0 ] && git config --global --add safe.directory "$FASTTEST_SOURCE" From 05e1e0f6013909838feec179ec3fdcc97b63e261 Mon Sep 17 00:00:00 2001 From: Nikita Fomichev Date: Wed, 3 Jul 2024 23:15:24 +0200 Subject: [PATCH 172/417] 03167_base64_url_functions_sh.sh add tag no-fasttest --- tests/queries/0_stateless/03167_base64_url_functions_sh.sh | 1 + 1 file changed, 1 insertion(+) diff --git a/tests/queries/0_stateless/03167_base64_url_functions_sh.sh b/tests/queries/0_stateless/03167_base64_url_functions_sh.sh index ec3170b165c..f4d5addf370 100755 --- a/tests/queries/0_stateless/03167_base64_url_functions_sh.sh +++ b/tests/queries/0_stateless/03167_base64_url_functions_sh.sh @@ -1,4 +1,5 @@ #!/usr/bin/env bash +# Tags: no-fasttest # shellcheck disable=SC2155 set -e From 41b9216dd1d862b46ed72d50e899197e2fec9daa Mon Sep 17 00:00:00 2001 From: Kruglov Pavel <48961922+Avogar@users.noreply.github.com> Date: Thu, 4 Jul 2024 00:22:41 +0200 Subject: [PATCH 173/417] Fix build --- src/Formats/JSONExtractTree.cpp | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/src/Formats/JSONExtractTree.cpp b/src/Formats/JSONExtractTree.cpp index b94981e7cb4..827f276311a 100644 --- a/src/Formats/JSONExtractTree.cpp +++ b/src/Formats/JSONExtractTree.cpp @@ -1465,7 +1465,7 @@ std::unique_ptr> buildJSONExtractTree(const Data case TypeIndex::LowCardinality: { /// To optimize inserting into LowCardinality we have special nodes for LowCardinality of numeric and string types. - const auto & lc_type = assert_cast(*type)); + const auto & lc_type = assert_cast(*type); auto dictionary_type = removeNullable(lc_type.getDictionaryType()); bool is_nullable = lc_type.isLowCardinalityNullable(); From 5c0d88221fb6794c94ed586ef781d3c26842276f Mon Sep 17 00:00:00 2001 From: kevinyhzou Date: Thu, 4 Jul 2024 09:43:22 +0800 Subject: [PATCH 174/417] check style --- tests/queries/0_stateless/03198_orc_read_time_zone.sh | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/tests/queries/0_stateless/03198_orc_read_time_zone.sh b/tests/queries/0_stateless/03198_orc_read_time_zone.sh index fec4e2bcc72..adef85ac6f1 100644 --- a/tests/queries/0_stateless/03198_orc_read_time_zone.sh +++ b/tests/queries/0_stateless/03198_orc_read_time_zone.sh @@ -6,6 +6,6 @@ CURDIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd) $CLICKHOUSE_CLIENT -q "drop table if exists test" $CLICKHOUSE_CLIENT -q "create table test(id UInt64, t DateTime64) Engine=MergeTree order by id" -$CLICKHOUSE_CLIENT -q "insert into test from infile '"$CURDIR"/data_orc/read_time_zone.snappy.orc' SETTINGS input_format_orc_read_use_writer_time_zone=true FORMAT ORC" +$CLICKHOUSE_CLIENT -q "insert into test from infile '$CURDIR/data_orc/read_time_zone.snappy.orc' SETTINGS input_format_orc_read_use_writer_time_zone=true FORMAT ORC" $CLICKHOUSE_CLIENT -q "select * from test" $CLICKHOUSE_CLIENT -q "drop table test" \ No newline at end of file From 45a73dc3b585597b4c9db99236ce4636726d6793 Mon Sep 17 00:00:00 2001 From: kevinyhzou Date: Thu, 4 Jul 2024 10:37:55 +0800 Subject: [PATCH 175/417] fast test --- src/Core/SettingsChangesHistory.cpp | 1 + 1 file changed, 1 insertion(+) diff --git a/src/Core/SettingsChangesHistory.cpp b/src/Core/SettingsChangesHistory.cpp index b0725340f46..f34dc15b777 100644 --- a/src/Core/SettingsChangesHistory.cpp +++ b/src/Core/SettingsChangesHistory.cpp @@ -59,6 +59,7 @@ static std::initializer_list Date: Thu, 4 Jul 2024 11:01:12 +0800 Subject: [PATCH 176/417] change permission of test shell --- tests/queries/0_stateless/03198_orc_read_time_zone.sh | 0 1 file changed, 0 insertions(+), 0 deletions(-) mode change 100644 => 100755 tests/queries/0_stateless/03198_orc_read_time_zone.sh diff --git a/tests/queries/0_stateless/03198_orc_read_time_zone.sh b/tests/queries/0_stateless/03198_orc_read_time_zone.sh old mode 100644 new mode 100755 From baa38ecd61109df0a8f19264dc8bb9e5c6b93297 Mon Sep 17 00:00:00 2001 From: kevinyhzou Date: Thu, 4 Jul 2024 11:32:37 +0800 Subject: [PATCH 177/417] add no-fasttest tag --- tests/queries/0_stateless/03198_orc_read_time_zone.sh | 1 + 1 file changed, 1 insertion(+) diff --git a/tests/queries/0_stateless/03198_orc_read_time_zone.sh b/tests/queries/0_stateless/03198_orc_read_time_zone.sh index adef85ac6f1..ba868e29dce 100755 --- a/tests/queries/0_stateless/03198_orc_read_time_zone.sh +++ b/tests/queries/0_stateless/03198_orc_read_time_zone.sh @@ -1,4 +1,5 @@ #!/usr/bin/env bash +# Tags: no-fasttest CURDIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd) # shellcheck source=../shell_config.sh From e38e105e20ae6406a60baa0a08beed518676b346 Mon Sep 17 00:00:00 2001 From: lgbo-ustc Date: Thu, 18 Apr 2024 16:44:20 +0800 Subject: [PATCH 178/417] add window function percent_rank --- .../sql-reference/window-functions/index.md | 1 + src/Processors/Transforms/WindowTransform.cpp | 171 ++++++++++++++++-- .../01592_window_functions.reference | 12 ++ .../0_stateless/01592_window_functions.sql | 18 ++ 4 files changed, 184 insertions(+), 18 deletions(-) diff --git a/docs/en/sql-reference/window-functions/index.md b/docs/en/sql-reference/window-functions/index.md index 49076f3cbe1..8097abc0b15 100644 --- a/docs/en/sql-reference/window-functions/index.md +++ b/docs/en/sql-reference/window-functions/index.md @@ -23,6 +23,7 @@ ClickHouse supports the standard grammar for defining windows and window functio | `GROUPS` frame | ❌ | | Calculating aggregate functions over a frame (`sum(value) over (order by time)`) | ✅ (All aggregate functions are supported) | | `rank()`, `dense_rank()`, `row_number()` | ✅ | +| `percent_rank()` | ✅ equal to `ifNull((rank() OVER(PARTITION BY x order by y) - 1) / nullif(count(1) OVER(PARTITION BY x) -1, 0), 0)`, but more efficent| | `lag/lead(value, offset)` | ❌
You can use one of the following workarounds:
1) `any(value) over (.... rows between preceding and preceding)`, or `following` for `lead`
2) `lagInFrame/leadInFrame`, which are analogous, but respect the window frame. To get behavior identical to `lag/lead`, use `rows between unbounded preceding and unbounded following` | | ntile(buckets) | ✅
Specify window like, (partition by x order by y rows between unbounded preceding and unrounded following). | diff --git a/src/Processors/Transforms/WindowTransform.cpp b/src/Processors/Transforms/WindowTransform.cpp index b9f61d30182..0c7caca9de5 100644 --- a/src/Processors/Transforms/WindowTransform.cpp +++ b/src/Processors/Transforms/WindowTransform.cpp @@ -16,6 +16,10 @@ #include #include #include +#include "WindowTransform.h" + +#include +#include #include @@ -1609,8 +1613,37 @@ struct WindowFunctionHelpers { recurrent_detail::setValueToOutputColumn(transform, function_index, value); } + + ALWAYS_INLINE static bool checkPartitionEnterFirstRow(const WindowTransform * transform) { return transform->current_row_number == 1; } + + ALWAYS_INLINE static bool checkPartitionEnterLastRow(const WindowTransform * transform) + { + /// when partition_ended is false, it means that we don't reach the last row in this partition. + /// But when partition_ended is true, it doesn't mean that we reach the last row in this partition. + /// partition_ended is true when + /// - the input has finished. or + /// - current block contains next partition's data. + /// This is for fast check. + if (!transform->partition_ended) + return false; + + auto current_row = transform->current_row; + current_row.row++; + const auto & partitoin_end_row = transform->partition_end; + /// If current_row == partitoin_end_row, return true. otherwise + if (current_row != partitoin_end_row) + { + if (current_row.row < transform->blockRowsNumber(current_row)) + return false; + /// Next row to current_row may belong to next block. + if (partitoin_end_row.block != current_row.block + 1 || partitoin_end_row.row) + return false; + } + return true; + } }; + template struct StatefulWindowFunction : public WindowFunction { @@ -1639,6 +1672,8 @@ struct StatefulWindowFunction : public WindowFunction { return *reinterpret_cast(workspace.aggregate_function_state.data()); } + + }; struct ExponentialTimeDecayedSumState @@ -2128,7 +2163,7 @@ namespace } } // new partition - if (transform->current_row_number == 1) [[unlikely]] + if (WindowFunctionHelpers::checkPartitionEnterFirstRow(transform)) [[unlikely]] { current_partition_rows = 0; current_partition_inserted_row = 0; @@ -2137,25 +2172,9 @@ namespace current_partition_rows++; // Only do the action when we meet the last row in this partition. - if (!transform->partition_ended) + if (!WindowFunctionHelpers::checkPartitionEnterLastRow(transform)) return; - else - { - auto current_row = transform->current_row; - current_row.row++; - const auto & end_row = transform->partition_end; - if (current_row != end_row) - { - if (current_row.row < transform->blockRowsNumber(current_row)) - return; - if (end_row.block != current_row.block + 1 || end_row.row) - { - return; - } - // else, current_row is the last input row. - } - } auto bucket_capacity = current_partition_rows / buckets; auto capacity_diff = current_partition_rows - bucket_capacity * buckets; @@ -2211,6 +2230,115 @@ namespace } } +namespace +{ +struct PercentRankState +{ + RowNumber start_row; + UInt64 current_partition_rows = 0; +}; +} + +struct WindowFunctionPercentRank final : public StatefulWindowFunction +{ +public: + WindowFunctionPercentRank(const std::string & name_, + const DataTypes & argument_types_, const Array & parameters_) + : StatefulWindowFunction(name_, argument_types_, parameters_, std::make_shared()) + {} + + bool allocatesMemoryInArena() const override { return false; } + + std::optional getDefaultFrame() const override + { + WindowFrame frame; + frame.type = WindowFrame::FrameType::ROWS; + frame.end_type = WindowFrame::BoundaryType::Unbounded; + return frame; + } + + void windowInsertResultInto(const WindowTransform * transform, size_t function_index) const override + { + checkFrameBoundType(transform); + + auto & state = getWorkspaceState(transform, function_index); + if (WindowFunctionHelpers::checkPartitionEnterFirstRow(transform)) + { + state.current_partition_rows = 0; + state.start_row = transform->current_row; + } + + insertRankIntoColumn(transform, function_index); + state.current_partition_rows++; + + if (!WindowFunctionHelpers::checkPartitionEnterLastRow(transform)) + { + return; + } + + UInt64 remaining_rows = state.current_partition_rows; + Float64 percent_rank_denominator = state.current_partition_rows - 1; + + if (remaining_rows <= 1) + return; + while(remaining_rows > 0) + { + auto block_rows_number = transform->blockRowsNumber(state.start_row); + auto available_block_rows = block_rows_number - state.start_row.row; + if (available_block_rows <= remaining_rows) + { + auto & to_column = *transform->blockAt(state.start_row).output_columns[function_index]; + auto & data = assert_cast(to_column).getData(); + for (size_t i = state.start_row.row; i < block_rows_number; ++i) + data[i] = data[i] / percent_rank_denominator; + + state.start_row.block++; + state.start_row.row = 0; + remaining_rows -= available_block_rows; + } + else + { + auto & to_column = *transform->blockAt(state.start_row).output_columns[function_index]; + auto & data = assert_cast(to_column).getData(); + for (size_t i = state.start_row.row, n = state.start_row.row + remaining_rows; i < n; ++i) + { + data[i] = data[i]/percent_rank_denominator; + } + state.start_row.row += remaining_rows; + remaining_rows = 0; + } + } + } + + + inline PercentRankState & getWorkspaceState(const WindowTransform * transform, size_t function_index) const + { + const auto & workspace = transform->workspaces[function_index]; + return getState(workspace); + } + + inline void insertRankIntoColumn(const WindowTransform * transform, size_t function_index) const + { + auto & to_column = *transform->blockAt(transform->current_row).output_columns[function_index]; + assert_cast(to_column).getData().push_back(static_cast(transform->peer_group_start_row_number) - 1); + } +private: + mutable bool has_check_frame_bound_type = false; + ALWAYS_INLINE void checkFrameBoundType(const WindowTransform * transform) const + { + if (has_check_frame_bound_type) + return; + if (transform->window_description.frame.begin_type != WindowFrame::BoundaryType::Unbounded + || transform->window_description.frame.end_type != WindowFrame::BoundaryType::Unbounded) + { + throw Exception( + ErrorCodes::BAD_ARGUMENTS, + "Window frame for function 'percent_rank' should be 'ROWS BETWEEN UNBOUNDED PRECEDING AND UNBOUNDED FOLLOWING'"); + } + has_check_frame_bound_type = true; + } +}; + // ClickHouse-specific variant of lag/lead that respects the window frame. template struct WindowFunctionLagLeadInFrame final : public WindowFunction @@ -2582,6 +2710,13 @@ void registerWindowFunctions(AggregateFunctionFactory & factory) parameters); }, properties}, AggregateFunctionFactory::CaseInsensitive); + factory.registerFunction("percent_rank", {[](const std::string & name, + const DataTypes & argument_types, const Array & parameters, const Settings *) + { + return std::make_shared(name, argument_types, + parameters); + }, properties}, AggregateFunctionFactory::CaseInsensitive); + factory.registerFunction("row_number", {[](const std::string & name, const DataTypes & argument_types, const Array & parameters, const Settings *) { diff --git a/tests/queries/0_stateless/01592_window_functions.reference b/tests/queries/0_stateless/01592_window_functions.reference index ec957dd7a02..0995def71e2 100644 --- a/tests/queries/0_stateless/01592_window_functions.reference +++ b/tests/queries/0_stateless/01592_window_functions.reference @@ -79,3 +79,15 @@ iPhone 900 Smartphone 500 500 Kindle Fire 150 Tablet 150 350 Samsung Galaxy Tab 200 Tablet 175 350 iPad 700 Tablet 350 350 +---- Q8 ---- +Lenovo Thinkpad Laptop 700 0 +Sony VAIO Laptop 700 0.3333333333333333 +Dell Vostro Laptop 800 0.6666666666666666 +HP Elite Laptop 1200 1 +Microsoft Lumia Smartphone 200 0 +HTC One Smartphone 400 0.3333333333333333 +Nexus Smartphone 500 0.6666666666666666 +iPhone Smartphone 900 1 +Kindle Fire Tablet 150 0 +Samsung Galaxy Tab Tablet 200 0.5 +iPad Tablet 700 1 diff --git a/tests/queries/0_stateless/01592_window_functions.sql b/tests/queries/0_stateless/01592_window_functions.sql index f0d173b1f20..e48e26b26d2 100644 --- a/tests/queries/0_stateless/01592_window_functions.sql +++ b/tests/queries/0_stateless/01592_window_functions.sql @@ -101,5 +101,23 @@ SELECT FROM products INNER JOIN product_groups USING (group_id)) t order by group_name, product_name, price; +select '---- Q8 ----'; +SELECT * +FROM +( + SELECT + product_name, + group_name, + price, + percent_rank() OVER (PARTITION BY group_name ORDER BY price ASC) AS r + FROM products + INNER JOIN product_groups USING (group_id) +) AS t +ORDER BY + group_name ASC, + r ASC, + product_name ASC, + price ASC; + drop table product_groups; drop table products; From 6e231eedcf04e4136ea56fcbbb4a43916241dd23 Mon Sep 17 00:00:00 2001 From: lgbo-ustc Date: Thu, 18 Apr 2024 17:16:20 +0800 Subject: [PATCH 179/417] fixed style --- src/Processors/Transforms/WindowTransform.cpp | 9 +-------- 1 file changed, 1 insertion(+), 8 deletions(-) diff --git a/src/Processors/Transforms/WindowTransform.cpp b/src/Processors/Transforms/WindowTransform.cpp index 0c7caca9de5..4758d5ca7f4 100644 --- a/src/Processors/Transforms/WindowTransform.cpp +++ b/src/Processors/Transforms/WindowTransform.cpp @@ -16,10 +16,6 @@ #include #include #include -#include "WindowTransform.h" - -#include -#include #include @@ -1643,7 +1639,6 @@ struct WindowFunctionHelpers } }; - template struct StatefulWindowFunction : public WindowFunction { @@ -1672,8 +1667,6 @@ struct StatefulWindowFunction : public WindowFunction { return *reinterpret_cast(workspace.aggregate_function_state.data()); } - - }; struct ExponentialTimeDecayedSumState @@ -2281,7 +2274,7 @@ public: if (remaining_rows <= 1) return; - while(remaining_rows > 0) + while (remaining_rows > 0) { auto block_rows_number = transform->blockRowsNumber(state.start_row); auto available_block_rows = block_rows_number - state.start_row.row; From 37d2ced74cd173c44015a89fcb9522ef9c3979ee Mon Sep 17 00:00:00 2001 From: lgbo-ustc Date: Thu, 18 Apr 2024 17:35:43 +0800 Subject: [PATCH 180/417] fixed typos --- docs/en/sql-reference/window-functions/index.md | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/docs/en/sql-reference/window-functions/index.md b/docs/en/sql-reference/window-functions/index.md index 8097abc0b15..814a7ac4aca 100644 --- a/docs/en/sql-reference/window-functions/index.md +++ b/docs/en/sql-reference/window-functions/index.md @@ -23,7 +23,7 @@ ClickHouse supports the standard grammar for defining windows and window functio | `GROUPS` frame | ❌ | | Calculating aggregate functions over a frame (`sum(value) over (order by time)`) | ✅ (All aggregate functions are supported) | | `rank()`, `dense_rank()`, `row_number()` | ✅ | -| `percent_rank()` | ✅ equal to `ifNull((rank() OVER(PARTITION BY x order by y) - 1) / nullif(count(1) OVER(PARTITION BY x) -1, 0), 0)`, but more efficent| +| `percent_rank()` | ✅ equal to `ifNull((rank() OVER(PARTITION BY x order by y) - 1) / nullif(count(1) OVER(PARTITION BY x) -1, 0), 0)`, but more efficient| | `lag/lead(value, offset)` | ❌
You can use one of the following workarounds:
1) `any(value) over (.... rows between preceding and preceding)`, or `following` for `lead`
2) `lagInFrame/leadInFrame`, which are analogous, but respect the window frame. To get behavior identical to `lag/lead`, use `rows between unbounded preceding and unbounded following` | | ntile(buckets) | ✅
Specify window like, (partition by x order by y rows between unbounded preceding and unrounded following). | From 13d5b336adb8021742f842bff457ee8b3267b743 Mon Sep 17 00:00:00 2001 From: lgbo-ustc Date: Fri, 19 Apr 2024 09:16:31 +0800 Subject: [PATCH 181/417] check window frame --- src/Processors/Transforms/WindowTransform.cpp | 91 +++++++++++-------- 1 file changed, 55 insertions(+), 36 deletions(-) diff --git a/src/Processors/Transforms/WindowTransform.cpp b/src/Processors/Transforms/WindowTransform.cpp index 4758d5ca7f4..45c9f4457b8 100644 --- a/src/Processors/Transforms/WindowTransform.cpp +++ b/src/Processors/Transforms/WindowTransform.cpp @@ -17,6 +17,9 @@ #include #include +#include +#include + #include @@ -71,6 +74,9 @@ public: size_t function_index) const = 0; virtual std::optional getDefaultFrame() const { return {}; } + + /// Is the frame type supported by this function. + virtual bool checkWindowFrameType(const WindowTransform * /*transform*/) const { return true; } }; // Compares ORDER BY column values at given rows to find the boundaries of frame: @@ -402,6 +408,19 @@ WindowTransform::WindowTransform(const Block & input_header_, } } } + + for (const auto & workspace : workspaces) + { + if (workspace.window_function_impl) + { + if (!workspace.window_function_impl->checkWindowFrameType(this)) + { + throw Exception(ErrorCodes::BAD_ARGUMENTS, "Unsupported window frame type for function '{}'", + workspace.aggregate_function->getName()); + } + } + + } } WindowTransform::~WindowTransform() @@ -2086,8 +2105,6 @@ namespace const WindowTransform * transform, size_t function_index, const DataTypes & argument_types); - - static void checkWindowFrameType(const WindowTransform * transform); }; } @@ -2107,6 +2124,29 @@ struct WindowFunctionNtile final : public StatefulWindowFunction } bool allocatesMemoryInArena() const override { return false; } + + bool checkWindowFrameType(const WindowTransform * transform) const override + { + if (transform->order_by_indices.empty()) + { + LOG_ERROR(getLogger("WindowFunctionNtile"), "Window frame for 'ntile' function must have ORDER BY clause"); + return false; + } + + // We must wait all for the partition end and get the total rows number in this + // partition. So before the end of this partition, there is no any block could be + // dropped out. + bool is_frame_supported = transform->window_description.frame.begin_type == WindowFrame::BoundaryType::Unbounded + && transform->window_description.frame.end_type == WindowFrame::BoundaryType::Unbounded; + if (!is_frame_supported) + { + LOG_ERROR( + getLogger("WindowFunctionNtile"), + "Window frame for function 'ntile' should be 'ROWS BETWEEN UNBOUNDED PRECEDING AND UNBOUNDED FOLLOWING'"); + return false; + } + return true; + } std::optional getDefaultFrame() const override { @@ -2134,7 +2174,6 @@ namespace { if (!buckets) [[unlikely]] { - checkWindowFrameType(transform); const auto & current_block = transform->blockAt(transform->current_row); const auto & workspace = transform->workspaces[function_index]; const auto & arg_col = *current_block.original_input_columns[workspace.argument_column_indices[0]]; @@ -2205,22 +2244,6 @@ namespace bucket_num += 1; } } - - void NtileState::checkWindowFrameType(const WindowTransform * transform) - { - if (transform->order_by_indices.empty()) - throw Exception(ErrorCodes::BAD_ARGUMENTS, "Window frame for 'ntile' function must have ORDER BY clause"); - - // We must wait all for the partition end and get the total rows number in this - // partition. So before the end of this partition, there is no any block could be - // dropped out. - bool is_frame_supported = transform->window_description.frame.begin_type == WindowFrame::BoundaryType::Unbounded - && transform->window_description.frame.end_type == WindowFrame::BoundaryType::Unbounded; - if (!is_frame_supported) - { - throw Exception(ErrorCodes::BAD_ARGUMENTS, "Window frame for function 'ntile' should be 'ROWS BETWEEN UNBOUNDED PRECEDING AND UNBOUNDED FOLLOWING'"); - } - } } namespace @@ -2249,11 +2272,22 @@ public: frame.end_type = WindowFrame::BoundaryType::Unbounded; return frame; } + + bool checkWindowFrameType(const WindowTransform * transform) const override + { + if (transform->window_description.frame.begin_type != WindowFrame::BoundaryType::Unbounded + || transform->window_description.frame.end_type != WindowFrame::BoundaryType::Unbounded) + { + LOG_ERROR(getLogger("WindowFunctionPercentRank"), + "Window frame for function 'percent_rank' should be 'ROWS BETWEEN UNBOUNDED PRECEDING AND UNBOUNDED FOLLOWING'"); + return false; + } + return true; + } + void windowInsertResultInto(const WindowTransform * transform, size_t function_index) const override { - checkFrameBoundType(transform); - auto & state = getWorkspaceState(transform, function_index); if (WindowFunctionHelpers::checkPartitionEnterFirstRow(transform)) { @@ -2315,21 +2349,6 @@ public: auto & to_column = *transform->blockAt(transform->current_row).output_columns[function_index]; assert_cast(to_column).getData().push_back(static_cast(transform->peer_group_start_row_number) - 1); } -private: - mutable bool has_check_frame_bound_type = false; - ALWAYS_INLINE void checkFrameBoundType(const WindowTransform * transform) const - { - if (has_check_frame_bound_type) - return; - if (transform->window_description.frame.begin_type != WindowFrame::BoundaryType::Unbounded - || transform->window_description.frame.end_type != WindowFrame::BoundaryType::Unbounded) - { - throw Exception( - ErrorCodes::BAD_ARGUMENTS, - "Window frame for function 'percent_rank' should be 'ROWS BETWEEN UNBOUNDED PRECEDING AND UNBOUNDED FOLLOWING'"); - } - has_check_frame_bound_type = true; - } }; // ClickHouse-specific variant of lag/lead that respects the window frame. From 04e7b11a6477ed8b554a12ca301d00ba01e0525d Mon Sep 17 00:00:00 2001 From: lgbo-ustc Date: Fri, 19 Apr 2024 10:33:44 +0800 Subject: [PATCH 182/417] fixed style --- src/Processors/Transforms/WindowTransform.cpp | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/src/Processors/Transforms/WindowTransform.cpp b/src/Processors/Transforms/WindowTransform.cpp index 45c9f4457b8..1cb447bb6d3 100644 --- a/src/Processors/Transforms/WindowTransform.cpp +++ b/src/Processors/Transforms/WindowTransform.cpp @@ -2124,7 +2124,7 @@ struct WindowFunctionNtile final : public StatefulWindowFunction } bool allocatesMemoryInArena() const override { return false; } - + bool checkWindowFrameType(const WindowTransform * transform) const override { if (transform->order_by_indices.empty()) @@ -2272,7 +2272,7 @@ public: frame.end_type = WindowFrame::BoundaryType::Unbounded; return frame; } - + bool checkWindowFrameType(const WindowTransform * transform) const override { if (transform->window_description.frame.begin_type != WindowFrame::BoundaryType::Unbounded From 7f706dd9d1e7bf0b982c9db86f73c1cc89a4a0a5 Mon Sep 17 00:00:00 2001 From: lgbo-ustc Date: Fri, 19 Apr 2024 14:40:58 +0800 Subject: [PATCH 183/417] fixed --- src/Processors/Transforms/WindowTransform.cpp | 11 +--------- .../01592_window_functions.reference | 22 +++++++++---------- .../0_stateless/01592_window_functions.sql | 8 +++---- 3 files changed, 16 insertions(+), 25 deletions(-) diff --git a/src/Processors/Transforms/WindowTransform.cpp b/src/Processors/Transforms/WindowTransform.cpp index 1cb447bb6d3..ce188ed47ae 100644 --- a/src/Processors/Transforms/WindowTransform.cpp +++ b/src/Processors/Transforms/WindowTransform.cpp @@ -2265,18 +2265,9 @@ public: bool allocatesMemoryInArena() const override { return false; } - std::optional getDefaultFrame() const override - { - WindowFrame frame; - frame.type = WindowFrame::FrameType::ROWS; - frame.end_type = WindowFrame::BoundaryType::Unbounded; - return frame; - } - bool checkWindowFrameType(const WindowTransform * transform) const override { - if (transform->window_description.frame.begin_type != WindowFrame::BoundaryType::Unbounded - || transform->window_description.frame.end_type != WindowFrame::BoundaryType::Unbounded) + if (transform->window_description.frame.begin_type != WindowFrame::BoundaryType::Unbounded) { LOG_ERROR(getLogger("WindowFunctionPercentRank"), "Window frame for function 'percent_rank' should be 'ROWS BETWEEN UNBOUNDED PRECEDING AND UNBOUNDED FOLLOWING'"); diff --git a/tests/queries/0_stateless/01592_window_functions.reference b/tests/queries/0_stateless/01592_window_functions.reference index 0995def71e2..f88360abcc1 100644 --- a/tests/queries/0_stateless/01592_window_functions.reference +++ b/tests/queries/0_stateless/01592_window_functions.reference @@ -80,14 +80,14 @@ Kindle Fire 150 Tablet 150 350 Samsung Galaxy Tab 200 Tablet 175 350 iPad 700 Tablet 350 350 ---- Q8 ---- -Lenovo Thinkpad Laptop 700 0 -Sony VAIO Laptop 700 0.3333333333333333 -Dell Vostro Laptop 800 0.6666666666666666 -HP Elite Laptop 1200 1 -Microsoft Lumia Smartphone 200 0 -HTC One Smartphone 400 0.3333333333333333 -Nexus Smartphone 500 0.6666666666666666 -iPhone Smartphone 900 1 -Kindle Fire Tablet 150 0 -Samsung Galaxy Tab Tablet 200 0.5 -iPad Tablet 700 1 +Lenovo Thinkpad Laptop 700 1 0 +Sony VAIO Laptop 700 1 0 +Dell Vostro Laptop 800 3 0.6666666666666666 +HP Elite Laptop 1200 4 1 +Microsoft Lumia Smartphone 200 1 0 +HTC One Smartphone 400 2 0.3333333333333333 +Nexus Smartphone 500 3 0.6666666666666666 +iPhone Smartphone 900 4 1 +Kindle Fire Tablet 150 1 0 +Samsung Galaxy Tab Tablet 200 2 0.5 +iPad Tablet 700 3 1 diff --git a/tests/queries/0_stateless/01592_window_functions.sql b/tests/queries/0_stateless/01592_window_functions.sql index e48e26b26d2..f4b868c36e4 100644 --- a/tests/queries/0_stateless/01592_window_functions.sql +++ b/tests/queries/0_stateless/01592_window_functions.sql @@ -109,15 +109,15 @@ FROM product_name, group_name, price, - percent_rank() OVER (PARTITION BY group_name ORDER BY price ASC) AS r + rank() OVER (PARTITION BY group_name ORDER BY price ASC) AS rank, + percent_rank() OVER (PARTITION BY group_name ORDER BY price ASC) AS percent FROM products INNER JOIN product_groups USING (group_id) ) AS t ORDER BY group_name ASC, - r ASC, - product_name ASC, - price ASC; + price ASC, + product_name ASC; drop table product_groups; drop table products; From 656a9a7260e3789b8fb671b788dbb3126d88ebe9 Mon Sep 17 00:00:00 2001 From: lgbo-ustc Date: Fri, 19 Apr 2024 14:42:24 +0800 Subject: [PATCH 184/417] update --- tests/queries/0_stateless/01592_window_functions.sql | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/tests/queries/0_stateless/01592_window_functions.sql b/tests/queries/0_stateless/01592_window_functions.sql index f4b868c36e4..2fc0e55bf02 100644 --- a/tests/queries/0_stateless/01592_window_functions.sql +++ b/tests/queries/0_stateless/01592_window_functions.sql @@ -109,14 +109,14 @@ FROM product_name, group_name, price, - rank() OVER (PARTITION BY group_name ORDER BY price ASC) AS rank, + rank() OVER (PARTITION BY group_name ORDER BY price ASC) AS rank, percent_rank() OVER (PARTITION BY group_name ORDER BY price ASC) AS percent FROM products INNER JOIN product_groups USING (group_id) ) AS t ORDER BY group_name ASC, - price ASC, + price ASC, product_name ASC; drop table product_groups; From 91d2e5c72b38bb607d5a75020d38230a6937f310 Mon Sep 17 00:00:00 2001 From: lgbo-ustc Date: Fri, 19 Apr 2024 14:48:42 +0800 Subject: [PATCH 185/417] more corver case --- tests/queries/0_stateless/01592_window_functions.reference | 1 + tests/queries/0_stateless/01592_window_functions.sql | 3 +++ 2 files changed, 4 insertions(+) diff --git a/tests/queries/0_stateless/01592_window_functions.reference b/tests/queries/0_stateless/01592_window_functions.reference index f88360abcc1..06ec67ee82d 100644 --- a/tests/queries/0_stateless/01592_window_functions.reference +++ b/tests/queries/0_stateless/01592_window_functions.reference @@ -91,3 +91,4 @@ iPhone Smartphone 900 4 1 Kindle Fire Tablet 150 1 0 Samsung Galaxy Tab Tablet 200 2 0.5 iPad Tablet 700 3 1 +Others Unknow 200 1 0 diff --git a/tests/queries/0_stateless/01592_window_functions.sql b/tests/queries/0_stateless/01592_window_functions.sql index 2fc0e55bf02..a660fcca7b2 100644 --- a/tests/queries/0_stateless/01592_window_functions.sql +++ b/tests/queries/0_stateless/01592_window_functions.sql @@ -102,6 +102,9 @@ FROM products INNER JOIN product_groups USING (group_id)) t order by group_name, product_name, price; select '---- Q8 ----'; +INSERT INTO product_groups VALUES (4, 'Unknow'); +INSERT INTO products (product_id,product_name, group_id,price) VALUES (12, 'Others', 4, 200); + SELECT * FROM ( From e52828abf91c2e407fbdf5371e5c794a31b86b1e Mon Sep 17 00:00:00 2001 From: lgbo-ustc Date: Mon, 13 May 2024 11:19:33 +0800 Subject: [PATCH 186/417] fixed typos --- src/Processors/Transforms/WindowTransform.cpp | 6 +++--- 1 file changed, 3 insertions(+), 3 deletions(-) diff --git a/src/Processors/Transforms/WindowTransform.cpp b/src/Processors/Transforms/WindowTransform.cpp index ce188ed47ae..ad592613da2 100644 --- a/src/Processors/Transforms/WindowTransform.cpp +++ b/src/Processors/Transforms/WindowTransform.cpp @@ -1644,14 +1644,14 @@ struct WindowFunctionHelpers auto current_row = transform->current_row; current_row.row++; - const auto & partitoin_end_row = transform->partition_end; + const auto & partition_end_row = transform->partition_end; /// If current_row == partitoin_end_row, return true. otherwise - if (current_row != partitoin_end_row) + if (current_row != partition_end_row) { if (current_row.row < transform->blockRowsNumber(current_row)) return false; /// Next row to current_row may belong to next block. - if (partitoin_end_row.block != current_row.block + 1 || partitoin_end_row.row) + if (partition_end_row.block != current_row.block + 1 || partition_end_row.row) return false; } return true; From fa234cadcbd287145ce93211dd239fdf034d9335 Mon Sep 17 00:00:00 2001 From: lgbo-ustc Date: Tue, 14 May 2024 11:47:50 +0800 Subject: [PATCH 187/417] update doc --- docs/en/sql-reference/window-functions/index.md | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/docs/en/sql-reference/window-functions/index.md b/docs/en/sql-reference/window-functions/index.md index 814a7ac4aca..16225d4b0e2 100644 --- a/docs/en/sql-reference/window-functions/index.md +++ b/docs/en/sql-reference/window-functions/index.md @@ -23,7 +23,7 @@ ClickHouse supports the standard grammar for defining windows and window functio | `GROUPS` frame | ❌ | | Calculating aggregate functions over a frame (`sum(value) over (order by time)`) | ✅ (All aggregate functions are supported) | | `rank()`, `dense_rank()`, `row_number()` | ✅ | -| `percent_rank()` | ✅ equal to `ifNull((rank() OVER(PARTITION BY x order by y) - 1) / nullif(count(1) OVER(PARTITION BY x) -1, 0), 0)`, but more efficient| +| `percent_rank()` | ✅ Efficiently computes the relative standing of a value within a partition in a dataset. This function effectively replaces the more verbose and computationally intensive manual SQL calculation expressed as `ifNull((rank() OVER(PARTITION BY x ORDER BY y) - 1) / nullif(count(1) OVER(PARTITION BY x) - 1, 0), 0)`| | `lag/lead(value, offset)` | ❌
You can use one of the following workarounds:
1) `any(value) over (.... rows between preceding and preceding)`, or `following` for `lead`
2) `lagInFrame/leadInFrame`, which are analogous, but respect the window frame. To get behavior identical to `lag/lead`, use `rows between unbounded preceding and unbounded following` | | ntile(buckets) | ✅
Specify window like, (partition by x order by y rows between unbounded preceding and unrounded following). | From 297b65dbbe1859bac7c237d644452b03e3e5849d Mon Sep 17 00:00:00 2001 From: lgbo-ustc Date: Wed, 15 May 2024 17:16:44 +0800 Subject: [PATCH 188/417] fixed --- src/Processors/Transforms/WindowTransform.cpp | 20 ++++++++++++++----- 1 file changed, 15 insertions(+), 5 deletions(-) diff --git a/src/Processors/Transforms/WindowTransform.cpp b/src/Processors/Transforms/WindowTransform.cpp index ad592613da2..517e202556b 100644 --- a/src/Processors/Transforms/WindowTransform.cpp +++ b/src/Processors/Transforms/WindowTransform.cpp @@ -2267,15 +2267,25 @@ public: bool checkWindowFrameType(const WindowTransform * transform) const override { - if (transform->window_description.frame.begin_type != WindowFrame::BoundaryType::Unbounded) - { - LOG_ERROR(getLogger("WindowFunctionPercentRank"), - "Window frame for function 'percent_rank' should be 'ROWS BETWEEN UNBOUNDED PRECEDING AND UNBOUNDED FOLLOWING'"); - return false; + if (transform->window_description.frame.type != WindowFrame::FrameType::RANGE + || transform->window_description.frame.begin_type != WindowFrame::BoundaryType::Unbounded) + { + LOG_ERROR( + getLogger("WindowFunctionPercentRank"), + "Window frame for function 'percent_rank' should be 'RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT'"); + return false; } return true; } + std::optional getDefaultFrame() const override + { + WindowFrame frame; + frame.type = WindowFrame::FrameType::RANGE; + frame.begin_type = WindowFrame::BoundaryType::Unbounded; + frame.end_type = WindowFrame::BoundaryType::Current; + return frame; + } void windowInsertResultInto(const WindowTransform * transform, size_t function_index) const override { From b6782d4b2d98e72002b691a3a421d689831fc1bf Mon Sep 17 00:00:00 2001 From: lgbo-ustc Date: Fri, 24 May 2024 09:39:20 +0800 Subject: [PATCH 189/417] update --- src/Processors/Transforms/WindowTransform.cpp | 28 ++++++++++--------- 1 file changed, 15 insertions(+), 13 deletions(-) diff --git a/src/Processors/Transforms/WindowTransform.cpp b/src/Processors/Transforms/WindowTransform.cpp index 517e202556b..729fef5c05d 100644 --- a/src/Processors/Transforms/WindowTransform.cpp +++ b/src/Processors/Transforms/WindowTransform.cpp @@ -1633,24 +1633,24 @@ struct WindowFunctionHelpers ALWAYS_INLINE static bool checkPartitionEnterLastRow(const WindowTransform * transform) { - /// when partition_ended is false, it means that we don't reach the last row in this partition. - /// But when partition_ended is true, it doesn't mean that we reach the last row in this partition. - /// partition_ended is true when - /// - the input has finished. or - /// - current block contains next partition's data. /// This is for fast check. if (!transform->partition_ended) return false; auto current_row = transform->current_row; + /// checkPartitionEnterLastRow is called on each row, also move on current_row.row here. current_row.row++; const auto & partition_end_row = transform->partition_end; - /// If current_row == partitoin_end_row, return true. otherwise + + /// The partition end is reached, when following is true + /// - current row is the partition end row, + /// - or current row is the last row of all input. if (current_row != partition_end_row) { + /// when current row is not the partition end row, we need to check whether it's the last + /// input row. if (current_row.row < transform->blockRowsNumber(current_row)) return false; - /// Next row to current_row may belong to next block. if (partition_end_row.block != current_row.block + 1 || partition_end_row.row) return false; } @@ -2268,7 +2268,8 @@ public: bool checkWindowFrameType(const WindowTransform * transform) const override { if (transform->window_description.frame.type != WindowFrame::FrameType::RANGE - || transform->window_description.frame.begin_type != WindowFrame::BoundaryType::Unbounded) + || transform->window_description.frame.begin_type != WindowFrame::BoundaryType::Unbounded + || transform->window_description.frame.end_type != WindowFrame::BoundaryType::Current) { LOG_ERROR( getLogger("WindowFunctionPercentRank"), @@ -2305,20 +2306,20 @@ public: } UInt64 remaining_rows = state.current_partition_rows; - Float64 percent_rank_denominator = state.current_partition_rows - 1; + Float64 percent_rank_denominator = remaining_rows == 1 ? 1 : remaining_rows - 1; - if (remaining_rows <= 1) - return; while (remaining_rows > 0) { auto block_rows_number = transform->blockRowsNumber(state.start_row); auto available_block_rows = block_rows_number - state.start_row.row; if (available_block_rows <= remaining_rows) { + /// This partition involves multiple blocks. Finish current block and move on to the + /// next block. auto & to_column = *transform->blockAt(state.start_row).output_columns[function_index]; auto & data = assert_cast(to_column).getData(); for (size_t i = state.start_row.row; i < block_rows_number; ++i) - data[i] = data[i] / percent_rank_denominator; + data[i] = (data[i] - 1) / percent_rank_denominator; state.start_row.block++; state.start_row.row = 0; @@ -2326,11 +2327,12 @@ public: } else { + /// The partition ends in current block.s auto & to_column = *transform->blockAt(state.start_row).output_columns[function_index]; auto & data = assert_cast(to_column).getData(); for (size_t i = state.start_row.row, n = state.start_row.row + remaining_rows; i < n; ++i) { - data[i] = data[i]/percent_rank_denominator; + data[i] = (data[i] - 1) / percent_rank_denominator; } state.start_row.row += remaining_rows; remaining_rows = 0; From 87978327d6bab9509c0aef945dfbf76b4437e300 Mon Sep 17 00:00:00 2001 From: lgbo-ustc Date: Mon, 27 May 2024 09:00:17 +0800 Subject: [PATCH 190/417] fixed --- src/Processors/Transforms/WindowTransform.cpp | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/src/Processors/Transforms/WindowTransform.cpp b/src/Processors/Transforms/WindowTransform.cpp index 729fef5c05d..a694fa43e46 100644 --- a/src/Processors/Transforms/WindowTransform.cpp +++ b/src/Processors/Transforms/WindowTransform.cpp @@ -2350,7 +2350,7 @@ public: inline void insertRankIntoColumn(const WindowTransform * transform, size_t function_index) const { auto & to_column = *transform->blockAt(transform->current_row).output_columns[function_index]; - assert_cast(to_column).getData().push_back(static_cast(transform->peer_group_start_row_number) - 1); + assert_cast(to_column).getData().push_back(static_cast(transform->peer_group_start_row_number)); } }; From 1c82488eca1f3e4a12c66cc8b4779fd441cb5102 Mon Sep 17 00:00:00 2001 From: Nikita Fomichev Date: Thu, 4 Jul 2024 10:59:00 +0200 Subject: [PATCH 191/417] 03167_base64_url_functions_sh.sh make test simpler --- .../03167_base64_url_functions_sh.sh | 47 +++++-------------- 1 file changed, 13 insertions(+), 34 deletions(-) diff --git a/tests/queries/0_stateless/03167_base64_url_functions_sh.sh b/tests/queries/0_stateless/03167_base64_url_functions_sh.sh index f4d5addf370..57060b8c525 100755 --- a/tests/queries/0_stateless/03167_base64_url_functions_sh.sh +++ b/tests/queries/0_stateless/03167_base64_url_functions_sh.sh @@ -132,51 +132,30 @@ base64URLDecode() { echo "$result" | tr '_-' '/+' | base64 -w0 -d } -test_compare_to_gold_encode() { +test() { local input="$1" - local encode=$(${CLICKHOUSE_CLIENT} --query="SELECT base64URLEncode('$input')") + local encode_ch=$(${CLICKHOUSE_CLIENT} --query="SELECT base64URLEncode('$input')") local encode_gold=$(base64URLEncode $input) - if [ "$encode" != "$encode_gold" ]; then - echo "Input: $input" - echo "Expected: $encode_gold" - echo "Got: $encode" - fi -} - -test_compare_to_gold_decode() { - local input="$1" - local encode_gold=$(base64URLEncode $input) - local decode=$(${CLICKHOUSE_CLIENT} --query="SELECT base64URLDecode('$encode_gold')") + local decode_ch=$(${CLICKHOUSE_CLIENT} --query="SELECT base64URLDecode('$encode_gold')") local decode_gold=$(base64URLDecode $encode_gold) - if [ "$decode" != "$decode_gold" ]; then + if [ "$encode_ch" != "$encode_gold" ]; then echo "Input: $input" - echo "Expected: $decode_gold" - echo "Got: $decode" + echo "Expected: $encode_gold" + echo "Got: $encode_ch" + fi + + if [ "$decode_ch" != "$input" ] || [ "$decode_ch" != "$decode_gold" ]; then + echo "Input: $input" + echo "Decode gold: $decode_gold" + echo "Got: $decode_ch" fi } -test_compare_to_self() { - local input="$1" - local decode=$(${CLICKHOUSE_CLIENT} --query="SELECT base64URLDecode(base64URLEncode('$input'))") - - if [ "$decode" != "$input" ]; then - echo "Input: $input" - echo "Got: $decode" - fi -} for url in "${urls[@]}"; do - test_compare_to_gold_encode "$url" -done - -for url in "${urls[@]}"; do - test_compare_to_gold_decode "$url" -done - -for url in "${urls[@]}"; do - test_compare_to_self "$url" + test "$url" done # special case for ' From 75828c6e817e0c2a2c68040a63a46a083fc56e7a Mon Sep 17 00:00:00 2001 From: alesapin Date: Thu, 4 Jul 2024 12:16:00 +0200 Subject: [PATCH 192/417] Try to disable sccache --- tests/ci/build_check.py | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/tests/ci/build_check.py b/tests/ci/build_check.py index 39f34ed9ccf..ac3ff9a0b5a 100644 --- a/tests/ci/build_check.py +++ b/tests/ci/build_check.py @@ -62,7 +62,7 @@ def get_packager_cmd( if build_config.tidy: cmd += " --clang-tidy" - cmd += " --cache=sccache" + cmd += " --cache=ccache" cmd += " --s3-rw-access" cmd += f" --s3-bucket={S3_BUILDS_BUCKET}" From 036d0c3fad88e7092af2c2e749ce7132db811d26 Mon Sep 17 00:00:00 2001 From: alesapin Date: Thu, 4 Jul 2024 14:08:08 +0200 Subject: [PATCH 193/417] Revert "Try to disable sccache" This reverts commit 75828c6e817e0c2a2c68040a63a46a083fc56e7a. --- tests/ci/build_check.py | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/tests/ci/build_check.py b/tests/ci/build_check.py index ac3ff9a0b5a..39f34ed9ccf 100644 --- a/tests/ci/build_check.py +++ b/tests/ci/build_check.py @@ -62,7 +62,7 @@ def get_packager_cmd( if build_config.tidy: cmd += " --clang-tidy" - cmd += " --cache=ccache" + cmd += " --cache=sccache" cmd += " --s3-rw-access" cmd += f" --s3-bucket={S3_BUILDS_BUCKET}" From 8040150de8d0a27ffa72cfa4eaf7563d419dca71 Mon Sep 17 00:00:00 2001 From: alesapin Date: Thu, 4 Jul 2024 14:22:30 +0200 Subject: [PATCH 194/417] Very dirty hack --- docker/test/fasttest/Dockerfile | 1 + 1 file changed, 1 insertion(+) diff --git a/docker/test/fasttest/Dockerfile b/docker/test/fasttest/Dockerfile index 5f92db5a3ee..d6cc17c4126 100644 --- a/docker/test/fasttest/Dockerfile +++ b/docker/test/fasttest/Dockerfile @@ -36,6 +36,7 @@ RUN ln -s /usr/bin/lld-${LLVM_VERSION} /usr/bin/ld.lld # FIXME: workaround for "The imported target "merge-fdata" references the file" error # https://salsa.debian.org/pkg-llvm-team/llvm-toolchain/-/commit/992e52c0b156a5ba9c6a8a54f8c4857ddd3d371d RUN sed -i '/_IMPORT_CHECK_FILES_FOR_\(mlir-\|llvm-bolt\|merge-fdata\|MLIR\)/ {s|^|#|}' /usr/lib/llvm-${LLVM_VERSION}/lib/cmake/llvm/LLVMExports-*.cmake +RUN cp -r /usr/lib/llvm-18/lib/clang/18/lib/x86_64-pc-linux-gnu /usr/lib/llvm-18/lib/clang/18/lib/linux ARG CCACHE_VERSION=4.6.1 RUN mkdir /tmp/ccache \ From 2c77371b8b6a5b1d397fadea17fe0ce97c2af106 Mon Sep 17 00:00:00 2001 From: alesapin Date: Thu, 4 Jul 2024 15:27:23 +0200 Subject: [PATCH 195/417] Better --- docker/test/fasttest/Dockerfile | 3 ++- 1 file changed, 2 insertions(+), 1 deletion(-) diff --git a/docker/test/fasttest/Dockerfile b/docker/test/fasttest/Dockerfile index d6cc17c4126..fa58261ee97 100644 --- a/docker/test/fasttest/Dockerfile +++ b/docker/test/fasttest/Dockerfile @@ -36,7 +36,8 @@ RUN ln -s /usr/bin/lld-${LLVM_VERSION} /usr/bin/ld.lld # FIXME: workaround for "The imported target "merge-fdata" references the file" error # https://salsa.debian.org/pkg-llvm-team/llvm-toolchain/-/commit/992e52c0b156a5ba9c6a8a54f8c4857ddd3d371d RUN sed -i '/_IMPORT_CHECK_FILES_FOR_\(mlir-\|llvm-bolt\|merge-fdata\|MLIR\)/ {s|^|#|}' /usr/lib/llvm-${LLVM_VERSION}/lib/cmake/llvm/LLVMExports-*.cmake -RUN cp -r /usr/lib/llvm-18/lib/clang/18/lib/x86_64-pc-linux-gnu /usr/lib/llvm-18/lib/clang/18/lib/linux +RUN ls /usr/lib/llvm-18/lib/clang/18/lib/aarch64-unknown-linux-gnu 2>/dev/null 1>/dev/null && ln -s /usr/lib/llvm-18/lib/clang/18/lib/aarch64-unknown-linux-gnu /usr/lib/llvm-18/lib/clang/18/lib/linux 2>/dev/null ||: +RUN ls /usr/lib/llvm-18/lib/clang/18/lib/x86_64-pc-linux-gnu 2>/dev/null 1>/dev/null && ln -s /usr/lib/llvm-18/lib/clang/18/lib/x86_64-pc-linux-gnu /usr/lib/llvm-18/lib/clang/18/lib/linux 2>/dev/null ||: ARG CCACHE_VERSION=4.6.1 RUN mkdir /tmp/ccache \ From 8d6c0147e258fd9bd939c6a52b51a422c43aaa15 Mon Sep 17 00:00:00 2001 From: alesapin Date: Thu, 4 Jul 2024 15:30:27 +0200 Subject: [PATCH 196/417] Add comment --- docker/test/fasttest/Dockerfile | 2 ++ 1 file changed, 2 insertions(+) diff --git a/docker/test/fasttest/Dockerfile b/docker/test/fasttest/Dockerfile index fa58261ee97..a818e01ccca 100644 --- a/docker/test/fasttest/Dockerfile +++ b/docker/test/fasttest/Dockerfile @@ -36,6 +36,8 @@ RUN ln -s /usr/bin/lld-${LLVM_VERSION} /usr/bin/ld.lld # FIXME: workaround for "The imported target "merge-fdata" references the file" error # https://salsa.debian.org/pkg-llvm-team/llvm-toolchain/-/commit/992e52c0b156a5ba9c6a8a54f8c4857ddd3d371d RUN sed -i '/_IMPORT_CHECK_FILES_FOR_\(mlir-\|llvm-bolt\|merge-fdata\|MLIR\)/ {s|^|#|}' /usr/lib/llvm-${LLVM_VERSION}/lib/cmake/llvm/LLVMExports-*.cmake +# LLVM changes paths for compiler-rt libraries. For some reason clang-18.1.8 cannot catch up libraries from default install path. +# It's very dirty workaround, better to build compiler and LLVM ourself and use it. RUN ls /usr/lib/llvm-18/lib/clang/18/lib/aarch64-unknown-linux-gnu 2>/dev/null 1>/dev/null && ln -s /usr/lib/llvm-18/lib/clang/18/lib/aarch64-unknown-linux-gnu /usr/lib/llvm-18/lib/clang/18/lib/linux 2>/dev/null ||: RUN ls /usr/lib/llvm-18/lib/clang/18/lib/x86_64-pc-linux-gnu 2>/dev/null 1>/dev/null && ln -s /usr/lib/llvm-18/lib/clang/18/lib/x86_64-pc-linux-gnu /usr/lib/llvm-18/lib/clang/18/lib/linux 2>/dev/null ||: From b4be9d5c6f64cb108e961180aa90453ac8efd5ff Mon Sep 17 00:00:00 2001 From: divanik Date: Thu, 4 Jul 2024 13:40:36 +0000 Subject: [PATCH 197/417] Fix address --- programs/disks/DisksApp.cpp | 1 + 1 file changed, 1 insertion(+) diff --git a/programs/disks/DisksApp.cpp b/programs/disks/DisksApp.cpp index 0898b692095..59ba45b9451 100644 --- a/programs/disks/DisksApp.cpp +++ b/programs/disks/DisksApp.cpp @@ -514,6 +514,7 @@ int DisksApp::main(const std::vector & /*args*/) DisksApp::~DisksApp() { + client.reset(nullptr); if (global_context) global_context->shutdown(); } From 7be481decf23d39afbb1fac9ced2d67d391a859c Mon Sep 17 00:00:00 2001 From: Sema Checherinda Date: Wed, 3 Jul 2024 18:01:14 +0200 Subject: [PATCH 198/417] init --- src/Interpreters/DatabaseCatalog.cpp | 220 +++++++++++++------- src/Interpreters/DatabaseCatalog.h | 11 +- src/Interpreters/InterpreterUndropQuery.cpp | 2 +- 3 files changed, 155 insertions(+), 78 deletions(-) diff --git a/src/Interpreters/DatabaseCatalog.cpp b/src/Interpreters/DatabaseCatalog.cpp index aaec94a4fb0..0a71a842452 100644 --- a/src/Interpreters/DatabaseCatalog.cpp +++ b/src/Interpreters/DatabaseCatalog.cpp @@ -1,5 +1,7 @@ +#include #include #include +#include #include #include #include @@ -26,6 +28,7 @@ #include #include #include +#include #include @@ -190,6 +193,7 @@ void DatabaseCatalog::initializeAndLoadTemporaryDatabase() unused_dir_rm_timeout_sec = getContext()->getConfigRef().getInt64("database_catalog_unused_dir_rm_timeout_sec", unused_dir_rm_timeout_sec); unused_dir_cleanup_period_sec = getContext()->getConfigRef().getInt64("database_catalog_unused_dir_cleanup_period_sec", unused_dir_cleanup_period_sec); drop_error_cooldown_sec = getContext()->getConfigRef().getInt64("database_catalog_drop_error_cooldown_sec", drop_error_cooldown_sec); + drop_table_concurrency = getContext()->getConfigRef().getInt64("database_catalog_drop_table_concurrency", drop_table_concurrency); auto db_for_temporary_and_external_tables = std::make_shared(TEMPORARY_DATABASE, getContext()); attachDatabase(TEMPORARY_DATABASE, db_for_temporary_and_external_tables); @@ -1141,7 +1145,7 @@ void DatabaseCatalog::enqueueDroppedTableCleanup(StorageID table_id, StoragePtr (*drop_task)->schedule(); } -void DatabaseCatalog::dequeueDroppedTableCleanup(StorageID table_id) +void DatabaseCatalog::undropTable(StorageID table_id) { String latest_metadata_dropped_path; TableMarkedAsDropped dropped_table; @@ -1216,91 +1220,155 @@ void DatabaseCatalog::dequeueDroppedTableCleanup(StorageID table_id) LOG_INFO(log, "Table {} was successfully undropped.", dropped_table.table_id.getNameForLogs()); } +std::tuple DatabaseCatalog::getDroppedTablesCountAndInuseCount() +{ + std::lock_guard lock(tables_marked_dropped_mutex); + + size_t in_use_count = 0; + for (const auto & item : tables_marked_dropped) + { + bool in_use = item.table && !item.table.unique(); + in_use_count += in_use; + } + return {tables_marked_dropped.size(), in_use_count}; +} + +time_t DatabaseCatalog::getMinDropTime() +{ + time_t min_drop_time = std::numeric_limits::max(); + for (const auto & item : tables_marked_dropped) + { + min_drop_time = std::min(min_drop_time, item.drop_time); + } + return min_drop_time; +} + +DatabaseCatalog::TablesMarkedAsDropped DatabaseCatalog::getTablesToDrop() +{ + time_t current_time = std::chrono::system_clock::to_time_t(std::chrono::system_clock::now()); + DatabaseCatalog::TablesMarkedAsDropped result; + + std::lock_guard lock(tables_marked_dropped_mutex); + + auto it = tables_marked_dropped.begin(); + while (it != tables_marked_dropped.end()) + { + bool in_use = it->table && !it->table.unique(); + bool old_enough = it->drop_time <= current_time; + LOG_DEBUG(log, "check {}: in_use {}, old_enough {}", it->table_id.getFullTableName(), in_use, old_enough); + + if (in_use || !old_enough) + { + ++it; + continue; + } + + if (it == first_async_drop_in_queue) + ++first_async_drop_in_queue; + + result.emplace_back(std::move(*it)); + it = tables_marked_dropped.erase(it); + } + + return result; +} + +void DatabaseCatalog::rescheduleDropTableTask() +{ + std::lock_guard lock(tables_marked_dropped_mutex); + + if (tables_marked_dropped.empty()) + return; + + if (first_async_drop_in_queue != tables_marked_dropped.begin()) + { + (*drop_task)->scheduleAfter(0); + return; + } + + time_t current_time = std::chrono::system_clock::to_time_t(std::chrono::system_clock::now()); + auto min_drop_time = getMinDropTime(); + time_t schedule_after = min_drop_time < current_time ? (min_drop_time - current_time) * 1000 : 0; + (*drop_task)->scheduleAfter(schedule_after); +} + +void DatabaseCatalog::dropTablesParallel(TablesMarkedAsDropped tables_to_drop) +{ + if (tables_to_drop.empty()) + return; + + SCOPE_EXIT({ + std::lock_guard lock(tables_marked_dropped_mutex); + if (first_async_drop_in_queue == tables_marked_dropped.end()) + first_async_drop_in_queue = tables_to_drop.begin(); + + tables_marked_dropped.splice(tables_marked_dropped.end(), tables_to_drop); + }); + + ThreadPool pool( + CurrentMetrics::DatabaseCatalogThreads, + CurrentMetrics::DatabaseCatalogThreadsActive, + CurrentMetrics::DatabaseCatalogThreadsScheduled, + /* max_threads */drop_table_concurrency, + /* max_free_threads */0, + /* queue_size */tables_to_drop.size()); + + while (!tables_to_drop.empty()) + { + auto front_table = std::move(tables_to_drop.front()); + tables_to_drop.pop_front(); + + pool.scheduleOrThrowOnError([&, table = front_table] () + { + try + { + dropTableFinally(table); + std::lock_guard lock(tables_marked_dropped_mutex); + [[maybe_unused]] auto removed = tables_marked_dropped_ids.erase(table.table_id.uuid); + chassert(removed); + wait_table_finally_dropped.notify_all(); + } + catch (...) + { + tryLogCurrentException(log, "Cannot drop table " + table.table_id.getNameForLogs() + + ". Will retry later."); + { + std::lock_guard lock(tables_marked_dropped_mutex); + tables_marked_dropped.emplace_back(table); + tables_marked_dropped.back().drop_time = std::chrono::system_clock::to_time_t(std::chrono::system_clock::now()) + drop_error_cooldown_sec; + if (first_async_drop_in_queue == tables_marked_dropped.end()) + --first_async_drop_in_queue; + } + } + }); + } + + pool.wait(); +} + void DatabaseCatalog::dropTableDataTask() { /// Background task that removes data of tables which were marked as dropped by Atomic databases. /// Table can be removed when it's not used by queries and drop_delay_sec elapsed since it was marked as dropped. - bool need_reschedule = true; - /// Default reschedule time for the case when we are waiting for reference count to become 1. - size_t schedule_after_ms = reschedule_time_ms; - TableMarkedAsDropped table; - try + auto [drop_tables_count, drop_tables_in_use_count] = getDroppedTablesCountAndInuseCount(); + + auto tables_to_drop = getTablesToDrop(); + + if (!tables_to_drop.empty()) { - std::lock_guard lock(tables_marked_dropped_mutex); - if (tables_marked_dropped.empty()) - return; - time_t current_time = std::chrono::system_clock::to_time_t(std::chrono::system_clock::now()); - time_t min_drop_time = std::numeric_limits::max(); - size_t tables_in_use_count = 0; - auto it = std::find_if(tables_marked_dropped.begin(), tables_marked_dropped.end(), [&](const auto & elem) - { - bool not_in_use = !elem.table || elem.table.unique(); - bool old_enough = elem.drop_time <= current_time; - min_drop_time = std::min(min_drop_time, elem.drop_time); - tables_in_use_count += !not_in_use; - return not_in_use && old_enough; - }); - if (it != tables_marked_dropped.end()) - { - table = std::move(*it); - LOG_INFO(log, "Have {} tables in drop queue ({} of them are in use), will try drop {}", - tables_marked_dropped.size(), tables_in_use_count, table.table_id.getNameForLogs()); - if (first_async_drop_in_queue == it) - ++first_async_drop_in_queue; - tables_marked_dropped.erase(it); - /// Schedule the task as soon as possible, while there are suitable tables to drop. - schedule_after_ms = 0; - } - else if (current_time < min_drop_time) - { - /// We are waiting for drop_delay_sec to exceed, no sense to wakeup until min_drop_time. - /// If new table is added to the queue with ignore_delay flag, schedule() is called to wakeup the task earlier. - schedule_after_ms = (min_drop_time - current_time) * 1000; - LOG_TRACE(log, "Not found any suitable tables to drop, still have {} tables in drop queue ({} of them are in use). " - "Will check again after {} seconds", tables_marked_dropped.size(), tables_in_use_count, min_drop_time - current_time); - } - need_reschedule = !tables_marked_dropped.empty(); + LOG_INFO(log, "Have {} tables in drop queue ({} of them are in use), will try drop {} tables", + drop_tables_count, drop_tables_in_use_count, tables_to_drop.size()); + + dropTablesParallel(tables_to_drop); } - catch (...) + else { - tryLogCurrentException(log, __PRETTY_FUNCTION__); + LOG_TRACE(log, "Not found any suitable tables to drop, still have {} tables in drop queue ({} of them are in use). " + "Will check again later", drop_tables_count, drop_tables_in_use_count); } - if (table.table_id) - { - try - { - dropTableFinally(table); - std::lock_guard lock(tables_marked_dropped_mutex); - [[maybe_unused]] auto removed = tables_marked_dropped_ids.erase(table.table_id.uuid); - assert(removed); - } - catch (...) - { - tryLogCurrentException(log, "Cannot drop table " + table.table_id.getNameForLogs() + - ". Will retry later."); - { - table.drop_time = std::chrono::system_clock::to_time_t(std::chrono::system_clock::now()) + drop_error_cooldown_sec; - std::lock_guard lock(tables_marked_dropped_mutex); - tables_marked_dropped.emplace_back(std::move(table)); - if (first_async_drop_in_queue == tables_marked_dropped.end()) - --first_async_drop_in_queue; - /// If list of dropped tables was empty, schedule a task to retry deletion. - if (tables_marked_dropped.size() == 1) - { - need_reschedule = true; - schedule_after_ms = drop_error_cooldown_sec * 1000; - } - } - } - - wait_table_finally_dropped.notify_all(); - } - - /// Do not schedule a task if there is no tables to drop - if (need_reschedule) - (*drop_task)->scheduleAfter(schedule_after_ms); + rescheduleDropTableTask(); } void DatabaseCatalog::dropTableFinally(const TableMarkedAsDropped & table) diff --git a/src/Interpreters/DatabaseCatalog.h b/src/Interpreters/DatabaseCatalog.h index 17d34e96245..f9b8233e85a 100644 --- a/src/Interpreters/DatabaseCatalog.h +++ b/src/Interpreters/DatabaseCatalog.h @@ -225,7 +225,7 @@ public: String getPathForDroppedMetadata(const StorageID & table_id) const; String getPathForMetadata(const StorageID & table_id) const; void enqueueDroppedTableCleanup(StorageID table_id, StoragePtr table, String dropped_metadata_path, bool ignore_delay = false); - void dequeueDroppedTableCleanup(StorageID table_id); + void undropTable(StorageID table_id); void waitTableFinallyDropped(const UUID & uuid); @@ -296,6 +296,12 @@ private: void dropTableDataTask(); void dropTableFinally(const TableMarkedAsDropped & table); + time_t getMinDropTime() TSA_REQUIRES(tables_marked_dropped_mutex); + std::tuple getDroppedTablesCountAndInuseCount(); + TablesMarkedAsDropped getTablesToDrop(); + void dropTablesParallel(TablesMarkedAsDropped tables); + void rescheduleDropTableTask(); + void cleanupStoreDirectoryTask(); bool maybeRemoveDirectory(const String & disk_name, const DiskPtr & disk, const String & unused_dir); @@ -363,6 +369,9 @@ private: static constexpr time_t default_drop_error_cooldown_sec = 5; time_t drop_error_cooldown_sec = default_drop_error_cooldown_sec; + static constexpr size_t default_drop_table_concurrency = 10; + size_t drop_table_concurrency = default_drop_table_concurrency; + std::unique_ptr reload_disks_task; std::mutex reload_disks_mutex; std::set disks_to_reload; diff --git a/src/Interpreters/InterpreterUndropQuery.cpp b/src/Interpreters/InterpreterUndropQuery.cpp index 920df3d6aed..8f935e951ef 100644 --- a/src/Interpreters/InterpreterUndropQuery.cpp +++ b/src/Interpreters/InterpreterUndropQuery.cpp @@ -64,7 +64,7 @@ BlockIO InterpreterUndropQuery::executeToTable(ASTUndropQuery & query) database->checkMetadataFilenameAvailability(table_id.table_name); - DatabaseCatalog::instance().dequeueDroppedTableCleanup(table_id); + DatabaseCatalog::instance().undropTable(table_id); return {}; } From 3776fafc881bf4725bfbb356e6e81df66ad336b6 Mon Sep 17 00:00:00 2001 From: Nikolai Kochetov Date: Thu, 4 Jul 2024 13:44:44 +0000 Subject: [PATCH 199/417] Print stacktrace in case of about after logical error. --- src/Common/Exception.cpp | 9 ++++++--- src/Common/StackTrace.cpp | 2 +- src/Common/StackTrace.h | 2 +- 3 files changed, 8 insertions(+), 5 deletions(-) diff --git a/src/Common/Exception.cpp b/src/Common/Exception.cpp index 1f4b0aea8f2..181b4f1488e 100644 --- a/src/Common/Exception.cpp +++ b/src/Common/Exception.cpp @@ -38,9 +38,12 @@ namespace ErrorCodes extern const int CANNOT_MREMAP; } -void abortOnFailedAssertion(const String & description) +void abortOnFailedAssertion(const String & description, const Exception::FramePointers * trace = nullptr) { - LOG_FATAL(&Poco::Logger::root(), "Logical error: '{}'.", description); + auto & logger = Poco::Logger::root(); + LOG_FATAL(&logger, "Logical error: '{}'.", description); + if (trace) + LOG_FATAL(&logger, "Stack trace (when copying this message, always include the lines below):\n\n{}", StackTrace::toString(trace->data(), 0, trace->size())); abort(); } @@ -58,7 +61,7 @@ void handle_error_code(const std::string & msg, int code, bool remote, const Exc #ifdef ABORT_ON_LOGICAL_ERROR if (code == ErrorCodes::LOGICAL_ERROR) { - abortOnFailedAssertion(msg); + abortOnFailedAssertion(msg, &trace); } #endif diff --git a/src/Common/StackTrace.cpp b/src/Common/StackTrace.cpp index 239e957bdfe..34f6f0b7535 100644 --- a/src/Common/StackTrace.cpp +++ b/src/Common/StackTrace.cpp @@ -545,7 +545,7 @@ std::string StackTrace::toString() const return toStringCached(frame_pointers, offset, size); } -std::string StackTrace::toString(void ** frame_pointers_raw, size_t offset, size_t size) +std::string StackTrace::toString(void * const * frame_pointers_raw, size_t offset, size_t size) { __msan_unpoison(frame_pointers_raw, size * sizeof(*frame_pointers_raw)); diff --git a/src/Common/StackTrace.h b/src/Common/StackTrace.h index 4ce9a9281f3..2078828f3d7 100644 --- a/src/Common/StackTrace.h +++ b/src/Common/StackTrace.h @@ -59,7 +59,7 @@ public: const FramePointers & getFramePointers() const { return frame_pointers; } std::string toString() const; - static std::string toString(void ** frame_pointers, size_t offset, size_t size); + static std::string toString(void * const * frame_pointers, size_t offset, size_t size); static void dropCache(); /// @param fatal - if true, will process inline frames (slower) From 1af3caf2b7eca87186de0aa73502f941ea913e14 Mon Sep 17 00:00:00 2001 From: alesapin Date: Thu, 4 Jul 2024 15:48:12 +0200 Subject: [PATCH 200/417] Update docker/test/fasttest/Dockerfile Co-authored-by: Azat Khuzhin --- docker/test/fasttest/Dockerfile | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/docker/test/fasttest/Dockerfile b/docker/test/fasttest/Dockerfile index a818e01ccca..46490276003 100644 --- a/docker/test/fasttest/Dockerfile +++ b/docker/test/fasttest/Dockerfile @@ -38,8 +38,8 @@ RUN ln -s /usr/bin/lld-${LLVM_VERSION} /usr/bin/ld.lld RUN sed -i '/_IMPORT_CHECK_FILES_FOR_\(mlir-\|llvm-bolt\|merge-fdata\|MLIR\)/ {s|^|#|}' /usr/lib/llvm-${LLVM_VERSION}/lib/cmake/llvm/LLVMExports-*.cmake # LLVM changes paths for compiler-rt libraries. For some reason clang-18.1.8 cannot catch up libraries from default install path. # It's very dirty workaround, better to build compiler and LLVM ourself and use it. -RUN ls /usr/lib/llvm-18/lib/clang/18/lib/aarch64-unknown-linux-gnu 2>/dev/null 1>/dev/null && ln -s /usr/lib/llvm-18/lib/clang/18/lib/aarch64-unknown-linux-gnu /usr/lib/llvm-18/lib/clang/18/lib/linux 2>/dev/null ||: -RUN ls /usr/lib/llvm-18/lib/clang/18/lib/x86_64-pc-linux-gnu 2>/dev/null 1>/dev/null && ln -s /usr/lib/llvm-18/lib/clang/18/lib/x86_64-pc-linux-gnu /usr/lib/llvm-18/lib/clang/18/lib/linux 2>/dev/null ||: +RUN ls /usr/lib/llvm-18/lib/clang/18/lib/aarch64-unknown-linux-gnu 2>/dev/null 1>/dev/null && ln -s /usr/lib/llvm-18/lib/clang/18/lib/aarch64-unknown-linux-gnu /usr/lib/llvm-18/lib/clang/18/lib/linux 2>/dev/null +RUN ls /usr/lib/llvm-18/lib/clang/18/lib/x86_64-pc-linux-gnu 2>/dev/null 1>/dev/null && ln -s /usr/lib/llvm-18/lib/clang/18/lib/x86_64-pc-linux-gnu /usr/lib/llvm-18/lib/clang/18/lib/linux 2>/dev/null ARG CCACHE_VERSION=4.6.1 RUN mkdir /tmp/ccache \ From 34697c0bfa85df5b0f07d6457f36e1b1eec4e680 Mon Sep 17 00:00:00 2001 From: Nikita Fomichev Date: Thu, 4 Jul 2024 15:55:03 +0200 Subject: [PATCH 201/417] Tests: increase timeout for flaky check --- tests/clickhouse-test | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/tests/clickhouse-test b/tests/clickhouse-test index 36870d59c3a..ef61d493f4b 100755 --- a/tests/clickhouse-test +++ b/tests/clickhouse-test @@ -69,7 +69,7 @@ TEST_FILE_EXTENSIONS = [".sql", ".sql.j2", ".sh", ".py", ".expect"] VERSION_PATTERN = r"^((\d+\.)?(\d+\.)?(\d+\.)?\d+)$" -TEST_MAX_RUN_TIME_IN_SECONDS = 120 +TEST_MAX_RUN_TIME_IN_SECONDS = 180 class SharedEngineReplacer: From 963f39ede89ecdfafd0731d0c5fed27dbad7bc11 Mon Sep 17 00:00:00 2001 From: alesapin Date: Thu, 4 Jul 2024 15:56:44 +0200 Subject: [PATCH 202/417] Revert "Update docker/test/fasttest/Dockerfile" This reverts commit 1af3caf2b7eca87186de0aa73502f941ea913e14. --- docker/test/fasttest/Dockerfile | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/docker/test/fasttest/Dockerfile b/docker/test/fasttest/Dockerfile index 46490276003..a818e01ccca 100644 --- a/docker/test/fasttest/Dockerfile +++ b/docker/test/fasttest/Dockerfile @@ -38,8 +38,8 @@ RUN ln -s /usr/bin/lld-${LLVM_VERSION} /usr/bin/ld.lld RUN sed -i '/_IMPORT_CHECK_FILES_FOR_\(mlir-\|llvm-bolt\|merge-fdata\|MLIR\)/ {s|^|#|}' /usr/lib/llvm-${LLVM_VERSION}/lib/cmake/llvm/LLVMExports-*.cmake # LLVM changes paths for compiler-rt libraries. For some reason clang-18.1.8 cannot catch up libraries from default install path. # It's very dirty workaround, better to build compiler and LLVM ourself and use it. -RUN ls /usr/lib/llvm-18/lib/clang/18/lib/aarch64-unknown-linux-gnu 2>/dev/null 1>/dev/null && ln -s /usr/lib/llvm-18/lib/clang/18/lib/aarch64-unknown-linux-gnu /usr/lib/llvm-18/lib/clang/18/lib/linux 2>/dev/null -RUN ls /usr/lib/llvm-18/lib/clang/18/lib/x86_64-pc-linux-gnu 2>/dev/null 1>/dev/null && ln -s /usr/lib/llvm-18/lib/clang/18/lib/x86_64-pc-linux-gnu /usr/lib/llvm-18/lib/clang/18/lib/linux 2>/dev/null +RUN ls /usr/lib/llvm-18/lib/clang/18/lib/aarch64-unknown-linux-gnu 2>/dev/null 1>/dev/null && ln -s /usr/lib/llvm-18/lib/clang/18/lib/aarch64-unknown-linux-gnu /usr/lib/llvm-18/lib/clang/18/lib/linux 2>/dev/null ||: +RUN ls /usr/lib/llvm-18/lib/clang/18/lib/x86_64-pc-linux-gnu 2>/dev/null 1>/dev/null && ln -s /usr/lib/llvm-18/lib/clang/18/lib/x86_64-pc-linux-gnu /usr/lib/llvm-18/lib/clang/18/lib/linux 2>/dev/null ||: ARG CCACHE_VERSION=4.6.1 RUN mkdir /tmp/ccache \ From f8ea14da5d46dc09d1e8c2df62e6b03258416628 Mon Sep 17 00:00:00 2001 From: Anton Popov Date: Thu, 4 Jul 2024 02:23:43 +0000 Subject: [PATCH 203/417] fix function 'has' with LowCardinality and FixedString --- src/Functions/array/arrayIndex.h | 445 +++++++----------- .../03199_has_lc_fixed_string.reference | 2 + .../0_stateless/03199_has_lc_fixed_string.sql | 7 + 3 files changed, 181 insertions(+), 273 deletions(-) create mode 100644 tests/queries/0_stateless/03199_has_lc_fixed_string.reference create mode 100644 tests/queries/0_stateless/03199_has_lc_fixed_string.sql diff --git a/src/Functions/array/arrayIndex.h b/src/Functions/array/arrayIndex.h index fa9b3dc92dd..067957f0d4b 100644 --- a/src/Functions/array/arrayIndex.h +++ b/src/Functions/array/arrayIndex.h @@ -28,6 +28,7 @@ namespace ErrorCodes { extern const int ILLEGAL_COLUMN; extern const int ILLEGAL_TYPE_OF_ARGUMENT; + extern const int LOGICAL_ERROR; } using NullMap = PaddedPODArray; @@ -424,31 +425,21 @@ public: ColumnPtr executeImpl(const ColumnsWithTypeAndName & arguments, const DataTypePtr & result_type, size_t /*input_rows_count*/) const override { - if constexpr (std::is_same_v) + if (auto res = executeMap(arguments, result_type)) + return res; + + if (auto res = executeArrayLowCardinality(arguments)) + return res; + + auto new_arguments = arguments; + + for (auto & argument : new_arguments) { - if (isMap(arguments[0].type)) - { - auto non_const_map_column = arguments[0].column->convertToFullColumnIfConst(); - - const auto & map_column = assert_cast(*non_const_map_column); - const auto & map_array_column = map_column.getNestedColumn(); - auto offsets = map_array_column.getOffsetsPtr(); - auto keys = map_column.getNestedData().getColumnPtr(0); - auto array_column = ColumnArray::create(keys, offsets); - - const auto & type_map = assert_cast(*arguments[0].type); - auto array_type = std::make_shared(type_map.getKeyType()); - - auto arguments_copy = arguments; - arguments_copy[0].column = std::move(array_column); - arguments_copy[0].type = std::move(array_type); - arguments_copy[0].name = arguments[0].name; - - return executeArrayImpl(arguments_copy, result_type); - } + argument.column = recursiveRemoveLowCardinality(argument.column); + argument.type = recursiveRemoveLowCardinality(argument.type); } - return executeArrayImpl(arguments, result_type); + return executeArrayImpl(new_arguments, result_type); } private: @@ -458,18 +449,6 @@ private: using NullMaps = std::pair; - struct ExecutionData - { - const IColumn& left; - const IColumn& right; - const ColumnArray::Offsets& offsets; - ColumnPtr result_column; - NullMaps maps; - ResultColumnPtr result { ResultColumnType::create() }; - - void moveResult() { result_column = std::move(result); } - }; - static bool allowArguments(const DataTypePtr & inner_type, const DataTypePtr & arg) { auto inner_type_decayed = removeNullable(removeLowCardinality(inner_type)); @@ -574,23 +553,13 @@ private: } } -#define INTEGRAL_TPL_PACK UInt8, UInt16, UInt32, UInt64, Int8, Int16, Int32, Int64, Float32, Float64 +#define INTEGRAL_PACK UInt8, UInt16, UInt32, UInt64, Int8, Int16, Int32, Int64, Float32, Float64 ColumnPtr executeOnNonNullable(const ColumnsWithTypeAndName & arguments, const DataTypePtr & result_type) const { - if (const auto* const left_arr = checkAndGetColumn(arguments[0].column.get())) - { - if (checkAndGetColumn(&left_arr->getData())) - { - if (auto res = executeLowCardinality(arguments)) - return res; - - throw Exception(ErrorCodes::ILLEGAL_COLUMN, "Illegal internal type of first argument of function {}", getName()); - } - } - ColumnPtr res; - if (!((res = executeIntegral(arguments)) + if (!((res = executeNothing(arguments)) + || (res = executeIntegral(arguments)) || (res = executeConst(arguments, result_type)) || (res = executeString(arguments)) || (res = executeGeneric(arguments)))) @@ -599,6 +568,8 @@ private: return res; } +#undef INTEGRAL_PACK + /** * The Array's internal data type may be quite tricky (containing a Nullable type somewhere). To process the * Nullable types correctly, for each data type specialisation we provide two null maps (one for the data and one @@ -627,6 +598,14 @@ private: return {null_map_data, null_map_item}; } + struct ExecutionData + { + const IColumn & left; + const IColumn & right; + const ColumnArray::Offsets & offsets; + NullMaps null_maps; + }; + /** * Given a variadic pack #Integral, apply executeIntegralExpanded with such parameters: * Integral s = {s1, s2, ...} @@ -635,39 +614,33 @@ private: template static ColumnPtr executeIntegral(const ColumnsWithTypeAndName & arguments) { - const ColumnArray * const left = checkAndGetColumn(arguments[0].column.get()); - - if (!left) + const auto * array = checkAndGetColumn(arguments[0].column.get()); + if (!array) return nullptr; - const ColumnPtr right_converted_ptr = arguments[1].column->convertToFullColumnIfLowCardinality(); - const IColumn& right = *right_converted_ptr.get(); - - ExecutionData data = { - left->getData(), - right, - left->getOffsets(), - nullptr, - getNullMaps(arguments) + ExecutionData data + { + .left = array->getData(), + .right = *arguments[1].column, + .offsets = array->getOffsets(), + .null_maps = getNullMaps(arguments), }; - if (executeIntegral(data)) - return data.result_column; - - return nullptr; + auto result = ResultColumnType::create(); + return executeIntegral(data, *result) ? std::move(result) : nullptr; } template - static bool executeIntegral(ExecutionData& data) + static bool executeIntegral(const ExecutionData & data, ResultColumnType & result) { - return (executeIntegralExpanded(data) || ...); + return (executeIntegralExpanded(data, result) || ...); } /// Invoke executeIntegralImpl with such parameters: (A, other1), (A, other2), ... template - static bool executeIntegralExpanded(ExecutionData& data) + static bool executeIntegralExpanded(const ExecutionData & data, ResultColumnType & result) { - return (executeIntegralImpl(data) || ...); + return (executeIntegralImpl(data, result) || ...); } /** @@ -676,40 +649,31 @@ private: * so we have to check all possible variants for #Initial and #Resulting types. */ template - static bool executeIntegralImpl(ExecutionData& data) + static bool executeIntegralImpl(const ExecutionData & data, ResultColumnType & result) { - const ColumnVector * col_nested = checkAndGetColumn>(&data.left); - - if (!col_nested) + const auto * left_typed = checkAndGetColumn>(&data.left); + if (!left_typed) return false; - const auto [null_map_data, null_map_item] = data.maps; - - if (data.right.onlyNull()) - Impl::Null::process( - data.offsets, - data.result->getData(), - null_map_data); - else if (const auto item_arg_const = checkAndGetColumnConst>(&data.right)) + if (const auto * item_arg_const = checkAndGetColumnConst>(&data.right)) Impl::Main::vector( - col_nested->getData(), + left_typed->getData(), data.offsets, item_arg_const->template getValue(), - data.result->getData(), - null_map_data, + result.getData(), + data.null_maps.first, nullptr); - else if (const auto item_arg_vector = checkAndGetColumn>(&data.right)) + else if (const auto * item_arg_vector = checkAndGetColumn>(&data.right)) Impl::Main::vector( - col_nested->getData(), + left_typed->getData(), data.offsets, item_arg_vector->getData(), - data.result->getData(), - null_map_data, - null_map_item); + result.getData(), + data.null_maps.first, + data.null_maps.second); else return false; - data.moveResult(); return true; } @@ -724,227 +688,153 @@ private: * * Tips and tricks tried can be found at https://github.com/ClickHouse/ClickHouse/pull/12550 . */ - static ColumnPtr executeLowCardinality(const ColumnsWithTypeAndName & arguments) + static ColumnPtr executeArrayLowCardinality(const ColumnsWithTypeAndName & arguments) { - const ColumnArray * const col_array = checkAndGetColumn(arguments[0].column.get()); + const auto * col_array = checkAndGetColumn(arguments[0].column.get()); + const auto * col_array_const = checkAndGetColumnConstData(arguments[0].column.get()); - if (!col_array) + if (!col_array && !col_array_const) return nullptr; - const ColumnLowCardinality * const col_lc = checkAndGetColumn(&col_array->getData()); + if (col_array_const) + col_array = col_array_const; - if (!col_lc) + const auto * left_lc = checkAndGetColumn(&col_array->getData()); + if (!left_lc) return nullptr; - const auto [null_map_data, null_map_item] = getNullMaps(arguments); + const auto * right_const = checkAndGetColumn(arguments[1].column.get()); + if (!right_const) + return nullptr; - if (const ColumnConst * col_arg_const = checkAndGetColumn(&*arguments[1].column)) + const auto & array_type = assert_cast(*arguments[0].type); + const auto target_type = recursiveRemoveLowCardinality(array_type.getNestedType()); + auto right = recursiveRemoveLowCardinality(right_const->getDataColumnPtr()); + + UInt64 index = 0; + ResultColumnPtr col_result = ResultColumnType::create(); + + if (!right->isNullAt(0)) { - const IColumnUnique & col_lc_dict = col_lc->getDictionary(); + auto right_type = recursiveRemoveLowCardinality(arguments[1].type); + right = castColumn({right, right_type, ""}, target_type); - const DataTypeArray * const array_type = checkAndGetDataType(arguments[0].type.get()); - const DataTypePtr target_type_ptr = recursiveRemoveLowCardinality(array_type->getNestedType()); + if (right->isNullable()) + right = checkAndGetColumn(*right).getNestedColumnPtr(); - ColumnPtr col_arg_cloned = castColumn( - {col_arg_const->getDataColumnPtr(), arguments[1].type, arguments[1].name}, target_type_ptr); + StringRef elem = right->getDataAt(0); + const auto & left_dict = left_lc->getDictionary(); - ResultColumnPtr col_result = ResultColumnType::create(); - UInt64 index = 0; - - if (!col_arg_cloned->isNullAt(0)) + if (std::optional maybe_index = left_dict.getOrFindValueIndex(elem); maybe_index) { - if (col_arg_cloned->isNullable()) - col_arg_cloned = checkAndGetColumn(*col_arg_cloned).getNestedColumnPtr(); - - StringRef elem = col_arg_cloned->getDataAt(0); - - if (std::optional maybe_index = col_lc_dict.getOrFindValueIndex(elem); maybe_index) - { - index = *maybe_index; - } - else - { - const size_t offsets_size = col_array->getOffsets().size(); - auto & data = col_result->getData(); - - data.resize_fill(offsets_size); - - return col_result; - } + index = *maybe_index; } - - Impl::Main::vector( - col_lc->getIndexes(), - col_array->getOffsets(), - index, /** Assuming LowCardinality has index of NULL always as zero. */ - col_result->getData(), - null_map_data, - null_map_item); - - return col_result; - } - else if (col_lc->nestedIsNullable()) // LowCardinality(Nullable(T)) and U - { - const ColumnPtr left_casted = col_lc->convertToFullColumnIfLowCardinality(); // Nullable(T) - const ColumnNullable & left_nullable = checkAndGetColumn(*left_casted); - - const NullMap * const null_map_left_casted = &left_nullable.getNullMapColumn().getData(); - - const IColumn & left_ptr = left_nullable.getNestedColumn(); - - const ColumnPtr right_casted = arguments[1].column->convertToFullColumnIfLowCardinality(); - const ColumnNullable * const right_nullable = checkAndGetColumn(right_casted.get()); - - const NullMap * const null_map_right_casted = right_nullable - ? &right_nullable->getNullMapColumn().getData() - : null_map_item; - - const IColumn& right_ptr = right_nullable - ? right_nullable->getNestedColumn() - : *right_casted.get(); - - ExecutionData data = + else { - left_ptr, right_ptr, - col_array->getOffsets(), - nullptr, - {null_map_left_casted, null_map_right_casted}}; - - if (dispatchConvertedLowCardinalityColumns(data)) - return data.result_column; + col_result->getData().resize_fill(col_array->size()); + return col_result; + } } - else // LowCardinality(T) and U, T not Nullable - { - if (arguments[1].column->isNullable()) - return nullptr; - - if (const auto* const arg_lc = checkAndGetColumn(arguments[1].column.get()); - arg_lc && arg_lc->isNullable()) - return nullptr; - - // LowCardinality(T) and U (possibly LowCardinality(V)) - - const ColumnPtr left_casted = col_lc->convertToFullColumnIfLowCardinality(); - const ColumnPtr right_casted = arguments[1].column->convertToFullColumnIfLowCardinality(); - - ExecutionData data = - { - *left_casted.get(), *right_casted.get(), col_array->getOffsets(), - nullptr, {null_map_data, null_map_item} - }; - - if (dispatchConvertedLowCardinalityColumns(data)) - return data.result_column; - } - - return nullptr; - } - - static bool dispatchConvertedLowCardinalityColumns(ExecutionData & data) - { - if (data.left.isNumeric() && data.right.isNumeric()) // ColumnArrays - return executeIntegral(data); - - if (checkAndGetColumn(&data.left)) - return executeStringImpl(data); Impl::Main::vector( - data.left, - data.offsets, data.right, - data.result->getData(), - data.maps.first, data.maps.second); + left_lc->getIndexes(), + col_array->getOffsets(), + index, /** Assuming LowCardinality has index of NULL always as zero. */ + col_result->getData(), + nullptr, + nullptr); - data.moveResult(); - return true; + return col_result; } -#undef INTEGRAL_TPL_PACK + ColumnPtr executeMap(const ColumnsWithTypeAndName & arguments, const DataTypePtr & result_type) const + { + if constexpr (!std::is_same_v) + return nullptr; + + if (!isMap(arguments[0].type)) + return nullptr; + + auto non_const_map_column = arguments[0].column->convertToFullColumnIfConst(); + + const auto & map_column = assert_cast(*non_const_map_column); + const auto & map_array_column = map_column.getNestedColumn(); + auto offsets = map_array_column.getOffsetsPtr(); + auto keys = map_column.getNestedData().getColumnPtr(0); + auto array_column = ColumnArray::create(keys, offsets); + + const auto & type_map = assert_cast(*arguments[0].type); + auto array_type = std::make_shared(type_map.getKeyType()); + + auto arguments_copy = arguments; + arguments_copy[0].column = std::move(array_column); + arguments_copy[0].type = std::move(array_type); + arguments_copy[0].name = arguments[0].name; + + return executeArrayImpl(arguments_copy, result_type); + } static ColumnPtr executeString(const ColumnsWithTypeAndName & arguments) { - const ColumnArray * array = checkAndGetColumn(arguments[0].column.get()); - + const auto * array = checkAndGetColumn(arguments[0].column.get()); if (!array) return nullptr; - const ColumnString * left = checkAndGetColumn(&array->getData()); - + const auto * left = checkAndGetColumn(&array->getData()); if (!left) return nullptr; - const ColumnPtr right_ptr = arguments[1].column->convertToFullColumnIfLowCardinality(); - const IColumn & right = *right_ptr.get(); + const auto & right = *arguments[1].column; + const auto [null_map_data, null_map_item] = getNullMaps(arguments); - ExecutionData data = { - *left, right, array->getOffsets(), - nullptr, getNullMaps(arguments), - std::move(ResultColumnType::create()) - }; + auto result = ResultColumnType::create(); - if (executeStringImpl(data)) - return data.result_column; - - return nullptr; - } - - static bool executeStringImpl(ExecutionData& data) - { - const auto [null_map_data, null_map_item] = data.maps; - const ColumnString& left = *typeid_cast(&data.left); - - if (data.right.onlyNull()) - Impl::Null::process( - data.offsets, - data.result->getData(), - null_map_data); - else if (const auto *const item_arg_const = checkAndGetColumnConstStringOrFixedString(&data.right)) + if (const auto * item_arg_const = checkAndGetColumnConstStringOrFixedString(&right)) { - const ColumnString * item_const_string = - checkAndGetColumn(&item_arg_const->getDataColumn()); - - const ColumnFixedString * item_const_fixedstring = - checkAndGetColumn(&item_arg_const->getDataColumn()); + const auto * item_const_string = checkAndGetColumn(&item_arg_const->getDataColumn()); + const auto * item_const_fixedstring = checkAndGetColumn(&item_arg_const->getDataColumn()); if (item_const_string) Impl::String::process( - left.getChars(), - data.offsets, - left.getOffsets(), + left->getChars(), + array->getOffsets(), + left->getOffsets(), item_const_string->getChars(), item_const_string->getDataAt(0).size, - data.result->getData(), + result->getData(), null_map_data, null_map_item); else if (item_const_fixedstring) Impl::String::process( - left.getChars(), - data.offsets, - left.getOffsets(), + left->getChars(), + array->getOffsets(), + left->getOffsets(), item_const_fixedstring->getChars(), item_const_fixedstring->getN(), - data.result->getData(), + result->getData(), null_map_data, null_map_item); else - throw Exception(ErrorCodes::ILLEGAL_COLUMN, "Logical error: ColumnConst contains not String nor FixedString column"); + throw Exception(ErrorCodes::LOGICAL_ERROR, "ColumnConst contains not String nor FixedString column"); } - else if (const auto *const item_arg_vector = checkAndGetColumn(&data.right)) + else if (const auto * item_arg_vector = checkAndGetColumn(&right)) { Impl::String::process( - left.getChars(), - data.offsets, - left.getOffsets(), + left->getChars(), + array->getOffsets(), + left->getOffsets(), item_arg_vector->getChars(), item_arg_vector->getOffsets(), - data.result->getData(), + result->getData(), null_map_data, null_map_item); } else - return false; + { + return nullptr; + } - data.moveResult(); - return true; + return result; } static ColumnPtr executeConst(const ColumnsWithTypeAndName & arguments, const DataTypePtr & result_type) @@ -955,9 +845,7 @@ private: return nullptr; Array arr = col_array->getValue(); - - const ColumnPtr right_ptr = arguments[1].column->convertToFullColumnIfLowCardinality(); - const IColumn * item_arg = right_ptr.get(); + const IColumn * item_arg = arguments[1].column.get(); if (isColumnConst(*item_arg)) { @@ -1026,48 +914,59 @@ private: } } + static ColumnPtr executeNothing(const ColumnsWithTypeAndName & arguments) + { + const auto * array = checkAndGetColumn(arguments[0].column.get()); + if (!array) + return nullptr; + + if (arguments[1].column->onlyNull()) + { + auto result = ResultColumnType::create(); + Impl::Null::process(array->getOffsets(), result->getData(), getNullMaps(arguments).first); + return result; + } + + return nullptr; + } + static ColumnPtr executeGeneric(const ColumnsWithTypeAndName & arguments) { - const ColumnArray * col = checkAndGetColumn(arguments[0].column.get()); - - if (!col) + const auto * col_array = checkAndGetColumn(arguments[0].column.get()); + if (!col_array) return nullptr; DataTypePtr array_elements_type = assert_cast(*arguments[0].type).getNestedType(); const DataTypePtr & index_type = arguments[1].type; - DataTypePtr common_type = getLeastSupertype(DataTypes{array_elements_type, index_type}); - - ColumnPtr col_nested = castColumn({ col->getDataPtr(), array_elements_type, "" }, common_type); - - const ColumnPtr right_ptr = arguments[1].column->convertToFullColumnIfLowCardinality(); - ColumnPtr item_arg = castColumn({ right_ptr, removeLowCardinality(index_type), "" }, common_type); + DataTypePtr common_type = getLeastSupertype(DataTypes{array_elements_type, arguments[1].type}); + ColumnPtr col_nested = castColumn({ col_array->getDataPtr(), array_elements_type, "" }, common_type); + ColumnPtr item_arg = castColumn({ arguments[1].column, removeLowCardinality(index_type), "" }, common_type); auto col_res = ResultColumnType::create(); auto [null_map_data, null_map_item] = getNullMaps(arguments); - if (item_arg->onlyNull()) - Impl::Null::process( - col->getOffsets(), - col_res->getData(), - null_map_data); - else if (isColumnConst(*item_arg)) + if (const auto * item_arg_const = checkAndGetColumn(item_arg.get())) + { Impl::Main::vector( *col_nested, - col->getOffsets(), - typeid_cast(*item_arg).getDataColumn(), + col_array->getOffsets(), + item_arg_const->getDataColumn(), col_res->getData(), /// TODO This is wrong. null_map_data, nullptr); + } else + { Impl::Main::vector( *col_nested, - col->getOffsets(), + col_array->getOffsets(), *item_arg, col_res->getData(), null_map_data, null_map_item); + } return col_res; } diff --git a/tests/queries/0_stateless/03199_has_lc_fixed_string.reference b/tests/queries/0_stateless/03199_has_lc_fixed_string.reference new file mode 100644 index 00000000000..b261da18d51 --- /dev/null +++ b/tests/queries/0_stateless/03199_has_lc_fixed_string.reference @@ -0,0 +1,2 @@ +1 +0 diff --git a/tests/queries/0_stateless/03199_has_lc_fixed_string.sql b/tests/queries/0_stateless/03199_has_lc_fixed_string.sql new file mode 100644 index 00000000000..3cb551804b7 --- /dev/null +++ b/tests/queries/0_stateless/03199_has_lc_fixed_string.sql @@ -0,0 +1,7 @@ +DROP TABLE IF EXISTS 03199_fixedstring_array; +CREATE TABLE 03199_fixedstring_array (arr Array(LowCardinality(FixedString(8)))) ENGINE = Memory; +INSERT INTO 03199_fixedstring_array VALUES (['a', 'b']), (['c', 'd']); + +SELECT has(arr, toFixedString(materialize('a'), 1)) FROM 03199_fixedstring_array; + +DROP TABLE 03199_fixedstring_array; From 93afc8e6133365007488c4d8340f434f6e8a876f Mon Sep 17 00:00:00 2001 From: Anton Popov Date: Thu, 4 Jul 2024 15:11:29 +0000 Subject: [PATCH 204/417] more precise warning message about sanitizers --- programs/server/Server.cpp | 34 +++++++++++++++++++++++++++++++++- 1 file changed, 33 insertions(+), 1 deletion(-) diff --git a/programs/server/Server.cpp b/programs/server/Server.cpp index f992fdc13a9..d51d959a42a 100644 --- a/programs/server/Server.cpp +++ b/programs/server/Server.cpp @@ -625,6 +625,28 @@ static void initializeAzureSDKLogger( #endif } +#if defined(SANITIZER) +static std::vector getSanitizerNames() +{ + std::vector names; + +#if defined(ADDRESS_SANITIZER) + names.push_back("address"); +#endif +#if defined(THREAD_SANITIZER) + names.push_back("thread"); +#endif +#if defined(MEMORY_SANITIZER) + names.push_back("memory"); +#endif +#if defined(UNDEFINED_BEHAVIOR_SANITIZER) + names.push_back("undefined behavior"); +#endif + + return names; +} +#endif + int Server::main(const std::vector & /*args*/) try { @@ -711,7 +733,17 @@ try global_context->addWarningMessage("ThreadFuzzer is enabled. Application will run slowly and unstable."); #if defined(SANITIZER) - global_context->addWarningMessage("Server was built with sanitizer. It will work slowly."); + auto sanitizers = getSanitizerNames(); + + String log_message; + if (sanitizers.empty()) + log_message = "sanitizer"; + else if (sanitizers.size() == 1) + log_message = fmt::format("{} sanitizer", sanitizers.front()); + else + log_message = fmt::format("sanitizers ({})", fmt::join(sanitizers, ", ")); + + global_context->addWarningMessage(fmt::format("Server was built with {}. It will work slowly.", log_message)); #endif #if defined(SANITIZE_COVERAGE) || WITH_COVERAGE From e176587c592370cb78801d364ba428e9935877eb Mon Sep 17 00:00:00 2001 From: alesapin Date: Thu, 4 Jul 2024 17:16:25 +0200 Subject: [PATCH 205/417] Trying even worse fix --- docker/test/fasttest/Dockerfile | 10 ++++++++-- 1 file changed, 8 insertions(+), 2 deletions(-) diff --git a/docker/test/fasttest/Dockerfile b/docker/test/fasttest/Dockerfile index a818e01ccca..dba31525b78 100644 --- a/docker/test/fasttest/Dockerfile +++ b/docker/test/fasttest/Dockerfile @@ -36,10 +36,16 @@ RUN ln -s /usr/bin/lld-${LLVM_VERSION} /usr/bin/ld.lld # FIXME: workaround for "The imported target "merge-fdata" references the file" error # https://salsa.debian.org/pkg-llvm-team/llvm-toolchain/-/commit/992e52c0b156a5ba9c6a8a54f8c4857ddd3d371d RUN sed -i '/_IMPORT_CHECK_FILES_FOR_\(mlir-\|llvm-bolt\|merge-fdata\|MLIR\)/ {s|^|#|}' /usr/lib/llvm-${LLVM_VERSION}/lib/cmake/llvm/LLVMExports-*.cmake + # LLVM changes paths for compiler-rt libraries. For some reason clang-18.1.8 cannot catch up libraries from default install path. # It's very dirty workaround, better to build compiler and LLVM ourself and use it. -RUN ls /usr/lib/llvm-18/lib/clang/18/lib/aarch64-unknown-linux-gnu 2>/dev/null 1>/dev/null && ln -s /usr/lib/llvm-18/lib/clang/18/lib/aarch64-unknown-linux-gnu /usr/lib/llvm-18/lib/clang/18/lib/linux 2>/dev/null ||: -RUN ls /usr/lib/llvm-18/lib/clang/18/lib/x86_64-pc-linux-gnu 2>/dev/null 1>/dev/null && ln -s /usr/lib/llvm-18/lib/clang/18/lib/x86_64-pc-linux-gnu /usr/lib/llvm-18/lib/clang/18/lib/linux 2>/dev/null ||: +RUN ls /usr/lib/llvm-18/lib/clang/18/lib/aarch64-unknown-linux-gnu 2>/dev/null 1>/dev/null \ + && ln -s /usr/lib/llvm-18/lib/clang/18/lib/aarch64-unknown-linux-gnu /usr/lib/llvm-18/lib/clang/18/lib/linux 2>/dev/null \ + && cd /usr/lib/llvm-18/lib/clang/18/lib/linux && rename 's/.a.syms$/-aarch64.a.syms/' *.a.syms && rename 's/.a$/-aarch64.a/' *.a && rename 's/.so$/-aarch64.so/' *.so && rename 's/.o$/-aarch64.o/' *.o ||: + +RUN ls /usr/lib/llvm-18/lib/clang/18/lib/x86_64-pc-linux-gnu 2>/dev/null 1>/dev/null \ + && ln -s /usr/lib/llvm-18/lib/clang/18/lib/x86_64-pc-linux-gnu /usr/lib/llvm-18/lib/clang/18/lib/linux 2>/dev/null \ + && cd /usr/lib/llvm-18/lib/clang/18/lib/linux && rename 's/.a.syms$/-x86_64.a.syms/' *.a.syms && rename 's/.a$/-x86_64.a/' *.a && rename 's/.so$/-x86_64.so/' *.so && rename 's/.o$/-x86_64.o/' *.o ||: ARG CCACHE_VERSION=4.6.1 RUN mkdir /tmp/ccache \ From c49d26bc239847f701b8d9942ed136b338b26024 Mon Sep 17 00:00:00 2001 From: Azat Khuzhin Date: Thu, 4 Jul 2024 17:19:24 +0200 Subject: [PATCH 206/417] Avoid using source directory for generated files Signed-off-by: Azat Khuzhin --- contrib/aws-cmake/CMakeLists.txt | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/contrib/aws-cmake/CMakeLists.txt b/contrib/aws-cmake/CMakeLists.txt index abde20addaf..250b47b7c2c 100644 --- a/contrib/aws-cmake/CMakeLists.txt +++ b/contrib/aws-cmake/CMakeLists.txt @@ -125,7 +125,7 @@ configure_file("${AWS_SDK_CORE_DIR}/include/aws/core/SDKConfig.h.in" "${CMAKE_CURRENT_BINARY_DIR}/include/aws/core/SDKConfig.h" @ONLY) aws_get_version(AWS_CRT_CPP_VERSION_MAJOR AWS_CRT_CPP_VERSION_MINOR AWS_CRT_CPP_VERSION_PATCH FULL_VERSION GIT_HASH) -configure_file("${AWS_CRT_DIR}/include/aws/crt/Config.h.in" "${AWS_CRT_DIR}/include/aws/crt/Config.h" @ONLY) +configure_file("${AWS_CRT_DIR}/include/aws/crt/Config.h.in" "${CMAKE_CURRENT_BINARY_DIR}/include/aws/crt/Config.h" @ONLY) list(APPEND AWS_SOURCES ${AWS_SDK_CORE_SRC} ${AWS_SDK_CORE_NET_SRC} ${AWS_SDK_CORE_PLATFORM_SRC}) From 14f54cb6e96066d90946a7e97ebd87b76160ab14 Mon Sep 17 00:00:00 2001 From: Anton Popov Date: Thu, 4 Jul 2024 15:44:16 +0000 Subject: [PATCH 207/417] slightly better calculation of primary index --- .../MergeTree/IMergeTreeDataPartWriter.cpp | 19 +++++- .../MergeTreeDataPartWriterOnDisk.cpp | 65 ++++++++++--------- .../MergeTree/MergeTreeDataPartWriterOnDisk.h | 9 +-- 3 files changed, 55 insertions(+), 38 deletions(-) diff --git a/src/Storages/MergeTree/IMergeTreeDataPartWriter.cpp b/src/Storages/MergeTree/IMergeTreeDataPartWriter.cpp index 6152da78395..c87f66b64f3 100644 --- a/src/Storages/MergeTree/IMergeTreeDataPartWriter.cpp +++ b/src/Storages/MergeTree/IMergeTreeDataPartWriter.cpp @@ -1,4 +1,5 @@ #include +#include namespace DB { @@ -71,9 +72,21 @@ IMergeTreeDataPartWriter::IMergeTreeDataPartWriter( Columns IMergeTreeDataPartWriter::releaseIndexColumns() { - return Columns( - std::make_move_iterator(index_columns.begin()), - std::make_move_iterator(index_columns.end())); + /// The memory for index was allocated without thread memory tracker. + /// We need to deallocate it in shrinkToFit without memory tracker as well. + MemoryTrackerBlockerInThread temporarily_disable_memory_tracker; + + Columns result; + result.reserve(index_columns.size()); + + for (auto & column : index_columns) + { + column->shrinkToFit(); + result.push_back(std::move(column)); + } + + index_columns.clear(); + return result; } SerializationPtr IMergeTreeDataPartWriter::getSerialization(const String & column_name) const diff --git a/src/Storages/MergeTree/MergeTreeDataPartWriterOnDisk.cpp b/src/Storages/MergeTree/MergeTreeDataPartWriterOnDisk.cpp index a576720294f..5c9191dbb54 100644 --- a/src/Storages/MergeTree/MergeTreeDataPartWriterOnDisk.cpp +++ b/src/Storages/MergeTree/MergeTreeDataPartWriterOnDisk.cpp @@ -254,6 +254,12 @@ void MergeTreeDataPartWriterOnDisk::initPrimaryIndex() index_compressor_stream = std::make_unique(*index_file_hashing_stream, primary_key_compression_codec, settings.primary_key_compress_block_size); index_source_hashing_stream = std::make_unique(*index_compressor_stream); } + + const auto & primary_key_types = metadata_snapshot->getPrimaryKey().data_types; + index_serializations.reserve(primary_key_types.size()); + + for (const auto & type : primary_key_types) + index_serializations.push_back(type->getDefaultSerialization()); } } @@ -299,22 +305,33 @@ void MergeTreeDataPartWriterOnDisk::initSkipIndices() store = std::make_shared(stream_name, data_part_storage, data_part_storage, storage_settings->max_digestion_size_per_segment); gin_index_stores[stream_name] = store; } + skip_indices_aggregators.push_back(skip_index->createIndexAggregatorForPart(store, settings)); skip_index_accumulated_marks.push_back(0); } } +void MergeTreeDataPartWriterOnDisk::calculateAndSerializePrimaryIndexRow(const Block & index_block, size_t row) +{ + chassert(index_block.columns() == index_serializations.size()); + auto & index_stream = compress_primary_key ? *index_source_hashing_stream : *index_file_hashing_stream; + + for (size_t i = 0; i < index_block.columns(); ++i) + { + const auto & column = index_block.getByPosition(i).column; + + index_columns[i]->insertFrom(*column, row); + index_serializations[i]->serializeBinary(*column, row, index_stream, {}); + } +} + void MergeTreeDataPartWriterOnDisk::calculateAndSerializePrimaryIndex(const Block & primary_index_block, const Granules & granules_to_write) { - size_t primary_columns_num = primary_index_block.columns(); + if (!metadata_snapshot->hasPrimaryKey()) + return; + if (index_columns.empty()) - { - index_types = primary_index_block.getDataTypes(); - index_columns.resize(primary_columns_num); - last_block_index_columns.resize(primary_columns_num); - for (size_t i = 0; i < primary_columns_num; ++i) - index_columns[i] = primary_index_block.getByPosition(i).column->cloneEmpty(); - } + index_columns = primary_index_block.cloneEmptyColumns(); { /** While filling index (index_columns), disable memory tracker. @@ -328,22 +345,14 @@ void MergeTreeDataPartWriterOnDisk::calculateAndSerializePrimaryIndex(const Bloc /// Write index. The index contains Primary Key value for each `index_granularity` row. for (const auto & granule : granules_to_write) { - if (metadata_snapshot->hasPrimaryKey() && granule.mark_on_start) - { - for (size_t j = 0; j < primary_columns_num; ++j) - { - const auto & primary_column = primary_index_block.getByPosition(j); - index_columns[j]->insertFrom(*primary_column.column, granule.start_row); - primary_column.type->getDefaultSerialization()->serializeBinary( - *primary_column.column, granule.start_row, compress_primary_key ? *index_source_hashing_stream : *index_file_hashing_stream, {}); - } - } + if (granule.mark_on_start) + calculateAndSerializePrimaryIndexRow(primary_index_block, granule.start_row); } } - /// store last index row to write final mark at the end of column - for (size_t j = 0; j < primary_columns_num; ++j) - last_block_index_columns[j] = primary_index_block.getByPosition(j).column; + /// Store block with last index row to write final mark at the end of column + if (with_final_mark) + last_index_block = primary_index_block; } void MergeTreeDataPartWriterOnDisk::calculateAndSerializeStatistics(const Block & block) @@ -420,17 +429,11 @@ void MergeTreeDataPartWriterOnDisk::fillPrimaryIndexChecksums(MergeTreeData::Dat if (index_file_hashing_stream) { - if (write_final_mark) + if (write_final_mark && last_index_block) { - for (size_t j = 0; j < index_columns.size(); ++j) - { - const auto & column = *last_block_index_columns[j]; - size_t last_row_number = column.size() - 1; - index_columns[j]->insertFrom(column, last_row_number); - index_types[j]->getDefaultSerialization()->serializeBinary( - column, last_row_number, compress_primary_key ? *index_source_hashing_stream : *index_file_hashing_stream, {}); - } - last_block_index_columns.clear(); + MemoryTrackerBlockerInThread temporarily_disable_memory_tracker; + calculateAndSerializePrimaryIndexRow(last_index_block, last_index_block.rows() - 1); + last_index_block.clear(); } if (compress_primary_key) diff --git a/src/Storages/MergeTree/MergeTreeDataPartWriterOnDisk.h b/src/Storages/MergeTree/MergeTreeDataPartWriterOnDisk.h index bdf0fdb7f32..8d84442981e 100644 --- a/src/Storages/MergeTree/MergeTreeDataPartWriterOnDisk.h +++ b/src/Storages/MergeTree/MergeTreeDataPartWriterOnDisk.h @@ -173,10 +173,10 @@ protected: std::unique_ptr index_source_hashing_stream; bool compress_primary_key; - DataTypes index_types; - /// Index columns from the last block - /// It's written to index file in the `writeSuffixAndFinalizePart` method - Columns last_block_index_columns; + /// Last block with index columns. + /// It's written to index file in the `writeSuffixAndFinalizePart` method. + Block last_index_block; + Serializations index_serializations; bool data_written = false; @@ -193,6 +193,7 @@ private: void initStatistics(); virtual void fillIndexGranularity(size_t index_granularity_for_block, size_t rows_in_block) = 0; + void calculateAndSerializePrimaryIndexRow(const Block & index_block, size_t row); struct ExecutionStatistics { From a55f39bc195bb462d10d479d004920777ba0fccc Mon Sep 17 00:00:00 2001 From: Anton Popov Date: Thu, 4 Jul 2024 15:55:08 +0000 Subject: [PATCH 208/417] add test for 58998 --- .../03200_subcolumns_join_use_nulls.reference | 1 + .../0_stateless/03200_subcolumns_join_use_nulls.sql | 13 +++++++++++++ 2 files changed, 14 insertions(+) create mode 100644 tests/queries/0_stateless/03200_subcolumns_join_use_nulls.reference create mode 100644 tests/queries/0_stateless/03200_subcolumns_join_use_nulls.sql diff --git a/tests/queries/0_stateless/03200_subcolumns_join_use_nulls.reference b/tests/queries/0_stateless/03200_subcolumns_join_use_nulls.reference new file mode 100644 index 00000000000..573541ac970 --- /dev/null +++ b/tests/queries/0_stateless/03200_subcolumns_join_use_nulls.reference @@ -0,0 +1 @@ +0 diff --git a/tests/queries/0_stateless/03200_subcolumns_join_use_nulls.sql b/tests/queries/0_stateless/03200_subcolumns_join_use_nulls.sql new file mode 100644 index 00000000000..2dd0a37657d --- /dev/null +++ b/tests/queries/0_stateless/03200_subcolumns_join_use_nulls.sql @@ -0,0 +1,13 @@ +DROP TABLE IF EXISTS t_subcolumns_join; + +CREATE TABLE t_subcolumns_join (id UInt64) ENGINE=MergeTree ORDER BY tuple(); + +INSERT INTO t_subcolumns_join SELECT number as number FROM numbers(10000); + +SELECT + count() +FROM (SELECT number FROM numbers(10)) as tbl LEFT JOIN t_subcolumns_join ON number = id +WHERE id is null +SETTINGS allow_experimental_analyzer = 1, optimize_functions_to_subcolumns = 1, join_use_nulls = 1; + +DROP TABLE t_subcolumns_join; From eb12d4e37e63212e831ccb92fc3c9818da7ba866 Mon Sep 17 00:00:00 2001 From: Blargian Date: Thu, 4 Jul 2024 18:02:10 +0200 Subject: [PATCH 209/417] update time-window-functions --- .../functions/time-window-functions.md | 198 ++++++++++++++---- 1 file changed, 162 insertions(+), 36 deletions(-) diff --git a/docs/en/sql-reference/functions/time-window-functions.md b/docs/en/sql-reference/functions/time-window-functions.md index 2cec1987c20..bad545fc5a5 100644 --- a/docs/en/sql-reference/functions/time-window-functions.md +++ b/docs/en/sql-reference/functions/time-window-functions.md @@ -6,44 +6,120 @@ sidebar_label: Time Window # Time Window Functions -Time window functions return the inclusive lower and exclusive upper bound of the corresponding window. The functions for working with WindowView are listed below: +Time window functions return the inclusive lower and exclusive upper bound of the corresponding window. The functions for working with [WindowView](../statements/create/view.md/#window-view-experimental) are listed below: ## tumble A tumbling time window assigns records to non-overlapping, continuous windows with a fixed duration (`interval`). +**Syntax** + ``` sql tumble(time_attr, interval [, timezone]) ``` **Arguments** -- `time_attr` - Date and time. [DateTime](../data-types/datetime.md) data type. -- `interval` - Window interval in [Interval](../data-types/special-data-types/interval.md) data type. +- `time_attr` — Date and time. [DateTime](../data-types/datetime.md). +- `interval` — Window interval in [Interval](../data-types/special-data-types/interval.md). - `timezone` — [Timezone name](../../operations/server-configuration-parameters/settings.md#server_configuration_parameters-timezone) (optional). **Returned values** -- The inclusive lower and exclusive upper bound of the corresponding tumbling window. [Tuple](../data-types/tuple.md)([DateTime](../data-types/datetime.md), [DateTime](../data-types/datetime.md))`. +- The inclusive lower and exclusive upper bound of the corresponding tumbling window. [Tuple](../data-types/tuple.md)([DateTime](../data-types/datetime.md), [DateTime](../data-types/datetime.md)). **Example** Query: ``` sql -SELECT tumble(now(), toIntervalDay('1')) +SELECT tumble(now(), toIntervalDay('1')); ``` Result: ``` text ┌─tumble(now(), toIntervalDay('1'))─────────────┐ -│ ['2020-01-01 00:00:00','2020-01-02 00:00:00'] │ +│ ('2024-07-04 00:00:00','2024-07-05 00:00:00') │ └───────────────────────────────────────────────┘ ``` +## tumbleStart + +Returns the inclusive lower bound of the corresponding [tumbling window](#tumble). + +**Syntax** + +``` sql +tumbleStart(bounds_tuple); +tumbleStart(time_attr, interval [, timezone]); +``` + +**Arguments** + +- `time_attr` — Date and time. [DateTime](../data-types/datetime.md). +- `interval` — Window interval in [Interval](../data-types/special-data-types/interval.md). +- `timezone` — [Timezone name](../../operations/server-configuration-parameters/settings.md#server_configuration_parameters-timezone) (optional). + +**Returned values** + +- The inclusive lower bound of the corresponding tumbling window. [DateTime](../data-types/datetime.md), [Tuple](../data-types/tuple.md) or [UInt32](../data-types/int-uint.md). + +**Example** + +Query: + +```sql +SELECT tumbleStart(now(), toIntervalDay('1')); +``` + +Result: + +```response +┌─tumbleStart(now(), toIntervalDay('1'))─┐ +│ 2024-07-04 00:00:00 │ +└────────────────────────────────────────┘ +``` + +## tumbleEnd + +Returns the exclusive upper bound of the corresponding [tumbling window](#tumble). + +**Syntax** + +``` sql +tumbleEnd(bounds_tuple); +tumbleEnd(time_attr, interval [, timezone]); +``` + +**Arguments** + +- `time_attr` — Date and time. [DateTime](../data-types/datetime.md). +- `interval` — Window interval in [Interval](../data-types/special-data-types/interval.md). +- `timezone` — [Timezone name](../../operations/server-configuration-parameters/settings.md#server_configuration_parameters-timezone) (optional). + +**Returned values** + +- The inclusive lower bound of the corresponding tumbling window. [DateTime](../data-types/datetime.md), [Tuple](../data-types/tuple.md) or [UInt32](../data-types/int-uint.md). + +**Example** + +Query: + +```sql +SELECT tumbleEnd(now(), toIntervalDay('1')); +``` + +Result: + +```response +┌─tumbleEnd(now(), toIntervalDay('1'))─┐ +│ 2024-07-05 00:00:00 │ +└──────────────────────────────────────┘ +``` + ## hop -A hopping time window has a fixed duration (`window_interval`) and hops by a specified hop interval (`hop_interval`). If the `hop_interval` is smaller than the `window_interval`, hopping windows are overlapping. Thus, records can be assigned to multiple windows. +A hopping time window has a fixed duration (`window_interval`) and hops by a specified hop interval (`hop_interval`). If the `hop_interval` is smaller than the `window_interval`, hopping windows are overlapping. Thus, records can be assigned to multiple windows. ``` sql hop(time_attr, hop_interval, window_interval [, timezone]) @@ -51,66 +127,116 @@ hop(time_attr, hop_interval, window_interval [, timezone]) **Arguments** -- `time_attr` - Date and time. [DateTime](../data-types/datetime.md) data type. -- `hop_interval` - Hop interval in [Interval](../data-types/special-data-types/interval.md) data type. Should be a positive number. -- `window_interval` - Window interval in [Interval](../data-types/special-data-types/interval.md) data type. Should be a positive number. -- `timezone` — [Timezone name](../../operations/server-configuration-parameters/settings.md#server_configuration_parameters-timezone) (optional). +- `time_attr` — Date and time. [DateTime](../data-types/datetime.md). +- `hop_interval` — Positive Hop interval. [Interval](../data-types/special-data-types/interval.md). +- `window_interval` — Positive Window interval. [Interval](../data-types/special-data-types/interval.md). +- `timezone` — [Timezone name](../../operations/server-configuration-parameters/settings.md#server_configuration_parameters-timezone) (optional). **Returned values** -- The inclusive lower and exclusive upper bound of the corresponding hopping window. Since one record can be assigned to multiple hop windows, the function only returns the bound of the **first** window when hop function is used **without** `WINDOW VIEW`. [Tuple](../data-types/tuple.md)([DateTime](../data-types/datetime.md), [DateTime](../data-types/datetime.md))`. +- The inclusive lower and exclusive upper bound of the corresponding hopping window. [Tuple](../data-types/tuple.md)([DateTime](../data-types/datetime.md), [DateTime](../data-types/datetime.md))`. + +:::note +Since one record can be assigned to multiple hop windows, the function only returns the bound of the **first** window when hop function is used **without** `WINDOW VIEW`. +::: **Example** Query: ``` sql -SELECT hop(now(), INTERVAL '1' SECOND, INTERVAL '2' SECOND) +SELECT hop(now(), INTERVAL '1' DAY, INTERVAL '2' DAY); ``` Result: ``` text -┌─hop(now(), toIntervalSecond('1'), toIntervalSecond('2'))──┐ -│ ('2020-01-14 16:58:22','2020-01-14 16:58:24') │ -└───────────────────────────────────────────────────────────┘ -``` - -## tumbleStart - -Returns the inclusive lower bound of the corresponding tumbling window. - -``` sql -tumbleStart(bounds_tuple); -tumbleStart(time_attr, interval [, timezone]); -``` - -## tumbleEnd - -Returns the exclusive upper bound of the corresponding tumbling window. - -``` sql -tumbleEnd(bounds_tuple); -tumbleEnd(time_attr, interval [, timezone]); +┌─hop(now(), toIntervalDay('1'), toIntervalDay('2'))─┐ +│ ('2024-07-03 00:00:00','2024-07-05 00:00:00') │ +└────────────────────────────────────────────────────┘ ``` ## hopStart -Returns the inclusive lower bound of the corresponding hopping window. +Returns the inclusive lower bound of the corresponding [hopping window](#hop). + +**Syntax** ``` sql hopStart(bounds_tuple); hopStart(time_attr, hop_interval, window_interval [, timezone]); ``` +**Arguments** + +- `time_attr` — Date and time. [DateTime](../data-types/datetime.md). +- `hop_interval` — Positive Hop interval. [Interval](../data-types/special-data-types/interval.md). +- `window_interval` — Positive Window interval. [Interval](../data-types/special-data-types/interval.md). +- `timezone` — [Timezone name](../../operations/server-configuration-parameters/settings.md#server_configuration_parameters-timezone) (optional). + +**Returned values** + +- The inclusive lower bound of the corresponding hopping window. [DateTime](../data-types/datetime.md), [Tuple](../data-types/tuple.md) or [UInt32](../data-types/int-uint.md). + +:::note +Since one record can be assigned to multiple hop windows, the function only returns the bound of the **first** window when hop function is used **without** `WINDOW VIEW`. +::: + +**Example** + +Query: + +``` sql +SELECT hopStart(now(), INTERVAL '1' DAY, INTERVAL '2' DAY); +``` + +Result: + +``` text +┌─hopStart(now(), toIntervalDay('1'), toIntervalDay('2'))─┐ +│ 2024-07-03 00:00:00 │ +└─────────────────────────────────────────────────────────┘ +``` ## hopEnd -Returns the exclusive upper bound of the corresponding hopping window. +Returns the exclusive upper bound of the corresponding [hopping window](#hop). + +**Syntax** ``` sql hopEnd(bounds_tuple); hopEnd(time_attr, hop_interval, window_interval [, timezone]); ``` +**Arguments** + +- `time_attr` — Date and time. [DateTime](../data-types/datetime.md). +- `hop_interval` — Positive Hop interval. [Interval](../data-types/special-data-types/interval.md). +- `window_interval` — Positive Window interval. [Interval](../data-types/special-data-types/interval.md). +- `timezone` — [Timezone name](../../operations/server-configuration-parameters/settings.md#server_configuration_parameters-timezone) (optional). + +**Returned values** + +- The exclusive upper bound of the corresponding hopping window. [DateTime](../data-types/datetime.md), [Tuple](../data-types/tuple.md) or [UInt32](../data-types/int-uint.md). + +:::note +Since one record can be assigned to multiple hop windows, the function only returns the bound of the **first** window when hop function is used **without** `WINDOW VIEW`. +::: + +**Example** + +Query: + +``` sql +SELECT hopEnd(now(), INTERVAL '1' DAY, INTERVAL '2' DAY); +``` + +Result: + +``` text +┌─hopEnd(now(), toIntervalDay('1'), toIntervalDay('2'))─┐ +│ 2024-07-05 00:00:00 │ +└───────────────────────────────────────────────────────┘ +``` ## Related content From c98b411edd34450c9954f8d086ae014fb80d1d8a Mon Sep 17 00:00:00 2001 From: Anton Popov Date: Thu, 4 Jul 2024 16:11:12 +0000 Subject: [PATCH 210/417] fix tests --- .../0_stateless/02993_lazy_index_loading.reference | 2 +- .../03127_system_unload_primary_key_table.reference | 8 ++++---- .../0_stateless/03128_system_unload_primary_key.reference | 4 ++-- 3 files changed, 7 insertions(+), 7 deletions(-) diff --git a/tests/queries/0_stateless/02993_lazy_index_loading.reference b/tests/queries/0_stateless/02993_lazy_index_loading.reference index 5bc329ae4eb..08f07a92815 100644 --- a/tests/queries/0_stateless/02993_lazy_index_loading.reference +++ b/tests/queries/0_stateless/02993_lazy_index_loading.reference @@ -1,4 +1,4 @@ -100000000 140000000 +100000000 100000000 0 0 1 100000000 100000000 diff --git a/tests/queries/0_stateless/03127_system_unload_primary_key_table.reference b/tests/queries/0_stateless/03127_system_unload_primary_key_table.reference index 3ac6127fb21..2d33f7f6683 100644 --- a/tests/queries/0_stateless/03127_system_unload_primary_key_table.reference +++ b/tests/queries/0_stateless/03127_system_unload_primary_key_table.reference @@ -1,8 +1,8 @@ -100000000 140000000 -100000000 140000000 -100000000 140000000 +100000000 100000000 +100000000 100000000 +100000000 100000000 0 0 -100000000 140000000 +100000000 100000000 0 0 0 0 1 diff --git a/tests/queries/0_stateless/03128_system_unload_primary_key.reference b/tests/queries/0_stateless/03128_system_unload_primary_key.reference index c7b40ae5b06..2646dc7247f 100644 --- a/tests/queries/0_stateless/03128_system_unload_primary_key.reference +++ b/tests/queries/0_stateless/03128_system_unload_primary_key.reference @@ -1,4 +1,4 @@ -100000000 140000000 -100000000 140000000 +100000000 100000000 +100000000 100000000 0 0 0 0 From 4271b2b6e3d6940603ef0d1836fbabf42b092d65 Mon Sep 17 00:00:00 2001 From: Nikolai Kochetov Date: Thu, 4 Jul 2024 16:29:32 +0000 Subject: [PATCH 211/417] Add noreturn/ --- src/Common/Exception.cpp | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/src/Common/Exception.cpp b/src/Common/Exception.cpp index 181b4f1488e..07bda6a75be 100644 --- a/src/Common/Exception.cpp +++ b/src/Common/Exception.cpp @@ -38,7 +38,7 @@ namespace ErrorCodes extern const int CANNOT_MREMAP; } -void abortOnFailedAssertion(const String & description, const Exception::FramePointers * trace = nullptr) +[[noreturn]] void abortOnFailedAssertion(const String & description, const Exception::FramePointers * trace = nullptr) { auto & logger = Poco::Logger::root(); LOG_FATAL(&logger, "Logical error: '{}'.", description); From 163f4c2199983f719aed3b859534b58548b57800 Mon Sep 17 00:00:00 2001 From: Kruglov Pavel <48961922+Avogar@users.noreply.github.com> Date: Thu, 4 Jul 2024 18:48:59 +0200 Subject: [PATCH 212/417] Better docs --- .../data-types/data-types-binary-encoding.md | 10 ++++++++++ 1 file changed, 10 insertions(+) diff --git a/docs/en/sql-reference/data-types/data-types-binary-encoding.md b/docs/en/sql-reference/data-types/data-types-binary-encoding.md index d7eddf848d6..c9720da0f1c 100644 --- a/docs/en/sql-reference/data-types/data-types-binary-encoding.md +++ b/docs/en/sql-reference/data-types/data-types-binary-encoding.md @@ -7,6 +7,11 @@ sidebar_label: Data types binary encoding specification. # Data types binary encoding specification +This specification describes the binary format that can be used for binary encoding and decoding of ClickHouse data types. This format is used in `Dynamic` column [binary serialization](dynamic.md#binary-output-format) and can be used in input/output formats [RowBinaryWithNamesAndTypes](../../interfaces/formats.md#rowbinarywithnamesandtypes) and [Native](../../interfaces/formats.md#native) under corresponding settings. + +The table below describes how each data type is represented in bunary format. Each data type encoding consist of 1 byte that indicates the type and some optional additional information. +`var_uint` in the binary encoding means that the size is encoded using Variable-Length Quantity compression. + | ClickHouse data type | Binary encoding | |--------------------------------------------------------------------------------------|------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------| | `Nothing` | `0x00` | @@ -61,6 +66,8 @@ sidebar_label: Data types binary encoding specification. ### Interval kind binary encoding +The table below describes how different interval kinds of `Interval` data type are endoced. + | Interval kind | Binary encoding | |---------------|-----------------| | `Nanosecond` | `0x00` | @@ -77,6 +84,9 @@ sidebar_label: Data types binary encoding specification. ### Aggregate function parameter binary encoding +The table below describes how parameters of `AggragateFunction` and `SimpleAggregateFunction` are encoded. +The encoding of a parameter consists of 1 byte indicating the type of the parameter and the value itself. + | Parameter type | Binary encoding | |--------------------------|--------------------------------------------------------------------------------------------------------------------------------| | `Null` | `0x00` | From e7c0ff54c936fb619f89011719f47bd4ba27064b Mon Sep 17 00:00:00 2001 From: Anton Popov Date: Thu, 4 Jul 2024 17:19:35 +0000 Subject: [PATCH 213/417] fix tests --- src/Functions/array/arrayIndex.h | 3 +++ .../queries/0_stateless/02010_array_index_bad_cast.reference | 3 +++ tests/queries/0_stateless/02010_array_index_bad_cast.sql | 5 +++-- 3 files changed, 9 insertions(+), 2 deletions(-) diff --git a/src/Functions/array/arrayIndex.h b/src/Functions/array/arrayIndex.h index 067957f0d4b..111ab92b006 100644 --- a/src/Functions/array/arrayIndex.h +++ b/src/Functions/array/arrayIndex.h @@ -744,6 +744,9 @@ private: nullptr, nullptr); + if (col_array_const) + return ColumnConst::create(std::move(col_result), arguments[0].column->size()); + return col_result; } diff --git a/tests/queries/0_stateless/02010_array_index_bad_cast.reference b/tests/queries/0_stateless/02010_array_index_bad_cast.reference index e69de29bb2d..e22493782f0 100644 --- a/tests/queries/0_stateless/02010_array_index_bad_cast.reference +++ b/tests/queries/0_stateless/02010_array_index_bad_cast.reference @@ -0,0 +1,3 @@ +1 +0 +0 diff --git a/tests/queries/0_stateless/02010_array_index_bad_cast.sql b/tests/queries/0_stateless/02010_array_index_bad_cast.sql index 14162e0d2e2..590e60eb42e 100644 --- a/tests/queries/0_stateless/02010_array_index_bad_cast.sql +++ b/tests/queries/0_stateless/02010_array_index_bad_cast.sql @@ -1,3 +1,4 @@ --- This query throws exception about uncomparable data types (but at least it does not introduce bad cast in code). SET allow_suspicious_low_cardinality_types=1; -SELECT has(materialize(CAST(['2021-07-14'] AS Array(LowCardinality(Nullable(DateTime))))), materialize('2021-07-14'::DateTime64(7))); -- { serverError ILLEGAL_COLUMN } +SELECT has(materialize(CAST(['2021-07-14'] AS Array(LowCardinality(Nullable(DateTime))))), materialize('2021-07-14'::DateTime64(7))); +SELECT has(materialize(CAST(['2021-07-14'] AS Array(LowCardinality(Nullable(DateTime))))), materialize('2021-07-14 00:00:01'::DateTime64(7))); +SELECT has(materialize(CAST(['2021-07-14'] AS Array(LowCardinality(Nullable(DateTime))))), materialize(NULL)); From ad23d211b9a1c1c8dbe0ecec73fe6277f930b6e2 Mon Sep 17 00:00:00 2001 From: Nikita Taranov Date: Thu, 4 Jul 2024 18:56:26 +0100 Subject: [PATCH 214/417] impl --- src/Common/BinStringDecodeHelper.h | 6 ++-- .../FunctionsBinaryRepresentation.cpp | 31 ++++++++++++----- src/Parsers/ExpressionElementParsers.cpp | 4 +-- .../03199_unbin_buffer_overflow.reference | 0 .../03199_unbin_buffer_overflow.sh | 33 +++++++++++++++++++ 5 files changed, 60 insertions(+), 14 deletions(-) create mode 100644 tests/queries/0_stateless/03199_unbin_buffer_overflow.reference create mode 100755 tests/queries/0_stateless/03199_unbin_buffer_overflow.sh diff --git a/src/Common/BinStringDecodeHelper.h b/src/Common/BinStringDecodeHelper.h index df3e014cfad..03c175fd37f 100644 --- a/src/Common/BinStringDecodeHelper.h +++ b/src/Common/BinStringDecodeHelper.h @@ -5,7 +5,7 @@ namespace DB { -static void inline hexStringDecode(const char * pos, const char * end, char *& out, size_t word_size = 2) +static void inline hexStringDecode(const char * pos, const char * end, char *& out, size_t word_size) { if ((end - pos) & 1) { @@ -23,7 +23,7 @@ static void inline hexStringDecode(const char * pos, const char * end, char *& o ++out; } -static void inline binStringDecode(const char * pos, const char * end, char *& out) +static void inline binStringDecode(const char * pos, const char * end, char *& out, size_t word_size) { if (pos == end) { @@ -53,7 +53,7 @@ static void inline binStringDecode(const char * pos, const char * end, char *& o ++out; } - assert((end - pos) % 8 == 0); + chassert((end - pos) % word_size == 0); while (end - pos != 0) { diff --git a/src/Functions/FunctionsBinaryRepresentation.cpp b/src/Functions/FunctionsBinaryRepresentation.cpp index 0f3f8be96a7..ab10d402df4 100644 --- a/src/Functions/FunctionsBinaryRepresentation.cpp +++ b/src/Functions/FunctionsBinaryRepresentation.cpp @@ -3,14 +3,14 @@ #include #include #include -#include -#include #include #include #include #include #include #include +#include +#include namespace DB { @@ -218,10 +218,7 @@ struct UnbinImpl static constexpr auto name = "unbin"; static constexpr size_t word_size = 8; - static void decode(const char * pos, const char * end, char *& out) - { - binStringDecode(pos, end, out); - } + static void decode(const char * pos, const char * end, char *& out) { binStringDecode(pos, end, out, word_size); } }; /// Encode number or string to string with binary or hexadecimal representation @@ -651,7 +648,15 @@ public: size_t size = in_offsets.size(); out_offsets.resize(size); - out_vec.resize(in_vec.size() / word_size + size); + + size_t max_out_len = 0; + for (size_t i = 0; i < in_offsets.size(); ++i) + { + const size_t len = in_offsets[i] - (i == 0 ? 0 : in_offsets[i - 1]) + - /* trailing zero symbol that is always added in ColumnString and that is ignored while decoding */ 1; + max_out_len += (len + word_size - 1) / word_size + /* trailing zero symbol that is always added by Impl::decode */ 1; + } + out_vec.resize(max_out_len); char * begin = reinterpret_cast(out_vec.data()); char * pos = begin; @@ -661,6 +666,7 @@ public: { size_t new_offset = in_offsets[i]; + /// `new_offset - 1` because in ColumnString each string is stored with trailing zero byte Impl::decode(reinterpret_cast(&in_vec[prev_offset]), reinterpret_cast(&in_vec[new_offset - 1]), pos); out_offsets[i] = pos - begin; @@ -668,6 +674,9 @@ public: prev_offset = new_offset; } + chassert( + static_cast(pos - begin) <= out_vec.size(), + fmt::format("too small amount of memory was preallocated: needed {}, but have only {}", pos - begin, out_vec.size())); out_vec.resize(pos - begin); return col_res; @@ -680,11 +689,11 @@ public: ColumnString::Offsets & out_offsets = col_res->getOffsets(); const ColumnString::Chars & in_vec = col_fix_string->getChars(); - size_t n = col_fix_string->getN(); + const size_t n = col_fix_string->getN(); size_t size = col_fix_string->size(); out_offsets.resize(size); - out_vec.resize(in_vec.size() / word_size + size); + out_vec.resize(((n + word_size - 1) / word_size + /* trailing zero symbol that is always added by Impl::decode */ 1) * size); char * begin = reinterpret_cast(out_vec.data()); char * pos = begin; @@ -694,6 +703,7 @@ public: { size_t new_offset = prev_offset + n; + /// here we don't subtract 1 from `new_offset` because in ColumnFixedString strings are stored without trailing zero byte Impl::decode(reinterpret_cast(&in_vec[prev_offset]), reinterpret_cast(&in_vec[new_offset]), pos); out_offsets[i] = pos - begin; @@ -701,6 +711,9 @@ public: prev_offset = new_offset; } + chassert( + static_cast(pos - begin) <= out_vec.size(), + fmt::format("too small amount of memory was preallocated: needed {}, but have only {}", pos - begin, out_vec.size())); out_vec.resize(pos - begin); return col_res; diff --git a/src/Parsers/ExpressionElementParsers.cpp b/src/Parsers/ExpressionElementParsers.cpp index 5997452bcf3..d4fc9a4bc4d 100644 --- a/src/Parsers/ExpressionElementParsers.cpp +++ b/src/Parsers/ExpressionElementParsers.cpp @@ -1129,11 +1129,11 @@ inline static bool makeHexOrBinStringLiteral(IParser::Pos & pos, ASTPtr & node, if (hex) { - hexStringDecode(str_begin, str_end, res_pos); + hexStringDecode(str_begin, str_end, res_pos, word_size); } else { - binStringDecode(str_begin, str_end, res_pos); + binStringDecode(str_begin, str_end, res_pos, word_size); } return makeStringLiteral(pos, node, String(reinterpret_cast(res.data()), (res_pos - res_begin - 1))); diff --git a/tests/queries/0_stateless/03199_unbin_buffer_overflow.reference b/tests/queries/0_stateless/03199_unbin_buffer_overflow.reference new file mode 100644 index 00000000000..e69de29bb2d diff --git a/tests/queries/0_stateless/03199_unbin_buffer_overflow.sh b/tests/queries/0_stateless/03199_unbin_buffer_overflow.sh new file mode 100755 index 00000000000..337debebb14 --- /dev/null +++ b/tests/queries/0_stateless/03199_unbin_buffer_overflow.sh @@ -0,0 +1,33 @@ +#!/usr/bin/env bash + +CURDIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd) +# shellcheck source=../shell_config.sh +. "$CURDIR"/../shell_config.sh + + +# check for buffer overflow in unbin (due to not enough memory preallocate for output buffer) +# we iterate over all remainders of input string length modulo word_size and check that no assertions are triggered + +word_size=8 +for i in $(seq 1 $((word_size+1))); do + str=$(printf "%${i}s" | tr ' ' 'x') + $CLICKHOUSE_CLIENT -q "SELECT count() FROM numbers(99) GROUP BY unbin(toFixedString(materialize('$str'), $i)) WITH ROLLUP WITH TOTALS FORMAT NULL" +done + +word_size=8 +for i in $(seq 1 $((word_size+1))); do + str=$(printf "%${i}s" | tr ' ' 'x') + $CLICKHOUSE_CLIENT -q "SELECT count() FROM numbers(99) GROUP BY unbin(materialize('$str')) WITH ROLLUP WITH TOTALS FORMAT NULL" +done + +word_size=2 +for i in $(seq 1 $((word_size+1))); do + str=$(printf "%${i}s" | tr ' ' 'x') + $CLICKHOUSE_CLIENT -q "SELECT count() FROM numbers(99) GROUP BY unhex(toFixedString(materialize('$str'), $i)) WITH ROLLUP WITH TOTALS FORMAT NULL" +done + +word_size=2 +for i in $(seq 1 $((word_size+1))); do + str=$(printf "%${i}s" | tr ' ' 'x') + $CLICKHOUSE_CLIENT -q "SELECT count() FROM numbers(99) GROUP BY unhex(materialize('$str')) WITH ROLLUP WITH TOTALS FORMAT NULL" +done From f556f2cd9529acfdf796df91c20abec0ce405a95 Mon Sep 17 00:00:00 2001 From: avogar Date: Thu, 4 Jul 2024 18:28:22 +0000 Subject: [PATCH 215/417] Try to fix special build --- src/Formats/JSONExtractTree.cpp | 1 + src/Functions/FunctionsJSON.cpp | 13 ++----------- 2 files changed, 3 insertions(+), 11 deletions(-) diff --git a/src/Formats/JSONExtractTree.cpp b/src/Formats/JSONExtractTree.cpp index 827f276311a..8fe472930d3 100644 --- a/src/Formats/JSONExtractTree.cpp +++ b/src/Formats/JSONExtractTree.cpp @@ -1558,6 +1558,7 @@ template std::unique_ptr> buildJSONExtractTr #if USE_RAPIDJSON template void jsonElementToString(const RapidJSONParser::Element & element, WriteBuffer & buf, const FormatSettings & format_settings); template std::unique_ptr> buildJSONExtractTree(const DataTypePtr & type, const char * source_for_exception_message); +template bool tryGetNumericValueFromJSONElement(Float64 & value, const RapidJSONParser::Element & element, bool convert_bool_to_integer, String & error); #else template void jsonElementToString(const DummyJSONParser::Element & element, WriteBuffer & buf, const FormatSettings & format_settings); template std::unique_ptr> buildJSONExtractTree(const DataTypePtr & type, const char * source_for_exception_message); diff --git a/src/Functions/FunctionsJSON.cpp b/src/Functions/FunctionsJSON.cpp index c6af0674db7..ca233becb63 100644 --- a/src/Functions/FunctionsJSON.cpp +++ b/src/Functions/FunctionsJSON.cpp @@ -736,17 +736,8 @@ public: NumberType value; tryGetNumericValueFromJSONElement(value, element, convert_bool_to_integer, error); - - if (dest.getDataType() == TypeIndex::LowCardinality) - { - ColumnLowCardinality & col_low = assert_cast(dest); - col_low.insertData(reinterpret_cast(&value), sizeof(value)); - } - else - { - auto & col_vec = assert_cast &>(dest); - col_vec.insertValue(value); - } + auto & col_vec = assert_cast &>(dest); + col_vec.insertValue(value); return true; } }; From b98c123f3a8516788ba9b93168d3ea718ea7c565 Mon Sep 17 00:00:00 2001 From: Michael Kolupaev Date: Thu, 4 Jul 2024 18:41:20 +0000 Subject: [PATCH 216/417] Remove logging, if this happens again we'll have to guess again --- src/IO/ReadWriteBufferFromHTTP.cpp | 4 ---- 1 file changed, 4 deletions(-) diff --git a/src/IO/ReadWriteBufferFromHTTP.cpp b/src/IO/ReadWriteBufferFromHTTP.cpp index f17c19f3cc2..b753e66da48 100644 --- a/src/IO/ReadWriteBufferFromHTTP.cpp +++ b/src/IO/ReadWriteBufferFromHTTP.cpp @@ -717,10 +717,6 @@ ReadWriteBufferFromHTTP::HTTPFileInfo ReadWriteBufferFromHTTP::getFileInfo() e.getHTTPStatus() != Poco::Net::HTTPResponse::HTTP_REQUEST_TIMEOUT && e.getHTTPStatus() != Poco::Net::HTTPResponse::HTTP_MISDIRECTED_REQUEST) { - LOG_DEBUG(log, - "HEAD request to '{}'{} failed with HTTP status {}", - initial_uri.toString(), current_uri == initial_uri ? String() : fmt::format(" redirect to '{}'", current_uri.toString()), - e.getHTTPStatus()); return HTTPFileInfo{}; } From 1d8389ddeefcc35c0fd04e3b0bb3cf99e55fdcbd Mon Sep 17 00:00:00 2001 From: Robert Schulze Date: Wed, 3 Jul 2024 19:27:25 +0000 Subject: [PATCH 217/417] Bump rocksdb to v6.23.3 --- contrib/rocksdb | 2 +- contrib/rocksdb-cmake/CMakeLists.txt | 4 ++-- 2 files changed, 3 insertions(+), 3 deletions(-) diff --git a/contrib/rocksdb b/contrib/rocksdb index 3a0b80ca9d6..078fa563869 160000 --- a/contrib/rocksdb +++ b/contrib/rocksdb @@ -1 +1 @@ -Subproject commit 3a0b80ca9d6eebb38fad7ea3f41dfc9db4f6a984 +Subproject commit 078fa5638690004e1f744076d1bdcc4e93767304 diff --git a/contrib/rocksdb-cmake/CMakeLists.txt b/contrib/rocksdb-cmake/CMakeLists.txt index c4220ba90ac..943e1d8acbd 100644 --- a/contrib/rocksdb-cmake/CMakeLists.txt +++ b/contrib/rocksdb-cmake/CMakeLists.txt @@ -1,7 +1,7 @@ -option (ENABLE_ROCKSDB "Enable rocksdb library" ${ENABLE_LIBRARIES}) +option (ENABLE_ROCKSDB "Enable RocksDB" ${ENABLE_LIBRARIES}) if (NOT ENABLE_ROCKSDB) - message (STATUS "Not using rocksdb") + message (STATUS "Not using RocksDB") return() endif() From 3fc4d8988bd24dfff65636c4febd0c18a3e41bfb Mon Sep 17 00:00:00 2001 From: Yakov Olkhovskiy Date: Thu, 4 Jul 2024 21:51:57 +0000 Subject: [PATCH 218/417] better test --- .../0_stateless/03174_projection_deduplicate.reference | 1 + tests/queries/0_stateless/03174_projection_deduplicate.sql | 6 ++++++ 2 files changed, 7 insertions(+) diff --git a/tests/queries/0_stateless/03174_projection_deduplicate.reference b/tests/queries/0_stateless/03174_projection_deduplicate.reference index 586b63a392e..1796b2f1dee 100644 --- a/tests/queries/0_stateless/03174_projection_deduplicate.reference +++ b/tests/queries/0_stateless/03174_projection_deduplicate.reference @@ -1,2 +1,3 @@ 1 one 1 one +1 one diff --git a/tests/queries/0_stateless/03174_projection_deduplicate.sql b/tests/queries/0_stateless/03174_projection_deduplicate.sql index ed6da5d73fc..46222b69dc7 100644 --- a/tests/queries/0_stateless/03174_projection_deduplicate.sql +++ b/tests/queries/0_stateless/03174_projection_deduplicate.sql @@ -21,4 +21,10 @@ OPTIMIZE TABLE test_projection_deduplicate DEDUPLICATE; -- { serverError NOT_IMP SELECT * FROM test_projection_deduplicate; +ALTER TABLE test_projection_deduplicate DROP PROJECTION test_projection; + +OPTIMIZE TABLE test_projection_deduplicate DEDUPLICATE; + +SELECT * FROM test_projection_deduplicate; + DROP TABLE test_projection_deduplicate; From dcdad686ee5413d0c9530f2a679ae5f42379527f Mon Sep 17 00:00:00 2001 From: kevinyhzou Date: Fri, 5 Jul 2024 10:25:00 +0800 Subject: [PATCH 219/417] fix stateless test --- .../queries/0_stateless/03198_orc_read_time_zone.sh | 4 ++-- ....snappy.orc => test_reader_time_zone.snappy.orc} | Bin 2 files changed, 2 insertions(+), 2 deletions(-) rename tests/queries/0_stateless/data_orc/{read_time_zone.snappy.orc => test_reader_time_zone.snappy.orc} (100%) diff --git a/tests/queries/0_stateless/03198_orc_read_time_zone.sh b/tests/queries/0_stateless/03198_orc_read_time_zone.sh index ba868e29dce..27530c06237 100755 --- a/tests/queries/0_stateless/03198_orc_read_time_zone.sh +++ b/tests/queries/0_stateless/03198_orc_read_time_zone.sh @@ -7,6 +7,6 @@ CURDIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd) $CLICKHOUSE_CLIENT -q "drop table if exists test" $CLICKHOUSE_CLIENT -q "create table test(id UInt64, t DateTime64) Engine=MergeTree order by id" -$CLICKHOUSE_CLIENT -q "insert into test from infile '$CURDIR/data_orc/read_time_zone.snappy.orc' SETTINGS input_format_orc_read_use_writer_time_zone=true FORMAT ORC" -$CLICKHOUSE_CLIENT -q "select * from test" +$CLICKHOUSE_CLIENT -q "insert into test from infile '$CURDIR/data_orc/test_reader_time_zone.snappy.orc' SETTINGS input_format_orc_read_use_writer_time_zone=true FORMAT ORC" +$CLICKHOUSE_CLIENT -q "select * from test SETTINGS session_timezone='Asia/Shanghai'" $CLICKHOUSE_CLIENT -q "drop table test" \ No newline at end of file diff --git a/tests/queries/0_stateless/data_orc/read_time_zone.snappy.orc b/tests/queries/0_stateless/data_orc/test_reader_time_zone.snappy.orc similarity index 100% rename from tests/queries/0_stateless/data_orc/read_time_zone.snappy.orc rename to tests/queries/0_stateless/data_orc/test_reader_time_zone.snappy.orc From 1f07f33bd259e2fc8b6ff930f3eb8dd72d255865 Mon Sep 17 00:00:00 2001 From: alesapin Date: Fri, 5 Jul 2024 12:18:18 +0200 Subject: [PATCH 220/417] Less dirty still hack --- docker/test/fasttest/Dockerfile | 10 ++-------- 1 file changed, 2 insertions(+), 8 deletions(-) diff --git a/docker/test/fasttest/Dockerfile b/docker/test/fasttest/Dockerfile index dba31525b78..cdbfc3f0beb 100644 --- a/docker/test/fasttest/Dockerfile +++ b/docker/test/fasttest/Dockerfile @@ -38,14 +38,8 @@ RUN ln -s /usr/bin/lld-${LLVM_VERSION} /usr/bin/ld.lld RUN sed -i '/_IMPORT_CHECK_FILES_FOR_\(mlir-\|llvm-bolt\|merge-fdata\|MLIR\)/ {s|^|#|}' /usr/lib/llvm-${LLVM_VERSION}/lib/cmake/llvm/LLVMExports-*.cmake # LLVM changes paths for compiler-rt libraries. For some reason clang-18.1.8 cannot catch up libraries from default install path. -# It's very dirty workaround, better to build compiler and LLVM ourself and use it. -RUN ls /usr/lib/llvm-18/lib/clang/18/lib/aarch64-unknown-linux-gnu 2>/dev/null 1>/dev/null \ - && ln -s /usr/lib/llvm-18/lib/clang/18/lib/aarch64-unknown-linux-gnu /usr/lib/llvm-18/lib/clang/18/lib/linux 2>/dev/null \ - && cd /usr/lib/llvm-18/lib/clang/18/lib/linux && rename 's/.a.syms$/-aarch64.a.syms/' *.a.syms && rename 's/.a$/-aarch64.a/' *.a && rename 's/.so$/-aarch64.so/' *.so && rename 's/.o$/-aarch64.o/' *.o ||: - -RUN ls /usr/lib/llvm-18/lib/clang/18/lib/x86_64-pc-linux-gnu 2>/dev/null 1>/dev/null \ - && ln -s /usr/lib/llvm-18/lib/clang/18/lib/x86_64-pc-linux-gnu /usr/lib/llvm-18/lib/clang/18/lib/linux 2>/dev/null \ - && cd /usr/lib/llvm-18/lib/clang/18/lib/linux && rename 's/.a.syms$/-x86_64.a.syms/' *.a.syms && rename 's/.a$/-x86_64.a/' *.a && rename 's/.so$/-x86_64.so/' *.so && rename 's/.o$/-x86_64.o/' *.o ||: +# It's very dirty workaround, better to build compiler and LLVM ourself and use it. Details: https://github.com/llvm/llvm-project/issues/95792 +RUN test ! -d /usr/lib/llvm-18/lib/clang/18/lib/x86_64-pc-linux-gnu || mv /usr/lib/llvm-18/lib/clang/18/lib/x86_64-pc-linux-gnu /usr/lib/llvm-18/lib/clang/18/lib/x86_64-unknown-linux-gnu ARG CCACHE_VERSION=4.6.1 RUN mkdir /tmp/ccache \ From fddbe5222732ffee676457dfe76f88c44791e043 Mon Sep 17 00:00:00 2001 From: Nikolai Kochetov Date: Fri, 5 Jul 2024 10:32:53 +0000 Subject: [PATCH 221/417] Disable merge filters optimization by default. --- src/Core/Settings.h | 1 + .../QueryPlan/Optimizations/Optimizations.h | 7 +++- .../QueryPlanOptimizationSettings.cpp | 2 ++ .../QueryPlanOptimizationSettings.h | 3 ++ .../Optimizations/mergeExpressions.cpp | 19 ++++++++-- .../01655_plan_optimizations.reference | 1 + ...01655_plan_optimizations_merge_filters.sql | 2 ++ .../02496_remove_redundant_sorting.reference | 13 +++---- ...rouping_sets_predicate_push_down.reference | 36 +++++++++++-------- 9 files changed, 61 insertions(+), 23 deletions(-) diff --git a/src/Core/Settings.h b/src/Core/Settings.h index f6d282792db..591cf2be4c9 100644 --- a/src/Core/Settings.h +++ b/src/Core/Settings.h @@ -758,6 +758,7 @@ class IColumn; M(Bool, query_plan_push_down_limit, true, "Allow to move LIMITs down in the query plan", 0) \ M(Bool, query_plan_split_filter, true, "Allow to split filters in the query plan", 0) \ M(Bool, query_plan_merge_expressions, true, "Allow to merge expressions in the query plan", 0) \ + M(Bool, query_plan_merge_filters, false, "Allow to merge filters in the query plan", 0) \ M(Bool, query_plan_filter_push_down, true, "Allow to push down filter by predicate query plan step", 0) \ M(Bool, query_plan_convert_outer_join_to_inner_join, true, "Allow to convert OUTER JOIN to INNER JOIN if filter after JOIN always filters default values", 0) \ M(Bool, query_plan_optimize_prewhere, true, "Allow to push down filter to PREWHERE expression for supported storages", 0) \ diff --git a/src/Processors/QueryPlan/Optimizations/Optimizations.h b/src/Processors/QueryPlan/Optimizations/Optimizations.h index b1ab5561958..c48bdf1552a 100644 --- a/src/Processors/QueryPlan/Optimizations/Optimizations.h +++ b/src/Processors/QueryPlan/Optimizations/Optimizations.h @@ -46,6 +46,10 @@ size_t trySplitFilter(QueryPlan::Node * node, QueryPlan::Nodes & nodes); /// Replace chain `FilterStep -> ExpressionStep` to single FilterStep size_t tryMergeExpressions(QueryPlan::Node * parent_node, QueryPlan::Nodes &); +/// Replace chain `FilterStep -> FilterStep` to single FilterStep +/// Note: this breaks short-circuit logic, so it is disabled for now. +size_t tryMergeFilters(QueryPlan::Node * parent_node, QueryPlan::Nodes &); + /// Move FilterStep down if possible. /// May split FilterStep and push down only part of it. size_t tryPushDownFilter(QueryPlan::Node * parent_node, QueryPlan::Nodes & nodes); @@ -81,11 +85,12 @@ size_t tryAggregatePartitionsIndependently(QueryPlan::Node * node, QueryPlan::No inline const auto & getOptimizations() { - static const std::array optimizations = {{ + static const std::array optimizations = {{ {tryLiftUpArrayJoin, "liftUpArrayJoin", &QueryPlanOptimizationSettings::lift_up_array_join}, {tryPushDownLimit, "pushDownLimit", &QueryPlanOptimizationSettings::push_down_limit}, {trySplitFilter, "splitFilter", &QueryPlanOptimizationSettings::split_filter}, {tryMergeExpressions, "mergeExpressions", &QueryPlanOptimizationSettings::merge_expressions}, + {tryMergeFilters, "mergeFilters", &QueryPlanOptimizationSettings::merge_filters}, {tryPushDownFilter, "pushDownFilter", &QueryPlanOptimizationSettings::filter_push_down}, {tryConvertOuterJoinToInnerJoin, "convertOuterJoinToInnerJoin", &QueryPlanOptimizationSettings::convert_outer_join_to_inner_join}, {tryExecuteFunctionsAfterSorting, "liftUpFunctions", &QueryPlanOptimizationSettings::execute_functions_after_sorting}, diff --git a/src/Processors/QueryPlan/Optimizations/QueryPlanOptimizationSettings.cpp b/src/Processors/QueryPlan/Optimizations/QueryPlanOptimizationSettings.cpp index 2738de1ff5f..4d984133efd 100644 --- a/src/Processors/QueryPlan/Optimizations/QueryPlanOptimizationSettings.cpp +++ b/src/Processors/QueryPlan/Optimizations/QueryPlanOptimizationSettings.cpp @@ -20,6 +20,8 @@ QueryPlanOptimizationSettings QueryPlanOptimizationSettings::fromSettings(const settings.merge_expressions = from.query_plan_enable_optimizations && from.query_plan_merge_expressions; + settings.merge_filters = from.query_plan_enable_optimizations && from.query_plan_merge_filters; + settings.filter_push_down = from.query_plan_enable_optimizations && from.query_plan_filter_push_down; settings.convert_outer_join_to_inner_join = from.query_plan_enable_optimizations && from.query_plan_convert_outer_join_to_inner_join; diff --git a/src/Processors/QueryPlan/Optimizations/QueryPlanOptimizationSettings.h b/src/Processors/QueryPlan/Optimizations/QueryPlanOptimizationSettings.h index 85042cea4ed..539ff2eafbb 100644 --- a/src/Processors/QueryPlan/Optimizations/QueryPlanOptimizationSettings.h +++ b/src/Processors/QueryPlan/Optimizations/QueryPlanOptimizationSettings.h @@ -31,6 +31,9 @@ struct QueryPlanOptimizationSettings /// If merge-expressions optimization is enabled. bool merge_expressions = true; + /// If merge-filters optimization is enabled. + bool merge_filters = false; + /// If filter push down optimization is enabled. bool filter_push_down = true; diff --git a/src/Processors/QueryPlan/Optimizations/mergeExpressions.cpp b/src/Processors/QueryPlan/Optimizations/mergeExpressions.cpp index 6ace1b3b5ce..118abdd701f 100644 --- a/src/Processors/QueryPlan/Optimizations/mergeExpressions.cpp +++ b/src/Processors/QueryPlan/Optimizations/mergeExpressions.cpp @@ -34,7 +34,6 @@ size_t tryMergeExpressions(QueryPlan::Node * parent_node, QueryPlan::Nodes &) auto * parent_expr = typeid_cast(parent.get()); auto * parent_filter = typeid_cast(parent.get()); auto * child_expr = typeid_cast(child.get()); - auto * child_filter = typeid_cast(child.get()); if (parent_expr && child_expr) { @@ -76,7 +75,23 @@ size_t tryMergeExpressions(QueryPlan::Node * parent_node, QueryPlan::Nodes &) parent_node->children.swap(child_node->children); return 1; } - else if (parent_filter && child_filter) + + return 0; +} +size_t tryMergeFilters(QueryPlan::Node * parent_node, QueryPlan::Nodes &) +{ + if (parent_node->children.size() != 1) + return false; + + QueryPlan::Node * child_node = parent_node->children.front(); + + auto & parent = parent_node->step; + auto & child = child_node->step; + + auto * parent_filter = typeid_cast(parent.get()); + auto * child_filter = typeid_cast(child.get()); + + if (parent_filter && child_filter) { const auto & child_actions = child_filter->getExpression(); const auto & parent_actions = parent_filter->getExpression(); diff --git a/tests/queries/0_stateless/01655_plan_optimizations.reference b/tests/queries/0_stateless/01655_plan_optimizations.reference index a6af1f2170d..edf93b4b39f 100644 --- a/tests/queries/0_stateless/01655_plan_optimizations.reference +++ b/tests/queries/0_stateless/01655_plan_optimizations.reference @@ -163,6 +163,7 @@ Filter column: notEquals(__table1.y, 2_UInt8) > filter is pushed down before CreatingSets CreatingSets Filter +Filter 1 3 > one condition of filter is pushed down before LEFT JOIN diff --git a/tests/queries/0_stateless/01655_plan_optimizations_merge_filters.sql b/tests/queries/0_stateless/01655_plan_optimizations_merge_filters.sql index 1301135b4cb..2193fc7a8f4 100644 --- a/tests/queries/0_stateless/01655_plan_optimizations_merge_filters.sql +++ b/tests/queries/0_stateless/01655_plan_optimizations_merge_filters.sql @@ -1,3 +1,5 @@ +set query_plan_merge_filters=1; + set allow_experimental_analyzer=1; select explain from (explain actions = 1 select * from (select sum(number) as v, bitAnd(number, 15) as key from numbers(1e8) group by key having v != 0) where key = 7) where explain like '%Filter%' or explain like '%Aggregating%'; diff --git a/tests/queries/0_stateless/02496_remove_redundant_sorting.reference b/tests/queries/0_stateless/02496_remove_redundant_sorting.reference index 4a4e898c5bd..77ef213b36d 100644 --- a/tests/queries/0_stateless/02496_remove_redundant_sorting.reference +++ b/tests/queries/0_stateless/02496_remove_redundant_sorting.reference @@ -332,12 +332,13 @@ SETTINGS optimize_aggregators_of_group_by_keys=0 -- avoid removing any() as it d Expression (Projection) Sorting (Sorting for ORDER BY) Expression (Before ORDER BY) - Filter (((WHERE + (Projection + Before ORDER BY)) + HAVING)) - Aggregating - Expression ((Before GROUP BY + Projection)) - Sorting (Sorting for ORDER BY) - Expression ((Before ORDER BY + (Projection + Before ORDER BY))) - ReadFromSystemNumbers + Filter ((WHERE + (Projection + Before ORDER BY))) + Filter (HAVING) + Aggregating + Expression ((Before GROUP BY + Projection)) + Sorting (Sorting for ORDER BY) + Expression ((Before ORDER BY + (Projection + Before ORDER BY))) + ReadFromSystemNumbers -- execute 1 2 diff --git a/tests/queries/0_stateless/02554_fix_grouping_sets_predicate_push_down.reference b/tests/queries/0_stateless/02554_fix_grouping_sets_predicate_push_down.reference index 70bcd7f255b..9bb0c022752 100644 --- a/tests/queries/0_stateless/02554_fix_grouping_sets_predicate_push_down.reference +++ b/tests/queries/0_stateless/02554_fix_grouping_sets_predicate_push_down.reference @@ -29,16 +29,20 @@ WHERE type_1 = \'all\' ExpressionTransform × 2 (Filter) FilterTransform × 2 - (Aggregating) - ExpressionTransform × 2 - AggregatingTransform × 2 - Copy 1 → 2 - (Expression) - ExpressionTransform - (Expression) - ExpressionTransform - (ReadFromMergeTree) - MergeTreeSelect(pool: ReadPoolInOrder, algorithm: InOrder) 0 → 1 + (Filter) + FilterTransform × 2 + (Filter) + FilterTransform × 2 + (Aggregating) + ExpressionTransform × 2 + AggregatingTransform × 2 + Copy 1 → 2 + (Expression) + ExpressionTransform + (Expression) + ExpressionTransform + (ReadFromMergeTree) + MergeTreeSelect(pool: ReadPoolInOrder, algorithm: InOrder) 0 → 1 (Expression) ExpressionTransform × 2 (Filter) @@ -64,10 +68,14 @@ ExpressionTransform × 2 ExpressionTransform × 2 AggregatingTransform × 2 Copy 1 → 2 - (Expression) - ExpressionTransform - (ReadFromMergeTree) - MergeTreeSelect(pool: ReadPoolInOrder, algorithm: InOrder) 0 → 1 + (Filter) + FilterTransform + (Filter) + FilterTransform + (Expression) + ExpressionTransform + (ReadFromMergeTree) + MergeTreeSelect(pool: ReadPoolInOrder, algorithm: InOrder) 0 → 1 (Expression) ExpressionTransform × 2 (Aggregating) From e084496d5f0d1dc2c60c84c5b9d1d9c2aa53fa21 Mon Sep 17 00:00:00 2001 From: Nikolai Kochetov Date: Fri, 5 Jul 2024 10:35:19 +0000 Subject: [PATCH 222/417] Add a test. --- .../03199_merge_filters_bug.reference | 0 .../0_stateless/03199_merge_filters_bug.sql | 70 +++++++++++++++++++ 2 files changed, 70 insertions(+) create mode 100644 tests/queries/0_stateless/03199_merge_filters_bug.reference create mode 100644 tests/queries/0_stateless/03199_merge_filters_bug.sql diff --git a/tests/queries/0_stateless/03199_merge_filters_bug.reference b/tests/queries/0_stateless/03199_merge_filters_bug.reference new file mode 100644 index 00000000000..e69de29bb2d diff --git a/tests/queries/0_stateless/03199_merge_filters_bug.sql b/tests/queries/0_stateless/03199_merge_filters_bug.sql new file mode 100644 index 00000000000..ed2ec2ea217 --- /dev/null +++ b/tests/queries/0_stateless/03199_merge_filters_bug.sql @@ -0,0 +1,70 @@ +drop table if exists t1; +drop table if exists t2; + +CREATE TABLE t1 +( + `s1` String, + `s2` String, + `s3` String +) +ENGINE = MergeTree +ORDER BY tuple(); + + +CREATE TABLE t2 +( + `fs1` FixedString(10), + `fs2` FixedString(10) +) +ENGINE = MergeTree +ORDER BY tuple(); + +INSERT INTO t1 SELECT + repeat('t', 15) s1, + 'test' s2, + 'test' s3; + +INSERT INTO t1 SELECT + substring(s1, 1, 10), + s2, + s3 +FROM generateRandom('s1 String, s2 String, s3 String') +LIMIT 10000; + +INSERT INTO t2 SELECT * +FROM generateRandom() +LIMIT 10000; + +WITH +tmp1 AS +( + SELECT + CAST(s1, 'FixedString(10)') AS fs1, + s2 AS sector, + s3 + FROM t1 + WHERE (s3 != 'test') +) + SELECT + fs1 + FROM t2 + LEFT JOIN tmp1 USING (fs1) + WHERE (fs1 IN ('test')) SETTINGS enable_multiple_prewhere_read_steps = 0; + +optimize table t1 final; + +WITH +tmp1 AS +( + SELECT + CAST(s1, 'FixedString(10)') AS fs1, + s2 AS sector, + s3 + FROM t1 + WHERE (s3 != 'test') +) + SELECT + fs1 + FROM t2 + LEFT JOIN tmp1 USING (fs1) + WHERE (fs1 IN ('test')); From b189902a365e6f86759e0b1c4b64b852d3aaf843 Mon Sep 17 00:00:00 2001 From: robot-clickhouse Date: Fri, 5 Jul 2024 10:50:22 +0000 Subject: [PATCH 223/417] Update version_date.tsv and changelogs after v24.6.2.17-stable --- docker/keeper/Dockerfile | 2 +- docker/server/Dockerfile.alpine | 2 +- docker/server/Dockerfile.ubuntu | 2 +- docs/changelogs/v24.6.2.17-stable.md | 26 ++++++++++++++++++++++++++ utils/list-versions/version_date.tsv | 2 ++ 5 files changed, 31 insertions(+), 3 deletions(-) create mode 100644 docs/changelogs/v24.6.2.17-stable.md diff --git a/docker/keeper/Dockerfile b/docker/keeper/Dockerfile index 018fe57bf56..c59ef1b919a 100644 --- a/docker/keeper/Dockerfile +++ b/docker/keeper/Dockerfile @@ -34,7 +34,7 @@ RUN arch=${TARGETARCH:-amd64} \ # lts / testing / prestable / etc ARG REPO_CHANNEL="stable" ARG REPOSITORY="https://packages.clickhouse.com/tgz/${REPO_CHANNEL}" -ARG VERSION="24.6.1.4423" +ARG VERSION="24.6.2.17" ARG PACKAGES="clickhouse-keeper" ARG DIRECT_DOWNLOAD_URLS="" diff --git a/docker/server/Dockerfile.alpine b/docker/server/Dockerfile.alpine index a86406e5129..240df79aeb1 100644 --- a/docker/server/Dockerfile.alpine +++ b/docker/server/Dockerfile.alpine @@ -32,7 +32,7 @@ RUN arch=${TARGETARCH:-amd64} \ # lts / testing / prestable / etc ARG REPO_CHANNEL="stable" ARG REPOSITORY="https://packages.clickhouse.com/tgz/${REPO_CHANNEL}" -ARG VERSION="24.6.1.4423" +ARG VERSION="24.6.2.17" ARG PACKAGES="clickhouse-client clickhouse-server clickhouse-common-static" ARG DIRECT_DOWNLOAD_URLS="" diff --git a/docker/server/Dockerfile.ubuntu b/docker/server/Dockerfile.ubuntu index 25f3273a648..ac64655991a 100644 --- a/docker/server/Dockerfile.ubuntu +++ b/docker/server/Dockerfile.ubuntu @@ -28,7 +28,7 @@ RUN sed -i "s|http://archive.ubuntu.com|${apt_archive}|g" /etc/apt/sources.list ARG REPO_CHANNEL="stable" ARG REPOSITORY="deb [signed-by=/usr/share/keyrings/clickhouse-keyring.gpg] https://packages.clickhouse.com/deb ${REPO_CHANNEL} main" -ARG VERSION="24.6.1.4423" +ARG VERSION="24.6.2.17" ARG PACKAGES="clickhouse-client clickhouse-server clickhouse-common-static" #docker-official-library:off diff --git a/docs/changelogs/v24.6.2.17-stable.md b/docs/changelogs/v24.6.2.17-stable.md new file mode 100644 index 00000000000..820937f6291 --- /dev/null +++ b/docs/changelogs/v24.6.2.17-stable.md @@ -0,0 +1,26 @@ +--- +sidebar_position: 1 +sidebar_label: 2024 +--- + +# 2024 Changelog + +### ClickHouse release v24.6.2.17-stable (5710a8b5c0c) FIXME as compared to v24.6.1.4423-stable (dcced7c8478) + +#### New Feature +* Backported in [#66002](https://github.com/ClickHouse/ClickHouse/issues/66002): Add AzureQueue storage. [#65458](https://github.com/ClickHouse/ClickHouse/pull/65458) ([Kseniia Sumarokova](https://github.com/kssenii)). + +#### Improvement +* Backported in [#65898](https://github.com/ClickHouse/ClickHouse/issues/65898): Respect cgroup CPU limit in Keeper. [#65819](https://github.com/ClickHouse/ClickHouse/pull/65819) ([Antonio Andelic](https://github.com/antonio2368)). + +#### Bug Fix (user-visible misbehavior in an official stable release) +* Backported in [#65935](https://github.com/ClickHouse/ClickHouse/issues/65935): For queries that read from `PostgreSQL`, cancel the internal `PostgreSQL` query if the ClickHouse query is finished. Otherwise, `ClickHouse` query cannot be canceled until the internal `PostgreSQL` query is finished. [#65771](https://github.com/ClickHouse/ClickHouse/pull/65771) ([Maksim Kita](https://github.com/kitaisreal)). + +#### NOT FOR CHANGELOG / INSIGNIFICANT + +* Backported in [#65907](https://github.com/ClickHouse/ClickHouse/issues/65907): Fix bug with session closing in Keeper. [#65735](https://github.com/ClickHouse/ClickHouse/pull/65735) ([Antonio Andelic](https://github.com/antonio2368)). +* Backported in [#65962](https://github.com/ClickHouse/ClickHouse/issues/65962): Add missing workload identity changes. [#65848](https://github.com/ClickHouse/ClickHouse/pull/65848) ([SmitaRKulkarni](https://github.com/SmitaRKulkarni)). +* Backported in [#66033](https://github.com/ClickHouse/ClickHouse/issues/66033): Follow up to [#65046](https://github.com/ClickHouse/ClickHouse/issues/65046). [#65928](https://github.com/ClickHouse/ClickHouse/pull/65928) ([Kseniia Sumarokova](https://github.com/kssenii)). +* Backported in [#66076](https://github.com/ClickHouse/ClickHouse/issues/66076): Fix support of non-const scale arguments in rounding functions. [#65983](https://github.com/ClickHouse/ClickHouse/pull/65983) ([Mikhail Gorshkov](https://github.com/mgorshkov)). +* Backported in [#66017](https://github.com/ClickHouse/ClickHouse/issues/66017): Fix race in s3queue. [#65986](https://github.com/ClickHouse/ClickHouse/pull/65986) ([Kseniia Sumarokova](https://github.com/kssenii)). + diff --git a/utils/list-versions/version_date.tsv b/utils/list-versions/version_date.tsv index 8112ed9083b..271065a78fb 100644 --- a/utils/list-versions/version_date.tsv +++ b/utils/list-versions/version_date.tsv @@ -1,3 +1,4 @@ +v24.6.2.17-stable 2024-07-05 v24.6.1.4423-stable 2024-07-01 v24.5.4.49-stable 2024-07-01 v24.5.3.5-stable 2024-06-13 @@ -6,6 +7,7 @@ v24.5.1.1763-stable 2024-06-01 v24.4.3.25-stable 2024-06-14 v24.4.2.141-stable 2024-06-07 v24.4.1.2088-stable 2024-05-01 +v24.3.5.46-lts 2024-07-03 v24.3.4.147-lts 2024-06-13 v24.3.3.102-lts 2024-05-01 v24.3.2.23-lts 2024-04-03 From 23f3f36207c567427b70fc84a0557cd5ebcc4d31 Mon Sep 17 00:00:00 2001 From: Nikolai Kochetov Date: Fri, 5 Jul 2024 11:18:10 +0000 Subject: [PATCH 224/417] update settings history. --- src/Core/SettingsChangesHistory.cpp | 1 + 1 file changed, 1 insertion(+) diff --git a/src/Core/SettingsChangesHistory.cpp b/src/Core/SettingsChangesHistory.cpp index 60da43afbde..ddcfbb2eab5 100644 --- a/src/Core/SettingsChangesHistory.cpp +++ b/src/Core/SettingsChangesHistory.cpp @@ -62,6 +62,7 @@ static std::initializer_list Date: Fri, 5 Jul 2024 11:19:06 +0000 Subject: [PATCH 225/417] fix deadlock --- src/Databases/DatabaseAtomic.cpp | 15 ++++++++++----- 1 file changed, 10 insertions(+), 5 deletions(-) diff --git a/src/Databases/DatabaseAtomic.cpp b/src/Databases/DatabaseAtomic.cpp index b30b05bb7a7..a48eb2abce6 100644 --- a/src/Databases/DatabaseAtomic.cpp +++ b/src/Databases/DatabaseAtomic.cpp @@ -106,12 +106,17 @@ void DatabaseAtomic::attachTable(ContextPtr /* context_ */, const String & name, StoragePtr DatabaseAtomic::detachTable(ContextPtr /* context */, const String & name) { + // it is important to call destructures not_in_use without + // blocking mutex for avoid potential deadlock. DetachedTables not_in_use; - std::lock_guard lock(mutex); - auto table = DatabaseOrdinary::detachTableUnlocked(name); - table_name_to_path.erase(name); - detached_tables.emplace(table->getStorageID().uuid, table); - not_in_use = cleanupDetachedTables(); + StoragePtr table; + { + std::lock_guard lock(mutex); + table = DatabaseOrdinary::detachTableUnlocked(name); + table_name_to_path.erase(name); + detached_tables.emplace(table->getStorageID().uuid, table); + not_in_use = cleanupDetachedTables(); + } if (!not_in_use.empty()) { From 3ce470c57f4632030890cdabfd7f5eef3c5eba14 Mon Sep 17 00:00:00 2001 From: slvrtrn Date: Fri, 5 Jul 2024 14:42:43 +0200 Subject: [PATCH 226/417] Update AVRO submodule with the array block size fix --- contrib/avro | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/contrib/avro b/contrib/avro index d43acc84d3d..545e7002683 160000 --- a/contrib/avro +++ b/contrib/avro @@ -1 +1 @@ -Subproject commit d43acc84d3d455b016f847d6666fbc3cd27f16a9 +Subproject commit 545e7002683cbc2198164d93088ac8e4955b4628 From 77e60543fd795737dfc41f8b90cc4be7e770dcb5 Mon Sep 17 00:00:00 2001 From: Sema Checherinda <104093494+CheSema@users.noreply.github.com> Date: Fri, 5 Jul 2024 14:58:17 +0200 Subject: [PATCH 227/417] Revert "insertion deduplication on retries for materialised views" --- src/Columns/ColumnObject.cpp | 6 - src/Columns/ColumnObject.h | 2 +- src/Common/CollectionOfDerived.h | 184 ---- src/Core/Settings.h | 6 +- src/Interpreters/AsynchronousInsertQueue.cpp | 19 +- src/Interpreters/InterpreterCheckQuery.cpp | 18 +- src/Interpreters/InterpreterCreateQuery.cpp | 9 +- src/Interpreters/InterpreterExplainQuery.cpp | 8 +- src/Interpreters/InterpreterInsertQuery.cpp | 682 ++++++------- src/Interpreters/InterpreterInsertQuery.h | 17 +- src/Interpreters/Squashing.cpp | 124 ++- src/Interpreters/Squashing.h | 50 +- src/Interpreters/SystemLog.cpp | 8 +- src/Interpreters/TreeRewriter.cpp | 2 +- src/Processors/Chunk.cpp | 20 +- src/Processors/Chunk.h | 58 +- .../PullingAsyncPipelineExecutor.cpp | 9 +- .../Executors/PullingPipelineExecutor.cpp | 9 +- .../Formats/Impl/ParquetBlockOutputFormat.cpp | 4 +- src/Processors/IAccumulatingTransform.cpp | 5 +- .../FinishAggregatingInOrderAlgorithm.cpp | 10 +- .../Algorithms/MergeTreePartLevelInfo.h | 12 +- .../Algorithms/ReplacingSortedAlgorithm.cpp | 2 +- .../Algorithms/ReplacingSortedAlgorithm.h | 7 +- src/Processors/Merges/IMergingTransform.cpp | 2 +- src/Processors/Merges/IMergingTransform.h | 2 +- src/Processors/Sinks/RemoteSink.h | 2 +- src/Processors/Sinks/SinkToStorage.cpp | 5 +- src/Processors/Sinks/SinkToStorage.h | 5 +- src/Processors/Sources/BlocksSource.h | 5 +- src/Processors/Sources/RemoteSource.cpp | 2 +- .../Sources/SourceFromSingleChunk.cpp | 6 +- .../AggregatingInOrderTransform.cpp | 9 +- .../Transforms/AggregatingInOrderTransform.h | 5 +- .../Transforms/AggregatingTransform.cpp | 16 +- .../Transforms/AggregatingTransform.h | 3 +- .../Transforms/ApplySquashingTransform.h | 14 +- .../Transforms/CountingTransform.cpp | 3 +- .../DeduplicationTokenTransforms.cpp | 236 ----- .../Transforms/DeduplicationTokenTransforms.h | 237 ----- .../Transforms/ExpressionTransform.cpp | 2 - .../Transforms/JoiningTransform.cpp | 9 +- src/Processors/Transforms/JoiningTransform.h | 6 +- .../Transforms/MaterializingTransform.cpp | 1 - .../Transforms/MemoryBoundMerging.h | 6 +- ...gingAggregatedMemoryEfficientTransform.cpp | 36 +- ...ergingAggregatedMemoryEfficientTransform.h | 5 +- .../Transforms/MergingAggregatedTransform.cpp | 10 +- .../Transforms/PlanSquashingTransform.cpp | 15 +- .../Transforms/PlanSquashingTransform.h | 3 +- .../Transforms/SelectByIndicesTransform.h | 3 +- .../Transforms/SquashingTransform.cpp | 18 +- .../Transforms/TotalsHavingTransform.cpp | 6 +- .../Transforms/buildPushingToViewsChain.cpp | 127 +-- src/QueryPipeline/QueryPipelineBuilder.h | 2 +- src/QueryPipeline/QueryPlanResourceHolder.cpp | 8 +- src/QueryPipeline/QueryPlanResourceHolder.h | 3 - src/Server/TCPHandler.cpp | 20 +- src/Storages/Distributed/DistributedSink.cpp | 20 +- src/Storages/Distributed/DistributedSink.h | 2 +- src/Storages/FileLog/StorageFileLog.cpp | 9 +- src/Storages/Kafka/StorageKafka.cpp | 8 +- src/Storages/LiveView/LiveViewSink.h | 4 +- src/Storages/LiveView/StorageLiveView.cpp | 18 +- src/Storages/LiveView/StorageLiveView.h | 2 +- src/Storages/MaterializedView/RefreshTask.cpp | 8 +- src/Storages/MergeTree/IMergeTreeDataPart.cpp | 17 +- src/Storages/MergeTree/IMergeTreeDataPart.h | 1 - .../MergeTree/MergeTreeSelectProcessor.cpp | 6 +- .../MergeTree/MergeTreeSequentialSource.cpp | 5 +- src/Storages/MergeTree/MergeTreeSink.cpp | 68 +- src/Storages/MergeTree/MergeTreeSink.h | 3 +- src/Storages/MergeTree/MutateTask.cpp | 19 +- .../MergeTree/ReplicatedMergeTreeSink.cpp | 94 +- .../MergeTree/ReplicatedMergeTreeSink.h | 13 +- src/Storages/MessageQueueSink.cpp | 2 +- src/Storages/MessageQueueSink.h | 2 +- src/Storages/NATS/StorageNATS.cpp | 8 +- .../StorageObjectStorageSink.cpp | 4 +- .../ObjectStorage/StorageObjectStorageSink.h | 2 +- .../StorageObjectStorageQueue.cpp | 8 +- src/Storages/PartitionedSink.cpp | 4 +- src/Storages/PartitionedSink.h | 2 +- .../MaterializedPostgreSQLConsumer.cpp | 8 +- .../PostgreSQLReplicationHandler.cpp | 8 +- src/Storages/RabbitMQ/StorageRabbitMQ.cpp | 8 +- .../RocksDB/EmbeddedRocksDBBulkSink.cpp | 7 +- .../RocksDB/EmbeddedRocksDBBulkSink.h | 2 +- src/Storages/RocksDB/EmbeddedRocksDBSink.cpp | 2 +- src/Storages/RocksDB/EmbeddedRocksDBSink.h | 2 +- .../RocksDB/StorageEmbeddedRocksDB.cpp | 3 +- src/Storages/StorageBuffer.cpp | 10 +- src/Storages/StorageDistributed.cpp | 8 +- src/Storages/StorageFile.cpp | 4 +- src/Storages/StorageKeeperMap.cpp | 9 +- src/Storages/StorageLog.cpp | 8 +- src/Storages/StorageMemory.cpp | 2 +- src/Storages/StorageMongoDB.cpp | 5 +- src/Storages/StorageMySQL.cpp | 4 +- src/Storages/StoragePostgreSQL.cpp | 4 +- src/Storages/StorageRedis.cpp | 9 +- src/Storages/StorageSQLite.cpp | 2 +- src/Storages/StorageSet.cpp | 6 +- src/Storages/StorageStripeLog.cpp | 4 +- src/Storages/StorageURL.cpp | 4 +- src/Storages/StorageURL.h | 2 +- .../System/StorageSystemZooKeeper.cpp | 2 +- src/Storages/WindowView/StorageWindowView.cpp | 59 +- src/Storages/WindowView/StorageWindowView.h | 2 +- .../test_force_deduplication/test.py | 85 +- ...view_and_deduplication_zookeeper.reference | 2 +- ...lized_view_and_deduplication_zookeeper.sql | 2 +- ...lized_view_and_too_many_parts_zookeeper.sh | 4 +- .../0_stateless/01275_parallel_mv.reference | 4 +- ...01927_query_views_log_current_database.sql | 1 - ...ication_token_materialized_views.reference | 14 +- ...deduplication_token_materialized_views.sql | 8 +- .../0_stateless/02125_query_views_log.sql | 2 +- ...02912_ingestion_mv_deduplication.reference | 5 +- .../02912_ingestion_mv_deduplication.sql | 5 +- .../0_stateless/03008_deduplication.python | 657 ------------ ...08_deduplication_cases_from_docs.reference | 41 - .../03008_deduplication_cases_from_docs.sql | 331 ------ ...on_insert_into_partitioned_table.reference | 35 - ...lication_insert_into_partitioned_table.sql | 83 -- ...ert_several_blocks_nonreplicated.reference | 962 ------------------ ...ion_insert_several_blocks_nonreplicated.sh | 59 -- ...insert_several_blocks_replicated.reference | 962 ------------------ ...cation_insert_several_blocks_replicated.sh | 59 -- ...tes_several_blocks_nonreplicated.reference | 962 ------------------ ..._generates_several_blocks_nonreplicated.sh | 59 -- ...erates_several_blocks_replicated.reference | 962 ------------------ ..._mv_generates_several_blocks_replicated.sh | 59 -- ..._mv_into_one_table_nonreplicated.reference | 706 ------------- ...several_mv_into_one_table_nonreplicated.sh | 59 -- ...ral_mv_into_one_table_replicated.reference | 706 ------------- ...on_several_mv_into_one_table_replicated.sh | 59 -- .../03035_max_insert_threads_support.sh | 2 +- 138 files changed, 865 insertions(+), 8646 deletions(-) delete mode 100644 src/Common/CollectionOfDerived.h delete mode 100644 src/Processors/Transforms/DeduplicationTokenTransforms.cpp delete mode 100644 src/Processors/Transforms/DeduplicationTokenTransforms.h delete mode 100644 tests/queries/0_stateless/03008_deduplication.python delete mode 100644 tests/queries/0_stateless/03008_deduplication_cases_from_docs.reference delete mode 100644 tests/queries/0_stateless/03008_deduplication_cases_from_docs.sql delete mode 100644 tests/queries/0_stateless/03008_deduplication_insert_into_partitioned_table.reference delete mode 100644 tests/queries/0_stateless/03008_deduplication_insert_into_partitioned_table.sql delete mode 100644 tests/queries/0_stateless/03008_deduplication_insert_several_blocks_nonreplicated.reference delete mode 100755 tests/queries/0_stateless/03008_deduplication_insert_several_blocks_nonreplicated.sh delete mode 100644 tests/queries/0_stateless/03008_deduplication_insert_several_blocks_replicated.reference delete mode 100755 tests/queries/0_stateless/03008_deduplication_insert_several_blocks_replicated.sh delete mode 100644 tests/queries/0_stateless/03008_deduplication_mv_generates_several_blocks_nonreplicated.reference delete mode 100755 tests/queries/0_stateless/03008_deduplication_mv_generates_several_blocks_nonreplicated.sh delete mode 100644 tests/queries/0_stateless/03008_deduplication_mv_generates_several_blocks_replicated.reference delete mode 100755 tests/queries/0_stateless/03008_deduplication_mv_generates_several_blocks_replicated.sh delete mode 100644 tests/queries/0_stateless/03008_deduplication_several_mv_into_one_table_nonreplicated.reference delete mode 100755 tests/queries/0_stateless/03008_deduplication_several_mv_into_one_table_nonreplicated.sh delete mode 100644 tests/queries/0_stateless/03008_deduplication_several_mv_into_one_table_replicated.reference delete mode 100755 tests/queries/0_stateless/03008_deduplication_several_mv_into_one_table_replicated.sh diff --git a/src/Columns/ColumnObject.cpp b/src/Columns/ColumnObject.cpp index ded56b60e64..90ef974010c 100644 --- a/src/Columns/ColumnObject.cpp +++ b/src/Columns/ColumnObject.cpp @@ -1093,10 +1093,4 @@ void ColumnObject::finalize() checkObjectHasNoAmbiguosPaths(getKeys()); } -void ColumnObject::updateHashFast(SipHash & hash) const -{ - for (const auto & entry : subcolumns) - for (auto & part : entry->data.data) - part->updateHashFast(hash); -} } diff --git a/src/Columns/ColumnObject.h b/src/Columns/ColumnObject.h index b1b8827622f..e2936b27994 100644 --- a/src/Columns/ColumnObject.h +++ b/src/Columns/ColumnObject.h @@ -242,7 +242,7 @@ public: const char * skipSerializedInArena(const char *) const override { throwMustBeConcrete(); } void updateHashWithValue(size_t, SipHash &) const override { throwMustBeConcrete(); } void updateWeakHash32(WeakHash32 &) const override { throwMustBeConcrete(); } - void updateHashFast(SipHash & hash) const override; + void updateHashFast(SipHash &) const override { throwMustBeConcrete(); } void expand(const Filter &, bool) override { throwMustBeConcrete(); } bool hasEqualValues() const override { throwMustBeConcrete(); } size_t byteSizeAt(size_t) const override { throwMustBeConcrete(); } diff --git a/src/Common/CollectionOfDerived.h b/src/Common/CollectionOfDerived.h deleted file mode 100644 index 97c0c3fbc06..00000000000 --- a/src/Common/CollectionOfDerived.h +++ /dev/null @@ -1,184 +0,0 @@ -#pragma once - -#include - -#include - -#include -#include -#include -#include -#include - - -namespace DB -{ - -namespace ErrorCodes -{ - extern const int LOGICAL_ERROR; -} - -/* This is a collections of objects derived from ItemBase. -* Collection contains no more than one instance for each derived type. -* The derived type is used to access the instance. -*/ - -template -class CollectionOfDerivedItems -{ -public: - using Self = CollectionOfDerivedItems; - using ItemPtr = std::shared_ptr; - -private: - struct Rec - { - std::type_index type_idx; - ItemPtr ptr; - - bool operator<(const Rec & other) const - { - return type_idx < other.type_idx; - } - - bool operator<(const std::type_index & value) const - { - return type_idx < value; - } - - bool operator==(const Rec & other) const - { - return type_idx == other.type_idx; - } - }; - using Records = std::vector; - -public: - void swap(Self & other) noexcept - { - records.swap(other.records); - } - - void clear() - { - records.clear(); - } - - bool empty() const - { - return records.empty(); - } - - size_t size() const - { - return records.size(); - } - - Self clone() const - { - Self result; - result.records.reserve(records.size()); - for (const auto & rec : records) - result.records.emplace_back(rec.type_idx, rec.ptr->clone()); - return result; - } - - void append(Self && other) - { - auto middle_idx = records.size(); - std::move(other.records.begin(), other.records.end(), std::back_inserter(records)); - std::inplace_merge(records.begin(), records.begin() + middle_idx, records.end()); - chassert(isUniqTypes()); - } - - template - void add(std::shared_ptr info) - { - static_assert(std::is_base_of_v, "Template parameter must inherit items base class"); - return addImpl(std::type_index(typeid(T)), std::move(info)); - } - - template - std::shared_ptr get() const - { - static_assert(std::is_base_of_v, "Template parameter must inherit items base class"); - auto it = getImpl(std::type_index(typeid(T))); - if (it == records.cend()) - return nullptr; - auto cast = std::dynamic_pointer_cast(it->ptr); - chassert(cast); - return cast; - } - - template - std::shared_ptr extract() - { - static_assert(std::is_base_of_v, "Template parameter must inherit items base class"); - auto it = getImpl(std::type_index(typeid(T))); - if (it == records.cend()) - return nullptr; - auto cast = std::dynamic_pointer_cast(it->ptr); - chassert(cast); - - records.erase(it); - return cast; - } - - std::string debug() const - { - std::string result; - - for (auto & rec : records) - { - result.append(rec.type_idx.name()); - result.append(" "); - } - - return result; - } - -private: - bool isUniqTypes() const - { - auto uniq_it = std::adjacent_find(records.begin(), records.end()); - - return uniq_it == records.end(); - } - - void addImpl(std::type_index type_idx, ItemPtr item) - { - auto it = std::lower_bound(records.begin(), records.end(), type_idx); - - if (it == records.end()) - { - records.emplace_back(type_idx, item); - return; - } - - if (it->type_idx == type_idx) - throw Exception(ErrorCodes::LOGICAL_ERROR, "inserted items must be unique by their type, type {} is inserted twice", type_idx.name()); - - - records.emplace(it, type_idx, item); - - chassert(isUniqTypes()); - } - - Records::const_iterator getImpl(std::type_index type_idx) const - { - auto it = std::lower_bound(records.cbegin(), records.cend(), type_idx); - - if (it == records.cend()) - return records.cend(); - - if (it->type_idx != type_idx) - return records.cend(); - - return it; - } - - Records records; -}; - -} diff --git a/src/Core/Settings.h b/src/Core/Settings.h index 81d0aa0c51d..5903dbd32eb 100644 --- a/src/Core/Settings.h +++ b/src/Core/Settings.h @@ -36,7 +36,7 @@ class IColumn; M(Dialect, dialect, Dialect::clickhouse, "Which dialect will be used to parse query", 0)\ M(UInt64, min_compress_block_size, 65536, "The actual size of the block to compress, if the uncompressed data less than max_compress_block_size is no less than this value and no less than the volume of data for one mark.", 0) \ M(UInt64, max_compress_block_size, 1048576, "The maximum size of blocks of uncompressed data before compressing for writing to a table.", 0) \ - M(UInt64, max_block_size, DEFAULT_BLOCK_SIZE, "Maximum block size in rows for reading", 0) \ + M(UInt64, max_block_size, DEFAULT_BLOCK_SIZE, "Maximum block size for reading", 0) \ M(UInt64, max_insert_block_size, DEFAULT_INSERT_BLOCK_SIZE, "The maximum block size for insertion, if we control the creation of blocks for insertion.", 0) \ M(UInt64, min_insert_block_size_rows, DEFAULT_INSERT_BLOCK_SIZE, "Squash blocks passed to INSERT query to specified size in rows, if blocks are not big enough.", 0) \ M(UInt64, min_insert_block_size_bytes, (DEFAULT_INSERT_BLOCK_SIZE * 256), "Squash blocks passed to INSERT query to specified size in bytes, if blocks are not big enough.", 0) \ @@ -634,8 +634,9 @@ class IColumn; M(Bool, optimize_time_filter_with_preimage, true, "Optimize Date and DateTime predicates by converting functions into equivalent comparisons without conversions (e.g. toYear(col) = 2023 -> col >= '2023-01-01' AND col <= '2023-12-31')", 0) \ M(Bool, normalize_function_names, true, "Normalize function names to their canonical names", 0) \ M(Bool, enable_early_constant_folding, true, "Enable query optimization where we analyze function and subqueries results and rewrite query if there are constants there", 0) \ - M(Bool, deduplicate_blocks_in_dependent_materialized_views, false, "Should deduplicate blocks for materialized views. Use true to always deduplicate in dependent tables.", 0) \ + M(Bool, deduplicate_blocks_in_dependent_materialized_views, false, "Should deduplicate blocks for materialized views if the block is not a duplicate for the table. Use true to always deduplicate in dependent tables.", 0) \ M(Bool, throw_if_deduplication_in_dependent_materialized_views_enabled_with_async_insert, true, "Throw exception on INSERT query when the setting `deduplicate_blocks_in_dependent_materialized_views` is enabled along with `async_insert`. It guarantees correctness, because these features can't work together.", 0) \ + M(Bool, update_insert_deduplication_token_in_dependent_materialized_views, false, "Should update insert deduplication token with table identifier during insert in dependent materialized views.", 0) \ M(Bool, materialized_views_ignore_errors, false, "Allows to ignore errors for MATERIALIZED VIEW, and deliver original block to the table regardless of MVs", 0) \ M(Bool, ignore_materialized_views_with_dropped_target_table, false, "Ignore MVs with dropped target table during pushing to views", 0) \ M(Bool, allow_experimental_refreshable_materialized_view, false, "Allow refreshable materialized views (CREATE MATERIALIZED VIEW REFRESH ...).", 0) \ @@ -952,7 +953,6 @@ class IColumn; #define OBSOLETE_SETTINGS(M, ALIAS) \ /** Obsolete settings that do nothing but left for compatibility reasons. Remove each one after half a year of obsolescence. */ \ - MAKE_OBSOLETE(M, Bool, update_insert_deduplication_token_in_dependent_materialized_views, 1) \ MAKE_OBSOLETE(M, UInt64, max_memory_usage_for_all_queries, 0) \ MAKE_OBSOLETE(M, UInt64, multiple_joins_rewriter_version, 0) \ MAKE_OBSOLETE(M, Bool, enable_debug_queries, false) \ diff --git a/src/Interpreters/AsynchronousInsertQueue.cpp b/src/Interpreters/AsynchronousInsertQueue.cpp index dd1166a9228..d72f3d81549 100644 --- a/src/Interpreters/AsynchronousInsertQueue.cpp +++ b/src/Interpreters/AsynchronousInsertQueue.cpp @@ -301,13 +301,7 @@ void AsynchronousInsertQueue::preprocessInsertQuery(const ASTPtr & query, const auto & insert_query = query->as(); insert_query.async_insert_flush = true; - InterpreterInsertQuery interpreter( - query, - query_context, - query_context->getSettingsRef().insert_allow_materialized_columns, - /* no_squash */ false, - /* no_destination */ false, - /* async_insert */ false); + InterpreterInsertQuery interpreter(query, query_context, query_context->getSettingsRef().insert_allow_materialized_columns); auto table = interpreter.getTable(insert_query); auto sample_block = InterpreterInsertQuery::getSampleBlock(insert_query, table, table->getInMemoryMetadataPtr(), query_context); @@ -787,12 +781,7 @@ try try { interpreter = std::make_unique( - key.query, - insert_context, - key.settings.insert_allow_materialized_columns, - false, - false, - true); + key.query, insert_context, key.settings.insert_allow_materialized_columns, false, false, true); pipeline = interpreter->execute().pipeline; chassert(pipeline.pushing()); @@ -1011,7 +1000,7 @@ Chunk AsynchronousInsertQueue::processEntriesWithParsing( } Chunk chunk(executor.getResultColumns(), total_rows); - chunk.getChunkInfos().add(std::move(chunk_info)); + chunk.setChunkInfo(std::move(chunk_info)); return chunk; } @@ -1063,7 +1052,7 @@ Chunk AsynchronousInsertQueue::processPreprocessedEntries( } Chunk chunk(std::move(result_columns), total_rows); - chunk.getChunkInfos().add(std::move(chunk_info)); + chunk.setChunkInfo(std::move(chunk_info)); return chunk; } diff --git a/src/Interpreters/InterpreterCheckQuery.cpp b/src/Interpreters/InterpreterCheckQuery.cpp index 81bb6290acb..4a84a7bf570 100644 --- a/src/Interpreters/InterpreterCheckQuery.cpp +++ b/src/Interpreters/InterpreterCheckQuery.cpp @@ -2,7 +2,6 @@ #include #include -#include #include @@ -23,7 +22,6 @@ #include #include -#include #include #include #include @@ -93,7 +91,7 @@ Chunk getChunkFromCheckResult(const String & database, const String & table, con return Chunk(std::move(columns), 1); } -class TableCheckTask : public ChunkInfoCloneable +class TableCheckTask : public ChunkInfo { public: TableCheckTask(StorageID table_id, const std::variant & partition_or_part, ContextPtr context) @@ -112,12 +110,6 @@ public: context->checkAccess(AccessType::SHOW_TABLES, table_->getStorageID()); } - TableCheckTask(const TableCheckTask & other) - : table(other.table) - , check_data_tasks(other.check_data_tasks) - , is_finished(other.is_finished.load()) - {} - std::optional checkNext() const { if (isFinished()) @@ -129,8 +121,8 @@ public: std::this_thread::sleep_for(sleep_time); }); - IStorage::DataValidationTasksPtr tmp = check_data_tasks; - auto result = table->checkDataNext(tmp); + IStorage::DataValidationTasksPtr check_data_tasks_ = check_data_tasks; + auto result = table->checkDataNext(check_data_tasks_); is_finished = !result.has_value(); return result; } @@ -188,7 +180,7 @@ protected: /// source should return at least one row to start pipeline result.addColumn(ColumnUInt8::create(1, 1)); /// actual data stored in chunk info - result.getChunkInfos().add(std::move(current_check_task)); + result.setChunkInfo(std::move(current_check_task)); return result; } @@ -288,7 +280,7 @@ public: protected: void transform(Chunk & chunk) override { - auto table_check_task = chunk.getChunkInfos().get(); + auto table_check_task = std::dynamic_pointer_cast(chunk.getChunkInfo()); auto check_result = table_check_task->checkNext(); if (!check_result) { diff --git a/src/Interpreters/InterpreterCreateQuery.cpp b/src/Interpreters/InterpreterCreateQuery.cpp index ee191c02ff8..0ee2bb6c0e9 100644 --- a/src/Interpreters/InterpreterCreateQuery.cpp +++ b/src/Interpreters/InterpreterCreateQuery.cpp @@ -1776,13 +1776,8 @@ BlockIO InterpreterCreateQuery::fillTableIfNeeded(const ASTCreateQuery & create) else insert->select = create.select->clone(); - return InterpreterInsertQuery( - insert, - getContext(), - getContext()->getSettingsRef().insert_allow_materialized_columns, - /* no_squash */ false, - /* no_destination */ false, - /* async_isnert */ false).execute(); + return InterpreterInsertQuery(insert, getContext(), + getContext()->getSettingsRef().insert_allow_materialized_columns).execute(); } return {}; diff --git a/src/Interpreters/InterpreterExplainQuery.cpp b/src/Interpreters/InterpreterExplainQuery.cpp index 26b7e074fdf..7c7b4b3f95a 100644 --- a/src/Interpreters/InterpreterExplainQuery.cpp +++ b/src/Interpreters/InterpreterExplainQuery.cpp @@ -534,13 +534,7 @@ QueryPipeline InterpreterExplainQuery::executeImpl() } else if (dynamic_cast(ast.getExplainedQuery().get())) { - InterpreterInsertQuery insert( - ast.getExplainedQuery(), - getContext(), - /* allow_materialized */ false, - /* no_squash */ false, - /* no_destination */ false, - /* async_isnert */ false); + InterpreterInsertQuery insert(ast.getExplainedQuery(), getContext()); auto io = insert.execute(); printPipeline(io.pipeline.getProcessors(), buf); } diff --git a/src/Interpreters/InterpreterInsertQuery.cpp b/src/Interpreters/InterpreterInsertQuery.cpp index 2cbfc55d008..f396db70d21 100644 --- a/src/Interpreters/InterpreterInsertQuery.cpp +++ b/src/Interpreters/InterpreterInsertQuery.cpp @@ -16,7 +16,6 @@ #include #include #include -#include #include #include #include @@ -27,7 +26,6 @@ #include #include #include -#include #include #include #include @@ -40,7 +38,6 @@ #include #include #include -#include "base/defines.h" namespace ProfileEvents @@ -397,358 +394,28 @@ Chain InterpreterInsertQuery::buildPreSinkChain( return out; } -std::pair, std::vector> InterpreterInsertQuery::buildPreAndSinkChains(size_t presink_streams, size_t sink_streams, StoragePtr table, const StorageMetadataPtr & metadata_snapshot, const Block & query_sample_block) -{ - chassert(presink_streams > 0); - chassert(sink_streams > 0); - - ThreadGroupPtr running_group; - if (current_thread) - running_group = current_thread->getThreadGroup(); - if (!running_group) - running_group = std::make_shared(getContext()); - - std::vector sink_chains; - std::vector presink_chains; - - for (size_t i = 0; i < sink_streams; ++i) - { - auto out = buildSink(table, metadata_snapshot, /* thread_status_holder= */ nullptr, - running_group, /* elapsed_counter_ms= */ nullptr); - - sink_chains.emplace_back(std::move(out)); - } - - for (size_t i = 0; i < presink_streams; ++i) - { - auto out = buildPreSinkChain(sink_chains[0].getInputHeader(), table, metadata_snapshot, query_sample_block); - presink_chains.emplace_back(std::move(out)); - } - - return {std::move(presink_chains), std::move(sink_chains)}; -} - - -QueryPipeline InterpreterInsertQuery::buildInsertSelectPipeline(ASTInsertQuery & query, StoragePtr table) -{ - const Settings & settings = getContext()->getSettingsRef(); - - auto metadata_snapshot = table->getInMemoryMetadataPtr(); - auto query_sample_block = getSampleBlock(query, table, metadata_snapshot, getContext(), no_destination, allow_materialized); - - bool is_trivial_insert_select = false; - - if (settings.optimize_trivial_insert_select) - { - const auto & select_query = query.select->as(); - const auto & selects = select_query.list_of_selects->children; - const auto & union_modes = select_query.list_of_modes; - - /// ASTSelectWithUnionQuery is not normalized now, so it may pass some queries which can be Trivial select queries - const auto mode_is_all = [](const auto & mode) { return mode == SelectUnionMode::UNION_ALL; }; - - is_trivial_insert_select = - std::all_of(union_modes.begin(), union_modes.end(), std::move(mode_is_all)) - && std::all_of(selects.begin(), selects.end(), isTrivialSelect); - } - - ContextPtr select_context = getContext(); - - if (is_trivial_insert_select) - { - /** When doing trivial INSERT INTO ... SELECT ... FROM table, - * don't need to process SELECT with more than max_insert_threads - * and it's reasonable to set block size for SELECT to the desired block size for INSERT - * to avoid unnecessary squashing. - */ - - Settings new_settings = select_context->getSettings(); - - new_settings.max_threads = std::max(1, settings.max_insert_threads); - - if (table->prefersLargeBlocks()) - { - if (settings.min_insert_block_size_rows) - new_settings.max_block_size = settings.min_insert_block_size_rows; - if (settings.min_insert_block_size_bytes) - new_settings.preferred_block_size_bytes = settings.min_insert_block_size_bytes; - } - - auto context_for_trivial_select = Context::createCopy(context); - context_for_trivial_select->setSettings(new_settings); - context_for_trivial_select->setInsertionTable(getContext()->getInsertionTable(), getContext()->getInsertionTableColumnNames()); - - select_context = context_for_trivial_select; - } - - QueryPipelineBuilder pipeline; - - { - auto select_query_options = SelectQueryOptions(QueryProcessingStage::Complete, 1); - - if (settings.allow_experimental_analyzer) - { - InterpreterSelectQueryAnalyzer interpreter_select_analyzer(query.select, select_context, select_query_options); - pipeline = interpreter_select_analyzer.buildQueryPipeline(); - } - else - { - InterpreterSelectWithUnionQuery interpreter_select(query.select, select_context, select_query_options); - pipeline = interpreter_select.buildQueryPipeline(); - } - } - - pipeline.dropTotalsAndExtremes(); - - /// Allow to insert Nullable into non-Nullable columns, NULL values will be added as defaults values. - if (getContext()->getSettingsRef().insert_null_as_default) - { - const auto & input_columns = pipeline.getHeader().getColumnsWithTypeAndName(); - const auto & query_columns = query_sample_block.getColumnsWithTypeAndName(); - const auto & output_columns = metadata_snapshot->getColumns(); - - if (input_columns.size() == query_columns.size()) - { - for (size_t col_idx = 0; col_idx < query_columns.size(); ++col_idx) - { - /// Change query sample block columns to Nullable to allow inserting nullable columns, where NULL values will be substituted with - /// default column values (in AddingDefaultsTransform), so all values will be cast correctly. - if (isNullableOrLowCardinalityNullable(input_columns[col_idx].type) - && !isNullableOrLowCardinalityNullable(query_columns[col_idx].type) - && !isVariant(query_columns[col_idx].type) - && !isDynamic(query_columns[col_idx].type) - && output_columns.has(query_columns[col_idx].name)) - { - query_sample_block.setColumn( - col_idx, - ColumnWithTypeAndName( - makeNullableOrLowCardinalityNullable(query_columns[col_idx].column), - makeNullableOrLowCardinalityNullable(query_columns[col_idx].type), - query_columns[col_idx].name)); - } - } - } - } - - auto actions_dag = ActionsDAG::makeConvertingActions( - pipeline.getHeader().getColumnsWithTypeAndName(), - query_sample_block.getColumnsWithTypeAndName(), - ActionsDAG::MatchColumnsMode::Position); - auto actions = std::make_shared(actions_dag, ExpressionActionsSettings::fromContext(getContext(), CompileExpressions::yes)); - - pipeline.addSimpleTransform([&](const Block & in_header) -> ProcessorPtr - { - return std::make_shared(in_header, actions); - }); - - /// We need to convert Sparse columns to full, because it's destination storage - /// may not support it or may have different settings for applying Sparse serialization. - pipeline.addSimpleTransform([&](const Block & in_header) -> ProcessorPtr - { - return std::make_shared(in_header); - }); - - pipeline.addSimpleTransform([&](const Block & in_header) -> ProcessorPtr - { - auto context_ptr = getContext(); - auto counting = std::make_shared(in_header, nullptr, context_ptr->getQuota()); - counting->setProcessListElement(context_ptr->getProcessListElement()); - counting->setProgressCallback(context_ptr->getProgressCallback()); - - return counting; - }); - - size_t num_select_threads = pipeline.getNumThreads(); - - pipeline.resize(1); - - if (shouldAddSquashingFroStorage(table)) - { - pipeline.addSimpleTransform([&](const Block & in_header) -> ProcessorPtr - { - return std::make_shared( - in_header, - table->prefersLargeBlocks() ? settings.min_insert_block_size_rows : settings.max_block_size, - table->prefersLargeBlocks() ? settings.min_insert_block_size_bytes : 0ULL); - }); - } - - pipeline.addSimpleTransform([&](const Block &in_header) -> ProcessorPtr - { - return std::make_shared(in_header); - }); - - if (!settings.insert_deduplication_token.value.empty()) - { - pipeline.addSimpleTransform([&](const Block & in_header) -> ProcessorPtr - { - return std::make_shared(settings.insert_deduplication_token.value, in_header); - }); - - pipeline.addSimpleTransform([&](const Block & in_header) -> ProcessorPtr - { - return std::make_shared(in_header); - }); - } - - /// Number of streams works like this: - /// * For the SELECT, use `max_threads`, or `max_insert_threads`, or whatever - /// InterpreterSelectQuery ends up with. - /// * Use `max_insert_threads` streams for various insert-preparation steps, e.g. - /// materializing and squashing (too slow to do in one thread). That's `presink_chains`. - /// * If the table supports parallel inserts, use max_insert_threads for writing to IStorage. - /// Otherwise ResizeProcessor them down to 1 stream. - - size_t presink_streams_size = std::max(settings.max_insert_threads, pipeline.getNumStreams()); - - size_t sink_streams_size = table->supportsParallelInsert() ? std::max(1, settings.max_insert_threads) : 1; - - if (!settings.parallel_view_processing) - { - auto table_id = table->getStorageID(); - auto views = DatabaseCatalog::instance().getDependentViews(table_id); - - if (table->isView() || !views.empty()) - sink_streams_size = 1; - } - - auto [presink_chains, sink_chains] = buildPreAndSinkChains( - presink_streams_size, sink_streams_size, - table, metadata_snapshot, query_sample_block); - - pipeline.resize(presink_chains.size()); - - if (shouldAddSquashingFroStorage(table)) - { - pipeline.addSimpleTransform([&](const Block & in_header) -> ProcessorPtr - { - return std::make_shared( - in_header, - table->prefersLargeBlocks() ? settings.min_insert_block_size_rows : settings.max_block_size, - table->prefersLargeBlocks() ? settings.min_insert_block_size_bytes : 0ULL); - }); - } - - for (auto & chain : presink_chains) - pipeline.addResources(chain.detachResources()); - pipeline.addChains(std::move(presink_chains)); - - pipeline.resize(sink_streams_size); - - for (auto & chain : sink_chains) - pipeline.addResources(chain.detachResources()); - pipeline.addChains(std::move(sink_chains)); - - if (!settings.parallel_view_processing) - { - /// Don't use more threads for INSERT than for SELECT to reduce memory consumption. - if (pipeline.getNumThreads() > num_select_threads) - pipeline.setMaxThreads(num_select_threads); - } - else if (pipeline.getNumThreads() < settings.max_threads) - { - /// It is possible for query to have max_threads=1, due to optimize_trivial_insert_select, - /// however in case of parallel_view_processing and multiple views, views can still be processed in parallel. - /// - /// Note, number of threads will be limited by buildPushingToViewsChain() to max_threads. - pipeline.setMaxThreads(settings.max_threads); - } - - pipeline.setSinks([&](const Block & cur_header, QueryPipelineBuilder::StreamType) -> ProcessorPtr - { - return std::make_shared(cur_header); - }); - - return QueryPipelineBuilder::getPipeline(std::move(pipeline)); -} - - -QueryPipeline InterpreterInsertQuery::buildInsertPipeline(ASTInsertQuery & query, StoragePtr table) -{ - const Settings & settings = getContext()->getSettingsRef(); - - auto metadata_snapshot = table->getInMemoryMetadataPtr(); - auto query_sample_block = getSampleBlock(query, table, metadata_snapshot, getContext(), no_destination, allow_materialized); - - Chain chain; - - { - auto [presink_chains, sink_chains] = buildPreAndSinkChains( - /* presink_streams */1, /* sink_streams */1, - table, metadata_snapshot, query_sample_block); - - chain = std::move(presink_chains.front()); - chain.appendChain(std::move(sink_chains.front())); - } - - if (!settings.insert_deduplication_token.value.empty()) - { - chain.addSource(std::make_shared(chain.getInputHeader())); - chain.addSource(std::make_shared(settings.insert_deduplication_token.value, chain.getInputHeader())); - } - - chain.addSource(std::make_shared(chain.getInputHeader())); - - if (shouldAddSquashingFroStorage(table)) - { - bool table_prefers_large_blocks = table->prefersLargeBlocks(); - - auto squashing = std::make_shared( - chain.getInputHeader(), - table_prefers_large_blocks ? settings.min_insert_block_size_rows : settings.max_block_size, - table_prefers_large_blocks ? settings.min_insert_block_size_bytes : 0ULL); - - chain.addSource(std::move(squashing)); - - auto balancing = std::make_shared( - chain.getInputHeader(), - table_prefers_large_blocks ? settings.min_insert_block_size_rows : settings.max_block_size, - table_prefers_large_blocks ? settings.min_insert_block_size_bytes : 0ULL); - - chain.addSource(std::move(balancing)); - } - - auto context_ptr = getContext(); - auto counting = std::make_shared(chain.getInputHeader(), nullptr, context_ptr->getQuota()); - counting->setProcessListElement(context_ptr->getProcessListElement()); - counting->setProgressCallback(context_ptr->getProgressCallback()); - chain.addSource(std::move(counting)); - - QueryPipeline pipeline = QueryPipeline(std::move(chain)); - - pipeline.setNumThreads(std::min(pipeline.getNumThreads(), settings.max_threads)); - pipeline.setConcurrencyControl(settings.use_concurrency_control); - - if (query.hasInlinedData() && !async_insert) - { - /// can execute without additional data - auto format = getInputFormatFromASTInsertQuery(query_ptr, true, query_sample_block, getContext(), nullptr); - for (auto && buffer : owned_buffers) - format->addBuffer(std::move(buffer)); - - auto pipe = getSourceFromInputFormat(query_ptr, std::move(format), getContext(), nullptr); - pipeline.complete(std::move(pipe)); - } - - return pipeline; -} - - BlockIO InterpreterInsertQuery::execute() { const Settings & settings = getContext()->getSettingsRef(); auto & query = query_ptr->as(); + QueryPipelineBuilder pipeline; + std::optional distributed_pipeline; + QueryPlanResourceHolder resources; StoragePtr table = getTable(query); checkStorageSupportsTransactionsIfNeeded(table, getContext()); + StoragePtr inner_table; + if (const auto * mv = dynamic_cast(table.get())) + inner_table = mv->getTargetTable(); + if (query.partition_by && !table->supportsPartitionBy()) throw Exception(ErrorCodes::NOT_IMPLEMENTED, "PARTITION BY clause is not supported by storage"); auto table_lock = table->lockForShare(getContext()->getInitialQueryId(), settings.lock_acquire_timeout); - auto metadata_snapshot = table->getInMemoryMetadataPtr(); + auto query_sample_block = getSampleBlock(query, table, metadata_snapshot, getContext(), no_destination, allow_materialized); /// For table functions we check access while executing @@ -756,43 +423,320 @@ BlockIO InterpreterInsertQuery::execute() if (!query.table_function) getContext()->checkAccess(AccessType::INSERT, query.table_id, query_sample_block.getNames()); - if (!allow_materialized) + if (query.select && settings.parallel_distributed_insert_select) + // Distributed INSERT SELECT + distributed_pipeline = table->distributedWrite(query, getContext()); + + std::vector presink_chains; + std::vector sink_chains; + if (!distributed_pipeline) { - for (const auto & column : metadata_snapshot->getColumns()) - if (column.default_desc.kind == ColumnDefaultKind::Materialized && query_sample_block.has(column.name)) - throw Exception(ErrorCodes::ILLEGAL_COLUMN, "Cannot insert column {}, because it is MATERIALIZED column.", column.name); + /// Number of streams works like this: + /// * For the SELECT, use `max_threads`, or `max_insert_threads`, or whatever + /// InterpreterSelectQuery ends up with. + /// * Use `max_insert_threads` streams for various insert-preparation steps, e.g. + /// materializing and squashing (too slow to do in one thread). That's `presink_chains`. + /// * If the table supports parallel inserts, use the same streams for writing to IStorage. + /// Otherwise ResizeProcessor them down to 1 stream. + /// * If it's not an INSERT SELECT, forget all that and use one stream. + size_t pre_streams_size = 1; + size_t sink_streams_size = 1; + + if (query.select) + { + bool is_trivial_insert_select = false; + + if (settings.optimize_trivial_insert_select) + { + const auto & select_query = query.select->as(); + const auto & selects = select_query.list_of_selects->children; + const auto & union_modes = select_query.list_of_modes; + + /// ASTSelectWithUnionQuery is not normalized now, so it may pass some queries which can be Trivial select queries + const auto mode_is_all = [](const auto & mode) { return mode == SelectUnionMode::UNION_ALL; }; + + is_trivial_insert_select = + std::all_of(union_modes.begin(), union_modes.end(), std::move(mode_is_all)) + && std::all_of(selects.begin(), selects.end(), isTrivialSelect); + } + + if (is_trivial_insert_select) + { + /** When doing trivial INSERT INTO ... SELECT ... FROM table, + * don't need to process SELECT with more than max_insert_threads + * and it's reasonable to set block size for SELECT to the desired block size for INSERT + * to avoid unnecessary squashing. + */ + + Settings new_settings = getContext()->getSettings(); + + new_settings.max_threads = std::max(1, settings.max_insert_threads); + + if (table->prefersLargeBlocks()) + { + if (settings.min_insert_block_size_rows) + new_settings.max_block_size = settings.min_insert_block_size_rows; + if (settings.min_insert_block_size_bytes) + new_settings.preferred_block_size_bytes = settings.min_insert_block_size_bytes; + } + + auto new_context = Context::createCopy(context); + new_context->setSettings(new_settings); + new_context->setInsertionTable(getContext()->getInsertionTable(), getContext()->getInsertionTableColumnNames()); + + auto select_query_options = SelectQueryOptions(QueryProcessingStage::Complete, 1); + + if (settings.allow_experimental_analyzer) + { + InterpreterSelectQueryAnalyzer interpreter_select_analyzer(query.select, new_context, select_query_options); + pipeline = interpreter_select_analyzer.buildQueryPipeline(); + } + else + { + InterpreterSelectWithUnionQuery interpreter_select(query.select, new_context, select_query_options); + pipeline = interpreter_select.buildQueryPipeline(); + } + } + else + { + /// Passing 1 as subquery_depth will disable limiting size of intermediate result. + auto select_query_options = SelectQueryOptions(QueryProcessingStage::Complete, 1); + + if (settings.allow_experimental_analyzer) + { + InterpreterSelectQueryAnalyzer interpreter_select_analyzer(query.select, getContext(), select_query_options); + pipeline = interpreter_select_analyzer.buildQueryPipeline(); + } + else + { + InterpreterSelectWithUnionQuery interpreter_select(query.select, getContext(), select_query_options); + pipeline = interpreter_select.buildQueryPipeline(); + } + } + + pipeline.dropTotalsAndExtremes(); + + if (settings.max_insert_threads > 1) + { + auto table_id = table->getStorageID(); + auto views = DatabaseCatalog::instance().getDependentViews(table_id); + + /// It breaks some views-related tests and we have dedicated `parallel_view_processing` for views, so let's just skip them. + /// Also it doesn't make sense to reshuffle data if storage doesn't support parallel inserts. + const bool resize_to_max_insert_threads = !table->isView() && views.empty() && table->supportsParallelInsert(); + pre_streams_size = resize_to_max_insert_threads ? settings.max_insert_threads + : std::min(settings.max_insert_threads, pipeline.getNumStreams()); + + /// Deduplication when passing insert_deduplication_token breaks if using more than one thread + if (!settings.insert_deduplication_token.toString().empty()) + { + LOG_DEBUG( + getLogger("InsertQuery"), + "Insert-select query using insert_deduplication_token, setting streams to 1 to avoid deduplication issues"); + pre_streams_size = 1; + } + + if (table->supportsParallelInsert()) + sink_streams_size = pre_streams_size; + } + + pipeline.resize(pre_streams_size); + + /// Allow to insert Nullable into non-Nullable columns, NULL values will be added as defaults values. + if (getContext()->getSettingsRef().insert_null_as_default) + { + const auto & input_columns = pipeline.getHeader().getColumnsWithTypeAndName(); + const auto & query_columns = query_sample_block.getColumnsWithTypeAndName(); + const auto & output_columns = metadata_snapshot->getColumns(); + + if (input_columns.size() == query_columns.size()) + { + for (size_t col_idx = 0; col_idx < query_columns.size(); ++col_idx) + { + /// Change query sample block columns to Nullable to allow inserting nullable columns, where NULL values will be substituted with + /// default column values (in AddingDefaultsTransform), so all values will be cast correctly. + if (isNullableOrLowCardinalityNullable(input_columns[col_idx].type) + && !isNullableOrLowCardinalityNullable(query_columns[col_idx].type) + && !isVariant(query_columns[col_idx].type) + && !isDynamic(query_columns[col_idx].type) + && output_columns.has(query_columns[col_idx].name)) + query_sample_block.setColumn(col_idx, ColumnWithTypeAndName(makeNullableOrLowCardinalityNullable(query_columns[col_idx].column), makeNullableOrLowCardinalityNullable(query_columns[col_idx].type), query_columns[col_idx].name)); + } + } + } + } + + ThreadGroupPtr running_group; + if (current_thread) + running_group = current_thread->getThreadGroup(); + if (!running_group) + running_group = std::make_shared(getContext()); + for (size_t i = 0; i < sink_streams_size; ++i) + { + auto out = buildSink(table, metadata_snapshot, /* thread_status_holder= */ nullptr, + running_group, /* elapsed_counter_ms= */ nullptr); + sink_chains.emplace_back(std::move(out)); + } + for (size_t i = 0; i < pre_streams_size; ++i) + { + auto out = buildPreSinkChain(sink_chains[0].getInputHeader(), table, metadata_snapshot, query_sample_block); + presink_chains.emplace_back(std::move(out)); + } } BlockIO res; - if (query.select) + /// What type of query: INSERT or INSERT SELECT or INSERT WATCH? + if (distributed_pipeline) { - if (settings.parallel_distributed_insert_select) + res.pipeline = std::move(*distributed_pipeline); + } + else if (query.select) + { + const auto & header = presink_chains.at(0).getInputHeader(); + auto actions_dag = ActionsDAG::makeConvertingActions( + pipeline.getHeader().getColumnsWithTypeAndName(), + header.getColumnsWithTypeAndName(), + ActionsDAG::MatchColumnsMode::Position); + auto actions = std::make_shared(actions_dag, ExpressionActionsSettings::fromContext(getContext(), CompileExpressions::yes)); + + pipeline.addSimpleTransform([&](const Block & in_header) -> ProcessorPtr { - auto distributed = table->distributedWrite(query, getContext()); - if (distributed) - { - res.pipeline = std::move(*distributed); - } - else - { - res.pipeline = buildInsertSelectPipeline(query, table); - } - } - else + return std::make_shared(in_header, actions); + }); + + /// We need to convert Sparse columns to full, because it's destination storage + /// may not support it or may have different settings for applying Sparse serialization. + pipeline.addSimpleTransform([&](const Block & in_header) -> ProcessorPtr { - res.pipeline = buildInsertSelectPipeline(query, table); + return std::make_shared(in_header); + }); + + pipeline.addSimpleTransform([&](const Block & in_header) -> ProcessorPtr + { + auto context_ptr = getContext(); + auto counting = std::make_shared(in_header, nullptr, context_ptr->getQuota()); + counting->setProcessListElement(context_ptr->getProcessListElement()); + counting->setProgressCallback(context_ptr->getProgressCallback()); + + return counting; + }); + + if (shouldAddSquashingFroStorage(table)) + { + bool table_prefers_large_blocks = table->prefersLargeBlocks(); + + size_t threads = presink_chains.size(); + + pipeline.resize(1); + + pipeline.addTransform(std::make_shared( + header, + table_prefers_large_blocks ? settings.min_insert_block_size_rows : settings.max_block_size, + table_prefers_large_blocks ? settings.min_insert_block_size_bytes : 0ULL)); + + pipeline.resize(threads); + + pipeline.addSimpleTransform([&](const Block & in_header) -> ProcessorPtr + { + return std::make_shared( + in_header, + table_prefers_large_blocks ? settings.min_insert_block_size_rows : settings.max_block_size, + table_prefers_large_blocks ? settings.min_insert_block_size_bytes : 0ULL); + }); } + + size_t num_select_threads = pipeline.getNumThreads(); + + for (auto & chain : presink_chains) + resources = chain.detachResources(); + for (auto & chain : sink_chains) + resources = chain.detachResources(); + + pipeline.addChains(std::move(presink_chains)); + pipeline.resize(sink_chains.size()); + pipeline.addChains(std::move(sink_chains)); + + if (!settings.parallel_view_processing) + { + /// Don't use more threads for INSERT than for SELECT to reduce memory consumption. + if (pipeline.getNumThreads() > num_select_threads) + pipeline.setMaxThreads(num_select_threads); + } + else if (pipeline.getNumThreads() < settings.max_threads) + { + /// It is possible for query to have max_threads=1, due to optimize_trivial_insert_select, + /// however in case of parallel_view_processing and multiple views, views can still be processed in parallel. + /// + /// Note, number of threads will be limited by buildPushingToViewsChain() to max_threads. + pipeline.setMaxThreads(settings.max_threads); + } + + pipeline.setSinks([&](const Block & cur_header, QueryPipelineBuilder::StreamType) -> ProcessorPtr + { + return std::make_shared(cur_header); + }); + + if (!allow_materialized) + { + for (const auto & column : metadata_snapshot->getColumns()) + if (column.default_desc.kind == ColumnDefaultKind::Materialized && header.has(column.name)) + throw Exception(ErrorCodes::ILLEGAL_COLUMN, "Cannot insert column {}, because it is MATERIALIZED column.", column.name); + } + + res.pipeline = QueryPipelineBuilder::getPipeline(std::move(pipeline)); } else { - res.pipeline = buildInsertPipeline(query, table); + auto & chain = presink_chains.at(0); + chain.appendChain(std::move(sink_chains.at(0))); + + if (shouldAddSquashingFroStorage(table)) + { + bool table_prefers_large_blocks = table->prefersLargeBlocks(); + + auto squashing = std::make_shared( + chain.getInputHeader(), + table_prefers_large_blocks ? settings.min_insert_block_size_rows : settings.max_block_size, + table_prefers_large_blocks ? settings.min_insert_block_size_bytes : 0ULL); + + chain.addSource(std::move(squashing)); + + auto balancing = std::make_shared( + chain.getInputHeader(), + table_prefers_large_blocks ? settings.min_insert_block_size_rows : settings.max_block_size, + table_prefers_large_blocks ? settings.min_insert_block_size_bytes : 0ULL); + + chain.addSource(std::move(balancing)); + } + + auto context_ptr = getContext(); + auto counting = std::make_shared(chain.getInputHeader(), nullptr, context_ptr->getQuota()); + counting->setProcessListElement(context_ptr->getProcessListElement()); + counting->setProgressCallback(context_ptr->getProgressCallback()); + chain.addSource(std::move(counting)); + + res.pipeline = QueryPipeline(std::move(presink_chains[0])); + res.pipeline.setNumThreads(std::min(res.pipeline.getNumThreads(), settings.max_threads)); + res.pipeline.setConcurrencyControl(settings.use_concurrency_control); + + if (query.hasInlinedData() && !async_insert) + { + /// can execute without additional data + auto format = getInputFormatFromASTInsertQuery(query_ptr, true, query_sample_block, getContext(), nullptr); + for (auto && buffer : owned_buffers) + format->addBuffer(std::move(buffer)); + + auto pipe = getSourceFromInputFormat(query_ptr, std::move(format), getContext(), nullptr); + res.pipeline.complete(std::move(pipe)); + } } - res.pipeline.addStorageHolder(table); + res.pipeline.addResources(std::move(resources)); - if (const auto * mv = dynamic_cast(table.get())) - res.pipeline.addStorageHolder(mv->getTargetTable()); + res.pipeline.addStorageHolder(table); + if (inner_table) + res.pipeline.addStorageHolder(inner_table); return res; } @@ -813,27 +757,17 @@ void InterpreterInsertQuery::extendQueryLogElemImpl(QueryLogElement & elem, Cont } } - void InterpreterInsertQuery::extendQueryLogElemImpl(QueryLogElement & elem, const ASTPtr &, ContextPtr context_) const { extendQueryLogElemImpl(elem, context_); } - void registerInterpreterInsertQuery(InterpreterFactory & factory) { auto create_fn = [] (const InterpreterFactory::Arguments & args) { - return std::make_unique( - args.query, - args.context, - args.allow_materialized, - /* no_squash */false, - /* no_destination */false, - /* async_insert */false); + return std::make_unique(args.query, args.context, args.allow_materialized); }; factory.registerInterpreter("InterpreterInsertQuery", create_fn); } - - } diff --git a/src/Interpreters/InterpreterInsertQuery.h b/src/Interpreters/InterpreterInsertQuery.h index 894c7c42144..bf73fb2a319 100644 --- a/src/Interpreters/InterpreterInsertQuery.h +++ b/src/Interpreters/InterpreterInsertQuery.h @@ -23,10 +23,10 @@ public: InterpreterInsertQuery( const ASTPtr & query_ptr_, ContextPtr context_, - bool allow_materialized_, - bool no_squash_, - bool no_destination, - bool async_insert_); + bool allow_materialized_ = false, + bool no_squash_ = false, + bool no_destination_ = false, + bool async_insert_ = false); /** Prepare a request for execution. Return block streams * - the stream into which you can write data to execute the query, if INSERT; @@ -73,17 +73,12 @@ private: ASTPtr query_ptr; const bool allow_materialized; - bool no_squash = false; - bool no_destination = false; + const bool no_squash; + const bool no_destination; const bool async_insert; std::vector> owned_buffers; - std::pair, std::vector> buildPreAndSinkChains(size_t presink_streams, size_t sink_streams, StoragePtr table, const StorageMetadataPtr & metadata_snapshot, const Block & query_sample_block); - - QueryPipeline buildInsertSelectPipeline(ASTInsertQuery & query, StoragePtr table); - QueryPipeline buildInsertPipeline(ASTInsertQuery & query, StoragePtr table); - Chain buildSink( const StoragePtr & table, const StorageMetadataPtr & metadata_snapshot, diff --git a/src/Interpreters/Squashing.cpp b/src/Interpreters/Squashing.cpp index 25434d1103e..f8b6a6542cc 100644 --- a/src/Interpreters/Squashing.cpp +++ b/src/Interpreters/Squashing.cpp @@ -1,7 +1,6 @@ #include #include #include -#include namespace DB @@ -12,33 +11,24 @@ namespace ErrorCodes } Squashing::Squashing(Block header_, size_t min_block_size_rows_, size_t min_block_size_bytes_) - : min_block_size_rows(min_block_size_rows_) + : header(header_) + , min_block_size_rows(min_block_size_rows_) , min_block_size_bytes(min_block_size_bytes_) - , header(header_) { } Chunk Squashing::flush() { - if (!accumulated) - return {}; - - auto result = convertToChunk(accumulated.extract()); - chassert(result); - return result; + return convertToChunk(std::move(chunks_to_merge_vec)); } Chunk Squashing::squash(Chunk && input_chunk) { - if (!input_chunk) + if (!input_chunk.hasChunkInfo()) return Chunk(); - auto squash_info = input_chunk.getChunkInfos().extract(); - - if (!squash_info) - throw Exception(ErrorCodes::LOGICAL_ERROR, "There is no ChunksToSquash in ChunkInfoPtr"); - - return squash(std::move(squash_info->chunks), std::move(input_chunk.getChunkInfos())); + const auto *info = getInfoFromChunk(input_chunk); + return squash(info->chunks); } Chunk Squashing::add(Chunk && input_chunk) @@ -47,37 +37,48 @@ Chunk Squashing::add(Chunk && input_chunk) return {}; /// Just read block is already enough. - if (isEnoughSize(input_chunk)) + if (isEnoughSize(input_chunk.getNumRows(), input_chunk.bytes())) { /// If no accumulated data, return just read block. - if (!accumulated) + if (chunks_to_merge_vec.empty()) { - accumulated.add(std::move(input_chunk)); - return convertToChunk(accumulated.extract()); + chunks_to_merge_vec.push_back(std::move(input_chunk)); + Chunk res_chunk = convertToChunk(std::move(chunks_to_merge_vec)); + chunks_to_merge_vec.clear(); + return res_chunk; } /// Return accumulated data (maybe it has small size) and place new block to accumulated data. - Chunk res_chunk = convertToChunk(accumulated.extract()); - accumulated.add(std::move(input_chunk)); + Chunk res_chunk = convertToChunk(std::move(chunks_to_merge_vec)); + chunks_to_merge_vec.clear(); + changeCurrentSize(input_chunk.getNumRows(), input_chunk.bytes()); + chunks_to_merge_vec.push_back(std::move(input_chunk)); return res_chunk; } /// Accumulated block is already enough. - if (isEnoughSize()) + if (isEnoughSize(accumulated_size.rows, accumulated_size.bytes)) { /// Return accumulated data and place new block to accumulated data. - Chunk res_chunk = convertToChunk(accumulated.extract()); - accumulated.add(std::move(input_chunk)); + Chunk res_chunk = convertToChunk(std::move(chunks_to_merge_vec)); + chunks_to_merge_vec.clear(); + changeCurrentSize(input_chunk.getNumRows(), input_chunk.bytes()); + chunks_to_merge_vec.push_back(std::move(input_chunk)); return res_chunk; } /// Pushing data into accumulating vector - accumulated.add(std::move(input_chunk)); + expandCurrentSize(input_chunk.getNumRows(), input_chunk.bytes()); + chunks_to_merge_vec.push_back(std::move(input_chunk)); /// If accumulated data is big enough, we send it - if (isEnoughSize()) - return convertToChunk(accumulated.extract()); - + if (isEnoughSize(accumulated_size.rows, accumulated_size.bytes)) + { + Chunk res_chunk = convertToChunk(std::move(chunks_to_merge_vec)); + changeCurrentSize(0, 0); + chunks_to_merge_vec.clear(); + return res_chunk; + } return {}; } @@ -89,15 +90,14 @@ Chunk Squashing::convertToChunk(std::vector && chunks) const auto info = std::make_shared(); info->chunks = std::move(chunks); - // It is imortant that chunk is not empty, it has to have columns even if they are empty - auto aggr_chunk = Chunk(header.getColumns(), 0); - aggr_chunk.getChunkInfos().add(std::move(info)); - chassert(aggr_chunk); - return aggr_chunk; + chunks.clear(); + + return Chunk(header.cloneEmptyColumns(), 0, info); } -Chunk Squashing::squash(std::vector && input_chunks, Chunk::ChunkInfoCollection && infos) +Chunk Squashing::squash(std::vector & input_chunks) { + Chunk accumulated_chunk; std::vector mutable_columns = {}; size_t rows = 0; for (const Chunk & chunk : input_chunks) @@ -119,17 +119,35 @@ Chunk Squashing::squash(std::vector && input_chunks, Chunk::ChunkInfoColl for (size_t j = 0, size = mutable_columns.size(); j < size; ++j) { const auto source_column = columns[j]; + mutable_columns[j]->insertRangeFrom(*source_column, 0, source_column->size()); } } + accumulated_chunk.setColumns(std::move(mutable_columns), rows); + return accumulated_chunk; +} - Chunk result; - result.setColumns(std::move(mutable_columns), rows); - result.setChunkInfos(infos); - result.getChunkInfos().append(std::move(input_chunks.back().getChunkInfos())); +const ChunksToSquash* Squashing::getInfoFromChunk(const Chunk & chunk) +{ + const auto& info = chunk.getChunkInfo(); + const auto * agg_info = typeid_cast(info.get()); - chassert(result); - return result; + if (!agg_info) + throw Exception(ErrorCodes::LOGICAL_ERROR, "There is no ChunksToSquash in ChunkInfoPtr"); + + return agg_info; +} + +void Squashing::expandCurrentSize(size_t rows, size_t bytes) +{ + accumulated_size.rows += rows; + accumulated_size.bytes += bytes; +} + +void Squashing::changeCurrentSize(size_t rows, size_t bytes) +{ + accumulated_size.rows = rows; + accumulated_size.bytes = bytes; } bool Squashing::isEnoughSize(size_t rows, size_t bytes) const @@ -138,28 +156,4 @@ bool Squashing::isEnoughSize(size_t rows, size_t bytes) const || (min_block_size_rows && rows >= min_block_size_rows) || (min_block_size_bytes && bytes >= min_block_size_bytes); } - -bool Squashing::isEnoughSize() const -{ - return isEnoughSize(accumulated.getRows(), accumulated.getBytes()); -}; - -bool Squashing::isEnoughSize(const Chunk & chunk) const -{ - return isEnoughSize(chunk.getNumRows(), chunk.bytes()); -} - -void Squashing::CurrentSize::add(Chunk && chunk) -{ - rows += chunk.getNumRows(); - bytes += chunk.bytes(); - chunks.push_back(std::move(chunk)); -} - -std::vector Squashing::CurrentSize::extract() -{ - auto result = std::move(chunks); - *this = {}; - return result; -} } diff --git a/src/Interpreters/Squashing.h b/src/Interpreters/Squashing.h index 64a9768a71f..d76cca60e41 100644 --- a/src/Interpreters/Squashing.h +++ b/src/Interpreters/Squashing.h @@ -8,18 +8,9 @@ namespace DB { -class ChunksToSquash : public ChunkInfoCloneable +struct ChunksToSquash : public ChunkInfo { -public: - ChunksToSquash() = default; - ChunksToSquash(const ChunksToSquash & other) - { - chunks.reserve(other.chunks.size()); - for (const auto & chunk: other.chunks) - chunks.push_back(chunk.clone()); - } - - std::vector chunks = {}; + mutable std::vector chunks = {}; }; /** Merging consecutive passed blocks to specified minimum size. @@ -45,35 +36,32 @@ public: static Chunk squash(Chunk && input_chunk); Chunk flush(); - void setHeader(Block header_) { header = std::move(header_); } - const Block & getHeader() const { return header; } - -private: - class CurrentSize + bool isDataLeft() + { + return !chunks_to_merge_vec.empty(); + } + + Block header; +private: + struct CurrentSize { - std::vector chunks = {}; size_t rows = 0; size_t bytes = 0; - - public: - explicit operator bool () const { return !chunks.empty(); } - size_t getRows() const { return rows; } - size_t getBytes() const { return bytes; } - void add(Chunk && chunk); - std::vector extract(); }; - const size_t min_block_size_rows; - const size_t min_block_size_bytes; - Block header; + std::vector chunks_to_merge_vec = {}; + size_t min_block_size_rows; + size_t min_block_size_bytes; - CurrentSize accumulated; + CurrentSize accumulated_size; - static Chunk squash(std::vector && input_chunks, Chunk::ChunkInfoCollection && infos); + static const ChunksToSquash * getInfoFromChunk(const Chunk & chunk); - bool isEnoughSize() const; + static Chunk squash(std::vector & input_chunks); + + void expandCurrentSize(size_t rows, size_t bytes); + void changeCurrentSize(size_t rows, size_t bytes); bool isEnoughSize(size_t rows, size_t bytes) const; - bool isEnoughSize(const Chunk & chunk) const; Chunk convertToChunk(std::vector && chunks) const; }; diff --git a/src/Interpreters/SystemLog.cpp b/src/Interpreters/SystemLog.cpp index f386e157b14..557065b23ff 100644 --- a/src/Interpreters/SystemLog.cpp +++ b/src/Interpreters/SystemLog.cpp @@ -538,13 +538,7 @@ void SystemLog::flushImpl(const std::vector & to_flush, insert_context->makeQueryContext(); addSettingsForQuery(insert_context, IAST::QueryKind::Insert); - InterpreterInsertQuery interpreter( - query_ptr, - insert_context, - /* allow_materialized */ false, - /* no_squash */ false, - /* no_destination */ false, - /* async_isnert */ false); + InterpreterInsertQuery interpreter(query_ptr, insert_context); BlockIO io = interpreter.execute(); PushingPipelineExecutor executor(io.pipeline); diff --git a/src/Interpreters/TreeRewriter.cpp b/src/Interpreters/TreeRewriter.cpp index 6ce6f5e454e..a3c5a7ed3ed 100644 --- a/src/Interpreters/TreeRewriter.cpp +++ b/src/Interpreters/TreeRewriter.cpp @@ -1188,7 +1188,7 @@ bool TreeRewriterResult::collectUsedColumns(const ASTPtr & query, bool is_select } } - /// Check for dynamic subcolumns in unknown required columns. + /// Check for dynamic subcolums in unknown required columns. if (!unknown_required_source_columns.empty()) { for (const NameAndTypePair & pair : source_columns_ordinary) diff --git a/src/Processors/Chunk.cpp b/src/Processors/Chunk.cpp index 4466be5b3a7..5f6cf2f7230 100644 --- a/src/Processors/Chunk.cpp +++ b/src/Processors/Chunk.cpp @@ -19,6 +19,14 @@ Chunk::Chunk(DB::Columns columns_, UInt64 num_rows_) : columns(std::move(columns checkNumRowsIsConsistent(); } +Chunk::Chunk(Columns columns_, UInt64 num_rows_, ChunkInfoPtr chunk_info_) + : columns(std::move(columns_)) + , num_rows(num_rows_) + , chunk_info(std::move(chunk_info_)) +{ + checkNumRowsIsConsistent(); +} + static Columns unmuteColumns(MutableColumns && mutable_columns) { Columns columns; @@ -35,11 +43,17 @@ Chunk::Chunk(MutableColumns columns_, UInt64 num_rows_) checkNumRowsIsConsistent(); } +Chunk::Chunk(MutableColumns columns_, UInt64 num_rows_, ChunkInfoPtr chunk_info_) + : columns(unmuteColumns(std::move(columns_))) + , num_rows(num_rows_) + , chunk_info(std::move(chunk_info_)) +{ + checkNumRowsIsConsistent(); +} + Chunk Chunk::clone() const { - auto tmp = Chunk(getColumns(), getNumRows()); - tmp.setChunkInfos(chunk_infos.clone()); - return tmp; + return Chunk(getColumns(), getNumRows(), chunk_info); } void Chunk::setColumns(Columns columns_, UInt64 num_rows_) diff --git a/src/Processors/Chunk.h b/src/Processors/Chunk.h index 1348966c0d3..4f753798eaa 100644 --- a/src/Processors/Chunk.h +++ b/src/Processors/Chunk.h @@ -1,9 +1,7 @@ #pragma once -#include #include - -#include +#include namespace DB { @@ -11,29 +9,11 @@ namespace DB class ChunkInfo { public: - using Ptr = std::shared_ptr; - - ChunkInfo() = default; - ChunkInfo(const ChunkInfo&) = default; - ChunkInfo(ChunkInfo&&) = default; - - virtual Ptr clone() const = 0; virtual ~ChunkInfo() = default; + ChunkInfo() = default; }; - -template -class ChunkInfoCloneable : public ChunkInfo -{ -public: - ChunkInfoCloneable() = default; - ChunkInfoCloneable(const ChunkInfoCloneable & other) = default; - - Ptr clone() const override - { - return std::static_pointer_cast(std::make_shared(*static_cast(this))); - } -}; +using ChunkInfoPtr = std::shared_ptr; /** * Chunk is a list of columns with the same length. @@ -52,26 +32,26 @@ public: class Chunk { public: - using ChunkInfoCollection = CollectionOfDerivedItems; - Chunk() = default; Chunk(const Chunk & other) = delete; Chunk(Chunk && other) noexcept : columns(std::move(other.columns)) , num_rows(other.num_rows) - , chunk_infos(std::move(other.chunk_infos)) + , chunk_info(std::move(other.chunk_info)) { other.num_rows = 0; } Chunk(Columns columns_, UInt64 num_rows_); + Chunk(Columns columns_, UInt64 num_rows_, ChunkInfoPtr chunk_info_); Chunk(MutableColumns columns_, UInt64 num_rows_); + Chunk(MutableColumns columns_, UInt64 num_rows_, ChunkInfoPtr chunk_info_); Chunk & operator=(const Chunk & other) = delete; Chunk & operator=(Chunk && other) noexcept { columns = std::move(other.columns); - chunk_infos = std::move(other.chunk_infos); + chunk_info = std::move(other.chunk_info); num_rows = other.num_rows; other.num_rows = 0; return *this; @@ -82,15 +62,15 @@ public: void swap(Chunk & other) noexcept { columns.swap(other.columns); + chunk_info.swap(other.chunk_info); std::swap(num_rows, other.num_rows); - chunk_infos.swap(other.chunk_infos); } void clear() { num_rows = 0; columns.clear(); - chunk_infos.clear(); + chunk_info.reset(); } const Columns & getColumns() const { return columns; } @@ -101,9 +81,9 @@ public: /** Get empty columns with the same types as in block. */ MutableColumns cloneEmptyColumns() const; - ChunkInfoCollection & getChunkInfos() { return chunk_infos; } - const ChunkInfoCollection & getChunkInfos() const { return chunk_infos; } - void setChunkInfos(ChunkInfoCollection chunk_infos_) { chunk_infos = std::move(chunk_infos_); } + const ChunkInfoPtr & getChunkInfo() const { return chunk_info; } + bool hasChunkInfo() const { return chunk_info != nullptr; } + void setChunkInfo(ChunkInfoPtr chunk_info_) { chunk_info = std::move(chunk_info_); } UInt64 getNumRows() const { return num_rows; } UInt64 getNumColumns() const { return columns.size(); } @@ -127,7 +107,7 @@ public: private: Columns columns; UInt64 num_rows = 0; - ChunkInfoCollection chunk_infos; + ChunkInfoPtr chunk_info; void checkNumRowsIsConsistent(); }; @@ -137,15 +117,11 @@ using Chunks = std::vector; /// AsyncInsert needs two kinds of information: /// - offsets of different sub-chunks /// - tokens of different sub-chunks, which are assigned by setting `insert_deduplication_token`. -class AsyncInsertInfo : public ChunkInfoCloneable +class AsyncInsertInfo : public ChunkInfo { public: AsyncInsertInfo() = default; - AsyncInsertInfo(const AsyncInsertInfo & other) = default; - AsyncInsertInfo(const std::vector & offsets_, const std::vector & tokens_) - : offsets(offsets_) - , tokens(tokens_) - {} + explicit AsyncInsertInfo(const std::vector & offsets_, const std::vector & tokens_) : offsets(offsets_), tokens(tokens_) {} std::vector offsets; std::vector tokens; @@ -154,11 +130,9 @@ public: using AsyncInsertInfoPtr = std::shared_ptr; /// Extension to support delayed defaults. AddingDefaultsProcessor uses it to replace missing values with column defaults. -class ChunkMissingValues : public ChunkInfoCloneable +class ChunkMissingValues : public ChunkInfo { public: - ChunkMissingValues(const ChunkMissingValues & other) = default; - using RowsBitMask = std::vector; /// a bit per row for a column const RowsBitMask & getDefaultsBitmask(size_t column_idx) const; diff --git a/src/Processors/Executors/PullingAsyncPipelineExecutor.cpp b/src/Processors/Executors/PullingAsyncPipelineExecutor.cpp index d9fab88fe1f..d27002197d2 100644 --- a/src/Processors/Executors/PullingAsyncPipelineExecutor.cpp +++ b/src/Processors/Executors/PullingAsyncPipelineExecutor.cpp @@ -147,10 +147,13 @@ bool PullingAsyncPipelineExecutor::pull(Block & block, uint64_t milliseconds) block = lazy_format->getPort(IOutputFormat::PortKind::Main).getHeader().cloneWithColumns(chunk.detachColumns()); - if (auto agg_info = chunk.getChunkInfos().get()) + if (auto chunk_info = chunk.getChunkInfo()) { - block.info.bucket_num = agg_info->bucket_num; - block.info.is_overflows = agg_info->is_overflows; + if (const auto * agg_info = typeid_cast(chunk_info.get())) + { + block.info.bucket_num = agg_info->bucket_num; + block.info.is_overflows = agg_info->is_overflows; + } } return true; diff --git a/src/Processors/Executors/PullingPipelineExecutor.cpp b/src/Processors/Executors/PullingPipelineExecutor.cpp index 25c15d40c9a..cbf73c5cb07 100644 --- a/src/Processors/Executors/PullingPipelineExecutor.cpp +++ b/src/Processors/Executors/PullingPipelineExecutor.cpp @@ -73,10 +73,13 @@ bool PullingPipelineExecutor::pull(Block & block) } block = pulling_format->getPort(IOutputFormat::PortKind::Main).getHeader().cloneWithColumns(chunk.detachColumns()); - if (auto agg_info = chunk.getChunkInfos().get()) + if (auto chunk_info = chunk.getChunkInfo()) { - block.info.bucket_num = agg_info->bucket_num; - block.info.is_overflows = agg_info->is_overflows; + if (const auto * agg_info = typeid_cast(chunk_info.get())) + { + block.info.bucket_num = agg_info->bucket_num; + block.info.is_overflows = agg_info->is_overflows; + } } return true; diff --git a/src/Processors/Formats/Impl/ParquetBlockOutputFormat.cpp b/src/Processors/Formats/Impl/ParquetBlockOutputFormat.cpp index 9e499e2c400..a5d334f4f1d 100644 --- a/src/Processors/Formats/Impl/ParquetBlockOutputFormat.cpp +++ b/src/Processors/Formats/Impl/ParquetBlockOutputFormat.cpp @@ -179,9 +179,7 @@ void ParquetBlockOutputFormat::consume(Chunk chunk) columns[i]->insertRangeFrom(*concatenated.getColumns()[i], offset, count); Chunks piece; - piece.emplace_back(std::move(columns), count); - piece.back().setChunkInfos(concatenated.getChunkInfos()); - + piece.emplace_back(std::move(columns), count, concatenated.getChunkInfo()); writeRowGroup(std::move(piece)); } } diff --git a/src/Processors/IAccumulatingTransform.cpp b/src/Processors/IAccumulatingTransform.cpp index 46be6e74693..4136fc5a5f2 100644 --- a/src/Processors/IAccumulatingTransform.cpp +++ b/src/Processors/IAccumulatingTransform.cpp @@ -8,9 +8,8 @@ namespace ErrorCodes } IAccumulatingTransform::IAccumulatingTransform(Block input_header, Block output_header) - : IProcessor({std::move(input_header)}, {std::move(output_header)}) - , input(inputs.front()) - , output(outputs.front()) + : IProcessor({std::move(input_header)}, {std::move(output_header)}), + input(inputs.front()), output(outputs.front()) { } diff --git a/src/Processors/Merges/Algorithms/FinishAggregatingInOrderAlgorithm.cpp b/src/Processors/Merges/Algorithms/FinishAggregatingInOrderAlgorithm.cpp index 86675bcb237..466adf93538 100644 --- a/src/Processors/Merges/Algorithms/FinishAggregatingInOrderAlgorithm.cpp +++ b/src/Processors/Merges/Algorithms/FinishAggregatingInOrderAlgorithm.cpp @@ -53,11 +53,13 @@ void FinishAggregatingInOrderAlgorithm::consume(Input & input, size_t source_num if (!input.chunk.hasRows()) return; - if (input.chunk.getChunkInfos().empty()) + const auto & info = input.chunk.getChunkInfo(); + if (!info) throw Exception(ErrorCodes::LOGICAL_ERROR, "Chunk info was not set for chunk in FinishAggregatingInOrderAlgorithm"); Int64 allocated_bytes = 0; - if (auto arenas_info = input.chunk.getChunkInfos().get()) + /// Will be set by AggregatingInOrderTransform during local aggregation; will be nullptr during merging on initiator. + if (const auto * arenas_info = typeid_cast(info.get())) allocated_bytes = arenas_info->allocated_bytes; states[source_num] = State{input.chunk, description, allocated_bytes}; @@ -134,7 +136,7 @@ Chunk FinishAggregatingInOrderAlgorithm::prepareToMerge() info->chunk_num = chunk_num++; Chunk chunk; - chunk.getChunkInfos().add(std::move(info)); + chunk.setChunkInfo(std::move(info)); return chunk; } @@ -161,7 +163,7 @@ void FinishAggregatingInOrderAlgorithm::addToAggregation() chunks.emplace_back(std::move(new_columns), current_rows); } - chunks.back().getChunkInfos().add(std::make_shared()); + chunks.back().setChunkInfo(std::make_shared()); states[i].current_row = states[i].to_row; /// We assume that sizes in bytes of rows are almost the same. diff --git a/src/Processors/Merges/Algorithms/MergeTreePartLevelInfo.h b/src/Processors/Merges/Algorithms/MergeTreePartLevelInfo.h index e4f22deec8d..bcf4e759024 100644 --- a/src/Processors/Merges/Algorithms/MergeTreePartLevelInfo.h +++ b/src/Processors/Merges/Algorithms/MergeTreePartLevelInfo.h @@ -6,22 +6,18 @@ namespace DB { /// To carry part level if chunk is produced by a merge tree source -class MergeTreePartLevelInfo : public ChunkInfoCloneable +class MergeTreePartLevelInfo : public ChunkInfo { public: MergeTreePartLevelInfo() = delete; - explicit MergeTreePartLevelInfo(ssize_t part_level) - : origin_merge_tree_part_level(part_level) - { } - MergeTreePartLevelInfo(const MergeTreePartLevelInfo & other) = default; - + explicit MergeTreePartLevelInfo(ssize_t part_level) : origin_merge_tree_part_level(part_level) { } size_t origin_merge_tree_part_level = 0; }; inline size_t getPartLevelFromChunk(const Chunk & chunk) { - const auto part_level_info = chunk.getChunkInfos().get(); - if (part_level_info) + const auto & info = chunk.getChunkInfo(); + if (const auto * part_level_info = typeid_cast(info.get())) return part_level_info->origin_merge_tree_part_level; return 0; } diff --git a/src/Processors/Merges/Algorithms/ReplacingSortedAlgorithm.cpp b/src/Processors/Merges/Algorithms/ReplacingSortedAlgorithm.cpp index cd347d371d9..7b2c7d82a01 100644 --- a/src/Processors/Merges/Algorithms/ReplacingSortedAlgorithm.cpp +++ b/src/Processors/Merges/Algorithms/ReplacingSortedAlgorithm.cpp @@ -17,7 +17,7 @@ namespace ErrorCodes static IMergingAlgorithm::Status emitChunk(detail::SharedChunkPtr & chunk, bool finished = false) { - chunk->getChunkInfos().add(std::make_shared(std::move(chunk->replace_final_selection))); + chunk->setChunkInfo(std::make_shared(std::move(chunk->replace_final_selection))); return IMergingAlgorithm::Status(std::move(*chunk), finished); } diff --git a/src/Processors/Merges/Algorithms/ReplacingSortedAlgorithm.h b/src/Processors/Merges/Algorithms/ReplacingSortedAlgorithm.h index 2f23f2a5c4d..a3ccccf0845 100644 --- a/src/Processors/Merges/Algorithms/ReplacingSortedAlgorithm.h +++ b/src/Processors/Merges/Algorithms/ReplacingSortedAlgorithm.h @@ -3,7 +3,6 @@ #include #include #include -#include namespace Poco { @@ -15,13 +14,11 @@ namespace DB /** Use in skipping final to keep list of indices of selected row after merging final */ -struct ChunkSelectFinalIndices : public ChunkInfoCloneable +struct ChunkSelectFinalIndices : public ChunkInfo { - explicit ChunkSelectFinalIndices(MutableColumnPtr select_final_indices_); - ChunkSelectFinalIndices(const ChunkSelectFinalIndices & other) = default; - const ColumnPtr column_holder; const ColumnUInt64 * select_final_indices = nullptr; + explicit ChunkSelectFinalIndices(MutableColumnPtr select_final_indices_); }; /** Merges several sorted inputs into one. diff --git a/src/Processors/Merges/IMergingTransform.cpp b/src/Processors/Merges/IMergingTransform.cpp index b1b0182a113..fbb47969b2f 100644 --- a/src/Processors/Merges/IMergingTransform.cpp +++ b/src/Processors/Merges/IMergingTransform.cpp @@ -157,7 +157,7 @@ IProcessor::Status IMergingTransformBase::prepare() bool is_port_full = !output.canPush(); /// Push if has data. - if ((state.output_chunk || !state.output_chunk.getChunkInfos().empty()) && !is_port_full) + if ((state.output_chunk || state.output_chunk.hasChunkInfo()) && !is_port_full) output.push(std::move(state.output_chunk)); if (!is_initialized) diff --git a/src/Processors/Merges/IMergingTransform.h b/src/Processors/Merges/IMergingTransform.h index be629271736..c218f622870 100644 --- a/src/Processors/Merges/IMergingTransform.h +++ b/src/Processors/Merges/IMergingTransform.h @@ -129,7 +129,7 @@ public: IMergingAlgorithm::Status status = algorithm.merge(); - if ((status.chunk && status.chunk.hasRows()) || !status.chunk.getChunkInfos().empty()) + if ((status.chunk && status.chunk.hasRows()) || status.chunk.hasChunkInfo()) { // std::cerr << "Got chunk with " << status.chunk.getNumRows() << " rows" << std::endl; state.output_chunk = std::move(status.chunk); diff --git a/src/Processors/Sinks/RemoteSink.h b/src/Processors/Sinks/RemoteSink.h index c05cc1defcb..30cf958c072 100644 --- a/src/Processors/Sinks/RemoteSink.h +++ b/src/Processors/Sinks/RemoteSink.h @@ -20,7 +20,7 @@ public: } String getName() const override { return "RemoteSink"; } - void consume (Chunk & chunk) override { write(RemoteInserter::getHeader().cloneWithColumns(chunk.getColumns())); } + void consume (Chunk chunk) override { write(RemoteInserter::getHeader().cloneWithColumns(chunk.detachColumns())); } void onFinish() override { RemoteInserter::onFinish(); } }; diff --git a/src/Processors/Sinks/SinkToStorage.cpp b/src/Processors/Sinks/SinkToStorage.cpp index 36bb70f493f..5f9f9f9b1a1 100644 --- a/src/Processors/Sinks/SinkToStorage.cpp +++ b/src/Processors/Sinks/SinkToStorage.cpp @@ -15,8 +15,9 @@ void SinkToStorage::onConsume(Chunk chunk) */ Nested::validateArraySizes(getHeader().cloneWithColumns(chunk.getColumns())); - consume(chunk); - cur_chunk = std::move(chunk); + consume(chunk.clone()); + if (!lastBlockIsDuplicate()) + cur_chunk = std::move(chunk); } SinkToStorage::GenerateResult SinkToStorage::onGenerate() diff --git a/src/Processors/Sinks/SinkToStorage.h b/src/Processors/Sinks/SinkToStorage.h index c728fa87b1e..023bbd8b094 100644 --- a/src/Processors/Sinks/SinkToStorage.h +++ b/src/Processors/Sinks/SinkToStorage.h @@ -18,7 +18,8 @@ public: void addTableLock(const TableLockHolder & lock) { table_locks.push_back(lock); } protected: - virtual void consume(Chunk & chunk) = 0; + virtual void consume(Chunk chunk) = 0; + virtual bool lastBlockIsDuplicate() const { return false; } private: std::vector table_locks; @@ -37,7 +38,7 @@ class NullSinkToStorage : public SinkToStorage public: using SinkToStorage::SinkToStorage; std::string getName() const override { return "NullSinkToStorage"; } - void consume(Chunk &) override {} + void consume(Chunk) override {} }; using SinkPtr = std::shared_ptr; diff --git a/src/Processors/Sources/BlocksSource.h b/src/Processors/Sources/BlocksSource.h index 7ac460c14e2..ec0dc9609f1 100644 --- a/src/Processors/Sources/BlocksSource.h +++ b/src/Processors/Sources/BlocksSource.h @@ -43,10 +43,7 @@ protected: info->bucket_num = res.info.bucket_num; info->is_overflows = res.info.is_overflows; - auto chunk = Chunk(res.getColumns(), res.rows()); - chunk.getChunkInfos().add(std::move(info)); - - return chunk; + return Chunk(res.getColumns(), res.rows(), std::move(info)); } private: diff --git a/src/Processors/Sources/RemoteSource.cpp b/src/Processors/Sources/RemoteSource.cpp index 1578bd389c9..3d7dd3f76b8 100644 --- a/src/Processors/Sources/RemoteSource.cpp +++ b/src/Processors/Sources/RemoteSource.cpp @@ -176,7 +176,7 @@ std::optional RemoteSource::tryGenerate() auto info = std::make_shared(); info->bucket_num = block.info.bucket_num; info->is_overflows = block.info.is_overflows; - chunk.getChunkInfos().add(std::move(info)); + chunk.setChunkInfo(std::move(info)); } return chunk; diff --git a/src/Processors/Sources/SourceFromSingleChunk.cpp b/src/Processors/Sources/SourceFromSingleChunk.cpp index 9abe0504d10..00f40a34361 100644 --- a/src/Processors/Sources/SourceFromSingleChunk.cpp +++ b/src/Processors/Sources/SourceFromSingleChunk.cpp @@ -5,9 +5,7 @@ namespace DB { -SourceFromSingleChunk::SourceFromSingleChunk(Block header, Chunk chunk_) : ISource(std::move(header)), chunk(std::move(chunk_)) -{ -} +SourceFromSingleChunk::SourceFromSingleChunk(Block header, Chunk chunk_) : ISource(std::move(header)), chunk(std::move(chunk_)) {} SourceFromSingleChunk::SourceFromSingleChunk(Block data) : ISource(data.cloneEmpty()), chunk(data.getColumns(), data.rows()) { @@ -22,7 +20,7 @@ SourceFromSingleChunk::SourceFromSingleChunk(Block data) : ISource(data.cloneEmp auto info = std::make_shared(); info->bucket_num = data.info.bucket_num; info->is_overflows = data.info.is_overflows; - chunk.getChunkInfos().add(std::move(info)); + chunk.setChunkInfo(std::move(info)); } } diff --git a/src/Processors/Transforms/AggregatingInOrderTransform.cpp b/src/Processors/Transforms/AggregatingInOrderTransform.cpp index 45b0960ec8f..9ffe15d0f85 100644 --- a/src/Processors/Transforms/AggregatingInOrderTransform.cpp +++ b/src/Processors/Transforms/AggregatingInOrderTransform.cpp @@ -332,7 +332,7 @@ void AggregatingInOrderTransform::generate() variants.aggregates_pool = variants.aggregates_pools.at(0).get(); /// Pass info about used memory by aggregate functions further. - to_push_chunk.getChunkInfos().add(std::make_shared(cur_block_bytes)); + to_push_chunk.setChunkInfo(std::make_shared(cur_block_bytes)); cur_block_bytes = 0; cur_block_size = 0; @@ -351,12 +351,11 @@ FinalizeAggregatedTransform::FinalizeAggregatedTransform(Block header, Aggregati void FinalizeAggregatedTransform::transform(Chunk & chunk) { if (params->final) - { finalizeChunk(chunk, aggregates_mask); - } - else if (!chunk.getChunkInfos().get()) + else if (!chunk.getChunkInfo()) { - chunk.getChunkInfos().add(std::make_shared()); + auto info = std::make_shared(); + chunk.setChunkInfo(std::move(info)); } } diff --git a/src/Processors/Transforms/AggregatingInOrderTransform.h b/src/Processors/Transforms/AggregatingInOrderTransform.h index 41a0d7fc7f1..5d50e97f552 100644 --- a/src/Processors/Transforms/AggregatingInOrderTransform.h +++ b/src/Processors/Transforms/AggregatingInOrderTransform.h @@ -5,7 +5,6 @@ #include #include #include -#include namespace DB { @@ -13,12 +12,10 @@ namespace DB struct InputOrderInfo; using InputOrderInfoPtr = std::shared_ptr; -struct ChunkInfoWithAllocatedBytes : public ChunkInfoCloneable +struct ChunkInfoWithAllocatedBytes : public ChunkInfo { - ChunkInfoWithAllocatedBytes(const ChunkInfoWithAllocatedBytes & other) = default; explicit ChunkInfoWithAllocatedBytes(Int64 allocated_bytes_) : allocated_bytes(allocated_bytes_) {} - Int64 allocated_bytes; }; diff --git a/src/Processors/Transforms/AggregatingTransform.cpp b/src/Processors/Transforms/AggregatingTransform.cpp index 517f035667f..65f0612d738 100644 --- a/src/Processors/Transforms/AggregatingTransform.cpp +++ b/src/Processors/Transforms/AggregatingTransform.cpp @@ -35,7 +35,7 @@ Chunk convertToChunk(const Block & block) UInt64 num_rows = block.rows(); Chunk chunk(block.getColumns(), num_rows); - chunk.getChunkInfos().add(std::move(info)); + chunk.setChunkInfo(std::move(info)); return chunk; } @@ -44,11 +44,15 @@ namespace { const AggregatedChunkInfo * getInfoFromChunk(const Chunk & chunk) { - auto agg_info = chunk.getChunkInfos().get(); + const auto & info = chunk.getChunkInfo(); + if (!info) + throw Exception(ErrorCodes::LOGICAL_ERROR, "Chunk info was not set for chunk."); + + const auto * agg_info = typeid_cast(info.get()); if (!agg_info) throw Exception(ErrorCodes::LOGICAL_ERROR, "Chunk should have AggregatedChunkInfo."); - return agg_info.get(); + return agg_info; } /// Reads chunks from file in native format. Provide chunks with aggregation info. @@ -206,7 +210,11 @@ private: void process(Chunk && chunk) { - auto chunks_to_merge = chunk.getChunkInfos().get(); + if (!chunk.hasChunkInfo()) + throw Exception(ErrorCodes::LOGICAL_ERROR, "Expected chunk with chunk info in {}", getName()); + + const auto & info = chunk.getChunkInfo(); + const auto * chunks_to_merge = typeid_cast(info.get()); if (!chunks_to_merge) throw Exception(ErrorCodes::LOGICAL_ERROR, "Expected chunk with ChunksToMerge info in {}", getName()); diff --git a/src/Processors/Transforms/AggregatingTransform.h b/src/Processors/Transforms/AggregatingTransform.h index 95983c39d1e..e167acde067 100644 --- a/src/Processors/Transforms/AggregatingTransform.h +++ b/src/Processors/Transforms/AggregatingTransform.h @@ -2,7 +2,6 @@ #include #include #include -#include #include #include #include @@ -20,7 +19,7 @@ namespace CurrentMetrics namespace DB { -class AggregatedChunkInfo : public ChunkInfoCloneable +class AggregatedChunkInfo : public ChunkInfo { public: bool is_overflows = false; diff --git a/src/Processors/Transforms/ApplySquashingTransform.h b/src/Processors/Transforms/ApplySquashingTransform.h index 49a6581e685..965a084bb13 100644 --- a/src/Processors/Transforms/ApplySquashingTransform.h +++ b/src/Processors/Transforms/ApplySquashingTransform.h @@ -27,12 +27,18 @@ public: } ExceptionKeepingTransform::work(); + if (finish_chunk) + { + data.chunk = std::move(finish_chunk); + ready_output = true; + } } protected: void onConsume(Chunk chunk) override { - cur_chunk = Squashing::squash(std::move(chunk)); + if (auto res_chunk = DB::Squashing::squash(std::move(chunk))) + cur_chunk.setColumns(res_chunk.getColumns(), res_chunk.getNumRows()); } GenerateResult onGenerate() override @@ -42,10 +48,16 @@ protected: res.is_done = true; return res; } + void onFinish() override + { + auto chunk = DB::Squashing::squash({}); + finish_chunk.setColumns(chunk.getColumns(), chunk.getNumRows()); + } private: Squashing squashing; Chunk cur_chunk; + Chunk finish_chunk; }; } diff --git a/src/Processors/Transforms/CountingTransform.cpp b/src/Processors/Transforms/CountingTransform.cpp index 2c6b3bd8638..3dfb9fe178f 100644 --- a/src/Processors/Transforms/CountingTransform.cpp +++ b/src/Processors/Transforms/CountingTransform.cpp @@ -1,7 +1,6 @@ -#include -#include #include +#include #include #include diff --git a/src/Processors/Transforms/DeduplicationTokenTransforms.cpp b/src/Processors/Transforms/DeduplicationTokenTransforms.cpp deleted file mode 100644 index 6786f76cbef..00000000000 --- a/src/Processors/Transforms/DeduplicationTokenTransforms.cpp +++ /dev/null @@ -1,236 +0,0 @@ -#include - -#include - -#include -#include -#include - - -#include - - -namespace DB -{ - -namespace ErrorCodes -{ - extern const int LOGICAL_ERROR; -} - -void RestoreChunkInfosTransform::transform(Chunk & chunk) -{ - chunk.getChunkInfos().append(chunk_infos.clone()); -} - -namespace DeduplicationToken -{ - -String TokenInfo::getToken() const -{ - if (!isDefined()) - throw Exception(ErrorCodes::LOGICAL_ERROR, "token is not defined, stage {}, token {}", stage, debugToken()); - - return getTokenImpl(); -} - -String TokenInfo::getTokenImpl() const -{ - String result; - result.reserve(getTotalSize()); - - for (const auto & part : parts) - { - if (!result.empty()) - result.append(":"); - result.append(part); - } - - return result; -} - -String TokenInfo::debugToken() const -{ - return getTokenImpl(); -} - -void TokenInfo::addChunkHash(String part) -{ - if (stage == UNDEFINED && empty()) - stage = DEFINE_SOURCE_WITH_HASHES; - - if (stage != DEFINE_SOURCE_WITH_HASHES) - throw Exception(ErrorCodes::LOGICAL_ERROR, "token is in wrong stage {}, token {}", stage, debugToken()); - - addTokenPart(std::move(part)); -} - -void TokenInfo::finishChunkHashes() -{ - if (stage == UNDEFINED && empty()) - stage = DEFINE_SOURCE_WITH_HASHES; - - if (stage != DEFINE_SOURCE_WITH_HASHES) - throw Exception(ErrorCodes::LOGICAL_ERROR, "token is in wrong stage {}, token {}", stage, debugToken()); - - stage = DEFINED; -} - -void TokenInfo::setUserToken(const String & token) -{ - if (stage == UNDEFINED && empty()) - stage = DEFINE_SOURCE_USER_TOKEN; - - if (stage != DEFINE_SOURCE_USER_TOKEN) - throw Exception(ErrorCodes::LOGICAL_ERROR, "token is in wrong stage {}, token {}", stage, debugToken()); - - addTokenPart(fmt::format("user-token-{}", token)); -} - -void TokenInfo::setSourceWithUserToken(size_t block_number) -{ - if (stage != DEFINE_SOURCE_USER_TOKEN) - throw Exception(ErrorCodes::LOGICAL_ERROR, "token is in wrong stage {}, token {}", stage, debugToken()); - - addTokenPart(fmt::format("source-number-{}", block_number)); - - stage = DEFINED; -} - -void TokenInfo::setViewID(const String & id) -{ - if (stage == DEFINED) - stage = DEFINE_VIEW; - - if (stage != DEFINE_VIEW) - throw Exception(ErrorCodes::LOGICAL_ERROR, "token is in wrong stage {}, token {}", stage, debugToken()); - - addTokenPart(fmt::format("view-id-{}", id)); -} - -void TokenInfo::setViewBlockNumber(size_t block_number) -{ - if (stage != DEFINE_VIEW) - throw Exception(ErrorCodes::LOGICAL_ERROR, "token is in wrong stage {}, token {}", stage, debugToken()); - - addTokenPart(fmt::format("view-block-{}", block_number)); - - stage = DEFINED; -} - -void TokenInfo::reset() -{ - stage = UNDEFINED; - parts.clear(); -} - -void TokenInfo::addTokenPart(String part) -{ - parts.push_back(std::move(part)); -} - -size_t TokenInfo::getTotalSize() const -{ - if (parts.empty()) - return 0; - - size_t size = 0; - for (const auto & part : parts) - size += part.size(); - - // we reserve more size here to be able to add delimenter between parts. - return size + parts.size() - 1; -} - -#ifdef ABORT_ON_LOGICAL_ERROR -void CheckTokenTransform::transform(Chunk & chunk) -{ - auto token_info = chunk.getChunkInfos().get(); - - if (!token_info) - throw Exception(ErrorCodes::LOGICAL_ERROR, "Chunk has to have DedupTokenInfo as ChunkInfo, {}", debug); - - LOG_DEBUG(log, "debug: {}, token: {}", debug, token_info->debugToken()); -} -#endif - -String DefineSourceWithChunkHashTransform::getChunkHash(const Chunk & chunk) -{ - SipHash hash; - for (const auto & colunm : chunk.getColumns()) - colunm->updateHashFast(hash); - - const auto hash_value = hash.get128(); - return toString(hash_value.items[0]) + "_" + toString(hash_value.items[1]); -} - - -void DefineSourceWithChunkHashTransform::transform(Chunk & chunk) -{ - auto token_info = chunk.getChunkInfos().get(); - - if (!token_info) - throw Exception( - ErrorCodes::LOGICAL_ERROR, - "TokenInfo is expected for consumed chunk in DefineSourceWithChunkHashesTransform"); - - if (token_info->isDefined()) - return; - - token_info->addChunkHash(getChunkHash(chunk)); - token_info->finishChunkHashes(); -} - -void SetUserTokenTransform::transform(Chunk & chunk) -{ - auto token_info = chunk.getChunkInfos().get(); - if (!token_info) - throw Exception( - ErrorCodes::LOGICAL_ERROR, - "TokenInfo is expected for consumed chunk in SetUserTokenTransform"); - token_info->setUserToken(user_token); -} - -void SetSourceBlockNumberTransform::transform(Chunk & chunk) -{ - auto token_info = chunk.getChunkInfos().get(); - if (!token_info) - throw Exception( - ErrorCodes::LOGICAL_ERROR, - "TokenInfo is expected for consumed chunk in SetSourceBlockNumberTransform"); - token_info->setSourceWithUserToken(block_number++); -} - -void SetViewIDTransform::transform(Chunk & chunk) -{ - auto token_info = chunk.getChunkInfos().get(); - if (!token_info) - throw Exception( - ErrorCodes::LOGICAL_ERROR, - "TokenInfo is expected for consumed chunk in SetViewIDTransform"); - token_info->setViewID(view_id); -} - -void SetViewBlockNumberTransform::transform(Chunk & chunk) -{ - auto token_info = chunk.getChunkInfos().get(); - if (!token_info) - throw Exception( - ErrorCodes::LOGICAL_ERROR, - "TokenInfo is expected for consumed chunk in SetViewBlockNumberTransform"); - token_info->setViewBlockNumber(block_number++); -} - -void ResetTokenTransform::transform(Chunk & chunk) -{ - auto token_info = chunk.getChunkInfos().get(); - if (!token_info) - throw Exception( - ErrorCodes::LOGICAL_ERROR, - "TokenInfo is expected for consumed chunk in ResetTokenTransform"); - - token_info->reset(); -} - -} -} diff --git a/src/Processors/Transforms/DeduplicationTokenTransforms.h b/src/Processors/Transforms/DeduplicationTokenTransforms.h deleted file mode 100644 index d6aff9e1370..00000000000 --- a/src/Processors/Transforms/DeduplicationTokenTransforms.h +++ /dev/null @@ -1,237 +0,0 @@ -#pragma once - -#include -#include - -#include -#include "Common/Logger.h" - - -namespace DB -{ - class RestoreChunkInfosTransform : public ISimpleTransform - { - public: - RestoreChunkInfosTransform(Chunk::ChunkInfoCollection chunk_infos_, const Block & header_) - : ISimpleTransform(header_, header_, true) - , chunk_infos(std::move(chunk_infos_)) - {} - - String getName() const override { return "RestoreChunkInfosTransform"; } - - void transform(Chunk & chunk) override; - - private: - Chunk::ChunkInfoCollection chunk_infos; - }; - - -namespace DeduplicationToken -{ - class TokenInfo : public ChunkInfoCloneable - { - public: - TokenInfo() = default; - TokenInfo(const TokenInfo & other) = default; - - String getToken() const; - String debugToken() const; - - bool empty() const { return parts.empty(); } - - bool isDefined() const { return stage == DEFINED; } - - void addChunkHash(String part); - void finishChunkHashes(); - - void setUserToken(const String & token); - void setSourceWithUserToken(size_t block_number); - - void setViewID(const String & id); - void setViewBlockNumber(size_t block_number); - - void reset(); - - private: - String getTokenImpl() const; - - void addTokenPart(String part); - size_t getTotalSize() const; - - /* Token has to be prepared in a particular order. - * BuildingStage ensures that token is expanded according the following order. - * Firstly token is expanded with information about the source. - * It could be done with two ways: add several hash sums from the source chunks or provide user defined deduplication token and its sequentional block number. - * - * transition // method - * UNDEFINED -> DEFINE_SOURCE_WITH_HASHES // addChunkHash - * DEFINE_SOURCE_WITH_HASHES -> DEFINE_SOURCE_WITH_HASHES // addChunkHash - * DEFINE_SOURCE_WITH_HASHES -> DEFINED // defineSourceWithChankHashes - * - * transition // method - * UNDEFINED -> DEFINE_SOURCE_USER_TOKEN // setUserToken - * DEFINE_SOURCE_USER_TOKEN -> DEFINED // defineSourceWithUserToken - * - * After token is defined, it could be extended with view id and view block number. Actually it has to be expanded with view details if there is one or several views. - * - * transition // method - * DEFINED -> DEFINE_VIEW // setViewID - * DEFINE_VIEW -> DEFINED // defineViewID - */ - - enum BuildingStage - { - UNDEFINED, - DEFINE_SOURCE_WITH_HASHES, - DEFINE_SOURCE_USER_TOKEN, - DEFINE_VIEW, - DEFINED, - }; - - BuildingStage stage = UNDEFINED; - std::vector parts; - }; - - -#ifdef ABORT_ON_LOGICAL_ERROR - /// use that class only with debug builds in CI for introspection - class CheckTokenTransform : public ISimpleTransform - { - public: - CheckTokenTransform(String debug_, const Block & header_) - : ISimpleTransform(header_, header_, true) - , debug(std::move(debug_)) - { - } - - String getName() const override { return "DeduplicationToken::CheckTokenTransform"; } - - void transform(Chunk & chunk) override; - - private: - String debug; - LoggerPtr log = getLogger("CheckInsertDeduplicationTokenTransform"); - }; -#endif - - - class AddTokenInfoTransform : public ISimpleTransform - { - public: - explicit AddTokenInfoTransform(const Block & header_) - : ISimpleTransform(header_, header_, true) - { - } - - String getName() const override { return "DeduplicationToken::AddTokenInfoTransform"; } - - void transform(Chunk & chunk) override - { - chunk.getChunkInfos().add(std::make_shared()); - } - }; - - - class DefineSourceWithChunkHashTransform : public ISimpleTransform - { - public: - explicit DefineSourceWithChunkHashTransform(const Block & header_) - : ISimpleTransform(header_, header_, true) - { - } - - String getName() const override { return "DeduplicationToken::DefineSourceWithChunkHashesTransform"; } - - // Usually MergeTreeSink/ReplicatedMergeTreeSink calls addChunkHash for the deduplication token with hashes from the parts. - // But if there is some table with different engine, we still need to define the source of the data in deduplication token - // We use that transform to define the source as a hash of entire block in deduplication token - void transform(Chunk & chunk) override; - - static String getChunkHash(const Chunk & chunk); - }; - - class ResetTokenTransform : public ISimpleTransform - { - public: - explicit ResetTokenTransform(const Block & header_) - : ISimpleTransform(header_, header_, true) - { - } - - String getName() const override { return "DeduplicationToken::ResetTokenTransform"; } - - void transform(Chunk & chunk) override; - }; - - - class SetUserTokenTransform : public ISimpleTransform - { - public: - SetUserTokenTransform(String user_token_, const Block & header_) - : ISimpleTransform(header_, header_, true) - , user_token(std::move(user_token_)) - { - } - - String getName() const override { return "DeduplicationToken::SetUserTokenTransform"; } - - void transform(Chunk & chunk) override; - - private: - String user_token; - }; - - - class SetSourceBlockNumberTransform : public ISimpleTransform - { - public: - explicit SetSourceBlockNumberTransform(const Block & header_) - : ISimpleTransform(header_, header_, true) - { - } - - String getName() const override { return "DeduplicationToken::SetSourceBlockNumberTransform"; } - - void transform(Chunk & chunk) override; - - private: - size_t block_number = 0; - }; - - - class SetViewIDTransform : public ISimpleTransform - { - public: - SetViewIDTransform(String view_id_, const Block & header_) - : ISimpleTransform(header_, header_, true) - , view_id(std::move(view_id_)) - { - } - - String getName() const override { return "DeduplicationToken::SetViewIDTransform"; } - - void transform(Chunk & chunk) override; - - private: - String view_id; - }; - - - class SetViewBlockNumberTransform : public ISimpleTransform - { - public: - explicit SetViewBlockNumberTransform(const Block & header_) - : ISimpleTransform(header_, header_, true) - { - } - - String getName() const override { return "DeduplicationToken::SetViewBlockNumberTransform"; } - - void transform(Chunk & chunk) override; - - private: - size_t block_number = 0; - }; - -} -} diff --git a/src/Processors/Transforms/ExpressionTransform.cpp b/src/Processors/Transforms/ExpressionTransform.cpp index 04fabc9a3c6..2fbd2c21b8d 100644 --- a/src/Processors/Transforms/ExpressionTransform.cpp +++ b/src/Processors/Transforms/ExpressionTransform.cpp @@ -1,7 +1,5 @@ #include #include - - namespace DB { diff --git a/src/Processors/Transforms/JoiningTransform.cpp b/src/Processors/Transforms/JoiningTransform.cpp index ca204bcb482..3e2a9462e54 100644 --- a/src/Processors/Transforms/JoiningTransform.cpp +++ b/src/Processors/Transforms/JoiningTransform.cpp @@ -365,9 +365,10 @@ IProcessor::Status DelayedJoinedBlocksWorkerTransform::prepare() return Status::Finished; } - task = data.chunk.getChunkInfos().get(); - if (!task) + if (!data.chunk.hasChunkInfo()) throw Exception(ErrorCodes::LOGICAL_ERROR, "DelayedJoinedBlocksWorkerTransform must have chunk info"); + + task = std::dynamic_pointer_cast(data.chunk.getChunkInfo()); } else { @@ -478,7 +479,7 @@ IProcessor::Status DelayedJoinedBlocksTransform::prepare() if (output.isFinished()) continue; Chunk chunk; - chunk.getChunkInfos().add(std::make_shared()); + chunk.setChunkInfo(std::make_shared()); output.push(std::move(chunk)); output.finish(); } @@ -495,7 +496,7 @@ IProcessor::Status DelayedJoinedBlocksTransform::prepare() { Chunk chunk; auto task = std::make_shared(delayed_blocks, left_delayed_stream_finished_counter); - chunk.getChunkInfos().add(std::move(task)); + chunk.setChunkInfo(task); output.push(std::move(chunk)); } delayed_blocks = nullptr; diff --git a/src/Processors/Transforms/JoiningTransform.h b/src/Processors/Transforms/JoiningTransform.h index 5f6d9d6fff2..a308af03662 100644 --- a/src/Processors/Transforms/JoiningTransform.h +++ b/src/Processors/Transforms/JoiningTransform.h @@ -1,7 +1,6 @@ #pragma once #include -#include -#include + namespace DB { @@ -112,12 +111,11 @@ private: }; -class DelayedBlocksTask : public ChunkInfoCloneable +class DelayedBlocksTask : public ChunkInfo { public: DelayedBlocksTask() = default; - DelayedBlocksTask(const DelayedBlocksTask & other) = default; explicit DelayedBlocksTask(IBlocksStreamPtr delayed_blocks_, JoiningTransform::FinishCounterPtr left_delayed_stream_finish_counter_) : delayed_blocks(std::move(delayed_blocks_)) , left_delayed_stream_finish_counter(left_delayed_stream_finish_counter_) diff --git a/src/Processors/Transforms/MaterializingTransform.cpp b/src/Processors/Transforms/MaterializingTransform.cpp index 9ae80e21a68..1eaa5458d37 100644 --- a/src/Processors/Transforms/MaterializingTransform.cpp +++ b/src/Processors/Transforms/MaterializingTransform.cpp @@ -1,7 +1,6 @@ #include #include - namespace DB { diff --git a/src/Processors/Transforms/MemoryBoundMerging.h b/src/Processors/Transforms/MemoryBoundMerging.h index d7bc320173b..607087fb39c 100644 --- a/src/Processors/Transforms/MemoryBoundMerging.h +++ b/src/Processors/Transforms/MemoryBoundMerging.h @@ -150,7 +150,11 @@ private: if (!chunk.hasRows()) return; - const auto & agg_info = chunk.getChunkInfos().get(); + const auto & info = chunk.getChunkInfo(); + if (!info) + throw Exception(ErrorCodes::LOGICAL_ERROR, "Chunk info was not set for chunk in SortingAggregatedForMemoryBoundMergingTransform."); + + const auto * agg_info = typeid_cast(info.get()); if (!agg_info) throw Exception( ErrorCodes::LOGICAL_ERROR, "Chunk should have AggregatedChunkInfo in SortingAggregatedForMemoryBoundMergingTransform."); diff --git a/src/Processors/Transforms/MergingAggregatedMemoryEfficientTransform.cpp b/src/Processors/Transforms/MergingAggregatedMemoryEfficientTransform.cpp index ea9ebb0f96e..fc40c6894bb 100644 --- a/src/Processors/Transforms/MergingAggregatedMemoryEfficientTransform.cpp +++ b/src/Processors/Transforms/MergingAggregatedMemoryEfficientTransform.cpp @@ -30,10 +30,10 @@ void GroupingAggregatedTransform::pushData(Chunks chunks, Int32 bucket, bool is_ auto info = std::make_shared(); info->bucket_num = bucket; info->is_overflows = is_overflows; - info->chunks = std::make_shared(std::move(chunks)); + info->chunks = std::make_unique(std::move(chunks)); Chunk chunk; - chunk.getChunkInfos().add(std::move(info)); + chunk.setChunkInfo(std::move(info)); output.push(std::move(chunk)); } @@ -255,10 +255,11 @@ void GroupingAggregatedTransform::addChunk(Chunk chunk, size_t input) if (!chunk.hasRows()) return; - if (chunk.getChunkInfos().empty()) + const auto & info = chunk.getChunkInfo(); + if (!info) throw Exception(ErrorCodes::LOGICAL_ERROR, "Chunk info was not set for chunk in GroupingAggregatedTransform."); - if (auto agg_info = chunk.getChunkInfos().get()) + if (const auto * agg_info = typeid_cast(info.get())) { Int32 bucket = agg_info->bucket_num; bool is_overflows = agg_info->is_overflows; @@ -274,7 +275,7 @@ void GroupingAggregatedTransform::addChunk(Chunk chunk, size_t input) last_bucket_number[input] = bucket; } } - else if (chunk.getChunkInfos().get()) + else if (typeid_cast(info.get())) { single_level_chunks.emplace_back(std::move(chunk)); } @@ -303,11 +304,7 @@ void GroupingAggregatedTransform::work() Int32 bucket = cur_block.info.bucket_num; auto chunk_info = std::make_shared(); chunk_info->bucket_num = bucket; - - auto chunk = Chunk(cur_block.getColumns(), cur_block.rows()); - chunk.getChunkInfos().add(std::move(chunk_info)); - - chunks_map[bucket].emplace_back(std::move(chunk)); + chunks_map[bucket].emplace_back(Chunk(cur_block.getColumns(), cur_block.rows(), std::move(chunk_info))); } } } @@ -322,7 +319,9 @@ MergingAggregatedBucketTransform::MergingAggregatedBucketTransform( void MergingAggregatedBucketTransform::transform(Chunk & chunk) { - auto chunks_to_merge = chunk.getChunkInfos().get(); + const auto & info = chunk.getChunkInfo(); + const auto * chunks_to_merge = typeid_cast(info.get()); + if (!chunks_to_merge) throw Exception(ErrorCodes::LOGICAL_ERROR, "MergingAggregatedSimpleTransform chunk must have ChunkInfo with type ChunksToMerge."); @@ -331,10 +330,11 @@ void MergingAggregatedBucketTransform::transform(Chunk & chunk) BlocksList blocks_list; for (auto & cur_chunk : *chunks_to_merge->chunks) { - if (cur_chunk.getChunkInfos().empty()) + const auto & cur_info = cur_chunk.getChunkInfo(); + if (!cur_info) throw Exception(ErrorCodes::LOGICAL_ERROR, "Chunk info was not set for chunk in MergingAggregatedBucketTransform."); - if (auto agg_info = cur_chunk.getChunkInfos().get()) + if (const auto * agg_info = typeid_cast(cur_info.get())) { Block block = header.cloneWithColumns(cur_chunk.detachColumns()); block.info.is_overflows = agg_info->is_overflows; @@ -342,7 +342,7 @@ void MergingAggregatedBucketTransform::transform(Chunk & chunk) blocks_list.emplace_back(std::move(block)); } - else if (cur_chunk.getChunkInfos().get()) + else if (typeid_cast(cur_info.get())) { Block block = header.cloneWithColumns(cur_chunk.detachColumns()); block.info.is_overflows = false; @@ -361,7 +361,7 @@ void MergingAggregatedBucketTransform::transform(Chunk & chunk) res_info->is_overflows = chunks_to_merge->is_overflows; res_info->bucket_num = chunks_to_merge->bucket_num; res_info->chunk_num = chunks_to_merge->chunk_num; - chunk.getChunkInfos().add(std::move(res_info)); + chunk.setChunkInfo(std::move(res_info)); auto block = params->aggregator.mergeBlocks(blocks_list, params->final, is_cancelled); @@ -405,7 +405,11 @@ bool SortingAggregatedTransform::tryPushChunk() void SortingAggregatedTransform::addChunk(Chunk chunk, size_t from_input) { - auto agg_info = chunk.getChunkInfos().get(); + const auto & info = chunk.getChunkInfo(); + if (!info) + throw Exception(ErrorCodes::LOGICAL_ERROR, "Chunk info was not set for chunk in SortingAggregatedTransform."); + + const auto * agg_info = typeid_cast(info.get()); if (!agg_info) throw Exception(ErrorCodes::LOGICAL_ERROR, "Chunk should have AggregatedChunkInfo in SortingAggregatedTransform."); diff --git a/src/Processors/Transforms/MergingAggregatedMemoryEfficientTransform.h b/src/Processors/Transforms/MergingAggregatedMemoryEfficientTransform.h index 3a3c1bd9c1e..77ee3034ffc 100644 --- a/src/Processors/Transforms/MergingAggregatedMemoryEfficientTransform.h +++ b/src/Processors/Transforms/MergingAggregatedMemoryEfficientTransform.h @@ -3,7 +3,6 @@ #include #include #include -#include #include #include #include @@ -143,9 +142,9 @@ private: void addChunk(Chunk chunk, size_t from_input); }; -struct ChunksToMerge : public ChunkInfoCloneable +struct ChunksToMerge : public ChunkInfo { - std::shared_ptr chunks; + std::unique_ptr chunks; Int32 bucket_num = -1; bool is_overflows = false; UInt64 chunk_num = 0; // chunk number in order of generation, used during memory bound merging to restore chunks order diff --git a/src/Processors/Transforms/MergingAggregatedTransform.cpp b/src/Processors/Transforms/MergingAggregatedTransform.cpp index 446e60a0b81..ad723da7527 100644 --- a/src/Processors/Transforms/MergingAggregatedTransform.cpp +++ b/src/Processors/Transforms/MergingAggregatedTransform.cpp @@ -32,10 +32,11 @@ void MergingAggregatedTransform::consume(Chunk chunk) total_input_rows += input_rows; ++total_input_blocks; - if (chunk.getChunkInfos().empty()) + const auto & info = chunk.getChunkInfo(); + if (!info) throw Exception(ErrorCodes::LOGICAL_ERROR, "Chunk info was not set for chunk in MergingAggregatedTransform."); - if (auto agg_info = chunk.getChunkInfos().get()) + if (const auto * agg_info = typeid_cast(info.get())) { /** If the remote servers used a two-level aggregation method, * then blocks will contain information about the number of the bucket. @@ -48,7 +49,7 @@ void MergingAggregatedTransform::consume(Chunk chunk) bucket_to_blocks[agg_info->bucket_num].emplace_back(std::move(block)); } - else if (chunk.getChunkInfos().get()) + else if (typeid_cast(info.get())) { auto block = getInputPort().getHeader().cloneWithColumns(chunk.getColumns()); block.info.is_overflows = false; @@ -88,8 +89,7 @@ Chunk MergingAggregatedTransform::generate() UInt64 num_rows = block.rows(); Chunk chunk(block.getColumns(), num_rows); - - chunk.getChunkInfos().add(std::move(info)); + chunk.setChunkInfo(std::move(info)); return chunk; } diff --git a/src/Processors/Transforms/PlanSquashingTransform.cpp b/src/Processors/Transforms/PlanSquashingTransform.cpp index ee4dfa6a64e..0f433165f14 100644 --- a/src/Processors/Transforms/PlanSquashingTransform.cpp +++ b/src/Processors/Transforms/PlanSquashingTransform.cpp @@ -10,20 +10,20 @@ namespace ErrorCodes } PlanSquashingTransform::PlanSquashingTransform( - Block header_, size_t min_block_size_rows, size_t min_block_size_bytes) - : IInflatingTransform(header_, header_) - , squashing(header_, min_block_size_rows, min_block_size_bytes) + const Block & header, size_t min_block_size_rows, size_t min_block_size_bytes) + : IInflatingTransform(header, header), squashing(header, min_block_size_rows, min_block_size_bytes) { } void PlanSquashingTransform::consume(Chunk chunk) { - squashed_chunk = squashing.add(std::move(chunk)); + if (Chunk current_chunk = squashing.add(std::move(chunk)); current_chunk.hasChunkInfo()) + squashed_chunk.swap(current_chunk); } Chunk PlanSquashingTransform::generate() { - if (!squashed_chunk) + if (!squashed_chunk.hasChunkInfo()) throw Exception(ErrorCodes::LOGICAL_ERROR, "Can't generate chunk in SimpleSquashingChunksTransform"); Chunk result_chunk; @@ -33,11 +33,12 @@ Chunk PlanSquashingTransform::generate() bool PlanSquashingTransform::canGenerate() { - return bool(squashed_chunk); + return squashed_chunk.hasChunkInfo(); } Chunk PlanSquashingTransform::getRemaining() { - return squashing.flush(); + Chunk current_chunk = squashing.flush(); + return current_chunk; } } diff --git a/src/Processors/Transforms/PlanSquashingTransform.h b/src/Processors/Transforms/PlanSquashingTransform.h index e6db245499e..4ad2ec2d089 100644 --- a/src/Processors/Transforms/PlanSquashingTransform.h +++ b/src/Processors/Transforms/PlanSquashingTransform.h @@ -10,7 +10,7 @@ class PlanSquashingTransform : public IInflatingTransform { public: PlanSquashingTransform( - Block header_, size_t min_block_size_rows, size_t min_block_size_bytes); + const Block & header, size_t min_block_size_rows, size_t min_block_size_bytes); String getName() const override { return "PlanSquashingTransform"; } @@ -23,6 +23,7 @@ protected: private: Squashing squashing; Chunk squashed_chunk; + Chunk finish_chunk; }; } diff --git a/src/Processors/Transforms/SelectByIndicesTransform.h b/src/Processors/Transforms/SelectByIndicesTransform.h index b44f5a3203e..480ab1a0f61 100644 --- a/src/Processors/Transforms/SelectByIndicesTransform.h +++ b/src/Processors/Transforms/SelectByIndicesTransform.h @@ -26,7 +26,7 @@ public: void transform(Chunk & chunk) override { size_t num_rows = chunk.getNumRows(); - auto select_final_indices_info = chunk.getChunkInfos().extract(); + const auto * select_final_indices_info = typeid_cast(chunk.getChunkInfo().get()); if (!select_final_indices_info || !select_final_indices_info->select_final_indices) throw Exception(ErrorCodes::LOGICAL_ERROR, "Chunk passed to SelectByIndicesTransform without indices column"); @@ -41,6 +41,7 @@ public: chunk.setColumns(std::move(columns), index_column->size()); } + chunk.setChunkInfo(nullptr); } }; diff --git a/src/Processors/Transforms/SquashingTransform.cpp b/src/Processors/Transforms/SquashingTransform.cpp index 1fb4433240a..34b733cde5e 100644 --- a/src/Processors/Transforms/SquashingTransform.cpp +++ b/src/Processors/Transforms/SquashingTransform.cpp @@ -18,7 +18,9 @@ SquashingTransform::SquashingTransform( void SquashingTransform::onConsume(Chunk chunk) { - cur_chunk = Squashing::squash(squashing.add(std::move(chunk))); + Chunk planned_chunk = squashing.add(std::move(chunk)); + if (planned_chunk.hasChunkInfo()) + cur_chunk = DB::Squashing::squash(std::move(planned_chunk)); } SquashingTransform::GenerateResult SquashingTransform::onGenerate() @@ -31,7 +33,10 @@ SquashingTransform::GenerateResult SquashingTransform::onGenerate() void SquashingTransform::onFinish() { - finish_chunk = Squashing::squash(squashing.flush()); + Chunk chunk = squashing.flush(); + if (chunk.hasChunkInfo()) + chunk = DB::Squashing::squash(std::move(chunk)); + finish_chunk.setColumns(chunk.getColumns(), chunk.getNumRows()); } void SquashingTransform::work() @@ -44,7 +49,6 @@ void SquashingTransform::work() } ExceptionKeepingTransform::work(); - if (finish_chunk) { data.chunk = std::move(finish_chunk); @@ -63,14 +67,18 @@ void SimpleSquashingTransform::transform(Chunk & chunk) { if (!finished) { - chunk = Squashing::squash(squashing.add(std::move(chunk))); + Chunk planned_chunk = squashing.add(std::move(chunk)); + if (planned_chunk.hasChunkInfo()) + chunk = DB::Squashing::squash(std::move(planned_chunk)); } else { if (chunk.hasRows()) throw Exception(ErrorCodes::LOGICAL_ERROR, "Chunk expected to be empty, otherwise it will be lost"); - chunk = Squashing::squash(squashing.flush()); + chunk = squashing.flush(); + if (chunk.hasChunkInfo()) + chunk = DB::Squashing::squash(std::move(chunk)); } } diff --git a/src/Processors/Transforms/TotalsHavingTransform.cpp b/src/Processors/Transforms/TotalsHavingTransform.cpp index 59fceccb538..aa86879e62c 100644 --- a/src/Processors/Transforms/TotalsHavingTransform.cpp +++ b/src/Processors/Transforms/TotalsHavingTransform.cpp @@ -150,7 +150,11 @@ void TotalsHavingTransform::transform(Chunk & chunk) /// Block with values not included in `max_rows_to_group_by`. We'll postpone it. if (overflow_row) { - const auto & agg_info = chunk.getChunkInfos().get(); + const auto & info = chunk.getChunkInfo(); + if (!info) + throw Exception(ErrorCodes::LOGICAL_ERROR, "Chunk info was not set for chunk in TotalsHavingTransform."); + + const auto * agg_info = typeid_cast(info.get()); if (!agg_info) throw Exception(ErrorCodes::LOGICAL_ERROR, "Chunk should have AggregatedChunkInfo in TotalsHavingTransform."); diff --git a/src/Processors/Transforms/buildPushingToViewsChain.cpp b/src/Processors/Transforms/buildPushingToViewsChain.cpp index 312b333ab33..25fbf13b0e7 100644 --- a/src/Processors/Transforms/buildPushingToViewsChain.cpp +++ b/src/Processors/Transforms/buildPushingToViewsChain.cpp @@ -5,9 +5,7 @@ #include #include #include -#include #include -#include #include #include #include @@ -18,7 +16,6 @@ #include #include #include -#include #include #include #include @@ -27,12 +24,9 @@ #include #include #include -#include "base/defines.h" -#include #include #include -#include namespace ProfileEvents @@ -111,7 +105,7 @@ private: class ExecutingInnerQueryFromViewTransform final : public ExceptionKeepingTransform { public: - ExecutingInnerQueryFromViewTransform(const Block & header, ViewRuntimeData & view_, ViewsDataPtr views_data_, bool disable_deduplication_for_children_); + ExecutingInnerQueryFromViewTransform(const Block & header, ViewRuntimeData & view_, ViewsDataPtr views_data_); String getName() const override { return "ExecutingInnerQueryFromView"; } @@ -122,7 +116,6 @@ protected: private: ViewsDataPtr views_data; ViewRuntimeData & view; - bool disable_deduplication_for_children; struct State { @@ -145,7 +138,7 @@ class PushingToLiveViewSink final : public SinkToStorage public: PushingToLiveViewSink(const Block & header, StorageLiveView & live_view_, StoragePtr storage_holder_, ContextPtr context_); String getName() const override { return "PushingToLiveViewSink"; } - void consume(Chunk & chunk) override; + void consume(Chunk chunk) override; private: StorageLiveView & live_view; @@ -159,7 +152,7 @@ class PushingToWindowViewSink final : public SinkToStorage public: PushingToWindowViewSink(const Block & header, StorageWindowView & window_view_, StoragePtr storage_holder_, ContextPtr context_); String getName() const override { return "PushingToWindowViewSink"; } - void consume(Chunk & chunk) override; + void consume(Chunk chunk) override; private: StorageWindowView & window_view; @@ -223,10 +216,45 @@ std::optional generateViewChain( const auto & insert_settings = insert_context->getSettingsRef(); + // Do not deduplicate insertions into MV if the main insertion is Ok if (disable_deduplication_for_children) { insert_context->setSetting("insert_deduplicate", Field{false}); } + else if (insert_settings.update_insert_deduplication_token_in_dependent_materialized_views && + !insert_settings.insert_deduplication_token.value.empty()) + { + /** Update deduplication token passed to dependent MV with current view id. So it is possible to properly handle + * deduplication in complex INSERT flows. + * + * Example: + * + * landing -┬--> mv_1_1 ---> ds_1_1 ---> mv_2_1 --┬-> ds_2_1 ---> mv_3_1 ---> ds_3_1 + * | | + * └--> mv_1_2 ---> ds_1_2 ---> mv_2_2 --┘ + * + * Here we want to avoid deduplication for two different blocks generated from `mv_2_1` and `mv_2_2` that will + * be inserted into `ds_2_1`. + * + * We are forced to use view id instead of table id because there are some possible INSERT flows where no tables + * are involved. + * + * Example: + * + * landing -┬--> mv_1_1 --┬-> ds_1_1 + * | | + * └--> mv_1_2 --┘ + * + */ + auto insert_deduplication_token = insert_settings.insert_deduplication_token.value; + + if (view_id.hasUUID()) + insert_deduplication_token += "_" + toString(view_id.uuid); + else + insert_deduplication_token += "_" + view_id.getFullNameNotQuoted(); + + insert_context->setSetting("insert_deduplication_token", insert_deduplication_token); + } // Processing of blocks for MVs is done block by block, and there will // be no parallel reading after (plus it is not a costless operation) @@ -333,13 +361,7 @@ std::optional generateViewChain( insert_columns.emplace_back(column.name); } - InterpreterInsertQuery interpreter( - nullptr, - insert_context, - /* allow_materialized */ false, - /* no_squash */ false, - /* no_destination */ false, - /* async_isnert */ false); + InterpreterInsertQuery interpreter(nullptr, insert_context, false, false, false); /// TODO: remove sql_security_type check after we turn `ignore_empty_sql_security_in_create_view_query=false` bool check_access = !materialized_view->hasInnerTable() && materialized_view->getInMemoryMetadataPtr()->sql_security_type; @@ -356,10 +378,6 @@ std::optional generateViewChain( table_prefers_large_blocks ? settings.min_insert_block_size_bytes : 0ULL)); } -#ifdef ABORT_ON_LOGICAL_ERROR - out.addSource(std::make_shared("Before squashing", out.getInputHeader())); -#endif - auto counting = std::make_shared(out.getInputHeader(), current_thread, insert_context->getQuota()); counting->setProcessListElement(insert_context->getProcessListElement()); counting->setProgressCallback(insert_context->getProgressCallback()); @@ -402,19 +420,11 @@ std::optional generateViewChain( if (type == QueryViewsLogElement::ViewType::MATERIALIZED) { -#ifdef ABORT_ON_LOGICAL_ERROR - out.addSource(std::make_shared("Right after Inner query", out.getInputHeader())); -#endif - auto executing_inner_query = std::make_shared( - storage_header, views_data->views.back(), views_data, disable_deduplication_for_children); + storage_header, views_data->views.back(), views_data); executing_inner_query->setRuntimeData(view_thread_status, view_counter_ms); out.addSource(std::move(executing_inner_query)); - -#ifdef ABORT_ON_LOGICAL_ERROR - out.addSource(std::make_shared("Right before Inner query", out.getInputHeader())); -#endif } return out; @@ -455,7 +465,11 @@ Chain buildPushingToViewsChain( */ result_chain.addTableLock(storage->lockForShare(context->getInitialQueryId(), context->getSettingsRef().lock_acquire_timeout)); - bool disable_deduplication_for_children = !context->getSettingsRef().deduplicate_blocks_in_dependent_materialized_views; + /// If the "root" table deduplicates blocks, there are no need to make deduplication for children + /// Moreover, deduplication for AggregatingMergeTree children could produce false positives due to low size of inserting blocks + bool disable_deduplication_for_children = false; + if (!context->getSettingsRef().deduplicate_blocks_in_dependent_materialized_views) + disable_deduplication_for_children = !no_destination && storage->supportsDeduplication(); auto table_id = storage->getStorageID(); auto views = DatabaseCatalog::instance().getDependentViews(table_id); @@ -546,25 +560,12 @@ Chain buildPushingToViewsChain( auto sink = std::make_shared(live_view_header, *live_view, storage, context); sink->setRuntimeData(thread_status, elapsed_counter_ms); result_chain.addSource(std::move(sink)); - - result_chain.addSource(std::make_shared(result_chain.getInputHeader())); } else if (auto * window_view = dynamic_cast(storage.get())) { auto sink = std::make_shared(window_view->getInputHeader(), *window_view, storage, context); sink->setRuntimeData(thread_status, elapsed_counter_ms); result_chain.addSource(std::move(sink)); - - result_chain.addSource(std::make_shared(result_chain.getInputHeader())); - } - else if (dynamic_cast(storage.get())) - { - auto sink = storage->write(query_ptr, metadata_snapshot, context, async_insert); - metadata_snapshot->check(sink->getHeader().getColumnsWithTypeAndName()); - sink->setRuntimeData(thread_status, elapsed_counter_ms); - result_chain.addSource(std::move(sink)); - - result_chain.addSource(std::make_shared(result_chain.getInputHeader())); } /// Do not push to destination table if the flag is set else if (!no_destination) @@ -572,15 +573,8 @@ Chain buildPushingToViewsChain( auto sink = storage->write(query_ptr, metadata_snapshot, context, async_insert); metadata_snapshot->check(sink->getHeader().getColumnsWithTypeAndName()); sink->setRuntimeData(thread_status, elapsed_counter_ms); - - result_chain.addSource(std::make_shared(sink->getHeader())); - result_chain.addSource(std::move(sink)); } - else - { - result_chain.addSource(std::make_shared(storage_header)); - } if (result_chain.empty()) result_chain.addSink(std::make_shared(storage_header)); @@ -596,7 +590,7 @@ Chain buildPushingToViewsChain( return result_chain; } -static QueryPipeline process(Block block, ViewRuntimeData & view, const ViewsData & views_data, Chunk::ChunkInfoCollection && chunk_infos, bool disable_deduplication_for_children) +static QueryPipeline process(Block block, ViewRuntimeData & view, const ViewsData & views_data) { const auto & context = view.context; @@ -643,19 +637,6 @@ static QueryPipeline process(Block block, ViewRuntimeData & view, const ViewsDat pipeline.getHeader(), std::make_shared(std::move(converting)))); - pipeline.addTransform(std::make_shared(std::move(chunk_infos), pipeline.getHeader())); - - if (!disable_deduplication_for_children) - { - String materialize_view_id = view.table_id.hasUUID() ? toString(view.table_id.uuid) : view.table_id.getFullNameNotQuoted(); - pipeline.addTransform(std::make_shared(std::move(materialize_view_id), pipeline.getHeader())); - pipeline.addTransform(std::make_shared(pipeline.getHeader())); - } - else - { - pipeline.addTransform(std::make_shared(pipeline.getHeader())); - } - return QueryPipelineBuilder::getPipeline(std::move(pipeline)); } @@ -747,19 +728,17 @@ IProcessor::Status CopyingDataToViewsTransform::prepare() ExecutingInnerQueryFromViewTransform::ExecutingInnerQueryFromViewTransform( const Block & header, ViewRuntimeData & view_, - std::shared_ptr views_data_, - bool disable_deduplication_for_children_) + std::shared_ptr views_data_) : ExceptionKeepingTransform(header, view_.sample_block) , views_data(std::move(views_data_)) , view(view_) - , disable_deduplication_for_children(disable_deduplication_for_children_) { } void ExecutingInnerQueryFromViewTransform::onConsume(Chunk chunk) { - auto block = getInputPort().getHeader().cloneWithColumns(chunk.detachColumns()); - state.emplace(process(std::move(block), view, *views_data, std::move(chunk.getChunkInfos()), disable_deduplication_for_children)); + auto block = getInputPort().getHeader().cloneWithColumns(chunk.getColumns()); + state.emplace(process(block, view, *views_data)); } @@ -791,10 +770,10 @@ PushingToLiveViewSink::PushingToLiveViewSink(const Block & header, StorageLiveVi { } -void PushingToLiveViewSink::consume(Chunk & chunk) +void PushingToLiveViewSink::consume(Chunk chunk) { Progress local_progress(chunk.getNumRows(), chunk.bytes(), 0); - live_view.writeBlock(live_view, getHeader().cloneWithColumns(chunk.detachColumns()), std::move(chunk.getChunkInfos()), context); + live_view.writeBlock(getHeader().cloneWithColumns(chunk.detachColumns()), context); if (auto process = context->getProcessListElement()) process->updateProgressIn(local_progress); @@ -814,11 +793,11 @@ PushingToWindowViewSink::PushingToWindowViewSink( { } -void PushingToWindowViewSink::consume(Chunk & chunk) +void PushingToWindowViewSink::consume(Chunk chunk) { Progress local_progress(chunk.getNumRows(), chunk.bytes(), 0); StorageWindowView::writeIntoWindowView( - window_view, getHeader().cloneWithColumns(chunk.detachColumns()), std::move(chunk.getChunkInfos()), context); + window_view, getHeader().cloneWithColumns(chunk.detachColumns()), context); if (auto process = context->getProcessListElement()) process->updateProgressIn(local_progress); diff --git a/src/QueryPipeline/QueryPipelineBuilder.h b/src/QueryPipeline/QueryPipelineBuilder.h index a9e5b1535c0..f0b2ead687e 100644 --- a/src/QueryPipeline/QueryPipelineBuilder.h +++ b/src/QueryPipeline/QueryPipelineBuilder.h @@ -193,7 +193,7 @@ public: return concurrency_control; } - void addResources(QueryPlanResourceHolder resources_) { resources.append(std::move(resources_)); } + void addResources(QueryPlanResourceHolder resources_) { resources = std::move(resources_); } void setQueryIdHolder(std::shared_ptr query_id_holder) { resources.query_id_holders.emplace_back(std::move(query_id_holder)); } void addContext(ContextPtr context) { resources.interpreter_context.emplace_back(std::move(context)); } diff --git a/src/QueryPipeline/QueryPlanResourceHolder.cpp b/src/QueryPipeline/QueryPlanResourceHolder.cpp index bb2be2c8ffb..2cd4dc42a83 100644 --- a/src/QueryPipeline/QueryPlanResourceHolder.cpp +++ b/src/QueryPipeline/QueryPlanResourceHolder.cpp @@ -5,7 +5,7 @@ namespace DB { -QueryPlanResourceHolder & QueryPlanResourceHolder::append(QueryPlanResourceHolder && rhs) noexcept +QueryPlanResourceHolder & QueryPlanResourceHolder::operator=(QueryPlanResourceHolder && rhs) noexcept { table_locks.insert(table_locks.end(), rhs.table_locks.begin(), rhs.table_locks.end()); storage_holders.insert(storage_holders.end(), rhs.storage_holders.begin(), rhs.storage_holders.end()); @@ -16,12 +16,6 @@ QueryPlanResourceHolder & QueryPlanResourceHolder::append(QueryPlanResourceHolde return *this; } -QueryPlanResourceHolder & QueryPlanResourceHolder::operator=(QueryPlanResourceHolder && rhs) noexcept -{ - append(std::move(rhs)); - return *this; -} - QueryPlanResourceHolder::QueryPlanResourceHolder() = default; QueryPlanResourceHolder::QueryPlanResourceHolder(QueryPlanResourceHolder &&) noexcept = default; QueryPlanResourceHolder::~QueryPlanResourceHolder() = default; diff --git a/src/QueryPipeline/QueryPlanResourceHolder.h b/src/QueryPipeline/QueryPlanResourceHolder.h index 10f7f39ab09..ed9eb68b7ba 100644 --- a/src/QueryPipeline/QueryPlanResourceHolder.h +++ b/src/QueryPipeline/QueryPlanResourceHolder.h @@ -20,11 +20,8 @@ struct QueryPlanResourceHolder QueryPlanResourceHolder(QueryPlanResourceHolder &&) noexcept; ~QueryPlanResourceHolder(); - QueryPlanResourceHolder & operator=(QueryPlanResourceHolder &) = delete; - /// Custom move assignment does not destroy data from lhs. It appends data from rhs to lhs. QueryPlanResourceHolder & operator=(QueryPlanResourceHolder &&) noexcept; - QueryPlanResourceHolder & append(QueryPlanResourceHolder &&) noexcept; /// Some processors may implicitly use Context or temporary Storage created by Interpreter. /// But lifetime of Streams is not nested in lifetime of Interpreters, so we have to store it here, diff --git a/src/Server/TCPHandler.cpp b/src/Server/TCPHandler.cpp index fccea9e258e..ac1423f87c1 100644 --- a/src/Server/TCPHandler.cpp +++ b/src/Server/TCPHandler.cpp @@ -888,11 +888,12 @@ AsynchronousInsertQueue::PushResult TCPHandler::processAsyncInsertQuery(Asynchro while (readDataNext()) { - squashing.setHeader(state.block_for_insert.cloneEmpty()); - auto result_chunk = Squashing::squash(squashing.add({state.block_for_insert.getColumns(), state.block_for_insert.rows()})); - if (result_chunk) + squashing.header = state.block_for_insert; + auto planned_chunk = squashing.add({state.block_for_insert.getColumns(), state.block_for_insert.rows()}); + if (planned_chunk.hasChunkInfo()) { - auto result = squashing.getHeader().cloneWithColumns(result_chunk.detachColumns()); + Chunk result_chunk = DB::Squashing::squash(std::move(planned_chunk)); + auto result = state.block_for_insert.cloneWithColumns(result_chunk.getColumns()); return PushResult { .status = PushResult::TOO_MUCH_DATA, @@ -901,13 +902,12 @@ AsynchronousInsertQueue::PushResult TCPHandler::processAsyncInsertQuery(Asynchro } } - Chunk result_chunk = Squashing::squash(squashing.flush()); - if (!result_chunk) - { - return insert_queue.pushQueryWithBlock(state.parsed_query, squashing.getHeader(), query_context); - } + auto planned_chunk = squashing.flush(); + Chunk result_chunk; + if (planned_chunk.hasChunkInfo()) + result_chunk = DB::Squashing::squash(std::move(planned_chunk)); - auto result = squashing.getHeader().cloneWithColumns(result_chunk.detachColumns()); + auto result = squashing.header.cloneWithColumns(result_chunk.getColumns()); return insert_queue.pushQueryWithBlock(state.parsed_query, std::move(result), query_context); } diff --git a/src/Storages/Distributed/DistributedSink.cpp b/src/Storages/Distributed/DistributedSink.cpp index 8791668cd89..e556bda2561 100644 --- a/src/Storages/Distributed/DistributedSink.cpp +++ b/src/Storages/Distributed/DistributedSink.cpp @@ -134,7 +134,7 @@ DistributedSink::DistributedSink( } -void DistributedSink::consume(Chunk & chunk) +void DistributedSink::consume(Chunk chunk) { if (is_first_chunk) { @@ -142,7 +142,7 @@ void DistributedSink::consume(Chunk & chunk) is_first_chunk = false; } - auto ordinary_block = getHeader().cloneWithColumns(chunk.getColumns()); + auto ordinary_block = getHeader().cloneWithColumns(chunk.detachColumns()); if (insert_sync) writeSync(ordinary_block); @@ -420,13 +420,7 @@ DistributedSink::runWritingJob(JobReplica & job, const Block & current_block, si /// to resolve tables (in InterpreterInsertQuery::getTable()) auto copy_query_ast = query_ast->clone(); - InterpreterInsertQuery interp( - copy_query_ast, - job.local_context, - allow_materialized, - /* no_squash */ false, - /* no_destination */ false, - /* async_isnert */ false); + InterpreterInsertQuery interp(copy_query_ast, job.local_context, allow_materialized); auto block_io = interp.execute(); job.pipeline = std::move(block_io.pipeline); @@ -721,13 +715,7 @@ void DistributedSink::writeToLocal(const Cluster::ShardInfo & shard_info, const try { - InterpreterInsertQuery interp( - query_ast, - context, - allow_materialized, - /* no_squash */ false, - /* no_destination */ false, - /* async_isnert */ false); + InterpreterInsertQuery interp(query_ast, context, allow_materialized); auto block_io = interp.execute(); PushingPipelineExecutor executor(block_io.pipeline); diff --git a/src/Storages/Distributed/DistributedSink.h b/src/Storages/Distributed/DistributedSink.h index 5b7396f2c6f..a4c95633595 100644 --- a/src/Storages/Distributed/DistributedSink.h +++ b/src/Storages/Distributed/DistributedSink.h @@ -49,7 +49,7 @@ public: const Names & columns_to_send_); String getName() const override { return "DistributedSink"; } - void consume(Chunk & chunk) override; + void consume(Chunk chunk) override; void onFinish() override; private: diff --git a/src/Storages/FileLog/StorageFileLog.cpp b/src/Storages/FileLog/StorageFileLog.cpp index 0f9bd8b6ff9..abd4b4ce23b 100644 --- a/src/Storages/FileLog/StorageFileLog.cpp +++ b/src/Storages/FileLog/StorageFileLog.cpp @@ -740,14 +740,7 @@ bool StorageFileLog::streamToViews() auto new_context = Context::createCopy(getContext()); - InterpreterInsertQuery interpreter( - insert, - new_context, - /* allow_materialized */ false, - /* no_squash */ true, - /* no_destination */ true, - /* async_isnert */ false); - + InterpreterInsertQuery interpreter(insert, new_context, false, true, true); auto block_io = interpreter.execute(); /// Each stream responsible for closing it's files and store meta diff --git a/src/Storages/Kafka/StorageKafka.cpp b/src/Storages/Kafka/StorageKafka.cpp index 809401bb279..f5c5d093ce1 100644 --- a/src/Storages/Kafka/StorageKafka.cpp +++ b/src/Storages/Kafka/StorageKafka.cpp @@ -1099,13 +1099,7 @@ bool StorageKafka::streamToViews() // Create a stream for each consumer and join them in a union stream // Only insert into dependent views and expect that input blocks contain virtual columns - InterpreterInsertQuery interpreter( - insert, - kafka_context, - /* allow_materialized */ false, - /* no_squash */ true, - /* no_destination */ true, - /* async_isnert */ false); + InterpreterInsertQuery interpreter(insert, kafka_context, false, true, true); auto block_io = interpreter.execute(); // Create a stream for each consumer and join them in a union stream diff --git a/src/Storages/LiveView/LiveViewSink.h b/src/Storages/LiveView/LiveViewSink.h index 9803fa0a160..792133ced64 100644 --- a/src/Storages/LiveView/LiveViewSink.h +++ b/src/Storages/LiveView/LiveViewSink.h @@ -71,9 +71,9 @@ public: new_hash.reset(); } - void consume(Chunk & chunk) override + void consume(Chunk chunk) override { - auto block = getHeader().cloneWithColumns(chunk.getColumns()); + auto block = getHeader().cloneWithColumns(chunk.detachColumns()); block.updateHash(*new_hash); new_blocks->push_back(std::move(block)); } diff --git a/src/Storages/LiveView/StorageLiveView.cpp b/src/Storages/LiveView/StorageLiveView.cpp index 82759e8a851..57a1ea302f9 100644 --- a/src/Storages/LiveView/StorageLiveView.cpp +++ b/src/Storages/LiveView/StorageLiveView.cpp @@ -21,7 +21,6 @@ limitations under the License. */ #include #include #include -#include #include #include #include @@ -331,7 +330,7 @@ Pipe StorageLiveView::watch( return reader; } -void StorageLiveView::writeBlock(StorageLiveView & live_view, Block && block, Chunk::ChunkInfoCollection && chunk_infos, ContextPtr local_context) +void StorageLiveView::writeBlock(const Block & block, ContextPtr local_context) { auto output = std::make_shared(*this); @@ -408,21 +407,6 @@ void StorageLiveView::writeBlock(StorageLiveView & live_view, Block && block, Ch builder = interpreter.buildQueryPipeline(); } - builder.addSimpleTransform([&](const Block & cur_header) - { - return std::make_shared(chunk_infos.clone(), cur_header); - }); - - String live_view_id = live_view.getStorageID().hasUUID() ? toString(live_view.getStorageID().uuid) : live_view.getStorageID().getFullNameNotQuoted(); - builder.addSimpleTransform([&](const Block & stream_header) - { - return std::make_shared(live_view_id, stream_header); - }); - builder.addSimpleTransform([&](const Block & stream_header) - { - return std::make_shared(stream_header); - }); - builder.addSimpleTransform([&](const Block & cur_header) { return std::make_shared(cur_header); diff --git a/src/Storages/LiveView/StorageLiveView.h b/src/Storages/LiveView/StorageLiveView.h index 12d8e898347..91daac32c7b 100644 --- a/src/Storages/LiveView/StorageLiveView.h +++ b/src/Storages/LiveView/StorageLiveView.h @@ -118,7 +118,7 @@ public: return 0; } - void writeBlock(StorageLiveView & live_view, Block && block, Chunk::ChunkInfoCollection && chunk_infos, ContextPtr context); + void writeBlock(const Block & block, ContextPtr context); void refresh(); diff --git a/src/Storages/MaterializedView/RefreshTask.cpp b/src/Storages/MaterializedView/RefreshTask.cpp index ff5214a5e51..bc8cb0ce69a 100644 --- a/src/Storages/MaterializedView/RefreshTask.cpp +++ b/src/Storages/MaterializedView/RefreshTask.cpp @@ -377,13 +377,7 @@ void RefreshTask::executeRefreshUnlocked(std::shared_ptr(task->getInfo().data_part->info.level)); - return ChunkAndProgress{ - .chunk = std::move(chunk), + .chunk = Chunk(ordered_columns, res.row_count, add_part_level ? std::make_shared(task->getInfo().data_part->info.level) : nullptr), .num_read_rows = res.num_read_rows, .num_read_bytes = res.num_read_bytes, .is_finished = false}; diff --git a/src/Storages/MergeTree/MergeTreeSequentialSource.cpp b/src/Storages/MergeTree/MergeTreeSequentialSource.cpp index 4f90f7131da..02f8d6f4f6a 100644 --- a/src/Storages/MergeTree/MergeTreeSequentialSource.cpp +++ b/src/Storages/MergeTree/MergeTreeSequentialSource.cpp @@ -264,10 +264,7 @@ try ++it; } - auto result = Chunk(std::move(res_columns), rows_read); - if (add_part_level) - result.getChunkInfos().add(std::make_shared(data_part->info.level)); - return result; + return Chunk(std::move(res_columns), rows_read, add_part_level ? std::make_shared(data_part->info.level) : nullptr); } } else diff --git a/src/Storages/MergeTree/MergeTreeSink.cpp b/src/Storages/MergeTree/MergeTreeSink.cpp index d8cfce1ca99..05751e0fa6f 100644 --- a/src/Storages/MergeTree/MergeTreeSink.cpp +++ b/src/Storages/MergeTree/MergeTreeSink.cpp @@ -1,27 +1,14 @@ -#include -#include -#include -#include -#include -#include -#include -#include -#include #include #include - -#include +#include +#include +#include namespace ProfileEvents { extern const Event DuplicatedInsertedBlocks; } -namespace ErrorCodes -{ - extern const int LOGICAL_ERROR; -} - namespace DB { @@ -71,12 +58,12 @@ void MergeTreeSink::onCancel() { } -void MergeTreeSink::consume(Chunk & chunk) +void MergeTreeSink::consume(Chunk chunk) { if (num_blocks_processed > 0) storage.delayInsertOrThrowIfNeeded(nullptr, context, false); - auto block = getHeader().cloneWithColumns(chunk.getColumns()); + auto block = getHeader().cloneWithColumns(chunk.detachColumns()); if (!storage_snapshot->object_columns.empty()) convertDynamicColumnsToTuples(block, storage_snapshot); @@ -89,18 +76,6 @@ void MergeTreeSink::consume(Chunk & chunk) size_t streams = 0; bool support_parallel_write = false; - auto token_info = chunk.getChunkInfos().get(); - if (!token_info) - throw Exception(ErrorCodes::LOGICAL_ERROR, - "TokenInfo is expected for consumed chunk in MergeTreeSink for table: {}", - storage.getStorageID().getNameForLogs()); - - const bool need_to_define_dedup_token = !token_info->isDefined(); - - String block_dedup_token; - if (token_info->isDefined()) - block_dedup_token = token_info->getToken(); - for (auto & current_block : part_blocks) { ProfileEvents::Counters part_counters; @@ -125,16 +100,22 @@ void MergeTreeSink::consume(Chunk & chunk) if (!temp_part.part) continue; - if (need_to_define_dedup_token) - { - chassert(temp_part.part); - const auto hash_value = temp_part.part->getPartBlockIDHash(); - token_info->addChunkHash(toString(hash_value.items[0]) + "_" + toString(hash_value.items[1])); - } - if (!support_parallel_write && temp_part.part->getDataPartStorage().supportParallelWrite()) support_parallel_write = true; + String block_dedup_token; + if (storage.getDeduplicationLog()) + { + const String & dedup_token = settings.insert_deduplication_token; + if (!dedup_token.empty()) + { + /// multiple blocks can be inserted within the same insert query + /// an ordinal number is added to dedup token to generate a distinctive block id for each block + block_dedup_token = fmt::format("{}_{}", dedup_token, chunk_dedup_seqnum); + ++chunk_dedup_seqnum; + } + } + size_t max_insert_delayed_streams_for_parallel_write; if (settings.max_insert_delayed_streams_for_parallel_write.changed) @@ -146,7 +127,6 @@ void MergeTreeSink::consume(Chunk & chunk) /// In case of too much columns/parts in block, flush explicitly. streams += temp_part.streams.size(); - if (streams > max_insert_delayed_streams_for_parallel_write) { finishDelayedChunk(); @@ -163,16 +143,11 @@ void MergeTreeSink::consume(Chunk & chunk) { .temp_part = std::move(temp_part), .elapsed_ns = elapsed_ns, - .block_dedup_token = block_dedup_token, + .block_dedup_token = std::move(block_dedup_token), .part_counters = std::move(part_counters), }); } - if (need_to_define_dedup_token) - { - token_info->finishChunkHashes(); - } - finishDelayedChunk(); delayed_chunk = std::make_unique(); delayed_chunk->partitions = std::move(partitions); @@ -185,8 +160,6 @@ void MergeTreeSink::finishDelayedChunk() if (!delayed_chunk) return; - const Settings & settings = context->getSettingsRef(); - for (auto & partition : delayed_chunk->partitions) { ProfileEventsScope scoped_attach(&partition.part_counters); @@ -205,8 +178,7 @@ void MergeTreeSink::finishDelayedChunk() storage.fillNewPartName(part, lock); auto * deduplication_log = storage.getDeduplicationLog(); - - if (settings.insert_deduplicate && deduplication_log) + if (deduplication_log) { const String block_id = part->getZeroLevelPartBlockID(partition.block_dedup_token); auto res = deduplication_log->addPart(block_id, part->info); diff --git a/src/Storages/MergeTree/MergeTreeSink.h b/src/Storages/MergeTree/MergeTreeSink.h index 90976020d52..cf6715a3415 100644 --- a/src/Storages/MergeTree/MergeTreeSink.h +++ b/src/Storages/MergeTree/MergeTreeSink.h @@ -25,7 +25,7 @@ public: ~MergeTreeSink() override; String getName() const override { return "MergeTreeSink"; } - void consume(Chunk & chunk) override; + void consume(Chunk chunk) override; void onStart() override; void onFinish() override; void onCancel() override; @@ -36,6 +36,7 @@ private: size_t max_parts_per_block; ContextPtr context; StorageSnapshotPtr storage_snapshot; + UInt64 chunk_dedup_seqnum = 0; /// input chunk ordinal number in case of dedup token UInt64 num_blocks_processed = 0; /// We can delay processing for previous chunk and start writing a new one. diff --git a/src/Storages/MergeTree/MutateTask.cpp b/src/Storages/MergeTree/MutateTask.cpp index 3dbcb5e5bda..a552ee89aee 100644 --- a/src/Storages/MergeTree/MutateTask.cpp +++ b/src/Storages/MergeTree/MutateTask.cpp @@ -1297,7 +1297,6 @@ void PartMergerWriter::prepare() bool PartMergerWriter::mutateOriginalPartAndPrepareProjections() { Block cur_block; - Block projection_header; if (MutationHelpers::checkOperationIsNotCanceled(*ctx->merges_blocker, ctx->mutate_entry) && ctx->mutating_executor->pull(cur_block)) { if (ctx->minmax_idx) @@ -1315,12 +1314,14 @@ bool PartMergerWriter::mutateOriginalPartAndPrepareProjections() ProfileEventTimeIncrement watch(ProfileEvents::MutateTaskProjectionsCalculationMicroseconds); Block block_to_squash = projection.calculate(cur_block, ctx->context); - projection_squashes[i].setHeader(block_to_squash.cloneEmpty()); + projection_squashes[i].header = block_to_squash; + Chunk planned_chunk = projection_squashes[i].add({block_to_squash.getColumns(), block_to_squash.rows()}); - Chunk squashed_chunk = Squashing::squash(projection_squashes[i].add({block_to_squash.getColumns(), block_to_squash.rows()})); - if (squashed_chunk) + if (planned_chunk.hasChunkInfo()) { - auto result = projection_squashes[i].getHeader().cloneWithColumns(squashed_chunk.detachColumns()); + Chunk projection_chunk = DB::Squashing::squash(std::move(planned_chunk)); + + auto result = block_to_squash.cloneWithColumns(projection_chunk.getColumns()); auto tmp_part = MergeTreeDataWriter::writeTempProjectionPart( *ctx->data, ctx->log, result, projection, ctx->new_data_part.get(), ++block_num); tmp_part.finalize(); @@ -1341,10 +1342,12 @@ bool PartMergerWriter::mutateOriginalPartAndPrepareProjections() { const auto & projection = *ctx->projections_to_build[i]; auto & projection_squash_plan = projection_squashes[i]; - auto squashed_chunk = Squashing::squash(projection_squash_plan.flush()); - if (squashed_chunk) + auto planned_chunk = projection_squash_plan.flush(); + if (planned_chunk.hasChunkInfo()) { - auto result = projection_squash_plan.getHeader().cloneWithColumns(squashed_chunk.detachColumns()); + Chunk projection_chunk = DB::Squashing::squash(std::move(planned_chunk)); + + auto result = projection_squash_plan.header.cloneWithColumns(projection_chunk.getColumns()); auto temp_part = MergeTreeDataWriter::writeTempProjectionPart( *ctx->data, ctx->log, result, projection, ctx->new_data_part.get(), ++block_num); temp_part.finalize(); diff --git a/src/Storages/MergeTree/ReplicatedMergeTreeSink.cpp b/src/Storages/MergeTree/ReplicatedMergeTreeSink.cpp index bbae054fbed..4b4f4c33e7d 100644 --- a/src/Storages/MergeTree/ReplicatedMergeTreeSink.cpp +++ b/src/Storages/MergeTree/ReplicatedMergeTreeSink.cpp @@ -1,25 +1,21 @@ +#include +#include +#include +#include +#include #include "Common/Exception.h" #include #include #include -#include #include -#include -#include -#include -#include -#include -#include -#include +#include #include #include -#include -#include -#include - +#include +#include +#include +#include #include -#include - namespace ProfileEvents { @@ -257,12 +253,12 @@ size_t ReplicatedMergeTreeSinkImpl::checkQuorumPrecondition(const } template -void ReplicatedMergeTreeSinkImpl::consume(Chunk & chunk) +void ReplicatedMergeTreeSinkImpl::consume(Chunk chunk) { if (num_blocks_processed > 0) storage.delayInsertOrThrowIfNeeded(&storage.partial_shutdown_event, context, false); - auto block = getHeader().cloneWithColumns(chunk.getColumns()); + auto block = getHeader().cloneWithColumns(chunk.detachColumns()); const auto & settings = context->getSettingsRef(); @@ -288,25 +284,13 @@ void ReplicatedMergeTreeSinkImpl::consume(Chunk & chunk) if constexpr (async_insert) { - const auto async_insert_info_ptr = chunk.getChunkInfos().get(); - if (async_insert_info_ptr) + const auto & chunk_info = chunk.getChunkInfo(); + if (const auto * async_insert_info_ptr = typeid_cast(chunk_info.get())) async_insert_info = std::make_shared(async_insert_info_ptr->offsets, async_insert_info_ptr->tokens); else throw Exception(ErrorCodes::LOGICAL_ERROR, "No chunk info for async inserts"); } - String block_dedup_token; - auto token_info = chunk.getChunkInfos().get(); - if (!token_info) - throw Exception(ErrorCodes::LOGICAL_ERROR, - "TokenInfo is expected for consumed chunk in ReplicatedMergeTreeSink for table: {}", - storage.getStorageID().getNameForLogs()); - - const bool need_to_define_dedup_token = !token_info->isDefined(); - - if (token_info->isDefined()) - block_dedup_token = token_info->getToken(); - auto part_blocks = MergeTreeDataWriter::splitBlockIntoParts(std::move(block), max_parts_per_block, metadata_snapshot, context, async_insert_info); using DelayedPartition = typename ReplicatedMergeTreeSinkImpl::DelayedChunk::Partition; @@ -358,10 +342,23 @@ void ReplicatedMergeTreeSinkImpl::consume(Chunk & chunk) } else { + if (deduplicate) { + String block_dedup_token; + /// We add the hash from the data and partition identifier to deduplication ID. /// That is, do not insert the same data to the same partition twice. + + const String & dedup_token = settings.insert_deduplication_token; + if (!dedup_token.empty()) + { + /// multiple blocks can be inserted within the same insert query + /// an ordinal number is added to dedup token to generate a distinctive block id for each block + block_dedup_token = fmt::format("{}_{}", dedup_token, chunk_dedup_seqnum); + ++chunk_dedup_seqnum; + } + block_id = temp_part.part->getZeroLevelPartBlockID(block_dedup_token); LOG_DEBUG(log, "Wrote block with ID '{}', {} rows{}", block_id, current_block.block.rows(), quorumLogMessage(replicas_num)); } @@ -369,13 +366,6 @@ void ReplicatedMergeTreeSinkImpl::consume(Chunk & chunk) { LOG_DEBUG(log, "Wrote block with {} rows{}", current_block.block.rows(), quorumLogMessage(replicas_num)); } - - if (need_to_define_dedup_token) - { - chassert(temp_part.part); - const auto hash_value = temp_part.part->getPartBlockIDHash(); - token_info->addChunkHash(toString(hash_value.items[0]) + "_" + toString(hash_value.items[1])); - } } profile_events_scope.reset(); @@ -421,15 +411,17 @@ void ReplicatedMergeTreeSinkImpl::consume(Chunk & chunk) )); } - if (need_to_define_dedup_token) - { - token_info->finishChunkHashes(); - } - finishDelayedChunk(zookeeper); delayed_chunk = std::make_unique(); delayed_chunk->partitions = std::move(partitions); + /// If deduplicated data should not be inserted into MV, we need to set proper + /// value for `last_block_is_duplicate`, which is possible only after the part is committed. + /// Othervide we can delay commit. + /// TODO: we can also delay commit if there is no MVs. + if (!settings.deduplicate_blocks_in_dependent_materialized_views) + finishDelayedChunk(zookeeper); + ++num_blocks_processed; } @@ -439,6 +431,8 @@ void ReplicatedMergeTreeSinkImpl::finishDelayedChunk(const ZooKeeperWithF if (!delayed_chunk) return; + last_block_is_duplicate = false; + for (auto & partition : delayed_chunk->partitions) { ProfileEventsScope scoped_attach(&partition.part_counters); @@ -451,6 +445,8 @@ void ReplicatedMergeTreeSinkImpl::finishDelayedChunk(const ZooKeeperWithF { bool deduplicated = commitPart(zookeeper, part, partition.block_id, delayed_chunk->replicas_num).second; + last_block_is_duplicate = last_block_is_duplicate || deduplicated; + /// Set a special error code if the block is duplicate int error = (deduplicate && deduplicated) ? ErrorCodes::INSERT_WAS_DEDUPLICATED : 0; auto counters_snapshot = std::make_shared(partition.part_counters.getPartiallyAtomicSnapshot()); @@ -539,7 +535,7 @@ bool ReplicatedMergeTreeSinkImpl::writeExistingPart(MergeTreeData::Mutabl ProfileEventsScope profile_events_scope; String original_part_dir = part->getDataPartStorage().getPartDirectory(); - auto try_rollback_part_rename = [this, &part, &original_part_dir] () + auto try_rollback_part_rename = [this, &part, &original_part_dir]() { if (original_part_dir == part->getDataPartStorage().getPartDirectory()) return; @@ -1155,16 +1151,8 @@ void ReplicatedMergeTreeSinkImpl::onStart() template void ReplicatedMergeTreeSinkImpl::onFinish() { - const auto & settings = context->getSettingsRef(); - - ZooKeeperWithFaultInjectionPtr zookeeper = ZooKeeperWithFaultInjection::createInstance( - settings.insert_keeper_fault_injection_probability, - settings.insert_keeper_fault_injection_seed, - storage.getZooKeeper(), - "ReplicatedMergeTreeSink::onFinish", - log); - - finishDelayedChunk(zookeeper); + auto zookeeper = storage.getZooKeeper(); + finishDelayedChunk(std::make_shared(zookeeper)); } template diff --git a/src/Storages/MergeTree/ReplicatedMergeTreeSink.h b/src/Storages/MergeTree/ReplicatedMergeTreeSink.h index 7d025361717..39623c20584 100644 --- a/src/Storages/MergeTree/ReplicatedMergeTreeSink.h +++ b/src/Storages/MergeTree/ReplicatedMergeTreeSink.h @@ -51,7 +51,7 @@ public: ~ReplicatedMergeTreeSinkImpl() override; void onStart() override; - void consume(Chunk & chunk) override; + void consume(Chunk chunk) override; void onFinish() override; String getName() const override { return "ReplicatedMergeTreeSink"; } @@ -59,6 +59,16 @@ public: /// For ATTACHing existing data on filesystem. bool writeExistingPart(MergeTreeData::MutableDataPartPtr & part); + /// For proper deduplication in MaterializedViews + bool lastBlockIsDuplicate() const override + { + /// If MV is responsible for deduplication, block is not considered duplicating. + if (context->getSettingsRef().deduplicate_blocks_in_dependent_materialized_views) + return false; + + return last_block_is_duplicate; + } + struct DelayedChunk; private: std::vector detectConflictsInAsyncBlockIDs(const std::vector & ids); @@ -116,6 +126,7 @@ private: bool allow_attach_while_readonly = false; bool quorum_parallel = false; const bool deduplicate = true; + bool last_block_is_duplicate = false; UInt64 num_blocks_processed = 0; LoggerPtr log; diff --git a/src/Storages/MessageQueueSink.cpp b/src/Storages/MessageQueueSink.cpp index 36899011e33..4fb81d69070 100644 --- a/src/Storages/MessageQueueSink.cpp +++ b/src/Storages/MessageQueueSink.cpp @@ -40,7 +40,7 @@ void MessageQueueSink::onFinish() producer->finish(); } -void MessageQueueSink::consume(Chunk & chunk) +void MessageQueueSink::consume(Chunk chunk) { const auto & columns = chunk.getColumns(); if (columns.empty()) diff --git a/src/Storages/MessageQueueSink.h b/src/Storages/MessageQueueSink.h index 4a9248c6c4d..b3c1e61734f 100644 --- a/src/Storages/MessageQueueSink.h +++ b/src/Storages/MessageQueueSink.h @@ -35,7 +35,7 @@ public: String getName() const override { return storage_name + "Sink"; } - void consume(Chunk & chunk) override; + void consume(Chunk chunk) override; void onStart() override; void onFinish() override; diff --git a/src/Storages/NATS/StorageNATS.cpp b/src/Storages/NATS/StorageNATS.cpp index 8f0e2d76473..0b88a9e8929 100644 --- a/src/Storages/NATS/StorageNATS.cpp +++ b/src/Storages/NATS/StorageNATS.cpp @@ -644,13 +644,7 @@ bool StorageNATS::streamToViews() insert->table_id = table_id; // Only insert into dependent views and expect that input blocks contain virtual columns - InterpreterInsertQuery interpreter( - insert, - nats_context, - /* allow_materialized */ false, - /* no_squash */ true, - /* no_destination */ true, - /* async_isnert */ false); + InterpreterInsertQuery interpreter(insert, nats_context, false, true, true); auto block_io = interpreter.execute(); auto storage_snapshot = getStorageSnapshot(getInMemoryMetadataPtr(), getContext()); diff --git a/src/Storages/ObjectStorage/StorageObjectStorageSink.cpp b/src/Storages/ObjectStorage/StorageObjectStorageSink.cpp index d2bdd0af302..f2f6eac333c 100644 --- a/src/Storages/ObjectStorage/StorageObjectStorageSink.cpp +++ b/src/Storages/ObjectStorage/StorageObjectStorageSink.cpp @@ -39,12 +39,12 @@ StorageObjectStorageSink::StorageObjectStorageSink( configuration->format, *write_buf, sample_block, context, format_settings_); } -void StorageObjectStorageSink::consume(Chunk & chunk) +void StorageObjectStorageSink::consume(Chunk chunk) { std::lock_guard lock(cancel_mutex); if (cancelled) return; - writer->write(getHeader().cloneWithColumns(chunk.getColumns())); + writer->write(getHeader().cloneWithColumns(chunk.detachColumns())); } void StorageObjectStorageSink::onCancel() diff --git a/src/Storages/ObjectStorage/StorageObjectStorageSink.h b/src/Storages/ObjectStorage/StorageObjectStorageSink.h index 6ab531bb21a..e0081193686 100644 --- a/src/Storages/ObjectStorage/StorageObjectStorageSink.h +++ b/src/Storages/ObjectStorage/StorageObjectStorageSink.h @@ -20,7 +20,7 @@ public: String getName() const override { return "StorageObjectStorageSink"; } - void consume(Chunk & chunk) override; + void consume(Chunk chunk) override; void onCancel() override; diff --git a/src/Storages/ObjectStorageQueue/StorageObjectStorageQueue.cpp b/src/Storages/ObjectStorageQueue/StorageObjectStorageQueue.cpp index 14b828e7268..4388864434e 100644 --- a/src/Storages/ObjectStorageQueue/StorageObjectStorageQueue.cpp +++ b/src/Storages/ObjectStorageQueue/StorageObjectStorageQueue.cpp @@ -454,13 +454,7 @@ bool StorageObjectStorageQueue::streamToViews() while (!shutdown_called && !file_iterator->isFinished()) { - InterpreterInsertQuery interpreter( - insert, - queue_context, - /* allow_materialized */ false, - /* no_squash */ true, - /* no_destination */ true, - /* async_isnert */ false); + InterpreterInsertQuery interpreter(insert, queue_context, false, true, true); auto block_io = interpreter.execute(); auto read_from_format_info = prepareReadingFromFormat( block_io.pipeline.getHeader().getNames(), diff --git a/src/Storages/PartitionedSink.cpp b/src/Storages/PartitionedSink.cpp index ee2570756ed..09b009b26d8 100644 --- a/src/Storages/PartitionedSink.cpp +++ b/src/Storages/PartitionedSink.cpp @@ -51,7 +51,7 @@ SinkPtr PartitionedSink::getSinkForPartitionKey(StringRef partition_key) return it->second; } -void PartitionedSink::consume(Chunk & chunk) +void PartitionedSink::consume(Chunk chunk) { const auto & columns = chunk.getColumns(); @@ -104,7 +104,7 @@ void PartitionedSink::consume(Chunk & chunk) for (const auto & [partition_key, partition_index] : partition_id_to_chunk_index) { auto sink = getSinkForPartitionKey(partition_key); - sink->consume(partition_index_to_chunk[partition_index]); + sink->consume(std::move(partition_index_to_chunk[partition_index])); } } diff --git a/src/Storages/PartitionedSink.h b/src/Storages/PartitionedSink.h index fcd67556dc9..68edeb6fd73 100644 --- a/src/Storages/PartitionedSink.h +++ b/src/Storages/PartitionedSink.h @@ -20,7 +20,7 @@ public: String getName() const override { return "PartitionedSink"; } - void consume(Chunk & chunk) override; + void consume(Chunk chunk) override; void onException(std::exception_ptr exception) override; diff --git a/src/Storages/PostgreSQL/MaterializedPostgreSQLConsumer.cpp b/src/Storages/PostgreSQL/MaterializedPostgreSQLConsumer.cpp index 44479bd01e2..ba3cc6f58d0 100644 --- a/src/Storages/PostgreSQL/MaterializedPostgreSQLConsumer.cpp +++ b/src/Storages/PostgreSQL/MaterializedPostgreSQLConsumer.cpp @@ -697,13 +697,7 @@ void MaterializedPostgreSQLConsumer::syncTables() insert->table_id = storage->getStorageID(); insert->columns = std::make_shared(buffer->columns_ast); - InterpreterInsertQuery interpreter( - insert, - insert_context, - /* allow_materialized */ true, - /* no_squash */ false, - /* no_destination */ false, - /* async_isnert */ false); + InterpreterInsertQuery interpreter(insert, insert_context, true); auto io = interpreter.execute(); auto input = std::make_shared( result_rows.cloneEmpty(), Chunk(result_rows.getColumns(), result_rows.rows())); diff --git a/src/Storages/PostgreSQL/PostgreSQLReplicationHandler.cpp b/src/Storages/PostgreSQL/PostgreSQLReplicationHandler.cpp index f632e553a0d..2bb1e2dde0d 100644 --- a/src/Storages/PostgreSQL/PostgreSQLReplicationHandler.cpp +++ b/src/Storages/PostgreSQL/PostgreSQLReplicationHandler.cpp @@ -437,13 +437,7 @@ StorageInfo PostgreSQLReplicationHandler::loadFromSnapshot(postgres::Connection auto insert_context = materialized_storage->getNestedTableContext(); - InterpreterInsertQuery interpreter( - insert, - insert_context, - /* allow_materialized */ false, - /* no_squash */ false, - /* no_destination */ false, - /* async_isnert */ false); + InterpreterInsertQuery interpreter(insert, insert_context); auto block_io = interpreter.execute(); const StorageInMemoryMetadata & storage_metadata = nested_storage->getInMemoryMetadata(); diff --git a/src/Storages/RabbitMQ/StorageRabbitMQ.cpp b/src/Storages/RabbitMQ/StorageRabbitMQ.cpp index f3d2aff68c8..e4b19992151 100644 --- a/src/Storages/RabbitMQ/StorageRabbitMQ.cpp +++ b/src/Storages/RabbitMQ/StorageRabbitMQ.cpp @@ -1129,13 +1129,7 @@ bool StorageRabbitMQ::tryStreamToViews() } // Only insert into dependent views and expect that input blocks contain virtual columns - InterpreterInsertQuery interpreter( - insert, - rabbitmq_context, - /* allow_materialized */ false, - /* no_squash */ true, - /* no_destination */ true, - /* async_isnert */ false); + InterpreterInsertQuery interpreter(insert, rabbitmq_context, /* allow_materialized_ */ false, /* no_squash_ */ true, /* no_destination_ */ true); auto block_io = interpreter.execute(); block_io.pipeline.complete(Pipe::unitePipes(std::move(pipes))); diff --git a/src/Storages/RocksDB/EmbeddedRocksDBBulkSink.cpp b/src/Storages/RocksDB/EmbeddedRocksDBBulkSink.cpp index 4b5188ca9f2..90792c59d38 100644 --- a/src/Storages/RocksDB/EmbeddedRocksDBBulkSink.cpp +++ b/src/Storages/RocksDB/EmbeddedRocksDBBulkSink.cpp @@ -218,7 +218,7 @@ std::pair EmbeddedRocksDBBulkSink::seriali return {std::move(serialized_key_column), std::move(serialized_value_column)}; } -void EmbeddedRocksDBBulkSink::consume(Chunk & chunk_) +void EmbeddedRocksDBBulkSink::consume(Chunk chunk_) { std::vector chunks_to_write = squash(std::move(chunk_)); @@ -247,10 +247,7 @@ void EmbeddedRocksDBBulkSink::onFinish() { /// If there is any data left, write it. if (!chunks.empty()) - { - Chunk empty; - consume(empty); - } + consume({}); } String EmbeddedRocksDBBulkSink::getTemporarySSTFilePath() diff --git a/src/Storages/RocksDB/EmbeddedRocksDBBulkSink.h b/src/Storages/RocksDB/EmbeddedRocksDBBulkSink.h index 64190c8c86f..1f548e7813d 100644 --- a/src/Storages/RocksDB/EmbeddedRocksDBBulkSink.h +++ b/src/Storages/RocksDB/EmbeddedRocksDBBulkSink.h @@ -32,7 +32,7 @@ public: ~EmbeddedRocksDBBulkSink() override; - void consume(Chunk & chunk) override; + void consume(Chunk chunk) override; void onFinish() override; diff --git a/src/Storages/RocksDB/EmbeddedRocksDBSink.cpp b/src/Storages/RocksDB/EmbeddedRocksDBSink.cpp index 1f7f6939f40..c451cfd1bf5 100644 --- a/src/Storages/RocksDB/EmbeddedRocksDBSink.cpp +++ b/src/Storages/RocksDB/EmbeddedRocksDBSink.cpp @@ -29,7 +29,7 @@ EmbeddedRocksDBSink::EmbeddedRocksDBSink( serializations = getHeader().getSerializations(); } -void EmbeddedRocksDBSink::consume(Chunk & chunk) +void EmbeddedRocksDBSink::consume(Chunk chunk) { auto rows = chunk.getNumRows(); const auto & columns = chunk.getColumns(); diff --git a/src/Storages/RocksDB/EmbeddedRocksDBSink.h b/src/Storages/RocksDB/EmbeddedRocksDBSink.h index 2e1e0c7b429..011322df829 100644 --- a/src/Storages/RocksDB/EmbeddedRocksDBSink.h +++ b/src/Storages/RocksDB/EmbeddedRocksDBSink.h @@ -17,7 +17,7 @@ public: StorageEmbeddedRocksDB & storage_, const StorageMetadataPtr & metadata_snapshot_); - void consume(Chunk & chunk) override; + void consume(Chunk chunk) override; String getName() const override { return "EmbeddedRocksDBSink"; } private: diff --git a/src/Storages/RocksDB/StorageEmbeddedRocksDB.cpp b/src/Storages/RocksDB/StorageEmbeddedRocksDB.cpp index 3473166a080..b9d3e071b6c 100644 --- a/src/Storages/RocksDB/StorageEmbeddedRocksDB.cpp +++ b/src/Storages/RocksDB/StorageEmbeddedRocksDB.cpp @@ -313,8 +313,7 @@ void StorageEmbeddedRocksDB::mutate(const MutationCommands & commands, ContextPt Block block; while (executor.pull(block)) { - auto chunk = Chunk(block.getColumns(), block.rows()); - sink->consume(chunk); + sink->consume(Chunk{block.getColumns(), block.rows()}); } } diff --git a/src/Storages/StorageBuffer.cpp b/src/Storages/StorageBuffer.cpp index b064fba223a..a3f6b6afc5d 100644 --- a/src/Storages/StorageBuffer.cpp +++ b/src/Storages/StorageBuffer.cpp @@ -607,7 +607,7 @@ public: String getName() const override { return "BufferSink"; } - void consume(Chunk & chunk) override + void consume(Chunk chunk) override { size_t rows = chunk.getNumRows(); if (!rows) @@ -1020,13 +1020,7 @@ void StorageBuffer::writeBlockToDestination(const Block & block, StoragePtr tabl auto insert_context = Context::createCopy(getContext()); insert_context->makeQueryContext(); - InterpreterInsertQuery interpreter( - insert, - insert_context, - allow_materialized, - /* no_squash */ false, - /* no_destination */ false, - /* async_isnert */ false); + InterpreterInsertQuery interpreter{insert, insert_context, allow_materialized}; auto block_io = interpreter.execute(); PushingPipelineExecutor executor(block_io.pipeline); diff --git a/src/Storages/StorageDistributed.cpp b/src/Storages/StorageDistributed.cpp index 67586985ce8..849fa5dbe0b 100644 --- a/src/Storages/StorageDistributed.cpp +++ b/src/Storages/StorageDistributed.cpp @@ -1050,13 +1050,7 @@ std::optional StorageDistributed::distributedWriteBetweenDistribu const auto & shard_info = shards_info[shard_index]; if (shard_info.isLocal()) { - InterpreterInsertQuery interpreter( - new_query, - query_context, - /* allow_materialized */ false, - /* no_squash */ false, - /* no_destination */ false, - /* async_isnert */ false); + InterpreterInsertQuery interpreter(new_query, query_context); pipeline.addCompletedPipeline(interpreter.execute().pipeline); } else diff --git a/src/Storages/StorageFile.cpp b/src/Storages/StorageFile.cpp index 3fb397c7b81..7f39ff615f0 100644 --- a/src/Storages/StorageFile.cpp +++ b/src/Storages/StorageFile.cpp @@ -1778,12 +1778,12 @@ public: String getName() const override { return "StorageFileSink"; } - void consume(Chunk & chunk) override + void consume(Chunk chunk) override { std::lock_guard cancel_lock(cancel_mutex); if (cancelled) return; - writer->write(getHeader().cloneWithColumns(chunk.getColumns())); + writer->write(getHeader().cloneWithColumns(chunk.detachColumns())); } void onCancel() override diff --git a/src/Storages/StorageKeeperMap.cpp b/src/Storages/StorageKeeperMap.cpp index c80e799a92b..20f99070000 100644 --- a/src/Storages/StorageKeeperMap.cpp +++ b/src/Storages/StorageKeeperMap.cpp @@ -119,10 +119,10 @@ public: std::string getName() const override { return "StorageKeeperMapSink"; } - void consume(Chunk & chunk) override + void consume(Chunk chunk) override { auto rows = chunk.getNumRows(); - auto block = getHeader().cloneWithColumns(chunk.getColumns()); + auto block = getHeader().cloneWithColumns(chunk.detachColumns()); WriteBufferFromOwnString wb_key; WriteBufferFromOwnString wb_value; @@ -1248,10 +1248,7 @@ void StorageKeeperMap::mutate(const MutationCommands & commands, ContextPtr loca Block block; while (executor.pull(block)) - { - auto chunk = Chunk(block.getColumns(), block.rows()); - sink->consume(chunk); - } + sink->consume(Chunk{block.getColumns(), block.rows()}); sink->finalize(strict); } diff --git a/src/Storages/StorageLog.cpp b/src/Storages/StorageLog.cpp index 463694c63aa..de0324d7998 100644 --- a/src/Storages/StorageLog.cpp +++ b/src/Storages/StorageLog.cpp @@ -1,6 +1,5 @@ #include #include -#include #include #include @@ -22,6 +21,7 @@ #include #include +#include "StorageLogSettings.h" #include #include #include @@ -341,7 +341,7 @@ public: } } - void consume(Chunk & chunk) override; + void consume(Chunk chunk) override; void onFinish() override; private: @@ -398,9 +398,9 @@ private: }; -void LogSink::consume(Chunk & chunk) +void LogSink::consume(Chunk chunk) { - auto block = getHeader().cloneWithColumns(chunk.getColumns()); + auto block = getHeader().cloneWithColumns(chunk.detachColumns()); metadata_snapshot->check(block, true); for (auto & stream : streams | boost::adaptors::map_values) diff --git a/src/Storages/StorageMemory.cpp b/src/Storages/StorageMemory.cpp index b1bd7053c2e..f69c4adb552 100644 --- a/src/Storages/StorageMemory.cpp +++ b/src/Storages/StorageMemory.cpp @@ -63,7 +63,7 @@ public: String getName() const override { return "MemorySink"; } - void consume(Chunk & chunk) override + void consume(Chunk chunk) override { auto block = getHeader().cloneWithColumns(chunk.getColumns()); storage_snapshot->metadata->check(block, true); diff --git a/src/Storages/StorageMongoDB.cpp b/src/Storages/StorageMongoDB.cpp index e0818fafae9..62a2a048642 100644 --- a/src/Storages/StorageMongoDB.cpp +++ b/src/Storages/StorageMongoDB.cpp @@ -17,6 +17,7 @@ #include #include #include +#include #include @@ -106,12 +107,12 @@ public: String getName() const override { return "StorageMongoDBSink"; } - void consume(Chunk & chunk) override + void consume(Chunk chunk) override { Poco::MongoDB::Database db(db_name); Poco::MongoDB::Document::Vector documents; - auto block = getHeader().cloneWithColumns(chunk.getColumns()); + auto block = getHeader().cloneWithColumns(chunk.detachColumns()); size_t num_rows = block.rows(); size_t num_cols = block.columns(); diff --git a/src/Storages/StorageMySQL.cpp b/src/Storages/StorageMySQL.cpp index 2a8a7bd2ee7..da391909dff 100644 --- a/src/Storages/StorageMySQL.cpp +++ b/src/Storages/StorageMySQL.cpp @@ -151,9 +151,9 @@ public: String getName() const override { return "StorageMySQLSink"; } - void consume(Chunk & chunk) override + void consume(Chunk chunk) override { - auto block = getHeader().cloneWithColumns(chunk.getColumns()); + auto block = getHeader().cloneWithColumns(chunk.detachColumns()); auto blocks = splitBlocks(block, max_batch_rows); mysqlxx::Transaction trans(entry); try diff --git a/src/Storages/StoragePostgreSQL.cpp b/src/Storages/StoragePostgreSQL.cpp index cdfeab62b58..a8713c61e4d 100644 --- a/src/Storages/StoragePostgreSQL.cpp +++ b/src/Storages/StoragePostgreSQL.cpp @@ -227,9 +227,9 @@ public: String getName() const override { return "PostgreSQLSink"; } - void consume(Chunk & chunk) override + void consume(Chunk chunk) override { - auto block = getHeader().cloneWithColumns(chunk.getColumns()); + auto block = getHeader().cloneWithColumns(chunk.detachColumns()); if (!inserter) { if (on_conflict.empty()) diff --git a/src/Storages/StorageRedis.cpp b/src/Storages/StorageRedis.cpp index 1a275320f43..83bb3c606c9 100644 --- a/src/Storages/StorageRedis.cpp +++ b/src/Storages/StorageRedis.cpp @@ -147,7 +147,7 @@ class RedisSink : public SinkToStorage public: RedisSink(StorageRedis & storage_, const StorageMetadataPtr & metadata_snapshot_); - void consume(Chunk & chunk) override; + void consume(Chunk chunk) override; String getName() const override { return "RedisSink"; } private: @@ -169,10 +169,10 @@ RedisSink::RedisSink(StorageRedis & storage_, const StorageMetadataPtr & metadat } } -void RedisSink::consume(Chunk & chunk) +void RedisSink::consume(Chunk chunk) { auto rows = chunk.getNumRows(); - auto block = getHeader().cloneWithColumns(chunk.getColumns()); + auto block = getHeader().cloneWithColumns(chunk.detachColumns()); WriteBufferFromOwnString wb_key; WriteBufferFromOwnString wb_value; @@ -567,8 +567,7 @@ void StorageRedis::mutate(const MutationCommands & commands, ContextPtr context_ Block block; while (executor.pull(block)) { - Chunk chunk(block.getColumns(), block.rows()); - sink->consume(chunk); + sink->consume(Chunk{block.getColumns(), block.rows()}); } } diff --git a/src/Storages/StorageSQLite.cpp b/src/Storages/StorageSQLite.cpp index 85417a2f2a4..179e4cee199 100644 --- a/src/Storages/StorageSQLite.cpp +++ b/src/Storages/StorageSQLite.cpp @@ -141,7 +141,7 @@ public: String getName() const override { return "SQLiteSink"; } - void consume(Chunk & chunk) override + void consume(Chunk chunk) override { auto block = getHeader().cloneWithColumns(chunk.getColumns()); WriteBufferFromOwnString sqlbuf; diff --git a/src/Storages/StorageSet.cpp b/src/Storages/StorageSet.cpp index 0d094c15880..5b7f9fc0ac2 100644 --- a/src/Storages/StorageSet.cpp +++ b/src/Storages/StorageSet.cpp @@ -44,7 +44,7 @@ public: const String & backup_file_name_, bool persistent_); String getName() const override { return "SetOrJoinSink"; } - void consume(Chunk & chunk) override; + void consume(Chunk chunk) override; void onFinish() override; private: @@ -82,9 +82,9 @@ SetOrJoinSink::SetOrJoinSink( { } -void SetOrJoinSink::consume(Chunk & chunk) +void SetOrJoinSink::consume(Chunk chunk) { - Block block = getHeader().cloneWithColumns(chunk.getColumns()); + Block block = getHeader().cloneWithColumns(chunk.detachColumns()); table.insertBlock(block, getContext()); if (persistent) diff --git a/src/Storages/StorageStripeLog.cpp b/src/Storages/StorageStripeLog.cpp index 9b6d9f041e1..8df87d6290f 100644 --- a/src/Storages/StorageStripeLog.cpp +++ b/src/Storages/StorageStripeLog.cpp @@ -226,9 +226,9 @@ public: } } - void consume(Chunk & chunk) override + void consume(Chunk chunk) override { - block_out->write(getHeader().cloneWithColumns(chunk.getColumns())); + block_out->write(getHeader().cloneWithColumns(chunk.detachColumns())); } void onFinish() override diff --git a/src/Storages/StorageURL.cpp b/src/Storages/StorageURL.cpp index 90e05c44e31..895da028fc2 100644 --- a/src/Storages/StorageURL.cpp +++ b/src/Storages/StorageURL.cpp @@ -565,12 +565,12 @@ StorageURLSink::StorageURLSink( } -void StorageURLSink::consume(Chunk & chunk) +void StorageURLSink::consume(Chunk chunk) { std::lock_guard lock(cancel_mutex); if (cancelled) return; - writer->write(getHeader().cloneWithColumns(chunk.getColumns())); + writer->write(getHeader().cloneWithColumns(chunk.detachColumns())); } void StorageURLSink::onCancel() diff --git a/src/Storages/StorageURL.h b/src/Storages/StorageURL.h index 1804079e75f..fa7cc6eeeef 100644 --- a/src/Storages/StorageURL.h +++ b/src/Storages/StorageURL.h @@ -251,7 +251,7 @@ public: const String & method = Poco::Net::HTTPRequest::HTTP_POST); std::string getName() const override { return "StorageURLSink"; } - void consume(Chunk & chunk) override; + void consume(Chunk chunk) override; void onCancel() override; void onException(std::exception_ptr exception) override; void onFinish() override; diff --git a/src/Storages/System/StorageSystemZooKeeper.cpp b/src/Storages/System/StorageSystemZooKeeper.cpp index c9c606de049..cb46cd19517 100644 --- a/src/Storages/System/StorageSystemZooKeeper.cpp +++ b/src/Storages/System/StorageSystemZooKeeper.cpp @@ -119,7 +119,7 @@ public: ZooKeeperSink(const Block & header, ContextPtr context) : SinkToStorage(header), zookeeper(context->getZooKeeper()) { } String getName() const override { return "ZooKeeperSink"; } - void consume(Chunk & chunk) override + void consume(Chunk chunk) override { auto block = getHeader().cloneWithColumns(chunk.getColumns()); size_t rows = block.rows(); diff --git a/src/Storages/WindowView/StorageWindowView.cpp b/src/Storages/WindowView/StorageWindowView.cpp index e36247103c7..77e6ee9cb24 100644 --- a/src/Storages/WindowView/StorageWindowView.cpp +++ b/src/Storages/WindowView/StorageWindowView.cpp @@ -32,7 +32,6 @@ #include #include #include -#include #include #include #include @@ -305,7 +304,7 @@ namespace public: explicit AddingAggregatedChunkInfoTransform(Block header) : ISimpleTransform(header, header, false) { } - void transform(Chunk & chunk) override { chunk.getChunkInfos().add(std::make_shared()); } + void transform(Chunk & chunk) override { chunk.setChunkInfo(std::make_shared()); } String getName() const override { return "AddingAggregatedChunkInfoTransform"; } }; @@ -690,13 +689,7 @@ inline void StorageWindowView::fire(UInt32 watermark) StoragePtr target_table = getTargetTable(); auto insert = std::make_shared(); insert->table_id = target_table->getStorageID(); - InterpreterInsertQuery interpreter( - insert, - getContext(), - /* allow_materialized */ false, - /* no_squash */ false, - /* no_destination */ false, - /* async_isnert */ false); + InterpreterInsertQuery interpreter(insert, getContext()); auto block_io = interpreter.execute(); auto pipe = Pipe(std::make_shared(blocks, header)); @@ -1420,7 +1413,7 @@ void StorageWindowView::eventTimeParser(const ASTCreateQuery & query) } void StorageWindowView::writeIntoWindowView( - StorageWindowView & window_view, Block && block, Chunk::ChunkInfoCollection && chunk_infos, ContextPtr local_context) + StorageWindowView & window_view, const Block & block, ContextPtr local_context) { window_view.throwIfWindowViewIsDisabled(local_context); while (window_view.modifying_query) @@ -1435,7 +1428,7 @@ void StorageWindowView::writeIntoWindowView( window_view.max_watermark = window_view.getWindowUpperBound(first_record_timestamp); } - Pipe pipe(std::make_shared(block)); + Pipe pipe(std::make_shared(block.cloneEmpty(), Chunk(block.getColumns(), block.rows()))); UInt32 lateness_bound = 0; UInt32 t_max_watermark = 0; @@ -1480,10 +1473,10 @@ void StorageWindowView::writeIntoWindowView( auto syntax_result = TreeRewriter(local_context).analyze(query, columns); auto filter_expression = ExpressionAnalyzer(filter_function, syntax_result, local_context).getActionsDAG(false); - pipe.addSimpleTransform([&](const Block & header_) + pipe.addSimpleTransform([&](const Block & header) { return std::make_shared( - header_, std::make_shared(filter_expression), + header, std::make_shared(filter_expression), filter_function->getColumnName(), true); }); } @@ -1538,30 +1531,6 @@ void StorageWindowView::writeIntoWindowView( QueryProcessingStage::WithMergeableState); builder = select_block.buildQueryPipeline(); - - builder.addSimpleTransform([&](const Block & stream_header) - { - // Can't move chunk_infos here, that function could be called several times - return std::make_shared(chunk_infos.clone(), stream_header); - }); - - String window_view_id = window_view.getStorageID().hasUUID() ? toString(window_view.getStorageID().uuid) : window_view.getStorageID().getFullNameNotQuoted(); - builder.addSimpleTransform([&](const Block & stream_header) - { - return std::make_shared(window_view_id, stream_header); - }); - builder.addSimpleTransform([&](const Block & stream_header) - { - return std::make_shared(stream_header); - }); - -#ifdef ABORT_ON_LOGICAL_ERROR - builder.addSimpleTransform([&](const Block & stream_header) - { - return std::make_shared("StorageWindowView: Afrer tmp table before squashing", stream_header); - }); -#endif - builder.addSimpleTransform([&](const Block & current_header) { return std::make_shared( @@ -1601,13 +1570,6 @@ void StorageWindowView::writeIntoWindowView( lateness_upper_bound); }); -#ifdef ABORT_ON_LOGICAL_ERROR - builder.addSimpleTransform([&](const Block & stream_header) - { - return std::make_shared("StorageWindowView: Afrer WatermarkTransform", stream_header); - }); -#endif - auto inner_table = window_view.getInnerTable(); auto lock = inner_table->lockForShare( local_context->getCurrentQueryId(), local_context->getSettingsRef().lock_acquire_timeout); @@ -1624,16 +1586,9 @@ void StorageWindowView::writeIntoWindowView( auto convert_actions = std::make_shared( convert_actions_dag, ExpressionActionsSettings::fromContext(local_context, CompileExpressions::yes)); - builder.addSimpleTransform([&](const Block & header_) { return std::make_shared(header_, convert_actions); }); + builder.addSimpleTransform([&](const Block & header) { return std::make_shared(header, convert_actions); }); } -#ifdef ABORT_ON_LOGICAL_ERROR - builder.addSimpleTransform([&](const Block & stream_header) - { - return std::make_shared("StorageWindowView: Before out", stream_header); - }); -#endif - builder.addChain(Chain(std::move(output))); builder.setSinks([&](const Block & cur_header, Pipe::StreamType) { diff --git a/src/Storages/WindowView/StorageWindowView.h b/src/Storages/WindowView/StorageWindowView.h index 14ac65091d3..f79867df424 100644 --- a/src/Storages/WindowView/StorageWindowView.h +++ b/src/Storages/WindowView/StorageWindowView.h @@ -166,7 +166,7 @@ public: BlockIO populate(); - static void writeIntoWindowView(StorageWindowView & window_view, Block && block, Chunk::ChunkInfoCollection && chunk_infos, ContextPtr context); + static void writeIntoWindowView(StorageWindowView & window_view, const Block & block, ContextPtr context); ASTPtr getMergeableQuery() const { return mergeable_query->clone(); } diff --git a/tests/integration/test_force_deduplication/test.py b/tests/integration/test_force_deduplication/test.py index 14c11bc8500..87b2c45bbc5 100644 --- a/tests/integration/test_force_deduplication/test.py +++ b/tests/integration/test_force_deduplication/test.py @@ -29,8 +29,6 @@ def get_counts(): def test_basic(start_cluster): - old_src, old_a, old_b, old_c = 0, 0, 0, 0 - node.query( """ CREATE TABLE test (A Int64) ENGINE = ReplicatedMergeTree ('/clickhouse/test/tables/test','1') ORDER BY tuple(); @@ -41,15 +39,6 @@ def test_basic(start_cluster): INSERT INTO test values(999); """ ) - - src, a, b, c = get_counts() - assert src == old_src + 1 - assert a == old_a + 2 - assert b == old_b + 2 - assert c == old_c + 2 - old_src, old_a, old_b, old_c = src, a, b, c - - # that issert fails on test_mv_b due to partitions by A with pytest.raises(QueryRuntimeException): node.query( """ @@ -57,51 +46,34 @@ def test_basic(start_cluster): INSERT INTO test SELECT number FROM numbers(10); """ ) - src, a, b, c = get_counts() - assert src == old_src + 10 - assert a == old_a + 10 - assert b == old_b - assert c == old_c + 10 - old_src, old_a, old_b, old_c = src, a, b, c - # deduplication only for src table + old_src, old_a, old_b, old_c = get_counts() + # number of rows in test_mv_a and test_mv_c depends on order of inserts into views + assert old_src == 11 + assert old_a in (1, 11) + assert old_b == 1 + assert old_c in (1, 11) + node.query("INSERT INTO test SELECT number FROM numbers(10)") src, a, b, c = get_counts() - assert src == old_src - assert a == old_a + 10 - assert b == old_b + 10 - assert c == old_c + 10 - old_src, old_a, old_b, old_c = src, a, b, c - - # deduplication for MV tables does not work, because previous inserts have not written their deduplications tokens to the log due to `deduplicate_blocks_in_dependent_materialized_views = 0`. - node.query( - """ - SET deduplicate_blocks_in_dependent_materialized_views = 1; - INSERT INTO test SELECT number FROM numbers(10); - """ - ) - src, a, b, c = get_counts() - assert src == old_src - assert a == old_a + 10 - assert b == old_b + 10 - assert c == old_c + 10 - old_src, old_a, old_b, old_c = src, a, b, c - - # deduplication for all the tables - node.query( - """ - SET deduplicate_blocks_in_dependent_materialized_views = 1; - INSERT INTO test SELECT number FROM numbers(10); - """ - ) - src, a, b, c = get_counts() + # no changes because of deduplication in source table assert src == old_src assert a == old_a assert b == old_b assert c == old_c - old_src, old_a, old_b, old_c = src, a, b, c - # that issert fails on test_mv_b due to partitions by A, it is an uniq data which is not deduplicated + node.query( + """ + SET deduplicate_blocks_in_dependent_materialized_views = 1; + INSERT INTO test SELECT number FROM numbers(10); + """ + ) + src, a, b, c = get_counts() + assert src == 11 + assert a == old_a + 10 # first insert could be succesfull with disabled dedup + assert b == 11 + assert c == old_c + 10 + with pytest.raises(QueryRuntimeException): node.query( """ @@ -110,23 +82,16 @@ def test_basic(start_cluster): INSERT INTO test SELECT number FROM numbers(100,10); """ ) - src, a, b, c = get_counts() - assert src == old_src + 10 - assert a == old_a + 10 - assert b == old_b - assert c == old_c + 10 - old_src, old_a, old_b, old_c = src, a, b, c - # deduplication for all tables, except test_mv_b. For test_mv_b it is an uniq data which is not deduplicated due to exception at previous insert node.query( """ SET deduplicate_blocks_in_dependent_materialized_views = 1; INSERT INTO test SELECT number FROM numbers(100,10); """ ) + src, a, b, c = get_counts() - assert src == old_src - assert a == old_a - assert b == old_b + 10 - assert c == old_c - old_src, old_a, old_b, old_c = src, a, b, c + assert src == 21 + assert a == old_a + 20 + assert b == 21 + assert c == old_c + 20 diff --git a/tests/queries/0_stateless/00510_materizlized_view_and_deduplication_zookeeper.reference b/tests/queries/0_stateless/00510_materizlized_view_and_deduplication_zookeeper.reference index 9c9281dc7e4..adf6abb7298 100644 --- a/tests/queries/0_stateless/00510_materizlized_view_and_deduplication_zookeeper.reference +++ b/tests/queries/0_stateless/00510_materizlized_view_and_deduplication_zookeeper.reference @@ -1,7 +1,7 @@ 2 3 -3 +2 3 1 diff --git a/tests/queries/0_stateless/00510_materizlized_view_and_deduplication_zookeeper.sql b/tests/queries/0_stateless/00510_materizlized_view_and_deduplication_zookeeper.sql index 51e6a513608..d3c4da86b41 100644 --- a/tests/queries/0_stateless/00510_materizlized_view_and_deduplication_zookeeper.sql +++ b/tests/queries/0_stateless/00510_materizlized_view_and_deduplication_zookeeper.sql @@ -29,7 +29,7 @@ INSERT INTO without_deduplication VALUES (43); SELECT count() FROM with_deduplication; SELECT count() FROM without_deduplication; --- Implicit insert isn't deduplicated, because deduplicate_blocks_in_dependent_materialized_views = 0 by default +-- Implicit insert isn't deduplicated SELECT ''; SELECT countMerge(cnt) FROM with_deduplication_mv; SELECT countMerge(cnt) FROM without_deduplication_mv; diff --git a/tests/queries/0_stateless/00633_materialized_view_and_too_many_parts_zookeeper.sh b/tests/queries/0_stateless/00633_materialized_view_and_too_many_parts_zookeeper.sh index 8f7d19028b0..1fb219108da 100755 --- a/tests/queries/0_stateless/00633_materialized_view_and_too_many_parts_zookeeper.sh +++ b/tests/queries/0_stateless/00633_materialized_view_and_too_many_parts_zookeeper.sh @@ -36,8 +36,8 @@ ${CLICKHOUSE_CLIENT} --query "DROP TABLE c" echo ${CLICKHOUSE_CLIENT} --query "CREATE TABLE root (d UInt64) ENGINE = Null" ${CLICKHOUSE_CLIENT} --query "CREATE MATERIALIZED VIEW d (d UInt64) ENGINE = ReplicatedMergeTree('/clickhouse/$CLICKHOUSE_TEST_ZOOKEEPER_PREFIX/d', '1') ORDER BY d AS SELECT * FROM root" -${CLICKHOUSE_CLIENT} --query "INSERT INTO root SETTINGS deduplicate_blocks_in_dependent_materialized_views=1 VALUES (1)"; -${CLICKHOUSE_CLIENT} --query "INSERT INTO root SETTINGS deduplicate_blocks_in_dependent_materialized_views=1 VALUES (1)"; +${CLICKHOUSE_CLIENT} --query "INSERT INTO root VALUES (1)"; +${CLICKHOUSE_CLIENT} --query "INSERT INTO root VALUES (1)"; ${CLICKHOUSE_CLIENT} --query "SELECT * FROM d"; ${CLICKHOUSE_CLIENT} --query "DROP TABLE root" ${CLICKHOUSE_CLIENT} --query "DROP TABLE d" diff --git a/tests/queries/0_stateless/01275_parallel_mv.reference b/tests/queries/0_stateless/01275_parallel_mv.reference index dadf2f35e6e..a9801e3b910 100644 --- a/tests/queries/0_stateless/01275_parallel_mv.reference +++ b/tests/queries/0_stateless/01275_parallel_mv.reference @@ -137,7 +137,7 @@ select arrayUniq(thread_ids) from system.query_log where Settings['parallel_view_processing'] = '1' and Settings['optimize_trivial_insert_select'] = '0' and Settings['max_insert_threads'] = '16'; -18 +5 select count() from testX; 60 select count() from testXA; @@ -185,7 +185,7 @@ select arrayUniq(thread_ids) from system.query_log where Settings['parallel_view_processing'] = '1' and Settings['optimize_trivial_insert_select'] = '1' and Settings['max_insert_threads'] = '16'; -18 +5 select count() from testX; 80 select count() from testXA; diff --git a/tests/queries/0_stateless/01927_query_views_log_current_database.sql b/tests/queries/0_stateless/01927_query_views_log_current_database.sql index 6287156daaf..ba42795333c 100644 --- a/tests/queries/0_stateless/01927_query_views_log_current_database.sql +++ b/tests/queries/0_stateless/01927_query_views_log_current_database.sql @@ -16,7 +16,6 @@ CREATE MATERIALIZED VIEW matview_b_to_c TO table_c AS SELECT SUM(a + sleepEachRo CREATE MATERIALIZED VIEW matview_join_d_e TO table_f AS SELECT table_d.a as a, table_e.count + sleepEachRow(0.000003) as count FROM table_d LEFT JOIN table_e ON table_d.a = table_e.a; -- ENABLE LOGS -SET parallel_view_processing=0; SET log_query_views=1; SET log_queries_min_type='QUERY_FINISH'; SET log_queries=1; diff --git a/tests/queries/0_stateless/02124_insert_deduplication_token_materialized_views.reference b/tests/queries/0_stateless/02124_insert_deduplication_token_materialized_views.reference index 2d9f236ada9..e0cc8f0ce63 100644 --- a/tests/queries/0_stateless/02124_insert_deduplication_token_materialized_views.reference +++ b/tests/queries/0_stateless/02124_insert_deduplication_token_materialized_views.reference @@ -1,8 +1,8 @@ -deduplicate_blocks_in_dependent_materialized_views=0, insert_deduplication_token = no, results: test_mv_a and test_mv_c have all data, test_mv_b has data obly with max_partitions_per_insert_block=0 -18 36 27 36 -deduplicate_blocks_in_dependent_materialized_views=1, insert_deduplication_token = no, results: all tables have deduplicated data -18 18 18 18 -deduplicate_blocks_in_dependent_materialized_views=0, insert_deduplication_token = yes, results: test_mv_a and test_mv_c have all data, test_mv_b has data obly with max_partitions_per_insert_block=0 -18 36 27 36 -deduplicate_blocks_in_dependent_materialized_views=1, insert_deduplication_token = yes, results: all tables have deduplicated data +deduplicate_blocks_in_dependent_materialized_views=0, insert_deduplication_token = no, results inconsitent +18 18 9 18 +deduplicate_blocks_in_dependent_materialized_views=1, insert_deduplication_token = no, results inconsitent +18 9 9 9 +deduplicate_blocks_in_dependent_materialized_views=0, insert_deduplication_token = yes, results inconsitent +18 18 9 18 +deduplicate_blocks_in_dependent_materialized_views=1, insert_deduplication_token = yes, results consitent 18 18 18 18 diff --git a/tests/queries/0_stateless/02124_insert_deduplication_token_materialized_views.sql b/tests/queries/0_stateless/02124_insert_deduplication_token_materialized_views.sql index 465c8d6136c..fdd75b91b1f 100644 --- a/tests/queries/0_stateless/02124_insert_deduplication_token_materialized_views.sql +++ b/tests/queries/0_stateless/02124_insert_deduplication_token_materialized_views.sql @@ -1,6 +1,6 @@ -- Tags: long -select 'deduplicate_blocks_in_dependent_materialized_views=0, insert_deduplication_token = no, results: test_mv_a and test_mv_c have all data, test_mv_b has data obly with max_partitions_per_insert_block=0'; +select 'deduplicate_blocks_in_dependent_materialized_views=0, insert_deduplication_token = no, results inconsitent'; drop table if exists test sync; drop table if exists test_mv_a sync; @@ -35,7 +35,7 @@ select (select sum(c) from test_mv_c where test='case1'); -select 'deduplicate_blocks_in_dependent_materialized_views=1, insert_deduplication_token = no, results: all tables have deduplicated data'; +select 'deduplicate_blocks_in_dependent_materialized_views=1, insert_deduplication_token = no, results inconsitent'; set deduplicate_blocks_in_dependent_materialized_views=1; @@ -53,7 +53,7 @@ select (select sum(c) from test_mv_c where test='case2'); -select 'deduplicate_blocks_in_dependent_materialized_views=0, insert_deduplication_token = yes, results: test_mv_a and test_mv_c have all data, test_mv_b has data obly with max_partitions_per_insert_block=0'; +select 'deduplicate_blocks_in_dependent_materialized_views=0, insert_deduplication_token = yes, results inconsitent'; set deduplicate_blocks_in_dependent_materialized_views=0; @@ -70,7 +70,7 @@ select (select sum(c) from test_mv_b where test='case3'), (select sum(c) from test_mv_c where test='case3'); -select 'deduplicate_blocks_in_dependent_materialized_views=1, insert_deduplication_token = yes, results: all tables have deduplicated data'; +select 'deduplicate_blocks_in_dependent_materialized_views=1, insert_deduplication_token = yes, results consitent'; set deduplicate_blocks_in_dependent_materialized_views=1; diff --git a/tests/queries/0_stateless/02125_query_views_log.sql b/tests/queries/0_stateless/02125_query_views_log.sql index ba50902ebea..d2d19b76a1f 100644 --- a/tests/queries/0_stateless/02125_query_views_log.sql +++ b/tests/queries/0_stateless/02125_query_views_log.sql @@ -8,7 +8,7 @@ create table dst (key Int) engine=Null(); create materialized view mv1 to dst as select * from src; create materialized view mv2 to dst as select * from src; -insert into src select * from numbers(1e6) settings log_queries=1, max_untracked_memory=0, parallel_view_processing=0; +insert into src select * from numbers(1e6) settings log_queries=1, max_untracked_memory=0, parallel_view_processing=1; system flush logs; -- { echo } diff --git a/tests/queries/0_stateless/02912_ingestion_mv_deduplication.reference b/tests/queries/0_stateless/02912_ingestion_mv_deduplication.reference index 07deb7c2565..335b55f05c8 100644 --- a/tests/queries/0_stateless/02912_ingestion_mv_deduplication.reference +++ b/tests/queries/0_stateless/02912_ingestion_mv_deduplication.reference @@ -10,14 +10,13 @@ 2022-09-01 12:23:34 42 2023-09-01 12:23:34 42 -- MV -2022-09-01 12:00:00 84 -2023-09-01 12:00:00 42 +2022-09-01 12:00:00 42 -- Original issue with deduplicate_blocks_in_dependent_materialized_views = 1 AND max_insert_delayed_streams_for_parallel_write > 1 -- Landing 2022-09-01 12:23:34 42 2023-09-01 12:23:34 42 -- MV -2022-09-01 12:00:00 84 +2022-09-01 12:00:00 42 2023-09-01 12:00:00 42 -- Regression introduced in https://github.com/ClickHouse/ClickHouse/pull/54184 -- Landing (Agg/Replacing)MergeTree diff --git a/tests/queries/0_stateless/02912_ingestion_mv_deduplication.sql b/tests/queries/0_stateless/02912_ingestion_mv_deduplication.sql index a2378fd8f67..f206f0d7775 100644 --- a/tests/queries/0_stateless/02912_ingestion_mv_deduplication.sql +++ b/tests/queries/0_stateless/02912_ingestion_mv_deduplication.sql @@ -54,9 +54,8 @@ SELECT '-- Original issue with deduplicate_blocks_in_dependent_materialized_view - 1st insert works for landing and mv tables - 2nd insert gets first block 20220901 deduplicated and second one inserted in landing table - - 2nd insert is not inserting anything in mv table due to a bug computing blocks to be discarded, now that block is inserted because deduplicate_blocks_in_dependent_materialized_views=0 + - 2nd insert is not inserting anything in mv table due to a bug computing blocks to be discarded - Now it is fixed. */ SET deduplicate_blocks_in_dependent_materialized_views = 0, max_insert_delayed_streams_for_parallel_write = 1000; @@ -98,7 +97,7 @@ SELECT '-- Original issue with deduplicate_blocks_in_dependent_materialized_view This is what happens now: - 1st insert works for landing and mv tables - - 2nd insert gets first block 20220901 deduplicated for landing and both rows are inserted for mv tables + - 2nd insert gets first block 20220901 deduplicated and second one inserted for landing and mv tables */ SET deduplicate_blocks_in_dependent_materialized_views = 1, max_insert_delayed_streams_for_parallel_write = 1000; diff --git a/tests/queries/0_stateless/03008_deduplication.python b/tests/queries/0_stateless/03008_deduplication.python deleted file mode 100644 index dd1058518c9..00000000000 --- a/tests/queries/0_stateless/03008_deduplication.python +++ /dev/null @@ -1,657 +0,0 @@ -#!/usr/bin/env python3 - -import os -import sys -import argparse -import string - - -CURDIR = os.path.dirname(os.path.realpath(__file__)) -sys.path.insert(0, os.path.join(CURDIR, "helpers")) - - -def __format(template, **params): - field_names = [v[1] for v in string.Formatter().parse(template) if v[1] is not None] - kv_args = {} - for field in field_names: - if field in params: - kv_args[field] = params[field] - else: - kv_args[field] = "" - - return template.format(**kv_args) - - -def instance_create_statement( - table_name, - table_columns, - table_keys, - table_engine, - with_deduplication, - no_merges=True, -): - template = """ - CREATE TABLE {table_name} - {table_columns} - ENGINE = {table_engine} - ORDER BY {table_keys} - {table_settings}; - {table_no_merges} - """ - - params = dict() - params["table_name"] = table_name - params["table_columns"] = table_columns - params["table_keys"] = table_keys - params["table_no_merges"] = f"SYSTEM STOP MERGES {table_name};" if no_merges else "" - params["table_engine"] = ( - "MergeTree()" - if table_engine == "MergeTree" - else f"ReplicatedMergeTree('/clickhouse/tables/{{database}}/{table_name}', '1')" - ) - - deduplication_window_setting_name = ( - "non_replicated_deduplication_window" - if table_engine == "MergeTree" - else "replicated_deduplication_window" - ) - deduplication_window_setting_value = 1000 if with_deduplication else 0 - - settings = list() - settings += [ - f"{deduplication_window_setting_name}={deduplication_window_setting_value}" - ] - params["table_settings"] = "SETTINGS " + ",".join(settings) - - return __format(template, **params) - - -def instance_insert_statement( - table_name, count, insert_method, insert_unique_blocks, use_insert_token -): - insert_settings = ( - "" if not use_insert_token else "SETTINGS insert_deduplication_token='UDT'" - ) - - if insert_method == "InsertSelect": - template = """ - INSERT INTO {table_name} - SELECT {insert_columns} - FROM numbers({count}) {insert_settings}; - """ - return __format( - template, - table_name=table_name, - count=count, - insert_columns="'src_4', 4" - if not insert_unique_blocks - else "'src_' || toString(number), number", - insert_settings=insert_settings, - ) - - else: - template = """ - INSERT INTO {table_name} - {insert_settings} VALUES {insert_values}; - """ - - values = [] - for i in range(count): - values += ( - [f"('src_{i}', {i})"] if insert_unique_blocks else ["('src_4', 4)"] - ) - insert_values = ", ".join(values) - - return __format( - template, - table_name=table_name, - insert_settings=insert_settings, - insert_values=insert_values, - ) - - -def get_drop_tables_statements(tables): - return "".join( - [f"DROP TABLE IF EXISTS {table_name};\n" for table_name in tables[::-1]] - ) - - -def get_logs_statement(args): - if args.get_logs: - return "SET send_logs_level='test';" - return "" - - -def str2bool(v): - if isinstance(v, bool): - return v - if v.lower() in ("yes", "true", "t", "y", "1"): - return True - elif v.lower() in ("no", "false", "f", "n", "0"): - return False - else: - raise argparse.ArgumentTypeError("Boolean value expected.") - - -class ArgsFactory: - def __init__(self, parser): - self.__parser = parser - - def add_opt_engine(self): - self.__parser.add_argument( - "--table-engine", - choices=["ReplicatedMergeTree", "MergeTree"], - default="MergeTree", - ) - - def add_opt_user_token(self): - self.__parser.add_argument( - "--use-insert-token", type=str2bool, nargs="?", const=True, default=False - ) - - def add_opt_single_thread(self): - self.__parser.add_argument( - "--single-thread", type=str2bool, nargs="?", const=True, default=True - ) - - def add_opt_dedup_src(self): - self.__parser.add_argument( - "--deduplicate-src-table", - type=str2bool, - nargs="?", - const=True, - default=True, - ) - - def add_opt_dedup_dst(self): - self.__parser.add_argument( - "--deduplicate-dst-table", - type=str2bool, - nargs="?", - const=True, - default=True, - ) - - def add_opt_get_logs(self): - self.__parser.add_argument( - "--get-logs", type=str2bool, nargs="?", const=True, default=False - ) - - def add_opt_uniq_blocks(self): - self.__parser.add_argument( - "--insert-unique-blocks", type=str2bool, nargs="?", const=True, default=True - ) - - def add_opt_insert_method(self): - self.__parser.add_argument( - "--insert-method", - choices=["InsertSelect", "InsertValues"], - default="InsertSelect", - ) - - def add_all(self): - self.add_opt_engine() - self.add_opt_user_token() - self.add_opt_single_thread() - self.add_opt_dedup_src() - self.add_opt_dedup_dst() - self.add_opt_get_logs() - self.add_opt_insert_method() - self.add_opt_uniq_blocks() - - -def test_insert_several_blocks(parser): - ArgsFactory(parser).add_all() - - def calle(args): - create_table_a_b_statement = instance_create_statement( - table_name="table_a_b", - table_columns="(a String, b UInt64)", - table_keys="(a, b)", - table_engine=args.table_engine, - with_deduplication=args.deduplicate_src_table, - ) - - create_table_when_b_even_statement = instance_create_statement( - table_name="table_when_b_even", - table_columns="(a String, b UInt64)", - table_keys="(a, b)", - table_engine=args.table_engine, - with_deduplication=args.deduplicate_dst_table, - ) - - create_mv_statement = """ - CREATE MATERIALIZED VIEW mv_b_even - TO table_when_b_even - AS - SELECT a, b - FROM table_a_b - WHERE b % 2 = 0; - """ - - drop_tables_statements = get_drop_tables_statements( - ["table_a_b", "table_when_b_even", "mv_b_even"] - ) - - insert_statement = instance_insert_statement( - "table_a_b", - 10, - args.insert_method, - args.insert_unique_blocks, - args.use_insert_token, - ) - - print_details_statements = f""" - SELECT 'table_a_b'; - SELECT 'count', count() FROM table_a_b; - {"" if not args.get_logs else "SELECT _part, count() FROM table_a_b GROUP BY _part ORDER BY _part;"} - - SELECT 'table_when_b_even'; - SELECT 'count', count() FROM table_when_b_even; - {"" if not args.get_logs else "SELECT _part, count() FROM table_when_b_even GROUP BY _part ORDER BY _part;"} - """ - - if args.insert_unique_blocks: - assert_first_insert_statements = f""" - SELECT throwIf( count() != 10 ) - FROM table_a_b; - SELECT throwIf( count() != 5 ) - FROM table_when_b_even; - """ - assert_second_insert_statements = f""" - SELECT throwIf( count() != {10 if args.deduplicate_src_table else 20} ) - FROM table_a_b; - SELECT throwIf( count() != {5 if args.deduplicate_dst_table else 10} ) - FROM table_when_b_even; - """ - else: - if args.use_insert_token: - assert_first_insert_statements = """ - SELECT throwIf( count() != 10 ) - FROM table_a_b; - SELECT throwIf( count() != 10 ) - FROM table_when_b_even; - """ - assert_second_insert_statements = f""" - SELECT throwIf( count() != {10 if args.deduplicate_src_table else 20} ) - FROM table_a_b; - SELECT throwIf( count() != {10 if args.deduplicate_dst_table else 20} ) - FROM table_when_b_even; - """ - else: - assert_first_insert_statements = f""" - SELECT throwIf( count() != {1 if args.deduplicate_src_table else 10} ) - FROM table_a_b; - SELECT throwIf( count() != {1 if args.deduplicate_dst_table else 10} ) - FROM table_when_b_even; - """ - assert_second_insert_statements = f""" - SELECT throwIf( count() != {1 if args.deduplicate_src_table else 20} ) - FROM table_a_b; - SELECT throwIf( count() != {1 if args.deduplicate_dst_table else 20} ) - FROM table_when_b_even; - """ - - script = f""" - {get_logs_statement(args)} - - SET max_insert_threads={1 if args.single_thread else 10}; - SET update_insert_deduplication_token_in_dependent_materialized_views=1; - SET deduplicate_blocks_in_dependent_materialized_views=1; - - SET max_block_size=1; - SET min_insert_block_size_rows=0; - SET min_insert_block_size_bytes=0; - - {drop_tables_statements} - - {create_table_a_b_statement} - - {create_table_when_b_even_statement} - - {create_mv_statement} - - -- first insert - {insert_statement} - - {print_details_statements} - - {assert_first_insert_statements} - - -- second insert, it is retry - {insert_statement} - - {print_details_statements} - - {assert_second_insert_statements} - - {drop_tables_statements} - """ - - print(script) - - parser.set_defaults(func=calle) - - -def test_mv_generates_several_blocks(parser): - ArgsFactory(parser).add_all() - - def calle(args): - tables = [ - "table_for_join_with", - "table_a_b", - "table_when_b_even_and_joined", - "mv_b_even", - ] - drop_tables_statements = get_drop_tables_statements(tables) - - details_print_for_table_for_join_with = "" - if args.get_logs: - details_print_for_table_for_join_with = """ - SELECT 'table_for_join_with'; - SELECT a_join, b, _part FROM table_for_join_with ORDER BY _part, a_join, b; - """ - - create_table_a_b_statement = instance_create_statement( - table_name="table_a_b", - table_columns="(a_src String, b UInt64)", - table_keys="(a_src, b)", - table_engine=args.table_engine, - with_deduplication=args.deduplicate_src_table, - ) - - create_table_when_b_even_and_joined_statement = instance_create_statement( - table_name="table_when_b_even_and_joined", - table_columns="(a_src String, a_join String, b UInt64)", - table_keys="(a_src, a_join, b)", - table_engine=args.table_engine, - with_deduplication=args.deduplicate_dst_table, - ) - - insert_statement = instance_insert_statement( - "table_a_b", - 5, - args.insert_method, - args.insert_unique_blocks, - args.use_insert_token, - ) - - details_print_statements = f""" - SELECT 'table_a_b'; - SELECT 'count', count() FROM table_a_b; - - SELECT 'table_when_b_even_and_joined'; - SELECT 'count', count() FROM table_when_b_even_and_joined; - {"" if not args.get_logs else "SELECT _part, a_src, a_join, b FROM table_when_b_even_and_joined ORDER BY _part;"} - """ - - if args.insert_unique_blocks: - assert_first_insert_statements = f""" - SELECT throwIf( count() != 5 ) - FROM table_a_b; - - SELECT throwIf( count() != 9 ) - FROM table_when_b_even_and_joined; - """ - assert_second_insert_statements = f""" - SELECT throwIf( count() != {5 if args.deduplicate_src_table else 10} ) - FROM table_a_b; - - SELECT throwIf( count() != {9 if args.deduplicate_dst_table else 18} ) - FROM table_when_b_even_and_joined; - """ - else: - if args.use_insert_token: - assert_first_insert_statements = f""" - SELECT throwIf( count() != {5 if args.deduplicate_src_table else 5} ) - FROM table_a_b; - - SELECT throwIf( count() != {10 if args.deduplicate_dst_table else 10} ) - FROM table_when_b_even_and_joined; - """ - assert_second_insert_statements = f""" - SELECT throwIf( count() != {5 if args.deduplicate_src_table else 10} ) - FROM table_a_b; - - SELECT throwIf( count() != {10 if args.deduplicate_dst_table else 20} ) - FROM table_when_b_even_and_joined; - """ - else: - assert_first_insert_statements = f""" - SELECT throwIf( count() != {1 if args.deduplicate_src_table else 5} ) - FROM table_a_b; - - SELECT throwIf( count() != {2 if args.deduplicate_dst_table else 10} ) - FROM table_when_b_even_and_joined; - """ - assert_second_insert_statements = f""" - SELECT throwIf( count() != {1 if args.deduplicate_src_table else 10} ) - FROM table_a_b; - - SELECT throwIf( count() != {2 if args.deduplicate_dst_table else 20} ) - FROM table_when_b_even_and_joined; - """ - - script = f""" - {get_logs_statement(args)} - - SET max_insert_threads={1 if args.single_thread else 10}; - SET update_insert_deduplication_token_in_dependent_materialized_views=1; - SET deduplicate_blocks_in_dependent_materialized_views=1; - - SET max_block_size=1; - SET min_insert_block_size_rows=0; - SET min_insert_block_size_bytes=0; - - {drop_tables_statements} - - CREATE TABLE table_for_join_with - (a_join String, b UInt64) - ENGINE = MergeTree() - ORDER BY (a_join, b); - INSERT INTO table_for_join_with - SELECT 'joined_' || toString(number), number - FROM numbers(1); - {details_print_for_table_for_join_with} - - {create_table_a_b_statement} - SYSTEM STOP MERGES table_a_b; - - {create_table_when_b_even_and_joined_statement} - SYSTEM STOP MERGES table_when_b_even_and_joined; - - CREATE MATERIALIZED VIEW mv_b_even - TO table_when_b_even_and_joined - AS - SELECT a_src, a_join, table_for_join_with.b as b - FROM table_a_b - FULL OUTER JOIN table_for_join_with - ON table_a_b.b = table_for_join_with.b AND table_a_b.b % 2 = 0 - ORDER BY a_src, a_join, b; - - -- first insert - {insert_statement} - - {details_print_statements} - - -- first assertion - {assert_first_insert_statements} - - -- second insert - {insert_statement} - - {details_print_statements} - - -- second assertion - {assert_second_insert_statements} - - {drop_tables_statements} - """ - - print(script) - - parser.set_defaults(func=calle) - - -def test_several_mv_into_one_table(parser): - ArgsFactory(parser).add_all() - - def calle(args): - tables = ["table_src", "table_dst", "mv_b_even", "mv_b_even_even"] - drop_tables_statements = get_drop_tables_statements(tables) - - create_table_src_statement = instance_create_statement( - table_name="table_src", - table_columns="(a String, b UInt64)", - table_keys="(a, b)", - table_engine=args.table_engine, - with_deduplication=args.deduplicate_src_table, - ) - - create_table_dst_statement = instance_create_statement( - table_name="table_dst", - table_columns="(a String, b UInt64)", - table_keys="(a, b)", - table_engine=args.table_engine, - with_deduplication=args.deduplicate_dst_table, - ) - - insert_statement = instance_insert_statement( - "table_src", - 8, - args.insert_method, - args.insert_unique_blocks, - args.use_insert_token, - ) - - details_print_statements = f""" - SELECT 'table_src count', count() FROM table_src; - - SELECT 'table_dst count', count() FROM table_dst; - {"" if not args.get_logs else "SELECT _part, count() FROM table_dst GROUP BY _part ORDER BY _part;"} - """ - - if args.insert_unique_blocks: - assert_first_insert_statements = f""" - SELECT throwIf( count() != 8 ) - FROM table_src; - - SELECT throwIf( count() != 6 ) - FROM table_dst; - """ - assert_second_insert_statements = f""" - SELECT throwIf( count() != {8 if args.deduplicate_src_table else 16} ) - FROM table_src; - - SELECT throwIf( count() != {6 if args.deduplicate_dst_table else 12} ) - FROM table_dst; - """ - else: - if args.use_insert_token: - assert_first_insert_statements = f""" - SELECT throwIf( count() != {8 if args.deduplicate_src_table else 8} ) - FROM table_src; - - SELECT throwIf( count() != {16 if args.deduplicate_dst_table else 16} ) - FROM table_dst; - """ - assert_second_insert_statements = f""" - SELECT throwIf( count() != {8 if args.deduplicate_src_table else 16} ) - FROM table_src; - - SELECT throwIf( count() != {16 if args.deduplicate_dst_table else 32} ) - FROM table_dst; - """ - else: - assert_first_insert_statements = f""" - SELECT throwIf( count() != {1 if args.deduplicate_src_table else 8} ) - FROM table_src; - - SELECT throwIf( count() != {2 if args.deduplicate_dst_table else 16} ) - FROM table_dst; - """ - assert_second_insert_statements = f""" - SELECT throwIf( count() != {1 if args.deduplicate_src_table else 16} ) - FROM table_src; - - SELECT throwIf( count() != {2 if args.deduplicate_dst_table else 32} ) - FROM table_dst; - """ - - script = f""" - {get_logs_statement(args)} - - SET max_insert_threads={1 if args.single_thread else 10}; - SET update_insert_deduplication_token_in_dependent_materialized_views=1; - SET deduplicate_blocks_in_dependent_materialized_views=1; - - SET max_block_size=1; - SET min_insert_block_size_rows=0; - SET min_insert_block_size_bytes=0; - - {drop_tables_statements} - - {create_table_src_statement} - - {create_table_dst_statement} - - CREATE MATERIALIZED VIEW mv_b_even - TO table_dst - AS - SELECT a, b - FROM table_src - WHERE b % 2 = 0; - - CREATE MATERIALIZED VIEW mv_b_even_even - TO table_dst - AS - SELECT a, b - FROM table_src - WHERE b % 4 = 0; - - -- first insert - {insert_statement} - - {details_print_statements} - - {assert_first_insert_statements} - - -- second insert, retry - {insert_statement} - - {details_print_statements} - - {assert_second_insert_statements} - - {drop_tables_statements} - """ - - print(script) - - parser.set_defaults(func=calle) - - -def parse_args(): - parser = argparse.ArgumentParser() - subparsers = parser.add_subparsers(dest="test") - test_insert_several_blocks( - subparsers.add_parser("insert_several_blocks_into_table") - ) - test_mv_generates_several_blocks( - subparsers.add_parser("mv_generates_several_blocks") - ) - test_several_mv_into_one_table(subparsers.add_parser("several_mv_into_one_table")) - args = parser.parse_args() - if args.test is None: - parser.print_help() - return args - - -def main(): - args = parse_args() - if args.test is not None: - args.func(args) - - -if __name__ == "__main__": - main() diff --git a/tests/queries/0_stateless/03008_deduplication_cases_from_docs.reference b/tests/queries/0_stateless/03008_deduplication_cases_from_docs.reference deleted file mode 100644 index 4893274c1cd..00000000000 --- a/tests/queries/0_stateless/03008_deduplication_cases_from_docs.reference +++ /dev/null @@ -1,41 +0,0 @@ -Different materialized view insert into one underlayed table equal data. -first attempt -from dst 1 A all_1_1_0 -from mv_dst 0 A all_1_1_0 -from mv_dst 0 A all_2_2_0 -second attempt -from dst 1 A all_1_1_0 -from mv_dst 0 A all_1_1_0 -from mv_dst 0 A all_2_2_0 -Different insert operations generate the same data after transformation in underlied table of materialized view. -first attempt -from dst 1 A all_1_1_0 -from mv_dst 0 A all_1_1_0 -second attempt -from dst 1 A all_1_1_0 -from dst 2 A all_2_2_0 -from mv_dst 0 A all_1_1_0 -from mv_dst 0 A all_2_2_0 -Indentical blocks in insertion with `insert_deduplication_token` -first attempt -from dst 0 A all_1_1_0 -from dst 0 A all_2_2_0 -second attempt -from dst 0 A all_1_1_0 -from dst 0 A all_2_2_0 -third attempt -from dst 0 A all_1_1_0 -from dst 0 A all_2_2_0 -Indentical blocks in insertion -from dst 0 A all_1_1_0 -Indentical blocks after materialised view`s transformation -first attempt -from dst 1 B all_1_1_0 -from dst 2 B all_2_2_0 -from mv_dst 0 B all_1_1_0 -from mv_dst 0 B all_2_2_0 -second attempt -from dst 1 B all_1_1_0 -from dst 2 B all_2_2_0 -from mv_dst 0 B all_1_1_0 -from mv_dst 0 B all_2_2_0 diff --git a/tests/queries/0_stateless/03008_deduplication_cases_from_docs.sql b/tests/queries/0_stateless/03008_deduplication_cases_from_docs.sql deleted file mode 100644 index 7927a6b1edf..00000000000 --- a/tests/queries/0_stateless/03008_deduplication_cases_from_docs.sql +++ /dev/null @@ -1,331 +0,0 @@ --- ######### -select 'Different materialized view insert into one underlayed table equal data.'; - -DROP TABLE IF EXISTS dst; -DROP TABLE IF EXISTS mv_dst; -DROP TABLE IF EXISTS mv_first; -DROP TABLE IF EXISTS mv_second; - -CREATE TABLE dst -( - `key` Int64, - `value` String -) -ENGINE = MergeTree -ORDER BY tuple() -SETTINGS non_replicated_deduplication_window=1000; - -CREATE TABLE mv_dst -( - `key` Int64, - `value` String -) -ENGINE = MergeTree -ORDER BY tuple() -SETTINGS non_replicated_deduplication_window=1000; - -CREATE MATERIALIZED VIEW mv_first -TO mv_dst -AS SELECT - 0 AS key, - value AS value -FROM dst; - -CREATE MATERIALIZED VIEW mv_second -TO mv_dst -AS SELECT - 0 AS key, - value AS value -FROM dst; - -SET deduplicate_blocks_in_dependent_materialized_views=1; - -select 'first attempt'; - -INSERT INTO dst VALUES (1, 'A'); - -SELECT - 'from dst', - *, - _part -FROM dst -ORDER by all; - -SELECT - 'from mv_dst', - *, - _part -FROM mv_dst -ORDER by all; - -select 'second attempt'; - -INSERT INTO dst VALUES (1, 'A'); - -SELECT - 'from dst', - *, - _part -FROM dst -ORDER by all; - -SELECT - 'from mv_dst', - *, - _part -FROM mv_dst -ORDER by all; - -DROP TABLE mv_second; -DROP TABLE mv_first; -DROP TABLE mv_dst; -DROP TABLE dst; - - --- ######### -select 'Different insert operations generate the same data after transformation in underlied table of materialized view.'; - -DROP TABLE IF EXISTS dst; -DROP TABLE IF EXISTS mv_dst; - -CREATE TABLE dst -( - `key` Int64, - `value` String -) -ENGINE = MergeTree -ORDER BY tuple() -SETTINGS non_replicated_deduplication_window=1000; - -CREATE MATERIALIZED VIEW mv_dst -( - `key` Int64, - `value` String -) -ENGINE = MergeTree -ORDER BY tuple() -SETTINGS non_replicated_deduplication_window=1000 -AS SELECT - 0 AS key, - value AS value -FROM dst; - -SET deduplicate_blocks_in_dependent_materialized_views=1; - -select 'first attempt'; - -INSERT INTO dst VALUES (1, 'A'); - -SELECT - 'from dst', - *, - _part -FROM dst -ORDER by all; - -SELECT - 'from mv_dst', - *, - _part -FROM mv_dst -ORDER by all; - -select 'second attempt'; - -INSERT INTO dst VALUES (2, 'A'); - -SELECT - 'from dst', - *, - _part -FROM dst -ORDER by all; - -SELECT - 'from mv_dst', - *, - _part -FROM mv_dst -ORDER by all; - -DROP TABLE mv_dst; -DROP TABLE dst; - - --- ######### -select 'Indentical blocks in insertion with `insert_deduplication_token`'; - -DROP TABLE IF EXISTS dst; - -CREATE TABLE dst -( - `key` Int64, - `value` String -) -ENGINE = MergeTree -ORDER BY tuple() -SETTINGS non_replicated_deduplication_window=1000; - -SET max_block_size=1; -SET min_insert_block_size_rows=0; -SET min_insert_block_size_bytes=0; - -select 'first attempt'; - -INSERT INTO dst SELECT - 0 AS key, - 'A' AS value -FROM numbers(2) -SETTINGS insert_deduplication_token='some_user_token'; - -SELECT - 'from dst', - *, - _part -FROM dst -ORDER by all; - -select 'second attempt'; - -INSERT INTO dst SELECT - 0 AS key, - 'A' AS value -FROM numbers(2) -SETTINGS insert_deduplication_token='some_user_token'; - -SELECT - 'from dst', - *, - _part -FROM dst -ORDER by all; - -select 'third attempt'; - -INSERT INTO dst SELECT - 1 AS key, - 'b' AS value -FROM numbers(2) -SETTINGS insert_deduplication_token='some_user_token'; - -SELECT - 'from dst', - *, - _part -FROM dst -ORDER by all; - -DROP TABLE dst; - - --- ######### -select 'Indentical blocks in insertion'; - -DROP TABLE IF EXISTS dst; - -CREATE TABLE dst -( - `key` Int64, - `value` String -) -ENGINE = MergeTree -ORDER BY tuple() -SETTINGS non_replicated_deduplication_window=1000; - -SET max_block_size=1; -SET min_insert_block_size_rows=0; -SET min_insert_block_size_bytes=0; - -INSERT INTO dst SELECT - 0 AS key, - 'A' AS value -FROM numbers(2); - -SELECT - 'from dst', - *, - _part -FROM dst -ORDER by all; - -DROP TABLE dst; - - --- ######### -select 'Indentical blocks after materialised view`s transformation'; - -DROP TABLE IF EXISTS dst; -DROP TABLE IF EXISTS mv_dst; - -CREATE TABLE dst -( - `key` Int64, - `value` String -) -ENGINE = MergeTree -ORDER BY tuple() -SETTINGS non_replicated_deduplication_window=1000; - -CREATE MATERIALIZED VIEW mv_dst -( - `key` Int64, - `value` String -) -ENGINE = MergeTree -ORDER BY tuple() -SETTINGS non_replicated_deduplication_window=1000 -AS SELECT - 0 AS key, - value AS value -FROM dst; - -SET max_block_size=1; -SET min_insert_block_size_rows=0; -SET min_insert_block_size_bytes=0; - -SET deduplicate_blocks_in_dependent_materialized_views=1; - -select 'first attempt'; - -INSERT INTO dst SELECT - number + 1 AS key, - IF(key = 0, 'A', 'B') AS value -FROM numbers(2); - -SELECT - 'from dst', - *, - _part -FROM dst -ORDER by all; - -SELECT - 'from mv_dst', - *, - _part -FROM mv_dst -ORDER by all; - -select 'second attempt'; - -INSERT INTO dst SELECT - number + 1 AS key, - IF(key = 0, 'A', 'B') AS value -FROM numbers(2); - -SELECT - 'from dst', - *, - _part -FROM dst -ORDER by all; - -SELECT - 'from mv_dst', - *, - _part -FROM mv_dst -ORDER by all; - -DROP TABLE mv_dst; -DROP TABLE dst; diff --git a/tests/queries/0_stateless/03008_deduplication_insert_into_partitioned_table.reference b/tests/queries/0_stateless/03008_deduplication_insert_into_partitioned_table.reference deleted file mode 100644 index c82a6eaa213..00000000000 --- a/tests/queries/0_stateless/03008_deduplication_insert_into_partitioned_table.reference +++ /dev/null @@ -1,35 +0,0 @@ -no user deduplication token -partitioned_table is deduplicated bacause deduplication works in scope of one partiotion: -1 A -1 D -2 B -2 C -mv_table is not deduplicated because the inserted blocks was different: -1 A -1 A -1 D -2 B -2 B -2 C -with user deduplication token -partitioned_table is not deduplicated because different tokens: -1 A -1 A -1 D -2 B -2 B -2 C -mv_table is not deduplicated because different tokens: -1 A -1 A -1 D -2 B -2 B -2 C -with incorrect ussage of user deduplication token -partitioned_table is deduplicated because equal tokens: -1 A -2 B -mv_table is deduplicated because equal tokens: -1 A -2 B diff --git a/tests/queries/0_stateless/03008_deduplication_insert_into_partitioned_table.sql b/tests/queries/0_stateless/03008_deduplication_insert_into_partitioned_table.sql deleted file mode 100644 index 2eb931f7f73..00000000000 --- a/tests/queries/0_stateless/03008_deduplication_insert_into_partitioned_table.sql +++ /dev/null @@ -1,83 +0,0 @@ -DROP TABLE IF EXISTS partitioned_table; -DROP TABLE IF EXISTS mv_table; - - -SET deduplicate_blocks_in_dependent_materialized_views = 1; - - -SELECT 'no user deduplication token'; - -CREATE TABLE partitioned_table - (key Int64, value String) - ENGINE = ReplicatedMergeTree('/clickhouse/tables/{database}/03008_deduplication_insert_into_partitioned_table', '{replica}') - partition by key % 10 - order by tuple(); - -CREATE MATERIALIZED VIEW mv_table (key Int64, value String) - ENGINE = ReplicatedMergeTree('/clickhouse/tables/{database}/03008_deduplication_insert_into_partitioned_table_mv', '{replica}') - ORDER BY tuple() - AS SELECT key, value FROM partitioned_table; - -INSERT INTO partitioned_table VALUES (1, 'A'), (2, 'B'); -INSERT INTO partitioned_table VALUES (1, 'A'), (2, 'C'); -INSERT INTO partitioned_table VALUES (1, 'D'), (2, 'B'); - -SELECT 'partitioned_table is deduplicated bacause deduplication works in scope of one partiotion:'; -SELECT * FROM partitioned_table ORDER BY ALL; -SELECT 'mv_table is not deduplicated because the inserted blocks was different:'; -SELECT * FROM mv_table ORDER BY ALL; - -DROP TABLE partitioned_table; -DROP TABLE mv_table; - - -SELECT 'with user deduplication token'; - -CREATE TABLE partitioned_table - (key Int64, value String) - ENGINE = ReplicatedMergeTree('/clickhouse/tables/{database}/03008_deduplication_insert_into_partitioned_table', '{replica}') - partition by key % 10 - order by tuple(); - -CREATE MATERIALIZED VIEW mv_table (key Int64, value String) - ENGINE = ReplicatedMergeTree('/clickhouse/tables/{database}/03008_deduplication_insert_into_partitioned_table_mv', '{replica}') - ORDER BY tuple() - AS SELECT key, value FROM partitioned_table; - -INSERT INTO partitioned_table SETTINGS insert_deduplication_token='token_1' VALUES (1, 'A'), (2, 'B'); -INSERT INTO partitioned_table SETTINGS insert_deduplication_token='token_2' VALUES (1, 'A'), (2, 'C'); -INSERT INTO partitioned_table SETTINGS insert_deduplication_token='token_3' VALUES (1, 'D'), (2, 'B'); - -SELECT 'partitioned_table is not deduplicated because different tokens:'; -SELECT * FROM partitioned_table ORDER BY ALL; -SELECT 'mv_table is not deduplicated because different tokens:'; -SELECT * FROM mv_table ORDER BY ALL; - -DROP TABLE partitioned_table; -DROP TABLE mv_table; - - -SELECT 'with incorrect ussage of user deduplication token'; - -CREATE TABLE partitioned_table - (key Int64, value String) - ENGINE = ReplicatedMergeTree('/clickhouse/tables/{database}/03008_deduplication_insert_into_partitioned_table', '{replica}') - partition by key % 10 - order by tuple(); - -CREATE MATERIALIZED VIEW mv_table (key Int64, value String) - ENGINE = ReplicatedMergeTree('/clickhouse/tables/{database}/03008_deduplication_insert_into_partitioned_table_mv', '{replica}') - ORDER BY tuple() - AS SELECT key, value FROM partitioned_table; - -INSERT INTO partitioned_table SETTINGS insert_deduplication_token='token_0' VALUES (1, 'A'), (2, 'B'); -INSERT INTO partitioned_table SETTINGS insert_deduplication_token='token_0' VALUES (1, 'A'), (2, 'C'); -INSERT INTO partitioned_table SETTINGS insert_deduplication_token='token_0' VALUES (1, 'D'), (2, 'B'); - -SELECT 'partitioned_table is deduplicated because equal tokens:'; -SELECT * FROM partitioned_table ORDER BY ALL; -SELECT 'mv_table is deduplicated because equal tokens:'; -SELECT * FROM mv_table ORDER BY ALL; - -DROP TABLE partitioned_table; -DROP TABLE mv_table; diff --git a/tests/queries/0_stateless/03008_deduplication_insert_several_blocks_nonreplicated.reference b/tests/queries/0_stateless/03008_deduplication_insert_several_blocks_nonreplicated.reference deleted file mode 100644 index bf900aa84d2..00000000000 --- a/tests/queries/0_stateless/03008_deduplication_insert_several_blocks_nonreplicated.reference +++ /dev/null @@ -1,962 +0,0 @@ - -Test case 0: insert_method=InsertSelect engine=MergeTree use_insert_token=True single_thread=True deduplicate_src_table=True deduplicate_dst_table=True insert_unique_blocks=True -table_a_b -count 10 -table_when_b_even -count 5 -0 -0 -table_a_b -count 10 -table_when_b_even -count 5 -0 -0 -OK - -Test case 1: insert_method=InsertSelect engine=MergeTree use_insert_token=True single_thread=True deduplicate_src_table=True deduplicate_dst_table=True insert_unique_blocks=False -table_a_b -count 10 -table_when_b_even -count 10 -0 -0 -table_a_b -count 10 -table_when_b_even -count 10 -0 -0 -OK - -Test case 2: insert_method=InsertSelect engine=MergeTree use_insert_token=True single_thread=True deduplicate_src_table=True deduplicate_dst_table=False insert_unique_blocks=True -table_a_b -count 10 -table_when_b_even -count 5 -0 -0 -table_a_b -count 10 -table_when_b_even -count 10 -0 -0 -OK - -Test case 3: insert_method=InsertSelect engine=MergeTree use_insert_token=True single_thread=True deduplicate_src_table=True deduplicate_dst_table=False insert_unique_blocks=False -table_a_b -count 10 -table_when_b_even -count 10 -0 -0 -table_a_b -count 10 -table_when_b_even -count 20 -0 -0 -OK - -Test case 4: insert_method=InsertSelect engine=MergeTree use_insert_token=True single_thread=True deduplicate_src_table=False deduplicate_dst_table=True insert_unique_blocks=True -table_a_b -count 10 -table_when_b_even -count 5 -0 -0 -table_a_b -count 20 -table_when_b_even -count 5 -0 -0 -OK - -Test case 5: insert_method=InsertSelect engine=MergeTree use_insert_token=True single_thread=True deduplicate_src_table=False deduplicate_dst_table=True insert_unique_blocks=False -table_a_b -count 10 -table_when_b_even -count 10 -0 -0 -table_a_b -count 20 -table_when_b_even -count 10 -0 -0 -OK - -Test case 6: insert_method=InsertSelect engine=MergeTree use_insert_token=True single_thread=True deduplicate_src_table=False deduplicate_dst_table=False insert_unique_blocks=True -table_a_b -count 10 -table_when_b_even -count 5 -0 -0 -table_a_b -count 20 -table_when_b_even -count 10 -0 -0 -OK - -Test case 7: insert_method=InsertSelect engine=MergeTree use_insert_token=True single_thread=True deduplicate_src_table=False deduplicate_dst_table=False insert_unique_blocks=False -table_a_b -count 10 -table_when_b_even -count 10 -0 -0 -table_a_b -count 20 -table_when_b_even -count 20 -0 -0 -OK - -Test case 8: insert_method=InsertSelect engine=MergeTree use_insert_token=True single_thread=False deduplicate_src_table=True deduplicate_dst_table=True insert_unique_blocks=True -table_a_b -count 10 -table_when_b_even -count 5 -0 -0 -table_a_b -count 10 -table_when_b_even -count 5 -0 -0 -OK - -Test case 9: insert_method=InsertSelect engine=MergeTree use_insert_token=True single_thread=False deduplicate_src_table=True deduplicate_dst_table=True insert_unique_blocks=False -table_a_b -count 10 -table_when_b_even -count 10 -0 -0 -table_a_b -count 10 -table_when_b_even -count 10 -0 -0 -OK - -Test case 10: insert_method=InsertSelect engine=MergeTree use_insert_token=True single_thread=False deduplicate_src_table=True deduplicate_dst_table=False insert_unique_blocks=True -table_a_b -count 10 -table_when_b_even -count 5 -0 -0 -table_a_b -count 10 -table_when_b_even -count 10 -0 -0 -OK - -Test case 11: insert_method=InsertSelect engine=MergeTree use_insert_token=True single_thread=False deduplicate_src_table=True deduplicate_dst_table=False insert_unique_blocks=False -table_a_b -count 10 -table_when_b_even -count 10 -0 -0 -table_a_b -count 10 -table_when_b_even -count 20 -0 -0 -OK - -Test case 12: insert_method=InsertSelect engine=MergeTree use_insert_token=True single_thread=False deduplicate_src_table=False deduplicate_dst_table=True insert_unique_blocks=True -table_a_b -count 10 -table_when_b_even -count 5 -0 -0 -table_a_b -count 20 -table_when_b_even -count 5 -0 -0 -OK - -Test case 13: insert_method=InsertSelect engine=MergeTree use_insert_token=True single_thread=False deduplicate_src_table=False deduplicate_dst_table=True insert_unique_blocks=False -table_a_b -count 10 -table_when_b_even -count 10 -0 -0 -table_a_b -count 20 -table_when_b_even -count 10 -0 -0 -OK - -Test case 14: insert_method=InsertSelect engine=MergeTree use_insert_token=True single_thread=False deduplicate_src_table=False deduplicate_dst_table=False insert_unique_blocks=True -table_a_b -count 10 -table_when_b_even -count 5 -0 -0 -table_a_b -count 20 -table_when_b_even -count 10 -0 -0 -OK - -Test case 15: insert_method=InsertSelect engine=MergeTree use_insert_token=True single_thread=False deduplicate_src_table=False deduplicate_dst_table=False insert_unique_blocks=False -table_a_b -count 10 -table_when_b_even -count 10 -0 -0 -table_a_b -count 20 -table_when_b_even -count 20 -0 -0 -OK - -Test case 16: insert_method=InsertSelect engine=MergeTree use_insert_token=False single_thread=True deduplicate_src_table=True deduplicate_dst_table=True insert_unique_blocks=True -table_a_b -count 10 -table_when_b_even -count 5 -0 -0 -table_a_b -count 10 -table_when_b_even -count 5 -0 -0 -OK - -Test case 17: insert_method=InsertSelect engine=MergeTree use_insert_token=False single_thread=True deduplicate_src_table=True deduplicate_dst_table=True insert_unique_blocks=False -table_a_b -count 1 -table_when_b_even -count 1 -0 -0 -table_a_b -count 1 -table_when_b_even -count 1 -0 -0 -OK - -Test case 18: insert_method=InsertSelect engine=MergeTree use_insert_token=False single_thread=True deduplicate_src_table=True deduplicate_dst_table=False insert_unique_blocks=True -table_a_b -count 10 -table_when_b_even -count 5 -0 -0 -table_a_b -count 10 -table_when_b_even -count 10 -0 -0 -OK - -Test case 19: insert_method=InsertSelect engine=MergeTree use_insert_token=False single_thread=True deduplicate_src_table=True deduplicate_dst_table=False insert_unique_blocks=False -table_a_b -count 1 -table_when_b_even -count 10 -0 -0 -table_a_b -count 1 -table_when_b_even -count 20 -0 -0 -OK - -Test case 20: insert_method=InsertSelect engine=MergeTree use_insert_token=False single_thread=True deduplicate_src_table=False deduplicate_dst_table=True insert_unique_blocks=True -table_a_b -count 10 -table_when_b_even -count 5 -0 -0 -table_a_b -count 20 -table_when_b_even -count 5 -0 -0 -OK - -Test case 21: insert_method=InsertSelect engine=MergeTree use_insert_token=False single_thread=True deduplicate_src_table=False deduplicate_dst_table=True insert_unique_blocks=False -table_a_b -count 10 -table_when_b_even -count 1 -0 -0 -table_a_b -count 20 -table_when_b_even -count 1 -0 -0 -OK - -Test case 22: insert_method=InsertSelect engine=MergeTree use_insert_token=False single_thread=True deduplicate_src_table=False deduplicate_dst_table=False insert_unique_blocks=True -table_a_b -count 10 -table_when_b_even -count 5 -0 -0 -table_a_b -count 20 -table_when_b_even -count 10 -0 -0 -OK - -Test case 23: insert_method=InsertSelect engine=MergeTree use_insert_token=False single_thread=True deduplicate_src_table=False deduplicate_dst_table=False insert_unique_blocks=False -table_a_b -count 10 -table_when_b_even -count 10 -0 -0 -table_a_b -count 20 -table_when_b_even -count 20 -0 -0 -OK - -Test case 24: insert_method=InsertSelect engine=MergeTree use_insert_token=False single_thread=False deduplicate_src_table=True deduplicate_dst_table=True insert_unique_blocks=True -table_a_b -count 10 -table_when_b_even -count 5 -0 -0 -table_a_b -count 10 -table_when_b_even -count 5 -0 -0 -OK - -Test case 25: insert_method=InsertSelect engine=MergeTree use_insert_token=False single_thread=False deduplicate_src_table=True deduplicate_dst_table=True insert_unique_blocks=False -table_a_b -count 1 -table_when_b_even -count 1 -0 -0 -table_a_b -count 1 -table_when_b_even -count 1 -0 -0 -OK - -Test case 26: insert_method=InsertSelect engine=MergeTree use_insert_token=False single_thread=False deduplicate_src_table=True deduplicate_dst_table=False insert_unique_blocks=True -table_a_b -count 10 -table_when_b_even -count 5 -0 -0 -table_a_b -count 10 -table_when_b_even -count 10 -0 -0 -OK - -Test case 27: insert_method=InsertSelect engine=MergeTree use_insert_token=False single_thread=False deduplicate_src_table=True deduplicate_dst_table=False insert_unique_blocks=False -table_a_b -count 1 -table_when_b_even -count 10 -0 -0 -table_a_b -count 1 -table_when_b_even -count 20 -0 -0 -OK - -Test case 28: insert_method=InsertSelect engine=MergeTree use_insert_token=False single_thread=False deduplicate_src_table=False deduplicate_dst_table=True insert_unique_blocks=True -table_a_b -count 10 -table_when_b_even -count 5 -0 -0 -table_a_b -count 20 -table_when_b_even -count 5 -0 -0 -OK - -Test case 29: insert_method=InsertSelect engine=MergeTree use_insert_token=False single_thread=False deduplicate_src_table=False deduplicate_dst_table=True insert_unique_blocks=False -table_a_b -count 10 -table_when_b_even -count 1 -0 -0 -table_a_b -count 20 -table_when_b_even -count 1 -0 -0 -OK - -Test case 30: insert_method=InsertSelect engine=MergeTree use_insert_token=False single_thread=False deduplicate_src_table=False deduplicate_dst_table=False insert_unique_blocks=True -table_a_b -count 10 -table_when_b_even -count 5 -0 -0 -table_a_b -count 20 -table_when_b_even -count 10 -0 -0 -OK - -Test case 31: insert_method=InsertSelect engine=MergeTree use_insert_token=False single_thread=False deduplicate_src_table=False deduplicate_dst_table=False insert_unique_blocks=False -table_a_b -count 10 -table_when_b_even -count 10 -0 -0 -table_a_b -count 20 -table_when_b_even -count 20 -0 -0 -OK - -Test case 32: insert_method=InsertValues engine=MergeTree use_insert_token=True single_thread=True deduplicate_src_table=True deduplicate_dst_table=True insert_unique_blocks=True -table_a_b -count 10 -table_when_b_even -count 5 -0 -0 -table_a_b -count 10 -table_when_b_even -count 5 -0 -0 -OK - -Test case 33: insert_method=InsertValues engine=MergeTree use_insert_token=True single_thread=True deduplicate_src_table=True deduplicate_dst_table=True insert_unique_blocks=False -table_a_b -count 10 -table_when_b_even -count 10 -0 -0 -table_a_b -count 10 -table_when_b_even -count 10 -0 -0 -OK - -Test case 34: insert_method=InsertValues engine=MergeTree use_insert_token=True single_thread=True deduplicate_src_table=True deduplicate_dst_table=False insert_unique_blocks=True -table_a_b -count 10 -table_when_b_even -count 5 -0 -0 -table_a_b -count 10 -table_when_b_even -count 10 -0 -0 -OK - -Test case 35: insert_method=InsertValues engine=MergeTree use_insert_token=True single_thread=True deduplicate_src_table=True deduplicate_dst_table=False insert_unique_blocks=False -table_a_b -count 10 -table_when_b_even -count 10 -0 -0 -table_a_b -count 10 -table_when_b_even -count 20 -0 -0 -OK - -Test case 36: insert_method=InsertValues engine=MergeTree use_insert_token=True single_thread=True deduplicate_src_table=False deduplicate_dst_table=True insert_unique_blocks=True -table_a_b -count 10 -table_when_b_even -count 5 -0 -0 -table_a_b -count 20 -table_when_b_even -count 5 -0 -0 -OK - -Test case 37: insert_method=InsertValues engine=MergeTree use_insert_token=True single_thread=True deduplicate_src_table=False deduplicate_dst_table=True insert_unique_blocks=False -table_a_b -count 10 -table_when_b_even -count 10 -0 -0 -table_a_b -count 20 -table_when_b_even -count 10 -0 -0 -OK - -Test case 38: insert_method=InsertValues engine=MergeTree use_insert_token=True single_thread=True deduplicate_src_table=False deduplicate_dst_table=False insert_unique_blocks=True -table_a_b -count 10 -table_when_b_even -count 5 -0 -0 -table_a_b -count 20 -table_when_b_even -count 10 -0 -0 -OK - -Test case 39: insert_method=InsertValues engine=MergeTree use_insert_token=True single_thread=True deduplicate_src_table=False deduplicate_dst_table=False insert_unique_blocks=False -table_a_b -count 10 -table_when_b_even -count 10 -0 -0 -table_a_b -count 20 -table_when_b_even -count 20 -0 -0 -OK - -Test case 40: insert_method=InsertValues engine=MergeTree use_insert_token=True single_thread=False deduplicate_src_table=True deduplicate_dst_table=True insert_unique_blocks=True -table_a_b -count 10 -table_when_b_even -count 5 -0 -0 -table_a_b -count 10 -table_when_b_even -count 5 -0 -0 -OK - -Test case 41: insert_method=InsertValues engine=MergeTree use_insert_token=True single_thread=False deduplicate_src_table=True deduplicate_dst_table=True insert_unique_blocks=False -table_a_b -count 10 -table_when_b_even -count 10 -0 -0 -table_a_b -count 10 -table_when_b_even -count 10 -0 -0 -OK - -Test case 42: insert_method=InsertValues engine=MergeTree use_insert_token=True single_thread=False deduplicate_src_table=True deduplicate_dst_table=False insert_unique_blocks=True -table_a_b -count 10 -table_when_b_even -count 5 -0 -0 -table_a_b -count 10 -table_when_b_even -count 10 -0 -0 -OK - -Test case 43: insert_method=InsertValues engine=MergeTree use_insert_token=True single_thread=False deduplicate_src_table=True deduplicate_dst_table=False insert_unique_blocks=False -table_a_b -count 10 -table_when_b_even -count 10 -0 -0 -table_a_b -count 10 -table_when_b_even -count 20 -0 -0 -OK - -Test case 44: insert_method=InsertValues engine=MergeTree use_insert_token=True single_thread=False deduplicate_src_table=False deduplicate_dst_table=True insert_unique_blocks=True -table_a_b -count 10 -table_when_b_even -count 5 -0 -0 -table_a_b -count 20 -table_when_b_even -count 5 -0 -0 -OK - -Test case 45: insert_method=InsertValues engine=MergeTree use_insert_token=True single_thread=False deduplicate_src_table=False deduplicate_dst_table=True insert_unique_blocks=False -table_a_b -count 10 -table_when_b_even -count 10 -0 -0 -table_a_b -count 20 -table_when_b_even -count 10 -0 -0 -OK - -Test case 46: insert_method=InsertValues engine=MergeTree use_insert_token=True single_thread=False deduplicate_src_table=False deduplicate_dst_table=False insert_unique_blocks=True -table_a_b -count 10 -table_when_b_even -count 5 -0 -0 -table_a_b -count 20 -table_when_b_even -count 10 -0 -0 -OK - -Test case 47: insert_method=InsertValues engine=MergeTree use_insert_token=True single_thread=False deduplicate_src_table=False deduplicate_dst_table=False insert_unique_blocks=False -table_a_b -count 10 -table_when_b_even -count 10 -0 -0 -table_a_b -count 20 -table_when_b_even -count 20 -0 -0 -OK - -Test case 48: insert_method=InsertValues engine=MergeTree use_insert_token=False single_thread=True deduplicate_src_table=True deduplicate_dst_table=True insert_unique_blocks=True -table_a_b -count 10 -table_when_b_even -count 5 -0 -0 -table_a_b -count 10 -table_when_b_even -count 5 -0 -0 -OK - -Test case 49: insert_method=InsertValues engine=MergeTree use_insert_token=False single_thread=True deduplicate_src_table=True deduplicate_dst_table=True insert_unique_blocks=False -table_a_b -count 1 -table_when_b_even -count 1 -0 -0 -table_a_b -count 1 -table_when_b_even -count 1 -0 -0 -OK - -Test case 50: insert_method=InsertValues engine=MergeTree use_insert_token=False single_thread=True deduplicate_src_table=True deduplicate_dst_table=False insert_unique_blocks=True -table_a_b -count 10 -table_when_b_even -count 5 -0 -0 -table_a_b -count 10 -table_when_b_even -count 10 -0 -0 -OK - -Test case 51: insert_method=InsertValues engine=MergeTree use_insert_token=False single_thread=True deduplicate_src_table=True deduplicate_dst_table=False insert_unique_blocks=False -table_a_b -count 1 -table_when_b_even -count 10 -0 -0 -table_a_b -count 1 -table_when_b_even -count 20 -0 -0 -OK - -Test case 52: insert_method=InsertValues engine=MergeTree use_insert_token=False single_thread=True deduplicate_src_table=False deduplicate_dst_table=True insert_unique_blocks=True -table_a_b -count 10 -table_when_b_even -count 5 -0 -0 -table_a_b -count 20 -table_when_b_even -count 5 -0 -0 -OK - -Test case 53: insert_method=InsertValues engine=MergeTree use_insert_token=False single_thread=True deduplicate_src_table=False deduplicate_dst_table=True insert_unique_blocks=False -table_a_b -count 10 -table_when_b_even -count 1 -0 -0 -table_a_b -count 20 -table_when_b_even -count 1 -0 -0 -OK - -Test case 54: insert_method=InsertValues engine=MergeTree use_insert_token=False single_thread=True deduplicate_src_table=False deduplicate_dst_table=False insert_unique_blocks=True -table_a_b -count 10 -table_when_b_even -count 5 -0 -0 -table_a_b -count 20 -table_when_b_even -count 10 -0 -0 -OK - -Test case 55: insert_method=InsertValues engine=MergeTree use_insert_token=False single_thread=True deduplicate_src_table=False deduplicate_dst_table=False insert_unique_blocks=False -table_a_b -count 10 -table_when_b_even -count 10 -0 -0 -table_a_b -count 20 -table_when_b_even -count 20 -0 -0 -OK - -Test case 56: insert_method=InsertValues engine=MergeTree use_insert_token=False single_thread=False deduplicate_src_table=True deduplicate_dst_table=True insert_unique_blocks=True -table_a_b -count 10 -table_when_b_even -count 5 -0 -0 -table_a_b -count 10 -table_when_b_even -count 5 -0 -0 -OK - -Test case 57: insert_method=InsertValues engine=MergeTree use_insert_token=False single_thread=False deduplicate_src_table=True deduplicate_dst_table=True insert_unique_blocks=False -table_a_b -count 1 -table_when_b_even -count 1 -0 -0 -table_a_b -count 1 -table_when_b_even -count 1 -0 -0 -OK - -Test case 58: insert_method=InsertValues engine=MergeTree use_insert_token=False single_thread=False deduplicate_src_table=True deduplicate_dst_table=False insert_unique_blocks=True -table_a_b -count 10 -table_when_b_even -count 5 -0 -0 -table_a_b -count 10 -table_when_b_even -count 10 -0 -0 -OK - -Test case 59: insert_method=InsertValues engine=MergeTree use_insert_token=False single_thread=False deduplicate_src_table=True deduplicate_dst_table=False insert_unique_blocks=False -table_a_b -count 1 -table_when_b_even -count 10 -0 -0 -table_a_b -count 1 -table_when_b_even -count 20 -0 -0 -OK - -Test case 60: insert_method=InsertValues engine=MergeTree use_insert_token=False single_thread=False deduplicate_src_table=False deduplicate_dst_table=True insert_unique_blocks=True -table_a_b -count 10 -table_when_b_even -count 5 -0 -0 -table_a_b -count 20 -table_when_b_even -count 5 -0 -0 -OK - -Test case 61: insert_method=InsertValues engine=MergeTree use_insert_token=False single_thread=False deduplicate_src_table=False deduplicate_dst_table=True insert_unique_blocks=False -table_a_b -count 10 -table_when_b_even -count 1 -0 -0 -table_a_b -count 20 -table_when_b_even -count 1 -0 -0 -OK - -Test case 62: insert_method=InsertValues engine=MergeTree use_insert_token=False single_thread=False deduplicate_src_table=False deduplicate_dst_table=False insert_unique_blocks=True -table_a_b -count 10 -table_when_b_even -count 5 -0 -0 -table_a_b -count 20 -table_when_b_even -count 10 -0 -0 -OK - -Test case 63: insert_method=InsertValues engine=MergeTree use_insert_token=False single_thread=False deduplicate_src_table=False deduplicate_dst_table=False insert_unique_blocks=False -table_a_b -count 10 -table_when_b_even -count 10 -0 -0 -table_a_b -count 20 -table_when_b_even -count 20 -0 -0 -OK - -All cases executed diff --git a/tests/queries/0_stateless/03008_deduplication_insert_several_blocks_nonreplicated.sh b/tests/queries/0_stateless/03008_deduplication_insert_several_blocks_nonreplicated.sh deleted file mode 100755 index 49eb52b47fd..00000000000 --- a/tests/queries/0_stateless/03008_deduplication_insert_several_blocks_nonreplicated.sh +++ /dev/null @@ -1,59 +0,0 @@ -#!/usr/bin/env bash -# Tags: long, no-fasttest, no-parallel - -CURDIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd) -# shellcheck source=../shell_config.sh -. "$CURDIR"/../shell_config.sh - -ENGINE="MergeTree" - -RUN_ONLY="" -#RUN_ONLY="Test case 52: insert_method=InsertSelect engine=MergeTree use_insert_token=False single_thread=True deduplicate_src_table=False deduplicate_dst_table=True insert_unique_blocks=True" - -i=0 -for insert_method in "InsertSelect" "InsertValues"; do - for use_insert_token in "True" "False"; do - for single_thread in "True" "False"; do - for deduplicate_src_table in "True" "False"; do - for deduplicate_dst_table in "True" "False"; do - for insert_unique_blocks in "True" "False"; do - - THIS_RUN="Test case $i:" - THIS_RUN+=" insert_method=$insert_method" - THIS_RUN+=" engine=$ENGINE" - THIS_RUN+=" use_insert_token=$use_insert_token" - THIS_RUN+=" single_thread=$single_thread" - THIS_RUN+=" deduplicate_src_table=$deduplicate_src_table" - THIS_RUN+=" deduplicate_dst_table=$deduplicate_dst_table" - THIS_RUN+=" insert_unique_blocks=$insert_unique_blocks" - - i=$((i+1)) - - echo - if [ -n "$RUN_ONLY" ] && [ "$RUN_ONLY" != "$THIS_RUN" ]; then - echo "skip $THIS_RUN" - continue - fi - echo "$THIS_RUN" - - $CLICKHOUSE_CLIENT --max_insert_block_size 1 -nmq " - $(python3 $CURDIR/03008_deduplication.python insert_several_blocks_into_table \ - --insert-method $insert_method \ - --table-engine $ENGINE \ - --use-insert-token $use_insert_token \ - --single-thread $single_thread \ - --deduplicate-src-table $deduplicate_src_table \ - --deduplicate-dst-table $deduplicate_dst_table \ - --insert-unique-blocks $insert_unique_blocks \ - --get-logs false \ - ) - " && echo OK || echo FAIL - done - done - done - done - done -done - -echo -echo "All cases executed" diff --git a/tests/queries/0_stateless/03008_deduplication_insert_several_blocks_replicated.reference b/tests/queries/0_stateless/03008_deduplication_insert_several_blocks_replicated.reference deleted file mode 100644 index c815324b455..00000000000 --- a/tests/queries/0_stateless/03008_deduplication_insert_several_blocks_replicated.reference +++ /dev/null @@ -1,962 +0,0 @@ - -Test case 0: insert_method=InsertSelect engine=ReplicatedMergeTree use_insert_token=True single_thread=True deduplicate_src_table=True deduplicate_dst_table=True insert_unique_blocks=True -table_a_b -count 10 -table_when_b_even -count 5 -0 -0 -table_a_b -count 10 -table_when_b_even -count 5 -0 -0 -OK - -Test case 1: insert_method=InsertSelect engine=ReplicatedMergeTree use_insert_token=True single_thread=True deduplicate_src_table=True deduplicate_dst_table=True insert_unique_blocks=False -table_a_b -count 10 -table_when_b_even -count 10 -0 -0 -table_a_b -count 10 -table_when_b_even -count 10 -0 -0 -OK - -Test case 2: insert_method=InsertSelect engine=ReplicatedMergeTree use_insert_token=True single_thread=True deduplicate_src_table=True deduplicate_dst_table=False insert_unique_blocks=True -table_a_b -count 10 -table_when_b_even -count 5 -0 -0 -table_a_b -count 10 -table_when_b_even -count 10 -0 -0 -OK - -Test case 3: insert_method=InsertSelect engine=ReplicatedMergeTree use_insert_token=True single_thread=True deduplicate_src_table=True deduplicate_dst_table=False insert_unique_blocks=False -table_a_b -count 10 -table_when_b_even -count 10 -0 -0 -table_a_b -count 10 -table_when_b_even -count 20 -0 -0 -OK - -Test case 4: insert_method=InsertSelect engine=ReplicatedMergeTree use_insert_token=True single_thread=True deduplicate_src_table=False deduplicate_dst_table=True insert_unique_blocks=True -table_a_b -count 10 -table_when_b_even -count 5 -0 -0 -table_a_b -count 20 -table_when_b_even -count 5 -0 -0 -OK - -Test case 5: insert_method=InsertSelect engine=ReplicatedMergeTree use_insert_token=True single_thread=True deduplicate_src_table=False deduplicate_dst_table=True insert_unique_blocks=False -table_a_b -count 10 -table_when_b_even -count 10 -0 -0 -table_a_b -count 20 -table_when_b_even -count 10 -0 -0 -OK - -Test case 6: insert_method=InsertSelect engine=ReplicatedMergeTree use_insert_token=True single_thread=True deduplicate_src_table=False deduplicate_dst_table=False insert_unique_blocks=True -table_a_b -count 10 -table_when_b_even -count 5 -0 -0 -table_a_b -count 20 -table_when_b_even -count 10 -0 -0 -OK - -Test case 7: insert_method=InsertSelect engine=ReplicatedMergeTree use_insert_token=True single_thread=True deduplicate_src_table=False deduplicate_dst_table=False insert_unique_blocks=False -table_a_b -count 10 -table_when_b_even -count 10 -0 -0 -table_a_b -count 20 -table_when_b_even -count 20 -0 -0 -OK - -Test case 8: insert_method=InsertSelect engine=ReplicatedMergeTree use_insert_token=True single_thread=False deduplicate_src_table=True deduplicate_dst_table=True insert_unique_blocks=True -table_a_b -count 10 -table_when_b_even -count 5 -0 -0 -table_a_b -count 10 -table_when_b_even -count 5 -0 -0 -OK - -Test case 9: insert_method=InsertSelect engine=ReplicatedMergeTree use_insert_token=True single_thread=False deduplicate_src_table=True deduplicate_dst_table=True insert_unique_blocks=False -table_a_b -count 10 -table_when_b_even -count 10 -0 -0 -table_a_b -count 10 -table_when_b_even -count 10 -0 -0 -OK - -Test case 10: insert_method=InsertSelect engine=ReplicatedMergeTree use_insert_token=True single_thread=False deduplicate_src_table=True deduplicate_dst_table=False insert_unique_blocks=True -table_a_b -count 10 -table_when_b_even -count 5 -0 -0 -table_a_b -count 10 -table_when_b_even -count 10 -0 -0 -OK - -Test case 11: insert_method=InsertSelect engine=ReplicatedMergeTree use_insert_token=True single_thread=False deduplicate_src_table=True deduplicate_dst_table=False insert_unique_blocks=False -table_a_b -count 10 -table_when_b_even -count 10 -0 -0 -table_a_b -count 10 -table_when_b_even -count 20 -0 -0 -OK - -Test case 12: insert_method=InsertSelect engine=ReplicatedMergeTree use_insert_token=True single_thread=False deduplicate_src_table=False deduplicate_dst_table=True insert_unique_blocks=True -table_a_b -count 10 -table_when_b_even -count 5 -0 -0 -table_a_b -count 20 -table_when_b_even -count 5 -0 -0 -OK - -Test case 13: insert_method=InsertSelect engine=ReplicatedMergeTree use_insert_token=True single_thread=False deduplicate_src_table=False deduplicate_dst_table=True insert_unique_blocks=False -table_a_b -count 10 -table_when_b_even -count 10 -0 -0 -table_a_b -count 20 -table_when_b_even -count 10 -0 -0 -OK - -Test case 14: insert_method=InsertSelect engine=ReplicatedMergeTree use_insert_token=True single_thread=False deduplicate_src_table=False deduplicate_dst_table=False insert_unique_blocks=True -table_a_b -count 10 -table_when_b_even -count 5 -0 -0 -table_a_b -count 20 -table_when_b_even -count 10 -0 -0 -OK - -Test case 15: insert_method=InsertSelect engine=ReplicatedMergeTree use_insert_token=True single_thread=False deduplicate_src_table=False deduplicate_dst_table=False insert_unique_blocks=False -table_a_b -count 10 -table_when_b_even -count 10 -0 -0 -table_a_b -count 20 -table_when_b_even -count 20 -0 -0 -OK - -Test case 16: insert_method=InsertSelect engine=ReplicatedMergeTree use_insert_token=False single_thread=True deduplicate_src_table=True deduplicate_dst_table=True insert_unique_blocks=True -table_a_b -count 10 -table_when_b_even -count 5 -0 -0 -table_a_b -count 10 -table_when_b_even -count 5 -0 -0 -OK - -Test case 17: insert_method=InsertSelect engine=ReplicatedMergeTree use_insert_token=False single_thread=True deduplicate_src_table=True deduplicate_dst_table=True insert_unique_blocks=False -table_a_b -count 1 -table_when_b_even -count 1 -0 -0 -table_a_b -count 1 -table_when_b_even -count 1 -0 -0 -OK - -Test case 18: insert_method=InsertSelect engine=ReplicatedMergeTree use_insert_token=False single_thread=True deduplicate_src_table=True deduplicate_dst_table=False insert_unique_blocks=True -table_a_b -count 10 -table_when_b_even -count 5 -0 -0 -table_a_b -count 10 -table_when_b_even -count 10 -0 -0 -OK - -Test case 19: insert_method=InsertSelect engine=ReplicatedMergeTree use_insert_token=False single_thread=True deduplicate_src_table=True deduplicate_dst_table=False insert_unique_blocks=False -table_a_b -count 1 -table_when_b_even -count 10 -0 -0 -table_a_b -count 1 -table_when_b_even -count 20 -0 -0 -OK - -Test case 20: insert_method=InsertSelect engine=ReplicatedMergeTree use_insert_token=False single_thread=True deduplicate_src_table=False deduplicate_dst_table=True insert_unique_blocks=True -table_a_b -count 10 -table_when_b_even -count 5 -0 -0 -table_a_b -count 20 -table_when_b_even -count 5 -0 -0 -OK - -Test case 21: insert_method=InsertSelect engine=ReplicatedMergeTree use_insert_token=False single_thread=True deduplicate_src_table=False deduplicate_dst_table=True insert_unique_blocks=False -table_a_b -count 10 -table_when_b_even -count 1 -0 -0 -table_a_b -count 20 -table_when_b_even -count 1 -0 -0 -OK - -Test case 22: insert_method=InsertSelect engine=ReplicatedMergeTree use_insert_token=False single_thread=True deduplicate_src_table=False deduplicate_dst_table=False insert_unique_blocks=True -table_a_b -count 10 -table_when_b_even -count 5 -0 -0 -table_a_b -count 20 -table_when_b_even -count 10 -0 -0 -OK - -Test case 23: insert_method=InsertSelect engine=ReplicatedMergeTree use_insert_token=False single_thread=True deduplicate_src_table=False deduplicate_dst_table=False insert_unique_blocks=False -table_a_b -count 10 -table_when_b_even -count 10 -0 -0 -table_a_b -count 20 -table_when_b_even -count 20 -0 -0 -OK - -Test case 24: insert_method=InsertSelect engine=ReplicatedMergeTree use_insert_token=False single_thread=False deduplicate_src_table=True deduplicate_dst_table=True insert_unique_blocks=True -table_a_b -count 10 -table_when_b_even -count 5 -0 -0 -table_a_b -count 10 -table_when_b_even -count 5 -0 -0 -OK - -Test case 25: insert_method=InsertSelect engine=ReplicatedMergeTree use_insert_token=False single_thread=False deduplicate_src_table=True deduplicate_dst_table=True insert_unique_blocks=False -table_a_b -count 1 -table_when_b_even -count 1 -0 -0 -table_a_b -count 1 -table_when_b_even -count 1 -0 -0 -OK - -Test case 26: insert_method=InsertSelect engine=ReplicatedMergeTree use_insert_token=False single_thread=False deduplicate_src_table=True deduplicate_dst_table=False insert_unique_blocks=True -table_a_b -count 10 -table_when_b_even -count 5 -0 -0 -table_a_b -count 10 -table_when_b_even -count 10 -0 -0 -OK - -Test case 27: insert_method=InsertSelect engine=ReplicatedMergeTree use_insert_token=False single_thread=False deduplicate_src_table=True deduplicate_dst_table=False insert_unique_blocks=False -table_a_b -count 1 -table_when_b_even -count 10 -0 -0 -table_a_b -count 1 -table_when_b_even -count 20 -0 -0 -OK - -Test case 28: insert_method=InsertSelect engine=ReplicatedMergeTree use_insert_token=False single_thread=False deduplicate_src_table=False deduplicate_dst_table=True insert_unique_blocks=True -table_a_b -count 10 -table_when_b_even -count 5 -0 -0 -table_a_b -count 20 -table_when_b_even -count 5 -0 -0 -OK - -Test case 29: insert_method=InsertSelect engine=ReplicatedMergeTree use_insert_token=False single_thread=False deduplicate_src_table=False deduplicate_dst_table=True insert_unique_blocks=False -table_a_b -count 10 -table_when_b_even -count 1 -0 -0 -table_a_b -count 20 -table_when_b_even -count 1 -0 -0 -OK - -Test case 30: insert_method=InsertSelect engine=ReplicatedMergeTree use_insert_token=False single_thread=False deduplicate_src_table=False deduplicate_dst_table=False insert_unique_blocks=True -table_a_b -count 10 -table_when_b_even -count 5 -0 -0 -table_a_b -count 20 -table_when_b_even -count 10 -0 -0 -OK - -Test case 31: insert_method=InsertSelect engine=ReplicatedMergeTree use_insert_token=False single_thread=False deduplicate_src_table=False deduplicate_dst_table=False insert_unique_blocks=False -table_a_b -count 10 -table_when_b_even -count 10 -0 -0 -table_a_b -count 20 -table_when_b_even -count 20 -0 -0 -OK - -Test case 32: insert_method=InsertValues engine=ReplicatedMergeTree use_insert_token=True single_thread=True deduplicate_src_table=True deduplicate_dst_table=True insert_unique_blocks=True -table_a_b -count 10 -table_when_b_even -count 5 -0 -0 -table_a_b -count 10 -table_when_b_even -count 5 -0 -0 -OK - -Test case 33: insert_method=InsertValues engine=ReplicatedMergeTree use_insert_token=True single_thread=True deduplicate_src_table=True deduplicate_dst_table=True insert_unique_blocks=False -table_a_b -count 10 -table_when_b_even -count 10 -0 -0 -table_a_b -count 10 -table_when_b_even -count 10 -0 -0 -OK - -Test case 34: insert_method=InsertValues engine=ReplicatedMergeTree use_insert_token=True single_thread=True deduplicate_src_table=True deduplicate_dst_table=False insert_unique_blocks=True -table_a_b -count 10 -table_when_b_even -count 5 -0 -0 -table_a_b -count 10 -table_when_b_even -count 10 -0 -0 -OK - -Test case 35: insert_method=InsertValues engine=ReplicatedMergeTree use_insert_token=True single_thread=True deduplicate_src_table=True deduplicate_dst_table=False insert_unique_blocks=False -table_a_b -count 10 -table_when_b_even -count 10 -0 -0 -table_a_b -count 10 -table_when_b_even -count 20 -0 -0 -OK - -Test case 36: insert_method=InsertValues engine=ReplicatedMergeTree use_insert_token=True single_thread=True deduplicate_src_table=False deduplicate_dst_table=True insert_unique_blocks=True -table_a_b -count 10 -table_when_b_even -count 5 -0 -0 -table_a_b -count 20 -table_when_b_even -count 5 -0 -0 -OK - -Test case 37: insert_method=InsertValues engine=ReplicatedMergeTree use_insert_token=True single_thread=True deduplicate_src_table=False deduplicate_dst_table=True insert_unique_blocks=False -table_a_b -count 10 -table_when_b_even -count 10 -0 -0 -table_a_b -count 20 -table_when_b_even -count 10 -0 -0 -OK - -Test case 38: insert_method=InsertValues engine=ReplicatedMergeTree use_insert_token=True single_thread=True deduplicate_src_table=False deduplicate_dst_table=False insert_unique_blocks=True -table_a_b -count 10 -table_when_b_even -count 5 -0 -0 -table_a_b -count 20 -table_when_b_even -count 10 -0 -0 -OK - -Test case 39: insert_method=InsertValues engine=ReplicatedMergeTree use_insert_token=True single_thread=True deduplicate_src_table=False deduplicate_dst_table=False insert_unique_blocks=False -table_a_b -count 10 -table_when_b_even -count 10 -0 -0 -table_a_b -count 20 -table_when_b_even -count 20 -0 -0 -OK - -Test case 40: insert_method=InsertValues engine=ReplicatedMergeTree use_insert_token=True single_thread=False deduplicate_src_table=True deduplicate_dst_table=True insert_unique_blocks=True -table_a_b -count 10 -table_when_b_even -count 5 -0 -0 -table_a_b -count 10 -table_when_b_even -count 5 -0 -0 -OK - -Test case 41: insert_method=InsertValues engine=ReplicatedMergeTree use_insert_token=True single_thread=False deduplicate_src_table=True deduplicate_dst_table=True insert_unique_blocks=False -table_a_b -count 10 -table_when_b_even -count 10 -0 -0 -table_a_b -count 10 -table_when_b_even -count 10 -0 -0 -OK - -Test case 42: insert_method=InsertValues engine=ReplicatedMergeTree use_insert_token=True single_thread=False deduplicate_src_table=True deduplicate_dst_table=False insert_unique_blocks=True -table_a_b -count 10 -table_when_b_even -count 5 -0 -0 -table_a_b -count 10 -table_when_b_even -count 10 -0 -0 -OK - -Test case 43: insert_method=InsertValues engine=ReplicatedMergeTree use_insert_token=True single_thread=False deduplicate_src_table=True deduplicate_dst_table=False insert_unique_blocks=False -table_a_b -count 10 -table_when_b_even -count 10 -0 -0 -table_a_b -count 10 -table_when_b_even -count 20 -0 -0 -OK - -Test case 44: insert_method=InsertValues engine=ReplicatedMergeTree use_insert_token=True single_thread=False deduplicate_src_table=False deduplicate_dst_table=True insert_unique_blocks=True -table_a_b -count 10 -table_when_b_even -count 5 -0 -0 -table_a_b -count 20 -table_when_b_even -count 5 -0 -0 -OK - -Test case 45: insert_method=InsertValues engine=ReplicatedMergeTree use_insert_token=True single_thread=False deduplicate_src_table=False deduplicate_dst_table=True insert_unique_blocks=False -table_a_b -count 10 -table_when_b_even -count 10 -0 -0 -table_a_b -count 20 -table_when_b_even -count 10 -0 -0 -OK - -Test case 46: insert_method=InsertValues engine=ReplicatedMergeTree use_insert_token=True single_thread=False deduplicate_src_table=False deduplicate_dst_table=False insert_unique_blocks=True -table_a_b -count 10 -table_when_b_even -count 5 -0 -0 -table_a_b -count 20 -table_when_b_even -count 10 -0 -0 -OK - -Test case 47: insert_method=InsertValues engine=ReplicatedMergeTree use_insert_token=True single_thread=False deduplicate_src_table=False deduplicate_dst_table=False insert_unique_blocks=False -table_a_b -count 10 -table_when_b_even -count 10 -0 -0 -table_a_b -count 20 -table_when_b_even -count 20 -0 -0 -OK - -Test case 48: insert_method=InsertValues engine=ReplicatedMergeTree use_insert_token=False single_thread=True deduplicate_src_table=True deduplicate_dst_table=True insert_unique_blocks=True -table_a_b -count 10 -table_when_b_even -count 5 -0 -0 -table_a_b -count 10 -table_when_b_even -count 5 -0 -0 -OK - -Test case 49: insert_method=InsertValues engine=ReplicatedMergeTree use_insert_token=False single_thread=True deduplicate_src_table=True deduplicate_dst_table=True insert_unique_blocks=False -table_a_b -count 1 -table_when_b_even -count 1 -0 -0 -table_a_b -count 1 -table_when_b_even -count 1 -0 -0 -OK - -Test case 50: insert_method=InsertValues engine=ReplicatedMergeTree use_insert_token=False single_thread=True deduplicate_src_table=True deduplicate_dst_table=False insert_unique_blocks=True -table_a_b -count 10 -table_when_b_even -count 5 -0 -0 -table_a_b -count 10 -table_when_b_even -count 10 -0 -0 -OK - -Test case 51: insert_method=InsertValues engine=ReplicatedMergeTree use_insert_token=False single_thread=True deduplicate_src_table=True deduplicate_dst_table=False insert_unique_blocks=False -table_a_b -count 1 -table_when_b_even -count 10 -0 -0 -table_a_b -count 1 -table_when_b_even -count 20 -0 -0 -OK - -Test case 52: insert_method=InsertValues engine=ReplicatedMergeTree use_insert_token=False single_thread=True deduplicate_src_table=False deduplicate_dst_table=True insert_unique_blocks=True -table_a_b -count 10 -table_when_b_even -count 5 -0 -0 -table_a_b -count 20 -table_when_b_even -count 5 -0 -0 -OK - -Test case 53: insert_method=InsertValues engine=ReplicatedMergeTree use_insert_token=False single_thread=True deduplicate_src_table=False deduplicate_dst_table=True insert_unique_blocks=False -table_a_b -count 10 -table_when_b_even -count 1 -0 -0 -table_a_b -count 20 -table_when_b_even -count 1 -0 -0 -OK - -Test case 54: insert_method=InsertValues engine=ReplicatedMergeTree use_insert_token=False single_thread=True deduplicate_src_table=False deduplicate_dst_table=False insert_unique_blocks=True -table_a_b -count 10 -table_when_b_even -count 5 -0 -0 -table_a_b -count 20 -table_when_b_even -count 10 -0 -0 -OK - -Test case 55: insert_method=InsertValues engine=ReplicatedMergeTree use_insert_token=False single_thread=True deduplicate_src_table=False deduplicate_dst_table=False insert_unique_blocks=False -table_a_b -count 10 -table_when_b_even -count 10 -0 -0 -table_a_b -count 20 -table_when_b_even -count 20 -0 -0 -OK - -Test case 56: insert_method=InsertValues engine=ReplicatedMergeTree use_insert_token=False single_thread=False deduplicate_src_table=True deduplicate_dst_table=True insert_unique_blocks=True -table_a_b -count 10 -table_when_b_even -count 5 -0 -0 -table_a_b -count 10 -table_when_b_even -count 5 -0 -0 -OK - -Test case 57: insert_method=InsertValues engine=ReplicatedMergeTree use_insert_token=False single_thread=False deduplicate_src_table=True deduplicate_dst_table=True insert_unique_blocks=False -table_a_b -count 1 -table_when_b_even -count 1 -0 -0 -table_a_b -count 1 -table_when_b_even -count 1 -0 -0 -OK - -Test case 58: insert_method=InsertValues engine=ReplicatedMergeTree use_insert_token=False single_thread=False deduplicate_src_table=True deduplicate_dst_table=False insert_unique_blocks=True -table_a_b -count 10 -table_when_b_even -count 5 -0 -0 -table_a_b -count 10 -table_when_b_even -count 10 -0 -0 -OK - -Test case 59: insert_method=InsertValues engine=ReplicatedMergeTree use_insert_token=False single_thread=False deduplicate_src_table=True deduplicate_dst_table=False insert_unique_blocks=False -table_a_b -count 1 -table_when_b_even -count 10 -0 -0 -table_a_b -count 1 -table_when_b_even -count 20 -0 -0 -OK - -Test case 60: insert_method=InsertValues engine=ReplicatedMergeTree use_insert_token=False single_thread=False deduplicate_src_table=False deduplicate_dst_table=True insert_unique_blocks=True -table_a_b -count 10 -table_when_b_even -count 5 -0 -0 -table_a_b -count 20 -table_when_b_even -count 5 -0 -0 -OK - -Test case 61: insert_method=InsertValues engine=ReplicatedMergeTree use_insert_token=False single_thread=False deduplicate_src_table=False deduplicate_dst_table=True insert_unique_blocks=False -table_a_b -count 10 -table_when_b_even -count 1 -0 -0 -table_a_b -count 20 -table_when_b_even -count 1 -0 -0 -OK - -Test case 62: insert_method=InsertValues engine=ReplicatedMergeTree use_insert_token=False single_thread=False deduplicate_src_table=False deduplicate_dst_table=False insert_unique_blocks=True -table_a_b -count 10 -table_when_b_even -count 5 -0 -0 -table_a_b -count 20 -table_when_b_even -count 10 -0 -0 -OK - -Test case 63: insert_method=InsertValues engine=ReplicatedMergeTree use_insert_token=False single_thread=False deduplicate_src_table=False deduplicate_dst_table=False insert_unique_blocks=False -table_a_b -count 10 -table_when_b_even -count 10 -0 -0 -table_a_b -count 20 -table_when_b_even -count 20 -0 -0 -OK - -All cases executed diff --git a/tests/queries/0_stateless/03008_deduplication_insert_several_blocks_replicated.sh b/tests/queries/0_stateless/03008_deduplication_insert_several_blocks_replicated.sh deleted file mode 100755 index 53af06d4a6f..00000000000 --- a/tests/queries/0_stateless/03008_deduplication_insert_several_blocks_replicated.sh +++ /dev/null @@ -1,59 +0,0 @@ -#!/usr/bin/env bash -# Tags: long, no-fasttest, no-parallel - -CURDIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd) -# shellcheck source=../shell_config.sh -. "$CURDIR"/../shell_config.sh - -ENGINE="ReplicatedMergeTree" - -RUN_ONLY="" -#RUN_ONLY="Test case 52: insert_method=InsertSelect engine=ReplicatedMergeTree use_insert_token=False single_thread=True deduplicate_src_table=False deduplicate_dst_table=True insert_unique_blocks=True" - -i=0 -for insert_method in "InsertSelect" "InsertValues"; do - for use_insert_token in "True" "False"; do - for single_thread in "True" "False"; do - for deduplicate_src_table in "True" "False"; do - for deduplicate_dst_table in "True" "False"; do - for insert_unique_blocks in "True" "False"; do - - THIS_RUN="Test case $i:" - THIS_RUN+=" insert_method=$insert_method" - THIS_RUN+=" engine=$ENGINE" - THIS_RUN+=" use_insert_token=$use_insert_token" - THIS_RUN+=" single_thread=$single_thread" - THIS_RUN+=" deduplicate_src_table=$deduplicate_src_table" - THIS_RUN+=" deduplicate_dst_table=$deduplicate_dst_table" - THIS_RUN+=" insert_unique_blocks=$insert_unique_blocks" - - i=$((i+1)) - - echo - if [ -n "$RUN_ONLY" ] && [ "$RUN_ONLY" != "$THIS_RUN" ]; then - echo "skip $THIS_RUN" - continue - fi - echo "$THIS_RUN" - - $CLICKHOUSE_CLIENT --max_insert_block_size 1 -nmq " - $(python3 $CURDIR/03008_deduplication.python insert_several_blocks_into_table \ - --insert-method $insert_method \ - --table-engine $ENGINE \ - --use-insert-token $use_insert_token \ - --single-thread $single_thread \ - --deduplicate-src-table $deduplicate_src_table \ - --deduplicate-dst-table $deduplicate_dst_table \ - --insert-unique-blocks $insert_unique_blocks \ - --get-logs false \ - ) - " && echo OK || echo FAIL - done - done - done - done - done -done - -echo -echo "All cases executed" diff --git a/tests/queries/0_stateless/03008_deduplication_mv_generates_several_blocks_nonreplicated.reference b/tests/queries/0_stateless/03008_deduplication_mv_generates_several_blocks_nonreplicated.reference deleted file mode 100644 index 6e76ec46aa8..00000000000 --- a/tests/queries/0_stateless/03008_deduplication_mv_generates_several_blocks_nonreplicated.reference +++ /dev/null @@ -1,962 +0,0 @@ - -Test case 0: insert_method=InsertSelect engine=MergeTree use_insert_token=True single_thread=True deduplicate_src_table=True deduplicate_dst_table=True insert_unique_blocks=True -table_a_b -count 5 -table_when_b_even_and_joined -count 9 -0 -0 -table_a_b -count 5 -table_when_b_even_and_joined -count 9 -0 -0 -OK - -Test case 1: insert_method=InsertSelect engine=MergeTree use_insert_token=True single_thread=True deduplicate_src_table=True deduplicate_dst_table=True insert_unique_blocks=False -table_a_b -count 5 -table_when_b_even_and_joined -count 10 -0 -0 -table_a_b -count 5 -table_when_b_even_and_joined -count 10 -0 -0 -OK - -Test case 2: insert_method=InsertSelect engine=MergeTree use_insert_token=True single_thread=True deduplicate_src_table=True deduplicate_dst_table=False insert_unique_blocks=True -table_a_b -count 5 -table_when_b_even_and_joined -count 9 -0 -0 -table_a_b -count 5 -table_when_b_even_and_joined -count 18 -0 -0 -OK - -Test case 3: insert_method=InsertSelect engine=MergeTree use_insert_token=True single_thread=True deduplicate_src_table=True deduplicate_dst_table=False insert_unique_blocks=False -table_a_b -count 5 -table_when_b_even_and_joined -count 10 -0 -0 -table_a_b -count 5 -table_when_b_even_and_joined -count 20 -0 -0 -OK - -Test case 4: insert_method=InsertSelect engine=MergeTree use_insert_token=True single_thread=True deduplicate_src_table=False deduplicate_dst_table=True insert_unique_blocks=True -table_a_b -count 5 -table_when_b_even_and_joined -count 9 -0 -0 -table_a_b -count 10 -table_when_b_even_and_joined -count 9 -0 -0 -OK - -Test case 5: insert_method=InsertSelect engine=MergeTree use_insert_token=True single_thread=True deduplicate_src_table=False deduplicate_dst_table=True insert_unique_blocks=False -table_a_b -count 5 -table_when_b_even_and_joined -count 10 -0 -0 -table_a_b -count 10 -table_when_b_even_and_joined -count 10 -0 -0 -OK - -Test case 6: insert_method=InsertSelect engine=MergeTree use_insert_token=True single_thread=True deduplicate_src_table=False deduplicate_dst_table=False insert_unique_blocks=True -table_a_b -count 5 -table_when_b_even_and_joined -count 9 -0 -0 -table_a_b -count 10 -table_when_b_even_and_joined -count 18 -0 -0 -OK - -Test case 7: insert_method=InsertSelect engine=MergeTree use_insert_token=True single_thread=True deduplicate_src_table=False deduplicate_dst_table=False insert_unique_blocks=False -table_a_b -count 5 -table_when_b_even_and_joined -count 10 -0 -0 -table_a_b -count 10 -table_when_b_even_and_joined -count 20 -0 -0 -OK - -Test case 8: insert_method=InsertSelect engine=MergeTree use_insert_token=True single_thread=False deduplicate_src_table=True deduplicate_dst_table=True insert_unique_blocks=True -table_a_b -count 5 -table_when_b_even_and_joined -count 9 -0 -0 -table_a_b -count 5 -table_when_b_even_and_joined -count 9 -0 -0 -OK - -Test case 9: insert_method=InsertSelect engine=MergeTree use_insert_token=True single_thread=False deduplicate_src_table=True deduplicate_dst_table=True insert_unique_blocks=False -table_a_b -count 5 -table_when_b_even_and_joined -count 10 -0 -0 -table_a_b -count 5 -table_when_b_even_and_joined -count 10 -0 -0 -OK - -Test case 10: insert_method=InsertSelect engine=MergeTree use_insert_token=True single_thread=False deduplicate_src_table=True deduplicate_dst_table=False insert_unique_blocks=True -table_a_b -count 5 -table_when_b_even_and_joined -count 9 -0 -0 -table_a_b -count 5 -table_when_b_even_and_joined -count 18 -0 -0 -OK - -Test case 11: insert_method=InsertSelect engine=MergeTree use_insert_token=True single_thread=False deduplicate_src_table=True deduplicate_dst_table=False insert_unique_blocks=False -table_a_b -count 5 -table_when_b_even_and_joined -count 10 -0 -0 -table_a_b -count 5 -table_when_b_even_and_joined -count 20 -0 -0 -OK - -Test case 12: insert_method=InsertSelect engine=MergeTree use_insert_token=True single_thread=False deduplicate_src_table=False deduplicate_dst_table=True insert_unique_blocks=True -table_a_b -count 5 -table_when_b_even_and_joined -count 9 -0 -0 -table_a_b -count 10 -table_when_b_even_and_joined -count 9 -0 -0 -OK - -Test case 13: insert_method=InsertSelect engine=MergeTree use_insert_token=True single_thread=False deduplicate_src_table=False deduplicate_dst_table=True insert_unique_blocks=False -table_a_b -count 5 -table_when_b_even_and_joined -count 10 -0 -0 -table_a_b -count 10 -table_when_b_even_and_joined -count 10 -0 -0 -OK - -Test case 14: insert_method=InsertSelect engine=MergeTree use_insert_token=True single_thread=False deduplicate_src_table=False deduplicate_dst_table=False insert_unique_blocks=True -table_a_b -count 5 -table_when_b_even_and_joined -count 9 -0 -0 -table_a_b -count 10 -table_when_b_even_and_joined -count 18 -0 -0 -OK - -Test case 15: insert_method=InsertSelect engine=MergeTree use_insert_token=True single_thread=False deduplicate_src_table=False deduplicate_dst_table=False insert_unique_blocks=False -table_a_b -count 5 -table_when_b_even_and_joined -count 10 -0 -0 -table_a_b -count 10 -table_when_b_even_and_joined -count 20 -0 -0 -OK - -Test case 16: insert_method=InsertSelect engine=MergeTree use_insert_token=False single_thread=True deduplicate_src_table=True deduplicate_dst_table=True insert_unique_blocks=True -table_a_b -count 5 -table_when_b_even_and_joined -count 9 -0 -0 -table_a_b -count 5 -table_when_b_even_and_joined -count 9 -0 -0 -OK - -Test case 17: insert_method=InsertSelect engine=MergeTree use_insert_token=False single_thread=True deduplicate_src_table=True deduplicate_dst_table=True insert_unique_blocks=False -table_a_b -count 1 -table_when_b_even_and_joined -count 2 -0 -0 -table_a_b -count 1 -table_when_b_even_and_joined -count 2 -0 -0 -OK - -Test case 18: insert_method=InsertSelect engine=MergeTree use_insert_token=False single_thread=True deduplicate_src_table=True deduplicate_dst_table=False insert_unique_blocks=True -table_a_b -count 5 -table_when_b_even_and_joined -count 9 -0 -0 -table_a_b -count 5 -table_when_b_even_and_joined -count 18 -0 -0 -OK - -Test case 19: insert_method=InsertSelect engine=MergeTree use_insert_token=False single_thread=True deduplicate_src_table=True deduplicate_dst_table=False insert_unique_blocks=False -table_a_b -count 1 -table_when_b_even_and_joined -count 10 -0 -0 -table_a_b -count 1 -table_when_b_even_and_joined -count 20 -0 -0 -OK - -Test case 20: insert_method=InsertSelect engine=MergeTree use_insert_token=False single_thread=True deduplicate_src_table=False deduplicate_dst_table=True insert_unique_blocks=True -table_a_b -count 5 -table_when_b_even_and_joined -count 9 -0 -0 -table_a_b -count 10 -table_when_b_even_and_joined -count 9 -0 -0 -OK - -Test case 21: insert_method=InsertSelect engine=MergeTree use_insert_token=False single_thread=True deduplicate_src_table=False deduplicate_dst_table=True insert_unique_blocks=False -table_a_b -count 5 -table_when_b_even_and_joined -count 2 -0 -0 -table_a_b -count 10 -table_when_b_even_and_joined -count 2 -0 -0 -OK - -Test case 22: insert_method=InsertSelect engine=MergeTree use_insert_token=False single_thread=True deduplicate_src_table=False deduplicate_dst_table=False insert_unique_blocks=True -table_a_b -count 5 -table_when_b_even_and_joined -count 9 -0 -0 -table_a_b -count 10 -table_when_b_even_and_joined -count 18 -0 -0 -OK - -Test case 23: insert_method=InsertSelect engine=MergeTree use_insert_token=False single_thread=True deduplicate_src_table=False deduplicate_dst_table=False insert_unique_blocks=False -table_a_b -count 5 -table_when_b_even_and_joined -count 10 -0 -0 -table_a_b -count 10 -table_when_b_even_and_joined -count 20 -0 -0 -OK - -Test case 24: insert_method=InsertSelect engine=MergeTree use_insert_token=False single_thread=False deduplicate_src_table=True deduplicate_dst_table=True insert_unique_blocks=True -table_a_b -count 5 -table_when_b_even_and_joined -count 9 -0 -0 -table_a_b -count 5 -table_when_b_even_and_joined -count 9 -0 -0 -OK - -Test case 25: insert_method=InsertSelect engine=MergeTree use_insert_token=False single_thread=False deduplicate_src_table=True deduplicate_dst_table=True insert_unique_blocks=False -table_a_b -count 1 -table_when_b_even_and_joined -count 2 -0 -0 -table_a_b -count 1 -table_when_b_even_and_joined -count 2 -0 -0 -OK - -Test case 26: insert_method=InsertSelect engine=MergeTree use_insert_token=False single_thread=False deduplicate_src_table=True deduplicate_dst_table=False insert_unique_blocks=True -table_a_b -count 5 -table_when_b_even_and_joined -count 9 -0 -0 -table_a_b -count 5 -table_when_b_even_and_joined -count 18 -0 -0 -OK - -Test case 27: insert_method=InsertSelect engine=MergeTree use_insert_token=False single_thread=False deduplicate_src_table=True deduplicate_dst_table=False insert_unique_blocks=False -table_a_b -count 1 -table_when_b_even_and_joined -count 10 -0 -0 -table_a_b -count 1 -table_when_b_even_and_joined -count 20 -0 -0 -OK - -Test case 28: insert_method=InsertSelect engine=MergeTree use_insert_token=False single_thread=False deduplicate_src_table=False deduplicate_dst_table=True insert_unique_blocks=True -table_a_b -count 5 -table_when_b_even_and_joined -count 9 -0 -0 -table_a_b -count 10 -table_when_b_even_and_joined -count 9 -0 -0 -OK - -Test case 29: insert_method=InsertSelect engine=MergeTree use_insert_token=False single_thread=False deduplicate_src_table=False deduplicate_dst_table=True insert_unique_blocks=False -table_a_b -count 5 -table_when_b_even_and_joined -count 2 -0 -0 -table_a_b -count 10 -table_when_b_even_and_joined -count 2 -0 -0 -OK - -Test case 30: insert_method=InsertSelect engine=MergeTree use_insert_token=False single_thread=False deduplicate_src_table=False deduplicate_dst_table=False insert_unique_blocks=True -table_a_b -count 5 -table_when_b_even_and_joined -count 9 -0 -0 -table_a_b -count 10 -table_when_b_even_and_joined -count 18 -0 -0 -OK - -Test case 31: insert_method=InsertSelect engine=MergeTree use_insert_token=False single_thread=False deduplicate_src_table=False deduplicate_dst_table=False insert_unique_blocks=False -table_a_b -count 5 -table_when_b_even_and_joined -count 10 -0 -0 -table_a_b -count 10 -table_when_b_even_and_joined -count 20 -0 -0 -OK - -Test case 32: insert_method=InsertValues engine=MergeTree use_insert_token=True single_thread=True deduplicate_src_table=True deduplicate_dst_table=True insert_unique_blocks=True -table_a_b -count 5 -table_when_b_even_and_joined -count 9 -0 -0 -table_a_b -count 5 -table_when_b_even_and_joined -count 9 -0 -0 -OK - -Test case 33: insert_method=InsertValues engine=MergeTree use_insert_token=True single_thread=True deduplicate_src_table=True deduplicate_dst_table=True insert_unique_blocks=False -table_a_b -count 5 -table_when_b_even_and_joined -count 10 -0 -0 -table_a_b -count 5 -table_when_b_even_and_joined -count 10 -0 -0 -OK - -Test case 34: insert_method=InsertValues engine=MergeTree use_insert_token=True single_thread=True deduplicate_src_table=True deduplicate_dst_table=False insert_unique_blocks=True -table_a_b -count 5 -table_when_b_even_and_joined -count 9 -0 -0 -table_a_b -count 5 -table_when_b_even_and_joined -count 18 -0 -0 -OK - -Test case 35: insert_method=InsertValues engine=MergeTree use_insert_token=True single_thread=True deduplicate_src_table=True deduplicate_dst_table=False insert_unique_blocks=False -table_a_b -count 5 -table_when_b_even_and_joined -count 10 -0 -0 -table_a_b -count 5 -table_when_b_even_and_joined -count 20 -0 -0 -OK - -Test case 36: insert_method=InsertValues engine=MergeTree use_insert_token=True single_thread=True deduplicate_src_table=False deduplicate_dst_table=True insert_unique_blocks=True -table_a_b -count 5 -table_when_b_even_and_joined -count 9 -0 -0 -table_a_b -count 10 -table_when_b_even_and_joined -count 9 -0 -0 -OK - -Test case 37: insert_method=InsertValues engine=MergeTree use_insert_token=True single_thread=True deduplicate_src_table=False deduplicate_dst_table=True insert_unique_blocks=False -table_a_b -count 5 -table_when_b_even_and_joined -count 10 -0 -0 -table_a_b -count 10 -table_when_b_even_and_joined -count 10 -0 -0 -OK - -Test case 38: insert_method=InsertValues engine=MergeTree use_insert_token=True single_thread=True deduplicate_src_table=False deduplicate_dst_table=False insert_unique_blocks=True -table_a_b -count 5 -table_when_b_even_and_joined -count 9 -0 -0 -table_a_b -count 10 -table_when_b_even_and_joined -count 18 -0 -0 -OK - -Test case 39: insert_method=InsertValues engine=MergeTree use_insert_token=True single_thread=True deduplicate_src_table=False deduplicate_dst_table=False insert_unique_blocks=False -table_a_b -count 5 -table_when_b_even_and_joined -count 10 -0 -0 -table_a_b -count 10 -table_when_b_even_and_joined -count 20 -0 -0 -OK - -Test case 40: insert_method=InsertValues engine=MergeTree use_insert_token=True single_thread=False deduplicate_src_table=True deduplicate_dst_table=True insert_unique_blocks=True -table_a_b -count 5 -table_when_b_even_and_joined -count 9 -0 -0 -table_a_b -count 5 -table_when_b_even_and_joined -count 9 -0 -0 -OK - -Test case 41: insert_method=InsertValues engine=MergeTree use_insert_token=True single_thread=False deduplicate_src_table=True deduplicate_dst_table=True insert_unique_blocks=False -table_a_b -count 5 -table_when_b_even_and_joined -count 10 -0 -0 -table_a_b -count 5 -table_when_b_even_and_joined -count 10 -0 -0 -OK - -Test case 42: insert_method=InsertValues engine=MergeTree use_insert_token=True single_thread=False deduplicate_src_table=True deduplicate_dst_table=False insert_unique_blocks=True -table_a_b -count 5 -table_when_b_even_and_joined -count 9 -0 -0 -table_a_b -count 5 -table_when_b_even_and_joined -count 18 -0 -0 -OK - -Test case 43: insert_method=InsertValues engine=MergeTree use_insert_token=True single_thread=False deduplicate_src_table=True deduplicate_dst_table=False insert_unique_blocks=False -table_a_b -count 5 -table_when_b_even_and_joined -count 10 -0 -0 -table_a_b -count 5 -table_when_b_even_and_joined -count 20 -0 -0 -OK - -Test case 44: insert_method=InsertValues engine=MergeTree use_insert_token=True single_thread=False deduplicate_src_table=False deduplicate_dst_table=True insert_unique_blocks=True -table_a_b -count 5 -table_when_b_even_and_joined -count 9 -0 -0 -table_a_b -count 10 -table_when_b_even_and_joined -count 9 -0 -0 -OK - -Test case 45: insert_method=InsertValues engine=MergeTree use_insert_token=True single_thread=False deduplicate_src_table=False deduplicate_dst_table=True insert_unique_blocks=False -table_a_b -count 5 -table_when_b_even_and_joined -count 10 -0 -0 -table_a_b -count 10 -table_when_b_even_and_joined -count 10 -0 -0 -OK - -Test case 46: insert_method=InsertValues engine=MergeTree use_insert_token=True single_thread=False deduplicate_src_table=False deduplicate_dst_table=False insert_unique_blocks=True -table_a_b -count 5 -table_when_b_even_and_joined -count 9 -0 -0 -table_a_b -count 10 -table_when_b_even_and_joined -count 18 -0 -0 -OK - -Test case 47: insert_method=InsertValues engine=MergeTree use_insert_token=True single_thread=False deduplicate_src_table=False deduplicate_dst_table=False insert_unique_blocks=False -table_a_b -count 5 -table_when_b_even_and_joined -count 10 -0 -0 -table_a_b -count 10 -table_when_b_even_and_joined -count 20 -0 -0 -OK - -Test case 48: insert_method=InsertValues engine=MergeTree use_insert_token=False single_thread=True deduplicate_src_table=True deduplicate_dst_table=True insert_unique_blocks=True -table_a_b -count 5 -table_when_b_even_and_joined -count 9 -0 -0 -table_a_b -count 5 -table_when_b_even_and_joined -count 9 -0 -0 -OK - -Test case 49: insert_method=InsertValues engine=MergeTree use_insert_token=False single_thread=True deduplicate_src_table=True deduplicate_dst_table=True insert_unique_blocks=False -table_a_b -count 1 -table_when_b_even_and_joined -count 2 -0 -0 -table_a_b -count 1 -table_when_b_even_and_joined -count 2 -0 -0 -OK - -Test case 50: insert_method=InsertValues engine=MergeTree use_insert_token=False single_thread=True deduplicate_src_table=True deduplicate_dst_table=False insert_unique_blocks=True -table_a_b -count 5 -table_when_b_even_and_joined -count 9 -0 -0 -table_a_b -count 5 -table_when_b_even_and_joined -count 18 -0 -0 -OK - -Test case 51: insert_method=InsertValues engine=MergeTree use_insert_token=False single_thread=True deduplicate_src_table=True deduplicate_dst_table=False insert_unique_blocks=False -table_a_b -count 1 -table_when_b_even_and_joined -count 10 -0 -0 -table_a_b -count 1 -table_when_b_even_and_joined -count 20 -0 -0 -OK - -Test case 52: insert_method=InsertValues engine=MergeTree use_insert_token=False single_thread=True deduplicate_src_table=False deduplicate_dst_table=True insert_unique_blocks=True -table_a_b -count 5 -table_when_b_even_and_joined -count 9 -0 -0 -table_a_b -count 10 -table_when_b_even_and_joined -count 9 -0 -0 -OK - -Test case 53: insert_method=InsertValues engine=MergeTree use_insert_token=False single_thread=True deduplicate_src_table=False deduplicate_dst_table=True insert_unique_blocks=False -table_a_b -count 5 -table_when_b_even_and_joined -count 2 -0 -0 -table_a_b -count 10 -table_when_b_even_and_joined -count 2 -0 -0 -OK - -Test case 54: insert_method=InsertValues engine=MergeTree use_insert_token=False single_thread=True deduplicate_src_table=False deduplicate_dst_table=False insert_unique_blocks=True -table_a_b -count 5 -table_when_b_even_and_joined -count 9 -0 -0 -table_a_b -count 10 -table_when_b_even_and_joined -count 18 -0 -0 -OK - -Test case 55: insert_method=InsertValues engine=MergeTree use_insert_token=False single_thread=True deduplicate_src_table=False deduplicate_dst_table=False insert_unique_blocks=False -table_a_b -count 5 -table_when_b_even_and_joined -count 10 -0 -0 -table_a_b -count 10 -table_when_b_even_and_joined -count 20 -0 -0 -OK - -Test case 56: insert_method=InsertValues engine=MergeTree use_insert_token=False single_thread=False deduplicate_src_table=True deduplicate_dst_table=True insert_unique_blocks=True -table_a_b -count 5 -table_when_b_even_and_joined -count 9 -0 -0 -table_a_b -count 5 -table_when_b_even_and_joined -count 9 -0 -0 -OK - -Test case 57: insert_method=InsertValues engine=MergeTree use_insert_token=False single_thread=False deduplicate_src_table=True deduplicate_dst_table=True insert_unique_blocks=False -table_a_b -count 1 -table_when_b_even_and_joined -count 2 -0 -0 -table_a_b -count 1 -table_when_b_even_and_joined -count 2 -0 -0 -OK - -Test case 58: insert_method=InsertValues engine=MergeTree use_insert_token=False single_thread=False deduplicate_src_table=True deduplicate_dst_table=False insert_unique_blocks=True -table_a_b -count 5 -table_when_b_even_and_joined -count 9 -0 -0 -table_a_b -count 5 -table_when_b_even_and_joined -count 18 -0 -0 -OK - -Test case 59: insert_method=InsertValues engine=MergeTree use_insert_token=False single_thread=False deduplicate_src_table=True deduplicate_dst_table=False insert_unique_blocks=False -table_a_b -count 1 -table_when_b_even_and_joined -count 10 -0 -0 -table_a_b -count 1 -table_when_b_even_and_joined -count 20 -0 -0 -OK - -Test case 60: insert_method=InsertValues engine=MergeTree use_insert_token=False single_thread=False deduplicate_src_table=False deduplicate_dst_table=True insert_unique_blocks=True -table_a_b -count 5 -table_when_b_even_and_joined -count 9 -0 -0 -table_a_b -count 10 -table_when_b_even_and_joined -count 9 -0 -0 -OK - -Test case 61: insert_method=InsertValues engine=MergeTree use_insert_token=False single_thread=False deduplicate_src_table=False deduplicate_dst_table=True insert_unique_blocks=False -table_a_b -count 5 -table_when_b_even_and_joined -count 2 -0 -0 -table_a_b -count 10 -table_when_b_even_and_joined -count 2 -0 -0 -OK - -Test case 62: insert_method=InsertValues engine=MergeTree use_insert_token=False single_thread=False deduplicate_src_table=False deduplicate_dst_table=False insert_unique_blocks=True -table_a_b -count 5 -table_when_b_even_and_joined -count 9 -0 -0 -table_a_b -count 10 -table_when_b_even_and_joined -count 18 -0 -0 -OK - -Test case 63: insert_method=InsertValues engine=MergeTree use_insert_token=False single_thread=False deduplicate_src_table=False deduplicate_dst_table=False insert_unique_blocks=False -table_a_b -count 5 -table_when_b_even_and_joined -count 10 -0 -0 -table_a_b -count 10 -table_when_b_even_and_joined -count 20 -0 -0 -OK - -All cases executed diff --git a/tests/queries/0_stateless/03008_deduplication_mv_generates_several_blocks_nonreplicated.sh b/tests/queries/0_stateless/03008_deduplication_mv_generates_several_blocks_nonreplicated.sh deleted file mode 100755 index 7d4f5240cd1..00000000000 --- a/tests/queries/0_stateless/03008_deduplication_mv_generates_several_blocks_nonreplicated.sh +++ /dev/null @@ -1,59 +0,0 @@ -#!/usr/bin/env bash -# Tags: long, no-fasttest, no-parallel - -CURDIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd) -# shellcheck source=../shell_config.sh -. "$CURDIR"/../shell_config.sh - -ENGINE="MergeTree" - -RUN_ONLY="" -#RUN_ONLY="Test case 20: engine=MergeTree use_insert_token=False single_thread=True deduplicate_src_table=False deduplicate_dst_table=True insert_unique_blocks=True" - -i=0 -for insert_method in "InsertSelect" "InsertValues"; do - for use_insert_token in "True" "False"; do - for single_thread in "True" "False"; do - for deduplicate_src_table in "True" "False"; do - for deduplicate_dst_table in "True" "False"; do - for insert_unique_blocks in "True" "False"; do - - THIS_RUN="Test case $i:" - THIS_RUN+=" insert_method=$insert_method" - THIS_RUN+=" engine=$ENGINE" - THIS_RUN+=" use_insert_token=$use_insert_token" - THIS_RUN+=" single_thread=$single_thread" - THIS_RUN+=" deduplicate_src_table=$deduplicate_src_table" - THIS_RUN+=" deduplicate_dst_table=$deduplicate_dst_table" - THIS_RUN+=" insert_unique_blocks=$insert_unique_blocks" - - i=$((i+1)) - - echo - if [ -n "$RUN_ONLY" ] && [ "$RUN_ONLY" != "$THIS_RUN" ]; then - echo "skip $THIS_RUN" - continue - fi - echo "$THIS_RUN" - - $CLICKHOUSE_CLIENT --max_insert_block_size 1 -nmq " - $(python3 $CURDIR/03008_deduplication.python mv_generates_several_blocks \ - --insert-method $insert_method \ - --table-engine $ENGINE \ - --use-insert-token $use_insert_token \ - --single-thread $single_thread \ - --deduplicate-src-table $deduplicate_src_table \ - --deduplicate-dst-table $deduplicate_dst_table \ - --insert-unique-blocks $insert_unique_blocks \ - --get-logs false \ - ) - " && echo OK || echo FAIL - done - done - done - done - done -done - -echo -echo "All cases executed" diff --git a/tests/queries/0_stateless/03008_deduplication_mv_generates_several_blocks_replicated.reference b/tests/queries/0_stateless/03008_deduplication_mv_generates_several_blocks_replicated.reference deleted file mode 100644 index a25e8713c61..00000000000 --- a/tests/queries/0_stateless/03008_deduplication_mv_generates_several_blocks_replicated.reference +++ /dev/null @@ -1,962 +0,0 @@ - -Test case 0: insert_method=InsertSelect engine=ReplicatedMergeTree use_insert_token=True single_thread=True deduplicate_src_table=True deduplicate_dst_table=True insert_unique_blocks=True -table_a_b -count 5 -table_when_b_even_and_joined -count 9 -0 -0 -table_a_b -count 5 -table_when_b_even_and_joined -count 9 -0 -0 -OK - -Test case 1: insert_method=InsertSelect engine=ReplicatedMergeTree use_insert_token=True single_thread=True deduplicate_src_table=True deduplicate_dst_table=True insert_unique_blocks=False -table_a_b -count 5 -table_when_b_even_and_joined -count 10 -0 -0 -table_a_b -count 5 -table_when_b_even_and_joined -count 10 -0 -0 -OK - -Test case 2: insert_method=InsertSelect engine=ReplicatedMergeTree use_insert_token=True single_thread=True deduplicate_src_table=True deduplicate_dst_table=False insert_unique_blocks=True -table_a_b -count 5 -table_when_b_even_and_joined -count 9 -0 -0 -table_a_b -count 5 -table_when_b_even_and_joined -count 18 -0 -0 -OK - -Test case 3: insert_method=InsertSelect engine=ReplicatedMergeTree use_insert_token=True single_thread=True deduplicate_src_table=True deduplicate_dst_table=False insert_unique_blocks=False -table_a_b -count 5 -table_when_b_even_and_joined -count 10 -0 -0 -table_a_b -count 5 -table_when_b_even_and_joined -count 20 -0 -0 -OK - -Test case 4: insert_method=InsertSelect engine=ReplicatedMergeTree use_insert_token=True single_thread=True deduplicate_src_table=False deduplicate_dst_table=True insert_unique_blocks=True -table_a_b -count 5 -table_when_b_even_and_joined -count 9 -0 -0 -table_a_b -count 10 -table_when_b_even_and_joined -count 9 -0 -0 -OK - -Test case 5: insert_method=InsertSelect engine=ReplicatedMergeTree use_insert_token=True single_thread=True deduplicate_src_table=False deduplicate_dst_table=True insert_unique_blocks=False -table_a_b -count 5 -table_when_b_even_and_joined -count 10 -0 -0 -table_a_b -count 10 -table_when_b_even_and_joined -count 10 -0 -0 -OK - -Test case 6: insert_method=InsertSelect engine=ReplicatedMergeTree use_insert_token=True single_thread=True deduplicate_src_table=False deduplicate_dst_table=False insert_unique_blocks=True -table_a_b -count 5 -table_when_b_even_and_joined -count 9 -0 -0 -table_a_b -count 10 -table_when_b_even_and_joined -count 18 -0 -0 -OK - -Test case 7: insert_method=InsertSelect engine=ReplicatedMergeTree use_insert_token=True single_thread=True deduplicate_src_table=False deduplicate_dst_table=False insert_unique_blocks=False -table_a_b -count 5 -table_when_b_even_and_joined -count 10 -0 -0 -table_a_b -count 10 -table_when_b_even_and_joined -count 20 -0 -0 -OK - -Test case 8: insert_method=InsertSelect engine=ReplicatedMergeTree use_insert_token=True single_thread=False deduplicate_src_table=True deduplicate_dst_table=True insert_unique_blocks=True -table_a_b -count 5 -table_when_b_even_and_joined -count 9 -0 -0 -table_a_b -count 5 -table_when_b_even_and_joined -count 9 -0 -0 -OK - -Test case 9: insert_method=InsertSelect engine=ReplicatedMergeTree use_insert_token=True single_thread=False deduplicate_src_table=True deduplicate_dst_table=True insert_unique_blocks=False -table_a_b -count 5 -table_when_b_even_and_joined -count 10 -0 -0 -table_a_b -count 5 -table_when_b_even_and_joined -count 10 -0 -0 -OK - -Test case 10: insert_method=InsertSelect engine=ReplicatedMergeTree use_insert_token=True single_thread=False deduplicate_src_table=True deduplicate_dst_table=False insert_unique_blocks=True -table_a_b -count 5 -table_when_b_even_and_joined -count 9 -0 -0 -table_a_b -count 5 -table_when_b_even_and_joined -count 18 -0 -0 -OK - -Test case 11: insert_method=InsertSelect engine=ReplicatedMergeTree use_insert_token=True single_thread=False deduplicate_src_table=True deduplicate_dst_table=False insert_unique_blocks=False -table_a_b -count 5 -table_when_b_even_and_joined -count 10 -0 -0 -table_a_b -count 5 -table_when_b_even_and_joined -count 20 -0 -0 -OK - -Test case 12: insert_method=InsertSelect engine=ReplicatedMergeTree use_insert_token=True single_thread=False deduplicate_src_table=False deduplicate_dst_table=True insert_unique_blocks=True -table_a_b -count 5 -table_when_b_even_and_joined -count 9 -0 -0 -table_a_b -count 10 -table_when_b_even_and_joined -count 9 -0 -0 -OK - -Test case 13: insert_method=InsertSelect engine=ReplicatedMergeTree use_insert_token=True single_thread=False deduplicate_src_table=False deduplicate_dst_table=True insert_unique_blocks=False -table_a_b -count 5 -table_when_b_even_and_joined -count 10 -0 -0 -table_a_b -count 10 -table_when_b_even_and_joined -count 10 -0 -0 -OK - -Test case 14: insert_method=InsertSelect engine=ReplicatedMergeTree use_insert_token=True single_thread=False deduplicate_src_table=False deduplicate_dst_table=False insert_unique_blocks=True -table_a_b -count 5 -table_when_b_even_and_joined -count 9 -0 -0 -table_a_b -count 10 -table_when_b_even_and_joined -count 18 -0 -0 -OK - -Test case 15: insert_method=InsertSelect engine=ReplicatedMergeTree use_insert_token=True single_thread=False deduplicate_src_table=False deduplicate_dst_table=False insert_unique_blocks=False -table_a_b -count 5 -table_when_b_even_and_joined -count 10 -0 -0 -table_a_b -count 10 -table_when_b_even_and_joined -count 20 -0 -0 -OK - -Test case 16: insert_method=InsertSelect engine=ReplicatedMergeTree use_insert_token=False single_thread=True deduplicate_src_table=True deduplicate_dst_table=True insert_unique_blocks=True -table_a_b -count 5 -table_when_b_even_and_joined -count 9 -0 -0 -table_a_b -count 5 -table_when_b_even_and_joined -count 9 -0 -0 -OK - -Test case 17: insert_method=InsertSelect engine=ReplicatedMergeTree use_insert_token=False single_thread=True deduplicate_src_table=True deduplicate_dst_table=True insert_unique_blocks=False -table_a_b -count 1 -table_when_b_even_and_joined -count 2 -0 -0 -table_a_b -count 1 -table_when_b_even_and_joined -count 2 -0 -0 -OK - -Test case 18: insert_method=InsertSelect engine=ReplicatedMergeTree use_insert_token=False single_thread=True deduplicate_src_table=True deduplicate_dst_table=False insert_unique_blocks=True -table_a_b -count 5 -table_when_b_even_and_joined -count 9 -0 -0 -table_a_b -count 5 -table_when_b_even_and_joined -count 18 -0 -0 -OK - -Test case 19: insert_method=InsertSelect engine=ReplicatedMergeTree use_insert_token=False single_thread=True deduplicate_src_table=True deduplicate_dst_table=False insert_unique_blocks=False -table_a_b -count 1 -table_when_b_even_and_joined -count 10 -0 -0 -table_a_b -count 1 -table_when_b_even_and_joined -count 20 -0 -0 -OK - -Test case 20: insert_method=InsertSelect engine=ReplicatedMergeTree use_insert_token=False single_thread=True deduplicate_src_table=False deduplicate_dst_table=True insert_unique_blocks=True -table_a_b -count 5 -table_when_b_even_and_joined -count 9 -0 -0 -table_a_b -count 10 -table_when_b_even_and_joined -count 9 -0 -0 -OK - -Test case 21: insert_method=InsertSelect engine=ReplicatedMergeTree use_insert_token=False single_thread=True deduplicate_src_table=False deduplicate_dst_table=True insert_unique_blocks=False -table_a_b -count 5 -table_when_b_even_and_joined -count 2 -0 -0 -table_a_b -count 10 -table_when_b_even_and_joined -count 2 -0 -0 -OK - -Test case 22: insert_method=InsertSelect engine=ReplicatedMergeTree use_insert_token=False single_thread=True deduplicate_src_table=False deduplicate_dst_table=False insert_unique_blocks=True -table_a_b -count 5 -table_when_b_even_and_joined -count 9 -0 -0 -table_a_b -count 10 -table_when_b_even_and_joined -count 18 -0 -0 -OK - -Test case 23: insert_method=InsertSelect engine=ReplicatedMergeTree use_insert_token=False single_thread=True deduplicate_src_table=False deduplicate_dst_table=False insert_unique_blocks=False -table_a_b -count 5 -table_when_b_even_and_joined -count 10 -0 -0 -table_a_b -count 10 -table_when_b_even_and_joined -count 20 -0 -0 -OK - -Test case 24: insert_method=InsertSelect engine=ReplicatedMergeTree use_insert_token=False single_thread=False deduplicate_src_table=True deduplicate_dst_table=True insert_unique_blocks=True -table_a_b -count 5 -table_when_b_even_and_joined -count 9 -0 -0 -table_a_b -count 5 -table_when_b_even_and_joined -count 9 -0 -0 -OK - -Test case 25: insert_method=InsertSelect engine=ReplicatedMergeTree use_insert_token=False single_thread=False deduplicate_src_table=True deduplicate_dst_table=True insert_unique_blocks=False -table_a_b -count 1 -table_when_b_even_and_joined -count 2 -0 -0 -table_a_b -count 1 -table_when_b_even_and_joined -count 2 -0 -0 -OK - -Test case 26: insert_method=InsertSelect engine=ReplicatedMergeTree use_insert_token=False single_thread=False deduplicate_src_table=True deduplicate_dst_table=False insert_unique_blocks=True -table_a_b -count 5 -table_when_b_even_and_joined -count 9 -0 -0 -table_a_b -count 5 -table_when_b_even_and_joined -count 18 -0 -0 -OK - -Test case 27: insert_method=InsertSelect engine=ReplicatedMergeTree use_insert_token=False single_thread=False deduplicate_src_table=True deduplicate_dst_table=False insert_unique_blocks=False -table_a_b -count 1 -table_when_b_even_and_joined -count 10 -0 -0 -table_a_b -count 1 -table_when_b_even_and_joined -count 20 -0 -0 -OK - -Test case 28: insert_method=InsertSelect engine=ReplicatedMergeTree use_insert_token=False single_thread=False deduplicate_src_table=False deduplicate_dst_table=True insert_unique_blocks=True -table_a_b -count 5 -table_when_b_even_and_joined -count 9 -0 -0 -table_a_b -count 10 -table_when_b_even_and_joined -count 9 -0 -0 -OK - -Test case 29: insert_method=InsertSelect engine=ReplicatedMergeTree use_insert_token=False single_thread=False deduplicate_src_table=False deduplicate_dst_table=True insert_unique_blocks=False -table_a_b -count 5 -table_when_b_even_and_joined -count 2 -0 -0 -table_a_b -count 10 -table_when_b_even_and_joined -count 2 -0 -0 -OK - -Test case 30: insert_method=InsertSelect engine=ReplicatedMergeTree use_insert_token=False single_thread=False deduplicate_src_table=False deduplicate_dst_table=False insert_unique_blocks=True -table_a_b -count 5 -table_when_b_even_and_joined -count 9 -0 -0 -table_a_b -count 10 -table_when_b_even_and_joined -count 18 -0 -0 -OK - -Test case 31: insert_method=InsertSelect engine=ReplicatedMergeTree use_insert_token=False single_thread=False deduplicate_src_table=False deduplicate_dst_table=False insert_unique_blocks=False -table_a_b -count 5 -table_when_b_even_and_joined -count 10 -0 -0 -table_a_b -count 10 -table_when_b_even_and_joined -count 20 -0 -0 -OK - -Test case 32: insert_method=InsertValues engine=ReplicatedMergeTree use_insert_token=True single_thread=True deduplicate_src_table=True deduplicate_dst_table=True insert_unique_blocks=True -table_a_b -count 5 -table_when_b_even_and_joined -count 9 -0 -0 -table_a_b -count 5 -table_when_b_even_and_joined -count 9 -0 -0 -OK - -Test case 33: insert_method=InsertValues engine=ReplicatedMergeTree use_insert_token=True single_thread=True deduplicate_src_table=True deduplicate_dst_table=True insert_unique_blocks=False -table_a_b -count 5 -table_when_b_even_and_joined -count 10 -0 -0 -table_a_b -count 5 -table_when_b_even_and_joined -count 10 -0 -0 -OK - -Test case 34: insert_method=InsertValues engine=ReplicatedMergeTree use_insert_token=True single_thread=True deduplicate_src_table=True deduplicate_dst_table=False insert_unique_blocks=True -table_a_b -count 5 -table_when_b_even_and_joined -count 9 -0 -0 -table_a_b -count 5 -table_when_b_even_and_joined -count 18 -0 -0 -OK - -Test case 35: insert_method=InsertValues engine=ReplicatedMergeTree use_insert_token=True single_thread=True deduplicate_src_table=True deduplicate_dst_table=False insert_unique_blocks=False -table_a_b -count 5 -table_when_b_even_and_joined -count 10 -0 -0 -table_a_b -count 5 -table_when_b_even_and_joined -count 20 -0 -0 -OK - -Test case 36: insert_method=InsertValues engine=ReplicatedMergeTree use_insert_token=True single_thread=True deduplicate_src_table=False deduplicate_dst_table=True insert_unique_blocks=True -table_a_b -count 5 -table_when_b_even_and_joined -count 9 -0 -0 -table_a_b -count 10 -table_when_b_even_and_joined -count 9 -0 -0 -OK - -Test case 37: insert_method=InsertValues engine=ReplicatedMergeTree use_insert_token=True single_thread=True deduplicate_src_table=False deduplicate_dst_table=True insert_unique_blocks=False -table_a_b -count 5 -table_when_b_even_and_joined -count 10 -0 -0 -table_a_b -count 10 -table_when_b_even_and_joined -count 10 -0 -0 -OK - -Test case 38: insert_method=InsertValues engine=ReplicatedMergeTree use_insert_token=True single_thread=True deduplicate_src_table=False deduplicate_dst_table=False insert_unique_blocks=True -table_a_b -count 5 -table_when_b_even_and_joined -count 9 -0 -0 -table_a_b -count 10 -table_when_b_even_and_joined -count 18 -0 -0 -OK - -Test case 39: insert_method=InsertValues engine=ReplicatedMergeTree use_insert_token=True single_thread=True deduplicate_src_table=False deduplicate_dst_table=False insert_unique_blocks=False -table_a_b -count 5 -table_when_b_even_and_joined -count 10 -0 -0 -table_a_b -count 10 -table_when_b_even_and_joined -count 20 -0 -0 -OK - -Test case 40: insert_method=InsertValues engine=ReplicatedMergeTree use_insert_token=True single_thread=False deduplicate_src_table=True deduplicate_dst_table=True insert_unique_blocks=True -table_a_b -count 5 -table_when_b_even_and_joined -count 9 -0 -0 -table_a_b -count 5 -table_when_b_even_and_joined -count 9 -0 -0 -OK - -Test case 41: insert_method=InsertValues engine=ReplicatedMergeTree use_insert_token=True single_thread=False deduplicate_src_table=True deduplicate_dst_table=True insert_unique_blocks=False -table_a_b -count 5 -table_when_b_even_and_joined -count 10 -0 -0 -table_a_b -count 5 -table_when_b_even_and_joined -count 10 -0 -0 -OK - -Test case 42: insert_method=InsertValues engine=ReplicatedMergeTree use_insert_token=True single_thread=False deduplicate_src_table=True deduplicate_dst_table=False insert_unique_blocks=True -table_a_b -count 5 -table_when_b_even_and_joined -count 9 -0 -0 -table_a_b -count 5 -table_when_b_even_and_joined -count 18 -0 -0 -OK - -Test case 43: insert_method=InsertValues engine=ReplicatedMergeTree use_insert_token=True single_thread=False deduplicate_src_table=True deduplicate_dst_table=False insert_unique_blocks=False -table_a_b -count 5 -table_when_b_even_and_joined -count 10 -0 -0 -table_a_b -count 5 -table_when_b_even_and_joined -count 20 -0 -0 -OK - -Test case 44: insert_method=InsertValues engine=ReplicatedMergeTree use_insert_token=True single_thread=False deduplicate_src_table=False deduplicate_dst_table=True insert_unique_blocks=True -table_a_b -count 5 -table_when_b_even_and_joined -count 9 -0 -0 -table_a_b -count 10 -table_when_b_even_and_joined -count 9 -0 -0 -OK - -Test case 45: insert_method=InsertValues engine=ReplicatedMergeTree use_insert_token=True single_thread=False deduplicate_src_table=False deduplicate_dst_table=True insert_unique_blocks=False -table_a_b -count 5 -table_when_b_even_and_joined -count 10 -0 -0 -table_a_b -count 10 -table_when_b_even_and_joined -count 10 -0 -0 -OK - -Test case 46: insert_method=InsertValues engine=ReplicatedMergeTree use_insert_token=True single_thread=False deduplicate_src_table=False deduplicate_dst_table=False insert_unique_blocks=True -table_a_b -count 5 -table_when_b_even_and_joined -count 9 -0 -0 -table_a_b -count 10 -table_when_b_even_and_joined -count 18 -0 -0 -OK - -Test case 47: insert_method=InsertValues engine=ReplicatedMergeTree use_insert_token=True single_thread=False deduplicate_src_table=False deduplicate_dst_table=False insert_unique_blocks=False -table_a_b -count 5 -table_when_b_even_and_joined -count 10 -0 -0 -table_a_b -count 10 -table_when_b_even_and_joined -count 20 -0 -0 -OK - -Test case 48: insert_method=InsertValues engine=ReplicatedMergeTree use_insert_token=False single_thread=True deduplicate_src_table=True deduplicate_dst_table=True insert_unique_blocks=True -table_a_b -count 5 -table_when_b_even_and_joined -count 9 -0 -0 -table_a_b -count 5 -table_when_b_even_and_joined -count 9 -0 -0 -OK - -Test case 49: insert_method=InsertValues engine=ReplicatedMergeTree use_insert_token=False single_thread=True deduplicate_src_table=True deduplicate_dst_table=True insert_unique_blocks=False -table_a_b -count 1 -table_when_b_even_and_joined -count 2 -0 -0 -table_a_b -count 1 -table_when_b_even_and_joined -count 2 -0 -0 -OK - -Test case 50: insert_method=InsertValues engine=ReplicatedMergeTree use_insert_token=False single_thread=True deduplicate_src_table=True deduplicate_dst_table=False insert_unique_blocks=True -table_a_b -count 5 -table_when_b_even_and_joined -count 9 -0 -0 -table_a_b -count 5 -table_when_b_even_and_joined -count 18 -0 -0 -OK - -Test case 51: insert_method=InsertValues engine=ReplicatedMergeTree use_insert_token=False single_thread=True deduplicate_src_table=True deduplicate_dst_table=False insert_unique_blocks=False -table_a_b -count 1 -table_when_b_even_and_joined -count 10 -0 -0 -table_a_b -count 1 -table_when_b_even_and_joined -count 20 -0 -0 -OK - -Test case 52: insert_method=InsertValues engine=ReplicatedMergeTree use_insert_token=False single_thread=True deduplicate_src_table=False deduplicate_dst_table=True insert_unique_blocks=True -table_a_b -count 5 -table_when_b_even_and_joined -count 9 -0 -0 -table_a_b -count 10 -table_when_b_even_and_joined -count 9 -0 -0 -OK - -Test case 53: insert_method=InsertValues engine=ReplicatedMergeTree use_insert_token=False single_thread=True deduplicate_src_table=False deduplicate_dst_table=True insert_unique_blocks=False -table_a_b -count 5 -table_when_b_even_and_joined -count 2 -0 -0 -table_a_b -count 10 -table_when_b_even_and_joined -count 2 -0 -0 -OK - -Test case 54: insert_method=InsertValues engine=ReplicatedMergeTree use_insert_token=False single_thread=True deduplicate_src_table=False deduplicate_dst_table=False insert_unique_blocks=True -table_a_b -count 5 -table_when_b_even_and_joined -count 9 -0 -0 -table_a_b -count 10 -table_when_b_even_and_joined -count 18 -0 -0 -OK - -Test case 55: insert_method=InsertValues engine=ReplicatedMergeTree use_insert_token=False single_thread=True deduplicate_src_table=False deduplicate_dst_table=False insert_unique_blocks=False -table_a_b -count 5 -table_when_b_even_and_joined -count 10 -0 -0 -table_a_b -count 10 -table_when_b_even_and_joined -count 20 -0 -0 -OK - -Test case 56: insert_method=InsertValues engine=ReplicatedMergeTree use_insert_token=False single_thread=False deduplicate_src_table=True deduplicate_dst_table=True insert_unique_blocks=True -table_a_b -count 5 -table_when_b_even_and_joined -count 9 -0 -0 -table_a_b -count 5 -table_when_b_even_and_joined -count 9 -0 -0 -OK - -Test case 57: insert_method=InsertValues engine=ReplicatedMergeTree use_insert_token=False single_thread=False deduplicate_src_table=True deduplicate_dst_table=True insert_unique_blocks=False -table_a_b -count 1 -table_when_b_even_and_joined -count 2 -0 -0 -table_a_b -count 1 -table_when_b_even_and_joined -count 2 -0 -0 -OK - -Test case 58: insert_method=InsertValues engine=ReplicatedMergeTree use_insert_token=False single_thread=False deduplicate_src_table=True deduplicate_dst_table=False insert_unique_blocks=True -table_a_b -count 5 -table_when_b_even_and_joined -count 9 -0 -0 -table_a_b -count 5 -table_when_b_even_and_joined -count 18 -0 -0 -OK - -Test case 59: insert_method=InsertValues engine=ReplicatedMergeTree use_insert_token=False single_thread=False deduplicate_src_table=True deduplicate_dst_table=False insert_unique_blocks=False -table_a_b -count 1 -table_when_b_even_and_joined -count 10 -0 -0 -table_a_b -count 1 -table_when_b_even_and_joined -count 20 -0 -0 -OK - -Test case 60: insert_method=InsertValues engine=ReplicatedMergeTree use_insert_token=False single_thread=False deduplicate_src_table=False deduplicate_dst_table=True insert_unique_blocks=True -table_a_b -count 5 -table_when_b_even_and_joined -count 9 -0 -0 -table_a_b -count 10 -table_when_b_even_and_joined -count 9 -0 -0 -OK - -Test case 61: insert_method=InsertValues engine=ReplicatedMergeTree use_insert_token=False single_thread=False deduplicate_src_table=False deduplicate_dst_table=True insert_unique_blocks=False -table_a_b -count 5 -table_when_b_even_and_joined -count 2 -0 -0 -table_a_b -count 10 -table_when_b_even_and_joined -count 2 -0 -0 -OK - -Test case 62: insert_method=InsertValues engine=ReplicatedMergeTree use_insert_token=False single_thread=False deduplicate_src_table=False deduplicate_dst_table=False insert_unique_blocks=True -table_a_b -count 5 -table_when_b_even_and_joined -count 9 -0 -0 -table_a_b -count 10 -table_when_b_even_and_joined -count 18 -0 -0 -OK - -Test case 63: insert_method=InsertValues engine=ReplicatedMergeTree use_insert_token=False single_thread=False deduplicate_src_table=False deduplicate_dst_table=False insert_unique_blocks=False -table_a_b -count 5 -table_when_b_even_and_joined -count 10 -0 -0 -table_a_b -count 10 -table_when_b_even_and_joined -count 20 -0 -0 -OK - -All cases executed diff --git a/tests/queries/0_stateless/03008_deduplication_mv_generates_several_blocks_replicated.sh b/tests/queries/0_stateless/03008_deduplication_mv_generates_several_blocks_replicated.sh deleted file mode 100755 index 109d1674f3a..00000000000 --- a/tests/queries/0_stateless/03008_deduplication_mv_generates_several_blocks_replicated.sh +++ /dev/null @@ -1,59 +0,0 @@ -#!/usr/bin/env bash -# Tags: long, no-fasttest, no-parallel - -CURDIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd) -# shellcheck source=../shell_config.sh -. "$CURDIR"/../shell_config.sh - -ENGINE="ReplicatedMergeTree" - -RUN_ONLY="" -#RUN_ONLY="Test case 20: engine=ReplicatedMergeTree use_insert_token=False single_thread=True deduplicate_src_table=False deduplicate_dst_table=True insert_unique_blocks=True" - -i=0 -for insert_method in "InsertSelect" "InsertValues"; do - for use_insert_token in "True" "False"; do - for single_thread in "True" "False"; do - for deduplicate_src_table in "True" "False"; do - for deduplicate_dst_table in "True" "False"; do - for insert_unique_blocks in "True" "False"; do - - THIS_RUN="Test case $i:" - THIS_RUN+=" insert_method=$insert_method" - THIS_RUN+=" engine=$ENGINE" - THIS_RUN+=" use_insert_token=$use_insert_token" - THIS_RUN+=" single_thread=$single_thread" - THIS_RUN+=" deduplicate_src_table=$deduplicate_src_table" - THIS_RUN+=" deduplicate_dst_table=$deduplicate_dst_table" - THIS_RUN+=" insert_unique_blocks=$insert_unique_blocks" - - i=$((i+1)) - - echo - if [ -n "$RUN_ONLY" ] && [ "$RUN_ONLY" != "$THIS_RUN" ]; then - echo "skip $THIS_RUN" - continue - fi - echo "$THIS_RUN" - - $CLICKHOUSE_CLIENT --max_insert_block_size 1 -nmq " - $(python3 $CURDIR/03008_deduplication.python mv_generates_several_blocks \ - --insert-method $insert_method \ - --table-engine $ENGINE \ - --use-insert-token $use_insert_token \ - --single-thread $single_thread \ - --deduplicate-src-table $deduplicate_src_table \ - --deduplicate-dst-table $deduplicate_dst_table \ - --insert-unique-blocks $insert_unique_blocks \ - --get-logs false \ - ) - " && echo OK || echo FAIL - done - done - done - done - done -done - -echo -echo "All cases executed" diff --git a/tests/queries/0_stateless/03008_deduplication_several_mv_into_one_table_nonreplicated.reference b/tests/queries/0_stateless/03008_deduplication_several_mv_into_one_table_nonreplicated.reference deleted file mode 100644 index b6a3e0175a7..00000000000 --- a/tests/queries/0_stateless/03008_deduplication_several_mv_into_one_table_nonreplicated.reference +++ /dev/null @@ -1,706 +0,0 @@ - -Test case 0: insert_method=InsertSelect engine=MergeTree use_insert_token=True single_thread=True deduplicate_src_table=True deduplicate_dst_table=True insert_unique_blocks=True -table_src count 8 -table_dst count 6 -0 -0 -table_src count 8 -table_dst count 6 -0 -0 -OK - -Test case 1: insert_method=InsertSelect engine=MergeTree use_insert_token=True single_thread=True deduplicate_src_table=True deduplicate_dst_table=True insert_unique_blocks=False -table_src count 8 -table_dst count 16 -0 -0 -table_src count 8 -table_dst count 16 -0 -0 -OK - -Test case 2: insert_method=InsertSelect engine=MergeTree use_insert_token=True single_thread=True deduplicate_src_table=True deduplicate_dst_table=False insert_unique_blocks=True -table_src count 8 -table_dst count 6 -0 -0 -table_src count 8 -table_dst count 12 -0 -0 -OK - -Test case 3: insert_method=InsertSelect engine=MergeTree use_insert_token=True single_thread=True deduplicate_src_table=True deduplicate_dst_table=False insert_unique_blocks=False -table_src count 8 -table_dst count 16 -0 -0 -table_src count 8 -table_dst count 32 -0 -0 -OK - -Test case 4: insert_method=InsertSelect engine=MergeTree use_insert_token=True single_thread=True deduplicate_src_table=False deduplicate_dst_table=True insert_unique_blocks=True -table_src count 8 -table_dst count 6 -0 -0 -table_src count 16 -table_dst count 6 -0 -0 -OK - -Test case 5: insert_method=InsertSelect engine=MergeTree use_insert_token=True single_thread=True deduplicate_src_table=False deduplicate_dst_table=True insert_unique_blocks=False -table_src count 8 -table_dst count 16 -0 -0 -table_src count 16 -table_dst count 16 -0 -0 -OK - -Test case 6: insert_method=InsertSelect engine=MergeTree use_insert_token=True single_thread=True deduplicate_src_table=False deduplicate_dst_table=False insert_unique_blocks=True -table_src count 8 -table_dst count 6 -0 -0 -table_src count 16 -table_dst count 12 -0 -0 -OK - -Test case 7: insert_method=InsertSelect engine=MergeTree use_insert_token=True single_thread=True deduplicate_src_table=False deduplicate_dst_table=False insert_unique_blocks=False -table_src count 8 -table_dst count 16 -0 -0 -table_src count 16 -table_dst count 32 -0 -0 -OK - -Test case 8: insert_method=InsertSelect engine=MergeTree use_insert_token=True single_thread=False deduplicate_src_table=True deduplicate_dst_table=True insert_unique_blocks=True -table_src count 8 -table_dst count 6 -0 -0 -table_src count 8 -table_dst count 6 -0 -0 -OK - -Test case 9: insert_method=InsertSelect engine=MergeTree use_insert_token=True single_thread=False deduplicate_src_table=True deduplicate_dst_table=True insert_unique_blocks=False -table_src count 8 -table_dst count 16 -0 -0 -table_src count 8 -table_dst count 16 -0 -0 -OK - -Test case 10: insert_method=InsertSelect engine=MergeTree use_insert_token=True single_thread=False deduplicate_src_table=True deduplicate_dst_table=False insert_unique_blocks=True -table_src count 8 -table_dst count 6 -0 -0 -table_src count 8 -table_dst count 12 -0 -0 -OK - -Test case 11: insert_method=InsertSelect engine=MergeTree use_insert_token=True single_thread=False deduplicate_src_table=True deduplicate_dst_table=False insert_unique_blocks=False -table_src count 8 -table_dst count 16 -0 -0 -table_src count 8 -table_dst count 32 -0 -0 -OK - -Test case 12: insert_method=InsertSelect engine=MergeTree use_insert_token=True single_thread=False deduplicate_src_table=False deduplicate_dst_table=True insert_unique_blocks=True -table_src count 8 -table_dst count 6 -0 -0 -table_src count 16 -table_dst count 6 -0 -0 -OK - -Test case 13: insert_method=InsertSelect engine=MergeTree use_insert_token=True single_thread=False deduplicate_src_table=False deduplicate_dst_table=True insert_unique_blocks=False -table_src count 8 -table_dst count 16 -0 -0 -table_src count 16 -table_dst count 16 -0 -0 -OK - -Test case 14: insert_method=InsertSelect engine=MergeTree use_insert_token=True single_thread=False deduplicate_src_table=False deduplicate_dst_table=False insert_unique_blocks=True -table_src count 8 -table_dst count 6 -0 -0 -table_src count 16 -table_dst count 12 -0 -0 -OK - -Test case 15: insert_method=InsertSelect engine=MergeTree use_insert_token=True single_thread=False deduplicate_src_table=False deduplicate_dst_table=False insert_unique_blocks=False -table_src count 8 -table_dst count 16 -0 -0 -table_src count 16 -table_dst count 32 -0 -0 -OK - -Test case 16: insert_method=InsertSelect engine=MergeTree use_insert_token=False single_thread=True deduplicate_src_table=True deduplicate_dst_table=True insert_unique_blocks=True -table_src count 8 -table_dst count 6 -0 -0 -table_src count 8 -table_dst count 6 -0 -0 -OK - -Test case 17: insert_method=InsertSelect engine=MergeTree use_insert_token=False single_thread=True deduplicate_src_table=True deduplicate_dst_table=True insert_unique_blocks=False -table_src count 1 -table_dst count 2 -0 -0 -table_src count 1 -table_dst count 2 -0 -0 -OK - -Test case 18: insert_method=InsertSelect engine=MergeTree use_insert_token=False single_thread=True deduplicate_src_table=True deduplicate_dst_table=False insert_unique_blocks=True -table_src count 8 -table_dst count 6 -0 -0 -table_src count 8 -table_dst count 12 -0 -0 -OK - -Test case 19: insert_method=InsertSelect engine=MergeTree use_insert_token=False single_thread=True deduplicate_src_table=True deduplicate_dst_table=False insert_unique_blocks=False -table_src count 1 -table_dst count 16 -0 -0 -table_src count 1 -table_dst count 32 -0 -0 -OK - -Test case 20: insert_method=InsertSelect engine=MergeTree use_insert_token=False single_thread=True deduplicate_src_table=False deduplicate_dst_table=True insert_unique_blocks=True -table_src count 8 -table_dst count 6 -0 -0 -table_src count 16 -table_dst count 6 -0 -0 -OK - -Test case 21: insert_method=InsertSelect engine=MergeTree use_insert_token=False single_thread=True deduplicate_src_table=False deduplicate_dst_table=True insert_unique_blocks=False -table_src count 8 -table_dst count 2 -0 -0 -table_src count 16 -table_dst count 2 -0 -0 -OK - -Test case 22: insert_method=InsertSelect engine=MergeTree use_insert_token=False single_thread=True deduplicate_src_table=False deduplicate_dst_table=False insert_unique_blocks=True -table_src count 8 -table_dst count 6 -0 -0 -table_src count 16 -table_dst count 12 -0 -0 -OK - -Test case 23: insert_method=InsertSelect engine=MergeTree use_insert_token=False single_thread=True deduplicate_src_table=False deduplicate_dst_table=False insert_unique_blocks=False -table_src count 8 -table_dst count 16 -0 -0 -table_src count 16 -table_dst count 32 -0 -0 -OK - -Test case 24: insert_method=InsertSelect engine=MergeTree use_insert_token=False single_thread=False deduplicate_src_table=True deduplicate_dst_table=True insert_unique_blocks=True -table_src count 8 -table_dst count 6 -0 -0 -table_src count 8 -table_dst count 6 -0 -0 -OK - -Test case 25: insert_method=InsertSelect engine=MergeTree use_insert_token=False single_thread=False deduplicate_src_table=True deduplicate_dst_table=True insert_unique_blocks=False -table_src count 1 -table_dst count 2 -0 -0 -table_src count 1 -table_dst count 2 -0 -0 -OK - -Test case 26: insert_method=InsertSelect engine=MergeTree use_insert_token=False single_thread=False deduplicate_src_table=True deduplicate_dst_table=False insert_unique_blocks=True -table_src count 8 -table_dst count 6 -0 -0 -table_src count 8 -table_dst count 12 -0 -0 -OK - -Test case 27: insert_method=InsertSelect engine=MergeTree use_insert_token=False single_thread=False deduplicate_src_table=True deduplicate_dst_table=False insert_unique_blocks=False -table_src count 1 -table_dst count 16 -0 -0 -table_src count 1 -table_dst count 32 -0 -0 -OK - -Test case 28: insert_method=InsertSelect engine=MergeTree use_insert_token=False single_thread=False deduplicate_src_table=False deduplicate_dst_table=True insert_unique_blocks=True -table_src count 8 -table_dst count 6 -0 -0 -table_src count 16 -table_dst count 6 -0 -0 -OK - -Test case 29: insert_method=InsertSelect engine=MergeTree use_insert_token=False single_thread=False deduplicate_src_table=False deduplicate_dst_table=True insert_unique_blocks=False -table_src count 8 -table_dst count 2 -0 -0 -table_src count 16 -table_dst count 2 -0 -0 -OK - -Test case 30: insert_method=InsertSelect engine=MergeTree use_insert_token=False single_thread=False deduplicate_src_table=False deduplicate_dst_table=False insert_unique_blocks=True -table_src count 8 -table_dst count 6 -0 -0 -table_src count 16 -table_dst count 12 -0 -0 -OK - -Test case 31: insert_method=InsertSelect engine=MergeTree use_insert_token=False single_thread=False deduplicate_src_table=False deduplicate_dst_table=False insert_unique_blocks=False -table_src count 8 -table_dst count 16 -0 -0 -table_src count 16 -table_dst count 32 -0 -0 -OK - -Test case 32: insert_method=InsertValues engine=MergeTree use_insert_token=True single_thread=True deduplicate_src_table=True deduplicate_dst_table=True insert_unique_blocks=True -table_src count 8 -table_dst count 6 -0 -0 -table_src count 8 -table_dst count 6 -0 -0 -OK - -Test case 33: insert_method=InsertValues engine=MergeTree use_insert_token=True single_thread=True deduplicate_src_table=True deduplicate_dst_table=True insert_unique_blocks=False -table_src count 8 -table_dst count 16 -0 -0 -table_src count 8 -table_dst count 16 -0 -0 -OK - -Test case 34: insert_method=InsertValues engine=MergeTree use_insert_token=True single_thread=True deduplicate_src_table=True deduplicate_dst_table=False insert_unique_blocks=True -table_src count 8 -table_dst count 6 -0 -0 -table_src count 8 -table_dst count 12 -0 -0 -OK - -Test case 35: insert_method=InsertValues engine=MergeTree use_insert_token=True single_thread=True deduplicate_src_table=True deduplicate_dst_table=False insert_unique_blocks=False -table_src count 8 -table_dst count 16 -0 -0 -table_src count 8 -table_dst count 32 -0 -0 -OK - -Test case 36: insert_method=InsertValues engine=MergeTree use_insert_token=True single_thread=True deduplicate_src_table=False deduplicate_dst_table=True insert_unique_blocks=True -table_src count 8 -table_dst count 6 -0 -0 -table_src count 16 -table_dst count 6 -0 -0 -OK - -Test case 37: insert_method=InsertValues engine=MergeTree use_insert_token=True single_thread=True deduplicate_src_table=False deduplicate_dst_table=True insert_unique_blocks=False -table_src count 8 -table_dst count 16 -0 -0 -table_src count 16 -table_dst count 16 -0 -0 -OK - -Test case 38: insert_method=InsertValues engine=MergeTree use_insert_token=True single_thread=True deduplicate_src_table=False deduplicate_dst_table=False insert_unique_blocks=True -table_src count 8 -table_dst count 6 -0 -0 -table_src count 16 -table_dst count 12 -0 -0 -OK - -Test case 39: insert_method=InsertValues engine=MergeTree use_insert_token=True single_thread=True deduplicate_src_table=False deduplicate_dst_table=False insert_unique_blocks=False -table_src count 8 -table_dst count 16 -0 -0 -table_src count 16 -table_dst count 32 -0 -0 -OK - -Test case 40: insert_method=InsertValues engine=MergeTree use_insert_token=True single_thread=False deduplicate_src_table=True deduplicate_dst_table=True insert_unique_blocks=True -table_src count 8 -table_dst count 6 -0 -0 -table_src count 8 -table_dst count 6 -0 -0 -OK - -Test case 41: insert_method=InsertValues engine=MergeTree use_insert_token=True single_thread=False deduplicate_src_table=True deduplicate_dst_table=True insert_unique_blocks=False -table_src count 8 -table_dst count 16 -0 -0 -table_src count 8 -table_dst count 16 -0 -0 -OK - -Test case 42: insert_method=InsertValues engine=MergeTree use_insert_token=True single_thread=False deduplicate_src_table=True deduplicate_dst_table=False insert_unique_blocks=True -table_src count 8 -table_dst count 6 -0 -0 -table_src count 8 -table_dst count 12 -0 -0 -OK - -Test case 43: insert_method=InsertValues engine=MergeTree use_insert_token=True single_thread=False deduplicate_src_table=True deduplicate_dst_table=False insert_unique_blocks=False -table_src count 8 -table_dst count 16 -0 -0 -table_src count 8 -table_dst count 32 -0 -0 -OK - -Test case 44: insert_method=InsertValues engine=MergeTree use_insert_token=True single_thread=False deduplicate_src_table=False deduplicate_dst_table=True insert_unique_blocks=True -table_src count 8 -table_dst count 6 -0 -0 -table_src count 16 -table_dst count 6 -0 -0 -OK - -Test case 45: insert_method=InsertValues engine=MergeTree use_insert_token=True single_thread=False deduplicate_src_table=False deduplicate_dst_table=True insert_unique_blocks=False -table_src count 8 -table_dst count 16 -0 -0 -table_src count 16 -table_dst count 16 -0 -0 -OK - -Test case 46: insert_method=InsertValues engine=MergeTree use_insert_token=True single_thread=False deduplicate_src_table=False deduplicate_dst_table=False insert_unique_blocks=True -table_src count 8 -table_dst count 6 -0 -0 -table_src count 16 -table_dst count 12 -0 -0 -OK - -Test case 47: insert_method=InsertValues engine=MergeTree use_insert_token=True single_thread=False deduplicate_src_table=False deduplicate_dst_table=False insert_unique_blocks=False -table_src count 8 -table_dst count 16 -0 -0 -table_src count 16 -table_dst count 32 -0 -0 -OK - -Test case 48: insert_method=InsertValues engine=MergeTree use_insert_token=False single_thread=True deduplicate_src_table=True deduplicate_dst_table=True insert_unique_blocks=True -table_src count 8 -table_dst count 6 -0 -0 -table_src count 8 -table_dst count 6 -0 -0 -OK - -Test case 49: insert_method=InsertValues engine=MergeTree use_insert_token=False single_thread=True deduplicate_src_table=True deduplicate_dst_table=True insert_unique_blocks=False -table_src count 1 -table_dst count 2 -0 -0 -table_src count 1 -table_dst count 2 -0 -0 -OK - -Test case 50: insert_method=InsertValues engine=MergeTree use_insert_token=False single_thread=True deduplicate_src_table=True deduplicate_dst_table=False insert_unique_blocks=True -table_src count 8 -table_dst count 6 -0 -0 -table_src count 8 -table_dst count 12 -0 -0 -OK - -Test case 51: insert_method=InsertValues engine=MergeTree use_insert_token=False single_thread=True deduplicate_src_table=True deduplicate_dst_table=False insert_unique_blocks=False -table_src count 1 -table_dst count 16 -0 -0 -table_src count 1 -table_dst count 32 -0 -0 -OK - -Test case 52: insert_method=InsertValues engine=MergeTree use_insert_token=False single_thread=True deduplicate_src_table=False deduplicate_dst_table=True insert_unique_blocks=True -table_src count 8 -table_dst count 6 -0 -0 -table_src count 16 -table_dst count 6 -0 -0 -OK - -Test case 53: insert_method=InsertValues engine=MergeTree use_insert_token=False single_thread=True deduplicate_src_table=False deduplicate_dst_table=True insert_unique_blocks=False -table_src count 8 -table_dst count 2 -0 -0 -table_src count 16 -table_dst count 2 -0 -0 -OK - -Test case 54: insert_method=InsertValues engine=MergeTree use_insert_token=False single_thread=True deduplicate_src_table=False deduplicate_dst_table=False insert_unique_blocks=True -table_src count 8 -table_dst count 6 -0 -0 -table_src count 16 -table_dst count 12 -0 -0 -OK - -Test case 55: insert_method=InsertValues engine=MergeTree use_insert_token=False single_thread=True deduplicate_src_table=False deduplicate_dst_table=False insert_unique_blocks=False -table_src count 8 -table_dst count 16 -0 -0 -table_src count 16 -table_dst count 32 -0 -0 -OK - -Test case 56: insert_method=InsertValues engine=MergeTree use_insert_token=False single_thread=False deduplicate_src_table=True deduplicate_dst_table=True insert_unique_blocks=True -table_src count 8 -table_dst count 6 -0 -0 -table_src count 8 -table_dst count 6 -0 -0 -OK - -Test case 57: insert_method=InsertValues engine=MergeTree use_insert_token=False single_thread=False deduplicate_src_table=True deduplicate_dst_table=True insert_unique_blocks=False -table_src count 1 -table_dst count 2 -0 -0 -table_src count 1 -table_dst count 2 -0 -0 -OK - -Test case 58: insert_method=InsertValues engine=MergeTree use_insert_token=False single_thread=False deduplicate_src_table=True deduplicate_dst_table=False insert_unique_blocks=True -table_src count 8 -table_dst count 6 -0 -0 -table_src count 8 -table_dst count 12 -0 -0 -OK - -Test case 59: insert_method=InsertValues engine=MergeTree use_insert_token=False single_thread=False deduplicate_src_table=True deduplicate_dst_table=False insert_unique_blocks=False -table_src count 1 -table_dst count 16 -0 -0 -table_src count 1 -table_dst count 32 -0 -0 -OK - -Test case 60: insert_method=InsertValues engine=MergeTree use_insert_token=False single_thread=False deduplicate_src_table=False deduplicate_dst_table=True insert_unique_blocks=True -table_src count 8 -table_dst count 6 -0 -0 -table_src count 16 -table_dst count 6 -0 -0 -OK - -Test case 61: insert_method=InsertValues engine=MergeTree use_insert_token=False single_thread=False deduplicate_src_table=False deduplicate_dst_table=True insert_unique_blocks=False -table_src count 8 -table_dst count 2 -0 -0 -table_src count 16 -table_dst count 2 -0 -0 -OK - -Test case 62: insert_method=InsertValues engine=MergeTree use_insert_token=False single_thread=False deduplicate_src_table=False deduplicate_dst_table=False insert_unique_blocks=True -table_src count 8 -table_dst count 6 -0 -0 -table_src count 16 -table_dst count 12 -0 -0 -OK - -Test case 63: insert_method=InsertValues engine=MergeTree use_insert_token=False single_thread=False deduplicate_src_table=False deduplicate_dst_table=False insert_unique_blocks=False -table_src count 8 -table_dst count 16 -0 -0 -table_src count 16 -table_dst count 32 -0 -0 -OK - -All cases executed diff --git a/tests/queries/0_stateless/03008_deduplication_several_mv_into_one_table_nonreplicated.sh b/tests/queries/0_stateless/03008_deduplication_several_mv_into_one_table_nonreplicated.sh deleted file mode 100755 index fe3d610a758..00000000000 --- a/tests/queries/0_stateless/03008_deduplication_several_mv_into_one_table_nonreplicated.sh +++ /dev/null @@ -1,59 +0,0 @@ -#!/usr/bin/env bash -# Tags: long, no-fasttest, no-parallel - -CURDIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd) -# shellcheck source=../shell_config.sh -. "$CURDIR"/../shell_config.sh - -ENGINE="MergeTree" - -RUN_ONLY="" -#RUN_ONLY="Test case 17: insert_method=InsertSelect engine=MergeTree use_insert_token=False single_thread=True deduplicate_src_table=True deduplicate_dst_table=True insert_unique_blocks=False" - -i=0 -for insert_method in "InsertSelect" "InsertValues"; do - for use_insert_token in "True" "False"; do - for single_thread in "True" "False"; do - for deduplicate_src_table in "True" "False"; do - for deduplicate_dst_table in "True" "False"; do - for insert_unique_blocks in "True" "False"; do - - THIS_RUN="Test case $i:" - THIS_RUN+=" insert_method=$insert_method" - THIS_RUN+=" engine=$ENGINE" - THIS_RUN+=" use_insert_token=$use_insert_token" - THIS_RUN+=" single_thread=$single_thread" - THIS_RUN+=" deduplicate_src_table=$deduplicate_src_table" - THIS_RUN+=" deduplicate_dst_table=$deduplicate_dst_table" - THIS_RUN+=" insert_unique_blocks=$insert_unique_blocks" - - i=$((i+1)) - - echo - if [ -n "$RUN_ONLY" ] && [ "$RUN_ONLY" != "$THIS_RUN" ]; then - echo "skip $THIS_RUN" - continue - fi - echo "$THIS_RUN" - - $CLICKHOUSE_CLIENT --max_insert_block_size 1 -nmq " - $(python3 $CURDIR/03008_deduplication.python several_mv_into_one_table \ - --insert-method $insert_method \ - --table-engine $ENGINE \ - --use-insert-token $use_insert_token \ - --single-thread $single_thread \ - --deduplicate-src-table $deduplicate_src_table \ - --deduplicate-dst-table $deduplicate_dst_table \ - --insert-unique-blocks $insert_unique_blocks \ - --get-logs false \ - ) - " && echo OK || echo FAIL - done - done - done - done - done -done - -echo -echo "All cases executed" diff --git a/tests/queries/0_stateless/03008_deduplication_several_mv_into_one_table_replicated.reference b/tests/queries/0_stateless/03008_deduplication_several_mv_into_one_table_replicated.reference deleted file mode 100644 index 1921103f49e..00000000000 --- a/tests/queries/0_stateless/03008_deduplication_several_mv_into_one_table_replicated.reference +++ /dev/null @@ -1,706 +0,0 @@ - -Test case 0: insert_method=InsertSelect engine=ReplicatedMergeTree use_insert_token=True single_thread=True deduplicate_src_table=True deduplicate_dst_table=True insert_unique_blocks=True -table_src count 8 -table_dst count 6 -0 -0 -table_src count 8 -table_dst count 6 -0 -0 -OK - -Test case 1: insert_method=InsertSelect engine=ReplicatedMergeTree use_insert_token=True single_thread=True deduplicate_src_table=True deduplicate_dst_table=True insert_unique_blocks=False -table_src count 8 -table_dst count 16 -0 -0 -table_src count 8 -table_dst count 16 -0 -0 -OK - -Test case 2: insert_method=InsertSelect engine=ReplicatedMergeTree use_insert_token=True single_thread=True deduplicate_src_table=True deduplicate_dst_table=False insert_unique_blocks=True -table_src count 8 -table_dst count 6 -0 -0 -table_src count 8 -table_dst count 12 -0 -0 -OK - -Test case 3: insert_method=InsertSelect engine=ReplicatedMergeTree use_insert_token=True single_thread=True deduplicate_src_table=True deduplicate_dst_table=False insert_unique_blocks=False -table_src count 8 -table_dst count 16 -0 -0 -table_src count 8 -table_dst count 32 -0 -0 -OK - -Test case 4: insert_method=InsertSelect engine=ReplicatedMergeTree use_insert_token=True single_thread=True deduplicate_src_table=False deduplicate_dst_table=True insert_unique_blocks=True -table_src count 8 -table_dst count 6 -0 -0 -table_src count 16 -table_dst count 6 -0 -0 -OK - -Test case 5: insert_method=InsertSelect engine=ReplicatedMergeTree use_insert_token=True single_thread=True deduplicate_src_table=False deduplicate_dst_table=True insert_unique_blocks=False -table_src count 8 -table_dst count 16 -0 -0 -table_src count 16 -table_dst count 16 -0 -0 -OK - -Test case 6: insert_method=InsertSelect engine=ReplicatedMergeTree use_insert_token=True single_thread=True deduplicate_src_table=False deduplicate_dst_table=False insert_unique_blocks=True -table_src count 8 -table_dst count 6 -0 -0 -table_src count 16 -table_dst count 12 -0 -0 -OK - -Test case 7: insert_method=InsertSelect engine=ReplicatedMergeTree use_insert_token=True single_thread=True deduplicate_src_table=False deduplicate_dst_table=False insert_unique_blocks=False -table_src count 8 -table_dst count 16 -0 -0 -table_src count 16 -table_dst count 32 -0 -0 -OK - -Test case 8: insert_method=InsertSelect engine=ReplicatedMergeTree use_insert_token=True single_thread=False deduplicate_src_table=True deduplicate_dst_table=True insert_unique_blocks=True -table_src count 8 -table_dst count 6 -0 -0 -table_src count 8 -table_dst count 6 -0 -0 -OK - -Test case 9: insert_method=InsertSelect engine=ReplicatedMergeTree use_insert_token=True single_thread=False deduplicate_src_table=True deduplicate_dst_table=True insert_unique_blocks=False -table_src count 8 -table_dst count 16 -0 -0 -table_src count 8 -table_dst count 16 -0 -0 -OK - -Test case 10: insert_method=InsertSelect engine=ReplicatedMergeTree use_insert_token=True single_thread=False deduplicate_src_table=True deduplicate_dst_table=False insert_unique_blocks=True -table_src count 8 -table_dst count 6 -0 -0 -table_src count 8 -table_dst count 12 -0 -0 -OK - -Test case 11: insert_method=InsertSelect engine=ReplicatedMergeTree use_insert_token=True single_thread=False deduplicate_src_table=True deduplicate_dst_table=False insert_unique_blocks=False -table_src count 8 -table_dst count 16 -0 -0 -table_src count 8 -table_dst count 32 -0 -0 -OK - -Test case 12: insert_method=InsertSelect engine=ReplicatedMergeTree use_insert_token=True single_thread=False deduplicate_src_table=False deduplicate_dst_table=True insert_unique_blocks=True -table_src count 8 -table_dst count 6 -0 -0 -table_src count 16 -table_dst count 6 -0 -0 -OK - -Test case 13: insert_method=InsertSelect engine=ReplicatedMergeTree use_insert_token=True single_thread=False deduplicate_src_table=False deduplicate_dst_table=True insert_unique_blocks=False -table_src count 8 -table_dst count 16 -0 -0 -table_src count 16 -table_dst count 16 -0 -0 -OK - -Test case 14: insert_method=InsertSelect engine=ReplicatedMergeTree use_insert_token=True single_thread=False deduplicate_src_table=False deduplicate_dst_table=False insert_unique_blocks=True -table_src count 8 -table_dst count 6 -0 -0 -table_src count 16 -table_dst count 12 -0 -0 -OK - -Test case 15: insert_method=InsertSelect engine=ReplicatedMergeTree use_insert_token=True single_thread=False deduplicate_src_table=False deduplicate_dst_table=False insert_unique_blocks=False -table_src count 8 -table_dst count 16 -0 -0 -table_src count 16 -table_dst count 32 -0 -0 -OK - -Test case 16: insert_method=InsertSelect engine=ReplicatedMergeTree use_insert_token=False single_thread=True deduplicate_src_table=True deduplicate_dst_table=True insert_unique_blocks=True -table_src count 8 -table_dst count 6 -0 -0 -table_src count 8 -table_dst count 6 -0 -0 -OK - -Test case 17: insert_method=InsertSelect engine=ReplicatedMergeTree use_insert_token=False single_thread=True deduplicate_src_table=True deduplicate_dst_table=True insert_unique_blocks=False -table_src count 1 -table_dst count 2 -0 -0 -table_src count 1 -table_dst count 2 -0 -0 -OK - -Test case 18: insert_method=InsertSelect engine=ReplicatedMergeTree use_insert_token=False single_thread=True deduplicate_src_table=True deduplicate_dst_table=False insert_unique_blocks=True -table_src count 8 -table_dst count 6 -0 -0 -table_src count 8 -table_dst count 12 -0 -0 -OK - -Test case 19: insert_method=InsertSelect engine=ReplicatedMergeTree use_insert_token=False single_thread=True deduplicate_src_table=True deduplicate_dst_table=False insert_unique_blocks=False -table_src count 1 -table_dst count 16 -0 -0 -table_src count 1 -table_dst count 32 -0 -0 -OK - -Test case 20: insert_method=InsertSelect engine=ReplicatedMergeTree use_insert_token=False single_thread=True deduplicate_src_table=False deduplicate_dst_table=True insert_unique_blocks=True -table_src count 8 -table_dst count 6 -0 -0 -table_src count 16 -table_dst count 6 -0 -0 -OK - -Test case 21: insert_method=InsertSelect engine=ReplicatedMergeTree use_insert_token=False single_thread=True deduplicate_src_table=False deduplicate_dst_table=True insert_unique_blocks=False -table_src count 8 -table_dst count 2 -0 -0 -table_src count 16 -table_dst count 2 -0 -0 -OK - -Test case 22: insert_method=InsertSelect engine=ReplicatedMergeTree use_insert_token=False single_thread=True deduplicate_src_table=False deduplicate_dst_table=False insert_unique_blocks=True -table_src count 8 -table_dst count 6 -0 -0 -table_src count 16 -table_dst count 12 -0 -0 -OK - -Test case 23: insert_method=InsertSelect engine=ReplicatedMergeTree use_insert_token=False single_thread=True deduplicate_src_table=False deduplicate_dst_table=False insert_unique_blocks=False -table_src count 8 -table_dst count 16 -0 -0 -table_src count 16 -table_dst count 32 -0 -0 -OK - -Test case 24: insert_method=InsertSelect engine=ReplicatedMergeTree use_insert_token=False single_thread=False deduplicate_src_table=True deduplicate_dst_table=True insert_unique_blocks=True -table_src count 8 -table_dst count 6 -0 -0 -table_src count 8 -table_dst count 6 -0 -0 -OK - -Test case 25: insert_method=InsertSelect engine=ReplicatedMergeTree use_insert_token=False single_thread=False deduplicate_src_table=True deduplicate_dst_table=True insert_unique_blocks=False -table_src count 1 -table_dst count 2 -0 -0 -table_src count 1 -table_dst count 2 -0 -0 -OK - -Test case 26: insert_method=InsertSelect engine=ReplicatedMergeTree use_insert_token=False single_thread=False deduplicate_src_table=True deduplicate_dst_table=False insert_unique_blocks=True -table_src count 8 -table_dst count 6 -0 -0 -table_src count 8 -table_dst count 12 -0 -0 -OK - -Test case 27: insert_method=InsertSelect engine=ReplicatedMergeTree use_insert_token=False single_thread=False deduplicate_src_table=True deduplicate_dst_table=False insert_unique_blocks=False -table_src count 1 -table_dst count 16 -0 -0 -table_src count 1 -table_dst count 32 -0 -0 -OK - -Test case 28: insert_method=InsertSelect engine=ReplicatedMergeTree use_insert_token=False single_thread=False deduplicate_src_table=False deduplicate_dst_table=True insert_unique_blocks=True -table_src count 8 -table_dst count 6 -0 -0 -table_src count 16 -table_dst count 6 -0 -0 -OK - -Test case 29: insert_method=InsertSelect engine=ReplicatedMergeTree use_insert_token=False single_thread=False deduplicate_src_table=False deduplicate_dst_table=True insert_unique_blocks=False -table_src count 8 -table_dst count 2 -0 -0 -table_src count 16 -table_dst count 2 -0 -0 -OK - -Test case 30: insert_method=InsertSelect engine=ReplicatedMergeTree use_insert_token=False single_thread=False deduplicate_src_table=False deduplicate_dst_table=False insert_unique_blocks=True -table_src count 8 -table_dst count 6 -0 -0 -table_src count 16 -table_dst count 12 -0 -0 -OK - -Test case 31: insert_method=InsertSelect engine=ReplicatedMergeTree use_insert_token=False single_thread=False deduplicate_src_table=False deduplicate_dst_table=False insert_unique_blocks=False -table_src count 8 -table_dst count 16 -0 -0 -table_src count 16 -table_dst count 32 -0 -0 -OK - -Test case 32: insert_method=InsertValues engine=ReplicatedMergeTree use_insert_token=True single_thread=True deduplicate_src_table=True deduplicate_dst_table=True insert_unique_blocks=True -table_src count 8 -table_dst count 6 -0 -0 -table_src count 8 -table_dst count 6 -0 -0 -OK - -Test case 33: insert_method=InsertValues engine=ReplicatedMergeTree use_insert_token=True single_thread=True deduplicate_src_table=True deduplicate_dst_table=True insert_unique_blocks=False -table_src count 8 -table_dst count 16 -0 -0 -table_src count 8 -table_dst count 16 -0 -0 -OK - -Test case 34: insert_method=InsertValues engine=ReplicatedMergeTree use_insert_token=True single_thread=True deduplicate_src_table=True deduplicate_dst_table=False insert_unique_blocks=True -table_src count 8 -table_dst count 6 -0 -0 -table_src count 8 -table_dst count 12 -0 -0 -OK - -Test case 35: insert_method=InsertValues engine=ReplicatedMergeTree use_insert_token=True single_thread=True deduplicate_src_table=True deduplicate_dst_table=False insert_unique_blocks=False -table_src count 8 -table_dst count 16 -0 -0 -table_src count 8 -table_dst count 32 -0 -0 -OK - -Test case 36: insert_method=InsertValues engine=ReplicatedMergeTree use_insert_token=True single_thread=True deduplicate_src_table=False deduplicate_dst_table=True insert_unique_blocks=True -table_src count 8 -table_dst count 6 -0 -0 -table_src count 16 -table_dst count 6 -0 -0 -OK - -Test case 37: insert_method=InsertValues engine=ReplicatedMergeTree use_insert_token=True single_thread=True deduplicate_src_table=False deduplicate_dst_table=True insert_unique_blocks=False -table_src count 8 -table_dst count 16 -0 -0 -table_src count 16 -table_dst count 16 -0 -0 -OK - -Test case 38: insert_method=InsertValues engine=ReplicatedMergeTree use_insert_token=True single_thread=True deduplicate_src_table=False deduplicate_dst_table=False insert_unique_blocks=True -table_src count 8 -table_dst count 6 -0 -0 -table_src count 16 -table_dst count 12 -0 -0 -OK - -Test case 39: insert_method=InsertValues engine=ReplicatedMergeTree use_insert_token=True single_thread=True deduplicate_src_table=False deduplicate_dst_table=False insert_unique_blocks=False -table_src count 8 -table_dst count 16 -0 -0 -table_src count 16 -table_dst count 32 -0 -0 -OK - -Test case 40: insert_method=InsertValues engine=ReplicatedMergeTree use_insert_token=True single_thread=False deduplicate_src_table=True deduplicate_dst_table=True insert_unique_blocks=True -table_src count 8 -table_dst count 6 -0 -0 -table_src count 8 -table_dst count 6 -0 -0 -OK - -Test case 41: insert_method=InsertValues engine=ReplicatedMergeTree use_insert_token=True single_thread=False deduplicate_src_table=True deduplicate_dst_table=True insert_unique_blocks=False -table_src count 8 -table_dst count 16 -0 -0 -table_src count 8 -table_dst count 16 -0 -0 -OK - -Test case 42: insert_method=InsertValues engine=ReplicatedMergeTree use_insert_token=True single_thread=False deduplicate_src_table=True deduplicate_dst_table=False insert_unique_blocks=True -table_src count 8 -table_dst count 6 -0 -0 -table_src count 8 -table_dst count 12 -0 -0 -OK - -Test case 43: insert_method=InsertValues engine=ReplicatedMergeTree use_insert_token=True single_thread=False deduplicate_src_table=True deduplicate_dst_table=False insert_unique_blocks=False -table_src count 8 -table_dst count 16 -0 -0 -table_src count 8 -table_dst count 32 -0 -0 -OK - -Test case 44: insert_method=InsertValues engine=ReplicatedMergeTree use_insert_token=True single_thread=False deduplicate_src_table=False deduplicate_dst_table=True insert_unique_blocks=True -table_src count 8 -table_dst count 6 -0 -0 -table_src count 16 -table_dst count 6 -0 -0 -OK - -Test case 45: insert_method=InsertValues engine=ReplicatedMergeTree use_insert_token=True single_thread=False deduplicate_src_table=False deduplicate_dst_table=True insert_unique_blocks=False -table_src count 8 -table_dst count 16 -0 -0 -table_src count 16 -table_dst count 16 -0 -0 -OK - -Test case 46: insert_method=InsertValues engine=ReplicatedMergeTree use_insert_token=True single_thread=False deduplicate_src_table=False deduplicate_dst_table=False insert_unique_blocks=True -table_src count 8 -table_dst count 6 -0 -0 -table_src count 16 -table_dst count 12 -0 -0 -OK - -Test case 47: insert_method=InsertValues engine=ReplicatedMergeTree use_insert_token=True single_thread=False deduplicate_src_table=False deduplicate_dst_table=False insert_unique_blocks=False -table_src count 8 -table_dst count 16 -0 -0 -table_src count 16 -table_dst count 32 -0 -0 -OK - -Test case 48: insert_method=InsertValues engine=ReplicatedMergeTree use_insert_token=False single_thread=True deduplicate_src_table=True deduplicate_dst_table=True insert_unique_blocks=True -table_src count 8 -table_dst count 6 -0 -0 -table_src count 8 -table_dst count 6 -0 -0 -OK - -Test case 49: insert_method=InsertValues engine=ReplicatedMergeTree use_insert_token=False single_thread=True deduplicate_src_table=True deduplicate_dst_table=True insert_unique_blocks=False -table_src count 1 -table_dst count 2 -0 -0 -table_src count 1 -table_dst count 2 -0 -0 -OK - -Test case 50: insert_method=InsertValues engine=ReplicatedMergeTree use_insert_token=False single_thread=True deduplicate_src_table=True deduplicate_dst_table=False insert_unique_blocks=True -table_src count 8 -table_dst count 6 -0 -0 -table_src count 8 -table_dst count 12 -0 -0 -OK - -Test case 51: insert_method=InsertValues engine=ReplicatedMergeTree use_insert_token=False single_thread=True deduplicate_src_table=True deduplicate_dst_table=False insert_unique_blocks=False -table_src count 1 -table_dst count 16 -0 -0 -table_src count 1 -table_dst count 32 -0 -0 -OK - -Test case 52: insert_method=InsertValues engine=ReplicatedMergeTree use_insert_token=False single_thread=True deduplicate_src_table=False deduplicate_dst_table=True insert_unique_blocks=True -table_src count 8 -table_dst count 6 -0 -0 -table_src count 16 -table_dst count 6 -0 -0 -OK - -Test case 53: insert_method=InsertValues engine=ReplicatedMergeTree use_insert_token=False single_thread=True deduplicate_src_table=False deduplicate_dst_table=True insert_unique_blocks=False -table_src count 8 -table_dst count 2 -0 -0 -table_src count 16 -table_dst count 2 -0 -0 -OK - -Test case 54: insert_method=InsertValues engine=ReplicatedMergeTree use_insert_token=False single_thread=True deduplicate_src_table=False deduplicate_dst_table=False insert_unique_blocks=True -table_src count 8 -table_dst count 6 -0 -0 -table_src count 16 -table_dst count 12 -0 -0 -OK - -Test case 55: insert_method=InsertValues engine=ReplicatedMergeTree use_insert_token=False single_thread=True deduplicate_src_table=False deduplicate_dst_table=False insert_unique_blocks=False -table_src count 8 -table_dst count 16 -0 -0 -table_src count 16 -table_dst count 32 -0 -0 -OK - -Test case 56: insert_method=InsertValues engine=ReplicatedMergeTree use_insert_token=False single_thread=False deduplicate_src_table=True deduplicate_dst_table=True insert_unique_blocks=True -table_src count 8 -table_dst count 6 -0 -0 -table_src count 8 -table_dst count 6 -0 -0 -OK - -Test case 57: insert_method=InsertValues engine=ReplicatedMergeTree use_insert_token=False single_thread=False deduplicate_src_table=True deduplicate_dst_table=True insert_unique_blocks=False -table_src count 1 -table_dst count 2 -0 -0 -table_src count 1 -table_dst count 2 -0 -0 -OK - -Test case 58: insert_method=InsertValues engine=ReplicatedMergeTree use_insert_token=False single_thread=False deduplicate_src_table=True deduplicate_dst_table=False insert_unique_blocks=True -table_src count 8 -table_dst count 6 -0 -0 -table_src count 8 -table_dst count 12 -0 -0 -OK - -Test case 59: insert_method=InsertValues engine=ReplicatedMergeTree use_insert_token=False single_thread=False deduplicate_src_table=True deduplicate_dst_table=False insert_unique_blocks=False -table_src count 1 -table_dst count 16 -0 -0 -table_src count 1 -table_dst count 32 -0 -0 -OK - -Test case 60: insert_method=InsertValues engine=ReplicatedMergeTree use_insert_token=False single_thread=False deduplicate_src_table=False deduplicate_dst_table=True insert_unique_blocks=True -table_src count 8 -table_dst count 6 -0 -0 -table_src count 16 -table_dst count 6 -0 -0 -OK - -Test case 61: insert_method=InsertValues engine=ReplicatedMergeTree use_insert_token=False single_thread=False deduplicate_src_table=False deduplicate_dst_table=True insert_unique_blocks=False -table_src count 8 -table_dst count 2 -0 -0 -table_src count 16 -table_dst count 2 -0 -0 -OK - -Test case 62: insert_method=InsertValues engine=ReplicatedMergeTree use_insert_token=False single_thread=False deduplicate_src_table=False deduplicate_dst_table=False insert_unique_blocks=True -table_src count 8 -table_dst count 6 -0 -0 -table_src count 16 -table_dst count 12 -0 -0 -OK - -Test case 63: insert_method=InsertValues engine=ReplicatedMergeTree use_insert_token=False single_thread=False deduplicate_src_table=False deduplicate_dst_table=False insert_unique_blocks=False -table_src count 8 -table_dst count 16 -0 -0 -table_src count 16 -table_dst count 32 -0 -0 -OK - -All cases executed diff --git a/tests/queries/0_stateless/03008_deduplication_several_mv_into_one_table_replicated.sh b/tests/queries/0_stateless/03008_deduplication_several_mv_into_one_table_replicated.sh deleted file mode 100755 index 9adee6d53d4..00000000000 --- a/tests/queries/0_stateless/03008_deduplication_several_mv_into_one_table_replicated.sh +++ /dev/null @@ -1,59 +0,0 @@ -#!/usr/bin/env bash -# Tags: long, no-fasttest, no-parallel - -CURDIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd) -# shellcheck source=../shell_config.sh -. "$CURDIR"/../shell_config.sh - -ENGINE="ReplicatedMergeTree" - -RUN_ONLY="" -#RUN_ONLY="Test case 17: insert_method=InsertSelect engine=ReplicatedMergeTree use_insert_token=False single_thread=True deduplicate_src_table=True deduplicate_dst_table=True insert_unique_blocks=False" - -i=0 -for insert_method in "InsertSelect" "InsertValues"; do - for use_insert_token in "True" "False"; do - for single_thread in "True" "False"; do - for deduplicate_src_table in "True" "False"; do - for deduplicate_dst_table in "True" "False"; do - for insert_unique_blocks in "True" "False"; do - - THIS_RUN="Test case $i:" - THIS_RUN+=" insert_method=$insert_method" - THIS_RUN+=" engine=$ENGINE" - THIS_RUN+=" use_insert_token=$use_insert_token" - THIS_RUN+=" single_thread=$single_thread" - THIS_RUN+=" deduplicate_src_table=$deduplicate_src_table" - THIS_RUN+=" deduplicate_dst_table=$deduplicate_dst_table" - THIS_RUN+=" insert_unique_blocks=$insert_unique_blocks" - - i=$((i+1)) - - echo - if [ -n "$RUN_ONLY" ] && [ "$RUN_ONLY" != "$THIS_RUN" ]; then - echo "skip $THIS_RUN" - continue - fi - echo "$THIS_RUN" - - $CLICKHOUSE_CLIENT --max_insert_block_size 1 -nmq " - $(python3 $CURDIR/03008_deduplication.python several_mv_into_one_table \ - --insert-method $insert_method \ - --table-engine $ENGINE \ - --use-insert-token $use_insert_token \ - --single-thread $single_thread \ - --deduplicate-src-table $deduplicate_src_table \ - --deduplicate-dst-table $deduplicate_dst_table \ - --insert-unique-blocks $insert_unique_blocks \ - --get-logs false \ - ) - " && echo OK || echo FAIL - done - done - done - done - done -done - -echo -echo "All cases executed" diff --git a/tests/queries/0_stateless/03035_max_insert_threads_support.sh b/tests/queries/0_stateless/03035_max_insert_threads_support.sh index cedb651a430..1e6bfb414d8 100755 --- a/tests/queries/0_stateless/03035_max_insert_threads_support.sh +++ b/tests/queries/0_stateless/03035_max_insert_threads_support.sh @@ -8,7 +8,7 @@ DATA_FILE="data_$CLICKHOUSE_TEST_UNIQUE_NAME.csv" $CLICKHOUSE_CLIENT --max_insert_threads=4 --query=" EXPLAIN PIPELINE INSERT INTO FUNCTION file('$DATA_FILE') SELECT * FROM numbers_mt(1000000) ORDER BY number DESC -" | grep -o StorageFileSink | wc -l +" | grep -o MaterializingTransform | wc -l DATA_FILE_PATH=$($CLICKHOUSE_CLIENT_BINARY --query "select _path from file('$DATA_FILE', 'One')") rm $DATA_FILE_PATH From bf0b782960dd57250eaf5b48e1a55843ac6a1e5c Mon Sep 17 00:00:00 2001 From: Antonio Andelic Date: Fri, 5 Jul 2024 12:34:23 +0200 Subject: [PATCH 228/417] Correctly print long processing requests in Keeper --- src/Coordination/KeeperDispatcher.cpp | 27 +++++---- src/Coordination/KeeperDispatcher.h | 4 +- src/Coordination/KeeperStateMachine.cpp | 31 +++++----- src/Coordination/KeeperStorage.h | 1 + src/Server/KeeperTCPHandler.cpp | 75 ++++++++++++++----------- src/Server/KeeperTCPHandler.h | 10 +++- 6 files changed, 86 insertions(+), 62 deletions(-) diff --git a/src/Coordination/KeeperDispatcher.cpp b/src/Coordination/KeeperDispatcher.cpp index f36b1ef151f..85960ac659a 100644 --- a/src/Coordination/KeeperDispatcher.cpp +++ b/src/Coordination/KeeperDispatcher.cpp @@ -6,6 +6,7 @@ #include #include "Common/ZooKeeper/IKeeper.h" +#include "Common/ZooKeeper/ZooKeeperCommon.h" #include #include #include @@ -320,7 +321,7 @@ void KeeperDispatcher::responseThread() try { - setResponse(response_for_session.session_id, response_for_session.response); + setResponse(response_for_session.session_id, response_for_session.response, response_for_session.request); } catch (...) { @@ -355,7 +356,7 @@ void KeeperDispatcher::snapshotThread() } } -void KeeperDispatcher::setResponse(int64_t session_id, const Coordination::ZooKeeperResponsePtr & response) +void KeeperDispatcher::setResponse(int64_t session_id, const Coordination::ZooKeeperResponsePtr & response, Coordination::ZooKeeperRequestPtr request) { std::lock_guard lock(session_to_response_callback_mutex); @@ -369,7 +370,7 @@ void KeeperDispatcher::setResponse(int64_t session_id, const Coordination::ZooKe return; auto callback = new_session_id_response_callback[session_id_resp.internal_id]; - callback(response); + callback(response, request); new_session_id_response_callback.erase(session_id_resp.internal_id); } else /// Normal response, just write to client @@ -380,7 +381,7 @@ void KeeperDispatcher::setResponse(int64_t session_id, const Coordination::ZooKe if (session_response_callback == session_to_response_callback.end()) return; - session_response_callback->second(response); + session_response_callback->second(response, request); /// Session closed, no more writes if (response->xid != Coordination::WATCH_XID && response->getOpNum() == Coordination::OpNum::Close) @@ -771,21 +772,27 @@ int64_t KeeperDispatcher::getSessionID(int64_t session_timeout_ms) { std::lock_guard lock(session_to_response_callback_mutex); - new_session_id_response_callback[request->internal_id] = [promise, internal_id = request->internal_id] (const Coordination::ZooKeeperResponsePtr & response) + new_session_id_response_callback[request->internal_id] + = [promise, internal_id = request->internal_id]( + const Coordination::ZooKeeperResponsePtr & response, Coordination::ZooKeeperRequestPtr /*request*/) { if (response->getOpNum() != Coordination::OpNum::SessionID) - promise->set_exception(std::make_exception_ptr(Exception(ErrorCodes::LOGICAL_ERROR, - "Incorrect response of type {} instead of SessionID response", response->getOpNum()))); + promise->set_exception(std::make_exception_ptr(Exception( + ErrorCodes::LOGICAL_ERROR, "Incorrect response of type {} instead of SessionID response", response->getOpNum()))); auto session_id_response = dynamic_cast(*response); if (session_id_response.internal_id != internal_id) { - promise->set_exception(std::make_exception_ptr(Exception(ErrorCodes::LOGICAL_ERROR, - "Incorrect response with internal id {} instead of {}", session_id_response.internal_id, internal_id))); + promise->set_exception(std::make_exception_ptr(Exception( + ErrorCodes::LOGICAL_ERROR, + "Incorrect response with internal id {} instead of {}", + session_id_response.internal_id, + internal_id))); } if (response->error != Coordination::Error::ZOK) - promise->set_exception(std::make_exception_ptr(zkutil::KeeperException::fromMessage(response->error, "SessionID request failed with error"))); + promise->set_exception( + std::make_exception_ptr(zkutil::KeeperException::fromMessage(response->error, "SessionID request failed with error"))); promise->set_value(session_id_response.session_id); }; diff --git a/src/Coordination/KeeperDispatcher.h b/src/Coordination/KeeperDispatcher.h index 2e0c73131d5..a487b886d98 100644 --- a/src/Coordination/KeeperDispatcher.h +++ b/src/Coordination/KeeperDispatcher.h @@ -20,7 +20,7 @@ namespace DB { -using ZooKeeperResponseCallback = std::function; +using ZooKeeperResponseCallback = std::function; /// Highlevel wrapper for ClickHouse Keeper. /// Process user requests via consensus and return responses. @@ -92,7 +92,7 @@ private: void clusterUpdateWithReconfigDisabledThread(); void clusterUpdateThread(); - void setResponse(int64_t session_id, const Coordination::ZooKeeperResponsePtr & response); + void setResponse(int64_t session_id, const Coordination::ZooKeeperResponsePtr & response, Coordination::ZooKeeperRequestPtr request = nullptr); /// Add error responses for requests to responses queue. /// Clears requests. diff --git a/src/Coordination/KeeperStateMachine.cpp b/src/Coordination/KeeperStateMachine.cpp index e7cae714ba6..3d3d862e1dd 100644 --- a/src/Coordination/KeeperStateMachine.cpp +++ b/src/Coordination/KeeperStateMachine.cpp @@ -407,7 +407,7 @@ nuraft::ptr KeeperStateMachine::commit(const uint64_t log_idx, n if (!keeper_context->localLogsPreprocessed() && !preprocess(*request_for_session)) return nullptr; - auto try_push = [&](const KeeperStorage::ResponseForSession& response) + auto try_push = [&](const KeeperStorage::ResponseForSession & response) { if (!responses_queue.push(response)) { @@ -416,17 +416,6 @@ nuraft::ptr KeeperStateMachine::commit(const uint64_t log_idx, n "Failed to push response with session id {} to the queue, probably because of shutdown", response.session_id); } - - using namespace std::chrono; - uint64_t elapsed = duration_cast(system_clock::now().time_since_epoch()).count() - request_for_session->time; - if (elapsed > keeper_context->getCoordinationSettings()->log_slow_total_threshold_ms) - { - LOG_INFO( - log, - "Total time to process a request took too long ({}ms).\nRequest info: {}", - elapsed, - request_for_session->request->toString(/*short_format=*/true)); - } }; try @@ -443,6 +432,7 @@ nuraft::ptr KeeperStateMachine::commit(const uint64_t log_idx, n KeeperStorage::ResponseForSession response_for_session; response_for_session.session_id = -1; response_for_session.response = response; + response_for_session.request = request_for_session->request; LockGuardWithStats lock(storage_and_responses_lock); session_id = storage->getSessionID(session_id_request.session_timeout_ms); @@ -462,8 +452,14 @@ nuraft::ptr KeeperStateMachine::commit(const uint64_t log_idx, n LockGuardWithStats lock(storage_and_responses_lock); KeeperStorage::ResponsesForSessions responses_for_sessions = storage->processRequest(request_for_session->request, request_for_session->session_id, request_for_session->zxid); + for (auto & response_for_session : responses_for_sessions) + { + if (response_for_session.response->xid != Coordination::WATCH_XID) + response_for_session.request = request_for_session->request; + try_push(response_for_session); + } if (keeper_context->digestEnabled() && request_for_session->digest) assertDigest(*request_for_session->digest, storage->getNodesDigest(true), *request_for_session->request, request_for_session->log_idx, true); @@ -797,9 +793,14 @@ void KeeperStateMachine::processReadRequest(const KeeperStorage::RequestForSessi LockGuardWithStats lock(storage_and_responses_lock); auto responses = storage->processRequest( request_for_session.request, request_for_session.session_id, std::nullopt, true /*check_acl*/, true /*is_local*/); - for (const auto & response : responses) - if (!responses_queue.push(response)) - LOG_WARNING(log, "Failed to push response with session id {} to the queue, probably because of shutdown", response.session_id); + + for (auto & response_for_session : responses) + { + if (response_for_session.response->xid != Coordination::WATCH_XID) + response_for_session.request = request_for_session.request; + if (!responses_queue.push(response_for_session)) + LOG_WARNING(log, "Failed to push response with session id {} to the queue, probably because of shutdown", response_for_session.session_id); + } } void KeeperStateMachine::shutdownStorage() diff --git a/src/Coordination/KeeperStorage.h b/src/Coordination/KeeperStorage.h index d5e9a64e69c..f7812ad8877 100644 --- a/src/Coordination/KeeperStorage.h +++ b/src/Coordination/KeeperStorage.h @@ -206,6 +206,7 @@ public: { int64_t session_id; Coordination::ZooKeeperResponsePtr response; + Coordination::ZooKeeperRequestPtr request = nullptr; }; using ResponsesForSessions = std::vector; diff --git a/src/Server/KeeperTCPHandler.cpp b/src/Server/KeeperTCPHandler.cpp index 47064b467e7..5f26542d39c 100644 --- a/src/Server/KeeperTCPHandler.cpp +++ b/src/Server/KeeperTCPHandler.cpp @@ -2,31 +2,31 @@ #if USE_NURAFT -#include -#include -#include -#include -#include -#include -#include -#include -#include -#include -#include -#include -#include -#include -#include -#include -#include -#include +# include +# include +# include +# include +# include +# include +# include +# include +# include +# include +# include +# include +# include +# include +# include +# include +# include +# include -#ifdef POCO_HAVE_FD_EPOLL - #include -#else - #include -#endif +# ifdef POCO_HAVE_FD_EPOLL +# include +# else +# include +# endif namespace ProfileEvents { @@ -400,13 +400,11 @@ void KeeperTCPHandler::runImpl() } auto response_fd = poll_wrapper->getResponseFD(); - auto response_callback = [responses_ = this->responses, response_fd](const Coordination::ZooKeeperResponsePtr & response) + auto response_callback = [my_responses = this->responses, + response_fd](const Coordination::ZooKeeperResponsePtr & response, Coordination::ZooKeeperRequestPtr request) { - if (!responses_->push(response)) - throw Exception(ErrorCodes::SYSTEM_ERROR, - "Could not push response with xid {} and zxid {}", - response->xid, - response->zxid); + if (!my_responses->push(RequestWithResponse{response, std::move(request)})) + throw Exception(ErrorCodes::SYSTEM_ERROR, "Could not push response with xid {} and zxid {}", response->xid, response->zxid); UInt8 single_byte = 1; [[maybe_unused]] ssize_t result = write(response_fd, &single_byte, sizeof(single_byte)); @@ -470,19 +468,20 @@ void KeeperTCPHandler::runImpl() /// became inconsistent and race condition is possible. while (result.responses_count != 0) { - Coordination::ZooKeeperResponsePtr response; + RequestWithResponse request_with_response; - if (!responses->tryPop(response)) + if (!responses->tryPop(request_with_response)) throw Exception(ErrorCodes::LOGICAL_ERROR, "We must have ready response, but queue is empty. It's a bug."); log_long_operation("Waiting for response to be ready"); + auto & response = request_with_response.response; if (response->xid == close_xid) { LOG_DEBUG(log, "Session #{} successfully closed", session_id); return; } - updateStats(response); + updateStats(response, request_with_response.request); packageSent(); response->write(getWriteBuffer()); @@ -609,7 +608,7 @@ void KeeperTCPHandler::packageReceived() keeper_dispatcher->incrementPacketsReceived(); } -void KeeperTCPHandler::updateStats(Coordination::ZooKeeperResponsePtr & response) +void KeeperTCPHandler::updateStats(Coordination::ZooKeeperResponsePtr & response, const Coordination::ZooKeeperRequestPtr & request) { /// update statistics ignoring watch response and heartbeat. if (response->xid != Coordination::WATCH_XID && response->getOpNum() != Coordination::OpNum::Heartbeat) @@ -617,6 +616,16 @@ void KeeperTCPHandler::updateStats(Coordination::ZooKeeperResponsePtr & response Int64 elapsed = (Poco::Timestamp() - operations[response->xid]); ProfileEvents::increment(ProfileEvents::KeeperTotalElapsedMicroseconds, elapsed); Int64 elapsed_ms = elapsed / 1000; + + if (request && elapsed_ms > static_cast(keeper_dispatcher->getKeeperContext()->getCoordinationSettings()->log_slow_total_threshold_ms)) + { + LOG_INFO( + log, + "Total time to process a request took too long ({}ms).\nRequest info: {}", + elapsed, + request->toString(/*short_format=*/true)); + } + conn_stats.updateLatency(elapsed_ms); operations.erase(response->xid); diff --git a/src/Server/KeeperTCPHandler.h b/src/Server/KeeperTCPHandler.h index c1c522eee89..7c2b8acf624 100644 --- a/src/Server/KeeperTCPHandler.h +++ b/src/Server/KeeperTCPHandler.h @@ -26,7 +26,13 @@ namespace DB struct SocketInterruptablePollWrapper; using SocketInterruptablePollWrapperPtr = std::unique_ptr; -using ThreadSafeResponseQueue = ConcurrentBoundedQueue; +struct RequestWithResponse +{ + Coordination::ZooKeeperResponsePtr response; + Coordination::ZooKeeperRequestPtr request; /// it can be nullptr for some responses +}; + +using ThreadSafeResponseQueue = ConcurrentBoundedQueue; using ThreadSafeResponseQueuePtr = std::shared_ptr; struct LastOp; @@ -104,7 +110,7 @@ private: void packageSent(); void packageReceived(); - void updateStats(Coordination::ZooKeeperResponsePtr & response); + void updateStats(Coordination::ZooKeeperResponsePtr & response, const Coordination::ZooKeeperRequestPtr & request); Poco::Timestamp established; From 7180ae03467b05fb0495d744c066d4df758c37a3 Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?J=C3=A1nos=20Benjamin=20Antal?= Date: Fri, 5 Jul 2024 13:18:48 +0000 Subject: [PATCH 229/417] Add `use_same_s3_credentials_for_base_backup` to docs --- docs/en/operations/backup.md | 1 + 1 file changed, 1 insertion(+) diff --git a/docs/en/operations/backup.md b/docs/en/operations/backup.md index 2ba50b39934..7c102c38fa6 100644 --- a/docs/en/operations/backup.md +++ b/docs/en/operations/backup.md @@ -84,6 +84,7 @@ The BACKUP and RESTORE statements take a list of DATABASE and TABLE names, a des - [`compression_method`](/docs/en/sql-reference/statements/create/table.md/#column-compression-codecs) and compression_level - `password` for the file on disk - `base_backup`: the destination of the previous backup of this source. For example, `Disk('backups', '1.zip')` + - `use_same_s3_credentials_for_base_backup`: whether base backup to S3 should inherit credentials from the query. Only works with `S3`. - `structure_only`: if enabled, allows to only backup or restore the CREATE statements without the data of tables - `storage_policy`: storage policy for the tables being restored. See [Using Multiple Block Devices for Data Storage](../engines/table-engines/mergetree-family/mergetree.md#table_engine-mergetree-multiple-volumes). This setting is only applicable to the `RESTORE` command. The specified storage policy applies only to tables with an engine from the `MergeTree` family. - `s3_storage_class`: the storage class used for S3 backup. For example, `STANDARD` From d3f23c2753ff2b1ac935268c1af0609616381782 Mon Sep 17 00:00:00 2001 From: Robert Schulze Date: Fri, 5 Jul 2024 13:29:34 +0000 Subject: [PATCH 230/417] Bump s2geometry again --- contrib/s2geometry | 2 +- contrib/s2geometry-cmake/CMakeLists.txt | 3 +-- 2 files changed, 2 insertions(+), 3 deletions(-) diff --git a/contrib/s2geometry b/contrib/s2geometry index 0146e2d1355..6522a40338d 160000 --- a/contrib/s2geometry +++ b/contrib/s2geometry @@ -1 +1 @@ -Subproject commit 0146e2d1355828f8f633cb050948250ad7406c57 +Subproject commit 6522a40338d58752c2a4227a3fc2bc4107c73e43 diff --git a/contrib/s2geometry-cmake/CMakeLists.txt b/contrib/s2geometry-cmake/CMakeLists.txt index 5eabe71b538..48562b8cead 100644 --- a/contrib/s2geometry-cmake/CMakeLists.txt +++ b/contrib/s2geometry-cmake/CMakeLists.txt @@ -1,7 +1,6 @@ option(ENABLE_S2_GEOMETRY "Enable S2 Geometry" ${ENABLE_LIBRARIES}) -# ARCH_S390X broke upstream, it can be re-enabled once https://github.com/google/s2geometry/pull/372 is merged -if (NOT ENABLE_S2_GEOMETRY OR ARCH_S390X) +if (NOT ENABLE_S2_GEOMETRY) message(STATUS "Not using S2 Geometry") return() endif() From 40fd1502794a199d576ae8c57c5e1d08238b00f7 Mon Sep 17 00:00:00 2001 From: Vitaly Baranov Date: Fri, 5 Jul 2024 15:30:32 +0200 Subject: [PATCH 231/417] Switch submodule contrib/orc to proper commit in the main branch. --- contrib/orc | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/contrib/orc b/contrib/orc index 947cebaf943..bcc025c0982 160000 --- a/contrib/orc +++ b/contrib/orc @@ -1 +1 @@ -Subproject commit 947cebaf9432d708253ac08dc3012daa6b4ede6f +Subproject commit bcc025c09828c556f54cfbdf83a66b9acae7d17f From 3a79b9dc8f672e3993057c989618bd7f17d622de Mon Sep 17 00:00:00 2001 From: Blargian Date: Fri, 5 Jul 2024 15:35:00 +0200 Subject: [PATCH 232/417] start and end variants also accept tuple of parameters --- .../functions/time-window-functions.md | 13 +++++++++---- 1 file changed, 9 insertions(+), 4 deletions(-) diff --git a/docs/en/sql-reference/functions/time-window-functions.md b/docs/en/sql-reference/functions/time-window-functions.md index bad545fc5a5..5169d4487ec 100644 --- a/docs/en/sql-reference/functions/time-window-functions.md +++ b/docs/en/sql-reference/functions/time-window-functions.md @@ -50,7 +50,6 @@ Returns the inclusive lower bound of the corresponding [tumbling window](#tumble **Syntax** ``` sql -tumbleStart(bounds_tuple); tumbleStart(time_attr, interval [, timezone]); ``` @@ -60,6 +59,8 @@ tumbleStart(time_attr, interval [, timezone]); - `interval` — Window interval in [Interval](../data-types/special-data-types/interval.md). - `timezone` — [Timezone name](../../operations/server-configuration-parameters/settings.md#server_configuration_parameters-timezone) (optional). +The parameters above can also be passed to the function as a [tuple](../data-types/tuple.md). + **Returned values** - The inclusive lower bound of the corresponding tumbling window. [DateTime](../data-types/datetime.md), [Tuple](../data-types/tuple.md) or [UInt32](../data-types/int-uint.md). @@ -87,7 +88,6 @@ Returns the exclusive upper bound of the corresponding [tumbling window](#tumble **Syntax** ``` sql -tumbleEnd(bounds_tuple); tumbleEnd(time_attr, interval [, timezone]); ``` @@ -97,6 +97,8 @@ tumbleEnd(time_attr, interval [, timezone]); - `interval` — Window interval in [Interval](../data-types/special-data-types/interval.md). - `timezone` — [Timezone name](../../operations/server-configuration-parameters/settings.md#server_configuration_parameters-timezone) (optional). +The parameters above can also be passed to the function as a [tuple](../data-types/tuple.md). + **Returned values** - The inclusive lower bound of the corresponding tumbling window. [DateTime](../data-types/datetime.md), [Tuple](../data-types/tuple.md) or [UInt32](../data-types/int-uint.md). @@ -163,7 +165,6 @@ Returns the inclusive lower bound of the corresponding [hopping window](#hop). **Syntax** ``` sql -hopStart(bounds_tuple); hopStart(time_attr, hop_interval, window_interval [, timezone]); ``` **Arguments** @@ -173,6 +174,8 @@ hopStart(time_attr, hop_interval, window_interval [, timezone]); - `window_interval` — Positive Window interval. [Interval](../data-types/special-data-types/interval.md). - `timezone` — [Timezone name](../../operations/server-configuration-parameters/settings.md#server_configuration_parameters-timezone) (optional). +The parameters above can also be passed to the function as a [tuple](../data-types/tuple.md). + **Returned values** - The inclusive lower bound of the corresponding hopping window. [DateTime](../data-types/datetime.md), [Tuple](../data-types/tuple.md) or [UInt32](../data-types/int-uint.md). @@ -204,7 +207,6 @@ Returns the exclusive upper bound of the corresponding [hopping window](#hop). **Syntax** ``` sql -hopEnd(bounds_tuple); hopEnd(time_attr, hop_interval, window_interval [, timezone]); ``` **Arguments** @@ -214,6 +216,8 @@ hopEnd(time_attr, hop_interval, window_interval [, timezone]); - `window_interval` — Positive Window interval. [Interval](../data-types/special-data-types/interval.md). - `timezone` — [Timezone name](../../operations/server-configuration-parameters/settings.md#server_configuration_parameters-timezone) (optional). +The parameters above can also be passed to the function as a [tuple](../data-types/tuple.md). + **Returned values** - The exclusive upper bound of the corresponding hopping window. [DateTime](../data-types/datetime.md), [Tuple](../data-types/tuple.md) or [UInt32](../data-types/int-uint.md). @@ -236,6 +240,7 @@ Result: ┌─hopEnd(now(), toIntervalDay('1'), toIntervalDay('2'))─┐ │ 2024-07-05 00:00:00 │ └───────────────────────────────────────────────────────┘ + ``` ## Related content From dfef400e158b8065b2b7f32d33f5023445c848a4 Mon Sep 17 00:00:00 2001 From: Nikita Taranov Date: Fri, 5 Jul 2024 14:39:45 +0100 Subject: [PATCH 233/417] impl --- .../MergeTree/MergedBlockOutputStream.cpp | 15 ++++++++++++++- 1 file changed, 14 insertions(+), 1 deletion(-) diff --git a/src/Storages/MergeTree/MergedBlockOutputStream.cpp b/src/Storages/MergeTree/MergedBlockOutputStream.cpp index 164658c914e..9d696b70d9f 100644 --- a/src/Storages/MergeTree/MergedBlockOutputStream.cpp +++ b/src/Storages/MergeTree/MergedBlockOutputStream.cpp @@ -93,6 +93,7 @@ struct MergedBlockOutputStream::Finalizer::Impl void MergedBlockOutputStream::Finalizer::finish() { std::unique_ptr to_finish = std::move(impl); + impl.reset(); if (to_finish) to_finish->finish(); } @@ -130,7 +131,19 @@ MergedBlockOutputStream::Finalizer::Finalizer(Finalizer &&) noexcept = default; MergedBlockOutputStream::Finalizer & MergedBlockOutputStream::Finalizer::operator=(Finalizer &&) noexcept = default; MergedBlockOutputStream::Finalizer::Finalizer(std::unique_ptr impl_) : impl(std::move(impl_)) {} -MergedBlockOutputStream::Finalizer::~Finalizer() = default; +MergedBlockOutputStream::Finalizer::~Finalizer() +{ + try + { + if (impl) + finish(); + } + catch (...) + { + tryLogCurrentException(__PRETTY_FUNCTION__); + } +} + void MergedBlockOutputStream::finalizePart( const MergeTreeMutableDataPartPtr & new_part, From 75d3ffd8df5d0910ad02bab6260d6d637c647391 Mon Sep 17 00:00:00 2001 From: Antonio Andelic Date: Fri, 5 Jul 2024 15:13:11 +0100 Subject: [PATCH 234/417] Fix style --- src/Client/HedgedConnections.cpp | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/src/Client/HedgedConnections.cpp b/src/Client/HedgedConnections.cpp index 9211f39f0ae..4effc3adafa 100644 --- a/src/Client/HedgedConnections.cpp +++ b/src/Client/HedgedConnections.cpp @@ -200,7 +200,7 @@ void HedgedConnections::sendQuery( /// In other words, the initiator always controls whether the analyzer enabled or not for /// all servers involved in the distributed query processing. modified_settings.set("allow_experimental_analyzer", static_cast(modified_settings.allow_experimental_analyzer)); - + replica.connection->sendQuery( timeouts, query, /* query_parameters */ {}, query_id, stage, &modified_settings, &client_info, with_pending_data, {}); replica.change_replica_timeout.setRelative(timeouts.receive_data_timeout); From 964047bf8c676c520d28d30da7ed237af76147df Mon Sep 17 00:00:00 2001 From: alesapin Date: Fri, 5 Jul 2024 16:31:00 +0200 Subject: [PATCH 235/417] Yarrrr --- cmake/target.cmake | 2 +- docker/test/fasttest/Dockerfile | 2 +- 2 files changed, 2 insertions(+), 2 deletions(-) diff --git a/cmake/target.cmake b/cmake/target.cmake index d6c497955f6..3d0ecd032f9 100644 --- a/cmake/target.cmake +++ b/cmake/target.cmake @@ -84,5 +84,5 @@ if (CMAKE_CROSSCOMPILING) message (FATAL_ERROR "Trying to cross-compile to unsupported system: ${CMAKE_SYSTEM_NAME}!") endif () - message (STATUS "Cross-compiling for target: ${CMAKE_CXX_COMPILE_TARGET}") + message (STATUS "Cross-compiling for target: ${CMAKE_CXX_COMPILER_TARGET}") endif () diff --git a/docker/test/fasttest/Dockerfile b/docker/test/fasttest/Dockerfile index cdbfc3f0beb..2512268be0f 100644 --- a/docker/test/fasttest/Dockerfile +++ b/docker/test/fasttest/Dockerfile @@ -39,7 +39,7 @@ RUN sed -i '/_IMPORT_CHECK_FILES_FOR_\(mlir-\|llvm-bolt\|merge-fdata\|MLIR\)/ {s # LLVM changes paths for compiler-rt libraries. For some reason clang-18.1.8 cannot catch up libraries from default install path. # It's very dirty workaround, better to build compiler and LLVM ourself and use it. Details: https://github.com/llvm/llvm-project/issues/95792 -RUN test ! -d /usr/lib/llvm-18/lib/clang/18/lib/x86_64-pc-linux-gnu || mv /usr/lib/llvm-18/lib/clang/18/lib/x86_64-pc-linux-gnu /usr/lib/llvm-18/lib/clang/18/lib/x86_64-unknown-linux-gnu +RUN test ! -d /usr/lib/llvm-18/lib/clang/18/lib/x86_64-pc-linux-gnu || ln -s /usr/lib/llvm-18/lib/clang/18/lib/x86_64-pc-linux-gnu /usr/lib/llvm-18/lib/clang/18/lib/x86_64-unknown-linux-gnu ARG CCACHE_VERSION=4.6.1 RUN mkdir /tmp/ccache \ From ad70710f4bf284f7c41c49d7675eaff4094752ef Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?Ra=C3=BAl=20Mar=C3=ADn?= Date: Fri, 5 Jul 2024 11:57:54 +0200 Subject: [PATCH 236/417] stateless/run.sh cleanup --- docker/test/stateless/run.sh | 13 +++---------- 1 file changed, 3 insertions(+), 10 deletions(-) diff --git a/docker/test/stateless/run.sh b/docker/test/stateless/run.sh index 5ece9743498..983e7f55d43 100755 --- a/docker/test/stateless/run.sh +++ b/docker/test/stateless/run.sh @@ -11,14 +11,13 @@ set -e -x -a # NOTE: that clickhouse-test will randomize session_timezone by itself as well # (it will choose between default server timezone and something specific). TZ="$(rg -v '#' /usr/share/zoneinfo/zone.tab | awk '{print $3}' | shuf | head -n1)" -echo "Choosen random timezone $TZ" +echo "Chosen random timezone $TZ" ln -snf "/usr/share/zoneinfo/$TZ" /etc/localtime && echo "$TZ" > /etc/timezone dpkg -i package_folder/clickhouse-common-static_*.deb dpkg -i package_folder/clickhouse-common-static-dbg_*.deb -# Accept failure in the next two commands until 24.4 is released (for compatibility and Bugfix validation run) -dpkg -i package_folder/clickhouse-odbc-bridge_*.deb || true -dpkg -i package_folder/clickhouse-library-bridge_*.deb || true +dpkg -i package_folder/clickhouse-odbc-bridge_*.deb +dpkg -i package_folder/clickhouse-library-bridge_*.deb dpkg -i package_folder/clickhouse-server_*.deb dpkg -i package_folder/clickhouse-client_*.deb @@ -55,12 +54,6 @@ if [[ -n "$BUGFIX_VALIDATE_CHECK" ]] && [[ "$BUGFIX_VALIDATE_CHECK" -eq 1 ]]; th rm /etc/clickhouse-server/users.d/s3_cache_new.xml rm /etc/clickhouse-server/config.d/zero_copy_destructive_operations.xml - #todo: remove these after 24.3 released. - sudo sed -i "s|azure<|azure_blob_storage<|" /etc/clickhouse-server/config.d/azure_storage_conf.xml - - #todo: remove these after 24.3 released. - sudo sed -i "s|local<|local_blob_storage<|" /etc/clickhouse-server/config.d/storage_conf.xml - function remove_keeper_config() { sudo sed -i "/<$1>$2<\/$1>/d" /etc/clickhouse-server/config.d/keeper_port.xml From 8a1dd3c7c805b3382cf1ed40965cbb3fabb04c2c Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?Ra=C3=BAl=20Mar=C3=ADn?= Date: Fri, 5 Jul 2024 11:58:35 +0200 Subject: [PATCH 237/417] Fix download of debug files for BugFix checker --- tests/ci/download_release_packages.py | 10 +++++----- tests/ci/functional_test_check.py | 2 +- tests/ci/integration_test_check.py | 2 +- 3 files changed, 7 insertions(+), 7 deletions(-) diff --git a/tests/ci/download_release_packages.py b/tests/ci/download_release_packages.py index 8f3a2190ae8..887c94f619f 100755 --- a/tests/ci/download_release_packages.py +++ b/tests/ci/download_release_packages.py @@ -13,26 +13,26 @@ from get_previous_release_tag import ( PACKAGES_DIR = Path("previous_release_package_folder") -def download_packages(release: ReleaseInfo, dest_path: Path = PACKAGES_DIR) -> None: +def download_packages(release: ReleaseInfo, dest_path: Path = PACKAGES_DIR, debug: bool = False) -> None: dest_path.mkdir(parents=True, exist_ok=True) logging.info("Will download %s", release) for pkg, url in release.assets.items(): - if not pkg.endswith("_amd64.deb") or "-dbg_" in pkg: + if not pkg.endswith("_amd64.deb") or (not debug and "-dbg_" in pkg): continue pkg_name = dest_path / pkg download_build_with_progress(url, pkg_name) -def download_last_release(dest_path: Path) -> None: +def download_last_release(dest_path: Path, debug: bool = False) -> None: current_release = get_previous_release(None) if current_release is None: raise DownloadException("The current release is not found") - download_packages(current_release, dest_path=dest_path) + download_packages(current_release, dest_path=dest_path, debug=debug) if __name__ == "__main__": logging.basicConfig(level=logging.INFO) release = get_release_by_tag(input()) - download_packages(release) + download_packages(release, debug=True) diff --git a/tests/ci/functional_test_check.py b/tests/ci/functional_test_check.py index ee459ce35a0..d8e5a7fa27f 100644 --- a/tests/ci/functional_test_check.py +++ b/tests/ci/functional_test_check.py @@ -253,7 +253,7 @@ def main(): packages_path.mkdir(parents=True, exist_ok=True) if validate_bugfix_check: - download_last_release(packages_path) + download_last_release(packages_path, debug=True) else: download_all_deb_packages(check_name, reports_path, packages_path) diff --git a/tests/ci/integration_test_check.py b/tests/ci/integration_test_check.py index 7c74f52b610..80ac1935d95 100644 --- a/tests/ci/integration_test_check.py +++ b/tests/ci/integration_test_check.py @@ -185,7 +185,7 @@ def main(): build_path.mkdir(parents=True, exist_ok=True) if validate_bugfix_check: - download_last_release(build_path) + download_last_release(build_path, debug=True) else: download_all_deb_packages(check_name, reports_path, build_path) From 3435e25b8a2ece06dca6a23dfd1d6f74cec0ee46 Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?Ra=C3=BAl=20Mar=C3=ADn?= Date: Fri, 5 Jul 2024 12:43:49 +0200 Subject: [PATCH 238/417] Linter --- tests/ci/download_release_packages.py | 4 +++- 1 file changed, 3 insertions(+), 1 deletion(-) diff --git a/tests/ci/download_release_packages.py b/tests/ci/download_release_packages.py index 887c94f619f..e8260cf68f1 100755 --- a/tests/ci/download_release_packages.py +++ b/tests/ci/download_release_packages.py @@ -13,7 +13,9 @@ from get_previous_release_tag import ( PACKAGES_DIR = Path("previous_release_package_folder") -def download_packages(release: ReleaseInfo, dest_path: Path = PACKAGES_DIR, debug: bool = False) -> None: +def download_packages( + release: ReleaseInfo, dest_path: Path = PACKAGES_DIR, debug: bool = False +) -> None: dest_path.mkdir(parents=True, exist_ok=True) logging.info("Will download %s", release) From 918502202f06b45332b1ed62c11aab060f94baea Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?Ra=C3=BAl=20Mar=C3=ADn?= Date: Fri, 5 Jul 2024 14:07:39 +0200 Subject: [PATCH 239/417] Modify a random test file to enable BugFix checker --- tests/queries/0_stateless/00027_argMinMax.sql | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/tests/queries/0_stateless/00027_argMinMax.sql b/tests/queries/0_stateless/00027_argMinMax.sql index dbf7c9176d2..57f815add27 100644 --- a/tests/queries/0_stateless/00027_argMinMax.sql +++ b/tests/queries/0_stateless/00027_argMinMax.sql @@ -13,4 +13,4 @@ FROM SELECT arrayJoin([[10, 4, 3], [7, 5, 6], [8, 8, 2]]) AS num, arrayJoin([[1, 2, 4]]) AS id -) +); From 1b1922a2e022211a150157b5d0861f1547d69de7 Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?Ra=C3=BAl=20Mar=C3=ADn?= Date: Fri, 5 Jul 2024 17:44:44 +0200 Subject: [PATCH 240/417] Fix issue in SumIfToCountIfVisitor and signed integers --- src/Analyzer/Passes/SumIfToCountIfPass.cpp | 2 +- ...201_sumIf_to_countIf_return_type.reference | 24 +++++++++++++++++++ .../03201_sumIf_to_countIf_return_type.sql | 2 ++ 3 files changed, 27 insertions(+), 1 deletion(-) create mode 100644 tests/queries/0_stateless/03201_sumIf_to_countIf_return_type.reference create mode 100644 tests/queries/0_stateless/03201_sumIf_to_countIf_return_type.sql diff --git a/src/Analyzer/Passes/SumIfToCountIfPass.cpp b/src/Analyzer/Passes/SumIfToCountIfPass.cpp index f52d724f346..e5ee8a0d0b2 100644 --- a/src/Analyzer/Passes/SumIfToCountIfPass.cpp +++ b/src/Analyzer/Passes/SumIfToCountIfPass.cpp @@ -33,7 +33,7 @@ public: return; auto * function_node = node->as(); - if (!function_node || !function_node->isAggregateFunction()) + if (!function_node || !function_node->isAggregateFunction() || !function_node->getResultType()->equals(DataTypeUInt64())) return; auto function_name = function_node->getFunctionName(); diff --git a/tests/queries/0_stateless/03201_sumIf_to_countIf_return_type.reference b/tests/queries/0_stateless/03201_sumIf_to_countIf_return_type.reference new file mode 100644 index 00000000000..62f5eb45106 --- /dev/null +++ b/tests/queries/0_stateless/03201_sumIf_to_countIf_return_type.reference @@ -0,0 +1,24 @@ +QUERY id: 0 + PROJECTION COLUMNS + (sumIf(toInt64(1), 1)) Tuple(Int64) + PROJECTION + LIST id: 1, nodes: 1 + FUNCTION id: 2, function_name: tuple, function_type: ordinary, result_type: Tuple(Int64) + ARGUMENTS + LIST id: 3, nodes: 1 + FUNCTION id: 4, function_name: sumIf, function_type: aggregate, result_type: Int64 + ARGUMENTS + LIST id: 5, nodes: 2 + CONSTANT id: 6, constant_value: Int64_1, constant_value_type: Int64 + EXPRESSION + FUNCTION id: 7, function_name: toInt64, function_type: ordinary, result_type: Int64 + ARGUMENTS + LIST id: 8, nodes: 1 + CONSTANT id: 9, constant_value: UInt64_1, constant_value_type: UInt8 + CONSTANT id: 10, constant_value: UInt64_1, constant_value_type: UInt8 + JOIN TREE + TABLE_FUNCTION id: 11, alias: __table1, table_function_name: numbers + ARGUMENTS + LIST id: 12, nodes: 1 + CONSTANT id: 13, constant_value: UInt64_100, constant_value_type: UInt8 + SETTINGS optimize_rewrite_sum_if_to_count_if=1 diff --git a/tests/queries/0_stateless/03201_sumIf_to_countIf_return_type.sql b/tests/queries/0_stateless/03201_sumIf_to_countIf_return_type.sql new file mode 100644 index 00000000000..24369fd6497 --- /dev/null +++ b/tests/queries/0_stateless/03201_sumIf_to_countIf_return_type.sql @@ -0,0 +1,2 @@ +SET allow_experimental_analyzer = 1; +EXPLAIN QUERY TREE SELECT tuple(sumIf(toInt64(1), 1)) FROM numbers(100) settings optimize_rewrite_sum_if_to_count_if=1; From 6e8edf008ff0090762faa875af14b9694f026a86 Mon Sep 17 00:00:00 2001 From: Nikita Taranov Date: Fri, 5 Jul 2024 18:36:50 +0200 Subject: [PATCH 241/417] try fix --- contrib/libpq-cmake/CMakeLists.txt | 1 - src/Common/examples/CMakeLists.txt | 6 ++++-- src/IO/examples/CMakeLists.txt | 4 ++-- 3 files changed, 6 insertions(+), 5 deletions(-) diff --git a/contrib/libpq-cmake/CMakeLists.txt b/contrib/libpq-cmake/CMakeLists.txt index 6a0012c01bf..246e19593f6 100644 --- a/contrib/libpq-cmake/CMakeLists.txt +++ b/contrib/libpq-cmake/CMakeLists.txt @@ -54,7 +54,6 @@ set(SRCS "${LIBPQ_SOURCE_DIR}/port/pgstrcasecmp.c" "${LIBPQ_SOURCE_DIR}/port/thread.c" "${LIBPQ_SOURCE_DIR}/port/path.c" - "${LIBPQ_SOURCE_DIR}/port/explicit_bzero.c" ) add_library(_libpq ${SRCS}) diff --git a/src/Common/examples/CMakeLists.txt b/src/Common/examples/CMakeLists.txt index 410576c2b4a..c133e9f5617 100644 --- a/src/Common/examples/CMakeLists.txt +++ b/src/Common/examples/CMakeLists.txt @@ -31,8 +31,10 @@ target_link_libraries (arena_with_free_lists PRIVATE dbms) clickhouse_add_executable (lru_hash_map_perf lru_hash_map_perf.cpp) target_link_libraries (lru_hash_map_perf PRIVATE dbms) -clickhouse_add_executable (thread_creation_latency thread_creation_latency.cpp) -target_link_libraries (thread_creation_latency PRIVATE clickhouse_common_io) +if (OS_LINUX) + clickhouse_add_executable (thread_creation_latency thread_creation_latency.cpp) + target_link_libraries (thread_creation_latency PRIVATE clickhouse_common_io) +endif() clickhouse_add_executable (array_cache array_cache.cpp) target_link_libraries (array_cache PRIVATE clickhouse_common_io) diff --git a/src/IO/examples/CMakeLists.txt b/src/IO/examples/CMakeLists.txt index 12b85c483a1..fc9d9c7dcd1 100644 --- a/src/IO/examples/CMakeLists.txt +++ b/src/IO/examples/CMakeLists.txt @@ -59,10 +59,10 @@ clickhouse_add_executable (parse_date_time_best_effort parse_date_time_best_effo target_link_libraries (parse_date_time_best_effort PRIVATE clickhouse_common_io) clickhouse_add_executable (zlib_ng_bug zlib_ng_bug.cpp) -target_link_libraries (zlib_ng_bug PRIVATE ch_contrib::zlib) +target_link_libraries (zlib_ng_bug PRIVATE ch_contrib::zlib clickhouse_common_io) clickhouse_add_executable (dragonbox_test dragonbox_test.cpp) -target_link_libraries (dragonbox_test PRIVATE ch_contrib::dragonbox_to_chars) +target_link_libraries (dragonbox_test PRIVATE ch_contrib::dragonbox_to_chars clickhouse_common_io) clickhouse_add_executable (zstd_buffers zstd_buffers.cpp) target_link_libraries (zstd_buffers PRIVATE clickhouse_common_io) From d745a66c7e17196bc7dd70fced3b086059ad05d5 Mon Sep 17 00:00:00 2001 From: slvrtrn Date: Fri, 5 Jul 2024 18:49:04 +0200 Subject: [PATCH 242/417] Add a test for Avro negative block size arrays --- ...01_avro_negative_block_size_arrays.reference | 11 +++++++++++ .../03201_avro_negative_block_size_arrays.sh | 14 ++++++++++++++ .../data_avro/negative_block_size_arrays.avro | Bin 0 -> 37115 bytes 3 files changed, 25 insertions(+) create mode 100644 tests/queries/0_stateless/03201_avro_negative_block_size_arrays.reference create mode 100755 tests/queries/0_stateless/03201_avro_negative_block_size_arrays.sh create mode 100644 tests/queries/0_stateless/data_avro/negative_block_size_arrays.avro diff --git a/tests/queries/0_stateless/03201_avro_negative_block_size_arrays.reference b/tests/queries/0_stateless/03201_avro_negative_block_size_arrays.reference new file mode 100644 index 00000000000..912bff45da5 --- /dev/null +++ b/tests/queries/0_stateless/03201_avro_negative_block_size_arrays.reference @@ -0,0 +1,11 @@ +str_array Array(String) +1318 +5779 +1715 +6422 +5875 +1887 +3763 +4245 +4270 +758 diff --git a/tests/queries/0_stateless/03201_avro_negative_block_size_arrays.sh b/tests/queries/0_stateless/03201_avro_negative_block_size_arrays.sh new file mode 100755 index 00000000000..dcecd7b3bea --- /dev/null +++ b/tests/queries/0_stateless/03201_avro_negative_block_size_arrays.sh @@ -0,0 +1,14 @@ +#!/usr/bin/env bash +# Tags: no-parallel, no-fasttest + +set -e + +CUR_DIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd) +# shellcheck source=../shell_config.sh +. "$CUR_DIR"/../shell_config.sh + +DATA_DIR=$CUR_DIR/data_avro + +# See https://github.com/ClickHouse/ClickHouse/issues/60438 +$CLICKHOUSE_LOCAL -q "DESC file('$DATA_DIR/negative_block_size_arrays.avro')" +$CLICKHOUSE_LOCAL -q "SELECT arraySum(arrayMap(x -> length(x), str_array)) AS res FROM file('$DATA_DIR/negative_block_size_arrays.avro')" diff --git a/tests/queries/0_stateless/data_avro/negative_block_size_arrays.avro b/tests/queries/0_stateless/data_avro/negative_block_size_arrays.avro new file mode 100644 index 0000000000000000000000000000000000000000..ec785a885dc84d85043d7ae60e174ae2b8756e32 GIT binary patch literal 37115 zcmX`TTgWwGdlr`US6T_Bl@KM^Aj6MHDQR%_GoeUlzu&LfHpb4(n(dm+n*Bxt60C@X zkOPrYJg_1~LMcUTpg9l^Y$`!Sf`}9;5+nzbqavkv5c;fN6yEP!^Un5pXP&w5>%Ok* zURKsV{)6BC!5>NYdHmTWuZuqYt3UX&KOEA&_~Fm}FdzG$&ATRl*WX!2xrTkMj%sUN%MdT7LL*|INQY*RsxkCY|T>`uBVP*?)J` zzgyj`MGuboA=tGUsvrL1FaGIY{K+2;EFWJ*{s)Jx>wfJ2_$$BN|EIqg|Lza|mE8P` zKm5P{#!vpkzxN0J%U}M(|NJ38*BAex77-0;$xHZV=$SAXYxPARy-vtAOO85?yfG11 z-~Mn8iNx1#zCPph+3(NB%(CD4YyaK#&W+0~SIfFzPb&U=S0Q!MZ+le2MagILX25(@ z|I3GAZv7aGuh&hA&iT}sA$)tz4RZu!-BBhCCI|CLB$X)3WgQBDe5KkvgvL$oy@zwk0*rr{!%YE3$`-y_)lonF|hXD)n4XIU*E_{BVhD2_G~ z$A9gSufhG93j#gtBuT;UI1x(6d4MI=o%?h-`d042Z-k+tCw;4aH4r#`U1x1zBGrX* zUxKaNRh+Ic2$pH$Ps*i42&}x`JNVy`rYlWa$b(6?8QHpS5mD0+UM*-Yb*%%UU$9m# zH%5Y;N4*}8#yK@43=yMlpsZq2I^Br&cOP7Y>tiG9aO3NVbYI~9^GudVr@&sU?p&o{ z(tMqF>vzr;8EyDFD&{vIBJhS^sQ=1^y#9z$ny9LD**Hb+MW4Va(&r`Z#1&+;B&STD z>Z%XvC77O}|Fk52r1d@oIeDQ88>GeoZH%L0uJuNIDImR4Q2MJ9Qn~19@Sw*UtJgYX z5c|(v>6Y{F80xNaCDqL($jA~tjbm$0J;KC_T7f$Ci<9-KM6Ng(vm`djt=tr6o6KK* z*dv}50#Q>?8zIkwY*1G7K9JsL_WT@V*7l8n#%C;Rnuf+V9d#`*#PEuCeax12_AaKQ zYl@CC`p5pa$KUyRwd9#lAiLU(LW;QkEu!i$C0}Iveopv(vNqyIUB787#bGTj?x0@^aqOJb8|5zb* zOwJf1EsBxn9DiycV9@Y<~?Ego>M ze&ao(J=!e#D7xs@Bby(Lb1QFAqtmvB>c94|p6C`nX~%j`XH|A~6R^C7WOKSHM2PQ0 zOz+`)eKSV`b0jG$fkS-iY#G7UeG-xIkFH$V}H-l^$lY3qM2cO{oZ57D!E{(!%Il>{4{BdGu`FB83qpD zSdK5Ka0#8Q;?>#_FOg{WYOV%t)<1Ymm%bs6tyyQ?vJzQZ0Ix z7I)W{qszp*XX@B~5{lb5fAjI&v}il((`4LznT}I?ZxoKKJ0-=`+rTBt`v!0Q5~noS zh|bIL4$|J)z)=!M7wwzUhmq{SE??*~2-C*ter#V-ZA0fFFztm^8;V#6e?Z)FkL^Ia zL#|+pFrRwRL`JrHjF|?@=rP~~VfZ2m)2sF6EfB?}if%TzaFw4j`Zo9G|zTH<_0T8ITnPmoip^ICH#{Omiir zw&LWiEYmdB(|6qd>j!^ri2+Wa>>RFt?%Sd1ZN+YNwe-$S5d28ec{I&t0Q(raBw^+X zmYR-)?%?hyj+&7wvcVko2qP}}7;gJfa7HUyptGP6$0S%=4l(QG0VPMn7QolZC8$i#_3=Y z?P=bZm#J9-=G;`|uLnH8L2Pm1LzGb2MIt&ZoAU<8=odb%%OGne^b|+ZU9&JUD|)t> zH*t8vKY6tK4Pc$=hwNDBbBuAG5ijD|S_C|u5xLZ%Fnb@@w0FHdC@brE!cX;$W600G zoaIdv;40ReNG|IcSff>aFi9gDoZrbJu1%8Oh{6NAyoDbpB~XC;0RsOw4+UynJTi5I zPm^vpmTB=szq@xX%)j^hzV<7HY}ZZ&85goGBUVh-iLD`DMqzXTb zUV*w1`{HP#@O@yxCxv0~sVi1gxf|`ixSsB=Zog~`v--dIxN9iT6ZlmgDjipO$2mEO zPYPPWAt`jw{;ZzE8lo%yLI-v61`YX{v^dp%$V!D|)sM$! z0%Jf-HrR)D5$Y}U;Q}DIBu}FnR*AaYnE&~qT0Fy*7De=3!9s_Lg@LE@9Ub)`a(ro6Y}{|Q0yA!cbA1|htPrseB zmvF1u@tM(sA_sSA5oh4AkR#Zq`UR7^aK*fG=F#F-tsq+n)BE7%MiUiwFemH!0H5(2 zqUraoG47$yj4q~W2PyH9EuDAbs&z({%GZbR*|+0HC#=vD<<}q9Tj8;YqKlDME35%) zSDbz$jWcV-m^7TRRKoBYQ~L6^^=#d@DcY%vY)`4id%5WLwiH#fBm5`+sZn^vpv-K! z0ZEzrrvOT5AT}6WI1PS=KS^lGzj0AerTA5TUb4Wizx0FMyiT-abDV9jiRw>B(fC-c zvRCk-lYsuF>)V}60uo-8-ZmY=UN$JY(=kV6xHYY1vP4z=zuXjg}1us3X4!P`|e!jgDyr~en$5-M1jXb|Kw_jTXwy{Ub zNxSOQu_jg~x|a;!^i3hHPgU+H4Ip?aWd6U$yRjK#_teS#&KVBrnPG`z?HvyH70Y!N z<{xLZ2^wsyl~_foE8JJL%uJI-K7zm8FC;d+qXBW5dev^^yw5y$CR9^XNU*Z^%(PBZ zrN=Bf8>J`v&{*2%secMIAb%!?MV?(#=x%d759h|n!DYr@aJh+14}`k>#hq!0B$+VLgdMgw-PNbQ)bwvZTCwt4>1tdYm18+1@_pvisg&~jI9Q84zTsn+ zJZf?TKC{)@ZJ@O@yApF9hBgt6vzXZuLsAsLjtf0FrY3jIA}UzT`MY+1D`NR-+NTQ^ z{Gyh-V9)x-FzIM8rbrsXPNHP6EifxH{UQ_fcOH{WJ{!3r0xLzHb-?9I+EL=pdw50$ zX*q~A8#=7)*0l(=rkfDPxKr^~>&*MzJoT}Nu0<8bsm=!i3qCNIZRh?nRr2jU^Fo}+ zvBb2g_ndMWGuVy)VG-bGQgmdy8dBvBwj=DU7X8Y9BYO=(1Gi}S>k zHYyb2GdV?VqRYEtr&5&HJqFj9B6*RzH2-Bp!S%z^;7oXve%Czu|T*iU(-(xu|(D&#wMHy|1A%}X(zXW}n> z*YF2px&?ccaSsZETX12K&XfbcLa%^8h)tPhe6mFlSe1qX9>?2-g(^Kw-|qbzeY-b$ zB)5K&QWL`+TCJ>hIz4p7>3;%Maf)# zL4vpY(lo=`x;$T=hD!Vk$UsR8$Rb?rJkpK;Vy1X#u9x@{uxg-s3GaMuQKopa zW$34lXLaCgj!Z)6|L%h@yhP-0y>p_bJ@455zC;ATkIxpq)QvTt9hDNWMBLxMW`AA% zTV1*1(u31`X=MsXf|b=SUdDoT{;*vxU#*Pj`i$orUz8VTR{@Wh+3P#slxD)#gGbgG zcQtmIc@?%e_L+|Sfmukgy@PMo46;0D!}#Tg-G34J6Q7!8Ca|e) z1{)7iHEjHSnpL)CxINi8h5XJx?j?GQXbj5DE7cbFh8-U18*HHesPV@3GZ&Y5p<|QP zX6WPuD6E$+G=>0>7p+UopR-z%4iBTzk5k2lhyUs!n#{iX;WDyE;@J*qqC>~TZXc$| zk=W*|aFUkJSn|bVdoMLm8Lb7X07?&^GgF|S<&srX92t25BiEmj*=0mavu zd~!;rS^<~NGj6GXH`Q(!f9(Ol!L>wM_U_(1ETZ{%@tH-`>oFE*l?0Oy6L!8cto_>1 zGwQTs8YG!cDKdkhb0nq>{?=VZS3#Rj+R4Iry#@Woqo3Ef>aQEtF^rN4uHl3DI)SWq?>;&?axzRA%a0K-0vCI0Fw-K9qsUX3@`_EtW7OqIN;-8fx1mBx zH4CjV&G`(gXG=;3T6CvXnH<=sub~~*Q#zOpGj$U=kI!#EK7u1=gL7fx?k8A&cEg=dA#}!_r<$#>DRmuwt9#pznGe-c44S(`o=x;x5w7ozqA*vR#2z zmy9I8nUEr~)PaDcFX83k#jRBKg!)*^Drn@CIV6Haw%b=D;NEu<8+DPG_ zV+j7N#e*=+FsDLKJCL!E(2t_Y5#0W_tHeOHNQ9Hba4mt#gnu42E-1;8^xuawod#XL zDouKYuE>AdUw@=<8CtAkH8@Tzl%^YS*P`faBKgaEi-mI&uhbY{T=L%9BPK&+*65z4 zP=*(YqqW zq7Wy8pFV8%C~j#kZ|K6FxQae%b?+@j(JbJo@}0)1&Zmaku^nvZ{67Vv3iY;7cHT{Y zB27F$^#N6rDOTQ)_3YrdwYaRru(iCR+u+*QK`v-Ai=`UniRyd%Q=d6GTMsdVAbS=Y zYMq{d3-q&q-Q(BJ9nWLoC#hQm=kGtzD+8l=JC;yrHTAEa!a5-+M(l~yhfONf^f zV+56+YMVYH)FZVnA6sG+0Co;|op3&VUtE)fjCfQeudg7(p!q!h82q>Yv;X|SidJDL zZPH&v@aGPw-R;KKUL`4UZSf;^Qk5Duhq(el?v@AMuW?Vo==!OE(`0`=vSx2~0d z>?Ql#mErxfEeea6r>-sV0OzWJw14$+WdhzZjG+Y-BkH@}WVr)`3=R(&+h#w{hN)aL zajE39of29~V; zZpBE1q;{&JC&!CzdLs0P`j(k4K3s(V9-0V>X-|2%%@cYpH3};I7xW5a$1^?3I%4ib zP=YU^Z{q9Nbo^}06N`ZcHWI{m0I-(3eBnP&M;^;|j`hKlbtLW5Z@}Kj?_IHO))Z6L zphyhxSjO)^d?wC9hA7BwD>N?w6TYtBti;=(N*F+Hx9g(?1(c3$STI>b*L!{9M(FlJ zOdzD7^oG3~4918YMjh)5l40cD#I1yb0p$0mUTfYd>3b{@5vUJwUnES9(Z&>j5$r-k(l znEZuUKcBqpVGUZZeW5Wi63Q}YK&W-_WmBAcxfy-vz{Aa6Ti>NP5E(FPK=0zY9_qT4 zWQ4s{l*?8e_s6gr6u2X5O;ZJwmKcK5Qb$%5?In%^N1TNT-$pVp7Z24&7X{P1j(_Rl z?3pWfS=yMT2e{d&`MB9EHy;~woG<9AE0ypZx`akIsZF{xB+LAGZ|NH(kx{|Y-+7dG zK!1cc=`GL;Cn+Ps>Q9tIobVJEj2z`1%?Q=#0w79NP{PB2dmkZ(7BoRkJ?JM7>4Fkxm8#e7zdkR67F1DG<6KGKid_;Zi-Hg2Et@3fKdRIhtIy16XPf6t z3&`MK@M3y8qDYjZ3&L@I1onO#wFUA+LzRIbsMkRm2Kw0|#JId){lS;J1V1r}iibWX z3Lq12d&<+s5^qSX^5@AbmK|_G@WtBww={MZjUR6CPS{!pO8oszyeWckBpW#iZfhY# zY%+Vu;=uBYs$FCa)U=6DS8d)Q|n2kH1m;(OAR6271B#qPI>7*F^}D z0$-)Qi+;j#N%4eCFIEFU+td#-~LA)6Q_vbUfqpEGK4pZ(*j;zrqRnepCW)t)fJ==@h6 zc4c6GhhzZ+qnTP3Q6J+CeL@8g(J8*kfyS_?l~&QZ3#7GOiFCi(fR>02ya&?GEztbL ztI!ghKUuNR?#_;$N5A}#34PkyXKN<9(x!U2bq$bby4QeMnf^vj9~gt4x1TWc)x&dD zx|o%Zg-LHZBotm9Dc|V#{&QpmJ}|g>*ZGNxfH33J)wRme*wC#*gp8drZQ!izTVp6oi1!Z$*k@w+9Z37`bvwol-u{}x>r#ZLpYo@Z+Rt}8g z1cczn=HhT}|2H30>w^^GG!1tb`{*rIpm1<_LW8;m)yC6}4b4vhg%;W{sfsi2E6QgO z*Fn8{Pmc5a7$q@TJ=B@&@p?rg1podc(_0K6Aa1-yOVd~aQfNIe(OG)=2f^yJpzPhe zhI5!KgLAE(Klyk`-);t|bH^-g=qMl=T=4DEd_~$;T2F%FkkMs#?^42%(L%bT25ezb z9~mKTms6HoY1`mx0A+Fi&toels)m=amv0{3(?qxVXOuOTH&(sln&f2P1iwV-l%{ei z?LnJgemRtbgzM}8B&uhSDfh>)d&UdEUz%A%lmH>+)zw}ZBG2|syExp0DX2}`9fAP0 z+{XTW_tH;LZ}x?|YfAiuAFO>M`Zr#PfH%WeH7MA(q-zxdxfIYK@kX)ui$t~5w}iH7 zG&Y{Kc!p{q(Nq{xlX?1b4@9LmxHfFJahK=9i5bi%Ff_sni-XU(q{#X% z`gGf>eao7l&=-kbWoM{tyRmF#&Pd@^aAJ31D1v6tmalhp7(Dx7@gc~pwafOT-n|zN z+grMDzxR;lnrr2_$tQLCksrb0r%8($pu@)d=UW5uJ%yQPh4MsqnX~gc+UNT1DJJIpmJ2lUwPmqz+ALQ zOcZ?t07C(rjum2@#kWtuk$)g=u=P@Q0aBe|pZ`iX3O^u`!(K;*yqhESYbh?cKamxL zl~zJsO`SmPxzmT{??3X6#_Ft$xgN7lPA#6ba1MrC@9Fc)bFN;!5{~6s;|F#5yQ~xT z%kDCJo+6(*=3NUhD=5MRc%6rC6$qSo$_fN-oC$V&_HO>UOqlh)2DG;dld-zP5;RhU z+kQc(_}@PesqKLo2FVaW;!n7Qze>l#jW24{ak)_Q*0TA4y?_e=tg6v>C?esTVz$V4 zG(i%eq*Ace;{oq6|I!cmhIq9cX~8Q2{Do?9#!GE~lXiAB%!l(%%#514oQ|yvIS7*x zo5C(>(Fh1c<|y{~Bt!71?S%uej;DGjCPXWRM;aV2nr$haN=wrOuNvOA_QqKMvML92ecaZjV)g61P+4nx(=K6G*HA|P*w zX+Bb7$dEYq$YfODC3=c7)}LDSQhS+7{K_eN_}NP)Z1cV;?c0Y~?wzDzhw}8-n+59C zB4_Ds`@(Kxe5HCibhCGfsCxLRUe*>1bRr4wj30Hbh2Aei51*%7+knph61W!nrS0w3 z%e`+>GYMs8Z|GA+?m`*+UwXV>nuxF8x5S1RYN8ewnFxxpn=UZvsyFv|X<}&sI16MY zi5_)!BHo=JurjHiBwKxW=k?b?B1^mjOuWl%`KuL}+|Y+%$p^a8aCOWR+;!!gH&cp! zJ=g17r4t+NYvV?Gxvt$Qt-LSN+z zlu~>nGw26cA?CWRS?SXt^5v|;Z#_H_&OW^500ZYDsRS+h@tfhzt-8~ySW25-EWBkU z(vjj%()fOsUIM@s)5HOztpB8f8`K|ZAz``tK9=JBg|^ZQXnx~dbbK;=N7pb4}!&Y;eP=FCfyiGsYljk5Wo@*-~g8Ds}8K6wkiyk-Ojbo-RFcGXy6D5Uxo?>~tj`s@X3hg+g*iOi2xmABIr9-%l|RPdokT0JQt)JM zjHzg=TAl*|Y=r&yA=yBfe2*Vt{Yi22TA?DR99<3Ggd}&Ia(`7}yQ@V@W)L$%L)`vk z=q1Q%RP{42rZtg)*mSsaE3yZ zOSd`wP083<3m?6P)Dn1QLM?8!Aik5S!1H`w38)BEN%@h-sNWnefHOmVYpR7-V?sts z+7OkWk*->T5SKvG*R1xXIeatiV};puN%V%0hD~`8c??bgx(X1zD20Sy$>FU(Q&1)FH|pWA%M1l1gEQ2yHAL=W@DhKxa0hg&SnfD@#;pv@qvy zdvFd)0-Q=YaBZ@dZ~r^$;(b?mD(GCvut8>80b>m|`n1LJw;msQ0#-}%HhjN17QO5J z!VohCwiJ2e&f$(TDcU#M|FW#nkWD_Qahhijz`d~)(w3$qX9eg|VSt5!g|}aqO-0TS zbFPI87-S#R-5vZ(KZpnF41rfyXB=$nDGbu39Davje;C#1lj;$X{gF;cvDT`*a>i}< zlJM#M!p=y36ELkIY( zR~d-)(ita9c{rpyj*^x!V`oI*ZUOmByxv{}8U(=5J)E|0+uiUTYiEI)vJ6n$4&o=j zfG1N@MTiJl>V8R+Q1b!y7U8KjejzfZvS&n~#rMXgAe* zgmR2WY^)~mtzl-TYQf#&-c_?$$j;00X*n>%JZAVtPxP9sIY3dIciNQWk_aA=Zfj{# z?u5h!s^WnH3-%eJc9}rc{!btA#1t;*B-l0;=quF#&Oy+E1S^M>L3w&BVK8H(b_PYB zmm@Y6zwF~hk;M;ZhEp=me=yfsO1XLhflgg}D@Qwn}=jr{?n;pBX1! z@gZXaW`pGu^m|bJt)xzf)qT4(#z}l9V|U&mluQ3H((SI4{l+6i6g?US00^r9P*o>a z81+;dqRfAz2N8U5JA9VQwK5D>>>*;-&vQMc}9m}yOCX6pU zrC#m+cOIY7#tg&fB2XC!1keU#Z@|?E%wY*c8<`GoHVqW7jj^*GsH|c9x3ifWZY!Yx zw@Ie;z_0Jf1sO3&1`@SseiBp{{!8GByE^w1GCns@T6W0+ew&_RKep9rhT|m%--iT%2uGD6GO%7j zv4`<@mgQ0KhfG`8rsAf5;tfcyVU`+t@hyX*oDU>` zJYbz^=6&Be;CdVDCLtE%zfe1%mX^R3s%*v@khn&;F!e#@}e4^R<@G2>gd&M8}{izimO!uE>uC!&07RY)|wFD?O zK{MeHw^z6jY8u4^>{QKVX)010eEsEX;g0Z%(v6ioeQzN8W2GVg!tsHczES``j8(0>9|<9nP! zku@Z3zH_03jgYJN^=OEYwhQh%3|Ir*L*DjFKaib+AfCo64EMWDCfYPbPS-#$5z^Yc|_&SiBR_BJKW262m^K4%mC@xB6SkCGwtahsB|1 zI8*M4MG1dI#o{Slg95(cG=`y7p5amRC$&;eIkYj!3vi_t;8opdeg zZGS;zq(YSeUN8fy8)U+V5~udCzYG1Ts8$`(m`>ad^-9%d=3k53&h<=ATA|pdA-OdB zV}JJ%7dElkjBv&iDedeKK*%DcjNPwZfv_U@<3NKIJEAu3_+4C*T@fq#N+Ksa&lm>HA+Dn^{!)b899-0kC#?Dqaf8RU-6#m*I)2smnc@5{3)e)FM6N>`{6mSj<> zcUZA*s=Rga=!qB5z(CiWxA&hK$mYVq?YMfj)R$09T=GOhW#TK%!8vZUZ(-_>7?{bn z+c}XDsJ-@=rSRzikAL~m`#oS<14m~3guZE}!^Y^dsbgFCEN03ZsT*(KS_uNqv++e- z@wJN~PC95zH*Ymr;q{VjqD!N2x2H#(sWKI_G%EF1flv&gDccK`%FpD)%O*hRjV z34@rJyO!$t>PD9F^NINmBlLWI_SWo1_4_nz4XKDVdH=Kxn28rYm87}Pa`-zB$T~hn zbzZWBiE6zqXhnEiH%V`iA`V^6S9#UHseuteD_Iml6ty`g>}^)7z%u9+M6<|BCG8sR zH72h@r-8p3*cZXXN5362<>z~F?=NIE+rTD-?|=D0lU~x8@9Mpd@%=3;=dXDMv#zi9 z1Sn65qH?*X9D4&Gav_}!^yiF=%w8`kK#4UrlOBnF_pwu;%hMjXsDLtUY@C-usXx5AaT|eKi4~|Q z6|?5=gyxZiBA$|cA#23+L;)#J6l4F|13EmDc)lUWY>(mmNJ5r>%Ya#kNBcxAz?@&7 zM%)$E`P7bEkb;#a?-a;!gHY=LM9R>>;VfyKEN+hTAJ4W^vqqMnCh3E=7Q21^!2_38 z%=bR$7pI-##)1Mid}ZMa?xq~dx2AxA1^(p`628wKt%0cyBF_}aAz7B}0$d!GGA9Ce2ow3f;Ghm#gM6M= zWoqq2D(USPA|*T&j1WQF;U5wPI!mhC8VCwfxEsQ$5m#&{&lzCZU65@20%;Sk;VGK()$C%+&nfTpS9*AWP)o^lnQ zE1K^qH$(=54+>!*8N)7G?|)GI0p9(w|MBsAfBJ7cEZ~4SJp;^aGwn(l`Jn7;U~@Di z8LW2?uk8T&{8{h;xNm6Jw#?dC)ilq7D7D;J=iirUCX8n1etQBed)2@6P=QTRUdlT1 z+!n|XT{L2*$Din(tJ)Zi3%QSz!}baa&C-MQ4%;vPJHTKFIoIo4-%Q%qv{cV z(mx+F3&g$FeW+rr`tmv8HO*o@HC?Ip^v&Q!O__2wMZ-e67#?zK7&sP!7Uzf$(V{@R zJA6>%G3;IC1j^%cvLxodxp>k%m45=-7jJKrqIK6eX?L-kQOr~-<#AJmYH)yu+?LE*XVJH zO`P(6nJyn{TD>odQdAzADohG({O41Q!9RJ}?p2L8_jy;kX%;mR12Rmk*s%JZr`ze1 z)>o$k&1Xf(FZ=UL8x%=q`o%KiUYazjXt4|rY{}HSRyK#p=otn3sD2N+U_kFw_6Bl) zIso?Rk0&q6`(kp^8J)S5M}zo_5BnQgK)uZe;uVorV)wxA#OOSk|g^h zmL{J7g7GB)5^c7!R1 z#};9pxnGnraBo27zV}83V(&>BKpP_8(32zKp&-Bh&GC9}DryO9Qt8P*c-&8?b|_#6 zxt&ZQpKGP;@ZR`E!>6j@mLSt7yHSK7B~L#hw1~;^jWqAcW>qTTPH)xVlOTOQ%@1ha zvsRrzd6vfS*9{yp2pSB=CjHi9NY(EnY=PV3OAYdgt7g`2V(%(qa$6BC;o`{v-J3mn z_d95n(;-(sw0Hg7=+6Nt3rTU3fn^via@*Ha_rATsbVVxx0D?}-s8J9kzxv=jnybyD zFGpI(<9Aa#y!xef4sxx4brHqqh3(#;ercHithRJ#*r?V8d>VDnzLz%*jAYKz>Ts}@ zK2bz4_ksAg>a(H^zUTrR(!Af;n=1~+E&~%-NeF=OSO&yJrSFHrVhQ(jthTFOi}A(r zp;@fLv4U?puoK9tp+*2kk-_FiT8Z1TP7RI<$?7RIMu*swdw@e65|D(`0i)X;OI;*& z4fqm>kX{;<5Hs`(JcQRw$x9KC%O0h)4;<~Z58N<5>jO9TAL)b(ICfR1ke2`V@pYC8 zFAEK3@TCU=0{Q_?-(CYz# zCShv3gY>VbSp7Vx=Q=bO+XVjh&Txg$*1 zRfOg|^(6NMkxihDO!VOH&o9l<&>;iHF+5)v{WGHcEQjWGrue9hK*)VwMS}kSCB0e5 zG!38D_w`2@Le|| zH20LEpI6+bZ-lYh!$UB9?;u@F`29;!Setr~b0dM35F?sT7a(_5O+grI_b^8VOUz4D zxONE=DLevMrr%F9k!&BFbOSqvcM&m3tev#}JL+$~{7BfFJvssu1f5|vEo}79mUL*f zW~?Rhbx$YG5ZU1$U)sF8hH+vO8`cYZO2X!j%CAZ&gfH^&A}tXue^NuFo%@B$oXvxA0gJL;eRn4Q|J; z?E~=8<}tEq55h33SQl$PUesT|5Y2(X-Ra&89`uGU4j!2q-N*je(q>sX@eUl1r&g&; zJLJfBydh%9D5EA4h*iu*a;rLF5L1J*$y|~$FLt*yt8FfpefTa`byx}(7Te%x<<>h!(llaT& ziFuh^L*UZKwSjp!XM&T_u#M(A>F7dolD)9>Rx4Tskc4_}e%yiQW-kMMa~>L3KW z(ZoyPxQOts*L8LJ=2w5pv&QV?#eVftYn;0Jtw%2y^)9dTjd4JR@F?0Q&drcd*uy^; zaNpLw;mQZPnEo-7vQ$i?dTYuAquW_@IpS{ECP*8`NaLofnR#O14?> zSLC#OjbcKj{6_L#yMyIaL(1JSj_LRr1enN61zX7he_dc#_$luMArfH722DLq(Yya4 ze&r(9Nacb$t3L0@9}T?L+v|LaU|~>Udq0iySHI7T(#VHkV;jEc1O4@3js{m&5MkYZ z<)kv2v0MB=i{a>t(71)Rn1k2m;DpQ;uK{a zHITySoy!3AMk(J@qt?sg4c^d$BA}7y`a8U)Eh_@U9Jc12cHp>c+TXm87THH{$=$o< zx`Fsl+WnT)uo4dFg%Xfl?o;rk+b)jDyhTK5?ZbZMJi_-WrQlg}CC2HmTfyR4EQr}B zAcq>q$!K{}p&&mzGwXls#oS-+`@`5?*IhN2I)8+el#)pkX6znm&+sTmH9_WDisJL0 z-oK*t|92`CxMX5%Cj5dptbgxH$B0+HYND)r8YR zpSs^Dbq}a|wf-eVrS}DVjq8Y`$?fOo3nwI*E(L#rf`Xs@)ZysO!XMH`0}Vt*a{{w> zK(~B8ho~7aMY*yAPBozd#X>o$w95)ac8W+I!42J@?8~Wjrr_@=m^U(a%3^gk#xF?U zEuVlz+2J|JyN8m^2rylmyD9oS94Xo3Y5X>y>#iYt`a2+6yJm3vcZPd{_!aSpUk>eS zUZy(g^cg2c^FLuPyt(f)1Jxh{COP2wbl56ey;PnrD%1RYJ}=h#P|Cs+s35$*#rK;T zRVkdyg(t+OBGBu%l(`Vv?t@oF=($x?!Dn_X}w2;pG89eVNA4?e| zYd$cxsMz5#PT!hB?6L6GJNW*0D?a-n7U@L4H7qla!|K`0X@-)~jSohyj=l>8mR~ zmW6zS&p)^%9p5saF{0P24B|~Pk26MBK~CqLw~M+v-&DH~r>Ja$9cEbhLxB(W9OZ?f z30mPAv@z6Y#l@t)SE`ITUg6wTxPJ>Z`0r9LiUCxyw9s4(Eu-*mnI+`MGw{#}mM=%O z;^e>yxRh@e)FSlVef>|Dx?b$1DAGcldi5w$+)cxPv>AtT?(1ZT+s(O>_+x9@&%=g_ zdlbwOeE(u*dd8P|`8?q3Ow>B|`1%DA+x?NQW#Zj>z~ zszAn^JYITW$rF}Ri6iad(Vb@@?MYW}7;Yj+=sK;g&VEU!?g9=}D`b^313@rXQ9=d= ze%2N#ciUSAFm@$CCHVw~|7af_hFk|~Iwxus;J1kLVZrLtfa{zT5ta{lW%@rZOa3rA ziV-9*l9^TNy|*MB+r?#wegQ5So0dN&TYg&oe zCYjz|#`Hy= zY1`wTW&ntFgg7pQf#J(Hd38?}1v|E^DMOb>43OAYi)t}x!2CfkUZ|)+~NpC0< z-Zu6`h!?!oEwGR7&ZfHC3Kg{+owt+J;*_iJ>Y7e@Z|WRt)*K_i&PiGC5**KzkB%eZ zo?VxwK1iC7e$)qPMH0@n55vPnPyLjpb|wyWx?#0{^3?E2WPo2L@z zpv|1+MG=XzS}aK*fWjgm0Pac zMPL3_W&7ZL3FAoEM0u|3z%VFqi$<+hk#c>s=xccpH;mqe37|;MKe-HGNPCG9IU7o1*$=RG41gy9 z7Jx7lyvHu(5PTnZ1fvEvi^FVMK$XN<&%v(>{BRpc9P$i2)Am=YCv`L&K>EaWraa^kI--Ah z5k77TE^qO+?BJ6!^rg|X76}ODDb5zFa6`Wn!10M#+L6_JqkBg|-N`5U9Bu~FDw+ao zlt;kTr@|Kdu+Nqw-YISTP%(Z8iYx8KsGD!`6Z+?W5)-d3gS-Ltv@+&)jv80l2Xi2! z{rpk|2#8hn?ENgW9p4Py`nH8>3$h^GRxr*g34PXHK*BUaAW_8Xy57$74z3cnFLe*Q zfR4vISlS#`(tqOuKtJ!Z?goEe^Xu0+t?j!>>z%?Z_3>RKt zMkTx@k3*|}dtb|c=@~!!RlSCw1$%znvH~p9`}4d~kpOT(bQgZ*LgEYIC0`Sl&X*$s zXVRfsISXvL`Nn%w?7n%P`6>V~Rxerc$dsPQ7yiYZ%+0g#zjFz$L0(C4`l*P5RMJlB z5{GoXibJh5x`?WNj@ks(Ea>2TRed=reSMVZ?R-@5w-Ly^B@UWaOZQb1?X*+dPt~)2 z>!P8R+;rQvKDM(@>ut%q!43 zT&)tUqMm8Mj_xa6hdNR|%{L?S>vn`QJ`dYQct{VdT=zWAFda6DFMfMkYdjAo>==G% z;Zt~XZgkv}*g6cT@%xt@2U!M~Bc*tuc_xO&auzF>oHFFpl(=WXk2PpoZ-^o3%MS#p ztV9gl5*if-{K8!XWhHC!VggJ$)KJy8{XHV^jujDx>`dE7%T zeV2Ah&qJncJKxY)t2hidt8%9Q)EJD`_raDAL{3qRZeG8@HNvm_l=!}_R%a{$5P-o} zpEu%?5-HtW%56MQUPndc*%r7Q)iC>KNT)sS`N-{Rb(dFYmq`Xg;ER#nWJEuh9c+wg z#O5p@CWp$S*PSk5VRLCLh2LeT@9Gxb!xuEShM-n^h5U)6i1mj{w)r7&zj`S!wf^p1 zrN$ViGVu~+uQIs!8Vq`Caf2GWEpWgy>rFyeCyl^0?MtoRGv=lUBL zew-gNSwvkSiltjT!@CKR5Ptivu+dgwSEw0%xv;5%J^3Q4=i9d8PbC1qPsH^rLT>mq$@l5}hsJE_dhs?%bzbZ;*$w~k`oD2tLTni zKiExTv2XkxUq#1Xo#ZRzXv=^i)6-wC#U{B--7&M?@_Xom!&|{>I+TWGji`kOs7m!B z|1e4j%kiHOc!!GqIVb0Y?)NU9jIwvIerrKK;r6(>qtVraewL!Tz>YX==V!;a1@`$K zJm*i40OGkmYx%_vuazO$miL)<@qYIO8r3KGIQo}L4BQm^zO!ut+d&^0Rz34F?mX=D z#^4a`U%9w|$EE%E`9>k%C(o_$iNT-2J4Ogo7V#@)_h8e^F!7@L5V=7Vrx9K!~&$vzo{%Gr_w*7Mv#a zN3W1S4g`c5Aik5Z&Og88T_uL$-BMw(EHvUiX)vbu2GUvLBITHL+_7E(ZX2FRE^2h= zodrys4z%nRL+Tb-Z;V0rz(o6Wyx9WS2~XeA+;NRa$~nPK*w3~rpYxgE;PSGZ?5NL6 z+0QSf-q=~_3vmJJpcRmVA>IT>hUXjS`dr_g13ZiCUE-qKlPfeJ4V%e-Uyqr zU#zQ9>57OsLNOhBEmF*t{@{|bu`9{o14+3@c%5Y!K)H!^WF2-EDHtPRw$x_Q&9@cL zc!I#}(;7rkF6NSx%uJ8o4OYK>nZYqea6s+Qv^0)6gzbQ5$imijdS3|*0m)bXHHidH zR50}%0suL@Liv+yscW<$G{9>bY1aMUFE1kGS2s2-Ke^u00Vk^gaE{8dT8N3HqpI}| z%{OngI|!eYy?1?|qiXP2npuhT`Rc9nw3|OHqxjvT1rwON0+W4p;!7N}CO5fCOv;aV z@19|0VJ2EEe0;M3qPUqLn?oM)lZ}YC{%e=>I^o;}0Ry3wdwqmWnSYoU$oSyjJE6z> zwl=E+oaQgBV;EHySHH>5^IZ8VaP|@I5F+p@C!YiVJ&#Fv^yU-aLh@(R^HGNd&oRvF z_X^mGmSh<2J-qNY4jOU_9)qImr)n`}1p~gc1ahgxJ-zG(cNPGHM)7fAi46AKL^Jzg zpuM>qJ6*YfDf;?2&!fd?a9UNr^?=!IA07@%zq-(?i=Ok~~FW^WVp%p8>Y zLtONawFiU^4!}J(E5==cX5BNtc1cs)mFyo{y$bakj>qMja!44i2fLV&5kHqP{-|weq5u$1OK>=)Fn7aciWn(B>HoaKN+Lr!*I3)GI*jSr4G8n z@N00RfL_V^pgy@=!aUz!CM_3~51bTZ43xMl(t#?har?T5lnmaA5Akb@Y%{0|WwI2k z>r4L%8Z_C{ucyrhbm2_1WehzvL5Z1`L!|j*=EXD*vG`fsIZ%;yLc~NOCQCp0OPl%P z@1Z@t;#XBGxu*nE%vPEBl%XPj5rL4esEA~p8Yw1%-_f646!D~D`(0}dhKw}&-qlwo z2CTwFq#j&nNmv5a^Yu($AyEWfn5-wTUH?F+!+lMzTUZt)5P@c#K?9t7W;$>nW!yyj z_Zk8DHAt4LMkW%Kd>VTAV^`$RvzJFo`oFypLp8DvmDwLGv1k<8r)v6UHnjqySORfM z&<}w<-x zZj0Fm@^JpLK}W~-GTBS)RwC1Zegf@~5DQZ0E4MmzN%s85OlMK@b9QETMp--Hi@BdWVhv^qs=Bi}i($O@G|F#^ zg7I2jFn$|sckUNHd}B>@mX=X>G-WEfLRUfRhv`p;0foj}@*u*1?=#Q}bb#Z?`c~<% zYC3Ge88@FW0}EIXBnBr@OFci@|8>EuIPM}>g^A&CP4~M2K8OxU?r%8$M_Ni}W8!IG zsMXn|ptQT0#LcEn@ueFDc*B@%+X>Q-AQvRQk!ap~+gS{s)uNEGxA)rvb2bcNSO(Zn zxZ*p7{g=Yo{ld#w9wX#qz}buU!va-6cCTP&A$IXbZ1Z-DJ~Q}12^$IiEofl1H_WK! z7SXMwN$=&$y6V6>j%TepE9J??V5P9^GtL!Q8p_T=bF4DzH`6|ZSBa5FT0o?QtDrXH=@Xc| zzRZDD@1ddUlInhb$>hi9$!gm70S-%|#>;cMhi^d?=paDdh0k{E&pY;X6-oY76*1TE zkk$Yxs2opA$!zdHWIahm0Q(K`|NMAC&;NLPQvcEsx*@(0zHpVstLTzK+n?G_1Y{Io zWrQXT0A_!1fpO@E#@BQGYbEl~kLBKzemR~O#aAdio+?2~=OrEQGo_SPrVp?^036q& zHEC{}18`E!GD&3gu)$Y}2?!St@kHbHo}8<&=F%N42d!Rnw*B9@{Mfi$$9wMd*e`G3 zHIkkM;2OWF>lBSX#Cyb-AcocwQhnD)W+Ga&`8Xo9q88zPa%x7JA&^m8qyNR_UB=%4 z>5ePYOa01(Uzl@?JPQ|XS3Sl)iDKFQ0&o$}Wes|TzV^UHBt0n!9GDplu@BaQt9JFx z7ymsYjC4yt5H=d?Py%S1K3W=L~-+fj!Q4CoUK-oWI1rSt)!UUfzG@&v2&i-rltOQe;E;Qszr)=N zJQT(ncc0^M&=3edkQ=Vdi`p6d9!go!mU3u5AqnW(dTh^jWPW_7?)8a-I?-JF>`tD3 zuUyC1c>o!`-j{etz~EjMp~)rS21f&)QYa-@)Mv>&`zXVICn00|Z#qzqu%I4n7KG z$Y2ux9;Y9@UdxOp8K%(we|;>CEXOJEoIOK6!y0+H%1afw?aFaq$trZ{iC=!`_0VOJ!fGSgQRdr zQL*WP0+)G~WnL}XA1};3V#~H?Mz1~zUF!6cwd_%T<-k__?TfeyXiK>-!dpiPo=iY~ z10R+yu}06OK#l{D)jyZVpHDspSP)*vAvXcB=F8p9TTA*2J~z0G8q(Q4 z=Lq0F&rkQpfbDEKS7n6%cQ1O5f4hQyfvQ~z)0Fb{tM^W9_sd4kvjs>c%6s5wl9vgX zCM`NGUeBc^co0}=SHQHknJP zuq$rnb*rvlJPQHlhMZfP{z<@pg@l;j#%5#x=5iKM=Mosa?^&fnTL!5sFc}Qgc}!;T z%_kkk4fo@UljOUrnm6iL{_a!?6EAK3`065iZXSzo_mj;0j7a^dyT5zU4&4$WHtq)%L)yZ2xiFpk^DUPkund0lvvcsVm;!90 zytO*V6JGvKzJz4HrX0Da59%(VEWVMWz1b{Rm0xM<|J_BS-jC<=IA#rQ4WiegU$f88 zt)xRlhzbLstA4SVXSK?0x}#P?U>Juc3=t03c>wdW0?R_s~vDB(4O-XdMjS@|9W|-yslMW zru%?@8P*Cegh_?=RxHQc%04}rOj`-QPQV1+=5dYncz6&N`Bd{jrvc@DuE(z*{CHAr zx2n#AL3-$pjEVpBtu_nbHF`AeKu3Ty|f3| z1d%cH0UD&bJeXqX0*U}5FHJ{yrEU513ldHqP3s$#Jq5V)gaa6O9smGNZDSC=FPj3j zJS4jeD!!hCZU&e~6TzwUi?02}+Yy-}%rD_h7LVc9f3_a2PrC{uMytSgH!JhMGE_l- zklLH=Q2PVE%MqZQifFR5rIQ1=ytOL*2EPqZ}SiHku zfBUjO=6YSYL>k^q$1gW@2Xf#BBAUSm|W=>(Or#sCYK)&%y|p)`kWqlchwPj28G4NXXak`w_+#odeXBBs*;t%3`8}YQ zFWfq_w2Skq=swc2#BOC)+f+n$$Cz{d;{b|GWcVqX=pX3vBU`fW5cVYp?ifPsz{`NF zE=Avj$z-eAx~(sfY*wi)es+n0l>JSL>F79Ec*yoK)8TuM6#a4iOf7FGu-%a9ECqrp z!Q66SbCM&VMJE^XXxu(=*kJv&3kkd6`lgnz(K}2_fSHAI9yiL71wx@>_)gL{+sd?d zm#<1isB@Q){PDuh*F#psNbeD+a`5AWkHEt$VQ>HDg}l;N@acoVz+EK zZ}jhZ2R-U;j=J<~ju6N(3KTVxplo$YmB+mYFh?uc9JIJ8EdYuL^=gA}`R^`J*5C`l zHU@m)X*qo)hz0^xfsFV5lQCtBn5d#bhx>=>XXd;f@#8IcuItN!NF--{-s-kTTKc-u z>E~ycw}*?*SAo1>Ng~|#JI}GB{6ln*N%w$=ryA3&vvr%Eu~YMvdS~VBVJS4rhv$hJ zv&}y{S4EU$z`*^t{8a7*m?}f`1}3oxKILvk7x$6YK9T;* z;dRC57?cqxb(UJx{X7AYXZ0X!fsx`T{|?<(3|-p8d5-(iw zV0Pl4T==gu>L&MM9oT`@VAh4lK><}*S-Xgf6xV~Z3{us+@&R7p@4|mQ2pg?!{;>iw z5Dke0X!c}VDq(U%&9@^=pa*yXp8!JCH+8;CDbWWp=>wQSiJ}pl7Ev0|`T=u)CuBO4D6=yo=)%iVFZ}V6)zqPNA7JU^4gS zL0L7|EpU_9|J`NnqT@5$!!Btf!sF_xr)tU1!v!=X=+e3Ht^(8mqC$xt_F!M!dUIq| zZ4uXTS}fs@#!h`mqkO&YpvAremHHhTXG@25y>Oo7@GE*jRF(WHK^vXXtzBMOG99g! z?~Uv4T-L=60r%mza%+!A$W%o9bpbWj_1+y(+_hL;Cyb0DU$sZcFk}&wO$_zQeOk(L z&u>=<-9&*l0whUb4CQp<`i3|jP*=AMBVSQ0XGehkp%%e|s&2`*pqu_wg{%zGs$_&{ zrm+RS@~MAI?t+XzaEWQuzNGz ziI&j?%R#qasZ0<#?TBjB!}(~%z5k~`nes$hKxKAZxxs*gT6fbRO@p~PxEQ<$-#x?B zudd@EfDNHuyyLLXPC!8b>`&rs1C)R$z8g@fQ;Wy8YU)gZO)kG zi40&F1?hG!0L}#@x_!>-+?=mLY@Z|(k`2o|3fK9I)Eqxl$t?{` zFOhqfhSPMS>Hq@QV#e2v-((W;HF&)1RfU z8KhO^VB(^_q5_j5%&SCr_2R>XU}^;+hnNIS{JvkoXV<4=+`IU$jqhTTHwv9SeR!{czq_ zwNV;zjlCU<-`7LNCVu#eXF zt}PE0pPr}G_Aj(v>F?CpI+C^>ZTYCqP#dDiB6222FwRD z@Qj-~cUAW35kR6UIqqMpOWV)80g7q;9@G%)N#|URl2RwP4*0W^MrD5g0wNnQ6KCuL zXg&B5MgsOJczKuQ&li<~4~}@Aet+Tya!%G$46>eG#LUAMeEEGvSaQmLu&s;-;YaLbL^NHr4{{Sie%<^Nc&cXMx0I7iA(E0g? zko`LU?xj5kD$()@)kDTb%!bHI81!{cQ-X_f020;@x9KdWs*Mv_S*Jt6e;GE&!TSMD zI3Du%+9w(&@9vOkqX%y%;o^|D1fot>ynmH^AV0}HxxHBMmFvX>Ag?1#rBaZf){Ojg z^n~kGh@vZi8aA>t?gt|S`cV)+4Tu*T3voQsh{fkWx&U8D6#(+*j}W>E5ehp8aDgQs z0Ju2bU|~dm!R!X%gmfz?NBH3$$Y)ewce?JIt0LnKKg)$F=I;i`Ilj*$>4aH+;sX4u z>sV2X?mO-xdJjlqm}~L$p`=SkN6&#Tc`Mo63H;F>AbWcfJtvjCAwP%v^!=l~^GK`k zv8ZS25l^q0$-;c(4hIRiUfxtwGn&~#b~U6&m)14nyj6mzE(tV9IFKEyg!=-lk|Pk8 z^Fg7CbFa;(X~(HSj0rzRaoh8P_w7pq*0r1lpw7Q{A+y?P!|+;zZkXI&khN>Ihx|lA zy0$`e^F4Im;!GCLhr0qb3`)Tui9WD51O&!?*B?+}(5a}7wE-4<#+vle>TVRo!nm8l zWcWVoAbkKDA}>xm@DXmTitS$r`Sz5NC!f$Kil7{G(ZxNk4LJGbQTtACJ-^iuPZUR5 zT$3MVLqBLLeQ*kgknKN=e*)gQ_1^(g3lEE?gK5vzyp?$9y@PKEQO_U$&$kYJRj-?{MXfLQ+PBua+t_}h z6n0G5^A9eqN(&&>zW3LUMnm!q&XX9Qr*YdN@A2Uc4@YFe+$PHekRFHNVk>l9QgEPA z608DPV+kQ>lDGPUI@{$1P(NNIWSU*CPW4VcUP^)Nn$f=iVhm8pT1>=;ss66~?5A4t z{!%;pX~z+-1&x?$v42ONM?ucUb9pMTnNg{$Q8HByPJle4vQc!%70~%R?9PBp%0uvL zgKg)vww^vv8T#PyLuBQ>rLe&Ov<#iM7(SKQNAau~rNmBO{pXeS}*q=2T+r_T+o}YJUD<7sG9YD5|-YPbfdCaWE zZZH(f#E-}P(rd@kA3(6UB|J4PdAr_Ahqm$dc6M50ME&jsKQ~$L$Lre5*SfF;?F&Oq z*8)lCp#l4RREm;ca_lTC&Tkpv^MD$QY(5jvd3WT9#|7vY3E1Dj1-;~cV|8~Psm$d~ zCyogUd?|vOjbsNDN?Dn7qU_OafBgcLwir^7tI~KiQ7>ccRztROz#-Ia{6}<)??ycS zVX)lSa?sLp9!kpsBb%tU+%#!#<#fu~yN-bhf&A|;!||fMC2=(_njwICw@l=DhQufF z12H<-ToJfrQ1XX>VHG`JA+!(w;vl)IH1}&QEhL|IqPna%PiW0I#nXQS@Q}Suc0?w# zS4mQ;*|t%DYU|W)Q+ZHH%yT+N^yhn`LrNvV4=G4l)?N2vP?`7J>VAc5-^C+ zc=9(N45lvc-rTSgzbFexv(`i@9(L>C!*!x>iD7OmfwRlG1Aj+{O}fPW!;5?qUeyQ! zX!@WO8=kShHvJ3a$}Ix9tQa&8L$$?L^nBq6`T!fCS4=7U=0Is568cROIJh|$H( zD|-X12ZcY)S@i+h{QX}pkf^1Bk`M@hY`f>p%^twspRd9@gp-S@8D$?h_tQTy;fPCD zf-42zJwUyPEII%zjMXiubI=WUrB+6r-7Q-&*=X20^3Gl`Kpl4E-?+RIZmh*Ru+yR{ zAck+h1!Rd3&uRzp3VipYDEH3<3_wFc{Jfz8Qo%gPaRRI!8;iT4bW*e8BTC>c@5+$u z1>V*-jQVoxJr!8dPOw2pi{8y`e3*5bn41K9L83Shg4g4A1(ezIQ$p&iAnKWA&{X^M zu3@(z#PthaTJ^78EC|76NW}t>LWXp*`l&q$Kav<7>a#>+T@kKQqj9(bo2&;^g>W5%uS|4+ zsqGuEaGHRg+(xd(^&Jw#?cN_DQJVoljC79>8k&kdDYoJ_k^&9d|9uG|aVdB4uVUYA zg?V}a_69n(QPT<5!TM`G`+fg9gSe<*QT|mkh(z=g?ChtkP?nnh9UQWJ$vw~gq_xt|K=S4Eh#aH8C zDv%Sior1uV*Rb~YQj;uVDIv&3wQ=f#TyFpW!}FJg_BD+ z9$;eUbJCo2Wd2;glF66L`~ZcPPh{LedBS2+krhv>f{H7I@}4NVWRz<29DohdgFt-! zR(f+)BDB8!^NTh)6yRISwYeDsWI{TkA)y zq+&>I4F++R`KJoQQTd$f?#o)Lg0s9nlUQ9N5B@&>F+%KYd1%f&AJGnkxaal=E1V+!C8ejj@ABP=~H%FJGV8Z8K<4Uoq?dNs= z6s@PSjk;19Pq~~jgL3ueycP&qWy=%R2>k5%{TNlwF@6%Fhz| zE5iZ{epit59!+3159An+dHg7^vs_l#@t<9oLxprMzR{vNH~1mLyO}L)Q~=Wa^)bic^uu07V_&%GIqxG ztsb;lxpGtfnilC+ zlj^M%X*whxGEzYSnz7>smMeF_!dW@Sd6n$gauE2@n_IipYnTMk(>*8RNk z0wkKskKs#K8#+cNqq4Fuiz3g~pW>Om+GOF6agM`c`?E%`zhLd!uU-7TSmI>?f$VL@ zW8ns0D^U7%A(@!rVF?OB7#y}1YIZwdk|Dr;`UMGjoIOCmaulqVrV{Vt=|ENh+{wSb zK1e^iaQRDHfWl6J9Pga&&5A5R7_@c3^6#L*UUj{yU+Dr*gX_vQp@S0(M!>X9kC{3(5A52dS`0g@e$*p58_K#*xmXzD6F{ad0aa z32^BvedOVLyK~P(qAdsuU*yMzA)vAKEq|PuhR1|+pa3Ws2Xbr4{2yPe?ulk@2tsZ6 zL#JSrkwBnD)PM%78;%*K!XM&6t6twz?V6-$+iNcbwr+&r7ydCb*m2O`JFGdh*K!Q) zSoT8Tiw)gfOPzti;eP?Cqwp?4?0ETs9nM$7ufUaWPrzv$kz| z79Nk^dv)DDngxs4=teCs4#g+1M4O5xy-g4+E_RpjH=~04I*h_+V1lxlhEwXtyClQNtPVx~C^|NrDCN=JpS+*K+EMwu2I*DoAq>T&>War8 z(px|C@FPb5d@af$_D)S0Z?)%R#d|?7Fw?-CZ z1i#)lxMip!h1k7=-RoHnU6H0wWh%ETI3+-{=YUTCeZSSr&fGpDj>59nw+aY7&7Vj6 z&KvSVEul%_IZ7o!vJAhpmNKV!p%eb7=~1q1h! b|M|cD&E`M+%fWB|^mm2oU;oGd Date: Fri, 5 Jul 2024 17:09:17 +0000 Subject: [PATCH 243/417] fix for const arrays --- src/Functions/array/arrayIndex.h | 10 +++++++++- 1 file changed, 9 insertions(+), 1 deletion(-) diff --git a/src/Functions/array/arrayIndex.h b/src/Functions/array/arrayIndex.h index 111ab92b006..0782f109187 100644 --- a/src/Functions/array/arrayIndex.h +++ b/src/Functions/array/arrayIndex.h @@ -1,4 +1,5 @@ #pragma once +#include #include #include #include @@ -13,6 +14,8 @@ #include #include #include +#include "Common/Logger.h" +#include "Common/logger_useful.h" #include #include #include @@ -712,6 +715,7 @@ private: auto right = recursiveRemoveLowCardinality(right_const->getDataColumnPtr()); UInt64 index = 0; + UInt64 left_size = arguments[0].column->size(); ResultColumnPtr col_result = ResultColumnType::create(); if (!right->isNullAt(0)) @@ -732,6 +736,10 @@ private: else { col_result->getData().resize_fill(col_array->size()); + + if (col_array_const) + return ColumnConst::create(std::move(col_result), left_size); + return col_result; } } @@ -745,7 +753,7 @@ private: nullptr); if (col_array_const) - return ColumnConst::create(std::move(col_result), arguments[0].column->size()); + return ColumnConst::create(std::move(col_result), left_size); return col_result; } From c327f49f0cba7f3c1a8dab00bb355b0c6a921650 Mon Sep 17 00:00:00 2001 From: Blargian Date: Fri, 5 Jul 2024 20:04:43 +0200 Subject: [PATCH 244/417] add fullHostName which is an alias --- docs/en/sql-reference/functions/other-functions.md | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/docs/en/sql-reference/functions/other-functions.md b/docs/en/sql-reference/functions/other-functions.md index 58fc1eba02e..95ef5e7bd21 100644 --- a/docs/en/sql-reference/functions/other-functions.md +++ b/docs/en/sql-reference/functions/other-functions.md @@ -86,7 +86,7 @@ Returns the fully qualified domain name of the ClickHouse server. fqdn(); ``` -This function is case-insensitive. +Aliases: `fullHostName`, 'FQDN'. **Returned value** From a502933f5ea33f9f7ee07944afbd926d21603731 Mon Sep 17 00:00:00 2001 From: Nikita Taranov Date: Fri, 5 Jul 2024 20:09:08 +0200 Subject: [PATCH 245/417] Fix clang-tidy error in BufferWithOwnMemory.h --- src/IO/BufferWithOwnMemory.h | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/src/IO/BufferWithOwnMemory.h b/src/IO/BufferWithOwnMemory.h index 0ec733f7840..da38bccdea1 100644 --- a/src/IO/BufferWithOwnMemory.h +++ b/src/IO/BufferWithOwnMemory.h @@ -44,7 +44,7 @@ struct Memory : boost::noncopyable, Allocator char * m_data = nullptr; size_t alignment = 0; - [[maybe_unused]] bool allow_gwp_asan_force_sample; + [[maybe_unused]] bool allow_gwp_asan_force_sample{false}; Memory() = default; From e572dfd2dc4465b8a23db3fadb071b6eda8693b7 Mon Sep 17 00:00:00 2001 From: Vitaly Baranov Date: Fri, 5 Jul 2024 20:15:19 +0200 Subject: [PATCH 246/417] Use Common/Base64.h --- src/Server/HTTP/authenticateUserByHTTP.cpp | 29 +--------------------- 1 file changed, 1 insertion(+), 28 deletions(-) diff --git a/src/Server/HTTP/authenticateUserByHTTP.cpp b/src/Server/HTTP/authenticateUserByHTTP.cpp index 3bcf6552e54..40ac5f82657 100644 --- a/src/Server/HTTP/authenticateUserByHTTP.cpp +++ b/src/Server/HTTP/authenticateUserByHTTP.cpp @@ -4,16 +4,13 @@ #include #include #include +#include #include #include #include #include #include -#include -#include -#include -#include #include #if USE_SSL @@ -32,30 +29,6 @@ namespace ErrorCodes } -namespace -{ - String base64Decode(const String & encoded) - { - String decoded; - Poco::MemoryInputStream istr(encoded.data(), encoded.size()); - Poco::Base64Decoder decoder(istr); - Poco::StreamCopier::copyToString(decoder, decoded); - return decoded; - } - - String base64Encode(const String & decoded) - { - std::ostringstream ostr; // STYLE_CHECK_ALLOW_STD_STRING_STREAM - ostr.exceptions(std::ios::failbit); - Poco::Base64Encoder encoder(ostr); - encoder.rdbuf()->setLineLength(0); - encoder << decoded; - encoder.close(); - return ostr.str(); - } -} - - bool authenticateUserByHTTP( const HTTPServerRequest & request, const HTMLForm & params, From c42b5eddd4eaf2e8a185a7b768aab9bd557547ab Mon Sep 17 00:00:00 2001 From: Vitaly Baranov Date: Fri, 5 Jul 2024 21:10:39 +0200 Subject: [PATCH 247/417] Fix error messages in authenticateUserByHTTP() when multiple authentication methods are used simultaneously. --- src/Server/HTTP/authenticateUserByHTTP.cpp | 69 ++++++++++++---------- 1 file changed, 38 insertions(+), 31 deletions(-) diff --git a/src/Server/HTTP/authenticateUserByHTTP.cpp b/src/Server/HTTP/authenticateUserByHTTP.cpp index 40ac5f82657..8929c912601 100644 --- a/src/Server/HTTP/authenticateUserByHTTP.cpp +++ b/src/Server/HTTP/authenticateUserByHTTP.cpp @@ -29,6 +29,17 @@ namespace ErrorCodes } +namespace +{ + /// Throws an exception that multiple authorization schemes are used simultaneously. + [[noreturn]] void throwMultipleAuthenticationMethods(std::string_view method1, std::string_view method2) + { + throw Exception(ErrorCodes::AUTHENTICATION_FAILED, + "Invalid authentication: it is not allowed to use {} and {} simultaneously", method1, method2); + } +} + + bool authenticateUserByHTTP( const HTTPServerRequest & request, const HTMLForm & params, @@ -46,11 +57,11 @@ bool authenticateUserByHTTP( std::string user = request.get("X-ClickHouse-User", ""); std::string password = request.get("X-ClickHouse-Key", ""); std::string quota_key = request.get("X-ClickHouse-Quota", ""); + bool has_auth_headers = !user.empty() || !password.empty(); /// The header 'X-ClickHouse-SSL-Certificate-Auth: on' enables checking the common name /// extracted from the SSL certificate used for this connection instead of checking password. bool has_ssl_certificate_auth = (request.get("X-ClickHouse-SSL-Certificate-Auth", "") == "on"); - bool has_auth_headers = !user.empty() || !password.empty() || has_ssl_certificate_auth; /// User name and password can be passed using HTTP Basic auth or query parameters /// (both methods are insecure). @@ -60,45 +71,41 @@ bool authenticateUserByHTTP( std::string spnego_challenge; SSLCertificateSubjects certificate_subjects; - if (has_auth_headers) + if (has_ssl_certificate_auth) + { +#if USE_SSL + /// It is prohibited to mix different authorization schemes. + if (!password.empty()) + throwMultipleAuthenticationMethods("SSL certificate authentication", "authentication via password"); + if (has_http_credentials) + throwMultipleAuthenticationMethods("SSL certificate authentication", "Authorization HTTP header"); + if (has_credentials_in_query_params) + throwMultipleAuthenticationMethods("SSL certificate authentication", "authentication via parameters"); + + if (request.havePeerCertificate()) + certificate_subjects = extractSSLCertificateSubjects(request.peerCertificate()); + + if (certificate_subjects.empty()) + throw Exception(ErrorCodes::AUTHENTICATION_FAILED, + "Invalid authentication: SSL certificate authentication requires nonempty certificate's Common Name or Subject Alternative Name"); +#else + throw Exception(ErrorCodes::SUPPORT_IS_DISABLED, + "SSL certificate authentication disabled because ClickHouse was built without SSL library"); +#endif + } + else if (has_auth_headers) { /// It is prohibited to mix different authorization schemes. if (has_http_credentials) - throw Exception(ErrorCodes::AUTHENTICATION_FAILED, - "Invalid authentication: it is not allowed " - "to use SSL certificate authentication and Authorization HTTP header simultaneously"); + throwMultipleAuthenticationMethods("X-ClickHouse HTTP headers", "Authorization HTTP header"); if (has_credentials_in_query_params) - throw Exception(ErrorCodes::AUTHENTICATION_FAILED, - "Invalid authentication: it is not allowed " - "to use SSL certificate authentication and authentication via parameters simultaneously simultaneously"); - - if (has_ssl_certificate_auth) - { -#if USE_SSL - if (!password.empty()) - throw Exception(ErrorCodes::AUTHENTICATION_FAILED, - "Invalid authentication: it is not allowed " - "to use SSL certificate authentication and authentication via password simultaneously"); - - if (request.havePeerCertificate()) - certificate_subjects = extractSSLCertificateSubjects(request.peerCertificate()); - - if (certificate_subjects.empty()) - throw Exception(ErrorCodes::AUTHENTICATION_FAILED, - "Invalid authentication: SSL certificate authentication requires nonempty certificate's Common Name or Subject Alternative Name"); -#else - throw Exception(ErrorCodes::SUPPORT_IS_DISABLED, - "SSL certificate authentication disabled because ClickHouse was built without SSL library"); -#endif - } + throwMultipleAuthenticationMethods("X-ClickHouse HTTP headers", "authentication via parameters"); } else if (has_http_credentials) { /// It is prohibited to mix different authorization schemes. if (has_credentials_in_query_params) - throw Exception(ErrorCodes::AUTHENTICATION_FAILED, - "Invalid authentication: it is not allowed " - "to use Authorization HTTP header and authentication via parameters simultaneously"); + throwMultipleAuthenticationMethods("Authorization HTTP header", "authentication via parameters"); std::string scheme; std::string auth_info; From eb7c2e43412b4ba7274d993e2a8fc658beeab9b2 Mon Sep 17 00:00:00 2001 From: Vitaly Baranov Date: Fri, 5 Jul 2024 21:42:54 +0200 Subject: [PATCH 248/417] Remove unnecessary "const" from return type. --- base/poco/Net/include/Poco/Net/NameValueCollection.h | 2 +- base/poco/Net/src/NameValueCollection.cpp | 2 +- 2 files changed, 2 insertions(+), 2 deletions(-) diff --git a/base/poco/Net/include/Poco/Net/NameValueCollection.h b/base/poco/Net/include/Poco/Net/NameValueCollection.h index b66819840b1..2337535bd11 100644 --- a/base/poco/Net/include/Poco/Net/NameValueCollection.h +++ b/base/poco/Net/include/Poco/Net/NameValueCollection.h @@ -79,7 +79,7 @@ namespace Net /// Returns the value of the first name-value pair with the given name. /// If no value with the given name has been found, the defaultValue is returned. - const std::vector getAll(const std::string & name) const; + std::vector getAll(const std::string & name) const; /// Returns all values of all name-value pairs with the given name. /// /// Returns an empty vector if there are no name-value pairs with the given name. diff --git a/base/poco/Net/src/NameValueCollection.cpp b/base/poco/Net/src/NameValueCollection.cpp index 0cb31924579..e35d66d3bde 100644 --- a/base/poco/Net/src/NameValueCollection.cpp +++ b/base/poco/Net/src/NameValueCollection.cpp @@ -102,7 +102,7 @@ const std::string& NameValueCollection::get(const std::string& name, const std:: return defaultValue; } -const std::vector NameValueCollection::getAll(const std::string& name) const +std::vector NameValueCollection::getAll(const std::string& name) const { std::vector values; for (ConstIterator it = _map.find(name); it != _map.end(); it++) From 937ce8f780bb8d5ebb6ba115ee2d0c681a053299 Mon Sep 17 00:00:00 2001 From: Blargian Date: Fri, 5 Jul 2024 21:50:56 +0200 Subject: [PATCH 249/417] add aggThrow function to docs --- .../aggregate-functions/reference/aggthrow.md | 37 +++++++++++++++++++ .../aggregate-functions/reference/index.md | 1 + 2 files changed, 38 insertions(+) create mode 100644 docs/en/sql-reference/aggregate-functions/reference/aggthrow.md diff --git a/docs/en/sql-reference/aggregate-functions/reference/aggthrow.md b/docs/en/sql-reference/aggregate-functions/reference/aggthrow.md new file mode 100644 index 00000000000..fdbfd5b9e41 --- /dev/null +++ b/docs/en/sql-reference/aggregate-functions/reference/aggthrow.md @@ -0,0 +1,37 @@ +--- +slug: /en/sql-reference/aggregate-functions/reference/aggthrow +sidebar_position: 101 +--- + +# aggThrow + +This function can be used for the purpose of testing exception safety. It will throw an exception on creation with the specified probability. + +**Syntax** + +```sql +aggThrow(throw_prob) +``` + +**Arguments** + +- `throw_prob` — Probability to throw on creation. [Float64](../../data-types/float.md). + +**Returned value** + +- An exception: `Code: 503. DB::Exception: Aggregate function aggThrow has thrown exception successfully`. + +**Example** + +Query: + +```sql +SELECT number % 2 AS even, aggThrow(number) FROM numbers(10) GROUP BY even; +``` + +Result: + +```response +Received exception: +Code: 503. DB::Exception: Aggregate function aggThrow has thrown exception successfully: While executing AggregatingTransform. (AGGREGATE_FUNCTION_THROW) +``` diff --git a/docs/en/sql-reference/aggregate-functions/reference/index.md b/docs/en/sql-reference/aggregate-functions/reference/index.md index e3725b6a430..b0e5582bd87 100644 --- a/docs/en/sql-reference/aggregate-functions/reference/index.md +++ b/docs/en/sql-reference/aggregate-functions/reference/index.md @@ -43,6 +43,7 @@ Standard aggregate functions: ClickHouse-specific aggregate functions: +- [aggThrow](../reference/aggthrow.md) - [analysisOfVariance](../reference/analysis_of_variance.md) - [any](../reference/any_respect_nulls.md) - [anyHeavy](../reference/anyheavy.md) From f94076ce8f7ea8c6eeec3637d72e67b6dd6fb1d1 Mon Sep 17 00:00:00 2001 From: alesapin Date: Fri, 5 Jul 2024 22:20:46 +0200 Subject: [PATCH 250/417] Fix clang tidy --- src/IO/BufferWithOwnMemory.h | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/src/IO/BufferWithOwnMemory.h b/src/IO/BufferWithOwnMemory.h index 0ec733f7840..da38bccdea1 100644 --- a/src/IO/BufferWithOwnMemory.h +++ b/src/IO/BufferWithOwnMemory.h @@ -44,7 +44,7 @@ struct Memory : boost::noncopyable, Allocator char * m_data = nullptr; size_t alignment = 0; - [[maybe_unused]] bool allow_gwp_asan_force_sample; + [[maybe_unused]] bool allow_gwp_asan_force_sample{false}; Memory() = default; From fdd7353c92a4fc9bcf517633e049fcb301fdf6c0 Mon Sep 17 00:00:00 2001 From: Blargian Date: Fri, 5 Jul 2024 23:47:11 +0200 Subject: [PATCH 251/417] update initCap --- .../functions/string-functions.md | 216 ++++++++++++++---- 1 file changed, 167 insertions(+), 49 deletions(-) diff --git a/docs/en/sql-reference/functions/string-functions.md b/docs/en/sql-reference/functions/string-functions.md index c068b0e9d17..1e801cb1887 100644 --- a/docs/en/sql-reference/functions/string-functions.md +++ b/docs/en/sql-reference/functions/string-functions.md @@ -12,9 +12,7 @@ Functions for [searching](string-search-functions.md) in strings and for [replac ## empty -Checks whether the input string is empty. - -A string is considered non-empty if it contains at least one byte, even if this byte is a space or the null byte. +Checks whether the input string is empty. A string is considered non-empty if it contains at least one byte, even if this byte is a space or the null byte. The function is also available for [arrays](array-functions.md#function-empty) and [UUIDs](uuid-functions.md#empty). @@ -48,9 +46,7 @@ Result: ## notEmpty -Checks whether the input string is non-empty. - -A string is considered non-empty if it contains at least one byte, even if this byte is a space or the null byte. +Checks whether the input string is non-empty. A string is considered non-empty if it contains at least one byte, even if this byte is a space or the null byte. The function is also available for [arrays](array-functions.md#function-notempty) and [UUIDs](uuid-functions.md#notempty). @@ -96,7 +92,7 @@ length(s) **Parameters** -- `s`: An input string or array. [String](../data-types/string)/[Array](../data-types/array). +- `s` — An input string or array. [String](../data-types/string)/[Array](../data-types/array). **Returned value** @@ -149,7 +145,7 @@ lengthUTF8(s) **Parameters** -- `s`: String containing valid UTF-8 encoded text. [String](../data-types/string). +- `s` — String containing valid UTF-8 encoded text. [String](../data-types/string). **Returned value** @@ -183,8 +179,8 @@ left(s, offset) **Parameters** -- `s`: The string to calculate a substring from. [String](../data-types/string.md) or [FixedString](../data-types/fixedstring.md). -- `offset`: The number of bytes of the offset. [UInt*](../data-types/int-uint). +- `s` — The string to calculate a substring from. [String](../data-types/string.md) or [FixedString](../data-types/fixedstring.md). +- `offset` — The number of bytes of the offset. [UInt*](../data-types/int-uint). **Returned value** @@ -230,8 +226,8 @@ leftUTF8(s, offset) **Parameters** -- `s`: The UTF-8 encoded string to calculate a substring from. [String](../data-types/string.md) or [FixedString](../data-types/fixedstring.md). -- `offset`: The number of bytes of the offset. [UInt*](../data-types/int-uint). +- `s` — The UTF-8 encoded string to calculate a substring from. [String](../data-types/string.md) or [FixedString](../data-types/fixedstring.md). +- `offset` — The number of bytes of the offset. [UInt*](../data-types/int-uint). **Returned value** @@ -347,8 +343,8 @@ right(s, offset) **Parameters** -- `s`: The string to calculate a substring from. [String](../data-types/string.md) or [FixedString](../data-types/fixedstring.md). -- `offset`: The number of bytes of the offset. [UInt*](../data-types/int-uint). +- `s` — The string to calculate a substring from. [String](../data-types/string.md) or [FixedString](../data-types/fixedstring.md). +- `offset` — The number of bytes of the offset. [UInt*](../data-types/int-uint). **Returned value** @@ -394,8 +390,8 @@ rightUTF8(s, offset) **Parameters** -- `s`: The UTF-8 encoded string to calculate a substring from. [String](../data-types/string.md) or [FixedString](../data-types/fixedstring.md). -- `offset`: The number of bytes of the offset. [UInt*](../data-types/int-uint). +- `s` — The UTF-8 encoded string to calculate a substring from. [String](../data-types/string.md) or [FixedString](../data-types/fixedstring.md). +- `offset` — The number of bytes of the offset. [UInt*](../data-types/int-uint). **Returned value** @@ -547,7 +543,7 @@ Alias: `ucase` **Parameters** -- `input`: A string type [String](../data-types/string.md). +- `input` — A string type [String](../data-types/string.md). **Returned value** @@ -571,17 +567,47 @@ SELECT upper('clickhouse'); Converts a string to lowercase, assuming that the string contains valid UTF-8 encoded text. If this assumption is violated, no exception is thrown and the result is undefined. -Does not detect the language, e.g. for Turkish the result might not be exactly correct (i/İ vs. i/I). +:::note +Does not detect the language, e.g. for Turkish the result might not be exactly correct (i/İ vs. i/I). If the length of the UTF-8 byte sequence is different for upper and lower case of a code point, the result may be incorrect for this code point. +::: -If the length of the UTF-8 byte sequence is different for upper and lower case of a code point, the result may be incorrect for this code point. +**Syntax** + +```sql +lowerUTF8(input) +``` + +**Parameters** + +- `input` — A string type [String](../data-types/string.md). + +**Returned value** + +- A [String](../data-types/string.md) data type value. + +**Example** + +Query: + +``` sql +SELECT lowerUTF8('MÜNCHEN') as Lowerutf8; +``` + +Result: + +``` response +┌─Lowerutf8─┐ +│ münchen │ +└───────────┘ +``` ## upperUTF8 Converts a string to uppercase, assuming that the string contains valid UTF-8 encoded text. If this assumption is violated, no exception is thrown and the result is undefined. -Does not detect the language, e.g. for Turkish the result might not be exactly correct (i/İ vs. i/I). - -If the length of the UTF-8 byte sequence is different for upper and lower case of a code point, the result may be incorrect for this code point. +:::note +Does not detect the language, e.g. for Turkish the result might not be exactly correct (i/İ vs. i/I). If the length of the UTF-8 byte sequence is different for upper and lower case of a code point, the result may be incorrect for this code point. +::: **Syntax** @@ -591,7 +617,7 @@ upperUTF8(input) **Parameters** -- `input`: A string type [String](../data-types/string.md). +- `input` — A string type [String](../data-types/string.md). **Returned value** @@ -605,6 +631,8 @@ Query: SELECT upperUTF8('München') as Upperutf8; ``` +Result: + ``` response ┌─Upperutf8─┐ │ MÜNCHEN │ @@ -615,6 +643,34 @@ SELECT upperUTF8('München') as Upperutf8; Returns 1, if the set of bytes constitutes valid UTF-8-encoded text, otherwise 0. +**Syntax** + +``` sql +isValidUTF8(input) +``` + +**Parameters** + +- `input` — A string type [String](../data-types/string.md). + +**Returned value** + +- Returns `1`, if the set of bytes constitutes valid UTF-8-encoded text, otherwise `0`. + +Query: + +``` sql +SELECT isValidUTF8('\xc3\xb1') AS valid, isValidUTF8('\xc3\x28') AS invalid; +``` + +Result: + +``` response +┌─valid─┬─invalid─┐ +│ 1 │ 0 │ +└───────┴─────────┘ +``` + ## toValidUTF8 Replaces invalid UTF-8 characters by the `�` (U+FFFD) character. All running in a row invalid characters are collapsed into the one replacement character. @@ -884,7 +940,7 @@ Returns the substring of a string `s` which starts at the specified byte index ` substring(s, offset[, length]) ``` -Alias: +Aliases: - `substr` - `mid` - `byteSlice` @@ -927,9 +983,9 @@ substringUTF8(s, offset[, length]) **Arguments** -- `s`: The string to calculate a substring from. [String](../data-types/string.md), [FixedString](../data-types/fixedstring.md) or [Enum](../data-types/enum.md) -- `offset`: The starting position of the substring in `s` . [(U)Int*](../data-types/int-uint.md). -- `length`: The maximum length of the substring. [(U)Int*](../data-types/int-uint.md). Optional. +- `s` — The string to calculate a substring from. [String](../data-types/string.md), [FixedString](../data-types/fixedstring.md) or [Enum](../data-types/enum.md) +- `offset` — The starting position of the substring in `s` . [(U)Int*](../data-types/int-uint.md). +- `length` — The maximum length of the substring. [(U)Int*](../data-types/int-uint.md). Optional. **Returned value** @@ -965,9 +1021,9 @@ Alias: `SUBSTRING_INDEX` **Arguments** -- s: The string to extract substring from. [String](../data-types/string.md). -- delim: The character to split. [String](../data-types/string.md). -- count: The number of occurrences of the delimiter to count before extracting the substring. If count is positive, everything to the left of the final delimiter (counting from the left) is returned. If count is negative, everything to the right of the final delimiter (counting from the right) is returned. [UInt or Int](../data-types/int-uint.md) +- s — The string to extract substring from. [String](../data-types/string.md). +- delim — The character to split. [String](../data-types/string.md). +- count — The number of occurrences of the delimiter to count before extracting the substring. If count is positive, everything to the left of the final delimiter (counting from the left) is returned. If count is negative, everything to the right of the final delimiter (counting from the right) is returned. [UInt or Int](../data-types/int-uint.md) **Example** @@ -996,9 +1052,9 @@ substringIndexUTF8(s, delim, count) **Arguments** -- `s`: The string to extract substring from. [String](../data-types/string.md). -- `delim`: The character to split. [String](../data-types/string.md). -- `count`: The number of occurrences of the delimiter to count before extracting the substring. If count is positive, everything to the left of the final delimiter (counting from the left) is returned. If count is negative, everything to the right of the final delimiter (counting from the right) is returned. [UInt or Int](../data-types/int-uint.md) +- `s` — The string to extract substring from. [String](../data-types/string.md). +- `delim` — The character to split. [String](../data-types/string.md). +- `count` — The number of occurrences of the delimiter to count before extracting the substring. If count is positive, everything to the left of the final delimiter (counting from the left) is returned. If count is negative, everything to the right of the final delimiter (counting from the right) is returned. [UInt or Int](../data-types/int-uint.md) **Returned value** @@ -1278,7 +1334,7 @@ tryBase64Decode(encoded) **Arguments** -- `encoded`: [String](../data-types/string.md) column or constant. If the string is not a valid Base64-encoded value, returns an empty string. +- `encoded` — [String](../data-types/string.md) column or constant. If the string is not a valid Base64-encoded value, returns an empty string. **Returned value** @@ -1310,7 +1366,7 @@ tryBase64URLDecode(encodedUrl) **Parameters** -- `encodedURL`: [String](../data-types/string.md) column or constant. If the string is not a valid Base64-encoded value with URL-specific modifications, returns an empty string. +- `encodedURL` — [String](../data-types/string.md) column or constant. If the string is not a valid Base64-encoded value with URL-specific modifications, returns an empty string. **Returned value** @@ -1936,7 +1992,7 @@ soundex(val) **Arguments** -- `val` - Input value. [String](../data-types/string.md) +- `val` — Input value. [String](../data-types/string.md) **Returned value** @@ -1969,7 +2025,7 @@ punycodeEncode(val) **Arguments** -- `val` - Input value. [String](../data-types/string.md) +- `val` — Input value. [String](../data-types/string.md) **Returned value** @@ -2002,7 +2058,7 @@ punycodeEncode(val) **Arguments** -- `val` - Punycode-encoded string. [String](../data-types/string.md) +- `val` — Punycode-encoded string. [String](../data-types/string.md) **Returned value** @@ -2028,7 +2084,7 @@ Like `punycodeDecode` but returns an empty string if no valid Punycode-encoded s ## idnaEncode -Returns the the ASCII representation (ToASCII algorithm) of a domain name according to the [Internationalized Domain Names in Applications](https://en.wikipedia.org/wiki/Internationalized_domain_name#Internationalizing_Domain_Names_in_Applications) (IDNA) mechanism. +Returns the ASCII representation (ToASCII algorithm) of a domain name according to the [Internationalized Domain Names in Applications](https://en.wikipedia.org/wiki/Internationalized_domain_name#Internationalizing_Domain_Names_in_Applications) (IDNA) mechanism. The input string must be UTF-encoded and translatable to an ASCII string, otherwise an exception is thrown. Note: No percent decoding or trimming of tabs, spaces or control characters is performed. @@ -2040,7 +2096,7 @@ idnaEncode(val) **Arguments** -- `val` - Input value. [String](../data-types/string.md) +- `val` — Input value. [String](../data-types/string.md) **Returned value** @@ -2066,7 +2122,7 @@ Like `idnaEncode` but returns an empty string in case of an error instead of thr ## idnaDecode -Returns the the Unicode (UTF-8) representation (ToUnicode algorithm) of a domain name according to the [Internationalized Domain Names in Applications](https://en.wikipedia.org/wiki/Internationalized_domain_name#Internationalizing_Domain_Names_in_Applications) (IDNA) mechanism. +Returns the Unicode (UTF-8) representation (ToUnicode algorithm) of a domain name according to the [Internationalized Domain Names in Applications](https://en.wikipedia.org/wiki/Internationalized_domain_name#Internationalizing_Domain_Names_in_Applications) (IDNA) mechanism. In case of an error (e.g. because the input is invalid), the input string is returned. Note that repeated application of `idnaEncode()` and `idnaDecode()` does not necessarily return the original string due to case normalization. @@ -2078,7 +2134,7 @@ idnaDecode(val) **Arguments** -- `val` - Input value. [String](../data-types/string.md) +- `val` — Input value. [String](../data-types/string.md) **Returned value** @@ -2122,7 +2178,7 @@ Result: └───────────────────────────────────────────┘ ``` -Alias: mismatches +Alias: `mismatches` ## stringJaccardIndex @@ -2176,7 +2232,7 @@ Result: └─────────────────────────────────────┘ ``` -Alias: levenshteinDistance +Alias: `levenshteinDistance` ## editDistanceUTF8 @@ -2202,7 +2258,7 @@ Result: └─────────────────────────────────────┘ ``` -Alias: levenshteinDistanceUTF8 +Alias: `levenshteinDistanceUTF8` ## damerauLevenshteinDistance @@ -2280,13 +2336,75 @@ Result: Convert the first letter of each word to upper case and the rest to lower case. Words are sequences of alphanumeric characters separated by non-alphanumeric characters. +**Syntax** + +```sql +initcap(val) +``` + +**Arguments** + +- `val` — Input value. [String](../data-types/string.md). + +**Returned value** + +- `val` with the first letter of each word converted to upper case. [String](../data-types/string.md). + +**Example** + +Query: + +```sql +SELECT initcap('building for fast'); +``` + +Result: + +```text +┌─initcap('building for fast')─┐ +│ Building For Fast │ +└──────────────────────────────┘ +``` + ## initcapUTF8 -Like [initcap](#initcap), assuming that the string contains valid UTF-8 encoded text. If this assumption is violated, no exception is thrown and the result is undefined. - -Does not detect the language, e.g. for Turkish the result might not be exactly correct (i/İ vs. i/I). +Like [initcap](#initcap), сonvert the first letter of each word to upper case and the rest to lower case. Assumes that the string contains valid UTF-8 encoded text. +If this assumption is violated, no exception is thrown and the result is undefined. +:::note +This function does not detect the language, e.g. for Turkish the result might not be exactly correct (i/İ vs. i/I). If the length of the UTF-8 byte sequence is different for upper and lower case of a code point, the result may be incorrect for this code point. +::: + +**Syntax** + +```sql +initcapUTF8(val) +``` + +**Arguments** + +- `val` — Input value. [String](../data-types/string.md). + +**Returned value** + +- `val` with the first letter of each word converted to upper case. [String](../data-types/string.md). + +**Example** + +Query: + +```sql +SELECT initcapUTF8('не тормозит'); +``` + +Result: + +```text +┌─initcapUTF8('не тормозит')─┐ +│ Не Тормозит │ +└────────────────────────────┘ +``` ## firstLine @@ -2300,7 +2418,7 @@ firstLine(val) **Arguments** -- `val` - Input value. [String](../data-types/string.md) +- `val` — Input value. [String](../data-types/string.md) **Returned value** From d4a4e4ec2dd8a8a141a705ea6b90506bb486b7e8 Mon Sep 17 00:00:00 2001 From: Michael Kolupaev Date: Fri, 5 Jul 2024 22:42:22 +0000 Subject: [PATCH 252/417] Fix crash when adding empty tuple to query cache --- src/Columns/ColumnTuple.cpp | 8 +++++++- .../0_stateless/03201_query_cache_empty_tuple.reference | 2 ++ .../queries/0_stateless/03201_query_cache_empty_tuple.sql | 2 ++ 3 files changed, 11 insertions(+), 1 deletion(-) create mode 100644 tests/queries/0_stateless/03201_query_cache_empty_tuple.reference create mode 100644 tests/queries/0_stateless/03201_query_cache_empty_tuple.sql diff --git a/src/Columns/ColumnTuple.cpp b/src/Columns/ColumnTuple.cpp index 2159495b68f..f262a8676b7 100644 --- a/src/Columns/ColumnTuple.cpp +++ b/src/Columns/ColumnTuple.cpp @@ -711,7 +711,13 @@ void ColumnTuple::takeDynamicStructureFromSourceColumns(const Columns & source_c ColumnPtr ColumnTuple::compress() const { if (columns.empty()) - return Ptr(); + { + return ColumnCompressed::create(size(), 0, + [n = column_length] + { + return ColumnTuple::create(n); + }); + } size_t byte_size = 0; Columns compressed; diff --git a/tests/queries/0_stateless/03201_query_cache_empty_tuple.reference b/tests/queries/0_stateless/03201_query_cache_empty_tuple.reference new file mode 100644 index 00000000000..50e44edaecb --- /dev/null +++ b/tests/queries/0_stateless/03201_query_cache_empty_tuple.reference @@ -0,0 +1,2 @@ +() 0 +() 0 diff --git a/tests/queries/0_stateless/03201_query_cache_empty_tuple.sql b/tests/queries/0_stateless/03201_query_cache_empty_tuple.sql new file mode 100644 index 00000000000..8e133143ef8 --- /dev/null +++ b/tests/queries/0_stateless/03201_query_cache_empty_tuple.sql @@ -0,0 +1,2 @@ +SELECT tuple(), 0 FROM numbers(1) SETTINGS use_query_cache = true; +SELECT tuple(), 0 FROM numbers(1) SETTINGS use_query_cache = true; From fbe172dd593108ba6728a93ed9be6b0300c631f8 Mon Sep 17 00:00:00 2001 From: Justin de Guzman Date: Fri, 5 Jul 2024 18:28:59 -0700 Subject: [PATCH 253/417] [Docs] Make it obvious that Object Data Type is deprecated --- docs/en/sql-reference/data-types/json.md | 9 +++------ 1 file changed, 3 insertions(+), 6 deletions(-) diff --git a/docs/en/sql-reference/data-types/json.md b/docs/en/sql-reference/data-types/json.md index 9e9c3fdb489..f2eac12594d 100644 --- a/docs/en/sql-reference/data-types/json.md +++ b/docs/en/sql-reference/data-types/json.md @@ -5,14 +5,11 @@ sidebar_label: Object Data Type keywords: [object, data type] --- -:::note -**ClickHouse handles JSON documents in a variety of way, allowing you to bring structure to this data and query it quickly and efficiently. See [this guide](/docs/en/integrations/data-formats/json) for detailed guidance on working with JSON.**::: +# Object Data Type (deprecated) -# Object Data Type +**This feature is not production-ready and is now deprecated.** If you need to work with JSON documents, consider using [this guide](/docs/en/integrations/data-ingestion/data-formats/json) instead. A new implementation to support JSON object is in progress and can be tracked [here](https://github.com/ClickHouse/ClickHouse/issues/54864). -:::note -This feature is not production-ready and is now deprecated. If you need to work with JSON documents, consider using [this guide](/docs/en/integrations/data-ingestion/data-formats/json) instead. A new implementation to support JSON object is in progress and can be tracked [here](https://github.com/ClickHouse/ClickHouse/issues/54864) -::: +
Stores JavaScript Object Notation (JSON) documents in a single column. From ffd6bf28b1f844c360cd054e55dcedb325854d53 Mon Sep 17 00:00:00 2001 From: Azat Khuzhin Date: Sat, 6 Jul 2024 07:42:15 +0200 Subject: [PATCH 254/417] tests: fix 01563_distributed_query_finish flakiness (due to system.*_log_sender) From CI logs [1], during this test was executing on server: 2024.07.05 19:29:45.856853 [ 1328 ] {} system.zookeeper_log_sender.DistributedInsertQueue.default: Code: 210. DB::NetException: Connection reset by peer, while writing to socket (172.17.0.2:38546 -> 3.16.142.177:9440): While sending /var/lib/clickhouse/store/aa8/aa8f6e66-486b-4dc3-85a1-4941e69cb99f/shard1_replica1/447.bin. (NETWORK_ERROR), Stack trace (when copying this message, always include the lines below): [1]: https://s3.amazonaws.com/clickhouse-test-reports/66162/daae5d4d4661c780b6368950ec484415ca3a0492/stateless_tests__aarch64_.html So let's add retries Signed-off-by: Azat Khuzhin --- .../01563_distributed_query_finish.reference | 1 - .../01563_distributed_query_finish.sh | 29 +++++++++++-------- 2 files changed, 17 insertions(+), 13 deletions(-) diff --git a/tests/queries/0_stateless/01563_distributed_query_finish.reference b/tests/queries/0_stateless/01563_distributed_query_finish.reference index c3688b553c4..b48979a492e 100644 --- a/tests/queries/0_stateless/01563_distributed_query_finish.reference +++ b/tests/queries/0_stateless/01563_distributed_query_finish.reference @@ -1,2 +1 @@ -1,0 NETWORK_ERROR=0 diff --git a/tests/queries/0_stateless/01563_distributed_query_finish.sh b/tests/queries/0_stateless/01563_distributed_query_finish.sh index 0019c714e40..e3c5928f108 100755 --- a/tests/queries/0_stateless/01563_distributed_query_finish.sh +++ b/tests/queries/0_stateless/01563_distributed_query_finish.sh @@ -19,20 +19,25 @@ create table dist_01247 as data_01247 engine=Distributed(test_cluster_two_shards select * from dist_01247 format Null; EOL -network_errors_before=$($CLICKHOUSE_CLIENT -q "SELECT value FROM system.errors WHERE name = 'NETWORK_ERROR'") +# NOTE: it is possible to got NETWORK_ERROR even with no-parallel, at least due to system.*_log_sender to the cloud +for ((i = 0; i < 100; ++i)); do + network_errors_before=$($CLICKHOUSE_CLIENT -q "SELECT value FROM system.errors WHERE name = 'NETWORK_ERROR'") -opts=( - "--max_distributed_connections=1" - "--optimize_skip_unused_shards=1" - "--optimize_distributed_group_by_sharding_key=1" - "--prefer_localhost_replica=0" -) -$CLICKHOUSE_CLIENT "${opts[@]}" --format CSV -nm < Date: Sat, 6 Jul 2024 11:17:21 +0200 Subject: [PATCH 255/417] add note on known issues --- .../functions/string-functions.md | 22 +++++++++++++++++-- 1 file changed, 20 insertions(+), 2 deletions(-) diff --git a/docs/en/sql-reference/functions/string-functions.md b/docs/en/sql-reference/functions/string-functions.md index d451f763e18..6734630cd10 100644 --- a/docs/en/sql-reference/functions/string-functions.md +++ b/docs/en/sql-reference/functions/string-functions.md @@ -568,7 +568,7 @@ SELECT upper('clickhouse'); Converts a string to lowercase, assuming that the string contains valid UTF-8 encoded text. If this assumption is violated, no exception is thrown and the result is undefined. :::note -Does not detect the language, e.g. for Turkish the result might not be exactly correct (i/İ vs. i/I). If the length of the UTF-8 byte sequence is different for upper and lower case of a code point, the result may be incorrect for this code point. +Does not detect the language, e.g. for Turkish the result might not be exactly correct (i/İ vs. i/I). If the length of the UTF-8 byte sequence is different for upper and lower case of a code point (such as `ẞ` and `ß`), the result may be incorrect for this code point. ::: **Syntax** @@ -606,7 +606,7 @@ Result: Converts a string to uppercase, assuming that the string contains valid UTF-8 encoded text. If this assumption is violated, no exception is thrown and the result is undefined. :::note -Does not detect the language, e.g. for Turkish the result might not be exactly correct (i/İ vs. i/I). If the length of the UTF-8 byte sequence is different for upper and lower case of a code point, the result may be incorrect for this code point. +Does not detect the language, e.g. for Turkish the result might not be exactly correct (i/İ vs. i/I). If the length of the UTF-8 byte sequence is different for upper and lower case of a code point (such as `ẞ` and `ß`), the result may be incorrect for this code point. ::: **Syntax** @@ -2336,6 +2336,24 @@ Result: Convert the first letter of each word to upper case and the rest to lower case. Words are sequences of alphanumeric characters separated by non-alphanumeric characters. +:::note +Because `initCap` converts only the first letter of each word to upper case you may observe unexpected behaviour for words containing apostrophes or capital letters. For example: + +```sql +SELECT initCap('mother''s daughter'), initCap('joe McAdam'); +``` + +will return + +```response +┌─initCap('mother\'s daughter')─┬─initCap('joe McAdam')─┐ +│ Mother'S Daughter │ Joe Mcadam │ +└───────────────────────────────┴───────────────────────┘ +``` + +This is a known behaviour, with no plans currently to fix it. +::: + **Syntax** ```sql From 764f9244470414696f4de968edf0d64cb95f2146 Mon Sep 17 00:00:00 2001 From: Blargian Date: Sat, 6 Jul 2024 12:40:58 +0200 Subject: [PATCH 256/417] add mapPartialSort and mapPartialReverseSor functions t documentation --- .../functions/tuple-map-functions.md | 68 ++++++++++++++++++- 1 file changed, 66 insertions(+), 2 deletions(-) diff --git a/docs/en/sql-reference/functions/tuple-map-functions.md b/docs/en/sql-reference/functions/tuple-map-functions.md index ad40725d680..24b356eca87 100644 --- a/docs/en/sql-reference/functions/tuple-map-functions.md +++ b/docs/en/sql-reference/functions/tuple-map-functions.md @@ -600,7 +600,7 @@ mapApply(func, map) **Arguments** -- `func` - [Lambda function](../../sql-reference/functions/index.md#higher-order-functions---operator-and-lambdaparams-expr-function). +- `func` — [Lambda function](../../sql-reference/functions/index.md#higher-order-functions---operator-and-lambdaparams-expr-function). - `map` — [Map](../data-types/map.md). **Returned value** @@ -831,7 +831,39 @@ SELECT mapSort((k, v) -> v, map('key2', 2, 'key3', 1, 'key1', 3)) AS map; └──────────────────────────────┘ ``` -For more details see the [reference](../../sql-reference/functions/array-functions.md#array_functions-sort) for `arraySort` function. +For more details see the [reference](../../sql-reference/functions/array-functions.md#array_functions-sort) for `arraySort` function. + +## mapPartialSort + +Sorts the elements of a map in ascending order with additional `limit` argument allowing partial sorting. +If the `func` function is specified, the sorting order is determined by the result of the `func` function applied to the keys and values of the map. + +**Syntax** + +```sql +mapPartialSort([func,] limit, map) +``` +**Arguments** + +- `func` – Optional function to apply to the keys and values of the map. [Lambda function](../../sql-reference/functions/index.md#higher-order-functions---operator-and-lambdaparams-expr-function). +- `limit` – Elements in range [1..limit] are sorted. [(U)Int](../data-types/int-uint.md). +- `map` – Map to sort. [Map](../data-types/map.md). + +**Returned value** + +- Partially sorted map. [Map](../data-types/map.md). + +**Example** + +``` sql +SELECT mapPartialSort((k, v) -> v, 2, map('k1', 3, 'k2', 1, 'k3', 2)); +``` + +``` text +┌─mapPartialSort(lambda(tuple(k, v), v), 2, map('k1', 3, 'k2', 1, 'k3', 2))─┐ +│ {'k2':1,'k3':2,'k1':3} │ +└───────────────────────────────────────────────────────────────────────────┘ +``` ## mapReverseSort(\[func,\], map) @@ -861,3 +893,35 @@ SELECT mapReverseSort((k, v) -> v, map('key2', 2, 'key3', 1, 'key1', 3)) AS map; ``` For more details see function [arrayReverseSort](../../sql-reference/functions/array-functions.md#array_functions-reverse-sort). + +## mapPartialReverseSort + +Sorts the elements of a map in descending order with additional `limit` argument allowing partial sorting. +If the `func` function is specified, the sorting order is determined by the result of the `func` function applied to the keys and values of the map. + +**Syntax** + +```sql +mapPartialReverseSort([func,] limit, map) +``` +**Arguments** + +- `func` – Optional function to apply to the keys and values of the map. [Lambda function](../../sql-reference/functions/index.md#higher-order-functions---operator-and-lambdaparams-expr-function). +- `limit` – Elements in range [1..limit] are sorted. [(U)Int](../data-types/int-uint.md). +- `map` – Map to sort. [Map](../data-types/map.md). + +**Returned value** + +- Partially sorted map. [Map](../data-types/map.md). + +**Example** + +``` sql +SELECT mapPartialReverseSort((k, v) -> v, 2, map('k1', 3, 'k2', 1, 'k3', 2)); +``` + +``` text +┌─mapPartialReverseSort(lambda(tuple(k, v), v), 2, map('k1', 3, 'k2', 1, 'k3', 2))─┐ +│ {'k1':3,'k3':2,'k2':1} │ +└──────────────────────────────────────────────────────────────────────────────────┘ +``` \ No newline at end of file From ce785c38d65d3392b1dc91e056b8f669d48fc639 Mon Sep 17 00:00:00 2001 From: Shaun Struwig <41984034+Blargian@users.noreply.github.com> Date: Sat, 6 Jul 2024 12:46:23 +0200 Subject: [PATCH 257/417] Update aspell-dict.txt --- utils/check-style/aspell-ignore/en/aspell-dict.txt | 2 ++ 1 file changed, 2 insertions(+) diff --git a/utils/check-style/aspell-ignore/en/aspell-dict.txt b/utils/check-style/aspell-ignore/en/aspell-dict.txt index 229eccefa48..d47bbd86347 100644 --- a/utils/check-style/aspell-ignore/en/aspell-dict.txt +++ b/utils/check-style/aspell-ignore/en/aspell-dict.txt @@ -1102,6 +1102,8 @@ aggregatefunction aggregatingmergetree aggregatio aggretate +aggthrow +aggThrow aiochclient allocator alphaTokens From d6bf7ca15573ee43a05bab98f40d271d11400c0b Mon Sep 17 00:00:00 2001 From: Shaun Struwig <41984034+Blargian@users.noreply.github.com> Date: Sat, 6 Jul 2024 12:58:21 +0200 Subject: [PATCH 258/417] remove space from aspell-dict.txt --- utils/check-style/aspell-ignore/en/aspell-dict.txt | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/utils/check-style/aspell-ignore/en/aspell-dict.txt b/utils/check-style/aspell-ignore/en/aspell-dict.txt index d47bbd86347..e62dd57db76 100644 --- a/utils/check-style/aspell-ignore/en/aspell-dict.txt +++ b/utils/check-style/aspell-ignore/en/aspell-dict.txt @@ -1102,7 +1102,7 @@ aggregatefunction aggregatingmergetree aggregatio aggretate -aggthrow +aggthrow aggThrow aiochclient allocator From 3a8747992633f64beb0962ecc91ed0a9e650e627 Mon Sep 17 00:00:00 2001 From: Sema Checherinda Date: Sat, 6 Jul 2024 16:06:36 +0200 Subject: [PATCH 259/417] fix a crash --- src/Interpreters/DatabaseCatalog.cpp | 10 +++++++--- 1 file changed, 7 insertions(+), 3 deletions(-) diff --git a/src/Interpreters/DatabaseCatalog.cpp b/src/Interpreters/DatabaseCatalog.cpp index 0a71a842452..230deeab8d5 100644 --- a/src/Interpreters/DatabaseCatalog.cpp +++ b/src/Interpreters/DatabaseCatalog.cpp @@ -1250,6 +1250,8 @@ DatabaseCatalog::TablesMarkedAsDropped DatabaseCatalog::getTablesToDrop() std::lock_guard lock(tables_marked_dropped_mutex); + const auto was_count = tables_marked_dropped.size(); + auto it = tables_marked_dropped.begin(); while (it != tables_marked_dropped.end()) { @@ -1270,6 +1272,8 @@ DatabaseCatalog::TablesMarkedAsDropped DatabaseCatalog::getTablesToDrop() it = tables_marked_dropped.erase(it); } + chassert(was_count == tables_marked_dropped.size() + result.size()); + return result; } @@ -1299,10 +1303,10 @@ void DatabaseCatalog::dropTablesParallel(TablesMarkedAsDropped tables_to_drop) SCOPE_EXIT({ std::lock_guard lock(tables_marked_dropped_mutex); - if (first_async_drop_in_queue == tables_marked_dropped.end()) - first_async_drop_in_queue = tables_to_drop.begin(); - tables_marked_dropped.splice(tables_marked_dropped.end(), tables_to_drop); + + if (first_async_drop_in_queue == tables_marked_dropped.end()) + first_async_drop_in_queue = tables_marked_dropped.begin(); }); ThreadPool pool( From 9dc52217f44580f694e0d6f4460e83be9401ea23 Mon Sep 17 00:00:00 2001 From: vdimir Date: Sat, 6 Jul 2024 15:07:35 +0000 Subject: [PATCH 260/417] Fix SimpleSquashingTransform --- .../Transforms/AggregatingTransform.cpp | 2 +- .../Transforms/SquashingTransform.cpp | 62 +++++++------------ .../Transforms/SquashingTransform.h | 19 +++--- src/Storages/buildQueryTreeForShard.cpp | 2 +- 4 files changed, 35 insertions(+), 50 deletions(-) diff --git a/src/Processors/Transforms/AggregatingTransform.cpp b/src/Processors/Transforms/AggregatingTransform.cpp index 65f0612d738..cdbe194cfac 100644 --- a/src/Processors/Transforms/AggregatingTransform.cpp +++ b/src/Processors/Transforms/AggregatingTransform.cpp @@ -783,7 +783,7 @@ void AggregatingTransform::initGenerate() { /// Just a reasonable constant, matches default value for the setting `preferred_block_size_bytes` static constexpr size_t oneMB = 1024 * 1024; - return std::make_shared(header, params->params.max_block_size, oneMB); + return std::make_shared(header, params->params.max_block_size, oneMB); }); } /// AggregatingTransform::expandPipeline expects single output port. diff --git a/src/Processors/Transforms/SquashingTransform.cpp b/src/Processors/Transforms/SquashingTransform.cpp index 34b733cde5e..624b41a6d98 100644 --- a/src/Processors/Transforms/SquashingTransform.cpp +++ b/src/Processors/Transforms/SquashingTransform.cpp @@ -56,53 +56,37 @@ void SquashingTransform::work() } } -SimpleSquashingTransform::SimpleSquashingTransform( +SimpleSquashingChunksTransform::SimpleSquashingChunksTransform( const Block & header, size_t min_block_size_rows, size_t min_block_size_bytes) - : ISimpleTransform(header, header, false) - , squashing(header, min_block_size_rows, min_block_size_bytes) + : IInflatingTransform(header, header), squashing(min_block_size_rows, min_block_size_bytes) { } -void SimpleSquashingTransform::transform(Chunk & chunk) +void SimpleSquashingChunksTransform::consume(Chunk chunk) { - if (!finished) - { - Chunk planned_chunk = squashing.add(std::move(chunk)); - if (planned_chunk.hasChunkInfo()) - chunk = DB::Squashing::squash(std::move(planned_chunk)); - } - else - { - if (chunk.hasRows()) - throw Exception(ErrorCodes::LOGICAL_ERROR, "Chunk expected to be empty, otherwise it will be lost"); - - chunk = squashing.flush(); - if (chunk.hasChunkInfo()) - chunk = DB::Squashing::squash(std::move(chunk)); - } + Block current_block = squashing.add(getInputPort().getHeader().cloneWithColumns(chunk.detachColumns())); + squashed_chunk.setColumns(current_block.getColumns(), current_block.rows()); } -IProcessor::Status SimpleSquashingTransform::prepare() +Chunk SimpleSquashingChunksTransform::generate() { - if (!finished && input.isFinished()) - { - if (output.isFinished()) - return Status::Finished; + if (squashed_chunk.empty()) + throw Exception(ErrorCodes::LOGICAL_ERROR, "Can't generate chunk in SimpleSquashingChunksTransform"); - if (!output.canPush()) - return Status::PortFull; - - if (has_output) - { - output.pushData(std::move(output_data)); - has_output = false; - return Status::PortFull; - } - - finished = true; - /// On the next call to transform() we will return all data buffered in `squashing` (if any) - return Status::Ready; - } - return ISimpleTransform::prepare(); + return std::move(squashed_chunk); } + +bool SimpleSquashingChunksTransform::canGenerate() +{ + return !squashed_chunk.empty(); +} + +Chunk SimpleSquashingChunksTransform::getRemaining() +{ + Block current_block = squashing.add({}); + squashed_chunk.setColumns(current_block.getColumns(), current_block.rows()); + return std::move(squashed_chunk); +} + + } diff --git a/src/Processors/Transforms/SquashingTransform.h b/src/Processors/Transforms/SquashingTransform.h index c5b727ac6ec..8b09722ebbd 100644 --- a/src/Processors/Transforms/SquashingTransform.h +++ b/src/Processors/Transforms/SquashingTransform.h @@ -2,6 +2,7 @@ #include #include +#include #include #include @@ -29,22 +30,22 @@ private: Chunk finish_chunk; }; -/// Doesn't care about propagating exceptions and thus doesn't throw LOGICAL_ERROR if the following transform closes its input port. -class SimpleSquashingTransform : public ISimpleTransform +class SimpleSquashingChunksTransform : public IInflatingTransform { public: - explicit SimpleSquashingTransform(const Block & header, size_t min_block_size_rows, size_t min_block_size_bytes); + explicit SimpleSquashingChunksTransform(const Block & header, size_t min_block_size_rows, size_t min_block_size_bytes); String getName() const override { return "SimpleSquashingTransform"; } protected: - void transform(Chunk &) override; - - IProcessor::Status prepare() override; + void consume(Chunk chunk) override; + bool canGenerate() override; + Chunk generate() override; + Chunk getRemaining() override; private: - Squashing squashing; - - bool finished = false; + SquashingTransform squashing; + Chunk squashed_chunk; }; + } diff --git a/src/Storages/buildQueryTreeForShard.cpp b/src/Storages/buildQueryTreeForShard.cpp index ed378169381..84ba92bba00 100644 --- a/src/Storages/buildQueryTreeForShard.cpp +++ b/src/Storages/buildQueryTreeForShard.cpp @@ -290,7 +290,7 @@ TableNodePtr executeSubqueryNode(const QueryTreeNodePtr & subquery_node, size_t min_block_size_rows = mutable_context->getSettingsRef().min_external_table_block_size_rows; size_t min_block_size_bytes = mutable_context->getSettingsRef().min_external_table_block_size_bytes; - auto squashing = std::make_shared(builder->getHeader(), min_block_size_rows, min_block_size_bytes); + auto squashing = std::make_shared(builder->getHeader(), min_block_size_rows, min_block_size_bytes); builder->resize(1); builder->addTransform(std::move(squashing)); From f1f5dfc83ac3fb1ac2d96cd184283557a327709e Mon Sep 17 00:00:00 2001 From: vdimir Date: Sat, 6 Jul 2024 16:02:01 +0000 Subject: [PATCH 261/417] fix SimpleSquashingChunksTransform --- src/Processors/Transforms/SquashingTransform.cpp | 8 +++----- src/Processors/Transforms/SquashingTransform.h | 2 +- 2 files changed, 4 insertions(+), 6 deletions(-) diff --git a/src/Processors/Transforms/SquashingTransform.cpp b/src/Processors/Transforms/SquashingTransform.cpp index 624b41a6d98..7b04e4341e0 100644 --- a/src/Processors/Transforms/SquashingTransform.cpp +++ b/src/Processors/Transforms/SquashingTransform.cpp @@ -58,14 +58,13 @@ void SquashingTransform::work() SimpleSquashingChunksTransform::SimpleSquashingChunksTransform( const Block & header, size_t min_block_size_rows, size_t min_block_size_bytes) - : IInflatingTransform(header, header), squashing(min_block_size_rows, min_block_size_bytes) + : IInflatingTransform(header, header), squashing(header, min_block_size_rows, min_block_size_bytes) { } void SimpleSquashingChunksTransform::consume(Chunk chunk) { - Block current_block = squashing.add(getInputPort().getHeader().cloneWithColumns(chunk.detachColumns())); - squashed_chunk.setColumns(current_block.getColumns(), current_block.rows()); + squashed_chunk = squashing.add(std::move(chunk)); } Chunk SimpleSquashingChunksTransform::generate() @@ -83,8 +82,7 @@ bool SimpleSquashingChunksTransform::canGenerate() Chunk SimpleSquashingChunksTransform::getRemaining() { - Block current_block = squashing.add({}); - squashed_chunk.setColumns(current_block.getColumns(), current_block.rows()); + squashed_chunk = squashing.flush(); return std::move(squashed_chunk); } diff --git a/src/Processors/Transforms/SquashingTransform.h b/src/Processors/Transforms/SquashingTransform.h index 8b09722ebbd..092f58f2fe0 100644 --- a/src/Processors/Transforms/SquashingTransform.h +++ b/src/Processors/Transforms/SquashingTransform.h @@ -44,7 +44,7 @@ protected: Chunk getRemaining() override; private: - SquashingTransform squashing; + Squashing squashing; Chunk squashed_chunk; }; From 06734b351ece7166822a47849596483e899cda7e Mon Sep 17 00:00:00 2001 From: vdimir Date: Sat, 6 Jul 2024 16:35:54 +0000 Subject: [PATCH 262/417] Revert "fix SimpleSquashingChunksTransform" This reverts commit f1f5dfc83ac3fb1ac2d96cd184283557a327709e. --- src/Processors/Transforms/SquashingTransform.cpp | 8 +++++--- src/Processors/Transforms/SquashingTransform.h | 2 +- 2 files changed, 6 insertions(+), 4 deletions(-) diff --git a/src/Processors/Transforms/SquashingTransform.cpp b/src/Processors/Transforms/SquashingTransform.cpp index 7b04e4341e0..624b41a6d98 100644 --- a/src/Processors/Transforms/SquashingTransform.cpp +++ b/src/Processors/Transforms/SquashingTransform.cpp @@ -58,13 +58,14 @@ void SquashingTransform::work() SimpleSquashingChunksTransform::SimpleSquashingChunksTransform( const Block & header, size_t min_block_size_rows, size_t min_block_size_bytes) - : IInflatingTransform(header, header), squashing(header, min_block_size_rows, min_block_size_bytes) + : IInflatingTransform(header, header), squashing(min_block_size_rows, min_block_size_bytes) { } void SimpleSquashingChunksTransform::consume(Chunk chunk) { - squashed_chunk = squashing.add(std::move(chunk)); + Block current_block = squashing.add(getInputPort().getHeader().cloneWithColumns(chunk.detachColumns())); + squashed_chunk.setColumns(current_block.getColumns(), current_block.rows()); } Chunk SimpleSquashingChunksTransform::generate() @@ -82,7 +83,8 @@ bool SimpleSquashingChunksTransform::canGenerate() Chunk SimpleSquashingChunksTransform::getRemaining() { - squashed_chunk = squashing.flush(); + Block current_block = squashing.add({}); + squashed_chunk.setColumns(current_block.getColumns(), current_block.rows()); return std::move(squashed_chunk); } diff --git a/src/Processors/Transforms/SquashingTransform.h b/src/Processors/Transforms/SquashingTransform.h index 092f58f2fe0..8b09722ebbd 100644 --- a/src/Processors/Transforms/SquashingTransform.h +++ b/src/Processors/Transforms/SquashingTransform.h @@ -44,7 +44,7 @@ protected: Chunk getRemaining() override; private: - Squashing squashing; + SquashingTransform squashing; Chunk squashed_chunk; }; From c195537a1883e153c32971c5180ad30f9aba402a Mon Sep 17 00:00:00 2001 From: vdimir Date: Sat, 6 Jul 2024 16:41:18 +0000 Subject: [PATCH 263/417] fix SimpleSquashingChunksTransform --- .../Transforms/SquashingTransform.cpp | 134 ++++++++++++++++++ .../Transforms/SquashingTransform.h | 31 +++- 2 files changed, 164 insertions(+), 1 deletion(-) diff --git a/src/Processors/Transforms/SquashingTransform.cpp b/src/Processors/Transforms/SquashingTransform.cpp index 624b41a6d98..b5a40c75c5b 100644 --- a/src/Processors/Transforms/SquashingTransform.cpp +++ b/src/Processors/Transforms/SquashingTransform.cpp @@ -7,6 +7,7 @@ namespace DB namespace ErrorCodes { extern const int LOGICAL_ERROR; +extern const int SIZES_OF_COLUMNS_DOESNT_MATCH; } SquashingTransform::SquashingTransform( @@ -88,5 +89,138 @@ Chunk SimpleSquashingChunksTransform::getRemaining() return std::move(squashed_chunk); } +SquashingLegacy::SquashingLegacy(size_t min_block_size_rows_, size_t min_block_size_bytes_) + : min_block_size_rows(min_block_size_rows_) + , min_block_size_bytes(min_block_size_bytes_) +{ +} + +Block SquashingLegacy::add(Block && input_block) +{ + return addImpl(std::move(input_block)); +} + +Block SquashingLegacy::add(const Block & input_block) +{ + return addImpl(input_block); +} + +/* + * To minimize copying, accept two types of argument: const reference for output + * stream, and rvalue reference for input stream, and decide whether to copy + * inside this function. This allows us not to copy Block unless we absolutely + * have to. + */ +template +Block SquashingLegacy::addImpl(ReferenceType input_block) +{ + /// End of input stream. + if (!input_block) + { + Block to_return; + std::swap(to_return, accumulated_block); + return to_return; + } + + /// Just read block is already enough. + if (isEnoughSize(input_block)) + { + /// If no accumulated data, return just read block. + if (!accumulated_block) + { + return std::move(input_block); + } + + /// Return accumulated data (maybe it has small size) and place new block to accumulated data. + Block to_return = std::move(input_block); + std::swap(to_return, accumulated_block); + return to_return; + } + + /// Accumulated block is already enough. + if (isEnoughSize(accumulated_block)) + { + /// Return accumulated data and place new block to accumulated data. + Block to_return = std::move(input_block); + std::swap(to_return, accumulated_block); + return to_return; + } + + append(std::move(input_block)); + if (isEnoughSize(accumulated_block)) + { + Block to_return; + std::swap(to_return, accumulated_block); + return to_return; + } + + /// Squashed block is not ready. + return {}; +} + + +template +void SquashingLegacy::append(ReferenceType input_block) +{ + if (!accumulated_block) + { + accumulated_block = std::move(input_block); + return; + } + + assert(blocksHaveEqualStructure(input_block, accumulated_block)); + + try + { + for (size_t i = 0, size = accumulated_block.columns(); i < size; ++i) + { + const auto source_column = input_block.getByPosition(i).column; + + auto mutable_column = IColumn::mutate(std::move(accumulated_block.getByPosition(i).column)); + mutable_column->insertRangeFrom(*source_column, 0, source_column->size()); + accumulated_block.getByPosition(i).column = std::move(mutable_column); + } + } + catch (...) + { + /// add() may be called again even after a previous add() threw an exception. + /// Keep accumulated_block in a valid state. + /// Seems ok to discard accumulated data because we're throwing an exception, which the caller will + /// hopefully interpret to mean "this block and all *previous* blocks are potentially lost". + accumulated_block.clear(); + throw; + } +} + + +bool SquashingLegacy::isEnoughSize(const Block & block) +{ + size_t rows = 0; + size_t bytes = 0; + + for (const auto & [column, type, name] : block) + { + if (!column) + throw Exception(ErrorCodes::LOGICAL_ERROR, "Invalid column in block."); + + if (!rows) + rows = column->size(); + else if (rows != column->size()) + throw Exception(ErrorCodes::SIZES_OF_COLUMNS_DOESNT_MATCH, "Sizes of columns doesn't match"); + + bytes += column->byteSize(); + } + + return isEnoughSize(rows, bytes); +} + + +bool SquashingLegacy::isEnoughSize(size_t rows, size_t bytes) const +{ + return (!min_block_size_rows && !min_block_size_bytes) + || (min_block_size_rows && rows >= min_block_size_rows) + || (min_block_size_bytes && bytes >= min_block_size_bytes); +} + } diff --git a/src/Processors/Transforms/SquashingTransform.h b/src/Processors/Transforms/SquashingTransform.h index 8b09722ebbd..452317e7d5e 100644 --- a/src/Processors/Transforms/SquashingTransform.h +++ b/src/Processors/Transforms/SquashingTransform.h @@ -30,6 +30,35 @@ private: Chunk finish_chunk; }; + +class SquashingLegacy +{ +public: + /// Conditions on rows and bytes are OR-ed. If one of them is zero, then corresponding condition is ignored. + SquashingLegacy(size_t min_block_size_rows_, size_t min_block_size_bytes_); + + /** Add next block and possibly returns squashed block. + * At end, you need to pass empty block. As the result for last (empty) block, you will get last Result with ready = true. + */ + Block add(Block && block); + Block add(const Block & block); + +private: + size_t min_block_size_rows; + size_t min_block_size_bytes; + + Block accumulated_block; + + template + Block addImpl(ReferenceType block); + + template + void append(ReferenceType block); + + bool isEnoughSize(const Block & block); + bool isEnoughSize(size_t rows, size_t bytes) const; +}; + class SimpleSquashingChunksTransform : public IInflatingTransform { public: @@ -44,7 +73,7 @@ protected: Chunk getRemaining() override; private: - SquashingTransform squashing; + SquashingLegacy squashing; Chunk squashed_chunk; }; From 7f2bfa221e552957d45cc19b4137bf1d4638ee24 Mon Sep 17 00:00:00 2001 From: Blargian Date: Sat, 6 Jul 2024 19:37:00 +0200 Subject: [PATCH 264/417] Update formatting of maxMap and minMap and add missing aliases --- .../aggregate-functions/reference/maxmap.md | 34 +++++++++++++++---- .../aggregate-functions/reference/minmap.md | 34 +++++++++++++++---- 2 files changed, 56 insertions(+), 12 deletions(-) diff --git a/docs/en/sql-reference/aggregate-functions/reference/maxmap.md b/docs/en/sql-reference/aggregate-functions/reference/maxmap.md index c9c6913249c..73075c0823d 100644 --- a/docs/en/sql-reference/aggregate-functions/reference/maxmap.md +++ b/docs/en/sql-reference/aggregate-functions/reference/maxmap.md @@ -5,23 +5,45 @@ sidebar_position: 165 # maxMap -Syntax: `maxMap(key, value)` or `maxMap(Tuple(key, value))` - Calculates the maximum from `value` array according to the keys specified in the `key` array. -Passing a tuple of keys and value arrays is identical to passing two arrays of keys and values. +**Syntax** -The number of elements in `key` and `value` must be the same for each row that is totaled. +```sql +maxMap(key, value) +``` +or +```sql +maxMap(Tuple(key, value)) +``` -Returns a tuple of two arrays: keys and values calculated for the corresponding keys. +Alias: `maxMappedArrays` -Example: +:::note +- Passing a tuple of keys and value arrays is identical to passing two arrays of keys and values. +- The number of elements in `key` and `value` must be the same for each row that is totaled. +::: + +**Parameters** + +- `key` — Array of keys. [Array](../../data-types/array.md). +- `value` — Array of values. [Array](../../data-types/array.md). + +**Returned value** + +- Returns a tuple of two arrays: keys in sorted order, and values calculated for the corresponding keys. [Tuple](../../data-types/tuple.md)([Array](../../data-types/array.md), [Array](../../data-types/array.md)). + +**Example** + +Query: ``` sql SELECT maxMap(a, b) FROM values('a Array(Char), b Array(Int64)', (['x', 'y'], [2, 2]), (['y', 'z'], [3, 1])) ``` +Result: + ``` text ┌─maxMap(a, b)───────────┐ │ [['x','y','z'],[2,3,1]]│ diff --git a/docs/en/sql-reference/aggregate-functions/reference/minmap.md b/docs/en/sql-reference/aggregate-functions/reference/minmap.md index b1fbb9e49f3..c0f340b3f3f 100644 --- a/docs/en/sql-reference/aggregate-functions/reference/minmap.md +++ b/docs/en/sql-reference/aggregate-functions/reference/minmap.md @@ -5,23 +5,45 @@ sidebar_position: 169 # minMap -Syntax: `minMap(key, value)` or `minMap(Tuple(key, value))` - Calculates the minimum from `value` array according to the keys specified in the `key` array. -Passing a tuple of keys and value ​​arrays is identical to passing two arrays of keys and values. +**Syntax** -The number of elements in `key` and `value` must be the same for each row that is totaled. +```sql +`minMap(key, value)` +``` +or +```sql +minMap(Tuple(key, value)) +``` -Returns a tuple of two arrays: keys in sorted order, and values calculated for the corresponding keys. +Alias: `minMappedArrays` -Example: +:::note +- Passing a tuple of keys and value arrays is identical to passing an array of keys and an array of values. +- The number of elements in `key` and `value` must be the same for each row that is totaled. +::: + +**Parameters** + +- `key` — Array of keys. [Array](../../data-types/array.md). +- `value` — Array of values. [Array](../../data-types/array.md). + +**Returned value** + +- Returns a tuple of two arrays: keys in sorted order, and values calculated for the corresponding keys. [Tuple](../../data-types/tuple.md)([Array](../../data-types/array.md), [Array](../../data-types/array.md)). + +**Example** + +Query: ``` sql SELECT minMap(a, b) FROM values('a Array(Int32), b Array(Int64)', ([1, 2], [2, 2]), ([2, 3], [1, 1])) ``` +Result: + ``` text ┌─minMap(a, b)──────┐ │ ([1,2,3],[2,1,1]) │ From 3eb2de3d856a3e67eafde36fa34388242e1cbc2c Mon Sep 17 00:00:00 2001 From: Shaun Struwig <41984034+Blargian@users.noreply.github.com> Date: Sat, 6 Jul 2024 19:41:58 +0200 Subject: [PATCH 265/417] Update aspell-dict.txt --- utils/check-style/aspell-ignore/en/aspell-dict.txt | 2 ++ 1 file changed, 2 insertions(+) diff --git a/utils/check-style/aspell-ignore/en/aspell-dict.txt b/utils/check-style/aspell-ignore/en/aspell-dict.txt index 229eccefa48..eb19b3d2871 100644 --- a/utils/check-style/aspell-ignore/en/aspell-dict.txt +++ b/utils/check-style/aspell-ignore/en/aspell-dict.txt @@ -1996,6 +1996,7 @@ maxMap maxintersections maxintersectionsposition maxmap +minMappedArrays maxmind mdadm meanZTest @@ -2013,6 +2014,7 @@ metrica metroHash mfedotov minMap +minMappedArrays minSampleSizeContinuous minSampleSizeConversion mindsdb From 262972c0ee47f6782e8e073047f4e8a6f6db3229 Mon Sep 17 00:00:00 2001 From: Blargian Date: Sat, 6 Jul 2024 20:13:37 +0200 Subject: [PATCH 266/417] add makeDate32 and makeDateTime64 --- .../functions/date-time-functions.md | 82 ++++++++++++++++++- 1 file changed, 79 insertions(+), 3 deletions(-) diff --git a/docs/en/sql-reference/functions/date-time-functions.md b/docs/en/sql-reference/functions/date-time-functions.md index b532e0de8f0..46b1167fa33 100644 --- a/docs/en/sql-reference/functions/date-time-functions.md +++ b/docs/en/sql-reference/functions/date-time-functions.md @@ -83,7 +83,57 @@ Result: ``` ## makeDate32 -Like [makeDate](#makedate) but produces a [Date32](../data-types/date32.md). +Creates a date of type [Date32](../../sql-reference/data-types/date32.md) from a year, month, day (or optionally a year and a day). + +**Syntax** + +```sql +makeDate32(year, [month,] day) +``` + +**Arguments** + +- `year` — Year. [Integer](../../sql-reference/data-types/int-uint.md), [Float](../../sql-reference/data-types/float.md) or [Decimal](../../sql-reference/data-types/decimal.md). +- `month` — Month (optional). [Integer](../../sql-reference/data-types/int-uint.md), [Float](../../sql-reference/data-types/float.md) or [Decimal](../../sql-reference/data-types/decimal.md). +- `day` — Day. [Integer](../../sql-reference/data-types/int-uint.md), [Float](../../sql-reference/data-types/float.md) or [Decimal](../../sql-reference/data-types/decimal.md). + +:::note +If `month` is omitted then `day` should take a value between `1` and `365`, otherwise it should take a value between `1` and `31`. +::: + +**Returned values** + +- A date created from the arguments. [Date32](../../sql-reference/data-types/date32.md). + +**Examples** + +Create a date from a year, month, and day: + +Query: + +```sql +SELECT makeDate32(2024, 1, 1); +``` + +Result: + +```response +2024-01-01 +``` + +Create a Date from a year and day of year: + +Query: + +``` sql +SELECT makeDate32(2024, 100); +``` + +Result: + +```response +2024-04-09 +``` ## makeDateTime @@ -125,12 +175,38 @@ Result: ## makeDateTime64 -Like [makeDateTime](#makedatetime) but produces a [DateTime64](../data-types/datetime64.md). +Creates a [DateTime64](../../sql-reference/data-types/datetime64.md) data type value from its components: year, month, day, hour, minute, second. With optional sub-second precision. **Syntax** +```sql +makeDateTime64(year, month, day, hour, minute, second[, precision]) +``` + +**Arguments** + +- `year` — Year (0-9999). [Integer](../../sql-reference/data-types/int-uint.md), [Float](../../sql-reference/data-types/float.md) or [Decimal](../../sql-reference/data-types/decimal.md). +- `month` — Month (1-12). [Integer](../../sql-reference/data-types/int-uint.md), [Float](../../sql-reference/data-types/float.md) or [Decimal](../../sql-reference/data-types/decimal.md). +- `day` — Day (1-31). [Integer](../../sql-reference/data-types/int-uint.md), [Float](../../sql-reference/data-types/float.md) or [Decimal](../../sql-reference/data-types/decimal.md). +- `hour` — Hour (0-23). [Integer](../../sql-reference/data-types/int-uint.md), [Float](../../sql-reference/data-types/float.md) or [Decimal](../../sql-reference/data-types/decimal.md). +- `minute` — Minute (0-59). [Integer](../../sql-reference/data-types/int-uint.md), [Float](../../sql-reference/data-types/float.md) or [Decimal](../../sql-reference/data-types/decimal.md). +- `second` — Second (0-59). [Integer](../../sql-reference/data-types/int-uint.md), [Float](../../sql-reference/data-types/float.md) or [Decimal](../../sql-reference/data-types/decimal.md). +- `precision` — Optional precision of the sub-second component (0-9). [Integer](../../sql-reference/data-types/int-uint.md). + +**Returned value** + +- A date and time created from the supplied arguments. [DateTime64](../../sql-reference/data-types/datetime64.md). + +**Example** + ``` sql -makeDateTime64(year, month, day, hour, minute, second[, fraction[, precision[, timezone]]]) +SELECT makeDateTime64(2023, 5, 15, 10, 30, 45, 779, 5); +``` + +```response +┌─makeDateTime64(2023, 5, 15, 10, 30, 45, 779, 5)─┐ +│ 2023-05-15 10:30:45.00779 │ +└─────────────────────────────────────────────────┘ ``` ## timestamp From bbe72326ba567f67f57a7f3d2b2a6535691ab033 Mon Sep 17 00:00:00 2001 From: Blargian Date: Sat, 6 Jul 2024 20:14:23 +0200 Subject: [PATCH 267/417] update aspell-dict.txt --- utils/check-style/aspell-ignore/en/aspell-dict.txt | 2 ++ 1 file changed, 2 insertions(+) diff --git a/utils/check-style/aspell-ignore/en/aspell-dict.txt b/utils/check-style/aspell-ignore/en/aspell-dict.txt index 229eccefa48..43dcbed5c4c 100644 --- a/utils/check-style/aspell-ignore/en/aspell-dict.txt +++ b/utils/check-style/aspell-ignore/en/aspell-dict.txt @@ -1966,7 +1966,9 @@ macOS mailrugo mailto makeDate +makeDate32 makeDateTime +makeDateTime64 mannWhitneyUTest mannwhitneyutest mapAdd From 0f8b72af25fa8f35c4292d942942409c736ade6c Mon Sep 17 00:00:00 2001 From: Shaun Struwig <41984034+Blargian@users.noreply.github.com> Date: Sat, 6 Jul 2024 20:44:45 +0200 Subject: [PATCH 268/417] Update aspell-dict.txt --- utils/check-style/aspell-ignore/en/aspell-dict.txt | 2 ++ 1 file changed, 2 insertions(+) diff --git a/utils/check-style/aspell-ignore/en/aspell-dict.txt b/utils/check-style/aspell-ignore/en/aspell-dict.txt index 229eccefa48..40cdd04bd27 100644 --- a/utils/check-style/aspell-ignore/en/aspell-dict.txt +++ b/utils/check-style/aspell-ignore/en/aspell-dict.txt @@ -1980,6 +1980,8 @@ mapExtractKeyLike mapFilter mapFromArrays mapKeys +mapPartialReverseSort +mapPartialSort mapPopulateSeries mapReverseSort mapSort From d70d93f362ebceae4d3c981a84af6974c50c3f0b Mon Sep 17 00:00:00 2001 From: Shaun Struwig <41984034+Blargian@users.noreply.github.com> Date: Sat, 6 Jul 2024 20:45:57 +0200 Subject: [PATCH 269/417] Update aspell-dict.txt From 912218d9fc7fcf12a3ac86ed3d5402d94080d3fd Mon Sep 17 00:00:00 2001 From: Shaun Struwig <41984034+Blargian@users.noreply.github.com> Date: Sat, 6 Jul 2024 20:46:18 +0200 Subject: [PATCH 270/417] Update aspell-dict.txt From 458d4ca8628b82a98a2165c1a86433af0fa6a9e6 Mon Sep 17 00:00:00 2001 From: Shaun Struwig <41984034+Blargian@users.noreply.github.com> Date: Sat, 6 Jul 2024 20:56:47 +0200 Subject: [PATCH 271/417] Update aspell-dict.txt --- utils/check-style/aspell-ignore/en/aspell-dict.txt | 1 - 1 file changed, 1 deletion(-) diff --git a/utils/check-style/aspell-ignore/en/aspell-dict.txt b/utils/check-style/aspell-ignore/en/aspell-dict.txt index 43dcbed5c4c..b5417719c31 100644 --- a/utils/check-style/aspell-ignore/en/aspell-dict.txt +++ b/utils/check-style/aspell-ignore/en/aspell-dict.txt @@ -1966,7 +1966,6 @@ macOS mailrugo mailto makeDate -makeDate32 makeDateTime makeDateTime64 mannWhitneyUTest From aea4734edd78eadccc5b30db6d09369a3ede708f Mon Sep 17 00:00:00 2001 From: Shaun Struwig <41984034+Blargian@users.noreply.github.com> Date: Sat, 6 Jul 2024 21:28:16 +0200 Subject: [PATCH 272/417] Update aspell-dict.txt --- utils/check-style/aspell-ignore/en/aspell-dict.txt | 1 - 1 file changed, 1 deletion(-) diff --git a/utils/check-style/aspell-ignore/en/aspell-dict.txt b/utils/check-style/aspell-ignore/en/aspell-dict.txt index b5417719c31..229eccefa48 100644 --- a/utils/check-style/aspell-ignore/en/aspell-dict.txt +++ b/utils/check-style/aspell-ignore/en/aspell-dict.txt @@ -1967,7 +1967,6 @@ mailrugo mailto makeDate makeDateTime -makeDateTime64 mannWhitneyUTest mannwhitneyutest mapAdd From 84f876f0983ab425f162d0b69092770484392abc Mon Sep 17 00:00:00 2001 From: avogar Date: Sat, 6 Jul 2024 20:35:43 +0000 Subject: [PATCH 273/417] Support null map subcolumn for Variant and Dynamic subcolumns --- src/DataTypes/DataTypeDynamic.cpp | 46 +- src/DataTypes/IDataType.cpp | 2 +- .../Serializations/ISerialization.cpp | 8 +- src/DataTypes/Serializations/ISerialization.h | 14 +- .../SerializationDynamicElement.cpp | 21 +- .../SerializationDynamicElement.h | 6 +- .../Serializations/SerializationVariant.cpp | 29 +- .../Serializations/SerializationVariant.h | 2 + .../SerializationVariantElement.cpp | 31 +- .../SerializationVariantElement.h | 12 +- .../SerializationVariantElementNullMap.cpp | 190 +++ .../SerializationVariantElementNullMap.h | 107 ++ .../02941_variant_type_1.reference | 1380 ++++++++--------- .../02941_variant_type_2.reference | 15 - .../0_stateless/02941_variant_type_2.sh | 5 +- .../02941_variant_type_3.reference | 15 - .../0_stateless/02941_variant_type_3.sh | 5 +- .../02941_variant_type_4.reference | 15 - .../0_stateless/02941_variant_type_4.sh | 5 +- .../03040_dynamic_type_alters_1.reference | 858 +++++----- .../03040_dynamic_type_alters_2.reference | 276 ++-- .../03041_dynamic_type_check_table.reference | 72 +- ...03201_variant_null_map_subcolumn.reference | 402 +++++ .../03201_variant_null_map_subcolumn.sh | 44 + ...03202_dynamic_null_map_subcolumn.reference | 57 + .../03202_dynamic_null_map_subcolumn.sh | 62 + 26 files changed, 2299 insertions(+), 1380 deletions(-) create mode 100644 src/DataTypes/Serializations/SerializationVariantElementNullMap.cpp create mode 100644 src/DataTypes/Serializations/SerializationVariantElementNullMap.h create mode 100644 tests/queries/0_stateless/03201_variant_null_map_subcolumn.reference create mode 100755 tests/queries/0_stateless/03201_variant_null_map_subcolumn.sh create mode 100644 tests/queries/0_stateless/03202_dynamic_null_map_subcolumn.reference create mode 100755 tests/queries/0_stateless/03202_dynamic_null_map_subcolumn.sh diff --git a/src/DataTypes/DataTypeDynamic.cpp b/src/DataTypes/DataTypeDynamic.cpp index c920e69c13b..5302cdb18f9 100644 --- a/src/DataTypes/DataTypeDynamic.cpp +++ b/src/DataTypes/DataTypeDynamic.cpp @@ -2,9 +2,11 @@ #include #include #include +#include #include #include #include +#include #include #include #include @@ -110,28 +112,58 @@ std::unique_ptr DataTypeDynamic::getDynamicSubcolumnDa } /// Extract nested subcolumn of requested dynamic subcolumn if needed. - if (!subcolumn_nested_name.empty()) + /// If requested subcolumn is null map, it's processed separately as there is no Nullable type yet. + bool is_null_map_subcolumn = subcolumn_nested_name == "null"; + if (is_null_map_subcolumn) + { + res->type = std::make_shared(); + } + else if (!subcolumn_nested_name.empty()) { res = getSubcolumnData(subcolumn_nested_name, *res, throw_if_null); if (!res) return nullptr; } - res->serialization = std::make_shared(res->serialization, subcolumn_type->getName()); - res->type = makeNullableOrLowCardinalityNullableSafe(res->type); + res->serialization = std::make_shared(res->serialization, subcolumn_type->getName(), is_null_map_subcolumn); + /// Make resulting subcolumn Nullable only if type subcolumn can be inside Nullable or can be LowCardinality(Nullable()). + bool make_subcolumn_nullable = subcolumn_type->canBeInsideNullable() || subcolumn_type->lowCardinality(); + if (!is_null_map_subcolumn && make_subcolumn_nullable) + res->type = makeNullableOrLowCardinalityNullableSafe(res->type); + if (data.column) { if (discriminator) { - /// Provided Dynamic column has subcolumn of this type, we should use VariantSubcolumnCreator to + /// Provided Dynamic column has subcolumn of this type, we should use VariantSubcolumnCreator/VariantNullMapSubcolumnCreator to /// create full subcolumn from variant according to discriminators. const auto & variant_column = assert_cast(*data.column).getVariantColumn(); - auto creator = SerializationVariantElement::VariantSubcolumnCreator(variant_column.getLocalDiscriminatorsPtr(), "", *discriminator, variant_column.localDiscriminatorByGlobal(*discriminator)); - res->column = creator.create(res->column); + std::unique_ptr creator; + if (is_null_map_subcolumn) + creator = std::make_unique( + variant_column.getLocalDiscriminatorsPtr(), + "", + *discriminator, + variant_column.localDiscriminatorByGlobal(*discriminator)); + else + creator = std::make_unique( + variant_column.getLocalDiscriminatorsPtr(), + "", + *discriminator, + variant_column.localDiscriminatorByGlobal(*discriminator), + make_subcolumn_nullable); + res->column = creator->create(res->column); + } + /// Provided Dynamic column doesn't have subcolumn of this type, just create column filled with default values. + else if (is_null_map_subcolumn) + { + /// Fill null map with 1 when there is no such Dynamic subcolumn. + auto column = ColumnUInt8::create(); + assert_cast(*column).getData().resize_fill(data.column->size(), 1); + res->column = std::move(column); } else { - /// Provided Dynamic column doesn't have subcolumn of this type, just create column filled with default values. auto column = res->type->createColumn(); column->insertManyDefaults(data.column->size()); res->column = std::move(column); diff --git a/src/DataTypes/IDataType.cpp b/src/DataTypes/IDataType.cpp index 1c9715bbf53..1cb64b65d3a 100644 --- a/src/DataTypes/IDataType.cpp +++ b/src/DataTypes/IDataType.cpp @@ -173,7 +173,7 @@ bool IDataType::hasDynamicSubcolumns() const auto data = SubstreamData(getDefaultSerialization()).withType(getPtr()); auto callback = [&](const SubstreamPath &, const String &, const SubstreamData & subcolumn_data) { - has_dynamic_subcolumns |= subcolumn_data.type->hasDynamicSubcolumnsData(); + has_dynamic_subcolumns |= subcolumn_data.type && subcolumn_data.type->hasDynamicSubcolumnsData(); }; forEachSubcolumn(callback, data); return has_dynamic_subcolumns; diff --git a/src/DataTypes/Serializations/ISerialization.cpp b/src/DataTypes/Serializations/ISerialization.cpp index bbb1d1a6cd1..7642a6619b3 100644 --- a/src/DataTypes/Serializations/ISerialization.cpp +++ b/src/DataTypes/Serializations/ISerialization.cpp @@ -64,6 +64,9 @@ String ISerialization::Substream::toString() const if (type == VariantElement) return fmt::format("VariantElement({})", variant_element_name); + if (type == VariantElementNullMap) + return fmt::format("VariantElementNullMap({}.null)", variant_element_name); + return String(magic_enum::enum_name(type)); } @@ -195,6 +198,8 @@ String getNameForSubstreamPath( stream_name += ".variant_offsets"; else if (it->type == Substream::VariantElement) stream_name += "." + it->variant_element_name; + else if (it->type == Substream::VariantElementNullMap) + stream_name += "." + it->variant_element_name + ".null"; else if (it->type == SubstreamType::DynamicStructure) stream_name += ".dynamic_structure"; } @@ -395,7 +400,8 @@ bool ISerialization::hasSubcolumnForPath(const SubstreamPath & path, size_t pref return path[last_elem].type == Substream::NullMap || path[last_elem].type == Substream::TupleElement || path[last_elem].type == Substream::ArraySizes - || path[last_elem].type == Substream::VariantElement; + || path[last_elem].type == Substream::VariantElement + || path[last_elem].type == Substream::VariantElementNullMap; } ISerialization::SubstreamData ISerialization::createFromPath(const SubstreamPath & path, size_t prefix_len) diff --git a/src/DataTypes/Serializations/ISerialization.h b/src/DataTypes/Serializations/ISerialization.h index 89e2079490e..6007eca94d4 100644 --- a/src/DataTypes/Serializations/ISerialization.h +++ b/src/DataTypes/Serializations/ISerialization.h @@ -184,6 +184,7 @@ public: VariantOffsets, VariantElements, VariantElement, + VariantElementNullMap, DynamicData, DynamicStructure, @@ -436,6 +437,9 @@ protected: template State * checkAndGetState(const StatePtr & state) const; + template + static State * checkAndGetState(const StatePtr & state, const ISerialization * serialization); + [[noreturn]] void throwUnexpectedDataAfterParsedValue(IColumn & column, ReadBuffer & istr, const FormatSettings &, const String & type_name) const; }; @@ -446,10 +450,16 @@ using SubstreamType = ISerialization::Substream::Type; template State * ISerialization::checkAndGetState(const StatePtr & state) const +{ + return checkAndGetState(state, this); +} + +template +State * ISerialization::checkAndGetState(const StatePtr & state, const ISerialization * serialization) { if (!state) throw Exception(ErrorCodes::LOGICAL_ERROR, - "Got empty state for {}", demangle(typeid(*this).name())); + "Got empty state for {}", demangle(typeid(*serialization).name())); auto * state_concrete = typeid_cast(state.get()); if (!state_concrete) @@ -457,7 +467,7 @@ State * ISerialization::checkAndGetState(const StatePtr & state) const auto & state_ref = *state; throw Exception(ErrorCodes::LOGICAL_ERROR, "Invalid State for {}. Expected: {}, got {}", - demangle(typeid(*this).name()), + demangle(typeid(*serialization).name()), demangle(typeid(State).name()), demangle(typeid(state_ref).name())); } diff --git a/src/DataTypes/Serializations/SerializationDynamicElement.cpp b/src/DataTypes/Serializations/SerializationDynamicElement.cpp index dafd6d663b0..211f0ac9377 100644 --- a/src/DataTypes/Serializations/SerializationDynamicElement.cpp +++ b/src/DataTypes/Serializations/SerializationDynamicElement.cpp @@ -1,5 +1,6 @@ #include #include +#include #include #include #include @@ -77,7 +78,10 @@ void SerializationDynamicElement::deserializeBinaryBulkStatePrefix( if (auto global_discr = assert_cast(*variant_type).tryGetVariantDiscriminator(dynamic_element_name)) { settings.path.push_back(Substream::DynamicData); - dynamic_element_state->variant_serialization = std::make_shared(nested_serialization, dynamic_element_name, *global_discr); + if (is_null_map_subcolumn) + dynamic_element_state->variant_serialization = std::make_shared(dynamic_element_name, *global_discr); + else + dynamic_element_state->variant_serialization = std::make_shared(nested_serialization, dynamic_element_name, *global_discr); dynamic_element_state->variant_serialization->deserializeBinaryBulkStatePrefix(settings, dynamic_element_state->variant_element_state, cache); settings.path.pop_back(); } @@ -98,7 +102,16 @@ void SerializationDynamicElement::deserializeBinaryBulkWithMultipleStreams( SubstreamsCache * cache) const { if (!state) + { + if (is_null_map_subcolumn) + { + auto mutable_column = result_column->assumeMutable(); + auto & data = assert_cast(*mutable_column).getData(); + data.resize_fill(data.size() + limit, 1); + } + return; + } auto * dynamic_element_state = checkAndGetState(state); @@ -108,6 +121,12 @@ void SerializationDynamicElement::deserializeBinaryBulkWithMultipleStreams( dynamic_element_state->variant_serialization->deserializeBinaryBulkWithMultipleStreams(result_column, limit, settings, dynamic_element_state->variant_element_state, cache); settings.path.pop_back(); } + else if (is_null_map_subcolumn) + { + auto mutable_column = result_column->assumeMutable(); + auto & data = assert_cast(*mutable_column).getData(); + data.resize_fill(data.size() + limit, 1); + } else { auto mutable_column = result_column->assumeMutable(); diff --git a/src/DataTypes/Serializations/SerializationDynamicElement.h b/src/DataTypes/Serializations/SerializationDynamicElement.h index 2ddc3324139..127d14a55e0 100644 --- a/src/DataTypes/Serializations/SerializationDynamicElement.h +++ b/src/DataTypes/Serializations/SerializationDynamicElement.h @@ -13,11 +13,11 @@ private: /// To be able to deserialize Dynamic element as a subcolumn /// we need its type name and global discriminator. String dynamic_element_name; + bool is_null_map_subcolumn; public: - SerializationDynamicElement(const SerializationPtr & nested_, const String & dynamic_element_name_) - : SerializationWrapper(nested_) - , dynamic_element_name(dynamic_element_name_) + SerializationDynamicElement(const SerializationPtr & nested_, const String & dynamic_element_name_, bool is_null_map_subcolumn_ = false) + : SerializationWrapper(nested_), dynamic_element_name(dynamic_element_name_), is_null_map_subcolumn(is_null_map_subcolumn_) { } diff --git a/src/DataTypes/Serializations/SerializationVariant.cpp b/src/DataTypes/Serializations/SerializationVariant.cpp index 8cdd312a707..e4d71e84cc7 100644 --- a/src/DataTypes/Serializations/SerializationVariant.cpp +++ b/src/DataTypes/Serializations/SerializationVariant.cpp @@ -1,5 +1,6 @@ #include #include +#include #include #include #include @@ -71,10 +72,16 @@ void SerializationVariant::enumerateStreams( for (size_t i = 0; i < variants.size(); ++i) { - settings.path.back().creator = std::make_shared(local_discriminators, variant_names[i], i, column_variant ? column_variant->localDiscriminatorByGlobal(i) : i); + DataTypePtr type = type_variant ? type_variant->getVariant(i) : nullptr; + settings.path.back().creator = std::make_shared( + local_discriminators, + variant_names[i], + i, + column_variant ? column_variant->localDiscriminatorByGlobal(i) : i, + !type || type->canBeInsideNullable() || type->lowCardinality()); auto variant_data = SubstreamData(variants[i]) - .withType(type_variant ? type_variant->getVariant(i) : nullptr) + .withType(type) .withColumn(column_variant ? column_variant->getVariantPtrByGlobalDiscriminator(i) : nullptr) .withSerializationInfo(data.serialization_info) .withDeserializeState(variant_deserialize_state ? variant_deserialize_state->variant_states[i] : nullptr); @@ -85,6 +92,24 @@ void SerializationVariant::enumerateStreams( settings.path.pop_back(); } + /// Variant subcolumns like variant.Type have type Nullable(Type), so we want to support reading null map subcolumn from it: variant.Type.null. + /// Nullable column is created during deserialization of a variant subcolumn according to the discriminators, so we don't have actual Nullable + /// serialization with null map subcolumn. To be able to read null map subcolumn from the variant subcolumn we use special serialization + /// SerializationVariantElementNullMap. + auto null_map_data = SubstreamData(std::make_shared>()) + .withType(type_variant ? std::make_shared() : nullptr) + .withColumn(column_variant ? ColumnUInt8::create() : nullptr); + + for (size_t i = 0; i < variants.size(); ++i) + { + settings.path.back().creator = std::make_shared(local_discriminators, variant_names[i], i, column_variant ? column_variant->localDiscriminatorByGlobal(i) : i); + settings.path.push_back(Substream::VariantElementNullMap); + settings.path.back().variant_element_name = variant_names[i]; + settings.path.back().data = null_map_data; + callback(settings.path); + settings.path.pop_back(); + } + settings.path.pop_back(); } diff --git a/src/DataTypes/Serializations/SerializationVariant.h b/src/DataTypes/Serializations/SerializationVariant.h index f777ef1203d..af89632cf81 100644 --- a/src/DataTypes/Serializations/SerializationVariant.h +++ b/src/DataTypes/Serializations/SerializationVariant.h @@ -2,6 +2,7 @@ #include #include +#include namespace DB { @@ -161,6 +162,7 @@ public: private: friend SerializationVariantElement; + friend SerializationVariantElementNullMap; void addVariantElementToPath(SubstreamPath & path, size_t i) const; diff --git a/src/DataTypes/Serializations/SerializationVariantElement.cpp b/src/DataTypes/Serializations/SerializationVariantElement.cpp index 46f9194baa9..8ceab17cba4 100644 --- a/src/DataTypes/Serializations/SerializationVariantElement.cpp +++ b/src/DataTypes/Serializations/SerializationVariantElement.cpp @@ -115,7 +115,14 @@ void SerializationVariantElement::deserializeBinaryBulkWithMultipleStreams( if (discriminators_state->mode.value == SerializationVariant::DiscriminatorsSerializationMode::BASIC) SerializationNumber().deserializeBinaryBulk(*variant_element_state->discriminators->assumeMutable(), *discriminators_stream, limit, 0); else - variant_limit = deserializeCompactDiscriminators(variant_element_state->discriminators, limit, discriminators_stream, settings.continuous_reading, *variant_element_state); + variant_limit = deserializeCompactDiscriminators( + variant_element_state->discriminators, + variant_discriminator, + limit, + discriminators_stream, + settings.continuous_reading, + variant_element_state->discriminators_state, + this); addToSubstreamsCache(cache, settings.path, variant_element_state->discriminators); } @@ -224,12 +231,14 @@ void SerializationVariantElement::deserializeBinaryBulkWithMultipleStreams( size_t SerializationVariantElement::deserializeCompactDiscriminators( DB::ColumnPtr & discriminators_column, + ColumnVariant::Discriminator variant_discriminator, size_t limit, DB::ReadBuffer * stream, bool continuous_reading, - DeserializeBinaryBulkStateVariantElement & variant_element_state) const + DeserializeBinaryBulkStatePtr & discriminators_state_, + const ISerialization * serialization) { - auto * discriminators_state = checkAndGetState(variant_element_state.discriminators_state); + auto * discriminators_state = checkAndGetState(discriminators_state_, serialization); auto & discriminators = assert_cast(*discriminators_column->assumeMutable()); auto & discriminators_data = discriminators.getData(); @@ -290,17 +299,19 @@ SerializationVariantElement::VariantSubcolumnCreator::VariantSubcolumnCreator( const ColumnPtr & local_discriminators_, const String & variant_element_name_, ColumnVariant::Discriminator global_variant_discriminator_, - ColumnVariant::Discriminator local_variant_discriminator_) + ColumnVariant::Discriminator local_variant_discriminator_, + bool make_nullable_) : local_discriminators(local_discriminators_) , variant_element_name(variant_element_name_) , global_variant_discriminator(global_variant_discriminator_) , local_variant_discriminator(local_variant_discriminator_) + , make_nullable(make_nullable_) { } DataTypePtr SerializationVariantElement::VariantSubcolumnCreator::create(const DB::DataTypePtr & prev) const { - return makeNullableOrLowCardinalityNullableSafe(prev); + return make_nullable ? makeNullableOrLowCardinalityNullableSafe(prev) : prev; } SerializationPtr SerializationVariantElement::VariantSubcolumnCreator::create(const DB::SerializationPtr & prev) const @@ -313,12 +324,12 @@ ColumnPtr SerializationVariantElement::VariantSubcolumnCreator::create(const DB: /// Case when original Variant column contained only one non-empty variant and no NULLs. /// In this case just use this variant. if (prev->size() == local_discriminators->size()) - return makeNullableOrLowCardinalityNullableSafe(prev); + return make_nullable ? makeNullableOrLowCardinalityNullableSafe(prev) : prev; /// If this variant is empty, fill result column with default values. if (prev->empty()) { - auto res = makeNullableOrLowCardinalityNullableSafe(prev)->cloneEmpty(); + auto res = make_nullable ? makeNullableOrLowCardinalityNullableSafe(prev)->cloneEmpty() : prev->cloneEmpty(); res->insertManyDefaults(local_discriminators->size()); return res; } @@ -333,16 +344,16 @@ ColumnPtr SerializationVariantElement::VariantSubcolumnCreator::create(const DB: /// Now we can create new column from null-map and variant column using IColumn::expand. auto res_column = IColumn::mutate(prev); - /// Special case for LowCardinality. We want the result to be LowCardinality(Nullable), + /// Special case for LowCardinality when we want the result to be LowCardinality(Nullable), /// but we don't have a good way to apply null-mask for LowCardinality(), so, we first /// convert our column to LowCardinality(Nullable()) and then use expand which will /// fill rows with 0 in mask with default value (that is NULL). - if (prev->lowCardinality()) + if (make_nullable && prev->lowCardinality()) res_column = assert_cast(*res_column).cloneNullable(); res_column->expand(null_map, /*inverted = */ true); - if (res_column->canBeInsideNullable()) + if (make_nullable && prev->canBeInsideNullable()) { auto null_map_col = ColumnUInt8::create(); null_map_col->getData() = std::move(null_map); diff --git a/src/DataTypes/Serializations/SerializationVariantElement.h b/src/DataTypes/Serializations/SerializationVariantElement.h index f6d4e069633..69101aea0f5 100644 --- a/src/DataTypes/Serializations/SerializationVariantElement.h +++ b/src/DataTypes/Serializations/SerializationVariantElement.h @@ -9,6 +9,7 @@ namespace DB { class SerializationVariant; +class SerializationVariantElementNullMap; /// Serialization for Variant element when we read it as a subcolumn. class SerializationVariantElement final : public SerializationWrapper @@ -66,12 +67,14 @@ public: const String variant_element_name; const ColumnVariant::Discriminator global_variant_discriminator; const ColumnVariant::Discriminator local_variant_discriminator; + bool make_nullable; VariantSubcolumnCreator( const ColumnPtr & local_discriminators_, const String & variant_element_name_, ColumnVariant::Discriminator global_variant_discriminator_, - ColumnVariant::Discriminator local_variant_discriminator_); + ColumnVariant::Discriminator local_variant_discriminator_, + bool make_nullable_); DataTypePtr create(const DataTypePtr & prev) const override; ColumnPtr create(const ColumnPtr & prev) const override; @@ -79,15 +82,18 @@ public: }; private: friend SerializationVariant; + friend SerializationVariantElementNullMap; struct DeserializeBinaryBulkStateVariantElement; - size_t deserializeCompactDiscriminators( + static size_t deserializeCompactDiscriminators( ColumnPtr & discriminators_column, + ColumnVariant::Discriminator variant_discriminator, size_t limit, ReadBuffer * stream, bool continuous_reading, - DeserializeBinaryBulkStateVariantElement & variant_element_state) const; + DeserializeBinaryBulkStatePtr & discriminators_state_, + const ISerialization * serialization); void addVariantToPath(SubstreamPath & path) const; void removeVariantFromPath(SubstreamPath & path) const; diff --git a/src/DataTypes/Serializations/SerializationVariantElementNullMap.cpp b/src/DataTypes/Serializations/SerializationVariantElementNullMap.cpp new file mode 100644 index 00000000000..4e355fbb8ef --- /dev/null +++ b/src/DataTypes/Serializations/SerializationVariantElementNullMap.cpp @@ -0,0 +1,190 @@ +#include +#include +#include +#include +#include +#include +#include +#include + +namespace DB +{ + +namespace ErrorCodes +{ + extern const int NOT_IMPLEMENTED; +} + +struct DeserializeBinaryBulkStateVariantElementNullMap : public ISerialization::DeserializeBinaryBulkState +{ + /// During deserialization discriminators streams can be shared. + /// For example we can read several variant elements together: "select v.UInt32, v.String.null from table", + /// or we can read the whole variant and some of variant elements or their subcolumns: "select v, v.UInt32.null from table". + /// To read the same column from the same stream more than once we use substream cache, + /// but this cache stores the whole column, not only the current range. + /// During deserialization of variant elements or their subcolumns discriminators column is not stored + /// in the result column, so we need to store them inside deserialization state, so we can use + /// substream cache correctly. + ColumnPtr discriminators; + ISerialization::DeserializeBinaryBulkStatePtr discriminators_state; +}; + +void SerializationVariantElementNullMap::enumerateStreams( + DB::ISerialization::EnumerateStreamsSettings & settings, + const DB::ISerialization::StreamCallback & callback, + const DB::ISerialization::SubstreamData &) const +{ + /// We will need stream for discriminators during deserialization. + settings.path.push_back(Substream::VariantDiscriminators); + callback(settings.path); + settings.path.pop_back(); +} + +void SerializationVariantElementNullMap::serializeBinaryBulkStatePrefix( + const IColumn &, SerializeBinaryBulkSettings &, SerializeBinaryBulkStatePtr &) const +{ + throw Exception( + ErrorCodes::NOT_IMPLEMENTED, "Method serializeBinaryBulkStatePrefix is not implemented for SerializationVariantElementNullMap"); +} + +void SerializationVariantElementNullMap::serializeBinaryBulkStateSuffix(SerializeBinaryBulkSettings &, SerializeBinaryBulkStatePtr &) const +{ + throw Exception( + ErrorCodes::NOT_IMPLEMENTED, "Method serializeBinaryBulkStateSuffix is not implemented for SerializationVariantElementNullMap"); +} + +void SerializationVariantElementNullMap::deserializeBinaryBulkStatePrefix( + DeserializeBinaryBulkSettings & settings, DeserializeBinaryBulkStatePtr & state, SubstreamsDeserializeStatesCache * cache) const +{ + DeserializeBinaryBulkStatePtr discriminators_state = SerializationVariant::deserializeDiscriminatorsStatePrefix(settings, cache); + if (!discriminators_state) + return; + + auto variant_element_null_map_state = std::make_shared(); + variant_element_null_map_state->discriminators_state = std::move(discriminators_state); + state = std::move(variant_element_null_map_state); +} + +void SerializationVariantElementNullMap::serializeBinaryBulkWithMultipleStreams( + const IColumn &, size_t, size_t, SerializeBinaryBulkSettings &, SerializeBinaryBulkStatePtr &) const +{ + throw Exception( + ErrorCodes::NOT_IMPLEMENTED, + "Method serializeBinaryBulkWithMultipleStreams is not implemented for SerializationVariantElementNullMap"); +} + +void SerializationVariantElementNullMap::deserializeBinaryBulkWithMultipleStreams( + ColumnPtr & result_column, + size_t limit, + DeserializeBinaryBulkSettings & settings, + DeserializeBinaryBulkStatePtr & state, + SubstreamsCache * cache) const +{ + /// Deserialize discriminators from Variant column. + settings.path.push_back(Substream::VariantDiscriminators); + + DeserializeBinaryBulkStateVariantElementNullMap * variant_element_null_map_state = nullptr; + std::optional variant_limit; + if (auto cached_discriminators = getFromSubstreamsCache(cache, settings.path)) + { + variant_element_null_map_state = checkAndGetState(state); + variant_element_null_map_state->discriminators = cached_discriminators; + } + else if (auto * discriminators_stream = settings.getter(settings.path)) + { + variant_element_null_map_state = checkAndGetState(state); + auto * discriminators_state = checkAndGetState( + variant_element_null_map_state->discriminators_state); + + /// If we started to read a new column, reinitialize discriminators column in deserialization state. + if (!variant_element_null_map_state->discriminators || result_column->empty()) + variant_element_null_map_state->discriminators = ColumnVariant::ColumnDiscriminators::create(); + + /// Deserialize discriminators according to serialization mode. + if (discriminators_state->mode.value == SerializationVariant::DiscriminatorsSerializationMode::BASIC) + SerializationNumber().deserializeBinaryBulk( + *variant_element_null_map_state->discriminators->assumeMutable(), *discriminators_stream, limit, 0); + else + variant_limit = SerializationVariantElement::deserializeCompactDiscriminators( + variant_element_null_map_state->discriminators, + variant_discriminator, + limit, + discriminators_stream, + settings.continuous_reading, + variant_element_null_map_state->discriminators_state, + this); + + addToSubstreamsCache(cache, settings.path, variant_element_null_map_state->discriminators); + } + else + { + /// There is no such stream or cached data, it means that there is no Variant column in this part (it could happend after alter table add column). + /// In such cases columns are filled with default values, but for null-map column default value should be 1, not 0. Fill column with 1 here instead + MutableColumnPtr mutable_column = result_column->assumeMutable(); + auto & data = assert_cast(*mutable_column).getData(); + data.resize_fill(data.size() + limit, 1); + settings.path.pop_back(); + return; + } + settings.path.pop_back(); + + MutableColumnPtr mutable_column = result_column->assumeMutable(); + auto & data = assert_cast(*mutable_column).getData(); + /// Check if there are no such variant in read range. + if (variant_limit && *variant_limit == 0) + { + data.resize_fill(data.size() + limit, 1); + } + /// Check if there is only our variant in read range. + else if (variant_limit && *variant_limit == limit) + { + data.resize_fill(data.size() + limit, 0); + } + /// Iterate through new discriminators to calculate the null map of our variant. + else + { + const auto & discriminators_data + = assert_cast(*variant_element_null_map_state->discriminators).getData(); + size_t discriminators_offset = variant_element_null_map_state->discriminators->size() - limit; + for (size_t i = discriminators_offset; i != discriminators_data.size(); ++i) + data.push_back(discriminators_data[i] != variant_discriminator); + } +} + +SerializationVariantElementNullMap::VariantNullMapSubcolumnCreator::VariantNullMapSubcolumnCreator( + const ColumnPtr & local_discriminators_, + const String & variant_element_name_, + ColumnVariant::Discriminator global_variant_discriminator_, + ColumnVariant::Discriminator local_variant_discriminator_) + : local_discriminators(local_discriminators_) + , variant_element_name(variant_element_name_) + , global_variant_discriminator(global_variant_discriminator_) + , local_variant_discriminator(local_variant_discriminator_) +{ +} + +DataTypePtr SerializationVariantElementNullMap::VariantNullMapSubcolumnCreator::create(const DB::DataTypePtr &) const +{ + return std::make_shared(); +} + +SerializationPtr SerializationVariantElementNullMap::VariantNullMapSubcolumnCreator::create(const DB::SerializationPtr &) const +{ + return std::make_shared(variant_element_name, global_variant_discriminator); +} + +ColumnPtr SerializationVariantElementNullMap::VariantNullMapSubcolumnCreator::create(const DB::ColumnPtr &) const +{ + /// Iterate through discriminators and create null-map for our variant. + auto null_map_col = ColumnUInt8::create(); + auto & null_map_data = null_map_col->getData(); + null_map_data.reserve(local_discriminators->size()); + const auto & local_discriminators_data = assert_cast(*local_discriminators).getData(); + for (auto local_discr : local_discriminators_data) + null_map_data.push_back(local_discr != local_variant_discriminator); + + return null_map_col; +} + + +} diff --git a/src/DataTypes/Serializations/SerializationVariantElementNullMap.h b/src/DataTypes/Serializations/SerializationVariantElementNullMap.h new file mode 100644 index 00000000000..cd81b445189 --- /dev/null +++ b/src/DataTypes/Serializations/SerializationVariantElementNullMap.h @@ -0,0 +1,107 @@ +#pragma once + +#include +#include +#include +#include + +namespace DB +{ + +namespace ErrorCodes +{ + extern const int NOT_IMPLEMENTED; +} + +class SerializationVariant; +class SerializationVariantElement; + +/// Serialization for Variant element null map when we read it as a subcolumn. +/// For example, variant.UInt64.null. +/// It requires separate serialization because there is no actual Nullable column +/// and we should construct null map from variant discriminators. +/// The implementation of deserializeBinaryBulk* methods is similar to SerializationVariantElement, +/// but differs in that there is no need to read the actual data of the variant, only discriminators. +class SerializationVariantElementNullMap final : public SimpleTextSerialization +{ +public: + SerializationVariantElementNullMap(const String & variant_element_name_, ColumnVariant::Discriminator variant_discriminator_) + : variant_element_name(variant_element_name_), variant_discriminator(variant_discriminator_) + { + } + + void enumerateStreams( + EnumerateStreamsSettings & settings, + const StreamCallback & callback, + const SubstreamData & data) const override; + + void serializeBinaryBulkStatePrefix( + const IColumn & column, + SerializeBinaryBulkSettings & settings, + SerializeBinaryBulkStatePtr & state) const override; + + void serializeBinaryBulkStateSuffix( + SerializeBinaryBulkSettings & settings, + SerializeBinaryBulkStatePtr & state) const override; + + void deserializeBinaryBulkStatePrefix( + DeserializeBinaryBulkSettings & settings, + DeserializeBinaryBulkStatePtr & state, + SubstreamsDeserializeStatesCache * cache) const override; + + void serializeBinaryBulkWithMultipleStreams( + const IColumn & column, + size_t offset, + size_t limit, + SerializeBinaryBulkSettings & settings, + SerializeBinaryBulkStatePtr & state) const override; + + void deserializeBinaryBulkWithMultipleStreams( + ColumnPtr & column, + size_t limit, + DeserializeBinaryBulkSettings & settings, + DeserializeBinaryBulkStatePtr & state, + SubstreamsCache * cache) const override; + + void serializeBinary(const Field &, WriteBuffer &, const FormatSettings &) const override { throwNoSerialization(); } + void deserializeBinary(Field &, ReadBuffer &, const FormatSettings &) const override { throwNoSerialization(); } + void serializeBinary(const IColumn &, size_t, WriteBuffer &, const FormatSettings &) const override { throwNoSerialization(); } + void deserializeBinary(IColumn &, ReadBuffer &, const FormatSettings &) const override { throwNoSerialization(); } + void serializeText(const IColumn &, size_t, WriteBuffer &, const FormatSettings &) const override { throwNoSerialization(); } + void deserializeText(IColumn &, ReadBuffer &, const FormatSettings &, bool) const override { throwNoSerialization(); } + bool tryDeserializeText(IColumn &, ReadBuffer &, const FormatSettings &, bool) const override { throwNoSerialization(); } + + struct VariantNullMapSubcolumnCreator : public ISubcolumnCreator + { + const ColumnPtr local_discriminators; + const String variant_element_name; + const ColumnVariant::Discriminator global_variant_discriminator; + const ColumnVariant::Discriminator local_variant_discriminator; + + VariantNullMapSubcolumnCreator( + const ColumnPtr & local_discriminators_, + const String & variant_element_name_, + ColumnVariant::Discriminator global_variant_discriminator_, + ColumnVariant::Discriminator local_variant_discriminator_); + + DataTypePtr create(const DataTypePtr & prev) const override; + ColumnPtr create(const ColumnPtr & prev) const override; + SerializationPtr create(const SerializationPtr & prev) const override; + }; +private: + [[noreturn]] static void throwNoSerialization() + { + throw Exception(ErrorCodes::NOT_IMPLEMENTED, "Text/binary serialization is not implemented for variant element null map subcolumn"); + } + + friend SerializationVariant; + friend SerializationVariantElement; + + /// To be able to deserialize Variant element null map as a subcolumn + /// we need variant element type name and global discriminator. + String variant_element_name; + ColumnVariant::Discriminator variant_discriminator; + +}; + +} diff --git a/tests/queries/0_stateless/02941_variant_type_1.reference b/tests/queries/0_stateless/02941_variant_type_1.reference index 8a6e77d4f6d..53e5a556821 100644 --- a/tests/queries/0_stateless/02941_variant_type_1.reference +++ b/tests/queries/0_stateless/02941_variant_type_1.reference @@ -91,42 +91,42 @@ lc_str_2 (0,0) (0,0) (0,0) -\N -\N -\N -\N -\N -\N -\N -\N -\N -\N -\N -\N +0 +0 +0 +0 +0 +0 +0 +0 +0 +0 +0 +0 0 1 2 -\N -\N -\N -\N -\N -\N -\N -\N -\N -\N -\N -\N -\N -\N -\N +0 +0 +0 +0 +0 +0 +0 +0 +0 +0 +0 +0 +0 +0 +0 1 2 3 -\N -\N -\N +0 +0 +0 [] [] [] @@ -145,21 +145,21 @@ lc_str_2 [0] [0,1] [0,1,2] -\N -\N -\N -\N -\N -\N -\N -\N -\N -\N -\N -\N -\N -\N -\N +0 +0 +0 +0 +0 +0 +0 +0 +0 +0 +0 +0 +0 +0 +0 1 2 3 @@ -256,42 +256,42 @@ lc_str_2 (0,0) (0,0) (0,0) -\N -\N -\N -\N -\N -\N -\N -\N -\N -\N -\N -\N 0 -\N +0 +0 +0 +0 +0 +0 +0 +0 +0 +0 +0 +0 +0 2 -\N -\N -\N -\N -\N -\N -\N -\N -\N -\N -\N -\N -\N -\N -\N +0 +0 +0 +0 +0 +0 +0 +0 +0 +0 +0 +0 +0 +0 +0 1 -\N +0 3 -\N -\N -\N +0 +0 +0 [] [] [] @@ -310,23 +310,23 @@ lc_str_2 [0] [] [0,1,2] -\N -\N -\N -\N -\N -\N -\N -\N -\N -\N -\N -\N -\N -\N -\N +0 +0 +0 +0 +0 +0 +0 +0 +0 +0 +0 +0 +0 +0 +0 1 -\N +0 3 ----------------------------------------------------------------------------------------------------------- test3 insert @@ -421,42 +421,42 @@ lc_str_15 (0,0) (16,17) (0,0) -\N -\N -\N -\N +0 +0 +0 +0 4 -\N -\N -\N -\N -\N +0 +0 +0 +0 +0 10 -\N -\N -\N -\N -\N +0 +0 +0 +0 +0 16 -\N -\N -\N -\N -\N +0 +0 +0 +0 +0 5 -\N -\N -\N -\N -\N +0 +0 +0 +0 +0 11 -\N -\N -\N -\N -\N +0 +0 +0 +0 +0 17 -\N +0 [] [] [] @@ -475,23 +475,23 @@ lc_str_15 [] [] [0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17] -\N -\N -\N -\N -\N +0 +0 +0 +0 +0 6 -\N -\N -\N -\N -\N +0 +0 +0 +0 +0 12 -\N -\N -\N -\N -\N +0 +0 +0 +0 +0 18 ----------------------------------------------------------------------------------------------------------- MergeTree compact @@ -587,42 +587,42 @@ lc_str_2 (0,0) (0,0) (0,0) -\N -\N -\N -\N -\N -\N -\N -\N -\N -\N -\N -\N +0 +0 +0 +0 +0 +0 +0 +0 +0 +0 +0 +0 0 1 2 -\N -\N -\N -\N -\N -\N -\N -\N -\N -\N -\N -\N -\N -\N -\N +0 +0 +0 +0 +0 +0 +0 +0 +0 +0 +0 +0 +0 +0 +0 1 2 3 -\N -\N -\N +0 +0 +0 [] [] [] @@ -641,21 +641,21 @@ lc_str_2 [0] [0,1] [0,1,2] -\N -\N -\N -\N -\N -\N -\N -\N -\N -\N -\N -\N -\N -\N -\N +0 +0 +0 +0 +0 +0 +0 +0 +0 +0 +0 +0 +0 +0 +0 1 2 3 @@ -751,42 +751,42 @@ lc_str_2 (0,0) (0,0) (0,0) -\N -\N -\N -\N -\N -\N -\N -\N -\N -\N -\N -\N +0 +0 +0 +0 +0 +0 +0 +0 +0 +0 +0 +0 0 1 2 -\N -\N -\N -\N -\N -\N -\N -\N -\N -\N -\N -\N -\N -\N -\N +0 +0 +0 +0 +0 +0 +0 +0 +0 +0 +0 +0 +0 +0 +0 1 2 3 -\N -\N -\N +0 +0 +0 [] [] [] @@ -805,21 +805,21 @@ lc_str_2 [0] [0,1] [0,1,2] -\N -\N -\N -\N -\N -\N -\N -\N -\N -\N -\N -\N -\N -\N -\N +0 +0 +0 +0 +0 +0 +0 +0 +0 +0 +0 +0 +0 +0 +0 1 2 3 @@ -916,42 +916,42 @@ lc_str_2 (0,0) (0,0) (0,0) -\N -\N -\N -\N -\N -\N -\N -\N -\N -\N -\N -\N 0 -\N +0 +0 +0 +0 +0 +0 +0 +0 +0 +0 +0 +0 +0 2 -\N -\N -\N -\N -\N -\N -\N -\N -\N -\N -\N -\N -\N -\N -\N +0 +0 +0 +0 +0 +0 +0 +0 +0 +0 +0 +0 +0 +0 +0 1 -\N +0 3 -\N -\N -\N +0 +0 +0 [] [] [] @@ -970,23 +970,23 @@ lc_str_2 [0] [] [0,1,2] -\N -\N -\N -\N -\N -\N -\N -\N -\N -\N -\N -\N -\N -\N -\N +0 +0 +0 +0 +0 +0 +0 +0 +0 +0 +0 +0 +0 +0 +0 1 -\N +0 3 ----------------------------------------------------------------------------------------------------------- test2 select @@ -1080,42 +1080,42 @@ lc_str_2 (0,0) (0,0) (0,0) -\N -\N -\N -\N -\N -\N -\N -\N -\N -\N -\N -\N 0 -\N +0 +0 +0 +0 +0 +0 +0 +0 +0 +0 +0 +0 +0 2 -\N -\N -\N -\N -\N -\N -\N -\N -\N -\N -\N -\N -\N -\N -\N +0 +0 +0 +0 +0 +0 +0 +0 +0 +0 +0 +0 +0 +0 +0 1 -\N +0 3 -\N -\N -\N +0 +0 +0 [] [] [] @@ -1134,23 +1134,23 @@ lc_str_2 [0] [] [0,1,2] -\N -\N -\N -\N -\N -\N -\N -\N -\N -\N -\N -\N -\N -\N -\N +0 +0 +0 +0 +0 +0 +0 +0 +0 +0 +0 +0 +0 +0 +0 1 -\N +0 3 ----------------------------------------------------------------------------------------------------------- test3 insert @@ -1245,42 +1245,42 @@ lc_str_15 (0,0) (16,17) (0,0) -\N -\N -\N -\N +0 +0 +0 +0 4 -\N -\N -\N -\N -\N +0 +0 +0 +0 +0 10 -\N -\N -\N -\N -\N +0 +0 +0 +0 +0 16 -\N -\N -\N -\N -\N +0 +0 +0 +0 +0 5 -\N -\N -\N -\N -\N +0 +0 +0 +0 +0 11 -\N -\N -\N -\N -\N +0 +0 +0 +0 +0 17 -\N +0 [] [] [] @@ -1299,23 +1299,23 @@ lc_str_15 [] [] [0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17] -\N -\N -\N -\N -\N +0 +0 +0 +0 +0 6 -\N -\N -\N -\N -\N +0 +0 +0 +0 +0 12 -\N -\N -\N -\N -\N +0 +0 +0 +0 +0 18 ----------------------------------------------------------------------------------------------------------- test3 select @@ -1409,42 +1409,42 @@ lc_str_15 (0,0) (16,17) (0,0) -\N -\N -\N -\N +0 +0 +0 +0 4 -\N -\N -\N -\N -\N +0 +0 +0 +0 +0 10 -\N -\N -\N -\N -\N +0 +0 +0 +0 +0 16 -\N -\N -\N -\N -\N +0 +0 +0 +0 +0 5 -\N -\N -\N -\N -\N +0 +0 +0 +0 +0 11 -\N -\N -\N -\N -\N +0 +0 +0 +0 +0 17 -\N +0 [] [] [] @@ -1463,23 +1463,23 @@ lc_str_15 [] [] [0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17] -\N -\N -\N -\N -\N +0 +0 +0 +0 +0 6 -\N -\N -\N -\N -\N +0 +0 +0 +0 +0 12 -\N -\N -\N -\N -\N +0 +0 +0 +0 +0 18 ----------------------------------------------------------------------------------------------------------- MergeTree wide @@ -1575,42 +1575,42 @@ lc_str_2 (0,0) (0,0) (0,0) -\N -\N -\N -\N -\N -\N -\N -\N -\N -\N -\N -\N +0 +0 +0 +0 +0 +0 +0 +0 +0 +0 +0 +0 0 1 2 -\N -\N -\N -\N -\N -\N -\N -\N -\N -\N -\N -\N -\N -\N -\N +0 +0 +0 +0 +0 +0 +0 +0 +0 +0 +0 +0 +0 +0 +0 1 2 3 -\N -\N -\N +0 +0 +0 [] [] [] @@ -1629,21 +1629,21 @@ lc_str_2 [0] [0,1] [0,1,2] -\N -\N -\N -\N -\N -\N -\N -\N -\N -\N -\N -\N -\N -\N -\N +0 +0 +0 +0 +0 +0 +0 +0 +0 +0 +0 +0 +0 +0 +0 1 2 3 @@ -1739,42 +1739,42 @@ lc_str_2 (0,0) (0,0) (0,0) -\N -\N -\N -\N -\N -\N -\N -\N -\N -\N -\N -\N +0 +0 +0 +0 +0 +0 +0 +0 +0 +0 +0 +0 0 1 2 -\N -\N -\N -\N -\N -\N -\N -\N -\N -\N -\N -\N -\N -\N -\N +0 +0 +0 +0 +0 +0 +0 +0 +0 +0 +0 +0 +0 +0 +0 1 2 3 -\N -\N -\N +0 +0 +0 [] [] [] @@ -1793,21 +1793,21 @@ lc_str_2 [0] [0,1] [0,1,2] -\N -\N -\N -\N -\N -\N -\N -\N -\N -\N -\N -\N -\N -\N -\N +0 +0 +0 +0 +0 +0 +0 +0 +0 +0 +0 +0 +0 +0 +0 1 2 3 @@ -1904,42 +1904,42 @@ lc_str_2 (0,0) (0,0) (0,0) -\N -\N -\N -\N -\N -\N -\N -\N -\N -\N -\N -\N 0 -\N +0 +0 +0 +0 +0 +0 +0 +0 +0 +0 +0 +0 +0 2 -\N -\N -\N -\N -\N -\N -\N -\N -\N -\N -\N -\N -\N -\N -\N +0 +0 +0 +0 +0 +0 +0 +0 +0 +0 +0 +0 +0 +0 +0 1 -\N +0 3 -\N -\N -\N +0 +0 +0 [] [] [] @@ -1958,23 +1958,23 @@ lc_str_2 [0] [] [0,1,2] -\N -\N -\N -\N -\N -\N -\N -\N -\N -\N -\N -\N -\N -\N -\N +0 +0 +0 +0 +0 +0 +0 +0 +0 +0 +0 +0 +0 +0 +0 1 -\N +0 3 ----------------------------------------------------------------------------------------------------------- test2 select @@ -2068,42 +2068,42 @@ lc_str_2 (0,0) (0,0) (0,0) -\N -\N -\N -\N -\N -\N -\N -\N -\N -\N -\N -\N 0 -\N +0 +0 +0 +0 +0 +0 +0 +0 +0 +0 +0 +0 +0 2 -\N -\N -\N -\N -\N -\N -\N -\N -\N -\N -\N -\N -\N -\N -\N +0 +0 +0 +0 +0 +0 +0 +0 +0 +0 +0 +0 +0 +0 +0 1 -\N +0 3 -\N -\N -\N +0 +0 +0 [] [] [] @@ -2122,23 +2122,23 @@ lc_str_2 [0] [] [0,1,2] -\N -\N -\N -\N -\N -\N -\N -\N -\N -\N -\N -\N -\N -\N -\N +0 +0 +0 +0 +0 +0 +0 +0 +0 +0 +0 +0 +0 +0 +0 1 -\N +0 3 ----------------------------------------------------------------------------------------------------------- test3 insert @@ -2233,42 +2233,42 @@ lc_str_15 (0,0) (16,17) (0,0) -\N -\N -\N -\N +0 +0 +0 +0 4 -\N -\N -\N -\N -\N +0 +0 +0 +0 +0 10 -\N -\N -\N -\N -\N +0 +0 +0 +0 +0 16 -\N -\N -\N -\N -\N +0 +0 +0 +0 +0 5 -\N -\N -\N -\N -\N +0 +0 +0 +0 +0 11 -\N -\N -\N -\N -\N +0 +0 +0 +0 +0 17 -\N +0 [] [] [] @@ -2287,23 +2287,23 @@ lc_str_15 [] [] [0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17] -\N -\N -\N -\N -\N +0 +0 +0 +0 +0 6 -\N -\N -\N -\N -\N +0 +0 +0 +0 +0 12 -\N -\N -\N -\N -\N +0 +0 +0 +0 +0 18 ----------------------------------------------------------------------------------------------------------- test3 select @@ -2397,42 +2397,42 @@ lc_str_15 (0,0) (16,17) (0,0) -\N -\N -\N -\N +0 +0 +0 +0 4 -\N -\N -\N -\N -\N +0 +0 +0 +0 +0 10 -\N -\N -\N -\N -\N +0 +0 +0 +0 +0 16 -\N -\N -\N -\N -\N +0 +0 +0 +0 +0 5 -\N -\N -\N -\N -\N +0 +0 +0 +0 +0 11 -\N -\N -\N -\N -\N +0 +0 +0 +0 +0 17 -\N +0 [] [] [] @@ -2451,22 +2451,22 @@ lc_str_15 [] [] [0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17] -\N -\N -\N -\N -\N +0 +0 +0 +0 +0 6 -\N -\N -\N -\N -\N +0 +0 +0 +0 +0 12 -\N -\N -\N -\N -\N +0 +0 +0 +0 +0 18 ----------------------------------------------------------------------------------------------------------- diff --git a/tests/queries/0_stateless/02941_variant_type_2.reference b/tests/queries/0_stateless/02941_variant_type_2.reference index 20a5176cb5e..1d9126aa230 100644 --- a/tests/queries/0_stateless/02941_variant_type_2.reference +++ b/tests/queries/0_stateless/02941_variant_type_2.reference @@ -6,9 +6,6 @@ test4 select 100000 100000 100000 -100000 -100000 -100000 MergeTree compact test4 insert test4 select @@ -17,18 +14,12 @@ test4 select 100000 100000 100000 -100000 -100000 -100000 test4 select 500000 100000 100000 100000 100000 -100000 -100000 -100000 MergeTree wide test4 insert test4 select @@ -37,15 +28,9 @@ test4 select 100000 100000 100000 -100000 -100000 -100000 test4 select 500000 100000 100000 100000 100000 -100000 -100000 -100000 diff --git a/tests/queries/0_stateless/02941_variant_type_2.sh b/tests/queries/0_stateless/02941_variant_type_2.sh index f43cd2bb0d6..8453bce98dc 100755 --- a/tests/queries/0_stateless/02941_variant_type_2.sh +++ b/tests/queries/0_stateless/02941_variant_type_2.sh @@ -33,13 +33,10 @@ select v.\`LowCardinality(String)\` from test format Null; select count() from test where isNotNull(v.\`LowCardinality(String)\`); select v.\`Tuple(a UInt32, b UInt32)\` from test format Null; select v.\`Tuple(a UInt32, b UInt32)\`.a from test format Null; -select count() from test where isNotNull(v.\`Tuple(a UInt32, b UInt32)\`.a); select v.\`Tuple(a UInt32, b UInt32)\`.b from test format Null; -select count() from test where isNotNull(v.\`Tuple(a UInt32, b UInt32)\`.b); select v.\`Array(UInt64)\` from test format Null; select count() from test where not empty(v.\`Array(UInt64)\`); -select v.\`Array(UInt64)\`.size0 from test format Null; -select count() from test where isNotNull(v.\`Array(UInt64)\`.size0);" +select v.\`Array(UInt64)\`.size0 from test format Null;" } function run() diff --git a/tests/queries/0_stateless/02941_variant_type_3.reference b/tests/queries/0_stateless/02941_variant_type_3.reference index 1ccdb3acdff..d28aa7a594b 100644 --- a/tests/queries/0_stateless/02941_variant_type_3.reference +++ b/tests/queries/0_stateless/02941_variant_type_3.reference @@ -6,9 +6,6 @@ test5 select 100000 100000 100000 -100000 -100000 -100000 MergeTree compact test5 insert test5 select @@ -17,18 +14,12 @@ test5 select 100000 100000 100000 -100000 -100000 -100000 test5 select 500000 100000 100000 100000 100000 -100000 -100000 -100000 MergeTree wide test5 insert test5 select @@ -37,15 +28,9 @@ test5 select 100000 100000 100000 -100000 -100000 -100000 test5 select 500000 100000 100000 100000 100000 -100000 -100000 -100000 diff --git a/tests/queries/0_stateless/02941_variant_type_3.sh b/tests/queries/0_stateless/02941_variant_type_3.sh index f4b2b304f56..990eb25b5be 100755 --- a/tests/queries/0_stateless/02941_variant_type_3.sh +++ b/tests/queries/0_stateless/02941_variant_type_3.sh @@ -35,13 +35,10 @@ select v.\`LowCardinality(String)\` from test format Null; select count() from test where isNotNull(v.\`LowCardinality(String)\`); select v.\`Tuple(a UInt32, b UInt32)\` from test format Null; select v.\`Tuple(a UInt32, b UInt32)\`.a from test format Null; -select count() from test where isNotNull(v.\`Tuple(a UInt32, b UInt32)\`.a); select v.\`Tuple(a UInt32, b UInt32)\`.b from test format Null; -select count() from test where isNotNull(v.\`Tuple(a UInt32, b UInt32)\`.b); select v.\`Array(UInt64)\` from test format Null; select count() from test where not empty(v.\`Array(UInt64)\`); -select v.\`Array(UInt64)\`.size0 from test format Null; -select count() from test where isNotNull(v.\`Array(UInt64)\`.size0);" +select v.\`Array(UInt64)\`.size0 from test format Null;" } function run() diff --git a/tests/queries/0_stateless/02941_variant_type_4.reference b/tests/queries/0_stateless/02941_variant_type_4.reference index e13d5820343..d1630b04347 100644 --- a/tests/queries/0_stateless/02941_variant_type_4.reference +++ b/tests/queries/0_stateless/02941_variant_type_4.reference @@ -6,9 +6,6 @@ test6 select 200000 200000 200000 -200000 -200000 -200000 ----------------------------------------------------------------------------------------------------------- MergeTree compact test6 insert @@ -18,9 +15,6 @@ test6 select 200000 200000 200000 -200000 -200000 -200000 ----------------------------------------------------------------------------------------------------------- test6 select 1000000 @@ -28,9 +22,6 @@ test6 select 200000 200000 200000 -200000 -200000 -200000 ----------------------------------------------------------------------------------------------------------- MergeTree wide test6 insert @@ -40,9 +31,6 @@ test6 select 200000 200000 200000 -200000 -200000 -200000 ----------------------------------------------------------------------------------------------------------- test6 select 1000000 @@ -50,7 +38,4 @@ test6 select 200000 200000 200000 -200000 -200000 -200000 ----------------------------------------------------------------------------------------------------------- diff --git a/tests/queries/0_stateless/02941_variant_type_4.sh b/tests/queries/0_stateless/02941_variant_type_4.sh index f9a16847864..b8f619694b0 100755 --- a/tests/queries/0_stateless/02941_variant_type_4.sh +++ b/tests/queries/0_stateless/02941_variant_type_4.sh @@ -29,13 +29,10 @@ function test6_select() select count() from test where isNotNull(v.\`LowCardinality(String)\`); select v.\`Tuple(a UInt32, b UInt32)\` from test format Null; select v.\`Tuple(a UInt32, b UInt32)\`.a from test format Null; - select count() from test where isNotNull(v.\`Tuple(a UInt32, b UInt32)\`.a); select v.\`Tuple(a UInt32, b UInt32)\`.b from test format Null; - select count() from test where isNotNull(v.\`Tuple(a UInt32, b UInt32)\`.b); select v.\`Array(UInt64)\` from test format Null; select count() from test where not empty(v.\`Array(UInt64)\`); - select v.\`Array(UInt64)\`.size0 from test format Null; - select count() from test where isNotNull(v.\`Array(UInt64)\`.size0);" + select v.\`Array(UInt64)\`.size0 from test format Null;" echo "-----------------------------------------------------------------------------------------------------------" } diff --git a/tests/queries/0_stateless/03040_dynamic_type_alters_1.reference b/tests/queries/0_stateless/03040_dynamic_type_alters_1.reference index ca98ec0963c..a9c785d1e48 100644 --- a/tests/queries/0_stateless/03040_dynamic_type_alters_1.reference +++ b/tests/queries/0_stateless/03040_dynamic_type_alters_1.reference @@ -2,525 +2,525 @@ Memory initial insert alter add column 1 3 None -0 0 \N \N \N \N -1 1 \N \N \N \N -2 2 \N \N \N \N +0 0 \N \N \N 0 +1 1 \N \N \N 0 +2 2 \N \N \N 0 insert after alter add column 1 4 String 4 UInt64 7 None -0 0 \N \N \N \N \N -1 1 \N \N \N \N \N -2 2 \N \N \N \N \N -3 3 3 \N 3 \N \N -4 4 4 \N 4 \N \N -5 5 5 \N 5 \N \N -6 6 str_6 str_6 \N \N \N -7 7 str_7 str_7 \N \N \N -8 8 str_8 str_8 \N \N \N -9 9 \N \N \N \N \N -10 10 \N \N \N \N \N -11 11 \N \N \N \N \N -12 12 12 \N 12 \N \N -13 13 str_13 str_13 \N \N \N -14 14 \N \N \N \N \N +0 0 \N \N \N \N 0 +1 1 \N \N \N \N 0 +2 2 \N \N \N \N 0 +3 3 3 \N 3 \N 0 +4 4 4 \N 4 \N 0 +5 5 5 \N 5 \N 0 +6 6 str_6 str_6 \N \N 0 +7 7 str_7 str_7 \N \N 0 +8 8 str_8 str_8 \N \N 0 +9 9 \N \N \N \N 0 +10 10 \N \N \N \N 0 +11 11 \N \N \N \N 0 +12 12 12 \N 12 \N 0 +13 13 str_13 str_13 \N \N 0 +14 14 \N \N \N \N 0 alter modify column 1 7 None 8 String -0 0 \N \N \N \N \N -1 1 \N \N \N \N \N -2 2 \N \N \N \N \N -3 3 3 3 \N \N \N -4 4 4 4 \N \N \N -5 5 5 5 \N \N \N -6 6 str_6 str_6 \N \N \N -7 7 str_7 str_7 \N \N \N -8 8 str_8 str_8 \N \N \N -9 9 \N \N \N \N \N -10 10 \N \N \N \N \N -11 11 \N \N \N \N \N -12 12 12 12 \N \N \N -13 13 str_13 str_13 \N \N \N -14 14 \N \N \N \N \N +0 0 \N \N \N \N 0 +1 1 \N \N \N \N 0 +2 2 \N \N \N \N 0 +3 3 3 3 \N \N 0 +4 4 4 4 \N \N 0 +5 5 5 5 \N \N 0 +6 6 str_6 str_6 \N \N 0 +7 7 str_7 str_7 \N \N 0 +8 8 str_8 str_8 \N \N 0 +9 9 \N \N \N \N 0 +10 10 \N \N \N \N 0 +11 11 \N \N \N \N 0 +12 12 12 12 \N \N 0 +13 13 str_13 str_13 \N \N 0 +14 14 \N \N \N \N 0 insert after alter modify column 1 8 None 11 String -0 0 \N \N \N \N \N -1 1 \N \N \N \N \N -2 2 \N \N \N \N \N -3 3 3 3 \N \N \N -4 4 4 4 \N \N \N -5 5 5 5 \N \N \N -6 6 str_6 str_6 \N \N \N -7 7 str_7 str_7 \N \N \N -8 8 str_8 str_8 \N \N \N -9 9 \N \N \N \N \N -10 10 \N \N \N \N \N -11 11 \N \N \N \N \N -12 12 12 12 \N \N \N -13 13 str_13 str_13 \N \N \N -14 14 \N \N \N \N \N -15 15 \N \N \N \N \N -16 16 16 16 \N \N \N -17 17 str_17 str_17 \N \N \N -18 18 1970-01-19 1970-01-19 \N \N \N +0 0 \N \N \N \N 0 +1 1 \N \N \N \N 0 +2 2 \N \N \N \N 0 +3 3 3 3 \N \N 0 +4 4 4 4 \N \N 0 +5 5 5 5 \N \N 0 +6 6 str_6 str_6 \N \N 0 +7 7 str_7 str_7 \N \N 0 +8 8 str_8 str_8 \N \N 0 +9 9 \N \N \N \N 0 +10 10 \N \N \N \N 0 +11 11 \N \N \N \N 0 +12 12 12 12 \N \N 0 +13 13 str_13 str_13 \N \N 0 +14 14 \N \N \N \N 0 +15 15 \N \N \N \N 0 +16 16 16 16 \N \N 0 +17 17 str_17 str_17 \N \N 0 +18 18 1970-01-19 1970-01-19 \N \N 0 alter modify column 2 4 UInt64 7 String 8 None -0 0 \N \N \N \N \N -1 1 \N \N \N \N \N -2 2 \N \N \N \N \N -3 3 3 \N 3 \N \N -4 4 4 \N 4 \N \N -5 5 5 \N 5 \N \N -6 6 str_6 str_6 \N \N \N -7 7 str_7 str_7 \N \N \N -8 8 str_8 str_8 \N \N \N -9 9 \N \N \N \N \N -10 10 \N \N \N \N \N -11 11 \N \N \N \N \N -12 12 12 \N 12 \N \N -13 13 str_13 str_13 \N \N \N -14 14 \N \N \N \N \N -15 15 \N \N \N \N \N -16 16 16 16 \N \N \N -17 17 str_17 str_17 \N \N \N -18 18 1970-01-19 1970-01-19 \N \N \N +0 0 \N \N \N \N 0 +1 1 \N \N \N \N 0 +2 2 \N \N \N \N 0 +3 3 3 \N 3 \N 0 +4 4 4 \N 4 \N 0 +5 5 5 \N 5 \N 0 +6 6 str_6 str_6 \N \N 0 +7 7 str_7 str_7 \N \N 0 +8 8 str_8 str_8 \N \N 0 +9 9 \N \N \N \N 0 +10 10 \N \N \N \N 0 +11 11 \N \N \N \N 0 +12 12 12 \N 12 \N 0 +13 13 str_13 str_13 \N \N 0 +14 14 \N \N \N \N 0 +15 15 \N \N \N \N 0 +16 16 16 16 \N \N 0 +17 17 str_17 str_17 \N \N 0 +18 18 1970-01-19 1970-01-19 \N \N 0 insert after alter modify column 2 1 Date 5 UInt64 8 String 9 None -0 0 \N \N \N \N \N -1 1 \N \N \N \N \N -2 2 \N \N \N \N \N -3 3 3 \N 3 \N \N -4 4 4 \N 4 \N \N -5 5 5 \N 5 \N \N -6 6 str_6 str_6 \N \N \N -7 7 str_7 str_7 \N \N \N -8 8 str_8 str_8 \N \N \N -9 9 \N \N \N \N \N -10 10 \N \N \N \N \N -11 11 \N \N \N \N \N -12 12 12 \N 12 \N \N -13 13 str_13 str_13 \N \N \N -14 14 \N \N \N \N \N -15 15 \N \N \N \N \N -16 16 16 16 \N \N \N -17 17 str_17 str_17 \N \N \N -18 18 1970-01-19 1970-01-19 \N \N \N -19 19 \N \N \N \N \N -20 20 20 \N 20 \N \N -21 21 str_21 str_21 \N \N \N -22 22 1970-01-23 \N \N 1970-01-23 \N +0 0 \N \N \N \N 0 +1 1 \N \N \N \N 0 +2 2 \N \N \N \N 0 +3 3 3 \N 3 \N 0 +4 4 4 \N 4 \N 0 +5 5 5 \N 5 \N 0 +6 6 str_6 str_6 \N \N 0 +7 7 str_7 str_7 \N \N 0 +8 8 str_8 str_8 \N \N 0 +9 9 \N \N \N \N 0 +10 10 \N \N \N \N 0 +11 11 \N \N \N \N 0 +12 12 12 \N 12 \N 0 +13 13 str_13 str_13 \N \N 0 +14 14 \N \N \N \N 0 +15 15 \N \N \N \N 0 +16 16 16 16 \N \N 0 +17 17 str_17 str_17 \N \N 0 +18 18 1970-01-19 1970-01-19 \N \N 0 +19 19 \N \N \N \N 0 +20 20 20 \N 20 \N 0 +21 21 str_21 str_21 \N \N 0 +22 22 1970-01-23 \N \N 1970-01-23 0 alter modify column 3 1 Date 5 UInt64 8 String 9 None -0 0 0 \N \N \N \N \N \N -1 1 1 \N \N \N \N \N \N -2 2 2 \N \N \N \N \N \N -3 3 3 \N \N \N 3 \N \N -4 4 4 \N \N \N 4 \N \N -5 5 5 \N \N \N 5 \N \N -6 6 6 \N \N str_6 \N \N \N -7 7 7 \N \N str_7 \N \N \N -8 8 8 \N \N str_8 \N \N \N -9 9 9 \N \N \N \N \N \N -10 10 10 \N \N \N \N \N \N -11 11 11 \N \N \N \N \N \N -12 12 12 \N \N \N 12 \N \N -13 13 13 \N \N str_13 \N \N \N -14 14 14 \N \N \N \N \N \N -15 15 15 \N \N \N \N \N \N -16 16 16 \N \N 16 \N \N \N -17 17 17 \N \N str_17 \N \N \N -18 18 18 \N \N 1970-01-19 \N \N \N -19 19 19 \N \N \N \N \N \N -20 20 20 \N \N \N 20 \N \N -21 21 21 \N \N str_21 \N \N \N -22 22 22 \N \N \N \N 1970-01-23 \N +0 0 0 \N 0 \N \N \N 0 +1 1 1 \N 0 \N \N \N 0 +2 2 2 \N 0 \N \N \N 0 +3 3 3 \N 0 \N 3 \N 0 +4 4 4 \N 0 \N 4 \N 0 +5 5 5 \N 0 \N 5 \N 0 +6 6 6 \N 0 str_6 \N \N 0 +7 7 7 \N 0 str_7 \N \N 0 +8 8 8 \N 0 str_8 \N \N 0 +9 9 9 \N 0 \N \N \N 0 +10 10 10 \N 0 \N \N \N 0 +11 11 11 \N 0 \N \N \N 0 +12 12 12 \N 0 \N 12 \N 0 +13 13 13 \N 0 str_13 \N \N 0 +14 14 14 \N 0 \N \N \N 0 +15 15 15 \N 0 \N \N \N 0 +16 16 16 \N 0 16 \N \N 0 +17 17 17 \N 0 str_17 \N \N 0 +18 18 18 \N 0 1970-01-19 \N \N 0 +19 19 19 \N 0 \N \N \N 0 +20 20 20 \N 0 \N 20 \N 0 +21 21 21 \N 0 str_21 \N \N 0 +22 22 22 \N 0 \N \N 1970-01-23 0 insert after alter modify column 3 1 Date 5 UInt64 8 String 12 None -0 0 0 \N \N \N \N \N \N -1 1 1 \N \N \N \N \N \N -2 2 2 \N \N \N \N \N \N -3 3 3 \N \N \N 3 \N \N -4 4 4 \N \N \N 4 \N \N -5 5 5 \N \N \N 5 \N \N -6 6 6 \N \N str_6 \N \N \N -7 7 7 \N \N str_7 \N \N \N -8 8 8 \N \N str_8 \N \N \N -9 9 9 \N \N \N \N \N \N -10 10 10 \N \N \N \N \N \N -11 11 11 \N \N \N \N \N \N -12 12 12 \N \N \N 12 \N \N -13 13 13 \N \N str_13 \N \N \N -14 14 14 \N \N \N \N \N \N -15 15 15 \N \N \N \N \N \N -16 16 16 \N \N 16 \N \N \N -17 17 17 \N \N str_17 \N \N \N -18 18 18 \N \N 1970-01-19 \N \N \N -19 19 19 \N \N \N \N \N \N -20 20 20 \N \N \N 20 \N \N -21 21 21 \N \N str_21 \N \N \N -22 22 22 \N \N \N \N 1970-01-23 \N -23 \N \N \N \N \N \N \N \N -24 24 24 \N \N \N \N \N \N -25 str_25 \N str_25 \N \N \N \N \N +0 0 0 \N 0 \N \N \N 0 +1 1 1 \N 0 \N \N \N 0 +2 2 2 \N 0 \N \N \N 0 +3 3 3 \N 0 \N 3 \N 0 +4 4 4 \N 0 \N 4 \N 0 +5 5 5 \N 0 \N 5 \N 0 +6 6 6 \N 0 str_6 \N \N 0 +7 7 7 \N 0 str_7 \N \N 0 +8 8 8 \N 0 str_8 \N \N 0 +9 9 9 \N 0 \N \N \N 0 +10 10 10 \N 0 \N \N \N 0 +11 11 11 \N 0 \N \N \N 0 +12 12 12 \N 0 \N 12 \N 0 +13 13 13 \N 0 str_13 \N \N 0 +14 14 14 \N 0 \N \N \N 0 +15 15 15 \N 0 \N \N \N 0 +16 16 16 \N 0 16 \N \N 0 +17 17 17 \N 0 str_17 \N \N 0 +18 18 18 \N 0 1970-01-19 \N \N 0 +19 19 19 \N 0 \N \N \N 0 +20 20 20 \N 0 \N 20 \N 0 +21 21 21 \N 0 str_21 \N \N 0 +22 22 22 \N 0 \N \N 1970-01-23 0 +23 \N \N \N 0 \N \N \N 0 +24 24 24 \N 0 \N \N \N 0 +25 str_25 \N str_25 0 \N \N \N 0 MergeTree compact initial insert alter add column 1 3 None -0 0 \N \N \N \N -1 1 \N \N \N \N -2 2 \N \N \N \N +0 0 \N \N \N 0 +1 1 \N \N \N 0 +2 2 \N \N \N 0 insert after alter add column 1 4 String 4 UInt64 7 None -0 0 \N \N \N \N \N -1 1 \N \N \N \N \N -2 2 \N \N \N \N \N -3 3 3 \N 3 \N \N -4 4 4 \N 4 \N \N -5 5 5 \N 5 \N \N -6 6 str_6 str_6 \N \N \N -7 7 str_7 str_7 \N \N \N -8 8 str_8 str_8 \N \N \N -9 9 \N \N \N \N \N -10 10 \N \N \N \N \N -11 11 \N \N \N \N \N -12 12 12 \N 12 \N \N -13 13 str_13 str_13 \N \N \N -14 14 \N \N \N \N \N +0 0 \N \N \N \N 0 +1 1 \N \N \N \N 0 +2 2 \N \N \N \N 0 +3 3 3 \N 3 \N 0 +4 4 4 \N 4 \N 0 +5 5 5 \N 5 \N 0 +6 6 str_6 str_6 \N \N 0 +7 7 str_7 str_7 \N \N 0 +8 8 str_8 str_8 \N \N 0 +9 9 \N \N \N \N 0 +10 10 \N \N \N \N 0 +11 11 \N \N \N \N 0 +12 12 12 \N 12 \N 0 +13 13 str_13 str_13 \N \N 0 +14 14 \N \N \N \N 0 alter modify column 1 7 None 8 String -0 0 \N \N \N \N \N -1 1 \N \N \N \N \N -2 2 \N \N \N \N \N -3 3 3 3 \N \N \N -4 4 4 4 \N \N \N -5 5 5 5 \N \N \N -6 6 str_6 str_6 \N \N \N -7 7 str_7 str_7 \N \N \N -8 8 str_8 str_8 \N \N \N -9 9 \N \N \N \N \N -10 10 \N \N \N \N \N -11 11 \N \N \N \N \N -12 12 12 12 \N \N \N -13 13 str_13 str_13 \N \N \N -14 14 \N \N \N \N \N +0 0 \N \N \N \N 0 +1 1 \N \N \N \N 0 +2 2 \N \N \N \N 0 +3 3 3 3 \N \N 0 +4 4 4 4 \N \N 0 +5 5 5 5 \N \N 0 +6 6 str_6 str_6 \N \N 0 +7 7 str_7 str_7 \N \N 0 +8 8 str_8 str_8 \N \N 0 +9 9 \N \N \N \N 0 +10 10 \N \N \N \N 0 +11 11 \N \N \N \N 0 +12 12 12 12 \N \N 0 +13 13 str_13 str_13 \N \N 0 +14 14 \N \N \N \N 0 insert after alter modify column 1 8 None 11 String -0 0 \N \N \N \N \N -1 1 \N \N \N \N \N -2 2 \N \N \N \N \N -3 3 3 3 \N \N \N -4 4 4 4 \N \N \N -5 5 5 5 \N \N \N -6 6 str_6 str_6 \N \N \N -7 7 str_7 str_7 \N \N \N -8 8 str_8 str_8 \N \N \N -9 9 \N \N \N \N \N -10 10 \N \N \N \N \N -11 11 \N \N \N \N \N -12 12 12 12 \N \N \N -13 13 str_13 str_13 \N \N \N -14 14 \N \N \N \N \N -15 15 \N \N \N \N \N -16 16 16 16 \N \N \N -17 17 str_17 str_17 \N \N \N -18 18 1970-01-19 1970-01-19 \N \N \N +0 0 \N \N \N \N 0 +1 1 \N \N \N \N 0 +2 2 \N \N \N \N 0 +3 3 3 3 \N \N 0 +4 4 4 4 \N \N 0 +5 5 5 5 \N \N 0 +6 6 str_6 str_6 \N \N 0 +7 7 str_7 str_7 \N \N 0 +8 8 str_8 str_8 \N \N 0 +9 9 \N \N \N \N 0 +10 10 \N \N \N \N 0 +11 11 \N \N \N \N 0 +12 12 12 12 \N \N 0 +13 13 str_13 str_13 \N \N 0 +14 14 \N \N \N \N 0 +15 15 \N \N \N \N 0 +16 16 16 16 \N \N 0 +17 17 str_17 str_17 \N \N 0 +18 18 1970-01-19 1970-01-19 \N \N 0 alter modify column 2 8 None 11 String -0 0 \N \N \N \N \N -1 1 \N \N \N \N \N -2 2 \N \N \N \N \N -3 3 3 3 \N \N \N -4 4 4 4 \N \N \N -5 5 5 5 \N \N \N -6 6 str_6 str_6 \N \N \N -7 7 str_7 str_7 \N \N \N -8 8 str_8 str_8 \N \N \N -9 9 \N \N \N \N \N -10 10 \N \N \N \N \N -11 11 \N \N \N \N \N -12 12 12 12 \N \N \N -13 13 str_13 str_13 \N \N \N -14 14 \N \N \N \N \N -15 15 \N \N \N \N \N -16 16 16 16 \N \N \N -17 17 str_17 str_17 \N \N \N -18 18 1970-01-19 1970-01-19 \N \N \N +0 0 \N \N \N \N 0 +1 1 \N \N \N \N 0 +2 2 \N \N \N \N 0 +3 3 3 3 \N \N 0 +4 4 4 4 \N \N 0 +5 5 5 5 \N \N 0 +6 6 str_6 str_6 \N \N 0 +7 7 str_7 str_7 \N \N 0 +8 8 str_8 str_8 \N \N 0 +9 9 \N \N \N \N 0 +10 10 \N \N \N \N 0 +11 11 \N \N \N \N 0 +12 12 12 12 \N \N 0 +13 13 str_13 str_13 \N \N 0 +14 14 \N \N \N \N 0 +15 15 \N \N \N \N 0 +16 16 16 16 \N \N 0 +17 17 str_17 str_17 \N \N 0 +18 18 1970-01-19 1970-01-19 \N \N 0 insert after alter modify column 2 1 Date 1 UInt64 9 None 12 String -0 0 \N \N \N \N \N -1 1 \N \N \N \N \N -2 2 \N \N \N \N \N -3 3 3 3 \N \N \N -4 4 4 4 \N \N \N -5 5 5 5 \N \N \N -6 6 str_6 str_6 \N \N \N -7 7 str_7 str_7 \N \N \N -8 8 str_8 str_8 \N \N \N -9 9 \N \N \N \N \N -10 10 \N \N \N \N \N -11 11 \N \N \N \N \N -12 12 12 12 \N \N \N -13 13 str_13 str_13 \N \N \N -14 14 \N \N \N \N \N -15 15 \N \N \N \N \N -16 16 16 16 \N \N \N -17 17 str_17 str_17 \N \N \N -18 18 1970-01-19 1970-01-19 \N \N \N -19 19 \N \N \N \N \N -20 20 20 \N 20 \N \N -21 21 str_21 str_21 \N \N \N -22 22 1970-01-23 \N \N 1970-01-23 \N +0 0 \N \N \N \N 0 +1 1 \N \N \N \N 0 +2 2 \N \N \N \N 0 +3 3 3 3 \N \N 0 +4 4 4 4 \N \N 0 +5 5 5 5 \N \N 0 +6 6 str_6 str_6 \N \N 0 +7 7 str_7 str_7 \N \N 0 +8 8 str_8 str_8 \N \N 0 +9 9 \N \N \N \N 0 +10 10 \N \N \N \N 0 +11 11 \N \N \N \N 0 +12 12 12 12 \N \N 0 +13 13 str_13 str_13 \N \N 0 +14 14 \N \N \N \N 0 +15 15 \N \N \N \N 0 +16 16 16 16 \N \N 0 +17 17 str_17 str_17 \N \N 0 +18 18 1970-01-19 1970-01-19 \N \N 0 +19 19 \N \N \N \N 0 +20 20 20 \N 20 \N 0 +21 21 str_21 str_21 \N \N 0 +22 22 1970-01-23 \N \N 1970-01-23 0 alter modify column 3 1 Date 1 UInt64 9 None 12 String -0 0 0 \N \N \N \N \N \N -1 1 1 \N \N \N \N \N \N -2 2 2 \N \N \N \N \N \N -3 3 3 \N \N 3 \N \N \N -4 4 4 \N \N 4 \N \N \N -5 5 5 \N \N 5 \N \N \N -6 6 6 \N \N str_6 \N \N \N -7 7 7 \N \N str_7 \N \N \N -8 8 8 \N \N str_8 \N \N \N -9 9 9 \N \N \N \N \N \N -10 10 10 \N \N \N \N \N \N -11 11 11 \N \N \N \N \N \N -12 12 12 \N \N 12 \N \N \N -13 13 13 \N \N str_13 \N \N \N -14 14 14 \N \N \N \N \N \N -15 15 15 \N \N \N \N \N \N -16 16 16 \N \N 16 \N \N \N -17 17 17 \N \N str_17 \N \N \N -18 18 18 \N \N 1970-01-19 \N \N \N -19 19 19 \N \N \N \N \N \N -20 20 20 \N \N \N 20 \N \N -21 21 21 \N \N str_21 \N \N \N -22 22 22 \N \N \N \N 1970-01-23 \N +0 0 0 \N 0 \N \N \N 0 +1 1 1 \N 0 \N \N \N 0 +2 2 2 \N 0 \N \N \N 0 +3 3 3 \N 0 3 \N \N 0 +4 4 4 \N 0 4 \N \N 0 +5 5 5 \N 0 5 \N \N 0 +6 6 6 \N 0 str_6 \N \N 0 +7 7 7 \N 0 str_7 \N \N 0 +8 8 8 \N 0 str_8 \N \N 0 +9 9 9 \N 0 \N \N \N 0 +10 10 10 \N 0 \N \N \N 0 +11 11 11 \N 0 \N \N \N 0 +12 12 12 \N 0 12 \N \N 0 +13 13 13 \N 0 str_13 \N \N 0 +14 14 14 \N 0 \N \N \N 0 +15 15 15 \N 0 \N \N \N 0 +16 16 16 \N 0 16 \N \N 0 +17 17 17 \N 0 str_17 \N \N 0 +18 18 18 \N 0 1970-01-19 \N \N 0 +19 19 19 \N 0 \N \N \N 0 +20 20 20 \N 0 \N 20 \N 0 +21 21 21 \N 0 str_21 \N \N 0 +22 22 22 \N 0 \N \N 1970-01-23 0 insert after alter modify column 3 1 Date 1 UInt64 12 None 12 String -0 0 0 \N \N \N \N \N \N -1 1 1 \N \N \N \N \N \N -2 2 2 \N \N \N \N \N \N -3 3 3 \N \N 3 \N \N \N -4 4 4 \N \N 4 \N \N \N -5 5 5 \N \N 5 \N \N \N -6 6 6 \N \N str_6 \N \N \N -7 7 7 \N \N str_7 \N \N \N -8 8 8 \N \N str_8 \N \N \N -9 9 9 \N \N \N \N \N \N -10 10 10 \N \N \N \N \N \N -11 11 11 \N \N \N \N \N \N -12 12 12 \N \N 12 \N \N \N -13 13 13 \N \N str_13 \N \N \N -14 14 14 \N \N \N \N \N \N -15 15 15 \N \N \N \N \N \N -16 16 16 \N \N 16 \N \N \N -17 17 17 \N \N str_17 \N \N \N -18 18 18 \N \N 1970-01-19 \N \N \N -19 19 19 \N \N \N \N \N \N -20 20 20 \N \N \N 20 \N \N -21 21 21 \N \N str_21 \N \N \N -22 22 22 \N \N \N \N 1970-01-23 \N -23 \N \N \N \N \N \N \N \N -24 24 24 \N \N \N \N \N \N -25 str_25 \N str_25 \N \N \N \N \N +0 0 0 \N 0 \N \N \N 0 +1 1 1 \N 0 \N \N \N 0 +2 2 2 \N 0 \N \N \N 0 +3 3 3 \N 0 3 \N \N 0 +4 4 4 \N 0 4 \N \N 0 +5 5 5 \N 0 5 \N \N 0 +6 6 6 \N 0 str_6 \N \N 0 +7 7 7 \N 0 str_7 \N \N 0 +8 8 8 \N 0 str_8 \N \N 0 +9 9 9 \N 0 \N \N \N 0 +10 10 10 \N 0 \N \N \N 0 +11 11 11 \N 0 \N \N \N 0 +12 12 12 \N 0 12 \N \N 0 +13 13 13 \N 0 str_13 \N \N 0 +14 14 14 \N 0 \N \N \N 0 +15 15 15 \N 0 \N \N \N 0 +16 16 16 \N 0 16 \N \N 0 +17 17 17 \N 0 str_17 \N \N 0 +18 18 18 \N 0 1970-01-19 \N \N 0 +19 19 19 \N 0 \N \N \N 0 +20 20 20 \N 0 \N 20 \N 0 +21 21 21 \N 0 str_21 \N \N 0 +22 22 22 \N 0 \N \N 1970-01-23 0 +23 \N \N \N 0 \N \N \N 0 +24 24 24 \N 0 \N \N \N 0 +25 str_25 \N str_25 0 \N \N \N 0 MergeTree wide initial insert alter add column 1 3 None -0 0 \N \N \N \N -1 1 \N \N \N \N -2 2 \N \N \N \N +0 0 \N \N \N 0 +1 1 \N \N \N 0 +2 2 \N \N \N 0 insert after alter add column 1 4 String 4 UInt64 7 None -0 0 \N \N \N \N \N -1 1 \N \N \N \N \N -2 2 \N \N \N \N \N -3 3 3 \N 3 \N \N -4 4 4 \N 4 \N \N -5 5 5 \N 5 \N \N -6 6 str_6 str_6 \N \N \N -7 7 str_7 str_7 \N \N \N -8 8 str_8 str_8 \N \N \N -9 9 \N \N \N \N \N -10 10 \N \N \N \N \N -11 11 \N \N \N \N \N -12 12 12 \N 12 \N \N -13 13 str_13 str_13 \N \N \N -14 14 \N \N \N \N \N +0 0 \N \N \N \N 0 +1 1 \N \N \N \N 0 +2 2 \N \N \N \N 0 +3 3 3 \N 3 \N 0 +4 4 4 \N 4 \N 0 +5 5 5 \N 5 \N 0 +6 6 str_6 str_6 \N \N 0 +7 7 str_7 str_7 \N \N 0 +8 8 str_8 str_8 \N \N 0 +9 9 \N \N \N \N 0 +10 10 \N \N \N \N 0 +11 11 \N \N \N \N 0 +12 12 12 \N 12 \N 0 +13 13 str_13 str_13 \N \N 0 +14 14 \N \N \N \N 0 alter modify column 1 7 None 8 String -0 0 \N \N \N \N \N -1 1 \N \N \N \N \N -2 2 \N \N \N \N \N -3 3 3 3 \N \N \N -4 4 4 4 \N \N \N -5 5 5 5 \N \N \N -6 6 str_6 str_6 \N \N \N -7 7 str_7 str_7 \N \N \N -8 8 str_8 str_8 \N \N \N -9 9 \N \N \N \N \N -10 10 \N \N \N \N \N -11 11 \N \N \N \N \N -12 12 12 12 \N \N \N -13 13 str_13 str_13 \N \N \N -14 14 \N \N \N \N \N +0 0 \N \N \N \N 0 +1 1 \N \N \N \N 0 +2 2 \N \N \N \N 0 +3 3 3 3 \N \N 0 +4 4 4 4 \N \N 0 +5 5 5 5 \N \N 0 +6 6 str_6 str_6 \N \N 0 +7 7 str_7 str_7 \N \N 0 +8 8 str_8 str_8 \N \N 0 +9 9 \N \N \N \N 0 +10 10 \N \N \N \N 0 +11 11 \N \N \N \N 0 +12 12 12 12 \N \N 0 +13 13 str_13 str_13 \N \N 0 +14 14 \N \N \N \N 0 insert after alter modify column 1 8 None 11 String -0 0 \N \N \N \N \N -1 1 \N \N \N \N \N -2 2 \N \N \N \N \N -3 3 3 3 \N \N \N -4 4 4 4 \N \N \N -5 5 5 5 \N \N \N -6 6 str_6 str_6 \N \N \N -7 7 str_7 str_7 \N \N \N -8 8 str_8 str_8 \N \N \N -9 9 \N \N \N \N \N -10 10 \N \N \N \N \N -11 11 \N \N \N \N \N -12 12 12 12 \N \N \N -13 13 str_13 str_13 \N \N \N -14 14 \N \N \N \N \N -15 15 \N \N \N \N \N -16 16 16 16 \N \N \N -17 17 str_17 str_17 \N \N \N -18 18 1970-01-19 1970-01-19 \N \N \N +0 0 \N \N \N \N 0 +1 1 \N \N \N \N 0 +2 2 \N \N \N \N 0 +3 3 3 3 \N \N 0 +4 4 4 4 \N \N 0 +5 5 5 5 \N \N 0 +6 6 str_6 str_6 \N \N 0 +7 7 str_7 str_7 \N \N 0 +8 8 str_8 str_8 \N \N 0 +9 9 \N \N \N \N 0 +10 10 \N \N \N \N 0 +11 11 \N \N \N \N 0 +12 12 12 12 \N \N 0 +13 13 str_13 str_13 \N \N 0 +14 14 \N \N \N \N 0 +15 15 \N \N \N \N 0 +16 16 16 16 \N \N 0 +17 17 str_17 str_17 \N \N 0 +18 18 1970-01-19 1970-01-19 \N \N 0 alter modify column 2 8 None 11 String -0 0 \N \N \N \N \N -1 1 \N \N \N \N \N -2 2 \N \N \N \N \N -3 3 3 3 \N \N \N -4 4 4 4 \N \N \N -5 5 5 5 \N \N \N -6 6 str_6 str_6 \N \N \N -7 7 str_7 str_7 \N \N \N -8 8 str_8 str_8 \N \N \N -9 9 \N \N \N \N \N -10 10 \N \N \N \N \N -11 11 \N \N \N \N \N -12 12 12 12 \N \N \N -13 13 str_13 str_13 \N \N \N -14 14 \N \N \N \N \N -15 15 \N \N \N \N \N -16 16 16 16 \N \N \N -17 17 str_17 str_17 \N \N \N -18 18 1970-01-19 1970-01-19 \N \N \N +0 0 \N \N \N \N 0 +1 1 \N \N \N \N 0 +2 2 \N \N \N \N 0 +3 3 3 3 \N \N 0 +4 4 4 4 \N \N 0 +5 5 5 5 \N \N 0 +6 6 str_6 str_6 \N \N 0 +7 7 str_7 str_7 \N \N 0 +8 8 str_8 str_8 \N \N 0 +9 9 \N \N \N \N 0 +10 10 \N \N \N \N 0 +11 11 \N \N \N \N 0 +12 12 12 12 \N \N 0 +13 13 str_13 str_13 \N \N 0 +14 14 \N \N \N \N 0 +15 15 \N \N \N \N 0 +16 16 16 16 \N \N 0 +17 17 str_17 str_17 \N \N 0 +18 18 1970-01-19 1970-01-19 \N \N 0 insert after alter modify column 2 1 Date 1 UInt64 9 None 12 String -0 0 \N \N \N \N \N -1 1 \N \N \N \N \N -2 2 \N \N \N \N \N -3 3 3 3 \N \N \N -4 4 4 4 \N \N \N -5 5 5 5 \N \N \N -6 6 str_6 str_6 \N \N \N -7 7 str_7 str_7 \N \N \N -8 8 str_8 str_8 \N \N \N -9 9 \N \N \N \N \N -10 10 \N \N \N \N \N -11 11 \N \N \N \N \N -12 12 12 12 \N \N \N -13 13 str_13 str_13 \N \N \N -14 14 \N \N \N \N \N -15 15 \N \N \N \N \N -16 16 16 16 \N \N \N -17 17 str_17 str_17 \N \N \N -18 18 1970-01-19 1970-01-19 \N \N \N -19 19 \N \N \N \N \N -20 20 20 \N 20 \N \N -21 21 str_21 str_21 \N \N \N -22 22 1970-01-23 \N \N 1970-01-23 \N +0 0 \N \N \N \N 0 +1 1 \N \N \N \N 0 +2 2 \N \N \N \N 0 +3 3 3 3 \N \N 0 +4 4 4 4 \N \N 0 +5 5 5 5 \N \N 0 +6 6 str_6 str_6 \N \N 0 +7 7 str_7 str_7 \N \N 0 +8 8 str_8 str_8 \N \N 0 +9 9 \N \N \N \N 0 +10 10 \N \N \N \N 0 +11 11 \N \N \N \N 0 +12 12 12 12 \N \N 0 +13 13 str_13 str_13 \N \N 0 +14 14 \N \N \N \N 0 +15 15 \N \N \N \N 0 +16 16 16 16 \N \N 0 +17 17 str_17 str_17 \N \N 0 +18 18 1970-01-19 1970-01-19 \N \N 0 +19 19 \N \N \N \N 0 +20 20 20 \N 20 \N 0 +21 21 str_21 str_21 \N \N 0 +22 22 1970-01-23 \N \N 1970-01-23 0 alter modify column 3 1 Date 1 UInt64 9 None 12 String -0 0 0 \N \N \N \N \N \N -1 1 1 \N \N \N \N \N \N -2 2 2 \N \N \N \N \N \N -3 3 3 \N \N 3 \N \N \N -4 4 4 \N \N 4 \N \N \N -5 5 5 \N \N 5 \N \N \N -6 6 6 \N \N str_6 \N \N \N -7 7 7 \N \N str_7 \N \N \N -8 8 8 \N \N str_8 \N \N \N -9 9 9 \N \N \N \N \N \N -10 10 10 \N \N \N \N \N \N -11 11 11 \N \N \N \N \N \N -12 12 12 \N \N 12 \N \N \N -13 13 13 \N \N str_13 \N \N \N -14 14 14 \N \N \N \N \N \N -15 15 15 \N \N \N \N \N \N -16 16 16 \N \N 16 \N \N \N -17 17 17 \N \N str_17 \N \N \N -18 18 18 \N \N 1970-01-19 \N \N \N -19 19 19 \N \N \N \N \N \N -20 20 20 \N \N \N 20 \N \N -21 21 21 \N \N str_21 \N \N \N -22 22 22 \N \N \N \N 1970-01-23 \N +0 0 0 \N 0 \N \N \N 0 +1 1 1 \N 0 \N \N \N 0 +2 2 2 \N 0 \N \N \N 0 +3 3 3 \N 0 3 \N \N 0 +4 4 4 \N 0 4 \N \N 0 +5 5 5 \N 0 5 \N \N 0 +6 6 6 \N 0 str_6 \N \N 0 +7 7 7 \N 0 str_7 \N \N 0 +8 8 8 \N 0 str_8 \N \N 0 +9 9 9 \N 0 \N \N \N 0 +10 10 10 \N 0 \N \N \N 0 +11 11 11 \N 0 \N \N \N 0 +12 12 12 \N 0 12 \N \N 0 +13 13 13 \N 0 str_13 \N \N 0 +14 14 14 \N 0 \N \N \N 0 +15 15 15 \N 0 \N \N \N 0 +16 16 16 \N 0 16 \N \N 0 +17 17 17 \N 0 str_17 \N \N 0 +18 18 18 \N 0 1970-01-19 \N \N 0 +19 19 19 \N 0 \N \N \N 0 +20 20 20 \N 0 \N 20 \N 0 +21 21 21 \N 0 str_21 \N \N 0 +22 22 22 \N 0 \N \N 1970-01-23 0 insert after alter modify column 3 1 Date 1 UInt64 12 None 12 String -0 0 0 \N \N \N \N \N \N -1 1 1 \N \N \N \N \N \N -2 2 2 \N \N \N \N \N \N -3 3 3 \N \N 3 \N \N \N -4 4 4 \N \N 4 \N \N \N -5 5 5 \N \N 5 \N \N \N -6 6 6 \N \N str_6 \N \N \N -7 7 7 \N \N str_7 \N \N \N -8 8 8 \N \N str_8 \N \N \N -9 9 9 \N \N \N \N \N \N -10 10 10 \N \N \N \N \N \N -11 11 11 \N \N \N \N \N \N -12 12 12 \N \N 12 \N \N \N -13 13 13 \N \N str_13 \N \N \N -14 14 14 \N \N \N \N \N \N -15 15 15 \N \N \N \N \N \N -16 16 16 \N \N 16 \N \N \N -17 17 17 \N \N str_17 \N \N \N -18 18 18 \N \N 1970-01-19 \N \N \N -19 19 19 \N \N \N \N \N \N -20 20 20 \N \N \N 20 \N \N -21 21 21 \N \N str_21 \N \N \N -22 22 22 \N \N \N \N 1970-01-23 \N -23 \N \N \N \N \N \N \N \N -24 24 24 \N \N \N \N \N \N -25 str_25 \N str_25 \N \N \N \N \N +0 0 0 \N 0 \N \N \N 0 +1 1 1 \N 0 \N \N \N 0 +2 2 2 \N 0 \N \N \N 0 +3 3 3 \N 0 3 \N \N 0 +4 4 4 \N 0 4 \N \N 0 +5 5 5 \N 0 5 \N \N 0 +6 6 6 \N 0 str_6 \N \N 0 +7 7 7 \N 0 str_7 \N \N 0 +8 8 8 \N 0 str_8 \N \N 0 +9 9 9 \N 0 \N \N \N 0 +10 10 10 \N 0 \N \N \N 0 +11 11 11 \N 0 \N \N \N 0 +12 12 12 \N 0 12 \N \N 0 +13 13 13 \N 0 str_13 \N \N 0 +14 14 14 \N 0 \N \N \N 0 +15 15 15 \N 0 \N \N \N 0 +16 16 16 \N 0 16 \N \N 0 +17 17 17 \N 0 str_17 \N \N 0 +18 18 18 \N 0 1970-01-19 \N \N 0 +19 19 19 \N 0 \N \N \N 0 +20 20 20 \N 0 \N 20 \N 0 +21 21 21 \N 0 str_21 \N \N 0 +22 22 22 \N 0 \N \N 1970-01-23 0 +23 \N \N \N 0 \N \N \N 0 +24 24 24 \N 0 \N \N \N 0 +25 str_25 \N str_25 0 \N \N \N 0 diff --git a/tests/queries/0_stateless/03040_dynamic_type_alters_2.reference b/tests/queries/0_stateless/03040_dynamic_type_alters_2.reference index 18a181464e9..f7c00bd8c44 100644 --- a/tests/queries/0_stateless/03040_dynamic_type_alters_2.reference +++ b/tests/queries/0_stateless/03040_dynamic_type_alters_2.reference @@ -2,181 +2,181 @@ MergeTree compact initial insert alter add column 3 None -0 0 \N \N \N \N -1 1 \N \N \N \N -2 2 \N \N \N \N +0 0 \N \N \N 0 +1 1 \N \N \N 0 +2 2 \N \N \N 0 insert after alter add column 1 4 String 4 UInt64 7 None -0 0 \N \N \N \N \N -1 1 \N \N \N \N \N -2 2 \N \N \N \N \N -3 3 3 \N 3 \N \N -4 4 4 \N 4 \N \N -5 5 5 \N 5 \N \N -6 6 str_6 str_6 \N \N \N -7 7 str_7 str_7 \N \N \N -8 8 str_8 str_8 \N \N \N -9 9 \N \N \N \N \N -10 10 \N \N \N \N \N -11 11 \N \N \N \N \N -12 12 12 \N 12 \N \N -13 13 str_13 str_13 \N \N \N -14 14 \N \N \N \N \N +0 0 \N \N \N \N 0 +1 1 \N \N \N \N 0 +2 2 \N \N \N \N 0 +3 3 3 \N 3 \N 0 +4 4 4 \N 4 \N 0 +5 5 5 \N 5 \N 0 +6 6 str_6 str_6 \N \N 0 +7 7 str_7 str_7 \N \N 0 +8 8 str_8 str_8 \N \N 0 +9 9 \N \N \N \N 0 +10 10 \N \N \N \N 0 +11 11 \N \N \N \N 0 +12 12 12 \N 12 \N 0 +13 13 str_13 str_13 \N \N 0 +14 14 \N \N \N \N 0 alter rename column 1 4 String 4 UInt64 7 None -0 0 \N \N \N \N \N -1 1 \N \N \N \N \N -2 2 \N \N \N \N \N -3 3 3 \N 3 \N \N -4 4 4 \N 4 \N \N -5 5 5 \N 5 \N \N -6 6 str_6 str_6 \N \N \N -7 7 str_7 str_7 \N \N \N -8 8 str_8 str_8 \N \N \N -9 9 \N \N \N \N \N -10 10 \N \N \N \N \N -11 11 \N \N \N \N \N -12 12 12 \N 12 \N \N -13 13 str_13 str_13 \N \N \N -14 14 \N \N \N \N \N +0 0 \N \N \N \N 0 +1 1 \N \N \N \N 0 +2 2 \N \N \N \N 0 +3 3 3 \N 3 \N 0 +4 4 4 \N 4 \N 0 +5 5 5 \N 5 \N 0 +6 6 str_6 str_6 \N \N 0 +7 7 str_7 str_7 \N \N 0 +8 8 str_8 str_8 \N \N 0 +9 9 \N \N \N \N 0 +10 10 \N \N \N \N 0 +11 11 \N \N \N \N 0 +12 12 12 \N 12 \N 0 +13 13 str_13 str_13 \N \N 0 +14 14 \N \N \N \N 0 insert nested dynamic 3 Array(Dynamic) 4 String 4 UInt64 7 None -0 0 \N \N \N \N \N [] [] [] -1 1 \N \N \N \N \N [] [] [] -2 2 \N \N \N \N \N [] [] [] -3 3 3 \N 3 \N \N [] [] [] -4 4 4 \N 4 \N \N [] [] [] -5 5 5 \N 5 \N \N [] [] [] -6 6 str_6 str_6 \N \N \N [] [] [] -7 7 str_7 str_7 \N \N \N [] [] [] -8 8 str_8 str_8 \N \N \N [] [] [] -9 9 \N \N \N \N \N [] [] [] -10 10 \N \N \N \N \N [] [] [] -11 11 \N \N \N \N \N [] [] [] -12 12 12 \N 12 \N \N [] [] [] -13 13 str_13 str_13 \N \N \N [] [] [] -14 14 \N \N \N \N \N [] [] [] -15 15 [15] \N \N \N \N [15] [NULL] [NULL] -16 16 ['str_16'] \N \N \N \N [NULL] ['str_16'] [NULL] -17 17 [17] \N \N \N \N [17] [NULL] [NULL] +0 0 \N \N \N \N 0 [] [] [] +1 1 \N \N \N \N 0 [] [] [] +2 2 \N \N \N \N 0 [] [] [] +3 3 3 \N 3 \N 0 [] [] [] +4 4 4 \N 4 \N 0 [] [] [] +5 5 5 \N 5 \N 0 [] [] [] +6 6 str_6 str_6 \N \N 0 [] [] [] +7 7 str_7 str_7 \N \N 0 [] [] [] +8 8 str_8 str_8 \N \N 0 [] [] [] +9 9 \N \N \N \N 0 [] [] [] +10 10 \N \N \N \N 0 [] [] [] +11 11 \N \N \N \N 0 [] [] [] +12 12 12 \N 12 \N 0 [] [] [] +13 13 str_13 str_13 \N \N 0 [] [] [] +14 14 \N \N \N \N 0 [] [] [] +15 15 [15] \N \N \N 0 [15] [NULL] [NULL] +16 16 ['str_16'] \N \N \N 0 [NULL] ['str_16'] [NULL] +17 17 [17] \N \N \N 0 [17] [NULL] [NULL] alter rename column 2 3 Array(Dynamic) 4 String 4 UInt64 7 None -0 0 \N \N \N \N \N [] [] [] -1 1 \N \N \N \N \N [] [] [] -2 2 \N \N \N \N \N [] [] [] -3 3 3 \N 3 \N \N [] [] [] -4 4 4 \N 4 \N \N [] [] [] -5 5 5 \N 5 \N \N [] [] [] -6 6 str_6 str_6 \N \N \N [] [] [] -7 7 str_7 str_7 \N \N \N [] [] [] -8 8 str_8 str_8 \N \N \N [] [] [] -9 9 \N \N \N \N \N [] [] [] -10 10 \N \N \N \N \N [] [] [] -11 11 \N \N \N \N \N [] [] [] -12 12 12 \N 12 \N \N [] [] [] -13 13 str_13 str_13 \N \N \N [] [] [] -14 14 \N \N \N \N \N [] [] [] -15 15 [15] \N \N \N \N [15] [NULL] [NULL] -16 16 ['str_16'] \N \N \N \N [NULL] ['str_16'] [NULL] -17 17 [17] \N \N \N \N [17] [NULL] [NULL] +0 0 \N \N \N \N 0 [] [] [] +1 1 \N \N \N \N 0 [] [] [] +2 2 \N \N \N \N 0 [] [] [] +3 3 3 \N 3 \N 0 [] [] [] +4 4 4 \N 4 \N 0 [] [] [] +5 5 5 \N 5 \N 0 [] [] [] +6 6 str_6 str_6 \N \N 0 [] [] [] +7 7 str_7 str_7 \N \N 0 [] [] [] +8 8 str_8 str_8 \N \N 0 [] [] [] +9 9 \N \N \N \N 0 [] [] [] +10 10 \N \N \N \N 0 [] [] [] +11 11 \N \N \N \N 0 [] [] [] +12 12 12 \N 12 \N 0 [] [] [] +13 13 str_13 str_13 \N \N 0 [] [] [] +14 14 \N \N \N \N 0 [] [] [] +15 15 [15] \N \N \N 0 [15] [NULL] [NULL] +16 16 ['str_16'] \N \N \N 0 [NULL] ['str_16'] [NULL] +17 17 [17] \N \N \N 0 [17] [NULL] [NULL] MergeTree wide initial insert alter add column 3 None -0 0 \N \N \N \N -1 1 \N \N \N \N -2 2 \N \N \N \N +0 0 \N \N \N 0 +1 1 \N \N \N 0 +2 2 \N \N \N 0 insert after alter add column 1 4 String 4 UInt64 7 None -0 0 \N \N \N \N \N -1 1 \N \N \N \N \N -2 2 \N \N \N \N \N -3 3 3 \N 3 \N \N -4 4 4 \N 4 \N \N -5 5 5 \N 5 \N \N -6 6 str_6 str_6 \N \N \N -7 7 str_7 str_7 \N \N \N -8 8 str_8 str_8 \N \N \N -9 9 \N \N \N \N \N -10 10 \N \N \N \N \N -11 11 \N \N \N \N \N -12 12 12 \N 12 \N \N -13 13 str_13 str_13 \N \N \N -14 14 \N \N \N \N \N +0 0 \N \N \N \N 0 +1 1 \N \N \N \N 0 +2 2 \N \N \N \N 0 +3 3 3 \N 3 \N 0 +4 4 4 \N 4 \N 0 +5 5 5 \N 5 \N 0 +6 6 str_6 str_6 \N \N 0 +7 7 str_7 str_7 \N \N 0 +8 8 str_8 str_8 \N \N 0 +9 9 \N \N \N \N 0 +10 10 \N \N \N \N 0 +11 11 \N \N \N \N 0 +12 12 12 \N 12 \N 0 +13 13 str_13 str_13 \N \N 0 +14 14 \N \N \N \N 0 alter rename column 1 4 String 4 UInt64 7 None -0 0 \N \N \N \N \N -1 1 \N \N \N \N \N -2 2 \N \N \N \N \N -3 3 3 \N 3 \N \N -4 4 4 \N 4 \N \N -5 5 5 \N 5 \N \N -6 6 str_6 str_6 \N \N \N -7 7 str_7 str_7 \N \N \N -8 8 str_8 str_8 \N \N \N -9 9 \N \N \N \N \N -10 10 \N \N \N \N \N -11 11 \N \N \N \N \N -12 12 12 \N 12 \N \N -13 13 str_13 str_13 \N \N \N -14 14 \N \N \N \N \N +0 0 \N \N \N \N 0 +1 1 \N \N \N \N 0 +2 2 \N \N \N \N 0 +3 3 3 \N 3 \N 0 +4 4 4 \N 4 \N 0 +5 5 5 \N 5 \N 0 +6 6 str_6 str_6 \N \N 0 +7 7 str_7 str_7 \N \N 0 +8 8 str_8 str_8 \N \N 0 +9 9 \N \N \N \N 0 +10 10 \N \N \N \N 0 +11 11 \N \N \N \N 0 +12 12 12 \N 12 \N 0 +13 13 str_13 str_13 \N \N 0 +14 14 \N \N \N \N 0 insert nested dynamic 3 Array(Dynamic) 4 String 4 UInt64 7 None -0 0 \N \N \N \N \N [] [] [] -1 1 \N \N \N \N \N [] [] [] -2 2 \N \N \N \N \N [] [] [] -3 3 3 \N 3 \N \N [] [] [] -4 4 4 \N 4 \N \N [] [] [] -5 5 5 \N 5 \N \N [] [] [] -6 6 str_6 str_6 \N \N \N [] [] [] -7 7 str_7 str_7 \N \N \N [] [] [] -8 8 str_8 str_8 \N \N \N [] [] [] -9 9 \N \N \N \N \N [] [] [] -10 10 \N \N \N \N \N [] [] [] -11 11 \N \N \N \N \N [] [] [] -12 12 12 \N 12 \N \N [] [] [] -13 13 str_13 str_13 \N \N \N [] [] [] -14 14 \N \N \N \N \N [] [] [] -15 15 [15] \N \N \N \N [15] [NULL] [NULL] -16 16 ['str_16'] \N \N \N \N [NULL] ['str_16'] [NULL] -17 17 [17] \N \N \N \N [17] [NULL] [NULL] +0 0 \N \N \N \N 0 [] [] [] +1 1 \N \N \N \N 0 [] [] [] +2 2 \N \N \N \N 0 [] [] [] +3 3 3 \N 3 \N 0 [] [] [] +4 4 4 \N 4 \N 0 [] [] [] +5 5 5 \N 5 \N 0 [] [] [] +6 6 str_6 str_6 \N \N 0 [] [] [] +7 7 str_7 str_7 \N \N 0 [] [] [] +8 8 str_8 str_8 \N \N 0 [] [] [] +9 9 \N \N \N \N 0 [] [] [] +10 10 \N \N \N \N 0 [] [] [] +11 11 \N \N \N \N 0 [] [] [] +12 12 12 \N 12 \N 0 [] [] [] +13 13 str_13 str_13 \N \N 0 [] [] [] +14 14 \N \N \N \N 0 [] [] [] +15 15 [15] \N \N \N 0 [15] [NULL] [NULL] +16 16 ['str_16'] \N \N \N 0 [NULL] ['str_16'] [NULL] +17 17 [17] \N \N \N 0 [17] [NULL] [NULL] alter rename column 2 3 Array(Dynamic) 4 String 4 UInt64 7 None -0 0 \N \N \N \N \N [] [] [] -1 1 \N \N \N \N \N [] [] [] -2 2 \N \N \N \N \N [] [] [] -3 3 3 \N 3 \N \N [] [] [] -4 4 4 \N 4 \N \N [] [] [] -5 5 5 \N 5 \N \N [] [] [] -6 6 str_6 str_6 \N \N \N [] [] [] -7 7 str_7 str_7 \N \N \N [] [] [] -8 8 str_8 str_8 \N \N \N [] [] [] -9 9 \N \N \N \N \N [] [] [] -10 10 \N \N \N \N \N [] [] [] -11 11 \N \N \N \N \N [] [] [] -12 12 12 \N 12 \N \N [] [] [] -13 13 str_13 str_13 \N \N \N [] [] [] -14 14 \N \N \N \N \N [] [] [] -15 15 [15] \N \N \N \N [15] [NULL] [NULL] -16 16 ['str_16'] \N \N \N \N [NULL] ['str_16'] [NULL] -17 17 [17] \N \N \N \N [17] [NULL] [NULL] +0 0 \N \N \N \N 0 [] [] [] +1 1 \N \N \N \N 0 [] [] [] +2 2 \N \N \N \N 0 [] [] [] +3 3 3 \N 3 \N 0 [] [] [] +4 4 4 \N 4 \N 0 [] [] [] +5 5 5 \N 5 \N 0 [] [] [] +6 6 str_6 str_6 \N \N 0 [] [] [] +7 7 str_7 str_7 \N \N 0 [] [] [] +8 8 str_8 str_8 \N \N 0 [] [] [] +9 9 \N \N \N \N 0 [] [] [] +10 10 \N \N \N \N 0 [] [] [] +11 11 \N \N \N \N 0 [] [] [] +12 12 12 \N 12 \N 0 [] [] [] +13 13 str_13 str_13 \N \N 0 [] [] [] +14 14 \N \N \N \N 0 [] [] [] +15 15 [15] \N \N \N 0 [15] [NULL] [NULL] +16 16 ['str_16'] \N \N \N 0 [NULL] ['str_16'] [NULL] +17 17 [17] \N \N \N 0 [17] [NULL] [NULL] diff --git a/tests/queries/0_stateless/03041_dynamic_type_check_table.reference b/tests/queries/0_stateless/03041_dynamic_type_check_table.reference index b1ea186a917..0dab4ea0d20 100644 --- a/tests/queries/0_stateless/03041_dynamic_type_check_table.reference +++ b/tests/queries/0_stateless/03041_dynamic_type_check_table.reference @@ -2,55 +2,55 @@ MergeTree compact initial insert alter add column 3 None -0 0 \N \N \N \N -1 1 \N \N \N \N -2 2 \N \N \N \N +0 0 \N \N \N 0 +1 1 \N \N \N 0 +2 2 \N \N \N 0 insert after alter add column 4 String 4 UInt64 7 None -0 0 \N \N \N \N \N -1 1 \N \N \N \N \N -2 2 \N \N \N \N \N -3 3 3 \N 3 \N \N -4 4 4 \N 4 \N \N -5 5 5 \N 5 \N \N -6 6 str_6 str_6 \N \N \N -7 7 str_7 str_7 \N \N \N -8 8 str_8 str_8 \N \N \N -9 9 \N \N \N \N \N -10 10 \N \N \N \N \N -11 11 \N \N \N \N \N -12 12 12 \N 12 \N \N -13 13 str_13 str_13 \N \N \N -14 14 \N \N \N \N \N +0 0 \N \N \N \N 0 +1 1 \N \N \N \N 0 +2 2 \N \N \N \N 0 +3 3 3 \N 3 \N 0 +4 4 4 \N 4 \N 0 +5 5 5 \N 5 \N 0 +6 6 str_6 str_6 \N \N 0 +7 7 str_7 str_7 \N \N 0 +8 8 str_8 str_8 \N \N 0 +9 9 \N \N \N \N 0 +10 10 \N \N \N \N 0 +11 11 \N \N \N \N 0 +12 12 12 \N 12 \N 0 +13 13 str_13 str_13 \N \N 0 +14 14 \N \N \N \N 0 check table 1 MergeTree wide initial insert alter add column 3 None -0 0 \N \N \N \N -1 1 \N \N \N \N -2 2 \N \N \N \N +0 0 \N \N \N 0 +1 1 \N \N \N 0 +2 2 \N \N \N 0 insert after alter add column 4 String 4 UInt64 7 None -0 0 \N \N \N \N \N -1 1 \N \N \N \N \N -2 2 \N \N \N \N \N -3 3 3 \N 3 \N \N -4 4 4 \N 4 \N \N -5 5 5 \N 5 \N \N -6 6 str_6 str_6 \N \N \N -7 7 str_7 str_7 \N \N \N -8 8 str_8 str_8 \N \N \N -9 9 \N \N \N \N \N -10 10 \N \N \N \N \N -11 11 \N \N \N \N \N -12 12 12 \N 12 \N \N -13 13 str_13 str_13 \N \N \N -14 14 \N \N \N \N \N +0 0 \N \N \N \N 0 +1 1 \N \N \N \N 0 +2 2 \N \N \N \N 0 +3 3 3 \N 3 \N 0 +4 4 4 \N 4 \N 0 +5 5 5 \N 5 \N 0 +6 6 str_6 str_6 \N \N 0 +7 7 str_7 str_7 \N \N 0 +8 8 str_8 str_8 \N \N 0 +9 9 \N \N \N \N 0 +10 10 \N \N \N \N 0 +11 11 \N \N \N \N 0 +12 12 12 \N 12 \N 0 +13 13 str_13 str_13 \N \N 0 +14 14 \N \N \N \N 0 check table 1 diff --git a/tests/queries/0_stateless/03201_variant_null_map_subcolumn.reference b/tests/queries/0_stateless/03201_variant_null_map_subcolumn.reference new file mode 100644 index 00000000000..8565fe3d0fa --- /dev/null +++ b/tests/queries/0_stateless/03201_variant_null_map_subcolumn.reference @@ -0,0 +1,402 @@ +Memory +test +[] 1 0 0 [] +1 0 1 0 [] +\N 1 1 0 [] +['str_3','str_3','str_3'] 1 0 3 [1,1,1] +4 0 1 0 [] +\N 1 1 0 [] +[6,6,6,6,6,6] 1 0 6 [0,0,0,0,0,0] +7 0 1 0 [] +\N 1 1 0 [] +[NULL,NULL,NULL,NULL,NULL,NULL,NULL,NULL,NULL] 1 0 9 [1,1,1,1,1,1,1,1,1] +10 0 1 0 [] +\N 1 1 0 [] +['str_12','str_12'] 1 0 2 [1,1] +13 0 1 0 [] +\N 1 1 0 [] +[15,15,15,15,15] 1 0 5 [0,0,0,0,0] +16 0 1 0 [] +\N 1 1 0 [] +[NULL,NULL,NULL,NULL,NULL,NULL,NULL,NULL] 1 0 8 [1,1,1,1,1,1,1,1] +19 0 1 0 [] +\N 1 1 0 [] +['str_21'] 1 0 1 [1] +22 0 1 0 [] +\N 1 1 0 [] +[24,24,24,24] 1 0 4 [0,0,0,0] +25 0 1 0 [] +\N 1 1 0 [] +[NULL,NULL,NULL,NULL,NULL,NULL,NULL] 1 0 7 [1,1,1,1,1,1,1] +28 0 1 0 [] +\N 1 1 0 [] +[] 1 0 0 [] +31 0 1 0 [] +\N 1 1 0 [] +[33,33,33] 1 0 3 [0,0,0] +34 0 1 0 [] +\N 1 1 0 [] +1 0 0 [] +0 1 0 [] +1 1 0 [] +1 0 3 [1,1,1] +0 1 0 [] +1 1 0 [] +1 0 6 [0,0,0,0,0,0] +0 1 0 [] +1 1 0 [] +1 0 9 [1,1,1,1,1,1,1,1,1] +0 1 0 [] +1 1 0 [] +1 0 2 [1,1] +0 1 0 [] +1 1 0 [] +1 0 5 [0,0,0,0,0] +0 1 0 [] +1 1 0 [] +1 0 8 [1,1,1,1,1,1,1,1] +0 1 0 [] +1 1 0 [] +1 0 1 [1] +0 1 0 [] +1 1 0 [] +1 0 4 [0,0,0,0] +0 1 0 [] +1 1 0 [] +1 0 7 [1,1,1,1,1,1,1] +0 1 0 [] +1 1 0 [] +1 0 0 [] +0 1 0 [] +1 1 0 [] +1 0 3 [0,0,0] +0 1 0 [] +1 1 0 [] +0 0 [] [] +1 0 [] [] +1 0 [] [] +0 3 [1,1,1] [0,0,0] +1 0 [] [] +1 0 [] [] +0 6 [0,0,0,0,0,0] [1,1,1,1,1,1] +1 0 [] [] +1 0 [] [] +0 9 [1,1,1,1,1,1,1,1,1] [1,1,1,1,1,1,1,1,1] +1 0 [] [] +1 0 [] [] +0 2 [1,1] [0,0] +1 0 [] [] +1 0 [] [] +0 5 [0,0,0,0,0] [1,1,1,1,1] +1 0 [] [] +1 0 [] [] +0 8 [1,1,1,1,1,1,1,1] [1,1,1,1,1,1,1,1] +1 0 [] [] +1 0 [] [] +0 1 [1] [0] +1 0 [] [] +1 0 [] [] +0 4 [0,0,0,0] [1,1,1,1] +1 0 [] [] +1 0 [] [] +0 7 [1,1,1,1,1,1,1] [1,1,1,1,1,1,1] +1 0 [] [] +1 0 [] [] +0 0 [] [] +1 0 [] [] +1 0 [] [] +0 3 [0,0,0] [1,1,1] +1 0 [] [] +1 0 [] [] +0 +2 +3 +5 +6 +8 +9 +11 +12 +14 +15 +17 +18 +20 +21 +23 +24 +26 +27 +29 +30 +32 +33 +35 +MergeTree compact +test +[] 1 0 0 [] +1 0 1 0 [] +\N 1 1 0 [] +['str_3','str_3','str_3'] 1 0 3 [1,1,1] +4 0 1 0 [] +\N 1 1 0 [] +[6,6,6,6,6,6] 1 0 6 [0,0,0,0,0,0] +7 0 1 0 [] +\N 1 1 0 [] +[NULL,NULL,NULL,NULL,NULL,NULL,NULL,NULL,NULL] 1 0 9 [1,1,1,1,1,1,1,1,1] +10 0 1 0 [] +\N 1 1 0 [] +['str_12','str_12'] 1 0 2 [1,1] +13 0 1 0 [] +\N 1 1 0 [] +[15,15,15,15,15] 1 0 5 [0,0,0,0,0] +16 0 1 0 [] +\N 1 1 0 [] +[NULL,NULL,NULL,NULL,NULL,NULL,NULL,NULL] 1 0 8 [1,1,1,1,1,1,1,1] +19 0 1 0 [] +\N 1 1 0 [] +['str_21'] 1 0 1 [1] +22 0 1 0 [] +\N 1 1 0 [] +[24,24,24,24] 1 0 4 [0,0,0,0] +25 0 1 0 [] +\N 1 1 0 [] +[NULL,NULL,NULL,NULL,NULL,NULL,NULL] 1 0 7 [1,1,1,1,1,1,1] +28 0 1 0 [] +\N 1 1 0 [] +[] 1 0 0 [] +31 0 1 0 [] +\N 1 1 0 [] +[33,33,33] 1 0 3 [0,0,0] +34 0 1 0 [] +\N 1 1 0 [] +1 0 0 [] +0 1 0 [] +1 1 0 [] +1 0 3 [1,1,1] +0 1 0 [] +1 1 0 [] +1 0 6 [0,0,0,0,0,0] +0 1 0 [] +1 1 0 [] +1 0 9 [1,1,1,1,1,1,1,1,1] +0 1 0 [] +1 1 0 [] +1 0 2 [1,1] +0 1 0 [] +1 1 0 [] +1 0 5 [0,0,0,0,0] +0 1 0 [] +1 1 0 [] +1 0 8 [1,1,1,1,1,1,1,1] +0 1 0 [] +1 1 0 [] +1 0 1 [1] +0 1 0 [] +1 1 0 [] +1 0 4 [0,0,0,0] +0 1 0 [] +1 1 0 [] +1 0 7 [1,1,1,1,1,1,1] +0 1 0 [] +1 1 0 [] +1 0 0 [] +0 1 0 [] +1 1 0 [] +1 0 3 [0,0,0] +0 1 0 [] +1 1 0 [] +0 0 [] [] +1 0 [] [] +1 0 [] [] +0 3 [1,1,1] [0,0,0] +1 0 [] [] +1 0 [] [] +0 6 [0,0,0,0,0,0] [1,1,1,1,1,1] +1 0 [] [] +1 0 [] [] +0 9 [1,1,1,1,1,1,1,1,1] [1,1,1,1,1,1,1,1,1] +1 0 [] [] +1 0 [] [] +0 2 [1,1] [0,0] +1 0 [] [] +1 0 [] [] +0 5 [0,0,0,0,0] [1,1,1,1,1] +1 0 [] [] +1 0 [] [] +0 8 [1,1,1,1,1,1,1,1] [1,1,1,1,1,1,1,1] +1 0 [] [] +1 0 [] [] +0 1 [1] [0] +1 0 [] [] +1 0 [] [] +0 4 [0,0,0,0] [1,1,1,1] +1 0 [] [] +1 0 [] [] +0 7 [1,1,1,1,1,1,1] [1,1,1,1,1,1,1] +1 0 [] [] +1 0 [] [] +0 0 [] [] +1 0 [] [] +1 0 [] [] +0 3 [0,0,0] [1,1,1] +1 0 [] [] +1 0 [] [] +0 +2 +3 +5 +6 +8 +9 +11 +12 +14 +15 +17 +18 +20 +21 +23 +24 +26 +27 +29 +30 +32 +33 +35 +MergeTree wide +test +[] 1 0 0 [] +1 0 1 0 [] +\N 1 1 0 [] +['str_3','str_3','str_3'] 1 0 3 [1,1,1] +4 0 1 0 [] +\N 1 1 0 [] +[6,6,6,6,6,6] 1 0 6 [0,0,0,0,0,0] +7 0 1 0 [] +\N 1 1 0 [] +[NULL,NULL,NULL,NULL,NULL,NULL,NULL,NULL,NULL] 1 0 9 [1,1,1,1,1,1,1,1,1] +10 0 1 0 [] +\N 1 1 0 [] +['str_12','str_12'] 1 0 2 [1,1] +13 0 1 0 [] +\N 1 1 0 [] +[15,15,15,15,15] 1 0 5 [0,0,0,0,0] +16 0 1 0 [] +\N 1 1 0 [] +[NULL,NULL,NULL,NULL,NULL,NULL,NULL,NULL] 1 0 8 [1,1,1,1,1,1,1,1] +19 0 1 0 [] +\N 1 1 0 [] +['str_21'] 1 0 1 [1] +22 0 1 0 [] +\N 1 1 0 [] +[24,24,24,24] 1 0 4 [0,0,0,0] +25 0 1 0 [] +\N 1 1 0 [] +[NULL,NULL,NULL,NULL,NULL,NULL,NULL] 1 0 7 [1,1,1,1,1,1,1] +28 0 1 0 [] +\N 1 1 0 [] +[] 1 0 0 [] +31 0 1 0 [] +\N 1 1 0 [] +[33,33,33] 1 0 3 [0,0,0] +34 0 1 0 [] +\N 1 1 0 [] +1 0 0 [] +0 1 0 [] +1 1 0 [] +1 0 3 [1,1,1] +0 1 0 [] +1 1 0 [] +1 0 6 [0,0,0,0,0,0] +0 1 0 [] +1 1 0 [] +1 0 9 [1,1,1,1,1,1,1,1,1] +0 1 0 [] +1 1 0 [] +1 0 2 [1,1] +0 1 0 [] +1 1 0 [] +1 0 5 [0,0,0,0,0] +0 1 0 [] +1 1 0 [] +1 0 8 [1,1,1,1,1,1,1,1] +0 1 0 [] +1 1 0 [] +1 0 1 [1] +0 1 0 [] +1 1 0 [] +1 0 4 [0,0,0,0] +0 1 0 [] +1 1 0 [] +1 0 7 [1,1,1,1,1,1,1] +0 1 0 [] +1 1 0 [] +1 0 0 [] +0 1 0 [] +1 1 0 [] +1 0 3 [0,0,0] +0 1 0 [] +1 1 0 [] +0 0 [] [] +1 0 [] [] +1 0 [] [] +0 3 [1,1,1] [0,0,0] +1 0 [] [] +1 0 [] [] +0 6 [0,0,0,0,0,0] [1,1,1,1,1,1] +1 0 [] [] +1 0 [] [] +0 9 [1,1,1,1,1,1,1,1,1] [1,1,1,1,1,1,1,1,1] +1 0 [] [] +1 0 [] [] +0 2 [1,1] [0,0] +1 0 [] [] +1 0 [] [] +0 5 [0,0,0,0,0] [1,1,1,1,1] +1 0 [] [] +1 0 [] [] +0 8 [1,1,1,1,1,1,1,1] [1,1,1,1,1,1,1,1] +1 0 [] [] +1 0 [] [] +0 1 [1] [0] +1 0 [] [] +1 0 [] [] +0 4 [0,0,0,0] [1,1,1,1] +1 0 [] [] +1 0 [] [] +0 7 [1,1,1,1,1,1,1] [1,1,1,1,1,1,1] +1 0 [] [] +1 0 [] [] +0 0 [] [] +1 0 [] [] +1 0 [] [] +0 3 [0,0,0] [1,1,1] +1 0 [] [] +1 0 [] [] +0 +2 +3 +5 +6 +8 +9 +11 +12 +14 +15 +17 +18 +20 +21 +23 +24 +26 +27 +29 +30 +32 +33 +35 diff --git a/tests/queries/0_stateless/03201_variant_null_map_subcolumn.sh b/tests/queries/0_stateless/03201_variant_null_map_subcolumn.sh new file mode 100755 index 00000000000..8231691e184 --- /dev/null +++ b/tests/queries/0_stateless/03201_variant_null_map_subcolumn.sh @@ -0,0 +1,44 @@ +#!/usr/bin/env bash +# Tags: long + +CUR_DIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd) +# reset --log_comment +CLICKHOUSE_LOG_COMMENT= +# shellcheck source=../shell_config.sh +. "$CUR_DIR"/../shell_config.sh + +CH_CLIENT="$CLICKHOUSE_CLIENT --allow_experimental_variant_type=1 --use_variant_as_common_type=1 --allow_suspicious_variant_types=1" + +function test() +{ + echo "test" + $CH_CLIENT -q "insert into test select number, multiIf(number % 3 == 2, NULL, number % 3 == 1, number, arrayMap(x -> multiIf(number % 9 == 0, NULL, number % 9 == 3, 'str_' || toString(number), number), range(number % 10))) from numbers(36)" + $CH_CLIENT -q "select v, v.UInt64.null, v.\`Array(Variant(String, UInt64))\`.null, v.\`Array(Variant(String, UInt64))\`.size0, v.\`Array(Variant(String, UInt64))\`.UInt64.null from test order by id" + $CH_CLIENT -q "select v.UInt64.null, v.\`Array(Variant(String, UInt64))\`.null, v.\`Array(Variant(String, UInt64))\`.size0, v.\`Array(Variant(String, UInt64))\`.UInt64.null from test order by id" + $CH_CLIENT -q "select v.\`Array(Variant(String, UInt64))\`.null, v.\`Array(Variant(String, UInt64))\`.size0, v.\`Array(Variant(String, UInt64))\`.UInt64.null, v.\`Array(Variant(String, UInt64))\`.String.null from test order by id" + $CH_CLIENT -q "select id from test where v.UInt64 is null order by id" + + $CH_CLIENT -q "insert into test select number, multiIf(number % 3 == 2, NULL, number % 3 == 1, number, arrayMap(x -> multiIf(number % 9 == 0, NULL, number % 9 == 3, 'str_' || toString(number), number), range(number % 10))) from numbers(1000000) settings min_insert_block_size_rows=100000" + $CH_CLIENT -q "select v, v.UInt64.null, v.\`Array(Variant(String, UInt64))\`.null, v.\`Array(Variant(String, UInt64))\`.size0, v.\`Array(Variant(String, UInt64))\`.UInt64.null from test order by id format Null" + $CH_CLIENT -q "select v.UInt64.null, v.\`Array(Variant(String, UInt64))\`.null, v.\`Array(Variant(String, UInt64))\`.size0, v.\`Array(Variant(String, UInt64))\`.UInt64.null from test order by id format Null" + $CH_CLIENT -q "select v.\`Array(Variant(String, UInt64))\`.null, v.\`Array(Variant(String, UInt64))\`.size0, v.\`Array(Variant(String, UInt64))\`.UInt64.null, v.\`Array(Variant(String, UInt64))\`.String.null from test order by id format Null" + $CH_CLIENT -q "select id from test where v.UInt64 is null order by id format Null" +} + +$CH_CLIENT -q "drop table if exists test;" + +echo "Memory" +$CH_CLIENT -q "create table test (id UInt64, v Variant(UInt64, Array(Variant(String, UInt64)))) engine=Memory" +test +$CH_CLIENT -q "drop table test;" + +echo "MergeTree compact" +$CH_CLIENT -q "create table test (id UInt64, v Variant(UInt64, Array(Variant(String, UInt64)))) engine=MergeTree order by id settings min_rows_for_wide_part=1000000000, min_bytes_for_wide_part=10000000000;" +test +$CH_CLIENT -q "drop table test;" + +echo "MergeTree wide" +$CH_CLIENT -q "create table test (id UInt64, v Variant(UInt64, Array(Variant(String, UInt64)))) engine=MergeTree order by id settings min_rows_for_wide_part=1, min_bytes_for_wide_part=1;" +test +$CH_CLIENT -q "drop table test;" + diff --git a/tests/queries/0_stateless/03202_dynamic_null_map_subcolumn.reference b/tests/queries/0_stateless/03202_dynamic_null_map_subcolumn.reference new file mode 100644 index 00000000000..8740726c7ef --- /dev/null +++ b/tests/queries/0_stateless/03202_dynamic_null_map_subcolumn.reference @@ -0,0 +1,57 @@ +Memory +test +Array(Array(Dynamic)) +Array(Variant(String, UInt64)) +None +String +UInt64 +20 +20 +20 +20 +0 +0 +20 +20 +10 +10 +20 +0 +MergeTree compact +test +Array(Array(Dynamic)) +Array(Variant(String, UInt64)) +None +String +UInt64 +20 +20 +20 +20 +0 +0 +20 +20 +10 +10 +20 +0 +MergeTree wide +test +Array(Array(Dynamic)) +Array(Variant(String, UInt64)) +None +String +UInt64 +20 +20 +20 +20 +0 +0 +20 +20 +10 +10 +20 +0 diff --git a/tests/queries/0_stateless/03202_dynamic_null_map_subcolumn.sh b/tests/queries/0_stateless/03202_dynamic_null_map_subcolumn.sh new file mode 100755 index 00000000000..aa06e48376c --- /dev/null +++ b/tests/queries/0_stateless/03202_dynamic_null_map_subcolumn.sh @@ -0,0 +1,62 @@ +#!/usr/bin/env bash +# Tags: long + +CUR_DIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd) +# reset --log_comment +CLICKHOUSE_LOG_COMMENT= +# shellcheck source=../shell_config.sh +. "$CUR_DIR"/../shell_config.sh + +CH_CLIENT="$CLICKHOUSE_CLIENT --allow_experimental_variant_type=1 --use_variant_as_common_type=1 --allow_experimental_dynamic_type=1" + + +function test() +{ + echo "test" + $CH_CLIENT -q "insert into test select number, number from numbers(10) settings min_insert_block_size_rows=50000" + $CH_CLIENT -q "insert into test select number, 'str_' || toString(number) from numbers(10, 10) settings min_insert_block_size_rows=50000" + $CH_CLIENT -q "insert into test select number, arrayMap(x -> multiIf(number % 9 == 0, NULL, number % 9 == 3, 'str_' || toString(number), number), range(number % 10 + 1)) from numbers(20, 10) settings min_insert_block_size_rows=50000" + $CH_CLIENT -q "insert into test select number, NULL from numbers(30, 10) settings min_insert_block_size_rows=50000" + $CH_CLIENT -q "insert into test select number, multiIf(number % 4 == 3, 'str_' || toString(number), number % 4 == 2, NULL, number % 4 == 1, number, arrayMap(x -> multiIf(number % 9 == 0, NULL, number % 9 == 3, 'str_' || toString(number), number), range(number % 10 + 1))) from numbers(40, 40) settings min_insert_block_size_rows=50000" + $CH_CLIENT -q "insert into test select number, [range((number % 10 + 1)::UInt64)]::Array(Array(Dynamic)) from numbers(10, 10) settings min_insert_block_size_rows=50000" + + $CH_CLIENT -q "select distinct dynamicType(d) as type from test order by type" + $CH_CLIENT -q "select count() from test where dynamicType(d) == 'UInt64'" + $CH_CLIENT -q "select count() from test where d.UInt64 is not NULL" + $CH_CLIENT -q "select count() from test where dynamicType(d) == 'String'" + $CH_CLIENT -q "select count() from test where d.String is not NULL" + $CH_CLIENT -q "select count() from test where dynamicType(d) == 'Date'" + $CH_CLIENT -q "select count() from test where d.Date is not NULL" + $CH_CLIENT -q "select count() from test where dynamicType(d) == 'Array(Variant(String, UInt64))'" + $CH_CLIENT -q "select count() from test where not empty(d.\`Array(Variant(String, UInt64))\`)" + $CH_CLIENT -q "select count() from test where dynamicType(d) == 'Array(Array(Dynamic))'" + $CH_CLIENT -q "select count() from test where not empty(d.\`Array(Array(Dynamic))\`)" + $CH_CLIENT -q "select count() from test where d is NULL" + $CH_CLIENT -q "select count() from test where not empty(d.\`Tuple(a Array(Dynamic))\`.a.String)" + + $CH_CLIENT -q "select d, d.UInt64.null, d.String.null, d.\`Array(Variant(String, UInt64))\`.null from test format Null" + $CH_CLIENT -q "select d.UInt64.null, d.String.null, d.\`Array(Variant(String, UInt64))\`.null from test format Null" + $CH_CLIENT -q "select d.Int8.null, d.Date.null, d.\`Array(String)\`.null from test format Null" + $CH_CLIENT -q "select d, d.UInt64.null, d.Date.null, d.\`Array(Variant(String, UInt64))\`.null, d.\`Array(Variant(String, UInt64))\`.size0, d.\`Array(Variant(String, UInt64))\`.UInt64.null from test format Null" + $CH_CLIENT -q "select d.UInt64.null, d.Date.null, d.\`Array(Variant(String, UInt64))\`.null, d.\`Array(Variant(String, UInt64))\`.size0, d.\`Array(Variant(String, UInt64))\`.UInt64.null, d.\`Array(Variant(String, UInt64))\`.String.null from test format Null" + $CH_CLIENT -q "select d, d.\`Tuple(a UInt64, b String)\`.a, d.\`Array(Dynamic)\`.\`Variant(String, UInt64)\`.UInt64.null, d.\`Array(Variant(String, UInt64))\`.UInt64.null from test format Null" + $CH_CLIENT -q "select d.\`Array(Dynamic)\`.\`Variant(String, UInt64)\`.UInt64.null, d.\`Array(Dynamic)\`.size0, d.\`Array(Variant(String, UInt64))\`.UInt64.null from test format Null" + $CH_CLIENT -q "select d.\`Array(Array(Dynamic))\`.size1, d.\`Array(Array(Dynamic))\`.UInt64.null, d.\`Array(Array(Dynamic))\`.\`Map(String, Tuple(a UInt64))\`.values.a from test format Null" +} + +$CH_CLIENT -q "drop table if exists test;" + +echo "Memory" +$CH_CLIENT -q "create table test (id UInt64, d Dynamic) engine=Memory" +test +$CH_CLIENT -q "drop table test;" + +echo "MergeTree compact" +$CH_CLIENT -q "create table test (id UInt64, d Dynamic) engine=MergeTree order by id settings min_rows_for_wide_part=1000000000, min_bytes_for_wide_part=10000000000;" +test +$CH_CLIENT -q "drop table test;" + +echo "MergeTree wide" +$CH_CLIENT -q "create table test (id UInt64, d Dynamic) engine=MergeTree order by id settings min_rows_for_wide_part=1, min_bytes_for_wide_part=1;" +test +$CH_CLIENT -q "drop table test;" From def7408b48f0e6b557d143a2efedc243e4416dc8 Mon Sep 17 00:00:00 2001 From: Kruglov Pavel <48961922+Avogar@users.noreply.github.com> Date: Sat, 6 Jul 2024 22:57:41 +0200 Subject: [PATCH 274/417] Fix typo --- .../Serializations/SerializationVariantElementNullMap.cpp | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/src/DataTypes/Serializations/SerializationVariantElementNullMap.cpp b/src/DataTypes/Serializations/SerializationVariantElementNullMap.cpp index 4e355fbb8ef..f30da4fecf9 100644 --- a/src/DataTypes/Serializations/SerializationVariantElementNullMap.cpp +++ b/src/DataTypes/Serializations/SerializationVariantElementNullMap.cpp @@ -118,8 +118,8 @@ void SerializationVariantElementNullMap::deserializeBinaryBulkWithMultipleStream } else { - /// There is no such stream or cached data, it means that there is no Variant column in this part (it could happend after alter table add column). - /// In such cases columns are filled with default values, but for null-map column default value should be 1, not 0. Fill column with 1 here instead + /// There is no such stream or cached data, it means that there is no Variant column in this part (it could happen after alter table add column). + /// In such cases columns are filled with default values, but for null-map column default value should be 1, not 0. Fill column with 1 here instead. MutableColumnPtr mutable_column = result_column->assumeMutable(); auto & data = assert_cast(*mutable_column).getData(); data.resize_fill(data.size() + limit, 1); From e3c036e2d902e3fc0994a5d31135f19fe0e929c5 Mon Sep 17 00:00:00 2001 From: Sema Checherinda Date: Sun, 7 Jul 2024 00:21:22 +0200 Subject: [PATCH 275/417] remove debug logs --- src/Interpreters/DatabaseCatalog.cpp | 2 -- 1 file changed, 2 deletions(-) diff --git a/src/Interpreters/DatabaseCatalog.cpp b/src/Interpreters/DatabaseCatalog.cpp index 230deeab8d5..61ad5bf96de 100644 --- a/src/Interpreters/DatabaseCatalog.cpp +++ b/src/Interpreters/DatabaseCatalog.cpp @@ -1257,8 +1257,6 @@ DatabaseCatalog::TablesMarkedAsDropped DatabaseCatalog::getTablesToDrop() { bool in_use = it->table && !it->table.unique(); bool old_enough = it->drop_time <= current_time; - LOG_DEBUG(log, "check {}: in_use {}, old_enough {}", it->table_id.getFullTableName(), in_use, old_enough); - if (in_use || !old_enough) { ++it; From 82e1d82cb354de6ece3d525c4c71b7d08bfcf6d9 Mon Sep 17 00:00:00 2001 From: Robert Schulze Date: Sun, 7 Jul 2024 09:07:18 +0000 Subject: [PATCH 276/417] Cosmetics --- .../functions/string-replace-functions.md | 2 +- src/Functions/ReplaceRegexpImpl.h | 79 ++++++++++--------- 2 files changed, 43 insertions(+), 38 deletions(-) diff --git a/docs/en/sql-reference/functions/string-replace-functions.md b/docs/en/sql-reference/functions/string-replace-functions.md index 7aeb1f5b2a7..8793ebdd1a3 100644 --- a/docs/en/sql-reference/functions/string-replace-functions.md +++ b/docs/en/sql-reference/functions/string-replace-functions.md @@ -34,7 +34,7 @@ Alias: `replace`. Replaces the first occurrence of the substring matching the regular expression `pattern` (in [re2 syntax](https://github.com/google/re2/wiki/Syntax)) in `haystack` by the `replacement` string. -`replacement` can containing substitutions `\0-\9`. +`replacement` can contain substitutions `\0-\9`. Substitutions `\1-\9` correspond to the 1st to 9th capturing group (submatch), substitution `\0` corresponds to the entire match. To use a verbatim `\` character in the `pattern` or `replacement` strings, escape it using `\`. diff --git a/src/Functions/ReplaceRegexpImpl.h b/src/Functions/ReplaceRegexpImpl.h index 24a40c45c6e..c5095e3039f 100644 --- a/src/Functions/ReplaceRegexpImpl.h +++ b/src/Functions/ReplaceRegexpImpl.h @@ -48,42 +48,57 @@ struct ReplaceRegexpImpl static constexpr int max_captures = 10; - static Instructions createInstructions(std::string_view replacement, int num_captures) + /// The replacement string references must not contain non-existing capturing groups. + static void checkSubstitutions(std::string_view replacement, int num_captures) { - Instructions instructions; - - String literals; for (size_t i = 0; i < replacement.size(); ++i) { if (replacement[i] == '\\' && i + 1 < replacement.size()) { - if (isNumericASCII(replacement[i + 1])) /// Substitution + if (isNumericASCII(replacement[i + 1])) /// substitution + { + int substitution_num = replacement[i + 1] - '0'; + if (substitution_num >= num_captures) + throw Exception(ErrorCodes::BAD_ARGUMENTS, "Substitution '\\{}' in replacement argument is invalid, regexp has only {} capturing groups", substitution_num, num_captures - 1); + } + } + } + } + + static Instructions createInstructions(std::string_view replacement, int num_captures) + { + checkSubstitutions(replacement, num_captures); + + Instructions instructions; + + String literals; + literals.reserve(replacement.size()); + + for (size_t i = 0; i < replacement.size(); ++i) + { + if (replacement[i] == '\\' && i + 1 < replacement.size()) + { + if (isNumericASCII(replacement[i + 1])) /// substitution { if (!literals.empty()) { instructions.emplace_back(literals); literals = ""; } - instructions.emplace_back(replacement[i + 1] - '0'); + int substitution_num = replacement[i + 1] - '0'; + instructions.emplace_back(substitution_num); } else - literals += replacement[i + 1]; /// Escaping + literals += replacement[i + 1]; /// escaping ++i; } else - literals += replacement[i]; /// Plain character + literals += replacement[i]; /// plain character } if (!literals.empty()) instructions.emplace_back(literals); - for (const auto & instr : instructions) - if (instr.substitution_num >= num_captures) - throw Exception( - ErrorCodes::BAD_ARGUMENTS, - "Id {} in replacement string is an invalid substitution, regexp has only {} capturing groups", - instr.substitution_num, num_captures - 1); - return instructions; } @@ -124,7 +139,7 @@ struct ReplaceRegexpImpl { std::string_view replacement; if (instr.substitution_num >= 0) - replacement = std::string_view(matches[instr.substitution_num].data(), matches[instr.substitution_num].size()); + replacement = {matches[instr.substitution_num].data(), matches[instr.substitution_num].size()}; else replacement = instr.literal; res_data.resize(res_data.size() + replacement.size()); @@ -179,19 +194,15 @@ struct ReplaceRegexpImpl res_offsets.resize(haystack_size); re2::RE2::Options regexp_options; - /// Don't write error messages to stderr. - regexp_options.set_log_errors(false); + regexp_options.set_log_errors(false); /// don't write error messages to stderr re2::RE2 searcher(needle, regexp_options); - if (!searcher.ok()) throw Exception(ErrorCodes::BAD_ARGUMENTS, "The pattern argument is not a valid re2 pattern: {}", searcher.error()); int num_captures = std::min(searcher.NumberOfCapturingGroups() + 1, max_captures); - Instructions instructions = createInstructions(replacement, num_captures); - /// Cannot perform search for whole columns. Will process each string separately. for (size_t i = 0; i < haystack_size; ++i) { size_t from = i > 0 ? haystack_offsets[i - 1] : 0; @@ -221,10 +232,8 @@ struct ReplaceRegexpImpl res_offsets.resize(haystack_size); re2::RE2::Options regexp_options; - /// Don't write error messages to stderr. - regexp_options.set_log_errors(false); + regexp_options.set_log_errors(false); /// don't write error messages to stderr - /// Cannot perform search for whole columns. Will process each string separately. for (size_t i = 0; i < haystack_size; ++i) { size_t hs_from = i > 0 ? haystack_offsets[i - 1] : 0; @@ -242,6 +251,7 @@ struct ReplaceRegexpImpl re2::RE2 searcher(needle, regexp_options); if (!searcher.ok()) throw Exception(ErrorCodes::BAD_ARGUMENTS, "The pattern argument is not a valid re2 pattern: {}", searcher.error()); + int num_captures = std::min(searcher.NumberOfCapturingGroups() + 1, max_captures); Instructions instructions = createInstructions(replacement, num_captures); @@ -270,17 +280,14 @@ struct ReplaceRegexpImpl res_offsets.resize(haystack_size); re2::RE2::Options regexp_options; - /// Don't write error messages to stderr. - regexp_options.set_log_errors(false); + regexp_options.set_log_errors(false); /// don't write error messages to stderr re2::RE2 searcher(needle, regexp_options); - if (!searcher.ok()) throw Exception(ErrorCodes::BAD_ARGUMENTS, "The pattern argument is not a valid re2 pattern: {}", searcher.error()); int num_captures = std::min(searcher.NumberOfCapturingGroups() + 1, max_captures); - /// Cannot perform search for whole columns. Will process each string separately. for (size_t i = 0; i < haystack_size; ++i) { size_t hs_from = i > 0 ? haystack_offsets[i - 1] : 0; @@ -290,8 +297,9 @@ struct ReplaceRegexpImpl size_t repl_from = i > 0 ? replacement_offsets[i - 1] : 0; const char * repl_data = reinterpret_cast(replacement_data.data() + repl_from); const size_t repl_length = static_cast(replacement_offsets[i] - repl_from - 1); + std::string_view replacement(repl_data, repl_length); - Instructions instructions = createInstructions(std::string_view(repl_data, repl_length), num_captures); + Instructions instructions = createInstructions(replacement, num_captures); processString(hs_data, hs_length, res_data, res_offset, searcher, num_captures, instructions); res_offsets[i] = res_offset; @@ -317,10 +325,8 @@ struct ReplaceRegexpImpl res_offsets.resize(haystack_size); re2::RE2::Options regexp_options; - /// Don't write error messages to stderr. - regexp_options.set_log_errors(false); + regexp_options.set_log_errors(false); /// don't write error messages to stderr - /// Cannot perform search for whole columns. Will process each string separately. for (size_t i = 0; i < haystack_size; ++i) { size_t hs_from = i > 0 ? haystack_offsets[i - 1] : 0; @@ -338,12 +344,14 @@ struct ReplaceRegexpImpl size_t repl_from = i > 0 ? replacement_offsets[i - 1] : 0; const char * repl_data = reinterpret_cast(replacement_data.data() + repl_from); const size_t repl_length = static_cast(replacement_offsets[i] - repl_from - 1); + std::string_view replacement(repl_data, repl_length); re2::RE2 searcher(needle, regexp_options); if (!searcher.ok()) throw Exception(ErrorCodes::BAD_ARGUMENTS, "The pattern argument is not a valid re2 pattern: {}", searcher.error()); + int num_captures = std::min(searcher.NumberOfCapturingGroups() + 1, max_captures); - Instructions instructions = createInstructions(std::string_view(repl_data, repl_length), num_captures); + Instructions instructions = createInstructions(replacement, num_captures); processString(hs_data, hs_length, res_data, res_offset, searcher, num_captures, instructions); res_offsets[i] = res_offset; @@ -367,16 +375,13 @@ struct ReplaceRegexpImpl res_offsets.resize(haystack_size); re2::RE2::Options regexp_options; - /// Don't write error messages to stderr. - regexp_options.set_log_errors(false); + regexp_options.set_log_errors(false); /// don't write error messages to stderr re2::RE2 searcher(needle, regexp_options); - if (!searcher.ok()) throw Exception(ErrorCodes::BAD_ARGUMENTS, "The pattern argument is not a valid re2 pattern: {}", searcher.error()); int num_captures = std::min(searcher.NumberOfCapturingGroups() + 1, max_captures); - Instructions instructions = createInstructions(replacement, num_captures); for (size_t i = 0; i < haystack_size; ++i) From 17e089c490efbf1ac4224fa8cf0b74bb3f50739a Mon Sep 17 00:00:00 2001 From: zhongyuankai <872237106@qq.com> Date: Sun, 7 Jul 2024 18:22:55 +0800 Subject: [PATCH 277/417] Refactor `OptimizeIfWithConstantConditionVisitor` using `InDepthNodeVisitor` --- ...OptimizeIfWithConstantConditionVisitor.cpp | 93 ++++++++----------- .../OptimizeIfWithConstantConditionVisitor.h | 17 ++-- src/Interpreters/TreeOptimizer.cpp | 3 +- 3 files changed, 52 insertions(+), 61 deletions(-) diff --git a/src/Interpreters/OptimizeIfWithConstantConditionVisitor.cpp b/src/Interpreters/OptimizeIfWithConstantConditionVisitor.cpp index 20451fb20ad..48c9988b6fc 100644 --- a/src/Interpreters/OptimizeIfWithConstantConditionVisitor.cpp +++ b/src/Interpreters/OptimizeIfWithConstantConditionVisitor.cpp @@ -73,66 +73,55 @@ static bool tryExtractConstValueFromCondition(const ASTPtr & condition, bool & v return false; } -void OptimizeIfWithConstantConditionVisitor::visit(ASTPtr & current_ast) +void OptimizeIfWithConstantConditionVisitorData::visit(ASTFunction & function_node, ASTPtr & ast) { - if (!current_ast) - return; - checkStackSize(); - for (ASTPtr & child : current_ast->children) + if (function_node.name != "if") + return; + + if (!function_node.arguments) + throw Exception(ErrorCodes::NUMBER_OF_ARGUMENTS_DOESNT_MATCH, "Wrong number of arguments for function 'if' (0 instead of 3)"); + + if (function_node.arguments->children.size() != 3) + throw Exception(ErrorCodes::NUMBER_OF_ARGUMENTS_DOESNT_MATCH, + "Wrong number of arguments for function 'if' ({} instead of 3)", + function_node.arguments->children.size()); + + const auto * args = function_node.arguments->as(); + + ASTPtr condition_expr = args->children[0]; + ASTPtr then_expr = args->children[1]; + ASTPtr else_expr = args->children[2]; + + bool condition; + if (tryExtractConstValueFromCondition(condition_expr, condition)) { - auto * function_node = child->as(); - if (!function_node || function_node->name != "if") + ASTPtr replace_ast = condition ? then_expr : else_expr; + ASTPtr child_copy = ast; + String replace_alias = replace_ast->tryGetAlias(); + String if_alias = ast->tryGetAlias(); + + if (replace_alias.empty()) { - visit(child); - continue; + replace_ast->setAlias(if_alias); + ast = replace_ast; + } + else + { + /// Only copy of one node is required here. + /// But IAST has only method for deep copy of subtree. + /// This can be a reason of performance degradation in case of deep queries. + ASTPtr replace_ast_deep_copy = replace_ast->clone(); + replace_ast_deep_copy->setAlias(if_alias); + ast = replace_ast_deep_copy; } - if (!function_node->arguments) - throw Exception(ErrorCodes::NUMBER_OF_ARGUMENTS_DOESNT_MATCH, "Wrong number of arguments for function 'if' (0 instead of 3)"); - - if (function_node->arguments->children.size() != 3) - throw Exception(ErrorCodes::NUMBER_OF_ARGUMENTS_DOESNT_MATCH, - "Wrong number of arguments for function 'if' ({} instead of 3)", - function_node->arguments->children.size()); - - visit(function_node->arguments); - const auto * args = function_node->arguments->as(); - - ASTPtr condition_expr = args->children[0]; - ASTPtr then_expr = args->children[1]; - ASTPtr else_expr = args->children[2]; - - bool condition; - if (tryExtractConstValueFromCondition(condition_expr, condition)) + if (!if_alias.empty()) { - ASTPtr replace_ast = condition ? then_expr : else_expr; - ASTPtr child_copy = child; - String replace_alias = replace_ast->tryGetAlias(); - String if_alias = child->tryGetAlias(); - - if (replace_alias.empty()) - { - replace_ast->setAlias(if_alias); - child = replace_ast; - } - else - { - /// Only copy of one node is required here. - /// But IAST has only method for deep copy of subtree. - /// This can be a reason of performance degradation in case of deep queries. - ASTPtr replace_ast_deep_copy = replace_ast->clone(); - replace_ast_deep_copy->setAlias(if_alias); - child = replace_ast_deep_copy; - } - - if (!if_alias.empty()) - { - auto alias_it = aliases.find(if_alias); - if (alias_it != aliases.end() && alias_it->second.get() == child_copy.get()) - alias_it->second = child; - } + auto alias_it = aliases.find(if_alias); + if (alias_it != aliases.end() && alias_it->second.get() == child_copy.get()) + alias_it->second = ast; } } } diff --git a/src/Interpreters/OptimizeIfWithConstantConditionVisitor.h b/src/Interpreters/OptimizeIfWithConstantConditionVisitor.h index ad98f92bafd..3b46f90f07c 100644 --- a/src/Interpreters/OptimizeIfWithConstantConditionVisitor.h +++ b/src/Interpreters/OptimizeIfWithConstantConditionVisitor.h @@ -1,23 +1,24 @@ #pragma once #include +#include namespace DB { - -/// It removes Function_if node from AST if condition is constant. -/// TODO: rewrite with InDepthNodeVisitor -class OptimizeIfWithConstantConditionVisitor +struct OptimizeIfWithConstantConditionVisitorData { -public: - explicit OptimizeIfWithConstantConditionVisitor(Aliases & aliases_) + using TypeToVisit = ASTFunction; + + explicit OptimizeIfWithConstantConditionVisitorData(Aliases & aliases_) : aliases(aliases_) {} - void visit(ASTPtr & ast); - + void visit(ASTFunction & function_node, ASTPtr & ast); private: Aliases & aliases; }; +/// It removes Function_if node from AST if condition is constant. +using OptimizeIfWithConstantConditionVisitor = InDepthNodeVisitor, false>; + } diff --git a/src/Interpreters/TreeOptimizer.cpp b/src/Interpreters/TreeOptimizer.cpp index b88d75cd5a2..b872eb94fde 100644 --- a/src/Interpreters/TreeOptimizer.cpp +++ b/src/Interpreters/TreeOptimizer.cpp @@ -577,7 +577,8 @@ void TreeOptimizer::optimizeIf(ASTPtr & query, Aliases & aliases, bool if_chain_ optimizeMultiIfToIf(query); /// Optimize if with constant condition after constants was substituted instead of scalar subqueries. - OptimizeIfWithConstantConditionVisitor(aliases).visit(query); + OptimizeIfWithConstantConditionVisitorData visitor_data(aliases); + OptimizeIfWithConstantConditionVisitor(visitor_data).visit(query); if (if_chain_to_multiif) OptimizeIfChainsVisitor().visit(query); From 948565f5544d29b37a7368b8f61c5bfbd66905c1 Mon Sep 17 00:00:00 2001 From: Robert Schulze Date: Sun, 7 Jul 2024 10:43:48 +0000 Subject: [PATCH 278/417] Speed up replaceRegexp(All|One) if the pattern is trivial --- src/Functions/ReplaceRegexpImpl.h | 37 ++++++++++++++++++- tests/performance/replaceRegexp_fallback.xml | 12 ++++++ ...4_replace_regexp_string_fallback.reference | 1 + .../02864_replace_regexp_string_fallback.sql | 11 ++++++ 4 files changed, 60 insertions(+), 1 deletion(-) create mode 100644 tests/performance/replaceRegexp_fallback.xml create mode 100644 tests/queries/0_stateless/02864_replace_regexp_string_fallback.reference create mode 100644 tests/queries/0_stateless/02864_replace_regexp_string_fallback.sql diff --git a/src/Functions/ReplaceRegexpImpl.h b/src/Functions/ReplaceRegexpImpl.h index c5095e3039f..f5fb08f71d2 100644 --- a/src/Functions/ReplaceRegexpImpl.h +++ b/src/Functions/ReplaceRegexpImpl.h @@ -1,9 +1,12 @@ #pragma once -#include #include +#include #include +#include +#include #include +#include namespace DB { @@ -102,6 +105,21 @@ struct ReplaceRegexpImpl return instructions; } + static bool canFallbackToStringReplacement(const String & needle, const String & replacement, const re2::RE2 & searcher, int num_captures) + { + if (searcher.NumberOfCapturingGroups()) + return false; + + checkSubstitutions(replacement, num_captures); + + String required_substring; + bool is_trivial; + bool required_substring_is_prefix; + std::vector alternatives; + OptimizedRegularExpression::analyze(needle, required_substring, is_trivial, required_substring_is_prefix, alternatives); + return is_trivial && required_substring_is_prefix && required_substring == needle; + } + static void processString( const char * haystack_data, size_t haystack_length, @@ -201,6 +219,23 @@ struct ReplaceRegexpImpl throw Exception(ErrorCodes::BAD_ARGUMENTS, "The pattern argument is not a valid re2 pattern: {}", searcher.error()); int num_captures = std::min(searcher.NumberOfCapturingGroups() + 1, max_captures); + + /// Try to use non-regexp string replacement. This shortcut is implemented only for const-needles + const-replacement as + /// pattern analysis incurs some cost too. + if (canFallbackToStringReplacement(needle, replacement, searcher, num_captures)) + { + auto convertTrait = [](ReplaceRegexpTraits::Replace first_or_all) + { + switch (first_or_all) + { + case ReplaceRegexpTraits::Replace::First: return ReplaceStringTraits::Replace::First; + case ReplaceRegexpTraits::Replace::All: return ReplaceStringTraits::Replace::All; + } + }; + ReplaceStringImpl::vectorConstantConstant(haystack_data, haystack_offsets, needle, replacement, res_data, res_offsets); + return; + } + Instructions instructions = createInstructions(replacement, num_captures); for (size_t i = 0; i < haystack_size; ++i) diff --git a/tests/performance/replaceRegexp_fallback.xml b/tests/performance/replaceRegexp_fallback.xml new file mode 100644 index 00000000000..926e66c702f --- /dev/null +++ b/tests/performance/replaceRegexp_fallback.xml @@ -0,0 +1,12 @@ +> +> + + > + WITH 'Many years later as he faced the firing squad, Colonel Aureliano Buendia was to remember that distant afternoon when his father took him to discover ice.' AS s SELECT replaceRegexpAll(materialize(s), ' ', '\n') AS w FROM numbers(5000000) FORMAT Null + WITH 'Many years later as he faced the firing squad, Colonel Aureliano Buendia was to remember that distant afternoon when his father took him to discover ice.' AS s SELECT replaceRegexpOne(materialize(s), ' ', '\n') AS w FROM numbers(5000000) FORMAT Null + + > + > + WITH 'Many years later as he faced the firing squad, Colonel Aureliano Buendia was to remember that distant afternoon when his father took him to discover ice.' AS s SELECT replaceRegexpAll(materialize(s), '\s+', '\\0\n') AS w FROM numbers(500000) FORMAT Null + WITH 'Many years later as he faced the firing squad, Colonel Aureliano Buendia was to remember that distant afternoon when his father took him to discover ice.' AS s SELECT replaceRegexpOne(materialize(s), '\s+', '\\0\n') AS w FROM numbers(500000) FORMAT Null + diff --git a/tests/queries/0_stateless/02864_replace_regexp_string_fallback.reference b/tests/queries/0_stateless/02864_replace_regexp_string_fallback.reference new file mode 100644 index 00000000000..dd52d49eea3 --- /dev/null +++ b/tests/queries/0_stateless/02864_replace_regexp_string_fallback.reference @@ -0,0 +1 @@ +Hello l x Hexlo Hexxo diff --git a/tests/queries/0_stateless/02864_replace_regexp_string_fallback.sql b/tests/queries/0_stateless/02864_replace_regexp_string_fallback.sql new file mode 100644 index 00000000000..917c11fe8dd --- /dev/null +++ b/tests/queries/0_stateless/02864_replace_regexp_string_fallback.sql @@ -0,0 +1,11 @@ +-- Tests functions replaceRegexpAll and replaceRegexpOne with trivial patterns. These trigger internally a fallback to simple string replacement. + +-- _materialize_ because the shortcut is only implemented for non-const haystack + const needle + const replacement strings + +SELECT 'Hello' AS haystack, 'l' AS needle, 'x' AS replacement, replaceRegexpOne(materialize(haystack), needle, replacement), replaceRegexpAll(materialize(haystack), needle, replacement); + +-- negative tests + +-- Even if the fallback is used, invalid substitutions must throw an exception. +SELECT 'Hello' AS haystack, 'l' AS needle, '\\1' AS replacement, replaceRegexpOne(materialize(haystack), needle, replacement); -- { serverError BAD_ARGUMENTS } +SELECT 'Hello' AS haystack, 'l' AS needle, '\\1' AS replacement, replaceRegexpAll(materialize(haystack), needle, replacement); -- { serverError BAD_ARGUMENTS } From e64e9f51ece90fcbfc98234ac65cc7e6bac2e2bd Mon Sep 17 00:00:00 2001 From: Tyler Hannan Date: Sun, 7 Jul 2024 14:57:26 +0200 Subject: [PATCH 279/417] Update README.md removing completed events. release call update on monday --- README.md | 2 -- 1 file changed, 2 deletions(-) diff --git a/README.md b/README.md index dc253d4db2d..3d7d7441081 100644 --- a/README.md +++ b/README.md @@ -40,8 +40,6 @@ Every month we get together with the community (users, contributors, customers, Keep an eye out for upcoming meetups and events around the world. Somewhere else you want us to be? Please feel free to reach out to tyler `` clickhouse `` com. You can also peruse [ClickHouse Events](https://clickhouse.com/company/news-events) for a list of all upcoming trainings, meetups, speaking engagements, etc. -* [AWS Summit in DC](https://clickhouse.com/company/events/2024-06-aws-summit-dc) - Jun 26 -* [ClickHouse Meetup in Amsterdam](https://www.meetup.com/clickhouse-netherlands-user-group/events/300781068/) - Jun 27 * [ClickHouse Meetup in Paris](https://www.meetup.com/clickhouse-france-user-group/events/300783448/) - Jul 9 * [ClickHouse Cloud - Live Update Call](https://clickhouse.com/company/events/202407-cloud-update-live) - Jul 9 * [ClickHouse Meetup @ Ramp - New York City](https://www.meetup.com/clickhouse-new-york-user-group/events/300595845/) - Jul 9 From 5aedbac37d482644523ae3c4c720971cea6c5502 Mon Sep 17 00:00:00 2001 From: Azat Khuzhin Date: Sun, 7 Jul 2024 16:52:29 +0200 Subject: [PATCH 280/417] Fix 01246_buffer_flush flakiness - reduce min_time for Buffer's min test - rewrite the test to .sh to avoid extra sleeping time (with .sql we have to wait the max time) - change the assertion for min test, the time there should not exceed max time (100 seconds), this should fix with test flakiness [1] even after [2]. [1]: https://s3.amazonaws.com/clickhouse-test-reports/0/76119a4567ce2ac9c0aff715c1a9ba2607e806e0/stateless_tests__tsan__[3_5].html [2]: https://github.com/ClickHouse/ClickHouse/pull/65310 Signed-off-by: Azat Khuzhin --- .../queries/0_stateless/01246_buffer_flush.sh | 76 +++++++++++++++++++ .../0_stateless/01246_buffer_flush.sql | 50 ------------ 2 files changed, 76 insertions(+), 50 deletions(-) create mode 100755 tests/queries/0_stateless/01246_buffer_flush.sh delete mode 100644 tests/queries/0_stateless/01246_buffer_flush.sql diff --git a/tests/queries/0_stateless/01246_buffer_flush.sh b/tests/queries/0_stateless/01246_buffer_flush.sh new file mode 100755 index 00000000000..1ca953c80d9 --- /dev/null +++ b/tests/queries/0_stateless/01246_buffer_flush.sh @@ -0,0 +1,76 @@ +#!/usr/bin/env bash +# Tags: no-fasttest + +CUR_DIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd) +# shellcheck source=../shell_config.sh +. "$CUR_DIR"/../shell_config.sh + +function elapsed_sec() +{ + local expr=$1 && shift + local start end + start=$(date +%s.%N) + while ! eval "$expr"; do + sleep 0.5 + done + end=$(date +%s.%N) + $CLICKHOUSE_LOCAL -q "select floor($end-$start)" +} + +$CLICKHOUSE_CLIENT -nm -q " + drop table if exists data_01256; + drop table if exists buffer_01256; + + create table data_01256 as system.numbers Engine=Memory(); +" + +echo "min" +$CLICKHOUSE_CLIENT -nm -q " + create table buffer_01256 as system.numbers Engine=Buffer(currentDatabase(), data_01256, 1, + 2, 100, /* time */ + 4, 100, /* rows */ + 1, 1e6 /* bytes */ + ); + insert into buffer_01256 select * from system.numbers limit 5; + select count() from data_01256; +" +sec=$(elapsed_sec '[[ $($CLICKHOUSE_CLIENT -q "select count() from data_01256") -eq 5 ]]') +[[ $sec -ge 2 ]] || echo "Buffer flushed too early, min_time=2, flushed after $sec sec" +[[ $sec -lt 100 ]] || echo "Buffer flushed too late, max_time=100, flushed after $sec sec" +$CLICKHOUSE_CLIENT -q "select count() from data_01256" +$CLICKHOUSE_CLIENT -q "drop table buffer_01256" + +echo "max" +$CLICKHOUSE_CLIENT -nm -q " + create table buffer_01256 as system.numbers Engine=Buffer(currentDatabase(), data_01256, 1, + 100, 2, /* time */ + 0, 100, /* rows */ + 0, 1e6 /* bytes */ + ); + insert into buffer_01256 select * from system.numbers limit 5; + select count() from data_01256; +" +sec=$(elapsed_sec '[[ $($CLICKHOUSE_CLIENT -q "select count() from data_01256") -eq 10 ]]') +[[ $sec -ge 2 ]] || echo "Buffer flushed too early, max_time=2, flushed after $sec sec" +$CLICKHOUSE_CLIENT -q "select count() from data_01256" +$CLICKHOUSE_CLIENT -q "drop table buffer_01256" + +echo "direct" +$CLICKHOUSE_CLIENT -nm -q " + create table buffer_01256 as system.numbers Engine=Buffer(currentDatabase(), data_01256, 1, + 100, 100, /* time */ + 0, 9, /* rows */ + 0, 1e6 /* bytes */ + ); + insert into buffer_01256 select * from system.numbers limit 10; + select count() from data_01256; +" + +echo "drop" +$CLICKHOUSE_CLIENT -nm -q " + insert into buffer_01256 select * from system.numbers limit 10; + drop table if exists buffer_01256; + select count() from data_01256; +" + +$CLICKHOUSE_CLIENT -q "drop table data_01256" diff --git a/tests/queries/0_stateless/01246_buffer_flush.sql b/tests/queries/0_stateless/01246_buffer_flush.sql deleted file mode 100644 index 66f93371c29..00000000000 --- a/tests/queries/0_stateless/01246_buffer_flush.sql +++ /dev/null @@ -1,50 +0,0 @@ --- Tags: no-fasttest - -SET function_sleep_max_microseconds_per_block = 4000000; - -drop table if exists data_01256; -drop table if exists buffer_01256; - -create table data_01256 as system.numbers Engine=Memory(); - -select 'min'; -create table buffer_01256 as system.numbers Engine=Buffer(currentDatabase(), data_01256, 1, - 5, 100, /* time */ - 4, 100, /* rows */ - 1, 1e6 /* bytes */ -); -insert into buffer_01256 select * from system.numbers limit 5; -select count() from data_01256; --- It is enough to ensure that the buffer will be flushed earlier then 2*min_time (10 sec) -select sleepEachRow(9) FORMAT Null SETTINGS function_sleep_max_microseconds_per_block=10e6; -select count() from data_01256; -drop table buffer_01256; - -select 'max'; -create table buffer_01256 as system.numbers Engine=Buffer(currentDatabase(), data_01256, 1, - 100, 2, /* time */ - 0, 100, /* rows */ - 0, 1e6 /* bytes */ -); -insert into buffer_01256 select * from system.numbers limit 5; -select count() from data_01256; --- sleep 2 (min time) + 1 (round up) + bias (1) = 4 -select sleepEachRow(2) from numbers(2) FORMAT Null; -select count() from data_01256; -drop table buffer_01256; - -select 'direct'; -create table buffer_01256 as system.numbers Engine=Buffer(currentDatabase(), data_01256, 1, - 100, 100, /* time */ - 0, 9, /* rows */ - 0, 1e6 /* bytes */ -); -insert into buffer_01256 select * from system.numbers limit 10; -select count() from data_01256; - -select 'drop'; -insert into buffer_01256 select * from system.numbers limit 10; -drop table if exists buffer_01256; -select count() from data_01256; - -drop table data_01256; From 126417cc61440019f0346b2d0fd46a2758a54d2c Mon Sep 17 00:00:00 2001 From: Blargian Date: Sun, 7 Jul 2024 18:46:54 +0200 Subject: [PATCH 281/417] add normalizeQueryKeepNames and normalizedQueryHashKeepNames --- .../functions/string-functions.md | 81 ++++++++++++++++++- 1 file changed, 79 insertions(+), 2 deletions(-) diff --git a/docs/en/sql-reference/functions/string-functions.md b/docs/en/sql-reference/functions/string-functions.md index c068b0e9d17..a4259c0d7f3 100644 --- a/docs/en/sql-reference/functions/string-functions.md +++ b/docs/en/sql-reference/functions/string-functions.md @@ -1556,7 +1556,8 @@ The result type is UInt64. ## normalizeQuery -Replaces literals, sequences of literals and complex aliases with placeholders. +Replaces literals, sequences of literals and complex aliases (containing whitespace, more than two digits +or at least 36 bytes long such as UUIDs) with placeholder `?`. **Syntax** @@ -1574,6 +1575,8 @@ normalizeQuery(x) **Example** +Query: + ``` sql SELECT normalizeQuery('[1, 2, 3, x]') AS query; ``` @@ -1586,9 +1589,44 @@ Result: └──────────┘ ``` +## normalizeQueryKeepNames + +Replaces literals, sequences of literals with placeholder `?` but does not replace complex aliases (containing whitespace, more than two digits +or at least 36 bytes long such as UUIDs). This helps better analyze complex query logs. + +**Syntax** + +``` sql +normalizeQueryKeepNames(x) +``` + +**Arguments** + +- `x` — Sequence of characters. [String](../data-types/string.md). + +**Returned value** + +- Sequence of characters with placeholders. [String](../data-types/string.md). + +**Example** + +Query: + +``` sql +SELECT normalizeQuery('SELECT 1 AS aComplexName123'), normalizeQueryKeepNames('SELECT 1 AS aComplexName123'); +``` + +Result: + +```result +┌─normalizeQuery('SELECT 1 AS aComplexName123')─┬─normalizeQueryKeepNames('SELECT 1 AS aComplexName123')─┐ +│ SELECT ? AS `?` │ SELECT ? AS aComplexName123 │ +└───────────────────────────────────────────────┴────────────────────────────────────────────────────────┘ +``` + ## normalizedQueryHash -Returns identical 64bit hash values without the values of literals for similar queries. Can be helpful to analyze query log. +Returns identical 64bit hash values without the values of literals for similar queries. Can be helpful to analyze query logs. **Syntax** @@ -1606,6 +1644,8 @@ normalizedQueryHash(x) **Example** +Query: + ``` sql SELECT normalizedQueryHash('SELECT 1 AS `xyz`') != normalizedQueryHash('SELECT 1 AS `abc`') AS res; ``` @@ -1618,6 +1658,43 @@ Result: └─────┘ ``` +## normalizedQueryHashKeepNames + +Like [normalizedQueryHash](#normalizedqueryhash) it returns identical 64bit hash values without the values of literals for similar queries but it does not replace complex aliases (containing whitespace, more than two digits +or at least 36 bytes long such as UUIDs) with a placeholder before hashing. Can be helpful to analyze query logs. + +**Syntax** + +``` sql +normalizedQueryHashKeepNames(x) +``` + +**Arguments** + +- `x` — Sequence of characters. [String](../data-types/string.md). + +**Returned value** + +- Hash value. [UInt64](../data-types/int-uint.md#uint-ranges). + +**Example** + +``` sql +SELECT normalizedQueryHash('SELECT 1 AS `xyz123`') != normalizedQueryHash('SELECT 1 AS `abc123`') AS normalizedQueryHash; +SELECT normalizedQueryHashKeepNames('SELECT 1 AS `xyz123`') != normalizedQueryHashKeepNames('SELECT 1 AS `abc123`') AS normalizedQueryHashKeepNames; +``` + +Result: + +```result +┌─normalizedQueryHash─┐ +│ 0 │ +└─────────────────────┘ +┌─normalizedQueryHashKeepNames─┐ +│ 1 │ +└──────────────────────────────┘ +``` + ## normalizeUTF8NFC Converts a string to [NFC normalized form](https://en.wikipedia.org/wiki/Unicode_equivalence#Normal_forms), assuming the string is valid UTF8-encoded text. From 2113915bde5d60c429aeb5d45a74a083d4d3e570 Mon Sep 17 00:00:00 2001 From: Blargian Date: Sun, 7 Jul 2024 18:51:52 +0200 Subject: [PATCH 282/417] correct formatting problem --- docs/en/sql-reference/functions/string-functions.md | 3 +-- 1 file changed, 1 insertion(+), 2 deletions(-) diff --git a/docs/en/sql-reference/functions/string-functions.md b/docs/en/sql-reference/functions/string-functions.md index a4259c0d7f3..d759da4a106 100644 --- a/docs/en/sql-reference/functions/string-functions.md +++ b/docs/en/sql-reference/functions/string-functions.md @@ -1556,8 +1556,7 @@ The result type is UInt64. ## normalizeQuery -Replaces literals, sequences of literals and complex aliases (containing whitespace, more than two digits -or at least 36 bytes long such as UUIDs) with placeholder `?`. +Replaces literals, sequences of literals and complex aliases (containing whitespace, more than two digits or at least 36 bytes long such as UUIDs) with placeholder `?`. **Syntax** From 7e20f26735a179948cb61186986b84ce4e9f1300 Mon Sep 17 00:00:00 2001 From: Amos Bird Date: Tue, 18 Jun 2024 20:35:33 +0800 Subject: [PATCH 283/417] Extend tuple() and add tupleNames() --- .../functions/tuple-functions.md | 56 ++++++++- src/Core/Settings.h | 1 + src/Core/SettingsChangesHistory.cpp | 1 + src/Functions/tuple.cpp | 12 +- src/Functions/tuple.h | 39 ++++-- src/Functions/tupleNames.cpp | 118 ++++++++++++++++++ src/Parsers/isUnquotedIdentifier.cpp | 20 +++ src/Parsers/isUnquotedIdentifier.h | 10 ++ .../Formats/Impl/ValuesBlockInputFormat.h | 13 +- .../queries/0_stateless/00307_format_xml.sql | 2 +- tests/queries/0_stateless/00309_formats.sql | 2 + .../01144_multiword_data_types.sql | 4 +- .../0_stateless/01232_untuple.reference | 2 +- tests/queries/0_stateless/01232_untuple.sql | 1 + .../02294_nothing_arguments_in_functions.sql | 2 + ...new_functions_must_be_documented.reference | 1 - .../02541_tuple_element_with_null.sql | 2 +- .../02890_named_tuple_functions.reference | 8 ++ .../02890_named_tuple_functions.sql | 22 ++++ .../02890_untuple_column_names.reference | 4 + .../02890_untuple_column_names.sql | 7 +- .../03038_nested_dynamic_merges.sh | 2 +- .../aspell-ignore/en/aspell-dict.txt | 1 + 23 files changed, 311 insertions(+), 19 deletions(-) create mode 100644 src/Functions/tupleNames.cpp create mode 100644 src/Parsers/isUnquotedIdentifier.cpp create mode 100644 src/Parsers/isUnquotedIdentifier.h create mode 100644 tests/queries/0_stateless/02890_named_tuple_functions.reference create mode 100644 tests/queries/0_stateless/02890_named_tuple_functions.sql diff --git a/docs/en/sql-reference/functions/tuple-functions.md b/docs/en/sql-reference/functions/tuple-functions.md index 0663be08240..3b4d68e44b2 100644 --- a/docs/en/sql-reference/functions/tuple-functions.md +++ b/docs/en/sql-reference/functions/tuple-functions.md @@ -7,7 +7,7 @@ sidebar_label: Tuples ## tuple A function that allows grouping multiple columns. -For columns with the types T1, T2, ..., it returns a Tuple(T1, T2, ...) type tuple containing these columns. There is no cost to execute the function. +For columns C1, C2, ... with the types T1, T2, ..., it returns a named Tuple(C1 T1, C2 T2, ...) type tuple containing these columns if their names are unique and can be treated as unquoted identifiers, otherwise a Tuple(T1, T2, ...) is returned. There is no cost to execute the function. Tuples are normally used as intermediate values for an argument of IN operators, or for creating a list of formal parameters of lambda functions. Tuples can’t be written to a table. The function implements the operator `(x, y, ...)`. @@ -259,6 +259,60 @@ Result: └───────────────────────────────────────┘ ``` +## tupleNames + +Converts a tuple into an array of column names. For a tuple in the form `Tuple(a T, b T, ...)`, it returns an array of strings representing the named columns of the tuple. If the tuple elements do not have explicit names, their indices will be used as the column names instead. + +**Syntax** + +``` sql +tupleNames(tuple) +``` + +**Arguments** + +- `tuple` — Named tuple. [Tuple](../../sql-reference/data-types/tuple.md) with any types of values. + +**Returned value** + +- An array with strings. + +Type: [Array](../../sql-reference/data-types/array.md)([Tuple](../../sql-reference/data-types/tuple.md)([String](../../sql-reference/data-types/string.md), ...)). + +**Example** + +Query: + +``` sql +CREATE TABLE tupletest (col Tuple(user_ID UInt64, session_ID UInt64)) ENGINE = Memory; + +INSERT INTO tupletest VALUES (tuple(1, 2)); + +SELECT tupleNames(col) FROM tupletest; +``` + +Result: + +``` text +┌─tupleNames(col)──────────┐ +│ ['user_ID','session_ID'] │ +└──────────────────────────┘ +``` + +If you pass a simple tuple to the function, ClickHouse uses the indexes of the columns as their names: + +``` sql +SELECT tupleNames(tuple(3, 2, 1)); +``` + +Result: + +``` text +┌─tupleNames((3, 2, 1))─┐ +│ ['1','2','3'] │ +└───────────────────────┘ +``` + ## tuplePlus Calculates the sum of corresponding values of two tuples of the same size. diff --git a/src/Core/Settings.h b/src/Core/Settings.h index 5903dbd32eb..e175bdfcbbe 100644 --- a/src/Core/Settings.h +++ b/src/Core/Settings.h @@ -755,6 +755,7 @@ class IColumn; M(Bool, optimize_group_by_function_keys, true, "Eliminates functions of other keys in GROUP BY section", 0) \ M(Bool, optimize_group_by_constant_keys, true, "Optimize GROUP BY when all keys in block are constant", 0) \ M(Bool, legacy_column_name_of_tuple_literal, false, "List all names of element of large tuple literals in their column names instead of hash. This settings exists only for compatibility reasons. It makes sense to set to 'true', while doing rolling update of cluster from version lower than 21.7 to higher.", 0) \ + M(Bool, enable_named_columns_in_function_tuple, true, "Generate named tuples in function tuple() when all names are unique and can be treated as unquoted identifiers.", 0) \ \ M(Bool, query_plan_enable_optimizations, true, "Globally enable/disable query optimization at the query plan level", 0) \ M(UInt64, query_plan_max_optimizations_to_apply, 10000, "Limit the total number of optimizations applied to query plan. If zero, ignored. If limit reached, throw exception", 0) \ diff --git a/src/Core/SettingsChangesHistory.cpp b/src/Core/SettingsChangesHistory.cpp index 8c096c13634..4e8cbf07509 100644 --- a/src/Core/SettingsChangesHistory.cpp +++ b/src/Core/SettingsChangesHistory.cpp @@ -59,6 +59,7 @@ static std::initializer_list(); + factory.registerFunction(FunctionDocumentation{ + .description = R"( +Returns a tuple by grouping input arguments. + +For columns C1, C2, ... with the types T1, T2, ..., it returns a named Tuple(C1 T1, C2 T2, ...) type tuple containing these columns if their names are unique and can be treated as unquoted identifiers, otherwise a Tuple(T1, T2, ...) is returned. There is no cost to execute the function. +Tuples are normally used as intermediate values for an argument of IN operators, or for creating a list of formal parameters of lambda functions. Tuples can’t be written to a table. + +The function implements the operator `(x, y, ...)`. +)", + .examples{{"typical", "SELECT tuple(1, 2)", "(1,2)"}}, + .categories{"Miscellaneous"}}); } } diff --git a/src/Functions/tuple.h b/src/Functions/tuple.h index 8b3e041f781..94529d86861 100644 --- a/src/Functions/tuple.h +++ b/src/Functions/tuple.h @@ -6,20 +6,28 @@ #include #include #include +#include namespace DB { -/** tuple(x, y, ...) is a function that allows you to group several columns +/** tuple(x, y, ...) is a function that allows you to group several columns. * tupleElement(tuple, n) is a function that allows you to retrieve a column from tuple. */ class FunctionTuple : public IFunction { + bool enable_named_columns; + public: static constexpr auto name = "tuple"; /// maybe_unused: false-positive - [[ maybe_unused ]] static FunctionPtr create(ContextPtr) { return std::make_shared(); } + [[maybe_unused]] static FunctionPtr create(ContextPtr context) + { + return std::make_shared(context->getSettingsRef().enable_named_columns_in_function_tuple); + } + + explicit FunctionTuple(bool enable_named_columns_ = false) : enable_named_columns(enable_named_columns_) { } String getName() const override { return name; } @@ -38,9 +46,26 @@ public: bool useDefaultImplementationForConstants() const override { return true; } bool useDefaultImplementationForLowCardinalityColumns() const override { return false; } - DataTypePtr getReturnTypeImpl(const DataTypes & arguments) const override + DataTypePtr getReturnTypeImpl(const ColumnsWithTypeAndName & arguments) const override { - return std::make_shared(arguments); + if (arguments.empty()) + return std::make_shared(DataTypes{}); + + DataTypes types; + Names names; + NameSet name_set; + for (const auto & argument : arguments) + { + types.emplace_back(argument.type); + names.emplace_back(argument.name); + name_set.emplace(argument.name); + } + + if (enable_named_columns && name_set.size() == names.size() + && std::all_of(names.cbegin(), names.cend(), [](const auto & n) { return isUnquotedIdentifier(n); })) + return std::make_shared(types, names); + else + return std::make_shared(types); } ColumnPtr executeImpl(const ColumnsWithTypeAndName & arguments, const DataTypePtr &, size_t input_rows_count) const override @@ -53,9 +78,9 @@ public: for (size_t i = 0; i < tuple_size; ++i) { /** If tuple is mixed of constant and not constant columns, - * convert all to non-constant columns, - * because many places in code expect all non-constant columns in non-constant tuple. - */ + * convert all to non-constant columns, + * because many places in code expect all non-constant columns in non-constant tuple. + */ tuple_columns[i] = arguments[i].column->convertToFullColumnIfConst(); } return ColumnTuple::create(tuple_columns); diff --git a/src/Functions/tupleNames.cpp b/src/Functions/tupleNames.cpp new file mode 100644 index 00000000000..e444478c224 --- /dev/null +++ b/src/Functions/tupleNames.cpp @@ -0,0 +1,118 @@ +#include +#include +#include +#include +#include +#include + +namespace DB +{ +namespace ErrorCodes +{ +extern const int ILLEGAL_TYPE_OF_ARGUMENT; +} + +namespace +{ + +/** Transform a named tuple into names, which is a constant array of strings. + */ +class ExecutableFunctionTupleNames : public IExecutableFunction +{ +public: + static constexpr auto name = "tupleNames"; + + explicit ExecutableFunctionTupleNames(Array name_fields_) : name_fields(std::move(name_fields_)) { } + + String getName() const override { return name; } + + ColumnPtr executeImpl(const ColumnsWithTypeAndName &, const DataTypePtr & result_type, size_t input_rows_count) const override + { + return result_type->createColumnConst(input_rows_count, name_fields); + } + +private: + Array name_fields; +}; + +class FunctionBaseTupleNames : public IFunctionBase +{ +public: + static constexpr auto name = "tupleNames"; + + explicit FunctionBaseTupleNames(DataTypePtr argument_type, DataTypePtr result_type_, Array name_fields_) + : argument_types({std::move(argument_type)}), result_type(std::move(result_type_)), name_fields(std::move(name_fields_)) + { + } + + String getName() const override { return name; } + + bool isSuitableForConstantFolding() const override { return true; } + + bool isSuitableForShortCircuitArgumentsExecution(const DataTypesWithConstInfo & /*arguments*/) const override { return true; } + + const DataTypes & getArgumentTypes() const override { return argument_types; } + + const DataTypePtr & getResultType() const override { return result_type; } + + ExecutableFunctionPtr prepare(const ColumnsWithTypeAndName &) const override + { + return std::make_unique(name_fields); + } + +private: + DataTypes argument_types; + DataTypePtr result_type; + Array name_fields; +}; + +class TupleNamesOverloadResolver : public IFunctionOverloadResolver +{ +public: + static constexpr auto name = "tupleNames"; + + static FunctionOverloadResolverPtr create(ContextPtr) { return std::make_unique(); } + + String getName() const override { return name; } + + size_t getNumberOfArguments() const override { return 1; } + + DataTypePtr getReturnTypeImpl(const ColumnsWithTypeAndName & arguments) const override + { + const DataTypeTuple * tuple = checkAndGetDataType(arguments[0].type.get()); + + if (!tuple) + throw Exception(ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT, "First argument for function {} must be a tuple", getName()); + + return std::make_shared(std::make_shared()); + } + + FunctionBasePtr buildImpl(const ColumnsWithTypeAndName & arguments, const DataTypePtr & result_type) const override + { + const DataTypeTuple * tuple = checkAndGetDataType(arguments[0].type.get()); + + if (!tuple) + throw Exception(ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT, "First argument for function {} must be a tuple", getName()); + + DataTypes types = tuple->getElements(); + Array name_fields; + for (const auto & elem_name : tuple->getElementNames()) + name_fields.emplace_back(elem_name); + + return std::make_unique(arguments[0].type, result_type, std::move(name_fields)); + } +}; + +} + +REGISTER_FUNCTION(TupleNames) +{ + factory.registerFunction(FunctionDocumentation{ + .description = R"( +Converts a tuple into an array of column names. For a tuple in the form `Tuple(a T, b T, ...)`, it returns an array of strings representing the named columns of the tuple. If the tuple elements do not have explicit names, their indices will be used as the column names instead. +)", + .examples{{"typical", "SELECT tupleNames(tuple(1 as a, 2 as b))", "['a','b']"}}, + .categories{"Miscellaneous"}}); +} + +} diff --git a/src/Parsers/isUnquotedIdentifier.cpp b/src/Parsers/isUnquotedIdentifier.cpp new file mode 100644 index 00000000000..6f2442635ec --- /dev/null +++ b/src/Parsers/isUnquotedIdentifier.cpp @@ -0,0 +1,20 @@ +#include + +#include + +namespace DB +{ + +bool isUnquotedIdentifier(const String & name) +{ + Lexer lexer(name.data(), name.data() + name.size()); + + auto maybe_ident = lexer.nextToken(); + + if (maybe_ident.type != TokenType::BareWord) + return false; + + return lexer.nextToken().isEnd(); +} + +} diff --git a/src/Parsers/isUnquotedIdentifier.h b/src/Parsers/isUnquotedIdentifier.h new file mode 100644 index 00000000000..839e5860ad3 --- /dev/null +++ b/src/Parsers/isUnquotedIdentifier.h @@ -0,0 +1,10 @@ +#pragma once + +#include + +namespace DB +{ + +bool isUnquotedIdentifier(const String & name); + +} diff --git a/src/Processors/Formats/Impl/ValuesBlockInputFormat.h b/src/Processors/Formats/Impl/ValuesBlockInputFormat.h index 0abafc896ff..b1bce098e99 100644 --- a/src/Processors/Formats/Impl/ValuesBlockInputFormat.h +++ b/src/Processors/Formats/Impl/ValuesBlockInputFormat.h @@ -37,7 +37,18 @@ public: void resetReadBuffer() override; /// TODO: remove context somehow. - void setContext(const ContextPtr & context_) { context = Context::createCopy(context_); } + void setContext(const ContextPtr & context_) + { + auto context_copy = Context::createCopy(context_); + + /// ConstantExpressionTemplate generates placeholder names (_dummy_N) + /// for all literals, which are valid names for creating named tuples. + /// This behavior needs to be explicitly disabled, because if named + /// tuples with different names are inserted into a named tuple, it will + /// only insert default values. + context_copy->setSetting("enable_named_columns_in_function_tuple", false); + context = context_copy; + } const BlockMissingValues & getMissingValues() const override { return block_missing_values; } diff --git a/tests/queries/0_stateless/00307_format_xml.sql b/tests/queries/0_stateless/00307_format_xml.sql index 7fdca83b69f..29c733bb186 100644 --- a/tests/queries/0_stateless/00307_format_xml.sql +++ b/tests/queries/0_stateless/00307_format_xml.sql @@ -1,2 +1,2 @@ SET output_format_write_statistics = 0; -SELECT 'Hello & world' AS s, 'Hello\n', toDateTime('2001-02-03 04:05:06') AS time, arrayMap(x -> toString(x), range(10)) AS arr, (s, time) AS tpl SETTINGS extremes = 1 FORMAT XML; +SELECT 'Hello & world' AS s, 'Hello\n', toDateTime('2001-02-03 04:05:06') AS time, arrayMap(x -> toString(x), range(10)) AS arr, (s, time) AS tpl SETTINGS extremes = 1, enable_named_columns_in_function_tuple = 0 FORMAT XML; diff --git a/tests/queries/0_stateless/00309_formats.sql b/tests/queries/0_stateless/00309_formats.sql index 87a1ea454d0..b0939c00a10 100644 --- a/tests/queries/0_stateless/00309_formats.sql +++ b/tests/queries/0_stateless/00309_formats.sql @@ -1,4 +1,6 @@ SET output_format_write_statistics = 0; +SET enable_named_columns_in_function_tuple = 0; + SELECT number * 246 + 10 AS n, toDate('2000-01-01') + n AS d, range(n) AS arr, arrayStringConcat(arrayMap(x -> reinterpretAsString(x), arr)) AS s, (n, d) AS tuple FROM system.numbers LIMIT 2 FORMAT RowBinary; SELECT number * 246 + 10 AS n, toDate('2000-01-01') + n AS d, range(n) AS arr, arrayStringConcat(arrayMap(x -> reinterpretAsString(x), arr)) AS s, (n, d) AS tuple FROM system.numbers LIMIT 2 FORMAT RowBinaryWithNamesAndTypes; SELECT number * 246 + 10 AS n, toDate('2000-01-01') + n AS d, range(n) AS arr, arrayStringConcat(arrayMap(x -> reinterpretAsString(x), arr)) AS s, (n, d) AS tuple FROM system.numbers LIMIT 2 FORMAT TabSeparatedWithNamesAndTypes; diff --git a/tests/queries/0_stateless/01144_multiword_data_types.sql b/tests/queries/0_stateless/01144_multiword_data_types.sql index cc380f82d63..56def658ae0 100644 --- a/tests/queries/0_stateless/01144_multiword_data_types.sql +++ b/tests/queries/0_stateless/01144_multiword_data_types.sql @@ -23,7 +23,7 @@ CREATE TABLE multiword_types ( SHOW CREATE TABLE multiword_types; INSERT INTO multiword_types(a) VALUES (1); -SELECT toTypeName((*,)) FROM multiword_types; +SELECT toTypeName((*,)) FROM multiword_types SETTINGS enable_named_columns_in_function_tuple = 0; CREATE TABLE unsigned_types ( a TINYINT SIGNED, @@ -43,7 +43,7 @@ CREATE TABLE unsigned_types ( SHOW CREATE TABLE unsigned_types; INSERT INTO unsigned_types(a) VALUES (1); -SELECT toTypeName((*,)) FROM unsigned_types; +SELECT toTypeName((*,)) FROM unsigned_types SETTINGS enable_named_columns_in_function_tuple = 0; SELECT CAST('42' AS DOUBLE PRECISION), CAST(42, 'NATIONAL CHARACTER VARYING'), CAST(-1 AS tinyint UnSiGnEd), CAST(65535, ' sMaLlInT signed '); diff --git a/tests/queries/0_stateless/01232_untuple.reference b/tests/queries/0_stateless/01232_untuple.reference index 0358cde1354..3cd8eaa5611 100644 --- a/tests/queries/0_stateless/01232_untuple.reference +++ b/tests/queries/0_stateless/01232_untuple.reference @@ -2,7 +2,7 @@ hello 1 3 world 9 9 (0,1) -key tupleElement(argMax((v1, v2, v3, v4, v5), v1), \'1\') tupleElement(argMax((v1, v2, v3, v4, v5), v1), \'2\') tupleElement(argMax((v1, v2, v3, v4, v5), v1), \'3\') tupleElement(argMax((v1, v2, v3, v4, v5), v1), \'4\') tupleElement(argMax((v1, v2, v3, v4, v5), v1), \'5\') +key tupleElement(argMax((v1, v2, v3, v4, v5), v1), \'v1\') tupleElement(argMax((v1, v2, v3, v4, v5), v1), \'v2\') tupleElement(argMax((v1, v2, v3, v4, v5), v1), \'v3\') tupleElement(argMax((v1, v2, v3, v4, v5), v1), \'v4\') tupleElement(argMax((v1, v2, v3, v4, v5), v1), \'v5\') 1 20 20 10 20 30 2 11 20 10 20 30 3 70 20 10 20 30 diff --git a/tests/queries/0_stateless/01232_untuple.sql b/tests/queries/0_stateless/01232_untuple.sql index ccefd13a772..391d08ab859 100644 --- a/tests/queries/0_stateless/01232_untuple.sql +++ b/tests/queries/0_stateless/01232_untuple.sql @@ -1,4 +1,5 @@ SET allow_experimental_analyzer = 1; +SET enable_named_columns_in_function_tuple = 1; select untuple((* except (b),)) from (select 1 a, 2 b, 3 c); select 'hello', untuple((* except (b),)), 'world' from (select 1 a, 2 b, 3 c); diff --git a/tests/queries/0_stateless/02294_nothing_arguments_in_functions.sql b/tests/queries/0_stateless/02294_nothing_arguments_in_functions.sql index 4406a05df0c..ecf4f9cab93 100644 --- a/tests/queries/0_stateless/02294_nothing_arguments_in_functions.sql +++ b/tests/queries/0_stateless/02294_nothing_arguments_in_functions.sql @@ -1,3 +1,5 @@ +set enable_named_columns_in_function_tuple = 0; + select arrayMap(x -> 2 * x, []); select toTypeName(arrayMap(x -> 2 * x, [])); select arrayMap((x, y) -> x + y, [], []); diff --git a/tests/queries/0_stateless/02415_all_new_functions_must_be_documented.reference b/tests/queries/0_stateless/02415_all_new_functions_must_be_documented.reference index a152066a460..8dd8910c858 100644 --- a/tests/queries/0_stateless/02415_all_new_functions_must_be_documented.reference +++ b/tests/queries/0_stateless/02415_all_new_functions_must_be_documented.reference @@ -876,7 +876,6 @@ tryBase58Decode tumble tumbleEnd tumbleStart -tuple tupleConcat tupleDivide tupleDivideByNumber diff --git a/tests/queries/0_stateless/02541_tuple_element_with_null.sql b/tests/queries/0_stateless/02541_tuple_element_with_null.sql index d2062b60d49..e1581ce3755 100644 --- a/tests/queries/0_stateless/02541_tuple_element_with_null.sql +++ b/tests/queries/0_stateless/02541_tuple_element_with_null.sql @@ -9,7 +9,7 @@ SETTINGS index_granularity = 8192; INSERT INTO test_tuple_element VALUES (tuple(1,2)), (tuple(NULL, 3)); -SELECT +SELECT tupleElement(tuple, 'k1', 0) fine_k1_with_0, tupleElement(tuple, 'k1', NULL) k1_with_null, tupleElement(tuple, 'k2', 0) k2_with_0, diff --git a/tests/queries/0_stateless/02890_named_tuple_functions.reference b/tests/queries/0_stateless/02890_named_tuple_functions.reference new file mode 100644 index 00000000000..efccfef0817 --- /dev/null +++ b/tests/queries/0_stateless/02890_named_tuple_functions.reference @@ -0,0 +1,8 @@ +Tuple(\n i Int32,\n j Int32) +['i','j'] +Tuple(UInt8, Int32) +['1','2'] +Tuple(\n k UInt8,\n j Int32) +['k','j'] +Tuple(Int32, Int32, Int32, Int32) +['1','2','3','4'] diff --git a/tests/queries/0_stateless/02890_named_tuple_functions.sql b/tests/queries/0_stateless/02890_named_tuple_functions.sql new file mode 100644 index 00000000000..abd24e1cbfe --- /dev/null +++ b/tests/queries/0_stateless/02890_named_tuple_functions.sql @@ -0,0 +1,22 @@ +set enable_named_columns_in_function_tuple = 1; +set allow_experimental_analyzer = 1; + +drop table if exists x; +create table x (i int, j int) engine MergeTree order by i; +insert into x values (1, 2); + +select toTypeName(tuple(i, j)) from x; +select tupleNames(tuple(i, j)) from x; + +select toTypeName(tuple(1, j)) from x; +select tupleNames(tuple(1, j)) from x; + +select toTypeName(tuple(1 as k, j)) from x; +select tupleNames(tuple(1 as k, j)) from x; + +select toTypeName(tuple(i, i, j, j)) from x; +select tupleNames(tuple(i, i, j, j)) from x; + +select tupleNames(1); -- { serverError 43 } + +drop table x; diff --git a/tests/queries/0_stateless/02890_untuple_column_names.reference b/tests/queries/0_stateless/02890_untuple_column_names.reference index 388f974c45f..13a85c70138 100644 --- a/tests/queries/0_stateless/02890_untuple_column_names.reference +++ b/tests/queries/0_stateless/02890_untuple_column_names.reference @@ -57,6 +57,10 @@ t.1: 1 Row 1: ────── t.1: 1 +-- tuple() with enable_named_columns_in_function_tuple = 1 and allow_experimental_analyzer = 1 keeps the column names +Row 1: +────── +t.a: 1 -- thankfully JSONExtract() keeps them Row 1: ────── diff --git a/tests/queries/0_stateless/02890_untuple_column_names.sql b/tests/queries/0_stateless/02890_untuple_column_names.sql index ab6748cb54d..cd490ca3522 100644 --- a/tests/queries/0_stateless/02890_untuple_column_names.sql +++ b/tests/queries/0_stateless/02890_untuple_column_names.sql @@ -37,8 +37,11 @@ SELECT untuple(tuple(1)::Tuple(Int)), untuple(tuple(1)::Tuple(Int)) FORMAT Verti SELECT untuple(tuple(1)::Tuple(Int)), untuple(tuple(1)::Tuple(Int)) FORMAT Vertical SETTINGS allow_experimental_analyzer = 1; -- Bug: doesn't throw an exception SELECT '-- tuple() loses the column names (would be good to fix, see #36773)'; -SELECT untuple(tuple(1 as a)) as t FORMAT Vertical SETTINGS allow_experimental_analyzer = 0; -SELECT untuple(tuple(1 as a)) as t FORMAT Vertical SETTINGS allow_experimental_analyzer = 1; +SELECT untuple(tuple(1 as a)) as t FORMAT Vertical SETTINGS allow_experimental_analyzer = 0, enable_named_columns_in_function_tuple = 0; +SELECT untuple(tuple(1 as a)) as t FORMAT Vertical SETTINGS allow_experimental_analyzer = 1, enable_named_columns_in_function_tuple = 0; + +SELECT '-- tuple() with enable_named_columns_in_function_tuple = 1 and allow_experimental_analyzer = 1 keeps the column names'; +SELECT untuple(tuple(1 as a)) as t FORMAT Vertical SETTINGS allow_experimental_analyzer = 1, enable_named_columns_in_function_tuple = 1; SELECT '-- thankfully JSONExtract() keeps them'; SELECT untuple(JSONExtract('{"key": "value"}', 'Tuple(key String)')) x FORMAT Vertical SETTINGS allow_experimental_analyzer = 0; diff --git a/tests/queries/0_stateless/03038_nested_dynamic_merges.sh b/tests/queries/0_stateless/03038_nested_dynamic_merges.sh index b82ddb3813e..5d8eac082cf 100755 --- a/tests/queries/0_stateless/03038_nested_dynamic_merges.sh +++ b/tests/queries/0_stateless/03038_nested_dynamic_merges.sh @@ -7,7 +7,7 @@ CLICKHOUSE_LOG_COMMENT= # shellcheck source=../shell_config.sh . "$CUR_DIR"/../shell_config.sh -CH_CLIENT="$CLICKHOUSE_CLIENT --allow_experimental_variant_type=1 --use_variant_as_common_type=1 --allow_experimental_dynamic_type=1" +CH_CLIENT="$CLICKHOUSE_CLIENT --allow_experimental_variant_type=1 --use_variant_as_common_type=1 --allow_experimental_dynamic_type=1 --enable_named_columns_in_function_tuple=0" function test() diff --git a/utils/check-style/aspell-ignore/en/aspell-dict.txt b/utils/check-style/aspell-ignore/en/aspell-dict.txt index 02bc520743f..88d94ff3825 100644 --- a/utils/check-style/aspell-ignore/en/aspell-dict.txt +++ b/utils/check-style/aspell-ignore/en/aspell-dict.txt @@ -2797,6 +2797,7 @@ tupleModulo tupleModuloByNumber tupleMultiply tupleMultiplyByNumber +tupleNames tupleNegate tuplePlus tupleToNameValuePairs From 7b19076ebee9a89628d6ef564e155b767f279118 Mon Sep 17 00:00:00 2001 From: Vitaly Baranov Date: Sun, 7 Jul 2024 22:31:45 +0200 Subject: [PATCH 284/417] Avoid using harmful function rand() in grpc. --- contrib/grpc | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/contrib/grpc b/contrib/grpc index 77b2737a709..f5b7fdc2dff 160000 --- a/contrib/grpc +++ b/contrib/grpc @@ -1 +1 @@ -Subproject commit 77b2737a709d43d8c6895e3f03ca62b00bd9201c +Subproject commit f5b7fdc2dff09ada06dbf6c75df298fb40f898df From 97c6cbec46f5c93c2c6199576592a9262aff56f0 Mon Sep 17 00:00:00 2001 From: Blargian Date: Sun, 7 Jul 2024 22:38:43 +0200 Subject: [PATCH 285/417] ad individual window function pages --- .../window-functions/dense_rank.md | 73 ++++++++++++++++++ .../sql-reference/window-functions/index.md | 35 ++++----- .../en/sql-reference/window-functions/rank.md | 74 +++++++++++++++++++ .../window-functions/row_number.md | 0 4 files changed, 165 insertions(+), 17 deletions(-) create mode 100644 docs/en/sql-reference/window-functions/dense_rank.md create mode 100644 docs/en/sql-reference/window-functions/rank.md create mode 100644 docs/en/sql-reference/window-functions/row_number.md diff --git a/docs/en/sql-reference/window-functions/dense_rank.md b/docs/en/sql-reference/window-functions/dense_rank.md new file mode 100644 index 00000000000..17ab894707e --- /dev/null +++ b/docs/en/sql-reference/window-functions/dense_rank.md @@ -0,0 +1,73 @@ +--- +slug: /en/sql-reference/window-functions/dense_rank +sidebar_label: dense_rank +sidebar_position: 2 +--- + +# dense_rank + +This window function ranks the current row within its partition without gaps. In other words, if the value of any new row encountered is equal to the value of one of the previous rows then it will receive the next successive rank without any gaps in ranking. + +The [rank](./rank.md) function provides the same behaviour, but with gaps in ranking. + +**Syntax** + +```sql +dense_rank (column_name) + OVER ([[PARTITION BY grouping_column] [ORDER BY sorting_column] + [ROWS or RANGE expression_to_bound_rows_withing_the_group]] | [window_name]) +FROM table_name +WINDOW window_name as ([[PARTITION BY grouping_column] [ORDER BY sorting_column]) +``` + +For more detail on window function syntax see: [Window Functions - Syntax](./index.md/#syntax). + +**Returned value** + +- A number for the current row within its partition, without gaps in ranking. [UInt64](../data-types/int-uint.md). + +**Example** + +The following example is based on the example provided in the video instructional [Ranking window functions in ClickHouse](https://youtu.be/Yku9mmBYm_4?si=XIMu1jpYucCQEoXA). + +Query: + +```sql +CREATE TABLE salaries +( + `team` String, + `player` String, + `salary` UInt32, + `position` String +) +Engine = Memory; + +INSERT INTO salaries FORMAT Values + ('Port Elizabeth Barbarians', 'Gary Chen', 195000, 'F'), + ('New Coreystad Archdukes', 'Charles Juarez', 190000, 'F'), + ('Port Elizabeth Barbarians', 'Michael Stanley', 150000, 'D'), + ('New Coreystad Archdukes', 'Scott Harrison', 150000, 'D'), + ('Port Elizabeth Barbarians', 'Robert George', 195000, 'M'), + ('South Hampton Seagulls', 'Douglas Benson', 150000, 'M'), + ('South Hampton Seagulls', 'James Henderson', 140000, 'M'); +``` + +```sql +SELECT player, salary, + dense_rank() OVER (ORDER BY salary DESC) AS dense_rank +FROM salaries; +``` + +Result: + +```response + ┌─player──────────┬─salary─┬─dense_rank─┐ +1. │ Gary Chen │ 195000 │ 1 │ +2. │ Robert George │ 195000 │ 1 │ +3. │ Charles Juarez │ 190000 │ 2 │ +4. │ Michael Stanley │ 150000 │ 3 │ +5. │ Douglas Benson │ 150000 │ 3 │ +6. │ Scott Harrison │ 150000 │ 3 │ +7. │ James Henderson │ 140000 │ 4 │ + └─────────────────┴────────┴────────────┘ +``` \ No newline at end of file diff --git a/docs/en/sql-reference/window-functions/index.md b/docs/en/sql-reference/window-functions/index.md index 3a8afd10359..a0246af610f 100644 --- a/docs/en/sql-reference/window-functions/index.md +++ b/docs/en/sql-reference/window-functions/index.md @@ -1,10 +1,11 @@ --- slug: /en/sql-reference/window-functions/ -sidebar_position: 62 sidebar_label: Window Functions -title: Window Functions +sidebar_position: 1 --- +# Window Functions + Windows functions let you perform calculations across a set of rows that are related to the current row. Some of the calculations that you can do are similar to those that can be done with an aggregate function, but a window function doesn't cause rows to be grouped into a single output - the individual rows are still returned. @@ -12,19 +13,19 @@ Some of the calculations that you can do are similar to those that can be done w ClickHouse supports the standard grammar for defining windows and window functions. The table below indicates whether a feature is currently supported. -| Feature | Supported? | -|------------------------------------------------------------------------------------|---------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------| +| Feature | Supported? | +|--------------------------------------------------------------------------|---------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------| | ad hoc window specification (`count(*) over (partition by id order by time desc)`) | ✅ | -| expressions involving window functions, e.g. `(count(*) over ()) / 2)` | ✅ | -| `WINDOW` clause (`select ... from table window w as (partition by id)`) | ✅ | -| `ROWS` frame | ✅ | -| `RANGE` frame | ✅ (the default) | -| `INTERVAL` syntax for `DateTime` `RANGE OFFSET` frame | ❌ (specify the number of seconds instead (`RANGE` works with any numeric type).) | -| `GROUPS` frame | ❌ | -| Calculating aggregate functions over a frame (`sum(value) over (order by time)`) | ✅ (All aggregate functions are supported) | -| `rank()`, `dense_rank()`, `row_number()` | ✅ | -| `lag/lead(value, offset)` | ❌
You can use one of the following workarounds:
1) `any(value) over (.... rows between preceding and preceding)`, or `following` for `lead`
2) `lagInFrame/leadInFrame`, which are analogous, but respect the window frame. To get behavior identical to `lag/lead`, use `rows between unbounded preceding and unbounded following` | -| ntile(buckets) | ✅
Specify window like, (partition by x order by y rows between unbounded preceding and unrounded following). | +| expressions involving window functions, e.g. `(count(*) over ()) / 2)` | ✅ | +| `WINDOW` clause (`select ... from table window w as (partition by id)`) | ✅ | +| `ROWS` frame | ✅ | +| `RANGE` frame | ✅ (the default) | +| `INTERVAL` syntax for `DateTime` `RANGE OFFSET` frame | ❌ (specify the number of seconds instead (`RANGE` works with any numeric type).) | +| `GROUPS` frame | ❌ | +| Calculating aggregate functions over a frame (`sum(value) over (order by time)`) | ✅ (All aggregate functions are supported) | +| `rank()`, `dense_rank()`, `row_number()` | ✅ | +| `lag/lead(value, offset)` | ❌
You can use one of the following workarounds:
1) `any(value) over (.... rows between preceding and preceding)`, or `following` for `lead`
2) `lagInFrame/leadInFrame`, which are analogous, but respect the window frame. To get behavior identical to `lag/lead`, use `rows between unbounded preceding and unbounded following` | +| ntile(buckets) | ✅
Specify window like, (partition by x order by y rows between unbounded preceding and unrounded following). | ## ClickHouse-specific Window Functions @@ -74,12 +75,12 @@ WINDOW window_name as ([[PARTITION BY grouping_column] [ORDER BY sorting_column] These functions can be used only as a window function. -- `row_number()` - Number the current row within its partition starting from 1. +- [`row_number()`](./row_number.md) - Number the current row within its partition starting from 1. - `first_value(x)` - Return the first non-NULL value evaluated within its ordered frame. - `last_value(x)` - Return the last non-NULL value evaluated within its ordered frame. - `nth_value(x, offset)` - Return the first non-NULL value evaluated against the nth row (offset) in its ordered frame. -- `rank()` - Rank the current row within its partition with gaps. -- `dense_rank()` - Rank the current row within its partition without gaps. +- [`rank()`](./rank.md) - Rank the current row within its partition with gaps. +- [`dense_rank()`](./dense_rank.md) - Rank the current row within its partition without gaps. - `lagInFrame(x)` - Return a value evaluated at the row that is at a specified physical offset row before the current row within the ordered frame. - `leadInFrame(x)` - Return a value evaluated at the row that is offset rows after the current row within the ordered frame. diff --git a/docs/en/sql-reference/window-functions/rank.md b/docs/en/sql-reference/window-functions/rank.md new file mode 100644 index 00000000000..17db889ef92 --- /dev/null +++ b/docs/en/sql-reference/window-functions/rank.md @@ -0,0 +1,74 @@ +--- +slug: /en/sql-reference/window-functions/rank +sidebar_label: rank +sidebar_position: 3 +--- + +# rank + +This window function ranks the current row within its partition with gaps. In other words, if the value of any row it encounters is equal to the value of a previous row then it will receive the same rank as that previous row. +The rank of the next row is then equal to the rank of the previous row plus a gap equal to the number of times the previous rank was given. + +The [dense_rank](./dense_rank.md) function provides the same behaviour but without gaps in ranking. + +**Syntax** + +```sql +rank (column_name) + OVER ([[PARTITION BY grouping_column] [ORDER BY sorting_column] + [ROWS or RANGE expression_to_bound_rows_withing_the_group]] | [window_name]) +FROM table_name +WINDOW window_name as ([[PARTITION BY grouping_column] [ORDER BY sorting_column]) +``` + +For more detail on window function syntax see: [Window Functions - Syntax](./index.md/#syntax). + +**Returned value** + +- A number for the current row within its partition, including gaps. [UInt64](../data-types/int-uint.md). + +**Example** + +The following example is based on the example provided in the video instructional [Ranking window functions in ClickHouse](https://youtu.be/Yku9mmBYm_4?si=XIMu1jpYucCQEoXA). + +Query: + +```sql +CREATE TABLE salaries +( + `team` String, + `player` String, + `salary` UInt32, + `position` String +) +Engine = Memory; + +INSERT INTO salaries FORMAT Values + ('Port Elizabeth Barbarians', 'Gary Chen', 195000, 'F'), + ('New Coreystad Archdukes', 'Charles Juarez', 190000, 'F'), + ('Port Elizabeth Barbarians', 'Michael Stanley', 150000, 'D'), + ('New Coreystad Archdukes', 'Scott Harrison', 150000, 'D'), + ('Port Elizabeth Barbarians', 'Robert George', 195000, 'M'), + ('South Hampton Seagulls', 'Douglas Benson', 150000, 'M'), + ('South Hampton Seagulls', 'James Henderson', 140000, 'M'); +``` + +```sql +SELECT player, salary, + rank() OVER (ORDER BY salary DESC) AS rank +FROM salaries; +``` + +Result: + +```response + ┌─player──────────┬─salary─┬─rank─┐ +1. │ Gary Chen │ 195000 │ 1 │ +2. │ Robert George │ 195000 │ 1 │ +3. │ Charles Juarez │ 190000 │ 3 │ +4. │ Douglas Benson │ 150000 │ 4 │ +5. │ Michael Stanley │ 150000 │ 4 │ +6. │ Scott Harrison │ 150000 │ 4 │ +7. │ James Henderson │ 140000 │ 7 │ + └─────────────────┴────────┴──────┘ +``` \ No newline at end of file diff --git a/docs/en/sql-reference/window-functions/row_number.md b/docs/en/sql-reference/window-functions/row_number.md new file mode 100644 index 00000000000..e69de29bb2d From 0b1e6be8ea6ee68d501c6d1f9bd1a5d7eb02d738 Mon Sep 17 00:00:00 2001 From: Blargian Date: Sun, 7 Jul 2024 22:46:18 +0200 Subject: [PATCH 286/417] add disclaimer --- .../aggregate-functions/reference/singlevalueornull.md | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/docs/en/sql-reference/aggregate-functions/reference/singlevalueornull.md b/docs/en/sql-reference/aggregate-functions/reference/singlevalueornull.md index 21344b58ba6..19154c488d9 100644 --- a/docs/en/sql-reference/aggregate-functions/reference/singlevalueornull.md +++ b/docs/en/sql-reference/aggregate-functions/reference/singlevalueornull.md @@ -16,7 +16,7 @@ singleValueOrNull(x) **Parameters** -- `x` — Column of any [data type](../../data-types/index.md). +- `x` — Column of any [data type](../../data-types/index.md) (except [Map](../../data-types/map.md), [Array](../../data-types/array.md) or [Tuple](../../data-types/tuple) which cannot be of type [Nullable](../../data-types/nullable.md)). **Returned values** From e16cb83ca3071eb9aecf924aa17af21e1bb76450 Mon Sep 17 00:00:00 2001 From: gun9nir Date: Sun, 7 Jul 2024 19:24:29 -0700 Subject: [PATCH 287/417] style --- src/Storages/StorageFile.cpp | 7 +++++-- 1 file changed, 5 insertions(+), 2 deletions(-) diff --git a/src/Storages/StorageFile.cpp b/src/Storages/StorageFile.cpp index 9ff2a6667af..8797f6a3dfa 100644 --- a/src/Storages/StorageFile.cpp +++ b/src/Storages/StorageFile.cpp @@ -366,14 +366,17 @@ Strings StorageFile::getPathsList(const String & table_path, const String & user } else if (path.find_first_of("*?{") == std::string::npos) { - if (!fs::is_directory(path)) { + if (!fs::is_directory(path)) + { std::error_code error; size_t size = fs::file_size(path, error); if (!error) total_bytes_to_read += size; paths.push_back(path); - } else { + } + else + { /// We list non-directory files under that directory. paths = listFilesWithRegexpMatching(path / fs::path("*"), total_bytes_to_read); can_be_directory = false; From 7b3ce3c3b38f698eb80923061c8ec0e309e2cff6 Mon Sep 17 00:00:00 2001 From: Blargian Date: Mon, 8 Jul 2024 06:20:10 +0200 Subject: [PATCH 288/417] add leadInFrame, lagInFrame, row_number --- .../window-functions/lagInFrame.md | 79 +++++++++++++++++++ .../window-functions/leadInFrame.md | 60 ++++++++++++++ .../window-functions/row_number.md | 67 ++++++++++++++++ 3 files changed, 206 insertions(+) create mode 100644 docs/en/sql-reference/window-functions/lagInFrame.md create mode 100644 docs/en/sql-reference/window-functions/leadInFrame.md diff --git a/docs/en/sql-reference/window-functions/lagInFrame.md b/docs/en/sql-reference/window-functions/lagInFrame.md new file mode 100644 index 00000000000..ea9f6d9dea2 --- /dev/null +++ b/docs/en/sql-reference/window-functions/lagInFrame.md @@ -0,0 +1,79 @@ +--- +slug: /en/sql-reference/window-functions/lagInFrame +sidebar_label: lagInFrame +sidebar_position: 5 +--- + +# lagInFrame + +Return a value evaluated at the row that is at a specified physical offset before the current row within the ordered frame. The offset parameter, if not specified, defaults to 1, meaning it will fetch the value from the next row. If the calculated row exceeds the boundaries of the window frame, the specified default value is returned. + +**Syntax** + +```sql +lagInFrame(x[, offset[, default]]) + OVER ([[PARTITION BY grouping_column] [ORDER BY sorting_column] + [ROWS or RANGE expression_to_bound_rows_withing_the_group]] | [window_name]) +FROM table_name +WINDOW window_name as ([[PARTITION BY grouping_column] [ORDER BY sorting_column]) +``` + +For more detail on window function syntax see: [Window Functions - Syntax](./index.md/#syntax). + +**Parameters** +- `x` — Column name. +- `offset` — Offset to apply. [(U)Int*](../data-types/int-uint.md). (Optional - `1` by default). +- `default` — Value to return if calculated row exceeds the boundaries of the window frame. (Optional - `null` by default). + +**Returned value** + +- Value evaluated at the row that is at a specified physical offset before the current row within the ordered frame. + +**Example** + +This example looks at historical data for a specific stock and uses the `lagInFrame` function to calculate a day-to-day delta and percentage change in the closing price of the stock. + +Query: + +```sql +CREATE TABLE stock_prices +( + `date` Date, + `open` Float32, -- opening price + `high` Float32, -- daily high + `low` Float32, -- daily low + `close` Float32, -- closing price + `volume` UInt32 -- trade volume +) +Engine = Memory; + +INSERT INTO stock_prices FORMAT Values + ('2024-06-03', 113.62, 115.00, 112.00, 115.00, 438392000), + ('2024-06-04', 115.72, 116.60, 114.04, 116.44, 403324000), + ('2024-06-05', 118.37, 122.45, 117.47, 122.44, 528402000), + ('2024-06-06', 124.05, 125.59, 118.32, 121.00, 664696000), + ('2024-06-07', 119.77, 121.69, 118.02, 120.89, 412386000); +``` + +```sql +SELECT + date, + close, + lagInFrame(close, 1, close) OVER (ORDER BY date ASC) AS previous_day_close, + COALESCE(ROUND(close - previous_day_close, 2)) AS delta, + COALESCE(ROUND((delta / previous_day_close) * 100, 2)) AS percent_change +FROM stock_prices +ORDER BY date DESC; +``` + +Result: + +```response + ┌───────date─┬──close─┬─previous_day_close─┬─delta─┬─percent_change─┐ +1. │ 2024-06-07 │ 120.89 │ 121 │ -0.11 │ -0.09 │ +2. │ 2024-06-06 │ 121 │ 122.44 │ -1.44 │ -1.18 │ +3. │ 2024-06-05 │ 122.44 │ 116.44 │ 6 │ 5.15 │ +4. │ 2024-06-04 │ 116.44 │ 115 │ 1.44 │ 1.25 │ +5. │ 2024-06-03 │ 115 │ 115 │ 0 │ 0 │ + └────────────┴────────┴────────────────────┴───────┴────────────────┘ +``` \ No newline at end of file diff --git a/docs/en/sql-reference/window-functions/leadInFrame.md b/docs/en/sql-reference/window-functions/leadInFrame.md new file mode 100644 index 00000000000..e3b65af9a4d --- /dev/null +++ b/docs/en/sql-reference/window-functions/leadInFrame.md @@ -0,0 +1,60 @@ +--- +slug: /en/sql-reference/window-functions/leadInFrame +sidebar_label: leadInFrame +sidebar_position: 6 +--- + +# leadInFrame + +Return a value evaluated at the row that is offset rows after the current row within the ordered frame. + +**Syntax** + +```sql +leadInFrame(x[, offset[, default]]) + OVER ([[PARTITION BY grouping_column] [ORDER BY sorting_column] + [ROWS or RANGE expression_to_bound_rows_withing_the_group]] | [window_name]) +FROM table_name +WINDOW window_name as ([[PARTITION BY grouping_column] [ORDER BY sorting_column]) +``` + +For more detail on window function syntax see: [Window Functions - Syntax](./index.md/#syntax). + +**Parameters** +- `x` — Column name. +- `offset` — Offset to apply. [(U)Int*](../data-types/int-uint.md). (Optional - `1` by default). +- `default` — Value to return if calculated row exceeds the boundaries of the window frame. (Optional - `null` by default). + +**Returned value** + +- value evaluated at the row that is offset rows after the current row within the ordered frame. + +**Example** + +This example looks at [historical data](https://www.kaggle.com/datasets/sazidthe1/nobel-prize-data) for Nobel Prize winners and uses the `leadInFrame` function to return a list of successive winners in the physics category. + +Query: + +```sql +CREATE OR REPLACE VIEW nobel_prize_laureates AS FROM file('nobel_laureates_data.csv') SELECT *; +``` + +```sql +FROM nobel_prize_laureates SELECT fullName, leadInFrame(year, 1, year) OVER (PARTITION BY category ORDER BY year) AS year, category, motivation WHERE category == 'physics' ORDER BY year DESC LIMIT 9; +``` + +Result: + +```response + ┌─fullName─────────┬─year─┬─category─┬─motivation─────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────┐ +1. │ Pierre Agostini │ 2023 │ physics │ for experimental methods that generate attosecond pulses of light for the study of electron dynamics in matter │ +2. │ Ferenc Krausz │ 2023 │ physics │ for experimental methods that generate attosecond pulses of light for the study of electron dynamics in matter │ +3. │ Anne L Huillier │ 2023 │ physics │ for experimental methods that generate attosecond pulses of light for the study of electron dynamics in matter │ +4. │ Alain Aspect │ 2022 │ physics │ for experiments with entangled photons establishing the violation of Bell inequalities and pioneering quantum information science │ +5. │ Anton Zeilinger │ 2022 │ physics │ for experiments with entangled photons establishing the violation of Bell inequalities and pioneering quantum information science │ +6. │ John Clauser │ 2022 │ physics │ for experiments with entangled photons establishing the violation of Bell inequalities and pioneering quantum information science │ +7. │ Syukuro Manabe │ 2021 │ physics │ for the physical modelling of Earths climate quantifying variability and reliably predicting global warming │ +8. │ Klaus Hasselmann │ 2021 │ physics │ for the physical modelling of Earths climate quantifying variability and reliably predicting global warming │ +9. │ Giorgio Parisi │ 2021 │ physics │ for the discovery of the interplay of disorder and fluctuations in physical systems from atomic to planetary scales │ + └──────────────────┴──────┴──────────┴────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────┘ +``` \ No newline at end of file diff --git a/docs/en/sql-reference/window-functions/row_number.md b/docs/en/sql-reference/window-functions/row_number.md index e69de29bb2d..428bb34a8ba 100644 --- a/docs/en/sql-reference/window-functions/row_number.md +++ b/docs/en/sql-reference/window-functions/row_number.md @@ -0,0 +1,67 @@ +--- +slug: /en/sql-reference/window-functions/row_number +sidebar_label: row_number +sidebar_position: 4 +--- + +# row_number + +Numbers the current row within its partition starting from 1 + +**Syntax** + +```sql +row_number (column_name) + OVER ([[PARTITION BY grouping_column] [ORDER BY sorting_column] + [ROWS or RANGE expression_to_bound_rows_withing_the_group]] | [window_name]) +FROM table_name +WINDOW window_name as ([[PARTITION BY grouping_column] [ORDER BY sorting_column]) +``` + +For more detail on window function syntax see: [Window Functions - Syntax](./index.md/#syntax). + +**Returned value** + +- A number for the current row within its partition. [UInt64](../data-types/int-uint.md). + +**Example** + +The following example is based on the example provided in the video instructional [Ranking window functions in ClickHouse](https://youtu.be/Yku9mmBYm_4?si=XIMu1jpYucCQEoXA). + +Query: + +```sql +CREATE TABLE salaries +( + `team` String, + `player` String, + `salary` UInt32, + `position` String +) +Engine = Memory; + +INSERT INTO salaries FORMAT Values + ('Port Elizabeth Barbarians', 'Gary Chen', 195000, 'F'), + ('New Coreystad Archdukes', 'Charles Juarez', 190000, 'F'), + ('Port Elizabeth Barbarians', 'Michael Stanley', 150000, 'D'), + ('New Coreystad Archdukes', 'Scott Harrison', 150000, 'D'), + ('Port Elizabeth Barbarians', 'Robert George', 195000, 'M'); +``` + +```sql +SELECT player, salary, + row_number() OVER (ORDER BY salary DESC) AS row_number +FROM salaries; +``` + +Result: + +```response + ┌─player──────────┬─salary─┬─row_number─┐ +1. │ Gary Chen │ 195000 │ 1 │ +2. │ Robert George │ 195000 │ 2 │ +3. │ Charles Juarez │ 190000 │ 3 │ +4. │ Scott Harrison │ 150000 │ 4 │ +5. │ Michael Stanley │ 150000 │ 5 │ + └─────────────────┴────────┴────────────┘ +``` \ No newline at end of file From 3fa1fd321fe3e2be5155b87795912f4ab3ca8f48 Mon Sep 17 00:00:00 2001 From: Robert Schulze Date: Mon, 8 Jul 2024 07:38:07 +0000 Subject: [PATCH 289/417] Fix typo --- docs/en/operations/system-tables/metrics.md | 2 +- src/Common/CurrentMetrics.cpp | 2 +- src/Coordination/KeeperConstants.cpp | 2 +- src/Coordination/KeeperDispatcher.cpp | 14 +++++++------- utils/check-style/aspell-ignore/en/aspell-dict.txt | 2 +- 5 files changed, 11 insertions(+), 11 deletions(-) diff --git a/docs/en/operations/system-tables/metrics.md b/docs/en/operations/system-tables/metrics.md index 83ce817b7db..f253b164e2a 100644 --- a/docs/en/operations/system-tables/metrics.md +++ b/docs/en/operations/system-tables/metrics.md @@ -357,7 +357,7 @@ Number of currently running inserts to Kafka Number of alive connections -### KeeperOutstandingRequets +### KeeperOutstandingRequests Number of outstanding requests diff --git a/src/Common/CurrentMetrics.cpp b/src/Common/CurrentMetrics.cpp index 8516a88c7af..7c97e73f278 100644 --- a/src/Common/CurrentMetrics.cpp +++ b/src/Common/CurrentMetrics.cpp @@ -267,7 +267,7 @@ M(AsyncInsertCacheSize, "Number of async insert hash id in cache") \ M(S3Requests, "S3 requests count") \ M(KeeperAliveConnections, "Number of alive connections") \ - M(KeeperOutstandingRequets, "Number of outstanding requests") \ + M(KeeperOutstandingRequests, "Number of outstanding requests") \ M(ThreadsInOvercommitTracker, "Number of waiting threads inside of OvercommitTracker") \ M(IOUringPendingEvents, "Number of io_uring SQEs waiting to be submitted") \ M(IOUringInFlightEvents, "Number of io_uring SQEs in flight") \ diff --git a/src/Coordination/KeeperConstants.cpp b/src/Coordination/KeeperConstants.cpp index b4241235cc7..7589e3393be 100644 --- a/src/Coordination/KeeperConstants.cpp +++ b/src/Coordination/KeeperConstants.cpp @@ -372,7 +372,7 @@ extern const std::vector keeper_profile_events M(AsynchronousReadWait) \ M(S3Requests) \ M(KeeperAliveConnections) \ - M(KeeperOutstandingRequets) \ + M(KeeperOutstandingRequests) \ M(ThreadsInOvercommitTracker) \ M(IOUringPendingEvents) \ M(IOUringInFlightEvents) \ diff --git a/src/Coordination/KeeperDispatcher.cpp b/src/Coordination/KeeperDispatcher.cpp index f36b1ef151f..6f57fa6d2e2 100644 --- a/src/Coordination/KeeperDispatcher.cpp +++ b/src/Coordination/KeeperDispatcher.cpp @@ -28,7 +28,7 @@ namespace CurrentMetrics { extern const Metric KeeperAliveConnections; - extern const Metric KeeperOutstandingRequets; + extern const Metric KeeperOutstandingRequests; } namespace ProfileEvents @@ -139,7 +139,7 @@ void KeeperDispatcher::requestThread() { if (requests_queue->tryPop(request, max_wait)) { - CurrentMetrics::sub(CurrentMetrics::KeeperOutstandingRequets); + CurrentMetrics::sub(CurrentMetrics::KeeperOutstandingRequests); if (shutdown_called) break; @@ -171,7 +171,7 @@ void KeeperDispatcher::requestThread() /// Trying to get batch requests as fast as possible if (requests_queue->tryPop(request)) { - CurrentMetrics::sub(CurrentMetrics::KeeperOutstandingRequets); + CurrentMetrics::sub(CurrentMetrics::KeeperOutstandingRequests); /// Don't append read request into batch, we have to process them separately if (!coordination_settings->quorum_reads && request.request->isReadRequest()) { @@ -419,7 +419,7 @@ bool KeeperDispatcher::putRequest(const Coordination::ZooKeeperRequestPtr & requ { throw Exception(ErrorCodes::TIMEOUT_EXCEEDED, "Cannot push request to queue within operation timeout"); } - CurrentMetrics::add(CurrentMetrics::KeeperOutstandingRequets); + CurrentMetrics::add(CurrentMetrics::KeeperOutstandingRequests); return true; } @@ -543,7 +543,7 @@ void KeeperDispatcher::shutdown() /// Set session expired for all pending requests while (requests_queue && requests_queue->tryPop(request_for_session)) { - CurrentMetrics::sub(CurrentMetrics::KeeperOutstandingRequets); + CurrentMetrics::sub(CurrentMetrics::KeeperOutstandingRequests); auto response = request_for_session.request->makeResponse(); response->error = Coordination::Error::ZSESSIONEXPIRED; setResponse(request_for_session.session_id, response); @@ -670,7 +670,7 @@ void KeeperDispatcher::sessionCleanerTask() }; if (!requests_queue->push(std::move(request_info))) LOG_INFO(log, "Cannot push close request to queue while cleaning outdated sessions"); - CurrentMetrics::add(CurrentMetrics::KeeperOutstandingRequets); + CurrentMetrics::add(CurrentMetrics::KeeperOutstandingRequests); /// Remove session from registered sessions finishSession(dead_session); @@ -794,7 +794,7 @@ int64_t KeeperDispatcher::getSessionID(int64_t session_timeout_ms) /// Push new session request to queue if (!requests_queue->tryPush(std::move(request_info), session_timeout_ms)) throw Exception(ErrorCodes::TIMEOUT_EXCEEDED, "Cannot push session id request to queue within session timeout"); - CurrentMetrics::add(CurrentMetrics::KeeperOutstandingRequets); + CurrentMetrics::add(CurrentMetrics::KeeperOutstandingRequests); if (future.wait_for(std::chrono::milliseconds(session_timeout_ms)) != std::future_status::ready) throw Exception(ErrorCodes::TIMEOUT_EXCEEDED, "Cannot receive session id within session timeout"); diff --git a/utils/check-style/aspell-ignore/en/aspell-dict.txt b/utils/check-style/aspell-ignore/en/aspell-dict.txt index 02bc520743f..cbda66b7bf9 100644 --- a/utils/check-style/aspell-ignore/en/aspell-dict.txt +++ b/utils/check-style/aspell-ignore/en/aspell-dict.txt @@ -449,7 +449,7 @@ Kahan Kaser KeeperAliveConnections KeeperMap -KeeperOutstandingRequets +KeeperOutstandingRequests Kerberos Khanna KittenHouse From 5812a65ea091b4455123d5e29a8c161c29c5fb8b Mon Sep 17 00:00:00 2001 From: Robert Schulze Date: Mon, 8 Jul 2024 07:46:57 +0000 Subject: [PATCH 290/417] Rename test (query cache tests start with 02494) --- ...ty_tuple.reference => 02494_query_cache_empty_tuple.reference} | 0 ...ry_cache_empty_tuple.sql => 02494_query_cache_empty_tuple.sql} | 0 2 files changed, 0 insertions(+), 0 deletions(-) rename tests/queries/0_stateless/{03201_query_cache_empty_tuple.reference => 02494_query_cache_empty_tuple.reference} (100%) rename tests/queries/0_stateless/{03201_query_cache_empty_tuple.sql => 02494_query_cache_empty_tuple.sql} (100%) diff --git a/tests/queries/0_stateless/03201_query_cache_empty_tuple.reference b/tests/queries/0_stateless/02494_query_cache_empty_tuple.reference similarity index 100% rename from tests/queries/0_stateless/03201_query_cache_empty_tuple.reference rename to tests/queries/0_stateless/02494_query_cache_empty_tuple.reference diff --git a/tests/queries/0_stateless/03201_query_cache_empty_tuple.sql b/tests/queries/0_stateless/02494_query_cache_empty_tuple.sql similarity index 100% rename from tests/queries/0_stateless/03201_query_cache_empty_tuple.sql rename to tests/queries/0_stateless/02494_query_cache_empty_tuple.sql From e0d3213481769299881044512d78d23551c2aa8a Mon Sep 17 00:00:00 2001 From: Eduard Karacharov Date: Sun, 16 Jun 2024 20:43:26 +0300 Subject: [PATCH 291/417] support set transformation in partition pruning --- src/Storages/MergeTree/KeyCondition.cpp | 202 +++++++++++++- src/Storages/MergeTree/KeyCondition.h | 14 +- ...et_transformed_partition_pruning.reference | 50 ++++ ...3173_set_transformed_partition_pruning.sql | 258 ++++++++++++++++++ 4 files changed, 521 insertions(+), 3 deletions(-) create mode 100644 tests/queries/0_stateless/03173_set_transformed_partition_pruning.reference create mode 100644 tests/queries/0_stateless/03173_set_transformed_partition_pruning.sql diff --git a/src/Storages/MergeTree/KeyCondition.cpp b/src/Storages/MergeTree/KeyCondition.cpp index 7e4b1db4c89..85eca4644e7 100644 --- a/src/Storages/MergeTree/KeyCondition.cpp +++ b/src/Storages/MergeTree/KeyCondition.cpp @@ -1186,15 +1186,62 @@ bool KeyCondition::canConstantBeWrappedByFunctions( }); } +/// Sequentially applies functions to set column, returns `true` +/// if all function arguments are compatible with functions +/// signatures, and none of the functions produce `NULL` output. +/// +/// After functions chain execution, fills result set column and +/// its type. +bool applyFunctionChainToSetValues( + const ColumnPtr & set_column, + const DataTypePtr & set_data_type, + const std::vector & functions, + ColumnPtr & out_set_column, + DataTypePtr & out_set_type) +{ + auto result_column = set_column->convertToFullColumnIfLowCardinality(); + auto result_type = removeLowCardinality(set_data_type); + + for (const auto & func : functions) + { + if (func->getArgumentTypes().empty()) + return false; + + auto argument_type = func->getArgumentTypes()[0]; + if (!canBeSafelyCasted(result_type, argument_type)) + return false; + + result_column = castColumnAccurate({result_column, result_type, ""}, argument_type); + result_column = func->execute({{result_column, argument_type, ""}}, func->getResultType(), result_column->size()); + if (result_column->isNullable()) + { + const auto & result_column_nullable = assert_cast(*result_column); + const auto & null_map_data = result_column_nullable.getNullMapData(); + for (char8_t i : null_map_data) + { + if (i != 0) + return false; + } + } + result_type = func->getResultType(); + } + out_set_column = result_column; + out_set_type = result_type; + + return true; +} + bool KeyCondition::tryPrepareSetIndex( const RPNBuilderFunctionTreeNode & func, RPNElement & out, - size_t & out_key_column_num) + size_t & out_key_column_num, + bool & is_constant_transformed) { const auto & left_arg = func.getArgumentAt(0); out_key_column_num = 0; std::vector indexes_mapping; + std::vector set_transforming_chains; DataTypes data_types; auto get_key_tuple_position_mapping = [&](const RPNBuilderTreeNode & node, size_t tuple_index) @@ -1203,6 +1250,7 @@ bool KeyCondition::tryPrepareSetIndex( index_mapping.tuple_index = tuple_index; DataTypePtr data_type; std::optional key_space_filling_curve_argument_pos; + MonotonicFunctionsChain set_transforming_chain; if (isKeyPossiblyWrappedByMonotonicFunctions( node, index_mapping.key_index, key_space_filling_curve_argument_pos, data_type, index_mapping.functions) && !key_space_filling_curve_argument_pos) /// We don't support the analysis of space-filling curves and IN set. @@ -1210,6 +1258,14 @@ bool KeyCondition::tryPrepareSetIndex( indexes_mapping.push_back(index_mapping); data_types.push_back(data_type); out_key_column_num = std::max(out_key_column_num, index_mapping.key_index); + set_transforming_chains.push_back(set_transforming_chain); + } + else if (single_point && canSetValuesBeWrappedByFunctions(node, index_mapping.key_index, data_type, set_transforming_chain)) + { + indexes_mapping.push_back(index_mapping); + data_types.push_back(data_type); + out_key_column_num = std::max(out_key_column_num, index_mapping.key_index); + set_transforming_chains.push_back(set_transforming_chain); } }; @@ -1275,6 +1331,18 @@ bool KeyCondition::tryPrepareSetIndex( auto set_element_type = set_types[set_element_index]; auto set_column = set_columns[set_element_index]; + if (!set_transforming_chains[indexes_mapping_index].empty()) + { + ColumnPtr transformed_set_column; + DataTypePtr transformed_set_type; + if (!applyFunctionChainToSetValues(set_column, set_element_type, set_transforming_chains[indexes_mapping_index], transformed_set_column, transformed_set_type)) + return false; + + set_column = transformed_set_column; + set_element_type = transformed_set_type; + is_constant_transformed = true; + } + if (canBeSafelyCasted(set_element_type, key_column_type)) { set_columns[set_element_index] = castColumn({set_column, set_element_type, {}}, key_column_type); @@ -1571,6 +1639,136 @@ bool KeyCondition::isKeyPossiblyWrappedByMonotonicFunctionsImpl( return false; } +bool KeyCondition::canSetValuesBeWrappedByFunctions( + const RPNBuilderTreeNode & node, + size_t & out_key_column_num, + DataTypePtr & out_key_res_column_type, + MonotonicFunctionsChain & out_functions_chain) +{ + // Checking if column name matches any of key subexpressions + String expr_name = node.getColumnName(); + + if (array_joined_column_names.contains(expr_name)) + return false; + + if (!key_subexpr_names.contains(expr_name)) + { + expr_name = node.getColumnNameWithModuloLegacy(); + + if (!key_subexpr_names.contains(expr_name)) + return false; + } + + // If match has been found, need to identify key column, and + // sequence of function nodes, which forms key column + for (const auto & key_expr_node : key_expr->getNodes()) + { + auto it = key_columns.find(key_expr_node.result_name); + if (it != key_columns.end()) + { + std::vector chain; + + const auto * cur_node = &key_expr_node; + bool is_valid_chain = true; + + while (is_valid_chain) + { + if (cur_node->result_name == expr_name) + break; + + if (cur_node->type == ActionsDAG::ActionType::FUNCTION && cur_node->children.size() <= 2) + { + chain.push_back(cur_node); + is_valid_chain = cur_node->function_base->isDeterministic(); + + const ActionsDAG::Node * next_node = nullptr; + for (const auto * arg : cur_node->children) + { + if (arg->column && isColumnConst(*arg->column)) + continue; + + if (next_node) + is_valid_chain = false; + + next_node = arg; + } + + if (!next_node) + is_valid_chain = false; + + cur_node = next_node; + } + else if (cur_node->type == ActionsDAG::ActionType::ALIAS) + cur_node = cur_node->children.front(); + else + is_valid_chain = false; + } + + // If key column has been found, and it consists of deterministic function, + // in order to make them applicable to constants from query predicate, + // reversing the sequence of functions, and binding constant arguments + if (is_valid_chain) + { + // Removing low cardinality from `node` output type to avoid + // passing it through functions signatures + auto nonconst_input_type = removeLowCardinality(node.getDAGNode()->result_type); + + for (auto iter = chain.rbegin(); iter != chain.rend(); ++iter) + { + const auto * function_node = *iter; + auto function = function_node->function_base; + auto func_builder = FunctionFactory::instance().tryGet(function->getName(), node.getTreeContext().getQueryContext()); + if (!func_builder) + return false; + ColumnsWithTypeAndName arguments; + ColumnWithTypeAndName const_arg; + FunctionWithOptionalConstArg::Kind kind = FunctionWithOptionalConstArg::Kind::NO_CONST; + if (function_node->children.size() == 2) + { + const auto * left = function_node->children[0]; + const auto * right = function_node->children[1]; + if (left->column && isColumnConst(*left->column)) + { + const_arg = {left->result_type->createColumnConst(0, (*left->column)[0]), left->result_type, ""}; + arguments.push_back(const_arg); + arguments.push_back({nullptr, nonconst_input_type, ""}); + kind = FunctionWithOptionalConstArg::Kind::LEFT_CONST; + } + else if (right->column && isColumnConst(*right->column)) + { + const_arg = {right->result_type->createColumnConst(0, (*right->column)[0]), right->result_type, ""}; + arguments.push_back({nullptr, nonconst_input_type, ""}); + arguments.push_back(const_arg); + kind = FunctionWithOptionalConstArg::Kind::RIGHT_CONST; + } + + /// If constant arg of binary operator is NULL, there will be no monotonicity. + if (const_arg.column->isNullAt(0)) + return false; + } + else + arguments.push_back({nullptr, nonconst_input_type, ""}); + auto func = func_builder->build(arguments); + + if (kind == FunctionWithOptionalConstArg::Kind::NO_CONST) + out_functions_chain.push_back(func); + else + out_functions_chain.push_back(std::make_shared(func, const_arg, kind)); + + nonconst_input_type = func->getResultType(); + } + + const auto & sample_block = key_expr->getSampleBlock(); + out_key_column_num = it->second; + out_key_res_column_type = sample_block.getByName(it->first).type; + + return true; + } + } + } + + return false; +} static void castValueToType(const DataTypePtr & desired_type, Field & src_value, const DataTypePtr & src_type, const String & node_column_name) { @@ -1649,7 +1847,7 @@ bool KeyCondition::extractAtomFromTree(const RPNBuilderTreeNode & node, RPNEleme if (functionIsInOrGlobalInOperator(func_name)) { - if (tryPrepareSetIndex(func, out, key_column_num)) + if (tryPrepareSetIndex(func, out, key_column_num, is_constant_transformed)) { key_arg_pos = 0; is_set_const = true; diff --git a/src/Storages/MergeTree/KeyCondition.h b/src/Storages/MergeTree/KeyCondition.h index 6e5956706aa..11dd1c85c9d 100644 --- a/src/Storages/MergeTree/KeyCondition.h +++ b/src/Storages/MergeTree/KeyCondition.h @@ -276,13 +276,25 @@ private: Field & out_value, DataTypePtr & out_type); + /// Checks if node is a subexpression of any of key columns expressions, + /// wrapped by deterministic functions, and if so, returns `true`, and + /// specifies key column position / type. Besides that it produces the + /// chain of functions which should be executed on set, to transform it + /// into key column values. + bool canSetValuesBeWrappedByFunctions( + const RPNBuilderTreeNode & node, + size_t & out_key_column_num, + DataTypePtr & out_key_res_column_type, + MonotonicFunctionsChain & out_functions_chain); + /// If it's possible to make an RPNElement /// that will filter values (possibly tuples) by the content of 'prepared_set', /// do it and return true. bool tryPrepareSetIndex( const RPNBuilderFunctionTreeNode & func, RPNElement & out, - size_t & out_key_column_num); + size_t & out_key_column_num, + bool & is_constant_transformed); /// Checks that the index can not be used. /// diff --git a/tests/queries/0_stateless/03173_set_transformed_partition_pruning.reference b/tests/queries/0_stateless/03173_set_transformed_partition_pruning.reference new file mode 100644 index 00000000000..3a6727b70e8 --- /dev/null +++ b/tests/queries/0_stateless/03173_set_transformed_partition_pruning.reference @@ -0,0 +1,50 @@ +-- Single partition by function +0 +2 +-- Nested partition by function +1 +2 +1 +1 +-- Nested partition by function, LowCardinality +1 +2 +1 +1 +-- Nested partition by function, Nullable +1 +2 +1 +1 +-- Nested partition by function, LowCardinality + Nullable +1 +2 +1 +1 +-- Non-safe cast +2 +2 +-- Multiple partition columns +1 +1 +1 +2 +-- LowCardinality set +1 +1 +-- Nullable set +1 +1 +-- LowCardinality + Nullable set +1 +1 +-- Not failing with date parsing functions +1 +0 +-- Pruning + not failing with nested date parsing functions +1 +2 +0 +-- Empty transform functions +2 +1 diff --git a/tests/queries/0_stateless/03173_set_transformed_partition_pruning.sql b/tests/queries/0_stateless/03173_set_transformed_partition_pruning.sql new file mode 100644 index 00000000000..8ffabacaa8c --- /dev/null +++ b/tests/queries/0_stateless/03173_set_transformed_partition_pruning.sql @@ -0,0 +1,258 @@ +SELECT '-- Single partition by function'; + +DROP TABLE IF EXISTS 03173_single_function; +CREATE TABLE 03173_single_function ( + dt Date, +) +ENGINE = MergeTree +ORDER BY tuple() +PARTITION BY toMonth(dt); + +INSERT INTO 03173_single_function +SELECT toDate('2000-01-01') + 10 * number FROM numbers(50) +UNION ALL +SELECT toDate('2100-01-01') + 10 * number FROM numbers(50); +OPTIMIZE TABLE 03173_single_function FINAL; + +SELECT count() FROM 03173_single_function WHERE dt IN ('2024-01-20', '2024-05-25') SETTINGS log_comment='03173_single_function'; +SYSTEM FLUSH LOGS; +SELECT ProfileEvents['SelectedParts'] FROM system.query_log WHERE type = 'QueryFinish' AND current_database = currentDatabase() AND log_comment = '03173_single_function'; + +DROP TABLE IF EXISTS 03173_single_function; + +SELECT '-- Nested partition by function'; + +DROP TABLE IF EXISTS 03173_nested_function; +CREATE TABLE 03173_nested_function( + id Int32, +) +ENGINE = MergeTree +ORDER BY tuple() +PARTITION BY xxHash32(id) % 3; + +INSERT INTO 03173_nested_function SELECT number FROM numbers(100); +OPTIMIZE TABLE 03173_nested_function FINAL; + +SELECT count() FROM 03173_nested_function WHERE id IN (10) SETTINGS log_comment='03173_nested_function'; +SELECT count() FROM 03173_nested_function WHERE xxHash32(id) IN (2158931063, 1449383981) SETTINGS log_comment='03173_nested_function_subexpr'; +SYSTEM FLUSH LOGS; +SELECT ProfileEvents['SelectedParts'] FROM system.query_log WHERE type = 'QueryFinish' AND current_database = currentDatabase() AND log_comment = '03173_nested_function'; +SELECT ProfileEvents['SelectedParts'] FROM system.query_log WHERE type = 'QueryFinish' AND current_database = currentDatabase() AND log_comment = '03173_nested_function_subexpr'; + +DROP TABLE IF EXISTS 03173_nested_function; + +SELECT '-- Nested partition by function, LowCardinality'; + +SET allow_suspicious_low_cardinality_types = 1; + +DROP TABLE IF EXISTS 03173_nested_function_lc; +CREATE TABLE 03173_nested_function_lc( + id LowCardinality(Int32), +) +ENGINE = MergeTree +ORDER BY tuple() +PARTITION BY xxHash32(id) % 3; + +INSERT INTO 03173_nested_function_lc SELECT number FROM numbers(100); +OPTIMIZE TABLE 03173_nested_function_lc FINAL; + +SELECT count() FROM 03173_nested_function_lc WHERE id IN (10) SETTINGS log_comment='03173_nested_function_lc'; +SELECT count() FROM 03173_nested_function_lc WHERE xxHash32(id) IN (2158931063, 1449383981) SETTINGS log_comment='03173_nested_function_subexpr_lc'; +SYSTEM FLUSH LOGS; +SELECT ProfileEvents['SelectedParts'] FROM system.query_log WHERE type = 'QueryFinish' AND current_database = currentDatabase() AND log_comment = '03173_nested_function_lc'; +SELECT ProfileEvents['SelectedParts'] FROM system.query_log WHERE type = 'QueryFinish' AND current_database = currentDatabase() AND log_comment = '03173_nested_function_subexpr_lc'; + +DROP TABLE IF EXISTS 03173_nested_function_lc; + +SELECT '-- Nested partition by function, Nullable'; + +DROP TABLE IF EXISTS 03173_nested_function_null; +CREATE TABLE 03173_nested_function_null( + id Nullable(Int32), +) +ENGINE = MergeTree +ORDER BY tuple() +PARTITION BY xxHash32(id) % 3 +SETTINGS allow_nullable_key=1; + +INSERT INTO 03173_nested_function_null SELECT number FROM numbers(100); +OPTIMIZE TABLE 03173_nested_function_null FINAL; + +SELECT count() FROM 03173_nested_function_null WHERE id IN (10) SETTINGS log_comment='03173_nested_function_null'; +SELECT count() FROM 03173_nested_function_null WHERE xxHash32(id) IN (2158931063, 1449383981) SETTINGS log_comment='03173_nested_function_subexpr_null'; +SYSTEM FLUSH LOGS; +SELECT ProfileEvents['SelectedParts'] FROM system.query_log WHERE type = 'QueryFinish' AND current_database = currentDatabase() AND log_comment = '03173_nested_function_null'; +SELECT ProfileEvents['SelectedParts'] FROM system.query_log WHERE type = 'QueryFinish' AND current_database = currentDatabase() AND log_comment = '03173_nested_function_subexpr_null'; + +DROP TABLE IF EXISTS 03173_nested_function_null; + +SELECT '-- Nested partition by function, LowCardinality + Nullable'; + +DROP TABLE IF EXISTS 03173_nested_function_lc_null; + +SET allow_suspicious_low_cardinality_types = 1; +CREATE TABLE 03173_nested_function_lc_null( + id LowCardinality(Nullable(Int32)), +) +ENGINE = MergeTree +ORDER BY tuple() +PARTITION BY xxHash32(id) % 3 +SETTINGS allow_nullable_key=1; + +INSERT INTO 03173_nested_function_lc_null SELECT number FROM numbers(100); +OPTIMIZE TABLE 03173_nested_function_lc_null FINAL; + +SELECT count() FROM 03173_nested_function_lc_null WHERE id IN (10) SETTINGS log_comment='03173_nested_function_lc_null'; +SELECT count() FROM 03173_nested_function_lc_null WHERE xxHash32(id) IN (2158931063, 1449383981) SETTINGS log_comment='03173_nested_function_subexpr_lc_null'; +SYSTEM FLUSH LOGS; +SELECT ProfileEvents['SelectedParts'] FROM system.query_log WHERE type = 'QueryFinish' AND current_database = currentDatabase() AND log_comment = '03173_nested_function_lc_null'; +SELECT ProfileEvents['SelectedParts'] FROM system.query_log WHERE type = 'QueryFinish' AND current_database = currentDatabase() AND log_comment = '03173_nested_function_subexpr_lc_null'; + +DROP TABLE IF EXISTS 03173_nested_function_lc_null; + +SELECT '-- Non-safe cast'; + +DROP TABLE IF EXISTS 03173_nonsafe_cast; +CREATE TABLE 03173_nonsafe_cast( + id Int64, +) +ENGINE = MergeTree +ORDER BY tuple() +PARTITION BY xxHash32(id) % 3; + +INSERT INTO 03173_nonsafe_cast SELECT number FROM numbers(100); +OPTIMIZE TABLE 03173_nonsafe_cast FINAL; + +SELECT count() FROM 03173_nonsafe_cast WHERE id IN (SELECT '50' UNION ALL SELECT '99') SETTINGS log_comment='03173_nonsafe_cast'; +SYSTEM FLUSH LOGS; +SELECT ProfileEvents['SelectedParts'] FROM system.query_log WHERE type = 'QueryFinish' AND current_database = currentDatabase() AND log_comment = '03173_nonsafe_cast'; + +DROP TABLE IF EXISTS 03173_nonsafe_cast; + +SELECT '-- Multiple partition columns'; + +DROP TABLE IF EXISTS 03173_multiple_partition_cols; +CREATE TABLE 03173_multiple_partition_cols ( + key1 Int32, + key2 Int32 +) +ENGINE = MergeTree +ORDER BY tuple() +PARTITION BY (intDiv(key1, 50), xxHash32(key2) % 3); + +INSERT INTO 03173_multiple_partition_cols SELECT number, number FROM numbers(100); +OPTIMIZE TABLE 03173_multiple_partition_cols FINAL; + +SELECT count() FROM 03173_multiple_partition_cols WHERE key2 IN (4) SETTINGS log_comment='03173_multiple_columns'; +SELECT count() FROM 03173_multiple_partition_cols WHERE xxHash32(key2) IN (4251411170) SETTINGS log_comment='03173_multiple_columns_subexpr'; +SYSTEM FLUSH LOGS; +SELECT ProfileEvents['SelectedParts'] FROM system.query_log WHERE type = 'QueryFinish' AND current_database = currentDatabase() AND log_comment = '03173_multiple_columns'; +-- Due to xxHash32() in WHERE condition, MinMax is unable to eliminate any parts, +-- so partition pruning leave two parts (for key1 // 50 = 0 and key1 // 50 = 1) +SELECT ProfileEvents['SelectedParts'] FROM system.query_log WHERE type = 'QueryFinish' AND current_database = currentDatabase() AND log_comment = '03173_multiple_columns_subexpr'; + +-- Preparing base table for filtering by LowCardinality/Nullable sets +DROP TABLE IF EXISTS 03173_base_data_source; +CREATE TABLE 03173_base_data_source( + id Int32, +) +ENGINE = MergeTree +ORDER BY tuple() +PARTITION BY xxHash32(id) % 3; + +INSERT INTO 03173_base_data_source SELECT number FROM numbers(100); +OPTIMIZE TABLE 03173_base_data_source FINAL; + +SELECT '-- LowCardinality set'; + +SET allow_suspicious_low_cardinality_types = 1; +DROP TABLE IF EXISTS 03173_low_cardinality_set; +CREATE TABLE 03173_low_cardinality_set (id LowCardinality(Int32)) ENGINE=Memory AS SELECT 10; + +SELECT count() FROM 03173_base_data_source WHERE id IN (SELECT id FROM 03173_low_cardinality_set) SETTINGS log_comment='03173_low_cardinality_set'; +SYSTEM FLUSH LOGS; +SELECT ProfileEvents['SelectedParts'] FROM system.query_log WHERE type = 'QueryFinish' AND current_database = currentDatabase() AND log_comment = '03173_low_cardinality_set'; + +DROP TABLE IF EXISTS 03173_low_cardinality_set; + +SELECT '-- Nullable set'; + +DROP TABLE IF EXISTS 03173_nullable_set; +CREATE TABLE 03173_nullable_set (id Nullable(Int32)) ENGINE=Memory AS SELECT 10; + +SELECT count() FROM 03173_base_data_source WHERE id IN (SELECT id FROM 03173_nullable_set) SETTINGS log_comment='03173_nullable_set'; +SYSTEM FLUSH LOGS; +SELECT ProfileEvents['SelectedParts'] FROM system.query_log WHERE type = 'QueryFinish' AND current_database = currentDatabase() AND log_comment = '03173_nullable_set'; + +DROP TABLE IF EXISTS 03173_nullable_set; + +SELECT '-- LowCardinality + Nullable set'; + +DROP TABLE IF EXISTS 03173_lc_nullable_set; +CREATE TABLE 03173_lc_nullable_set (id LowCardinality(Nullable(Int32))) ENGINE=Memory AS SELECT 10 UNION ALL SELECT NULL; + +SELECT count() FROM 03173_base_data_source WHERE id IN (SELECT id FROM 03173_lc_nullable_set) SETTINGS log_comment='03173_lc_nullable_set'; +SYSTEM FLUSH LOGS; +SELECT ProfileEvents['SelectedParts'] FROM system.query_log WHERE type = 'QueryFinish' AND current_database = currentDatabase() AND log_comment = '03173_lc_nullable_set'; + +DROP TABLE IF EXISTS 03173_lc_nullable_set; + +SELECT '-- Not failing with date parsing functions'; + +DROP TABLE IF EXISTS 03173_date_parsing; +CREATE TABLE 03173_date_parsing ( + id String +) +ENGINE=MergeTree +ORDER BY tuple() +PARTITION BY toDate(id); + +INSERT INTO 03173_date_parsing +SELECT toString(toDate('2023-04-01') + number) +FROM numbers(20); + +SELECT count() FROM 03173_date_parsing WHERE id IN ('2023-04-02', '2023-05-02'); +SELECT count() FROM 03173_date_parsing WHERE id IN ('not a date'); + +DROP TABLE IF EXISTS 03173_date_parsing; + +SELECT '-- Pruning + not failing with nested date parsing functions'; + +DROP TABLE IF EXISTS 03173_nested_date_parsing; +CREATE TABLE 03173_nested_date_parsing ( + id String +) +ENGINE=MergeTree +ORDER BY tuple() +PARTITION BY toMonth(toDate(id)); + +INSERT INTO 03173_nested_date_parsing +SELECT toString(toDate('2000-01-01') + 10 * number) FROM numbers(50) +UNION ALL +SELECT toString(toDate('2100-01-01') + 10 * number) FROM numbers(50); + +SELECT count() FROM 03173_nested_date_parsing WHERE id IN ('2000-01-21', '2023-05-02') SETTINGS log_comment='03173_nested_date_parsing'; +SYSTEM FLUSH LOGS; +SELECT ProfileEvents['SelectedParts'] FROM system.query_log WHERE type = 'QueryFinish' AND current_database = currentDatabase() AND log_comment = '03173_nested_date_parsing'; +SELECT count() FROM 03173_nested_date_parsing WHERE id IN ('not a date'); + +DROP TABLE IF EXISTS 03173_nested_date_parsing; + +SELECT '-- Empty transform functions'; + +DROP TABLE IF EXISTS 03173_empty_transform; +CREATE TABLE 03173_empty_transform( + id Int32, +) +ENGINE = MergeTree +ORDER BY tuple() +PARTITION BY xxHash32(id) % 3; + +INSERT INTO 03173_empty_transform SELECT number FROM numbers(6); +OPTIMIZE TABLE 03173_empty_transform FINAL; + +SELECT id FROM 03173_empty_transform WHERE xxHash32(id) % 3 IN (xxHash32(2::Int32) % 3) SETTINGS log_comment='03173_empty_transform'; +SYSTEM FLUSH LOGS; +SELECT ProfileEvents['SelectedParts'] FROM system.query_log WHERE type = 'QueryFinish' AND current_database = currentDatabase() AND log_comment = '03173_empty_transform'; + +DROP TABLE IF EXISTS 03173_empty_transform; From b2306fc67acb992afbe25ef1a0ee9714ed19fbd0 Mon Sep 17 00:00:00 2001 From: Eduard Karacharov Date: Wed, 19 Jun 2024 09:32:29 +0300 Subject: [PATCH 292/417] process sets and scalars by same functions --- src/Storages/MergeTree/KeyCondition.cpp | 549 +++++++++++------------- src/Storages/MergeTree/KeyCondition.h | 6 +- 2 files changed, 249 insertions(+), 306 deletions(-) diff --git a/src/Storages/MergeTree/KeyCondition.cpp b/src/Storages/MergeTree/KeyCondition.cpp index 85eca4644e7..d38001a0feb 100644 --- a/src/Storages/MergeTree/KeyCondition.cpp +++ b/src/Storages/MergeTree/KeyCondition.cpp @@ -874,46 +874,6 @@ static Field applyFunctionForField( return (*col)[0]; } -/// The case when arguments may have types different than in the primary key. -static std::pair applyFunctionForFieldOfUnknownType( - const FunctionBasePtr & func, - const DataTypePtr & arg_type, - const Field & arg_value) -{ - ColumnsWithTypeAndName arguments{{ arg_type->createColumnConst(1, arg_value), arg_type, "x" }}; - DataTypePtr return_type = func->getResultType(); - - auto col = func->execute(arguments, return_type, 1); - - Field result = (*col)[0]; - - return {std::move(result), std::move(return_type)}; -} - - -/// Same as above but for binary operators -static std::pair applyBinaryFunctionForFieldOfUnknownType( - const FunctionOverloadResolverPtr & func, - const DataTypePtr & arg_type, - const Field & arg_value, - const DataTypePtr & arg_type2, - const Field & arg_value2) -{ - ColumnsWithTypeAndName arguments{ - {arg_type->createColumnConst(1, arg_value), arg_type, "x"}, {arg_type2->createColumnConst(1, arg_value2), arg_type2, "y"}}; - - FunctionBasePtr func_base = func->build(arguments); - - DataTypePtr return_type = func_base->getResultType(); - - auto col = func_base->execute(arguments, return_type, 1); - - Field result = (*col)[0]; - - return {std::move(result), std::move(return_type)}; -} - - static FieldRef applyFunction(const FunctionBasePtr & func, const DataTypePtr & current_type, const FieldRef & field) { /// Fallback for fields without block reference. @@ -940,164 +900,92 @@ static FieldRef applyFunction(const FunctionBasePtr & func, const DataTypePtr & return {field.columns, field.row_idx, result_idx}; } -/** When table's key has expression with these functions from a column, - * and when a column in a query is compared with a constant, such as: - * CREATE TABLE (x String) ORDER BY toDate(x) - * SELECT ... WHERE x LIKE 'Hello%' - * we want to apply the function to the constant for index analysis, - * but should modify it to pass on un-parsable values. - */ -static std::set date_time_parsing_functions = { - "toDate", - "toDate32", - "toDateTime", - "toDateTime64", - "parseDateTimeBestEffort", - "parseDateTimeBestEffortUS", - "parseDateTime32BestEffort", - "parseDateTime64BestEffort", - "parseDateTime", - "parseDateTimeInJodaSyntax", -}; - -/** The key functional expression constraint may be inferred from a plain column in the expression. - * For example, if the key contains `toStartOfHour(Timestamp)` and query contains `WHERE Timestamp >= now()`, - * it can be assumed that if `toStartOfHour()` is monotonic on [now(), inf), the `toStartOfHour(Timestamp) >= toStartOfHour(now())` - * condition also holds, so the index may be used to select only parts satisfying this condition. - * - * To check the assumption, we'd need to assert that the inverse function to this transformation is also monotonic, however the - * inversion isn't exported (or even viable for not strictly monotonic functions such as `toStartOfHour()`). - * Instead, we can qualify only functions that do not transform the range (for example rounding), - * which while not strictly monotonic, are monotonic everywhere on the input range. - */ -bool KeyCondition::transformConstantWithValidFunctions( - ContextPtr context, - const String & expr_name, - size_t & out_key_column_num, - DataTypePtr & out_key_column_type, - Field & out_value, - DataTypePtr & out_type, - std::function always_monotonic) const +/// Sequentially applies functions to the column, returns `true` +/// if all function arguments are compatible with functions +/// signatures, and none of the functions produce `NULL` output. +/// +/// After functions chain execution, fills result column and its type. +bool applyFunctionChainToColumn( + const ColumnPtr & in_column, + const DataTypePtr & in_data_type, + const std::vector & functions, + ColumnPtr & out_column, + DataTypePtr & out_data_type) { - const auto & sample_block = key_expr->getSampleBlock(); + // Remove LowCardinality from input column, and convert it to regular one + auto result_column = in_column->convertToFullIfNeeded(); + auto result_type = removeLowCardinality(in_data_type); - for (const auto & node : key_expr->getNodes()) + // In case function sequence is empty, return full non-LowCardinality column + if (functions.empty()) { - auto it = key_columns.find(node.result_name); - if (it != key_columns.end()) - { - std::stack chain; - - const auto * cur_node = &node; - bool is_valid_chain = true; - - while (is_valid_chain) - { - if (cur_node->result_name == expr_name) - break; - - chain.push(cur_node); - - if (cur_node->type == ActionsDAG::ActionType::FUNCTION && cur_node->children.size() <= 2) - { - is_valid_chain = always_monotonic(*cur_node->function_base, *cur_node->result_type); - - const ActionsDAG::Node * next_node = nullptr; - for (const auto * arg : cur_node->children) - { - if (arg->column && isColumnConst(*arg->column)) - continue; - - if (next_node) - is_valid_chain = false; - - next_node = arg; - } - - if (!next_node) - is_valid_chain = false; - - cur_node = next_node; - } - else if (cur_node->type == ActionsDAG::ActionType::ALIAS) - cur_node = cur_node->children.front(); - else - is_valid_chain = false; - } - - if (is_valid_chain) - { - out_type = removeLowCardinality(out_type); - auto const_type = removeLowCardinality(cur_node->result_type); - auto const_column = out_type->createColumnConst(1, out_value); - auto const_value = (*castColumnAccurateOrNull({const_column, out_type, ""}, const_type))[0]; - - if (const_value.isNull()) - return false; - - while (!chain.empty()) - { - const auto * func = chain.top(); - chain.pop(); - - if (func->type != ActionsDAG::ActionType::FUNCTION) - continue; - - const auto & func_name = func->function_base->getName(); - auto func_base = func->function_base; - const auto & arg_types = func_base->getArgumentTypes(); - if (date_time_parsing_functions.contains(func_name) && !arg_types.empty() && isStringOrFixedString(arg_types[0])) - { - auto func_or_null = FunctionFactory::instance().get(func_name + "OrNull", context); - ColumnsWithTypeAndName arguments; - int i = 0; - for (const auto & type : func->function_base->getArgumentTypes()) - arguments.push_back({nullptr, type, fmt::format("_{}", i++)}); - - func_base = func_or_null->build(arguments); - } - - if (func->children.size() == 1) - { - std::tie(const_value, const_type) - = applyFunctionForFieldOfUnknownType(func_base, const_type, const_value); - } - else if (func->children.size() == 2) - { - const auto * left = func->children[0]; - const auto * right = func->children[1]; - if (left->column && isColumnConst(*left->column)) - { - auto left_arg_type = left->result_type; - auto left_arg_value = (*left->column)[0]; - std::tie(const_value, const_type) = applyBinaryFunctionForFieldOfUnknownType( - FunctionFactory::instance().get(func_base->getName(), context), - left_arg_type, left_arg_value, const_type, const_value); - } - else - { - auto right_arg_type = right->result_type; - auto right_arg_value = (*right->column)[0]; - std::tie(const_value, const_type) = applyBinaryFunctionForFieldOfUnknownType( - FunctionFactory::instance().get(func_base->getName(), context), - const_type, const_value, right_arg_type, right_arg_value); - } - } - - if (const_value.isNull()) - return false; - } - - out_key_column_num = it->second; - out_key_column_type = sample_block.getByName(it->first).type; - out_value = const_value; - out_type = const_type; - return true; - } - } + out_column = result_column; + out_data_type = result_type; + return true; } - return false; + // If first function arguments are empty, cannot transform input column + if (functions[0]->getArgumentTypes().empty()) + { + return false; + } + + // And cast it to the argument type of the first function in the chain + auto in_argument_type = functions[0]->getArgumentTypes()[0]; + if (canBeSafelyCasted(result_type, in_argument_type)) + { + result_column = castColumnAccurate({result_column, result_type, ""}, in_argument_type); + result_type = in_argument_type; + } + // If column cannot be casted accurate, casting with OrNull, and in case all + // values has been casted (no nulls), unpacking nested column from nullable. + // In case any further functions require Nullable input, they'll be able + // to cast it. + else + { + result_column = castColumnAccurateOrNull({result_column, result_type, ""}, in_argument_type); + const auto & result_column_nullable = assert_cast(*result_column); + const auto & null_map_data = result_column_nullable.getNullMapData(); + for (char8_t i : null_map_data) + { + if (i != 0) + return false; + } + result_column = result_column_nullable.getNestedColumnPtr(); + result_type = removeNullable(in_argument_type); + } + + for (const auto & func : functions) + { + if (func->getArgumentTypes().empty()) + return false; + + auto argument_type = func->getArgumentTypes()[0]; + if (!canBeSafelyCasted(result_type, argument_type)) + return false; + + result_column = castColumnAccurate({result_column, result_type, ""}, argument_type); + result_column = func->execute({{result_column, argument_type, ""}}, func->getResultType(), result_column->size()); + result_type = func->getResultType(); + + // Transforming nullable columns to the nested ones, in case no nulls found + if (result_column->isNullable()) + { + const auto & result_column_nullable = assert_cast(*result_column); + const auto & null_map_data = result_column_nullable.getNullMapData(); + for (char8_t i : null_map_data) + { + if (i != 0) + return false; + } + result_column = result_column_nullable.getNestedColumnPtr(); + result_type = removeNullable(func->getResultType()); + } + } + out_column = result_column; + out_data_type = result_type; + + return true; } bool KeyCondition::canConstantBeWrappedByMonotonicFunctions( @@ -1118,13 +1006,13 @@ bool KeyCondition::canConstantBeWrappedByMonotonicFunctions( if (out_value.isNull()) return false; - return transformConstantWithValidFunctions( + MonotonicFunctionsChain transform_functions; + auto can_transform_constant = extractMonotonicFunctionsChainFromKey( node.getTreeContext().getQueryContext(), expr_name, out_key_column_num, out_key_column_type, - out_value, - out_type, + transform_functions, [](const IFunctionBase & func, const IDataType & type) { if (!func.hasInformationAboutMonotonicity()) @@ -1138,6 +1026,27 @@ bool KeyCondition::canConstantBeWrappedByMonotonicFunctions( } return true; }); + + if (!can_transform_constant) + return false; + + auto const_column = out_type->createColumnConst(1, out_value); + + ColumnPtr transformed_const_column; + DataTypePtr transformed_const_type; + bool constant_transformed = applyFunctionChainToColumn( + const_column, + out_type, + transform_functions, + transformed_const_column, + transformed_const_type); + + if (!constant_transformed) + return false; + + out_value = (*transformed_const_column)[0]; + out_type = transformed_const_type; + return true; } /// Looking for possible transformation of `column = constant` into `partition_expr = function(constant)` @@ -1173,61 +1082,34 @@ bool KeyCondition::canConstantBeWrappedByFunctions( if (out_value.isNull()) return false; - return transformConstantWithValidFunctions( + MonotonicFunctionsChain transform_functions; + auto can_transform_constant = extractMonotonicFunctionsChainFromKey( node.getTreeContext().getQueryContext(), expr_name, out_key_column_num, out_key_column_type, - out_value, + transform_functions, + [](const IFunctionBase & func, const IDataType &) { return func.isDeterministic(); }); + + if (!can_transform_constant) + return false; + + auto const_column = out_type->createColumnConst(1, out_value); + + ColumnPtr transformed_const_column; + DataTypePtr transformed_const_type; + bool constant_transformed = applyFunctionChainToColumn( + const_column, out_type, - [](const IFunctionBase & func, const IDataType &) - { - return func.isDeterministic(); - }); -} + transform_functions, + transformed_const_column, + transformed_const_type); -/// Sequentially applies functions to set column, returns `true` -/// if all function arguments are compatible with functions -/// signatures, and none of the functions produce `NULL` output. -/// -/// After functions chain execution, fills result set column and -/// its type. -bool applyFunctionChainToSetValues( - const ColumnPtr & set_column, - const DataTypePtr & set_data_type, - const std::vector & functions, - ColumnPtr & out_set_column, - DataTypePtr & out_set_type) -{ - auto result_column = set_column->convertToFullColumnIfLowCardinality(); - auto result_type = removeLowCardinality(set_data_type); - - for (const auto & func : functions) - { - if (func->getArgumentTypes().empty()) - return false; - - auto argument_type = func->getArgumentTypes()[0]; - if (!canBeSafelyCasted(result_type, argument_type)) - return false; - - result_column = castColumnAccurate({result_column, result_type, ""}, argument_type); - result_column = func->execute({{result_column, argument_type, ""}}, func->getResultType(), result_column->size()); - if (result_column->isNullable()) - { - const auto & result_column_nullable = assert_cast(*result_column); - const auto & null_map_data = result_column_nullable.getNullMapData(); - for (char8_t i : null_map_data) - { - if (i != 0) - return false; - } - } - result_type = func->getResultType(); - } - out_set_column = result_column; - out_set_type = result_type; + if (!constant_transformed) + return false; + out_value = (*transformed_const_column)[0]; + out_type = transformed_const_type; return true; } @@ -1260,6 +1142,7 @@ bool KeyCondition::tryPrepareSetIndex( out_key_column_num = std::max(out_key_column_num, index_mapping.key_index); set_transforming_chains.push_back(set_transforming_chain); } + // For partition index, checking if set can be transformed to prune any partitions else if (single_point && canSetValuesBeWrappedByFunctions(node, index_mapping.key_index, data_type, set_transforming_chain)) { indexes_mapping.push_back(index_mapping); @@ -1335,7 +1218,12 @@ bool KeyCondition::tryPrepareSetIndex( { ColumnPtr transformed_set_column; DataTypePtr transformed_set_type; - if (!applyFunctionChainToSetValues(set_column, set_element_type, set_transforming_chains[indexes_mapping_index], transformed_set_column, transformed_set_type)) + if (!applyFunctionChainToColumn( + set_column, + set_element_type, + set_transforming_chains[indexes_mapping_index], + transformed_set_column, + transformed_set_type)) return false; set_column = transformed_set_column; @@ -1639,36 +1527,54 @@ bool KeyCondition::isKeyPossiblyWrappedByMonotonicFunctionsImpl( return false; } -bool KeyCondition::canSetValuesBeWrappedByFunctions( - const RPNBuilderTreeNode & node, +/** When table's key has expression with these functions from a column, + * and when a column in a query is compared with a constant, such as: + * CREATE TABLE (x String) ORDER BY toDate(x) + * SELECT ... WHERE x LIKE 'Hello%' + * we want to apply the function to the constant for index analysis, + * but should modify it to pass on un-parsable values. + */ +static std::set date_time_parsing_functions = { + "toDate", + "toDate32", + "toDateTime", + "toDateTime64", + "parseDateTimeBestEffort", + "parseDateTimeBestEffortUS", + "parseDateTime32BestEffort", + "parseDateTime64BestEffort", + "parseDateTime", + "parseDateTimeInJodaSyntax", +}; + +/** The key functional expression constraint may be inferred from a plain column in the expression. + * For example, if the key contains `toStartOfHour(Timestamp)` and query contains `WHERE Timestamp >= now()`, + * it can be assumed that if `toStartOfHour()` is monotonic on [now(), inf), the `toStartOfHour(Timestamp) >= toStartOfHour(now())` + * condition also holds, so the index may be used to select only parts satisfying this condition. + * + * To check the assumption, we'd need to assert that the inverse function to this transformation is also monotonic, however the + * inversion isn't exported (or even viable for not strictly monotonic functions such as `toStartOfHour()`). + * Instead, we can qualify only functions that do not transform the range (for example rounding), + * which while not strictly monotonic, are monotonic everywhere on the input range. + */ +bool KeyCondition::extractMonotonicFunctionsChainFromKey( + ContextPtr context, + const String & expr_name, size_t & out_key_column_num, - DataTypePtr & out_key_res_column_type, - MonotonicFunctionsChain & out_functions_chain) + DataTypePtr & out_key_column_type, + MonotonicFunctionsChain & out_functions_chain, + std::function always_monotonic) const { - // Checking if column name matches any of key subexpressions - String expr_name = node.getColumnName(); + const auto & sample_block = key_expr->getSampleBlock(); - if (array_joined_column_names.contains(expr_name)) - return false; - - if (!key_subexpr_names.contains(expr_name)) + for (const auto & node : key_expr->getNodes()) { - expr_name = node.getColumnNameWithModuloLegacy(); - - if (!key_subexpr_names.contains(expr_name)) - return false; - } - - // If match has been found, need to identify key column, and - // sequence of function nodes, which forms key column - for (const auto & key_expr_node : key_expr->getNodes()) - { - auto it = key_columns.find(key_expr_node.result_name); + auto it = key_columns.find(node.result_name); if (it != key_columns.end()) { - std::vector chain; + std::stack chain; - const auto * cur_node = &key_expr_node; + const auto * cur_node = &node; bool is_valid_chain = true; while (is_valid_chain) @@ -1676,10 +1582,11 @@ bool KeyCondition::canSetValuesBeWrappedByFunctions( if (cur_node->result_name == expr_name) break; + chain.push(cur_node); + if (cur_node->type == ActionsDAG::ActionType::FUNCTION && cur_node->children.size() <= 2) { - chain.push_back(cur_node); - is_valid_chain = cur_node->function_base->isDeterministic(); + is_valid_chain = always_monotonic(*cur_node->function_base, *cur_node->result_type); const ActionsDAG::Node * next_node = nullptr; for (const auto * arg : cur_node->children) @@ -1704,64 +1611,68 @@ bool KeyCondition::canSetValuesBeWrappedByFunctions( is_valid_chain = false; } - // If key column has been found, and it consists of deterministic function, - // in order to make them applicable to constants from query predicate, - // reversing the sequence of functions, and binding constant arguments if (is_valid_chain) { - // Removing low cardinality from `node` output type to avoid - // passing it through functions signatures - auto nonconst_input_type = removeLowCardinality(node.getDAGNode()->result_type); - - for (auto iter = chain.rbegin(); iter != chain.rend(); ++iter) + while (!chain.empty()) { - const auto * function_node = *iter; - auto function = function_node->function_base; - auto func_builder = FunctionFactory::instance().tryGet(function->getName(), node.getTreeContext().getQueryContext()); - if (!func_builder) - return false; + const auto * func = chain.top(); + chain.pop(); + + if (func->type != ActionsDAG::ActionType::FUNCTION) + continue; + + auto func_name = func->function_base->getName(); + auto func_base = func->function_base; + ColumnsWithTypeAndName arguments; ColumnWithTypeAndName const_arg; FunctionWithOptionalConstArg::Kind kind = FunctionWithOptionalConstArg::Kind::NO_CONST; - if (function_node->children.size() == 2) + + if (date_time_parsing_functions.contains(func_name)) { - const auto * left = function_node->children[0]; - const auto * right = function_node->children[1]; + const auto & arg_types = func_base->getArgumentTypes(); + if (!arg_types.empty() && isStringOrFixedString(arg_types[0])) + { + func_name = func_name + "OrNull"; + } + + } + + auto func_builder = FunctionFactory::instance().tryGet(func_name, context); + + if (func->children.size() == 1) + { + arguments.push_back({nullptr, removeLowCardinality(func->children[0]->result_type), ""}); + } + else if (func->children.size() == 2) + { + const auto * left = func->children[0]; + const auto * right = func->children[1]; if (left->column && isColumnConst(*left->column)) { const_arg = {left->result_type->createColumnConst(0, (*left->column)[0]), left->result_type, ""}; arguments.push_back(const_arg); - arguments.push_back({nullptr, nonconst_input_type, ""}); + arguments.push_back({nullptr, removeLowCardinality(right->result_type), ""}); kind = FunctionWithOptionalConstArg::Kind::LEFT_CONST; } - else if (right->column && isColumnConst(*right->column)) + else { const_arg = {right->result_type->createColumnConst(0, (*right->column)[0]), right->result_type, ""}; - arguments.push_back({nullptr, nonconst_input_type, ""}); + arguments.push_back({nullptr, removeLowCardinality(left->result_type), ""}); arguments.push_back(const_arg); kind = FunctionWithOptionalConstArg::Kind::RIGHT_CONST; } - - /// If constant arg of binary operator is NULL, there will be no monotonicity. - if (const_arg.column->isNullAt(0)) - return false; } - else - arguments.push_back({nullptr, nonconst_input_type, ""}); - auto func = func_builder->build(arguments); + auto out_func = func_builder->build(arguments); if (kind == FunctionWithOptionalConstArg::Kind::NO_CONST) - out_functions_chain.push_back(func); + out_functions_chain.push_back(out_func); else - out_functions_chain.push_back(std::make_shared(func, const_arg, kind)); - - nonconst_input_type = func->getResultType(); + out_functions_chain.push_back(std::make_shared(out_func, const_arg, kind)); } - const auto & sample_block = key_expr->getSampleBlock(); out_key_column_num = it->second; - out_key_res_column_type = sample_block.getByName(it->first).type; - + out_key_column_type = sample_block.getByName(it->first).type; return true; } } @@ -1770,6 +1681,38 @@ bool KeyCondition::canSetValuesBeWrappedByFunctions( return false; } +bool KeyCondition::canSetValuesBeWrappedByFunctions( + const RPNBuilderTreeNode & node, + size_t & out_key_column_num, + DataTypePtr & out_key_res_column_type, + MonotonicFunctionsChain & out_functions_chain) +{ + // Checking if column name matches any of key subexpressions + String expr_name = node.getColumnName(); + + if (array_joined_column_names.contains(expr_name)) + return false; + + if (!key_subexpr_names.contains(expr_name)) + { + expr_name = node.getColumnNameWithModuloLegacy(); + + if (!key_subexpr_names.contains(expr_name)) + return false; + } + + return extractMonotonicFunctionsChainFromKey( + node.getTreeContext().getQueryContext(), + expr_name, + out_key_column_num, + out_key_res_column_type, + out_functions_chain, + [](const IFunctionBase & func, const IDataType &) + { + return func.isDeterministic(); + }); +} + static void castValueToType(const DataTypePtr & desired_type, Field & src_value, const DataTypePtr & src_type, const String & node_column_name) { try diff --git a/src/Storages/MergeTree/KeyCondition.h b/src/Storages/MergeTree/KeyCondition.h index 11dd1c85c9d..9e2218d7a29 100644 --- a/src/Storages/MergeTree/KeyCondition.h +++ b/src/Storages/MergeTree/KeyCondition.h @@ -14,6 +14,7 @@ #include #include +#include "DataTypes/Serializations/ISerialization.h" namespace DB @@ -253,13 +254,12 @@ private: DataTypePtr & out_key_column_type, std::vector & out_functions_chain); - bool transformConstantWithValidFunctions( + bool extractMonotonicFunctionsChainFromKey( ContextPtr context, const String & expr_name, size_t & out_key_column_num, DataTypePtr & out_key_column_type, - Field & out_value, - DataTypePtr & out_type, + MonotonicFunctionsChain & out_functions_chain, std::function always_monotonic) const; bool canConstantBeWrappedByMonotonicFunctions( From 9b1003527dc8aba15729a63a86428390470bff07 Mon Sep 17 00:00:00 2001 From: kssenii Date: Mon, 8 Jul 2024 11:34:39 +0200 Subject: [PATCH 293/417] Fix the order --- .../DataLakes/DeltaLakeMetadata.cpp | 75 ++++++++++--------- 1 file changed, 38 insertions(+), 37 deletions(-) diff --git a/src/Storages/ObjectStorage/DataLakes/DeltaLakeMetadata.cpp b/src/Storages/ObjectStorage/DataLakes/DeltaLakeMetadata.cpp index bc64ef15cf1..12341c877e2 100644 --- a/src/Storages/ObjectStorage/DataLakes/DeltaLakeMetadata.cpp +++ b/src/Storages/ObjectStorage/DataLakes/DeltaLakeMetadata.cpp @@ -209,43 +209,6 @@ struct DeltaLakeMetadataImpl // object->stringify(oss); // LOG_TEST(log, "Metadata: {}", oss.str()); - if (object->has("add")) - { - auto add_object = object->get("add").extract(); - auto path = add_object->getValue("path"); - result.insert(fs::path(configuration->getPath()) / path); - - auto filename = fs::path(path).filename().string(); - auto it = file_partition_columns.find(filename); - if (it == file_partition_columns.end()) - { - if (add_object->has("partitionValues")) - { - auto partition_values = add_object->get("partitionValues").extract(); - if (partition_values->size()) - { - auto & current_partition_columns = file_partition_columns[filename]; - for (const auto & partition_name : partition_values->getNames()) - { - const auto value = partition_values->getValue(partition_name); - auto name_and_type = file_schema.tryGetByName(partition_name); - if (!name_and_type) - throw Exception(ErrorCodes::LOGICAL_ERROR, "No such column in schema: {}", partition_name); - - auto field = getFieldValue(value, name_and_type->type); - current_partition_columns.emplace_back(*name_and_type, field); - - LOG_TEST(log, "Partition {} value is {} (for {})", partition_name, value, filename); - } - } - } - } - } - else if (object->has("remove")) - { - auto path = object->get("remove").extract()->getValue("path"); - result.erase(fs::path(configuration->getPath()) / path); - } if (object->has("metaData")) { const auto metadata_object = object->get("metaData").extract(); @@ -289,6 +252,44 @@ struct DeltaLakeMetadataImpl file_schema.toString(), current_schema.toString()); } } + + if (object->has("add")) + { + auto add_object = object->get("add").extract(); + auto path = add_object->getValue("path"); + result.insert(fs::path(configuration->getPath()) / path); + + auto filename = fs::path(path).filename().string(); + auto it = file_partition_columns.find(filename); + if (it == file_partition_columns.end()) + { + if (add_object->has("partitionValues")) + { + auto partition_values = add_object->get("partitionValues").extract(); + if (partition_values->size()) + { + auto & current_partition_columns = file_partition_columns[filename]; + for (const auto & partition_name : partition_values->getNames()) + { + const auto value = partition_values->getValue(partition_name); + auto name_and_type = file_schema.tryGetByName(partition_name); + if (!name_and_type) + throw Exception(ErrorCodes::LOGICAL_ERROR, "No such column in schema: {}", partition_name); + + auto field = getFieldValue(value, name_and_type->type); + current_partition_columns.emplace_back(*name_and_type, field); + + LOG_TEST(log, "Partition {} value is {} (for {})", partition_name, value, filename); + } + } + } + } + } + else if (object->has("remove")) + { + auto path = object->get("remove").extract()->getValue("path"); + result.erase(fs::path(configuration->getPath()) / path); + } } } From db53c2c5f79a3b65d10dec37fcad78e1192c8c96 Mon Sep 17 00:00:00 2001 From: Sema Checherinda Date: Mon, 8 Jul 2024 12:39:58 +0200 Subject: [PATCH 294/417] remove iterator from queue after removing table --- src/Interpreters/DatabaseCatalog.cpp | 62 +++++++++++++--------------- src/Interpreters/DatabaseCatalog.h | 4 +- 2 files changed, 31 insertions(+), 35 deletions(-) diff --git a/src/Interpreters/DatabaseCatalog.cpp b/src/Interpreters/DatabaseCatalog.cpp index 61ad5bf96de..4131e214278 100644 --- a/src/Interpreters/DatabaseCatalog.cpp +++ b/src/Interpreters/DatabaseCatalog.cpp @@ -1243,15 +1243,13 @@ time_t DatabaseCatalog::getMinDropTime() return min_drop_time; } -DatabaseCatalog::TablesMarkedAsDropped DatabaseCatalog::getTablesToDrop() +std::vector DatabaseCatalog::getTablesToDrop() { time_t current_time = std::chrono::system_clock::to_time_t(std::chrono::system_clock::now()); - DatabaseCatalog::TablesMarkedAsDropped result; + decltype(getTablesToDrop()) result; std::lock_guard lock(tables_marked_dropped_mutex); - const auto was_count = tables_marked_dropped.size(); - auto it = tables_marked_dropped.begin(); while (it != tables_marked_dropped.end()) { @@ -1263,15 +1261,9 @@ DatabaseCatalog::TablesMarkedAsDropped DatabaseCatalog::getTablesToDrop() continue; } - if (it == first_async_drop_in_queue) - ++first_async_drop_in_queue; - - result.emplace_back(std::move(*it)); - it = tables_marked_dropped.erase(it); + result.emplace_back(it); } - chassert(was_count == tables_marked_dropped.size() + result.size()); - return result; } @@ -1294,19 +1286,11 @@ void DatabaseCatalog::rescheduleDropTableTask() (*drop_task)->scheduleAfter(schedule_after); } -void DatabaseCatalog::dropTablesParallel(TablesMarkedAsDropped tables_to_drop) +void DatabaseCatalog::dropTablesParallel(std::vector tables_to_drop) { if (tables_to_drop.empty()) return; - SCOPE_EXIT({ - std::lock_guard lock(tables_marked_dropped_mutex); - tables_marked_dropped.splice(tables_marked_dropped.end(), tables_to_drop); - - if (first_async_drop_in_queue == tables_marked_dropped.end()) - first_async_drop_in_queue = tables_marked_dropped.begin(); - }); - ThreadPool pool( CurrentMetrics::DatabaseCatalogThreads, CurrentMetrics::DatabaseCatalogThreadsActive, @@ -1315,29 +1299,41 @@ void DatabaseCatalog::dropTablesParallel(TablesMarkedAsDropped tables_to_drop) /* max_free_threads */0, /* queue_size */tables_to_drop.size()); - while (!tables_to_drop.empty()) + for (const auto & item : tables_to_drop) { - auto front_table = std::move(tables_to_drop.front()); - tables_to_drop.pop_front(); - - pool.scheduleOrThrowOnError([&, table = front_table] () + pool.scheduleOrThrowOnError([&, table_iterator = item] () { try { - dropTableFinally(table); - std::lock_guard lock(tables_marked_dropped_mutex); - [[maybe_unused]] auto removed = tables_marked_dropped_ids.erase(table.table_id.uuid); - chassert(removed); - wait_table_finally_dropped.notify_all(); + dropTableFinally(*table_iterator); + + { + std::lock_guard lock(tables_marked_dropped_mutex); + + if (first_async_drop_in_queue == table_iterator) + ++first_async_drop_in_queue; + + [[maybe_unused]] auto removed = tables_marked_dropped_ids.erase(table_iterator->table_id.uuid); + chassert(removed); + + tables_marked_dropped.erase(table_iterator); + + wait_table_finally_dropped.notify_all(); + } } catch (...) { - tryLogCurrentException(log, "Cannot drop table " + table.table_id.getNameForLogs() + + tryLogCurrentException(log, "Cannot drop table " + table_iterator->table_id.getNameForLogs() + ". Will retry later."); { std::lock_guard lock(tables_marked_dropped_mutex); - tables_marked_dropped.emplace_back(table); - tables_marked_dropped.back().drop_time = std::chrono::system_clock::to_time_t(std::chrono::system_clock::now()) + drop_error_cooldown_sec; + + if (first_async_drop_in_queue == table_iterator) + ++first_async_drop_in_queue; + + tables_marked_dropped.splice(tables_marked_dropped.end(), tables_marked_dropped, table_iterator); + table_iterator->drop_time = std::chrono::system_clock::to_time_t(std::chrono::system_clock::now()) + drop_error_cooldown_sec; + if (first_async_drop_in_queue == tables_marked_dropped.end()) --first_async_drop_in_queue; } diff --git a/src/Interpreters/DatabaseCatalog.h b/src/Interpreters/DatabaseCatalog.h index f9b8233e85a..23e38a6445e 100644 --- a/src/Interpreters/DatabaseCatalog.h +++ b/src/Interpreters/DatabaseCatalog.h @@ -298,8 +298,8 @@ private: time_t getMinDropTime() TSA_REQUIRES(tables_marked_dropped_mutex); std::tuple getDroppedTablesCountAndInuseCount(); - TablesMarkedAsDropped getTablesToDrop(); - void dropTablesParallel(TablesMarkedAsDropped tables); + std::vector getTablesToDrop(); + void dropTablesParallel(std::vector tables); void rescheduleDropTableTask(); void cleanupStoreDirectoryTask(); From 4227447eac30dd77c6ba70d4b1685bbf11a8221f Mon Sep 17 00:00:00 2001 From: Blargian Date: Mon, 8 Jul 2024 12:53:55 +0200 Subject: [PATCH 295/417] add nth_value and update ordering --- .../sql-reference/window-functions/index.md | 6 +- .../window-functions/lagInFrame.md | 2 +- .../window-functions/leadInFrame.md | 2 +- .../window-functions/nth_value.md | 77 +++++++++++++++++++ .../en/sql-reference/window-functions/rank.md | 2 +- .../window-functions/row_number.md | 2 +- 6 files changed, 84 insertions(+), 7 deletions(-) create mode 100644 docs/en/sql-reference/window-functions/nth_value.md diff --git a/docs/en/sql-reference/window-functions/index.md b/docs/en/sql-reference/window-functions/index.md index a0246af610f..ee54a679ba1 100644 --- a/docs/en/sql-reference/window-functions/index.md +++ b/docs/en/sql-reference/window-functions/index.md @@ -78,11 +78,11 @@ These functions can be used only as a window function. - [`row_number()`](./row_number.md) - Number the current row within its partition starting from 1. - `first_value(x)` - Return the first non-NULL value evaluated within its ordered frame. - `last_value(x)` - Return the last non-NULL value evaluated within its ordered frame. -- `nth_value(x, offset)` - Return the first non-NULL value evaluated against the nth row (offset) in its ordered frame. +- [`nth_value(x, offset)`](./nth_value.md) - Return the first non-NULL value evaluated against the nth row (offset) in its ordered frame. - [`rank()`](./rank.md) - Rank the current row within its partition with gaps. - [`dense_rank()`](./dense_rank.md) - Rank the current row within its partition without gaps. -- `lagInFrame(x)` - Return a value evaluated at the row that is at a specified physical offset row before the current row within the ordered frame. -- `leadInFrame(x)` - Return a value evaluated at the row that is offset rows after the current row within the ordered frame. +- [`lagInFrame(x)`](./lagInFrame.md) - Return a value evaluated at the row that is at a specified physical offset row before the current row within the ordered frame. +- [`leadInFrame(x)`](./leadInFrame.md) - Return a value evaluated at the row that is offset rows after the current row within the ordered frame. ## Examples diff --git a/docs/en/sql-reference/window-functions/lagInFrame.md b/docs/en/sql-reference/window-functions/lagInFrame.md index ea9f6d9dea2..b67cf252283 100644 --- a/docs/en/sql-reference/window-functions/lagInFrame.md +++ b/docs/en/sql-reference/window-functions/lagInFrame.md @@ -1,7 +1,7 @@ --- slug: /en/sql-reference/window-functions/lagInFrame sidebar_label: lagInFrame -sidebar_position: 5 +sidebar_position: 3 --- # lagInFrame diff --git a/docs/en/sql-reference/window-functions/leadInFrame.md b/docs/en/sql-reference/window-functions/leadInFrame.md index e3b65af9a4d..0cb4eea52b2 100644 --- a/docs/en/sql-reference/window-functions/leadInFrame.md +++ b/docs/en/sql-reference/window-functions/leadInFrame.md @@ -1,7 +1,7 @@ --- slug: /en/sql-reference/window-functions/leadInFrame sidebar_label: leadInFrame -sidebar_position: 6 +sidebar_position: 4 --- # leadInFrame diff --git a/docs/en/sql-reference/window-functions/nth_value.md b/docs/en/sql-reference/window-functions/nth_value.md new file mode 100644 index 00000000000..26c90110aaa --- /dev/null +++ b/docs/en/sql-reference/window-functions/nth_value.md @@ -0,0 +1,77 @@ +--- +slug: /en/sql-reference/window-functions/leadInFrame +sidebar_label: leadInFrame +sidebar_position: 5 +--- + +# nth_value + +Return the first non-NULL value evaluated against the nth row (offset) in its ordered frame. + +The [dense_rank](./dense_rank.md) function provides the same behaviour but without gaps in ranking. + +**Syntax** + +```sql +nth_value (x, offset) + OVER ([[PARTITION BY grouping_column] [ORDER BY sorting_column] + [ROWS or RANGE expression_to_bound_rows_withing_the_group]] | [window_name]) +FROM table_name +WINDOW window_name as ([[PARTITION BY grouping_column] [ORDER BY sorting_column]) +``` + +For more detail on window function syntax see: [Window Functions - Syntax](./index.md/#syntax). + +**Parameters** + +- `x` — Column name. +- `offset` — nth row to evaluate current row against. + +**Returned value** + +- The first non-NULL value evaluated against the nth row (offset) in its ordered frame. + +**Example** + +In this example the `nth-value` function is used to find the third-highest salary from a fictional dataset of salaries of Premier League football players. + +Query: + +```sql +DROP TABLE IF EXISTS salaries; +CREATE TABLE salaries +( + `team` String, + `player` String, + `salary` UInt32, + `position` String +) +Engine = Memory; + +INSERT INTO salaries FORMAT Values + ('Port Elizabeth Barbarians', 'Gary Chen', 195000, 'F'), + ('New Coreystad Archdukes', 'Charles Juarez', 190000, 'F'), + ('Port Elizabeth Barbarians', 'Michael Stanley', 10000, 'D'), + ('New Coreystad Archdukes', 'Scott Harrison', 180000, 'D'), + ('Port Elizabeth Barbarians', 'Robert George', 195000, 'M'), + ('South Hampton Seagulls', 'Douglas Benson', 150000, 'M'), + ('South Hampton Seagulls', 'James Henderson', 140000, 'M'); +``` + +```sql +SELECT salary, nth_value(salary,3) OVER(ORDER BY salary DESC) FROM salaries GROUP BY salary; +``` + +Result: + +```response + ┌─player──────────┬─salary─┬─rank─┐ +1. │ Gary Chen │ 195000 │ 1 │ +2. │ Robert George │ 195000 │ 1 │ +3. │ Charles Juarez │ 190000 │ 3 │ +4. │ Douglas Benson │ 150000 │ 4 │ +5. │ Michael Stanley │ 150000 │ 4 │ +6. │ Scott Harrison │ 150000 │ 4 │ +7. │ James Henderson │ 140000 │ 7 │ + └─────────────────┴────────┴──────┘ +``` \ No newline at end of file diff --git a/docs/en/sql-reference/window-functions/rank.md b/docs/en/sql-reference/window-functions/rank.md index 17db889ef92..9ac99dde6df 100644 --- a/docs/en/sql-reference/window-functions/rank.md +++ b/docs/en/sql-reference/window-functions/rank.md @@ -1,7 +1,7 @@ --- slug: /en/sql-reference/window-functions/rank sidebar_label: rank -sidebar_position: 3 +sidebar_position: 6 --- # rank diff --git a/docs/en/sql-reference/window-functions/row_number.md b/docs/en/sql-reference/window-functions/row_number.md index 428bb34a8ba..e7165d60169 100644 --- a/docs/en/sql-reference/window-functions/row_number.md +++ b/docs/en/sql-reference/window-functions/row_number.md @@ -1,7 +1,7 @@ --- slug: /en/sql-reference/window-functions/row_number sidebar_label: row_number -sidebar_position: 4 +sidebar_position: 7 --- # row_number From 8f604c892ade829ebff91961faf550ec45f6c1a9 Mon Sep 17 00:00:00 2001 From: Blargian Date: Mon, 8 Jul 2024 12:58:29 +0200 Subject: [PATCH 296/417] Fix typo --- docs/en/sql-reference/functions/string-functions.md | 2 +- utils/check-style/aspell-ignore/en/aspell-dict.txt | 4 ++-- 2 files changed, 3 insertions(+), 3 deletions(-) diff --git a/docs/en/sql-reference/functions/string-functions.md b/docs/en/sql-reference/functions/string-functions.md index 6734630cd10..8259d780577 100644 --- a/docs/en/sql-reference/functions/string-functions.md +++ b/docs/en/sql-reference/functions/string-functions.md @@ -2386,7 +2386,7 @@ Result: ## initcapUTF8 -Like [initcap](#initcap), сonvert the first letter of each word to upper case and the rest to lower case. Assumes that the string contains valid UTF-8 encoded text. +Like [initcap](#initcap), `initcapUTF8` converts the first letter of each word to upper case and the rest to lower case. Assumes that the string contains valid UTF-8 encoded text. If this assumption is violated, no exception is thrown and the result is undefined. :::note diff --git a/utils/check-style/aspell-ignore/en/aspell-dict.txt b/utils/check-style/aspell-ignore/en/aspell-dict.txt index fa2bfef935a..4d615340357 100644 --- a/utils/check-style/aspell-ignore/en/aspell-dict.txt +++ b/utils/check-style/aspell-ignore/en/aspell-dict.txt @@ -1,4 +1,4 @@ -personal_ws-1.1 en 2758 +personal_ws-1.1 en 2942 AArch ACLs ALTERs @@ -1656,9 +1656,9 @@ fsync func fuzzBits fuzzJSON +fuzzQuery fuzzer fuzzers -fuzzQuery gRPC gccMurmurHash gcem From 5ac5a4b2fe91ee5efa37a602f109bf6534e12de6 Mon Sep 17 00:00:00 2001 From: Vitaly Baranov Date: Fri, 5 Jul 2024 22:15:32 +0200 Subject: [PATCH 297/417] Add check that user name is not empty to authenticateUserByHTTP(). --- src/Server/HTTP/authenticateUserByHTTP.cpp | 16 ++++++++++++++++ 1 file changed, 16 insertions(+) diff --git a/src/Server/HTTP/authenticateUserByHTTP.cpp b/src/Server/HTTP/authenticateUserByHTTP.cpp index 8929c912601..ac43bfd64c0 100644 --- a/src/Server/HTTP/authenticateUserByHTTP.cpp +++ b/src/Server/HTTP/authenticateUserByHTTP.cpp @@ -37,6 +37,13 @@ namespace throw Exception(ErrorCodes::AUTHENTICATION_FAILED, "Invalid authentication: it is not allowed to use {} and {} simultaneously", method1, method2); } + + /// Checks that a specified user name is not empty, and throws an exception if it's empty. + void checkUserNameNotEmpty(const String & user_name, std::string_view method) + { + if (user_name.empty()) + throw Exception(ErrorCodes::AUTHENTICATION_FAILED, "Got an empty user name from {}", method); + } } @@ -74,6 +81,9 @@ bool authenticateUserByHTTP( if (has_ssl_certificate_auth) { #if USE_SSL + /// For SSL certificate authentication we extract the user name from the "X-ClickHouse-User" HTTP header. + checkUserNameNotEmpty(user, "X-ClickHouse HTTP headers"); + /// It is prohibited to mix different authorization schemes. if (!password.empty()) throwMultipleAuthenticationMethods("SSL certificate authentication", "authentication via password"); @@ -95,6 +105,8 @@ bool authenticateUserByHTTP( } else if (has_auth_headers) { + checkUserNameNotEmpty(user, "X-ClickHouse HTTP headers"); + /// It is prohibited to mix different authorization schemes. if (has_http_credentials) throwMultipleAuthenticationMethods("X-ClickHouse HTTP headers", "Authorization HTTP header"); @@ -116,6 +128,7 @@ bool authenticateUserByHTTP( Poco::Net::HTTPBasicCredentials credentials(auth_info); user = credentials.getUsername(); password = credentials.getPassword(); + checkUserNameNotEmpty(user, "Authorization HTTP header"); } else if (Poco::icompare(scheme, "Negotiate") == 0) { @@ -134,10 +147,12 @@ bool authenticateUserByHTTP( /// If the user name is not set we assume it's the 'default' user. user = params.get("user", "default"); password = params.get("password", ""); + checkUserNameNotEmpty(user, "authentication via parameters"); } if (!certificate_subjects.empty()) { + chassert(!user.empty()); if (!current_credentials) current_credentials = std::make_unique(user, std::move(certificate_subjects)); @@ -183,6 +198,7 @@ bool authenticateUserByHTTP( if (!basic_credentials) throw Exception(ErrorCodes::AUTHENTICATION_FAILED, "Invalid authentication: expected 'Basic' HTTP Authorization scheme"); + chassert(!user.empty()); basic_credentials->setUserName(user); basic_credentials->setPassword(password); } From 21e7ef6e42770bbd3d544dab1e8a78e980a0de0e Mon Sep 17 00:00:00 2001 From: Tyler Hannan Date: Mon, 8 Jul 2024 13:44:41 +0200 Subject: [PATCH 298/417] Update README.md --- README.md | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/README.md b/README.md index 3d7d7441081..98f9108f14c 100644 --- a/README.md +++ b/README.md @@ -34,7 +34,7 @@ curl https://clickhouse.com/ | sh Every month we get together with the community (users, contributors, customers, those interested in learning more about ClickHouse) to discuss what is coming in the latest release. If you are interested in sharing what you've built on ClickHouse, let us know. -* [v24.6 Community Call](https://clickhouse.com/company/events/v24-6-community-release-call) - Jul 2 +* [v24.7 Community Call](https://clickhouse.com/company/events/v24-7-community-release-call) - Jul 30 ## Upcoming Events From 5be51a4e6269155184ac465a760f055f6b4eb3f7 Mon Sep 17 00:00:00 2001 From: Shaun Struwig <41984034+Blargian@users.noreply.github.com> Date: Mon, 8 Jul 2024 14:29:16 +0200 Subject: [PATCH 299/417] Update aspell-dict.txt --- utils/check-style/aspell-ignore/en/aspell-dict.txt | 2 ++ 1 file changed, 2 insertions(+) diff --git a/utils/check-style/aspell-ignore/en/aspell-dict.txt b/utils/check-style/aspell-ignore/en/aspell-dict.txt index 229eccefa48..d71bd599c8b 100644 --- a/utils/check-style/aspell-ignore/en/aspell-dict.txt +++ b/utils/check-style/aspell-ignore/en/aspell-dict.txt @@ -2120,8 +2120,10 @@ noaa nonNegativeDerivative noop normalizeQuery +normalizeQueryKeepNames normalizeUTF normalizedQueryHash +normalizedQueryHashKeepNames notEmpty notEquals notILike From 486ac7f287f0486fedb60f0339b81b2688af6760 Mon Sep 17 00:00:00 2001 From: Denny Crane Date: Mon, 8 Jul 2024 09:30:25 -0300 Subject: [PATCH 300/417] Update settings.md --- docs/en/operations/settings/settings.md | 4 ++++ 1 file changed, 4 insertions(+) diff --git a/docs/en/operations/settings/settings.md b/docs/en/operations/settings/settings.md index 1d74a63b972..ed0b29aa851 100644 --- a/docs/en/operations/settings/settings.md +++ b/docs/en/operations/settings/settings.md @@ -1170,6 +1170,10 @@ Data in the VALUES clause of INSERT queries is processed by a separate stream pa Default value: 262144 (= 256 KiB). +:::note +`max_query_size` cannot be set within an SQL query (e.g., `SELECT now() SETTINGS max_query_size=10000`) because ClickHouse needs to allocate a buffer to parse the query, and this buffer size is determined by the `max_query_size` setting, which must be configured before the query is executed. +::: + ## max_parser_depth {#max_parser_depth} Limits maximum recursion depth in the recursive descent parser. Allows controlling the stack size. From 312dd824254a8518b35c9a3bed75f2887edb769e Mon Sep 17 00:00:00 2001 From: Robert Schulze Date: Mon, 8 Jul 2024 11:35:00 +0000 Subject: [PATCH 301/417] Bump rocksdb to v6.24.2 --- contrib/rocksdb | 2 +- contrib/rocksdb-cmake/CMakeLists.txt | 112 ++++++------------ ...ksdb_build_version.cc => build_version.cc} | 0 3 files changed, 34 insertions(+), 80 deletions(-) rename contrib/rocksdb-cmake/{rocksdb_build_version.cc => build_version.cc} (100%) diff --git a/contrib/rocksdb b/contrib/rocksdb index 078fa563869..2aed45919b9 160000 --- a/contrib/rocksdb +++ b/contrib/rocksdb @@ -1 +1 @@ -Subproject commit 078fa5638690004e1f744076d1bdcc4e93767304 +Subproject commit 2aed45919b9fee4208221e01f368483fef11be61 diff --git a/contrib/rocksdb-cmake/CMakeLists.txt b/contrib/rocksdb-cmake/CMakeLists.txt index 943e1d8acbd..5502d3b6205 100644 --- a/contrib/rocksdb-cmake/CMakeLists.txt +++ b/contrib/rocksdb-cmake/CMakeLists.txt @@ -5,20 +5,13 @@ if (NOT ENABLE_ROCKSDB) return() endif() -## this file is extracted from `contrib/rocksdb/CMakeLists.txt` -set(ROCKSDB_SOURCE_DIR "${ClickHouse_SOURCE_DIR}/contrib/rocksdb") -list(APPEND CMAKE_MODULE_PATH "${ROCKSDB_SOURCE_DIR}/cmake/modules/") - -set(PORTABLE ON) -## always disable jemalloc for rocksdb by default -## because it introduces non-standard jemalloc APIs +# Always disable jemalloc for rocksdb by default because it introduces non-standard jemalloc APIs option(WITH_JEMALLOC "build with JeMalloc" OFF) -set(USE_SNAPPY OFF) -if (TARGET ch_contrib::snappy) - set(USE_SNAPPY ON) -endif() -option(WITH_SNAPPY "build with SNAPPY" ${USE_SNAPPY}) -## lz4, zlib, zstd is enabled in ClickHouse by default + +option(WITH_LIBURING "build with liburing" OFF) # TODO could try to enable this conditionally, depending on ClickHouse's ENABLE_LIBURING + +# ClickHouse cannot be compiled without snappy, lz4, zlib, zstd +option(WITH_SNAPPY "build with SNAPPY" ON) option(WITH_LZ4 "build with lz4" ON) option(WITH_ZLIB "build with zlib" ON) option(WITH_ZSTD "build with zstd" ON) @@ -26,74 +19,34 @@ option(WITH_ZSTD "build with zstd" ON) # third-party/folly is only validated to work on Linux and Windows for now. # So only turn it on there by default. if(CMAKE_SYSTEM_NAME MATCHES "Linux|Windows") - if(MSVC AND MSVC_VERSION LESS 1910) - # Folly does not compile with MSVC older than VS2017 - option(WITH_FOLLY_DISTRIBUTED_MUTEX "build with folly::DistributedMutex" OFF) - else() - option(WITH_FOLLY_DISTRIBUTED_MUTEX "build with folly::DistributedMutex" ON) - endif() + option(WITH_FOLLY_DISTRIBUTED_MUTEX "build with folly::DistributedMutex" ON) else() option(WITH_FOLLY_DISTRIBUTED_MUTEX "build with folly::DistributedMutex" OFF) endif() -if( NOT DEFINED CMAKE_CXX_STANDARD ) - set(CMAKE_CXX_STANDARD 11) +if(WITH_SNAPPY) + add_definitions(-DSNAPPY) + list(APPEND THIRDPARTY_LIBS ch_contrib::snappy) endif() -if(MSVC) - option(WITH_XPRESS "build with windows built in compression" OFF) - include("${ROCKSDB_SOURCE_DIR}/thirdparty.inc") -else() - if(CMAKE_SYSTEM_NAME MATCHES "FreeBSD" AND NOT CMAKE_SYSTEM_NAME MATCHES "kFreeBSD") - # FreeBSD has jemalloc as default malloc - # but it does not have all the jemalloc files in include/... - set(WITH_JEMALLOC ON) - else() - if(WITH_JEMALLOC AND TARGET ch_contrib::jemalloc) - add_definitions(-DROCKSDB_JEMALLOC -DJEMALLOC_NO_DEMANGLE) - list(APPEND THIRDPARTY_LIBS ch_contrib::jemalloc) - endif() - endif() - - if(WITH_SNAPPY) - add_definitions(-DSNAPPY) - list(APPEND THIRDPARTY_LIBS ch_contrib::snappy) - endif() - - if(WITH_ZLIB) - add_definitions(-DZLIB) - list(APPEND THIRDPARTY_LIBS ch_contrib::zlib) - endif() - - if(WITH_LZ4) - add_definitions(-DLZ4) - list(APPEND THIRDPARTY_LIBS ch_contrib::lz4) - endif() - - if(WITH_ZSTD) - add_definitions(-DZSTD) - list(APPEND THIRDPARTY_LIBS ch_contrib::zstd) - endif() +if(WITH_ZLIB) + add_definitions(-DZLIB) + list(APPEND THIRDPARTY_LIBS ch_contrib::zlib) endif() -if(CMAKE_SYSTEM_PROCESSOR MATCHES "^(powerpc|ppc)64") - if(POWER9) - set(HAS_POWER9 1) - set(HAS_ALTIVEC 1) - else() - set(HAS_POWER8 1) - set(HAS_ALTIVEC 1) - endif(POWER9) -endif(CMAKE_SYSTEM_PROCESSOR MATCHES "^(powerpc|ppc)64") +if(WITH_LZ4) + add_definitions(-DLZ4) + list(APPEND THIRDPARTY_LIBS ch_contrib::lz4) +endif() -if(CMAKE_SYSTEM_PROCESSOR MATCHES "aarch64|AARCH64|arm64|ARM64") - set(HAS_ARMV8_CRC 1) - set(CMAKE_C_FLAGS "${CMAKE_C_FLAGS} -march=armv8-a+crc+crypto -Wno-unused-function") - set(CMAKE_CXX_FLAGS "${CMAKE_CXX_FLAGS} -march=armv8-a+crc+crypto -Wno-unused-function") -endif(CMAKE_SYSTEM_PROCESSOR MATCHES "aarch64|AARCH64|arm64|ARM64") +if(WITH_ZSTD) + add_definitions(-DZSTD) + list(APPEND THIRDPARTY_LIBS ch_contrib::zstd) +endif() +option(PORTABLE "build a portable binary" ON) -if(ENABLE_AVX2 AND ENABLE_PCLMULQDQ) +if(ENABLE_SSE42 AND ENABLE_PCLMULQDQ) add_definitions(-DHAVE_SSE42) add_definitions(-DHAVE_PCLMUL) endif() @@ -107,8 +60,6 @@ if(CMAKE_SYSTEM_NAME MATCHES "Darwin") add_definitions(-DOS_MACOSX) elseif(CMAKE_SYSTEM_NAME MATCHES "Linux") add_definitions(-DOS_LINUX) -elseif(CMAKE_SYSTEM_NAME MATCHES "SunOS") - add_definitions(-DOS_SOLARIS) elseif(CMAKE_SYSTEM_NAME MATCHES "FreeBSD") add_definitions(-DOS_FREEBSD) elseif(CMAKE_SYSTEM_NAME MATCHES "Android") @@ -123,12 +74,10 @@ endif() if (OS_LINUX) add_definitions(-DROCKSDB_SCHED_GETCPU_PRESENT) - add_definitions(-DROCKSDB_AUXV_SYSAUXV_PRESENT) add_definitions(-DROCKSDB_AUXV_GETAUXVAL_PRESENT) -elseif (OS_FREEBSD) - add_definitions(-DROCKSDB_AUXV_SYSAUXV_PRESENT) endif() +set(ROCKSDB_SOURCE_DIR "${ClickHouse_SOURCE_DIR}/contrib/rocksdb") include_directories(${ROCKSDB_SOURCE_DIR}) include_directories("${ROCKSDB_SOURCE_DIR}/include") @@ -136,8 +85,6 @@ if(WITH_FOLLY_DISTRIBUTED_MUTEX) include_directories("${ROCKSDB_SOURCE_DIR}/third-party/folly") endif() -# Main library source code - set(SOURCES ${ROCKSDB_SOURCE_DIR}/cache/cache.cc ${ROCKSDB_SOURCE_DIR}/cache/cache_entry_roles.cc @@ -333,9 +280,12 @@ set(SOURCES ${ROCKSDB_SOURCE_DIR}/tools/ldb_tool.cc ${ROCKSDB_SOURCE_DIR}/tools/sst_dump_tool.cc ${ROCKSDB_SOURCE_DIR}/tools/trace_analyzer_tool.cc - ${ROCKSDB_SOURCE_DIR}/trace_replay/trace_replay.cc ${ROCKSDB_SOURCE_DIR}/trace_replay/block_cache_tracer.cc ${ROCKSDB_SOURCE_DIR}/trace_replay/io_tracer.cc + ${ROCKSDB_SOURCE_DIR}/trace_replay/trace_record_handler.cc + ${ROCKSDB_SOURCE_DIR}/trace_replay/trace_record_result.cc + ${ROCKSDB_SOURCE_DIR}/trace_replay/trace_record.cc + ${ROCKSDB_SOURCE_DIR}/trace_replay/trace_replay.cc ${ROCKSDB_SOURCE_DIR}/util/coding.cc ${ROCKSDB_SOURCE_DIR}/util/compaction_job_stats_impl.cc ${ROCKSDB_SOURCE_DIR}/util/comparator.cc @@ -366,6 +316,7 @@ set(SOURCES ${ROCKSDB_SOURCE_DIR}/utilities/cassandra/format.cc ${ROCKSDB_SOURCE_DIR}/utilities/cassandra/merge_operator.cc ${ROCKSDB_SOURCE_DIR}/utilities/checkpoint/checkpoint_impl.cc + ${ROCKSDB_SOURCE_DIR}/utilities/compaction_filters.cc ${ROCKSDB_SOURCE_DIR}/utilities/compaction_filters/remove_emptyvalue_compactionfilter.cc ${ROCKSDB_SOURCE_DIR}/utilities/debug.cc ${ROCKSDB_SOURCE_DIR}/utilities/env_mirror.cc @@ -374,6 +325,7 @@ set(SOURCES ${ROCKSDB_SOURCE_DIR}/utilities/fault_injection_fs.cc ${ROCKSDB_SOURCE_DIR}/utilities/leveldb_options/leveldb_options.cc ${ROCKSDB_SOURCE_DIR}/utilities/memory/memory_util.cc + ${ROCKSDB_SOURCE_DIR}/utilities/merge_operators.cc ${ROCKSDB_SOURCE_DIR}/utilities/merge_operators/bytesxor.cc ${ROCKSDB_SOURCE_DIR}/utilities/merge_operators/max.cc ${ROCKSDB_SOURCE_DIR}/utilities/merge_operators/put.cc @@ -393,6 +345,7 @@ set(SOURCES ${ROCKSDB_SOURCE_DIR}/utilities/simulator_cache/sim_cache.cc ${ROCKSDB_SOURCE_DIR}/utilities/table_properties_collectors/compact_on_deletion_collector.cc ${ROCKSDB_SOURCE_DIR}/utilities/trace/file_trace_reader_writer.cc + ${ROCKSDB_SOURCE_DIR}/utilities/trace/replayer_impl.cc ${ROCKSDB_SOURCE_DIR}/utilities/transactions/lock/lock_manager.cc ${ROCKSDB_SOURCE_DIR}/utilities/transactions/lock/point/point_lock_tracker.cc ${ROCKSDB_SOURCE_DIR}/utilities/transactions/lock/point/point_lock_manager.cc @@ -425,7 +378,7 @@ set(SOURCES ${ROCKSDB_SOURCE_DIR}/utilities/transactions/lock/range/range_tree/lib/standalone_port.cc ${ROCKSDB_SOURCE_DIR}/utilities/transactions/lock/range/range_tree/lib/util/dbt.cc ${ROCKSDB_SOURCE_DIR}/utilities/transactions/lock/range/range_tree/lib/util/memarena.cc - rocksdb_build_version.cc) + build_version.cc) # generated by hand if(ENABLE_SSE42 AND ENABLE_PCLMULQDQ) set_source_files_properties( @@ -462,5 +415,6 @@ endif() add_library(_rocksdb ${SOURCES}) add_library(ch_contrib::rocksdb ALIAS _rocksdb) target_link_libraries(_rocksdb PRIVATE ${THIRDPARTY_LIBS} ${SYSTEM_LIBS}) + # SYSTEM is required to overcome some issues target_include_directories(_rocksdb SYSTEM BEFORE INTERFACE "${ROCKSDB_SOURCE_DIR}/include") diff --git a/contrib/rocksdb-cmake/rocksdb_build_version.cc b/contrib/rocksdb-cmake/build_version.cc similarity index 100% rename from contrib/rocksdb-cmake/rocksdb_build_version.cc rename to contrib/rocksdb-cmake/build_version.cc From da5bde7f3a8957e61c4a113dd20cee5c60fa5c99 Mon Sep 17 00:00:00 2001 From: Sema Checherinda Date: Mon, 8 Jul 2024 15:00:23 +0200 Subject: [PATCH 302/417] fix peak_threads_usage --- src/Interpreters/ThreadStatusExt.cpp | 6 ++++-- 1 file changed, 4 insertions(+), 2 deletions(-) diff --git a/src/Interpreters/ThreadStatusExt.cpp b/src/Interpreters/ThreadStatusExt.cpp index 9ca521a4ab3..6ec6a64b13d 100644 --- a/src/Interpreters/ThreadStatusExt.cpp +++ b/src/Interpreters/ThreadStatusExt.cpp @@ -233,7 +233,8 @@ void ThreadStatus::attachToGroupImpl(const ThreadGroupPtr & thread_group_) { /// Attach or init current thread to thread group and copy useful information from it thread_group = thread_group_; - thread_group->linkThread(thread_id); + if (!internal_thread) + thread_group->linkThread(thread_id); performance_counters.setParent(&thread_group->performance_counters); memory_tracker.setParent(&thread_group->memory_tracker); @@ -269,7 +270,8 @@ void ThreadStatus::detachFromGroup() /// Extract MemoryTracker out from query and user context memory_tracker.setParent(&total_memory_tracker); - thread_group->unlinkThread(); + if (!internal_thread) + thread_group->unlinkThread(); thread_group.reset(); From c5eebf3b9eec37b9c3117071e03b262cf28e612f Mon Sep 17 00:00:00 2001 From: Sema Checherinda Date: Mon, 8 Jul 2024 15:23:52 +0200 Subject: [PATCH 303/417] fix getTablesToDrop --- src/Interpreters/DatabaseCatalog.cpp | 12 +++--------- 1 file changed, 3 insertions(+), 9 deletions(-) diff --git a/src/Interpreters/DatabaseCatalog.cpp b/src/Interpreters/DatabaseCatalog.cpp index 9df6fd93020..afe89e7b9a7 100644 --- a/src/Interpreters/DatabaseCatalog.cpp +++ b/src/Interpreters/DatabaseCatalog.cpp @@ -1252,18 +1252,12 @@ std::vector DatabaseCatalog::g std::lock_guard lock(tables_marked_dropped_mutex); - auto it = tables_marked_dropped.begin(); - while (it != tables_marked_dropped.end()) + for (auto it = tables_marked_dropped.begin(); it != tables_marked_dropped.end(); ++it) { bool in_use = it->table && !it->table.unique(); bool old_enough = it->drop_time <= current_time; - if (in_use || !old_enough) - { - ++it; - continue; - } - - result.emplace_back(it); + if (!in_use && old_enough) + result.emplace_back(it); } return result; From adcee80b2d637d9d79a3ecfe4501828339efe050 Mon Sep 17 00:00:00 2001 From: Robert Schulze Date: Mon, 8 Jul 2024 13:03:19 +0000 Subject: [PATCH 304/417] Bump rocksdb to v6.25.3 --- contrib/rocksdb | 2 +- contrib/rocksdb-cmake/CMakeLists.txt | 3 +++ 2 files changed, 4 insertions(+), 1 deletion(-) diff --git a/contrib/rocksdb b/contrib/rocksdb index 2aed45919b9..6df587a7eb3 160000 --- a/contrib/rocksdb +++ b/contrib/rocksdb @@ -1 +1 @@ -Subproject commit 2aed45919b9fee4208221e01f368483fef11be61 +Subproject commit 6df587a7eb3e7bb835a71c2f4a668a51cabefd67 diff --git a/contrib/rocksdb-cmake/CMakeLists.txt b/contrib/rocksdb-cmake/CMakeLists.txt index 5502d3b6205..48c97257d94 100644 --- a/contrib/rocksdb-cmake/CMakeLists.txt +++ b/contrib/rocksdb-cmake/CMakeLists.txt @@ -88,6 +88,7 @@ endif() set(SOURCES ${ROCKSDB_SOURCE_DIR}/cache/cache.cc ${ROCKSDB_SOURCE_DIR}/cache/cache_entry_roles.cc + ${ROCKSDB_SOURCE_DIR}/cache/cache_reservation_manager.cc ${ROCKSDB_SOURCE_DIR}/cache/clock_cache.cc ${ROCKSDB_SOURCE_DIR}/cache/lru_cache.cc ${ROCKSDB_SOURCE_DIR}/cache/sharded_cache.cc @@ -176,6 +177,7 @@ set(SOURCES ${ROCKSDB_SOURCE_DIR}/env/file_system_tracer.cc ${ROCKSDB_SOURCE_DIR}/env/fs_remap.cc ${ROCKSDB_SOURCE_DIR}/env/mock_env.cc + ${ROCKSDB_SOURCE_DIR}/env/unique_id.cc ${ROCKSDB_SOURCE_DIR}/file/delete_scheduler.cc ${ROCKSDB_SOURCE_DIR}/file/file_prefetch_buffer.cc ${ROCKSDB_SOURCE_DIR}/file/file_util.cc @@ -297,6 +299,7 @@ set(SOURCES ${ROCKSDB_SOURCE_DIR}/util/murmurhash.cc ${ROCKSDB_SOURCE_DIR}/util/random.cc ${ROCKSDB_SOURCE_DIR}/util/rate_limiter.cc + ${ROCKSDB_SOURCE_DIR}/util/regex.cc ${ROCKSDB_SOURCE_DIR}/util/ribbon_config.cc ${ROCKSDB_SOURCE_DIR}/util/slice.cc ${ROCKSDB_SOURCE_DIR}/util/file_checksum_helper.cc From 31f9bed44285eeaae964f98e5a4ce3150d57ac5f Mon Sep 17 00:00:00 2001 From: Robert Schulze Date: Mon, 8 Jul 2024 13:43:28 +0000 Subject: [PATCH 305/417] Bump rocksdb to 6.26.1 --- contrib/rocksdb | 2 +- contrib/rocksdb-cmake/CMakeLists.txt | 6 +++++- 2 files changed, 6 insertions(+), 2 deletions(-) diff --git a/contrib/rocksdb b/contrib/rocksdb index 6df587a7eb3..19ab8db7a73 160000 --- a/contrib/rocksdb +++ b/contrib/rocksdb @@ -1 +1 @@ -Subproject commit 6df587a7eb3e7bb835a71c2f4a668a51cabefd67 +Subproject commit 19ab8db7a736306d6d12992a21e545e0336ab34a diff --git a/contrib/rocksdb-cmake/CMakeLists.txt b/contrib/rocksdb-cmake/CMakeLists.txt index 48c97257d94..525b301f31f 100644 --- a/contrib/rocksdb-cmake/CMakeLists.txt +++ b/contrib/rocksdb-cmake/CMakeLists.txt @@ -177,7 +177,7 @@ set(SOURCES ${ROCKSDB_SOURCE_DIR}/env/file_system_tracer.cc ${ROCKSDB_SOURCE_DIR}/env/fs_remap.cc ${ROCKSDB_SOURCE_DIR}/env/mock_env.cc - ${ROCKSDB_SOURCE_DIR}/env/unique_id.cc + ${ROCKSDB_SOURCE_DIR}/env/unique_id_gen.cc ${ROCKSDB_SOURCE_DIR}/file/delete_scheduler.cc ${ROCKSDB_SOURCE_DIR}/file/file_prefetch_buffer.cc ${ROCKSDB_SOURCE_DIR}/file/file_util.cc @@ -271,6 +271,7 @@ set(SOURCES ${ROCKSDB_SOURCE_DIR}/table/table_factory.cc ${ROCKSDB_SOURCE_DIR}/table/table_properties.cc ${ROCKSDB_SOURCE_DIR}/table/two_level_iterator.cc + ${ROCKSDB_SOURCE_DIR}/table/unique_id.cc ${ROCKSDB_SOURCE_DIR}/test_util/sync_point.cc ${ROCKSDB_SOURCE_DIR}/test_util/sync_point_impl.cc ${ROCKSDB_SOURCE_DIR}/test_util/testutil.cc @@ -315,6 +316,8 @@ set(SOURCES ${ROCKSDB_SOURCE_DIR}/utilities/blob_db/blob_db_impl_filesnapshot.cc ${ROCKSDB_SOURCE_DIR}/utilities/blob_db/blob_dump_tool.cc ${ROCKSDB_SOURCE_DIR}/utilities/blob_db/blob_file.cc + ${ROCKSDB_SOURCE_DIR}/utilities/cache_dump_load.cc + ${ROCKSDB_SOURCE_DIR}/utilities/cache_dump_load_impl.cc ${ROCKSDB_SOURCE_DIR}/utilities/cassandra/cassandra_compaction_filter.cc ${ROCKSDB_SOURCE_DIR}/utilities/cassandra/format.cc ${ROCKSDB_SOURCE_DIR}/utilities/cassandra/merge_operator.cc @@ -367,6 +370,7 @@ set(SOURCES ${ROCKSDB_SOURCE_DIR}/utilities/transactions/write_unprepared_txn.cc ${ROCKSDB_SOURCE_DIR}/utilities/transactions/write_unprepared_txn_db.cc ${ROCKSDB_SOURCE_DIR}/utilities/ttl/db_ttl_impl.cc + ${ROCKSDB_SOURCE_DIR}/utilities/wal_filter.cc ${ROCKSDB_SOURCE_DIR}/utilities/write_batch_with_index/write_batch_with_index.cc ${ROCKSDB_SOURCE_DIR}/utilities/write_batch_with_index/write_batch_with_index_internal.cc ${ROCKSDB_SOURCE_DIR}/utilities/transactions/lock/range/range_tree/lib/locktree/concurrent_tree.cc From 9509802866206df50b7802ab74a556e9fd979852 Mon Sep 17 00:00:00 2001 From: Robert Schulze Date: Mon, 8 Jul 2024 13:53:24 +0000 Subject: [PATCH 306/417] Bump rocksdb to v6.27.3 --- contrib/rocksdb | 2 +- contrib/rocksdb-cmake/CMakeLists.txt | 2 ++ 2 files changed, 3 insertions(+), 1 deletion(-) diff --git a/contrib/rocksdb b/contrib/rocksdb index 19ab8db7a73..630bc2d1c3b 160000 --- a/contrib/rocksdb +++ b/contrib/rocksdb @@ -1 +1 @@ -Subproject commit 19ab8db7a736306d6d12992a21e545e0336ab34a +Subproject commit 630bc2d1c3bcf654ebada4d7a092996de8cfb779 diff --git a/contrib/rocksdb-cmake/CMakeLists.txt b/contrib/rocksdb-cmake/CMakeLists.txt index 525b301f31f..d6e2a1afd50 100644 --- a/contrib/rocksdb-cmake/CMakeLists.txt +++ b/contrib/rocksdb-cmake/CMakeLists.txt @@ -104,6 +104,7 @@ set(SOURCES ${ROCKSDB_SOURCE_DIR}/db/blob/blob_log_format.cc ${ROCKSDB_SOURCE_DIR}/db/blob/blob_log_sequential_reader.cc ${ROCKSDB_SOURCE_DIR}/db/blob/blob_log_writer.cc + ${ROCKSDB_SOURCE_DIR}/db/blob/prefetch_buffer_collection.cc ${ROCKSDB_SOURCE_DIR}/db/builder.cc ${ROCKSDB_SOURCE_DIR}/db/c.cc ${ROCKSDB_SOURCE_DIR}/db/column_family.cc @@ -329,6 +330,7 @@ set(SOURCES ${ROCKSDB_SOURCE_DIR}/utilities/env_timed.cc ${ROCKSDB_SOURCE_DIR}/utilities/fault_injection_env.cc ${ROCKSDB_SOURCE_DIR}/utilities/fault_injection_fs.cc + ${ROCKSDB_SOURCE_DIR}/utilities/fault_injection_secondary_cache.cc ${ROCKSDB_SOURCE_DIR}/utilities/leveldb_options/leveldb_options.cc ${ROCKSDB_SOURCE_DIR}/utilities/memory/memory_util.cc ${ROCKSDB_SOURCE_DIR}/utilities/merge_operators.cc From 1bfafa42be11ad338a76746f695ffa1710d198d1 Mon Sep 17 00:00:00 2001 From: Robert Schulze Date: Mon, 8 Jul 2024 14:38:51 +0000 Subject: [PATCH 307/417] Bump rocksdb to v6.28.2 --- contrib/rocksdb | 2 +- contrib/rocksdb-cmake/CMakeLists.txt | 2 ++ 2 files changed, 3 insertions(+), 1 deletion(-) diff --git a/contrib/rocksdb b/contrib/rocksdb index 630bc2d1c3b..b8a996ce196 160000 --- a/contrib/rocksdb +++ b/contrib/rocksdb @@ -1 +1 @@ -Subproject commit 630bc2d1c3bcf654ebada4d7a092996de8cfb779 +Subproject commit b8a996ce1969a3f7141aca7fb5c54196a58a654a diff --git a/contrib/rocksdb-cmake/CMakeLists.txt b/contrib/rocksdb-cmake/CMakeLists.txt index d6e2a1afd50..96558b40174 100644 --- a/contrib/rocksdb-cmake/CMakeLists.txt +++ b/contrib/rocksdb-cmake/CMakeLists.txt @@ -88,6 +88,7 @@ endif() set(SOURCES ${ROCKSDB_SOURCE_DIR}/cache/cache.cc ${ROCKSDB_SOURCE_DIR}/cache/cache_entry_roles.cc + ${ROCKSDB_SOURCE_DIR}/cache/cache_key.cc ${ROCKSDB_SOURCE_DIR}/cache/cache_reservation_manager.cc ${ROCKSDB_SOURCE_DIR}/cache/clock_cache.cc ${ROCKSDB_SOURCE_DIR}/cache/lru_cache.cc @@ -197,6 +198,7 @@ set(SOURCES ${ROCKSDB_SOURCE_DIR}/memory/concurrent_arena.cc ${ROCKSDB_SOURCE_DIR}/memory/jemalloc_nodump_allocator.cc ${ROCKSDB_SOURCE_DIR}/memory/memkind_kmem_allocator.cc + ${ROCKSDB_SOURCE_DIR}/memory/memory_allocator.cc ${ROCKSDB_SOURCE_DIR}/memtable/alloc_tracker.cc ${ROCKSDB_SOURCE_DIR}/memtable/hash_linklist_rep.cc ${ROCKSDB_SOURCE_DIR}/memtable/hash_skiplist_rep.cc From 93d0f7a4850de40080f18b49b15f26c8e618ed55 Mon Sep 17 00:00:00 2001 From: Nikita Taranov Date: Tue, 25 Jun 2024 15:03:36 +0100 Subject: [PATCH 308/417] refactor --- src/Columns/ColumnAggregateFunction.cpp | 4 +-- src/Columns/ColumnAggregateFunction.h | 8 +++--- src/Columns/ColumnArray.cpp | 6 ++--- src/Columns/ColumnArray.h | 6 ++--- src/Columns/ColumnCompressed.h | 4 +-- src/Columns/ColumnConst.h | 8 +++--- src/Columns/ColumnDecimal.cpp | 4 +-- src/Columns/ColumnDecimal.h | 8 +++--- src/Columns/ColumnDynamic.cpp | 8 +++--- src/Columns/ColumnDynamic.h | 8 +++--- src/Columns/ColumnFixedString.cpp | 6 ++--- src/Columns/ColumnFixedString.h | 8 +++--- src/Columns/ColumnFunction.cpp | 4 +-- src/Columns/ColumnFunction.h | 6 ++--- src/Columns/ColumnLowCardinality.cpp | 6 ++--- src/Columns/ColumnLowCardinality.h | 6 ++--- src/Columns/ColumnMap.cpp | 8 +++--- src/Columns/ColumnMap.h | 8 +++--- src/Columns/ColumnNullable.cpp | 8 +++--- src/Columns/ColumnNullable.h | 8 +++--- src/Columns/ColumnObject.cpp | 4 +-- src/Columns/ColumnObject.h | 6 ++--- src/Columns/ColumnSparse.cpp | 6 ++--- src/Columns/ColumnSparse.h | 6 ++--- src/Columns/ColumnString.cpp | 4 +-- src/Columns/ColumnString.h | 8 +++--- src/Columns/ColumnTuple.cpp | 8 +++--- src/Columns/ColumnTuple.h | 8 +++--- src/Columns/ColumnUnique.h | 4 +-- src/Columns/ColumnVariant.cpp | 8 +++--- src/Columns/ColumnVariant.h | 11 +++++--- src/Columns/ColumnVector.cpp | 2 +- src/Columns/ColumnVector.h | 8 +++--- src/Columns/IColumn.cpp | 2 +- src/Columns/IColumn.h | 27 +++++++++++++------ src/Columns/IColumnDummy.h | 6 ++--- src/Columns/IColumnUnique.h | 2 +- .../benchmark_column_insert_many_from.cpp | 2 +- 38 files changed, 135 insertions(+), 119 deletions(-) diff --git a/src/Columns/ColumnAggregateFunction.cpp b/src/Columns/ColumnAggregateFunction.cpp index f7e6b1a1ccc..be00f086ccf 100644 --- a/src/Columns/ColumnAggregateFunction.cpp +++ b/src/Columns/ColumnAggregateFunction.cpp @@ -267,7 +267,7 @@ bool ColumnAggregateFunction::structureEquals(const IColumn & to) const } -void ColumnAggregateFunction::insertRangeFrom(const IColumn & from, size_t start, size_t length) +void ColumnAggregateFunction::doInsertRangeFrom(const IColumn & from, size_t start, size_t length) { const ColumnAggregateFunction & from_concrete = assert_cast(from); @@ -462,7 +462,7 @@ void ColumnAggregateFunction::insertFromWithOwnership(const IColumn & from, size insertMergeFrom(from, n); } -void ColumnAggregateFunction::insertFrom(const IColumn & from, size_t n) +void ColumnAggregateFunction::doInsertFrom(const IColumn & from, size_t n) { insertRangeFrom(from, n, 1); } diff --git a/src/Columns/ColumnAggregateFunction.h b/src/Columns/ColumnAggregateFunction.h index a75b27e835c..a34def52d08 100644 --- a/src/Columns/ColumnAggregateFunction.h +++ b/src/Columns/ColumnAggregateFunction.h @@ -145,7 +145,9 @@ public: void insertData(const char * pos, size_t length) override; - void insertFrom(const IColumn & from, size_t n) override; + void doInsertFrom(const IColumn & from, size_t n) override; + + using IColumn::insertFrom; void insertFrom(ConstAggregateDataPtr place); @@ -182,7 +184,7 @@ public: void protect() override; - void insertRangeFrom(const IColumn & from, size_t start, size_t length) override; + void doInsertRangeFrom(const IColumn & from, size_t start, size_t length) override; void popBack(size_t n) override; @@ -201,7 +203,7 @@ public: MutableColumns scatter(ColumnIndex num_columns, const Selector & selector) const override; - int compareAt(size_t, size_t, const IColumn &, int) const override + int doCompareAt(size_t, size_t, const IColumn &, int) const override { return 0; } diff --git a/src/Columns/ColumnArray.cpp b/src/Columns/ColumnArray.cpp index 0b7e6541560..756fce07f8e 100644 --- a/src/Columns/ColumnArray.cpp +++ b/src/Columns/ColumnArray.cpp @@ -337,7 +337,7 @@ bool ColumnArray::tryInsert(const Field & x) return true; } -void ColumnArray::insertFrom(const IColumn & src_, size_t n) +void ColumnArray::doInsertFrom(const IColumn & src_, size_t n) { const ColumnArray & src = assert_cast(src_); size_t size = src.sizeAt(n); @@ -392,7 +392,7 @@ int ColumnArray::compareAtImpl(size_t n, size_t m, const IColumn & rhs_, int nan : 1); } -int ColumnArray::compareAt(size_t n, size_t m, const IColumn & rhs_, int nan_direction_hint) const +int ColumnArray::doCompareAt(size_t n, size_t m, const IColumn & rhs_, int nan_direction_hint) const { return compareAtImpl(n, m, rhs_, nan_direction_hint); } @@ -535,7 +535,7 @@ void ColumnArray::getExtremes(Field & min, Field & max) const } -void ColumnArray::insertRangeFrom(const IColumn & src, size_t start, size_t length) +void ColumnArray::doInsertRangeFrom(const IColumn & src, size_t start, size_t length) { if (length == 0) return; diff --git a/src/Columns/ColumnArray.h b/src/Columns/ColumnArray.h index 53eb5166df8..b609c64598c 100644 --- a/src/Columns/ColumnArray.h +++ b/src/Columns/ColumnArray.h @@ -84,10 +84,10 @@ public: void updateHashWithValue(size_t n, SipHash & hash) const override; void updateWeakHash32(WeakHash32 & hash) const override; void updateHashFast(SipHash & hash) const override; - void insertRangeFrom(const IColumn & src, size_t start, size_t length) override; + void doInsertRangeFrom(const IColumn & src, size_t start, size_t length) override; void insert(const Field & x) override; bool tryInsert(const Field & x) override; - void insertFrom(const IColumn & src_, size_t n) override; + void doInsertFrom(const IColumn & src_, size_t n) override; void insertDefault() override; void popBack(size_t n) override; ColumnPtr filter(const Filter & filt, ssize_t result_size_hint) const override; @@ -95,7 +95,7 @@ public: ColumnPtr permute(const Permutation & perm, size_t limit) const override; ColumnPtr index(const IColumn & indexes, size_t limit) const override; template ColumnPtr indexImpl(const PaddedPODArray & indexes, size_t limit) const; - int compareAt(size_t n, size_t m, const IColumn & rhs_, int nan_direction_hint) const override; + int doCompareAt(size_t n, size_t m, const IColumn & rhs_, int nan_direction_hint) const override; int compareAtWithCollation(size_t n, size_t m, const IColumn & rhs_, int nan_direction_hint, const Collator & collator) const override; void getPermutation(PermutationSortDirection direction, PermutationSortStability stability, size_t limit, int nan_direction_hint, Permutation & res) const override; diff --git a/src/Columns/ColumnCompressed.h b/src/Columns/ColumnCompressed.h index 934adf07cf4..f6e56b15d30 100644 --- a/src/Columns/ColumnCompressed.h +++ b/src/Columns/ColumnCompressed.h @@ -85,7 +85,7 @@ public: bool isDefaultAt(size_t) const override { throwMustBeDecompressed(); } void insert(const Field &) override { throwMustBeDecompressed(); } bool tryInsert(const Field &) override { throwMustBeDecompressed(); } - void insertRangeFrom(const IColumn &, size_t, size_t) override { throwMustBeDecompressed(); } + void doInsertRangeFrom(const IColumn &, size_t, size_t) override { throwMustBeDecompressed(); } void insertData(const char *, size_t) override { throwMustBeDecompressed(); } void insertDefault() override { throwMustBeDecompressed(); } void popBack(size_t) override { throwMustBeDecompressed(); } @@ -100,7 +100,7 @@ public: void expand(const Filter &, bool) override { throwMustBeDecompressed(); } ColumnPtr permute(const Permutation &, size_t) const override { throwMustBeDecompressed(); } ColumnPtr index(const IColumn &, size_t) const override { throwMustBeDecompressed(); } - int compareAt(size_t, size_t, const IColumn &, int) const override { throwMustBeDecompressed(); } + int doCompareAt(size_t, size_t, const IColumn &, int) const override { throwMustBeDecompressed(); } void compareColumn(const IColumn &, size_t, PaddedPODArray *, PaddedPODArray &, int, int) const override { throwMustBeDecompressed(); diff --git a/src/Columns/ColumnConst.h b/src/Columns/ColumnConst.h index c2c0fa3027c..e782d06a6ca 100644 --- a/src/Columns/ColumnConst.h +++ b/src/Columns/ColumnConst.h @@ -121,7 +121,7 @@ public: return data->isNullAt(0); } - void insertRangeFrom(const IColumn &, size_t /*start*/, size_t length) override + void doInsertRangeFrom(const IColumn &, size_t /*start*/, size_t length) override { s += length; } @@ -145,12 +145,12 @@ public: ++s; } - void insertFrom(const IColumn &, size_t) override + void doInsertFrom(const IColumn &, size_t) override { ++s; } - void insertManyFrom(const IColumn & /*src*/, size_t /* position */, size_t length) override { s += length; } + void doInsertManyFrom(const IColumn & /*src*/, size_t /* position */, size_t length) override { s += length; } void insertDefault() override { @@ -223,7 +223,7 @@ public: return data->allocatedBytes() + sizeof(s); } - int compareAt(size_t, size_t, const IColumn & rhs, int nan_direction_hint) const override + int doCompareAt(size_t, size_t, const IColumn & rhs, int nan_direction_hint) const override { return data->compareAt(0, 0, *assert_cast(rhs).data, nan_direction_hint); } diff --git a/src/Columns/ColumnDecimal.cpp b/src/Columns/ColumnDecimal.cpp index eb9784c14dd..ad17ccce778 100644 --- a/src/Columns/ColumnDecimal.cpp +++ b/src/Columns/ColumnDecimal.cpp @@ -32,7 +32,7 @@ namespace ErrorCodes } template -int ColumnDecimal::compareAt(size_t n, size_t m, const IColumn & rhs_, int) const +int ColumnDecimal::doCompareAt(size_t n, size_t m, const IColumn & rhs_, int) const { auto & other = static_cast(rhs_); const T & a = data[n]; @@ -331,7 +331,7 @@ void ColumnDecimal::insertData(const char * src, size_t /*length*/) } template -void ColumnDecimal::insertRangeFrom(const IColumn & src, size_t start, size_t length) +void ColumnDecimal::doInsertRangeFrom(const IColumn & src, size_t start, size_t length) { const ColumnDecimal & src_vec = assert_cast(src); diff --git a/src/Columns/ColumnDecimal.h b/src/Columns/ColumnDecimal.h index c4510ba2922..019ce442bcc 100644 --- a/src/Columns/ColumnDecimal.h +++ b/src/Columns/ColumnDecimal.h @@ -55,9 +55,9 @@ public: void reserve(size_t n) override { data.reserve_exact(n); } void shrinkToFit() override { data.shrink_to_fit(); } - void insertFrom(const IColumn & src, size_t n) override { data.push_back(static_cast(src).getData()[n]); } + void doInsertFrom(const IColumn & src, size_t n) override { data.push_back(static_cast(src).getData()[n]); } - void insertManyFrom(const IColumn & src, size_t position, size_t length) override + void doInsertManyFrom(const IColumn & src, size_t position, size_t length) override { ValueType v = assert_cast(src).getData()[position]; data.resize_fill(data.size() + length, v); @@ -68,7 +68,7 @@ public: void insertManyDefaults(size_t length) override { data.resize_fill(data.size() + length); } void insert(const Field & x) override { data.push_back(x.get()); } bool tryInsert(const Field & x) override; - void insertRangeFrom(const IColumn & src, size_t start, size_t length) override; + void doInsertRangeFrom(const IColumn & src, size_t start, size_t length) override; void popBack(size_t n) override { @@ -92,7 +92,7 @@ public: void updateHashWithValue(size_t n, SipHash & hash) const override; void updateWeakHash32(WeakHash32 & hash) const override; void updateHashFast(SipHash & hash) const override; - int compareAt(size_t n, size_t m, const IColumn & rhs_, int nan_direction_hint) const override; + int doCompareAt(size_t n, size_t m, const IColumn & rhs_, int nan_direction_hint) const override; void getPermutation(IColumn::PermutationSortDirection direction, IColumn::PermutationSortStability stability, size_t limit, int nan_direction_hint, IColumn::Permutation & res) const override; void updatePermutation(IColumn::PermutationSortDirection direction, IColumn::PermutationSortStability stability, diff --git a/src/Columns/ColumnDynamic.cpp b/src/Columns/ColumnDynamic.cpp index 3c147b6f123..63ddd5e566c 100644 --- a/src/Columns/ColumnDynamic.cpp +++ b/src/Columns/ColumnDynamic.cpp @@ -213,7 +213,7 @@ bool ColumnDynamic::tryInsert(const DB::Field & x) } -void ColumnDynamic::insertFrom(const DB::IColumn & src_, size_t n) +void ColumnDynamic::doInsertFrom(const DB::IColumn & src_, size_t n) { const auto & dynamic_src = assert_cast(src_); @@ -263,7 +263,7 @@ void ColumnDynamic::insertFrom(const DB::IColumn & src_, size_t n) variant_col.insertIntoVariantFrom(string_variant_discr, *tmp_string_column, 0); } -void ColumnDynamic::insertRangeFrom(const DB::IColumn & src_, size_t start, size_t length) +void ColumnDynamic::doInsertRangeFrom(const DB::IColumn & src_, size_t start, size_t length) { if (start + length > src_.size()) throw Exception(ErrorCodes::PARAMETER_OUT_OF_BOUND, "Parameter out of bound in ColumnDynamic::insertRangeFrom method. " @@ -429,7 +429,7 @@ void ColumnDynamic::insertRangeFrom(const DB::IColumn & src_, size_t start, size } } -void ColumnDynamic::insertManyFrom(const DB::IColumn & src_, size_t position, size_t length) +void ColumnDynamic::doInsertManyFrom(const DB::IColumn & src_, size_t position, size_t length) { const auto & dynamic_src = assert_cast(src_); @@ -587,7 +587,7 @@ void ColumnDynamic::updateHashWithValue(size_t n, SipHash & hash) const variant_col.getVariantByGlobalDiscriminator(discr).updateHashWithValue(variant_col.offsetAt(n), hash); } -int ColumnDynamic::compareAt(size_t n, size_t m, const DB::IColumn & rhs, int nan_direction_hint) const +int ColumnDynamic::doCompareAt(size_t n, size_t m, const DB::IColumn & rhs, int nan_direction_hint) const { const auto & left_variant = assert_cast(*variant_column); const auto & right_dynamic = assert_cast(rhs); diff --git a/src/Columns/ColumnDynamic.h b/src/Columns/ColumnDynamic.h index 27ad0dd583f..7156cd49678 100644 --- a/src/Columns/ColumnDynamic.h +++ b/src/Columns/ColumnDynamic.h @@ -142,9 +142,9 @@ public: void insert(const Field & x) override; bool tryInsert(const Field & x) override; - void insertFrom(const IColumn & src_, size_t n) override; - void insertRangeFrom(const IColumn & src, size_t start, size_t length) override; - void insertManyFrom(const IColumn & src, size_t position, size_t length) override; + void doInsertFrom(const IColumn & src_, size_t n) override; + void doInsertRangeFrom(const IColumn & src, size_t start, size_t length) override; + void doInsertManyFrom(const IColumn & src, size_t position, size_t length) override; void insertDefault() override { @@ -213,7 +213,7 @@ public: return scattered_columns; } - int compareAt(size_t n, size_t m, const IColumn & rhs, int nan_direction_hint) const override; + int doCompareAt(size_t n, size_t m, const IColumn & rhs, int nan_direction_hint) const override; bool hasEqualValues() const override { diff --git a/src/Columns/ColumnFixedString.cpp b/src/Columns/ColumnFixedString.cpp index d7e4eff2727..081a46b78d9 100644 --- a/src/Columns/ColumnFixedString.cpp +++ b/src/Columns/ColumnFixedString.cpp @@ -74,7 +74,7 @@ bool ColumnFixedString::tryInsert(const Field & x) return true; } -void ColumnFixedString::insertFrom(const IColumn & src_, size_t index) +void ColumnFixedString::doInsertFrom(const IColumn & src_, size_t index) { const ColumnFixedString & src = assert_cast(src_); @@ -86,7 +86,7 @@ void ColumnFixedString::insertFrom(const IColumn & src_, size_t index) memcpySmallAllowReadWriteOverflow15(chars.data() + old_size, &src.chars[n * index], n); } -void ColumnFixedString::insertManyFrom(const IColumn & src, size_t position, size_t length) +void ColumnFixedString::doInsertManyFrom(const IColumn & src, size_t position, size_t length) { const ColumnFixedString & src_concrete = assert_cast(src); if (n != src_concrete.getN()) @@ -219,7 +219,7 @@ size_t ColumnFixedString::estimateCardinalityInPermutedRange(const Permutation & return elements.size(); } -void ColumnFixedString::insertRangeFrom(const IColumn & src, size_t start, size_t length) +void ColumnFixedString::doInsertRangeFrom(const IColumn & src, size_t start, size_t length) { const ColumnFixedString & src_concrete = assert_cast(src); chassert(this->n == src_concrete.n); diff --git a/src/Columns/ColumnFixedString.h b/src/Columns/ColumnFixedString.h index 7b46dc11cd6..47df6bec4cc 100644 --- a/src/Columns/ColumnFixedString.h +++ b/src/Columns/ColumnFixedString.h @@ -98,9 +98,9 @@ public: bool tryInsert(const Field & x) override; - void insertFrom(const IColumn & src_, size_t index) override; + void doInsertFrom(const IColumn & src_, size_t index) override; - void insertManyFrom(const IColumn & src, size_t position, size_t length) override; + void doInsertManyFrom(const IColumn & src, size_t position, size_t length) override; void insertData(const char * pos, size_t length) override; @@ -129,7 +129,7 @@ public: void updateHashFast(SipHash & hash) const override; - int compareAt(size_t p1, size_t p2, const IColumn & rhs_, int /*nan_direction_hint*/) const override + int doCompareAt(size_t p1, size_t p2, const IColumn & rhs_, int /*nan_direction_hint*/) const override { const ColumnFixedString & rhs = assert_cast(rhs_); chassert(this->n == rhs.n); @@ -144,7 +144,7 @@ public: size_t estimateCardinalityInPermutedRange(const Permutation & permutation, const EqualRange & equal_range) const override; - void insertRangeFrom(const IColumn & src, size_t start, size_t length) override; + void doInsertRangeFrom(const IColumn & src, size_t start, size_t length) override; ColumnPtr filter(const IColumn::Filter & filt, ssize_t result_size_hint) const override; diff --git a/src/Columns/ColumnFunction.cpp b/src/Columns/ColumnFunction.cpp index 0ab9d15ad50..f40b951ef13 100644 --- a/src/Columns/ColumnFunction.cpp +++ b/src/Columns/ColumnFunction.cpp @@ -72,7 +72,7 @@ ColumnPtr ColumnFunction::cut(size_t start, size_t length) const return ColumnFunction::create(length, function, capture, is_short_circuit_argument, is_function_compiled); } -void ColumnFunction::insertFrom(const IColumn & src, size_t n) +void ColumnFunction::doInsertFrom(const IColumn & src, size_t n) { const ColumnFunction & src_func = assert_cast(src); @@ -89,7 +89,7 @@ void ColumnFunction::insertFrom(const IColumn & src, size_t n) ++elements_size; } -void ColumnFunction::insertRangeFrom(const IColumn & src, size_t start, size_t length) +void ColumnFunction::doInsertRangeFrom(const IColumn & src, size_t start, size_t length) { const ColumnFunction & src_func = assert_cast(src); diff --git a/src/Columns/ColumnFunction.h b/src/Columns/ColumnFunction.h index 6fdc6679d3e..ab5a63d1afe 100644 --- a/src/Columns/ColumnFunction.h +++ b/src/Columns/ColumnFunction.h @@ -94,8 +94,8 @@ public: throw Exception(ErrorCodes::NOT_IMPLEMENTED, "Cannot insert into {}", getName()); } - void insertFrom(const IColumn & src, size_t n) override; - void insertRangeFrom(const IColumn &, size_t start, size_t length) override; + void doInsertFrom(const IColumn & src, size_t n) override; + void doInsertRangeFrom(const IColumn &, size_t start, size_t length) override; void insertData(const char *, size_t) override { @@ -137,7 +137,7 @@ public: throw Exception(ErrorCodes::NOT_IMPLEMENTED, "popBack is not implemented for {}", getName()); } - int compareAt(size_t, size_t, const IColumn &, int) const override + int doCompareAt(size_t, size_t, const IColumn &, int) const override { throw Exception(ErrorCodes::NOT_IMPLEMENTED, "compareAt is not implemented for {}", getName()); } diff --git a/src/Columns/ColumnLowCardinality.cpp b/src/Columns/ColumnLowCardinality.cpp index 208326fe629..a2d4903284e 100644 --- a/src/Columns/ColumnLowCardinality.cpp +++ b/src/Columns/ColumnLowCardinality.cpp @@ -159,7 +159,7 @@ void ColumnLowCardinality::insertDefault() idx.insertPosition(getDictionary().getDefaultValueIndex()); } -void ColumnLowCardinality::insertFrom(const IColumn & src, size_t n) +void ColumnLowCardinality::doInsertFrom(const IColumn & src, size_t n) { const auto * low_cardinality_src = typeid_cast(&src); @@ -187,7 +187,7 @@ void ColumnLowCardinality::insertFromFullColumn(const IColumn & src, size_t n) idx.insertPosition(getDictionary().uniqueInsertFrom(src, n)); } -void ColumnLowCardinality::insertRangeFrom(const IColumn & src, size_t start, size_t length) +void ColumnLowCardinality::doInsertRangeFrom(const IColumn & src, size_t start, size_t length) { const auto * low_cardinality_src = typeid_cast(&src); @@ -364,7 +364,7 @@ int ColumnLowCardinality::compareAtImpl(size_t n, size_t m, const IColumn & rhs, return getDictionary().compareAt(n_index, m_index, low_cardinality_column.getDictionary(), nan_direction_hint); } -int ColumnLowCardinality::compareAt(size_t n, size_t m, const IColumn & rhs, int nan_direction_hint) const +int ColumnLowCardinality::doCompareAt(size_t n, size_t m, const IColumn & rhs, int nan_direction_hint) const { return compareAtImpl(n, m, rhs, nan_direction_hint); } diff --git a/src/Columns/ColumnLowCardinality.h b/src/Columns/ColumnLowCardinality.h index ac3b725b22f..43984cbcf8a 100644 --- a/src/Columns/ColumnLowCardinality.h +++ b/src/Columns/ColumnLowCardinality.h @@ -78,10 +78,10 @@ public: bool tryInsert(const Field & x) override; void insertDefault() override; - void insertFrom(const IColumn & src, size_t n) override; + void doInsertFrom(const IColumn & src, size_t n) override; void insertFromFullColumn(const IColumn & src, size_t n); - void insertRangeFrom(const IColumn & src, size_t start, size_t length) override; + void doInsertRangeFrom(const IColumn & src, size_t start, size_t length) override; void insertRangeFromFullColumn(const IColumn & src, size_t start, size_t length); void insertRangeFromDictionaryEncodedColumn(const IColumn & keys, const IColumn & positions); @@ -127,7 +127,7 @@ public: return ColumnLowCardinality::create(dictionary.getColumnUniquePtr(), getIndexes().index(indexes_, limit)); } - int compareAt(size_t n, size_t m, const IColumn & rhs, int nan_direction_hint) const override; + int doCompareAt(size_t n, size_t m, const IColumn & rhs, int nan_direction_hint) const override; int compareAtWithCollation(size_t n, size_t m, const IColumn & rhs, int nan_direction_hint, const Collator &) const override; diff --git a/src/Columns/ColumnMap.cpp b/src/Columns/ColumnMap.cpp index eecea1a273f..5b4cd04d67a 100644 --- a/src/Columns/ColumnMap.cpp +++ b/src/Columns/ColumnMap.cpp @@ -153,17 +153,17 @@ void ColumnMap::updateHashFast(SipHash & hash) const nested->updateHashFast(hash); } -void ColumnMap::insertFrom(const IColumn & src, size_t n) +void ColumnMap::doInsertFrom(const IColumn & src, size_t n) { nested->insertFrom(assert_cast(src).getNestedColumn(), n); } -void ColumnMap::insertManyFrom(const IColumn & src, size_t position, size_t length) +void ColumnMap::doInsertManyFrom(const IColumn & src, size_t position, size_t length) { assert_cast(*nested).insertManyFrom(assert_cast(src).getNestedColumn(), position, length); } -void ColumnMap::insertRangeFrom(const IColumn & src, size_t start, size_t length) +void ColumnMap::doInsertRangeFrom(const IColumn & src, size_t start, size_t length) { nested->insertRangeFrom( assert_cast(src).getNestedColumn(), @@ -210,7 +210,7 @@ MutableColumns ColumnMap::scatter(ColumnIndex num_columns, const Selector & sele return res; } -int ColumnMap::compareAt(size_t n, size_t m, const IColumn & rhs, int nan_direction_hint) const +int ColumnMap::doCompareAt(size_t n, size_t m, const IColumn & rhs, int nan_direction_hint) const { const auto & rhs_map = assert_cast(rhs); return nested->compareAt(n, m, rhs_map.getNestedColumn(), nan_direction_hint); diff --git a/src/Columns/ColumnMap.h b/src/Columns/ColumnMap.h index 52165d0d74e..9834b2ed0ee 100644 --- a/src/Columns/ColumnMap.h +++ b/src/Columns/ColumnMap.h @@ -66,16 +66,16 @@ public: void updateHashWithValue(size_t n, SipHash & hash) const override; void updateWeakHash32(WeakHash32 & hash) const override; void updateHashFast(SipHash & hash) const override; - void insertFrom(const IColumn & src_, size_t n) override; - void insertManyFrom(const IColumn & src, size_t position, size_t length) override; - void insertRangeFrom(const IColumn & src, size_t start, size_t length) override; + void doInsertFrom(const IColumn & src_, size_t n) override; + void doInsertManyFrom(const IColumn & src, size_t position, size_t length) override; + void doInsertRangeFrom(const IColumn & src, size_t start, size_t length) override; ColumnPtr filter(const Filter & filt, ssize_t result_size_hint) const override; void expand(const Filter & mask, bool inverted) override; ColumnPtr permute(const Permutation & perm, size_t limit) const override; ColumnPtr index(const IColumn & indexes, size_t limit) const override; ColumnPtr replicate(const Offsets & offsets) const override; MutableColumns scatter(ColumnIndex num_columns, const Selector & selector) const override; - int compareAt(size_t n, size_t m, const IColumn & rhs, int nan_direction_hint) const override; + int doCompareAt(size_t n, size_t m, const IColumn & rhs, int nan_direction_hint) const override; void getExtremes(Field & min, Field & max) const override; void getPermutation(IColumn::PermutationSortDirection direction, IColumn::PermutationSortStability stability, size_t limit, int nan_direction_hint, IColumn::Permutation & res) const override; diff --git a/src/Columns/ColumnNullable.cpp b/src/Columns/ColumnNullable.cpp index 1d12a59fd59..8e57204721f 100644 --- a/src/Columns/ColumnNullable.cpp +++ b/src/Columns/ColumnNullable.cpp @@ -221,7 +221,7 @@ const char * ColumnNullable::skipSerializedInArena(const char * pos) const return pos; } -void ColumnNullable::insertRangeFrom(const IColumn & src, size_t start, size_t length) +void ColumnNullable::doInsertRangeFrom(const IColumn & src, size_t start, size_t length) { const ColumnNullable & nullable_col = assert_cast(src); getNullMapColumn().insertRangeFrom(*nullable_col.null_map, start, length); @@ -258,7 +258,7 @@ bool ColumnNullable::tryInsert(const Field & x) return true; } -void ColumnNullable::insertFrom(const IColumn & src, size_t n) +void ColumnNullable::doInsertFrom(const IColumn & src, size_t n) { const ColumnNullable & src_concrete = assert_cast(src); getNestedColumn().insertFrom(src_concrete.getNestedColumn(), n); @@ -266,7 +266,7 @@ void ColumnNullable::insertFrom(const IColumn & src, size_t n) } -void ColumnNullable::insertManyFrom(const IColumn & src, size_t position, size_t length) +void ColumnNullable::doInsertManyFrom(const IColumn & src, size_t position, size_t length) { const ColumnNullable & src_concrete = assert_cast(src); getNestedColumn().insertManyFrom(src_concrete.getNestedColumn(), position, length); @@ -402,7 +402,7 @@ int ColumnNullable::compareAtImpl(size_t n, size_t m, const IColumn & rhs_, int return getNestedColumn().compareAt(n, m, nested_rhs, null_direction_hint); } -int ColumnNullable::compareAt(size_t n, size_t m, const IColumn & rhs_, int null_direction_hint) const +int ColumnNullable::doCompareAt(size_t n, size_t m, const IColumn & rhs_, int null_direction_hint) const { return compareAtImpl(n, m, rhs_, null_direction_hint); } diff --git a/src/Columns/ColumnNullable.h b/src/Columns/ColumnNullable.h index 510a4cacf1e..1e5866b8abb 100644 --- a/src/Columns/ColumnNullable.h +++ b/src/Columns/ColumnNullable.h @@ -69,11 +69,11 @@ public: char * serializeValueIntoMemory(size_t n, char * memory) const override; const char * deserializeAndInsertFromArena(const char * pos) override; const char * skipSerializedInArena(const char * pos) const override; - void insertRangeFrom(const IColumn & src, size_t start, size_t length) override; + void doInsertRangeFrom(const IColumn & src, size_t start, size_t length) override; void insert(const Field & x) override; bool tryInsert(const Field & x) override; - void insertFrom(const IColumn & src, size_t n) override; - void insertManyFrom(const IColumn & src, size_t position, size_t length) override; + void doInsertFrom(const IColumn & src, size_t n) override; + void doInsertManyFrom(const IColumn & src, size_t position, size_t length) override; void insertFromNotNullable(const IColumn & src, size_t n); void insertRangeFromNotNullable(const IColumn & src, size_t start, size_t length); @@ -90,7 +90,7 @@ public: void expand(const Filter & mask, bool inverted) override; ColumnPtr permute(const Permutation & perm, size_t limit) const override; ColumnPtr index(const IColumn & indexes, size_t limit) const override; - int compareAt(size_t n, size_t m, const IColumn & rhs_, int null_direction_hint) const override; + int doCompareAt(size_t n, size_t m, const IColumn & rhs_, int null_direction_hint) const override; #if USE_EMBEDDED_COMPILER diff --git a/src/Columns/ColumnObject.cpp b/src/Columns/ColumnObject.cpp index ded56b60e64..86e94d9397a 100644 --- a/src/Columns/ColumnObject.cpp +++ b/src/Columns/ColumnObject.cpp @@ -763,12 +763,12 @@ void ColumnObject::get(size_t n, Field & res) const } } -void ColumnObject::insertFrom(const IColumn & src, size_t n) +void ColumnObject::doInsertFrom(const IColumn & src, size_t n) { insert(src[n]); } -void ColumnObject::insertRangeFrom(const IColumn & src, size_t start, size_t length) +void ColumnObject::doInsertRangeFrom(const IColumn & src, size_t start, size_t length) { const auto & src_object = assert_cast(src); diff --git a/src/Columns/ColumnObject.h b/src/Columns/ColumnObject.h index b1b8827622f..2fe5adb70b3 100644 --- a/src/Columns/ColumnObject.h +++ b/src/Columns/ColumnObject.h @@ -209,8 +209,8 @@ public: void insert(const Field & field) override; bool tryInsert(const Field & field) override; void insertDefault() override; - void insertFrom(const IColumn & src, size_t n) override; - void insertRangeFrom(const IColumn & src, size_t start, size_t length) override; + void doInsertFrom(const IColumn & src, size_t n) override; + void doInsertRangeFrom(const IColumn & src, size_t start, size_t length) override; void popBack(size_t length) override; Field operator[](size_t n) const override; void get(size_t n, Field & res) const override; @@ -228,7 +228,7 @@ public: /// Order of rows in ColumnObject is undefined. void getPermutation(PermutationSortDirection, PermutationSortStability, size_t, int, Permutation & res) const override; void updatePermutation(PermutationSortDirection, PermutationSortStability, size_t, int, Permutation &, EqualRanges &) const override {} - int compareAt(size_t, size_t, const IColumn &, int) const override { return 0; } + int doCompareAt(size_t, size_t, const IColumn &, int) const override { return 0; } void getExtremes(Field & min, Field & max) const override; /// All other methods throw exception. diff --git a/src/Columns/ColumnSparse.cpp b/src/Columns/ColumnSparse.cpp index 5190ceb49e5..2100ac0cc03 100644 --- a/src/Columns/ColumnSparse.cpp +++ b/src/Columns/ColumnSparse.cpp @@ -174,7 +174,7 @@ const char * ColumnSparse::skipSerializedInArena(const char * pos) const return values->skipSerializedInArena(pos); } -void ColumnSparse::insertRangeFrom(const IColumn & src, size_t start, size_t length) +void ColumnSparse::doInsertRangeFrom(const IColumn & src, size_t start, size_t length) { if (length == 0) return; @@ -248,7 +248,7 @@ bool ColumnSparse::tryInsert(const Field & x) return true; } -void ColumnSparse::insertFrom(const IColumn & src, size_t n) +void ColumnSparse::doInsertFrom(const IColumn & src, size_t n) { if (const auto * src_sparse = typeid_cast(&src)) { @@ -446,7 +446,7 @@ ColumnPtr ColumnSparse::indexImpl(const PaddedPODArray & indexes, size_t l return ColumnSparse::create(std::move(res_values), std::move(res_offsets), limit); } -int ColumnSparse::compareAt(size_t n, size_t m, const IColumn & rhs_, int null_direction_hint) const +int ColumnSparse::doCompareAt(size_t n, size_t m, const IColumn & rhs_, int null_direction_hint) const { if (const auto * rhs_sparse = typeid_cast(&rhs_)) return values->compareAt(getValueIndex(n), rhs_sparse->getValueIndex(m), rhs_sparse->getValuesColumn(), null_direction_hint); diff --git a/src/Columns/ColumnSparse.h b/src/Columns/ColumnSparse.h index 12b2def7cf1..85040255e2e 100644 --- a/src/Columns/ColumnSparse.h +++ b/src/Columns/ColumnSparse.h @@ -81,10 +81,10 @@ public: char * serializeValueIntoMemory(size_t n, char * memory) const override; const char * deserializeAndInsertFromArena(const char * pos) override; const char * skipSerializedInArena(const char *) const override; - void insertRangeFrom(const IColumn & src, size_t start, size_t length) override; + void doInsertRangeFrom(const IColumn & src, size_t start, size_t length) override; void insert(const Field & x) override; bool tryInsert(const Field & x) override; - void insertFrom(const IColumn & src, size_t n) override; + void doInsertFrom(const IColumn & src, size_t n) override; void insertDefault() override; void insertManyDefaults(size_t length) override; @@ -98,7 +98,7 @@ public: template ColumnPtr indexImpl(const PaddedPODArray & indexes, size_t limit) const; - int compareAt(size_t n, size_t m, const IColumn & rhs_, int null_direction_hint) const override; + int doCompareAt(size_t n, size_t m, const IColumn & rhs_, int null_direction_hint) const override; void compareColumn(const IColumn & rhs, size_t rhs_row_num, PaddedPODArray * row_indexes, PaddedPODArray & compare_results, int direction, int nan_direction_hint) const override; diff --git a/src/Columns/ColumnString.cpp b/src/Columns/ColumnString.cpp index a84aea73486..b93488ab1be 100644 --- a/src/Columns/ColumnString.cpp +++ b/src/Columns/ColumnString.cpp @@ -39,7 +39,7 @@ ColumnString::ColumnString(const ColumnString & src) last_offset, chars.size()); } -void ColumnString::insertManyFrom(const IColumn & src, size_t position, size_t length) +void ColumnString::doInsertManyFrom(const IColumn & src, size_t position, size_t length) { const ColumnString & src_concrete = assert_cast(src); const UInt8 * src_buf = &src_concrete.chars[src_concrete.offsets[position - 1]]; @@ -129,7 +129,7 @@ void ColumnString::updateWeakHash32(WeakHash32 & hash) const } -void ColumnString::insertRangeFrom(const IColumn & src, size_t start, size_t length) +void ColumnString::doInsertRangeFrom(const IColumn & src, size_t start, size_t length) { if (length == 0) return; diff --git a/src/Columns/ColumnString.h b/src/Columns/ColumnString.h index 39d4684fd89..c0f2d4c6e50 100644 --- a/src/Columns/ColumnString.h +++ b/src/Columns/ColumnString.h @@ -142,7 +142,7 @@ public: return true; } - void insertFrom(const IColumn & src_, size_t n) override + void doInsertFrom(const IColumn & src_, size_t n) override { const ColumnString & src = assert_cast(src_); const size_t size_to_append = src.offsets[n] - src.offsets[n - 1]; /// -1th index is Ok, see PaddedPODArray. @@ -165,7 +165,7 @@ public: } } - void insertManyFrom(const IColumn & src, size_t position, size_t length) override; + void doInsertManyFrom(const IColumn & src, size_t position, size_t length) override; void insertData(const char * pos, size_t length) override { @@ -212,7 +212,7 @@ public: hash.update(reinterpret_cast(chars.data()), chars.size() * sizeof(chars[0])); } - void insertRangeFrom(const IColumn & src, size_t start, size_t length) override; + void doInsertRangeFrom(const IColumn & src, size_t start, size_t length) override; ColumnPtr filter(const Filter & filt, ssize_t result_size_hint) const override; @@ -238,7 +238,7 @@ public: offsets.push_back(offsets.back() + 1); } - int compareAt(size_t n, size_t m, const IColumn & rhs_, int /*nan_direction_hint*/) const override + int doCompareAt(size_t n, size_t m, const IColumn & rhs_, int /*nan_direction_hint*/) const override { const ColumnString & rhs = assert_cast(rhs_); return memcmpSmallAllowOverflow15(chars.data() + offsetAt(n), sizeAt(n) - 1, rhs.chars.data() + rhs.offsetAt(m), rhs.sizeAt(m) - 1); diff --git a/src/Columns/ColumnTuple.cpp b/src/Columns/ColumnTuple.cpp index 2159495b68f..fd79afaea96 100644 --- a/src/Columns/ColumnTuple.cpp +++ b/src/Columns/ColumnTuple.cpp @@ -205,7 +205,7 @@ bool ColumnTuple::tryInsert(const Field & x) return true; } -void ColumnTuple::insertFrom(const IColumn & src_, size_t n) +void ColumnTuple::doInsertFrom(const IColumn & src_, size_t n) { const ColumnTuple & src = assert_cast(src_); @@ -218,7 +218,7 @@ void ColumnTuple::insertFrom(const IColumn & src_, size_t n) columns[i]->insertFrom(*src.columns[i], n); } -void ColumnTuple::insertManyFrom(const IColumn & src, size_t position, size_t length) +void ColumnTuple::doInsertManyFrom(const IColumn & src, size_t position, size_t length) { const ColumnTuple & src_tuple = assert_cast(src); @@ -318,7 +318,7 @@ void ColumnTuple::updateHashFast(SipHash & hash) const column->updateHashFast(hash); } -void ColumnTuple::insertRangeFrom(const IColumn & src, size_t start, size_t length) +void ColumnTuple::doInsertRangeFrom(const IColumn & src, size_t start, size_t length) { column_length += length; const size_t tuple_size = columns.size(); @@ -470,7 +470,7 @@ int ColumnTuple::compareAtImpl(size_t n, size_t m, const IColumn & rhs, int nan_ return 0; } -int ColumnTuple::compareAt(size_t n, size_t m, const IColumn & rhs, int nan_direction_hint) const +int ColumnTuple::doCompareAt(size_t n, size_t m, const IColumn & rhs, int nan_direction_hint) const { return compareAtImpl(n, m, rhs, nan_direction_hint); } diff --git a/src/Columns/ColumnTuple.h b/src/Columns/ColumnTuple.h index 0103f81b242..78ecb75862e 100644 --- a/src/Columns/ColumnTuple.h +++ b/src/Columns/ColumnTuple.h @@ -65,8 +65,8 @@ public: void insertData(const char * pos, size_t length) override; void insert(const Field & x) override; bool tryInsert(const Field & x) override; - void insertFrom(const IColumn & src_, size_t n) override; - void insertManyFrom(const IColumn & src, size_t position, size_t length) override; + void doInsertFrom(const IColumn & src_, size_t n) override; + void doInsertManyFrom(const IColumn & src, size_t position, size_t length) override; void insertDefault() override; void popBack(size_t n) override; StringRef serializeValueIntoArena(size_t n, Arena & arena, char const *& begin) const override; @@ -76,14 +76,14 @@ public: void updateHashWithValue(size_t n, SipHash & hash) const override; void updateWeakHash32(WeakHash32 & hash) const override; void updateHashFast(SipHash & hash) const override; - void insertRangeFrom(const IColumn & src, size_t start, size_t length) override; + void doInsertRangeFrom(const IColumn & src, size_t start, size_t length) override; ColumnPtr filter(const Filter & filt, ssize_t result_size_hint) const override; void expand(const Filter & mask, bool inverted) override; ColumnPtr permute(const Permutation & perm, size_t limit) const override; ColumnPtr index(const IColumn & indexes, size_t limit) const override; ColumnPtr replicate(const Offsets & offsets) const override; MutableColumns scatter(ColumnIndex num_columns, const Selector & selector) const override; - int compareAt(size_t n, size_t m, const IColumn & rhs, int nan_direction_hint) const override; + int doCompareAt(size_t n, size_t m, const IColumn & rhs, int nan_direction_hint) const override; int compareAtWithCollation(size_t n, size_t m, const IColumn & rhs, int nan_direction_hint, const Collator & collator) const override; void getExtremes(Field & min, Field & max) const override; void getPermutation(IColumn::PermutationSortDirection direction, IColumn::PermutationSortStability stability, diff --git a/src/Columns/ColumnUnique.h b/src/Columns/ColumnUnique.h index 0311efd4c83..12f765f42af 100644 --- a/src/Columns/ColumnUnique.h +++ b/src/Columns/ColumnUnique.h @@ -90,7 +90,7 @@ public: return getNestedColumn()->updateHashWithValue(n, hash_func); } - int compareAt(size_t n, size_t m, const IColumn & rhs, int nan_direction_hint) const override; + int doCompareAt(size_t n, size_t m, const IColumn & rhs, int nan_direction_hint) const override; void getExtremes(Field & min, Field & max) const override { column_holder->getExtremes(min, max); } bool valuesHaveFixedSize() const override { return column_holder->valuesHaveFixedSize(); } @@ -488,7 +488,7 @@ const char * ColumnUnique::skipSerializedInArena(const char *) const } template -int ColumnUnique::compareAt(size_t n, size_t m, const IColumn & rhs, int nan_direction_hint) const +int ColumnUnique::doCompareAt(size_t n, size_t m, const IColumn & rhs, int nan_direction_hint) const { if (is_nullable) { diff --git a/src/Columns/ColumnVariant.cpp b/src/Columns/ColumnVariant.cpp index ec47f5dfa74..137a396d3c4 100644 --- a/src/Columns/ColumnVariant.cpp +++ b/src/Columns/ColumnVariant.cpp @@ -595,17 +595,17 @@ void ColumnVariant::insertManyFromImpl(const DB::IColumn & src_, size_t position } } -void ColumnVariant::insertFrom(const IColumn & src_, size_t n) +void ColumnVariant::doInsertFrom(const IColumn & src_, size_t n) { insertFromImpl(src_, n, nullptr); } -void ColumnVariant::insertRangeFrom(const IColumn & src_, size_t start, size_t length) +void ColumnVariant::doInsertRangeFrom(const IColumn & src_, size_t start, size_t length) { insertRangeFromImpl(src_, start, length, nullptr); } -void ColumnVariant::insertManyFrom(const DB::IColumn & src_, size_t position, size_t length) +void ColumnVariant::doInsertManyFrom(const DB::IColumn & src_, size_t position, size_t length) { insertManyFromImpl(src_, position, length, nullptr); } @@ -1174,7 +1174,7 @@ bool ColumnVariant::hasEqualValues() const return local_discriminators->hasEqualValues() && variants[localDiscriminatorAt(0)]->hasEqualValues(); } -int ColumnVariant::compareAt(size_t n, size_t m, const IColumn & rhs, int nan_direction_hint) const +int ColumnVariant::doCompareAt(size_t n, size_t m, const IColumn & rhs, int nan_direction_hint) const { const auto & rhs_variant = assert_cast(rhs); Discriminator left_discr = globalDiscriminatorAt(n); diff --git a/src/Columns/ColumnVariant.h b/src/Columns/ColumnVariant.h index e5a4498f340..f516b616e02 100644 --- a/src/Columns/ColumnVariant.h +++ b/src/Columns/ColumnVariant.h @@ -180,9 +180,12 @@ public: void insert(const Field & x) override; bool tryInsert(const Field & x) override; - void insertFrom(const IColumn & src_, size_t n) override; - void insertRangeFrom(const IColumn & src_, size_t start, size_t length) override; - void insertManyFrom(const IColumn & src_, size_t position, size_t length) override; + void doInsertFrom(const IColumn & src_, size_t n) override; + void doInsertRangeFrom(const IColumn & src_, size_t start, size_t length) override; + void doInsertManyFrom(const IColumn & src_, size_t position, size_t length) override; + + using IColumn::insertFrom; + using IColumn::insertRangeFrom; /// Methods for insertion from another Variant but with known mapping between global discriminators. void insertFrom(const IColumn & src_, size_t n, const std::vector & global_discriminators_mapping); @@ -213,7 +216,7 @@ public: ColumnPtr indexImpl(const PaddedPODArray & indexes, size_t limit) const; ColumnPtr replicate(const Offsets & replicate_offsets) const override; MutableColumns scatter(ColumnIndex num_columns, const Selector & selector) const override; - int compareAt(size_t n, size_t m, const IColumn & rhs, int nan_direction_hint) const override; + int doCompareAt(size_t n, size_t m, const IColumn & rhs, int nan_direction_hint) const override; bool hasEqualValues() const override; void getExtremes(Field & min, Field & max) const override; void getPermutation(IColumn::PermutationSortDirection direction, IColumn::PermutationSortStability stability, diff --git a/src/Columns/ColumnVector.cpp b/src/Columns/ColumnVector.cpp index 35d9f5386ed..f46d244e9d9 100644 --- a/src/Columns/ColumnVector.cpp +++ b/src/Columns/ColumnVector.cpp @@ -503,7 +503,7 @@ bool ColumnVector::tryInsert(const DB::Field & x) } template -void ColumnVector::insertRangeFrom(const IColumn & src, size_t start, size_t length) +void ColumnVector::doInsertRangeFrom(const IColumn & src, size_t start, size_t length) { const ColumnVector & src_vec = assert_cast(src); diff --git a/src/Columns/ColumnVector.h b/src/Columns/ColumnVector.h index bbd27c91a70..f0bc8257a3f 100644 --- a/src/Columns/ColumnVector.h +++ b/src/Columns/ColumnVector.h @@ -64,12 +64,12 @@ public: return data.size(); } - void insertFrom(const IColumn & src, size_t n) override + void doInsertFrom(const IColumn & src, size_t n) override { data.push_back(assert_cast(src).getData()[n]); } - void insertManyFrom(const IColumn & src, size_t position, size_t length) override + void doInsertManyFrom(const IColumn & src, size_t position, size_t length) override { ValueType v = assert_cast(src).getData()[position]; data.resize_fill(data.size() + length, v); @@ -142,7 +142,7 @@ public: } /// This method implemented in header because it could be possibly devirtualized. - int compareAt(size_t n, size_t m, const IColumn & rhs_, int nan_direction_hint) const override + int doCompareAt(size_t n, size_t m, const IColumn & rhs_, int nan_direction_hint) const override { return CompareHelper::compare(data[n], assert_cast(rhs_).data[m], nan_direction_hint); } @@ -228,7 +228,7 @@ public: bool tryInsert(const DB::Field & x) override; - void insertRangeFrom(const IColumn & src, size_t start, size_t length) override; + void doInsertRangeFrom(const IColumn & src, size_t start, size_t length) override; ColumnPtr filter(const IColumn::Filter & filt, ssize_t result_size_hint) const override; diff --git a/src/Columns/IColumn.cpp b/src/Columns/IColumn.cpp index 90cccef2b03..1b4bd8f6e62 100644 --- a/src/Columns/IColumn.cpp +++ b/src/Columns/IColumn.cpp @@ -46,7 +46,7 @@ String IColumn::dumpStructure() const return res.str(); } -void IColumn::insertFrom(const IColumn & src, size_t n) +void IColumn::doInsertFrom(const IColumn & src, size_t n) { insert(src[n]); } diff --git a/src/Columns/IColumn.h b/src/Columns/IColumn.h index afa301d5c1c..6c69b40ecc8 100644 --- a/src/Columns/IColumn.h +++ b/src/Columns/IColumn.h @@ -180,18 +180,14 @@ public: /// Appends n-th element from other column with the same type. /// Is used in merge-sort and merges. It could be implemented in inherited classes more optimally than default implementation. - virtual void insertFrom(const IColumn & src, size_t n); + void insertFrom(const IColumn & src, size_t n) { doInsertFrom(src, n); } /// Appends range of elements from other column with the same type. /// Could be used to concatenate columns. - virtual void insertRangeFrom(const IColumn & src, size_t start, size_t length) = 0; + void insertRangeFrom(const IColumn & src, size_t start, size_t length) { doInsertRangeFrom(src, start, length); } /// Appends one element from other column with the same type multiple times. - virtual void insertManyFrom(const IColumn & src, size_t position, size_t length) - { - for (size_t i = 0; i < length; ++i) - insertFrom(src, position); - } + void insertManyFrom(const IColumn & src, size_t position, size_t length) { doInsertManyFrom(src, position, length); } /// Appends one field multiple times. Can be optimized in inherited classes. virtual void insertMany(const Field & field, size_t length) @@ -322,7 +318,10 @@ public: * * For non Nullable and non floating point types, nan_direction_hint is ignored. */ - [[nodiscard]] virtual int compareAt(size_t n, size_t m, const IColumn & rhs, int nan_direction_hint) const = 0; + [[nodiscard]] int compareAt(size_t n, size_t m, const IColumn & rhs, int nan_direction_hint) const + { + return doCompareAt(n, m, rhs, nan_direction_hint); + } #if USE_EMBEDDED_COMPILER @@ -633,6 +632,18 @@ protected: Equals equals, Sort full_sort, PartialSort partial_sort) const; + + virtual void doInsertFrom(const IColumn & src, size_t n); + + virtual void doInsertRangeFrom(const IColumn & src, size_t start, size_t length) = 0; + + virtual void doInsertManyFrom(const IColumn & src, size_t position, size_t length) + { + for (size_t i = 0; i < length; ++i) + insertFrom(src, position); + } + + virtual int doCompareAt(size_t n, size_t m, const IColumn & rhs, int nan_direction_hint) const = 0; }; using ColumnPtr = IColumn::Ptr; diff --git a/src/Columns/IColumnDummy.h b/src/Columns/IColumnDummy.h index 27f420fbc71..f6a1109d0a2 100644 --- a/src/Columns/IColumnDummy.h +++ b/src/Columns/IColumnDummy.h @@ -26,7 +26,7 @@ public: size_t byteSize() const override { return 0; } size_t byteSizeAt(size_t) const override { return 0; } size_t allocatedBytes() const override { return 0; } - int compareAt(size_t, size_t, const IColumn &, int) const override { return 0; } + int doCompareAt(size_t, size_t, const IColumn &, int) const override { return 0; } void compareColumn(const IColumn &, size_t, PaddedPODArray *, PaddedPODArray &, int, int) const override { } @@ -67,12 +67,12 @@ public: { } - void insertFrom(const IColumn &, size_t) override + void doInsertFrom(const IColumn &, size_t) override { ++s; } - void insertRangeFrom(const IColumn & /*src*/, size_t /*start*/, size_t length) override + void doInsertRangeFrom(const IColumn & /*src*/, size_t /*start*/, size_t length) override { s += length; } diff --git a/src/Columns/IColumnUnique.h b/src/Columns/IColumnUnique.h index f71f19a5da6..3ced489c04d 100644 --- a/src/Columns/IColumnUnique.h +++ b/src/Columns/IColumnUnique.h @@ -85,7 +85,7 @@ public: throw Exception(ErrorCodes::NOT_IMPLEMENTED, "Method tryInsert is not supported for ColumnUnique."); } - void insertRangeFrom(const IColumn &, size_t, size_t) override + void doInsertRangeFrom(const IColumn &, size_t, size_t) override { throw Exception(ErrorCodes::NOT_IMPLEMENTED, "Method insertRangeFrom is not supported for ColumnUnique."); } diff --git a/src/Columns/benchmarks/benchmark_column_insert_many_from.cpp b/src/Columns/benchmarks/benchmark_column_insert_many_from.cpp index 325cf5559cd..6b9c329cd55 100644 --- a/src/Columns/benchmarks/benchmark_column_insert_many_from.cpp +++ b/src/Columns/benchmarks/benchmark_column_insert_many_from.cpp @@ -52,7 +52,7 @@ static ColumnPtr mockColumn(const DataTypePtr & type, size_t rows) } -static NO_INLINE void insertManyFrom(IColumn & dst, const IColumn & src) +static NO_INLINE void doInsertManyFrom(IColumn & dst, const IColumn & src) { size_t size = src.size(); dst.insertManyFrom(src, size / 2, size); From dd1b85e63d69c7e461922ca15aae9b436bf20053 Mon Sep 17 00:00:00 2001 From: Nikita Taranov Date: Tue, 25 Jun 2024 19:42:58 +0100 Subject: [PATCH 309/417] add type check --- src/Columns/IColumn.h | 32 ++++++++++++++++++++++++-------- 1 file changed, 24 insertions(+), 8 deletions(-) diff --git a/src/Columns/IColumn.h b/src/Columns/IColumn.h index 6c69b40ecc8..14bcbf60c1b 100644 --- a/src/Columns/IColumn.h +++ b/src/Columns/IColumn.h @@ -1,11 +1,11 @@ #pragma once -#include -#include -#include -#include -#include #include +#include +#include +#include +#include +#include #include "config.h" @@ -180,14 +180,26 @@ public: /// Appends n-th element from other column with the same type. /// Is used in merge-sort and merges. It could be implemented in inherited classes more optimally than default implementation. - void insertFrom(const IColumn & src, size_t n) { doInsertFrom(src, n); } + void insertFrom(const IColumn & src, size_t n) + { + assertTypeEquality(src); + doInsertFrom(src, n); + } /// Appends range of elements from other column with the same type. /// Could be used to concatenate columns. - void insertRangeFrom(const IColumn & src, size_t start, size_t length) { doInsertRangeFrom(src, start, length); } + void insertRangeFrom(const IColumn & src, size_t start, size_t length) + { + assertTypeEquality(src); + doInsertRangeFrom(src, start, length); + } /// Appends one element from other column with the same type multiple times. - void insertManyFrom(const IColumn & src, size_t position, size_t length) { doInsertManyFrom(src, position, length); } + void insertManyFrom(const IColumn & src, size_t position, size_t length) + { + assertTypeEquality(src); + doInsertManyFrom(src, position, length); + } /// Appends one field multiple times. Can be optimized in inherited classes. virtual void insertMany(const Field & field, size_t length) @@ -320,6 +332,7 @@ public: */ [[nodiscard]] int compareAt(size_t n, size_t m, const IColumn & rhs, int nan_direction_hint) const { + assertTypeEquality(rhs); return doCompareAt(n, m, rhs, nan_direction_hint); } @@ -644,6 +657,9 @@ protected: } virtual int doCompareAt(size_t n, size_t m, const IColumn & rhs, int nan_direction_hint) const = 0; + +private: + void assertTypeEquality(const IColumn & rhs) const { chassert(typeid(*this) == typeid(rhs)); } }; using ColumnPtr = IColumn::Ptr; From 2302fcf9ac8237ae736fe7a0aa924c4f4dd58da1 Mon Sep 17 00:00:00 2001 From: Nikita Taranov Date: Wed, 26 Jun 2024 12:01:24 +0100 Subject: [PATCH 310/417] fix build --- src/Columns/ColumnVariant.h | 1 + 1 file changed, 1 insertion(+) diff --git a/src/Columns/ColumnVariant.h b/src/Columns/ColumnVariant.h index f516b616e02..fe8cf20e20d 100644 --- a/src/Columns/ColumnVariant.h +++ b/src/Columns/ColumnVariant.h @@ -185,6 +185,7 @@ public: void doInsertManyFrom(const IColumn & src_, size_t position, size_t length) override; using IColumn::insertFrom; + using IColumn::insertManyFrom; using IColumn::insertRangeFrom; /// Methods for insertion from another Variant but with known mapping between global discriminators. From 286f2abca25e4a4a5479984e8d9a8fe4037dd70d Mon Sep 17 00:00:00 2001 From: Nikita Taranov Date: Wed, 26 Jun 2024 23:18:35 +0100 Subject: [PATCH 311/417] try fix --- src/Columns/IColumn.cpp | 8 ++++++++ src/Columns/IColumn.h | 5 ++++- 2 files changed, 12 insertions(+), 1 deletion(-) diff --git a/src/Columns/IColumn.cpp b/src/Columns/IColumn.cpp index 1b4bd8f6e62..985d791cfbc 100644 --- a/src/Columns/IColumn.cpp +++ b/src/Columns/IColumn.cpp @@ -104,6 +104,14 @@ void IColumn::forEachSubcolumnRecursively(RecursiveColumnCallback callback) cons }); } +void IColumn::assertTypeEquality(const IColumn & rhs) const +{ + if (typeid(*this) != typeid(rhs)) + LOG_DEBUG(&Poco::Logger::get("IColumn"), "typeid(*this) = {}, typeid(rhs) = {}", typeid(*this).name(), typeid(rhs).name()); + + chassert(isSparse() || typeid(*this) == typeid(rhs)); +} + bool isColumnNullable(const IColumn & column) { return checkColumn(column); diff --git a/src/Columns/IColumn.h b/src/Columns/IColumn.h index 14bcbf60c1b..a586214ab0f 100644 --- a/src/Columns/IColumn.h +++ b/src/Columns/IColumn.h @@ -9,6 +9,9 @@ #include "config.h" +#include +#include + class SipHash; class Collator; @@ -659,7 +662,7 @@ protected: virtual int doCompareAt(size_t n, size_t m, const IColumn & rhs, int nan_direction_hint) const = 0; private: - void assertTypeEquality(const IColumn & rhs) const { chassert(typeid(*this) == typeid(rhs)); } + void assertTypeEquality(const IColumn & rhs) const; }; using ColumnPtr = IColumn::Ptr; From f972e80589ef09164ee51b783f9ebf6361e03df4 Mon Sep 17 00:00:00 2001 From: Nikita Taranov Date: Thu, 27 Jun 2024 13:48:03 +0100 Subject: [PATCH 312/417] fix --- src/Columns/IColumn.cpp | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/src/Columns/IColumn.cpp b/src/Columns/IColumn.cpp index 985d791cfbc..d054d3ad8d4 100644 --- a/src/Columns/IColumn.cpp +++ b/src/Columns/IColumn.cpp @@ -109,7 +109,7 @@ void IColumn::assertTypeEquality(const IColumn & rhs) const if (typeid(*this) != typeid(rhs)) LOG_DEBUG(&Poco::Logger::get("IColumn"), "typeid(*this) = {}, typeid(rhs) = {}", typeid(*this).name(), typeid(rhs).name()); - chassert(isSparse() || typeid(*this) == typeid(rhs)); + chassert(isColumnConst(*this) || isSparse() || typeid(*this) == typeid(rhs)); } bool isColumnNullable(const IColumn & column) From 55f363d5f06968c8eb3316e8176f2d1a5071c255 Mon Sep 17 00:00:00 2001 From: Nikita Taranov Date: Thu, 27 Jun 2024 23:59:53 +0100 Subject: [PATCH 313/417] better --- src/Columns/IColumn.cpp | 5 +---- 1 file changed, 1 insertion(+), 4 deletions(-) diff --git a/src/Columns/IColumn.cpp b/src/Columns/IColumn.cpp index d054d3ad8d4..6d4d4a73df9 100644 --- a/src/Columns/IColumn.cpp +++ b/src/Columns/IColumn.cpp @@ -106,10 +106,7 @@ void IColumn::forEachSubcolumnRecursively(RecursiveColumnCallback callback) cons void IColumn::assertTypeEquality(const IColumn & rhs) const { - if (typeid(*this) != typeid(rhs)) - LOG_DEBUG(&Poco::Logger::get("IColumn"), "typeid(*this) = {}, typeid(rhs) = {}", typeid(*this).name(), typeid(rhs).name()); - - chassert(isColumnConst(*this) || isSparse() || typeid(*this) == typeid(rhs)); + chassert(((isColumnConst(*this) || isSparse()) && getDataType() == rhs.getDataType()) || typeid(*this) == typeid(rhs)); } bool isColumnNullable(const IColumn & column) From 7dc3ae381fe7525d9f1a9ce2d8a33ef09053eb7e Mon Sep 17 00:00:00 2001 From: Nikita Taranov Date: Fri, 28 Jun 2024 12:09:48 +0100 Subject: [PATCH 314/417] better --- src/Columns/ColumnConst.h | 2 ++ src/Columns/IColumn.cpp | 5 ----- src/Columns/IColumn.h | 14 +++++++++----- 3 files changed, 11 insertions(+), 10 deletions(-) diff --git a/src/Columns/ColumnConst.h b/src/Columns/ColumnConst.h index e782d06a6ca..98f9c8f4266 100644 --- a/src/Columns/ColumnConst.h +++ b/src/Columns/ColumnConst.h @@ -32,6 +32,8 @@ private: ColumnConst(const ColumnConst & src) = default; public: + bool isConst() const override { return true; } + ColumnPtr convertToFullColumn() const; ColumnPtr convertToFullColumnIfConst() const override diff --git a/src/Columns/IColumn.cpp b/src/Columns/IColumn.cpp index 6d4d4a73df9..1b4bd8f6e62 100644 --- a/src/Columns/IColumn.cpp +++ b/src/Columns/IColumn.cpp @@ -104,11 +104,6 @@ void IColumn::forEachSubcolumnRecursively(RecursiveColumnCallback callback) cons }); } -void IColumn::assertTypeEquality(const IColumn & rhs) const -{ - chassert(((isColumnConst(*this) || isSparse()) && getDataType() == rhs.getDataType()) || typeid(*this) == typeid(rhs)); -} - bool isColumnNullable(const IColumn & column) { return checkColumn(column); diff --git a/src/Columns/IColumn.h b/src/Columns/IColumn.h index a586214ab0f..b482a16a42e 100644 --- a/src/Columns/IColumn.h +++ b/src/Columns/IColumn.h @@ -9,10 +9,6 @@ #include "config.h" -#include -#include - - class SipHash; class Collator; @@ -625,6 +621,8 @@ public: [[nodiscard]] virtual bool isSparse() const { return false; } + [[nodiscard]] virtual bool isConst() const { return false; } + [[nodiscard]] virtual bool isCollationSupported() const { return false; } virtual ~IColumn() = default; @@ -662,7 +660,13 @@ protected: virtual int doCompareAt(size_t n, size_t m, const IColumn & rhs, int nan_direction_hint) const = 0; private: - void assertTypeEquality(const IColumn & rhs) const; + void assertTypeEquality(const IColumn & rhs) const + { + /// For Sparse and Const columns, we can compare only internal types. It is considered normal to e.g. insert from normal vector column to a sparse vector column. + /// This case is specifically handled in ColumnSparse implementation. Similar situation with Const column. + /// For the rest of column types we can compare the types directly. + chassert((isConst() || isSparse()) ? getDataType() == rhs.getDataType() : typeid(*this) == typeid(rhs)); + } }; using ColumnPtr = IColumn::Ptr; From 053228eea1bc1e733603f70448dca559dd580039 Mon Sep 17 00:00:00 2001 From: Konstantin Bogdanov Date: Wed, 3 Jul 2024 23:48:18 +0200 Subject: [PATCH 315/417] Empty user when JWT is set --- programs/client/Client.cpp | 1 + 1 file changed, 1 insertion(+) diff --git a/programs/client/Client.cpp b/programs/client/Client.cpp index 6343dc85d00..0126613f797 100644 --- a/programs/client/Client.cpp +++ b/programs/client/Client.cpp @@ -1117,6 +1117,7 @@ void Client::processOptions(const OptionsDescription & options_description, if (!options["user"].defaulted()) throw Exception(ErrorCodes::BAD_ARGUMENTS, "User and JWT flags can't be specified together"); config().setString("jwt", options["jwt"].as()); + config().setString("user", ""); } if (options.count("accept-invalid-certificate")) { From d320f0f3f29e685b6f7e2a3ed2fcac6ee457a3cf Mon Sep 17 00:00:00 2001 From: Konstantin Bogdanov Date: Wed, 3 Jul 2024 23:48:30 +0200 Subject: [PATCH 316/417] Typo --- src/Core/Protocol.h | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/src/Core/Protocol.h b/src/Core/Protocol.h index 4c0848c0706..2e5b91e9b1b 100644 --- a/src/Core/Protocol.h +++ b/src/Core/Protocol.h @@ -63,7 +63,7 @@ const char USER_INTERSERVER_MARKER[] = " INTERSERVER SECRET "; /// Marker for SSH-keys-based authentication (passed as the user name) const char SSH_KEY_AUTHENTICAION_MARKER[] = " SSH KEY AUTHENTICATION "; -/// Market for JSON Web Token authentication +/// Marker for JSON Web Token authentication const char JWT_AUTHENTICAION_MARKER[] = " JWT AUTHENTICATION "; }; From 9ba10ca604ad6705ad46a60b9d03569c4729afcc Mon Sep 17 00:00:00 2001 From: Azat Khuzhin Date: Sun, 30 Jun 2024 17:22:42 +0200 Subject: [PATCH 317/417] Remove mysqlxx::Pool::Entry assignment operator v2: fix tidy https://s3.amazonaws.com/clickhouse-builds/PRs/65920/86789491be1a945602f6ebf0b3b93bf5272e52ab/binary_tidy/build_log.log Signed-off-by: Azat Khuzhin --- src/Common/mysqlxx/Pool.cpp | 1 - src/Common/mysqlxx/mysqlxx/Pool.h | 11 ----------- src/Common/mysqlxx/tests/mysqlxx_pool_test.cpp | 4 +--- src/Databases/MySQL/MaterializedMySQLSyncThread.cpp | 12 +++++++----- 4 files changed, 8 insertions(+), 20 deletions(-) diff --git a/src/Common/mysqlxx/Pool.cpp b/src/Common/mysqlxx/Pool.cpp index cc5b18214c8..546e9e91dc7 100644 --- a/src/Common/mysqlxx/Pool.cpp +++ b/src/Common/mysqlxx/Pool.cpp @@ -228,7 +228,6 @@ Pool::Entry Pool::tryGet() for (auto connection_it = connections.cbegin(); connection_it != connections.cend();) { Connection * connection_ptr = *connection_it; - /// Fixme: There is a race condition here b/c we do not synchronize with Pool::Entry's copy-assignment operator if (connection_ptr->ref_count == 0) { { diff --git a/src/Common/mysqlxx/mysqlxx/Pool.h b/src/Common/mysqlxx/mysqlxx/Pool.h index 6e509d8bdd6..f1ef81e28dd 100644 --- a/src/Common/mysqlxx/mysqlxx/Pool.h +++ b/src/Common/mysqlxx/mysqlxx/Pool.h @@ -64,17 +64,6 @@ public: decrementRefCount(); } - Entry & operator= (const Entry & src) /// NOLINT - { - pool = src.pool; - if (data) - decrementRefCount(); - data = src.data; - if (data) - incrementRefCount(); - return * this; - } - bool isNull() const { return data == nullptr; diff --git a/src/Common/mysqlxx/tests/mysqlxx_pool_test.cpp b/src/Common/mysqlxx/tests/mysqlxx_pool_test.cpp index 61d6a117285..121767edc84 100644 --- a/src/Common/mysqlxx/tests/mysqlxx_pool_test.cpp +++ b/src/Common/mysqlxx/tests/mysqlxx_pool_test.cpp @@ -13,13 +13,11 @@ mysqlxx::Pool::Entry getWithFailover(mysqlxx::Pool & connections_pool) constexpr size_t max_tries = 3; - mysqlxx::Pool::Entry worker_connection; - for (size_t try_no = 1; try_no <= max_tries; ++try_no) { try { - worker_connection = connections_pool.tryGet(); + mysqlxx::Pool::Entry worker_connection = connections_pool.tryGet(); if (!worker_connection.isNull()) { diff --git a/src/Databases/MySQL/MaterializedMySQLSyncThread.cpp b/src/Databases/MySQL/MaterializedMySQLSyncThread.cpp index 7ab4235feeb..27ebe0b6d21 100644 --- a/src/Databases/MySQL/MaterializedMySQLSyncThread.cpp +++ b/src/Databases/MySQL/MaterializedMySQLSyncThread.cpp @@ -29,6 +29,7 @@ #include #include #include +#include #include #include #include @@ -532,13 +533,17 @@ static inline void dumpDataForTables( bool MaterializedMySQLSyncThread::prepareSynchronized(MaterializeMetadata & metadata) { bool opened_transaction = false; - mysqlxx::PoolWithFailover::Entry connection; while (!isCancelled()) { try { - connection = pool.tryGet(); + mysqlxx::PoolWithFailover::Entry connection = pool.tryGet(); + SCOPE_EXIT({ + if (opened_transaction) + connection->query("ROLLBACK").execute(); + }); + if (connection.isNull()) { if (settings->max_wait_time_when_mysql_unavailable < 0) @@ -602,9 +607,6 @@ bool MaterializedMySQLSyncThread::prepareSynchronized(MaterializeMetadata & meta { tryLogCurrentException(log); - if (opened_transaction) - connection->query("ROLLBACK").execute(); - if (settings->max_wait_time_when_mysql_unavailable < 0) throw; From 24dc3b40209db9e1bc59186cc16c978a1b9d2a27 Mon Sep 17 00:00:00 2001 From: Nikita Taranov Date: Mon, 8 Jul 2024 16:57:00 +0100 Subject: [PATCH 318/417] insertRangeFrom --- src/Columns/ColumnAggregateFunction.cpp | 4 ++++ src/Columns/ColumnAggregateFunction.h | 4 ++++ src/Columns/ColumnArray.cpp | 4 ++++ src/Columns/ColumnArray.h | 4 ++++ src/Columns/ColumnCompressed.h | 4 ++++ src/Columns/ColumnConst.h | 4 ++++ src/Columns/ColumnDecimal.cpp | 4 ++++ src/Columns/ColumnDecimal.h | 4 ++++ src/Columns/ColumnDynamic.cpp | 4 ++++ src/Columns/ColumnDynamic.h | 4 ++++ src/Columns/ColumnFixedString.cpp | 4 ++++ src/Columns/ColumnFixedString.h | 4 ++++ src/Columns/ColumnFunction.cpp | 4 ++++ src/Columns/ColumnFunction.h | 4 ++++ src/Columns/ColumnLowCardinality.cpp | 4 ++++ src/Columns/ColumnLowCardinality.h | 4 ++++ src/Columns/ColumnMap.cpp | 4 ++++ src/Columns/ColumnMap.h | 4 ++++ src/Columns/ColumnNullable.cpp | 4 ++++ src/Columns/ColumnNullable.h | 4 ++++ src/Columns/ColumnObject.cpp | 4 ++++ src/Columns/ColumnObject.h | 4 ++++ src/Columns/ColumnSparse.cpp | 4 ++++ src/Columns/ColumnSparse.h | 4 ++++ src/Columns/ColumnString.cpp | 4 ++++ src/Columns/ColumnString.h | 4 ++++ src/Columns/ColumnTuple.cpp | 4 ++++ src/Columns/ColumnTuple.h | 4 ++++ src/Columns/ColumnVariant.cpp | 4 ++++ src/Columns/ColumnVariant.h | 4 ++++ src/Columns/ColumnVector.cpp | 4 ++++ src/Columns/ColumnVector.h | 4 ++++ src/Columns/IColumn.h | 6 ++++++ src/Columns/IColumnDummy.h | 4 ++++ src/Columns/IColumnUnique.h | 4 ++++ 35 files changed, 142 insertions(+) diff --git a/src/Columns/ColumnAggregateFunction.cpp b/src/Columns/ColumnAggregateFunction.cpp index be00f086ccf..7124a03c605 100644 --- a/src/Columns/ColumnAggregateFunction.cpp +++ b/src/Columns/ColumnAggregateFunction.cpp @@ -267,7 +267,11 @@ bool ColumnAggregateFunction::structureEquals(const IColumn & to) const } +#if !defined(ABORT_ON_LOGICAL_ERROR) +void ColumnAggregateFunction::insertRangeFrom(const IColumn & from, size_t start, size_t length) +#else void ColumnAggregateFunction::doInsertRangeFrom(const IColumn & from, size_t start, size_t length) +#endif { const ColumnAggregateFunction & from_concrete = assert_cast(from); diff --git a/src/Columns/ColumnAggregateFunction.h b/src/Columns/ColumnAggregateFunction.h index a34def52d08..b33fd7dfc47 100644 --- a/src/Columns/ColumnAggregateFunction.h +++ b/src/Columns/ColumnAggregateFunction.h @@ -184,7 +184,11 @@ public: void protect() override; +#if !defined(ABORT_ON_LOGICAL_ERROR) + void insertRangeFrom(const IColumn & from, size_t start, size_t length) override; +#else void doInsertRangeFrom(const IColumn & from, size_t start, size_t length) override; +#endif void popBack(size_t n) override; diff --git a/src/Columns/ColumnArray.cpp b/src/Columns/ColumnArray.cpp index 756fce07f8e..bf89341d5a0 100644 --- a/src/Columns/ColumnArray.cpp +++ b/src/Columns/ColumnArray.cpp @@ -535,7 +535,11 @@ void ColumnArray::getExtremes(Field & min, Field & max) const } +#if !defined(ABORT_ON_LOGICAL_ERROR) +void ColumnArray::insertRangeFrom(const IColumn & src, size_t start, size_t length) +#else void ColumnArray::doInsertRangeFrom(const IColumn & src, size_t start, size_t length) +#endif { if (length == 0) return; diff --git a/src/Columns/ColumnArray.h b/src/Columns/ColumnArray.h index b609c64598c..bdadbd18a49 100644 --- a/src/Columns/ColumnArray.h +++ b/src/Columns/ColumnArray.h @@ -84,7 +84,11 @@ public: void updateHashWithValue(size_t n, SipHash & hash) const override; void updateWeakHash32(WeakHash32 & hash) const override; void updateHashFast(SipHash & hash) const override; +#if !defined(ABORT_ON_LOGICAL_ERROR) + void insertRangeFrom(const IColumn & src, size_t start, size_t length) override; +#else void doInsertRangeFrom(const IColumn & src, size_t start, size_t length) override; +#endif void insert(const Field & x) override; bool tryInsert(const Field & x) override; void doInsertFrom(const IColumn & src_, size_t n) override; diff --git a/src/Columns/ColumnCompressed.h b/src/Columns/ColumnCompressed.h index f6e56b15d30..d0f4c2c5910 100644 --- a/src/Columns/ColumnCompressed.h +++ b/src/Columns/ColumnCompressed.h @@ -85,7 +85,11 @@ public: bool isDefaultAt(size_t) const override { throwMustBeDecompressed(); } void insert(const Field &) override { throwMustBeDecompressed(); } bool tryInsert(const Field &) override { throwMustBeDecompressed(); } +#if !defined(ABORT_ON_LOGICAL_ERROR) + void insertRangeFrom(const IColumn &, size_t, size_t) override { throwMustBeDecompressed(); } +#else void doInsertRangeFrom(const IColumn &, size_t, size_t) override { throwMustBeDecompressed(); } +#endif void insertData(const char *, size_t) override { throwMustBeDecompressed(); } void insertDefault() override { throwMustBeDecompressed(); } void popBack(size_t) override { throwMustBeDecompressed(); } diff --git a/src/Columns/ColumnConst.h b/src/Columns/ColumnConst.h index 98f9c8f4266..df51991e22f 100644 --- a/src/Columns/ColumnConst.h +++ b/src/Columns/ColumnConst.h @@ -123,7 +123,11 @@ public: return data->isNullAt(0); } +#if !defined(ABORT_ON_LOGICAL_ERROR) + void insertRangeFrom(const IColumn &, size_t /*start*/, size_t length) override +#else void doInsertRangeFrom(const IColumn &, size_t /*start*/, size_t length) override +#endif { s += length; } diff --git a/src/Columns/ColumnDecimal.cpp b/src/Columns/ColumnDecimal.cpp index ad17ccce778..b09d7b1ee90 100644 --- a/src/Columns/ColumnDecimal.cpp +++ b/src/Columns/ColumnDecimal.cpp @@ -331,7 +331,11 @@ void ColumnDecimal::insertData(const char * src, size_t /*length*/) } template +#if !defined(ABORT_ON_LOGICAL_ERROR) +void ColumnDecimal::insertRangeFrom(const IColumn & src, size_t start, size_t length) +#else void ColumnDecimal::doInsertRangeFrom(const IColumn & src, size_t start, size_t length) +#endif { const ColumnDecimal & src_vec = assert_cast(src); diff --git a/src/Columns/ColumnDecimal.h b/src/Columns/ColumnDecimal.h index 019ce442bcc..af2e9286a24 100644 --- a/src/Columns/ColumnDecimal.h +++ b/src/Columns/ColumnDecimal.h @@ -68,7 +68,11 @@ public: void insertManyDefaults(size_t length) override { data.resize_fill(data.size() + length); } void insert(const Field & x) override { data.push_back(x.get()); } bool tryInsert(const Field & x) override; +#if !defined(ABORT_ON_LOGICAL_ERROR) + void insertRangeFrom(const IColumn & src, size_t start, size_t length) override; +#else void doInsertRangeFrom(const IColumn & src, size_t start, size_t length) override; +#endif void popBack(size_t n) override { diff --git a/src/Columns/ColumnDynamic.cpp b/src/Columns/ColumnDynamic.cpp index 63ddd5e566c..c80cb79384e 100644 --- a/src/Columns/ColumnDynamic.cpp +++ b/src/Columns/ColumnDynamic.cpp @@ -263,7 +263,11 @@ void ColumnDynamic::doInsertFrom(const DB::IColumn & src_, size_t n) variant_col.insertIntoVariantFrom(string_variant_discr, *tmp_string_column, 0); } +#if !defined(ABORT_ON_LOGICAL_ERROR) +void ColumnDynamic::insertRangeFrom(const DB::IColumn & src_, size_t start, size_t length) +#else void ColumnDynamic::doInsertRangeFrom(const DB::IColumn & src_, size_t start, size_t length) +#endif { if (start + length > src_.size()) throw Exception(ErrorCodes::PARAMETER_OUT_OF_BOUND, "Parameter out of bound in ColumnDynamic::insertRangeFrom method. " diff --git a/src/Columns/ColumnDynamic.h b/src/Columns/ColumnDynamic.h index 7156cd49678..59e889a3644 100644 --- a/src/Columns/ColumnDynamic.h +++ b/src/Columns/ColumnDynamic.h @@ -143,7 +143,11 @@ public: void insert(const Field & x) override; bool tryInsert(const Field & x) override; void doInsertFrom(const IColumn & src_, size_t n) override; +#if !defined(ABORT_ON_LOGICAL_ERROR) + void insertRangeFrom(const IColumn & src, size_t start, size_t length) override; +#else void doInsertRangeFrom(const IColumn & src, size_t start, size_t length) override; +#endif void doInsertManyFrom(const IColumn & src, size_t position, size_t length) override; void insertDefault() override diff --git a/src/Columns/ColumnFixedString.cpp b/src/Columns/ColumnFixedString.cpp index 081a46b78d9..ed0f1301a07 100644 --- a/src/Columns/ColumnFixedString.cpp +++ b/src/Columns/ColumnFixedString.cpp @@ -219,7 +219,11 @@ size_t ColumnFixedString::estimateCardinalityInPermutedRange(const Permutation & return elements.size(); } +#if !defined(ABORT_ON_LOGICAL_ERROR) +void ColumnFixedString::insertRangeFrom(const IColumn & src, size_t start, size_t length) +#else void ColumnFixedString::doInsertRangeFrom(const IColumn & src, size_t start, size_t length) +#endif { const ColumnFixedString & src_concrete = assert_cast(src); chassert(this->n == src_concrete.n); diff --git a/src/Columns/ColumnFixedString.h b/src/Columns/ColumnFixedString.h index 47df6bec4cc..cc99634adf8 100644 --- a/src/Columns/ColumnFixedString.h +++ b/src/Columns/ColumnFixedString.h @@ -144,7 +144,11 @@ public: size_t estimateCardinalityInPermutedRange(const Permutation & permutation, const EqualRange & equal_range) const override; +#if !defined(ABORT_ON_LOGICAL_ERROR) + void insertRangeFrom(const IColumn & src, size_t start, size_t length) override; +#else void doInsertRangeFrom(const IColumn & src, size_t start, size_t length) override; +#endif ColumnPtr filter(const IColumn::Filter & filt, ssize_t result_size_hint) const override; diff --git a/src/Columns/ColumnFunction.cpp b/src/Columns/ColumnFunction.cpp index f40b951ef13..4f19cf87fc6 100644 --- a/src/Columns/ColumnFunction.cpp +++ b/src/Columns/ColumnFunction.cpp @@ -89,7 +89,11 @@ void ColumnFunction::doInsertFrom(const IColumn & src, size_t n) ++elements_size; } +#if !defined(ABORT_ON_LOGICAL_ERROR) +void ColumnFunction::insertRangeFrom(const IColumn & src, size_t start, size_t length) +#else void ColumnFunction::doInsertRangeFrom(const IColumn & src, size_t start, size_t length) +#endif { const ColumnFunction & src_func = assert_cast(src); diff --git a/src/Columns/ColumnFunction.h b/src/Columns/ColumnFunction.h index ab5a63d1afe..b5b0c53feb5 100644 --- a/src/Columns/ColumnFunction.h +++ b/src/Columns/ColumnFunction.h @@ -95,7 +95,11 @@ public: } void doInsertFrom(const IColumn & src, size_t n) override; +#if !defined(ABORT_ON_LOGICAL_ERROR) + void insertRangeFrom(const IColumn &, size_t start, size_t length) override; +#else void doInsertRangeFrom(const IColumn &, size_t start, size_t length) override; +#endif void insertData(const char *, size_t) override { diff --git a/src/Columns/ColumnLowCardinality.cpp b/src/Columns/ColumnLowCardinality.cpp index a2d4903284e..cf43640a669 100644 --- a/src/Columns/ColumnLowCardinality.cpp +++ b/src/Columns/ColumnLowCardinality.cpp @@ -187,7 +187,11 @@ void ColumnLowCardinality::insertFromFullColumn(const IColumn & src, size_t n) idx.insertPosition(getDictionary().uniqueInsertFrom(src, n)); } +#if !defined(ABORT_ON_LOGICAL_ERROR) +void ColumnLowCardinality::insertRangeFrom(const IColumn & src, size_t start, size_t length) +#else void ColumnLowCardinality::doInsertRangeFrom(const IColumn & src, size_t start, size_t length) +#endif { const auto * low_cardinality_src = typeid_cast(&src); diff --git a/src/Columns/ColumnLowCardinality.h b/src/Columns/ColumnLowCardinality.h index 43984cbcf8a..ef0df669729 100644 --- a/src/Columns/ColumnLowCardinality.h +++ b/src/Columns/ColumnLowCardinality.h @@ -81,7 +81,11 @@ public: void doInsertFrom(const IColumn & src, size_t n) override; void insertFromFullColumn(const IColumn & src, size_t n); +#if !defined(ABORT_ON_LOGICAL_ERROR) + void insertRangeFrom(const IColumn & src, size_t start, size_t length) override; +#else void doInsertRangeFrom(const IColumn & src, size_t start, size_t length) override; +#endif void insertRangeFromFullColumn(const IColumn & src, size_t start, size_t length); void insertRangeFromDictionaryEncodedColumn(const IColumn & keys, const IColumn & positions); diff --git a/src/Columns/ColumnMap.cpp b/src/Columns/ColumnMap.cpp index 5b4cd04d67a..072df6a91c5 100644 --- a/src/Columns/ColumnMap.cpp +++ b/src/Columns/ColumnMap.cpp @@ -163,7 +163,11 @@ void ColumnMap::doInsertManyFrom(const IColumn & src, size_t position, size_t le assert_cast(*nested).insertManyFrom(assert_cast(src).getNestedColumn(), position, length); } +#if !defined(ABORT_ON_LOGICAL_ERROR) +void ColumnMap::insertRangeFrom(const IColumn & src, size_t start, size_t length) +#else void ColumnMap::doInsertRangeFrom(const IColumn & src, size_t start, size_t length) +#endif { nested->insertRangeFrom( assert_cast(src).getNestedColumn(), diff --git a/src/Columns/ColumnMap.h b/src/Columns/ColumnMap.h index 9834b2ed0ee..54370c15650 100644 --- a/src/Columns/ColumnMap.h +++ b/src/Columns/ColumnMap.h @@ -68,7 +68,11 @@ public: void updateHashFast(SipHash & hash) const override; void doInsertFrom(const IColumn & src_, size_t n) override; void doInsertManyFrom(const IColumn & src, size_t position, size_t length) override; +#if !defined(ABORT_ON_LOGICAL_ERROR) + void insertRangeFrom(const IColumn & src, size_t start, size_t length) override; +#else void doInsertRangeFrom(const IColumn & src, size_t start, size_t length) override; +#endif ColumnPtr filter(const Filter & filt, ssize_t result_size_hint) const override; void expand(const Filter & mask, bool inverted) override; ColumnPtr permute(const Permutation & perm, size_t limit) const override; diff --git a/src/Columns/ColumnNullable.cpp b/src/Columns/ColumnNullable.cpp index 8e57204721f..6268c802be7 100644 --- a/src/Columns/ColumnNullable.cpp +++ b/src/Columns/ColumnNullable.cpp @@ -221,7 +221,11 @@ const char * ColumnNullable::skipSerializedInArena(const char * pos) const return pos; } +#if !defined(ABORT_ON_LOGICAL_ERROR) +void ColumnNullable::insertRangeFrom(const IColumn & src, size_t start, size_t length) +#else void ColumnNullable::doInsertRangeFrom(const IColumn & src, size_t start, size_t length) +#endif { const ColumnNullable & nullable_col = assert_cast(src); getNullMapColumn().insertRangeFrom(*nullable_col.null_map, start, length); diff --git a/src/Columns/ColumnNullable.h b/src/Columns/ColumnNullable.h index 1e5866b8abb..45104577d12 100644 --- a/src/Columns/ColumnNullable.h +++ b/src/Columns/ColumnNullable.h @@ -69,7 +69,11 @@ public: char * serializeValueIntoMemory(size_t n, char * memory) const override; const char * deserializeAndInsertFromArena(const char * pos) override; const char * skipSerializedInArena(const char * pos) const override; +#if !defined(ABORT_ON_LOGICAL_ERROR) + void insertRangeFrom(const IColumn & src, size_t start, size_t length) override; +#else void doInsertRangeFrom(const IColumn & src, size_t start, size_t length) override; +#endif void insert(const Field & x) override; bool tryInsert(const Field & x) override; void doInsertFrom(const IColumn & src, size_t n) override; diff --git a/src/Columns/ColumnObject.cpp b/src/Columns/ColumnObject.cpp index 86e94d9397a..69dbf780eeb 100644 --- a/src/Columns/ColumnObject.cpp +++ b/src/Columns/ColumnObject.cpp @@ -768,7 +768,11 @@ void ColumnObject::doInsertFrom(const IColumn & src, size_t n) insert(src[n]); } +#if !defined(ABORT_ON_LOGICAL_ERROR) +void ColumnObject::insertRangeFrom(const IColumn & src, size_t start, size_t length) +#else void ColumnObject::doInsertRangeFrom(const IColumn & src, size_t start, size_t length) +#endif { const auto & src_object = assert_cast(src); diff --git a/src/Columns/ColumnObject.h b/src/Columns/ColumnObject.h index 2fe5adb70b3..eb8ac2874c2 100644 --- a/src/Columns/ColumnObject.h +++ b/src/Columns/ColumnObject.h @@ -210,7 +210,11 @@ public: bool tryInsert(const Field & field) override; void insertDefault() override; void doInsertFrom(const IColumn & src, size_t n) override; +#if !defined(ABORT_ON_LOGICAL_ERROR) + void insertRangeFrom(const IColumn & src, size_t start, size_t length) override; +#else void doInsertRangeFrom(const IColumn & src, size_t start, size_t length) override; +#endif void popBack(size_t length) override; Field operator[](size_t n) const override; void get(size_t n, Field & res) const override; diff --git a/src/Columns/ColumnSparse.cpp b/src/Columns/ColumnSparse.cpp index 2100ac0cc03..937d016c55c 100644 --- a/src/Columns/ColumnSparse.cpp +++ b/src/Columns/ColumnSparse.cpp @@ -174,7 +174,11 @@ const char * ColumnSparse::skipSerializedInArena(const char * pos) const return values->skipSerializedInArena(pos); } +#if !defined(ABORT_ON_LOGICAL_ERROR) +void ColumnSparse::insertRangeFrom(const IColumn & src, size_t start, size_t length) +#else void ColumnSparse::doInsertRangeFrom(const IColumn & src, size_t start, size_t length) +#endif { if (length == 0) return; diff --git a/src/Columns/ColumnSparse.h b/src/Columns/ColumnSparse.h index 85040255e2e..73bcdb78984 100644 --- a/src/Columns/ColumnSparse.h +++ b/src/Columns/ColumnSparse.h @@ -81,7 +81,11 @@ public: char * serializeValueIntoMemory(size_t n, char * memory) const override; const char * deserializeAndInsertFromArena(const char * pos) override; const char * skipSerializedInArena(const char *) const override; +#if !defined(ABORT_ON_LOGICAL_ERROR) + void insertRangeFrom(const IColumn & src, size_t start, size_t length) override; +#else void doInsertRangeFrom(const IColumn & src, size_t start, size_t length) override; +#endif void insert(const Field & x) override; bool tryInsert(const Field & x) override; void doInsertFrom(const IColumn & src, size_t n) override; diff --git a/src/Columns/ColumnString.cpp b/src/Columns/ColumnString.cpp index b93488ab1be..24ef2f26b95 100644 --- a/src/Columns/ColumnString.cpp +++ b/src/Columns/ColumnString.cpp @@ -129,7 +129,11 @@ void ColumnString::updateWeakHash32(WeakHash32 & hash) const } +#if !defined(ABORT_ON_LOGICAL_ERROR) +void ColumnString::insertRangeFrom(const IColumn & src, size_t start, size_t length) +#else void ColumnString::doInsertRangeFrom(const IColumn & src, size_t start, size_t length) +#endif { if (length == 0) return; diff --git a/src/Columns/ColumnString.h b/src/Columns/ColumnString.h index c0f2d4c6e50..69b2c71e107 100644 --- a/src/Columns/ColumnString.h +++ b/src/Columns/ColumnString.h @@ -212,7 +212,11 @@ public: hash.update(reinterpret_cast(chars.data()), chars.size() * sizeof(chars[0])); } +#if !defined(ABORT_ON_LOGICAL_ERROR) + void insertRangeFrom(const IColumn & src, size_t start, size_t length) override; +#else void doInsertRangeFrom(const IColumn & src, size_t start, size_t length) override; +#endif ColumnPtr filter(const Filter & filt, ssize_t result_size_hint) const override; diff --git a/src/Columns/ColumnTuple.cpp b/src/Columns/ColumnTuple.cpp index fd79afaea96..9d99ce9837a 100644 --- a/src/Columns/ColumnTuple.cpp +++ b/src/Columns/ColumnTuple.cpp @@ -318,7 +318,11 @@ void ColumnTuple::updateHashFast(SipHash & hash) const column->updateHashFast(hash); } +#if !defined(ABORT_ON_LOGICAL_ERROR) +void ColumnTuple::insertRangeFrom(const IColumn & src, size_t start, size_t length) +#else void ColumnTuple::doInsertRangeFrom(const IColumn & src, size_t start, size_t length) +#endif { column_length += length; const size_t tuple_size = columns.size(); diff --git a/src/Columns/ColumnTuple.h b/src/Columns/ColumnTuple.h index 78ecb75862e..1536dfcaa0b 100644 --- a/src/Columns/ColumnTuple.h +++ b/src/Columns/ColumnTuple.h @@ -76,7 +76,11 @@ public: void updateHashWithValue(size_t n, SipHash & hash) const override; void updateWeakHash32(WeakHash32 & hash) const override; void updateHashFast(SipHash & hash) const override; +#if !defined(ABORT_ON_LOGICAL_ERROR) + void insertRangeFrom(const IColumn & src, size_t start, size_t length) override; +#else void doInsertRangeFrom(const IColumn & src, size_t start, size_t length) override; +#endif ColumnPtr filter(const Filter & filt, ssize_t result_size_hint) const override; void expand(const Filter & mask, bool inverted) override; ColumnPtr permute(const Permutation & perm, size_t limit) const override; diff --git a/src/Columns/ColumnVariant.cpp b/src/Columns/ColumnVariant.cpp index 137a396d3c4..d9803846ca2 100644 --- a/src/Columns/ColumnVariant.cpp +++ b/src/Columns/ColumnVariant.cpp @@ -600,7 +600,11 @@ void ColumnVariant::doInsertFrom(const IColumn & src_, size_t n) insertFromImpl(src_, n, nullptr); } +#if !defined(ABORT_ON_LOGICAL_ERROR) +void ColumnVariant::insertRangeFrom(const IColumn & src_, size_t start, size_t length) +#else void ColumnVariant::doInsertRangeFrom(const IColumn & src_, size_t start, size_t length) +#endif { insertRangeFromImpl(src_, start, length, nullptr); } diff --git a/src/Columns/ColumnVariant.h b/src/Columns/ColumnVariant.h index fe8cf20e20d..cfb6d254cec 100644 --- a/src/Columns/ColumnVariant.h +++ b/src/Columns/ColumnVariant.h @@ -181,7 +181,11 @@ public: bool tryInsert(const Field & x) override; void doInsertFrom(const IColumn & src_, size_t n) override; +#if !defined(ABORT_ON_LOGICAL_ERROR) + void insertRangeFrom(const IColumn & src_, size_t start, size_t length) override; +#else void doInsertRangeFrom(const IColumn & src_, size_t start, size_t length) override; +#endif void doInsertManyFrom(const IColumn & src_, size_t position, size_t length) override; using IColumn::insertFrom; diff --git a/src/Columns/ColumnVector.cpp b/src/Columns/ColumnVector.cpp index f46d244e9d9..19d1b800961 100644 --- a/src/Columns/ColumnVector.cpp +++ b/src/Columns/ColumnVector.cpp @@ -503,7 +503,11 @@ bool ColumnVector::tryInsert(const DB::Field & x) } template +#if !defined(ABORT_ON_LOGICAL_ERROR) +void ColumnVector::insertRangeFrom(const IColumn & src, size_t start, size_t length) +#else void ColumnVector::doInsertRangeFrom(const IColumn & src, size_t start, size_t length) +#endif { const ColumnVector & src_vec = assert_cast(src); diff --git a/src/Columns/ColumnVector.h b/src/Columns/ColumnVector.h index f0bc8257a3f..f078386cfcd 100644 --- a/src/Columns/ColumnVector.h +++ b/src/Columns/ColumnVector.h @@ -228,7 +228,11 @@ public: bool tryInsert(const DB::Field & x) override; +#if !defined(ABORT_ON_LOGICAL_ERROR) + void insertRangeFrom(const IColumn & src, size_t start, size_t length) override; +#else void doInsertRangeFrom(const IColumn & src, size_t start, size_t length) override; +#endif ColumnPtr filter(const IColumn::Filter & filt, ssize_t result_size_hint) const override; diff --git a/src/Columns/IColumn.h b/src/Columns/IColumn.h index b482a16a42e..5b1de8a1f39 100644 --- a/src/Columns/IColumn.h +++ b/src/Columns/IColumn.h @@ -187,11 +187,15 @@ public: /// Appends range of elements from other column with the same type. /// Could be used to concatenate columns. +#if !defined(ABORT_ON_LOGICAL_ERROR) + virtual void insertRangeFrom(const IColumn & src, size_t start, size_t length) = 0; +#else void insertRangeFrom(const IColumn & src, size_t start, size_t length) { assertTypeEquality(src); doInsertRangeFrom(src, start, length); } +#endif /// Appends one element from other column with the same type multiple times. void insertManyFrom(const IColumn & src, size_t position, size_t length) @@ -649,7 +653,9 @@ protected: virtual void doInsertFrom(const IColumn & src, size_t n); +#if defined(ABORT_ON_LOGICAL_ERROR) virtual void doInsertRangeFrom(const IColumn & src, size_t start, size_t length) = 0; +#endif virtual void doInsertManyFrom(const IColumn & src, size_t position, size_t length) { diff --git a/src/Columns/IColumnDummy.h b/src/Columns/IColumnDummy.h index f6a1109d0a2..023cbbc463a 100644 --- a/src/Columns/IColumnDummy.h +++ b/src/Columns/IColumnDummy.h @@ -72,7 +72,11 @@ public: ++s; } +#if !defined(ABORT_ON_LOGICAL_ERROR) + void insertRangeFrom(const IColumn & /*src*/, size_t /*start*/, size_t length) override +#else void doInsertRangeFrom(const IColumn & /*src*/, size_t /*start*/, size_t length) override +#endif { s += length; } diff --git a/src/Columns/IColumnUnique.h b/src/Columns/IColumnUnique.h index 3ced489c04d..3398452b7ee 100644 --- a/src/Columns/IColumnUnique.h +++ b/src/Columns/IColumnUnique.h @@ -85,7 +85,11 @@ public: throw Exception(ErrorCodes::NOT_IMPLEMENTED, "Method tryInsert is not supported for ColumnUnique."); } +#if !defined(ABORT_ON_LOGICAL_ERROR) + void insertRangeFrom(const IColumn &, size_t, size_t) override +#else void doInsertRangeFrom(const IColumn &, size_t, size_t) override +#endif { throw Exception(ErrorCodes::NOT_IMPLEMENTED, "Method insertRangeFrom is not supported for ColumnUnique."); } From 3c0ccb0e294526bc25c20caf5132dee6b71a27c9 Mon Sep 17 00:00:00 2001 From: Amos Bird Date: Tue, 9 Jul 2024 00:03:25 +0800 Subject: [PATCH 319/417] Use a better placeholder name --- .../Formats/Impl/ConstantExpressionTemplate.cpp | 7 ++++++- .../Formats/Impl/ValuesBlockInputFormat.h | 13 +------------ .../02890_named_tuple_functions.reference | 1 + .../0_stateless/02890_named_tuple_functions.sql | 9 +++++++++ 4 files changed, 17 insertions(+), 13 deletions(-) diff --git a/src/Processors/Formats/Impl/ConstantExpressionTemplate.cpp b/src/Processors/Formats/Impl/ConstantExpressionTemplate.cpp index 9d056b42101..fe82d1b1c53 100644 --- a/src/Processors/Formats/Impl/ConstantExpressionTemplate.cpp +++ b/src/Processors/Formats/Impl/ConstantExpressionTemplate.cpp @@ -227,7 +227,12 @@ private: return true; } - String column_name = "_dummy_" + std::to_string(replaced_literals.size()); + /// When generating placeholder names, ensure that we use names + /// requiring quotes to be valid identifiers. This prevents the + /// tuple() function from generating named tuples. Otherwise, + /// inserting named tuples with different names into another named + /// tuple will result in only default values being inserted. + String column_name = "-dummy-" + std::to_string(replaced_literals.size()); replaced_literals.emplace_back(literal, column_name, force_nullable); setDataType(replaced_literals.back()); ast = std::make_shared(column_name); diff --git a/src/Processors/Formats/Impl/ValuesBlockInputFormat.h b/src/Processors/Formats/Impl/ValuesBlockInputFormat.h index b1bce098e99..0abafc896ff 100644 --- a/src/Processors/Formats/Impl/ValuesBlockInputFormat.h +++ b/src/Processors/Formats/Impl/ValuesBlockInputFormat.h @@ -37,18 +37,7 @@ public: void resetReadBuffer() override; /// TODO: remove context somehow. - void setContext(const ContextPtr & context_) - { - auto context_copy = Context::createCopy(context_); - - /// ConstantExpressionTemplate generates placeholder names (_dummy_N) - /// for all literals, which are valid names for creating named tuples. - /// This behavior needs to be explicitly disabled, because if named - /// tuples with different names are inserted into a named tuple, it will - /// only insert default values. - context_copy->setSetting("enable_named_columns_in_function_tuple", false); - context = context_copy; - } + void setContext(const ContextPtr & context_) { context = Context::createCopy(context_); } const BlockMissingValues & getMissingValues() const override { return block_missing_values; } diff --git a/tests/queries/0_stateless/02890_named_tuple_functions.reference b/tests/queries/0_stateless/02890_named_tuple_functions.reference index efccfef0817..f7a0c440b5a 100644 --- a/tests/queries/0_stateless/02890_named_tuple_functions.reference +++ b/tests/queries/0_stateless/02890_named_tuple_functions.reference @@ -6,3 +6,4 @@ Tuple(\n k UInt8,\n j Int32) ['k','j'] Tuple(Int32, Int32, Int32, Int32) ['1','2','3','4'] +(1,2,3) diff --git a/tests/queries/0_stateless/02890_named_tuple_functions.sql b/tests/queries/0_stateless/02890_named_tuple_functions.sql index abd24e1cbfe..8e0c9c2b10e 100644 --- a/tests/queries/0_stateless/02890_named_tuple_functions.sql +++ b/tests/queries/0_stateless/02890_named_tuple_functions.sql @@ -20,3 +20,12 @@ select tupleNames(tuple(i, i, j, j)) from x; select tupleNames(1); -- { serverError 43 } drop table x; + +drop table if exists tbl; + +-- Make sure named tuple won't break Values insert +create table tbl (x Tuple(a Int32, b Int32, c Int32)) engine MergeTree order by (); +insert into tbl values (tuple(1, 2, 3)); -- without tuple it's interpreted differently inside values block. +select * from tbl; + +drop table tbl From 3be0ab5c4b6ab964e0e97269dd5e265bff1b064a Mon Sep 17 00:00:00 2001 From: Nikita Taranov Date: Mon, 8 Jul 2024 17:05:55 +0100 Subject: [PATCH 320/417] insertManyFrom --- src/Columns/ColumnConst.h | 4 ++++ src/Columns/ColumnDecimal.h | 4 ++++ src/Columns/ColumnDynamic.cpp | 4 ++++ src/Columns/ColumnDynamic.h | 4 ++++ src/Columns/ColumnFixedString.cpp | 4 ++++ src/Columns/ColumnFixedString.h | 4 ++++ src/Columns/ColumnMap.cpp | 4 ++++ src/Columns/ColumnMap.h | 4 ++++ src/Columns/ColumnNullable.cpp | 4 ++++ src/Columns/ColumnNullable.h | 4 ++++ src/Columns/ColumnString.cpp | 4 ++++ src/Columns/ColumnString.h | 4 ++++ src/Columns/ColumnTuple.cpp | 4 ++++ src/Columns/ColumnTuple.h | 4 ++++ src/Columns/ColumnVariant.cpp | 4 ++++ src/Columns/ColumnVariant.h | 4 ++++ src/Columns/ColumnVector.h | 4 ++++ src/Columns/IColumn.h | 10 ++++++++++ .../benchmarks/benchmark_column_insert_many_from.cpp | 4 ++++ 19 files changed, 82 insertions(+) diff --git a/src/Columns/ColumnConst.h b/src/Columns/ColumnConst.h index df51991e22f..72c3d1f0f0e 100644 --- a/src/Columns/ColumnConst.h +++ b/src/Columns/ColumnConst.h @@ -156,7 +156,11 @@ public: ++s; } +#if !defined(ABORT_ON_LOGICAL_ERROR) + void insertManyFrom(const IColumn & /*src*/, size_t /* position */, size_t length) override { s += length; } +#else void doInsertManyFrom(const IColumn & /*src*/, size_t /* position */, size_t length) override { s += length; } +#endif void insertDefault() override { diff --git a/src/Columns/ColumnDecimal.h b/src/Columns/ColumnDecimal.h index af2e9286a24..27b5c765710 100644 --- a/src/Columns/ColumnDecimal.h +++ b/src/Columns/ColumnDecimal.h @@ -57,7 +57,11 @@ public: void doInsertFrom(const IColumn & src, size_t n) override { data.push_back(static_cast(src).getData()[n]); } +#if !defined(ABORT_ON_LOGICAL_ERROR) + void insertManyFrom(const IColumn & src, size_t position, size_t length) override +#else void doInsertManyFrom(const IColumn & src, size_t position, size_t length) override +#endif { ValueType v = assert_cast(src).getData()[position]; data.resize_fill(data.size() + length, v); diff --git a/src/Columns/ColumnDynamic.cpp b/src/Columns/ColumnDynamic.cpp index c80cb79384e..6cc3e27b7ff 100644 --- a/src/Columns/ColumnDynamic.cpp +++ b/src/Columns/ColumnDynamic.cpp @@ -433,7 +433,11 @@ void ColumnDynamic::doInsertRangeFrom(const DB::IColumn & src_, size_t start, si } } +#if !defined(ABORT_ON_LOGICAL_ERROR) +void ColumnDynamic::insertManyFrom(const DB::IColumn & src_, size_t position, size_t length) +#else void ColumnDynamic::doInsertManyFrom(const DB::IColumn & src_, size_t position, size_t length) +#endif { const auto & dynamic_src = assert_cast(src_); diff --git a/src/Columns/ColumnDynamic.h b/src/Columns/ColumnDynamic.h index 59e889a3644..e3fd944a6bc 100644 --- a/src/Columns/ColumnDynamic.h +++ b/src/Columns/ColumnDynamic.h @@ -148,7 +148,11 @@ public: #else void doInsertRangeFrom(const IColumn & src, size_t start, size_t length) override; #endif +#if !defined(ABORT_ON_LOGICAL_ERROR) + void insertManyFrom(const IColumn & src, size_t position, size_t length) override; +#else void doInsertManyFrom(const IColumn & src, size_t position, size_t length) override; +#endif void insertDefault() override { diff --git a/src/Columns/ColumnFixedString.cpp b/src/Columns/ColumnFixedString.cpp index ed0f1301a07..21435d405b3 100644 --- a/src/Columns/ColumnFixedString.cpp +++ b/src/Columns/ColumnFixedString.cpp @@ -86,7 +86,11 @@ void ColumnFixedString::doInsertFrom(const IColumn & src_, size_t index) memcpySmallAllowReadWriteOverflow15(chars.data() + old_size, &src.chars[n * index], n); } +#if !defined(ABORT_ON_LOGICAL_ERROR) +void ColumnFixedString::insertManyFrom(const IColumn & src, size_t position, size_t length) +#else void ColumnFixedString::doInsertManyFrom(const IColumn & src, size_t position, size_t length) +#endif { const ColumnFixedString & src_concrete = assert_cast(src); if (n != src_concrete.getN()) diff --git a/src/Columns/ColumnFixedString.h b/src/Columns/ColumnFixedString.h index cc99634adf8..faa63910c00 100644 --- a/src/Columns/ColumnFixedString.h +++ b/src/Columns/ColumnFixedString.h @@ -100,7 +100,11 @@ public: void doInsertFrom(const IColumn & src_, size_t index) override; +#if !defined(ABORT_ON_LOGICAL_ERROR) + void insertManyFrom(const IColumn & src, size_t position, size_t length) override; +#else void doInsertManyFrom(const IColumn & src, size_t position, size_t length) override; +#endif void insertData(const char * pos, size_t length) override; diff --git a/src/Columns/ColumnMap.cpp b/src/Columns/ColumnMap.cpp index 072df6a91c5..53f8a81e40f 100644 --- a/src/Columns/ColumnMap.cpp +++ b/src/Columns/ColumnMap.cpp @@ -158,7 +158,11 @@ void ColumnMap::doInsertFrom(const IColumn & src, size_t n) nested->insertFrom(assert_cast(src).getNestedColumn(), n); } +#if !defined(ABORT_ON_LOGICAL_ERROR) +void ColumnMap::insertManyFrom(const IColumn & src, size_t position, size_t length) +#else void ColumnMap::doInsertManyFrom(const IColumn & src, size_t position, size_t length) +#endif { assert_cast(*nested).insertManyFrom(assert_cast(src).getNestedColumn(), position, length); } diff --git a/src/Columns/ColumnMap.h b/src/Columns/ColumnMap.h index 54370c15650..05a32682e23 100644 --- a/src/Columns/ColumnMap.h +++ b/src/Columns/ColumnMap.h @@ -67,7 +67,11 @@ public: void updateWeakHash32(WeakHash32 & hash) const override; void updateHashFast(SipHash & hash) const override; void doInsertFrom(const IColumn & src_, size_t n) override; +#if !defined(ABORT_ON_LOGICAL_ERROR) + void insertManyFrom(const IColumn & src, size_t position, size_t length) override; +#else void doInsertManyFrom(const IColumn & src, size_t position, size_t length) override; +#endif #if !defined(ABORT_ON_LOGICAL_ERROR) void insertRangeFrom(const IColumn & src, size_t start, size_t length) override; #else diff --git a/src/Columns/ColumnNullable.cpp b/src/Columns/ColumnNullable.cpp index 6268c802be7..c24278d02c5 100644 --- a/src/Columns/ColumnNullable.cpp +++ b/src/Columns/ColumnNullable.cpp @@ -270,7 +270,11 @@ void ColumnNullable::doInsertFrom(const IColumn & src, size_t n) } +#if !defined(ABORT_ON_LOGICAL_ERROR) +void ColumnNullable::insertManyFrom(const IColumn & src, size_t position, size_t length) +#else void ColumnNullable::doInsertManyFrom(const IColumn & src, size_t position, size_t length) +#endif { const ColumnNullable & src_concrete = assert_cast(src); getNestedColumn().insertManyFrom(src_concrete.getNestedColumn(), position, length); diff --git a/src/Columns/ColumnNullable.h b/src/Columns/ColumnNullable.h index 45104577d12..47d007c6dcf 100644 --- a/src/Columns/ColumnNullable.h +++ b/src/Columns/ColumnNullable.h @@ -77,7 +77,11 @@ public: void insert(const Field & x) override; bool tryInsert(const Field & x) override; void doInsertFrom(const IColumn & src, size_t n) override; +#if !defined(ABORT_ON_LOGICAL_ERROR) + void insertManyFrom(const IColumn & src, size_t position, size_t length) override; +#else void doInsertManyFrom(const IColumn & src, size_t position, size_t length) override; +#endif void insertFromNotNullable(const IColumn & src, size_t n); void insertRangeFromNotNullable(const IColumn & src, size_t start, size_t length); diff --git a/src/Columns/ColumnString.cpp b/src/Columns/ColumnString.cpp index 24ef2f26b95..1eda9714d62 100644 --- a/src/Columns/ColumnString.cpp +++ b/src/Columns/ColumnString.cpp @@ -39,7 +39,11 @@ ColumnString::ColumnString(const ColumnString & src) last_offset, chars.size()); } +#if !defined(ABORT_ON_LOGICAL_ERROR) +void ColumnString::insertManyFrom(const IColumn & src, size_t position, size_t length) +#else void ColumnString::doInsertManyFrom(const IColumn & src, size_t position, size_t length) +#endif { const ColumnString & src_concrete = assert_cast(src); const UInt8 * src_buf = &src_concrete.chars[src_concrete.offsets[position - 1]]; diff --git a/src/Columns/ColumnString.h b/src/Columns/ColumnString.h index 69b2c71e107..6a1b4be0505 100644 --- a/src/Columns/ColumnString.h +++ b/src/Columns/ColumnString.h @@ -165,7 +165,11 @@ public: } } +#if !defined(ABORT_ON_LOGICAL_ERROR) + void insertManyFrom(const IColumn & src, size_t position, size_t length) override; +#else void doInsertManyFrom(const IColumn & src, size_t position, size_t length) override; +#endif void insertData(const char * pos, size_t length) override { diff --git a/src/Columns/ColumnTuple.cpp b/src/Columns/ColumnTuple.cpp index 9d99ce9837a..382d2182b61 100644 --- a/src/Columns/ColumnTuple.cpp +++ b/src/Columns/ColumnTuple.cpp @@ -218,7 +218,11 @@ void ColumnTuple::doInsertFrom(const IColumn & src_, size_t n) columns[i]->insertFrom(*src.columns[i], n); } +#if !defined(ABORT_ON_LOGICAL_ERROR) +void ColumnTuple::insertManyFrom(const IColumn & src, size_t position, size_t length) +#else void ColumnTuple::doInsertManyFrom(const IColumn & src, size_t position, size_t length) +#endif { const ColumnTuple & src_tuple = assert_cast(src); diff --git a/src/Columns/ColumnTuple.h b/src/Columns/ColumnTuple.h index 1536dfcaa0b..7ca065a8439 100644 --- a/src/Columns/ColumnTuple.h +++ b/src/Columns/ColumnTuple.h @@ -66,7 +66,11 @@ public: void insert(const Field & x) override; bool tryInsert(const Field & x) override; void doInsertFrom(const IColumn & src_, size_t n) override; +#if !defined(ABORT_ON_LOGICAL_ERROR) + void insertManyFrom(const IColumn & src, size_t position, size_t length) override; +#else void doInsertManyFrom(const IColumn & src, size_t position, size_t length) override; +#endif void insertDefault() override; void popBack(size_t n) override; StringRef serializeValueIntoArena(size_t n, Arena & arena, char const *& begin) const override; diff --git a/src/Columns/ColumnVariant.cpp b/src/Columns/ColumnVariant.cpp index d9803846ca2..c571913614d 100644 --- a/src/Columns/ColumnVariant.cpp +++ b/src/Columns/ColumnVariant.cpp @@ -609,7 +609,11 @@ void ColumnVariant::doInsertRangeFrom(const IColumn & src_, size_t start, size_t insertRangeFromImpl(src_, start, length, nullptr); } +#if !defined(ABORT_ON_LOGICAL_ERROR) +void ColumnVariant::insertManyFrom(const DB::IColumn & src_, size_t position, size_t length) +#else void ColumnVariant::doInsertManyFrom(const DB::IColumn & src_, size_t position, size_t length) +#endif { insertManyFromImpl(src_, position, length, nullptr); } diff --git a/src/Columns/ColumnVariant.h b/src/Columns/ColumnVariant.h index cfb6d254cec..6ec2529a48e 100644 --- a/src/Columns/ColumnVariant.h +++ b/src/Columns/ColumnVariant.h @@ -186,7 +186,11 @@ public: #else void doInsertRangeFrom(const IColumn & src_, size_t start, size_t length) override; #endif +#if !defined(ABORT_ON_LOGICAL_ERROR) + void insertManyFrom(const IColumn & src_, size_t position, size_t length) override; +#else void doInsertManyFrom(const IColumn & src_, size_t position, size_t length) override; +#endif using IColumn::insertFrom; using IColumn::insertManyFrom; diff --git a/src/Columns/ColumnVector.h b/src/Columns/ColumnVector.h index f078386cfcd..19dacdad775 100644 --- a/src/Columns/ColumnVector.h +++ b/src/Columns/ColumnVector.h @@ -69,7 +69,11 @@ public: data.push_back(assert_cast(src).getData()[n]); } +#if !defined(ABORT_ON_LOGICAL_ERROR) + void insertManyFrom(const IColumn & src, size_t position, size_t length) override +#else void doInsertManyFrom(const IColumn & src, size_t position, size_t length) override +#endif { ValueType v = assert_cast(src).getData()[position]; data.resize_fill(data.size() + length, v); diff --git a/src/Columns/IColumn.h b/src/Columns/IColumn.h index 5b1de8a1f39..e5a82d5b772 100644 --- a/src/Columns/IColumn.h +++ b/src/Columns/IColumn.h @@ -198,11 +198,19 @@ public: #endif /// Appends one element from other column with the same type multiple times. +#if !defined(ABORT_ON_LOGICAL_ERROR) + virtual void insertManyFrom(const IColumn & src, size_t position, size_t length) + { + for (size_t i = 0; i < length; ++i) + insertFrom(src, position); + } +#else void insertManyFrom(const IColumn & src, size_t position, size_t length) { assertTypeEquality(src); doInsertManyFrom(src, position, length); } +#endif /// Appends one field multiple times. Can be optimized in inherited classes. virtual void insertMany(const Field & field, size_t length) @@ -657,11 +665,13 @@ protected: virtual void doInsertRangeFrom(const IColumn & src, size_t start, size_t length) = 0; #endif +#if defined(ABORT_ON_LOGICAL_ERROR) virtual void doInsertManyFrom(const IColumn & src, size_t position, size_t length) { for (size_t i = 0; i < length; ++i) insertFrom(src, position); } +#endif virtual int doCompareAt(size_t n, size_t m, const IColumn & rhs, int nan_direction_hint) const = 0; diff --git a/src/Columns/benchmarks/benchmark_column_insert_many_from.cpp b/src/Columns/benchmarks/benchmark_column_insert_many_from.cpp index 6b9c329cd55..645f6ed79f3 100644 --- a/src/Columns/benchmarks/benchmark_column_insert_many_from.cpp +++ b/src/Columns/benchmarks/benchmark_column_insert_many_from.cpp @@ -52,7 +52,11 @@ static ColumnPtr mockColumn(const DataTypePtr & type, size_t rows) } +#if !defined(ABORT_ON_LOGICAL_ERROR) +static NO_INLINE void insertManyFrom(IColumn & dst, const IColumn & src) +#else static NO_INLINE void doInsertManyFrom(IColumn & dst, const IColumn & src) +#endif { size_t size = src.size(); dst.insertManyFrom(src, size / 2, size); From 1b45ac30a2aecb67a297ca91f8fc092122d2fc9c Mon Sep 17 00:00:00 2001 From: Nikita Taranov Date: Mon, 8 Jul 2024 17:14:25 +0100 Subject: [PATCH 321/417] insertFrom --- src/Columns/ColumnAggregateFunction.cpp | 4 ++++ src/Columns/ColumnAggregateFunction.h | 4 ++++ src/Columns/ColumnArray.cpp | 4 ++++ src/Columns/ColumnArray.h | 4 ++++ src/Columns/ColumnConst.h | 4 ++++ src/Columns/ColumnDecimal.h | 4 ++++ src/Columns/ColumnDynamic.cpp | 4 ++++ src/Columns/ColumnDynamic.h | 4 ++++ src/Columns/ColumnFixedString.cpp | 4 ++++ src/Columns/ColumnFixedString.h | 4 ++++ src/Columns/ColumnFunction.cpp | 4 ++++ src/Columns/ColumnFunction.h | 4 ++++ src/Columns/ColumnLowCardinality.cpp | 4 ++++ src/Columns/ColumnLowCardinality.h | 4 ++++ src/Columns/ColumnMap.cpp | 4 ++++ src/Columns/ColumnMap.h | 4 ++++ src/Columns/ColumnNullable.cpp | 4 ++++ src/Columns/ColumnNullable.h | 4 ++++ src/Columns/ColumnObject.cpp | 4 ++++ src/Columns/ColumnObject.h | 4 ++++ src/Columns/ColumnSparse.cpp | 4 ++++ src/Columns/ColumnSparse.h | 4 ++++ src/Columns/ColumnString.h | 4 ++++ src/Columns/ColumnTuple.cpp | 4 ++++ src/Columns/ColumnTuple.h | 4 ++++ src/Columns/ColumnVariant.cpp | 4 ++++ src/Columns/ColumnVariant.h | 4 ++++ src/Columns/ColumnVector.h | 4 ++++ src/Columns/IColumn.cpp | 4 ++++ src/Columns/IColumn.h | 8 +++++--- src/Columns/IColumnDummy.h | 4 ++++ 31 files changed, 125 insertions(+), 3 deletions(-) diff --git a/src/Columns/ColumnAggregateFunction.cpp b/src/Columns/ColumnAggregateFunction.cpp index 7124a03c605..cfd07c27765 100644 --- a/src/Columns/ColumnAggregateFunction.cpp +++ b/src/Columns/ColumnAggregateFunction.cpp @@ -466,7 +466,11 @@ void ColumnAggregateFunction::insertFromWithOwnership(const IColumn & from, size insertMergeFrom(from, n); } +#if !defined(ABORT_ON_LOGICAL_ERROR) +void ColumnAggregateFunction::insertFrom(const IColumn & from, size_t n) +#else void ColumnAggregateFunction::doInsertFrom(const IColumn & from, size_t n) +#endif { insertRangeFrom(from, n, 1); } diff --git a/src/Columns/ColumnAggregateFunction.h b/src/Columns/ColumnAggregateFunction.h index b33fd7dfc47..8d12d0fb851 100644 --- a/src/Columns/ColumnAggregateFunction.h +++ b/src/Columns/ColumnAggregateFunction.h @@ -145,7 +145,11 @@ public: void insertData(const char * pos, size_t length) override; +#if !defined(ABORT_ON_LOGICAL_ERROR) + void insertFrom(const IColumn & from, size_t n) override; +#else void doInsertFrom(const IColumn & from, size_t n) override; +#endif using IColumn::insertFrom; diff --git a/src/Columns/ColumnArray.cpp b/src/Columns/ColumnArray.cpp index bf89341d5a0..fba3ca420d7 100644 --- a/src/Columns/ColumnArray.cpp +++ b/src/Columns/ColumnArray.cpp @@ -337,7 +337,11 @@ bool ColumnArray::tryInsert(const Field & x) return true; } +#if !defined(ABORT_ON_LOGICAL_ERROR) +void ColumnArray::insertFrom(const IColumn & src_, size_t n) +#else void ColumnArray::doInsertFrom(const IColumn & src_, size_t n) +#endif { const ColumnArray & src = assert_cast(src_); size_t size = src.sizeAt(n); diff --git a/src/Columns/ColumnArray.h b/src/Columns/ColumnArray.h index bdadbd18a49..ad3db6a4822 100644 --- a/src/Columns/ColumnArray.h +++ b/src/Columns/ColumnArray.h @@ -91,7 +91,11 @@ public: #endif void insert(const Field & x) override; bool tryInsert(const Field & x) override; +#if !defined(ABORT_ON_LOGICAL_ERROR) + void insertFrom(const IColumn & src_, size_t n) override; +#else void doInsertFrom(const IColumn & src_, size_t n) override; +#endif void insertDefault() override; void popBack(size_t n) override; ColumnPtr filter(const Filter & filt, ssize_t result_size_hint) const override; diff --git a/src/Columns/ColumnConst.h b/src/Columns/ColumnConst.h index 72c3d1f0f0e..28892f3f09a 100644 --- a/src/Columns/ColumnConst.h +++ b/src/Columns/ColumnConst.h @@ -151,7 +151,11 @@ public: ++s; } +#if !defined(ABORT_ON_LOGICAL_ERROR) + void insertFrom(const IColumn &, size_t) override +#else void doInsertFrom(const IColumn &, size_t) override +#endif { ++s; } diff --git a/src/Columns/ColumnDecimal.h b/src/Columns/ColumnDecimal.h index 27b5c765710..cea1554c2bd 100644 --- a/src/Columns/ColumnDecimal.h +++ b/src/Columns/ColumnDecimal.h @@ -55,7 +55,11 @@ public: void reserve(size_t n) override { data.reserve_exact(n); } void shrinkToFit() override { data.shrink_to_fit(); } +#if !defined(ABORT_ON_LOGICAL_ERROR) + void insertFrom(const IColumn & src, size_t n) override { data.push_back(static_cast(src).getData()[n]); } +#else void doInsertFrom(const IColumn & src, size_t n) override { data.push_back(static_cast(src).getData()[n]); } +#endif #if !defined(ABORT_ON_LOGICAL_ERROR) void insertManyFrom(const IColumn & src, size_t position, size_t length) override diff --git a/src/Columns/ColumnDynamic.cpp b/src/Columns/ColumnDynamic.cpp index 6cc3e27b7ff..4d2474219fb 100644 --- a/src/Columns/ColumnDynamic.cpp +++ b/src/Columns/ColumnDynamic.cpp @@ -213,7 +213,11 @@ bool ColumnDynamic::tryInsert(const DB::Field & x) } +#if !defined(ABORT_ON_LOGICAL_ERROR) +void ColumnDynamic::insertFrom(const DB::IColumn & src_, size_t n) +#else void ColumnDynamic::doInsertFrom(const DB::IColumn & src_, size_t n) +#endif { const auto & dynamic_src = assert_cast(src_); diff --git a/src/Columns/ColumnDynamic.h b/src/Columns/ColumnDynamic.h index e3fd944a6bc..b394f5e38ad 100644 --- a/src/Columns/ColumnDynamic.h +++ b/src/Columns/ColumnDynamic.h @@ -142,7 +142,11 @@ public: void insert(const Field & x) override; bool tryInsert(const Field & x) override; +#if !defined(ABORT_ON_LOGICAL_ERROR) + void insertFrom(const IColumn & src_, size_t n) override; +#else void doInsertFrom(const IColumn & src_, size_t n) override; +#endif #if !defined(ABORT_ON_LOGICAL_ERROR) void insertRangeFrom(const IColumn & src, size_t start, size_t length) override; #else diff --git a/src/Columns/ColumnFixedString.cpp b/src/Columns/ColumnFixedString.cpp index 21435d405b3..1c2de203a94 100644 --- a/src/Columns/ColumnFixedString.cpp +++ b/src/Columns/ColumnFixedString.cpp @@ -74,7 +74,11 @@ bool ColumnFixedString::tryInsert(const Field & x) return true; } +#if !defined(ABORT_ON_LOGICAL_ERROR) +void ColumnFixedString::insertFrom(const IColumn & src_, size_t index) +#else void ColumnFixedString::doInsertFrom(const IColumn & src_, size_t index) +#endif { const ColumnFixedString & src = assert_cast(src_); diff --git a/src/Columns/ColumnFixedString.h b/src/Columns/ColumnFixedString.h index faa63910c00..5dbb514e639 100644 --- a/src/Columns/ColumnFixedString.h +++ b/src/Columns/ColumnFixedString.h @@ -98,7 +98,11 @@ public: bool tryInsert(const Field & x) override; +#if !defined(ABORT_ON_LOGICAL_ERROR) + void insertFrom(const IColumn & src_, size_t index) override; +#else void doInsertFrom(const IColumn & src_, size_t index) override; +#endif #if !defined(ABORT_ON_LOGICAL_ERROR) void insertManyFrom(const IColumn & src, size_t position, size_t length) override; diff --git a/src/Columns/ColumnFunction.cpp b/src/Columns/ColumnFunction.cpp index 4f19cf87fc6..fa57f35a823 100644 --- a/src/Columns/ColumnFunction.cpp +++ b/src/Columns/ColumnFunction.cpp @@ -72,7 +72,11 @@ ColumnPtr ColumnFunction::cut(size_t start, size_t length) const return ColumnFunction::create(length, function, capture, is_short_circuit_argument, is_function_compiled); } +#if !defined(ABORT_ON_LOGICAL_ERROR) +void ColumnFunction::insertFrom(const IColumn & src, size_t n) +#else void ColumnFunction::doInsertFrom(const IColumn & src, size_t n) +#endif { const ColumnFunction & src_func = assert_cast(src); diff --git a/src/Columns/ColumnFunction.h b/src/Columns/ColumnFunction.h index b5b0c53feb5..0af6c525599 100644 --- a/src/Columns/ColumnFunction.h +++ b/src/Columns/ColumnFunction.h @@ -94,7 +94,11 @@ public: throw Exception(ErrorCodes::NOT_IMPLEMENTED, "Cannot insert into {}", getName()); } +#if !defined(ABORT_ON_LOGICAL_ERROR) + void insertFrom(const IColumn & src, size_t n) override; +#else void doInsertFrom(const IColumn & src, size_t n) override; +#endif #if !defined(ABORT_ON_LOGICAL_ERROR) void insertRangeFrom(const IColumn &, size_t start, size_t length) override; #else diff --git a/src/Columns/ColumnLowCardinality.cpp b/src/Columns/ColumnLowCardinality.cpp index cf43640a669..3da7af7f168 100644 --- a/src/Columns/ColumnLowCardinality.cpp +++ b/src/Columns/ColumnLowCardinality.cpp @@ -159,7 +159,11 @@ void ColumnLowCardinality::insertDefault() idx.insertPosition(getDictionary().getDefaultValueIndex()); } +#if !defined(ABORT_ON_LOGICAL_ERROR) +void ColumnLowCardinality::insertFrom(const IColumn & src, size_t n) +#else void ColumnLowCardinality::doInsertFrom(const IColumn & src, size_t n) +#endif { const auto * low_cardinality_src = typeid_cast(&src); diff --git a/src/Columns/ColumnLowCardinality.h b/src/Columns/ColumnLowCardinality.h index ef0df669729..26c83a49107 100644 --- a/src/Columns/ColumnLowCardinality.h +++ b/src/Columns/ColumnLowCardinality.h @@ -78,7 +78,11 @@ public: bool tryInsert(const Field & x) override; void insertDefault() override; +#if !defined(ABORT_ON_LOGICAL_ERROR) + void insertFrom(const IColumn & src, size_t n) override; +#else void doInsertFrom(const IColumn & src, size_t n) override; +#endif void insertFromFullColumn(const IColumn & src, size_t n); #if !defined(ABORT_ON_LOGICAL_ERROR) diff --git a/src/Columns/ColumnMap.cpp b/src/Columns/ColumnMap.cpp index 53f8a81e40f..8c4a7f2f5d3 100644 --- a/src/Columns/ColumnMap.cpp +++ b/src/Columns/ColumnMap.cpp @@ -153,7 +153,11 @@ void ColumnMap::updateHashFast(SipHash & hash) const nested->updateHashFast(hash); } +#if !defined(ABORT_ON_LOGICAL_ERROR) +void ColumnMap::insertFrom(const IColumn & src, size_t n) +#else void ColumnMap::doInsertFrom(const IColumn & src, size_t n) +#endif { nested->insertFrom(assert_cast(src).getNestedColumn(), n); } diff --git a/src/Columns/ColumnMap.h b/src/Columns/ColumnMap.h index 05a32682e23..dae39d32fe1 100644 --- a/src/Columns/ColumnMap.h +++ b/src/Columns/ColumnMap.h @@ -66,7 +66,11 @@ public: void updateHashWithValue(size_t n, SipHash & hash) const override; void updateWeakHash32(WeakHash32 & hash) const override; void updateHashFast(SipHash & hash) const override; +#if !defined(ABORT_ON_LOGICAL_ERROR) + void insertFrom(const IColumn & src_, size_t n) override; +#else void doInsertFrom(const IColumn & src_, size_t n) override; +#endif #if !defined(ABORT_ON_LOGICAL_ERROR) void insertManyFrom(const IColumn & src, size_t position, size_t length) override; #else diff --git a/src/Columns/ColumnNullable.cpp b/src/Columns/ColumnNullable.cpp index c24278d02c5..adc7ce40a42 100644 --- a/src/Columns/ColumnNullable.cpp +++ b/src/Columns/ColumnNullable.cpp @@ -262,7 +262,11 @@ bool ColumnNullable::tryInsert(const Field & x) return true; } +#if !defined(ABORT_ON_LOGICAL_ERROR) +void ColumnNullable::insertFrom(const IColumn & src, size_t n) +#else void ColumnNullable::doInsertFrom(const IColumn & src, size_t n) +#endif { const ColumnNullable & src_concrete = assert_cast(src); getNestedColumn().insertFrom(src_concrete.getNestedColumn(), n); diff --git a/src/Columns/ColumnNullable.h b/src/Columns/ColumnNullable.h index 47d007c6dcf..a8d1352e44c 100644 --- a/src/Columns/ColumnNullable.h +++ b/src/Columns/ColumnNullable.h @@ -76,7 +76,11 @@ public: #endif void insert(const Field & x) override; bool tryInsert(const Field & x) override; +#if !defined(ABORT_ON_LOGICAL_ERROR) + void insertFrom(const IColumn & src, size_t n) override; +#else void doInsertFrom(const IColumn & src, size_t n) override; +#endif #if !defined(ABORT_ON_LOGICAL_ERROR) void insertManyFrom(const IColumn & src, size_t position, size_t length) override; #else diff --git a/src/Columns/ColumnObject.cpp b/src/Columns/ColumnObject.cpp index 69dbf780eeb..9c9dade3dd8 100644 --- a/src/Columns/ColumnObject.cpp +++ b/src/Columns/ColumnObject.cpp @@ -763,7 +763,11 @@ void ColumnObject::get(size_t n, Field & res) const } } +#if !defined(ABORT_ON_LOGICAL_ERROR) +void ColumnObject::insertFrom(const IColumn & src, size_t n) +#else void ColumnObject::doInsertFrom(const IColumn & src, size_t n) +#endif { insert(src[n]); } diff --git a/src/Columns/ColumnObject.h b/src/Columns/ColumnObject.h index eb8ac2874c2..ac227f1ed05 100644 --- a/src/Columns/ColumnObject.h +++ b/src/Columns/ColumnObject.h @@ -209,7 +209,11 @@ public: void insert(const Field & field) override; bool tryInsert(const Field & field) override; void insertDefault() override; +#if !defined(ABORT_ON_LOGICAL_ERROR) + void insertFrom(const IColumn & src, size_t n) override; +#else void doInsertFrom(const IColumn & src, size_t n) override; +#endif #if !defined(ABORT_ON_LOGICAL_ERROR) void insertRangeFrom(const IColumn & src, size_t start, size_t length) override; #else diff --git a/src/Columns/ColumnSparse.cpp b/src/Columns/ColumnSparse.cpp index 937d016c55c..0937bc92c26 100644 --- a/src/Columns/ColumnSparse.cpp +++ b/src/Columns/ColumnSparse.cpp @@ -252,7 +252,11 @@ bool ColumnSparse::tryInsert(const Field & x) return true; } +#if !defined(ABORT_ON_LOGICAL_ERROR) +void ColumnSparse::insertFrom(const IColumn & src, size_t n) +#else void ColumnSparse::doInsertFrom(const IColumn & src, size_t n) +#endif { if (const auto * src_sparse = typeid_cast(&src)) { diff --git a/src/Columns/ColumnSparse.h b/src/Columns/ColumnSparse.h index 73bcdb78984..8de7584bddc 100644 --- a/src/Columns/ColumnSparse.h +++ b/src/Columns/ColumnSparse.h @@ -88,7 +88,11 @@ public: #endif void insert(const Field & x) override; bool tryInsert(const Field & x) override; +#if !defined(ABORT_ON_LOGICAL_ERROR) + void insertFrom(const IColumn & src, size_t n) override; +#else void doInsertFrom(const IColumn & src, size_t n) override; +#endif void insertDefault() override; void insertManyDefaults(size_t length) override; diff --git a/src/Columns/ColumnString.h b/src/Columns/ColumnString.h index 6a1b4be0505..be88f4a49c0 100644 --- a/src/Columns/ColumnString.h +++ b/src/Columns/ColumnString.h @@ -142,7 +142,11 @@ public: return true; } +#if !defined(ABORT_ON_LOGICAL_ERROR) + void insertFrom(const IColumn & src_, size_t n) override +#else void doInsertFrom(const IColumn & src_, size_t n) override +#endif { const ColumnString & src = assert_cast(src_); const size_t size_to_append = src.offsets[n] - src.offsets[n - 1]; /// -1th index is Ok, see PaddedPODArray. diff --git a/src/Columns/ColumnTuple.cpp b/src/Columns/ColumnTuple.cpp index 382d2182b61..8cf2dec2452 100644 --- a/src/Columns/ColumnTuple.cpp +++ b/src/Columns/ColumnTuple.cpp @@ -205,7 +205,11 @@ bool ColumnTuple::tryInsert(const Field & x) return true; } +#if !defined(ABORT_ON_LOGICAL_ERROR) +void ColumnTuple::insertFrom(const IColumn & src_, size_t n) +#else void ColumnTuple::doInsertFrom(const IColumn & src_, size_t n) +#endif { const ColumnTuple & src = assert_cast(src_); diff --git a/src/Columns/ColumnTuple.h b/src/Columns/ColumnTuple.h index 7ca065a8439..ac4f713d341 100644 --- a/src/Columns/ColumnTuple.h +++ b/src/Columns/ColumnTuple.h @@ -65,7 +65,11 @@ public: void insertData(const char * pos, size_t length) override; void insert(const Field & x) override; bool tryInsert(const Field & x) override; +#if !defined(ABORT_ON_LOGICAL_ERROR) + void insertFrom(const IColumn & src_, size_t n) override; +#else void doInsertFrom(const IColumn & src_, size_t n) override; +#endif #if !defined(ABORT_ON_LOGICAL_ERROR) void insertManyFrom(const IColumn & src, size_t position, size_t length) override; #else diff --git a/src/Columns/ColumnVariant.cpp b/src/Columns/ColumnVariant.cpp index c571913614d..f8dcae258f3 100644 --- a/src/Columns/ColumnVariant.cpp +++ b/src/Columns/ColumnVariant.cpp @@ -595,7 +595,11 @@ void ColumnVariant::insertManyFromImpl(const DB::IColumn & src_, size_t position } } +#if !defined(ABORT_ON_LOGICAL_ERROR) +void ColumnVariant::insertFrom(const IColumn & src_, size_t n) +#else void ColumnVariant::doInsertFrom(const IColumn & src_, size_t n) +#endif { insertFromImpl(src_, n, nullptr); } diff --git a/src/Columns/ColumnVariant.h b/src/Columns/ColumnVariant.h index 6ec2529a48e..48719d4e9ca 100644 --- a/src/Columns/ColumnVariant.h +++ b/src/Columns/ColumnVariant.h @@ -180,7 +180,11 @@ public: void insert(const Field & x) override; bool tryInsert(const Field & x) override; +#if !defined(ABORT_ON_LOGICAL_ERROR) + void insertFrom(const IColumn & src_, size_t n) override; +#else void doInsertFrom(const IColumn & src_, size_t n) override; +#endif #if !defined(ABORT_ON_LOGICAL_ERROR) void insertRangeFrom(const IColumn & src_, size_t start, size_t length) override; #else diff --git a/src/Columns/ColumnVector.h b/src/Columns/ColumnVector.h index 19dacdad775..49ca42cc57b 100644 --- a/src/Columns/ColumnVector.h +++ b/src/Columns/ColumnVector.h @@ -64,7 +64,11 @@ public: return data.size(); } +#if !defined(ABORT_ON_LOGICAL_ERROR) + void insertFrom(const IColumn & src, size_t n) override +#else void doInsertFrom(const IColumn & src, size_t n) override +#endif { data.push_back(assert_cast(src).getData()[n]); } diff --git a/src/Columns/IColumn.cpp b/src/Columns/IColumn.cpp index 1b4bd8f6e62..552e52cf51c 100644 --- a/src/Columns/IColumn.cpp +++ b/src/Columns/IColumn.cpp @@ -46,7 +46,11 @@ String IColumn::dumpStructure() const return res.str(); } +#if !defined(ABORT_ON_LOGICAL_ERROR) +void IColumn::insertFrom(const IColumn & src, size_t n) +#else void IColumn::doInsertFrom(const IColumn & src, size_t n) +#endif { insert(src[n]); } diff --git a/src/Columns/IColumn.h b/src/Columns/IColumn.h index e5a82d5b772..8dbbf6bf9ea 100644 --- a/src/Columns/IColumn.h +++ b/src/Columns/IColumn.h @@ -179,11 +179,15 @@ public: /// Appends n-th element from other column with the same type. /// Is used in merge-sort and merges. It could be implemented in inherited classes more optimally than default implementation. +#if !defined(ABORT_ON_LOGICAL_ERROR) + virtual void insertFrom(const IColumn & src, size_t n); +#else void insertFrom(const IColumn & src, size_t n) { assertTypeEquality(src); doInsertFrom(src, n); } +#endif /// Appends range of elements from other column with the same type. /// Could be used to concatenate columns. @@ -659,13 +663,11 @@ protected: Sort full_sort, PartialSort partial_sort) const; +#if defined(ABORT_ON_LOGICAL_ERROR) virtual void doInsertFrom(const IColumn & src, size_t n); -#if defined(ABORT_ON_LOGICAL_ERROR) virtual void doInsertRangeFrom(const IColumn & src, size_t start, size_t length) = 0; -#endif -#if defined(ABORT_ON_LOGICAL_ERROR) virtual void doInsertManyFrom(const IColumn & src, size_t position, size_t length) { for (size_t i = 0; i < length; ++i) diff --git a/src/Columns/IColumnDummy.h b/src/Columns/IColumnDummy.h index 023cbbc463a..e52c143ace7 100644 --- a/src/Columns/IColumnDummy.h +++ b/src/Columns/IColumnDummy.h @@ -67,7 +67,11 @@ public: { } +#if !defined(ABORT_ON_LOGICAL_ERROR) + void insertFrom(const IColumn &, size_t) override +#else void doInsertFrom(const IColumn &, size_t) override +#endif { ++s; } From b3993532a9abc41360f3011c30b446894a2f424f Mon Sep 17 00:00:00 2001 From: Nikita Taranov Date: Mon, 8 Jul 2024 17:17:39 +0100 Subject: [PATCH 322/417] compareAt --- src/Columns/ColumnAggregateFunction.h | 4 ++++ src/Columns/ColumnArray.cpp | 4 ++++ src/Columns/ColumnArray.h | 4 ++++ src/Columns/ColumnCompressed.h | 4 ++++ src/Columns/ColumnConst.h | 4 ++++ src/Columns/ColumnDecimal.cpp | 4 ++++ src/Columns/ColumnDecimal.h | 4 ++++ src/Columns/ColumnDynamic.cpp | 4 ++++ src/Columns/ColumnDynamic.h | 4 ++++ src/Columns/ColumnFixedString.h | 4 ++++ src/Columns/ColumnFunction.h | 4 ++++ src/Columns/ColumnLowCardinality.cpp | 4 ++++ src/Columns/ColumnLowCardinality.h | 4 ++++ src/Columns/ColumnMap.cpp | 4 ++++ src/Columns/ColumnMap.h | 4 ++++ src/Columns/ColumnNullable.cpp | 4 ++++ src/Columns/ColumnNullable.h | 4 ++++ src/Columns/ColumnObject.h | 4 ++++ src/Columns/ColumnSparse.cpp | 4 ++++ src/Columns/ColumnSparse.h | 4 ++++ src/Columns/ColumnString.h | 4 ++++ src/Columns/ColumnTuple.cpp | 4 ++++ src/Columns/ColumnTuple.h | 4 ++++ src/Columns/ColumnUnique.h | 8 ++++++++ src/Columns/ColumnVariant.cpp | 4 ++++ src/Columns/ColumnVariant.h | 4 ++++ src/Columns/ColumnVector.h | 4 ++++ src/Columns/IColumn.h | 6 +++++- src/Columns/IColumnDummy.h | 4 ++++ 29 files changed, 121 insertions(+), 1 deletion(-) diff --git a/src/Columns/ColumnAggregateFunction.h b/src/Columns/ColumnAggregateFunction.h index 8d12d0fb851..fada5fc0c68 100644 --- a/src/Columns/ColumnAggregateFunction.h +++ b/src/Columns/ColumnAggregateFunction.h @@ -211,7 +211,11 @@ public: MutableColumns scatter(ColumnIndex num_columns, const Selector & selector) const override; +#if !defined(ABORT_ON_LOGICAL_ERROR) + int compareAt(size_t, size_t, const IColumn &, int) const override +#else int doCompareAt(size_t, size_t, const IColumn &, int) const override +#endif { return 0; } diff --git a/src/Columns/ColumnArray.cpp b/src/Columns/ColumnArray.cpp index fba3ca420d7..5d7350f3a79 100644 --- a/src/Columns/ColumnArray.cpp +++ b/src/Columns/ColumnArray.cpp @@ -396,7 +396,11 @@ int ColumnArray::compareAtImpl(size_t n, size_t m, const IColumn & rhs_, int nan : 1); } +#if !defined(ABORT_ON_LOGICAL_ERROR) +int ColumnArray::compareAt(size_t n, size_t m, const IColumn & rhs_, int nan_direction_hint) const +#else int ColumnArray::doCompareAt(size_t n, size_t m, const IColumn & rhs_, int nan_direction_hint) const +#endif { return compareAtImpl(n, m, rhs_, nan_direction_hint); } diff --git a/src/Columns/ColumnArray.h b/src/Columns/ColumnArray.h index ad3db6a4822..6cd3e2f6c3b 100644 --- a/src/Columns/ColumnArray.h +++ b/src/Columns/ColumnArray.h @@ -103,7 +103,11 @@ public: ColumnPtr permute(const Permutation & perm, size_t limit) const override; ColumnPtr index(const IColumn & indexes, size_t limit) const override; template ColumnPtr indexImpl(const PaddedPODArray & indexes, size_t limit) const; +#if !defined(ABORT_ON_LOGICAL_ERROR) + int compareAt(size_t n, size_t m, const IColumn & rhs_, int nan_direction_hint) const override; +#else int doCompareAt(size_t n, size_t m, const IColumn & rhs_, int nan_direction_hint) const override; +#endif int compareAtWithCollation(size_t n, size_t m, const IColumn & rhs_, int nan_direction_hint, const Collator & collator) const override; void getPermutation(PermutationSortDirection direction, PermutationSortStability stability, size_t limit, int nan_direction_hint, Permutation & res) const override; diff --git a/src/Columns/ColumnCompressed.h b/src/Columns/ColumnCompressed.h index d0f4c2c5910..5e455709fec 100644 --- a/src/Columns/ColumnCompressed.h +++ b/src/Columns/ColumnCompressed.h @@ -104,7 +104,11 @@ public: void expand(const Filter &, bool) override { throwMustBeDecompressed(); } ColumnPtr permute(const Permutation &, size_t) const override { throwMustBeDecompressed(); } ColumnPtr index(const IColumn &, size_t) const override { throwMustBeDecompressed(); } +#if !defined(ABORT_ON_LOGICAL_ERROR) + int compareAt(size_t, size_t, const IColumn &, int) const override { throwMustBeDecompressed(); } +#else int doCompareAt(size_t, size_t, const IColumn &, int) const override { throwMustBeDecompressed(); } +#endif void compareColumn(const IColumn &, size_t, PaddedPODArray *, PaddedPODArray &, int, int) const override { throwMustBeDecompressed(); diff --git a/src/Columns/ColumnConst.h b/src/Columns/ColumnConst.h index 28892f3f09a..b55a1f42037 100644 --- a/src/Columns/ColumnConst.h +++ b/src/Columns/ColumnConst.h @@ -237,7 +237,11 @@ public: return data->allocatedBytes() + sizeof(s); } +#if !defined(ABORT_ON_LOGICAL_ERROR) + int compareAt(size_t, size_t, const IColumn & rhs, int nan_direction_hint) const override +#else int doCompareAt(size_t, size_t, const IColumn & rhs, int nan_direction_hint) const override +#endif { return data->compareAt(0, 0, *assert_cast(rhs).data, nan_direction_hint); } diff --git a/src/Columns/ColumnDecimal.cpp b/src/Columns/ColumnDecimal.cpp index b09d7b1ee90..cf413f790a7 100644 --- a/src/Columns/ColumnDecimal.cpp +++ b/src/Columns/ColumnDecimal.cpp @@ -32,7 +32,11 @@ namespace ErrorCodes } template +#if !defined(ABORT_ON_LOGICAL_ERROR) +int ColumnDecimal::compareAt(size_t n, size_t m, const IColumn & rhs_, int) const +#else int ColumnDecimal::doCompareAt(size_t n, size_t m, const IColumn & rhs_, int) const +#endif { auto & other = static_cast(rhs_); const T & a = data[n]; diff --git a/src/Columns/ColumnDecimal.h b/src/Columns/ColumnDecimal.h index cea1554c2bd..32efeb643a6 100644 --- a/src/Columns/ColumnDecimal.h +++ b/src/Columns/ColumnDecimal.h @@ -104,7 +104,11 @@ public: void updateHashWithValue(size_t n, SipHash & hash) const override; void updateWeakHash32(WeakHash32 & hash) const override; void updateHashFast(SipHash & hash) const override; +#if !defined(ABORT_ON_LOGICAL_ERROR) + int compareAt(size_t n, size_t m, const IColumn & rhs_, int nan_direction_hint) const override; +#else int doCompareAt(size_t n, size_t m, const IColumn & rhs_, int nan_direction_hint) const override; +#endif void getPermutation(IColumn::PermutationSortDirection direction, IColumn::PermutationSortStability stability, size_t limit, int nan_direction_hint, IColumn::Permutation & res) const override; void updatePermutation(IColumn::PermutationSortDirection direction, IColumn::PermutationSortStability stability, diff --git a/src/Columns/ColumnDynamic.cpp b/src/Columns/ColumnDynamic.cpp index 4d2474219fb..7bf378bb0ac 100644 --- a/src/Columns/ColumnDynamic.cpp +++ b/src/Columns/ColumnDynamic.cpp @@ -599,7 +599,11 @@ void ColumnDynamic::updateHashWithValue(size_t n, SipHash & hash) const variant_col.getVariantByGlobalDiscriminator(discr).updateHashWithValue(variant_col.offsetAt(n), hash); } +#if !defined(ABORT_ON_LOGICAL_ERROR) +int ColumnDynamic::compareAt(size_t n, size_t m, const DB::IColumn & rhs, int nan_direction_hint) const +#else int ColumnDynamic::doCompareAt(size_t n, size_t m, const DB::IColumn & rhs, int nan_direction_hint) const +#endif { const auto & left_variant = assert_cast(*variant_column); const auto & right_dynamic = assert_cast(rhs); diff --git a/src/Columns/ColumnDynamic.h b/src/Columns/ColumnDynamic.h index b394f5e38ad..eb8a2ad6ed6 100644 --- a/src/Columns/ColumnDynamic.h +++ b/src/Columns/ColumnDynamic.h @@ -225,7 +225,11 @@ public: return scattered_columns; } +#if !defined(ABORT_ON_LOGICAL_ERROR) + int compareAt(size_t n, size_t m, const IColumn & rhs, int nan_direction_hint) const override; +#else int doCompareAt(size_t n, size_t m, const IColumn & rhs, int nan_direction_hint) const override; +#endif bool hasEqualValues() const override { diff --git a/src/Columns/ColumnFixedString.h b/src/Columns/ColumnFixedString.h index 5dbb514e639..6e88136fc50 100644 --- a/src/Columns/ColumnFixedString.h +++ b/src/Columns/ColumnFixedString.h @@ -137,7 +137,11 @@ public: void updateHashFast(SipHash & hash) const override; +#if !defined(ABORT_ON_LOGICAL_ERROR) + int compareAt(size_t p1, size_t p2, const IColumn & rhs_, int /*nan_direction_hint*/) const override +#else int doCompareAt(size_t p1, size_t p2, const IColumn & rhs_, int /*nan_direction_hint*/) const override +#endif { const ColumnFixedString & rhs = assert_cast(rhs_); chassert(this->n == rhs.n); diff --git a/src/Columns/ColumnFunction.h b/src/Columns/ColumnFunction.h index 0af6c525599..ba924c49a82 100644 --- a/src/Columns/ColumnFunction.h +++ b/src/Columns/ColumnFunction.h @@ -145,7 +145,11 @@ public: throw Exception(ErrorCodes::NOT_IMPLEMENTED, "popBack is not implemented for {}", getName()); } +#if !defined(ABORT_ON_LOGICAL_ERROR) + int compareAt(size_t, size_t, const IColumn &, int) const override +#else int doCompareAt(size_t, size_t, const IColumn &, int) const override +#endif { throw Exception(ErrorCodes::NOT_IMPLEMENTED, "compareAt is not implemented for {}", getName()); } diff --git a/src/Columns/ColumnLowCardinality.cpp b/src/Columns/ColumnLowCardinality.cpp index 3da7af7f168..eb694a10b0f 100644 --- a/src/Columns/ColumnLowCardinality.cpp +++ b/src/Columns/ColumnLowCardinality.cpp @@ -372,7 +372,11 @@ int ColumnLowCardinality::compareAtImpl(size_t n, size_t m, const IColumn & rhs, return getDictionary().compareAt(n_index, m_index, low_cardinality_column.getDictionary(), nan_direction_hint); } +#if !defined(ABORT_ON_LOGICAL_ERROR) +int ColumnLowCardinality::compareAt(size_t n, size_t m, const IColumn & rhs, int nan_direction_hint) const +#else int ColumnLowCardinality::doCompareAt(size_t n, size_t m, const IColumn & rhs, int nan_direction_hint) const +#endif { return compareAtImpl(n, m, rhs, nan_direction_hint); } diff --git a/src/Columns/ColumnLowCardinality.h b/src/Columns/ColumnLowCardinality.h index 26c83a49107..e99be07cd8d 100644 --- a/src/Columns/ColumnLowCardinality.h +++ b/src/Columns/ColumnLowCardinality.h @@ -135,7 +135,11 @@ public: return ColumnLowCardinality::create(dictionary.getColumnUniquePtr(), getIndexes().index(indexes_, limit)); } +#if !defined(ABORT_ON_LOGICAL_ERROR) + int compareAt(size_t n, size_t m, const IColumn & rhs, int nan_direction_hint) const override; +#else int doCompareAt(size_t n, size_t m, const IColumn & rhs, int nan_direction_hint) const override; +#endif int compareAtWithCollation(size_t n, size_t m, const IColumn & rhs, int nan_direction_hint, const Collator &) const override; diff --git a/src/Columns/ColumnMap.cpp b/src/Columns/ColumnMap.cpp index 8c4a7f2f5d3..2dffddb2dc9 100644 --- a/src/Columns/ColumnMap.cpp +++ b/src/Columns/ColumnMap.cpp @@ -222,7 +222,11 @@ MutableColumns ColumnMap::scatter(ColumnIndex num_columns, const Selector & sele return res; } +#if !defined(ABORT_ON_LOGICAL_ERROR) +int ColumnMap::compareAt(size_t n, size_t m, const IColumn & rhs, int nan_direction_hint) const +#else int ColumnMap::doCompareAt(size_t n, size_t m, const IColumn & rhs, int nan_direction_hint) const +#endif { const auto & rhs_map = assert_cast(rhs); return nested->compareAt(n, m, rhs_map.getNestedColumn(), nan_direction_hint); diff --git a/src/Columns/ColumnMap.h b/src/Columns/ColumnMap.h index dae39d32fe1..d15829f4147 100644 --- a/src/Columns/ColumnMap.h +++ b/src/Columns/ColumnMap.h @@ -87,7 +87,11 @@ public: ColumnPtr index(const IColumn & indexes, size_t limit) const override; ColumnPtr replicate(const Offsets & offsets) const override; MutableColumns scatter(ColumnIndex num_columns, const Selector & selector) const override; +#if !defined(ABORT_ON_LOGICAL_ERROR) + int compareAt(size_t n, size_t m, const IColumn & rhs, int nan_direction_hint) const override; +#else int doCompareAt(size_t n, size_t m, const IColumn & rhs, int nan_direction_hint) const override; +#endif void getExtremes(Field & min, Field & max) const override; void getPermutation(IColumn::PermutationSortDirection direction, IColumn::PermutationSortStability stability, size_t limit, int nan_direction_hint, IColumn::Permutation & res) const override; diff --git a/src/Columns/ColumnNullable.cpp b/src/Columns/ColumnNullable.cpp index adc7ce40a42..f060e74b315 100644 --- a/src/Columns/ColumnNullable.cpp +++ b/src/Columns/ColumnNullable.cpp @@ -414,7 +414,11 @@ int ColumnNullable::compareAtImpl(size_t n, size_t m, const IColumn & rhs_, int return getNestedColumn().compareAt(n, m, nested_rhs, null_direction_hint); } +#if !defined(ABORT_ON_LOGICAL_ERROR) +int ColumnNullable::compareAt(size_t n, size_t m, const IColumn & rhs_, int null_direction_hint) const +#else int ColumnNullable::doCompareAt(size_t n, size_t m, const IColumn & rhs_, int null_direction_hint) const +#endif { return compareAtImpl(n, m, rhs_, null_direction_hint); } diff --git a/src/Columns/ColumnNullable.h b/src/Columns/ColumnNullable.h index a8d1352e44c..f89e67b6000 100644 --- a/src/Columns/ColumnNullable.h +++ b/src/Columns/ColumnNullable.h @@ -102,7 +102,11 @@ public: void expand(const Filter & mask, bool inverted) override; ColumnPtr permute(const Permutation & perm, size_t limit) const override; ColumnPtr index(const IColumn & indexes, size_t limit) const override; +#if !defined(ABORT_ON_LOGICAL_ERROR) + int compareAt(size_t n, size_t m, const IColumn & rhs_, int null_direction_hint) const override; +#else int doCompareAt(size_t n, size_t m, const IColumn & rhs_, int null_direction_hint) const override; +#endif #if USE_EMBEDDED_COMPILER diff --git a/src/Columns/ColumnObject.h b/src/Columns/ColumnObject.h index ac227f1ed05..9de2adb87de 100644 --- a/src/Columns/ColumnObject.h +++ b/src/Columns/ColumnObject.h @@ -236,7 +236,11 @@ public: /// Order of rows in ColumnObject is undefined. void getPermutation(PermutationSortDirection, PermutationSortStability, size_t, int, Permutation & res) const override; void updatePermutation(PermutationSortDirection, PermutationSortStability, size_t, int, Permutation &, EqualRanges &) const override {} +#if !defined(ABORT_ON_LOGICAL_ERROR) + int compareAt(size_t, size_t, const IColumn &, int) const override { return 0; } +#else int doCompareAt(size_t, size_t, const IColumn &, int) const override { return 0; } +#endif void getExtremes(Field & min, Field & max) const override; /// All other methods throw exception. diff --git a/src/Columns/ColumnSparse.cpp b/src/Columns/ColumnSparse.cpp index 0937bc92c26..809586d8810 100644 --- a/src/Columns/ColumnSparse.cpp +++ b/src/Columns/ColumnSparse.cpp @@ -454,7 +454,11 @@ ColumnPtr ColumnSparse::indexImpl(const PaddedPODArray & indexes, size_t l return ColumnSparse::create(std::move(res_values), std::move(res_offsets), limit); } +#if !defined(ABORT_ON_LOGICAL_ERROR) +int ColumnSparse::compareAt(size_t n, size_t m, const IColumn & rhs_, int null_direction_hint) const +#else int ColumnSparse::doCompareAt(size_t n, size_t m, const IColumn & rhs_, int null_direction_hint) const +#endif { if (const auto * rhs_sparse = typeid_cast(&rhs_)) return values->compareAt(getValueIndex(n), rhs_sparse->getValueIndex(m), rhs_sparse->getValuesColumn(), null_direction_hint); diff --git a/src/Columns/ColumnSparse.h b/src/Columns/ColumnSparse.h index 8de7584bddc..3e34d1de94a 100644 --- a/src/Columns/ColumnSparse.h +++ b/src/Columns/ColumnSparse.h @@ -106,7 +106,11 @@ public: template ColumnPtr indexImpl(const PaddedPODArray & indexes, size_t limit) const; +#if !defined(ABORT_ON_LOGICAL_ERROR) + int compareAt(size_t n, size_t m, const IColumn & rhs_, int null_direction_hint) const override; +#else int doCompareAt(size_t n, size_t m, const IColumn & rhs_, int null_direction_hint) const override; +#endif void compareColumn(const IColumn & rhs, size_t rhs_row_num, PaddedPODArray * row_indexes, PaddedPODArray & compare_results, int direction, int nan_direction_hint) const override; diff --git a/src/Columns/ColumnString.h b/src/Columns/ColumnString.h index be88f4a49c0..602ffac65e8 100644 --- a/src/Columns/ColumnString.h +++ b/src/Columns/ColumnString.h @@ -250,7 +250,11 @@ public: offsets.push_back(offsets.back() + 1); } +#if !defined(ABORT_ON_LOGICAL_ERROR) + int compareAt(size_t n, size_t m, const IColumn & rhs_, int /*nan_direction_hint*/) const override +#else int doCompareAt(size_t n, size_t m, const IColumn & rhs_, int /*nan_direction_hint*/) const override +#endif { const ColumnString & rhs = assert_cast(rhs_); return memcmpSmallAllowOverflow15(chars.data() + offsetAt(n), sizeAt(n) - 1, rhs.chars.data() + rhs.offsetAt(m), rhs.sizeAt(m) - 1); diff --git a/src/Columns/ColumnTuple.cpp b/src/Columns/ColumnTuple.cpp index 8cf2dec2452..b471725f516 100644 --- a/src/Columns/ColumnTuple.cpp +++ b/src/Columns/ColumnTuple.cpp @@ -482,7 +482,11 @@ int ColumnTuple::compareAtImpl(size_t n, size_t m, const IColumn & rhs, int nan_ return 0; } +#if !defined(ABORT_ON_LOGICAL_ERROR) +int ColumnTuple::compareAt(size_t n, size_t m, const IColumn & rhs, int nan_direction_hint) const +#else int ColumnTuple::doCompareAt(size_t n, size_t m, const IColumn & rhs, int nan_direction_hint) const +#endif { return compareAtImpl(n, m, rhs, nan_direction_hint); } diff --git a/src/Columns/ColumnTuple.h b/src/Columns/ColumnTuple.h index ac4f713d341..11c09f827f5 100644 --- a/src/Columns/ColumnTuple.h +++ b/src/Columns/ColumnTuple.h @@ -95,7 +95,11 @@ public: ColumnPtr index(const IColumn & indexes, size_t limit) const override; ColumnPtr replicate(const Offsets & offsets) const override; MutableColumns scatter(ColumnIndex num_columns, const Selector & selector) const override; +#if !defined(ABORT_ON_LOGICAL_ERROR) + int compareAt(size_t n, size_t m, const IColumn & rhs, int nan_direction_hint) const override; +#else int doCompareAt(size_t n, size_t m, const IColumn & rhs, int nan_direction_hint) const override; +#endif int compareAtWithCollation(size_t n, size_t m, const IColumn & rhs, int nan_direction_hint, const Collator & collator) const override; void getExtremes(Field & min, Field & max) const override; void getPermutation(IColumn::PermutationSortDirection direction, IColumn::PermutationSortStability stability, diff --git a/src/Columns/ColumnUnique.h b/src/Columns/ColumnUnique.h index 12f765f42af..ec1f8e0a4d5 100644 --- a/src/Columns/ColumnUnique.h +++ b/src/Columns/ColumnUnique.h @@ -90,7 +90,11 @@ public: return getNestedColumn()->updateHashWithValue(n, hash_func); } +#if !defined(ABORT_ON_LOGICAL_ERROR) + int compareAt(size_t n, size_t m, const IColumn & rhs, int nan_direction_hint) const override; +#else int doCompareAt(size_t n, size_t m, const IColumn & rhs, int nan_direction_hint) const override; +#endif void getExtremes(Field & min, Field & max) const override { column_holder->getExtremes(min, max); } bool valuesHaveFixedSize() const override { return column_holder->valuesHaveFixedSize(); } @@ -488,7 +492,11 @@ const char * ColumnUnique::skipSerializedInArena(const char *) const } template +#if !defined(ABORT_ON_LOGICAL_ERROR) +int ColumnUnique::compareAt(size_t n, size_t m, const IColumn & rhs, int nan_direction_hint) const +#else int ColumnUnique::doCompareAt(size_t n, size_t m, const IColumn & rhs, int nan_direction_hint) const +#endif { if (is_nullable) { diff --git a/src/Columns/ColumnVariant.cpp b/src/Columns/ColumnVariant.cpp index f8dcae258f3..ee5de4c2dde 100644 --- a/src/Columns/ColumnVariant.cpp +++ b/src/Columns/ColumnVariant.cpp @@ -1186,7 +1186,11 @@ bool ColumnVariant::hasEqualValues() const return local_discriminators->hasEqualValues() && variants[localDiscriminatorAt(0)]->hasEqualValues(); } +#if !defined(ABORT_ON_LOGICAL_ERROR) +int ColumnVariant::compareAt(size_t n, size_t m, const IColumn & rhs, int nan_direction_hint) const +#else int ColumnVariant::doCompareAt(size_t n, size_t m, const IColumn & rhs, int nan_direction_hint) const +#endif { const auto & rhs_variant = assert_cast(rhs); Discriminator left_discr = globalDiscriminatorAt(n); diff --git a/src/Columns/ColumnVariant.h b/src/Columns/ColumnVariant.h index 48719d4e9ca..b492d4b49c7 100644 --- a/src/Columns/ColumnVariant.h +++ b/src/Columns/ColumnVariant.h @@ -229,7 +229,11 @@ public: ColumnPtr indexImpl(const PaddedPODArray & indexes, size_t limit) const; ColumnPtr replicate(const Offsets & replicate_offsets) const override; MutableColumns scatter(ColumnIndex num_columns, const Selector & selector) const override; +#if !defined(ABORT_ON_LOGICAL_ERROR) + int compareAt(size_t n, size_t m, const IColumn & rhs, int nan_direction_hint) const override; +#else int doCompareAt(size_t n, size_t m, const IColumn & rhs, int nan_direction_hint) const override; +#endif bool hasEqualValues() const override; void getExtremes(Field & min, Field & max) const override; void getPermutation(IColumn::PermutationSortDirection direction, IColumn::PermutationSortStability stability, diff --git a/src/Columns/ColumnVector.h b/src/Columns/ColumnVector.h index 49ca42cc57b..3a0acf5e312 100644 --- a/src/Columns/ColumnVector.h +++ b/src/Columns/ColumnVector.h @@ -150,7 +150,11 @@ public: } /// This method implemented in header because it could be possibly devirtualized. +#if !defined(ABORT_ON_LOGICAL_ERROR) + int compareAt(size_t n, size_t m, const IColumn & rhs_, int nan_direction_hint) const override +#else int doCompareAt(size_t n, size_t m, const IColumn & rhs_, int nan_direction_hint) const override +#endif { return CompareHelper::compare(data[n], assert_cast(rhs_).data[m], nan_direction_hint); } diff --git a/src/Columns/IColumn.h b/src/Columns/IColumn.h index 8dbbf6bf9ea..4b6f34e5aa2 100644 --- a/src/Columns/IColumn.h +++ b/src/Columns/IColumn.h @@ -345,11 +345,15 @@ public: * * For non Nullable and non floating point types, nan_direction_hint is ignored. */ +#if !defined(ABORT_ON_LOGICAL_ERROR) + [[nodiscard]] virtual int compareAt(size_t n, size_t m, const IColumn & rhs, int nan_direction_hint) const = 0; +#else [[nodiscard]] int compareAt(size_t n, size_t m, const IColumn & rhs, int nan_direction_hint) const { assertTypeEquality(rhs); return doCompareAt(n, m, rhs, nan_direction_hint); } +#endif #if USE_EMBEDDED_COMPILER @@ -673,7 +677,6 @@ protected: for (size_t i = 0; i < length; ++i) insertFrom(src, position); } -#endif virtual int doCompareAt(size_t n, size_t m, const IColumn & rhs, int nan_direction_hint) const = 0; @@ -685,6 +688,7 @@ private: /// For the rest of column types we can compare the types directly. chassert((isConst() || isSparse()) ? getDataType() == rhs.getDataType() : typeid(*this) == typeid(rhs)); } +#endif }; using ColumnPtr = IColumn::Ptr; diff --git a/src/Columns/IColumnDummy.h b/src/Columns/IColumnDummy.h index e52c143ace7..c19fb704d9b 100644 --- a/src/Columns/IColumnDummy.h +++ b/src/Columns/IColumnDummy.h @@ -26,7 +26,11 @@ public: size_t byteSize() const override { return 0; } size_t byteSizeAt(size_t) const override { return 0; } size_t allocatedBytes() const override { return 0; } +#if !defined(ABORT_ON_LOGICAL_ERROR) + int compareAt(size_t, size_t, const IColumn &, int) const override { return 0; } +#else int doCompareAt(size_t, size_t, const IColumn &, int) const override { return 0; } +#endif void compareColumn(const IColumn &, size_t, PaddedPODArray *, PaddedPODArray &, int, int) const override { } From bd42a096b7e023195316d798188adfe0a98555f5 Mon Sep 17 00:00:00 2001 From: Robert Schulze Date: Mon, 8 Jul 2024 16:52:36 +0000 Subject: [PATCH 323/417] Bump rocksdb to v6.29.5 --- contrib/rocksdb | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/contrib/rocksdb b/contrib/rocksdb index b8a996ce196..be366233921 160000 --- a/contrib/rocksdb +++ b/contrib/rocksdb @@ -1 +1 @@ -Subproject commit b8a996ce1969a3f7141aca7fb5c54196a58a654a +Subproject commit be366233921293bd07a84dc4ea6991858665f202 From bb4115b4e6ee1291115a509f555f1d0c591769e9 Mon Sep 17 00:00:00 2001 From: Azat Khuzhin Date: Mon, 8 Jul 2024 18:34:31 +0200 Subject: [PATCH 324/417] Reduce inaccuracy of input_wait_elapsed_us/input_wait_elapsed_us/elapsed_us By collecting them with nanoseconds precision, and only store them in system.processors_profile_log in microseconds. This should fix 02210_processors_profile_log test failures like this one [1]: ExpressionTransform 999989 1 1 1 1 When the total elapsed_us is less then 1 second. [1]: https://s3.amazonaws.com/clickhouse-test-reports/65920/ce417c78be566d8a616df3544e2801b845277f44/stateless_tests__release__old_analyzer__s3__databasereplicated__[1_4].html Signed-off-by: Azat Khuzhin --- src/Interpreters/executeQuery.cpp | 6 +++--- src/Processors/Executors/ExecutingGraph.cpp | 4 ++-- .../Executors/ExecutionThreadContext.cpp | 6 +++--- src/Processors/IProcessor.h | 18 +++++++++--------- 4 files changed, 17 insertions(+), 17 deletions(-) diff --git a/src/Interpreters/executeQuery.cpp b/src/Interpreters/executeQuery.cpp index 9f33cbf1c27..336e226a691 100644 --- a/src/Interpreters/executeQuery.cpp +++ b/src/Interpreters/executeQuery.cpp @@ -476,9 +476,9 @@ void logQueryFinish( processor_elem.processor_name = processor->getName(); /// NOTE: convert this to UInt64 - processor_elem.elapsed_us = static_cast(processor->getElapsedUs()); - processor_elem.input_wait_elapsed_us = static_cast(processor->getInputWaitElapsedUs()); - processor_elem.output_wait_elapsed_us = static_cast(processor->getOutputWaitElapsedUs()); + processor_elem.elapsed_us = static_cast(processor->getElapsedNs() / 1000U); + processor_elem.input_wait_elapsed_us = static_cast(processor->getInputWaitElapsedNs() / 1000U); + processor_elem.output_wait_elapsed_us = static_cast(processor->getOutputWaitElapsedNs() / 1000U); auto stats = processor->getProcessorDataStats(); processor_elem.input_rows = stats.input_rows; diff --git a/src/Processors/Executors/ExecutingGraph.cpp b/src/Processors/Executors/ExecutingGraph.cpp index 27f6a454b24..6d5b60d8159 100644 --- a/src/Processors/Executors/ExecutingGraph.cpp +++ b/src/Processors/Executors/ExecutingGraph.cpp @@ -292,7 +292,7 @@ bool ExecutingGraph::updateNode(uint64_t pid, Queue & queue, Queue & async_queue } else if (last_status == IProcessor::Status::NeedData && status != IProcessor::Status::NeedData) { - processor.input_wait_elapsed_us += processor.input_wait_watch.elapsedMicroseconds(); + processor.input_wait_elapsed_ns += processor.input_wait_watch.elapsedNanoseconds(); } /// PortFull @@ -302,7 +302,7 @@ bool ExecutingGraph::updateNode(uint64_t pid, Queue & queue, Queue & async_queue } else if (last_status == IProcessor::Status::PortFull && status != IProcessor::Status::PortFull) { - processor.output_wait_elapsed_us += processor.output_wait_watch.elapsedMicroseconds(); + processor.output_wait_elapsed_ns += processor.output_wait_watch.elapsedNanoseconds(); } } } diff --git a/src/Processors/Executors/ExecutionThreadContext.cpp b/src/Processors/Executors/ExecutionThreadContext.cpp index 05669725f9a..17b6773ad83 100644 --- a/src/Processors/Executors/ExecutionThreadContext.cpp +++ b/src/Processors/Executors/ExecutionThreadContext.cpp @@ -103,10 +103,10 @@ bool ExecutionThreadContext::executeTask() if (profile_processors) { - UInt64 elapsed_microseconds = execution_time_watch->elapsedMicroseconds(); - node->processor->elapsed_us += elapsed_microseconds; + UInt64 elapsed_ns = execution_time_watch->elapsedNanoseconds(); + node->processor->elapsed_ns += elapsed_ns; if (trace_processors) - span->addAttribute("execution_time_ms", elapsed_microseconds); + span->addAttribute("execution_time_ms", elapsed_ns / 1000U); } #ifndef NDEBUG execution_time_ns += execution_time_watch->elapsed(); diff --git a/src/Processors/IProcessor.h b/src/Processors/IProcessor.h index 6f779e7a8d4..02f7b6b3d12 100644 --- a/src/Processors/IProcessor.h +++ b/src/Processors/IProcessor.h @@ -303,9 +303,9 @@ public: IQueryPlanStep * getQueryPlanStep() const { return query_plan_step; } size_t getQueryPlanStepGroup() const { return query_plan_step_group; } - uint64_t getElapsedUs() const { return elapsed_us; } - uint64_t getInputWaitElapsedUs() const { return input_wait_elapsed_us; } - uint64_t getOutputWaitElapsedUs() const { return output_wait_elapsed_us; } + uint64_t getElapsedNs() const { return elapsed_ns; } + uint64_t getInputWaitElapsedNs() const { return input_wait_elapsed_ns; } + uint64_t getOutputWaitElapsedNs() const { return output_wait_elapsed_ns; } struct ProcessorDataStats { @@ -369,21 +369,21 @@ protected: private: /// For: - /// - elapsed_us + /// - elapsed_ns friend class ExecutionThreadContext; /// For - /// - input_wait_elapsed_us - /// - output_wait_elapsed_us + /// - input_wait_elapsed_ns + /// - output_wait_elapsed_ns friend class ExecutingGraph; std::string processor_description; /// For processors_profile_log - uint64_t elapsed_us = 0; + uint64_t elapsed_ns = 0; Stopwatch input_wait_watch; - uint64_t input_wait_elapsed_us = 0; + uint64_t input_wait_elapsed_ns = 0; Stopwatch output_wait_watch; - uint64_t output_wait_elapsed_us = 0; + uint64_t output_wait_elapsed_ns = 0; size_t stream_number = NO_STREAM; From 8550e64352dc98bf6e8002fbbadbed1b1e014027 Mon Sep 17 00:00:00 2001 From: Azat Khuzhin Date: Mon, 8 Jul 2024 19:03:35 +0200 Subject: [PATCH 325/417] Fix type of input_wait_elapsed_us/input_wait_elapsed_us/elapsed_us (was UInt32) Even though type in the system.processors_profile_log is UInt64, intermediate type was UInt32 (sigh). Signed-off-by: Azat Khuzhin --- src/Interpreters/ProcessorsProfileLog.h | 6 +++--- src/Interpreters/executeQuery.cpp | 7 +++---- 2 files changed, 6 insertions(+), 7 deletions(-) diff --git a/src/Interpreters/ProcessorsProfileLog.h b/src/Interpreters/ProcessorsProfileLog.h index 8319d373f39..abece2604f2 100644 --- a/src/Interpreters/ProcessorsProfileLog.h +++ b/src/Interpreters/ProcessorsProfileLog.h @@ -25,11 +25,11 @@ struct ProcessorProfileLogElement String processor_name; /// Milliseconds spend in IProcessor::work() - UInt32 elapsed_us{}; + UInt64 elapsed_us{}; /// IProcessor::NeedData - UInt32 input_wait_elapsed_us{}; + UInt64 input_wait_elapsed_us{}; /// IProcessor::PortFull - UInt32 output_wait_elapsed_us{}; + UInt64 output_wait_elapsed_us{}; size_t input_rows{}; size_t input_bytes{}; diff --git a/src/Interpreters/executeQuery.cpp b/src/Interpreters/executeQuery.cpp index 336e226a691..d9d3ba58160 100644 --- a/src/Interpreters/executeQuery.cpp +++ b/src/Interpreters/executeQuery.cpp @@ -475,10 +475,9 @@ void logQueryFinish( processor_elem.processor_name = processor->getName(); - /// NOTE: convert this to UInt64 - processor_elem.elapsed_us = static_cast(processor->getElapsedNs() / 1000U); - processor_elem.input_wait_elapsed_us = static_cast(processor->getInputWaitElapsedNs() / 1000U); - processor_elem.output_wait_elapsed_us = static_cast(processor->getOutputWaitElapsedNs() / 1000U); + processor_elem.elapsed_us = static_cast(processor->getElapsedNs() / 1000U); + processor_elem.input_wait_elapsed_us = static_cast(processor->getInputWaitElapsedNs() / 1000U); + processor_elem.output_wait_elapsed_us = static_cast(processor->getOutputWaitElapsedNs() / 1000U); auto stats = processor->getProcessorDataStats(); processor_elem.input_rows = stats.input_rows; From 1e48831d76d90c13cb60fc4e96fdd98a7c42cb2a Mon Sep 17 00:00:00 2001 From: Nikita Taranov Date: Mon, 8 Jul 2024 18:21:06 +0100 Subject: [PATCH 326/417] better --- src/Columns/ColumnAggregateFunction.h | 3 ++- src/Columns/ColumnDynamic.h | 11 +++-------- src/Columns/ColumnMap.h | 12 ++++-------- src/Columns/ColumnNullable.h | 6 ++---- src/Columns/ColumnObject.h | 7 +++---- src/Columns/ColumnTuple.h | 7 +++---- src/Columns/ColumnVariant.h | 16 +++++----------- 7 files changed, 22 insertions(+), 40 deletions(-) diff --git a/src/Columns/ColumnAggregateFunction.h b/src/Columns/ColumnAggregateFunction.h index fada5fc0c68..1be7a862438 100644 --- a/src/Columns/ColumnAggregateFunction.h +++ b/src/Columns/ColumnAggregateFunction.h @@ -148,10 +148,11 @@ public: #if !defined(ABORT_ON_LOGICAL_ERROR) void insertFrom(const IColumn & from, size_t n) override; #else + using IColumn::insertFrom; + void doInsertFrom(const IColumn & from, size_t n) override; #endif - using IColumn::insertFrom; void insertFrom(ConstAggregateDataPtr place); diff --git a/src/Columns/ColumnDynamic.h b/src/Columns/ColumnDynamic.h index eb8a2ad6ed6..9abddc7a26d 100644 --- a/src/Columns/ColumnDynamic.h +++ b/src/Columns/ColumnDynamic.h @@ -142,19 +142,14 @@ public: void insert(const Field & x) override; bool tryInsert(const Field & x) override; + #if !defined(ABORT_ON_LOGICAL_ERROR) void insertFrom(const IColumn & src_, size_t n) override; -#else - void doInsertFrom(const IColumn & src_, size_t n) override; -#endif -#if !defined(ABORT_ON_LOGICAL_ERROR) void insertRangeFrom(const IColumn & src, size_t start, size_t length) override; -#else - void doInsertRangeFrom(const IColumn & src, size_t start, size_t length) override; -#endif -#if !defined(ABORT_ON_LOGICAL_ERROR) void insertManyFrom(const IColumn & src, size_t position, size_t length) override; #else + void doInsertFrom(const IColumn & src_, size_t n) override; + void doInsertRangeFrom(const IColumn & src, size_t start, size_t length) override; void doInsertManyFrom(const IColumn & src, size_t position, size_t length) override; #endif diff --git a/src/Columns/ColumnMap.h b/src/Columns/ColumnMap.h index d15829f4147..a54071a2974 100644 --- a/src/Columns/ColumnMap.h +++ b/src/Columns/ColumnMap.h @@ -66,21 +66,17 @@ public: void updateHashWithValue(size_t n, SipHash & hash) const override; void updateWeakHash32(WeakHash32 & hash) const override; void updateHashFast(SipHash & hash) const override; + #if !defined(ABORT_ON_LOGICAL_ERROR) void insertFrom(const IColumn & src_, size_t n) override; -#else - void doInsertFrom(const IColumn & src_, size_t n) override; -#endif -#if !defined(ABORT_ON_LOGICAL_ERROR) void insertManyFrom(const IColumn & src, size_t position, size_t length) override; -#else - void doInsertManyFrom(const IColumn & src, size_t position, size_t length) override; -#endif -#if !defined(ABORT_ON_LOGICAL_ERROR) void insertRangeFrom(const IColumn & src, size_t start, size_t length) override; #else + void doInsertFrom(const IColumn & src_, size_t n) override; + void doInsertManyFrom(const IColumn & src, size_t position, size_t length) override; void doInsertRangeFrom(const IColumn & src, size_t start, size_t length) override; #endif + ColumnPtr filter(const Filter & filt, ssize_t result_size_hint) const override; void expand(const Filter & mask, bool inverted) override; ColumnPtr permute(const Permutation & perm, size_t limit) const override; diff --git a/src/Columns/ColumnNullable.h b/src/Columns/ColumnNullable.h index f89e67b6000..a6d0483e527 100644 --- a/src/Columns/ColumnNullable.h +++ b/src/Columns/ColumnNullable.h @@ -76,14 +76,12 @@ public: #endif void insert(const Field & x) override; bool tryInsert(const Field & x) override; + #if !defined(ABORT_ON_LOGICAL_ERROR) void insertFrom(const IColumn & src, size_t n) override; -#else - void doInsertFrom(const IColumn & src, size_t n) override; -#endif -#if !defined(ABORT_ON_LOGICAL_ERROR) void insertManyFrom(const IColumn & src, size_t position, size_t length) override; #else + void doInsertFrom(const IColumn & src, size_t n) override; void doInsertManyFrom(const IColumn & src, size_t position, size_t length) override; #endif diff --git a/src/Columns/ColumnObject.h b/src/Columns/ColumnObject.h index 9de2adb87de..7470dfa6302 100644 --- a/src/Columns/ColumnObject.h +++ b/src/Columns/ColumnObject.h @@ -209,16 +209,15 @@ public: void insert(const Field & field) override; bool tryInsert(const Field & field) override; void insertDefault() override; + #if !defined(ABORT_ON_LOGICAL_ERROR) void insertFrom(const IColumn & src, size_t n) override; -#else - void doInsertFrom(const IColumn & src, size_t n) override; -#endif -#if !defined(ABORT_ON_LOGICAL_ERROR) void insertRangeFrom(const IColumn & src, size_t start, size_t length) override; #else + void doInsertFrom(const IColumn & src, size_t n) override; void doInsertRangeFrom(const IColumn & src, size_t start, size_t length) override; #endif + void popBack(size_t length) override; Field operator[](size_t n) const override; void get(size_t n, Field & res) const override; diff --git a/src/Columns/ColumnTuple.h b/src/Columns/ColumnTuple.h index 11c09f827f5..38e479791d4 100644 --- a/src/Columns/ColumnTuple.h +++ b/src/Columns/ColumnTuple.h @@ -65,16 +65,15 @@ public: void insertData(const char * pos, size_t length) override; void insert(const Field & x) override; bool tryInsert(const Field & x) override; + #if !defined(ABORT_ON_LOGICAL_ERROR) void insertFrom(const IColumn & src_, size_t n) override; -#else - void doInsertFrom(const IColumn & src_, size_t n) override; -#endif -#if !defined(ABORT_ON_LOGICAL_ERROR) void insertManyFrom(const IColumn & src, size_t position, size_t length) override; #else + void doInsertFrom(const IColumn & src_, size_t n) override; void doInsertManyFrom(const IColumn & src, size_t position, size_t length) override; #endif + void insertDefault() override; void popBack(size_t n) override; StringRef serializeValueIntoArena(size_t n, Arena & arena, char const *& begin) const override; diff --git a/src/Columns/ColumnVariant.h b/src/Columns/ColumnVariant.h index b492d4b49c7..d91b8e93a7d 100644 --- a/src/Columns/ColumnVariant.h +++ b/src/Columns/ColumnVariant.h @@ -182,24 +182,18 @@ public: #if !defined(ABORT_ON_LOGICAL_ERROR) void insertFrom(const IColumn & src_, size_t n) override; -#else - void doInsertFrom(const IColumn & src_, size_t n) override; -#endif -#if !defined(ABORT_ON_LOGICAL_ERROR) void insertRangeFrom(const IColumn & src_, size_t start, size_t length) override; -#else - void doInsertRangeFrom(const IColumn & src_, size_t start, size_t length) override; -#endif -#if !defined(ABORT_ON_LOGICAL_ERROR) void insertManyFrom(const IColumn & src_, size_t position, size_t length) override; #else - void doInsertManyFrom(const IColumn & src_, size_t position, size_t length) override; -#endif - using IColumn::insertFrom; using IColumn::insertManyFrom; using IColumn::insertRangeFrom; + void doInsertFrom(const IColumn & src_, size_t n) override; + void doInsertRangeFrom(const IColumn & src_, size_t start, size_t length) override; + void doInsertManyFrom(const IColumn & src_, size_t position, size_t length) override; +#endif + /// Methods for insertion from another Variant but with known mapping between global discriminators. void insertFrom(const IColumn & src_, size_t n, const std::vector & global_discriminators_mapping); void insertRangeFrom(const IColumn & src_, size_t start, size_t length, const std::vector & global_discriminators_mapping); From 8a352fdd30190ae342140441d2012754bbdd58c2 Mon Sep 17 00:00:00 2001 From: gun9nir Date: Mon, 8 Jul 2024 10:28:24 -0700 Subject: [PATCH 327/417] add no-parallel tag on the test --- .../queries/0_stateless/03198_table_function_directory_path.sql | 2 ++ 1 file changed, 2 insertions(+) diff --git a/tests/queries/0_stateless/03198_table_function_directory_path.sql b/tests/queries/0_stateless/03198_table_function_directory_path.sql index 671074ab45a..90b81b3fe5e 100644 --- a/tests/queries/0_stateless/03198_table_function_directory_path.sql +++ b/tests/queries/0_stateless/03198_table_function_directory_path.sql @@ -1,3 +1,5 @@ +-- Tags: no-parallel + INSERT INTO FUNCTION file('data_03198_table_function_directory_path/1.csv', 'csv') SELECT '1.csv'; INSERT INTO FUNCTION file('data_03198_table_function_directory_path/2.csv', 'csv') SELECT '2.csv'; INSERT INTO FUNCTION file('data_03198_table_function_directory_path/dir/3.csv', 'csv') SELECT '3.csv'; From fb4fb2ca5e9a69570b99a0dfaf89dd68aaf9022c Mon Sep 17 00:00:00 2001 From: gun9nir Date: Mon, 8 Jul 2024 10:39:35 -0700 Subject: [PATCH 328/417] add docs --- docs/en/sql-reference/table-functions/file.md | 10 +++++++++- 1 file changed, 9 insertions(+), 1 deletion(-) diff --git a/docs/en/sql-reference/table-functions/file.md b/docs/en/sql-reference/table-functions/file.md index 3a3162dad9a..44b1b50620a 100644 --- a/docs/en/sql-reference/table-functions/file.md +++ b/docs/en/sql-reference/table-functions/file.md @@ -130,7 +130,9 @@ SELECT * FROM file('user_files/archives/archive{1..2}.zip :: table.csv'); ## Globs in path -Paths may use globbing. Files must match the whole path pattern, not only the suffix or prefix. +Paths may use globbing. Files must match the whole path pattern, not only the suffix or prefix. There is one exception that if the path refers to an existing +directory and does not use globs, a `*` will be implicitly added to the path so +all the files in the directory are selected. - `*` — Represents arbitrarily many characters except `/` but including the empty string. - `?` — Represents an arbitrary single character. @@ -163,6 +165,12 @@ An alternative path expression which achieves the same: SELECT count(*) FROM file('{some,another}_dir/*', 'TSV', 'name String, value UInt32'); ``` +Query the total number of rows in `some_dir` using the implicit `*`: + +```sql +SELECT count(*) FROM file('some_dir', 'TSV', 'name String, value UInt32'); +``` + :::note If your listing of files contains number ranges with leading zeros, use the construction with braces for each digit separately or use `?`. ::: From 5b9c74b4b88ac4a9592edba8e642fda2e91a904d Mon Sep 17 00:00:00 2001 From: Sema Checherinda Date: Mon, 8 Jul 2024 21:13:30 +0200 Subject: [PATCH 329/417] fix tests, lesser logs --- src/Interpreters/DatabaseCatalog.cpp | 41 +++++++++++++++++++++------- 1 file changed, 31 insertions(+), 10 deletions(-) diff --git a/src/Interpreters/DatabaseCatalog.cpp b/src/Interpreters/DatabaseCatalog.cpp index afe89e7b9a7..2abda981a8c 100644 --- a/src/Interpreters/DatabaseCatalog.cpp +++ b/src/Interpreters/DatabaseCatalog.cpp @@ -1272,14 +1272,23 @@ void DatabaseCatalog::rescheduleDropTableTask() if (first_async_drop_in_queue != tables_marked_dropped.begin()) { + LOG_TRACE( + log, + "Have {} tables in queue to drop. Some of them are being dropped in sync mode. Schedule background task ASAP", + tables_marked_dropped.size()); (*drop_task)->scheduleAfter(0); return; } time_t current_time = std::chrono::system_clock::to_time_t(std::chrono::system_clock::now()); auto min_drop_time = getMinDropTime(); - time_t schedule_after = min_drop_time < current_time ? (min_drop_time - current_time) * 1000 : 0; - (*drop_task)->scheduleAfter(schedule_after); + time_t schedule_after_ms = min_drop_time > current_time ? (min_drop_time - current_time) * 1000 : 0; + + LOG_TRACE( + log, + "Have {} tables in queue to drop. Schedule background task in {} seconds", + tables_marked_dropped.size(), schedule_after_ms / 1000); + (*drop_task)->scheduleAfter(schedule_after_ms); } void DatabaseCatalog::dropTablesParallel(std::vector tables_to_drop) @@ -1297,7 +1306,7 @@ void DatabaseCatalog::dropTablesParallel(std::vector Date: Mon, 8 Jul 2024 10:14:09 +0000 Subject: [PATCH 330/417] slightly better --- src/Processors/QueryPlan/BufferChunksTransform.cpp | 11 ++++++----- src/Processors/QueryPlan/BufferChunksTransform.h | 6 ++++++ 2 files changed, 12 insertions(+), 5 deletions(-) diff --git a/src/Processors/QueryPlan/BufferChunksTransform.cpp b/src/Processors/QueryPlan/BufferChunksTransform.cpp index 283a0acf172..3601a68d36e 100644 --- a/src/Processors/QueryPlan/BufferChunksTransform.cpp +++ b/src/Processors/QueryPlan/BufferChunksTransform.cpp @@ -26,6 +26,12 @@ IProcessor::Status BufferChunksTransform::prepare() return Status::Finished; } + if (input.isFinished() && chunks.empty()) + { + output.finish(); + return Status::Finished; + } + if (output.canPush()) { input.setNeeded(); @@ -45,11 +51,6 @@ IProcessor::Status BufferChunksTransform::prepare() auto chunk = pullChunk(); output.push(std::move(chunk)); } - else if (input.isFinished()) - { - output.finish(); - return Status::Finished; - } } if (input.hasData() && (num_buffered_rows < max_rows_to_buffer || num_buffered_bytes < max_bytes_to_buffer)) diff --git a/src/Processors/QueryPlan/BufferChunksTransform.h b/src/Processors/QueryPlan/BufferChunksTransform.h index 84c35431364..752f9910734 100644 --- a/src/Processors/QueryPlan/BufferChunksTransform.h +++ b/src/Processors/QueryPlan/BufferChunksTransform.h @@ -5,9 +5,15 @@ namespace DB { +/// Transform that buffers chunks from the input +/// up to the certain limit and pushes chunks to +/// the output whenever it is ready. It can be used +/// to increase parallelism of execution, for example +/// when it is adeded before MergingSortedTransform. class BufferChunksTransform : public IProcessor { public: + /// OR condition is used for the limits on rows and bytes. BufferChunksTransform( const Block & header_, size_t max_rows_to_buffer_, From be343c7dddfa388f46282e0aa3cf264b2f1e0f2f Mon Sep 17 00:00:00 2001 From: Alexander Tokmakov Date: Mon, 8 Jul 2024 14:08:03 +0200 Subject: [PATCH 331/417] don't throw TIMEOUT_EXCEEDED for none_only_active --- src/Interpreters/executeDDLQueryOnCluster.cpp | 7 ++++--- .../02447_drop_database_replica.reference | 17 +++++++++++++-- .../02447_drop_database_replica.sh | 21 +++++++++++++++++-- 3 files changed, 38 insertions(+), 7 deletions(-) diff --git a/src/Interpreters/executeDDLQueryOnCluster.cpp b/src/Interpreters/executeDDLQueryOnCluster.cpp index e372f036073..9c3f85128cf 100644 --- a/src/Interpreters/executeDDLQueryOnCluster.cpp +++ b/src/Interpreters/executeDDLQueryOnCluster.cpp @@ -237,6 +237,7 @@ private: Int64 timeout_seconds = 120; bool is_replicated_database = false; bool throw_on_timeout = true; + bool throw_on_timeout_only_active = false; bool only_running_hosts = false; bool timeout_exceeded = false; @@ -316,8 +317,8 @@ DDLQueryStatusSource::DDLQueryStatusSource( , log(getLogger("DDLQueryStatusSource")) { auto output_mode = context->getSettingsRef().distributed_ddl_output_mode; - throw_on_timeout = output_mode == DistributedDDLOutputMode::THROW || output_mode == DistributedDDLOutputMode::THROW_ONLY_ACTIVE - || output_mode == DistributedDDLOutputMode::NONE || output_mode == DistributedDDLOutputMode::NONE_ONLY_ACTIVE; + throw_on_timeout = output_mode == DistributedDDLOutputMode::THROW || output_mode == DistributedDDLOutputMode::NONE; + throw_on_timeout_only_active = output_mode == DistributedDDLOutputMode::THROW_ONLY_ACTIVE || output_mode == DistributedDDLOutputMode::NONE_ONLY_ACTIVE; if (hosts_to_wait) { @@ -451,7 +452,7 @@ Chunk DDLQueryStatusSource::generate() "({} of them are currently executing the task, {} are inactive). " "They are going to execute the query in background. Was waiting for {} seconds{}"; - if (throw_on_timeout) + if (throw_on_timeout || (throw_on_timeout_only_active && !stop_waiting_offline_hosts)) { if (!first_exception) first_exception = std::make_unique(Exception(ErrorCodes::TIMEOUT_EXCEEDED, diff --git a/tests/queries/0_stateless/02447_drop_database_replica.reference b/tests/queries/0_stateless/02447_drop_database_replica.reference index bd3b689ca3c..d997b7ba830 100644 --- a/tests/queries/0_stateless/02447_drop_database_replica.reference +++ b/tests/queries/0_stateless/02447_drop_database_replica.reference @@ -12,11 +12,21 @@ t 2 rdb_default 1 1 s1 r1 1 2 -2 -2 +skip inactive s1 r1 OK 2 0 s1 r2 QUEUED 2 0 s2 r1 QUEUED 2 0 +s1 r1 OK 2 0 +s1 r2 QUEUED 2 0 +s2 r1 QUEUED 2 0 +timeout on active +2 +2 +s1 r1 OK 3 0 +s1 r2 QUEUED 3 0 +s2 r1 QUEUED 3 0 +s9 r9 QUEUED 3 0 +drop replica 2 rdb_default 1 1 s1 r1 1 rdb_default 1 2 s1 r2 0 @@ -24,6 +34,9 @@ rdb_default 1 2 s1 r2 0 2 t t2 +t22 t3 +t33 t4 +t44 rdb_default_4 1 1 s1 r1 1 diff --git a/tests/queries/0_stateless/02447_drop_database_replica.sh b/tests/queries/0_stateless/02447_drop_database_replica.sh index 1604d527f2b..93a5fcee8e2 100755 --- a/tests/queries/0_stateless/02447_drop_database_replica.sh +++ b/tests/queries/0_stateless/02447_drop_database_replica.sh @@ -33,10 +33,27 @@ $CLICKHOUSE_CLIENT -q "select cluster, shard_num, replica_num, database_shard_na $CLICKHOUSE_CLIENT -q "system drop database replica 's1|r1' from database $db2" 2>&1| grep -Fac "is active, cannot drop it" # Also check that it doesn't exceed distributed_ddl_task_timeout waiting for inactive replicas -timeout 60s $CLICKHOUSE_CLIENT --distributed_ddl_task_timeout=1000 --distributed_ddl_output_mode=none_only_active -q "create table $db.t2 (n int) engine=Log" 2>&1| grep -Fac "TIMEOUT_EXCEEDED" -timeout 60s $CLICKHOUSE_CLIENT --distributed_ddl_task_timeout=1000 --distributed_ddl_output_mode=throw_only_active -q "create table $db.t3 (n int) engine=Log" 2>&1| grep -Fac "TIMEOUT_EXCEEDED" +echo 'skip inactive' +timeout 60s $CLICKHOUSE_CLIENT --distributed_ddl_task_timeout=1000 --distributed_ddl_output_mode=none_only_active -q "create table $db.t2 (n int) engine=Log" +timeout 60s $CLICKHOUSE_CLIENT --distributed_ddl_task_timeout=1000 --distributed_ddl_output_mode=throw_only_active -q "create table $db.t3 (n int) engine=Log" | sort timeout 60s $CLICKHOUSE_CLIENT --distributed_ddl_task_timeout=1000 --distributed_ddl_output_mode=null_status_on_timeout_only_active -q "create table $db.t4 (n int) engine=Log" | sort +# And that it still throws TIMEOUT_EXCEEDED for active replicas +echo 'timeout on active' +db9="${db}_9" +$CLICKHOUSE_CLIENT -q "create database $db9 engine=Replicated('/test/$CLICKHOUSE_DATABASE/rdb', 's9', 'r9')" +$CLICKHOUSE_CLIENT -q "detach database $db9" +$CLICKHOUSE_CLIENT -q "insert into system.zookeeper(name, path, value) values ('active', '/test/$CLICKHOUSE_DATABASE/rdb/replicas/s9|r9', '$($CLICKHOUSE_CLIENT -q "select serverUUID()")')" + +$CLICKHOUSE_CLIENT --distributed_ddl_task_timeout=5 --distributed_ddl_output_mode=none_only_active -q "create table $db.t22 (n int) engine=Log" 2>&1| grep -Fac "TIMEOUT_EXCEEDED" +$CLICKHOUSE_CLIENT --distributed_ddl_task_timeout=5 --distributed_ddl_output_mode=throw_only_active -q "create table $db.t33 (n int) engine=Log" 2>&1| grep -Fac "TIMEOUT_EXCEEDED" +$CLICKHOUSE_CLIENT --distributed_ddl_task_timeout=5 --distributed_ddl_output_mode=null_status_on_timeout_only_active -q "create table $db.t44 (n int) engine=Log" | sort + +$CLICKHOUSE_CLIENT -q "attach database $db9" +$CLICKHOUSE_CLIENT -q "drop database $db9" + +echo 'drop replica' + $CLICKHOUSE_CLIENT -q "detach database $db3" $CLICKHOUSE_CLIENT -q "system drop database replica 'r1' from shard 's2' from database $db" $CLICKHOUSE_CLIENT -q "attach database $db3" 2>/dev/null From 708c81f6e2a81b530a47f543795bb8ff6ec663ed Mon Sep 17 00:00:00 2001 From: Anton Popov Date: Tue, 9 Jul 2024 01:20:31 +0200 Subject: [PATCH 332/417] Update 00504_mergetree_arrays_rw.sql --- tests/queries/0_stateless/00504_mergetree_arrays_rw.sql | 3 +++ 1 file changed, 3 insertions(+) diff --git a/tests/queries/0_stateless/00504_mergetree_arrays_rw.sql b/tests/queries/0_stateless/00504_mergetree_arrays_rw.sql index 7c939d060ea..14929045356 100644 --- a/tests/queries/0_stateless/00504_mergetree_arrays_rw.sql +++ b/tests/queries/0_stateless/00504_mergetree_arrays_rw.sql @@ -1,5 +1,8 @@ set allow_deprecated_syntax_for_merge_tree=1; +set max_threads = 1; +set max_insert_threads = 1; + drop table if exists test_ins_arr; create table test_ins_arr (date Date, val Array(UInt64)) engine = MergeTree(date, (date), 8192); insert into test_ins_arr select toDate('2017-10-02'), [number, 42] from system.numbers limit 10000; From 05fe5c9f2ca14c9704a965c8224f6d49bfcbbd42 Mon Sep 17 00:00:00 2001 From: gun9nir Date: Mon, 8 Jul 2024 19:04:08 -0700 Subject: [PATCH 333/417] truncate file in insert --- .../03198_table_function_directory_path.sql | 10 +++++----- 1 file changed, 5 insertions(+), 5 deletions(-) diff --git a/tests/queries/0_stateless/03198_table_function_directory_path.sql b/tests/queries/0_stateless/03198_table_function_directory_path.sql index 90b81b3fe5e..9e2791847af 100644 --- a/tests/queries/0_stateless/03198_table_function_directory_path.sql +++ b/tests/queries/0_stateless/03198_table_function_directory_path.sql @@ -1,10 +1,10 @@ -- Tags: no-parallel -INSERT INTO FUNCTION file('data_03198_table_function_directory_path/1.csv', 'csv') SELECT '1.csv'; -INSERT INTO FUNCTION file('data_03198_table_function_directory_path/2.csv', 'csv') SELECT '2.csv'; -INSERT INTO FUNCTION file('data_03198_table_function_directory_path/dir/3.csv', 'csv') SELECT '3.csv'; -INSERT INTO FUNCTION file('data_03198_table_function_directory_path/dir1/dir/4.csv', 'csv') SELECT '4.csv'; -INSERT INTO FUNCTION file('data_03198_table_function_directory_path/dir2/dir/5.csv', 'csv') SELECT '5.csv'; +INSERT INTO FUNCTION file('data_03198_table_function_directory_path/1.csv', 'csv') SELECT '1.csv' SETTINGS engine_file_truncate_on_insert=1; +INSERT INTO FUNCTION file('data_03198_table_function_directory_path/2.csv', 'csv') SELECT '2.csv' SETTINGS engine_file_truncate_on_insert=1; +INSERT INTO FUNCTION file('data_03198_table_function_directory_path/dir/3.csv', 'csv') SELECT '3.csv' SETTINGS engine_file_truncate_on_insert=1; +INSERT INTO FUNCTION file('data_03198_table_function_directory_path/dir1/dir/4.csv', 'csv') SELECT '4.csv' SETTINGS engine_file_truncate_on_insert=1; +INSERT INTO FUNCTION file('data_03198_table_function_directory_path/dir2/dir/5.csv', 'csv') SELECT '5.csv' SETTINGS engine_file_truncate_on_insert=1; SELECT COUNT(*) FROM file('data_03198_table_function_directory_path'); SELECT COUNT(*) FROM file('data_03198_table_function_directory_path/'); From 9153e65456dd3a90d9be85c1a8a52592ce054e77 Mon Sep 17 00:00:00 2001 From: Kruglov Pavel <48961922+Avogar@users.noreply.github.com> Date: Tue, 9 Jul 2024 10:15:41 +0200 Subject: [PATCH 334/417] Remove unneded include --- src/DataTypes/DataTypeDynamic.cpp | 1 - 1 file changed, 1 deletion(-) diff --git a/src/DataTypes/DataTypeDynamic.cpp b/src/DataTypes/DataTypeDynamic.cpp index 6826c46a1a7..c920e69c13b 100644 --- a/src/DataTypes/DataTypeDynamic.cpp +++ b/src/DataTypes/DataTypeDynamic.cpp @@ -12,7 +12,6 @@ #include #include #include -#include namespace DB { From b4dd700c26c43a6910c7394e0bdb246afef47884 Mon Sep 17 00:00:00 2001 From: Kruglov Pavel <48961922+Avogar@users.noreply.github.com> Date: Tue, 9 Jul 2024 12:26:39 +0200 Subject: [PATCH 335/417] Fix typos --- .../en/sql-reference/data-types/data-types-binary-encoding.md | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/docs/en/sql-reference/data-types/data-types-binary-encoding.md b/docs/en/sql-reference/data-types/data-types-binary-encoding.md index c9720da0f1c..812e946e43e 100644 --- a/docs/en/sql-reference/data-types/data-types-binary-encoding.md +++ b/docs/en/sql-reference/data-types/data-types-binary-encoding.md @@ -9,7 +9,7 @@ sidebar_label: Data types binary encoding specification. This specification describes the binary format that can be used for binary encoding and decoding of ClickHouse data types. This format is used in `Dynamic` column [binary serialization](dynamic.md#binary-output-format) and can be used in input/output formats [RowBinaryWithNamesAndTypes](../../interfaces/formats.md#rowbinarywithnamesandtypes) and [Native](../../interfaces/formats.md#native) under corresponding settings. -The table below describes how each data type is represented in bunary format. Each data type encoding consist of 1 byte that indicates the type and some optional additional information. +The table below describes how each data type is represented in binary format. Each data type encoding consist of 1 byte that indicates the type and some optional additional information. `var_uint` in the binary encoding means that the size is encoded using Variable-Length Quantity compression. | ClickHouse data type | Binary encoding | @@ -66,7 +66,7 @@ The table below describes how each data type is represented in bunary format. Ea ### Interval kind binary encoding -The table below describes how different interval kinds of `Interval` data type are endoced. +The table below describes how different interval kinds of `Interval` data type are encoded. | Interval kind | Binary encoding | |---------------|-----------------| From 58eb57dffb6be5033beb6dd79d05a3292512a9f9 Mon Sep 17 00:00:00 2001 From: Dmitry Novik Date: Mon, 8 Jul 2024 15:43:16 +0200 Subject: [PATCH 336/417] Add settings to control connection to PostgreSQL --- src/Core/PostgreSQL/PoolWithFailover.cpp | 23 +++++++++++++++---- src/Core/PostgreSQL/PoolWithFailover.h | 7 +++--- src/Core/PostgreSQL/Utils.cpp | 4 ++-- src/Core/PostgreSQL/Utils.h | 2 +- src/Core/Settings.h | 2 ++ src/Core/SettingsChangesHistory.cpp | 2 ++ .../DatabaseMaterializedPostgreSQL.cpp | 7 +++++- .../PostgreSQL/DatabasePostgreSQL.cpp | 5 ++-- .../PostgreSQLDictionarySource.cpp | 5 ++-- .../StorageMaterializedPostgreSQL.cpp | 3 ++- src/Storages/StorageExternalDistributed.cpp | 5 ++-- src/Storages/StoragePostgreSQL.cpp | 5 ++-- .../TableFunctionPostgreSQL.cpp | 5 ++-- 13 files changed, 52 insertions(+), 23 deletions(-) diff --git a/src/Core/PostgreSQL/PoolWithFailover.cpp b/src/Core/PostgreSQL/PoolWithFailover.cpp index a034c50094d..5014564dbe0 100644 --- a/src/Core/PostgreSQL/PoolWithFailover.cpp +++ b/src/Core/PostgreSQL/PoolWithFailover.cpp @@ -27,7 +27,8 @@ PoolWithFailover::PoolWithFailover( size_t pool_size, size_t pool_wait_timeout_, size_t max_tries_, - bool auto_close_connection_) + bool auto_close_connection_, + size_t connection_attempt_timeout_) : pool_wait_timeout(pool_wait_timeout_) , max_tries(max_tries_) , auto_close_connection(auto_close_connection_) @@ -39,8 +40,13 @@ PoolWithFailover::PoolWithFailover( { for (const auto & replica_configuration : configurations) { - auto connection_info = formatConnectionString(replica_configuration.database, - replica_configuration.host, replica_configuration.port, replica_configuration.username, replica_configuration.password); + auto connection_info = formatConnectionString( + replica_configuration.database, + replica_configuration.host, + replica_configuration.port, + replica_configuration.username, + replica_configuration.password, + connection_attempt_timeout_); replicas_with_priority[priority].emplace_back(connection_info, pool_size); } } @@ -51,7 +57,8 @@ PoolWithFailover::PoolWithFailover( size_t pool_size, size_t pool_wait_timeout_, size_t max_tries_, - bool auto_close_connection_) + bool auto_close_connection_, + size_t connection_attempt_timeout_) : pool_wait_timeout(pool_wait_timeout_) , max_tries(max_tries_) , auto_close_connection(auto_close_connection_) @@ -63,7 +70,13 @@ PoolWithFailover::PoolWithFailover( for (const auto & [host, port] : configuration.addresses) { LOG_DEBUG(getLogger("PostgreSQLPoolWithFailover"), "Adding address host: {}, port: {} to connection pool", host, port); - auto connection_string = formatConnectionString(configuration.database, host, port, configuration.username, configuration.password); + auto connection_string = formatConnectionString( + configuration.database, + host, + port, + configuration.username, + configuration.password, + connection_attempt_timeout_); replicas_with_priority[0].emplace_back(connection_string, pool_size); } } diff --git a/src/Core/PostgreSQL/PoolWithFailover.h b/src/Core/PostgreSQL/PoolWithFailover.h index 3c538fc3dea..502a9a9b7d7 100644 --- a/src/Core/PostgreSQL/PoolWithFailover.h +++ b/src/Core/PostgreSQL/PoolWithFailover.h @@ -14,7 +14,6 @@ static constexpr inline auto POSTGRESQL_POOL_DEFAULT_SIZE = 16; static constexpr inline auto POSTGRESQL_POOL_WAIT_TIMEOUT = 5000; -static constexpr inline auto POSTGRESQL_POOL_WITH_FAILOVER_DEFAULT_MAX_TRIES = 2; namespace postgres { @@ -30,14 +29,16 @@ public: size_t pool_size, size_t pool_wait_timeout, size_t max_tries_, - bool auto_close_connection_); + bool auto_close_connection_, + size_t connection_attempt_timeout_); explicit PoolWithFailover( const DB::StoragePostgreSQL::Configuration & configuration, size_t pool_size, size_t pool_wait_timeout, size_t max_tries_, - bool auto_close_connection_); + bool auto_close_connection_, + size_t connection_attempt_timeout_); PoolWithFailover(const PoolWithFailover & other) = delete; diff --git a/src/Core/PostgreSQL/Utils.cpp b/src/Core/PostgreSQL/Utils.cpp index 810bf62fdab..9dc010c1c69 100644 --- a/src/Core/PostgreSQL/Utils.cpp +++ b/src/Core/PostgreSQL/Utils.cpp @@ -8,7 +8,7 @@ namespace postgres { -ConnectionInfo formatConnectionString(String dbname, String host, UInt16 port, String user, String password) +ConnectionInfo formatConnectionString(String dbname, String host, UInt16 port, String user, String password, UInt64 timeout) { DB::WriteBufferFromOwnString out; out << "dbname=" << DB::quote << dbname @@ -16,7 +16,7 @@ ConnectionInfo formatConnectionString(String dbname, String host, UInt16 port, S << " port=" << port << " user=" << DB::quote << user << " password=" << DB::quote << password - << " connect_timeout=2"; + << " connect_timeout=" << timeout; return {out.str(), host + ':' + DB::toString(port)}; } diff --git a/src/Core/PostgreSQL/Utils.h b/src/Core/PostgreSQL/Utils.h index f179ab14c89..f2b8f1ac084 100644 --- a/src/Core/PostgreSQL/Utils.h +++ b/src/Core/PostgreSQL/Utils.h @@ -18,7 +18,7 @@ namespace pqxx namespace postgres { -ConnectionInfo formatConnectionString(String dbname, String host, UInt16 port, String user, String password); +ConnectionInfo formatConnectionString(String dbname, String host, UInt16 port, String user, String password, UInt64 timeout); String getConnectionForLog(const String & host, UInt16 port); diff --git a/src/Core/Settings.h b/src/Core/Settings.h index 6764076349a..cb1dbd2929e 100644 --- a/src/Core/Settings.h +++ b/src/Core/Settings.h @@ -561,7 +561,9 @@ class IColumn; M(UInt64, max_partition_size_to_drop, 50000000000lu, "Same as max_table_size_to_drop, but for the partitions.", 0) \ \ M(UInt64, postgresql_connection_pool_size, 16, "Connection pool size for PostgreSQL table engine and database engine.", 0) \ + M(UInt64, postgresql_connection_attempt_timeout, 2, "Connection timeout to PostgreSQL table engine and database engine in seconds.", 0) \ M(UInt64, postgresql_connection_pool_wait_timeout, 5000, "Connection pool push/pop timeout on empty pool for PostgreSQL table engine and database engine. By default it will block on empty pool.", 0) \ + M(UInt64, postgresql_connection_pool_retries, 2, "Connection pool push/pop retries number for PostgreSQL table engine and database engine.", 0) \ M(Bool, postgresql_connection_pool_auto_close_connection, false, "Close connection before returning connection to the pool.", 0) \ M(UInt64, glob_expansion_max_elements, 1000, "Maximum number of allowed addresses (For external storages, table functions, etc).", 0) \ M(UInt64, odbc_bridge_connection_pool_size, 16, "Connection pool size for each connection settings string in ODBC bridge.", 0) \ diff --git a/src/Core/SettingsChangesHistory.cpp b/src/Core/SettingsChangesHistory.cpp index 2e2853a9170..3bae67e7a43 100644 --- a/src/Core/SettingsChangesHistory.cpp +++ b/src/Core/SettingsChangesHistory.cpp @@ -69,6 +69,8 @@ static std::initializer_listgetSettingsRef().postgresql_connection_attempt_timeout); auto postgresql_replica_settings = std::make_unique(); if (engine_define->settings) diff --git a/src/Databases/PostgreSQL/DatabasePostgreSQL.cpp b/src/Databases/PostgreSQL/DatabasePostgreSQL.cpp index 136fb7fd6d2..b22356bebea 100644 --- a/src/Databases/PostgreSQL/DatabasePostgreSQL.cpp +++ b/src/Databases/PostgreSQL/DatabasePostgreSQL.cpp @@ -545,8 +545,9 @@ void registerDatabasePostgreSQL(DatabaseFactory & factory) configuration, settings.postgresql_connection_pool_size, settings.postgresql_connection_pool_wait_timeout, - POSTGRESQL_POOL_WITH_FAILOVER_DEFAULT_MAX_TRIES, - settings.postgresql_connection_pool_auto_close_connection); + settings.postgresql_connection_pool_retries, + settings.postgresql_connection_pool_auto_close_connection, + settings.postgresql_connection_attempt_timeout); return std::make_shared( args.context, diff --git a/src/Dictionaries/PostgreSQLDictionarySource.cpp b/src/Dictionaries/PostgreSQLDictionarySource.cpp index c7401386e40..b35e14577a8 100644 --- a/src/Dictionaries/PostgreSQLDictionarySource.cpp +++ b/src/Dictionaries/PostgreSQLDictionarySource.cpp @@ -205,8 +205,9 @@ void registerDictionarySourcePostgreSQL(DictionarySourceFactory & factory) configuration.replicas_configurations, settings.postgresql_connection_pool_size, settings.postgresql_connection_pool_wait_timeout, - POSTGRESQL_POOL_WITH_FAILOVER_DEFAULT_MAX_TRIES, - settings.postgresql_connection_pool_auto_close_connection); + settings.postgresql_connection_pool_retries, + settings.postgresql_connection_pool_auto_close_connection, + settings.postgresql_connection_attempt_timeout); PostgreSQLDictionarySource::Configuration dictionary_configuration { diff --git a/src/Storages/PostgreSQL/StorageMaterializedPostgreSQL.cpp b/src/Storages/PostgreSQL/StorageMaterializedPostgreSQL.cpp index f686fbda664..b9edff39b82 100644 --- a/src/Storages/PostgreSQL/StorageMaterializedPostgreSQL.cpp +++ b/src/Storages/PostgreSQL/StorageMaterializedPostgreSQL.cpp @@ -592,7 +592,8 @@ void registerStorageMaterializedPostgreSQL(StorageFactory & factory) auto configuration = StoragePostgreSQL::getConfiguration(args.engine_args, args.getContext()); auto connection_info = postgres::formatConnectionString( configuration.database, configuration.host, configuration.port, - configuration.username, configuration.password); + configuration.username, configuration.password, + args.getContext()->getSettingsRef().postgresql_connection_attempt_timeout); bool has_settings = args.storage_def->settings; auto postgresql_replication_settings = std::make_unique(); diff --git a/src/Storages/StorageExternalDistributed.cpp b/src/Storages/StorageExternalDistributed.cpp index beb93afc972..d712bd10da4 100644 --- a/src/Storages/StorageExternalDistributed.cpp +++ b/src/Storages/StorageExternalDistributed.cpp @@ -167,8 +167,9 @@ void registerStorageExternalDistributed(StorageFactory & factory) current_configuration, settings.postgresql_connection_pool_size, settings.postgresql_connection_pool_wait_timeout, - POSTGRESQL_POOL_WITH_FAILOVER_DEFAULT_MAX_TRIES, - settings.postgresql_connection_pool_auto_close_connection); + settings.postgresql_connection_pool_retries, + settings.postgresql_connection_pool_auto_close_connection, + settings.postgresql_connection_attempt_timeout); shards.insert(std::make_shared( args.table_id, std::move(pool), configuration.table, args.columns, args.constraints, String{}, context)); } diff --git a/src/Storages/StoragePostgreSQL.cpp b/src/Storages/StoragePostgreSQL.cpp index a8713c61e4d..b5a388e8159 100644 --- a/src/Storages/StoragePostgreSQL.cpp +++ b/src/Storages/StoragePostgreSQL.cpp @@ -613,8 +613,9 @@ void registerStoragePostgreSQL(StorageFactory & factory) auto pool = std::make_shared(configuration, settings.postgresql_connection_pool_size, settings.postgresql_connection_pool_wait_timeout, - POSTGRESQL_POOL_WITH_FAILOVER_DEFAULT_MAX_TRIES, - settings.postgresql_connection_pool_auto_close_connection); + settings.postgresql_connection_pool_retries, + settings.postgresql_connection_pool_auto_close_connection, + settings.postgresql_connection_attempt_timeout); return std::make_shared( args.table_id, diff --git a/src/TableFunctions/TableFunctionPostgreSQL.cpp b/src/TableFunctions/TableFunctionPostgreSQL.cpp index 8d94988cd65..508f85df6a3 100644 --- a/src/TableFunctions/TableFunctionPostgreSQL.cpp +++ b/src/TableFunctions/TableFunctionPostgreSQL.cpp @@ -80,8 +80,9 @@ void TableFunctionPostgreSQL::parseArguments(const ASTPtr & ast_function, Contex *configuration, settings.postgresql_connection_pool_size, settings.postgresql_connection_pool_wait_timeout, - POSTGRESQL_POOL_WITH_FAILOVER_DEFAULT_MAX_TRIES, - settings.postgresql_connection_pool_auto_close_connection); + settings.postgresql_connection_pool_retries, + settings.postgresql_connection_pool_auto_close_connection, + settings.postgresql_connection_attempt_timeout); } } From eeac41fa455cf1886f535259f485da17f70276f1 Mon Sep 17 00:00:00 2001 From: Dmitry Novik Date: Mon, 8 Jul 2024 16:17:22 +0200 Subject: [PATCH 337/417] Add a documentation --- docs/en/operations/settings/settings.md | 13 +++++++++++++ 1 file changed, 13 insertions(+) diff --git a/docs/en/operations/settings/settings.md b/docs/en/operations/settings/settings.md index ed0b29aa851..c3f697c3bdc 100644 --- a/docs/en/operations/settings/settings.md +++ b/docs/en/operations/settings/settings.md @@ -1358,12 +1358,25 @@ Connection pool size for PostgreSQL table engine and database engine. Default value: 16 +## postgresql_connection_attempt_timeout {#postgresql-connection-attempt-timeout} + +Connection timeout in seconds of a single attempt to connect PostgreSQL end-point. +The value is passed as a `connect_timeout` parameter of the connection URL. + +Default value: `2`. + ## postgresql_connection_pool_wait_timeout {#postgresql-connection-pool-wait-timeout} Connection pool push/pop timeout on empty pool for PostgreSQL table engine and database engine. By default it will block on empty pool. Default value: 5000 +## postgresql_connection_pool_retries {#postgresql-connection-pool-retries} + +The maximum number of retries to establish a connection with the PostgreSQL end-point. + +Default value: `2`. + ## postgresql_connection_pool_auto_close_connection {#postgresql-connection-pool-auto-close-connection} Close connection before returning connection to the pool. From 0174a43d17b61cbb0d57eeeea5068eac463b31e3 Mon Sep 17 00:00:00 2001 From: Nikolai Kochetov Date: Tue, 9 Jul 2024 11:54:21 +0000 Subject: [PATCH 338/417] Fixing Not-ready set in constraint. --- .../Transforms/CheckConstraintsTransform.cpp | 6 ++++++ .../Transforms/CheckConstraintsTransform.h | 1 + .../System/StorageSystemDetachedParts.cpp | 2 +- .../System/StorageSystemPartsBase.cpp | 4 ++-- src/Storages/VirtualColumnUtils.cpp | 6 +++--- src/Storages/VirtualColumnUtils.h | 2 +- .../02841_not_ready_set_constraints.reference | 1 + .../02841_not_ready_set_constraints.sql | 19 +++++++++++++++++++ 8 files changed, 34 insertions(+), 7 deletions(-) create mode 100644 tests/queries/0_stateless/02841_not_ready_set_constraints.reference create mode 100644 tests/queries/0_stateless/02841_not_ready_set_constraints.sql diff --git a/src/Processors/Transforms/CheckConstraintsTransform.cpp b/src/Processors/Transforms/CheckConstraintsTransform.cpp index e43aa6028da..cdae8c23a3e 100644 --- a/src/Processors/Transforms/CheckConstraintsTransform.cpp +++ b/src/Processors/Transforms/CheckConstraintsTransform.cpp @@ -10,6 +10,7 @@ #include #include #include +#include namespace DB @@ -31,6 +32,7 @@ CheckConstraintsTransform::CheckConstraintsTransform( , table_id(table_id_) , constraints_to_check(constraints_.filterConstraints(ConstraintsDescription::ConstraintType::CHECK)) , expressions(constraints_.getExpressions(context_, header.getNamesAndTypesList())) + , context(std::move(context_)) { } @@ -39,6 +41,10 @@ void CheckConstraintsTransform::onConsume(Chunk chunk) { if (chunk.getNumRows() > 0) { + if (rows_written == 0) + for (const auto & expression : expressions) + VirtualColumnUtils::buildSetsForDAG(expression->getActionsDAG(), context); + Block block_to_calculate = getInputPort().getHeader().cloneWithColumns(chunk.getColumns()); for (size_t i = 0; i < expressions.size(); ++i) { diff --git a/src/Processors/Transforms/CheckConstraintsTransform.h b/src/Processors/Transforms/CheckConstraintsTransform.h index 09833ff396b..f92d0ab855e 100644 --- a/src/Processors/Transforms/CheckConstraintsTransform.h +++ b/src/Processors/Transforms/CheckConstraintsTransform.h @@ -35,6 +35,7 @@ private: StorageID table_id; const ASTs constraints_to_check; const ConstraintsExpressions expressions; + ContextPtr context; size_t rows_written = 0; Chunk cur_chunk; }; diff --git a/src/Storages/System/StorageSystemDetachedParts.cpp b/src/Storages/System/StorageSystemDetachedParts.cpp index f48a8c67971..fbc99ab865e 100644 --- a/src/Storages/System/StorageSystemDetachedParts.cpp +++ b/src/Storages/System/StorageSystemDetachedParts.cpp @@ -328,7 +328,7 @@ void ReadFromSystemDetachedParts::applyFilters(ActionDAGNodes added_filter_nodes filter = VirtualColumnUtils::splitFilterDagForAllowedInputs(predicate, &block); if (filter) - VirtualColumnUtils::buildSetsForDAG(filter, context); + VirtualColumnUtils::buildSetsForDAG(*filter, context); } } diff --git a/src/Storages/System/StorageSystemPartsBase.cpp b/src/Storages/System/StorageSystemPartsBase.cpp index 175c0834bcb..f7d1c1b3eb8 100644 --- a/src/Storages/System/StorageSystemPartsBase.cpp +++ b/src/Storages/System/StorageSystemPartsBase.cpp @@ -274,7 +274,7 @@ void ReadFromSystemPartsBase::applyFilters(ActionDAGNodes added_filter_nodes) filter_by_database = VirtualColumnUtils::splitFilterDagForAllowedInputs(predicate, &block); if (filter_by_database) - VirtualColumnUtils::buildSetsForDAG(filter_by_database, context); + VirtualColumnUtils::buildSetsForDAG(*filter_by_database, context); block.insert(ColumnWithTypeAndName({}, std::make_shared(), table_column_name)); block.insert(ColumnWithTypeAndName({}, std::make_shared(), engine_column_name)); @@ -283,7 +283,7 @@ void ReadFromSystemPartsBase::applyFilters(ActionDAGNodes added_filter_nodes) filter_by_other_columns = VirtualColumnUtils::splitFilterDagForAllowedInputs(predicate, &block); if (filter_by_other_columns) - VirtualColumnUtils::buildSetsForDAG(filter_by_other_columns, context); + VirtualColumnUtils::buildSetsForDAG(*filter_by_other_columns, context); } } diff --git a/src/Storages/VirtualColumnUtils.cpp b/src/Storages/VirtualColumnUtils.cpp index 778c9e13adb..27c52124e9c 100644 --- a/src/Storages/VirtualColumnUtils.cpp +++ b/src/Storages/VirtualColumnUtils.cpp @@ -54,9 +54,9 @@ namespace DB namespace VirtualColumnUtils { -void buildSetsForDAG(const ActionsDAGPtr & dag, const ContextPtr & context) +void buildSetsForDAG(const ActionsDAG & dag, const ContextPtr & context) { - for (const auto & node : dag->getNodes()) + for (const auto & node : dag.getNodes()) { if (node.type == ActionsDAG::ActionType::COLUMN) { @@ -79,7 +79,7 @@ void buildSetsForDAG(const ActionsDAGPtr & dag, const ContextPtr & context) void filterBlockWithDAG(ActionsDAGPtr dag, Block & block, ContextPtr context) { - buildSetsForDAG(dag, context); + buildSetsForDAG(*dag, context); auto actions = std::make_shared(dag); Block block_with_filter = block; actions->execute(block_with_filter, /*dry_run=*/ false, /*allow_duplicates_in_input=*/ true); diff --git a/src/Storages/VirtualColumnUtils.h b/src/Storages/VirtualColumnUtils.h index fbfbdd6c6cc..9045a2f5481 100644 --- a/src/Storages/VirtualColumnUtils.h +++ b/src/Storages/VirtualColumnUtils.h @@ -26,7 +26,7 @@ void filterBlockWithPredicate(const ActionsDAG::Node * predicate, Block & block, void filterBlockWithDAG(ActionsDAGPtr dag, Block & block, ContextPtr context); /// Builds sets used by ActionsDAG inplace. -void buildSetsForDAG(const ActionsDAGPtr & dag, const ContextPtr & context); +void buildSetsForDAG(const ActionsDAG & dag, const ContextPtr & context); /// Recursively checks if all functions used in DAG are deterministic in scope of query. bool isDeterministicInScopeOfQuery(const ActionsDAG::Node * node); diff --git a/tests/queries/0_stateless/02841_not_ready_set_constraints.reference b/tests/queries/0_stateless/02841_not_ready_set_constraints.reference new file mode 100644 index 00000000000..d81cc0710eb --- /dev/null +++ b/tests/queries/0_stateless/02841_not_ready_set_constraints.reference @@ -0,0 +1 @@ +42 diff --git a/tests/queries/0_stateless/02841_not_ready_set_constraints.sql b/tests/queries/0_stateless/02841_not_ready_set_constraints.sql new file mode 100644 index 00000000000..ecdf4d50635 --- /dev/null +++ b/tests/queries/0_stateless/02841_not_ready_set_constraints.sql @@ -0,0 +1,19 @@ +DROP TABLE IF EXISTS t1; +DROP TABLE IF EXISTS t2; + +CREATE TABLE t1 ( + `id` UInt64 +) +ENGINE = MergeTree ORDER BY id; + +INSERT INTO t1(id) VALUES (42); + +CREATE TABLE t2 ( + `conversation` UInt64, + CONSTRAINT constraint_conversation CHECK conversation IN (SELECT id FROM t1) +) +ENGINE = MergeTree ORDER BY conversation; + +INSERT INTO t2(conversation) VALUES (42); + +select * from t2; From 6baa52d10176369fefc6249dbb256a26eb7b1bdc Mon Sep 17 00:00:00 2001 From: Kruglov Pavel <48961922+Avogar@users.noreply.github.com> Date: Tue, 9 Jul 2024 14:01:28 +0200 Subject: [PATCH 339/417] Fix null insertion into dynamic column --- src/Formats/JSONExtractTree.cpp | 9 ++++++++- 1 file changed, 8 insertions(+), 1 deletion(-) diff --git a/src/Formats/JSONExtractTree.cpp b/src/Formats/JSONExtractTree.cpp index 8fe472930d3..9efb1392583 100644 --- a/src/Formats/JSONExtractTree.cpp +++ b/src/Formats/JSONExtractTree.cpp @@ -1265,9 +1265,16 @@ public: bool insertResultToColumn(IColumn & column, const typename JSONParser::Element & element, const JSONExtractInsertSettings & insert_settings, const FormatSettings & format_settings, String & error) const override { auto & column_dynamic = assert_cast(column); + /// First, check if element is NULL. + if (element.isNull()) + { + column_dynamic.insertDefault(); + return true; + } + auto & variant_column = column_dynamic.getVariantColumn(); auto variant_info = column_dynamic.getVariantInfo(); - /// First, infer ClickHouse type for this element and add it as a new variant. + /// Second, infer ClickHouse type for this element and add it as a new variant. auto element_type = elementToDataType(element, format_settings); if (column_dynamic.addNewVariant(element_type)) { From 61f827b5698754af84e7d75a2b83bd0191139820 Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?J=C3=A1nos=20Benjamin=20Antal?= Date: Tue, 9 Jul 2024 15:26:33 +0200 Subject: [PATCH 340/417] Update src/Databases/DatabaseAtomic.cpp --- src/Databases/DatabaseAtomic.cpp | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/src/Databases/DatabaseAtomic.cpp b/src/Databases/DatabaseAtomic.cpp index a48eb2abce6..5b816e4f282 100644 --- a/src/Databases/DatabaseAtomic.cpp +++ b/src/Databases/DatabaseAtomic.cpp @@ -106,8 +106,8 @@ void DatabaseAtomic::attachTable(ContextPtr /* context_ */, const String & name, StoragePtr DatabaseAtomic::detachTable(ContextPtr /* context */, const String & name) { - // it is important to call destructures not_in_use without - // blocking mutex for avoid potential deadlock. + // it is important to call the destructors of not_in_use without + // locked mutex to avoid potential deadlock. DetachedTables not_in_use; StoragePtr table; { From 830653b1f6fc0fdac9f0a143b99cdba94009c1aa Mon Sep 17 00:00:00 2001 From: Max K Date: Tue, 9 Jul 2024 15:40:44 +0200 Subject: [PATCH 341/417] CI: do not finalize CI running status unless all success --- .github/workflows/pull_request.yml | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/.github/workflows/pull_request.yml b/.github/workflows/pull_request.yml index 4764e6d3c1a..259e6d41110 100644 --- a/.github/workflows/pull_request.yml +++ b/.github/workflows/pull_request.yml @@ -172,7 +172,7 @@ jobs: ################################# Stage Final ################################# # FinishCheck: - if: ${{ !cancelled() }} + if: ${{ !failure() }} needs: [RunConfig, BuildDockers, StyleCheck, FastTest, Builds_1, Builds_2, Builds_Report, Tests_1, Tests_2, Tests_3] runs-on: [self-hosted, style-checker-aarch64] steps: From 61e50a346aecd33eed510580f2ab52eb2f816dc1 Mon Sep 17 00:00:00 2001 From: Azat Khuzhin Date: Fri, 5 Jul 2024 20:23:45 +0200 Subject: [PATCH 342/417] Use peak_threads_usage instead of arrayUniq(thread_ids) in tests Should fix flakiness like [1]. [1]: https://s3.amazonaws.com/clickhouse-test-reports/66098/93afc8e6133365007488c4d8340f434f6e8a876f/stateless_tests__aarch64_.html Signed-off-by: Azat Khuzhin --- .../0_stateless/01275_parallel_mv.reference | 16 ++++++++-------- .../queries/0_stateless/01275_parallel_mv.sql.j2 | 2 +- .../0_stateless/01323_too_many_threads_bug.sql | 4 ++-- .../02350_views_max_insert_threads.sql | 2 +- 4 files changed, 12 insertions(+), 12 deletions(-) diff --git a/tests/queries/0_stateless/01275_parallel_mv.reference b/tests/queries/0_stateless/01275_parallel_mv.reference index a9801e3b910..f5f31c4a563 100644 --- a/tests/queries/0_stateless/01275_parallel_mv.reference +++ b/tests/queries/0_stateless/01275_parallel_mv.reference @@ -10,7 +10,7 @@ insert into testX select number from numbers(10) settings optimize_trivial_insert_select=0, max_insert_threads=0; -- { serverError FUNCTION_THROW_IF_VALUE_IS_NON_ZERO } system flush logs; -select arrayUniq(thread_ids) from system.query_log where +select peak_threads_usage from system.query_log where current_database = currentDatabase() and type != 'QueryStart' and query like '%insert into testX %' and @@ -34,7 +34,7 @@ insert into testX select number from numbers(10) settings optimize_trivial_insert_select=0, max_insert_threads=16; -- { serverError FUNCTION_THROW_IF_VALUE_IS_NON_ZERO } system flush logs; -select arrayUniq(thread_ids) from system.query_log where +select peak_threads_usage from system.query_log where current_database = currentDatabase() and type != 'QueryStart' and query like '%insert into testX %' and @@ -58,7 +58,7 @@ insert into testX select number from numbers(10) settings optimize_trivial_insert_select=1, max_insert_threads=0; -- { serverError FUNCTION_THROW_IF_VALUE_IS_NON_ZERO } system flush logs; -select arrayUniq(thread_ids) from system.query_log where +select peak_threads_usage from system.query_log where current_database = currentDatabase() and type != 'QueryStart' and query like '%insert into testX %' and @@ -82,7 +82,7 @@ insert into testX select number from numbers(10) settings optimize_trivial_insert_select=1, max_insert_threads=16; -- { serverError FUNCTION_THROW_IF_VALUE_IS_NON_ZERO } system flush logs; -select arrayUniq(thread_ids) from system.query_log where +select peak_threads_usage from system.query_log where current_database = currentDatabase() and type != 'QueryStart' and query like '%insert into testX %' and @@ -106,7 +106,7 @@ insert into testX select number from numbers(10) settings optimize_trivial_insert_select=0, max_insert_threads=0; -- { serverError FUNCTION_THROW_IF_VALUE_IS_NON_ZERO } system flush logs; -select arrayUniq(thread_ids) from system.query_log where +select peak_threads_usage from system.query_log where current_database = currentDatabase() and type != 'QueryStart' and query like '%insert into testX %' and @@ -130,7 +130,7 @@ insert into testX select number from numbers(10) settings optimize_trivial_insert_select=0, max_insert_threads=16; -- { serverError FUNCTION_THROW_IF_VALUE_IS_NON_ZERO } system flush logs; -select arrayUniq(thread_ids) from system.query_log where +select peak_threads_usage from system.query_log where current_database = currentDatabase() and type != 'QueryStart' and query like '%insert into testX %' and @@ -154,7 +154,7 @@ insert into testX select number from numbers(10) settings optimize_trivial_insert_select=1, max_insert_threads=0; -- { serverError FUNCTION_THROW_IF_VALUE_IS_NON_ZERO } system flush logs; -select arrayUniq(thread_ids) from system.query_log where +select peak_threads_usage from system.query_log where current_database = currentDatabase() and type != 'QueryStart' and query like '%insert into testX %' and @@ -178,7 +178,7 @@ insert into testX select number from numbers(10) settings optimize_trivial_insert_select=1, max_insert_threads=16; -- { serverError FUNCTION_THROW_IF_VALUE_IS_NON_ZERO } system flush logs; -select arrayUniq(thread_ids) from system.query_log where +select peak_threads_usage from system.query_log where current_database = currentDatabase() and type != 'QueryStart' and query like '%insert into testX %' and diff --git a/tests/queries/0_stateless/01275_parallel_mv.sql.j2 b/tests/queries/0_stateless/01275_parallel_mv.sql.j2 index 9d74474c1a4..5918035e9c3 100644 --- a/tests/queries/0_stateless/01275_parallel_mv.sql.j2 +++ b/tests/queries/0_stateless/01275_parallel_mv.sql.j2 @@ -28,7 +28,7 @@ insert into testX select number from numbers(10) settings optimize_trivial_insert_select={{ optimize_trivial_insert_select }}, max_insert_threads={{ max_insert_threads }}; -- { serverError FUNCTION_THROW_IF_VALUE_IS_NON_ZERO } system flush logs; -select arrayUniq(thread_ids) from system.query_log where +select peak_threads_usage from system.query_log where current_database = currentDatabase() and type != 'QueryStart' and query like '%insert into testX %' and diff --git a/tests/queries/0_stateless/01323_too_many_threads_bug.sql b/tests/queries/0_stateless/01323_too_many_threads_bug.sql index c377e2c7570..5bf282808c3 100644 --- a/tests/queries/0_stateless/01323_too_many_threads_bug.sql +++ b/tests/queries/0_stateless/01323_too_many_threads_bug.sql @@ -14,11 +14,11 @@ set log_queries = 1; select x from table_01323_many_parts limit 10 format Null; system flush logs; -select arrayUniq(thread_ids) <= 4 from system.query_log where current_database = currentDatabase() AND event_date >= today() - 1 and query ilike '%select x from table_01323_many_parts%' and query not like '%system.query_log%' and type = 'QueryFinish' order by query_start_time desc limit 1; +select peak_threads_usage <= 4 from system.query_log where current_database = currentDatabase() AND event_date >= today() - 1 and query ilike '%select x from table_01323_many_parts%' and query not like '%system.query_log%' and type = 'QueryFinish' order by query_start_time desc limit 1; select x from table_01323_many_parts order by x limit 10 format Null; system flush logs; -select arrayUniq(thread_ids) <= 36 from system.query_log where current_database = currentDatabase() AND event_date >= today() - 1 and query ilike '%select x from table_01323_many_parts order by x%' and query not like '%system.query_log%' and type = 'QueryFinish' order by query_start_time desc limit 1; +select peak_threads_usage <= 36 from system.query_log where current_database = currentDatabase() AND event_date >= today() - 1 and query ilike '%select x from table_01323_many_parts order by x%' and query not like '%system.query_log%' and type = 'QueryFinish' order by query_start_time desc limit 1; drop table if exists table_01323_many_parts; diff --git a/tests/queries/0_stateless/02350_views_max_insert_threads.sql b/tests/queries/0_stateless/02350_views_max_insert_threads.sql index 25e0fdeadba..a4f7e2546ed 100644 --- a/tests/queries/0_stateless/02350_views_max_insert_threads.sql +++ b/tests/queries/0_stateless/02350_views_max_insert_threads.sql @@ -10,7 +10,7 @@ create materialized view t_mv Engine = Null AS select now() as ts, max(a) from t insert into t select * from numbers_mt(10e6) settings max_threads = 16, max_insert_threads=16, max_block_size=100000; system flush logs; -select arrayUniq(thread_ids)>=16 from system.query_log where +select peak_threads_usage>=16 from system.query_log where event_date >= yesterday() and current_database = currentDatabase() and type = 'QueryFinish' and From 8a202d91ad745089adaff4ebf2cde5e6754503ce Mon Sep 17 00:00:00 2001 From: kssenii Date: Tue, 9 Jul 2024 16:24:35 +0200 Subject: [PATCH 343/417] Properly read schema and partition columns from checkpoint file --- .../DataLakes/DeltaLakeMetadata.cpp | 169 ++++++++++++------ .../DataLakes/IStorageDataLake.h | 10 +- .../StorageObjectStorageSource.cpp | 28 +-- tests/integration/test_storage_delta/test.py | 129 +++++++++++-- 4 files changed, 255 insertions(+), 81 deletions(-) diff --git a/src/Storages/ObjectStorage/DataLakes/DeltaLakeMetadata.cpp b/src/Storages/ObjectStorage/DataLakes/DeltaLakeMetadata.cpp index 12341c877e2..d37bffc42c4 100644 --- a/src/Storages/ObjectStorage/DataLakes/DeltaLakeMetadata.cpp +++ b/src/Storages/ObjectStorage/DataLakes/DeltaLakeMetadata.cpp @@ -8,6 +8,7 @@ #include #include #include +#include #include #include @@ -30,6 +31,7 @@ #include #include #include +#include #include #include @@ -111,7 +113,7 @@ struct DeltaLakeMetadataImpl std::set result_files; NamesAndTypesList current_schema; DataLakePartitionColumns current_partition_columns; - const auto checkpoint_version = getCheckpointIfExists(result_files); + const auto checkpoint_version = getCheckpointIfExists(result_files, current_schema, current_partition_columns); if (checkpoint_version) { @@ -205,9 +207,9 @@ struct DeltaLakeMetadataImpl Poco::Dynamic::Var json = parser.parse(json_str); Poco::JSON::Object::Ptr object = json.extract(); - // std::ostringstream oss; // STYLE_CHECK_ALLOW_STD_STRING_STREAM - // object->stringify(oss); - // LOG_TEST(log, "Metadata: {}", oss.str()); + std::ostringstream oss; // STYLE_CHECK_ALLOW_STD_STRING_STREAM + object->stringify(oss); + LOG_TEST(log, "Metadata: {}", oss.str()); if (object->has("metaData")) { @@ -216,30 +218,9 @@ struct DeltaLakeMetadataImpl Poco::JSON::Parser p; Poco::Dynamic::Var fields_json = parser.parse(schema_object); - Poco::JSON::Object::Ptr fields_object = fields_json.extract(); - - const auto fields = fields_object->get("fields").extract(); - NamesAndTypesList current_schema; - for (size_t i = 0; i < fields->size(); ++i) - { - const auto field = fields->getObject(static_cast(i)); - auto column_name = field->getValue("name"); - auto type = field->getValue("type"); - auto is_nullable = field->getValue("nullable"); - - std::string physical_name; - auto schema_metadata_object = field->get("metadata").extract(); - if (schema_metadata_object->has("delta.columnMapping.physicalName")) - physical_name = schema_metadata_object->getValue("delta.columnMapping.physicalName"); - else - physical_name = column_name; - - LOG_TEST(log, "Found column: {}, type: {}, nullable: {}, physical name: {}", - column_name, type, is_nullable, physical_name); - - current_schema.push_back({physical_name, getFieldType(field, "type", is_nullable)}); - } + const Poco::JSON::Object::Ptr & fields_object = fields_json.extract(); + auto current_schema = parseMetadata(fields_object); if (file_schema.empty()) { file_schema = current_schema; @@ -274,7 +255,12 @@ struct DeltaLakeMetadataImpl const auto value = partition_values->getValue(partition_name); auto name_and_type = file_schema.tryGetByName(partition_name); if (!name_and_type) - throw Exception(ErrorCodes::LOGICAL_ERROR, "No such column in schema: {}", partition_name); + { + throw Exception( + ErrorCodes::LOGICAL_ERROR, + "No such column in schema: {} (schema: {})", + partition_name, file_schema.toNamesAndTypesDescription()); + } auto field = getFieldValue(value, name_and_type->type); current_partition_columns.emplace_back(*name_and_type, field); @@ -293,6 +279,32 @@ struct DeltaLakeMetadataImpl } } + NamesAndTypesList parseMetadata(const Poco::JSON::Object::Ptr & metadata_json) + { + NamesAndTypesList schema; + const auto fields = metadata_json->get("fields").extract(); + for (size_t i = 0; i < fields->size(); ++i) + { + const auto field = fields->getObject(static_cast(i)); + auto column_name = field->getValue("name"); + auto type = field->getValue("type"); + auto is_nullable = field->getValue("nullable"); + + std::string physical_name; + auto schema_metadata_object = field->get("metadata").extract(); + if (schema_metadata_object->has("delta.columnMapping.physicalName")) + physical_name = schema_metadata_object->getValue("delta.columnMapping.physicalName"); + else + physical_name = column_name; + + LOG_TEST(log, "Found column: {}, type: {}, nullable: {}, physical name: {}", + column_name, type, is_nullable, physical_name); + + schema.push_back({physical_name, getFieldType(field, "type", is_nullable)}); + } + return schema; + } + DataTypePtr getFieldType(const Poco::JSON::Object::Ptr & field, const String & type_key, bool is_nullable) { if (field->isObject(type_key)) @@ -506,7 +518,10 @@ struct DeltaLakeMetadataImpl throw Exception(ErrorCodes::BAD_ARGUMENTS, "Arrow error: {}", _s.ToString()); \ } while (false) - size_t getCheckpointIfExists(std::set & result) + size_t getCheckpointIfExists( + std::set & result, + NamesAndTypesList & file_schema, + DataLakePartitionColumns & file_partition_columns) { const auto version = readLastCheckpointIfExists(); if (!version) @@ -527,7 +542,8 @@ struct DeltaLakeMetadataImpl auto columns = ParquetSchemaReader(*buf, format_settings).readSchema(); /// Read only columns that we need. - columns.filterColumns(NameSet{"add", "remove"}); + auto filter_column_names = NameSet{"add", "metaData"}; + columns.filterColumns(filter_column_names); Block header; for (const auto & column : columns) header.insert({column.type->createColumn(), column.type, column.name}); @@ -541,9 +557,6 @@ struct DeltaLakeMetadataImpl ArrowMemoryPool::instance(), &reader)); - std::shared_ptr file_schema; - THROW_ARROW_NOT_OK(reader->GetSchema(&file_schema)); - ArrowColumnToCHColumn column_reader( header, "Parquet", format_settings.parquet.allow_missing_columns, @@ -554,29 +567,85 @@ struct DeltaLakeMetadataImpl std::shared_ptr table; THROW_ARROW_NOT_OK(reader->ReadTable(&table)); - Chunk res = column_reader.arrowTableToCHChunk(table, reader->parquet_reader()->metadata()->num_rows()); - const auto & res_columns = res.getColumns(); + Chunk chunk = column_reader.arrowTableToCHChunk(table, reader->parquet_reader()->metadata()->num_rows()); + auto res_block = header.cloneWithColumns(chunk.detachColumns()); + res_block = Nested::flatten(res_block); - if (res_columns.size() != 2) - { - throw Exception( - ErrorCodes::INCORRECT_DATA, - "Unexpected number of columns: {} (having: {}, expected: {})", - res_columns.size(), res.dumpStructure(), header.dumpStructure()); - } + const auto * nullable_path_column = assert_cast(res_block.getByName("add.path").column.get()); + const auto & path_column = assert_cast(nullable_path_column->getNestedColumn()); + + const auto * nullable_schema_column = assert_cast(res_block.getByName("metaData.schemaString").column.get()); + const auto & schema_column = assert_cast(nullable_schema_column->getNestedColumn()); + + auto partition_values_column_raw = res_block.getByName("add.partitionValues").column; + const auto & partition_values_column = assert_cast(*partition_values_column_raw); - const auto * tuple_column = assert_cast(res_columns[0].get()); - const auto & nullable_column = assert_cast(tuple_column->getColumn(0)); - const auto & path_column = assert_cast(nullable_column.getNestedColumn()); for (size_t i = 0; i < path_column.size(); ++i) { - const auto filename = String(path_column.getDataAt(i)); - if (filename.empty()) + const auto metadata = String(schema_column.getDataAt(i)); + if (!metadata.empty()) + { + Poco::JSON::Parser parser; + Poco::Dynamic::Var json = parser.parse(metadata); + const Poco::JSON::Object::Ptr & object = json.extract(); + + auto current_schema = parseMetadata(object); + if (file_schema.empty()) + { + file_schema = current_schema; + LOG_TEST(log, "Processed schema from checkpoint: {}", file_schema.toString()); + } + else if (file_schema != current_schema) + { + throw Exception(ErrorCodes::NOT_IMPLEMENTED, + "Reading from files with different schema is not possible " + "({} is different from {})", + file_schema.toString(), current_schema.toString()); + } + } + } + + for (size_t i = 0; i < path_column.size(); ++i) + { + const auto path = String(path_column.getDataAt(i)); + if (path.empty()) continue; - LOG_TEST(log, "Adding {}", filename); - const auto [_, inserted] = result.insert(std::filesystem::path(configuration->getPath()) / filename); + + auto filename = fs::path(path).filename().string(); + auto it = file_partition_columns.find(filename); + if (it == file_partition_columns.end()) + { + Field map; + partition_values_column.get(i, map); + auto partition_values_map = map.safeGet(); + if (!partition_values_map.empty()) + { + auto & current_partition_columns = file_partition_columns[filename]; + for (const auto & map_value : partition_values_map) + { + const auto tuple = map_value.safeGet(); + const auto partition_name = tuple[0].safeGet(); + auto name_and_type = file_schema.tryGetByName(partition_name); + if (!name_and_type) + { + throw Exception( + ErrorCodes::LOGICAL_ERROR, + "No such column in schema: {} (schema: {})", + partition_name, file_schema.toString()); + } + const auto value = tuple[1].safeGet(); + auto field = getFieldValue(value, name_and_type->type); + current_partition_columns.emplace_back(*name_and_type, field); + + LOG_TEST(log, "Partition {} value is {} (for {})", partition_name, value, filename); + } + } + } + + LOG_TEST(log, "Adding {}", path); + const auto [_, inserted] = result.insert(std::filesystem::path(configuration->getPath()) / path); if (!inserted) - throw Exception(ErrorCodes::INCORRECT_DATA, "File already exists {}", filename); + throw Exception(ErrorCodes::INCORRECT_DATA, "File already exists {}", path); } return version; diff --git a/src/Storages/ObjectStorage/DataLakes/IStorageDataLake.h b/src/Storages/ObjectStorage/DataLakes/IStorageDataLake.h index f1217bc9729..d6935c706d9 100644 --- a/src/Storages/ObjectStorage/DataLakes/IStorageDataLake.h +++ b/src/Storages/ObjectStorage/DataLakes/IStorageDataLake.h @@ -17,6 +17,10 @@ namespace DB { +namespace ErrorCodes +{ + extern const int LOGICAL_ERROR; +} /// Storage for read-only integration with Apache Iceberg tables in Amazon S3 (see https://iceberg.apache.org/) /// Right now it's implemented on top of StorageS3 and right now it doesn't support @@ -41,6 +45,7 @@ public: auto object_storage = base_configuration->createObjectStorage(context, /* is_readonly */true); DataLakeMetadataPtr metadata; NamesAndTypesList schema_from_metadata; + const bool use_schema_from_metadata = columns_.empty(); if (base_configuration->format == "auto") base_configuration->format = "Parquet"; @@ -50,8 +55,9 @@ public: try { metadata = DataLakeMetadata::create(object_storage, base_configuration, context); - schema_from_metadata = metadata->getTableSchema(); configuration->setPaths(metadata->getDataFiles()); + if (use_schema_from_metadata) + schema_from_metadata = metadata->getTableSchema(); } catch (...) { @@ -66,7 +72,7 @@ public: return std::make_shared>( base_configuration, std::move(metadata), configuration, object_storage, context, table_id_, - columns_.empty() ? ColumnsDescription(schema_from_metadata) : columns_, + use_schema_from_metadata ? ColumnsDescription(schema_from_metadata) : columns_, constraints_, comment_, format_settings_); } diff --git a/src/Storages/ObjectStorage/StorageObjectStorageSource.cpp b/src/Storages/ObjectStorage/StorageObjectStorageSource.cpp index 6940f10cb91..a9a7e062076 100644 --- a/src/Storages/ObjectStorage/StorageObjectStorageSource.cpp +++ b/src/Storages/ObjectStorage/StorageObjectStorageSource.cpp @@ -206,23 +206,25 @@ Chunk StorageObjectStorageSource::generate() if (!partition_columns.empty() && chunk_size && chunk.hasColumns()) { auto partition_values = partition_columns.find(filename); - - for (const auto & [name_and_type, value] : partition_values->second) + if (partition_values != partition_columns.end()) { - if (!read_from_format_info.source_header.has(name_and_type.name)) - continue; + for (const auto & [name_and_type, value] : partition_values->second) + { + if (!read_from_format_info.source_header.has(name_and_type.name)) + continue; - const auto column_pos = read_from_format_info.source_header.getPositionByName(name_and_type.name); - auto partition_column = name_and_type.type->createColumnConst(chunk.getNumRows(), value)->convertToFullColumnIfConst(); + const auto column_pos = read_from_format_info.source_header.getPositionByName(name_and_type.name); + auto partition_column = name_and_type.type->createColumnConst(chunk.getNumRows(), value)->convertToFullColumnIfConst(); - /// This column is filled with default value now, remove it. - chunk.erase(column_pos); + /// This column is filled with default value now, remove it. + chunk.erase(column_pos); - /// Add correct values. - if (chunk.hasColumns()) - chunk.addColumn(column_pos, std::move(partition_column)); - else - chunk.addColumn(std::move(partition_column)); + /// Add correct values. + if (column_pos < chunk.getNumColumns()) + chunk.addColumn(column_pos, std::move(partition_column)); + else + chunk.addColumn(std::move(partition_column)); + } } } return chunk; diff --git a/tests/integration/test_storage_delta/test.py b/tests/integration/test_storage_delta/test.py index 4cb71895881..d3dd7cfe52a 100644 --- a/tests/integration/test_storage_delta/test.py +++ b/tests/integration/test_storage_delta/test.py @@ -596,19 +596,116 @@ def test_partition_columns(started_cluster): ) assert result == 1 - # instance.query( - # f""" - # DROP TABLE IF EXISTS {TABLE_NAME}; - # CREATE TABLE {TABLE_NAME} (a Int32, b String, c DateTime) - # ENGINE=DeltaLake('http://{started_cluster.minio_ip}:{started_cluster.minio_port}/{bucket}/{result_file}/', 'minio', 'minio123')""" - # ) - # assert ( - # int( - # instance.query( - # f"SELECT count() FROM {TABLE_NAME} WHERE c != toDateTime('2000/01/05')" - # ) - # ) - # == num_rows - 1 - # ) - # instance.query(f"SELECT a, b, c, FROM {TABLE_NAME}") - # assert False + instance.query( + f""" + DROP TABLE IF EXISTS {TABLE_NAME}; + CREATE TABLE {TABLE_NAME} (a Nullable(Int32), b Nullable(String), c Nullable(Date32), d Nullable(Int32), e Nullable(Bool)) + ENGINE=DeltaLake('http://{started_cluster.minio_ip}:{started_cluster.minio_port}/{bucket}/{result_file}/', 'minio', 'minio123')""" + ) + assert ( + """1 test1 2000-01-01 1 false +2 test2 2000-01-02 2 false +3 test3 2000-01-03 3 false +4 test4 2000-01-04 4 false +5 test5 2000-01-05 5 false +6 test6 2000-01-06 6 false +7 test7 2000-01-07 7 false +8 test8 2000-01-08 8 false +9 test9 2000-01-09 9 false""" + == instance.query(f"SELECT * FROM {TABLE_NAME} ORDER BY b").strip() + ) + + assert ( + int( + instance.query( + f"SELECT count() FROM {TABLE_NAME} WHERE c == toDateTime('2000/01/05')" + ) + ) + == 1 + ) + + # Subset of columns should work. + instance.query( + f""" + DROP TABLE IF EXISTS {TABLE_NAME}; + CREATE TABLE {TABLE_NAME} (b Nullable(String), c Nullable(Date32), d Nullable(Int32)) + ENGINE=DeltaLake('http://{started_cluster.minio_ip}:{started_cluster.minio_port}/{bucket}/{result_file}/', 'minio', 'minio123')""" + ) + assert ( + """test1 2000-01-01 1 +test2 2000-01-02 2 +test3 2000-01-03 3 +test4 2000-01-04 4 +test5 2000-01-05 5 +test6 2000-01-06 6 +test7 2000-01-07 7 +test8 2000-01-08 8 +test9 2000-01-09 9""" + == instance.query(f"SELECT * FROM {TABLE_NAME} ORDER BY b").strip() + ) + + for i in range(num_rows + 1, 2 * num_rows + 1): + data = [ + ( + i, + "test" + str(i), + datetime.strptime(f"2000-01-{i}", "%Y-%m-%d"), + i, + False, + ) + ] + df = spark.createDataFrame(data=data, schema=schema) + df.printSchema() + df.write.mode("append").format("delta").partitionBy(partition_columns).save( + f"/{TABLE_NAME}" + ) + + files = upload_directory(minio_client, bucket, f"/{TABLE_NAME}", "") + ok = False + for file in files: + if file.endswith("last_checkpoint"): + ok = True + assert ok + + result = int( + instance.query( + f"""SELECT count() + FROM deltaLake('http://{started_cluster.minio_ip}:{started_cluster.minio_port}/{bucket}/{result_file}/', 'minio', 'minio123') + """ + ) + ) + assert result == num_rows * 2 + + assert ( + """1 test1 2000-01-01 1 false +2 test2 2000-01-02 2 false +3 test3 2000-01-03 3 false +4 test4 2000-01-04 4 false +5 test5 2000-01-05 5 false +6 test6 2000-01-06 6 false +7 test7 2000-01-07 7 false +8 test8 2000-01-08 8 false +9 test9 2000-01-09 9 false +10 test10 2000-01-10 10 false +11 test11 2000-01-11 11 false +12 test12 2000-01-12 12 false +13 test13 2000-01-13 13 false +14 test14 2000-01-14 14 false +15 test15 2000-01-15 15 false +16 test16 2000-01-16 16 false +17 test17 2000-01-17 17 false +18 test18 2000-01-18 18 false""" + == instance.query( + f""" +SELECT * FROM deltaLake('http://{started_cluster.minio_ip}:{started_cluster.minio_port}/{bucket}/{result_file}/', 'minio', 'minio123') ORDER BY c + """ + ).strip() + ) + assert ( + int( + instance.query( + f"SELECT count() FROM {TABLE_NAME} WHERE c == toDateTime('2000/01/15')" + ) + ) + == 1 + ) From 0821d72f1cef93e437f5b38477f2a2d9d9611f8e Mon Sep 17 00:00:00 2001 From: Antonio Andelic Date: Tue, 9 Jul 2024 17:06:37 +0200 Subject: [PATCH 344/417] Collect core dumps in more tests --- docker/test/stateful/run.sh | 5 +++++ docker/test/stateless/run.sh | 2 ++ docker/test/stateless/stress_tests.lib | 11 ----------- docker/test/stateless/utils.lib | 11 +++++++++++ docker/test/stress/run.sh | 3 +++ 5 files changed, 21 insertions(+), 11 deletions(-) diff --git a/docker/test/stateful/run.sh b/docker/test/stateful/run.sh index 2215ac2b37c..80e5e81a4b1 100755 --- a/docker/test/stateful/run.sh +++ b/docker/test/stateful/run.sh @@ -16,6 +16,9 @@ dpkg -i package_folder/clickhouse-client_*.deb ln -s /usr/share/clickhouse-test/clickhouse-test /usr/bin/clickhouse-test +# shellcheck disable=SC1091 +source /utils.lib + # install test configs /usr/share/clickhouse-test/config/install.sh @@ -272,3 +275,5 @@ if [[ -n "$USE_DATABASE_REPLICATED" ]] && [[ "$USE_DATABASE_REPLICATED" -eq 1 ]] mv /var/log/clickhouse-server/stderr1.log /test_output/ ||: mv /var/log/clickhouse-server/stderr2.log /test_output/ ||: fi + +collect_core_dumps diff --git a/docker/test/stateless/run.sh b/docker/test/stateless/run.sh index 43d3c698d8a..fe551f5d6d6 100755 --- a/docker/test/stateless/run.sh +++ b/docker/test/stateless/run.sh @@ -399,3 +399,5 @@ if [[ -n "$USE_SHARED_CATALOG" ]] && [[ "$USE_SHARED_CATALOG" -eq 1 ]]; then mv /var/log/clickhouse-server/stderr1.log /test_output/ ||: tar -chf /test_output/coordination1.tar /var/lib/clickhouse1/coordination ||: fi + +collect_core_dumps diff --git a/docker/test/stateless/stress_tests.lib b/docker/test/stateless/stress_tests.lib index c069ccbdd8d..682da1df837 100644 --- a/docker/test/stateless/stress_tests.lib +++ b/docker/test/stateless/stress_tests.lib @@ -1,8 +1,5 @@ #!/bin/bash -# core.COMM.PID-TID -sysctl kernel.core_pattern='core.%e.%p-%P' - OK="\tOK\t\\N\t" FAIL="\tFAIL\t\\N\t" @@ -315,12 +312,4 @@ function collect_query_and_trace_logs() done } -function collect_core_dumps() -{ - find . -type f -maxdepth 1 -name 'core.*' | while read -r core; do - zstd --threads=0 "$core" - mv "$core.zst" /test_output/ - done -} - # vi: ft=bash diff --git a/docker/test/stateless/utils.lib b/docker/test/stateless/utils.lib index 833e1a05384..90efb5b1a8e 100644 --- a/docker/test/stateless/utils.lib +++ b/docker/test/stateless/utils.lib @@ -1,5 +1,8 @@ #!/bin/bash +# core.COMM.PID-TID +sysctl kernel.core_pattern='core.%e.%p-%P' + function run_with_retry() { if [[ $- =~ e ]]; then @@ -48,4 +51,12 @@ function timeout_with_logging() { return $exit_code } +function collect_core_dumps() +{ + find . -type f -maxdepth 1 -name 'core.*' | while read -r core; do + zstd --threads=0 "$core" + mv "$core.zst" /test_output/ + done +} + # vi: ft=bash diff --git a/docker/test/stress/run.sh b/docker/test/stress/run.sh index 323944591b1..86467394513 100644 --- a/docker/test/stress/run.sh +++ b/docker/test/stress/run.sh @@ -21,6 +21,9 @@ source /attach_gdb.lib # shellcheck source=../stateless/stress_tests.lib source /stress_tests.lib +# shellcheck disable=SC1091 +source /utils.lib + install_packages package_folder # Thread Fuzzer allows to check more permutations of possible thread scheduling From b539d999cddf263e0b9fa6f9aa9381dea62a5ed3 Mon Sep 17 00:00:00 2001 From: Sema Checherinda Date: Tue, 9 Jul 2024 17:12:55 +0200 Subject: [PATCH 345/417] limit the logs about rescheduling --- src/Interpreters/DatabaseCatalog.cpp | 3 ++- src/Interpreters/DatabaseCatalog.h | 2 ++ 2 files changed, 4 insertions(+), 1 deletion(-) diff --git a/src/Interpreters/DatabaseCatalog.cpp b/src/Interpreters/DatabaseCatalog.cpp index 2abda981a8c..e34aef125fb 100644 --- a/src/Interpreters/DatabaseCatalog.cpp +++ b/src/Interpreters/DatabaseCatalog.cpp @@ -837,6 +837,7 @@ DatabaseCatalog::DatabaseCatalog(ContextMutablePtr global_context_) , loading_dependencies{"LoadingDeps"} , view_dependencies{"ViewDeps"} , log(getLogger("DatabaseCatalog")) + , limitedLog(std::make_shared(log, 1, 5)) , first_async_drop_in_queue(tables_marked_dropped.end()) { } @@ -1273,7 +1274,7 @@ void DatabaseCatalog::rescheduleDropTableTask() if (first_async_drop_in_queue != tables_marked_dropped.begin()) { LOG_TRACE( - log, + limitedLog, "Have {} tables in queue to drop. Some of them are being dropped in sync mode. Schedule background task ASAP", tables_marked_dropped.size()); (*drop_task)->scheduleAfter(0); diff --git a/src/Interpreters/DatabaseCatalog.h b/src/Interpreters/DatabaseCatalog.h index 23e38a6445e..73e330dedbc 100644 --- a/src/Interpreters/DatabaseCatalog.h +++ b/src/Interpreters/DatabaseCatalog.h @@ -7,6 +7,7 @@ #include #include #include +#include #include #include @@ -325,6 +326,7 @@ private: TablesDependencyGraph view_dependencies TSA_GUARDED_BY(databases_mutex); LoggerPtr log; + LogSeriesLimiterPtr limitedLog; std::atomic_bool is_shutting_down = false; From 19a508747bfc8ece6eaf9f0d245003e1ac1abeb1 Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Tue, 9 Jul 2024 18:03:56 +0200 Subject: [PATCH 346/417] Update opentelemetry.md --- docs/en/operations/opentelemetry.md | 6 +----- 1 file changed, 1 insertion(+), 5 deletions(-) diff --git a/docs/en/operations/opentelemetry.md b/docs/en/operations/opentelemetry.md index 70f64d08ba3..fe60ceedc0b 100644 --- a/docs/en/operations/opentelemetry.md +++ b/docs/en/operations/opentelemetry.md @@ -2,15 +2,11 @@ slug: /en/operations/opentelemetry sidebar_position: 62 sidebar_label: Tracing ClickHouse with OpenTelemetry -title: "[experimental] Tracing ClickHouse with OpenTelemetry" +title: "Tracing ClickHouse with OpenTelemetry" --- [OpenTelemetry](https://opentelemetry.io/) is an open standard for collecting traces and metrics from the distributed application. ClickHouse has some support for OpenTelemetry. -:::note -This is an experimental feature that will change in backwards-incompatible ways in future releases. -::: - ## Supplying Trace Context to ClickHouse ClickHouse accepts trace context HTTP headers, as described by the [W3C recommendation](https://www.w3.org/TR/trace-context/). It also accepts trace context over a native protocol that is used for communication between ClickHouse servers or between the client and server. For manual testing, trace context headers conforming to the Trace Context recommendation can be supplied to `clickhouse-client` using `--opentelemetry-traceparent` and `--opentelemetry-tracestate` flags. From 55468caeaee9c7c5074897d50f5aa2c4fe4d584e Mon Sep 17 00:00:00 2001 From: Robert Schulze Date: Tue, 9 Jul 2024 16:38:22 +0000 Subject: [PATCH 347/417] Fix ARM build --- contrib/rocksdb-cmake/CMakeLists.txt | 8 ++++++++ 1 file changed, 8 insertions(+) diff --git a/contrib/rocksdb-cmake/CMakeLists.txt b/contrib/rocksdb-cmake/CMakeLists.txt index 96558b40174..3a14407166c 100644 --- a/contrib/rocksdb-cmake/CMakeLists.txt +++ b/contrib/rocksdb-cmake/CMakeLists.txt @@ -51,6 +51,14 @@ if(ENABLE_SSE42 AND ENABLE_PCLMULQDQ) add_definitions(-DHAVE_PCLMUL) endif() +if(CMAKE_SYSTEM_PROCESSOR MATCHES "arm64|aarch64|AARCH64") + set (HAS_ARMV8_CRC 1) + # the original build descriptions set specific flags for ARM. These flags are already subsumed by ClickHouse's general + # ARM flags, see cmake/cpu_features.cmake + # set(CMAKE_C_FLAGS "${CMAKE_C_FLAGS} -march=armv8-a+crc+crypto -Wno-unused-function") + # set(CMAKE_CXX_FLAGS "${CMAKE_CXX_FLAGS} -march=armv8-a+crc+crypto -Wno-unused-function") +endif() + set (HAVE_THREAD_LOCAL 1) if(HAVE_THREAD_LOCAL) add_definitions(-DROCKSDB_SUPPORT_THREAD_LOCAL) From 004d8e0d49d77879e63bcb16456af84db784200f Mon Sep 17 00:00:00 2001 From: Antonio Andelic Date: Tue, 9 Jul 2024 18:39:21 +0200 Subject: [PATCH 348/417] Run privileged --- tests/ci/functional_test_check.py | 2 ++ 1 file changed, 2 insertions(+) diff --git a/tests/ci/functional_test_check.py b/tests/ci/functional_test_check.py index d8e5a7fa27f..4440d0d332c 100644 --- a/tests/ci/functional_test_check.py +++ b/tests/ci/functional_test_check.py @@ -104,6 +104,8 @@ def get_run_command( return ( f"docker run --volume={builds_path}:/package_folder " + # For dmesg and sysctl + "--privileged " f"{ci_logs_args}" f"--volume={repo_path}/tests:/usr/share/clickhouse-test " f"{volume_with_broken_test}" From b4f59b96c274fcde50050e172a91d455eddcb17f Mon Sep 17 00:00:00 2001 From: Kseniia Sumarokova <54203879+kssenii@users.noreply.github.com> Date: Tue, 9 Jul 2024 19:30:15 +0200 Subject: [PATCH 349/417] Update IStorageDataLake.h --- src/Storages/ObjectStorage/DataLakes/IStorageDataLake.h | 4 ---- 1 file changed, 4 deletions(-) diff --git a/src/Storages/ObjectStorage/DataLakes/IStorageDataLake.h b/src/Storages/ObjectStorage/DataLakes/IStorageDataLake.h index d6935c706d9..c8603fccb86 100644 --- a/src/Storages/ObjectStorage/DataLakes/IStorageDataLake.h +++ b/src/Storages/ObjectStorage/DataLakes/IStorageDataLake.h @@ -17,10 +17,6 @@ namespace DB { -namespace ErrorCodes -{ - extern const int LOGICAL_ERROR; -} /// Storage for read-only integration with Apache Iceberg tables in Amazon S3 (see https://iceberg.apache.org/) /// Right now it's implemented on top of StorageS3 and right now it doesn't support From 9fc557ad65ab0a306e417d01ea0b4636a0569824 Mon Sep 17 00:00:00 2001 From: MikhailBurdukov Date: Tue, 9 Jul 2024 17:36:09 +0000 Subject: [PATCH 350/417] Ignore ON CLUSTER clause in queries for management of replicated named collections --- .../NamedCollectionsFactory.cpp | 9 +++++- .../NamedCollectionsFactory.h | 2 ++ .../NamedCollectionsMetadataStorage.cpp | 12 ++++---- .../NamedCollectionsMetadataStorage.h | 2 +- src/Core/Settings.h | 1 + .../InterpreterAlterNamedCollectionQuery.cpp | 7 +++-- .../InterpreterCreateNamedCollectionQuery.cpp | 7 +++-- .../InterpreterDropNamedCollectionQuery.cpp | 7 +++-- .../removeOnClusterClauseIfNeeded.cpp | 16 ++++++++++- .../named_collections_with_zookeeper.xml | 17 +++++++++++ .../configs/users.d/users.xml | 5 ++++ .../test_named_collections/test.py | 28 +++++++++++++++++++ 12 files changed, 98 insertions(+), 15 deletions(-) diff --git a/src/Common/NamedCollections/NamedCollectionsFactory.cpp b/src/Common/NamedCollections/NamedCollectionsFactory.cpp index 14105a8651d..2faea1957ba 100644 --- a/src/Common/NamedCollections/NamedCollectionsFactory.cpp +++ b/src/Common/NamedCollections/NamedCollectionsFactory.cpp @@ -235,7 +235,7 @@ bool NamedCollectionFactory::loadIfNot(std::lock_guard & lock) loadFromConfig(context->getConfigRef(), lock); loadFromSQL(lock); - if (metadata_storage->supportsPeriodicUpdate()) + if (metadata_storage->isReplicated()) { update_task = context->getSchedulePool().createTask("NamedCollectionsMetadataStorage", [this]{ updateFunc(); }); update_task->activate(); @@ -357,6 +357,13 @@ void NamedCollectionFactory::reloadFromSQL() add(std::move(collections), lock); } +bool NamedCollectionFactory::usesReplicatedStorage() +{ + std::lock_guard lock(mutex); + loadIfNot(lock); + return metadata_storage->isReplicated(); +} + void NamedCollectionFactory::updateFunc() { LOG_TRACE(log, "Named collections background updating thread started"); diff --git a/src/Common/NamedCollections/NamedCollectionsFactory.h b/src/Common/NamedCollections/NamedCollectionsFactory.h index 6ee5940e686..a0721ad8a50 100644 --- a/src/Common/NamedCollections/NamedCollectionsFactory.h +++ b/src/Common/NamedCollections/NamedCollectionsFactory.h @@ -34,6 +34,8 @@ public: void updateFromSQL(const ASTAlterNamedCollectionQuery & query); + bool usesReplicatedStorage(); + void loadIfNot(); void shutdown(); diff --git a/src/Common/NamedCollections/NamedCollectionsMetadataStorage.cpp b/src/Common/NamedCollections/NamedCollectionsMetadataStorage.cpp index 32fdb25abd3..b3671350f92 100644 --- a/src/Common/NamedCollections/NamedCollectionsMetadataStorage.cpp +++ b/src/Common/NamedCollections/NamedCollectionsMetadataStorage.cpp @@ -67,7 +67,7 @@ public: virtual bool removeIfExists(const std::string & path) = 0; - virtual bool supportsPeriodicUpdate() const = 0; + virtual bool isReplicated() const = 0; virtual bool waitUpdate(size_t /* timeout */) { return false; } }; @@ -89,7 +89,7 @@ public: ~LocalStorage() override = default; - bool supportsPeriodicUpdate() const override { return false; } + bool isReplicated() const override { return false; } std::vector list() const override { @@ -221,7 +221,7 @@ public: ~ZooKeeperStorage() override = default; - bool supportsPeriodicUpdate() const override { return true; } + bool isReplicated() const override { return true; } /// Return true if children changed. bool waitUpdate(size_t timeout) override @@ -465,14 +465,14 @@ void NamedCollectionsMetadataStorage::writeCreateQuery(const ASTCreateNamedColle storage->write(getFileName(query.collection_name), serializeAST(*normalized_query), replace); } -bool NamedCollectionsMetadataStorage::supportsPeriodicUpdate() const +bool NamedCollectionsMetadataStorage::isReplicated() const { - return storage->supportsPeriodicUpdate(); + return storage->isReplicated(); } bool NamedCollectionsMetadataStorage::waitUpdate() { - if (!storage->supportsPeriodicUpdate()) + if (!storage->isReplicated()) throw Exception(ErrorCodes::LOGICAL_ERROR, "Periodic updates are not supported"); const auto & config = Context::getGlobalContextInstance()->getConfigRef(); diff --git a/src/Common/NamedCollections/NamedCollectionsMetadataStorage.h b/src/Common/NamedCollections/NamedCollectionsMetadataStorage.h index 3c089fe2fa2..c3468fbc468 100644 --- a/src/Common/NamedCollections/NamedCollectionsMetadataStorage.h +++ b/src/Common/NamedCollections/NamedCollectionsMetadataStorage.h @@ -30,7 +30,7 @@ public: /// Return true if update was made bool waitUpdate(); - bool supportsPeriodicUpdate() const; + bool isReplicated() const; private: class INamedCollectionsStorage; diff --git a/src/Core/Settings.h b/src/Core/Settings.h index d84e5b149f6..6c53837138b 100644 --- a/src/Core/Settings.h +++ b/src/Core/Settings.h @@ -364,6 +364,7 @@ class IColumn; \ M(Bool, ignore_on_cluster_for_replicated_udf_queries, false, "Ignore ON CLUSTER clause for replicated UDF management queries.", 0) \ M(Bool, ignore_on_cluster_for_replicated_access_entities_queries, false, "Ignore ON CLUSTER clause for replicated access entities management queries.", 0) \ + M(Bool, ignore_on_cluster_for_replicated_named_collections_queries, false, "Ignore ON CLUSTER clause for replicated named collections management queries.", 0) \ /** Settings for testing hedged requests */ \ M(Milliseconds, sleep_in_send_tables_status_ms, 0, "Time to sleep in sending tables status response in TCPHandler", 0) \ M(Milliseconds, sleep_in_send_data_ms, 0, "Time to sleep in sending data in TCPHandler", 0) \ diff --git a/src/Interpreters/InterpreterAlterNamedCollectionQuery.cpp b/src/Interpreters/InterpreterAlterNamedCollectionQuery.cpp index 79a17fd1844..0e83e2039f6 100644 --- a/src/Interpreters/InterpreterAlterNamedCollectionQuery.cpp +++ b/src/Interpreters/InterpreterAlterNamedCollectionQuery.cpp @@ -4,6 +4,7 @@ #include #include #include +#include #include @@ -13,14 +14,16 @@ namespace DB BlockIO InterpreterAlterNamedCollectionQuery::execute() { auto current_context = getContext(); - const auto & query = query_ptr->as(); + + const auto updated_query = removeOnClusterClauseIfNeeded(query_ptr, getContext()); + const auto & query = updated_query->as(); current_context->checkAccess(AccessType::ALTER_NAMED_COLLECTION, query.collection_name); if (!query.cluster.empty()) { DDLQueryOnClusterParams params; - return executeDDLQueryOnCluster(query_ptr, current_context, params); + return executeDDLQueryOnCluster(updated_query, current_context, params); } NamedCollectionFactory::instance().updateFromSQL(query); diff --git a/src/Interpreters/InterpreterCreateNamedCollectionQuery.cpp b/src/Interpreters/InterpreterCreateNamedCollectionQuery.cpp index c71441daa8c..b4920b1729f 100644 --- a/src/Interpreters/InterpreterCreateNamedCollectionQuery.cpp +++ b/src/Interpreters/InterpreterCreateNamedCollectionQuery.cpp @@ -4,6 +4,7 @@ #include #include #include +#include #include @@ -13,14 +14,16 @@ namespace DB BlockIO InterpreterCreateNamedCollectionQuery::execute() { auto current_context = getContext(); - const auto & query = query_ptr->as(); + + const auto updated_query = removeOnClusterClauseIfNeeded(query_ptr, getContext()); + const auto & query = updated_query->as(); current_context->checkAccess(AccessType::CREATE_NAMED_COLLECTION, query.collection_name); if (!query.cluster.empty()) { DDLQueryOnClusterParams params; - return executeDDLQueryOnCluster(query_ptr, current_context, params); + return executeDDLQueryOnCluster(updated_query, current_context, params); } NamedCollectionFactory::instance().createFromSQL(query); diff --git a/src/Interpreters/InterpreterDropNamedCollectionQuery.cpp b/src/Interpreters/InterpreterDropNamedCollectionQuery.cpp index 2edaef1b2f2..6233d21b439 100644 --- a/src/Interpreters/InterpreterDropNamedCollectionQuery.cpp +++ b/src/Interpreters/InterpreterDropNamedCollectionQuery.cpp @@ -4,6 +4,7 @@ #include #include #include +#include #include @@ -13,14 +14,16 @@ namespace DB BlockIO InterpreterDropNamedCollectionQuery::execute() { auto current_context = getContext(); - const auto & query = query_ptr->as(); + + const auto updated_query = removeOnClusterClauseIfNeeded(query_ptr, getContext()); + const auto & query = updated_query->as(); current_context->checkAccess(AccessType::DROP_NAMED_COLLECTION, query.collection_name); if (!query.cluster.empty()) { DDLQueryOnClusterParams params; - return executeDDLQueryOnCluster(query_ptr, current_context, params); + return executeDDLQueryOnCluster(updated_query, current_context, params); } NamedCollectionFactory::instance().removeFromSQL(query); diff --git a/src/Interpreters/removeOnClusterClauseIfNeeded.cpp b/src/Interpreters/removeOnClusterClauseIfNeeded.cpp index 44167fe7242..dd20164925c 100644 --- a/src/Interpreters/removeOnClusterClauseIfNeeded.cpp +++ b/src/Interpreters/removeOnClusterClauseIfNeeded.cpp @@ -15,6 +15,10 @@ #include #include #include +#include +#include +#include +#include namespace DB @@ -38,6 +42,13 @@ static bool isAccessControlQuery(const ASTPtr & query) || query->as(); } +static bool isNamedCollectionQuery(const ASTPtr & query) +{ + return query->as() + || query->as() + || query->as(); +} + ASTPtr removeOnClusterClauseIfNeeded(const ASTPtr & query, ContextPtr context, const WithoutOnClusterASTRewriteParams & params) { auto * query_on_cluster = dynamic_cast(query.get()); @@ -50,7 +61,10 @@ ASTPtr removeOnClusterClauseIfNeeded(const ASTPtr & query, ContextPtr context, c && context->getUserDefinedSQLObjectsStorage().isReplicated()) || (isAccessControlQuery(query) && context->getSettings().ignore_on_cluster_for_replicated_access_entities_queries - && context->getAccessControl().containsStorage(ReplicatedAccessStorage::STORAGE_TYPE))) + && context->getAccessControl().containsStorage(ReplicatedAccessStorage::STORAGE_TYPE)) + || (isNamedCollectionQuery(query) + && context->getSettings().ignore_on_cluster_for_replicated_named_collections_queries + && NamedCollectionFactory::instance().usesReplicatedStorage())) { LOG_DEBUG(getLogger("removeOnClusterClauseIfNeeded"), "ON CLUSTER clause was ignored for query {}", query->getID()); return query_on_cluster->getRewrittenASTWithoutOnCluster(params); diff --git a/tests/integration/test_named_collections/configs/config.d/named_collections_with_zookeeper.xml b/tests/integration/test_named_collections/configs/config.d/named_collections_with_zookeeper.xml index 2d7946d1587..43d80ee6f69 100644 --- a/tests/integration/test_named_collections/configs/config.d/named_collections_with_zookeeper.xml +++ b/tests/integration/test_named_collections/configs/config.d/named_collections_with_zookeeper.xml @@ -9,4 +9,21 @@ value1 + + + + + true + + node_with_keeper + 9000 + + + node_with_keeper_2 + 9000 + + + true + + diff --git a/tests/integration/test_named_collections/configs/users.d/users.xml b/tests/integration/test_named_collections/configs/users.d/users.xml index 15da914f666..7d4f0543ff1 100644 --- a/tests/integration/test_named_collections/configs/users.d/users.xml +++ b/tests/integration/test_named_collections/configs/users.d/users.xml @@ -1,4 +1,9 @@ + + + 0 + + diff --git a/tests/integration/test_named_collections/test.py b/tests/integration/test_named_collections/test.py index dbc502236c0..5d38047e885 100644 --- a/tests/integration/test_named_collections/test.py +++ b/tests/integration/test_named_collections/test.py @@ -3,6 +3,8 @@ import pytest import os import time from helpers.cluster import ClickHouseCluster +from contextlib import nullcontext as does_not_raise +from helpers.client import QueryRuntimeException SCRIPT_DIR = os.path.dirname(os.path.realpath(__file__)) NAMED_COLLECTIONS_CONFIG = os.path.join( @@ -761,3 +763,29 @@ def test_keeper_storage(cluster): check_dropped(node1) check_dropped(node2) + + +@pytest.mark.parametrize( + "ignore, expected_raise", + [(True, does_not_raise()), (False, pytest.raises(QueryRuntimeException))], +) +def test_keeper_storage_remove_on_cluster(cluster, ignore, expected_raise): + node = cluster.instances["node_with_keeper"] + + replace_in_users_config( + node, + "ignore_on_cluster_for_replicated_named_collections_queries>.", + f"ignore_on_cluster_for_replicated_named_collections_queries>{int(ignore)}", + ) + node.query("SYSTEM RELOAD CONFIG") + + with expected_raise: + node.query( + f"CREATE NAMED COLLECTION test_nc ON CLUSTER `replicated_nc_nodes_cluster` AS key1=1, key2=2 OVERRIDABLE" + ) + node.query( + f"ALTER NAMED COLLECTION test_nc ON CLUSTER `replicated_nc_nodes_cluster` SET key2=3" + ) + node.query( + f"DROP NAMED COLLECTION test_nc ON CLUSTER `replicated_nc_nodes_cluster`" + ) From eeb3561c37f0554c021e100039ffb197b1058d62 Mon Sep 17 00:00:00 2001 From: Nikita Fomichev Date: Tue, 9 Jul 2024 18:16:38 +0200 Subject: [PATCH 351/417] Stateless tests: run sequential tests in parallel to other tests --- docker/test/fasttest/run.sh | 5 + docker/test/stateless/run.sh | 112 ++++++++++-- tests/ci/ci_config.py | 20 +-- tests/clickhouse-test | 169 +++++++++++++++--- tests/config/install.sh | 1 - ..._log_and_exception_messages_formatting.sql | 2 +- tests/queries/shell_config.sh | 8 + 7 files changed, 265 insertions(+), 52 deletions(-) diff --git a/docker/test/fasttest/run.sh b/docker/test/fasttest/run.sh index c015d3a3542..0d975d64010 100755 --- a/docker/test/fasttest/run.sh +++ b/docker/test/fasttest/run.sh @@ -284,6 +284,11 @@ function run_tests NPROC=1 fi + export CLICKHOUSE_CONFIG_DIR=$FASTTEST_DATA + export CLICKHOUSE_CONFIG="$FASTTEST_DATA/config.xml" + export CLICKHOUSE_USER_FILES="$FASTTEST_DATA/user_files" + export CLICKHOUSE_SCHEMA_FILES="$FASTTEST_DATA/format_schemas" + local test_opts=( --hung-check --fast-tests-only diff --git a/docker/test/stateless/run.sh b/docker/test/stateless/run.sh index 43d3c698d8a..30079073ea2 100755 --- a/docker/test/stateless/run.sh +++ b/docker/test/stateless/run.sh @@ -9,6 +9,16 @@ set -e -x -a MAX_RUN_TIME=${MAX_RUN_TIME:-10800} MAX_RUN_TIME=$((MAX_RUN_TIME == 0 ? 10800 : MAX_RUN_TIME)) +USE_DATABASE_REPLICATED=${USE_DATABASE_REPLICATED:=0} +USE_SHARED_CATALOG=${USE_SHARED_CATALOG:=0} + +# disable for now +RUN_SEQUENTIAL_TESTS_IN_PARALLEL=0 + +if [[ "$USE_DATABASE_REPLICATED" -eq 1 ]] || [[ "$USE_SHARED_CATALOG" -eq 1 ]]; then + RUN_SEQUENTIAL_TESTS_IN_PARALLEL=0 +fi + # Choose random timezone for this test run. # # NOTE: that clickhouse-test will randomize session_timezone by itself as well @@ -89,10 +99,57 @@ if [ "$NUM_TRIES" -gt "1" ]; then mkdir -p /var/run/clickhouse-server fi +# Run a CH instance to execute sequential tests on it in parallel with all other tests. +if [[ "$RUN_SEQUENTIAL_TESTS_IN_PARALLEL" -eq 1 ]]; then + mkdir -p /var/run/clickhouse-server3 /etc/clickhouse-server3 /var/lib/clickhouse3 + cp -r -L /etc/clickhouse-server/* /etc/clickhouse-server3/ + + sudo chown clickhouse:clickhouse /var/run/clickhouse-server3 /var/lib/clickhouse3 /etc/clickhouse-server3/ + sudo chown -R clickhouse:clickhouse /etc/clickhouse-server3/* + + function replace(){ + sudo find /etc/clickhouse-server3/ -type f -name '*.xml' -exec sed -i "$1" {} \; + } + + replace "s|9000|19000|g" + replace "s|9440|19440|g" + replace "s|9988|19988|g" + replace "s|9234|19234|g" + replace "s|9181|19181|g" + replace "s|8443|18443|g" + replace "s|9000|19000|g" + replace "s|9181|19181|g" + replace "s|9440|19440|g" + replace "s|9010|19010|g" + replace "s|9004|19004|g" + replace "s|9005|19005|g" + replace "s|9009|19009|g" + replace "s|8123|18123|g" + replace "s|/var/lib/clickhouse/|/var/lib/clickhouse3/|g" + replace "s|/etc/clickhouse-server/|/etc/clickhouse-server3/|g" + # distributed cache + replace "s|10001|10003|g" + replace "s|10002|10004|g" + + sudo -E -u clickhouse /usr/bin/clickhouse server --daemon --config /etc/clickhouse-server3/config.xml \ + --pid-file /var/run/clickhouse-server3/clickhouse-server.pid \ + -- --path /var/lib/clickhouse3/ --logger.stderr /var/log/clickhouse-server/stderr3.log \ + --logger.log /var/log/clickhouse-server/clickhouse-server3.log --logger.errorlog /var/log/clickhouse-server/clickhouse-server3.err.log \ + --tcp_port 19000 --tcp_port_secure 19440 --http_port 18123 --https_port 18443 --interserver_http_port 19009 --tcp_with_proxy_port 19010 \ + --prometheus.port 19988 --keeper_server.raft_configuration.server.port 19234 --keeper_server.tcp_port 19181 \ + --mysql_port 19004 --postgresql_port 19005 + + for _ in {1..100} + do + clickhouse-client --port 19000 --query "SELECT 1" && break + sleep 1 + done +fi + # simplest way to forward env variables to server sudo -E -u clickhouse /usr/bin/clickhouse-server --config /etc/clickhouse-server/config.xml --daemon --pid-file /var/run/clickhouse-server/clickhouse-server.pid -if [[ -n "$USE_DATABASE_REPLICATED" ]] && [[ "$USE_DATABASE_REPLICATED" -eq 1 ]]; then +if [[ "$USE_DATABASE_REPLICATED" -eq 1 ]]; then sudo sed -i "s|/var/lib/clickhouse/filesystem_caches/|/var/lib/clickhouse/filesystem_caches_1/|" /etc/clickhouse-server1/config.d/filesystem_caches_path.xml sudo sed -i "s|/var/lib/clickhouse/filesystem_caches/|/var/lib/clickhouse/filesystem_caches_2/|" /etc/clickhouse-server2/config.d/filesystem_caches_path.xml @@ -129,7 +186,7 @@ if [[ -n "$USE_DATABASE_REPLICATED" ]] && [[ "$USE_DATABASE_REPLICATED" -eq 1 ]] MAX_RUN_TIME=$((MAX_RUN_TIME != 0 ? MAX_RUN_TIME : 9000)) # set to 2.5 hours if 0 (unlimited) fi -if [[ -n "$USE_SHARED_CATALOG" ]] && [[ "$USE_SHARED_CATALOG" -eq 1 ]]; then +if [[ "$USE_SHARED_CATALOG" -eq 1 ]]; then sudo cat /etc/clickhouse-server1/config.d/filesystem_caches_path.xml \ | sed "s|/var/lib/clickhouse/filesystem_caches/|/var/lib/clickhouse/filesystem_caches_1/|" \ > /etc/clickhouse-server1/config.d/filesystem_caches_path.xml.tmp @@ -209,15 +266,15 @@ function run_tests() ADDITIONAL_OPTIONS+=('--no-random-merge-tree-settings') fi - if [[ -n "$USE_SHARED_CATALOG" ]] && [[ "$USE_SHARED_CATALOG" -eq 1 ]]; then + if [[ "$USE_SHARED_CATALOG" -eq 1 ]]; then ADDITIONAL_OPTIONS+=('--shared-catalog') fi - if [[ -n "$USE_DATABASE_REPLICATED" ]] && [[ "$USE_DATABASE_REPLICATED" -eq 1 ]]; then + if [[ "$USE_DATABASE_REPLICATED" -eq 1 ]]; then ADDITIONAL_OPTIONS+=('--replicated-database') # Too many tests fail for DatabaseReplicated in parallel. ADDITIONAL_OPTIONS+=('--jobs') - ADDITIONAL_OPTIONS+=('2') + ADDITIONAL_OPTIONS+=('3') elif [[ 1 == $(clickhouse-client --query "SELECT value LIKE '%SANITIZE_COVERAGE%' FROM system.build_options WHERE name = 'CXX_FLAGS'") ]]; then # Coverage on a per-test basis could only be collected sequentially. # Do not set the --jobs parameter. @@ -225,7 +282,11 @@ function run_tests() else # All other configurations are OK. ADDITIONAL_OPTIONS+=('--jobs') - ADDITIONAL_OPTIONS+=('8') + ADDITIONAL_OPTIONS+=('5') + fi + + if [[ "$RUN_SEQUENTIAL_TESTS_IN_PARALLEL" -eq 1 ]]; then + ADDITIONAL_OPTIONS+=('--run-sequential-tests-in-parallel') fi if [[ -n "$RUN_BY_HASH_NUM" ]] && [[ -n "$RUN_BY_HASH_TOTAL" ]]; then @@ -289,7 +350,7 @@ do err=$(clickhouse-client -q "select * from system.$table into outfile '/test_output/$table.tsv.gz' format TSVWithNamesAndTypes") echo "$err" [[ "0" != "${#err}" ]] && failed_to_save_logs=1 - if [[ -n "$USE_DATABASE_REPLICATED" ]] && [[ "$USE_DATABASE_REPLICATED" -eq 1 ]]; then + if [[ "$USE_DATABASE_REPLICATED" -eq 1 ]]; then err=$( { clickhouse-client --port 19000 -q "select * from system.$table format TSVWithNamesAndTypes" | zstd --threads=0 > /test_output/$table.1.tsv.zst; } 2>&1 ) echo "$err" [[ "0" != "${#err}" ]] && failed_to_save_logs=1 @@ -298,7 +359,7 @@ do [[ "0" != "${#err}" ]] && failed_to_save_logs=1 fi - if [[ -n "$USE_SHARED_CATALOG" ]] && [[ "$USE_SHARED_CATALOG" -eq 1 ]]; then + if [[ "$USE_SHARED_CATALOG" -eq 1 ]]; then err=$( { clickhouse-client --port 19000 -q "select * from system.$table format TSVWithNamesAndTypes" | zstd --threads=0 > /test_output/$table.1.tsv.zst; } 2>&1 ) echo "$err" [[ "0" != "${#err}" ]] && failed_to_save_logs=1 @@ -309,12 +370,17 @@ done # Why do we read data with clickhouse-local? # Because it's the simplest way to read it when server has crashed. sudo clickhouse stop ||: -if [[ -n "$USE_DATABASE_REPLICATED" ]] && [[ "$USE_DATABASE_REPLICATED" -eq 1 ]]; then + +if [[ "$RUN_SEQUENTIAL_TESTS_IN_PARALLEL" -eq 1 ]]; then + sudo clickhouse stop --pid-path /var/run/clickhouse-server3 ||: +fi + +if [[ "$USE_DATABASE_REPLICATED" -eq 1 ]]; then sudo clickhouse stop --pid-path /var/run/clickhouse-server1 ||: sudo clickhouse stop --pid-path /var/run/clickhouse-server2 ||: fi -if [[ -n "$USE_SHARED_CATALOG" ]] && [[ "$USE_SHARED_CATALOG" -eq 1 ]]; then +if [[ "$USE_SHARED_CATALOG" -eq 1 ]]; then sudo clickhouse stop --pid-path /var/run/clickhouse-server1 ||: fi @@ -322,6 +388,12 @@ rg -Fa "" /var/log/clickhouse-server/clickhouse-server.log ||: rg -A50 -Fa "============" /var/log/clickhouse-server/stderr.log ||: zstd --threads=0 < /var/log/clickhouse-server/clickhouse-server.log > /test_output/clickhouse-server.log.zst & +if [[ "$RUN_SEQUENTIAL_TESTS_IN_PARALLEL" -eq 1 ]]; then + rg -Fa "" /var/log/clickhouse-server3/clickhouse-server.log ||: + rg -A50 -Fa "============" /var/log/clickhouse-server3/stderr.log ||: + zstd --threads=0 < /var/log/clickhouse-server3/clickhouse-server.log > /test_output/clickhouse-server3.log.zst & +fi + data_path_config="--path=/var/lib/clickhouse/" if [[ -n "$USE_S3_STORAGE_FOR_MERGE_TREE" ]] && [[ "$USE_S3_STORAGE_FOR_MERGE_TREE" -eq 1 ]]; then # We need s3 storage configuration (but it's more likely that clickhouse-local will fail for some reason) @@ -341,12 +413,17 @@ if [ $failed_to_save_logs -ne 0 ]; then for table in query_log zookeeper_log trace_log transactions_info_log metric_log blob_storage_log error_log do clickhouse-local "$data_path_config" --only-system-tables --stacktrace -q "select * from system.$table format TSVWithNamesAndTypes" | zstd --threads=0 > /test_output/$table.tsv.zst ||: - if [[ -n "$USE_DATABASE_REPLICATED" ]] && [[ "$USE_DATABASE_REPLICATED" -eq 1 ]]; then + + if [[ "$RUN_SEQUENTIAL_TESTS_IN_PARALLEL" -eq 1 ]]; then + clickhouse-local --path /var/lib/clickhouse3/ --only-system-tables --stacktrace -q "select * from system.$table format TSVWithNamesAndTypes" | zstd --threads=0 > /test_output/$table.3.tsv.zst ||: + fi + + if [[ "$USE_DATABASE_REPLICATED" -eq 1 ]]; then clickhouse-local --path /var/lib/clickhouse1/ --only-system-tables --stacktrace -q "select * from system.$table format TSVWithNamesAndTypes" | zstd --threads=0 > /test_output/$table.1.tsv.zst ||: clickhouse-local --path /var/lib/clickhouse2/ --only-system-tables --stacktrace -q "select * from system.$table format TSVWithNamesAndTypes" | zstd --threads=0 > /test_output/$table.2.tsv.zst ||: fi - if [[ -n "$USE_SHARED_CATALOG" ]] && [[ "$USE_SHARED_CATALOG" -eq 1 ]]; then + if [[ "$USE_SHARED_CATALOG" -eq 1 ]]; then clickhouse-local --path /var/lib/clickhouse1/ --only-system-tables --stacktrace -q "select * from system.$table format TSVWithNamesAndTypes" | zstd --threads=0 > /test_output/$table.1.tsv.zst ||: fi done @@ -382,7 +459,14 @@ rm -rf /var/lib/clickhouse/data/system/*/ tar -chf /test_output/store.tar /var/lib/clickhouse/store ||: tar -chf /test_output/metadata.tar /var/lib/clickhouse/metadata/*.sql ||: -if [[ -n "$USE_DATABASE_REPLICATED" ]] && [[ "$USE_DATABASE_REPLICATED" -eq 1 ]]; then +if [[ "$RUN_SEQUENTIAL_TESTS_IN_PARALLEL" -eq 1 ]]; then + rm -rf /var/lib/clickhouse3/data/system/*/ + tar -chf /test_output/store.tar /var/lib/clickhouse3/store ||: + tar -chf /test_output/metadata.tar /var/lib/clickhouse3/metadata/*.sql ||: +fi + + +if [[ "$USE_DATABASE_REPLICATED" -eq 1 ]]; then rg -Fa "" /var/log/clickhouse-server/clickhouse-server1.log ||: rg -Fa "" /var/log/clickhouse-server/clickhouse-server2.log ||: zstd --threads=0 < /var/log/clickhouse-server/clickhouse-server1.log > /test_output/clickhouse-server1.log.zst ||: @@ -393,7 +477,7 @@ if [[ -n "$USE_DATABASE_REPLICATED" ]] && [[ "$USE_DATABASE_REPLICATED" -eq 1 ]] tar -chf /test_output/coordination2.tar /var/lib/clickhouse2/coordination ||: fi -if [[ -n "$USE_SHARED_CATALOG" ]] && [[ "$USE_SHARED_CATALOG" -eq 1 ]]; then +if [[ "$USE_SHARED_CATALOG" -eq 1 ]]; then rg -Fa "" /var/log/clickhouse-server/clickhouse-server1.log ||: zstd --threads=0 < /var/log/clickhouse-server/clickhouse-server1.log > /test_output/clickhouse-server1.log.zst ||: mv /var/log/clickhouse-server/stderr1.log /test_output/ ||: diff --git a/tests/ci/ci_config.py b/tests/ci/ci_config.py index bef43083a35..8eda6e6b96f 100644 --- a/tests/ci/ci_config.py +++ b/tests/ci/ci_config.py @@ -311,42 +311,42 @@ class CI: random_bucket="parrepl_with_sanitizer", ), JobNames.STATELESS_TEST_ASAN: CommonJobConfigs.STATELESS_TEST.with_properties( - required_builds=[BuildNames.PACKAGE_ASAN], num_batches=4 + required_builds=[BuildNames.PACKAGE_ASAN], num_batches=2 ), JobNames.STATELESS_TEST_TSAN: CommonJobConfigs.STATELESS_TEST.with_properties( - required_builds=[BuildNames.PACKAGE_TSAN], num_batches=5 + required_builds=[BuildNames.PACKAGE_TSAN], num_batches=2 ), JobNames.STATELESS_TEST_MSAN: CommonJobConfigs.STATELESS_TEST.with_properties( - required_builds=[BuildNames.PACKAGE_MSAN], num_batches=6 + required_builds=[BuildNames.PACKAGE_MSAN], num_batches=3 ), JobNames.STATELESS_TEST_UBSAN: CommonJobConfigs.STATELESS_TEST.with_properties( - required_builds=[BuildNames.PACKAGE_UBSAN], num_batches=2 + required_builds=[BuildNames.PACKAGE_UBSAN], num_batches=1 ), JobNames.STATELESS_TEST_DEBUG: CommonJobConfigs.STATELESS_TEST.with_properties( - required_builds=[BuildNames.PACKAGE_DEBUG], num_batches=5 + required_builds=[BuildNames.PACKAGE_DEBUG], num_batches=2 ), JobNames.STATELESS_TEST_RELEASE: CommonJobConfigs.STATELESS_TEST.with_properties( required_builds=[BuildNames.PACKAGE_RELEASE], ), JobNames.STATELESS_TEST_RELEASE_COVERAGE: CommonJobConfigs.STATELESS_TEST.with_properties( - required_builds=[BuildNames.PACKAGE_RELEASE_COVERAGE], num_batches=6 + required_builds=[BuildNames.PACKAGE_RELEASE_COVERAGE], num_batches=5 ), JobNames.STATELESS_TEST_AARCH64: CommonJobConfigs.STATELESS_TEST.with_properties( required_builds=[BuildNames.PACKAGE_AARCH64], runner_type=Runners.FUNC_TESTER_ARM, ), JobNames.STATELESS_TEST_OLD_ANALYZER_S3_REPLICATED_RELEASE: CommonJobConfigs.STATELESS_TEST.with_properties( - required_builds=[BuildNames.PACKAGE_RELEASE], num_batches=4 + required_builds=[BuildNames.PACKAGE_RELEASE], num_batches=3 ), JobNames.STATELESS_TEST_S3_DEBUG: CommonJobConfigs.STATELESS_TEST.with_properties( - required_builds=[BuildNames.PACKAGE_DEBUG], num_batches=6 + required_builds=[BuildNames.PACKAGE_DEBUG], num_batches=2 ), JobNames.STATELESS_TEST_AZURE_ASAN: CommonJobConfigs.STATELESS_TEST.with_properties( - required_builds=[BuildNames.PACKAGE_ASAN], num_batches=4, release_only=True + required_builds=[BuildNames.PACKAGE_ASAN], num_batches=2, release_only=True ), JobNames.STATELESS_TEST_S3_TSAN: CommonJobConfigs.STATELESS_TEST.with_properties( required_builds=[BuildNames.PACKAGE_TSAN], - num_batches=5, + num_batches=3, ), JobNames.STRESS_TEST_DEBUG: CommonJobConfigs.STRESS_TEST.with_properties( required_builds=[BuildNames.PACKAGE_DEBUG], diff --git a/tests/clickhouse-test b/tests/clickhouse-test index 8e7002af889..8dea6297a61 100755 --- a/tests/clickhouse-test +++ b/tests/clickhouse-test @@ -709,9 +709,9 @@ def get_localzone(): class SettingsRandomizer: settings = { - "max_insert_threads": lambda: ( - 0 if random.random() < 0.5 else random.randint(1, 16) - ), + "max_insert_threads": lambda: 32 + if random.random() < 0.03 + else random.randint(1, 3), "group_by_two_level_threshold": threshold_generator(0.2, 0.2, 1, 1000000), "group_by_two_level_threshold_bytes": threshold_generator( 0.2, 0.2, 1, 50000000 @@ -727,7 +727,7 @@ class SettingsRandomizer: "prefer_localhost_replica": lambda: random.randint(0, 1), "max_block_size": lambda: random.randint(8000, 100000), "max_joined_block_size_rows": lambda: random.randint(8000, 100000), - "max_threads": lambda: random.randint(1, 64), + "max_threads": lambda: 64 if random.random() < 0.03 else random.randint(1, 3), "optimize_append_index": lambda: random.randint(0, 1), "optimize_if_chain_to_multiif": lambda: random.randint(0, 1), "optimize_if_transform_strings_to_enum": lambda: random.randint(0, 1), @@ -1217,6 +1217,11 @@ class TestCase: ): return FailureReason.OBJECT_STORAGE + elif "no-batch" in tags and ( + args.run_by_hash_num is not None or args.run_by_hash_total is not None + ): + return FailureReason.SKIP + elif tags: for build_flag in args.build_flags: if "no-" + build_flag in tags: @@ -1447,8 +1452,7 @@ class TestCase: description_full = messages[result.status] description_full += self.print_test_time(result.total_time) if result.reason is not None: - description_full += " - " - description_full += result.reason.value + description_full += f"\nReason: {result.reason.value} " description_full += result.description @@ -1575,10 +1579,11 @@ class TestCase: # pylint:disable-next=consider-using-with; TODO: fix proc = Popen(command, shell=True, env=os.environ, start_new_session=True) - while ( - datetime.now() - start_time - ).total_seconds() < args.timeout and proc.poll() is None: - sleep(0.01) + try: + proc.wait(args.timeout) + except subprocess.TimeoutExpired: + # Whether the test timed out will be decided later + pass debug_log = "" if os.path.exists(self.testcase_args.debug_log_file): @@ -1600,6 +1605,44 @@ class TestCase: # Normalize hostname in stdout file. replace_in_file(self.stdout_file, socket.gethostname(), "localhost") + if os.environ.get("CLICKHOUSE_PORT_TCP"): + replace_in_file( + self.stdout_file, + f"PORT {os.environ['CLICKHOUSE_PORT_TCP']}", + "PORT 9000", + ) + replace_in_file( + self.stdout_file, + f"localhost {os.environ['CLICKHOUSE_PORT_TCP']}", + "localhost 9000", + ) + + if os.environ.get("CLICKHOUSE_PORT_TCP_SECURE"): + replace_in_file( + self.stdout_file, + f"PORT {os.environ['CLICKHOUSE_PORT_TCP_SECURE']}", + "PORT 9440", + ) + replace_in_file( + self.stdout_file, + f"localhost {os.environ['CLICKHOUSE_PORT_TCP_SECURE']}", + "localhost 9440", + ) + + if os.environ.get("CLICKHOUSE_PATH"): + replace_in_file( + self.stdout_file, + os.environ["CLICKHOUSE_PATH"], + "/var/lib/clickhouse", + ) + + if os.environ.get("CLICKHOUSE_PORT_HTTPS"): + replace_in_file( + self.stdout_file, + f"https://localhost:{os.environ['CLICKHOUSE_PORT_HTTPS']}/", + "https://localhost:8443/", + ) + stdout = "" if os.path.exists(self.stdout_file): with open(self.stdout_file, "rb") as stdfd: @@ -2056,8 +2099,13 @@ class GlobalTimeout(Exception): pass -def run_tests_array(all_tests_with_params: Tuple[List[str], int, TestSuite]): - all_tests, num_tests, test_suite = all_tests_with_params +def run_tests_array(all_tests_with_params: Tuple[List[str], int, TestSuite, bool]): + ( + all_tests, + num_tests, + test_suite, + is_concurrent, + ) = all_tests_with_params global stop_time global exit_code global server_died @@ -2100,14 +2148,12 @@ def run_tests_array(all_tests_with_params: Tuple[List[str], int, TestSuite]): failures_chain = 0 start_time = datetime.now() - is_concurrent = multiprocessing.current_process().name != "MainProcess" - client_options = get_additional_client_options(args) if num_tests > 0: about = "about " if is_concurrent else "" proc_name = multiprocessing.current_process().name - print(f"\nRunning {about}{num_tests} {test_suite.suite} tests ({proc_name}).\n") + print(f"Running {about}{num_tests} {test_suite.suite} tests ({proc_name}).") while True: if all_tests: @@ -2128,16 +2174,17 @@ def run_tests_array(all_tests_with_params: Tuple[List[str], int, TestSuite]): try: description = "" - test_cace_name = removesuffix(test_case.name, ".gen", ".sql") + ": " - if not is_concurrent: + test_case_name = removesuffix(test_case.name, ".gen", ".sql") + ": " + + if is_concurrent or args.run_sequential_tests_in_parallel: + description = f"{test_case_name:72}" + else: sys.stdout.flush() - sys.stdout.write(f"{test_cace_name:72}") + sys.stdout.write(f"{test_case_name:72}") # This flush is needed so you can see the test name of the long # running test before it will finish. But don't do it in parallel # mode, so that the lines don't mix. sys.stdout.flush() - else: - description = f"{test_cace_name:72}" while True: test_result = test_case.run( @@ -2372,6 +2419,35 @@ def extract_key(key: str) -> str: )[1] +def override_envs(*args_, **kwargs): + global args + args.client += " --port 19000" + args.http_port = 18123 + args.https_port = 18443 + + updated_env = { + "CLICKHOUSE_CONFIG": "/etc/clickhouse-server3/config.xml", + "CLICKHOUSE_CONFIG_DIR": "/etc/clickhouse-server3", + "CLICKHOUSE_CONFIG_GREP": "/etc/clickhouse-server3/preprocessed/config.xml", + "CLICKHOUSE_USER_FILES": "/var/lib/clickhouse3/user_files", + "CLICKHOUSE_SCHEMA_FILES": "/var/lib/clickhouse3/format_schemas", + "CLICKHOUSE_PATH": "/var/lib/clickhouse3", + "CLICKHOUSE_PORT_TCP": "19000", + "CLICKHOUSE_PORT_TCP_SECURE": "19440", + "CLICKHOUSE_PORT_TCP_WITH_PROXY": "19010", + "CLICKHOUSE_PORT_HTTP": "18123", + "CLICKHOUSE_PORT_HTTPS": "18443", + "CLICKHOUSE_PORT_INTERSERVER": "19009", + "CLICKHOUSE_PORT_KEEPER": "19181", + "CLICKHOUSE_PORT_PROMTHEUS_PORT": "19988", + "CLICKHOUSE_PORT_MYSQL": "19004", + "CLICKHOUSE_PORT_POSTGRESQL": "19005", + } + os.environ.update(updated_env) + + run_tests_array(*args_, **kwargs) + + def do_run_tests(jobs, test_suite: TestSuite): if jobs > 1 and len(test_suite.parallel_tests) > 0: print( @@ -2400,24 +2476,55 @@ def do_run_tests(jobs, test_suite: TestSuite): for job in range(jobs): range_ = job * batch_size, job * batch_size + batch_size batch = test_suite.parallel_tests[range_[0] : range_[1]] - parallel_tests_array.append((batch, batch_size, test_suite)) + parallel_tests_array.append((batch, batch_size, test_suite, True)) try: - with multiprocessing.Pool(processes=jobs) as pool: + with multiprocessing.Pool(processes=jobs + 1) as pool: future = pool.map_async(run_tests_array, parallel_tests_array) + + if args.run_sequential_tests_in_parallel: + # Run parallel tests and sequential tests at the same time + # Sequential tests will use different ClickHouse instance + # In this process we can safely override values in `args` and `os.environ` + future_seq = pool.map_async( + override_envs, + [ + ( + test_suite.sequential_tests, + len(test_suite.sequential_tests), + test_suite, + False, + ) + ], + ) + future_seq.wait() + future.wait() finally: pool.terminate() pool.close() pool.join() - run_tests_array( - (test_suite.sequential_tests, len(test_suite.sequential_tests), test_suite) - ) + if not args.run_sequential_tests_in_parallel: + run_tests_array( + ( + test_suite.sequential_tests, + len(test_suite.sequential_tests), + test_suite, + False, + ) + ) return len(test_suite.sequential_tests) + len(test_suite.parallel_tests) else: num_tests = len(test_suite.all_tests) - run_tests_array((test_suite.all_tests, num_tests, test_suite)) + run_tests_array( + ( + test_suite.all_tests, + num_tests, + test_suite, + False, + ) + ) return num_tests @@ -2722,6 +2829,7 @@ def main(args): f"{get_db_engine(args, db_name)}", settings=get_create_database_settings(args, None), ) + break except HTTPError as e: total_time = (datetime.now() - start_time).total_seconds() if not need_retry(args, e.message, e.message, total_time): @@ -3234,6 +3342,15 @@ def parse_args(): help="Replace ordinary MergeTree engine with SharedMergeTree", ) + parser.add_argument( + "--run-sequential-tests-in-parallel", + action="store_true", + default=False, + help="If `true`, tests with the tag `no-parallel` will run on a " + "separate ClickHouse instance in parallel with other tests. " + "This is used in CI to make test jobs run faster.", + ) + return parser.parse_args() diff --git a/tests/config/install.sh b/tests/config/install.sh index 08ee11a7407..8b58a519bc9 100755 --- a/tests/config/install.sh +++ b/tests/config/install.sh @@ -57,7 +57,6 @@ ln -sf $SRC_PATH/config.d/forbidden_headers.xml $DEST_SERVER_PATH/config.d/ ln -sf $SRC_PATH/config.d/enable_keeper_map.xml $DEST_SERVER_PATH/config.d/ ln -sf $SRC_PATH/config.d/custom_disks_base_path.xml $DEST_SERVER_PATH/config.d/ ln -sf $SRC_PATH/config.d/display_name.xml $DEST_SERVER_PATH/config.d/ -ln -sf $SRC_PATH/config.d/reverse_dns_query_function.xml $DEST_SERVER_PATH/config.d/ ln -sf $SRC_PATH/config.d/compressed_marks_and_index.xml $DEST_SERVER_PATH/config.d/ ln -sf $SRC_PATH/config.d/disable_s3_env_credentials.xml $DEST_SERVER_PATH/config.d/ ln -sf $SRC_PATH/config.d/enable_wait_for_shutdown_replicated_tables.xml $DEST_SERVER_PATH/config.d/ diff --git a/tests/queries/0_stateless/00002_log_and_exception_messages_formatting.sql b/tests/queries/0_stateless/00002_log_and_exception_messages_formatting.sql index 0ca7df8ecd3..07c42d6d039 100644 --- a/tests/queries/0_stateless/00002_log_and_exception_messages_formatting.sql +++ b/tests/queries/0_stateless/00002_log_and_exception_messages_formatting.sql @@ -1,4 +1,4 @@ --- Tags: no-parallel, no-fasttest +-- Tags: no-parallel, no-fasttest, no-ubsan, no-batch -- no-parallel because we want to run this test when most of the other tests already passed -- If this test fails, see the "Top patterns of log messages" diagnostics in the end of run.log diff --git a/tests/queries/shell_config.sh b/tests/queries/shell_config.sh index 614bfcece8f..ef2d89f0218 100644 --- a/tests/queries/shell_config.sh +++ b/tests/queries/shell_config.sh @@ -54,9 +54,17 @@ export CLICKHOUSE_OBFUSCATOR=${CLICKHOUSE_OBFUSCATOR:="${CLICKHOUSE_BINARY}-obfu export CLICKHOUSE_COMPRESSOR=${CLICKHOUSE_COMPRESSOR:="${CLICKHOUSE_BINARY}-compressor"} export CLICKHOUSE_GIT_IMPORT=${CLICKHOUSE_GIT_IMPORT="${CLICKHOUSE_BINARY}-git-import"} +export CLICKHOUSE_CONFIG_DIR=${CLICKHOUSE_CONFIG_DIR:="/etc/clickhouse-server"} export CLICKHOUSE_CONFIG=${CLICKHOUSE_CONFIG:="/etc/clickhouse-server/config.xml"} export CLICKHOUSE_CONFIG_CLIENT=${CLICKHOUSE_CONFIG_CLIENT:="/etc/clickhouse-client/config.xml"} +export CLICKHOUSE_USER_FILES=${CLICKHOUSE_USER_FILES:="/var/lib/clickhouse/user_files"} +export CLICKHOUSE_USER_FILES_UNIQUE=${CLICKHOUSE_USER_FILES_UNIQUE:="${CLICKHOUSE_USER_FILES}/${CLICKHOUSE_TEST_UNIQUE_NAME}"} +# synonym +export USER_FILES_PATH=$CLICKHOUSE_USER_FILES + +export CLICKHOUSE_SCHEMA_FILES=${CLICKHOUSE_SCHEMA_FILES:="/var/lib/clickhouse/format_schemas"} + [ -x "${CLICKHOUSE_BINARY}-extract-from-config" ] && CLICKHOUSE_EXTRACT_CONFIG=${CLICKHOUSE_EXTRACT_CONFIG:="$CLICKHOUSE_BINARY-extract-from-config --config=$CLICKHOUSE_CONFIG"} [ -x "${CLICKHOUSE_BINARY}" ] && CLICKHOUSE_EXTRACT_CONFIG=${CLICKHOUSE_EXTRACT_CONFIG:="$CLICKHOUSE_BINARY extract-from-config --config=$CLICKHOUSE_CONFIG"} export CLICKHOUSE_EXTRACT_CONFIG=${CLICKHOUSE_EXTRACT_CONFIG:="$CLICKHOUSE_BINARY-extract-from-config --config=$CLICKHOUSE_CONFIG"} From 1e72335030a4de446275ece652876887910963e2 Mon Sep 17 00:00:00 2001 From: Igor Nikonov Date: Tue, 9 Jul 2024 17:53:34 +0000 Subject: [PATCH 352/417] Failed to connect to replica log message --- src/QueryPipeline/RemoteQueryExecutor.cpp | 4 ++++ 1 file changed, 4 insertions(+) diff --git a/src/QueryPipeline/RemoteQueryExecutor.cpp b/src/QueryPipeline/RemoteQueryExecutor.cpp index bde8ce78f55..61a6bf82270 100644 --- a/src/QueryPipeline/RemoteQueryExecutor.cpp +++ b/src/QueryPipeline/RemoteQueryExecutor.cpp @@ -104,6 +104,10 @@ RemoteQueryExecutor::RemoteQueryExecutor( connection_entries.emplace_back(std::move(result.entry)); } + else + { + LOG_DEBUG(log, "Failed to connect to replica {}. {}", pool->getAddress(), fail_message); + } auto res = std::make_unique(std::move(connection_entries), context, throttler); if (extension_ && extension_->replica_info) From 5a12659f43f74aa501610404c4b2ee6b1b4a02c9 Mon Sep 17 00:00:00 2001 From: Kseniia Sumarokova <54203879+kssenii@users.noreply.github.com> Date: Tue, 9 Jul 2024 20:18:03 +0200 Subject: [PATCH 353/417] Update run.sh --- docker/test/stateless/run.sh | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/docker/test/stateless/run.sh b/docker/test/stateless/run.sh index 43d3c698d8a..637d277e6f8 100755 --- a/docker/test/stateless/run.sh +++ b/docker/test/stateless/run.sh @@ -249,7 +249,7 @@ function run_tests() try_run_with_retry 10 clickhouse-client -q "insert into system.zookeeper (name, path, value) values ('auxiliary_zookeeper2', '/test/chroot/', '')" set +e - timeout -s TERM --preserve-status 120m clickhouse-test --testname --shard --zookeeper --check-zookeeper-session --hung-check --print-time \ + timeout -s KILL --preserve-status 120m clickhouse-test --testname --shard --zookeeper --check-zookeeper-session --hung-check --print-time \ --no-drop-if-fail --test-runs "$NUM_TRIES" "${ADDITIONAL_OPTIONS[@]}" 2>&1 \ | ts '%Y-%m-%d %H:%M:%S' \ | tee -a test_output/test_result.txt From beca3b98341e9cf52a9a9840b3f216a67b454e29 Mon Sep 17 00:00:00 2001 From: Igor Nikonov Date: Tue, 9 Jul 2024 19:31:33 +0000 Subject: [PATCH 354/417] Proper placing for DistributedConnectionFailTry + better logs --- src/Client/ConnectionEstablisher.cpp | 3 +++ src/Client/HedgedConnectionsFactory.cpp | 2 -- src/Common/PoolWithFailoverBase.h | 2 -- src/QueryPipeline/RemoteQueryExecutor.cpp | 6 +++++- 4 files changed, 8 insertions(+), 5 deletions(-) diff --git a/src/Client/ConnectionEstablisher.cpp b/src/Client/ConnectionEstablisher.cpp index 303105751ad..05839b44452 100644 --- a/src/Client/ConnectionEstablisher.cpp +++ b/src/Client/ConnectionEstablisher.cpp @@ -8,6 +8,7 @@ namespace ProfileEvents extern const Event DistributedConnectionUsable; extern const Event DistributedConnectionMissingTable; extern const Event DistributedConnectionStaleReplica; + extern const Event DistributedConnectionFailTry; } namespace DB @@ -97,6 +98,8 @@ void ConnectionEstablisher::run(ConnectionEstablisher::TryResult & result, std:: } catch (const Exception & e) { + ProfileEvents::increment(ProfileEvents::DistributedConnectionFailTry); + if (e.code() != ErrorCodes::NETWORK_ERROR && e.code() != ErrorCodes::SOCKET_TIMEOUT && e.code() != ErrorCodes::ATTEMPT_TO_READ_AFTER_EOF && e.code() != ErrorCodes::DNS_ERROR) throw; diff --git a/src/Client/HedgedConnectionsFactory.cpp b/src/Client/HedgedConnectionsFactory.cpp index 0fa2bc12924..be7397b0fad 100644 --- a/src/Client/HedgedConnectionsFactory.cpp +++ b/src/Client/HedgedConnectionsFactory.cpp @@ -7,7 +7,6 @@ namespace ProfileEvents { extern const Event HedgedRequestsChangeReplica; - extern const Event DistributedConnectionFailTry; extern const Event DistributedConnectionFailAtAll; } @@ -327,7 +326,6 @@ HedgedConnectionsFactory::State HedgedConnectionsFactory::processFinishedConnect { ShuffledPool & shuffled_pool = shuffled_pools[index]; LOG_INFO(log, "Connection failed at try №{}, reason: {}", (shuffled_pool.error_count + 1), fail_message); - ProfileEvents::increment(ProfileEvents::DistributedConnectionFailTry); shuffled_pool.error_count = std::min(pool->getMaxErrorCup(), shuffled_pool.error_count + 1); shuffled_pool.slowdown_count = 0; diff --git a/src/Common/PoolWithFailoverBase.h b/src/Common/PoolWithFailoverBase.h index 2359137012c..3d4de773a36 100644 --- a/src/Common/PoolWithFailoverBase.h +++ b/src/Common/PoolWithFailoverBase.h @@ -28,7 +28,6 @@ namespace ErrorCodes namespace ProfileEvents { - extern const Event DistributedConnectionFailTry; extern const Event DistributedConnectionFailAtAll; extern const Event DistributedConnectionSkipReadOnlyReplica; } @@ -285,7 +284,6 @@ PoolWithFailoverBase::getMany( else { LOG_WARNING(log, "Connection failed at try №{}, reason: {}", (shuffled_pool.error_count + 1), fail_message); - ProfileEvents::increment(ProfileEvents::DistributedConnectionFailTry); shuffled_pool.error_count = std::min(max_error_cap, shuffled_pool.error_count + 1); diff --git a/src/QueryPipeline/RemoteQueryExecutor.cpp b/src/QueryPipeline/RemoteQueryExecutor.cpp index 61a6bf82270..14457d2df43 100644 --- a/src/QueryPipeline/RemoteQueryExecutor.cpp +++ b/src/QueryPipeline/RemoteQueryExecutor.cpp @@ -106,7 +106,11 @@ RemoteQueryExecutor::RemoteQueryExecutor( } else { - LOG_DEBUG(log, "Failed to connect to replica {}. {}", pool->getAddress(), fail_message); + chassert(!fail_message.empty()); + if (result.entry.isNull()) + LOG_DEBUG(log, "Failed to connect to replica {}. {}", pool->getAddress(), fail_message); + else + LOG_DEBUG(log, "Replica is not usable for remote query execution: {}. {}", pool->getAddress(), fail_message); } auto res = std::make_unique(std::move(connection_entries), context, throttler); From 0d54151cb81421b8eaa99df0c8abb224b776570b Mon Sep 17 00:00:00 2001 From: Nikita Mikhaylov Date: Tue, 9 Jul 2024 19:55:37 +0000 Subject: [PATCH 355/417] Make the pocketfft to point to the upstream/master branch --- contrib/pocketfft | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/contrib/pocketfft b/contrib/pocketfft index 9efd4da52cf..f4c1aa8aa9c 160000 --- a/contrib/pocketfft +++ b/contrib/pocketfft @@ -1 +1 @@ -Subproject commit 9efd4da52cf8d28d14531d14e43ad9d913807546 +Subproject commit f4c1aa8aa9ce79ad39e80f2c9c41b92ead90fda3 From 2b091983e8df97a5a103be8aa03ad2c0a836ff46 Mon Sep 17 00:00:00 2001 From: Nikita Mikhaylov Date: Tue, 9 Jul 2024 19:59:49 +0000 Subject: [PATCH 356/417] Bump Azure to https://github.com/ClickHouse/azure-sdk-for-cpp/commit/ea3e19a7be08519134c643177d56c7484dfec884 --- contrib/azure | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/contrib/azure b/contrib/azure index 92c94d7f37a..ea3e19a7be0 160000 --- a/contrib/azure +++ b/contrib/azure @@ -1 +1 @@ -Subproject commit 92c94d7f37a43cc8fc4d466884a95f610c0593bf +Subproject commit ea3e19a7be08519134c643177d56c7484dfec884 From 7ff447de686d687fab0e08a3094eb173d8d128ff Mon Sep 17 00:00:00 2001 From: Nikita Mikhaylov Date: Mon, 8 Jul 2024 16:13:03 +0000 Subject: [PATCH 357/417] Modify the code for build with new libcxx (cherry picked from commit c896f0bf677bcc948d988cf83b2108e9f7cd761d) --- base/base/demangle.h | 1 + base/base/extended_types.h | 16 ++++++++ base/base/isSharedPtrUnique.h | 9 +++++ base/poco/Foundation/include/Poco/Format.h | 2 +- .../include/Poco/RefCountedObject.h | 2 + base/poco/Foundation/src/Format.cpp | 40 +++++++++---------- base/poco/MongoDB/src/ObjectId.cpp | 2 +- base/poco/MongoDB/src/OpMsgCursor.cpp | 8 ++-- base/poco/Net/src/HTTPMessage.cpp | 12 +++--- src/Backups/BackupOperationInfo.h | 2 + src/Common/AtomicLogger.h | 2 + src/Common/ConcurrencyControl.h | 1 + src/Common/RemoteHostFilter.h | 1 + src/Coordination/Changelog.h | 1 + src/Coordination/FourLetterCommand.h | 2 + src/Databases/DatabaseAtomic.cpp | 5 ++- src/Databases/DatabaseLazy.cpp | 3 +- src/Databases/MySQL/DatabaseMySQL.cpp | 3 +- src/Functions/FunctionsBitToArray.cpp | 9 ++++- src/Interpreters/ActionsVisitor.cpp | 9 +++-- src/Interpreters/Cache/Metadata.h | 4 +- src/Interpreters/DatabaseCatalog.cpp | 6 ++- src/Interpreters/ExternalLoader.h | 1 + src/Interpreters/JIT/CHJIT.cpp | 6 +-- src/Interpreters/Session.cpp | 5 ++- src/Storages/MergeTree/MergeTreeData.cpp | 11 ++--- 26 files changed, 108 insertions(+), 55 deletions(-) create mode 100644 base/base/isSharedPtrUnique.h diff --git a/base/base/demangle.h b/base/base/demangle.h index ddca264ecab..af9ccad16c1 100644 --- a/base/base/demangle.h +++ b/base/base/demangle.h @@ -1,5 +1,6 @@ #pragma once +#include #include #include diff --git a/base/base/extended_types.h b/base/base/extended_types.h index 796167ab45d..3bf3f4ed31d 100644 --- a/base/base/extended_types.h +++ b/base/base/extended_types.h @@ -108,6 +108,14 @@ struct make_unsigned // NOLINT(readability-identifier-naming) using type = std::make_unsigned_t; }; +template <> struct make_unsigned { using type = UInt8; }; +template <> struct make_unsigned { using type = UInt8; }; +template <> struct make_unsigned { using type = UInt16; }; +template <> struct make_unsigned { using type = UInt16; }; +template <> struct make_unsigned { using type = UInt32; }; +template <> struct make_unsigned { using type = UInt32; }; +template <> struct make_unsigned { using type = UInt64; }; +template <> struct make_unsigned { using type = UInt64; }; template <> struct make_unsigned { using type = UInt128; }; template <> struct make_unsigned { using type = UInt128; }; template <> struct make_unsigned { using type = UInt256; }; @@ -121,6 +129,14 @@ struct make_signed // NOLINT(readability-identifier-naming) using type = std::make_signed_t; }; +template <> struct make_signed { using type = Int8; }; +template <> struct make_signed { using type = Int8; }; +template <> struct make_signed { using type = Int16; }; +template <> struct make_signed { using type = Int16; }; +template <> struct make_signed { using type = Int32; }; +template <> struct make_signed { using type = Int32; }; +template <> struct make_signed { using type = Int64; }; +template <> struct make_signed { using type = Int64; }; template <> struct make_signed { using type = Int128; }; template <> struct make_signed { using type = Int128; }; template <> struct make_signed { using type = Int256; }; diff --git a/base/base/isSharedPtrUnique.h b/base/base/isSharedPtrUnique.h new file mode 100644 index 00000000000..c153605ecb1 --- /dev/null +++ b/base/base/isSharedPtrUnique.h @@ -0,0 +1,9 @@ +#pragma once + +#include + +template +bool isSharedPtrUnique(const std::shared_ptr & ptr) +{ + return ptr.use_count() == 1; +} diff --git a/base/poco/Foundation/include/Poco/Format.h b/base/poco/Foundation/include/Poco/Format.h index f84be16d3ad..4f91dd44ca5 100644 --- a/base/poco/Foundation/include/Poco/Format.h +++ b/base/poco/Foundation/include/Poco/Format.h @@ -232,7 +232,7 @@ void Foundation_API format( const Any & value10); -void Foundation_API format(std::string & result, const std::string & fmt, const std::vector & values); +void Foundation_API formatVector(std::string & result, const std::string & fmt, const std::vector & values); /// Supports a variable number of arguments and is used by /// all other variants of format(). diff --git a/base/poco/Foundation/include/Poco/RefCountedObject.h b/base/poco/Foundation/include/Poco/RefCountedObject.h index db966089e00..d0d964d8390 100644 --- a/base/poco/Foundation/include/Poco/RefCountedObject.h +++ b/base/poco/Foundation/include/Poco/RefCountedObject.h @@ -21,6 +21,8 @@ #include "Poco/AtomicCounter.h" #include "Poco/Foundation.h" +#include + namespace Poco { diff --git a/base/poco/Foundation/src/Format.cpp b/base/poco/Foundation/src/Format.cpp index 9872ddff042..94ab124510d 100644 --- a/base/poco/Foundation/src/Format.cpp +++ b/base/poco/Foundation/src/Format.cpp @@ -51,8 +51,8 @@ namespace } if (width != 0) str.width(width); } - - + + void parsePrec(std::ostream& str, std::string::const_iterator& itFmt, const std::string::const_iterator& endFmt) { if (itFmt != endFmt && *itFmt == '.') @@ -67,7 +67,7 @@ namespace if (prec >= 0) str.precision(prec); } } - + char parseMod(std::string::const_iterator& itFmt, const std::string::const_iterator& endFmt) { char mod = 0; @@ -77,13 +77,13 @@ namespace { case 'l': case 'h': - case 'L': + case 'L': case '?': mod = *itFmt++; break; } } return mod; } - + std::size_t parseIndex(std::string::const_iterator& itFmt, const std::string::const_iterator& endFmt) { int index = 0; @@ -110,8 +110,8 @@ namespace case 'f': str << std::fixed; break; } } - - + + void writeAnyInt(std::ostream& str, const Any& any) { if (any.type() == typeid(char)) @@ -201,7 +201,7 @@ namespace str << RefAnyCast(*itVal++); break; case 'z': - str << AnyCast(*itVal++); + str << AnyCast(*itVal++); break; case 'I': case 'D': @@ -303,7 +303,7 @@ void format(std::string& result, const std::string& fmt, const Any& value) { std::vector args; args.push_back(value); - format(result, fmt, args); + formatVector(result, fmt, args); } @@ -312,7 +312,7 @@ void format(std::string& result, const std::string& fmt, const Any& value1, cons std::vector args; args.push_back(value1); args.push_back(value2); - format(result, fmt, args); + formatVector(result, fmt, args); } @@ -322,7 +322,7 @@ void format(std::string& result, const std::string& fmt, const Any& value1, cons args.push_back(value1); args.push_back(value2); args.push_back(value3); - format(result, fmt, args); + formatVector(result, fmt, args); } @@ -333,7 +333,7 @@ void format(std::string& result, const std::string& fmt, const Any& value1, cons args.push_back(value2); args.push_back(value3); args.push_back(value4); - format(result, fmt, args); + formatVector(result, fmt, args); } @@ -345,7 +345,7 @@ void format(std::string& result, const std::string& fmt, const Any& value1, cons args.push_back(value3); args.push_back(value4); args.push_back(value5); - format(result, fmt, args); + formatVector(result, fmt, args); } @@ -358,7 +358,7 @@ void format(std::string& result, const std::string& fmt, const Any& value1, cons args.push_back(value4); args.push_back(value5); args.push_back(value6); - format(result, fmt, args); + formatVector(result, fmt, args); } @@ -372,7 +372,7 @@ void format(std::string& result, const std::string& fmt, const Any& value1, cons args.push_back(value5); args.push_back(value6); args.push_back(value7); - format(result, fmt, args); + formatVector(result, fmt, args); } @@ -387,7 +387,7 @@ void format(std::string& result, const std::string& fmt, const Any& value1, cons args.push_back(value6); args.push_back(value7); args.push_back(value8); - format(result, fmt, args); + formatVector(result, fmt, args); } @@ -403,7 +403,7 @@ void format(std::string& result, const std::string& fmt, const Any& value1, cons args.push_back(value7); args.push_back(value8); args.push_back(value9); - format(result, fmt, args); + formatVector(result, fmt, args); } @@ -420,16 +420,16 @@ void format(std::string& result, const std::string& fmt, const Any& value1, cons args.push_back(value8); args.push_back(value9); args.push_back(value10); - format(result, fmt, args); + formatVector(result, fmt, args); } -void format(std::string& result, const std::string& fmt, const std::vector& values) +void formatVector(std::string& result, const std::string& fmt, const std::vector& values) { std::string::const_iterator itFmt = fmt.begin(); std::string::const_iterator endFmt = fmt.end(); std::vector::const_iterator itVal = values.begin(); - std::vector::const_iterator endVal = values.end(); + std::vector::const_iterator endVal = values.end(); while (itFmt != endFmt) { switch (*itFmt) diff --git a/base/poco/MongoDB/src/ObjectId.cpp b/base/poco/MongoDB/src/ObjectId.cpp index 0125c246c2d..e360d129843 100644 --- a/base/poco/MongoDB/src/ObjectId.cpp +++ b/base/poco/MongoDB/src/ObjectId.cpp @@ -57,7 +57,7 @@ std::string ObjectId::toString(const std::string& fmt) const for (int i = 0; i < 12; ++i) { - s += format(fmt, (unsigned int) _id[i]); + s += Poco::format(fmt, (unsigned int) _id[i]); } return s; } diff --git a/base/poco/MongoDB/src/OpMsgCursor.cpp b/base/poco/MongoDB/src/OpMsgCursor.cpp index bc95851ae33..6abd45ecf76 100644 --- a/base/poco/MongoDB/src/OpMsgCursor.cpp +++ b/base/poco/MongoDB/src/OpMsgCursor.cpp @@ -43,9 +43,9 @@ namespace Poco { namespace MongoDB { -static const std::string keyCursor {"cursor"}; -static const std::string keyFirstBatch {"firstBatch"}; -static const std::string keyNextBatch {"nextBatch"}; +[[ maybe_unused ]] static const std::string keyCursor {"cursor"}; +[[ maybe_unused ]] static const std::string keyFirstBatch {"firstBatch"}; +[[ maybe_unused ]] static const std::string keyNextBatch {"nextBatch"}; static Poco::Int64 cursorIdFromResponse(const MongoDB::Document& doc); @@ -131,7 +131,7 @@ OpMsgMessage& OpMsgCursor::next(Connection& connection) connection.readResponse(_response); } else -#endif +#endif { _response.clear(); _query.setCursor(_cursorID, _batchSize); diff --git a/base/poco/Net/src/HTTPMessage.cpp b/base/poco/Net/src/HTTPMessage.cpp index c0083ec410c..b7ab5543a85 100644 --- a/base/poco/Net/src/HTTPMessage.cpp +++ b/base/poco/Net/src/HTTPMessage.cpp @@ -17,9 +17,9 @@ #include "Poco/NumberFormatter.h" #include "Poco/NumberParser.h" #include "Poco/String.h" +#include #include - using Poco::NumberFormatter; using Poco::NumberParser; using Poco::icompare; @@ -75,7 +75,7 @@ void HTTPMessage::setContentLength(std::streamsize length) erase(CONTENT_LENGTH); } - + std::streamsize HTTPMessage::getContentLength() const { const std::string& contentLength = get(CONTENT_LENGTH, EMPTY); @@ -98,7 +98,7 @@ void HTTPMessage::setContentLength64(Poco::Int64 length) erase(CONTENT_LENGTH); } - + Poco::Int64 HTTPMessage::getContentLength64() const { const std::string& contentLength = get(CONTENT_LENGTH, EMPTY); @@ -133,13 +133,13 @@ void HTTPMessage::setChunkedTransferEncoding(bool flag) setTransferEncoding(IDENTITY_TRANSFER_ENCODING); } - + bool HTTPMessage::getChunkedTransferEncoding() const { return icompare(getTransferEncoding(), CHUNKED_TRANSFER_ENCODING) == 0; } - + void HTTPMessage::setContentType(const std::string& mediaType) { if (mediaType.empty()) @@ -154,7 +154,7 @@ void HTTPMessage::setContentType(const MediaType& mediaType) setContentType(mediaType.toString()); } - + const std::string& HTTPMessage::getContentType() const { return get(CONTENT_TYPE, UNKNOWN_CONTENT_TYPE); diff --git a/src/Backups/BackupOperationInfo.h b/src/Backups/BackupOperationInfo.h index 21b5284458c..71589ec3b30 100644 --- a/src/Backups/BackupOperationInfo.h +++ b/src/Backups/BackupOperationInfo.h @@ -3,6 +3,8 @@ #include #include +#include + namespace DB { diff --git a/src/Common/AtomicLogger.h b/src/Common/AtomicLogger.h index 0ece9e8a09a..c1bbdb41866 100644 --- a/src/Common/AtomicLogger.h +++ b/src/Common/AtomicLogger.h @@ -1,5 +1,7 @@ #pragma once +#include +#include #include #include diff --git a/src/Common/ConcurrencyControl.h b/src/Common/ConcurrencyControl.h index ba94502962c..9d35d7cb8b0 100644 --- a/src/Common/ConcurrencyControl.h +++ b/src/Common/ConcurrencyControl.h @@ -1,5 +1,6 @@ #pragma once +#include #include #include #include diff --git a/src/Common/RemoteHostFilter.h b/src/Common/RemoteHostFilter.h index 2b91306f405..4c8983205fa 100644 --- a/src/Common/RemoteHostFilter.h +++ b/src/Common/RemoteHostFilter.h @@ -1,5 +1,6 @@ #pragma once +#include #include #include #include diff --git a/src/Coordination/Changelog.h b/src/Coordination/Changelog.h index c9b45d9a344..0f833c17e1b 100644 --- a/src/Coordination/Changelog.h +++ b/src/Coordination/Changelog.h @@ -5,6 +5,7 @@ #include #include +#include #include #include #include diff --git a/src/Coordination/FourLetterCommand.h b/src/Coordination/FourLetterCommand.h index 2a53bade62f..e3289982b0d 100644 --- a/src/Coordination/FourLetterCommand.h +++ b/src/Coordination/FourLetterCommand.h @@ -2,9 +2,11 @@ #include "config.h" +#include #include #include #include +#include #include namespace DB diff --git a/src/Databases/DatabaseAtomic.cpp b/src/Databases/DatabaseAtomic.cpp index ccab72cfbae..0ed995c6180 100644 --- a/src/Databases/DatabaseAtomic.cpp +++ b/src/Databases/DatabaseAtomic.cpp @@ -1,4 +1,5 @@ #include +#include #include #include #include @@ -12,7 +13,7 @@ #include #include #include -#include "Common/logger_useful.h" +#include #include #include #include @@ -397,7 +398,7 @@ DatabaseAtomic::DetachedTables DatabaseAtomic::cleanupDetachedTables() LOG_DEBUG(log, "There are {} detached tables. Start searching non used tables.", detached_tables.size()); while (it != detached_tables.end()) { - if (it->second.unique()) + if (isSharedPtrUnique(it->second)) { not_in_use.emplace(it->first, it->second); it = detached_tables.erase(it); diff --git a/src/Databases/DatabaseLazy.cpp b/src/Databases/DatabaseLazy.cpp index 233db07cd68..da942cebf8f 100644 --- a/src/Databases/DatabaseLazy.cpp +++ b/src/Databases/DatabaseLazy.cpp @@ -1,6 +1,7 @@ #include #include +#include #include #include #include @@ -305,7 +306,7 @@ try String table_name = expired_tables.front().table_name; auto it = tables_cache.find(table_name); - if (!it->second.table || it->second.table.unique()) + if (!it->second.table || isSharedPtrUnique(it->second.table)) { LOG_DEBUG(log, "Drop table {} from cache.", backQuote(it->first)); it->second.table.reset(); diff --git a/src/Databases/MySQL/DatabaseMySQL.cpp b/src/Databases/MySQL/DatabaseMySQL.cpp index 1c82131af0d..bb24373a7e1 100644 --- a/src/Databases/MySQL/DatabaseMySQL.cpp +++ b/src/Databases/MySQL/DatabaseMySQL.cpp @@ -2,6 +2,7 @@ #if USE_MYSQL # include +# include # include # include # include @@ -354,7 +355,7 @@ void DatabaseMySQL::cleanOutdatedTables() { for (auto iterator = outdated_tables.begin(); iterator != outdated_tables.end();) { - if (!iterator->unique()) + if (!isSharedPtrUnique(*iterator)) ++iterator; else { diff --git a/src/Functions/FunctionsBitToArray.cpp b/src/Functions/FunctionsBitToArray.cpp index adabda1a7f8..81c80ae07bf 100644 --- a/src/Functions/FunctionsBitToArray.cpp +++ b/src/Functions/FunctionsBitToArray.cpp @@ -284,7 +284,13 @@ public: { while (x) { - result_array_values_data.push_back(std::countr_zero(x)); + /// С++20 char8_t is not an unsigned type anymore + /// https://stackoverflow.com/questions/57402464/is-c20-char8-t-the-same-as-our-old-char + // and thus you cannot use std::countr_zero on it. + if constexpr (std::is_same_v) + result_array_values_data.push_back(std::countr_zero(static_cast(x))); + else + result_array_values_data.push_back(std::countr_zero(x)); x &= (x - 1); } } @@ -336,4 +342,3 @@ REGISTER_FUNCTION(BitToArray) } } - diff --git a/src/Interpreters/ActionsVisitor.cpp b/src/Interpreters/ActionsVisitor.cpp index c3285d73145..9efb1d89a47 100644 --- a/src/Interpreters/ActionsVisitor.cpp +++ b/src/Interpreters/ActionsVisitor.cpp @@ -405,10 +405,6 @@ Block createBlockForSet( } -ScopeStack::Level::Level() = default; -ScopeStack::Level::~Level() = default; -ScopeStack::Level::Level(Level &&) noexcept = default; - FutureSetPtr makeExplicitSet( const ASTFunction * node, const ActionsDAG & actions, ContextPtr context, PreparedSets & prepared_sets) { @@ -462,6 +458,7 @@ public: for (const auto * node : index) map.emplace(node->result_name, node); } + ~Index() = default; void addNode(const ActionsDAG::Node * node) { @@ -502,6 +499,10 @@ public: } }; +ScopeStack::Level::Level() = default; +ScopeStack::Level::~Level() = default; +ScopeStack::Level::Level(Level &&) noexcept = default; + ActionsMatcher::Data::Data( ContextPtr context_, SizeLimits set_size_limit_, diff --git a/src/Interpreters/Cache/Metadata.h b/src/Interpreters/Cache/Metadata.h index a5c8f3c0cf4..d2158457a44 100644 --- a/src/Interpreters/Cache/Metadata.h +++ b/src/Interpreters/Cache/Metadata.h @@ -6,6 +6,8 @@ #include #include #include + +#include #include namespace DB @@ -30,7 +32,7 @@ struct FileSegmentMetadata : private boost::noncopyable explicit FileSegmentMetadata(FileSegmentPtr && file_segment_); - bool releasable() const { return file_segment.unique(); } + bool releasable() const { return file_segment.use_count() == 1; } size_t size() const; diff --git a/src/Interpreters/DatabaseCatalog.cpp b/src/Interpreters/DatabaseCatalog.cpp index 841decf29c5..0cc88ac62f5 100644 --- a/src/Interpreters/DatabaseCatalog.cpp +++ b/src/Interpreters/DatabaseCatalog.cpp @@ -1,3 +1,4 @@ +#include #include #include #include @@ -27,6 +28,7 @@ #include #include +#include #include #include "config.h" @@ -1197,7 +1199,7 @@ void DatabaseCatalog::dequeueDroppedTableCleanup(StorageID table_id) /// It's unsafe to create another instance while the old one exists /// We cannot wait on shared_ptr's refcount, so it's busy wait - while (!dropped_table.table.unique()) + while (!isSharedPtrUnique(dropped_table.table)) std::this_thread::sleep_for(std::chrono::milliseconds(100)); dropped_table.table.reset(); @@ -1237,7 +1239,7 @@ void DatabaseCatalog::dropTableDataTask() size_t tables_in_use_count = 0; auto it = std::find_if(tables_marked_dropped.begin(), tables_marked_dropped.end(), [&](const auto & elem) { - bool not_in_use = !elem.table || elem.table.unique(); + bool not_in_use = !elem.table || isSharedPtrUnique(elem.table); bool old_enough = elem.drop_time <= current_time; min_drop_time = std::min(min_drop_time, elem.drop_time); tables_in_use_count += !not_in_use; diff --git a/src/Interpreters/ExternalLoader.h b/src/Interpreters/ExternalLoader.h index 49b5e68d821..6356a174a01 100644 --- a/src/Interpreters/ExternalLoader.h +++ b/src/Interpreters/ExternalLoader.h @@ -2,6 +2,7 @@ #include #include +#include #include #include #include diff --git a/src/Interpreters/JIT/CHJIT.cpp b/src/Interpreters/JIT/CHJIT.cpp index 21c773ee1d7..c35d2442d1f 100644 --- a/src/Interpreters/JIT/CHJIT.cpp +++ b/src/Interpreters/JIT/CHJIT.cpp @@ -18,12 +18,12 @@ #include #include #include -#include +// #include #include #include -#include +// #include #include -#include +// #include #include #include diff --git a/src/Interpreters/Session.cpp b/src/Interpreters/Session.cpp index bb8c415602f..fb80b12ee60 100644 --- a/src/Interpreters/Session.cpp +++ b/src/Interpreters/Session.cpp @@ -1,5 +1,6 @@ #include +#include #include #include #include @@ -130,7 +131,7 @@ public: LOG_TRACE(log, "Reuse session from storage with session_id: {}, user_id: {}", key.second, key.first); - if (!session.unique()) + if (!isSharedPtrUnique(session)) throw Exception(ErrorCodes::SESSION_IS_LOCKED, "Session {} is locked by a concurrent client", session_id); return {session, false}; } @@ -156,7 +157,7 @@ public: return; } - if (!it->second.unique()) + if (!isSharedPtrUnique(it->second)) throw Exception(ErrorCodes::LOGICAL_ERROR, "Cannot close session {} with refcount {}", session_id, it->second.use_count()); sessions.erase(it); diff --git a/src/Storages/MergeTree/MergeTreeData.cpp b/src/Storages/MergeTree/MergeTreeData.cpp index 467a5c82141..e31f6db5409 100644 --- a/src/Storages/MergeTree/MergeTreeData.cpp +++ b/src/Storages/MergeTree/MergeTreeData.cpp @@ -87,6 +87,7 @@ #include #include +#include #include #include @@ -2464,7 +2465,7 @@ MergeTreeData::DataPartsVector MergeTreeData::grabOldParts(bool force) } /// Grab only parts that are not used by anyone (SELECTs for example). - if (!part.unique()) + if (!isSharedPtrUnique(part)) { part->removal_state.store(DataPartRemovalState::NON_UNIQUE_OWNERSHIP, std::memory_order_relaxed); skipped_parts.push_back(part->info); @@ -4360,13 +4361,13 @@ bool MergeTreeData::tryRemovePartImmediately(DataPartPtr && part) part.reset(); - if (!((*it)->getState() == DataPartState::Outdated && it->unique())) + if (!((*it)->getState() == DataPartState::Outdated && isSharedPtrUnique(*it))) { if ((*it)->getState() != DataPartState::Outdated) LOG_WARNING(log, "Cannot immediately remove part {} because it's not in Outdated state " "usage counter {}", part_name_with_state, it->use_count()); - if (!it->unique()) + if (!isSharedPtrUnique(*it)) LOG_WARNING(log, "Cannot immediately remove part {} because someone using it right now " "usage counter {}", part_name_with_state, it->use_count()); return false; @@ -4432,7 +4433,7 @@ size_t MergeTreeData::getNumberOfOutdatedPartsWithExpiredRemovalTime() const for (const auto & part : outdated_parts_range) { auto part_remove_time = part->remove_time.load(std::memory_order_relaxed); - if (part_remove_time <= time_now && time_now - part_remove_time >= getSettings()->old_parts_lifetime.totalSeconds() && part.unique()) + if (part_remove_time <= time_now && time_now - part_remove_time >= getSettings()->old_parts_lifetime.totalSeconds() && isSharedPtrUnique(part)) ++res; } @@ -8640,7 +8641,7 @@ size_t MergeTreeData::unloadPrimaryKeysOfOutdatedParts() /// Outdated part may be hold by SELECT query and still needs the index. /// This check requires lock of index_mutex but if outdated part is unique then there is no /// contention on it, so it's relatively cheap and it's ok to check under a global parts lock. - if (part.unique() && part->isIndexLoaded()) + if (isSharedPtrUnique(part) && part->isIndexLoaded()) parts_to_unload_index.push_back(part); } } From ab70de1dc9f2a604f1c5cc0933133ce812a2216c Mon Sep 17 00:00:00 2001 From: Nikita Mikhaylov Date: Mon, 8 Jul 2024 16:16:54 +0000 Subject: [PATCH 358/417] Delete unneeded changes (cherry picked from commit e28e83673d975f00b6df4c0d2a61e8015b2d434b) --- src/Interpreters/Cache/Metadata.h | 5 ++++- src/Interpreters/JIT/CHJIT.cpp | 6 +++--- 2 files changed, 7 insertions(+), 4 deletions(-) diff --git a/src/Interpreters/Cache/Metadata.h b/src/Interpreters/Cache/Metadata.h index d2158457a44..0e85ead3265 100644 --- a/src/Interpreters/Cache/Metadata.h +++ b/src/Interpreters/Cache/Metadata.h @@ -1,5 +1,8 @@ #pragma once + + #include +#include #include #include #include @@ -32,7 +35,7 @@ struct FileSegmentMetadata : private boost::noncopyable explicit FileSegmentMetadata(FileSegmentPtr && file_segment_); - bool releasable() const { return file_segment.use_count() == 1; } + bool releasable() const { return isSharedPtrUnique(file_segment); } size_t size() const; diff --git a/src/Interpreters/JIT/CHJIT.cpp b/src/Interpreters/JIT/CHJIT.cpp index c35d2442d1f..21c773ee1d7 100644 --- a/src/Interpreters/JIT/CHJIT.cpp +++ b/src/Interpreters/JIT/CHJIT.cpp @@ -18,12 +18,12 @@ #include #include #include -// #include +#include #include #include -// #include +#include #include -// #include +#include #include #include From 8bb48fa9022e55f5f37eca2ca06019fe9f4eabb7 Mon Sep 17 00:00:00 2001 From: Nikita Mikhaylov Date: Tue, 9 Jul 2024 20:24:16 +0000 Subject: [PATCH 359/417] Correct comment for std::countr_zero --- src/Functions/FunctionsBitToArray.cpp | 5 ++--- 1 file changed, 2 insertions(+), 3 deletions(-) diff --git a/src/Functions/FunctionsBitToArray.cpp b/src/Functions/FunctionsBitToArray.cpp index 81c80ae07bf..beaaccad6db 100644 --- a/src/Functions/FunctionsBitToArray.cpp +++ b/src/Functions/FunctionsBitToArray.cpp @@ -284,9 +284,8 @@ public: { while (x) { - /// С++20 char8_t is not an unsigned type anymore - /// https://stackoverflow.com/questions/57402464/is-c20-char8-t-the-same-as-our-old-char - // and thus you cannot use std::countr_zero on it. + /// С++20 char8_t is not an unsigned integral type anymore https://godbolt.org/z/Mqcb7qn58 + /// and thus you cannot use std::countr_zero on it. if constexpr (std::is_same_v) result_array_values_data.push_back(std::countr_zero(static_cast(x))); else From 65bdf3bccc2a2c4a033a9b49cbee04ba88aa6842 Mon Sep 17 00:00:00 2001 From: Sema Checherinda <104093494+CheSema@users.noreply.github.com> Date: Tue, 9 Jul 2024 22:36:36 +0200 Subject: [PATCH 360/417] Update src/Interpreters/DatabaseCatalog.cpp --- src/Interpreters/DatabaseCatalog.cpp | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/src/Interpreters/DatabaseCatalog.cpp b/src/Interpreters/DatabaseCatalog.cpp index e34aef125fb..7087a04e9b4 100644 --- a/src/Interpreters/DatabaseCatalog.cpp +++ b/src/Interpreters/DatabaseCatalog.cpp @@ -837,7 +837,7 @@ DatabaseCatalog::DatabaseCatalog(ContextMutablePtr global_context_) , loading_dependencies{"LoadingDeps"} , view_dependencies{"ViewDeps"} , log(getLogger("DatabaseCatalog")) - , limitedLog(std::make_shared(log, 1, 5)) + , limitedLog(std::make_shared(log, 1, 20)) , first_async_drop_in_queue(tables_marked_dropped.end()) { } From ebc87d0c702e9bb26814718fec97e4c938735dec Mon Sep 17 00:00:00 2001 From: Kseniia Sumarokova <54203879+kssenii@users.noreply.github.com> Date: Tue, 9 Jul 2024 22:58:06 +0200 Subject: [PATCH 361/417] Update run.sh --- docker/test/stateless/run.sh | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/docker/test/stateless/run.sh b/docker/test/stateless/run.sh index 637d277e6f8..1c03f5107b0 100755 --- a/docker/test/stateless/run.sh +++ b/docker/test/stateless/run.sh @@ -249,7 +249,7 @@ function run_tests() try_run_with_retry 10 clickhouse-client -q "insert into system.zookeeper (name, path, value) values ('auxiliary_zookeeper2', '/test/chroot/', '')" set +e - timeout -s KILL --preserve-status 120m clickhouse-test --testname --shard --zookeeper --check-zookeeper-session --hung-check --print-time \ + timeout -s TERM --preserve-status 120m -k 60m clickhouse-test --testname --shard --zookeeper --check-zookeeper-session --hung-check --print-time \ --no-drop-if-fail --test-runs "$NUM_TRIES" "${ADDITIONAL_OPTIONS[@]}" 2>&1 \ | ts '%Y-%m-%d %H:%M:%S' \ | tee -a test_output/test_result.txt From 73db17b499377b3a85a7f4c651649e88e79aad2c Mon Sep 17 00:00:00 2001 From: Alexander Tokmakov Date: Wed, 10 Jul 2024 00:19:55 +0200 Subject: [PATCH 362/417] Update parallel_skip.json --- tests/integration/parallel_skip.json | 1 + 1 file changed, 1 insertion(+) diff --git a/tests/integration/parallel_skip.json b/tests/integration/parallel_skip.json index 33dd85aceaf..3c3d1b6cc96 100644 --- a/tests/integration/parallel_skip.json +++ b/tests/integration/parallel_skip.json @@ -48,6 +48,7 @@ "test_system_metrics/test.py::test_readonly_metrics", "test_system_replicated_fetches/test.py::test_system_replicated_fetches", "test_zookeeper_config_load_balancing/test.py::test_round_robin", + "test_zookeeper_config_load_balancing/test.py::test_az", "test_zookeeper_fallback_session/test.py::test_fallback_session", "test_global_overcommit_tracker/test.py::test_global_overcommit", From 88d92367b7dc69a6df1407edaaa58e17f9eacfcf Mon Sep 17 00:00:00 2001 From: Nikita Fomichev Date: Wed, 10 Jul 2024 00:24:53 +0200 Subject: [PATCH 363/417] Stateless tests: run sequential tests in parallel to other tests --- ...ard_memory_tracker_and_exception_safety.sh | 2 +- .../00429_long_http_bufferization.sh | 2 +- .../00600_replace_running_query.sh | 12 +- .../0_stateless/00623_truncate_all_tables.sql | 81 +- .../0_stateless/00623_truncate_table.sql | 3 - .../0_stateless/00719_parallel_ddl_db.sh | 11 +- .../0_stateless/00763_lock_buffer_long.sh | 2 +- .../00816_long_concurrent_alter_column.sh | 2 +- ...ong_concurrent_select_and_drop_deadlock.sh | 4 +- .../0_stateless/00910_buffer_prewhere.sql | 13 +- .../00938_template_input_format.sh | 5 +- .../00989_parallel_parts_loading.sql | 2 - ...rrent_recreate_reattach_and_show_tables.sh | 4 +- ...ent_move_partition_from_table_zookeeper.sh | 4 +- ...rfluous_dict_reload_on_create_database.sql | 34 +- .../01037_polygon_dicts_correctness_all.ans | 2000 ++++++++--------- .../01037_polygon_dicts_correctness_all.sh | 31 +- .../01037_polygon_dicts_correctness_fast.ans | 2000 ++++++++--------- .../01037_polygon_dicts_correctness_fast.sh | 29 +- .../01038_dictionary_lifetime_min_zero_sec.sh | 28 +- ...ionary_invalidate_query_switchover_long.sh | 37 +- ...em_reload_dictionary_reloads_completely.sh | 35 +- .../0_stateless/01053_ssd_dictionary.sh | 2 - .../0_stateless/01055_compact_parts_1.sql | 7 - tests/queries/0_stateless/01060_avro.sh | 5 +- .../0_stateless/01069_database_memory.sql | 2 - ...cache_dictionary_datarace_exception_ptr.sh | 22 +- ...arallel_alter_add_drop_column_zookeeper.sh | 4 +- ...79_parallel_alter_modify_zookeeper_long.sh | 4 +- .../01083_expressions_in_engine_arguments.sql | 4 +- .../0_stateless/01098_msgpack_format.sh | 5 +- .../01111_create_drop_replicated_db_stress.sh | 2 +- ...01113_local_dictionary_type_conversion.sql | 18 +- .../0_stateless/01114_database_atomic.sh | 2 +- .../01114_mysql_database_engine_segfault.sql | 2 +- .../01125_dict_ddl_cannot_add_column.sql | 12 +- ..._month_partitioning_consistency_select.sql | 3 - .../0_stateless/01154_move_partition_long.sh | 2 +- .../01171_mv_select_insert_isolation_long.sh | 2 +- .../01185_create_or_replace_table.sql | 2 +- .../01188_attach_table_from_path.sql | 2 +- .../01225_drop_dictionary_as_table.sql | 19 +- .../01254_dict_create_without_db.sql | 21 +- .../01254_dict_load_after_detach_attach.sql | 27 +- .../01259_dictionary_custom_settings_ddl.sql | 20 +- .../0_stateless/01269_alias_type_differs.sql | 2 - .../0_stateless/01272_suspicious_codecs.sql | 4 - .../01280_ssd_complex_key_dictionary.sh | 2 - ..._recreate_reattach_and_show_tables_long.sh | 4 +- ...1_aggregate_state_exception_memory_leak.sh | 2 +- ...2_aggregate_state_exception_memory_leak.sh | 2 +- .../01338_long_select_and_alter.sh | 2 +- .../01338_long_select_and_alter_zookeeper.sh | 2 +- .../01355_alter_column_with_order.sql | 34 +- tests/queries/0_stateless/01355_ilike.sql | 6 +- .../0_stateless/01388_clear_all_columns.sql | 2 - .../0_stateless/01391_join_on_dict_crash.sql | 17 +- .../0_stateless/01392_column_resolve.sql | 31 +- ...nactive_replica_cleanup_nodes_zookeeper.sh | 2 +- .../01412_cache_dictionary_race.sh | 4 +- .../0_stateless/01415_sticking_mutations.sh | 2 +- .../01442_merge_detach_attach_long.sh | 4 +- .../01444_create_table_drop_database_race.sh | 12 +- ...01454_storagememory_data_race_challenge.sh | 6 +- .../01543_avro_deserialization_with_lc.sh | 2 - .../01593_concurrent_alter_mutations_kill.sh | 4 +- .../01658_read_file_to_stringcolumn.sh | 27 +- .../01684_ssd_cache_dictionary_simple_key.sh | 2 - .../01685_ssd_cache_dictionary_complex_key.sh | 2 - .../01710_projection_vertical_merges.sql | 2 +- .../01747_join_view_filter_dictionary.sql | 2 +- .../01747_system_session_log_long.sh | 12 +- .../01748_dictionary_table_dot.sql | 2 +- ...1780_clickhouse_dictionary_source_loop.sql | 6 +- .../0_stateless/01825_type_json_btc.sh | 9 +- .../01825_type_json_multiple_files.sh | 17 +- .../01825_type_json_schema_inference.sh | 7 +- ...75_ssd_cache_dictionary_decimal256_type.sh | 2 - ..._row_policy_defined_using_user_function.sh | 35 +- .../0_stateless/01889_sqlite_read_write.sh | 8 +- .../01903_ssd_cache_dictionary_array_type.sh | 2 - ..._cache_dictionary_default_nullable_type.sh | 2 - .../0_stateless/01910_view_dictionary.sql | 2 +- ...nt_ttl_and_normal_merges_zookeeper_long.sh | 4 +- ...th_escape_sequence_at_the_end_of_buffer.sh | 6 +- .../queries/0_stateless/02003_compress_bz2.sh | 1 - .../queries/0_stateless/02012_compress_lz4.sh | 1 - .../02022_storage_filelog_one_file.sh | 14 +- .../0_stateless/02023_storage_filelog.sh | 43 +- .../0_stateless/02024_storage_filelog_mv.sh | 25 +- .../02025_storage_filelog_virtual_col.sh | 31 +- .../02026_storage_filelog_largefile.sh | 21 +- .../queries/0_stateless/02030_capnp_format.sh | 7 +- .../02051_symlinks_to_user_files.sh | 5 +- ...2103_tsv_csv_custom_null_representation.sh | 2 +- ...3_with_names_and_types_parallel_parsing.sh | 7 +- .../02104_json_strings_nullable_string.sh | 1 - ...02105_table_function_file_partiotion_by.sh | 8 +- .../02115_write_buffers_finalize.sh | 2 +- ...7_custom_separated_with_names_and_types.sh | 1 - .../02118_deserialize_whole_text.sh | 42 +- .../02125_tskv_proper_names_reading.sh | 2 - .../queries/0_stateless/02126_fix_filelog.sh | 14 +- .../0_stateless/02129_skip_quoted_fields.sh | 1 - .../0_stateless/02130_parse_quoted_null.sh | 6 +- .../02149_external_schema_inference.sh | 9 +- .../0_stateless/02149_schema_inference.sh | 8 +- ...49_schema_inference_create_table_syntax.sh | 7 +- .../02167_format_from_file_extension.sh | 30 +- .../0_stateless/02185_orc_corrupted_file.sh | 1 - .../02207_allow_plaintext_and_no_password.sh | 2 +- ...2_create_table_without_columns_metadata.sh | 2 - .../02227_test_create_empty_sqlite_db.sh | 5 +- .../02228_merge_tree_insert_memory_usage.sql | 2 +- ...w_orc_parquet_nullable_schema_inference.sh | 1 - .../02245_parquet_skip_unknown_type.sh | 1 - ...46_tsv_csv_best_effort_schema_inference.sh | 3 +- .../02247_names_order_in_json_and_tskv.sh | 1 - .../02247_read_bools_as_numbers_json.sh | 5 +- .../0_stateless/02270_errors_in_files.sh | 14 +- .../02286_mysql_dump_input_format.sh | 2 - .../0_stateless/02293_formats_json_columns.sh | 2 - .../02297_regex_parsing_file_names.sh | 36 +- ...02327_capnproto_protobuf_empty_messages.sh | 3 +- .../0_stateless/02353_compression_level.sh | 3 +- .../0_stateless/02358_file_default_value.sh | 1 - .../02360_clickhouse_local_config-option.sh | 2 +- .../0_stateless/02372_data_race_in_avro.sh | 2 +- .../02373_heap_buffer_overflow_in_avro.sh | 2 - .../02383_arrow_dict_special_cases.sh | 1 - .../02402_capnp_format_segments_overflow.sh | 3 +- ...02421_record_errors_row_by_input_format.sh | 13 +- .../02422_allow_implicit_no_password.sh | 2 +- .../02455_one_row_from_csv_memory_usage.sh | 10 +- .../0_stateless/02457_bz2_concatenated.sh | 1 - ..._glob_for_recursive_directory_traversal.sh | 32 +- .../0_stateless/02475_bson_each_row_format.sh | 2 +- ...condition_between_insert_and_droppin_mv.sh | 2 +- .../02482_capnp_list_of_structs.sh | 3 +- .../0_stateless/02483_capnp_decimals.sh | 12 +- .../02504_regexp_dictionary_ua_parser.sh | 21 +- .../02504_regexp_dictionary_yaml_source.sh | 3 - .../0_stateless/02661_read_from_archive.lib | 22 +- ...02703_keeper_map_concurrent_create_drop.sh | 4 +- .../0_stateless/02722_database_filesystem.sh | 5 +- .../02724_decompress_filename_exception.sh | 1 - .../02732_rename_after_processing.sh | 5 +- ...02771_multidirectory_globs_storage_file.sh | 29 +- .../0_stateless/02889_file_log_save_errors.sh | 18 +- .../02892_input_csv_cr_end_count_many_rows.sh | 4 +- .../0_stateless/02895_npy_output_format.sh | 99 +- .../queries/0_stateless/02931_file_cluster.sh | 2 - ...33_change_cache_setting_without_restart.sh | 2 +- ...ynamically_change_filesystem_cache_size.sh | 2 +- ...2950_dictionary_ssd_cache_short_circuit.sh | 4 +- .../02961_storage_config_volume_priority.sh | 2 +- ..._sync_replica_lightweight_from_modifier.sh | 2 +- .../02968_file_log_multiple_read.sh | 7 +- .../0_stateless/02971_analyzer_remote_id.sh | 10 +- .../02973_parse_crlf_with_tsv_files.sh | 2 - .../queries/0_stateless/02984_form_format.sh | 4 +- ...2_dynamically_resize_filesystem_cache_2.sh | 2 +- ...r_add_drop_column_zookeeper_on_steroids.sh | 2 +- .../03153_format_regexp_usability.sh | 2 +- 164 files changed, 2594 insertions(+), 2944 deletions(-) diff --git a/tests/queries/0_stateless/00133_long_shard_memory_tracker_and_exception_safety.sh b/tests/queries/0_stateless/00133_long_shard_memory_tracker_and_exception_safety.sh index a42fd58190a..d57efaa1f0e 100755 --- a/tests/queries/0_stateless/00133_long_shard_memory_tracker_and_exception_safety.sh +++ b/tests/queries/0_stateless/00133_long_shard_memory_tracker_and_exception_safety.sh @@ -1,5 +1,5 @@ #!/usr/bin/env bash -# Tags: long, shard, no-parallel +# Tags: long, shard CURDIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd) # shellcheck source=../shell_config.sh diff --git a/tests/queries/0_stateless/00429_long_http_bufferization.sh b/tests/queries/0_stateless/00429_long_http_bufferization.sh index 98dd300e6ab..83a6a4e8043 100755 --- a/tests/queries/0_stateless/00429_long_http_bufferization.sh +++ b/tests/queries/0_stateless/00429_long_http_bufferization.sh @@ -1,5 +1,5 @@ #!/usr/bin/env bash -# Tags: long, no-parallel +# Tags: long set -e diff --git a/tests/queries/0_stateless/00600_replace_running_query.sh b/tests/queries/0_stateless/00600_replace_running_query.sh index 6a682210489..7a71d17f19b 100755 --- a/tests/queries/0_stateless/00600_replace_running_query.sh +++ b/tests/queries/0_stateless/00600_replace_running_query.sh @@ -1,5 +1,4 @@ #!/usr/bin/env bash -# Tags: no-parallel CLICKHOUSE_CLIENT_SERVER_LOGS_LEVEL=none @@ -7,9 +6,10 @@ CURDIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd) # shellcheck source=../shell_config.sh . "$CURDIR"/../shell_config.sh -${CLICKHOUSE_CLIENT} -q "drop user if exists u_00600" -${CLICKHOUSE_CLIENT} -q "create user u_00600 settings max_execution_time=60, readonly=1" -${CLICKHOUSE_CLIENT} -q "grant select on system.numbers to u_00600" +TEST_PREFIX=$RANDOM +${CLICKHOUSE_CLIENT} -q "drop user if exists u_00600${TEST_PREFIX}" +${CLICKHOUSE_CLIENT} -q "create user u_00600${TEST_PREFIX} settings max_execution_time=60, readonly=1" +${CLICKHOUSE_CLIENT} -q "grant select on system.numbers to u_00600${TEST_PREFIX}" function wait_for_query_to_start() { @@ -26,7 +26,7 @@ $CLICKHOUSE_CURL -sS "$CLICKHOUSE_URL&query_id=hello&replace_running_query=1" -d # Wait for it to be replaced wait -${CLICKHOUSE_CLIENT_BINARY} --user=u_00600 --query_id=42 --query='SELECT 2, count() FROM system.numbers' 2>&1 | grep -cF 'was cancelled' & +${CLICKHOUSE_CLIENT_BINARY} --user=u_00600${TEST_PREFIX} --query_id=42 --query='SELECT 2, count() FROM system.numbers' 2>&1 | grep -cF 'was cancelled' & wait_for_query_to_start '42' # Trying to run another query with the same query_id @@ -43,4 +43,4 @@ wait_for_query_to_start '42' ${CLICKHOUSE_CLIENT} --query_id=42 --replace_running_query=1 --replace_running_query_max_wait_ms=500 --query='SELECT 43' 2>&1 | grep -F "can't be stopped" > /dev/null wait ${CLICKHOUSE_CLIENT} --query_id=42 --replace_running_query=1 --query='SELECT 44' -${CLICKHOUSE_CLIENT} -q "drop user u_00600" +${CLICKHOUSE_CLIENT} -q "drop user u_00600${TEST_PREFIX}" diff --git a/tests/queries/0_stateless/00623_truncate_all_tables.sql b/tests/queries/0_stateless/00623_truncate_all_tables.sql index 2d5e9d48f59..2626f7ed285 100644 --- a/tests/queries/0_stateless/00623_truncate_all_tables.sql +++ b/tests/queries/0_stateless/00623_truncate_all_tables.sql @@ -1,50 +1,43 @@ --- Tags: no-parallel - -DROP DATABASE IF EXISTS truncate_test; - -CREATE DATABASE IF NOT EXISTS truncate_test; -CREATE TABLE IF NOT EXISTS truncate_test.truncate_test_set(id UInt64) ENGINE = Set; -CREATE TABLE IF NOT EXISTS truncate_test.truncate_test_log(id UInt64) ENGINE = Log; -CREATE TABLE IF NOT EXISTS truncate_test.truncate_test_memory(id UInt64) ENGINE = Memory; -CREATE TABLE IF NOT EXISTS truncate_test.truncate_test_tiny_log(id UInt64) ENGINE = TinyLog; -CREATE TABLE IF NOT EXISTS truncate_test.truncate_test_stripe_log(id UInt64) ENGINE = StripeLog; -CREATE TABLE IF NOT EXISTS truncate_test.truncate_test_merge_tree(p Date, k UInt64) ENGINE = MergeTree ORDER BY p; +CREATE TABLE IF NOT EXISTS truncate_test_set(id UInt64) ENGINE = Set; +CREATE TABLE IF NOT EXISTS truncate_test_log(id UInt64) ENGINE = Log; +CREATE TABLE IF NOT EXISTS truncate_test_memory(id UInt64) ENGINE = Memory; +CREATE TABLE IF NOT EXISTS truncate_test_tiny_log(id UInt64) ENGINE = TinyLog; +CREATE TABLE IF NOT EXISTS truncate_test_stripe_log(id UInt64) ENGINE = StripeLog; +CREATE TABLE IF NOT EXISTS truncate_test_merge_tree(p Date, k UInt64) ENGINE = MergeTree ORDER BY p; SELECT '======Before Truncate======'; -INSERT INTO truncate_test.truncate_test_set VALUES(0); -INSERT INTO truncate_test.truncate_test_log VALUES(1); -INSERT INTO truncate_test.truncate_test_memory VALUES(1); -INSERT INTO truncate_test.truncate_test_tiny_log VALUES(1); -INSERT INTO truncate_test.truncate_test_stripe_log VALUES(1); -INSERT INTO truncate_test.truncate_test_merge_tree VALUES('2000-01-01', 1); -SELECT * FROM system.numbers WHERE number NOT IN truncate_test.truncate_test_set LIMIT 1; -SELECT * FROM truncate_test.truncate_test_log; -SELECT * FROM truncate_test.truncate_test_memory; -SELECT * FROM truncate_test.truncate_test_tiny_log; -SELECT * FROM truncate_test.truncate_test_stripe_log; -SELECT * FROM truncate_test.truncate_test_merge_tree; +INSERT INTO truncate_test_set VALUES(0); +INSERT INTO truncate_test_log VALUES(1); +INSERT INTO truncate_test_memory VALUES(1); +INSERT INTO truncate_test_tiny_log VALUES(1); +INSERT INTO truncate_test_stripe_log VALUES(1); +INSERT INTO truncate_test_merge_tree VALUES('2000-01-01', 1); +SELECT * FROM system.numbers WHERE number NOT IN truncate_test_set LIMIT 1; +SELECT * FROM truncate_test_log; +SELECT * FROM truncate_test_memory; +SELECT * FROM truncate_test_tiny_log; +SELECT * FROM truncate_test_stripe_log; +SELECT * FROM truncate_test_merge_tree; SELECT '======After Truncate And Empty======'; -TRUNCATE ALL TABLES FROM IF EXISTS truncate_test; -SELECT * FROM system.numbers WHERE number NOT IN truncate_test.truncate_test_set LIMIT 1; -SELECT * FROM truncate_test.truncate_test_log; -SELECT * FROM truncate_test.truncate_test_memory; -SELECT * FROM truncate_test.truncate_test_tiny_log; -SELECT * FROM truncate_test.truncate_test_stripe_log; -SELECT * FROM truncate_test.truncate_test_merge_tree; +TRUNCATE ALL TABLES FROM IF EXISTS {CLICKHOUSE_DATABASE:Identifier}; +SELECT * FROM system.numbers WHERE number NOT IN truncate_test_set LIMIT 1; +SELECT * FROM truncate_test_log; +SELECT * FROM truncate_test_memory; +SELECT * FROM truncate_test_tiny_log; +SELECT * FROM truncate_test_stripe_log; +SELECT * FROM truncate_test_merge_tree; SELECT '======After Truncate And Insert Data======'; -INSERT INTO truncate_test.truncate_test_set VALUES(0); -INSERT INTO truncate_test.truncate_test_log VALUES(1); -INSERT INTO truncate_test.truncate_test_memory VALUES(1); -INSERT INTO truncate_test.truncate_test_tiny_log VALUES(1); -INSERT INTO truncate_test.truncate_test_stripe_log VALUES(1); -INSERT INTO truncate_test.truncate_test_merge_tree VALUES('2000-01-01', 1); -SELECT * FROM system.numbers WHERE number NOT IN truncate_test.truncate_test_set LIMIT 1; -SELECT * FROM truncate_test.truncate_test_log; -SELECT * FROM truncate_test.truncate_test_memory; -SELECT * FROM truncate_test.truncate_test_tiny_log; -SELECT * FROM truncate_test.truncate_test_stripe_log; -SELECT * FROM truncate_test.truncate_test_merge_tree; - -DROP DATABASE IF EXISTS truncate_test; +INSERT INTO truncate_test_set VALUES(0); +INSERT INTO truncate_test_log VALUES(1); +INSERT INTO truncate_test_memory VALUES(1); +INSERT INTO truncate_test_tiny_log VALUES(1); +INSERT INTO truncate_test_stripe_log VALUES(1); +INSERT INTO truncate_test_merge_tree VALUES('2000-01-01', 1); +SELECT * FROM system.numbers WHERE number NOT IN truncate_test_set LIMIT 1; +SELECT * FROM truncate_test_log; +SELECT * FROM truncate_test_memory; +SELECT * FROM truncate_test_tiny_log; +SELECT * FROM truncate_test_stripe_log; +SELECT * FROM truncate_test_merge_tree; diff --git a/tests/queries/0_stateless/00623_truncate_table.sql b/tests/queries/0_stateless/00623_truncate_table.sql index 4a67e49acda..e35803db1d9 100644 --- a/tests/queries/0_stateless/00623_truncate_table.sql +++ b/tests/queries/0_stateless/00623_truncate_table.sql @@ -1,6 +1,5 @@ set allow_deprecated_syntax_for_merge_tree=1; -DROP DATABASE IF EXISTS truncate_test; DROP TABLE IF EXISTS truncate_test_log; DROP TABLE IF EXISTS truncate_test_memory; DROP TABLE IF EXISTS truncate_test_tiny_log; @@ -9,7 +8,6 @@ DROP TABLE IF EXISTS truncate_test_merge_tree; DROP TABLE IF EXISTS truncate_test_materialized_view; DROP TABLE IF EXISTS truncate_test_materialized_depend; -CREATE DATABASE truncate_test; CREATE TABLE truncate_test_set(id UInt64) ENGINE = Set; CREATE TABLE truncate_test_log(id UInt64) ENGINE = Log; CREATE TABLE truncate_test_memory(id UInt64) ENGINE = Memory; @@ -75,4 +73,3 @@ DROP TABLE IF EXISTS truncate_test_stripe_log; DROP TABLE IF EXISTS truncate_test_merge_tree; DROP TABLE IF EXISTS truncate_test_materialized_view; DROP TABLE IF EXISTS truncate_test_materialized_depend; -DROP DATABASE IF EXISTS truncate_test; diff --git a/tests/queries/0_stateless/00719_parallel_ddl_db.sh b/tests/queries/0_stateless/00719_parallel_ddl_db.sh index b7dea25c182..ceba24df7e4 100755 --- a/tests/queries/0_stateless/00719_parallel_ddl_db.sh +++ b/tests/queries/0_stateless/00719_parallel_ddl_db.sh @@ -1,13 +1,12 @@ #!/usr/bin/env bash -# Tags: no-parallel - set -e CURDIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd) # shellcheck source=../shell_config.sh . "$CURDIR"/../shell_config.sh -${CLICKHOUSE_CLIENT} --query "DROP DATABASE IF EXISTS parallel_ddl" +DB_SUFFIX=$RANDOM +${CLICKHOUSE_CLIENT} --query "DROP DATABASE IF EXISTS parallel_ddl_${DB_SUFFIX}" function query() { @@ -16,8 +15,8 @@ function query() while [ $SECONDS -lt "$TIMELIMIT" ] && [ $it -lt 50 ]; do it=$((it+1)) - ${CLICKHOUSE_CLIENT} --query "CREATE DATABASE IF NOT EXISTS parallel_ddl" - ${CLICKHOUSE_CLIENT} --query "DROP DATABASE IF EXISTS parallel_ddl" + ${CLICKHOUSE_CLIENT} --query "CREATE DATABASE IF NOT EXISTS parallel_ddl_${DB_SUFFIX}" + ${CLICKHOUSE_CLIENT} --query "DROP DATABASE IF EXISTS parallel_ddl_${DB_SUFFIX}" done } @@ -27,4 +26,4 @@ done wait -${CLICKHOUSE_CLIENT} --query "DROP DATABASE IF EXISTS parallel_ddl" +${CLICKHOUSE_CLIENT} --query "DROP DATABASE IF EXISTS parallel_ddl_${DB_SUFFIX}" diff --git a/tests/queries/0_stateless/00763_lock_buffer_long.sh b/tests/queries/0_stateless/00763_lock_buffer_long.sh index 2006d43cdd2..444a66767aa 100755 --- a/tests/queries/0_stateless/00763_lock_buffer_long.sh +++ b/tests/queries/0_stateless/00763_lock_buffer_long.sh @@ -21,7 +21,7 @@ function thread1() function thread2() { - seq 1 1000 | sed -r -e 's/.+/SELECT count() FROM buffer_00763_2;/' | ${CLICKHOUSE_CLIENT} --multiquery --server_logs_file='/dev/null' --ignore-error 2>&1 | grep -vP '^0$|^10$|^Received exception|^Code: 60|^Code: 218|^Code: 473' | grep -v '(query: ' + seq 1 500 | sed -r -e 's/.+/SELECT count() FROM buffer_00763_2;/' | ${CLICKHOUSE_CLIENT} --multiquery --server_logs_file='/dev/null' --ignore-error 2>&1 | grep -vP '^0$|^10$|^Received exception|^Code: 60|^Code: 218|^Code: 473' | grep -v '(query: ' } thread1 & diff --git a/tests/queries/0_stateless/00816_long_concurrent_alter_column.sh b/tests/queries/0_stateless/00816_long_concurrent_alter_column.sh index 71acc11b971..0ed9593c689 100755 --- a/tests/queries/0_stateless/00816_long_concurrent_alter_column.sh +++ b/tests/queries/0_stateless/00816_long_concurrent_alter_column.sh @@ -50,7 +50,7 @@ export -f thread2; export -f thread3; export -f thread4; -TIMEOUT=30 +TIMEOUT=20 timeout $TIMEOUT bash -c thread1 2> /dev/null & timeout $TIMEOUT bash -c thread2 2> /dev/null & diff --git a/tests/queries/0_stateless/00840_long_concurrent_select_and_drop_deadlock.sh b/tests/queries/0_stateless/00840_long_concurrent_select_and_drop_deadlock.sh index 238cdcea547..ae728c8d10d 100755 --- a/tests/queries/0_stateless/00840_long_concurrent_select_and_drop_deadlock.sh +++ b/tests/queries/0_stateless/00840_long_concurrent_select_and_drop_deadlock.sh @@ -1,5 +1,5 @@ #!/usr/bin/env bash -# Tags: deadlock, no-parallel, no-debug +# Tags: deadlock, no-debug # NOTE: database = $CLICKHOUSE_DATABASE is unwanted @@ -49,7 +49,7 @@ function thread_select() export -f thread_drop_create export -f thread_select -TIMEOUT=60 +TIMEOUT=30 thread_drop_create $TIMEOUT & thread_select $TIMEOUT & diff --git a/tests/queries/0_stateless/00910_buffer_prewhere.sql b/tests/queries/0_stateless/00910_buffer_prewhere.sql index deda0db85fb..e6b1cc424ad 100644 --- a/tests/queries/0_stateless/00910_buffer_prewhere.sql +++ b/tests/queries/0_stateless/00910_buffer_prewhere.sql @@ -1,9 +1,4 @@ --- Tags: no-parallel - -DROP DATABASE IF EXISTS test_buffer; -CREATE DATABASE test_buffer; -CREATE TABLE test_buffer.mt (uid UInt64, ts DateTime, val Float64) ENGINE = MergeTree PARTITION BY toDate(ts) ORDER BY (uid, ts); -CREATE TABLE test_buffer.buf as test_buffer.mt ENGINE = Buffer(test_buffer, mt, 2, 10, 60, 10000, 100000, 1000000, 10000000); -INSERT INTO test_buffer.buf VALUES (1, '2019-03-01 10:00:00', 0.5), (2, '2019-03-02 10:00:00', 0.15), (1, '2019-03-03 10:00:00', 0.25); -SELECT count() from test_buffer.buf prewhere ts > toDateTime('2019-03-01 12:00:00') and ts < toDateTime('2019-03-02 12:00:00'); -DROP DATABASE test_buffer; +CREATE TABLE mt (uid UInt64, ts DateTime, val Float64) ENGINE = MergeTree PARTITION BY toDate(ts) ORDER BY (uid, ts); +CREATE TABLE buf as mt ENGINE = Buffer({CLICKHOUSE_DATABASE:Identifier}, mt, 2, 10, 60, 10000, 100000, 1000000, 10000000); +INSERT INTO buf VALUES (1, '2019-03-01 10:00:00', 0.5), (2, '2019-03-02 10:00:00', 0.15), (1, '2019-03-03 10:00:00', 0.25); +SELECT count() from buf prewhere ts > toDateTime('2019-03-01 12:00:00') and ts < toDateTime('2019-03-02 12:00:00'); diff --git a/tests/queries/0_stateless/00938_template_input_format.sh b/tests/queries/0_stateless/00938_template_input_format.sh index be75edcdb61..016e662ea3b 100755 --- a/tests/queries/0_stateless/00938_template_input_format.sh +++ b/tests/queries/0_stateless/00938_template_input_format.sh @@ -1,12 +1,13 @@ #!/usr/bin/env bash -# Tags: no-parallel - # shellcheck disable=SC2016,SC2028 CURDIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd) # shellcheck source=../shell_config.sh . "$CURDIR"/../shell_config.sh +CURDIR=$CURDIR/${CLICKHOUSE_DATABASE} +mkdir -p $CURDIR + $CLICKHOUSE_CLIENT --query="DROP TABLE IF EXISTS template1"; $CLICKHOUSE_CLIENT --query="DROP TABLE IF EXISTS template2"; $CLICKHOUSE_CLIENT --query="CREATE TABLE template1 (s1 String, s2 String, s3 String, s4 String, n UInt64, d Date) ENGINE = Memory"; diff --git a/tests/queries/0_stateless/00989_parallel_parts_loading.sql b/tests/queries/0_stateless/00989_parallel_parts_loading.sql index a05515cf756..407e124f137 100644 --- a/tests/queries/0_stateless/00989_parallel_parts_loading.sql +++ b/tests/queries/0_stateless/00989_parallel_parts_loading.sql @@ -1,5 +1,3 @@ --- Tags: no-parallel - DROP TABLE IF EXISTS mt; CREATE TABLE mt (x UInt64) ENGINE = MergeTree ORDER BY x SETTINGS parts_to_delay_insert = 100000, parts_to_throw_insert = 100000; diff --git a/tests/queries/0_stateless/01014_lazy_database_concurrent_recreate_reattach_and_show_tables.sh b/tests/queries/0_stateless/01014_lazy_database_concurrent_recreate_reattach_and_show_tables.sh index 0a6888a5c69..3046fcbcd73 100755 --- a/tests/queries/0_stateless/01014_lazy_database_concurrent_recreate_reattach_and_show_tables.sh +++ b/tests/queries/0_stateless/01014_lazy_database_concurrent_recreate_reattach_and_show_tables.sh @@ -1,5 +1,5 @@ #!/usr/bin/env bash -# Tags: no-parallel, no-fasttest +# Tags: no-fasttest CURDIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd) # shellcheck source=../shell_config.sh @@ -89,7 +89,7 @@ ${CLICKHOUSE_CLIENT} -n -q " " -TIMEOUT=30 +TIMEOUT=20 timeout $TIMEOUT bash -c recreate_lazy_func1 2> /dev/null & timeout $TIMEOUT bash -c recreate_lazy_func2 2> /dev/null & diff --git a/tests/queries/0_stateless/01035_concurrent_move_partition_from_table_zookeeper.sh b/tests/queries/0_stateless/01035_concurrent_move_partition_from_table_zookeeper.sh index 06a460f3600..0d57bb25543 100755 --- a/tests/queries/0_stateless/01035_concurrent_move_partition_from_table_zookeeper.sh +++ b/tests/queries/0_stateless/01035_concurrent_move_partition_from_table_zookeeper.sh @@ -1,5 +1,5 @@ #!/usr/bin/env bash -# Tags: zookeeper, no-parallel, no-fasttest +# Tags: zookeeper, no-fasttest set -e @@ -61,7 +61,7 @@ export -f thread3; export -f thread4; export -f thread5; -TIMEOUT=30 +TIMEOUT=20 timeout $TIMEOUT bash -c thread1 2> /dev/null & timeout $TIMEOUT bash -c thread2 2> /dev/null & diff --git a/tests/queries/0_stateless/01036_no_superfluous_dict_reload_on_create_database.sql b/tests/queries/0_stateless/01036_no_superfluous_dict_reload_on_create_database.sql index 9040d7b3231..d0841124706 100644 --- a/tests/queries/0_stateless/01036_no_superfluous_dict_reload_on_create_database.sql +++ b/tests/queries/0_stateless/01036_no_superfluous_dict_reload_on_create_database.sql @@ -1,35 +1,25 @@ --- Tags: no-parallel - -DROP DATABASE IF EXISTS dict_db_01036; -CREATE DATABASE dict_db_01036; - -CREATE TABLE dict_db_01036.dict_data (key UInt64, val UInt64) Engine=Memory(); -CREATE DICTIONARY dict_db_01036.dict +CREATE TABLE dict_data (key UInt64, val UInt64) Engine=Memory(); +CREATE DICTIONARY dict ( key UInt64 DEFAULT 0, val UInt64 DEFAULT 10 ) PRIMARY KEY key -SOURCE(CLICKHOUSE(HOST 'localhost' PORT tcpPort() USER 'default' TABLE 'dict_data' PASSWORD '' DB 'dict_db_01036')) +SOURCE(CLICKHOUSE(HOST 'localhost' PORT tcpPort() USER 'default' TABLE 'dict_data' PASSWORD '' DB currentDatabase())) LIFETIME(MIN 0 MAX 0) LAYOUT(FLAT()); -SELECT query_count FROM system.dictionaries WHERE database = 'dict_db_01036' AND name = 'dict'; -SELECT dictGetUInt64('dict_db_01036.dict', 'val', toUInt64(0)); -SELECT query_count FROM system.dictionaries WHERE database = 'dict_db_01036' AND name = 'dict'; +SELECT query_count FROM system.dictionaries WHERE database = currentDatabase() AND name = 'dict'; +SELECT dictGetUInt64('dict', 'val', toUInt64(0)); +SELECT query_count FROM system.dictionaries WHERE database = currentDatabase() AND name = 'dict'; SELECT 'SYSTEM RELOAD DICTIONARY'; -SYSTEM RELOAD DICTIONARY dict_db_01036.dict; -SELECT query_count FROM system.dictionaries WHERE database = 'dict_db_01036' AND name = 'dict'; -SELECT dictGetUInt64('dict_db_01036.dict', 'val', toUInt64(0)); -SELECT query_count FROM system.dictionaries WHERE database = 'dict_db_01036' AND name = 'dict'; +SYSTEM RELOAD DICTIONARY dict; +SELECT query_count FROM system.dictionaries WHERE database = currentDatabase() AND name = 'dict'; +SELECT dictGetUInt64('dict', 'val', toUInt64(0)); +SELECT query_count FROM system.dictionaries WHERE database = currentDatabase() AND name = 'dict'; SELECT 'CREATE DATABASE'; DROP DATABASE IF EXISTS empty_db_01036; -CREATE DATABASE empty_db_01036; -SELECT query_count FROM system.dictionaries WHERE database = 'dict_db_01036' AND name = 'dict'; - -DROP DICTIONARY dict_db_01036.dict; -DROP TABLE dict_db_01036.dict_data; -DROP DATABASE dict_db_01036; -DROP DATABASE empty_db_01036; +CREATE DATABASE IF NOT EXISTS empty_db_01036; +SELECT query_count FROM system.dictionaries WHERE database = currentDatabase() AND name = 'dict'; diff --git a/tests/queries/0_stateless/01037_polygon_dicts_correctness_all.ans b/tests/queries/0_stateless/01037_polygon_dicts_correctness_all.ans index 6e31edbdd40..0a3f4123eb8 100644 --- a/tests/queries/0_stateless/01037_polygon_dicts_correctness_all.ans +++ b/tests/queries/0_stateless/01037_polygon_dicts_correctness_all.ans @@ -1,1000 +1,1000 @@ -dictGet test_01037.dict_array (29.5699,2.50068) 101 -dictGet test_01037.dict_array (29.5796,1.55456) 101 -dictGet test_01037.dict_array (29.5796,2.36864) 101 -dictGet test_01037.dict_array (29.5844,1.59626) 101 -dictGet test_01037.dict_array (29.5886,4.03321) 101 -dictGet test_01037.dict_array (29.5914,3.02628) 101 -dictGet test_01037.dict_array (29.5926,-0.0965169) 101 -dictGet test_01037.dict_array (29.5968,2.37773) 101 -dictGet test_01037.dict_array (29.5984,0.755853) 101 -dictGet test_01037.dict_array (29.6066,3.47173) 101 -dictGet test_01037.dict_array (29.6085,-1.26007) 101 -dictGet test_01037.dict_array (29.6131,0.246565) 101 -dictGet test_01037.dict_array (29.6157,-0.266687) 101 -dictGet test_01037.dict_array (29.6164,2.94674) 101 -dictGet test_01037.dict_array (29.6195,-0.591941) 101 -dictGet test_01037.dict_array (29.6231,1.54818) 101 -dictGet test_01037.dict_array (29.6379,0.764114) 101 -dictGet test_01037.dict_array (29.6462,-0.772059) 934570 -dictGet test_01037.dict_array (29.6579,-1.07336) 101 -dictGet test_01037.dict_array (29.6618,-0.271842) 101 -dictGet test_01037.dict_array (29.6629,-0.303602) 101 -dictGet test_01037.dict_array (29.6659,-0.782823) 934570 -dictGet test_01037.dict_array (29.6736,-0.113832) 101 -dictGet test_01037.dict_array (29.6759,3.02905) 101 -dictGet test_01037.dict_array (29.6778,3.71898) 101 -dictGet test_01037.dict_array (29.6796,1.10433) 101 -dictGet test_01037.dict_array (29.6809,2.13677) 101 -dictGet test_01037.dict_array (29.6935,4.11894) 101 -dictGet test_01037.dict_array (29.6991,-1.4458199999999999) 101 -dictGet test_01037.dict_array (29.6997,3.17297) 101 -dictGet test_01037.dict_array (29.7043,3.6145899999999997) 101 -dictGet test_01037.dict_array (29.7065,3.24885) 101 -dictGet test_01037.dict_array (29.7126,0.28108) 101 -dictGet test_01037.dict_array (29.7192,0.174273) 101 -dictGet test_01037.dict_array (29.7217,-0.523481) 934570 -dictGet test_01037.dict_array (29.7271,1.67967) 101 -dictGet test_01037.dict_array (29.7311,4.12444) 101 -dictGet test_01037.dict_array (29.7347,1.88378) 101 -dictGet test_01037.dict_array (29.7358,0.67944) 101 -dictGet test_01037.dict_array (29.7366,-0.2973) 101 -dictGet test_01037.dict_array (29.7446,0.646536) 101 -dictGet test_01037.dict_array (29.7453,-0.567963) 101 -dictGet test_01037.dict_array (29.764,4.04217) 101 -dictGet test_01037.dict_array (29.7655,1.51372) 101 -dictGet test_01037.dict_array (29.7744,1.12435) 101 -dictGet test_01037.dict_array (29.7774,-0.0681196) 101 -dictGet test_01037.dict_array (29.7784,1.54864) 101 -dictGet test_01037.dict_array (29.7785,2.24139) 101 -dictGet test_01037.dict_array (29.7922,0.220808) 101 -dictGet test_01037.dict_array (29.7936,2.37709) 101 -dictGet test_01037.dict_array (29.8008,0.948536) 101 -dictGet test_01037.dict_array (29.8115,0.201227) 101 -dictGet test_01037.dict_array (29.814,0.149601) 101 -dictGet test_01037.dict_array (29.8193,-1.35858) 101 -dictGet test_01037.dict_array (29.8201,0.965518) 101 -dictGet test_01037.dict_array (29.8265,-0.727286) 101 -dictGet test_01037.dict_array (29.8277,-0.531746) 101 -dictGet test_01037.dict_array (29.8289,3.63009) 101 -dictGet test_01037.dict_array (29.8548,0.838047) 101 -dictGet test_01037.dict_array (29.8641,-0.845265) 101 -dictGet test_01037.dict_array (29.8649,0.0562212) 101 -dictGet test_01037.dict_array (29.8701,-1.02045) 101 -dictGet test_01037.dict_array (29.8733,2.76654) 101 -dictGet test_01037.dict_array (29.876,0.555475) 101 -dictGet test_01037.dict_array (29.8794,-0.800108) 101 -dictGet test_01037.dict_array (29.8813,2.7426399999999997) 101 -dictGet test_01037.dict_array (29.897100000000002,2.66193) 101 -dictGet test_01037.dict_array (29.908,4.01339) 101 -dictGet test_01037.dict_array (29.9165,-1.08246) 101 -dictGet test_01037.dict_array (29.9201,-0.420861) 101 -dictGet test_01037.dict_array (29.9217,3.03778) 101 -dictGet test_01037.dict_array (29.9355,0.773833) 101 -dictGet test_01037.dict_array (29.947,3.76517) 101 -dictGet test_01037.dict_array (29.9518,-0.60557) 101 -dictGet test_01037.dict_array (29.9564,-0.600163) 101 -dictGet test_01037.dict_array (29.959600000000002,4.16591) 101 -dictGet test_01037.dict_array (29.9615,-1.33708) 101 -dictGet test_01037.dict_array (29.9699,-0.392375) 101 -dictGet test_01037.dict_array (29.9776,1.04552) 101 -dictGet test_01037.dict_array (29.9784,4.02756) 101 -dictGet test_01037.dict_array (29.9819,4.00597) 101 -dictGet test_01037.dict_array (29.9826,1.2816100000000001) 101 -dictGet test_01037.dict_array (30.0026,2.76257) 101 -dictGet test_01037.dict_array (30.0126,3.68255) 101 -dictGet test_01037.dict_array (30.0131,0.796576) 101 -dictGet test_01037.dict_array (30.018,1.16523) 101 -dictGet test_01037.dict_array (30.0261,-0.210653) 101 -dictGet test_01037.dict_array (30.0472,-1.11007) 101 -dictGet test_01037.dict_array (30.0542,-0.479585) 101 -dictGet test_01037.dict_array (30.0613,1.6278000000000001) 101 -dictGet test_01037.dict_array (30.0617,-0.0551152) 101 -dictGet test_01037.dict_array (30.0637,2.62066) 101 -dictGet test_01037.dict_array (30.0721,1.6424400000000001) 101 -dictGet test_01037.dict_array (30.0769,-0.402636) 101 -dictGet test_01037.dict_array (30.0791,-0.277435) 101 -dictGet test_01037.dict_array (30.0931,0.0327512) 101 -dictGet test_01037.dict_array (30.1059,3.52623) 101 -dictGet test_01037.dict_array (30.1103,0.865466) 101 -dictGet test_01037.dict_array (30.1115,2.95243) 101 -dictGet test_01037.dict_array (30.1144,1.71029) 101 -dictGet test_01037.dict_array (30.1311,-0.864751) 101 -dictGet test_01037.dict_array (30.1336,-0.851386) 101 -dictGet test_01037.dict_array (30.1393,3.89901) 101 -dictGet test_01037.dict_array (30.1456,-0.531898) 101 -dictGet test_01037.dict_array (30.1492,2.07833) 101 -dictGet test_01037.dict_array (30.1575,2.43856) 101 -dictGet test_01037.dict_array (30.1682,1.19771) 101 -dictGet test_01037.dict_array (30.1716,3.9853300000000003) 101 -dictGet test_01037.dict_array (30.1849,2.78374) 101 -dictGet test_01037.dict_array (30.1866,0.65658) 101 -dictGet test_01037.dict_array (30.1885,1.56943) 101 -dictGet test_01037.dict_array (30.1959,-1.38202) 101 -dictGet test_01037.dict_array (30.1999,1.58413) 101 -dictGet test_01037.dict_array (30.2024,0.713081) 101 -dictGet test_01037.dict_array (30.2054,0.620143) 101 -dictGet test_01037.dict_array (30.2091,1.51641) 101 -dictGet test_01037.dict_array (30.2124,-0.331782) 101 -dictGet test_01037.dict_array (30.226,3.03527) 101 -dictGet test_01037.dict_array (30.2261,3.18486) 101 -dictGet test_01037.dict_array (30.2288,2.48407) 101 -dictGet test_01037.dict_array (30.2345,3.7462400000000002) 101 -dictGet test_01037.dict_array (30.2375,0.62046) 101 -dictGet test_01037.dict_array (30.2425,-0.472914) 101 -dictGet test_01037.dict_array (30.247,3.95863) 101 -dictGet test_01037.dict_array (30.2494,-0.305093) 101 -dictGet test_01037.dict_array (30.2499,2.54337) 101 -dictGet test_01037.dict_array (30.2606,2.16644) 101 -dictGet test_01037.dict_array (30.2672,3.94847) 101 -dictGet test_01037.dict_array (30.2709,-0.136264) 101 -dictGet test_01037.dict_array (30.2764,1.18654) 101 -dictGet test_01037.dict_array (30.2765,1.20383) 101 -dictGet test_01037.dict_array (30.2839,1.05762) 101 -dictGet test_01037.dict_array (30.286,0.469327) 101 -dictGet test_01037.dict_array (30.2927,3.1693) 101 -dictGet test_01037.dict_array (30.2935,3.49854) 101 -dictGet test_01037.dict_array (30.307,0.312338) 101 -dictGet test_01037.dict_array (30.3085,1.07791) 101 -dictGet test_01037.dict_array (30.3139,2.77248) 101 -dictGet test_01037.dict_array (30.314,0.822823) 101 -dictGet test_01037.dict_array (30.3227,-0.587351) 101 -dictGet test_01037.dict_array (30.332,1.00174) 101 -dictGet test_01037.dict_array (30.3388,0.844148) 101 -dictGet test_01037.dict_array (30.3485,0.561902) 101 -dictGet test_01037.dict_array (30.3497,0.180362) 101 -dictGet test_01037.dict_array (30.361,4.13016) 101 -dictGet test_01037.dict_array (30.3623,-0.0484027) 101 -dictGet test_01037.dict_array (30.3638,3.9845800000000002) 101 -dictGet test_01037.dict_array (30.3853,3.16051) 101 -dictGet test_01037.dict_array (30.3974,2.6617800000000003) 101 -dictGet test_01037.dict_array (30.4002,-1.15886) 101 -dictGet test_01037.dict_array (30.4008,-0.387015) 101 -dictGet test_01037.dict_array (30.4018,1.86493) 101 -dictGet test_01037.dict_array (30.4239,1.16818) 101 -dictGet test_01037.dict_array (30.4363,3.63938) 101 -dictGet test_01037.dict_array (30.4377,-0.81315) 101 -dictGet test_01037.dict_array (30.4391,3.54703) 101 -dictGet test_01037.dict_array (30.4424,-1.39435) 101 -dictGet test_01037.dict_array (30.4441,2.8463000000000003) 101 -dictGet test_01037.dict_array (30.4517,3.28117) 101 -dictGet test_01037.dict_array (30.4658,2.6928) 101 -dictGet test_01037.dict_array (30.4734,2.66161) 101 -dictGet test_01037.dict_array (30.4799,-1.07578) 101 -dictGet test_01037.dict_array (30.4837,-1.02486) 101 -dictGet test_01037.dict_array (30.485,1.06326) 101 -dictGet test_01037.dict_array (30.495,1.12306) 101 -dictGet test_01037.dict_array (30.501,2.27264) 101 -dictGet test_01037.dict_array (30.5027,1.99382) 101 -dictGet test_01037.dict_array (30.5194,-1.03943) 101 -dictGet test_01037.dict_array (30.5239,1.04328) 101 -dictGet test_01037.dict_array (30.528,3.82041) 101 -dictGet test_01037.dict_array (30.5299,-0.715248) 101 -dictGet test_01037.dict_array (30.5331,1.19603) 101 -dictGet test_01037.dict_array (30.535800000000002,2.71485) 101 -dictGet test_01037.dict_array (30.5405,0.804694) 101 -dictGet test_01037.dict_array (30.542,1.23739) 101 -dictGet test_01037.dict_array (30.5432,4.04189) 101 -dictGet test_01037.dict_array (30.5457,-0.956121) 101 -dictGet test_01037.dict_array (30.5506,3.07443) 101 -dictGet test_01037.dict_array (30.5539,3.87084) 101 -dictGet test_01037.dict_array (30.5578,3.78837) 101 -dictGet test_01037.dict_array (30.5588,0.966135) 101 -dictGet test_01037.dict_array (30.5637,2.5605) 101 -dictGet test_01037.dict_array (30.5647,-1.27328) 101 -dictGet test_01037.dict_array (30.5656,-0.0581332) 101 -dictGet test_01037.dict_array (30.5715,0.65755) 101 -dictGet test_01037.dict_array (30.5727,3.01604) 101 -dictGet test_01037.dict_array (30.5729,-0.976857) 101 -dictGet test_01037.dict_array (30.5751,0.60204) 101 -dictGet test_01037.dict_array (30.5854,3.02473) 101 -dictGet test_01037.dict_array (30.5866,0.174099) 101 -dictGet test_01037.dict_array (30.5947,0.875193) 101 -dictGet test_01037.dict_array (30.5992,-0.403901) 101 -dictGet test_01037.dict_array (30.6002,4.18891) 101 -dictGet test_01037.dict_array (30.6025,0.217712) 101 -dictGet test_01037.dict_array (30.6054,0.927203) 101 -dictGet test_01037.dict_array (30.6075,3.79359) 101 -dictGet test_01037.dict_array (30.6159,3.82773) 101 -dictGet test_01037.dict_array (30.627,3.84039) 101 -dictGet test_01037.dict_array (30.6308,0.77517) 101 -dictGet test_01037.dict_array (30.6338,0.179565) 101 -dictGet test_01037.dict_array (30.6461,1.3293599999999999) 101 -dictGet test_01037.dict_array (30.6674,-0.424547) 101 -dictGet test_01037.dict_array (30.669,1.76539) 101 -dictGet test_01037.dict_array (30.6788,4.01239) 101 -dictGet test_01037.dict_array (30.6864,3.59158) 101 -dictGet test_01037.dict_array (30.7049,-0.875413) 101 -dictGet test_01037.dict_array (30.705,1.3307) 101 -dictGet test_01037.dict_array (30.7063,-0.473192) 101 -dictGet test_01037.dict_array (30.7075,-1.1958199999999999) 101 -dictGet test_01037.dict_array (30.7101,-0.367562) 101 -dictGet test_01037.dict_array (30.7203,2.98725) 101 -dictGet test_01037.dict_array (30.7213,2.2745699999999998) 101 -dictGet test_01037.dict_array (30.7446,-0.334144) 101 -dictGet test_01037.dict_array (30.7468,3.82967) 101 -dictGet test_01037.dict_array (30.747,-0.384779) 101 -dictGet test_01037.dict_array (30.7681,0.904198) 101 -dictGet test_01037.dict_array (30.7757,1.78743) 101 -dictGet test_01037.dict_array (30.8021,-0.479212) 101 -dictGet test_01037.dict_array (30.8079,-1.40869) 101 -dictGet test_01037.dict_array (30.8206,-0.0608489) 101 -dictGet test_01037.dict_array (30.8218,0.43909) 101 -dictGet test_01037.dict_array (30.8239,0.10014) 101 -dictGet test_01037.dict_array (30.8282,4.15409) 101 -dictGet test_01037.dict_array (30.8288,-0.709528) 101 -dictGet test_01037.dict_array (30.8326,0.156011) 101 -dictGet test_01037.dict_array (30.8328,-1.03704) 101 -dictGet test_01037.dict_array (30.839,2.15528) 101 -dictGet test_01037.dict_array (30.8452,0.219377) 101 -dictGet test_01037.dict_array (30.8463,0.0515355) 101 -dictGet test_01037.dict_array (30.8526,2.06614) 101 -dictGet test_01037.dict_array (30.8566,0.517876) 101 -dictGet test_01037.dict_array (30.8588,-1.31738) 101 -dictGet test_01037.dict_array (30.8681,0.44207) 101 -dictGet test_01037.dict_array (30.8914,1.0072) 101 -dictGet test_01037.dict_array (30.897,0.483425) 101 -dictGet test_01037.dict_array (30.905,2.8731999999999998) 101 -dictGet test_01037.dict_array (30.9051,2.21956) 101 -dictGet test_01037.dict_array (30.9115,4.00663) 101 -dictGet test_01037.dict_array (30.9167,-0.834462) 101 -dictGet test_01037.dict_array (30.9252,-1.3289900000000001) 101 -dictGet test_01037.dict_array (30.9314,1.85384) 101 -dictGet test_01037.dict_array (30.9392,2.53236) 101 -dictGet test_01037.dict_array (30.9569,2.82038) 101 -dictGet test_01037.dict_array (30.9598,-0.641011) 101 -dictGet test_01037.dict_array (30.9601,-0.254928) 101 -dictGet test_01037.dict_array (30.9623,-1.3886) 101 -dictGet test_01037.dict_array (30.9707,0.888854) 101 -dictGet test_01037.dict_array (30.9766,2.81957) 101 -dictGet test_01037.dict_array (30.9775,2.69273) 101 -dictGet test_01037.dict_array (30.9821,0.587715) 101 -dictGet test_01037.dict_array (30.9887,4.0233) 101 -dictGet test_01037.dict_array (30.9914,0.259542) 101 -dictGet test_01037.dict_array (30.9986,-1.36832) 101 -dictGet test_01037.dict_array (31.008,0.628999) 101 -dictGet test_01037.dict_array (31.0168,-1.17462) 101 -dictGet test_01037.dict_array (31.0237,3.52547) 101 -dictGet test_01037.dict_array (31.0306,3.78522) 101 -dictGet test_01037.dict_array (31.0308,-0.72453) 101 -dictGet test_01037.dict_array (31.0463,2.41997) 101 -dictGet test_01037.dict_array (31.047,0.624184) 101 -dictGet test_01037.dict_array (31.0569,0.0706393) 5994232 -dictGet test_01037.dict_array (31.0583,1.3244099999999999) 101 -dictGet test_01037.dict_array (31.063,3.23861) 101 -dictGet test_01037.dict_array (31.068,0.695575) 101 -dictGet test_01037.dict_array (31.0687,1.85675) 101 -dictGet test_01037.dict_array (31.0692,0.254793) 101 -dictGet test_01037.dict_array (31.0766,0.828128) 101 -dictGet test_01037.dict_array (31.0833,0.0612782) 5994232 -dictGet test_01037.dict_array (31.0833,2.59748) 101 -dictGet test_01037.dict_array (31.0861,-1.3778299999999999) 101 -dictGet test_01037.dict_array (31.0874,3.07258) 101 -dictGet test_01037.dict_array (31.0882,1.4882) 101 -dictGet test_01037.dict_array (31.0924,3.42242) 101 -dictGet test_01037.dict_array (31.0927,2.67448) 101 -dictGet test_01037.dict_array (31.0936,1.12292) 101 -dictGet test_01037.dict_array (31.0952,-0.336928) 101 -dictGet test_01037.dict_array (31.0978,3.48482) 101 -dictGet test_01037.dict_array (31.1107,3.7513199999999998) 101 -dictGet test_01037.dict_array (31.1156,1.19171) 101 -dictGet test_01037.dict_array (31.1176,0.223509) 5994232 -dictGet test_01037.dict_array (31.1249,0.946838) 101 -dictGet test_01037.dict_array (31.1267,1.48983) 101 -dictGet test_01037.dict_array (31.138,-0.289981) 101 -dictGet test_01037.dict_array (31.1382,3.02904) 101 -dictGet test_01037.dict_array (31.1475,2.6178) 101 -dictGet test_01037.dict_array (31.1491,1.37873) 101 -dictGet test_01037.dict_array (31.1525,3.72105) 101 -dictGet test_01037.dict_array (31.1526,-1.4129800000000001) 101 -dictGet test_01037.dict_array (31.1526,-0.186457) 101 -dictGet test_01037.dict_array (31.1539,2.78789) 101 -dictGet test_01037.dict_array (31.1548,-1.08552) 101 -dictGet test_01037.dict_array (31.1567,-0.0768925) 101 -dictGet test_01037.dict_array (31.1613,1.49617) 101 -dictGet test_01037.dict_array (31.1653,1.03777) 101 -dictGet test_01037.dict_array (31.1662,3.4214700000000002) 101 -dictGet test_01037.dict_array (31.1672,-0.0813169) 101 -dictGet test_01037.dict_array (31.177,0.440843) 101 -dictGet test_01037.dict_array (31.1788,-0.737151) 101 -dictGet test_01037.dict_array (31.1856,-0.144396) 101 -dictGet test_01037.dict_array (31.1959,3.66813) 101 -dictGet test_01037.dict_array (31.1996,-0.353983) 101 -dictGet test_01037.dict_array (31.2019,2.86802) 101 -dictGet test_01037.dict_array (31.2087,2.31245) 101 -dictGet test_01037.dict_array (31.2125,3.2713200000000002) 101 -dictGet test_01037.dict_array (31.2137,-0.108129) 101 -dictGet test_01037.dict_array (31.216,3.9156) 101 -dictGet test_01037.dict_array (31.2201,-0.202141) 101 -dictGet test_01037.dict_array (31.2285,2.09058) 101 -dictGet test_01037.dict_array (31.2502,4.01526) 101 -dictGet test_01037.dict_array (31.2585,3.11524) 101 -dictGet test_01037.dict_array (31.2645,-0.620418) 101 -dictGet test_01037.dict_array (31.2684,2.74277) 101 -dictGet test_01037.dict_array (31.2821,-1.12772) 101 -dictGet test_01037.dict_array (31.2821,2.46769) 101 -dictGet test_01037.dict_array (31.2887,3.91396) 101 -dictGet test_01037.dict_array (31.295,1.49942) 101 -dictGet test_01037.dict_array (31.2997,3.46122) 101 -dictGet test_01037.dict_array (31.3017,3.3263) 101 -dictGet test_01037.dict_array (31.3022,3.16754) 101 -dictGet test_01037.dict_array (31.3048,0.364962) 101 -dictGet test_01037.dict_array (31.305,3.1967) 101 -dictGet test_01037.dict_array (31.3061,1.84303) 101 -dictGet test_01037.dict_array (31.3082,-0.173851) 101 -dictGet test_01037.dict_array (31.3315,3.90932) 101 -dictGet test_01037.dict_array (31.3351,2.80164) 101 -dictGet test_01037.dict_array (31.3388,0.168765) 5994233 -dictGet test_01037.dict_array (31.339,0.25535) 101 -dictGet test_01037.dict_array (31.3423,1.7036799999999999) 101 -dictGet test_01037.dict_array (31.349,0.386456) 101 -dictGet test_01037.dict_array (31.3558,-1.04336) 101 -dictGet test_01037.dict_array (31.3564,0.478876) 101 -dictGet test_01037.dict_array (31.3607,-0.0860507) 5994233 -dictGet test_01037.dict_array (31.3831,3.84469) 101 -dictGet test_01037.dict_array (31.3886,-0.731137) 101 -dictGet test_01037.dict_array (31.4043,-0.348907) 101 -dictGet test_01037.dict_array (31.4081,1.47391) 101 -dictGet test_01037.dict_array (31.4176,-0.583645) 101 -dictGet test_01037.dict_array (31.4177,1.36972) 101 -dictGet test_01037.dict_array (31.4182,0.958303) 101 -dictGet test_01037.dict_array (31.4199,3.1738) 101 -dictGet test_01037.dict_array (31.4221,2.74876) 101 -dictGet test_01037.dict_array (31.4301,-0.122643) 5994233 -dictGet test_01037.dict_array (31.4344,1.00661) 101 -dictGet test_01037.dict_array (31.4375,4.20304) 101 -dictGet test_01037.dict_array (31.4377,0.289608) 101 -dictGet test_01037.dict_array (31.4379,0.54744) 101 -dictGet test_01037.dict_array (31.4459,3.94945) 101 -dictGet test_01037.dict_array (31.4559,-0.345063) 101 -dictGet test_01037.dict_array (31.464,0.726129) 101 -dictGet test_01037.dict_array (31.4662,-0.299019) 5994233 -dictGet test_01037.dict_array (31.4671,1.9605299999999999) 101 -dictGet test_01037.dict_array (31.4673,-0.403676) 101 -dictGet test_01037.dict_array (31.4712,-0.237941) 5994233 -dictGet test_01037.dict_array (31.4816,0.120264) 5994233 -dictGet test_01037.dict_array (31.4875,0.323483) 101 -dictGet test_01037.dict_array (31.490099999999998,-0.338163) 101 -dictGet test_01037.dict_array (31.4932,0.517674) 101 -dictGet test_01037.dict_array (31.5112,1.9689299999999998) 101 -dictGet test_01037.dict_array (31.5122,2.92785) 101 -dictGet test_01037.dict_array (31.5151,0.166429) 101 -dictGet test_01037.dict_array (31.5174,2.94802) 101 -dictGet test_01037.dict_array (31.5182,4.18776) 101 -dictGet test_01037.dict_array (31.5238,1.18793) 101 -dictGet test_01037.dict_array (31.5271,3.07446) 101 -dictGet test_01037.dict_array (31.5393,1.58061) 101 -dictGet test_01037.dict_array (31.5421,3.13711) 101 -dictGet test_01037.dict_array (31.5479,2.39897) 101 -dictGet test_01037.dict_array (31.5519,0.99285) 101 -dictGet test_01037.dict_array (31.5685,3.47987) 101 -dictGet test_01037.dict_array (31.5959,0.437382) 101 -dictGet test_01037.dict_array (31.6003,0.194376) 101 -dictGet test_01037.dict_array (31.6026,2.15457) 101 -dictGet test_01037.dict_array (31.606,2.45365) 101 -dictGet test_01037.dict_array (31.6062,-0.453441) 101 -dictGet test_01037.dict_array (31.6107,1.35247) 101 -dictGet test_01037.dict_array (31.6155,3.85588) 101 -dictGet test_01037.dict_array (31.6222,2.03326) 101 -dictGet test_01037.dict_array (31.6231,-0.123059) 101 -dictGet test_01037.dict_array (31.6244,1.6885599999999998) 101 -dictGet test_01037.dict_array (31.6459,0.669716) 101 -dictGet test_01037.dict_array (31.6563,-0.0644741) 101 -dictGet test_01037.dict_array (31.6618,-0.551121) 101 -dictGet test_01037.dict_array (31.6725,-0.38922) 101 -dictGet test_01037.dict_array (31.6727,4.10336) 101 -dictGet test_01037.dict_array (31.6739,4.1391) 101 -dictGet test_01037.dict_array (31.6897,2.8694699999999997) 101 -dictGet test_01037.dict_array (31.6902,3.98792) 101 -dictGet test_01037.dict_array (31.6945,2.46687) 101 -dictGet test_01037.dict_array (31.6987,-1.3796) 101 -dictGet test_01037.dict_array (31.7012,2.34845) 101 -dictGet test_01037.dict_array (31.7036,0.0228348) 101 -dictGet test_01037.dict_array (31.7046,3.68111) 101 -dictGet test_01037.dict_array (31.7055,2.92556) 101 -dictGet test_01037.dict_array (31.7102,1.04532) 101 -dictGet test_01037.dict_array (31.7149,-0.443302) 101 -dictGet test_01037.dict_array (31.7195,2.99311) 101 -dictGet test_01037.dict_array (31.7274,0.166719) 101 -dictGet test_01037.dict_array (31.7565,-0.565382) 101 -dictGet test_01037.dict_array (31.7615,0.771626) 101 -dictGet test_01037.dict_array (31.7739,1.8970099999999999) 101 -dictGet test_01037.dict_array (31.7848,1.2623199999999999) 101 -dictGet test_01037.dict_array (31.7912,-0.788599) 101 -dictGet test_01037.dict_array (31.8011,2.65853) 101 -dictGet test_01037.dict_array (31.8032,-0.0590108) 101 -dictGet test_01037.dict_array (31.8038,1.9618799999999998) 101 -dictGet test_01037.dict_array (31.8098,-1.46851) 101 -dictGet test_01037.dict_array (31.8131,3.41982) 101 -dictGet test_01037.dict_array (31.8169,3.31059) 101 -dictGet test_01037.dict_array (31.8202,-0.193692) 101 -dictGet test_01037.dict_array (31.8306,1.57586) 101 -dictGet test_01037.dict_array (31.8382,-0.787948) 101 -dictGet test_01037.dict_array (31.8433,2.49692) 101 -dictGet test_01037.dict_array (31.8436,2.41851) 101 -dictGet test_01037.dict_array (31.8563,-1.10787) 101 -dictGet test_01037.dict_array (31.8683,0.996504) 101 -dictGet test_01037.dict_array (31.8693,-0.828142) 101 -dictGet test_01037.dict_array (31.8723,1.08929) 101 -dictGet test_01037.dict_array (31.8737,0.881127) 101 -dictGet test_01037.dict_array (31.8881,-0.58441) 101 -dictGet test_01037.dict_array (31.9011,0.121349) 101 -dictGet test_01037.dict_array (31.9066,2.13045) 101 -dictGet test_01037.dict_array (31.9142,1.03368) 101 -dictGet test_01037.dict_array (31.9155,3.38363) 101 -dictGet test_01037.dict_array (31.9168,1.3166) 101 -dictGet test_01037.dict_array (31.9185,-1.11879) 101 -dictGet test_01037.dict_array (31.9186,-0.647948) 101 -dictGet test_01037.dict_array (31.9311,3.96928) 101 -dictGet test_01037.dict_array (31.9335,1.47048) 101 -dictGet test_01037.dict_array (31.9443,-1.36175) 101 -dictGet test_01037.dict_array (31.9481,2.34231) 101 -dictGet test_01037.dict_array (31.9526,1.36565) 101 -dictGet test_01037.dict_array (31.9629,2.5208399999999997) 101 -dictGet test_01037.dict_array (31.9765,0.975783) 101 -dictGet test_01037.dict_array (31.9923,3.31773) 101 -dictGet test_01037.dict_array (31.9994,0.972816) 101 -dictGet test_01037.dict_array (32.001,3.47425) 101 -dictGet test_01037.dict_array (32.0127,2.13874) 101 -dictGet test_01037.dict_array (32.0244,3.2092) 101 -dictGet test_01037.dict_array (32.029,1.18039) 101 -dictGet test_01037.dict_array (32.0315,0.566073) 101 -dictGet test_01037.dict_array (32.0354,1.0766499999999999) 101 -dictGet test_01037.dict_array (32.0399,-1.11576) 101 -dictGet test_01037.dict_array (32.053,2.16849) 101 -dictGet test_01037.dict_array (32.0542,0.042328) 101 -dictGet test_01037.dict_array (32.0576,2.47001) 101 -dictGet test_01037.dict_array (32.061,3.7498899999999997) 101 -dictGet test_01037.dict_array (32.0623,1.25134) 101 -dictGet test_01037.dict_array (32.0626,1.9611399999999999) 101 -dictGet test_01037.dict_array (32.0666,-0.0904247) 101 -dictGet test_01037.dict_array (32.0681,2.28442) 101 -dictGet test_01037.dict_array (32.0692,1.50869) 101 -dictGet test_01037.dict_array (32.0724,4.03314) 101 -dictGet test_01037.dict_array (32.0729,-0.064324) 101 -dictGet test_01037.dict_array (32.079,0.293758) 101 -dictGet test_01037.dict_array (32.0847,-1.19814) 101 -dictGet test_01037.dict_array (32.0974,-0.91927) 101 -dictGet test_01037.dict_array (32.0979,-0.736979) 101 -dictGet test_01037.dict_array (32.106,-1.33063) 101 -dictGet test_01037.dict_array (32.1189,0.246715) 101 -dictGet test_01037.dict_array (32.1207,4.00883) 101 -dictGet test_01037.dict_array (32.1396,1.12402) 101 -dictGet test_01037.dict_array (32.1413,1.5668) 101 -dictGet test_01037.dict_array (32.143,1.35559) 101 -dictGet test_01037.dict_array (32.1538,1.32881) 101 -dictGet test_01037.dict_array (32.1549,4.06552) 101 -dictGet test_01037.dict_array (32.1555,-0.79275) 101 -dictGet test_01037.dict_array (32.163,1.17733) 101 -dictGet test_01037.dict_array (32.1634,2.94273) 101 -dictGet test_01037.dict_array (32.1644,1.85666) 101 -dictGet test_01037.dict_array (32.1745,0.435458) 101 -dictGet test_01037.dict_array (32.1765,1.65149) 101 -dictGet test_01037.dict_array (32.1893,2.08924) 101 -dictGet test_01037.dict_array (32.2024,0.222191) 101 -dictGet test_01037.dict_array (32.2107,1.34379) 101 -dictGet test_01037.dict_array (32.2109,3.9018699999999997) 101 -dictGet test_01037.dict_array (32.2123,1.85233) 101 -dictGet test_01037.dict_array (32.2144,3.72534) 101 -dictGet test_01037.dict_array (32.2218,2.5386699999999998) 101 -dictGet test_01037.dict_array (32.2279,2.84267) 101 -dictGet test_01037.dict_array (32.2345,3.33295) 101 -dictGet test_01037.dict_array (32.2435,3.85283) 101 -dictGet test_01037.dict_array (32.2527,-0.480608) 101 -dictGet test_01037.dict_array (32.2566,-0.837882) 101 -dictGet test_01037.dict_array (32.2627,2.57708) 101 -dictGet test_01037.dict_array (32.2733,0.244931) 101 -dictGet test_01037.dict_array (32.2761,4.05808) 101 -dictGet test_01037.dict_array (32.2764,3.78472) 101 -dictGet test_01037.dict_array (32.2814,-1.26011) 101 -dictGet test_01037.dict_array (32.2861,3.02427) 101 -dictGet test_01037.dict_array (32.2924,0.928609) 101 -dictGet test_01037.dict_array (32.2963,-0.78543) 101 -dictGet test_01037.dict_array (32.3039,3.21175) 101 -dictGet test_01037.dict_array (32.3107,0.698287) 101 -dictGet test_01037.dict_array (32.3138,0.0595677) 101 -dictGet test_01037.dict_array (32.3339,0.707056) 101 -dictGet test_01037.dict_array (32.3351,0.415474) 101 -dictGet test_01037.dict_array (32.342,-0.681023) 101 -dictGet test_01037.dict_array (32.3463,1.83196) 101 -dictGet test_01037.dict_array (32.3494,2.43799) 101 -dictGet test_01037.dict_array (32.3524,3.47049) 101 -dictGet test_01037.dict_array (32.3531,2.33115) 101 -dictGet test_01037.dict_array (32.3602,0.116106) 101 -dictGet test_01037.dict_array (32.3612,1.1598) 101 -dictGet test_01037.dict_array (32.3689,3.34847) 101 -dictGet test_01037.dict_array (32.3695,0.734055) 101 -dictGet test_01037.dict_array (32.3825,3.85017) 101 -dictGet test_01037.dict_array (32.3835,-1.25491) 101 -dictGet test_01037.dict_array (32.4018,-0.728568) 101 -dictGet test_01037.dict_array (32.4044,2.96727) 101 -dictGet test_01037.dict_array (32.4101,2.9988) 101 -dictGet test_01037.dict_array (32.417,-1.12908) 101 -dictGet test_01037.dict_array (32.4172,4.1952) 101 -dictGet test_01037.dict_array (32.4239,2.49512) 101 -dictGet test_01037.dict_array (32.4258,4.05137) 101 -dictGet test_01037.dict_array (32.4264,-0.427357) 101 -dictGet test_01037.dict_array (32.4274,3.59377) 101 -dictGet test_01037.dict_array (32.4286,-1.24757) 101 -dictGet test_01037.dict_array (32.4294,3.0665) 101 -dictGet test_01037.dict_array (32.4333,-0.353347) 101 -dictGet test_01037.dict_array (32.4391,3.64421) 101 -dictGet test_01037.dict_array (32.4401,3.70635) 101 -dictGet test_01037.dict_array (32.45,1.68918) 101 -dictGet test_01037.dict_array (32.4507,-0.133471) 101 -dictGet test_01037.dict_array (32.4592,0.976458) 101 -dictGet test_01037.dict_array (32.4595,1.89135) 101 -dictGet test_01037.dict_array (32.4604,0.280248) 101 -dictGet test_01037.dict_array (32.4835,0.472731) 101 -dictGet test_01037.dict_array (32.4855,2.01938) 101 -dictGet test_01037.dict_array (32.4872,2.01697) 101 -dictGet test_01037.dict_array (32.4911,0.613106) 101 -dictGet test_01037.dict_array (32.4918,2.17834) 101 -dictGet test_01037.dict_array (32.4947,2.34595) 101 -dictGet test_01037.dict_array (32.5035,2.92234) 101 -dictGet test_01037.dict_array (32.5132,-0.331206) 101 -dictGet test_01037.dict_array (32.5156,-0.412604) 7652581 -dictGet test_01037.dict_array (32.5158,2.9067499999999997) 101 -dictGet test_01037.dict_array (32.5249,2.44519) 101 -dictGet test_01037.dict_array (32.5293,-0.790952) 101 -dictGet test_01037.dict_array (32.5319,3.96854) 101 -dictGet test_01037.dict_array (32.5518,3.6093) 101 -dictGet test_01037.dict_array (32.5541,3.5225400000000002) 101 -dictGet test_01037.dict_array (32.5569,0.816123) 101 -dictGet test_01037.dict_array (32.5646,1.9775) 101 -dictGet test_01037.dict_array (32.5733,3.81271) 101 -dictGet test_01037.dict_array (32.5767,0.948327) 101 -dictGet test_01037.dict_array (32.5971,1.76179) 101 -dictGet test_01037.dict_array (32.6035,-0.716157) 101 -dictGet test_01037.dict_array (32.6087,4.21614) 101 -dictGet test_01037.dict_array (32.6171,0.024481) 101 -dictGet test_01037.dict_array (32.6189,-0.775391) 101 -dictGet test_01037.dict_array (32.6198,2.92081) 101 -dictGet test_01037.dict_array (32.621,-0.970784) 101 -dictGet test_01037.dict_array (32.6266,0.650009) 101 -dictGet test_01037.dict_array (32.6315,2.15144) 101 -dictGet test_01037.dict_array (32.6385,-0.436803) 101 -dictGet test_01037.dict_array (32.6449,-0.191292) 101 -dictGet test_01037.dict_array (32.6535,2.10385) 101 -dictGet test_01037.dict_array (32.6592,3.49973) 101 -dictGet test_01037.dict_array (32.6598,2.5980600000000003) 101 -dictGet test_01037.dict_array (32.6612,2.95681) 101 -dictGet test_01037.dict_array (32.6636,-0.57235) 101 -dictGet test_01037.dict_array (32.669,-0.382702) 101 -dictGet test_01037.dict_array (32.6752,1.30748) 101 -dictGet test_01037.dict_array (32.6811,2.9559800000000003) 101 -dictGet test_01037.dict_array (32.6821,0.57336) 101 -dictGet test_01037.dict_array (32.6828,3.91304) 101 -dictGet test_01037.dict_array (32.6979,3.96868) 101 -dictGet test_01037.dict_array (32.6983,3.15784) 101 -dictGet test_01037.dict_array (32.7122,0.794293) 101 -dictGet test_01037.dict_array (32.7131,-0.847256) 101 -dictGet test_01037.dict_array (32.7219,0.883461) 101 -dictGet test_01037.dict_array (32.7228,1.78808) 101 -dictGet test_01037.dict_array (32.7273,-0.206908) 101 -dictGet test_01037.dict_array (32.7292,0.259331) 101 -dictGet test_01037.dict_array (32.7304,-1.38317) 101 -dictGet test_01037.dict_array (32.7353,1.01601) 101 -dictGet test_01037.dict_array (32.7354,4.17574) 101 -dictGet test_01037.dict_array (32.7357,-0.190194) 101 -dictGet test_01037.dict_array (32.7465,-1.37598) 101 -dictGet test_01037.dict_array (32.7494,-0.275675) 101 -dictGet test_01037.dict_array (32.7514,0.128951) 101 -dictGet test_01037.dict_array (32.753,3.44207) 101 -dictGet test_01037.dict_array (32.7686,2.11713) 101 -dictGet test_01037.dict_array (32.7694,1.47159) 101 -dictGet test_01037.dict_array (32.7768,0.0401042) 101 -dictGet test_01037.dict_array (32.781,-1.34283) 101 -dictGet test_01037.dict_array (32.7814,1.73876) 101 -dictGet test_01037.dict_array (32.7856,-1.06363) 101 -dictGet test_01037.dict_array (32.792699999999996,-1.1255600000000001) 101 -dictGet test_01037.dict_array (32.7941,-0.645447) 101 -dictGet test_01037.dict_array (32.7946,1.48889) 101 -dictGet test_01037.dict_array (32.797,0.791753) 101 -dictGet test_01037.dict_array (32.7982,-0.537798) 101 -dictGet test_01037.dict_array (32.8091,2.3611) 101 -dictGet test_01037.dict_array (32.81,1.7130800000000002) 101 -dictGet test_01037.dict_array (32.8174,-0.288322) 101 -dictGet test_01037.dict_array (32.823,1.6546699999999999) 101 -dictGet test_01037.dict_array (32.8233,1.62108) 101 -dictGet test_01037.dict_array (32.8428,-0.400045) 101 -dictGet test_01037.dict_array (32.8479,2.13598) 101 -dictGet test_01037.dict_array (32.8524,0.199902) 101 -dictGet test_01037.dict_array (32.8543,3.23553) 101 -dictGet test_01037.dict_array (32.8562,1.31371) 101 -dictGet test_01037.dict_array (32.87,1.44256) 101 -dictGet test_01037.dict_array (32.8789,2.38192) 101 -dictGet test_01037.dict_array (32.8812,2.20734) 5999168 -dictGet test_01037.dict_array (32.8815,-0.54427) 101 -dictGet test_01037.dict_array (32.8853,2.4859) 5999168 -dictGet test_01037.dict_array (32.8909,0.513964) 101 -dictGet test_01037.dict_array (32.9035,2.38999) 101 -dictGet test_01037.dict_array (32.9097,2.48131) 5999168 -dictGet test_01037.dict_array (32.928,-0.943269) 101 -dictGet test_01037.dict_array (32.9322,1.13165) 101 -dictGet test_01037.dict_array (32.9348,1.22606) 101 -dictGet test_01037.dict_array (32.9417,3.77998) 101 -dictGet test_01037.dict_array (32.9428,3.11936) 101 -dictGet test_01037.dict_array (32.9482,1.18092) 101 -dictGet test_01037.dict_array (32.9506,0.0609364) 101 -dictGet test_01037.dict_array (32.953,-0.828308) 101 -dictGet test_01037.dict_array (32.9593,3.5209099999999998) 101 -dictGet test_01037.dict_array (32.9617,2.07711) 5999168 -dictGet test_01037.dict_array (32.966,0.693749) 101 -dictGet test_01037.dict_array (32.9668,-0.716432) 101 -dictGet test_01037.dict_array (32.9702,1.98555) 101 -dictGet test_01037.dict_array (32.9782,1.73819) 101 -dictGet test_01037.dict_array (32.9805,3.71151) 101 -dictGet test_01037.dict_array (32.9821,2.97225) 101 -dictGet test_01037.dict_array (32.995,-0.830301) 101 -dictGet test_01037.dict_array (33.0234,0.770848) 101 -dictGet test_01037.dict_array (33.0312,-0.340964) 101 -dictGet test_01037.dict_array (33.0366,-0.756795) 101 -dictGet test_01037.dict_array (33.0438,0.812871) 101 -dictGet test_01037.dict_array (33.0455,1.84843) 101 -dictGet test_01037.dict_array (33.0498,0.0913292) 101 -dictGet test_01037.dict_array (33.0506,1.53739) 101 -dictGet test_01037.dict_array (33.0554,2.4265) 101 -dictGet test_01037.dict_array (33.0741,3.61332) 101 -dictGet test_01037.dict_array (33.0765,-0.179985) 101 -dictGet test_01037.dict_array (33.087,1.46465) 101 -dictGet test_01037.dict_array (33.0906,-0.620383) 101 -dictGet test_01037.dict_array (33.1047,-1.28027) 101 -dictGet test_01037.dict_array (33.1072,1.96303) 101 -dictGet test_01037.dict_array (33.1081,-0.897874) 101 -dictGet test_01037.dict_array (33.1122,1.8950200000000001) 101 -dictGet test_01037.dict_array (33.1237,2.63993) 101 -dictGet test_01037.dict_array (33.1238,0.753963) 101 -dictGet test_01037.dict_array (33.1257,0.495668) 101 -dictGet test_01037.dict_array (33.1258,1.78341) 101 -dictGet test_01037.dict_array (33.127,2.59646) 101 -dictGet test_01037.dict_array (33.1324,-1.23742) 101 -dictGet test_01037.dict_array (33.1359,3.83491) 101 -dictGet test_01037.dict_array (33.1628,-0.379588) 101 -dictGet test_01037.dict_array (33.1679,1.25601) 101 -dictGet test_01037.dict_array (33.1688,-1.35553) 101 -dictGet test_01037.dict_array (33.181,2.10943) 101 -dictGet test_01037.dict_array (33.1871,2.81171) 101 -dictGet test_01037.dict_array (33.1877,0.771297) 101 -dictGet test_01037.dict_array (33.1883,-0.204797) 101 -dictGet test_01037.dict_array (33.1886,3.27998) 101 -dictGet test_01037.dict_array (33.1955,0.708907) 101 -dictGet test_01037.dict_array (33.2044,-0.769275) 101 -dictGet test_01037.dict_array (33.2182,3.36103) 101 -dictGet test_01037.dict_array (33.2192,3.43586) 101 -dictGet test_01037.dict_array (33.2322,-0.916753) 101 -dictGet test_01037.dict_array (33.2359,-0.81321) 101 -dictGet test_01037.dict_array (33.238,0.635072) 101 -dictGet test_01037.dict_array (33.2398,3.02588) 101 -dictGet test_01037.dict_array (33.2469,2.35698) 101 -dictGet test_01037.dict_array (33.247,2.3327) 101 -dictGet test_01037.dict_array (33.2579,2.8027100000000003) 101 -dictGet test_01037.dict_array (33.2607,0.321082) 101 -dictGet test_01037.dict_array (33.2653,0.243336) 101 -dictGet test_01037.dict_array (33.2758,0.831836) 101 -dictGet test_01037.dict_array (33.2771,0.886536) 101 -dictGet test_01037.dict_array (33.2914,1.16026) 101 -dictGet test_01037.dict_array (33.2914,1.38882) 101 -dictGet test_01037.dict_array (33.2982,-1.16604) 101 -dictGet test_01037.dict_array (33.2985,0.842556) 101 -dictGet test_01037.dict_array (33.3005,2.8338900000000002) 101 -dictGet test_01037.dict_array (33.305,0.0969475) 101 -dictGet test_01037.dict_array (33.3072,3.82163) 101 -dictGet test_01037.dict_array (33.312,3.41475) 101 -dictGet test_01037.dict_array (33.3129,2.46048) 101 -dictGet test_01037.dict_array (33.3134,3.46863) 101 -dictGet test_01037.dict_array (33.3203,2.33139) 101 -dictGet test_01037.dict_array (33.324,0.433701) 101 -dictGet test_01037.dict_array (33.3338,2.44705) 101 -dictGet test_01037.dict_array (33.337,4.06475) 101 -dictGet test_01037.dict_array (33.3469,1.08172) 101 -dictGet test_01037.dict_array (33.3538,0.717896) 101 -dictGet test_01037.dict_array (33.3618,1.37899) 101 -dictGet test_01037.dict_array (33.3698,0.547744) 101 -dictGet test_01037.dict_array (33.3705,0.957619) 101 -dictGet test_01037.dict_array (33.3821,3.07258) 101 -dictGet test_01037.dict_array (33.3881,3.0626) 101 -dictGet test_01037.dict_array (33.393,-0.816186) 101 -dictGet test_01037.dict_array (33.3945,0.869508) 101 -dictGet test_01037.dict_array (33.4001,1.24186) 101 -dictGet test_01037.dict_array (33.4008,2.34911) 101 -dictGet test_01037.dict_array (33.4166,-1.2808899999999999) 101 -dictGet test_01037.dict_array (33.4167,3.0655) 101 -dictGet test_01037.dict_array (33.4204,2.81887) 101 -dictGet test_01037.dict_array (33.4211,1.71128) 101 -dictGet test_01037.dict_array (33.4237,2.91761) 101 -dictGet test_01037.dict_array (33.4266,1.5955599999999999) 101 -dictGet test_01037.dict_array (33.4353,-0.391392) 101 -dictGet test_01037.dict_array (33.4362,-0.134658) 101 -dictGet test_01037.dict_array (33.4386,0.15396) 101 -dictGet test_01037.dict_array (33.4421,-0.50712) 101 -dictGet test_01037.dict_array (33.452,0.915829) 101 -dictGet test_01037.dict_array (33.463,-0.0882717) 101 -dictGet test_01037.dict_array (33.464,-1.00949) 101 -dictGet test_01037.dict_array (33.4692,0.954092) 101 -dictGet test_01037.dict_array (33.4716,1.9538799999999998) 101 -dictGet test_01037.dict_array (33.4756,1.85836) 101 -dictGet test_01037.dict_array (33.4859,4.0751) 101 -dictGet test_01037.dict_array (33.4899,3.54193) 101 -dictGet test_01037.dict_array (33.4935,3.49794) 101 -dictGet test_01037.dict_array (33.494,-0.983356) 101 -dictGet test_01037.dict_array (33.4955,-1.28128) 101 -dictGet test_01037.dict_array (33.4965,-0.278687) 101 -dictGet test_01037.dict_array (33.4991,0.647491) 101 -dictGet test_01037.dict_array (33.5076,2.2272) 101 -dictGet test_01037.dict_array (33.5079,-0.498199) 101 -dictGet test_01037.dict_array (33.5157,0.535034) 101 -dictGet test_01037.dict_array (33.5171,2.49677) 101 -dictGet test_01037.dict_array (33.5255,2.4447200000000002) 101 -dictGet test_01037.dict_array (33.526,4.01194) 101 -dictGet test_01037.dict_array (33.5288,0.789434) 101 -dictGet test_01037.dict_array (33.5356,-1.17671) 101 -dictGet test_01037.dict_array (33.5402,1.49152) 101 -dictGet test_01037.dict_array (33.5418,3.45757) 101 -dictGet test_01037.dict_array (33.5428,1.90712) 101 -dictGet test_01037.dict_array (33.5556,-0.55741) 101 -dictGet test_01037.dict_array (33.5564,0.876858) 101 -dictGet test_01037.dict_array (33.5567,-0.10208) 101 -dictGet test_01037.dict_array (33.5645,-0.124824) 101 -dictGet test_01037.dict_array (33.5663,3.4872) 101 -dictGet test_01037.dict_array (33.5716,-0.0107611) 101 -dictGet test_01037.dict_array (33.578,3.55714) 101 -dictGet test_01037.dict_array (33.5826,-0.49076) 101 -dictGet test_01037.dict_array (33.5909,0.773737) 101 -dictGet test_01037.dict_array (33.5958,2.9619999999999997) 5994231 -dictGet test_01037.dict_array (33.6193,-0.919755) 101 -dictGet test_01037.dict_array (33.6313,0.652132) 101 -dictGet test_01037.dict_array (33.632,0.823351) 101 -dictGet test_01037.dict_array (33.66,2.18998) 101 -dictGet test_01037.dict_array (33.6621,0.535395) 101 -dictGet test_01037.dict_array (33.6726,3.19367) 101 -dictGet test_01037.dict_array (33.6912,1.74522) 101 -dictGet test_01037.dict_array (33.705,0.706397) 101 -dictGet test_01037.dict_array (33.7076,0.7622) 101 -dictGet test_01037.dict_array (33.7112,1.70187) 101 -dictGet test_01037.dict_array (33.7246,-1.14837) 101 -dictGet test_01037.dict_array (33.7326,2.62413) 5994231 -dictGet test_01037.dict_array (33.7332,2.82137) 5994231 -dictGet test_01037.dict_array (33.7434,0.394672) 101 -dictGet test_01037.dict_array (33.7443,1.54557) 101 -dictGet test_01037.dict_array (33.7506,1.57317) 101 -dictGet test_01037.dict_array (33.7526,1.8578999999999999) 101 -dictGet test_01037.dict_array (33.766,4.15013) 101 -dictGet test_01037.dict_array (33.7834,2.41789) 101 -dictGet test_01037.dict_array (33.7864,0.230935) 101 -dictGet test_01037.dict_array (33.7965,3.05709) 101 -dictGet test_01037.dict_array (33.7998,3.32881) 101 -dictGet test_01037.dict_array (33.8003,2.97338) 5994231 -dictGet test_01037.dict_array (33.8007,-1.08962) 101 -dictGet test_01037.dict_array (33.8022,-0.139488) 101 -dictGet test_01037.dict_array (33.8065,2.70857) 5994231 -dictGet test_01037.dict_array (33.8169,-0.607788) 101 -dictGet test_01037.dict_array (33.8203,0.108512) 101 -dictGet test_01037.dict_array (33.8231,-1.03449) 101 -dictGet test_01037.dict_array (33.8312,3.49458) 101 -dictGet test_01037.dict_array (33.8342,0.297518) 101 -dictGet test_01037.dict_array (33.8352,0.165872) 101 -dictGet test_01037.dict_array (33.8354,1.87277) 101 -dictGet test_01037.dict_array (33.8371,1.60103) 101 -dictGet test_01037.dict_array (33.8387,1.9968) 101 -dictGet test_01037.dict_array (33.8403,3.5805) 101 -dictGet test_01037.dict_array (33.8414,-0.703067) 101 -dictGet test_01037.dict_array (33.844,-0.179472) 101 -dictGet test_01037.dict_array (33.8468,3.40137) 101 -dictGet test_01037.dict_array (33.8509,4.15334) 101 -dictGet test_01037.dict_array (33.8539,2.38339) 101 -dictGet test_01037.dict_array (33.858,-1.3122500000000001) 101 -dictGet test_01037.dict_array (33.859,3.72626) 101 -dictGet test_01037.dict_array (33.8616,2.24433) 101 -dictGet test_01037.dict_array (33.8621,3.01035) 101 -dictGet test_01037.dict_array (33.8623,1.17559) 101 -dictGet test_01037.dict_array (33.8682,2.706) 5994231 -dictGet test_01037.dict_array (33.8684,0.189231) 101 -dictGet test_01037.dict_array (33.872,1.93574) 101 -dictGet test_01037.dict_array (33.8844,3.80404) 101 -dictGet test_01037.dict_array (33.8888,0.594884) 101 -dictGet test_01037.dict_array (33.8946,2.74161) 101 -dictGet test_01037.dict_array (33.9023,0.6239) 101 -dictGet test_01037.dict_array (33.9057,0.873222) 101 -dictGet test_01037.dict_array (33.9157,-1.26607) 101 -dictGet test_01037.dict_array (33.92,2.06848) 101 -dictGet test_01037.dict_array (33.9298,-0.00526229) 101 -dictGet test_01037.dict_array (33.932,3.07063) 101 -dictGet test_01037.dict_array (33.9322,0.629385) 101 -dictGet test_01037.dict_array (33.9367,-1.41955) 101 -dictGet test_01037.dict_array (33.937,1.42532) 101 -dictGet test_01037.dict_array (33.9375,1.1467100000000001) 101 -dictGet test_01037.dict_array (33.9434,-1.05739) 101 -dictGet test_01037.dict_array (33.9477,3.34809) 101 -dictGet test_01037.dict_array (33.95,2.21715) 101 -dictGet test_01037.dict_array (33.955799999999996,0.305176) 101 -dictGet test_01037.dict_array (33.9686,-0.28273) 101 -dictGet test_01037.dict_array (33.9703,4.1255) 101 -dictGet test_01037.dict_array (33.9707,3.08199) 101 -dictGet test_01037.dict_array (33.9754,1.06203) 101 -dictGet test_01037.dict_array (33.9757,3.72468) 101 -dictGet test_01037.dict_array (33.9775,-0.0440599) 101 -dictGet test_01037.dict_array (33.9777,-0.251484) 101 -dictGet test_01037.dict_array (33.9789,-0.339374) 101 -dictGet test_01037.dict_array (33.9849,2.54515) 5994231 -dictGet test_01037.dict_array (33.9885,-0.318557) 101 -dictGet test_01037.dict_array (33.9977,1.07175) 101 -dictGet test_01037.dict_array (33.9984,-0.700517) 101 -dictGet test_01037.dict_array (34.0149,3.53338) 101 -dictGet test_01037.dict_array (34.0173,3.39155) 101 -dictGet test_01037.dict_array (34.0317,3.9579) 101 -dictGet test_01037.dict_array (34.0369,3.83612) 101 -dictGet test_01037.dict_array (34.043,-0.0887221) 101 -dictGet test_01037.dict_array (34.0487,1.14252) 101 -dictGet test_01037.dict_array (34.052,1.74832) 101 -dictGet test_01037.dict_array (34.0711,-0.898071) 101 -dictGet test_01037.dict_array (34.0747,1.55057) 101 -dictGet test_01037.dict_array (34.0803,3.16763) 101 -dictGet test_01037.dict_array (34.0872,3.75555) 101 -dictGet test_01037.dict_array (34.0965,1.62038) 101 -dictGet test_01037.dict_array (34.0977,-0.412691) 101 -dictGet test_01037.dict_array (34.0986,0.0294206) 101 -dictGet test_01037.dict_array (34.1072,3.15823) 101 -dictGet test_01037.dict_array (34.1092,3.09599) 101 -dictGet test_01037.dict_array (34.1206,1.04637) 5940222 -dictGet test_01037.dict_array (34.1209,3.13826) 101 -dictGet test_01037.dict_array (34.1265,3.95881) 101 -dictGet test_01037.dict_array (34.1286,-0.539319) 101 -dictGet test_01037.dict_array (34.1358,3.67451) 101 -dictGet test_01037.dict_array (34.1428,0.136115) 101 -dictGet test_01037.dict_array (34.157,1.73522) 101 -dictGet test_01037.dict_array (34.1581,1.48001) 101 -dictGet test_01037.dict_array (34.1682,3.42373) 101 -dictGet test_01037.dict_array (34.1683,-1.26511) 101 -dictGet test_01037.dict_array (34.1684,4.20007) 101 -dictGet test_01037.dict_array (34.1854,3.32089) 101 -dictGet test_01037.dict_array (34.2022,0.749536) 101 -dictGet test_01037.dict_array (34.2044,3.04865) 101 -dictGet test_01037.dict_array (34.22,-0.500055) 101 -dictGet test_01037.dict_array (34.2249,0.743775) 101 -dictGet test_01037.dict_array (34.2254,1.34702) 101 -dictGet test_01037.dict_array (34.2355,-0.898843) 101 -dictGet test_01037.dict_array (34.2394,2.0203699999999998) 101 -dictGet test_01037.dict_array (34.2466,1.83785) 101 -dictGet test_01037.dict_array (34.247,4.09563) 101 -dictGet test_01037.dict_array (34.2508,2.61312) 101 -dictGet test_01037.dict_array (34.2517,1.69642) 101 -dictGet test_01037.dict_array (34.2564,4.13033) 101 -dictGet test_01037.dict_array (34.2574,4.18928) 101 -dictGet test_01037.dict_array (34.2614,-0.478719) 101 -dictGet test_01037.dict_array (34.2625,2.38088) 101 -dictGet test_01037.dict_array (34.2666,3.1503) 101 -dictGet test_01037.dict_array (34.271,4.02223) 101 -dictGet test_01037.dict_array (34.2727,0.514755) 101 -dictGet test_01037.dict_array (34.278,1.98929) 101 -dictGet test_01037.dict_array (34.2798,-0.199208) 101 -dictGet test_01037.dict_array (34.2804,2.05184) 101 -dictGet test_01037.dict_array (34.2945,-1.11051) 101 -dictGet test_01037.dict_array (34.3168,-0.0829721) 101 -dictGet test_01037.dict_array (34.3345,3.4358) 101 -dictGet test_01037.dict_array (34.3377,1.13527) 5940222 -dictGet test_01037.dict_array (34.3383,1.27891) 5940222 -dictGet test_01037.dict_array (34.3391,1.47945) 5940222 -dictGet test_01037.dict_array (34.3441,0.627014) 101 -dictGet test_01037.dict_array (34.347,2.4853) 101 -dictGet test_01037.dict_array (34.3514,2.16247) 101 -dictGet test_01037.dict_array (34.3627,2.64533) 101 -dictGet test_01037.dict_array (34.3682,-0.227501) 101 -dictGet test_01037.dict_array (34.3756,4.21248) 101 -dictGet test_01037.dict_array (34.379,3.96604) 101 -dictGet test_01037.dict_array (34.3827,1.7518) 101 -dictGet test_01037.dict_array (34.3912,2.8834) 101 -dictGet test_01037.dict_array (34.3919,0.668829) 101 -dictGet test_01037.dict_array (34.3949,2.00338) 101 -dictGet test_01037.dict_array (34.3987,0.557268) 101 -dictGet test_01037.dict_array (34.4111,0.768558) 101 -dictGet test_01037.dict_array (34.4119,2.8742) 101 -dictGet test_01037.dict_array (34.416,3.50841) 101 -dictGet test_01037.dict_array (34.4212,1.24916) 5940222 -dictGet test_01037.dict_array (34.4251,0.457029) 101 -dictGet test_01037.dict_array (34.4274,-0.902559) 101 -dictGet test_01037.dict_array (34.4325,4.03159) 101 -dictGet test_01037.dict_array (34.438,1.63994) 101 -dictGet test_01037.dict_array (34.4403,-0.177594) 101 -dictGet test_01037.dict_array (34.4421,0.726712) 101 -dictGet test_01037.dict_array (34.4517,2.98611) 101 -dictGet test_01037.dict_array (34.4658,-1.312) 101 -dictGet test_01037.dict_array (34.4732,-0.0681338) 101 -dictGet test_01037.dict_array (34.4752,2.81646) 101 -dictGet test_01037.dict_array (34.4914,2.3858) 101 -dictGet test_01037.dict_array (34.4923,0.855231) 101 -dictGet test_01037.dict_array (34.5235,1.78468) 101 -dictGet test_01037.dict_array (34.5305,4.10608) 101 -dictGet test_01037.dict_array (34.5389,0.621937) 101 -dictGet test_01037.dict_array (34.5406,3.17145) 101 -dictGet test_01037.dict_array (34.5434,-0.56306) 101 -dictGet test_01037.dict_array (34.5449,3.13311) 101 -dictGet test_01037.dict_array (34.5491,2.31572) 101 -dictGet test_01037.dict_array (34.5539,2.94028) 101 -dictGet test_01037.dict_array (34.5546,-0.208825) 101 -dictGet test_01037.dict_array (34.5549,3.78486) 101 -dictGet test_01037.dict_array (34.5676,0.307148) 101 -dictGet test_01037.dict_array (34.5743,1.5217399999999999) 101 -dictGet test_01037.dict_array (34.5775,3.48046) 101 -dictGet test_01037.dict_array (34.5815,2.5243700000000002) 101 -dictGet test_01037.dict_array (34.5841,4.21191) 101 -dictGet test_01037.dict_array (34.5887,2.65083) 101 -dictGet test_01037.dict_array (34.5937,3.2143) 101 -dictGet test_01037.dict_array (34.6013,-1.0612) 101 -dictGet test_01037.dict_array (34.6089,1.36066) 101 -dictGet test_01037.dict_array (34.6103,3.40227) 101 -dictGet test_01037.dict_array (34.6128,1.92276) 101 -dictGet test_01037.dict_array (34.6175,2.43627) 101 -dictGet test_01037.dict_array (34.6209,3.43776) 101 -dictGet test_01037.dict_array (34.6234,2.60237) 101 -dictGet test_01037.dict_array (34.6275,3.52479) 101 -dictGet test_01037.dict_array (34.635,0.568558) 101 -dictGet test_01037.dict_array (34.6373,2.37692) 101 -dictGet test_01037.dict_array (34.6375,3.52234) 101 -dictGet test_01037.dict_array (34.6426,2.12397) 101 -dictGet test_01037.dict_array (34.6513,2.80915) 101 -dictGet test_01037.dict_array (34.6632,2.30039) 101 -dictGet test_01037.dict_array (34.6691,1.86582) 101 -dictGet test_01037.dict_array (34.6739,0.15342) 101 -dictGet test_01037.dict_array (34.6825,0.0499679) 101 -dictGet test_01037.dict_array (34.6893,0.454326) 101 -dictGet test_01037.dict_array (34.6957,-0.358598) 101 -dictGet test_01037.dict_array (34.6986,0.562679) 101 -dictGet test_01037.dict_array (34.712,1.12114) 101 -dictGet test_01037.dict_array (34.7126,-0.0057301) 101 -dictGet test_01037.dict_array (34.7137,0.0248501) 101 -dictGet test_01037.dict_array (34.7162,1.15623) 101 -dictGet test_01037.dict_array (34.7258,3.95142) 101 -dictGet test_01037.dict_array (34.7347,3.5232099999999997) 101 -dictGet test_01037.dict_array (34.7363,2.23374) 101 -dictGet test_01037.dict_array (34.7375,0.397841) 101 -dictGet test_01037.dict_array (34.7423,3.09198) 101 -dictGet test_01037.dict_array (34.7452,3.09029) 101 -dictGet test_01037.dict_array (34.7539,-1.06943) 101 -dictGet test_01037.dict_array (34.7733,-0.00912717) 101 -dictGet test_01037.dict_array (34.774,2.71088) 101 -dictGet test_01037.dict_array (34.7771,1.46009) 101 -dictGet test_01037.dict_array (34.7782,-1.28308) 101 -dictGet test_01037.dict_array (34.7924,3.63564) 101 -dictGet test_01037.dict_array (34.7939,-0.416676) 101 -dictGet test_01037.dict_array (34.7964,-0.401773) 101 -dictGet test_01037.dict_array (34.7974,0.0286873) 101 -dictGet test_01037.dict_array (34.7975,3.05965) 101 -dictGet test_01037.dict_array (34.8037,3.07263) 101 -dictGet test_01037.dict_array (34.8254,-0.390284) 101 -dictGet test_01037.dict_array (34.828,1.91869) 101 -dictGet test_01037.dict_array (34.8289,3.71058) 101 -dictGet test_01037.dict_array (34.8403,2.14606) 101 -dictGet test_01037.dict_array (34.8437,2.20617) 101 -dictGet test_01037.dict_array (34.8469,2.38435) 101 -dictGet test_01037.dict_array (34.86,1.45705) 101 -dictGet test_01037.dict_array (34.8612,0.914248) 101 -dictGet test_01037.dict_array (34.8663,3.4215400000000002) 101 -dictGet test_01037.dict_array (34.8724,-0.375144) 101 -dictGet test_01037.dict_array (34.8795,3.29317) 101 -dictGet test_01037.dict_array (34.8823,1.21988) 101 -dictGet test_01037.dict_array (34.8834,1.07657) 101 -dictGet test_01037.dict_array (34.8837,0.157648) 101 -dictGet test_01037.dict_array (34.8871,-0.9755) 101 -dictGet test_01037.dict_array (34.8871,1.8943699999999999) 101 -dictGet test_01037.dict_array (34.889,3.36756) 101 -dictGet test_01037.dict_array (34.8907,1.24874) 101 -dictGet test_01037.dict_array (34.8965,3.13508) 101 -dictGet test_01037.dict_array (34.9042,2.62092) 101 -dictGet test_01037.dict_array (34.9055,-0.0448967) 101 -dictGet test_01037.dict_array (34.9122,0.110576) 101 -dictGet test_01037.dict_array (34.9228,3.60183) 101 -dictGet test_01037.dict_array (34.9237,1.21715) 101 -dictGet test_01037.dict_array (34.9296,1.70459) 101 -dictGet test_01037.dict_array (34.941,-1.14663) 101 -dictGet test_01037.dict_array (34.9448,1.18923) 101 -dictGet test_01037.dict_array (34.9462,3.81678) 101 -dictGet test_01037.dict_array (34.9466,0.593463) 101 -dictGet test_01037.dict_array (34.9485,0.150307) 101 -dictGet test_01037.dict_array (34.9542,0.487238) 101 -dictGet test_01037.dict_array (34.9559,2.03473) 101 -dictGet test_01037.dict_array (34.9671,-0.960225) 101 -dictGet test_01037.dict_array (34.9711,2.63444) 101 -dictGet test_01037.dict_array (34.9892,0.354775) 101 -dictGet test_01037.dict_array (34.9907,1.40724) 101 -dictGet test_01037.dict_array (34.9916,-0.00173097) 101 -dictGet test_01037.dict_array (34.9919,2.06167) 101 +dictGet dict_array (29.5699,2.50068) 101 +dictGet dict_array (29.5796,1.55456) 101 +dictGet dict_array (29.5796,2.36864) 101 +dictGet dict_array (29.5844,1.59626) 101 +dictGet dict_array (29.5886,4.03321) 101 +dictGet dict_array (29.5914,3.02628) 101 +dictGet dict_array (29.5926,-0.0965169) 101 +dictGet dict_array (29.5968,2.37773) 101 +dictGet dict_array (29.5984,0.755853) 101 +dictGet dict_array (29.6066,3.47173) 101 +dictGet dict_array (29.6085,-1.26007) 101 +dictGet dict_array (29.6131,0.246565) 101 +dictGet dict_array (29.6157,-0.266687) 101 +dictGet dict_array (29.6164,2.94674) 101 +dictGet dict_array (29.6195,-0.591941) 101 +dictGet dict_array (29.6231,1.54818) 101 +dictGet dict_array (29.6379,0.764114) 101 +dictGet dict_array (29.6462,-0.772059) 934570 +dictGet dict_array (29.6579,-1.07336) 101 +dictGet dict_array (29.6618,-0.271842) 101 +dictGet dict_array (29.6629,-0.303602) 101 +dictGet dict_array (29.6659,-0.782823) 934570 +dictGet dict_array (29.6736,-0.113832) 101 +dictGet dict_array (29.6759,3.02905) 101 +dictGet dict_array (29.6778,3.71898) 101 +dictGet dict_array (29.6796,1.10433) 101 +dictGet dict_array (29.6809,2.13677) 101 +dictGet dict_array (29.6935,4.11894) 101 +dictGet dict_array (29.6991,-1.4458199999999999) 101 +dictGet dict_array (29.6997,3.17297) 101 +dictGet dict_array (29.7043,3.6145899999999997) 101 +dictGet dict_array (29.7065,3.24885) 101 +dictGet dict_array (29.7126,0.28108) 101 +dictGet dict_array (29.7192,0.174273) 101 +dictGet dict_array (29.7217,-0.523481) 934570 +dictGet dict_array (29.7271,1.67967) 101 +dictGet dict_array (29.7311,4.12444) 101 +dictGet dict_array (29.7347,1.88378) 101 +dictGet dict_array (29.7358,0.67944) 101 +dictGet dict_array (29.7366,-0.2973) 101 +dictGet dict_array (29.7446,0.646536) 101 +dictGet dict_array (29.7453,-0.567963) 101 +dictGet dict_array (29.764,4.04217) 101 +dictGet dict_array (29.7655,1.51372) 101 +dictGet dict_array (29.7744,1.12435) 101 +dictGet dict_array (29.7774,-0.0681196) 101 +dictGet dict_array (29.7784,1.54864) 101 +dictGet dict_array (29.7785,2.24139) 101 +dictGet dict_array (29.7922,0.220808) 101 +dictGet dict_array (29.7936,2.37709) 101 +dictGet dict_array (29.8008,0.948536) 101 +dictGet dict_array (29.8115,0.201227) 101 +dictGet dict_array (29.814,0.149601) 101 +dictGet dict_array (29.8193,-1.35858) 101 +dictGet dict_array (29.8201,0.965518) 101 +dictGet dict_array (29.8265,-0.727286) 101 +dictGet dict_array (29.8277,-0.531746) 101 +dictGet dict_array (29.8289,3.63009) 101 +dictGet dict_array (29.8548,0.838047) 101 +dictGet dict_array (29.8641,-0.845265) 101 +dictGet dict_array (29.8649,0.0562212) 101 +dictGet dict_array (29.8701,-1.02045) 101 +dictGet dict_array (29.8733,2.76654) 101 +dictGet dict_array (29.876,0.555475) 101 +dictGet dict_array (29.8794,-0.800108) 101 +dictGet dict_array (29.8813,2.7426399999999997) 101 +dictGet dict_array (29.897100000000002,2.66193) 101 +dictGet dict_array (29.908,4.01339) 101 +dictGet dict_array (29.9165,-1.08246) 101 +dictGet dict_array (29.9201,-0.420861) 101 +dictGet dict_array (29.9217,3.03778) 101 +dictGet dict_array (29.9355,0.773833) 101 +dictGet dict_array (29.947,3.76517) 101 +dictGet dict_array (29.9518,-0.60557) 101 +dictGet dict_array (29.9564,-0.600163) 101 +dictGet dict_array (29.959600000000002,4.16591) 101 +dictGet dict_array (29.9615,-1.33708) 101 +dictGet dict_array (29.9699,-0.392375) 101 +dictGet dict_array (29.9776,1.04552) 101 +dictGet dict_array (29.9784,4.02756) 101 +dictGet dict_array (29.9819,4.00597) 101 +dictGet dict_array (29.9826,1.2816100000000001) 101 +dictGet dict_array (30.0026,2.76257) 101 +dictGet dict_array (30.0126,3.68255) 101 +dictGet dict_array (30.0131,0.796576) 101 +dictGet dict_array (30.018,1.16523) 101 +dictGet dict_array (30.0261,-0.210653) 101 +dictGet dict_array (30.0472,-1.11007) 101 +dictGet dict_array (30.0542,-0.479585) 101 +dictGet dict_array (30.0613,1.6278000000000001) 101 +dictGet dict_array (30.0617,-0.0551152) 101 +dictGet dict_array (30.0637,2.62066) 101 +dictGet dict_array (30.0721,1.6424400000000001) 101 +dictGet dict_array (30.0769,-0.402636) 101 +dictGet dict_array (30.0791,-0.277435) 101 +dictGet dict_array (30.0931,0.0327512) 101 +dictGet dict_array (30.1059,3.52623) 101 +dictGet dict_array (30.1103,0.865466) 101 +dictGet dict_array (30.1115,2.95243) 101 +dictGet dict_array (30.1144,1.71029) 101 +dictGet dict_array (30.1311,-0.864751) 101 +dictGet dict_array (30.1336,-0.851386) 101 +dictGet dict_array (30.1393,3.89901) 101 +dictGet dict_array (30.1456,-0.531898) 101 +dictGet dict_array (30.1492,2.07833) 101 +dictGet dict_array (30.1575,2.43856) 101 +dictGet dict_array (30.1682,1.19771) 101 +dictGet dict_array (30.1716,3.9853300000000003) 101 +dictGet dict_array (30.1849,2.78374) 101 +dictGet dict_array (30.1866,0.65658) 101 +dictGet dict_array (30.1885,1.56943) 101 +dictGet dict_array (30.1959,-1.38202) 101 +dictGet dict_array (30.1999,1.58413) 101 +dictGet dict_array (30.2024,0.713081) 101 +dictGet dict_array (30.2054,0.620143) 101 +dictGet dict_array (30.2091,1.51641) 101 +dictGet dict_array (30.2124,-0.331782) 101 +dictGet dict_array (30.226,3.03527) 101 +dictGet dict_array (30.2261,3.18486) 101 +dictGet dict_array (30.2288,2.48407) 101 +dictGet dict_array (30.2345,3.7462400000000002) 101 +dictGet dict_array (30.2375,0.62046) 101 +dictGet dict_array (30.2425,-0.472914) 101 +dictGet dict_array (30.247,3.95863) 101 +dictGet dict_array (30.2494,-0.305093) 101 +dictGet dict_array (30.2499,2.54337) 101 +dictGet dict_array (30.2606,2.16644) 101 +dictGet dict_array (30.2672,3.94847) 101 +dictGet dict_array (30.2709,-0.136264) 101 +dictGet dict_array (30.2764,1.18654) 101 +dictGet dict_array (30.2765,1.20383) 101 +dictGet dict_array (30.2839,1.05762) 101 +dictGet dict_array (30.286,0.469327) 101 +dictGet dict_array (30.2927,3.1693) 101 +dictGet dict_array (30.2935,3.49854) 101 +dictGet dict_array (30.307,0.312338) 101 +dictGet dict_array (30.3085,1.07791) 101 +dictGet dict_array (30.3139,2.77248) 101 +dictGet dict_array (30.314,0.822823) 101 +dictGet dict_array (30.3227,-0.587351) 101 +dictGet dict_array (30.332,1.00174) 101 +dictGet dict_array (30.3388,0.844148) 101 +dictGet dict_array (30.3485,0.561902) 101 +dictGet dict_array (30.3497,0.180362) 101 +dictGet dict_array (30.361,4.13016) 101 +dictGet dict_array (30.3623,-0.0484027) 101 +dictGet dict_array (30.3638,3.9845800000000002) 101 +dictGet dict_array (30.3853,3.16051) 101 +dictGet dict_array (30.3974,2.6617800000000003) 101 +dictGet dict_array (30.4002,-1.15886) 101 +dictGet dict_array (30.4008,-0.387015) 101 +dictGet dict_array (30.4018,1.86493) 101 +dictGet dict_array (30.4239,1.16818) 101 +dictGet dict_array (30.4363,3.63938) 101 +dictGet dict_array (30.4377,-0.81315) 101 +dictGet dict_array (30.4391,3.54703) 101 +dictGet dict_array (30.4424,-1.39435) 101 +dictGet dict_array (30.4441,2.8463000000000003) 101 +dictGet dict_array (30.4517,3.28117) 101 +dictGet dict_array (30.4658,2.6928) 101 +dictGet dict_array (30.4734,2.66161) 101 +dictGet dict_array (30.4799,-1.07578) 101 +dictGet dict_array (30.4837,-1.02486) 101 +dictGet dict_array (30.485,1.06326) 101 +dictGet dict_array (30.495,1.12306) 101 +dictGet dict_array (30.501,2.27264) 101 +dictGet dict_array (30.5027,1.99382) 101 +dictGet dict_array (30.5194,-1.03943) 101 +dictGet dict_array (30.5239,1.04328) 101 +dictGet dict_array (30.528,3.82041) 101 +dictGet dict_array (30.5299,-0.715248) 101 +dictGet dict_array (30.5331,1.19603) 101 +dictGet dict_array (30.535800000000002,2.71485) 101 +dictGet dict_array (30.5405,0.804694) 101 +dictGet dict_array (30.542,1.23739) 101 +dictGet dict_array (30.5432,4.04189) 101 +dictGet dict_array (30.5457,-0.956121) 101 +dictGet dict_array (30.5506,3.07443) 101 +dictGet dict_array (30.5539,3.87084) 101 +dictGet dict_array (30.5578,3.78837) 101 +dictGet dict_array (30.5588,0.966135) 101 +dictGet dict_array (30.5637,2.5605) 101 +dictGet dict_array (30.5647,-1.27328) 101 +dictGet dict_array (30.5656,-0.0581332) 101 +dictGet dict_array (30.5715,0.65755) 101 +dictGet dict_array (30.5727,3.01604) 101 +dictGet dict_array (30.5729,-0.976857) 101 +dictGet dict_array (30.5751,0.60204) 101 +dictGet dict_array (30.5854,3.02473) 101 +dictGet dict_array (30.5866,0.174099) 101 +dictGet dict_array (30.5947,0.875193) 101 +dictGet dict_array (30.5992,-0.403901) 101 +dictGet dict_array (30.6002,4.18891) 101 +dictGet dict_array (30.6025,0.217712) 101 +dictGet dict_array (30.6054,0.927203) 101 +dictGet dict_array (30.6075,3.79359) 101 +dictGet dict_array (30.6159,3.82773) 101 +dictGet dict_array (30.627,3.84039) 101 +dictGet dict_array (30.6308,0.77517) 101 +dictGet dict_array (30.6338,0.179565) 101 +dictGet dict_array (30.6461,1.3293599999999999) 101 +dictGet dict_array (30.6674,-0.424547) 101 +dictGet dict_array (30.669,1.76539) 101 +dictGet dict_array (30.6788,4.01239) 101 +dictGet dict_array (30.6864,3.59158) 101 +dictGet dict_array (30.7049,-0.875413) 101 +dictGet dict_array (30.705,1.3307) 101 +dictGet dict_array (30.7063,-0.473192) 101 +dictGet dict_array (30.7075,-1.1958199999999999) 101 +dictGet dict_array (30.7101,-0.367562) 101 +dictGet dict_array (30.7203,2.98725) 101 +dictGet dict_array (30.7213,2.2745699999999998) 101 +dictGet dict_array (30.7446,-0.334144) 101 +dictGet dict_array (30.7468,3.82967) 101 +dictGet dict_array (30.747,-0.384779) 101 +dictGet dict_array (30.7681,0.904198) 101 +dictGet dict_array (30.7757,1.78743) 101 +dictGet dict_array (30.8021,-0.479212) 101 +dictGet dict_array (30.8079,-1.40869) 101 +dictGet dict_array (30.8206,-0.0608489) 101 +dictGet dict_array (30.8218,0.43909) 101 +dictGet dict_array (30.8239,0.10014) 101 +dictGet dict_array (30.8282,4.15409) 101 +dictGet dict_array (30.8288,-0.709528) 101 +dictGet dict_array (30.8326,0.156011) 101 +dictGet dict_array (30.8328,-1.03704) 101 +dictGet dict_array (30.839,2.15528) 101 +dictGet dict_array (30.8452,0.219377) 101 +dictGet dict_array (30.8463,0.0515355) 101 +dictGet dict_array (30.8526,2.06614) 101 +dictGet dict_array (30.8566,0.517876) 101 +dictGet dict_array (30.8588,-1.31738) 101 +dictGet dict_array (30.8681,0.44207) 101 +dictGet dict_array (30.8914,1.0072) 101 +dictGet dict_array (30.897,0.483425) 101 +dictGet dict_array (30.905,2.8731999999999998) 101 +dictGet dict_array (30.9051,2.21956) 101 +dictGet dict_array (30.9115,4.00663) 101 +dictGet dict_array (30.9167,-0.834462) 101 +dictGet dict_array (30.9252,-1.3289900000000001) 101 +dictGet dict_array (30.9314,1.85384) 101 +dictGet dict_array (30.9392,2.53236) 101 +dictGet dict_array (30.9569,2.82038) 101 +dictGet dict_array (30.9598,-0.641011) 101 +dictGet dict_array (30.9601,-0.254928) 101 +dictGet dict_array (30.9623,-1.3886) 101 +dictGet dict_array (30.9707,0.888854) 101 +dictGet dict_array (30.9766,2.81957) 101 +dictGet dict_array (30.9775,2.69273) 101 +dictGet dict_array (30.9821,0.587715) 101 +dictGet dict_array (30.9887,4.0233) 101 +dictGet dict_array (30.9914,0.259542) 101 +dictGet dict_array (30.9986,-1.36832) 101 +dictGet dict_array (31.008,0.628999) 101 +dictGet dict_array (31.0168,-1.17462) 101 +dictGet dict_array (31.0237,3.52547) 101 +dictGet dict_array (31.0306,3.78522) 101 +dictGet dict_array (31.0308,-0.72453) 101 +dictGet dict_array (31.0463,2.41997) 101 +dictGet dict_array (31.047,0.624184) 101 +dictGet dict_array (31.0569,0.0706393) 5994232 +dictGet dict_array (31.0583,1.3244099999999999) 101 +dictGet dict_array (31.063,3.23861) 101 +dictGet dict_array (31.068,0.695575) 101 +dictGet dict_array (31.0687,1.85675) 101 +dictGet dict_array (31.0692,0.254793) 101 +dictGet dict_array (31.0766,0.828128) 101 +dictGet dict_array (31.0833,0.0612782) 5994232 +dictGet dict_array (31.0833,2.59748) 101 +dictGet dict_array (31.0861,-1.3778299999999999) 101 +dictGet dict_array (31.0874,3.07258) 101 +dictGet dict_array (31.0882,1.4882) 101 +dictGet dict_array (31.0924,3.42242) 101 +dictGet dict_array (31.0927,2.67448) 101 +dictGet dict_array (31.0936,1.12292) 101 +dictGet dict_array (31.0952,-0.336928) 101 +dictGet dict_array (31.0978,3.48482) 101 +dictGet dict_array (31.1107,3.7513199999999998) 101 +dictGet dict_array (31.1156,1.19171) 101 +dictGet dict_array (31.1176,0.223509) 5994232 +dictGet dict_array (31.1249,0.946838) 101 +dictGet dict_array (31.1267,1.48983) 101 +dictGet dict_array (31.138,-0.289981) 101 +dictGet dict_array (31.1382,3.02904) 101 +dictGet dict_array (31.1475,2.6178) 101 +dictGet dict_array (31.1491,1.37873) 101 +dictGet dict_array (31.1525,3.72105) 101 +dictGet dict_array (31.1526,-1.4129800000000001) 101 +dictGet dict_array (31.1526,-0.186457) 101 +dictGet dict_array (31.1539,2.78789) 101 +dictGet dict_array (31.1548,-1.08552) 101 +dictGet dict_array (31.1567,-0.0768925) 101 +dictGet dict_array (31.1613,1.49617) 101 +dictGet dict_array (31.1653,1.03777) 101 +dictGet dict_array (31.1662,3.4214700000000002) 101 +dictGet dict_array (31.1672,-0.0813169) 101 +dictGet dict_array (31.177,0.440843) 101 +dictGet dict_array (31.1788,-0.737151) 101 +dictGet dict_array (31.1856,-0.144396) 101 +dictGet dict_array (31.1959,3.66813) 101 +dictGet dict_array (31.1996,-0.353983) 101 +dictGet dict_array (31.2019,2.86802) 101 +dictGet dict_array (31.2087,2.31245) 101 +dictGet dict_array (31.2125,3.2713200000000002) 101 +dictGet dict_array (31.2137,-0.108129) 101 +dictGet dict_array (31.216,3.9156) 101 +dictGet dict_array (31.2201,-0.202141) 101 +dictGet dict_array (31.2285,2.09058) 101 +dictGet dict_array (31.2502,4.01526) 101 +dictGet dict_array (31.2585,3.11524) 101 +dictGet dict_array (31.2645,-0.620418) 101 +dictGet dict_array (31.2684,2.74277) 101 +dictGet dict_array (31.2821,-1.12772) 101 +dictGet dict_array (31.2821,2.46769) 101 +dictGet dict_array (31.2887,3.91396) 101 +dictGet dict_array (31.295,1.49942) 101 +dictGet dict_array (31.2997,3.46122) 101 +dictGet dict_array (31.3017,3.3263) 101 +dictGet dict_array (31.3022,3.16754) 101 +dictGet dict_array (31.3048,0.364962) 101 +dictGet dict_array (31.305,3.1967) 101 +dictGet dict_array (31.3061,1.84303) 101 +dictGet dict_array (31.3082,-0.173851) 101 +dictGet dict_array (31.3315,3.90932) 101 +dictGet dict_array (31.3351,2.80164) 101 +dictGet dict_array (31.3388,0.168765) 5994233 +dictGet dict_array (31.339,0.25535) 101 +dictGet dict_array (31.3423,1.7036799999999999) 101 +dictGet dict_array (31.349,0.386456) 101 +dictGet dict_array (31.3558,-1.04336) 101 +dictGet dict_array (31.3564,0.478876) 101 +dictGet dict_array (31.3607,-0.0860507) 5994233 +dictGet dict_array (31.3831,3.84469) 101 +dictGet dict_array (31.3886,-0.731137) 101 +dictGet dict_array (31.4043,-0.348907) 101 +dictGet dict_array (31.4081,1.47391) 101 +dictGet dict_array (31.4176,-0.583645) 101 +dictGet dict_array (31.4177,1.36972) 101 +dictGet dict_array (31.4182,0.958303) 101 +dictGet dict_array (31.4199,3.1738) 101 +dictGet dict_array (31.4221,2.74876) 101 +dictGet dict_array (31.4301,-0.122643) 5994233 +dictGet dict_array (31.4344,1.00661) 101 +dictGet dict_array (31.4375,4.20304) 101 +dictGet dict_array (31.4377,0.289608) 101 +dictGet dict_array (31.4379,0.54744) 101 +dictGet dict_array (31.4459,3.94945) 101 +dictGet dict_array (31.4559,-0.345063) 101 +dictGet dict_array (31.464,0.726129) 101 +dictGet dict_array (31.4662,-0.299019) 5994233 +dictGet dict_array (31.4671,1.9605299999999999) 101 +dictGet dict_array (31.4673,-0.403676) 101 +dictGet dict_array (31.4712,-0.237941) 5994233 +dictGet dict_array (31.4816,0.120264) 5994233 +dictGet dict_array (31.4875,0.323483) 101 +dictGet dict_array (31.490099999999998,-0.338163) 101 +dictGet dict_array (31.4932,0.517674) 101 +dictGet dict_array (31.5112,1.9689299999999998) 101 +dictGet dict_array (31.5122,2.92785) 101 +dictGet dict_array (31.5151,0.166429) 101 +dictGet dict_array (31.5174,2.94802) 101 +dictGet dict_array (31.5182,4.18776) 101 +dictGet dict_array (31.5238,1.18793) 101 +dictGet dict_array (31.5271,3.07446) 101 +dictGet dict_array (31.5393,1.58061) 101 +dictGet dict_array (31.5421,3.13711) 101 +dictGet dict_array (31.5479,2.39897) 101 +dictGet dict_array (31.5519,0.99285) 101 +dictGet dict_array (31.5685,3.47987) 101 +dictGet dict_array (31.5959,0.437382) 101 +dictGet dict_array (31.6003,0.194376) 101 +dictGet dict_array (31.6026,2.15457) 101 +dictGet dict_array (31.606,2.45365) 101 +dictGet dict_array (31.6062,-0.453441) 101 +dictGet dict_array (31.6107,1.35247) 101 +dictGet dict_array (31.6155,3.85588) 101 +dictGet dict_array (31.6222,2.03326) 101 +dictGet dict_array (31.6231,-0.123059) 101 +dictGet dict_array (31.6244,1.6885599999999998) 101 +dictGet dict_array (31.6459,0.669716) 101 +dictGet dict_array (31.6563,-0.0644741) 101 +dictGet dict_array (31.6618,-0.551121) 101 +dictGet dict_array (31.6725,-0.38922) 101 +dictGet dict_array (31.6727,4.10336) 101 +dictGet dict_array (31.6739,4.1391) 101 +dictGet dict_array (31.6897,2.8694699999999997) 101 +dictGet dict_array (31.6902,3.98792) 101 +dictGet dict_array (31.6945,2.46687) 101 +dictGet dict_array (31.6987,-1.3796) 101 +dictGet dict_array (31.7012,2.34845) 101 +dictGet dict_array (31.7036,0.0228348) 101 +dictGet dict_array (31.7046,3.68111) 101 +dictGet dict_array (31.7055,2.92556) 101 +dictGet dict_array (31.7102,1.04532) 101 +dictGet dict_array (31.7149,-0.443302) 101 +dictGet dict_array (31.7195,2.99311) 101 +dictGet dict_array (31.7274,0.166719) 101 +dictGet dict_array (31.7565,-0.565382) 101 +dictGet dict_array (31.7615,0.771626) 101 +dictGet dict_array (31.7739,1.8970099999999999) 101 +dictGet dict_array (31.7848,1.2623199999999999) 101 +dictGet dict_array (31.7912,-0.788599) 101 +dictGet dict_array (31.8011,2.65853) 101 +dictGet dict_array (31.8032,-0.0590108) 101 +dictGet dict_array (31.8038,1.9618799999999998) 101 +dictGet dict_array (31.8098,-1.46851) 101 +dictGet dict_array (31.8131,3.41982) 101 +dictGet dict_array (31.8169,3.31059) 101 +dictGet dict_array (31.8202,-0.193692) 101 +dictGet dict_array (31.8306,1.57586) 101 +dictGet dict_array (31.8382,-0.787948) 101 +dictGet dict_array (31.8433,2.49692) 101 +dictGet dict_array (31.8436,2.41851) 101 +dictGet dict_array (31.8563,-1.10787) 101 +dictGet dict_array (31.8683,0.996504) 101 +dictGet dict_array (31.8693,-0.828142) 101 +dictGet dict_array (31.8723,1.08929) 101 +dictGet dict_array (31.8737,0.881127) 101 +dictGet dict_array (31.8881,-0.58441) 101 +dictGet dict_array (31.9011,0.121349) 101 +dictGet dict_array (31.9066,2.13045) 101 +dictGet dict_array (31.9142,1.03368) 101 +dictGet dict_array (31.9155,3.38363) 101 +dictGet dict_array (31.9168,1.3166) 101 +dictGet dict_array (31.9185,-1.11879) 101 +dictGet dict_array (31.9186,-0.647948) 101 +dictGet dict_array (31.9311,3.96928) 101 +dictGet dict_array (31.9335,1.47048) 101 +dictGet dict_array (31.9443,-1.36175) 101 +dictGet dict_array (31.9481,2.34231) 101 +dictGet dict_array (31.9526,1.36565) 101 +dictGet dict_array (31.9629,2.5208399999999997) 101 +dictGet dict_array (31.9765,0.975783) 101 +dictGet dict_array (31.9923,3.31773) 101 +dictGet dict_array (31.9994,0.972816) 101 +dictGet dict_array (32.001,3.47425) 101 +dictGet dict_array (32.0127,2.13874) 101 +dictGet dict_array (32.0244,3.2092) 101 +dictGet dict_array (32.029,1.18039) 101 +dictGet dict_array (32.0315,0.566073) 101 +dictGet dict_array (32.0354,1.0766499999999999) 101 +dictGet dict_array (32.0399,-1.11576) 101 +dictGet dict_array (32.053,2.16849) 101 +dictGet dict_array (32.0542,0.042328) 101 +dictGet dict_array (32.0576,2.47001) 101 +dictGet dict_array (32.061,3.7498899999999997) 101 +dictGet dict_array (32.0623,1.25134) 101 +dictGet dict_array (32.0626,1.9611399999999999) 101 +dictGet dict_array (32.0666,-0.0904247) 101 +dictGet dict_array (32.0681,2.28442) 101 +dictGet dict_array (32.0692,1.50869) 101 +dictGet dict_array (32.0724,4.03314) 101 +dictGet dict_array (32.0729,-0.064324) 101 +dictGet dict_array (32.079,0.293758) 101 +dictGet dict_array (32.0847,-1.19814) 101 +dictGet dict_array (32.0974,-0.91927) 101 +dictGet dict_array (32.0979,-0.736979) 101 +dictGet dict_array (32.106,-1.33063) 101 +dictGet dict_array (32.1189,0.246715) 101 +dictGet dict_array (32.1207,4.00883) 101 +dictGet dict_array (32.1396,1.12402) 101 +dictGet dict_array (32.1413,1.5668) 101 +dictGet dict_array (32.143,1.35559) 101 +dictGet dict_array (32.1538,1.32881) 101 +dictGet dict_array (32.1549,4.06552) 101 +dictGet dict_array (32.1555,-0.79275) 101 +dictGet dict_array (32.163,1.17733) 101 +dictGet dict_array (32.1634,2.94273) 101 +dictGet dict_array (32.1644,1.85666) 101 +dictGet dict_array (32.1745,0.435458) 101 +dictGet dict_array (32.1765,1.65149) 101 +dictGet dict_array (32.1893,2.08924) 101 +dictGet dict_array (32.2024,0.222191) 101 +dictGet dict_array (32.2107,1.34379) 101 +dictGet dict_array (32.2109,3.9018699999999997) 101 +dictGet dict_array (32.2123,1.85233) 101 +dictGet dict_array (32.2144,3.72534) 101 +dictGet dict_array (32.2218,2.5386699999999998) 101 +dictGet dict_array (32.2279,2.84267) 101 +dictGet dict_array (32.2345,3.33295) 101 +dictGet dict_array (32.2435,3.85283) 101 +dictGet dict_array (32.2527,-0.480608) 101 +dictGet dict_array (32.2566,-0.837882) 101 +dictGet dict_array (32.2627,2.57708) 101 +dictGet dict_array (32.2733,0.244931) 101 +dictGet dict_array (32.2761,4.05808) 101 +dictGet dict_array (32.2764,3.78472) 101 +dictGet dict_array (32.2814,-1.26011) 101 +dictGet dict_array (32.2861,3.02427) 101 +dictGet dict_array (32.2924,0.928609) 101 +dictGet dict_array (32.2963,-0.78543) 101 +dictGet dict_array (32.3039,3.21175) 101 +dictGet dict_array (32.3107,0.698287) 101 +dictGet dict_array (32.3138,0.0595677) 101 +dictGet dict_array (32.3339,0.707056) 101 +dictGet dict_array (32.3351,0.415474) 101 +dictGet dict_array (32.342,-0.681023) 101 +dictGet dict_array (32.3463,1.83196) 101 +dictGet dict_array (32.3494,2.43799) 101 +dictGet dict_array (32.3524,3.47049) 101 +dictGet dict_array (32.3531,2.33115) 101 +dictGet dict_array (32.3602,0.116106) 101 +dictGet dict_array (32.3612,1.1598) 101 +dictGet dict_array (32.3689,3.34847) 101 +dictGet dict_array (32.3695,0.734055) 101 +dictGet dict_array (32.3825,3.85017) 101 +dictGet dict_array (32.3835,-1.25491) 101 +dictGet dict_array (32.4018,-0.728568) 101 +dictGet dict_array (32.4044,2.96727) 101 +dictGet dict_array (32.4101,2.9988) 101 +dictGet dict_array (32.417,-1.12908) 101 +dictGet dict_array (32.4172,4.1952) 101 +dictGet dict_array (32.4239,2.49512) 101 +dictGet dict_array (32.4258,4.05137) 101 +dictGet dict_array (32.4264,-0.427357) 101 +dictGet dict_array (32.4274,3.59377) 101 +dictGet dict_array (32.4286,-1.24757) 101 +dictGet dict_array (32.4294,3.0665) 101 +dictGet dict_array (32.4333,-0.353347) 101 +dictGet dict_array (32.4391,3.64421) 101 +dictGet dict_array (32.4401,3.70635) 101 +dictGet dict_array (32.45,1.68918) 101 +dictGet dict_array (32.4507,-0.133471) 101 +dictGet dict_array (32.4592,0.976458) 101 +dictGet dict_array (32.4595,1.89135) 101 +dictGet dict_array (32.4604,0.280248) 101 +dictGet dict_array (32.4835,0.472731) 101 +dictGet dict_array (32.4855,2.01938) 101 +dictGet dict_array (32.4872,2.01697) 101 +dictGet dict_array (32.4911,0.613106) 101 +dictGet dict_array (32.4918,2.17834) 101 +dictGet dict_array (32.4947,2.34595) 101 +dictGet dict_array (32.5035,2.92234) 101 +dictGet dict_array (32.5132,-0.331206) 101 +dictGet dict_array (32.5156,-0.412604) 7652581 +dictGet dict_array (32.5158,2.9067499999999997) 101 +dictGet dict_array (32.5249,2.44519) 101 +dictGet dict_array (32.5293,-0.790952) 101 +dictGet dict_array (32.5319,3.96854) 101 +dictGet dict_array (32.5518,3.6093) 101 +dictGet dict_array (32.5541,3.5225400000000002) 101 +dictGet dict_array (32.5569,0.816123) 101 +dictGet dict_array (32.5646,1.9775) 101 +dictGet dict_array (32.5733,3.81271) 101 +dictGet dict_array (32.5767,0.948327) 101 +dictGet dict_array (32.5971,1.76179) 101 +dictGet dict_array (32.6035,-0.716157) 101 +dictGet dict_array (32.6087,4.21614) 101 +dictGet dict_array (32.6171,0.024481) 101 +dictGet dict_array (32.6189,-0.775391) 101 +dictGet dict_array (32.6198,2.92081) 101 +dictGet dict_array (32.621,-0.970784) 101 +dictGet dict_array (32.6266,0.650009) 101 +dictGet dict_array (32.6315,2.15144) 101 +dictGet dict_array (32.6385,-0.436803) 101 +dictGet dict_array (32.6449,-0.191292) 101 +dictGet dict_array (32.6535,2.10385) 101 +dictGet dict_array (32.6592,3.49973) 101 +dictGet dict_array (32.6598,2.5980600000000003) 101 +dictGet dict_array (32.6612,2.95681) 101 +dictGet dict_array (32.6636,-0.57235) 101 +dictGet dict_array (32.669,-0.382702) 101 +dictGet dict_array (32.6752,1.30748) 101 +dictGet dict_array (32.6811,2.9559800000000003) 101 +dictGet dict_array (32.6821,0.57336) 101 +dictGet dict_array (32.6828,3.91304) 101 +dictGet dict_array (32.6979,3.96868) 101 +dictGet dict_array (32.6983,3.15784) 101 +dictGet dict_array (32.7122,0.794293) 101 +dictGet dict_array (32.7131,-0.847256) 101 +dictGet dict_array (32.7219,0.883461) 101 +dictGet dict_array (32.7228,1.78808) 101 +dictGet dict_array (32.7273,-0.206908) 101 +dictGet dict_array (32.7292,0.259331) 101 +dictGet dict_array (32.7304,-1.38317) 101 +dictGet dict_array (32.7353,1.01601) 101 +dictGet dict_array (32.7354,4.17574) 101 +dictGet dict_array (32.7357,-0.190194) 101 +dictGet dict_array (32.7465,-1.37598) 101 +dictGet dict_array (32.7494,-0.275675) 101 +dictGet dict_array (32.7514,0.128951) 101 +dictGet dict_array (32.753,3.44207) 101 +dictGet dict_array (32.7686,2.11713) 101 +dictGet dict_array (32.7694,1.47159) 101 +dictGet dict_array (32.7768,0.0401042) 101 +dictGet dict_array (32.781,-1.34283) 101 +dictGet dict_array (32.7814,1.73876) 101 +dictGet dict_array (32.7856,-1.06363) 101 +dictGet dict_array (32.792699999999996,-1.1255600000000001) 101 +dictGet dict_array (32.7941,-0.645447) 101 +dictGet dict_array (32.7946,1.48889) 101 +dictGet dict_array (32.797,0.791753) 101 +dictGet dict_array (32.7982,-0.537798) 101 +dictGet dict_array (32.8091,2.3611) 101 +dictGet dict_array (32.81,1.7130800000000002) 101 +dictGet dict_array (32.8174,-0.288322) 101 +dictGet dict_array (32.823,1.6546699999999999) 101 +dictGet dict_array (32.8233,1.62108) 101 +dictGet dict_array (32.8428,-0.400045) 101 +dictGet dict_array (32.8479,2.13598) 101 +dictGet dict_array (32.8524,0.199902) 101 +dictGet dict_array (32.8543,3.23553) 101 +dictGet dict_array (32.8562,1.31371) 101 +dictGet dict_array (32.87,1.44256) 101 +dictGet dict_array (32.8789,2.38192) 101 +dictGet dict_array (32.8812,2.20734) 5999168 +dictGet dict_array (32.8815,-0.54427) 101 +dictGet dict_array (32.8853,2.4859) 5999168 +dictGet dict_array (32.8909,0.513964) 101 +dictGet dict_array (32.9035,2.38999) 101 +dictGet dict_array (32.9097,2.48131) 5999168 +dictGet dict_array (32.928,-0.943269) 101 +dictGet dict_array (32.9322,1.13165) 101 +dictGet dict_array (32.9348,1.22606) 101 +dictGet dict_array (32.9417,3.77998) 101 +dictGet dict_array (32.9428,3.11936) 101 +dictGet dict_array (32.9482,1.18092) 101 +dictGet dict_array (32.9506,0.0609364) 101 +dictGet dict_array (32.953,-0.828308) 101 +dictGet dict_array (32.9593,3.5209099999999998) 101 +dictGet dict_array (32.9617,2.07711) 5999168 +dictGet dict_array (32.966,0.693749) 101 +dictGet dict_array (32.9668,-0.716432) 101 +dictGet dict_array (32.9702,1.98555) 101 +dictGet dict_array (32.9782,1.73819) 101 +dictGet dict_array (32.9805,3.71151) 101 +dictGet dict_array (32.9821,2.97225) 101 +dictGet dict_array (32.995,-0.830301) 101 +dictGet dict_array (33.0234,0.770848) 101 +dictGet dict_array (33.0312,-0.340964) 101 +dictGet dict_array (33.0366,-0.756795) 101 +dictGet dict_array (33.0438,0.812871) 101 +dictGet dict_array (33.0455,1.84843) 101 +dictGet dict_array (33.0498,0.0913292) 101 +dictGet dict_array (33.0506,1.53739) 101 +dictGet dict_array (33.0554,2.4265) 101 +dictGet dict_array (33.0741,3.61332) 101 +dictGet dict_array (33.0765,-0.179985) 101 +dictGet dict_array (33.087,1.46465) 101 +dictGet dict_array (33.0906,-0.620383) 101 +dictGet dict_array (33.1047,-1.28027) 101 +dictGet dict_array (33.1072,1.96303) 101 +dictGet dict_array (33.1081,-0.897874) 101 +dictGet dict_array (33.1122,1.8950200000000001) 101 +dictGet dict_array (33.1237,2.63993) 101 +dictGet dict_array (33.1238,0.753963) 101 +dictGet dict_array (33.1257,0.495668) 101 +dictGet dict_array (33.1258,1.78341) 101 +dictGet dict_array (33.127,2.59646) 101 +dictGet dict_array (33.1324,-1.23742) 101 +dictGet dict_array (33.1359,3.83491) 101 +dictGet dict_array (33.1628,-0.379588) 101 +dictGet dict_array (33.1679,1.25601) 101 +dictGet dict_array (33.1688,-1.35553) 101 +dictGet dict_array (33.181,2.10943) 101 +dictGet dict_array (33.1871,2.81171) 101 +dictGet dict_array (33.1877,0.771297) 101 +dictGet dict_array (33.1883,-0.204797) 101 +dictGet dict_array (33.1886,3.27998) 101 +dictGet dict_array (33.1955,0.708907) 101 +dictGet dict_array (33.2044,-0.769275) 101 +dictGet dict_array (33.2182,3.36103) 101 +dictGet dict_array (33.2192,3.43586) 101 +dictGet dict_array (33.2322,-0.916753) 101 +dictGet dict_array (33.2359,-0.81321) 101 +dictGet dict_array (33.238,0.635072) 101 +dictGet dict_array (33.2398,3.02588) 101 +dictGet dict_array (33.2469,2.35698) 101 +dictGet dict_array (33.247,2.3327) 101 +dictGet dict_array (33.2579,2.8027100000000003) 101 +dictGet dict_array (33.2607,0.321082) 101 +dictGet dict_array (33.2653,0.243336) 101 +dictGet dict_array (33.2758,0.831836) 101 +dictGet dict_array (33.2771,0.886536) 101 +dictGet dict_array (33.2914,1.16026) 101 +dictGet dict_array (33.2914,1.38882) 101 +dictGet dict_array (33.2982,-1.16604) 101 +dictGet dict_array (33.2985,0.842556) 101 +dictGet dict_array (33.3005,2.8338900000000002) 101 +dictGet dict_array (33.305,0.0969475) 101 +dictGet dict_array (33.3072,3.82163) 101 +dictGet dict_array (33.312,3.41475) 101 +dictGet dict_array (33.3129,2.46048) 101 +dictGet dict_array (33.3134,3.46863) 101 +dictGet dict_array (33.3203,2.33139) 101 +dictGet dict_array (33.324,0.433701) 101 +dictGet dict_array (33.3338,2.44705) 101 +dictGet dict_array (33.337,4.06475) 101 +dictGet dict_array (33.3469,1.08172) 101 +dictGet dict_array (33.3538,0.717896) 101 +dictGet dict_array (33.3618,1.37899) 101 +dictGet dict_array (33.3698,0.547744) 101 +dictGet dict_array (33.3705,0.957619) 101 +dictGet dict_array (33.3821,3.07258) 101 +dictGet dict_array (33.3881,3.0626) 101 +dictGet dict_array (33.393,-0.816186) 101 +dictGet dict_array (33.3945,0.869508) 101 +dictGet dict_array (33.4001,1.24186) 101 +dictGet dict_array (33.4008,2.34911) 101 +dictGet dict_array (33.4166,-1.2808899999999999) 101 +dictGet dict_array (33.4167,3.0655) 101 +dictGet dict_array (33.4204,2.81887) 101 +dictGet dict_array (33.4211,1.71128) 101 +dictGet dict_array (33.4237,2.91761) 101 +dictGet dict_array (33.4266,1.5955599999999999) 101 +dictGet dict_array (33.4353,-0.391392) 101 +dictGet dict_array (33.4362,-0.134658) 101 +dictGet dict_array (33.4386,0.15396) 101 +dictGet dict_array (33.4421,-0.50712) 101 +dictGet dict_array (33.452,0.915829) 101 +dictGet dict_array (33.463,-0.0882717) 101 +dictGet dict_array (33.464,-1.00949) 101 +dictGet dict_array (33.4692,0.954092) 101 +dictGet dict_array (33.4716,1.9538799999999998) 101 +dictGet dict_array (33.4756,1.85836) 101 +dictGet dict_array (33.4859,4.0751) 101 +dictGet dict_array (33.4899,3.54193) 101 +dictGet dict_array (33.4935,3.49794) 101 +dictGet dict_array (33.494,-0.983356) 101 +dictGet dict_array (33.4955,-1.28128) 101 +dictGet dict_array (33.4965,-0.278687) 101 +dictGet dict_array (33.4991,0.647491) 101 +dictGet dict_array (33.5076,2.2272) 101 +dictGet dict_array (33.5079,-0.498199) 101 +dictGet dict_array (33.5157,0.535034) 101 +dictGet dict_array (33.5171,2.49677) 101 +dictGet dict_array (33.5255,2.4447200000000002) 101 +dictGet dict_array (33.526,4.01194) 101 +dictGet dict_array (33.5288,0.789434) 101 +dictGet dict_array (33.5356,-1.17671) 101 +dictGet dict_array (33.5402,1.49152) 101 +dictGet dict_array (33.5418,3.45757) 101 +dictGet dict_array (33.5428,1.90712) 101 +dictGet dict_array (33.5556,-0.55741) 101 +dictGet dict_array (33.5564,0.876858) 101 +dictGet dict_array (33.5567,-0.10208) 101 +dictGet dict_array (33.5645,-0.124824) 101 +dictGet dict_array (33.5663,3.4872) 101 +dictGet dict_array (33.5716,-0.0107611) 101 +dictGet dict_array (33.578,3.55714) 101 +dictGet dict_array (33.5826,-0.49076) 101 +dictGet dict_array (33.5909,0.773737) 101 +dictGet dict_array (33.5958,2.9619999999999997) 5994231 +dictGet dict_array (33.6193,-0.919755) 101 +dictGet dict_array (33.6313,0.652132) 101 +dictGet dict_array (33.632,0.823351) 101 +dictGet dict_array (33.66,2.18998) 101 +dictGet dict_array (33.6621,0.535395) 101 +dictGet dict_array (33.6726,3.19367) 101 +dictGet dict_array (33.6912,1.74522) 101 +dictGet dict_array (33.705,0.706397) 101 +dictGet dict_array (33.7076,0.7622) 101 +dictGet dict_array (33.7112,1.70187) 101 +dictGet dict_array (33.7246,-1.14837) 101 +dictGet dict_array (33.7326,2.62413) 5994231 +dictGet dict_array (33.7332,2.82137) 5994231 +dictGet dict_array (33.7434,0.394672) 101 +dictGet dict_array (33.7443,1.54557) 101 +dictGet dict_array (33.7506,1.57317) 101 +dictGet dict_array (33.7526,1.8578999999999999) 101 +dictGet dict_array (33.766,4.15013) 101 +dictGet dict_array (33.7834,2.41789) 101 +dictGet dict_array (33.7864,0.230935) 101 +dictGet dict_array (33.7965,3.05709) 101 +dictGet dict_array (33.7998,3.32881) 101 +dictGet dict_array (33.8003,2.97338) 5994231 +dictGet dict_array (33.8007,-1.08962) 101 +dictGet dict_array (33.8022,-0.139488) 101 +dictGet dict_array (33.8065,2.70857) 5994231 +dictGet dict_array (33.8169,-0.607788) 101 +dictGet dict_array (33.8203,0.108512) 101 +dictGet dict_array (33.8231,-1.03449) 101 +dictGet dict_array (33.8312,3.49458) 101 +dictGet dict_array (33.8342,0.297518) 101 +dictGet dict_array (33.8352,0.165872) 101 +dictGet dict_array (33.8354,1.87277) 101 +dictGet dict_array (33.8371,1.60103) 101 +dictGet dict_array (33.8387,1.9968) 101 +dictGet dict_array (33.8403,3.5805) 101 +dictGet dict_array (33.8414,-0.703067) 101 +dictGet dict_array (33.844,-0.179472) 101 +dictGet dict_array (33.8468,3.40137) 101 +dictGet dict_array (33.8509,4.15334) 101 +dictGet dict_array (33.8539,2.38339) 101 +dictGet dict_array (33.858,-1.3122500000000001) 101 +dictGet dict_array (33.859,3.72626) 101 +dictGet dict_array (33.8616,2.24433) 101 +dictGet dict_array (33.8621,3.01035) 101 +dictGet dict_array (33.8623,1.17559) 101 +dictGet dict_array (33.8682,2.706) 5994231 +dictGet dict_array (33.8684,0.189231) 101 +dictGet dict_array (33.872,1.93574) 101 +dictGet dict_array (33.8844,3.80404) 101 +dictGet dict_array (33.8888,0.594884) 101 +dictGet dict_array (33.8946,2.74161) 101 +dictGet dict_array (33.9023,0.6239) 101 +dictGet dict_array (33.9057,0.873222) 101 +dictGet dict_array (33.9157,-1.26607) 101 +dictGet dict_array (33.92,2.06848) 101 +dictGet dict_array (33.9298,-0.00526229) 101 +dictGet dict_array (33.932,3.07063) 101 +dictGet dict_array (33.9322,0.629385) 101 +dictGet dict_array (33.9367,-1.41955) 101 +dictGet dict_array (33.937,1.42532) 101 +dictGet dict_array (33.9375,1.1467100000000001) 101 +dictGet dict_array (33.9434,-1.05739) 101 +dictGet dict_array (33.9477,3.34809) 101 +dictGet dict_array (33.95,2.21715) 101 +dictGet dict_array (33.955799999999996,0.305176) 101 +dictGet dict_array (33.9686,-0.28273) 101 +dictGet dict_array (33.9703,4.1255) 101 +dictGet dict_array (33.9707,3.08199) 101 +dictGet dict_array (33.9754,1.06203) 101 +dictGet dict_array (33.9757,3.72468) 101 +dictGet dict_array (33.9775,-0.0440599) 101 +dictGet dict_array (33.9777,-0.251484) 101 +dictGet dict_array (33.9789,-0.339374) 101 +dictGet dict_array (33.9849,2.54515) 5994231 +dictGet dict_array (33.9885,-0.318557) 101 +dictGet dict_array (33.9977,1.07175) 101 +dictGet dict_array (33.9984,-0.700517) 101 +dictGet dict_array (34.0149,3.53338) 101 +dictGet dict_array (34.0173,3.39155) 101 +dictGet dict_array (34.0317,3.9579) 101 +dictGet dict_array (34.0369,3.83612) 101 +dictGet dict_array (34.043,-0.0887221) 101 +dictGet dict_array (34.0487,1.14252) 101 +dictGet dict_array (34.052,1.74832) 101 +dictGet dict_array (34.0711,-0.898071) 101 +dictGet dict_array (34.0747,1.55057) 101 +dictGet dict_array (34.0803,3.16763) 101 +dictGet dict_array (34.0872,3.75555) 101 +dictGet dict_array (34.0965,1.62038) 101 +dictGet dict_array (34.0977,-0.412691) 101 +dictGet dict_array (34.0986,0.0294206) 101 +dictGet dict_array (34.1072,3.15823) 101 +dictGet dict_array (34.1092,3.09599) 101 +dictGet dict_array (34.1206,1.04637) 5940222 +dictGet dict_array (34.1209,3.13826) 101 +dictGet dict_array (34.1265,3.95881) 101 +dictGet dict_array (34.1286,-0.539319) 101 +dictGet dict_array (34.1358,3.67451) 101 +dictGet dict_array (34.1428,0.136115) 101 +dictGet dict_array (34.157,1.73522) 101 +dictGet dict_array (34.1581,1.48001) 101 +dictGet dict_array (34.1682,3.42373) 101 +dictGet dict_array (34.1683,-1.26511) 101 +dictGet dict_array (34.1684,4.20007) 101 +dictGet dict_array (34.1854,3.32089) 101 +dictGet dict_array (34.2022,0.749536) 101 +dictGet dict_array (34.2044,3.04865) 101 +dictGet dict_array (34.22,-0.500055) 101 +dictGet dict_array (34.2249,0.743775) 101 +dictGet dict_array (34.2254,1.34702) 101 +dictGet dict_array (34.2355,-0.898843) 101 +dictGet dict_array (34.2394,2.0203699999999998) 101 +dictGet dict_array (34.2466,1.83785) 101 +dictGet dict_array (34.247,4.09563) 101 +dictGet dict_array (34.2508,2.61312) 101 +dictGet dict_array (34.2517,1.69642) 101 +dictGet dict_array (34.2564,4.13033) 101 +dictGet dict_array (34.2574,4.18928) 101 +dictGet dict_array (34.2614,-0.478719) 101 +dictGet dict_array (34.2625,2.38088) 101 +dictGet dict_array (34.2666,3.1503) 101 +dictGet dict_array (34.271,4.02223) 101 +dictGet dict_array (34.2727,0.514755) 101 +dictGet dict_array (34.278,1.98929) 101 +dictGet dict_array (34.2798,-0.199208) 101 +dictGet dict_array (34.2804,2.05184) 101 +dictGet dict_array (34.2945,-1.11051) 101 +dictGet dict_array (34.3168,-0.0829721) 101 +dictGet dict_array (34.3345,3.4358) 101 +dictGet dict_array (34.3377,1.13527) 5940222 +dictGet dict_array (34.3383,1.27891) 5940222 +dictGet dict_array (34.3391,1.47945) 5940222 +dictGet dict_array (34.3441,0.627014) 101 +dictGet dict_array (34.347,2.4853) 101 +dictGet dict_array (34.3514,2.16247) 101 +dictGet dict_array (34.3627,2.64533) 101 +dictGet dict_array (34.3682,-0.227501) 101 +dictGet dict_array (34.3756,4.21248) 101 +dictGet dict_array (34.379,3.96604) 101 +dictGet dict_array (34.3827,1.7518) 101 +dictGet dict_array (34.3912,2.8834) 101 +dictGet dict_array (34.3919,0.668829) 101 +dictGet dict_array (34.3949,2.00338) 101 +dictGet dict_array (34.3987,0.557268) 101 +dictGet dict_array (34.4111,0.768558) 101 +dictGet dict_array (34.4119,2.8742) 101 +dictGet dict_array (34.416,3.50841) 101 +dictGet dict_array (34.4212,1.24916) 5940222 +dictGet dict_array (34.4251,0.457029) 101 +dictGet dict_array (34.4274,-0.902559) 101 +dictGet dict_array (34.4325,4.03159) 101 +dictGet dict_array (34.438,1.63994) 101 +dictGet dict_array (34.4403,-0.177594) 101 +dictGet dict_array (34.4421,0.726712) 101 +dictGet dict_array (34.4517,2.98611) 101 +dictGet dict_array (34.4658,-1.312) 101 +dictGet dict_array (34.4732,-0.0681338) 101 +dictGet dict_array (34.4752,2.81646) 101 +dictGet dict_array (34.4914,2.3858) 101 +dictGet dict_array (34.4923,0.855231) 101 +dictGet dict_array (34.5235,1.78468) 101 +dictGet dict_array (34.5305,4.10608) 101 +dictGet dict_array (34.5389,0.621937) 101 +dictGet dict_array (34.5406,3.17145) 101 +dictGet dict_array (34.5434,-0.56306) 101 +dictGet dict_array (34.5449,3.13311) 101 +dictGet dict_array (34.5491,2.31572) 101 +dictGet dict_array (34.5539,2.94028) 101 +dictGet dict_array (34.5546,-0.208825) 101 +dictGet dict_array (34.5549,3.78486) 101 +dictGet dict_array (34.5676,0.307148) 101 +dictGet dict_array (34.5743,1.5217399999999999) 101 +dictGet dict_array (34.5775,3.48046) 101 +dictGet dict_array (34.5815,2.5243700000000002) 101 +dictGet dict_array (34.5841,4.21191) 101 +dictGet dict_array (34.5887,2.65083) 101 +dictGet dict_array (34.5937,3.2143) 101 +dictGet dict_array (34.6013,-1.0612) 101 +dictGet dict_array (34.6089,1.36066) 101 +dictGet dict_array (34.6103,3.40227) 101 +dictGet dict_array (34.6128,1.92276) 101 +dictGet dict_array (34.6175,2.43627) 101 +dictGet dict_array (34.6209,3.43776) 101 +dictGet dict_array (34.6234,2.60237) 101 +dictGet dict_array (34.6275,3.52479) 101 +dictGet dict_array (34.635,0.568558) 101 +dictGet dict_array (34.6373,2.37692) 101 +dictGet dict_array (34.6375,3.52234) 101 +dictGet dict_array (34.6426,2.12397) 101 +dictGet dict_array (34.6513,2.80915) 101 +dictGet dict_array (34.6632,2.30039) 101 +dictGet dict_array (34.6691,1.86582) 101 +dictGet dict_array (34.6739,0.15342) 101 +dictGet dict_array (34.6825,0.0499679) 101 +dictGet dict_array (34.6893,0.454326) 101 +dictGet dict_array (34.6957,-0.358598) 101 +dictGet dict_array (34.6986,0.562679) 101 +dictGet dict_array (34.712,1.12114) 101 +dictGet dict_array (34.7126,-0.0057301) 101 +dictGet dict_array (34.7137,0.0248501) 101 +dictGet dict_array (34.7162,1.15623) 101 +dictGet dict_array (34.7258,3.95142) 101 +dictGet dict_array (34.7347,3.5232099999999997) 101 +dictGet dict_array (34.7363,2.23374) 101 +dictGet dict_array (34.7375,0.397841) 101 +dictGet dict_array (34.7423,3.09198) 101 +dictGet dict_array (34.7452,3.09029) 101 +dictGet dict_array (34.7539,-1.06943) 101 +dictGet dict_array (34.7733,-0.00912717) 101 +dictGet dict_array (34.774,2.71088) 101 +dictGet dict_array (34.7771,1.46009) 101 +dictGet dict_array (34.7782,-1.28308) 101 +dictGet dict_array (34.7924,3.63564) 101 +dictGet dict_array (34.7939,-0.416676) 101 +dictGet dict_array (34.7964,-0.401773) 101 +dictGet dict_array (34.7974,0.0286873) 101 +dictGet dict_array (34.7975,3.05965) 101 +dictGet dict_array (34.8037,3.07263) 101 +dictGet dict_array (34.8254,-0.390284) 101 +dictGet dict_array (34.828,1.91869) 101 +dictGet dict_array (34.8289,3.71058) 101 +dictGet dict_array (34.8403,2.14606) 101 +dictGet dict_array (34.8437,2.20617) 101 +dictGet dict_array (34.8469,2.38435) 101 +dictGet dict_array (34.86,1.45705) 101 +dictGet dict_array (34.8612,0.914248) 101 +dictGet dict_array (34.8663,3.4215400000000002) 101 +dictGet dict_array (34.8724,-0.375144) 101 +dictGet dict_array (34.8795,3.29317) 101 +dictGet dict_array (34.8823,1.21988) 101 +dictGet dict_array (34.8834,1.07657) 101 +dictGet dict_array (34.8837,0.157648) 101 +dictGet dict_array (34.8871,-0.9755) 101 +dictGet dict_array (34.8871,1.8943699999999999) 101 +dictGet dict_array (34.889,3.36756) 101 +dictGet dict_array (34.8907,1.24874) 101 +dictGet dict_array (34.8965,3.13508) 101 +dictGet dict_array (34.9042,2.62092) 101 +dictGet dict_array (34.9055,-0.0448967) 101 +dictGet dict_array (34.9122,0.110576) 101 +dictGet dict_array (34.9228,3.60183) 101 +dictGet dict_array (34.9237,1.21715) 101 +dictGet dict_array (34.9296,1.70459) 101 +dictGet dict_array (34.941,-1.14663) 101 +dictGet dict_array (34.9448,1.18923) 101 +dictGet dict_array (34.9462,3.81678) 101 +dictGet dict_array (34.9466,0.593463) 101 +dictGet dict_array (34.9485,0.150307) 101 +dictGet dict_array (34.9542,0.487238) 101 +dictGet dict_array (34.9559,2.03473) 101 +dictGet dict_array (34.9671,-0.960225) 101 +dictGet dict_array (34.9711,2.63444) 101 +dictGet dict_array (34.9892,0.354775) 101 +dictGet dict_array (34.9907,1.40724) 101 +dictGet dict_array (34.9916,-0.00173097) 101 +dictGet dict_array (34.9919,2.06167) 101 diff --git a/tests/queries/0_stateless/01037_polygon_dicts_correctness_all.sh b/tests/queries/0_stateless/01037_polygon_dicts_correctness_all.sh index c2a35a3ef63..fff786d6c06 100755 --- a/tests/queries/0_stateless/01037_polygon_dicts_correctness_all.sh +++ b/tests/queries/0_stateless/01037_polygon_dicts_correctness_all.sh @@ -1,5 +1,5 @@ #!/usr/bin/env bash -# Tags: no-debug, no-parallel +# Tags: no-debug CURDIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd) # shellcheck source=../shell_config.sh @@ -12,20 +12,18 @@ declare -a SearchTypes=("POLYGON" "POLYGON_SIMPLE" "POLYGON_INDEX_EACH" "POLYGON tar -xf "${CURDIR}"/01037_test_data_search.tar.gz -C "${CURDIR}" $CLICKHOUSE_CLIENT -n --query=" -DROP DATABASE IF EXISTS test_01037; -CREATE DATABASE test_01037; -DROP TABLE IF EXISTS test_01037.points; -CREATE TABLE test_01037.points (x Float64, y Float64) ENGINE = Memory; +DROP TABLE IF EXISTS points; +CREATE TABLE points (x Float64, y Float64) ENGINE = Memory; " -$CLICKHOUSE_CLIENT --query="INSERT INTO test_01037.points FORMAT TSV" --max_insert_block_size=100000 < "${CURDIR}/01037_point_data" +$CLICKHOUSE_CLIENT --query="INSERT INTO points FORMAT TSV" --max_insert_block_size=100000 < "${CURDIR}/01037_point_data" rm "${CURDIR}"/01037_point_data $CLICKHOUSE_CLIENT -n --query=" -DROP TABLE IF EXISTS test_01037.polygons_array; +DROP TABLE IF EXISTS polygons_array; -CREATE TABLE test_01037.polygons_array +CREATE TABLE polygons_array ( key Array(Array(Array(Array(Float64)))), name String, @@ -34,7 +32,7 @@ CREATE TABLE test_01037.polygons_array ENGINE = Memory; " -$CLICKHOUSE_CLIENT --query="INSERT INTO test_01037.polygons_array FORMAT JSONEachRow" --min_chunk_bytes_for_parallel_parsing=10485760 --max_insert_block_size=100000 < "${CURDIR}/01037_polygon_data" +$CLICKHOUSE_CLIENT --query="INSERT INTO polygons_array FORMAT JSONEachRow" --min_chunk_bytes_for_parallel_parsing=10485760 --max_insert_block_size=100000 < "${CURDIR}/01037_polygon_data" rm "${CURDIR}"/01037_polygon_data @@ -43,27 +41,22 @@ do outputFile="${TMP_DIR}/results${type}.out" $CLICKHOUSE_CLIENT -n --query=" - DROP DICTIONARY IF EXISTS test_01037.dict_array; + DROP DICTIONARY IF EXISTS dict_array; - CREATE DICTIONARY test_01037.dict_array + CREATE DICTIONARY dict_array ( key Array(Array(Array(Array(Float64)))), name String DEFAULT 'qqq', value UInt64 DEFAULT 101 ) PRIMARY KEY key - SOURCE(CLICKHOUSE(HOST 'localhost' PORT tcpPort() USER 'default' TABLE 'polygons_array' PASSWORD '' DB 'test_01037')) + SOURCE(CLICKHOUSE(HOST 'localhost' PORT tcpPort() USER 'default' TABLE 'polygons_array' PASSWORD '' DB currentDatabase())) LIFETIME(0) LAYOUT($type()); - select 'dictGet', 'test_01037.dict_array' as dict_name, tuple(x, y) as key, - dictGet(dict_name, 'value', key) from test_01037.points order by x, y; + select 'dictGet', 'dict_array' as dict_name, tuple(x, y) as key, + dictGet(dict_name, 'value', key) from points order by x, y; " > "$outputFile" diff -q "${CURDIR}/01037_polygon_dicts_correctness_all.ans" "$outputFile" done - -$CLICKHOUSE_CLIENT -n --query=" -DROP TABLE test_01037.points; -DROP DATABASE test_01037; -" diff --git a/tests/queries/0_stateless/01037_polygon_dicts_correctness_fast.ans b/tests/queries/0_stateless/01037_polygon_dicts_correctness_fast.ans index 45fa7637421..297c8416096 100644 --- a/tests/queries/0_stateless/01037_polygon_dicts_correctness_fast.ans +++ b/tests/queries/0_stateless/01037_polygon_dicts_correctness_fast.ans @@ -1,1000 +1,1000 @@ -dictGet test_01037.dict_array (29.5699,2.50068) 101 -dictGet test_01037.dict_array (29.5796,1.55456) 101 -dictGet test_01037.dict_array (29.5796,2.36864) 101 -dictGet test_01037.dict_array (29.5844,1.59626) 101 -dictGet test_01037.dict_array (29.5886,4.03321) 101 -dictGet test_01037.dict_array (29.5914,3.02628) 101 -dictGet test_01037.dict_array (29.5926,-0.0965169) 101 -dictGet test_01037.dict_array (29.5968,2.37773) 101 -dictGet test_01037.dict_array (29.5984,0.755853) 101 -dictGet test_01037.dict_array (29.6066,3.47173) 101 -dictGet test_01037.dict_array (29.6085,-1.26007) 6489978 -dictGet test_01037.dict_array (29.6131,0.246565) 101 -dictGet test_01037.dict_array (29.6157,-0.266687) 101 -dictGet test_01037.dict_array (29.6164,2.94674) 101 -dictGet test_01037.dict_array (29.6195,-0.591941) 101 -dictGet test_01037.dict_array (29.6231,1.54818) 101 -dictGet test_01037.dict_array (29.6379,0.764114) 101 -dictGet test_01037.dict_array (29.6462,-0.772059) 934530 -dictGet test_01037.dict_array (29.6579,-1.07336) 6489978 -dictGet test_01037.dict_array (29.6618,-0.271842) 101 -dictGet test_01037.dict_array (29.6629,-0.303602) 101 -dictGet test_01037.dict_array (29.6659,-0.782823) 934530 -dictGet test_01037.dict_array (29.6736,-0.113832) 101 -dictGet test_01037.dict_array (29.6759,3.02905) 101 -dictGet test_01037.dict_array (29.6778,3.71898) 101 -dictGet test_01037.dict_array (29.6796,1.10433) 101 -dictGet test_01037.dict_array (29.6809,2.13677) 101 -dictGet test_01037.dict_array (29.6935,4.11894) 101 -dictGet test_01037.dict_array (29.6991,-1.4458199999999999) 101 -dictGet test_01037.dict_array (29.6997,3.17297) 101 -dictGet test_01037.dict_array (29.7043,3.6145899999999997) 101 -dictGet test_01037.dict_array (29.7065,3.24885) 101 -dictGet test_01037.dict_array (29.7126,0.28108) 101 -dictGet test_01037.dict_array (29.7192,0.174273) 101 -dictGet test_01037.dict_array (29.7217,-0.523481) 3501900 -dictGet test_01037.dict_array (29.7271,1.67967) 101 -dictGet test_01037.dict_array (29.7311,4.12444) 101 -dictGet test_01037.dict_array (29.7347,1.88378) 101 -dictGet test_01037.dict_array (29.7358,0.67944) 101 -dictGet test_01037.dict_array (29.7366,-0.2973) 101 -dictGet test_01037.dict_array (29.7446,0.646536) 101 -dictGet test_01037.dict_array (29.7453,-0.567963) 3501900 -dictGet test_01037.dict_array (29.764,4.04217) 101 -dictGet test_01037.dict_array (29.7655,1.51372) 101 -dictGet test_01037.dict_array (29.7744,1.12435) 101 -dictGet test_01037.dict_array (29.7774,-0.0681196) 3501895 -dictGet test_01037.dict_array (29.7784,1.54864) 101 -dictGet test_01037.dict_array (29.7785,2.24139) 101 -dictGet test_01037.dict_array (29.7922,0.220808) 101 -dictGet test_01037.dict_array (29.7936,2.37709) 101 -dictGet test_01037.dict_array (29.8008,0.948536) 101 -dictGet test_01037.dict_array (29.8115,0.201227) 101 -dictGet test_01037.dict_array (29.814,0.149601) 3501895 -dictGet test_01037.dict_array (29.8193,-1.35858) 101 -dictGet test_01037.dict_array (29.8201,0.965518) 101 -dictGet test_01037.dict_array (29.8265,-0.727286) 3501900 -dictGet test_01037.dict_array (29.8277,-0.531746) 3501900 -dictGet test_01037.dict_array (29.8289,3.63009) 101 -dictGet test_01037.dict_array (29.8548,0.838047) 101 -dictGet test_01037.dict_array (29.8641,-0.845265) 3501900 -dictGet test_01037.dict_array (29.8649,0.0562212) 3501895 -dictGet test_01037.dict_array (29.8701,-1.02045) 934530 -dictGet test_01037.dict_array (29.8733,2.76654) 101 -dictGet test_01037.dict_array (29.876,0.555475) 101 -dictGet test_01037.dict_array (29.8794,-0.800108) 3501900 -dictGet test_01037.dict_array (29.8813,2.7426399999999997) 101 -dictGet test_01037.dict_array (29.897100000000002,2.66193) 101 -dictGet test_01037.dict_array (29.908,4.01339) 101 -dictGet test_01037.dict_array (29.9165,-1.08246) 3501894 -dictGet test_01037.dict_array (29.9201,-0.420861) 3498054 -dictGet test_01037.dict_array (29.9217,3.03778) 101 -dictGet test_01037.dict_array (29.9355,0.773833) 101 -dictGet test_01037.dict_array (29.947,3.76517) 101 -dictGet test_01037.dict_array (29.9518,-0.60557) 3498056 -dictGet test_01037.dict_array (29.9564,-0.600163) 3498056 -dictGet test_01037.dict_array (29.959600000000002,4.16591) 101 -dictGet test_01037.dict_array (29.9615,-1.33708) 3501894 -dictGet test_01037.dict_array (29.9699,-0.392375) 3498054 -dictGet test_01037.dict_array (29.9776,1.04552) 101 -dictGet test_01037.dict_array (29.9784,4.02756) 101 -dictGet test_01037.dict_array (29.9819,4.00597) 101 -dictGet test_01037.dict_array (29.9826,1.2816100000000001) 101 -dictGet test_01037.dict_array (30.0026,2.76257) 101 -dictGet test_01037.dict_array (30.0126,3.68255) 101 -dictGet test_01037.dict_array (30.0131,0.796576) 3501892 -dictGet test_01037.dict_array (30.018,1.16523) 101 -dictGet test_01037.dict_array (30.0261,-0.210653) 3501896 -dictGet test_01037.dict_array (30.0472,-1.11007) 3501894 -dictGet test_01037.dict_array (30.0542,-0.479585) 3498054 -dictGet test_01037.dict_array (30.0613,1.6278000000000001) 101 -dictGet test_01037.dict_array (30.0617,-0.0551152) 3501895 -dictGet test_01037.dict_array (30.0637,2.62066) 101 -dictGet test_01037.dict_array (30.0721,1.6424400000000001) 101 -dictGet test_01037.dict_array (30.0769,-0.402636) 3498054 -dictGet test_01037.dict_array (30.0791,-0.277435) 3501896 -dictGet test_01037.dict_array (30.0931,0.0327512) 3501895 -dictGet test_01037.dict_array (30.1059,3.52623) 101 -dictGet test_01037.dict_array (30.1103,0.865466) 3501892 -dictGet test_01037.dict_array (30.1115,2.95243) 101 -dictGet test_01037.dict_array (30.1144,1.71029) 101 -dictGet test_01037.dict_array (30.1311,-0.864751) 3501899 -dictGet test_01037.dict_array (30.1336,-0.851386) 3501899 -dictGet test_01037.dict_array (30.1393,3.89901) 101 -dictGet test_01037.dict_array (30.1456,-0.531898) 3498054 -dictGet test_01037.dict_array (30.1492,2.07833) 101 -dictGet test_01037.dict_array (30.1575,2.43856) 101 -dictGet test_01037.dict_array (30.1682,1.19771) 101 -dictGet test_01037.dict_array (30.1716,3.9853300000000003) 101 -dictGet test_01037.dict_array (30.1849,2.78374) 101 -dictGet test_01037.dict_array (30.1866,0.65658) 3498021 -dictGet test_01037.dict_array (30.1885,1.56943) 101 -dictGet test_01037.dict_array (30.1959,-1.38202) 101 -dictGet test_01037.dict_array (30.1999,1.58413) 101 -dictGet test_01037.dict_array (30.2024,0.713081) 3498021 -dictGet test_01037.dict_array (30.2054,0.620143) 3498021 -dictGet test_01037.dict_array (30.2091,1.51641) 101 -dictGet test_01037.dict_array (30.2124,-0.331782) 3498031 -dictGet test_01037.dict_array (30.226,3.03527) 101 -dictGet test_01037.dict_array (30.2261,3.18486) 101 -dictGet test_01037.dict_array (30.2288,2.48407) 101 -dictGet test_01037.dict_array (30.2345,3.7462400000000002) 101 -dictGet test_01037.dict_array (30.2375,0.62046) 3498021 -dictGet test_01037.dict_array (30.2425,-0.472914) 3498054 -dictGet test_01037.dict_array (30.247,3.95863) 101 -dictGet test_01037.dict_array (30.2494,-0.305093) 3498031 -dictGet test_01037.dict_array (30.2499,2.54337) 101 -dictGet test_01037.dict_array (30.2606,2.16644) 101 -dictGet test_01037.dict_array (30.2672,3.94847) 101 -dictGet test_01037.dict_array (30.2709,-0.136264) 6088794 -dictGet test_01037.dict_array (30.2764,1.18654) 101 -dictGet test_01037.dict_array (30.2765,1.20383) 101 -dictGet test_01037.dict_array (30.2839,1.05762) 3498024 -dictGet test_01037.dict_array (30.286,0.469327) 3498021 -dictGet test_01037.dict_array (30.2927,3.1693) 101 -dictGet test_01037.dict_array (30.2935,3.49854) 101 -dictGet test_01037.dict_array (30.307,0.312338) 3498021 -dictGet test_01037.dict_array (30.3085,1.07791) 3498024 -dictGet test_01037.dict_array (30.3139,2.77248) 101 -dictGet test_01037.dict_array (30.314,0.822823) 3498024 -dictGet test_01037.dict_array (30.3227,-0.587351) 3498055 -dictGet test_01037.dict_array (30.332,1.00174) 3498024 -dictGet test_01037.dict_array (30.3388,0.844148) 3498024 -dictGet test_01037.dict_array (30.3485,0.561902) 3498021 -dictGet test_01037.dict_array (30.3497,0.180362) 6489998 -dictGet test_01037.dict_array (30.361,4.13016) 101 -dictGet test_01037.dict_array (30.3623,-0.0484027) 6489998 -dictGet test_01037.dict_array (30.3638,3.9845800000000002) 101 -dictGet test_01037.dict_array (30.3853,3.16051) 101 -dictGet test_01037.dict_array (30.3974,2.6617800000000003) 101 -dictGet test_01037.dict_array (30.4002,-1.15886) 101 -dictGet test_01037.dict_array (30.4008,-0.387015) 3498031 -dictGet test_01037.dict_array (30.4018,1.86493) 101 -dictGet test_01037.dict_array (30.4239,1.16818) 3498024 -dictGet test_01037.dict_array (30.4363,3.63938) 101 -dictGet test_01037.dict_array (30.4377,-0.81315) 3498063 -dictGet test_01037.dict_array (30.4391,3.54703) 101 -dictGet test_01037.dict_array (30.4424,-1.39435) 101 -dictGet test_01037.dict_array (30.4441,2.8463000000000003) 101 -dictGet test_01037.dict_array (30.4517,3.28117) 101 -dictGet test_01037.dict_array (30.4658,2.6928) 101 -dictGet test_01037.dict_array (30.4734,2.66161) 101 -dictGet test_01037.dict_array (30.4799,-1.07578) 101 -dictGet test_01037.dict_array (30.4837,-1.02486) 3501899 -dictGet test_01037.dict_array (30.485,1.06326) 3498024 -dictGet test_01037.dict_array (30.495,1.12306) 101 -dictGet test_01037.dict_array (30.501,2.27264) 101 -dictGet test_01037.dict_array (30.5027,1.99382) 101 -dictGet test_01037.dict_array (30.5194,-1.03943) 3501893 -dictGet test_01037.dict_array (30.5239,1.04328) 101 -dictGet test_01037.dict_array (30.528,3.82041) 101 -dictGet test_01037.dict_array (30.5299,-0.715248) 3498063 -dictGet test_01037.dict_array (30.5331,1.19603) 101 -dictGet test_01037.dict_array (30.535800000000002,2.71485) 101 -dictGet test_01037.dict_array (30.5405,0.804694) 3498023 -dictGet test_01037.dict_array (30.542,1.23739) 101 -dictGet test_01037.dict_array (30.5432,4.04189) 101 -dictGet test_01037.dict_array (30.5457,-0.956121) 3501893 -dictGet test_01037.dict_array (30.5506,3.07443) 101 -dictGet test_01037.dict_array (30.5539,3.87084) 101 -dictGet test_01037.dict_array (30.5578,3.78837) 101 -dictGet test_01037.dict_array (30.5588,0.966135) 3498022 -dictGet test_01037.dict_array (30.5637,2.5605) 101 -dictGet test_01037.dict_array (30.5647,-1.27328) 101 -dictGet test_01037.dict_array (30.5656,-0.0581332) 6088794 -dictGet test_01037.dict_array (30.5715,0.65755) 3498023 -dictGet test_01037.dict_array (30.5727,3.01604) 101 -dictGet test_01037.dict_array (30.5729,-0.976857) 3501893 -dictGet test_01037.dict_array (30.5751,0.60204) 3498023 -dictGet test_01037.dict_array (30.5854,3.02473) 101 -dictGet test_01037.dict_array (30.5866,0.174099) 6489998 -dictGet test_01037.dict_array (30.5947,0.875193) 3498023 -dictGet test_01037.dict_array (30.5992,-0.403901) 3498063 -dictGet test_01037.dict_array (30.6002,4.18891) 101 -dictGet test_01037.dict_array (30.6025,0.217712) 6489998 -dictGet test_01037.dict_array (30.6054,0.927203) 3498022 -dictGet test_01037.dict_array (30.6075,3.79359) 101 -dictGet test_01037.dict_array (30.6159,3.82773) 101 -dictGet test_01037.dict_array (30.627,3.84039) 101 -dictGet test_01037.dict_array (30.6308,0.77517) 3498023 -dictGet test_01037.dict_array (30.6338,0.179565) 6489998 -dictGet test_01037.dict_array (30.6461,1.3293599999999999) 101 -dictGet test_01037.dict_array (30.6674,-0.424547) 3498063 -dictGet test_01037.dict_array (30.669,1.76539) 101 -dictGet test_01037.dict_array (30.6788,4.01239) 101 -dictGet test_01037.dict_array (30.6864,3.59158) 101 -dictGet test_01037.dict_array (30.7049,-0.875413) 3501893 -dictGet test_01037.dict_array (30.705,1.3307) 101 -dictGet test_01037.dict_array (30.7063,-0.473192) 3498063 -dictGet test_01037.dict_array (30.7075,-1.1958199999999999) 101 -dictGet test_01037.dict_array (30.7101,-0.367562) 3498012 -dictGet test_01037.dict_array (30.7203,2.98725) 101 -dictGet test_01037.dict_array (30.7213,2.2745699999999998) 101 -dictGet test_01037.dict_array (30.7446,-0.334144) 3498012 -dictGet test_01037.dict_array (30.7468,3.82967) 101 -dictGet test_01037.dict_array (30.747,-0.384779) 3498012 -dictGet test_01037.dict_array (30.7681,0.904198) 3498022 -dictGet test_01037.dict_array (30.7757,1.78743) 101 -dictGet test_01037.dict_array (30.8021,-0.479212) 3498012 -dictGet test_01037.dict_array (30.8079,-1.40869) 101 -dictGet test_01037.dict_array (30.8206,-0.0608489) 3498012 -dictGet test_01037.dict_array (30.8218,0.43909) 3498023 -dictGet test_01037.dict_array (30.8239,0.10014) 3498012 -dictGet test_01037.dict_array (30.8282,4.15409) 101 -dictGet test_01037.dict_array (30.8288,-0.709528) 3501893 -dictGet test_01037.dict_array (30.8326,0.156011) 3498012 -dictGet test_01037.dict_array (30.8328,-1.03704) 101 -dictGet test_01037.dict_array (30.839,2.15528) 101 -dictGet test_01037.dict_array (30.8452,0.219377) 3498013 -dictGet test_01037.dict_array (30.8463,0.0515355) 3498012 -dictGet test_01037.dict_array (30.8526,2.06614) 101 -dictGet test_01037.dict_array (30.8566,0.517876) 3498023 -dictGet test_01037.dict_array (30.8588,-1.31738) 101 -dictGet test_01037.dict_array (30.8681,0.44207) 3498013 -dictGet test_01037.dict_array (30.8914,1.0072) 3498022 -dictGet test_01037.dict_array (30.897,0.483425) 3498013 -dictGet test_01037.dict_array (30.905,2.8731999999999998) 3501793 -dictGet test_01037.dict_array (30.9051,2.21956) 101 -dictGet test_01037.dict_array (30.9115,4.00663) 101 -dictGet test_01037.dict_array (30.9167,-0.834462) 3501893 -dictGet test_01037.dict_array (30.9252,-1.3289900000000001) 101 -dictGet test_01037.dict_array (30.9314,1.85384) 101 -dictGet test_01037.dict_array (30.9392,2.53236) 3501827 -dictGet test_01037.dict_array (30.9569,2.82038) 3501793 -dictGet test_01037.dict_array (30.9598,-0.641011) 3498012 -dictGet test_01037.dict_array (30.9601,-0.254928) 3498012 -dictGet test_01037.dict_array (30.9623,-1.3886) 101 -dictGet test_01037.dict_array (30.9707,0.888854) 3498022 -dictGet test_01037.dict_array (30.9766,2.81957) 3501793 -dictGet test_01037.dict_array (30.9775,2.69273) 3501793 -dictGet test_01037.dict_array (30.9821,0.587715) 3498013 -dictGet test_01037.dict_array (30.9887,4.0233) 101 -dictGet test_01037.dict_array (30.9914,0.259542) 3498013 -dictGet test_01037.dict_array (30.9986,-1.36832) 101 -dictGet test_01037.dict_array (31.008,0.628999) 3498013 -dictGet test_01037.dict_array (31.0168,-1.17462) 101 -dictGet test_01037.dict_array (31.0237,3.52547) 3501821 -dictGet test_01037.dict_array (31.0306,3.78522) 101 -dictGet test_01037.dict_array (31.0308,-0.72453) 3501893 -dictGet test_01037.dict_array (31.0463,2.41997) 3501825 -dictGet test_01037.dict_array (31.047,0.624184) 3498013 -dictGet test_01037.dict_array (31.0569,0.0706393) 3498015 -dictGet test_01037.dict_array (31.0583,1.3244099999999999) 3501926 -dictGet test_01037.dict_array (31.063,3.23861) 3501793 -dictGet test_01037.dict_array (31.068,0.695575) 3498022 -dictGet test_01037.dict_array (31.0687,1.85675) 101 -dictGet test_01037.dict_array (31.0692,0.254793) 3498014 -dictGet test_01037.dict_array (31.0766,0.828128) 3498022 -dictGet test_01037.dict_array (31.0833,0.0612782) 3498015 -dictGet test_01037.dict_array (31.0833,2.59748) 3501793 -dictGet test_01037.dict_array (31.0861,-1.3778299999999999) 101 -dictGet test_01037.dict_array (31.0874,3.07258) 3501793 -dictGet test_01037.dict_array (31.0882,1.4882) 3501926 -dictGet test_01037.dict_array (31.0924,3.42242) 3501821 -dictGet test_01037.dict_array (31.0927,2.67448) 3501793 -dictGet test_01037.dict_array (31.0936,1.12292) 3498022 -dictGet test_01037.dict_array (31.0952,-0.336928) 3498012 -dictGet test_01037.dict_array (31.0978,3.48482) 3501826 -dictGet test_01037.dict_array (31.1107,3.7513199999999998) 3501826 -dictGet test_01037.dict_array (31.1156,1.19171) 3501926 -dictGet test_01037.dict_array (31.1176,0.223509) 3498015 -dictGet test_01037.dict_array (31.1249,0.946838) 3498022 -dictGet test_01037.dict_array (31.1267,1.48983) 3501926 -dictGet test_01037.dict_array (31.138,-0.289981) 3501898 -dictGet test_01037.dict_array (31.1382,3.02904) 3501793 -dictGet test_01037.dict_array (31.1475,2.6178) 3501793 -dictGet test_01037.dict_array (31.1491,1.37873) 3501926 -dictGet test_01037.dict_array (31.1525,3.72105) 3501826 -dictGet test_01037.dict_array (31.1526,-1.4129800000000001) 101 -dictGet test_01037.dict_array (31.1526,-0.186457) 3501898 -dictGet test_01037.dict_array (31.1539,2.78789) 3501793 -dictGet test_01037.dict_array (31.1548,-1.08552) 101 -dictGet test_01037.dict_array (31.1567,-0.0768925) 3501898 -dictGet test_01037.dict_array (31.1613,1.49617) 3501926 -dictGet test_01037.dict_array (31.1653,1.03777) 3498022 -dictGet test_01037.dict_array (31.1662,3.4214700000000002) 3501826 -dictGet test_01037.dict_array (31.1672,-0.0813169) 3501898 -dictGet test_01037.dict_array (31.177,0.440843) 3498014 -dictGet test_01037.dict_array (31.1788,-0.737151) 3501893 -dictGet test_01037.dict_array (31.1856,-0.144396) 3501898 -dictGet test_01037.dict_array (31.1959,3.66813) 3501826 -dictGet test_01037.dict_array (31.1996,-0.353983) 3501898 -dictGet test_01037.dict_array (31.2019,2.86802) 3501793 -dictGet test_01037.dict_array (31.2087,2.31245) 3501825 -dictGet test_01037.dict_array (31.2125,3.2713200000000002) 3501793 -dictGet test_01037.dict_array (31.2137,-0.108129) 3501898 -dictGet test_01037.dict_array (31.216,3.9156) 101 -dictGet test_01037.dict_array (31.2201,-0.202141) 3501898 -dictGet test_01037.dict_array (31.2285,2.09058) 101 -dictGet test_01037.dict_array (31.2502,4.01526) 101 -dictGet test_01037.dict_array (31.2585,3.11524) 3501793 -dictGet test_01037.dict_array (31.2645,-0.620418) 3501890 -dictGet test_01037.dict_array (31.2684,2.74277) 3501793 -dictGet test_01037.dict_array (31.2821,-1.12772) 101 -dictGet test_01037.dict_array (31.2821,2.46769) 3501825 -dictGet test_01037.dict_array (31.2887,3.91396) 101 -dictGet test_01037.dict_array (31.295,1.49942) 3501926 -dictGet test_01037.dict_array (31.2997,3.46122) 3501826 -dictGet test_01037.dict_array (31.3017,3.3263) 3501826 -dictGet test_01037.dict_array (31.3022,3.16754) 3501793 -dictGet test_01037.dict_array (31.3048,0.364962) 3498014 -dictGet test_01037.dict_array (31.305,3.1967) 3501793 -dictGet test_01037.dict_array (31.3061,1.84303) 101 -dictGet test_01037.dict_array (31.3082,-0.173851) 3501898 -dictGet test_01037.dict_array (31.3315,3.90932) 101 -dictGet test_01037.dict_array (31.3351,2.80164) 3501793 -dictGet test_01037.dict_array (31.3388,0.168765) 3498015 -dictGet test_01037.dict_array (31.339,0.25535) 3498094 -dictGet test_01037.dict_array (31.3423,1.7036799999999999) 3501926 -dictGet test_01037.dict_array (31.349,0.386456) 3498014 -dictGet test_01037.dict_array (31.3558,-1.04336) 101 -dictGet test_01037.dict_array (31.3564,0.478876) 3498014 -dictGet test_01037.dict_array (31.3607,-0.0860507) 3498015 -dictGet test_01037.dict_array (31.3831,3.84469) 101 -dictGet test_01037.dict_array (31.3886,-0.731137) 3501890 -dictGet test_01037.dict_array (31.4043,-0.348907) 5457271 -dictGet test_01037.dict_array (31.4081,1.47391) 3501926 -dictGet test_01037.dict_array (31.4176,-0.583645) 5457271 -dictGet test_01037.dict_array (31.4177,1.36972) 3501926 -dictGet test_01037.dict_array (31.4182,0.958303) 3498022 -dictGet test_01037.dict_array (31.4199,3.1738) 3501793 -dictGet test_01037.dict_array (31.4221,2.74876) 3501825 -dictGet test_01037.dict_array (31.4301,-0.122643) 3498015 -dictGet test_01037.dict_array (31.4344,1.00661) 3498022 -dictGet test_01037.dict_array (31.4375,4.20304) 101 -dictGet test_01037.dict_array (31.4377,0.289608) 3498094 -dictGet test_01037.dict_array (31.4379,0.54744) 3498014 -dictGet test_01037.dict_array (31.4459,3.94945) 101 -dictGet test_01037.dict_array (31.4559,-0.345063) 5457271 -dictGet test_01037.dict_array (31.464,0.726129) 3498014 -dictGet test_01037.dict_array (31.4662,-0.299019) 3498015 -dictGet test_01037.dict_array (31.4671,1.9605299999999999) 3501794 -dictGet test_01037.dict_array (31.4673,-0.403676) 5457271 -dictGet test_01037.dict_array (31.4712,-0.237941) 3498015 -dictGet test_01037.dict_array (31.4816,0.120264) 3498015 -dictGet test_01037.dict_array (31.4875,0.323483) 3498014 -dictGet test_01037.dict_array (31.490099999999998,-0.338163) 5457271 -dictGet test_01037.dict_array (31.4932,0.517674) 3498014 -dictGet test_01037.dict_array (31.5112,1.9689299999999998) 3501794 -dictGet test_01037.dict_array (31.5122,2.92785) 3501791 -dictGet test_01037.dict_array (31.5151,0.166429) 3498094 -dictGet test_01037.dict_array (31.5174,2.94802) 3501791 -dictGet test_01037.dict_array (31.5182,4.18776) 101 -dictGet test_01037.dict_array (31.5238,1.18793) 3498003 -dictGet test_01037.dict_array (31.5271,3.07446) 3501791 -dictGet test_01037.dict_array (31.5393,1.58061) 3501794 -dictGet test_01037.dict_array (31.5421,3.13711) 3501791 -dictGet test_01037.dict_array (31.5479,2.39897) 3497970 -dictGet test_01037.dict_array (31.5519,0.99285) 3498003 -dictGet test_01037.dict_array (31.5685,3.47987) 3501824 -dictGet test_01037.dict_array (31.5959,0.437382) 3498014 -dictGet test_01037.dict_array (31.6003,0.194376) 3498094 -dictGet test_01037.dict_array (31.6026,2.15457) 3501794 -dictGet test_01037.dict_array (31.606,2.45365) 3497970 -dictGet test_01037.dict_array (31.6062,-0.453441) 3501890 -dictGet test_01037.dict_array (31.6107,1.35247) 3497974 -dictGet test_01037.dict_array (31.6155,3.85588) 101 -dictGet test_01037.dict_array (31.6222,2.03326) 3501794 -dictGet test_01037.dict_array (31.6231,-0.123059) 3498083 -dictGet test_01037.dict_array (31.6244,1.6885599999999998) 3497974 -dictGet test_01037.dict_array (31.6459,0.669716) 3498014 -dictGet test_01037.dict_array (31.6563,-0.0644741) 3498083 -dictGet test_01037.dict_array (31.6618,-0.551121) 3501890 -dictGet test_01037.dict_array (31.6725,-0.38922) 3498085 -dictGet test_01037.dict_array (31.6727,4.10336) 101 -dictGet test_01037.dict_array (31.6739,4.1391) 101 -dictGet test_01037.dict_array (31.6897,2.8694699999999997) 3501792 -dictGet test_01037.dict_array (31.6902,3.98792) 101 -dictGet test_01037.dict_array (31.6945,2.46687) 3497970 -dictGet test_01037.dict_array (31.6987,-1.3796) 101 -dictGet test_01037.dict_array (31.7012,2.34845) 3497970 -dictGet test_01037.dict_array (31.7036,0.0228348) 3501888 -dictGet test_01037.dict_array (31.7046,3.68111) 3501824 -dictGet test_01037.dict_array (31.7055,2.92556) 3501792 -dictGet test_01037.dict_array (31.7102,1.04532) 3498003 -dictGet test_01037.dict_array (31.7149,-0.443302) 3498085 -dictGet test_01037.dict_array (31.7195,2.99311) 3501791 -dictGet test_01037.dict_array (31.7274,0.166719) 3498094 -dictGet test_01037.dict_array (31.7565,-0.565382) 3498085 -dictGet test_01037.dict_array (31.7615,0.771626) 3498014 -dictGet test_01037.dict_array (31.7739,1.8970099999999999) 3497974 -dictGet test_01037.dict_array (31.7848,1.2623199999999999) 3498003 -dictGet test_01037.dict_array (31.7912,-0.788599) 101 -dictGet test_01037.dict_array (31.8011,2.65853) 3497970 -dictGet test_01037.dict_array (31.8032,-0.0590108) 3501888 -dictGet test_01037.dict_array (31.8038,1.9618799999999998) 3497974 -dictGet test_01037.dict_array (31.8098,-1.46851) 101 -dictGet test_01037.dict_array (31.8131,3.41982) 3501791 -dictGet test_01037.dict_array (31.8169,3.31059) 3501791 -dictGet test_01037.dict_array (31.8202,-0.193692) 3501888 -dictGet test_01037.dict_array (31.8306,1.57586) 3497974 -dictGet test_01037.dict_array (31.8382,-0.787948) 101 -dictGet test_01037.dict_array (31.8433,2.49692) 3497970 -dictGet test_01037.dict_array (31.8436,2.41851) 3497970 -dictGet test_01037.dict_array (31.8563,-1.10787) 101 -dictGet test_01037.dict_array (31.8683,0.996504) 3498002 -dictGet test_01037.dict_array (31.8693,-0.828142) 101 -dictGet test_01037.dict_array (31.8723,1.08929) 3498003 -dictGet test_01037.dict_array (31.8737,0.881127) 3498002 -dictGet test_01037.dict_array (31.8881,-0.58441) 101 -dictGet test_01037.dict_array (31.9011,0.121349) 3498094 -dictGet test_01037.dict_array (31.9066,2.13045) 3497965 -dictGet test_01037.dict_array (31.9142,1.03368) 3498002 -dictGet test_01037.dict_array (31.9155,3.38363) 3501791 -dictGet test_01037.dict_array (31.9168,1.3166) 3498004 -dictGet test_01037.dict_array (31.9185,-1.11879) 101 -dictGet test_01037.dict_array (31.9186,-0.647948) 101 -dictGet test_01037.dict_array (31.9311,3.96928) 101 -dictGet test_01037.dict_array (31.9335,1.47048) 3497974 -dictGet test_01037.dict_array (31.9443,-1.36175) 101 -dictGet test_01037.dict_array (31.9481,2.34231) 3497970 -dictGet test_01037.dict_array (31.9526,1.36565) 3498004 -dictGet test_01037.dict_array (31.9629,2.5208399999999997) 3497970 -dictGet test_01037.dict_array (31.9765,0.975783) 3498002 -dictGet test_01037.dict_array (31.9923,3.31773) 3501791 -dictGet test_01037.dict_array (31.9994,0.972816) 3498002 -dictGet test_01037.dict_array (32.001,3.47425) 3501791 -dictGet test_01037.dict_array (32.0127,2.13874) 3497965 -dictGet test_01037.dict_array (32.0244,3.2092) 3501792 -dictGet test_01037.dict_array (32.029,1.18039) 3498004 -dictGet test_01037.dict_array (32.0315,0.566073) 3498095 -dictGet test_01037.dict_array (32.0354,1.0766499999999999) 3498004 -dictGet test_01037.dict_array (32.0399,-1.11576) 101 -dictGet test_01037.dict_array (32.053,2.16849) 3497965 -dictGet test_01037.dict_array (32.0542,0.042328) 3498096 -dictGet test_01037.dict_array (32.0576,2.47001) 3497970 -dictGet test_01037.dict_array (32.061,3.7498899999999997) 101 -dictGet test_01037.dict_array (32.0623,1.25134) 3498004 -dictGet test_01037.dict_array (32.0626,1.9611399999999999) 3497965 -dictGet test_01037.dict_array (32.0666,-0.0904247) 3498096 -dictGet test_01037.dict_array (32.0681,2.28442) 3497970 -dictGet test_01037.dict_array (32.0692,1.50869) 3497981 -dictGet test_01037.dict_array (32.0724,4.03314) 101 -dictGet test_01037.dict_array (32.0729,-0.064324) 101 -dictGet test_01037.dict_array (32.079,0.293758) 3498094 -dictGet test_01037.dict_array (32.0847,-1.19814) 101 -dictGet test_01037.dict_array (32.0974,-0.91927) 101 -dictGet test_01037.dict_array (32.0979,-0.736979) 101 -dictGet test_01037.dict_array (32.106,-1.33063) 101 -dictGet test_01037.dict_array (32.1189,0.246715) 3498094 -dictGet test_01037.dict_array (32.1207,4.00883) 101 -dictGet test_01037.dict_array (32.1396,1.12402) 3498004 -dictGet test_01037.dict_array (32.1413,1.5668) 3497981 -dictGet test_01037.dict_array (32.143,1.35559) 3498004 -dictGet test_01037.dict_array (32.1538,1.32881) 3498004 -dictGet test_01037.dict_array (32.1549,4.06552) 101 -dictGet test_01037.dict_array (32.1555,-0.79275) 101 -dictGet test_01037.dict_array (32.163,1.17733) 3498004 -dictGet test_01037.dict_array (32.1634,2.94273) 3501792 -dictGet test_01037.dict_array (32.1644,1.85666) 3497965 -dictGet test_01037.dict_array (32.1745,0.435458) 3498095 -dictGet test_01037.dict_array (32.1765,1.65149) 3497981 -dictGet test_01037.dict_array (32.1893,2.08924) 3497965 -dictGet test_01037.dict_array (32.2024,0.222191) 3498093 -dictGet test_01037.dict_array (32.2107,1.34379) 3497981 -dictGet test_01037.dict_array (32.2109,3.9018699999999997) 101 -dictGet test_01037.dict_array (32.2123,1.85233) 3497965 -dictGet test_01037.dict_array (32.2144,3.72534) 101 -dictGet test_01037.dict_array (32.2218,2.5386699999999998) 3497970 -dictGet test_01037.dict_array (32.2279,2.84267) 3497245 -dictGet test_01037.dict_array (32.2345,3.33295) 3501792 -dictGet test_01037.dict_array (32.2435,3.85283) 101 -dictGet test_01037.dict_array (32.2527,-0.480608) 101 -dictGet test_01037.dict_array (32.2566,-0.837882) 101 -dictGet test_01037.dict_array (32.2627,2.57708) 3497970 -dictGet test_01037.dict_array (32.2733,0.244931) 3498096 -dictGet test_01037.dict_array (32.2761,4.05808) 101 -dictGet test_01037.dict_array (32.2764,3.78472) 101 -dictGet test_01037.dict_array (32.2814,-1.26011) 101 -dictGet test_01037.dict_array (32.2861,3.02427) 3497245 -dictGet test_01037.dict_array (32.2924,0.928609) 3498004 -dictGet test_01037.dict_array (32.2963,-0.78543) 101 -dictGet test_01037.dict_array (32.3039,3.21175) 3501792 -dictGet test_01037.dict_array (32.3107,0.698287) 3498004 -dictGet test_01037.dict_array (32.3138,0.0595677) 3498106 -dictGet test_01037.dict_array (32.3339,0.707056) 3498004 -dictGet test_01037.dict_array (32.3351,0.415474) 3498106 -dictGet test_01037.dict_array (32.342,-0.681023) 101 -dictGet test_01037.dict_array (32.3463,1.83196) 3497126 -dictGet test_01037.dict_array (32.3494,2.43799) 3497114 -dictGet test_01037.dict_array (32.3524,3.47049) 3501822 -dictGet test_01037.dict_array (32.3531,2.33115) 3497114 -dictGet test_01037.dict_array (32.3602,0.116106) 3498106 -dictGet test_01037.dict_array (32.3612,1.1598) 3498004 -dictGet test_01037.dict_array (32.3689,3.34847) 3501822 -dictGet test_01037.dict_array (32.3695,0.734055) 3498004 -dictGet test_01037.dict_array (32.3825,3.85017) 101 -dictGet test_01037.dict_array (32.3835,-1.25491) 101 -dictGet test_01037.dict_array (32.4018,-0.728568) 101 -dictGet test_01037.dict_array (32.4044,2.96727) 3497245 -dictGet test_01037.dict_array (32.4101,2.9988) 3497245 -dictGet test_01037.dict_array (32.417,-1.12908) 101 -dictGet test_01037.dict_array (32.4172,4.1952) 101 -dictGet test_01037.dict_array (32.4239,2.49512) 3497245 -dictGet test_01037.dict_array (32.4258,4.05137) 101 -dictGet test_01037.dict_array (32.4264,-0.427357) 101 -dictGet test_01037.dict_array (32.4274,3.59377) 3501822 -dictGet test_01037.dict_array (32.4286,-1.24757) 101 -dictGet test_01037.dict_array (32.4294,3.0665) 3497245 -dictGet test_01037.dict_array (32.4333,-0.353347) 101 -dictGet test_01037.dict_array (32.4391,3.64421) 3501822 -dictGet test_01037.dict_array (32.4401,3.70635) 3501822 -dictGet test_01037.dict_array (32.45,1.68918) 3497126 -dictGet test_01037.dict_array (32.4507,-0.133471) 101 -dictGet test_01037.dict_array (32.4592,0.976458) 3498105 -dictGet test_01037.dict_array (32.4595,1.89135) 3497126 -dictGet test_01037.dict_array (32.4604,0.280248) 3498106 -dictGet test_01037.dict_array (32.4835,0.472731) 3498106 -dictGet test_01037.dict_array (32.4855,2.01938) 3497126 -dictGet test_01037.dict_array (32.4872,2.01697) 3497126 -dictGet test_01037.dict_array (32.4911,0.613106) 3498105 -dictGet test_01037.dict_array (32.4918,2.17834) 3497114 -dictGet test_01037.dict_array (32.4947,2.34595) 3497114 -dictGet test_01037.dict_array (32.5035,2.92234) 3497245 -dictGet test_01037.dict_array (32.5132,-0.331206) 101 -dictGet test_01037.dict_array (32.5156,-0.412604) 3501887 -dictGet test_01037.dict_array (32.5158,2.9067499999999997) 3497245 -dictGet test_01037.dict_array (32.5249,2.44519) 3497114 -dictGet test_01037.dict_array (32.5293,-0.790952) 101 -dictGet test_01037.dict_array (32.5319,3.96854) 101 -dictGet test_01037.dict_array (32.5518,3.6093) 3501822 -dictGet test_01037.dict_array (32.5541,3.5225400000000002) 3501822 -dictGet test_01037.dict_array (32.5569,0.816123) 3498105 -dictGet test_01037.dict_array (32.5646,1.9775) 3497126 -dictGet test_01037.dict_array (32.5733,3.81271) 101 -dictGet test_01037.dict_array (32.5767,0.948327) 3498105 -dictGet test_01037.dict_array (32.5971,1.76179) 3497126 -dictGet test_01037.dict_array (32.6035,-0.716157) 101 -dictGet test_01037.dict_array (32.6087,4.21614) 101 -dictGet test_01037.dict_array (32.6171,0.024481) 101 -dictGet test_01037.dict_array (32.6189,-0.775391) 101 -dictGet test_01037.dict_array (32.6198,2.92081) 3497167 -dictGet test_01037.dict_array (32.621,-0.970784) 101 -dictGet test_01037.dict_array (32.6266,0.650009) 3498105 -dictGet test_01037.dict_array (32.6315,2.15144) 3497126 -dictGet test_01037.dict_array (32.6385,-0.436803) 101 -dictGet test_01037.dict_array (32.6449,-0.191292) 101 -dictGet test_01037.dict_array (32.6535,2.10385) 3497126 -dictGet test_01037.dict_array (32.6592,3.49973) 3501822 -dictGet test_01037.dict_array (32.6598,2.5980600000000003) 3497114 -dictGet test_01037.dict_array (32.6612,2.95681) 3497167 -dictGet test_01037.dict_array (32.6636,-0.57235) 101 -dictGet test_01037.dict_array (32.669,-0.382702) 101 -dictGet test_01037.dict_array (32.6752,1.30748) 3497981 -dictGet test_01037.dict_array (32.6811,2.9559800000000003) 3497167 -dictGet test_01037.dict_array (32.6821,0.57336) 3498105 -dictGet test_01037.dict_array (32.6828,3.91304) 101 -dictGet test_01037.dict_array (32.6979,3.96868) 101 -dictGet test_01037.dict_array (32.6983,3.15784) 3497167 -dictGet test_01037.dict_array (32.7122,0.794293) 3498105 -dictGet test_01037.dict_array (32.7131,-0.847256) 101 -dictGet test_01037.dict_array (32.7219,0.883461) 3498105 -dictGet test_01037.dict_array (32.7228,1.78808) 3497126 -dictGet test_01037.dict_array (32.7273,-0.206908) 101 -dictGet test_01037.dict_array (32.7292,0.259331) 3501889 -dictGet test_01037.dict_array (32.7304,-1.38317) 101 -dictGet test_01037.dict_array (32.7353,1.01601) 3498105 -dictGet test_01037.dict_array (32.7354,4.17574) 101 -dictGet test_01037.dict_array (32.7357,-0.190194) 101 -dictGet test_01037.dict_array (32.7465,-1.37598) 101 -dictGet test_01037.dict_array (32.7494,-0.275675) 101 -dictGet test_01037.dict_array (32.7514,0.128951) 3501889 -dictGet test_01037.dict_array (32.753,3.44207) 3501822 -dictGet test_01037.dict_array (32.7686,2.11713) 3497126 -dictGet test_01037.dict_array (32.7694,1.47159) 3497388 -dictGet test_01037.dict_array (32.7768,0.0401042) 101 -dictGet test_01037.dict_array (32.781,-1.34283) 101 -dictGet test_01037.dict_array (32.7814,1.73876) 3497388 -dictGet test_01037.dict_array (32.7856,-1.06363) 101 -dictGet test_01037.dict_array (32.792699999999996,-1.1255600000000001) 101 -dictGet test_01037.dict_array (32.7941,-0.645447) 101 -dictGet test_01037.dict_array (32.7946,1.48889) 3497388 -dictGet test_01037.dict_array (32.797,0.791753) 3501889 -dictGet test_01037.dict_array (32.7982,-0.537798) 101 -dictGet test_01037.dict_array (32.8091,2.3611) 3490438 -dictGet test_01037.dict_array (32.81,1.7130800000000002) 3497388 -dictGet test_01037.dict_array (32.8174,-0.288322) 101 -dictGet test_01037.dict_array (32.823,1.6546699999999999) 3497388 -dictGet test_01037.dict_array (32.8233,1.62108) 3497388 -dictGet test_01037.dict_array (32.8428,-0.400045) 101 -dictGet test_01037.dict_array (32.8479,2.13598) 3490438 -dictGet test_01037.dict_array (32.8524,0.199902) 3501889 -dictGet test_01037.dict_array (32.8543,3.23553) 3501820 -dictGet test_01037.dict_array (32.8562,1.31371) 3498117 -dictGet test_01037.dict_array (32.87,1.44256) 3498117 -dictGet test_01037.dict_array (32.8789,2.38192) 3490438 -dictGet test_01037.dict_array (32.8812,2.20734) 3497128 -dictGet test_01037.dict_array (32.8815,-0.54427) 101 -dictGet test_01037.dict_array (32.8853,2.4859) 3497128 -dictGet test_01037.dict_array (32.8909,0.513964) 3501889 -dictGet test_01037.dict_array (32.9035,2.38999) 3490438 -dictGet test_01037.dict_array (32.9097,2.48131) 3497128 -dictGet test_01037.dict_array (32.928,-0.943269) 101 -dictGet test_01037.dict_array (32.9322,1.13165) 3498104 -dictGet test_01037.dict_array (32.9348,1.22606) 3498117 -dictGet test_01037.dict_array (32.9417,3.77998) 3501822 -dictGet test_01037.dict_array (32.9428,3.11936) 3497167 -dictGet test_01037.dict_array (32.9482,1.18092) 3498118 -dictGet test_01037.dict_array (32.9506,0.0609364) 101 -dictGet test_01037.dict_array (32.953,-0.828308) 101 -dictGet test_01037.dict_array (32.9593,3.5209099999999998) 3501822 -dictGet test_01037.dict_array (32.9617,2.07711) 3497128 -dictGet test_01037.dict_array (32.966,0.693749) 3498104 -dictGet test_01037.dict_array (32.9668,-0.716432) 101 -dictGet test_01037.dict_array (32.9702,1.98555) 3497127 -dictGet test_01037.dict_array (32.9782,1.73819) 3497388 -dictGet test_01037.dict_array (32.9805,3.71151) 3501822 -dictGet test_01037.dict_array (32.9821,2.97225) 3497167 -dictGet test_01037.dict_array (32.995,-0.830301) 101 -dictGet test_01037.dict_array (33.0234,0.770848) 3498104 -dictGet test_01037.dict_array (33.0312,-0.340964) 101 -dictGet test_01037.dict_array (33.0366,-0.756795) 101 -dictGet test_01037.dict_array (33.0438,0.812871) 3498118 -dictGet test_01037.dict_array (33.0455,1.84843) 3497127 -dictGet test_01037.dict_array (33.0498,0.0913292) 101 -dictGet test_01037.dict_array (33.0506,1.53739) 3497364 -dictGet test_01037.dict_array (33.0554,2.4265) 3497363 -dictGet test_01037.dict_array (33.0741,3.61332) 3501822 -dictGet test_01037.dict_array (33.0765,-0.179985) 101 -dictGet test_01037.dict_array (33.087,1.46465) 3497399 -dictGet test_01037.dict_array (33.0906,-0.620383) 101 -dictGet test_01037.dict_array (33.1047,-1.28027) 101 -dictGet test_01037.dict_array (33.1072,1.96303) 3497127 -dictGet test_01037.dict_array (33.1081,-0.897874) 101 -dictGet test_01037.dict_array (33.1122,1.8950200000000001) 3497127 -dictGet test_01037.dict_array (33.1237,2.63993) 3497165 -dictGet test_01037.dict_array (33.1238,0.753963) 3498118 -dictGet test_01037.dict_array (33.1257,0.495668) 3498102 -dictGet test_01037.dict_array (33.1258,1.78341) 3497364 -dictGet test_01037.dict_array (33.127,2.59646) 3497166 -dictGet test_01037.dict_array (33.1324,-1.23742) 101 -dictGet test_01037.dict_array (33.1359,3.83491) 101 -dictGet test_01037.dict_array (33.1628,-0.379588) 101 -dictGet test_01037.dict_array (33.1679,1.25601) 3498117 -dictGet test_01037.dict_array (33.1688,-1.35553) 101 -dictGet test_01037.dict_array (33.181,2.10943) 3497363 -dictGet test_01037.dict_array (33.1871,2.81171) 3497165 -dictGet test_01037.dict_array (33.1877,0.771297) 3498118 -dictGet test_01037.dict_array (33.1883,-0.204797) 101 -dictGet test_01037.dict_array (33.1886,3.27998) 3501820 -dictGet test_01037.dict_array (33.1955,0.708907) 3498118 -dictGet test_01037.dict_array (33.2044,-0.769275) 101 -dictGet test_01037.dict_array (33.2182,3.36103) 3501820 -dictGet test_01037.dict_array (33.2192,3.43586) 3501822 -dictGet test_01037.dict_array (33.2322,-0.916753) 101 -dictGet test_01037.dict_array (33.2359,-0.81321) 101 -dictGet test_01037.dict_array (33.238,0.635072) 3498111 -dictGet test_01037.dict_array (33.2398,3.02588) 3497165 -dictGet test_01037.dict_array (33.2469,2.35698) 3497363 -dictGet test_01037.dict_array (33.247,2.3327) 3497363 -dictGet test_01037.dict_array (33.2579,2.8027100000000003) 3497165 -dictGet test_01037.dict_array (33.2607,0.321082) 101 -dictGet test_01037.dict_array (33.2653,0.243336) 101 -dictGet test_01037.dict_array (33.2758,0.831836) 3498118 -dictGet test_01037.dict_array (33.2771,0.886536) 3498118 -dictGet test_01037.dict_array (33.2914,1.16026) 3498117 -dictGet test_01037.dict_array (33.2914,1.38882) 3497399 -dictGet test_01037.dict_array (33.2982,-1.16604) 101 -dictGet test_01037.dict_array (33.2985,0.842556) 3498112 -dictGet test_01037.dict_array (33.3005,2.8338900000000002) 3497165 -dictGet test_01037.dict_array (33.305,0.0969475) 101 -dictGet test_01037.dict_array (33.3072,3.82163) 101 -dictGet test_01037.dict_array (33.312,3.41475) 3501820 -dictGet test_01037.dict_array (33.3129,2.46048) 3497166 -dictGet test_01037.dict_array (33.3134,3.46863) 3501820 -dictGet test_01037.dict_array (33.3203,2.33139) 3497166 -dictGet test_01037.dict_array (33.324,0.433701) 101 -dictGet test_01037.dict_array (33.3338,2.44705) 3497166 -dictGet test_01037.dict_array (33.337,4.06475) 101 -dictGet test_01037.dict_array (33.3469,1.08172) 3498126 -dictGet test_01037.dict_array (33.3538,0.717896) 3498112 -dictGet test_01037.dict_array (33.3618,1.37899) 3497399 -dictGet test_01037.dict_array (33.3698,0.547744) 3501862 -dictGet test_01037.dict_array (33.3705,0.957619) 3498112 -dictGet test_01037.dict_array (33.3821,3.07258) 3497165 -dictGet test_01037.dict_array (33.3881,3.0626) 3497165 -dictGet test_01037.dict_array (33.393,-0.816186) 101 -dictGet test_01037.dict_array (33.3945,0.869508) 3498110 -dictGet test_01037.dict_array (33.4001,1.24186) 3498117 -dictGet test_01037.dict_array (33.4008,2.34911) 3497166 -dictGet test_01037.dict_array (33.4166,-1.2808899999999999) 101 -dictGet test_01037.dict_array (33.4167,3.0655) 3497165 -dictGet test_01037.dict_array (33.4204,2.81887) 3497165 -dictGet test_01037.dict_array (33.4211,1.71128) 3497400 -dictGet test_01037.dict_array (33.4237,2.91761) 3497165 -dictGet test_01037.dict_array (33.4266,1.5955599999999999) 3497399 -dictGet test_01037.dict_array (33.4353,-0.391392) 101 -dictGet test_01037.dict_array (33.4362,-0.134658) 101 -dictGet test_01037.dict_array (33.4386,0.15396) 101 -dictGet test_01037.dict_array (33.4421,-0.50712) 101 -dictGet test_01037.dict_array (33.452,0.915829) 3498126 -dictGet test_01037.dict_array (33.463,-0.0882717) 101 -dictGet test_01037.dict_array (33.464,-1.00949) 101 -dictGet test_01037.dict_array (33.4692,0.954092) 3498126 -dictGet test_01037.dict_array (33.4716,1.9538799999999998) 3497400 -dictGet test_01037.dict_array (33.4756,1.85836) 3497400 -dictGet test_01037.dict_array (33.4859,4.0751) 101 -dictGet test_01037.dict_array (33.4899,3.54193) 3501820 -dictGet test_01037.dict_array (33.4935,3.49794) 3501820 -dictGet test_01037.dict_array (33.494,-0.983356) 101 -dictGet test_01037.dict_array (33.4955,-1.28128) 101 -dictGet test_01037.dict_array (33.4965,-0.278687) 101 -dictGet test_01037.dict_array (33.4991,0.647491) 3498110 -dictGet test_01037.dict_array (33.5076,2.2272) 3497424 -dictGet test_01037.dict_array (33.5079,-0.498199) 101 -dictGet test_01037.dict_array (33.5157,0.535034) 3501862 -dictGet test_01037.dict_array (33.5171,2.49677) 3497166 -dictGet test_01037.dict_array (33.5255,2.4447200000000002) 3497166 -dictGet test_01037.dict_array (33.526,4.01194) 101 -dictGet test_01037.dict_array (33.5288,0.789434) 3498110 -dictGet test_01037.dict_array (33.5356,-1.17671) 101 -dictGet test_01037.dict_array (33.5402,1.49152) 3497399 -dictGet test_01037.dict_array (33.5418,3.45757) 3501820 -dictGet test_01037.dict_array (33.5428,1.90712) 3497400 -dictGet test_01037.dict_array (33.5556,-0.55741) 101 -dictGet test_01037.dict_array (33.5564,0.876858) 3498128 -dictGet test_01037.dict_array (33.5567,-0.10208) 101 -dictGet test_01037.dict_array (33.5645,-0.124824) 101 -dictGet test_01037.dict_array (33.5663,3.4872) 3501820 -dictGet test_01037.dict_array (33.5716,-0.0107611) 101 -dictGet test_01037.dict_array (33.578,3.55714) 3501820 -dictGet test_01037.dict_array (33.5826,-0.49076) 101 -dictGet test_01037.dict_array (33.5909,0.773737) 3498110 -dictGet test_01037.dict_array (33.5958,2.9619999999999997) 3497425 -dictGet test_01037.dict_array (33.6193,-0.919755) 101 -dictGet test_01037.dict_array (33.6313,0.652132) 3498110 -dictGet test_01037.dict_array (33.632,0.823351) 3498128 -dictGet test_01037.dict_array (33.66,2.18998) 3497424 -dictGet test_01037.dict_array (33.6621,0.535395) 3498135 -dictGet test_01037.dict_array (33.6726,3.19367) 3497438 -dictGet test_01037.dict_array (33.6912,1.74522) 3497400 -dictGet test_01037.dict_array (33.705,0.706397) 3498135 -dictGet test_01037.dict_array (33.7076,0.7622) 3498128 -dictGet test_01037.dict_array (33.7112,1.70187) 3497400 -dictGet test_01037.dict_array (33.7246,-1.14837) 101 -dictGet test_01037.dict_array (33.7326,2.62413) 3497425 -dictGet test_01037.dict_array (33.7332,2.82137) 3497425 -dictGet test_01037.dict_array (33.7434,0.394672) 3498135 -dictGet test_01037.dict_array (33.7443,1.54557) 3497398 -dictGet test_01037.dict_array (33.7506,1.57317) 3497398 -dictGet test_01037.dict_array (33.7526,1.8578999999999999) 3497424 -dictGet test_01037.dict_array (33.766,4.15013) 101 -dictGet test_01037.dict_array (33.7834,2.41789) 3497439 -dictGet test_01037.dict_array (33.7864,0.230935) 101 -dictGet test_01037.dict_array (33.7965,3.05709) 3497438 -dictGet test_01037.dict_array (33.7998,3.32881) 3497438 -dictGet test_01037.dict_array (33.8003,2.97338) 3497425 -dictGet test_01037.dict_array (33.8007,-1.08962) 101 -dictGet test_01037.dict_array (33.8022,-0.139488) 101 -dictGet test_01037.dict_array (33.8065,2.70857) 3497425 -dictGet test_01037.dict_array (33.8169,-0.607788) 101 -dictGet test_01037.dict_array (33.8203,0.108512) 3501863 -dictGet test_01037.dict_array (33.8231,-1.03449) 101 -dictGet test_01037.dict_array (33.8312,3.49458) 3501829 -dictGet test_01037.dict_array (33.8342,0.297518) 3501863 -dictGet test_01037.dict_array (33.8352,0.165872) 101 -dictGet test_01037.dict_array (33.8354,1.87277) 3497424 -dictGet test_01037.dict_array (33.8371,1.60103) 3497398 -dictGet test_01037.dict_array (33.8387,1.9968) 3497424 -dictGet test_01037.dict_array (33.8403,3.5805) 3501829 -dictGet test_01037.dict_array (33.8414,-0.703067) 101 -dictGet test_01037.dict_array (33.844,-0.179472) 101 -dictGet test_01037.dict_array (33.8468,3.40137) 3501829 -dictGet test_01037.dict_array (33.8509,4.15334) 101 -dictGet test_01037.dict_array (33.8539,2.38339) 3497439 -dictGet test_01037.dict_array (33.858,-1.3122500000000001) 101 -dictGet test_01037.dict_array (33.859,3.72626) 3501829 -dictGet test_01037.dict_array (33.8616,2.24433) 3497424 -dictGet test_01037.dict_array (33.8621,3.01035) 3497438 -dictGet test_01037.dict_array (33.8623,1.17559) 3498129 -dictGet test_01037.dict_array (33.8682,2.706) 3497425 -dictGet test_01037.dict_array (33.8684,0.189231) 3501863 -dictGet test_01037.dict_array (33.872,1.93574) 3497424 -dictGet test_01037.dict_array (33.8844,3.80404) 3501829 -dictGet test_01037.dict_array (33.8888,0.594884) 3498135 -dictGet test_01037.dict_array (33.8946,2.74161) 3497438 -dictGet test_01037.dict_array (33.9023,0.6239) 3498135 -dictGet test_01037.dict_array (33.9057,0.873222) 3498136 -dictGet test_01037.dict_array (33.9157,-1.26607) 101 -dictGet test_01037.dict_array (33.92,2.06848) 3497397 -dictGet test_01037.dict_array (33.9298,-0.00526229) 101 -dictGet test_01037.dict_array (33.932,3.07063) 3497438 -dictGet test_01037.dict_array (33.9322,0.629385) 3501864 -dictGet test_01037.dict_array (33.9367,-1.41955) 101 -dictGet test_01037.dict_array (33.937,1.42532) 3498173 -dictGet test_01037.dict_array (33.9375,1.1467100000000001) 3498159 -dictGet test_01037.dict_array (33.9434,-1.05739) 101 -dictGet test_01037.dict_array (33.9477,3.34809) 3501829 -dictGet test_01037.dict_array (33.95,2.21715) 3497397 -dictGet test_01037.dict_array (33.955799999999996,0.305176) 3501859 -dictGet test_01037.dict_array (33.9686,-0.28273) 101 -dictGet test_01037.dict_array (33.9703,4.1255) 3501829 -dictGet test_01037.dict_array (33.9707,3.08199) 3497438 -dictGet test_01037.dict_array (33.9754,1.06203) 3498159 -dictGet test_01037.dict_array (33.9757,3.72468) 3501829 -dictGet test_01037.dict_array (33.9775,-0.0440599) 101 -dictGet test_01037.dict_array (33.9777,-0.251484) 101 -dictGet test_01037.dict_array (33.9789,-0.339374) 101 -dictGet test_01037.dict_array (33.9849,2.54515) 3497425 -dictGet test_01037.dict_array (33.9885,-0.318557) 101 -dictGet test_01037.dict_array (33.9977,1.07175) 3498159 -dictGet test_01037.dict_array (33.9984,-0.700517) 101 -dictGet test_01037.dict_array (34.0149,3.53338) 3501829 -dictGet test_01037.dict_array (34.0173,3.39155) 3501829 -dictGet test_01037.dict_array (34.0317,3.9579) 3501829 -dictGet test_01037.dict_array (34.0369,3.83612) 3501829 -dictGet test_01037.dict_array (34.043,-0.0887221) 101 -dictGet test_01037.dict_array (34.0487,1.14252) 3498159 -dictGet test_01037.dict_array (34.052,1.74832) 3497397 -dictGet test_01037.dict_array (34.0711,-0.898071) 101 -dictGet test_01037.dict_array (34.0747,1.55057) 3498173 -dictGet test_01037.dict_array (34.0803,3.16763) 3497438 -dictGet test_01037.dict_array (34.0872,3.75555) 3501829 -dictGet test_01037.dict_array (34.0965,1.62038) 3498173 -dictGet test_01037.dict_array (34.0977,-0.412691) 101 -dictGet test_01037.dict_array (34.0986,0.0294206) 101 -dictGet test_01037.dict_array (34.1072,3.15823) 3497438 -dictGet test_01037.dict_array (34.1092,3.09599) 3497438 -dictGet test_01037.dict_array (34.1206,1.04637) 3498160 -dictGet test_01037.dict_array (34.1209,3.13826) 3497438 -dictGet test_01037.dict_array (34.1265,3.95881) 3501829 -dictGet test_01037.dict_array (34.1286,-0.539319) 101 -dictGet test_01037.dict_array (34.1358,3.67451) 3501829 -dictGet test_01037.dict_array (34.1428,0.136115) 101 -dictGet test_01037.dict_array (34.157,1.73522) 3497397 -dictGet test_01037.dict_array (34.1581,1.48001) 3498172 -dictGet test_01037.dict_array (34.1682,3.42373) 3501829 -dictGet test_01037.dict_array (34.1683,-1.26511) 101 -dictGet test_01037.dict_array (34.1684,4.20007) 101 -dictGet test_01037.dict_array (34.1854,3.32089) 3501829 -dictGet test_01037.dict_array (34.2022,0.749536) 3501864 -dictGet test_01037.dict_array (34.2044,3.04865) 3497438 -dictGet test_01037.dict_array (34.22,-0.500055) 101 -dictGet test_01037.dict_array (34.2249,0.743775) 3501864 -dictGet test_01037.dict_array (34.2254,1.34702) 3498172 -dictGet test_01037.dict_array (34.2355,-0.898843) 101 -dictGet test_01037.dict_array (34.2394,2.0203699999999998) 3497439 -dictGet test_01037.dict_array (34.2466,1.83785) 3498251 -dictGet test_01037.dict_array (34.247,4.09563) 101 -dictGet test_01037.dict_array (34.2508,2.61312) 3497439 -dictGet test_01037.dict_array (34.2517,1.69642) 3498251 -dictGet test_01037.dict_array (34.2564,4.13033) 101 -dictGet test_01037.dict_array (34.2574,4.18928) 101 -dictGet test_01037.dict_array (34.2614,-0.478719) 101 -dictGet test_01037.dict_array (34.2625,2.38088) 3497439 -dictGet test_01037.dict_array (34.2666,3.1503) 3501829 -dictGet test_01037.dict_array (34.271,4.02223) 101 -dictGet test_01037.dict_array (34.2727,0.514755) 101 -dictGet test_01037.dict_array (34.278,1.98929) 3497439 -dictGet test_01037.dict_array (34.2798,-0.199208) 101 -dictGet test_01037.dict_array (34.2804,2.05184) 3497439 -dictGet test_01037.dict_array (34.2945,-1.11051) 101 -dictGet test_01037.dict_array (34.3168,-0.0829721) 101 -dictGet test_01037.dict_array (34.3345,3.4358) 3501829 -dictGet test_01037.dict_array (34.3377,1.13527) 3498162 -dictGet test_01037.dict_array (34.3383,1.27891) 3498161 -dictGet test_01037.dict_array (34.3391,1.47945) 3498161 -dictGet test_01037.dict_array (34.3441,0.627014) 101 -dictGet test_01037.dict_array (34.347,2.4853) 3497439 -dictGet test_01037.dict_array (34.3514,2.16247) 3497439 -dictGet test_01037.dict_array (34.3627,2.64533) 3497439 -dictGet test_01037.dict_array (34.3682,-0.227501) 101 -dictGet test_01037.dict_array (34.3756,4.21248) 101 -dictGet test_01037.dict_array (34.379,3.96604) 101 -dictGet test_01037.dict_array (34.3827,1.7518) 3498251 -dictGet test_01037.dict_array (34.3912,2.8834) 3501830 -dictGet test_01037.dict_array (34.3919,0.668829) 101 -dictGet test_01037.dict_array (34.3949,2.00338) 3497439 -dictGet test_01037.dict_array (34.3987,0.557268) 101 -dictGet test_01037.dict_array (34.4111,0.768558) 101 -dictGet test_01037.dict_array (34.4119,2.8742) 3501830 -dictGet test_01037.dict_array (34.416,3.50841) 3501829 -dictGet test_01037.dict_array (34.4212,1.24916) 3498161 -dictGet test_01037.dict_array (34.4251,0.457029) 101 -dictGet test_01037.dict_array (34.4274,-0.902559) 101 -dictGet test_01037.dict_array (34.4325,4.03159) 101 -dictGet test_01037.dict_array (34.438,1.63994) 3498251 -dictGet test_01037.dict_array (34.4403,-0.177594) 101 -dictGet test_01037.dict_array (34.4421,0.726712) 101 -dictGet test_01037.dict_array (34.4517,2.98611) 3501830 -dictGet test_01037.dict_array (34.4658,-1.312) 101 -dictGet test_01037.dict_array (34.4732,-0.0681338) 101 -dictGet test_01037.dict_array (34.4752,2.81646) 3501830 -dictGet test_01037.dict_array (34.4914,2.3858) 3497439 -dictGet test_01037.dict_array (34.4923,0.855231) 101 -dictGet test_01037.dict_array (34.5235,1.78468) 3498251 -dictGet test_01037.dict_array (34.5305,4.10608) 101 -dictGet test_01037.dict_array (34.5389,0.621937) 101 -dictGet test_01037.dict_array (34.5406,3.17145) 101 -dictGet test_01037.dict_array (34.5434,-0.56306) 101 -dictGet test_01037.dict_array (34.5449,3.13311) 3501829 -dictGet test_01037.dict_array (34.5491,2.31572) 3497439 -dictGet test_01037.dict_array (34.5539,2.94028) 3501830 -dictGet test_01037.dict_array (34.5546,-0.208825) 101 -dictGet test_01037.dict_array (34.5549,3.78486) 101 -dictGet test_01037.dict_array (34.5676,0.307148) 101 -dictGet test_01037.dict_array (34.5743,1.5217399999999999) 3501838 -dictGet test_01037.dict_array (34.5775,3.48046) 101 -dictGet test_01037.dict_array (34.5815,2.5243700000000002) 3501830 -dictGet test_01037.dict_array (34.5841,4.21191) 101 -dictGet test_01037.dict_array (34.5887,2.65083) 3501830 -dictGet test_01037.dict_array (34.5937,3.2143) 101 -dictGet test_01037.dict_array (34.6013,-1.0612) 101 -dictGet test_01037.dict_array (34.6089,1.36066) 3501838 -dictGet test_01037.dict_array (34.6103,3.40227) 101 -dictGet test_01037.dict_array (34.6128,1.92276) 3498251 -dictGet test_01037.dict_array (34.6175,2.43627) 3498251 -dictGet test_01037.dict_array (34.6209,3.43776) 101 -dictGet test_01037.dict_array (34.6234,2.60237) 3501830 -dictGet test_01037.dict_array (34.6275,3.52479) 101 -dictGet test_01037.dict_array (34.635,0.568558) 101 -dictGet test_01037.dict_array (34.6373,2.37692) 3498251 -dictGet test_01037.dict_array (34.6375,3.52234) 101 -dictGet test_01037.dict_array (34.6426,2.12397) 3498251 -dictGet test_01037.dict_array (34.6513,2.80915) 3501830 -dictGet test_01037.dict_array (34.6632,2.30039) 3498251 -dictGet test_01037.dict_array (34.6691,1.86582) 3498251 -dictGet test_01037.dict_array (34.6739,0.15342) 101 -dictGet test_01037.dict_array (34.6825,0.0499679) 101 -dictGet test_01037.dict_array (34.6893,0.454326) 101 -dictGet test_01037.dict_array (34.6957,-0.358598) 101 -dictGet test_01037.dict_array (34.6986,0.562679) 101 -dictGet test_01037.dict_array (34.712,1.12114) 101 -dictGet test_01037.dict_array (34.7126,-0.0057301) 101 -dictGet test_01037.dict_array (34.7137,0.0248501) 101 -dictGet test_01037.dict_array (34.7162,1.15623) 101 -dictGet test_01037.dict_array (34.7258,3.95142) 101 -dictGet test_01037.dict_array (34.7347,3.5232099999999997) 101 -dictGet test_01037.dict_array (34.7363,2.23374) 3501830 -dictGet test_01037.dict_array (34.7375,0.397841) 101 -dictGet test_01037.dict_array (34.7423,3.09198) 101 -dictGet test_01037.dict_array (34.7452,3.09029) 101 -dictGet test_01037.dict_array (34.7539,-1.06943) 101 -dictGet test_01037.dict_array (34.7733,-0.00912717) 101 -dictGet test_01037.dict_array (34.774,2.71088) 3501830 -dictGet test_01037.dict_array (34.7771,1.46009) 3501835 -dictGet test_01037.dict_array (34.7782,-1.28308) 101 -dictGet test_01037.dict_array (34.7924,3.63564) 101 -dictGet test_01037.dict_array (34.7939,-0.416676) 101 -dictGet test_01037.dict_array (34.7964,-0.401773) 101 -dictGet test_01037.dict_array (34.7974,0.0286873) 101 -dictGet test_01037.dict_array (34.7975,3.05965) 101 -dictGet test_01037.dict_array (34.8037,3.07263) 101 -dictGet test_01037.dict_array (34.8254,-0.390284) 101 -dictGet test_01037.dict_array (34.828,1.91869) 3498251 -dictGet test_01037.dict_array (34.8289,3.71058) 101 -dictGet test_01037.dict_array (34.8403,2.14606) 3501835 -dictGet test_01037.dict_array (34.8437,2.20617) 3501830 -dictGet test_01037.dict_array (34.8469,2.38435) 3501830 -dictGet test_01037.dict_array (34.86,1.45705) 101 -dictGet test_01037.dict_array (34.8612,0.914248) 101 -dictGet test_01037.dict_array (34.8663,3.4215400000000002) 101 -dictGet test_01037.dict_array (34.8724,-0.375144) 101 -dictGet test_01037.dict_array (34.8795,3.29317) 101 -dictGet test_01037.dict_array (34.8823,1.21988) 101 -dictGet test_01037.dict_array (34.8834,1.07657) 101 -dictGet test_01037.dict_array (34.8837,0.157648) 101 -dictGet test_01037.dict_array (34.8871,-0.9755) 101 -dictGet test_01037.dict_array (34.8871,1.8943699999999999) 3501835 -dictGet test_01037.dict_array (34.889,3.36756) 101 -dictGet test_01037.dict_array (34.8907,1.24874) 101 -dictGet test_01037.dict_array (34.8965,3.13508) 101 -dictGet test_01037.dict_array (34.9042,2.62092) 101 -dictGet test_01037.dict_array (34.9055,-0.0448967) 101 -dictGet test_01037.dict_array (34.9122,0.110576) 101 -dictGet test_01037.dict_array (34.9228,3.60183) 101 -dictGet test_01037.dict_array (34.9237,1.21715) 101 -dictGet test_01037.dict_array (34.9296,1.70459) 3501835 -dictGet test_01037.dict_array (34.941,-1.14663) 101 -dictGet test_01037.dict_array (34.9448,1.18923) 101 -dictGet test_01037.dict_array (34.9462,3.81678) 101 -dictGet test_01037.dict_array (34.9466,0.593463) 101 -dictGet test_01037.dict_array (34.9485,0.150307) 101 -dictGet test_01037.dict_array (34.9542,0.487238) 101 -dictGet test_01037.dict_array (34.9559,2.03473) 3501835 -dictGet test_01037.dict_array (34.9671,-0.960225) 101 -dictGet test_01037.dict_array (34.9711,2.63444) 101 -dictGet test_01037.dict_array (34.9892,0.354775) 101 -dictGet test_01037.dict_array (34.9907,1.40724) 101 -dictGet test_01037.dict_array (34.9916,-0.00173097) 101 -dictGet test_01037.dict_array (34.9919,2.06167) 101 +dictGet dict_array (29.5699,2.50068) 101 +dictGet dict_array (29.5796,1.55456) 101 +dictGet dict_array (29.5796,2.36864) 101 +dictGet dict_array (29.5844,1.59626) 101 +dictGet dict_array (29.5886,4.03321) 101 +dictGet dict_array (29.5914,3.02628) 101 +dictGet dict_array (29.5926,-0.0965169) 101 +dictGet dict_array (29.5968,2.37773) 101 +dictGet dict_array (29.5984,0.755853) 101 +dictGet dict_array (29.6066,3.47173) 101 +dictGet dict_array (29.6085,-1.26007) 6489978 +dictGet dict_array (29.6131,0.246565) 101 +dictGet dict_array (29.6157,-0.266687) 101 +dictGet dict_array (29.6164,2.94674) 101 +dictGet dict_array (29.6195,-0.591941) 101 +dictGet dict_array (29.6231,1.54818) 101 +dictGet dict_array (29.6379,0.764114) 101 +dictGet dict_array (29.6462,-0.772059) 934530 +dictGet dict_array (29.6579,-1.07336) 6489978 +dictGet dict_array (29.6618,-0.271842) 101 +dictGet dict_array (29.6629,-0.303602) 101 +dictGet dict_array (29.6659,-0.782823) 934530 +dictGet dict_array (29.6736,-0.113832) 101 +dictGet dict_array (29.6759,3.02905) 101 +dictGet dict_array (29.6778,3.71898) 101 +dictGet dict_array (29.6796,1.10433) 101 +dictGet dict_array (29.6809,2.13677) 101 +dictGet dict_array (29.6935,4.11894) 101 +dictGet dict_array (29.6991,-1.4458199999999999) 101 +dictGet dict_array (29.6997,3.17297) 101 +dictGet dict_array (29.7043,3.6145899999999997) 101 +dictGet dict_array (29.7065,3.24885) 101 +dictGet dict_array (29.7126,0.28108) 101 +dictGet dict_array (29.7192,0.174273) 101 +dictGet dict_array (29.7217,-0.523481) 3501900 +dictGet dict_array (29.7271,1.67967) 101 +dictGet dict_array (29.7311,4.12444) 101 +dictGet dict_array (29.7347,1.88378) 101 +dictGet dict_array (29.7358,0.67944) 101 +dictGet dict_array (29.7366,-0.2973) 101 +dictGet dict_array (29.7446,0.646536) 101 +dictGet dict_array (29.7453,-0.567963) 3501900 +dictGet dict_array (29.764,4.04217) 101 +dictGet dict_array (29.7655,1.51372) 101 +dictGet dict_array (29.7744,1.12435) 101 +dictGet dict_array (29.7774,-0.0681196) 3501895 +dictGet dict_array (29.7784,1.54864) 101 +dictGet dict_array (29.7785,2.24139) 101 +dictGet dict_array (29.7922,0.220808) 101 +dictGet dict_array (29.7936,2.37709) 101 +dictGet dict_array (29.8008,0.948536) 101 +dictGet dict_array (29.8115,0.201227) 101 +dictGet dict_array (29.814,0.149601) 3501895 +dictGet dict_array (29.8193,-1.35858) 101 +dictGet dict_array (29.8201,0.965518) 101 +dictGet dict_array (29.8265,-0.727286) 3501900 +dictGet dict_array (29.8277,-0.531746) 3501900 +dictGet dict_array (29.8289,3.63009) 101 +dictGet dict_array (29.8548,0.838047) 101 +dictGet dict_array (29.8641,-0.845265) 3501900 +dictGet dict_array (29.8649,0.0562212) 3501895 +dictGet dict_array (29.8701,-1.02045) 934530 +dictGet dict_array (29.8733,2.76654) 101 +dictGet dict_array (29.876,0.555475) 101 +dictGet dict_array (29.8794,-0.800108) 3501900 +dictGet dict_array (29.8813,2.7426399999999997) 101 +dictGet dict_array (29.897100000000002,2.66193) 101 +dictGet dict_array (29.908,4.01339) 101 +dictGet dict_array (29.9165,-1.08246) 3501894 +dictGet dict_array (29.9201,-0.420861) 3498054 +dictGet dict_array (29.9217,3.03778) 101 +dictGet dict_array (29.9355,0.773833) 101 +dictGet dict_array (29.947,3.76517) 101 +dictGet dict_array (29.9518,-0.60557) 3498056 +dictGet dict_array (29.9564,-0.600163) 3498056 +dictGet dict_array (29.959600000000002,4.16591) 101 +dictGet dict_array (29.9615,-1.33708) 3501894 +dictGet dict_array (29.9699,-0.392375) 3498054 +dictGet dict_array (29.9776,1.04552) 101 +dictGet dict_array (29.9784,4.02756) 101 +dictGet dict_array (29.9819,4.00597) 101 +dictGet dict_array (29.9826,1.2816100000000001) 101 +dictGet dict_array (30.0026,2.76257) 101 +dictGet dict_array (30.0126,3.68255) 101 +dictGet dict_array (30.0131,0.796576) 3501892 +dictGet dict_array (30.018,1.16523) 101 +dictGet dict_array (30.0261,-0.210653) 3501896 +dictGet dict_array (30.0472,-1.11007) 3501894 +dictGet dict_array (30.0542,-0.479585) 3498054 +dictGet dict_array (30.0613,1.6278000000000001) 101 +dictGet dict_array (30.0617,-0.0551152) 3501895 +dictGet dict_array (30.0637,2.62066) 101 +dictGet dict_array (30.0721,1.6424400000000001) 101 +dictGet dict_array (30.0769,-0.402636) 3498054 +dictGet dict_array (30.0791,-0.277435) 3501896 +dictGet dict_array (30.0931,0.0327512) 3501895 +dictGet dict_array (30.1059,3.52623) 101 +dictGet dict_array (30.1103,0.865466) 3501892 +dictGet dict_array (30.1115,2.95243) 101 +dictGet dict_array (30.1144,1.71029) 101 +dictGet dict_array (30.1311,-0.864751) 3501899 +dictGet dict_array (30.1336,-0.851386) 3501899 +dictGet dict_array (30.1393,3.89901) 101 +dictGet dict_array (30.1456,-0.531898) 3498054 +dictGet dict_array (30.1492,2.07833) 101 +dictGet dict_array (30.1575,2.43856) 101 +dictGet dict_array (30.1682,1.19771) 101 +dictGet dict_array (30.1716,3.9853300000000003) 101 +dictGet dict_array (30.1849,2.78374) 101 +dictGet dict_array (30.1866,0.65658) 3498021 +dictGet dict_array (30.1885,1.56943) 101 +dictGet dict_array (30.1959,-1.38202) 101 +dictGet dict_array (30.1999,1.58413) 101 +dictGet dict_array (30.2024,0.713081) 3498021 +dictGet dict_array (30.2054,0.620143) 3498021 +dictGet dict_array (30.2091,1.51641) 101 +dictGet dict_array (30.2124,-0.331782) 3498031 +dictGet dict_array (30.226,3.03527) 101 +dictGet dict_array (30.2261,3.18486) 101 +dictGet dict_array (30.2288,2.48407) 101 +dictGet dict_array (30.2345,3.7462400000000002) 101 +dictGet dict_array (30.2375,0.62046) 3498021 +dictGet dict_array (30.2425,-0.472914) 3498054 +dictGet dict_array (30.247,3.95863) 101 +dictGet dict_array (30.2494,-0.305093) 3498031 +dictGet dict_array (30.2499,2.54337) 101 +dictGet dict_array (30.2606,2.16644) 101 +dictGet dict_array (30.2672,3.94847) 101 +dictGet dict_array (30.2709,-0.136264) 6088794 +dictGet dict_array (30.2764,1.18654) 101 +dictGet dict_array (30.2765,1.20383) 101 +dictGet dict_array (30.2839,1.05762) 3498024 +dictGet dict_array (30.286,0.469327) 3498021 +dictGet dict_array (30.2927,3.1693) 101 +dictGet dict_array (30.2935,3.49854) 101 +dictGet dict_array (30.307,0.312338) 3498021 +dictGet dict_array (30.3085,1.07791) 3498024 +dictGet dict_array (30.3139,2.77248) 101 +dictGet dict_array (30.314,0.822823) 3498024 +dictGet dict_array (30.3227,-0.587351) 3498055 +dictGet dict_array (30.332,1.00174) 3498024 +dictGet dict_array (30.3388,0.844148) 3498024 +dictGet dict_array (30.3485,0.561902) 3498021 +dictGet dict_array (30.3497,0.180362) 6489998 +dictGet dict_array (30.361,4.13016) 101 +dictGet dict_array (30.3623,-0.0484027) 6489998 +dictGet dict_array (30.3638,3.9845800000000002) 101 +dictGet dict_array (30.3853,3.16051) 101 +dictGet dict_array (30.3974,2.6617800000000003) 101 +dictGet dict_array (30.4002,-1.15886) 101 +dictGet dict_array (30.4008,-0.387015) 3498031 +dictGet dict_array (30.4018,1.86493) 101 +dictGet dict_array (30.4239,1.16818) 3498024 +dictGet dict_array (30.4363,3.63938) 101 +dictGet dict_array (30.4377,-0.81315) 3498063 +dictGet dict_array (30.4391,3.54703) 101 +dictGet dict_array (30.4424,-1.39435) 101 +dictGet dict_array (30.4441,2.8463000000000003) 101 +dictGet dict_array (30.4517,3.28117) 101 +dictGet dict_array (30.4658,2.6928) 101 +dictGet dict_array (30.4734,2.66161) 101 +dictGet dict_array (30.4799,-1.07578) 101 +dictGet dict_array (30.4837,-1.02486) 3501899 +dictGet dict_array (30.485,1.06326) 3498024 +dictGet dict_array (30.495,1.12306) 101 +dictGet dict_array (30.501,2.27264) 101 +dictGet dict_array (30.5027,1.99382) 101 +dictGet dict_array (30.5194,-1.03943) 3501893 +dictGet dict_array (30.5239,1.04328) 101 +dictGet dict_array (30.528,3.82041) 101 +dictGet dict_array (30.5299,-0.715248) 3498063 +dictGet dict_array (30.5331,1.19603) 101 +dictGet dict_array (30.535800000000002,2.71485) 101 +dictGet dict_array (30.5405,0.804694) 3498023 +dictGet dict_array (30.542,1.23739) 101 +dictGet dict_array (30.5432,4.04189) 101 +dictGet dict_array (30.5457,-0.956121) 3501893 +dictGet dict_array (30.5506,3.07443) 101 +dictGet dict_array (30.5539,3.87084) 101 +dictGet dict_array (30.5578,3.78837) 101 +dictGet dict_array (30.5588,0.966135) 3498022 +dictGet dict_array (30.5637,2.5605) 101 +dictGet dict_array (30.5647,-1.27328) 101 +dictGet dict_array (30.5656,-0.0581332) 6088794 +dictGet dict_array (30.5715,0.65755) 3498023 +dictGet dict_array (30.5727,3.01604) 101 +dictGet dict_array (30.5729,-0.976857) 3501893 +dictGet dict_array (30.5751,0.60204) 3498023 +dictGet dict_array (30.5854,3.02473) 101 +dictGet dict_array (30.5866,0.174099) 6489998 +dictGet dict_array (30.5947,0.875193) 3498023 +dictGet dict_array (30.5992,-0.403901) 3498063 +dictGet dict_array (30.6002,4.18891) 101 +dictGet dict_array (30.6025,0.217712) 6489998 +dictGet dict_array (30.6054,0.927203) 3498022 +dictGet dict_array (30.6075,3.79359) 101 +dictGet dict_array (30.6159,3.82773) 101 +dictGet dict_array (30.627,3.84039) 101 +dictGet dict_array (30.6308,0.77517) 3498023 +dictGet dict_array (30.6338,0.179565) 6489998 +dictGet dict_array (30.6461,1.3293599999999999) 101 +dictGet dict_array (30.6674,-0.424547) 3498063 +dictGet dict_array (30.669,1.76539) 101 +dictGet dict_array (30.6788,4.01239) 101 +dictGet dict_array (30.6864,3.59158) 101 +dictGet dict_array (30.7049,-0.875413) 3501893 +dictGet dict_array (30.705,1.3307) 101 +dictGet dict_array (30.7063,-0.473192) 3498063 +dictGet dict_array (30.7075,-1.1958199999999999) 101 +dictGet dict_array (30.7101,-0.367562) 3498012 +dictGet dict_array (30.7203,2.98725) 101 +dictGet dict_array (30.7213,2.2745699999999998) 101 +dictGet dict_array (30.7446,-0.334144) 3498012 +dictGet dict_array (30.7468,3.82967) 101 +dictGet dict_array (30.747,-0.384779) 3498012 +dictGet dict_array (30.7681,0.904198) 3498022 +dictGet dict_array (30.7757,1.78743) 101 +dictGet dict_array (30.8021,-0.479212) 3498012 +dictGet dict_array (30.8079,-1.40869) 101 +dictGet dict_array (30.8206,-0.0608489) 3498012 +dictGet dict_array (30.8218,0.43909) 3498023 +dictGet dict_array (30.8239,0.10014) 3498012 +dictGet dict_array (30.8282,4.15409) 101 +dictGet dict_array (30.8288,-0.709528) 3501893 +dictGet dict_array (30.8326,0.156011) 3498012 +dictGet dict_array (30.8328,-1.03704) 101 +dictGet dict_array (30.839,2.15528) 101 +dictGet dict_array (30.8452,0.219377) 3498013 +dictGet dict_array (30.8463,0.0515355) 3498012 +dictGet dict_array (30.8526,2.06614) 101 +dictGet dict_array (30.8566,0.517876) 3498023 +dictGet dict_array (30.8588,-1.31738) 101 +dictGet dict_array (30.8681,0.44207) 3498013 +dictGet dict_array (30.8914,1.0072) 3498022 +dictGet dict_array (30.897,0.483425) 3498013 +dictGet dict_array (30.905,2.8731999999999998) 3501793 +dictGet dict_array (30.9051,2.21956) 101 +dictGet dict_array (30.9115,4.00663) 101 +dictGet dict_array (30.9167,-0.834462) 3501893 +dictGet dict_array (30.9252,-1.3289900000000001) 101 +dictGet dict_array (30.9314,1.85384) 101 +dictGet dict_array (30.9392,2.53236) 3501827 +dictGet dict_array (30.9569,2.82038) 3501793 +dictGet dict_array (30.9598,-0.641011) 3498012 +dictGet dict_array (30.9601,-0.254928) 3498012 +dictGet dict_array (30.9623,-1.3886) 101 +dictGet dict_array (30.9707,0.888854) 3498022 +dictGet dict_array (30.9766,2.81957) 3501793 +dictGet dict_array (30.9775,2.69273) 3501793 +dictGet dict_array (30.9821,0.587715) 3498013 +dictGet dict_array (30.9887,4.0233) 101 +dictGet dict_array (30.9914,0.259542) 3498013 +dictGet dict_array (30.9986,-1.36832) 101 +dictGet dict_array (31.008,0.628999) 3498013 +dictGet dict_array (31.0168,-1.17462) 101 +dictGet dict_array (31.0237,3.52547) 3501821 +dictGet dict_array (31.0306,3.78522) 101 +dictGet dict_array (31.0308,-0.72453) 3501893 +dictGet dict_array (31.0463,2.41997) 3501825 +dictGet dict_array (31.047,0.624184) 3498013 +dictGet dict_array (31.0569,0.0706393) 3498015 +dictGet dict_array (31.0583,1.3244099999999999) 3501926 +dictGet dict_array (31.063,3.23861) 3501793 +dictGet dict_array (31.068,0.695575) 3498022 +dictGet dict_array (31.0687,1.85675) 101 +dictGet dict_array (31.0692,0.254793) 3498014 +dictGet dict_array (31.0766,0.828128) 3498022 +dictGet dict_array (31.0833,0.0612782) 3498015 +dictGet dict_array (31.0833,2.59748) 3501793 +dictGet dict_array (31.0861,-1.3778299999999999) 101 +dictGet dict_array (31.0874,3.07258) 3501793 +dictGet dict_array (31.0882,1.4882) 3501926 +dictGet dict_array (31.0924,3.42242) 3501821 +dictGet dict_array (31.0927,2.67448) 3501793 +dictGet dict_array (31.0936,1.12292) 3498022 +dictGet dict_array (31.0952,-0.336928) 3498012 +dictGet dict_array (31.0978,3.48482) 3501826 +dictGet dict_array (31.1107,3.7513199999999998) 3501826 +dictGet dict_array (31.1156,1.19171) 3501926 +dictGet dict_array (31.1176,0.223509) 3498015 +dictGet dict_array (31.1249,0.946838) 3498022 +dictGet dict_array (31.1267,1.48983) 3501926 +dictGet dict_array (31.138,-0.289981) 3501898 +dictGet dict_array (31.1382,3.02904) 3501793 +dictGet dict_array (31.1475,2.6178) 3501793 +dictGet dict_array (31.1491,1.37873) 3501926 +dictGet dict_array (31.1525,3.72105) 3501826 +dictGet dict_array (31.1526,-1.4129800000000001) 101 +dictGet dict_array (31.1526,-0.186457) 3501898 +dictGet dict_array (31.1539,2.78789) 3501793 +dictGet dict_array (31.1548,-1.08552) 101 +dictGet dict_array (31.1567,-0.0768925) 3501898 +dictGet dict_array (31.1613,1.49617) 3501926 +dictGet dict_array (31.1653,1.03777) 3498022 +dictGet dict_array (31.1662,3.4214700000000002) 3501826 +dictGet dict_array (31.1672,-0.0813169) 3501898 +dictGet dict_array (31.177,0.440843) 3498014 +dictGet dict_array (31.1788,-0.737151) 3501893 +dictGet dict_array (31.1856,-0.144396) 3501898 +dictGet dict_array (31.1959,3.66813) 3501826 +dictGet dict_array (31.1996,-0.353983) 3501898 +dictGet dict_array (31.2019,2.86802) 3501793 +dictGet dict_array (31.2087,2.31245) 3501825 +dictGet dict_array (31.2125,3.2713200000000002) 3501793 +dictGet dict_array (31.2137,-0.108129) 3501898 +dictGet dict_array (31.216,3.9156) 101 +dictGet dict_array (31.2201,-0.202141) 3501898 +dictGet dict_array (31.2285,2.09058) 101 +dictGet dict_array (31.2502,4.01526) 101 +dictGet dict_array (31.2585,3.11524) 3501793 +dictGet dict_array (31.2645,-0.620418) 3501890 +dictGet dict_array (31.2684,2.74277) 3501793 +dictGet dict_array (31.2821,-1.12772) 101 +dictGet dict_array (31.2821,2.46769) 3501825 +dictGet dict_array (31.2887,3.91396) 101 +dictGet dict_array (31.295,1.49942) 3501926 +dictGet dict_array (31.2997,3.46122) 3501826 +dictGet dict_array (31.3017,3.3263) 3501826 +dictGet dict_array (31.3022,3.16754) 3501793 +dictGet dict_array (31.3048,0.364962) 3498014 +dictGet dict_array (31.305,3.1967) 3501793 +dictGet dict_array (31.3061,1.84303) 101 +dictGet dict_array (31.3082,-0.173851) 3501898 +dictGet dict_array (31.3315,3.90932) 101 +dictGet dict_array (31.3351,2.80164) 3501793 +dictGet dict_array (31.3388,0.168765) 3498015 +dictGet dict_array (31.339,0.25535) 3498094 +dictGet dict_array (31.3423,1.7036799999999999) 3501926 +dictGet dict_array (31.349,0.386456) 3498014 +dictGet dict_array (31.3558,-1.04336) 101 +dictGet dict_array (31.3564,0.478876) 3498014 +dictGet dict_array (31.3607,-0.0860507) 3498015 +dictGet dict_array (31.3831,3.84469) 101 +dictGet dict_array (31.3886,-0.731137) 3501890 +dictGet dict_array (31.4043,-0.348907) 5457271 +dictGet dict_array (31.4081,1.47391) 3501926 +dictGet dict_array (31.4176,-0.583645) 5457271 +dictGet dict_array (31.4177,1.36972) 3501926 +dictGet dict_array (31.4182,0.958303) 3498022 +dictGet dict_array (31.4199,3.1738) 3501793 +dictGet dict_array (31.4221,2.74876) 3501825 +dictGet dict_array (31.4301,-0.122643) 3498015 +dictGet dict_array (31.4344,1.00661) 3498022 +dictGet dict_array (31.4375,4.20304) 101 +dictGet dict_array (31.4377,0.289608) 3498094 +dictGet dict_array (31.4379,0.54744) 3498014 +dictGet dict_array (31.4459,3.94945) 101 +dictGet dict_array (31.4559,-0.345063) 5457271 +dictGet dict_array (31.464,0.726129) 3498014 +dictGet dict_array (31.4662,-0.299019) 3498015 +dictGet dict_array (31.4671,1.9605299999999999) 3501794 +dictGet dict_array (31.4673,-0.403676) 5457271 +dictGet dict_array (31.4712,-0.237941) 3498015 +dictGet dict_array (31.4816,0.120264) 3498015 +dictGet dict_array (31.4875,0.323483) 3498014 +dictGet dict_array (31.490099999999998,-0.338163) 5457271 +dictGet dict_array (31.4932,0.517674) 3498014 +dictGet dict_array (31.5112,1.9689299999999998) 3501794 +dictGet dict_array (31.5122,2.92785) 3501791 +dictGet dict_array (31.5151,0.166429) 3498094 +dictGet dict_array (31.5174,2.94802) 3501791 +dictGet dict_array (31.5182,4.18776) 101 +dictGet dict_array (31.5238,1.18793) 3498003 +dictGet dict_array (31.5271,3.07446) 3501791 +dictGet dict_array (31.5393,1.58061) 3501794 +dictGet dict_array (31.5421,3.13711) 3501791 +dictGet dict_array (31.5479,2.39897) 3497970 +dictGet dict_array (31.5519,0.99285) 3498003 +dictGet dict_array (31.5685,3.47987) 3501824 +dictGet dict_array (31.5959,0.437382) 3498014 +dictGet dict_array (31.6003,0.194376) 3498094 +dictGet dict_array (31.6026,2.15457) 3501794 +dictGet dict_array (31.606,2.45365) 3497970 +dictGet dict_array (31.6062,-0.453441) 3501890 +dictGet dict_array (31.6107,1.35247) 3497974 +dictGet dict_array (31.6155,3.85588) 101 +dictGet dict_array (31.6222,2.03326) 3501794 +dictGet dict_array (31.6231,-0.123059) 3498083 +dictGet dict_array (31.6244,1.6885599999999998) 3497974 +dictGet dict_array (31.6459,0.669716) 3498014 +dictGet dict_array (31.6563,-0.0644741) 3498083 +dictGet dict_array (31.6618,-0.551121) 3501890 +dictGet dict_array (31.6725,-0.38922) 3498085 +dictGet dict_array (31.6727,4.10336) 101 +dictGet dict_array (31.6739,4.1391) 101 +dictGet dict_array (31.6897,2.8694699999999997) 3501792 +dictGet dict_array (31.6902,3.98792) 101 +dictGet dict_array (31.6945,2.46687) 3497970 +dictGet dict_array (31.6987,-1.3796) 101 +dictGet dict_array (31.7012,2.34845) 3497970 +dictGet dict_array (31.7036,0.0228348) 3501888 +dictGet dict_array (31.7046,3.68111) 3501824 +dictGet dict_array (31.7055,2.92556) 3501792 +dictGet dict_array (31.7102,1.04532) 3498003 +dictGet dict_array (31.7149,-0.443302) 3498085 +dictGet dict_array (31.7195,2.99311) 3501791 +dictGet dict_array (31.7274,0.166719) 3498094 +dictGet dict_array (31.7565,-0.565382) 3498085 +dictGet dict_array (31.7615,0.771626) 3498014 +dictGet dict_array (31.7739,1.8970099999999999) 3497974 +dictGet dict_array (31.7848,1.2623199999999999) 3498003 +dictGet dict_array (31.7912,-0.788599) 101 +dictGet dict_array (31.8011,2.65853) 3497970 +dictGet dict_array (31.8032,-0.0590108) 3501888 +dictGet dict_array (31.8038,1.9618799999999998) 3497974 +dictGet dict_array (31.8098,-1.46851) 101 +dictGet dict_array (31.8131,3.41982) 3501791 +dictGet dict_array (31.8169,3.31059) 3501791 +dictGet dict_array (31.8202,-0.193692) 3501888 +dictGet dict_array (31.8306,1.57586) 3497974 +dictGet dict_array (31.8382,-0.787948) 101 +dictGet dict_array (31.8433,2.49692) 3497970 +dictGet dict_array (31.8436,2.41851) 3497970 +dictGet dict_array (31.8563,-1.10787) 101 +dictGet dict_array (31.8683,0.996504) 3498002 +dictGet dict_array (31.8693,-0.828142) 101 +dictGet dict_array (31.8723,1.08929) 3498003 +dictGet dict_array (31.8737,0.881127) 3498002 +dictGet dict_array (31.8881,-0.58441) 101 +dictGet dict_array (31.9011,0.121349) 3498094 +dictGet dict_array (31.9066,2.13045) 3497965 +dictGet dict_array (31.9142,1.03368) 3498002 +dictGet dict_array (31.9155,3.38363) 3501791 +dictGet dict_array (31.9168,1.3166) 3498004 +dictGet dict_array (31.9185,-1.11879) 101 +dictGet dict_array (31.9186,-0.647948) 101 +dictGet dict_array (31.9311,3.96928) 101 +dictGet dict_array (31.9335,1.47048) 3497974 +dictGet dict_array (31.9443,-1.36175) 101 +dictGet dict_array (31.9481,2.34231) 3497970 +dictGet dict_array (31.9526,1.36565) 3498004 +dictGet dict_array (31.9629,2.5208399999999997) 3497970 +dictGet dict_array (31.9765,0.975783) 3498002 +dictGet dict_array (31.9923,3.31773) 3501791 +dictGet dict_array (31.9994,0.972816) 3498002 +dictGet dict_array (32.001,3.47425) 3501791 +dictGet dict_array (32.0127,2.13874) 3497965 +dictGet dict_array (32.0244,3.2092) 3501792 +dictGet dict_array (32.029,1.18039) 3498004 +dictGet dict_array (32.0315,0.566073) 3498095 +dictGet dict_array (32.0354,1.0766499999999999) 3498004 +dictGet dict_array (32.0399,-1.11576) 101 +dictGet dict_array (32.053,2.16849) 3497965 +dictGet dict_array (32.0542,0.042328) 3498096 +dictGet dict_array (32.0576,2.47001) 3497970 +dictGet dict_array (32.061,3.7498899999999997) 101 +dictGet dict_array (32.0623,1.25134) 3498004 +dictGet dict_array (32.0626,1.9611399999999999) 3497965 +dictGet dict_array (32.0666,-0.0904247) 3498096 +dictGet dict_array (32.0681,2.28442) 3497970 +dictGet dict_array (32.0692,1.50869) 3497981 +dictGet dict_array (32.0724,4.03314) 101 +dictGet dict_array (32.0729,-0.064324) 101 +dictGet dict_array (32.079,0.293758) 3498094 +dictGet dict_array (32.0847,-1.19814) 101 +dictGet dict_array (32.0974,-0.91927) 101 +dictGet dict_array (32.0979,-0.736979) 101 +dictGet dict_array (32.106,-1.33063) 101 +dictGet dict_array (32.1189,0.246715) 3498094 +dictGet dict_array (32.1207,4.00883) 101 +dictGet dict_array (32.1396,1.12402) 3498004 +dictGet dict_array (32.1413,1.5668) 3497981 +dictGet dict_array (32.143,1.35559) 3498004 +dictGet dict_array (32.1538,1.32881) 3498004 +dictGet dict_array (32.1549,4.06552) 101 +dictGet dict_array (32.1555,-0.79275) 101 +dictGet dict_array (32.163,1.17733) 3498004 +dictGet dict_array (32.1634,2.94273) 3501792 +dictGet dict_array (32.1644,1.85666) 3497965 +dictGet dict_array (32.1745,0.435458) 3498095 +dictGet dict_array (32.1765,1.65149) 3497981 +dictGet dict_array (32.1893,2.08924) 3497965 +dictGet dict_array (32.2024,0.222191) 3498093 +dictGet dict_array (32.2107,1.34379) 3497981 +dictGet dict_array (32.2109,3.9018699999999997) 101 +dictGet dict_array (32.2123,1.85233) 3497965 +dictGet dict_array (32.2144,3.72534) 101 +dictGet dict_array (32.2218,2.5386699999999998) 3497970 +dictGet dict_array (32.2279,2.84267) 3497245 +dictGet dict_array (32.2345,3.33295) 3501792 +dictGet dict_array (32.2435,3.85283) 101 +dictGet dict_array (32.2527,-0.480608) 101 +dictGet dict_array (32.2566,-0.837882) 101 +dictGet dict_array (32.2627,2.57708) 3497970 +dictGet dict_array (32.2733,0.244931) 3498096 +dictGet dict_array (32.2761,4.05808) 101 +dictGet dict_array (32.2764,3.78472) 101 +dictGet dict_array (32.2814,-1.26011) 101 +dictGet dict_array (32.2861,3.02427) 3497245 +dictGet dict_array (32.2924,0.928609) 3498004 +dictGet dict_array (32.2963,-0.78543) 101 +dictGet dict_array (32.3039,3.21175) 3501792 +dictGet dict_array (32.3107,0.698287) 3498004 +dictGet dict_array (32.3138,0.0595677) 3498106 +dictGet dict_array (32.3339,0.707056) 3498004 +dictGet dict_array (32.3351,0.415474) 3498106 +dictGet dict_array (32.342,-0.681023) 101 +dictGet dict_array (32.3463,1.83196) 3497126 +dictGet dict_array (32.3494,2.43799) 3497114 +dictGet dict_array (32.3524,3.47049) 3501822 +dictGet dict_array (32.3531,2.33115) 3497114 +dictGet dict_array (32.3602,0.116106) 3498106 +dictGet dict_array (32.3612,1.1598) 3498004 +dictGet dict_array (32.3689,3.34847) 3501822 +dictGet dict_array (32.3695,0.734055) 3498004 +dictGet dict_array (32.3825,3.85017) 101 +dictGet dict_array (32.3835,-1.25491) 101 +dictGet dict_array (32.4018,-0.728568) 101 +dictGet dict_array (32.4044,2.96727) 3497245 +dictGet dict_array (32.4101,2.9988) 3497245 +dictGet dict_array (32.417,-1.12908) 101 +dictGet dict_array (32.4172,4.1952) 101 +dictGet dict_array (32.4239,2.49512) 3497245 +dictGet dict_array (32.4258,4.05137) 101 +dictGet dict_array (32.4264,-0.427357) 101 +dictGet dict_array (32.4274,3.59377) 3501822 +dictGet dict_array (32.4286,-1.24757) 101 +dictGet dict_array (32.4294,3.0665) 3497245 +dictGet dict_array (32.4333,-0.353347) 101 +dictGet dict_array (32.4391,3.64421) 3501822 +dictGet dict_array (32.4401,3.70635) 3501822 +dictGet dict_array (32.45,1.68918) 3497126 +dictGet dict_array (32.4507,-0.133471) 101 +dictGet dict_array (32.4592,0.976458) 3498105 +dictGet dict_array (32.4595,1.89135) 3497126 +dictGet dict_array (32.4604,0.280248) 3498106 +dictGet dict_array (32.4835,0.472731) 3498106 +dictGet dict_array (32.4855,2.01938) 3497126 +dictGet dict_array (32.4872,2.01697) 3497126 +dictGet dict_array (32.4911,0.613106) 3498105 +dictGet dict_array (32.4918,2.17834) 3497114 +dictGet dict_array (32.4947,2.34595) 3497114 +dictGet dict_array (32.5035,2.92234) 3497245 +dictGet dict_array (32.5132,-0.331206) 101 +dictGet dict_array (32.5156,-0.412604) 3501887 +dictGet dict_array (32.5158,2.9067499999999997) 3497245 +dictGet dict_array (32.5249,2.44519) 3497114 +dictGet dict_array (32.5293,-0.790952) 101 +dictGet dict_array (32.5319,3.96854) 101 +dictGet dict_array (32.5518,3.6093) 3501822 +dictGet dict_array (32.5541,3.5225400000000002) 3501822 +dictGet dict_array (32.5569,0.816123) 3498105 +dictGet dict_array (32.5646,1.9775) 3497126 +dictGet dict_array (32.5733,3.81271) 101 +dictGet dict_array (32.5767,0.948327) 3498105 +dictGet dict_array (32.5971,1.76179) 3497126 +dictGet dict_array (32.6035,-0.716157) 101 +dictGet dict_array (32.6087,4.21614) 101 +dictGet dict_array (32.6171,0.024481) 101 +dictGet dict_array (32.6189,-0.775391) 101 +dictGet dict_array (32.6198,2.92081) 3497167 +dictGet dict_array (32.621,-0.970784) 101 +dictGet dict_array (32.6266,0.650009) 3498105 +dictGet dict_array (32.6315,2.15144) 3497126 +dictGet dict_array (32.6385,-0.436803) 101 +dictGet dict_array (32.6449,-0.191292) 101 +dictGet dict_array (32.6535,2.10385) 3497126 +dictGet dict_array (32.6592,3.49973) 3501822 +dictGet dict_array (32.6598,2.5980600000000003) 3497114 +dictGet dict_array (32.6612,2.95681) 3497167 +dictGet dict_array (32.6636,-0.57235) 101 +dictGet dict_array (32.669,-0.382702) 101 +dictGet dict_array (32.6752,1.30748) 3497981 +dictGet dict_array (32.6811,2.9559800000000003) 3497167 +dictGet dict_array (32.6821,0.57336) 3498105 +dictGet dict_array (32.6828,3.91304) 101 +dictGet dict_array (32.6979,3.96868) 101 +dictGet dict_array (32.6983,3.15784) 3497167 +dictGet dict_array (32.7122,0.794293) 3498105 +dictGet dict_array (32.7131,-0.847256) 101 +dictGet dict_array (32.7219,0.883461) 3498105 +dictGet dict_array (32.7228,1.78808) 3497126 +dictGet dict_array (32.7273,-0.206908) 101 +dictGet dict_array (32.7292,0.259331) 3501889 +dictGet dict_array (32.7304,-1.38317) 101 +dictGet dict_array (32.7353,1.01601) 3498105 +dictGet dict_array (32.7354,4.17574) 101 +dictGet dict_array (32.7357,-0.190194) 101 +dictGet dict_array (32.7465,-1.37598) 101 +dictGet dict_array (32.7494,-0.275675) 101 +dictGet dict_array (32.7514,0.128951) 3501889 +dictGet dict_array (32.753,3.44207) 3501822 +dictGet dict_array (32.7686,2.11713) 3497126 +dictGet dict_array (32.7694,1.47159) 3497388 +dictGet dict_array (32.7768,0.0401042) 101 +dictGet dict_array (32.781,-1.34283) 101 +dictGet dict_array (32.7814,1.73876) 3497388 +dictGet dict_array (32.7856,-1.06363) 101 +dictGet dict_array (32.792699999999996,-1.1255600000000001) 101 +dictGet dict_array (32.7941,-0.645447) 101 +dictGet dict_array (32.7946,1.48889) 3497388 +dictGet dict_array (32.797,0.791753) 3501889 +dictGet dict_array (32.7982,-0.537798) 101 +dictGet dict_array (32.8091,2.3611) 3490438 +dictGet dict_array (32.81,1.7130800000000002) 3497388 +dictGet dict_array (32.8174,-0.288322) 101 +dictGet dict_array (32.823,1.6546699999999999) 3497388 +dictGet dict_array (32.8233,1.62108) 3497388 +dictGet dict_array (32.8428,-0.400045) 101 +dictGet dict_array (32.8479,2.13598) 3490438 +dictGet dict_array (32.8524,0.199902) 3501889 +dictGet dict_array (32.8543,3.23553) 3501820 +dictGet dict_array (32.8562,1.31371) 3498117 +dictGet dict_array (32.87,1.44256) 3498117 +dictGet dict_array (32.8789,2.38192) 3490438 +dictGet dict_array (32.8812,2.20734) 3497128 +dictGet dict_array (32.8815,-0.54427) 101 +dictGet dict_array (32.8853,2.4859) 3497128 +dictGet dict_array (32.8909,0.513964) 3501889 +dictGet dict_array (32.9035,2.38999) 3490438 +dictGet dict_array (32.9097,2.48131) 3497128 +dictGet dict_array (32.928,-0.943269) 101 +dictGet dict_array (32.9322,1.13165) 3498104 +dictGet dict_array (32.9348,1.22606) 3498117 +dictGet dict_array (32.9417,3.77998) 3501822 +dictGet dict_array (32.9428,3.11936) 3497167 +dictGet dict_array (32.9482,1.18092) 3498118 +dictGet dict_array (32.9506,0.0609364) 101 +dictGet dict_array (32.953,-0.828308) 101 +dictGet dict_array (32.9593,3.5209099999999998) 3501822 +dictGet dict_array (32.9617,2.07711) 3497128 +dictGet dict_array (32.966,0.693749) 3498104 +dictGet dict_array (32.9668,-0.716432) 101 +dictGet dict_array (32.9702,1.98555) 3497127 +dictGet dict_array (32.9782,1.73819) 3497388 +dictGet dict_array (32.9805,3.71151) 3501822 +dictGet dict_array (32.9821,2.97225) 3497167 +dictGet dict_array (32.995,-0.830301) 101 +dictGet dict_array (33.0234,0.770848) 3498104 +dictGet dict_array (33.0312,-0.340964) 101 +dictGet dict_array (33.0366,-0.756795) 101 +dictGet dict_array (33.0438,0.812871) 3498118 +dictGet dict_array (33.0455,1.84843) 3497127 +dictGet dict_array (33.0498,0.0913292) 101 +dictGet dict_array (33.0506,1.53739) 3497364 +dictGet dict_array (33.0554,2.4265) 3497363 +dictGet dict_array (33.0741,3.61332) 3501822 +dictGet dict_array (33.0765,-0.179985) 101 +dictGet dict_array (33.087,1.46465) 3497399 +dictGet dict_array (33.0906,-0.620383) 101 +dictGet dict_array (33.1047,-1.28027) 101 +dictGet dict_array (33.1072,1.96303) 3497127 +dictGet dict_array (33.1081,-0.897874) 101 +dictGet dict_array (33.1122,1.8950200000000001) 3497127 +dictGet dict_array (33.1237,2.63993) 3497165 +dictGet dict_array (33.1238,0.753963) 3498118 +dictGet dict_array (33.1257,0.495668) 3498102 +dictGet dict_array (33.1258,1.78341) 3497364 +dictGet dict_array (33.127,2.59646) 3497166 +dictGet dict_array (33.1324,-1.23742) 101 +dictGet dict_array (33.1359,3.83491) 101 +dictGet dict_array (33.1628,-0.379588) 101 +dictGet dict_array (33.1679,1.25601) 3498117 +dictGet dict_array (33.1688,-1.35553) 101 +dictGet dict_array (33.181,2.10943) 3497363 +dictGet dict_array (33.1871,2.81171) 3497165 +dictGet dict_array (33.1877,0.771297) 3498118 +dictGet dict_array (33.1883,-0.204797) 101 +dictGet dict_array (33.1886,3.27998) 3501820 +dictGet dict_array (33.1955,0.708907) 3498118 +dictGet dict_array (33.2044,-0.769275) 101 +dictGet dict_array (33.2182,3.36103) 3501820 +dictGet dict_array (33.2192,3.43586) 3501822 +dictGet dict_array (33.2322,-0.916753) 101 +dictGet dict_array (33.2359,-0.81321) 101 +dictGet dict_array (33.238,0.635072) 3498111 +dictGet dict_array (33.2398,3.02588) 3497165 +dictGet dict_array (33.2469,2.35698) 3497363 +dictGet dict_array (33.247,2.3327) 3497363 +dictGet dict_array (33.2579,2.8027100000000003) 3497165 +dictGet dict_array (33.2607,0.321082) 101 +dictGet dict_array (33.2653,0.243336) 101 +dictGet dict_array (33.2758,0.831836) 3498118 +dictGet dict_array (33.2771,0.886536) 3498118 +dictGet dict_array (33.2914,1.16026) 3498117 +dictGet dict_array (33.2914,1.38882) 3497399 +dictGet dict_array (33.2982,-1.16604) 101 +dictGet dict_array (33.2985,0.842556) 3498112 +dictGet dict_array (33.3005,2.8338900000000002) 3497165 +dictGet dict_array (33.305,0.0969475) 101 +dictGet dict_array (33.3072,3.82163) 101 +dictGet dict_array (33.312,3.41475) 3501820 +dictGet dict_array (33.3129,2.46048) 3497166 +dictGet dict_array (33.3134,3.46863) 3501820 +dictGet dict_array (33.3203,2.33139) 3497166 +dictGet dict_array (33.324,0.433701) 101 +dictGet dict_array (33.3338,2.44705) 3497166 +dictGet dict_array (33.337,4.06475) 101 +dictGet dict_array (33.3469,1.08172) 3498126 +dictGet dict_array (33.3538,0.717896) 3498112 +dictGet dict_array (33.3618,1.37899) 3497399 +dictGet dict_array (33.3698,0.547744) 3501862 +dictGet dict_array (33.3705,0.957619) 3498112 +dictGet dict_array (33.3821,3.07258) 3497165 +dictGet dict_array (33.3881,3.0626) 3497165 +dictGet dict_array (33.393,-0.816186) 101 +dictGet dict_array (33.3945,0.869508) 3498110 +dictGet dict_array (33.4001,1.24186) 3498117 +dictGet dict_array (33.4008,2.34911) 3497166 +dictGet dict_array (33.4166,-1.2808899999999999) 101 +dictGet dict_array (33.4167,3.0655) 3497165 +dictGet dict_array (33.4204,2.81887) 3497165 +dictGet dict_array (33.4211,1.71128) 3497400 +dictGet dict_array (33.4237,2.91761) 3497165 +dictGet dict_array (33.4266,1.5955599999999999) 3497399 +dictGet dict_array (33.4353,-0.391392) 101 +dictGet dict_array (33.4362,-0.134658) 101 +dictGet dict_array (33.4386,0.15396) 101 +dictGet dict_array (33.4421,-0.50712) 101 +dictGet dict_array (33.452,0.915829) 3498126 +dictGet dict_array (33.463,-0.0882717) 101 +dictGet dict_array (33.464,-1.00949) 101 +dictGet dict_array (33.4692,0.954092) 3498126 +dictGet dict_array (33.4716,1.9538799999999998) 3497400 +dictGet dict_array (33.4756,1.85836) 3497400 +dictGet dict_array (33.4859,4.0751) 101 +dictGet dict_array (33.4899,3.54193) 3501820 +dictGet dict_array (33.4935,3.49794) 3501820 +dictGet dict_array (33.494,-0.983356) 101 +dictGet dict_array (33.4955,-1.28128) 101 +dictGet dict_array (33.4965,-0.278687) 101 +dictGet dict_array (33.4991,0.647491) 3498110 +dictGet dict_array (33.5076,2.2272) 3497424 +dictGet dict_array (33.5079,-0.498199) 101 +dictGet dict_array (33.5157,0.535034) 3501862 +dictGet dict_array (33.5171,2.49677) 3497166 +dictGet dict_array (33.5255,2.4447200000000002) 3497166 +dictGet dict_array (33.526,4.01194) 101 +dictGet dict_array (33.5288,0.789434) 3498110 +dictGet dict_array (33.5356,-1.17671) 101 +dictGet dict_array (33.5402,1.49152) 3497399 +dictGet dict_array (33.5418,3.45757) 3501820 +dictGet dict_array (33.5428,1.90712) 3497400 +dictGet dict_array (33.5556,-0.55741) 101 +dictGet dict_array (33.5564,0.876858) 3498128 +dictGet dict_array (33.5567,-0.10208) 101 +dictGet dict_array (33.5645,-0.124824) 101 +dictGet dict_array (33.5663,3.4872) 3501820 +dictGet dict_array (33.5716,-0.0107611) 101 +dictGet dict_array (33.578,3.55714) 3501820 +dictGet dict_array (33.5826,-0.49076) 101 +dictGet dict_array (33.5909,0.773737) 3498110 +dictGet dict_array (33.5958,2.9619999999999997) 3497425 +dictGet dict_array (33.6193,-0.919755) 101 +dictGet dict_array (33.6313,0.652132) 3498110 +dictGet dict_array (33.632,0.823351) 3498128 +dictGet dict_array (33.66,2.18998) 3497424 +dictGet dict_array (33.6621,0.535395) 3498135 +dictGet dict_array (33.6726,3.19367) 3497438 +dictGet dict_array (33.6912,1.74522) 3497400 +dictGet dict_array (33.705,0.706397) 3498135 +dictGet dict_array (33.7076,0.7622) 3498128 +dictGet dict_array (33.7112,1.70187) 3497400 +dictGet dict_array (33.7246,-1.14837) 101 +dictGet dict_array (33.7326,2.62413) 3497425 +dictGet dict_array (33.7332,2.82137) 3497425 +dictGet dict_array (33.7434,0.394672) 3498135 +dictGet dict_array (33.7443,1.54557) 3497398 +dictGet dict_array (33.7506,1.57317) 3497398 +dictGet dict_array (33.7526,1.8578999999999999) 3497424 +dictGet dict_array (33.766,4.15013) 101 +dictGet dict_array (33.7834,2.41789) 3497439 +dictGet dict_array (33.7864,0.230935) 101 +dictGet dict_array (33.7965,3.05709) 3497438 +dictGet dict_array (33.7998,3.32881) 3497438 +dictGet dict_array (33.8003,2.97338) 3497425 +dictGet dict_array (33.8007,-1.08962) 101 +dictGet dict_array (33.8022,-0.139488) 101 +dictGet dict_array (33.8065,2.70857) 3497425 +dictGet dict_array (33.8169,-0.607788) 101 +dictGet dict_array (33.8203,0.108512) 3501863 +dictGet dict_array (33.8231,-1.03449) 101 +dictGet dict_array (33.8312,3.49458) 3501829 +dictGet dict_array (33.8342,0.297518) 3501863 +dictGet dict_array (33.8352,0.165872) 101 +dictGet dict_array (33.8354,1.87277) 3497424 +dictGet dict_array (33.8371,1.60103) 3497398 +dictGet dict_array (33.8387,1.9968) 3497424 +dictGet dict_array (33.8403,3.5805) 3501829 +dictGet dict_array (33.8414,-0.703067) 101 +dictGet dict_array (33.844,-0.179472) 101 +dictGet dict_array (33.8468,3.40137) 3501829 +dictGet dict_array (33.8509,4.15334) 101 +dictGet dict_array (33.8539,2.38339) 3497439 +dictGet dict_array (33.858,-1.3122500000000001) 101 +dictGet dict_array (33.859,3.72626) 3501829 +dictGet dict_array (33.8616,2.24433) 3497424 +dictGet dict_array (33.8621,3.01035) 3497438 +dictGet dict_array (33.8623,1.17559) 3498129 +dictGet dict_array (33.8682,2.706) 3497425 +dictGet dict_array (33.8684,0.189231) 3501863 +dictGet dict_array (33.872,1.93574) 3497424 +dictGet dict_array (33.8844,3.80404) 3501829 +dictGet dict_array (33.8888,0.594884) 3498135 +dictGet dict_array (33.8946,2.74161) 3497438 +dictGet dict_array (33.9023,0.6239) 3498135 +dictGet dict_array (33.9057,0.873222) 3498136 +dictGet dict_array (33.9157,-1.26607) 101 +dictGet dict_array (33.92,2.06848) 3497397 +dictGet dict_array (33.9298,-0.00526229) 101 +dictGet dict_array (33.932,3.07063) 3497438 +dictGet dict_array (33.9322,0.629385) 3501864 +dictGet dict_array (33.9367,-1.41955) 101 +dictGet dict_array (33.937,1.42532) 3498173 +dictGet dict_array (33.9375,1.1467100000000001) 3498159 +dictGet dict_array (33.9434,-1.05739) 101 +dictGet dict_array (33.9477,3.34809) 3501829 +dictGet dict_array (33.95,2.21715) 3497397 +dictGet dict_array (33.955799999999996,0.305176) 3501859 +dictGet dict_array (33.9686,-0.28273) 101 +dictGet dict_array (33.9703,4.1255) 3501829 +dictGet dict_array (33.9707,3.08199) 3497438 +dictGet dict_array (33.9754,1.06203) 3498159 +dictGet dict_array (33.9757,3.72468) 3501829 +dictGet dict_array (33.9775,-0.0440599) 101 +dictGet dict_array (33.9777,-0.251484) 101 +dictGet dict_array (33.9789,-0.339374) 101 +dictGet dict_array (33.9849,2.54515) 3497425 +dictGet dict_array (33.9885,-0.318557) 101 +dictGet dict_array (33.9977,1.07175) 3498159 +dictGet dict_array (33.9984,-0.700517) 101 +dictGet dict_array (34.0149,3.53338) 3501829 +dictGet dict_array (34.0173,3.39155) 3501829 +dictGet dict_array (34.0317,3.9579) 3501829 +dictGet dict_array (34.0369,3.83612) 3501829 +dictGet dict_array (34.043,-0.0887221) 101 +dictGet dict_array (34.0487,1.14252) 3498159 +dictGet dict_array (34.052,1.74832) 3497397 +dictGet dict_array (34.0711,-0.898071) 101 +dictGet dict_array (34.0747,1.55057) 3498173 +dictGet dict_array (34.0803,3.16763) 3497438 +dictGet dict_array (34.0872,3.75555) 3501829 +dictGet dict_array (34.0965,1.62038) 3498173 +dictGet dict_array (34.0977,-0.412691) 101 +dictGet dict_array (34.0986,0.0294206) 101 +dictGet dict_array (34.1072,3.15823) 3497438 +dictGet dict_array (34.1092,3.09599) 3497438 +dictGet dict_array (34.1206,1.04637) 3498160 +dictGet dict_array (34.1209,3.13826) 3497438 +dictGet dict_array (34.1265,3.95881) 3501829 +dictGet dict_array (34.1286,-0.539319) 101 +dictGet dict_array (34.1358,3.67451) 3501829 +dictGet dict_array (34.1428,0.136115) 101 +dictGet dict_array (34.157,1.73522) 3497397 +dictGet dict_array (34.1581,1.48001) 3498172 +dictGet dict_array (34.1682,3.42373) 3501829 +dictGet dict_array (34.1683,-1.26511) 101 +dictGet dict_array (34.1684,4.20007) 101 +dictGet dict_array (34.1854,3.32089) 3501829 +dictGet dict_array (34.2022,0.749536) 3501864 +dictGet dict_array (34.2044,3.04865) 3497438 +dictGet dict_array (34.22,-0.500055) 101 +dictGet dict_array (34.2249,0.743775) 3501864 +dictGet dict_array (34.2254,1.34702) 3498172 +dictGet dict_array (34.2355,-0.898843) 101 +dictGet dict_array (34.2394,2.0203699999999998) 3497439 +dictGet dict_array (34.2466,1.83785) 3498251 +dictGet dict_array (34.247,4.09563) 101 +dictGet dict_array (34.2508,2.61312) 3497439 +dictGet dict_array (34.2517,1.69642) 3498251 +dictGet dict_array (34.2564,4.13033) 101 +dictGet dict_array (34.2574,4.18928) 101 +dictGet dict_array (34.2614,-0.478719) 101 +dictGet dict_array (34.2625,2.38088) 3497439 +dictGet dict_array (34.2666,3.1503) 3501829 +dictGet dict_array (34.271,4.02223) 101 +dictGet dict_array (34.2727,0.514755) 101 +dictGet dict_array (34.278,1.98929) 3497439 +dictGet dict_array (34.2798,-0.199208) 101 +dictGet dict_array (34.2804,2.05184) 3497439 +dictGet dict_array (34.2945,-1.11051) 101 +dictGet dict_array (34.3168,-0.0829721) 101 +dictGet dict_array (34.3345,3.4358) 3501829 +dictGet dict_array (34.3377,1.13527) 3498162 +dictGet dict_array (34.3383,1.27891) 3498161 +dictGet dict_array (34.3391,1.47945) 3498161 +dictGet dict_array (34.3441,0.627014) 101 +dictGet dict_array (34.347,2.4853) 3497439 +dictGet dict_array (34.3514,2.16247) 3497439 +dictGet dict_array (34.3627,2.64533) 3497439 +dictGet dict_array (34.3682,-0.227501) 101 +dictGet dict_array (34.3756,4.21248) 101 +dictGet dict_array (34.379,3.96604) 101 +dictGet dict_array (34.3827,1.7518) 3498251 +dictGet dict_array (34.3912,2.8834) 3501830 +dictGet dict_array (34.3919,0.668829) 101 +dictGet dict_array (34.3949,2.00338) 3497439 +dictGet dict_array (34.3987,0.557268) 101 +dictGet dict_array (34.4111,0.768558) 101 +dictGet dict_array (34.4119,2.8742) 3501830 +dictGet dict_array (34.416,3.50841) 3501829 +dictGet dict_array (34.4212,1.24916) 3498161 +dictGet dict_array (34.4251,0.457029) 101 +dictGet dict_array (34.4274,-0.902559) 101 +dictGet dict_array (34.4325,4.03159) 101 +dictGet dict_array (34.438,1.63994) 3498251 +dictGet dict_array (34.4403,-0.177594) 101 +dictGet dict_array (34.4421,0.726712) 101 +dictGet dict_array (34.4517,2.98611) 3501830 +dictGet dict_array (34.4658,-1.312) 101 +dictGet dict_array (34.4732,-0.0681338) 101 +dictGet dict_array (34.4752,2.81646) 3501830 +dictGet dict_array (34.4914,2.3858) 3497439 +dictGet dict_array (34.4923,0.855231) 101 +dictGet dict_array (34.5235,1.78468) 3498251 +dictGet dict_array (34.5305,4.10608) 101 +dictGet dict_array (34.5389,0.621937) 101 +dictGet dict_array (34.5406,3.17145) 101 +dictGet dict_array (34.5434,-0.56306) 101 +dictGet dict_array (34.5449,3.13311) 3501829 +dictGet dict_array (34.5491,2.31572) 3497439 +dictGet dict_array (34.5539,2.94028) 3501830 +dictGet dict_array (34.5546,-0.208825) 101 +dictGet dict_array (34.5549,3.78486) 101 +dictGet dict_array (34.5676,0.307148) 101 +dictGet dict_array (34.5743,1.5217399999999999) 3501838 +dictGet dict_array (34.5775,3.48046) 101 +dictGet dict_array (34.5815,2.5243700000000002) 3501830 +dictGet dict_array (34.5841,4.21191) 101 +dictGet dict_array (34.5887,2.65083) 3501830 +dictGet dict_array (34.5937,3.2143) 101 +dictGet dict_array (34.6013,-1.0612) 101 +dictGet dict_array (34.6089,1.36066) 3501838 +dictGet dict_array (34.6103,3.40227) 101 +dictGet dict_array (34.6128,1.92276) 3498251 +dictGet dict_array (34.6175,2.43627) 3498251 +dictGet dict_array (34.6209,3.43776) 101 +dictGet dict_array (34.6234,2.60237) 3501830 +dictGet dict_array (34.6275,3.52479) 101 +dictGet dict_array (34.635,0.568558) 101 +dictGet dict_array (34.6373,2.37692) 3498251 +dictGet dict_array (34.6375,3.52234) 101 +dictGet dict_array (34.6426,2.12397) 3498251 +dictGet dict_array (34.6513,2.80915) 3501830 +dictGet dict_array (34.6632,2.30039) 3498251 +dictGet dict_array (34.6691,1.86582) 3498251 +dictGet dict_array (34.6739,0.15342) 101 +dictGet dict_array (34.6825,0.0499679) 101 +dictGet dict_array (34.6893,0.454326) 101 +dictGet dict_array (34.6957,-0.358598) 101 +dictGet dict_array (34.6986,0.562679) 101 +dictGet dict_array (34.712,1.12114) 101 +dictGet dict_array (34.7126,-0.0057301) 101 +dictGet dict_array (34.7137,0.0248501) 101 +dictGet dict_array (34.7162,1.15623) 101 +dictGet dict_array (34.7258,3.95142) 101 +dictGet dict_array (34.7347,3.5232099999999997) 101 +dictGet dict_array (34.7363,2.23374) 3501830 +dictGet dict_array (34.7375,0.397841) 101 +dictGet dict_array (34.7423,3.09198) 101 +dictGet dict_array (34.7452,3.09029) 101 +dictGet dict_array (34.7539,-1.06943) 101 +dictGet dict_array (34.7733,-0.00912717) 101 +dictGet dict_array (34.774,2.71088) 3501830 +dictGet dict_array (34.7771,1.46009) 3501835 +dictGet dict_array (34.7782,-1.28308) 101 +dictGet dict_array (34.7924,3.63564) 101 +dictGet dict_array (34.7939,-0.416676) 101 +dictGet dict_array (34.7964,-0.401773) 101 +dictGet dict_array (34.7974,0.0286873) 101 +dictGet dict_array (34.7975,3.05965) 101 +dictGet dict_array (34.8037,3.07263) 101 +dictGet dict_array (34.8254,-0.390284) 101 +dictGet dict_array (34.828,1.91869) 3498251 +dictGet dict_array (34.8289,3.71058) 101 +dictGet dict_array (34.8403,2.14606) 3501835 +dictGet dict_array (34.8437,2.20617) 3501830 +dictGet dict_array (34.8469,2.38435) 3501830 +dictGet dict_array (34.86,1.45705) 101 +dictGet dict_array (34.8612,0.914248) 101 +dictGet dict_array (34.8663,3.4215400000000002) 101 +dictGet dict_array (34.8724,-0.375144) 101 +dictGet dict_array (34.8795,3.29317) 101 +dictGet dict_array (34.8823,1.21988) 101 +dictGet dict_array (34.8834,1.07657) 101 +dictGet dict_array (34.8837,0.157648) 101 +dictGet dict_array (34.8871,-0.9755) 101 +dictGet dict_array (34.8871,1.8943699999999999) 3501835 +dictGet dict_array (34.889,3.36756) 101 +dictGet dict_array (34.8907,1.24874) 101 +dictGet dict_array (34.8965,3.13508) 101 +dictGet dict_array (34.9042,2.62092) 101 +dictGet dict_array (34.9055,-0.0448967) 101 +dictGet dict_array (34.9122,0.110576) 101 +dictGet dict_array (34.9228,3.60183) 101 +dictGet dict_array (34.9237,1.21715) 101 +dictGet dict_array (34.9296,1.70459) 3501835 +dictGet dict_array (34.941,-1.14663) 101 +dictGet dict_array (34.9448,1.18923) 101 +dictGet dict_array (34.9462,3.81678) 101 +dictGet dict_array (34.9466,0.593463) 101 +dictGet dict_array (34.9485,0.150307) 101 +dictGet dict_array (34.9542,0.487238) 101 +dictGet dict_array (34.9559,2.03473) 3501835 +dictGet dict_array (34.9671,-0.960225) 101 +dictGet dict_array (34.9711,2.63444) 101 +dictGet dict_array (34.9892,0.354775) 101 +dictGet dict_array (34.9907,1.40724) 101 +dictGet dict_array (34.9916,-0.00173097) 101 +dictGet dict_array (34.9919,2.06167) 101 diff --git a/tests/queries/0_stateless/01037_polygon_dicts_correctness_fast.sh b/tests/queries/0_stateless/01037_polygon_dicts_correctness_fast.sh index f6880ae5009..c9cd151a2d9 100755 --- a/tests/queries/0_stateless/01037_polygon_dicts_correctness_fast.sh +++ b/tests/queries/0_stateless/01037_polygon_dicts_correctness_fast.sh @@ -1,5 +1,5 @@ #!/usr/bin/env bash -# Tags: no-debug, no-parallel +# Tags: no-debug CURDIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd) # shellcheck source=../shell_config.sh @@ -12,20 +12,17 @@ declare -a SearchTypes=("POLYGON_INDEX_EACH" "POLYGON_INDEX_CELL") tar -xf "${CURDIR}"/01037_test_data_perf.tar.gz -C "${CURDIR}" $CLICKHOUSE_CLIENT -n --query=" -DROP DATABASE IF EXISTS test_01037; -CREATE DATABASE test_01037; -DROP TABLE IF EXISTS test_01037.points; -CREATE TABLE test_01037.points (x Float64, y Float64) ENGINE = Memory; +CREATE TABLE points (x Float64, y Float64) ENGINE = Memory; " -$CLICKHOUSE_CLIENT --query="INSERT INTO test_01037.points FORMAT TSV" --min_chunk_bytes_for_parallel_parsing=10485760 --max_insert_block_size=100000 < "${CURDIR}/01037_point_data" +$CLICKHOUSE_CLIENT --query="INSERT INTO points FORMAT TSV" --min_chunk_bytes_for_parallel_parsing=10485760 --max_insert_block_size=100000 < "${CURDIR}/01037_point_data" rm "${CURDIR}"/01037_point_data $CLICKHOUSE_CLIENT -n --query=" -DROP TABLE IF EXISTS test_01037.polygons_array; +DROP TABLE IF EXISTS polygons_array; -CREATE TABLE test_01037.polygons_array +CREATE TABLE polygons_array ( key Array(Array(Array(Array(Float64)))), name String, @@ -34,7 +31,7 @@ CREATE TABLE test_01037.polygons_array ENGINE = Memory; " -$CLICKHOUSE_CLIENT --query="INSERT INTO test_01037.polygons_array FORMAT JSONEachRow" --min_chunk_bytes_for_parallel_parsing=10485760 --max_insert_block_size=100000 < "${CURDIR}/01037_polygon_data" +$CLICKHOUSE_CLIENT --query="INSERT INTO polygons_array FORMAT JSONEachRow" --min_chunk_bytes_for_parallel_parsing=10485760 --max_insert_block_size=100000 < "${CURDIR}/01037_polygon_data" rm "${CURDIR}"/01037_polygon_data @@ -43,27 +40,23 @@ do outputFile="${TMP_DIR}/results${type}.out" $CLICKHOUSE_CLIENT -n --query=" - DROP DICTIONARY IF EXISTS test_01037.dict_array; + DROP DICTIONARY IF EXISTS dict_array; - CREATE DICTIONARY test_01037.dict_array + CREATE DICTIONARY dict_array ( key Array(Array(Array(Array(Float64)))), name String DEFAULT 'qqq', value UInt64 DEFAULT 101 ) PRIMARY KEY key - SOURCE(CLICKHOUSE(HOST 'localhost' PORT tcpPort() USER 'default' TABLE 'polygons_array' PASSWORD '' DB 'test_01037')) + SOURCE(CLICKHOUSE(HOST 'localhost' PORT tcpPort() USER 'default' TABLE 'polygons_array' PASSWORD '' DB currentDatabase())) LIFETIME(0) LAYOUT($type()); - select 'dictGet', 'test_01037.dict_array' as dict_name, tuple(x, y) as key, - dictGet(dict_name, 'value', key) from test_01037.points order by x, y; + select 'dictGet', 'dict_array' as dict_name, tuple(x, y) as key, + dictGet(dict_name, 'value', key) from points order by x, y; " > "$outputFile" diff -q "${CURDIR}/01037_polygon_dicts_correctness_fast.ans" "$outputFile" done -$CLICKHOUSE_CLIENT -n --query=" -DROP TABLE test_01037.points; -DROP DATABASE test_01037; -" diff --git a/tests/queries/0_stateless/01038_dictionary_lifetime_min_zero_sec.sh b/tests/queries/0_stateless/01038_dictionary_lifetime_min_zero_sec.sh index 1e754dce786..66732205f95 100755 --- a/tests/queries/0_stateless/01038_dictionary_lifetime_min_zero_sec.sh +++ b/tests/queries/0_stateless/01038_dictionary_lifetime_min_zero_sec.sh @@ -5,13 +5,9 @@ CURDIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd) # shellcheck source=../shell_config.sh . "$CURDIR"/../shell_config.sh -$CLICKHOUSE_CLIENT --query "DROP DATABASE IF EXISTS db_01038" - -$CLICKHOUSE_CLIENT --query "CREATE DATABASE db_01038" - $CLICKHOUSE_CLIENT --query " -CREATE TABLE db_01038.table_for_dict +CREATE TABLE ${CLICKHOUSE_DATABASE}.table_for_dict ( key_column UInt64, value Float64 @@ -19,34 +15,34 @@ CREATE TABLE db_01038.table_for_dict ENGINE = MergeTree() ORDER BY key_column" -$CLICKHOUSE_CLIENT --query "INSERT INTO db_01038.table_for_dict VALUES (1, 1.1)" +$CLICKHOUSE_CLIENT --query "INSERT INTO ${CLICKHOUSE_DATABASE}.table_for_dict VALUES (1, 1.1)" $CLICKHOUSE_CLIENT --query " -CREATE DICTIONARY db_01038.dict_with_zero_min_lifetime +CREATE DICTIONARY ${CLICKHOUSE_DATABASE}.dict_with_zero_min_lifetime ( key_column UInt64, value Float64 DEFAULT 77.77 ) PRIMARY KEY key_column -SOURCE(CLICKHOUSE(HOST 'localhost' PORT tcpPort() USER 'default' TABLE 'table_for_dict' DB 'db_01038')) +SOURCE(CLICKHOUSE(HOST 'localhost' PORT tcpPort() USER 'default' TABLE 'table_for_dict' DB '${CLICKHOUSE_DATABASE}')) LIFETIME(1) LAYOUT(FLAT())" -$CLICKHOUSE_CLIENT --query "SELECT dictGetFloat64('db_01038.dict_with_zero_min_lifetime', 'value', toUInt64(1))" +$CLICKHOUSE_CLIENT --query "SELECT dictGetFloat64('${CLICKHOUSE_DATABASE}.dict_with_zero_min_lifetime', 'value', toUInt64(1))" -$CLICKHOUSE_CLIENT --query "SELECT dictGetFloat64('db_01038.dict_with_zero_min_lifetime', 'value', toUInt64(2))" +$CLICKHOUSE_CLIENT --query "SELECT dictGetFloat64('${CLICKHOUSE_DATABASE}.dict_with_zero_min_lifetime', 'value', toUInt64(2))" -$CLICKHOUSE_CLIENT --query "INSERT INTO db_01038.table_for_dict VALUES (2, 2.2)" +$CLICKHOUSE_CLIENT --query "INSERT INTO ${CLICKHOUSE_DATABASE}.table_for_dict VALUES (2, 2.2)" function check() { - query_result=$($CLICKHOUSE_CLIENT --query "SELECT dictGetFloat64('db_01038.dict_with_zero_min_lifetime', 'value', toUInt64(2))") + query_result=$($CLICKHOUSE_CLIENT --query "SELECT dictGetFloat64('${CLICKHOUSE_DATABASE}.dict_with_zero_min_lifetime', 'value', toUInt64(2))") while [ "$query_result" != "2.2" ] do - query_result=$($CLICKHOUSE_CLIENT --query "SELECT dictGetFloat64('db_01038.dict_with_zero_min_lifetime', 'value', toUInt64(2))") + query_result=$($CLICKHOUSE_CLIENT --query "SELECT dictGetFloat64('${CLICKHOUSE_DATABASE}.dict_with_zero_min_lifetime', 'value', toUInt64(2))") done } @@ -55,8 +51,6 @@ export -f check; timeout 10 bash -c check -$CLICKHOUSE_CLIENT --query "SELECT dictGetFloat64('db_01038.dict_with_zero_min_lifetime', 'value', toUInt64(1))" +$CLICKHOUSE_CLIENT --query "SELECT dictGetFloat64('${CLICKHOUSE_DATABASE}.dict_with_zero_min_lifetime', 'value', toUInt64(1))" -$CLICKHOUSE_CLIENT --query "SELECT dictGetFloat64('db_01038.dict_with_zero_min_lifetime', 'value', toUInt64(2))" - -$CLICKHOUSE_CLIENT --query "DROP DATABASE IF EXISTS db_01038" +$CLICKHOUSE_CLIENT --query "SELECT dictGetFloat64('${CLICKHOUSE_DATABASE}.dict_with_zero_min_lifetime', 'value', toUInt64(2))" diff --git a/tests/queries/0_stateless/01040_dictionary_invalidate_query_switchover_long.sh b/tests/queries/0_stateless/01040_dictionary_invalidate_query_switchover_long.sh index 6856f952a47..d558fbf465e 100755 --- a/tests/queries/0_stateless/01040_dictionary_invalidate_query_switchover_long.sh +++ b/tests/queries/0_stateless/01040_dictionary_invalidate_query_switchover_long.sh @@ -1,17 +1,12 @@ #!/usr/bin/env bash -# Tags: long, no-parallel +# Tags: long CURDIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd) # shellcheck source=../shell_config.sh . "$CURDIR"/../shell_config.sh - -$CLICKHOUSE_CLIENT --query "DROP DATABASE IF EXISTS dictdb_01041_01040" - -$CLICKHOUSE_CLIENT --query "CREATE DATABASE dictdb_01041_01040" - $CLICKHOUSE_CLIENT --query " -CREATE TABLE dictdb_01041_01040.dict_invalidate +CREATE TABLE dict_invalidate ENGINE = Memory AS SELECT 122 as dummy, @@ -20,31 +15,31 @@ FROM system.one" $CLICKHOUSE_CLIENT --query " -CREATE DICTIONARY dictdb_01041_01040.invalidate +CREATE DICTIONARY invalidate ( dummy UInt64, two UInt8 EXPRESSION dummy ) PRIMARY KEY dummy -SOURCE(CLICKHOUSE(HOST 'localhost' PORT tcpPort() USER 'default' TABLE 'dict_invalidate' DB 'dictdb_01041_01040' INVALIDATE_QUERY 'select max(last_time) from dictdb_01041_01040.dict_invalidate')) +SOURCE(CLICKHOUSE(HOST 'localhost' PORT tcpPort() USER 'default' TABLE 'dict_invalidate' DB currentDatabase() INVALIDATE_QUERY 'select max(last_time) from dict_invalidate')) LIFETIME(MIN 0 MAX 1) LAYOUT(FLAT())" -$CLICKHOUSE_CLIENT --query "SELECT dictGetUInt8('dictdb_01041_01040.invalidate', 'two', toUInt64(122))" +$CLICKHOUSE_CLIENT --query "SELECT dictGetUInt8('invalidate', 'two', toUInt64(122))" # No exception happened -$CLICKHOUSE_CLIENT --query "SELECT last_exception FROM system.dictionaries WHERE database = 'dictdb_01041_01040' AND name = 'invalidate'" +$CLICKHOUSE_CLIENT --query "SELECT last_exception FROM system.dictionaries WHERE database = currentDatabase() AND name = 'invalidate'" -$CLICKHOUSE_CLIENT --check_table_dependencies=0 --query "DROP TABLE dictdb_01041_01040.dict_invalidate" +$CLICKHOUSE_CLIENT --check_table_dependencies=0 --query "DROP TABLE dict_invalidate" function check_exception_detected() { - query_result=$($CLICKHOUSE_CLIENT --query "SELECT last_exception FROM system.dictionaries WHERE database = 'dictdb_01041_01040' AND name = 'invalidate'" 2>&1) + query_result=$($CLICKHOUSE_CLIENT --query "SELECT last_exception FROM system.dictionaries WHERE database = currentDatabase() AND name = 'invalidate'" 2>&1) while [ -z "$query_result" ] do - query_result=$($CLICKHOUSE_CLIENT --query "SELECT last_exception FROM system.dictionaries WHERE database = 'dictdb_01041_01040' AND name = 'invalidate'" 2>&1) + query_result=$($CLICKHOUSE_CLIENT --query "SELECT last_exception FROM system.dictionaries WHERE database = currentDatabase() AND name = 'invalidate'" 2>&1) sleep 0.1 done } @@ -53,10 +48,10 @@ function check_exception_detected() export -f check_exception_detected; timeout 30 bash -c check_exception_detected 2> /dev/null -$CLICKHOUSE_CLIENT --query "SELECT last_exception FROM system.dictionaries WHERE database = 'dictdb_01041_01040' AND name = 'invalidate'" 2>&1 | grep -Eo "dictdb_01041_01040.dict_invalidate.*UNKNOWN_TABLE" | wc -l +$CLICKHOUSE_CLIENT --query "SELECT last_exception FROM system.dictionaries WHERE database = currentDatabase() AND name = 'invalidate'" 2>&1 | grep -Eo "dict_invalidate.*UNKNOWN_TABLE" | wc -l $CLICKHOUSE_CLIENT --query " -CREATE TABLE dictdb_01041_01040.dict_invalidate +CREATE TABLE dict_invalidate ENGINE = Memory AS SELECT 133 as dummy, @@ -65,11 +60,11 @@ FROM system.one" function check_exception_fixed() { - query_result=$($CLICKHOUSE_CLIENT --query "SELECT last_exception FROM system.dictionaries WHERE database = 'dictdb_01041_01040' AND name = 'invalidate'" 2>&1) + query_result=$($CLICKHOUSE_CLIENT --query "SELECT last_exception FROM system.dictionaries WHERE database = currentDatabase() AND name = 'invalidate'" 2>&1) while [ "$query_result" ] do - query_result=$($CLICKHOUSE_CLIENT --query "SELECT last_exception FROM system.dictionaries WHERE database = 'dictdb_01041_01040' AND name = 'invalidate'" 2>&1) + query_result=$($CLICKHOUSE_CLIENT --query "SELECT last_exception FROM system.dictionaries WHERE database = currentDatabase() AND name = 'invalidate'" 2>&1) sleep 0.1 done } @@ -78,7 +73,5 @@ export -f check_exception_fixed; # it may take a while until dictionary reloads timeout 60 bash -c check_exception_fixed 2> /dev/null -$CLICKHOUSE_CLIENT --query "SELECT last_exception FROM system.dictionaries WHERE database = 'dictdb_01041_01040' AND name = 'invalidate'" 2>&1 -$CLICKHOUSE_CLIENT --query "SELECT dictGetUInt8('dictdb_01041_01040.invalidate', 'two', toUInt64(133))" - -$CLICKHOUSE_CLIENT --query "DROP DATABASE IF EXISTS dictdb_01041_01040" +$CLICKHOUSE_CLIENT --query "SELECT last_exception FROM system.dictionaries WHERE database = currentDatabase() AND name = 'invalidate'" 2>&1 +$CLICKHOUSE_CLIENT --query "SELECT dictGetUInt8('invalidate', 'two', toUInt64(133))" diff --git a/tests/queries/0_stateless/01042_system_reload_dictionary_reloads_completely.sh b/tests/queries/0_stateless/01042_system_reload_dictionary_reloads_completely.sh index 9d34470c38d..2b075566ac3 100755 --- a/tests/queries/0_stateless/01042_system_reload_dictionary_reloads_completely.sh +++ b/tests/queries/0_stateless/01042_system_reload_dictionary_reloads_completely.sh @@ -1,5 +1,4 @@ #!/usr/bin/env bash -# Tags: no-parallel CURDIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd) # shellcheck source=../shell_config.sh @@ -8,41 +7,37 @@ CURDIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd) set -e -o pipefail # NOTE: dictionaries TTLs works with server timezone, so session_timeout cannot be used -$CLICKHOUSE_CLIENT --session_timezone '' --multiquery <<'EOF' -DROP DATABASE IF EXISTS dictdb_01042; -CREATE DATABASE dictdb_01042; -CREATE TABLE dictdb_01042.table(x Int64, y Int64, insert_time DateTime) ENGINE = MergeTree ORDER BY tuple(); -INSERT INTO dictdb_01042.table VALUES (12, 102, now()); +$CLICKHOUSE_CLIENT --session_timezone '' --multiquery < ', dictGetInt64('dictdb_01042.dict', 'y', toUInt64(12))" +$CLICKHOUSE_CLIENT --query "SELECT '12 -> ', dictGetInt64('${CLICKHOUSE_DATABASE}.dict', 'y', toUInt64(12))" -$CLICKHOUSE_CLIENT --query "INSERT INTO dictdb_01042.table VALUES (13, 103, now())" -$CLICKHOUSE_CLIENT --query "INSERT INTO dictdb_01042.table VALUES (14, 104, now() - INTERVAL 1 DAY)" +$CLICKHOUSE_CLIENT --query "INSERT INTO ${CLICKHOUSE_DATABASE}.table VALUES (13, 103, now())" +$CLICKHOUSE_CLIENT --query "INSERT INTO ${CLICKHOUSE_DATABASE}.table VALUES (14, 104, now() - INTERVAL 1 DAY)" -while [ "$(${CLICKHOUSE_CLIENT} --query "SELECT dictGetInt64('dictdb_01042.dict', 'y', toUInt64(13))")" = -1 ] +while [ "$(${CLICKHOUSE_CLIENT} --query "SELECT dictGetInt64('${CLICKHOUSE_DATABASE}.dict', 'y', toUInt64(13))")" = -1 ] do sleep 0.5 done -$CLICKHOUSE_CLIENT --query "SELECT '13 -> ', dictGetInt64('dictdb_01042.dict', 'y', toUInt64(13))" -$CLICKHOUSE_CLIENT --query "SELECT '14 -> ', dictGetInt64('dictdb_01042.dict', 'y', toUInt64(14))" +$CLICKHOUSE_CLIENT --query "SELECT '13 -> ', dictGetInt64('${CLICKHOUSE_DATABASE}.dict', 'y', toUInt64(13))" +$CLICKHOUSE_CLIENT --query "SELECT '14 -> ', dictGetInt64('${CLICKHOUSE_DATABASE}.dict', 'y', toUInt64(14))" -$CLICKHOUSE_CLIENT --query "SYSTEM RELOAD DICTIONARY 'dictdb_01042.dict'" +$CLICKHOUSE_CLIENT --query "SYSTEM RELOAD DICTIONARY '${CLICKHOUSE_DATABASE}.dict'" -$CLICKHOUSE_CLIENT --query "SELECT '12(r) -> ', dictGetInt64('dictdb_01042.dict', 'y', toUInt64(12))" -$CLICKHOUSE_CLIENT --query "SELECT '13(r) -> ', dictGetInt64('dictdb_01042.dict', 'y', toUInt64(13))" -$CLICKHOUSE_CLIENT --query "SELECT '14(r) -> ', dictGetInt64('dictdb_01042.dict', 'y', toUInt64(14))" - -$CLICKHOUSE_CLIENT --query "DROP DATABASE IF EXISTS dictdb_01042" +$CLICKHOUSE_CLIENT --query "SELECT '12(r) -> ', dictGetInt64('${CLICKHOUSE_DATABASE}.dict', 'y', toUInt64(12))" +$CLICKHOUSE_CLIENT --query "SELECT '13(r) -> ', dictGetInt64('${CLICKHOUSE_DATABASE}.dict', 'y', toUInt64(13))" +$CLICKHOUSE_CLIENT --query "SELECT '14(r) -> ', dictGetInt64('${CLICKHOUSE_DATABASE}.dict', 'y', toUInt64(14))" diff --git a/tests/queries/0_stateless/01053_ssd_dictionary.sh b/tests/queries/0_stateless/01053_ssd_dictionary.sh index b49144c9b1a..00e5719a9a9 100755 --- a/tests/queries/0_stateless/01053_ssd_dictionary.sh +++ b/tests/queries/0_stateless/01053_ssd_dictionary.sh @@ -6,8 +6,6 @@ CURDIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd) # shellcheck source=../shell_config.sh . "$CURDIR"/../shell_config.sh -USER_FILES_PATH=$($CLICKHOUSE_CLIENT_BINARY --query "select _path,_file from file('nonexist.txt', 'CSV', 'val1 char')" 2>&1 | grep Exception | awk '{gsub("/nonexist.txt","",$9); print $9}') - $CLICKHOUSE_CLIENT --allow_deprecated_database_ordinary=1 -n --query=" DROP DATABASE IF EXISTS 01053_db; diff --git a/tests/queries/0_stateless/01055_compact_parts_1.sql b/tests/queries/0_stateless/01055_compact_parts_1.sql index ff5ab722e0f..72048c59a41 100644 --- a/tests/queries/0_stateless/01055_compact_parts_1.sql +++ b/tests/queries/0_stateless/01055_compact_parts_1.sql @@ -1,8 +1,3 @@ --- Tags: no-parallel - -drop table if exists mt_compact; -drop table if exists mt_compact_2; - create table mt_compact (a Int, s String) engine = MergeTree order by a partition by a settings index_granularity_bytes = 0; alter table mt_compact modify setting min_rows_for_wide_part = 1000; -- { serverError NOT_IMPLEMENTED } @@ -25,5 +20,3 @@ alter table mt_compact modify setting parts_to_delay_insert = 300; alter table mt_compact modify setting min_rows_for_wide_part = 0; show create table mt_compact; - -drop table mt_compact diff --git a/tests/queries/0_stateless/01060_avro.sh b/tests/queries/0_stateless/01060_avro.sh index 3c70927db25..6ed26c8565f 100755 --- a/tests/queries/0_stateless/01060_avro.sh +++ b/tests/queries/0_stateless/01060_avro.sh @@ -1,5 +1,5 @@ #!/usr/bin/env bash -# Tags: no-parallel, no-fasttest +# Tags: no-fasttest set -e @@ -69,9 +69,6 @@ cat "$DATA_DIR"/simple.null.avro | ${CLICKHOUSE_LOCAL} --input-format Avro --out - - - # output echo '===' output diff --git a/tests/queries/0_stateless/01069_database_memory.sql b/tests/queries/0_stateless/01069_database_memory.sql index 5aab9175c58..5d2fa4ea11e 100644 --- a/tests/queries/0_stateless/01069_database_memory.sql +++ b/tests/queries/0_stateless/01069_database_memory.sql @@ -1,5 +1,3 @@ --- Tags: no-parallel - DROP DATABASE IF EXISTS memory_01069; CREATE DATABASE memory_01069 ENGINE = Memory; SHOW CREATE DATABASE memory_01069; diff --git a/tests/queries/0_stateless/01076_cache_dictionary_datarace_exception_ptr.sh b/tests/queries/0_stateless/01076_cache_dictionary_datarace_exception_ptr.sh index 17068dcbdf9..dcd15718416 100755 --- a/tests/queries/0_stateless/01076_cache_dictionary_datarace_exception_ptr.sh +++ b/tests/queries/0_stateless/01076_cache_dictionary_datarace_exception_ptr.sh @@ -1,5 +1,5 @@ #!/usr/bin/env bash -# Tags: race, no-parallel +# Tags: race # This is a monkey test used to trigger sanitizers. @@ -7,11 +7,8 @@ CURDIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd) # shellcheck source=../shell_config.sh . "$CURDIR"/../shell_config.sh -$CLICKHOUSE_CLIENT --query="DROP DATABASE IF EXISTS dictdb_01076;" -$CLICKHOUSE_CLIENT --query="CREATE DATABASE dictdb_01076;" - $CLICKHOUSE_CLIENT --query=" -CREATE TABLE dictdb_01076.table_datarace +CREATE TABLE ${CLICKHOUSE_DATABASE}.table_datarace ( key_column UUID, value Float64 @@ -21,17 +18,17 @@ ORDER BY key_column; " $CLICKHOUSE_CLIENT --query=" -INSERT INTO dictdb_01076.table_datarace VALUES ('cd5db34f-0c25-4375-b10e-bfb3708ddc72', 1.1), ('cd5db34f-0c25-4375-b10e-bfb3708ddc72', 2.2), ('cd5db34f-0c25-4375-b10e-bfb3708ddc72', 3.3); +INSERT INTO ${CLICKHOUSE_DATABASE}.table_datarace VALUES ('cd5db34f-0c25-4375-b10e-bfb3708ddc72', 1.1), ('cd5db34f-0c25-4375-b10e-bfb3708ddc72', 2.2), ('cd5db34f-0c25-4375-b10e-bfb3708ddc72', 3.3); " $CLICKHOUSE_CLIENT --query=" -CREATE DICTIONARY IF NOT EXISTS dictdb_01076.dict_datarace +CREATE DICTIONARY IF NOT EXISTS ${CLICKHOUSE_DATABASE}.dict_datarace ( key_column UInt64, value Float64 DEFAULT 77.77 ) PRIMARY KEY key_column -SOURCE(CLICKHOUSE(HOST 'localhost' PORT tcpPort() USER 'default' TABLE 'table_datarace' DB 'dictdb_01076')) +SOURCE(CLICKHOUSE(HOST 'localhost' PORT tcpPort() USER 'default' TABLE 'table_datarace' DB '${CLICKHOUSE_DATABASE}')) LIFETIME(1) LAYOUT(CACHE(SIZE_IN_CELLS 10)); " @@ -41,7 +38,7 @@ function thread1() for _ in {1..50} do # This query will be ended with exception, because source dictionary has UUID as a key type. - $CLICKHOUSE_CLIENT --query="SELECT dictGetFloat64('dictdb_01076.dict_datarace', 'value', toUInt64(1));" + $CLICKHOUSE_CLIENT --query="SELECT dictGetFloat64('${CLICKHOUSE_DATABASE}.dict_datarace', 'value', toUInt64(1));" done } @@ -51,7 +48,7 @@ function thread2() for _ in {1..50} do # This query will be ended with exception, because source dictionary has UUID as a key type. - $CLICKHOUSE_CLIENT --query="SELECT dictGetFloat64('dictdb_01076.dict_datarace', 'value', toUInt64(2));" + $CLICKHOUSE_CLIENT --query="SELECT dictGetFloat64('${CLICKHOUSE_DATABASE}.dict_datarace', 'value', toUInt64(2));" done } @@ -67,6 +64,5 @@ wait echo OK -$CLICKHOUSE_CLIENT --query="DROP DICTIONARY dictdb_01076.dict_datarace;" -$CLICKHOUSE_CLIENT --query="DROP TABLE dictdb_01076.table_datarace;" -$CLICKHOUSE_CLIENT --query="DROP DATABASE dictdb_01076;" +$CLICKHOUSE_CLIENT --query="DROP DICTIONARY ${CLICKHOUSE_DATABASE}.dict_datarace;" +$CLICKHOUSE_CLIENT --query="DROP TABLE ${CLICKHOUSE_DATABASE}.table_datarace;" diff --git a/tests/queries/0_stateless/01079_parallel_alter_add_drop_column_zookeeper.sh b/tests/queries/0_stateless/01079_parallel_alter_add_drop_column_zookeeper.sh index bfdea95fa9e..f05a0fed965 100755 --- a/tests/queries/0_stateless/01079_parallel_alter_add_drop_column_zookeeper.sh +++ b/tests/queries/0_stateless/01079_parallel_alter_add_drop_column_zookeeper.sh @@ -1,5 +1,5 @@ #!/usr/bin/env bash -# Tags: zookeeper, no-parallel, no-fasttest +# Tags: zookeeper, no-fasttest, no-parallel CURDIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd) # shellcheck source=../shell_config.sh @@ -63,7 +63,7 @@ export -f optimize_thread; export -f insert_thread; -TIMEOUT=30 +TIMEOUT=20 # Sometimes we detach and attach tables timeout $TIMEOUT bash -c alter_thread 2> /dev/null & diff --git a/tests/queries/0_stateless/01079_parallel_alter_modify_zookeeper_long.sh b/tests/queries/0_stateless/01079_parallel_alter_modify_zookeeper_long.sh index ba8d89aad3c..399c9e488a4 100755 --- a/tests/queries/0_stateless/01079_parallel_alter_modify_zookeeper_long.sh +++ b/tests/queries/0_stateless/01079_parallel_alter_modify_zookeeper_long.sh @@ -1,5 +1,5 @@ #!/usr/bin/env bash -# Tags: long, zookeeper, no-parallel, no-fasttest +# Tags: long, zookeeper, no-fasttest CURDIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd) # shellcheck source=../shell_config.sh @@ -76,7 +76,7 @@ export -f insert_thread; export -f select_thread; -TIMEOUT=30 +TIMEOUT=20 # Selects should run successfully diff --git a/tests/queries/0_stateless/01083_expressions_in_engine_arguments.sql b/tests/queries/0_stateless/01083_expressions_in_engine_arguments.sql index bdfbf2a47cf..697843be27f 100644 --- a/tests/queries/0_stateless/01083_expressions_in_engine_arguments.sql +++ b/tests/queries/0_stateless/01083_expressions_in_engine_arguments.sql @@ -28,7 +28,7 @@ CREATE TABLE url (n UInt64, col String) ENGINE=URL ( replace ( - 'https://localhost:8443/?query=' || 'select n, _table from ' || currentDatabase() || '.merge format CSV', ' ', '+' + 'https://localhost:' || getServerPort('https_port') || '/?query=' || 'select n, _table from ' || currentDatabase() || '.merge format CSV', ' ', '+' ), CSV ); @@ -39,7 +39,7 @@ CREATE VIEW view AS SELECT toInt64(n) as n FROM (SELECT toString(n) as n from me SELECT nonexistentsomething; -- { serverError UNKNOWN_IDENTIFIER } CREATE DICTIONARY dict (n UInt64, col String DEFAULT '42') PRIMARY KEY n -SOURCE(CLICKHOUSE(HOST 'localhost' PORT 9440 SECURE 1 USER 'default' TABLE 'url')) LIFETIME(1) LAYOUT(CACHE(SIZE_IN_CELLS 1)); +SOURCE(CLICKHOUSE(HOST 'localhost' PORT getServerPort('tcp_port_secure') SECURE 1 USER 'default' TABLE 'url')) LIFETIME(1) LAYOUT(CACHE(SIZE_IN_CELLS 1)); -- dict --> url --> merge |-> distributed -> file (1) -- |-> distributed_tf -> buffer -> file (1) diff --git a/tests/queries/0_stateless/01098_msgpack_format.sh b/tests/queries/0_stateless/01098_msgpack_format.sh index e2ae026eb27..30956ac6c7f 100755 --- a/tests/queries/0_stateless/01098_msgpack_format.sh +++ b/tests/queries/0_stateless/01098_msgpack_format.sh @@ -1,14 +1,11 @@ #!/usr/bin/env bash -# Tags: no-fasttest, no-parallel +# Tags: no-fasttest CURDIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd) # shellcheck source=../shell_config.sh . "$CURDIR"/../shell_config.sh -USER_FILES_PATH=$($CLICKHOUSE_CLIENT_BINARY --query "select _path,_file from file('nonexist.txt', 'CSV', 'val1 char')" 2>&1 | grep Exception | awk '{gsub("/nonexist.txt","",$9); print $9}') - - $CLICKHOUSE_CLIENT --query="DROP TABLE IF EXISTS msgpack"; $CLICKHOUSE_CLIENT --query="CREATE TABLE msgpack (uint8 UInt8, uint16 UInt16, uint32 UInt32, uint64 UInt64, int8 Int8, int16 Int16, int32 Int32, int64 Int64, float Float32, double Float64, string String, date Date, datetime DateTime('Asia/Istanbul'), datetime64 DateTime64(3, 'Asia/Istanbul'), array Array(UInt32)) ENGINE = Memory"; diff --git a/tests/queries/0_stateless/01111_create_drop_replicated_db_stress.sh b/tests/queries/0_stateless/01111_create_drop_replicated_db_stress.sh index 91f7a276ea3..90128d7a8ad 100755 --- a/tests/queries/0_stateless/01111_create_drop_replicated_db_stress.sh +++ b/tests/queries/0_stateless/01111_create_drop_replicated_db_stress.sh @@ -87,7 +87,7 @@ function insert() -TIMEOUT=30 +TIMEOUT=20 create_db $TIMEOUT & sync_db $TIMEOUT & diff --git a/tests/queries/0_stateless/01113_local_dictionary_type_conversion.sql b/tests/queries/0_stateless/01113_local_dictionary_type_conversion.sql index 65a03993295..1dc727930ab 100644 --- a/tests/queries/0_stateless/01113_local_dictionary_type_conversion.sql +++ b/tests/queries/0_stateless/01113_local_dictionary_type_conversion.sql @@ -1,29 +1,21 @@ --- Tags: no-parallel - -DROP DATABASE IF EXISTS database_for_dict; - -CREATE DATABASE database_for_dict; - -CREATE TABLE database_for_dict.table_for_dict ( +CREATE TABLE table_for_dict ( CompanyID String, OSType Enum('UNKNOWN' = 0, 'WINDOWS' = 1, 'LINUX' = 2, 'ANDROID' = 3, 'MAC' = 4), SomeID Int32 ) ENGINE = Memory(); -INSERT INTO database_for_dict.table_for_dict VALUES ('First', 'WINDOWS', 1), ('Second', 'LINUX', 2); +INSERT INTO table_for_dict VALUES ('First', 'WINDOWS', 1), ('Second', 'LINUX', 2); -CREATE DICTIONARY database_for_dict.dict_with_conversion +CREATE DICTIONARY dict_with_conversion ( CompanyID String DEFAULT '', OSType String DEFAULT '', SomeID Int32 DEFAULT 0 ) PRIMARY KEY CompanyID -SOURCE(CLICKHOUSE(HOST 'localhost' PORT tcpPort() USER 'default' TABLE 'table_for_dict' DB 'database_for_dict')) +SOURCE(CLICKHOUSE(HOST 'localhost' PORT tcpPort() USER 'default' TABLE 'table_for_dict' DB currentDatabase())) LIFETIME(MIN 1 MAX 20) LAYOUT(COMPLEX_KEY_HASHED()); -SELECT * FROM database_for_dict.dict_with_conversion ORDER BY CompanyID; - -DROP DATABASE IF EXISTS database_for_dict; +SELECT * FROM dict_with_conversion ORDER BY CompanyID; diff --git a/tests/queries/0_stateless/01114_database_atomic.sh b/tests/queries/0_stateless/01114_database_atomic.sh index 1b1f064ae0b..fed76727a27 100755 --- a/tests/queries/0_stateless/01114_database_atomic.sh +++ b/tests/queries/0_stateless/01114_database_atomic.sh @@ -1,5 +1,5 @@ #!/usr/bin/env bash -# Tags: no-parallel, no-fasttest +# Tags: no-fasttest # Tag no-fasttest: 45 seconds running # Creation of a database with Ordinary engine emits a warning. diff --git a/tests/queries/0_stateless/01114_mysql_database_engine_segfault.sql b/tests/queries/0_stateless/01114_mysql_database_engine_segfault.sql index 3379acf4d7b..783a728e336 100644 --- a/tests/queries/0_stateless/01114_mysql_database_engine_segfault.sql +++ b/tests/queries/0_stateless/01114_mysql_database_engine_segfault.sql @@ -1,4 +1,4 @@ --- Tags: no-parallel, no-fasttest +-- Tags: no-fasttest DROP DATABASE IF EXISTS conv_main; CREATE DATABASE conv_main ENGINE = MySQL('127.0.0.1:3456', conv_main, 'metrika', 'password'); -- { serverError CANNOT_CREATE_DATABASE } diff --git a/tests/queries/0_stateless/01125_dict_ddl_cannot_add_column.sql b/tests/queries/0_stateless/01125_dict_ddl_cannot_add_column.sql index a324d278c12..6a818d94a58 100644 --- a/tests/queries/0_stateless/01125_dict_ddl_cannot_add_column.sql +++ b/tests/queries/0_stateless/01125_dict_ddl_cannot_add_column.sql @@ -1,11 +1,3 @@ --- Tags: no-parallel - -DROP DATABASE IF EXISTS database_for_dict; - -CREATE DATABASE database_for_dict; - -use database_for_dict; - CREATE TABLE date_table ( id UInt32, @@ -24,7 +16,7 @@ CREATE DICTIONARY somedict end Date ) PRIMARY KEY id -SOURCE(CLICKHOUSE(HOST 'localhost' PORT tcpPort() USER 'default' TABLE 'date_table' DB 'database_for_dict')) +SOURCE(CLICKHOUSE(HOST 'localhost' PORT tcpPort() USER 'default' TABLE 'date_table' DB currentDatabase())) LAYOUT(RANGE_HASHED()) RANGE (MIN start MAX end) LIFETIME(MIN 300 MAX 360); @@ -35,5 +27,3 @@ SELECT * from somedict; SELECT 1 FROM somedict; SHOW TABLES; - -DROP DATABASE database_for_dict; diff --git a/tests/queries/0_stateless/01127_month_partitioning_consistency_select.sql b/tests/queries/0_stateless/01127_month_partitioning_consistency_select.sql index 2a1d04e6074..78632ab2463 100644 --- a/tests/queries/0_stateless/01127_month_partitioning_consistency_select.sql +++ b/tests/queries/0_stateless/01127_month_partitioning_consistency_select.sql @@ -1,6 +1,3 @@ --- Tags: no-parallel - -DROP TABLE IF EXISTS mt; set allow_deprecated_syntax_for_merge_tree=1; CREATE TABLE mt (d Date, x String) ENGINE = MergeTree(d, x, 8192); INSERT INTO mt VALUES ('2106-02-07', 'Hello'), ('1970-01-01', 'World'); diff --git a/tests/queries/0_stateless/01154_move_partition_long.sh b/tests/queries/0_stateless/01154_move_partition_long.sh index 24ec58c9c17..bdffa028846 100755 --- a/tests/queries/0_stateless/01154_move_partition_long.sh +++ b/tests/queries/0_stateless/01154_move_partition_long.sh @@ -107,7 +107,7 @@ export -f drop_partition_thread; export -f optimize_thread; export -f drop_part_thread; -TIMEOUT=60 +TIMEOUT=40 #timeout $TIMEOUT bash -c "create_drop_thread ${engines[@]}" & timeout $TIMEOUT bash -c 'insert_thread src' & diff --git a/tests/queries/0_stateless/01171_mv_select_insert_isolation_long.sh b/tests/queries/0_stateless/01171_mv_select_insert_isolation_long.sh index 8344bb6f426..2ab7f883367 100755 --- a/tests/queries/0_stateless/01171_mv_select_insert_isolation_long.sh +++ b/tests/queries/0_stateless/01171_mv_select_insert_isolation_long.sh @@ -1,5 +1,5 @@ #!/usr/bin/env bash -# Tags: long, no-parallel, no-ordinary-database, no-debug +# Tags: long, no-ordinary-database, no-debug # Test is too heavy, avoid parallel run in Flaky Check # shellcheck disable=SC2119 diff --git a/tests/queries/0_stateless/01185_create_or_replace_table.sql b/tests/queries/0_stateless/01185_create_or_replace_table.sql index 11759d0bb0c..801a775e024 100644 --- a/tests/queries/0_stateless/01185_create_or_replace_table.sql +++ b/tests/queries/0_stateless/01185_create_or_replace_table.sql @@ -1,4 +1,4 @@ --- Tags: no-ordinary-database, no-parallel +-- Tags: no-ordinary-database drop table if exists t1; diff --git a/tests/queries/0_stateless/01188_attach_table_from_path.sql b/tests/queries/0_stateless/01188_attach_table_from_path.sql index d1b9493b6c2..026979a0132 100644 --- a/tests/queries/0_stateless/01188_attach_table_from_path.sql +++ b/tests/queries/0_stateless/01188_attach_table_from_path.sql @@ -1,4 +1,4 @@ --- Tags: no-replicated-database, no-parallel +-- Tags: no-replicated-database drop table if exists test; drop table if exists file; diff --git a/tests/queries/0_stateless/01225_drop_dictionary_as_table.sql b/tests/queries/0_stateless/01225_drop_dictionary_as_table.sql index 3e497f9e3a4..a0cacd8bc7a 100644 --- a/tests/queries/0_stateless/01225_drop_dictionary_as_table.sql +++ b/tests/queries/0_stateless/01225_drop_dictionary_as_table.sql @@ -1,22 +1,15 @@ --- Tags: no-parallel - -DROP DATABASE IF EXISTS dict_db_01225; -CREATE DATABASE dict_db_01225; - -CREATE TABLE dict_db_01225.dict_data (key UInt64, val UInt64) Engine=Memory(); -CREATE DICTIONARY dict_db_01225.dict +CREATE TABLE dict_data (key UInt64, val UInt64) Engine=Memory(); +CREATE DICTIONARY dict ( key UInt64 DEFAULT 0, val UInt64 DEFAULT 10 ) PRIMARY KEY key -SOURCE(CLICKHOUSE(HOST 'localhost' PORT tcpPort() USER 'default' TABLE 'dict_data' PASSWORD '' DB 'dict_db_01225')) +SOURCE(CLICKHOUSE(HOST 'localhost' PORT tcpPort() USER 'default' TABLE 'dict_data' PASSWORD '' DB currentDatabase())) LIFETIME(MIN 0 MAX 0) LAYOUT(FLAT()); -SYSTEM RELOAD DICTIONARY dict_db_01225.dict; +SYSTEM RELOAD DICTIONARY dict; -DROP TABLE dict_db_01225.dict; -- { serverError CANNOT_DETACH_DICTIONARY_AS_TABLE } -DROP DICTIONARY dict_db_01225.dict; - -DROP DATABASE dict_db_01225; +DROP TABLE dict; -- { serverError CANNOT_DETACH_DICTIONARY_AS_TABLE } +DROP DICTIONARY dict; diff --git a/tests/queries/0_stateless/01254_dict_create_without_db.sql b/tests/queries/0_stateless/01254_dict_create_without_db.sql index 65a2ab52d23..2d4da5af9a9 100644 --- a/tests/queries/0_stateless/01254_dict_create_without_db.sql +++ b/tests/queries/0_stateless/01254_dict_create_without_db.sql @@ -1,9 +1,3 @@ --- Tags: no-parallel - -DROP DATABASE IF EXISTS dict_db_01254; -CREATE DATABASE dict_db_01254; -USE dict_db_01254; - CREATE TABLE dict_data (key UInt64, val UInt64) Engine=Memory(); CREATE DICTIONARY dict ( @@ -11,15 +5,12 @@ CREATE DICTIONARY dict val UInt64 DEFAULT 10 ) PRIMARY KEY key -SOURCE(CLICKHOUSE(HOST 'localhost' PORT tcpPort() USER 'default' TABLE 'dict_data' PASSWORD '' DB 'dict_db_01254')) +SOURCE(CLICKHOUSE(HOST 'localhost' PORT tcpPort() USER 'default' TABLE 'dict_data' PASSWORD '' DB currentDatabase())) LIFETIME(MIN 0 MAX 0) LAYOUT(FLAT()); -SELECT query_count, status FROM system.dictionaries WHERE database = 'dict_db_01254' AND name = 'dict'; -SYSTEM RELOAD DICTIONARY dict_db_01254.dict; -SELECT query_count, status FROM system.dictionaries WHERE database = 'dict_db_01254' AND name = 'dict'; -SELECT dictGetUInt64('dict_db_01254.dict', 'val', toUInt64(0)); -SELECT query_count, status FROM system.dictionaries WHERE database = 'dict_db_01254' AND name = 'dict'; - -USE system; -DROP DATABASE dict_db_01254; +SELECT query_count, status FROM system.dictionaries WHERE database = currentDatabase() AND name = 'dict'; +SYSTEM RELOAD DICTIONARY dict; +SELECT query_count, status FROM system.dictionaries WHERE database = currentDatabase() AND name = 'dict'; +SELECT dictGetUInt64('dict', 'val', toUInt64(0)); +SELECT query_count, status FROM system.dictionaries WHERE database = currentDatabase() AND name = 'dict'; diff --git a/tests/queries/0_stateless/01254_dict_load_after_detach_attach.sql b/tests/queries/0_stateless/01254_dict_load_after_detach_attach.sql index 206ddeac612..11473c6ce32 100644 --- a/tests/queries/0_stateless/01254_dict_load_after_detach_attach.sql +++ b/tests/queries/0_stateless/01254_dict_load_after_detach_attach.sql @@ -1,26 +1,19 @@ --- Tags: no-parallel - -DROP DATABASE IF EXISTS dict_db_01254; -CREATE DATABASE dict_db_01254; - -CREATE TABLE dict_db_01254.dict_data (key UInt64, val UInt64) Engine=Memory(); -CREATE DICTIONARY dict_db_01254.dict +CREATE TABLE dict_data (key UInt64, val UInt64) Engine=Memory(); +CREATE DICTIONARY dict ( key UInt64 DEFAULT 0, val UInt64 DEFAULT 10 ) PRIMARY KEY key -SOURCE(CLICKHOUSE(HOST 'localhost' PORT tcpPort() USER 'default' TABLE 'dict_data' PASSWORD '' DB 'dict_db_01254')) +SOURCE(CLICKHOUSE(HOST 'localhost' PORT tcpPort() USER 'default' TABLE 'dict_data' PASSWORD '' DB currentDatabase())) LIFETIME(MIN 0 MAX 0) LAYOUT(FLAT()); -DETACH DATABASE dict_db_01254; -ATTACH DATABASE dict_db_01254; +DETACH DATABASE {CLICKHOUSE_DATABASE:Identifier}; +ATTACH DATABASE {CLICKHOUSE_DATABASE:Identifier}; -SELECT COALESCE((SELECT status FROM system.dictionaries WHERE database = 'dict_db_01254' AND name = 'dict')::Nullable(String), 'NOT_LOADED'); -SYSTEM RELOAD DICTIONARY dict_db_01254.dict; -SELECT query_count, status FROM system.dictionaries WHERE database = 'dict_db_01254' AND name = 'dict'; -SELECT dictGetUInt64('dict_db_01254.dict', 'val', toUInt64(0)); -SELECT query_count, status FROM system.dictionaries WHERE database = 'dict_db_01254' AND name = 'dict'; - -DROP DATABASE dict_db_01254; +SELECT COALESCE((SELECT status FROM system.dictionaries WHERE database = currentDatabase() AND name = 'dict')::Nullable(String), 'NOT_LOADED'); +SYSTEM RELOAD DICTIONARY dict; +SELECT query_count, status FROM system.dictionaries WHERE database = currentDatabase() AND name = 'dict'; +SELECT dictGetUInt64('dict', 'val', toUInt64(0)); +SELECT query_count, status FROM system.dictionaries WHERE database = currentDatabase() AND name = 'dict'; diff --git a/tests/queries/0_stateless/01259_dictionary_custom_settings_ddl.sql b/tests/queries/0_stateless/01259_dictionary_custom_settings_ddl.sql index 432256d33c2..be56806f8d6 100644 --- a/tests/queries/0_stateless/01259_dictionary_custom_settings_ddl.sql +++ b/tests/queries/0_stateless/01259_dictionary_custom_settings_ddl.sql @@ -1,12 +1,6 @@ --- Tags: no-parallel, no-fasttest +-- Tags: no-fasttest -DROP DATABASE IF EXISTS database_for_dict; - -CREATE DATABASE database_for_dict; - -DROP TABLE IF EXISTS database_for_dict.table_for_dict; - -CREATE TABLE database_for_dict.table_for_dict +CREATE TABLE table_for_dict ( key_column UInt64, second_column UInt64, @@ -15,7 +9,7 @@ CREATE TABLE database_for_dict.table_for_dict ENGINE = MergeTree() ORDER BY key_column; -INSERT INTO database_for_dict.table_for_dict VALUES (100500, 10000000, 'Hello world'); +INSERT INTO table_for_dict VALUES (100500, 10000000, 'Hello world'); DROP DATABASE IF EXISTS ordinary_db; @@ -30,7 +24,7 @@ CREATE DICTIONARY ordinary_db.dict1 third_column String DEFAULT 'qqq' ) PRIMARY KEY key_column -SOURCE(CLICKHOUSE(HOST 'localhost' PORT tcpPort() USER 'default' TABLE 'table_for_dict' PASSWORD '' DB 'database_for_dict')) +SOURCE(CLICKHOUSE(HOST 'localhost' PORT tcpPort() USER 'default' TABLE 'table_for_dict' PASSWORD '' DB currentDatabase())) LIFETIME(MIN 1 MAX 10) LAYOUT(FLAT()) SETTINGS(max_result_bytes=1); @@ -40,10 +34,6 @@ SELECT dictGetUInt64('ordinary_db.dict1', 'second_column', toUInt64(100500)); -- SELECT 'END'; -DROP DICTIONARY IF EXISTS ordinary_db.dict1; - DROP DATABASE IF EXISTS ordinary_db; -DROP TABLE IF EXISTS database_for_dict.table_for_dict; - -DROP DATABASE IF EXISTS database_for_dict; +DROP TABLE IF EXISTS table_for_dict; diff --git a/tests/queries/0_stateless/01269_alias_type_differs.sql b/tests/queries/0_stateless/01269_alias_type_differs.sql index 64abcf9e367..b78e46f62c8 100644 --- a/tests/queries/0_stateless/01269_alias_type_differs.sql +++ b/tests/queries/0_stateless/01269_alias_type_differs.sql @@ -1,5 +1,3 @@ --- Tags: no-parallel - DROP TABLE IF EXISTS data_01269; CREATE TABLE data_01269 ( diff --git a/tests/queries/0_stateless/01272_suspicious_codecs.sql b/tests/queries/0_stateless/01272_suspicious_codecs.sql index 082a8d08675..1c1d7b58dd0 100644 --- a/tests/queries/0_stateless/01272_suspicious_codecs.sql +++ b/tests/queries/0_stateless/01272_suspicious_codecs.sql @@ -1,9 +1,5 @@ --- Tags: no-parallel - DROP TABLE IF EXISTS codecs; --- test what should work - CREATE TABLE codecs ( a UInt8 CODEC(LZ4), diff --git a/tests/queries/0_stateless/01280_ssd_complex_key_dictionary.sh b/tests/queries/0_stateless/01280_ssd_complex_key_dictionary.sh index fb7bf5c6fc1..9a80820dd58 100755 --- a/tests/queries/0_stateless/01280_ssd_complex_key_dictionary.sh +++ b/tests/queries/0_stateless/01280_ssd_complex_key_dictionary.sh @@ -5,8 +5,6 @@ CURDIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd) # shellcheck source=../shell_config.sh . "$CURDIR"/../shell_config.sh -USER_FILES_PATH=$($CLICKHOUSE_CLIENT_BINARY --query "select _path,_file from file('nonexist.txt', 'CSV', 'val1 char')" 2>&1 | grep Exception | awk '{gsub("/nonexist.txt","",$9); print $9}') - $CLICKHOUSE_CLIENT -n --query=" set allow_deprecated_database_ordinary=1; DROP DATABASE IF EXISTS 01280_db; diff --git a/tests/queries/0_stateless/01294_lazy_database_concurrent_recreate_reattach_and_show_tables_long.sh b/tests/queries/0_stateless/01294_lazy_database_concurrent_recreate_reattach_and_show_tables_long.sh index 3c11dc5f772..21f46a34514 100755 --- a/tests/queries/0_stateless/01294_lazy_database_concurrent_recreate_reattach_and_show_tables_long.sh +++ b/tests/queries/0_stateless/01294_lazy_database_concurrent_recreate_reattach_and_show_tables_long.sh @@ -1,5 +1,5 @@ #!/usr/bin/env bash -# Tags: long, no-parallel, no-fasttest +# Tags: long, no-fasttest CURDIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd) # shellcheck source=../shell_config.sh @@ -91,7 +91,7 @@ ${CLICKHOUSE_CLIENT} -n -q " " -TIMEOUT=30 +TIMEOUT=20 timeout $TIMEOUT bash -c recreate_lazy_func1 2> /dev/null & timeout $TIMEOUT bash -c recreate_lazy_func2 2> /dev/null & diff --git a/tests/queries/0_stateless/01301_aggregate_state_exception_memory_leak.sh b/tests/queries/0_stateless/01301_aggregate_state_exception_memory_leak.sh index 2d18c45406c..47fe7a9c7d9 100755 --- a/tests/queries/0_stateless/01301_aggregate_state_exception_memory_leak.sh +++ b/tests/queries/0_stateless/01301_aggregate_state_exception_memory_leak.sh @@ -1,5 +1,5 @@ #!/usr/bin/env bash -# Tags: no-parallel, no-fasttest +# Tags: no-fasttest CURDIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd) # shellcheck source=../shell_config.sh diff --git a/tests/queries/0_stateless/01302_aggregate_state_exception_memory_leak.sh b/tests/queries/0_stateless/01302_aggregate_state_exception_memory_leak.sh index b9f1f81da1a..a521accb082 100755 --- a/tests/queries/0_stateless/01302_aggregate_state_exception_memory_leak.sh +++ b/tests/queries/0_stateless/01302_aggregate_state_exception_memory_leak.sh @@ -1,5 +1,5 @@ #!/usr/bin/env bash -# Tags: no-parallel, no-fasttest +# Tags: no-fasttest CURDIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd) # shellcheck source=../shell_config.sh diff --git a/tests/queries/0_stateless/01338_long_select_and_alter.sh b/tests/queries/0_stateless/01338_long_select_and_alter.sh index fcdfa2dec82..2b0709162a3 100755 --- a/tests/queries/0_stateless/01338_long_select_and_alter.sh +++ b/tests/queries/0_stateless/01338_long_select_and_alter.sh @@ -1,5 +1,5 @@ #!/usr/bin/env bash -# Tags: long, no-parallel +# Tags: long CURDIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd) # shellcheck source=../shell_config.sh diff --git a/tests/queries/0_stateless/01338_long_select_and_alter_zookeeper.sh b/tests/queries/0_stateless/01338_long_select_and_alter_zookeeper.sh index 50ade3fad45..41e0a12f369 100755 --- a/tests/queries/0_stateless/01338_long_select_and_alter_zookeeper.sh +++ b/tests/queries/0_stateless/01338_long_select_and_alter_zookeeper.sh @@ -1,5 +1,5 @@ #!/usr/bin/env bash -# Tags: long, zookeeper, no-parallel +# Tags: long, zookeeper CURDIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd) # shellcheck source=../shell_config.sh diff --git a/tests/queries/0_stateless/01355_alter_column_with_order.sql b/tests/queries/0_stateless/01355_alter_column_with_order.sql index 0b1b4c42cce..405157fd891 100644 --- a/tests/queries/0_stateless/01355_alter_column_with_order.sql +++ b/tests/queries/0_stateless/01355_alter_column_with_order.sql @@ -1,28 +1,26 @@ --- Tags: no-parallel - -DROP TABLE IF EXISTS alter_test; +DROP TABLE IF EXISTS alter_01355; set allow_deprecated_syntax_for_merge_tree=1; -CREATE TABLE alter_test (CounterID UInt32, StartDate Date, UserID UInt32, VisitID UInt32, NestedColumn Nested(A UInt8, S String), ToDrop UInt32) ENGINE = MergeTree(StartDate, intHash32(UserID), (CounterID, StartDate, intHash32(UserID), VisitID), 8192); +CREATE TABLE alter_01355 (CounterID UInt32, StartDate Date, UserID UInt32, VisitID UInt32, NestedColumn Nested(A UInt8, S String), ToDrop UInt32) ENGINE = MergeTree(StartDate, intHash32(UserID), (CounterID, StartDate, intHash32(UserID), VisitID), 8192); -ALTER TABLE alter_test ADD COLUMN Added1 UInt32 FIRST; +ALTER TABLE alter_01355 ADD COLUMN Added1 UInt32 FIRST; -ALTER TABLE alter_test ADD COLUMN Added2 UInt32 AFTER NestedColumn; +ALTER TABLE alter_01355 ADD COLUMN Added2 UInt32 AFTER NestedColumn; -ALTER TABLE alter_test ADD COLUMN Added3 UInt32 AFTER ToDrop; +ALTER TABLE alter_01355 ADD COLUMN Added3 UInt32 AFTER ToDrop; -DESC alter_test; -DETACH TABLE alter_test; -ATTACH TABLE alter_test; -DESC alter_test; +DESC alter_01355; +DETACH TABLE alter_01355; +ATTACH TABLE alter_01355; +DESC alter_01355; -ALTER TABLE alter_test MODIFY COLUMN Added2 UInt32 FIRST; +ALTER TABLE alter_01355 MODIFY COLUMN Added2 UInt32 FIRST; -ALTER TABLE alter_test MODIFY COLUMN Added3 UInt32 AFTER CounterID; +ALTER TABLE alter_01355 MODIFY COLUMN Added3 UInt32 AFTER CounterID; -DESC alter_test; -DETACH TABLE alter_test; -ATTACH TABLE alter_test; -DESC alter_test; +DESC alter_01355; +DETACH TABLE alter_01355; +ATTACH TABLE alter_01355; +DESC alter_01355; -DROP TABLE IF EXISTS alter_test; +DROP TABLE IF EXISTS alter_01355; diff --git a/tests/queries/0_stateless/01355_ilike.sql b/tests/queries/0_stateless/01355_ilike.sql index 6bde62bf47e..1ceb878a5ef 100644 --- a/tests/queries/0_stateless/01355_ilike.sql +++ b/tests/queries/0_stateless/01355_ilike.sql @@ -1,4 +1,4 @@ --- Tags: no-parallel, no-fasttest +-- Tags: no-fasttest SELECT 'Hello' ILIKE ''; SELECT 'Hello' ILIKE '%'; @@ -53,11 +53,7 @@ SELECT 'ощщЁё' ILIKE '%щ%'; SELECT 'ощЩЁё' ILIKE '%ё%'; SHOW TABLES NOT ILIKE '%'; -DROP DATABASE IF EXISTS test_01355; -CREATE DATABASE test_01355; -USE test_01355; CREATE TABLE test1 (x UInt8) ENGINE = Memory; CREATE TABLE test2 (x UInt8) ENGINE = Memory; SHOW TABLES ILIKE 'tES%'; SHOW TABLES NOT ILIKE 'TeS%'; -DROP DATABASE test_01355; diff --git a/tests/queries/0_stateless/01388_clear_all_columns.sql b/tests/queries/0_stateless/01388_clear_all_columns.sql index cc395aa7fb4..07b4fb3de90 100644 --- a/tests/queries/0_stateless/01388_clear_all_columns.sql +++ b/tests/queries/0_stateless/01388_clear_all_columns.sql @@ -1,5 +1,3 @@ --- Tags: no-parallel - DROP TABLE IF EXISTS test; CREATE TABLE test (x UInt8) ENGINE = MergeTree ORDER BY tuple(); INSERT INTO test (x) VALUES (1), (2), (3); diff --git a/tests/queries/0_stateless/01391_join_on_dict_crash.sql b/tests/queries/0_stateless/01391_join_on_dict_crash.sql index 854da04b334..e056e147501 100644 --- a/tests/queries/0_stateless/01391_join_on_dict_crash.sql +++ b/tests/queries/0_stateless/01391_join_on_dict_crash.sql @@ -1,13 +1,3 @@ --- Tags: no-parallel - -DROP DATABASE IF EXISTS db_01391; -CREATE DATABASE db_01391; -USE db_01391; - -DROP TABLE IF EXISTS t; -DROP TABLE IF EXISTS d_src; -DROP DICTIONARY IF EXISTS d; - CREATE TABLE t (click_city_id UInt32, click_country_id UInt32) Engine = Memory; CREATE TABLE d_src (id UInt64, country_id UInt8, name String) Engine = Memory; @@ -16,14 +6,9 @@ INSERT INTO d_src VALUES (0, 0, 'n'); CREATE DICTIONARY d (id UInt32, country_id UInt8, name String) PRIMARY KEY id -SOURCE(CLICKHOUSE(HOST 'localhost' PORT tcpPort() USER 'default' DB 'db_01391' table 'd_src')) +SOURCE(CLICKHOUSE(HOST 'localhost' PORT tcpPort() USER 'default' DB currentDatabase() table 'd_src')) LIFETIME(MIN 1 MAX 1) LAYOUT(HASHED()); SELECT click_country_id FROM t AS cc LEFT JOIN d ON toUInt32(d.id) = cc.click_city_id; SELECT click_country_id FROM t AS cc LEFT JOIN d ON d.country_id < 99 AND d.id = cc.click_city_id; - -DROP DICTIONARY d; -DROP TABLE t; -DROP TABLE d_src; -DROP DATABASE IF EXISTS db_01391; diff --git a/tests/queries/0_stateless/01392_column_resolve.sql b/tests/queries/0_stateless/01392_column_resolve.sql index 72b6af4576a..90a7d9b169a 100644 --- a/tests/queries/0_stateless/01392_column_resolve.sql +++ b/tests/queries/0_stateless/01392_column_resolve.sql @@ -1,16 +1,11 @@ --- Tags: no-parallel +CREATE TABLE tableConversion (conversionId String, value Nullable(Double)) ENGINE = Log(); +CREATE TABLE tableClick (clickId String, conversionId String, value Nullable(Double)) ENGINE = Log(); +CREATE TABLE leftjoin (id String) ENGINE = Log(); -DROP DATABASE IF EXISTS test_01392; -CREATE DATABASE test_01392; - -CREATE TABLE test_01392.tableConversion (conversionId String, value Nullable(Double)) ENGINE = Log(); -CREATE TABLE test_01392.tableClick (clickId String, conversionId String, value Nullable(Double)) ENGINE = Log(); -CREATE TABLE test_01392.leftjoin (id String) ENGINE = Log(); - -INSERT INTO test_01392.tableConversion(conversionId, value) VALUES ('Conversion 1', 1); -INSERT INTO test_01392.tableClick(clickId, conversionId, value) VALUES ('Click 1', 'Conversion 1', 14); -INSERT INTO test_01392.tableClick(clickId, conversionId, value) VALUES ('Click 2', 'Conversion 1', 15); -INSERT INTO test_01392.tableClick(clickId, conversionId, value) VALUES ('Click 3', 'Conversion 1', 16); +INSERT INTO tableConversion(conversionId, value) VALUES ('Conversion 1', 1); +INSERT INTO tableClick(clickId, conversionId, value) VALUES ('Click 1', 'Conversion 1', 14); +INSERT INTO tableClick(clickId, conversionId, value) VALUES ('Click 2', 'Conversion 1', 15); +INSERT INTO tableClick(clickId, conversionId, value) VALUES ('Click 3', 'Conversion 1', 16); SELECT conversion.conversionId AS myConversionId, @@ -18,19 +13,13 @@ SELECT click.myValue AS myValue FROM ( SELECT conversionId, value as myValue - FROM test_01392.tableConversion + FROM tableConversion ) AS conversion INNER JOIN ( SELECT clickId, conversionId, value as myValue - FROM test_01392.tableClick + FROM tableClick ) AS click ON click.conversionId = conversion.conversionId LEFT JOIN ( - SELECT * FROM test_01392.leftjoin + SELECT * FROM leftjoin ) AS dummy ON (dummy.id = conversion.conversionId) ORDER BY myValue; - -DROP TABLE test_01392.tableConversion; -DROP TABLE test_01392.tableClick; -DROP TABLE test_01392.leftjoin; - -DROP DATABASE test_01392; diff --git a/tests/queries/0_stateless/01396_inactive_replica_cleanup_nodes_zookeeper.sh b/tests/queries/0_stateless/01396_inactive_replica_cleanup_nodes_zookeeper.sh index 1c1eb4489ee..b81bb75891d 100755 --- a/tests/queries/0_stateless/01396_inactive_replica_cleanup_nodes_zookeeper.sh +++ b/tests/queries/0_stateless/01396_inactive_replica_cleanup_nodes_zookeeper.sh @@ -1,5 +1,5 @@ #!/usr/bin/env bash -# Tags: replica, no-debug, no-parallel +# Tags: replica, no-debug CURDIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd) # shellcheck source=../shell_config.sh diff --git a/tests/queries/0_stateless/01412_cache_dictionary_race.sh b/tests/queries/0_stateless/01412_cache_dictionary_race.sh index 165a461193d..9aa39652021 100755 --- a/tests/queries/0_stateless/01412_cache_dictionary_race.sh +++ b/tests/queries/0_stateless/01412_cache_dictionary_race.sh @@ -1,5 +1,5 @@ #!/usr/bin/env bash -# Tags: race, no-parallel +# Tags: race CURDIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd) # shellcheck source=../shell_config.sh @@ -54,7 +54,7 @@ function drop_create_table_thread() export -f dict_get_thread; export -f drop_create_table_thread; -TIMEOUT=30 +TIMEOUT=20 timeout $TIMEOUT bash -c dict_get_thread 2> /dev/null & timeout $TIMEOUT bash -c dict_get_thread 2> /dev/null & diff --git a/tests/queries/0_stateless/01415_sticking_mutations.sh b/tests/queries/0_stateless/01415_sticking_mutations.sh index 821c83fe728..b7c8768a65d 100755 --- a/tests/queries/0_stateless/01415_sticking_mutations.sh +++ b/tests/queries/0_stateless/01415_sticking_mutations.sh @@ -1,5 +1,5 @@ #!/usr/bin/env bash -# Tags: no-replicated-database, no-parallel +# Tags: no-replicated-database set -e diff --git a/tests/queries/0_stateless/01442_merge_detach_attach_long.sh b/tests/queries/0_stateless/01442_merge_detach_attach_long.sh index 85fdf7ed764..9ba1fe93543 100755 --- a/tests/queries/0_stateless/01442_merge_detach_attach_long.sh +++ b/tests/queries/0_stateless/01442_merge_detach_attach_long.sh @@ -1,5 +1,5 @@ #!/usr/bin/env bash -# Tags: long, no-parallel, no-debug +# Tags: long, no-debug set -e @@ -27,7 +27,7 @@ function thread_ops() } export -f thread_ops -TIMEOUT=60 +TIMEOUT=30 thread_ops $TIMEOUT & wait diff --git a/tests/queries/0_stateless/01444_create_table_drop_database_race.sh b/tests/queries/0_stateless/01444_create_table_drop_database_race.sh index eb231e71525..ae74efa4e20 100755 --- a/tests/queries/0_stateless/01444_create_table_drop_database_race.sh +++ b/tests/queries/0_stateless/01444_create_table_drop_database_race.sh @@ -1,5 +1,5 @@ #!/usr/bin/env bash -# Tags: race, no-parallel +# Tags: race set -e @@ -8,20 +8,20 @@ CUR_DIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd) . "$CUR_DIR"/../shell_config.sh # This test reproduces "Directory not empty" error in DROP DATABASE query. +export DB=test_$RANDOM function thread1() { while true; do -# ${CLICKHOUSE_CLIENT} --query="SHOW TABLES FROM test_01444" - ${CLICKHOUSE_CLIENT} --query="DROP DATABASE IF EXISTS test_01444" 2>&1| grep -F "Code: " | grep -Fv "Code: 219" - ${CLICKHOUSE_CLIENT} --query="CREATE DATABASE IF NOT EXISTS test_01444" + ${CLICKHOUSE_CLIENT} --query="DROP DATABASE IF EXISTS $DB" 2>&1| grep -F "Code: " | grep -Fv "Code: 219" + ${CLICKHOUSE_CLIENT} --query="CREATE DATABASE IF NOT EXISTS $DB" done } function thread2() { while true; do - ${CLICKHOUSE_CLIENT} --query="CREATE TABLE IF NOT EXISTS test_01444.t$RANDOM (x UInt8) ENGINE = MergeTree ORDER BY tuple()" 2>/dev/null + ${CLICKHOUSE_CLIENT} --query="CREATE TABLE IF NOT EXISTS $DB.t$RANDOM (x UInt8) ENGINE = MergeTree ORDER BY tuple()" 2>/dev/null done } @@ -36,4 +36,4 @@ timeout $TIMEOUT bash -c thread2 & wait -${CLICKHOUSE_CLIENT} --query="DROP DATABASE IF EXISTS test_01444" 2>&1| grep -F "Code: " | grep -Fv "Code: 219" || exit 0 +${CLICKHOUSE_CLIENT} --query="DROP DATABASE IF EXISTS ${DB}" 2>&1| grep -F "Code: " | grep -Fv "Code: 219" || exit 0 diff --git a/tests/queries/0_stateless/01454_storagememory_data_race_challenge.sh b/tests/queries/0_stateless/01454_storagememory_data_race_challenge.sh index d83343b3cb3..2bfd350ec51 100755 --- a/tests/queries/0_stateless/01454_storagememory_data_race_challenge.sh +++ b/tests/queries/0_stateless/01454_storagememory_data_race_challenge.sh @@ -1,5 +1,5 @@ #!/usr/bin/env bash -# Tags: race, no-parallel +# Tags: race set -e @@ -36,8 +36,8 @@ function g { export -f f; export -f g; -timeout 30 bash -c f > /dev/null & -timeout 30 bash -c g > /dev/null & +timeout 20 bash -c f > /dev/null & +timeout 20 bash -c g > /dev/null & wait $CLICKHOUSE_CLIENT -q "DROP TABLE mem" diff --git a/tests/queries/0_stateless/01543_avro_deserialization_with_lc.sh b/tests/queries/0_stateless/01543_avro_deserialization_with_lc.sh index 697b32a77ae..a5697a62dc2 100755 --- a/tests/queries/0_stateless/01543_avro_deserialization_with_lc.sh +++ b/tests/queries/0_stateless/01543_avro_deserialization_with_lc.sh @@ -5,8 +5,6 @@ CURDIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd) # shellcheck source=../shell_config.sh . "$CURDIR"/../shell_config.sh -USER_FILES_PATH=$($CLICKHOUSE_CLIENT_BINARY --query "select _path,_file from file('nonexist.txt', 'CSV', 'val1 char')" 2>&1 | grep Exception | awk '{gsub("/nonexist.txt","",$9); print $9}') - $CLICKHOUSE_CLIENT --multiquery --query " SET allow_suspicious_low_cardinality_types=1; CREATE TABLE IF NOT EXISTS test_01543 (value LowCardinality(String), value2 LowCardinality(UInt64)) ENGINE=Memory(); diff --git a/tests/queries/0_stateless/01593_concurrent_alter_mutations_kill.sh b/tests/queries/0_stateless/01593_concurrent_alter_mutations_kill.sh index d68f9bc1837..68c09932f2c 100755 --- a/tests/queries/0_stateless/01593_concurrent_alter_mutations_kill.sh +++ b/tests/queries/0_stateless/01593_concurrent_alter_mutations_kill.sh @@ -1,5 +1,5 @@ #!/usr/bin/env bash -# Tags: no-parallel, no-fasttest +# Tags: no-fasttest, no-parallel CURDIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd) # shellcheck source=../shell_config.sh @@ -39,7 +39,7 @@ function kill_mutation_thread export -f alter_thread; export -f kill_mutation_thread; -TIMEOUT=30 +TIMEOUT=20 timeout $TIMEOUT bash -c alter_thread 2> /dev/null & timeout $TIMEOUT bash -c kill_mutation_thread 2> /dev/null & diff --git a/tests/queries/0_stateless/01658_read_file_to_stringcolumn.sh b/tests/queries/0_stateless/01658_read_file_to_stringcolumn.sh index 96327536f89..685fe69642a 100755 --- a/tests/queries/0_stateless/01658_read_file_to_stringcolumn.sh +++ b/tests/queries/0_stateless/01658_read_file_to_stringcolumn.sh @@ -7,18 +7,11 @@ CURDIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd) # shellcheck source=../shell_config.sh . "$CURDIR"/../shell_config.sh -# Data preparation. - -# Now we can get the user_files_path by use the table file function for trick. also we can get it by query as: -# "insert into function file('exist.txt', 'CSV', 'val1 char') values ('aaaa'); select _path from file('exist.txt', 'CSV', 'val1 char')" -CLICKHOUSE_USER_FILES_PATH=$(clickhouse-client --query "select _path, _file from file('nonexist.txt', 'CSV', 'val1 char')" 2>&1 | grep Exception | awk '{gsub("/nonexist.txt","",$9); print $9}') - -mkdir -p ${CLICKHOUSE_USER_FILES_PATH}/ -echo -n aaaaaaaaa > ${CLICKHOUSE_USER_FILES_PATH}/a.txt -echo -n bbbbbbbbb > ${CLICKHOUSE_USER_FILES_PATH}/b.txt -echo -n ccccccccc > ${CLICKHOUSE_USER_FILES_PATH}/c.txt +echo -n aaaaaaaaa > ${USER_FILES_PATH}/a.txt +echo -n bbbbbbbbb > ${USER_FILES_PATH}/b.txt +echo -n ccccccccc > ${USER_FILES_PATH}/c.txt echo -n ccccccccc > /tmp/c.txt -mkdir -p ${CLICKHOUSE_USER_FILES_PATH}/dir +mkdir -p ${USER_FILES_PATH}/dir ### 1st TEST in CLIENT mode. @@ -85,15 +78,15 @@ echo "${CLICKHOUSE_LOCAL} --query "'"select file('"'dir'), file('b.txt')"'";echo # Test that the function is not injective -echo -n Hello > ${CLICKHOUSE_USER_FILES_PATH}/a -echo -n Hello > ${CLICKHOUSE_USER_FILES_PATH}/b -echo -n World > ${CLICKHOUSE_USER_FILES_PATH}/c +echo -n Hello > ${USER_FILES_PATH}/a +echo -n Hello > ${USER_FILES_PATH}/b +echo -n World > ${USER_FILES_PATH}/c ${CLICKHOUSE_CLIENT} --query "SELECT file(arrayJoin(['a', 'b', 'c'])) AS s, count() GROUP BY s ORDER BY s" ${CLICKHOUSE_CLIENT} --query "SELECT s, count() FROM file('?', TSV, 's String') GROUP BY s ORDER BY s" # Restore -rm ${CLICKHOUSE_USER_FILES_PATH}/{a,b,c}.txt -rm ${CLICKHOUSE_USER_FILES_PATH}/{a,b,c} +rm ${USER_FILES_PATH}/{a,b,c}.txt +rm ${USER_FILES_PATH}/{a,b,c} rm /tmp/c.txt -rm -rf ${CLICKHOUSE_USER_FILES_PATH}/dir +rm -rf ${USER_FILES_PATH}/dir diff --git a/tests/queries/0_stateless/01684_ssd_cache_dictionary_simple_key.sh b/tests/queries/0_stateless/01684_ssd_cache_dictionary_simple_key.sh index 9167a2d306f..0e5c2862066 100755 --- a/tests/queries/0_stateless/01684_ssd_cache_dictionary_simple_key.sh +++ b/tests/queries/0_stateless/01684_ssd_cache_dictionary_simple_key.sh @@ -5,8 +5,6 @@ CURDIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd) # shellcheck source=../shell_config.sh . "$CURDIR"/../shell_config.sh -USER_FILES_PATH=$($CLICKHOUSE_CLIENT_BINARY --query "select _path,_file from file('nonexist.txt', 'CSV', 'val1 char')" 2>&1 | grep Exception | awk '{gsub("/nonexist.txt","",$9); print $9}') - $CLICKHOUSE_CLIENT -n --query=" DROP DATABASE IF EXISTS 01684_database_for_cache_dictionary; CREATE DATABASE 01684_database_for_cache_dictionary; diff --git a/tests/queries/0_stateless/01685_ssd_cache_dictionary_complex_key.sh b/tests/queries/0_stateless/01685_ssd_cache_dictionary_complex_key.sh index 9dd8a41ce5a..5583a9dd5e7 100755 --- a/tests/queries/0_stateless/01685_ssd_cache_dictionary_complex_key.sh +++ b/tests/queries/0_stateless/01685_ssd_cache_dictionary_complex_key.sh @@ -5,8 +5,6 @@ CURDIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd) # shellcheck source=../shell_config.sh . "$CURDIR"/../shell_config.sh -USER_FILES_PATH=$($CLICKHOUSE_CLIENT_BINARY --query "select _path,_file from file('nonexist.txt', 'CSV', 'val1 char')" 2>&1 | grep Exception | awk '{gsub("/nonexist.txt","",$9); print $9}') - $CLICKHOUSE_CLIENT -n --query=" DROP DATABASE IF EXISTS 01685_database_for_cache_dictionary; CREATE DATABASE 01685_database_for_cache_dictionary; diff --git a/tests/queries/0_stateless/01710_projection_vertical_merges.sql b/tests/queries/0_stateless/01710_projection_vertical_merges.sql index 2c4378bb7a4..0f80d659e92 100644 --- a/tests/queries/0_stateless/01710_projection_vertical_merges.sql +++ b/tests/queries/0_stateless/01710_projection_vertical_merges.sql @@ -1,4 +1,4 @@ --- Tags: long, no-parallel +-- Tags: long drop table if exists t; diff --git a/tests/queries/0_stateless/01747_join_view_filter_dictionary.sql b/tests/queries/0_stateless/01747_join_view_filter_dictionary.sql index 050aa33464e..c7c525ba20e 100644 --- a/tests/queries/0_stateless/01747_join_view_filter_dictionary.sql +++ b/tests/queries/0_stateless/01747_join_view_filter_dictionary.sql @@ -25,7 +25,7 @@ create table dictst01747(some_name String, field1 String, field2 UInt8) Engine = as select 'name', 'test', 33; CREATE DICTIONARY default.dict01747 (some_name String, field1 String, field2 UInt8) -PRIMARY KEY some_name SOURCE(CLICKHOUSE(HOST 'localhost' PORT 9000 +PRIMARY KEY some_name SOURCE(CLICKHOUSE(HOST 'localhost' PORT tcpPort() TABLE dictst01747 DB currentDatabase() USER 'default')) LIFETIME(MIN 0 MAX 0) LAYOUT(COMPLEX_KEY_HASHED()); diff --git a/tests/queries/0_stateless/01747_system_session_log_long.sh b/tests/queries/0_stateless/01747_system_session_log_long.sh index ecddcb627b8..022bf488886 100755 --- a/tests/queries/0_stateless/01747_system_session_log_long.sh +++ b/tests/queries/0_stateless/01747_system_session_log_long.sh @@ -228,13 +228,13 @@ function testMySQL() echo "MySQL 'successful login' case is skipped for ${auth_type}." else executeQuery \ - <<< "SELECT 1 FROM mysql('127.0.0.1:9004', 'system', 'one', '${username}', '${password}') LIMIT 1 \ + <<< "SELECT 1 FROM mysql('127.0.0.1:${CLICKHOUSE_PORT_MYSQL}', 'system', 'one', '${username}', '${password}') LIMIT 1 \ FORMAT Null" fi echo 'Wrong username' executeQueryExpectError \ - <<< "SELECT 1 FROM mysql('127.0.0.1:9004', 'system', 'one', 'invalid_${username}', '${password}') LIMIT 1 \ + <<< "SELECT 1 FROM mysql('127.0.0.1:${CLICKHOUSE_PORT_MYSQL}', 'system', 'one', 'invalid_${username}', '${password}') LIMIT 1 \ FORMAT Null" \ | grep -Eq "Code: 279\. DB::Exception: .* invalid_${username}" @@ -246,7 +246,7 @@ function testMySQL() echo "MySQL 'wrong password' case is skipped for ${auth_type}." else executeQueryExpectError \ - <<< "SELECT 1 FROM mysql('127.0.0.1:9004', 'system', 'one', '${username}', 'invalid_${password}') LIMIT 1 \ + <<< "SELECT 1 FROM mysql('127.0.0.1:${CLICKHOUSE_PORT_MYSQL}', 'system', 'one', '${username}', 'invalid_${password}') LIMIT 1 \ FORMAT Null" | grep -Eq "Code: 279\. DB::Exception: .* ${username}" fi } @@ -267,11 +267,11 @@ function testMySQL() ## Loging\Logout ## CH is being able to log into itself via PostgreSQL protocol but query fails. #executeQueryExpectError \ - # <<< "SELECT 1 FROM postgresql('localhost:9005', 'system', 'one', '${username}', '${password}') LIMIT 1 FORMAT Null" \ + # <<< "SELECT 1 FROM postgresql('localhost:${CLICKHOUSE_PORT_POSTGRESQL', 'system', 'one', '${username}', '${password}') LIMIT 1 FORMAT Null" \ # Wrong username executeQueryExpectError \ - <<< "SELECT 1 FROM postgresql('localhost:9005', 'system', 'one', 'invalid_${username}', '${password}') LIMIT 1 FORMAT Null" \ + <<< "SELECT 1 FROM postgresql('localhost:${CLICKHOUSE_PORT_POSTGRESQL}', 'system', 'one', 'invalid_${username}', '${password}') LIMIT 1 FORMAT Null" \ | grep -Eq "Invalid user or password" if [[ "${auth_type}" == "no_password" ]] @@ -281,7 +281,7 @@ function testMySQL() else # Wrong password executeQueryExpectError \ - <<< "SELECT 1 FROM postgresql('localhost:9005', 'system', 'one', '${username}', 'invalid_${password}') LIMIT 1 FORMAT Null" \ + <<< "SELECT 1 FROM postgresql('localhost:${CLICKHOUSE_PORT_POSTGRESQL}', 'system', 'one', '${username}', 'invalid_${password}') LIMIT 1 FORMAT Null" \ | grep -Eq "Invalid user or password" fi } diff --git a/tests/queries/0_stateless/01748_dictionary_table_dot.sql b/tests/queries/0_stateless/01748_dictionary_table_dot.sql index a2364fdf823..993d2e1a635 100644 --- a/tests/queries/0_stateless/01748_dictionary_table_dot.sql +++ b/tests/queries/0_stateless/01748_dictionary_table_dot.sql @@ -22,7 +22,7 @@ CREATE DICTIONARY test_dict `value` String ) PRIMARY KEY key1, key2 -SOURCE(CLICKHOUSE(HOST 'localhost' PORT 9000 USER 'default' TABLE `test.txt` PASSWORD '' DB currentDatabase())) +SOURCE(CLICKHOUSE(HOST 'localhost' PORT tcpPort() USER 'default' TABLE `test.txt` PASSWORD '' DB currentDatabase())) LIFETIME(MIN 1 MAX 3600) LAYOUT(COMPLEX_KEY_HASHED()); diff --git a/tests/queries/0_stateless/01780_clickhouse_dictionary_source_loop.sql b/tests/queries/0_stateless/01780_clickhouse_dictionary_source_loop.sql index 1eee4090112..3ebc85c47f7 100644 --- a/tests/queries/0_stateless/01780_clickhouse_dictionary_source_loop.sql +++ b/tests/queries/0_stateless/01780_clickhouse_dictionary_source_loop.sql @@ -10,7 +10,7 @@ CREATE DICTIONARY dict1 value String ) PRIMARY KEY id -SOURCE(CLICKHOUSE(HOST 'localhost' PORT 9000 TABLE 'dict1')) +SOURCE(CLICKHOUSE(HOST 'localhost' PORT tcpPort() TABLE 'dict1')) LAYOUT(DIRECT()); SELECT * FROM dict1; --{serverError BAD_ARGUMENTS} @@ -24,7 +24,7 @@ CREATE DICTIONARY 01780_db.dict2 value String ) PRIMARY KEY id -SOURCE(CLICKHOUSE(HOST 'localhost' PORT 9000 DATABASE '01780_db' TABLE 'dict2')) +SOURCE(CLICKHOUSE(HOST 'localhost' PORT tcpPort() DATABASE '01780_db' TABLE 'dict2')) LAYOUT(DIRECT()); SELECT * FROM 01780_db.dict2; --{serverError BAD_ARGUMENTS} @@ -45,7 +45,7 @@ CREATE DICTIONARY 01780_db.dict3 value String ) PRIMARY KEY id -SOURCE(CLICKHOUSE(HOST 'localhost' PORT 9000 TABLE 'dict3_source' DATABASE '01780_db')) +SOURCE(CLICKHOUSE(HOST 'localhost' PORT tcpPort() TABLE 'dict3_source' DATABASE '01780_db')) LAYOUT(DIRECT()); SELECT * FROM 01780_db.dict3; diff --git a/tests/queries/0_stateless/01825_type_json_btc.sh b/tests/queries/0_stateless/01825_type_json_btc.sh index 1e74166e7a7..ebc5482de7a 100755 --- a/tests/queries/0_stateless/01825_type_json_btc.sh +++ b/tests/queries/0_stateless/01825_type_json_btc.sh @@ -5,10 +5,9 @@ CUR_DIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd) # shellcheck source=../shell_config.sh . "$CUR_DIR"/../shell_config.sh -user_files_path=$($CLICKHOUSE_CLIENT_BINARY --query "select _path,_file from file('nonexist.txt', 'CSV', 'val1 char')" 2>&1 | grep Exception | awk '{gsub("/nonexist.txt","",$9); print $9}') -mkdir -p ${user_files_path}/${CLICKHOUSE_TEST_UNIQUE_NAME}/ -rm -rf ${user_files_path}/${CLICKHOUSE_TEST_UNIQUE_NAME:?}/* -cp $CUR_DIR/data_json/btc_transactions.json ${user_files_path}/${CLICKHOUSE_TEST_UNIQUE_NAME}/ +mkdir -p ${CLICKHOUSE_USER_FILES_UNIQUE}/ +rm -rf "${CLICKHOUSE_USER_FILES_UNIQUE:?}"/* +cp $CUR_DIR/data_json/btc_transactions.json ${CLICKHOUSE_USER_FILES_UNIQUE}/ ${CLICKHOUSE_CLIENT} -q "DROP TABLE IF EXISTS btc" @@ -27,4 +26,4 @@ ${CLICKHOUSE_CLIENT} -q "SELECT data.out.spending_outpoints AS outpoints FROM bt ${CLICKHOUSE_CLIENT} -q "DROP TABLE IF EXISTS btc" -rm ${user_files_path}/${CLICKHOUSE_TEST_UNIQUE_NAME}/btc_transactions.json +rm ${CLICKHOUSE_USER_FILES_UNIQUE}/btc_transactions.json diff --git a/tests/queries/0_stateless/01825_type_json_multiple_files.sh b/tests/queries/0_stateless/01825_type_json_multiple_files.sh index 089b7991784..453e7a3c78e 100755 --- a/tests/queries/0_stateless/01825_type_json_multiple_files.sh +++ b/tests/queries/0_stateless/01825_type_json_multiple_files.sh @@ -1,23 +1,22 @@ #!/usr/bin/env bash -# Tags: no-fasttest, no-parallel +# Tags: no-fasttest CUR_DIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd) # shellcheck source=../shell_config.sh . "$CUR_DIR"/../shell_config.sh -user_files_path=$($CLICKHOUSE_CLIENT --query "select _path,_file from file('nonexist.txt', 'CSV', 'val1 char')" 2>&1 | grep -E '^Code: 107.*FILE_DOESNT_EXIST' | head -1 | awk '{gsub("/nonexist.txt","",$9); print $9}') -for f in "$user_files_path"/01825_file_*.json; do +for f in "${USER_FILES_PATH:?}/${CLICKHOUSE_DATABASE}"_*.json; do [ -e $f ] && rm $f done for i in {0..5}; do - echo "{\"k$i\": 100}" > "$user_files_path"/01825_file_$i.json + echo "{\"k$i\": 100}" > "$USER_FILES_PATH/${CLICKHOUSE_DATABASE}_$i.json" done ${CLICKHOUSE_CLIENT} -q "DROP TABLE IF EXISTS t_json_files" ${CLICKHOUSE_CLIENT} -q "CREATE TABLE t_json_files (file String, data JSON) ENGINE = MergeTree ORDER BY tuple()" --allow_experimental_object_type 1 -${CLICKHOUSE_CLIENT} -q "INSERT INTO t_json_files SELECT _file, data FROM file('01825_file_*.json', 'JSONAsObject', 'data JSON')" --allow_experimental_object_type 1 +${CLICKHOUSE_CLIENT} -q "INSERT INTO t_json_files SELECT _file, data FROM file('${CLICKHOUSE_DATABASE}_*.json', 'JSONAsObject', 'data JSON')" --allow_experimental_object_type 1 ${CLICKHOUSE_CLIENT} -q "SELECT data FROM t_json_files ORDER BY file FORMAT JSONEachRow" --output_format_json_named_tuples_as_objects 1 ${CLICKHOUSE_CLIENT} -q "SELECT toTypeName(data) FROM t_json_files LIMIT 1" @@ -25,7 +24,7 @@ ${CLICKHOUSE_CLIENT} -q "SELECT toTypeName(data) FROM t_json_files LIMIT 1" ${CLICKHOUSE_CLIENT} -q "TRUNCATE TABLE IF EXISTS t_json_files" ${CLICKHOUSE_CLIENT} -q "INSERT INTO t_json_files \ - SELECT _file, data FROM file('01825_file_*.json', 'JSONAsObject', 'data JSON') \ + SELECT _file, data FROM file('${CLICKHOUSE_DATABASE}_*.json', 'JSONAsObject', 'data JSON') \ ORDER BY _file LIMIT 3" --max_threads 1 --min_insert_block_size_rows 1 --max_insert_block_size 1 --max_block_size 1 --allow_experimental_object_type 1 ${CLICKHOUSE_CLIENT} -q "SELECT data FROM t_json_files ORDER BY file, data FORMAT JSONEachRow" --output_format_json_named_tuples_as_objects 1 @@ -34,11 +33,11 @@ ${CLICKHOUSE_CLIENT} -q "SELECT toTypeName(data) FROM t_json_files LIMIT 1" ${CLICKHOUSE_CLIENT} -q "TRUNCATE TABLE IF EXISTS t_json_files" ${CLICKHOUSE_CLIENT} -q "INSERT INTO t_json_files \ - SELECT _file, data FROM file('01825_file_*.json', 'JSONAsObject', 'data JSON') \ - WHERE _file IN ('01825_file_1.json', '01825_file_3.json')" --allow_experimental_object_type 1 + SELECT _file, data FROM file('${CLICKHOUSE_DATABASE}_*.json', 'JSONAsObject', 'data JSON') \ + WHERE _file IN ('${CLICKHOUSE_DATABASE}_1.json', '${CLICKHOUSE_DATABASE}_3.json')" --allow_experimental_object_type 1 ${CLICKHOUSE_CLIENT} -q "SELECT data FROM t_json_files ORDER BY file FORMAT JSONEachRow" --output_format_json_named_tuples_as_objects 1 ${CLICKHOUSE_CLIENT} -q "SELECT toTypeName(data) FROM t_json_files LIMIT 1" ${CLICKHOUSE_CLIENT} -q "DROP TABLE IF EXISTS t_json_files" -rm "$user_files_path"/01825_file_*.json +rm "$USER_FILES_PATH"/${CLICKHOUSE_DATABASE}_*.json diff --git a/tests/queries/0_stateless/01825_type_json_schema_inference.sh b/tests/queries/0_stateless/01825_type_json_schema_inference.sh index 5fca608d8bb..e0c283b2230 100755 --- a/tests/queries/0_stateless/01825_type_json_schema_inference.sh +++ b/tests/queries/0_stateless/01825_type_json_schema_inference.sh @@ -10,11 +10,10 @@ ${CLICKHOUSE_CLIENT} -q "DROP TABLE IF EXISTS t_json_inference" ${CLICKHOUSE_CLIENT} -q "CREATE TABLE t_json_inference (id UInt64, obj Object(Nullable('json')), s String) \ ENGINE = MergeTree ORDER BY id" --allow_experimental_object_type 1 -user_files_path=$($CLICKHOUSE_CLIENT_BINARY --query "select _path,_file from file('nonexist.txt', 'CSV', 'val1 char')" 2>&1 | grep Exception | awk '{gsub("/nonexist.txt","",$9); print $9}') -mkdir -p ${user_files_path}/${CLICKHOUSE_TEST_UNIQUE_NAME}/ -rm -rf ${user_files_path}/${CLICKHOUSE_TEST_UNIQUE_NAME:?}/* +mkdir -p ${USER_FILES_PATH}/${CLICKHOUSE_TEST_UNIQUE_NAME}/ +rm -rf ${USER_FILES_PATH}/${CLICKHOUSE_TEST_UNIQUE_NAME:?}/* -filename="${user_files_path}/${CLICKHOUSE_TEST_UNIQUE_NAME}/data.json" +filename="${USER_FILES_PATH}/${CLICKHOUSE_TEST_UNIQUE_NAME}/data.json" echo '{"id": 1, "obj": {"k1": 1, "k2": {"k3": 2, "k4": [{"k5": 3}, {"k5": 4}]}}, "s": "foo"}' > $filename echo '{"id": 2, "obj": {"k2": {"k3": "str", "k4": [{"k6": 55}]}, "some": 42}, "s": "bar"}' >> $filename diff --git a/tests/queries/0_stateless/01875_ssd_cache_dictionary_decimal256_type.sh b/tests/queries/0_stateless/01875_ssd_cache_dictionary_decimal256_type.sh index 36a2165329b..8336229a643 100755 --- a/tests/queries/0_stateless/01875_ssd_cache_dictionary_decimal256_type.sh +++ b/tests/queries/0_stateless/01875_ssd_cache_dictionary_decimal256_type.sh @@ -5,8 +5,6 @@ CURDIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd) # shellcheck source=../shell_config.sh . "$CURDIR"/../shell_config.sh -USER_FILES_PATH=$($CLICKHOUSE_CLIENT_BINARY --query "select _path,_file from file('nonexist.txt', 'CSV', 'val1 char')" 2>&1 | grep Exception | awk '{gsub("/nonexist.txt","",$9); print $9}') - $CLICKHOUSE_CLIENT -n --query=" SET allow_experimental_bigint_types = 1; diff --git a/tests/queries/0_stateless/01889_check_row_policy_defined_using_user_function.sh b/tests/queries/0_stateless/01889_check_row_policy_defined_using_user_function.sh index b5be39a91df..f79637a7635 100755 --- a/tests/queries/0_stateless/01889_check_row_policy_defined_using_user_function.sh +++ b/tests/queries/0_stateless/01889_check_row_policy_defined_using_user_function.sh @@ -1,5 +1,4 @@ #!/usr/bin/env bash -# Tags: no-parallel CLICKHOUSE_CLIENT_SERVER_LOGS_LEVEL=none @@ -7,23 +6,27 @@ CURDIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd) # shellcheck source=../shell_config.sh . "$CURDIR"/../shell_config.sh -${CLICKHOUSE_CLIENT} -q "drop user if exists u_01889" -${CLICKHOUSE_CLIENT} -q "drop role if exists r_01889" -${CLICKHOUSE_CLIENT} -q "drop policy if exists t_01889_filter on t_01889" -${CLICKHOUSE_CLIENT} -q "create user u_01889 identified with plaintext_password by 'dfsdffdf5t123'" -${CLICKHOUSE_CLIENT} -q "revoke all on *.* from u_01889" -${CLICKHOUSE_CLIENT} -q "create role r_01889" +USER=u_01889$RANDOM +ROLE=r_01889$RANDOM +POLICY=t_01889_filter$RANDOM + +${CLICKHOUSE_CLIENT} -q "drop user if exists $USER" +${CLICKHOUSE_CLIENT} -q "drop role if exists ${ROLE}" +${CLICKHOUSE_CLIENT} -q "drop policy if exists ${POLICY} on t_01889" +${CLICKHOUSE_CLIENT} -q "create user $USER identified with plaintext_password by 'dfsdffdf5t123'" +${CLICKHOUSE_CLIENT} -q "revoke all on *.* from $USER" +${CLICKHOUSE_CLIENT} -q "create role ${ROLE}" ${CLICKHOUSE_CLIENT} -q "create table t_01889(a Int64, user_id String) Engine=MergeTree order by a" -${CLICKHOUSE_CLIENT} -q "insert into t_01889 select number, 'u_01889' from numbers(1000)" +${CLICKHOUSE_CLIENT} -q "insert into t_01889 select number, '$USER' from numbers(1000)" ${CLICKHOUSE_CLIENT} -q "insert into t_01889 select number, 'xxxxxxx' from numbers(1000)" -${CLICKHOUSE_CLIENT} -q "grant select on t_01889 to r_01889" -${CLICKHOUSE_CLIENT} -q "create row policy t_01889_filter ON t_01889 FOR SELECT USING user_id = user() TO r_01889" -${CLICKHOUSE_CLIENT} -q "grant r_01889 to u_01889" -${CLICKHOUSE_CLIENT} -q "alter user u_01889 default role r_01889 settings none" +${CLICKHOUSE_CLIENT} -q "grant select on t_01889 to ${ROLE}" +${CLICKHOUSE_CLIENT} -q "create row policy ${POLICY} ON t_01889 FOR SELECT USING user_id = user() TO ${ROLE}" +${CLICKHOUSE_CLIENT} -q "grant ${ROLE} to $USER" +${CLICKHOUSE_CLIENT} -q "alter user $USER default role ${ROLE} settings none" -${CLICKHOUSE_CLIENT_BINARY} --database=${CLICKHOUSE_DATABASE} --user=u_01889 --password=dfsdffdf5t123 --query="select count() from t_01889" +${CLICKHOUSE_CLIENT_BINARY} --database=${CLICKHOUSE_DATABASE} --user=$USER --password=dfsdffdf5t123 --query="select count() from t_01889" -${CLICKHOUSE_CLIENT} -q "drop user u_01889" -${CLICKHOUSE_CLIENT} -q "drop policy t_01889_filter on t_01889" -${CLICKHOUSE_CLIENT} -q "drop role r_01889" +${CLICKHOUSE_CLIENT} -q "drop user $USER" +${CLICKHOUSE_CLIENT} -q "drop policy ${POLICY} on t_01889" +${CLICKHOUSE_CLIENT} -q "drop role ${ROLE}" ${CLICKHOUSE_CLIENT} -q "drop table t_01889" diff --git a/tests/queries/0_stateless/01889_sqlite_read_write.sh b/tests/queries/0_stateless/01889_sqlite_read_write.sh index 30496af46f6..63ce5dc909c 100755 --- a/tests/queries/0_stateless/01889_sqlite_read_write.sh +++ b/tests/queries/0_stateless/01889_sqlite_read_write.sh @@ -6,15 +6,9 @@ CUR_DIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd) # shellcheck source=../shell_config.sh . "$CUR_DIR"/../shell_config.sh -# See 01658_read_file_to_string_column.sh -user_files_path=$($CLICKHOUSE_CLIENT_BINARY --query "select _path,_file from file('nonexist.txt', 'CSV', 'val1 char')" 2>&1 | grep Exception | awk '{gsub("/nonexist.txt","",$9); print $9}') - -mkdir -p "${user_files_path}/" -chmod 777 "${user_files_path}" - export CURR_DATABASE="test_01889_sqllite_${CLICKHOUSE_DATABASE}" -DB_PATH=${user_files_path}/${CURR_DATABASE}_db1 +DB_PATH=${USER_FILES_PATH}/${CURR_DATABASE}_db1 DB_PATH2=$CUR_DIR/${CURR_DATABASE}_db2 function cleanup() diff --git a/tests/queries/0_stateless/01903_ssd_cache_dictionary_array_type.sh b/tests/queries/0_stateless/01903_ssd_cache_dictionary_array_type.sh index 3676f1429b2..853445daf3f 100755 --- a/tests/queries/0_stateless/01903_ssd_cache_dictionary_array_type.sh +++ b/tests/queries/0_stateless/01903_ssd_cache_dictionary_array_type.sh @@ -5,8 +5,6 @@ CURDIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd) # shellcheck source=../shell_config.sh . "$CURDIR"/../shell_config.sh -USER_FILES_PATH=$($CLICKHOUSE_CLIENT_BINARY --query "select _path,_file from file('nonexist.txt', 'CSV', 'val1 char')" 2>&1 | grep Exception | awk '{gsub("/nonexist.txt","",$9); print $9}') - $CLICKHOUSE_CLIENT -n --query=" DROP TABLE IF EXISTS dictionary_array_source_table; CREATE TABLE dictionary_array_source_table diff --git a/tests/queries/0_stateless/01904_ssd_cache_dictionary_default_nullable_type.sh b/tests/queries/0_stateless/01904_ssd_cache_dictionary_default_nullable_type.sh index 6aecb20329a..0b555cf82c2 100755 --- a/tests/queries/0_stateless/01904_ssd_cache_dictionary_default_nullable_type.sh +++ b/tests/queries/0_stateless/01904_ssd_cache_dictionary_default_nullable_type.sh @@ -5,8 +5,6 @@ CURDIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd) # shellcheck source=../shell_config.sh . "$CURDIR"/../shell_config.sh -USER_FILES_PATH=$($CLICKHOUSE_CLIENT_BINARY --query "select _path,_file from file('nonexist.txt', 'CSV', 'val1 char')" 2>&1 | grep Exception | awk '{gsub("/nonexist.txt","",$9); print $9}') - $CLICKHOUSE_CLIENT -n --query=" DROP TABLE IF EXISTS dictionary_nullable_source_table; CREATE TABLE dictionary_nullable_source_table diff --git a/tests/queries/0_stateless/01910_view_dictionary.sql b/tests/queries/0_stateless/01910_view_dictionary.sql index 05a67889825..51f46decadd 100644 --- a/tests/queries/0_stateless/01910_view_dictionary.sql +++ b/tests/queries/0_stateless/01910_view_dictionary.sql @@ -34,7 +34,7 @@ CREATE DICTIONARY flat_dictionary value_ru String ) PRIMARY KEY id -SOURCE(CLICKHOUSE(HOST 'localhost' PORT 9000 USER 'default' PASSWORD '' TABLE 'dictionary_source_view')) +SOURCE(CLICKHOUSE(HOST 'localhost' PORT tcpPort() USER 'default' PASSWORD '' TABLE 'dictionary_source_view')) LIFETIME(MIN 1 MAX 1000) LAYOUT(FLAT()); diff --git a/tests/queries/0_stateless/01921_concurrent_ttl_and_normal_merges_zookeeper_long.sh b/tests/queries/0_stateless/01921_concurrent_ttl_and_normal_merges_zookeeper_long.sh index 5e1600a0673..edffc0a3807 100755 --- a/tests/queries/0_stateless/01921_concurrent_ttl_and_normal_merges_zookeeper_long.sh +++ b/tests/queries/0_stateless/01921_concurrent_ttl_and_normal_merges_zookeeper_long.sh @@ -1,5 +1,5 @@ #!/usr/bin/env bash -# Tags: long, zookeeper, no-parallel +# Tags: long, zookeeper CLICKHOUSE_CLIENT_SERVER_LOGS_LEVEL=error @@ -50,7 +50,7 @@ function insert_thread export -f insert_thread; export -f optimize_thread; -TIMEOUT=30 +TIMEOUT=20 timeout $TIMEOUT bash -c insert_thread 2> /dev/null & timeout $TIMEOUT bash -c insert_thread 2> /dev/null & diff --git a/tests/queries/0_stateless/01946_test_zstd_decompression_with_escape_sequence_at_the_end_of_buffer.sh b/tests/queries/0_stateless/01946_test_zstd_decompression_with_escape_sequence_at_the_end_of_buffer.sh index 4b230e4f738..0aedef028a2 100755 --- a/tests/queries/0_stateless/01946_test_zstd_decompression_with_escape_sequence_at_the_end_of_buffer.sh +++ b/tests/queries/0_stateless/01946_test_zstd_decompression_with_escape_sequence_at_the_end_of_buffer.sh @@ -6,10 +6,8 @@ CUR_DIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd) . "$CUR_DIR"/../shell_config.sh -# See 01658_read_file_to_string_column.sh -user_files_path=$($CLICKHOUSE_CLIENT_BINARY --query "select _path,_file from file('nonexist.txt', 'CSV', 'val1 char')" 2>&1 | grep Exception | awk '{gsub("/nonexist.txt","",$9); print $9}') -mkdir -p ${user_files_path}/ -cp $CUR_DIR/data_zstd/test_01946.zstd ${user_files_path}/ +mkdir -p ${USER_FILES_PATH}/ +cp $CUR_DIR/data_zstd/test_01946.zstd ${USER_FILES_PATH}/ ${CLICKHOUSE_CLIENT} --multiline --multiquery --query " set min_chunk_bytes_for_parallel_parsing=10485760; diff --git a/tests/queries/0_stateless/02003_compress_bz2.sh b/tests/queries/0_stateless/02003_compress_bz2.sh index b17effb20b6..edc433ad69b 100755 --- a/tests/queries/0_stateless/02003_compress_bz2.sh +++ b/tests/queries/0_stateless/02003_compress_bz2.sh @@ -6,7 +6,6 @@ CURDIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd) # shellcheck source=../shell_config.sh . "$CURDIR"/../shell_config.sh -USER_FILES_PATH=$($CLICKHOUSE_CLIENT_BINARY --query "select _path,_file from file('nonexist.txt', 'CSV', 'val1 char')" 2>&1 | grep Exception | awk '{gsub("/nonexist.txt","",$9); print $9}') WORKING_FOLDER_02003="${USER_FILES_PATH}/${CLICKHOUSE_DATABASE}" rm -rf "${WORKING_FOLDER_02003}" diff --git a/tests/queries/0_stateless/02012_compress_lz4.sh b/tests/queries/0_stateless/02012_compress_lz4.sh index aad437c8011..700bff613da 100755 --- a/tests/queries/0_stateless/02012_compress_lz4.sh +++ b/tests/queries/0_stateless/02012_compress_lz4.sh @@ -4,7 +4,6 @@ CURDIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd) # shellcheck source=../shell_config.sh . "$CURDIR"/../shell_config.sh -USER_FILES_PATH=$($CLICKHOUSE_CLIENT_BINARY --query "select _path,_file from file('nonexist.txt', 'CSV', 'val1 char')" 2>&1 | grep Exception | awk '{gsub("/nonexist.txt","",$9); print $9}') WORKING_FOLDER_02012="${USER_FILES_PATH}/${CLICKHOUSE_DATABASE}" rm -rf "${WORKING_FOLDER_02012}" diff --git a/tests/queries/0_stateless/02022_storage_filelog_one_file.sh b/tests/queries/0_stateless/02022_storage_filelog_one_file.sh index ea703d69aa5..29bc28849c8 100755 --- a/tests/queries/0_stateless/02022_storage_filelog_one_file.sh +++ b/tests/queries/0_stateless/02022_storage_filelog_one_file.sh @@ -6,30 +6,26 @@ CURDIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd) # shellcheck source=../shell_config.sh . "$CURDIR"/../shell_config.sh -# Data preparation. -# Now we can get the user_files_path by use the table file function for trick. also we can get it by query as: -# "insert into function file('exist.txt', 'CSV', 'val1 char') values ('aaaa'); select _path from file('exist.txt', 'CSV', 'val1 char')" -user_files_path=$($CLICKHOUSE_CLIENT_BINARY --query "select _path,_file from file('nonexist.txt', 'CSV', 'val1 char')" 2>&1 | grep Exception | awk '{gsub("/nonexist.txt","",$9); print $9}') for i in {1..20} do - echo $i, $i >> ${user_files_path}/${CLICKHOUSE_TEST_UNIQUE_NAME}.txt + echo $i, $i >> ${USER_FILES_PATH}/${CLICKHOUSE_TEST_UNIQUE_NAME}.txt done ${CLICKHOUSE_CLIENT} --query "drop table if exists file_log;" -${CLICKHOUSE_CLIENT} --query "create table file_log(k UInt8, v UInt8) engine=FileLog('${user_files_path}/${CLICKHOUSE_TEST_UNIQUE_NAME}.txt', 'CSV');" +${CLICKHOUSE_CLIENT} --query "create table file_log(k UInt8, v UInt8) engine=FileLog('${USER_FILES_PATH}/${CLICKHOUSE_TEST_UNIQUE_NAME}.txt', 'CSV');" ${CLICKHOUSE_CLIENT} --query "select * from file_log order by k settings stream_like_engine_allow_direct_select=1;" for i in {100..120} do - echo $i, $i >> ${user_files_path}/${CLICKHOUSE_TEST_UNIQUE_NAME}.txt + echo $i, $i >> ${USER_FILES_PATH}/${CLICKHOUSE_TEST_UNIQUE_NAME}.txt done ${CLICKHOUSE_CLIENT} --query "select * from file_log order by k settings stream_like_engine_allow_direct_select=1;" # touch does not change file content, no event -touch ${user_files_path}/${CLICKHOUSE_TEST_UNIQUE_NAME}.txt +touch ${USER_FILES_PATH}/${CLICKHOUSE_TEST_UNIQUE_NAME}.txt ${CLICKHOUSE_CLIENT} --query "select * from file_log order by k settings stream_like_engine_allow_direct_select=1;" ${CLICKHOUSE_CLIENT} --query "detach table file_log;" @@ -40,4 +36,4 @@ ${CLICKHOUSE_CLIENT} --query "select * from file_log order by k settings stream_ ${CLICKHOUSE_CLIENT} --query "drop table file_log;" -rm -rf ${user_files_path}/${CLICKHOUSE_TEST_UNIQUE_NAME}.txt +rm -rf ${USER_FILES_PATH}/${CLICKHOUSE_TEST_UNIQUE_NAME}.txt diff --git a/tests/queries/0_stateless/02023_storage_filelog.sh b/tests/queries/0_stateless/02023_storage_filelog.sh index 51c8dc8ab3e..798ffe66ed9 100755 --- a/tests/queries/0_stateless/02023_storage_filelog.sh +++ b/tests/queries/0_stateless/02023_storage_filelog.sh @@ -6,57 +6,52 @@ CURDIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd) # shellcheck source=../shell_config.sh . "$CURDIR"/../shell_config.sh -# Data preparation. -# Now we can get the user_files_path by use the table file function for trick. also we can get it by query as: -# "insert into function file('exist.txt', 'CSV', 'val1 char') values ('aaaa'); select _path from file('exist.txt', 'CSV', 'val1 char')" -user_files_path=$($CLICKHOUSE_CLIENT_BINARY --query "select _path,_file from file('nonexist.txt', 'CSV', 'val1 char')" 2>&1 | grep Exception | awk '{gsub("/nonexist.txt","",$9); print $9}') +mkdir -p ${USER_FILES_PATH}/${CLICKHOUSE_TEST_UNIQUE_NAME}/ -mkdir -p ${user_files_path}/${CLICKHOUSE_TEST_UNIQUE_NAME}/ - -rm -rf ${user_files_path}/${CLICKHOUSE_TEST_UNIQUE_NAME:?}/* +rm -rf ${USER_FILES_PATH}/${CLICKHOUSE_TEST_UNIQUE_NAME:?}/* for i in {1..20} do - echo $i, $i >> ${user_files_path}/${CLICKHOUSE_TEST_UNIQUE_NAME}/a.txt + echo $i, $i >> ${USER_FILES_PATH}/${CLICKHOUSE_TEST_UNIQUE_NAME}/a.txt done ${CLICKHOUSE_CLIENT} --query "drop table if exists file_log;" -${CLICKHOUSE_CLIENT} --query "create table file_log(k UInt8, v UInt8) engine=FileLog('${user_files_path}/${CLICKHOUSE_TEST_UNIQUE_NAME}/', 'CSV');" +${CLICKHOUSE_CLIENT} --query "create table file_log(k UInt8, v UInt8) engine=FileLog('${USER_FILES_PATH}/${CLICKHOUSE_TEST_UNIQUE_NAME}/', 'CSV');" ${CLICKHOUSE_CLIENT} --query "select * from file_log order by k settings stream_like_engine_allow_direct_select=1;" -cp ${user_files_path}/${CLICKHOUSE_TEST_UNIQUE_NAME}/a.txt ${user_files_path}/${CLICKHOUSE_TEST_UNIQUE_NAME}/b.txt +cp ${USER_FILES_PATH}/${CLICKHOUSE_TEST_UNIQUE_NAME}/a.txt ${USER_FILES_PATH}/${CLICKHOUSE_TEST_UNIQUE_NAME}/b.txt ${CLICKHOUSE_CLIENT} --query "select * from file_log order by k settings stream_like_engine_allow_direct_select=1;" for i in {100..120} do - echo $i, $i >> ${user_files_path}/${CLICKHOUSE_TEST_UNIQUE_NAME}/a.txt + echo $i, $i >> ${USER_FILES_PATH}/${CLICKHOUSE_TEST_UNIQUE_NAME}/a.txt done # touch does not change file content, no event -touch ${user_files_path}/${CLICKHOUSE_TEST_UNIQUE_NAME}/a.txt +touch ${USER_FILES_PATH}/${CLICKHOUSE_TEST_UNIQUE_NAME}/a.txt -cp ${user_files_path}/${CLICKHOUSE_TEST_UNIQUE_NAME}/a.txt ${user_files_path}/${CLICKHOUSE_TEST_UNIQUE_NAME}/c.txt -cp ${user_files_path}/${CLICKHOUSE_TEST_UNIQUE_NAME}/a.txt ${user_files_path}/${CLICKHOUSE_TEST_UNIQUE_NAME}/d.txt -cp ${user_files_path}/${CLICKHOUSE_TEST_UNIQUE_NAME}/a.txt ${user_files_path}/${CLICKHOUSE_TEST_UNIQUE_NAME}/e.txt -mv ${user_files_path}/${CLICKHOUSE_TEST_UNIQUE_NAME}/b.txt ${user_files_path}/${CLICKHOUSE_TEST_UNIQUE_NAME}/j.txt +cp ${USER_FILES_PATH}/${CLICKHOUSE_TEST_UNIQUE_NAME}/a.txt ${USER_FILES_PATH}/${CLICKHOUSE_TEST_UNIQUE_NAME}/c.txt +cp ${USER_FILES_PATH}/${CLICKHOUSE_TEST_UNIQUE_NAME}/a.txt ${USER_FILES_PATH}/${CLICKHOUSE_TEST_UNIQUE_NAME}/d.txt +cp ${USER_FILES_PATH}/${CLICKHOUSE_TEST_UNIQUE_NAME}/a.txt ${USER_FILES_PATH}/${CLICKHOUSE_TEST_UNIQUE_NAME}/e.txt +mv ${USER_FILES_PATH}/${CLICKHOUSE_TEST_UNIQUE_NAME}/b.txt ${USER_FILES_PATH}/${CLICKHOUSE_TEST_UNIQUE_NAME}/j.txt -rm ${user_files_path}/${CLICKHOUSE_TEST_UNIQUE_NAME}/d.txt +rm ${USER_FILES_PATH}/${CLICKHOUSE_TEST_UNIQUE_NAME}/d.txt ${CLICKHOUSE_CLIENT} --query "select * from file_log order by k settings stream_like_engine_allow_direct_select=1;" ${CLICKHOUSE_CLIENT} --query "detach table file_log;" -cp ${user_files_path}/${CLICKHOUSE_TEST_UNIQUE_NAME}/e.txt ${user_files_path}/${CLICKHOUSE_TEST_UNIQUE_NAME}/f.txt -mv ${user_files_path}/${CLICKHOUSE_TEST_UNIQUE_NAME}/e.txt ${user_files_path}/${CLICKHOUSE_TEST_UNIQUE_NAME}/g.txt -mv ${user_files_path}/${CLICKHOUSE_TEST_UNIQUE_NAME}/c.txt ${user_files_path}/${CLICKHOUSE_TEST_UNIQUE_NAME}/h.txt +cp ${USER_FILES_PATH}/${CLICKHOUSE_TEST_UNIQUE_NAME}/e.txt ${USER_FILES_PATH}/${CLICKHOUSE_TEST_UNIQUE_NAME}/f.txt +mv ${USER_FILES_PATH}/${CLICKHOUSE_TEST_UNIQUE_NAME}/e.txt ${USER_FILES_PATH}/${CLICKHOUSE_TEST_UNIQUE_NAME}/g.txt +mv ${USER_FILES_PATH}/${CLICKHOUSE_TEST_UNIQUE_NAME}/c.txt ${USER_FILES_PATH}/${CLICKHOUSE_TEST_UNIQUE_NAME}/h.txt for i in {150..200} do - echo $i, $i >> ${user_files_path}/${CLICKHOUSE_TEST_UNIQUE_NAME}/h.txt + echo $i, $i >> ${USER_FILES_PATH}/${CLICKHOUSE_TEST_UNIQUE_NAME}/h.txt done for i in {200..250} do - echo $i, $i >> ${user_files_path}/${CLICKHOUSE_TEST_UNIQUE_NAME}/i.txt + echo $i, $i >> ${USER_FILES_PATH}/${CLICKHOUSE_TEST_UNIQUE_NAME}/i.txt done ${CLICKHOUSE_CLIENT} --query "attach table file_log;" @@ -68,11 +63,11 @@ ${CLICKHOUSE_CLIENT} --query "attach table file_log;" # should no records return ${CLICKHOUSE_CLIENT} --query "select * from file_log order by k settings stream_like_engine_allow_direct_select=1;" -truncate ${user_files_path}/${CLICKHOUSE_TEST_UNIQUE_NAME}/a.txt --size 0 +truncate ${USER_FILES_PATH}/${CLICKHOUSE_TEST_UNIQUE_NAME}/a.txt --size 0 # exception happend ${CLICKHOUSE_CLIENT} --query "select * from file_log order by k settings stream_like_engine_allow_direct_select=1;" 2>&1 | grep -q "Code: 33" && echo 'OK' || echo 'FAIL' ${CLICKHOUSE_CLIENT} --query "drop table file_log;" -rm -rf ${user_files_path}/${CLICKHOUSE_TEST_UNIQUE_NAME:?} +rm -rf ${USER_FILES_PATH}/${CLICKHOUSE_TEST_UNIQUE_NAME:?} diff --git a/tests/queries/0_stateless/02024_storage_filelog_mv.sh b/tests/queries/0_stateless/02024_storage_filelog_mv.sh index 33c8693648c..a47ca6c7507 100755 --- a/tests/queries/0_stateless/02024_storage_filelog_mv.sh +++ b/tests/queries/0_stateless/02024_storage_filelog_mv.sh @@ -7,21 +7,16 @@ CURDIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd) # shellcheck source=../shell_config.sh . "$CURDIR"/../shell_config.sh -# Data preparation. -# Now we can get the user_files_path by use the table file function for trick. also we can get it by query as: -# "insert into function file('exist.txt', 'CSV', 'val1 char') values ('aaaa'); select _path from file('exist.txt', 'CSV', 'val1 char')" -user_files_path=$($CLICKHOUSE_CLIENT_BINARY --query "select _path,_file from file('nonexist.txt', 'CSV', 'val1 char')" 2>&1 | grep Exception | awk '{gsub("/nonexist.txt","",$9); print $9}') - -mkdir -p ${user_files_path}/${CLICKHOUSE_TEST_UNIQUE_NAME}/ -rm -rf ${user_files_path}/${CLICKHOUSE_TEST_UNIQUE_NAME:?}/* +mkdir -p ${USER_FILES_PATH}/${CLICKHOUSE_TEST_UNIQUE_NAME}/ +rm -rf ${USER_FILES_PATH}/${CLICKHOUSE_TEST_UNIQUE_NAME:?}/* for i in {1..20} do - echo $i, $i >> ${user_files_path}/${CLICKHOUSE_TEST_UNIQUE_NAME}/a.txt + echo $i, $i >> ${USER_FILES_PATH}/${CLICKHOUSE_TEST_UNIQUE_NAME}/a.txt done ${CLICKHOUSE_CLIENT} --query "drop table if exists file_log;" -${CLICKHOUSE_CLIENT} --query "create table file_log(k UInt8, v UInt8) engine=FileLog('${user_files_path}/${CLICKHOUSE_TEST_UNIQUE_NAME}/', 'CSV');" +${CLICKHOUSE_CLIENT} --query "create table file_log(k UInt8, v UInt8) engine=FileLog('${USER_FILES_PATH}/${CLICKHOUSE_TEST_UNIQUE_NAME}/', 'CSV');" ${CLICKHOUSE_CLIENT} --query "drop table if exists mv;" ${CLICKHOUSE_CLIENT} --query "create Materialized View mv engine=MergeTree order by k as select * from file_log;" @@ -39,17 +34,17 @@ done ${CLICKHOUSE_CLIENT} --query "select * from mv order by k;" -cp ${user_files_path}/${CLICKHOUSE_TEST_UNIQUE_NAME}/a.txt ${user_files_path}/${CLICKHOUSE_TEST_UNIQUE_NAME}/b.txt +cp ${USER_FILES_PATH}/${CLICKHOUSE_TEST_UNIQUE_NAME}/a.txt ${USER_FILES_PATH}/${CLICKHOUSE_TEST_UNIQUE_NAME}/b.txt # touch does not change file content, no event -touch ${user_files_path}/${CLICKHOUSE_TEST_UNIQUE_NAME}/a.txt +touch ${USER_FILES_PATH}/${CLICKHOUSE_TEST_UNIQUE_NAME}/a.txt -cp ${user_files_path}/${CLICKHOUSE_TEST_UNIQUE_NAME}/a.txt ${user_files_path}/${CLICKHOUSE_TEST_UNIQUE_NAME}/c.txt -cp ${user_files_path}/${CLICKHOUSE_TEST_UNIQUE_NAME}/a.txt ${user_files_path}/${CLICKHOUSE_TEST_UNIQUE_NAME}/d.txt +cp ${USER_FILES_PATH}/${CLICKHOUSE_TEST_UNIQUE_NAME}/a.txt ${USER_FILES_PATH}/${CLICKHOUSE_TEST_UNIQUE_NAME}/c.txt +cp ${USER_FILES_PATH}/${CLICKHOUSE_TEST_UNIQUE_NAME}/a.txt ${USER_FILES_PATH}/${CLICKHOUSE_TEST_UNIQUE_NAME}/d.txt for i in {100..120} do - echo $i, $i >> ${user_files_path}/${CLICKHOUSE_TEST_UNIQUE_NAME}/d.txt + echo $i, $i >> ${USER_FILES_PATH}/${CLICKHOUSE_TEST_UNIQUE_NAME}/d.txt done while true; do @@ -62,4 +57,4 @@ ${CLICKHOUSE_CLIENT} --query "select * from mv order by k;" ${CLICKHOUSE_CLIENT} --query "drop table mv;" ${CLICKHOUSE_CLIENT} --query "drop table file_log;" -rm -rf ${user_files_path}/${CLICKHOUSE_TEST_UNIQUE_NAME:?} +rm -rf ${USER_FILES_PATH}/${CLICKHOUSE_TEST_UNIQUE_NAME:?} diff --git a/tests/queries/0_stateless/02025_storage_filelog_virtual_col.sh b/tests/queries/0_stateless/02025_storage_filelog_virtual_col.sh index f027b61c3ef..7e414b8863e 100755 --- a/tests/queries/0_stateless/02025_storage_filelog_virtual_col.sh +++ b/tests/queries/0_stateless/02025_storage_filelog_virtual_col.sh @@ -6,42 +6,37 @@ CURDIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd) # shellcheck source=../shell_config.sh . "$CURDIR"/../shell_config.sh -# Data preparation. -# Now we can get the user_files_path by use the table file function for trick. also we can get it by query as: -# "insert into function file('exist.txt', 'CSV', 'val1 char') values ('aaaa'); select _path from file('exist.txt', 'CSV', 'val1 char')" -user_files_path=$($CLICKHOUSE_CLIENT_BINARY --query "select _path,_file from file('nonexist.txt', 'CSV', 'val1 char')" 2>&1 | grep Exception | awk '{gsub("/nonexist.txt","",$9); print $9}') +mkdir -p ${USER_FILES_PATH}/${CLICKHOUSE_TEST_UNIQUE_NAME}/ -mkdir -p ${user_files_path}/${CLICKHOUSE_TEST_UNIQUE_NAME}/ - -rm -rf ${user_files_path}/${CLICKHOUSE_TEST_UNIQUE_NAME:?}/* +rm -rf ${USER_FILES_PATH}/${CLICKHOUSE_TEST_UNIQUE_NAME:?}/* for i in {1..20} do - echo $i, $i >> ${user_files_path}/${CLICKHOUSE_TEST_UNIQUE_NAME}/a.txt + echo $i, $i >> ${USER_FILES_PATH}/${CLICKHOUSE_TEST_UNIQUE_NAME}/a.txt done ${CLICKHOUSE_CLIENT} --query "drop table if exists file_log;" -${CLICKHOUSE_CLIENT} --query "create table file_log(k UInt8, v UInt8) engine=FileLog('${user_files_path}/${CLICKHOUSE_TEST_UNIQUE_NAME}/', 'CSV');" +${CLICKHOUSE_CLIENT} --query "create table file_log(k UInt8, v UInt8) engine=FileLog('${USER_FILES_PATH}/${CLICKHOUSE_TEST_UNIQUE_NAME}/', 'CSV');" ${CLICKHOUSE_CLIENT} --query "select *, _filename, _offset from file_log order by _filename, _offset settings stream_like_engine_allow_direct_select=1;" -cp ${user_files_path}/${CLICKHOUSE_TEST_UNIQUE_NAME}/a.txt ${user_files_path}/${CLICKHOUSE_TEST_UNIQUE_NAME}/b.txt +cp ${USER_FILES_PATH}/${CLICKHOUSE_TEST_UNIQUE_NAME}/a.txt ${USER_FILES_PATH}/${CLICKHOUSE_TEST_UNIQUE_NAME}/b.txt ${CLICKHOUSE_CLIENT} --query "select *, _filename, _offset from file_log order by _filename, _offset settings stream_like_engine_allow_direct_select=1;" for i in {100..120} do - echo $i, $i >> ${user_files_path}/${CLICKHOUSE_TEST_UNIQUE_NAME}/a.txt + echo $i, $i >> ${USER_FILES_PATH}/${CLICKHOUSE_TEST_UNIQUE_NAME}/a.txt done # touch does not change file content, no event -touch ${user_files_path}/${CLICKHOUSE_TEST_UNIQUE_NAME}/a.txt +touch ${USER_FILES_PATH}/${CLICKHOUSE_TEST_UNIQUE_NAME}/a.txt -cp ${user_files_path}/${CLICKHOUSE_TEST_UNIQUE_NAME}/a.txt ${user_files_path}/${CLICKHOUSE_TEST_UNIQUE_NAME}/c.txt -cp ${user_files_path}/${CLICKHOUSE_TEST_UNIQUE_NAME}/a.txt ${user_files_path}/${CLICKHOUSE_TEST_UNIQUE_NAME}/d.txt -cp ${user_files_path}/${CLICKHOUSE_TEST_UNIQUE_NAME}/a.txt ${user_files_path}/${CLICKHOUSE_TEST_UNIQUE_NAME}/e.txt +cp ${USER_FILES_PATH}/${CLICKHOUSE_TEST_UNIQUE_NAME}/a.txt ${USER_FILES_PATH}/${CLICKHOUSE_TEST_UNIQUE_NAME}/c.txt +cp ${USER_FILES_PATH}/${CLICKHOUSE_TEST_UNIQUE_NAME}/a.txt ${USER_FILES_PATH}/${CLICKHOUSE_TEST_UNIQUE_NAME}/d.txt +cp ${USER_FILES_PATH}/${CLICKHOUSE_TEST_UNIQUE_NAME}/a.txt ${USER_FILES_PATH}/${CLICKHOUSE_TEST_UNIQUE_NAME}/e.txt -rm ${user_files_path}/${CLICKHOUSE_TEST_UNIQUE_NAME}/d.txt +rm ${USER_FILES_PATH}/${CLICKHOUSE_TEST_UNIQUE_NAME}/d.txt ${CLICKHOUSE_CLIENT} --query "select *, _filename, _offset from file_log order by _filename, _offset settings stream_like_engine_allow_direct_select=1;" @@ -51,11 +46,11 @@ ${CLICKHOUSE_CLIENT} --query "attach table file_log;" # should no records return ${CLICKHOUSE_CLIENT} --query "select *, _filename, _offset from file_log order by _filename, _offset settings stream_like_engine_allow_direct_select=1;" -truncate ${user_files_path}/${CLICKHOUSE_TEST_UNIQUE_NAME}/a.txt --size 0 +truncate ${USER_FILES_PATH}/${CLICKHOUSE_TEST_UNIQUE_NAME}/a.txt --size 0 # exception happend ${CLICKHOUSE_CLIENT} --query "select * from file_log order by k settings stream_like_engine_allow_direct_select=1;" 2>&1 | grep -q "Code: 33" && echo 'OK' || echo 'FAIL' ${CLICKHOUSE_CLIENT} --query "drop table file_log;" -rm -rf ${user_files_path}/${CLICKHOUSE_TEST_UNIQUE_NAME:?} +rm -rf ${USER_FILES_PATH}/${CLICKHOUSE_TEST_UNIQUE_NAME:?} diff --git a/tests/queries/0_stateless/02026_storage_filelog_largefile.sh b/tests/queries/0_stateless/02026_storage_filelog_largefile.sh index b0a9a4357f3..b7377fb026a 100755 --- a/tests/queries/0_stateless/02026_storage_filelog_largefile.sh +++ b/tests/queries/0_stateless/02026_storage_filelog_largefile.sh @@ -7,34 +7,27 @@ CURDIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd) # shellcheck source=../shell_config.sh . "$CURDIR"/../shell_config.sh -# Data preparation. -# Now we can get the user_files_path by use the table file function for trick. also we can get it by query as: -# "insert into function file('exist.txt', 'CSV', 'val1 char') values ('aaaa'); select _path from file('exist.txt', 'CSV', 'val1 char')" -user_files_path=$($CLICKHOUSE_CLIENT_BINARY --query "select _path,_file from file('nonexist.txt', 'CSV', 'val1 char')" 2>&1 | grep Exception | awk '{gsub("/nonexist.txt","",$9); print $9}') - -mkdir -p ${user_files_path}/${CLICKHOUSE_TEST_UNIQUE_NAME}/ - -rm -rf ${user_files_path}/${CLICKHOUSE_TEST_UNIQUE_NAME:?}/* - -chmod 777 ${user_files_path}/${CLICKHOUSE_TEST_UNIQUE_NAME}/ +mkdir -p ${USER_FILES_PATH}/${CLICKHOUSE_TEST_UNIQUE_NAME}/ +rm -rf ${USER_FILES_PATH}/${CLICKHOUSE_TEST_UNIQUE_NAME:?}/* +chmod 777 ${USER_FILES_PATH}/${CLICKHOUSE_TEST_UNIQUE_NAME}/ for i in {1..10} do - ${CLICKHOUSE_CLIENT} --query "insert into function file('${user_files_path}/${CLICKHOUSE_TEST_UNIQUE_NAME}/test$i.csv', 'CSV', 'k UInt32, v UInt32') select number, number from numbers(10000);" + ${CLICKHOUSE_CLIENT} --query "insert into function file('${USER_FILES_PATH}/${CLICKHOUSE_TEST_UNIQUE_NAME}/test$i.csv', 'CSV', 'k UInt32, v UInt32') select number, number from numbers(10000);" done ${CLICKHOUSE_CLIENT} --query "drop table if exists file_log;" -${CLICKHOUSE_CLIENT} --query "create table file_log(k UInt32, v UInt32) engine=FileLog('${user_files_path}/${CLICKHOUSE_TEST_UNIQUE_NAME}/', 'CSV');" +${CLICKHOUSE_CLIENT} --query "create table file_log(k UInt32, v UInt32) engine=FileLog('${USER_FILES_PATH}/${CLICKHOUSE_TEST_UNIQUE_NAME}/', 'CSV');" ${CLICKHOUSE_CLIENT} --query "select count() from file_log settings stream_like_engine_allow_direct_select=1;" for i in {11..20} do - ${CLICKHOUSE_CLIENT} --query "insert into function file('${user_files_path}/${CLICKHOUSE_TEST_UNIQUE_NAME}/test$i.csv', 'CSV', 'k UInt32, v UInt32') select number, number from numbers(10000);" + ${CLICKHOUSE_CLIENT} --query "insert into function file('${USER_FILES_PATH}/${CLICKHOUSE_TEST_UNIQUE_NAME}/test$i.csv', 'CSV', 'k UInt32, v UInt32') select number, number from numbers(10000);" done ${CLICKHOUSE_CLIENT} --query "select count() from file_log settings stream_like_engine_allow_direct_select=1;" ${CLICKHOUSE_CLIENT} --query "drop table file_log;" -rm -rf ${user_files_path}/${CLICKHOUSE_TEST_UNIQUE_NAME:?} +rm -rf ${USER_FILES_PATH}/${CLICKHOUSE_TEST_UNIQUE_NAME:?} diff --git a/tests/queries/0_stateless/02030_capnp_format.sh b/tests/queries/0_stateless/02030_capnp_format.sh index 6fcfef23cc7..b34bdc9f670 100755 --- a/tests/queries/0_stateless/02030_capnp_format.sh +++ b/tests/queries/0_stateless/02030_capnp_format.sh @@ -1,17 +1,16 @@ #!/usr/bin/env bash -# Tags: no-fasttest, no-parallel, no-replicated-database +# Tags: no-fasttest, no-replicated-database CURDIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd) # shellcheck source=../shell_config.sh . "$CURDIR"/../shell_config.sh -USER_FILES_PATH=$($CLICKHOUSE_CLIENT_BINARY --query "select _path,_file from file('nonexist.txt', 'CSV', 'val1 char')" 2>&1 | grep Exception | awk '{gsub("/nonexist.txt","",$9); print $9}') CAPN_PROTO_FILE=$USER_FILES_PATH/data.capnp touch $CAPN_PROTO_FILE -SCHEMADIR=$($CLICKHOUSE_CLIENT_BINARY --query "select * from file('data.capnp', 'CapnProto', 'val1 char') settings format_schema='nonexist:Message'" 2>&1 | grep Exception | grep -oP "file \K.*(?=/nonexist.capnp)") +SCHEMADIR=${CLICKHOUSE_SCHEMA_FILES} CLIENT_SCHEMADIR=$CURDIR/format_schemas -SERVER_SCHEMADIR=test_02030 +SERVER_SCHEMADIR=${CLICKHOUSE_DATABASE} mkdir -p $SCHEMADIR/$SERVER_SCHEMADIR cp -r $CLIENT_SCHEMADIR/02030_* $SCHEMADIR/$SERVER_SCHEMADIR/ diff --git a/tests/queries/0_stateless/02051_symlinks_to_user_files.sh b/tests/queries/0_stateless/02051_symlinks_to_user_files.sh index eab44e74d88..afdb14c4191 100755 --- a/tests/queries/0_stateless/02051_symlinks_to_user_files.sh +++ b/tests/queries/0_stateless/02051_symlinks_to_user_files.sh @@ -5,10 +5,7 @@ CUR_DIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd) # shellcheck source=../shell_config.sh . "$CUR_DIR"/../shell_config.sh -# See 01658_read_file_to_string_column.sh -user_files_path=$($CLICKHOUSE_CLIENT_BINARY --query "select _path,_file from file('nonexist.txt', 'CSV', 'val1 char')" 2>&1 | grep Exception | awk '{gsub("/nonexist.txt","",$9); print $9}') - -FILE_PATH="${user_files_path}/file" +FILE_PATH="${USER_FILES_PATH}/file" mkdir -p ${FILE_PATH} chmod 777 ${FILE_PATH} diff --git a/tests/queries/0_stateless/02103_tsv_csv_custom_null_representation.sh b/tests/queries/0_stateless/02103_tsv_csv_custom_null_representation.sh index 7ea6739e932..25ad4f37e45 100755 --- a/tests/queries/0_stateless/02103_tsv_csv_custom_null_representation.sh +++ b/tests/queries/0_stateless/02103_tsv_csv_custom_null_representation.sh @@ -5,7 +5,7 @@ CURDIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd) # shellcheck source=../shell_config.sh . "$CURDIR"/../shell_config.sh -DATA_FILE=$CLICKHOUSE_TMP/test_02103_null.data +DATA_FILE=${CLICKHOUSE_TMP}/${CLICKHOUSE_DATABASE}_null.data # Wrapper for clickhouse-client to always output in JSONEachRow format, that # way format settings will not affect output. diff --git a/tests/queries/0_stateless/02103_with_names_and_types_parallel_parsing.sh b/tests/queries/0_stateless/02103_with_names_and_types_parallel_parsing.sh index a6e704093a2..07d40539358 100755 --- a/tests/queries/0_stateless/02103_with_names_and_types_parallel_parsing.sh +++ b/tests/queries/0_stateless/02103_with_names_and_types_parallel_parsing.sh @@ -1,20 +1,17 @@ #!/usr/bin/env bash -# Tags: no-parallel CURDIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd) # shellcheck source=../shell_config.sh . "$CURDIR"/../shell_config.sh -USER_FILES_PATH=$(clickhouse client --query "select _path,_file from file('nonexist.txt', 'CSV', 'val1 char')" 2>&1 | grep Exception | awk '{gsub("/nonexist.txt","",$9); print $9}') - -DATA_FILE=$USER_FILES_PATH/test_02103.data +DATA_FILE=$USER_FILES_PATH/${CLICKHOUSE_DATABASE}.data FORMATS=('TSVWithNames' 'TSVWithNamesAndTypes' 'TSVRawWithNames' 'TSVRawWithNamesAndTypes' 'CSVWithNames' 'CSVWithNamesAndTypes' 'JSONCompactEachRowWithNames' 'JSONCompactEachRowWithNamesAndTypes') for format in "${FORMATS[@]}" do $CLICKHOUSE_CLIENT -q "SELECT number, range(number + 10) AS array, toString(number) AS string FROM numbers(10) FORMAT $format" > $DATA_FILE - $CLICKHOUSE_CLIENT -q "SELECT * FROM file('test_02103.data', '$format', 'number UInt64, array Array(UInt64), string String') ORDER BY number SETTINGS input_format_parallel_parsing=1, min_chunk_bytes_for_parallel_parsing=40" + $CLICKHOUSE_CLIENT -q "SELECT * FROM file('${CLICKHOUSE_DATABASE}.data', '$format', 'number UInt64, array Array(UInt64), string String') ORDER BY number SETTINGS input_format_parallel_parsing=1, min_chunk_bytes_for_parallel_parsing=40" done rm $DATA_FILE diff --git a/tests/queries/0_stateless/02104_json_strings_nullable_string.sh b/tests/queries/0_stateless/02104_json_strings_nullable_string.sh index b3b156b5787..d46b0704b0e 100755 --- a/tests/queries/0_stateless/02104_json_strings_nullable_string.sh +++ b/tests/queries/0_stateless/02104_json_strings_nullable_string.sh @@ -5,7 +5,6 @@ CURDIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd) # shellcheck source=../shell_config.sh . "$CURDIR"/../shell_config.sh -USER_FILES_PATH=$($CLICKHOUSE_CLIENT_BINARY --query "select _path,_file from file('nonexist.txt', 'CSV', 'val1 char')" 2>&1 | grep Exception | awk '{gsub("/nonexist.txt","",$9); print $9}') DATA_FILE=$USER_FILES_PATH/test_02104_null.data echo -e '{"s" : "NULLSome string"}' > $DATA_FILE diff --git a/tests/queries/0_stateless/02105_table_function_file_partiotion_by.sh b/tests/queries/0_stateless/02105_table_function_file_partiotion_by.sh index c79b5d0eee5..38c68ca0005 100755 --- a/tests/queries/0_stateless/02105_table_function_file_partiotion_by.sh +++ b/tests/queries/0_stateless/02105_table_function_file_partiotion_by.sh @@ -5,13 +5,7 @@ CUR_DIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd) # shellcheck source=../shell_config.sh . "$CUR_DIR"/../shell_config.sh -# See 01658_read_file_to_string_column.sh -user_files_path=$($CLICKHOUSE_CLIENT_BINARY --query "select _path,_file from file('nonexist.txt', 'CSV', 'val1 char')" 2>&1 | grep Exception | awk '{gsub("/nonexist.txt","",$9); print $9}') - -mkdir -p "${user_files_path}/" -chmod 777 ${user_files_path} - -FILE_PATH="${user_files_path}/test_table_function_file" +FILE_PATH="${USER_FILES_PATH}/test_table_function_file" function cleanup() { diff --git a/tests/queries/0_stateless/02115_write_buffers_finalize.sh b/tests/queries/0_stateless/02115_write_buffers_finalize.sh index d8a3c29bbbd..fb3a77f6105 100755 --- a/tests/queries/0_stateless/02115_write_buffers_finalize.sh +++ b/tests/queries/0_stateless/02115_write_buffers_finalize.sh @@ -1,5 +1,5 @@ #!/usr/bin/env bash -# Tags: no-fasttest, no-parallel +# Tags: no-fasttest # Tag no-fasttest: depends on brotli and bzip2 CURDIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd) diff --git a/tests/queries/0_stateless/02117_custom_separated_with_names_and_types.sh b/tests/queries/0_stateless/02117_custom_separated_with_names_and_types.sh index 400bf2a56fa..d925c962da4 100755 --- a/tests/queries/0_stateless/02117_custom_separated_with_names_and_types.sh +++ b/tests/queries/0_stateless/02117_custom_separated_with_names_and_types.sh @@ -1,5 +1,4 @@ #!/usr/bin/env bash -# Tags: no-parallel CURDIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd) # shellcheck source=../shell_config.sh diff --git a/tests/queries/0_stateless/02118_deserialize_whole_text.sh b/tests/queries/0_stateless/02118_deserialize_whole_text.sh index ccbfc5abe97..d544f1452a9 100755 --- a/tests/queries/0_stateless/02118_deserialize_whole_text.sh +++ b/tests/queries/0_stateless/02118_deserialize_whole_text.sh @@ -1,65 +1,65 @@ #!/usr/bin/env bash -# Tags: no-fasttest, no-parallel, no-replicated-database +# Tags: no-fasttest, no-replicated-database CURDIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd) # shellcheck source=../shell_config.sh . "$CURDIR"/../shell_config.sh -USER_FILES_PATH=$($CLICKHOUSE_CLIENT_BINARY --query "select _path,_file from file('nonexist.txt', 'CSV', 'val1 char')" 2>&1 | grep Exception | awk '{gsub("/nonexist.txt","",$9); print $9}') -DATA_FILE=$USER_FILES_PATH/data_02118 +DATA_FILE=$USER_FILES_PATH/${CLICKHOUSE_DATABASE} +FILE=${CLICKHOUSE_DATABASE} echo "[\"[1,2,3]trash\"]" > $DATA_FILE -$CLICKHOUSE_CLIENT -q "SELECT * FROM file('data_02118', 'JSONCompactStringsEachRow', 'x Array(UInt32)')" 2>&1 | grep -F -q "UNEXPECTED_DATA_AFTER_PARSED_VALUE" && echo 'OK' || echo 'FAIL' +$CLICKHOUSE_CLIENT -q "SELECT * FROM file('${FILE}', 'JSONCompactStringsEachRow', 'x Array(UInt32)')" 2>&1 | grep -F -q "UNEXPECTED_DATA_AFTER_PARSED_VALUE" && echo 'OK' || echo 'FAIL' echo "[\"1970-01-02trash\"]" > $DATA_FILE -$CLICKHOUSE_CLIENT -q "SELECT * FROM file('data_02118', 'JSONCompactStringsEachRow', 'x Date')" 2>&1 | grep -F -q "UNEXPECTED_DATA_AFTER_PARSED_VALUE" && echo 'OK' || echo 'FAIL' +$CLICKHOUSE_CLIENT -q "SELECT * FROM file('${FILE}', 'JSONCompactStringsEachRow', 'x Date')" 2>&1 | grep -F -q "UNEXPECTED_DATA_AFTER_PARSED_VALUE" && echo 'OK' || echo 'FAIL' echo "[\"1970-01-02trash\"]" > $DATA_FILE -$CLICKHOUSE_CLIENT -q "SELECT * FROM file('data_02118', 'JSONCompactStringsEachRow', 'x Date32')" 2>&1 | grep -F -q "UNEXPECTED_DATA_AFTER_PARSED_VALUE" && echo 'OK' || echo 'FAIL' +$CLICKHOUSE_CLIENT -q "SELECT * FROM file('${FILE}', 'JSONCompactStringsEachRow', 'x Date32')" 2>&1 | grep -F -q "UNEXPECTED_DATA_AFTER_PARSED_VALUE" && echo 'OK' || echo 'FAIL' echo "[\"1970-01-01 03:00:01trash\"]" > $DATA_FILE -$CLICKHOUSE_CLIENT -q "SELECT * FROM file('data_02118', 'JSONCompactStringsEachRow', 'x DateTime')" 2>&1 | grep -F -q "UNEXPECTED_DATA_AFTER_PARSED_VALUE" && echo 'OK' || echo 'FAIL' +$CLICKHOUSE_CLIENT -q "SELECT * FROM file('${FILE}', 'JSONCompactStringsEachRow', 'x DateTime')" 2>&1 | grep -F -q "UNEXPECTED_DATA_AFTER_PARSED_VALUE" && echo 'OK' || echo 'FAIL' echo "[\"1970-01-01 03:00:01.0000trash\"]" > $DATA_FILE -$CLICKHOUSE_CLIENT -q "SELECT * FROM file('data_02118', 'JSONCompactStringsEachRow', 'x DateTime64(4)')" 2>&1 | grep -F -q "UNEXPECTED_DATA_AFTER_PARSED_VALUE" && echo 'OK' || echo 'FAIL' +$CLICKHOUSE_CLIENT -q "SELECT * FROM file('${FILE}', 'JSONCompactStringsEachRow', 'x DateTime64(4)')" 2>&1 | grep -F -q "UNEXPECTED_DATA_AFTER_PARSED_VALUE" && echo 'OK' || echo 'FAIL' echo "[\"42trash\"]" > $DATA_FILE -$CLICKHOUSE_CLIENT -q "SELECT * FROM file('data_02118', 'JSONCompactStringsEachRow', 'x UInt32')" 2>&1 | grep -F -q "UNEXPECTED_DATA_AFTER_PARSED_VALUE" && echo 'OK' || echo 'FAIL' +$CLICKHOUSE_CLIENT -q "SELECT * FROM file('${FILE}', 'JSONCompactStringsEachRow', 'x UInt32')" 2>&1 | grep -F -q "UNEXPECTED_DATA_AFTER_PARSED_VALUE" && echo 'OK' || echo 'FAIL' echo "[\"42.4242trash\"]" > $DATA_FILE -$CLICKHOUSE_CLIENT -q "SELECT * FROM file('data_02118', 'JSONCompactStringsEachRow', 'x Decimal32(4)')" 2>&1 | grep -F -q "UNEXPECTED_DATA_AFTER_PARSED_VALUE" && echo 'OK' || echo 'FAIL' +$CLICKHOUSE_CLIENT -q "SELECT * FROM file('${FILE}', 'JSONCompactStringsEachRow', 'x Decimal32(4)')" 2>&1 | grep -F -q "UNEXPECTED_DATA_AFTER_PARSED_VALUE" && echo 'OK' || echo 'FAIL' echo "[\"255.255.255.255trash\"]" > $DATA_FILE -$CLICKHOUSE_CLIENT -q "SELECT * FROM file('data_02118', 'JSONCompactStringsEachRow', 'x IPv4')" 2>&1 | grep -F -q "UNEXPECTED_DATA_AFTER_PARSED_VALUE" && echo 'OK' || echo 'FAIL' +$CLICKHOUSE_CLIENT -q "SELECT * FROM file('${FILE}', 'JSONCompactStringsEachRow', 'x IPv4')" 2>&1 | grep -F -q "UNEXPECTED_DATA_AFTER_PARSED_VALUE" && echo 'OK' || echo 'FAIL' echo "255.255.255.255trash" > $DATA_FILE -$CLICKHOUSE_CLIENT -q "SELECT * FROM file('data_02118', 'TSV', 'x IPv4')" 2>&1 | grep -F -q "CANNOT_PARSE_INPUT_ASSERTION_FAILED" && echo 'OK' || echo 'FAIL' +$CLICKHOUSE_CLIENT -q "SELECT * FROM file('${FILE}', 'TSV', 'x IPv4')" 2>&1 | grep -F -q "CANNOT_PARSE_INPUT_ASSERTION_FAILED" && echo 'OK' || echo 'FAIL' echo "255.255.255.255trash" > $DATA_FILE -$CLICKHOUSE_CLIENT -q "SELECT * FROM file('data_02118', 'CSV', 'x IPv4')" 2>&1 | grep -F -q "INCORRECT_DATA" && echo 'OK' || echo 'FAIL' +$CLICKHOUSE_CLIENT -q "SELECT * FROM file('${FILE}', 'CSV', 'x IPv4')" 2>&1 | grep -F -q "INCORRECT_DATA" && echo 'OK' || echo 'FAIL' echo "[\"255.255.255.255trash\"]" > $DATA_FILE -$CLICKHOUSE_CLIENT -q "SELECT * FROM file('data_02118', 'JSONCompactEachRow', 'x IPv4')" 2>&1 | grep -F -q "UNEXPECTED_DATA_AFTER_PARSED_VALUE" && echo 'OK' || echo 'FAIL' +$CLICKHOUSE_CLIENT -q "SELECT * FROM file('${FILE}', 'JSONCompactEachRow', 'x IPv4')" 2>&1 | grep -F -q "UNEXPECTED_DATA_AFTER_PARSED_VALUE" && echo 'OK' || echo 'FAIL' echo "[\"0000:0000:0000:0000:0000:ffff:192.168.100.228b1trash\"]" > $DATA_FILE -$CLICKHOUSE_CLIENT -q "SELECT * FROM file('data_02118', 'JSONCompactStringsEachRow', 'x IPv6')" 2>&1 | grep -F -q "UNEXPECTED_DATA_AFTER_PARSED_VALUE" && echo 'OK' || echo 'FAIL' +$CLICKHOUSE_CLIENT -q "SELECT * FROM file('${FILE}', 'JSONCompactStringsEachRow', 'x IPv6')" 2>&1 | grep -F -q "UNEXPECTED_DATA_AFTER_PARSED_VALUE" && echo 'OK' || echo 'FAIL' echo "0000:0000:0000:0000:0000:ffff:192.168.100.228b1trash" > $DATA_FILE -$CLICKHOUSE_CLIENT -q "SELECT * FROM file('data_02118', 'TSV', 'x IPv6')" 2>&1 | grep -F -q "CANNOT_PARSE_INPUT_ASSERTION_FAILED" && echo 'OK' || echo 'FAIL' +$CLICKHOUSE_CLIENT -q "SELECT * FROM file('${FILE}', 'TSV', 'x IPv6')" 2>&1 | grep -F -q "CANNOT_PARSE_INPUT_ASSERTION_FAILED" && echo 'OK' || echo 'FAIL' echo "0000:0000:0000:0000:0000:ffff:192.168.100.228b1trash" > $DATA_FILE -$CLICKHOUSE_CLIENT -q "SELECT * FROM file('data_02118', 'CSV', 'x IPv6')" 2>&1 | grep -F -q "INCORRECT_DATA" && echo 'OK' || echo 'FAIL' +$CLICKHOUSE_CLIENT -q "SELECT * FROM file('${FILE}', 'CSV', 'x IPv6')" 2>&1 | grep -F -q "INCORRECT_DATA" && echo 'OK' || echo 'FAIL' echo "[\"0000:0000:0000:0000:0000:ffff:192.168.100.228b1trash\"]" > $DATA_FILE -$CLICKHOUSE_CLIENT -q "SELECT * FROM file('data_02118', 'JSONCompactEachRow', 'x IPv6')" 2>&1 | grep -F -q "UNEXPECTED_DATA_AFTER_PARSED_VALUE" && echo 'OK' || echo 'FAIL' +$CLICKHOUSE_CLIENT -q "SELECT * FROM file('${FILE}', 'JSONCompactEachRow', 'x IPv6')" 2>&1 | grep -F -q "UNEXPECTED_DATA_AFTER_PARSED_VALUE" && echo 'OK' || echo 'FAIL' echo "[\"{1:2, 2:3}trash\"]" > $DATA_FILE -$CLICKHOUSE_CLIENT -q "SELECT * FROM file('data_02118', 'JSONCompactStringsEachRow', 'x Map(UInt32, UInt32)')" 2>&1 | grep -F -q "UNEXPECTED_DATA_AFTER_PARSED_VALUE" && echo 'OK' || echo 'FAIL' +$CLICKHOUSE_CLIENT -q "SELECT * FROM file('${FILE}', 'JSONCompactStringsEachRow', 'x Map(UInt32, UInt32)')" 2>&1 | grep -F -q "UNEXPECTED_DATA_AFTER_PARSED_VALUE" && echo 'OK' || echo 'FAIL' echo "[\"(1, 2)trash\"]" > $DATA_FILE -$CLICKHOUSE_CLIENT -q "SELECT * FROM file('data_02118', 'JSONCompactStringsEachRow', 'x Tuple(UInt32, UInt32)')" 2>&1 | grep -F -q "UNEXPECTED_DATA_AFTER_PARSED_VALUE" && echo 'OK' || echo 'FAIL' +$CLICKHOUSE_CLIENT -q "SELECT * FROM file('${FILE}', 'JSONCompactStringsEachRow', 'x Tuple(UInt32, UInt32)')" 2>&1 | grep -F -q "UNEXPECTED_DATA_AFTER_PARSED_VALUE" && echo 'OK' || echo 'FAIL' echo "[\"ed9fd45d-6287-47c1-ad9f-d45d628767c1trash\"]" > $DATA_FILE -$CLICKHOUSE_CLIENT -q "SELECT * FROM file('data_02118', 'JSONCompactStringsEachRow', 'x UUID')" 2>&1 | grep -F -q "UNEXPECTED_DATA_AFTER_PARSED_VALUE" && echo 'OK' || echo 'FAIL' +$CLICKHOUSE_CLIENT -q "SELECT * FROM file('${FILE}', 'JSONCompactStringsEachRow', 'x UUID')" 2>&1 | grep -F -q "UNEXPECTED_DATA_AFTER_PARSED_VALUE" && echo 'OK' || echo 'FAIL' rm $DATA_FILE diff --git a/tests/queries/0_stateless/02125_tskv_proper_names_reading.sh b/tests/queries/0_stateless/02125_tskv_proper_names_reading.sh index 0abf411d38f..4a169897520 100755 --- a/tests/queries/0_stateless/02125_tskv_proper_names_reading.sh +++ b/tests/queries/0_stateless/02125_tskv_proper_names_reading.sh @@ -5,8 +5,6 @@ CURDIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd) # shellcheck source=../shell_config.sh . "$CURDIR"/../shell_config.sh -USER_FILES_PATH=$($CLICKHOUSE_CLIENT_BINARY --query "select _path,_file from file('nonexist.txt', 'CSV', 'val1 char')" 2>&1 | grep Exception | awk '{gsub("/nonexist.txt","",$9); print $9}') - DATA_FILE=$USER_FILES_PATH/test_02125.data echo "number=1" > $DATA_FILE diff --git a/tests/queries/0_stateless/02126_fix_filelog.sh b/tests/queries/0_stateless/02126_fix_filelog.sh index b266b582428..0e136a34c62 100755 --- a/tests/queries/0_stateless/02126_fix_filelog.sh +++ b/tests/queries/0_stateless/02126_fix_filelog.sh @@ -6,14 +6,8 @@ CURDIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd) # shellcheck source=../shell_config.sh . "$CURDIR"/../shell_config.sh -# Data preparation. -# Now we can get the user_files_path by use the table file function for trick. also we can get it by query as: -# "insert into function file('exist.txt', 'CSV', 'val1 char') values ('aaaa'); select _path from file('exist.txt', 'CSV', 'val1 char')" -user_files_path=$($CLICKHOUSE_CLIENT_BINARY --query "select _path,_file from file('nonexist.txt', 'CSV', 'val1 char')" 2>&1 | grep Exception | awk '{gsub("/nonexist.txt","",$9); print $9}') - -mkdir -p ${user_files_path}/${CLICKHOUSE_TEST_UNIQUE_NAME}/ - -rm -rf ${user_files_path}/${CLICKHOUSE_TEST_UNIQUE_NAME:?}/* +mkdir -p ${USER_FILES_PATH}/${CLICKHOUSE_TEST_UNIQUE_NAME}/ +rm -rf ${USER_FILES_PATH}/${CLICKHOUSE_TEST_UNIQUE_NAME:?}/* ${CLICKHOUSE_CLIENT} --query "drop table if exists file_log;" @@ -21,8 +15,8 @@ ${CLICKHOUSE_CLIENT} --query "create table file_log(k UInt8, v UInt8) engine=Fil ${CLICKHOUSE_CLIENT} --query "create table file_log(k UInt8, v UInt8) engine=FileLog('/tmp/aaa.csv', 'CSV');" 2>&1 | grep -q "Code: 36" && echo 'OK' || echo 'FAIL'; ${CLICKHOUSE_CLIENT} --query "create table file_log(k UInt8, v UInt8) engine=FileLog('/tmp/aaa.csv', 'CSV');" 2>&1 | grep -q "Code: 36" && echo 'OK' || echo 'FAIL'; -${CLICKHOUSE_CLIENT} --query "create table file_log(k UInt8, v UInt8) engine=FileLog('${user_files_path}/${CLICKHOUSE_TEST_UNIQUE_NAME}/', 'CSV');" +${CLICKHOUSE_CLIENT} --query "create table file_log(k UInt8, v UInt8) engine=FileLog('${USER_FILES_PATH}/${CLICKHOUSE_TEST_UNIQUE_NAME}/', 'CSV');" ${CLICKHOUSE_CLIENT} --query "drop table file_log;" -rm -rf ${user_files_path}/${CLICKHOUSE_TEST_UNIQUE_NAME:?} +rm -rf ${USER_FILES_PATH}/${CLICKHOUSE_TEST_UNIQUE_NAME:?} diff --git a/tests/queries/0_stateless/02129_skip_quoted_fields.sh b/tests/queries/0_stateless/02129_skip_quoted_fields.sh index ac702d3c750..701d7a30b68 100755 --- a/tests/queries/0_stateless/02129_skip_quoted_fields.sh +++ b/tests/queries/0_stateless/02129_skip_quoted_fields.sh @@ -1,5 +1,4 @@ #!/usr/bin/env bash -# Tags: no-parallel CURDIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd) # shellcheck source=../shell_config.sh diff --git a/tests/queries/0_stateless/02130_parse_quoted_null.sh b/tests/queries/0_stateless/02130_parse_quoted_null.sh index 0c72d0e85a7..44e6ee93599 100755 --- a/tests/queries/0_stateless/02130_parse_quoted_null.sh +++ b/tests/queries/0_stateless/02130_parse_quoted_null.sh @@ -1,14 +1,12 @@ #!/usr/bin/env bash -# Tags: no-parallel CURDIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd) # shellcheck source=../shell_config.sh . "$CURDIR"/../shell_config.sh -USER_FILES_PATH=$($CLICKHOUSE_CLIENT_BINARY --query "select _path,_file from file('nonexist.txt', 'CSV', 'val1 char')" 2>&1 | grep Exception | awk '{gsub("/nonexist.txt","",$9); print $9}') -DATA_FILE=$USER_FILES_PATH/test_02130.data -SELECT_QUERY="select * from file('test_02130.data', 'CustomSeparated', 'x Nullable(Float64), y Nullable(UInt64)') settings input_format_parallel_parsing=0, format_custom_escaping_rule='Quoted'" +DATA_FILE=$USER_FILES_PATH/${CLICKHOUSE_DATABASE}.data +SELECT_QUERY="select * from file('${CLICKHOUSE_DATABASE}.data', 'CustomSeparated', 'x Nullable(Float64), y Nullable(UInt64)') settings input_format_parallel_parsing=0, format_custom_escaping_rule='Quoted'" $CLICKHOUSE_CLIENT -q "drop table if exists test_02130" diff --git a/tests/queries/0_stateless/02149_external_schema_inference.sh b/tests/queries/0_stateless/02149_external_schema_inference.sh index 41f8bfee2bc..edb4e915701 100755 --- a/tests/queries/0_stateless/02149_external_schema_inference.sh +++ b/tests/queries/0_stateless/02149_external_schema_inference.sh @@ -1,20 +1,19 @@ #!/usr/bin/env bash -# Tags: no-parallel, no-fasttest +# Tags: no-fasttest CURDIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd) # shellcheck source=../shell_config.sh . "$CURDIR"/../shell_config.sh -USER_FILES_PATH=$($CLICKHOUSE_CLIENT_BINARY --query "select _path,_file from file('nonexist.txt', 'CSV', 'val1 char')" 2>&1 | grep Exception | awk '{gsub("/nonexist.txt","",$9); print $9}') -FILE_NAME=test_$CLICKHOUSE_TEST_UNIQUE_NAME.data +FILE_NAME=test_${CLICKHOUSE_TEST_UNIQUE_NAME}_${CLICKHOUSE_DATABASE}.data DATA_FILE=$USER_FILES_PATH/$FILE_NAME touch $DATA_FILE -SCHEMADIR=$($CLICKHOUSE_CLIENT_BINARY --query "select * from file('$FILE_NAME', 'CapnProto', 'val1 char') settings format_schema='nonexist:Message'" 2>&1 | grep Exception | grep -oP "file \K.*(?=/nonexist.capnp)") +SCHEMADIR=${CLICKHOUSE_SCHEMA_FILES} CLIENT_SCHEMADIR=$CURDIR/format_schemas -SERVER_SCHEMADIR=test_02149 +SERVER_SCHEMADIR=${CLICKHOUSE_DATABASE} mkdir -p $SCHEMADIR/$SERVER_SCHEMADIR cp -r $CLIENT_SCHEMADIR/* $SCHEMADIR/$SERVER_SCHEMADIR/ diff --git a/tests/queries/0_stateless/02149_schema_inference.sh b/tests/queries/0_stateless/02149_schema_inference.sh index 856549f2215..fba1e6e9137 100755 --- a/tests/queries/0_stateless/02149_schema_inference.sh +++ b/tests/queries/0_stateless/02149_schema_inference.sh @@ -1,18 +1,16 @@ #!/usr/bin/env bash -# Tags: no-parallel, no-fasttest +# Tags: no-fasttest CURDIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd) # shellcheck source=../shell_config.sh . "$CURDIR"/../shell_config.sh -USER_FILES_PATH=$($CLICKHOUSE_CLIENT_BINARY --query "select _path,_file from file('nonexist.txt', 'CSV', 'val1 char')" 2>&1 | grep Exception | awk '{gsub("/nonexist.txt","",$9); print $9}') FILE_NAME=test_$CLICKHOUSE_TEST_UNIQUE_NAME.data -DATA_FILE=${USER_FILES_PATH:?}/$FILE_NAME - +DATA_FILE=${CLICKHOUSE_USER_FILES:?}/$FILE_NAME touch $DATA_FILE -SCHEMADIR=$($CLICKHOUSE_CLIENT_BINARY --query "select * from file('$FILE_NAME', 'Template', 'val1 char') settings format_template_row='nonexist'" 2>&1 | grep Exception | grep -oP "file \K.*(?=/nonexist)") +SCHEMADIR=${CLICKHOUSE_SCHEMA_FILES} echo "TSV" diff --git a/tests/queries/0_stateless/02149_schema_inference_create_table_syntax.sh b/tests/queries/0_stateless/02149_schema_inference_create_table_syntax.sh index 8de2ab8c57a..bf247817323 100755 --- a/tests/queries/0_stateless/02149_schema_inference_create_table_syntax.sh +++ b/tests/queries/0_stateless/02149_schema_inference_create_table_syntax.sh @@ -1,14 +1,13 @@ #!/usr/bin/env bash -# Tags: no-parallel, no-fasttest +# Tags: no-fasttest CURDIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd) # shellcheck source=../shell_config.sh . "$CURDIR"/../shell_config.sh -USER_FILES_PATH=$($CLICKHOUSE_CLIENT_BINARY --query "select _path,_file from file('nonexist.txt', 'CSV', 'val1 char')" 2>&1 | grep Exception | awk '{gsub("/nonexist.txt","",$9); print $9}') -mkdir $USER_FILES_PATH/test_02149 -FILE_NAME=test_02149/data.Parquet +mkdir $USER_FILES_PATH/${CLICKHOUSE_DATABASE}/ +FILE_NAME=data.Parquet DATA_FILE=$USER_FILES_PATH/$FILE_NAME $CLICKHOUSE_CLIENT -q "select number as num, concat('Str: ', toString(number)) as str, [number, number + 1] as arr from numbers(10) format Parquet" > $DATA_FILE diff --git a/tests/queries/0_stateless/02167_format_from_file_extension.sh b/tests/queries/0_stateless/02167_format_from_file_extension.sh index 14985233524..0a0efff3228 100755 --- a/tests/queries/0_stateless/02167_format_from_file_extension.sh +++ b/tests/queries/0_stateless/02167_format_from_file_extension.sh @@ -1,5 +1,5 @@ #!/usr/bin/env bash -# Tags: no-parallel, no-fasttest +# Tags: no-fasttest CURDIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd) # shellcheck source=../shell_config.sh @@ -7,28 +7,26 @@ CURDIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd) function cleanup() { - # this command expects an error message like 'Code: 107. DB::Exception: Received <...> nonexist.txt doesn't exist. (FILE_DOESNT_EXIST)' - user_files_path=$($CLICKHOUSE_CLIENT --query "select _path,_file from file('nonexist.txt', 'CSV', 'val1 char')" 2>&1 | grep -E '^Code: 107.*FILE_DOESNT_EXIST' | head -1 | awk '{gsub("/nonexist.txt","",$9); print $9}') - rm $user_files_path/test_02167.* + rm ${USER_FILES_PATH}/${CLICKHOUSE_DATABASE}_test_02167.* } trap cleanup EXIT for format in TSV TabSeparated TSVWithNames TSVWithNamesAndTypes CSV Parquet ORC Arrow JSONEachRow JSONCompactEachRow CustomSeparatedWithNamesAndTypes do - $CLICKHOUSE_CLIENT -q "insert into table function file('test_02167.$format', 'auto', 'x UInt64') select * from numbers(2)" - $CLICKHOUSE_CLIENT -q "select * from file('test_02167.$format')" - $CLICKHOUSE_CLIENT -q "select * from file('test_02167.$format', '$format')" + $CLICKHOUSE_CLIENT -q "insert into table function file('${CLICKHOUSE_DATABASE}_test_02167.$format', 'auto', 'x UInt64') select * from numbers(2)" + $CLICKHOUSE_CLIENT -q "select * from file('${CLICKHOUSE_DATABASE}_test_02167.$format')" + $CLICKHOUSE_CLIENT -q "select * from file('${CLICKHOUSE_DATABASE}_test_02167.$format', '$format')" done -$CLICKHOUSE_CLIENT -q "insert into table function file('test_02167.bin', 'auto', 'x UInt64') select * from numbers(2)" -$CLICKHOUSE_CLIENT -q "select * from file('test_02167.bin', 'auto', 'x UInt64')" -$CLICKHOUSE_CLIENT -q "select * from file('test_02167.bin', 'RowBinary', 'x UInt64')" +$CLICKHOUSE_CLIENT -q "insert into table function file('${CLICKHOUSE_DATABASE}_test_02167.bin', 'auto', 'x UInt64') select * from numbers(2)" +$CLICKHOUSE_CLIENT -q "select * from file('${CLICKHOUSE_DATABASE}_test_02167.bin', 'auto', 'x UInt64')" +$CLICKHOUSE_CLIENT -q "select * from file('${CLICKHOUSE_DATABASE}_test_02167.bin', 'RowBinary', 'x UInt64')" -$CLICKHOUSE_CLIENT -q "insert into table function file('test_02167.ndjson', 'auto', 'x UInt64') select * from numbers(2)" -$CLICKHOUSE_CLIENT -q "select * from file('test_02167.ndjson')" -$CLICKHOUSE_CLIENT -q "select * from file('test_02167.ndjson', 'JSONEachRow', 'x UInt64')" +$CLICKHOUSE_CLIENT -q "insert into table function file('${CLICKHOUSE_DATABASE}_test_02167.ndjson', 'auto', 'x UInt64') select * from numbers(2)" +$CLICKHOUSE_CLIENT -q "select * from file('${CLICKHOUSE_DATABASE}_test_02167.ndjson')" +$CLICKHOUSE_CLIENT -q "select * from file('${CLICKHOUSE_DATABASE}_test_02167.ndjson', 'JSONEachRow', 'x UInt64')" -$CLICKHOUSE_CLIENT -q "insert into table function file('test_02167.messagepack', 'auto', 'x UInt64') select * from numbers(2)" -$CLICKHOUSE_CLIENT -q "select * from file('test_02167.messagepack') settings input_format_msgpack_number_of_columns=1" -$CLICKHOUSE_CLIENT -q "select * from file('test_02167.messagepack', 'MsgPack', 'x UInt64')" +$CLICKHOUSE_CLIENT -q "insert into table function file('${CLICKHOUSE_DATABASE}_test_02167.messagepack', 'auto', 'x UInt64') select * from numbers(2)" +$CLICKHOUSE_CLIENT -q "select * from file('${CLICKHOUSE_DATABASE}_test_02167.messagepack') settings input_format_msgpack_number_of_columns=1" +$CLICKHOUSE_CLIENT -q "select * from file('${CLICKHOUSE_DATABASE}_test_02167.messagepack', 'MsgPack', 'x UInt64')" diff --git a/tests/queries/0_stateless/02185_orc_corrupted_file.sh b/tests/queries/0_stateless/02185_orc_corrupted_file.sh index 12510ae3836..8cf4334845d 100755 --- a/tests/queries/0_stateless/02185_orc_corrupted_file.sh +++ b/tests/queries/0_stateless/02185_orc_corrupted_file.sh @@ -5,7 +5,6 @@ CUR_DIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd) # shellcheck source=../shell_config.sh . "$CUR_DIR"/../shell_config.sh -USER_FILES_PATH=$($CLICKHOUSE_CLIENT_BINARY --query "select _path,_file from file('nonexist.txt', 'CSV', 'val1 char')" 2>&1 | grep Exception | awk '{gsub("/nonexist.txt","",$9); print $9}') cp $CUR_DIR/data_orc/corrupted.orc $USER_FILES_PATH/ ${CLICKHOUSE_CLIENT} --query="select * from file('corrupted.orc')" 2>&1 | grep -F -q 'CANNOT_EXTRACT_TABLE_STRUCTURE' && echo 'OK' || echo 'FAIL' diff --git a/tests/queries/0_stateless/02207_allow_plaintext_and_no_password.sh b/tests/queries/0_stateless/02207_allow_plaintext_and_no_password.sh index 0345a0e6394..dc3cb0de110 100755 --- a/tests/queries/0_stateless/02207_allow_plaintext_and_no_password.sh +++ b/tests/queries/0_stateless/02207_allow_plaintext_and_no_password.sh @@ -5,7 +5,7 @@ CURDIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd) # shellcheck source=../shell_config.sh . "$CURDIR"/../shell_config.sh -cp /etc/clickhouse-server/users.xml "$CURDIR"/users.xml +cp ${CLICKHOUSE_CONFIG_DIR}/users.xml "$CURDIR"/users.xml sed -i 's/<\/password>/c64c5e4e53ea1a9f1427d2713b3a22bbebe8940bc807adaf654744b1568c70ab<\/password_sha256_hex>/g' "$CURDIR"/users.xml sed -i 's//1<\/access_management>/g' "$CURDIR"/users.xml diff --git a/tests/queries/0_stateless/02222_create_table_without_columns_metadata.sh b/tests/queries/0_stateless/02222_create_table_without_columns_metadata.sh index d49c3610852..73d1c2b9b42 100755 --- a/tests/queries/0_stateless/02222_create_table_without_columns_metadata.sh +++ b/tests/queries/0_stateless/02222_create_table_without_columns_metadata.sh @@ -5,8 +5,6 @@ CURDIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd) # shellcheck source=../shell_config.sh . "$CURDIR"/../shell_config.sh -USER_FILES_PATH=$($CLICKHOUSE_CLIENT_BINARY --query "select _path,_file from file('nonexist.txt', 'CSV', 'val1 char')" 2>&1 | grep Exception | awk '{gsub("/nonexist.txt","",$9); print $9}') - $CLICKHOUSE_CLIENT -q "insert into table function file(data.jsonl, 'JSONEachRow', 'x UInt32 default 42, y String') select number as x, 'String' as y from numbers(10)" $CLICKHOUSE_CLIENT -q "drop table if exists test" diff --git a/tests/queries/0_stateless/02227_test_create_empty_sqlite_db.sh b/tests/queries/0_stateless/02227_test_create_empty_sqlite_db.sh index 37fdde95ea7..344452767cc 100755 --- a/tests/queries/0_stateless/02227_test_create_empty_sqlite_db.sh +++ b/tests/queries/0_stateless/02227_test_create_empty_sqlite_db.sh @@ -6,9 +6,6 @@ CUR_DIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd) # shellcheck source=../shell_config.sh . "$CUR_DIR"/../shell_config.sh -# See 01658_read_file_to_string_column.sh -user_files_path=$($CLICKHOUSE_CLIENT_BINARY --query "select _path,_file from file('nonexist.txt', 'CSV', 'val1 char')" 2>&1 | grep Exception | awk '{gsub("/nonexist.txt","",$9); print $9}') - function cleanup() { ${CLICKHOUSE_CLIENT} --query="DROP DATABASE IF EXISTS ${CURR_DATABASE}" @@ -18,7 +15,7 @@ trap cleanup EXIT export CURR_DATABASE="test_01889_sqllite_${CLICKHOUSE_DATABASE}" -DB_PATH=${user_files_path}/${CURR_DATABASE}_db1 +DB_PATH=${USER_FILES_PATH}/${CURR_DATABASE}_db1 ${CLICKHOUSE_CLIENT} --multiquery --multiline --query=""" DROP DATABASE IF EXISTS ${CURR_DATABASE}; diff --git a/tests/queries/0_stateless/02228_merge_tree_insert_memory_usage.sql b/tests/queries/0_stateless/02228_merge_tree_insert_memory_usage.sql index 6d86d995143..72b6cc06f26 100644 --- a/tests/queries/0_stateless/02228_merge_tree_insert_memory_usage.sql +++ b/tests/queries/0_stateless/02228_merge_tree_insert_memory_usage.sql @@ -1,4 +1,4 @@ --- Tags: long, no-parallel, no-object-storage +-- Tags: long, no-object-storage -- no-object-storage: Avoid flakiness due to cache / buffer usage SET insert_keeper_fault_injection_probability=0; -- to succeed this test can require too many retries due to 100 partitions, so disable fault injections diff --git a/tests/queries/0_stateless/02242_arrow_orc_parquet_nullable_schema_inference.sh b/tests/queries/0_stateless/02242_arrow_orc_parquet_nullable_schema_inference.sh index e03c62cfc5f..0f37bff45d6 100755 --- a/tests/queries/0_stateless/02242_arrow_orc_parquet_nullable_schema_inference.sh +++ b/tests/queries/0_stateless/02242_arrow_orc_parquet_nullable_schema_inference.sh @@ -6,7 +6,6 @@ CURDIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd) . "$CURDIR"/../shell_config.sh -USER_FILES_PATH=$($CLICKHOUSE_CLIENT_BINARY --query "select _path,_file from file('nonexist.txt', 'CSV', 'val1 char')" 2>&1 | grep Exception | awk '{gsub("/nonexist.txt","",$9); print $9}') FILE_NAME=test_02242.data DATA_FILE=$USER_FILES_PATH/$FILE_NAME diff --git a/tests/queries/0_stateless/02245_parquet_skip_unknown_type.sh b/tests/queries/0_stateless/02245_parquet_skip_unknown_type.sh index 8ff6e28b123..7bfeb747cc2 100755 --- a/tests/queries/0_stateless/02245_parquet_skip_unknown_type.sh +++ b/tests/queries/0_stateless/02245_parquet_skip_unknown_type.sh @@ -5,7 +5,6 @@ CUR_DIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd) # shellcheck source=../shell_config.sh . "$CUR_DIR"/../shell_config.sh -USER_FILES_PATH=$($CLICKHOUSE_CLIENT_BINARY --query "select _path,_file from file('nonexist.txt', 'CSV', 'val1 char')" 2>&1 | grep Exception | awk '{gsub("/nonexist.txt","",$9); print $9}') FILE_NAME=test_02245.parquet DATA_FILE=$USER_FILES_PATH/$FILE_NAME diff --git a/tests/queries/0_stateless/02246_tsv_csv_best_effort_schema_inference.sh b/tests/queries/0_stateless/02246_tsv_csv_best_effort_schema_inference.sh index 233db7a534d..07c2a33c4d5 100755 --- a/tests/queries/0_stateless/02246_tsv_csv_best_effort_schema_inference.sh +++ b/tests/queries/0_stateless/02246_tsv_csv_best_effort_schema_inference.sh @@ -1,12 +1,11 @@ #!/usr/bin/env bash -# Tags: no-parallel, no-fasttest +# Tags: no-fasttest CURDIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd) # shellcheck source=../shell_config.sh . "$CURDIR"/../shell_config.sh -USER_FILES_PATH=$($CLICKHOUSE_CLIENT_BINARY --query "select _path,_file from file('nonexist.txt', 'CSV', 'val1 char')" 2>&1 | grep Exception | awk '{gsub("/nonexist.txt","",$9); print $9}') FILE_NAME=test_02149.data DATA_FILE=${USER_FILES_PATH:?}/$FILE_NAME diff --git a/tests/queries/0_stateless/02247_names_order_in_json_and_tskv.sh b/tests/queries/0_stateless/02247_names_order_in_json_and_tskv.sh index e8e3bf88ac4..3385f62af38 100755 --- a/tests/queries/0_stateless/02247_names_order_in_json_and_tskv.sh +++ b/tests/queries/0_stateless/02247_names_order_in_json_and_tskv.sh @@ -6,7 +6,6 @@ CURDIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd) . "$CURDIR"/../shell_config.sh -USER_FILES_PATH=$($CLICKHOUSE_CLIENT_BINARY --query "select _path,_file from file('nonexist.txt', 'CSV', 'val1 char')" 2>&1 | grep Exception | awk '{gsub("/nonexist.txt","",$9); print $9}') FILE_NAME=test_02247.data DATA_FILE=${USER_FILES_PATH:?}/$FILE_NAME diff --git a/tests/queries/0_stateless/02247_read_bools_as_numbers_json.sh b/tests/queries/0_stateless/02247_read_bools_as_numbers_json.sh index 523b5934543..76133df2b37 100755 --- a/tests/queries/0_stateless/02247_read_bools_as_numbers_json.sh +++ b/tests/queries/0_stateless/02247_read_bools_as_numbers_json.sh @@ -1,13 +1,12 @@ #!/usr/bin/env bash -# Tags: no-parallel, no-fasttest +# Tags: no-fasttest CURDIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd) # shellcheck source=../shell_config.sh . "$CURDIR"/../shell_config.sh -USER_FILES_PATH=$($CLICKHOUSE_CLIENT_BINARY --query "select _path,_file from file('nonexist.txt', 'CSV', 'val1 char')" 2>&1 | grep Exception | awk '{gsub("/nonexist.txt","",$9); print $9}') -FILE_NAME=test_02247.data +FILE_NAME=${CLICKHOUSE_DATABASE}.data DATA_FILE=${USER_FILES_PATH:?}/$FILE_NAME touch $DATA_FILE diff --git a/tests/queries/0_stateless/02270_errors_in_files.sh b/tests/queries/0_stateless/02270_errors_in_files.sh index 517547c6ef8..ab8bb28787d 100755 --- a/tests/queries/0_stateless/02270_errors_in_files.sh +++ b/tests/queries/0_stateless/02270_errors_in_files.sh @@ -13,8 +13,6 @@ echo "Error" > "${CLICKHOUSE_TMP}"/test_02270_2.csv ${CLICKHOUSE_LOCAL} --query "SELECT * FROM file('${CLICKHOUSE_TMP}/test_02270*.csv', CSV, 'a String, b String')" 2>&1 | grep -o "test_02270_2.csv" ${CLICKHOUSE_LOCAL} --query "SELECT * FROM file('${CLICKHOUSE_TMP}/test_02270*.csv', CSV, 'a String, b String')" --input_format_parallel_parsing 0 2>&1 | grep -o "test_02270_2.csv" -user_files_path=$($CLICKHOUSE_CLIENT --query "select _path,_file from file('nonexist.txt', 'CSV', 'val1 char')" 2>&1 | grep -E '^Code: 107.*FILE_DOESNT_EXIST' | head -1 | awk '{gsub("/nonexist.txt","",$9); print $9}') - ${CLICKHOUSE_CLIENT} --query "INSERT INTO TABLE FUNCTION file('test_02270_1.csv') SELECT 'Hello', 'World'" ${CLICKHOUSE_CLIENT} --query "INSERT INTO TABLE FUNCTION file('test_02270_2.csv') SELECT 'Error'" @@ -27,9 +25,9 @@ ${CLICKHOUSE_CLIENT} --query "INSERT INTO TABLE FUNCTION file('test_02270_2.csv. ${CLICKHOUSE_CLIENT} --query "SELECT * FROM file('test_02270*.csv.gz', 'CSV', 'a String, b String')" 2>&1 | grep -o -m1 "test_02270_2.csv.gz" ${CLICKHOUSE_CLIENT} --query "SELECT * FROM file('test_02270*.csv.gz', 'CSV', 'a String, b String')" --input_format_parallel_parsing 0 2>&1 | grep -o -m1 "test_02270_2.csv.gz" -rm "${CLICKHOUSE_TMP}"/test_02270_1.csv -rm "${CLICKHOUSE_TMP}"/test_02270_2.csv -rm "${user_files_path}"/test_02270_1.csv -rm "${user_files_path}"/test_02270_2.csv -rm "${user_files_path}"/test_02270_1.csv.gz -rm "${user_files_path}"/test_02270_2.csv.gz +rm -f "${CLICKHOUSE_TMP}"/test_02270_1.csv +rm -f "${CLICKHOUSE_TMP}"/test_02270_2.csv +rm -f "${USER_FILES_PATH}"/test_02270_1.csv +rm -f "${USER_FILES_PATH}"/test_02270_2.csv +rm -f "${USER_FILES_PATH}"/test_02270_1.csv.gz +rm -f "${USER_FILES_PATH}"/test_02270_2.csv.gz diff --git a/tests/queries/0_stateless/02286_mysql_dump_input_format.sh b/tests/queries/0_stateless/02286_mysql_dump_input_format.sh index 2f6167c3ddf..59528d97b93 100755 --- a/tests/queries/0_stateless/02286_mysql_dump_input_format.sh +++ b/tests/queries/0_stateless/02286_mysql_dump_input_format.sh @@ -5,8 +5,6 @@ CURDIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd) # shellcheck source=../shell_config.sh . "$CURDIR"/../shell_config.sh -USER_FILES_PATH=$($CLICKHOUSE_CLIENT_BINARY --query "select _path,_file from file('nonexist.txt', 'CSV', 'val1 char')" 2>&1 | grep Exception | awk '{gsub("/nonexist.txt","",$9); print $9}') - cp $CURDIR/data_mysql_dump/dump*.sql $USER_FILES_PATH $CLICKHOUSE_CLIENT -q "select * from file(dump1.sql, MySQLDump, 'x Nullable(Int32), y Nullable(Int32)') order by x, y" diff --git a/tests/queries/0_stateless/02293_formats_json_columns.sh b/tests/queries/0_stateless/02293_formats_json_columns.sh index 4eae5a1abb4..bf605d6f591 100755 --- a/tests/queries/0_stateless/02293_formats_json_columns.sh +++ b/tests/queries/0_stateless/02293_formats_json_columns.sh @@ -5,8 +5,6 @@ CUR_DIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd) # shellcheck source=../shell_config.sh . "$CUR_DIR"/../shell_config.sh -USER_FILES_PATH=$(clickhouse client --query "select _path,_file from file('nonexist.txt', 'CSV', 'val1 char')" 2>&1 | grep Exception | awk '{gsub("/nonexist.txt","",$9); print $9}') - DATA_FILE=$USER_FILES_PATH/data_02293 $CLICKHOUSE_CLIENT -q "drop table if exists test_02293" diff --git a/tests/queries/0_stateless/02297_regex_parsing_file_names.sh b/tests/queries/0_stateless/02297_regex_parsing_file_names.sh index 5973e24844a..666cb4d87fc 100755 --- a/tests/queries/0_stateless/02297_regex_parsing_file_names.sh +++ b/tests/queries/0_stateless/02297_regex_parsing_file_names.sh @@ -5,29 +5,21 @@ CURDIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd) # shellcheck source=../shell_config.sh . "$CURDIR"/../shell_config.sh -# Data preparation. +rm -rf ${USER_FILES_PATH}/file_{0..10}.csv -# Now we can get the user_files_path by use the table file function for trick. also we can get it by query as: -# "insert into function file('exist.txt', 'CSV', 'val1 char') values ('aaaa'); select _path from file('exist.txt', 'CSV', 'val1 char')" -CLICKHOUSE_USER_FILES_PATH=$($CLICKHOUSE_CLIENT_BINARY --query "select _path, _file from file('nonexist.txt', 'CSV', 'val1 char')" 2>&1 | grep Exception | awk '{gsub("/nonexist.txt","",$9); print $9}') +echo '0' > ${USER_FILES_PATH}/file_0.csv +echo '0' > ${USER_FILES_PATH}/file_1.csv +echo '0' > ${USER_FILES_PATH}/file_2.csv +echo '0' > ${USER_FILES_PATH}/file_3.csv +echo '0' > ${USER_FILES_PATH}/file_4.csv +echo '0' > ${USER_FILES_PATH}/file_5.csv +echo '0' > ${USER_FILES_PATH}/file_6.csv +echo '0' > ${USER_FILES_PATH}/file_7.csv +echo '0' > ${USER_FILES_PATH}/file_8.csv +echo '0' > ${USER_FILES_PATH}/file_9.csv +echo '0' > ${USER_FILES_PATH}/file_10.csv -mkdir -p ${CLICKHOUSE_USER_FILES_PATH}/ - -rm -rf ${CLICKHOUSE_USER_FILES_PATH}/file_{0..10}.csv - -echo '0' > ${CLICKHOUSE_USER_FILES_PATH}/file_0.csv -echo '0' > ${CLICKHOUSE_USER_FILES_PATH}/file_1.csv -echo '0' > ${CLICKHOUSE_USER_FILES_PATH}/file_2.csv -echo '0' > ${CLICKHOUSE_USER_FILES_PATH}/file_3.csv -echo '0' > ${CLICKHOUSE_USER_FILES_PATH}/file_4.csv -echo '0' > ${CLICKHOUSE_USER_FILES_PATH}/file_5.csv -echo '0' > ${CLICKHOUSE_USER_FILES_PATH}/file_6.csv -echo '0' > ${CLICKHOUSE_USER_FILES_PATH}/file_7.csv -echo '0' > ${CLICKHOUSE_USER_FILES_PATH}/file_8.csv -echo '0' > ${CLICKHOUSE_USER_FILES_PATH}/file_9.csv -echo '0' > ${CLICKHOUSE_USER_FILES_PATH}/file_10.csv - -# echo '' > ${CLICKHOUSE_USER_FILES_PATH}/file_10.csv +# echo '' > ${USER_FILES_PATH}/file_10.csv ${CLICKHOUSE_CLIENT} -q "DROP TABLE IF EXISTS t_regex;" @@ -36,5 +28,5 @@ ${CLICKHOUSE_CLIENT} -q "CREATE TABLE t_regex (id UInt64) ENGINE = MergeTree() o ${CLICKHOUSE_CLIENT} -q "INSERT INTO t_regex SELECT * FROM file('file_{0..10}.csv','CSV');" ${CLICKHOUSE_CLIENT} -q "SELECT count() from t_regex;" -rm -rf ${CLICKHOUSE_USER_FILES_PATH}/file_{0..10}.csv; +rm -rf ${USER_FILES_PATH}/file_{0..10}.csv; ${CLICKHOUSE_CLIENT} -q "DROP TABLE IF EXISTS t_regex;" diff --git a/tests/queries/0_stateless/02327_capnproto_protobuf_empty_messages.sh b/tests/queries/0_stateless/02327_capnproto_protobuf_empty_messages.sh index 650faf6985e..89e5c827a48 100755 --- a/tests/queries/0_stateless/02327_capnproto_protobuf_empty_messages.sh +++ b/tests/queries/0_stateless/02327_capnproto_protobuf_empty_messages.sh @@ -5,10 +5,9 @@ CURDIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd) # shellcheck source=../shell_config.sh . "$CURDIR"/../shell_config.sh -USER_FILES_PATH=$($CLICKHOUSE_CLIENT_BINARY --query "select _path,_file from file('nonexist.txt', 'CSV', 'val1 char')" 2>&1 | grep Exception | awk '{gsub("/nonexist.txt","",$9); print $9}') touch $USER_FILES_PATH/data.capnp -SCHEMADIR=$($CLICKHOUSE_CLIENT_BINARY --query "select * from file('data.capnp', 'CapnProto', 'val1 char') settings format_schema='nonexist:Message'" 2>&1 | grep Exception | grep -oP "file \K.*(?=/nonexist.capnp)") +SCHEMADIR=${CLICKHOUSE_SCHEMA_FILES} CLIENT_SCHEMADIR=$CURDIR/format_schemas SERVER_SCHEMADIR=test_02327 mkdir -p $SCHEMADIR/$SERVER_SCHEMADIR diff --git a/tests/queries/0_stateless/02353_compression_level.sh b/tests/queries/0_stateless/02353_compression_level.sh index 8d6a9c899ad..9e102d12fed 100755 --- a/tests/queries/0_stateless/02353_compression_level.sh +++ b/tests/queries/0_stateless/02353_compression_level.sh @@ -1,12 +1,11 @@ #!/usr/bin/env bash -# Tags: no-fasttest, no-parallel +# Tags: no-fasttest # Tag no-fasttest: depends on brotli and bzip2 CURDIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd) # shellcheck source=../shell_config.sh . "$CURDIR"/../shell_config.sh -USER_FILES_PATH=$($CLICKHOUSE_CLIENT_BINARY --query "select _path,_file from file('nonexist.txt', 'CSV', 'val1 char')" 2>&1 | grep Exception | awk '{gsub("/nonexist.txt","",$9); print $9}') WORKING_FOLDER_02353="${USER_FILES_PATH}/${CLICKHOUSE_DATABASE}" rm -rf "${WORKING_FOLDER_02353}" diff --git a/tests/queries/0_stateless/02358_file_default_value.sh b/tests/queries/0_stateless/02358_file_default_value.sh index a7c4c17c129..0fd97a09546 100755 --- a/tests/queries/0_stateless/02358_file_default_value.sh +++ b/tests/queries/0_stateless/02358_file_default_value.sh @@ -4,7 +4,6 @@ CURDIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd) # shellcheck source=../shell_config.sh . "$CURDIR"/../shell_config.sh -USER_FILES_PATH=$($CLICKHOUSE_CLIENT_BINARY --query "select _path,_file from file('nonexist.txt', 'CSV', 'val1 char')" 2>&1 | grep Exception | awk '{gsub("/nonexist.txt","",$9); print $9}') WORKING_FOLDER_02357="${USER_FILES_PATH}/${CLICKHOUSE_DATABASE}" rm -rf "${WORKING_FOLDER_02357}" diff --git a/tests/queries/0_stateless/02360_clickhouse_local_config-option.sh b/tests/queries/0_stateless/02360_clickhouse_local_config-option.sh index b58cfd7ec21..50e07ca8612 100755 --- a/tests/queries/0_stateless/02360_clickhouse_local_config-option.sh +++ b/tests/queries/0_stateless/02360_clickhouse_local_config-option.sh @@ -15,7 +15,7 @@ echo " true - 9000 + ${CLICKHOUSE_PORT_TCP} ${SAFE_DIR} diff --git a/tests/queries/0_stateless/02372_data_race_in_avro.sh b/tests/queries/0_stateless/02372_data_race_in_avro.sh index 50a7ae1e3c5..49c34e31923 100755 --- a/tests/queries/0_stateless/02372_data_race_in_avro.sh +++ b/tests/queries/0_stateless/02372_data_race_in_avro.sh @@ -1,5 +1,5 @@ #!/usr/bin/env bash -# Tags: no-fasttest, no-parallel +# Tags: no-fasttest CURDIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd) # shellcheck source=../shell_config.sh diff --git a/tests/queries/0_stateless/02373_heap_buffer_overflow_in_avro.sh b/tests/queries/0_stateless/02373_heap_buffer_overflow_in_avro.sh index 3461287d28a..e95bda2adfb 100755 --- a/tests/queries/0_stateless/02373_heap_buffer_overflow_in_avro.sh +++ b/tests/queries/0_stateless/02373_heap_buffer_overflow_in_avro.sh @@ -5,8 +5,6 @@ CURDIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd) # shellcheck source=../shell_config.sh . "$CURDIR"/../shell_config.sh -USER_FILES_PATH=$($CLICKHOUSE_CLIENT_BINARY --query "select _path,_file from file('nonexist.txt', 'CSV', 'val1 char')" 2>&1 | grep Exception | awk '{gsub("/nonexist.txt","",$9); print $9}') - cp $CURDIR/data_avro/corrupted.avro $USER_FILES_PATH/ $CLICKHOUSE_CLIENT -q "select * from file(corrupted.avro)" 2>&1 | grep -F -q "Cannot read compressed data" && echo "OK" || echo "FAIL" diff --git a/tests/queries/0_stateless/02383_arrow_dict_special_cases.sh b/tests/queries/0_stateless/02383_arrow_dict_special_cases.sh index 80743a97dd0..ae08941da63 100755 --- a/tests/queries/0_stateless/02383_arrow_dict_special_cases.sh +++ b/tests/queries/0_stateless/02383_arrow_dict_special_cases.sh @@ -5,7 +5,6 @@ CURDIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd) # shellcheck source=../shell_config.sh . "$CURDIR"/../shell_config.sh -USER_FILES_PATH=$($CLICKHOUSE_CLIENT_BINARY --query "select _path,_file from file('nonexist.txt', 'CSV', 'val1 char')" 2>&1 | grep Exception | awk '{gsub("/nonexist.txt","",$9); print $9}') UNIQ_DEST_PATH=$USER_FILES_PATH/test-02383-$RANDOM-$RANDOM mkdir -p $UNIQ_DEST_PATH diff --git a/tests/queries/0_stateless/02402_capnp_format_segments_overflow.sh b/tests/queries/0_stateless/02402_capnp_format_segments_overflow.sh index 8aad68ffe5c..3028451a5f5 100755 --- a/tests/queries/0_stateless/02402_capnp_format_segments_overflow.sh +++ b/tests/queries/0_stateless/02402_capnp_format_segments_overflow.sh @@ -5,11 +5,10 @@ CURDIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd) # shellcheck source=../shell_config.sh . "$CURDIR"/../shell_config.sh -USER_FILES_PATH=$($CLICKHOUSE_CLIENT_BINARY --query "select _path,_file from file('nonexist.txt', 'CSV', 'val1 char')" 2>&1 | grep Exception | awk '{gsub("/nonexist.txt","",$9); print $9}') mkdir -p $USER_FILES_PATH/test_02402 cp $CURDIR/data_capnp/overflow.capnp $USER_FILES_PATH/test_02402/ -SCHEMADIR=$($CLICKHOUSE_CLIENT_BINARY --query "select * from file('test_02402/overflow.capnp', 'CapnProto', 'val1 char') settings format_schema='nonexist:Message'" 2>&1 | grep Exception | grep -oP "file \K.*(?=/nonexist.capnp)") +SCHEMADIR=${CLICKHOUSE_SCHEMA_FILES} CLIENT_SCHEMADIR=$CURDIR/format_schemas SERVER_SCHEMADIR=test_02402 diff --git a/tests/queries/0_stateless/02421_record_errors_row_by_input_format.sh b/tests/queries/0_stateless/02421_record_errors_row_by_input_format.sh index df304eeeba5..72b55144348 100755 --- a/tests/queries/0_stateless/02421_record_errors_row_by_input_format.sh +++ b/tests/queries/0_stateless/02421_record_errors_row_by_input_format.sh @@ -9,10 +9,7 @@ CURDIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd) # Data preparation. -CLICKHOUSE_USER_FILES_PATH=$($CLICKHOUSE_CLIENT_BINARY --query "select _path, _file from file('nonexist.txt', 'CSV', 'val1 char')" 2>&1 | grep Exception | awk '{gsub("/nonexist.txt","",$9); print $9}') - -mkdir -p ${CLICKHOUSE_USER_FILES_PATH}/ -echo -e "1,1\n2,a\nb,3\n4,4\n5,c\n6,6" > ${CLICKHOUSE_USER_FILES_PATH}/a.csv +echo -e "1,1\n2,a\nb,3\n4,4\n5,c\n6,6" > ${USER_FILES_PATH}/a.csv ${CLICKHOUSE_CLIENT} --query "drop table if exists data;" ${CLICKHOUSE_CLIENT} --query "create table data (A UInt8, B UInt8) engine=MergeTree() order by A;" @@ -23,12 +20,12 @@ sleep 2 ${CLICKHOUSE_CLIENT} --query "select * except (time) from file('errors_server', 'CSV', 'time DateTime, database Nullable(String), table Nullable(String), offset UInt32, reason String, raw_data String');" # Client side -${CLICKHOUSE_CLIENT} --input_format_allow_errors_num 4 --input_format_record_errors_file_path "${CLICKHOUSE_USER_FILES_PATH}/errors_client" --query "insert into data(A, B) format CSV" < ${CLICKHOUSE_USER_FILES_PATH}/a.csv +${CLICKHOUSE_CLIENT} --input_format_allow_errors_num 4 --input_format_record_errors_file_path "${USER_FILES_PATH}/errors_client" --query "insert into data(A, B) format CSV" < ${USER_FILES_PATH}/a.csv sleep 2 ${CLICKHOUSE_CLIENT} --query "select * except (time) from file('errors_client', 'CSV', 'time DateTime, database Nullable(String), table Nullable(String), offset UInt32, reason String, raw_data String');" # Restore ${CLICKHOUSE_CLIENT} --query "drop table if exists data;" -rm ${CLICKHOUSE_USER_FILES_PATH}/a.csv -rm ${CLICKHOUSE_USER_FILES_PATH}/errors_server -rm ${CLICKHOUSE_USER_FILES_PATH}/errors_client +rm ${USER_FILES_PATH}/a.csv +rm ${USER_FILES_PATH}/errors_server +rm ${USER_FILES_PATH}/errors_client diff --git a/tests/queries/0_stateless/02422_allow_implicit_no_password.sh b/tests/queries/0_stateless/02422_allow_implicit_no_password.sh index 013c367e079..3c433856be2 100755 --- a/tests/queries/0_stateless/02422_allow_implicit_no_password.sh +++ b/tests/queries/0_stateless/02422_allow_implicit_no_password.sh @@ -9,7 +9,7 @@ CURDIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd) # shellcheck source=../shell_config.sh . "$CURDIR"/../shell_config.sh -cp /etc/clickhouse-server/users.xml "$CURDIR"/users.xml +cp ${CLICKHOUSE_CONFIG_DIR}/users.xml "$CURDIR"/users.xml sed -i 's/<\/password>/c64c5e4e53ea1a9f1427d2713b3a22bbebe8940bc807adaf654744b1568c70ab<\/password_sha256_hex>/g' "$CURDIR"/users.xml sed -i 's//1<\/access_management>/g' "$CURDIR"/users.xml diff --git a/tests/queries/0_stateless/02455_one_row_from_csv_memory_usage.sh b/tests/queries/0_stateless/02455_one_row_from_csv_memory_usage.sh index 05de3f05562..7906f2917c4 100755 --- a/tests/queries/0_stateless/02455_one_row_from_csv_memory_usage.sh +++ b/tests/queries/0_stateless/02455_one_row_from_csv_memory_usage.sh @@ -5,11 +5,9 @@ CUR_DIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd) # shellcheck source=../shell_config.sh . "$CUR_DIR"/../shell_config.sh -USER_FILES_PATH=$($CLICKHOUSE_CLIENT --query "select _path,_file from file('nonexist.txt', 'CSV', 'val1 char')" 2>&1 | grep -E '^Code: 107.*FILE_DOESNT_EXIST' | head -1 | awk '{gsub("/nonexist.txt","",$9); print $9}') +cp "$CUR_DIR"/data_csv/10m_rows.csv.xz $USER_FILES_PATH/${CLICKHOUSE_DATABASE}_10m_rows.csv.xz -cp "$CUR_DIR"/data_csv/10m_rows.csv.xz $USER_FILES_PATH/ +${CLICKHOUSE_CLIENT} --query="SELECT * FROM file('${CLICKHOUSE_DATABASE}_10m_rows.csv.xz' , 'CSVWithNames') order by identifier, number, name, surname, birthday LIMIT 1 settings input_format_parallel_parsing=1, max_threads=1, max_parsing_threads=16, min_chunk_bytes_for_parallel_parsing=10485760, max_memory_usage=1000000000" +${CLICKHOUSE_CLIENT} --query="SELECT * FROM file('${CLICKHOUSE_DATABASE}_10m_rows.csv.xz' , 'CSVWithNames') order by identifier, number, name, surname, birthday LIMIT 1 settings input_format_parallel_parsing=1, max_threads=1, max_parsing_threads=16, min_chunk_bytes_for_parallel_parsing=10485760, max_memory_usage=100000000" -${CLICKHOUSE_CLIENT} --query="SELECT * FROM file('10m_rows.csv.xz' , 'CSVWithNames') order by identifier, number, name, surname, birthday LIMIT 1 settings input_format_parallel_parsing=1, max_threads=1, max_parsing_threads=16, min_chunk_bytes_for_parallel_parsing=10485760, max_memory_usage=1000000000" -${CLICKHOUSE_CLIENT} --query="SELECT * FROM file('10m_rows.csv.xz' , 'CSVWithNames') order by identifier, number, name, surname, birthday LIMIT 1 settings input_format_parallel_parsing=1, max_threads=1, max_parsing_threads=16, min_chunk_bytes_for_parallel_parsing=10485760, max_memory_usage=100000000" - -rm $USER_FILES_PATH/10m_rows.csv.xz +rm $USER_FILES_PATH/${CLICKHOUSE_DATABASE}_10m_rows.csv.xz diff --git a/tests/queries/0_stateless/02457_bz2_concatenated.sh b/tests/queries/0_stateless/02457_bz2_concatenated.sh index 96e23cbfa2a..a9991cf44e7 100755 --- a/tests/queries/0_stateless/02457_bz2_concatenated.sh +++ b/tests/queries/0_stateless/02457_bz2_concatenated.sh @@ -6,7 +6,6 @@ CURDIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd) # shellcheck source=../shell_config.sh . "$CURDIR"/../shell_config.sh -USER_FILES_PATH=$($CLICKHOUSE_CLIENT_BINARY --query "select _path,_file from file('nonexist.txt', 'CSV', 'val1 char')" 2>&1 | grep Exception | awk '{gsub("/nonexist.txt","",$9); print $9}') WORKING_FOLDER_02457="${USER_FILES_PATH}/${CLICKHOUSE_DATABASE}" rm -rf "${WORKING_FOLDER_02457}" diff --git a/tests/queries/0_stateless/02459_glob_for_recursive_directory_traversal.sh b/tests/queries/0_stateless/02459_glob_for_recursive_directory_traversal.sh index b8430307ea3..b86385b72c4 100755 --- a/tests/queries/0_stateless/02459_glob_for_recursive_directory_traversal.sh +++ b/tests/queries/0_stateless/02459_glob_for_recursive_directory_traversal.sh @@ -5,28 +5,26 @@ CUR_DIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd) # shellcheck source=../shell_config.sh . "$CUR_DIR"/../shell_config.sh -user_files_path=$($CLICKHOUSE_CLIENT_BINARY --query "select _path,_file from file('nonexist.txt', 'CSV', 'val1 char')" 2>&1 | grep Exception | awk '{gsub("/nonexist.txt","",$9); print $9}') - -mkdir $user_files_path/d1 -touch $user_files_path/d1/text1.txt +mkdir $USER_FILES_PATH/d1 +touch $USER_FILES_PATH/d1/text1.txt for i in {1..2} do - echo $i$'\t'$i >> $user_files_path/d1/text1.txt + echo $i$'\t'$i >> $USER_FILES_PATH/d1/text1.txt done -mkdir $user_files_path/d1/d2 -touch $user_files_path/d1/d2/text2.txt +mkdir $USER_FILES_PATH/d1/d2 +touch $USER_FILES_PATH/d1/d2/text2.txt for i in {3..4} do - echo $i$'\t'$i >> $user_files_path/d1/d2/text2.txt + echo $i$'\t'$i >> $USER_FILES_PATH/d1/d2/text2.txt done -mkdir $user_files_path/d1/d2/d3 -touch $user_files_path/d1/d2/d3/text3.txt +mkdir $USER_FILES_PATH/d1/d2/d3 +touch $USER_FILES_PATH/d1/d2/d3/text3.txt for i in {5..6} do - echo $i$'\t'$i >> $user_files_path/d1/d2/d3/text3.txt + echo $i$'\t'$i >> $USER_FILES_PATH/d1/d2/d3/text3.txt done ${CLICKHOUSE_CLIENT} -q "SELECT * from file ('d1/*','TSV', 'Index UInt8, Number UInt8')" | sort --numeric-sort @@ -35,9 +33,9 @@ ${CLICKHOUSE_CLIENT} -q "SELECT * from file ('d1/*/tex*','TSV', 'Index UInt8, Nu ${CLICKHOUSE_CLIENT} -q "SELECT * from file ('d1/**/tex*','TSV', 'Index UInt8, Number UInt8')" | sort --numeric-sort -rm $user_files_path/d1/d2/d3/text3.txt -rmdir $user_files_path/d1/d2/d3 -rm $user_files_path/d1/d2/text2.txt -rmdir $user_files_path/d1/d2 -rm $user_files_path/d1/text1.txt -rmdir $user_files_path/d1 +rm $USER_FILES_PATH/d1/d2/d3/text3.txt +rmdir $USER_FILES_PATH/d1/d2/d3 +rm $USER_FILES_PATH/d1/d2/text2.txt +rmdir $USER_FILES_PATH/d1/d2 +rm $USER_FILES_PATH/d1/text1.txt +rmdir $USER_FILES_PATH/d1 diff --git a/tests/queries/0_stateless/02475_bson_each_row_format.sh b/tests/queries/0_stateless/02475_bson_each_row_format.sh index 474a6cd0e47..2975b40c868 100755 --- a/tests/queries/0_stateless/02475_bson_each_row_format.sh +++ b/tests/queries/0_stateless/02475_bson_each_row_format.sh @@ -1,5 +1,5 @@ #!/usr/bin/env bash -# Tags: no-parallel, no-debug +# Tags: no-debug CURDIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd) # shellcheck source=../shell_config.sh diff --git a/tests/queries/0_stateless/02479_race_condition_between_insert_and_droppin_mv.sh b/tests/queries/0_stateless/02479_race_condition_between_insert_and_droppin_mv.sh index 6899b31d1d9..935ea03a947 100755 --- a/tests/queries/0_stateless/02479_race_condition_between_insert_and_droppin_mv.sh +++ b/tests/queries/0_stateless/02479_race_condition_between_insert_and_droppin_mv.sh @@ -38,7 +38,7 @@ ${CLICKHOUSE_CLIENT} -q "CREATE TABLE test_race_condition_landing (number Int64, export -f drop_mv; export -f insert; -TIMEOUT=55 +TIMEOUT=50 for i in {1..4} do diff --git a/tests/queries/0_stateless/02482_capnp_list_of_structs.sh b/tests/queries/0_stateless/02482_capnp_list_of_structs.sh index 9d78b9893dd..a04c631c411 100755 --- a/tests/queries/0_stateless/02482_capnp_list_of_structs.sh +++ b/tests/queries/0_stateless/02482_capnp_list_of_structs.sh @@ -5,10 +5,9 @@ CURDIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd) # shellcheck source=../shell_config.sh . "$CURDIR"/../shell_config.sh -USER_FILES_PATH=$($CLICKHOUSE_CLIENT_BINARY --query "select _path,_file from file('nonexist.txt', 'CSV', 'val1 char')" 2>&1 | grep Exception | awk '{gsub("/nonexist.txt","",$9); print $9}') touch $USER_FILES_PATH/data.capnp -SCHEMADIR=$($CLICKHOUSE_CLIENT_BINARY --query "select * from file('data.capnp', 'CapnProto', 'val1 char') settings format_schema='nonexist:Message'" 2>&1 | grep Exception | grep -oP "file \K.*(?=/nonexist.capnp)") +SCHEMADIR=${CLICKHOUSE_SCHEMA_FILES} CLIENT_SCHEMADIR=$CURDIR/format_schemas SERVER_SCHEMADIR=test_02482 mkdir -p $SCHEMADIR/$SERVER_SCHEMADIR diff --git a/tests/queries/0_stateless/02483_capnp_decimals.sh b/tests/queries/0_stateless/02483_capnp_decimals.sh index ef545a5539f..bc19b63fc8b 100755 --- a/tests/queries/0_stateless/02483_capnp_decimals.sh +++ b/tests/queries/0_stateless/02483_capnp_decimals.sh @@ -5,19 +5,17 @@ CURDIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd) # shellcheck source=../shell_config.sh . "$CURDIR"/../shell_config.sh -USER_FILES_PATH=$($CLICKHOUSE_CLIENT_BINARY --query "select _path,_file from file('nonexist.txt', 'CSV', 'val1 char')" 2>&1 | grep Exception | awk '{gsub("/nonexist.txt","",$9); print $9}') -touch $USER_FILES_PATH/data.capnp +touch $CLICKHOUSE_USER_FILES/data.capnp -SCHEMADIR=$($CLICKHOUSE_CLIENT_BINARY --query "select * from file('data.capnp', 'CapnProto', 'val1 char') settings format_schema='nonexist:Message'" 2>&1 | grep Exception | grep -oP "file \K.*(?=/nonexist.capnp)") CLIENT_SCHEMADIR=$CURDIR/format_schemas SERVER_SCHEMADIR=test_02483 -mkdir -p $SCHEMADIR/$SERVER_SCHEMADIR -cp -r $CLIENT_SCHEMADIR/02483_* $SCHEMADIR/$SERVER_SCHEMADIR/ +mkdir -p $CLICKHOUSE_SCHEMA_FILES/$SERVER_SCHEMADIR +cp -r $CLIENT_SCHEMADIR/02483_* $CLICKHOUSE_SCHEMA_FILES/$SERVER_SCHEMADIR/ $CLICKHOUSE_CLIENT -q "insert into function file(02483_data.capnp, auto, 'decimal32 Decimal32(3), decimal64 Decimal64(6)') select 42.42, 4242.424242 settings format_schema='$SERVER_SCHEMADIR/02483_decimals.capnp:Message', engine_file_truncate_on_insert=1" $CLICKHOUSE_CLIENT -q "select * from file(02483_data.capnp) settings format_schema='$SERVER_SCHEMADIR/02483_decimals.capnp:Message'" $CLICKHOUSE_CLIENT -q "select * from file(02483_data.capnp, auto, 'decimal64 Decimal64(6), decimal32 Decimal32(3)') settings format_schema='$SERVER_SCHEMADIR/02483_decimals.capnp:Message'" -rm $USER_FILES_PATH/data.capnp -rm $USER_FILES_PATH/02483_data.capnp +rm $CLICKHOUSE_USER_FILES/data.capnp +rm $CLICKHOUSE_USER_FILES/02483_data.capnp diff --git a/tests/queries/0_stateless/02504_regexp_dictionary_ua_parser.sh b/tests/queries/0_stateless/02504_regexp_dictionary_ua_parser.sh index d3a8743b880..e389cf410e8 100755 --- a/tests/queries/0_stateless/02504_regexp_dictionary_ua_parser.sh +++ b/tests/queries/0_stateless/02504_regexp_dictionary_ua_parser.sh @@ -1,18 +1,15 @@ #!/usr/bin/env bash - -# Tags: no-fasttest, no-parallel +# Tags: no-fasttest CURDIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd) # shellcheck source=../shell_config.sh . "$CURDIR"/../shell_config.sh -user_files_path=$($CLICKHOUSE_CLIENT_BINARY --query "select _path,_file from file('nonexist.txt', 'CSV', 'val1 char')" 2>&1 | grep Exception | awk '{gsub("/nonexist.txt","",$9); print $9}') +mkdir -p ${USER_FILES_PATH:?}/${CLICKHOUSE_DATABASE} -mkdir -p $user_files_path/test_02504 - -cp $CURDIR/data_ua_parser/os.yaml ${user_files_path}/test_02504/ -cp $CURDIR/data_ua_parser/browser.yaml ${user_files_path}/test_02504/ -cp $CURDIR/data_ua_parser/device.yaml ${user_files_path}/test_02504/ +cp $CURDIR/data_ua_parser/os.yaml ${USER_FILES_PATH}/${CLICKHOUSE_DATABASE}/ +cp $CURDIR/data_ua_parser/browser.yaml ${USER_FILES_PATH}/${CLICKHOUSE_DATABASE}/ +cp $CURDIR/data_ua_parser/device.yaml ${USER_FILES_PATH}/${CLICKHOUSE_DATABASE}/ $CLICKHOUSE_CLIENT -n --query=" drop dictionary if exists regexp_os; @@ -29,7 +26,7 @@ create dictionary regexp_os os_v4_replacement String default '0' ) PRIMARY KEY(regex) -SOURCE(YAMLRegExpTree(PATH '${user_files_path}/test_02504/os.yaml')) +SOURCE(YAMLRegExpTree(PATH '${USER_FILES_PATH}/${CLICKHOUSE_DATABASE}/os.yaml')) LIFETIME(0) LAYOUT(regexp_tree); @@ -41,7 +38,7 @@ create dictionary regexp_browser v2_replacement String default '0' ) PRIMARY KEY(regex) -SOURCE(YAMLRegExpTree(PATH '${user_files_path}/test_02504/browser.yaml')) +SOURCE(YAMLRegExpTree(PATH '${USER_FILES_PATH}/${CLICKHOUSE_DATABASE}/browser.yaml')) LIFETIME(0) LAYOUT(regexp_tree); @@ -53,7 +50,7 @@ create dictionary regexp_device model_replacement String ) PRIMARY KEY(regex) -SOURCE(YAMLRegExpTree(PATH '${user_files_path}/test_02504/device.yaml')) +SOURCE(YAMLRegExpTree(PATH '${USER_FILES_PATH}/${CLICKHOUSE_DATABASE}/device.yaml')) LIFETIME(0) LAYOUT(regexp_tree); @@ -84,4 +81,4 @@ drop dictionary if exists regexp_device; drop table if exists user_agents; " -rm -rf "$user_files_path/test_02504" +rm -rf ${USER_FILES_PATH:?}/${CLICKHOUSE_DATABASE} diff --git a/tests/queries/0_stateless/02504_regexp_dictionary_yaml_source.sh b/tests/queries/0_stateless/02504_regexp_dictionary_yaml_source.sh index 7211372f2f7..68a87a14320 100755 --- a/tests/queries/0_stateless/02504_regexp_dictionary_yaml_source.sh +++ b/tests/queries/0_stateless/02504_regexp_dictionary_yaml_source.sh @@ -1,13 +1,10 @@ #!/usr/bin/env bash - # Tags: use-vectorscan, no-fasttest, no-parallel CURDIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd) # shellcheck source=../shell_config.sh . "$CURDIR"/../shell_config.sh -USER_FILES_PATH=$($CLICKHOUSE_CLIENT_BINARY --query "select _path,_file from file('nonexist.txt', 'CSV', 'val1 char')" 2>&1 | grep Exception | awk '{gsub("/nonexist.txt","",$9); print $9}') - mkdir -p $USER_FILES_PATH/test_02504 yaml=$USER_FILES_PATH/test_02504/test.yaml diff --git a/tests/queries/0_stateless/02661_read_from_archive.lib b/tests/queries/0_stateless/02661_read_from_archive.lib index 908b6bd38d2..56f1a0f163c 100644 --- a/tests/queries/0_stateless/02661_read_from_archive.lib +++ b/tests/queries/0_stateless/02661_read_from_archive.lib @@ -6,10 +6,10 @@ CUR_DIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd) . "$CUR_DIR"/../shell_config.sh function read_archive_file() { - $CLICKHOUSE_LOCAL --query "SELECT * FROM file('${user_files_path}/$1') ORDER BY 1, 2" - $CLICKHOUSE_CLIENT --query "SELECT * FROM file('${user_files_path}/$1') ORDER BY 1, 2" - $CLICKHOUSE_CLIENT --query "DESC file('${user_files_path}/$1')" - $CLICKHOUSE_CLIENT --query "CREATE TABLE 02661_archive_table Engine=File('CSV', '${user_files_path}/$1')" + $CLICKHOUSE_LOCAL --query "SELECT * FROM file('${USER_FILES_PATH}/$1') ORDER BY 1, 2" + $CLICKHOUSE_CLIENT --query "SELECT * FROM file('${USER_FILES_PATH}/$1') ORDER BY 1, 2" + $CLICKHOUSE_CLIENT --query "DESC file('${USER_FILES_PATH}/$1')" + $CLICKHOUSE_CLIENT --query "CREATE TABLE 02661_archive_table Engine=File('CSV', '${USER_FILES_PATH}/$1')" $CLICKHOUSE_CLIENT --query "SELECT * FROM 02661_archive_table ORDER BY 1, 2" $CLICKHOUSE_CLIENT --query "DROP TABLE 02661_archive_table" } @@ -20,16 +20,14 @@ function run_archive_test() { extension_without_dot=$(echo $1 | sed -e 's/\.//g') FILE_PREFIX="02661_read_from_archive_${CLICKHOUSE_DATABASE}_$extension_without_dot" - user_files_path=$(clickhouse-client --query "select _path, _file from file('nonexist.txt', 'CSV', 'val1 char')" 2>&1 | grep -o "/[^[:space:]]*nonexist.txt" | awk '{gsub("/nonexist.txt","",$1); print $1}') - touch ${FILE_PREFIX}_data0.csv echo -e "1,2\n3,4" > ${FILE_PREFIX}_data1.csv echo -e "5,6\n7,8" > ${FILE_PREFIX}_data2.csv echo -e "9,10\n11,12" > ${FILE_PREFIX}_data3.csv - eval "$2 ${user_files_path}/${FILE_PREFIX}_archive1.$1 ${FILE_PREFIX}_data0.csv ${FILE_PREFIX}_data1.csv ${FILE_PREFIX}_data2.csv > /dev/null" - eval "$2 ${user_files_path}/${FILE_PREFIX}_archive2.$1 ${FILE_PREFIX}_data1.csv ${FILE_PREFIX}_data3.csv > /dev/null" - eval "$2 ${user_files_path}/${FILE_PREFIX}_archive3.$1 ${FILE_PREFIX}_data2.csv ${FILE_PREFIX}_data3.csv > /dev/null" + eval "$2 ${USER_FILES_PATH}/${FILE_PREFIX}_archive1.$1 ${FILE_PREFIX}_data0.csv ${FILE_PREFIX}_data1.csv ${FILE_PREFIX}_data2.csv > /dev/null" + eval "$2 ${USER_FILES_PATH}/${FILE_PREFIX}_archive2.$1 ${FILE_PREFIX}_data1.csv ${FILE_PREFIX}_data3.csv > /dev/null" + eval "$2 ${USER_FILES_PATH}/${FILE_PREFIX}_archive3.$1 ${FILE_PREFIX}_data2.csv ${FILE_PREFIX}_data3.csv > /dev/null" echo "archive1 data1.csv" read_archive_file "${FILE_PREFIX}_archive1.$1 :: ${FILE_PREFIX}_data1.csv" @@ -44,10 +42,10 @@ function run_archive_test() { echo "archive* {2..3}.csv" read_archive_file "${FILE_PREFIX}_archive*.$1 :: ${FILE_PREFIX}_data{2..3}.csv" - $CLICKHOUSE_LOCAL --query "SELECT * FROM file('${user_files_path}/${FILE_PREFIX}_archive1.$1::nonexistent.csv')" 2>&1 | grep -q "CANNOT_EXTRACT_TABLE_STRUCTURE" && echo "OK" || echo "FAIL" - $CLICKHOUSE_LOCAL --query "SELECT * FROM file('${user_files_path}/${FILE_PREFIX}_archive3.$1::{2..3}.csv')" 2>&1 | grep -q "CANNOT_EXTRACT_TABLE_STRUCTURE" && echo "OK" || echo "FAIL" + $CLICKHOUSE_LOCAL --query "SELECT * FROM file('${USER_FILES_PATH}/${FILE_PREFIX}_archive1.$1::nonexistent.csv')" 2>&1 | grep -q "CANNOT_EXTRACT_TABLE_STRUCTURE" && echo "OK" || echo "FAIL" + $CLICKHOUSE_LOCAL --query "SELECT * FROM file('${USER_FILES_PATH}/${FILE_PREFIX}_archive3.$1::{2..3}.csv')" 2>&1 | grep -q "CANNOT_EXTRACT_TABLE_STRUCTURE" && echo "OK" || echo "FAIL" - rm ${user_files_path}/${FILE_PREFIX}_archive{1..3}.$1 + rm ${USER_FILES_PATH}/${FILE_PREFIX}_archive{1..3}.$1 rm ${FILE_PREFIX}_data{0..3}.csv } diff --git a/tests/queries/0_stateless/02703_keeper_map_concurrent_create_drop.sh b/tests/queries/0_stateless/02703_keeper_map_concurrent_create_drop.sh index 17d1fa92377..3e629ece33f 100755 --- a/tests/queries/0_stateless/02703_keeper_map_concurrent_create_drop.sh +++ b/tests/queries/0_stateless/02703_keeper_map_concurrent_create_drop.sh @@ -1,5 +1,5 @@ #!/usr/bin/env bash -# Tags: no-ordinary-database, zookeeper, no-fasttest, no-parallel +# Tags: no-ordinary-database, zookeeper, no-fasttest CURDIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd) # shellcheck source=../shell_config.sh @@ -37,7 +37,7 @@ function create_drop_loop() export -f create_drop_loop; THREADS=10 -TIMEOUT=30 +TIMEOUT=20 for i in `seq $THREADS` do diff --git a/tests/queries/0_stateless/02722_database_filesystem.sh b/tests/queries/0_stateless/02722_database_filesystem.sh index 374dd246c96..2d0ff256c95 100755 --- a/tests/queries/0_stateless/02722_database_filesystem.sh +++ b/tests/queries/0_stateless/02722_database_filesystem.sh @@ -5,12 +5,9 @@ CURDIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd) # shellcheck source=../shell_config.sh . "$CURDIR"/../shell_config.sh -# see 01658_read_file_to_stringcolumn.sh -CLICKHOUSE_USER_FILES_PATH=$($CLICKHOUSE_CLIENT_BINARY --query "select _path, _file from file('nonexist.txt', 'CSV', 'val1 char')" 2>&1 | grep Exception | awk '{gsub("/nonexist.txt","",$9); print $9}') - # Prepare data unique_name=${CLICKHOUSE_TEST_UNIQUE_NAME} -user_files_tmp_dir=${CLICKHOUSE_USER_FILES_PATH}/${unique_name} +user_files_tmp_dir=${USER_FILES_PATH}/${unique_name} mkdir -p ${user_files_tmp_dir}/tmp/ echo '"id","str","int","text"' > ${user_files_tmp_dir}/tmp.csv echo '1,"abc",123,"abacaba"' >> ${user_files_tmp_dir}/tmp.csv diff --git a/tests/queries/0_stateless/02724_decompress_filename_exception.sh b/tests/queries/0_stateless/02724_decompress_filename_exception.sh index e413910b934..8b5a2f23aa9 100755 --- a/tests/queries/0_stateless/02724_decompress_filename_exception.sh +++ b/tests/queries/0_stateless/02724_decompress_filename_exception.sh @@ -5,7 +5,6 @@ CURDIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd) # shellcheck source=../shell_config.sh . "$CURDIR"/../shell_config.sh -USER_FILES_PATH=$($CLICKHOUSE_CLIENT_BINARY --query "select _path,_file from file('nonexist.txt', 'CSV', 'val1 char')" 2>&1 | grep Exception | awk '{gsub("/nonexist.txt","",$9); print $9}') FILENAME="${USER_FILES_PATH}/corrupted_file.tsv.xx" echo 'corrupted file' > $FILENAME; diff --git a/tests/queries/0_stateless/02732_rename_after_processing.sh b/tests/queries/0_stateless/02732_rename_after_processing.sh index 9d44ff9fc34..c3f2274570e 100755 --- a/tests/queries/0_stateless/02732_rename_after_processing.sh +++ b/tests/queries/0_stateless/02732_rename_after_processing.sh @@ -4,12 +4,9 @@ CURDIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd) # shellcheck source=../shell_config.sh . "$CURDIR"/../shell_config.sh -# see 01658_read_file_to_stringcolumn.sh -CLICKHOUSE_USER_FILES_PATH=$($CLICKHOUSE_CLIENT_BINARY --query "select _path, _file from file('nonexist.txt', 'CSV', 'val1 char')" 2>&1 | grep Exception | awk '{gsub("/nonexist.txt","",$9); print $9}') - # Prepare data unique_name=${CLICKHOUSE_TEST_UNIQUE_NAME} -tmp_dir=${CLICKHOUSE_USER_FILES_PATH}/${unique_name} +tmp_dir=${USER_FILES_PATH}/${unique_name} mkdir -p $tmp_dir rm -rf ${tmp_dir:?}/* diff --git a/tests/queries/0_stateless/02771_multidirectory_globs_storage_file.sh b/tests/queries/0_stateless/02771_multidirectory_globs_storage_file.sh index 932837b83db..60df3ed2762 100755 --- a/tests/queries/0_stateless/02771_multidirectory_globs_storage_file.sh +++ b/tests/queries/0_stateless/02771_multidirectory_globs_storage_file.sh @@ -7,27 +7,22 @@ CURDIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd) # shellcheck source=../shell_config.sh . "$CURDIR"/../shell_config.sh -# Data preparation. -# Now we can get the user_files_path by using the file function, we can also get it by this query: -# "insert into function file('exist.txt', 'CSV', 'val1 char') values ('aaaa'); select _path from file('exist.txt', 'CSV', 'val1 char')" -user_files_path=$($CLICKHOUSE_CLIENT_BINARY --query "select _path,_file from file('nonexist.txt', 'CSV', 'val1 char')" 2>&1 | grep Exception | awk '{gsub("/nonexist.txt","",$9); print $9}') +rm -rf ${USER_FILES_PATH:?}/${CLICKHOUSE_TEST_UNIQUE_NAME:?}/* +mkdir -p ${USER_FILES_PATH}/${CLICKHOUSE_TEST_UNIQUE_NAME}/ -rm -rf ${user_files_path:?}/${CLICKHOUSE_TEST_UNIQUE_NAME:?}/* -mkdir -p ${user_files_path}/${CLICKHOUSE_TEST_UNIQUE_NAME}/ - -${CLICKHOUSE_CLIENT} --query "SELECT *, _file FROM file('${user_files_path}/${CLICKHOUSE_TEST_UNIQUE_NAME}/dir{?/subdir?1/da,2/subdir2?/da}ta/non_existing.csv', CSV);" 2>&1 | grep -q "CANNOT_EXTRACT_TABLE_STRUCTURE" && echo 'OK' || echo 'FAIL' +${CLICKHOUSE_CLIENT} --query "SELECT *, _file FROM file('${USER_FILES_PATH}/${CLICKHOUSE_TEST_UNIQUE_NAME}/dir{?/subdir?1/da,2/subdir2?/da}ta/non_existing.csv', CSV);" 2>&1 | grep -q "CANNOT_EXTRACT_TABLE_STRUCTURE" && echo 'OK' || echo 'FAIL' # Create two files in different directories -mkdir -p ${user_files_path}/${CLICKHOUSE_TEST_UNIQUE_NAME}/dir1/subdir11/ -mkdir -p ${user_files_path}/${CLICKHOUSE_TEST_UNIQUE_NAME}/dir2/subdir22/ +mkdir -p ${USER_FILES_PATH}/${CLICKHOUSE_TEST_UNIQUE_NAME}/dir1/subdir11/ +mkdir -p ${USER_FILES_PATH}/${CLICKHOUSE_TEST_UNIQUE_NAME}/dir2/subdir22/ -echo 'This is file data1' > ${user_files_path}/${CLICKHOUSE_TEST_UNIQUE_NAME}/dir1/subdir11/data1.csv -echo 'This is file data2' > ${user_files_path}/${CLICKHOUSE_TEST_UNIQUE_NAME}/dir2/subdir22/data2.csv +echo 'This is file data1' > ${USER_FILES_PATH}/${CLICKHOUSE_TEST_UNIQUE_NAME}/dir1/subdir11/data1.csv +echo 'This is file data2' > ${USER_FILES_PATH}/${CLICKHOUSE_TEST_UNIQUE_NAME}/dir2/subdir22/data2.csv -${CLICKHOUSE_CLIENT} --query "SELECT *, _file FROM file('${user_files_path}/${CLICKHOUSE_TEST_UNIQUE_NAME}/dir{?/subdir?1/da,2/subdir2?/da}ta1.csv', CSV);" -${CLICKHOUSE_CLIENT} --query "SELECT *, _file FROM file('${user_files_path}/${CLICKHOUSE_TEST_UNIQUE_NAME}/dir{?/subdir?1/da,2/subdir2?/da}ta2.csv', CSV);" +${CLICKHOUSE_CLIENT} --query "SELECT *, _file FROM file('${USER_FILES_PATH}/${CLICKHOUSE_TEST_UNIQUE_NAME}/dir{?/subdir?1/da,2/subdir2?/da}ta1.csv', CSV);" +${CLICKHOUSE_CLIENT} --query "SELECT *, _file FROM file('${USER_FILES_PATH}/${CLICKHOUSE_TEST_UNIQUE_NAME}/dir{?/subdir?1/da,2/subdir2?/da}ta2.csv', CSV);" -${CLICKHOUSE_CLIENT} --query "SELECT *, _file FROM file('${user_files_path}/${CLICKHOUSE_TEST_UNIQUE_NAME}/dir?/{subdir?1/data1,subdir2?/data2}.csv', CSV) WHERE _file == 'data1.csv';" -${CLICKHOUSE_CLIENT} --query "SELECT *, _file FROM file('${user_files_path}/${CLICKHOUSE_TEST_UNIQUE_NAME}/dir?/{subdir?1/data1,subdir2?/data2}.csv', CSV) WHERE _file == 'data2.csv';" +${CLICKHOUSE_CLIENT} --query "SELECT *, _file FROM file('${USER_FILES_PATH}/${CLICKHOUSE_TEST_UNIQUE_NAME}/dir?/{subdir?1/data1,subdir2?/data2}.csv', CSV) WHERE _file == 'data1.csv';" +${CLICKHOUSE_CLIENT} --query "SELECT *, _file FROM file('${USER_FILES_PATH}/${CLICKHOUSE_TEST_UNIQUE_NAME}/dir?/{subdir?1/data1,subdir2?/data2}.csv', CSV) WHERE _file == 'data2.csv';" -rm -rf ${user_files_path:?}/${CLICKHOUSE_TEST_UNIQUE_NAME:?} +rm -rf ${USER_FILES_PATH:?}/${CLICKHOUSE_TEST_UNIQUE_NAME:?} diff --git a/tests/queries/0_stateless/02889_file_log_save_errors.sh b/tests/queries/0_stateless/02889_file_log_save_errors.sh index 8ef7816d57d..cf7ced0bd08 100755 --- a/tests/queries/0_stateless/02889_file_log_save_errors.sh +++ b/tests/queries/0_stateless/02889_file_log_save_errors.sh @@ -4,27 +4,25 @@ CURDIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd) # shellcheck source=../shell_config.sh . "$CURDIR"/../shell_config.sh -user_files_path=$($CLICKHOUSE_CLIENT_BINARY --query "select _path,_file from file('nonexist.txt', 'CSV', 'val1 char')" 2>&1 | grep Exception | awk '{gsub("/nonexist.txt","",$9); print $9}') - ${CLICKHOUSE_CLIENT} --query "drop table if exists file_log;" ${CLICKHOUSE_CLIENT} --query "drop table if exists log_errors;" -mkdir -p ${user_files_path}/${CLICKHOUSE_TEST_UNIQUE_NAME}/ -rm -rf ${user_files_path}/${CLICKHOUSE_TEST_UNIQUE_NAME:?}/* +mkdir -p ${USER_FILES_PATH}/${CLICKHOUSE_TEST_UNIQUE_NAME}/ +rm -rf ${USER_FILES_PATH}/${CLICKHOUSE_TEST_UNIQUE_NAME:?}/* for i in {0..9} do - echo "{\"key\" : $i, \"value\" : $i}" >> ${user_files_path}/${CLICKHOUSE_TEST_UNIQUE_NAME}/a.jsonl - echo "Error $i" >> ${user_files_path}/${CLICKHOUSE_TEST_UNIQUE_NAME}/a.jsonl + echo "{\"key\" : $i, \"value\" : $i}" >> ${USER_FILES_PATH}/${CLICKHOUSE_TEST_UNIQUE_NAME}/a.jsonl + echo "Error $i" >> ${USER_FILES_PATH}/${CLICKHOUSE_TEST_UNIQUE_NAME}/a.jsonl done for i in {10..19} do - echo "{\"key\" : $i, \"value\" : $i}" >> ${user_files_path}/${CLICKHOUSE_TEST_UNIQUE_NAME}/b.jsonl - echo "Error $i" >> ${user_files_path}/${CLICKHOUSE_TEST_UNIQUE_NAME}/b.jsonl + echo "{\"key\" : $i, \"value\" : $i}" >> ${USER_FILES_PATH}/${CLICKHOUSE_TEST_UNIQUE_NAME}/b.jsonl + echo "Error $i" >> ${USER_FILES_PATH}/${CLICKHOUSE_TEST_UNIQUE_NAME}/b.jsonl done -${CLICKHOUSE_CLIENT} --query "create table file_log(key UInt8, value UInt8) engine=FileLog('${user_files_path}/${CLICKHOUSE_TEST_UNIQUE_NAME}/', 'JSONEachRow') settings handle_error_mode='stream';" +${CLICKHOUSE_CLIENT} --query "create table file_log(key UInt8, value UInt8) engine=FileLog('${USER_FILES_PATH}/${CLICKHOUSE_TEST_UNIQUE_NAME}/', 'JSONEachRow') settings handle_error_mode='stream';" ${CLICKHOUSE_CLIENT} --query "create Materialized View log_errors engine=MergeTree order by tuple() as select _error as error, _raw_record as record, _filename as file from file_log where not isNull(_error);" function count() @@ -42,4 +40,4 @@ ${CLICKHOUSE_CLIENT} --query "select * from log_errors order by file, record;" ${CLICKHOUSE_CLIENT} --query "drop table file_log;" ${CLICKHOUSE_CLIENT} --query "drop table log_errors;" -rm -rf ${user_files_path}/${CLICKHOUSE_TEST_UNIQUE_NAME:?} +rm -rf ${USER_FILES_PATH}/${CLICKHOUSE_TEST_UNIQUE_NAME:?} diff --git a/tests/queries/0_stateless/02892_input_csv_cr_end_count_many_rows.sh b/tests/queries/0_stateless/02892_input_csv_cr_end_count_many_rows.sh index 42dde18de00..9f93396e368 100755 --- a/tests/queries/0_stateless/02892_input_csv_cr_end_count_many_rows.sh +++ b/tests/queries/0_stateless/02892_input_csv_cr_end_count_many_rows.sh @@ -6,11 +6,9 @@ CURDIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd) # shellcheck source=../shell_config.sh . "$CURDIR"/../shell_config.sh -USER_FILES_PATH=$($CLICKHOUSE_CLIENT --query "select _path,_file from file('nonexist.txt', 'CSV', 'val1 char')" 2>&1 | grep -E '^Code: 107.*FILE_DOESNT_EXIST' | head -1 | awk '{gsub("/nonexist.txt","",$9); print $9}') - cp "$CURDIR"/data_csv/1m_rows_cr_end_of_line.csv.xz $USER_FILES_PATH/ $CLICKHOUSE_CLIENT -q "SELECT count(1) from file('1m_rows_cr_end_of_line.csv.xz') settings input_format_csv_allow_cr_end_of_line=1, optimize_count_from_files=1" $CLICKHOUSE_CLIENT -q "SELECT count(1) from file('1m_rows_cr_end_of_line.csv.xz') settings input_format_csv_allow_cr_end_of_line=1, optimize_count_from_files=0" -rm $USER_FILES_PATH/1m_rows_cr_end_of_line.csv.xz \ No newline at end of file +rm $USER_FILES_PATH/1m_rows_cr_end_of_line.csv.xz diff --git a/tests/queries/0_stateless/02895_npy_output_format.sh b/tests/queries/0_stateless/02895_npy_output_format.sh index 934c80830c5..a364e447062 100755 --- a/tests/queries/0_stateless/02895_npy_output_format.sh +++ b/tests/queries/0_stateless/02895_npy_output_format.sh @@ -5,10 +5,9 @@ CURDIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd) # shellcheck source=../shell_config.sh . "$CURDIR"/../shell_config.sh -user_files_path=$($CLICKHOUSE_CLIENT_BINARY -q "select _path,_file from file('nonexist.txt', 'CSV', 'val1 char')" 2>&1 | grep Exception | awk '{gsub("/nonexist.txt","",$9); print $9}') -mkdir -p ${user_files_path}/${CLICKHOUSE_TEST_UNIQUE_NAME}/ -rm -rf ${user_files_path}/${CLICKHOUSE_TEST_UNIQUE_NAME:?}/* -chmod 777 ${user_files_path}/${CLICKHOUSE_TEST_UNIQUE_NAME}/ +mkdir -p ${USER_FILES_PATH}/${CLICKHOUSE_TEST_UNIQUE_NAME}/ +rm -rf ${USER_FILES_PATH}/${CLICKHOUSE_TEST_UNIQUE_NAME:?}/* +chmod 777 ${USER_FILES_PATH}/${CLICKHOUSE_TEST_UNIQUE_NAME}/ ${CLICKHOUSE_CLIENT} -n -q --ignore-error " DROP DATABASE IF EXISTS npy_output_02895; @@ -33,43 +32,43 @@ ${CLICKHOUSE_CLIENT} -n -q --ignore-error " INSERT INTO npy_output_02895.data_types VALUES (1, 1, 1, 1, 1, 1, 1, 1, 0.1, 0.01, 'npy', 'npy'), (-1, -1, -1, -1, 0, 0, 0, 0, 0.2, 0.02, 'npy', 'npynpy'); - INSERT INTO TABLE FUNCTION file('${user_files_path}/${CLICKHOUSE_TEST_UNIQUE_NAME}/02895_dtype_int8.npy') SELECT i1 FROM npy_output_02895.data_types; - INSERT INTO TABLE FUNCTION file('${user_files_path}/${CLICKHOUSE_TEST_UNIQUE_NAME}/02895_dtype_int16.npy') SELECT i2 FROM npy_output_02895.data_types; - INSERT INTO TABLE FUNCTION file('${user_files_path}/${CLICKHOUSE_TEST_UNIQUE_NAME}/02895_dtype_int32.npy') SELECT i4 FROM npy_output_02895.data_types; - INSERT INTO TABLE FUNCTION file('${user_files_path}/${CLICKHOUSE_TEST_UNIQUE_NAME}/02895_dtype_int64.npy') SELECT i8 FROM npy_output_02895.data_types; - INSERT INTO TABLE FUNCTION file('${user_files_path}/${CLICKHOUSE_TEST_UNIQUE_NAME}/02895_dtype_uint8.npy') SELECT u1 FROM npy_output_02895.data_types; - INSERT INTO TABLE FUNCTION file('${user_files_path}/${CLICKHOUSE_TEST_UNIQUE_NAME}/02895_dtype_uint16.npy') SELECT u2 FROM npy_output_02895.data_types; - INSERT INTO TABLE FUNCTION file('${user_files_path}/${CLICKHOUSE_TEST_UNIQUE_NAME}/02895_dtype_uint32.npy') SELECT u4 FROM npy_output_02895.data_types; - INSERT INTO TABLE FUNCTION file('${user_files_path}/${CLICKHOUSE_TEST_UNIQUE_NAME}/02895_dtype_uint64.npy') SELECT u8 FROM npy_output_02895.data_types; - INSERT INTO TABLE FUNCTION file('${user_files_path}/${CLICKHOUSE_TEST_UNIQUE_NAME}/02895_dtype_float32.npy') SELECT f4 FROM npy_output_02895.data_types; - INSERT INTO TABLE FUNCTION file('${user_files_path}/${CLICKHOUSE_TEST_UNIQUE_NAME}/02895_dtype_float64.npy') SELECT f8 FROM npy_output_02895.data_types; - INSERT INTO TABLE FUNCTION file('${user_files_path}/${CLICKHOUSE_TEST_UNIQUE_NAME}/02895_dtype_fixedstring.npy') SELECT fs FROM npy_output_02895.data_types; - INSERT INTO TABLE FUNCTION file('${user_files_path}/${CLICKHOUSE_TEST_UNIQUE_NAME}/02895_dtype_string.npy') SELECT s FROM npy_output_02895.data_types; + INSERT INTO TABLE FUNCTION file('${USER_FILES_PATH}/${CLICKHOUSE_TEST_UNIQUE_NAME}/02895_dtype_int8.npy') SELECT i1 FROM npy_output_02895.data_types; + INSERT INTO TABLE FUNCTION file('${USER_FILES_PATH}/${CLICKHOUSE_TEST_UNIQUE_NAME}/02895_dtype_int16.npy') SELECT i2 FROM npy_output_02895.data_types; + INSERT INTO TABLE FUNCTION file('${USER_FILES_PATH}/${CLICKHOUSE_TEST_UNIQUE_NAME}/02895_dtype_int32.npy') SELECT i4 FROM npy_output_02895.data_types; + INSERT INTO TABLE FUNCTION file('${USER_FILES_PATH}/${CLICKHOUSE_TEST_UNIQUE_NAME}/02895_dtype_int64.npy') SELECT i8 FROM npy_output_02895.data_types; + INSERT INTO TABLE FUNCTION file('${USER_FILES_PATH}/${CLICKHOUSE_TEST_UNIQUE_NAME}/02895_dtype_uint8.npy') SELECT u1 FROM npy_output_02895.data_types; + INSERT INTO TABLE FUNCTION file('${USER_FILES_PATH}/${CLICKHOUSE_TEST_UNIQUE_NAME}/02895_dtype_uint16.npy') SELECT u2 FROM npy_output_02895.data_types; + INSERT INTO TABLE FUNCTION file('${USER_FILES_PATH}/${CLICKHOUSE_TEST_UNIQUE_NAME}/02895_dtype_uint32.npy') SELECT u4 FROM npy_output_02895.data_types; + INSERT INTO TABLE FUNCTION file('${USER_FILES_PATH}/${CLICKHOUSE_TEST_UNIQUE_NAME}/02895_dtype_uint64.npy') SELECT u8 FROM npy_output_02895.data_types; + INSERT INTO TABLE FUNCTION file('${USER_FILES_PATH}/${CLICKHOUSE_TEST_UNIQUE_NAME}/02895_dtype_float32.npy') SELECT f4 FROM npy_output_02895.data_types; + INSERT INTO TABLE FUNCTION file('${USER_FILES_PATH}/${CLICKHOUSE_TEST_UNIQUE_NAME}/02895_dtype_float64.npy') SELECT f8 FROM npy_output_02895.data_types; + INSERT INTO TABLE FUNCTION file('${USER_FILES_PATH}/${CLICKHOUSE_TEST_UNIQUE_NAME}/02895_dtype_fixedstring.npy') SELECT fs FROM npy_output_02895.data_types; + INSERT INTO TABLE FUNCTION file('${USER_FILES_PATH}/${CLICKHOUSE_TEST_UNIQUE_NAME}/02895_dtype_string.npy') SELECT s FROM npy_output_02895.data_types; - SELECT * FROM file('${user_files_path}/${CLICKHOUSE_TEST_UNIQUE_NAME}/02895_dtype_int8.npy'); - SELECT * FROM file('${user_files_path}/${CLICKHOUSE_TEST_UNIQUE_NAME}/02895_dtype_int16.npy'); - SELECT * FROM file('${user_files_path}/${CLICKHOUSE_TEST_UNIQUE_NAME}/02895_dtype_int32.npy'); - SELECT * FROM file('${user_files_path}/${CLICKHOUSE_TEST_UNIQUE_NAME}/02895_dtype_int64.npy'); - SELECT * FROM file('${user_files_path}/${CLICKHOUSE_TEST_UNIQUE_NAME}/02895_dtype_uint8.npy'); - SELECT * FROM file('${user_files_path}/${CLICKHOUSE_TEST_UNIQUE_NAME}/02895_dtype_uint16.npy'); - SELECT * FROM file('${user_files_path}/${CLICKHOUSE_TEST_UNIQUE_NAME}/02895_dtype_uint32.npy'); - SELECT * FROM file('${user_files_path}/${CLICKHOUSE_TEST_UNIQUE_NAME}/02895_dtype_uint64.npy'); - SELECT * FROM file('${user_files_path}/${CLICKHOUSE_TEST_UNIQUE_NAME}/02895_dtype_float32.npy'); - SELECT * FROM file('${user_files_path}/${CLICKHOUSE_TEST_UNIQUE_NAME}/02895_dtype_float64.npy'); - SELECT * FROM file('${user_files_path}/${CLICKHOUSE_TEST_UNIQUE_NAME}/02895_dtype_fixedstring.npy'); - SELECT * FROM file('${user_files_path}/${CLICKHOUSE_TEST_UNIQUE_NAME}/02895_dtype_string.npy'); - DESC file('${user_files_path}/${CLICKHOUSE_TEST_UNIQUE_NAME}/02895_dtype_int8.npy'); - DESC file('${user_files_path}/${CLICKHOUSE_TEST_UNIQUE_NAME}/02895_dtype_int16.npy'); - DESC file('${user_files_path}/${CLICKHOUSE_TEST_UNIQUE_NAME}/02895_dtype_int32.npy'); - DESC file('${user_files_path}/${CLICKHOUSE_TEST_UNIQUE_NAME}/02895_dtype_int64.npy'); - DESC file('${user_files_path}/${CLICKHOUSE_TEST_UNIQUE_NAME}/02895_dtype_uint8.npy'); - DESC file('${user_files_path}/${CLICKHOUSE_TEST_UNIQUE_NAME}/02895_dtype_uint16.npy'); - DESC file('${user_files_path}/${CLICKHOUSE_TEST_UNIQUE_NAME}/02895_dtype_uint32.npy'); - DESC file('${user_files_path}/${CLICKHOUSE_TEST_UNIQUE_NAME}/02895_dtype_uint64.npy'); - DESC file('${user_files_path}/${CLICKHOUSE_TEST_UNIQUE_NAME}/02895_dtype_float32.npy'); - DESC file('${user_files_path}/${CLICKHOUSE_TEST_UNIQUE_NAME}/02895_dtype_float64.npy'); - DESC file('${user_files_path}/${CLICKHOUSE_TEST_UNIQUE_NAME}/02895_dtype_fixedstring.npy'); - DESC file('${user_files_path}/${CLICKHOUSE_TEST_UNIQUE_NAME}/02895_dtype_string.npy'); + SELECT * FROM file('${USER_FILES_PATH}/${CLICKHOUSE_TEST_UNIQUE_NAME}/02895_dtype_int8.npy'); + SELECT * FROM file('${USER_FILES_PATH}/${CLICKHOUSE_TEST_UNIQUE_NAME}/02895_dtype_int16.npy'); + SELECT * FROM file('${USER_FILES_PATH}/${CLICKHOUSE_TEST_UNIQUE_NAME}/02895_dtype_int32.npy'); + SELECT * FROM file('${USER_FILES_PATH}/${CLICKHOUSE_TEST_UNIQUE_NAME}/02895_dtype_int64.npy'); + SELECT * FROM file('${USER_FILES_PATH}/${CLICKHOUSE_TEST_UNIQUE_NAME}/02895_dtype_uint8.npy'); + SELECT * FROM file('${USER_FILES_PATH}/${CLICKHOUSE_TEST_UNIQUE_NAME}/02895_dtype_uint16.npy'); + SELECT * FROM file('${USER_FILES_PATH}/${CLICKHOUSE_TEST_UNIQUE_NAME}/02895_dtype_uint32.npy'); + SELECT * FROM file('${USER_FILES_PATH}/${CLICKHOUSE_TEST_UNIQUE_NAME}/02895_dtype_uint64.npy'); + SELECT * FROM file('${USER_FILES_PATH}/${CLICKHOUSE_TEST_UNIQUE_NAME}/02895_dtype_float32.npy'); + SELECT * FROM file('${USER_FILES_PATH}/${CLICKHOUSE_TEST_UNIQUE_NAME}/02895_dtype_float64.npy'); + SELECT * FROM file('${USER_FILES_PATH}/${CLICKHOUSE_TEST_UNIQUE_NAME}/02895_dtype_fixedstring.npy'); + SELECT * FROM file('${USER_FILES_PATH}/${CLICKHOUSE_TEST_UNIQUE_NAME}/02895_dtype_string.npy'); + DESC file('${USER_FILES_PATH}/${CLICKHOUSE_TEST_UNIQUE_NAME}/02895_dtype_int8.npy'); + DESC file('${USER_FILES_PATH}/${CLICKHOUSE_TEST_UNIQUE_NAME}/02895_dtype_int16.npy'); + DESC file('${USER_FILES_PATH}/${CLICKHOUSE_TEST_UNIQUE_NAME}/02895_dtype_int32.npy'); + DESC file('${USER_FILES_PATH}/${CLICKHOUSE_TEST_UNIQUE_NAME}/02895_dtype_int64.npy'); + DESC file('${USER_FILES_PATH}/${CLICKHOUSE_TEST_UNIQUE_NAME}/02895_dtype_uint8.npy'); + DESC file('${USER_FILES_PATH}/${CLICKHOUSE_TEST_UNIQUE_NAME}/02895_dtype_uint16.npy'); + DESC file('${USER_FILES_PATH}/${CLICKHOUSE_TEST_UNIQUE_NAME}/02895_dtype_uint32.npy'); + DESC file('${USER_FILES_PATH}/${CLICKHOUSE_TEST_UNIQUE_NAME}/02895_dtype_uint64.npy'); + DESC file('${USER_FILES_PATH}/${CLICKHOUSE_TEST_UNIQUE_NAME}/02895_dtype_float32.npy'); + DESC file('${USER_FILES_PATH}/${CLICKHOUSE_TEST_UNIQUE_NAME}/02895_dtype_float64.npy'); + DESC file('${USER_FILES_PATH}/${CLICKHOUSE_TEST_UNIQUE_NAME}/02895_dtype_fixedstring.npy'); + DESC file('${USER_FILES_PATH}/${CLICKHOUSE_TEST_UNIQUE_NAME}/02895_dtype_string.npy'); SELECT '-- test nested data types --'; CREATE TABLE IF NOT EXISTS npy_output_02895.nested_data_types @@ -81,16 +80,16 @@ ${CLICKHOUSE_CLIENT} -n -q --ignore-error " INSERT INTO npy_output_02895.nested_data_types VALUES ([[[1], [2]], [[3], [4]]], [[0.1], [0.2]], ['a', 'bb']), ([[[1], [2]], [[3], [4]]], [[0.1], [0.2]], ['ccc', 'dddd']); - INSERT INTO TABLE FUNCTION file('${user_files_path}/${CLICKHOUSE_TEST_UNIQUE_NAME}/02895_nested_dtype_int32.npy') SELECT i4 FROM npy_output_02895.nested_data_types; - INSERT INTO TABLE FUNCTION file('${user_files_path}/${CLICKHOUSE_TEST_UNIQUE_NAME}/02895_nested_dtype_float64.npy') SELECT f8 FROM npy_output_02895.nested_data_types; - INSERT INTO TABLE FUNCTION file('${user_files_path}/${CLICKHOUSE_TEST_UNIQUE_NAME}/02895_nested_dtype_string.npy') SELECT s FROM npy_output_02895.nested_data_types; + INSERT INTO TABLE FUNCTION file('${USER_FILES_PATH}/${CLICKHOUSE_TEST_UNIQUE_NAME}/02895_nested_dtype_int32.npy') SELECT i4 FROM npy_output_02895.nested_data_types; + INSERT INTO TABLE FUNCTION file('${USER_FILES_PATH}/${CLICKHOUSE_TEST_UNIQUE_NAME}/02895_nested_dtype_float64.npy') SELECT f8 FROM npy_output_02895.nested_data_types; + INSERT INTO TABLE FUNCTION file('${USER_FILES_PATH}/${CLICKHOUSE_TEST_UNIQUE_NAME}/02895_nested_dtype_string.npy') SELECT s FROM npy_output_02895.nested_data_types; - SELECT * FROM file('${user_files_path}/${CLICKHOUSE_TEST_UNIQUE_NAME}/02895_nested_dtype_int32.npy'); - SELECT * FROM file('${user_files_path}/${CLICKHOUSE_TEST_UNIQUE_NAME}/02895_nested_dtype_float64.npy'); - SELECT * FROM file('${user_files_path}/${CLICKHOUSE_TEST_UNIQUE_NAME}/02895_nested_dtype_string.npy'); - DESC file('${user_files_path}/${CLICKHOUSE_TEST_UNIQUE_NAME}/02895_nested_dtype_int32.npy'); - DESC file('${user_files_path}/${CLICKHOUSE_TEST_UNIQUE_NAME}/02895_nested_dtype_float64.npy'); - DESC file('${user_files_path}/${CLICKHOUSE_TEST_UNIQUE_NAME}/02895_nested_dtype_string.npy'); + SELECT * FROM file('${USER_FILES_PATH}/${CLICKHOUSE_TEST_UNIQUE_NAME}/02895_nested_dtype_int32.npy'); + SELECT * FROM file('${USER_FILES_PATH}/${CLICKHOUSE_TEST_UNIQUE_NAME}/02895_nested_dtype_float64.npy'); + SELECT * FROM file('${USER_FILES_PATH}/${CLICKHOUSE_TEST_UNIQUE_NAME}/02895_nested_dtype_string.npy'); + DESC file('${USER_FILES_PATH}/${CLICKHOUSE_TEST_UNIQUE_NAME}/02895_nested_dtype_int32.npy'); + DESC file('${USER_FILES_PATH}/${CLICKHOUSE_TEST_UNIQUE_NAME}/02895_nested_dtype_float64.npy'); + DESC file('${USER_FILES_PATH}/${CLICKHOUSE_TEST_UNIQUE_NAME}/02895_nested_dtype_string.npy'); SELECT '-- test exceptions --'; CREATE TABLE IF NOT EXISTS npy_output_02895.exceptions @@ -115,4 +114,4 @@ ${CLICKHOUSE_CLIENT} -n -q --ignore-error " DROP DATABASE IF EXISTS npy_output_02895;" -rm -rf ${user_files_path}/${CLICKHOUSE_TEST_UNIQUE_NAME:?} +rm -rf ${USER_FILES_PATH}/${CLICKHOUSE_TEST_UNIQUE_NAME:?} diff --git a/tests/queries/0_stateless/02931_file_cluster.sh b/tests/queries/0_stateless/02931_file_cluster.sh index 8566e2ab08e..ebd3792e1dc 100755 --- a/tests/queries/0_stateless/02931_file_cluster.sh +++ b/tests/queries/0_stateless/02931_file_cluster.sh @@ -4,8 +4,6 @@ CURDIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd) # shellcheck source=../shell_config.sh . "$CURDIR"/../shell_config.sh -USER_FILES_PATH=$($CLICKHOUSE_CLIENT_BINARY --query "select _path,_file from file('nonexist.txt', 'CSV', 'val1 char')" 2>&1 | grep Exception | awk '{gsub("/nonexist.txt","",$9); print $9}') - mkdir -p "${USER_FILES_PATH}"/"${CLICKHOUSE_TEST_UNIQUE_NAME}"/ for i in {1..10} diff --git a/tests/queries/0_stateless/02933_change_cache_setting_without_restart.sh b/tests/queries/0_stateless/02933_change_cache_setting_without_restart.sh index 76ada756f47..2e5a538007c 100755 --- a/tests/queries/0_stateless/02933_change_cache_setting_without_restart.sh +++ b/tests/queries/0_stateless/02933_change_cache_setting_without_restart.sh @@ -8,7 +8,7 @@ CUR_DIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd) disk_name="s3_cache_02933" $CLICKHOUSE_CLIENT --query "DESCRIBE FILESYSTEM CACHE '${disk_name}'" -config_path=/etc/clickhouse-server/config.d/storage_conf.xml +config_path=${CLICKHOUSE_CONFIG_DIR}/config.d/storage_conf.xml config_path_tmp=$config_path.tmp cat $config_path \ diff --git a/tests/queries/0_stateless/02944_dynamically_change_filesystem_cache_size.sh b/tests/queries/0_stateless/02944_dynamically_change_filesystem_cache_size.sh index 6f454da40da..cb099bb59ae 100755 --- a/tests/queries/0_stateless/02944_dynamically_change_filesystem_cache_size.sh +++ b/tests/queries/0_stateless/02944_dynamically_change_filesystem_cache_size.sh @@ -24,7 +24,7 @@ $CLICKHOUSE_CLIENT --query "SELECT * FROM test FORMAT Null" $CLICKHOUSE_CLIENT --query "SELECT count() FROM system.filesystem_cache WHERE state = 'DOWNLOADED'" $CLICKHOUSE_CLIENT --query "SELECT sum(size) FROM system.filesystem_cache WHERE state = 'DOWNLOADED'" -config_path=/etc/clickhouse-server/config.d/storage_conf_02944.xml +config_path=${CLICKHOUSE_CONFIG_DIR}/config.d/storage_conf_02944.xml config_path_tmp=$config_path.tmp echo 'set max_size from 100 to 10' diff --git a/tests/queries/0_stateless/02950_dictionary_ssd_cache_short_circuit.sh b/tests/queries/0_stateless/02950_dictionary_ssd_cache_short_circuit.sh index a02bdd0a1d2..3d2fe5d664d 100755 --- a/tests/queries/0_stateless/02950_dictionary_ssd_cache_short_circuit.sh +++ b/tests/queries/0_stateless/02950_dictionary_ssd_cache_short_circuit.sh @@ -5,8 +5,6 @@ CURDIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd) # shellcheck source=../shell_config.sh . "$CURDIR"/../shell_config.sh -USER_FILES_PATH=$($CLICKHOUSE_CLIENT_BINARY --query "select _path,_file from file('nonexist.txt', 'CSV', 'val1 char')" 2>&1 | grep Exception | awk '{gsub("/nonexist.txt","",$9); print $9}') - $CLICKHOUSE_CLIENT -n --query=" DROP DATABASE IF EXISTS 02950_database_for_ssd_cache_dictionary; CREATE DATABASE 02950_database_for_ssd_cache_dictionary; @@ -32,7 +30,7 @@ $CLICKHOUSE_CLIENT -n --query=" PRIMARY KEY id SOURCE(CLICKHOUSE(HOST 'localhost' PORT tcpPort() USER 'default' TABLE 'source_table')) LIFETIME(MIN 1 MAX 1000) - LAYOUT(SSD_CACHE(BLOCK_SIZE 4096 FILE_SIZE 8192 PATH '$USER_FILES_PATH/0d')); + LAYOUT(SSD_CACHE(BLOCK_SIZE 4096 FILE_SIZE 8192 PATH '$CLICKHOUSE_USER_FILES/0d')); SELECT dictGetOrDefault('02950_database_for_ssd_cache_dictionary.ssd_cache_dictionary', ('v1', 'v2'), 0, (intDiv(1, id), intDiv(1, id))) FROM 02950_database_for_ssd_cache_dictionary.source_table; SELECT dictGetOrDefault('02950_database_for_ssd_cache_dictionary.ssd_cache_dictionary', 'v2', id+1, intDiv(NULL, id)) FROM 02950_database_for_ssd_cache_dictionary.source_table; diff --git a/tests/queries/0_stateless/02961_storage_config_volume_priority.sh b/tests/queries/0_stateless/02961_storage_config_volume_priority.sh index 4e085541a8d..145b921a750 100755 --- a/tests/queries/0_stateless/02961_storage_config_volume_priority.sh +++ b/tests/queries/0_stateless/02961_storage_config_volume_priority.sh @@ -15,7 +15,7 @@ WHERE policy_name = 'policy_02961' ORDER BY volume_priority ASC; " -config_path=/etc/clickhouse-server/config.d/storage_conf_02961.xml +config_path=${CLICKHOUSE_CONFIG_DIR}/config.d/storage_conf_02961.xml config_path_tmp=$config_path.tmp echo 'check non-unique values dont work' diff --git a/tests/queries/0_stateless/02962_system_sync_replica_lightweight_from_modifier.sh b/tests/queries/0_stateless/02962_system_sync_replica_lightweight_from_modifier.sh index b61be87411d..9ef271632d0 100755 --- a/tests/queries/0_stateless/02962_system_sync_replica_lightweight_from_modifier.sh +++ b/tests/queries/0_stateless/02962_system_sync_replica_lightweight_from_modifier.sh @@ -62,7 +62,7 @@ export -f sync_and_drop_replicas export -f optimize_thread export -f mutations_thread -TIMEOUT=60 +TIMEOUT=30 timeout $TIMEOUT bash -c insert_thread 2> /dev/null & timeout $TIMEOUT bash -c sync_and_drop_replicas 2> /dev/null & diff --git a/tests/queries/0_stateless/02968_file_log_multiple_read.sh b/tests/queries/0_stateless/02968_file_log_multiple_read.sh index 199893a9428..d9bae05270a 100755 --- a/tests/queries/0_stateless/02968_file_log_multiple_read.sh +++ b/tests/queries/0_stateless/02968_file_log_multiple_read.sh @@ -4,12 +4,7 @@ CUR_DIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd) # shellcheck source=../shell_config.sh . "$CUR_DIR"/../shell_config.sh -# Data preparation. -# Now we can get the user_files_path by use the table file function for trick. also we can get it by query as: -# "insert into function file('exist.txt', 'CSV', 'val1 char') values ('aaaa'); select _path from file('exist.txt', 'CSV', 'val1 char')" -user_files_path=$($CLICKHOUSE_CLIENT_BINARY --query "select _path,_file from file('nonexist.txt', 'CSV', 'val1 char')" 2>&1 | grep Exception | awk '{gsub("/nonexist.txt","",$9); print $9}') - -logs_dir=${user_files_path}/${CLICKHOUSE_TEST_UNIQUE_NAME} +logs_dir=${USER_FILES_PATH}/${CLICKHOUSE_TEST_UNIQUE_NAME} rm -rf ${logs_dir} diff --git a/tests/queries/0_stateless/02971_analyzer_remote_id.sh b/tests/queries/0_stateless/02971_analyzer_remote_id.sh index 463e4cc1f0c..ab3c5292529 100755 --- a/tests/queries/0_stateless/02971_analyzer_remote_id.sh +++ b/tests/queries/0_stateless/02971_analyzer_remote_id.sh @@ -1,15 +1,9 @@ #!/usr/bin/env bash -# Tags: no-parallel CURDIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd) # shellcheck source=../shell_config.sh . "$CURDIR"/../shell_config.sh -${CLICKHOUSE_CLIENT} --query="DROP DATABASE IF EXISTS test_02971" -${CLICKHOUSE_CLIENT} --query="CREATE DATABASE test_02971" - -${CLICKHOUSE_CLIENT} --query="CREATE TABLE test_02971.x ENGINE = MergeTree() ORDER BY number AS SELECT * FROM numbers(2)" -${CLICKHOUSE_LOCAL} --query="SELECT count() FROM remote('127.0.0.{2,3}', 'test_02971.x') SETTINGS allow_experimental_analyzer = 1" 2>&1 \ +${CLICKHOUSE_CLIENT} --query="CREATE TABLE ${CLICKHOUSE_DATABASE}.x ENGINE = MergeTree() ORDER BY number AS SELECT * FROM numbers(2)" +${CLICKHOUSE_LOCAL} --query="SELECT count() FROM remote('127.0.0.{2,3}', '${CLICKHOUSE_DATABASE}.x') SETTINGS allow_experimental_analyzer = 1" 2>&1 \ | grep -av "ASan doesn't fully support makecontext/swapcontext functions" - -${CLICKHOUSE_CLIENT} --query="DROP DATABASE IF EXISTS test_02971" diff --git a/tests/queries/0_stateless/02973_parse_crlf_with_tsv_files.sh b/tests/queries/0_stateless/02973_parse_crlf_with_tsv_files.sh index 14f28f1ba4a..6edac86be5b 100755 --- a/tests/queries/0_stateless/02973_parse_crlf_with_tsv_files.sh +++ b/tests/queries/0_stateless/02973_parse_crlf_with_tsv_files.sh @@ -4,8 +4,6 @@ CUR_DIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd) # shellcheck source=../shell_config.sh . "$CUR_DIR"/../shell_config.sh -# Data preparation step -USER_FILES_PATH=$($CLICKHOUSE_CLIENT_BINARY --query "select _path,_file from file('nonexist.txt', 'CSV', 'val1 char')" 2>&1 | grep Exception | awk '{gsub("/nonexist.txt","",$9); print $9}') UNIX_ENDINGS="${CLICKHOUSE_TEST_UNIQUE_NAME}_data_without_crlf.tsv" DOS_ENDINGS="${CLICKHOUSE_TEST_UNIQUE_NAME}_data_with_crlf.tsv" DATA_FILE_UNIX_ENDINGS="${USER_FILES_PATH:?}/${UNIX_ENDINGS}" diff --git a/tests/queries/0_stateless/02984_form_format.sh b/tests/queries/0_stateless/02984_form_format.sh index ce5feb60130..471b48e0f68 100755 --- a/tests/queries/0_stateless/02984_form_format.sh +++ b/tests/queries/0_stateless/02984_form_format.sh @@ -5,8 +5,6 @@ CURDIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd) # shellcheck source=../shell_config.sh . "$CURDIR"/../shell_config.sh -# Test setup -USER_FILES_PATH=$($CLICKHOUSE_CLIENT_BINARY --query "select _path,_file from file('nonexist.txt', 'CSV', 'val1 char')" 2>&1 | grep Exception | awk '{gsub("/nonexist.txt","",$9); print $9}') FILE_NAME="data.tmp" FORM_DATA="${USER_FILES_PATH}/${CLICKHOUSE_TEST_UNIQUE_NAME}/${FILE_NAME}" mkdir -p ${USER_FILES_PATH}/${CLICKHOUSE_TEST_UNIQUE_NAME}/ @@ -30,4 +28,4 @@ touch $FORM_DATA echo -ne "c.e=ls7xfkpm&c.tti.m=raf&rt.start=navigation&rt.bmr=390%2C11%2C10&rt.tstart=1707076768666&rt.bstart=1707076769091&rt.blstart=1707076769056&rt.end=1707076769078&t_resp=296&t_page=116&t_done=412&t_other=boomerang%7C6%2Cboomr_fb%7C425%2Cboomr_ld%7C390%2Cboomr_lat%7C35&rt.tt=2685&rt.obo=0&pt.fcp=407&nt_nav_st=1707076768666&nt_dns_st=1707076768683&nt_dns_end=1707076768684&nt_con_st=1707076768684&nt_con_end=1707076768850&nt_req_st=1707076768850&nt_res_st=1707076768962&nt_res_end=1707076768962&nt_domloading=1707076769040&nt_domint=1707076769066&nt_domcontloaded_st=1707076769067&nt_domcontloaded_end=1707076769068&nt_domcomp=1707076769069&nt_load_st=1707076769069&nt_load_end=1707076769078&nt_unload_st=1707076769040&nt_unload_end=1707076769041&nt_ssl_st=1707076768788&nt_enc_size=3209&nt_dec_size=10093&nt_trn_size=3940&nt_protocol=h2&nt_red_cnt=0&nt_nav_type=1&restiming=%7B%22https%3A%2F%2Fwww.basicrum.com%2F%22%3A%7B%22publications%2F%22%3A%226%2C88%2C88%2C54%2C54%2C3e%2Ci%2Ci%2Ch*12h5%2Ckb%2C5b8%22%2C%22assets%2Fjs%2F%22%3A%7B%22just-the-docs.js%22%3A%223am%2Ce%2Ce*12pc%2C_%2C8oj*20%22%2C%22boomerang-1.737.60.cutting-edge.min.js%22%3A%222au%2Cb%2Ca*1pu3%2C_%2C1m19*21*42%22%2C%22vendor%2Flunr.min.js%22%3A%223am%2Cd%2C8*16t2%2C_%2Cfym*20%22%7D%7D%7D&u=https%3A%2F%2Fwww.basicrum.com%2Fpublications%2F&r=https%3A%2F%2Fwww.basicrum.com%2Fcost-analyses%2F&v=1.737.60&sv=14&sm=p&rt.si=dd0c542f-7adf-4310-830a-6c0a3d157c90-s8cjr1&rt.ss=1707075325294&rt.sl=4&vis.st=visible&ua.plt=Linux%20x86_64&ua.vnd=&pid=8fftz949&n=1&c.t.fps=07*4*65*j*61&c.t.busy=2*4*0034&c.tti.vr=408&c.tti=408&c.b=2&c.f=60&c.f.d=2511&c.f.m=1&c.f.s=ls7xfl1h&dom.res=5&dom.doms=1&mem.lsln=0&mem.ssln=0&mem.lssz=2&mem.sssz=2&scr.xy=1920x1200&scr.bpp=24%2F24&scr.orn=0%2Flandscape-primary&cpu.cnc=16&dom.ln=114&dom.sz=10438&dom.ck=157&dom.img=0&dom.script=6&dom.script.ext=3&dom.iframe=0&dom.link=4&dom.link.css=1&sb=1" > $FORM_DATA $CLICKHOUSE_CLIENT -q "SELECT * FROM file('$FORM_DATA', Form) FORMAT Vertical" -rm $FORM_DATA \ No newline at end of file +rm $FORM_DATA diff --git a/tests/queries/0_stateless/03032_dynamically_resize_filesystem_cache_2.sh b/tests/queries/0_stateless/03032_dynamically_resize_filesystem_cache_2.sh index 09bdd7f6b56..cba5317fcfa 100755 --- a/tests/queries/0_stateless/03032_dynamically_resize_filesystem_cache_2.sh +++ b/tests/queries/0_stateless/03032_dynamically_resize_filesystem_cache_2.sh @@ -18,7 +18,7 @@ $CLICKHOUSE_CLIENT --query "SELECT * FROM test FORMAT Null" prev_max_size=$($CLICKHOUSE_CLIENT --query "SELECT max_size FROM system.filesystem_cache_settings WHERE cache_name = '$disk_name'") $CLICKHOUSE_CLIENT --query "SELECT current_size > 0 FROM system.filesystem_cache_settings WHERE cache_name = '$disk_name' FORMAT TabSeparated" -config_path=/etc/clickhouse-server/config.d/storage_conf.xml +config_path=${CLICKHOUSE_CONFIG_DIR}/config.d/storage_conf.xml new_max_size=$($CLICKHOUSE_CLIENT --query "SELECT divide(max_size, 2) FROM system.filesystem_cache_settings WHERE cache_name = '$disk_name'") sed -i "s|$prev_max_size<\/max_size>|$new_max_size<\/max_size>|" $config_path diff --git a/tests/queries/0_stateless/03144_parallel_alter_add_drop_column_zookeeper_on_steroids.sh b/tests/queries/0_stateless/03144_parallel_alter_add_drop_column_zookeeper_on_steroids.sh index ea7bb8f7ad0..c27dfffcfc2 100755 --- a/tests/queries/0_stateless/03144_parallel_alter_add_drop_column_zookeeper_on_steroids.sh +++ b/tests/queries/0_stateless/03144_parallel_alter_add_drop_column_zookeeper_on_steroids.sh @@ -85,7 +85,7 @@ export -f optimize_thread; export -f insert_thread; -TIMEOUT=30 +TIMEOUT=20 # Sometimes we detach and attach tables timeout $TIMEOUT bash -c alter_thread 2> /dev/null & diff --git a/tests/queries/0_stateless/03153_format_regexp_usability.sh b/tests/queries/0_stateless/03153_format_regexp_usability.sh index 03bed10dd17..561de3be893 100755 --- a/tests/queries/0_stateless/03153_format_regexp_usability.sh +++ b/tests/queries/0_stateless/03153_format_regexp_usability.sh @@ -1,5 +1,5 @@ #!/usr/bin/env bash -# Tags: no-fasttest, no-parallel, no-ordinary-database, long +# Tags: no-fasttest, no-ordinary-database, long CUR_DIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd) # shellcheck source=../shell_config.sh From 0365a82e538731bdb17407d0765d0ff81e722b54 Mon Sep 17 00:00:00 2001 From: Antonio Andelic Date: Wed, 10 Jul 2024 08:04:46 +0200 Subject: [PATCH 364/417] Fix test --- tests/queries/0_stateless/01268_procfs_metrics.sh | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/tests/queries/0_stateless/01268_procfs_metrics.sh b/tests/queries/0_stateless/01268_procfs_metrics.sh index 4f09d197596..7d6389bb86e 100755 --- a/tests/queries/0_stateless/01268_procfs_metrics.sh +++ b/tests/queries/0_stateless/01268_procfs_metrics.sh @@ -15,7 +15,7 @@ tmp_path=$(mktemp "$CURDIR/01268_procfs_metrics.XXXXXX") trap 'rm -f $tmp_path' EXIT truncate -s1025 "$tmp_path" -$CLICKHOUSE_LOCAL --profile-events-delay-ms=-1 --print-profile-events -q "SELECT * FROM file('$tmp_path', 'LineAsString') FORMAT Null" |& grep -m1 -F -o -e OSReadChars +$CLICKHOUSE_LOCAL --profile-events-delay-ms=-1 --print-profile-events --storage_file_read_method=pread -q "SELECT * FROM file('$tmp_path', 'LineAsString') FORMAT Null" |& grep -m1 -F -o -e OSReadChars # NOTE: that OSCPUVirtualTimeMicroseconds is in microseconds, so 1e6 is not enough. $CLICKHOUSE_LOCAL --profile-events-delay-ms=-1 --print-profile-events -q "SELECT * FROM numbers(1e8) FORMAT Null" |& grep -m1 -F -o -e OSCPUVirtualTimeMicroseconds exit 0 From bc02d8e66ecc82bee3c8d0402b01816c5005ece9 Mon Sep 17 00:00:00 2001 From: MikhailBurdukov Date: Wed, 10 Jul 2024 08:01:36 +0000 Subject: [PATCH 365/417] Fix settings changelog --- src/Core/SettingsChangesHistory.cpp | 1 + 1 file changed, 1 insertion(+) diff --git a/src/Core/SettingsChangesHistory.cpp b/src/Core/SettingsChangesHistory.cpp index b0725340f46..3ccc7321088 100644 --- a/src/Core/SettingsChangesHistory.cpp +++ b/src/Core/SettingsChangesHistory.cpp @@ -59,6 +59,7 @@ static std::initializer_list Date: Wed, 10 Jul 2024 11:33:48 +0300 Subject: [PATCH 366/417] Remove typo from distinctive-features.md --- docs/ru/introduction/distinctive-features.md | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/docs/ru/introduction/distinctive-features.md b/docs/ru/introduction/distinctive-features.md index dafaf055980..da820c90a1e 100644 --- a/docs/ru/introduction/distinctive-features.md +++ b/docs/ru/introduction/distinctive-features.md @@ -12,7 +12,7 @@ sidebar_label: "Отличительные возможности ClickHouse" Этот пункт пришлось выделить, так как существуют системы, которые могут хранить значения отдельных столбцов по отдельности, но не могут эффективно выполнять аналитические запросы в силу оптимизации под другой сценарий работы. Примеры: HBase, BigTable, Cassandra, HyperTable. В этих системах вы получите пропускную способность в районе сотен тысяч строк в секунду, но не сотен миллионов строк в секунду. -Также стоит заметить, что ClickHouse является системой управления базами данных, а не системой для одной базой данных. То есть, ClickHouse позволяет создавать таблицы и базы данных во время выполнения (runtime), загружать данные и выполнять запросы без переконфигурирования и перезапуска сервера. +Также стоит заметить, что ClickHouse является системой управления базами данных, а не системой для одной базы данных. То есть, ClickHouse позволяет создавать таблицы и базы данных во время выполнения (runtime), загружать данные и выполнять запросы без переконфигурирования и перезапуска сервера. ## Сжатие данных {#szhatie-dannykh} From 513ce9fa2f3bb0d2cc1774a07272a249b40f475f Mon Sep 17 00:00:00 2001 From: vdimir Date: Wed, 10 Jul 2024 08:40:09 +0000 Subject: [PATCH 367/417] Disable broken cases from 02911_join_on_nullsafe_optimization --- ...2911_join_on_nullsafe_optimization.reference | 17 ++++------------- .../02911_join_on_nullsafe_optimization.sql | 5 +++-- 2 files changed, 7 insertions(+), 15 deletions(-) diff --git a/tests/queries/0_stateless/02911_join_on_nullsafe_optimization.reference b/tests/queries/0_stateless/02911_join_on_nullsafe_optimization.reference index f0463509b80..4eb7e74446d 100644 --- a/tests/queries/0_stateless/02911_join_on_nullsafe_optimization.reference +++ b/tests/queries/0_stateless/02911_join_on_nullsafe_optimization.reference @@ -36,19 +36,10 @@ SELECT * FROM t1 JOIN t2 ON (t1.x == t2.x AND ((t2.x IS NOT NULL) AND (t1.x IS N 3 3 3 33 \N \N \N \N -- aliases defined in the join condition are valid -SELECT *, e, e2 FROM t1 FULL JOIN t2 ON ( ( ((t1.x == t2.x) AS e) AND ((t2.x IS NOT NULL) AND (t1.x IS NOT NULL)) ) OR ( (t2.x IS NULL) AND (t1.x IS NULL) ) AS e2 ) ORDER BY t1.x NULLS LAST, t2.x NULLS LAST; -1 42 \N \N \N 0 -2 2 2 2 1 1 -3 3 3 33 1 1 -\N \N 4 42 \N 0 -\N \N \N \N \N 1 -SELECT *, e, e2 FROM t1 FULL JOIN t2 ON ( ( ((t1.x == t2.x) AS e) AND ((t2.x IS NOT NULL) AND (t1.x IS NOT NULL)) ) AS e2 ) ORDER BY t1.x NULLS LAST, t2.x NULLS LAST; -1 42 \N \N \N 0 -2 2 2 2 1 1 -3 3 3 33 1 1 -\N \N 4 42 \N 0 -\N \N \N \N \N 0 -\N \N \N \N \N 0 +-- FIXME(@vdimir) broken query formatting for the following queries: +-- SELECT *, e, e2 FROM t1 FULL JOIN t2 ON ( ( ((t1.x == t2.x) AS e) AND ((t2.x IS NOT NULL) AND (t1.x IS NOT NULL)) ) OR ( (t2.x IS NULL) AND (t1.x IS NULL) ) AS e2 ) ORDER BY t1.x NULLS LAST, t2.x NULLS LAST; +-- SELECT *, e, e2 FROM t1 FULL JOIN t2 ON ( ( ((t1.x == t2.x) AS e) AND ((t2.x IS NOT NULL) AND (t1.x IS NOT NULL)) ) AS e2 ) ORDER BY t1.x NULLS LAST, t2.x NULLS LAST; + -- check for non-nullable columns for which `is null` is replaced with constant SELECT * FROM t1n as t1 JOIN t2n as t2 ON (t1.x == t2.x AND ((t2.x IS NOT NULL) AND (t1.x IS NOT NULL)) ) OR ( (t2.x IS NULL) AND (t1.x IS NULL) ) ORDER BY t1.x NULLS LAST; 2 2 2 2 diff --git a/tests/queries/0_stateless/02911_join_on_nullsafe_optimization.sql b/tests/queries/0_stateless/02911_join_on_nullsafe_optimization.sql index 67918f4302f..f7813e2a1b4 100644 --- a/tests/queries/0_stateless/02911_join_on_nullsafe_optimization.sql +++ b/tests/queries/0_stateless/02911_join_on_nullsafe_optimization.sql @@ -36,8 +36,9 @@ SELECT x = y OR (x IS NULL AND y IS NULL) FROM t1 ORDER BY x NULLS LAST; SELECT * FROM t1 JOIN t2 ON (t1.x == t2.x AND ((t2.x IS NOT NULL) AND (t1.x IS NOT NULL)) ) OR ( (t2.x IS NULL) AND (t1.x IS NULL) ) ORDER BY t1.x NULLS LAST; -- aliases defined in the join condition are valid -SELECT *, e, e2 FROM t1 FULL JOIN t2 ON ( ( ((t1.x == t2.x) AS e) AND ((t2.x IS NOT NULL) AND (t1.x IS NOT NULL)) ) OR ( (t2.x IS NULL) AND (t1.x IS NULL) ) AS e2 ) ORDER BY t1.x NULLS LAST, t2.x NULLS LAST; -SELECT *, e, e2 FROM t1 FULL JOIN t2 ON ( ( ((t1.x == t2.x) AS e) AND ((t2.x IS NOT NULL) AND (t1.x IS NOT NULL)) ) AS e2 ) ORDER BY t1.x NULLS LAST, t2.x NULLS LAST; +-- FIXME(@vdimir) broken query formatting for the following queries: +-- SELECT *, e, e2 FROM t1 FULL JOIN t2 ON ( ( ((t1.x == t2.x) AS e) AND ((t2.x IS NOT NULL) AND (t1.x IS NOT NULL)) ) OR ( (t2.x IS NULL) AND (t1.x IS NULL) ) AS e2 ) ORDER BY t1.x NULLS LAST, t2.x NULLS LAST; +-- SELECT *, e, e2 FROM t1 FULL JOIN t2 ON ( ( ((t1.x == t2.x) AS e) AND ((t2.x IS NOT NULL) AND (t1.x IS NOT NULL)) ) AS e2 ) ORDER BY t1.x NULLS LAST, t2.x NULLS LAST; -- check for non-nullable columns for which `is null` is replaced with constant SELECT * FROM t1n as t1 JOIN t2n as t2 ON (t1.x == t2.x AND ((t2.x IS NOT NULL) AND (t1.x IS NOT NULL)) ) OR ( (t2.x IS NULL) AND (t1.x IS NULL) ) ORDER BY t1.x NULLS LAST; From 36b9a5641f614de1fba593d42ba7670614cde5aa Mon Sep 17 00:00:00 2001 From: Kseniia Sumarokova <54203879+kssenii@users.noreply.github.com> Date: Wed, 10 Jul 2024 11:56:43 +0200 Subject: [PATCH 368/417] Update run.sh --- docker/test/stateless/run.sh | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/docker/test/stateless/run.sh b/docker/test/stateless/run.sh index 1c03f5107b0..8e66d2667f1 100755 --- a/docker/test/stateless/run.sh +++ b/docker/test/stateless/run.sh @@ -249,7 +249,7 @@ function run_tests() try_run_with_retry 10 clickhouse-client -q "insert into system.zookeeper (name, path, value) values ('auxiliary_zookeeper2', '/test/chroot/', '')" set +e - timeout -s TERM --preserve-status 120m -k 60m clickhouse-test --testname --shard --zookeeper --check-zookeeper-session --hung-check --print-time \ + timeout -k 60m -s TERM --preserve-status 120m clickhouse-test --testname --shard --zookeeper --check-zookeeper-session --hung-check --print-time \ --no-drop-if-fail --test-runs "$NUM_TRIES" "${ADDITIONAL_OPTIONS[@]}" 2>&1 \ | ts '%Y-%m-%d %H:%M:%S' \ | tee -a test_output/test_result.txt From a32795d116903c66c18263f47e5d1e622d83a362 Mon Sep 17 00:00:00 2001 From: avogar Date: Wed, 10 Jul 2024 10:07:02 +0000 Subject: [PATCH 369/417] Fix review comments --- src/Formats/JSONExtractTree.cpp | 174 ++++++++++++++++++++++++-------- src/Formats/JSONExtractTree.h | 6 ++ src/Functions/FunctionsJSON.cpp | 3 + 3 files changed, 139 insertions(+), 44 deletions(-) diff --git a/src/Formats/JSONExtractTree.cpp b/src/Formats/JSONExtractTree.cpp index 9efb1392583..242d2dc9f80 100644 --- a/src/Formats/JSONExtractTree.cpp +++ b/src/Formats/JSONExtractTree.cpp @@ -26,6 +26,7 @@ #include #include #include +#include #include #include #include @@ -35,9 +36,8 @@ #include #include #include -#include #include -#include +#include #include #include @@ -123,10 +123,7 @@ void jsonElementToString(const typename JSONParser::Element & element, WriteBuff template bool tryGetNumericValueFromJSONElement( - NumberType & value, - const typename JSONParser::Element & element, - bool convert_bool_to_integer, - String & error) + NumberType & value, const typename JSONParser::Element & element, bool convert_bool_to_integer, String & error) { switch (element.type()) { @@ -226,7 +223,11 @@ public: explicit NumericNode(bool is_bool_type_ = false) : is_bool_type(is_bool_type_) { } bool insertResultToColumn( - IColumn & column, const typename JSONParser::Element & element, const JSONExtractInsertSettings & insert_settings, const FormatSettings & format_settings, String & error) const override + IColumn & column, + const typename JSONParser::Element & element, + const JSONExtractInsertSettings & insert_settings, + const FormatSettings & format_settings, + String & error) const override { if (element.isNull()) { @@ -270,7 +271,11 @@ public: } bool insertResultToColumn( - IColumn & column, const typename JSONParser::Element & element, const JSONExtractInsertSettings & insert_settings, const FormatSettings & format_settings, String & error) const override + IColumn & column, + const typename JSONParser::Element & element, + const JSONExtractInsertSettings & insert_settings, + const FormatSettings & format_settings, + String & error) const override { if (element.isNull()) { @@ -309,7 +314,11 @@ class StringNode : public JSONExtractTreeNode { public: bool insertResultToColumn( - IColumn & column, const typename JSONParser::Element & element, const JSONExtractInsertSettings &, const FormatSettings & format_settings, String & error) const override + IColumn & column, + const typename JSONParser::Element & element, + const JSONExtractInsertSettings &, + const FormatSettings & format_settings, + String & error) const override { if (element.isNull()) { @@ -349,7 +358,11 @@ public: explicit LowCardinalityStringNode(bool is_nullable_) : is_nullable(is_nullable_) { } bool insertResultToColumn( - IColumn & column, const typename JSONParser::Element & element, const JSONExtractInsertSettings &, const FormatSettings & format_settings, String & error) const override + IColumn & column, + const typename JSONParser::Element & element, + const JSONExtractInsertSettings &, + const FormatSettings & format_settings, + String & error) const override { if (element.isNull()) { @@ -387,7 +400,11 @@ class FixedStringNode : public JSONExtractTreeNode public: explicit FixedStringNode(size_t fixed_length_) : fixed_length(fixed_length_) { } bool insertResultToColumn( - IColumn & column, const typename JSONParser::Element & element, const JSONExtractInsertSettings &, const FormatSettings & format_settings, String & error) const override + IColumn & column, + const typename JSONParser::Element & element, + const JSONExtractInsertSettings &, + const FormatSettings & format_settings, + String & error) const override { if (element.isNull()) { @@ -431,7 +448,11 @@ public: } bool insertResultToColumn( - IColumn & column, const typename JSONParser::Element & element, const JSONExtractInsertSettings &, const FormatSettings & format_settings, String & error) const override + IColumn & column, + const typename JSONParser::Element & element, + const JSONExtractInsertSettings &, + const FormatSettings & format_settings, + String & error) const override { if (element.isNull()) { @@ -484,7 +505,11 @@ class UUIDNode : public JSONExtractTreeNode { public: bool insertResultToColumn( - IColumn & column, const typename JSONParser::Element & element, const JSONExtractInsertSettings &, const FormatSettings & format_settings, String & error) const override + IColumn & column, + const typename JSONParser::Element & element, + const JSONExtractInsertSettings &, + const FormatSettings & format_settings, + String & error) const override { if (element.isNull() && format_settings.null_as_default) { @@ -525,7 +550,11 @@ public: explicit LowCardinalityUUIDNode(bool is_nullable_) : is_nullable(is_nullable_) { } bool insertResultToColumn( - IColumn & column, const typename JSONParser::Element & element, const JSONExtractInsertSettings &, const FormatSettings & format_settings, String & error) const override + IColumn & column, + const typename JSONParser::Element & element, + const JSONExtractInsertSettings &, + const FormatSettings & format_settings, + String & error) const override { if (element.isNull() && (is_nullable || format_settings.null_as_default)) { @@ -560,7 +589,11 @@ class DateNode : public JSONExtractTreeNode { public: bool insertResultToColumn( - IColumn & column, const typename JSONParser::Element & element, const JSONExtractInsertSettings &, const FormatSettings & format_settings, String & error) const override + IColumn & column, + const typename JSONParser::Element & element, + const JSONExtractInsertSettings &, + const FormatSettings & format_settings, + String & error) const override { if (element.isNull() && format_settings.null_as_default) { @@ -595,7 +628,11 @@ public: explicit DateTimeNode(const DataTypeDateTime & datetime_type) : TimezoneMixin(datetime_type) { } bool insertResultToColumn( - IColumn & column, const typename JSONParser::Element & element, const JSONExtractInsertSettings &, const FormatSettings & format_settings, String & error) const override + IColumn & column, + const typename JSONParser::Element & element, + const JSONExtractInsertSettings &, + const FormatSettings & format_settings, + String & error) const override { if (element.isNull() && format_settings.null_as_default) { @@ -656,7 +693,11 @@ public: explicit DecimalNode(const DataTypePtr & type) : scale(assert_cast &>(*type).getScale()) { } bool insertResultToColumn( - IColumn & column, const typename JSONParser::Element & element, const JSONExtractInsertSettings &, const FormatSettings & format_settings, String & error) const override + IColumn & column, + const typename JSONParser::Element & element, + const JSONExtractInsertSettings &, + const FormatSettings & format_settings, + String & error) const override { DecimalType value{}; @@ -688,7 +729,8 @@ public: } break; } - default: { + default: + { error = fmt::format("cannot read Decimal value from JSON element: {}", jsonElementToString(element, format_settings)); return false; } @@ -707,10 +749,16 @@ template class DateTime64Node : public JSONExtractTreeNode, public TimezoneMixin { public: - explicit DateTime64Node(const DataTypeDateTime64 & datetime64_type) : TimezoneMixin(datetime64_type), scale(datetime64_type.getScale()) { } + explicit DateTime64Node(const DataTypeDateTime64 & datetime64_type) : TimezoneMixin(datetime64_type), scale(datetime64_type.getScale()) + { + } bool insertResultToColumn( - IColumn & column, const typename JSONParser::Element & element, const JSONExtractInsertSettings &, const FormatSettings & format_settings, String & error) const override + IColumn & column, + const typename JSONParser::Element & element, + const JSONExtractInsertSettings &, + const FormatSettings & format_settings, + String & error) const override { if (element.isNull() && format_settings.null_as_default) { @@ -790,7 +838,11 @@ public: } bool insertResultToColumn( - IColumn & column, const typename JSONParser::Element & element, const JSONExtractInsertSettings &, const FormatSettings & format_settings, String & error) const override + IColumn & column, + const typename JSONParser::Element & element, + const JSONExtractInsertSettings &, + const FormatSettings & format_settings, + String & error) const override { if (element.isNull()) { @@ -857,7 +909,11 @@ class IPv4Node : public JSONExtractTreeNode { public: bool insertResultToColumn( - IColumn & column, const typename JSONParser::Element & element, const JSONExtractInsertSettings &, const FormatSettings & format_settings, String & error) const override + IColumn & column, + const typename JSONParser::Element & element, + const JSONExtractInsertSettings &, + const FormatSettings & format_settings, + String & error) const override { if (element.isNull() && format_settings.null_as_default) { @@ -895,7 +951,11 @@ class IPv6Node : public JSONExtractTreeNode { public: bool insertResultToColumn( - IColumn & column, const typename JSONParser::Element & element, const JSONExtractInsertSettings &, const FormatSettings & format_settings, String & error) const override + IColumn & column, + const typename JSONParser::Element & element, + const JSONExtractInsertSettings &, + const FormatSettings & format_settings, + String & error) const override { if (element.isNull() && format_settings.null_as_default) { @@ -936,7 +996,11 @@ public: explicit NullableNode(std::unique_ptr> nested_) : nested(std::move(nested_)) { } bool insertResultToColumn( - IColumn & column, const typename JSONParser::Element & element, const JSONExtractInsertSettings & insert_settings, const FormatSettings & format_settings, String & error) const override + IColumn & column, + const typename JSONParser::Element & element, + const JSONExtractInsertSettings & insert_settings, + const FormatSettings & format_settings, + String & error) const override { if (element.isNull()) { @@ -945,7 +1009,7 @@ public: } auto & col_null = assert_cast(column); - if (!nested-> insertResultToColumn(col_null.getNestedColumn(), element, insert_settings, format_settings, error)) + if (!nested->insertResultToColumn(col_null.getNestedColumn(), element, insert_settings, format_settings, error)) return false; col_null.getNullMapColumn().insertValue(0); return true; @@ -965,7 +1029,11 @@ public: } bool insertResultToColumn( - IColumn & column, const typename JSONParser::Element & element, const JSONExtractInsertSettings & insert_settings, const FormatSettings & format_settings, String & error) const override + IColumn & column, + const typename JSONParser::Element & element, + const JSONExtractInsertSettings & insert_settings, + const FormatSettings & format_settings, + String & error) const override { if (element.isNull() && (is_nullable || format_settings.null_as_default)) { @@ -975,7 +1043,7 @@ public: auto & col_lc = assert_cast(column); auto tmp_nested = col_lc.getDictionary().getNestedColumn()->cloneEmpty(); - if (!nested-> insertResultToColumn(*tmp_nested, element, insert_settings, format_settings, error)) + if (!nested->insertResultToColumn(*tmp_nested, element, insert_settings, format_settings, error)) return false; col_lc.insertFromFullColumn(*tmp_nested, 0); @@ -994,7 +1062,11 @@ public: explicit ArrayNode(std::unique_ptr> nested_) : nested(std::move(nested_)) { } bool insertResultToColumn( - IColumn & column, const typename JSONParser::Element & element, const JSONExtractInsertSettings & insert_settings, const FormatSettings & format_settings, String & error) const override + IColumn & column, + const typename JSONParser::Element & element, + const JSONExtractInsertSettings & insert_settings, + const FormatSettings & format_settings, + String & error) const override { if (element.isNull() && format_settings.null_as_default) { @@ -1017,7 +1089,7 @@ public: for (auto value : array) { - if (nested-> insertResultToColumn(data, value, insert_settings, format_settings, error)) + if (nested->insertResultToColumn(data, value, insert_settings, format_settings, error)) { were_valid_elements = true; } @@ -1058,7 +1130,11 @@ public: } bool insertResultToColumn( - IColumn & column, const typename JSONParser::Element & element, const JSONExtractInsertSettings & insert_settings, const FormatSettings & format_settings, String & error) const override + IColumn & column, + const typename JSONParser::Element & element, + const JSONExtractInsertSettings & insert_settings, + const FormatSettings & format_settings, + String & error) const override { auto & tuple = assert_cast(column); size_t old_size = column.size(); @@ -1087,7 +1163,7 @@ public: for (size_t index = 0; (index != nested.size()) && (it != array.end()); ++index) { - if (nested[index]-> insertResultToColumn(tuple.getColumn(index), *it++, insert_settings, format_settings, error)) + if (nested[index]->insertResultToColumn(tuple.getColumn(index), *it++, insert_settings, format_settings, error)) { were_valid_elements = true; } @@ -1115,7 +1191,7 @@ public: auto it = object.begin(); for (size_t index = 0; (index != nested.size()) && (it != object.end()); ++index) { - if (nested[index]-> insertResultToColumn(tuple.getColumn(index), (*it++).second, insert_settings, format_settings, error)) + if (nested[index]->insertResultToColumn(tuple.getColumn(index), (*it++).second, insert_settings, format_settings, error)) { were_valid_elements = true; } @@ -1138,7 +1214,7 @@ public: auto index = name_to_index_map.find(key); if (index != name_to_index_map.end()) { - if (nested[index->second]-> insertResultToColumn(tuple.getColumn(index->second), value, insert_settings, format_settings, error)) + if (nested[index->second]->insertResultToColumn(tuple.getColumn(index->second), value, insert_settings, format_settings, error)) { were_valid_elements = true; } @@ -1173,7 +1249,11 @@ public: explicit MapNode(std::unique_ptr> value_) : value(std::move(value_)) { } bool insertResultToColumn( - IColumn & column, const typename JSONParser::Element & element, const JSONExtractInsertSettings & insert_settings, const FormatSettings & format_settings, String & error) const override + IColumn & column, + const typename JSONParser::Element & element, + const JSONExtractInsertSettings & insert_settings, + const FormatSettings & format_settings, + String & error) const override { if (!element.isObject()) { @@ -1198,7 +1278,7 @@ public: key_col.insertData(pair.first.data(), pair.first.size()); /// Insert value - if (!value-> insertResultToColumn(value_col, pair.second, insert_settings, format_settings, error)) + if (!value->insertResultToColumn(value_col, pair.second, insert_settings, format_settings, error)) { if (insert_settings.insert_default_on_invalid_elements_in_complex_types) { @@ -1232,13 +1312,17 @@ public: } bool insertResultToColumn( - IColumn & column, const typename JSONParser::Element & element, const JSONExtractInsertSettings & insert_settings, const FormatSettings & format_settings, String & error) const override + IColumn & column, + const typename JSONParser::Element & element, + const JSONExtractInsertSettings & insert_settings, + const FormatSettings & format_settings, + String & error) const override { auto & column_variant = assert_cast(column); for (size_t i : order) { auto & variant = column_variant.getVariantByGlobalDiscriminator(i); - if (variant_nodes[i]-> insertResultToColumn(variant, element, insert_settings, format_settings, error)) + if (variant_nodes[i]->insertResultToColumn(variant, element, insert_settings, format_settings, error)) { column_variant.getLocalDiscriminators().push_back(column_variant.localDiscriminatorByGlobal(i)); column_variant.getOffsets().push_back(variant.size() - 1); @@ -1262,7 +1346,12 @@ template class DynamicNode : public JSONExtractTreeNode { public: - bool insertResultToColumn(IColumn & column, const typename JSONParser::Element & element, const JSONExtractInsertSettings & insert_settings, const FormatSettings & format_settings, String & error) const override + bool insertResultToColumn( + IColumn & column, + const typename JSONParser::Element & element, + const JSONExtractInsertSettings & insert_settings, + const FormatSettings & format_settings, + String & error) const override { auto & column_dynamic = assert_cast(column); /// First, check if element is NULL. @@ -1281,7 +1370,7 @@ public: auto node = buildJSONExtractTree(element_type, "Dynamic inference"); auto global_discriminator = variant_info.variant_name_to_discriminator[element_type->getName()]; auto & variant = variant_column.getVariantByGlobalDiscriminator(global_discriminator); - if (!node-> insertResultToColumn(variant, element, insert_settings, format_settings, error)) + if (!node->insertResultToColumn(variant, element, insert_settings, format_settings, error)) return false; variant_column.getLocalDiscriminators().push_back(variant_column.localDiscriminatorByGlobal(global_discriminator)); variant_column.getOffsets().push_back(variant.size() - 1); @@ -1290,14 +1379,14 @@ public: /// We couldn't add new variant. Try to insert element into current variants. auto variant_node = buildJSONExtractTree(variant_info.variant_type, "Dynamic inference"); - if (variant_node-> insertResultToColumn(variant_column, element, insert_settings, format_settings, error)) + if (variant_node->insertResultToColumn(variant_column, element, insert_settings, format_settings, error)) return true; /// We couldn't insert element into any existing variant, add String variant and read value as String. column_dynamic.addStringVariant(); auto string_global_discriminator = variant_info.variant_name_to_discriminator["String"]; auto & string_column = variant_column.getVariantByGlobalDiscriminator(string_global_discriminator); - if (!getStringNode()-> insertResultToColumn(string_column, element, insert_settings, format_settings, error)) + if (!getStringNode()->insertResultToColumn(string_column, element, insert_settings, format_settings, error)) return false; variant_column.getLocalDiscriminators().push_back(variant_column.localDiscriminatorByGlobal(string_global_discriminator)); variant_column.getOffsets().push_back(string_column.size() - 1); @@ -1348,12 +1437,9 @@ private: if (format_settings.json.try_infer_numbers_from_strings) { - bool is_negative = false; if (auto type = tryInferJSONNumberFromString(data, format_settings, &json_inference_info)) { json_inference_info.numbers_parsed_from_json_strings.insert(type.get()); - if (is_negative) - json_inference_info.negative_integers.insert(type.get()); return type; } } diff --git a/src/Formats/JSONExtractTree.h b/src/Formats/JSONExtractTree.h index 4735f568b1c..b5e82506548 100644 --- a/src/Formats/JSONExtractTree.h +++ b/src/Formats/JSONExtractTree.h @@ -9,7 +9,13 @@ namespace DB struct JSONExtractInsertSettings { + /// If false, JSON boolean values won't be inserted into columns with integer types + /// It's used in JSONExtractInt64/JSONExtractUInt64/... functions. bool convert_bool_to_integer = true; + /// If true, when complex type like Array/Map has both valid and invalid elements, + /// the default value will be inserted on invalid elements. + /// For example, if we have [1, "hello", 2] and type Array(UInt32), + /// we will insert [1, 0, 2] in the column. Used in all JSONExtract functions. bool insert_default_on_invalid_elements_in_complex_types = false; }; diff --git a/src/Functions/FunctionsJSON.cpp b/src/Functions/FunctionsJSON.cpp index ca233becb63..db1602b1939 100644 --- a/src/Functions/FunctionsJSON.cpp +++ b/src/Functions/FunctionsJSON.cpp @@ -354,7 +354,10 @@ public: explicit ExecutableFunctionJSON(const NullPresence & null_presence_, bool allow_simdjson_, const DataTypePtr & json_return_type_, const FormatSettings & format_settings_) : null_presence(null_presence_), allow_simdjson(allow_simdjson_), json_return_type(json_return_type_), format_settings(format_settings_) { + /// Don't escape forward slashes during converting JSON elements to raw string. format_settings.json.escape_forward_slashes = false; + /// Don't insert default values on null during traversing the JSON element. + /// We allow to insert null only to Nullable columns in JSONExtract functions. format_settings.null_as_default = false; } From 17d5ec28d4803f767f5c1f067116fe02147a1e1e Mon Sep 17 00:00:00 2001 From: Nikita Fomichev Date: Wed, 10 Jul 2024 12:08:33 +0200 Subject: [PATCH 370/417] debug tests hung on gdb error --- tests/clickhouse-test | 3 +++ 1 file changed, 3 insertions(+) diff --git a/tests/clickhouse-test b/tests/clickhouse-test index 8dea6297a61..f60e285e771 100755 --- a/tests/clickhouse-test +++ b/tests/clickhouse-test @@ -2190,6 +2190,7 @@ def run_tests_array(all_tests_with_params: Tuple[List[str], int, TestSuite, bool test_result = test_case.run( args, test_suite, client_options, server_logs_level ) + print(test_result, test_result.case_name, test_result.need_retry) test_result = test_case.process_result(test_result, MESSAGES) if not test_result.need_retry: break @@ -2211,7 +2212,9 @@ def run_tests_array(all_tests_with_params: Tuple[List[str], int, TestSuite, bool failures_total += 1 failures_chain += 1 if test_result.reason == FailureReason.SERVER_DIED: + stop_tests() server_died.set() + raise ServerDied("Server died") elif test_result.status == TestStatus.SKIPPED: skipped_total += 1 From 73c4eaa0f26c0e67f15b0e65ebeda93fd8214957 Mon Sep 17 00:00:00 2001 From: Alexander Gololobov Date: Wed, 10 Jul 2024 12:09:32 +0200 Subject: [PATCH 371/417] Clarify ordered mode description for s3Queue --- docs/en/engines/table-engines/integrations/s3queue.md | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/docs/en/engines/table-engines/integrations/s3queue.md b/docs/en/engines/table-engines/integrations/s3queue.md index 11181703645..c45de8760d6 100644 --- a/docs/en/engines/table-engines/integrations/s3queue.md +++ b/docs/en/engines/table-engines/integrations/s3queue.md @@ -75,7 +75,7 @@ SETTINGS Possible values: - unordered — With unordered mode, the set of all already processed files is tracked with persistent nodes in ZooKeeper. -- ordered — With ordered mode, only the max name of the successfully consumed file, and the names of files that will be retried after unsuccessful loading attempt are being stored in ZooKeeper. +- ordered — With ordered mode, the files are processed in lexicographic order. It means that if file named 'BBB' was processed and some point and later on a file named 'AAAA' is added to the bucket, it will be ignored. Only the max name (in lexicographic sense) of the successfully consumed file, and the names of files that will be retried after unsuccessful loading attempt are being stored in ZooKeeper. Default value: `ordered` in versions before 24.6. Starting with 24.6 there is no default value, the setting becomes required to be specified manually. For tables created on earlier versions the default value will remain `Ordered` for compatibility. From 56c751a10a0070d31e70313854cf1ade54f0c9a5 Mon Sep 17 00:00:00 2001 From: Alexander Gololobov Date: Wed, 10 Jul 2024 12:17:39 +0200 Subject: [PATCH 372/417] Update docs/en/engines/table-engines/integrations/s3queue.md Co-authored-by: Kseniia Sumarokova <54203879+kssenii@users.noreply.github.com> --- docs/en/engines/table-engines/integrations/s3queue.md | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/docs/en/engines/table-engines/integrations/s3queue.md b/docs/en/engines/table-engines/integrations/s3queue.md index c45de8760d6..dcce8c415db 100644 --- a/docs/en/engines/table-engines/integrations/s3queue.md +++ b/docs/en/engines/table-engines/integrations/s3queue.md @@ -75,7 +75,7 @@ SETTINGS Possible values: - unordered — With unordered mode, the set of all already processed files is tracked with persistent nodes in ZooKeeper. -- ordered — With ordered mode, the files are processed in lexicographic order. It means that if file named 'BBB' was processed and some point and later on a file named 'AAAA' is added to the bucket, it will be ignored. Only the max name (in lexicographic sense) of the successfully consumed file, and the names of files that will be retried after unsuccessful loading attempt are being stored in ZooKeeper. +- ordered — With ordered mode, the files are processed in lexicographic order. It means that if file named 'BBB' was processed at some point and later on a file named 'AAAA' is added to the bucket, it will be ignored. Only the max name (in lexicographic sense) of the successfully consumed file, and the names of files that will be retried after unsuccessful loading attempt are being stored in ZooKeeper. Default value: `ordered` in versions before 24.6. Starting with 24.6 there is no default value, the setting becomes required to be specified manually. For tables created on earlier versions the default value will remain `Ordered` for compatibility. From 74cc20b2868ffa88656daf0d30a95287291319ed Mon Sep 17 00:00:00 2001 From: Alexander Gololobov Date: Wed, 10 Jul 2024 12:18:50 +0200 Subject: [PATCH 373/417] Make spellcheck happy --- docs/en/engines/table-engines/integrations/s3queue.md | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/docs/en/engines/table-engines/integrations/s3queue.md b/docs/en/engines/table-engines/integrations/s3queue.md index dcce8c415db..06325fa15fb 100644 --- a/docs/en/engines/table-engines/integrations/s3queue.md +++ b/docs/en/engines/table-engines/integrations/s3queue.md @@ -75,7 +75,7 @@ SETTINGS Possible values: - unordered — With unordered mode, the set of all already processed files is tracked with persistent nodes in ZooKeeper. -- ordered — With ordered mode, the files are processed in lexicographic order. It means that if file named 'BBB' was processed at some point and later on a file named 'AAAA' is added to the bucket, it will be ignored. Only the max name (in lexicographic sense) of the successfully consumed file, and the names of files that will be retried after unsuccessful loading attempt are being stored in ZooKeeper. +- ordered — With ordered mode, the files are processed in lexicographic order. It means that if file named 'BBB' was processed at some point and later on a file named 'AA' is added to the bucket, it will be ignored. Only the max name (in lexicographic sense) of the successfully consumed file, and the names of files that will be retried after unsuccessful loading attempt are being stored in ZooKeeper. Default value: `ordered` in versions before 24.6. Starting with 24.6 there is no default value, the setting becomes required to be specified manually. For tables created on earlier versions the default value will remain `Ordered` for compatibility. From ef08055db972581b3a5b24e99fa7fa6414436bc2 Mon Sep 17 00:00:00 2001 From: Nikita Fomichev Date: Wed, 10 Jul 2024 12:47:32 +0200 Subject: [PATCH 374/417] debug tests hung on gdb error 2 --- tests/clickhouse-test | 4 +++- 1 file changed, 3 insertions(+), 1 deletion(-) diff --git a/tests/clickhouse-test b/tests/clickhouse-test index f60e285e771..d1f7ace2376 100755 --- a/tests/clickhouse-test +++ b/tests/clickhouse-test @@ -361,9 +361,11 @@ def clickhouse_execute_json( def stop_tests(): # send signal to all processes in group to avoid hung check triggering # (to avoid terminating clickhouse-test itself, the signal should be ignored) + print("Sending signals") signal.signal(signal.SIGTERM, signal.SIG_IGN) os.killpg(os.getpgid(os.getpid()), signal.SIGTERM) signal.signal(signal.SIGTERM, signal.SIG_DFL) + print("Sending signals DONE") def get_db_engine(args, database_name): @@ -2187,10 +2189,10 @@ def run_tests_array(all_tests_with_params: Tuple[List[str], int, TestSuite, bool sys.stdout.flush() while True: + print(test_case.name) test_result = test_case.run( args, test_suite, client_options, server_logs_level ) - print(test_result, test_result.case_name, test_result.need_retry) test_result = test_case.process_result(test_result, MESSAGES) if not test_result.need_retry: break From ede43721b4d43131ff2d55337bfbcc396772db44 Mon Sep 17 00:00:00 2001 From: Nikita Fomichev Date: Wed, 10 Jul 2024 13:30:06 +0200 Subject: [PATCH 375/417] debug tests hung on gdb error 4 --- docker/test/stateless/run.sh | 2 +- tests/clickhouse-test | 20 ++++++++++++++++---- 2 files changed, 17 insertions(+), 5 deletions(-) diff --git a/docker/test/stateless/run.sh b/docker/test/stateless/run.sh index 30079073ea2..b48b0731d92 100755 --- a/docker/test/stateless/run.sh +++ b/docker/test/stateless/run.sh @@ -311,7 +311,7 @@ function run_tests() set +e timeout -s TERM --preserve-status 120m clickhouse-test --testname --shard --zookeeper --check-zookeeper-session --hung-check --print-time \ - --no-drop-if-fail --test-runs "$NUM_TRIES" "${ADDITIONAL_OPTIONS[@]}" 2>&1 \ + --no-drop-if-fail --test-runs "$NUM_TRIES" "${ADDITIONAL_OPTIONS[@]}" 01710_aggregate_projections 01035_avg_weighted_long 00735_long_conditional 02911_join_on_nullsafe_optimization 02911_backup_restore_keeper_map 02552_siphash128_reference 2>&1 \ | ts '%Y-%m-%d %H:%M:%S' \ | tee -a test_output/test_result.txt set -e diff --git a/tests/clickhouse-test b/tests/clickhouse-test index d1f7ace2376..23273a0a28a 100755 --- a/tests/clickhouse-test +++ b/tests/clickhouse-test @@ -2189,7 +2189,10 @@ def run_tests_array(all_tests_with_params: Tuple[List[str], int, TestSuite, bool sys.stdout.flush() while True: - print(test_case.name) + sys.stdout.flush() + sys.stdout.write(test_case.name) + sys.stdout.flush() + test_result = test_case.run( args, test_suite, client_options, server_logs_level ) @@ -2505,10 +2508,19 @@ def do_run_tests(jobs, test_suite: TestSuite): future_seq.wait() future.wait() + print("future wait DONE") finally: - pool.terminate() - pool.close() - pool.join() + + + print("pool.join") + pool.join(timeout=2200) + print("pool.join DONE") + + # print("pool.terminate") + # pool.terminate() + # print("pool.close") + # pool.close(force=True) + # print("pool.join") if not args.run_sequential_tests_in_parallel: run_tests_array( From f96d40d82806cb195ec519f9a92a859e3e4ef873 Mon Sep 17 00:00:00 2001 From: Nikita Fomichev Date: Wed, 10 Jul 2024 13:35:19 +0200 Subject: [PATCH 376/417] debug tests hung on gdb error 5 --- tests/clickhouse-test | 14 +++++--------- 1 file changed, 5 insertions(+), 9 deletions(-) diff --git a/tests/clickhouse-test b/tests/clickhouse-test index 23273a0a28a..a90ffb79344 100755 --- a/tests/clickhouse-test +++ b/tests/clickhouse-test @@ -2510,18 +2510,14 @@ def do_run_tests(jobs, test_suite: TestSuite): future.wait() print("future wait DONE") finally: - - + print("pool.terminate") + pool.terminate() + print("pool.close") + pool.close() print("pool.join") - pool.join(timeout=2200) + pool.join() print("pool.join DONE") - # print("pool.terminate") - # pool.terminate() - # print("pool.close") - # pool.close(force=True) - # print("pool.join") - if not args.run_sequential_tests_in_parallel: run_tests_array( ( From bd62fecd317cf1e22ac51ddb45a771ff03754b2d Mon Sep 17 00:00:00 2001 From: Sema Checherinda Date: Wed, 10 Jul 2024 13:47:58 +0200 Subject: [PATCH 377/417] avoid lock-order-inversion --- src/Interpreters/DatabaseCatalog.cpp | 2 ++ 1 file changed, 2 insertions(+) diff --git a/src/Interpreters/DatabaseCatalog.cpp b/src/Interpreters/DatabaseCatalog.cpp index 889c6785217..2f83fe2ac6e 100644 --- a/src/Interpreters/DatabaseCatalog.cpp +++ b/src/Interpreters/DatabaseCatalog.cpp @@ -1314,6 +1314,7 @@ void DatabaseCatalog::dropTablesParallel(std::vectortable_id.uuid); chassert(removed); + table_to_delete_without_lock = std::move(*table_iterator); tables_marked_dropped.erase(table_iterator); wait_table_finally_dropped.notify_all(); From bde1d2fe929b37ce22812346a7df8e76b5ce8733 Mon Sep 17 00:00:00 2001 From: Sema Checherinda Date: Wed, 10 Jul 2024 13:48:52 +0200 Subject: [PATCH 378/417] less noisy logs --- src/Interpreters/DatabaseCatalog.cpp | 5 ----- src/Interpreters/DatabaseCatalog.h | 2 -- 2 files changed, 7 deletions(-) diff --git a/src/Interpreters/DatabaseCatalog.cpp b/src/Interpreters/DatabaseCatalog.cpp index 2f83fe2ac6e..964baea1891 100644 --- a/src/Interpreters/DatabaseCatalog.cpp +++ b/src/Interpreters/DatabaseCatalog.cpp @@ -838,7 +838,6 @@ DatabaseCatalog::DatabaseCatalog(ContextMutablePtr global_context_) , loading_dependencies{"LoadingDeps"} , view_dependencies{"ViewDeps"} , log(getLogger("DatabaseCatalog")) - , limitedLog(std::make_shared(log, 1, 20)) , first_async_drop_in_queue(tables_marked_dropped.end()) { } @@ -1274,10 +1273,6 @@ void DatabaseCatalog::rescheduleDropTableTask() if (first_async_drop_in_queue != tables_marked_dropped.begin()) { - LOG_TRACE( - limitedLog, - "Have {} tables in queue to drop. Some of them are being dropped in sync mode. Schedule background task ASAP", - tables_marked_dropped.size()); (*drop_task)->scheduleAfter(0); return; } diff --git a/src/Interpreters/DatabaseCatalog.h b/src/Interpreters/DatabaseCatalog.h index 73e330dedbc..23e38a6445e 100644 --- a/src/Interpreters/DatabaseCatalog.h +++ b/src/Interpreters/DatabaseCatalog.h @@ -7,7 +7,6 @@ #include #include #include -#include #include #include @@ -326,7 +325,6 @@ private: TablesDependencyGraph view_dependencies TSA_GUARDED_BY(databases_mutex); LoggerPtr log; - LogSeriesLimiterPtr limitedLog; std::atomic_bool is_shutting_down = false; From f8b9fe621a9b249764538d08dbf361e3ba4a1d49 Mon Sep 17 00:00:00 2001 From: MikhailBurdukov Date: Wed, 10 Jul 2024 12:47:19 +0000 Subject: [PATCH 379/417] Fix test --- tests/integration/test_named_collections/test.py | 3 +++ 1 file changed, 3 insertions(+) diff --git a/tests/integration/test_named_collections/test.py b/tests/integration/test_named_collections/test.py index 5d38047e885..32846c79d23 100644 --- a/tests/integration/test_named_collections/test.py +++ b/tests/integration/test_named_collections/test.py @@ -780,6 +780,9 @@ def test_keeper_storage_remove_on_cluster(cluster, ignore, expected_raise): node.query("SYSTEM RELOAD CONFIG") with expected_raise: + node.query( + "DROP NAMED COLLECTION IF EXISTS test_nc ON CLUSTER `replicated_nc_nodes_cluster`" + ) node.query( f"CREATE NAMED COLLECTION test_nc ON CLUSTER `replicated_nc_nodes_cluster` AS key1=1, key2=2 OVERRIDABLE" ) From ba176a9c4aa420a824571e818f74d370e06ec1ba Mon Sep 17 00:00:00 2001 From: Nikita Fomichev Date: Wed, 10 Jul 2024 14:51:27 +0200 Subject: [PATCH 380/417] debug tests hung on gdb error 6 --- docker/test/stateless/run.sh | 4 ++-- tests/clickhouse-test | 10 +++++++++- 2 files changed, 11 insertions(+), 3 deletions(-) diff --git a/docker/test/stateless/run.sh b/docker/test/stateless/run.sh index b48b0731d92..a3f8f9eea60 100755 --- a/docker/test/stateless/run.sh +++ b/docker/test/stateless/run.sh @@ -310,7 +310,7 @@ function run_tests() try_run_with_retry 10 clickhouse-client -q "insert into system.zookeeper (name, path, value) values ('auxiliary_zookeeper2', '/test/chroot/', '')" set +e - timeout -s TERM --preserve-status 120m clickhouse-test --testname --shard --zookeeper --check-zookeeper-session --hung-check --print-time \ + clickhouse-test --sequential --testname --shard --zookeeper --check-zookeeper-session --hung-check --print-time \ --no-drop-if-fail --test-runs "$NUM_TRIES" "${ADDITIONAL_OPTIONS[@]}" 01710_aggregate_projections 01035_avg_weighted_long 00735_long_conditional 02911_join_on_nullsafe_optimization 02911_backup_restore_keeper_map 02552_siphash128_reference 2>&1 \ | ts '%Y-%m-%d %H:%M:%S' \ | tee -a test_output/test_result.txt @@ -321,7 +321,7 @@ export -f run_tests # This should be enough to setup job and collect artifacts -TIMEOUT=$((MAX_RUN_TIME - 300)) +TIMEOUT=$((MAX_RUN_TIME - 600)) if [ "$NUM_TRIES" -gt "1" ]; then # We don't run tests with Ordinary database in PRs, only in master. # So run new/changed tests with Ordinary at least once in flaky check. diff --git a/tests/clickhouse-test b/tests/clickhouse-test index a90ffb79344..c735fc3bb04 100755 --- a/tests/clickhouse-test +++ b/tests/clickhouse-test @@ -2507,7 +2507,15 @@ def do_run_tests(jobs, test_suite: TestSuite): ) future_seq.wait() - future.wait() + while not future.ready(): + sleep(0.1) + if server_died.is_set(): + print("== Server died") + sleep(5) + pool.terminate() + pool.close() + + # future.wait() print("future wait DONE") finally: print("pool.terminate") From 211994333f363597885e8ac4d083e07ade1ec79c Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?J=C3=A1nos=20Benjamin=20Antal?= Date: Wed, 10 Jul 2024 14:56:28 +0200 Subject: [PATCH 381/417] Fix invalid XML --- tests/performance/replaceRegexp_fallback.xml | 10 +++++----- 1 file changed, 5 insertions(+), 5 deletions(-) diff --git a/tests/performance/replaceRegexp_fallback.xml b/tests/performance/replaceRegexp_fallback.xml index 926e66c702f..15dddf9cdac 100644 --- a/tests/performance/replaceRegexp_fallback.xml +++ b/tests/performance/replaceRegexp_fallback.xml @@ -1,12 +1,12 @@ -> -> + + - > + WITH 'Many years later as he faced the firing squad, Colonel Aureliano Buendia was to remember that distant afternoon when his father took him to discover ice.' AS s SELECT replaceRegexpAll(materialize(s), ' ', '\n') AS w FROM numbers(5000000) FORMAT Null WITH 'Many years later as he faced the firing squad, Colonel Aureliano Buendia was to remember that distant afternoon when his father took him to discover ice.' AS s SELECT replaceRegexpOne(materialize(s), ' ', '\n') AS w FROM numbers(5000000) FORMAT Null - > - > + + WITH 'Many years later as he faced the firing squad, Colonel Aureliano Buendia was to remember that distant afternoon when his father took him to discover ice.' AS s SELECT replaceRegexpAll(materialize(s), '\s+', '\\0\n') AS w FROM numbers(500000) FORMAT Null WITH 'Many years later as he faced the firing squad, Colonel Aureliano Buendia was to remember that distant afternoon when his father took him to discover ice.' AS s SELECT replaceRegexpOne(materialize(s), '\s+', '\\0\n') AS w FROM numbers(500000) FORMAT Null From ccffc056818c70cf16531f680ae7531bca1b376e Mon Sep 17 00:00:00 2001 From: Antonio Andelic Date: Wed, 10 Jul 2024 15:45:17 +0200 Subject: [PATCH 382/417] Fix asan --- docker/test/stateless/utils.lib | 2 ++ 1 file changed, 2 insertions(+) diff --git a/docker/test/stateless/utils.lib b/docker/test/stateless/utils.lib index 90efb5b1a8e..c3bb8ae9ea4 100644 --- a/docker/test/stateless/utils.lib +++ b/docker/test/stateless/utils.lib @@ -2,6 +2,8 @@ # core.COMM.PID-TID sysctl kernel.core_pattern='core.%e.%p-%P' +# ASAN doesn't work with suid_dumpable=2 +sysctl fs.suid_dumpable=1 function run_with_retry() { From 83cfe164bdab9b73741cf3e1c72d727fe65c1f97 Mon Sep 17 00:00:00 2001 From: Nikita Fomichev Date: Wed, 10 Jul 2024 15:55:30 +0200 Subject: [PATCH 383/417] debug tests hung on gdb error 7 --- docker/test/stateless/run.sh | 4 ++-- tests/clickhouse-test | 6 ++---- 2 files changed, 4 insertions(+), 6 deletions(-) diff --git a/docker/test/stateless/run.sh b/docker/test/stateless/run.sh index a3f8f9eea60..406ee75ff53 100755 --- a/docker/test/stateless/run.sh +++ b/docker/test/stateless/run.sh @@ -282,7 +282,7 @@ function run_tests() else # All other configurations are OK. ADDITIONAL_OPTIONS+=('--jobs') - ADDITIONAL_OPTIONS+=('5') + ADDITIONAL_OPTIONS+=('1') fi if [[ "$RUN_SEQUENTIAL_TESTS_IN_PARALLEL" -eq 1 ]]; then @@ -310,7 +310,7 @@ function run_tests() try_run_with_retry 10 clickhouse-client -q "insert into system.zookeeper (name, path, value) values ('auxiliary_zookeeper2', '/test/chroot/', '')" set +e - clickhouse-test --sequential --testname --shard --zookeeper --check-zookeeper-session --hung-check --print-time \ + clickhouse-test --testname --shard --zookeeper --check-zookeeper-session --hung-check --print-time \ --no-drop-if-fail --test-runs "$NUM_TRIES" "${ADDITIONAL_OPTIONS[@]}" 01710_aggregate_projections 01035_avg_weighted_long 00735_long_conditional 02911_join_on_nullsafe_optimization 02911_backup_restore_keeper_map 02552_siphash128_reference 2>&1 \ | ts '%Y-%m-%d %H:%M:%S' \ | tee -a test_output/test_result.txt diff --git a/tests/clickhouse-test b/tests/clickhouse-test index c735fc3bb04..27960b56ae3 100755 --- a/tests/clickhouse-test +++ b/tests/clickhouse-test @@ -2510,12 +2510,10 @@ def do_run_tests(jobs, test_suite: TestSuite): while not future.ready(): sleep(0.1) if server_died.is_set(): - print("== Server died") + print("== Server died ==") sleep(5) - pool.terminate() - pool.close() + break - # future.wait() print("future wait DONE") finally: print("pool.terminate") From e6e8b065bce976becee39e664b77f4b7ef20adf1 Mon Sep 17 00:00:00 2001 From: Azat Khuzhin Date: Wed, 10 Jul 2024 15:58:42 +0200 Subject: [PATCH 384/417] Fix possible deadlock for jemalloc with enabled profiler ClickHouse uses JEMALLOC_PROF_LIBGCC - _Unwind_Backtrace() for jemalloc, which calls _Unwind_Backtrace() during bootstrap of jemalloc, and if, dlsym() has allocations it will lead to deadlock: 0 __lll_lock_wait (futex=futex@entry=0x16516410 , private=0) at lowlevellock.c:52 1 0x00007ffff7f9d0a3 in __GI___pthread_mutex_lock (mutex=0x16516410 ) at ../nptl/pthread_mutex_lock.c:80 2 0x000000000ba6ec23 in pthread_mutex_lock (arg=arg@entry=0x16516410 ) at ./build/RelWithDebInfo/./src/Common/ThreadFuzzer.cpp:445 3 0x0000000014215fd4 in malloc_mutex_lock_final (mutex=0x165163d0 ) at ./contrib/jemalloc/include/jemalloc/internal/mutex.h:151 4 malloc_mutex_lock_slow (mutex=0x165163d0 ) at ./build/RelWithDebInfo/./contrib/jemalloc/src/mutex.c:90 5 0x00000000141defc6 in malloc_mutex_lock (tsdn=0x0, mutex=) at ./contrib/jemalloc/include/jemalloc/internal/mutex.h:217 6 malloc_init_hard () at ./build/RelWithDebInfo/./contrib/jemalloc/src/jemalloc.c:2118 7 0x00000000141d05db in malloc_init () at ./build/RelWithDebInfo/./contrib/jemalloc/src/jemalloc.c:298 8 imalloc_init_check (sopts=, dopts=) at ./build/RelWithDebInfo/./contrib/jemalloc/src/jemalloc.c:2658 9 imalloc (sopts=, dopts=) at ./build/RelWithDebInfo/./contrib/jemalloc/src/jemalloc.c:2689 10 calloc (num=num@entry=1, size=size@entry=32) at ./build/RelWithDebInfo/./contrib/jemalloc/src/jemalloc.c:2852 11 0x00007ffff7c42c05 in _dlerror_run (operate=operate@entry=0x7ffff7c42490 , args=args@entry=0x7fffffffda40) at dlerror.c:148 12 0x00007ffff7c42525 in __dlsym (handle=, name=0xe7f568 "dl_iterate_phdr") at dlsym.c:70 13 0x000000001408e1a0 in (anonymous namespace)::getOriginalDLIteratePHDR () at ./build/RelWithDebInfo/./base/base/phdr_cache.cpp:44 14 dl_iterate_phdr (callback=0x16287a60 , data=0x7fffffffdb58) at ./build/RelWithDebInfo/./base/base/phdr_cache.cpp:64 15 0x0000000016288468 in libunwind::LocalAddressSpace::findUnwindSections (this=, targetAddr=371765377, info=...) at ./contrib/libunwind/src/AddressSpace.hpp:605 16 libunwind::UnwindCursor::setInfoBasedOnIPRegister (this=this@entry=0x7fffffffde70, isReturnAddress=false) at ./contrib/libunwind/src/UnwindCursor.hpp:2553 17 0x0000000016287617 in __unw_init_local (cursor=cursor@entry=0x7fffffffde70, context=context@entry=0x7fffffffddc8) at ./build/RelWithDebInfo/./contrib/libunwind/src/libunwind.cpp:91 18 0x000000001628b094 in _Unwind_Backtrace (callback=0x14226f40 , ref=0x0) at ./build/RelWithDebInfo/./contrib/libunwind/src/UnwindLevel1-gcc-ext.c:106 19 0x000000001421a7e0 in prof_boot2 (tsd=tsd@entry=0x7ffff7c357c0, base=) at ./build/RelWithDebInfo/./contrib/jemalloc/src/prof.c:693 20 0x00000000141df3e7 in malloc_init_hard () at ./build/RelWithDebInfo/./contrib/jemalloc/src/jemalloc.c:2151 21 0x00000000141d96f1 in malloc_init () at ./build/RelWithDebInfo/./contrib/jemalloc/src/jemalloc.c:298 22 nallocx (size=32, flags=0) at ./build/RelWithDebInfo/./contrib/jemalloc/src/jemalloc.c:4011 23 0x000000000b9fdd9b in _ZN6Memory23getActualAllocationSizeITpTkNSt3__17same_asISt11align_val_tEEJEQsr2DBE16OptionalArgumentIDpT_EEEmmS5_ (size=32) at ./src/Common/memory.h:197 24 _ZN6Memory11trackMemoryITpTkNSt3__17same_asISt11align_val_tEEJEQsr2DBE16OptionalArgumentIDpT_EEEmmR15AllocationTraceS5_ (size=32, trace=...) at ./src/Common/memory.h:208 25 operator new (size=size@entry=32) at ./build/RelWithDebInfo/./src/Common/new_delete.cpp:53 26 0x000000001540e0a6 in google::protobuf::internal::ShutdownData::get () at ./build/RelWithDebInfo/./contrib/google-protobuf/src/google/protobuf/message_lite.cc:679 27 google::protobuf::internal::OnShutdownRun (f=0x15407420 , arg=0x167a6368 ) at ./build/RelWithDebInfo/./contrib/google-protobuf/src/google/protobuf/message_lite.cc:697 28 0x0000000015407560 in global constructors keyed to 000101 () at ./contrib/google-protobuf/src/google/protobuf/generated_message_util.h:180 29 0x000000001629101d in __libc_csu_init () 30 0x00007ffff7dba010 in __libc_start_main (main=0x66e3b00 , argc=4, argv=0x7fffffffe258, init=0x16290fd0 <__libc_csu_init>, fini=, rtld_fini=, stack_end=0x7fffffffe248) at ../csu/libc-start.c:264 31 0x00000000066e302e in _start () And this is indeed what happens for glibc prio 2.34, since only in [2] the allocation had been removed [2]: https://sourceware.org/git/?p=glibc.git;a=commit;h=fada9018199c21c469ff0e731ef75c6020074ac9 But, jemalloc provides also JEMALLOC_PROF_LIBUNWIND, yes it is not llvm libunwind, but gnu one, but ClickHouse version of llvm libunwind provides unw_backtrace(), so we can use it, and it will not be called during jemalloc bootstrap, so deadlock should not be possible. Funny thing that for this deadlock the workaround is to enable background_thread, because a thread created for it will initialize TLS for __dlerror(). Signed-off-by: Azat Khuzhin --- contrib/jemalloc-cmake/CMakeLists.txt | 17 ++++++++++++----- 1 file changed, 12 insertions(+), 5 deletions(-) diff --git a/contrib/jemalloc-cmake/CMakeLists.txt b/contrib/jemalloc-cmake/CMakeLists.txt index 38ebcc8f680..1fbfd29a3bd 100644 --- a/contrib/jemalloc-cmake/CMakeLists.txt +++ b/contrib/jemalloc-cmake/CMakeLists.txt @@ -179,12 +179,19 @@ endif () target_compile_definitions(_jemalloc PRIVATE -DJEMALLOC_PROF=1) -# jemalloc provides support for two different libunwind flavors: the original HP libunwind and the one coming with gcc / g++ / libstdc++. -# The latter is identified by `JEMALLOC_PROF_LIBGCC` and uses `_Unwind_Backtrace` method instead of `unw_backtrace`. -# At the time ClickHouse uses LLVM libunwind which follows libgcc's way of backtracking. +# jemalloc provides support two unwind flavors: +# - JEMALLOC_PROF_LIBUNWIND - unw_backtrace() - gnu libunwind (compatible with llvm libunwind) +# - JEMALLOC_PROF_LIBGCC - _Unwind_Backtrace() - the original HP libunwind and the one coming with gcc / g++ / libstdc++. # -# ClickHouse has to provide `unw_backtrace` method by the means of [commit 8e2b31e](https://github.com/ClickHouse/libunwind/commit/8e2b31e766dd502f6df74909e04a7dbdf5182eb1). -target_compile_definitions (_jemalloc PRIVATE -DJEMALLOC_PROF_LIBGCC=1) +# But for JEMALLOC_PROF_LIBGCC it also calls _Unwind_Backtrace() during +# bootstraping of jemalloc, which may lead to deadlock, if the dlsym will do +# allocations somewhere (like glibc does prio 2.34, see [1]). +# +# [1]: https://sourceware.org/git/?p=glibc.git;a=commit;h=fada9018199c21c469ff0e731ef75c6020074ac9 +# +# And since ClickHouse unwind already supports unw_backtrace() we can safely +# switch to it to avoid this deadlock. +target_compile_definitions (_jemalloc PRIVATE -DJEMALLOC_PROF_LIBUNWIND=1) target_link_libraries (_jemalloc PRIVATE unwind) # for RTLD_NEXT From b7964bc13ea59b1d812b8a453ddaada4f453af0d Mon Sep 17 00:00:00 2001 From: Thom O'Connor Date: Wed, 10 Jul 2024 14:34:54 +0000 Subject: [PATCH 385/417] Update kill.md Including extra verbiage for ON CLUSTER required in ClickHouse Cloud or self-managed clusters --- docs/en/sql-reference/statements/kill.md | 3 +++ 1 file changed, 3 insertions(+) diff --git a/docs/en/sql-reference/statements/kill.md b/docs/en/sql-reference/statements/kill.md index b665ad85a09..6e18ace10c7 100644 --- a/docs/en/sql-reference/statements/kill.md +++ b/docs/en/sql-reference/statements/kill.md @@ -58,6 +58,8 @@ KILL QUERY WHERE query_id='2-857d-4a57-9ee0-327da5d60a90' KILL QUERY WHERE user='username' SYNC ``` +:::tip If you are killing a query in ClickHouse Cloud or in a self-managed cluster, then be sure to use the ```ON CLUSTER [cluster-name]``` option, in order to ensure the query is killed on all replicas::: + Read-only users can only stop their own queries. By default, the asynchronous version of queries is used (`ASYNC`), which does not wait for confirmation that queries have stopped. @@ -131,6 +133,7 @@ KILL MUTATION WHERE database = 'default' AND table = 'table' -- Cancel the specific mutation: KILL MUTATION WHERE database = 'default' AND table = 'table' AND mutation_id = 'mutation_3.txt' ``` +:::tip If you are killing a mutation in ClickHouse Cloud or in a self-managed cluster, then be sure to use the ```ON CLUSTER [cluster-name]``` option, in order to ensure the mutation is killed on all replicas::: The query is useful when a mutation is stuck and cannot finish (e.g. if some function in the mutation query throws an exception when applied to the data contained in the table). From 26627605727518710cb6f1a345d3e4e8da2ac0c9 Mon Sep 17 00:00:00 2001 From: Nikita Fomichev Date: Wed, 10 Jul 2024 16:38:54 +0200 Subject: [PATCH 386/417] debug tests hung on gdb error 8 --- docker/test/stateless/run.sh | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/docker/test/stateless/run.sh b/docker/test/stateless/run.sh index 406ee75ff53..1e9eba94c4b 100755 --- a/docker/test/stateless/run.sh +++ b/docker/test/stateless/run.sh @@ -282,7 +282,7 @@ function run_tests() else # All other configurations are OK. ADDITIONAL_OPTIONS+=('--jobs') - ADDITIONAL_OPTIONS+=('1') + ADDITIONAL_OPTIONS+=('5') fi if [[ "$RUN_SEQUENTIAL_TESTS_IN_PARALLEL" -eq 1 ]]; then @@ -311,7 +311,7 @@ function run_tests() set +e clickhouse-test --testname --shard --zookeeper --check-zookeeper-session --hung-check --print-time \ - --no-drop-if-fail --test-runs "$NUM_TRIES" "${ADDITIONAL_OPTIONS[@]}" 01710_aggregate_projections 01035_avg_weighted_long 00735_long_conditional 02911_join_on_nullsafe_optimization 02911_backup_restore_keeper_map 02552_siphash128_reference 2>&1 \ + --no-drop-if-fail --test-runs "$NUM_TRIES" "${ADDITIONAL_OPTIONS[@]}" 2>&1 \ | ts '%Y-%m-%d %H:%M:%S' \ | tee -a test_output/test_result.txt set -e From 7cff918a8348afdce60650655200bbee7a8dee00 Mon Sep 17 00:00:00 2001 From: Antonio Andelic Date: Wed, 10 Jul 2024 17:04:07 +0200 Subject: [PATCH 387/417] Fix test_parallel_replicas_custom_key --- tests/integration/test_parallel_replicas_custom_key/test.py | 3 +++ 1 file changed, 3 insertions(+) diff --git a/tests/integration/test_parallel_replicas_custom_key/test.py b/tests/integration/test_parallel_replicas_custom_key/test.py index affa3f32cbe..375fe58d741 100644 --- a/tests/integration/test_parallel_replicas_custom_key/test.py +++ b/tests/integration/test_parallel_replicas_custom_key/test.py @@ -161,6 +161,9 @@ def test_parallel_replicas_custom_key_replicatedmergetree( insert_data("test_table_for_rmt", row_num, all_nodes=False) + for node in nodes: + node.query("SYSTEM SYNC REPLICA test_table_for_rmt LIGHTWEIGHT") + expected_result = "" for i in range(4): expected_result += f"{i}\t250\n" From e6b3368dc152059dd0b5367d1279585203c150f9 Mon Sep 17 00:00:00 2001 From: Nikita Fomichev Date: Wed, 10 Jul 2024 17:09:50 +0200 Subject: [PATCH 388/417] debug tests hung on gdb error 9 --- tests/clickhouse-test | 10 +++++----- 1 file changed, 5 insertions(+), 5 deletions(-) diff --git a/tests/clickhouse-test b/tests/clickhouse-test index 27960b56ae3..af4f60a50c5 100755 --- a/tests/clickhouse-test +++ b/tests/clickhouse-test @@ -2189,10 +2189,6 @@ def run_tests_array(all_tests_with_params: Tuple[List[str], int, TestSuite, bool sys.stdout.flush() while True: - sys.stdout.flush() - sys.stdout.write(test_case.name) - sys.stdout.flush() - test_result = test_case.run( args, test_suite, client_options, server_logs_level ) @@ -2505,7 +2501,11 @@ def do_run_tests(jobs, test_suite: TestSuite): ) ], ) - future_seq.wait() + while not future_seq.ready(): + sleep(0.1) + if server_died.is_set(): + sleep(5) + break while not future.ready(): sleep(0.1) From e5804ae5cd64e5a2972d9d5039a8d2df0f9f70b3 Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?J=C3=A1nos=20Benjamin=20Antal?= Date: Wed, 10 Jul 2024 17:28:10 +0200 Subject: [PATCH 389/417] Use `numbers_mt` to reduce duration of tests --- tests/performance/replaceRegexp_fallback.xml | 8 ++++---- 1 file changed, 4 insertions(+), 4 deletions(-) diff --git a/tests/performance/replaceRegexp_fallback.xml b/tests/performance/replaceRegexp_fallback.xml index 15dddf9cdac..509257efeb5 100644 --- a/tests/performance/replaceRegexp_fallback.xml +++ b/tests/performance/replaceRegexp_fallback.xml @@ -2,11 +2,11 @@ - WITH 'Many years later as he faced the firing squad, Colonel Aureliano Buendia was to remember that distant afternoon when his father took him to discover ice.' AS s SELECT replaceRegexpAll(materialize(s), ' ', '\n') AS w FROM numbers(5000000) FORMAT Null - WITH 'Many years later as he faced the firing squad, Colonel Aureliano Buendia was to remember that distant afternoon when his father took him to discover ice.' AS s SELECT replaceRegexpOne(materialize(s), ' ', '\n') AS w FROM numbers(5000000) FORMAT Null + WITH 'Many years later as he faced the firing squad, Colonel Aureliano Buendia was to remember that distant afternoon when his father took him to discover ice.' AS s SELECT replaceRegexpAll(materialize(s), ' ', '\n') AS w FROM numbers_mt(5000000) FORMAT Null + WITH 'Many years later as he faced the firing squad, Colonel Aureliano Buendia was to remember that distant afternoon when his father took him to discover ice.' AS s SELECT replaceRegexpOne(materialize(s), ' ', '\n') AS w FROM numbers_mt(5000000) FORMAT Null - WITH 'Many years later as he faced the firing squad, Colonel Aureliano Buendia was to remember that distant afternoon when his father took him to discover ice.' AS s SELECT replaceRegexpAll(materialize(s), '\s+', '\\0\n') AS w FROM numbers(500000) FORMAT Null - WITH 'Many years later as he faced the firing squad, Colonel Aureliano Buendia was to remember that distant afternoon when his father took him to discover ice.' AS s SELECT replaceRegexpOne(materialize(s), '\s+', '\\0\n') AS w FROM numbers(500000) FORMAT Null + WITH 'Many years later as he faced the firing squad, Colonel Aureliano Buendia was to remember that distant afternoon when his father took him to discover ice.' AS s SELECT replaceRegexpAll(materialize(s), '\s+', '\\0\n') AS w FROM numbers_mt(500000) FORMAT Null + WITH 'Many years later as he faced the firing squad, Colonel Aureliano Buendia was to remember that distant afternoon when his father took him to discover ice.' AS s SELECT replaceRegexpOne(materialize(s), '\s+', '\\0\n') AS w FROM numbers_mt(500000) FORMAT Null From 481a1570aab5c2cfaf868011ab68d4d16fff9d4e Mon Sep 17 00:00:00 2001 From: Nikita Fomichev Date: Wed, 10 Jul 2024 19:16:16 +0200 Subject: [PATCH 390/417] Run sequential tests in parallel to non-sequential --- docker/test/stateless/run.sh | 3 +-- 1 file changed, 1 insertion(+), 2 deletions(-) diff --git a/docker/test/stateless/run.sh b/docker/test/stateless/run.sh index 1e9eba94c4b..5747ead7986 100755 --- a/docker/test/stateless/run.sh +++ b/docker/test/stateless/run.sh @@ -12,8 +12,7 @@ MAX_RUN_TIME=$((MAX_RUN_TIME == 0 ? 10800 : MAX_RUN_TIME)) USE_DATABASE_REPLICATED=${USE_DATABASE_REPLICATED:=0} USE_SHARED_CATALOG=${USE_SHARED_CATALOG:=0} -# disable for now -RUN_SEQUENTIAL_TESTS_IN_PARALLEL=0 +RUN_SEQUENTIAL_TESTS_IN_PARALLEL=1 if [[ "$USE_DATABASE_REPLICATED" -eq 1 ]] || [[ "$USE_SHARED_CATALOG" -eq 1 ]]; then RUN_SEQUENTIAL_TESTS_IN_PARALLEL=0 From 9c0610ec2bf454ba4740a2117fb5b0d03510607f Mon Sep 17 00:00:00 2001 From: Blargian Date: Wed, 10 Jul 2024 21:27:15 +0200 Subject: [PATCH 391/417] add remaining window functions --- .../window-functions/first_value.md | 72 +++++++++++++++++++ .../sql-reference/window-functions/index.md | 4 +- .../window-functions/last_value.md | 72 +++++++++++++++++++ .../window-functions/leadInFrame.md | 2 +- .../window-functions/nth_value.md | 24 +++---- .../en/sql-reference/window-functions/rank.md | 2 +- .../window-functions/row_number.md | 2 +- 7 files changed, 161 insertions(+), 17 deletions(-) create mode 100644 docs/en/sql-reference/window-functions/first_value.md create mode 100644 docs/en/sql-reference/window-functions/last_value.md diff --git a/docs/en/sql-reference/window-functions/first_value.md b/docs/en/sql-reference/window-functions/first_value.md new file mode 100644 index 00000000000..575a6fc3f48 --- /dev/null +++ b/docs/en/sql-reference/window-functions/first_value.md @@ -0,0 +1,72 @@ +--- +slug: /en/sql-reference/window-functions/first_value +sidebar_label: first_value +sidebar_position: 3 +--- + +# first_value + +Returns the first non-NULL value evaluated within its ordered frame. + +**Syntax** + +```sql +first_value (column_name) + OVER ([[PARTITION BY grouping_column] [ORDER BY sorting_column] + [ROWS or RANGE expression_to_bound_rows_withing_the_group]] | [window_name]) +FROM table_name +WINDOW window_name as ([[PARTITION BY grouping_column] [ORDER BY sorting_column]) +``` + +For more detail on window function syntax see: [Window Functions - Syntax](./index.md/#syntax). + +**Returned value** + +- The first non-NULL value evaluated within its ordered frame. + +**Example** + +In this example the `first_value` function is used to find the highest paid footballer from a fictional dataset of salaries of Premier League football players. + +Query: + +```sql +DROP TABLE IF EXISTS salaries; +CREATE TABLE salaries +( + `team` String, + `player` String, + `salary` UInt32, + `position` String +) +Engine = Memory; + +INSERT INTO salaries FORMAT Values + ('Port Elizabeth Barbarians', 'Gary Chen', 196000, 'F'), + ('New Coreystad Archdukes', 'Charles Juarez', 190000, 'F'), + ('Port Elizabeth Barbarians', 'Michael Stanley', 100000, 'D'), + ('New Coreystad Archdukes', 'Scott Harrison', 180000, 'D'), + ('Port Elizabeth Barbarians', 'Robert George', 195000, 'M'), + ('South Hampton Seagulls', 'Douglas Benson', 150000, 'M'), + ('South Hampton Seagulls', 'James Henderson', 140000, 'M'); +``` + +```sql +SELECT player, salary, + first_value(player) OVER (ORDER BY salary DESC) AS highest_paid_player +FROM salaries; +``` + +Result: + +```response + ┌─player──────────┬─salary─┬─highest_paid_player─┐ +1. │ Gary Chen │ 196000 │ Gary Chen │ +2. │ Robert George │ 195000 │ Gary Chen │ +3. │ Charles Juarez │ 190000 │ Gary Chen │ +4. │ Scott Harrison │ 180000 │ Gary Chen │ +5. │ Douglas Benson │ 150000 │ Gary Chen │ +6. │ James Henderson │ 140000 │ Gary Chen │ +7. │ Michael Stanley │ 100000 │ Gary Chen │ + └─────────────────┴────────┴─────────────────────┘ +``` \ No newline at end of file diff --git a/docs/en/sql-reference/window-functions/index.md b/docs/en/sql-reference/window-functions/index.md index ee54a679ba1..d18dbcc189d 100644 --- a/docs/en/sql-reference/window-functions/index.md +++ b/docs/en/sql-reference/window-functions/index.md @@ -76,8 +76,8 @@ WINDOW window_name as ([[PARTITION BY grouping_column] [ORDER BY sorting_column] These functions can be used only as a window function. - [`row_number()`](./row_number.md) - Number the current row within its partition starting from 1. -- `first_value(x)` - Return the first non-NULL value evaluated within its ordered frame. -- `last_value(x)` - Return the last non-NULL value evaluated within its ordered frame. +- [`first_value(x)`](./first_value.md) - Return the first non-NULL value evaluated within its ordered frame. +- [`last_value(x)`](./last_value.md) - Return the last non-NULL value evaluated within its ordered frame. - [`nth_value(x, offset)`](./nth_value.md) - Return the first non-NULL value evaluated against the nth row (offset) in its ordered frame. - [`rank()`](./rank.md) - Rank the current row within its partition with gaps. - [`dense_rank()`](./dense_rank.md) - Rank the current row within its partition without gaps. diff --git a/docs/en/sql-reference/window-functions/last_value.md b/docs/en/sql-reference/window-functions/last_value.md new file mode 100644 index 00000000000..098ee81ceb3 --- /dev/null +++ b/docs/en/sql-reference/window-functions/last_value.md @@ -0,0 +1,72 @@ +--- +slug: /en/sql-reference/window-functions/lagInFrame +sidebar_label: lagInFrame +sidebar_position: 4 +--- + +# first_value + +Return the last non-NULL value evaluated within its ordered frame. + +**Syntax** + +```sql +first_value (column_name) + OVER ([[PARTITION BY grouping_column] [ORDER BY sorting_column] + [ROWS or RANGE expression_to_bound_rows_withing_the_group]] | [window_name]) +FROM table_name +WINDOW window_name as ([[PARTITION BY grouping_column] [ORDER BY sorting_column]) +``` + +For more detail on window function syntax see: [Window Functions - Syntax](./index.md/#syntax). + +**Returned value** + +- The last non-NULL value evaluated within its ordered frame. + +**Example** + +In this example the `last_value` function is used to find the highest paid footballer from a fictional dataset of salaries of Premier League football players. + +Query: + +```sql +DROP TABLE IF EXISTS salaries; +CREATE TABLE salaries +( + `team` String, + `player` String, + `salary` UInt32, + `position` String +) +Engine = Memory; + +INSERT INTO salaries FORMAT Values + ('Port Elizabeth Barbarians', 'Gary Chen', 196000, 'F'), + ('New Coreystad Archdukes', 'Charles Juarez', 190000, 'F'), + ('Port Elizabeth Barbarians', 'Michael Stanley', 100000, 'D'), + ('New Coreystad Archdukes', 'Scott Harrison', 180000, 'D'), + ('Port Elizabeth Barbarians', 'Robert George', 195000, 'M'), + ('South Hampton Seagulls', 'Douglas Benson', 150000, 'M'), + ('South Hampton Seagulls', 'James Henderson', 140000, 'M'); +``` + +```sql +SELECT player, salary, + last_value(player) OVER (ORDER BY salary DESC RANGE BETWEEN UNBOUNDED PRECEDING AND UNBOUNDED FOLLOWING) AS lowest_paid_player +FROM salaries; +``` + +Result: + +```response + ┌─player──────────┬─salary─┬─lowest_paid_player─┐ +1. │ Gary Chen │ 196000 │ Michael Stanley │ +2. │ Robert George │ 195000 │ Michael Stanley │ +3. │ Charles Juarez │ 190000 │ Michael Stanley │ +4. │ Scott Harrison │ 180000 │ Michael Stanley │ +5. │ Douglas Benson │ 150000 │ Michael Stanley │ +6. │ James Henderson │ 140000 │ Michael Stanley │ +7. │ Michael Stanley │ 100000 │ Michael Stanley │ + └─────────────────┴────────┴────────────────────┘ +``` \ No newline at end of file diff --git a/docs/en/sql-reference/window-functions/leadInFrame.md b/docs/en/sql-reference/window-functions/leadInFrame.md index 0cb4eea52b2..33f69c0dcae 100644 --- a/docs/en/sql-reference/window-functions/leadInFrame.md +++ b/docs/en/sql-reference/window-functions/leadInFrame.md @@ -1,7 +1,7 @@ --- slug: /en/sql-reference/window-functions/leadInFrame sidebar_label: leadInFrame -sidebar_position: 4 +sidebar_position: 5 --- # leadInFrame diff --git a/docs/en/sql-reference/window-functions/nth_value.md b/docs/en/sql-reference/window-functions/nth_value.md index 26c90110aaa..5c430707009 100644 --- a/docs/en/sql-reference/window-functions/nth_value.md +++ b/docs/en/sql-reference/window-functions/nth_value.md @@ -1,7 +1,7 @@ --- slug: /en/sql-reference/window-functions/leadInFrame sidebar_label: leadInFrame -sidebar_position: 5 +sidebar_position: 6 --- # nth_value @@ -51,7 +51,7 @@ Engine = Memory; INSERT INTO salaries FORMAT Values ('Port Elizabeth Barbarians', 'Gary Chen', 195000, 'F'), ('New Coreystad Archdukes', 'Charles Juarez', 190000, 'F'), - ('Port Elizabeth Barbarians', 'Michael Stanley', 10000, 'D'), + ('Port Elizabeth Barbarians', 'Michael Stanley', 100000, 'D'), ('New Coreystad Archdukes', 'Scott Harrison', 180000, 'D'), ('Port Elizabeth Barbarians', 'Robert George', 195000, 'M'), ('South Hampton Seagulls', 'Douglas Benson', 150000, 'M'), @@ -59,19 +59,19 @@ INSERT INTO salaries FORMAT Values ``` ```sql -SELECT salary, nth_value(salary,3) OVER(ORDER BY salary DESC) FROM salaries GROUP BY salary; +SELECT player, salary, nth_value(player,3) OVER(ORDER BY salary DESC) AS third_highest_salary FROM salaries; ``` Result: ```response - ┌─player──────────┬─salary─┬─rank─┐ -1. │ Gary Chen │ 195000 │ 1 │ -2. │ Robert George │ 195000 │ 1 │ -3. │ Charles Juarez │ 190000 │ 3 │ -4. │ Douglas Benson │ 150000 │ 4 │ -5. │ Michael Stanley │ 150000 │ 4 │ -6. │ Scott Harrison │ 150000 │ 4 │ -7. │ James Henderson │ 140000 │ 7 │ - └─────────────────┴────────┴──────┘ + ┌─player──────────┬─salary─┬─third_highest_salary─┐ +1. │ Gary Chen │ 195000 │ │ +2. │ Robert George │ 195000 │ │ +3. │ Charles Juarez │ 190000 │ Charles Juarez │ +4. │ Scott Harrison │ 180000 │ Charles Juarez │ +5. │ Douglas Benson │ 150000 │ Charles Juarez │ +6. │ James Henderson │ 140000 │ Charles Juarez │ +7. │ Michael Stanley │ 100000 │ Charles Juarez │ + └─────────────────┴────────┴──────────────────────┘ ``` \ No newline at end of file diff --git a/docs/en/sql-reference/window-functions/rank.md b/docs/en/sql-reference/window-functions/rank.md index 9ac99dde6df..d7ed8d79c35 100644 --- a/docs/en/sql-reference/window-functions/rank.md +++ b/docs/en/sql-reference/window-functions/rank.md @@ -1,7 +1,7 @@ --- slug: /en/sql-reference/window-functions/rank sidebar_label: rank -sidebar_position: 6 +sidebar_position: 7 --- # rank diff --git a/docs/en/sql-reference/window-functions/row_number.md b/docs/en/sql-reference/window-functions/row_number.md index e7165d60169..485ca355f12 100644 --- a/docs/en/sql-reference/window-functions/row_number.md +++ b/docs/en/sql-reference/window-functions/row_number.md @@ -1,7 +1,7 @@ --- slug: /en/sql-reference/window-functions/row_number sidebar_label: row_number -sidebar_position: 7 +sidebar_position: 8 --- # row_number From 4d60ff6a91b9d17744a8522e3da0b850215a76d2 Mon Sep 17 00:00:00 2001 From: Blargian Date: Wed, 10 Jul 2024 21:51:14 +0200 Subject: [PATCH 392/417] small updates --- docs/en/sql-reference/window-functions/dense_rank.md | 4 ++-- docs/en/sql-reference/window-functions/lagInFrame.md | 4 ++-- docs/en/sql-reference/window-functions/last_value.md | 10 +++++----- docs/en/sql-reference/window-functions/leadInFrame.md | 4 ++-- docs/en/sql-reference/window-functions/nth_value.md | 10 ++++------ docs/en/sql-reference/window-functions/rank.md | 4 ++-- docs/en/sql-reference/window-functions/row_number.md | 4 ++-- 7 files changed, 19 insertions(+), 21 deletions(-) diff --git a/docs/en/sql-reference/window-functions/dense_rank.md b/docs/en/sql-reference/window-functions/dense_rank.md index 17ab894707e..d6445b68c55 100644 --- a/docs/en/sql-reference/window-functions/dense_rank.md +++ b/docs/en/sql-reference/window-functions/dense_rank.md @@ -1,12 +1,12 @@ --- slug: /en/sql-reference/window-functions/dense_rank sidebar_label: dense_rank -sidebar_position: 2 +sidebar_position: 7 --- # dense_rank -This window function ranks the current row within its partition without gaps. In other words, if the value of any new row encountered is equal to the value of one of the previous rows then it will receive the next successive rank without any gaps in ranking. +Ranks the current row within its partition without gaps. In other words, if the value of any new row encountered is equal to the value of one of the previous rows then it will receive the next successive rank without any gaps in ranking. The [rank](./rank.md) function provides the same behaviour, but with gaps in ranking. diff --git a/docs/en/sql-reference/window-functions/lagInFrame.md b/docs/en/sql-reference/window-functions/lagInFrame.md index b67cf252283..049e095c10f 100644 --- a/docs/en/sql-reference/window-functions/lagInFrame.md +++ b/docs/en/sql-reference/window-functions/lagInFrame.md @@ -1,12 +1,12 @@ --- slug: /en/sql-reference/window-functions/lagInFrame sidebar_label: lagInFrame -sidebar_position: 3 +sidebar_position: 8 --- # lagInFrame -Return a value evaluated at the row that is at a specified physical offset before the current row within the ordered frame. The offset parameter, if not specified, defaults to 1, meaning it will fetch the value from the next row. If the calculated row exceeds the boundaries of the window frame, the specified default value is returned. +Returns a value evaluated at the row that is at a specified physical offset row before the current row within the ordered frame. **Syntax** diff --git a/docs/en/sql-reference/window-functions/last_value.md b/docs/en/sql-reference/window-functions/last_value.md index 098ee81ceb3..99b7ca4f75a 100644 --- a/docs/en/sql-reference/window-functions/last_value.md +++ b/docs/en/sql-reference/window-functions/last_value.md @@ -1,17 +1,17 @@ --- -slug: /en/sql-reference/window-functions/lagInFrame -sidebar_label: lagInFrame +slug: /en/sql-reference/window-functions/last_value +sidebar_label: last_value sidebar_position: 4 --- -# first_value +# last_value -Return the last non-NULL value evaluated within its ordered frame. +Returns the last non-NULL value evaluated within its ordered frame. **Syntax** ```sql -first_value (column_name) +last_value (column_name) OVER ([[PARTITION BY grouping_column] [ORDER BY sorting_column] [ROWS or RANGE expression_to_bound_rows_withing_the_group]] | [window_name]) FROM table_name diff --git a/docs/en/sql-reference/window-functions/leadInFrame.md b/docs/en/sql-reference/window-functions/leadInFrame.md index 33f69c0dcae..fc1b92cc266 100644 --- a/docs/en/sql-reference/window-functions/leadInFrame.md +++ b/docs/en/sql-reference/window-functions/leadInFrame.md @@ -1,12 +1,12 @@ --- slug: /en/sql-reference/window-functions/leadInFrame sidebar_label: leadInFrame -sidebar_position: 5 +sidebar_position: 9 --- # leadInFrame -Return a value evaluated at the row that is offset rows after the current row within the ordered frame. +Returns a value evaluated at the row that is offset rows after the current row within the ordered frame. **Syntax** diff --git a/docs/en/sql-reference/window-functions/nth_value.md b/docs/en/sql-reference/window-functions/nth_value.md index 5c430707009..aa5baf651a8 100644 --- a/docs/en/sql-reference/window-functions/nth_value.md +++ b/docs/en/sql-reference/window-functions/nth_value.md @@ -1,14 +1,12 @@ --- -slug: /en/sql-reference/window-functions/leadInFrame -sidebar_label: leadInFrame -sidebar_position: 6 +slug: /en/sql-reference/window-functions/nth_value +sidebar_label: nth_value +sidebar_position: 5 --- # nth_value -Return the first non-NULL value evaluated against the nth row (offset) in its ordered frame. - -The [dense_rank](./dense_rank.md) function provides the same behaviour but without gaps in ranking. +Returns the first non-NULL value evaluated against the nth row (offset) in its ordered frame. **Syntax** diff --git a/docs/en/sql-reference/window-functions/rank.md b/docs/en/sql-reference/window-functions/rank.md index d7ed8d79c35..dff5e154151 100644 --- a/docs/en/sql-reference/window-functions/rank.md +++ b/docs/en/sql-reference/window-functions/rank.md @@ -1,12 +1,12 @@ --- slug: /en/sql-reference/window-functions/rank sidebar_label: rank -sidebar_position: 7 +sidebar_position: 6 --- # rank -This window function ranks the current row within its partition with gaps. In other words, if the value of any row it encounters is equal to the value of a previous row then it will receive the same rank as that previous row. +Ranks the current row within its partition with gaps. In other words, if the value of any row it encounters is equal to the value of a previous row then it will receive the same rank as that previous row. The rank of the next row is then equal to the rank of the previous row plus a gap equal to the number of times the previous rank was given. The [dense_rank](./dense_rank.md) function provides the same behaviour but without gaps in ranking. diff --git a/docs/en/sql-reference/window-functions/row_number.md b/docs/en/sql-reference/window-functions/row_number.md index 485ca355f12..f1c331f89a3 100644 --- a/docs/en/sql-reference/window-functions/row_number.md +++ b/docs/en/sql-reference/window-functions/row_number.md @@ -1,12 +1,12 @@ --- slug: /en/sql-reference/window-functions/row_number sidebar_label: row_number -sidebar_position: 8 +sidebar_position: 2 --- # row_number -Numbers the current row within its partition starting from 1 +Numbers the current row within its partition starting from 1. **Syntax** From 41633cabb2e055a42db9e8899947358111470cf3 Mon Sep 17 00:00:00 2001 From: Blargian Date: Wed, 10 Jul 2024 22:16:03 +0200 Subject: [PATCH 393/417] Update first_value, last_value with possibility to use RESPECT NULLS --- docs/en/sql-reference/window-functions/first_value.md | 11 +++++++++-- docs/en/sql-reference/window-functions/index.md | 4 ++-- docs/en/sql-reference/window-functions/last_value.md | 11 +++++++++-- 3 files changed, 20 insertions(+), 6 deletions(-) diff --git a/docs/en/sql-reference/window-functions/first_value.md b/docs/en/sql-reference/window-functions/first_value.md index 575a6fc3f48..17ca1cacda8 100644 --- a/docs/en/sql-reference/window-functions/first_value.md +++ b/docs/en/sql-reference/window-functions/first_value.md @@ -6,18 +6,25 @@ sidebar_position: 3 # first_value -Returns the first non-NULL value evaluated within its ordered frame. +Returns the first value evaluated within its ordered frame. By default, NULL arguments are skipped, however the `RESPECT NULLS` modifier can be used to override this behaviour. **Syntax** ```sql -first_value (column_name) +first_value (column_name) [RESPECT NULLS] OVER ([[PARTITION BY grouping_column] [ORDER BY sorting_column] [ROWS or RANGE expression_to_bound_rows_withing_the_group]] | [window_name]) FROM table_name WINDOW window_name as ([[PARTITION BY grouping_column] [ORDER BY sorting_column]) ``` +Alias: `any`. + +:::note +Using the optional modifier `RESPECT NULLS` after `first_value(column_name)` will ensure that `NULL` arguments are not skipped. +See [NULL processing](../aggregate-functions/index.md/#null-processing) for more information. +::: + For more detail on window function syntax see: [Window Functions - Syntax](./index.md/#syntax). **Returned value** diff --git a/docs/en/sql-reference/window-functions/index.md b/docs/en/sql-reference/window-functions/index.md index 712b99992ea..0c3e2ea1cb6 100644 --- a/docs/en/sql-reference/window-functions/index.md +++ b/docs/en/sql-reference/window-functions/index.md @@ -77,8 +77,8 @@ WINDOW window_name as ([[PARTITION BY grouping_column] [ORDER BY sorting_column] These functions can be used only as a window function. - [`row_number()`](./row_number.md) - Number the current row within its partition starting from 1. -- [`first_value(x)`](./first_value.md) - Return the first non-NULL value evaluated within its ordered frame. -- [`last_value(x)`](./last_value.md) - Return the last non-NULL value evaluated within its ordered frame. +- [`first_value(x)`](./first_value.md) - Return the first value evaluated within its ordered frame. +- [`last_value(x)`](./last_value.md) - Return the last value evaluated within its ordered frame. - [`nth_value(x, offset)`](./nth_value.md) - Return the first non-NULL value evaluated against the nth row (offset) in its ordered frame. - [`rank()`](./rank.md) - Rank the current row within its partition with gaps. - [`dense_rank()`](./dense_rank.md) - Rank the current row within its partition without gaps. diff --git a/docs/en/sql-reference/window-functions/last_value.md b/docs/en/sql-reference/window-functions/last_value.md index 99b7ca4f75a..9d1ce81cc57 100644 --- a/docs/en/sql-reference/window-functions/last_value.md +++ b/docs/en/sql-reference/window-functions/last_value.md @@ -6,18 +6,25 @@ sidebar_position: 4 # last_value -Returns the last non-NULL value evaluated within its ordered frame. +Returns the last value evaluated within its ordered frame. By default, NULL arguments are skipped, however the `RESPECT NULLS` modifier can be used to override this behaviour. **Syntax** ```sql -last_value (column_name) +last_value (column_name) [RESPECT NULLS] OVER ([[PARTITION BY grouping_column] [ORDER BY sorting_column] [ROWS or RANGE expression_to_bound_rows_withing_the_group]] | [window_name]) FROM table_name WINDOW window_name as ([[PARTITION BY grouping_column] [ORDER BY sorting_column]) ``` +Alias: `anyLast`. + +:::note +Using the optional modifier `RESPECT NULLS` after `first_value(column_name)` will ensure that `NULL` arguments are not skipped. +See [NULL processing](../aggregate-functions/index.md/#null-processing) for more information. +::: + For more detail on window function syntax see: [Window Functions - Syntax](./index.md/#syntax). **Returned value** From 6f3ae4a6de7762fae8dde45c0fc83ecc1296f271 Mon Sep 17 00:00:00 2001 From: Nikita Fomichev Date: Wed, 10 Jul 2024 23:57:23 +0200 Subject: [PATCH 394/417] Fix path in test --- tests/clickhouse-test | 19 ++----------------- .../02240_tskv_schema_inference_bug.sh | 2 -- 2 files changed, 2 insertions(+), 19 deletions(-) diff --git a/tests/clickhouse-test b/tests/clickhouse-test index af4f60a50c5..958dde0606f 100755 --- a/tests/clickhouse-test +++ b/tests/clickhouse-test @@ -2501,28 +2501,13 @@ def do_run_tests(jobs, test_suite: TestSuite): ) ], ) - while not future_seq.ready(): - sleep(0.1) - if server_died.is_set(): - sleep(5) - break + future_seq.wait() - while not future.ready(): - sleep(0.1) - if server_died.is_set(): - print("== Server died ==") - sleep(5) - break - - print("future wait DONE") + future.wait() finally: - print("pool.terminate") pool.terminate() - print("pool.close") pool.close() - print("pool.join") pool.join() - print("pool.join DONE") if not args.run_sequential_tests_in_parallel: run_tests_array( diff --git a/tests/queries/0_stateless/02240_tskv_schema_inference_bug.sh b/tests/queries/0_stateless/02240_tskv_schema_inference_bug.sh index ce545a27317..d4a4e54acbd 100755 --- a/tests/queries/0_stateless/02240_tskv_schema_inference_bug.sh +++ b/tests/queries/0_stateless/02240_tskv_schema_inference_bug.sh @@ -5,8 +5,6 @@ CURDIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd) # shellcheck source=../shell_config.sh . "$CURDIR"/../shell_config.sh - -USER_FILES_PATH=$($CLICKHOUSE_CLIENT_BINARY --query "select _path,_file from file('nonexist.txt', 'CSV', 'val1 char')" 2>&1 | grep Exception | awk '{gsub("/nonexist.txt","",$9); print $9}') FILE_NAME=test_02240.data DATA_FILE=${USER_FILES_PATH:?}/$FILE_NAME From 22706b89b9927045e463286c53d82c6369f68bf2 Mon Sep 17 00:00:00 2001 From: Kruglov Pavel <48961922+Avogar@users.noreply.github.com> Date: Thu, 11 Jul 2024 00:10:59 +0200 Subject: [PATCH 395/417] Try to fix links in docs --- docs/en/sql-reference/data-types/dynamic.md | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/docs/en/sql-reference/data-types/dynamic.md b/docs/en/sql-reference/data-types/dynamic.md index e50f7e6ddaa..8b3c7479f4f 100644 --- a/docs/en/sql-reference/data-types/dynamic.md +++ b/docs/en/sql-reference/data-types/dynamic.md @@ -529,10 +529,10 @@ SELECT JSONExtractKeysAndValues('{"a" : 42, "b" : "Hello", "c" : [1,2,3]}', 'Var ### Binary output format -In [RowBinary](../../interfaces/formats.md#rowbinary-rowbinary) format values of `Dynamic` type are serialized in the following format: +In [RowBinary](/docs/en/interfaces/formats.md#rowbinary-rowbinary) format values of `Dynamic` type are serialized in the following format: ```text ``` -See the [data types binary encoding specification](../../sql-reference/data-types/data-types-binary-encoding.md) +See the [data types binary encoding specification](/docs/en/sql-reference/data-types/data-types-binary-encoding.md) From 3c52651b5580034e5d42433320c0d3de70a15b4e Mon Sep 17 00:00:00 2001 From: Ilya Golshtein Date: Wed, 10 Jul 2024 11:58:45 +0000 Subject: [PATCH 396/417] s3_off_fix: initial (proper ifdef for registerStorageAzureQueue) --- src/Storages/registerStorages.cpp | 5 ++++- 1 file changed, 4 insertions(+), 1 deletion(-) diff --git a/src/Storages/registerStorages.cpp b/src/Storages/registerStorages.cpp index 9f849052071..adc1074b1fe 100644 --- a/src/Storages/registerStorages.cpp +++ b/src/Storages/registerStorages.cpp @@ -35,7 +35,6 @@ void registerStorageFuzzJSON(StorageFactory & factory); void registerStorageS3(StorageFactory & factory); void registerStorageHudi(StorageFactory & factory); void registerStorageS3Queue(StorageFactory & factory); -void registerStorageAzureQueue(StorageFactory & factory); #if USE_PARQUET void registerStorageDeltaLake(StorageFactory & factory); @@ -45,6 +44,10 @@ void registerStorageIceberg(StorageFactory & factory); #endif #endif +#if USE_AZURE_BLOB_STORAGE +void registerStorageAzureQueue(StorageFactory & factory); +#endif + #if USE_HDFS #if USE_HIVE void registerStorageHive(StorageFactory & factory); From 35850da12e0dc775f117ed060f839671306cc26b Mon Sep 17 00:00:00 2001 From: Kruglov Pavel <48961922+Avogar@users.noreply.github.com> Date: Thu, 11 Jul 2024 11:03:03 +0200 Subject: [PATCH 397/417] Update dynamic.md --- docs/en/sql-reference/data-types/dynamic.md | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/docs/en/sql-reference/data-types/dynamic.md b/docs/en/sql-reference/data-types/dynamic.md index 8b3c7479f4f..b5781a7dd62 100644 --- a/docs/en/sql-reference/data-types/dynamic.md +++ b/docs/en/sql-reference/data-types/dynamic.md @@ -1,6 +1,6 @@ --- slug: /en/sql-reference/data-types/dynamic -sidebar_position: 56 +sidebar_position: 62 sidebar_label: Dynamic --- From 595bce4945cef4ef1822e610e5352a13e654e45f Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?Ra=C3=BAl=20Mar=C3=ADn?= Date: Thu, 11 Jul 2024 12:06:04 +0200 Subject: [PATCH 398/417] Update docs/en/sql-reference/window-functions/first_value.md --- docs/en/sql-reference/window-functions/first_value.md | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/docs/en/sql-reference/window-functions/first_value.md b/docs/en/sql-reference/window-functions/first_value.md index 17ca1cacda8..4f8a9d393b1 100644 --- a/docs/en/sql-reference/window-functions/first_value.md +++ b/docs/en/sql-reference/window-functions/first_value.md @@ -11,7 +11,7 @@ Returns the first value evaluated within its ordered frame. By default, NULL arg **Syntax** ```sql -first_value (column_name) [RESPECT NULLS] +first_value (column_name) [[RESPECT NULLS] | [IGNORE NULLS]] OVER ([[PARTITION BY grouping_column] [ORDER BY sorting_column] [ROWS or RANGE expression_to_bound_rows_withing_the_group]] | [window_name]) FROM table_name From 366ed8701e0e60bc6ca54258663987d3342d7763 Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?Ra=C3=BAl=20Mar=C3=ADn?= Date: Thu, 11 Jul 2024 12:06:09 +0200 Subject: [PATCH 399/417] Update docs/en/sql-reference/window-functions/first_value.md --- docs/en/sql-reference/window-functions/first_value.md | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/docs/en/sql-reference/window-functions/first_value.md b/docs/en/sql-reference/window-functions/first_value.md index 4f8a9d393b1..30c3b1f99dc 100644 --- a/docs/en/sql-reference/window-functions/first_value.md +++ b/docs/en/sql-reference/window-functions/first_value.md @@ -29,7 +29,7 @@ For more detail on window function syntax see: [Window Functions - Syntax](./ind **Returned value** -- The first non-NULL value evaluated within its ordered frame. +- The first value evaluated within its ordered frame. **Example** From 3d96bf298ceaf030c3e863ea2fabd0a6ebe90e4c Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?Ra=C3=BAl=20Mar=C3=ADn?= Date: Thu, 11 Jul 2024 12:06:13 +0200 Subject: [PATCH 400/417] Update docs/en/sql-reference/window-functions/last_value.md --- docs/en/sql-reference/window-functions/last_value.md | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/docs/en/sql-reference/window-functions/last_value.md b/docs/en/sql-reference/window-functions/last_value.md index 9d1ce81cc57..34170226cdd 100644 --- a/docs/en/sql-reference/window-functions/last_value.md +++ b/docs/en/sql-reference/window-functions/last_value.md @@ -29,7 +29,7 @@ For more detail on window function syntax see: [Window Functions - Syntax](./ind **Returned value** -- The last non-NULL value evaluated within its ordered frame. +- The last value evaluated within its ordered frame. **Example** From 330082c3d4d1ff075e66f2eaf72c1515ad64ffdb Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?Ra=C3=BAl=20Mar=C3=ADn?= Date: Thu, 11 Jul 2024 12:06:33 +0200 Subject: [PATCH 401/417] Update docs/en/sql-reference/window-functions/last_value.md --- docs/en/sql-reference/window-functions/last_value.md | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/docs/en/sql-reference/window-functions/last_value.md b/docs/en/sql-reference/window-functions/last_value.md index 34170226cdd..dd7f5fa078a 100644 --- a/docs/en/sql-reference/window-functions/last_value.md +++ b/docs/en/sql-reference/window-functions/last_value.md @@ -11,7 +11,7 @@ Returns the last value evaluated within its ordered frame. By default, NULL argu **Syntax** ```sql -last_value (column_name) [RESPECT NULLS] +last_value (column_name) [[RESPECT NULLS] | [IGNORE NULLS]] OVER ([[PARTITION BY grouping_column] [ORDER BY sorting_column] [ROWS or RANGE expression_to_bound_rows_withing_the_group]] | [window_name]) FROM table_name From 050240d89071f750516f2a38fea1909d58095aaa Mon Sep 17 00:00:00 2001 From: kssenii Date: Thu, 11 Jul 2024 12:08:16 +0200 Subject: [PATCH 402/417] Review fix --- src/Storages/ObjectStorage/DataLakes/DeltaLakeMetadata.cpp | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/src/Storages/ObjectStorage/DataLakes/DeltaLakeMetadata.cpp b/src/Storages/ObjectStorage/DataLakes/DeltaLakeMetadata.cpp index d37bffc42c4..c896a760597 100644 --- a/src/Storages/ObjectStorage/DataLakes/DeltaLakeMetadata.cpp +++ b/src/Storages/ObjectStorage/DataLakes/DeltaLakeMetadata.cpp @@ -635,7 +635,7 @@ struct DeltaLakeMetadataImpl } const auto value = tuple[1].safeGet(); auto field = getFieldValue(value, name_and_type->type); - current_partition_columns.emplace_back(*name_and_type, field); + current_partition_columns.emplace_back(std::move(name_and_type.value()), std::move(field)); LOG_TEST(log, "Partition {} value is {} (for {})", partition_name, value, filename); } From 80ceb63f5f194c4c99aa7502f64f7770933ae18f Mon Sep 17 00:00:00 2001 From: Nikolai Kochetov Date: Thu, 11 Jul 2024 10:45:36 +0000 Subject: [PATCH 403/417] Fixing build. --- src/Common/Exception.cpp | 2 +- src/Common/Exception.h | 3 +-- 2 files changed, 2 insertions(+), 3 deletions(-) diff --git a/src/Common/Exception.cpp b/src/Common/Exception.cpp index 07bda6a75be..09ba664baef 100644 --- a/src/Common/Exception.cpp +++ b/src/Common/Exception.cpp @@ -38,7 +38,7 @@ namespace ErrorCodes extern const int CANNOT_MREMAP; } -[[noreturn]] void abortOnFailedAssertion(const String & description, const Exception::FramePointers * trace = nullptr) +void abortOnFailedAssertion(const String & description, const Exception::FramePointers * trace) { auto & logger = Poco::Logger::root(); LOG_FATAL(&logger, "Logical error: '{}'.", description); diff --git a/src/Common/Exception.h b/src/Common/Exception.h index 87ef7101cdc..68cc305e67e 100644 --- a/src/Common/Exception.h +++ b/src/Common/Exception.h @@ -25,8 +25,6 @@ namespace DB class AtomicLogger; -[[noreturn]] void abortOnFailedAssertion(const String & description); - /// This flag can be set for testing purposes - to check that no exceptions are thrown. extern bool terminate_on_any_exception; @@ -167,6 +165,7 @@ protected: mutable std::vector capture_thread_frame_pointers; }; +[[noreturn]] void abortOnFailedAssertion(const String & description, const Exception::FramePointers * trace = nullptr); std::string getExceptionStackTraceString(const std::exception & e); std::string getExceptionStackTraceString(std::exception_ptr e); From 1bc02fb71d6a27a2bc83484dec667edb48b0ab84 Mon Sep 17 00:00:00 2001 From: Nikolai Kochetov Date: Thu, 11 Jul 2024 12:39:53 +0000 Subject: [PATCH 404/417] Ignore subquery for IN in DDLLoadingDependencyVisitor --- src/Databases/DDLLoadingDependencyVisitor.cpp | 8 +++++++ .../02841_not_ready_set_constraints.reference | 1 + .../02841_not_ready_set_constraints.sql | 24 +++++++++++++++++++ 3 files changed, 33 insertions(+) diff --git a/src/Databases/DDLLoadingDependencyVisitor.cpp b/src/Databases/DDLLoadingDependencyVisitor.cpp index 40234abb20f..67bce915168 100644 --- a/src/Databases/DDLLoadingDependencyVisitor.cpp +++ b/src/Databases/DDLLoadingDependencyVisitor.cpp @@ -11,6 +11,7 @@ #include #include #include +#include #include #include #include @@ -211,6 +212,13 @@ void DDLLoadingDependencyVisitor::extractTableNameFromArgument(const ASTFunction qualified_name.database = table_identifier->getDatabaseName(); qualified_name.table = table_identifier->shortName(); } + else if (arg->as()) + { + /// Allow IN subquery. + /// Do not add tables from the subquery into dependencies, + /// because CREATE will succeed anyway. + return; + } else { assert(false); diff --git a/tests/queries/0_stateless/02841_not_ready_set_constraints.reference b/tests/queries/0_stateless/02841_not_ready_set_constraints.reference index d81cc0710eb..daaac9e3030 100644 --- a/tests/queries/0_stateless/02841_not_ready_set_constraints.reference +++ b/tests/queries/0_stateless/02841_not_ready_set_constraints.reference @@ -1 +1,2 @@ 42 +42 diff --git a/tests/queries/0_stateless/02841_not_ready_set_constraints.sql b/tests/queries/0_stateless/02841_not_ready_set_constraints.sql index ecdf4d50635..274940f50a3 100644 --- a/tests/queries/0_stateless/02841_not_ready_set_constraints.sql +++ b/tests/queries/0_stateless/02841_not_ready_set_constraints.sql @@ -17,3 +17,27 @@ ENGINE = MergeTree ORDER BY conversation; INSERT INTO t2(conversation) VALUES (42); select * from t2; + +drop table t1; + +INSERT INTO t2(conversation) VALUES (42); -- { serverError UNKNOWN_TABLE } + +drop table t2; + +CREATE TABLE t2 ( + `conversation` UInt64, + CONSTRAINT constraint_conversation CHECK conversation IN (SELECT id FROM t1) +) +ENGINE = MergeTree ORDER BY conversation; + +INSERT INTO t2(conversation) VALUES (42); -- { serverError UNKNOWN_TABLE } + +CREATE TABLE t1 ( + `id` UInt64 +) +ENGINE = MergeTree ORDER BY id; + +INSERT INTO t1(id) VALUES (42); + +INSERT INTO t2(conversation) VALUES (42); +select * from t2; From 3e9f6265195ee6952d4963409de3bd4a1f344730 Mon Sep 17 00:00:00 2001 From: Kseniia Sumarokova <54203879+kssenii@users.noreply.github.com> Date: Thu, 11 Jul 2024 15:59:34 +0200 Subject: [PATCH 405/417] Update test.py --- tests/integration/test_storage_rabbitmq/test.py | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/tests/integration/test_storage_rabbitmq/test.py b/tests/integration/test_storage_rabbitmq/test.py index 3240039ee81..f885a3507ac 100644 --- a/tests/integration/test_storage_rabbitmq/test.py +++ b/tests/integration/test_storage_rabbitmq/test.py @@ -78,13 +78,13 @@ def wait_rabbitmq_to_start(rabbitmq_docker_id, cookie, timeout=180): def kill_rabbitmq(rabbitmq_id): p = subprocess.Popen(("docker", "stop", rabbitmq_id), stdout=subprocess.PIPE) - p.communicate() + p.wait(timeout=60) return p.returncode == 0 def revive_rabbitmq(rabbitmq_id, cookie): p = subprocess.Popen(("docker", "start", rabbitmq_id), stdout=subprocess.PIPE) - p.communicate() + p.wait(timeout=60) wait_rabbitmq_to_start(rabbitmq_id, cookie) From 262c1f9e77add2349c76777d9ddabfe06895c6d3 Mon Sep 17 00:00:00 2001 From: Kruglov Pavel <48961922+Avogar@users.noreply.github.com> Date: Thu, 11 Jul 2024 17:03:13 +0200 Subject: [PATCH 406/417] Update dynamic.md --- docs/en/sql-reference/data-types/dynamic.md | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/docs/en/sql-reference/data-types/dynamic.md b/docs/en/sql-reference/data-types/dynamic.md index b5781a7dd62..d0116e7158c 100644 --- a/docs/en/sql-reference/data-types/dynamic.md +++ b/docs/en/sql-reference/data-types/dynamic.md @@ -529,7 +529,7 @@ SELECT JSONExtractKeysAndValues('{"a" : 42, "b" : "Hello", "c" : [1,2,3]}', 'Var ### Binary output format -In [RowBinary](/docs/en/interfaces/formats.md#rowbinary-rowbinary) format values of `Dynamic` type are serialized in the following format: +In [RowBinary](/docs/en/interfaces/formats.md#rowbinary) format values of `Dynamic` type are serialized in the following format: ```text From 6868708a58f39fac83382695864459eb5fcffe5b Mon Sep 17 00:00:00 2001 From: Max Kainov Date: Thu, 11 Jul 2024 11:37:26 +0000 Subject: [PATCH 407/417] CI Buddy bot to notify about CI events --- pyproject.toml | 3 +- tests/ci/.mypy.ini | 1 + tests/ci/ci.py | 11 +++++- tests/ci/ci_buddy.py | 88 ++++++++++++++++++++++++++++++++++++++++++++ tests/ci/ci_utils.py | 41 +++++++++++++++++++++ 5 files changed, 141 insertions(+), 3 deletions(-) create mode 100644 tests/ci/ci_buddy.py diff --git a/pyproject.toml b/pyproject.toml index 279d077a695..90f089afa41 100644 --- a/pyproject.toml +++ b/pyproject.toml @@ -35,10 +35,9 @@ disable = ''' broad-except, bare-except, no-else-return, - global-statement + global-statement, ''' [tool.pylint.SIMILARITIES] # due to SQL min-similarity-lines=1000 - diff --git a/tests/ci/.mypy.ini b/tests/ci/.mypy.ini index 9bc44025826..f12d27979ce 100644 --- a/tests/ci/.mypy.ini +++ b/tests/ci/.mypy.ini @@ -15,3 +15,4 @@ warn_return_any = True no_implicit_reexport = True strict_equality = True extra_checks = True +ignore_missing_imports = True \ No newline at end of file diff --git a/tests/ci/ci.py b/tests/ci/ci.py index af2f4c0a1fc..b4a3c7ec849 100644 --- a/tests/ci/ci.py +++ b/tests/ci/ci.py @@ -15,7 +15,7 @@ import upload_result_helper from build_check import get_release_or_pr from ci_config import CI from ci_metadata import CiMetadata -from ci_utils import GHActions, normalize_string +from ci_utils import GHActions, normalize_string, Shell from clickhouse_helper import ( CiLogsCredentials, ClickHouseHelper, @@ -53,6 +53,7 @@ from stopwatch import Stopwatch from tee_popen import TeePopen from ci_cache import CiCache from ci_settings import CiSettings +from ci_buddy import CIBuddy from version_helper import get_version_from_repo # pylint: disable=too-many-lines @@ -262,6 +263,8 @@ def check_missing_images_on_dockerhub( def _pre_action(s3, indata, pr_info): + print("Clear dmesg") + Shell.run("sudo dmesg --clear ||:") CommitStatusData.cleanup() JobReport.cleanup() BuildResult.cleanup() @@ -1118,6 +1121,12 @@ def main() -> int: ### POST action: start elif args.post: + if Shell.check( + "sudo dmesg -T | grep -q -e 'Out of memory: Killed process' -e 'oom_reaper: reaped process' -e 'oom-kill:constraint=CONSTRAINT_NONE'" + ): + print("WARNING: OOM while job execution") + CIBuddy().post_error("Out Of Memory") + job_report = JobReport.load() if JobReport.exist() else None if job_report: ch_helper = ClickHouseHelper() diff --git a/tests/ci/ci_buddy.py b/tests/ci/ci_buddy.py new file mode 100644 index 00000000000..d03f5d819ec --- /dev/null +++ b/tests/ci/ci_buddy.py @@ -0,0 +1,88 @@ +import json +import os + +import boto3 +import requests +from botocore.exceptions import ClientError + +from pr_info import PRInfo +from ci_utils import Shell + + +class CIBuddy: + _HEADERS = {"Content-Type": "application/json"} + + def __init__(self, dry_run=False): + self.repo = os.getenv("GITHUB_REPOSITORY", "") + self.dry_run = dry_run + res = self._get_webhooks() + self.test_channel = "" + self.dev_ci_channel = "" + if res: + self.test_channel = json.loads(res)["test_channel"] + self.dev_ci_channel = json.loads(res)["ci_channel"] + self.job_name = os.getenv("CHECK_NAME", "unknown") + pr_info = PRInfo() + self.pr_number = pr_info.number + self.head_ref = pr_info.head_ref + self.commit_url = pr_info.commit_html_url + + @staticmethod + def _get_webhooks(): + name = "ci_buddy_web_hooks" + + session = boto3.Session(region_name="us-east-1") # Replace with your region + ssm_client = session.client("ssm") + json_string = None + try: + response = ssm_client.get_parameter( + Name=name, + WithDecryption=True, # Set to True if the parameter is a SecureString + ) + json_string = response["Parameter"]["Value"] + except ClientError as e: + print(f"An error occurred: {e}") + + return json_string + + def post(self, message, dry_run=None): + if dry_run is None: + dry_run = self.dry_run + print(f"Posting slack message, dry_run [{dry_run}]") + if dry_run: + url = self.test_channel + else: + url = self.dev_ci_channel + data = {"text": message} + try: + requests.post(url, headers=self._HEADERS, data=json.dumps(data), timeout=10) + except Exception as e: + print(f"ERROR: Failed to post message, ex {e}") + + def post_error(self, error_description, job_name="", with_instance_info=True): + instance_id, instance_type = "unknown", "unknown" + if with_instance_info: + instance_id = Shell.run("ec2metadata --instance-id") or instance_id + instance_type = Shell.run("ec2metadata --instance-type") or instance_type + if not job_name: + job_name = os.getenv("CHECK_NAME", "unknown") + line_err = f":red_circle: {error_description} :red_circle:\n\n" + line_ghr = f" *Runner:* `{instance_type}`, `{instance_id}`\n" + line_job = f" *Job:* `{job_name}`\n" + line_pr_ = f" *PR:* \n" + line_br_ = f" *Branch:* `{self.head_ref}`, <{self.commit_url}|commit>\n" + message = line_err + message += line_job + if with_instance_info: + message += line_ghr + if self.pr_number > 0: + message += line_pr_ + else: + message += line_br_ + self.post(message) + + +if __name__ == "__main__": + # test + buddy = CIBuddy(dry_run=True) + buddy.post_error("Out of memory") diff --git a/tests/ci/ci_utils.py b/tests/ci/ci_utils.py index e7034d0b104..629f37289a9 100644 --- a/tests/ci/ci_utils.py +++ b/tests/ci/ci_utils.py @@ -1,4 +1,5 @@ import os +import subprocess from contextlib import contextmanager from pathlib import Path from typing import Any, Iterator, List, Union @@ -42,3 +43,43 @@ class GHActions: for line in lines: print(line) print("::endgroup::") + + +class Shell: + @classmethod + def run_strict(cls, command): + subprocess.run( + command + " 2>&1", + shell=True, + stdout=subprocess.PIPE, + stderr=subprocess.PIPE, + text=True, + check=True, + ) + + @classmethod + def run(cls, command): + res = "" + result = subprocess.run( + command, + shell=True, + stdout=subprocess.PIPE, + stderr=subprocess.PIPE, + text=True, + check=False, + ) + if result.returncode == 0: + res = result.stdout + return res.strip() + + @classmethod + def check(cls, command): + result = subprocess.run( + command + " 2>&1", + shell=True, + stdout=subprocess.PIPE, + stderr=subprocess.PIPE, + text=True, + check=False, + ) + return result.returncode == 0 From 808d875a760d81792691f4cd7c465ec2823aefa9 Mon Sep 17 00:00:00 2001 From: Kruglov Pavel <48961922+Avogar@users.noreply.github.com> Date: Thu, 11 Jul 2024 18:57:54 +0200 Subject: [PATCH 408/417] Remove links at all --- docs/en/sql-reference/data-types/dynamic.md | 4 +--- 1 file changed, 1 insertion(+), 3 deletions(-) diff --git a/docs/en/sql-reference/data-types/dynamic.md b/docs/en/sql-reference/data-types/dynamic.md index d0116e7158c..8be81471377 100644 --- a/docs/en/sql-reference/data-types/dynamic.md +++ b/docs/en/sql-reference/data-types/dynamic.md @@ -529,10 +529,8 @@ SELECT JSONExtractKeysAndValues('{"a" : 42, "b" : "Hello", "c" : [1,2,3]}', 'Var ### Binary output format -In [RowBinary](/docs/en/interfaces/formats.md#rowbinary) format values of `Dynamic` type are serialized in the following format: +In RowBinary format values of `Dynamic` type are serialized in the following format: ```text ``` - -See the [data types binary encoding specification](/docs/en/sql-reference/data-types/data-types-binary-encoding.md) From fe451ec25a3baa269cb47722e38dfd90f01b3734 Mon Sep 17 00:00:00 2001 From: Nikolai Kochetov Date: Thu, 11 Jul 2024 17:34:33 +0000 Subject: [PATCH 409/417] Fixing build. --- src/Common/Exception.cpp | 12 +++++++++--- src/Common/Exception.h | 3 ++- 2 files changed, 11 insertions(+), 4 deletions(-) diff --git a/src/Common/Exception.cpp b/src/Common/Exception.cpp index 09ba664baef..111280074dd 100644 --- a/src/Common/Exception.cpp +++ b/src/Common/Exception.cpp @@ -38,15 +38,21 @@ namespace ErrorCodes extern const int CANNOT_MREMAP; } -void abortOnFailedAssertion(const String & description, const Exception::FramePointers * trace) +void abortOnFailedAssertion(const String & description, void * const * trace, size_t trace_offset, size_t trace_size) { auto & logger = Poco::Logger::root(); LOG_FATAL(&logger, "Logical error: '{}'.", description); if (trace) - LOG_FATAL(&logger, "Stack trace (when copying this message, always include the lines below):\n\n{}", StackTrace::toString(trace->data(), 0, trace->size())); + LOG_FATAL(&logger, "Stack trace (when copying this message, always include the lines below):\n\n{}", StackTrace::toString(trace, trace_offset, trace_size)); abort(); } +void abortOnFailedAssertion(const String & description) +{ + StackTrace st; + abortOnFailedAssertion(description, st.getFramePointers().data(), st.getOffset(), st.getSize()); +} + bool terminate_on_any_exception = false; static int terminate_status_code = 128 + SIGABRT; thread_local bool update_error_statistics = true; @@ -61,7 +67,7 @@ void handle_error_code(const std::string & msg, int code, bool remote, const Exc #ifdef ABORT_ON_LOGICAL_ERROR if (code == ErrorCodes::LOGICAL_ERROR) { - abortOnFailedAssertion(msg, &trace); + abortOnFailedAssertion(msg, trace.data(), 0, trace.size()); } #endif diff --git a/src/Common/Exception.h b/src/Common/Exception.h index 68cc305e67e..a4774a89f6a 100644 --- a/src/Common/Exception.h +++ b/src/Common/Exception.h @@ -165,7 +165,8 @@ protected: mutable std::vector capture_thread_frame_pointers; }; -[[noreturn]] void abortOnFailedAssertion(const String & description, const Exception::FramePointers * trace = nullptr); +[[noreturn]] void abortOnFailedAssertion(const String & description, void * const * trace, size_t trace_offset, size_t trace_size); +[[noreturn]] void abortOnFailedAssertion(const String & description); std::string getExceptionStackTraceString(const std::exception & e); std::string getExceptionStackTraceString(std::exception_ptr e); From 8ef3fbf32333dea0be9ee3ebbd9a3c9529cb9fb6 Mon Sep 17 00:00:00 2001 From: Guspan Tanadi <36249910+guspan-tanadi@users.noreply.github.com> Date: Fri, 12 Jul 2024 07:50:35 +0700 Subject: [PATCH 410/417] docs(clickhouse-local): intended section link --- docs/en/operations/utilities/clickhouse-local.md | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/docs/en/operations/utilities/clickhouse-local.md b/docs/en/operations/utilities/clickhouse-local.md index f19643a3fa5..c20e4fc3b09 100644 --- a/docs/en/operations/utilities/clickhouse-local.md +++ b/docs/en/operations/utilities/clickhouse-local.md @@ -16,7 +16,7 @@ sidebar_label: clickhouse-local While `clickhouse-local` is a great tool for development and testing purposes, and for processing files, it is not suitable for serving end users or applications. In these scenarios, it is recommended to use the open-source [ClickHouse](https://clickhouse.com/docs/en/install). ClickHouse is a powerful OLAP database that is designed to handle large-scale analytical workloads. It provides fast and efficient processing of complex queries on large datasets, making it ideal for use in production environments where high-performance is critical. Additionally, ClickHouse offers a wide range of features such as replication, sharding, and high availability, which are essential for scaling up to handle large datasets and serving applications. If you need to handle larger datasets or serve end users or applications, we recommend using open-source ClickHouse instead of `clickhouse-local`. -Please read the docs below that show example use cases for `clickhouse-local`, such as [querying local CSVs](#query-data-in-a-csv-file-using-sql) or [reading a parquet file in S3](#query-data-in-a-parquet-file-in-aws-s3). +Please read the docs below that show example use cases for `clickhouse-local`, such as [querying local file](#query_data_in_file) or [reading a parquet file in S3](#query-data-in-a-parquet-file-in-aws-s3). ## Download clickhouse-local From b024bb736f6008de741bf6392d0b8432e8cdf16c Mon Sep 17 00:00:00 2001 From: Max K Date: Fri, 12 Jul 2024 10:14:41 +0200 Subject: [PATCH 411/417] CI: CiBuddy to post to salck channel from release branches only --- tests/ci/ci.py | 4 +++- tests/ci/ci_buddy.py | 10 +++++----- 2 files changed, 8 insertions(+), 6 deletions(-) diff --git a/tests/ci/ci.py b/tests/ci/ci.py index b4a3c7ec849..fac50d30022 100644 --- a/tests/ci/ci.py +++ b/tests/ci/ci.py @@ -1125,7 +1125,9 @@ def main() -> int: "sudo dmesg -T | grep -q -e 'Out of memory: Killed process' -e 'oom_reaper: reaped process' -e 'oom-kill:constraint=CONSTRAINT_NONE'" ): print("WARNING: OOM while job execution") - CIBuddy().post_error("Out Of Memory") + CIBuddy(dry_run=not pr_info.is_release).post_error( + "Out Of Memory", job_name=_get_ext_check_name(args.job_name) + ) job_report = JobReport.load() if JobReport.exist() else None if job_report: diff --git a/tests/ci/ci_buddy.py b/tests/ci/ci_buddy.py index d03f5d819ec..ea690bb602c 100644 --- a/tests/ci/ci_buddy.py +++ b/tests/ci/ci_buddy.py @@ -66,11 +66,11 @@ class CIBuddy: instance_type = Shell.run("ec2metadata --instance-type") or instance_type if not job_name: job_name = os.getenv("CHECK_NAME", "unknown") - line_err = f":red_circle: {error_description} :red_circle:\n\n" - line_ghr = f" *Runner:* `{instance_type}`, `{instance_id}`\n" - line_job = f" *Job:* `{job_name}`\n" - line_pr_ = f" *PR:* \n" - line_br_ = f" *Branch:* `{self.head_ref}`, <{self.commit_url}|commit>\n" + line_err = f":red_circle: *Error: {error_description}*\n\n" + line_ghr = f" *Runner:* `{instance_type}`, `{instance_id}`\n" + line_job = f" *Job:* `{job_name}`\n" + line_pr_ = f" *PR:* \n" + line_br_ = f" *Branch:* `{self.head_ref}`, <{self.commit_url}|commit>\n" message = line_err message += line_job if with_instance_info: From 7d9e1700d2a85330380c19777b788ce5a6c2f605 Mon Sep 17 00:00:00 2001 From: Blargian Date: Fri, 12 Jul 2024 11:24:24 +0200 Subject: [PATCH 412/417] update intHash32, intHash64 --- .../functions/array-functions.md | 40 ++++++++++++ .../sql-reference/functions/hash-functions.md | 63 ++++++++++++++++++- 2 files changed, 102 insertions(+), 1 deletion(-) diff --git a/docs/en/sql-reference/functions/array-functions.md b/docs/en/sql-reference/functions/array-functions.md index d87ca4a0fe7..4080dce883f 100644 --- a/docs/en/sql-reference/functions/array-functions.md +++ b/docs/en/sql-reference/functions/array-functions.md @@ -3081,3 +3081,43 @@ Result: ## Distance functions All supported functions are described in [distance functions documentation](../../sql-reference/functions/distance-functions.md). + +## kql_array_sort_asc + +Sorts an array from lowest to highest value. For use with [Kusto Query Language (KQL)](https://clickhouse.com/docs/en/guides/developer/alternative-query-languages#kusto-query-language-kql). + +:::note +For this function to work you should have Kusto enabled. To enable Kusto: + +```sql +SET dialect = 'kusto' +``` + +::: + +**Syntax** + +``` sql +kql_array_sort_asc(arr1 [, arr2 ... arrN]) +``` + +**Arguments** + +- `arr1` — [Array](../data-types/array.md) of numeric values. +- `arr1` — [Array](../data-types/array.md) of numeric values. + +**Returned value** + +- Returns an array of non-negative partial sums of elements in the source array. [UInt\*](https://clickhouse.com/docs/en/data_types/int_uint/#uint-ranges), [Int\*](https://clickhouse.com/docs/en/data_types/int_uint/#int-ranges), [Float\*](https://clickhouse.com/docs/en/data_types/float/). + +**Example** + +``` sql +SELECT arrayCumSumNonNegative([1, 1, -4, 1]) AS res +``` + +``` text +┌─res───────┐ +│ [1,2,0,1] │ +└───────────┘ +``` \ No newline at end of file diff --git a/docs/en/sql-reference/functions/hash-functions.md b/docs/en/sql-reference/functions/hash-functions.md index e431ed75465..d2ed4516fce 100644 --- a/docs/en/sql-reference/functions/hash-functions.md +++ b/docs/en/sql-reference/functions/hash-functions.md @@ -314,10 +314,71 @@ SELECT groupBitXor(cityHash64(*)) FROM table Calculates a 32-bit hash code from any type of integer. This is a relatively fast non-cryptographic hash function of average quality for numbers. +**Syntax** + +```sql +intHash32(int) +``` + +**Arguments** + +- `int` — Integer to hash. [(U)Int*](../data-types/int-uint.md). + +**Returned value** + +- 32-bit hash code. [UInt32](../data-types/int-uint.md). + +**Example** + +Query: + +```sql +SELECT intHash32(42); +``` + +Result: + +```response +┌─intHash32(42)─┐ +│ 1228623923 │ +└───────────────┘ +``` + ## intHash64 Calculates a 64-bit hash code from any type of integer. -It works faster than intHash32. Average quality. +This is a relatively fast non-cryptographic hash function of average quality for numbers. +It works faster than [intHash32](#inthash32). + +**Syntax** + +```sql +intHash32(int) +``` + +**Arguments** + +- `int` — Integer to hash. [(U)Int*](../data-types/int-uint.md). + +**Returned value** + +- 64-bit hash code. [UInt64](../data-types/int-uint.md). + +**Example** + +Query: + +```sql +SELECT intHash64(42); +``` + +Result: + +```response +┌────────intHash64(42)─┐ +│ 11490350930367293593 │ +└──────────────────────┘ +``` ## SHA1, SHA224, SHA256, SHA512, SHA512_256 From 18e411d35366a82e3a2c9a725ccfabfa6a9170b6 Mon Sep 17 00:00:00 2001 From: Blargian Date: Fri, 12 Jul 2024 11:28:41 +0200 Subject: [PATCH 413/417] remove unwanted change --- .../functions/array-functions.md | 42 +------------------ 1 file changed, 1 insertion(+), 41 deletions(-) diff --git a/docs/en/sql-reference/functions/array-functions.md b/docs/en/sql-reference/functions/array-functions.md index 4080dce883f..1b52440903d 100644 --- a/docs/en/sql-reference/functions/array-functions.md +++ b/docs/en/sql-reference/functions/array-functions.md @@ -3080,44 +3080,4 @@ Result: ## Distance functions -All supported functions are described in [distance functions documentation](../../sql-reference/functions/distance-functions.md). - -## kql_array_sort_asc - -Sorts an array from lowest to highest value. For use with [Kusto Query Language (KQL)](https://clickhouse.com/docs/en/guides/developer/alternative-query-languages#kusto-query-language-kql). - -:::note -For this function to work you should have Kusto enabled. To enable Kusto: - -```sql -SET dialect = 'kusto' -``` - -::: - -**Syntax** - -``` sql -kql_array_sort_asc(arr1 [, arr2 ... arrN]) -``` - -**Arguments** - -- `arr1` — [Array](../data-types/array.md) of numeric values. -- `arr1` — [Array](../data-types/array.md) of numeric values. - -**Returned value** - -- Returns an array of non-negative partial sums of elements in the source array. [UInt\*](https://clickhouse.com/docs/en/data_types/int_uint/#uint-ranges), [Int\*](https://clickhouse.com/docs/en/data_types/int_uint/#int-ranges), [Float\*](https://clickhouse.com/docs/en/data_types/float/). - -**Example** - -``` sql -SELECT arrayCumSumNonNegative([1, 1, -4, 1]) AS res -``` - -``` text -┌─res───────┐ -│ [1,2,0,1] │ -└───────────┘ -``` \ No newline at end of file +All supported functions are described in [distance functions documentation](../../sql-reference/functions/distance-functions.md). \ No newline at end of file From 7c6db58eec7d06ae216774b957df992c05e94454 Mon Sep 17 00:00:00 2001 From: Shaun Struwig <41984034+Blargian@users.noreply.github.com> Date: Fri, 12 Jul 2024 11:34:04 +0200 Subject: [PATCH 414/417] Update aspell-dict.txt --- utils/check-style/aspell-ignore/en/aspell-dict.txt | 2 ++ 1 file changed, 2 insertions(+) diff --git a/utils/check-style/aspell-ignore/en/aspell-dict.txt b/utils/check-style/aspell-ignore/en/aspell-dict.txt index 78c4b6bde95..ca2c4ec4192 100644 --- a/utils/check-style/aspell-ignore/en/aspell-dict.txt +++ b/utils/check-style/aspell-ignore/en/aspell-dict.txt @@ -1900,11 +1900,13 @@ kurtosis kurtpop kurtsamp laion +lagInFrame lang laravel largestTriangleThreeBuckets latencies ldap +leadInFrame leftPad leftPadUTF leftUTF From d9a05bca89f3578bb2cf965b1ce373657de0474a Mon Sep 17 00:00:00 2001 From: Blargian Date: Fri, 12 Jul 2024 11:43:04 +0200 Subject: [PATCH 415/417] add alias to anyLast_respect_nulls --- .../aggregate-functions/reference/anylast_respect_nulls.md | 2 ++ 1 file changed, 2 insertions(+) diff --git a/docs/en/sql-reference/aggregate-functions/reference/anylast_respect_nulls.md b/docs/en/sql-reference/aggregate-functions/reference/anylast_respect_nulls.md index 8f093cfdb61..a28b965f7ea 100644 --- a/docs/en/sql-reference/aggregate-functions/reference/anylast_respect_nulls.md +++ b/docs/en/sql-reference/aggregate-functions/reference/anylast_respect_nulls.md @@ -13,6 +13,8 @@ Selects the last value encountered, irregardless of whether it is `NULL` or not. anyLast_respect_nulls(column) ``` +Alias: `last_value_respect_nulls`. + **Parameters** - `column`: The column name. From 3806ab7ef1ed8f298cbe0a1d3b186a0d29e7d3a6 Mon Sep 17 00:00:00 2001 From: Blargian Date: Fri, 12 Jul 2024 12:50:19 +0200 Subject: [PATCH 416/417] remove *_respect_nulls and modify any, anyLast to reflect that they can use modifier RESPECT NULLS --- .../aggregate-functions/index.md | 2 +- .../aggregate-functions/reference/any.md | 8 ++-- .../reference/any_respect_nulls.md | 44 ------------------- .../aggregate-functions/reference/anylast.md | 8 +++- .../reference/anylast_respect_nulls.md | 41 ----------------- .../aggregate-functions/reference/index.md | 3 +- 6 files changed, 13 insertions(+), 93 deletions(-) delete mode 100644 docs/en/sql-reference/aggregate-functions/reference/any_respect_nulls.md delete mode 100644 docs/en/sql-reference/aggregate-functions/reference/anylast_respect_nulls.md diff --git a/docs/en/sql-reference/aggregate-functions/index.md b/docs/en/sql-reference/aggregate-functions/index.md index 96bf0c5d93b..5056ef2c7aa 100644 --- a/docs/en/sql-reference/aggregate-functions/index.md +++ b/docs/en/sql-reference/aggregate-functions/index.md @@ -18,7 +18,7 @@ ClickHouse also supports: During aggregation, all `NULL` arguments are skipped. If the aggregation has several arguments it will ignore any row in which one or more of them are NULL. -There is an exception to this rule, which are the functions [`first_value`](../../sql-reference/aggregate-functions/reference/first_value.md), [`last_value`](../../sql-reference/aggregate-functions/reference/last_value.md) and their aliases when followed by the modifier `RESPECT NULLS`: `FIRST_VALUE(b) RESPECT NULLS`. +There is an exception to this rule, which are the functions [`first_value`](../../sql-reference/aggregate-functions/reference/first_value.md), [`last_value`](../../sql-reference/aggregate-functions/reference/last_value.md) and their aliases (`any` and `anyLast` respectively) when followed by the modifier `RESPECT NULLS`. For example, `FIRST_VALUE(b) RESPECT NULLS`. **Examples:** diff --git a/docs/en/sql-reference/aggregate-functions/reference/any.md b/docs/en/sql-reference/aggregate-functions/reference/any.md index cdff7dde4a9..972263585f2 100644 --- a/docs/en/sql-reference/aggregate-functions/reference/any.md +++ b/docs/en/sql-reference/aggregate-functions/reference/any.md @@ -5,12 +5,12 @@ sidebar_position: 102 # any -Selects the first encountered value of a column. +Selects the first encountered value of a column, ignoring any `NULL` values. **Syntax** ```sql -any(column) +any(column) [RESPECT NULLS] ``` Aliases: `any_value`, [`first_value`](../reference/first_value.md). @@ -20,7 +20,9 @@ Aliases: `any_value`, [`first_value`](../reference/first_value.md). **Returned value** -By default, it ignores NULL values and returns the first NOT NULL value found in the column. Like [`first_value`](../../../sql-reference/aggregate-functions/reference/first_value.md) it supports `RESPECT NULLS`, in which case it will select the first value passed, independently on whether it's NULL or not. +:::note +Supports the `RESPECT NULLS` modifier after the function name. Using this modifier will ensure the function selects the first value passed, regardless of whether it is `NULL` or not. +::: :::note The return type of the function is the same as the input, except for LowCardinality which is discarded. This means that given no rows as input it will return the default value of that type (0 for integers, or Null for a Nullable() column). You might use the `-OrNull` [combinator](../../../sql-reference/aggregate-functions/combinators.md) ) to modify this behaviour. diff --git a/docs/en/sql-reference/aggregate-functions/reference/any_respect_nulls.md b/docs/en/sql-reference/aggregate-functions/reference/any_respect_nulls.md deleted file mode 100644 index 99104a9b8c7..00000000000 --- a/docs/en/sql-reference/aggregate-functions/reference/any_respect_nulls.md +++ /dev/null @@ -1,44 +0,0 @@ ---- -slug: /en/sql-reference/aggregate-functions/reference/any_respect_nulls -sidebar_position: 103 ---- - -# any_respect_nulls - -Selects the first encountered value of a column, irregardless of whether it is a `NULL` value or not. - -Alias: `any_value_respect_nulls`, `first_value_repect_nulls`. - -**Syntax** - -```sql -any_respect_nulls(column) -``` - -**Parameters** -- `column`: The column name. - -**Returned value** - -- The last value encountered, irregardless of whether it is a `NULL` value or not. - -**Example** - -Query: - -```sql -CREATE TABLE any_nulls (city Nullable(String)) ENGINE=Log; - -INSERT INTO any_nulls (city) VALUES (NULL), ('Amsterdam'), ('New York'), ('Tokyo'), ('Valencia'), (NULL); - -SELECT any(city), any_respect_nulls(city) FROM any_nulls; -``` - -```response -┌─any(city)─┬─any_respect_nulls(city)─┐ -│ Amsterdam │ ᴺᵁᴸᴸ │ -└───────────┴─────────────────────────┘ -``` - -**See Also** -- [any](../reference/any.md) diff --git a/docs/en/sql-reference/aggregate-functions/reference/anylast.md b/docs/en/sql-reference/aggregate-functions/reference/anylast.md index e43bc07fbdc..202d2e9fb10 100644 --- a/docs/en/sql-reference/aggregate-functions/reference/anylast.md +++ b/docs/en/sql-reference/aggregate-functions/reference/anylast.md @@ -5,17 +5,21 @@ sidebar_position: 105 # anyLast -Selects the last value encountered. The result is just as indeterminate as for the [any](../../../sql-reference/aggregate-functions/reference/any.md) function. +Selects the last value encountered, ignoring any `NULL` values by default. The result is just as indeterminate as for the [any](../../../sql-reference/aggregate-functions/reference/any.md) function. **Syntax** ```sql -anyLast(column) +anyLast(column) [RESPECT NULLS] ``` **Parameters** - `column`: The column name. +:::note +Supports the `RESPECT NULLS` modifier after the function name. Using this modifier will ensure the function selects the first value passed, regardless of whether it is `NULL` or not. +::: + **Returned value** - The last value encountered. diff --git a/docs/en/sql-reference/aggregate-functions/reference/anylast_respect_nulls.md b/docs/en/sql-reference/aggregate-functions/reference/anylast_respect_nulls.md deleted file mode 100644 index a28b965f7ea..00000000000 --- a/docs/en/sql-reference/aggregate-functions/reference/anylast_respect_nulls.md +++ /dev/null @@ -1,41 +0,0 @@ ---- -slug: /en/sql-reference/aggregate-functions/reference/anylast_respect_nulls -sidebar_position: 106 ---- - -# anyLast_respect_nulls - -Selects the last value encountered, irregardless of whether it is `NULL` or not. - -**Syntax** - -```sql -anyLast_respect_nulls(column) -``` - -Alias: `last_value_respect_nulls`. - -**Parameters** -- `column`: The column name. - -**Returned value** - -- The last value encountered, irregardless of whether it is `NULL` or not. - -**Example** - -Query: - -```sql -CREATE TABLE any_last_nulls (city Nullable(String)) ENGINE=Log; - -INSERT INTO any_last_nulls (city) VALUES ('Amsterdam'),(NULL),('New York'),('Tokyo'),('Valencia'),(NULL); - -SELECT anyLast(city), anyLast_respect_nulls(city) FROM any_last_nulls; -``` - -```response -┌─anyLast(city)─┬─anyLast_respect_nulls(city)─┐ -│ Valencia │ ᴺᵁᴸᴸ │ -└───────────────┴─────────────────────────────┘ -``` \ No newline at end of file diff --git a/docs/en/sql-reference/aggregate-functions/reference/index.md b/docs/en/sql-reference/aggregate-functions/reference/index.md index e3725b6a430..323a99d276f 100644 --- a/docs/en/sql-reference/aggregate-functions/reference/index.md +++ b/docs/en/sql-reference/aggregate-functions/reference/index.md @@ -44,10 +44,9 @@ Standard aggregate functions: ClickHouse-specific aggregate functions: - [analysisOfVariance](../reference/analysis_of_variance.md) -- [any](../reference/any_respect_nulls.md) +- [any](../reference/any.md) - [anyHeavy](../reference/anyheavy.md) - [anyLast](../reference/anylast.md) -- [anyLast](../reference/anylast_respect_nulls.md) - [boundingRatio](../reference/boundrat.md) - [first_value](../reference/first_value.md) - [last_value](../reference/last_value.md) From 633db10d397a030b9e0f5aa4435fe5d6c002b54f Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?Ra=C3=BAl=20Mar=C3=ADn?= Date: Fri, 12 Jul 2024 12:58:14 +0200 Subject: [PATCH 417/417] Update docs/en/sql-reference/functions/hash-functions.md --- docs/en/sql-reference/functions/hash-functions.md | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/docs/en/sql-reference/functions/hash-functions.md b/docs/en/sql-reference/functions/hash-functions.md index d2ed4516fce..7c977e7d6dc 100644 --- a/docs/en/sql-reference/functions/hash-functions.md +++ b/docs/en/sql-reference/functions/hash-functions.md @@ -353,7 +353,7 @@ It works faster than [intHash32](#inthash32). **Syntax** ```sql -intHash32(int) +intHash64(int) ``` **Arguments**