From dd3ea8d31b1b5615bcdd6b671f5e6e86315e8dda Mon Sep 17 00:00:00 2001 From: liyang830 Date: Fri, 10 Mar 2023 20:52:27 +0800 Subject: [PATCH 001/117] feat: modify materalized view query, check inner table structure --- src/Storages/StorageMaterializedView.cpp | 13 +++++++++++++ ...erialized_view_query_has_inner_table.reference | 2 ++ ...er_materialized_view_query_has_inner_table.sql | 15 +++++++++++++++ 3 files changed, 30 insertions(+) create mode 100644 tests/queries/0_stateless/25340_alter_materialized_view_query_has_inner_table.reference create mode 100644 tests/queries/0_stateless/25340_alter_materialized_view_query_has_inner_table.sql diff --git a/src/Storages/StorageMaterializedView.cpp b/src/Storages/StorageMaterializedView.cpp index ae3fa62b38c..0bbd689043c 100644 --- a/src/Storages/StorageMaterializedView.cpp +++ b/src/Storages/StorageMaterializedView.cpp @@ -7,6 +7,7 @@ #include #include #include +#include #include #include #include @@ -269,6 +270,18 @@ void StorageMaterializedView::alter( DatabaseCatalog::instance().updateViewDependency(old_select.select_table_id, table_id, new_select.select_table_id, table_id); new_metadata.setSelectQuery(new_select); + + /// check materialized view inner table structure + if (has_inner_table) + { + const Block & block = InterpreterSelectWithUnionQuery::getSampleBlock(new_select.select_query, local_context); + for (const auto & col : block.getColumnsWithTypeAndName()) + { + if (!tryGetTargetTable()->getInMemoryMetadata().columns.has(col.name)) + throw Exception(ErrorCodes::QUERY_IS_NOT_SUPPORTED_IN_MATERIALIZED_VIEW, "column {} is not in materialized view inner table", col.name); + } + } + } /// end modify query diff --git a/tests/queries/0_stateless/25340_alter_materialized_view_query_has_inner_table.reference b/tests/queries/0_stateless/25340_alter_materialized_view_query_has_inner_table.reference new file mode 100644 index 00000000000..1191247b6d9 --- /dev/null +++ b/tests/queries/0_stateless/25340_alter_materialized_view_query_has_inner_table.reference @@ -0,0 +1,2 @@ +1 +2 diff --git a/tests/queries/0_stateless/25340_alter_materialized_view_query_has_inner_table.sql b/tests/queries/0_stateless/25340_alter_materialized_view_query_has_inner_table.sql new file mode 100644 index 00000000000..73bbac59a95 --- /dev/null +++ b/tests/queries/0_stateless/25340_alter_materialized_view_query_has_inner_table.sql @@ -0,0 +1,15 @@ +DROP TABLE IF EXISTS src_table; +DROP TABLE IF EXISTS mv; + +CREATE TABLE src_table (`a` UInt32, `b` UInt32) ENGINE = MergeTree ORDER BY a; +CREATE MATERIALIZED VIEW mv UUID '2bad6d75-86fe-4da0-815b-2c7410253941' (`a` UInt32) ENGINE = MergeTree ORDER BY a AS SELECT a FROM src_table; + +INSERT INTO src_table (a, b) VALUES (1, 1), (2, 2); + +SELECT * FROM mv; + +SET allow_experimental_alter_materialized_view_structure = 1; +ALTER TABLE mv MODIFY QUERY SELECT a, b FROM src_table; -- {serverError QUERY_IS_NOT_SUPPORTED_IN_MATERIALIZED_VIEW} + +DROP TABLE src_table; +DROP TABLE mv; \ No newline at end of file From 3f5853b970dd205465a5593d5786c1f8a4d82cc7 Mon Sep 17 00:00:00 2001 From: AN Date: Fri, 27 Oct 2023 19:17:13 +0300 Subject: [PATCH 002/117] Update index.md MIME-Version: 1.0 Content-Type: text/plain; charset=UTF-8 Content-Transfer-Encoding: 8bit Punctuation fixes, узел→сервер as suggested by alexei-milovidov at https://github.com/ClickHouse/ClickHouse/pull/56040#issuecomment-1783155867, консистентность → согласованность (standard translation instead of calque) --- docs/ru/index.md | 18 +++++++++--------- 1 file changed, 9 insertions(+), 9 deletions(-) diff --git a/docs/ru/index.md b/docs/ru/index.md index 78bb382753b..a9a666b18db 100644 --- a/docs/ru/index.md +++ b/docs/ru/index.md @@ -41,7 +41,7 @@ ClickHouse — столбцовая система управления база Разный порядок хранения данных лучше подходит для разных сценариев работы. Сценарий работы с данными — это то, какие производятся запросы, как часто и в каком соотношении; сколько читается данных на запросы каждого вида — строк, столбцов, байтов; как соотносятся чтения и обновления данных; какой рабочий размер данных и насколько локально он используется; используются ли транзакции и с какой изолированностью; какие требования к дублированию данных и логической целостности; требования к задержкам на выполнение и пропускной способности запросов каждого вида и т. п. -Чем больше нагрузка на систему, тем более важной становится специализация под сценарий работы, и тем более конкретной становится эта специализация. Не существует системы, одинаково хорошо подходящей под существенно различные сценарии работы. Если система подходит под широкое множество сценариев работы, то при достаточно большой нагрузке, система будет справляться со всеми сценариями работы плохо, или справляться хорошо только с одним из сценариев работы. +Чем больше нагрузка на систему, тем более важной становится специализация под сценарий работы, и тем более конкретной становится эта специализация. Не существует системы, одинаково хорошо подходящей под существенно различные сценарии работы. Если система подходит под широкое множество сценариев работы, то при достаточно большой нагрузке система будет справляться со всеми сценариями работы плохо, или справляться хорошо только с одним из сценариев работы. ## Ключевые особенности OLAP-сценария работы {#kliuchevye-osobennosti-olap-stsenariia-raboty} @@ -53,11 +53,11 @@ ClickHouse — столбцовая система управления база - запросы идут сравнительно редко (обычно не более сотни в секунду на сервер); - при выполнении простых запросов, допустимы задержки в районе 50 мс; - значения в столбцах достаточно мелкие — числа и небольшие строки (например, 60 байт на URL); -- требуется высокая пропускная способность при обработке одного запроса (до миллиардов строк в секунду на один узел); +- требуется высокая пропускная способность при обработке одного запроса (до миллиардов строк в секунду на один сервер); - транзакции отсутствуют; -- низкие требования к консистентности данных; -- в запросе одна большая таблица, все таблицы кроме одной маленькие; -- результат выполнения запроса существенно меньше исходных данных — то есть данные фильтруются или агрегируются; результат выполнения помещается в оперативную память одного узла. +- низкие требования к согласованности данных; +- в запросе одна большая таблица, все остальные таблицы из запроса — маленькие; +- результат выполнения запроса существенно меньше исходных данных — то есть данные фильтруются или агрегируются; результат выполнения помещается в оперативную память одного сервера. Легко видеть, что OLAP-сценарий работы существенно отличается от других распространённых сценариев работы (например, OLTP или Key-Value сценариев работы). Таким образом, не имеет никакого смысла пытаться использовать OLTP-системы или системы класса «ключ — значение» для обработки аналитических запросов, если вы хотите получить приличную производительность («выше плинтуса»). Например, если вы попытаетесь использовать для аналитики MongoDB или Redis — вы получите анекдотически низкую производительность по сравнению с OLAP-СУБД. @@ -77,11 +77,11 @@ ClickHouse — столбцовая система управления база ### По вводу-выводу {#po-vvodu-vyvodu} -1. Для выполнения аналитического запроса, требуется прочитать небольшое количество столбцов таблицы. В столбцовой БД для этого можно читать только нужные данные. Например, если вам требуется только 5 столбцов из 100, то следует рассчитывать на 20-кратное уменьшение ввода-вывода. -2. Так как данные читаются пачками, то их проще сжимать. Данные, лежащие по столбцам также лучше сжимаются. За счёт этого, дополнительно уменьшается объём ввода-вывода. -3. За счёт уменьшения ввода-вывода, больше данных влезает в системный кэш. +1. Для выполнения аналитического запроса требуется прочитать небольшое количество столбцов таблицы. В столбцовой БД для этого можно читать только нужные данные. Например, если вам требуется только 5 столбцов из 100, то следует рассчитывать на 20-кратное уменьшение ввода-вывода. +2. Так как данные читаются пачками, то их проще сжимать. Данные, лежащие по столбцам, также лучше сжимаются. За счёт этого, дополнительно уменьшается объём ввода-вывода. +3. За счёт уменьшения ввода-вывода больше данных влезает в системный кэш. -Например, для запроса «посчитать количество записей для каждой рекламной системы», требуется прочитать один столбец «идентификатор рекламной системы», который занимает 1 байт в несжатом виде. Если большинство переходов было не с рекламных систем, то можно рассчитывать хотя бы на десятикратное сжатие этого столбца. При использовании быстрого алгоритма сжатия, возможно разжатие данных со скоростью более нескольких гигабайт несжатых данных в секунду. То есть, такой запрос может выполняться со скоростью около нескольких миллиардов строк в секунду на одном сервере. На практике, такая скорость действительно достигается. +Например, для запроса «посчитать количество записей для каждой рекламной системы» требуется прочитать один столбец «идентификатор рекламной системы», который занимает 1 байт в несжатом виде. Если большинство переходов было не с рекламных систем, то можно рассчитывать хотя бы на десятикратное сжатие этого столбца. При использовании быстрого алгоритма сжатия возможно разжатие данных со скоростью более нескольких гигабайт несжатых данных в секунду. То есть такой запрос может выполняться со скоростью около нескольких миллиардов строк в секунду на одном сервере. На практике такая скорость действительно достигается. ### По вычислениям {#po-vychisleniiam} From 1549725eddb6db299ba0297de21a51411607d2a3 Mon Sep 17 00:00:00 2001 From: unashi Date: Sun, 18 Feb 2024 19:26:12 +0800 Subject: [PATCH 003/117] [feature]: allow to attach parts from a different disk --- src/Storages/MergeTree/MergeTreeData.cpp | 13 ++ src/Storages/MergeTree/MergeTreeData.h | 9 + .../MergeTree/MergeTreeDataPartCloner.cpp | 70 ++++++- src/Storages/StorageMergeTree.cpp | 45 +++-- src/Storages/StorageReplicatedMergeTree.cpp | 42 ++-- .../__init__.py | 0 .../configs/remote_servers.xml | 17 ++ .../test_attach_partition_using_copy/test.py | 183 ++++++++++++++++++ 8 files changed, 353 insertions(+), 26 deletions(-) create mode 100644 tests/integration/test_attach_partition_using_copy/__init__.py create mode 100644 tests/integration/test_attach_partition_using_copy/configs/remote_servers.xml create mode 100644 tests/integration/test_attach_partition_using_copy/test.py diff --git a/src/Storages/MergeTree/MergeTreeData.cpp b/src/Storages/MergeTree/MergeTreeData.cpp index 3ca746a7197..56710b157de 100644 --- a/src/Storages/MergeTree/MergeTreeData.cpp +++ b/src/Storages/MergeTree/MergeTreeData.cpp @@ -7085,6 +7085,19 @@ std::pair MergeTreeData::cloneAn this, src_part, metadata_snapshot, dst_part_info, tmp_part_prefix, require_part_metadata, params, read_settings, write_settings); } +std::pair MergeTreeData::cloneAndLoadDataPartOnOtherDisk( + const MergeTreeData::DataPartPtr & src_part, + const String & tmp_part_prefix, + const MergeTreePartInfo & dst_part_info, + const StorageMetadataPtr & metadata_snapshot, + const IDataPartStorage::ClonePartParams & params, + const ReadSettings & read_settings, + const WriteSettings & write_settings) +{ + return MergeTreeDataPartCloner::clone( + this, src_part, metadata_snapshot, dst_part_info, tmp_part_prefix, require_part_metadata, params, read_settings, write_settings); +} + std::pair MergeTreeData::cloneAndLoadPartOnSameDiskWithDifferentPartitionKey( const MergeTreeData::DataPartPtr & src_part, const MergeTreePartition & new_partition, diff --git a/src/Storages/MergeTree/MergeTreeData.h b/src/Storages/MergeTree/MergeTreeData.h index dfdc22baa8f..a24362f68fc 100644 --- a/src/Storages/MergeTree/MergeTreeData.h +++ b/src/Storages/MergeTree/MergeTreeData.h @@ -866,6 +866,15 @@ public: ContextPtr local_context, Int64 min_block, Int64 max_block); + + std::pair cloneAndLoadDataPartOnOtherDisk( + const MergeTreeData::DataPartPtr & src_part, + const String & tmp_part_prefix, + const MergeTreePartInfo & dst_part_info, + const StorageMetadataPtr & metadata_snapshot, + const IDataPartStorage::ClonePartParams & params, + const ReadSettings & read_settings, + const WriteSettings & write_settings); static std::pair createPartitionAndMinMaxIndexFromSourcePart( const MergeTreeData::DataPartPtr & src_part, diff --git a/src/Storages/MergeTree/MergeTreeDataPartCloner.cpp b/src/Storages/MergeTree/MergeTreeDataPartCloner.cpp index 04019d2c665..69b7abacc93 100644 --- a/src/Storages/MergeTree/MergeTreeDataPartCloner.cpp +++ b/src/Storages/MergeTree/MergeTreeDataPartCloner.cpp @@ -142,6 +142,30 @@ std::shared_ptr hardlinkAllFiles( params); } +std::shared_ptr cloneAllFiles( + MergeTreeData * merge_tree_data, + const DB::ReadSettings & read_settings, + const DB::WriteSettings & write_settings, + const DataPartStoragePtr & storage, + const String & path) +{ + for (const DiskPtr & disk : merge_tree_data->getStoragePolicy()->getDisks()) + { + try{ + return storage->clonePart( + merge_tree_data->getRelativeDataPath(), + path, + disk, + read_settings, + write_settings,{},{}); + }catch(...) { + LOG_TRACE(&Poco::Logger::get("MergeTreeDataPartCloner"), "Clone part on disk {} fail", disk->getName()); + } + } + LOG_FATAL(&Poco::Logger::get("MergeTreeDataPartCloner"), "Clone part on disks all fail"); + throw; +} + std::pair cloneSourcePart( MergeTreeData * merge_tree_data, const MergeTreeData::DataPartPtr & src_part, @@ -165,8 +189,18 @@ std::pair cloneSourcePart( auto src_part_storage = flushPartStorageToDiskIfInMemory( merge_tree_data, src_part, metadata_snapshot, tmp_part_prefix, tmp_dst_part_name, src_flushed_tmp_dir_lock, src_flushed_tmp_part); - - auto dst_part_storage = hardlinkAllFiles(merge_tree_data, read_settings, write_settings, src_part_storage, tmp_dst_part_name, params); + std::shared_ptr dst_part_storage {}; + if (params.copy_instead_of_hardlink) { + dst_part_storage = cloneAllFiles(merge_tree_data, read_settings, write_settings, src_part_storage, tmp_dst_part_name); + } else { + try{ + dst_part_storage = hardlinkAllFiles(merge_tree_data, read_settings, write_settings, src_part_storage, tmp_dst_part_name, params); + } catch(...){ + // Hard link fail. Try copy. + LOG_WARNING(&Poco::Logger::get("MergeTreeDataPartCloner"), "Hard link fail, try tp copy directly. to:{}, path:{}", merge_tree_data->getRelativeDataPath(),tmp_dst_part_name); + dst_part_storage = cloneAllFiles(merge_tree_data, read_settings, write_settings, src_part_storage, tmp_dst_part_name); + } + } if (params.metadata_version_to_write.has_value()) { @@ -275,6 +309,25 @@ std::pair cloneAndHand return std::make_pair(destination_part, std::move(temporary_directory_lock)); } + +std::pair cloneInsteadOfHardlinksAndProjections( + MergeTreeData * merge_tree_data, + const DataPartPtr & src_part, + const StorageMetadataPtr & metadata_snapshot, + const MergeTreePartInfo & dst_part_info, + const String & tmp_part_prefix, + const ReadSettings & read_settings, + const WriteSettings & write_settings, + const IDataPartStorage::ClonePartParams & params) +{ + chassert(!merge_tree_data->isStaticStorage()); + + auto [destination_part, temporary_directory_lock] = cloneSourcePart( + merge_tree_data, src_part, metadata_snapshot, dst_part_info, tmp_part_prefix, read_settings, write_settings, params); + + return std::make_pair(destination_part, std::move(temporary_directory_lock)); +} + } std::pair MergeTreeDataPartCloner::clone( @@ -288,10 +341,19 @@ std::pair MergeTreeDat const ReadSettings & read_settings, const WriteSettings & write_settings) { - auto [destination_part, temporary_directory_lock] = cloneAndHandleHardlinksAndProjections( + if (params.copy_instead_of_hardlink) + { + auto [destination_part, temporary_directory_lock] = cloneInsteadOfHardlinksAndProjections( merge_tree_data, src_part, metadata_snapshot, dst_part_info, tmp_part_prefix, read_settings, write_settings, params); + return std::make_pair(finalizePart(destination_part, params, require_part_metadata), std::move(temporary_directory_lock)); + } + else + { + auto [destination_part, temporary_directory_lock] = cloneAndHandleHardlinksAndProjections( + merge_tree_data, src_part, metadata_snapshot, dst_part_info, tmp_part_prefix, read_settings, write_settings, params); + return std::make_pair(finalizePart(destination_part, params, require_part_metadata), std::move(temporary_directory_lock)); - return std::make_pair(finalizePart(destination_part, params, require_part_metadata), std::move(temporary_directory_lock)); + } } std::pair MergeTreeDataPartCloner::cloneWithDistinctPartitionExpression( diff --git a/src/Storages/StorageMergeTree.cpp b/src/Storages/StorageMergeTree.cpp index 0f75c726bce..0f95fef9c6e 100644 --- a/src/Storages/StorageMergeTree.cpp +++ b/src/Storages/StorageMergeTree.cpp @@ -2118,17 +2118,40 @@ void StorageMergeTree::replacePartitionFrom(const StoragePtr & source_table, con else { MergeTreePartInfo dst_part_info(partition_id, index, index, src_part->info.level); - - auto [dst_part, part_lock] = cloneAndLoadDataPartOnSameDisk( - src_part, - TMP_PREFIX, - dst_part_info, - my_metadata_snapshot, - clone_params, - local_context->getReadSettings(), - local_context->getWriteSettings()); - dst_parts.emplace_back(std::move(dst_part)); - dst_parts_locks.emplace_back(std::move(part_lock)); + LOG_TRACE(log, "Partition exps are the same:part id: {}; number of disks:{}",dst_part_info.partition_id, this->getStoragePolicy()->getDisks().size()); + bool on_same_disk = false; + for (const DiskPtr & disk : this->getStoragePolicy()->getDisks()) + { + if (disk->getName() == src_part->getDataPartStorage().getDiskName()) + on_same_disk = true; + } + if (on_same_disk) + { + auto [dst_part, part_lock] = cloneAndLoadDataPartOnSameDisk( + src_part, + TMP_PREFIX, + dst_part_info, + my_metadata_snapshot, + clone_params, + local_context->getReadSettings(), + local_context->getWriteSettings()); + dst_parts.emplace_back(std::move(dst_part)); + dst_parts_locks.emplace_back(std::move(part_lock)); + } + else + { + clone_params.copy_instead_of_hardlink = true; + auto [dst_part, part_lock] = cloneAndLoadDataPartOnOtherDisk( + src_part, + TMP_PREFIX, + dst_part_info, + my_metadata_snapshot, + clone_params, + local_context->getReadSettings(), + local_context->getWriteSettings()); + dst_parts.emplace_back(std::move(dst_part)); + dst_parts_locks.emplace_back(std::move(part_lock)); + } } } diff --git a/src/Storages/StorageReplicatedMergeTree.cpp b/src/Storages/StorageReplicatedMergeTree.cpp index 6bd57cc4d6d..ba0d27fe612 100644 --- a/src/Storages/StorageReplicatedMergeTree.cpp +++ b/src/Storages/StorageReplicatedMergeTree.cpp @@ -8015,17 +8015,37 @@ void StorageReplicatedMergeTree::replacePartitionFrom( { MergeTreePartInfo dst_part_info(partition_id, index, index, src_part->info.level); - auto [dst_part, part_lock] = cloneAndLoadDataPartOnSameDisk( - src_part, - TMP_PREFIX, - dst_part_info, - metadata_snapshot, - clone_params, - query_context->getReadSettings(), - query_context->getWriteSettings()); - - dst_parts.emplace_back(dst_part); - dst_parts_locks.emplace_back(std::move(part_lock)); + bool on_same_disk = false; + for (const DiskPtr & disk : this->getStoragePolicy()->getDisks()) + if (disk->getName() == src_part->getDataPartStorage().getDiskName()) + on_same_disk = true; + if (on_same_disk) + { + auto [dst_part, part_lock] = cloneAndLoadDataPartOnSameDisk( + src_part, + TMP_PREFIX, + dst_part_info, + metadata_snapshot, + clone_params, + query_context->getReadSettings(), + query_context->getWriteSettings()); + dst_parts.emplace_back(dst_part); + dst_parts_locks.emplace_back(std::move(part_lock)); + } + else + { + clone_params.copy_instead_of_hardlink = true; + auto [dst_part, part_lock] = cloneAndLoadDataPartOnOtherDisk( + src_part, + TMP_PREFIX, + dst_part_info, + metadata_snapshot, + clone_params, + query_context->getReadSettings(), + query_context->getWriteSettings()); + dst_parts.emplace_back(dst_part); + dst_parts_locks.emplace_back(std::move(part_lock)); + } } src_parts.emplace_back(src_part); diff --git a/tests/integration/test_attach_partition_using_copy/__init__.py b/tests/integration/test_attach_partition_using_copy/__init__.py new file mode 100644 index 00000000000..e69de29bb2d diff --git a/tests/integration/test_attach_partition_using_copy/configs/remote_servers.xml b/tests/integration/test_attach_partition_using_copy/configs/remote_servers.xml new file mode 100644 index 00000000000..b40730e9f7d --- /dev/null +++ b/tests/integration/test_attach_partition_using_copy/configs/remote_servers.xml @@ -0,0 +1,17 @@ + + + + + true + + replica1 + 9000 + + + replica2 + 9000 + + + + + diff --git a/tests/integration/test_attach_partition_using_copy/test.py b/tests/integration/test_attach_partition_using_copy/test.py new file mode 100644 index 00000000000..effb5708cf3 --- /dev/null +++ b/tests/integration/test_attach_partition_using_copy/test.py @@ -0,0 +1,183 @@ +import pytest +from helpers.cluster import ClickHouseCluster +from helpers.test_tools import assert_eq_with_retry + +cluster = ClickHouseCluster(__file__) + +replica1 = cluster.add_instance( + "replica1", with_zookeeper=True, main_configs=["configs/remote_servers.xml"] +) +replica2 = cluster.add_instance( + "replica2", with_zookeeper=True, main_configs=["configs/remote_servers.xml"] +) + + +@pytest.fixture(scope="module") +def start_cluster(): + try: + cluster.start() + yield cluster + except Exception as ex: + print(ex) + finally: + cluster.shutdown() + + +def cleanup(nodes): + for node in nodes: + node.query("DROP TABLE IF EXISTS source SYNC") + node.query("DROP TABLE IF EXISTS destination SYNC") + + +def create_source_table(node, table_name, replicated): + replica = node.name + engine = ( + f"ReplicatedMergeTree('/clickhouse/tables/1/{table_name}', '{replica}')" + if replicated + else "MergeTree()" + ) + node.query_with_retry( + """ + ATTACH TABLE {table_name} UUID 'cf712b4f-2ca8-435c-ac23-c4393efe52f7' + ( + price UInt32, + date Date, + postcode1 LowCardinality(String), + postcode2 LowCardinality(String), + type Enum8('other' = 0, 'terraced' = 1, 'semi-detached' = 2, 'detached' = 3, 'flat' = 4), + is_new UInt8, + duration Enum8('unknown' = 0, 'freehold' = 1, 'leasehold' = 2), + addr1 String, + addr2 String, + street LowCardinality(String), + locality LowCardinality(String), + town LowCardinality(String), + district LowCardinality(String), + county LowCardinality(String) + ) + ENGINE = {engine} + ORDER BY (postcode1, postcode2, addr1, addr2) + SETTINGS disk = disk(type = web, endpoint = 'https://raw.githubusercontent.com/ClickHouse/web-tables-demo/main/web/') + """.format( + table_name=table_name, + engine=engine + ) + ) + + + +def create_destination_table(node, table_name, replicated): + replica = node.name + engine = ( + f"ReplicatedMergeTree('/clickhouse/tables/1/{table_name}', '{replica}')" + if replicated + else "MergeTree()" + ) + node.query_with_retry( + """ + CREATE TABLE {table_name} + ( + price UInt32, + date Date, + postcode1 LowCardinality(String), + postcode2 LowCardinality(String), + type Enum8('other' = 0, 'terraced' = 1, 'semi-detached' = 2, 'detached' = 3, 'flat' = 4), + is_new UInt8, + duration Enum8('unknown' = 0, 'freehold' = 1, 'leasehold' = 2), + addr1 String, + addr2 String, + street LowCardinality(String), + locality LowCardinality(String), + town LowCardinality(String), + district LowCardinality(String), + county LowCardinality(String) + ) + ENGINE = {engine} + ORDER BY (postcode1, postcode2, addr1, addr2) + """.format( + table_name=table_name, + engine=engine + ) + ) + +def test_both_mergtree(start_cluster): + create_source_table(replica1, "source", False) + create_destination_table(replica1, "destination", False) + + replica1.query( + f"ALTER TABLE destination ATTACH PARTITION tuple() FROM source" + ) + + assert_eq_with_retry( + replica1, f"SELECT toYear(date) AS year,round(avg(price)) AS price,bar(price, 0, 1000000, 80) FROM destination GROUP BY year ORDER BY year ASC", + replica1.query(f"SELECT toYear(date) AS year,round(avg(price)) AS price,bar(price, 0, 1000000, 80) FROM source GROUP BY year ORDER BY year ASC"), + ) + + assert_eq_with_retry( + replica1, f"SELECT town from destination LIMIT 1", + "SCARBOROUGH" + ) + + cleanup([replica1]) + +def test_all_replicated(start_cluster): + create_source_table(replica1, "source", True) + create_destination_table(replica1, "destination", True) + create_destination_table(replica2, "destination", True) + + replica1.query("SYSTEM SYNC REPLICA destination") + replica1.query( + f"ALTER TABLE destination ATTACH PARTITION tuple() FROM source" + ) + + assert_eq_with_retry( + replica1, f"SELECT toYear(date) AS year,round(avg(price)) AS price,bar(price, 0, 1000000, 80) FROM destination GROUP BY year ORDER BY year ASC", + replica1.query(f"SELECT toYear(date) AS year,round(avg(price)) AS price,bar(price, 0, 1000000, 80) FROM source GROUP BY year ORDER BY year ASC"), + ) + assert_eq_with_retry( + replica1, f"SELECT toYear(date) AS year,round(avg(price)) AS price,bar(price, 0, 1000000, 80) FROM source GROUP BY year ORDER BY year ASC", + replica2.query(f"SELECT toYear(date) AS year,round(avg(price)) AS price,bar(price, 0, 1000000, 80) FROM destination GROUP BY year ORDER BY year ASC"), + ) + + assert_eq_with_retry( + replica1, f"SELECT town from destination LIMIT 1", + "SCARBOROUGH" + ) + + assert_eq_with_retry( + replica2, f"SELECT town from destination LIMIT 1", + "SCARBOROUGH" + ) + + cleanup([replica1, replica2]) + +def test_only_destination_replicated(start_cluster): + create_source_table(replica1, "source", False) + create_destination_table(replica1, "destination", True) + create_destination_table(replica2, "destination", True) + + replica1.query("SYSTEM SYNC REPLICA destination") + replica1.query( + f"ALTER TABLE destination ATTACH PARTITION tuple() FROM source" + ) + + assert_eq_with_retry( + replica1, f"SELECT toYear(date) AS year,round(avg(price)) AS price,bar(price, 0, 1000000, 80) FROM destination GROUP BY year ORDER BY year ASC", + replica1.query(f"SELECT toYear(date) AS year,round(avg(price)) AS price,bar(price, 0, 1000000, 80) FROM source GROUP BY year ORDER BY year ASC"), + ) + assert_eq_with_retry( + replica1, f"SELECT toYear(date) AS year,round(avg(price)) AS price,bar(price, 0, 1000000, 80) FROM source GROUP BY year ORDER BY year ASC", + replica2.query(f"SELECT toYear(date) AS year,round(avg(price)) AS price,bar(price, 0, 1000000, 80) FROM destination GROUP BY year ORDER BY year ASC"), + ) + + assert_eq_with_retry( + replica1, f"SELECT town from destination LIMIT 1", + "SCARBOROUGH" + ) + + assert_eq_with_retry( + replica2, f"SELECT town from destination LIMIT 1", + "SCARBOROUGH" + ) + + cleanup([replica1, replica2]) From 80fe3f78d99caeaed733548ca65b6bd466730d51 Mon Sep 17 00:00:00 2001 From: unashi Date: Tue, 20 Feb 2024 11:12:09 +0800 Subject: [PATCH 004/117] [fix] black the python script --- .../test_attach_partition_using_copy/test.py | 92 ++++++++++--------- 1 file changed, 48 insertions(+), 44 deletions(-) diff --git a/tests/integration/test_attach_partition_using_copy/test.py b/tests/integration/test_attach_partition_using_copy/test.py index effb5708cf3..df5378742ae 100644 --- a/tests/integration/test_attach_partition_using_copy/test.py +++ b/tests/integration/test_attach_partition_using_copy/test.py @@ -59,13 +59,11 @@ def create_source_table(node, table_name, replicated): ORDER BY (postcode1, postcode2, addr1, addr2) SETTINGS disk = disk(type = web, endpoint = 'https://raw.githubusercontent.com/ClickHouse/web-tables-demo/main/web/') """.format( - table_name=table_name, - engine=engine + table_name=table_name, engine=engine ) ) - def create_destination_table(node, table_name, replicated): replica = node.name engine = ( @@ -95,89 +93,95 @@ def create_destination_table(node, table_name, replicated): ENGINE = {engine} ORDER BY (postcode1, postcode2, addr1, addr2) """.format( - table_name=table_name, - engine=engine + table_name=table_name, engine=engine ) ) + def test_both_mergtree(start_cluster): create_source_table(replica1, "source", False) create_destination_table(replica1, "destination", False) - replica1.query( - f"ALTER TABLE destination ATTACH PARTITION tuple() FROM source" - ) - + replica1.query(f"ALTER TABLE destination ATTACH PARTITION tuple() FROM source") + assert_eq_with_retry( - replica1, f"SELECT toYear(date) AS year,round(avg(price)) AS price,bar(price, 0, 1000000, 80) FROM destination GROUP BY year ORDER BY year ASC", - replica1.query(f"SELECT toYear(date) AS year,round(avg(price)) AS price,bar(price, 0, 1000000, 80) FROM source GROUP BY year ORDER BY year ASC"), + replica1, + f"SELECT toYear(date) AS year,round(avg(price)) AS price,bar(price, 0, 1000000, 80) FROM destination GROUP BY year ORDER BY year ASC", + replica1.query( + f"SELECT toYear(date) AS year,round(avg(price)) AS price,bar(price, 0, 1000000, 80) FROM source GROUP BY year ORDER BY year ASC" + ), ) - + assert_eq_with_retry( - replica1, f"SELECT town from destination LIMIT 1", - "SCARBOROUGH" + replica1, f"SELECT town from destination LIMIT 1", "SCARBOROUGH" ) cleanup([replica1]) + def test_all_replicated(start_cluster): create_source_table(replica1, "source", True) create_destination_table(replica1, "destination", True) create_destination_table(replica2, "destination", True) replica1.query("SYSTEM SYNC REPLICA destination") - replica1.query( - f"ALTER TABLE destination ATTACH PARTITION tuple() FROM source" + replica1.query(f"ALTER TABLE destination ATTACH PARTITION tuple() FROM source") + + assert_eq_with_retry( + replica1, + f"SELECT toYear(date) AS year,round(avg(price)) AS price,bar(price, 0, 1000000, 80) FROM destination GROUP BY year ORDER BY year ASC", + replica1.query( + f"SELECT toYear(date) AS year,round(avg(price)) AS price,bar(price, 0, 1000000, 80) FROM source GROUP BY year ORDER BY year ASC" + ), + ) + assert_eq_with_retry( + replica1, + f"SELECT toYear(date) AS year,round(avg(price)) AS price,bar(price, 0, 1000000, 80) FROM source GROUP BY year ORDER BY year ASC", + replica2.query( + f"SELECT toYear(date) AS year,round(avg(price)) AS price,bar(price, 0, 1000000, 80) FROM destination GROUP BY year ORDER BY year ASC" + ), ) assert_eq_with_retry( - replica1, f"SELECT toYear(date) AS year,round(avg(price)) AS price,bar(price, 0, 1000000, 80) FROM destination GROUP BY year ORDER BY year ASC", - replica1.query(f"SELECT toYear(date) AS year,round(avg(price)) AS price,bar(price, 0, 1000000, 80) FROM source GROUP BY year ORDER BY year ASC"), - ) - assert_eq_with_retry( - replica1, f"SELECT toYear(date) AS year,round(avg(price)) AS price,bar(price, 0, 1000000, 80) FROM source GROUP BY year ORDER BY year ASC", - replica2.query(f"SELECT toYear(date) AS year,round(avg(price)) AS price,bar(price, 0, 1000000, 80) FROM destination GROUP BY year ORDER BY year ASC"), + replica1, f"SELECT town from destination LIMIT 1", "SCARBOROUGH" ) assert_eq_with_retry( - replica1, f"SELECT town from destination LIMIT 1", - "SCARBOROUGH" - ) - - assert_eq_with_retry( - replica2, f"SELECT town from destination LIMIT 1", - "SCARBOROUGH" + replica2, f"SELECT town from destination LIMIT 1", "SCARBOROUGH" ) cleanup([replica1, replica2]) + def test_only_destination_replicated(start_cluster): create_source_table(replica1, "source", False) create_destination_table(replica1, "destination", True) create_destination_table(replica2, "destination", True) replica1.query("SYSTEM SYNC REPLICA destination") - replica1.query( - f"ALTER TABLE destination ATTACH PARTITION tuple() FROM source" + replica1.query(f"ALTER TABLE destination ATTACH PARTITION tuple() FROM source") + + assert_eq_with_retry( + replica1, + f"SELECT toYear(date) AS year,round(avg(price)) AS price,bar(price, 0, 1000000, 80) FROM destination GROUP BY year ORDER BY year ASC", + replica1.query( + f"SELECT toYear(date) AS year,round(avg(price)) AS price,bar(price, 0, 1000000, 80) FROM source GROUP BY year ORDER BY year ASC" + ), + ) + assert_eq_with_retry( + replica1, + f"SELECT toYear(date) AS year,round(avg(price)) AS price,bar(price, 0, 1000000, 80) FROM source GROUP BY year ORDER BY year ASC", + replica2.query( + f"SELECT toYear(date) AS year,round(avg(price)) AS price,bar(price, 0, 1000000, 80) FROM destination GROUP BY year ORDER BY year ASC" + ), ) assert_eq_with_retry( - replica1, f"SELECT toYear(date) AS year,round(avg(price)) AS price,bar(price, 0, 1000000, 80) FROM destination GROUP BY year ORDER BY year ASC", - replica1.query(f"SELECT toYear(date) AS year,round(avg(price)) AS price,bar(price, 0, 1000000, 80) FROM source GROUP BY year ORDER BY year ASC"), - ) - assert_eq_with_retry( - replica1, f"SELECT toYear(date) AS year,round(avg(price)) AS price,bar(price, 0, 1000000, 80) FROM source GROUP BY year ORDER BY year ASC", - replica2.query(f"SELECT toYear(date) AS year,round(avg(price)) AS price,bar(price, 0, 1000000, 80) FROM destination GROUP BY year ORDER BY year ASC"), + replica1, f"SELECT town from destination LIMIT 1", "SCARBOROUGH" ) assert_eq_with_retry( - replica1, f"SELECT town from destination LIMIT 1", - "SCARBOROUGH" - ) - - assert_eq_with_retry( - replica2, f"SELECT town from destination LIMIT 1", - "SCARBOROUGH" + replica2, f"SELECT town from destination LIMIT 1", "SCARBOROUGH" ) cleanup([replica1, replica2]) From 8de4a9dbfd32b7e82764a5c8efff3916b5c7ccda Mon Sep 17 00:00:00 2001 From: unashi Date: Tue, 20 Feb 2024 11:42:40 +0800 Subject: [PATCH 005/117] [fix] delete trailing whitespaces --- src/Storages/MergeTree/MergeTreeData.h | 2 +- src/Storages/StorageMergeTree.cpp | 2 +- 2 files changed, 2 insertions(+), 2 deletions(-) diff --git a/src/Storages/MergeTree/MergeTreeData.h b/src/Storages/MergeTree/MergeTreeData.h index 5f387385d38..081087acbaa 100644 --- a/src/Storages/MergeTree/MergeTreeData.h +++ b/src/Storages/MergeTree/MergeTreeData.h @@ -853,7 +853,7 @@ public: const IDataPartStorage::ClonePartParams & params, const ReadSettings & read_settings, const WriteSettings & write_settings); - + std::pair cloneAndLoadDataPartOnOtherDisk( const MergeTreeData::DataPartPtr & src_part, const String & tmp_part_prefix, diff --git a/src/Storages/StorageMergeTree.cpp b/src/Storages/StorageMergeTree.cpp index a2713775e65..47684925182 100644 --- a/src/Storages/StorageMergeTree.cpp +++ b/src/Storages/StorageMergeTree.cpp @@ -2081,7 +2081,7 @@ void StorageMergeTree::replacePartitionFrom(const StoragePtr & source_table, con /// This will generate unique name in scope of current server process. Int64 temp_index = insert_increment.get(); MergeTreePartInfo dst_part_info(partition_id, temp_index, temp_index, src_part->info.level); - + IDataPartStorage::ClonePartParams clone_params{.txn = local_context->getCurrentTransaction()}; LOG_TRACE(log, "Partition exps are the same:part id: {}; number of disks:{}",dst_part_info.partition_id, this->getStoragePolicy()->getDisks().size()); bool on_same_disk = false; From 6437877a712bfaf4a36c180b332a0d6a37981af1 Mon Sep 17 00:00:00 2001 From: unashi Date: Tue, 20 Feb 2024 20:31:59 +0800 Subject: [PATCH 006/117] [fix] add changelog; change some feature logic --- CHANGELOG.md | 1 + src/Storages/MergeTree/MergeTreeData.cpp | 10 +++++++--- src/Storages/StorageMergeTree.cpp | 1 - src/Storages/StorageReplicatedMergeTree.cpp | 3 +-- 4 files changed, 9 insertions(+), 6 deletions(-) diff --git a/CHANGELOG.md b/CHANGELOG.md index b3e5dd709ab..fd4ff90f841 100644 --- a/CHANGELOG.md +++ b/CHANGELOG.md @@ -42,6 +42,7 @@ * Added `FROM ` modifier for `SYSTEM SYNC REPLICA LIGHTWEIGHT` query. With the `FROM` modifier ensures we wait for fetches and drop-ranges only for the specified source replicas, as well as any replica not in zookeeper or with an empty source_replica. [#58393](https://github.com/ClickHouse/ClickHouse/pull/58393) ([Jayme Bird](https://github.com/jaymebrd)). * Added setting `update_insert_deduplication_token_in_dependent_materialized_views`. This setting allows to update insert deduplication token with table identifier during insert in dependent materialized views. Closes [#59165](https://github.com/ClickHouse/ClickHouse/issues/59165). [#59238](https://github.com/ClickHouse/ClickHouse/pull/59238) ([Maksim Kita](https://github.com/kitaisreal)). * Added statement `SYSTEM RELOAD ASYNCHRONOUS METRICS` which updates the asynchronous metrics. Mostly useful for testing and development. [#53710](https://github.com/ClickHouse/ClickHouse/pull/53710) ([Robert Schulze](https://github.com/rschu1ze)). +* Attach parts from a different disk `ALTER TABLE destination ATTACH PARTITION tuple() FROM source` where source is an [instant table](https://github.com/ClickHouse/web-tables-demo). [#60112](https://github.com/ClickHouse/ClickHouse/pull/60112)([Unalian](https://github.com/Unalian)). #### Performance Improvement * Coordination for parallel replicas is rewritten for better parallelism and cache locality. It has been tested for linear scalability on hundreds of replicas. It also got support for reading in order. [#57968](https://github.com/ClickHouse/ClickHouse/pull/57968) ([Nikita Taranov](https://github.com/nickitat)). diff --git a/src/Storages/MergeTree/MergeTreeData.cpp b/src/Storages/MergeTree/MergeTreeData.cpp index 64787d3509b..be1346e0ea2 100644 --- a/src/Storages/MergeTree/MergeTreeData.cpp +++ b/src/Storages/MergeTree/MergeTreeData.cpp @@ -8,6 +8,7 @@ #include #include #include +#include "Common/logger_useful.h" #include #include #include @@ -7170,7 +7171,9 @@ std::pair MergeTreeData::cloneAn } } if (!copy_successful) - throw; + { + LOG_FATAL(&Poco::Logger::get("MergeTreeData"), "Hard link fail, clone fail"); + } } @@ -7301,8 +7304,9 @@ std::pair MergeTreeData::cloneAn } } if (!copy_successful) - throw; - + { + LOG_FATAL( &Poco::Logger::get("MergeTreeData"), "Hard link fail, clone fail."); + } if (params.metadata_version_to_write.has_value()) { chassert(!params.keep_metadata_version); diff --git a/src/Storages/StorageMergeTree.cpp b/src/Storages/StorageMergeTree.cpp index 47684925182..0748ac2dbdf 100644 --- a/src/Storages/StorageMergeTree.cpp +++ b/src/Storages/StorageMergeTree.cpp @@ -2104,7 +2104,6 @@ void StorageMergeTree::replacePartitionFrom(const StoragePtr & source_table, con } else { - clone_params.copy_instead_of_hardlink = true; auto [dst_part, part_lock] = cloneAndLoadDataPartOnOtherDisk( src_part, TMP_PREFIX, diff --git a/src/Storages/StorageReplicatedMergeTree.cpp b/src/Storages/StorageReplicatedMergeTree.cpp index df261053360..2460d2704c4 100644 --- a/src/Storages/StorageReplicatedMergeTree.cpp +++ b/src/Storages/StorageReplicatedMergeTree.cpp @@ -7933,7 +7933,7 @@ void StorageReplicatedMergeTree::replacePartitionFrom( for (const DiskPtr & disk : this->getStoragePolicy()->getDisks()) if (disk->getName() == src_part->getDataPartStorage().getDiskName()) on_same_disk = true; - if (on_same_disk) + if (on_same_disk && !clone_params.copy_instead_of_hardlink) { auto [dst_part, part_lock] = cloneAndLoadDataPartOnSameDisk( src_part, @@ -7948,7 +7948,6 @@ void StorageReplicatedMergeTree::replacePartitionFrom( } else { - clone_params.copy_instead_of_hardlink = true; auto [dst_part, part_lock] = cloneAndLoadDataPartOnOtherDisk( src_part, TMP_PREFIX, From fc3ebe007b3b5dc905ecbd63ed402547a1cde3a5 Mon Sep 17 00:00:00 2001 From: unashi Date: Tue, 20 Feb 2024 20:54:32 +0800 Subject: [PATCH 007/117] [fix] rm whitespaces --- src/Storages/MergeTree/MergeTreeData.cpp | 4 ---- 1 file changed, 4 deletions(-) diff --git a/src/Storages/MergeTree/MergeTreeData.cpp b/src/Storages/MergeTree/MergeTreeData.cpp index be1346e0ea2..18bb0966bfc 100644 --- a/src/Storages/MergeTree/MergeTreeData.cpp +++ b/src/Storages/MergeTree/MergeTreeData.cpp @@ -7171,9 +7171,7 @@ std::pair MergeTreeData::cloneAn } } if (!copy_successful) - { LOG_FATAL(&Poco::Logger::get("MergeTreeData"), "Hard link fail, clone fail"); - } } @@ -7304,9 +7302,7 @@ std::pair MergeTreeData::cloneAn } } if (!copy_successful) - { LOG_FATAL( &Poco::Logger::get("MergeTreeData"), "Hard link fail, clone fail."); - } if (params.metadata_version_to_write.has_value()) { chassert(!params.keep_metadata_version); From f829a97d9130de5609e07e237b9486847422bc8c Mon Sep 17 00:00:00 2001 From: unashi Date: Tue, 20 Feb 2024 21:08:24 +0800 Subject: [PATCH 008/117] [fix] rm whitespaces --- src/Storages/MergeTree/MergeTreeData.cpp | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/src/Storages/MergeTree/MergeTreeData.cpp b/src/Storages/MergeTree/MergeTreeData.cpp index 18bb0966bfc..849ceb1b66d 100644 --- a/src/Storages/MergeTree/MergeTreeData.cpp +++ b/src/Storages/MergeTree/MergeTreeData.cpp @@ -7302,7 +7302,7 @@ std::pair MergeTreeData::cloneAn } } if (!copy_successful) - LOG_FATAL( &Poco::Logger::get("MergeTreeData"), "Hard link fail, clone fail."); + LOG_FATAL(&Poco::Logger::get("MergeTreeData"), "Hard link fail, clone fail."); if (params.metadata_version_to_write.has_value()) { chassert(!params.keep_metadata_version); From 28282eee91add78e5b18202bd38566d1d3797083 Mon Sep 17 00:00:00 2001 From: unashi Date: Tue, 20 Feb 2024 21:37:09 +0800 Subject: [PATCH 009/117] [fix] Add description in partition.md --- docs/en/sql-reference/statements/alter/partition.md | 2 ++ 1 file changed, 2 insertions(+) diff --git a/docs/en/sql-reference/statements/alter/partition.md b/docs/en/sql-reference/statements/alter/partition.md index 114b8d5ffe3..277e174bb05 100644 --- a/docs/en/sql-reference/statements/alter/partition.md +++ b/docs/en/sql-reference/statements/alter/partition.md @@ -116,6 +116,8 @@ For the query to run successfully, the following conditions must be met: - Both tables must have the same indices and projections. - Both tables must have the same storage policy. +If both tables have the same storage policy, use hardlink to attach partition. Otherwise, use copying the data to attach partition. + ## REPLACE PARTITION ``` sql From 1731a5a8afba5a48ce01cea20e0cdc1f91316841 Mon Sep 17 00:00:00 2001 From: unashi Date: Wed, 21 Feb 2024 10:55:32 +0800 Subject: [PATCH 010/117] [improve]change the integration test test_multiple_disks::test_move_across_policies_not_work to test_move_across_policies_work_for_attach_not_work_for_move --- tests/integration/test_multiple_disks/test.py | 36 +++++++------------ 1 file changed, 13 insertions(+), 23 deletions(-) diff --git a/tests/integration/test_multiple_disks/test.py b/tests/integration/test_multiple_disks/test.py index fdd81284b2a..9584ace7f45 100644 --- a/tests/integration/test_multiple_disks/test.py +++ b/tests/integration/test_multiple_disks/test.py @@ -5,6 +5,7 @@ import string import threading import time from multiprocessing.dummy import Pool +from helpers.test_tools import assert_eq_with_retry import pytest from helpers.client import QueryRuntimeException @@ -1745,9 +1746,9 @@ def test_move_while_merge(start_cluster): node1.query(f"DROP TABLE IF EXISTS {name} SYNC") -def test_move_across_policies_does_not_work(start_cluster): +def test_move_across_policies_work_for_attach_not_work_for_move(start_cluster): try: - name = "test_move_across_policies_does_not_work" + name = "test_move_across_policies_work_for_attach_not_work_for_move" node1.query( """ @@ -1783,25 +1784,18 @@ def test_move_across_policies_does_not_work(start_cluster): except QueryRuntimeException: """All parts of partition 'all' are already on disk 'jbod2'.""" - with pytest.raises( - QueryRuntimeException, - match=".*because disk does not belong to storage policy.*", - ): - node1.query( - """ALTER TABLE {name}2 ATTACH PARTITION tuple() FROM {name}""".format( - name=name - ) + node1.query( + """ALTER TABLE {name}2 ATTACH PARTITION tuple() FROM {name}""".format( + name=name ) - - with pytest.raises( - QueryRuntimeException, - match=".*because disk does not belong to storage policy.*", - ): + ) + assert_eq_with_retry( + node1, + """SELECT * FROM {name}2""".format(name=name), node1.query( - """ALTER TABLE {name}2 REPLACE PARTITION tuple() FROM {name}""".format( - name=name - ) - ) + """SELECT * FROM {name}""".format(name=name), + ), + ) with pytest.raises( QueryRuntimeException, @@ -1813,10 +1807,6 @@ def test_move_across_policies_does_not_work(start_cluster): ) ) - assert node1.query( - """SELECT * FROM {name}""".format(name=name) - ).splitlines() == ["1"] - finally: node1.query(f"DROP TABLE IF EXISTS {name} SYNC") node1.query(f"DROP TABLE IF EXISTS {name}2 SYNC") From 835b47519a7c575d70542e5a37c97dbf5a2b25f9 Mon Sep 17 00:00:00 2001 From: Nikita Taranov Date: Wed, 14 Feb 2024 00:44:38 +0100 Subject: [PATCH 011/117] impl --- src/Backups/BackupIO_S3.cpp | 1 + src/Coordination/KeeperSnapshotManagerS3.cpp | 1 + src/Disks/ObjectStorages/S3/diskSettings.cpp | 1 + src/IO/S3/Client.cpp | 19 +++++++++--- src/IO/S3/Client.h | 6 ++++ src/IO/S3/Requests.h | 32 ++++++++++++++++++-- src/IO/S3/URI.cpp | 18 +++++------ src/IO/S3/tests/gtest_aws_s3_client.cpp | 25 ++++++++++++++- src/IO/WriteBufferFromS3.cpp | 15 ++++++++- src/IO/WriteBufferFromS3.h | 1 + src/IO/tests/gtest_s3_uri.cpp | 8 +++++ src/IO/tests/gtest_writebuffer_s3.cpp | 21 +++++++------ src/Storages/StorageS3.cpp | 1 + 13 files changed, 119 insertions(+), 30 deletions(-) diff --git a/src/Backups/BackupIO_S3.cpp b/src/Backups/BackupIO_S3.cpp index 9359602a651..2063af2061c 100644 --- a/src/Backups/BackupIO_S3.cpp +++ b/src/Backups/BackupIO_S3.cpp @@ -73,6 +73,7 @@ namespace .use_virtual_addressing = s3_uri.is_virtual_hosted_style, .disable_checksum = local_settings.s3_disable_checksum, .gcs_issue_compose_request = context->getConfigRef().getBool("s3.gcs_issue_compose_request", false), + .is_s3express_bucket = S3::isS3ExpressEndpoint(s3_uri.endpoint), }; return S3::ClientFactory::instance().create( diff --git a/src/Coordination/KeeperSnapshotManagerS3.cpp b/src/Coordination/KeeperSnapshotManagerS3.cpp index 0337a564660..9779a041095 100644 --- a/src/Coordination/KeeperSnapshotManagerS3.cpp +++ b/src/Coordination/KeeperSnapshotManagerS3.cpp @@ -103,6 +103,7 @@ void KeeperSnapshotManagerS3::updateS3Configuration(const Poco::Util::AbstractCo .use_virtual_addressing = new_uri.is_virtual_hosted_style, .disable_checksum = false, .gcs_issue_compose_request = false, + .is_s3express_bucket = S3::isS3ExpressEndpoint(new_uri.endpoint), }; auto client = S3::ClientFactory::instance().create( diff --git a/src/Disks/ObjectStorages/S3/diskSettings.cpp b/src/Disks/ObjectStorages/S3/diskSettings.cpp index 4fd4b17aabe..b8688cd3de6 100644 --- a/src/Disks/ObjectStorages/S3/diskSettings.cpp +++ b/src/Disks/ObjectStorages/S3/diskSettings.cpp @@ -97,6 +97,7 @@ std::unique_ptr getClient( .use_virtual_addressing = uri.is_virtual_hosted_style, .disable_checksum = local_settings.s3_disable_checksum, .gcs_issue_compose_request = config.getBool("s3.gcs_issue_compose_request", false), + .is_s3express_bucket = S3::isS3ExpressEndpoint(endpoint), }; return S3::ClientFactory::instance().create( diff --git a/src/IO/S3/Client.cpp b/src/IO/S3/Client.cpp index 1b6b245b89a..a75d41df3d1 100644 --- a/src/IO/S3/Client.cpp +++ b/src/IO/S3/Client.cpp @@ -304,6 +304,9 @@ Model::HeadObjectOutcome Client::HeadObject(HeadObjectRequest & request) const request.setApiMode(api_mode); + if (isS3ExpressBucket()) + request.setIsS3ExpressBucket(); + addAdditionalAMZHeadersToCanonicalHeadersList(request, client_configuration.extra_headers); if (auto region = getRegionForBucket(bucket); !region.empty()) @@ -530,7 +533,11 @@ Client::doRequest(RequestType & request, RequestFn request_fn) const addAdditionalAMZHeadersToCanonicalHeadersList(request, client_configuration.extra_headers); const auto & bucket = request.GetBucket(); request.setApiMode(api_mode); - if (client_settings.disable_checksum) + + /// We have to use checksums for S3Express buckets, so the order of checks should be the following + if (client_settings.is_s3express_bucket) + request.setIsS3ExpressBucket(); + else if (client_settings.disable_checksum) request.disableChecksum(); if (auto region = getRegionForBucket(bucket); !region.empty()) @@ -915,9 +922,9 @@ std::unique_ptr ClientFactory::create( // NOLINT std::move(sse_kms_config), credentials_provider, client_configuration, // Client configuration. - Aws::Client::AWSAuthV4Signer::PayloadSigningPolicy::Never, - client_settings - ); + client_settings.is_s3express_bucket ? Aws::Client::AWSAuthV4Signer::PayloadSigningPolicy::RequestDependent + : Aws::Client::AWSAuthV4Signer::PayloadSigningPolicy::Never, + client_settings); } PocoHTTPClientConfiguration ClientFactory::createClientConfiguration( // NOLINT @@ -956,6 +963,10 @@ PocoHTTPClientConfiguration ClientFactory::createClientConfiguration( // NOLINT return config; } +bool isS3ExpressEndpoint(const std::string & endpoint) +{ + return endpoint.contains("s3express"); +} } } diff --git a/src/IO/S3/Client.h b/src/IO/S3/Client.h index 8da21bd2c2c..c7bc727bf32 100644 --- a/src/IO/S3/Client.h +++ b/src/IO/S3/Client.h @@ -92,6 +92,8 @@ private: std::unordered_map> client_caches; }; +bool isS3ExpressEndpoint(const std::string & endpoint); + struct ClientSettings { bool use_virtual_addressing; @@ -107,6 +109,7 @@ struct ClientSettings /// Ability to enable it preserved since likely it is required for old /// files. bool gcs_issue_compose_request; + bool is_s3express_bucket; }; /// Client that improves the client from the AWS SDK @@ -208,6 +211,9 @@ public: const std::shared_ptr& httpRequest) const override; bool supportsMultiPartCopy() const; + + bool isS3ExpressBucket() const { return client_settings.is_s3express_bucket; } + private: friend struct ::MockS3::Client; diff --git a/src/IO/S3/Requests.h b/src/IO/S3/Requests.h index bfb94a5a67e..6f82a0f39d3 100644 --- a/src/IO/S3/Requests.h +++ b/src/IO/S3/Requests.h @@ -21,12 +21,32 @@ #include #include #include +#include +#include + +#include namespace DB::S3 { namespace Model = Aws::S3::Model; +/// Used only for S3Express +namespace RequestChecksum +{ +inline void setPartChecksum(Model::CompletedPart & part, const std::string & checksum) +{ + part.SetChecksumCRC32(checksum); +} + +template +inline void setChecksumAlgorithm(R & request) +{ + if constexpr (requires { request.SetChecksumAlgorithm(Model::ChecksumAlgorithm::CRC32); }) + request.SetChecksumAlgorithm(Model::ChecksumAlgorithm::CRC32); +} +}; + template class ExtendedRequest : public BaseRequest { @@ -49,11 +69,13 @@ public: Aws::String GetChecksumAlgorithmName() const override { + chassert(!is_s3express_bucket || checksum); + /// Return empty string is enough to disable checksums (see /// AWSClient::AddChecksumToRequest [1] for more details). /// /// [1]: https://github.com/aws/aws-sdk-cpp/blob/b0ee1c0d336dbb371c34358b68fba6c56aae2c92/src/aws-cpp-sdk-core/source/client/AWSClient.cpp#L783-L839 - if (!checksum) + if (!is_s3express_bucket && !checksum) return ""; return BaseRequest::GetChecksumAlgorithmName(); } @@ -84,9 +106,12 @@ public: } /// Disable checksum to avoid extra read of the input stream - void disableChecksum() const + void disableChecksum() const { checksum = false; } + + void setIsS3ExpressBucket() { - checksum = false; + is_s3express_bucket = true; + RequestChecksum::setChecksumAlgorithm(*this); } protected: @@ -94,6 +119,7 @@ protected: mutable std::optional uri_override; mutable ApiMode api_mode{ApiMode::AWS}; mutable bool checksum = true; + bool is_s3express_bucket = false; }; class CopyObjectRequest : public ExtendedRequest diff --git a/src/IO/S3/URI.cpp b/src/IO/S3/URI.cpp index 23f59420bfe..062d3b80850 100644 --- a/src/IO/S3/URI.cpp +++ b/src/IO/S3/URI.cpp @@ -35,7 +35,7 @@ URI::URI(const std::string & uri_) /// Case when bucket name represented in domain name of S3 URL. /// E.g. (https://bucket-name.s3.Region.amazonaws.com/key) /// https://docs.aws.amazon.com/AmazonS3/latest/dev/VirtualHosting.html#virtual-hosted-style-access - static const RE2 virtual_hosted_style_pattern(R"((.+)\.(s3|cos|obs|oss|eos)([.\-][a-z0-9\-.:]+))"); + static const RE2 virtual_hosted_style_pattern(R"((.+)\.(s3express[\-a-z0-9]+|s3|cos|obs|oss|eos)([.\-][a-z0-9\-.:]+))"); /// Case when bucket name and key represented in path of S3 URL. /// E.g. (https://s3.Region.amazonaws.com/bucket-name/key) @@ -43,6 +43,7 @@ URI::URI(const std::string & uri_) static const RE2 path_style_pattern("^/([^/]*)/(.*)"); static constexpr auto S3 = "S3"; + static constexpr auto S3EXPRESS = "S3EXPRESS"; static constexpr auto COSN = "COSN"; static constexpr auto COS = "COS"; static constexpr auto OBS = "OBS"; @@ -115,21 +116,16 @@ URI::URI(const std::string & uri_) } boost::to_upper(name); - if (name != S3 && name != COS && name != OBS && name != OSS && name != EOS) + /// For S3Express it will look like s3express-eun1-az1, i.e. contain region and AZ info + if (name != S3 && !name.starts_with(S3EXPRESS) && name != COS && name != OBS && name != OSS && name != EOS) throw Exception(ErrorCodes::BAD_ARGUMENTS, "Object storage system name is unrecognized in virtual hosted style S3 URI: {}", quoteString(name)); - if (name == S3) - storage_name = name; - else if (name == OBS) - storage_name = OBS; - else if (name == OSS) - storage_name = OSS; - else if (name == EOS) - storage_name = EOS; - else + if (name == COS || name == COSN) storage_name = COSN; + else + storage_name = name; } else if (re2::RE2::PartialMatch(uri.getPath(), path_style_pattern, &bucket, &key)) { diff --git a/src/IO/S3/tests/gtest_aws_s3_client.cpp b/src/IO/S3/tests/gtest_aws_s3_client.cpp index 33917314bca..8edbe12a22f 100644 --- a/src/IO/S3/tests/gtest_aws_s3_client.cpp +++ b/src/IO/S3/tests/gtest_aws_s3_client.cpp @@ -110,7 +110,8 @@ void testServerSideEncryption( bool disable_checksum, String server_side_encryption_customer_key_base64, DB::S3::ServerSideEncryptionKMSConfig sse_kms_config, - String expected_headers) + String expected_headers, + bool is_s3express_bucket = false) { TestPocoHTTPServer http; @@ -144,6 +145,7 @@ void testServerSideEncryption( .use_virtual_addressing = uri.is_virtual_hosted_style, .disable_checksum = disable_checksum, .gcs_issue_compose_request = false, + .is_s3express_bucket = is_s3express_bucket, }; std::shared_ptr client = DB::S3::ClientFactory::instance().create( @@ -295,4 +297,25 @@ TEST(IOTestAwsS3Client, AppendExtraSSEKMSHeadersWrite) "x-amz-server-side-encryption-context: arn:aws:s3:::bucket_ARN\n"); } +TEST(IOTestAwsS3Client, ChecksumHeaderIsPresentForS3Express) +{ + /// See https://github.com/ClickHouse/ClickHouse/pull/19748 + testServerSideEncryption( + doWriteRequest, + /* disable_checksum= */ true, + "", + {}, + "authorization: ... SignedHeaders=" + "amz-sdk-invocation-id;" + "amz-sdk-request;" + "content-length;" + "content-type;" + "host;" + "x-amz-checksum-crc32;" + "x-amz-content-sha256;" + "x-amz-date;" + "x-amz-sdk-checksum-algorithm, ...\n", + /*is_s3express_bucket=*/true); +} + #endif diff --git a/src/IO/WriteBufferFromS3.cpp b/src/IO/WriteBufferFromS3.cpp index 5bb01050591..a162992278f 100644 --- a/src/IO/WriteBufferFromS3.cpp +++ b/src/IO/WriteBufferFromS3.cpp @@ -18,7 +18,9 @@ #include #include +#include #include +#include #include @@ -456,6 +458,14 @@ S3::UploadPartRequest WriteBufferFromS3::getUploadRequest(size_t part_number, Pa /// If we don't do it, AWS SDK can mistakenly set it to application/xml, see https://github.com/aws/aws-sdk-cpp/issues/1840 req.SetContentType("binary/octet-stream"); + /// Checksums need to be provided on CompleteMultipartUpload requests, so we calculate then manually and store in multipart_checksums + if (client_ptr->isS3ExpressBucket()) + { + chassert(req.GetChecksumAlgorithm() == Aws::S3::Model::ChecksumAlgorithm::CRC32); + req.SetChecksumCRC32(Aws::Utils::HashingUtils::Base64Encode(Aws::Utils::HashingUtils::CalculateCRC32(*(req.GetBody())))); + multipart_checksums.push_back(req.GetChecksumCRC32()); + } + return req; } @@ -575,7 +585,10 @@ void WriteBufferFromS3::completeMultipartUpload() for (size_t i = 0; i < multipart_tags.size(); ++i) { Aws::S3::Model::CompletedPart part; - multipart_upload.AddParts(part.WithETag(multipart_tags[i]).WithPartNumber(static_cast(i + 1))); + part.WithETag(multipart_tags[i]).WithPartNumber(static_cast(i + 1)); + if (!multipart_checksums.empty()) + S3::RequestChecksum::setPartChecksum(part, multipart_checksums.at(i)); + multipart_upload.AddParts(part); } req.SetMultipartUpload(multipart_upload); diff --git a/src/IO/WriteBufferFromS3.h b/src/IO/WriteBufferFromS3.h index 5dc269990a1..148cd27f854 100644 --- a/src/IO/WriteBufferFromS3.h +++ b/src/IO/WriteBufferFromS3.h @@ -100,6 +100,7 @@ private: /// We initiate upload, then upload each part and get ETag as a response, and then finalizeImpl() upload with listing all our parts. String multipart_upload_id; std::deque multipart_tags; + std::deque multipart_checksums; bool multipart_upload_finished = false; /// Track that prefinalize() is called only once diff --git a/src/IO/tests/gtest_s3_uri.cpp b/src/IO/tests/gtest_s3_uri.cpp index c088e41f1e8..5bf0dfb962d 100644 --- a/src/IO/tests/gtest_s3_uri.cpp +++ b/src/IO/tests/gtest_s3_uri.cpp @@ -162,6 +162,14 @@ TEST(S3UriTest, validPatterns) ASSERT_EQ("", uri.version_id); ASSERT_EQ(false, uri.is_virtual_hosted_style); } + { + S3::URI uri("https://test-perf-bucket--eun1-az1--x-s3.s3express-eun1-az1.eu-north-1.amazonaws.com/test.csv"); + ASSERT_EQ("https://s3express-eun1-az1.eu-north-1.amazonaws.com", uri.endpoint); + ASSERT_EQ("test-perf-bucket--eun1-az1--x-s3", uri.bucket); + ASSERT_EQ("test.csv", uri.key); + ASSERT_EQ("", uri.version_id); + ASSERT_EQ(true, uri.is_virtual_hosted_style); + } } TEST_P(S3UriTest, invalidPatterns) diff --git a/src/IO/tests/gtest_writebuffer_s3.cpp b/src/IO/tests/gtest_writebuffer_s3.cpp index ae00bb2e9e2..d9cb486c09e 100644 --- a/src/IO/tests/gtest_writebuffer_s3.cpp +++ b/src/IO/tests/gtest_writebuffer_s3.cpp @@ -205,16 +205,17 @@ struct Client : DB::S3::Client { explicit Client(std::shared_ptr mock_s3_store) : DB::S3::Client( - 100, - DB::S3::ServerSideEncryptionKMSConfig(), - std::make_shared("", ""), - GetClientConfiguration(), - Aws::Client::AWSAuthV4Signer::PayloadSigningPolicy::Never, - DB::S3::ClientSettings{ - .use_virtual_addressing = true, - .disable_checksum= false, - .gcs_issue_compose_request = false, - }) + 100, + DB::S3::ServerSideEncryptionKMSConfig(), + std::make_shared("", ""), + GetClientConfiguration(), + Aws::Client::AWSAuthV4Signer::PayloadSigningPolicy::Never, + DB::S3::ClientSettings{ + .use_virtual_addressing = true, + .disable_checksum = false, + .gcs_issue_compose_request = false, + .is_s3express_bucket = false, + }) , store(mock_s3_store) {} diff --git a/src/Storages/StorageS3.cpp b/src/Storages/StorageS3.cpp index 2d8ef3df1c8..044a1ca5362 100644 --- a/src/Storages/StorageS3.cpp +++ b/src/Storages/StorageS3.cpp @@ -1427,6 +1427,7 @@ void StorageS3::Configuration::connect(const ContextPtr & context) .use_virtual_addressing = url.is_virtual_hosted_style, .disable_checksum = local_settings.s3_disable_checksum, .gcs_issue_compose_request = context->getConfigRef().getBool("s3.gcs_issue_compose_request", false), + .is_s3express_bucket = S3::isS3ExpressEndpoint(url.endpoint), }; auto credentials = Aws::Auth::AWSCredentials(auth_settings.access_key_id, auth_settings.secret_access_key, auth_settings.session_token); From 974ba7364f193838f735a9233c6dec4298172542 Mon Sep 17 00:00:00 2001 From: Nikita Taranov Date: Thu, 29 Feb 2024 00:55:17 +0100 Subject: [PATCH 012/117] better --- src/Disks/ObjectStorages/S3/diskSettings.cpp | 16 +++++++++++++--- src/IO/S3/Client.cpp | 2 ++ src/IO/S3/URI.cpp | 2 +- src/IO/WriteBufferFromS3.h | 2 +- src/Storages/StorageS3.cpp | 4 ++++ 5 files changed, 21 insertions(+), 5 deletions(-) diff --git a/src/Disks/ObjectStorages/S3/diskSettings.cpp b/src/Disks/ObjectStorages/S3/diskSettings.cpp index b8688cd3de6..10172805f06 100644 --- a/src/Disks/ObjectStorages/S3/diskSettings.cpp +++ b/src/Disks/ObjectStorages/S3/diskSettings.cpp @@ -1,5 +1,6 @@ #include -#include "IO/S3/Client.h" +#include +#include #if USE_AWS_S3 @@ -10,7 +11,7 @@ #include #include #include -#include "Disks/DiskFactory.h" +#include #include #include @@ -25,6 +26,11 @@ namespace DB { +namespace ErrorCodes +{ +extern const int NO_ELEMENTS_IN_CONFIG; +} + std::unique_ptr getSettings(const Poco::Util::AbstractConfiguration & config, const String & config_prefix, ContextPtr context) { const Settings & settings = context->getSettingsRef(); @@ -47,11 +53,15 @@ std::unique_ptr getClient( const Settings & global_settings = context->getGlobalContext()->getSettingsRef(); const Settings & local_settings = context->getSettingsRef(); - String endpoint = context->getMacros()->expand(config.getString(config_prefix + ".endpoint")); + const String endpoint = context->getMacros()->expand(config.getString(config_prefix + ".endpoint")); S3::URI uri(endpoint); if (!uri.key.ends_with('/')) uri.key.push_back('/'); + if (S3::isS3ExpressEndpoint(endpoint) && !config.has(config_prefix + ".region")) + throw Exception( + ErrorCodes::NO_ELEMENTS_IN_CONFIG, "Region should be explicitly specified for directory buckets ({})", config_prefix); + S3::PocoHTTPClientConfiguration client_configuration = S3::ClientFactory::instance().createClientConfiguration( config.getString(config_prefix + ".region", ""), context->getRemoteHostFilter(), diff --git a/src/IO/S3/Client.cpp b/src/IO/S3/Client.cpp index a75d41df3d1..4f93aba2f84 100644 --- a/src/IO/S3/Client.cpp +++ b/src/IO/S3/Client.cpp @@ -1,4 +1,5 @@ #include +#include #if USE_AWS_S3 @@ -965,6 +966,7 @@ PocoHTTPClientConfiguration ClientFactory::createClientConfiguration( // NOLINT bool isS3ExpressEndpoint(const std::string & endpoint) { + /// On one hand this check isn't 100% reliable, on the other - all it will change is whether we attach checksums to the requests. return endpoint.contains("s3express"); } } diff --git a/src/IO/S3/URI.cpp b/src/IO/S3/URI.cpp index 062d3b80850..027cb624ed5 100644 --- a/src/IO/S3/URI.cpp +++ b/src/IO/S3/URI.cpp @@ -122,7 +122,7 @@ URI::URI(const std::string & uri_) "Object storage system name is unrecognized in virtual hosted style S3 URI: {}", quoteString(name)); - if (name == COS || name == COSN) + if (name == COS) storage_name = COSN; else storage_name = name; diff --git a/src/IO/WriteBufferFromS3.h b/src/IO/WriteBufferFromS3.h index 148cd27f854..59f4e19e15b 100644 --- a/src/IO/WriteBufferFromS3.h +++ b/src/IO/WriteBufferFromS3.h @@ -100,7 +100,7 @@ private: /// We initiate upload, then upload each part and get ETag as a response, and then finalizeImpl() upload with listing all our parts. String multipart_upload_id; std::deque multipart_tags; - std::deque multipart_checksums; + std::deque multipart_checksums; // if enabled bool multipart_upload_finished = false; /// Track that prefinalize() is called only once diff --git a/src/Storages/StorageS3.cpp b/src/Storages/StorageS3.cpp index 044a1ca5362..f96ff8b7eb6 100644 --- a/src/Storages/StorageS3.cpp +++ b/src/Storages/StorageS3.cpp @@ -133,6 +133,7 @@ namespace ErrorCodes extern const int NOT_IMPLEMENTED; extern const int CANNOT_COMPILE_REGEXP; extern const int FILE_DOESNT_EXIST; + extern const int NO_ELEMENTS_IN_CONFIG; } @@ -1403,6 +1404,9 @@ void StorageS3::Configuration::connect(const ContextPtr & context) const Settings & global_settings = context->getGlobalContext()->getSettingsRef(); const Settings & local_settings = context->getSettingsRef(); + if (S3::isS3ExpressEndpoint(url.endpoint) && auth_settings.region.empty()) + throw Exception(ErrorCodes::NO_ELEMENTS_IN_CONFIG, "Region should be explicitly specified for directory buckets"); + S3::PocoHTTPClientConfiguration client_configuration = S3::ClientFactory::instance().createClientConfiguration( auth_settings.region, context->getRemoteHostFilter(), From 37917a3ed34df22756562a04a90d3c985ca23bd8 Mon Sep 17 00:00:00 2001 From: Nikita Taranov Date: Thu, 29 Feb 2024 01:42:32 +0100 Subject: [PATCH 013/117] better --- src/IO/S3/Requests.h | 12 ++++++++++++ src/IO/WriteBufferFromS3.cpp | 10 +++------- 2 files changed, 15 insertions(+), 7 deletions(-) diff --git a/src/IO/S3/Requests.h b/src/IO/S3/Requests.h index 6f82a0f39d3..196f074c9df 100644 --- a/src/IO/S3/Requests.h +++ b/src/IO/S3/Requests.h @@ -23,6 +23,7 @@ #include #include #include +#include #include @@ -39,6 +40,17 @@ inline void setPartChecksum(Model::CompletedPart & part, const std::string & che part.SetChecksumCRC32(checksum); } +inline void setRequestChecksum(Model::UploadPartRequest & req, const std::string & checksum) +{ + req.SetChecksumCRC32(checksum); +} + +inline std::string calculateChecksum(Model::UploadPartRequest & req) +{ + chassert(req.GetChecksumAlgorithm() == Aws::S3::Model::ChecksumAlgorithm::CRC32); + return Aws::Utils::HashingUtils::Base64Encode(Aws::Utils::HashingUtils::CalculateCRC32(*(req.GetBody()))); +} + template inline void setChecksumAlgorithm(R & request) { diff --git a/src/IO/WriteBufferFromS3.cpp b/src/IO/WriteBufferFromS3.cpp index a162992278f..80ca96b0382 100644 --- a/src/IO/WriteBufferFromS3.cpp +++ b/src/IO/WriteBufferFromS3.cpp @@ -18,10 +18,6 @@ #include #include -#include -#include -#include - #include @@ -461,9 +457,9 @@ S3::UploadPartRequest WriteBufferFromS3::getUploadRequest(size_t part_number, Pa /// Checksums need to be provided on CompleteMultipartUpload requests, so we calculate then manually and store in multipart_checksums if (client_ptr->isS3ExpressBucket()) { - chassert(req.GetChecksumAlgorithm() == Aws::S3::Model::ChecksumAlgorithm::CRC32); - req.SetChecksumCRC32(Aws::Utils::HashingUtils::Base64Encode(Aws::Utils::HashingUtils::CalculateCRC32(*(req.GetBody())))); - multipart_checksums.push_back(req.GetChecksumCRC32()); + auto checksum = S3::RequestChecksum::calculateChecksum(req); + S3::RequestChecksum::setRequestChecksum(req, checksum); + multipart_checksums.push_back(std::move(checksum)); } return req; From a7aeb4c00f106d396364bf2a21697e329d3d284d Mon Sep 17 00:00:00 2001 From: Peter Date: Fri, 1 Mar 2024 23:44:58 +0800 Subject: [PATCH 014/117] Add --now option to enable and start the service --- packages/clickhouse-server.postinstall | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/packages/clickhouse-server.postinstall b/packages/clickhouse-server.postinstall index d3b49db758f..41d4405a790 100644 --- a/packages/clickhouse-server.postinstall +++ b/packages/clickhouse-server.postinstall @@ -36,7 +36,7 @@ if [ "$1" = configure ] || [ -n "$not_deb_os" ]; then fi /bin/systemctl daemon-reload - /bin/systemctl enable clickhouse-server + /bin/systemctl enable --now clickhouse-server else # If you downgrading to version older than 1.1.54336 run: systemctl disable clickhouse-server if [ -x "/etc/init.d/clickhouse-server" ]; then From aa43885ac81924a73e9a151a550e7c1af43d23e2 Mon Sep 17 00:00:00 2001 From: unashi Date: Tue, 5 Mar 2024 10:57:25 +0800 Subject: [PATCH 015/117] [improve] add check the remaining disk size before copying --- src/Storages/MergeTree/MergeTreeData.cpp | 11 +++++++++-- 1 file changed, 9 insertions(+), 2 deletions(-) diff --git a/src/Storages/MergeTree/MergeTreeData.cpp b/src/Storages/MergeTree/MergeTreeData.cpp index 849ceb1b66d..d8680958c21 100644 --- a/src/Storages/MergeTree/MergeTreeData.cpp +++ b/src/Storages/MergeTree/MergeTreeData.cpp @@ -7160,14 +7160,18 @@ std::pair MergeTreeData::cloneAn { try { + auto reservation_space = src_part_storage->reserve(src_part->getBytesOnDisk()); + if (!reservation_space) { + throw Exception(ErrorCodes::NOT_ENOUGH_SPACE, "Not enough space on disk."); + } dst_part_storage = src_part_storage->clonePart(this->getRelativeDataPath(), tmp_dst_part_name, disk, read_settings, write_settings, {}, {}); copy_successful = true; break; } - catch (...) + catch (Exception & e) { - LOG_TRACE(&Poco::Logger::get("MergeTreeData"), "Clone part on disk {} fail", disk->getName()); + LOG_TRACE(&Poco::Logger::get("MergeTreeData"), "Clone part on disk {} fail: {}", disk->getName(), e.what()); } } if (!copy_successful) @@ -7291,6 +7295,9 @@ std::pair MergeTreeData::cloneAn { try { + auto reservation_space = src_part_storage->reserve(src_part->getBytesOnDisk()); + if (!reservation_space) + throw Exception(ErrorCodes::NOT_ENOUGH_SPACE, "Not enough space on disk."); dst_part_storage = src_part_storage->clonePart(this->getRelativeDataPath(), tmp_dst_part_name, disk, read_settings, write_settings, {}, {}); copy_successful = true; From 3dbb0a12fb433b29107d449099efbc99f5d71f34 Mon Sep 17 00:00:00 2001 From: unashi Date: Wed, 6 Mar 2024 16:15:37 +0800 Subject: [PATCH 016/117] [fix] style --- src/Storages/MergeTree/MergeTreeData.cpp | 7 +++---- tests/integration/helpers/cluster.py | 2 +- 2 files changed, 4 insertions(+), 5 deletions(-) diff --git a/src/Storages/MergeTree/MergeTreeData.cpp b/src/Storages/MergeTree/MergeTreeData.cpp index d8680958c21..c76ffeee874 100644 --- a/src/Storages/MergeTree/MergeTreeData.cpp +++ b/src/Storages/MergeTree/MergeTreeData.cpp @@ -7161,11 +7161,10 @@ std::pair MergeTreeData::cloneAn try { auto reservation_space = src_part_storage->reserve(src_part->getBytesOnDisk()); - if (!reservation_space) { + if (!reservation_space) throw Exception(ErrorCodes::NOT_ENOUGH_SPACE, "Not enough space on disk."); - } - dst_part_storage - = src_part_storage->clonePart(this->getRelativeDataPath(), tmp_dst_part_name, disk, read_settings, write_settings, {}, {}); + dst_part_storage = src_part_storage->clonePart( + this->getRelativeDataPath(), tmp_dst_part_name, disk, read_settings, write_settings, {}, {}); copy_successful = true; break; } diff --git a/tests/integration/helpers/cluster.py b/tests/integration/helpers/cluster.py index 1d96563251b..767ba5b6660 100644 --- a/tests/integration/helpers/cluster.py +++ b/tests/integration/helpers/cluster.py @@ -3216,7 +3216,7 @@ services: - timeout:1 - inet6 - rotate - {networks} + {123} {app_net} {ipv4_address} {ipv6_address} From b4dba828a4dcde93944e05b512818827fd3e5a85 Mon Sep 17 00:00:00 2001 From: unashi Date: Wed, 6 Mar 2024 16:19:07 +0800 Subject: [PATCH 017/117] [fix] --- tests/integration/helpers/cluster.py | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/tests/integration/helpers/cluster.py b/tests/integration/helpers/cluster.py index 767ba5b6660..1d96563251b 100644 --- a/tests/integration/helpers/cluster.py +++ b/tests/integration/helpers/cluster.py @@ -3216,7 +3216,7 @@ services: - timeout:1 - inet6 - rotate - {123} + {networks} {app_net} {ipv4_address} {ipv6_address} From d9b5f9a086d8bc1af5387edee77d0e3fdbf4d9b0 Mon Sep 17 00:00:00 2001 From: Nikolay Monkov Date: Wed, 6 Mar 2024 21:37:56 +0500 Subject: [PATCH 018/117] version has been added to docker labels --- tests/ci/docker_server.py | 5 +++-- 1 file changed, 3 insertions(+), 2 deletions(-) diff --git a/tests/ci/docker_server.py b/tests/ci/docker_server.py index 38d0ea6d86b..803dbfcd92a 100644 --- a/tests/ci/docker_server.py +++ b/tests/ci/docker_server.py @@ -216,11 +216,12 @@ def gen_tags(version: ClickHouseVersion, release_type: str) -> List[str]: return tags -def buildx_args(urls: Dict[str, str], arch: str, direct_urls: List[str]) -> List[str]: +def buildx_args(urls: Dict[str, str], arch: str, direct_urls: List[str], version: str) -> List[str]: args = [ f"--platform=linux/{arch}", f"--label=build-url={GITHUB_RUN_URL}", f"--label=com.clickhouse.build.githash={git.sha}", + f"--label=com.clickhouse.build.version={version}", ] if direct_urls: args.append(f"--build-arg=DIRECT_DOWNLOAD_URLS='{' '.join(direct_urls)}'") @@ -267,7 +268,7 @@ def build_and_push_image( urls = [url for url in direct_urls[arch] if ".deb" in url] else: urls = [url for url in direct_urls[arch] if ".tgz" in url] - cmd_args.extend(buildx_args(repo_urls, arch, direct_urls=urls)) + cmd_args.extend(buildx_args(repo_urls, arch, direct_urls=urls, version=version.describe)) if not push: cmd_args.append(f"--tag={image.repo}:{arch_tag}") cmd_args.extend( From fb17749b50ce1024ef8c0b6f7bb8b7a58321894c Mon Sep 17 00:00:00 2001 From: Nikolay Monkov Date: Thu, 7 Mar 2024 09:45:24 +0500 Subject: [PATCH 019/117] file has been reformatted to pass Style check --- tests/ci/docker_server.py | 8 ++++++-- 1 file changed, 6 insertions(+), 2 deletions(-) diff --git a/tests/ci/docker_server.py b/tests/ci/docker_server.py index 803dbfcd92a..35c86d8eadd 100644 --- a/tests/ci/docker_server.py +++ b/tests/ci/docker_server.py @@ -216,7 +216,9 @@ def gen_tags(version: ClickHouseVersion, release_type: str) -> List[str]: return tags -def buildx_args(urls: Dict[str, str], arch: str, direct_urls: List[str], version: str) -> List[str]: +def buildx_args( + urls: Dict[str, str], arch: str, direct_urls: List[str], version: str +) -> List[str]: args = [ f"--platform=linux/{arch}", f"--label=build-url={GITHUB_RUN_URL}", @@ -268,7 +270,9 @@ def build_and_push_image( urls = [url for url in direct_urls[arch] if ".deb" in url] else: urls = [url for url in direct_urls[arch] if ".tgz" in url] - cmd_args.extend(buildx_args(repo_urls, arch, direct_urls=urls, version=version.describe)) + cmd_args.extend( + buildx_args(repo_urls, arch, direct_urls=urls, version=version.describe) + ) if not push: cmd_args.append(f"--tag={image.repo}:{arch_tag}") cmd_args.extend( From b07e5d9f5a6420a9ba3d19d63070831a9db121ea Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Sun, 10 Mar 2024 08:11:59 +0300 Subject: [PATCH 020/117] Update StorageMergeTree.cpp --- src/Storages/StorageMergeTree.cpp | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/src/Storages/StorageMergeTree.cpp b/src/Storages/StorageMergeTree.cpp index 0748ac2dbdf..055a48ad998 100644 --- a/src/Storages/StorageMergeTree.cpp +++ b/src/Storages/StorageMergeTree.cpp @@ -2083,7 +2083,7 @@ void StorageMergeTree::replacePartitionFrom(const StoragePtr & source_table, con MergeTreePartInfo dst_part_info(partition_id, temp_index, temp_index, src_part->info.level); IDataPartStorage::ClonePartParams clone_params{.txn = local_context->getCurrentTransaction()}; - LOG_TRACE(log, "Partition exps are the same:part id: {}; number of disks:{}",dst_part_info.partition_id, this->getStoragePolicy()->getDisks().size()); + bool on_same_disk = false; for (const DiskPtr & disk : this->getStoragePolicy()->getDisks()) if (disk->getName() == src_part->getDataPartStorage().getDiskName()) From 29fce4143d1f177efdf1864d41429cfadea22ff1 Mon Sep 17 00:00:00 2001 From: unashi Date: Tue, 12 Mar 2024 12:07:24 +0800 Subject: [PATCH 021/117] [fix] log level from fatal->error when hardlink and copy both fail --- src/Storages/MergeTree/MergeTreeData.cpp | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/src/Storages/MergeTree/MergeTreeData.cpp b/src/Storages/MergeTree/MergeTreeData.cpp index c76ffeee874..97968f1b9c1 100644 --- a/src/Storages/MergeTree/MergeTreeData.cpp +++ b/src/Storages/MergeTree/MergeTreeData.cpp @@ -7174,7 +7174,7 @@ std::pair MergeTreeData::cloneAn } } if (!copy_successful) - LOG_FATAL(&Poco::Logger::get("MergeTreeData"), "Hard link fail, clone fail"); + LOG_ERROR(&Poco::Logger::get("MergeTreeData"), "Hard link fail, clone fail"); } From 10c7ea7a29d8426fcf4d0ca09c778cdd3e56fbbd Mon Sep 17 00:00:00 2001 From: unashi Date: Tue, 12 Mar 2024 14:32:07 +0800 Subject: [PATCH 022/117] [debug] fast test again --- src/Storages/StorageMergeTree.cpp | 1 - 1 file changed, 1 deletion(-) diff --git a/src/Storages/StorageMergeTree.cpp b/src/Storages/StorageMergeTree.cpp index 055a48ad998..928ee094583 100644 --- a/src/Storages/StorageMergeTree.cpp +++ b/src/Storages/StorageMergeTree.cpp @@ -2083,7 +2083,6 @@ void StorageMergeTree::replacePartitionFrom(const StoragePtr & source_table, con MergeTreePartInfo dst_part_info(partition_id, temp_index, temp_index, src_part->info.level); IDataPartStorage::ClonePartParams clone_params{.txn = local_context->getCurrentTransaction()}; - bool on_same_disk = false; for (const DiskPtr & disk : this->getStoragePolicy()->getDisks()) if (disk->getName() == src_part->getDataPartStorage().getDiskName()) From c1b94b2170acbf72d066928fd168c18dc571d505 Mon Sep 17 00:00:00 2001 From: unashi Date: Tue, 12 Mar 2024 15:33:43 +0800 Subject: [PATCH 023/117] [debug] fast test again again --- src/Storages/MergeTree/MergeTreeData.cpp | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/src/Storages/MergeTree/MergeTreeData.cpp b/src/Storages/MergeTree/MergeTreeData.cpp index 97968f1b9c1..7fc504d71f1 100644 --- a/src/Storages/MergeTree/MergeTreeData.cpp +++ b/src/Storages/MergeTree/MergeTreeData.cpp @@ -7174,7 +7174,7 @@ std::pair MergeTreeData::cloneAn } } if (!copy_successful) - LOG_ERROR(&Poco::Logger::get("MergeTreeData"), "Hard link fail, clone fail"); + LOG_ERROR(&Poco::Logger::get("MergeTreeData"), "Hard link fail, clone fail."); } From da06ba432e7a3232688a661aac337f9994fddead Mon Sep 17 00:00:00 2001 From: unashi Date: Wed, 13 Mar 2024 12:09:35 +0800 Subject: [PATCH 024/117] [fix] merge and resolve problems --- src/Storages/MergeTree/MergeTreeData.cpp | 21 --------------------- 1 file changed, 21 deletions(-) diff --git a/src/Storages/MergeTree/MergeTreeData.cpp b/src/Storages/MergeTree/MergeTreeData.cpp index 55ab8208472..f8448be7268 100644 --- a/src/Storages/MergeTree/MergeTreeData.cpp +++ b/src/Storages/MergeTree/MergeTreeData.cpp @@ -7233,27 +7233,6 @@ std::pair MergeTreeData::cloneAn scope_guard src_flushed_tmp_dir_lock; MergeTreeData::MutableDataPartPtr src_flushed_tmp_part; - /// If source part is in memory, flush it to disk and clone it already in on-disk format - /// Protect tmp dir from removing by cleanup thread with src_flushed_tmp_dir_lock - /// Construct src_flushed_tmp_part in order to delete part with its directory at destructor - if (auto src_part_in_memory = asInMemoryPart(src_part)) - { - auto flushed_part_path = *src_part_in_memory->getRelativePathForPrefix(tmp_part_prefix); - - auto tmp_src_part_file_name = fs::path(tmp_dst_part_name).filename(); - src_flushed_tmp_dir_lock = src_part->storage.getTemporaryPartDirectoryHolder(tmp_src_part_file_name); - - auto flushed_part_storage = src_part_in_memory->flushToDisk(flushed_part_path, metadata_snapshot); - - src_flushed_tmp_part = MergeTreeDataPartBuilder(*this, src_part->name, flushed_part_storage) - .withPartInfo(src_part->info) - .withPartFormatFromDisk() - .build(); - - src_flushed_tmp_part->is_temp = true; - src_part_storage = flushed_part_storage; - } - String with_copy; if (params.copy_instead_of_hardlink) with_copy = " (copying data)"; From cea5d460f8a14561d733fef69cff1721567c3fb2 Mon Sep 17 00:00:00 2001 From: unashi Date: Wed, 13 Mar 2024 15:06:28 +0800 Subject: [PATCH 025/117] [debug] replace 2 submodule to new version(whose version can't be found in test) --- contrib/idxd-config | 2 +- contrib/re2 | 2 +- 2 files changed, 2 insertions(+), 2 deletions(-) diff --git a/contrib/idxd-config b/contrib/idxd-config index a836ce0e420..3feff87d1f4 160000 --- a/contrib/idxd-config +++ b/contrib/idxd-config @@ -1 +1 @@ -Subproject commit a836ce0e42052a69bffbbc14239ab4097f3b77f1 +Subproject commit 3feff87d1f4add861b356f2126364750c7d00727 diff --git a/contrib/re2 b/contrib/re2 index a807e8a3aac..108914d28a7 160000 --- a/contrib/re2 +++ b/contrib/re2 @@ -1 +1 @@ -Subproject commit a807e8a3aac2cc33c77b7071efea54fcabe38e0c +Subproject commit 108914d28a79243d4300e7e651cd0a0d5883ca0f From 28bebe40a8bc457a6c743e11727269ad1edc6b0d Mon Sep 17 00:00:00 2001 From: unashi Date: Wed, 13 Mar 2024 15:17:35 +0800 Subject: [PATCH 026/117] [debug] change the submodule back to version which is similar to master --- contrib/idxd-config | 2 +- contrib/re2 | 2 +- 2 files changed, 2 insertions(+), 2 deletions(-) diff --git a/contrib/idxd-config b/contrib/idxd-config index 3feff87d1f4..a836ce0e420 160000 --- a/contrib/idxd-config +++ b/contrib/idxd-config @@ -1 +1 @@ -Subproject commit 3feff87d1f4add861b356f2126364750c7d00727 +Subproject commit a836ce0e42052a69bffbbc14239ab4097f3b77f1 diff --git a/contrib/re2 b/contrib/re2 index 108914d28a7..a807e8a3aac 160000 --- a/contrib/re2 +++ b/contrib/re2 @@ -1 +1 @@ -Subproject commit 108914d28a79243d4300e7e651cd0a0d5883ca0f +Subproject commit a807e8a3aac2cc33c77b7071efea54fcabe38e0c From a0044f3aa4d5c297229be6f6829f8d7b5e7d75c5 Mon Sep 17 00:00:00 2001 From: Nikolay Degterinsky Date: Thu, 14 Mar 2024 11:44:35 +0000 Subject: [PATCH 027/117] Fix ATTACH query with external ON CLUSTER --- src/Interpreters/InterpreterCreateQuery.cpp | 7 +++++-- tests/integration/test_external_cluster/test.py | 6 ++++++ 2 files changed, 11 insertions(+), 2 deletions(-) diff --git a/src/Interpreters/InterpreterCreateQuery.cpp b/src/Interpreters/InterpreterCreateQuery.cpp index edd7452c130..fb421ee0a5e 100644 --- a/src/Interpreters/InterpreterCreateQuery.cpp +++ b/src/Interpreters/InterpreterCreateQuery.cpp @@ -1087,8 +1087,8 @@ BlockIO InterpreterCreateQuery::createTable(ASTCreateQuery & create) // If this is a stub ATTACH query, read the query definition from the database if (create.attach && !create.storage && !create.columns_list) { - auto database = DatabaseCatalog::instance().getDatabase(database_name); - if (database->shouldReplicateQuery(getContext(), query_ptr)) + auto database = DatabaseCatalog::instance().tryGetDatabase(database_name); + if (database && database->shouldReplicateQuery(getContext(), query_ptr)) { auto guard = DatabaseCatalog::instance().getDDLGuard(database_name, create.getTable()); create.setDatabase(database_name); @@ -1099,6 +1099,9 @@ BlockIO InterpreterCreateQuery::createTable(ASTCreateQuery & create) if (!create.cluster.empty()) return executeQueryOnCluster(create); + if (!database) + throw Exception(ErrorCodes::UNKNOWN_DATABASE, "Database {} does not exist", backQuoteIfNeed(database_name)); + /// For short syntax of ATTACH query we have to lock table name here, before reading metadata /// and hold it until table is attached if (likely(need_ddl_guard)) diff --git a/tests/integration/test_external_cluster/test.py b/tests/integration/test_external_cluster/test.py index 2ed8ada3df4..6fa9cd16dbc 100644 --- a/tests/integration/test_external_cluster/test.py +++ b/tests/integration/test_external_cluster/test.py @@ -46,6 +46,12 @@ def test_ddl(started_cluster): control_node.query( "ALTER TABLE test_db.test_table ON CLUSTER 'external' add column data String" ) + control_node.query("DETACH TABLE test_db.test_table ON CLUSTER 'external'") + + expected = "" + assert_create_query(data_node, "test_db", "test_table", expected) + + control_node.query("ATTACH TABLE test_db.test_table ON CLUSTER 'external'") expected = "CREATE TABLE test_db.test_table (`id` Int64, `data` String) ENGINE = MergeTree ORDER BY id SETTINGS index_granularity = 8192" assert_create_query(data_node, "test_db", "test_table", expected) From 75aed5ce8651fa1aff93bb726418de3df413ead6 Mon Sep 17 00:00:00 2001 From: Anton Popov Date: Thu, 14 Mar 2024 16:26:38 +0000 Subject: [PATCH 028/117] fix consecutive keys optimization for nullable keys --- src/Common/ColumnsHashingImpl.h | 88 +++++++++++++++++++++++---------- 1 file changed, 62 insertions(+), 26 deletions(-) diff --git a/src/Common/ColumnsHashingImpl.h b/src/Common/ColumnsHashingImpl.h index 7116160e94c..c8d62fa7e6b 100644 --- a/src/Common/ColumnsHashingImpl.h +++ b/src/Common/ColumnsHashingImpl.h @@ -46,29 +46,45 @@ struct LastElementCacheStats namespace columns_hashing_impl { -template -struct LastElementCache +struct LastElementCacheBase { - static constexpr bool consecutive_keys_optimization = consecutive_keys_optimization_; - - Value value; bool empty = true; bool found = false; UInt64 misses = 0; - bool check(const Value & value_) const { return value == value_; } + void onNewValue(bool is_found) + { + empty = false; + found = is_found; + ++misses; + } + + bool hasOnlyOneValue() const { return found && misses == 1; } +}; + +template struct LastElementCache; + +template +struct LastElementCache : public LastElementCacheBase +{ + Value value; + bool is_null = false; + + template + bool check(const Key & key) const { return !is_null && value.first == key; } + + bool check(const Value & rhs) const { return !is_null && value == rhs; } +}; + +template +struct LastElementCache : public LastElementCacheBase +{ + Value value; template bool check(const Key & key) const { return value.first == key; } - bool hasOnlyOneValue() const { return found && misses == 1; } - UInt64 getMisses() const { return misses; } -}; - -template -struct LastElementCache -{ - static constexpr bool consecutive_keys_optimization = false; + bool check(const Value & rhs) const { return value == rhs; } }; template @@ -162,7 +178,7 @@ public: using EmplaceResult = EmplaceResultImpl; using FindResult = FindResultImpl; static constexpr bool has_mapped = !std::is_same_v; - using Cache = LastElementCache; + using Cache = LastElementCache; static HashMethodContextPtr createContext(const HashMethodContext::Settings &) { return nullptr; } @@ -173,6 +189,15 @@ public: { if (isNullAt(row)) { + if constexpr (consecutive_keys_optimization) + { + if (!cache.is_null) + { + cache.onNewValue(true); + cache.is_null = true; + } + } + bool has_null_key = data.hasNullKeyData(); data.hasNullKeyData() = true; @@ -194,10 +219,21 @@ public: { if (isNullAt(row)) { + bool has_null_key = data.hasNullKeyData(); + + if constexpr (consecutive_keys_optimization) + { + if (!cache.is_null) + { + cache.onNewValue(has_null_key); + cache.is_null = true; + } + } + if constexpr (has_mapped) - return FindResult(&data.getNullKeyData(), data.hasNullKeyData(), 0); + return FindResult(&data.getNullKeyData(), has_null_key, 0); else - return FindResult(data.hasNullKeyData(), 0); + return FindResult(has_null_key, 0); } } @@ -232,7 +268,7 @@ public: ALWAYS_INLINE UInt64 getCacheMissesSinceLastReset() const { if constexpr (consecutive_keys_optimization) - return cache.getMisses(); + return cache.misses; return 0; } @@ -304,9 +340,10 @@ protected: if constexpr (consecutive_keys_optimization) { - cache.found = true; - cache.empty = false; - ++cache.misses; + cache.onNewValue(true); + + if constexpr (nullable) + cache.is_null = false; if constexpr (has_mapped) { @@ -347,17 +384,16 @@ protected: if constexpr (consecutive_keys_optimization) { - cache.found = it != nullptr; - cache.empty = false; - ++cache.misses; + cache.onNewValue(it != nullptr); + + if constexpr (nullable) + cache.is_null = false; if constexpr (has_mapped) { cache.value.first = key; if (it) - { cache.value.second = it->getMapped(); - } } else { From e3666e42a9864623f0f509c90a81bba6581c350e Mon Sep 17 00:00:00 2001 From: Anton Popov Date: Thu, 14 Mar 2024 16:47:37 +0000 Subject: [PATCH 029/117] add a test --- .../03009_consecutive_keys_nullable.reference | 16 ++++++ .../03009_consecutive_keys_nullable.sql | 56 +++++++++++++++++++ 2 files changed, 72 insertions(+) create mode 100644 tests/queries/0_stateless/03009_consecutive_keys_nullable.reference create mode 100644 tests/queries/0_stateless/03009_consecutive_keys_nullable.sql diff --git a/tests/queries/0_stateless/03009_consecutive_keys_nullable.reference b/tests/queries/0_stateless/03009_consecutive_keys_nullable.reference new file mode 100644 index 00000000000..1c8064290c6 --- /dev/null +++ b/tests/queries/0_stateless/03009_consecutive_keys_nullable.reference @@ -0,0 +1,16 @@ +\N 1 1 +1 2 0 +\N 1 1 +1 2 0 +\N 3 3 +1 3 0 +\N 1 1 +1 2 0 +\N 2 2 +1 1 0 +t_nullable_keys_1 0 +t_nullable_keys_2 0 +t_nullable_keys_3 1 +t_nullable_keys_4 1 +t_nullable_keys_5 0 +t_nullable_keys_6 0 diff --git a/tests/queries/0_stateless/03009_consecutive_keys_nullable.sql b/tests/queries/0_stateless/03009_consecutive_keys_nullable.sql new file mode 100644 index 00000000000..ee2cb5a171f --- /dev/null +++ b/tests/queries/0_stateless/03009_consecutive_keys_nullable.sql @@ -0,0 +1,56 @@ +DROP TABLE IF EXISTS t_nullable_keys_1; + +CREATE TABLE t_nullable_keys_1 (x Nullable(Int64)) ENGINE = Memory; +INSERT INTO t_nullable_keys_1 VALUES (1), (1), (NULL); +SELECT x, count(), countIf(x IS NULL) FROM t_nullable_keys_1 GROUP BY x; + +DROP TABLE t_nullable_keys_1; + +DROP TABLE IF EXISTS t_nullable_keys_2; + +CREATE TABLE t_nullable_keys_2 (x Nullable(Int64)) ENGINE = Memory; +INSERT INTO t_nullable_keys_2 VALUES (NULL), (1), (1); +SELECT x, count(), countIf(x IS NULL) FROM t_nullable_keys_2 GROUP BY x; + +DROP TABLE t_nullable_keys_2; + +DROP TABLE IF EXISTS t_nullable_keys_3; + +CREATE TABLE t_nullable_keys_3 (x Nullable(Int64)) ENGINE = Memory; +INSERT INTO t_nullable_keys_3 VALUES (NULL), (NULL), (NULL); +SELECT x, count(), countIf(x IS NULL) FROM t_nullable_keys_3 GROUP BY x; + +DROP TABLE t_nullable_keys_3; + +DROP TABLE IF EXISTS t_nullable_keys_4; + +CREATE TABLE t_nullable_keys_4 (x Nullable(Int64)) ENGINE = Memory; +INSERT INTO t_nullable_keys_4 VALUES (1), (1), (1); +SELECT x, count(), countIf(x IS NULL) FROM t_nullable_keys_4 GROUP BY x; + +DROP TABLE t_nullable_keys_4; + +DROP TABLE IF EXISTS t_nullable_keys_5; + +CREATE TABLE t_nullable_keys_5 (x Nullable(Int64)) ENGINE = Memory; +INSERT INTO t_nullable_keys_5 VALUES (1), (NULL), (1); +SELECT x, count(), countIf(x IS NULL) FROM t_nullable_keys_5 GROUP BY x; + +DROP TABLE t_nullable_keys_5; + +DROP TABLE IF EXISTS t_nullable_keys_6; + +CREATE TABLE t_nullable_keys_6 (x Nullable(Int64)) ENGINE = Memory; +INSERT INTO t_nullable_keys_6 VALUES (NULL), (1), (NULL); +SELECT x, count(), countIf(x IS NULL) FROM t_nullable_keys_6 GROUP BY x; + +DROP TABLE t_nullable_keys_6; + +SYSTEM FLUSH LOGS; + +SELECT + splitByChar('.', tables[1])[2] AS table, + ProfileEvents['AggregationOptimizedEqualRangesOfKeys'] > 0 +FROM system.query_log +WHERE type = 'QueryFinish' AND current_database = currentDatabase() AND query LIKE '%SELECT%FROM%t_nullable_keys_%' +ORDER BY table; From 4e2e5a015b162d63dd192dc1cdb0cb622a524cf1 Mon Sep 17 00:00:00 2001 From: Nikolay Degterinsky <43110995+evillique@users.noreply.github.com> Date: Tue, 19 Mar 2024 16:29:50 +0100 Subject: [PATCH 030/117] Fixes --- src/Interpreters/InterpreterCreateQuery.cpp | 4 +++- 1 file changed, 3 insertions(+), 1 deletion(-) diff --git a/src/Interpreters/InterpreterCreateQuery.cpp b/src/Interpreters/InterpreterCreateQuery.cpp index fb421ee0a5e..38918918c92 100644 --- a/src/Interpreters/InterpreterCreateQuery.cpp +++ b/src/Interpreters/InterpreterCreateQuery.cpp @@ -1087,6 +1087,7 @@ BlockIO InterpreterCreateQuery::createTable(ASTCreateQuery & create) // If this is a stub ATTACH query, read the query definition from the database if (create.attach && !create.storage && !create.columns_list) { + // In case of an ON CLUSTER query, the database may not be present on the initiator node auto database = DatabaseCatalog::instance().tryGetDatabase(database_name); if (database && database->shouldReplicateQuery(getContext(), query_ptr)) { @@ -1253,6 +1254,7 @@ BlockIO InterpreterCreateQuery::createTable(ASTCreateQuery & create) DatabasePtr database; bool need_add_to_database = !create.temporary; + // In case of an ON CLUSTER query, the database may not be present on the initiator node if (need_add_to_database) database = DatabaseCatalog::instance().tryGetDatabase(database_name); @@ -1273,7 +1275,7 @@ BlockIO InterpreterCreateQuery::createTable(ASTCreateQuery & create) "CREATE AS SELECT is not supported with Replicated databases. Use separate CREATE and INSERT queries"); } - if (need_add_to_database && database && database->shouldReplicateQuery(getContext(), query_ptr)) + if (database && database->shouldReplicateQuery(getContext(), query_ptr)) { chassert(!ddl_guard); auto guard = DatabaseCatalog::instance().getDDLGuard(create.getDatabase(), create.getTable()); From 11d0b0a9fad7414cd5be6d3cccac80c1e5f66a91 Mon Sep 17 00:00:00 2001 From: Azat Khuzhin Date: Thu, 14 Mar 2024 19:22:02 +0100 Subject: [PATCH 031/117] Forbid SimpleAggregateFunction in ORDER BY of MergeTree tables Like AggregateFunction is forbidden, but they are forbidden because they are not comparable. New setting (allow_suspicious_primary_key) had been added for backard compatiblity (turned OFF by default). Signed-off-by: Azat Khuzhin --- src/Core/Settings.h | 1 + src/Core/SettingsChangesHistory.h | 1 + .../MergeTree/registerStorageMergeTree.cpp | 24 ++++++++++++++++--- .../01410_nullable_key_and_index.sql | 2 +- ...order_by_SimpleAggregateFunction.reference | 0 ...03020_order_by_SimpleAggregateFunction.sql | 14 +++++++++++ 6 files changed, 38 insertions(+), 4 deletions(-) create mode 100644 tests/queries/0_stateless/03020_order_by_SimpleAggregateFunction.reference create mode 100644 tests/queries/0_stateless/03020_order_by_SimpleAggregateFunction.sql diff --git a/src/Core/Settings.h b/src/Core/Settings.h index a3c5638d97f..7d53803e41f 100644 --- a/src/Core/Settings.h +++ b/src/Core/Settings.h @@ -161,6 +161,7 @@ class IColumn; M(Bool, allow_suspicious_indices, false, "Reject primary/secondary indexes and sorting keys with identical expressions", 0) \ M(Bool, allow_suspicious_ttl_expressions, false, "Reject TTL expressions that don't depend on any of table's columns. It indicates a user error most of the time.", 0) \ M(Bool, allow_suspicious_variant_types, false, "In CREATE TABLE statement allows specifying Variant type with similar variant types (for example, with different numeric or date types). Enabling this setting may introduce some ambiguity when working with values with similar types.", 0) \ + M(Bool, allow_suspicious_primary_key, false, "Forbid suspicious PRIMARY KEY/ORDER BY for MergeTree (i.e. SimpleAggregateFunction)", 0) \ M(Bool, compile_expressions, false, "Compile some scalar functions and operators to native code.", 0) \ M(UInt64, min_count_to_compile_expression, 3, "The number of identical expressions before they are JIT-compiled", 0) \ M(Bool, compile_aggregate_expressions, true, "Compile aggregate functions to native code.", 0) \ diff --git a/src/Core/SettingsChangesHistory.h b/src/Core/SettingsChangesHistory.h index 072b9803682..2d8c173d850 100644 --- a/src/Core/SettingsChangesHistory.h +++ b/src/Core/SettingsChangesHistory.h @@ -95,6 +95,7 @@ static std::map sett {"throw_if_deduplication_in_dependent_materialized_views_enabled_with_async_insert", false, true, "Deduplication is dependent materialized view cannot work together with async inserts."}, {"parallel_replicas_allow_in_with_subquery", false, true, "If true, subquery for IN will be executed on every follower replica"}, {"function_locate_has_mysql_compatible_argument_order", false, true, "Increase compatibility with MySQL's locate function."}, + {"allow_suspicious_primary_key", true, false, "Forbid suspicious PRIMARY KEY/ORDER BY for MergeTree (i.e. SimpleAggregateFunction)"}, {"filesystem_cache_reserve_space_wait_lock_timeout_milliseconds", 1000, 1000, "Wait time to lock cache for sapce reservation in filesystem cache"}, {"max_parser_backtracks", 0, 1000000, "Limiting the complexity of parsing"}, {"keeper_max_retries", 10, 10, "Max retries for general keeper operations"}, diff --git a/src/Storages/MergeTree/registerStorageMergeTree.cpp b/src/Storages/MergeTree/registerStorageMergeTree.cpp index 2b24a56e994..e89547952d0 100644 --- a/src/Storages/MergeTree/registerStorageMergeTree.cpp +++ b/src/Storages/MergeTree/registerStorageMergeTree.cpp @@ -13,6 +13,7 @@ #include #include +#include #include #include @@ -30,6 +31,7 @@ namespace ErrorCodes extern const int UNKNOWN_STORAGE; extern const int NO_REPLICA_NAME_GIVEN; extern const int CANNOT_EXTRACT_TABLE_STRUCTURE; + extern const int DATA_TYPE_CANNOT_BE_USED_IN_KEY; } @@ -110,6 +112,16 @@ static ColumnsDescription getColumnsDescriptionFromZookeeper(const String & raw_ return ColumnsDescription::parse(zookeeper->get(fs::path(zookeeper_path) / "columns", &columns_stat)); } +static void verifySortingKey(const KeyDescription & sorting_key) +{ + /// Aggregate functions already forbidden, but SimpleAggregateFunction are not + for (const auto & data_type : sorting_key.data_types) + { + if (dynamic_cast(data_type->getCustomName())) + throw Exception(ErrorCodes::DATA_TYPE_CANNOT_BE_USED_IN_KEY, "Column with type {} is not allowed in key expression", data_type->getCustomName()->getName()); + } +} + static StoragePtr create(const StorageFactory::Arguments & args) { @@ -148,6 +160,8 @@ static StoragePtr create(const StorageFactory::Arguments & args) || (args.query.columns_list->indices && !args.query.columns_list->indices->children.empty()) || (args.query.columns_list->projections && !args.query.columns_list->projections->children.empty()); + const Settings & local_settings = args.getLocalContext()->getSettingsRef(); + String name_part = args.engine_name.substr(0, args.engine_name.size() - strlen("MergeTree")); bool replicated = startsWith(name_part, "Replicated"); @@ -293,7 +307,7 @@ static StoragePtr create(const StorageFactory::Arguments & args) arg_idx, e.message(), verbose_help_message); } } - else if (args.mode <= LoadingStrictnessLevel::CREATE && !args.getLocalContext()->getSettingsRef().allow_deprecated_syntax_for_merge_tree) + else if (args.mode <= LoadingStrictnessLevel::CREATE && !local_settings.allow_deprecated_syntax_for_merge_tree) { throw Exception(ErrorCodes::BAD_ARGUMENTS, "This syntax for *MergeTree engine is deprecated. " "Use extended storage definition syntax with ORDER BY/PRIMARY KEY clause. " @@ -532,7 +546,7 @@ static StoragePtr create(const StorageFactory::Arguments & args) if (!args.storage_def->order_by) { - if (args.getLocalContext()->getSettingsRef().create_table_empty_primary_key_by_default) + if (local_settings.create_table_empty_primary_key_by_default) { args.storage_def->set(args.storage_def->order_by, makeASTFunction("tuple")); } @@ -553,6 +567,8 @@ static StoragePtr create(const StorageFactory::Arguments & args) /// column if sorting key will be changed. metadata.sorting_key = KeyDescription::getSortingKeyFromAST( args.storage_def->order_by->ptr(), metadata.columns, context, merging_param_key_arg); + if (!local_settings.allow_suspicious_primary_key) + verifySortingKey(metadata.sorting_key); /// If primary key explicitly defined, than get it from AST if (args.storage_def->primary_key) @@ -577,7 +593,7 @@ static StoragePtr create(const StorageFactory::Arguments & args) if (args.storage_def->sample_by) metadata.sampling_key = KeyDescription::getKeyFromAST(args.storage_def->sample_by->ptr(), metadata.columns, context); - bool allow_suspicious_ttl = LoadingStrictnessLevel::SECONDARY_CREATE <= args.mode || args.getLocalContext()->getSettingsRef().allow_suspicious_ttl_expressions; + bool allow_suspicious_ttl = LoadingStrictnessLevel::SECONDARY_CREATE <= args.mode || local_settings.allow_suspicious_ttl_expressions; if (args.storage_def->ttl_table) { @@ -665,6 +681,8 @@ static StoragePtr create(const StorageFactory::Arguments & args) /// column if sorting key will be changed. metadata.sorting_key = KeyDescription::getSortingKeyFromAST(engine_args[arg_num], metadata.columns, context, merging_param_key_arg); + if (!local_settings.allow_suspicious_primary_key) + verifySortingKey(metadata.sorting_key); /// In old syntax primary_key always equals to sorting key. metadata.primary_key = KeyDescription::getKeyFromAST(engine_args[arg_num], metadata.columns, context); diff --git a/tests/queries/0_stateless/01410_nullable_key_and_index.sql b/tests/queries/0_stateless/01410_nullable_key_and_index.sql index 905d997d95c..7c28a7a6e70 100644 --- a/tests/queries/0_stateless/01410_nullable_key_and_index.sql +++ b/tests/queries/0_stateless/01410_nullable_key_and_index.sql @@ -73,5 +73,5 @@ CREATE TABLE invalid_lc_null (id LowCardinality(Nullable(String))) ENGINE = Merg CREATE TABLE invalid_array_null (id Array(Nullable(String))) ENGINE = MergeTree ORDER BY id; -- { serverError 44 } CREATE TABLE invalid_tuple_null (id Tuple(Nullable(String), UInt8)) ENGINE = MergeTree ORDER BY id; -- { serverError 44 } CREATE TABLE invalid_map_null (id Map(UInt8, Nullable(String))) ENGINE = MergeTree ORDER BY id; -- { serverError 44 } -CREATE TABLE invalid_simple_agg_state_null (id SimpleAggregateFunction(sum, Nullable(UInt64))) ENGINE = MergeTree ORDER BY id; -- { serverError 44 } +CREATE TABLE invalid_simple_agg_state_null (id SimpleAggregateFunction(sum, Nullable(UInt64))) ENGINE = MergeTree ORDER BY id; -- { serverError DATA_TYPE_CANNOT_BE_USED_IN_KEY } -- AggregateFunctions are not comparable and cannot be used in key expressions. No need to test it. diff --git a/tests/queries/0_stateless/03020_order_by_SimpleAggregateFunction.reference b/tests/queries/0_stateless/03020_order_by_SimpleAggregateFunction.reference new file mode 100644 index 00000000000..e69de29bb2d diff --git a/tests/queries/0_stateless/03020_order_by_SimpleAggregateFunction.sql b/tests/queries/0_stateless/03020_order_by_SimpleAggregateFunction.sql new file mode 100644 index 00000000000..fd3418fb50e --- /dev/null +++ b/tests/queries/0_stateless/03020_order_by_SimpleAggregateFunction.sql @@ -0,0 +1,14 @@ +set allow_suspicious_primary_key = 0; + +create table data (key Int, value AggregateFunction(sum, UInt64)) engine=AggregatingMergeTree() order by (key, value); -- { serverError DATA_TYPE_CANNOT_BE_USED_IN_KEY } +create table data (key Int, value SimpleAggregateFunction(sum, UInt64)) engine=AggregatingMergeTree() order by (key, value); -- { serverError DATA_TYPE_CANNOT_BE_USED_IN_KEY } + +create table data (key Int, value AggregateFunction(sum, UInt64)) engine=AggregatingMergeTree() primary key value; -- { serverError DATA_TYPE_CANNOT_BE_USED_IN_KEY } +create table data (key Int, value SimpleAggregateFunction(sum, UInt64)) engine=AggregatingMergeTree() primary key value; -- { serverError DATA_TYPE_CANNOT_BE_USED_IN_KEY } + +create table data (key Int, value AggregateFunction(sum, UInt64)) engine=AggregatingMergeTree() primary key value order by (value, key); -- { serverError DATA_TYPE_CANNOT_BE_USED_IN_KEY } +create table data (key Int, value SimpleAggregateFunction(sum, UInt64)) engine=AggregatingMergeTree() primary key value order by (value, key); -- { serverError DATA_TYPE_CANNOT_BE_USED_IN_KEY } + +set allow_suspicious_primary_key = 1; + +create table data (key Int, value SimpleAggregateFunction(sum, UInt64)) engine=AggregatingMergeTree() primary key value order by (value, key); From 624b3a6dbe198eb3c125fe37593b8c22b82c78f8 Mon Sep 17 00:00:00 2001 From: Nikolai Kochetov Date: Wed, 20 Mar 2024 12:08:46 +0000 Subject: [PATCH 032/117] Fixing 00223_shard_distributed_aggregation_memory_efficient with analyzer. --- src/Analyzer/Passes/QueryAnalysisPass.cpp | 17 ++++++++++++++++- src/Functions/randConstant.cpp | 16 +++++++++++----- tests/analyzer_tech_debt.txt | 1 - 3 files changed, 27 insertions(+), 7 deletions(-) diff --git a/src/Analyzer/Passes/QueryAnalysisPass.cpp b/src/Analyzer/Passes/QueryAnalysisPass.cpp index 1da5d86edf3..2a46417ee98 100644 --- a/src/Analyzer/Passes/QueryAnalysisPass.cpp +++ b/src/Analyzer/Passes/QueryAnalysisPass.cpp @@ -799,6 +799,8 @@ struct IdentifierResolveScope /// Node hash to mask id map std::shared_ptr> projection_mask_map; + std::map rand_constant_cache; + [[maybe_unused]] const IdentifierResolveScope * getNearestQueryScope() const { const IdentifierResolveScope * scope_to_check = this; @@ -5534,7 +5536,20 @@ ProjectionNames QueryAnalyzer::resolveFunction(QueryTreeNodePtr & node, Identifi if (!function) { - function = FunctionFactory::instance().tryGet(function_name, scope.context); + /// This is a hack to allow a query like `select randConstant(), randConstant(), randConstant()`. + /// Function randConstant() would return the same value for the same arguments (in scope). + if (function_name == "randConstant") + { + auto hash = function_node_ptr->getTreeHash(); + auto & func = scope.rand_constant_cache[hash]; + if (!func) + func = FunctionFactory::instance().tryGet(function_name, scope.context); + + function = func; + } + else + function = FunctionFactory::instance().tryGet(function_name, scope.context); + is_executable_udf = false; } diff --git a/src/Functions/randConstant.cpp b/src/Functions/randConstant.cpp index 22ce6d88ea6..5872ab8bb34 100644 --- a/src/Functions/randConstant.cpp +++ b/src/Functions/randConstant.cpp @@ -88,6 +88,14 @@ public: return std::make_unique>(); } + RandomConstantOverloadResolver() + { + typename ColumnVector::Container vec_to(1); + + TargetSpecific::Default::RandImpl::execute(reinterpret_cast(vec_to.data()), sizeof(ToType)); + value = vec_to[0]; + } + DataTypePtr getReturnTypeImpl(const DataTypes & data_types) const override { size_t number_of_arguments = data_types.size(); @@ -105,13 +113,11 @@ public: if (!arguments.empty()) argument_types.emplace_back(arguments.back().type); - typename ColumnVector::Container vec_to(1); - - TargetSpecific::Default::RandImpl::execute(reinterpret_cast(vec_to.data()), sizeof(ToType)); - ToType value = vec_to[0]; - return std::make_unique>(value, argument_types, return_type); } + +private: + ToType value; }; struct NameRandConstant { static constexpr auto name = "randConstant"; }; diff --git a/tests/analyzer_tech_debt.txt b/tests/analyzer_tech_debt.txt index 7cd73705e77..2d8497234e2 100644 --- a/tests/analyzer_tech_debt.txt +++ b/tests/analyzer_tech_debt.txt @@ -1,4 +1,3 @@ -00223_shard_distributed_aggregation_memory_efficient 00725_memory_tracking 01062_pm_all_join_with_block_continuation 01083_expressions_in_engine_arguments From ccf6657f4dd9354fb31c7c08889eb0b51f498f07 Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?J=C3=A1nos=20Benjamin=20Antal?= Date: Wed, 20 Mar 2024 16:20:39 +0000 Subject: [PATCH 033/117] Remove unused macro --- src/Dictionaries/DictionaryStructure.h | 27 -------------------------- 1 file changed, 27 deletions(-) diff --git a/src/Dictionaries/DictionaryStructure.h b/src/Dictionaries/DictionaryStructure.h index 56d11be9837..55060b1592f 100644 --- a/src/Dictionaries/DictionaryStructure.h +++ b/src/Dictionaries/DictionaryStructure.h @@ -41,33 +41,6 @@ enum class AttributeUnderlyingType : TypeIndexUnderlying #undef map_item - -#define CALL_FOR_ALL_DICTIONARY_ATTRIBUTE_TYPES(M) \ - M(UInt8) \ - M(UInt16) \ - M(UInt32) \ - M(UInt64) \ - M(UInt128) \ - M(UInt256) \ - M(Int8) \ - M(Int16) \ - M(Int32) \ - M(Int64) \ - M(Int128) \ - M(Int256) \ - M(Decimal32) \ - M(Decimal64) \ - M(Decimal128) \ - M(Decimal256) \ - M(DateTime64) \ - M(Float32) \ - M(Float64) \ - M(UUID) \ - M(IPv4) \ - M(IPv6) \ - M(String) \ - M(Array) - /// Min and max lifetimes for a dictionary or its entry using DictionaryLifetime = ExternalLoadableLifetime; From 9e8c731e1e96ff4f8dc131f312a13ca46acb8c72 Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?J=C3=A1nos=20Benjamin=20Antal?= Date: Wed, 20 Mar 2024 16:22:15 +0000 Subject: [PATCH 034/117] Remove some inclusion of magic enum --- src/Common/ExternalLoaderStatus.cpp | 18 +++ src/Common/ExternalLoaderStatus.h | 32 ++--- src/Core/SettingsEnums.cpp | 10 ++ src/Core/SettingsEnums.h | 102 ++++++++++++++++ src/Core/SettingsFields.h | 111 ------------------ src/IO/WriteHelpers.h | 1 - src/Parsers/ASTTablesInSelectQuery.h | 2 - .../System/StorageSystemDictionaries.cpp | 2 +- 8 files changed, 142 insertions(+), 136 deletions(-) create mode 100644 src/Common/ExternalLoaderStatus.cpp diff --git a/src/Common/ExternalLoaderStatus.cpp b/src/Common/ExternalLoaderStatus.cpp new file mode 100644 index 00000000000..f3298409f43 --- /dev/null +++ b/src/Common/ExternalLoaderStatus.cpp @@ -0,0 +1,18 @@ +#include + +#include + +namespace DB +{ + +std::vector> getExternalLoaderStatusEnumAllPossibleValues() +{ + std::vector> out; + out.reserve(magic_enum::enum_count()); + + for (const auto & [value, str] : magic_enum::enum_entries()) + out.emplace_back(std::string{str}, static_cast(value)); + + return out; +} +} diff --git a/src/Common/ExternalLoaderStatus.h b/src/Common/ExternalLoaderStatus.h index 6cd27837d8f..b6e98073fea 100644 --- a/src/Common/ExternalLoaderStatus.h +++ b/src/Common/ExternalLoaderStatus.h @@ -1,30 +1,20 @@ #pragma once #include -#include #include namespace DB { - enum class ExternalLoaderStatus : int8_t - { - NOT_LOADED, /// Object hasn't been tried to load. This is an initial state. - LOADED, /// Object has been loaded successfully. - FAILED, /// Object has been failed to load. - LOADING, /// Object is being loaded right now for the first time. - FAILED_AND_RELOADING, /// Object was failed to load before and it's being reloaded right now. - LOADED_AND_RELOADING, /// Object was loaded successfully before and it's being reloaded right now. - NOT_EXIST, /// Object with this name wasn't found in the configuration. - }; +enum class ExternalLoaderStatus : int8_t +{ + NOT_LOADED, /// Object hasn't been tried to load. This is an initial state. + LOADED, /// Object has been loaded successfully. + FAILED, /// Object has been failed to load. + LOADING, /// Object is being loaded right now for the first time. + FAILED_AND_RELOADING, /// Object was failed to load before and it's being reloaded right now. + LOADED_AND_RELOADING, /// Object was loaded successfully before and it's being reloaded right now. + NOT_EXIST, /// Object with this name wasn't found in the configuration. +}; - inline std::vector> getStatusEnumAllPossibleValues() - { - std::vector> out; - out.reserve(magic_enum::enum_count()); - - for (const auto & [value, str] : magic_enum::enum_entries()) - out.emplace_back(std::string{str}, static_cast(value)); - - return out; - } +std::vector> getExternalLoaderStatusEnumAllPossibleValues(); } diff --git a/src/Core/SettingsEnums.cpp b/src/Core/SettingsEnums.cpp index de30d6d8eb5..c3f0715ad68 100644 --- a/src/Core/SettingsEnums.cpp +++ b/src/Core/SettingsEnums.cpp @@ -18,6 +18,16 @@ namespace ErrorCodes extern const int UNKNOWN_UNION; } +template +constexpr auto getEnumValues() +{ + std::array, magic_enum::enum_count()> enum_values{}; + size_t index = 0; + for (auto value : magic_enum::enum_values()) + enum_values[index++] = std::pair{magic_enum::enum_name(value), value}; + return enum_values; +} + IMPLEMENT_SETTING_ENUM(LoadBalancing, ErrorCodes::UNKNOWN_LOAD_BALANCING, {{"random", LoadBalancing::RANDOM}, {"nearest_hostname", LoadBalancing::NEAREST_HOSTNAME}, diff --git a/src/Core/SettingsEnums.h b/src/Core/SettingsEnums.h index 22fcf0389d8..c11ae75bfc7 100644 --- a/src/Core/SettingsEnums.h +++ b/src/Core/SettingsEnums.h @@ -12,6 +12,108 @@ namespace DB { +template +constexpr auto getEnumValues(); + +/// NOLINTNEXTLINE +#define DECLARE_SETTING_ENUM(ENUM_TYPE) \ + DECLARE_SETTING_ENUM_WITH_RENAME(ENUM_TYPE, ENUM_TYPE) + +/// NOLINTNEXTLINE +#define DECLARE_SETTING_ENUM_WITH_RENAME(NEW_NAME, ENUM_TYPE) \ + struct SettingField##NEW_NAME##Traits \ + { \ + using EnumType = ENUM_TYPE; \ + using EnumValuePairs = std::pair[]; \ + static const String & toString(EnumType value); \ + static EnumType fromString(std::string_view str); \ + }; \ + \ + using SettingField##NEW_NAME = SettingFieldEnum; + +/// NOLINTNEXTLINE +#define IMPLEMENT_SETTING_ENUM(NEW_NAME, ERROR_CODE_FOR_UNEXPECTED_NAME, ...) \ + IMPLEMENT_SETTING_ENUM_IMPL(NEW_NAME, ERROR_CODE_FOR_UNEXPECTED_NAME, EnumValuePairs, __VA_ARGS__) + +/// NOLINTNEXTLINE +#define IMPLEMENT_SETTING_AUTO_ENUM(NEW_NAME, ERROR_CODE_FOR_UNEXPECTED_NAME) \ + IMPLEMENT_SETTING_ENUM_IMPL(NEW_NAME, ERROR_CODE_FOR_UNEXPECTED_NAME, , getEnumValues()) + +/// NOLINTNEXTLINE +#define IMPLEMENT_SETTING_ENUM_IMPL(NEW_NAME, ERROR_CODE_FOR_UNEXPECTED_NAME, PAIRS_TYPE, ...) \ + const String & SettingField##NEW_NAME##Traits::toString(typename SettingField##NEW_NAME::EnumType value) \ + { \ + static const std::unordered_map map = [] { \ + std::unordered_map res; \ + for (const auto & [name, val] : PAIRS_TYPE __VA_ARGS__) \ + res.emplace(val, name); \ + return res; \ + }(); \ + auto it = map.find(value); \ + if (it != map.end()) \ + return it->second; \ + throw Exception(ERROR_CODE_FOR_UNEXPECTED_NAME, \ + "Unexpected value of " #NEW_NAME ":{}", std::to_string(std::underlying_type_t(value))); \ + } \ + \ + typename SettingField##NEW_NAME::EnumType SettingField##NEW_NAME##Traits::fromString(std::string_view str) \ + { \ + static const std::unordered_map map = [] { \ + std::unordered_map res; \ + for (const auto & [name, val] : PAIRS_TYPE __VA_ARGS__) \ + res.emplace(name, val); \ + return res; \ + }(); \ + auto it = map.find(str); \ + if (it != map.end()) \ + return it->second; \ + String msg; \ + bool need_comma = false; \ + for (auto & name : map | boost::adaptors::map_keys) \ + { \ + if (std::exchange(need_comma, true)) \ + msg += ", "; \ + msg += "'" + String{name} + "'"; \ + } \ + throw Exception(ERROR_CODE_FOR_UNEXPECTED_NAME, "Unexpected value of " #NEW_NAME ": '{}'. Must be one of [{}]", String{str}, msg); \ + } + +/// NOLINTNEXTLINE +#define DECLARE_SETTING_MULTI_ENUM(ENUM_TYPE) \ + DECLARE_SETTING_MULTI_ENUM_WITH_RENAME(ENUM_TYPE, ENUM_TYPE) + +/// NOLINTNEXTLINE +#define DECLARE_SETTING_MULTI_ENUM_WITH_RENAME(ENUM_TYPE, NEW_NAME) \ + struct SettingField##NEW_NAME##Traits \ + { \ + using EnumType = ENUM_TYPE; \ + using EnumValuePairs = std::pair[]; \ + static size_t getEnumSize(); \ + static const String & toString(EnumType value); \ + static EnumType fromString(std::string_view str); \ + }; \ + \ + using SettingField##NEW_NAME = SettingFieldMultiEnum; \ + using NEW_NAME##List = typename SettingField##NEW_NAME::ValueType; + +/// NOLINTNEXTLINE +#define IMPLEMENT_SETTING_MULTI_ENUM(ENUM_TYPE, ERROR_CODE_FOR_UNEXPECTED_NAME, ...) \ + IMPLEMENT_SETTING_MULTI_ENUM_WITH_RENAME(ENUM_TYPE, ERROR_CODE_FOR_UNEXPECTED_NAME, __VA_ARGS__) + +/// NOLINTNEXTLINE +#define IMPLEMENT_SETTING_MULTI_ENUM_WITH_RENAME(NEW_NAME, ERROR_CODE_FOR_UNEXPECTED_NAME, ...) \ + IMPLEMENT_SETTING_ENUM(NEW_NAME, ERROR_CODE_FOR_UNEXPECTED_NAME, __VA_ARGS__)\ + size_t SettingField##NEW_NAME##Traits::getEnumSize() {\ + return std::initializer_list> __VA_ARGS__ .size();\ + } + +/// NOLINTNEXTLINE +#define IMPLEMENT_SETTING_MULTI_AUTO_ENUM(NEW_NAME, ERROR_CODE_FOR_UNEXPECTED_NAME) \ + IMPLEMENT_SETTING_AUTO_ENUM(NEW_NAME, ERROR_CODE_FOR_UNEXPECTED_NAME)\ + size_t SettingField##NEW_NAME##Traits::getEnumSize() {\ + return getEnumValues().size();\ + } + enum class LoadBalancing { /// among replicas with a minimum number of errors selected randomly diff --git a/src/Core/SettingsFields.h b/src/Core/SettingsFields.h index 452f3f149ab..cef30bb1916 100644 --- a/src/Core/SettingsFields.h +++ b/src/Core/SettingsFields.h @@ -7,9 +7,7 @@ #include #include #include -#include #include -#include namespace DB @@ -380,79 +378,6 @@ void SettingFieldEnum::readBinary(ReadBuffer & in) *this = Traits::fromString(SettingFieldEnumHelpers::readBinary(in)); } -template -constexpr auto getEnumValues() -{ - std::array, magic_enum::enum_count()> enum_values{}; - size_t index = 0; - for (auto value : magic_enum::enum_values()) - enum_values[index++] = std::pair{magic_enum::enum_name(value), value}; - return enum_values; -} - -/// NOLINTNEXTLINE -#define DECLARE_SETTING_ENUM(ENUM_TYPE) \ - DECLARE_SETTING_ENUM_WITH_RENAME(ENUM_TYPE, ENUM_TYPE) - -/// NOLINTNEXTLINE -#define DECLARE_SETTING_ENUM_WITH_RENAME(NEW_NAME, ENUM_TYPE) \ - struct SettingField##NEW_NAME##Traits \ - { \ - using EnumType = ENUM_TYPE; \ - using EnumValuePairs = std::pair[]; \ - static const String & toString(EnumType value); \ - static EnumType fromString(std::string_view str); \ - }; \ - \ - using SettingField##NEW_NAME = SettingFieldEnum; - -/// NOLINTNEXTLINE -#define IMPLEMENT_SETTING_ENUM(NEW_NAME, ERROR_CODE_FOR_UNEXPECTED_NAME, ...) \ - IMPLEMENT_SETTING_ENUM_IMPL(NEW_NAME, ERROR_CODE_FOR_UNEXPECTED_NAME, EnumValuePairs, __VA_ARGS__) - -/// NOLINTNEXTLINE -#define IMPLEMENT_SETTING_AUTO_ENUM(NEW_NAME, ERROR_CODE_FOR_UNEXPECTED_NAME) \ - IMPLEMENT_SETTING_ENUM_IMPL(NEW_NAME, ERROR_CODE_FOR_UNEXPECTED_NAME, , getEnumValues()) - -/// NOLINTNEXTLINE -#define IMPLEMENT_SETTING_ENUM_IMPL(NEW_NAME, ERROR_CODE_FOR_UNEXPECTED_NAME, PAIRS_TYPE, ...) \ - const String & SettingField##NEW_NAME##Traits::toString(typename SettingField##NEW_NAME::EnumType value) \ - { \ - static const std::unordered_map map = [] { \ - std::unordered_map res; \ - for (const auto & [name, val] : PAIRS_TYPE __VA_ARGS__) \ - res.emplace(val, name); \ - return res; \ - }(); \ - auto it = map.find(value); \ - if (it != map.end()) \ - return it->second; \ - throw Exception(ERROR_CODE_FOR_UNEXPECTED_NAME, \ - "Unexpected value of " #NEW_NAME ":{}", std::to_string(std::underlying_type_t(value))); \ - } \ - \ - typename SettingField##NEW_NAME::EnumType SettingField##NEW_NAME##Traits::fromString(std::string_view str) \ - { \ - static const std::unordered_map map = [] { \ - std::unordered_map res; \ - for (const auto & [name, val] : PAIRS_TYPE __VA_ARGS__) \ - res.emplace(name, val); \ - return res; \ - }(); \ - auto it = map.find(str); \ - if (it != map.end()) \ - return it->second; \ - String msg; \ - bool need_comma = false; \ - for (auto & name : map | boost::adaptors::map_keys) \ - { \ - if (std::exchange(need_comma, true)) \ - msg += ", "; \ - msg += "'" + String{name} + "'"; \ - } \ - throw Exception(ERROR_CODE_FOR_UNEXPECTED_NAME, "Unexpected value of " #NEW_NAME ": '{}'. Must be one of [{}]", String{str}, msg); \ - } - // Mostly like SettingFieldEnum, but can have multiple enum values (or none) set at once. template struct SettingFieldMultiEnum @@ -543,42 +468,6 @@ void SettingFieldMultiEnum::readBinary(ReadBuffer & in) parseFromString(SettingFieldEnumHelpers::readBinary(in)); } -/// NOLINTNEXTLINE -#define DECLARE_SETTING_MULTI_ENUM(ENUM_TYPE) \ - DECLARE_SETTING_MULTI_ENUM_WITH_RENAME(ENUM_TYPE, ENUM_TYPE) - -/// NOLINTNEXTLINE -#define DECLARE_SETTING_MULTI_ENUM_WITH_RENAME(ENUM_TYPE, NEW_NAME) \ - struct SettingField##NEW_NAME##Traits \ - { \ - using EnumType = ENUM_TYPE; \ - using EnumValuePairs = std::pair[]; \ - static size_t getEnumSize(); \ - static const String & toString(EnumType value); \ - static EnumType fromString(std::string_view str); \ - }; \ - \ - using SettingField##NEW_NAME = SettingFieldMultiEnum; \ - using NEW_NAME##List = typename SettingField##NEW_NAME::ValueType; - -/// NOLINTNEXTLINE -#define IMPLEMENT_SETTING_MULTI_ENUM(ENUM_TYPE, ERROR_CODE_FOR_UNEXPECTED_NAME, ...) \ - IMPLEMENT_SETTING_MULTI_ENUM_WITH_RENAME(ENUM_TYPE, ERROR_CODE_FOR_UNEXPECTED_NAME, __VA_ARGS__) - -/// NOLINTNEXTLINE -#define IMPLEMENT_SETTING_MULTI_ENUM_WITH_RENAME(NEW_NAME, ERROR_CODE_FOR_UNEXPECTED_NAME, ...) \ - IMPLEMENT_SETTING_ENUM(NEW_NAME, ERROR_CODE_FOR_UNEXPECTED_NAME, __VA_ARGS__)\ - size_t SettingField##NEW_NAME##Traits::getEnumSize() {\ - return std::initializer_list> __VA_ARGS__ .size();\ - } - -/// NOLINTNEXTLINE -#define IMPLEMENT_SETTING_MULTI_AUTO_ENUM(NEW_NAME, ERROR_CODE_FOR_UNEXPECTED_NAME) \ - IMPLEMENT_SETTING_AUTO_ENUM(NEW_NAME, ERROR_CODE_FOR_UNEXPECTED_NAME)\ - size_t SettingField##NEW_NAME##Traits::getEnumSize() {\ - return getEnumValues().size();\ - } - /// Setting field for specifying user-defined timezone. It is basically a string, but it needs validation. struct SettingFieldTimezone { diff --git a/src/IO/WriteHelpers.h b/src/IO/WriteHelpers.h index 8b743e6351b..b42b4e6e978 100644 --- a/src/IO/WriteHelpers.h +++ b/src/IO/WriteHelpers.h @@ -19,7 +19,6 @@ #include #include #include -#include #include #include diff --git a/src/Parsers/ASTTablesInSelectQuery.h b/src/Parsers/ASTTablesInSelectQuery.h index 67370eaee14..39ef0b21f6d 100644 --- a/src/Parsers/ASTTablesInSelectQuery.h +++ b/src/Parsers/ASTTablesInSelectQuery.h @@ -1,7 +1,5 @@ #pragma once -#include - #include #include diff --git a/src/Storages/System/StorageSystemDictionaries.cpp b/src/Storages/System/StorageSystemDictionaries.cpp index 8b528b4a298..a62a4a3f425 100644 --- a/src/Storages/System/StorageSystemDictionaries.cpp +++ b/src/Storages/System/StorageSystemDictionaries.cpp @@ -66,7 +66,7 @@ ColumnsDescription StorageSystemDictionaries::getColumnsDescription() {"database", std::make_shared(), "Name of the database containing the dictionary created by DDL query. Empty string for other dictionaries."}, {"name", std::make_shared(), "Dictionary name."}, {"uuid", std::make_shared(), "Dictionary UUID."}, - {"status", std::make_shared(getStatusEnumAllPossibleValues()), + {"status", std::make_shared(getExternalLoaderStatusEnumAllPossibleValues()), "Dictionary status. Possible values: " "NOT_LOADED — Dictionary was not loaded because it was not used, " "LOADED — Dictionary loaded successfully, " From 0b54b34b5b8339b163c17420e552aa1dbca2692d Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?J=C3=A1nos=20Benjamin=20Antal?= Date: Wed, 20 Mar 2024 16:27:32 +0000 Subject: [PATCH 035/117] Avoid instantiation of `CacheBase`'s ctor in header file --- src/IO/UncompressedCache.cpp | 11 +++++++++++ src/IO/UncompressedCache.h | 4 ++-- src/Storages/MarkCache.cpp | 11 +++++++++++ src/Storages/MarkCache.h | 5 ++--- 4 files changed, 26 insertions(+), 5 deletions(-) create mode 100644 src/IO/UncompressedCache.cpp create mode 100644 src/Storages/MarkCache.cpp diff --git a/src/IO/UncompressedCache.cpp b/src/IO/UncompressedCache.cpp new file mode 100644 index 00000000000..7309ef5d2f4 --- /dev/null +++ b/src/IO/UncompressedCache.cpp @@ -0,0 +1,11 @@ +#include + +namespace DB +{ +template class CacheBase; + +UncompressedCache::UncompressedCache(const String & cache_policy, size_t max_size_in_bytes, double size_ratio) + : Base(cache_policy, max_size_in_bytes, 0, size_ratio) +{ +} +} diff --git a/src/IO/UncompressedCache.h b/src/IO/UncompressedCache.h index 702804cdda3..aa515eec357 100644 --- a/src/IO/UncompressedCache.h +++ b/src/IO/UncompressedCache.h @@ -33,6 +33,7 @@ struct UncompressedSizeWeightFunction } }; +extern template class CacheBase; /** Cache of decompressed blocks for implementation of CachedCompressedReadBuffer. thread-safe. */ @@ -42,8 +43,7 @@ private: using Base = CacheBase; public: - UncompressedCache(const String & cache_policy, size_t max_size_in_bytes, double size_ratio) - : Base(cache_policy, max_size_in_bytes, 0, size_ratio) {} + UncompressedCache(const String & cache_policy, size_t max_size_in_bytes, double size_ratio); /// Calculate key from path to file and offset. static UInt128 hash(const String & path_to_file, size_t offset) diff --git a/src/Storages/MarkCache.cpp b/src/Storages/MarkCache.cpp new file mode 100644 index 00000000000..85f0aeac692 --- /dev/null +++ b/src/Storages/MarkCache.cpp @@ -0,0 +1,11 @@ +#include + +namespace DB +{ +template class CacheBase; + +MarkCache::MarkCache(const String & cache_policy, size_t max_size_in_bytes, double size_ratio) + : Base(cache_policy, max_size_in_bytes, 0, size_ratio) +{ +} +} diff --git a/src/Storages/MarkCache.h b/src/Storages/MarkCache.h index bcb6ae96c68..311fccdf810 100644 --- a/src/Storages/MarkCache.h +++ b/src/Storages/MarkCache.h @@ -31,7 +31,7 @@ struct MarksWeightFunction } }; - +extern template class CacheBase; /** Cache of 'marks' for StorageMergeTree. * Marks is an index structure that addresses ranges in column file, corresponding to ranges of primary key. */ @@ -41,8 +41,7 @@ private: using Base = CacheBase; public: - MarkCache(const String & cache_policy, size_t max_size_in_bytes, double size_ratio) - : Base(cache_policy, max_size_in_bytes, 0, size_ratio) {} + MarkCache(const String & cache_policy, size_t max_size_in_bytes, double size_ratio); /// Calculate key from path to file and offset. static UInt128 hash(const String & path_to_file) From adc964568c29e69131c37e600c867a98957c65da Mon Sep 17 00:00:00 2001 From: unashi Date: Thu, 21 Mar 2024 15:44:27 +0800 Subject: [PATCH 036/117] [update] Merge the on same disk and on another disk scenarios into cloneAndLoadDataPart; remove the try catch; instead of obtaining the copied destination hard disk through traversal, the destination hard disk is obtained through reservation. --- src/Storages/MergeTree/MergeTreeData.cpp | 166 ++------------------ src/Storages/MergeTree/MergeTreeData.h | 11 +- src/Storages/MergeTree/MutateTask.cpp | 2 +- src/Storages/StorageMergeTree.cpp | 43 ++--- src/Storages/StorageReplicatedMergeTree.cpp | 48 ++---- 5 files changed, 42 insertions(+), 228 deletions(-) diff --git a/src/Storages/MergeTree/MergeTreeData.cpp b/src/Storages/MergeTree/MergeTreeData.cpp index f8448be7268..aed2db16504 100644 --- a/src/Storages/MergeTree/MergeTreeData.cpp +++ b/src/Storages/MergeTree/MergeTreeData.cpp @@ -7063,7 +7063,7 @@ MergeTreeData & MergeTreeData::checkStructureAndGetMergeTreeData( return checkStructureAndGetMergeTreeData(*source_table, src_snapshot, my_snapshot); } -std::pair MergeTreeData::cloneAndLoadDataPartOnSameDisk( +std::pair MergeTreeData::cloneAndLoadDataPart( const MergeTreeData::DataPartPtr & src_part, const String & tmp_part_prefix, const MergeTreePartInfo & dst_part_info, @@ -7073,22 +7073,16 @@ std::pair MergeTreeData::cloneAn const WriteSettings & write_settings) { chassert(!isStaticStorage()); - - /// Check that the storage policy contains the disk where the src_part is located. - bool does_storage_policy_allow_same_disk = false; - for (const DiskPtr & disk : getStoragePolicy()->getDisks()) + bool on_same_disk = false; + for (const DiskPtr & disk : this->getStoragePolicy()->getDisks()) { if (disk->getName() == src_part->getDataPartStorage().getDiskName()) { - does_storage_policy_allow_same_disk = true; + on_same_disk = true; break; } } - if (!does_storage_policy_allow_same_disk) - throw Exception( - ErrorCodes::BAD_ARGUMENTS, - "Could not clone and load part {} because disk does not belong to storage policy", - quoteString(src_part->getDataPartStorage().getFullPath())); + String dst_part_name = src_part->getNewName(dst_part_info); String tmp_dst_part_name = tmp_part_prefix + dst_part_name; @@ -7103,11 +7097,12 @@ std::pair MergeTreeData::cloneAn MergeTreeData::MutableDataPartPtr src_flushed_tmp_part; String with_copy; - if (params.copy_instead_of_hardlink) + if (params.copy_instead_of_hardlink || !on_same_disk) with_copy = " (copying data)"; + std::shared_ptr dst_part_storage{}; - try + if (on_same_disk && !params.copy_instead_of_hardlink) { dst_part_storage = src_part_storage->freeze( relative_data_path, @@ -7117,34 +7112,13 @@ std::pair MergeTreeData::cloneAn /* save_metadata_callback= */ {}, params); } - catch (...) + else { - /// Hardlink fail. Try copy. - LOG_WARNING( - &Poco::Logger::get("MergeTreeData"), - "Hard link fail, try tp copy directly. to:{}, path:{}", - this->getRelativeDataPath(), - tmp_dst_part_name); - bool copy_successful = false; - for (const DiskPtr & disk : this->getStoragePolicy()->getDisks()) - { - try - { - auto reservation_space = src_part_storage->reserve(src_part->getBytesOnDisk()); - if (!reservation_space) - throw Exception(ErrorCodes::NOT_ENOUGH_SPACE, "Not enough space on disk."); - dst_part_storage = src_part_storage->clonePart( - this->getRelativeDataPath(), tmp_dst_part_name, disk, read_settings, write_settings, {}, {}); - copy_successful = true; - break; - } - catch (Exception & e) - { - LOG_TRACE(&Poco::Logger::get("MergeTreeData"), "Clone part on disk {} fail: {}", disk->getName(), e.what()); - } - } - if (!copy_successful) - LOG_ERROR(&Poco::Logger::get("MergeTreeData"), "Hard link fail, clone fail."); + auto reservation_on_dst = getStoragePolicy()->reserve(src_part->getBytesOnDisk()); + if (!reservation_on_dst) + throw Exception(ErrorCodes::NOT_ENOUGH_SPACE, "Not enough space on disk."); + dst_part_storage = src_part_storage->clonePart( + this->getRelativeDataPath(), tmp_dst_part_name, reservation_on_dst->getDisk(), read_settings, write_settings, {}, {}); } @@ -7168,117 +7142,7 @@ std::pair MergeTreeData::cloneAn .withPartFormatFromDisk() .build(); - if (!params.copy_instead_of_hardlink && params.hardlinked_files) - { - params.hardlinked_files->source_part_name = src_part->name; - params.hardlinked_files->source_table_shared_id = src_part->storage.getTableSharedID(); - - for (auto it = src_part->getDataPartStorage().iterate(); it->isValid(); it->next()) - { - if (!params.files_to_copy_instead_of_hardlinks.contains(it->name()) - && it->name() != IMergeTreeDataPart::DELETE_ON_DESTROY_MARKER_FILE_NAME_DEPRECATED - && it->name() != IMergeTreeDataPart::TXN_VERSION_METADATA_FILE_NAME) - { - params.hardlinked_files->hardlinks_from_source_part.insert(it->name()); - } - } - - auto projections = src_part->getProjectionParts(); - for (const auto & [name, projection_part] : projections) - { - const auto & projection_storage = projection_part->getDataPartStorage(); - for (auto it = projection_storage.iterate(); it->isValid(); it->next()) - { - auto file_name_with_projection_prefix = fs::path(projection_storage.getPartDirectory()) / it->name(); - if (!params.files_to_copy_instead_of_hardlinks.contains(file_name_with_projection_prefix) - && it->name() != IMergeTreeDataPart::DELETE_ON_DESTROY_MARKER_FILE_NAME_DEPRECATED - && it->name() != IMergeTreeDataPart::TXN_VERSION_METADATA_FILE_NAME) - { - params.hardlinked_files->hardlinks_from_source_part.insert(file_name_with_projection_prefix); - } - } - } - } - - /// We should write version metadata on part creation to distinguish it from parts that were created without transaction. - TransactionID tid = params.txn ? params.txn->tid : Tx::PrehistoricTID; - dst_data_part->version.setCreationTID(tid, nullptr); - dst_data_part->storeVersionMetadata(); - - dst_data_part->is_temp = true; - - dst_data_part->loadColumnsChecksumsIndexes(require_part_metadata, true); - dst_data_part->modification_time = dst_part_storage->getLastModified().epochTime(); - return std::make_pair(dst_data_part, std::move(temporary_directory_lock)); -} - -std::pair MergeTreeData::cloneAndLoadDataPartOnOtherDisk( - const MergeTreeData::DataPartPtr & src_part, - const String & tmp_part_prefix, - const MergeTreePartInfo & dst_part_info, - const StorageMetadataPtr & metadata_snapshot, - const IDataPartStorage::ClonePartParams & params, - const ReadSettings & read_settings, - const WriteSettings & write_settings) -{ - chassert(!isStaticStorage()); - - String dst_part_name = src_part->getNewName(dst_part_info); - String tmp_dst_part_name = tmp_part_prefix + dst_part_name; - auto temporary_directory_lock = getTemporaryPartDirectoryHolder(tmp_dst_part_name); - - auto reservation = src_part->getDataPartStorage().reserve(src_part->getBytesOnDisk()); - auto src_part_storage = src_part->getDataPartStoragePtr(); - - scope_guard src_flushed_tmp_dir_lock; - MergeTreeData::MutableDataPartPtr src_flushed_tmp_part; - - String with_copy; - if (params.copy_instead_of_hardlink) - with_copy = " (copying data)"; - - std::shared_ptr dst_part_storage{}; - bool copy_successful = false; - for (const DiskPtr & disk : this->getStoragePolicy()->getDisks()) - { - try - { - auto reservation_space = src_part_storage->reserve(src_part->getBytesOnDisk()); - if (!reservation_space) - throw Exception(ErrorCodes::NOT_ENOUGH_SPACE, "Not enough space on disk."); - dst_part_storage - = src_part_storage->clonePart(this->getRelativeDataPath(), tmp_dst_part_name, disk, read_settings, write_settings, {}, {}); - copy_successful = true; - break; - } - catch (...) - { - LOG_TRACE(&Poco::Logger::get("MergeTreeData"), "Clone part on disk {} fail", disk->getName()); - } - } - if (!copy_successful) - LOG_FATAL(&Poco::Logger::get("MergeTreeData"), "Hard link fail, clone fail."); - if (params.metadata_version_to_write.has_value()) - { - chassert(!params.keep_metadata_version); - auto out_metadata = dst_part_storage->writeFile(IMergeTreeDataPart::METADATA_VERSION_FILE_NAME, 4096, getContext()->getWriteSettings()); - writeText(metadata_snapshot->getMetadataVersion(), *out_metadata); - out_metadata->finalize(); - if (getSettings()->fsync_after_insert) - out_metadata->sync(); - } - - LOG_DEBUG(log, "Clone{} part {} to {}{}", - src_flushed_tmp_part ? " flushed" : "", - src_part_storage->getFullPath(), - std::string(fs::path(dst_part_storage->getFullRootPath()) / tmp_dst_part_name), - with_copy); - - auto dst_data_part = MergeTreeDataPartBuilder(*this, dst_part_name, dst_part_storage) - .withPartFormatFromDisk() - .build(); - - if (!params.copy_instead_of_hardlink && params.hardlinked_files) + if (on_same_disk && !params.copy_instead_of_hardlink && params.hardlinked_files) { params.hardlinked_files->source_part_name = src_part->name; params.hardlinked_files->source_table_shared_id = src_part->storage.getTableSharedID(); diff --git a/src/Storages/MergeTree/MergeTreeData.h b/src/Storages/MergeTree/MergeTreeData.h index 95bec1eeb2b..9b9e5f97f36 100644 --- a/src/Storages/MergeTree/MergeTreeData.h +++ b/src/Storages/MergeTree/MergeTreeData.h @@ -834,16 +834,7 @@ public: MergeTreeData & checkStructureAndGetMergeTreeData(const StoragePtr & source_table, const StorageMetadataPtr & src_snapshot, const StorageMetadataPtr & my_snapshot) const; MergeTreeData & checkStructureAndGetMergeTreeData(IStorage & source_table, const StorageMetadataPtr & src_snapshot, const StorageMetadataPtr & my_snapshot) const; - std::pair cloneAndLoadDataPartOnSameDisk( - const MergeTreeData::DataPartPtr & src_part, - const String & tmp_part_prefix, - const MergeTreePartInfo & dst_part_info, - const StorageMetadataPtr & metadata_snapshot, - const IDataPartStorage::ClonePartParams & params, - const ReadSettings & read_settings, - const WriteSettings & write_settings); - - std::pair cloneAndLoadDataPartOnOtherDisk( + std::pair cloneAndLoadDataPart( const MergeTreeData::DataPartPtr & src_part, const String & tmp_part_prefix, const MergeTreePartInfo & dst_part_info, diff --git a/src/Storages/MergeTree/MutateTask.cpp b/src/Storages/MergeTree/MutateTask.cpp index 150cc27c369..3ac103824bd 100644 --- a/src/Storages/MergeTree/MutateTask.cpp +++ b/src/Storages/MergeTree/MutateTask.cpp @@ -2097,7 +2097,7 @@ bool MutateTask::prepare() scope_guard lock; { - std::tie(part, lock) = ctx->data->cloneAndLoadDataPartOnSameDisk( + std::tie(part, lock) = ctx->data->cloneAndLoadDataPart( ctx->source_part, prefix, ctx->future_part->part_info, ctx->metadata_snapshot, clone_params, ctx->context->getReadSettings(), ctx->context->getWriteSettings()); part->getDataPartStorage().beginTransaction(); ctx->temporary_directory_lock = std::move(lock); diff --git a/src/Storages/StorageMergeTree.cpp b/src/Storages/StorageMergeTree.cpp index 89857156701..6adfc860cbc 100644 --- a/src/Storages/StorageMergeTree.cpp +++ b/src/Storages/StorageMergeTree.cpp @@ -2099,37 +2099,16 @@ void StorageMergeTree::replacePartitionFrom(const StoragePtr & source_table, con MergeTreePartInfo dst_part_info(partition_id, temp_index, temp_index, src_part->info.level); IDataPartStorage::ClonePartParams clone_params{.txn = local_context->getCurrentTransaction()}; - bool on_same_disk = false; - for (const DiskPtr & disk : this->getStoragePolicy()->getDisks()) - if (disk->getName() == src_part->getDataPartStorage().getDiskName()) - on_same_disk = true; - if (on_same_disk && !clone_params.copy_instead_of_hardlink) - { - - auto [dst_part, part_lock] = cloneAndLoadDataPartOnSameDisk( - src_part, - TMP_PREFIX, - dst_part_info, - my_metadata_snapshot, - clone_params, - local_context->getReadSettings(), - local_context->getWriteSettings()); - dst_parts.emplace_back(std::move(dst_part)); - dst_parts_locks.emplace_back(std::move(part_lock)); - } - else - { - auto [dst_part, part_lock] = cloneAndLoadDataPartOnOtherDisk( - src_part, - TMP_PREFIX, - dst_part_info, - my_metadata_snapshot, - clone_params, - local_context->getReadSettings(), - local_context->getWriteSettings()); - dst_parts.emplace_back(std::move(dst_part)); - dst_parts_locks.emplace_back(std::move(part_lock)); - } + auto [dst_part, part_lock] = cloneAndLoadDataPart( + src_part, + TMP_PREFIX, + dst_part_info, + my_metadata_snapshot, + clone_params, + local_context->getReadSettings(), + local_context->getWriteSettings()); + dst_parts.emplace_back(std::move(dst_part)); + dst_parts_locks.emplace_back(std::move(part_lock)); } /// ATTACH empty part set @@ -2231,7 +2210,7 @@ void StorageMergeTree::movePartitionToTable(const StoragePtr & dest_table, const .copy_instead_of_hardlink = getSettings()->always_use_copy_instead_of_hardlinks, }; - auto [dst_part, part_lock] = dest_table_storage->cloneAndLoadDataPartOnSameDisk( + auto [dst_part, part_lock] = dest_table_storage->cloneAndLoadDataPart( src_part, TMP_PREFIX, dst_part_info, diff --git a/src/Storages/StorageReplicatedMergeTree.cpp b/src/Storages/StorageReplicatedMergeTree.cpp index fdbd2d47fe4..cf9cc6f27e1 100644 --- a/src/Storages/StorageReplicatedMergeTree.cpp +++ b/src/Storages/StorageReplicatedMergeTree.cpp @@ -2751,7 +2751,7 @@ bool StorageReplicatedMergeTree::executeReplaceRange(LogEntry & entry) auto obtain_part = [&] (PartDescriptionPtr & part_desc) { - /// Fetches with zero-copy-replication are cheap, but cloneAndLoadDataPartOnSameDisk will do full copy. + /// Fetches with zero-copy-replication are cheap, but cloneAndLoadDataPart(OnSameDisk) will do full copy. /// It's okay to check the setting for current table and disk for the source table, because src and dst part are on the same disk. bool prefer_fetch_from_other_replica = !part_desc->replica.empty() && storage_settings_ptr->allow_remote_fs_zero_copy_replication && part_desc->src_table_part && part_desc->src_table_part->isStoredOnRemoteDiskWithZeroCopySupport(); @@ -2770,7 +2770,7 @@ bool StorageReplicatedMergeTree::executeReplaceRange(LogEntry & entry) .copy_instead_of_hardlink = storage_settings_ptr->always_use_copy_instead_of_hardlinks || ((our_zero_copy_enabled || source_zero_copy_enabled) && part_desc->src_table_part->isStoredOnRemoteDiskWithZeroCopySupport()), .metadata_version_to_write = metadata_snapshot->getMetadataVersion() }; - auto [res_part, temporary_part_lock] = cloneAndLoadDataPartOnSameDisk( + auto [res_part, temporary_part_lock] = cloneAndLoadDataPart( part_desc->src_table_part, TMP_PREFIX + "clone_", part_desc->new_part_info, @@ -4847,7 +4847,7 @@ bool StorageReplicatedMergeTree::fetchPart( .keep_metadata_version = true, }; - auto [cloned_part, lock] = cloneAndLoadDataPartOnSameDisk( + auto [cloned_part, lock] = cloneAndLoadDataPart( part_to_clone, "tmp_clone_", part_info, @@ -8023,36 +8023,16 @@ void StorageReplicatedMergeTree::replacePartitionFrom( .copy_instead_of_hardlink = storage_settings_ptr->always_use_copy_instead_of_hardlinks || (zero_copy_enabled && src_part->isStoredOnRemoteDiskWithZeroCopySupport()), .metadata_version_to_write = metadata_snapshot->getMetadataVersion() }; - bool on_same_disk = false; - for (const DiskPtr & disk : this->getStoragePolicy()->getDisks()) - if (disk->getName() == src_part->getDataPartStorage().getDiskName()) - on_same_disk = true; - if (on_same_disk && !clone_params.copy_instead_of_hardlink) - { - auto [dst_part, part_lock] = cloneAndLoadDataPartOnSameDisk( - src_part, - TMP_PREFIX, - dst_part_info, - metadata_snapshot, - clone_params, - query_context->getReadSettings(), - query_context->getWriteSettings()); - dst_parts.emplace_back(std::move(dst_part)); - dst_parts_locks.emplace_back(std::move(part_lock)); - } - else - { - auto [dst_part, part_lock] = cloneAndLoadDataPartOnOtherDisk( - src_part, - TMP_PREFIX, - dst_part_info, - metadata_snapshot, - clone_params, - query_context->getReadSettings(), - query_context->getWriteSettings()); - dst_parts.emplace_back(std::move(dst_part)); - dst_parts_locks.emplace_back(std::move(part_lock)); - } + auto [dst_part, part_lock] = cloneAndLoadDataPart( + src_part, + TMP_PREFIX, + dst_part_info, + metadata_snapshot, + clone_params, + query_context->getReadSettings(), + query_context->getWriteSettings()); + dst_parts.emplace_back(std::move(dst_part)); + dst_parts_locks.emplace_back(std::move(part_lock)); src_parts.emplace_back(src_part); ephemeral_locks.emplace_back(std::move(*lock)); block_id_paths.emplace_back(block_id_path); @@ -8291,7 +8271,7 @@ void StorageReplicatedMergeTree::movePartitionToTable(const StoragePtr & dest_ta .copy_instead_of_hardlink = storage_settings_ptr->always_use_copy_instead_of_hardlinks || (zero_copy_enabled && src_part->isStoredOnRemoteDiskWithZeroCopySupport()), .metadata_version_to_write = dest_metadata_snapshot->getMetadataVersion() }; - auto [dst_part, dst_part_lock] = dest_table_storage->cloneAndLoadDataPartOnSameDisk( + auto [dst_part, dst_part_lock] = dest_table_storage->cloneAndLoadDataPart( src_part, TMP_PREFIX, dst_part_info, From d1d72778021e9e13dc4a9a3bf4a34b53eb55ce7e Mon Sep 17 00:00:00 2001 From: Anton Popov Date: Thu, 21 Mar 2024 15:29:11 +0000 Subject: [PATCH 037/117] fix clang-tidy and test --- src/Common/ColumnsHashingImpl.h | 4 ++-- .../03009_consecutive_keys_nullable.reference | 6 +++--- .../0_stateless/03009_consecutive_keys_nullable.sql | 12 ++++++------ 3 files changed, 11 insertions(+), 11 deletions(-) diff --git a/src/Common/ColumnsHashingImpl.h b/src/Common/ColumnsHashingImpl.h index c8d62fa7e6b..6b3cc24d870 100644 --- a/src/Common/ColumnsHashingImpl.h +++ b/src/Common/ColumnsHashingImpl.h @@ -67,7 +67,7 @@ template struct LastElementCache; template struct LastElementCache : public LastElementCacheBase { - Value value; + Value value{}; bool is_null = false; template @@ -79,7 +79,7 @@ struct LastElementCache : public LastElementCacheBase template struct LastElementCache : public LastElementCacheBase { - Value value; + Value value{}; template bool check(const Key & key) const { return value.first == key; } diff --git a/tests/queries/0_stateless/03009_consecutive_keys_nullable.reference b/tests/queries/0_stateless/03009_consecutive_keys_nullable.reference index 1c8064290c6..e1b9e0cba62 100644 --- a/tests/queries/0_stateless/03009_consecutive_keys_nullable.reference +++ b/tests/queries/0_stateless/03009_consecutive_keys_nullable.reference @@ -1,13 +1,13 @@ -\N 1 1 1 2 0 \N 1 1 1 2 0 +\N 1 1 \N 3 3 1 3 0 -\N 1 1 1 2 0 -\N 2 2 +\N 1 1 1 1 0 +\N 2 2 t_nullable_keys_1 0 t_nullable_keys_2 0 t_nullable_keys_3 1 diff --git a/tests/queries/0_stateless/03009_consecutive_keys_nullable.sql b/tests/queries/0_stateless/03009_consecutive_keys_nullable.sql index ee2cb5a171f..7650cf14a49 100644 --- a/tests/queries/0_stateless/03009_consecutive_keys_nullable.sql +++ b/tests/queries/0_stateless/03009_consecutive_keys_nullable.sql @@ -2,7 +2,7 @@ DROP TABLE IF EXISTS t_nullable_keys_1; CREATE TABLE t_nullable_keys_1 (x Nullable(Int64)) ENGINE = Memory; INSERT INTO t_nullable_keys_1 VALUES (1), (1), (NULL); -SELECT x, count(), countIf(x IS NULL) FROM t_nullable_keys_1 GROUP BY x; +SELECT x, count(), countIf(x IS NULL) FROM t_nullable_keys_1 GROUP BY x ORDER BY x; DROP TABLE t_nullable_keys_1; @@ -10,7 +10,7 @@ DROP TABLE IF EXISTS t_nullable_keys_2; CREATE TABLE t_nullable_keys_2 (x Nullable(Int64)) ENGINE = Memory; INSERT INTO t_nullable_keys_2 VALUES (NULL), (1), (1); -SELECT x, count(), countIf(x IS NULL) FROM t_nullable_keys_2 GROUP BY x; +SELECT x, count(), countIf(x IS NULL) FROM t_nullable_keys_2 GROUP BY x ORDER BY x; DROP TABLE t_nullable_keys_2; @@ -18,7 +18,7 @@ DROP TABLE IF EXISTS t_nullable_keys_3; CREATE TABLE t_nullable_keys_3 (x Nullable(Int64)) ENGINE = Memory; INSERT INTO t_nullable_keys_3 VALUES (NULL), (NULL), (NULL); -SELECT x, count(), countIf(x IS NULL) FROM t_nullable_keys_3 GROUP BY x; +SELECT x, count(), countIf(x IS NULL) FROM t_nullable_keys_3 GROUP BY x ORDER BY x; DROP TABLE t_nullable_keys_3; @@ -26,7 +26,7 @@ DROP TABLE IF EXISTS t_nullable_keys_4; CREATE TABLE t_nullable_keys_4 (x Nullable(Int64)) ENGINE = Memory; INSERT INTO t_nullable_keys_4 VALUES (1), (1), (1); -SELECT x, count(), countIf(x IS NULL) FROM t_nullable_keys_4 GROUP BY x; +SELECT x, count(), countIf(x IS NULL) FROM t_nullable_keys_4 GROUP BY x ORDER BY x; DROP TABLE t_nullable_keys_4; @@ -34,7 +34,7 @@ DROP TABLE IF EXISTS t_nullable_keys_5; CREATE TABLE t_nullable_keys_5 (x Nullable(Int64)) ENGINE = Memory; INSERT INTO t_nullable_keys_5 VALUES (1), (NULL), (1); -SELECT x, count(), countIf(x IS NULL) FROM t_nullable_keys_5 GROUP BY x; +SELECT x, count(), countIf(x IS NULL) FROM t_nullable_keys_5 GROUP BY x ORDER BY x; DROP TABLE t_nullable_keys_5; @@ -42,7 +42,7 @@ DROP TABLE IF EXISTS t_nullable_keys_6; CREATE TABLE t_nullable_keys_6 (x Nullable(Int64)) ENGINE = Memory; INSERT INTO t_nullable_keys_6 VALUES (NULL), (1), (NULL); -SELECT x, count(), countIf(x IS NULL) FROM t_nullable_keys_6 GROUP BY x; +SELECT x, count(), countIf(x IS NULL) FROM t_nullable_keys_6 GROUP BY x ORDER BY x; DROP TABLE t_nullable_keys_6; From 89d80a4d65be42986f8d710c6f2ac5305491b80b Mon Sep 17 00:00:00 2001 From: Nikolai Kochetov Date: Thu, 21 Mar 2024 18:16:31 +0000 Subject: [PATCH 038/117] Redo fix. --- src/Analyzer/Passes/QueryAnalysisPass.cpp | 57 ++++++++++++++++++----- src/Functions/randConstant.cpp | 16 ++----- 2 files changed, 50 insertions(+), 23 deletions(-) diff --git a/src/Analyzer/Passes/QueryAnalysisPass.cpp b/src/Analyzer/Passes/QueryAnalysisPass.cpp index 2a46417ee98..834bc2e5813 100644 --- a/src/Analyzer/Passes/QueryAnalysisPass.cpp +++ b/src/Analyzer/Passes/QueryAnalysisPass.cpp @@ -799,7 +799,13 @@ struct IdentifierResolveScope /// Node hash to mask id map std::shared_ptr> projection_mask_map; - std::map rand_constant_cache; + struct ResolvedFunctionsCache + { + FunctionOverloadResolverPtr resolver; + std::map cache; + }; + + std::map functions_cache; [[maybe_unused]] const IdentifierResolveScope * getNearestQueryScope() const { @@ -925,6 +931,24 @@ struct IdentifierResolveScope } }; +IQueryTreeNode::Hash getHashForFunctionArguments(const ColumnsWithTypeAndName & arguments) +{ + SipHash hash; + for (const auto & arg : arguments) + { + auto type_name = arg.type->getName(); + hash.update(type_name.c_str(), type_name.size()); + + if (arg.column) + { + if (const auto * col_const = typeid_cast(arg.column.get())) + col_const->updateHashWithValue(0, hash); + } + } + + return getSipHash128AsPair(hash); +} + /** Visitor that extracts expression and function aliases from node and initialize scope tables with it. * Does not go into child lambdas and queries. @@ -5534,21 +5558,19 @@ ProjectionNames QueryAnalyzer::resolveFunction(QueryTreeNodePtr & node, Identifi FunctionOverloadResolverPtr function = UserDefinedExecutableFunctionFactory::instance().tryGet(function_name, scope.context, parameters); bool is_executable_udf = true; + IdentifierResolveScope::ResolvedFunctionsCache * function_cache = nullptr; + if (!function) { /// This is a hack to allow a query like `select randConstant(), randConstant(), randConstant()`. /// Function randConstant() would return the same value for the same arguments (in scope). - if (function_name == "randConstant") - { - auto hash = function_node_ptr->getTreeHash(); - auto & func = scope.rand_constant_cache[hash]; - if (!func) - func = FunctionFactory::instance().tryGet(function_name, scope.context); - function = func; - } - else - function = FunctionFactory::instance().tryGet(function_name, scope.context); + auto hash = function_node_ptr->getTreeHash(); + function_cache = &scope.functions_cache[hash]; + if (!function_cache->resolver) + function_cache->resolver = FunctionFactory::instance().tryGet(function_name, scope.context); + + function = function_cache->resolver; is_executable_udf = false; } @@ -5773,7 +5795,18 @@ ProjectionNames QueryAnalyzer::resolveFunction(QueryTreeNodePtr & node, Identifi try { - auto function_base = function->build(argument_columns); + FunctionBasePtr function_base; + if (function_cache) + { + auto args_hash = getHashForFunctionArguments(argument_columns); + auto & cached_function = function_cache->cache[args_hash]; + if (!cached_function) + cached_function = function->build(argument_columns); + + function_base = cached_function; + } + else + function_base = function->build(argument_columns); /// Do not constant fold get scalar functions bool disable_constant_folding = function_name == "__getScalar" || function_name == "shardNum" || diff --git a/src/Functions/randConstant.cpp b/src/Functions/randConstant.cpp index 5872ab8bb34..22ce6d88ea6 100644 --- a/src/Functions/randConstant.cpp +++ b/src/Functions/randConstant.cpp @@ -88,14 +88,6 @@ public: return std::make_unique>(); } - RandomConstantOverloadResolver() - { - typename ColumnVector::Container vec_to(1); - - TargetSpecific::Default::RandImpl::execute(reinterpret_cast(vec_to.data()), sizeof(ToType)); - value = vec_to[0]; - } - DataTypePtr getReturnTypeImpl(const DataTypes & data_types) const override { size_t number_of_arguments = data_types.size(); @@ -113,11 +105,13 @@ public: if (!arguments.empty()) argument_types.emplace_back(arguments.back().type); + typename ColumnVector::Container vec_to(1); + + TargetSpecific::Default::RandImpl::execute(reinterpret_cast(vec_to.data()), sizeof(ToType)); + ToType value = vec_to[0]; + return std::make_unique>(value, argument_types, return_type); } - -private: - ToType value; }; struct NameRandConstant { static constexpr auto name = "randConstant"; }; From 7ac41d7fc195004b4efccbd7891466fe956500e1 Mon Sep 17 00:00:00 2001 From: Michael Kolupaev Date: Thu, 21 Mar 2024 23:25:31 +0000 Subject: [PATCH 039/117] Fix segfault in SquashingTransform --- src/Interpreters/SquashingTransform.cpp | 25 ++++++++++++++++++++----- 1 file changed, 20 insertions(+), 5 deletions(-) diff --git a/src/Interpreters/SquashingTransform.cpp b/src/Interpreters/SquashingTransform.cpp index 4ed0dddc191..da608f5e7ce 100644 --- a/src/Interpreters/SquashingTransform.cpp +++ b/src/Interpreters/SquashingTransform.cpp @@ -89,13 +89,25 @@ void SquashingTransform::append(ReferenceType input_block) assert(blocksHaveEqualStructure(input_block, accumulated_block)); - for (size_t i = 0, size = accumulated_block.columns(); i < size; ++i) + try { - const auto source_column = input_block.getByPosition(i).column; + for (size_t i = 0, size = accumulated_block.columns(); i < size; ++i) + { + const auto source_column = input_block.getByPosition(i).column; - auto mutable_column = IColumn::mutate(std::move(accumulated_block.getByPosition(i).column)); - mutable_column->insertRangeFrom(*source_column, 0, source_column->size()); - accumulated_block.getByPosition(i).column = std::move(mutable_column); + auto mutable_column = IColumn::mutate(std::move(accumulated_block.getByPosition(i).column)); + mutable_column->insertRangeFrom(*source_column, 0, source_column->size()); + accumulated_block.getByPosition(i).column = std::move(mutable_column); + } + } + catch (...) + { + /// add() may be called again even after a previous add() threw an exception. + /// Keep accumulated_block in a valid state. + /// Seems ok to discard accumulated data because we're throwing an exception, which the caller will + /// hopefully interpret to mean "this block and all *previous* blocks are potentially lost". + accumulated_block.clear(); + throw; } } @@ -107,6 +119,9 @@ bool SquashingTransform::isEnoughSize(const Block & block) for (const auto & [column, type, name] : block) { + if (!column) + throw Exception(ErrorCodes::LOGICAL_ERROR, "Invalid column in block."); + if (!rows) rows = column->size(); else if (rows != column->size()) From 415ed79b1673c06ac6d50114ce33ca6e12c198ec Mon Sep 17 00:00:00 2001 From: Duc Canh Le Date: Fri, 15 Dec 2023 01:46:08 +0000 Subject: [PATCH 040/117] allow declaring enum in external table structure Signed-off-by: Duc Canh Le --- src/Core/ExternalTable.cpp | 73 ++++++++++++++++--- .../02935_external_table_enum_type.reference | 2 + .../02935_external_table_enum_type.sh | 10 +++ 3 files changed, 75 insertions(+), 10 deletions(-) create mode 100644 tests/queries/0_stateless/02935_external_table_enum_type.reference create mode 100755 tests/queries/0_stateless/02935_external_table_enum_type.sh diff --git a/src/Core/ExternalTable.cpp b/src/Core/ExternalTable.cpp index 58b705ca317..fa1e49d437c 100644 --- a/src/Core/ExternalTable.cpp +++ b/src/Core/ExternalTable.cpp @@ -17,6 +17,9 @@ #include #include +#include +#include +#include #include @@ -28,6 +31,18 @@ namespace ErrorCodes extern const int BAD_ARGUMENTS; } +/// Parsing a list of types with `,` as separator. For example, `Int, Enum('foo'=1,'bar'=2), Double` +/// Used in `parseStructureFromTypesField` +class ParserTypeList : public IParserBase +{ +protected: + const char * getName() const override { return "type pair list"; } + bool parseImpl(Pos & pos, ASTPtr & node, Expected & expected) override + { + return ParserList(std::make_unique(), std::make_unique(TokenType::Comma), false) + .parse(pos, node, expected); + } +}; ExternalTableDataPtr BaseExternalTable::getData(ContextPtr context) { @@ -55,23 +70,61 @@ void BaseExternalTable::clear() void BaseExternalTable::parseStructureFromStructureField(const std::string & argument) { - std::vector vals; - splitInto<' ', ','>(vals, argument, true); + /// First try to parse table structure with `ParserNameTypePairList`, this allows user to declare Enum types in the structure + ParserNameTypePairList parser; + const auto * pos = argument.data(); + String error; + ASTPtr columns_list_raw = tryParseQuery(parser, pos, pos+argument.size(), error, false, "", false, 0, 0); + bool parse_structure_with_parser = false; + if ((parse_structure_with_parser = columns_list_raw != nullptr)) + { + for (auto & child : columns_list_raw->children) + { + auto * column = child->as(); + if (column) + structure.emplace_back(column->name, column->type->getColumnNameWithoutAlias()); + else + { + structure.clear(); + parse_structure_with_parser = false; + break; + } + } + } - if (vals.size() % 2 != 0) - throw Exception(ErrorCodes::BAD_ARGUMENTS, "Odd number of attributes in section structure: {}", vals.size()); + if (!parse_structure_with_parser) + { + std::vector vals; + splitInto<' ', ','>(vals, argument, true); - for (size_t i = 0; i < vals.size(); i += 2) - structure.emplace_back(vals[i], vals[i + 1]); + if (vals.size() % 2 != 0) + throw Exception(ErrorCodes::BAD_ARGUMENTS, "Odd number of attributes in section structure: {}", vals.size()); + + for (size_t i = 0; i < vals.size(); i += 2) + structure.emplace_back(vals[i], vals[i + 1]); + } } void BaseExternalTable::parseStructureFromTypesField(const std::string & argument) { - std::vector vals; - splitInto<' ', ','>(vals, argument, true); + /// First try to parse table structure with `ParserTypeList`, this allows user to declare Enum types in the structure + ParserTypeList parser; + const auto * pos = argument.data(); + String error; + ASTPtr type_list_raw = tryParseQuery(parser, pos, pos+argument.size(), error, false, "", false, 0, 0); + if (type_list_raw != nullptr) + { + for (size_t i = 0; i < type_list_raw->children.size(); ++i) + structure.emplace_back("_" + toString(i + 1), type_list_raw->children[i]->getColumnNameWithoutAlias()); + } + else + { + std::vector vals; + splitInto<' ', ','>(vals, argument, true); - for (size_t i = 0; i < vals.size(); ++i) - structure.emplace_back("_" + toString(i + 1), vals[i]); + for (size_t i = 0; i < vals.size(); ++i) + structure.emplace_back("_" + toString(i + 1), vals[i]); + } } void BaseExternalTable::initSampleBlock() diff --git a/tests/queries/0_stateless/02935_external_table_enum_type.reference b/tests/queries/0_stateless/02935_external_table_enum_type.reference new file mode 100644 index 00000000000..1efccdbfc67 --- /dev/null +++ b/tests/queries/0_stateless/02935_external_table_enum_type.reference @@ -0,0 +1,2 @@ +foo 1 +bar 2 diff --git a/tests/queries/0_stateless/02935_external_table_enum_type.sh b/tests/queries/0_stateless/02935_external_table_enum_type.sh new file mode 100755 index 00000000000..ab4306a056e --- /dev/null +++ b/tests/queries/0_stateless/02935_external_table_enum_type.sh @@ -0,0 +1,10 @@ +#!/usr/bin/env bash + +CURDIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd) +# shellcheck source=../shell_config.sh +. "$CURDIR"/../shell_config.sh + +http_url="http://${CLICKHOUSE_HOST}:${CLICKHOUSE_PORT_HTTP}/?" + +curl -s "${http_url}temp_structure=x+Enum8('foo'%3D1,'bar'%3D2),y+Int" -F "$(printf 'temp='foo'\t1');filename=data1" -F "query=SELECT * FROM temp" +curl -s "${http_url}temp_types=Enum8('foo'%3D1,'bar'%3D2),Int" -F "$(printf 'temp='bar'\t2');filename=data1" -F "query=SELECT * FROM temp" From 3f0bba97a68016803c9c730a47872468a7c3d997 Mon Sep 17 00:00:00 2001 From: Duc Canh Le Date: Wed, 20 Dec 2023 03:14:47 +0000 Subject: [PATCH 041/117] fix shellcheck Signed-off-by: Duc Canh Le --- tests/queries/0_stateless/02935_external_table_enum_type.sh | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/tests/queries/0_stateless/02935_external_table_enum_type.sh b/tests/queries/0_stateless/02935_external_table_enum_type.sh index ab4306a056e..292fb647b27 100755 --- a/tests/queries/0_stateless/02935_external_table_enum_type.sh +++ b/tests/queries/0_stateless/02935_external_table_enum_type.sh @@ -6,5 +6,5 @@ CURDIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd) http_url="http://${CLICKHOUSE_HOST}:${CLICKHOUSE_PORT_HTTP}/?" -curl -s "${http_url}temp_structure=x+Enum8('foo'%3D1,'bar'%3D2),y+Int" -F "$(printf 'temp='foo'\t1');filename=data1" -F "query=SELECT * FROM temp" -curl -s "${http_url}temp_types=Enum8('foo'%3D1,'bar'%3D2),Int" -F "$(printf 'temp='bar'\t2');filename=data1" -F "query=SELECT * FROM temp" +curl -s "${http_url}temp_structure=x+Enum8('foo'%3D1,'bar'%3D2),y+Int" -F "$(printf 'temp='"foo"'\t1');filename=data1" -F "query=SELECT * FROM temp" +curl -s "${http_url}temp_types=Enum8('foo'%3D1,'bar'%3D2),Int" -F "$(printf 'temp='"bar"'\t2');filename=data1" -F "query=SELECT * FROM temp" From f45e3ba4327c09fd67746a15bdfa82ad79390dad Mon Sep 17 00:00:00 2001 From: Duc Canh Le Date: Mon, 4 Mar 2024 15:37:41 +0000 Subject: [PATCH 042/117] address review comments Signed-off-by: Duc Canh Le --- src/Core/ExternalTable.cpp | 57 ++++++------------- .../02935_external_table_enum_type.reference | 2 + .../02935_external_table_enum_type.sh | 2 + 3 files changed, 20 insertions(+), 41 deletions(-) diff --git a/src/Core/ExternalTable.cpp b/src/Core/ExternalTable.cpp index fa1e49d437c..23a181d8ea2 100644 --- a/src/Core/ExternalTable.cpp +++ b/src/Core/ExternalTable.cpp @@ -70,61 +70,36 @@ void BaseExternalTable::clear() void BaseExternalTable::parseStructureFromStructureField(const std::string & argument) { - /// First try to parse table structure with `ParserNameTypePairList`, this allows user to declare Enum types in the structure ParserNameTypePairList parser; const auto * pos = argument.data(); String error; - ASTPtr columns_list_raw = tryParseQuery(parser, pos, pos+argument.size(), error, false, "", false, 0, 0); - bool parse_structure_with_parser = false; - if ((parse_structure_with_parser = columns_list_raw != nullptr)) + ASTPtr columns_list_raw = tryParseQuery(parser, pos, pos + argument.size(), error, false, "", false, 0, 0); + + if (!columns_list_raw) + throw Exception(ErrorCodes::BAD_ARGUMENTS, "Error while parsing table structure: {}", error); + + for (auto & child : columns_list_raw->children) { - for (auto & child : columns_list_raw->children) - { - auto * column = child->as(); - if (column) - structure.emplace_back(column->name, column->type->getColumnNameWithoutAlias()); - else - { - structure.clear(); - parse_structure_with_parser = false; - break; - } - } - } - - if (!parse_structure_with_parser) - { - std::vector vals; - splitInto<' ', ','>(vals, argument, true); - - if (vals.size() % 2 != 0) - throw Exception(ErrorCodes::BAD_ARGUMENTS, "Odd number of attributes in section structure: {}", vals.size()); - - for (size_t i = 0; i < vals.size(); i += 2) - structure.emplace_back(vals[i], vals[i + 1]); + auto * column = child->as(); + if (column) + structure.emplace_back(column->name, column->type->getColumnNameWithoutAlias()); + else + throw Exception(ErrorCodes::BAD_ARGUMENTS, "Error while parsing table structure: expected column definition, got {}", child->formatForErrorMessage()); } } void BaseExternalTable::parseStructureFromTypesField(const std::string & argument) { - /// First try to parse table structure with `ParserTypeList`, this allows user to declare Enum types in the structure ParserTypeList parser; const auto * pos = argument.data(); String error; ASTPtr type_list_raw = tryParseQuery(parser, pos, pos+argument.size(), error, false, "", false, 0, 0); - if (type_list_raw != nullptr) - { - for (size_t i = 0; i < type_list_raw->children.size(); ++i) - structure.emplace_back("_" + toString(i + 1), type_list_raw->children[i]->getColumnNameWithoutAlias()); - } - else - { - std::vector vals; - splitInto<' ', ','>(vals, argument, true); - for (size_t i = 0; i < vals.size(); ++i) - structure.emplace_back("_" + toString(i + 1), vals[i]); - } + if (!type_list_raw) + throw Exception(ErrorCodes::BAD_ARGUMENTS, "Error while parsing table structure: {}", error); + + for (size_t i = 0; i < type_list_raw->children.size(); ++i) + structure.emplace_back("_" + toString(i + 1), type_list_raw->children[i]->getColumnNameWithoutAlias()); } void BaseExternalTable::initSampleBlock() diff --git a/tests/queries/0_stateless/02935_external_table_enum_type.reference b/tests/queries/0_stateless/02935_external_table_enum_type.reference index 1efccdbfc67..ed152e608b9 100644 --- a/tests/queries/0_stateless/02935_external_table_enum_type.reference +++ b/tests/queries/0_stateless/02935_external_table_enum_type.reference @@ -1,2 +1,4 @@ foo 1 bar 2 +foo 1 +bar 2 diff --git a/tests/queries/0_stateless/02935_external_table_enum_type.sh b/tests/queries/0_stateless/02935_external_table_enum_type.sh index 292fb647b27..61d5508e9f9 100755 --- a/tests/queries/0_stateless/02935_external_table_enum_type.sh +++ b/tests/queries/0_stateless/02935_external_table_enum_type.sh @@ -8,3 +8,5 @@ http_url="http://${CLICKHOUSE_HOST}:${CLICKHOUSE_PORT_HTTP}/?" curl -s "${http_url}temp_structure=x+Enum8('foo'%3D1,'bar'%3D2),y+Int" -F "$(printf 'temp='"foo"'\t1');filename=data1" -F "query=SELECT * FROM temp" curl -s "${http_url}temp_types=Enum8('foo'%3D1,'bar'%3D2),Int" -F "$(printf 'temp='"bar"'\t2');filename=data1" -F "query=SELECT * FROM temp" +echo -ne 'foo\t1' | ${CLICKHOUSE_CLIENT} --query="select * from tmp" --external --file=- --name=tmp --structure="x Enum8('foo'=1,'bar'=2),y Int" +echo -ne 'bar\t2' | ${CLICKHOUSE_CLIENT} --query="select * from tmp" --external --file=- --name=tmp --types="Enum8('foo'=1,'bar'=2),Int" From e39576917dfe2ba69c5181035c87182d2404aee5 Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Fri, 22 Mar 2024 00:40:28 +0100 Subject: [PATCH 043/117] Add a test --- .../03018_external_with_complex_data_types.reference | 1 + .../0_stateless/03018_external_with_complex_data_types.sh | 7 +++++++ 2 files changed, 8 insertions(+) create mode 100644 tests/queries/0_stateless/03018_external_with_complex_data_types.reference create mode 100755 tests/queries/0_stateless/03018_external_with_complex_data_types.sh diff --git a/tests/queries/0_stateless/03018_external_with_complex_data_types.reference b/tests/queries/0_stateless/03018_external_with_complex_data_types.reference new file mode 100644 index 00000000000..11277a62b06 --- /dev/null +++ b/tests/queries/0_stateless/03018_external_with_complex_data_types.reference @@ -0,0 +1 @@ +Hello world diff --git a/tests/queries/0_stateless/03018_external_with_complex_data_types.sh b/tests/queries/0_stateless/03018_external_with_complex_data_types.sh new file mode 100755 index 00000000000..fb3a22d9044 --- /dev/null +++ b/tests/queries/0_stateless/03018_external_with_complex_data_types.sh @@ -0,0 +1,7 @@ +#!/usr/bin/env bash + +CURDIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd) +# shellcheck source=../shell_config.sh +. "$CURDIR"/../shell_config.sh + +${CLICKHOUSE_CLIENT} --external --file <(echo "Hello, world") --name test --format CSV --structure "x Enum('Hello' = 1, 'world' = 2), y String" --query "SELECT * FROM test" From 828c555780c70a260029b13d07e61f2cdbf493a6 Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Fri, 22 Mar 2024 00:42:49 +0100 Subject: [PATCH 044/117] Fix build --- src/Core/ExternalTable.cpp | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/src/Core/ExternalTable.cpp b/src/Core/ExternalTable.cpp index 23a181d8ea2..e043a2f9492 100644 --- a/src/Core/ExternalTable.cpp +++ b/src/Core/ExternalTable.cpp @@ -73,7 +73,7 @@ void BaseExternalTable::parseStructureFromStructureField(const std::string & arg ParserNameTypePairList parser; const auto * pos = argument.data(); String error; - ASTPtr columns_list_raw = tryParseQuery(parser, pos, pos + argument.size(), error, false, "", false, 0, 0); + ASTPtr columns_list_raw = tryParseQuery(parser, pos, pos + argument.size(), error, false, "", false, DBMS_DEFAULT_MAX_QUERY_SIZE, DBMS_DEFAULT_MAX_PARSER_DEPTH, DBMS_DEFAULT_MAX_PARSER_BACKTRACKS, true); if (!columns_list_raw) throw Exception(ErrorCodes::BAD_ARGUMENTS, "Error while parsing table structure: {}", error); @@ -93,7 +93,7 @@ void BaseExternalTable::parseStructureFromTypesField(const std::string & argumen ParserTypeList parser; const auto * pos = argument.data(); String error; - ASTPtr type_list_raw = tryParseQuery(parser, pos, pos+argument.size(), error, false, "", false, 0, 0); + ASTPtr type_list_raw = tryParseQuery(parser, pos, pos+argument.size(), error, false, "", false, DBMS_DEFAULT_MAX_QUERY_SIZE, DBMS_DEFAULT_MAX_PARSER_DEPTH, DBMS_DEFAULT_MAX_PARSER_BACKTRACKS, true); if (!type_list_raw) throw Exception(ErrorCodes::BAD_ARGUMENTS, "Error while parsing table structure: {}", error); From 4675e709fc5a0e58bb1f75c2f94b4595bfe227e5 Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Fri, 22 Mar 2024 03:18:38 +0300 Subject: [PATCH 045/117] Unclog the CI --- docs/ru/index.md | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/docs/ru/index.md b/docs/ru/index.md index a9a666b18db..88a90fa4b21 100644 --- a/docs/ru/index.md +++ b/docs/ru/index.md @@ -96,4 +96,4 @@ ClickHouse — столбцовая система управления база В «обычных» СУБД этого не делается, так как не имеет смысла при выполнении простых запросов. Хотя есть исключения. Например, в MemSQL кодогенерация используется для уменьшения времени отклика при выполнении SQL-запросов. Для сравнения: в аналитических СУБД требуется оптимизация по пропускной способности (throughput, ГБ/с), а не времени отклика (latency, с). -Стоит заметить, что для эффективности по CPU требуется, чтобы язык запросов был декларативным (SQL, MDX) или хотя бы векторным (J, K). То есть необходимо, чтобы запрос содержал циклы только в неявном виде, открывая возможности для оптимизации. +Стоит заметить, что для эффективности по CPU требуется, чтобы язык запросов был декларативным (SQL, MDX) или хотя бы векторным (J, K, APL). То есть необходимо, чтобы запрос содержал циклы только в неявном виде, открывая возможности для оптимизации. From 5beabe071ce9cfbcfac28e4f0f21048588704faa Mon Sep 17 00:00:00 2001 From: Antonio Andelic Date: Fri, 22 Mar 2024 09:35:23 +0100 Subject: [PATCH 046/117] Destroy KeeperDispatcher first --- src/Coordination/Standalone/Context.cpp | 76 ++++++++++++++++++++++++- src/Interpreters/Context.cpp | 22 +++++-- 2 files changed, 90 insertions(+), 8 deletions(-) diff --git a/src/Coordination/Standalone/Context.cpp b/src/Coordination/Standalone/Context.cpp index 75b81187973..6652449340d 100644 --- a/src/Coordination/Standalone/Context.cpp +++ b/src/Coordination/Standalone/Context.cpp @@ -44,12 +44,78 @@ struct ContextSharedPart : boost::noncopyable : macros(std::make_unique()) {} + ~ContextSharedPart() + { + if (keeper_dispatcher) + { + try + { + keeper_dispatcher->shutdown(); + } + catch (...) + { + tryLogCurrentException(__PRETTY_FUNCTION__); + } + } + + /// Wait for thread pool for background reads and writes, + /// since it may use per-user MemoryTracker which will be destroyed here. + if (asynchronous_remote_fs_reader) + { + try + { + asynchronous_remote_fs_reader->wait(); + asynchronous_remote_fs_reader.reset(); + } + catch (...) + { + tryLogCurrentException(__PRETTY_FUNCTION__); + } + } + + if (asynchronous_local_fs_reader) + { + try + { + asynchronous_local_fs_reader->wait(); + asynchronous_local_fs_reader.reset(); + } + catch (...) + { + tryLogCurrentException(__PRETTY_FUNCTION__); + } + } + + if (synchronous_local_fs_reader) + { + try + { + synchronous_local_fs_reader->wait(); + synchronous_local_fs_reader.reset(); + } + catch (...) + { + tryLogCurrentException(__PRETTY_FUNCTION__); + } + } + + if (threadpool_writer) + { + try + { + threadpool_writer->wait(); + threadpool_writer.reset(); + } + catch (...) + { + tryLogCurrentException(__PRETTY_FUNCTION__); + } + } + } + /// For access of most of shared objects. mutable SharedMutex mutex; - mutable std::mutex keeper_dispatcher_mutex; - mutable std::shared_ptr keeper_dispatcher TSA_GUARDED_BY(keeper_dispatcher_mutex); - ServerSettings server_settings; String path; /// Path to the data directory, with a slash at the end. @@ -77,6 +143,10 @@ struct ContextSharedPart : boost::noncopyable mutable ThrottlerPtr local_read_throttler; /// A server-wide throttler for local IO reads mutable ThrottlerPtr local_write_throttler; /// A server-wide throttler for local IO writes + /// + mutable std::mutex keeper_dispatcher_mutex; + mutable std::shared_ptr keeper_dispatcher TSA_GUARDED_BY(keeper_dispatcher_mutex); + }; ContextData::ContextData() = default; diff --git a/src/Interpreters/Context.cpp b/src/Interpreters/Context.cpp index 038b9712b0c..9e7afcaaac1 100644 --- a/src/Interpreters/Context.cpp +++ b/src/Interpreters/Context.cpp @@ -221,10 +221,6 @@ struct ContextSharedPart : boost::noncopyable ConfigurationPtr sensitive_data_masker_config; -#if USE_NURAFT - mutable std::mutex keeper_dispatcher_mutex; - mutable std::shared_ptr keeper_dispatcher TSA_GUARDED_BY(keeper_dispatcher_mutex); -#endif mutable std::mutex auxiliary_zookeepers_mutex; mutable std::map auxiliary_zookeepers TSA_GUARDED_BY(auxiliary_zookeepers_mutex); /// Map for auxiliary ZooKeeper clients. ConfigurationPtr auxiliary_zookeepers_config TSA_GUARDED_BY(auxiliary_zookeepers_mutex); /// Stores auxiliary zookeepers configs @@ -417,6 +413,11 @@ struct ContextSharedPart : boost::noncopyable bool is_server_completely_started TSA_GUARDED_BY(mutex) = false; +#if USE_NURAFT + mutable std::mutex keeper_dispatcher_mutex; + mutable std::shared_ptr keeper_dispatcher TSA_GUARDED_BY(keeper_dispatcher_mutex); +#endif + ContextSharedPart() : access_control(std::make_unique()) , global_overcommit_tracker(&process_list) @@ -432,9 +433,20 @@ struct ContextSharedPart : boost::noncopyable } } - ~ContextSharedPart() { + if (keeper_dispatcher) + { + try + { + keeper_dispatcher->shutdown(); + } + catch (...) + { + tryLogCurrentException(__PRETTY_FUNCTION__); + } + } + /// Wait for thread pool for background reads and writes, /// since it may use per-user MemoryTracker which will be destroyed here. if (asynchronous_remote_fs_reader) From 409d2edba4d8781ac1c9e061a6646bfb8e160d41 Mon Sep 17 00:00:00 2001 From: Antonio Andelic Date: Fri, 22 Mar 2024 11:15:09 +0100 Subject: [PATCH 047/117] Fix flaky 03014_async_with_dedup_part_log_rmt --- .../0_stateless/03014_async_with_dedup_part_log_rmt.sql | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/tests/queries/0_stateless/03014_async_with_dedup_part_log_rmt.sql b/tests/queries/0_stateless/03014_async_with_dedup_part_log_rmt.sql index e14c1635853..6b441e951e1 100644 --- a/tests/queries/0_stateless/03014_async_with_dedup_part_log_rmt.sql +++ b/tests/queries/0_stateless/03014_async_with_dedup_part_log_rmt.sql @@ -10,7 +10,7 @@ INSERT INTO 03014_async_with_dedup_part_log VALUES (2); SYSTEM FLUSH LOGS; SELECT error, count() FROM system.part_log -WHERE table = '03014_async_with_dedup_part_log' and database = currentDatabase() +WHERE table = '03014_async_with_dedup_part_log' AND database = currentDatabase() AND event_type = 'NewPart' GROUP BY error ORDER BY error; @@ -19,6 +19,6 @@ INSERT INTO 03014_async_with_dedup_part_log VALUES (2); SYSTEM FLUSH LOGS; SELECT error, count() FROM system.part_log -WHERE table = '03014_async_with_dedup_part_log' and database = currentDatabase() +WHERE table = '03014_async_with_dedup_part_log' AND database = currentDatabase() AND event_type = 'NewPart' GROUP BY error ORDER BY error; From ec134d2642bca10a9f147006858af4656b008d45 Mon Sep 17 00:00:00 2001 From: Nikolai Kochetov Date: Fri, 22 Mar 2024 10:51:01 +0000 Subject: [PATCH 048/117] Another attempt. --- src/Analyzer/Passes/QueryAnalysisPass.cpp | 23 ++--------------------- 1 file changed, 2 insertions(+), 21 deletions(-) diff --git a/src/Analyzer/Passes/QueryAnalysisPass.cpp b/src/Analyzer/Passes/QueryAnalysisPass.cpp index 834bc2e5813..6082012445c 100644 --- a/src/Analyzer/Passes/QueryAnalysisPass.cpp +++ b/src/Analyzer/Passes/QueryAnalysisPass.cpp @@ -802,7 +802,7 @@ struct IdentifierResolveScope struct ResolvedFunctionsCache { FunctionOverloadResolverPtr resolver; - std::map cache; + FunctionBasePtr function_base; }; std::map functions_cache; @@ -931,24 +931,6 @@ struct IdentifierResolveScope } }; -IQueryTreeNode::Hash getHashForFunctionArguments(const ColumnsWithTypeAndName & arguments) -{ - SipHash hash; - for (const auto & arg : arguments) - { - auto type_name = arg.type->getName(); - hash.update(type_name.c_str(), type_name.size()); - - if (arg.column) - { - if (const auto * col_const = typeid_cast(arg.column.get())) - col_const->updateHashWithValue(0, hash); - } - } - - return getSipHash128AsPair(hash); -} - /** Visitor that extracts expression and function aliases from node and initialize scope tables with it. * Does not go into child lambdas and queries. @@ -5798,8 +5780,7 @@ ProjectionNames QueryAnalyzer::resolveFunction(QueryTreeNodePtr & node, Identifi FunctionBasePtr function_base; if (function_cache) { - auto args_hash = getHashForFunctionArguments(argument_columns); - auto & cached_function = function_cache->cache[args_hash]; + auto & cached_function = function_cache->function_base; if (!cached_function) cached_function = function->build(argument_columns); From 5c082a8cc52dfeeadeec89bcc38ef17d10c4855b Mon Sep 17 00:00:00 2001 From: yariks5s Date: Fri, 22 Mar 2024 13:19:18 +0000 Subject: [PATCH 049/117] init --- src/Functions/currentUser.cpp | 1 + 1 file changed, 1 insertion(+) diff --git a/src/Functions/currentUser.cpp b/src/Functions/currentUser.cpp index 67b5d7626bf..1679c56a929 100644 --- a/src/Functions/currentUser.cpp +++ b/src/Functions/currentUser.cpp @@ -55,6 +55,7 @@ REGISTER_FUNCTION(CurrentUser) { factory.registerFunction(); factory.registerAlias("user", FunctionCurrentUser::name, FunctionFactory::CaseInsensitive); + factory.registerAlias("current_user", FunctionCurrentUser::name, FunctionFactory::CaseInsensitive); } } From 7be46e117cf7535209d16e536f9accc7e80d0f9a Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?Ra=C3=BAl=20Mar=C3=ADn?= Date: Fri, 22 Mar 2024 14:57:34 +0100 Subject: [PATCH 050/117] Remove another batch of bad templates --- src/Functions/FunctionsConversion.cpp | 150 +++++++++++++++----------- 1 file changed, 85 insertions(+), 65 deletions(-) diff --git a/src/Functions/FunctionsConversion.cpp b/src/Functions/FunctionsConversion.cpp index 9262389e593..16a6e15cd10 100644 --- a/src/Functions/FunctionsConversion.cpp +++ b/src/Functions/FunctionsConversion.cpp @@ -1921,6 +1921,19 @@ struct NameParseDateTimeBestEffort; struct NameParseDateTimeBestEffortOrZero; struct NameParseDateTimeBestEffortOrNull; +template +constexpr bool mightBeDateTime() +{ + if constexpr (std::is_same_v) + return true; + else if constexpr ( + std::is_same_v || std::is_same_v + || std::is_same_v || std::is_same_v) + return true; + + return false; +} + template inline bool isDateTime64(const ColumnsWithTypeAndName & arguments) { @@ -2190,7 +2203,6 @@ private: result_column = ConvertImpl::execute(arguments, result_type, input_rows_count, from_string_tag, scale); break; } - } else if constexpr (IsDataTypeDateOrDateTime && std::is_same_v) { @@ -2208,12 +2220,25 @@ private: break; } } + else if constexpr ((IsDataTypeNumber + || IsDataTypeDateOrDateTime)&&IsDataTypeDateOrDateTime) + { #define GENERATE_OVERFLOW_MODE_CASE(OVERFLOW_MODE) \ - case FormatSettings::DateTimeOverflowBehavior::OVERFLOW_MODE: \ - result_column = ConvertImpl::execute( \ - arguments, result_type, input_rows_count, from_string_tag); \ - break; + case FormatSettings::DateTimeOverflowBehavior::OVERFLOW_MODE: \ + result_column = ConvertImpl::execute( \ + arguments, result_type, input_rows_count, from_string_tag); \ + break; + static_assert( + !(std::is_same_v> && std::is_same_v>)); + switch (date_time_overflow_behavior) + { + GENERATE_OVERFLOW_MODE_CASE(Throw) + GENERATE_OVERFLOW_MODE_CASE(Ignore) + GENERATE_OVERFLOW_MODE_CASE(Saturate) + } +#undef GENERATE_OVERFLOW_MODE_CASE + } else if constexpr (IsDataTypeDecimalOrNumber && IsDataTypeDecimalOrNumber) { using LeftT = typename LeftDataType::FieldType; @@ -2232,44 +2257,36 @@ private: } else { - switch (date_time_overflow_behavior) - { - GENERATE_OVERFLOW_MODE_CASE(Throw) - GENERATE_OVERFLOW_MODE_CASE(Ignore) - GENERATE_OVERFLOW_MODE_CASE(Saturate) - } + result_column = ConvertImpl::execute( + arguments, result_type, input_rows_count, from_string_tag); } } - else if constexpr ((IsDataTypeNumber || IsDataTypeDateOrDateTime) - && IsDataTypeDateOrDateTime) - { - switch (date_time_overflow_behavior) - { - GENERATE_OVERFLOW_MODE_CASE(Throw) - GENERATE_OVERFLOW_MODE_CASE(Ignore) - GENERATE_OVERFLOW_MODE_CASE(Saturate) - } - } -#undef GENERATE_OVERFLOW_MODE_CASE else result_column = ConvertImpl::execute(arguments, result_type, input_rows_count, from_string_tag); return true; }; - if (isDateTime64(arguments)) + if constexpr (mightBeDateTime()) { - /// For toDateTime('xxxx-xx-xx xx:xx:xx.00', 2[, 'timezone']) we need to it convert to DateTime64 - const ColumnWithTypeAndName & scale_column = arguments[1]; - UInt32 scale = extractToDecimalScale(scale_column); - - if (to_datetime64 || scale != 0) /// When scale = 0, the data type is DateTime otherwise the data type is DateTime64 + if (isDateTime64(arguments)) { - if (!callOnIndexAndDataType(from_type->getTypeId(), call, BehaviourOnErrorFromString::ConvertDefaultBehaviorTag)) - throw Exception(ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT, "Illegal type {} of argument of function {}", - arguments[0].type->getName(), getName()); + /// For toDateTime('xxxx-xx-xx xx:xx:xx.00', 2[, 'timezone']) we need to it convert to DateTime64 + const ColumnWithTypeAndName & scale_column = arguments[1]; + UInt32 scale = extractToDecimalScale(scale_column); - return result_column; + if (to_datetime64 || scale != 0) /// When scale = 0, the data type is DateTime otherwise the data type is DateTime64 + { + if (!callOnIndexAndDataType( + from_type->getTypeId(), call, BehaviourOnErrorFromString::ConvertDefaultBehaviorTag)) + throw Exception( + ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT, + "Illegal type {} of argument of function {}", + arguments[0].type->getName(), + getName()); + + return result_column; + } } } @@ -2468,19 +2485,27 @@ public: result_column = executeInternal(arguments, result_type, input_rows_count, assert_cast(*removeNullable(result_type)).getScale()); } - else if (isDateTime64(arguments)) + else if constexpr (mightBeDateTime()) { - UInt64 scale = to_datetime64 ? DataTypeDateTime64::default_scale : 0; - if (arguments.size() > 1) - scale = extractToDecimalScale(arguments[1]); - - if (scale == 0) + if (isDateTime64(arguments)) { - result_column = executeInternal(arguments, result_type, input_rows_count, 0); + UInt64 scale = to_datetime64 ? DataTypeDateTime64::default_scale : 0; + if (arguments.size() > 1) + scale = extractToDecimalScale(arguments[1]); + + if (scale == 0) + { + result_column = executeInternal(arguments, result_type, input_rows_count, 0); + } + else + { + result_column + = executeInternal(arguments, result_type, input_rows_count, static_cast(scale)); + } } else { - result_column = executeInternal(arguments, result_type, input_rows_count, static_cast(scale)); + result_column = executeInternal(arguments, result_type, input_rows_count, 0); } } else @@ -3173,13 +3198,14 @@ private: if constexpr (IsDataTypeNumber) { - if constexpr (IsDataTypeNumber) + if constexpr (IsDataTypeDateOrDateTime) { #define GENERATE_OVERFLOW_MODE_CASE(OVERFLOW_MODE, ADDITIONS) \ - case FormatSettings::DateTimeOverflowBehavior::OVERFLOW_MODE: \ - result_column = ConvertImpl::execute( \ - arguments, result_type, input_rows_count, BehaviourOnErrorFromString::ConvertDefaultBehaviorTag, ADDITIONS()); \ - break; + case FormatSettings::DateTimeOverflowBehavior::OVERFLOW_MODE: \ + result_column \ + = ConvertImpl:: \ + execute(arguments, result_type, input_rows_count, BehaviourOnErrorFromString::ConvertDefaultBehaviorTag, ADDITIONS()); \ + break; if (wrapper_cast_type == CastType::accurate) { switch (date_time_overflow_behavior) @@ -3202,33 +3228,27 @@ private: return true; } - - if constexpr (std::is_same_v || std::is_same_v) + else if constexpr (IsDataTypeNumber) { -#define GENERATE_OVERFLOW_MODE_CASE(OVERFLOW_MODE, ADDITIONS) \ - case FormatSettings::DateTimeOverflowBehavior::OVERFLOW_MODE: \ - result_column = ConvertImpl::template execute( \ -arguments, result_type, input_rows_count, BehaviourOnErrorFromString::ConvertDefaultBehaviorTag); \ - break; if (wrapper_cast_type == CastType::accurate) { - switch (date_time_overflow_behavior) - { - GENERATE_OVERFLOW_MODE_CASE(Throw, DateTimeAccurateConvertStrategyAdditions) - GENERATE_OVERFLOW_MODE_CASE(Ignore, DateTimeAccurateConvertStrategyAdditions) - GENERATE_OVERFLOW_MODE_CASE(Saturate, DateTimeAccurateConvertStrategyAdditions) - } + result_column = ConvertImpl::execute( + arguments, + result_type, + input_rows_count, + BehaviourOnErrorFromString::ConvertDefaultBehaviorTag, + AccurateConvertStrategyAdditions()); } else { - switch (date_time_overflow_behavior) - { - GENERATE_OVERFLOW_MODE_CASE(Throw, DateTimeAccurateOrNullConvertStrategyAdditions) - GENERATE_OVERFLOW_MODE_CASE(Ignore, DateTimeAccurateOrNullConvertStrategyAdditions) - GENERATE_OVERFLOW_MODE_CASE(Saturate, DateTimeAccurateOrNullConvertStrategyAdditions) - } + result_column = ConvertImpl::execute( + arguments, + result_type, + input_rows_count, + BehaviourOnErrorFromString::ConvertDefaultBehaviorTag, + AccurateOrNullConvertStrategyAdditions()); } -#undef GENERATE_OVERFLOW_MODE_CASE + return true; } } From 6a681e074fbf981aa4e7002a3b17c7d4bba8bf31 Mon Sep 17 00:00:00 2001 From: Antonio Andelic Date: Fri, 22 Mar 2024 15:17:28 +0100 Subject: [PATCH 051/117] Fix build --- src/Coordination/Standalone/Context.cpp | 2 +- src/Interpreters/Context.cpp | 2 ++ 2 files changed, 3 insertions(+), 1 deletion(-) diff --git a/src/Coordination/Standalone/Context.cpp b/src/Coordination/Standalone/Context.cpp index 6652449340d..1095a11566f 100644 --- a/src/Coordination/Standalone/Context.cpp +++ b/src/Coordination/Standalone/Context.cpp @@ -143,7 +143,7 @@ struct ContextSharedPart : boost::noncopyable mutable ThrottlerPtr local_read_throttler; /// A server-wide throttler for local IO reads mutable ThrottlerPtr local_write_throttler; /// A server-wide throttler for local IO writes - /// + mutable std::mutex keeper_dispatcher_mutex; mutable std::shared_ptr keeper_dispatcher TSA_GUARDED_BY(keeper_dispatcher_mutex); diff --git a/src/Interpreters/Context.cpp b/src/Interpreters/Context.cpp index 9e7afcaaac1..1167b5d3472 100644 --- a/src/Interpreters/Context.cpp +++ b/src/Interpreters/Context.cpp @@ -435,6 +435,7 @@ struct ContextSharedPart : boost::noncopyable ~ContextSharedPart() { +#if USE_NURAFT if (keeper_dispatcher) { try @@ -446,6 +447,7 @@ struct ContextSharedPart : boost::noncopyable tryLogCurrentException(__PRETTY_FUNCTION__); } } +#endif /// Wait for thread pool for background reads and writes, /// since it may use per-user MemoryTracker which will be destroyed here. From de55ec24cc48503e8fe94dd795266e225b7041ce Mon Sep 17 00:00:00 2001 From: yariks5s Date: Fri, 22 Mar 2024 14:29:44 +0000 Subject: [PATCH 052/117] tests+docs --- docs/en/sql-reference/functions/other-functions.md | 2 +- .../0_stateless/00990_function_current_user.reference | 4 ++++ tests/queries/0_stateless/00990_function_current_user.sql | 5 +++++ 3 files changed, 10 insertions(+), 1 deletion(-) diff --git a/docs/en/sql-reference/functions/other-functions.md b/docs/en/sql-reference/functions/other-functions.md index e7fca31483a..0ca404274ec 100644 --- a/docs/en/sql-reference/functions/other-functions.md +++ b/docs/en/sql-reference/functions/other-functions.md @@ -405,7 +405,7 @@ Returns the name of the current user. In case of a distributed query, the name o SELECT currentUser(); ``` -Alias: `user()`, `USER()`. +Aliases: `user()`, `USER()`, `current_user()`. Aliases are case insensitive. **Returned values** diff --git a/tests/queries/0_stateless/00990_function_current_user.reference b/tests/queries/0_stateless/00990_function_current_user.reference index f1f321b1ecd..227c7c3b562 100644 --- a/tests/queries/0_stateless/00990_function_current_user.reference +++ b/tests/queries/0_stateless/00990_function_current_user.reference @@ -2,3 +2,7 @@ 1 1 1 1 +1 +1 +1 1 1 +1 diff --git a/tests/queries/0_stateless/00990_function_current_user.sql b/tests/queries/0_stateless/00990_function_current_user.sql index 38bd815ecef..c2d946e4185 100644 --- a/tests/queries/0_stateless/00990_function_current_user.sql +++ b/tests/queries/0_stateless/00990_function_current_user.sql @@ -3,3 +3,8 @@ select currentUser() IS NOT NULL; select length(currentUser()) > 0; select currentUser() = user(), currentUser() = USER(); select currentUser() = initial_user from system.processes where query like '%$!@#%'; + +select current_user() IS NOT NULL; +select length(current_user()) > 0; +select current_user() = user(), current_user() = USER(), current_user() = currentUser(); +select current_user() = initial_user from system.processes where query like '%$!@#%'; From 2377b1eb293c643de561d73c37524c249a8167cf Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?Ra=C3=BAl=20Mar=C3=ADn?= Date: Fri, 22 Mar 2024 17:55:50 +0100 Subject: [PATCH 053/117] FML --- src/Functions/FunctionsConversion.cpp | 14 ++++++-------- 1 file changed, 6 insertions(+), 8 deletions(-) diff --git a/src/Functions/FunctionsConversion.cpp b/src/Functions/FunctionsConversion.cpp index 16a6e15cd10..7049ca44110 100644 --- a/src/Functions/FunctionsConversion.cpp +++ b/src/Functions/FunctionsConversion.cpp @@ -2228,8 +2228,6 @@ private: result_column = ConvertImpl::execute( \ arguments, result_type, input_rows_count, from_string_tag); \ break; - static_assert( - !(std::is_same_v> && std::is_same_v>)); switch (date_time_overflow_behavior) { GENERATE_OVERFLOW_MODE_CASE(Throw) @@ -3210,18 +3208,18 @@ private: { switch (date_time_overflow_behavior) { - GENERATE_OVERFLOW_MODE_CASE(Throw, AccurateConvertStrategyAdditions) - GENERATE_OVERFLOW_MODE_CASE(Ignore, AccurateConvertStrategyAdditions) - GENERATE_OVERFLOW_MODE_CASE(Saturate, AccurateConvertStrategyAdditions) + GENERATE_OVERFLOW_MODE_CASE(Throw, DateTimeAccurateConvertStrategyAdditions) + GENERATE_OVERFLOW_MODE_CASE(Ignore, DateTimeAccurateConvertStrategyAdditions) + GENERATE_OVERFLOW_MODE_CASE(Saturate, DateTimeAccurateConvertStrategyAdditions) } } else { switch (date_time_overflow_behavior) { - GENERATE_OVERFLOW_MODE_CASE(Throw, AccurateOrNullConvertStrategyAdditions) - GENERATE_OVERFLOW_MODE_CASE(Ignore, AccurateOrNullConvertStrategyAdditions) - GENERATE_OVERFLOW_MODE_CASE(Saturate, AccurateOrNullConvertStrategyAdditions) + GENERATE_OVERFLOW_MODE_CASE(Throw, DateTimeAccurateOrNullConvertStrategyAdditions) + GENERATE_OVERFLOW_MODE_CASE(Ignore, DateTimeAccurateOrNullConvertStrategyAdditions) + GENERATE_OVERFLOW_MODE_CASE(Saturate, DateTimeAccurateOrNullConvertStrategyAdditions) } } #undef GENERATE_OVERFLOW_MODE_CASE From 2736b4ef6436b86d82fb8439c82b1b0fcbab1213 Mon Sep 17 00:00:00 2001 From: Dani Pozo Date: Fri, 22 Mar 2024 17:58:07 +0100 Subject: [PATCH 054/117] Use managed identity for backups IO in Azure Blob Storage Also adds option to prevent ClickHouse from trying to create a non-existing container, which requires a role assignment at the storage account level. --- docs/en/operations/backup.md | 1 + src/Backups/BackupFactory.h | 1 + src/Backups/BackupIO_AzureBlobStorage.cpp | 5 +- src/Backups/BackupIO_AzureBlobStorage.h | 2 +- src/Backups/BackupSettings.cpp | 1 + src/Backups/BackupSettings.h | 3 + src/Backups/BackupsWorker.cpp | 1 + .../registerBackupEngineAzureBlobStorage.cpp | 5 +- src/Storages/StorageAzureBlob.cpp | 56 ++++++++++++------- src/Storages/StorageAzureBlob.h | 2 +- 10 files changed, 50 insertions(+), 27 deletions(-) diff --git a/docs/en/operations/backup.md b/docs/en/operations/backup.md index 8639af468c2..2ba50b39934 100644 --- a/docs/en/operations/backup.md +++ b/docs/en/operations/backup.md @@ -87,6 +87,7 @@ The BACKUP and RESTORE statements take a list of DATABASE and TABLE names, a des - `structure_only`: if enabled, allows to only backup or restore the CREATE statements without the data of tables - `storage_policy`: storage policy for the tables being restored. See [Using Multiple Block Devices for Data Storage](../engines/table-engines/mergetree-family/mergetree.md#table_engine-mergetree-multiple-volumes). This setting is only applicable to the `RESTORE` command. The specified storage policy applies only to tables with an engine from the `MergeTree` family. - `s3_storage_class`: the storage class used for S3 backup. For example, `STANDARD` + - `azure_attempt_to_create_container`: when using Azure Blob Storage, whether the specified container will try to be created if it doesn't exist. Default: true. ### Usage examples diff --git a/src/Backups/BackupFactory.h b/src/Backups/BackupFactory.h index f0c64486da4..4e752508577 100644 --- a/src/Backups/BackupFactory.h +++ b/src/Backups/BackupFactory.h @@ -40,6 +40,7 @@ public: bool deduplicate_files = true; bool allow_s3_native_copy = true; bool use_same_s3_credentials_for_base_backup = false; + bool azure_attempt_to_create_container = true; ReadSettings read_settings; WriteSettings write_settings; }; diff --git a/src/Backups/BackupIO_AzureBlobStorage.cpp b/src/Backups/BackupIO_AzureBlobStorage.cpp index b3b92323109..7f3ee6b42a7 100644 --- a/src/Backups/BackupIO_AzureBlobStorage.cpp +++ b/src/Backups/BackupIO_AzureBlobStorage.cpp @@ -140,12 +140,13 @@ BackupWriterAzureBlobStorage::BackupWriterAzureBlobStorage( StorageAzureBlob::Configuration configuration_, const ReadSettings & read_settings_, const WriteSettings & write_settings_, - const ContextPtr & context_) + const ContextPtr & context_, + bool attempt_to_create_container) : BackupWriterDefault(read_settings_, write_settings_, getLogger("BackupWriterAzureBlobStorage")) , data_source_description{DataSourceType::ObjectStorage, ObjectStorageType::Azure, MetadataStorageType::None, configuration_.container, false, false} , configuration(configuration_) { - auto client_ptr = StorageAzureBlob::createClient(configuration, /* is_read_only */ false); + auto client_ptr = StorageAzureBlob::createClient(configuration, /* is_read_only */ false, attempt_to_create_container); object_storage = std::make_unique("BackupWriterAzureBlobStorage", std::move(client_ptr), StorageAzureBlob::createSettings(context_), diff --git a/src/Backups/BackupIO_AzureBlobStorage.h b/src/Backups/BackupIO_AzureBlobStorage.h index 95325044a62..f0b9aace4d4 100644 --- a/src/Backups/BackupIO_AzureBlobStorage.h +++ b/src/Backups/BackupIO_AzureBlobStorage.h @@ -37,7 +37,7 @@ private: class BackupWriterAzureBlobStorage : public BackupWriterDefault { public: - BackupWriterAzureBlobStorage(StorageAzureBlob::Configuration configuration_, const ReadSettings & read_settings_, const WriteSettings & write_settings_, const ContextPtr & context_); + BackupWriterAzureBlobStorage(StorageAzureBlob::Configuration configuration_, const ReadSettings & read_settings_, const WriteSettings & write_settings_, const ContextPtr & context_, bool attempt_to_create_container); ~BackupWriterAzureBlobStorage() override; bool fileExists(const String & file_name) override; diff --git a/src/Backups/BackupSettings.cpp b/src/Backups/BackupSettings.cpp index 68d825e9468..468e5651274 100644 --- a/src/Backups/BackupSettings.cpp +++ b/src/Backups/BackupSettings.cpp @@ -28,6 +28,7 @@ namespace ErrorCodes M(Bool, deduplicate_files) \ M(Bool, allow_s3_native_copy) \ M(Bool, use_same_s3_credentials_for_base_backup) \ + M(Bool, azure_attempt_to_create_container) \ M(Bool, read_from_filesystem_cache) \ M(UInt64, shard_num) \ M(UInt64, replica_num) \ diff --git a/src/Backups/BackupSettings.h b/src/Backups/BackupSettings.h index 10181ea464a..13709ca11c6 100644 --- a/src/Backups/BackupSettings.h +++ b/src/Backups/BackupSettings.h @@ -47,6 +47,9 @@ struct BackupSettings /// Whether base backup to S3 should inherit credentials from the BACKUP query. bool use_same_s3_credentials_for_base_backup = false; + /// Whether a new Azure container should be created if it does not exist (requires permissions at storage account level) + bool azure_attempt_to_create_container = true; + /// Allow to use the filesystem cache in passive mode - benefit from the existing cache entries, /// but don't put more entries into the cache. bool read_from_filesystem_cache = true; diff --git a/src/Backups/BackupsWorker.cpp b/src/Backups/BackupsWorker.cpp index d0853300edb..96fe770227c 100644 --- a/src/Backups/BackupsWorker.cpp +++ b/src/Backups/BackupsWorker.cpp @@ -597,6 +597,7 @@ void BackupsWorker::doBackup( backup_create_params.deduplicate_files = backup_settings.deduplicate_files; backup_create_params.allow_s3_native_copy = backup_settings.allow_s3_native_copy; backup_create_params.use_same_s3_credentials_for_base_backup = backup_settings.use_same_s3_credentials_for_base_backup; + backup_create_params.azure_attempt_to_create_container = backup_settings.azure_attempt_to_create_container; backup_create_params.read_settings = getReadSettingsForBackup(context, backup_settings); backup_create_params.write_settings = getWriteSettingsForBackup(context); backup = BackupFactory::instance().createBackup(backup_create_params); diff --git a/src/Backups/registerBackupEngineAzureBlobStorage.cpp b/src/Backups/registerBackupEngineAzureBlobStorage.cpp index 48f66569304..dbe5b555c31 100644 --- a/src/Backups/registerBackupEngineAzureBlobStorage.cpp +++ b/src/Backups/registerBackupEngineAzureBlobStorage.cpp @@ -86,7 +86,7 @@ void registerBackupEngineAzureBlobStorage(BackupFactory & factory) if (args.size() == 3) { configuration.connection_url = args[0].safeGet(); - configuration.is_connection_string = true; + configuration.is_connection_string = !configuration.connection_url.starts_with("http"); configuration.container = args[1].safeGet(); configuration.blob_path = args[2].safeGet(); @@ -147,7 +147,8 @@ void registerBackupEngineAzureBlobStorage(BackupFactory & factory) auto writer = std::make_shared(configuration, params.read_settings, params.write_settings, - params.context); + params.context, + params.azure_attempt_to_create_container); return std::make_unique( params.backup_info, diff --git a/src/Storages/StorageAzureBlob.cpp b/src/Storages/StorageAzureBlob.cpp index 6367cb0b077..289bd4643c4 100644 --- a/src/Storages/StorageAzureBlob.cpp +++ b/src/Storages/StorageAzureBlob.cpp @@ -17,6 +17,7 @@ #include #include +#include #include #include #include @@ -336,33 +337,37 @@ static bool containerExists(std::unique_ptr &blob_service_cli return false; } -AzureClientPtr StorageAzureBlob::createClient(StorageAzureBlob::Configuration configuration, bool is_read_only) +AzureClientPtr StorageAzureBlob::createClient(StorageAzureBlob::Configuration configuration, bool is_read_only, bool attempt_to_create_container) { AzureClientPtr result; if (configuration.is_connection_string) { + std::shared_ptr managed_identity_credential = std::make_shared(); std::unique_ptr blob_service_client = std::make_unique(BlobServiceClient::CreateFromConnectionString(configuration.connection_url)); result = std::make_unique(BlobContainerClient::CreateFromConnectionString(configuration.connection_url, configuration.container)); - bool container_exists = containerExists(blob_service_client,configuration.container); - if (!container_exists) + if (attempt_to_create_container) { - if (is_read_only) - throw Exception( - ErrorCodes::DATABASE_ACCESS_DENIED, - "AzureBlobStorage container does not exist '{}'", - configuration.container); + bool container_exists = containerExists(blob_service_client,configuration.container); + if (!container_exists) + { + if (is_read_only) + throw Exception( + ErrorCodes::DATABASE_ACCESS_DENIED, + "AzureBlobStorage container does not exist '{}'", + configuration.container); - try - { - result->CreateIfNotExists(); - } catch (const Azure::Storage::StorageException & e) - { - if (!(e.StatusCode == Azure::Core::Http::HttpStatusCode::Conflict - && e.ReasonPhrase == "The specified container already exists.")) + try { - throw; + result->CreateIfNotExists(); + } catch (const Azure::Storage::StorageException & e) + { + if (!(e.StatusCode == Azure::Core::Http::HttpStatusCode::Conflict + && e.ReasonPhrase == "The specified container already exists.")) + { + throw; + } } } } @@ -377,17 +382,17 @@ AzureClientPtr StorageAzureBlob::createClient(StorageAzureBlob::Configuration co } std::unique_ptr blob_service_client; + std::shared_ptr managed_identity_credential; if (storage_shared_key_credential) { blob_service_client = std::make_unique(configuration.connection_url, storage_shared_key_credential); } else { - blob_service_client = std::make_unique(configuration.connection_url); + managed_identity_credential = std::make_shared(); + blob_service_client = std::make_unique(configuration.connection_url, managed_identity_credential); } - bool container_exists = containerExists(blob_service_client,configuration.container); - std::string final_url; size_t pos = configuration.connection_url.find('?'); if (pos != std::string::npos) @@ -400,12 +405,21 @@ AzureClientPtr StorageAzureBlob::createClient(StorageAzureBlob::Configuration co final_url = configuration.connection_url + (configuration.connection_url.back() == '/' ? "" : "/") + configuration.container; + if (!attempt_to_create_container) + { + if (storage_shared_key_credential) + return std::make_unique(final_url, storage_shared_key_credential); + else + return std::make_unique(final_url, managed_identity_credential); + } + + bool container_exists = containerExists(blob_service_client,configuration.container); if (container_exists) { if (storage_shared_key_credential) result = std::make_unique(final_url, storage_shared_key_credential); else - result = std::make_unique(final_url); + result = std::make_unique(final_url, managed_identity_credential); } else { @@ -425,7 +439,7 @@ AzureClientPtr StorageAzureBlob::createClient(StorageAzureBlob::Configuration co if (storage_shared_key_credential) result = std::make_unique(final_url, storage_shared_key_credential); else - result = std::make_unique(final_url); + result = std::make_unique(final_url, managed_identity_credential); } else { diff --git a/src/Storages/StorageAzureBlob.h b/src/Storages/StorageAzureBlob.h index 63fd489dcaf..27ac7a5c368 100644 --- a/src/Storages/StorageAzureBlob.h +++ b/src/Storages/StorageAzureBlob.h @@ -69,7 +69,7 @@ public: ASTPtr partition_by_); static StorageAzureBlob::Configuration getConfiguration(ASTs & engine_args, const ContextPtr & local_context); - static AzureClientPtr createClient(StorageAzureBlob::Configuration configuration, bool is_read_only); + static AzureClientPtr createClient(StorageAzureBlob::Configuration configuration, bool is_read_only, bool attempt_to_create_container = true); static AzureObjectStorage::SettingsPtr createSettings(const ContextPtr & local_context); From df9aa3611f9ccce452fcbdfeeec28ea3250cf41a Mon Sep 17 00:00:00 2001 From: Nikolai Kochetov Date: Fri, 22 Mar 2024 18:12:38 +0100 Subject: [PATCH 055/117] Remove already not flaky tests with analyzer. --- tests/analyzer_tech_debt.txt | 2 -- 1 file changed, 2 deletions(-) diff --git a/tests/analyzer_tech_debt.txt b/tests/analyzer_tech_debt.txt index e2d8084df5e..cd9d262e5e4 100644 --- a/tests/analyzer_tech_debt.txt +++ b/tests/analyzer_tech_debt.txt @@ -6,6 +6,4 @@ # Check after constants refactoring 02901_parallel_replicas_rollup # Flaky. Please don't delete them without fixing them: -01287_max_execution_speed 02003_WithMergeableStateAfterAggregationAndLimit_LIMIT_BY_LIMIT_OFFSET -02404_memory_bound_merging From 510a3044c3b3ce3cccc45681bdc7dfafc3b49ac4 Mon Sep 17 00:00:00 2001 From: Yarik Briukhovetskyi <114298166+yariks5s@users.noreply.github.com> Date: Fri, 22 Mar 2024 18:23:20 +0100 Subject: [PATCH 056/117] simplify tests --- .../0_stateless/00990_function_current_user.reference | 4 ---- tests/queries/0_stateless/00990_function_current_user.sql | 7 +------ 2 files changed, 1 insertion(+), 10 deletions(-) diff --git a/tests/queries/0_stateless/00990_function_current_user.reference b/tests/queries/0_stateless/00990_function_current_user.reference index 227c7c3b562..95d2ef117ea 100644 --- a/tests/queries/0_stateless/00990_function_current_user.reference +++ b/tests/queries/0_stateless/00990_function_current_user.reference @@ -1,8 +1,4 @@ 1 1 -1 1 -1 -1 -1 1 1 1 1 diff --git a/tests/queries/0_stateless/00990_function_current_user.sql b/tests/queries/0_stateless/00990_function_current_user.sql index c2d946e4185..4cf59e72e11 100644 --- a/tests/queries/0_stateless/00990_function_current_user.sql +++ b/tests/queries/0_stateless/00990_function_current_user.sql @@ -1,10 +1,5 @@ -- since actual user name is unknown, have to perform just smoke tests select currentUser() IS NOT NULL; select length(currentUser()) > 0; -select currentUser() = user(), currentUser() = USER(); +select currentUser() = user(), currentUser() = USER(), current_user() = currentUser(); select currentUser() = initial_user from system.processes where query like '%$!@#%'; - -select current_user() IS NOT NULL; -select length(current_user()) > 0; -select current_user() = user(), current_user() = USER(), current_user() = currentUser(); -select current_user() = initial_user from system.processes where query like '%$!@#%'; From edc9e8fbc8eb91e00399ab844681c95e31df0f70 Mon Sep 17 00:00:00 2001 From: vdimir Date: Fri, 22 Mar 2024 17:30:50 +0000 Subject: [PATCH 057/117] Analyzer: Fix assert in JOIN with Distributed table --- src/Planner/PlannerContext.cpp | 3 +++ src/Planner/PlannerJoinTree.cpp | 3 ++- 2 files changed, 5 insertions(+), 1 deletion(-) diff --git a/src/Planner/PlannerContext.cpp b/src/Planner/PlannerContext.cpp index f939b959ce7..c9187e6aa2a 100644 --- a/src/Planner/PlannerContext.cpp +++ b/src/Planner/PlannerContext.cpp @@ -32,6 +32,9 @@ const ColumnIdentifier & GlobalPlannerContext::createColumnIdentifier(const Name column_identifier = column.name; auto [it, inserted] = column_identifiers.emplace(column_identifier); + if (!inserted) + throw Exception(ErrorCodes::LOGICAL_ERROR, "Column identifier {} is already registered", column_identifier); + assert(inserted); return *it; diff --git a/src/Planner/PlannerJoinTree.cpp b/src/Planner/PlannerJoinTree.cpp index 61a7a19f5be..4ca8b6d7d48 100644 --- a/src/Planner/PlannerJoinTree.cpp +++ b/src/Planner/PlannerJoinTree.cpp @@ -960,7 +960,8 @@ JoinTreeQueryPlan buildQueryPlanForTableExpression(QueryTreeNodePtr table_expres else { auto subquery_options = select_query_options.subquery(); - Planner subquery_planner(table_expression, subquery_options, planner_context->getGlobalPlannerContext()); + Planner subquery_planner(table_expression, subquery_options, + std::make_shared(nullptr, nullptr, FiltersForTableExpressionMap{})); /// Propagate storage limits to subquery subquery_planner.addStorageLimits(*select_query_info.storage_limits); subquery_planner.buildQueryPlanIfNeeded(); From f54e11176d6057f37c1f38e332d6ff5113accb50 Mon Sep 17 00:00:00 2001 From: vdimir Date: Fri, 22 Mar 2024 18:12:31 +0000 Subject: [PATCH 058/117] u --- src/Planner/PlannerJoinTree.cpp | 9 +++++++-- 1 file changed, 7 insertions(+), 2 deletions(-) diff --git a/src/Planner/PlannerJoinTree.cpp b/src/Planner/PlannerJoinTree.cpp index 4ca8b6d7d48..c2b888af578 100644 --- a/src/Planner/PlannerJoinTree.cpp +++ b/src/Planner/PlannerJoinTree.cpp @@ -959,9 +959,14 @@ JoinTreeQueryPlan buildQueryPlanForTableExpression(QueryTreeNodePtr table_expres } else { + std::shared_ptr subquery_planner_context; + if (wrap_read_columns_in_subquery) + subquery_planner_context = std::make_shared(nullptr, nullptr, FiltersForTableExpressionMap{}); + else + subquery_planner_context = planner_context->getGlobalPlannerContext(); + auto subquery_options = select_query_options.subquery(); - Planner subquery_planner(table_expression, subquery_options, - std::make_shared(nullptr, nullptr, FiltersForTableExpressionMap{})); + Planner subquery_planner(table_expression, subquery_options, subquery_planner_context); /// Propagate storage limits to subquery subquery_planner.addStorageLimits(*select_query_info.storage_limits); subquery_planner.buildQueryPlanIfNeeded(); From 4621fe6115fe7d62efa3309ef8efc59c02c6ce6e Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Fri, 22 Mar 2024 20:21:54 +0100 Subject: [PATCH 059/117] Fix wrong cases of numbers pretty printing --- .../Formats/Impl/PrettyBlockOutputFormat.cpp | 19 +++++++++++-------- 1 file changed, 11 insertions(+), 8 deletions(-) diff --git a/src/Processors/Formats/Impl/PrettyBlockOutputFormat.cpp b/src/Processors/Formats/Impl/PrettyBlockOutputFormat.cpp index c5db8f2e30a..92411864e93 100644 --- a/src/Processors/Formats/Impl/PrettyBlockOutputFormat.cpp +++ b/src/Processors/Formats/Impl/PrettyBlockOutputFormat.cpp @@ -419,16 +419,19 @@ void PrettyBlockOutputFormat::writeReadableNumberTip(const Chunk & chunk) auto is_single_number = readable_number_tip && chunk.getNumRows() == 1 && chunk.getNumColumns() == 1; if (!is_single_number) return; + auto value = columns[0]->getFloat64(0); auto threshold = format_settings.pretty.output_format_pretty_single_large_number_tip_threshold; - if (threshold == 0 || value <= threshold) - return; - if (color) - writeCString("\033[90m", out); - writeCString(" -- ", out); - formatReadableQuantity(value, out, 2); - if (color) - writeCString("\033[0m", out); + + if (threshold && isFinite(value) && abs(value) >= threshold) + { + if (color) + writeCString("\033[90m", out); + writeCString(" -- ", out); + formatReadableQuantity(value, out, 2); + if (color) + writeCString("\033[0m", out); + } } void registerOutputFormatPretty(FormatFactory & factory) From 1a7494f04f4525494eb40c0edd2669c8a515e49b Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Fri, 22 Mar 2024 20:25:58 +0100 Subject: [PATCH 060/117] Add a test Revert changes from another branch Revert changes from another branch Revert changes from another branch --- .../03019_numbers_pretty.reference | 25 +++++++++++++++++++ .../0_stateless/03019_numbers_pretty.sql | 6 +++++ 2 files changed, 31 insertions(+) create mode 100644 tests/queries/0_stateless/03019_numbers_pretty.reference create mode 100644 tests/queries/0_stateless/03019_numbers_pretty.sql diff --git a/tests/queries/0_stateless/03019_numbers_pretty.reference b/tests/queries/0_stateless/03019_numbers_pretty.reference new file mode 100644 index 00000000000..501df424ce7 --- /dev/null +++ b/tests/queries/0_stateless/03019_numbers_pretty.reference @@ -0,0 +1,25 @@ +┏━━━━━━━━━━━━━┓ +┃ 1230000000. ┃ +┡━━━━━━━━━━━━━┩ +│ 1230000000 │ -- 1.23 billion +└─────────────┘ +┏━━━━━━━━━━━━━━┓ +┃ -1230000000. ┃ +┡━━━━━━━━━━━━━━┩ +│ -1230000000 │-- -1.23 billion +└──────────────┘ +┏━━━━━┓ +┃ inf ┃ +┡━━━━━┩ +│ inf │ +└─────┘ +┏━━━━━━┓ +┃ -inf ┃ +┡━━━━━━┩ +│ -inf │ +└──────┘ +┏━━━━━┓ +┃ nan ┃ +┡━━━━━┩ +│ nan │ +└─────┘ diff --git a/tests/queries/0_stateless/03019_numbers_pretty.sql b/tests/queries/0_stateless/03019_numbers_pretty.sql new file mode 100644 index 00000000000..e6d47abcd17 --- /dev/null +++ b/tests/queries/0_stateless/03019_numbers_pretty.sql @@ -0,0 +1,6 @@ +SET output_format_pretty_row_numbers = 0; +SELECT 1.23e9 FORMAT Pretty; +SELECT -1.23e9 FORMAT Pretty; +SELECT inf FORMAT Pretty; +SELECT -inf FORMAT Pretty; +SELECT nan FORMAT Pretty; From 84daaa1c7ee8196fa16eb71dc2e7b354c39348d4 Mon Sep 17 00:00:00 2001 From: Michael Kolupaev Date: Fri, 22 Mar 2024 20:00:42 +0000 Subject: [PATCH 061/117] Style --- src/Interpreters/SquashingTransform.cpp | 1 + 1 file changed, 1 insertion(+) diff --git a/src/Interpreters/SquashingTransform.cpp b/src/Interpreters/SquashingTransform.cpp index da608f5e7ce..41f024df7a7 100644 --- a/src/Interpreters/SquashingTransform.cpp +++ b/src/Interpreters/SquashingTransform.cpp @@ -6,6 +6,7 @@ namespace DB namespace ErrorCodes { extern const int SIZES_OF_COLUMNS_DOESNT_MATCH; + extern const int LOGICAL_ERROR; } SquashingTransform::SquashingTransform(size_t min_block_size_rows_, size_t min_block_size_bytes_) From 18c016caaac6cab5635ed3149805015092e101ff Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Fri, 22 Mar 2024 21:06:36 +0100 Subject: [PATCH 062/117] Support for --output-format in client --- programs/local/LocalServer.cpp | 1 - src/Client/ClientBase.cpp | 3 ++- 2 files changed, 2 insertions(+), 2 deletions(-) diff --git a/programs/local/LocalServer.cpp b/programs/local/LocalServer.cpp index 6f7d3681c46..e0124f03a64 100644 --- a/programs/local/LocalServer.cpp +++ b/programs/local/LocalServer.cpp @@ -819,7 +819,6 @@ void LocalServer::addOptions(OptionsDescription & options_description) ("file,F", po::value(), "path to file with data of the initial table (stdin if not specified)") ("input-format", po::value(), "input format of the initial table data") - ("output-format", po::value(), "default output format") ("logger.console", po::value()->implicit_value(true), "Log to console") ("logger.log", po::value(), "Log file name") diff --git a/src/Client/ClientBase.cpp b/src/Client/ClientBase.cpp index 8b13f2a4ffd..4dab7f2403b 100644 --- a/src/Client/ClientBase.cpp +++ b/src/Client/ClientBase.cpp @@ -2892,7 +2892,8 @@ void ClientBase::init(int argc, char ** argv) ("suggestion_limit", po::value()->default_value(10000), "Suggestion limit for how many databases, tables and columns to fetch.") - ("format,f", po::value(), "default output format") + ("format,output-format,f", po::value(), "default output format (and input format for clickhouse-local)") + ("vertical,E", "vertical output format, same as --format=Vertical or FORMAT Vertical or \\G at end of command") ("highlight", po::value()->default_value(true), "enable or disable basic syntax highlight in interactive command line") From fd5d55588553ed7a4a8121eb244e484cbf194632 Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Fri, 22 Mar 2024 21:07:09 +0100 Subject: [PATCH 063/117] A test can be slow in debug build --- tests/queries/0_stateless/00938_fix_rwlock_segfault_long.sh | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/tests/queries/0_stateless/00938_fix_rwlock_segfault_long.sh b/tests/queries/0_stateless/00938_fix_rwlock_segfault_long.sh index 795f41528b2..088c266b017 100755 --- a/tests/queries/0_stateless/00938_fix_rwlock_segfault_long.sh +++ b/tests/queries/0_stateless/00938_fix_rwlock_segfault_long.sh @@ -1,5 +1,5 @@ #!/usr/bin/env bash -# Tags: long +# Tags: long, no-debug # Test fix for issue #5066 From 14c616e6964d7e98fcae36603206ca074903b776 Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Fri, 22 Mar 2024 21:29:33 +0100 Subject: [PATCH 064/117] Fix for clang-19 --- src/Interpreters/PreparedSets.h | 3 +-- 1 file changed, 1 insertion(+), 2 deletions(-) diff --git a/src/Interpreters/PreparedSets.h b/src/Interpreters/PreparedSets.h index 3419d3b6839..45d53a691a6 100644 --- a/src/Interpreters/PreparedSets.h +++ b/src/Interpreters/PreparedSets.h @@ -9,12 +9,11 @@ #include #include #include +#include namespace DB { -class QueryPlan; - class Set; using SetPtr = std::shared_ptr; struct SetKeyColumns; From a56c113c839f644b45f69dcb8030840f4872c0dc Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Fri, 22 Mar 2024 22:12:59 +0100 Subject: [PATCH 065/117] Fix for clang-19 --- .../AggregateFunctionQuantile.cpp | 3 -- .../AggregateFunctionQuantile.h | 31 +++++++++---------- .../AggregateFunctionQuantileBFloat16.cpp | 3 -- ...regateFunctionQuantileBFloat16Weighted.cpp | 3 -- .../AggregateFunctionQuantileDD.cpp | 3 -- ...AggregateFunctionQuantileDeterministic.cpp | 3 -- .../AggregateFunctionQuantileExact.cpp | 3 -- ...ggregateFunctionQuantileExactExclusive.cpp | 3 -- .../AggregateFunctionQuantileExactHigh.cpp | 5 +-- ...ggregateFunctionQuantileExactInclusive.cpp | 3 -- .../AggregateFunctionQuantileExactLow.cpp | 3 -- ...AggregateFunctionQuantileExactWeighted.cpp | 3 -- .../AggregateFunctionQuantileGK.cpp | 3 -- ...teFunctionQuantileInterpolatedWeighted.cpp | 3 -- .../AggregateFunctionQuantileTDigest.cpp | 3 -- ...gregateFunctionQuantileTDigestWeighted.cpp | 3 -- .../AggregateFunctionQuantileTiming.cpp | 3 -- ...ggregateFunctionQuantileTimingWeighted.cpp | 3 -- src/Interpreters/PreparedSets.cpp | 2 ++ src/Interpreters/PreparedSets.h | 5 ++- src/Server/MySQLHandler.cpp | 2 ++ src/Server/MySQLHandler.h | 2 ++ src/Storages/HDFS/StorageHDFS.cpp | 2 ++ src/Storages/HDFS/StorageHDFS.h | 2 ++ src/Storages/Kafka/StorageKafka.cpp | 2 ++ src/Storages/Kafka/StorageKafka.h | 2 ++ .../RocksDB/StorageEmbeddedRocksDB.cpp | 2 ++ src/Storages/RocksDB/StorageEmbeddedRocksDB.h | 2 ++ src/Storages/StorageURL.cpp | 2 ++ src/Storages/StorageURL.h | 2 ++ 30 files changed, 42 insertions(+), 69 deletions(-) diff --git a/src/AggregateFunctions/AggregateFunctionQuantile.cpp b/src/AggregateFunctions/AggregateFunctionQuantile.cpp index 110f6c6b4d6..1ba3cc0f516 100644 --- a/src/AggregateFunctions/AggregateFunctionQuantile.cpp +++ b/src/AggregateFunctions/AggregateFunctionQuantile.cpp @@ -116,9 +116,6 @@ template