From e3961c118a018c5bbb4e76d93a6be57636ef61bd Mon Sep 17 00:00:00 2001 From: alesapin Date: Mon, 27 Feb 2023 12:27:57 +0100 Subject: [PATCH 001/103] Revert "Revert "Fix several `RENAME COLUMN` bugs."" --- src/Storages/MergeTree/AlterConversions.cpp | 55 +++++ src/Storages/MergeTree/AlterConversions.h | 17 +- .../MergeTree/DataPartStorageOnDiskBase.cpp | 1 + src/Storages/MergeTree/DataPartsExchange.cpp | 16 +- src/Storages/MergeTree/IMergeTreeDataPart.cpp | 43 +++- src/Storages/MergeTree/IMergeTreeDataPart.h | 19 +- src/Storages/MergeTree/MergeTask.cpp | 2 +- src/Storages/MergeTree/MergeTreeData.cpp | 29 ++- src/Storages/MergeTree/MergeTreeData.h | 2 +- .../MergeTree/MergeTreeDataPartInMemory.cpp | 4 +- .../MergeTree/MergeTreeDataWriter.cpp | 4 +- .../MergeTree/MergeTreeMarksLoader.cpp | 16 +- .../MergeTree/MergeTreeWriteAheadLog.cpp | 2 +- .../MergeTree/MergedBlockOutputStream.cpp | 10 +- .../MergedColumnOnlyOutputStream.cpp | 2 +- src/Storages/MergeTree/MutateTask.cpp | 216 ++++++++++++++---- .../ReplicatedMergeTreeAttachThread.cpp | 2 +- .../MergeTree/ReplicatedMergeTreeQueue.cpp | 51 ++++- .../MergeTree/ReplicatedMergeTreeQueue.h | 6 +- src/Storages/MutationCommands.cpp | 16 ++ src/Storages/MutationCommands.h | 8 + src/Storages/StorageInMemoryMetadata.cpp | 14 ++ src/Storages/StorageInMemoryMetadata.h | 14 +- src/Storages/StorageMergeTree.cpp | 44 +++- src/Storages/StorageMergeTree.h | 2 +- src/Storages/StorageReplicatedMergeTree.cpp | 37 +-- src/Storages/StorageReplicatedMergeTree.h | 11 +- .../integration/test_merge_tree_hdfs/test.py | 14 +- tests/integration/test_merge_tree_s3/test.py | 15 +- .../test_merge_tree_s3_failover/test.py | 2 +- tests/integration/test_partition/test.py | 2 + .../test_replicated_merge_tree_s3/test.py | 14 +- .../test.py | 14 +- .../integration/test_s3_zero_copy_ttl/test.py | 4 +- .../01278_alter_rename_combination.reference | 4 +- .../01278_alter_rename_combination.sql | 2 +- ...1_alter_rename_and_other_renames.reference | 6 +- .../01281_alter_rename_and_other_renames.sql | 2 +- ...system_cache_on_write_operations.reference | 40 ++-- .../0_stateless/02361_fsync_profile_events.sh | 4 +- .../02538_alter_rename_sequence.reference | 8 + .../02538_alter_rename_sequence.sql | 59 +++++ .../02543_alter_rename_modify_stuck.reference | 1 + .../02543_alter_rename_modify_stuck.sh | 58 +++++ .../02555_davengers_rename_chain.reference | 26 +++ .../02555_davengers_rename_chain.sh | 143 ++++++++++++ 46 files changed, 892 insertions(+), 169 deletions(-) create mode 100644 src/Storages/MergeTree/AlterConversions.cpp create mode 100644 tests/queries/0_stateless/02538_alter_rename_sequence.reference create mode 100644 tests/queries/0_stateless/02538_alter_rename_sequence.sql create mode 100644 tests/queries/0_stateless/02543_alter_rename_modify_stuck.reference create mode 100755 tests/queries/0_stateless/02543_alter_rename_modify_stuck.sh create mode 100644 tests/queries/0_stateless/02555_davengers_rename_chain.reference create mode 100755 tests/queries/0_stateless/02555_davengers_rename_chain.sh diff --git a/src/Storages/MergeTree/AlterConversions.cpp b/src/Storages/MergeTree/AlterConversions.cpp new file mode 100644 index 00000000000..7a298b0f6ca --- /dev/null +++ b/src/Storages/MergeTree/AlterConversions.cpp @@ -0,0 +1,55 @@ +#include +#include + +namespace DB +{ + +namespace ErrorCodes +{ + extern const int LOGICAL_ERROR; +} + +bool AlterConversions::columnHasNewName(const std::string & old_name) const +{ + for (const auto & [new_name, prev_name] : rename_map) + { + if (old_name == prev_name) + return true; + } + + return false; +} + +std::string AlterConversions::getColumnNewName(const std::string & old_name) const +{ + for (const auto & [new_name, prev_name] : rename_map) + { + if (old_name == prev_name) + return new_name; + } + + throw Exception(ErrorCodes::LOGICAL_ERROR, "Column {} was not renamed", old_name); +} + + +bool AlterConversions::isColumnRenamed(const std::string & new_name) const +{ + for (const auto & [name_to, name_from] : rename_map) + { + if (name_to == new_name) + return true; + } + return false; +} +/// Get column old name before rename (lookup by key in rename_map) +std::string AlterConversions::getColumnOldName(const std::string & new_name) const +{ + for (const auto & [name_to, name_from] : rename_map) + { + if (name_to == new_name) + return name_from; + } + throw Exception(ErrorCodes::LOGICAL_ERROR, "Column {} was not renamed", new_name); +} + +} diff --git a/src/Storages/MergeTree/AlterConversions.h b/src/Storages/MergeTree/AlterConversions.h index 0d58499d424..ada385d6100 100644 --- a/src/Storages/MergeTree/AlterConversions.h +++ b/src/Storages/MergeTree/AlterConversions.h @@ -14,11 +14,22 @@ namespace DB /// part->getColumns() and storage->getColumns(). struct AlterConversions { + struct RenamePair + { + std::string rename_to; + std::string rename_from; + }; /// Rename map new_name -> old_name - std::unordered_map rename_map; + std::vector rename_map; - bool isColumnRenamed(const std::string & new_name) const { return rename_map.count(new_name) > 0; } - std::string getColumnOldName(const std::string & new_name) const { return rename_map.at(new_name); } + /// Column was renamed (lookup by value in rename_map) + bool columnHasNewName(const std::string & old_name) const; + /// Get new name for column (lookup by value in rename_map) + std::string getColumnNewName(const std::string & old_name) const; + /// Is this name is new name of column (lookup by key in rename_map) + bool isColumnRenamed(const std::string & new_name) const; + /// Get column old name before rename (lookup by key in rename_map) + std::string getColumnOldName(const std::string & new_name) const; }; } diff --git a/src/Storages/MergeTree/DataPartStorageOnDiskBase.cpp b/src/Storages/MergeTree/DataPartStorageOnDiskBase.cpp index d8769a94347..260bf524ed2 100644 --- a/src/Storages/MergeTree/DataPartStorageOnDiskBase.cpp +++ b/src/Storages/MergeTree/DataPartStorageOnDiskBase.cpp @@ -683,6 +683,7 @@ void DataPartStorageOnDiskBase::clearDirectory( request.emplace_back(fs::path(dir) / "default_compression_codec.txt", true); request.emplace_back(fs::path(dir) / "delete-on-destroy.txt", true); request.emplace_back(fs::path(dir) / "txn_version.txt", true); + request.emplace_back(fs::path(dir) / "metadata_version.txt", true); /// Inverted index request.emplace_back(fs::path(dir) / "skp_idx_af.gin_dict", true); diff --git a/src/Storages/MergeTree/DataPartsExchange.cpp b/src/Storages/MergeTree/DataPartsExchange.cpp index b671106f46a..987ad2f2fe5 100644 --- a/src/Storages/MergeTree/DataPartsExchange.cpp +++ b/src/Storages/MergeTree/DataPartsExchange.cpp @@ -63,8 +63,9 @@ constexpr auto REPLICATION_PROTOCOL_VERSION_WITH_PARTS_DEFAULT_COMPRESSION = 4; constexpr auto REPLICATION_PROTOCOL_VERSION_WITH_PARTS_UUID = 5; constexpr auto REPLICATION_PROTOCOL_VERSION_WITH_PARTS_ZERO_COPY = 6; constexpr auto REPLICATION_PROTOCOL_VERSION_WITH_PARTS_PROJECTION = 7; +constexpr auto REPLICATION_PROTOCOL_VERSION_WITH_METADATA_VERSION = 8; // Reserved for ALTER PRIMARY KEY -// constexpr auto REPLICATION_PROTOCOL_VERSION_WITH_PARTS_PRIMARY_KEY = 8; +// constexpr auto REPLICATION_PROTOCOL_VERSION_WITH_PARTS_PRIMARY_KEY = 9; std::string getEndpointId(const std::string & node_id) { @@ -120,7 +121,7 @@ void Service::processQuery(const HTMLForm & params, ReadBuffer & /*body*/, Write MergeTreePartInfo::fromPartName(part_name, data.format_version); /// We pretend to work as older server version, to be sure that client will correctly process our version - response.addCookie({"server_protocol_version", toString(std::min(client_protocol_version, REPLICATION_PROTOCOL_VERSION_WITH_PARTS_PROJECTION))}); + response.addCookie({"server_protocol_version", toString(std::min(client_protocol_version, REPLICATION_PROTOCOL_VERSION_WITH_METADATA_VERSION))}); LOG_TRACE(log, "Sending part {}", part_name); @@ -280,6 +281,10 @@ MergeTreeData::DataPart::Checksums Service::sendPartFromDisk( && name == IMergeTreeDataPart::DEFAULT_COMPRESSION_CODEC_FILE_NAME) continue; + if (client_protocol_version < REPLICATION_PROTOCOL_VERSION_WITH_METADATA_VERSION + && name == IMergeTreeDataPart::METADATA_VERSION_FILE_NAME) + continue; + files_to_replicate.insert(name); } @@ -407,7 +412,7 @@ MergeTreeData::MutableDataPartPtr Fetcher::fetchSelectedPart( { {"endpoint", getEndpointId(replica_path)}, {"part", part_name}, - {"client_protocol_version", toString(REPLICATION_PROTOCOL_VERSION_WITH_PARTS_PROJECTION)}, + {"client_protocol_version", toString(REPLICATION_PROTOCOL_VERSION_WITH_METADATA_VERSION)}, {"compress", "false"} }); @@ -692,7 +697,7 @@ MergeTreeData::MutableDataPartPtr Fetcher::downloadPartToMemory( auto block = block_in.read(); throttler->add(block.bytes()); - new_data_part->setColumns(block.getNamesAndTypesList(), {}); + new_data_part->setColumns(block.getNamesAndTypesList(), {}, metadata_snapshot->getMetadataVersion()); if (!is_projection) { @@ -768,7 +773,8 @@ void Fetcher::downloadBaseOrProjectionPartToDisk( if (file_name != "checksums.txt" && file_name != "columns.txt" && - file_name != IMergeTreeDataPart::DEFAULT_COMPRESSION_CODEC_FILE_NAME) + file_name != IMergeTreeDataPart::DEFAULT_COMPRESSION_CODEC_FILE_NAME && + file_name != IMergeTreeDataPart::METADATA_VERSION_FILE_NAME) checksums.addFile(file_name, file_size, expected_hash); } diff --git a/src/Storages/MergeTree/IMergeTreeDataPart.cpp b/src/Storages/MergeTree/IMergeTreeDataPart.cpp index 85420cabb8d..671c6ca67c1 100644 --- a/src/Storages/MergeTree/IMergeTreeDataPart.cpp +++ b/src/Storages/MergeTree/IMergeTreeDataPart.cpp @@ -416,10 +416,11 @@ std::pair IMergeTreeDataPart::getMinMaxTime() const } -void IMergeTreeDataPart::setColumns(const NamesAndTypesList & new_columns, const SerializationInfoByName & new_infos) +void IMergeTreeDataPart::setColumns(const NamesAndTypesList & new_columns, const SerializationInfoByName & new_infos, int32_t metadata_version_) { columns = new_columns; serialization_infos = new_infos; + metadata_version = metadata_version_; column_name_to_position.clear(); column_name_to_position.reserve(new_columns.size()); @@ -660,6 +661,7 @@ void IMergeTreeDataPart::appendFilesOfColumnsChecksumsIndexes(Strings & files, b appendFilesOfPartitionAndMinMaxIndex(files); appendFilesOfTTLInfos(files); appendFilesOfDefaultCompressionCodec(files); + appendFilesOfMetadataVersion(files); } if (!parent_part && include_projection) @@ -798,6 +800,9 @@ NameSet IMergeTreeDataPart::getFileNamesWithoutChecksums() const if (getDataPartStorage().exists(TXN_VERSION_METADATA_FILE_NAME)) result.emplace(TXN_VERSION_METADATA_FILE_NAME); + if (getDataPartStorage().exists(METADATA_VERSION_FILE_NAME)) + result.emplace(METADATA_VERSION_FILE_NAME); + return result; } @@ -971,11 +976,22 @@ void IMergeTreeDataPart::removeVersionMetadata() getDataPartStorage().removeFileIfExists("txn_version.txt"); } + +void IMergeTreeDataPart::removeMetadataVersion() +{ + getDataPartStorage().removeFileIfExists(METADATA_VERSION_FILE_NAME); +} + void IMergeTreeDataPart::appendFilesOfDefaultCompressionCodec(Strings & files) { files.push_back(DEFAULT_COMPRESSION_CODEC_FILE_NAME); } +void IMergeTreeDataPart::appendFilesOfMetadataVersion(Strings & files) +{ + files.push_back(METADATA_VERSION_FILE_NAME); +} + CompressionCodecPtr IMergeTreeDataPart::detectDefaultCompressionCodec() const { /// In memory parts doesn't have any compression @@ -1288,8 +1304,7 @@ void IMergeTreeDataPart::loadColumns(bool require) metadata_snapshot = metadata_snapshot->projections.get(name).metadata; NamesAndTypesList loaded_columns; - bool exists = metadata_manager->exists("columns.txt"); - if (!exists) + if (!metadata_manager->exists("columns.txt")) { /// We can get list of columns only from columns.txt in compact parts. if (require || part_type == Type::Compact) @@ -1322,16 +1337,32 @@ void IMergeTreeDataPart::loadColumns(bool require) }; SerializationInfoByName infos(loaded_columns, settings); - exists = metadata_manager->exists(SERIALIZATION_FILE_NAME); - if (exists) + if (metadata_manager->exists(SERIALIZATION_FILE_NAME)) { auto in = metadata_manager->read(SERIALIZATION_FILE_NAME); infos.readJSON(*in); } - setColumns(loaded_columns, infos); + int32_t loaded_metadata_version; + if (metadata_manager->exists(METADATA_VERSION_FILE_NAME)) + { + auto in = metadata_manager->read(METADATA_VERSION_FILE_NAME); + readIntText(loaded_metadata_version, *in); + } + else + { + loaded_metadata_version = metadata_snapshot->getMetadataVersion(); + + writeMetadata(METADATA_VERSION_FILE_NAME, {}, [loaded_metadata_version](auto & buffer) + { + writeIntText(loaded_metadata_version, buffer); + }); + } + + setColumns(loaded_columns, infos, loaded_metadata_version); } + /// Project part / part with project parts / compact part doesn't support LWD. bool IMergeTreeDataPart::supportLightweightDeleteMutate() const { diff --git a/src/Storages/MergeTree/IMergeTreeDataPart.h b/src/Storages/MergeTree/IMergeTreeDataPart.h index ea1fd209a20..4326aa630f9 100644 --- a/src/Storages/MergeTree/IMergeTreeDataPart.h +++ b/src/Storages/MergeTree/IMergeTreeDataPart.h @@ -137,7 +137,11 @@ public: String getTypeName() const { return getType().toString(); } - void setColumns(const NamesAndTypesList & new_columns, const SerializationInfoByName & new_infos); + /// We could have separate method like setMetadata, but it's much more convenient to set it up with columns + void setColumns(const NamesAndTypesList & new_columns, const SerializationInfoByName & new_infos, int32_t metadata_version_); + + /// Version of metadata for part (columns, pk and so on) + int32_t getMetadataVersion() const { return metadata_version; } const NamesAndTypesList & getColumns() const { return columns; } const ColumnsDescription & getColumnsDescription() const { return columns_description; } @@ -308,6 +312,9 @@ public: mutable VersionMetadata version; + /// Version of part metadata (columns, pk and so on). Managed properly only for replicated merge tree. + int32_t metadata_version; + /// For data in RAM ('index') UInt64 getIndexSizeInBytes() const; UInt64 getIndexSizeInAllocatedBytes() const; @@ -379,8 +386,12 @@ public: /// (number of rows, number of rows with default values, etc). static inline constexpr auto SERIALIZATION_FILE_NAME = "serialization.json"; + /// Version used for transactions. static inline constexpr auto TXN_VERSION_METADATA_FILE_NAME = "txn_version.txt"; + + static inline constexpr auto METADATA_VERSION_FILE_NAME = "metadata_version.txt"; + /// One of part files which is used to check how many references (I'd like /// to say hardlinks, but it will confuse even more) we have for the part /// for zero copy replication. Sadly it's very complex. @@ -443,7 +454,11 @@ public: void writeDeleteOnDestroyMarker(); void removeDeleteOnDestroyMarker(); + /// It may look like a stupid joke. but these two methods are absolutely unrelated. + /// This one is about removing file with metadata about part version (for transactions) void removeVersionMetadata(); + /// This one is about removing file with version of part's metadata (columns, pk and so on) + void removeMetadataVersion(); mutable std::atomic removal_state = DataPartRemovalState::NOT_ATTEMPTED; @@ -582,6 +597,8 @@ private: static void appendFilesOfDefaultCompressionCodec(Strings & files); + static void appendFilesOfMetadataVersion(Strings & files); + /// Found column without specific compression and return codec /// for this column with default parameters. CompressionCodecPtr detectDefaultCompressionCodec() const; diff --git a/src/Storages/MergeTree/MergeTask.cpp b/src/Storages/MergeTree/MergeTask.cpp index b961b70428e..ff72ef187f5 100644 --- a/src/Storages/MergeTree/MergeTask.cpp +++ b/src/Storages/MergeTree/MergeTask.cpp @@ -208,7 +208,7 @@ bool MergeTask::ExecuteAndFinalizeHorizontalPart::prepare() infos.add(part->getSerializationInfos()); } - global_ctx->new_data_part->setColumns(global_ctx->storage_columns, infos); + global_ctx->new_data_part->setColumns(global_ctx->storage_columns, infos, global_ctx->metadata_snapshot->getMetadataVersion()); const auto & local_part_min_ttl = global_ctx->new_data_part->ttl_infos.part_min_ttl; if (local_part_min_ttl && local_part_min_ttl <= global_ctx->time_of_merge) diff --git a/src/Storages/MergeTree/MergeTreeData.cpp b/src/Storages/MergeTree/MergeTreeData.cpp index e36ab125f97..e19eb57e387 100644 --- a/src/Storages/MergeTree/MergeTreeData.cpp +++ b/src/Storages/MergeTree/MergeTreeData.cpp @@ -4427,6 +4427,11 @@ MergeTreeData::DataPartPtr MergeTreeData::getPartIfExistsUnlocked(const MergeTre static void loadPartAndFixMetadataImpl(MergeTreeData::MutableDataPartPtr part) { + /// Remove metadata version file and take it from table. + /// Currently we cannot attach parts with different schema, so + /// we can assume that it's equal to table's current schema. + part->removeMetadataVersion(); + part->loadColumnsChecksumsIndexes(false, true); part->modification_time = part->getDataPartStorage().getLastModified().epochTime(); part->removeDeleteOnDestroyMarker(); @@ -7649,15 +7654,23 @@ bool MergeTreeData::canUsePolymorphicParts(const MergeTreeSettings & settings, S AlterConversions MergeTreeData::getAlterConversionsForPart(const MergeTreeDataPartPtr part) const { - MutationCommands commands = getFirstAlterMutationCommandsForPart(part); + std::map commands_map = getAlterMutationCommandsForPart(part); AlterConversions result{}; - for (const auto & command : commands) - /// Currently we need explicit conversions only for RENAME alter - /// all other conversions can be deduced from diff between part columns - /// and columns in storage. - if (command.type == MutationCommand::Type::RENAME_COLUMN) - result.rename_map[command.rename_to] = command.column_name; + auto & rename_map = result.rename_map; + for (const auto & [version, commands] : commands_map) + { + for (const auto & command : commands) + { + /// Currently we need explicit conversions only for RENAME alter + /// all other conversions can be deduced from diff between part columns + /// and columns in storage. + if (command.type == MutationCommand::Type::RENAME_COLUMN) + { + rename_map.emplace_back(AlterConversions::RenamePair{command.rename_to, command.column_name}); + } + } + } return result; } @@ -8044,7 +8057,7 @@ MergeTreeData::MutableDataPartPtr MergeTreeData::createEmptyPart( if (settings->assign_part_uuids) new_data_part->uuid = UUIDHelpers::generateV4(); - new_data_part->setColumns(columns, {}); + new_data_part->setColumns(columns, {}, metadata_snapshot->getMetadataVersion()); new_data_part->rows_count = block.rows(); new_data_part->partition = partition; diff --git a/src/Storages/MergeTree/MergeTreeData.h b/src/Storages/MergeTree/MergeTreeData.h index faee69b137f..cae791d6ffe 100644 --- a/src/Storages/MergeTree/MergeTreeData.h +++ b/src/Storages/MergeTree/MergeTreeData.h @@ -1313,7 +1313,7 @@ protected: /// Used to receive AlterConversions for part and apply them on fly. This /// method has different implementations for replicated and non replicated /// MergeTree because they store mutations in different way. - virtual MutationCommands getFirstAlterMutationCommandsForPart(const DataPartPtr & part) const = 0; + virtual std::map getAlterMutationCommandsForPart(const DataPartPtr & part) const = 0; /// Moves part to specified space, used in ALTER ... MOVE ... queries bool movePartsToSpace(const DataPartsVector & parts, SpacePtr space); diff --git a/src/Storages/MergeTree/MergeTreeDataPartInMemory.cpp b/src/Storages/MergeTree/MergeTreeDataPartInMemory.cpp index 20049976acf..5b1054d0a0e 100644 --- a/src/Storages/MergeTree/MergeTreeDataPartInMemory.cpp +++ b/src/Storages/MergeTree/MergeTreeDataPartInMemory.cpp @@ -73,7 +73,7 @@ MutableDataPartStoragePtr MergeTreeDataPartInMemory::flushToDisk(const String & new_data_part_storage->beginTransaction(); new_data_part->uuid = uuid; - new_data_part->setColumns(columns, {}); + new_data_part->setColumns(columns, {}, metadata_snapshot->getMetadataVersion()); new_data_part->partition.value = partition.value; new_data_part->minmax_idx = minmax_idx; @@ -104,7 +104,7 @@ MutableDataPartStoragePtr MergeTreeDataPartInMemory::flushToDisk(const String & .build(); new_projection_part->is_temp = false; // clean up will be done on parent part - new_projection_part->setColumns(projection->getColumns(), {}); + new_projection_part->setColumns(projection->getColumns(), {}, metadata_snapshot->getMetadataVersion()); auto new_projection_part_storage = new_projection_part->getDataPartStoragePtr(); if (new_projection_part_storage->exists()) diff --git a/src/Storages/MergeTree/MergeTreeDataWriter.cpp b/src/Storages/MergeTree/MergeTreeDataWriter.cpp index 37cfe4d065e..adb7505a8ba 100644 --- a/src/Storages/MergeTree/MergeTreeDataWriter.cpp +++ b/src/Storages/MergeTree/MergeTreeDataWriter.cpp @@ -464,7 +464,7 @@ MergeTreeDataWriter::TemporaryPart MergeTreeDataWriter::writeTempPartImpl( SerializationInfoByName infos(columns, settings); infos.add(block); - new_data_part->setColumns(columns, infos); + new_data_part->setColumns(columns, infos, metadata_snapshot->getMetadataVersion()); new_data_part->rows_count = block.rows(); new_data_part->partition = std::move(partition); new_data_part->minmax_idx = std::move(minmax_idx); @@ -586,7 +586,7 @@ MergeTreeDataWriter::TemporaryPart MergeTreeDataWriter::writeProjectionPartImpl( SerializationInfoByName infos(columns, settings); infos.add(block); - new_data_part->setColumns(columns, infos); + new_data_part->setColumns(columns, infos, metadata_snapshot->getMetadataVersion()); if (new_data_part->isStoredOnDisk()) { diff --git a/src/Storages/MergeTree/MergeTreeMarksLoader.cpp b/src/Storages/MergeTree/MergeTreeMarksLoader.cpp index 3fc7ff54c35..1d85ac1bd34 100644 --- a/src/Storages/MergeTree/MergeTreeMarksLoader.cpp +++ b/src/Storages/MergeTree/MergeTreeMarksLoader.cpp @@ -102,6 +102,15 @@ MarkCache::MappedPtr MergeTreeMarksLoader::loadMarksImpl() auto res = std::make_shared(marks_count * columns_in_mark); + if (file_size == 0 && marks_count != 0) + { + throw Exception( + ErrorCodes::CORRUPTED_DATA, + "Empty marks file '{}': {}, must be: {}", + std::string(fs::path(data_part_storage->getFullPath()) / mrk_path), + file_size, expected_uncompressed_size); + } + if (!index_granularity_info.mark_type.compressed && expected_uncompressed_size != file_size) throw Exception( ErrorCodes::CORRUPTED_DATA, @@ -138,7 +147,12 @@ MarkCache::MappedPtr MergeTreeMarksLoader::loadMarksImpl() } if (i * mark_size != expected_uncompressed_size) - throw Exception(ErrorCodes::CANNOT_READ_ALL_DATA, "Cannot read all marks from file {}", mrk_path); + { + throw Exception( + ErrorCodes::CANNOT_READ_ALL_DATA, + "Cannot read all marks from file {}, marks expected {} (bytes size {}), marks read {} (bytes size {})", + mrk_path, marks_count, expected_uncompressed_size, i, reader->count()); + } } res->protect(); diff --git a/src/Storages/MergeTree/MergeTreeWriteAheadLog.cpp b/src/Storages/MergeTree/MergeTreeWriteAheadLog.cpp index fabf2acdad3..87d9bb0f168 100644 --- a/src/Storages/MergeTree/MergeTreeWriteAheadLog.cpp +++ b/src/Storages/MergeTree/MergeTreeWriteAheadLog.cpp @@ -229,7 +229,7 @@ MergeTreeData::MutableDataPartsVector MergeTreeWriteAheadLog::restore( part->minmax_idx->update(block, storage.getMinMaxColumnsNames(metadata_snapshot->getPartitionKey())); part->partition.create(metadata_snapshot, block, 0, context); - part->setColumns(block.getNamesAndTypesList(), {}); + part->setColumns(block.getNamesAndTypesList(), {}, metadata_snapshot->getMetadataVersion()); if (metadata_snapshot->hasSortingKey()) metadata_snapshot->getSortingKey().expression->execute(block); diff --git a/src/Storages/MergeTree/MergedBlockOutputStream.cpp b/src/Storages/MergeTree/MergedBlockOutputStream.cpp index ced43ae25b0..53d4a32fc0e 100644 --- a/src/Storages/MergeTree/MergedBlockOutputStream.cpp +++ b/src/Storages/MergeTree/MergedBlockOutputStream.cpp @@ -175,7 +175,7 @@ MergedBlockOutputStream::Finalizer MergedBlockOutputStream::finalizePartAsync( serialization_infos.replaceData(new_serialization_infos); files_to_remove_after_sync = removeEmptyColumnsFromPart(new_part, part_columns, serialization_infos, checksums); - new_part->setColumns(part_columns, serialization_infos); + new_part->setColumns(part_columns, serialization_infos, metadata_snapshot->getMetadataVersion()); } auto finalizer = std::make_unique(*writer, new_part, files_to_remove_after_sync, sync); @@ -289,6 +289,14 @@ MergedBlockOutputStream::WrittenFiles MergedBlockOutputStream::finalizePartOnDis written_files.emplace_back(std::move(out)); } + { + /// Write a file with a description of columns. + auto out = new_part->getDataPartStorage().writeFile(IMergeTreeDataPart::METADATA_VERSION_FILE_NAME, 4096, write_settings); + DB::writeIntText(new_part->getMetadataVersion(), *out); + out->preFinalize(); + written_files.emplace_back(std::move(out)); + } + if (default_codec != nullptr) { auto out = new_part->getDataPartStorage().writeFile(IMergeTreeDataPart::DEFAULT_COMPRESSION_CODEC_FILE_NAME, 4096, write_settings); diff --git a/src/Storages/MergeTree/MergedColumnOnlyOutputStream.cpp b/src/Storages/MergeTree/MergedColumnOnlyOutputStream.cpp index 03829f1daf9..c70c5187b8b 100644 --- a/src/Storages/MergeTree/MergedColumnOnlyOutputStream.cpp +++ b/src/Storages/MergeTree/MergedColumnOnlyOutputStream.cpp @@ -85,7 +85,7 @@ MergedColumnOnlyOutputStream::fillChecksums( all_checksums.files.erase(removed_file); } - new_part->setColumns(columns, serialization_infos); + new_part->setColumns(columns, serialization_infos, metadata_snapshot->getMetadataVersion()); return checksums; } diff --git a/src/Storages/MergeTree/MutateTask.cpp b/src/Storages/MergeTree/MutateTask.cpp index bcb1d5d2c28..ae34b1b707c 100644 --- a/src/Storages/MergeTree/MutateTask.cpp +++ b/src/Storages/MergeTree/MutateTask.cpp @@ -53,7 +53,7 @@ static bool checkOperationIsNotCanceled(ActionBlocker & merges_blocker, MergeLis * First part should be executed by mutations interpreter. * Other is just simple drop/renames, so they can be executed without interpreter. */ -static void splitMutationCommands( +static void splitAndModifyMutationCommands( MergeTreeData::DataPartPtr part, const MutationCommands & commands, MutationCommands & for_interpreter, @@ -98,25 +98,48 @@ static void splitMutationCommands( else mutated_columns.emplace(command.column_name); } - - if (command.type == MutationCommand::Type::RENAME_COLUMN) - { - for_interpreter.push_back( - { - .type = MutationCommand::Type::READ_COLUMN, - .column_name = command.rename_to, - }); - part_columns.rename(command.column_name, command.rename_to); - } } } + + auto alter_conversions = part->storage.getAlterConversionsForPart(part); + + /// We don't add renames from commands, instead we take them from rename_map. + /// It's important because required renames depend not only on part's data version (i.e. mutation version) + /// but also on part's metadata version. Why we have such logic only for renames? Because all other types of alter + /// can be deduced based on difference between part's schema and table schema. + for (const auto & [rename_to, rename_from] : alter_conversions.rename_map) + { + if (part_columns.has(rename_from)) + { + /// Actual rename + for_interpreter.push_back( + { + .type = MutationCommand::Type::READ_COLUMN, + .column_name = rename_to, + }); + + /// Not needed for compact parts (not executed), added here only to produce correct + /// set of columns for new part and their serializations + for_file_renames.push_back( + { + .type = MutationCommand::Type::RENAME_COLUMN, + .column_name = rename_from, + .rename_to = rename_to + }); + + part_columns.rename(rename_from, rename_to); + } + } + /// If it's compact part, then we don't need to actually remove files /// from disk we just don't read dropped columns - for (const auto & column : part->getColumns()) + for (const auto & column : part_columns) { if (!mutated_columns.contains(column.name)) + { for_interpreter.emplace_back( MutationCommand{.type = MutationCommand::Type::READ_COLUMN, .column_name = column.name, .data_type = column.type}); + } } } else @@ -142,15 +165,59 @@ static void splitMutationCommands( { if (command.type == MutationCommand::Type::READ_COLUMN) for_interpreter.push_back(command); - else if (command.type == MutationCommand::Type::RENAME_COLUMN) - part_columns.rename(command.column_name, command.rename_to); for_file_renames.push_back(command); } } + + auto alter_conversions = part->storage.getAlterConversionsForPart(part); + /// We don't add renames from commands, instead we take them from rename_map. + /// It's important because required renames depend not only on part's data version (i.e. mutation version) + /// but also on part's metadata version. Why we have such logic only for renames? Because all other types of alter + /// can be deduced based on difference between part's schema and table schema. + + for (const auto & [rename_to, rename_from] : alter_conversions.rename_map) + { + for_file_renames.push_back({.type = MutationCommand::Type::RENAME_COLUMN, .column_name = rename_from, .rename_to = rename_to}); + } } } +/// It's legal to squash renames because commands with rename are always "barrier" +/// and executed separately from other types of commands. +static MutationCommands squashRenamesInCommands(const MutationCommands & commands) +{ + NameToNameMap squashed_renames; + for (const auto & command : commands) + { + std::string result_name = command.rename_to; + + bool squashed = false; + for (const auto & [name_from, name_to] : squashed_renames) + { + if (name_to == command.column_name) + { + squashed = true; + squashed_renames[name_from] = result_name; + break; + } + } + if (!squashed) + squashed_renames[command.column_name] = result_name; + } + + MutationCommands squashed_commands; + for (const auto & command : commands) + { + if (squashed_renames.contains(command.column_name)) + { + squashed_commands.push_back(command); + squashed_commands.back().rename_to = squashed_renames[command.column_name]; + } + } + return squashed_commands; +} + /// Get the columns list of the resulting part in the same order as storage_columns. static std::pair getColumnsForNewDataPart( @@ -158,8 +225,13 @@ getColumnsForNewDataPart( const Block & updated_header, NamesAndTypesList storage_columns, const SerializationInfoByName & serialization_infos, + const MutationCommands & commands_for_interpreter, const MutationCommands & commands_for_removes) { + MutationCommands all_commands; + all_commands.insert(all_commands.end(), commands_for_interpreter.begin(), commands_for_interpreter.end()); + all_commands.insert(all_commands.end(), commands_for_removes.begin(), commands_for_removes.end()); + NameSet removed_columns; NameToNameMap renamed_columns_to_from; NameToNameMap renamed_columns_from_to; @@ -175,8 +247,9 @@ getColumnsForNewDataPart( storage_columns.emplace_back(column); } - /// All commands are validated in AlterCommand so we don't care about order - for (const auto & command : commands_for_removes) + MutationCommands squashed_commands = squashRenamesInCommands(all_commands); + + for (const auto & command : squashed_commands) { if (command.type == MutationCommand::UPDATE) { @@ -269,20 +342,38 @@ getColumnsForNewDataPart( /// should it's previous version should be dropped or removed if (renamed_columns_to_from.contains(it->name) && !was_renamed && !was_removed) throw Exception( - ErrorCodes::LOGICAL_ERROR, - "Incorrect mutation commands, trying to rename column {} to {}, " - "but part {} already has column {}", - renamed_columns_to_from[it->name], it->name, source_part->name, it->name); + ErrorCodes::LOGICAL_ERROR, + "Incorrect mutation commands, trying to rename column {} to {}, " + "but part {} already has column {}", + renamed_columns_to_from[it->name], it->name, source_part->name, it->name); /// Column was renamed and no other column renamed to it's name /// or column is dropped. if (!renamed_columns_to_from.contains(it->name) && (was_renamed || was_removed)) + { it = storage_columns.erase(it); + } else { - /// Take a type from source part column. - /// It may differ from column type in storage. - it->type = source_col->second; + + if (was_removed) + { /// DROP COLUMN xxx, RENAME COLUMN yyy TO xxx + auto renamed_from = renamed_columns_to_from.at(it->name); + auto maybe_name_and_type = source_columns.tryGetByName(renamed_from); + if (!maybe_name_and_type) + throw Exception( + ErrorCodes::LOGICAL_ERROR, + "Got incorrect mutation commands, column {} was renamed from {}, but it doesn't exist in source columns {}", + it->name, renamed_from, source_columns.toString()); + + it->type = maybe_name_and_type->type; + } + else + { + /// Take a type from source part column. + /// It may differ from column type in storage. + it->type = source_col->second; + } ++it; } } @@ -548,27 +639,36 @@ static NameToNameVector collectFilesForRenames( /// Collect counts for shared streams of different columns. As an example, Nested columns have shared stream with array sizes. auto stream_counts = getStreamCounts(source_part, source_part->getColumns().getNames()); NameToNameVector rename_vector; + NameSet collected_names; + + auto add_rename = [&rename_vector, &collected_names] (const std::string & file_rename_from, const std::string & file_rename_to) + { + if (collected_names.emplace(file_rename_from).second) + rename_vector.emplace_back(file_rename_from, file_rename_to); + }; + + MutationCommands squashed_commands = squashRenamesInCommands(commands_for_removes); /// Remove old data - for (const auto & command : commands_for_removes) + for (const auto & command : squashed_commands) { if (command.type == MutationCommand::Type::DROP_INDEX) { if (source_part->checksums.has(INDEX_FILE_PREFIX + command.column_name + ".idx2")) { - rename_vector.emplace_back(INDEX_FILE_PREFIX + command.column_name + ".idx2", ""); - rename_vector.emplace_back(INDEX_FILE_PREFIX + command.column_name + mrk_extension, ""); + add_rename(INDEX_FILE_PREFIX + command.column_name + ".idx2", ""); + add_rename(INDEX_FILE_PREFIX + command.column_name + mrk_extension, ""); } else if (source_part->checksums.has(INDEX_FILE_PREFIX + command.column_name + ".idx")) { - rename_vector.emplace_back(INDEX_FILE_PREFIX + command.column_name + ".idx", ""); - rename_vector.emplace_back(INDEX_FILE_PREFIX + command.column_name + mrk_extension, ""); + add_rename(INDEX_FILE_PREFIX + command.column_name + ".idx", ""); + add_rename(INDEX_FILE_PREFIX + command.column_name + mrk_extension, ""); } } else if (command.type == MutationCommand::Type::DROP_PROJECTION) { if (source_part->checksums.has(command.column_name + ".proj")) - rename_vector.emplace_back(command.column_name + ".proj", ""); + add_rename(command.column_name + ".proj", ""); } else if (command.type == MutationCommand::Type::DROP_COLUMN) { @@ -578,8 +678,8 @@ static NameToNameVector collectFilesForRenames( /// Delete files if they are no longer shared with another column. if (--stream_counts[stream_name] == 0) { - rename_vector.emplace_back(stream_name + ".bin", ""); - rename_vector.emplace_back(stream_name + mrk_extension, ""); + add_rename(stream_name + ".bin", ""); + add_rename(stream_name + mrk_extension, ""); } }; @@ -598,8 +698,8 @@ static NameToNameVector collectFilesForRenames( if (stream_from != stream_to) { - rename_vector.emplace_back(stream_from + ".bin", stream_to + ".bin"); - rename_vector.emplace_back(stream_from + mrk_extension, stream_to + mrk_extension); + add_rename(stream_from + ".bin", stream_to + ".bin"); + add_rename(stream_from + mrk_extension, stream_to + mrk_extension); } }; @@ -619,8 +719,8 @@ static NameToNameVector collectFilesForRenames( { if (!new_streams.contains(old_stream) && --stream_counts[old_stream] == 0) { - rename_vector.emplace_back(old_stream + ".bin", ""); - rename_vector.emplace_back(old_stream + mrk_extension, ""); + add_rename(old_stream + ".bin", ""); + add_rename(old_stream + mrk_extension, ""); } } } @@ -637,6 +737,7 @@ void finalizeMutatedPart( ExecuteTTLType execute_ttl_type, const CompressionCodecPtr & codec, ContextPtr context, + StorageMetadataPtr metadata_snapshot, bool sync) { std::vector> written_files; @@ -685,6 +786,12 @@ void finalizeMutatedPart( written_files.push_back(std::move(out_comp)); } + { + auto out_metadata = new_data_part->getDataPartStorage().writeFile(IMergeTreeDataPart::METADATA_VERSION_FILE_NAME, 4096, context->getWriteSettings()); + DB::writeText(metadata_snapshot->getMetadataVersion(), *out_metadata); + written_files.push_back(std::move(out_metadata)); + } + { /// Write a file with a description of columns. auto out_columns = new_data_part->getDataPartStorage().writeFile("columns.txt", 4096, context->getWriteSettings()); @@ -1324,13 +1431,27 @@ private: ctx->new_data_part->storeVersionMetadata(); NameSet hardlinked_files; + + /// NOTE: Renames must be done in order + for (const auto & [rename_from, rename_to] : ctx->files_to_rename) + { + if (rename_to.empty()) /// It's DROP COLUMN + { + /// pass + } + else + { + ctx->new_data_part->getDataPartStorage().createHardLinkFrom( + ctx->source_part->getDataPartStorage(), rename_from, rename_to); + hardlinked_files.insert(rename_from); + } + } /// Create hardlinks for unchanged files for (auto it = ctx->source_part->getDataPartStorage().iterate(); it->isValid(); it->next()) { if (ctx->files_to_skip.contains(it->name())) continue; - String destination; String file_name = it->name(); auto rename_it = std::find_if(ctx->files_to_rename.begin(), ctx->files_to_rename.end(), [&file_name](const auto & rename_pair) @@ -1340,20 +1461,17 @@ private: if (rename_it != ctx->files_to_rename.end()) { - if (rename_it->second.empty()) - continue; - destination = rename_it->second; - } - else - { - destination = it->name(); + /// RENAMEs and DROPs already processed + continue; } + String destination = it->name(); + if (it->isFile()) { ctx->new_data_part->getDataPartStorage().createHardLinkFrom( - ctx->source_part->getDataPartStorage(), it->name(), destination); - hardlinked_files.insert(it->name()); + ctx->source_part->getDataPartStorage(), file_name, destination); + hardlinked_files.insert(file_name); } else if (!endsWith(it->name(), ".tmp_proj")) // ignore projection tmp merge dir { @@ -1449,7 +1567,7 @@ private: } } - MutationHelpers::finalizeMutatedPart(ctx->source_part, ctx->new_data_part, ctx->execute_ttl_type, ctx->compression_codec, ctx->context, ctx->need_sync); + MutationHelpers::finalizeMutatedPart(ctx->source_part, ctx->new_data_part, ctx->execute_ttl_type, ctx->compression_codec, ctx->context, ctx->metadata_snapshot, ctx->need_sync); } @@ -1611,7 +1729,7 @@ bool MutateTask::prepare() context_for_reading->setSetting("allow_asynchronous_read_from_io_pool_for_merge_tree", false); context_for_reading->setSetting("max_streams_for_merge_tree_reading", Field(0)); - MutationHelpers::splitMutationCommands(ctx->source_part, ctx->commands_for_part, ctx->for_interpreter, ctx->for_file_renames); + MutationHelpers::splitAndModifyMutationCommands(ctx->source_part, ctx->commands_for_part, ctx->for_interpreter, ctx->for_file_renames); ctx->stage_progress = std::make_unique(1.0); @@ -1656,9 +1774,9 @@ bool MutateTask::prepare() auto [new_columns, new_infos] = MutationHelpers::getColumnsForNewDataPart( ctx->source_part, ctx->updated_header, ctx->storage_columns, - ctx->source_part->getSerializationInfos(), ctx->commands_for_part); + ctx->source_part->getSerializationInfos(), ctx->for_interpreter, ctx->for_file_renames); - ctx->new_data_part->setColumns(new_columns, new_infos); + ctx->new_data_part->setColumns(new_columns, new_infos, ctx->metadata_snapshot->getMetadataVersion()); ctx->new_data_part->partition.assign(ctx->source_part->partition); /// Don't change granularity type while mutating subset of columns diff --git a/src/Storages/MergeTree/ReplicatedMergeTreeAttachThread.cpp b/src/Storages/MergeTree/ReplicatedMergeTreeAttachThread.cpp index 557123ddae2..f1bdc9f43af 100644 --- a/src/Storages/MergeTree/ReplicatedMergeTreeAttachThread.cpp +++ b/src/Storages/MergeTree/ReplicatedMergeTreeAttachThread.cpp @@ -149,7 +149,7 @@ void ReplicatedMergeTreeAttachThread::runImpl() const bool replica_metadata_version_exists = zookeeper->tryGet(replica_path + "/metadata_version", replica_metadata_version); if (replica_metadata_version_exists) { - storage.metadata_version = parse(replica_metadata_version); + storage.setInMemoryMetadata(metadata_snapshot->withMetadataVersion(parse(replica_metadata_version))); } else { diff --git a/src/Storages/MergeTree/ReplicatedMergeTreeQueue.cpp b/src/Storages/MergeTree/ReplicatedMergeTreeQueue.cpp index 125529e7b94..bac5beb6364 100644 --- a/src/Storages/MergeTree/ReplicatedMergeTreeQueue.cpp +++ b/src/Storages/MergeTree/ReplicatedMergeTreeQueue.cpp @@ -11,6 +11,7 @@ #include #include +#include namespace DB { @@ -1754,19 +1755,40 @@ ReplicatedMergeTreeMergePredicate ReplicatedMergeTreeQueue::getMergePredicate(zk } -MutationCommands ReplicatedMergeTreeQueue::getFirstAlterMutationCommandsForPart(const MergeTreeData::DataPartPtr & part) const +std::map ReplicatedMergeTreeQueue::getAlterMutationCommandsForPart(const MergeTreeData::DataPartPtr & part) const { - std::lock_guard lock(state_mutex); + std::unique_lock lock(state_mutex); auto in_partition = mutations_by_partition.find(part->info.partition_id); if (in_partition == mutations_by_partition.end()) - return MutationCommands{}; + return {}; - Int64 part_version = part->info.getDataVersion(); - for (auto [mutation_version, mutation_status] : in_partition->second) - if (mutation_version > part_version && mutation_status->entry->alter_version != -1) - return mutation_status->entry->commands; + Int64 part_metadata_version = part->getMetadataVersion(); + std::map result; + /// Here we return mutation commands for part which has bigger alter version than part metadata version. + /// Please note, we don't use getDataVersion(). It's because these alter commands are used for in-fly conversions + /// of part's metadata. + for (const auto & [mutation_version, mutation_status] : in_partition->second | std::views::reverse) + { + int32_t alter_version = mutation_status->entry->alter_version; + if (alter_version != -1) + { + if (!alter_sequence.canExecuteDataAlter(alter_version, lock)) + continue; - return MutationCommands{}; + /// we take commands with bigger metadata version + if (alter_version > part_metadata_version) + { + result[mutation_version] = mutation_status->entry->commands; + } + else + { + /// entries are ordered, we processing them in reverse order so we can break + break; + } + } + } + + return result; } MutationCommands ReplicatedMergeTreeQueue::getMutationCommands( @@ -1808,7 +1830,18 @@ MutationCommands ReplicatedMergeTreeQueue::getMutationCommands( MutationCommands commands; for (auto it = begin; it != end; ++it) - commands.insert(commands.end(), it->second->entry->commands.begin(), it->second->entry->commands.end()); + { + const auto & commands_from_entry = it->second->entry->commands; + + if (commands_from_entry.containBarrierCommand()) + { + if (commands.empty()) + commands.insert(commands.end(), commands_from_entry.begin(), commands_from_entry.end()); + break; + } + else + commands.insert(commands.end(), commands_from_entry.begin(), commands_from_entry.end()); + } return commands; } diff --git a/src/Storages/MergeTree/ReplicatedMergeTreeQueue.h b/src/Storages/MergeTree/ReplicatedMergeTreeQueue.h index 36552129690..df1fd5a3c85 100644 --- a/src/Storages/MergeTree/ReplicatedMergeTreeQueue.h +++ b/src/Storages/MergeTree/ReplicatedMergeTreeQueue.h @@ -393,10 +393,10 @@ public: MutationCommands getMutationCommands(const MergeTreeData::DataPartPtr & part, Int64 desired_mutation_version) const; - /// Return mutation commands for part with smallest mutation version bigger - /// than data part version. Used when we apply alter commands on fly, + /// Return mutation commands for part which could be not applied to + /// it according to part mutation version. Used when we apply alter commands on fly, /// without actual data modification on disk. - MutationCommands getFirstAlterMutationCommandsForPart(const MergeTreeData::DataPartPtr & part) const; + std::map getAlterMutationCommandsForPart(const MergeTreeData::DataPartPtr & part) const; /// Mark finished mutations as done. If the function needs to be called again at some later time /// (because some mutations are probably done but we are not sure yet), returns true. diff --git a/src/Storages/MutationCommands.cpp b/src/Storages/MutationCommands.cpp index 0c9e9223929..aa77988348d 100644 --- a/src/Storages/MutationCommands.cpp +++ b/src/Storages/MutationCommands.cpp @@ -23,6 +23,12 @@ namespace ErrorCodes extern const int MULTIPLE_ASSIGNMENTS_TO_COLUMN; } + +bool MutationCommand::isBarrierCommand() const +{ + return type == RENAME_COLUMN; +} + std::optional MutationCommand::parse(ASTAlterCommand * command, bool parse_alter_commands) { if (command->type == ASTAlterCommand::DELETE) @@ -212,4 +218,14 @@ bool MutationCommands::hasNonEmptyMutationCommands() const return false; } +bool MutationCommands::containBarrierCommand() const +{ + for (const auto & command : *this) + { + if (command.isBarrierCommand()) + return true; + } + return false; +} + } diff --git a/src/Storages/MutationCommands.h b/src/Storages/MutationCommands.h index aca91c16e85..5ef0cfda1be 100644 --- a/src/Storages/MutationCommands.h +++ b/src/Storages/MutationCommands.h @@ -67,6 +67,9 @@ struct MutationCommand /// If parse_alter_commands, than consider more Alter commands as mutation commands static std::optional parse(ASTAlterCommand * command, bool parse_alter_commands = false); + + /// This command shouldn't stick with other commands + bool isBarrierCommand() const; }; /// Multiple mutation commands, possible from different ALTER queries @@ -79,6 +82,11 @@ public: void readText(ReadBuffer & in); std::string toString() const; bool hasNonEmptyMutationCommands() const; + + /// These set of commands contain barrier command and shouldn't + /// stick with other commands. Commands from one set have already been validated + /// to be executed without issues on the creation state. + bool containBarrierCommand() const; }; using MutationCommandsConstPtr = std::shared_ptr; diff --git a/src/Storages/StorageInMemoryMetadata.cpp b/src/Storages/StorageInMemoryMetadata.cpp index f6550c6cd5d..45abd4bebef 100644 --- a/src/Storages/StorageInMemoryMetadata.cpp +++ b/src/Storages/StorageInMemoryMetadata.cpp @@ -41,6 +41,7 @@ StorageInMemoryMetadata::StorageInMemoryMetadata(const StorageInMemoryMetadata & , settings_changes(other.settings_changes ? other.settings_changes->clone() : nullptr) , select(other.select) , comment(other.comment) + , metadata_version(other.metadata_version) { } @@ -69,6 +70,7 @@ StorageInMemoryMetadata & StorageInMemoryMetadata::operator=(const StorageInMemo settings_changes.reset(); select = other.select; comment = other.comment; + metadata_version = other.metadata_version; return *this; } @@ -122,6 +124,18 @@ void StorageInMemoryMetadata::setSelectQuery(const SelectQueryDescription & sele select = select_; } +void StorageInMemoryMetadata::setMetadataVersion(int32_t metadata_version_) +{ + metadata_version = metadata_version_; +} + +StorageInMemoryMetadata StorageInMemoryMetadata::withMetadataVersion(int32_t metadata_version_) const +{ + StorageInMemoryMetadata copy(*this); + copy.setMetadataVersion(metadata_version_); + return copy; +} + const ColumnsDescription & StorageInMemoryMetadata::getColumns() const { return columns; diff --git a/src/Storages/StorageInMemoryMetadata.h b/src/Storages/StorageInMemoryMetadata.h index eadce581334..25618c5b03f 100644 --- a/src/Storages/StorageInMemoryMetadata.h +++ b/src/Storages/StorageInMemoryMetadata.h @@ -50,6 +50,10 @@ struct StorageInMemoryMetadata String comment; + /// Version of metadata. Managed properly by ReplicatedMergeTree only + /// (zero-initialization is important) + int32_t metadata_version = 0; + StorageInMemoryMetadata() = default; StorageInMemoryMetadata(const StorageInMemoryMetadata & other); @@ -58,7 +62,7 @@ struct StorageInMemoryMetadata StorageInMemoryMetadata(StorageInMemoryMetadata && other) = default; StorageInMemoryMetadata & operator=(StorageInMemoryMetadata && other) = default; - /// NOTE: Thread unsafe part. You should modify same StorageInMemoryMetadata + /// NOTE: Thread unsafe part. You should not modify same StorageInMemoryMetadata /// structure from different threads. It should be used as MultiVersion /// object. See example in IStorage. @@ -90,6 +94,11 @@ struct StorageInMemoryMetadata /// Set SELECT query for (Materialized)View void setSelectQuery(const SelectQueryDescription & select_); + /// Set version of metadata. + void setMetadataVersion(int32_t metadata_version_); + /// Get copy of current metadata with metadata_version_ + StorageInMemoryMetadata withMetadataVersion(int32_t metadata_version_) const; + /// Returns combined set of columns const ColumnsDescription & getColumns() const; @@ -218,6 +227,9 @@ struct StorageInMemoryMetadata const SelectQueryDescription & getSelectQuery() const; bool hasSelectQuery() const; + /// Get version of metadata + int32_t getMetadataVersion() const { return metadata_version; } + /// Check that all the requested names are in the table and have the correct types. void check(const NamesAndTypesList & columns) const; diff --git a/src/Storages/StorageMergeTree.cpp b/src/Storages/StorageMergeTree.cpp index 8210ef93172..11922bebe7b 100644 --- a/src/Storages/StorageMergeTree.cpp +++ b/src/Storages/StorageMergeTree.cpp @@ -1143,9 +1143,24 @@ MergeMutateSelectedEntryPtr StorageMergeTree::selectPartsToMutate( if (current_ast_elements + commands_size >= max_ast_elements) break; - current_ast_elements += commands_size; - commands->insert(commands->end(), it->second.commands.begin(), it->second.commands.end()); - last_mutation_to_apply = it; + const auto & single_mutation_commands = it->second.commands; + + if (single_mutation_commands.containBarrierCommand()) + { + if (commands->empty()) + { + commands->insert(commands->end(), single_mutation_commands.begin(), single_mutation_commands.end()); + last_mutation_to_apply = it; + } + break; + } + else + { + current_ast_elements += commands_size; + commands->insert(commands->end(), single_mutation_commands.begin(), single_mutation_commands.end()); + last_mutation_to_apply = it; + } + } assert(commands->empty() == (last_mutation_to_apply == mutations_end_it)); @@ -1240,7 +1255,10 @@ bool StorageMergeTree::scheduleDataProcessingJob(BackgroundJobsAssignee & assign } if (mutate_entry) { - auto task = std::make_shared(*this, metadata_snapshot, mutate_entry, shared_lock, common_assignee_trigger); + /// We take new metadata snapshot here. It's because mutation commands can be executed only with metadata snapshot + /// which is equal or more fresh than commands themselves. In extremely rare case it can happen that we will have alter + /// in between we took snapshot above and selected commands. That is why we take new snapshot here. + auto task = std::make_shared(*this, getInMemoryMetadataPtr(), mutate_entry, shared_lock, common_assignee_trigger); assignee.scheduleMergeMutateTask(task); return true; } @@ -2109,14 +2127,22 @@ void StorageMergeTree::attachRestoredParts(MutableDataPartsVector && parts) } -MutationCommands StorageMergeTree::getFirstAlterMutationCommandsForPart(const DataPartPtr & part) const +std::map StorageMergeTree::getAlterMutationCommandsForPart(const DataPartPtr & part) const { std::lock_guard lock(currently_processing_in_background_mutex); - auto it = current_mutations_by_version.upper_bound(part->info.getDataVersion()); - if (it == current_mutations_by_version.end()) - return {}; - return it->second.commands; + Int64 part_data_version = part->info.getDataVersion(); + + std::map result; + if (!current_mutations_by_version.empty()) + { + const auto & [latest_mutation_id, latest_commands] = *current_mutations_by_version.rbegin(); + if (part_data_version < static_cast(latest_mutation_id)) + { + result[latest_mutation_id] = latest_commands.commands; + } + } + return result; } void StorageMergeTree::startBackgroundMovesIfNeeded() diff --git a/src/Storages/StorageMergeTree.h b/src/Storages/StorageMergeTree.h index 052a672a187..a0629bb8d3e 100644 --- a/src/Storages/StorageMergeTree.h +++ b/src/Storages/StorageMergeTree.h @@ -267,7 +267,7 @@ private: protected: - MutationCommands getFirstAlterMutationCommandsForPart(const DataPartPtr & part) const override; + std::map getAlterMutationCommandsForPart(const DataPartPtr & part) const override; }; } diff --git a/src/Storages/StorageReplicatedMergeTree.cpp b/src/Storages/StorageReplicatedMergeTree.cpp index 429339386b4..8ae9b1baa96 100644 --- a/src/Storages/StorageReplicatedMergeTree.cpp +++ b/src/Storages/StorageReplicatedMergeTree.cpp @@ -461,7 +461,7 @@ StorageReplicatedMergeTree::StorageReplicatedMergeTree( Coordination::Stat metadata_stat; current_zookeeper->get(zookeeper_path + "/metadata", &metadata_stat); - metadata_version = metadata_stat.version; + setInMemoryMetadata(metadata_snapshot->withMetadataVersion(metadata_stat.version)); } catch (Coordination::Exception & e) { @@ -781,7 +781,7 @@ bool StorageReplicatedMergeTree::createTableIfNotExists(const StorageMetadataPtr zkutil::CreateMode::Persistent)); ops.emplace_back(zkutil::makeCreateRequest(replica_path + "/columns", metadata_snapshot->getColumns().toString(), zkutil::CreateMode::Persistent)); - ops.emplace_back(zkutil::makeCreateRequest(replica_path + "/metadata_version", std::to_string(metadata_version), + ops.emplace_back(zkutil::makeCreateRequest(replica_path + "/metadata_version", toString(metadata_snapshot->getMetadataVersion()), zkutil::CreateMode::Persistent)); /// The following 3 nodes were added in version 1.1.xxx, so we create them here, not in createNewZooKeeperNodes() @@ -854,7 +854,7 @@ void StorageReplicatedMergeTree::createReplica(const StorageMetadataPtr & metada zkutil::CreateMode::Persistent)); ops.emplace_back(zkutil::makeCreateRequest(replica_path + "/columns", metadata_snapshot->getColumns().toString(), zkutil::CreateMode::Persistent)); - ops.emplace_back(zkutil::makeCreateRequest(replica_path + "/metadata_version", std::to_string(metadata_version), + ops.emplace_back(zkutil::makeCreateRequest(replica_path + "/metadata_version", toString(metadata_snapshot->getMetadataVersion()), zkutil::CreateMode::Persistent)); /// The following 3 nodes were added in version 1.1.xxx, so we create them here, not in createNewZooKeeperNodes() @@ -1159,16 +1159,19 @@ void StorageReplicatedMergeTree::checkTableStructure(const String & zookeeper_pr } void StorageReplicatedMergeTree::setTableStructure(const StorageID & table_id, const ContextPtr & local_context, - ColumnsDescription new_columns, const ReplicatedMergeTreeTableMetadata::Diff & metadata_diff) + ColumnsDescription new_columns, const ReplicatedMergeTreeTableMetadata::Diff & metadata_diff, int32_t new_metadata_version) { StorageInMemoryMetadata old_metadata = getInMemoryMetadata(); + StorageInMemoryMetadata new_metadata = metadata_diff.getNewMetadata(new_columns, local_context, old_metadata); + new_metadata.setMetadataVersion(new_metadata_version); /// Even if the primary/sorting/partition keys didn't change we must reinitialize it /// because primary/partition key column types might have changed. checkTTLExpressions(new_metadata, old_metadata); setProperties(new_metadata, old_metadata); + DatabaseCatalog::instance().getDatabase(table_id.database_name)->alterTable(local_context, table_id, new_metadata); } @@ -2782,8 +2785,9 @@ void StorageReplicatedMergeTree::cloneMetadataIfNeeded(const String & source_rep return; } + auto metadata_snapshot = getInMemoryMetadataPtr(); Int32 source_metadata_version = parse(source_metadata_version_str); - if (metadata_version == source_metadata_version) + if (metadata_snapshot->getMetadataVersion() == source_metadata_version) return; /// Our metadata it not up to date with source replica metadata. @@ -2801,7 +2805,7 @@ void StorageReplicatedMergeTree::cloneMetadataIfNeeded(const String & source_rep /// if all such entries were cleaned up from the log and source_queue. LOG_WARNING(log, "Metadata version ({}) on replica is not up to date with metadata ({}) on source replica {}", - metadata_version, source_metadata_version, source_replica); + metadata_snapshot->getMetadataVersion(), source_metadata_version, source_replica); String source_metadata; String source_columns; @@ -4980,14 +4984,15 @@ bool StorageReplicatedMergeTree::optimize( bool StorageReplicatedMergeTree::executeMetadataAlter(const StorageReplicatedMergeTree::LogEntry & entry) { - if (entry.alter_version < metadata_version) + auto current_metadata = getInMemoryMetadataPtr(); + if (entry.alter_version < current_metadata->getMetadataVersion()) { /// TODO Can we replace it with LOGICAL_ERROR? /// As for now, it may rarely happen due to reordering of ALTER_METADATA entries in the queue of /// non-initial replica and also may happen after stale replica recovery. LOG_WARNING(log, "Attempt to update metadata of version {} " "to older version {} when processing log entry {}: {}", - metadata_version, entry.alter_version, entry.znode_name, entry.toString()); + current_metadata->getMetadataVersion(), entry.alter_version, entry.znode_name, entry.toString()); return true; } @@ -5035,10 +5040,10 @@ bool StorageReplicatedMergeTree::executeMetadataAlter(const StorageReplicatedMer LOG_INFO(log, "Metadata changed in ZooKeeper. Applying changes locally."); auto metadata_diff = ReplicatedMergeTreeTableMetadata(*this, getInMemoryMetadataPtr()).checkAndFindDiff(metadata_from_entry, getInMemoryMetadataPtr()->getColumns(), getContext()); - setTableStructure(table_id, alter_context, std::move(columns_from_entry), metadata_diff); - metadata_version = entry.alter_version; + setTableStructure(table_id, alter_context, std::move(columns_from_entry), metadata_diff, entry.alter_version); - LOG_INFO(log, "Applied changes to the metadata of the table. Current metadata version: {}", metadata_version); + current_metadata = getInMemoryMetadataPtr(); + LOG_INFO(log, "Applied changes to the metadata of the table. Current metadata version: {}", current_metadata->getMetadataVersion()); } { @@ -5050,7 +5055,7 @@ bool StorageReplicatedMergeTree::executeMetadataAlter(const StorageReplicatedMer /// This transaction may not happen, but it's OK, because on the next retry we will eventually create/update this node /// TODO Maybe do in in one transaction for Replicated database? - zookeeper->createOrUpdate(fs::path(replica_path) / "metadata_version", std::to_string(metadata_version), zkutil::CreateMode::Persistent); + zookeeper->createOrUpdate(fs::path(replica_path) / "metadata_version", std::to_string(current_metadata->getMetadataVersion()), zkutil::CreateMode::Persistent); return true; } @@ -5174,7 +5179,7 @@ void StorageReplicatedMergeTree::alter( size_t mutation_path_idx = std::numeric_limits::max(); String new_metadata_str = future_metadata_in_zk.toString(); - ops.emplace_back(zkutil::makeSetRequest(fs::path(zookeeper_path) / "metadata", new_metadata_str, metadata_version)); + ops.emplace_back(zkutil::makeSetRequest(fs::path(zookeeper_path) / "metadata", new_metadata_str, current_metadata->getMetadataVersion())); String new_columns_str = future_metadata.columns.toString(); ops.emplace_back(zkutil::makeSetRequest(fs::path(zookeeper_path) / "columns", new_columns_str, -1)); @@ -5190,7 +5195,7 @@ void StorageReplicatedMergeTree::alter( /// We can be sure, that in case of successful commit in zookeeper our /// version will increments by 1. Because we update with version check. - int new_metadata_version = metadata_version + 1; + int new_metadata_version = current_metadata->getMetadataVersion() + 1; alter_entry->type = LogEntry::ALTER_METADATA; alter_entry->source_replica = replica_name; @@ -7964,9 +7969,9 @@ bool StorageReplicatedMergeTree::canUseAdaptiveGranularity() const } -MutationCommands StorageReplicatedMergeTree::getFirstAlterMutationCommandsForPart(const DataPartPtr & part) const +std::map StorageReplicatedMergeTree::getAlterMutationCommandsForPart(const DataPartPtr & part) const { - return queue.getFirstAlterMutationCommandsForPart(part); + return queue.getAlterMutationCommandsForPart(part); } diff --git a/src/Storages/StorageReplicatedMergeTree.h b/src/Storages/StorageReplicatedMergeTree.h index 46c78e9064a..ac8229481f9 100644 --- a/src/Storages/StorageReplicatedMergeTree.h +++ b/src/Storages/StorageReplicatedMergeTree.h @@ -215,8 +215,6 @@ public: /// It's used if not set in engine's arguments while creating a replicated table. static String getDefaultReplicaName(const ContextPtr & context_); - int getMetadataVersion() const { return metadata_version; } - /// Modify a CREATE TABLE query to make a variant which must be written to a backup. void adjustCreateQueryForBackup(ASTPtr & create_query) const override; @@ -428,7 +426,6 @@ private: std::atomic shutdown_called {false}; std::atomic flush_called {false}; - int metadata_version = 0; /// Threads. /// A task that keeps track of the updates in the logs of all replicas and loads them into the queue. @@ -505,8 +502,10 @@ private: /// A part of ALTER: apply metadata changes only (data parts are altered separately). /// Must be called under IStorage::lockForAlter() lock. - void setTableStructure(const StorageID & table_id, const ContextPtr & local_context, - ColumnsDescription new_columns, const ReplicatedMergeTreeTableMetadata::Diff & metadata_diff); + void setTableStructure( + const StorageID & table_id, const ContextPtr & local_context, + ColumnsDescription new_columns, const ReplicatedMergeTreeTableMetadata::Diff & metadata_diff, + int32_t new_metadata_version); /** Check that the set of parts corresponds to that in ZK (/replicas/me/parts/). * If any parts described in ZK are not locally, throw an exception. @@ -833,7 +832,7 @@ private: void waitMutationToFinishOnReplicas( const Strings & replicas, const String & mutation_id) const; - MutationCommands getFirstAlterMutationCommandsForPart(const DataPartPtr & part) const override; + std::map getAlterMutationCommandsForPart(const DataPartPtr & part) const override; void startBackgroundMovesIfNeeded() override; diff --git a/tests/integration/test_merge_tree_hdfs/test.py b/tests/integration/test_merge_tree_hdfs/test.py index 3950077e619..782237539fa 100644 --- a/tests/integration/test_merge_tree_hdfs/test.py +++ b/tests/integration/test_merge_tree_hdfs/test.py @@ -43,8 +43,18 @@ def create_table(cluster, table_name, additional_settings=None): FILES_OVERHEAD = 1 FILES_OVERHEAD_PER_COLUMN = 2 # Data and mark files -FILES_OVERHEAD_PER_PART_WIDE = FILES_OVERHEAD_PER_COLUMN * 3 + 2 + 6 + 1 -FILES_OVERHEAD_PER_PART_COMPACT = 10 + 1 +FILES_OVERHEAD_DEFAULT_COMPRESSION_CODEC = 1 +FILES_OVERHEAD_METADATA_VERSION = 1 +FILES_OVERHEAD_PER_PART_WIDE = ( + FILES_OVERHEAD_PER_COLUMN * 3 + + 2 + + 6 + + FILES_OVERHEAD_DEFAULT_COMPRESSION_CODEC + + FILES_OVERHEAD_METADATA_VERSION +) +FILES_OVERHEAD_PER_PART_COMPACT = ( + 10 + FILES_OVERHEAD_DEFAULT_COMPRESSION_CODEC + FILES_OVERHEAD_METADATA_VERSION +) @pytest.fixture(scope="module") diff --git a/tests/integration/test_merge_tree_s3/test.py b/tests/integration/test_merge_tree_s3/test.py index f0f81100320..696c016f760 100644 --- a/tests/integration/test_merge_tree_s3/test.py +++ b/tests/integration/test_merge_tree_s3/test.py @@ -52,8 +52,18 @@ def cluster(): FILES_OVERHEAD = 1 FILES_OVERHEAD_PER_COLUMN = 2 # Data and mark files -FILES_OVERHEAD_PER_PART_WIDE = FILES_OVERHEAD_PER_COLUMN * 3 + 2 + 6 + 1 -FILES_OVERHEAD_PER_PART_COMPACT = 10 + 1 +FILES_OVERHEAD_DEFAULT_COMPRESSION_CODEC = 1 +FILES_OVERHEAD_METADATA_VERSION = 1 +FILES_OVERHEAD_PER_PART_WIDE = ( + FILES_OVERHEAD_PER_COLUMN * 3 + + 2 + + 6 + + FILES_OVERHEAD_DEFAULT_COMPRESSION_CODEC + + FILES_OVERHEAD_METADATA_VERSION +) +FILES_OVERHEAD_PER_PART_COMPACT = ( + 10 + FILES_OVERHEAD_DEFAULT_COMPRESSION_CODEC + FILES_OVERHEAD_METADATA_VERSION +) def create_table(node, table_name, **additional_settings): @@ -232,7 +242,6 @@ def test_insert_same_partition_and_merge(cluster, merge_vertical, node_name): def test_alter_table_columns(cluster, node_name): node = cluster.instances[node_name] create_table(node, "s3_test") - minio = cluster.minio_client node.query( "INSERT INTO s3_test VALUES {}".format(generate_values("2020-01-03", 4096)) diff --git a/tests/integration/test_merge_tree_s3_failover/test.py b/tests/integration/test_merge_tree_s3_failover/test.py index c61cacc9d8c..fef2e26bd2e 100644 --- a/tests/integration/test_merge_tree_s3_failover/test.py +++ b/tests/integration/test_merge_tree_s3_failover/test.py @@ -89,7 +89,7 @@ def drop_table(cluster): # S3 request will be failed for an appropriate part file write. -FILES_PER_PART_BASE = 5 # partition.dat, default_compression_codec.txt, count.txt, columns.txt, checksums.txt +FILES_PER_PART_BASE = 6 # partition.dat, metadata_version.txt, default_compression_codec.txt, count.txt, columns.txt, checksums.txt FILES_PER_PART_WIDE = ( FILES_PER_PART_BASE + 1 + 1 + 3 * 2 ) # Primary index, MinMax, Mark and data file for column(s) diff --git a/tests/integration/test_partition/test.py b/tests/integration/test_partition/test.py index ae4393fc6f6..b59cc21f39f 100644 --- a/tests/integration/test_partition/test.py +++ b/tests/integration/test_partition/test.py @@ -105,6 +105,8 @@ def partition_complex_assert_checksums(): "c4ca4238a0b923820dcc509a6f75849b\tshadow/1/data/test/partition_complex/19700102_2_2_0/count.txt\n" "c4ca4238a0b923820dcc509a6f75849b\tshadow/1/data/test/partition_complex/19700201_1_1_0/count.txt\n" "cfcb770c3ecd0990dcceb1bde129e6c6\tshadow/1/data/test/partition_complex/19700102_2_2_0/p.bin\n" + "cfcd208495d565ef66e7dff9f98764da\tshadow/1/data/test/partition_complex/19700102_2_2_0/metadata_version.txt\n" + "cfcd208495d565ef66e7dff9f98764da\tshadow/1/data/test/partition_complex/19700201_1_1_0/metadata_version.txt\n" "e2af3bef1fd129aea73a890ede1e7a30\tshadow/1/data/test/partition_complex/19700201_1_1_0/k.bin\n" "f2312862cc01adf34a93151377be2ddf\tshadow/1/data/test/partition_complex/19700201_1_1_0/minmax_p.idx\n" ) diff --git a/tests/integration/test_replicated_merge_tree_s3/test.py b/tests/integration/test_replicated_merge_tree_s3/test.py index 0d978bb6967..b90e28dfdb2 100644 --- a/tests/integration/test_replicated_merge_tree_s3/test.py +++ b/tests/integration/test_replicated_merge_tree_s3/test.py @@ -44,8 +44,18 @@ def cluster(): FILES_OVERHEAD = 1 FILES_OVERHEAD_PER_COLUMN = 2 # Data and mark files -FILES_OVERHEAD_PER_PART_WIDE = FILES_OVERHEAD_PER_COLUMN * 3 + 2 + 6 + 1 -FILES_OVERHEAD_PER_PART_COMPACT = 10 + 1 +FILES_OVERHEAD_DEFAULT_COMPRESSION_CODEC = 1 +FILES_OVERHEAD_METADATA_VERSION = 1 +FILES_OVERHEAD_PER_PART_WIDE = ( + FILES_OVERHEAD_PER_COLUMN * 3 + + 2 + + 6 + + FILES_OVERHEAD_DEFAULT_COMPRESSION_CODEC + + FILES_OVERHEAD_METADATA_VERSION +) +FILES_OVERHEAD_PER_PART_COMPACT = ( + 10 + FILES_OVERHEAD_DEFAULT_COMPRESSION_CODEC + FILES_OVERHEAD_METADATA_VERSION +) def random_string(length): diff --git a/tests/integration/test_replicated_merge_tree_s3_zero_copy/test.py b/tests/integration/test_replicated_merge_tree_s3_zero_copy/test.py index f0bc12e3125..62ca73c210b 100644 --- a/tests/integration/test_replicated_merge_tree_s3_zero_copy/test.py +++ b/tests/integration/test_replicated_merge_tree_s3_zero_copy/test.py @@ -47,8 +47,18 @@ def cluster(): FILES_OVERHEAD = 1 FILES_OVERHEAD_PER_COLUMN = 2 # Data and mark files -FILES_OVERHEAD_PER_PART_WIDE = FILES_OVERHEAD_PER_COLUMN * 3 + 2 + 6 + 1 -FILES_OVERHEAD_PER_PART_COMPACT = 10 + 1 +FILES_OVERHEAD_DEFAULT_COMPRESSION_CODEC = 1 +FILES_OVERHEAD_METADATA_VERSION = 1 +FILES_OVERHEAD_PER_PART_WIDE = ( + FILES_OVERHEAD_PER_COLUMN * 3 + + 2 + + 6 + + FILES_OVERHEAD_DEFAULT_COMPRESSION_CODEC + + FILES_OVERHEAD_METADATA_VERSION +) +FILES_OVERHEAD_PER_PART_COMPACT = ( + 10 + FILES_OVERHEAD_DEFAULT_COMPRESSION_CODEC + FILES_OVERHEAD_METADATA_VERSION +) def random_string(length): diff --git a/tests/integration/test_s3_zero_copy_ttl/test.py b/tests/integration/test_s3_zero_copy_ttl/test.py index 9a782aacef6..7dcf3734653 100644 --- a/tests/integration/test_s3_zero_copy_ttl/test.py +++ b/tests/integration/test_s3_zero_copy_ttl/test.py @@ -86,9 +86,9 @@ def test_ttl_move_and_s3(started_cluster): print(f"Total objects: {counter}") - if counter == 300: + if counter == 330: break print(f"Attempts remaining: {attempt}") - assert counter == 300 + assert counter == 330 diff --git a/tests/queries/0_stateless/01278_alter_rename_combination.reference b/tests/queries/0_stateless/01278_alter_rename_combination.reference index cc912e9b265..e70c2d2e6f8 100644 --- a/tests/queries/0_stateless/01278_alter_rename_combination.reference +++ b/tests/queries/0_stateless/01278_alter_rename_combination.reference @@ -1,7 +1,7 @@ -CREATE TABLE default.rename_table\n(\n `key` Int32,\n `old_value1` Int32,\n `value1` Int32\n)\nENGINE = MergeTree\nORDER BY tuple()\nSETTINGS index_granularity = 8192 +CREATE TABLE default.rename_table\n(\n `key` Int32,\n `old_value1` Int32,\n `value1` Int32\n)\nENGINE = MergeTree\nORDER BY tuple()\nSETTINGS min_bytes_for_wide_part = 0, index_granularity = 8192 key old_value1 value1 1 2 3 -CREATE TABLE default.rename_table\n(\n `k` Int32,\n `v1` Int32,\n `v2` Int32\n)\nENGINE = MergeTree\nORDER BY tuple()\nSETTINGS index_granularity = 8192 +CREATE TABLE default.rename_table\n(\n `k` Int32,\n `v1` Int32,\n `v2` Int32\n)\nENGINE = MergeTree\nORDER BY tuple()\nSETTINGS min_bytes_for_wide_part = 0, index_granularity = 8192 k v1 v2 1 2 3 4 5 6 diff --git a/tests/queries/0_stateless/01278_alter_rename_combination.sql b/tests/queries/0_stateless/01278_alter_rename_combination.sql index fa73362622c..51322f5d86f 100644 --- a/tests/queries/0_stateless/01278_alter_rename_combination.sql +++ b/tests/queries/0_stateless/01278_alter_rename_combination.sql @@ -1,6 +1,6 @@ DROP TABLE IF EXISTS rename_table; -CREATE TABLE rename_table (key Int32, value1 Int32, value2 Int32) ENGINE = MergeTree ORDER BY tuple(); +CREATE TABLE rename_table (key Int32, value1 Int32, value2 Int32) ENGINE = MergeTree ORDER BY tuple() SETTINGS min_bytes_for_wide_part=0; INSERT INTO rename_table VALUES (1, 2, 3); diff --git a/tests/queries/0_stateless/01281_alter_rename_and_other_renames.reference b/tests/queries/0_stateless/01281_alter_rename_and_other_renames.reference index bf3358aea60..532b8ce8712 100644 --- a/tests/queries/0_stateless/01281_alter_rename_and_other_renames.reference +++ b/tests/queries/0_stateless/01281_alter_rename_and_other_renames.reference @@ -1,11 +1,11 @@ -CREATE TABLE default.rename_table_multiple\n(\n `key` Int32,\n `value1_string` String,\n `value2` Int32\n)\nENGINE = MergeTree\nORDER BY tuple()\nSETTINGS index_granularity = 8192 +CREATE TABLE default.rename_table_multiple\n(\n `key` Int32,\n `value1_string` String,\n `value2` Int32\n)\nENGINE = MergeTree\nORDER BY tuple()\nSETTINGS min_bytes_for_wide_part = 0, index_granularity = 8192 key value1_string value2 1 2 3 -CREATE TABLE default.rename_table_multiple\n(\n `key` Int32,\n `value1_string` String,\n `value2_old` Int32,\n `value2` Int64 DEFAULT 7\n)\nENGINE = MergeTree\nORDER BY tuple()\nSETTINGS index_granularity = 8192 +CREATE TABLE default.rename_table_multiple\n(\n `key` Int32,\n `value1_string` String,\n `value2_old` Int32,\n `value2` Int64 DEFAULT 7\n)\nENGINE = MergeTree\nORDER BY tuple()\nSETTINGS min_bytes_for_wide_part = 0, index_granularity = 8192 key value1_string value2_old value2 1 2 3 7 4 5 6 7 -CREATE TABLE default.rename_table_multiple\n(\n `key` Int32,\n `value1_string` String,\n `value2_old` Int64 DEFAULT 7\n)\nENGINE = MergeTree\nORDER BY tuple()\nSETTINGS index_granularity = 8192 +CREATE TABLE default.rename_table_multiple\n(\n `key` Int32,\n `value1_string` String,\n `value2_old` Int64 DEFAULT 7\n)\nENGINE = MergeTree\nORDER BY tuple()\nSETTINGS min_bytes_for_wide_part = 0, index_granularity = 8192 key value1_string value2_old 1 2 7 4 5 7 diff --git a/tests/queries/0_stateless/01281_alter_rename_and_other_renames.sql b/tests/queries/0_stateless/01281_alter_rename_and_other_renames.sql index f9462f0478e..b0ccd7751ab 100644 --- a/tests/queries/0_stateless/01281_alter_rename_and_other_renames.sql +++ b/tests/queries/0_stateless/01281_alter_rename_and_other_renames.sql @@ -1,6 +1,6 @@ DROP TABLE IF EXISTS rename_table_multiple; -CREATE TABLE rename_table_multiple (key Int32, value1 String, value2 Int32) ENGINE = MergeTree ORDER BY tuple(); +CREATE TABLE rename_table_multiple (key Int32, value1 String, value2 Int32) ENGINE = MergeTree ORDER BY tuple() SETTINGS min_bytes_for_wide_part=0; INSERT INTO rename_table_multiple VALUES (1, 2, 3); diff --git a/tests/queries/0_stateless/02241_filesystem_cache_on_write_operations.reference b/tests/queries/0_stateless/02241_filesystem_cache_on_write_operations.reference index bbca9bbbfee..f3fac9b32d3 100644 --- a/tests/queries/0_stateless/02241_filesystem_cache_on_write_operations.reference +++ b/tests/queries/0_stateless/02241_filesystem_cache_on_write_operations.reference @@ -7,25 +7,25 @@ file_segment_range_begin: 0 file_segment_range_end: 745 size: 746 state: DOWNLOADED -7 -7 +8 +8 0 2 2 -7 +8 Row 1: ────── file_segment_range_begin: 0 file_segment_range_end: 1659 size: 1660 state: DOWNLOADED -7 -7 -7 -7 -21 -31 -38 +8 +8 +8 +8 +24 +35 +43 5010500 18816 Using storage policy: local_cache @@ -37,24 +37,24 @@ file_segment_range_begin: 0 file_segment_range_end: 745 size: 746 state: DOWNLOADED -7 -7 +8 +8 0 2 2 -7 +8 Row 1: ────── file_segment_range_begin: 0 file_segment_range_end: 1659 size: 1660 state: DOWNLOADED -7 -7 -7 -7 -21 -31 -38 +8 +8 +8 +8 +24 +35 +43 5010500 18816 diff --git a/tests/queries/0_stateless/02361_fsync_profile_events.sh b/tests/queries/0_stateless/02361_fsync_profile_events.sh index 44a1bd58d36..5b603133f6c 100755 --- a/tests/queries/0_stateless/02361_fsync_profile_events.sh +++ b/tests/queries/0_stateless/02361_fsync_profile_events.sh @@ -44,8 +44,8 @@ for i in {1..100}; do ")" # Non retriable errors - if [[ $FileSync -ne 7 ]]; then - echo "FileSync: $FileSync != 11" >&2 + if [[ $FileSync -ne 8 ]]; then + echo "FileSync: $FileSync != 8" >&2 exit 2 fi # Check that all files was synced diff --git a/tests/queries/0_stateless/02538_alter_rename_sequence.reference b/tests/queries/0_stateless/02538_alter_rename_sequence.reference new file mode 100644 index 00000000000..73aa1b7e8d8 --- /dev/null +++ b/tests/queries/0_stateless/02538_alter_rename_sequence.reference @@ -0,0 +1,8 @@ +1 2 3 +4 5 6 +{"column1_renamed":"1","column2_renamed":"2","column3":"3"} +{"column1_renamed":"4","column2_renamed":"5","column3":"6"} +1 2 3 +4 5 6 +{"column1_renamed":"1","column2_renamed":"2","column3":"3"} +{"column1_renamed":"4","column2_renamed":"5","column3":"6"} diff --git a/tests/queries/0_stateless/02538_alter_rename_sequence.sql b/tests/queries/0_stateless/02538_alter_rename_sequence.sql new file mode 100644 index 00000000000..d7df27dc702 --- /dev/null +++ b/tests/queries/0_stateless/02538_alter_rename_sequence.sql @@ -0,0 +1,59 @@ +DROP TABLE IF EXISTS wrong_metadata; + +CREATE TABLE wrong_metadata( + column1 UInt64, + column2 UInt64, + column3 UInt64 +) +ENGINE ReplicatedMergeTree('/test/{database}/tables/wrong_metadata', '1') +ORDER BY tuple(); + +INSERT INTO wrong_metadata VALUES (1, 2, 3); + +SYSTEM STOP REPLICATION QUEUES wrong_metadata; + +ALTER TABLE wrong_metadata RENAME COLUMN column1 TO column1_renamed SETTINGS replication_alter_partitions_sync = 0; + +INSERT INTO wrong_metadata VALUES (4, 5, 6); + +SELECT * FROM wrong_metadata ORDER BY column1; + +SYSTEM START REPLICATION QUEUES wrong_metadata; + +SYSTEM SYNC REPLICA wrong_metadata; + +ALTER TABLE wrong_metadata RENAME COLUMN column2 to column2_renamed SETTINGS replication_alter_partitions_sync = 2; + +SELECT * FROM wrong_metadata ORDER BY column1_renamed FORMAT JSONEachRow; + +DROP TABLE IF EXISTS wrong_metadata; + + +CREATE TABLE wrong_metadata_wide( + column1 UInt64, + column2 UInt64, + column3 UInt64 +) +ENGINE ReplicatedMergeTree('/test/{database}/tables/wrong_metadata_wide', '1') +ORDER BY tuple() +SETTINGS min_bytes_for_wide_part = 0; + +INSERT INTO wrong_metadata_wide VALUES (1, 2, 3); + +SYSTEM STOP REPLICATION QUEUES wrong_metadata_wide; + +ALTER TABLE wrong_metadata_wide RENAME COLUMN column1 TO column1_renamed SETTINGS replication_alter_partitions_sync = 0; + +INSERT INTO wrong_metadata_wide VALUES (4, 5, 6); + +SELECT * FROM wrong_metadata_wide ORDER by column1; + +SYSTEM START REPLICATION QUEUES wrong_metadata_wide; + +SYSTEM SYNC REPLICA wrong_metadata_wide; + +ALTER TABLE wrong_metadata_wide RENAME COLUMN column2 to column2_renamed SETTINGS replication_alter_partitions_sync = 2; + +SELECT * FROM wrong_metadata_wide ORDER BY column1_renamed FORMAT JSONEachRow; + +DROP TABLE IF EXISTS wrong_metadata_wide; diff --git a/tests/queries/0_stateless/02543_alter_rename_modify_stuck.reference b/tests/queries/0_stateless/02543_alter_rename_modify_stuck.reference new file mode 100644 index 00000000000..156128e3dd2 --- /dev/null +++ b/tests/queries/0_stateless/02543_alter_rename_modify_stuck.reference @@ -0,0 +1 @@ +{"v":"1","v2":"77"} diff --git a/tests/queries/0_stateless/02543_alter_rename_modify_stuck.sh b/tests/queries/0_stateless/02543_alter_rename_modify_stuck.sh new file mode 100755 index 00000000000..adaf1846552 --- /dev/null +++ b/tests/queries/0_stateless/02543_alter_rename_modify_stuck.sh @@ -0,0 +1,58 @@ +#!/usr/bin/env bash + +CURDIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd) +# shellcheck source=../shell_config.sh +. "$CURDIR"/../shell_config.sh + + +$CLICKHOUSE_CLIENT --query="DROP TABLE IF EXISTS table_to_rename" + +$CLICKHOUSE_CLIENT --query="CREATE TABLE table_to_rename(v UInt64, v1 UInt64)ENGINE = MergeTree ORDER BY tuple() SETTINGS min_bytes_for_wide_part = 0" + +$CLICKHOUSE_CLIENT --query="INSERT INTO table_to_rename VALUES (1, 1)" + + +# we want to following mutations to stuck +# That is why we stop merges and wait in loops until they actually start +$CLICKHOUSE_CLIENT --query="SYSTEM STOP MERGES table_to_rename" + +$CLICKHOUSE_CLIENT --query="ALTER TABLE table_to_rename RENAME COLUMN v1 to v2" & + +counter=0 retries=60 + +I=0 +while [[ $counter -lt $retries ]]; do + I=$((I + 1)) + result=$($CLICKHOUSE_CLIENT --query "show create table table_to_rename") + if [[ $result == *"v2"* ]]; then + break; + fi + sleep 0.1 + ((++counter)) +done + + +$CLICKHOUSE_CLIENT --query="ALTER TABLE table_to_rename UPDATE v2 = 77 WHERE 1 = 1 SETTINGS mutations_sync = 2" & + +counter=0 retries=60 + +I=0 +while [[ $counter -lt $retries ]]; do + I=$((I + 1)) + result=$($CLICKHOUSE_CLIENT --query "SELECT count() from system.mutations where database='${CLICKHOUSE_DATABASE}' and table='table_to_rename'") + if [[ $result == "2" ]]; then + break; + fi + sleep 0.1 + ((++counter)) +done + + +$CLICKHOUSE_CLIENT --query="SYSTEM START MERGES table_to_rename" + +wait + +$CLICKHOUSE_CLIENT --query="SELECT * FROM table_to_rename FORMAT JSONEachRow" + + + $CLICKHOUSE_CLIENT --query="DROP TABLE IF EXISTS table_to_rename" diff --git a/tests/queries/0_stateless/02555_davengers_rename_chain.reference b/tests/queries/0_stateless/02555_davengers_rename_chain.reference new file mode 100644 index 00000000000..a9fc4b395e2 --- /dev/null +++ b/tests/queries/0_stateless/02555_davengers_rename_chain.reference @@ -0,0 +1,26 @@ +{"a1":"1","b1":"2","c":"3"} +~~~~~~~ +{"a1":"1","b1":"2","c":"3"} +{"a1":"4","b1":"5","c":"6"} +~~~~~~~ +{"a1":"1","b1":"2","c":"3"} +{"a1":"4","b1":"5","c":"6"} +{"a1":"7","b1":"8","c":"9"} +~~~~~~~ +{"b":"1","a":"2","c":"3"} +{"b":"4","a":"5","c":"6"} +{"b":"7","a":"8","c":"9"} +~~~~~~~ +{"a1":"1","b1":"2","c":"3"} +~~~~~~~ +{"a1":"1","b1":"2","c":"3"} +{"a1":"4","b1":"5","c":"6"} +~~~~~~~ +{"a1":"1","b1":"2","c":"3"} +{"a1":"4","b1":"5","c":"6"} +{"a1":"7","b1":"8","c":"9"} +~~~~~~~ +{"b":"1","a":"2","c":"3"} +{"b":"4","a":"5","c":"6"} +{"b":"7","a":"8","c":"9"} +~~~~~~~ diff --git a/tests/queries/0_stateless/02555_davengers_rename_chain.sh b/tests/queries/0_stateless/02555_davengers_rename_chain.sh new file mode 100755 index 00000000000..b23f8085fd7 --- /dev/null +++ b/tests/queries/0_stateless/02555_davengers_rename_chain.sh @@ -0,0 +1,143 @@ +#!/usr/bin/env bash +# Tags: replica +CUR_DIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd) +# shellcheck source=../shell_config.sh +. "$CUR_DIR"/../shell_config.sh + +$CLICKHOUSE_CLIENT --query="DROP TABLE IF EXISTS wrong_metadata" + +$CLICKHOUSE_CLIENT -n --query="CREATE TABLE wrong_metadata( + a UInt64, + b UInt64, + c UInt64 +) +ENGINE ReplicatedMergeTree('/test/{database}/tables/wrong_metadata', '1') +ORDER BY tuple() +SETTINGS min_bytes_for_wide_part = 0" + +$CLICKHOUSE_CLIENT --query="INSERT INTO wrong_metadata VALUES (1, 2, 3)" + + +$CLICKHOUSE_CLIENT --query="SYSTEM STOP MERGES wrong_metadata" + + +$CLICKHOUSE_CLIENT --query="ALTER TABLE wrong_metadata RENAME COLUMN a TO a1, RENAME COLUMN b to b1 SETTINGS replication_alter_partitions_sync = 0" + +counter=0 retries=60 +I=0 +while [[ $counter -lt $retries ]]; do + I=$((I + 1)) + result=$($CLICKHOUSE_CLIENT --query "SHOW CREATE TABLE wrong_metadata") + if [[ $result == *"\`a1\` UInt64"* ]]; then + break; + fi + sleep 0.1 + ((++counter)) +done + + +$CLICKHOUSE_CLIENT --query="SELECT * FROM wrong_metadata ORDER BY a1 FORMAT JSONEachRow" + +$CLICKHOUSE_CLIENT --query="SELECT '~~~~~~~'" + +$CLICKHOUSE_CLIENT --query="INSERT INTO wrong_metadata VALUES (4, 5, 6)" + + +$CLICKHOUSE_CLIENT --query="SELECT * FROM wrong_metadata ORDER BY a1 FORMAT JSONEachRow" +$CLICKHOUSE_CLIENT --query="SELECT '~~~~~~~'" + + +$CLICKHOUSE_CLIENT --query="ALTER TABLE wrong_metadata RENAME COLUMN a1 TO b, RENAME COLUMN b1 to a SETTINGS replication_alter_partitions_sync = 0" + +counter=0 retries=60 +I=0 +while [[ $counter -lt $retries ]]; do + I=$((I + 1)) + result=$($CLICKHOUSE_CLIENT --query "SELECT * FROM system.mutations WHERE table = 'wrong_metadata' AND database='${CLICKHOUSE_DATABASE}'") + if [[ $result == *"b1 TO a"* ]]; then + break; + fi + sleep 0.1 + ((++counter)) +done + +$CLICKHOUSE_CLIENT --query="INSERT INTO wrong_metadata VALUES (7, 8, 9)" + +$CLICKHOUSE_CLIENT --query="SELECT * FROM wrong_metadata ORDER by a1 FORMAT JSONEachRow" +$CLICKHOUSE_CLIENT --query="SELECT '~~~~~~~'" + +$CLICKHOUSE_CLIENT --query="SYSTEM START MERGES wrong_metadata" + +$CLICKHOUSE_CLIENT --query="SYSTEM SYNC REPLICA wrong_metadata" + +$CLICKHOUSE_CLIENT --query="SELECT * FROM wrong_metadata order by a FORMAT JSONEachRow" + +$CLICKHOUSE_CLIENT --query="SELECT '~~~~~~~'" + + +$CLICKHOUSE_CLIENT --query="DROP TABLE IF EXISTS wrong_metadata" + +$CLICKHOUSE_CLIENT --query="DROP TABLE IF EXISTS wrong_metadata_compact" + +$CLICKHOUSE_CLIENT -n --query="CREATE TABLE wrong_metadata_compact( + a UInt64, + b UInt64, + c UInt64 +) +ENGINE ReplicatedMergeTree('/test/{database}/tables/wrong_metadata_compact', '1') +ORDER BY tuple() +SETTINGS min_bytes_for_wide_part = 10000000" + +$CLICKHOUSE_CLIENT --query="INSERT INTO wrong_metadata_compact VALUES (1, 2, 3)" + +$CLICKHOUSE_CLIENT --query="SYSTEM STOP MERGES wrong_metadata_compact" + +$CLICKHOUSE_CLIENT --query="ALTER TABLE wrong_metadata_compact RENAME COLUMN a TO a1, RENAME COLUMN b to b1 SETTINGS replication_alter_partitions_sync = 0" + +counter=0 retries=60 +I=0 +while [[ $counter -lt $retries ]]; do + I=$((I + 1)) + result=$($CLICKHOUSE_CLIENT --query "SHOW CREATE TABLE wrong_metadata_compact") + if [[ $result == *"\`a1\` UInt64"* ]]; then + break; + fi + sleep 0.1 + ((++counter)) +done + +$CLICKHOUSE_CLIENT --query="SELECT * FROM wrong_metadata_compact ORDER BY a1 FORMAT JSONEachRow" +$CLICKHOUSE_CLIENT --query="SELECT '~~~~~~~'" + +$CLICKHOUSE_CLIENT --query="INSERT INTO wrong_metadata_compact VALUES (4, 5, 6)" + +$CLICKHOUSE_CLIENT --query="SELECT * FROM wrong_metadata_compact ORDER BY a1 FORMAT JSONEachRow" +$CLICKHOUSE_CLIENT --query="SELECT '~~~~~~~'" + +$CLICKHOUSE_CLIENT --query="ALTER TABLE wrong_metadata_compact RENAME COLUMN a1 TO b, RENAME COLUMN b1 to a SETTINGS replication_alter_partitions_sync = 0" + +counter=0 retries=60 +I=0 +while [[ $counter -lt $retries ]]; do + I=$((I + 1)) + result=$($CLICKHOUSE_CLIENT --query "SELECT * FROM system.mutations WHERE table = 'wrong_metadata_compact' AND database='${CLICKHOUSE_DATABASE}'") + if [[ $result == *"b1 TO a"* ]]; then + break; + fi + sleep 0.1 + ((++counter)) +done + +$CLICKHOUSE_CLIENT --query="INSERT INTO wrong_metadata_compact VALUES (7, 8, 9)" + +$CLICKHOUSE_CLIENT --query="SELECT * FROM wrong_metadata_compact ORDER by a1 FORMAT JSONEachRow" +$CLICKHOUSE_CLIENT --query="SELECT '~~~~~~~'" + +$CLICKHOUSE_CLIENT --query="SYSTEM START MERGES wrong_metadata_compact" + +$CLICKHOUSE_CLIENT --query="SYSTEM SYNC REPLICA wrong_metadata_compact" + +$CLICKHOUSE_CLIENT --query="SELECT * FROM wrong_metadata_compact order by a FORMAT JSONEachRow" +$CLICKHOUSE_CLIENT --query="SELECT '~~~~~~~'" + +$CLICKHOUSE_CLIENT --query="DROP TABLE IF EXISTS wrong_metadata_compact" From 9dec64c94cbfb21dd259a1974b32f931f295edcd Mon Sep 17 00:00:00 2001 From: alesapin Date: Mon, 27 Feb 2023 14:35:26 +0100 Subject: [PATCH 002/103] Choose correct version for mutation --- src/Storages/MergeTree/MutateTask.cpp | 42 +------------------ .../MergeTree/ReplicatedMergeTreeQueue.cpp | 19 ++++++++- 2 files changed, 20 insertions(+), 41 deletions(-) diff --git a/src/Storages/MergeTree/MutateTask.cpp b/src/Storages/MergeTree/MutateTask.cpp index ae34b1b707c..5f466862856 100644 --- a/src/Storages/MergeTree/MutateTask.cpp +++ b/src/Storages/MergeTree/MutateTask.cpp @@ -183,40 +183,6 @@ static void splitAndModifyMutationCommands( } } -/// It's legal to squash renames because commands with rename are always "barrier" -/// and executed separately from other types of commands. -static MutationCommands squashRenamesInCommands(const MutationCommands & commands) -{ - NameToNameMap squashed_renames; - for (const auto & command : commands) - { - std::string result_name = command.rename_to; - - bool squashed = false; - for (const auto & [name_from, name_to] : squashed_renames) - { - if (name_to == command.column_name) - { - squashed = true; - squashed_renames[name_from] = result_name; - break; - } - } - if (!squashed) - squashed_renames[command.column_name] = result_name; - } - - MutationCommands squashed_commands; - for (const auto & command : commands) - { - if (squashed_renames.contains(command.column_name)) - { - squashed_commands.push_back(command); - squashed_commands.back().rename_to = squashed_renames[command.column_name]; - } - } - return squashed_commands; -} /// Get the columns list of the resulting part in the same order as storage_columns. static std::pair @@ -247,9 +213,7 @@ getColumnsForNewDataPart( storage_columns.emplace_back(column); } - MutationCommands squashed_commands = squashRenamesInCommands(all_commands); - - for (const auto & command : squashed_commands) + for (const auto & command : all_commands) { if (command.type == MutationCommand::UPDATE) { @@ -647,10 +611,8 @@ static NameToNameVector collectFilesForRenames( rename_vector.emplace_back(file_rename_from, file_rename_to); }; - MutationCommands squashed_commands = squashRenamesInCommands(commands_for_removes); - /// Remove old data - for (const auto & command : squashed_commands) + for (const auto & command : commands_for_removes) { if (command.type == MutationCommand::Type::DROP_INDEX) { diff --git a/src/Storages/MergeTree/ReplicatedMergeTreeQueue.cpp b/src/Storages/MergeTree/ReplicatedMergeTreeQueue.cpp index bac5beb6364..e54007e339c 100644 --- a/src/Storages/MergeTree/ReplicatedMergeTreeQueue.cpp +++ b/src/Storages/MergeTree/ReplicatedMergeTreeQueue.cpp @@ -2412,12 +2412,26 @@ std::optional> ReplicatedMergeTreeMergePredicate::getDesir return {}; Int64 current_version = queue.getCurrentMutationVersionImpl(part->info.partition_id, part->info.getDataVersion(), lock); - Int64 max_version = in_partition->second.rbegin()->first; + Int64 max_version = in_partition->second.begin()->first; int alter_version = -1; + bool barrier_found = false; for (auto [mutation_version, mutation_status] : in_partition->second) { + /// Some commands cannot stick together with other commands + if (mutation_status->entry->commands.containBarrierCommand()) + { + /// We already collected some mutation, we don't want to stick it with barrier + if (max_version != mutation_version && max_version > current_version) + break; + + /// This mutations is fresh, but it's barrier, let's execute only it + if (mutation_version > current_version) + barrier_found = true; + } + max_version = mutation_version; + if (mutation_status->entry->isAlterMutation()) { /// We want to assign mutations for part which version is bigger @@ -2430,6 +2444,9 @@ std::optional> ReplicatedMergeTreeMergePredicate::getDesir break; } } + + if (barrier_found == true) + break; } if (current_version >= max_version) From 7adbe93105948b3bf62d86aa98653a7c09df50f3 Mon Sep 17 00:00:00 2001 From: alesapin Date: Mon, 27 Feb 2023 14:48:47 +0100 Subject: [PATCH 003/103] Remove strange part --- src/Storages/MergeTree/ReplicatedMergeTreeQueue.cpp | 10 +--------- 1 file changed, 1 insertion(+), 9 deletions(-) diff --git a/src/Storages/MergeTree/ReplicatedMergeTreeQueue.cpp b/src/Storages/MergeTree/ReplicatedMergeTreeQueue.cpp index e54007e339c..eea21d6349c 100644 --- a/src/Storages/MergeTree/ReplicatedMergeTreeQueue.cpp +++ b/src/Storages/MergeTree/ReplicatedMergeTreeQueue.cpp @@ -1832,15 +1832,7 @@ MutationCommands ReplicatedMergeTreeQueue::getMutationCommands( for (auto it = begin; it != end; ++it) { const auto & commands_from_entry = it->second->entry->commands; - - if (commands_from_entry.containBarrierCommand()) - { - if (commands.empty()) - commands.insert(commands.end(), commands_from_entry.begin(), commands_from_entry.end()); - break; - } - else - commands.insert(commands.end(), commands_from_entry.begin(), commands_from_entry.end()); + commands.insert(commands.end(), commands_from_entry.begin(), commands_from_entry.end()); } return commands; From 967a47d870a11e8c8a07e36b82e9245c3517aa3a Mon Sep 17 00:00:00 2001 From: alesapin Date: Tue, 28 Feb 2023 12:17:43 +0100 Subject: [PATCH 004/103] Fix bug with readonly storages --- .../MergeTree/DataPartStorageOnDiskBase.cpp | 5 ++ .../MergeTree/DataPartStorageOnDiskBase.h | 1 + src/Storages/MergeTree/IDataPartStorage.h | 1 + src/Storages/MergeTree/IMergeTreeDataPart.cpp | 14 +++-- .../test_disk_over_web_server/test.py | 56 ++++++++++++++----- 5 files changed, 59 insertions(+), 18 deletions(-) diff --git a/src/Storages/MergeTree/DataPartStorageOnDiskBase.cpp b/src/Storages/MergeTree/DataPartStorageOnDiskBase.cpp index 260bf524ed2..c95a9b7f6f4 100644 --- a/src/Storages/MergeTree/DataPartStorageOnDiskBase.cpp +++ b/src/Storages/MergeTree/DataPartStorageOnDiskBase.cpp @@ -213,6 +213,11 @@ bool DataPartStorageOnDiskBase::isBroken() const return volume->getDisk()->isBroken(); } +bool DataPartStorageOnDiskBase::isReadonly() const +{ + return volume->getDisk()->isReadOnly(); +} + void DataPartStorageOnDiskBase::syncRevision(UInt64 revision) const { volume->getDisk()->syncRevision(revision); diff --git a/src/Storages/MergeTree/DataPartStorageOnDiskBase.h b/src/Storages/MergeTree/DataPartStorageOnDiskBase.h index 7c408dcf381..52544bb2457 100644 --- a/src/Storages/MergeTree/DataPartStorageOnDiskBase.h +++ b/src/Storages/MergeTree/DataPartStorageOnDiskBase.h @@ -39,6 +39,7 @@ public: bool supportZeroCopyReplication() const override; bool supportParallelWrite() const override; bool isBroken() const override; + bool isReadonly() const override; void syncRevision(UInt64 revision) const override; UInt64 getRevision() const override; std::string getDiskPath() const override; diff --git a/src/Storages/MergeTree/IDataPartStorage.h b/src/Storages/MergeTree/IDataPartStorage.h index f92784cb0da..065d4d42dcb 100644 --- a/src/Storages/MergeTree/IDataPartStorage.h +++ b/src/Storages/MergeTree/IDataPartStorage.h @@ -151,6 +151,7 @@ public: virtual bool supportZeroCopyReplication() const { return false; } virtual bool supportParallelWrite() const = 0; virtual bool isBroken() const = 0; + virtual bool isReadonly() const = 0; /// TODO: remove or at least remove const. virtual void syncRevision(UInt64 revision) const = 0; diff --git a/src/Storages/MergeTree/IMergeTreeDataPart.cpp b/src/Storages/MergeTree/IMergeTreeDataPart.cpp index 671c6ca67c1..a03a2619993 100644 --- a/src/Storages/MergeTree/IMergeTreeDataPart.cpp +++ b/src/Storages/MergeTree/IMergeTreeDataPart.cpp @@ -1304,6 +1304,8 @@ void IMergeTreeDataPart::loadColumns(bool require) metadata_snapshot = metadata_snapshot->projections.get(name).metadata; NamesAndTypesList loaded_columns; + bool is_readonly_storage = getDataPartStorage().isReadonly(); + if (!metadata_manager->exists("columns.txt")) { /// We can get list of columns only from columns.txt in compact parts. @@ -1319,7 +1321,8 @@ void IMergeTreeDataPart::loadColumns(bool require) if (columns.empty()) throw Exception(ErrorCodes::NO_FILE_IN_DATA_PART, "No columns in part {}", name); - writeColumns(loaded_columns, {}); + if (!is_readonly_storage) + writeColumns(loaded_columns, {}); } else { @@ -1353,10 +1356,13 @@ void IMergeTreeDataPart::loadColumns(bool require) { loaded_metadata_version = metadata_snapshot->getMetadataVersion(); - writeMetadata(METADATA_VERSION_FILE_NAME, {}, [loaded_metadata_version](auto & buffer) + if (!is_readonly_storage) { - writeIntText(loaded_metadata_version, buffer); - }); + writeMetadata(METADATA_VERSION_FILE_NAME, {}, [loaded_metadata_version](auto & buffer) + { + writeIntText(loaded_metadata_version, buffer); + }); + } } setColumns(loaded_columns, infos, loaded_metadata_version); diff --git a/tests/integration/test_disk_over_web_server/test.py b/tests/integration/test_disk_over_web_server/test.py index 363df4595b2..2991ece05ec 100644 --- a/tests/integration/test_disk_over_web_server/test.py +++ b/tests/integration/test_disk_over_web_server/test.py @@ -21,23 +21,27 @@ def cluster(): cluster.add_instance( "node3", main_configs=["configs/storage_conf_web.xml"], with_nginx=True ) + + cluster.add_instance( + "node4", main_configs=["configs/storage_conf.xml"], + with_nginx=True, stay_alive=True, with_installed_binary=True, + image="clickhouse/clickhouse-server", tag="22.8.14.53", + ) + cluster.start() - node1 = cluster.instances["node1"] - expected = "" - global uuids - for i in range(3): - node1.query( + def create_table_and_upload_data(node, i): + node.query( f"CREATE TABLE data{i} (id Int32) ENGINE = MergeTree() ORDER BY id SETTINGS storage_policy = 'def', min_bytes_for_wide_part=1;" ) for _ in range(10): - node1.query( + node.query( f"INSERT INTO data{i} SELECT number FROM numbers(500000 * {i+1})" ) - expected = node1.query(f"SELECT * FROM data{i} ORDER BY id") + node.query(f"SELECT * FROM data{i} ORDER BY id") - metadata_path = node1.query( + metadata_path = node.query( f"SELECT data_paths FROM system.tables WHERE name='data{i}'" ) metadata_path = metadata_path[ @@ -45,7 +49,7 @@ def cluster(): ] print(f"Metadata: {metadata_path}") - node1.exec_in_container( + node.exec_in_container( [ "bash", "-c", @@ -56,8 +60,21 @@ def cluster(): user="root", ) parts = metadata_path.split("/") - uuids.append(parts[3]) print(f"UUID: {parts[3]}") + return parts[3] + + + node1 = cluster.instances["node1"] + + global uuids + for i in range(2): + uuid = create_table_and_upload_data(node1, i) + uuids.append(uuid) + + node4 = cluster.instances["node4"] + + uuid = create_table_and_upload_data(node4, 2) + uuids.append(uuid) yield cluster @@ -68,6 +85,7 @@ def cluster(): @pytest.mark.parametrize("node_name", ["node2"]) def test_usage(cluster, node_name): node1 = cluster.instances["node1"] + node4 = cluster.instances["node4"] node2 = cluster.instances[node_name] global uuids assert len(uuids) == 3 @@ -90,7 +108,11 @@ def test_usage(cluster, node_name): result = node2.query( "SELECT id FROM test{} WHERE id % 56 = 3 ORDER BY id".format(i) ) - assert result == node1.query( + node = node1 + if i == 2: + node = node4 + + assert result == node.query( "SELECT id FROM data{} WHERE id % 56 = 3 ORDER BY id".format(i) ) @@ -99,7 +121,7 @@ def test_usage(cluster, node_name): i ) ) - assert result == node1.query( + assert result == node.query( "SELECT id FROM data{} WHERE id > 789999 AND id < 999999 ORDER BY id".format( i ) @@ -141,6 +163,7 @@ def test_incorrect_usage(cluster): @pytest.mark.parametrize("node_name", ["node2"]) def test_cache(cluster, node_name): node1 = cluster.instances["node1"] + node4 = cluster.instances["node4"] node2 = cluster.instances[node_name] global uuids assert len(uuids) == 3 @@ -178,7 +201,12 @@ def test_cache(cluster, node_name): result = node2.query( "SELECT id FROM test{} WHERE id % 56 = 3 ORDER BY id".format(i) ) - assert result == node1.query( + + node = node1 + if i == 2: + node = node4 + + assert result == node.query( "SELECT id FROM data{} WHERE id % 56 = 3 ORDER BY id".format(i) ) @@ -187,7 +215,7 @@ def test_cache(cluster, node_name): i ) ) - assert result == node1.query( + assert result == node.query( "SELECT id FROM data{} WHERE id > 789999 AND id < 999999 ORDER BY id".format( i ) From 979c62af4fe4e7932c8fd05945ca5ee1987b9005 Mon Sep 17 00:00:00 2001 From: robot-clickhouse Date: Tue, 28 Feb 2023 11:24:39 +0000 Subject: [PATCH 005/103] Automatic style fix --- tests/integration/test_disk_over_web_server/test.py | 11 +++++++---- 1 file changed, 7 insertions(+), 4 deletions(-) diff --git a/tests/integration/test_disk_over_web_server/test.py b/tests/integration/test_disk_over_web_server/test.py index 2991ece05ec..fd71389f71a 100644 --- a/tests/integration/test_disk_over_web_server/test.py +++ b/tests/integration/test_disk_over_web_server/test.py @@ -23,9 +23,13 @@ def cluster(): ) cluster.add_instance( - "node4", main_configs=["configs/storage_conf.xml"], - with_nginx=True, stay_alive=True, with_installed_binary=True, - image="clickhouse/clickhouse-server", tag="22.8.14.53", + "node4", + main_configs=["configs/storage_conf.xml"], + with_nginx=True, + stay_alive=True, + with_installed_binary=True, + image="clickhouse/clickhouse-server", + tag="22.8.14.53", ) cluster.start() @@ -63,7 +67,6 @@ def cluster(): print(f"UUID: {parts[3]}") return parts[3] - node1 = cluster.instances["node1"] global uuids From 95c51d29971d99d6a291a94e7984c053384e192a Mon Sep 17 00:00:00 2001 From: alesapin Date: Thu, 2 Mar 2023 12:28:26 +0100 Subject: [PATCH 006/103] Add broken test --- .../02543_alter_update_rename_stuck.reference | 0 .../02543_alter_update_rename_stuck.sh | 70 +++++++++++++++++++ 2 files changed, 70 insertions(+) create mode 100644 tests/queries/0_stateless/02543_alter_update_rename_stuck.reference create mode 100755 tests/queries/0_stateless/02543_alter_update_rename_stuck.sh diff --git a/tests/queries/0_stateless/02543_alter_update_rename_stuck.reference b/tests/queries/0_stateless/02543_alter_update_rename_stuck.reference new file mode 100644 index 00000000000..e69de29bb2d diff --git a/tests/queries/0_stateless/02543_alter_update_rename_stuck.sh b/tests/queries/0_stateless/02543_alter_update_rename_stuck.sh new file mode 100755 index 00000000000..c1c17505fc8 --- /dev/null +++ b/tests/queries/0_stateless/02543_alter_update_rename_stuck.sh @@ -0,0 +1,70 @@ +#!/usr/bin/env bash + +CURDIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd) +# shellcheck source=../shell_config.sh +. "$CURDIR"/../shell_config.sh + + +$CLICKHOUSE_CLIENT --query="DROP TABLE IF EXISTS table_to_rename" + +$CLICKHOUSE_CLIENT --query="CREATE TABLE table_to_rename(v UInt64, v1 UInt64)ENGINE = MergeTree ORDER BY tuple() SETTINGS min_bytes_for_wide_part = 0" + +$CLICKHOUSE_CLIENT --query="INSERT INTO table_to_rename VALUES (1, 1)" + + +# we want to following mutations to stuck +# That is why we stop merges and wait in loops until they actually start +$CLICKHOUSE_CLIENT --query="SYSTEM STOP MERGES table_to_rename" + +$CLICKHOUSE_CLIENT --query="ALTER TABLE table_to_rename UPDATE v1 = 77 WHERE 1 = 1 SETTINGS mutations_sync = 2" & + +counter=0 retries=60 + +I=0 +while [[ $counter -lt $retries ]]; do + I=$((I + 1)) + result=$($CLICKHOUSE_CLIENT --query "SELECT count() from system.mutations where database='${CLICKHOUSE_DATABASE}' and table='table_to_rename'") + if [[ $result == "1" ]]; then + break; + fi + sleep 0.1 + ((++counter)) +done + +$CLICKHOUSE_CLIENT --query="ALTER TABLE table_to_rename RENAME COLUMN v1 to v2" & + +counter=0 retries=60 + +I=0 +while [[ $counter -lt $retries ]]; do + I=$((I + 1)) + result=$($CLICKHOUSE_CLIENT --query "show create table table_to_rename") + if [[ $result == *"v2"* ]]; then + break; + fi + sleep 0.1 + ((++counter)) +done + +counter=0 retries=60 + +I=0 +while [[ $counter -lt $retries ]]; do + I=$((I + 1)) + result=$($CLICKHOUSE_CLIENT --query "SELECT count() from system.mutations where database='${CLICKHOUSE_DATABASE}' and table='table_to_rename'") + if [[ $result == "2" ]]; then + break; + fi + sleep 0.1 + ((++counter)) +done + + +$CLICKHOUSE_CLIENT --query="SYSTEM START MERGES table_to_rename" + +wait + +$CLICKHOUSE_CLIENT --query="SELECT * FROM table_to_rename FORMAT JSONEachRow" + + + $CLICKHOUSE_CLIENT --query="DROP TABLE IF EXISTS table_to_rename" From db0f9b657844ff9c3fcfc9f3c888e4d0a164e2fc Mon Sep 17 00:00:00 2001 From: alesapin Date: Thu, 2 Mar 2023 14:26:07 +0100 Subject: [PATCH 007/103] Fix one more bug --- src/Storages/MergeTree/MutateTask.cpp | 2 ++ src/Storages/StorageMergeTree.cpp | 18 ++++++++++++ .../02543_alter_update_rename_stuck.reference | 1 + .../02543_alter_update_rename_stuck.sh | 28 ++----------------- 4 files changed, 24 insertions(+), 25 deletions(-) diff --git a/src/Storages/MergeTree/MutateTask.cpp b/src/Storages/MergeTree/MutateTask.cpp index 5f466862856..945cbe3c3a5 100644 --- a/src/Storages/MergeTree/MutateTask.cpp +++ b/src/Storages/MergeTree/MutateTask.cpp @@ -165,6 +165,8 @@ static void splitAndModifyMutationCommands( { if (command.type == MutationCommand::Type::READ_COLUMN) for_interpreter.push_back(command); + else if (command.type == MutationCommand::Type::RENAME_COLUMN) + part_columns.rename(command.column_name, command.rename_to); for_file_renames.push_back(command); } diff --git a/src/Storages/StorageMergeTree.cpp b/src/Storages/StorageMergeTree.cpp index 11922bebe7b..f25fb04172c 100644 --- a/src/Storages/StorageMergeTree.cpp +++ b/src/Storages/StorageMergeTree.cpp @@ -319,6 +319,24 @@ void StorageMergeTree::alter( } else { + if (!maybe_mutation_commands.empty() && maybe_mutation_commands.containBarrierCommand()) + { + int64_t prev_mutation = 0; + { + std::lock_guard lock(currently_processing_in_background_mutex); + auto it = current_mutations_by_version.rbegin(); + if (it != current_mutations_by_version.rend()) + prev_mutation = it->first; + } + + if (prev_mutation != 0) + { + LOG_DEBUG(log, "Cannot change metadata with barrier alter query, will wait for mutation {}", prev_mutation); + waitForMutation(prev_mutation); + LOG_DEBUG(log, "Mutation {} finished", prev_mutation); + } + } + { changeSettings(new_metadata.settings_changes, table_lock_holder); checkTTLExpressions(new_metadata, old_metadata); diff --git a/tests/queries/0_stateless/02543_alter_update_rename_stuck.reference b/tests/queries/0_stateless/02543_alter_update_rename_stuck.reference index e69de29bb2d..156128e3dd2 100644 --- a/tests/queries/0_stateless/02543_alter_update_rename_stuck.reference +++ b/tests/queries/0_stateless/02543_alter_update_rename_stuck.reference @@ -0,0 +1 @@ +{"v":"1","v2":"77"} diff --git a/tests/queries/0_stateless/02543_alter_update_rename_stuck.sh b/tests/queries/0_stateless/02543_alter_update_rename_stuck.sh index c1c17505fc8..e801fbedab7 100755 --- a/tests/queries/0_stateless/02543_alter_update_rename_stuck.sh +++ b/tests/queries/0_stateless/02543_alter_update_rename_stuck.sh @@ -33,32 +33,10 @@ done $CLICKHOUSE_CLIENT --query="ALTER TABLE table_to_rename RENAME COLUMN v1 to v2" & -counter=0 retries=60 - -I=0 -while [[ $counter -lt $retries ]]; do - I=$((I + 1)) - result=$($CLICKHOUSE_CLIENT --query "show create table table_to_rename") - if [[ $result == *"v2"* ]]; then - break; - fi - sleep 0.1 - ((++counter)) -done - -counter=0 retries=60 - -I=0 -while [[ $counter -lt $retries ]]; do - I=$((I + 1)) - result=$($CLICKHOUSE_CLIENT --query "SELECT count() from system.mutations where database='${CLICKHOUSE_DATABASE}' and table='table_to_rename'") - if [[ $result == "2" ]]; then - break; - fi - sleep 0.1 - ((++counter)) -done +# it will not introduce any flakyness +# just wait that mutation doesn't start +sleep 3 $CLICKHOUSE_CLIENT --query="SYSTEM START MERGES table_to_rename" From 63f615cdcd1d4a6edda29b3d746729293ab6713b Mon Sep 17 00:00:00 2001 From: alesapin Date: Thu, 2 Mar 2023 19:17:18 +0100 Subject: [PATCH 008/103] Fix metadata version --- src/Storages/MergeTree/ReplicatedMergeTreeQueue.cpp | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/src/Storages/MergeTree/ReplicatedMergeTreeQueue.cpp b/src/Storages/MergeTree/ReplicatedMergeTreeQueue.cpp index eea21d6349c..48e4e30aff2 100644 --- a/src/Storages/MergeTree/ReplicatedMergeTreeQueue.cpp +++ b/src/Storages/MergeTree/ReplicatedMergeTreeQueue.cpp @@ -1772,7 +1772,7 @@ std::map ReplicatedMergeTreeQueue::getAlterMutationCo int32_t alter_version = mutation_status->entry->alter_version; if (alter_version != -1) { - if (!alter_sequence.canExecuteDataAlter(alter_version, lock)) + if (alter_version > storage.getInMemoryMetadataPtr()->getMetadataVersion()) continue; /// we take commands with bigger metadata version From 83b68caccc395a2166263ba92fa2c62d57ced56f Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?Ra=C3=BAl=20Mar=C3=ADn?= Date: Fri, 24 Mar 2023 17:27:15 +0100 Subject: [PATCH 009/103] Do not continue retrying to connect to ZK if the query is killed or over limits --- src/Backups/BackupCoordinationRemote.cpp | 6 +++--- src/Storages/MergeTree/ReplicatedMergeTreeSink.cpp | 6 +++--- src/Storages/MergeTree/ZooKeeperRetries.h | 8 +++++++- 3 files changed, 13 insertions(+), 7 deletions(-) diff --git a/src/Backups/BackupCoordinationRemote.cpp b/src/Backups/BackupCoordinationRemote.cpp index 5ad95490c95..557b00aecbe 100644 --- a/src/Backups/BackupCoordinationRemote.cpp +++ b/src/Backups/BackupCoordinationRemote.cpp @@ -612,7 +612,7 @@ std::vector BackupCoordinationRemote::getAllFileInfos() const std::vector batched_escaped_names; { - ZooKeeperRetriesControl retries_ctl("getAllFileInfos::getChildren", zookeeper_retries_info); + ZooKeeperRetriesControl retries_ctl("getAllFileInfos::getChildren", zookeeper_retries_info, nullptr); retries_ctl.retryLoop([&]() { auto zk = getZooKeeper(); @@ -633,7 +633,7 @@ std::vector BackupCoordinationRemote::getAllFileInfos() const file_names_paths.emplace_back(zookeeper_path + "/file_names/" + escaped_name); - ZooKeeperRetriesControl retries_ctl("getAllFileInfos::getSizesAndChecksums", zookeeper_retries_info); + ZooKeeperRetriesControl retries_ctl("getAllFileInfos::getSizesAndChecksums", zookeeper_retries_info, nullptr); retries_ctl.retryLoop([&] { auto zk = getZooKeeper(); @@ -675,7 +675,7 @@ std::vector BackupCoordinationRemote::getAllFileInfos() const } zkutil::ZooKeeper::MultiGetResponse non_empty_file_infos_serialized; - ZooKeeperRetriesControl retries_ctl("getAllFileInfos::getFileInfos", zookeeper_retries_info); + ZooKeeperRetriesControl retries_ctl("getAllFileInfos::getFileInfos", zookeeper_retries_info, nullptr); retries_ctl.retryLoop([&]() { auto zk = getZooKeeper(); diff --git a/src/Storages/MergeTree/ReplicatedMergeTreeSink.cpp b/src/Storages/MergeTree/ReplicatedMergeTreeSink.cpp index 035cbdac55e..c71f72d000a 100644 --- a/src/Storages/MergeTree/ReplicatedMergeTreeSink.cpp +++ b/src/Storages/MergeTree/ReplicatedMergeTreeSink.cpp @@ -384,7 +384,7 @@ void ReplicatedMergeTreeSinkImpl::consume(Chunk chunk) * TODO Too complex logic, you can do better. */ size_t replicas_num = 0; - ZooKeeperRetriesControl quorum_retries_ctl("checkQuorumPrecondition", zookeeper_retries_info); + ZooKeeperRetriesControl quorum_retries_ctl("checkQuorumPrecondition", zookeeper_retries_info, context->getProcessListElement()); quorum_retries_ctl.retryLoop( [&]() { @@ -641,7 +641,7 @@ std::vector ReplicatedMergeTreeSinkImpl::commitPart( Coordination::Error write_part_info_keeper_error = Coordination::Error::ZOK; std::vector conflict_block_ids; - ZooKeeperRetriesControl retries_ctl("commitPart", zookeeper_retries_info); + ZooKeeperRetriesControl retries_ctl("commitPart", zookeeper_retries_info, context->getProcessListElement()); retries_ctl.retryLoop([&]() { zookeeper->setKeeper(storage.getZooKeeper()); @@ -1077,7 +1077,7 @@ std::vector ReplicatedMergeTreeSinkImpl::commitPart( return conflict_block_ids; if (isQuorumEnabled()) { - ZooKeeperRetriesControl quorum_retries_ctl("waitForQuorum", zookeeper_retries_info); + ZooKeeperRetriesControl quorum_retries_ctl("waitForQuorum", zookeeper_retries_info, context->getProcessListElement()); quorum_retries_ctl.retryLoop([&]() { zookeeper->setKeeper(storage.getZooKeeper()); diff --git a/src/Storages/MergeTree/ZooKeeperRetries.h b/src/Storages/MergeTree/ZooKeeperRetries.h index 1a4d394857f..a24f9f63693 100644 --- a/src/Storages/MergeTree/ZooKeeperRetries.h +++ b/src/Storages/MergeTree/ZooKeeperRetries.h @@ -1,4 +1,5 @@ #pragma once +#include #include #include #include @@ -35,7 +36,8 @@ struct ZooKeeperRetriesInfo class ZooKeeperRetriesControl { public: - ZooKeeperRetriesControl(std::string name_, ZooKeeperRetriesInfo & retries_info_) : name(std::move(name_)), retries_info(retries_info_) + ZooKeeperRetriesControl(std::string name_, ZooKeeperRetriesInfo & retries_info_, QueryStatusPtr elem) + : name(std::move(name_)), retries_info(retries_info_), process_list_element(elem) { } @@ -160,6 +162,9 @@ private: if (0 == iteration_count) return true; + if (process_list_element && !process_list_element->checkTimeLimitSoft()) + return false; + if (unconditional_retry) { unconditional_retry = false; @@ -260,6 +265,7 @@ private: bool unconditional_retry = false; bool iteration_succeeded = true; bool stop_retries = false; + QueryStatusPtr process_list_element; }; } From d2cb4eb9fb6c63861aa03dd2ded71e906b7941a2 Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?Ra=C3=BAl=20Mar=C3=ADn?= Date: Mon, 27 Mar 2023 16:40:15 +0200 Subject: [PATCH 010/103] Add integration test --- .../test_inserts_with_keeper_retries/test.py | 16 ++++++++++++++++ 1 file changed, 16 insertions(+) diff --git a/tests/integration/test_inserts_with_keeper_retries/test.py b/tests/integration/test_inserts_with_keeper_retries/test.py index dbf18365053..41acac629f5 100644 --- a/tests/integration/test_inserts_with_keeper_retries/test.py +++ b/tests/integration/test_inserts_with_keeper_retries/test.py @@ -98,3 +98,19 @@ def test_replica_inserts_with_keeper_disconnect(started_cluster): finally: node1.query("DROP TABLE IF EXISTS r SYNC") + + +def test_query_timeout_with_zk_down(started_cluster): + node1.query( + "CREATE TABLE zk_down (a UInt64, b String) ENGINE=ReplicatedMergeTree('/test/zk_down', '0') ORDER BY tuple()" + ) + + cluster.stop_zookeeper_nodes(["zoo1", "zoo2", "zoo3"]) + + start_time = time.time() + with pytest.raises(QueryRuntimeException): + node1.query( + "INSERT INTO zk_down SELECT number, toString(number) FROM numbers(10) SETTINGS insert_keeper_max_retries=10000, insert_keeper_retry_max_backoff_ms=1000, max_execution_time=1" + ) + finish_time = time.time() + assert finish_time - start_time < 10 From 920bb62ae15f12029397f96adacf1325734c2522 Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?Ra=C3=BAl=20Mar=C3=ADn?= Date: Tue, 28 Mar 2023 10:46:22 +0200 Subject: [PATCH 011/103] Fix integration tests --- .../test_inserts_with_keeper_retries/test.py | 26 +++++++++++-------- 1 file changed, 15 insertions(+), 11 deletions(-) diff --git a/tests/integration/test_inserts_with_keeper_retries/test.py b/tests/integration/test_inserts_with_keeper_retries/test.py index 41acac629f5..3937823a37b 100644 --- a/tests/integration/test_inserts_with_keeper_retries/test.py +++ b/tests/integration/test_inserts_with_keeper_retries/test.py @@ -101,16 +101,20 @@ def test_replica_inserts_with_keeper_disconnect(started_cluster): def test_query_timeout_with_zk_down(started_cluster): - node1.query( - "CREATE TABLE zk_down (a UInt64, b String) ENGINE=ReplicatedMergeTree('/test/zk_down', '0') ORDER BY tuple()" - ) - - cluster.stop_zookeeper_nodes(["zoo1", "zoo2", "zoo3"]) - - start_time = time.time() - with pytest.raises(QueryRuntimeException): + try: node1.query( - "INSERT INTO zk_down SELECT number, toString(number) FROM numbers(10) SETTINGS insert_keeper_max_retries=10000, insert_keeper_retry_max_backoff_ms=1000, max_execution_time=1" + "CREATE TABLE zk_down (a UInt64, b String) ENGINE=ReplicatedMergeTree('/test/zk_down', '0') ORDER BY tuple()" ) - finish_time = time.time() - assert finish_time - start_time < 10 + + cluster.stop_zookeeper_nodes(["zoo1", "zoo2", "zoo3"]) + + start_time = time.time() + with pytest.raises(QueryRuntimeException): + node1.query( + "INSERT INTO zk_down SELECT number, toString(number) FROM numbers(10) SETTINGS insert_keeper_max_retries=10000, insert_keeper_retry_max_backoff_ms=1000, max_execution_time=1" + ) + finish_time = time.time() + assert finish_time - start_time < 10 + finally: + cluster.start_zookeeper_nodes(["zoo1", "zoo2", "zoo3"]) + node1.query("DROP TABLE IF EXISTS zk_down SYNC") From 499b58dcf6142382bcfe583e754fba067cacb475 Mon Sep 17 00:00:00 2001 From: Alexander Tokmakov Date: Wed, 29 Mar 2023 17:03:57 +0200 Subject: [PATCH 012/103] add scripts for sparse checkout of some contribs --- .../sparse-checkout/setup-sparse-checkout.sh | 11 +++++++++ contrib/sparse-checkout/update-arrow.sh | 12 ++++++++++ contrib/sparse-checkout/update-aws.sh | 13 +++++++++++ contrib/sparse-checkout/update-boringssl.sh | 11 +++++++++ contrib/sparse-checkout/update-croaring.sh | 12 ++++++++++ contrib/sparse-checkout/update-grpc.sh | 20 ++++++++++++++++ contrib/sparse-checkout/update-h3.sh | 12 ++++++++++ .../sparse-checkout/update-llvm-project.sh | 23 +++++++++++++++++++ contrib/sparse-checkout/update-openssl.sh | 12 ++++++++++ contrib/sparse-checkout/update-orc.sh | 12 ++++++++++ 10 files changed, 138 insertions(+) create mode 100755 contrib/sparse-checkout/setup-sparse-checkout.sh create mode 100755 contrib/sparse-checkout/update-arrow.sh create mode 100755 contrib/sparse-checkout/update-aws.sh create mode 100755 contrib/sparse-checkout/update-boringssl.sh create mode 100755 contrib/sparse-checkout/update-croaring.sh create mode 100755 contrib/sparse-checkout/update-grpc.sh create mode 100755 contrib/sparse-checkout/update-h3.sh create mode 100755 contrib/sparse-checkout/update-llvm-project.sh create mode 100755 contrib/sparse-checkout/update-openssl.sh create mode 100755 contrib/sparse-checkout/update-orc.sh diff --git a/contrib/sparse-checkout/setup-sparse-checkout.sh b/contrib/sparse-checkout/setup-sparse-checkout.sh new file mode 100755 index 00000000000..ac039b964e9 --- /dev/null +++ b/contrib/sparse-checkout/setup-sparse-checkout.sh @@ -0,0 +1,11 @@ +#!/bin/sh + +git config submodule."contrib/llvm-project".update '!../sparse-checkout/update-llvm-project.sh' +git config submodule."contrib/croaring".update '!../sparse-checkout/update-croaring.sh' +git config submodule."contrib/aws".update '!../sparse-checkout/update-aws.sh' +git config submodule."contrib/openssl".update '!../sparse-checkout/update-openssl.sh' +git config submodule."contrib/boringssl".update '!../sparse-checkout/update-boringssl.sh' +git config submodule."contrib/arrow".update '!../sparse-checkout/update-arrow.sh' +git config submodule."contrib/grpc".update '!../sparse-checkout/update-grpc.sh' +git config submodule."contrib/orc".update '!../sparse-checkout/update-orc.sh' +git config submodule."contrib/h3".update '!../sparse-checkout/update-h3.sh' diff --git a/contrib/sparse-checkout/update-arrow.sh b/contrib/sparse-checkout/update-arrow.sh new file mode 100755 index 00000000000..e004b60da02 --- /dev/null +++ b/contrib/sparse-checkout/update-arrow.sh @@ -0,0 +1,12 @@ +#!/bin/sh + +echo "Using sparse checkout for arrow" + +FILES_TO_CHECKOUT=$(git rev-parse --git-dir)/info/sparse-checkout +echo '/*' > $FILES_TO_CHECKOUT +echo '!/*/*' >> $FILES_TO_CHECKOUT +echo '/cpp/*' >> $FILES_TO_CHECKOUT + +git config core.sparsecheckout true +git checkout $1 +git read-tree -mu HEAD diff --git a/contrib/sparse-checkout/update-aws.sh b/contrib/sparse-checkout/update-aws.sh new file mode 100755 index 00000000000..c8d4c5a89c2 --- /dev/null +++ b/contrib/sparse-checkout/update-aws.sh @@ -0,0 +1,13 @@ +#!/bin/sh + +echo "Using sparse checkout for aws" + +FILES_TO_CHECKOUT=$(git rev-parse --git-dir)/info/sparse-checkout +echo '/*' > $FILES_TO_CHECKOUT +echo '!/*/*' >> $FILES_TO_CHECKOUT +echo '/aws-cpp-sdk-core/*' >> $FILES_TO_CHECKOUT +echo '/aws-cpp-sdk-s3/*' >> $FILES_TO_CHECKOUT + +git config core.sparsecheckout true +git checkout $1 +git read-tree -mu HEAD diff --git a/contrib/sparse-checkout/update-boringssl.sh b/contrib/sparse-checkout/update-boringssl.sh new file mode 100755 index 00000000000..dee19e3ca97 --- /dev/null +++ b/contrib/sparse-checkout/update-boringssl.sh @@ -0,0 +1,11 @@ +#!/bin/sh + +echo "Using sparse checkout for boringsll" + +FILES_TO_CHECKOUT=$(git rev-parse --git-dir)/info/sparse-checkout +echo '/*' > $FILES_TO_CHECKOUT +echo '!/fuzz/*' >> $FILES_TO_CHECKOUT + +git config core.sparsecheckout true +git checkout $1 +git read-tree -mu HEAD diff --git a/contrib/sparse-checkout/update-croaring.sh b/contrib/sparse-checkout/update-croaring.sh new file mode 100755 index 00000000000..9b7bba19df4 --- /dev/null +++ b/contrib/sparse-checkout/update-croaring.sh @@ -0,0 +1,12 @@ +#!/bin/sh + +echo "Using sparse checkout for croaring" + +FILES_TO_CHECKOUT=$(git rev-parse --git-dir)/info/sparse-checkout +echo '/*' > $FILES_TO_CHECKOUT +echo '!/benchmarks/*' >> $FILES_TO_CHECKOUT +echo '!/tests/*' >> $FILES_TO_CHECKOUT + +git config core.sparsecheckout true +git checkout $1 +git read-tree -mu HEAD diff --git a/contrib/sparse-checkout/update-grpc.sh b/contrib/sparse-checkout/update-grpc.sh new file mode 100755 index 00000000000..18045e6ec90 --- /dev/null +++ b/contrib/sparse-checkout/update-grpc.sh @@ -0,0 +1,20 @@ +#!/bin/sh + +echo "Using sparse checkout for grpc" + +FILES_TO_CHECKOUT=$(git rev-parse --git-dir)/info/sparse-checkout +echo '/*' > $FILES_TO_CHECKOUT +# FIXME for some reason we need it for build +# echo '!/test/*' >> $FILES_TO_CHECKOUT +# echo '!/tools/*' >> $FILES_TO_CHECKOUT +echo '!/examples/*' >> $FILES_TO_CHECKOUT +echo '!/doc/*' >> $FILES_TO_CHECKOUT +# echo '!/src/csharp/*' >> $FILES_TO_CHECKOUT +echo '!/src/python/*' >> $FILES_TO_CHECKOUT +echo '!/src/objective-c/*' >> $FILES_TO_CHECKOUT +echo '!/src/php/*' >> $FILES_TO_CHECKOUT +echo '!/src/ruby/*' >> $FILES_TO_CHECKOUT + +git config core.sparsecheckout true +git checkout $1 +git read-tree -mu HEAD diff --git a/contrib/sparse-checkout/update-h3.sh b/contrib/sparse-checkout/update-h3.sh new file mode 100755 index 00000000000..127885f89cc --- /dev/null +++ b/contrib/sparse-checkout/update-h3.sh @@ -0,0 +1,12 @@ +#!/bin/sh + +echo "Using sparse checkout for h3" + +FILES_TO_CHECKOUT=$(git rev-parse --git-dir)/info/sparse-checkout +echo '/*' > $FILES_TO_CHECKOUT +echo '!/tests/*' >> $FILES_TO_CHECKOUT +echo '!/website/*' >> $FILES_TO_CHECKOUT + +git config core.sparsecheckout true +git checkout $1 +git read-tree -mu HEAD diff --git a/contrib/sparse-checkout/update-llvm-project.sh b/contrib/sparse-checkout/update-llvm-project.sh new file mode 100755 index 00000000000..43c20bd8bfe --- /dev/null +++ b/contrib/sparse-checkout/update-llvm-project.sh @@ -0,0 +1,23 @@ +#!/bin/sh + +echo "Using sparse checkout for llvm-project" + +FILES_TO_CHECKOUT=$(git rev-parse --git-dir)/info/sparse-checkout +echo '/*' > $FILES_TO_CHECKOUT +echo '!/llvm/test/*' >> $FILES_TO_CHECKOUT +echo '!/llvm/docs/*' >> $FILES_TO_CHECKOUT +echo '!/llvm/unittests/*' >> $FILES_TO_CHECKOUT +echo '!/llvm/tools/*' >> $FILES_TO_CHECKOUT +echo '!/clang/*' >> $FILES_TO_CHECKOUT +echo '!/clang-tools-extra/*' >> $FILES_TO_CHECKOUT +echo '!/lldb/*' >> $FILES_TO_CHECKOUT +echo '!/mlir/*' >> $FILES_TO_CHECKOUT +echo '!/polly/*' >> $FILES_TO_CHECKOUT +echo '!/lld/*' >> $FILES_TO_CHECKOUT +echo '!/flang/*' >> $FILES_TO_CHECKOUT +echo '!/libcxx/test/*' >> $FILES_TO_CHECKOUT +echo '!/compiler-rt/test/*' >> $FILES_TO_CHECKOUT + +git config core.sparsecheckout true +git checkout $1 +git read-tree -mu HEAD diff --git a/contrib/sparse-checkout/update-openssl.sh b/contrib/sparse-checkout/update-openssl.sh new file mode 100755 index 00000000000..5528f0e70ba --- /dev/null +++ b/contrib/sparse-checkout/update-openssl.sh @@ -0,0 +1,12 @@ +#!/bin/sh + +echo "Using sparse checkout for opensll" + +FILES_TO_CHECKOUT=$(git rev-parse --git-dir)/info/sparse-checkout +echo '/*' > $FILES_TO_CHECKOUT +echo '!/fuzz/*' >> $FILES_TO_CHECKOUT +echo '!/test/*' >> $FILES_TO_CHECKOUT + +git config core.sparsecheckout true +git checkout $1 +git read-tree -mu HEAD diff --git a/contrib/sparse-checkout/update-orc.sh b/contrib/sparse-checkout/update-orc.sh new file mode 100755 index 00000000000..963db8f306c --- /dev/null +++ b/contrib/sparse-checkout/update-orc.sh @@ -0,0 +1,12 @@ +#!/bin/sh + +echo "Using sparse checkout for orc" + +FILES_TO_CHECKOUT=$(git rev-parse --git-dir)/info/sparse-checkout +echo '/*' > $FILES_TO_CHECKOUT +echo '!/*/*' >> $FILES_TO_CHECKOUT +echo '/c++/*' >> $FILES_TO_CHECKOUT + +git config core.sparsecheckout true +git checkout $1 +git read-tree -mu HEAD From 905975f6e6ae7dbae5683b4f4891fd40c08dffae Mon Sep 17 00:00:00 2001 From: Alexander Tokmakov Date: Wed, 29 Mar 2023 18:33:15 +0200 Subject: [PATCH 013/103] update docs --- contrib/sparse-checkout/setup-sparse-checkout.sh | 2 ++ contrib/update-submodules.sh | 11 +++++++++++ docs/en/development/developer-instruction.md | 6 ++++++ docs/ru/development/developer-instruction.md | 8 +++++++- 4 files changed, 26 insertions(+), 1 deletion(-) create mode 100755 contrib/update-submodules.sh diff --git a/contrib/sparse-checkout/setup-sparse-checkout.sh b/contrib/sparse-checkout/setup-sparse-checkout.sh index ac039b964e9..f646e16412f 100755 --- a/contrib/sparse-checkout/setup-sparse-checkout.sh +++ b/contrib/sparse-checkout/setup-sparse-checkout.sh @@ -1,5 +1,7 @@ #!/bin/sh +set -e + git config submodule."contrib/llvm-project".update '!../sparse-checkout/update-llvm-project.sh' git config submodule."contrib/croaring".update '!../sparse-checkout/update-croaring.sh' git config submodule."contrib/aws".update '!../sparse-checkout/update-aws.sh' diff --git a/contrib/update-submodules.sh b/contrib/update-submodules.sh new file mode 100755 index 00000000000..c94681e6240 --- /dev/null +++ b/contrib/update-submodules.sh @@ -0,0 +1,11 @@ +#!/bin/sh + +set -e + +WORKDIR=$(dirname "$0") +WORKDIR=$(readlink -f "${WORKDIR}") + +"$WORKDIR/sparse-checkout/setup-sparse-checkout.sh" +git submodule init +git submodule sync +git submodule update --depth=1 diff --git a/docs/en/development/developer-instruction.md b/docs/en/development/developer-instruction.md index ace5ab79bb4..ea98b2da5e6 100644 --- a/docs/en/development/developer-instruction.md +++ b/docs/en/development/developer-instruction.md @@ -42,6 +42,12 @@ In the command line terminal run: git clone --recursive --shallow-submodules git@github.com:your_github_username/ClickHouse.git cd ClickHouse +Or (if you'd like to use sparse checkout for submodules and avoid checking out unneeded files): + + git clone git@github.com:your_github_username/ClickHouse.git + cd ClickHouse + ./contrib/update-submodules.sh + Note: please, substitute *your_github_username* with what is appropriate! This command will create a directory `ClickHouse` containing the working copy of the project. diff --git a/docs/ru/development/developer-instruction.md b/docs/ru/development/developer-instruction.md index 80472178ae2..c208439678a 100644 --- a/docs/ru/development/developer-instruction.md +++ b/docs/ru/development/developer-instruction.md @@ -41,9 +41,15 @@ ClickHouse не работает и не собирается на 32-битны Выполните в терминале: - git clone git@github.com:your_github_username/ClickHouse.git --recursive + git clone --recursive --shallow-submodules git@github.com:your_github_username/ClickHouse.git cd ClickHouse +Или (если вы хотите использовать sparse checkout для submodules): + + git clone git@github.com:your_github_username/ClickHouse.git + cd ClickHouse + ./contrib/update-submodules.sh + Замените слово `your_github_username` в команде для git на имя вашего аккаунта на GitHub. Эта команда создаст директорию ClickHouse, содержащую рабочую копию проекта. From ee3da3854a64e6d7074634a3c0403d998e630f86 Mon Sep 17 00:00:00 2001 From: Alexander Tokmakov Date: Fri, 31 Mar 2023 17:47:23 +0200 Subject: [PATCH 014/103] check sparse checkout in CI --- .github/workflows/pull_request.yml | 5 +++++ 1 file changed, 5 insertions(+) diff --git a/.github/workflows/pull_request.yml b/.github/workflows/pull_request.yml index ab0cbbb7ec1..291c0fac05a 100644 --- a/.github/workflows/pull_request.yml +++ b/.github/workflows/pull_request.yml @@ -550,6 +550,11 @@ jobs: with: clear-repository: true submodules: true + - name: Apply sparse checkout for contrib # in order to check that it doesn't break build + run: | + rm -rf "$GITHUB_WORKSPACE/contrib" + git -c "$GITHUB_WORKSPACE" checkout . + "$GITHUB_WORKSPACE/contrib/update-submodules.sh" - name: Build run: | sudo rm -fr "$TEMP_PATH" From 8169f01a24e34c5e749cf699a9748564481774f5 Mon Sep 17 00:00:00 2001 From: Alexander Tokmakov Date: Fri, 31 Mar 2023 20:58:57 +0200 Subject: [PATCH 015/103] exclude more subdirs --- .../sparse-checkout/setup-sparse-checkout.sh | 6 ++ contrib/sparse-checkout/update-aws-s2n-tls.sh | 13 +++ contrib/sparse-checkout/update-boost.sh | 85 +++++++++++++++++++ contrib/sparse-checkout/update-boringssl.sh | 3 + contrib/sparse-checkout/update-brotli.sh | 12 +++ contrib/sparse-checkout/update-grpc.sh | 10 ++- contrib/sparse-checkout/update-icu.sh | 12 +++ contrib/sparse-checkout/update-libxml2.sh | 16 ++++ .../sparse-checkout/update-llvm-project.sh | 26 +++--- contrib/sparse-checkout/update-openssl.sh | 5 +- contrib/sparse-checkout/update-protobuf.sh | 13 +++ 11 files changed, 185 insertions(+), 16 deletions(-) create mode 100755 contrib/sparse-checkout/update-aws-s2n-tls.sh create mode 100755 contrib/sparse-checkout/update-boost.sh create mode 100755 contrib/sparse-checkout/update-brotli.sh create mode 100755 contrib/sparse-checkout/update-icu.sh create mode 100755 contrib/sparse-checkout/update-libxml2.sh create mode 100755 contrib/sparse-checkout/update-protobuf.sh diff --git a/contrib/sparse-checkout/setup-sparse-checkout.sh b/contrib/sparse-checkout/setup-sparse-checkout.sh index f646e16412f..3feba6c5adf 100755 --- a/contrib/sparse-checkout/setup-sparse-checkout.sh +++ b/contrib/sparse-checkout/setup-sparse-checkout.sh @@ -11,3 +11,9 @@ git config submodule."contrib/arrow".update '!../sparse-checkout/update-arrow.sh git config submodule."contrib/grpc".update '!../sparse-checkout/update-grpc.sh' git config submodule."contrib/orc".update '!../sparse-checkout/update-orc.sh' git config submodule."contrib/h3".update '!../sparse-checkout/update-h3.sh' +git config submodule."contrib/icu".update '!../sparse-checkout/update-icu.sh' +git config submodule."contrib/boost".update '!../sparse-checkout/update-boost.sh' +git config submodule."contrib/aws-s2n-tls".update '!../sparse-checkout/update-aws-s2n-tls.sh' +git config submodule."contrib/protobuf".update '!../sparse-checkout/update-protobuf.sh' +git config submodule."contrib/libxml2".update '!../sparse-checkout/update-libxml2.sh' +git config submodule."contrib/brotli".update '!../sparse-checkout/update-brotli.sh' diff --git a/contrib/sparse-checkout/update-aws-s2n-tls.sh b/contrib/sparse-checkout/update-aws-s2n-tls.sh new file mode 100755 index 00000000000..4d65dc4b81d --- /dev/null +++ b/contrib/sparse-checkout/update-aws-s2n-tls.sh @@ -0,0 +1,13 @@ +#!/bin/sh + +echo "Using sparse checkout for aws-s2n-tls" + +FILES_TO_CHECKOUT=$(git rev-parse --git-dir)/info/sparse-checkout +echo '/*' > $FILES_TO_CHECKOUT +echo '!/test/*' >> $FILES_TO_CHECKOUT +echo '!/docs/*' >> $FILES_TO_CHECKOUT +echo '!/compliance/*' >> $FILES_TO_CHECKOUT + +git config core.sparsecheckout true +git checkout $1 +git read-tree -mu HEAD diff --git a/contrib/sparse-checkout/update-boost.sh b/contrib/sparse-checkout/update-boost.sh new file mode 100755 index 00000000000..9bd1f6c1796 --- /dev/null +++ b/contrib/sparse-checkout/update-boost.sh @@ -0,0 +1,85 @@ +#!/bin/sh + +echo "Using sparse checkout for boost" + +FILES_TO_CHECKOUT=$(git rev-parse --git-dir)/info/sparse-checkout +echo '/*' > $FILES_TO_CHECKOUT +echo '!/*/*' >> $FILES_TO_CHECKOUT +echo '/boost/*' > $FILES_TO_CHECKOUT +echo '!/boost/*/*' >> $FILES_TO_CHECKOUT +echo '/boost/algorithm/*' >> $FILES_TO_CHECKOUT +echo '/boost/any/*' >> $FILES_TO_CHECKOUT +echo '/boost/atomic/*' >> $FILES_TO_CHECKOUT +echo '/boost/assert/*' >> $FILES_TO_CHECKOUT +echo '/boost/bind/*' >> $FILES_TO_CHECKOUT +echo '/boost/concept/*' >> $FILES_TO_CHECKOUT +echo '/boost/config/*' >> $FILES_TO_CHECKOUT +echo '/boost/container/*' >> $FILES_TO_CHECKOUT +echo '/boost/container_hash/*' >> $FILES_TO_CHECKOUT +echo '/boost/context/*' >> $FILES_TO_CHECKOUT +echo '/boost/convert/*' >> $FILES_TO_CHECKOUT +echo '/boost/coroutine/*' >> $FILES_TO_CHECKOUT +echo '/boost/core/*' >> $FILES_TO_CHECKOUT +echo '/boost/detail/*' >> $FILES_TO_CHECKOUT +echo '/boost/dynamic_bitset/*' >> $FILES_TO_CHECKOUT +echo '/boost/exception/*' >> $FILES_TO_CHECKOUT +echo '/boost/filesystem/*' >> $FILES_TO_CHECKOUT +echo '/boost/functional/*' >> $FILES_TO_CHECKOUT +echo '/boost/function/*' >> $FILES_TO_CHECKOUT +echo '/boost/geometry/*' >> $FILES_TO_CHECKOUT +echo '/boost/graph/*' >> $FILES_TO_CHECKOUT +echo '/boost/heap/*' >> $FILES_TO_CHECKOUT +echo '/boost/integer/*' >> $FILES_TO_CHECKOUT +echo '/boost/intrusive/*' >> $FILES_TO_CHECKOUT +echo '/boost/iostreams/*' >> $FILES_TO_CHECKOUT +echo '/boost/io/*' >> $FILES_TO_CHECKOUT +echo '/boost/iterator/*' >> $FILES_TO_CHECKOUT +echo '/boost/math/*' >> $FILES_TO_CHECKOUT +echo '/boost/move/*' >> $FILES_TO_CHECKOUT +echo '/boost/mpl/*' >> $FILES_TO_CHECKOUT +echo '/boost/multi_index/*' >> $FILES_TO_CHECKOUT +echo '/boost/multiprecision/*' >> $FILES_TO_CHECKOUT +echo '/boost/numeric/*' >> $FILES_TO_CHECKOUT +echo '/boost/predef/*' >> $FILES_TO_CHECKOUT +echo '/boost/preprocessor/*' >> $FILES_TO_CHECKOUT +echo '/boost/program_options/*' >> $FILES_TO_CHECKOUT +echo '/boost/range/*' >> $FILES_TO_CHECKOUT +echo '/boost/regex/*' >> $FILES_TO_CHECKOUT +echo '/boost/smart_ptr/*' >> $FILES_TO_CHECKOUT +echo '/boost/type_index/*' >> $FILES_TO_CHECKOUT +echo '/boost/type_traits/*' >> $FILES_TO_CHECKOUT +echo '/boost/system/*' >> $FILES_TO_CHECKOUT +echo '/boost/tti/*' >> $FILES_TO_CHECKOUT +echo '/boost/utility/*' >> $FILES_TO_CHECKOUT +echo '/boost/lexical_cast/*' >> $FILES_TO_CHECKOUT +echo '/boost/optional/*' >> $FILES_TO_CHECKOUT +echo '/boost/property_map/*' >> $FILES_TO_CHECKOUT +echo '/boost/pending/*' >> $FILES_TO_CHECKOUT +echo '/boost/multi_array/*' >> $FILES_TO_CHECKOUT +echo '/boost/tuple/*' >> $FILES_TO_CHECKOUT +echo '/boost/icl/*' >> $FILES_TO_CHECKOUT +echo '/boost/unordered/*' >> $FILES_TO_CHECKOUT +echo '/boost/typeof/*' >> $FILES_TO_CHECKOUT +echo '/boost/parameter/*' >> $FILES_TO_CHECKOUT +echo '/boost/mp11/*' >> $FILES_TO_CHECKOUT +echo '/boost/archive/*' >> $FILES_TO_CHECKOUT +echo '/boost/function_types/*' >> $FILES_TO_CHECKOUT +echo '/boost/serialization/*' >> $FILES_TO_CHECKOUT +echo '/boost/fusion/*' >> $FILES_TO_CHECKOUT +echo '/boost/variant/*' >> $FILES_TO_CHECKOUT +echo '/boost/format/*' >> $FILES_TO_CHECKOUT +echo '/boost/locale/*' >> $FILES_TO_CHECKOUT +echo '/boost/random/*' >> $FILES_TO_CHECKOUT +echo '/boost/spirit/*' >> $FILES_TO_CHECKOUT +echo '/boost/uuid/*' >> $FILES_TO_CHECKOUT +echo '/boost/xpressive/*' >> $FILES_TO_CHECKOUT +echo '/boost/asio/*' >> $FILES_TO_CHECKOUT +echo '/boost/circular_buffer/*' >> $FILES_TO_CHECKOUT +echo '/boost/proto/*' >> $FILES_TO_CHECKOUT +echo '/boost/qvm/*' >> $FILES_TO_CHECKOUT +echo '/boost/property_tree/*' >> $FILES_TO_CHECKOUT +echo '/libs/*' >> $FILES_TO_CHECKOUT + +git config core.sparsecheckout true +git checkout $1 +git read-tree -mu HEAD \ No newline at end of file diff --git a/contrib/sparse-checkout/update-boringssl.sh b/contrib/sparse-checkout/update-boringssl.sh index dee19e3ca97..f877a78afed 100755 --- a/contrib/sparse-checkout/update-boringssl.sh +++ b/contrib/sparse-checkout/update-boringssl.sh @@ -5,6 +5,9 @@ echo "Using sparse checkout for boringsll" FILES_TO_CHECKOUT=$(git rev-parse --git-dir)/info/sparse-checkout echo '/*' > $FILES_TO_CHECKOUT echo '!/fuzz/*' >> $FILES_TO_CHECKOUT +echo '!/crypto/cipher_extra/test/*' >> $FILES_TO_CHECKOUT +echo '!/third_party/wycheproof_testvectors/*' >> $FILES_TO_CHECKOUT +echo '!/third_party/googletest/*' >> $FILES_TO_CHECKOUT git config core.sparsecheckout true git checkout $1 diff --git a/contrib/sparse-checkout/update-brotli.sh b/contrib/sparse-checkout/update-brotli.sh new file mode 100755 index 00000000000..8784f5e4125 --- /dev/null +++ b/contrib/sparse-checkout/update-brotli.sh @@ -0,0 +1,12 @@ +#!/bin/sh + +echo "Using sparse checkout for brotli" + +FILES_TO_CHECKOUT=$(git rev-parse --git-dir)/info/sparse-checkout +echo '/*' > $FILES_TO_CHECKOUT +echo '!/*/*' >> $FILES_TO_CHECKOUT +echo '/c/*' >> $FILES_TO_CHECKOUT + +git config core.sparsecheckout true +git checkout $1 +git read-tree -mu HEAD diff --git a/contrib/sparse-checkout/update-grpc.sh b/contrib/sparse-checkout/update-grpc.sh index 18045e6ec90..38934fdbc1b 100755 --- a/contrib/sparse-checkout/update-grpc.sh +++ b/contrib/sparse-checkout/update-grpc.sh @@ -4,12 +4,14 @@ echo "Using sparse checkout for grpc" FILES_TO_CHECKOUT=$(git rev-parse --git-dir)/info/sparse-checkout echo '/*' > $FILES_TO_CHECKOUT -# FIXME for some reason we need it for build -# echo '!/test/*' >> $FILES_TO_CHECKOUT -# echo '!/tools/*' >> $FILES_TO_CHECKOUT +echo '!/test/*' >> $FILES_TO_CHECKOUT +echo '/test/build/*' >> $FILES_TO_CHECKOUT +echo '!/tools/*' >> $FILES_TO_CHECKOUT +echo '/tools/codegen/*' >> $FILES_TO_CHECKOUT echo '!/examples/*' >> $FILES_TO_CHECKOUT echo '!/doc/*' >> $FILES_TO_CHECKOUT -# echo '!/src/csharp/*' >> $FILES_TO_CHECKOUT +# FIXME why do we need csharp? +#echo '!/src/csharp/*' >> $FILES_TO_CHECKOUT echo '!/src/python/*' >> $FILES_TO_CHECKOUT echo '!/src/objective-c/*' >> $FILES_TO_CHECKOUT echo '!/src/php/*' >> $FILES_TO_CHECKOUT diff --git a/contrib/sparse-checkout/update-icu.sh b/contrib/sparse-checkout/update-icu.sh new file mode 100755 index 00000000000..76af39f07a4 --- /dev/null +++ b/contrib/sparse-checkout/update-icu.sh @@ -0,0 +1,12 @@ +#!/bin/sh + +echo "Using sparse checkout for icu" + +FILES_TO_CHECKOUT=$(git rev-parse --git-dir)/info/sparse-checkout +echo '/*' > $FILES_TO_CHECKOUT +echo '!/*/*' >> $FILES_TO_CHECKOUT +echo '/icu4c/*' >> $FILES_TO_CHECKOUT + +git config core.sparsecheckout true +git checkout $1 +git read-tree -mu HEAD \ No newline at end of file diff --git a/contrib/sparse-checkout/update-libxml2.sh b/contrib/sparse-checkout/update-libxml2.sh new file mode 100755 index 00000000000..24faf11eec9 --- /dev/null +++ b/contrib/sparse-checkout/update-libxml2.sh @@ -0,0 +1,16 @@ +#!/bin/sh + +echo "Using sparse checkout for libxml2" + +FILES_TO_CHECKOUT=$(git rev-parse --git-dir)/info/sparse-checkout +echo '/*' > $FILES_TO_CHECKOUT +echo '!/result/*' >> $FILES_TO_CHECKOUT +echo '!/test/*' >> $FILES_TO_CHECKOUT +echo '!/doc/*' >> $FILES_TO_CHECKOUT +echo '!/os400/*' >> $FILES_TO_CHECKOUT +echo '!/fuzz/*' >> $FILES_TO_CHECKOUT +echo '!/python/*' >> $FILES_TO_CHECKOUT + +git config core.sparsecheckout true +git checkout $1 +git read-tree -mu HEAD diff --git a/contrib/sparse-checkout/update-llvm-project.sh b/contrib/sparse-checkout/update-llvm-project.sh index 43c20bd8bfe..53c3b691d3a 100755 --- a/contrib/sparse-checkout/update-llvm-project.sh +++ b/contrib/sparse-checkout/update-llvm-project.sh @@ -4,19 +4,23 @@ echo "Using sparse checkout for llvm-project" FILES_TO_CHECKOUT=$(git rev-parse --git-dir)/info/sparse-checkout echo '/*' > $FILES_TO_CHECKOUT -echo '!/llvm/test/*' >> $FILES_TO_CHECKOUT -echo '!/llvm/docs/*' >> $FILES_TO_CHECKOUT -echo '!/llvm/unittests/*' >> $FILES_TO_CHECKOUT -echo '!/llvm/tools/*' >> $FILES_TO_CHECKOUT -echo '!/clang/*' >> $FILES_TO_CHECKOUT -echo '!/clang-tools-extra/*' >> $FILES_TO_CHECKOUT -echo '!/lldb/*' >> $FILES_TO_CHECKOUT -echo '!/mlir/*' >> $FILES_TO_CHECKOUT -echo '!/polly/*' >> $FILES_TO_CHECKOUT -echo '!/lld/*' >> $FILES_TO_CHECKOUT -echo '!/flang/*' >> $FILES_TO_CHECKOUT +echo '!/*/*' >> $FILES_TO_CHECKOUT +echo '/llvm/*' >> $FILES_TO_CHECKOUT +echo '!/llvm/*/*' >> $FILES_TO_CHECKOUT +echo '/llvm/cmake/*' >> $FILES_TO_CHECKOUT +echo '/llvm/projects/*' >> $FILES_TO_CHECKOUT +echo '/llvm/include/*' >> $FILES_TO_CHECKOUT +echo '/llvm/lib/*' >> $FILES_TO_CHECKOUT +echo '/llvm/utils/TableGen/*' >> $FILES_TO_CHECKOUT +echo '/libcxxabi/*' >> $FILES_TO_CHECKOUT +echo '!/libcxxabi/test/*' >> $FILES_TO_CHECKOUT +echo '/libcxx/*' >> $FILES_TO_CHECKOUT echo '!/libcxx/test/*' >> $FILES_TO_CHECKOUT +echo '/libunwind/*' >> $FILES_TO_CHECKOUT +echo '!/libunwind/test/*' >> $FILES_TO_CHECKOUT +echo '/compiler-rt/*' >> $FILES_TO_CHECKOUT echo '!/compiler-rt/test/*' >> $FILES_TO_CHECKOUT +echo '/cmake/*' >> $FILES_TO_CHECKOUT git config core.sparsecheckout true git checkout $1 diff --git a/contrib/sparse-checkout/update-openssl.sh b/contrib/sparse-checkout/update-openssl.sh index 5528f0e70ba..33e19f43cb7 100755 --- a/contrib/sparse-checkout/update-openssl.sh +++ b/contrib/sparse-checkout/update-openssl.sh @@ -1,11 +1,14 @@ #!/bin/sh -echo "Using sparse checkout for opensll" +echo "Using sparse checkout for openssl" FILES_TO_CHECKOUT=$(git rev-parse --git-dir)/info/sparse-checkout echo '/*' > $FILES_TO_CHECKOUT echo '!/fuzz/*' >> $FILES_TO_CHECKOUT echo '!/test/*' >> $FILES_TO_CHECKOUT +echo '!/doc/*' >> $FILES_TO_CHECKOUT +echo '!/providers/*' >> $FILES_TO_CHECKOUT +echo '!/apps/*' >> $FILES_TO_CHECKOUT git config core.sparsecheckout true git checkout $1 diff --git a/contrib/sparse-checkout/update-protobuf.sh b/contrib/sparse-checkout/update-protobuf.sh new file mode 100755 index 00000000000..31c037c2cf5 --- /dev/null +++ b/contrib/sparse-checkout/update-protobuf.sh @@ -0,0 +1,13 @@ +#!/bin/sh + +echo "Using sparse checkout for protobuf" + +FILES_TO_CHECKOUT=$(git rev-parse --git-dir)/info/sparse-checkout +echo '!/*' > $FILES_TO_CHECKOUT +echo '/*/*' >> $FILES_TO_CHECKOUT +echo '/src/*' >> $FILES_TO_CHECKOUT +echo '/cmake/*' >> $FILES_TO_CHECKOUT + +git config core.sparsecheckout true +git checkout $1 +git read-tree -mu HEAD From 409c8e3f547e208563dbb824c080ed226c2eab4f Mon Sep 17 00:00:00 2001 From: Alexander Tokmakov Date: Fri, 31 Mar 2023 22:35:28 +0200 Subject: [PATCH 016/103] fix --- .github/workflows/pull_request.yml | 6 +++--- 1 file changed, 3 insertions(+), 3 deletions(-) diff --git a/.github/workflows/pull_request.yml b/.github/workflows/pull_request.yml index 291c0fac05a..84a0d6ef0eb 100644 --- a/.github/workflows/pull_request.yml +++ b/.github/workflows/pull_request.yml @@ -552,9 +552,9 @@ jobs: submodules: true - name: Apply sparse checkout for contrib # in order to check that it doesn't break build run: | - rm -rf "$GITHUB_WORKSPACE/contrib" - git -c "$GITHUB_WORKSPACE" checkout . - "$GITHUB_WORKSPACE/contrib/update-submodules.sh" + rm -rf "$GITHUB_WORKSPACE/contrib" && echo 'removed' + git -C "$GITHUB_WORKSPACE" checkout . && echo 'restored' + "$GITHUB_WORKSPACE/contrib/update-submodules.sh" && echo 'OK' - name: Build run: | sudo rm -fr "$TEMP_PATH" From 043729e1ebb0cc6e61aa9437b815526515c347a9 Mon Sep 17 00:00:00 2001 From: Alexander Tokmakov Date: Sat, 1 Apr 2023 02:18:39 +0300 Subject: [PATCH 017/103] Update pull_request.yml --- .github/workflows/pull_request.yml | 2 ++ 1 file changed, 2 insertions(+) diff --git a/.github/workflows/pull_request.yml b/.github/workflows/pull_request.yml index 84a0d6ef0eb..6fccc0542b7 100644 --- a/.github/workflows/pull_request.yml +++ b/.github/workflows/pull_request.yml @@ -555,6 +555,8 @@ jobs: rm -rf "$GITHUB_WORKSPACE/contrib" && echo 'removed' git -C "$GITHUB_WORKSPACE" checkout . && echo 'restored' "$GITHUB_WORKSPACE/contrib/update-submodules.sh" && echo 'OK' + du -hs "$GITHUB_WORKSPACE/contrib" ||: + find "$GITHUB_WORKSPACE/contrib" -type f | wc -l ||: - name: Build run: | sudo rm -fr "$TEMP_PATH" From 686b284e6c1b48d1ad6e23df0d0a7f20c409dda6 Mon Sep 17 00:00:00 2001 From: Alexander Tokmakov Date: Sat, 1 Apr 2023 02:00:45 +0200 Subject: [PATCH 018/103] fix --- contrib/sparse-checkout/update-orc.sh | 1 + 1 file changed, 1 insertion(+) diff --git a/contrib/sparse-checkout/update-orc.sh b/contrib/sparse-checkout/update-orc.sh index 963db8f306c..57ab57a8d52 100755 --- a/contrib/sparse-checkout/update-orc.sh +++ b/contrib/sparse-checkout/update-orc.sh @@ -6,6 +6,7 @@ FILES_TO_CHECKOUT=$(git rev-parse --git-dir)/info/sparse-checkout echo '/*' > $FILES_TO_CHECKOUT echo '!/*/*' >> $FILES_TO_CHECKOUT echo '/c++/*' >> $FILES_TO_CHECKOUT +echo '/proto/*' >> $FILES_TO_CHECKOUT git config core.sparsecheckout true git checkout $1 From 368c120f427aa40f3f99aa8da594d10e96703b4b Mon Sep 17 00:00:00 2001 From: lzydmxy <13126752315@163.com> Date: Mon, 3 Apr 2023 15:48:40 +0800 Subject: [PATCH 019/103] check sample dictionary key is native unsigned integer --- src/Core/Settings.h | 1 + .../getDictionaryConfigurationFromAST.cpp | 27 +++++++++++- ...check_sample_dict_key_is_correct.reference | 0 ...02707_check_sample_dict_key_is_correct.sql | 41 +++++++++++++++++++ 4 files changed, 67 insertions(+), 2 deletions(-) create mode 100644 tests/queries/0_stateless/02707_check_sample_dict_key_is_correct.reference create mode 100644 tests/queries/0_stateless/02707_check_sample_dict_key_is_correct.sql diff --git a/src/Core/Settings.h b/src/Core/Settings.h index e9db155fb12..7452acd777f 100644 --- a/src/Core/Settings.h +++ b/src/Core/Settings.h @@ -944,6 +944,7 @@ class IColumn; M(Bool, regexp_dict_allow_hyperscan, true, "Allow regexp_tree dictionary using Hyperscan library.", 0) \ \ M(Bool, dictionary_use_async_executor, false, "Execute a pipeline for reading from a dictionary with several threads. It's supported only by DIRECT dictionary with CLICKHOUSE source.", 0) \ + M(Bool, check_sample_dict_key_is_correct, true, "Check primary key type for simple dictionary is Native unsigned integer", 0) \ // End of FORMAT_FACTORY_SETTINGS diff --git a/src/Dictionaries/getDictionaryConfigurationFromAST.cpp b/src/Dictionaries/getDictionaryConfigurationFromAST.cpp index 0de8b843604..081ed4f87e3 100644 --- a/src/Dictionaries/getDictionaryConfigurationFromAST.cpp +++ b/src/Dictionaries/getDictionaryConfigurationFromAST.cpp @@ -19,6 +19,7 @@ #include #include #include +#include namespace DB @@ -341,7 +342,9 @@ void buildPrimaryKeyConfiguration( AutoPtr root, bool complex, const Names & key_names, - const ASTExpressionList * dictionary_attributes) + const ASTExpressionList * dictionary_attributes, + const ASTDictionarySettings * dict_settings, + ContextPtr context) { const auto & children = dictionary_attributes->children; @@ -376,6 +379,26 @@ void buildPrimaryKeyConfiguration( const ASTDictionaryAttributeDeclaration * dict_attr = (*it)->as(); + auto key_type = DataTypeFactory::instance().tryGet(dict_attr->type); + + auto check_sample_dict_key_is_correct = context->getSettings().check_sample_dict_key_is_correct; + + if (dict_settings) + { + if (const auto * check_sample_dict_key_is_correct_change = dict_settings->changes.tryGet("check_sample_dict_key_is_correct")) + { + check_sample_dict_key_is_correct = check_sample_dict_key_is_correct_change->get(); + } + } + + if (check_sample_dict_key_is_correct && !WhichDataType(key_type).isNativeUInt()) + { + throw Exception(ErrorCodes::INCORRECT_DICTIONARY_DEFINITION, + "Invalid Primary key type for simple dictionary: {}. Must be native unsigned integer type. " + "To avoid checking it, please set check_sample_dict_key_is_correct=false", + dict_attr->name); + } + AutoPtr name(doc->createTextNode(dict_attr->name)); name_element->appendChild(name); @@ -614,7 +637,7 @@ getDictionaryConfigurationFromAST(const ASTCreateQuery & query, ContextPtr conte checkPrimaryKey(all_attr_names_and_types, pk_attrs); - buildPrimaryKeyConfiguration(xml_document, structure_element, complex, pk_attrs, query.dictionary_attributes_list); + buildPrimaryKeyConfiguration(xml_document, structure_element, complex, pk_attrs, query.dictionary_attributes_list, query.dictionary->dict_settings, context); buildLayoutConfiguration(xml_document, current_dictionary, query.dictionary->dict_settings, dictionary_layout); buildSourceConfiguration(xml_document, current_dictionary, query.dictionary->source, query.dictionary->dict_settings, context); diff --git a/tests/queries/0_stateless/02707_check_sample_dict_key_is_correct.reference b/tests/queries/0_stateless/02707_check_sample_dict_key_is_correct.reference new file mode 100644 index 00000000000..e69de29bb2d diff --git a/tests/queries/0_stateless/02707_check_sample_dict_key_is_correct.sql b/tests/queries/0_stateless/02707_check_sample_dict_key_is_correct.sql new file mode 100644 index 00000000000..d61d2d59fd7 --- /dev/null +++ b/tests/queries/0_stateless/02707_check_sample_dict_key_is_correct.sql @@ -0,0 +1,41 @@ +DROP DICTIONARY IF EXISTS test_sample_key_dict1; +DROP DICTIONARY IF EXISTS test_sample_key_dict2; +DROP table IF EXISTS test_sample_key_local; + +-- create local table +CREATE TABLE test_sample_key_local +( + `id` Int128, + `name` String +) +ENGINE = Memory; + + +-- create DICTIONARY with default settings check_sample_dict_key_is_correct = 1 +CREATE DICTIONARY test_sample_key_dict1 +( + `id` Int128, + `name` String +) +PRIMARY KEY id +SOURCE(CLICKHOUSE(TABLE 'test_sample_key_local')) +LIFETIME(MIN 0 MAX 300) +LAYOUT(HASHED()); -- { serverError 489 } + + +-- create DICTIONARY with settings check_sample_dict_key_is_correct = 0 +CREATE DICTIONARY test_sample_key_dict2 +( + `id` Int128, + `name` String +) +PRIMARY KEY id +SOURCE(CLICKHOUSE(TABLE 'test_sample_key_local')) +LIFETIME(MIN 0 MAX 300) +LAYOUT(HASHED()) +SETTINGS(check_sample_dict_key_is_correct = 0); + + +DROP DICTIONARY IF EXISTS test_sample_key_dict1; +DROP DICTIONARY IF EXISTS test_sample_key_dict2; +DROP table IF EXISTS test_sample_key_local; \ No newline at end of file From 529e1466df363b844076c7dcd9653fe7fd08bdd4 Mon Sep 17 00:00:00 2001 From: lzydmxy <13126752315@163.com> Date: Tue, 4 Apr 2023 12:04:17 +0800 Subject: [PATCH 020/103] use check_dictionary_primary_key instead of check_sample_dict_key_is_correct --- docs/en/operations/settings/settings.md | 29 +++++++++++++++++++ src/Core/Settings.h | 2 +- .../getDictionaryConfigurationFromAST.cpp | 10 +++---- ...02707_check_sample_dict_key_is_correct.sql | 6 ++-- 4 files changed, 38 insertions(+), 9 deletions(-) diff --git a/docs/en/operations/settings/settings.md b/docs/en/operations/settings/settings.md index 5ea555aa56a..960e20612f6 100644 --- a/docs/en/operations/settings/settings.md +++ b/docs/en/operations/settings/settings.md @@ -4049,3 +4049,32 @@ SELECT sum(number) FROM numbers(10000000000) SETTINGS partial_result_on_first_ca Possible values: `true`, `false` Default value: `false` + +## check_dictionary_primary_key {#check_dictionary_primary_key} + +Enables the check at dictionay creation, dictionaries without word complex-key* in a layout have a key with UInt64 type. The primary key data type must be one of unsigned [integer types](../../sql-reference/data-types/int-uint.md): `UInt8`, `UInt16`, `UInt32`, `UInt64`. +Possible values: + +- true — The check is enabled. +- false — The check is disabled at dictionay creation. + +Default value: `true`. + +If you already have dictionay with incorrect primar key and do not want the server to raise an exception during startup, set `check_dictionary_primary_key` to `false`. + +Or you can create dictionay with settings `check_dictionary_primary_key` to `false`. + +**Example** + +```sql +CREATE DICTIONARY test +( + `id` Int128, + `name` String +) +PRIMARY KEY id +SOURCE(CLICKHOUSE(TABLE 'test_local')) +LIFETIME(MIN 0 MAX 300) +LAYOUT(HASHED()) +SETTINGS(check_dictionary_primary_key = 0); +``` \ No newline at end of file diff --git a/src/Core/Settings.h b/src/Core/Settings.h index 7452acd777f..ae8f34986a8 100644 --- a/src/Core/Settings.h +++ b/src/Core/Settings.h @@ -944,7 +944,7 @@ class IColumn; M(Bool, regexp_dict_allow_hyperscan, true, "Allow regexp_tree dictionary using Hyperscan library.", 0) \ \ M(Bool, dictionary_use_async_executor, false, "Execute a pipeline for reading from a dictionary with several threads. It's supported only by DIRECT dictionary with CLICKHOUSE source.", 0) \ - M(Bool, check_sample_dict_key_is_correct, true, "Check primary key type for simple dictionary is Native unsigned integer", 0) \ + M(Bool, check_dictionary_primary_key, true, "Check primary key type for simple dictionary is native unsigned integer", 0) \ // End of FORMAT_FACTORY_SETTINGS diff --git a/src/Dictionaries/getDictionaryConfigurationFromAST.cpp b/src/Dictionaries/getDictionaryConfigurationFromAST.cpp index 081ed4f87e3..5aecd54c115 100644 --- a/src/Dictionaries/getDictionaryConfigurationFromAST.cpp +++ b/src/Dictionaries/getDictionaryConfigurationFromAST.cpp @@ -381,21 +381,21 @@ void buildPrimaryKeyConfiguration( auto key_type = DataTypeFactory::instance().tryGet(dict_attr->type); - auto check_sample_dict_key_is_correct = context->getSettings().check_sample_dict_key_is_correct; + auto check_dictionary_primary_key = context->getSettingsRef().check_dictionary_primary_key; if (dict_settings) { - if (const auto * check_sample_dict_key_is_correct_change = dict_settings->changes.tryGet("check_sample_dict_key_is_correct")) + if (const auto * check_dictionary_primary_key_change = dict_settings->changes.tryGet("check_dictionary_primary_key")) { - check_sample_dict_key_is_correct = check_sample_dict_key_is_correct_change->get(); + check_dictionary_primary_key = check_dictionary_primary_key_change->get(); } } - if (check_sample_dict_key_is_correct && !WhichDataType(key_type).isNativeUInt()) + if (check_dictionary_primary_key && !WhichDataType(key_type).isNativeUInt()) { throw Exception(ErrorCodes::INCORRECT_DICTIONARY_DEFINITION, "Invalid Primary key type for simple dictionary: {}. Must be native unsigned integer type. " - "To avoid checking it, please set check_sample_dict_key_is_correct=false", + "To avoid checking it, please set check_dictionary_primary_key=false", dict_attr->name); } diff --git a/tests/queries/0_stateless/02707_check_sample_dict_key_is_correct.sql b/tests/queries/0_stateless/02707_check_sample_dict_key_is_correct.sql index d61d2d59fd7..35cc14e73f1 100644 --- a/tests/queries/0_stateless/02707_check_sample_dict_key_is_correct.sql +++ b/tests/queries/0_stateless/02707_check_sample_dict_key_is_correct.sql @@ -11,7 +11,7 @@ CREATE TABLE test_sample_key_local ENGINE = Memory; --- create DICTIONARY with default settings check_sample_dict_key_is_correct = 1 +-- create DICTIONARY with default settings check_dictionary_primary_key = 1 CREATE DICTIONARY test_sample_key_dict1 ( `id` Int128, @@ -23,7 +23,7 @@ LIFETIME(MIN 0 MAX 300) LAYOUT(HASHED()); -- { serverError 489 } --- create DICTIONARY with settings check_sample_dict_key_is_correct = 0 +-- create DICTIONARY with settings check_dictionary_primary_key = 0 CREATE DICTIONARY test_sample_key_dict2 ( `id` Int128, @@ -33,7 +33,7 @@ PRIMARY KEY id SOURCE(CLICKHOUSE(TABLE 'test_sample_key_local')) LIFETIME(MIN 0 MAX 300) LAYOUT(HASHED()) -SETTINGS(check_sample_dict_key_is_correct = 0); +SETTINGS(check_dictionary_primary_key = 0); DROP DICTIONARY IF EXISTS test_sample_key_dict1; From 3cee537e73015052fcbeff6b09ab2b98a4141421 Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?Ra=C3=BAl=20Mar=C3=ADn?= Date: Tue, 4 Apr 2023 18:58:28 +0200 Subject: [PATCH 021/103] Changes for master --- src/Backups/BackupEntriesCollector.cpp | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/src/Backups/BackupEntriesCollector.cpp b/src/Backups/BackupEntriesCollector.cpp index 1adc4d41fee..4e904bdc402 100644 --- a/src/Backups/BackupEntriesCollector.cpp +++ b/src/Backups/BackupEntriesCollector.cpp @@ -490,7 +490,7 @@ std::vector> BackupEntriesCollector::findTablesInD { /// Database or table could be replicated - so may use ZooKeeper. We need to retry. auto zookeeper_retries_info = global_zookeeper_retries_info; - ZooKeeperRetriesControl retries_ctl("getTablesForBackup", zookeeper_retries_info); + ZooKeeperRetriesControl retries_ctl("getTablesForBackup", zookeeper_retries_info, context->getProcessListElement()); retries_ctl.retryLoop([&](){ db_tables = database->getTablesForBackup(filter_by_table_name, context); }); } catch (Exception & e) From 96a1304730cfc56dc5c51a8ffbf0a475e9988f45 Mon Sep 17 00:00:00 2001 From: Dmitry Novik Date: Tue, 4 Apr 2023 20:33:59 +0000 Subject: [PATCH 022/103] WIP on additional_table_filters --- src/Planner/PlannerJoinTree.cpp | 57 +++++++++++++++++++++++++++++++++ 1 file changed, 57 insertions(+) diff --git a/src/Planner/PlannerJoinTree.cpp b/src/Planner/PlannerJoinTree.cpp index 0479170eba1..780b1ffee0c 100644 --- a/src/Planner/PlannerJoinTree.cpp +++ b/src/Planner/PlannerJoinTree.cpp @@ -1,6 +1,10 @@ #include +#include "Common/logger_useful.h" #include +#include "Parsers/ExpressionListParsers.h" +#include "Parsers/parseQuery.h" +#include "Storages/SelectQueryInfo.h" #include @@ -468,6 +472,51 @@ FilterDAGInfo buildCustomKeyFilterIfNeeded(const StoragePtr & storage, return buildFilterInfo(parallel_replicas_custom_filter_ast, table_expression_query_info, planner_context); } +/// Apply filters from additional_table_filters setting +FilterDAGInfo buildAdditionalFiltersIfNeeded(const StoragePtr & storage, + const String & table_expression_alias, + SelectQueryInfo & table_expression_query_info, + PlannerContextPtr & planner_context) +{ + const auto & query_context = planner_context->getQueryContext(); + const auto & settings = query_context->getSettingsRef(); + + auto const & additional_filters = settings.additional_table_filters.value; + if (additional_filters.empty()) + return {}; + + auto const & storage_id = storage->getStorageID(); + + LOG_DEBUG(&Poco::Logger::get("buildAdditionalFiltersIfNeeded"), "Trying to find additional filters for table: {}", storage_id.getFullTableName()); + + ASTPtr additional_filter_ast; + + for (size_t i = 0; i < additional_filters.size(); ++i) + { + const auto & tuple = additional_filters[i].safeGet(); + auto const & table = tuple.at(0).safeGet(); + auto const & filter = tuple.at(1).safeGet(); + + if (table == table_expression_alias || + (table == storage_id.getTableName() && query_context->getCurrentDatabase() == storage_id.getDatabaseName()) || + (table == storage_id.getFullNameNotQuoted())) + { + ParserExpression parser; + additional_filter_ast = parseQuery( + parser, filter.data(), filter.data() + filter.size(), + "additional filter", settings.max_query_size, settings.max_parser_depth); + break; + } + } + + if (!additional_filter_ast) + return {}; + + LOG_DEBUG(&Poco::Logger::get("buildAdditionalFiltersIfNeeded"), "Found additional filter: {}", additional_filter_ast->formatForErrorMessage()); + + return buildFilterInfo(additional_filter_ast, table_expression_query_info, planner_context); +} + JoinTreeQueryPlan buildQueryPlanForTableExpression(QueryTreeNodePtr table_expression, const SelectQueryInfo & select_query_info, const SelectQueryOptions & select_query_options, @@ -696,6 +745,10 @@ JoinTreeQueryPlan buildQueryPlanForTableExpression(QueryTreeNodePtr table_expres } } + const auto & table_expression_alias = table_expression->getAlias(); + auto additional_filters_info = buildAdditionalFiltersIfNeeded(storage, table_expression_alias, table_expression_query_info, planner_context); + add_filter(additional_filters_info, "additional filter"); + from_stage = storage->getQueryProcessingStage(query_context, select_query_options.to_stage, storage_snapshot, table_expression_query_info); storage->read(query_plan, columns_names, storage_snapshot, table_expression_query_info, query_context, from_stage, max_block_size, max_streams); @@ -713,6 +766,10 @@ JoinTreeQueryPlan buildQueryPlanForTableExpression(QueryTreeNodePtr table_expres filter_step->setStepDescription(description); query_plan.addStep(std::move(filter_step)); } + else + { + LOG_DEBUG(&Poco::Logger::get("PlannerJoinTree"), "Can not add filter: {}", description); + } } if (query_context->hasQueryContext() && !select_query_options.is_internal) From 0d5d2a9b557c808fb2985824479c1c2c358fb2c4 Mon Sep 17 00:00:00 2001 From: Robert Schulze Date: Wed, 5 Apr 2023 08:26:00 +0000 Subject: [PATCH 023/103] Implement %f in parseDateTime() Fixes: #48394 @cc OP --- .../functions/type-conversion-functions.md | 1 - src/Functions/parseDateTime.cpp | 36 +++++++++++++++++-- .../02668_parse_datetime.reference | 7 ++++ .../0_stateless/02668_parse_datetime.sql | 7 ++++ 4 files changed, 48 insertions(+), 3 deletions(-) diff --git a/docs/en/sql-reference/functions/type-conversion-functions.md b/docs/en/sql-reference/functions/type-conversion-functions.md index 213ed187f15..5ce72caa3b9 100644 --- a/docs/en/sql-reference/functions/type-conversion-functions.md +++ b/docs/en/sql-reference/functions/type-conversion-functions.md @@ -1245,7 +1245,6 @@ Returns DateTime values parsed from input string according to a MySQL style form **Supported format specifiers** All format specifiers listed in [formatDateTime](/docs/en/sql-reference/functions/date-time-functions.md#date_time_functions-formatDateTime) except: -- %f: fractional second - %Q: Quarter (1-4) **Example** diff --git a/src/Functions/parseDateTime.cpp b/src/Functions/parseDateTime.cpp index 6077e00f915..b929bc45878 100644 --- a/src/Functions/parseDateTime.cpp +++ b/src/Functions/parseDateTime.cpp @@ -1035,6 +1035,36 @@ namespace return cur; } + static Pos mysqlMicrosecond(Pos cur, Pos end, const String & fragment, DateTime & /*date*/) + { + checkSpace(cur, end, 6, "mysqlMicrosecond requires size >= 6", fragment); + + Pos start = cur; + auto check_is_number = [&](Pos pos) { + if (*pos < '0' || *pos > '9') + throw Exception( + ErrorCodes::CANNOT_PARSE_DATETIME, + "Unable to parse fragment '{}' from '{}' because '{}'' is not a number ", + fragment, + std::string_view(start, end), + *cur); + }; + + check_is_number(cur); + ++cur; + check_is_number(cur); + ++cur; + check_is_number(cur); + ++cur; + check_is_number(cur); + ++cur; + check_is_number(cur); + ++cur; + check_is_number(cur); + ++cur; + return cur; + } + static Pos mysqlISO8601Time(Pos cur, Pos end, const String & fragment, DateTime & date) { checkSpace(cur, end, 8, "mysqlISO8601Time requires size >= 8", fragment); @@ -1446,6 +1476,10 @@ namespace instructions.emplace_back(ACTION_ARGS(Instruction::mysqlDayOfMonthSpacePadded)); break; + // Fractional seconds + case 'f': + instructions.emplace_back(ACTION_ARGS(Instruction::mysqlMicrosecond)); + break; // Short YYYY-MM-DD date, equivalent to %Y-%m-%d 2001-08-23 case 'F': @@ -1593,8 +1627,6 @@ namespace /// Unimplemented /// Fractional seconds - case 'f': - throw Exception(ErrorCodes::NOT_IMPLEMENTED, "format is not supported for fractional seconds"); case 'U': throw Exception(ErrorCodes::NOT_IMPLEMENTED, "format is not supported for WEEK (Sun-Sat)"); case 'v': diff --git a/tests/queries/0_stateless/02668_parse_datetime.reference b/tests/queries/0_stateless/02668_parse_datetime.reference index 6bcd4a42c10..a5b5ad7d109 100644 --- a/tests/queries/0_stateless/02668_parse_datetime.reference +++ b/tests/queries/0_stateless/02668_parse_datetime.reference @@ -190,6 +190,13 @@ select parseDateTime('00/', '%s/', 'UTC') = toDateTime('1970-01-01 00:00:00', 'U select parseDateTime('60', '%s', 'UTC'); -- { serverError CANNOT_PARSE_DATETIME } select parseDateTime('-1', '%s', 'UTC'); -- { serverError CANNOT_PARSE_DATETIME } select parseDateTime('123456789', '%s', 'UTC'); -- { serverError CANNOT_PARSE_DATETIME } +-- microsecond +select parseDateTime('000000', '%f', 'UTC') = toDateTime('1970-01-01 00:00:00', 'UTC'); +1 +select parseDateTime('456789', '%f', 'UTC') = toDateTime('1970-01-01 00:00:00', 'UTC'); +1 +select parseDateTime('42', '%f', 'UTC') = toDateTime('1970-01-01 00:00:00', 'UTC'); -- { serverError NOT_ENOUGH_SPACE } +select parseDateTime('12ABCD', '%f', 'UTC') = toDateTime('1970-01-01 00:00:00', 'UTC'); -- { serverError CANNOT_PARSE_DATETIME } -- mixed YMD format select parseDateTime('2021-01-04+23:00:00', '%Y-%m-%d+%H:%i:%s', 'UTC') = toDateTime('2021-01-04 23:00:00', 'UTC'); 1 diff --git a/tests/queries/0_stateless/02668_parse_datetime.sql b/tests/queries/0_stateless/02668_parse_datetime.sql index abe3505de03..33e84120521 100644 --- a/tests/queries/0_stateless/02668_parse_datetime.sql +++ b/tests/queries/0_stateless/02668_parse_datetime.sql @@ -127,6 +127,13 @@ select parseDateTime('00/', '%s/', 'UTC') = toDateTime('1970-01-01 00:00:00', 'U select parseDateTime('60', '%s', 'UTC'); -- { serverError CANNOT_PARSE_DATETIME } select parseDateTime('-1', '%s', 'UTC'); -- { serverError CANNOT_PARSE_DATETIME } select parseDateTime('123456789', '%s', 'UTC'); -- { serverError CANNOT_PARSE_DATETIME } + +-- microsecond +select parseDateTime('000000', '%f', 'UTC') = toDateTime('1970-01-01 00:00:00', 'UTC'); +select parseDateTime('456789', '%f', 'UTC') = toDateTime('1970-01-01 00:00:00', 'UTC'); +select parseDateTime('42', '%f', 'UTC') = toDateTime('1970-01-01 00:00:00', 'UTC'); -- { serverError NOT_ENOUGH_SPACE } +select parseDateTime('12ABCD', '%f', 'UTC') = toDateTime('1970-01-01 00:00:00', 'UTC'); -- { serverError CANNOT_PARSE_DATETIME } + -- mixed YMD format select parseDateTime('2021-01-04+23:00:00', '%Y-%m-%d+%H:%i:%s', 'UTC') = toDateTime('2021-01-04 23:00:00', 'UTC'); select parseDateTime('2019-07-03 11:04:10', '%Y-%m-%d %H:%i:%s', 'UTC') = toDateTime('2019-07-03 11:04:10', 'UTC'); From 2276d4feb478a133afe2779706b01bfcd726b731 Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?Ra=C3=BAl=20Mar=C3=ADn?= Date: Wed, 5 Apr 2023 11:19:04 +0200 Subject: [PATCH 024/103] Backups have no context and no process list element --- src/Backups/BackupEntriesCollector.cpp | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/src/Backups/BackupEntriesCollector.cpp b/src/Backups/BackupEntriesCollector.cpp index 4e904bdc402..14ecccb0c81 100644 --- a/src/Backups/BackupEntriesCollector.cpp +++ b/src/Backups/BackupEntriesCollector.cpp @@ -490,7 +490,7 @@ std::vector> BackupEntriesCollector::findTablesInD { /// Database or table could be replicated - so may use ZooKeeper. We need to retry. auto zookeeper_retries_info = global_zookeeper_retries_info; - ZooKeeperRetriesControl retries_ctl("getTablesForBackup", zookeeper_retries_info, context->getProcessListElement()); + ZooKeeperRetriesControl retries_ctl("getTablesForBackup", zookeeper_retries_info, nullptr); retries_ctl.retryLoop([&](){ db_tables = database->getTablesForBackup(filter_by_table_name, context); }); } catch (Exception & e) From a59b53775c0048f012e4d55042e2070b85858819 Mon Sep 17 00:00:00 2001 From: Robert Schulze Date: Wed, 5 Apr 2023 10:43:55 +0000 Subject: [PATCH 025/103] Fix style --- src/Functions/parseDateTime.cpp | 3 ++- 1 file changed, 2 insertions(+), 1 deletion(-) diff --git a/src/Functions/parseDateTime.cpp b/src/Functions/parseDateTime.cpp index b929bc45878..01239074852 100644 --- a/src/Functions/parseDateTime.cpp +++ b/src/Functions/parseDateTime.cpp @@ -1040,7 +1040,8 @@ namespace checkSpace(cur, end, 6, "mysqlMicrosecond requires size >= 6", fragment); Pos start = cur; - auto check_is_number = [&](Pos pos) { + auto check_is_number = [&](Pos pos) + { if (*pos < '0' || *pos > '9') throw Exception( ErrorCodes::CANNOT_PARSE_DATETIME, From 41f6e0e7b5916b6148bfbbad148c6a60ab5297c7 Mon Sep 17 00:00:00 2001 From: Igor Nikonov Date: Wed, 5 Apr 2023 13:34:56 +0000 Subject: [PATCH 026/103] More efficient original node search in ActionsDAG --- src/Interpreters/ActionsDAG.cpp | 29 ++++++++++++--------- src/Processors/QueryPlan/ExpressionStep.cpp | 14 +++++----- 2 files changed, 23 insertions(+), 20 deletions(-) diff --git a/src/Interpreters/ActionsDAG.cpp b/src/Interpreters/ActionsDAG.cpp index bb7b309d362..e1f38a704b7 100644 --- a/src/Interpreters/ActionsDAG.cpp +++ b/src/Interpreters/ActionsDAG.cpp @@ -2500,8 +2500,21 @@ ActionsDAGPtr ActionsDAG::buildFilterActionsDAG( FindOriginalNodeForOutputName::FindOriginalNodeForOutputName(const ActionsDAGPtr & actions_) :actions(actions_) { - for (const auto * node : actions->getOutputs()) - index.emplace(node->result_name, node); + const auto & actions_outputs = actions->getOutputs(); + for (const auto * output_node : actions_outputs) + { + /// find input node which refers to the output node + /// consider only aliases on the path + const auto * node = output_node; + while (node && node->type == ActionsDAG::ActionType::ALIAS) + { + /// alias has only one child + chassert(node->children.size() == 1); + node = node->children.front(); + } + if (node && node->type == ActionsDAG::ActionType::INPUT) + index.emplace(output_node->result_name, node); + } } const ActionsDAG::Node * FindOriginalNodeForOutputName::find(const String & output_name) @@ -2510,17 +2523,7 @@ const ActionsDAG::Node * FindOriginalNodeForOutputName::find(const String & outp if (it == index.end()) return nullptr; - /// find original(non alias) node it refers to - const ActionsDAG::Node * node = it->second; - while (node && node->type == ActionsDAG::ActionType::ALIAS) - { - chassert(!node->children.empty()); - node = node->children.front(); - } - if (node && node->type != ActionsDAG::ActionType::INPUT) - return nullptr; - - return node; + return it->second; } } diff --git a/src/Processors/QueryPlan/ExpressionStep.cpp b/src/Processors/QueryPlan/ExpressionStep.cpp index a93ccb2e020..fcd7164e6fe 100644 --- a/src/Processors/QueryPlan/ExpressionStep.cpp +++ b/src/Processors/QueryPlan/ExpressionStep.cpp @@ -80,24 +80,24 @@ void ExpressionStep::updateOutputStream() return; FindOriginalNodeForOutputName original_node_finder(actions_dag); + Names output_names = actions_dag->getNames(); const auto & input_sort_description = getInputStreams().front().sort_description; for (size_t i = 0, s = input_sort_description.size(); i < s; ++i) { - const auto & desc = input_sort_description[i]; String alias; - const auto & origin_column = desc.column_name; - for (const auto & column : output_stream->header) + const auto & original_column = input_sort_description[i].column_name; + for (const auto & column_name : output_names) { - const auto * original_node = original_node_finder.find(column.name); - if (original_node && original_node->result_name == origin_column) + const auto * original_node = original_node_finder.find(column_name); + if (original_node && original_node->result_name == original_column) { - alias = column.name; + alias = column_name; break; } } if (alias.empty()) - return; + break; output_stream->sort_description[i].column_name = alias; } From 46f7016d8c62e47511dcc1b79492db46a88f98b4 Mon Sep 17 00:00:00 2001 From: Igor Nikonov Date: Wed, 5 Apr 2023 15:56:06 +0000 Subject: [PATCH 027/103] Add search alias by input name in ActionsDAG --- src/Interpreters/ActionsDAG.cpp | 29 +++++++++++++++++++++ src/Interpreters/ActionsDAG.h | 15 ++++++++++- src/Processors/QueryPlan/ExpressionStep.cpp | 20 +++----------- src/Processors/QueryPlan/FilterStep.cpp | 22 ++++------------ 4 files changed, 52 insertions(+), 34 deletions(-) diff --git a/src/Interpreters/ActionsDAG.cpp b/src/Interpreters/ActionsDAG.cpp index e1f38a704b7..a34541f3e59 100644 --- a/src/Interpreters/ActionsDAG.cpp +++ b/src/Interpreters/ActionsDAG.cpp @@ -2526,4 +2526,33 @@ const ActionsDAG::Node * FindOriginalNodeForOutputName::find(const String & outp return it->second; } +FindAliasForInputName::FindAliasForInputName(const ActionsDAGPtr & actions_) + :actions(actions_) +{ + const auto & actions_outputs = actions->getOutputs(); + for (const auto * output_node : actions_outputs) + { + /// find input node which corresponds to alias + const auto * node = output_node; + while (node && node->type == ActionsDAG::ActionType::ALIAS) + { + /// alias has only one child + chassert(node->children.size() == 1); + node = node->children.front(); + } + if (node && node->type == ActionsDAG::ActionType::INPUT) + /// node can have several aliases but we consider only the first one + index.emplace(node->result_name, output_node); + } +} + +const ActionsDAG::Node * FindAliasForInputName::find(const String & name) +{ + const auto it = index.find(name); + if (it == index.end()) + return nullptr; + + return it->second; +} + } diff --git a/src/Interpreters/ActionsDAG.h b/src/Interpreters/ActionsDAG.h index b94dba99b67..2d6ede0db00 100644 --- a/src/Interpreters/ActionsDAG.h +++ b/src/Interpreters/ActionsDAG.h @@ -408,7 +408,20 @@ class FindOriginalNodeForOutputName public: explicit FindOriginalNodeForOutputName(const ActionsDAGPtr & actions); - const ActionsDAG::Node* find(const String& output_name); + const ActionsDAG::Node * find(const String & output_name); + +private: + ActionsDAGPtr actions; + NameToNodeIndex index; +}; + +class FindAliasForInputName +{ + using NameToNodeIndex = std::unordered_map; + +public: + explicit FindAliasForInputName(const ActionsDAGPtr & actions); + const ActionsDAG::Node * find(const String & name); private: ActionsDAGPtr actions; diff --git a/src/Processors/QueryPlan/ExpressionStep.cpp b/src/Processors/QueryPlan/ExpressionStep.cpp index fcd7164e6fe..5062ff5479e 100644 --- a/src/Processors/QueryPlan/ExpressionStep.cpp +++ b/src/Processors/QueryPlan/ExpressionStep.cpp @@ -79,27 +79,15 @@ void ExpressionStep::updateOutputStream() if (!getDataStreamTraits().preserves_sorting) return; - FindOriginalNodeForOutputName original_node_finder(actions_dag); - Names output_names = actions_dag->getNames(); + FindAliasForInputName alias_finder(actions_dag); const auto & input_sort_description = getInputStreams().front().sort_description; for (size_t i = 0, s = input_sort_description.size(); i < s; ++i) { String alias; const auto & original_column = input_sort_description[i].column_name; - for (const auto & column_name : output_names) - { - const auto * original_node = original_node_finder.find(column_name); - if (original_node && original_node->result_name == original_column) - { - alias = column_name; - break; - } - } - - if (alias.empty()) - break; - - output_stream->sort_description[i].column_name = alias; + const auto * alias_node = alias_finder.find(original_column); + if (alias_node) + output_stream->sort_description[i].column_name = alias_node->result_name; } } diff --git a/src/Processors/QueryPlan/FilterStep.cpp b/src/Processors/QueryPlan/FilterStep.cpp index 03a301d4077..1b9b47f46ac 100644 --- a/src/Processors/QueryPlan/FilterStep.cpp +++ b/src/Processors/QueryPlan/FilterStep.cpp @@ -109,27 +109,15 @@ void FilterStep::updateOutputStream() if (!getDataStreamTraits().preserves_sorting) return; - FindOriginalNodeForOutputName original_node_finder(actions_dag); + FindAliasForInputName alias_finder(actions_dag); const auto & input_sort_description = getInputStreams().front().sort_description; for (size_t i = 0, s = input_sort_description.size(); i < s; ++i) { - const auto & desc = input_sort_description[i]; String alias; - const auto & origin_column = desc.column_name; - for (const auto & column : output_stream->header) - { - const auto * original_node = original_node_finder.find(column.name); - if (original_node && original_node->result_name == origin_column) - { - alias = column.name; - break; - } - } - - if (alias.empty()) - return; - - output_stream->sort_description[i].column_name = alias; + const auto & original_column = input_sort_description[i].column_name; + const auto * alias_node = alias_finder.find(original_column); + if (alias_node) + output_stream->sort_description[i].column_name = alias_node->result_name; } } From 4cb1ffa2ebf707ce1c91db1c57cace2882eb0651 Mon Sep 17 00:00:00 2001 From: Dmitry Novik Date: Wed, 5 Apr 2023 16:13:03 +0000 Subject: [PATCH 028/103] Fix distributed case --- src/Planner/PlannerJoinTree.cpp | 2 ++ 1 file changed, 2 insertions(+) diff --git a/src/Planner/PlannerJoinTree.cpp b/src/Planner/PlannerJoinTree.cpp index 780b1ffee0c..c220e1eaa88 100644 --- a/src/Planner/PlannerJoinTree.cpp +++ b/src/Planner/PlannerJoinTree.cpp @@ -512,6 +512,8 @@ FilterDAGInfo buildAdditionalFiltersIfNeeded(const StoragePtr & storage, if (!additional_filter_ast) return {}; + table_expression_query_info.additional_filter_ast = additional_filter_ast; + LOG_DEBUG(&Poco::Logger::get("buildAdditionalFiltersIfNeeded"), "Found additional filter: {}", additional_filter_ast->formatForErrorMessage()); return buildFilterInfo(additional_filter_ast, table_expression_query_info, planner_context); From 1be6bceb2fc0a6030c8832053381475b5a2d7c86 Mon Sep 17 00:00:00 2001 From: avogar Date: Wed, 5 Apr 2023 16:23:06 +0000 Subject: [PATCH 029/103] Generate valid Decimals and Bools in generateRandom function --- src/Storages/StorageGenerateRandom.cpp | 55 +++++++++++++++---- ..._random_valid_decimals_and_bools.reference | 25 +++++++++ ...nerate_random_valid_decimals_and_bools.sql | 5 ++ 3 files changed, 73 insertions(+), 12 deletions(-) create mode 100644 tests/queries/0_stateless/02709_generate_random_valid_decimals_and_bools.reference create mode 100644 tests/queries/0_stateless/02709_generate_random_valid_decimals_and_bools.sql diff --git a/src/Storages/StorageGenerateRandom.cpp b/src/Storages/StorageGenerateRandom.cpp index 03f3aff412f..fc642eb7cc3 100644 --- a/src/Storages/StorageGenerateRandom.cpp +++ b/src/Storages/StorageGenerateRandom.cpp @@ -24,6 +24,7 @@ #include #include #include +#include #include #include @@ -232,8 +233,17 @@ ColumnPtr fillColumnWithRandomData( case TypeIndex::UInt8: { auto column = ColumnUInt8::create(); - column->getData().resize(limit); - fillBufferWithRandomData(reinterpret_cast(column->getData().data()), limit, sizeof(UInt8), rng); + auto & data = column->getData(); + data.resize(limit); + if (isBool(type)) + { + for (size_t i = 0; i < limit; ++i) + data[i] = rng() % 2; + } + else + { + fillBufferWithRandomData(reinterpret_cast(data.data()), limit, sizeof(UInt8), rng); + } return column; } case TypeIndex::UInt16: [[fallthrough]]; @@ -349,34 +359,55 @@ ColumnPtr fillColumnWithRandomData( } case TypeIndex::Decimal32: { - auto column = type->createColumn(); + const auto & decimal_type = assert_cast &>(*type); + auto column = decimal_type.createColumn(); auto & column_concrete = typeid_cast &>(*column); - column_concrete.getData().resize(limit); - fillBufferWithRandomData(reinterpret_cast(column_concrete.getData().data()), limit, sizeof(Decimal32), rng, true); + auto & data = column_concrete.getData(); + data.resize(limit); + /// Generate numbers from range [-10^P + 1, 10^P - 1] + Int32 range = common::exp10_i32(decimal_type.getPrecision()); + for (size_t i = 0; i != limit; ++i) + data[i] = static_cast(rng()) % range; return column; } - case TypeIndex::Decimal64: /// TODO Decimal may be generated out of range. + case TypeIndex::Decimal64: { + const auto & decimal_type = assert_cast &>(*type); auto column = type->createColumn(); auto & column_concrete = typeid_cast &>(*column); - column_concrete.getData().resize(limit); - fillBufferWithRandomData(reinterpret_cast(column_concrete.getData().data()), limit, sizeof(Decimal64), rng, true); + auto & data = column_concrete.getData(); + data.resize(limit); + /// Generate numbers from range [-10^P + 1, 10^P - 1] + Int64 range = common::exp10_i64(decimal_type.getPrecision()); + for (size_t i = 0; i != limit; ++i) + data[i] = static_cast(rng()) % range; + return column; } case TypeIndex::Decimal128: { + const auto & decimal_type = assert_cast &>(*type); auto column = type->createColumn(); auto & column_concrete = typeid_cast &>(*column); - column_concrete.getData().resize(limit); - fillBufferWithRandomData(reinterpret_cast(column_concrete.getData().data()), limit, sizeof(Decimal128), rng, true); + auto & data = column_concrete.getData(); + data.resize(limit); + /// Generate numbers from range [-10^P + 1, 10^P - 1] + Int128 range = common::exp10_i128(decimal_type.getPrecision()); + for (size_t i = 0; i != limit; ++i) + data[i] = Int128({rng(), rng()}) % range; return column; } case TypeIndex::Decimal256: { + const auto & decimal_type = assert_cast &>(*type); auto column = type->createColumn(); auto & column_concrete = typeid_cast &>(*column); - column_concrete.getData().resize(limit); - fillBufferWithRandomData(reinterpret_cast(column_concrete.getData().data()), limit, sizeof(Decimal256), rng, true); + auto & data = column_concrete.getData(); + data.resize(limit); + /// Generate numbers from range [-10^P + 1, 10^P - 1] + Int256 range = common::exp10_i256(decimal_type.getPrecision()); + for (size_t i = 0; i != limit; ++i) + data[i] = Int256({rng(), rng(), rng(), rng()}) % range; return column; } case TypeIndex::FixedString: diff --git a/tests/queries/0_stateless/02709_generate_random_valid_decimals_and_bools.reference b/tests/queries/0_stateless/02709_generate_random_valid_decimals_and_bools.reference new file mode 100644 index 00000000000..49f0c2c1360 --- /dev/null +++ b/tests/queries/0_stateless/02709_generate_random_valid_decimals_and_bools.reference @@ -0,0 +1,25 @@ +32.077 +-421.374 +401.741 +19.925 +-53.055 +-741245.011359027 +-108826.704302334 +-885710.601317107 +200615.252943765 +571119.753066497 +-1810705720.77468465228544079155 +4423020457.03124833705321108749 +-7357115755.03893179428185573375 +-3596476105.34116783307269095642 +-882544888.46147190610682821046 +2998274172057708048.9272057168211482653001963259921827760845 +-59397657133291422934.9333984240607897191609175175045361436671 +39869165044835399916.6747138660882932389363995495451540473418 +-57185968744047146404.1855920695644202095475491426441451681562 +-45866526235163110880.4305861667709353032099072504822212330478 +1 +0 +1 +1 +1 diff --git a/tests/queries/0_stateless/02709_generate_random_valid_decimals_and_bools.sql b/tests/queries/0_stateless/02709_generate_random_valid_decimals_and_bools.sql new file mode 100644 index 00000000000..c290ce4833a --- /dev/null +++ b/tests/queries/0_stateless/02709_generate_random_valid_decimals_and_bools.sql @@ -0,0 +1,5 @@ +select toString(x)::Decimal(6, 3) from generateRandom('x Decimal(6, 3)', 42) limit 5; +select toString(x)::Decimal(15, 9) from generateRandom('x Decimal(15, 9)', 42) limit 5; +select toString(x)::Decimal(30, 20) from generateRandom('x Decimal(30, 20)', 42) limit 5; +select toString(x)::Decimal(60, 40) from generateRandom('x Decimal(60, 40)', 42) limit 5; +select reinterpret(x, 'UInt8') from generateRandom('x Bool', 42) limit 5; From af257572eb5a8508395c6a327978e6bbc8f6f4c7 Mon Sep 17 00:00:00 2001 From: avogar Date: Wed, 5 Apr 2023 16:43:55 +0000 Subject: [PATCH 030/103] Update tests --- .../01087_storage_generate.reference | 200 +++++++++--------- .../01087_table_function_generate.reference | 60 +++--- 2 files changed, 130 insertions(+), 130 deletions(-) diff --git a/tests/queries/0_stateless/01087_storage_generate.reference b/tests/queries/0_stateless/01087_storage_generate.reference index 00ea8ac914f..3b388d0e80f 100644 --- a/tests/queries/0_stateless/01087_storage_generate.reference +++ b/tests/queries/0_stateless/01087_storage_generate.reference @@ -1,103 +1,103 @@ 100 - -[] -54259.6828 ('2088-03-01 13:26:24.094','d3c2a216-a98c-d56c-7bf7-62de9f264cf4') -[88] 34528.4014 ('2031-12-08 21:40:39.898','9ef777c8-de0e-d25e-e16c-5b624f88523c') -[-1] 121968.7945 ('2060-02-05 06:18:12.011','7655e515-d2ca-2f06-0950-e4f44f69aca7') -[-103,75] -135033.4349 ('2038-12-19 17:38:58.695','86b57d15-292d-2517-9acf-47cd053e7a3a') -[110] -202668.69 ('2009-06-17 21:53:29.808','bc630f78-7d58-0c46-dd4b-27fc35625e96') -[-22,2] 168636.9728 ('2074-09-03 06:20:20.936','7624ce27-9bff-4e9d-3f18-6851a97dd0ca') -[-22,-62] -75192.4989 ('2085-10-11 18:51:12.855','a4c4d0ed-f448-244e-1723-ca1bba816f2b') -[-2,-90] 133592.5064 ('2010-10-28 17:18:04.633','8ba9103b-f90c-b49b-38c1-223ae5f42bf7') -[-94,80] 197330.6359 ('2024-03-30 19:08:45.772','83442013-3677-5097-065d-72dfbe8a3506') -[23] 167557.6237 ('2078-07-25 18:54:42.480','be14d98e-5b24-54ee-c959-d24fa9a58fdd') -[46,-10,-63] 185107.1979 ('2040-10-07 03:06:53.504','5ed1fe6a-9313-41d7-4bf9-3948e961509f') -[-107,68] -163781.3045 ('2021-12-21 16:18:58.933','7b634f19-0863-829e-484b-be288aab54a1') -[-35,-116,73] -203577.5379 ('2093-08-01 17:21:09.407','d371bad4-b098-ffdd-f84c-6a02390c2939') -[61] 152284.9386 ('2089-12-20 16:21:33.149','9e8426c1-278a-4d9c-4076-364a95b065e3') -[75] 170968.4171 ('2020-07-17 12:45:31.975','47397a81-bda7-8bd9-59f7-d60e2204fe99') -[-115,93] -173740.5652 ('2098-04-25 19:10:33.327','117e31dd-102e-ee6c-0dbd-0a4203c18ca5') -[-20,4,21] 63834.8685 ('2000-07-08 14:09:40.271','10b0fa48-55a3-755a-4a44-36315ae04c1c') -[-110,117,91] -160640.1506 ('1998-04-18 06:58:04.479','6dfa3a8e-6e65-543c-5f50-1ff45835aa5a') -[62] 63817.7977 ('2043-01-23 23:07:18.972','98b8ef31-4f65-2f8b-1ea7-b1473900099e') -[-2] -175477.0173 ('2007-01-16 04:46:14.781','ec92f616-6e1f-003a-54c6-c5f9118d2f1b') -[] 197663.3035 ('2046-06-30 14:04:56.788','fb3244a4-8af2-104f-2a6f-25a7b7b9a112') -[-24] -174299.4691 ('2058-02-23 11:50:58.839','d63ee868-fa93-bf8b-0264-8ebbceb13e3b') -[95,38] -65083.7371 ('2015-03-10 10:33:16.429','47bd199c-f99e-51ea-84e9-b65cce9d167c') -[91,110,72] 130908.9643 ('2036-03-16 12:17:53.679','0dd4ca31-1e09-d7e0-f3df-60cad3cfa805') -[] 208972.3779 ('2034-03-05 19:29:21.994','1069d77c-dfd2-912e-60b8-3c5b964f7e11') -[-32] 167938.505 ('2093-09-10 17:39:39.050','9d1025b6-2d0c-1d84-dafd-02668eb29270') -[] 153744.6987 ('2088-10-02 08:02:11.024','a88e6cb7-2210-5ce5-6bcf-24afc0eca5b6') -[67] -74220.665 ('2074-12-30 15:43:40.817','68096065-18c8-8aca-fd21-15330ead669d') -[6] 66759.8938 ('2091-09-01 16:07:18.219','bb14f4cc-0b54-9a8c-e835-71333b28c03b') -[-28,-82,9] 168625.3131 ('2002-03-20 18:02:30.321','405bb877-6e28-8b91-cb62-bd82a3fa797c') -[] -19760.167 ('2044-11-08 04:52:03.325','13769348-9e58-0e75-3972-8bbadc150715') -[] 160663.7797 ('2025-04-12 10:17:53.501','e6370321-94f5-97e6-0348-a84e72ff5b42') -[-17,18] 99105.9856 ('1972-05-01 09:23:11.688','02618b9e-97cd-4698-d2e8-3f52f4c5a09a') -[86,77] -116990.3914 ('1981-12-31 02:06:54.198','3ac42bb4-8652-b1a8-10bb-98f0337261f8') -[-109,69,-63] -151527.3587 ('2001-01-17 08:19:56.504','77fe7ee2-f279-2855-bfd2-a7d7cee678cc') -[] -57762.3928 ('1978-08-16 15:47:37.660','ab9a110a-fd8d-3c4c-5a49-34c2005536ce') -[-77] 107274.6407 ('2017-01-12 09:03:02.657','c1ad4f17-cc54-45f3-9410-9c1011653f6d') -[] 107133.641 ('2050-10-05 03:29:27.154','36e576aa-c77f-994e-1925-4a4c40da3a0f') -[] 46672.2176 ('2094-01-21 17:25:39.144','e9ba850d-604e-bc7d-417c-1078e89d4615') -[-87,-122,-65] -86258.4663 ('2081-06-17 00:37:45.498','64795221-9719-7937-b4d2-be5f30065ece') -[-53] -48672.1424 ('1992-06-27 13:27:23.602','7c67bc31-c7bb-6197-fdca-f73329b976f2') -[34] -108954.782 ('2096-07-03 20:06:30.632','9c1b37d7-4ced-9428-a0ae-34c5436b14c4') -[] -168124.2364 ('1987-06-03 02:47:12.945','d1c39af4-f920-5095-b8e2-0f878950167b') -[] -112431.4799 ('2021-07-26 04:04:58.527','da07a72d-7e1f-8890-4c4b-326835d11b39') -[-35,-95,58] -181254.9139 ('2086-11-12 14:17:14.473','22f74d0b-dfc0-3f7a-33f4-8055d8fa7846') -[98,119] 11468.5238 ('2092-02-25 08:07:07.695','a1fb97bf-1885-6715-c233-b88a6cd111e4') -[] 82333.8963 ('1989-11-22 22:38:57.012','a2b82b5b-8331-555c-579b-de4b0eeb7e81') -[-5,-66,69] 32055.8376 ('2040-12-17 13:49:08.704','4537d25e-a2db-ea9a-8e24-a16ed7e0c6e4') -[81,-84,-24] -210815.2512 ('2047-06-09 10:30:06.922','ac3c5b5f-f977-2830-c398-d10a6076a498') -[84,-105] -175413.7733 ('1998-11-03 01:30:21.191','c535feac-1943-c0a1-23f0-645d5406db24') -[58,31] -335.8512 ('1973-07-09 09:21:10.444','24a7dd3d-2565-1de3-05d9-e45fd8ba7729') -[-49,-47] 177399.2836 ('2049-03-15 12:33:00.190','e4432b9b-61e9-d451-dc87-ae3b9da6fd35') -[] 211525.2349 ('2106-01-11 07:44:18.918','23315435-7132-05b5-5a9b-c2c738433a87') -[45,-95,-39] -15314.9732 ('2055-10-29 10:51:12.182','833b2efa-8c72-f5f6-3040-cb4831e8ceb9') -[] 213384.5774 ('2067-02-10 19:02:42.113','0cd7f438-caa7-0d21-867c-1fdb6d67d797') -[99] -147316.5599 ('2000-05-09 17:37:34.776','a3ea6796-38d5-72ff-910d-8b4300831916') -[] 8828.2471 ('1993-11-30 13:53:22.503','7209213f-38bb-cfed-1955-f1fad5a9577a') -[117,9,-35] -134812.6269 ('2065-09-04 20:47:26.589','d33d0d6f-b9c0-2850-4593-cfc9f1e20a4d') -[-35,-58,-101] -9101.5369 ('2023-08-24 17:56:11.695','87fbe3f9-b1f0-c030-a4c0-8662045923b4') -[-58,87] 122510.9099 ('2019-08-09 14:40:29.849','c1d3a2cc-878f-c2c3-4a0b-10e98cda8b4a') -[4,19,58] -13496.8672 ('2027-05-01 06:11:48.659','8996ae31-d670-cbfe-b735-b16b7c3b3476') -[23,-75,-89] -51218.286 ('2010-06-01 22:49:03.396','d32b8b61-cc3e-31fa-2a2a-abefa60bfcee') -[50] -45297.4315 ('2087-04-15 03:46:08.247','04fe9603-97fc-07a4-6248-0f21e408c884') -[-23,17,63] 89185.9462 ('2065-10-26 05:27:12.817','a5fbf764-70b4-8b65-4a8f-7550abca3859') -[-6] -129925.369 ('2013-11-05 03:44:45.233','11db26b3-e2b5-b9fa-6b0e-79c43a2e67ab') -[-72,-108] 203171.5475 ('2000-01-28 06:34:58.032','14d5399e-7949-20c7-0e47-85e2fce5836c') -[-73,34,-27] 2676.7265 ('2057-10-25 11:37:10.049','00049a92-4350-badb-3764-dd7f019b9b31') -[65,-7] -153472.9461 ('1973-04-11 23:34:41.245','e0a0324d-1552-d11e-f3a5-fbd822d206c5') -[] 81837.7838 ('2041-09-20 17:56:39.712','f7923f2c-e526-1706-79b9-58045d9deaa7') -[-113,8] 173192.6905 ('2066-04-02 06:59:59.356','e3013e5c-92e3-c03c-b57a-e1939e00a1a7') -[107] 9694.1102 ('1984-11-02 10:11:34.034','e973db18-07b7-2117-f3ba-e7002adfa939') -[] -76460.9664 ('2051-02-10 06:54:42.143','b8344c22-9e8a-7052-c644-9c3e5989cdf1') -[59,59,0] 27041.7606 ('2083-02-17 15:21:22.547','4d6b137b-a3e1-f36d-2c0c-c8d718dda388') -[-114] 133673.963 ('2005-10-02 16:34:27.452','04785b75-30e5-af8b-547e-d15bcb7f49fb') -[43] -169861.2 ('2006-12-13 06:26:13.923','cb865d38-d961-d7f9-acbb-583b9f31252f') -[] 197115.2174 ('2060-04-08 01:17:00.488','0f26c4b4-b24c-1fd5-c619-31bcf71a4831') -[-25] -200081.9506 ('2055-12-24 23:30:16.276','0b32ad69-2c84-4269-9718-e3171482878a') -[14,110] -40196.4463 ('2084-08-13 16:37:07.588','ed882071-acba-b3ab-5d77-d79a9544a834') -[-62,-71,-82] -154958.9747 ('2100-07-07 23:32:53.741','7711c7c1-0d22-e302-fc86-61ef5e68db96') -[96,-114,-101] 78910.332 ('2100-07-19 12:02:27.109','756bfd26-c4b3-94b8-e991-c7ab7a833b76') -[49] 80117.2267 ('1970-07-04 00:50:56.748','aebac019-9054-4a77-2ccd-8801fc4a7496') -[] 102078.4801 ('2055-01-06 22:22:33.624','21f2e59a-a1ca-5df3-27fd-aa95456cfbe5') -[-106] -108728.4237 ('2020-05-27 08:56:18.121','6b7b6674-9342-2360-4cc0-f7ef8a2404de') -[] 173213.5631 ('2034-01-18 16:04:16.059','2dc0038d-67c1-f0ee-280b-f3f0f536b01a') -[42] 139872.2503 ('2001-07-16 07:09:28.754','d6487da6-1077-1053-f314-9a1079f5df15') -[] 1107.5244 ('2031-02-26 12:06:00.846','b32bee8f-85b7-3c71-bb24-9a0093e6a08c') -[] 85892.8913 ('2088-04-13 11:54:18.514','84f3b59b-8d23-78a6-3032-91392344584f') -[43] -109644.2714 ('1974-07-04 11:45:43.139','cf722ca8-15f5-6fe2-997c-0cf88e95e902') -[] 212557.3762 ('2069-03-03 04:21:08.439','9e676cac-36e6-2962-f7b1-578214f0dfbd') -[-128,55] 80471.0777 ('1970-04-01 15:54:40.257','ca358854-416b-9c95-0b9b-c7fed7bb7cb5') -[-30,-54] -132205.4512 ('2017-12-15 19:54:15.750','3558faa4-2d2f-c533-437f-1e03d3600f1d') -[-116,-72] -91499.667 ('2105-09-23 18:06:17.755','07bb6e47-3234-c268-40d7-332388dc06f8') -[] -201636.5228 ('2085-01-27 04:54:42.717','86c3bdc3-ff0f-1723-07c2-845aa3c02370') -[-103,-39] 44330.7722 ('2064-07-02 08:08:28.068','0869c79d-6bdd-5d2d-a3d1-ffe13f6aa810') -[99] -31035.5391 ('2093-07-25 22:50:23.026','aeb59338-254f-dc09-fbd7-263da415e211') -[101] 157961.4729 ('2036-05-03 23:35:07.845','8b6221a9-8dad-4655-7460-6b3031b06893') -[111] 84732.4403 ('1997-04-06 12:10:18.624','08806a79-59f4-c833-eedc-a200bb851767') -[9,-48] -190491.559 ('2031-11-03 16:47:03.757','914e6166-c96e-e0e4-101a-0bb516cf5a2f') -[-41] -132501.8311 ('2089-11-21 18:38:28.848','6de6cc8d-3c49-641e-fb12-87ed5ecb97b0') -[77] 64903.6579 ('1985-04-17 13:08:03.998','26484b8a-f3f1-587f-7777-bc7a57a689c3') +[] -54259.6828 ('1973-07-09 09:21:10.444','9d1025b6-2d0c-1d84-dafd-02668eb29270') +[88] 21968.7945 ('2049-03-15 12:33:00.190','a88e6cb7-2210-5ce5-6bcf-24afc0eca5b6') +[-1] -2668.69 ('2106-01-11 07:44:18.918','68096065-18c8-8aca-fd21-15330ead669d') +[-103,75] -75192.4989 ('2055-10-29 10:51:12.182','bb14f4cc-0b54-9a8c-e835-71333b28c03b') +[110] 97330.6359 ('2067-02-10 19:02:42.113','405bb877-6e28-8b91-cb62-bd82a3fa797c') +[-22,2] 85107.1979 ('2000-05-09 17:37:34.776','13769348-9e58-0e75-3972-8bbadc150715') +[-22,-62] -3577.5379 ('1993-11-30 13:53:22.503','e6370321-94f5-97e6-0348-a84e72ff5b42') +[-2,-90] 70968.4171 ('2065-09-04 20:47:26.589','02618b9e-97cd-4698-d2e8-3f52f4c5a09a') +[-94,80] 63834.8685 ('2023-08-24 17:56:11.695','3ac42bb4-8652-b1a8-10bb-98f0337261f8') +[23] 63817.7977 ('2019-08-09 14:40:29.849','77fe7ee2-f279-2855-bfd2-a7d7cee678cc') +[46,-10,-63] 97663.3035 ('2027-05-01 06:11:48.659','ab9a110a-fd8d-3c4c-5a49-34c2005536ce') +[-107,68] -65083.7371 ('2010-06-01 22:49:03.396','c1ad4f17-cc54-45f3-9410-9c1011653f6d') +[-35,-116,73] 8972.3779 ('2087-04-15 03:46:08.247','36e576aa-c77f-994e-1925-4a4c40da3a0f') +[61] 53744.6987 ('2065-10-26 05:27:12.817','e9ba850d-604e-bc7d-417c-1078e89d4615') +[75] 66759.8938 ('2013-11-05 03:44:45.233','64795221-9719-7937-b4d2-be5f30065ece') +[-115,93] -19760.167 ('2000-01-28 06:34:58.032','7c67bc31-c7bb-6197-fdca-f73329b976f2') +[-20,4,21] 99105.9856 ('2057-10-25 11:37:10.049','9c1b37d7-4ced-9428-a0ae-34c5436b14c4') +[-110,117,91] -51527.3587 ('1973-04-11 23:34:41.245','d1c39af4-f920-5095-b8e2-0f878950167b') +[62] 7274.6407 ('2041-09-20 17:56:39.712','da07a72d-7e1f-8890-4c4b-326835d11b39') +[-2] 46672.2176 ('2066-04-02 06:59:59.356','22f74d0b-dfc0-3f7a-33f4-8055d8fa7846') +[] -48672.1424 ('1984-11-02 10:11:34.034','a1fb97bf-1885-6715-c233-b88a6cd111e4') +[-24] -68124.2364 ('2051-02-10 06:54:42.143','a2b82b5b-8331-555c-579b-de4b0eeb7e81') +[95,38] -81254.9139 ('2083-02-17 15:21:22.547','4537d25e-a2db-ea9a-8e24-a16ed7e0c6e4') +[91,110,72] 82333.8963 ('2005-10-02 16:34:27.452','ac3c5b5f-f977-2830-c398-d10a6076a498') +[] -10815.2512 ('2006-12-13 06:26:13.923','c535feac-1943-c0a1-23f0-645d5406db24') +[-32] -335.8512 ('2060-04-08 01:17:00.488','24a7dd3d-2565-1de3-05d9-e45fd8ba7729') +[] 11525.2349 ('2055-12-24 23:30:16.276','e4432b9b-61e9-d451-dc87-ae3b9da6fd35') +[67] 13384.5774 ('2084-08-13 16:37:07.588','23315435-7132-05b5-5a9b-c2c738433a87') +[6] 8828.2471 ('2100-07-07 23:32:53.741','833b2efa-8c72-f5f6-3040-cb4831e8ceb9') +[-28,-82,9] -9101.5369 ('2100-07-19 12:02:27.109','0cd7f438-caa7-0d21-867c-1fdb6d67d797') +[] -13496.8672 ('1970-07-04 00:50:56.748','a3ea6796-38d5-72ff-910d-8b4300831916') +[] -45297.4315 ('2055-01-06 22:22:33.624','7209213f-38bb-cfed-1955-f1fad5a9577a') +[-17,18] -29925.369 ('2020-05-27 08:56:18.121','d33d0d6f-b9c0-2850-4593-cfc9f1e20a4d') +[86,77] 2676.7265 ('2034-01-18 16:04:16.059','87fbe3f9-b1f0-c030-a4c0-8662045923b4') +[-109,69,-63] 81837.7838 ('2001-07-16 07:09:28.754','c1d3a2cc-878f-c2c3-4a0b-10e98cda8b4a') +[] 9694.1102 ('2031-02-26 12:06:00.846','8996ae31-d670-cbfe-b735-b16b7c3b3476') +[-77] 27041.7606 ('2088-04-13 11:54:18.514','d32b8b61-cc3e-31fa-2a2a-abefa60bfcee') +[] -69861.2 ('1974-07-04 11:45:43.139','04fe9603-97fc-07a4-6248-0f21e408c884') +[] -81.9506 ('2069-03-03 04:21:08.439','a5fbf764-70b4-8b65-4a8f-7550abca3859') +[-87,-122,-65] -54958.9747 ('1970-04-01 15:54:40.257','11db26b3-e2b5-b9fa-6b0e-79c43a2e67ab') +[-53] 80117.2267 ('2017-12-15 19:54:15.750','14d5399e-7949-20c7-0e47-85e2fce5836c') +[34] -8728.4237 ('2105-09-23 18:06:17.755','00049a92-4350-badb-3764-dd7f019b9b31') +[] 39872.2503 ('2085-01-27 04:54:42.717','e0a0324d-1552-d11e-f3a5-fbd822d206c5') +[] 85892.8913 ('2064-07-02 08:08:28.068','f7923f2c-e526-1706-79b9-58045d9deaa7') +[-35,-95,58] 12557.3762 ('2093-07-25 22:50:23.026','e3013e5c-92e3-c03c-b57a-e1939e00a1a7') +[98,119] -32205.4512 ('2036-05-03 23:35:07.845','e973db18-07b7-2117-f3ba-e7002adfa939') +[] -1636.5228 ('1997-04-06 12:10:18.624','b8344c22-9e8a-7052-c644-9c3e5989cdf1') +[-5,-66,69] -31035.5391 ('2031-11-03 16:47:03.757','4d6b137b-a3e1-f36d-2c0c-c8d718dda388') +[81,-84,-24] 84732.4403 ('2089-11-21 18:38:28.848','04785b75-30e5-af8b-547e-d15bcb7f49fb') +[84,-105] -32501.8311 ('1985-04-17 13:08:03.998','cb865d38-d961-d7f9-acbb-583b9f31252f') +[58,31] 95437.1166 ('1977-06-10 06:40:13.164','0f26c4b4-b24c-1fd5-c619-31bcf71a4831') +[-49,-47] 32232.0218 ('2006-10-31 16:14:17.332','0b32ad69-2c84-4269-9718-e3171482878a') +[] -8085.7941 ('1995-02-28 19:24:55.774','ed882071-acba-b3ab-5d77-d79a9544a834') +[45,-95,-39] -15528.0377 ('2090-01-30 21:38:09.084','7711c7c1-0d22-e302-fc86-61ef5e68db96') +[] -26490.6032 ('2064-06-06 03:33:21.798','756bfd26-c4b3-94b8-e991-c7ab7a833b76') +[99] 35137.0312 ('2029-05-20 02:21:06.599','aebac019-9054-4a77-2ccd-8801fc4a7496') +[] -35249.6041 ('2103-07-16 18:29:01.847','21f2e59a-a1ca-5df3-27fd-aa95456cfbe5') +[117,9,-35] -20390.4167 ('2051-04-04 01:10:56.314','6b7b6674-9342-2360-4cc0-f7ef8a2404de') +[-35,-58,-101] -86622.5332 ('2056-01-31 05:30:53.382','2dc0038d-67c1-f0ee-280b-f3f0f536b01a') +[-58,87] -38301.9728 ('1993-06-09 12:46:54.102','d6487da6-1077-1053-f314-9a1079f5df15') +[4,19,58] -19098.0416 ('2046-03-01 07:58:38.749','b32bee8f-85b7-3c71-bb24-9a0093e6a08c') +[23,-75,-89] -57396.8139 ('1994-03-07 15:06:46.346','84f3b59b-8d23-78a6-3032-91392344584f') +[50] 15536.4639 ('2101-06-20 20:16:49.230','cf722ca8-15f5-6fe2-997c-0cf88e95e902') +[-23,17,63] 6790.5373 ('2033-08-28 21:52:52.011','9e676cac-36e6-2962-f7b1-578214f0dfbd') +[-6] 55706.5159 ('2073-12-23 07:06:46.299','ca358854-416b-9c95-0b9b-c7fed7bb7cb5') +[-72,-108] -86272.6801 ('2011-10-08 23:52:57.079','3558faa4-2d2f-c533-437f-1e03d3600f1d') +[-73,34,-27] 99270.5967 ('2100-01-02 21:12:36.759','07bb6e47-3234-c268-40d7-332388dc06f8') +[65,-7] -47051.3089 ('2045-02-03 08:35:08.678','86c3bdc3-ff0f-1723-07c2-845aa3c02370') +[] -73019.898 ('2031-11-03 03:51:00.078','0869c79d-6bdd-5d2d-a3d1-ffe13f6aa810') +[-113,8] 69167.0269 ('2100-08-12 02:10:36.637','aeb59338-254f-dc09-fbd7-263da415e211') +[107] 20867.6436 ('2032-05-30 20:42:14.359','8b6221a9-8dad-4655-7460-6b3031b06893') +[] -44814.8969 ('2014-10-07 06:52:22.047','08806a79-59f4-c833-eedc-a200bb851767') +[59,59,0] 5445.4157 ('2089-11-28 09:02:07.006','914e6166-c96e-e0e4-101a-0bb516cf5a2f') +[-114] 92856.7823 ('2028-11-13 18:19:16.001','6de6cc8d-3c49-641e-fb12-87ed5ecb97b0') +[43] -2480.1718 ('2102-05-21 16:10:28.701','26484b8a-f3f1-587f-7777-bc7a57a689c3') +[] -68249.3014 ('2030-03-10 00:24:03.129','d2625c6e-dd01-7e12-e794-6601d4633dfc') +[-25] -96391.8384 ('2079-01-14 14:29:39.356','125c0ffe-0cb6-c0ef-e4fe-cfa824d91cfc') +[14,110] 99043.5601 ('2102-02-06 00:40:42.723','0bcc2764-a09e-7707-df0d-ba8de2228488') +[-62,-71,-82] -19952.4405 ('2003-12-28 16:25:30.841','b91d069b-8866-f2a5-37a6-d476cafcb331') +[96,-114,-101] -25989.8831 ('2026-05-13 15:46:53.209','cbbd4ea9-a80c-642f-2650-977c8e56adff') +[49] -3848.9475 ('2102-06-09 05:38:10.156','097a67d1-a39a-b133-03ab-304e954bcd95') +[] 69635.1325 ('2070-06-11 21:48:00.549','b9c6c1eb-2ee3-c50d-fb0f-05c7e06c77ce') +[-106] 54574.7656 ('2087-08-21 07:15:19.514','b6cc520e-a251-0027-6f4a-26b3a8fc47d1') +[] 65529.215 ('2011-07-24 06:41:14.268','bd1b3d45-44c7-f3c3-b521-5a42e095c66e') +[42] 46705.3016 ('2042-09-26 17:17:50.524','d3a14a33-e1c6-9c99-66cf-dd8a6b03bbfe') +[] 54751.8012 ('2048-12-26 02:02:23.706','ff698130-c509-0e1b-0a85-3989d6699004') +[] -84690.1759 ('2063-10-25 19:01:38.315','8040b7c2-ef8f-b180-927f-4745e7a106c3') +[43] 63776.0626 ('2021-01-25 10:03:59.774','0aaf8333-b1ea-4d85-33e1-8c715179c161') +[] -7703.4808 ('2001-05-12 17:53:23.514','047489f8-7294-e929-f98e-a2044d26ed22') +[-128,55] -23194.9926 ('2099-09-18 07:04:58.395','3035f0bf-bda6-9307-90f6-a34378b20d6c') +[-30,-54] 98203.9762 ('2050-02-06 14:51:29.743','ccc3d3fd-64dc-65a1-61e1-384e83e3d641') +[-116,-72] -13519.4648 ('1998-04-09 16:50:26.194','38810a09-9c5b-9a6f-4b4c-134aa551d6ba') +[] -6898.0943 ('2099-09-13 20:00:52.619','255e9952-300f-0153-3a7f-8865d1b6683c') +[-103,-39] -52030.6657 ('2047-10-28 16:47:28.315','385abde4-14d6-ed9e-bd01-e641dd0b5ed5') +[99] -42787.2791 ('1996-10-22 16:24:52.458','e522d1ea-ffae-14f6-d95c-14dfdaf2eb83') +[101] -11540.4113 ('1981-07-17 18:47:34.268','918f1eea-b5e7-b10b-96dd-47cdcd470a1a') +[111] 71231.1796 ('1997-05-16 14:19:48.064','28ae2849-7667-d36f-7010-fa020d71eb79') +[9,-48] 68698.7488 ('2078-03-06 02:00:34.565','3a37dc94-88d0-e885-8a2f-d37d135c5394') +[-41] -14738.4502 ('2092-08-10 16:54:45.294','a629c44c-14ac-c7de-c6bc-a5eaac0063ed') +[77] -47644.5561 ('2080-06-16 05:11:09.073','a46a60e2-c8ac-e8de-4b10-2a653c66c751') - diff --git a/tests/queries/0_stateless/01087_table_function_generate.reference b/tests/queries/0_stateless/01087_table_function_generate.reference index 53792bfb579..d6744ba0b33 100644 --- a/tests/queries/0_stateless/01087_table_function_generate.reference +++ b/tests/queries/0_stateless/01087_table_function_generate.reference @@ -94,16 +94,16 @@ Float32 Float64 7.317837e-36 -1.6511853645079817e-21 - Decimal(9, 4) Decimal(18, 8) Decimal(18, 8) --18731.5032 81241713112.39967992 -10576027963457111164764.0798899532879521 -65289.5061 -27889310937.24180887 5807515838469365530027.7612329616030438 --197586.1517 -751754543.85331084 3835903211857734974086.0358362773591932 -183596.0063 8217353434.4196403 13633006218585943284268.9826084812209912 -73041.2674 -88881500366.49430454 -148702703925022894263.3187064158377476 -101454.4494 -27768337.71540858 -634829280961262229789.4961995996929358 --174012.0101 -13636289325.35403038 -3611949395160064991369.2765012316944096 -138203.8526 13556098030.08819271 134470734594381953531.9736002591779584 -15395.1766 -8047388876.97332962 16804394201271843589306.4234533639925009 -8569.7048 -49271659843.47126295 -14851374957489266092927.8687987539036841 +-18731.5032 -27768337.71540858 -9393543543230357843716.0041459374484681 +-97586.1517 -3636289325.35403038 3364376403318670133825.8224672630083466 +73041.2674 3556098030.08819271 -2446472555280036491886.6248666210495333 +-74012.0101 -8047388876.97332962 -1097064725002692978976.3763759594690971 +15395.1766 -9271659843.47126295 5428073503721506689195.250326552245674 +-93143.0664 -1797039080.46100129 -8559656833301969702973.2550203153949345 +46848.5225 -7332762471.23822513 6692780327859072322513.5175470507222506 +-85331.598 -7937795415.83578394 6508457918219676369805.8207979274851797 +-31470.8994 1482606444.06230976 -8371832329716119186316.4300878863977591 +32104.097 -9071727536.35797124 1491705856202199878279.4124322332399575 - Tuple(Int32, Int64) (-187315032,8124171311239967992) @@ -225,25 +225,25 @@ U6 \'%Y~t9 RL,{Xs\\tw - -[] -27467.1221 ('2021-03-08 00:39:14.331','08ec773f-cded-8c46-727f-954768082cbf') -[] 204013.7193 ('2026-05-05 02:20:23.160','30f6d580-cb25-8d4f-f869-fc10128b3389') -[-122] -9432.2617 ('2001-08-23 04:05:41.222','f7bf2154-78c3-8920-e4d3-a374e22998a4') -[-30,61] -133488.2399 ('2048-05-14 06:05:06.021','a6af106c-b321-978b-fa79-338c9e342b5a') -[-1] 58720.0591 ('1976-06-07 20:26:18.162','fc038af0-ba31-8fdc-1847-37328ef161b0') -[1] -18736.7874 ('1977-03-10 01:41:16.215','3259d377-a92d-3557-9045-4ad1294d55d5') -[34,-10] -99367.9009 ('2031-05-08 07:00:41.084','0b38ebc5-20a6-be3d-8543-23ce3546f49c') -[110] 31562.7502 ('2045-02-27 08:46:14.976','74116384-cb3e-eb00-0102-fb30ddea5d5f') -[114] -84125.1554 ('2023-06-06 03:55:06.492','bf9ab359-ef9f-ad11-7e6c-160368b1e5ea') -[124] -114719.5228 ('2010-11-11 19:57:23.722','c1046ffb-3415-cc3a-509a-e0005856d7d7') +[] -89844.0836 ('2057-10-03 21:07:16.864','fa79338c-9e34-2b5a-64b7-ab28e4f8c281') +[] 58720.0591 ('2048-05-14 06:05:06.021','854323ce-3546-f49c-08ec-773fcded8c46') +[-122] -9432.2617 ('2010-11-11 19:57:23.722','7e6c1603-68b1-e5ea-3259-d377a92d3557') +[-30,61] -16883.7384 ('2067-03-02 22:51:41.201','8dab5bc5-a641-5a33-7a35-02d6b3af106f') +[-1] -82719.0473 ('2050-09-09 13:42:43.295','138fe3b2-602c-4249-f2dc-175ea47e2429') +[1] -99367.9009 ('1976-06-07 20:26:18.162','509ae000-5856-d7d7-0b38-ebc520a6be3d') +[34,-10] 97602.7584 ('1995-09-16 10:43:48.516','18473732-8ef1-61b0-a6af-106cb321978b') +[110] -84125.1554 ('2031-05-08 07:00:41.084','90454ad1-294d-55d5-30f6-d580cb258d4f') +[114] 4013.7193 ('2021-03-08 00:39:14.331','f869fc10-128b-3389-c104-6ffb3415cc3a') +[124] 10712.855 ('2005-06-14 02:52:57.504','727f9547-6808-2cbf-fc03-8af0ba318fdc') - -[] 1900051923 { -189530.5846 h -5.6279699579452485e47 ('1984-12-06','2028-08-17 03:05:01','2036-04-02 20:52:28.468','4b3d498c-dd44-95c1-5b75-921504ec5d8d') F743 -[-102,-118] 392272782 Eb -14818.02 o -2.664492247169164e59 ('2082-12-26','2052-09-09 03:50:50','2088-04-21 02:07:08.245','aeb9c26e-0ee7-2b8e-802b-2a96319b8e60') CBF4 -[-71] 775049089 \N -158115.1178 w 4.1323844687113747e-305 ('2108-04-19','2090-07-31 13:45:26','2076-07-10 06:11:06.385','57c69bc6-dddd-0975-e932-a7b5173a1304') EB1D -[-28,100] 3675466147 { -146685.1749 h 3.6676044396877755e142 ('2017-10-25','2100-02-28 15:07:18','2055-10-14 03:36:20.056','14949dae-dfa8-a124-af83-887348b2f609') 6D88 -[-23] 2514120753 (`u, -119659.6174 w 1.3231258347475906e34 ('2141-04-06','2074-08-10 03:25:12','1976-12-04 15:31:55.745','86a9b3c1-4593-4d56-7762-3aa1dd22cbbf') AD43 -[11,-36] 3308237300 \N 171205.1896 \N 5.634708707075817e195 ('1974-10-31','1993-12-24 06:38:45','2038-07-15 02:22:51.805','63d999b8-8cca-e237-c4a4-4dd7d0096f65') 609E -[39] 1614362420 `4A8P 157144.063 o -1.1843143253872814e-255 ('2147-08-18','2072-09-28 15:27:27','2073-07-10 09:19:58.146','6483f5c0-8733-364c-4fa0-9948d32e8903') A886 -[48,-120] 3848918261 1 Date: Wed, 5 Apr 2023 18:08:56 +0000 Subject: [PATCH 031/103] Add trailing commas to expression list in SELECT query --- src/Parsers/ExpressionListParsers.cpp | 70 +++++++++++++++---- src/Parsers/ExpressionListParsers.h | 18 +++-- src/Parsers/ParserSelectQuery.cpp | 2 +- src/Parsers/ParserTablesInSelectQuery.cpp | 1 - .../02676_trailing_commas.reference | 4 ++ .../0_stateless/02676_trailing_commas.sql | 4 ++ 6 files changed, 77 insertions(+), 22 deletions(-) create mode 100644 tests/queries/0_stateless/02676_trailing_commas.reference create mode 100644 tests/queries/0_stateless/02676_trailing_commas.sql diff --git a/src/Parsers/ExpressionListParsers.cpp b/src/Parsers/ExpressionListParsers.cpp index 054a22a0c3a..6008e89d038 100644 --- a/src/Parsers/ExpressionListParsers.cpp +++ b/src/Parsers/ExpressionListParsers.cpp @@ -303,9 +303,9 @@ ASTPtr makeBetweenOperator(bool negative, ASTs arguments) } } -ParserExpressionWithOptionalAlias::ParserExpressionWithOptionalAlias(bool allow_alias_without_as_keyword, bool is_table_function) +ParserExpressionWithOptionalAlias::ParserExpressionWithOptionalAlias(bool allow_alias_without_as_keyword, bool is_table_function, bool allow_trailing_commas) : impl(std::make_unique( - is_table_function ? ParserPtr(std::make_unique()) : ParserPtr(std::make_unique()), + is_table_function ? ParserPtr(std::make_unique()) : ParserPtr(std::make_unique(allow_trailing_commas)), allow_alias_without_as_keyword)) { } @@ -314,7 +314,7 @@ ParserExpressionWithOptionalAlias::ParserExpressionWithOptionalAlias(bool allow_ bool ParserExpressionList::parseImpl(Pos & pos, ASTPtr & node, Expected & expected) { return ParserList( - std::make_unique(allow_alias_without_as_keyword, is_table_function), + std::make_unique(allow_alias_without_as_keyword, is_table_function, allow_trailing_commas), std::make_unique(TokenType::Comma)) .parse(pos, node, expected); } @@ -783,9 +783,11 @@ class ExpressionLayer : public Layer { public: - explicit ExpressionLayer(bool is_table_function_) : Layer(false, false) + explicit ExpressionLayer(bool is_table_function_, bool allow_trailing_commas_ = false) + : Layer(false, false) { is_table_function = is_table_function_; + allow_trailing_commas = allow_trailing_commas_; } bool getResult(ASTPtr & node) override @@ -799,13 +801,52 @@ public: return Layer::getResultImpl(node); } - bool parse(IParser::Pos & pos, Expected & /*expected*/, Action & /*action*/) override + bool parse(IParser::Pos & pos, Expected & expected, Action & /*action*/) override { if (pos->type == TokenType::Comma) + { finished = true; + if (!allow_trailing_commas) + return true; + + /// We support trailing commas at the end of the column declaration: + /// - SELECT a, b, c, FROM table + /// - SELECT 1, + + /// For this purpose we eliminate the following cases: + /// 1. WITH 1 AS from SELECT 2, from + /// 2. SELECT to, from FROM table + /// 3. SELECT to, from AS alias FROM table + /// 4. SELECT to, from + to FROM table + + auto test_pos = pos; + ++test_pos; + + if (test_pos.isValid() && test_pos->type != TokenType::Semicolon) + { + if (!ParserKeyword("FROM").ignore(test_pos, expected)) + return true; + + if (ParserKeyword("FROM").ignore(test_pos, expected)) + return true; + + if (ParserAlias(false).ignore(test_pos, expected)) + return true; + + if (!ParserIdentifier(true).ignore(test_pos, expected)) + return true; + } + + ++pos; + return true; + } + return true; } + +private: + bool allow_trailing_commas; }; /// Basic layer for a function with certain separator and end tokens: @@ -2201,7 +2242,7 @@ struct ParserExpressionImpl bool ParserExpression::parseImpl(Pos & pos, ASTPtr & node, Expected & expected) { - auto start = std::make_unique(false); + auto start = std::make_unique(false, allow_trailing_commas); return ParserExpressionImpl().parse(std::move(start), pos, node, expected); } @@ -2543,18 +2584,17 @@ Action ParserExpressionImpl::tryParseOperator(Layers & layers, IParser::Pos & po if (cur_op == operators_table.end()) { + if (!layers.back()->allow_alias || layers.back()->parsed_alias) + return Action::NONE; + ASTPtr alias; ParserAlias alias_parser(layers.back()->allow_alias_without_as_keyword); - if (layers.back()->allow_alias && - !layers.back()->parsed_alias && - alias_parser.parse(pos, alias, expected) && - layers.back()->insertAlias(alias)) - { - layers.back()->parsed_alias = true; - return Action::OPERATOR; - } - return Action::NONE; + if (!alias_parser.parse(pos, alias, expected) || !layers.back()->insertAlias(alias)) + return Action::NONE; + + layers.back()->parsed_alias = true; + return Action::OPERATOR; } auto op = cur_op->second; diff --git a/src/Parsers/ExpressionListParsers.h b/src/Parsers/ExpressionListParsers.h index 653654e5a33..040a81bbe27 100644 --- a/src/Parsers/ExpressionListParsers.h +++ b/src/Parsers/ExpressionListParsers.h @@ -172,10 +172,15 @@ protected: class ParserExpression : public IParserBase { +public: + ParserExpression(bool allow_trailing_commas_ = false) : allow_trailing_commas(allow_trailing_commas_) {} + protected: const char * getName() const override { return "lambda expression"; } bool parseImpl(Pos & pos, ASTPtr & node, Expected & expected) override; + + bool allow_trailing_commas; }; @@ -192,7 +197,7 @@ protected: class ParserExpressionWithOptionalAlias : public IParserBase { public: - explicit ParserExpressionWithOptionalAlias(bool allow_alias_without_as_keyword_, bool is_table_function_ = false); + explicit ParserExpressionWithOptionalAlias(bool allow_alias_without_as_keyword_, bool is_table_function_ = false, bool allow_trailing_commas_ = false); protected: ParserPtr impl; @@ -209,12 +214,15 @@ protected: class ParserExpressionList : public IParserBase { public: - explicit ParserExpressionList(bool allow_alias_without_as_keyword_, bool is_table_function_ = false) - : allow_alias_without_as_keyword(allow_alias_without_as_keyword_), is_table_function(is_table_function_) {} + explicit ParserExpressionList(bool allow_alias_without_as_keyword_, bool is_table_function_ = false, bool allow_trailing_commas_ = false) + : allow_alias_without_as_keyword(allow_alias_without_as_keyword_) + , is_table_function(is_table_function_) + , allow_trailing_commas(allow_trailing_commas_) {} protected: bool allow_alias_without_as_keyword; bool is_table_function; // This expression list is used by a table function + bool allow_trailing_commas; const char * getName() const override { return "list of expressions"; } bool parseImpl(Pos & pos, ASTPtr & node, Expected & expected) override; @@ -224,8 +232,8 @@ protected: class ParserNotEmptyExpressionList : public IParserBase { public: - explicit ParserNotEmptyExpressionList(bool allow_alias_without_as_keyword) - : nested_parser(allow_alias_without_as_keyword) {} + explicit ParserNotEmptyExpressionList(bool allow_alias_without_as_keyword_, bool allow_trailing_commas_ = false) + : nested_parser(allow_alias_without_as_keyword_, false, allow_trailing_commas_) {} private: ParserExpressionList nested_parser; protected: diff --git a/src/Parsers/ParserSelectQuery.cpp b/src/Parsers/ParserSelectQuery.cpp index 17b082a2ddb..1c48f773823 100644 --- a/src/Parsers/ParserSelectQuery.cpp +++ b/src/Parsers/ParserSelectQuery.cpp @@ -68,7 +68,7 @@ bool ParserSelectQuery::parseImpl(Pos & pos, ASTPtr & node, Expected & expected) ParserNotEmptyExpressionList exp_list(false); ParserNotEmptyExpressionList exp_list_for_with_clause(false); - ParserNotEmptyExpressionList exp_list_for_select_clause(true); /// Allows aliases without AS keyword. + ParserNotEmptyExpressionList exp_list_for_select_clause(/*allow_alias_without_as_keyword*/ true, /*allow_trailing_commas*/ true); ParserExpressionWithOptionalAlias exp_elem(false); ParserOrderByExpressionList order_list; ParserGroupingSetsExpressionList grouping_sets_list; diff --git a/src/Parsers/ParserTablesInSelectQuery.cpp b/src/Parsers/ParserTablesInSelectQuery.cpp index 617ab7816d4..08b6f77fafa 100644 --- a/src/Parsers/ParserTablesInSelectQuery.cpp +++ b/src/Parsers/ParserTablesInSelectQuery.cpp @@ -225,7 +225,6 @@ bool ParserTablesInSelectQueryElement::parseImpl(Pos & pos, ASTPtr & node, Expec } else if (ParserKeyword("ON").ignore(pos, expected)) { - /// OR is operator with lowest priority, so start parsing from it. if (!ParserExpression().parse(pos, table_join->on_expression, expected)) return false; } diff --git a/tests/queries/0_stateless/02676_trailing_commas.reference b/tests/queries/0_stateless/02676_trailing_commas.reference new file mode 100644 index 00000000000..41ace3e47aa --- /dev/null +++ b/tests/queries/0_stateless/02676_trailing_commas.reference @@ -0,0 +1,4 @@ +1 +1 +1 +1 2 diff --git a/tests/queries/0_stateless/02676_trailing_commas.sql b/tests/queries/0_stateless/02676_trailing_commas.sql new file mode 100644 index 00000000000..5e2dafccb46 --- /dev/null +++ b/tests/queries/0_stateless/02676_trailing_commas.sql @@ -0,0 +1,4 @@ +SELECT 1,; +SELECT 1, FROM numbers(1); +WITH 1 as a SELECT a, FROM numbers(1); +WITH 1 as from SELECT from, from + from, FROM numbers(1); From cea820ffbaf8c8abfa3be5cce16d5e3a4f3ebe30 Mon Sep 17 00:00:00 2001 From: Denny Crane Date: Wed, 5 Apr 2023 16:10:54 -0300 Subject: [PATCH 032/103] test #16399 --- .../00700_to_decimal_or_something_1.sql.j2 | 32 +++++ ...00_to_decimal_or_something_1.sql.reference | 120 ++++++++++++++++++ 2 files changed, 152 insertions(+) create mode 100644 tests/queries/0_stateless/00700_to_decimal_or_something_1.sql.j2 create mode 100644 tests/queries/0_stateless/00700_to_decimal_or_something_1.sql.reference diff --git a/tests/queries/0_stateless/00700_to_decimal_or_something_1.sql.j2 b/tests/queries/0_stateless/00700_to_decimal_or_something_1.sql.j2 new file mode 100644 index 00000000000..03ae16778bc --- /dev/null +++ b/tests/queries/0_stateless/00700_to_decimal_or_something_1.sql.j2 @@ -0,0 +1,32 @@ +{% for func in [ "toDecimal32", "toDecimal64", "toDecimal128", "toDecimal256" ] -%} + +SELECT '---- {{ func }} ----'; +SELECT {{ func }} ('54.1234567', 4); +SELECT {{ func }} ('1.1111111111111111111111111111111111111', 4); +SELECT {{ func }} ('x123', 4); -- { serverError CANNOT_PARSE_TEXT } +SELECT {{ func }} ('', 4); -- { serverError ATTEMPT_TO_READ_AFTER_EOF } +SELECT {{ func }} ('\0', 4); -- { serverError CANNOT_PARSE_TEXT } +SELECT {{ func }} ('\0\0\0\0\0', 4); -- { serverError CANNOT_PARSE_TEXT } +SELECT {{ func }} ('\n\t\r', 4); -- { serverError CANNOT_PARSE_TEXT } +SELECT {{ func }} ('\'', 4); -- { serverError CANNOT_PARSE_TEXT } + +{% endfor -%} + +{% for func in [ "toDecimal32OrDefault", "toDecimal32OrNull", "toDecimal32OrZero", + "toDecimal64OrDefault", "toDecimal64OrZero", "toDecimal64OrNull", + "toDecimal128OrDefault", "toDecimal128OrNull", "toDecimal128OrZero", + "toDecimal256OrDefault", "toDecimal256OrNull", "toDecimal256OrZero" ] -%} + +SELECT '---- {{ func }} ----'; +SELECT {{ func }} ('54.1234567', 4); +SELECT {{ func }} ('1.1111111111111111111111111111111111111', 4); +SELECT {{ func }} ('x123', 4); +SELECT {{ func }} ('', 4); +SELECT {{ func }} ('\0', 4); +SELECT {{ func }} ('\0\0\0\0\0', 4); +SELECT {{ func }} ('\n\t\r', 4); +SELECT {{ func }} ('\'', 4); + +{% endfor -%} + + diff --git a/tests/queries/0_stateless/00700_to_decimal_or_something_1.sql.reference b/tests/queries/0_stateless/00700_to_decimal_or_something_1.sql.reference new file mode 100644 index 00000000000..fdc7dafa24e --- /dev/null +++ b/tests/queries/0_stateless/00700_to_decimal_or_something_1.sql.reference @@ -0,0 +1,120 @@ +---- toDecimal32 ---- +54.1234 +1.1111 +---- toDecimal64 ---- +54.1234 +1.1111 +---- toDecimal128 ---- +54.1234 +1.1111 +---- toDecimal256 ---- +54.1234 +1.1111 +---- toDecimal32OrDefault ---- +54.1234 +1.1111 +0 +0 +0 +0 +0 +0 +---- toDecimal32OrNull ---- +54.1234 +1.1111 +\N +\N +\N +\N +\N +\N +---- toDecimal32OrZero ---- +54.1234 +1.1111 +0 +0 +0 +0 +0 +0 +---- toDecimal64OrDefault ---- +54.1234 +1.1111 +0 +0 +0 +0 +0 +0 +---- toDecimal64OrZero ---- +54.1234 +1.1111 +0 +0 +0 +0 +0 +0 +---- toDecimal64OrNull ---- +54.1234 +1.1111 +\N +\N +\N +\N +\N +\N +---- toDecimal128OrDefault ---- +54.1234 +1.1111 +0 +0 +0 +0 +0 +0 +---- toDecimal128OrNull ---- +54.1234 +1.1111 +\N +\N +\N +\N +\N +\N +---- toDecimal128OrZero ---- +54.1234 +1.1111 +0 +0 +0 +0 +0 +0 +---- toDecimal256OrDefault ---- +54.1234 +1.1111 +0 +0 +0 +0 +0 +0 +---- toDecimal256OrNull ---- +54.1234 +1.1111 +\N +\N +\N +\N +\N +\N +---- toDecimal256OrZero ---- +54.1234 +1.1111 +0 +0 +0 +0 +0 +0 From 4fb7e0ec3c13a05129c60565daae44cd7b4d197e Mon Sep 17 00:00:00 2001 From: Anton Popov Date: Wed, 5 Apr 2023 21:23:57 +0000 Subject: [PATCH 033/103] add retries to loading of data parts --- src/Storages/MergeTree/MergeTreeData.cpp | 51 +++++++++++++++++-- src/Storages/MergeTree/MergeTreeData.h | 10 ++++ .../configs/config.d/storage_conf.xml | 19 +++++++ .../test_merge_tree_s3_failover/test.py | 26 ++++++++++ 4 files changed, 102 insertions(+), 4 deletions(-) diff --git a/src/Storages/MergeTree/MergeTreeData.cpp b/src/Storages/MergeTree/MergeTreeData.cpp index 66c52e6e24c..678dd008ba7 100644 --- a/src/Storages/MergeTree/MergeTreeData.cpp +++ b/src/Storages/MergeTree/MergeTreeData.cpp @@ -76,6 +76,7 @@ #include #include +#include #include #include #include @@ -1154,6 +1155,10 @@ static bool isRetryableException(const Exception & e) return false; } +static constexpr size_t loading_parts_initial_backoff_ms = 100; +static constexpr size_t loading_parts_max_backoff_ms = 5000; +static constexpr size_t loading_parts_max_tries = 3; + MergeTreeData::LoadPartResult MergeTreeData::loadDataPart( const MergeTreePartInfo & part_info, const String & part_name, @@ -1322,6 +1327,37 @@ MergeTreeData::LoadPartResult MergeTreeData::loadDataPart( return res; } +MergeTreeData::LoadPartResult MergeTreeData::loadDataPartWithRetries( + const MergeTreePartInfo & part_info, + const String & part_name, + const DiskPtr & part_disk_ptr, + MergeTreeDataPartState to_state, + std::mutex & part_loading_mutex, + size_t initial_backoff_ms, + size_t max_backoff_ms, + size_t max_tries) +{ + for (size_t try_no = 0; try_no < max_tries; ++try_no) + { + try + { + return loadDataPart(part_info, part_name, part_disk_ptr, to_state, part_loading_mutex); + } + catch (const Exception & e) + { + if (!isRetryableException(e) || try_no + 1 == max_tries) + throw; + + LOG_DEBUG(log, "Failed to load data part {} at try {} with retryable error: {}. Will retry in {} ms", + part_name, try_no, e.message(), initial_backoff_ms); + + std::this_thread::sleep_for(std::chrono::milliseconds(initial_backoff_ms)); + initial_backoff_ms = std::min(initial_backoff_ms * 2, max_backoff_ms); + } + } + UNREACHABLE(); +} + std::vector MergeTreeData::loadDataPartsFromDisk( ThreadPool & pool, size_t num_parts, @@ -1436,10 +1472,14 @@ std::vector MergeTreeData::loadDataPartsFromDisk( /// Pass a separate mutex to guard the set of parts, because this lambda /// is called concurrently but with already locked @data_parts_mutex. - auto res = loadDataPart(thread_part->info, thread_part->name, thread_part->disk, DataPartState::Active, part_loading_mutex); - thread_part->is_loaded = true; + auto res = loadDataPartWithRetries( + thread_part->info, thread_part->name, thread_part->disk, + DataPartState::Active, part_loading_mutex, loading_parts_initial_backoff_ms, + loading_parts_max_backoff_ms, loading_parts_max_tries); + thread_part->is_loaded = true; bool is_active_part = res.part->getState() == DataPartState::Active; + /// If part is broken or duplicate or should be removed according to transaction /// and it has any covered parts then try to load them to replace this part. if (!is_active_part && !thread_part->children.empty()) @@ -1834,9 +1874,12 @@ try outdated_unloaded_data_parts.pop_back(); } - auto res = loadDataPart(part->info, part->name, part->disk, MergeTreeDataPartState::Outdated, data_parts_mutex); - ++num_loaded_parts; + auto res = loadDataPartWithRetries( + part->info, part->name, part->disk, + DataPartState::Outdated, data_parts_mutex, loading_parts_initial_backoff_ms, + loading_parts_max_backoff_ms, loading_parts_max_tries); + ++num_loaded_parts; if (res.is_broken) res.part->renameToDetached("broken-on-start"); /// detached parts must not have '_' in prefixes else if (res.part->is_duplicate) diff --git a/src/Storages/MergeTree/MergeTreeData.h b/src/Storages/MergeTree/MergeTreeData.h index 0be932ccdaf..f08c7dfc55a 100644 --- a/src/Storages/MergeTree/MergeTreeData.h +++ b/src/Storages/MergeTree/MergeTreeData.h @@ -1500,6 +1500,16 @@ private: MergeTreeDataPartState to_state, std::mutex & part_loading_mutex); + LoadPartResult loadDataPartWithRetries( + const MergeTreePartInfo & part_info, + const String & part_name, + const DiskPtr & part_disk_ptr, + MergeTreeDataPartState to_state, + std::mutex & part_loading_mutex, + size_t backoff_ms, + size_t max_backoff_ms, + size_t max_tries); + std::vector loadDataPartsFromDisk( ThreadPool & pool, size_t num_parts, diff --git a/tests/integration/test_merge_tree_s3_failover/configs/config.d/storage_conf.xml b/tests/integration/test_merge_tree_s3_failover/configs/config.d/storage_conf.xml index 32d78468a71..976933b2d21 100644 --- a/tests/integration/test_merge_tree_s3_failover/configs/config.d/storage_conf.xml +++ b/tests/integration/test_merge_tree_s3_failover/configs/config.d/storage_conf.xml @@ -21,6 +21,18 @@ true + + s3 + + http://resolver:8080/root/data/ + minio + minio123 + + true + + 1 + 1 + @@ -48,6 +60,13 @@ + + +
+ s3_no_retries +
+
+
diff --git a/tests/integration/test_merge_tree_s3_failover/test.py b/tests/integration/test_merge_tree_s3_failover/test.py index c61cacc9d8c..cf71b423713 100644 --- a/tests/integration/test_merge_tree_s3_failover/test.py +++ b/tests/integration/test_merge_tree_s3_failover/test.py @@ -270,3 +270,29 @@ def test_throttle_retry(cluster): ) == "42\n" ) + + +# Check that loading of parts is retried. +def test_retry_loading_parts(cluster): + node = cluster.instances["node"] + + node.query( + """ + CREATE TABLE s3_retry_loading_parts ( + id Int64 + ) ENGINE=MergeTree() + ORDER BY id + SETTINGS storage_policy='s3_no_retries' + """ + ) + + node.query("INSERT INTO s3_retry_loading_parts VALUES (42)") + node.query("DETACH TABLE s3_retry_loading_parts") + + fail_request(cluster, 5) + node.query("ATTACH TABLE s3_retry_loading_parts") + + assert node.contains_in_log( + "Failed to load data part all_1_1_0 at try 0 with retryable error" + ) + assert node.query("SELECT * FROM s3_retry_loading_parts") == "42\n" From dc19e3352c57d00278ac4e34d361de709e7528fa Mon Sep 17 00:00:00 2001 From: Denny Crane Date: Wed, 5 Apr 2023 20:32:35 -0300 Subject: [PATCH 034/103] rename reference file --- ..._1.sql.reference => 00700_to_decimal_or_something_1.reference} | 0 1 file changed, 0 insertions(+), 0 deletions(-) rename tests/queries/0_stateless/{00700_to_decimal_or_something_1.sql.reference => 00700_to_decimal_or_something_1.reference} (100%) diff --git a/tests/queries/0_stateless/00700_to_decimal_or_something_1.sql.reference b/tests/queries/0_stateless/00700_to_decimal_or_something_1.reference similarity index 100% rename from tests/queries/0_stateless/00700_to_decimal_or_something_1.sql.reference rename to tests/queries/0_stateless/00700_to_decimal_or_something_1.reference From ecbf10e41cc99e5c544aab1c21a713f55a350148 Mon Sep 17 00:00:00 2001 From: Denny Crane Date: Wed, 5 Apr 2023 21:47:36 -0300 Subject: [PATCH 035/103] rename reference file, i named incorreclty again --- ...g_1.reference => 00700_to_decimal_or_something_1.reference.j2} | 0 1 file changed, 0 insertions(+), 0 deletions(-) rename tests/queries/0_stateless/{00700_to_decimal_or_something_1.reference => 00700_to_decimal_or_something_1.reference.j2} (100%) diff --git a/tests/queries/0_stateless/00700_to_decimal_or_something_1.reference b/tests/queries/0_stateless/00700_to_decimal_or_something_1.reference.j2 similarity index 100% rename from tests/queries/0_stateless/00700_to_decimal_or_something_1.reference rename to tests/queries/0_stateless/00700_to_decimal_or_something_1.reference.j2 From 0dfcc849105cb53f1c58d88d5508cdb5853adafb Mon Sep 17 00:00:00 2001 From: Nikolay Degterinsky Date: Thu, 6 Apr 2023 00:49:39 +0000 Subject: [PATCH 036/103] Add support for Date/Datetime datatypes to arrayMin, arrayMax, arrayDifference functions --- src/DataTypes/DataTypeDateTime64.h | 10 ++++++ src/DataTypes/IDataType.h | 3 ++ src/Functions/array/arrayAggregation.cpp | 31 +++++++++++++++++-- src/Functions/array/arrayDifference.cpp | 15 +++++++-- .../01602_array_aggregation.reference | 4 +++ .../0_stateless/01602_array_aggregation.sql | 5 +++ 6 files changed, 63 insertions(+), 5 deletions(-) diff --git a/src/DataTypes/DataTypeDateTime64.h b/src/DataTypes/DataTypeDateTime64.h index ac0f49613a2..6c828594f5a 100644 --- a/src/DataTypes/DataTypeDateTime64.h +++ b/src/DataTypes/DataTypeDateTime64.h @@ -41,5 +41,15 @@ protected: SerializationPtr doGetDefaultSerialization() const override; }; +inline std::string getDateTimeTimezone(const IDataType & data_type) +{ + if (const auto * type = typeid_cast(&data_type)) + return type->hasExplicitTimeZone() ? type->getTimeZone().getTimeZone() : std::string(); + if (const auto * type = typeid_cast(&data_type)) + return type->hasExplicitTimeZone() ? type->getTimeZone().getTimeZone() : std::string(); + + throw Exception(ErrorCodes::LOGICAL_ERROR, "Cannot get decimal scale from type {}", data_type.getName()); +} + } diff --git a/src/DataTypes/IDataType.h b/src/DataTypes/IDataType.h index 4816bd11ab7..1b512bca30d 100644 --- a/src/DataTypes/IDataType.h +++ b/src/DataTypes/IDataType.h @@ -556,6 +556,7 @@ inline bool isNullableOrLowCardinalityNullable(const DataTypePtr & data_type) template constexpr bool IsDataTypeDecimal = false; template constexpr bool IsDataTypeNumber = false; template constexpr bool IsDataTypeDateOrDateTime = false; +template constexpr bool IsDataTypeDate = false; template constexpr bool IsDataTypeEnum = false; template constexpr bool IsDataTypeDecimalOrNumber = IsDataTypeDecimal || IsDataTypeNumber; @@ -580,6 +581,8 @@ template <> inline constexpr bool IsDataTypeDateOrDateTime = true; template <> inline constexpr bool IsDataTypeDateOrDateTime = true; template <> inline constexpr bool IsDataTypeDateOrDateTime = true; template <> inline constexpr bool IsDataTypeDateOrDateTime = true; +template <> inline constexpr bool IsDataTypeDate = true; +template <> inline constexpr bool IsDataTypeDate = true; template class DataTypeEnum; diff --git a/src/Functions/array/arrayAggregation.cpp b/src/Functions/array/arrayAggregation.cpp index 8818ebde9f1..238d46202aa 100644 --- a/src/Functions/array/arrayAggregation.cpp +++ b/src/Functions/array/arrayAggregation.cpp @@ -5,6 +5,9 @@ #include #include +#include +#include +#include #include #include #include @@ -129,6 +132,30 @@ struct ArrayAggregateImpl return true; } + else if constexpr (IsDataTypeDateOrDateTime && (aggregate_operation == AggregateOperation::max || aggregate_operation == AggregateOperation::min)) + { + if constexpr (IsDataTypeDate) + { + result = std::make_shared(); + + return true; + } + else if constexpr (!IsDataTypeDecimal) + { + std::string timezone = getDateTimeTimezone(*expression_return); + result = std::make_shared(timezone); + + return true; + } + else + { + std::string timezone = getDateTimeTimezone(*expression_return); + UInt32 scale = getDecimalScale(*expression_return); + result = std::make_shared(scale, timezone); + + return true; + } + } return false; }; @@ -370,8 +397,8 @@ struct ArrayAggregateImpl executeType(mapped, offsets, res) || executeType(mapped, offsets, res) || executeType(mapped, offsets, res) || - executeType(mapped, offsets, res)) - { + executeType(mapped, offsets, res) || + executeType(mapped, offsets, res)) return res; } else diff --git a/src/Functions/array/arrayDifference.cpp b/src/Functions/array/arrayDifference.cpp index 2852c5b967c..d210a2559a2 100644 --- a/src/Functions/array/arrayDifference.cpp +++ b/src/Functions/array/arrayDifference.cpp @@ -35,10 +35,10 @@ struct ArrayDifferenceImpl if (which.isUInt8() || which.isInt8()) return std::make_shared(std::make_shared()); - if (which.isUInt16() || which.isInt16()) + if (which.isUInt16() || which.isInt16() || which.isDate()) return std::make_shared(std::make_shared()); - if (which.isUInt32() || which.isUInt64() || which.isInt32() || which.isInt64()) + if (which.isUInt32() || which.isUInt64() || which.isInt32() || which.isInt64() || which.isDate32() || which.isDateTime()) return std::make_shared(std::make_shared()); if (which.isFloat32() || which.isFloat64()) @@ -47,6 +47,14 @@ struct ArrayDifferenceImpl if (which.isDecimal()) return std::make_shared(expression_return); + if (which.isDateTime64()) + { + UInt32 scale = getDecimalScale(*expression_return); + UInt32 precision = getDecimalPrecision(*expression_return); + + return std::make_shared(std::make_shared>(precision, scale)); + } + throw Exception(ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT, "arrayDifference cannot process values of type {}", expression_return->getName()); } @@ -146,7 +154,8 @@ struct ArrayDifferenceImpl executeType(mapped, array, res) || executeType(mapped, array, res) || executeType(mapped, array, res) || - executeType(mapped, array, res)) + executeType(mapped, array, res) || + executeType(mapped, array, res)) return res; else throw Exception(ErrorCodes::ILLEGAL_COLUMN, "Unexpected column for arrayDifference: {}", mapped->getName()); diff --git a/tests/queries/0_stateless/01602_array_aggregation.reference b/tests/queries/0_stateless/01602_array_aggregation.reference index bc21fae692e..ec8a0838401 100644 --- a/tests/queries/0_stateless/01602_array_aggregation.reference +++ b/tests/queries/0_stateless/01602_array_aggregation.reference @@ -34,6 +34,10 @@ Table array decimal avg 3.5 0 2 +2023-04-05 00:25:24 2023-04-05 00:25:23 [0,1] +2023-04-05 00:25:24.124 2023-04-05 00:25:23.123 [0,1.001] +2023-04-06 2023-04-05 [0,1] +2023-04-06 2023-04-05 [0,1] Types of aggregation result array min Int8 Int16 Int32 Int64 UInt8 UInt16 UInt32 UInt64 diff --git a/tests/queries/0_stateless/01602_array_aggregation.sql b/tests/queries/0_stateless/01602_array_aggregation.sql index 7159eb1442b..7c0f6eb8267 100644 --- a/tests/queries/0_stateless/01602_array_aggregation.sql +++ b/tests/queries/0_stateless/01602_array_aggregation.sql @@ -34,6 +34,11 @@ SELECT arrayAvg(x) FROM test_aggregation; DROP TABLE test_aggregation; +WITH ['2023-04-05 00:25:23', '2023-04-05 00:25:24']::Array(DateTime) AS dt SELECT arrayMax(dt), arrayMin(dt), arrayDifference(dt); +WITH ['2023-04-05 00:25:23.123', '2023-04-05 00:25:24.124']::Array(DateTime64(3)) AS dt SELECT arrayMax(dt), arrayMin(dt), arrayDifference(dt); +WITH ['2023-04-05', '2023-04-06']::Array(Date) AS d SELECT arrayMax(d), arrayMin(d), arrayDifference(d); +WITH ['2023-04-05', '2023-04-06']::Array(Date32) AS d SELECT arrayMax(d), arrayMin(d), arrayDifference(d); + SELECT 'Types of aggregation result array min'; SELECT toTypeName(arrayMin([toInt8(0)])), toTypeName(arrayMin([toInt16(0)])), toTypeName(arrayMin([toInt32(0)])), toTypeName(arrayMin([toInt64(0)])); SELECT toTypeName(arrayMin([toUInt8(0)])), toTypeName(arrayMin([toUInt16(0)])), toTypeName(arrayMin([toUInt32(0)])), toTypeName(arrayMin([toUInt64(0)])); From 95086536b7d6bbbe89ab5c91bbc97de371b96e29 Mon Sep 17 00:00:00 2001 From: Nikolay Degterinsky Date: Thu, 6 Apr 2023 01:22:39 +0000 Subject: [PATCH 037/103] Fix style --- src/DataTypes/DataTypeDateTime64.h | 5 +++++ 1 file changed, 5 insertions(+) diff --git a/src/DataTypes/DataTypeDateTime64.h b/src/DataTypes/DataTypeDateTime64.h index 6c828594f5a..f3c9b30f232 100644 --- a/src/DataTypes/DataTypeDateTime64.h +++ b/src/DataTypes/DataTypeDateTime64.h @@ -9,6 +9,11 @@ class DateLUTImpl; namespace DB { +namespace ErrorCodes +{ + extern const int LOGICAL_ERROR; +} + /** DateTime64 is same as DateTime, but it stores values as Int64 and has configurable sub-second part. * * `scale` determines number of decimal places for sub-second part of the DateTime64. From 5eb31bba8743b33050a5711f7e3b1182e1bb5948 Mon Sep 17 00:00:00 2001 From: Robert Schulze Date: Thu, 6 Apr 2023 07:47:07 +0000 Subject: [PATCH 038/103] MySQL compat: Align behavior of %f in formatDateTime() with parseDateTime() --- .../functions/date-time-functions.md | 4 ++- src/Core/Settings.h | 3 +- src/Core/SettingsChangesHistory.h | 1 + src/Functions/formatDateTime.cpp | 36 ++++++++++++++++--- .../00718_format_datetime.reference | 25 +++++++++---- .../0_stateless/00718_format_datetime.sql | 19 ++++++++-- 6 files changed, 73 insertions(+), 15 deletions(-) diff --git a/docs/en/sql-reference/functions/date-time-functions.md b/docs/en/sql-reference/functions/date-time-functions.md index 11036d804dc..b49f8745468 100644 --- a/docs/en/sql-reference/functions/date-time-functions.md +++ b/docs/en/sql-reference/functions/date-time-functions.md @@ -1264,7 +1264,7 @@ Using replacement fields, you can define a pattern for the resulting string. “ | %d | day of the month, zero-padded (01-31) | 02 | | %D | Short MM/DD/YY date, equivalent to %m/%d/%y | 01/02/18 | | %e | day of the month, space-padded (1-31) |   2 | -| %f | fractional second from the fractional part of DateTime64 | 1234560 | +| %f | fractional second, see below (*) | 1234560 | | %F | short YYYY-MM-DD date, equivalent to %Y-%m-%d | 2018-01-02 | | %g | two-digit year format, aligned to ISO 8601, abbreviated from four-digit notation | 18 | | %G | four-digit year format for ISO week number, calculated from the week-based year [defined by the ISO 8601](https://en.wikipedia.org/wiki/ISO_8601#Week_dates) standard, normally useful only with %V | 2018 | @@ -1295,6 +1295,8 @@ Using replacement fields, you can define a pattern for the resulting string. “ | %z | Time offset from UTC as +HHMM or -HHMM | -0500 | | %% | a % sign | % | +(*) The behavior of `%f` is to print `000000` (six zeros) if the formatted value is a Date, Date32 or DateTime (which have no fractional seconds) or a DateTime64 with a precision of 0. Earlier versions of ClickHouse printed `0` in this case. The previous behavior can be restored using setting `formatdatetime_f_prints_single_zero = 1`. + **Example** Query: diff --git a/src/Core/Settings.h b/src/Core/Settings.h index b6a149aa4ca..1b565d421eb 100644 --- a/src/Core/Settings.h +++ b/src/Core/Settings.h @@ -467,7 +467,8 @@ class IColumn; M(Bool, allow_introspection_functions, false, "Allow functions for introspection of ELF and DWARF for query profiling. These functions are slow and may impose security considerations.", 0) \ \ M(Bool, allow_execute_multiif_columnar, true, "Allow execute multiIf function columnar", 0) \ - M(Bool, formatdatetime_parsedatetime_m_is_month_name, true, "Formatter '%M' in function 'formatDateTime' produces the month name instead of minutes.", 0) \ + M(Bool, formatdatetime_f_prints_single_zero, false, "Formatter '%f' in function 'formatDateTime()' produces a single zero instead of six zeros if the formatted value has no fractional seconds.", 0) \ + M(Bool, formatdatetime_parsedatetime_m_is_month_name, true, "Formatter '%M' in functions 'formatDateTime()' and 'parseDateTime()' produces the month name instead of minutes.", 0) \ \ M(UInt64, max_partitions_per_insert_block, 100, "Limit maximum number of partitions in single INSERTed block. Zero means unlimited. Throw exception if the block contains too many partitions. This setting is a safety threshold, because using large number of partitions is a common misconception.", 0) \ M(Int64, max_partitions_to_read, -1, "Limit the max number of partitions that can be accessed in one query. <= 0 means unlimited.", 0) \ diff --git a/src/Core/SettingsChangesHistory.h b/src/Core/SettingsChangesHistory.h index 4f89397ed9d..d7f80cc7a49 100644 --- a/src/Core/SettingsChangesHistory.h +++ b/src/Core/SettingsChangesHistory.h @@ -101,6 +101,7 @@ static std::map sett {"query_plan_aggregation_in_order", 0, 1, "Enable some refactoring around query plan"}, {"format_binary_max_string_size", 0, 1_GiB, "Prevent allocating large amount of memory"}}}, {"22.11", {{"use_structure_from_insertion_table_in_table_functions", 0, 2, "Improve using structure from insertion table in table functions"}}}, + {"23.4", {{"formatdatetime_f_prints_single_zero", true, false, "Improved compatibility with MySQL DATE_FORMAT()/STR_TO_DATE()"}}}, {"23.4", {{"formatdatetime_parsedatetime_m_is_month_name", false, true, "Improved compatibility with MySQL DATE_FORMAT/STR_TO_DATE"}}}, {"22.9", {{"force_grouping_standard_compatibility", false, true, "Make GROUPING function output the same as in SQL standard and other DBMS"}}}, {"22.7", {{"cross_to_inner_join_rewrite", 1, 2, "Force rewrite comma join to inner"}, diff --git a/src/Functions/formatDateTime.cpp b/src/Functions/formatDateTime.cpp index d6275a54c75..a015340fc5d 100644 --- a/src/Functions/formatDateTime.cpp +++ b/src/Functions/formatDateTime.cpp @@ -449,6 +449,20 @@ private: } size_t mysqlFractionalSecond(char * dest, Time /*source*/, UInt64 fractional_second, UInt32 scale, const DateLUTImpl & /*timezone*/) + { + if (scale == 0) + scale = 6; + + for (Int64 i = scale, value = fractional_second; i > 0; --i) + { + dest[i - 1] += value % 10; + value /= 10; + } + return scale; + } + + /// Same as mysqlFractionalSecond but prints a single zero if the value has no fractional seconds + size_t mysqlFractionalSecondSingleZero(char * dest, Time /*source*/, UInt64 fractional_second, UInt32 scale, const DateLUTImpl & /*timezone*/) { if (scale == 0) scale = 1; @@ -710,6 +724,7 @@ private: } const bool mysql_M_is_month_name; + const bool mysql_f_prints_single_zero; public: static constexpr auto name = Name::name; @@ -718,6 +733,7 @@ public: explicit FunctionFormatDateTimeImpl(ContextPtr context) : mysql_M_is_month_name(context->getSettings().formatdatetime_parsedatetime_m_is_month_name) + , mysql_f_prints_single_zero(context->getSettings().formatdatetime_f_prints_single_zero) { } @@ -1116,11 +1132,21 @@ public: // Fractional seconds case 'f': { - /// If the time data type has no fractional part, then we print '0' as the fractional part. - Instruction instruction; - instruction.setMysqlFunc(&Instruction::mysqlFractionalSecond); - instructions.push_back(std::move(instruction)); - out_template += String(std::max(1, scale), '0'); + /// If the time data type has no fractional part, we print (default) '000000' or (deprecated) '0' as fractional part. + if (mysql_f_prints_single_zero) + { + Instruction instruction; + instruction.setMysqlFunc(&Instruction::mysqlFractionalSecondSingleZero); + instructions.push_back(std::move(instruction)); + out_template += String(scale == 0 ? 1 : scale, '0'); + } + else + { + Instruction instruction; + instruction.setMysqlFunc(&Instruction::mysqlFractionalSecond); + instructions.push_back(std::move(instruction)); + out_template += String(scale == 0 ? 6 : scale, '0'); + } break; } diff --git a/tests/queries/0_stateless/00718_format_datetime.reference b/tests/queries/0_stateless/00718_format_datetime.reference index eb2c23576eb..50874ac9b2e 100644 --- a/tests/queries/0_stateless/00718_format_datetime.reference +++ b/tests/queries/0_stateless/00718_format_datetime.reference @@ -5,6 +5,7 @@ Jan Jan 02 02 01/02/18 01/02/18 2 2 +000000 000000 2018-01-02 2018-01-02 10 12 22 00 @@ -43,11 +44,23 @@ no formatting pattern no formatting pattern -1100 +0300 +0530 -1234560 -000340 +000000 +000000 +000000 +000000 +123 +123456 +123456789 +0 +0 +0 +0 +123 +123456 +123456789 2022-12-08 18:11:29.123400000 2022-12-08 18:11:29.1 -2022-12-08 18:11:29.0 -2022-12-08 18:11:29.0 -2022-12-08 00:00:00.0 -2022-12-08 00:00:00.0 +2022-12-08 18:11:29.000000 +2022-12-08 18:11:29.000000 +2022-12-08 00:00:00.000000 +2022-12-08 00:00:00.000000 diff --git a/tests/queries/0_stateless/00718_format_datetime.sql b/tests/queries/0_stateless/00718_format_datetime.sql index a77578fc7c3..c0db6a4f64e 100644 --- a/tests/queries/0_stateless/00718_format_datetime.sql +++ b/tests/queries/0_stateless/00718_format_datetime.sql @@ -17,6 +17,7 @@ SELECT formatDateTime(toDateTime('2018-01-02 22:33:44'), '%C'), formatDateTime(t SELECT formatDateTime(toDateTime('2018-01-02 22:33:44'), '%d'), formatDateTime(toDate32('2018-01-02'), '%d'); SELECT formatDateTime(toDateTime('2018-01-02 22:33:44'), '%D'), formatDateTime(toDate32('2018-01-02'), '%D'); SELECT formatDateTime(toDateTime('2018-01-02 22:33:44'), '%e'), formatDateTime(toDate32('2018-01-02'), '%e'); +SELECT formatDateTime(toDateTime('2018-01-02 22:33:44'), '%f'), formatDateTime(toDate32('2018-01-02'), '%f'); SELECT formatDateTime(toDateTime('2018-01-02 22:33:44'), '%F'), formatDateTime(toDate32('2018-01-02'), '%F'); SELECT formatDateTime(toDateTime('2018-01-02 22:33:44'), '%h'), formatDateTime(toDate32('2018-01-02'), '%h'); SELECT formatDateTime(toDateTime('2018-01-02 22:33:44'), '%H'), formatDateTime(toDate32('2018-01-02'), '%H'); @@ -66,8 +67,22 @@ SELECT formatDateTime(toDateTime('2020-01-01 01:00:00', 'US/Samoa'), '%z'); SELECT formatDateTime(toDateTime('2020-01-01 01:00:00', 'Europe/Moscow'), '%z'); SELECT formatDateTime(toDateTime('1970-01-01 00:00:00', 'Asia/Kolkata'), '%z'); -select formatDateTime(toDateTime64('2010-01-04 12:34:56.123456', 7), '%f'); -select formatDateTime(toDateTime64('2022-12-08 18:11:29.00034', 6, 'UTC'), '%f'); +-- %f (default settings) +select formatDateTime(toDate('2010-01-04'), '%f') SETTINGS formatdatetime_f_prints_single_zero = 0; +select formatDateTime(toDate32('2010-01-04'), '%f') SETTINGS formatdatetime_f_prints_single_zero = 0; +select formatDateTime(toDateTime('2010-01-04 12:34:56'), '%f') SETTINGS formatdatetime_f_prints_single_zero = 0; +select formatDateTime(toDateTime64('2010-01-04 12:34:56', 0), '%f') SETTINGS formatdatetime_f_prints_single_zero = 0; +select formatDateTime(toDateTime64('2010-01-04 12:34:56.123', 3), '%f') SETTINGS formatdatetime_f_prints_single_zero = 0; +select formatDateTime(toDateTime64('2010-01-04 12:34:56.123456', 6), '%f') SETTINGS formatdatetime_f_prints_single_zero = 0; +select formatDateTime(toDateTime64('2010-01-04 12:34:56.123456789', 9), '%f') SETTINGS formatdatetime_f_prints_single_zero = 0; +-- %f (legacy settings) +select formatDateTime(toDate('2010-01-04'), '%f') SETTINGS formatdatetime_f_prints_single_zero = 1; +select formatDateTime(toDate32('2010-01-04'), '%f') SETTINGS formatdatetime_f_prints_single_zero = 1; +select formatDateTime(toDateTime('2010-01-04 12:34:56'), '%f') SETTINGS formatdatetime_f_prints_single_zero = 1; +select formatDateTime(toDateTime64('2010-01-04 12:34:56', 0), '%f') SETTINGS formatdatetime_f_prints_single_zero = 1; +select formatDateTime(toDateTime64('2010-01-04 12:34:56.123', 3), '%f') SETTINGS formatdatetime_f_prints_single_zero = 1; +select formatDateTime(toDateTime64('2010-01-04 12:34:56.123456', 6), '%f') SETTINGS formatdatetime_f_prints_single_zero = 0; +select formatDateTime(toDateTime64('2010-01-04 12:34:56.123456789', 9), '%f') SETTINGS formatdatetime_f_prints_single_zero = 1; select formatDateTime(toDateTime64('2022-12-08 18:11:29.1234', 9, 'UTC'), '%F %T.%f'); select formatDateTime(toDateTime64('2022-12-08 18:11:29.1234', 1, 'UTC'), '%F %T.%f'); From b7be5fd89e825ae10be38e265a7289839d97b244 Mon Sep 17 00:00:00 2001 From: Robert Schulze Date: Thu, 6 Apr 2023 08:40:39 +0000 Subject: [PATCH 039/103] Improve code and tests --- src/Functions/parseDateTime.cpp | 48 ++++++++----------- .../02668_parse_datetime.reference | 6 +-- .../0_stateless/02668_parse_datetime.sql | 6 +-- 3 files changed, 27 insertions(+), 33 deletions(-) diff --git a/src/Functions/parseDateTime.cpp b/src/Functions/parseDateTime.cpp index 01239074852..12e2e113af4 100644 --- a/src/Functions/parseDateTime.cpp +++ b/src/Functions/parseDateTime.cpp @@ -716,7 +716,7 @@ namespace if constexpr (need_check_space == NeedCheckSpace::Yes) checkSpace(cur, end, 1, "assertChar requires size >= 1", fragment); - if (*cur != expected) + if (*cur != expected) [[unlikely]] throw Exception( ErrorCodes::CANNOT_PARSE_DATETIME, "Unable to parse fragment {} from {} because char {} is expected but {} provided", @@ -729,6 +729,24 @@ namespace return cur; } + template + static Pos assertNumber(Pos cur, Pos end, const String & fragment) + { + if constexpr (need_check_space == NeedCheckSpace::Yes) + checkSpace(cur, end, 1, "assertChar requires size >= 1", fragment); + + if (*cur < '0' || *cur > '9') [[unlikely]] + throw Exception( + ErrorCodes::CANNOT_PARSE_DATETIME, + "Unable to parse fragment {} from {} because {} is not a number", + fragment, + std::string_view(cur, end - cur), + String(*cur, 1)); + + ++cur; + return cur; + } + static Pos mysqlDayOfWeekTextShort(Pos cur, Pos end, const String & fragment, DateTime & date) { checkSpace(cur, end, 3, "mysqlDayOfWeekTextShort requires size >= 3", fragment); @@ -1037,32 +1055,8 @@ namespace static Pos mysqlMicrosecond(Pos cur, Pos end, const String & fragment, DateTime & /*date*/) { - checkSpace(cur, end, 6, "mysqlMicrosecond requires size >= 6", fragment); - - Pos start = cur; - auto check_is_number = [&](Pos pos) - { - if (*pos < '0' || *pos > '9') - throw Exception( - ErrorCodes::CANNOT_PARSE_DATETIME, - "Unable to parse fragment '{}' from '{}' because '{}'' is not a number ", - fragment, - std::string_view(start, end), - *cur); - }; - - check_is_number(cur); - ++cur; - check_is_number(cur); - ++cur; - check_is_number(cur); - ++cur; - check_is_number(cur); - ++cur; - check_is_number(cur); - ++cur; - check_is_number(cur); - ++cur; + for (size_t i = 0; i < 6; ++i) + cur = assertNumber(cur, end, fragment); return cur; } diff --git a/tests/queries/0_stateless/02668_parse_datetime.reference b/tests/queries/0_stateless/02668_parse_datetime.reference index a5b5ad7d109..3a6925ecb70 100644 --- a/tests/queries/0_stateless/02668_parse_datetime.reference +++ b/tests/queries/0_stateless/02668_parse_datetime.reference @@ -198,11 +198,11 @@ select parseDateTime('456789', '%f', 'UTC') = toDateTime('1970-01-01 00:00:00', select parseDateTime('42', '%f', 'UTC') = toDateTime('1970-01-01 00:00:00', 'UTC'); -- { serverError NOT_ENOUGH_SPACE } select parseDateTime('12ABCD', '%f', 'UTC') = toDateTime('1970-01-01 00:00:00', 'UTC'); -- { serverError CANNOT_PARSE_DATETIME } -- mixed YMD format -select parseDateTime('2021-01-04+23:00:00', '%Y-%m-%d+%H:%i:%s', 'UTC') = toDateTime('2021-01-04 23:00:00', 'UTC'); +select parseDateTime('2021-01-04+23:00:00.654321', '%Y-%m-%d+%H:%i:%s.%f', 'UTC') = toDateTime('2021-01-04 23:00:00', 'UTC'); 1 -select parseDateTime('2019-07-03 11:04:10', '%Y-%m-%d %H:%i:%s', 'UTC') = toDateTime('2019-07-03 11:04:10', 'UTC'); +select parseDateTime('2019-07-03 11:04:10.975319', '%Y-%m-%d %H:%i:%s.%f', 'UTC') = toDateTime('2019-07-03 11:04:10', 'UTC'); 1 -select parseDateTime('10:04:11 03-07-2019', '%s:%i:%H %d-%m-%Y', 'UTC') = toDateTime('2019-07-03 11:04:10', 'UTC'); +select parseDateTime('10:04:11 03-07-2019.242424', '%s:%i:%H %d-%m-%Y.%f', 'UTC') = toDateTime('2019-07-03 11:04:10', 'UTC'); 1 -- *OrZero, *OrNull, str_to_date select parseDateTimeOrZero('10:04:11 03-07-2019', '%s:%i:%H %d-%m-%Y', 'UTC') = toDateTime('2019-07-03 11:04:10', 'UTC'); diff --git a/tests/queries/0_stateless/02668_parse_datetime.sql b/tests/queries/0_stateless/02668_parse_datetime.sql index 33e84120521..b18375840c9 100644 --- a/tests/queries/0_stateless/02668_parse_datetime.sql +++ b/tests/queries/0_stateless/02668_parse_datetime.sql @@ -135,9 +135,9 @@ select parseDateTime('42', '%f', 'UTC') = toDateTime('1970-01-01 00:00:00', 'UTC select parseDateTime('12ABCD', '%f', 'UTC') = toDateTime('1970-01-01 00:00:00', 'UTC'); -- { serverError CANNOT_PARSE_DATETIME } -- mixed YMD format -select parseDateTime('2021-01-04+23:00:00', '%Y-%m-%d+%H:%i:%s', 'UTC') = toDateTime('2021-01-04 23:00:00', 'UTC'); -select parseDateTime('2019-07-03 11:04:10', '%Y-%m-%d %H:%i:%s', 'UTC') = toDateTime('2019-07-03 11:04:10', 'UTC'); -select parseDateTime('10:04:11 03-07-2019', '%s:%i:%H %d-%m-%Y', 'UTC') = toDateTime('2019-07-03 11:04:10', 'UTC'); +select parseDateTime('2021-01-04+23:00:00.654321', '%Y-%m-%d+%H:%i:%s.%f', 'UTC') = toDateTime('2021-01-04 23:00:00', 'UTC'); +select parseDateTime('2019-07-03 11:04:10.975319', '%Y-%m-%d %H:%i:%s.%f', 'UTC') = toDateTime('2019-07-03 11:04:10', 'UTC'); +select parseDateTime('10:04:11 03-07-2019.242424', '%s:%i:%H %d-%m-%Y.%f', 'UTC') = toDateTime('2019-07-03 11:04:10', 'UTC'); -- *OrZero, *OrNull, str_to_date select parseDateTimeOrZero('10:04:11 03-07-2019', '%s:%i:%H %d-%m-%Y', 'UTC') = toDateTime('2019-07-03 11:04:10', 'UTC'); From 1ec9f5e42c03d2a054f8aeecf8b8464de657f46b Mon Sep 17 00:00:00 2001 From: Kruglov Pavel <48961922+Avogar@users.noreply.github.com> Date: Thu, 6 Apr 2023 13:20:11 +0200 Subject: [PATCH 040/103] Avoid operation on uninitialised data in readDateTimeTextImpl --- src/IO/ReadHelpers.h | 11 +++++++---- 1 file changed, 7 insertions(+), 4 deletions(-) diff --git a/src/IO/ReadHelpers.h b/src/IO/ReadHelpers.h index 20ba73e0fa7..9c0c9525773 100644 --- a/src/IO/ReadHelpers.h +++ b/src/IO/ReadHelpers.h @@ -1028,12 +1028,15 @@ inline ReturnType readDateTimeTextImpl(DateTime64 & datetime64, UInt32 scale, Re bool is_ok = true; if constexpr (std::is_same_v) - datetime64 = DecimalUtils::decimalFromComponents(components, scale); + { + datetime64 = DecimalUtils::decimalFromComponents(components, scale) * negative_multiplier; + } else + { is_ok = DecimalUtils::tryGetDecimalFromComponents(components, scale, datetime64); - - datetime64 *= negative_multiplier; - + if (is_ok) + datetime64 *= negative_multiplier; + } return ReturnType(is_ok); } From 56c51043785c4c201816ec5b2b8a1b7f40ae70e4 Mon Sep 17 00:00:00 2001 From: avogar Date: Thu, 6 Apr 2023 11:43:56 +0000 Subject: [PATCH 041/103] Update tests --- tests/queries/0_stateless/01905_to_json_string.reference | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/tests/queries/0_stateless/01905_to_json_string.reference b/tests/queries/0_stateless/01905_to_json_string.reference index 33d435f8e1a..ec4f4e48bde 100644 --- a/tests/queries/0_stateless/01905_to_json_string.reference +++ b/tests/queries/0_stateless/01905_to_json_string.reference @@ -1,3 +1,3 @@ -[] 2947817982 "&" -69802.9769 "w" -1.9158530982937093e25 ["2003-05-15","1988-03-19 06:13:49","2090-04-14 03:58:26.029","91943d2e-480d-66b5-ee4c-1b5bb8eb7256"] "O" [] -[-115] 481807067 ",{MM" -170235.0663 "o" 3.3808659558052087e155 ["2055-01-12","2070-08-09 03:49:21","2068-11-30 09:36:49.672","20b0e7b5-ad0e-177b-3054-c779b2a8ebe0"] "I\\u001C" ["e57178f9-4d10-2fa1-7c2d-53c5a65c3463"] +[] 2947817982 "&" -69802.9769 "o" 3.3808659558052087e155 ["2142-01-24","2076-06-05 14:54:21","2068-11-30 09:36:49.672","ee4c1b5b-b8eb-7256-20b0-e7b5ad0e177b"] "" ["7c2d53c5-a65c-3463-a76e-e26583aca234"] +[-115] 481807067 ",{MM" -45534.1174 "w" 1.711178201812925e-166 ["1994-01-04","1971-12-29 08:41:23","2012-03-25 07:11:39.573","3054c779-b2a8-ebe0-ec50-64cb1c494fbd"] "\\u0000" ["055300b2-b400-653c-1ea0-2413e3a3af76"] {"1234":"5678"} From cfd9c4d85e26c2a77280582bcc5405a5c9d3d9cb Mon Sep 17 00:00:00 2001 From: Robert Schulze Date: Thu, 6 Apr 2023 14:01:05 +0000 Subject: [PATCH 042/103] Register aliases of date_diff in system.functions --- docs/en/sql-reference/functions/date-time-functions.md | 2 +- src/Functions/dateDiff.cpp | 5 +++++ .../queries/0_stateless/25342_date_diff_aliases.reference | 5 +++++ tests/queries/0_stateless/25342_date_diff_aliases.sql | 7 +++++++ 4 files changed, 18 insertions(+), 1 deletion(-) create mode 100644 tests/queries/0_stateless/25342_date_diff_aliases.reference create mode 100644 tests/queries/0_stateless/25342_date_diff_aliases.sql diff --git a/docs/en/sql-reference/functions/date-time-functions.md b/docs/en/sql-reference/functions/date-time-functions.md index 74ef9a28dc1..42a16b7185f 100644 --- a/docs/en/sql-reference/functions/date-time-functions.md +++ b/docs/en/sql-reference/functions/date-time-functions.md @@ -645,7 +645,7 @@ For an alternative to `date\_diff`, see function `age`. date_diff('unit', startdate, enddate, [timezone]) ``` -Aliases: `dateDiff`, `DATE_DIFF`. +Aliases: `dateDiff`, `DATE_DIFF`, `timestampDiff`, `timestamp_diff`, `TIMESTAMP_DIFF`. **Arguments** diff --git a/src/Functions/dateDiff.cpp b/src/Functions/dateDiff.cpp index 457b77b9843..b28c97d45b2 100644 --- a/src/Functions/dateDiff.cpp +++ b/src/Functions/dateDiff.cpp @@ -448,6 +448,11 @@ private: REGISTER_FUNCTION(DateDiff) { factory.registerFunction>({}, FunctionFactory::CaseInsensitive); + factory.registerAlias("date_diff", FunctionDateDiff::name); + factory.registerAlias("DATE_DIFF", FunctionDateDiff::name); + factory.registerAlias("timestampDiff", FunctionDateDiff::name); + factory.registerAlias("timestamp_diff", FunctionDateDiff::name); + factory.registerAlias("TIMESTAMP_DIFF", FunctionDateDiff::name); } REGISTER_FUNCTION(TimeDiff) diff --git a/tests/queries/0_stateless/25342_date_diff_aliases.reference b/tests/queries/0_stateless/25342_date_diff_aliases.reference new file mode 100644 index 00000000000..1eeb5a3a2fa --- /dev/null +++ b/tests/queries/0_stateless/25342_date_diff_aliases.reference @@ -0,0 +1,5 @@ +DATE_DIFF +TIMESTAMP_DIFF +date_diff +timestampDiff +timestamp_diff diff --git a/tests/queries/0_stateless/25342_date_diff_aliases.sql b/tests/queries/0_stateless/25342_date_diff_aliases.sql new file mode 100644 index 00000000000..c6b31c44f95 --- /dev/null +++ b/tests/queries/0_stateless/25342_date_diff_aliases.sql @@ -0,0 +1,7 @@ +SELECT name FROM system.functions +WHERE name = 'date_diff' + OR name = 'DATE_DIFF' + OR name = 'timestampDiff' + OR name = 'timestamp_diff' + OR name = 'TIMESTAMP_DIFF' +ORDER BY name; From e88938d6c8f21b3fe4a3e5d3cf2bbc0c18248d83 Mon Sep 17 00:00:00 2001 From: Anton Popov Date: Thu, 6 Apr 2023 16:17:45 +0200 Subject: [PATCH 043/103] Change error code. --- src/DataTypes/ObjectUtils.cpp | 3 ++- 1 file changed, 2 insertions(+), 1 deletion(-) diff --git a/src/DataTypes/ObjectUtils.cpp b/src/DataTypes/ObjectUtils.cpp index edda0235bcc..28f000b6f0d 100644 --- a/src/DataTypes/ObjectUtils.cpp +++ b/src/DataTypes/ObjectUtils.cpp @@ -30,6 +30,7 @@ namespace ErrorCodes extern const int TYPE_MISMATCH; extern const int LOGICAL_ERROR; extern const int INCOMPATIBLE_COLUMNS; + extern const int NOT_IMPLEMENTED; } size_t getNumberOfDimensions(const IDataType & type) @@ -121,7 +122,7 @@ DataTypePtr getDataTypeByColumn(const IColumn & column) return makeNullable(getDataTypeByColumn(column_nullable->getNestedColumn())); /// TODO: add more types. - throw Exception(ErrorCodes::LOGICAL_ERROR, "Cannot get data type of column {}", column.getFamilyName()); + throw Exception(ErrorCodes::NOT_IMPLEMENTED, "Cannot get data type of column {}", column.getFamilyName()); } template From 7bcb8c4970c37a4f6b5001454636effe7684d4f3 Mon Sep 17 00:00:00 2001 From: Robert Schulze Date: Thu, 6 Apr 2023 14:25:27 +0000 Subject: [PATCH 044/103] Register trim aliases in system.functions --- src/Functions/trim.cpp | 3 +++ tests/queries/0_stateless/25343_trim_aliases.reference | 3 +++ tests/queries/0_stateless/25343_trim_aliases.sql | 5 +++++ 3 files changed, 11 insertions(+) create mode 100644 tests/queries/0_stateless/25343_trim_aliases.reference create mode 100644 tests/queries/0_stateless/25343_trim_aliases.sql diff --git a/src/Functions/trim.cpp b/src/Functions/trim.cpp index acfab47a68b..dd51c606ff7 100644 --- a/src/Functions/trim.cpp +++ b/src/Functions/trim.cpp @@ -112,5 +112,8 @@ REGISTER_FUNCTION(Trim) factory.registerFunction(); factory.registerFunction(); factory.registerFunction(); + factory.registerAlias("ltrim", FunctionTrimLeft::name); + factory.registerAlias("rtrim", FunctionTrimRight::name); + factory.registerAlias("trim", FunctionTrimBoth::name); } } diff --git a/tests/queries/0_stateless/25343_trim_aliases.reference b/tests/queries/0_stateless/25343_trim_aliases.reference new file mode 100644 index 00000000000..fa0920cd079 --- /dev/null +++ b/tests/queries/0_stateless/25343_trim_aliases.reference @@ -0,0 +1,3 @@ +ltrim +rtrim +trim diff --git a/tests/queries/0_stateless/25343_trim_aliases.sql b/tests/queries/0_stateless/25343_trim_aliases.sql new file mode 100644 index 00000000000..d0d739805fd --- /dev/null +++ b/tests/queries/0_stateless/25343_trim_aliases.sql @@ -0,0 +1,5 @@ +SELECT name FROM system.functions +WHERE name = 'ltrim' + OR name = 'rtrim' + OR name = 'trim' +ORDER BY name; From 5282a20abad41f00c87a850166f972692308fc76 Mon Sep 17 00:00:00 2001 From: MikhailBurdukov Date: Thu, 6 Apr 2023 15:50:28 +0000 Subject: [PATCH 045/103] Test and fix --- src/Core/SettingsFields.h | 4 +- ...2705_settings_check_changed_flag.reference | 32 ++++++++ .../02705_settings_check_changed_flag.sql | 82 +++++++++++++++++++ 3 files changed, 116 insertions(+), 2 deletions(-) create mode 100644 tests/queries/0_stateless/02705_settings_check_changed_flag.reference create mode 100644 tests/queries/0_stateless/02705_settings_check_changed_flag.sql diff --git a/src/Core/SettingsFields.h b/src/Core/SettingsFields.h index 3994e402c9a..2cd55e6b4c5 100644 --- a/src/Core/SettingsFields.h +++ b/src/Core/SettingsFields.h @@ -453,8 +453,8 @@ struct SettingFieldMultiEnum explicit operator StorageType() const { return value.getValue(); } explicit operator Field() const { return toString(); } - SettingFieldMultiEnum & operator= (StorageType x) { changed = x != value.getValue(); value.setValue(x); return *this; } - SettingFieldMultiEnum & operator= (ValueType x) { changed = !(x == value); value = x; return *this; } + SettingFieldMultiEnum & operator= (StorageType x) { changed = true; value.setValue(x); return *this; } + SettingFieldMultiEnum & operator= (ValueType x) { changed = true; value = x; return *this; } SettingFieldMultiEnum & operator= (const Field & x) { parseFromString(x.safeGet()); return *this; } String toString() const diff --git a/tests/queries/0_stateless/02705_settings_check_changed_flag.reference b/tests/queries/0_stateless/02705_settings_check_changed_flag.reference new file mode 100644 index 00000000000..58a93072157 --- /dev/null +++ b/tests/queries/0_stateless/02705_settings_check_changed_flag.reference @@ -0,0 +1,32 @@ +0 +1 +1 +1 +0 +1 +1 +1 +0 +1 +1 +1 +0 +1 +1 +1 +0 +1 +1 +1 +0 +1 +1 +1 +0 +1 +1 +1 +0 +1 +1 +1 diff --git a/tests/queries/0_stateless/02705_settings_check_changed_flag.sql b/tests/queries/0_stateless/02705_settings_check_changed_flag.sql new file mode 100644 index 00000000000..151e7a66b71 --- /dev/null +++ b/tests/queries/0_stateless/02705_settings_check_changed_flag.sql @@ -0,0 +1,82 @@ +---SettingFieldNumber +SELECT changed from system.settings where name = 'mysql_max_rows_to_insert'; +SET mysql_max_rows_to_insert = 123123; + +select changed from system.settings where name = 'mysql_max_rows_to_insert'; +set mysql_max_rows_to_insert = 123123; +select changed from system.settings where name = 'mysql_max_rows_to_insert'; +set mysql_max_rows_to_insert = 65536; +select changed from system.settings where name = 'mysql_max_rows_to_insert'; + +---SettingAutoWrapper + +select changed from system.settings where name = 'insert_quorum'; +set insert_quorum = 123123; +select changed from system.settings where name = 'insert_quorum'; +set insert_quorum = 123123; +select changed from system.settings where name = 'insert_quorum'; +set insert_quorum = 0; +select changed from system.settings where name = 'insert_quorum'; + +---SettingFieldMaxThreads + +select changed from system.settings where name = 'max_alter_threads'; +set max_alter_threads = 123123; +select changed from system.settings where name = 'max_alter_threads'; +set max_alter_threads = 123123; +select changed from system.settings where name = 'max_alter_threads'; +set max_alter_threads = 0; +select changed from system.settings where name = 'max_alter_threads'; + +---SettingFieldTimespanUnit + +select changed from system.settings where name = 'drain_timeout'; +set drain_timeout = 123123; +select changed from system.settings where name = 'drain_timeout'; +set drain_timeout = 123123; +select changed from system.settings where name = 'drain_timeout'; +set drain_timeout = 3; +select changed from system.settings where name = 'drain_timeout'; + + +---SettingFieldChar + +select changed from system.settings where name = 'format_csv_delimiter'; +set format_csv_delimiter = ','; +select changed from system.settings where name = 'format_csv_delimiter'; +set format_csv_delimiter = ','; +select changed from system.settings where name = 'format_csv_delimiter'; +set format_csv_delimiter = ','; +select changed from system.settings where name = 'format_csv_delimiter'; + + +---SettingFieldURI + +select changed from system.settings where name = 'format_avro_schema_registry_url'; +set format_avro_schema_registry_url = 'https://github.com/ClickHouse/ClickHouse/tree/master/src/Core'; +select changed from system.settings where name = 'format_avro_schema_registry_url'; +set format_avro_schema_registry_url = 'https://github.com/ClickHouse/ClickHouse/tree/master/src/Core'; +select changed from system.settings where name = 'format_avro_schema_registry_url'; +set format_avro_schema_registry_url = ''; +select changed from system.settings where name = 'format_avro_schema_registry_url'; + + +--- SettingFieldEnum + +select changed from system.settings where name = 'output_format_orc_compression_method'; +set output_format_orc_compression_method = 'none'; +select changed from system.settings where name = 'output_format_orc_compression_method'; +set output_format_orc_compression_method = 'none'; +select changed from system.settings where name = 'output_format_orc_compression_method'; +set output_format_orc_compression_method = 'lz4'; +select changed from system.settings where name = 'output_format_orc_compression_method'; + +--- SettingFieldMultiEnum + +select changed from system.settings where name = 'join_algorithm'; +set join_algorithm = 'auto,direct'; +select changed from system.settings where name = 'join_algorithm'; +set join_algorithm = 'auto,direct'; +select changed from system.settings where name = 'join_algorithm'; +set join_algorithm = 'default'; +select changed from system.settings where name = 'join_algorithm'; From a60c315b3e87428fb5e56b642df8ae25509f3fcb Mon Sep 17 00:00:00 2001 From: Dmitry Novik Date: Thu, 6 Apr 2023 19:18:26 +0000 Subject: [PATCH 046/103] Support additional_result_filter --- src/Planner/Planner.cpp | 46 ++++++++++++++++++++++++ src/Planner/PlannerJoinTree.cpp | 46 ++---------------------- src/Planner/Utils.cpp | 63 +++++++++++++++++++++++++++++++++ src/Planner/Utils.h | 8 +++++ 4 files changed, 120 insertions(+), 43 deletions(-) diff --git a/src/Planner/Planner.cpp b/src/Planner/Planner.cpp index a8e0d80ce8c..ec243d123b0 100644 --- a/src/Planner/Planner.cpp +++ b/src/Planner/Planner.cpp @@ -33,8 +33,11 @@ #include #include +#include +#include #include +#include #include #include @@ -911,6 +914,46 @@ void addBuildSubqueriesForSetsStepIfNeeded(QueryPlan & query_plan, addCreatingSetsStep(query_plan, std::move(subqueries_for_sets), planner_context->getQueryContext()); } +/// Support for `additional_result_filter` setting +void addAdditionalFilterStepIfNeeded(QueryPlan & query_plan, + const QueryNode & query_node, + const SelectQueryOptions & select_query_options, + PlannerContextPtr & planner_context +) +{ + if (select_query_options.subquery_depth != 0) + return; + + const auto & query_context = planner_context->getQueryContext(); + const auto & settings = query_context->getSettingsRef(); + + auto additional_result_filter_ast = parseAdditionalResultFilter(settings); + if (!additional_result_filter_ast) + return; + + ColumnsDescription fake_column_descriptions; + NameSet fake_name_set; + for (const auto & column : query_node.getProjectionColumns()) + { + fake_column_descriptions.add(ColumnDescription(column.name, column.type)); + fake_name_set.emplace(column.name); + } + + auto storage = std::make_shared(StorageID{"dummy", "dummy"}, fake_column_descriptions); + auto fake_table_expression = std::make_shared(std::move(storage), query_context); + + auto filter_info = buildFilterInfo(additional_result_filter_ast, fake_table_expression, planner_context, std::move(fake_name_set)); + if (!filter_info.actions || !query_plan.isInitialized()) + return; + + auto filter_step = std::make_unique(query_plan.getCurrentDataStream(), + filter_info.actions, + filter_info.column_name, + filter_info.do_remove_column); + filter_step->setStepDescription("additional result filter"); + query_plan.addStep(std::move(filter_step)); +} + } PlannerContextPtr buildPlannerContext(const QueryTreeNodePtr & query_tree_node, @@ -1409,6 +1452,9 @@ void Planner::buildPlanForQueryNode() const auto & projection_analysis_result = expression_analysis_result.getProjection(); addExpressionStep(query_plan, projection_analysis_result.project_names_actions, "Project names", result_actions_to_execute); } + + // For additional_result_filter setting + addAdditionalFilterStepIfNeeded(query_plan, query_node, select_query_options, planner_context); } if (!select_query_options.only_analyze) diff --git a/src/Planner/PlannerJoinTree.cpp b/src/Planner/PlannerJoinTree.cpp index c220e1eaa88..08d10c6adba 100644 --- a/src/Planner/PlannerJoinTree.cpp +++ b/src/Planner/PlannerJoinTree.cpp @@ -387,46 +387,6 @@ void updatePrewhereOutputsIfNeeded(SelectQueryInfo & table_expression_query_info prewhere_outputs.insert(prewhere_outputs.end(), required_output_nodes.begin(), required_output_nodes.end()); } -FilterDAGInfo buildFilterInfo(ASTPtr filter_expression, - SelectQueryInfo & table_expression_query_info, - PlannerContextPtr & planner_context) -{ - const auto & query_context = planner_context->getQueryContext(); - - auto filter_query_tree = buildQueryTree(filter_expression, query_context); - - QueryAnalysisPass query_analysis_pass(table_expression_query_info.table_expression); - query_analysis_pass.run(filter_query_tree, query_context); - - auto & table_expression_data = planner_context->getTableExpressionDataOrThrow(table_expression_query_info.table_expression); - const auto table_expression_names = table_expression_data.getColumnNames(); - NameSet table_expression_required_names_without_filter(table_expression_names.begin(), table_expression_names.end()); - - collectSourceColumns(filter_query_tree, planner_context); - collectSets(filter_query_tree, *planner_context); - - auto filter_actions_dag = std::make_shared(); - - PlannerActionsVisitor actions_visitor(planner_context, false /*use_column_identifier_as_action_node_name*/); - auto expression_nodes = actions_visitor.visit(filter_actions_dag, filter_query_tree); - if (expression_nodes.size() != 1) - throw Exception(ErrorCodes::BAD_ARGUMENTS, - "Filter actions must return single output node. Actual {}", - expression_nodes.size()); - - auto & filter_actions_outputs = filter_actions_dag->getOutputs(); - filter_actions_outputs = std::move(expression_nodes); - - std::string filter_node_name = filter_actions_outputs[0]->result_name; - bool remove_filter_column = true; - - for (const auto & filter_input_node : filter_actions_dag->getInputs()) - if (table_expression_required_names_without_filter.contains(filter_input_node->result_name)) - filter_actions_outputs.push_back(filter_input_node); - - return {std::move(filter_actions_dag), std::move(filter_node_name), remove_filter_column}; -} - FilterDAGInfo buildRowPolicyFilterIfNeeded(const StoragePtr & storage, SelectQueryInfo & table_expression_query_info, PlannerContextPtr & planner_context) @@ -438,7 +398,7 @@ FilterDAGInfo buildRowPolicyFilterIfNeeded(const StoragePtr & storage, if (!row_policy_filter) return {}; - return buildFilterInfo(row_policy_filter->expression, table_expression_query_info, planner_context); + return buildFilterInfo(row_policy_filter->expression, table_expression_query_info.table_expression, planner_context); } FilterDAGInfo buildCustomKeyFilterIfNeeded(const StoragePtr & storage, @@ -469,7 +429,7 @@ FilterDAGInfo buildCustomKeyFilterIfNeeded(const StoragePtr & storage, *storage, query_context); - return buildFilterInfo(parallel_replicas_custom_filter_ast, table_expression_query_info, planner_context); + return buildFilterInfo(parallel_replicas_custom_filter_ast, table_expression_query_info.table_expression, planner_context); } /// Apply filters from additional_table_filters setting @@ -516,7 +476,7 @@ FilterDAGInfo buildAdditionalFiltersIfNeeded(const StoragePtr & storage, LOG_DEBUG(&Poco::Logger::get("buildAdditionalFiltersIfNeeded"), "Found additional filter: {}", additional_filter_ast->formatForErrorMessage()); - return buildFilterInfo(additional_filter_ast, table_expression_query_info, planner_context); + return buildFilterInfo(additional_filter_ast, table_expression_query_info.table_expression, planner_context); } JoinTreeQueryPlan buildQueryPlanForTableExpression(QueryTreeNodePtr table_expression, diff --git a/src/Planner/Utils.cpp b/src/Planner/Utils.cpp index e5d73bdd977..47da0e9483b 100644 --- a/src/Planner/Utils.cpp +++ b/src/Planner/Utils.cpp @@ -3,6 +3,8 @@ #include #include #include +#include +#include #include #include @@ -28,8 +30,12 @@ #include #include #include +#include +#include #include +#include +#include namespace DB { @@ -416,4 +422,61 @@ SelectQueryInfo buildSelectQueryInfo(const QueryTreeNodePtr & query_tree, const return select_query_info; } +FilterDAGInfo buildFilterInfo(ASTPtr filter_expression, + const QueryTreeNodePtr & table_expression, + PlannerContextPtr & planner_context, + NameSet table_expression_required_names_without_filter) +{ + const auto & query_context = planner_context->getQueryContext(); + + auto filter_query_tree = buildQueryTree(filter_expression, query_context); + + QueryAnalysisPass query_analysis_pass(table_expression); + query_analysis_pass.run(filter_query_tree, query_context); + + if (table_expression_required_names_without_filter.empty()) + { + auto & table_expression_data = planner_context->getTableExpressionDataOrThrow(table_expression); + const auto & table_expression_names = table_expression_data.getColumnNames(); + table_expression_required_names_without_filter.insert(table_expression_names.begin(), table_expression_names.end()); + } + + collectSourceColumns(filter_query_tree, planner_context); + collectSets(filter_query_tree, *planner_context); + + auto filter_actions_dag = std::make_shared(); + + PlannerActionsVisitor actions_visitor(planner_context, false /*use_column_identifier_as_action_node_name*/); + auto expression_nodes = actions_visitor.visit(filter_actions_dag, filter_query_tree); + if (expression_nodes.size() != 1) + throw Exception(ErrorCodes::BAD_ARGUMENTS, + "Filter actions must return single output node. Actual {}", + expression_nodes.size()); + + auto & filter_actions_outputs = filter_actions_dag->getOutputs(); + filter_actions_outputs = std::move(expression_nodes); + + std::string filter_node_name = filter_actions_outputs[0]->result_name; + bool remove_filter_column = true; + + for (const auto & filter_input_node : filter_actions_dag->getInputs()) + if (table_expression_required_names_without_filter.contains(filter_input_node->result_name)) + filter_actions_outputs.push_back(filter_input_node); + + return {std::move(filter_actions_dag), std::move(filter_node_name), remove_filter_column}; +} + +ASTPtr parseAdditionalResultFilter(const Settings & settings) +{ + const String & additional_result_filter = settings.additional_result_filter; + if (additional_result_filter.empty()) + return {}; + + ParserExpression parser; + auto additional_result_filter_ast = parseQuery( + parser, additional_result_filter.data(), additional_result_filter.data() + additional_result_filter.size(), + "additional result filter", settings.max_query_size, settings.max_parser_depth); + return additional_result_filter_ast; +} + } diff --git a/src/Planner/Utils.h b/src/Planner/Utils.h index f6123e1a1d3..d9412800e61 100644 --- a/src/Planner/Utils.h +++ b/src/Planner/Utils.h @@ -78,4 +78,12 @@ QueryTreeNodePtr buildSubqueryToReadColumnsFromTableExpression(const NamesAndTyp SelectQueryInfo buildSelectQueryInfo(const QueryTreeNodePtr & query_tree, const PlannerContextPtr & planner_context); +/// Build filter for specific table_expression +FilterDAGInfo buildFilterInfo(ASTPtr filter_expression, + const QueryTreeNodePtr & table_expression, + PlannerContextPtr & planner_context, + NameSet table_expression_required_names_without_filter = {}); + +ASTPtr parseAdditionalResultFilter(const Settings & settings); + } From a1552159e34f80005b07b711bd92d9624e3f288d Mon Sep 17 00:00:00 2001 From: Dmitry Novik Date: Thu, 6 Apr 2023 21:35:28 +0000 Subject: [PATCH 047/103] Fix style --- src/Planner/Utils.cpp | 1 + 1 file changed, 1 insertion(+) diff --git a/src/Planner/Utils.cpp b/src/Planner/Utils.cpp index 47da0e9483b..5c61b2fc2c7 100644 --- a/src/Planner/Utils.cpp +++ b/src/Planner/Utils.cpp @@ -42,6 +42,7 @@ namespace DB namespace ErrorCodes { + extern const int BAD_ARGUMENTS; extern const int LOGICAL_ERROR; extern const int UNION_ALL_RESULT_STRUCTURES_MISMATCH; extern const int INTERSECT_OR_EXCEPT_RESULT_STRUCTURES_MISMATCH; From 343a07179892efddc48488d6b211dad0fed866a8 Mon Sep 17 00:00:00 2001 From: serxa Date: Thu, 6 Apr 2023 22:38:58 +0000 Subject: [PATCH 048/103] add `lost_part_count` column to `system.replicas` --- .../MergeTree/ReplicatedTableStatus.h | 1 + src/Storages/StorageReplicatedMergeTree.cpp | 24 ++++++++++++++++++- src/Storages/System/StorageSystemReplicas.cpp | 3 +++ .../02117_show_create_table_system.reference | 1 + 4 files changed, 28 insertions(+), 1 deletion(-) diff --git a/src/Storages/MergeTree/ReplicatedTableStatus.h b/src/Storages/MergeTree/ReplicatedTableStatus.h index b9f84091e9b..46e971f562a 100644 --- a/src/Storages/MergeTree/ReplicatedTableStatus.h +++ b/src/Storages/MergeTree/ReplicatedTableStatus.h @@ -25,6 +25,7 @@ struct ReplicatedTableStatus UInt64 absolute_delay; UInt8 total_replicas; UInt8 active_replicas; + UInt64 lost_part_count; String last_queue_update_exception; /// If the error has happened fetching the info from ZooKeeper, this field will be set. String zookeeper_exception; diff --git a/src/Storages/StorageReplicatedMergeTree.cpp b/src/Storages/StorageReplicatedMergeTree.cpp index cbfe3f8cab2..600168d2637 100644 --- a/src/Storages/StorageReplicatedMergeTree.cpp +++ b/src/Storages/StorageReplicatedMergeTree.cpp @@ -652,6 +652,8 @@ void StorageReplicatedMergeTree::createNewZooKeeperNodes() futures.push_back(zookeeper->asyncTryCreateNoThrow(zookeeper_path + "/alter_partition_version", String(), zkutil::CreateMode::Persistent)); /// For deduplication of async inserts futures.push_back(zookeeper->asyncTryCreateNoThrow(zookeeper_path + "/async_blocks", String(), zkutil::CreateMode::Persistent)); + /// To track "lost forever" parts count, just for `system.replicas` table + futures.push_back(zookeeper->asyncTryCreateNoThrow(zookeeper_path + "/lost_part_count", String(), zkutil::CreateMode::Persistent)); /// As for now, "/temp" node must exist, but we want to be able to remove it in future if (zookeeper->exists(zookeeper_path + "/temp")) @@ -5960,6 +5962,7 @@ void StorageReplicatedMergeTree::getStatus(ReplicatedTableStatus & res, bool wit res.log_pointer = 0; res.total_replicas = 0; res.active_replicas = 0; + res.lost_part_count = 0; res.last_queue_update_exception = getLastQueueUpdateException(); if (with_zk_fields && !res.is_session_expired) @@ -5976,6 +5979,7 @@ void StorageReplicatedMergeTree::getStatus(ReplicatedTableStatus & res, bool wit paths.clear(); paths.push_back(fs::path(replica_path) / "log_pointer"); + paths.push_back(fs::path(zookeeper_path) / "lost_part_count"); for (const String & replica : all_replicas) paths.push_back(fs::path(zookeeper_path) / "replicas" / replica / "is_active"); @@ -5993,10 +5997,14 @@ void StorageReplicatedMergeTree::getStatus(ReplicatedTableStatus & res, bool wit res.log_pointer = log_pointer_str.empty() ? 0 : parse(log_pointer_str); res.total_replicas = all_replicas.size(); + if (get_result[1].error == Coordination::Error::ZNONODE) + res.lost_part_count = 0; + else + res.lost_part_count = get_result[1].data.empty() ? 0 : parse(get_result[1].data); for (size_t i = 0, size = all_replicas.size(); i < size; ++i) { - bool is_replica_active = get_result[i + 1].error != Coordination::Error::ZNONODE; + bool is_replica_active = get_result[i + 2].error != Coordination::Error::ZNONODE; res.active_replicas += static_cast(is_replica_active); res.replica_is_active.emplace(all_replicas[i], is_replica_active); } @@ -8862,6 +8870,20 @@ bool StorageReplicatedMergeTree::createEmptyPartInsteadOfLost(zkutil::ZooKeeperP getCommitPartOps(ops, new_data_part); + /// Increment lost_part_count + auto lost_part_count_path = fs::path(zookeeper_path) / "lost_part_count"; + Coordination::Stat lost_part_count_stat; + String lost_part_count_str; + if (zookeeper->tryGet(lost_part_count_path, lost_part_count_str, &lost_part_count_stat)) + { + UInt64 lost_part_count = lost_part_count_str.empty() ? 0 : parse(lost_part_count_str); + ops.emplace_back(zkutil::makeSetRequest(lost_part_count_path, fmt::format("{}", lost_part_count + 1), lost_part_count_stat.version)); + } + else + { + ops.emplace_back(zkutil::makeCreateRequest(lost_part_count_path, "1", zkutil::CreateMode::Persistent)); + } + Coordination::Responses responses; if (auto code = zookeeper->tryMulti(ops, responses); code == Coordination::Error::ZOK) { diff --git a/src/Storages/System/StorageSystemReplicas.cpp b/src/Storages/System/StorageSystemReplicas.cpp index 240d452fe29..1a09c8fb96f 100644 --- a/src/Storages/System/StorageSystemReplicas.cpp +++ b/src/Storages/System/StorageSystemReplicas.cpp @@ -59,6 +59,7 @@ StorageSystemReplicas::StorageSystemReplicas(const StorageID & table_id_) { "absolute_delay", std::make_shared() }, { "total_replicas", std::make_shared() }, { "active_replicas", std::make_shared() }, + { "lost_part_count", std::make_shared() }, { "last_queue_update_exception", std::make_shared() }, { "zookeeper_exception", std::make_shared() }, { "replica_is_active", std::make_shared(std::make_shared(), std::make_shared()) } @@ -112,6 +113,7 @@ Pipe StorageSystemReplicas::read( || column_name == "log_pointer" || column_name == "total_replicas" || column_name == "active_replicas" + || column_name == "lost_part_count" || column_name == "zookeeper_exception" || column_name == "replica_is_active") { @@ -212,6 +214,7 @@ Pipe StorageSystemReplicas::read( res_columns[col_num++]->insert(status.absolute_delay); res_columns[col_num++]->insert(status.total_replicas); res_columns[col_num++]->insert(status.active_replicas); + res_columns[col_num++]->insert(status.lost_part_count); res_columns[col_num++]->insert(status.last_queue_update_exception); res_columns[col_num++]->insert(status.zookeeper_exception); diff --git a/tests/queries/0_stateless/02117_show_create_table_system.reference b/tests/queries/0_stateless/02117_show_create_table_system.reference index b07d6e01161..c3598788b2c 100644 --- a/tests/queries/0_stateless/02117_show_create_table_system.reference +++ b/tests/queries/0_stateless/02117_show_create_table_system.reference @@ -867,6 +867,7 @@ CREATE TABLE system.replicas `absolute_delay` UInt64, `total_replicas` UInt8, `active_replicas` UInt8, + `lost_part_count` UInt64, `last_queue_update_exception` String, `zookeeper_exception` String, `replica_is_active` Map(String, UInt8) From 9976f17e12668346c37482e8acb7901e1d31f117 Mon Sep 17 00:00:00 2001 From: serxa Date: Thu, 6 Apr 2023 22:54:00 +0000 Subject: [PATCH 049/103] add docs --- docs/en/operations/system-tables/replicas.md | 2 ++ 1 file changed, 2 insertions(+) diff --git a/docs/en/operations/system-tables/replicas.md b/docs/en/operations/system-tables/replicas.md index e711d9a7784..15426eefbcc 100644 --- a/docs/en/operations/system-tables/replicas.md +++ b/docs/en/operations/system-tables/replicas.md @@ -50,6 +50,7 @@ last_queue_update: 2021-10-12 14:50:08 absolute_delay: 99 total_replicas: 5 active_replicas: 5 +lost_part_count: 0 last_queue_update_exception: zookeeper_exception: replica_is_active: {'r1':1,'r2':1} @@ -90,6 +91,7 @@ The next 4 columns have a non-zero value only where there is an active session w - `absolute_delay` (`UInt64`) - How big lag in seconds the current replica has. - `total_replicas` (`UInt8`) - The total number of known replicas of this table. - `active_replicas` (`UInt8`) - The number of replicas of this table that have a session in ClickHouse Keeper (i.e., the number of functioning replicas). +- `lost_part_count` (`UInt64`) - The number of data parts lost in the table by all replicas in total since table creation. Value is persisted in ClickHouse Keeper and can only increase. - `last_queue_update_exception` (`String`) - When the queue contains broken entries. Especially important when ClickHouse breaks backward compatibility between versions and log entries written by newer versions aren't parseable by old versions. - `zookeeper_exception` (`String`) - The last exception message, got if the error happened when fetching the info from ClickHouse Keeper. - `replica_is_active` ([Map(String, UInt8)](../../sql-reference/data-types/map.md)) — Map between replica name and is replica active. From 2d07704243fd8f740ec247c92e50553f95526a49 Mon Sep 17 00:00:00 2001 From: serxa Date: Thu, 6 Apr 2023 23:50:23 +0000 Subject: [PATCH 050/103] fix zk tests --- .../0_stateless/02221_system_zookeeper_unrestricted.reference | 2 ++ .../02221_system_zookeeper_unrestricted_like.reference | 2 ++ 2 files changed, 4 insertions(+) diff --git a/tests/queries/0_stateless/02221_system_zookeeper_unrestricted.reference b/tests/queries/0_stateless/02221_system_zookeeper_unrestricted.reference index 60d3c78d740..53b44764d5c 100644 --- a/tests/queries/0_stateless/02221_system_zookeeper_unrestricted.reference +++ b/tests/queries/0_stateless/02221_system_zookeeper_unrestricted.reference @@ -36,6 +36,8 @@ log log log_pointer log_pointer +lost_part_count +lost_part_count max_processed_insert_time max_processed_insert_time metadata diff --git a/tests/queries/0_stateless/02221_system_zookeeper_unrestricted_like.reference b/tests/queries/0_stateless/02221_system_zookeeper_unrestricted_like.reference index c59be6a3af5..ccc3064ccbd 100644 --- a/tests/queries/0_stateless/02221_system_zookeeper_unrestricted_like.reference +++ b/tests/queries/0_stateless/02221_system_zookeeper_unrestricted_like.reference @@ -17,6 +17,7 @@ leader_election leader_election-0 log log_pointer +lost_part_count max_processed_insert_time metadata metadata @@ -58,6 +59,7 @@ leader_election leader_election-0 log log_pointer +lost_part_count max_processed_insert_time metadata metadata From 54d7a412ec85fdedaf0ac2ce4ebdce05d0026868 Mon Sep 17 00:00:00 2001 From: MikhailBurdukov Date: Fri, 7 Apr 2023 11:04:00 +0000 Subject: [PATCH 051/103] Fixed unit tests --- src/Core/tests/gtest_settings.cpp | 6 +++--- 1 file changed, 3 insertions(+), 3 deletions(-) diff --git a/src/Core/tests/gtest_settings.cpp b/src/Core/tests/gtest_settings.cpp index 46d8f9665dc..cbeb84ef2e7 100644 --- a/src/Core/tests/gtest_settings.cpp +++ b/src/Core/tests/gtest_settings.cpp @@ -122,7 +122,7 @@ GTEST_TEST(SettingMySQLDataTypesSupport, SetString) // comma with spaces setting = " datetime64 , decimal "; - ASSERT_FALSE(setting.changed); // false since value is the same as previous one. + ASSERT_TRUE(setting.changed); ASSERT_TRUE(setting.value.isSet(MySQLDataTypesSupport::DECIMAL)); ASSERT_TRUE(setting.value.isSet(MySQLDataTypesSupport::DATETIME64)); ASSERT_EQ("decimal,datetime64", setting.toString()); @@ -136,7 +136,7 @@ GTEST_TEST(SettingMySQLDataTypesSupport, SetString) ASSERT_EQ(Field("decimal"), setting); setting = String(",decimal,decimal,decimal,decimal,decimal,decimal,decimal,decimal,decimal,"); - ASSERT_FALSE(setting.changed); //since previous value was DECIMAL + ASSERT_TRUE(setting.changed); //since previous value was DECIMAL ASSERT_TRUE(setting.value.isSet(MySQLDataTypesSupport::DECIMAL)); ASSERT_FALSE(setting.value.isSet(MySQLDataTypesSupport::DATETIME64)); ASSERT_EQ("decimal", setting.toString()); @@ -163,7 +163,7 @@ GTEST_TEST(SettingMySQLDataTypesSupport, SetInvalidString) ASSERT_EQ(0, setting.value.getValue()); EXPECT_NO_THROW(setting = String(", ")); - ASSERT_FALSE(setting.changed); + ASSERT_TRUE(setting.changed); ASSERT_EQ(0, setting.value.getValue()); } From 5b2b20a0b06e88ea1165cb59a6ff8f65653cd71f Mon Sep 17 00:00:00 2001 From: Azat Khuzhin Date: Fri, 7 Apr 2023 15:13:21 +0200 Subject: [PATCH 052/103] Rename ThreadGroupStatus to ThreadGroup There are methods like getThreadGroup() and ThreadGroupSwitcher class, so seems that this is logical. Signed-off-by: Azat Khuzhin --- src/Common/CurrentThread.cpp | 2 +- src/Common/CurrentThread.h | 6 ++-- src/Common/ThreadStatus.cpp | 6 ++-- src/Common/ThreadStatus.h | 24 +++++++-------- src/Interpreters/Aggregator.cpp | 4 +-- src/Interpreters/ExternalLoader.cpp | 2 +- src/Interpreters/ProcessList.cpp | 2 +- src/Interpreters/ProcessList.h | 4 +-- src/Interpreters/ThreadStatusExt.cpp | 30 +++++++++---------- .../Executors/CompletedPipelineExecutor.cpp | 2 +- .../PullingAsyncPipelineExecutor.cpp | 2 +- .../PushingAsyncPipelineExecutor.cpp | 2 +- .../Impl/ParallelFormattingOutputFormat.cpp | 4 +-- .../Impl/ParallelFormattingOutputFormat.h | 4 +-- .../Impl/ParallelParsingInputFormat.cpp | 4 +-- .../Formats/Impl/ParallelParsingInputFormat.h | 4 +-- .../Transforms/buildPushingToViewsChain.cpp | 6 ++-- src/Storages/MergeTree/MergeList.cpp | 4 +-- src/Storages/MergeTree/MergeList.h | 6 ++-- 19 files changed, 59 insertions(+), 59 deletions(-) diff --git a/src/Common/CurrentThread.cpp b/src/Common/CurrentThread.cpp index 6ec46d6508c..fd2ad0bbaf1 100644 --- a/src/Common/CurrentThread.cpp +++ b/src/Common/CurrentThread.cpp @@ -90,7 +90,7 @@ void CurrentThread::attachInternalTextLogsQueue(const std::shared_ptr & logs_queue, @@ -69,9 +69,9 @@ public: /// You must call one of these methods when create a query child thread: /// Add current thread to a group associated with the thread group - static void attachToGroup(const ThreadGroupStatusPtr & thread_group); + static void attachToGroup(const ThreadGroupPtr & thread_group); /// Is useful for a ThreadPool tasks - static void attachToGroupIfDetached(const ThreadGroupStatusPtr & thread_group); + static void attachToGroupIfDetached(const ThreadGroupPtr & thread_group); /// Non-master threads call this method in destructor automatically static void detachFromGroupIfNotDetached(); diff --git a/src/Common/ThreadStatus.cpp b/src/Common/ThreadStatus.cpp index 1b783aa9ec4..e00c9b168a9 100644 --- a/src/Common/ThreadStatus.cpp +++ b/src/Common/ThreadStatus.cpp @@ -61,7 +61,7 @@ static thread_local ThreadStack alt_stack; static thread_local bool has_alt_stack = false; #endif -ThreadGroupStatus::ThreadGroupStatus() +ThreadGroup::ThreadGroup() : master_thread_id(CurrentThread::get().thread_id) {} @@ -119,7 +119,7 @@ ThreadStatus::ThreadStatus() #endif } -ThreadGroupStatusPtr ThreadStatus::getThreadGroup() const +ThreadGroupPtr ThreadStatus::getThreadGroup() const { return thread_group; } @@ -139,7 +139,7 @@ ContextPtr ThreadStatus::getGlobalContext() const return global_context.lock(); } -void ThreadGroupStatus::attachInternalTextLogsQueue(const InternalTextLogsQueuePtr & logs_queue, LogsLevel logs_level) +void ThreadGroup::attachInternalTextLogsQueue(const InternalTextLogsQueuePtr & logs_queue, LogsLevel logs_level) { std::lock_guard lock(mutex); shared_data.logs_queue_ptr = logs_queue; diff --git a/src/Common/ThreadStatus.h b/src/Common/ThreadStatus.h index cb85aa67b11..16083fe0925 100644 --- a/src/Common/ThreadStatus.h +++ b/src/Common/ThreadStatus.h @@ -58,15 +58,15 @@ using ThreadStatusPtr = ThreadStatus *; * Create via CurrentThread::initializeQuery (for queries) or directly (for various background tasks). * Use via CurrentThread::getGroup. */ -class ThreadGroupStatus; -using ThreadGroupStatusPtr = std::shared_ptr; +class ThreadGroup; +using ThreadGroupPtr = std::shared_ptr; -class ThreadGroupStatus +class ThreadGroup { public: - ThreadGroupStatus(); + ThreadGroup(); using FatalErrorCallback = std::function; - ThreadGroupStatus(ContextPtr query_context_, FatalErrorCallback fatal_error_callback_ = {}); + ThreadGroup(ContextPtr query_context_, FatalErrorCallback fatal_error_callback_ = {}); /// The first thread created this thread group const UInt64 master_thread_id; @@ -104,9 +104,9 @@ public: void attachInternalProfileEventsQueue(const InternalProfileEventsQueuePtr & profile_queue); /// When new query starts, new thread group is created for it, current thread becomes master thread of the query - static ThreadGroupStatusPtr createForQuery(ContextPtr query_context_, FatalErrorCallback fatal_error_callback_ = {}); + static ThreadGroupPtr createForQuery(ContextPtr query_context_, FatalErrorCallback fatal_error_callback_ = {}); - static ThreadGroupStatusPtr createForBackgroundProcess(ContextPtr storage_context); + static ThreadGroupPtr createForBackgroundProcess(ContextPtr storage_context); std::vector getInvolvedThreadIds() const; void linkThread(UInt64 thread_it); @@ -163,7 +163,7 @@ public: private: /// Group of threads, to which this thread attached - ThreadGroupStatusPtr thread_group; + ThreadGroupPtr thread_group; /// Is set once ContextWeakPtr global_context; @@ -174,7 +174,7 @@ private: using FatalErrorCallback = std::function; FatalErrorCallback fatal_error_callback; - ThreadGroupStatus::SharedData local_data; + ThreadGroup::SharedData local_data; bool performance_counters_finalized = false; @@ -215,7 +215,7 @@ public: ThreadStatus(); ~ThreadStatus(); - ThreadGroupStatusPtr getThreadGroup() const; + ThreadGroupPtr getThreadGroup() const; const String & getQueryId() const; @@ -239,7 +239,7 @@ public: void setInternalThread(); /// Attaches slave thread to existing thread group - void attachToGroup(const ThreadGroupStatusPtr & thread_group_, bool check_detached = true); + void attachToGroup(const ThreadGroupPtr & thread_group_, bool check_detached = true); /// Detaches thread from the thread group and the query, dumps performance counters if they have not been dumped void detachFromGroup(); @@ -287,7 +287,7 @@ private: void logToQueryThreadLog(QueryThreadLog & thread_log, const String & current_database); - void attachToGroupImpl(const ThreadGroupStatusPtr & thread_group_); + void attachToGroupImpl(const ThreadGroupPtr & thread_group_); }; /** diff --git a/src/Interpreters/Aggregator.cpp b/src/Interpreters/Aggregator.cpp index d6fbf072d05..2d5de796e1c 100644 --- a/src/Interpreters/Aggregator.cpp +++ b/src/Interpreters/Aggregator.cpp @@ -2315,7 +2315,7 @@ BlocksList Aggregator::prepareBlocksAndFillTwoLevelImpl( std::atomic next_bucket_to_merge = 0; - auto converter = [&](size_t thread_id, ThreadGroupStatusPtr thread_group) + auto converter = [&](size_t thread_id, ThreadGroupPtr thread_group) { SCOPE_EXIT_SAFE( if (thread_group) @@ -3043,7 +3043,7 @@ void Aggregator::mergeBlocks(BucketToBlocks bucket_to_blocks, AggregatedDataVari LOG_TRACE(log, "Merging partially aggregated two-level data."); - auto merge_bucket = [&bucket_to_blocks, &result, this](Int32 bucket, Arena * aggregates_pool, ThreadGroupStatusPtr thread_group) + auto merge_bucket = [&bucket_to_blocks, &result, this](Int32 bucket, Arena * aggregates_pool, ThreadGroupPtr thread_group) { SCOPE_EXIT_SAFE( if (thread_group) diff --git a/src/Interpreters/ExternalLoader.cpp b/src/Interpreters/ExternalLoader.cpp index 04a116ec0c7..c1cbd8b75be 100644 --- a/src/Interpreters/ExternalLoader.cpp +++ b/src/Interpreters/ExternalLoader.cpp @@ -967,7 +967,7 @@ private: } /// Does the loading, possibly in the separate thread. - void doLoading(const String & name, size_t loading_id, bool forced_to_reload, size_t min_id_to_finish_loading_dependencies_, bool async, ThreadGroupStatusPtr thread_group = {}) + void doLoading(const String & name, size_t loading_id, bool forced_to_reload, size_t min_id_to_finish_loading_dependencies_, bool async, ThreadGroupPtr thread_group = {}) { SCOPE_EXIT_SAFE( if (thread_group) diff --git a/src/Interpreters/ProcessList.cpp b/src/Interpreters/ProcessList.cpp index 51053bd2884..aca474bf152 100644 --- a/src/Interpreters/ProcessList.cpp +++ b/src/Interpreters/ProcessList.cpp @@ -340,7 +340,7 @@ QueryStatus::QueryStatus( const String & query_, const ClientInfo & client_info_, QueryPriorities::Handle && priority_handle_, - ThreadGroupStatusPtr && thread_group_, + ThreadGroupPtr && thread_group_, IAST::QueryKind query_kind_, UInt64 watch_start_nanoseconds) : WithContext(context_) diff --git a/src/Interpreters/ProcessList.h b/src/Interpreters/ProcessList.h index a04beac4901..b593bcef395 100644 --- a/src/Interpreters/ProcessList.h +++ b/src/Interpreters/ProcessList.h @@ -86,7 +86,7 @@ protected: ClientInfo client_info; /// Info about all threads involved in query execution - ThreadGroupStatusPtr thread_group; + ThreadGroupPtr thread_group; Stopwatch watch; @@ -162,7 +162,7 @@ public: const String & query_, const ClientInfo & client_info_, QueryPriorities::Handle && priority_handle_, - ThreadGroupStatusPtr && thread_group_, + ThreadGroupPtr && thread_group_, IAST::QueryKind query_kind_, UInt64 watch_start_nanoseconds); diff --git a/src/Interpreters/ThreadStatusExt.cpp b/src/Interpreters/ThreadStatusExt.cpp index 72ce08bf653..070cd3f98e1 100644 --- a/src/Interpreters/ThreadStatusExt.cpp +++ b/src/Interpreters/ThreadStatusExt.cpp @@ -41,14 +41,14 @@ namespace ErrorCodes extern const int CANNOT_SET_THREAD_PRIORITY; } -ThreadGroupStatus::ThreadGroupStatus(ContextPtr query_context_, FatalErrorCallback fatal_error_callback_) +ThreadGroup::ThreadGroup(ContextPtr query_context_, FatalErrorCallback fatal_error_callback_) : master_thread_id(CurrentThread::get().thread_id) , query_context(query_context_) , global_context(query_context_->getGlobalContext()) , fatal_error_callback(fatal_error_callback_) {} -std::vector ThreadGroupStatus::getInvolvedThreadIds() const +std::vector ThreadGroup::getInvolvedThreadIds() const { std::vector res; @@ -60,22 +60,22 @@ std::vector ThreadGroupStatus::getInvolvedThreadIds() const return res; } -void ThreadGroupStatus::linkThread(UInt64 thread_it) +void ThreadGroup::linkThread(UInt64 thread_it) { std::lock_guard lock(mutex); thread_ids.insert(thread_it); } -ThreadGroupStatusPtr ThreadGroupStatus::createForQuery(ContextPtr query_context_, std::function fatal_error_callback_) +ThreadGroupPtr ThreadGroup::createForQuery(ContextPtr query_context_, std::function fatal_error_callback_) { - auto group = std::make_shared(query_context_, std::move(fatal_error_callback_)); + auto group = std::make_shared(query_context_, std::move(fatal_error_callback_)); group->memory_tracker.setDescription("(for query)"); return group; } -ThreadGroupStatusPtr ThreadGroupStatus::createForBackgroundProcess(ContextPtr storage_context) +ThreadGroupPtr ThreadGroup::createForBackgroundProcess(ContextPtr storage_context) { - auto group = std::make_shared(storage_context); + auto group = std::make_shared(storage_context); group->memory_tracker.setDescription("background process to apply mutate/merge in table"); /// However settings from storage context have to be applied @@ -89,7 +89,7 @@ ThreadGroupStatusPtr ThreadGroupStatus::createForBackgroundProcess(ContextPtr st return group; } -void ThreadGroupStatus::attachQueryForLog(const String & query_, UInt64 normalized_hash) +void ThreadGroup::attachQueryForLog(const String & query_, UInt64 normalized_hash) { auto hash = normalized_hash ? normalized_hash : normalizedQueryHash(query_); @@ -109,7 +109,7 @@ void ThreadStatus::attachQueryForLog(const String & query_) thread_group->attachQueryForLog(local_data.query_for_logs, local_data.normalized_query_hash); } -void ThreadGroupStatus::attachInternalProfileEventsQueue(const InternalProfileEventsQueuePtr & profile_queue) +void ThreadGroup::attachInternalProfileEventsQueue(const InternalProfileEventsQueuePtr & profile_queue) { std::lock_guard lock(mutex); shared_data.profile_queue_ptr = profile_queue; @@ -168,7 +168,7 @@ void ThreadStatus::applyQuerySettings() #endif } -void ThreadStatus::attachToGroupImpl(const ThreadGroupStatusPtr & thread_group_) +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_; @@ -234,7 +234,7 @@ void ThreadStatus::setInternalThread() internal_thread = true; } -void ThreadStatus::attachToGroup(const ThreadGroupStatusPtr & thread_group_, bool check_detached) +void ThreadStatus::attachToGroup(const ThreadGroupPtr & thread_group_, bool check_detached) { if (thread_group && check_detached) throw Exception(ErrorCodes::LOGICAL_ERROR, "Can't attach query to the thread, it is already attached"); @@ -541,14 +541,14 @@ void ThreadStatus::logToQueryViewsLog(const ViewRuntimeData & vinfo) views_log->add(element); } -void CurrentThread::attachToGroup(const ThreadGroupStatusPtr & thread_group) +void CurrentThread::attachToGroup(const ThreadGroupPtr & thread_group) { if (unlikely(!current_thread)) return; current_thread->attachToGroup(thread_group, true); } -void CurrentThread::attachToGroupIfDetached(const ThreadGroupStatusPtr & thread_group) +void CurrentThread::attachToGroupIfDetached(const ThreadGroupPtr & thread_group) { if (unlikely(!current_thread)) return; @@ -574,7 +574,7 @@ CurrentThread::QueryScope::QueryScope(ContextMutablePtr query_context, std::func if (!query_context->hasQueryContext()) query_context->makeQueryContext(); - auto group = ThreadGroupStatus::createForQuery(query_context, std::move(fatal_error_callback)); + auto group = ThreadGroup::createForQuery(query_context, std::move(fatal_error_callback)); CurrentThread::attachToGroup(group); } @@ -584,7 +584,7 @@ CurrentThread::QueryScope::QueryScope(ContextPtr query_context, std::functiongetPort(IOutputFormat::PortKind::Main).getHeader(); } -static void threadFunction(PullingAsyncPipelineExecutor::Data & data, ThreadGroupStatusPtr thread_group, size_t num_threads) +static void threadFunction(PullingAsyncPipelineExecutor::Data & data, ThreadGroupPtr thread_group, size_t num_threads) { SCOPE_EXIT_SAFE( if (thread_group) diff --git a/src/Processors/Executors/PushingAsyncPipelineExecutor.cpp b/src/Processors/Executors/PushingAsyncPipelineExecutor.cpp index 3aec7608e6d..ac40cef35d9 100644 --- a/src/Processors/Executors/PushingAsyncPipelineExecutor.cpp +++ b/src/Processors/Executors/PushingAsyncPipelineExecutor.cpp @@ -97,7 +97,7 @@ struct PushingAsyncPipelineExecutor::Data } }; -static void threadFunction(PushingAsyncPipelineExecutor::Data & data, ThreadGroupStatusPtr thread_group, size_t num_threads) +static void threadFunction(PushingAsyncPipelineExecutor::Data & data, ThreadGroupPtr thread_group, size_t num_threads) { SCOPE_EXIT_SAFE( if (thread_group) diff --git a/src/Processors/Formats/Impl/ParallelFormattingOutputFormat.cpp b/src/Processors/Formats/Impl/ParallelFormattingOutputFormat.cpp index 3fc57ca1c1e..62ee4e4a48d 100644 --- a/src/Processors/Formats/Impl/ParallelFormattingOutputFormat.cpp +++ b/src/Processors/Formats/Impl/ParallelFormattingOutputFormat.cpp @@ -96,7 +96,7 @@ namespace DB } - void ParallelFormattingOutputFormat::collectorThreadFunction(const ThreadGroupStatusPtr & thread_group) + void ParallelFormattingOutputFormat::collectorThreadFunction(const ThreadGroupPtr & thread_group) { SCOPE_EXIT_SAFE( if (thread_group) @@ -157,7 +157,7 @@ namespace DB } - void ParallelFormattingOutputFormat::formatterThreadFunction(size_t current_unit_number, size_t first_row_num, const ThreadGroupStatusPtr & thread_group) + void ParallelFormattingOutputFormat::formatterThreadFunction(size_t current_unit_number, size_t first_row_num, const ThreadGroupPtr & thread_group) { SCOPE_EXIT_SAFE( if (thread_group) diff --git a/src/Processors/Formats/Impl/ParallelFormattingOutputFormat.h b/src/Processors/Formats/Impl/ParallelFormattingOutputFormat.h index 790d05e83dd..4e5aaab5dcb 100644 --- a/src/Processors/Formats/Impl/ParallelFormattingOutputFormat.h +++ b/src/Processors/Formats/Impl/ParallelFormattingOutputFormat.h @@ -270,10 +270,10 @@ private: } /// Collects all temporary buffers into main WriteBuffer. - void collectorThreadFunction(const ThreadGroupStatusPtr & thread_group); + void collectorThreadFunction(const ThreadGroupPtr & thread_group); /// This function is executed in ThreadPool and the only purpose of it is to format one Chunk into a continuous buffer in memory. - void formatterThreadFunction(size_t current_unit_number, size_t first_row_num, const ThreadGroupStatusPtr & thread_group); + void formatterThreadFunction(size_t current_unit_number, size_t first_row_num, const ThreadGroupPtr & thread_group); void setRowsBeforeLimit(size_t rows_before_limit) override { diff --git a/src/Processors/Formats/Impl/ParallelParsingInputFormat.cpp b/src/Processors/Formats/Impl/ParallelParsingInputFormat.cpp index a2e5074efb1..f4d619a263b 100644 --- a/src/Processors/Formats/Impl/ParallelParsingInputFormat.cpp +++ b/src/Processors/Formats/Impl/ParallelParsingInputFormat.cpp @@ -8,7 +8,7 @@ namespace DB { -void ParallelParsingInputFormat::segmentatorThreadFunction(ThreadGroupStatusPtr thread_group) +void ParallelParsingInputFormat::segmentatorThreadFunction(ThreadGroupPtr thread_group) { SCOPE_EXIT_SAFE( if (thread_group) @@ -62,7 +62,7 @@ void ParallelParsingInputFormat::segmentatorThreadFunction(ThreadGroupStatusPtr } } -void ParallelParsingInputFormat::parserThreadFunction(ThreadGroupStatusPtr thread_group, size_t current_ticket_number) +void ParallelParsingInputFormat::parserThreadFunction(ThreadGroupPtr thread_group, size_t current_ticket_number) { SCOPE_EXIT_SAFE( if (thread_group) diff --git a/src/Processors/Formats/Impl/ParallelParsingInputFormat.h b/src/Processors/Formats/Impl/ParallelParsingInputFormat.h index 97df9308dbf..ae9f123d411 100644 --- a/src/Processors/Formats/Impl/ParallelParsingInputFormat.h +++ b/src/Processors/Formats/Impl/ParallelParsingInputFormat.h @@ -317,8 +317,8 @@ private: } } - void segmentatorThreadFunction(ThreadGroupStatusPtr thread_group); - void parserThreadFunction(ThreadGroupStatusPtr thread_group, size_t current_ticket_number); + void segmentatorThreadFunction(ThreadGroupPtr thread_group); + void parserThreadFunction(ThreadGroupPtr thread_group, size_t current_ticket_number); /// Save/log a background exception, set termination flag, wake up all /// threads. This function is used by segmentator and parsed threads. diff --git a/src/Processors/Transforms/buildPushingToViewsChain.cpp b/src/Processors/Transforms/buildPushingToViewsChain.cpp index 91845bc18ad..e32e2116f71 100644 --- a/src/Processors/Transforms/buildPushingToViewsChain.cpp +++ b/src/Processors/Transforms/buildPushingToViewsChain.cpp @@ -270,14 +270,14 @@ Chain buildPushingToViewsChain( ASTPtr query; Chain out; - /// NOTE: ThreadGroupStatus always should have context attached, + /// NOTE: ThreadGroup always should have context attached, /// otherwise entry to the system.query_views_log will not be added /// (see ThreadStatus::logToQueryViewsLog()) - ThreadGroupStatusPtr running_group; + ThreadGroupPtr running_group; if (current_thread) running_group = current_thread->getThreadGroup(); if (!running_group) - running_group = std::make_shared(context); + running_group = std::make_shared(context); /// We are creating a ThreadStatus per view to store its metrics individually /// Since calling ThreadStatus() changes current_thread we save it and restore it after the calls diff --git a/src/Storages/MergeTree/MergeList.cpp b/src/Storages/MergeTree/MergeList.cpp index 7970efd1a20..91e1cb1b078 100644 --- a/src/Storages/MergeTree/MergeList.cpp +++ b/src/Storages/MergeTree/MergeList.cpp @@ -11,7 +11,7 @@ namespace DB { -ThreadGroupSwitcher::ThreadGroupSwitcher(ThreadGroupStatusPtr thread_group) +ThreadGroupSwitcher::ThreadGroupSwitcher(ThreadGroupPtr thread_group) { chassert(thread_group); @@ -59,7 +59,7 @@ MergeListElement::MergeListElement( is_mutation = (result_part_info.getDataVersion() != source_data_version); } - thread_group = ThreadGroupStatus::createForBackgroundProcess(context); + thread_group = ThreadGroup::createForBackgroundProcess(context); } MergeInfo MergeListElement::getInfo() const diff --git a/src/Storages/MergeTree/MergeList.h b/src/Storages/MergeTree/MergeList.h index c3ba4f00f7f..308f00feda9 100644 --- a/src/Storages/MergeTree/MergeList.h +++ b/src/Storages/MergeTree/MergeList.h @@ -69,11 +69,11 @@ struct Settings; class ThreadGroupSwitcher : private boost::noncopyable { public: - explicit ThreadGroupSwitcher(ThreadGroupStatusPtr thread_group); + explicit ThreadGroupSwitcher(ThreadGroupPtr thread_group); ~ThreadGroupSwitcher(); private: - ThreadGroupStatusPtr prev_thread_group; + ThreadGroupPtr prev_thread_group; }; struct MergeListElement : boost::noncopyable @@ -113,7 +113,7 @@ struct MergeListElement : boost::noncopyable /// Detected after merge already started std::atomic merge_algorithm; - ThreadGroupStatusPtr thread_group; + ThreadGroupPtr thread_group; MergeListElement( const StorageID & table_id_, From aacf2a083882bf395d94a6f7dba1fa2d35db2991 Mon Sep 17 00:00:00 2001 From: Azat Khuzhin Date: Fri, 7 Apr 2023 15:15:10 +0200 Subject: [PATCH 053/103] Move ThreadGroupSwitcher to ThreadStatus.h (out from MergeTree code) Signed-off-by: Azat Khuzhin --- src/Common/ThreadStatus.h | 16 +++++++++++++++- src/Interpreters/ThreadStatusExt.cpp | 18 ++++++++++++++++++ src/Storages/MergeTree/MergeList.cpp | 18 ------------------ src/Storages/MergeTree/MergeList.h | 14 -------------- 4 files changed, 33 insertions(+), 33 deletions(-) diff --git a/src/Common/ThreadStatus.h b/src/Common/ThreadStatus.h index 16083fe0925..600dfc56d2b 100644 --- a/src/Common/ThreadStatus.h +++ b/src/Common/ThreadStatus.h @@ -41,7 +41,6 @@ class TaskStatsInfoGetter; class InternalTextLogsQueue; struct ViewRuntimeData; class QueryViewsLog; -class ThreadGroupSwitcher; using InternalTextLogsQueuePtr = std::shared_ptr; using InternalTextLogsQueueWeakPtr = std::weak_ptr; @@ -120,6 +119,21 @@ private: std::unordered_set thread_ids; }; +/** + * Since merge is executed with multiple threads, this class + * switches the parent MemoryTracker as part of the thread group to account all the memory used. + */ +class ThreadGroupSwitcher : private boost::noncopyable +{ +public: + explicit ThreadGroupSwitcher(ThreadGroupPtr thread_group); + ~ThreadGroupSwitcher(); + +private: + ThreadGroupPtr prev_thread_group; +}; + + /** * We use **constinit** here to tell the compiler the current_thread variable is initialized. * If we didn't help the compiler, then it would most likely add a check before every use of the variable to initialize it if needed. diff --git a/src/Interpreters/ThreadStatusExt.cpp b/src/Interpreters/ThreadStatusExt.cpp index 070cd3f98e1..c6e36263e7c 100644 --- a/src/Interpreters/ThreadStatusExt.cpp +++ b/src/Interpreters/ThreadStatusExt.cpp @@ -115,6 +115,24 @@ void ThreadGroup::attachInternalProfileEventsQueue(const InternalProfileEventsQu shared_data.profile_queue_ptr = profile_queue; } +ThreadGroupSwitcher::ThreadGroupSwitcher(ThreadGroupPtr thread_group) +{ + chassert(thread_group); + + /// might be nullptr + prev_thread_group = CurrentThread::getGroup(); + + CurrentThread::detachFromGroupIfNotDetached(); + CurrentThread::attachToGroup(thread_group); +} + +ThreadGroupSwitcher::~ThreadGroupSwitcher() +{ + CurrentThread::detachFromGroupIfNotDetached(); + if (prev_thread_group) + CurrentThread::attachToGroup(prev_thread_group); +} + void ThreadStatus::attachInternalProfileEventsQueue(const InternalProfileEventsQueuePtr & profile_queue) { if (!thread_group) diff --git a/src/Storages/MergeTree/MergeList.cpp b/src/Storages/MergeTree/MergeList.cpp index 91e1cb1b078..0bf662921ad 100644 --- a/src/Storages/MergeTree/MergeList.cpp +++ b/src/Storages/MergeTree/MergeList.cpp @@ -11,24 +11,6 @@ namespace DB { -ThreadGroupSwitcher::ThreadGroupSwitcher(ThreadGroupPtr thread_group) -{ - chassert(thread_group); - - /// might be nullptr - prev_thread_group = CurrentThread::getGroup(); - - CurrentThread::detachFromGroupIfNotDetached(); - CurrentThread::attachToGroup(thread_group); -} - -ThreadGroupSwitcher::~ThreadGroupSwitcher() -{ - CurrentThread::detachFromGroupIfNotDetached(); - if (prev_thread_group) - CurrentThread::attachToGroup(prev_thread_group); -} - MergeListElement::MergeListElement( const StorageID & table_id_, FutureMergedMutatedPartPtr future_part, diff --git a/src/Storages/MergeTree/MergeList.h b/src/Storages/MergeTree/MergeList.h index 308f00feda9..9c8c2ebd1e4 100644 --- a/src/Storages/MergeTree/MergeList.h +++ b/src/Storages/MergeTree/MergeList.h @@ -62,20 +62,6 @@ using MergeListEntry = BackgroundProcessListEntry; struct Settings; -/** - * Since merge is executed with multiple threads, this class - * switches the parent MemoryTracker as part of the thread group to account all the memory used. - */ -class ThreadGroupSwitcher : private boost::noncopyable -{ -public: - explicit ThreadGroupSwitcher(ThreadGroupPtr thread_group); - ~ThreadGroupSwitcher(); - -private: - ThreadGroupPtr prev_thread_group; -}; - struct MergeListElement : boost::noncopyable { const StorageID table_id; From 44ae8485f185ba43323d61a2e85007f5067b4e2b Mon Sep 17 00:00:00 2001 From: Azat Khuzhin Date: Fri, 7 Apr 2023 15:29:51 +0200 Subject: [PATCH 054/103] Use one ThreadGroup while pushing to materialized views Before this patch only the case when ThreadStatus (current_thread) already has ThreadGroup works that way, after they will be identical in this aspect. But this should not affect anything, but it just make sense. Signed-off-by: Azat Khuzhin --- src/Interpreters/InterpreterInsertQuery.cpp | 24 ++++++++++++++++--- src/Interpreters/InterpreterInsertQuery.h | 1 + .../Transforms/buildPushingToViewsChain.cpp | 22 ++++++++--------- .../Transforms/buildPushingToViewsChain.h | 4 ++++ 4 files changed, 36 insertions(+), 15 deletions(-) diff --git a/src/Interpreters/InterpreterInsertQuery.cpp b/src/Interpreters/InterpreterInsertQuery.cpp index b4a19ea7403..e78a61831a1 100644 --- a/src/Interpreters/InterpreterInsertQuery.cpp +++ b/src/Interpreters/InterpreterInsertQuery.cpp @@ -32,6 +32,7 @@ #include #include #include +#include #include @@ -233,8 +234,14 @@ Chain InterpreterInsertQuery::buildChain( ThreadStatusesHolderPtr thread_status_holder, std::atomic_uint64_t * elapsed_counter_ms) { + ThreadGroupPtr running_group; + if (current_thread) + running_group = current_thread->getThreadGroup(); + if (!running_group) + running_group = std::make_shared(getContext()); + auto sample = getSampleBlock(columns, table, metadata_snapshot); - return buildChainImpl(table, metadata_snapshot, sample, thread_status_holder, elapsed_counter_ms); + return buildChainImpl(table, metadata_snapshot, sample, thread_status_holder, running_group, elapsed_counter_ms); } Chain InterpreterInsertQuery::buildChainImpl( @@ -242,6 +249,7 @@ Chain InterpreterInsertQuery::buildChainImpl( const StorageMetadataPtr & metadata_snapshot, const Block & query_sample_block, ThreadStatusesHolderPtr thread_status_holder, + ThreadGroupPtr running_group, std::atomic_uint64_t * elapsed_counter_ms) { ThreadStatus * thread_status = current_thread; @@ -273,7 +281,9 @@ Chain InterpreterInsertQuery::buildChainImpl( } else { - out = buildPushingToViewsChain(table, metadata_snapshot, context_ptr, query_ptr, no_destination, thread_status_holder, elapsed_counter_ms); + out = buildPushingToViewsChain(table, metadata_snapshot, context_ptr, + query_ptr, no_destination, + thread_status_holder, running_group, elapsed_counter_ms); } /// Note that we wrap transforms one on top of another, so we write them in reverse of data processing order. @@ -461,9 +471,17 @@ BlockIO InterpreterInsertQuery::execute() pipeline = interpreter_watch.buildQueryPipeline(); } + 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 < out_streams_size; ++i) { - auto out = buildChainImpl(table, metadata_snapshot, query_sample_block, nullptr, nullptr); + auto out = buildChainImpl(table, metadata_snapshot, query_sample_block, + /* thread_status_holder= */ nullptr, + running_group, + /* elapsed_counter_ms= */ nullptr); out_chains.emplace_back(std::move(out)); } } diff --git a/src/Interpreters/InterpreterInsertQuery.h b/src/Interpreters/InterpreterInsertQuery.h index bb6509a9102..f60d6567d74 100644 --- a/src/Interpreters/InterpreterInsertQuery.h +++ b/src/Interpreters/InterpreterInsertQuery.h @@ -70,6 +70,7 @@ private: const StorageMetadataPtr & metadata_snapshot, const Block & query_sample_block, ThreadStatusesHolderPtr thread_status_holder, + ThreadGroupPtr running_group, std::atomic_uint64_t * elapsed_counter_ms); }; diff --git a/src/Processors/Transforms/buildPushingToViewsChain.cpp b/src/Processors/Transforms/buildPushingToViewsChain.cpp index e32e2116f71..0bdd7a88851 100644 --- a/src/Processors/Transforms/buildPushingToViewsChain.cpp +++ b/src/Processors/Transforms/buildPushingToViewsChain.cpp @@ -195,6 +195,7 @@ Chain buildPushingToViewsChain( const ASTPtr & query_ptr, bool no_destination, ThreadStatusesHolderPtr thread_status_holder, + ThreadGroupPtr running_group, std::atomic_uint64_t * elapsed_counter_ms, const Block & live_view_header) { @@ -270,15 +271,6 @@ Chain buildPushingToViewsChain( ASTPtr query; Chain out; - /// NOTE: ThreadGroup always should have context attached, - /// otherwise entry to the system.query_views_log will not be added - /// (see ThreadStatus::logToQueryViewsLog()) - ThreadGroupPtr running_group; - if (current_thread) - running_group = current_thread->getThreadGroup(); - if (!running_group) - running_group = std::make_shared(context); - /// We are creating a ThreadStatus per view to store its metrics individually /// Since calling ThreadStatus() changes current_thread we save it and restore it after the calls /// Later on, before doing any task related to a view, we'll switch to its ThreadStatus, do the work, @@ -354,18 +346,24 @@ Chain buildPushingToViewsChain( runtime_stats->type = QueryViewsLogElement::ViewType::LIVE; query = live_view->getInnerQuery(); // Used only to log in system.query_views_log out = buildPushingToViewsChain( - view, view_metadata_snapshot, insert_context, ASTPtr(), true, thread_status_holder, view_counter_ms, storage_header); + view, view_metadata_snapshot, insert_context, ASTPtr(), + /* no_destination= */ true, + thread_status_holder, running_group, view_counter_ms, storage_header); } else if (auto * window_view = dynamic_cast(view.get())) { runtime_stats->type = QueryViewsLogElement::ViewType::WINDOW; query = window_view->getMergeableQuery(); // Used only to log in system.query_views_log out = buildPushingToViewsChain( - view, view_metadata_snapshot, insert_context, ASTPtr(), true, thread_status_holder, view_counter_ms); + view, view_metadata_snapshot, insert_context, ASTPtr(), + /* no_destination= */ true, + thread_status_holder, running_group, view_counter_ms); } else out = buildPushingToViewsChain( - view, view_metadata_snapshot, insert_context, ASTPtr(), false, thread_status_holder, view_counter_ms); + view, view_metadata_snapshot, insert_context, ASTPtr(), + /* no_destination= */ false, + thread_status_holder, running_group, view_counter_ms); views_data->views.emplace_back(ViewRuntimeData{ std::move(query), diff --git a/src/Processors/Transforms/buildPushingToViewsChain.h b/src/Processors/Transforms/buildPushingToViewsChain.h index 76131a8df6e..a2e7e39ff23 100644 --- a/src/Processors/Transforms/buildPushingToViewsChain.h +++ b/src/Processors/Transforms/buildPushingToViewsChain.h @@ -61,6 +61,10 @@ Chain buildPushingToViewsChain( /// We could specify separate thread_status for each view. /// Needed mainly to collect counters separately. Should be improved. ThreadStatusesHolderPtr thread_status_holder, + /// Usually current_thread->getThreadGroup(), but sometimes ThreadStatus + /// may not have ThreadGroup (i.e. Buffer background flush), and in this + /// case it should be passed outside. + ThreadGroupPtr running_group, /// Counter to measure time spent separately per view. Should be improved. std::atomic_uint64_t * elapsed_counter_ms, /// LiveView executes query itself, it needs source block structure. From 396f6a6421b49f775119663718d1fdc98953e48b Mon Sep 17 00:00:00 2001 From: Sergei Trifonov Date: Fri, 7 Apr 2023 17:07:36 +0200 Subject: [PATCH 055/103] Update src/Storages/StorageReplicatedMergeTree.cpp Co-authored-by: Alexander Tokmakov --- src/Storages/StorageReplicatedMergeTree.cpp | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/src/Storages/StorageReplicatedMergeTree.cpp b/src/Storages/StorageReplicatedMergeTree.cpp index 600168d2637..1f081523ed9 100644 --- a/src/Storages/StorageReplicatedMergeTree.cpp +++ b/src/Storages/StorageReplicatedMergeTree.cpp @@ -8877,7 +8877,7 @@ bool StorageReplicatedMergeTree::createEmptyPartInsteadOfLost(zkutil::ZooKeeperP if (zookeeper->tryGet(lost_part_count_path, lost_part_count_str, &lost_part_count_stat)) { UInt64 lost_part_count = lost_part_count_str.empty() ? 0 : parse(lost_part_count_str); - ops.emplace_back(zkutil::makeSetRequest(lost_part_count_path, fmt::format("{}", lost_part_count + 1), lost_part_count_stat.version)); + ops.emplace_back(zkutil::makeSetRequest(lost_part_count_path, toString(lost_part_count + 1), lost_part_count_stat.version)); } else { From b88fef12dee8d40caf5b73fe0efd20a185190078 Mon Sep 17 00:00:00 2001 From: lzydmxy <13126752315@163.com> Date: Fri, 7 Apr 2023 23:57:08 +0800 Subject: [PATCH 056/103] update Failed tests --- tests/integration/test_backup_restore_new/test.py | 8 +++++--- tests/integration/test_backup_restore_on_cluster/test.py | 2 +- .../test_backward_compatibility/test_convert_ordinary.py | 4 ++-- tests/integration/test_dictionaries_access/test.py | 2 +- tests/integration/test_dictionaries_ddl/test.py | 2 ++ .../integration/test_dictionaries_dependency_xml/test.py | 4 ++-- .../configs/disable_check_dictionary_primary_key.xml | 7 +++++++ tests/integration/test_mask_sensitive_info/test.py | 3 +++ tests/integration/test_replicated_database/test.py | 6 +++--- .../01042_system_reload_dictionary_reloads_completely.sh | 3 ++- .../01043_dictionary_attribute_properties_values.sql | 3 ++- tests/queries/0_stateless/01160_table_dependencies.sh | 6 +++--- .../02364_dictionary_datetime_64_attribute_crash.sql | 3 ++- ...90_prometheus_ClickHouseStatusInfo_DictionaryStatus.sh | 2 +- .../0_stateless/02391_hashed_dictionary_shards.sql | 3 ++- .../02525_range_hashed_dictionary_update_field.sql | 3 ++- 16 files changed, 40 insertions(+), 21 deletions(-) create mode 100644 tests/integration/test_mask_sensitive_info/configs/disable_check_dictionary_primary_key.xml diff --git a/tests/integration/test_backup_restore_new/test.py b/tests/integration/test_backup_restore_new/test.py index ed9747f940b..e05fb5304f4 100644 --- a/tests/integration/test_backup_restore_new/test.py +++ b/tests/integration/test_backup_restore_new/test.py @@ -689,7 +689,7 @@ def test_dependencies(): "CREATE DICTIONARY test.dict1(x UInt32, w String) PRIMARY KEY x SOURCE(CLICKHOUSE(HOST 'localhost' PORT tcpPort() DB 'test' TABLE 'view')) LAYOUT(FLAT()) LIFETIME(0)" ) instance.query( - "CREATE DICTIONARY test.dict2(x UInt32, w String) PRIMARY KEY w SOURCE(CLICKHOUSE(HOST 'localhost' PORT tcpPort() DB 'test' TABLE 'dict1')) LAYOUT(FLAT()) LIFETIME(0)" + "CREATE DICTIONARY test.dict2(x UInt32, w String) PRIMARY KEY w SOURCE(CLICKHOUSE(HOST 'localhost' PORT tcpPort() DB 'test' TABLE 'dict1')) LAYOUT(FLAT()) LIFETIME(0) SETTINGS(check_dictionary_primary_key = 0)" ) instance.query( "CREATE TABLE test.table2(k String, v Int32 DEFAULT dictGet('test.dict2', 'x', k) - 1) ENGINE=MergeTree ORDER BY tuple()" @@ -1421,7 +1421,8 @@ def test_tables_dependency(): instance.query(f"CREATE MATERIALIZED VIEW {t3} TO {t2} AS SELECT x, y FROM {t1}") instance.query( - f"CREATE DICTIONARY {t4} (x Int64, y String) PRIMARY KEY x SOURCE(CLICKHOUSE(HOST 'localhost' PORT tcpPort() TABLE '{t1.split('.')[1]}' DB '{t1.split('.')[0]}')) LAYOUT(FLAT()) LIFETIME(4)" + f"CREATE DICTIONARY {t4} (x Int64, y String) PRIMARY KEY x SOURCE(CLICKHOUSE(HOST 'localhost' PORT tcpPort() TABLE '{t1.split('.')[1]}' DB '{t1.split('.')[0]}')) " + f"LAYOUT(FLAT()) LIFETIME(4) SETTINGS(check_dictionary_primary_key = 0)" ) instance.query(f"CREATE TABLE {t5} AS dictionary({t4})") @@ -1433,7 +1434,8 @@ def test_tables_dependency(): instance.query(f"CREATE VIEW {t7} AS SELECT sum(x) FROM (SELECT x FROM {t6})") instance.query( - f"CREATE DICTIONARY {t8} (x Int64, y String) PRIMARY KEY x SOURCE(CLICKHOUSE(TABLE '{t1.split('.')[1]}' DB '{t1.split('.')[0]}')) LAYOUT(FLAT()) LIFETIME(9)" + f"CREATE DICTIONARY {t8} (x Int64, y String) PRIMARY KEY x SOURCE(CLICKHOUSE(TABLE '{t1.split('.')[1]}' DB '{t1.split('.')[0]}')) " + f"LAYOUT(FLAT()) LIFETIME(9) SETTINGS(check_dictionary_primary_key = 0)" ) instance.query(f"CREATE TABLE {t9}(a Int64) ENGINE=Log") diff --git a/tests/integration/test_backup_restore_on_cluster/test.py b/tests/integration/test_backup_restore_on_cluster/test.py index 9ed39627d82..8f04e6a5ab0 100644 --- a/tests/integration/test_backup_restore_on_cluster/test.py +++ b/tests/integration/test_backup_restore_on_cluster/test.py @@ -884,7 +884,7 @@ def test_tables_dependency(): node1.query( "CREATE DICTIONARY mydb.dict ON CLUSTER 'cluster' (x Int64, y String) PRIMARY KEY x " - "SOURCE(CLICKHOUSE(HOST 'localhost' PORT tcpPort() DB 'mydb' TABLE 'src')) LAYOUT(FLAT()) LIFETIME(0)" + "SOURCE(CLICKHOUSE(HOST 'localhost' PORT tcpPort() DB 'mydb' TABLE 'src')) LAYOUT(FLAT()) LIFETIME(0) SETTINGS(check_dictionary_primary_key = 0)" ) node1.query( diff --git a/tests/integration/test_backward_compatibility/test_convert_ordinary.py b/tests/integration/test_backward_compatibility/test_convert_ordinary.py index 8b1afd358eb..13033752c4c 100644 --- a/tests/integration/test_backward_compatibility/test_convert_ordinary.py +++ b/tests/integration/test_backward_compatibility/test_convert_ordinary.py @@ -116,12 +116,12 @@ def create_some_tables(db): node.query( "CREATE DICTIONARY {}.d1 (n int DEFAULT 0, m int DEFAULT 1) PRIMARY KEY n " "SOURCE(CLICKHOUSE(HOST 'localhost' PORT 9000 USER 'default' TABLE 'rmt1' PASSWORD '' DB '{}')) " - "LIFETIME(MIN 1 MAX 10) LAYOUT(FLAT())".format(db, db) + "LIFETIME(MIN 1 MAX 10) LAYOUT(FLAT()) SETTINGS(check_dictionary_primary_key = 0)".format(db, db) ) node.query( "CREATE DICTIONARY {}.d2 (n int DEFAULT 0, m int DEFAULT 1) PRIMARY KEY n " "SOURCE(CLICKHOUSE(HOST 'localhost' PORT 9000 USER 'default' TABLE 'rmt2' PASSWORD '' DB '{}')) " - "LIFETIME(MIN 1 MAX 10) LAYOUT(FLAT())".format(db, db) + "LIFETIME(MIN 1 MAX 10) LAYOUT(FLAT()) SETTINGS(check_dictionary_primary_key = 0)".format(db, db) ) node.query( "CREATE TABLE {}.merge (n int) ENGINE=Merge('{}', '(mt)|(mv)')".format(db, db) diff --git a/tests/integration/test_dictionaries_access/test.py b/tests/integration/test_dictionaries_access/test.py index 993c8259f32..d54518477e8 100644 --- a/tests/integration/test_dictionaries_access/test.py +++ b/tests/integration/test_dictionaries_access/test.py @@ -33,7 +33,7 @@ create_query = """ CREATE DICTIONARY test_dict(x Int32, y Int32) PRIMARY KEY x LAYOUT(FLAT()) SOURCE(CLICKHOUSE(HOST 'localhost' PORT 9000 USER 'default' TABLE 'test_table' DB 'default')) - LIFETIME(0) + LIFETIME(0) SETTINGS(check_dictionary_primary_key = 0) """ drop_query = "DROP DICTIONARY test_dict" diff --git a/tests/integration/test_dictionaries_ddl/test.py b/tests/integration/test_dictionaries_ddl/test.py index 7dda6fc245a..6ae527223a5 100644 --- a/tests/integration/test_dictionaries_ddl/test.py +++ b/tests/integration/test_dictionaries_ddl/test.py @@ -156,6 +156,7 @@ def test_create_and_select_mysql(started_cluster, clickhouse, name, layout): )) {} LIFETIME(MIN 1 MAX 3) + SETTINGS(check_dictionary_primary_key = 0) """.format( name, name, layout ) @@ -398,6 +399,7 @@ def test_dictionary_with_where(started_cluster): )) LAYOUT(FLAT()) LIFETIME(MIN 1 MAX 3) + SETTINGS(check_dictionary_primary_key = 0) """ ) diff --git a/tests/integration/test_dictionaries_dependency_xml/test.py b/tests/integration/test_dictionaries_dependency_xml/test.py index 3f4c3320920..0f75aefa97a 100644 --- a/tests/integration/test_dictionaries_dependency_xml/test.py +++ b/tests/integration/test_dictionaries_dependency_xml/test.py @@ -114,7 +114,7 @@ def test_dependent_tables(started_cluster): query( "create dictionary a.d (n int default 0, m int default 42) primary key n " "source(clickhouse(host 'localhost' port tcpPort() user 'default' table 'src' password '' db 'lazy'))" - "lifetime(min 1 max 10) layout(flat())" + "lifetime(min 1 max 10) layout(flat()) SETTINGS(check_dictionary_primary_key = 0)" ) query("create table system.join (n int, m int) engine=Join(any, left, n)") query("insert into system.join values (1, 1)") @@ -126,7 +126,7 @@ def test_dependent_tables(started_cluster): query( "create dictionary test.d (n int default 0, m int default 42) primary key n " "source(clickhouse(host 'localhost' port tcpPort() user 'default' table 'src' password '' db 'default'))" - "lifetime(min 1 max 10) layout(flat())" + "lifetime(min 1 max 10) layout(flat()) SETTINGS(check_dictionary_primary_key = 0)" ) query( "create table join (n int, m default dictGet('a.d', 'm', toUInt64(3))," diff --git a/tests/integration/test_mask_sensitive_info/configs/disable_check_dictionary_primary_key.xml b/tests/integration/test_mask_sensitive_info/configs/disable_check_dictionary_primary_key.xml new file mode 100644 index 00000000000..c48d5f3f3ef --- /dev/null +++ b/tests/integration/test_mask_sensitive_info/configs/disable_check_dictionary_primary_key.xml @@ -0,0 +1,7 @@ + + + + 0 + + + diff --git a/tests/integration/test_mask_sensitive_info/test.py b/tests/integration/test_mask_sensitive_info/test.py index 92232f7e6a8..597dc162c35 100644 --- a/tests/integration/test_mask_sensitive_info/test.py +++ b/tests/integration/test_mask_sensitive_info/test.py @@ -9,6 +9,9 @@ node = cluster.add_instance( main_configs=[ "configs/named_collections.xml", ], + user_configs=[ + "configs/disable_check_dictionary_primary_key.xml" + ], with_zookeeper=True, ) diff --git a/tests/integration/test_replicated_database/test.py b/tests/integration/test_replicated_database/test.py index 2ab2fe499ff..16a9c55cc56 100644 --- a/tests/integration/test_replicated_database/test.py +++ b/tests/integration/test_replicated_database/test.py @@ -698,12 +698,12 @@ def create_some_tables(db): main_node.query( f"CREATE DICTIONARY {db}.d1 (n int DEFAULT 0, m int DEFAULT 1) PRIMARY KEY n " "SOURCE(CLICKHOUSE(HOST 'localhost' PORT 9000 USER 'default' TABLE 'rmt1' PASSWORD '' DB 'recover')) " - "LIFETIME(MIN 1 MAX 10) LAYOUT(FLAT())" + "LIFETIME(MIN 1 MAX 10) LAYOUT(FLAT()) SETTINGS(check_dictionary_primary_key = 0)" ) dummy_node.query( f"CREATE DICTIONARY {db}.d2 (n int DEFAULT 0, m int DEFAULT 1) PRIMARY KEY n " "SOURCE(CLICKHOUSE(HOST 'localhost' PORT 9000 USER 'default' TABLE 'rmt2' PASSWORD '' DB 'recover')) " - "LIFETIME(MIN 1 MAX 10) LAYOUT(FLAT())" + "LIFETIME(MIN 1 MAX 10) LAYOUT(FLAT()) SETTINGS(check_dictionary_primary_key = 0)" ) @@ -766,7 +766,7 @@ def test_recover_staled_replica(started_cluster): main_node.query_with_retry( "CREATE DICTIONARY recover.d2 (n int DEFAULT 0, m int DEFAULT 1) PRIMARY KEY n " "SOURCE(CLICKHOUSE(HOST 'localhost' PORT 9000 USER 'default' TABLE 'rmt1' PASSWORD '' DB 'recover')) " - "LIFETIME(MIN 1 MAX 10) LAYOUT(FLAT());", + "LIFETIME(MIN 1 MAX 10) LAYOUT(FLAT()) SETTINGS(check_dictionary_primary_key = 0);", settings=settings, ) 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 f2b30e05040..3cd945cfee3 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 @@ -23,7 +23,8 @@ CREATE DICTIONARY dictdb_01042.dict PRIMARY KEY x SOURCE(CLICKHOUSE(HOST 'localhost' PORT tcpPort() USER 'default' TABLE 'table' DB 'dictdb_01042' UPDATE_FIELD 'insert_time')) LAYOUT(FLAT()) -LIFETIME(1); +LIFETIME(1) +SETTINGS(check_dictionary_primary_key = 0); EOF $CLICKHOUSE_CLIENT --query "SELECT '12 -> ', dictGetInt64('dictdb_01042.dict', 'y', toUInt64(12))" diff --git a/tests/queries/0_stateless/01043_dictionary_attribute_properties_values.sql b/tests/queries/0_stateless/01043_dictionary_attribute_properties_values.sql index 5d629d4e8db..9d7c5d5196d 100644 --- a/tests/queries/0_stateless/01043_dictionary_attribute_properties_values.sql +++ b/tests/queries/0_stateless/01043_dictionary_attribute_properties_values.sql @@ -17,7 +17,8 @@ CREATE DICTIONARY dictdb_01043.dict PRIMARY KEY key SOURCE(CLICKHOUSE(HOST 'localhost' PORT tcpPort() USER 'default' TABLE 'dicttbl' DB 'dictdb_01043')) LAYOUT(FLAT()) -LIFETIME(1); +LIFETIME(1) +SETTINGS(check_dictionary_primary_key = 0); SELECT dictGetString('dictdb_01043.dict', 'value_default', toUInt64(12)); diff --git a/tests/queries/0_stateless/01160_table_dependencies.sh b/tests/queries/0_stateless/01160_table_dependencies.sh index a0a3f05c6a9..650b89816d7 100755 --- a/tests/queries/0_stateless/01160_table_dependencies.sh +++ b/tests/queries/0_stateless/01160_table_dependencies.sh @@ -16,14 +16,14 @@ $CLICKHOUSE_CLIENT -q "create table dict_src (n int, m int, s String) engine=Mer $CLICKHOUSE_CLIENT -q "create dictionary dict1 (n int default 0, m int default 1, s String default 'qqq') PRIMARY KEY n SOURCE(CLICKHOUSE(HOST 'localhost' PORT tcpPort() USER 'default' TABLE 'dict_src' PASSWORD '' DB '$CLICKHOUSE_DATABASE')) -LIFETIME(MIN 1 MAX 10) LAYOUT(FLAT());" +LIFETIME(MIN 1 MAX 10) LAYOUT(FLAT()) SETTINGS(check_dictionary_primary_key = 0);" $CLICKHOUSE_CLIENT -q "create table join(n int, m int default dictGet('$CLICKHOUSE_DATABASE.dict1', 'm', 42::UInt64)) engine=Join(any, left, n);" $CLICKHOUSE_CLIENT -q "create dictionary dict2 (n int default 0, m int DEFAULT 2) PRIMARY KEY n SOURCE(CLICKHOUSE(HOST 'localhost' PORT tcpPort() USER 'default' TABLE 'join' PASSWORD '' DB '$CLICKHOUSE_DATABASE')) -LIFETIME(MIN 1 MAX 10) LAYOUT(FLAT());" +LIFETIME(MIN 1 MAX 10) LAYOUT(FLAT()) SETTINGS(check_dictionary_primary_key = 0);" $CLICKHOUSE_CLIENT -q "create table s (x default joinGet($CLICKHOUSE_DATABASE.join, 'm', 42::int)) engine=Set" @@ -102,7 +102,7 @@ $CLICKHOUSE_CLIENT -q "create table ${CLICKHOUSE_DATABASE}_1.xdict_src (n int, m $CLICKHOUSE_CLIENT -q "create dictionary ${CLICKHOUSE_DATABASE}_1.ydict1 (n int default 0, m int default 1, s String default 'qqq') PRIMARY KEY n SOURCE(CLICKHOUSE(HOST 'localhost' PORT tcpPort() USER 'default' TABLE 'xdict_src' PASSWORD '' DB '${CLICKHOUSE_DATABASE}_1')) -LIFETIME(MIN 1 MAX 10) LAYOUT(FLAT());" +LIFETIME(MIN 1 MAX 10) LAYOUT(FLAT()) SETTINGS(check_dictionary_primary_key = 0);" $CLICKHOUSE_CLIENT -q "create table ${CLICKHOUSE_DATABASE}_1.zjoin(n int, m int default dictGet('${CLICKHOUSE_DATABASE}_1.ydict1', 'm', 42::UInt64)) engine=Join(any, left, n);" $CLICKHOUSE_CLIENT -q "drop database ${CLICKHOUSE_DATABASE}_1" diff --git a/tests/queries/0_stateless/02364_dictionary_datetime_64_attribute_crash.sql b/tests/queries/0_stateless/02364_dictionary_datetime_64_attribute_crash.sql index 77fc9e1183b..6dcea0ac442 100644 --- a/tests/queries/0_stateless/02364_dictionary_datetime_64_attribute_crash.sql +++ b/tests/queries/0_stateless/02364_dictionary_datetime_64_attribute_crash.sql @@ -9,7 +9,8 @@ CREATE DICTIONARY datDictionary PRIMARY KEY blockNum SOURCE(CLICKHOUSE(TABLE 'dat')) LIFETIME(MIN 0 MAX 1000) -LAYOUT(FLAT()); +LAYOUT(FLAT()) +SETTINGS(check_dictionary_primary_key = 0); select (select eventTimestamp from datDictionary); select count(*) from dat where eventTimestamp >= (select eventTimestamp from datDictionary); diff --git a/tests/queries/0_stateless/02390_prometheus_ClickHouseStatusInfo_DictionaryStatus.sh b/tests/queries/0_stateless/02390_prometheus_ClickHouseStatusInfo_DictionaryStatus.sh index 65025858e20..92879ae9cb4 100755 --- a/tests/queries/0_stateless/02390_prometheus_ClickHouseStatusInfo_DictionaryStatus.sh +++ b/tests/queries/0_stateless/02390_prometheus_ClickHouseStatusInfo_DictionaryStatus.sh @@ -13,7 +13,7 @@ function get_dictionary_status() } } -$CLICKHOUSE_CLIENT -q "CREATE DICTIONARY dict (key Int, value String) PRIMARY KEY key SOURCE(CLICKHOUSE(TABLE data)) LAYOUT(HASHED()) LIFETIME(0)" +$CLICKHOUSE_CLIENT -q "CREATE DICTIONARY dict (key Int, value String) PRIMARY KEY key SOURCE(CLICKHOUSE(TABLE data)) LAYOUT(HASHED()) LIFETIME(0) SETTINGS(check_dictionary_primary_key = 0)" uuid="$($CLICKHOUSE_CLIENT -q "SELECT uuid FROM system.dictionaries WHERE database = '$CLICKHOUSE_DATABASE' AND name = 'dict'")" echo 'status before reload' diff --git a/tests/queries/0_stateless/02391_hashed_dictionary_shards.sql b/tests/queries/0_stateless/02391_hashed_dictionary_shards.sql index ac43c12afc0..48f920e00eb 100644 --- a/tests/queries/0_stateless/02391_hashed_dictionary_shards.sql +++ b/tests/queries/0_stateless/02391_hashed_dictionary_shards.sql @@ -87,7 +87,8 @@ CREATE DICTIONARY test_dictionary_10_shards_string ) PRIMARY KEY key SOURCE(CLICKHOUSE(TABLE test_table_string)) LAYOUT(SPARSE_HASHED(SHARDS 10)) -LIFETIME(0); +LIFETIME(0) +SETTINGS(check_dictionary_primary_key = 0); SYSTEM RELOAD DICTIONARY test_dictionary_10_shards_string; -- { serverError CANNOT_PARSE_TEXT } diff --git a/tests/queries/0_stateless/02525_range_hashed_dictionary_update_field.sql b/tests/queries/0_stateless/02525_range_hashed_dictionary_update_field.sql index 2534333afcd..9f36f69236c 100644 --- a/tests/queries/0_stateless/02525_range_hashed_dictionary_update_field.sql +++ b/tests/queries/0_stateless/02525_range_hashed_dictionary_update_field.sql @@ -18,7 +18,8 @@ CREATE DICTIONARY test_dictionary LAYOUT(RANGE_HASHED()) RANGE(MIN start MAX end) SOURCE(CLICKHOUSE(TABLE 'test_table' UPDATE_FIELD 'insert_time' UPDATE_LAG 10)) -LIFETIME(MIN 1 MAX 2); +LIFETIME(MIN 1 MAX 2) +SETTINGS(check_dictionary_primary_key = 0); INSERT INTO test_table VALUES (1, 0, 100, '2022-12-26 11:38:34'), (1, 101, 200, '2022-12-26 11:38:34'), (2, 0, 999, '2022-12-26 11:38:34'), (2, 1000, 10000, '2022-12-26 11:38:34'); From f951ff2968221507cbc88e5c466c3aa2fd9de318 Mon Sep 17 00:00:00 2001 From: lzydmxy <13126752315@163.com> Date: Sat, 8 Apr 2023 00:38:44 +0800 Subject: [PATCH 057/103] fix format --- .../test_backward_compatibility/test_convert_ordinary.py | 8 ++++++-- tests/integration/test_mask_sensitive_info/test.py | 2 +- 2 files changed, 7 insertions(+), 3 deletions(-) diff --git a/tests/integration/test_backward_compatibility/test_convert_ordinary.py b/tests/integration/test_backward_compatibility/test_convert_ordinary.py index 13033752c4c..dd0699fe3af 100644 --- a/tests/integration/test_backward_compatibility/test_convert_ordinary.py +++ b/tests/integration/test_backward_compatibility/test_convert_ordinary.py @@ -116,12 +116,16 @@ def create_some_tables(db): node.query( "CREATE DICTIONARY {}.d1 (n int DEFAULT 0, m int DEFAULT 1) PRIMARY KEY n " "SOURCE(CLICKHOUSE(HOST 'localhost' PORT 9000 USER 'default' TABLE 'rmt1' PASSWORD '' DB '{}')) " - "LIFETIME(MIN 1 MAX 10) LAYOUT(FLAT()) SETTINGS(check_dictionary_primary_key = 0)".format(db, db) + "LIFETIME(MIN 1 MAX 10) LAYOUT(FLAT()) SETTINGS(check_dictionary_primary_key = 0)".format( + db, db + ) ) node.query( "CREATE DICTIONARY {}.d2 (n int DEFAULT 0, m int DEFAULT 1) PRIMARY KEY n " "SOURCE(CLICKHOUSE(HOST 'localhost' PORT 9000 USER 'default' TABLE 'rmt2' PASSWORD '' DB '{}')) " - "LIFETIME(MIN 1 MAX 10) LAYOUT(FLAT()) SETTINGS(check_dictionary_primary_key = 0)".format(db, db) + "LIFETIME(MIN 1 MAX 10) LAYOUT(FLAT()) SETTINGS(check_dictionary_primary_key = 0)".format( + db, db + ) ) node.query( "CREATE TABLE {}.merge (n int) ENGINE=Merge('{}', '(mt)|(mv)')".format(db, db) diff --git a/tests/integration/test_mask_sensitive_info/test.py b/tests/integration/test_mask_sensitive_info/test.py index 597dc162c35..1806134c486 100644 --- a/tests/integration/test_mask_sensitive_info/test.py +++ b/tests/integration/test_mask_sensitive_info/test.py @@ -10,7 +10,7 @@ node = cluster.add_instance( "configs/named_collections.xml", ], user_configs=[ - "configs/disable_check_dictionary_primary_key.xml" + "configs/disable_check_dictionary_primary_key.xml", ], with_zookeeper=True, ) From 3633f0b2f3d479b884d1e70c2d4bcb0e817d2d19 Mon Sep 17 00:00:00 2001 From: Alexander Tokmakov Date: Fri, 7 Apr 2023 23:50:09 +0200 Subject: [PATCH 058/103] fix --- .github/workflows/backport_branches.yml | 7 +++++++ .github/workflows/master.yml | 7 +++++++ .github/workflows/release_branches.yml | 7 +++++++ docs/en/development/developer-instruction.md | 2 +- docs/ru/development/developer-instruction.md | 2 +- 5 files changed, 23 insertions(+), 2 deletions(-) diff --git a/.github/workflows/backport_branches.yml b/.github/workflows/backport_branches.yml index 0d81a7b303c..d69168b01ee 100644 --- a/.github/workflows/backport_branches.yml +++ b/.github/workflows/backport_branches.yml @@ -349,6 +349,13 @@ jobs: with: clear-repository: true submodules: true + - name: Apply sparse checkout for contrib # in order to check that it doesn't break build + run: | + rm -rf "$GITHUB_WORKSPACE/contrib" && echo 'removed' + git -C "$GITHUB_WORKSPACE" checkout . && echo 'restored' + "$GITHUB_WORKSPACE/contrib/update-submodules.sh" && echo 'OK' + du -hs "$GITHUB_WORKSPACE/contrib" ||: + find "$GITHUB_WORKSPACE/contrib" -type f | wc -l ||: - name: Build run: | sudo rm -fr "$TEMP_PATH" diff --git a/.github/workflows/master.yml b/.github/workflows/master.yml index ecd5b85d320..1182481c897 100644 --- a/.github/workflows/master.yml +++ b/.github/workflows/master.yml @@ -487,6 +487,13 @@ jobs: with: clear-repository: true submodules: true + - name: Apply sparse checkout for contrib # in order to check that it doesn't break build + run: | + rm -rf "$GITHUB_WORKSPACE/contrib" && echo 'removed' + git -C "$GITHUB_WORKSPACE" checkout . && echo 'restored' + "$GITHUB_WORKSPACE/contrib/update-submodules.sh" && echo 'OK' + du -hs "$GITHUB_WORKSPACE/contrib" ||: + find "$GITHUB_WORKSPACE/contrib" -type f | wc -l ||: - name: Build run: | sudo rm -fr "$TEMP_PATH" diff --git a/.github/workflows/release_branches.yml b/.github/workflows/release_branches.yml index 1282dbef50b..21284815583 100644 --- a/.github/workflows/release_branches.yml +++ b/.github/workflows/release_branches.yml @@ -406,6 +406,13 @@ jobs: with: clear-repository: true submodules: true + - name: Apply sparse checkout for contrib # in order to check that it doesn't break build + run: | + rm -rf "$GITHUB_WORKSPACE/contrib" && echo 'removed' + git -C "$GITHUB_WORKSPACE" checkout . && echo 'restored' + "$GITHUB_WORKSPACE/contrib/update-submodules.sh" && echo 'OK' + du -hs "$GITHUB_WORKSPACE/contrib" ||: + find "$GITHUB_WORKSPACE/contrib" -type f | wc -l ||: - name: Build run: | sudo rm -fr "$TEMP_PATH" diff --git a/docs/en/development/developer-instruction.md b/docs/en/development/developer-instruction.md index ea98b2da5e6..6bcdadeb1eb 100644 --- a/docs/en/development/developer-instruction.md +++ b/docs/en/development/developer-instruction.md @@ -39,7 +39,7 @@ Next, you need to download the source files onto your working machine. This is c In the command line terminal run: - git clone --recursive --shallow-submodules git@github.com:your_github_username/ClickHouse.git + git clone --shallow-submodules git@github.com:your_github_username/ClickHouse.git cd ClickHouse Or (if you'd like to use sparse checkout for submodules and avoid checking out unneeded files): diff --git a/docs/ru/development/developer-instruction.md b/docs/ru/development/developer-instruction.md index c208439678a..7294bc2ae87 100644 --- a/docs/ru/development/developer-instruction.md +++ b/docs/ru/development/developer-instruction.md @@ -41,7 +41,7 @@ ClickHouse не работает и не собирается на 32-битны Выполните в терминале: - git clone --recursive --shallow-submodules git@github.com:your_github_username/ClickHouse.git + git clone --shallow-submodules git@github.com:your_github_username/ClickHouse.git cd ClickHouse Или (если вы хотите использовать sparse checkout для submodules): From 7161d3d380e8b083d01898a3c086bdaf5de32696 Mon Sep 17 00:00:00 2001 From: Nikolay Degterinsky Date: Fri, 7 Apr 2023 20:59:12 +0000 Subject: [PATCH 059/103] Fixes --- src/DataTypes/DataTypeDateTime64.cpp | 11 +++++++++++ src/DataTypes/DataTypeDateTime64.h | 15 +-------------- src/DataTypes/IDataType.h | 5 +++-- src/Functions/array/arrayAggregation.cpp | 1 + 4 files changed, 16 insertions(+), 16 deletions(-) diff --git a/src/DataTypes/DataTypeDateTime64.cpp b/src/DataTypes/DataTypeDateTime64.cpp index 8ba8ad1cfec..248eb3638b9 100644 --- a/src/DataTypes/DataTypeDateTime64.cpp +++ b/src/DataTypes/DataTypeDateTime64.cpp @@ -12,6 +12,7 @@ namespace DB namespace ErrorCodes { extern const int ARGUMENT_OUT_OF_BOUND; + extern const int LOGICAL_ERROR; } static constexpr UInt32 max_scale = 9; @@ -56,4 +57,14 @@ SerializationPtr DataTypeDateTime64::doGetDefaultSerialization() const return std::make_shared(scale, *this); } +inline std::string getDateTimeTimezone(const IDataType & data_type) +{ + if (const auto * type = typeid_cast(&data_type)) + return type->hasExplicitTimeZone() ? type->getTimeZone().getTimeZone() : std::string(); + if (const auto * type = typeid_cast(&data_type)) + return type->hasExplicitTimeZone() ? type->getTimeZone().getTimeZone() : std::string(); + + throw Exception(ErrorCodes::LOGICAL_ERROR, "Cannot get decimal scale from type {}", data_type.getName()); +} + } diff --git a/src/DataTypes/DataTypeDateTime64.h b/src/DataTypes/DataTypeDateTime64.h index f3c9b30f232..64547b30be9 100644 --- a/src/DataTypes/DataTypeDateTime64.h +++ b/src/DataTypes/DataTypeDateTime64.h @@ -9,11 +9,6 @@ class DateLUTImpl; namespace DB { -namespace ErrorCodes -{ - extern const int LOGICAL_ERROR; -} - /** DateTime64 is same as DateTime, but it stores values as Int64 and has configurable sub-second part. * * `scale` determines number of decimal places for sub-second part of the DateTime64. @@ -46,15 +41,7 @@ protected: SerializationPtr doGetDefaultSerialization() const override; }; -inline std::string getDateTimeTimezone(const IDataType & data_type) -{ - if (const auto * type = typeid_cast(&data_type)) - return type->hasExplicitTimeZone() ? type->getTimeZone().getTimeZone() : std::string(); - if (const auto * type = typeid_cast(&data_type)) - return type->hasExplicitTimeZone() ? type->getTimeZone().getTimeZone() : std::string(); - - throw Exception(ErrorCodes::LOGICAL_ERROR, "Cannot get decimal scale from type {}", data_type.getName()); -} +inline std::string getDateTimeTimezone(const IDataType & data_type); } diff --git a/src/DataTypes/IDataType.h b/src/DataTypes/IDataType.h index 1b512bca30d..4750db3f9ee 100644 --- a/src/DataTypes/IDataType.h +++ b/src/DataTypes/IDataType.h @@ -577,12 +577,13 @@ template <> inline constexpr bool IsDataTypeDecimal = true; template constexpr bool IsDataTypeNumber> = true; +template <> inline constexpr bool IsDataTypeDate = true; +template <> inline constexpr bool IsDataTypeDate = true; + template <> inline constexpr bool IsDataTypeDateOrDateTime = true; template <> inline constexpr bool IsDataTypeDateOrDateTime = true; template <> inline constexpr bool IsDataTypeDateOrDateTime = true; template <> inline constexpr bool IsDataTypeDateOrDateTime = true; -template <> inline constexpr bool IsDataTypeDate = true; -template <> inline constexpr bool IsDataTypeDate = true; template class DataTypeEnum; diff --git a/src/Functions/array/arrayAggregation.cpp b/src/Functions/array/arrayAggregation.cpp index 238d46202aa..1fe02870e4d 100644 --- a/src/Functions/array/arrayAggregation.cpp +++ b/src/Functions/array/arrayAggregation.cpp @@ -399,6 +399,7 @@ struct ArrayAggregateImpl executeType(mapped, offsets, res) || executeType(mapped, offsets, res) || executeType(mapped, offsets, res)) + { return res; } else From 6debaccd85c34a7e777f5bd9241abf4e09490540 Mon Sep 17 00:00:00 2001 From: Nikolay Degterinsky Date: Sun, 9 Apr 2023 11:12:08 +0000 Subject: [PATCH 060/103] Fix build --- src/DataTypes/DataTypeDateTime64.cpp | 4 +-- src/DataTypes/DataTypeDateTime64.h | 2 +- src/Functions/array/arrayAggregation.cpp | 44 +++++++++++++----------- 3 files changed, 27 insertions(+), 23 deletions(-) diff --git a/src/DataTypes/DataTypeDateTime64.cpp b/src/DataTypes/DataTypeDateTime64.cpp index 248eb3638b9..124fea1f458 100644 --- a/src/DataTypes/DataTypeDateTime64.cpp +++ b/src/DataTypes/DataTypeDateTime64.cpp @@ -57,14 +57,14 @@ SerializationPtr DataTypeDateTime64::doGetDefaultSerialization() const return std::make_shared(scale, *this); } -inline std::string getDateTimeTimezone(const IDataType & data_type) +std::string getDateTimeTimezone(const IDataType & data_type) { if (const auto * type = typeid_cast(&data_type)) return type->hasExplicitTimeZone() ? type->getTimeZone().getTimeZone() : std::string(); if (const auto * type = typeid_cast(&data_type)) return type->hasExplicitTimeZone() ? type->getTimeZone().getTimeZone() : std::string(); - throw Exception(ErrorCodes::LOGICAL_ERROR, "Cannot get decimal scale from type {}", data_type.getName()); + throw Exception(ErrorCodes::LOGICAL_ERROR, "Cannot get time zone from type {}", data_type.getName()); } } diff --git a/src/DataTypes/DataTypeDateTime64.h b/src/DataTypes/DataTypeDateTime64.h index 64547b30be9..aaa99485040 100644 --- a/src/DataTypes/DataTypeDateTime64.h +++ b/src/DataTypes/DataTypeDateTime64.h @@ -41,7 +41,7 @@ protected: SerializationPtr doGetDefaultSerialization() const override; }; -inline std::string getDateTimeTimezone(const IDataType & data_type); +std::string getDateTimeTimezone(const IDataType & data_type); } diff --git a/src/Functions/array/arrayAggregation.cpp b/src/Functions/array/arrayAggregation.cpp index 1fe02870e4d..e13b5b1d2f1 100644 --- a/src/Functions/array/arrayAggregation.cpp +++ b/src/Functions/array/arrayAggregation.cpp @@ -84,9 +84,10 @@ struct ArrayAggregateResultImpl std::conditional_t, Decimal128, std::conditional_t, Decimal128, std::conditional_t, Decimal256, + std::conditional_t, Decimal128, std::conditional_t, Float64, std::conditional_t, Int64, - UInt64>>>>>>>>>>; + UInt64>>>>>>>>>>>; }; template @@ -111,28 +112,31 @@ struct ArrayAggregateImpl using Types = std::decay_t; using DataType = typename Types::LeftType; - if constexpr (aggregate_operation == AggregateOperation::average || aggregate_operation == AggregateOperation::product) + if constexpr (!IsDataTypeDateOrDateTime) { - result = std::make_shared(); + if constexpr (aggregate_operation == AggregateOperation::average || aggregate_operation == AggregateOperation::product) + { + result = std::make_shared(); - return true; - } - else if constexpr (IsDataTypeNumber) - { - using NumberReturnType = ArrayAggregateResult; - result = std::make_shared>(); + return true; + } + else if constexpr (IsDataTypeNumber) + { + using NumberReturnType = ArrayAggregateResult; + result = std::make_shared>(); - return true; - } - else if constexpr (IsDataTypeDecimal && !IsDataTypeDateOrDateTime) - { - using DecimalReturnType = ArrayAggregateResult; - UInt32 scale = getDecimalScale(*expression_return); - result = std::make_shared>(DecimalUtils::max_precision, scale); + return true; + } + else if constexpr (IsDataTypeDecimal) + { + using DecimalReturnType = ArrayAggregateResult; + UInt32 scale = getDecimalScale(*expression_return); + result = std::make_shared>(DecimalUtils::max_precision, scale); - return true; + return true; + } } - else if constexpr (IsDataTypeDateOrDateTime && (aggregate_operation == AggregateOperation::max || aggregate_operation == AggregateOperation::min)) + else if constexpr (aggregate_operation == AggregateOperation::max || aggregate_operation == AggregateOperation::min) { if constexpr (IsDataTypeDate) { @@ -143,7 +147,7 @@ struct ArrayAggregateImpl else if constexpr (!IsDataTypeDecimal) { std::string timezone = getDateTimeTimezone(*expression_return); - result = std::make_shared(timezone); + result = std::make_shared(timezone); return true; } @@ -151,7 +155,7 @@ struct ArrayAggregateImpl { std::string timezone = getDateTimeTimezone(*expression_return); UInt32 scale = getDecimalScale(*expression_return); - result = std::make_shared(scale, timezone); + result = std::make_shared(scale, timezone); return true; } From 10883d8af5b407c5bbcf3b49bc0db76f558afa14 Mon Sep 17 00:00:00 2001 From: Nikolay Degterinsky Date: Sun, 9 Apr 2023 12:52:23 +0000 Subject: [PATCH 061/103] Allow commas with subqueries, add comments --- src/Parsers/ExpressionListParsers.cpp | 96 +++++++++++-------- .../02676_trailing_commas.reference | 3 +- .../0_stateless/02676_trailing_commas.sql | 3 +- 3 files changed, 59 insertions(+), 43 deletions(-) diff --git a/src/Parsers/ExpressionListParsers.cpp b/src/Parsers/ExpressionListParsers.cpp index 6008e89d038..0069821c949 100644 --- a/src/Parsers/ExpressionListParsers.cpp +++ b/src/Parsers/ExpressionListParsers.cpp @@ -779,6 +779,41 @@ protected: }; +struct ParserExpressionImpl +{ + static std::vector> operators_table; + static std::vector> unary_operators_table; + static const char * overlapping_operators_to_skip[]; + + static Operator finish_between_operator; + + ParserCompoundIdentifier identifier_parser{false, true}; + ParserNumber number_parser; + ParserAsterisk asterisk_parser; + ParserLiteral literal_parser; + ParserTupleOfLiterals tuple_literal_parser; + ParserArrayOfLiterals array_literal_parser; + ParserSubstitution substitution_parser; + ParserMySQLGlobalVariable mysql_global_variable_parser; + + ParserKeyword any_parser{"ANY"}; + ParserKeyword all_parser{"ALL"}; + + // Recursion + ParserQualifiedAsterisk qualified_asterisk_parser; + ParserColumnsMatcher columns_matcher_parser; + ParserQualifiedColumnsMatcher qualified_columns_matcher_parser; + ParserSubquery subquery_parser; + + bool parse(std::unique_ptr start, IParser::Pos & pos, ASTPtr & node, Expected & expected); + + using Layers = std::vector>; + + Action tryParseOperand(Layers & layers, IParser::Pos & pos, Expected & expected); + Action tryParseOperator(Layers & layers, IParser::Pos & pos, Expected & expected); +}; + + class ExpressionLayer : public Layer { public: @@ -801,7 +836,7 @@ public: return Layer::getResultImpl(node); } - bool parse(IParser::Pos & pos, Expected & expected, Action & /*action*/) override + bool parse(IParser::Pos & pos, Expected & /*expected*/, Action & /*action*/) override { if (pos->type == TokenType::Comma) { @@ -814,27 +849,40 @@ public: /// - SELECT a, b, c, FROM table /// - SELECT 1, - /// For this purpose we eliminate the following cases: + /// For this purpose we need to eliminate the following cases: /// 1. WITH 1 AS from SELECT 2, from /// 2. SELECT to, from FROM table /// 3. SELECT to, from AS alias FROM table - /// 4. SELECT to, from + to FROM table + /// 4. SELECT to, from + to, from IN [1,2,3], FROM table + Expected test_expected; auto test_pos = pos; ++test_pos; + /// End of query if (test_pos.isValid() && test_pos->type != TokenType::Semicolon) { - if (!ParserKeyword("FROM").ignore(test_pos, expected)) + /// If we can't parse FROM then return + if (!ParserKeyword("FROM").ignore(test_pos, test_expected)) return true; - if (ParserKeyword("FROM").ignore(test_pos, expected)) + /// If we parse a second FROM then the first one was a name of a column + if (ParserKeyword("FROM").ignore(test_pos, test_expected)) return true; - if (ParserAlias(false).ignore(test_pos, expected)) + /// If we parse an explicit alias to FROM, then it was a name of a column + if (ParserAlias(false).ignore(test_pos, test_expected)) return true; - if (!ParserIdentifier(true).ignore(test_pos, expected)) + /// If we parse an operator after FROM then it was a name of a column + auto cur_op = ParserExpressionImpl::operators_table.begin(); + for (; cur_op != ParserExpressionImpl::operators_table.end(); ++cur_op) + { + if (parseOperator(test_pos, cur_op->first, test_expected)) + break; + } + + if (cur_op != ParserExpressionImpl::operators_table.end()) return true; } @@ -2205,40 +2253,6 @@ bool ParseTimestampOperatorExpression(IParser::Pos & pos, ASTPtr & node, Expecte return true; } -struct ParserExpressionImpl -{ - static std::vector> operators_table; - static std::vector> unary_operators_table; - static const char * overlapping_operators_to_skip[]; - - static Operator finish_between_operator; - - ParserCompoundIdentifier identifier_parser{false, true}; - ParserNumber number_parser; - ParserAsterisk asterisk_parser; - ParserLiteral literal_parser; - ParserTupleOfLiterals tuple_literal_parser; - ParserArrayOfLiterals array_literal_parser; - ParserSubstitution substitution_parser; - ParserMySQLGlobalVariable mysql_global_variable_parser; - - ParserKeyword any_parser{"ANY"}; - ParserKeyword all_parser{"ALL"}; - - // Recursion - ParserQualifiedAsterisk qualified_asterisk_parser; - ParserColumnsMatcher columns_matcher_parser; - ParserQualifiedColumnsMatcher qualified_columns_matcher_parser; - ParserSubquery subquery_parser; - - bool parse(std::unique_ptr start, IParser::Pos & pos, ASTPtr & node, Expected & expected); - - using Layers = std::vector>; - - Action tryParseOperand(Layers & layers, IParser::Pos & pos, Expected & expected); - Action tryParseOperator(Layers & layers, IParser::Pos & pos, Expected & expected); -}; - bool ParserExpression::parseImpl(Pos & pos, ASTPtr & node, Expected & expected) { diff --git a/tests/queries/0_stateless/02676_trailing_commas.reference b/tests/queries/0_stateless/02676_trailing_commas.reference index 41ace3e47aa..76d173ca23e 100644 --- a/tests/queries/0_stateless/02676_trailing_commas.reference +++ b/tests/queries/0_stateless/02676_trailing_commas.reference @@ -1,4 +1,5 @@ 1 1 1 -1 2 +1 2 0 +1 diff --git a/tests/queries/0_stateless/02676_trailing_commas.sql b/tests/queries/0_stateless/02676_trailing_commas.sql index 5e2dafccb46..048405c4d20 100644 --- a/tests/queries/0_stateless/02676_trailing_commas.sql +++ b/tests/queries/0_stateless/02676_trailing_commas.sql @@ -1,4 +1,5 @@ SELECT 1,; SELECT 1, FROM numbers(1); WITH 1 as a SELECT a, FROM numbers(1); -WITH 1 as from SELECT from, from + from, FROM numbers(1); +WITH 1 as from SELECT from, from + from, from in [0], FROM numbers(1); +SELECT n, FROM (SELECT 1 AS n); From 553326c999d2febe1316ad372bab976f1f30043b Mon Sep 17 00:00:00 2001 From: serxa Date: Sun, 9 Apr 2023 15:13:40 +0000 Subject: [PATCH 062/103] add test --- .../0_stateless/01165_lost_part_empty_partition.reference | 2 ++ tests/queries/0_stateless/01165_lost_part_empty_partition.sql | 2 ++ 2 files changed, 4 insertions(+) diff --git a/tests/queries/0_stateless/01165_lost_part_empty_partition.reference b/tests/queries/0_stateless/01165_lost_part_empty_partition.reference index e69de29bb2d..6ed281c757a 100644 --- a/tests/queries/0_stateless/01165_lost_part_empty_partition.reference +++ b/tests/queries/0_stateless/01165_lost_part_empty_partition.reference @@ -0,0 +1,2 @@ +1 +1 diff --git a/tests/queries/0_stateless/01165_lost_part_empty_partition.sql b/tests/queries/0_stateless/01165_lost_part_empty_partition.sql index dc41b15118f..924798b0050 100644 --- a/tests/queries/0_stateless/01165_lost_part_empty_partition.sql +++ b/tests/queries/0_stateless/01165_lost_part_empty_partition.sql @@ -10,6 +10,7 @@ insert into rmt1 values (now(), rand()); drop table rmt1; system sync replica rmt2; +select lost_part_count from system.replicas where database = currentDatabase() and table = 'rmt2'; drop table rmt2; @@ -21,6 +22,7 @@ insert into rmt1 values (now(), rand()); drop table rmt1; system sync replica rmt2; +select lost_part_count from system.replicas where database = currentDatabase() and table = 'rmt2'; drop table rmt2; From 000c2b5b9128ce9c2c08d530aa0b14e5f91ed7a6 Mon Sep 17 00:00:00 2001 From: Aleksei Filatov Date: Mon, 10 Apr 2023 13:02:31 +0300 Subject: [PATCH 063/103] Correct FixupACL for auth scheme --- src/Coordination/KeeperStorage.cpp | 71 +++++++++++++++++++++--------- src/Coordination/KeeperStorage.h | 4 ++ 2 files changed, 54 insertions(+), 21 deletions(-) diff --git a/src/Coordination/KeeperStorage.cpp b/src/Coordination/KeeperStorage.cpp index 41a6af54204..6b2696034f1 100644 --- a/src/Coordination/KeeperStorage.cpp +++ b/src/Coordination/KeeperStorage.cpp @@ -61,16 +61,10 @@ String getSHA1(const String & userdata) return String{digest_id.begin(), digest_id.end()}; } -String generateDigest(const String & userdata) -{ - std::vector user_password; - boost::split(user_password, userdata, [](char character) { return character == ':'; }); - return user_password[0] + ":" + base64Encode(getSHA1(userdata)); -} - bool fixupACL( const std::vector & request_acls, - const std::vector & current_ids, + int64_t session_id, + const KeeperStorage::UncommittedState & uncommitted_state, std::vector & result_acls) { if (request_acls.empty()) @@ -81,14 +75,18 @@ bool fixupACL( { if (request_acl.scheme == "auth") { - for (const auto & current_id : current_ids) - { - valid_found = true; - Coordination::ACL new_acl = request_acl; - new_acl.scheme = current_id.scheme; - new_acl.id = current_id.id; - result_acls.push_back(new_acl); - } + uncommitted_state.forEachAuthInSession( + session_id, + [&](const KeeperStorage::AuthID & auth_id) + { + valid_found = true; + Coordination::ACL new_acl = request_acl; + + new_acl.scheme = auth_id.scheme; + new_acl.id = auth_id.id; + + result_acls.push_back(new_acl); + }); } else if (request_acl.scheme == "world" && request_acl.id == "anyone") { @@ -564,6 +562,32 @@ Coordination::ACLs KeeperStorage::UncommittedState::getACLs(StringRef path) cons return storage.acl_map.convertNumber(node_it->value.acl_id); } +void KeeperStorage::UncommittedState::forEachAuthInSession(int64_t session_id, std::function func) const +{ + const auto call_for_each_auth = [&func](const auto & auth_ids) + { + for (const auto & auth : auth_ids) + { + using TAuth = std::remove_reference_t; + + const AuthID * auth_ptr = nullptr; + if constexpr (std::is_pointer_v) + auth_ptr = auth; + else + auth_ptr = &auth; + + func(*auth_ptr); + } + }; + + // for committed + if (storage.session_and_auth.contains(session_id)) + call_for_each_auth(storage.session_and_auth.at(session_id)); + // for uncommitted + if (session_and_auth.contains(session_id)) + call_for_each_auth(session_and_auth.at(session_id)); +} + namespace { @@ -927,7 +951,7 @@ struct KeeperStorageCreateRequestProcessor final : public KeeperStorageRequestPr return {KeeperStorage::Delta{zxid, Coordination::Error::ZBADARGUMENTS}}; Coordination::ACLs node_acls; - if (!fixupACL(request.acls, storage.session_and_auth[session_id], node_acls)) + if (!fixupACL(request.acls, session_id, storage.uncommitted_state, node_acls)) return {KeeperStorage::Delta{zxid, Coordination::Error::ZINVALIDACL}}; if (request.is_ephemeral) @@ -1533,10 +1557,8 @@ struct KeeperStorageSetACLRequestProcessor final : public KeeperStorageRequestPr return {KeeperStorage::Delta{zxid, Coordination::Error::ZBADVERSION}}; - auto & session_auth_ids = storage.session_and_auth[session_id]; Coordination::ACLs node_acls; - - if (!fixupACL(request.acls, session_auth_ids, node_acls)) + if (!fixupACL(request.acls, session_id, uncommitted_state, node_acls)) return {KeeperStorage::Delta{zxid, Coordination::Error::ZINVALIDACL}}; std::vector new_deltas @@ -1840,7 +1862,7 @@ struct KeeperStorageAuthRequestProcessor final : public KeeperStorageRequestProc return {KeeperStorage::Delta{zxid, Coordination::Error::ZAUTHFAILED}}; std::vector new_deltas; - auto auth_digest = generateDigest(auth_request.data); + auto auth_digest = KeeperStorage::generateDigest(auth_request.data); if (auth_digest == storage.superdigest) { KeeperStorage::AuthID auth{"super", ""}; @@ -2420,5 +2442,12 @@ void KeeperStorage::recalculateStats() container.recalculateDataSize(); } +String KeeperStorage::generateDigest(const String & userdata) +{ + std::vector user_password; + boost::split(user_password, userdata, [](char character) { return character == ':'; }); + return user_password[0] + ":" + base64Encode(getSHA1(userdata)); +} + } diff --git a/src/Coordination/KeeperStorage.h b/src/Coordination/KeeperStorage.h index be528072df4..cfacdfc84de 100644 --- a/src/Coordination/KeeperStorage.h +++ b/src/Coordination/KeeperStorage.h @@ -105,6 +105,8 @@ public: return first.value == second.value; } + static String generateDigest(const String & userdata); + struct RequestForSession { int64_t session_id; @@ -263,6 +265,8 @@ public: return check_auth(auth_it->second); } + void forEachAuthInSession(int64_t session_id, std::function func) const; + std::shared_ptr tryGetNodeFromStorage(StringRef path) const; std::unordered_map> session_and_auth; From e8bf96d126fe08c7264bcec7effc8410aa1284a8 Mon Sep 17 00:00:00 2001 From: Aleksei Filatov Date: Mon, 10 Apr 2023 13:02:43 +0300 Subject: [PATCH 064/103] Add unit tests --- src/Coordination/tests/gtest_coordination.cpp | 107 ++++++++++++++++++ 1 file changed, 107 insertions(+) diff --git a/src/Coordination/tests/gtest_coordination.cpp b/src/Coordination/tests/gtest_coordination.cpp index 895d563327e..68e68ca1fa7 100644 --- a/src/Coordination/tests/gtest_coordination.cpp +++ b/src/Coordination/tests/gtest_coordination.cpp @@ -1579,6 +1579,113 @@ TEST_P(CoordinationTest, TestEphemeralNodeRemove) } +TEST_P(CoordinationTest, TestCreateNodeWithAuthSchemeForAclWhenAuthIsPrecommitted) +{ + using namespace Coordination; + using namespace DB; + + ChangelogDirTest snapshots("./snapshots"); + CoordinationSettingsPtr settings = std::make_shared(); + ResponsesQueue queue(std::numeric_limits::max()); + SnapshotsQueue snapshots_queue{1}; + + auto state_machine = std::make_shared(queue, snapshots_queue, "./snapshots", settings, keeper_context, nullptr); + state_machine->init(); + + String user_auth_data = "test_user:test_password"; + String digest = KeeperStorage::generateDigest(user_auth_data); + + std::shared_ptr auth_req = std::make_shared(); + auth_req->scheme = "digest"; + auth_req->data = user_auth_data; + + // Add auth data to the session + auto auth_entry = getLogEntryFromZKRequest(0, 1, state_machine->getNextZxid(), auth_req); + state_machine->pre_commit(1, auth_entry->get_buf()); + + // Create a node with 'auth' scheme for ACL + String node_path = "/hello"; + std::shared_ptr create_req = std::make_shared(); + create_req->path = node_path; + // When 'auth' scheme is used the creator must have been authenticated by the server (for example, using 'digest' scheme) before it can + // create nodes with this ACL. + create_req->acls = {{.permissions = 31, .scheme = "auth", .id = ""}}; + auto create_entry = getLogEntryFromZKRequest(0, 1, state_machine->getNextZxid(), create_req); + state_machine->pre_commit(2, create_entry->get_buf()); + + const auto & uncommitted_state = state_machine->getStorage().uncommitted_state; + ASSERT_TRUE(uncommitted_state.nodes.contains(node_path)); + + // commit log entries + state_machine->commit(1, auth_entry->get_buf()); + state_machine->commit(2, create_entry->get_buf()); + + auto node = uncommitted_state.getNode(node_path); + ASSERT_NE(node, nullptr); + auto acls = uncommitted_state.getACLs(node_path); + ASSERT_EQ(acls.size(), 1); + EXPECT_EQ(acls[0].scheme, "digest"); + EXPECT_EQ(acls[0].id, digest); + EXPECT_EQ(acls[0].permissions, 31); +} + +TEST_P(CoordinationTest, TestSetACLWithAuthSchemeForAclWhenAuthIsPrecommitted) +{ + using namespace Coordination; + using namespace DB; + + ChangelogDirTest snapshots("./snapshots"); + CoordinationSettingsPtr settings = std::make_shared(); + ResponsesQueue queue(std::numeric_limits::max()); + SnapshotsQueue snapshots_queue{1}; + + auto state_machine = std::make_shared(queue, snapshots_queue, "./snapshots", settings, keeper_context, nullptr); + state_machine->init(); + + String user_auth_data = "test_user:test_password"; + String digest = KeeperStorage::generateDigest(user_auth_data); + + std::shared_ptr auth_req = std::make_shared(); + auth_req->scheme = "digest"; + auth_req->data = user_auth_data; + + // Add auth data to the session + auto auth_entry = getLogEntryFromZKRequest(0, 1, state_machine->getNextZxid(), auth_req); + state_machine->pre_commit(1, auth_entry->get_buf()); + + // Create a node + String node_path = "/hello"; + std::shared_ptr create_req = std::make_shared(); + create_req->path = node_path; + auto create_entry = getLogEntryFromZKRequest(0, 1, state_machine->getNextZxid(), create_req); + state_machine->pre_commit(2, create_entry->get_buf()); + + // Set ACL with 'auth' scheme for ACL + std::shared_ptr set_acl_req = std::make_shared(); + set_acl_req->path = node_path; + // When 'auth' scheme is used the creator must have been authenticated by the server (for example, using 'digest' scheme) before it can + // set this ACL. + set_acl_req->acls = {{.permissions = 31, .scheme = "auth", .id = ""}}; + auto set_acl_entry = getLogEntryFromZKRequest(0, 1, state_machine->getNextZxid(), set_acl_req); + state_machine->pre_commit(3, set_acl_entry->get_buf()); + + // commit all entries + state_machine->commit(1, auth_entry->get_buf()); + state_machine->commit(2, create_entry->get_buf()); + state_machine->commit(2, set_acl_entry->get_buf()); + + const auto & uncommitted_state = state_machine->getStorage().uncommitted_state; + auto node = uncommitted_state.getNode(node_path); + + ASSERT_NE(node, nullptr); + auto acls = uncommitted_state.getACLs(node_path); + ASSERT_EQ(acls.size(), 1); + EXPECT_EQ(acls[0].scheme, "digest"); + EXPECT_EQ(acls[0].id, digest); + EXPECT_EQ(acls[0].permissions, 31); +} + + TEST_P(CoordinationTest, TestRotateIntervalChanges) { using namespace Coordination; From 61bf70be129e8d2f99b3333e5a4fffb30285a360 Mon Sep 17 00:00:00 2001 From: Azat Khuzhin Date: Mon, 10 Apr 2023 13:47:56 +0200 Subject: [PATCH 065/103] Add second_deadlock_stack=1 for TSan on CI To improve TSan reports. Signed-off-by: Azat Khuzhin --- docker/test/base/Dockerfile | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/docker/test/base/Dockerfile b/docker/test/base/Dockerfile index c6c9fbca421..f6836804454 100644 --- a/docker/test/base/Dockerfile +++ b/docker/test/base/Dockerfile @@ -18,13 +18,13 @@ RUN apt-get update \ # and MEMORY_LIMIT_EXCEEDED exceptions in Functional tests (total memory limit in Functional tests is ~55.24 GiB). # TSAN will flush shadow memory when reaching this limit. # It may cause false-negatives, but it's better than OOM. -RUN echo "TSAN_OPTIONS='verbosity=1000 halt_on_error=1 history_size=7 memory_limit_mb=46080'" >> /etc/environment +RUN echo "TSAN_OPTIONS='verbosity=1000 halt_on_error=1 history_size=7 memory_limit_mb=46080 second_deadlock_stack=1'" >> /etc/environment RUN echo "UBSAN_OPTIONS='print_stacktrace=1'" >> /etc/environment RUN echo "MSAN_OPTIONS='abort_on_error=1 poison_in_dtor=1'" >> /etc/environment RUN echo "LSAN_OPTIONS='suppressions=/usr/share/clickhouse-test/config/lsan_suppressions.txt'" >> /etc/environment # Sanitizer options for current shell (not current, but the one that will be spawned on "docker run") # (but w/o verbosity for TSAN, otherwise test.reference will not match) -ENV TSAN_OPTIONS='halt_on_error=1 history_size=7 memory_limit_mb=46080' +ENV TSAN_OPTIONS='halt_on_error=1 history_size=7 memory_limit_mb=46080 second_deadlock_stack=1' ENV UBSAN_OPTIONS='print_stacktrace=1' ENV MSAN_OPTIONS='abort_on_error=1 poison_in_dtor=1' From bbee1a36be8262f4ee10542933bb8ce23585c22a Mon Sep 17 00:00:00 2001 From: Azat Khuzhin Date: Mon, 10 Apr 2023 15:06:40 +0200 Subject: [PATCH 066/103] Add a comment for use_sigaltstack=0 ASan option Signed-off-by: Azat Khuzhin --- tests/integration/helpers/cluster.py | 3 +++ 1 file changed, 3 insertions(+) diff --git a/tests/integration/helpers/cluster.py b/tests/integration/helpers/cluster.py index b2aedfce3ca..243d6c03093 100644 --- a/tests/integration/helpers/cluster.py +++ b/tests/integration/helpers/cluster.py @@ -371,6 +371,9 @@ class ClickHouseCluster: self.env_file = p.join(self.instances_dir, DEFAULT_ENV_NAME) self.env_variables = {} self.env_variables["TSAN_OPTIONS"] = "second_deadlock_stack=1" + # Problems with glibc 2.36+ [1] + # + # [1]: https://github.com/ClickHouse/ClickHouse/issues/43426#issuecomment-1368512678 self.env_variables["ASAN_OPTIONS"] = "use_sigaltstack=0" self.env_variables["CLICKHOUSE_WATCHDOG_ENABLE"] = "0" self.env_variables["CLICKHOUSE_NATS_TLS_SECURE"] = "0" From a1272e8536265929255fdf5020836f057859e425 Mon Sep 17 00:00:00 2001 From: Azat Khuzhin Date: Mon, 10 Apr 2023 15:07:20 +0200 Subject: [PATCH 067/103] Pass existing TSAN_OPTIONS to integration tests and append new options Signed-off-by: Azat Khuzhin --- docker/test/integration/runner/Dockerfile | 6 ++++++ tests/integration/helpers/cluster.py | 1 - tests/integration/test_grpc_protocol/test.py | 5 +++-- tests/integration/test_grpc_protocol_ssl/test.py | 5 +++-- tests/integration/test_server_reload/test.py | 6 ++++-- 5 files changed, 16 insertions(+), 7 deletions(-) diff --git a/docker/test/integration/runner/Dockerfile b/docker/test/integration/runner/Dockerfile index ce5bae2a031..1c5f635b134 100644 --- a/docker/test/integration/runner/Dockerfile +++ b/docker/test/integration/runner/Dockerfile @@ -108,6 +108,12 @@ RUN set -x \ && echo 'dockremap:165536:65536' >> /etc/subuid \ && echo 'dockremap:165536:65536' >> /etc/subgid +# Same options as in test/base/Dockerfile +# (in case you need to override them in tests) +ENV TSAN_OPTIONS='halt_on_error=1 history_size=7 memory_limit_mb=46080 second_deadlock_stack=1' +ENV UBSAN_OPTIONS='print_stacktrace=1' +ENV MSAN_OPTIONS='abort_on_error=1 poison_in_dtor=1' + EXPOSE 2375 ENTRYPOINT ["dockerd-entrypoint.sh"] CMD ["sh", "-c", "pytest $PYTEST_OPTS"] diff --git a/tests/integration/helpers/cluster.py b/tests/integration/helpers/cluster.py index 243d6c03093..3b6d0694d5b 100644 --- a/tests/integration/helpers/cluster.py +++ b/tests/integration/helpers/cluster.py @@ -370,7 +370,6 @@ class ClickHouseCluster: self.docker_logs_path = p.join(self.instances_dir, "docker.log") self.env_file = p.join(self.instances_dir, DEFAULT_ENV_NAME) self.env_variables = {} - self.env_variables["TSAN_OPTIONS"] = "second_deadlock_stack=1" # Problems with glibc 2.36+ [1] # # [1]: https://github.com/ClickHouse/ClickHouse/issues/43426#issuecomment-1368512678 diff --git a/tests/integration/test_grpc_protocol/test.py b/tests/integration/test_grpc_protocol/test.py index 137d585f7d1..254b78667d5 100644 --- a/tests/integration/test_grpc_protocol/test.py +++ b/tests/integration/test_grpc_protocol/test.py @@ -41,8 +41,9 @@ node = cluster.add_instance( "node", main_configs=["configs/grpc_config.xml"], # Bug in TSAN reproduces in this test https://github.com/grpc/grpc/issues/29550#issuecomment-1188085387 - # second_deadlock_stack -- just ordinary option we use everywhere, don't want to overwrite it - env_variables={"TSAN_OPTIONS": "report_atomic_races=0 second_deadlock_stack=1"}, + env_variables={ + "TSAN_OPTIONS": "report_atomic_races=0 " + os.getenv("TSAN_OPTIONS") + }, ) main_channel = None diff --git a/tests/integration/test_grpc_protocol_ssl/test.py b/tests/integration/test_grpc_protocol_ssl/test.py index 2c2a7f6c61e..f1a4475c1a5 100644 --- a/tests/integration/test_grpc_protocol_ssl/test.py +++ b/tests/integration/test_grpc_protocol_ssl/test.py @@ -43,8 +43,9 @@ node = cluster.add_instance( "configs/ca-cert.pem", ], # Bug in TSAN reproduces in this test https://github.com/grpc/grpc/issues/29550#issuecomment-1188085387 - # second_deadlock_stack -- just ordinary option we use everywhere, don't want to overwrite it - env_variables={"TSAN_OPTIONS": "report_atomic_races=0 second_deadlock_stack=1"}, + env_variables={ + "TSAN_OPTIONS": "report_atomic_races=0 " + os.getenv("TSAN_OPTIONS") + }, ) diff --git a/tests/integration/test_server_reload/test.py b/tests/integration/test_server_reload/test.py index b06d424ee1c..1429713cb84 100644 --- a/tests/integration/test_server_reload/test.py +++ b/tests/integration/test_server_reload/test.py @@ -12,6 +12,7 @@ import pymysql.connections import pymysql.err import pytest import sys +import os import time import logging from helpers.cluster import ClickHouseCluster, run_and_check @@ -34,8 +35,9 @@ instance = cluster.add_instance( user_configs=["configs/default_passwd.xml"], with_zookeeper=True, # Bug in TSAN reproduces in this test https://github.com/grpc/grpc/issues/29550#issuecomment-1188085387 - # second_deadlock_stack -- just ordinary option we use everywhere, don't want to overwrite it - env_variables={"TSAN_OPTIONS": "report_atomic_races=0 second_deadlock_stack=1"}, + env_variables={ + "TSAN_OPTIONS": "report_atomic_races=0 " + os.getenv("TSAN_OPTIONS") + }, ) From d50c6a9e1b546a109c044cc6a2af3d445f75f519 Mon Sep 17 00:00:00 2001 From: Dale Mcdiarmid Date: Mon, 10 Apr 2023 15:23:00 +0100 Subject: [PATCH 068/103] update blog references --- docs/en/engines/database-engines/postgresql.md | 1 + docs/en/engines/table-engines/integrations/postgresql.md | 2 ++ .../mergetree-family/aggregatingmergetree.md | 4 ++++ .../table-engines/mergetree-family/invertedindexes.md | 4 ++++ .../table-engines/mergetree-family/summingmergetree.md | 4 ++++ docs/en/engines/table-engines/special/keepermap.md | 4 ++++ docs/en/getting-started/example-datasets/github.md | 8 +++++--- docs/en/operations/opentelemetry.md | 4 ++++ docs/en/operations/query-cache.md | 4 ++++ docs/en/operations/utilities/clickhouse-local.md | 4 ++++ docs/en/sql-reference/aggregate-functions/combinators.md | 5 +++++ docs/en/sql-reference/data-types/aggregatefunction.md | 5 +++++ docs/en/sql-reference/data-types/map.md | 5 +++++ docs/en/sql-reference/statements/alter/delete.md | 4 ++++ docs/en/sql-reference/statements/alter/index.md | 4 ++++ docs/en/sql-reference/statements/alter/update.md | 5 +++++ docs/en/sql-reference/statements/create/view.md | 1 + docs/en/sql-reference/statements/delete.md | 4 ++++ docs/en/sql-reference/statements/select/join.md | 4 ++++ docs/en/sql-reference/table-functions/postgresql.md | 2 ++ 20 files changed, 75 insertions(+), 3 deletions(-) diff --git a/docs/en/engines/database-engines/postgresql.md b/docs/en/engines/database-engines/postgresql.md index 939995a61c5..95d6a6192e3 100644 --- a/docs/en/engines/database-engines/postgresql.md +++ b/docs/en/engines/database-engines/postgresql.md @@ -140,3 +140,4 @@ DESCRIBE TABLE test_database.test_table; ## Related content - Blog: [ClickHouse and PostgreSQL - a match made in data heaven - part 1](https://clickhouse.com/blog/migrating-data-between-clickhouse-postgres) +- Blog: [ClickHouse and PostgreSQL - a Match Made in Data Heaven - part 2](https://clickhouse.com/blog/migrating-data-between-clickhouse-postgres-part-2) diff --git a/docs/en/engines/table-engines/integrations/postgresql.md b/docs/en/engines/table-engines/integrations/postgresql.md index 2222d1fc016..2fcc0b10e78 100644 --- a/docs/en/engines/table-engines/integrations/postgresql.md +++ b/docs/en/engines/table-engines/integrations/postgresql.md @@ -177,4 +177,6 @@ CREATE TABLE pg_table_schema_with_dots (a UInt32) - [Using PostgreSQL as a dictionary source](../../../sql-reference/dictionaries/index.md#dictionary-sources#dicts-external_dicts_dict_sources-postgresql) ## Related content + - Blog: [ClickHouse and PostgreSQL - a match made in data heaven - part 1](https://clickhouse.com/blog/migrating-data-between-clickhouse-postgres) +- Blog: [ClickHouse and PostgreSQL - a Match Made in Data Heaven - part 2](https://clickhouse.com/blog/migrating-data-between-clickhouse-postgres-part-2) diff --git a/docs/en/engines/table-engines/mergetree-family/aggregatingmergetree.md b/docs/en/engines/table-engines/mergetree-family/aggregatingmergetree.md index 6591f666244..2b8b43802ea 100644 --- a/docs/en/engines/table-engines/mergetree-family/aggregatingmergetree.md +++ b/docs/en/engines/table-engines/mergetree-family/aggregatingmergetree.md @@ -122,3 +122,7 @@ FROM test.mv_visits GROUP BY StartDate ORDER BY StartDate; ``` + +## Related Content + +- Blog: [Using Aggregate Combinators in ClickHouse](https://clickhouse.com/blog/aggregate-functions-combinators-in-clickhouse-for-arrays-maps-and-states) diff --git a/docs/en/engines/table-engines/mergetree-family/invertedindexes.md b/docs/en/engines/table-engines/mergetree-family/invertedindexes.md index 701615495de..31f5a87a2b6 100644 --- a/docs/en/engines/table-engines/mergetree-family/invertedindexes.md +++ b/docs/en/engines/table-engines/mergetree-family/invertedindexes.md @@ -191,3 +191,7 @@ is performance. In practice, users often search for multiple terms at once. For '%big%'` can be evaluated directly using an inverted index by forming the union of the row id lists for terms "little" and "big". This also means that the parameter `GRANULARITY` supplied to index creation has no meaning (it may be removed from the syntax in the future). ::: + +## Related Content + +- Blog: [Introducing Inverted Indices in ClickHouse](https://clickhouse.com/blog/clickhouse-search-with-inverted-indices) diff --git a/docs/en/engines/table-engines/mergetree-family/summingmergetree.md b/docs/en/engines/table-engines/mergetree-family/summingmergetree.md index d0078656b5d..d15323bbc88 100644 --- a/docs/en/engines/table-engines/mergetree-family/summingmergetree.md +++ b/docs/en/engines/table-engines/mergetree-family/summingmergetree.md @@ -186,3 +186,7 @@ ARRAY JOIN When requesting data, use the [sumMap(key, value)](../../../sql-reference/aggregate-functions/reference/summap.md) function for aggregation of `Map`. For nested data structure, you do not need to specify its columns in the tuple of columns for summation. + +## Related Content + +- Blog: [Using Aggregate Combinators in ClickHouse](https://clickhouse.com/blog/aggregate-functions-combinators-in-clickhouse-for-arrays-maps-and-states) diff --git a/docs/en/engines/table-engines/special/keepermap.md b/docs/en/engines/table-engines/special/keepermap.md index e5c4dea2339..a1c7009b712 100644 --- a/docs/en/engines/table-engines/special/keepermap.md +++ b/docs/en/engines/table-engines/special/keepermap.md @@ -112,3 +112,7 @@ If setting `keeper_map_strict_mode` is set to `true`, fetching and updating data ```sql ALTER TABLE keeper_map_table UPDATE v1 = v1 * 10 + 2 WHERE key LIKE 'some%' AND v3 > 3.1; ``` + +## Related content + +- Blog: [Building a Real-time Analytics Apps with ClickHouse and Hex](https://clickhouse.com/blog/building-real-time-applications-with-clickhouse-and-hex-notebook-keeper-engine) diff --git a/docs/en/getting-started/example-datasets/github.md b/docs/en/getting-started/example-datasets/github.md index e18c7dec1a6..02965ed5e33 100644 --- a/docs/en/getting-started/example-datasets/github.md +++ b/docs/en/getting-started/example-datasets/github.md @@ -2499,7 +2499,9 @@ LIMIT 20 We welcome exact and improved solutions here. -# Related Content +## Related Content -- [Git commits and our community](https://clickhouse.com/blog/clickhouse-git-community-commits) -- [Window and array functions for Git commit sequences](https://clickhouse.com/blog/clickhouse-window-array-functions-git-commits) +- Blog: [Git commits and our community](https://clickhouse.com/blog/clickhouse-git-community-commits) +- Blog: [Window and array functions for Git commit sequences](https://clickhouse.com/blog/clickhouse-window-array-functions-git-commits) +- Blog: [Building a Real-time Analytics Apps with ClickHouse and Hex](https://clickhouse.com/blog/building-real-time-applications-with-clickhouse-and-hex-notebook-keeper-engine) +- Blog: [A Story of Open-source GitHub Activity using ClickHouse + Grafana](https://clickhouse.com/blog/introduction-to-clickhouse-and-grafana-webinar) diff --git a/docs/en/operations/opentelemetry.md b/docs/en/operations/opentelemetry.md index 15185f7ae6b..70f64d08ba3 100644 --- a/docs/en/operations/opentelemetry.md +++ b/docs/en/operations/opentelemetry.md @@ -61,3 +61,7 @@ FROM system.opentelemetry_span_log ``` In case of any errors, the part of the log data for which the error has occurred will be silently lost. Check the server log for error messages if the data does not arrive. + +## Related Content + +- Blog: [Building an Observability Solution with ClickHouse - Part 2 - Traces](https://clickhouse.com/blog/storing-traces-and-spans-open-telemetry-in-clickhouse) diff --git a/docs/en/operations/query-cache.md b/docs/en/operations/query-cache.md index 7b106909cf0..e9301ef3051 100644 --- a/docs/en/operations/query-cache.md +++ b/docs/en/operations/query-cache.md @@ -124,3 +124,7 @@ Finally, entries in the query cache are not shared between users due to security row policy on a table by running the same query as another user B for whom no such policy exists. However, if necessary, cache entries can be marked accessible by other users (i.e. shared) by supplying setting [query_cache_share_between_users](settings/settings.md#query-cache-share-between-users). + +## Related Content + +- Blog: [Introducing the ClickHouse Query Cache](https://clickhouse.com/blog/introduction-to-the-clickhouse-query-cache-and-design) diff --git a/docs/en/operations/utilities/clickhouse-local.md b/docs/en/operations/utilities/clickhouse-local.md index 6363d9cab27..4439f823240 100644 --- a/docs/en/operations/utilities/clickhouse-local.md +++ b/docs/en/operations/utilities/clickhouse-local.md @@ -6,6 +6,10 @@ sidebar_label: clickhouse-local # clickhouse-local +## Related Content + +- Blog: [Extracting, Converting, and Querying Data in Local Files using clickhouse-local](https://clickhouse.com/blog/extracting-converting-querying-local-files-with-sql-clickhouse-local) + ## When to use clickhouse-local vs. ClickHouse `clickhouse-local` is an easy-to-use version of ClickHouse that is ideal for developers who need to perform fast processing on local and remote files using SQL without having to install a full database server. With `clickhouse-local`, developers can use SQL commands (using the [ClickHouse SQL dialect](../../sql-reference/index.md)) directly from the command line, providing a simple and efficient way to access ClickHouse features without the need for a full ClickHouse installation. One of the main benefits of `clickhouse-local` is that it is already included when installing [clickhouse-client](https://clickhouse.com/docs/en/integrations/sql-clients/clickhouse-client-local). This means that developers can get started with `clickhouse-local` quickly, without the need for a complex installation process. diff --git a/docs/en/sql-reference/aggregate-functions/combinators.md b/docs/en/sql-reference/aggregate-functions/combinators.md index 704e88c6313..a28e58ca880 100644 --- a/docs/en/sql-reference/aggregate-functions/combinators.md +++ b/docs/en/sql-reference/aggregate-functions/combinators.md @@ -285,3 +285,8 @@ FROM people │ [3,2] │ [11.5,12.949999809265137] │ └────────┴───────────────────────────┘ ``` + + +## Related Content + +- Blog: [Using Aggregate Combinators in ClickHouse](https://clickhouse.com/blog/aggregate-functions-combinators-in-clickhouse-for-arrays-maps-and-states) diff --git a/docs/en/sql-reference/data-types/aggregatefunction.md b/docs/en/sql-reference/data-types/aggregatefunction.md index d8547f03714..90dd8bd2311 100644 --- a/docs/en/sql-reference/data-types/aggregatefunction.md +++ b/docs/en/sql-reference/data-types/aggregatefunction.md @@ -63,3 +63,8 @@ SELECT uniqMerge(state) FROM (SELECT uniqState(UserID) AS state FROM table GROUP ## Usage Example See [AggregatingMergeTree](../../engines/table-engines/mergetree-family/aggregatingmergetree.md) engine description. + + +## Related Content + +- Blog: [Using Aggregate Combinators in ClickHouse](https://clickhouse.com/blog/aggregate-functions-combinators-in-clickhouse-for-arrays-maps-and-states) diff --git a/docs/en/sql-reference/data-types/map.md b/docs/en/sql-reference/data-types/map.md index b0659746ba7..ad99bc75f24 100644 --- a/docs/en/sql-reference/data-types/map.md +++ b/docs/en/sql-reference/data-types/map.md @@ -108,3 +108,8 @@ Result: - [map()](../../sql-reference/functions/tuple-map-functions.md#function-map) function - [CAST()](../../sql-reference/functions/type-conversion-functions.md#type_conversion_function-cast) function + + +## Related content + +- Blog: [Building an Observability Solution with ClickHouse - Part 2 - Traces](https://clickhouse.com/blog/storing-traces-and-spans-open-telemetry-in-clickhouse) diff --git a/docs/en/sql-reference/statements/alter/delete.md b/docs/en/sql-reference/statements/alter/delete.md index 30ed96c0b9c..21ae6a1e5d1 100644 --- a/docs/en/sql-reference/statements/alter/delete.md +++ b/docs/en/sql-reference/statements/alter/delete.md @@ -28,3 +28,7 @@ The synchronicity of the query processing is defined by the [mutations_sync](/do - [Mutations](/docs/en/sql-reference/statements/alter/index.md#mutations) - [Synchronicity of ALTER Queries](/docs/en/sql-reference/statements/alter/index.md#synchronicity-of-alter-queries) - [mutations_sync](/docs/en/operations/settings/settings.md/#mutations_sync) setting + +## Related content + +- Blog: [Handling Updates and Deletes in ClickHouse](https://clickhouse.com/blog/handling-updates-and-deletes-in-clickhouse) diff --git a/docs/en/sql-reference/statements/alter/index.md b/docs/en/sql-reference/statements/alter/index.md index 5d7b92bd34d..cbec9572bb1 100644 --- a/docs/en/sql-reference/statements/alter/index.md +++ b/docs/en/sql-reference/statements/alter/index.md @@ -61,3 +61,7 @@ For all `ALTER` queries, if `alter_sync = 2` and some replicas are not active fo ::: For `ALTER TABLE ... UPDATE|DELETE` queries the synchronicity is defined by the [mutations_sync](/docs/en/operations/settings/settings.md/#mutations_sync) setting. + +## Related content + +- Blog: [Handling Updates and Deletes in ClickHouse](https://clickhouse.com/blog/handling-updates-and-deletes-in-clickhouse) diff --git a/docs/en/sql-reference/statements/alter/update.md b/docs/en/sql-reference/statements/alter/update.md index 92f0f111b92..c0036c060eb 100644 --- a/docs/en/sql-reference/statements/alter/update.md +++ b/docs/en/sql-reference/statements/alter/update.md @@ -27,3 +27,8 @@ The synchronicity of the query processing is defined by the [mutations_sync](/do - [Mutations](/docs/en/sql-reference/statements/alter/index.md#mutations) - [Synchronicity of ALTER Queries](/docs/en/sql-reference/statements/alter/index.md#synchronicity-of-alter-queries) - [mutations_sync](/docs/en/operations/settings/settings.md/#mutations_sync) setting + + +## Related content + +- Blog: [Handling Updates and Deletes in ClickHouse](https://clickhouse.com/blog/handling-updates-and-deletes-in-clickhouse) diff --git a/docs/en/sql-reference/statements/create/view.md b/docs/en/sql-reference/statements/create/view.md index 0def42259ab..d7b16a88f6b 100644 --- a/docs/en/sql-reference/statements/create/view.md +++ b/docs/en/sql-reference/statements/create/view.md @@ -364,3 +364,4 @@ The window view is useful in the following scenarios: ## Related Content - Blog: [Working with time series data in ClickHouse](https://clickhouse.com/blog/working-with-time-series-data-and-functions-ClickHouse) +- Blog: [Building an Observability Solution with ClickHouse - Part 2 - Traces](https://clickhouse.com/blog/storing-traces-and-spans-open-telemetry-in-clickhouse) diff --git a/docs/en/sql-reference/statements/delete.md b/docs/en/sql-reference/statements/delete.md index 149e7ab371f..d18f94031ae 100644 --- a/docs/en/sql-reference/statements/delete.md +++ b/docs/en/sql-reference/statements/delete.md @@ -55,3 +55,7 @@ With the described implementation now we can see what can negatively affect 'DEL - Table having a very large number of data parts - Having a lot of data in Compact parts—in a Compact part, all columns are stored in one file. + +## Related content + +- Blog: [Handling Updates and Deletes in ClickHouse](https://clickhouse.com/blog/handling-updates-and-deletes-in-clickhouse) diff --git a/docs/en/sql-reference/statements/select/join.md b/docs/en/sql-reference/statements/select/join.md index ece60961aaf..62947dcefdd 100644 --- a/docs/en/sql-reference/statements/select/join.md +++ b/docs/en/sql-reference/statements/select/join.md @@ -18,6 +18,10 @@ FROM Expressions from `ON` clause and columns from `USING` clause are called “join keys”. Unless otherwise stated, join produces a [Cartesian product](https://en.wikipedia.org/wiki/Cartesian_product) from rows with matching “join keys”, which might produce results with much more rows than the source tables. +## Related Content + +- Blog: [ClickHouse: A Blazingly Fast DBMS with Full SQL Join Support - Part 1](https://clickhouse.com/blog/clickhouse-fully-supports-joins) + ## Supported Types of JOIN All standard [SQL JOIN](https://en.wikipedia.org/wiki/Join_(SQL)) types are supported: diff --git a/docs/en/sql-reference/table-functions/postgresql.md b/docs/en/sql-reference/table-functions/postgresql.md index 6cd13acaa77..975a04fa0de 100644 --- a/docs/en/sql-reference/table-functions/postgresql.md +++ b/docs/en/sql-reference/table-functions/postgresql.md @@ -133,4 +133,6 @@ CREATE TABLE pg_table_schema_with_dots (a UInt32) - [Using PostgreSQL as a dictionary source](../../sql-reference/dictionaries/index.md#dictionary-sources#dicts-external_dicts_dict_sources-postgresql) ## Related content + - Blog: [ClickHouse and PostgreSQL - a match made in data heaven - part 1](https://clickhouse.com/blog/migrating-data-between-clickhouse-postgres) +- Blog: [ClickHouse and PostgreSQL - a Match Made in Data Heaven - part 2](https://clickhouse.com/blog/migrating-data-between-clickhouse-postgres-part-2) From 6015c3be5698538dff0a195ff9217525e514959a Mon Sep 17 00:00:00 2001 From: lzydmxy <13126752315@163.com> Date: Mon, 10 Apr 2023 23:53:32 +0800 Subject: [PATCH 069/103] update Failed test --- .../0_stateless/00962_live_view_periodic_refresh_dictionary.py | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/tests/queries/0_stateless/00962_live_view_periodic_refresh_dictionary.py b/tests/queries/0_stateless/00962_live_view_periodic_refresh_dictionary.py index 9d2a26c83c0..6b824f18dde 100755 --- a/tests/queries/0_stateless/00962_live_view_periodic_refresh_dictionary.py +++ b/tests/queries/0_stateless/00962_live_view_periodic_refresh_dictionary.py @@ -38,7 +38,7 @@ with client(name="client1>", log=log) as client1, client( client1.expect(prompt) client1.send( "CREATE DICTIONARY test.dict(a Int32, b Int32) PRIMARY KEY a LAYOUT(FLAT()) " - + "SOURCE(CLICKHOUSE(db 'test' table 'mt')) LIFETIME(1)" + + "SOURCE(CLICKHOUSE(db 'test' table 'mt')) LIFETIME(1) SETTINGS(check_dictionary_primary_key = 0)" ) client1.expect(prompt) client1.send("CREATE LIVE VIEW test.lv WITH REFRESH 1 AS SELECT * FROM test.dict") From 013a3ed269a75f8719dbc626d77a6e5cb3ebe3c0 Mon Sep 17 00:00:00 2001 From: Azat Khuzhin Date: Mon, 10 Apr 2023 19:56:52 +0200 Subject: [PATCH 070/103] Fix possible lock order inversion in Context::setTemporaryStorageInCache() TSan report [1]: WARNING: ThreadSanitizer: lock-order-inversion (potential deadlock) (pid=1) Cycle in lock order graph: M0 (0x7b9000000008) => M1 (0x7b90000000a8) => M0 Mutex M1 acquired here while holding mutex M0 in main thread: 0 pthread_mutex_lock (clickhouse+0xc66a8da) (BuildId: 174c9c4bf3606a5fc390211bdc2feff052246280) 3 std::__1::lock_guard::lock_guard[abi:v15000](std::__1::mutex&) build_docker/./contrib/llvm-project/libcxx/include/__mutex_base:94:27 (clickhouse+0x1d0ccb3a) (BuildId: 174c9c4bf3606a5fc390211bdc2feff052246280) 4 DB::Context::getDisk() const build_docker/./src/Interpreters/Context.cpp:3281:21 (clickhouse+0x1d0ccb3a) 5 DB::Context::setTemporaryStorageInCache() build_docker/./src/Interpreters/Context.cpp:927:21 (clickhouse+0x1d0cc188) (BuildId: 174c9c4bf3606a5fc390211bdc2feff052246280) 6 DB::Server::main() build_docker/./programs/server/Server.cpp:1055:25 (clickhouse+0x148504c2) (BuildId: 174c9c4bf3606a5fc390211bdc2feff052246280) Mutex M0 previously acquired by the same thread here: 0 pthread_mutex_lock (clickhouse+0xc66a8da) (BuildId: 174c9c4bf3606a5fc390211bdc2feff052246280) 3 std::__1::unique_lock::unique_lock[abi:v15000](std::__1::recursive_mutex&) build_docker/./contrib/llvm-project/libcxx/include/__mutex_base:122:61 (clickhouse+0x1d0cc148) (BuildId: 174c9c4bf3606a5fc390211bdc2feff052246280) 4 DB::Context::getLock() const build_docker/./src/Interpreters/Context.cpp:690:12 (clickhouse+0x1d0cc148) 5 DB::Context::setTemporaryStorageInCache() build_docker/./src/Interpreters/Context.cpp:922:17 (clickhouse+0x1d0cc148) 6 DB::Server::main() build_docker/./programs/server/Server.cpp:1055:25 (clickhouse+0x148504c2) (BuildId: 174c9c4bf3606a5fc390211bdc2feff052246280) Muex M0 acquired here while holding mutex M1 in main thread: 0 pthread_mutex_lock (clickhouse+0xc66a8da) (BuildId: 174c9c4bf3606a5fc390211bdc2feff052246280) 3 std::__1::unique_lock::unique_lock[abi:v15000](std::__1::recursive_mutex&) build_docker/./contrib/llvm-project/libcxx/include/__mutex_base:122:61 (clickhouse+0x1d0c7ce1) (BuildId: 174c9c4bf3606a5fc390211bdc2feff052246280) 4 DB::Context::getLock() const build_docker/./src/Interpreters/Context.cpp:690:12 (clickhouse+0x1d0c7ce1) 5 DB::Context::getPath() const build_docker/./src/Interpreters/Context.cpp:713:17 (clickhouse+0x1d0c7ce1) 6 DB::loadDiskLocalConfig() build_docker/./src/Disks/DiskLocal.cpp:75:30 (clickhouse+0x1ce7ed8a) (BuildId: 174c9c4bf3606a5fc390211bdc2feff052246280) 7 DB::DiskLocal::applyNewSettings() build_docker/./src/Disks/DiskLocal.cpp:491:5 (clickhouse+0x1ce7e7ef) (BuildId: 174c9c4bf3606a5fc390211bdc2feff052246280) 8 DB::DiskSelector::updateFromConfig() const build_docker/./src/Disks/DiskSelector.cpp:92:19 (clickhouse+0x1d14088e) (BuildId: 174c9c4bf3606a5fc390211bdc2feff052246280) 9 DB::Context::updateStorageConfiguration() build_docker/./src/Interpreters/Context.cpp:3388:49 (clickhouse+0x1d0ec002) (BuildId: 174c9c4bf3606a5fc390211bdc2feff052246280) .. 17 DB::ConfigReloader::reloadIfNewer() build_docker/./src/Common/Config/ConfigReloader.cpp:149:13 (clickhouse+0x1fff4250) Mutex M1 previously acquired by the same thread here: 0 pthread_mutex_lock (clickhouse+0xc66a8da) (BuildId: 174c9c4bf3606a5fc390211bdc2feff052246280) 3 std::__1::lock_guard::lock_guard[abi:v15000](std::__1::mutex&) build_docker/./contrib/llvm-project/libcxx/include/__mutex_base:94:27 (clickhouse+0x1d0ebf25) (BuildId: 174c9c4bf3606a5fc390211bdc2feff052246280) 4 DB::Context::updateStorageConfiguration() build_docker/./src/Interpreters/Context.cpp:3384:21 (clickhouse+0x1d0ebf25) .. 12 DB::ConfigReloader::reloadIfNewer() build_docker/./src/Common/Config/ConfigReloader.cpp:149:13 (clickhouse+0x1fff4250) SUMMARY: ThreadSanitizer: lock-order-inversion (potential deadlock) (/usr/bin/clickhouse+0xc66a8da) (BuildId: 174c9c4bf3606a5fc390211bdc2feff052246280) in pthread_mutex_lock ================== [1]: https://s3.amazonaws.com/clickhouse-test-reports/48596/a1272e8536265929255fdf5020836f057859e425/integration_tests__tsan__[6/6].html Signed-off-by: Azat Khuzhin --- src/Interpreters/Context.cpp | 9 ++++----- 1 file changed, 4 insertions(+), 5 deletions(-) diff --git a/src/Interpreters/Context.cpp b/src/Interpreters/Context.cpp index 8af09217b63..0ae533ba623 100644 --- a/src/Interpreters/Context.cpp +++ b/src/Interpreters/Context.cpp @@ -919,15 +919,14 @@ void Context::setTemporaryStoragePolicy(const String & policy_name, size_t max_s void Context::setTemporaryStorageInCache(const String & cache_disk_name, size_t max_size) { - auto lock = getLock(); - - if (shared->root_temp_data_on_disk) - throw Exception(ErrorCodes::LOGICAL_ERROR, "Temporary storage is already set"); - auto disk_ptr = getDisk(cache_disk_name); if (!disk_ptr) throw Exception(ErrorCodes::NO_ELEMENTS_IN_CONFIG, "Disk '{}' is not found", cache_disk_name); + auto lock = getLock(); + if (shared->root_temp_data_on_disk) + throw Exception(ErrorCodes::LOGICAL_ERROR, "Temporary storage is already set"); + const auto * disk_object_storage_ptr = dynamic_cast(disk_ptr.get()); if (!disk_object_storage_ptr) throw Exception(ErrorCodes::NO_ELEMENTS_IN_CONFIG, "Disk '{}' does not use cache", cache_disk_name); From 7cd4009c3d8811cc5f13244edbbf430da14da854 Mon Sep 17 00:00:00 2001 From: Azat Khuzhin Date: Mon, 10 Apr 2023 20:23:56 +0200 Subject: [PATCH 071/103] Fix flakiness of test_store_cleanup in case of image rebuild The log level will be substituted from "test" to "trace" in case of the tag is not "latest", the assumption behind this I guess is that it should not try to use "test" log level for older versions. But, it could have per-PR image in case of changes in the Dockerfile, so it is better to check for self.with_installed_binary, since actually any parameters except this will use new clickhouse binary anyway. CI: https://s3.amazonaws.com/clickhouse-test-reports/48596/a1272e8536265929255fdf5020836f057859e425/integration_tests__tsan__[1/6].html Signed-off-by: Azat Khuzhin --- tests/integration/helpers/cluster.py | 5 +++-- 1 file changed, 3 insertions(+), 2 deletions(-) diff --git a/tests/integration/helpers/cluster.py b/tests/integration/helpers/cluster.py index b2aedfce3ca..a5788ce36eb 100644 --- a/tests/integration/helpers/cluster.py +++ b/tests/integration/helpers/cluster.py @@ -4161,9 +4161,10 @@ class ClickHouseInstance: logging.debug("Copy common configuration from helpers") # The file is named with 0_ prefix to be processed before other configuration overloads. if self.copy_common_configs: - need_fix_log_level = self.tag != "latest" write_embedded_config( - "0_common_instance_config.xml", self.config_d_dir, need_fix_log_level + "0_common_instance_config.xml", + self.config_d_dir, + self.with_installed_binary, ) write_embedded_config("0_common_instance_users.xml", users_d_dir) From a6c99e8e843c324153d9ffeba8a67fb58a14ead9 Mon Sep 17 00:00:00 2001 From: Yakov Olkhovskiy Date: Mon, 10 Apr 2023 18:59:50 +0000 Subject: [PATCH 072/103] add IPv4 to supertype calculation --- src/DataTypes/getLeastSupertype.cpp | 3 ++- 1 file changed, 2 insertions(+), 1 deletion(-) diff --git a/src/DataTypes/getLeastSupertype.cpp b/src/DataTypes/getLeastSupertype.cpp index 3c33289c304..783326c25e9 100644 --- a/src/DataTypes/getLeastSupertype.cpp +++ b/src/DataTypes/getLeastSupertype.cpp @@ -3,6 +3,7 @@ #include #include #include +#include "Core/Types.h" #include @@ -88,7 +89,7 @@ DataTypePtr getNumericType(const TypeIndexSet & types) maximize(max_bits_of_unsigned_integer, 8); else if (type == TypeIndex::UInt16) maximize(max_bits_of_unsigned_integer, 16); - else if (type == TypeIndex::UInt32) + else if (type == TypeIndex::UInt32 || type == TypeIndex::IPv4) maximize(max_bits_of_unsigned_integer, 32); else if (type == TypeIndex::UInt64) maximize(max_bits_of_unsigned_integer, 64); From e6d96c27f249936ae4caa92650788f7ff0febab4 Mon Sep 17 00:00:00 2001 From: Yakov Olkhovskiy Date: Mon, 10 Apr 2023 19:16:02 +0000 Subject: [PATCH 073/103] unnecessary include --- src/DataTypes/getLeastSupertype.cpp | 1 - 1 file changed, 1 deletion(-) diff --git a/src/DataTypes/getLeastSupertype.cpp b/src/DataTypes/getLeastSupertype.cpp index 783326c25e9..9d42d82ce91 100644 --- a/src/DataTypes/getLeastSupertype.cpp +++ b/src/DataTypes/getLeastSupertype.cpp @@ -3,7 +3,6 @@ #include #include #include -#include "Core/Types.h" #include From 9f420ecad097da3e9f450e519ed7cc093111d000 Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Mon, 10 Apr 2023 22:00:03 +0200 Subject: [PATCH 074/103] Update HDFS --- contrib/libhdfs3 | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/contrib/libhdfs3 b/contrib/libhdfs3 index 9ee3ce77215..3c91d96ff29 160000 --- a/contrib/libhdfs3 +++ b/contrib/libhdfs3 @@ -1 +1 @@ -Subproject commit 9ee3ce77215fca83b7fdfcfe2186a3db0d0bdb74 +Subproject commit 3c91d96ff29fe5928f055519c6d979c4b104db9e From 32174e9a6143e404643273d63e959357f998bd57 Mon Sep 17 00:00:00 2001 From: Yakov Olkhovskiy Date: Mon, 10 Apr 2023 22:47:59 +0000 Subject: [PATCH 075/103] test is added --- .../0_stateless/02713_ip4_uint_compare.reference | 1 + tests/queries/0_stateless/02713_ip4_uint_compare.sql | 10 ++++++++++ 2 files changed, 11 insertions(+) create mode 100644 tests/queries/0_stateless/02713_ip4_uint_compare.reference create mode 100644 tests/queries/0_stateless/02713_ip4_uint_compare.sql diff --git a/tests/queries/0_stateless/02713_ip4_uint_compare.reference b/tests/queries/0_stateless/02713_ip4_uint_compare.reference new file mode 100644 index 00000000000..fdc2de3fbcb --- /dev/null +++ b/tests/queries/0_stateless/02713_ip4_uint_compare.reference @@ -0,0 +1 @@ +1 0 1 1 1 1 0 diff --git a/tests/queries/0_stateless/02713_ip4_uint_compare.sql b/tests/queries/0_stateless/02713_ip4_uint_compare.sql new file mode 100644 index 00000000000..141fa1371b6 --- /dev/null +++ b/tests/queries/0_stateless/02713_ip4_uint_compare.sql @@ -0,0 +1,10 @@ +WITH toIPv4('127.0.0.10') AS ip +SELECT + ip = 2130706442::UInt32, + ip = 0::UInt32, + ip < 2130706443::UInt32, + ip > 2130706441::UInt32, + ip <= 2130706442::UInt32, + ip >= 2130706442::UInt32, + ip != 2130706442::UInt32; + \ No newline at end of file From 7581982e0a6af04ad953bb68ae151fefdfc1593f Mon Sep 17 00:00:00 2001 From: Yakov Olkhovskiy Date: Mon, 10 Apr 2023 22:50:09 +0000 Subject: [PATCH 076/103] fix test --- tests/queries/0_stateless/02713_ip4_uint_compare.sql | 1 - 1 file changed, 1 deletion(-) diff --git a/tests/queries/0_stateless/02713_ip4_uint_compare.sql b/tests/queries/0_stateless/02713_ip4_uint_compare.sql index 141fa1371b6..ec8d6584329 100644 --- a/tests/queries/0_stateless/02713_ip4_uint_compare.sql +++ b/tests/queries/0_stateless/02713_ip4_uint_compare.sql @@ -7,4 +7,3 @@ SELECT ip <= 2130706442::UInt32, ip >= 2130706442::UInt32, ip != 2130706442::UInt32; - \ No newline at end of file From 9de55d9ccc6d95daa9df63dccd8c94650634e463 Mon Sep 17 00:00:00 2001 From: Robert Schulze Date: Tue, 11 Apr 2023 06:07:58 +0000 Subject: [PATCH 077/103] Rename tests to fix style check --- ...e_diff_aliases.reference => 02710_date_diff_aliases.reference} | 0 .../{25342_date_diff_aliases.sql => 02710_date_diff_aliases.sql} | 0 ...{25343_trim_aliases.reference => 02711_trim_aliases.reference} | 0 .../{25343_trim_aliases.sql => 02711_trim_aliases.sql} | 0 4 files changed, 0 insertions(+), 0 deletions(-) rename tests/queries/0_stateless/{25342_date_diff_aliases.reference => 02710_date_diff_aliases.reference} (100%) rename tests/queries/0_stateless/{25342_date_diff_aliases.sql => 02710_date_diff_aliases.sql} (100%) rename tests/queries/0_stateless/{25343_trim_aliases.reference => 02711_trim_aliases.reference} (100%) rename tests/queries/0_stateless/{25343_trim_aliases.sql => 02711_trim_aliases.sql} (100%) diff --git a/tests/queries/0_stateless/25342_date_diff_aliases.reference b/tests/queries/0_stateless/02710_date_diff_aliases.reference similarity index 100% rename from tests/queries/0_stateless/25342_date_diff_aliases.reference rename to tests/queries/0_stateless/02710_date_diff_aliases.reference diff --git a/tests/queries/0_stateless/25342_date_diff_aliases.sql b/tests/queries/0_stateless/02710_date_diff_aliases.sql similarity index 100% rename from tests/queries/0_stateless/25342_date_diff_aliases.sql rename to tests/queries/0_stateless/02710_date_diff_aliases.sql diff --git a/tests/queries/0_stateless/25343_trim_aliases.reference b/tests/queries/0_stateless/02711_trim_aliases.reference similarity index 100% rename from tests/queries/0_stateless/25343_trim_aliases.reference rename to tests/queries/0_stateless/02711_trim_aliases.reference diff --git a/tests/queries/0_stateless/25343_trim_aliases.sql b/tests/queries/0_stateless/02711_trim_aliases.sql similarity index 100% rename from tests/queries/0_stateless/25343_trim_aliases.sql rename to tests/queries/0_stateless/02711_trim_aliases.sql From c3f976c58a4eec4b2cc280fe6ad0f62f2b54db07 Mon Sep 17 00:00:00 2001 From: Robert Schulze Date: Tue, 11 Apr 2023 06:19:10 +0000 Subject: [PATCH 078/103] Move space check out of loop --- src/Functions/parseDateTime.cpp | 5 ++++- 1 file changed, 4 insertions(+), 1 deletion(-) diff --git a/src/Functions/parseDateTime.cpp b/src/Functions/parseDateTime.cpp index 12e2e113af4..05d18a736e3 100644 --- a/src/Functions/parseDateTime.cpp +++ b/src/Functions/parseDateTime.cpp @@ -1055,8 +1055,11 @@ namespace static Pos mysqlMicrosecond(Pos cur, Pos end, const String & fragment, DateTime & /*date*/) { + checkSpace(cur, end, 6, "mysqlMicrosecond requires size >= 6", fragment); + for (size_t i = 0; i < 6; ++i) - cur = assertNumber(cur, end, fragment); + cur = assertNumber(cur, end, fragment); + return cur; } From 4b61685c6469a674a8f7f492d9a94fa0b98a8b64 Mon Sep 17 00:00:00 2001 From: Robert Schulze Date: Tue, 11 Apr 2023 07:29:38 +0000 Subject: [PATCH 079/103] Minor refactoring of formatDateTime() Don't explicitly specify literal length when the called functions can compute it. --- src/Functions/formatDateTime.cpp | 44 ++++++++++++++++---------------- 1 file changed, 22 insertions(+), 22 deletions(-) diff --git a/src/Functions/formatDateTime.cpp b/src/Functions/formatDateTime.cpp index d6275a54c75..168404eaf01 100644 --- a/src/Functions/formatDateTime.cpp +++ b/src/Functions/formatDateTime.cpp @@ -978,15 +978,15 @@ public: instructions.push_back(std::move(instruction)); }; - auto add_extra_shift_or_literal_instruction = [&](size_t amount, std::string_view literal) + auto add_extra_shift_or_literal_instruction = [&](std::string_view literal) { if (mysql_with_only_fixed_length_formatters) - add_extra_shift(amount); + add_extra_shift(literal.size()); else add_literal_instruction(literal); }; - auto add_time_instruction = [&]([[maybe_unused]] typename Instruction::FuncMysql && func, [[maybe_unused]] size_t amount, [[maybe_unused]] std::string_view literal) + auto add_time_instruction = [&]([[maybe_unused]] typename Instruction::FuncMysql && func, [[maybe_unused]] std::string_view literal) { /// DateTime/DateTime64 --> insert instruction /// Other types cannot provide the requested data --> write out template @@ -997,7 +997,7 @@ public: instructions.push_back(std::move(instruction)); } else - add_extra_shift_or_literal_instruction(amount, literal); + add_extra_shift_or_literal_instruction(literal); }; Pos pos = format.data(); @@ -1012,7 +1012,7 @@ public: if (pos < percent_pos) { /// Handle characters before next % - add_extra_shift_or_literal_instruction(percent_pos - pos, std::string_view(pos, percent_pos - pos)); + add_extra_shift_or_literal_instruction(std::string_view(pos, percent_pos - pos)); out_template += String(pos, percent_pos - pos); } @@ -1107,7 +1107,7 @@ public: else { static constexpr std::string_view val = "00"; - add_time_instruction(&Instruction::mysqlMinute, 2, val); + add_time_instruction(&Instruction::mysqlMinute, val); out_template += val; } break; @@ -1260,7 +1260,7 @@ public: case 'p': { static constexpr std::string_view val = "AM"; - add_time_instruction(&Instruction::mysqlAMPM, 2, val); + add_time_instruction(&Instruction::mysqlAMPM, val); out_template += val; break; } @@ -1269,7 +1269,7 @@ public: case 'r': { static constexpr std::string_view val = "12:00 AM"; - add_time_instruction(&Instruction::mysqlHHMM12, 8, val); + add_time_instruction(&Instruction::mysqlHHMM12, val); out_template += val; break; } @@ -1278,7 +1278,7 @@ public: case 'R': { static constexpr std::string_view val = "00:00"; - add_time_instruction(&Instruction::mysqlHHMM24, 5, val); + add_time_instruction(&Instruction::mysqlHHMM24, val); out_template += val; break; } @@ -1287,7 +1287,7 @@ public: case 's': { static constexpr std::string_view val = "00"; - add_time_instruction(&Instruction::mysqlSecond, 2, val); + add_time_instruction(&Instruction::mysqlSecond, val); out_template += val; break; } @@ -1296,7 +1296,7 @@ public: case 'S': { static constexpr std::string_view val = "00"; - add_time_instruction(&Instruction::mysqlSecond, 2, val); + add_time_instruction(&Instruction::mysqlSecond, val); out_template += val; break; } @@ -1305,7 +1305,7 @@ public: case 'T': { static constexpr std::string_view val = "00:00:00"; - add_time_instruction(&Instruction::mysqlISO8601Time, 8, val); + add_time_instruction(&Instruction::mysqlISO8601Time, val); out_template += val; break; } @@ -1314,7 +1314,7 @@ public: case 'h': { static constexpr std::string_view val = "12"; - add_time_instruction(&Instruction::mysqlHour12, 2, val); + add_time_instruction(&Instruction::mysqlHour12, val); out_template += val; break; } @@ -1323,7 +1323,7 @@ public: case 'H': { static constexpr std::string_view val = "00"; - add_time_instruction(&Instruction::mysqlHour24, 2, val); + add_time_instruction(&Instruction::mysqlHour24, val); out_template += val; break; } @@ -1332,7 +1332,7 @@ public: case 'i': { static constexpr std::string_view val = "00"; - add_time_instruction(&Instruction::mysqlMinute, 2, val); + add_time_instruction(&Instruction::mysqlMinute, val); out_template += val; break; } @@ -1341,7 +1341,7 @@ public: case 'I': { static constexpr std::string_view val = "12"; - add_time_instruction(&Instruction::mysqlHour12, 2, val); + add_time_instruction(&Instruction::mysqlHour12, val); out_template += val; break; } @@ -1350,7 +1350,7 @@ public: case 'k': { static constexpr std::string_view val = "00"; - add_time_instruction(&Instruction::mysqlHour24, 2, val); + add_time_instruction(&Instruction::mysqlHour24, val); out_template += val; break; } @@ -1359,7 +1359,7 @@ public: case 'l': { static constexpr std::string_view val = "12"; - add_time_instruction(&Instruction::mysqlHour12, 2, val); + add_time_instruction(&Instruction::mysqlHour12, val); out_template += val; break; } @@ -1367,7 +1367,7 @@ public: case 't': { static constexpr std::string_view val = "\t"; - add_extra_shift_or_literal_instruction(1, val); + add_extra_shift_or_literal_instruction(val); out_template += val; break; } @@ -1375,7 +1375,7 @@ public: case 'n': { static constexpr std::string_view val = "\n"; - add_extra_shift_or_literal_instruction(1, val); + add_extra_shift_or_literal_instruction(val); out_template += val; break; } @@ -1384,7 +1384,7 @@ public: case '%': { static constexpr std::string_view val = "%"; - add_extra_shift_or_literal_instruction(1, val); + add_extra_shift_or_literal_instruction(val); out_template += val; break; } @@ -1411,7 +1411,7 @@ public: else { /// Handle characters after last % - add_extra_shift_or_literal_instruction(end - pos, std::string_view(pos, end - pos)); + add_extra_shift_or_literal_instruction(std::string_view(pos, end - pos)); out_template += String(pos, end - pos); break; } From dd2364361c08bccab5f872e63a1e267827a7fe44 Mon Sep 17 00:00:00 2001 From: Aleksei Filatov <68555560+aalexfvk@users.noreply.github.com> Date: Tue, 11 Apr 2023 10:45:30 +0300 Subject: [PATCH 080/103] Update src/Coordination/tests/gtest_coordination.cpp Co-authored-by: Antonio Andelic --- src/Coordination/tests/gtest_coordination.cpp | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/src/Coordination/tests/gtest_coordination.cpp b/src/Coordination/tests/gtest_coordination.cpp index 68e68ca1fa7..b1bea8ddf24 100644 --- a/src/Coordination/tests/gtest_coordination.cpp +++ b/src/Coordination/tests/gtest_coordination.cpp @@ -1672,7 +1672,7 @@ TEST_P(CoordinationTest, TestSetACLWithAuthSchemeForAclWhenAuthIsPrecommitted) // commit all entries state_machine->commit(1, auth_entry->get_buf()); state_machine->commit(2, create_entry->get_buf()); - state_machine->commit(2, set_acl_entry->get_buf()); + state_machine->commit(3, set_acl_entry->get_buf()); const auto & uncommitted_state = state_machine->getStorage().uncommitted_state; auto node = uncommitted_state.getNode(node_path); From 363b97fab8a789742153d63f04ec4738a0c42ec3 Mon Sep 17 00:00:00 2001 From: Han Fei Date: Tue, 11 Apr 2023 11:45:29 +0200 Subject: [PATCH 081/103] refine some messages of exception in regexp tree --- src/Dictionaries/RegExpTreeDictionary.cpp | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/src/Dictionaries/RegExpTreeDictionary.cpp b/src/Dictionaries/RegExpTreeDictionary.cpp index c072ba78d46..4db88631a2c 100644 --- a/src/Dictionaries/RegExpTreeDictionary.cpp +++ b/src/Dictionaries/RegExpTreeDictionary.cpp @@ -272,7 +272,7 @@ void RegExpTreeDictionary::initGraph() if (value->parent_id == 0) // this is root node. initTopologyOrder(id, visited, topology_id); if (topology_order.size() != regex_nodes.size()) - throw Exception(ErrorCodes::INCORRECT_DICTIONARY_DEFINITION, "Invalid Regex tree"); + throw Exception(ErrorCodes::LOGICAL_ERROR, "The topology order cannot match the number of regex nodes. This is likely a internal bug."); } void RegExpTreeDictionary::initTopologyOrder(UInt64 node_idx, std::set & visited, UInt64 & topology_id) @@ -280,7 +280,7 @@ void RegExpTreeDictionary::initTopologyOrder(UInt64 node_idx, std::set & visited.insert(node_idx); for (UInt64 child_idx : regex_nodes[node_idx]->children) if (visited.contains(child_idx)) - throw Exception(ErrorCodes::INCORRECT_DICTIONARY_DEFINITION, "Invalid Regex tree. The input tree is cyclical"); + throw Exception(ErrorCodes::INCORRECT_DICTIONARY_DEFINITION, "The regexp tree is cyclical. Please check your config."); else initTopologyOrder(child_idx, visited, topology_id); topology_order[node_idx] = topology_id++; From 57a1919594b5affdc27e0093fc93e7c44585438c Mon Sep 17 00:00:00 2001 From: "Mikhail f. Shiryaev" Date: Tue, 11 Apr 2023 12:40:04 +0200 Subject: [PATCH 082/103] Partially revert e0252db8d and fix pr-bugfix labeling --- tests/ci/run_check.py | 5 +++-- 1 file changed, 3 insertions(+), 2 deletions(-) diff --git a/tests/ci/run_check.py b/tests/ci/run_check.py index 8dc136e01f4..de56137b7dd 100644 --- a/tests/ci/run_check.py +++ b/tests/ci/run_check.py @@ -35,8 +35,9 @@ LABELS = { "pr-backward-incompatible": ["Backward Incompatible Change"], "pr-bugfix": [ "Bug Fix", - "Bug Fix (user-visible misbehaviour in official stable release)", - "Bug Fix (user-visible misbehavior in official stable release)", + "Bug Fix (user-visible misbehavior in an official stable release)", + "Bug Fix (user-visible misbehaviour in official stable or prestable release)", + "Bug Fix (user-visible misbehavior in official stable or prestable release)", ], "pr-build": [ "Build/Testing/Packaging Improvement", From 4605d71947077067c0fed62ae3e0cb2179c1ef52 Mon Sep 17 00:00:00 2001 From: vdimir Date: Tue, 11 Apr 2023 11:17:02 +0000 Subject: [PATCH 083/103] Fix build src/Interpreters/InterpreterInsertQuery.h --- src/Interpreters/InterpreterInsertQuery.h | 1 + 1 file changed, 1 insertion(+) diff --git a/src/Interpreters/InterpreterInsertQuery.h b/src/Interpreters/InterpreterInsertQuery.h index f60d6567d74..5bff472270d 100644 --- a/src/Interpreters/InterpreterInsertQuery.h +++ b/src/Interpreters/InterpreterInsertQuery.h @@ -4,6 +4,7 @@ #include #include #include +#include namespace DB { From 30659c5b48cf10adf936ade1f1a23c9187febacf Mon Sep 17 00:00:00 2001 From: vdimir Date: Tue, 11 Apr 2023 11:46:53 +0000 Subject: [PATCH 084/103] Fix build ThreadGroupPtr --- src/Processors/Transforms/buildPushingToViewsChain.h | 1 + 1 file changed, 1 insertion(+) diff --git a/src/Processors/Transforms/buildPushingToViewsChain.h b/src/Processors/Transforms/buildPushingToViewsChain.h index a2e7e39ff23..c7effa77d5b 100644 --- a/src/Processors/Transforms/buildPushingToViewsChain.h +++ b/src/Processors/Transforms/buildPushingToViewsChain.h @@ -7,6 +7,7 @@ #include #include #include +#include namespace Poco { From 24453759719b15c8681e7f838a44e7e81b9d58ee Mon Sep 17 00:00:00 2001 From: Kruglov Pavel <48961922+Avogar@users.noreply.github.com> Date: Tue, 11 Apr 2023 14:11:24 +0200 Subject: [PATCH 085/103] Fix flaky test test_drop_replica_and_achieve_quorum --- tests/integration/test_quorum_inserts/test.py | 19 ------------------- 1 file changed, 19 deletions(-) diff --git a/tests/integration/test_quorum_inserts/test.py b/tests/integration/test_quorum_inserts/test.py index 779d1a69dcc..4dbd530dd17 100644 --- a/tests/integration/test_quorum_inserts/test.py +++ b/tests/integration/test_quorum_inserts/test.py @@ -144,25 +144,6 @@ def test_drop_replica_and_achieve_quorum(started_cluster): ) ) - print("Now we can insert some other data.") - zero.query( - "INSERT INTO test_drop_replica_and_achieve_quorum(a,d) VALUES (2, '2012-02-02')" - ) - - assert TSV("1\t2011-01-01\n2\t2012-02-02\n") == TSV( - zero.query("SELECT * FROM test_drop_replica_and_achieve_quorum ORDER BY a") - ) - assert TSV("1\t2011-01-01\n2\t2012-02-02\n") == TSV( - first.query("SELECT * FROM test_drop_replica_and_achieve_quorum ORDER BY a") - ) - assert TSV("1\t2011-01-01\n2\t2012-02-02\n") == TSV( - second.query("SELECT * FROM test_drop_replica_and_achieve_quorum ORDER BY a") - ) - - zero.query( - "DROP TABLE IF EXISTS test_drop_replica_and_achieve_quorum ON CLUSTER cluster" - ) - @pytest.mark.parametrize(("add_new_data"), [False, True]) def test_insert_quorum_with_drop_partition(started_cluster, add_new_data): From c29aa0cd49e635b4b2f929c9519a9f4d055af57f Mon Sep 17 00:00:00 2001 From: "Mikhail f. Shiryaev" Date: Tue, 11 Apr 2023 15:34:33 +0200 Subject: [PATCH 086/103] Fail the build on a wrong changelog category --- tests/ci/run_check.py | 5 +++++ 1 file changed, 5 insertions(+) diff --git a/tests/ci/run_check.py b/tests/ci/run_check.py index de56137b7dd..b7dc78af4d7 100644 --- a/tests/ci/run_check.py +++ b/tests/ci/run_check.py @@ -31,6 +31,8 @@ SUBMODULE_CHANGED_LABEL = "submodule changed" # They are used in .github/PULL_REQUEST_TEMPLATE.md, keep comments there # updated accordingly +# The following lists are append only, try to avoid editing them +# They atill could be cleaned out after the decent time though. LABELS = { "pr-backward-incompatible": ["Backward Incompatible Change"], "pr-bugfix": [ @@ -190,6 +192,9 @@ def check_pr_description(pr_info: PRInfo) -> Tuple[str, str]: ): return "", category + if category not in CATEGORY_TO_LABEL: + return f"Category '{category}' is not valid", "" + if not entry: return f"Changelog entry required for category '{category}'", category From c6907f2a327894480f93487f3a5e13c3d0613f71 Mon Sep 17 00:00:00 2001 From: DanRoscigno Date: Tue, 11 Apr 2023 09:53:40 -0400 Subject: [PATCH 087/103] add Observability to spell list --- utils/check-style/aspell-ignore/en/aspell-dict.txt | 1 + 1 file changed, 1 insertion(+) diff --git a/utils/check-style/aspell-ignore/en/aspell-dict.txt b/utils/check-style/aspell-ignore/en/aspell-dict.txt index 8f72f07d7ec..4eb0e9dc42d 100644 --- a/utils/check-style/aspell-ignore/en/aspell-dict.txt +++ b/utils/check-style/aspell-ignore/en/aspell-dict.txt @@ -108,6 +108,7 @@ NULLIF NVME NYPD NuRaft +Observability OLAP OLTP ObjectId From 5488fb0fec80ded48205afc048dd5d1a1a488c10 Mon Sep 17 00:00:00 2001 From: Dmitry Novik Date: Tue, 11 Apr 2023 14:07:17 +0000 Subject: [PATCH 088/103] Add documentation for additional_result_filter setting --- docs/en/operations/settings/settings.md | 33 +++++++++++++++++++++++++ 1 file changed, 33 insertions(+) diff --git a/docs/en/operations/settings/settings.md b/docs/en/operations/settings/settings.md index da503bc02aa..000349d726e 100644 --- a/docs/en/operations/settings/settings.md +++ b/docs/en/operations/settings/settings.md @@ -40,6 +40,39 @@ SETTINGS additional_table_filters = (('table_1', 'x != 2')) └───┴──────┘ ``` +## additional_result_filter + +An additional filter expression is applied after to the result of `SELECT` query. +This setting is not applied to any subquery. + +Default value: `''`. + +**Example** + +``` sql +insert into table_1 values (1, 'a'), (2, 'bb'), (3, 'ccc'), (4, 'dddd'); +``` +```response +┌─x─┬─y────┐ +│ 1 │ a │ +│ 2 │ bb │ +│ 3 │ ccc │ +│ 4 │ dddd │ +└───┴──────┘ +``` +```sql +SELECT * +FROM table_1 +SETTINGS additional_result_filter = 'x != 2' +``` +```response +┌─x─┬─y────┐ +│ 1 │ a │ +│ 3 │ ccc │ +│ 4 │ dddd │ +└───┴──────┘ +``` + ## allow_nondeterministic_mutations {#allow_nondeterministic_mutations} User-level setting that allows mutations on replicated tables to make use of non-deterministic functions such as `dictGet`. From 0e417b1a6fa0408be9b30f680b08c2d067c326ef Mon Sep 17 00:00:00 2001 From: Dmitry Novik Date: Tue, 11 Apr 2023 14:08:28 +0000 Subject: [PATCH 089/103] Cleanup the code --- src/Planner/PlannerJoinTree.cpp | 17 +++-------------- 1 file changed, 3 insertions(+), 14 deletions(-) diff --git a/src/Planner/PlannerJoinTree.cpp b/src/Planner/PlannerJoinTree.cpp index 08d10c6adba..f0572073e38 100644 --- a/src/Planner/PlannerJoinTree.cpp +++ b/src/Planner/PlannerJoinTree.cpp @@ -1,10 +1,6 @@ #include -#include "Common/logger_useful.h" #include -#include "Parsers/ExpressionListParsers.h" -#include "Parsers/parseQuery.h" -#include "Storages/SelectQueryInfo.h" #include @@ -37,6 +33,9 @@ #include #include +#include +#include + #include #include #include @@ -447,10 +446,7 @@ FilterDAGInfo buildAdditionalFiltersIfNeeded(const StoragePtr & storage, auto const & storage_id = storage->getStorageID(); - LOG_DEBUG(&Poco::Logger::get("buildAdditionalFiltersIfNeeded"), "Trying to find additional filters for table: {}", storage_id.getFullTableName()); - ASTPtr additional_filter_ast; - for (size_t i = 0; i < additional_filters.size(); ++i) { const auto & tuple = additional_filters[i].safeGet(); @@ -473,9 +469,6 @@ FilterDAGInfo buildAdditionalFiltersIfNeeded(const StoragePtr & storage, return {}; table_expression_query_info.additional_filter_ast = additional_filter_ast; - - LOG_DEBUG(&Poco::Logger::get("buildAdditionalFiltersIfNeeded"), "Found additional filter: {}", additional_filter_ast->formatForErrorMessage()); - return buildFilterInfo(additional_filter_ast, table_expression_query_info.table_expression, planner_context); } @@ -728,10 +721,6 @@ JoinTreeQueryPlan buildQueryPlanForTableExpression(QueryTreeNodePtr table_expres filter_step->setStepDescription(description); query_plan.addStep(std::move(filter_step)); } - else - { - LOG_DEBUG(&Poco::Logger::get("PlannerJoinTree"), "Can not add filter: {}", description); - } } if (query_context->hasQueryContext() && !select_query_options.is_internal) From cdceac662412e968096de800c35c47f650d2996a Mon Sep 17 00:00:00 2001 From: "Mikhail f. Shiryaev" Date: Tue, 11 Apr 2023 16:10:52 +0200 Subject: [PATCH 090/103] Reduce number of return statements --- tests/ci/run_check.py | 20 +++++++++----------- 1 file changed, 9 insertions(+), 11 deletions(-) diff --git a/tests/ci/run_check.py b/tests/ci/run_check.py index b7dc78af4d7..4f38007cb03 100644 --- a/tests/ci/run_check.py +++ b/tests/ci/run_check.py @@ -132,6 +132,7 @@ def check_pr_description(pr_info: PRInfo) -> Tuple[str, str]: category = "" entry = "" + description_error = "" i = 0 while i < len(lines): @@ -183,22 +184,19 @@ def check_pr_description(pr_info: PRInfo) -> Tuple[str, str]: i += 1 if not category: - return "Changelog category is empty", category - + description_error = "Changelog category is empty" # Filter out the PR categories that are not for changelog. - if re.match( + elif re.match( r"(?i)doc|((non|in|not|un)[-\s]*significant)|(not[ ]*for[ ]*changelog)", category, ): - return "", category + pass # to not check the rest of the conditions + elif category not in CATEGORY_TO_LABEL: + description_error, category = f"Category '{category}' is not valid", "" + elif not entry: + description_error = f"Changelog entry required for category '{category}'" - if category not in CATEGORY_TO_LABEL: - return f"Category '{category}' is not valid", "" - - if not entry: - return f"Changelog entry required for category '{category}'", category - - return "", category + return description_error, category if __name__ == "__main__": From 435a0ab9bbead50cf2ee29dca1460c71df723184 Mon Sep 17 00:00:00 2001 From: Dmitry Novik Date: Tue, 11 Apr 2023 14:13:37 +0000 Subject: [PATCH 091/103] Fix a typo --- docs/en/operations/settings/settings.md | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/docs/en/operations/settings/settings.md b/docs/en/operations/settings/settings.md index 000349d726e..f690a6d506c 100644 --- a/docs/en/operations/settings/settings.md +++ b/docs/en/operations/settings/settings.md @@ -42,7 +42,7 @@ SETTINGS additional_table_filters = (('table_1', 'x != 2')) ## additional_result_filter -An additional filter expression is applied after to the result of `SELECT` query. +An additional filter expression to apply to the result of `SELECT` query. This setting is not applied to any subquery. Default value: `''`. From bf28be8837f7cbeb9883b943cd04aa74678620c1 Mon Sep 17 00:00:00 2001 From: Han Fei Date: Tue, 11 Apr 2023 17:07:44 +0200 Subject: [PATCH 092/103] fix 02504_regexp_dictionary_table_source --- src/Dictionaries/RegExpTreeDictionary.cpp | 4 +++- 1 file changed, 3 insertions(+), 1 deletion(-) diff --git a/src/Dictionaries/RegExpTreeDictionary.cpp b/src/Dictionaries/RegExpTreeDictionary.cpp index 4db88631a2c..c56c611d02f 100644 --- a/src/Dictionaries/RegExpTreeDictionary.cpp +++ b/src/Dictionaries/RegExpTreeDictionary.cpp @@ -271,14 +271,16 @@ void RegExpTreeDictionary::initGraph() for (const auto & [id, value]: regex_nodes) if (value->parent_id == 0) // this is root node. initTopologyOrder(id, visited, topology_id); + /// If there is a cycle and all nodes have a parent, this condition will be met. if (topology_order.size() != regex_nodes.size()) - throw Exception(ErrorCodes::LOGICAL_ERROR, "The topology order cannot match the number of regex nodes. This is likely a internal bug."); + throw Exception(ErrorCodes::INCORRECT_DICTIONARY_DEFINITION, "The regexp tree is cyclical. Please check your config."); } void RegExpTreeDictionary::initTopologyOrder(UInt64 node_idx, std::set & visited, UInt64 & topology_id) { visited.insert(node_idx); for (UInt64 child_idx : regex_nodes[node_idx]->children) + /// there is a cycle when dfs the graph. if (visited.contains(child_idx)) throw Exception(ErrorCodes::INCORRECT_DICTIONARY_DEFINITION, "The regexp tree is cyclical. Please check your config."); else From 165edd332e58c280808a842331d9f6a9934dfa59 Mon Sep 17 00:00:00 2001 From: alesapin Date: Tue, 11 Apr 2023 18:30:53 +0200 Subject: [PATCH 093/103] Remove strange code from MutateTask --- src/Storages/MergeTree/MutateTask.cpp | 40 +-------------------------- 1 file changed, 1 insertion(+), 39 deletions(-) diff --git a/src/Storages/MergeTree/MutateTask.cpp b/src/Storages/MergeTree/MutateTask.cpp index b4d0a310785..9d1916ea15e 100644 --- a/src/Storages/MergeTree/MutateTask.cpp +++ b/src/Storages/MergeTree/MutateTask.cpp @@ -795,8 +795,6 @@ struct MutationContext NamesAndTypesList storage_columns; NameSet materialized_indices; NameSet materialized_projections; - MutationsInterpreter::MutationKind::MutationKindEnum mutation_kind - = MutationsInterpreter::MutationKind::MutationKindEnum::MUTATE_UNKNOWN; MergeTreeData::MutableDataPartPtr new_data_part; IMergedBlockOutputStreamPtr out{nullptr}; @@ -1686,7 +1684,6 @@ bool MutateTask::prepare() *ctx->data, ctx->source_part, ctx->metadata_snapshot, ctx->for_interpreter, context_for_reading, true); ctx->materialized_indices = ctx->interpreter->grabMaterializedIndices(); ctx->materialized_projections = ctx->interpreter->grabMaterializedProjections(); - ctx->mutation_kind = ctx->interpreter->getMutationKind(); /// Always disable filtering in mutations: we want to read and write all rows because for updates we rewrite only some of the /// columns and preserve the columns that are not affected, but after the update all columns must have the same number of rows. ctx->interpreter->setApplyDeletedMask(false); @@ -1696,8 +1693,6 @@ bool MutateTask::prepare() } auto single_disk_volume = std::make_shared("volume_" + ctx->future_part->name, ctx->space_reservation->getDisk(), 0); - /// FIXME new_data_part is not used in the case when we clone part with cloneAndLoadDataPartOnSameDisk and return false - /// Is it possible to handle this case earlier? std::string prefix; if (ctx->need_prefix) @@ -1739,7 +1734,7 @@ bool MutateTask::prepare() /// All columns from part are changed and may be some more that were missing before in part /// TODO We can materialize compact part without copying data if (!isWidePart(ctx->source_part) || !isFullPartStorage(ctx->source_part->getDataPartStorage()) - || (ctx->mutation_kind == MutationsInterpreter::MutationKind::MUTATE_OTHER && ctx->interpreter && ctx->interpreter->isAffectingAllColumns())) + || (ctx->interpreter && ctx->interpreter->isAffectingAllColumns())) { task = std::make_unique(ctx); } @@ -1768,39 +1763,6 @@ bool MutateTask::prepare() ctx->for_file_renames, ctx->mrk_extension); - if (ctx->indices_to_recalc.empty() && - ctx->projections_to_recalc.empty() && - ctx->mutation_kind != MutationsInterpreter::MutationKind::MUTATE_OTHER - && ctx->files_to_rename.empty()) - { - LOG_TRACE(ctx->log, "Part {} doesn't change up to mutation version {} (optimized)", ctx->source_part->name, ctx->future_part->part_info.mutation); - /// new_data_part is not used here, another part is created instead (see the comment above) - ctx->temporary_directory_lock = {}; - - /// In zero-copy replication checksums file path in s3 (blob path) is used for zero copy locks in ZooKeeper. If we will hardlink checksums file, we will have the same blob path - /// and two different parts (source and new mutated part) will use the same locks in ZooKeeper. To avoid this we copy checksums.txt to generate new blob path. - /// Example: - /// part: all_0_0_0/checksums.txt -> /s3/blobs/shjfgsaasdasdasdasdasdas - /// locks path in zk: /zero_copy/tbl_id/s3_blobs_shjfgsaasdasdasdasdasdas/replica_name - /// ^ part name don't participate in lock path - /// In case of full hardlink we will have: - /// part: all_0_0_0_1/checksums.txt -> /s3/blobs/shjfgsaasdasdasdasdasdas - /// locks path in zk: /zero_copy/tbl_id/s3_blobs_shjfgsaasdasdasdasdasdas/replica_name - /// So we need to copy to have a new name - NameSet files_to_copy_instead_of_hardlinks; - auto settings_ptr = ctx->data->getSettings(); - bool copy_checksumns = ctx->data->supportsReplication() && settings_ptr->allow_remote_fs_zero_copy_replication && ctx->source_part->isStoredOnRemoteDiskWithZeroCopySupport(); - if (copy_checksumns) - files_to_copy_instead_of_hardlinks.insert(IMergeTreeDataPart::FILE_FOR_REFERENCES_CHECK); - - auto [part, lock] = ctx->data->cloneAndLoadDataPartOnSameDisk(ctx->source_part, prefix, ctx->future_part->part_info, ctx->metadata_snapshot, ctx->txn, &ctx->hardlinked_files, false, files_to_copy_instead_of_hardlinks); - part->getDataPartStorage().beginTransaction(); - - ctx->temporary_directory_lock = std::move(lock); - promise.set_value(std::move(part)); - return false; - } - task = std::make_unique(ctx); } From d7f865037266ed87538cf4df7ec7e8165681871b Mon Sep 17 00:00:00 2001 From: Azat Khuzhin Date: Tue, 11 Apr 2023 20:18:11 +0200 Subject: [PATCH 094/103] Remove lock for duplicated parts UUIDs (allow_experimental_query_deduplication=1) It looks redundant, since sendQuery() cannot be executed in parallel with processPacket() (hence RemoteQueryExecutor::setPartUUIDs()) This likely will fix the lock-order-inversion in RemoteQueryExecutor, since I think it is false-positive. Fixes: #48534 Signed-off-by: Azat Khuzhin --- src/QueryPipeline/RemoteQueryExecutor.cpp | 8 ++------ src/QueryPipeline/RemoteQueryExecutor.h | 1 - 2 files changed, 2 insertions(+), 7 deletions(-) diff --git a/src/QueryPipeline/RemoteQueryExecutor.cpp b/src/QueryPipeline/RemoteQueryExecutor.cpp index e3a69958213..b7490a2ad9c 100644 --- a/src/QueryPipeline/RemoteQueryExecutor.cpp +++ b/src/QueryPipeline/RemoteQueryExecutor.cpp @@ -236,11 +236,8 @@ void RemoteQueryExecutor::sendQuery(ClientInfo::QueryKind query_kind) ClientInfo modified_client_info = context->getClientInfo(); modified_client_info.query_kind = query_kind; - { - std::lock_guard lock(duplicated_part_uuids_mutex); - if (!duplicated_part_uuids.empty()) - connections->sendIgnoredPartUUIDs(duplicated_part_uuids); - } + if (!duplicated_part_uuids.empty()) + connections->sendIgnoredPartUUIDs(duplicated_part_uuids); connections->sendQuery(timeouts, query, query_id, stage, modified_client_info, true); @@ -471,7 +468,6 @@ bool RemoteQueryExecutor::setPartUUIDs(const std::vector & uuids) if (!duplicates.empty()) { - std::lock_guard lock(duplicated_part_uuids_mutex); duplicated_part_uuids.insert(duplicated_part_uuids.begin(), duplicates.begin(), duplicates.end()); return false; } diff --git a/src/QueryPipeline/RemoteQueryExecutor.h b/src/QueryPipeline/RemoteQueryExecutor.h index 576efb97bb1..b114d7eecdd 100644 --- a/src/QueryPipeline/RemoteQueryExecutor.h +++ b/src/QueryPipeline/RemoteQueryExecutor.h @@ -255,7 +255,6 @@ private: std::atomic got_duplicated_part_uuids{ false }; /// Parts uuids, collected from remote replicas - std::mutex duplicated_part_uuids_mutex; std::vector duplicated_part_uuids; PoolMode pool_mode = PoolMode::GET_MANY; From f91309d83d4461ebaca0e6331d89127974bbde67 Mon Sep 17 00:00:00 2001 From: alesapin Date: Tue, 11 Apr 2023 21:21:42 +0200 Subject: [PATCH 095/103] Fix drop in compact parts --- src/Storages/MergeTree/MutateTask.cpp | 20 +++++++++++++++++++- 1 file changed, 19 insertions(+), 1 deletion(-) diff --git a/src/Storages/MergeTree/MutateTask.cpp b/src/Storages/MergeTree/MutateTask.cpp index 50c180e44fb..6f96683971b 100644 --- a/src/Storages/MergeTree/MutateTask.cpp +++ b/src/Storages/MergeTree/MutateTask.cpp @@ -64,7 +64,7 @@ static void splitAndModifyMutationCommands( if (!isWidePart(part) || !isFullPartStorage(part->getDataPartStorage())) { - NameSet mutated_columns; + NameSet mutated_columns, dropped_columns; for (const auto & command : commands) { if (command.type == MutationCommand::Type::MATERIALIZE_INDEX @@ -98,8 +98,12 @@ static void splitAndModifyMutationCommands( } else mutated_columns.emplace(command.column_name); + + if (command.type == MutationCommand::Type::DROP_COLUMN) + dropped_columns.emplace(command.column_name); } } + } auto alter_conversions = part->storage.getAlterConversionsForPart(part); @@ -141,6 +145,16 @@ static void splitAndModifyMutationCommands( for_interpreter.emplace_back( MutationCommand{.type = MutationCommand::Type::READ_COLUMN, .column_name = column.name, .data_type = column.type}); } + else if (dropped_columns.contains(column.name)) + { + /// Not needed for compact parts (not executed), added here only to produce correct + /// set of columns for new part and their serializations + for_file_renames.push_back( + { + .type = MutationCommand::Type::DROP_COLUMN, + .column_name = column.name, + }); + } } } else @@ -231,10 +245,14 @@ getColumnsForNewDataPart( /// If we don't have this column in source part, than we don't need to materialize it if (!part_columns.has(command.column_name)) + { continue; + } if (command.type == MutationCommand::DROP_COLUMN) + { removed_columns.insert(command.column_name); + } if (command.type == MutationCommand::RENAME_COLUMN) { From 38e89b892a3b95e337441e83b872da3cb919ebe4 Mon Sep 17 00:00:00 2001 From: Robert Schulze Date: Tue, 11 Apr 2023 20:31:35 +0000 Subject: [PATCH 096/103] Fix roundAge() --- src/Functions/roundAge.cpp | 3 +-- tests/queries/0_stateless/00968_roundAge.sql | 3 ++- 2 files changed, 3 insertions(+), 3 deletions(-) diff --git a/src/Functions/roundAge.cpp b/src/Functions/roundAge.cpp index d2503bb6938..5c6f9b65279 100644 --- a/src/Functions/roundAge.cpp +++ b/src/Functions/roundAge.cpp @@ -15,8 +15,7 @@ struct RoundAgeImpl static inline ResultType apply(A x) { - return x < 1 ? 0 - : (x < 18 ? 17 + return (x < 18 ? 0 : (x < 25 ? 18 : (x < 35 ? 25 : (x < 45 ? 35 diff --git a/tests/queries/0_stateless/00968_roundAge.sql b/tests/queries/0_stateless/00968_roundAge.sql index c8e5a5579f2..f092dbf6560 100644 --- a/tests/queries/0_stateless/00968_roundAge.sql +++ b/tests/queries/0_stateless/00968_roundAge.sql @@ -1,7 +1,8 @@ SELECT roundAge(0); +SELECT roundAge(12); SELECT roundAge(18); SELECT roundAge(25); SELECT roundAge(35); SELECT roundAge(45); SELECT roundAge(55); -SELECT roundAge(56); \ No newline at end of file +SELECT roundAge(56); From 58d98fa21f2e8de2ada4414c42f65d8314c2d48b Mon Sep 17 00:00:00 2001 From: Robert Schulze Date: Wed, 12 Apr 2023 07:57:00 +0000 Subject: [PATCH 097/103] Revert "Fix roundAge()" This reverts commit 38e89b892a3b95e337441e83b872da3cb919ebe4. --- src/Functions/roundAge.cpp | 3 ++- tests/queries/0_stateless/00968_roundAge.sql | 3 +-- 2 files changed, 3 insertions(+), 3 deletions(-) diff --git a/src/Functions/roundAge.cpp b/src/Functions/roundAge.cpp index 5c6f9b65279..d2503bb6938 100644 --- a/src/Functions/roundAge.cpp +++ b/src/Functions/roundAge.cpp @@ -15,7 +15,8 @@ struct RoundAgeImpl static inline ResultType apply(A x) { - return (x < 18 ? 0 + return x < 1 ? 0 + : (x < 18 ? 17 : (x < 25 ? 18 : (x < 35 ? 25 : (x < 45 ? 35 diff --git a/tests/queries/0_stateless/00968_roundAge.sql b/tests/queries/0_stateless/00968_roundAge.sql index f092dbf6560..c8e5a5579f2 100644 --- a/tests/queries/0_stateless/00968_roundAge.sql +++ b/tests/queries/0_stateless/00968_roundAge.sql @@ -1,8 +1,7 @@ SELECT roundAge(0); -SELECT roundAge(12); SELECT roundAge(18); SELECT roundAge(25); SELECT roundAge(35); SELECT roundAge(45); SELECT roundAge(55); -SELECT roundAge(56); +SELECT roundAge(56); \ No newline at end of file From 9c653197a4630b8a32b2e816ec1deb507c81a370 Mon Sep 17 00:00:00 2001 From: Robert Schulze Date: Wed, 12 Apr 2023 08:05:46 +0000 Subject: [PATCH 098/103] Update roundAge() docs --- docs/en/sql-reference/functions/rounding-functions.md | 9 ++++++++- 1 file changed, 8 insertions(+), 1 deletion(-) diff --git a/docs/en/sql-reference/functions/rounding-functions.md b/docs/en/sql-reference/functions/rounding-functions.md index 01ee720cfd3..40ff958e0ae 100644 --- a/docs/en/sql-reference/functions/rounding-functions.md +++ b/docs/en/sql-reference/functions/rounding-functions.md @@ -194,7 +194,14 @@ Accepts a number. If the number is less than one, it returns 0. Otherwise, it ro ## roundAge(num) -Accepts a number. If the number is less than 18, it returns 0. Otherwise, it rounds the number down to a number from the set: 18, 25, 35, 45, 55. +Accepts a number. If the number is +- smaller than 1, it returns 0, +- between 1 and 17, it returns 17, +- between 18 and 24, it returns 18, +- between 25 and 34, it returns 25, +- between 35 and 44, it returns 35, +- between 45 and 54, it returns 45, +- larger than 55, it returns 55. ## roundDown(num, arr) From f4573d64df2a5660275b8f3ed0d88f591d8575e7 Mon Sep 17 00:00:00 2001 From: alesapin Date: Wed, 12 Apr 2023 12:52:40 +0200 Subject: [PATCH 099/103] Fix test --- ...rofile_events_from_query_log_and_client.reference | 12 ++++++------ 1 file changed, 6 insertions(+), 6 deletions(-) diff --git a/tests/queries/0_stateless/02675_profile_events_from_query_log_and_client.reference b/tests/queries/0_stateless/02675_profile_events_from_query_log_and_client.reference index 2d41f5dae89..00e93b1db3d 100644 --- a/tests/queries/0_stateless/02675_profile_events_from_query_log_and_client.reference +++ b/tests/queries/0_stateless/02675_profile_events_from_query_log_and_client.reference @@ -9,15 +9,15 @@ CHECK WITH query_log QueryFinish S3CreateMultipartUpload 1 S3UploadPart 1 S3CompleteMultipartUpload 1 S3PutObject 0 CREATE INSERT - [ 0 ] FileOpen: 7 + [ 0 ] FileOpen: 8 READ INSERT and READ INSERT - [ 0 ] FileOpen: 7 - [ 0 ] FileOpen: 7 + [ 0 ] FileOpen: 8 + [ 0 ] FileOpen: 8 DROP CHECK with query_log -QueryFinish INSERT INTO times SELECT now() + INTERVAL 1 day SETTINGS optimize_on_insert = 0; FileOpen 7 +QueryFinish INSERT INTO times SELECT now() + INTERVAL 1 day SETTINGS optimize_on_insert = 0; FileOpen 8 QueryFinish SELECT \'1\', min(t) FROM times; FileOpen 0 -QueryFinish INSERT INTO times SELECT now() + INTERVAL 2 day SETTINGS optimize_on_insert = 0; FileOpen 7 +QueryFinish INSERT INTO times SELECT now() + INTERVAL 2 day SETTINGS optimize_on_insert = 0; FileOpen 8 QueryFinish SELECT \'2\', min(t) FROM times; FileOpen 0 -QueryFinish INSERT INTO times SELECT now() + INTERVAL 3 day SETTINGS optimize_on_insert = 0; FileOpen 7 +QueryFinish INSERT INTO times SELECT now() + INTERVAL 3 day SETTINGS optimize_on_insert = 0; FileOpen 8 From 6b441b40ddcb2a67432e42c577a42fccbbb3ddcc Mon Sep 17 00:00:00 2001 From: Robert Schulze Date: Wed, 12 Apr 2023 11:16:30 +0000 Subject: [PATCH 100/103] Update MergeTree syntax for optional index granularity argument This is follow up to #45451. --- src/Storages/MergeTree/registerStorageMergeTree.cpp | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/src/Storages/MergeTree/registerStorageMergeTree.cpp b/src/Storages/MergeTree/registerStorageMergeTree.cpp index 00478dea9d3..5209be0fd3b 100644 --- a/src/Storages/MergeTree/registerStorageMergeTree.cpp +++ b/src/Storages/MergeTree/registerStorageMergeTree.cpp @@ -70,8 +70,8 @@ CREATE TABLE [IF NOT EXISTS] [db.]table_name [ON CLUSTER cluster] name1 [type1] [DEFAULT|MATERIALIZED|ALIAS expr1] [TTL expr1], name2 [type2] [DEFAULT|MATERIALIZED|ALIAS expr2] [TTL expr2], ... - INDEX index_name1 expr1 TYPE type1(...) GRANULARITY value1, - INDEX index_name2 expr2 TYPE type2(...) GRANULARITY value2 + INDEX index_name1 expr1 TYPE type1(...) [GRANULARITY value1], + INDEX index_name2 expr2 TYPE type2(...) [GRANULARITY value2] ) ENGINE = MergeTree() ORDER BY expr [PARTITION BY expr] From 2cd3512a5ddcc6e88aced2a6379ef570c519bfe3 Mon Sep 17 00:00:00 2001 From: MikhailBurdukov <102754618+MikhailBurdukov@users.noreply.github.com> Date: Wed, 12 Apr 2023 15:06:02 +0300 Subject: [PATCH 101/103] ClickHouse startup error when loading a distributed table that depends on a dictionary (#48419) * Test for start failure. * Handling the dictionary as shard key. * Added integration test and fixed style * Revert extra test * Fix style * Fix style * Refactoring * Fix build * style fix --- src/Databases/DDLLoadingDependencyVisitor.cpp | 34 ++++++++++++++--- .../test_dictionaries_dependency/test.py | 37 +++++++++++++++++++ 2 files changed, 66 insertions(+), 5 deletions(-) diff --git a/src/Databases/DDLLoadingDependencyVisitor.cpp b/src/Databases/DDLLoadingDependencyVisitor.cpp index 3a61f821629..22dc0b4af7b 100644 --- a/src/Databases/DDLLoadingDependencyVisitor.cpp +++ b/src/Databases/DDLLoadingDependencyVisitor.cpp @@ -115,10 +115,13 @@ void DDLLoadingDependencyVisitor::visit(const ASTStorage & storage, Data & data) { if (!storage.engine) return; - if (storage.engine->name != "Dictionary") - return; - extractTableNameFromArgument(*storage.engine, data, 0); + if (storage.engine->name == "Distributed") + /// Checks that dict* expression was used as sharding_key and builds dependency between the dictionary and current table. + /// Distributed(logs, default, hits[, sharding_key[, policy_name]]) + extractTableNameFromArgument(*storage.engine, data, 3); + else if (storage.engine->name == "Dictionary") + extractTableNameFromArgument(*storage.engine, data, 0); } @@ -131,7 +134,29 @@ void DDLLoadingDependencyVisitor::extractTableNameFromArgument(const ASTFunction QualifiedTableName qualified_name; const auto * arg = function.arguments->as()->children[arg_idx].get(); - if (const auto * literal = arg->as()) + + if (const auto * dict_function = arg->as()) + { + if (!functionIsDictGet(dict_function->name)) + return; + + /// Get the dictionary name from `dict*` function. + const auto * literal_arg = dict_function->arguments->as()->children[0].get(); + const auto * dictionary_name = literal_arg->as(); + + if (!dictionary_name) + return; + + if (dictionary_name->value.getType() != Field::Types::String) + return; + + auto maybe_qualified_name = QualifiedTableName::tryParseFromString(dictionary_name->value.get()); + if (!maybe_qualified_name) + return; + + qualified_name = std::move(*maybe_qualified_name); + } + else if (const auto * literal = arg->as()) { if (literal->value.getType() != Field::Types::String) return; @@ -167,5 +192,4 @@ void DDLLoadingDependencyVisitor::extractTableNameFromArgument(const ASTFunction } data.dependencies.emplace(std::move(qualified_name)); } - } diff --git a/tests/integration/test_dictionaries_dependency/test.py b/tests/integration/test_dictionaries_dependency/test.py index 2042db69fa2..05d6afd35c7 100644 --- a/tests/integration/test_dictionaries_dependency/test.py +++ b/tests/integration/test_dictionaries_dependency/test.py @@ -154,3 +154,40 @@ def test_dependency_via_dictionary_database(node): node.query(f"DROP DICTIONARY IF EXISTS {d_name} SYNC") node.query("DROP DATABASE dict_db SYNC") node.restart_clickhouse() + + +@pytest.mark.parametrize("node", nodes) +def test_dependent_dict_table_distr(node): + query = node.query + query("CREATE DATABASE test_db;") + query( + "CREATE TABLE test_db.test(id UInt32,data UInt32,key1 UInt8,key2 UInt8) ENGINE=MergeTree ORDER BY id;" + ) + query( + "INSERT INTO test_db.test SELECT abs(rand32())%100, rand32()%1000, abs(rand32())%1, abs(rand32())%1 FROM numbers(100);" + ) + query( + "CREATE TABLE test_db.dictback (key1 UInt8,key2 UInt8, value UInt8) ENGINE=MergeTree ORDER BY key1;" + ) + query("INSERT INTO test_db.dictback VALUES (0,0,0);") + + query( + "CREATE DICTIONARY test_db.mdict (key1 UInt8,key2 UInt8, value UInt8) PRIMARY KEY key1,key2" + " SOURCE(CLICKHOUSE(HOST 'localhost' PORT tcpPort() DB 'test_db' TABLE 'dictback'))" + " LIFETIME(MIN 100 MAX 100) LAYOUT(COMPLEX_KEY_CACHE(SIZE_IN_CELLS 1000));" + ) + + query( + "CREATE TABLE test_db.distr (id UInt32, data UInt32, key1 UInt8, key2 UInt8)" + " ENGINE = Distributed('test_shard_localhost', test_db, test, dictGetOrDefault('test_db.mdict','value',(key1,key2),0));" + ) + + # Tables should load in the correct order. + node.restart_clickhouse() + + query("DETACH TABLE test_db.distr;") + query("ATTACH TABLE test_db.distr;") + + node.restart_clickhouse() + + query("DROP DATABASE IF EXISTS test_db;") From 5f223f24c624d261f5de43be18f7f1aeca8b6497 Mon Sep 17 00:00:00 2001 From: Alexander Tokmakov Date: Wed, 12 Apr 2023 15:50:06 +0200 Subject: [PATCH 102/103] better messages formatting --- utils/ci-slack-bot/ci-slack-bot.py | 48 +++++++++++++++++++++++------- 1 file changed, 37 insertions(+), 11 deletions(-) diff --git a/utils/ci-slack-bot/ci-slack-bot.py b/utils/ci-slack-bot/ci-slack-bot.py index 76f60a80c04..895b20b1ed7 100755 --- a/utils/ci-slack-bot/ci-slack-bot.py +++ b/utils/ci-slack-bot/ci-slack-bot.py @@ -26,20 +26,22 @@ else: DRY_RUN_MARK = "" -MAX_FAILURES_DEFAULT = 50 +MAX_FAILURES_DEFAULT = 40 SLACK_URL_DEFAULT = DRY_RUN_MARK -EXTENDED_CHECK_PERIOD_MUL = 3 FLAKY_ALERT_PROBABILITY = 0.20 -# Find tests that failed in master during the last check_period * 12 hours, +# Slack has a stupid limitation on message size, it splits long messages into multiple ones breaking formatting +MESSAGE_LENGTH_LIMIT = 4000 + +# Find tests that failed in master during the last check_period * 24 hours, # but did not fail during the last 2 weeks. Assuming these tests were broken recently. -# Counts number of failures in check_period and check_period * 12 time windows +# Counts number of failures in check_period and check_period * 24 time windows # to distinguish rare flaky tests from completely broken tests NEW_BROKEN_TESTS_QUERY = """ WITH 1 AS check_period, - check_period * 12 AS extended_check_period, + check_period * 24 AS extended_check_period, now() as now SELECT test_name, @@ -155,7 +157,7 @@ def format_failed_tests_list(failed_tests, failure_type): for name, report in failed_tests: cidb_url = get_play_url(ALL_RECENT_FAILURES_QUERY.format(name)) - res += " - *{}* - <{}|Report> - <{}|CI DB> \n".format( + res += "- *{}* - <{}|Report> - <{}|CI DB> \n".format( name, report, cidb_url ) return res @@ -173,11 +175,14 @@ def get_new_broken_tests_message(failed_tests): if len(broken_tests) > 0: msg += format_failed_tests_list(broken_tests, "*BROKEN*") elif random.random() > FLAKY_ALERT_PROBABILITY: - # Should we report fuzzers unconditionally? - print("Will not report flaky tests to avoid noise: ", flaky_tests) - return None + looks_like_fuzzer = [x[0].count(" ") > 2 for x in flaky_tests] + if not any(looks_like_fuzzer): + print("Will not report flaky tests to avoid noise: ", flaky_tests) + return None if len(flaky_tests) > 0: + if len(msg) > 0: + msg += "\n" msg += format_failed_tests_list(flaky_tests, "flaky") return msg @@ -187,7 +192,7 @@ def get_too_many_failures_message_impl(failures_count): MAX_FAILURES = int(os.environ.get("MAX_FAILURES", MAX_FAILURES_DEFAULT)) curr_failures = int(failures_count[0][0]) prev_failures = int(failures_count[0][1]) - if curr_failures == 0: + if curr_failures == 0 and prev_failures != 0: return ( "Looks like CI is completely broken: there are *no failures* at all... 0_o" ) @@ -213,7 +218,22 @@ def get_too_many_failures_message(failures_count): return msg -def send_to_slack(message): +def split_slack_message(long_message): + lines = long_message.split("\n") + messages = [] + curr_msg = "" + for line in lines: + if len(curr_msg) + len(line) < MESSAGE_LENGTH_LIMIT: + curr_msg += "\n" + curr_msg += line + else: + messages.append(curr_msg) + curr_msg = line + messages.append(curr_msg) + return messages + + +def send_to_slack_impl(message): SLACK_URL = os.environ.get("SLACK_URL", SLACK_URL_DEFAULT) if SLACK_URL == DRY_RUN_MARK: return @@ -230,6 +250,12 @@ def send_to_slack(message): ) +def send_to_slack(message): + messages = split_slack_message(message) + for msg in messages: + send_to_slack_impl(msg) + + def query_and_alert_if_needed(query, get_message_func): query_res = run_clickhouse_query(query) print("Got result {} for query {}", query_res, query) From 0a1815b9bb8b9cdb0a5ad7a98eaa7f16d53a1bdb Mon Sep 17 00:00:00 2001 From: Aleksei Filatov <68555560+aalexfvk@users.noreply.github.com> Date: Wed, 12 Apr 2023 20:38:05 +0300 Subject: [PATCH 103/103] Add new trusted contributors (#48715) Co-authored-by: Nikita Mikhaylov --- tests/ci/workflow_approve_rerun_lambda/app.py | 2 ++ 1 file changed, 2 insertions(+) diff --git a/tests/ci/workflow_approve_rerun_lambda/app.py b/tests/ci/workflow_approve_rerun_lambda/app.py index decf6ce0393..0f6d10de751 100644 --- a/tests/ci/workflow_approve_rerun_lambda/app.py +++ b/tests/ci/workflow_approve_rerun_lambda/app.py @@ -124,6 +124,8 @@ TRUSTED_CONTRIBUTORS = { "tylerhannan", # ClickHouse Employee "myrrc", # Mike Kot, DoubleCloud "thevar1able", # ClickHouse Employee + "aalexfvk", + "MikhailBurdukov", ] }